From 29809a6d58bfe3700350ce1988ff7083881c4382 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 17 Jul 2014 23:25:01 -0700 Subject: [PATCH 0001/1492] [SPARK-2570] [SQL] Fix the bug of ClassCastException Exception thrown when running the example of HiveFromSpark. Exception in thread "main" java.lang.ClassCastException: java.lang.Long cannot be cast to java.lang.Integer at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:106) at org.apache.spark.sql.catalyst.expressions.GenericRow.getInt(Row.scala:145) at org.apache.spark.examples.sql.hive.HiveFromSpark$.main(HiveFromSpark.scala:45) at org.apache.spark.examples.sql.hive.HiveFromSpark.main(HiveFromSpark.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:303) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:55) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Author: Cheng Hao Closes #1475 from chenghao-intel/hive_from_spark and squashes the following commits: d4c0500 [Cheng Hao] Fix the bug of ClassCastException --- .../org/apache/spark/examples/sql/hive/HiveFromSpark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index b262fabbe0e0d..66a23fac39999 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -42,7 +42,7 @@ object HiveFromSpark { hql("SELECT * FROM src").collect.foreach(println) // Aggregation queries are also supported. - val count = hql("SELECT COUNT(*) FROM src").collect().head.getInt(0) + val count = hql("SELECT COUNT(*) FROM src").collect().head.getLong(0) println(s"COUNT(*): $count") // The results of SQL queries are themselves RDDs and support all normal RDD functions. The From e52b8719cf0603e79ded51cbe1c9f88eea8b56de Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 17 Jul 2014 23:57:08 -0700 Subject: [PATCH 0002/1492] SPARK-2553. CoGroupedRDD unnecessarily allocates a Tuple2 per dependency... ... per key My humble opinion is that avoiding allocations in this performance-critical section is worth the extra code. Author: Sandy Ryza Closes #1461 from sryza/sandy-spark-2553 and squashes the following commits: 7eaf7f2 [Sandy Ryza] SPARK-2553. CoGroupedRDD unnecessarily allocates a Tuple2 per dependency per key --- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 5951865e56c9d..b284b636f2766 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -180,7 +180,11 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = (combiner1, combiner2) => { - combiner1.zip(combiner2).map { case (v1, v2) => v1 ++ v2 } + var depNum = 0 + while (depNum < numRdds) { + combiner1(depNum) ++= combiner2(depNum) + depNum += 1 + } } new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( createCombiner, mergeValue, mergeCombiners) From 30b8d369d459be5da133f12ebb33a9c74579f468 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 18 Jul 2014 00:47:43 -0700 Subject: [PATCH 0003/1492] SPARK-2553. Fix compile error Author: Sandy Ryza Closes #1479 from sryza/sandy-spark-2553 and squashes the following commits: 2cb5ed8 [Sandy Ryza] SPARK-2553. Fix compile error --- core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index b284b636f2766..5366c1a1cc1bd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -185,6 +185,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: combiner1(depNum) ++= combiner2(depNum) depNum += 1 } + combiner1 } new ExternalAppendOnlyMap[K, CoGroupValue, CoGroupCombiner]( createCombiner, mergeValue, mergeCombiners) From 7f87ab98138d00723e007471f1a7f506650978cb Mon Sep 17 00:00:00 2001 From: Basit Mustafa Date: Fri, 18 Jul 2014 12:23:47 -0700 Subject: [PATCH 0004/1492] Added t2 instance types New t2 instance types require HVM amis, bailout assumption of pvm causes failures when using t2 instance types. Author: Basit Mustafa Closes #1446 from 24601/master and squashes the following commits: 01fe128 [Basit Mustafa] Makin' it pretty 392a95e [Basit Mustafa] Added t2 instance types --- ec2/spark_ec2.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 44775ea479ece..02cfe4ec39c7d 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -240,7 +240,10 @@ def get_spark_ami(opts): "r3.xlarge": "hvm", "r3.2xlarge": "hvm", "r3.4xlarge": "hvm", - "r3.8xlarge": "hvm" + "r3.8xlarge": "hvm", + "t2.micro": "hvm", + "t2.small": "hvm", + "t2.medium": "hvm" } if opts.instance_type in instance_types: instance_type = instance_types[opts.instance_type] From 586e716e47305cd7c2c3ff35c0e828b63ef2f6a8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 18 Jul 2014 12:41:50 -0700 Subject: [PATCH 0005/1492] Reservoir sampling implementation. This is going to be used in https://issues.apache.org/jira/browse/SPARK-2568 Author: Reynold Xin Closes #1478 from rxin/reservoirSample and squashes the following commits: 17bcbf3 [Reynold Xin] Added seed. badf20d [Reynold Xin] Renamed the method. 6940010 [Reynold Xin] Reservoir sampling implementation. --- .../spark/util/random/SamplingUtils.scala | 46 +++++++++++++++++++ .../util/random/SamplingUtilsSuite.scala | 21 +++++++++ 2 files changed, 67 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index a79e3ee756fc6..d10141b90e621 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -17,8 +17,54 @@ package org.apache.spark.util.random +import scala.reflect.ClassTag +import scala.util.Random + private[spark] object SamplingUtils { + /** + * Reservoir sampling implementation that also returns the input size. + * + * @param input input size + * @param k reservoir size + * @param seed random seed + * @return (samples, input size) + */ + def reservoirSampleAndCount[T: ClassTag]( + input: Iterator[T], + k: Int, + seed: Long = Random.nextLong()) + : (Array[T], Int) = { + val reservoir = new Array[T](k) + // Put the first k elements in the reservoir. + var i = 0 + while (i < k && input.hasNext) { + val item = input.next() + reservoir(i) = item + i += 1 + } + + // If we have consumed all the elements, return them. Otherwise do the replacement. + if (i < k) { + // If input size < k, trim the array to return only an array of input size. + val trimReservoir = new Array[T](i) + System.arraycopy(reservoir, 0, trimReservoir, 0, i) + (trimReservoir, i) + } else { + // If input size > k, continue the sampling process. + val rand = new XORShiftRandom(seed) + while (input.hasNext) { + val item = input.next() + val replacementIndex = rand.nextInt(i) + if (replacementIndex < k) { + reservoir(replacementIndex) = item + } + i += 1 + } + (reservoir, i) + } + } + /** * Returns a sampling rate that guarantees a sample of size >= sampleSizeLowerBound 99.99% of * the time. diff --git a/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala index accfe2e9b7f2a..73a9d029b0248 100644 --- a/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/SamplingUtilsSuite.scala @@ -17,11 +17,32 @@ package org.apache.spark.util.random +import scala.util.Random + import org.apache.commons.math3.distribution.{BinomialDistribution, PoissonDistribution} import org.scalatest.FunSuite class SamplingUtilsSuite extends FunSuite { + test("reservoirSampleAndCount") { + val input = Seq.fill(100)(Random.nextInt()) + + // input size < k + val (sample1, count1) = SamplingUtils.reservoirSampleAndCount(input.iterator, 150) + assert(count1 === 100) + assert(input === sample1.toSeq) + + // input size == k + val (sample2, count2) = SamplingUtils.reservoirSampleAndCount(input.iterator, 100) + assert(count2 === 100) + assert(input === sample2.toSeq) + + // input size > k + val (sample3, count3) = SamplingUtils.reservoirSampleAndCount(input.iterator, 10) + assert(count3 === 100) + assert(sample3.length === 10) + } + test("computeFraction") { // test that the computed fraction guarantees enough data points // in the sample with a failure rate <= 0.0001 From d88f6be446e263251c446441c9ce7f5b11216909 Mon Sep 17 00:00:00 2001 From: Manish Amde Date: Fri, 18 Jul 2014 14:00:13 -0700 Subject: [PATCH 0006/1492] [MLlib] SPARK-1536: multiclass classification support for decision tree The ability to perform multiclass classification is a big advantage for using decision trees and was a highly requested feature for mllib. This pull request adds multiclass classification support to the MLlib decision tree. It also adds sample weights support using WeightedLabeledPoint class for handling unbalanced datasets during classification. It will also support algorithms such as AdaBoost which requires instances to be weighted. It handles the special case where the categorical variables cannot be ordered for multiclass classification and thus the optimizations used for speeding up binary classification cannot be directly used for multiclass classification with categorical variables. More specifically, for m categories in a categorical feature, it analyses all the ```2^(m-1) - 1``` categorical splits provided that #splits are less than the maxBins provided in the input. This condition will not be met for features with large number of categories -- using decision trees is not recommended for such datasets in general since the categorical features are favored over continuous features. Moreover, the user can use a combination of tricks (increasing bin size of the tree algorithms, use binary encoding for categorical features or use one-vs-all classification strategy) to avoid these constraints. The new code is accompanied by unit tests and has also been tested on the iris and covtype datasets. cc: mengxr, etrain, hirakendu, atalwalkar, srowen Author: Manish Amde Author: manishamde Author: Evan Sparks Closes #886 from manishamde/multiclass and squashes the following commits: 26f8acc [Manish Amde] another attempt at fixing mima c5b2d04 [Manish Amde] more MIMA fixes 1ce7212 [Manish Amde] change problem filter for mima 10fdd82 [Manish Amde] fixing MIMA excludes e1c970d [Manish Amde] merged master abf2901 [Manish Amde] adding classes to MimaExcludes.scala 45e767a [Manish Amde] adding developer api annotation for overriden methods c8428c4 [Manish Amde] fixing weird multiline bug afced16 [Manish Amde] removed label weights support 2d85a48 [Manish Amde] minor: fixed scalastyle issues reprise 4e85f2c [Manish Amde] minor: fixed scalastyle issues b2ae41f [Manish Amde] minor: scalastyle e4c1321 [Manish Amde] using while loop for regression histograms d75ac32 [Manish Amde] removed WeightedLabeledPoint from this PR 0fecd38 [Manish Amde] minor: add newline to EOF 2061cf5 [Manish Amde] merged from master 06b1690 [Manish Amde] fixed off-by-one error in bin to split conversion 9cc3e31 [Manish Amde] added implicit conversion import 5c1b2ca [Manish Amde] doc for PointConverter class 485eaae [Manish Amde] implicit conversion from LabeledPoint to WeightedLabeledPoint 3d7f911 [Manish Amde] updated doc 8e44ab8 [Manish Amde] updated doc adc7315 [Manish Amde] support ordered categorical splits for multiclass classification e3e8843 [Manish Amde] minor code formatting 23d4268 [Manish Amde] minor: another minor code style 34ee7b9 [Manish Amde] minor: code style 237762d [Manish Amde] renaming functions 12e6d0a [Manish Amde] minor: removing line in doc 9a90c93 [Manish Amde] Merge branch 'master' into multiclass 1892a2c [Manish Amde] tests and use multiclass binaggregate length when atleast one categorical feature is present f5f6b83 [Manish Amde] multiclass for continous variables 8cfd3b6 [Manish Amde] working for categorical multiclass classification 828ff16 [Manish Amde] added categorical variable test bce835f [Manish Amde] code cleanup 7e5f08c [Manish Amde] minor doc 1dd2735 [Manish Amde] bin search logic for multiclass f16a9bb [Manish Amde] fixing while loop d811425 [Manish Amde] multiclass bin aggregate logic ab5cb21 [Manish Amde] multiclass logic d8e4a11 [Manish Amde] sample weights ed5a2df [Manish Amde] fixed classification requirements d012be7 [Manish Amde] fixed while loop 18d2835 [Manish Amde] changing default values for num classes 6b912dc [Manish Amde] added numclasses to tree runner, predict logic for multiclass, add multiclass option to train 75f2bfc [Manish Amde] minor code style fix e547151 [Manish Amde] minor modifications 34549d0 [Manish Amde] fixing error during merge 098e8c5 [Manish Amde] merged master e006f9d [Manish Amde] changing variable names 5c78e1a [Manish Amde] added multiclass support 6c7af22 [Manish Amde] prepared for multiclass without breaking binary classification 46e06ee [Manish Amde] minor mods 3f85a17 [Manish Amde] tests for multiclass classification 4d5f70c [Manish Amde] added multiclass support for find splits bins 46f909c [Manish Amde] todo for multiclass support 455bea9 [Manish Amde] fixed tests 14aea48 [Manish Amde] changing instance format to weighted labeled point a1a6e09 [Manish Amde] added weighted point class 968ca9d [Manish Amde] merged master 7fc9545 [Manish Amde] added docs ce004a1 [Manish Amde] minor formatting b27ad2c [Manish Amde] formatting 426bb28 [Manish Amde] programming guide blurb 8053fed [Manish Amde] more formatting 5eca9e4 [Manish Amde] grammar 4731cda [Manish Amde] formatting 5e82202 [Manish Amde] added documentation, fixed off by 1 error in max level calculation cbd9f14 [Manish Amde] modified scala.math to math dad9652 [Manish Amde] removed unused imports e0426ee [Manish Amde] renamed parameter 718506b [Manish Amde] added unit test 1517155 [Manish Amde] updated documentation 9dbdabe [Manish Amde] merge from master 719d009 [Manish Amde] updating user documentation fecf89a [manishamde] Merge pull request #6 from etrain/deep_tree 0287772 [Evan Sparks] Fixing scalastyle issue. 2f1e093 [Manish Amde] minor: added doc for maxMemory parameter 2f6072c [manishamde] Merge pull request #5 from etrain/deep_tree abc5a23 [Evan Sparks] Parameterizing max memory. 50b143a [Manish Amde] adding support for very deep trees --- docs/mllib-decision-tree.md | 8 +- .../examples/mllib/DecisionTreeRunner.scala | 21 +- .../spark/mllib/tree/DecisionTree.scala | 732 +++++++++++++----- .../mllib/tree/configuration/Strategy.scala | 12 +- .../spark/mllib/tree/impurity/Entropy.scala | 36 +- .../spark/mllib/tree/impurity/Gini.scala | 33 +- .../spark/mllib/tree/impurity/Impurity.scala | 8 +- .../spark/mllib/tree/impurity/Variance.scala | 11 +- .../apache/spark/mllib/tree/model/Bin.scala | 2 +- .../tree/model/InformationGainStats.scala | 8 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 303 +++++++- project/MimaExcludes.scala | 10 +- 12 files changed, 926 insertions(+), 258 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 9cd768599e529..9cbd880897578 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -77,15 +77,17 @@ bins if the condition is not satisfied. **Categorical features** -For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for -binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the +For `$M$` categorical feature values, one could come up with `$2^(M-1)-1$` split candidates. For +binary classification, we can reduce the number of split candidates to `$M-1$` by ordering the categorical feature values by the proportion of labels falling in one of the two classes (see Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical features are ordered as A followed by C followed B or A, B, C. The two split candidates are A \| C, B -and A , B \| C where \| denotes the split. +and A , B \| C where \| denotes the split. A similar heuristic is used for multiclass classification +when `$2^(M-1)-1$` is greater than the number of bins -- the impurity for each categorical feature value +is used for ordering. ### Stopping rule diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index b3cc361154198..43f13fe24f0d0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -49,6 +49,7 @@ object DecisionTreeRunner { case class Params( input: String = null, algo: Algo = Classification, + numClassesForClassification: Int = 2, maxDepth: Int = 5, impurity: ImpurityType = Gini, maxBins: Int = 100) @@ -68,6 +69,10 @@ object DecisionTreeRunner { opt[Int]("maxDepth") .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("numClassesForClassification") + .text(s"number of classes for classification, " + + s"default: ${defaultParams.numClassesForClassification}") + .action((x, c) => c.copy(numClassesForClassification = x)) opt[Int]("maxBins") .text(s"max number of bins, default: ${defaultParams.maxBins}") .action((x, c) => c.copy(maxBins = x)) @@ -118,7 +123,13 @@ object DecisionTreeRunner { case Variance => impurity.Variance } - val strategy = new Strategy(params.algo, impurityCalculator, params.maxDepth, params.maxBins) + val strategy + = new Strategy( + algo = params.algo, + impurity = impurityCalculator, + maxDepth = params.maxDepth, + maxBins = params.maxBins, + numClassesForClassification = params.numClassesForClassification) val model = DecisionTree.train(training, strategy) if (params.algo == Classification) { @@ -139,12 +150,8 @@ object DecisionTreeRunner { */ private def accuracyScore( model: DecisionTreeModel, - data: RDD[LabeledPoint], - threshold: Double = 0.5): Double = { - def predictedValue(features: Vector): Double = { - if (model.predict(features) < threshold) 0.0 else 1.0 - } - val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() + data: RDD[LabeledPoint]): Double = { + val correctCount = data.filter(y => model.predict(y.features) == y.label).count() val count = data.count() correctCount.toDouble / count } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 74d5d7ba10960..ad32e3f4560fe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -77,11 +77,9 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Max memory usage for aggregates val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val numElementsPerNode = - strategy.algo match { - case Classification => 2 * numBins * numFeatures - case Regression => 3 * numBins * numFeatures - } + val numElementsPerNode = DecisionTree.getElementsPerNode(numFeatures, numBins, + strategy.numClassesForClassification, strategy.isMulticlassWithCategoricalFeatures, + strategy.algo) logDebug("numElementsPerNode = " + numElementsPerNode) val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array @@ -109,8 +107,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("#####################################") // Find best split for all nodes at a level. - val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, - level, filters, splits, bins, maxLevelForSingleGroup) + val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, + strategy, level, filters, splits, bins, maxLevelForSingleGroup) for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { // Extract info for nodes at the current level. @@ -212,7 +210,7 @@ object DecisionTree extends Serializable with Logging { * @return a DecisionTreeModel that can be used for prediction */ def train(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeModel = { - new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + new DecisionTree(strategy).train(input) } /** @@ -233,10 +231,33 @@ object DecisionTree extends Serializable with Logging { algo: Algo, impurity: Impurity, maxDepth: Int): DecisionTreeModel = { - val strategy = new Strategy(algo,impurity,maxDepth) - new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + val strategy = new Strategy(algo, impurity, maxDepth) + new DecisionTree(strategy).train(input) } + /** + * Method to train a decision tree model where the instances are represented as an RDD of + * (label, features) pairs. The method supports binary classification and regression. For the + * binary classification, the label for each instance should either be 0 or 1 to denote the two + * classes. + * + * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as + * training data + * @param algo algorithm, classification or regression + * @param impurity impurity criterion used for information gain calculation + * @param maxDepth maxDepth maximum depth of the tree + * @param numClassesForClassification number of classes for classification. Default value of 2. + * @return a DecisionTreeModel that can be used for prediction + */ + def train( + input: RDD[LabeledPoint], + algo: Algo, + impurity: Impurity, + maxDepth: Int, + numClassesForClassification: Int): DecisionTreeModel = { + val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification) + new DecisionTree(strategy).train(input) + } /** * Method to train a decision tree model where the instances are represented as an RDD of @@ -250,6 +271,7 @@ object DecisionTree extends Serializable with Logging { * @param algo classification or regression * @param impurity criterion used for information gain calculation * @param maxDepth maximum depth of the tree + * @param numClassesForClassification number of classes for classification. Default value of 2. * @param maxBins maximum number of bins used for splitting features * @param quantileCalculationStrategy algorithm for calculating quantiles * @param categoricalFeaturesInfo A map storing information about the categorical variables and @@ -264,12 +286,13 @@ object DecisionTree extends Serializable with Logging { algo: Algo, impurity: Impurity, maxDepth: Int, + numClassesForClassification: Int, maxBins: Int, quantileCalculationStrategy: QuantileStrategy, categoricalFeaturesInfo: Map[Int,Int]): DecisionTreeModel = { - val strategy = new Strategy(algo, impurity, maxDepth, maxBins, quantileCalculationStrategy, - categoricalFeaturesInfo) - new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification, maxBins, + quantileCalculationStrategy, categoricalFeaturesInfo) + new DecisionTree(strategy).train(input) } private val InvalidBinIndex = -1 @@ -381,6 +404,14 @@ object DecisionTree extends Serializable with Logging { logDebug("numFeatures = " + numFeatures) val numBins = bins(0).length logDebug("numBins = " + numBins) + val numClasses = strategy.numClassesForClassification + logDebug("numClasses = " + numClasses) + val isMulticlassClassification = strategy.isMulticlassClassification + logDebug("isMulticlassClassification = " + isMulticlassClassification) + val isMulticlassClassificationWithCategoricalFeatures + = strategy.isMulticlassWithCategoricalFeatures + logDebug("isMultiClassWithCategoricalFeatures = " + + isMulticlassClassificationWithCategoricalFeatures) // shift when more than one group is used at deep tree level val groupShift = numNodes * groupIndex @@ -436,10 +467,8 @@ object DecisionTree extends Serializable with Logging { /** * Find bin for one feature. */ - def findBin( - featureIndex: Int, - labeledPoint: LabeledPoint, - isFeatureContinuous: Boolean): Int = { + def findBin(featureIndex: Int, labeledPoint: LabeledPoint, + isFeatureContinuous: Boolean, isSpaceSufficientForAllCategoricalSplits: Boolean): Int = { val binForFeatures = bins(featureIndex) val feature = labeledPoint.features(featureIndex) @@ -467,17 +496,28 @@ object DecisionTree extends Serializable with Logging { -1 } + /** + * Sequential search helper method to find bin for categorical feature in multiclass + * classification. The category is returned since each category can belong to multiple + * splits. The actual left/right child allocation per split is performed in the + * sequential phase of the bin aggregate operation. + */ + def sequentialBinSearchForUnorderedCategoricalFeatureInClassification(): Int = { + labeledPoint.features(featureIndex).toInt + } + /** * Sequential search helper method to find bin for categorical feature. */ - def sequentialBinSearchForCategoricalFeature(): Int = { - val numCategoricalBins = strategy.categoricalFeaturesInfo(featureIndex) + def sequentialBinSearchForOrderedCategoricalFeatureInClassification(): Int = { + val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val numCategoricalBins = math.pow(2.0, featureCategories - 1).toInt - 1 var binIndex = 0 while (binIndex < numCategoricalBins) { val bin = bins(featureIndex)(binIndex) - val category = bin.category + val categories = bin.highSplit.categories val features = labeledPoint.features - if (category == features(featureIndex)) { + if (categories.contains(features(featureIndex))) { return binIndex } binIndex += 1 @@ -494,7 +534,13 @@ object DecisionTree extends Serializable with Logging { binIndex } else { // Perform sequential search to find bin for categorical features. - val binIndex = sequentialBinSearchForCategoricalFeature() + val binIndex = { + if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { + sequentialBinSearchForUnorderedCategoricalFeatureInClassification() + } else { + sequentialBinSearchForOrderedCategoricalFeatureInClassification() + } + } if (binIndex == -1){ throw new UnknownError("no bin was found for categorical variable.") } @@ -506,13 +552,16 @@ object DecisionTree extends Serializable with Logging { * Finds bins for all nodes (and all features) at a given level. * For l nodes, k features the storage is as follows: * label, b_11, b_12, .. , b_1k, b_21, b_22, .. , b_2k, b_l1, b_l2, .. , b_lk, - * where b_ij is an integer between 0 and numBins - 1. + * where b_ij is an integer between 0 and numBins - 1 for regressions and binary + * classification and the categorical feature value in multiclass classification. * Invalid sample is denoted by noting bin for feature 1 as -1. */ def findBinsForLevel(labeledPoint: LabeledPoint): Array[Double] = { // Calculate bin index and label per feature per node. val arr = new Array[Double](1 + (numFeatures * numNodes)) + // First element of the array is the label of the instance. arr(0) = labeledPoint.label + // Iterate over nodes. var nodeIndex = 0 while (nodeIndex < numNodes) { val parentFilters = findParentFilters(nodeIndex) @@ -525,8 +574,19 @@ object DecisionTree extends Serializable with Logging { } else { var featureIndex = 0 while (featureIndex < numFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - arr(shift + featureIndex) = findBin(featureIndex, labeledPoint,isFeatureContinuous) + val featureInfo = strategy.categoricalFeaturesInfo.get(featureIndex) + val isFeatureContinuous = featureInfo.isEmpty + if (isFeatureContinuous) { + arr(shift + featureIndex) + = findBin(featureIndex, labeledPoint, isFeatureContinuous, false) + } else { + val featureCategories = featureInfo.get + val isSpaceSufficientForAllCategoricalSplits + = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + arr(shift + featureIndex) + = findBin(featureIndex, labeledPoint, isFeatureContinuous, + isSpaceSufficientForAllCategoricalSplits) + } featureIndex += 1 } } @@ -535,18 +595,61 @@ object DecisionTree extends Serializable with Logging { arr } + // Find feature bins for all nodes at a level. + val binMappedRDD = input.map(x => findBinsForLevel(x)) + + def updateBinForOrderedFeature(arr: Array[Double], agg: Array[Double], nodeIndex: Int, + label: Double, featureIndex: Int) = { + + // Find the bin index for this feature. + val arrShift = 1 + numFeatures * nodeIndex + val arrIndex = arrShift + featureIndex + // Update the left or right count for one bin. + val aggShift = numClasses * numBins * numFeatures * nodeIndex + val aggIndex + = aggShift + numClasses * featureIndex * numBins + arr(arrIndex).toInt * numClasses + val labelInt = label.toInt + agg(aggIndex + labelInt) = agg(aggIndex + labelInt) + 1 + } + + def updateBinForUnorderedFeature(nodeIndex: Int, featureIndex: Int, arr: Array[Double], + label: Double, agg: Array[Double], rightChildShift: Int) = { + // Find the bin index for this feature. + val arrShift = 1 + numFeatures * nodeIndex + val arrIndex = arrShift + featureIndex + // Update the left or right count for one bin. + val aggShift = numClasses * numBins * numFeatures * nodeIndex + val aggIndex + = aggShift + numClasses * featureIndex * numBins + arr(arrIndex).toInt * numClasses + // Find all matching bins and increment their values + val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val numCategoricalBins = math.pow(2.0, featureCategories - 1).toInt - 1 + var binIndex = 0 + while (binIndex < numCategoricalBins) { + val labelInt = label.toInt + if (bins(featureIndex)(binIndex).highSplit.categories.contains(labelInt)) { + agg(aggIndex + binIndex) + = agg(aggIndex + binIndex) + 1 + } else { + agg(rightChildShift + aggIndex + binIndex) + = agg(rightChildShift + aggIndex + binIndex) + 1 + } + binIndex += 1 + } + } + /** * Performs a sequential aggregation over a partition for classification. For l nodes, * k features, either the left count or the right count of one of the p bins is * incremented based upon whether the feature is classified as 0 or 1. * * @param agg Array[Double] storing aggregate calculation of size - * 2 * numSplits * numFeatures*numNodes for classification + * numClasses * numSplits * numFeatures*numNodes for classification * @param arr Array[Double] of size 1 + (numFeatures * numNodes) * @return Array[Double] storing aggregate calculation of size * 2 * numSplits * numFeatures * numNodes for classification */ - def classificationBinSeqOp(arr: Array[Double], agg: Array[Double]) { + def orderedClassificationBinSeqOp(arr: Array[Double], agg: Array[Double]) = { // Iterate over all nodes. var nodeIndex = 0 while (nodeIndex < numNodes) { @@ -559,15 +662,52 @@ object DecisionTree extends Serializable with Logging { // Iterate over all features. var featureIndex = 0 while (featureIndex < numFeatures) { - // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex - // Update the left or right count for one bin. - val aggShift = 2 * numBins * numFeatures * nodeIndex - val aggIndex = aggShift + 2 * featureIndex * numBins + arr(arrIndex).toInt * 2 - label match { - case 0.0 => agg(aggIndex) = agg(aggIndex) + 1 - case 1.0 => agg(aggIndex + 1) = agg(aggIndex + 1) + 1 + updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) + featureIndex += 1 + } + } + nodeIndex += 1 + } + } + + /** + * Performs a sequential aggregation over a partition for classification. For l nodes, + * k features, either the left count or the right count of one of the p bins is + * incremented based upon whether the feature is classified as 0 or 1. + * + * @param agg Array[Double] storing aggregate calculation of size + * numClasses * numSplits * numFeatures*numNodes for classification + * @param arr Array[Double] of size 1 + (numFeatures * numNodes) + * @return Array[Double] storing aggregate calculation of size + * 2 * numClasses * numSplits * numFeatures * numNodes for classification + */ + def unorderedClassificationBinSeqOp(arr: Array[Double], agg: Array[Double]) = { + // Iterate over all nodes. + var nodeIndex = 0 + while (nodeIndex < numNodes) { + // Check whether the instance was valid for this nodeIndex. + val validSignalIndex = 1 + numFeatures * nodeIndex + val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex + if (isSampleValidForNode) { + val rightChildShift = numClasses * numBins * numFeatures * numNodes + // actual class label + val label = arr(0) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + if (isFeatureContinuous) { + updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) + } else { + val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val isSpaceSufficientForAllCategoricalSplits + = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + if (isSpaceSufficientForAllCategoricalSplits) { + updateBinForUnorderedFeature(nodeIndex, featureIndex, arr, label, agg, + rightChildShift) + } else { + updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) + } } featureIndex += 1 } @@ -586,7 +726,7 @@ object DecisionTree extends Serializable with Logging { * @return Array[Double] storing aggregate calculation of size * 3 * numSplits * numFeatures * numNodes for regression */ - def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]) { + def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]) = { // Iterate over all nodes. var nodeIndex = 0 while (nodeIndex < numNodes) { @@ -620,17 +760,20 @@ object DecisionTree extends Serializable with Logging { */ def binSeqOp(agg: Array[Double], arr: Array[Double]): Array[Double] = { strategy.algo match { - case Classification => classificationBinSeqOp(arr, agg) + case Classification => + if(isMulticlassClassificationWithCategoricalFeatures) { + unorderedClassificationBinSeqOp(arr, agg) + } else { + orderedClassificationBinSeqOp(arr, agg) + } case Regression => regressionBinSeqOp(arr, agg) } agg } // Calculate bin aggregate length for classification or regression. - val binAggregateLength = strategy.algo match { - case Classification => 2 * numBins * numFeatures * numNodes - case Regression => 3 * numBins * numFeatures * numNodes - } + val binAggregateLength = numNodes * getElementsPerNode(numFeatures, numBins, numClasses, + isMulticlassClassificationWithCategoricalFeatures, strategy.algo) logDebug("binAggregateLength = " + binAggregateLength) /** @@ -649,9 +792,6 @@ object DecisionTree extends Serializable with Logging { combinedAggregate } - // Find feature bins for all nodes at a level. - val binMappedRDD = input.map(x => findBinsForLevel(x)) - // Calculate bin aggregates. val binAggregates = { binMappedRDD.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp,binCombOp) @@ -668,42 +808,55 @@ object DecisionTree extends Serializable with Logging { * @return information gain and statistics for all splits */ def calculateGainForSplit( - leftNodeAgg: Array[Array[Double]], + leftNodeAgg: Array[Array[Array[Double]]], featureIndex: Int, splitIndex: Int, - rightNodeAgg: Array[Array[Double]], + rightNodeAgg: Array[Array[Array[Double]]], topImpurity: Double): InformationGainStats = { strategy.algo match { case Classification => - val left0Count = leftNodeAgg(featureIndex)(2 * splitIndex) - val left1Count = leftNodeAgg(featureIndex)(2 * splitIndex + 1) - val leftCount = left0Count + left1Count - - val right0Count = rightNodeAgg(featureIndex)(2 * splitIndex) - val right1Count = rightNodeAgg(featureIndex)(2 * splitIndex + 1) - val rightCount = right0Count + right1Count + var classIndex = 0 + val leftCounts: Array[Double] = new Array[Double](numClasses) + val rightCounts: Array[Double] = new Array[Double](numClasses) + var leftTotalCount = 0.0 + var rightTotalCount = 0.0 + while (classIndex < numClasses) { + val leftClassCount = leftNodeAgg(featureIndex)(splitIndex)(classIndex) + val rightClassCount = rightNodeAgg(featureIndex)(splitIndex)(classIndex) + leftCounts(classIndex) = leftClassCount + leftTotalCount += leftClassCount + rightCounts(classIndex) = rightClassCount + rightTotalCount += rightClassCount + classIndex += 1 + } val impurity = { if (level > 0) { topImpurity } else { // Calculate impurity for root node. - strategy.impurity.calculate(left0Count + right0Count, left1Count + right1Count) + val rootNodeCounts = new Array[Double](numClasses) + var classIndex = 0 + while (classIndex < numClasses) { + rootNodeCounts(classIndex) = leftCounts(classIndex) + rightCounts(classIndex) + classIndex += 1 + } + strategy.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) } } - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity,1) + if (leftTotalCount == 0) { + return new InformationGainStats(0, topImpurity, topImpurity, Double.MinValue, 1) } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity, topImpurity, Double.MinValue,0) + if (rightTotalCount == 0) { + return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, 1) } - val leftImpurity = strategy.impurity.calculate(left0Count, left1Count) - val rightImpurity = strategy.impurity.calculate(right0Count, right1Count) + val leftImpurity = strategy.impurity.calculate(leftCounts, leftTotalCount) + val rightImpurity = strategy.impurity.calculate(rightCounts, rightTotalCount) - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val leftWeight = leftTotalCount / (leftTotalCount + rightTotalCount) + val rightWeight = rightTotalCount / (leftTotalCount + rightTotalCount) val gain = { if (level > 0) { @@ -713,17 +866,34 @@ object DecisionTree extends Serializable with Logging { } } - val predict = (left1Count + right1Count) / (leftCount + rightCount) + val totalCount = leftTotalCount + rightTotalCount - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) + // Sum of count for each label + val leftRightCounts: Array[Double] + = leftCounts.zip(rightCounts) + .map{case (leftCount, rightCount) => leftCount + rightCount} + + def indexOfLargestArrayElement(array: Array[Double]): Int = { + val result = array.foldLeft(-1, Double.MinValue, 0) { + case ((maxIndex, maxValue, currentIndex), currentValue) => + if(currentValue > maxValue) (currentIndex, currentValue, currentIndex + 1) + else (maxIndex, maxValue, currentIndex + 1) + } + if (result._1 < 0) 0 else result._1 + } + + val predict = indexOfLargestArrayElement(leftRightCounts) + val prob = leftRightCounts(predict) / totalCount + + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) case Regression => - val leftCount = leftNodeAgg(featureIndex)(3 * splitIndex) - val leftSum = leftNodeAgg(featureIndex)(3 * splitIndex + 1) - val leftSumSquares = leftNodeAgg(featureIndex)(3 * splitIndex + 2) + val leftCount = leftNodeAgg(featureIndex)(splitIndex)(0) + val leftSum = leftNodeAgg(featureIndex)(splitIndex)(1) + val leftSumSquares = leftNodeAgg(featureIndex)(splitIndex)(2) - val rightCount = rightNodeAgg(featureIndex)(3 * splitIndex) - val rightSum = rightNodeAgg(featureIndex)(3 * splitIndex + 1) - val rightSumSquares = rightNodeAgg(featureIndex)(3 * splitIndex + 2) + val rightCount = rightNodeAgg(featureIndex)(splitIndex)(0) + val rightSum = rightNodeAgg(featureIndex)(splitIndex)(1) + val rightSumSquares = rightNodeAgg(featureIndex)(splitIndex)(2) val impurity = { if (level > 0) { @@ -768,104 +938,149 @@ object DecisionTree extends Serializable with Logging { /** * Extracts left and right split aggregates. * @param binData Array[Double] of size 2*numFeatures*numSplits - * @return (leftNodeAgg, rightNodeAgg) tuple of type (Array[Double], - * Array[Double]) where each array is of size(numFeature,2*(numSplits-1)) + * @return (leftNodeAgg, rightNodeAgg) tuple of type (Array[Array[Array[Double\]\]\], + * Array[Array[Array[Double\]\]\]) where each array is of size(numFeature, + * (numBins - 1), numClasses) */ def extractLeftRightNodeAggregates( - binData: Array[Double]): (Array[Array[Double]], Array[Array[Double]]) = { + binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { + + + def findAggForOrderedFeatureClassification( + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], + featureIndex: Int) { + + // shift for this featureIndex + val shift = numClasses * featureIndex * numBins + + var classIndex = 0 + while (classIndex < numClasses) { + // left node aggregate for the lowest split + leftNodeAgg(featureIndex)(0)(classIndex) = binData(shift + classIndex) + // right node aggregate for the highest split + rightNodeAgg(featureIndex)(numBins - 2)(classIndex) + = binData(shift + (numClasses * (numBins - 1)) + classIndex) + classIndex += 1 + } + + // Iterate over all splits. + var splitIndex = 1 + while (splitIndex < numBins - 1) { + // calculating left node aggregate for a split as a sum of left node aggregate of a + // lower split and the left bin aggregate of a bin where the split is a high split + var innerClassIndex = 0 + while (innerClassIndex < numClasses) { + leftNodeAgg(featureIndex)(splitIndex)(innerClassIndex) + = binData(shift + numClasses * splitIndex + innerClassIndex) + + leftNodeAgg(featureIndex)(splitIndex - 1)(innerClassIndex) + rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(innerClassIndex) = + binData(shift + (numClasses * (numBins - 1 - splitIndex) + innerClassIndex)) + + rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(innerClassIndex) + innerClassIndex += 1 + } + splitIndex += 1 + } + } + + def findAggForUnorderedFeatureClassification( + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], + featureIndex: Int) { + + val rightChildShift = numClasses * numBins * numFeatures + var splitIndex = 0 + while (splitIndex < numBins - 1) { + var classIndex = 0 + while (classIndex < numClasses) { + // shift for this featureIndex + val shift = numClasses * featureIndex * numBins + splitIndex * numClasses + val leftBinValue = binData(shift + classIndex) + val rightBinValue = binData(rightChildShift + shift + classIndex) + leftNodeAgg(featureIndex)(splitIndex)(classIndex) = leftBinValue + rightNodeAgg(featureIndex)(splitIndex)(classIndex) = rightBinValue + classIndex += 1 + } + splitIndex += 1 + } + } + + def findAggForRegression( + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], + featureIndex: Int) { + + // shift for this featureIndex + val shift = 3 * featureIndex * numBins + // left node aggregate for the lowest split + leftNodeAgg(featureIndex)(0)(0) = binData(shift + 0) + leftNodeAgg(featureIndex)(0)(1) = binData(shift + 1) + leftNodeAgg(featureIndex)(0)(2) = binData(shift + 2) + + // right node aggregate for the highest split + rightNodeAgg(featureIndex)(numBins - 2)(0) = + binData(shift + (3 * (numBins - 1))) + rightNodeAgg(featureIndex)(numBins - 2)(1) = + binData(shift + (3 * (numBins - 1)) + 1) + rightNodeAgg(featureIndex)(numBins - 2)(2) = + binData(shift + (3 * (numBins - 1)) + 2) + + // Iterate over all splits. + var splitIndex = 1 + while (splitIndex < numBins - 1) { + var i = 0 // index for regression histograms + while (i < 3) { // count, sum, sum^2 + // calculating left node aggregate for a split as a sum of left node aggregate of a + // lower split and the left bin aggregate of a bin where the split is a high split + leftNodeAgg(featureIndex)(splitIndex)(i) = binData(shift + 3 * splitIndex + i) + + leftNodeAgg(featureIndex)(splitIndex - 1)(i) + // calculating right node aggregate for a split as a sum of right node aggregate of a + // higher split and the right bin aggregate of a bin where the split is a low split + rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(i) = + binData(shift + (3 * (numBins - 1 - splitIndex) + i)) + + rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(i) + i += 1 + } + splitIndex += 1 + } + } + strategy.algo match { case Classification => // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1)) - val rightNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1)) - // Iterate over all features. + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) var featureIndex = 0 while (featureIndex < numFeatures) { - // shift for this featureIndex - val shift = 2 * featureIndex * numBins - - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0) = binData(shift + 0) - leftNodeAgg(featureIndex)(1) = binData(shift + 1) - - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(2 * (numBins - 2)) - = binData(shift + (2 * (numBins - 1))) - rightNodeAgg(featureIndex)(2 * (numBins - 2) + 1) - = binData(shift + (2 * (numBins - 1)) + 1) - - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - leftNodeAgg(featureIndex)(2 * splitIndex) = binData(shift + 2 * splitIndex) + - leftNodeAgg(featureIndex)(2 * splitIndex - 2) - leftNodeAgg(featureIndex)(2 * splitIndex + 1) = binData(shift + 2 * splitIndex + 1) + - leftNodeAgg(featureIndex)(2 * splitIndex - 2 + 1) - - // calculating right node aggregate for a split as a sum of right node aggregate of a - // higher split and the right bin aggregate of a bin where the split is a low split - rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) = - binData(shift + (2 *(numBins - 1 - splitIndex))) + - rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex)) - rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (2* (numBins - 1 - splitIndex) + 1)) + - rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex) + 1) - - splitIndex += 1 + if (isMulticlassClassificationWithCategoricalFeatures){ + val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + if (isFeatureContinuous) { + findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) + } else { + val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val isSpaceSufficientForAllCategoricalSplits + = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + if (isSpaceSufficientForAllCategoricalSplits) { + findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) + } else { + findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) + } + } + } else { + findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) } featureIndex += 1 } + (leftNodeAgg, rightNodeAgg) case Regression => // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1)) - val rightNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1)) + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) // Iterate over all features. var featureIndex = 0 while (featureIndex < numFeatures) { - // shift for this featureIndex - val shift = 3 * featureIndex * numBins - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0) = binData(shift + 0) - leftNodeAgg(featureIndex)(1) = binData(shift + 1) - leftNodeAgg(featureIndex)(2) = binData(shift + 2) - - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(3 * (numBins - 2)) = - binData(shift + (3 * (numBins - 1))) - rightNodeAgg(featureIndex)(3 * (numBins - 2) + 1) = - binData(shift + (3 * (numBins - 1)) + 1) - rightNodeAgg(featureIndex)(3 * (numBins - 2) + 2) = - binData(shift + (3 * (numBins - 1)) + 2) - - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - leftNodeAgg(featureIndex)(3 * splitIndex) = binData(shift + 3 * splitIndex) + - leftNodeAgg(featureIndex)(3 * splitIndex - 3) - leftNodeAgg(featureIndex)(3 * splitIndex + 1) = binData(shift + 3 * splitIndex + 1) + - leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 1) - leftNodeAgg(featureIndex)(3 * splitIndex + 2) = binData(shift + 3 * splitIndex + 2) + - leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 2) - - // calculating right node aggregate for a split as a sum of right node aggregate of a - // higher split and the right bin aggregate of a bin where the split is a low split - rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex)) = - binData(shift + (3 * (numBins - 1 - splitIndex))) + - rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex)) - rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 1) = - binData(shift + (3 * (numBins - 1 - splitIndex) + 1)) + - rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 1) - rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 2) = - binData(shift + (3 * (numBins - 1 - splitIndex) + 2)) + - rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 2) - - splitIndex += 1 - } + findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) featureIndex += 1 } (leftNodeAgg, rightNodeAgg) @@ -876,8 +1091,8 @@ object DecisionTree extends Serializable with Logging { * Calculates information gain for all nodes splits. */ def calculateGainsForAllNodeSplits( - leftNodeAgg: Array[Array[Double]], - rightNodeAgg: Array[Array[Double]], + leftNodeAgg: Array[Array[Array[Double]]], + rightNodeAgg: Array[Array[Array[Double]]], nodeImpurity: Double): Array[Array[InformationGainStats]] = { val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1) @@ -918,7 +1133,22 @@ object DecisionTree extends Serializable with Logging { while (featureIndex < numFeatures) { // Iterate over all splits. var splitIndex = 0 - while (splitIndex < numBins - 1) { + val maxSplitIndex : Double = { + val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + if (isFeatureContinuous) { + numBins - 1 + } else { // Categorical feature + val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val isSpaceSufficientForAllCategoricalSplits + = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { + math.pow(2.0, featureCategories - 1).toInt - 1 + } else { // Binary classification + featureCategories + } + } + } + while (splitIndex < maxSplitIndex) { val gainStats = gains(featureIndex)(splitIndex) if (gainStats.gain > bestGainStats.gain) { bestGainStats = gainStats @@ -944,9 +1174,23 @@ object DecisionTree extends Serializable with Logging { def getBinDataForNode(node: Int): Array[Double] = { strategy.algo match { case Classification => - val shift = 2 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 2 * numBins * numFeatures) - binsForNode + if (isMulticlassClassificationWithCategoricalFeatures) { + val shift = numClasses * node * numBins * numFeatures + val rightChildShift = numClasses * numBins * numFeatures * numNodes + val binsForNode = { + val leftChildData + = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) + val rightChildData + = binAggregates.slice(rightChildShift + shift, + rightChildShift + shift + numClasses * numBins * numFeatures) + leftChildData ++ rightChildData + } + binsForNode + } else { + val shift = numClasses * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) + binsForNode + } case Regression => val shift = 3 * node * numBins * numFeatures val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) @@ -963,14 +1207,26 @@ object DecisionTree extends Serializable with Logging { val binsForNode: Array[Double] = getBinDataForNode(node) logDebug("nodeImpurityIndex = " + nodeImpurityIndex) val parentNodeImpurity = parentImpurities(nodeImpurityIndex) - logDebug("node impurity = " + parentNodeImpurity) + logDebug("parent node impurity = " + parentNodeImpurity) bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity) node += 1 } - bestSplits } + private def getElementsPerNode(numFeatures: Int, numBins: Int, numClasses: Int, + isMulticlassClassificationWithCategoricalFeatures: Boolean, algo: Algo): Int = { + algo match { + case Classification => + if (isMulticlassClassificationWithCategoricalFeatures) { + 2 * numClasses * numBins * numFeatures + } else { + numClasses * numBins * numFeatures + } + case Regression => 3 * numBins * numFeatures + } + } + /** * Returns split and bins for decision tree calculation. * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data @@ -992,17 +1248,23 @@ object DecisionTree extends Serializable with Logging { val maxBins = strategy.maxBins val numBins = if (maxBins <= count) maxBins else count.toInt logDebug("numBins = " + numBins) + val isMulticlassClassification = strategy.isMulticlassClassification + logDebug("isMulticlassClassification = " + isMulticlassClassification) + /* - * TODO: Add a require statement ensuring #bins is always greater than the categories. + * Ensure #bins is always greater than the categories. For multiclass classification, + * #bins should be greater than 2^(maxCategories - 1) - 1. * It's a limitation of the current implementation but a reasonable trade-off since features * with large number of categories get favored over continuous features. */ if (strategy.categoricalFeaturesInfo.size > 0) { val maxCategoriesForFeatures = strategy.categoricalFeaturesInfo.maxBy(_._2)._2 - require(numBins >= maxCategoriesForFeatures) + require(numBins > maxCategoriesForFeatures, "numBins should be greater than max categories " + + "in categorical features") } + // Calculate the number of sample for approximate quantile calculation. val requiredSamples = numBins*numBins val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0 @@ -1036,48 +1298,93 @@ object DecisionTree extends Serializable with Logging { val split = new Split(featureIndex, featureSamples(sampleIndex), Continuous, List()) splits(featureIndex)(index) = split } - } else { - val maxFeatureValue = strategy.categoricalFeaturesInfo(featureIndex) - require(maxFeatureValue < numBins, "number of categories should be less than number " + - "of bins") - - // For categorical variables, each bin is a category. The bins are sorted and they - // are ordered by calculating the centroid of their corresponding labels. - val centroidForCategories = - sampledInput.map(lp => (lp.features(featureIndex),lp.label)) - .groupBy(_._1) - .mapValues(x => x.map(_._2).sum / x.map(_._1).length) - - // Check for missing categorical variables and putting them last in the sorted list. - val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() - for (i <- 0 until maxFeatureValue) { - if (centroidForCategories.contains(i)) { - fullCentroidForCategories(i) = centroidForCategories(i) - } else { - fullCentroidForCategories(i) = Double.MaxValue - } - } - - // bins sorted by centroids - val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) - - logDebug("centriod for categorical variable = " + categoriesSortedByCentroid) - - var categoriesForSplit = List[Double]() - categoriesSortedByCentroid.iterator.zipWithIndex.foreach { - case ((key, value), index) => - categoriesForSplit = key :: categoriesForSplit - splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, Categorical, - categoriesForSplit) + } else { // Categorical feature + val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val isSpaceSufficientForAllCategoricalSplits + = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + + // Use different bin/split calculation strategy for categorical features in multiclass + // classification that satisfy the space constraint + if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { + // 2^(maxFeatureValue- 1) - 1 combinations + var index = 0 + while (index < math.pow(2.0, featureCategories - 1).toInt - 1) { + val categories: List[Double] + = extractMultiClassCategories(index + 1, featureCategories) + splits(featureIndex)(index) + = new Split(featureIndex, Double.MinValue, Categorical, categories) bins(featureIndex)(index) = { if (index == 0) { - new Bin(new DummyCategoricalSplit(featureIndex, Categorical), - splits(featureIndex)(0), Categorical, key) + new Bin( + new DummyCategoricalSplit(featureIndex, Categorical), + splits(featureIndex)(0), + Categorical, + Double.MinValue) } else { - new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Categorical, key) + new Bin( + splits(featureIndex)(index - 1), + splits(featureIndex)(index), + Categorical, + Double.MinValue) } } + index += 1 + } + } else { + + val centroidForCategories = { + if (isMulticlassClassification) { + // For categorical variables in multiclass classification, + // each bin is a category. The bins are sorted and they + // are ordered by calculating the impurity of their corresponding labels. + sampledInput.map(lp => (lp.features(featureIndex), lp.label)) + .groupBy(_._1) + .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) + .map(x => (x._1, x._2.values.toArray)) + .map(x => (x._1, strategy.impurity.calculate(x._2,x._2.sum))) + } else { // regression or binary classification + // For categorical variables in regression and binary classification, + // each bin is a category. The bins are sorted and they + // are ordered by calculating the centroid of their corresponding labels. + sampledInput.map(lp => (lp.features(featureIndex), lp.label)) + .groupBy(_._1) + .mapValues(x => x.map(_._2).sum / x.map(_._1).length) + } + } + + logDebug("centriod for categories = " + centroidForCategories.mkString(",")) + + // Check for missing categorical variables and putting them last in the sorted list. + val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() + for (i <- 0 until featureCategories) { + if (centroidForCategories.contains(i)) { + fullCentroidForCategories(i) = centroidForCategories(i) + } else { + fullCentroidForCategories(i) = Double.MaxValue + } + } + + // bins sorted by centroids + val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) + + logDebug("centriod for categorical variable = " + categoriesSortedByCentroid) + + var categoriesForSplit = List[Double]() + categoriesSortedByCentroid.iterator.zipWithIndex.foreach { + case ((key, value), index) => + categoriesForSplit = key :: categoriesForSplit + splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, + Categorical, categoriesForSplit) + bins(featureIndex)(index) = { + if (index == 0) { + new Bin(new DummyCategoricalSplit(featureIndex, Categorical), + splits(featureIndex)(0), Categorical, key) + } else { + new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), + Categorical, key) + } + } + } } } featureIndex += 1 @@ -1107,4 +1414,29 @@ object DecisionTree extends Serializable with Logging { throw new UnsupportedOperationException("approximate histogram not supported yet.") } } + + /** + * Nested method to extract list of eligible categories given an index. It extracts the + * position of ones in a binary representation of the input. If binary + * representation of an number is 01101 (13), the output list should (3.0, 2.0, + * 0.0). The maxFeatureValue depict the number of rightmost digits that will be tested for ones. + */ + private[tree] def extractMultiClassCategories( + input: Int, + maxFeatureValue: Int): List[Double] = { + var categories = List[Double]() + var j = 0 + var bitShiftedInput = input + while (j < maxFeatureValue) { + if (bitShiftedInput % 2 != 0) { + // updating the list of categories. + categories = j.toDouble :: categories + } + // Right shift by one + bitShiftedInput = bitShiftedInput >> 1 + j += 1 + } + categories + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 1b505fd76eb75..7c027ac2fda6b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -28,6 +28,8 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * @param algo classification or regression * @param impurity criterion used for information gain calculation * @param maxDepth maximum depth of the tree + * @param numClassesForClassification number of classes for classification. Default value is 2 + * leads to binary classification * @param maxBins maximum number of bins used for splitting features * @param quantileCalculationStrategy algorithm for calculating quantiles * @param categoricalFeaturesInfo A map storing information about the categorical variables and the @@ -44,7 +46,15 @@ class Strategy ( val algo: Algo, val impurity: Impurity, val maxDepth: Int, + val numClassesForClassification: Int = 2, val maxBins: Int = 100, val quantileCalculationStrategy: QuantileStrategy = Sort, val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), - val maxMemoryInMB: Int = 128) extends Serializable + val maxMemoryInMB: Int = 128) extends Serializable { + + require(numClassesForClassification >= 2) + val isMulticlassClassification = numClassesForClassification > 2 + val isMulticlassWithCategoricalFeatures + = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 60f43e9278d2a..a0e2d91762782 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -31,23 +31,35 @@ object Entropy extends Impurity { /** * :: DeveloperApi :: - * entropy calculation - * @param c0 count of instances with label 0 - * @param c1 count of instances with label 1 - * @return entropy value + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels + * @return information value */ @DeveloperApi - override def calculate(c0: Double, c1: Double): Double = { - if (c0 == 0 || c1 == 0) { - 0 - } else { - val total = c0 + c1 - val f0 = c0 / total - val f1 = c1 / total - -(f0 * log2(f0)) - (f1 * log2(f1)) + override def calculate(counts: Array[Double], totalCount: Double): Double = { + val numClasses = counts.length + var impurity = 0.0 + var classIndex = 0 + while (classIndex < numClasses) { + val classCount = counts(classIndex) + if (classCount != 0) { + val freq = classCount / totalCount + impurity -= freq * log2(freq) + } + classIndex += 1 } + impurity } + /** + * :: DeveloperApi :: + * variance calculation + * @param count number of instances + * @param sum sum of labels + * @param sumSquares summation of squares of the labels + */ + @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c51d76d9b4c5b..48144b5e6d1e4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -30,23 +30,32 @@ object Gini extends Impurity { /** * :: DeveloperApi :: - * Gini coefficient calculation - * @param c0 count of instances with label 0 - * @param c1 count of instances with label 1 - * @return Gini coefficient value + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels + * @return information value */ @DeveloperApi - override def calculate(c0: Double, c1: Double): Double = { - if (c0 == 0 || c1 == 0) { - 0 - } else { - val total = c0 + c1 - val f0 = c0 / total - val f1 = c1 / total - 1 - f0 * f0 - f1 * f1 + override def calculate(counts: Array[Double], totalCount: Double): Double = { + val numClasses = counts.length + var impurity = 1.0 + var classIndex = 0 + while (classIndex < numClasses) { + val freq = counts(classIndex) / totalCount + impurity -= freq * freq + classIndex += 1 } + impurity } + /** + * :: DeveloperApi :: + * variance calculation + * @param count number of instances + * @param sum sum of labels + * @param sumSquares summation of squares of the labels + */ + @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 8eab247cf0932..7b2a9320cc21d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -28,13 +28,13 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * information calculation for binary classification - * @param c0 count of instances with label 0 - * @param c1 count of instances with label 1 + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels * @return information value */ @DeveloperApi - def calculate(c0 : Double, c1 : Double): Double + def calculate(counts: Array[Double], totalCount: Double): Double /** * :: DeveloperApi :: diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 47d07122af30f..97149a99ead59 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -25,7 +25,16 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} */ @Experimental object Variance extends Impurity { - override def calculate(c0: Double, c1: Double): Double = + + /** + * :: DeveloperApi :: + * information calculation for multiclass classification + * @param counts Array[Double] with counts for each label + * @param totalCount sum of counts for all labels + * @return information value + */ + @DeveloperApi + override def calculate(counts: Array[Double], totalCount: Double): Double = throw new UnsupportedOperationException("Variance.calculate") /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index 2d71e1e366069..c89c1e371a40e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * @param highSplit signifying the upper threshold for the continuous feature to be * accepted in the bin * @param featureType type of feature -- categorical or continuous - * @param category categorical label value accepted in the bin + * @param category categorical label value accepted in the bin for binary classification */ private[tree] case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index cc8a24cce9614..fb12298e0f5d3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -27,6 +27,7 @@ import org.apache.spark.annotation.DeveloperApi * @param leftImpurity left node impurity * @param rightImpurity right node impurity * @param predict predicted value + * @param prob probability of the label (classification only) */ @DeveloperApi class InformationGainStats( @@ -34,10 +35,11 @@ class InformationGainStats( val impurity: Double, val leftImpurity: Double, val rightImpurity: Double, - val predict: Double) extends Serializable { + val predict: Double, + val prob: Double = 0.0) extends Serializable { override def toString = { - "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f" - .format(gain, impurity, leftImpurity, rightImpurity, predict) + "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f, prob = %f" + .format(gain, impurity, leftImpurity, rightImpurity, predict, prob) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index bcb11876b8f4f..5961a618c59d9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.mllib.tree import org.scalatest.FunSuite -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.Filter import org.apache.spark.mllib.tree.model.Split @@ -28,6 +27,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.regression.LabeledPoint class DecisionTreeSuite extends FunSuite with LocalSparkContext { @@ -35,7 +35,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 100) + val strategy = new Strategy(Classification, Gini, 3, 2, 100) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) assert(splits.length === 2) assert(bins.length === 2) @@ -51,6 +51,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) @@ -130,8 +131,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100, - categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) // Check splits. @@ -231,6 +233,162 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(1)(3) === null) } + test("extract categories from a number for multiclass classification") { + val l = DecisionTree.extractMultiClassCategories(13, 10) + assert(l.length === 3) + assert(List(3.0, 2.0, 0.0).toSeq == l.toSeq) + } + + test("split and bin calculations for unordered categorical variables with multiclass " + + "classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Classification, + Gini, + maxDepth = 3, + numClassesForClassification = 100, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + + // Expecting 2^2 - 1 = 3 bins/splits + assert(splits(0)(0).feature === 0) + assert(splits(0)(0).threshold === Double.MinValue) + assert(splits(0)(0).featureType === Categorical) + assert(splits(0)(0).categories.length === 1) + assert(splits(0)(0).categories.contains(0.0)) + assert(splits(1)(0).feature === 1) + assert(splits(1)(0).threshold === Double.MinValue) + assert(splits(1)(0).featureType === Categorical) + assert(splits(1)(0).categories.length === 1) + assert(splits(1)(0).categories.contains(0.0)) + + assert(splits(0)(1).feature === 0) + assert(splits(0)(1).threshold === Double.MinValue) + assert(splits(0)(1).featureType === Categorical) + assert(splits(0)(1).categories.length === 1) + assert(splits(0)(1).categories.contains(1.0)) + assert(splits(1)(1).feature === 1) + assert(splits(1)(1).threshold === Double.MinValue) + assert(splits(1)(1).featureType === Categorical) + assert(splits(1)(1).categories.length === 1) + assert(splits(1)(1).categories.contains(1.0)) + + assert(splits(0)(2).feature === 0) + assert(splits(0)(2).threshold === Double.MinValue) + assert(splits(0)(2).featureType === Categorical) + assert(splits(0)(2).categories.length === 2) + assert(splits(0)(2).categories.contains(0.0)) + assert(splits(0)(2).categories.contains(1.0)) + assert(splits(1)(2).feature === 1) + assert(splits(1)(2).threshold === Double.MinValue) + assert(splits(1)(2).featureType === Categorical) + assert(splits(1)(2).categories.length === 2) + assert(splits(1)(2).categories.contains(0.0)) + assert(splits(1)(2).categories.contains(1.0)) + + assert(splits(0)(3) === null) + assert(splits(1)(3) === null) + + + // Check bins. + + assert(bins(0)(0).category === Double.MinValue) + assert(bins(0)(0).lowSplit.categories.length === 0) + assert(bins(0)(0).highSplit.categories.length === 1) + assert(bins(0)(0).highSplit.categories.contains(0.0)) + assert(bins(1)(0).category === Double.MinValue) + assert(bins(1)(0).lowSplit.categories.length === 0) + assert(bins(1)(0).highSplit.categories.length === 1) + assert(bins(1)(0).highSplit.categories.contains(0.0)) + + assert(bins(0)(1).category === Double.MinValue) + assert(bins(0)(1).lowSplit.categories.length === 1) + assert(bins(0)(1).lowSplit.categories.contains(0.0)) + assert(bins(0)(1).highSplit.categories.length === 1) + assert(bins(0)(1).highSplit.categories.contains(1.0)) + assert(bins(1)(1).category === Double.MinValue) + assert(bins(1)(1).lowSplit.categories.length === 1) + assert(bins(1)(1).lowSplit.categories.contains(0.0)) + assert(bins(1)(1).highSplit.categories.length === 1) + assert(bins(1)(1).highSplit.categories.contains(1.0)) + + assert(bins(0)(2).category === Double.MinValue) + assert(bins(0)(2).lowSplit.categories.length === 1) + assert(bins(0)(2).lowSplit.categories.contains(1.0)) + assert(bins(0)(2).highSplit.categories.length === 2) + assert(bins(0)(2).highSplit.categories.contains(1.0)) + assert(bins(0)(2).highSplit.categories.contains(0.0)) + assert(bins(1)(2).category === Double.MinValue) + assert(bins(1)(2).lowSplit.categories.length === 1) + assert(bins(1)(2).lowSplit.categories.contains(1.0)) + assert(bins(1)(2).highSplit.categories.length === 2) + assert(bins(1)(2).highSplit.categories.contains(1.0)) + assert(bins(1)(2).highSplit.categories.contains(0.0)) + + assert(bins(0)(3) === null) + assert(bins(1)(3) === null) + + } + + test("split and bin calculations for ordered categorical variables with multiclass " + + "classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + assert(arr.length === 3000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Classification, + Gini, + maxDepth = 3, + numClassesForClassification = 100, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + + // 2^10 - 1 > 100, so categorical variables will be ordered + + assert(splits(0)(0).feature === 0) + assert(splits(0)(0).threshold === Double.MinValue) + assert(splits(0)(0).featureType === Categorical) + assert(splits(0)(0).categories.length === 1) + assert(splits(0)(0).categories.contains(1.0)) + + assert(splits(0)(1).feature === 0) + assert(splits(0)(1).threshold === Double.MinValue) + assert(splits(0)(1).featureType === Categorical) + assert(splits(0)(1).categories.length === 2) + assert(splits(0)(1).categories.contains(2.0)) + + assert(splits(0)(2).feature === 0) + assert(splits(0)(2).threshold === Double.MinValue) + assert(splits(0)(2).featureType === Categorical) + assert(splits(0)(2).categories.length === 3) + assert(splits(0)(2).categories.contains(2.0)) + assert(splits(0)(2).categories.contains(1.0)) + + assert(splits(0)(10) === null) + assert(splits(1)(10) === null) + + + // Check bins. + + assert(bins(0)(0).category === 1.0) + assert(bins(0)(0).lowSplit.categories.length === 0) + assert(bins(0)(0).highSplit.categories.length === 1) + assert(bins(0)(0).highSplit.categories.contains(1.0)) + assert(bins(0)(1).category === 2.0) + assert(bins(0)(1).lowSplit.categories.length === 1) + assert(bins(0)(1).highSplit.categories.length === 2) + assert(bins(0)(1).highSplit.categories.contains(1.0)) + assert(bins(0)(1).highSplit.categories.contains(2.0)) + + assert(bins(0)(10) === null) + + } + + test("classification stump with all categorical variables") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) @@ -238,6 +396,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, + numClassesForClassification = 2, maxDepth = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) @@ -253,8 +412,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = bestSplits(0)._2 assert(stats.gain > 0) - assert(stats.predict > 0.5) - assert(stats.predict < 0.7) + assert(stats.predict === 1) + assert(stats.prob == 0.6) assert(stats.impurity > 0.2) } @@ -280,8 +439,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = bestSplits(0)._2 assert(stats.gain > 0) - assert(stats.predict > 0.5) - assert(stats.predict < 0.7) + assert(stats.predict == 0.6) assert(stats.impurity > 0.2) } @@ -289,7 +447,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 100) + val strategy = new Strategy(Classification, Gini, 3, 2, 100) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) assert(splits.length === 2) assert(splits(0).length === 99) @@ -312,7 +470,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 100) + val strategy = new Strategy(Classification, Gini, 3, 2, 100) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) assert(splits.length === 2) assert(splits(0).length === 99) @@ -336,7 +494,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 100) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) assert(splits.length === 2) assert(splits(0).length === 99) @@ -360,7 +518,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 100) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) assert(splits.length === 2) assert(splits(0).length === 99) @@ -380,11 +538,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 1) } - test("test second level node building with/without groups") { + test("second level node building with/without groups") { val arr = DecisionTreeSuite.generateOrderedLabeledPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 100) + val strategy = new Strategy(Classification, Entropy, 3, 2, 100) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) assert(splits.length === 2) assert(splits(0).length === 99) @@ -426,6 +584,82 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { } + test("stump with categorical variables for multiclass classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + assert(strategy.isMulticlassClassification) + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + Array[List[Filter]](), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + assert(bestSplit.feature === 0) + assert(bestSplit.categories.length === 1) + assert(bestSplit.categories.contains(1)) + assert(bestSplit.featureType === Categorical) + } + + test("stump with continuous variables for multiclass classification") { + val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClassesForClassification = 3) + assert(strategy.isMulticlassClassification) + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + Array[List[Filter]](), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + + assert(bestSplit.feature === 1) + assert(bestSplit.featureType === Continuous) + assert(bestSplit.threshold > 1980) + assert(bestSplit.threshold < 2020) + + } + + test("stump with continuous + categorical variables for multiclass classification") { + val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) + assert(strategy.isMulticlassClassification) + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + Array[List[Filter]](), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + + assert(bestSplit.feature === 1) + assert(bestSplit.featureType === Continuous) + assert(bestSplit.threshold > 1980) + assert(bestSplit.threshold < 2020) + } + + test("stump with categorical variables for ordered multiclass classification") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + assert(strategy.isMulticlassClassification) + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + Array[List[Filter]](), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + assert(bestSplit.feature === 0) + assert(bestSplit.categories.length === 1) + assert(bestSplit.categories.contains(1.0)) + assert(bestSplit.featureType === Categorical) + } + + } object DecisionTreeSuite { @@ -473,4 +707,47 @@ object DecisionTreeSuite { } arr } + + def generateCategoricalDataPointsForMulticlass(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](3000) + for (i <- 0 until 3000) { + if (i < 1000) { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, 2.0)) + } else if (i < 2000) { + arr(i) = new LabeledPoint(1.0, Vectors.dense(1.0, 2.0)) + } else { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, 2.0)) + } + } + arr + } + + def generateContinuousDataPointsForMulticlass(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](3000) + for (i <- 0 until 3000) { + if (i < 2000) { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, i)) + } else { + arr(i) = new LabeledPoint(1.0, Vectors.dense(2.0, i)) + } + } + arr + } + + def generateCategoricalDataPointsForMulticlassForOrderedFeatures(): + Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](3000) + for (i <- 0 until 3000) { + if (i < 1000) { + arr(i) = new LabeledPoint(2.0, Vectors.dense(2.0, 2.0)) + } else if (i < 2000) { + arr(i) = new LabeledPoint(1.0, Vectors.dense(1.0, 2.0)) + } else { + arr(i) = new LabeledPoint(1.0, Vectors.dense(2.0, 2.0)) + } + } + arr + } + + } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 3487f7c5c1255..e0f433b26f7ff 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -82,7 +82,15 @@ object MimaExcludes { MimaBuild.excludeSparkClass("util.SerializableHyperLogLog") ++ MimaBuild.excludeSparkClass("storage.Values") ++ MimaBuild.excludeSparkClass("storage.Entry") ++ - MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") + MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") ++ + Seq( + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Gini.calculate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Entropy.calculate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Variance.calculate") + ) case v if v.startsWith("1.0") => Seq( MimaBuild.excludeSparkPackage("api.java"), From 3a1709fa557f2bd6d101bc67a9e773882078c527 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 18 Jul 2014 16:24:00 -0500 Subject: [PATCH 0007/1492] [SPARK-2535][SQL] Add StringComparison case to NullPropagation. `StringComparison` expressions including `null` literal cases could be added to `NullPropagation`. Author: Takuya UESHIN Closes #1451 from ueshin/issues/SPARK-2535 and squashes the following commits: e99c237 [Takuya UESHIN] Add some tests. 8f9b984 [Takuya UESHIN] Add StringComparison case to NullPropagation. --- .../sql/catalyst/optimizer/Optimizer.scala | 5 ++ .../ExpressionEvaluationSuite.scala | 23 ++++- .../optimizer/ConstantFoldingSuite.scala | 10 ++- .../optimizer/LikeSimplificationSuite.scala | 90 +++++++++++++++++++ 4 files changed, 125 insertions(+), 3 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala 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 7f32f6b8bcf46..c65987b7120b2 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 @@ -188,6 +188,11 @@ object NullPropagation extends Rule[LogicalPlan] { case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) case _ => e } + case e: StringComparison => e.children match { + case Literal(null, _) :: right :: Nil => Literal(null, e.dataType) + case left :: Literal(null, _) :: Nil => Literal(null, e.dataType) + case _ => e + } } } } 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 143330bd64716..73f546455b67f 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,7 +466,28 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 === c2, false, row) checkEvaluation(c1 !== c2, true, row) } - + + test("StringComparison") { + val row = new GenericRow(Array[Any]("abc", null)) + val c1 = 'a.string.at(0) + val c2 = 'a.string.at(1) + + checkEvaluation(Contains(c1, "b"), true, row) + checkEvaluation(Contains(c1, "x"), false, row) + checkEvaluation(Contains(c2, "b"), null, row) + checkEvaluation(Contains(c1, Literal(null, StringType)), null, row) + + checkEvaluation(StartsWith(c1, "a"), true, row) + checkEvaluation(StartsWith(c1, "b"), false, row) + checkEvaluation(StartsWith(c2, "a"), null, row) + checkEvaluation(StartsWith(c1, Literal(null, StringType)), null, row) + + checkEvaluation(EndsWith(c1, "c"), true, row) + checkEvaluation(EndsWith(c1, "b"), false, row) + checkEvaluation(EndsWith(c2, "b"), null, row) + checkEvaluation(EndsWith(c1, Literal(null, StringType)), null, row) + } + test("Substring") { val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index ff8d0d06c45e6..d607eed1bea89 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -205,7 +205,10 @@ class ConstantFoldingSuite extends PlanTest { Substring(Literal(null, StringType), 0, 1) as 'c16, Substring("abc", Literal(null, IntegerType), 1) as 'c17, - Substring("abc", 0, Literal(null, IntegerType)) as 'c18 + Substring("abc", 0, Literal(null, IntegerType)) as 'c18, + + Contains(Literal(null, StringType), "abc") as 'c19, + Contains("abc", Literal(null, StringType)) as 'c20 ) val optimized = Optimize(originalQuery.analyze) @@ -237,7 +240,10 @@ class ConstantFoldingSuite extends PlanTest { Literal(null, StringType) as 'c16, Literal(null, StringType) as 'c17, - Literal(null, StringType) as 'c18 + Literal(null, StringType) as 'c18, + + Literal(null, BooleanType) as 'c19, + Literal(null, BooleanType) as 'c20 ).analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.scala new file mode 100644 index 0000000000000..b10577c8001e2 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LikeSimplificationSuite.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.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.rules._ + +/* Implicit conversions */ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ + +class LikeSimplificationSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Like Simplification", Once, + LikeSimplification) :: Nil + } + + val testRelation = LocalRelation('a.string) + + test("simplify Like into StartsWith") { + val originalQuery = + testRelation + .where(('a like "abc%") || ('a like "abc\\%")) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(StartsWith('a, "abc") || ('a like "abc\\%")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify Like into EndsWith") { + val originalQuery = + testRelation + .where('a like "%xyz") + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(EndsWith('a, "xyz")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify Like into Contains") { + val originalQuery = + testRelation + .where(('a like "%mn%") || ('a like "%mn\\%")) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(Contains('a, "mn") || ('a like "%mn\\%")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("simplify Like into EqualTo") { + val originalQuery = + testRelation + .where(('a like "") || ('a like "abc")) + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = testRelation + .where(('a === "") || ('a === "abc")) + .analyze + + comparePlans(optimized, correctAnswer) + } +} From 7f1720813793e155743b58eae5228298e894b90d Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 18 Jul 2014 16:38:11 -0500 Subject: [PATCH 0008/1492] [SPARK-2540] [SQL] Add HiveDecimal & HiveVarchar support in unwrapping data Author: Cheng Hao Closes #1436 from chenghao-intel/unwrapdata and squashes the following commits: 34cc21a [Cheng Hao] update the table scan accodringly since the unwrapData function changed afc39da [Cheng Hao] Polish the code 39d6475 [Cheng Hao] Add HiveDecimal & HiveVarchar support in unwrap data --- .../spark/sql/hive/execution/HiveTableScan.scala | 12 +----------- .../scala/org/apache/spark/sql/hive/hiveUdfs.scala | 4 ++++ 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index ef8bae74530ec..e7016fa16eea9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -96,19 +96,9 @@ case class HiveTableScan( .getOrElse(sys.error(s"Can't find attribute $a")) val fieldObjectInspector = ref.getFieldObjectInspector - val unwrapHiveData = fieldObjectInspector match { - case _: HiveVarcharObjectInspector => - (value: Any) => value.asInstanceOf[HiveVarchar].getValue - case _: HiveDecimalObjectInspector => - (value: Any) => BigDecimal(value.asInstanceOf[HiveDecimal].bigDecimalValue()) - case _ => - identity[Any] _ - } - (row: Any, _: Array[String]) => { val data = objectInspector.getStructFieldData(row, ref) - val hiveData = unwrapData(data, fieldObjectInspector) - if (hiveData != null) unwrapHiveData(hiveData) else null + unwrapData(data, fieldObjectInspector) } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 9b105308ab7cf..fc33c5b460d70 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -280,6 +280,10 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) private[hive] trait HiveInspectors { def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { + case hvoi: HiveVarcharObjectInspector => + if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue + case hdoi: HiveDecimalObjectInspector => + if (data == null) null else BigDecimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) case li: ListObjectInspector => Option(li.getList(data)) From 7b971b91caeebda57f1506ffc4fd266a1b379290 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 18 Jul 2014 14:40:32 -0700 Subject: [PATCH 0009/1492] [SPARK-2571] Correctly report shuffle read metrics. Currently, shuffle read metrics are incorrectly reported when stages have multiple shuffle dependencies (they are set to be the metrics from just one of the shuffle dependencies, rather than the accumulated metrics from all of the shuffle dependencies). This fixes that problem, and should probably be back-ported to the 0.9 branch. Thanks ryanra for discovering this problem! cc rxin andrewor14 Author: Kay Ousterhout Closes #1476 from kayousterhout/join_bug and squashes the following commits: 0203a16 [Kay Ousterhout] Fix broken unit tests. f463c2e [Kay Ousterhout] [SPARK-2571] Correctly report shuffle read metrics. --- .../apache/spark/executor/TaskMetrics.scala | 20 ++++++++++++++++++- .../hash/BlockStoreShuffleFetcher.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 5 +++-- .../spark/scheduler/SparkListenerSuite.scala | 4 ++-- .../ui/jobs/JobProgressListenerSuite.scala | 6 +----- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- 6 files changed, 27 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index ac73288442a74..5d59e00636ee6 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -75,7 +75,9 @@ class TaskMetrics extends Serializable { /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here */ - var shuffleReadMetrics: Option[ShuffleReadMetrics] = None + private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None + + def shuffleReadMetrics = _shuffleReadMetrics /** * If this task writes to shuffle output, metrics on the written shuffle data will be collected @@ -87,6 +89,22 @@ class TaskMetrics extends Serializable { * Storage statuses of any blocks that have been updated as a result of this task. */ var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None + + /** Adds the given ShuffleReadMetrics to any existing shuffle metrics for this task. */ + def updateShuffleReadMetrics(newMetrics: ShuffleReadMetrics) = synchronized { + _shuffleReadMetrics match { + case Some(existingMetrics) => + existingMetrics.shuffleFinishTime = math.max( + existingMetrics.shuffleFinishTime, newMetrics.shuffleFinishTime) + existingMetrics.fetchWaitTime += newMetrics.fetchWaitTime + existingMetrics.localBlocksFetched += newMetrics.localBlocksFetched + existingMetrics.remoteBlocksFetched += newMetrics.remoteBlocksFetched + existingMetrics.totalBlocksFetched += newMetrics.totalBlocksFetched + existingMetrics.remoteBytesRead += newMetrics.remoteBytesRead + case None => + _shuffleReadMetrics = Some(newMetrics) + } + } } private[spark] object TaskMetrics { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index a932455776e34..3795994cd920f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -84,7 +84,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks - context.taskMetrics.shuffleReadMetrics = Some(shuffleMetrics) + context.taskMetrics.updateShuffleReadMetrics(shuffleMetrics) }) new InterruptibleIterator[T](context, completionIter) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 47eb44b530379..2ff8b25a56d10 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -527,8 +527,9 @@ private[spark] object JsonProtocol { metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] - metrics.shuffleReadMetrics = - Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) + Utils.jsonOption(json \ "Shuffle Read Metrics").map { shuffleReadMetrics => + metrics.updateShuffleReadMetrics(shuffleReadMetricsFromJson(shuffleReadMetrics)) + } metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) metrics.inputMetrics = diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 71f48e295ecca..3b0b8e2f68c97 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -258,8 +258,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get - sm.totalBlocksFetched should be > (0) - sm.localBlocksFetched should be > (0) + sm.totalBlocksFetched should be (128) + sm.localBlocksFetched should be (128) sm.remoteBlocksFetched should be (0) sm.remoteBytesRead should be (0l) } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index a8556624804bb..b52f81877d557 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -63,7 +63,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc // finish this task, should get updated shuffleRead shuffleReadMetrics.remoteBytesRead = 1000 - taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) + taskMetrics.updateShuffleReadMetrics(shuffleReadMetrics) var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 var task = new ShuffleMapTask(0, null, null, 0, null) @@ -81,8 +81,6 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(listener.stageIdToData.size === 1) // finish this task, should get updated duration - shuffleReadMetrics.remoteBytesRead = 1000 - taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1235L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) @@ -91,8 +89,6 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc .shuffleRead === 2000) // finish this task, should get updated duration - shuffleReadMetrics.remoteBytesRead = 1000 - taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1236L, 0, 2, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0, null, null, 0, null) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 058d31453081a..11f70a6090d24 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -518,7 +518,7 @@ class JsonProtocolSuite extends FunSuite { sr.localBlocksFetched = e sr.fetchWaitTime = a + d sr.remoteBlocksFetched = f - t.shuffleReadMetrics = Some(sr) + t.updateShuffleReadMetrics(sr) } sw.shuffleBytesWritten = a + b + c sw.shuffleWriteTime = b + c + d From a243364b225da9a91813234027eafedffc495ecc Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Fri, 18 Jul 2014 17:25:32 -0700 Subject: [PATCH 0010/1492] [SPARK-2359][MLlib] Correlations Implementation for Pearson and Spearman's correlation. Author: Doris Xin Closes #1367 from dorx/correlation and squashes the following commits: c0dd7dc [Doris Xin] here we go 32d83a3 [Doris Xin] Reviewer comments 4db0da1 [Doris Xin] added private[stat] to Spearman b716f70 [Doris Xin] minor fixes 6e1b42a [Doris Xin] More comments addressed. Still some open questions 8104f44 [Doris Xin] addressed comments. some open questions still 39387c2 [Doris Xin] added missing header bd3cf19 [Doris Xin] Merge branch 'master' into correlation 6341884 [Doris Xin] race condition bug squished bd2bacf [Doris Xin] Race condition bug b775ff9 [Doris Xin] old wrong impl 534ebf2 [Doris Xin] Merge branch 'master' into correlation 818fa31 [Doris Xin] wip units 9d808ee [Doris Xin] wip units b843a13 [Doris Xin] revert change in stat counter 28561b6 [Doris Xin] wip bb2e977 [Doris Xin] minor fix 8e02c63 [Doris Xin] Merge branch 'master' into correlation 2a40aa1 [Doris Xin] initial, untested implementation of Pearson dfc4854 [Doris Xin] WIP --- .../apache/spark/mllib/stat/Statistics.scala | 78 +++++++++++ .../mllib/stat/correlation/Correlation.scala | 91 +++++++++++++ .../stat/correlation/PearsonCorrelation.scala | 107 +++++++++++++++ .../correlation/SpearmanCorrelation.scala | 127 ++++++++++++++++++ .../spark/mllib/stat/CorrelationSuite.scala | 116 ++++++++++++++++ 5 files changed, 519 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala new file mode 100644 index 0000000000000..68f3867ba6c11 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -0,0 +1,78 @@ +/* + * 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.stat + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.rdd.RDD + +/** + * API for statistical functions in MLlib + */ +@Experimental +object Statistics { + + /** + * Compute the Pearson correlation matrix for the input RDD of Vectors. + * Returns NaN if either vector has 0 variance. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @return Pearson correlation matrix comparing columns in X. + */ + def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) + + /** + * Compute the correlation matrix for the input RDD of Vectors using the specified method. + * Methods currently supported: `pearson` (default), `spearman` + * + * Note that for Spearman, a rank correlation, we need to create an RDD[Double] for each column + * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], + * which is fairly costly. Cache the input RDD before calling corr with `method = "spearman"` to + * avoid recomputing the common lineage. + * + * @param X an RDD[Vector] for which the correlation matrix is to be computed. + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + * @return Correlation matrix comparing columns in X. + */ + def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) + + /** + * Compute the Pearson correlation for the input RDDs. + * Columns with 0 covariance produce NaN entries in the correlation matrix. + * + * @param x RDD[Double] of the same cardinality as y + * @param y RDD[Double] of the same cardinality as x + * @return A Double containing the Pearson correlation between the two input RDD[Double]s + */ + def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) + + /** + * Compute the correlation for the input RDDs using the specified method. + * Methods currently supported: pearson (default), spearman + * + * @param x RDD[Double] of the same cardinality as y + * @param y RDD[Double] of the same cardinality as x + * @param method String specifying the method to use for computing correlation. + * Supported: `pearson` (default), `spearman` + *@return A Double containing the correlation between the two input RDD[Double]s using the + * specified method. + */ + def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala new file mode 100644 index 0000000000000..f23393d3da257 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.correlation + +import org.apache.spark.mllib.linalg.{DenseVector, Matrix, Vector} +import org.apache.spark.rdd.RDD + +/** + * Trait for correlation algorithms. + */ +private[stat] trait Correlation { + + /** + * Compute correlation for two datasets. + */ + def computeCorrelation(x: RDD[Double], y: RDD[Double]): Double + + /** + * Compute the correlation matrix S, for the input matrix, where S(i, j) is the correlation + * between column i and j. S(i, j) can be NaN if the correlation is undefined for column i and j. + */ + def computeCorrelationMatrix(X: RDD[Vector]): Matrix + + /** + * Combine the two input RDD[Double]s into an RDD[Vector] and compute the correlation using the + * correlation implementation for RDD[Vector]. Can be NaN if correlation is undefined for the + * input vectors. + */ + def computeCorrelationWithMatrixImpl(x: RDD[Double], y: RDD[Double]): Double = { + val mat: RDD[Vector] = x.zip(y).map { case (xi, yi) => new DenseVector(Array(xi, yi)) } + computeCorrelationMatrix(mat)(0, 1) + } + +} + +/** + * Delegates computation to the specific correlation object based on the input method name + * + * Currently supported correlations: pearson, spearman. + * After new correlation algorithms are added, please update the documentation here and in + * Statistics.scala for the correlation APIs. + * + * Maintains the default correlation type, pearson + */ +private[stat] object Correlations { + + // Note: after new types of correlations are implemented, please update this map + val nameToObjectMap = Map(("pearson", PearsonCorrelation), ("spearman", SpearmanCorrelation)) + val defaultCorrName: String = "pearson" + val defaultCorr: Correlation = nameToObjectMap(defaultCorrName) + + def corr(x: RDD[Double], y: RDD[Double], method: String = defaultCorrName): Double = { + val correlation = getCorrelationFromName(method) + correlation.computeCorrelation(x, y) + } + + def corrMatrix(X: RDD[Vector], method: String = defaultCorrName): Matrix = { + val correlation = getCorrelationFromName(method) + correlation.computeCorrelationMatrix(X) + } + + /** + * Match input correlation name with a known name via simple string matching + * + * private to stat for ease of unit testing + */ + private[stat] def getCorrelationFromName(method: String): Correlation = { + try { + nameToObjectMap(method) + } catch { + case nse: NoSuchElementException => + throw new IllegalArgumentException("Unrecognized method name. Supported correlations: " + + nameToObjectMap.keys.mkString(", ")) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala new file mode 100644 index 0000000000000..23b291eee070b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/PearsonCorrelation.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat.correlation + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.rdd.RDD + +/** + * Compute Pearson correlation for two RDDs of the type RDD[Double] or the correlation matrix + * for an RDD of the type RDD[Vector]. + * + * Definition of Pearson correlation can be found at + * http://en.wikipedia.org/wiki/Pearson_product-moment_correlation_coefficient + */ +private[stat] object PearsonCorrelation extends Correlation with Logging { + + /** + * Compute the Pearson correlation for two datasets. NaN if either vector has 0 variance. + */ + override def computeCorrelation(x: RDD[Double], y: RDD[Double]): Double = { + computeCorrelationWithMatrixImpl(x, y) + } + + /** + * Compute the Pearson correlation matrix S, for the input matrix, where S(i, j) is the + * correlation between column i and j. 0 covariance results in a correlation value of Double.NaN. + */ + override def computeCorrelationMatrix(X: RDD[Vector]): Matrix = { + val rowMatrix = new RowMatrix(X) + val cov = rowMatrix.computeCovariance() + computeCorrelationMatrixFromCovariance(cov) + } + + /** + * Compute the Pearson correlation matrix from the covariance matrix. + * 0 covariance results in a correlation value of Double.NaN. + */ + def computeCorrelationMatrixFromCovariance(covarianceMatrix: Matrix): Matrix = { + val cov = covarianceMatrix.toBreeze.asInstanceOf[BDM[Double]] + val n = cov.cols + + // Compute the standard deviation on the diagonals first + var i = 0 + while (i < n) { + // TODO remove once covariance numerical issue resolved. + cov(i, i) = if (closeToZero(cov(i, i))) 0.0 else math.sqrt(cov(i, i)) + i +=1 + } + + // Loop through columns since cov is column major + var j = 0 + var sigma = 0.0 + var containNaN = false + while (j < n) { + sigma = cov(j, j) + i = 0 + while (i < j) { + val corr = if (sigma == 0.0 || cov(i, i) == 0.0) { + containNaN = true + Double.NaN + } else { + cov(i, j) / (sigma * cov(i, i)) + } + cov(i, j) = corr + cov(j, i) = corr + i += 1 + } + j += 1 + } + + // put 1.0 on the diagonals + i = 0 + while (i < n) { + cov(i, i) = 1.0 + i +=1 + } + + if (containNaN) { + logWarning("Pearson correlation matrix contains NaN values.") + } + + Matrices.fromBreeze(cov) + } + + private def closeToZero(value: Double, threshhold: Double = 1e-12): Boolean = { + math.abs(value) <= threshhold + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala new file mode 100644 index 0000000000000..88de2c82479b7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -0,0 +1,127 @@ +/* + * 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.stat.correlation + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, HashPartitioner} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.{DenseVector, Matrix, Vector} +import org.apache.spark.rdd.{CoGroupedRDD, RDD} + +/** + * Compute Spearman's correlation for two RDDs of the type RDD[Double] or the correlation matrix + * for an RDD of the type RDD[Vector]. + * + * Definition of Spearman's correlation can be found at + * http://en.wikipedia.org/wiki/Spearman's_rank_correlation_coefficient + */ +private[stat] object SpearmanCorrelation extends Correlation with Logging { + + /** + * Compute Spearman's correlation for two datasets. + */ + override def computeCorrelation(x: RDD[Double], y: RDD[Double]): Double = { + computeCorrelationWithMatrixImpl(x, y) + } + + /** + * Compute Spearman's correlation matrix S, for the input matrix, where S(i, j) is the + * correlation between column i and j. + * + * Input RDD[Vector] should be cached or checkpointed if possible since it would be split into + * numCol RDD[Double]s, each of which sorted, and the joined back into a single RDD[Vector]. + */ + override def computeCorrelationMatrix(X: RDD[Vector]): Matrix = { + val indexed = X.zipWithUniqueId() + + val numCols = X.first.size + if (numCols > 50) { + logWarning("Computing the Spearman correlation matrix can be slow for large RDDs with more" + + " than 50 columns.") + } + val ranks = new Array[RDD[(Long, Double)]](numCols) + + // Note: we use a for loop here instead of a while loop with a single index variable + // to avoid race condition caused by closure serialization + for (k <- 0 until numCols) { + val column = indexed.map { case (vector, index) => (vector(k), index) } + ranks(k) = getRanks(column) + } + + val ranksMat: RDD[Vector] = makeRankMatrix(ranks, X) + PearsonCorrelation.computeCorrelationMatrix(ranksMat) + } + + /** + * Compute the ranks for elements in the input RDD, using the average method for ties. + * + * With the average method, elements with the same value receive the same rank that's computed + * by taking the average of their positions in the sorted list. + * e.g. ranks([2, 1, 0, 2]) = [2.5, 1.0, 0.0, 2.5] + * Note that positions here are 0-indexed, instead of the 1-indexed as in the definition for + * ranks in the standard definition for Spearman's correlation. This does not affect the final + * results and is slightly more performant. + * + * @param indexed RDD[(Double, Long)] containing pairs of the format (originalValue, uniqueId) + * @return RDD[(Long, Double)] containing pairs of the format (uniqueId, rank), where uniqueId is + * copied from the input RDD. + */ + private def getRanks(indexed: RDD[(Double, Long)]): RDD[(Long, Double)] = { + // Get elements' positions in the sorted list for computing average rank for duplicate values + val sorted = indexed.sortByKey().zipWithIndex() + + val ranks: RDD[(Long, Double)] = sorted.mapPartitions { iter => + // add an extra element to signify the end of the list so that flatMap can flush the last + // batch of duplicates + val padded = iter ++ + Iterator[((Double, Long), Long)](((Double.NaN, -1L), -1L)) + var lastVal = 0.0 + var firstRank = 0.0 + val idBuffer = new ArrayBuffer[Long]() + padded.flatMap { case ((v, id), rank) => + if (v == lastVal && id != Long.MinValue) { + idBuffer += id + Iterator.empty + } else { + val entries = if (idBuffer.size == 0) { + // edge case for the first value matching the initial value of lastVal + Iterator.empty + } else if (idBuffer.size == 1) { + Iterator((idBuffer(0), firstRank)) + } else { + val averageRank = firstRank + (idBuffer.size - 1.0) / 2.0 + idBuffer.map(id => (id, averageRank)) + } + lastVal = v + firstRank = rank + idBuffer.clear() + idBuffer += id + entries + } + } + } + ranks + } + + private def makeRankMatrix(ranks: Array[RDD[(Long, Double)]], input: RDD[Vector]): RDD[Vector] = { + val partitioner = new HashPartitioner(input.partitions.size) + val cogrouped = new CoGroupedRDD[Long](ranks, partitioner) + cogrouped.map { case (_, values: Seq[Seq[Double]]) => new DenseVector(values.flatten.toArray) } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala new file mode 100644 index 0000000000000..bce4251426df7 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.stat + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM, Matrix => BM} + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation, + SpearmanCorrelation} +import org.apache.spark.mllib.util.LocalSparkContext + +class CorrelationSuite extends FunSuite with LocalSparkContext { + + // test input data + val xData = Array(1.0, 0.0, -2.0) + val yData = Array(4.0, 5.0, 3.0) + val data = Seq( + Vectors.dense(1.0, 0.0, 0.0, -2.0), + Vectors.dense(4.0, 5.0, 0.0, 3.0), + Vectors.dense(6.0, 7.0, 0.0, 8.0), + Vectors.dense(9.0, 0.0, 0.0, 1.0) + ) + + test("corr(x, y) default, pearson") { + val x = sc.parallelize(xData) + val y = sc.parallelize(yData) + val expected = 0.6546537 + val default = Statistics.corr(x, y) + val p1 = Statistics.corr(x, y, "pearson") + assert(approxEqual(expected, default)) + assert(approxEqual(expected, p1)) + } + + test("corr(x, y) spearman") { + val x = sc.parallelize(xData) + val y = sc.parallelize(yData) + val expected = 0.5 + val s1 = Statistics.corr(x, y, "spearman") + assert(approxEqual(expected, s1)) + } + + test("corr(X) default, pearson") { + val X = sc.parallelize(data) + val defaultMat = Statistics.corr(X) + val pearsonMat = Statistics.corr(X, "pearson") + val expected = BDM( + (1.00000000, 0.05564149, Double.NaN, 0.4004714), + (0.05564149, 1.00000000, Double.NaN, 0.9135959), + (Double.NaN, Double.NaN, 1.00000000, Double.NaN), + (0.40047142, 0.91359586, Double.NaN,1.0000000)) + assert(matrixApproxEqual(defaultMat.toBreeze, expected)) + assert(matrixApproxEqual(pearsonMat.toBreeze, expected)) + } + + test("corr(X) spearman") { + val X = sc.parallelize(data) + val spearmanMat = Statistics.corr(X, "spearman") + val expected = BDM( + (1.0000000, 0.1054093, Double.NaN, 0.4000000), + (0.1054093, 1.0000000, Double.NaN, 0.9486833), + (Double.NaN, Double.NaN, 1.00000000, Double.NaN), + (0.4000000, 0.9486833, Double.NaN, 1.0000000)) + assert(matrixApproxEqual(spearmanMat.toBreeze, expected)) + } + + test("method identification") { + val pearson = PearsonCorrelation + val spearman = SpearmanCorrelation + + assert(Correlations.getCorrelationFromName("pearson") === pearson) + assert(Correlations.getCorrelationFromName("spearman") === spearman) + + // Should throw IllegalArgumentException + try { + Correlations.getCorrelationFromName("kendall") + assert(false) + } catch { + case ie: IllegalArgumentException => + } + } + + def approxEqual(v1: Double, v2: Double, threshold: Double = 1e-6): Boolean = { + if (v1.isNaN) { + v2.isNaN + } else { + math.abs(v1 - v2) <= threshold + } + } + + def matrixApproxEqual(A: BM[Double], B: BM[Double], threshold: Double = 1e-6): Boolean = { + for (i <- 0 until A.rows; j <- 0 until A.cols) { + if (!approxEqual(A(i, j), B(i, j), threshold)) { + println("i, j = " + i + ", " + j + " actual: " + A(i, j) + " expected:" + B(i, j)) + return false + } + } + true + } +} From 7b8cd175254d42c8e82f0aa8eb4b7f3508d8fde2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 18 Jul 2014 23:52:47 -0700 Subject: [PATCH 0011/1492] [SPARK-2521] Broadcast RDD object (instead of sending it along with every task). Currently (as of Spark 1.0.1), Spark sends RDD object (which contains closures) using Akka along with the task itself to the executors. This is inefficient because all tasks in the same stage use the same RDD object, but we have to send RDD object multiple times to the executors. This is especially bad when a closure references some variable that is very large. The current design led to users having to explicitly broadcast large variables. The patch uses broadcast to send RDD objects and the closures to executors, and use Akka to only send a reference to the broadcast RDD/closure along with the partition specific information for the task. For those of you who know more about the internals, Spark already relies on broadcast to send the Hadoop JobConf every time it uses the Hadoop input, because the JobConf is large. The user-facing impact of the change include: 1. Users won't need to decide what to broadcast anymore, unless they would want to use a large object multiple times in different operations 2. Task size will get smaller, resulting in faster scheduling and higher task dispatch throughput. In addition, the change will simplify some internals of Spark, eliminating the need to maintain task caches and the complex logic to broadcast JobConf (which also led to a deadlock recently). A simple way to test this: ```scala val a = new Array[Byte](1000*1000); scala.util.Random.nextBytes(a); sc.parallelize(1 to 1000, 1000).map { x => a; x }.groupBy { x => a; x }.count ``` Numbers on 3 r3.8xlarge instances on EC2 ``` master branch: 5.648436068 s, 4.715361895 s, 5.360161877 s with this change: 3.416348793 s, 1.477846558 s, 1.553432156 s ``` Author: Reynold Xin Closes #1452 from rxin/broadcast-task and squashes the following commits: 762e0be [Reynold Xin] Warn large broadcasts. ade6eac [Reynold Xin] Log broadcast size. c3b6f11 [Reynold Xin] Added a unit test for clean up. 754085f [Reynold Xin] Explain why broadcasting serialized copy of the task. 04b17f0 [Reynold Xin] [SPARK-2521] Broadcast RDD object once per TaskSet (instead of sending it for every task). --- .../scala/org/apache/spark/Dependency.scala | 28 ++-- .../scala/org/apache/spark/SparkContext.scala | 2 - .../main/scala/org/apache/spark/rdd/RDD.scala | 30 +++- .../apache/spark/rdd/RDDCheckpointData.scala | 9 +- .../apache/spark/scheduler/DAGScheduler.scala | 4 - .../apache/spark/scheduler/ResultTask.scala | 128 +++++------------- .../spark/scheduler/ShuffleMapTask.scala | 125 ++++------------- .../apache/spark/ContextCleanerSuite.scala | 62 +++++---- 8 files changed, 137 insertions(+), 251 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 09a60571238ea..3935c8772252e 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -27,7 +27,9 @@ import org.apache.spark.shuffle.ShuffleHandle * Base class for dependencies. */ @DeveloperApi -abstract class Dependency[T](val rdd: RDD[T]) extends Serializable +abstract class Dependency[T] extends Serializable { + def rdd: RDD[T] +} /** @@ -36,20 +38,24 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ @DeveloperApi -abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { +abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { /** * Get the parent partitions for a child partition. * @param partitionId a partition of the child RDD * @return the partitions of the parent RDD that the child partition depends upon */ def getParents(partitionId: Int): Seq[Int] + + override def rdd: RDD[T] = _rdd } /** * :: DeveloperApi :: - * Represents a dependency on the output of a shuffle stage. - * @param rdd the parent RDD + * Represents a dependency on the output of a shuffle stage. Note that in the case of shuffle, + * the RDD is transient since we don't need it on the executor side. + * + * @param _rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will @@ -57,20 +63,22 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { */ @DeveloperApi class ShuffleDependency[K, V, C]( - @transient rdd: RDD[_ <: Product2[K, V]], + @transient _rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, val mapSideCombine: Boolean = false) - extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { + extends Dependency[Product2[K, V]] { + + override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]] - val shuffleId: Int = rdd.context.newShuffleId() + val shuffleId: Int = _rdd.context.newShuffleId() - val shuffleHandle: ShuffleHandle = rdd.context.env.shuffleManager.registerShuffle( - shuffleId, rdd.partitions.size, this) + val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( + shuffleId, _rdd.partitions.size, this) - rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) + _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8052499ab7526..48a09657fde26 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -997,8 +997,6 @@ class SparkContext(config: SparkConf) extends Logging { // TODO: Cache.stop()? env.stop() SparkEnv.set(null) - ShuffleMapTask.clearCache() - ResultTask.clearCache() listenerBus.stop() eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") 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 88a918aebf763..2ee9a8f1a8e0d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,12 +35,13 @@ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.broadcast.Broadcast import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, CallSite, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} @@ -1195,21 +1196,36 @@ abstract class RDD[T: ClassTag]( /** * Return whether this RDD has been checkpointed or not */ - def isCheckpointed: Boolean = { - checkpointData.map(_.isCheckpointed).getOrElse(false) - } + def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) /** * Gets the name of the file to which this RDD was checkpointed */ - def getCheckpointFile: Option[String] = { - checkpointData.flatMap(_.getCheckpointFile) - } + def getCheckpointFile: Option[String] = checkpointData.flatMap(_.getCheckpointFile) // ======================================================================= // Other internal methods and fields // ======================================================================= + /** + * Broadcasted copy of this RDD, used to dispatch tasks to executors. Note that we broadcast + * the serialized copy of the RDD and for each task we will deserialize it, which means each + * task gets a different copy of the RDD. This provides stronger isolation between tasks that + * might modify state of objects referenced in their closures. This is necessary in Hadoop + * where the JobConf/Configuration object is not thread-safe. + */ + @transient private[spark] lazy val broadcasted: Broadcast[Array[Byte]] = { + val ser = SparkEnv.get.closureSerializer.newInstance() + val bytes = ser.serialize(this).array() + val size = Utils.bytesToString(bytes.length) + if (bytes.length > (1L << 20)) { + logWarning(s"Broadcasting RDD $id ($size), which contains large objects") + } else { + logDebug(s"Broadcasting RDD $id ($size)") + } + sc.broadcast(bytes) + } + private var storageLevel: StorageLevel = StorageLevel.NONE /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index c3b2a33fb54d0..f67e5f1857979 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -106,7 +106,6 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) cpRDD = Some(newRDD) rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions cpState = Checkpointed - RDDCheckpointData.clearTaskCaches() } logInfo("Done checkpointing RDD " + rdd.id + " to " + path + ", new parent is RDD " + newRDD.id) } @@ -131,9 +130,5 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -private[spark] object RDDCheckpointData { - def clearTaskCaches() { - ShuffleMapTask.clearCache() - ResultTask.clearCache() - } -} +// Used for synchronization +private[spark] object RDDCheckpointData diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ede3c7d9f01ae..88cb5feaaff2a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -376,9 +376,6 @@ class DAGScheduler( stageIdToStage -= stageId stageIdToJobIds -= stageId - ShuffleMapTask.removeStage(stageId) - ResultTask.removeStage(stageId) - logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } @@ -723,7 +720,6 @@ class DAGScheduler( } } - /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index bbf9f7388b074..62beb0d02a9c3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,134 +17,68 @@ package org.apache.spark.scheduler -import scala.language.existentials +import java.nio.ByteBuffer import java.io._ -import java.util.zip.{GZIPInputStream, GZIPOutputStream} - -import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.rdd.{RDD, RDDCheckpointData} - -private[spark] object ResultTask { - - // A simple map between the stage id to the serialized byte array of a task. - // Served as a cache for task serialization because serialization can be - // expensive on the master node if it needs to launch thousands of tasks. - private val serializedInfoCache = new HashMap[Int, Array[Byte]] - - def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = - { - synchronized { - val old = serializedInfoCache.get(stageId).orNull - if (old != null) { - old - } else { - val out = new ByteArrayOutputStream - val ser = SparkEnv.get.closureSerializer.newInstance() - val objOut = ser.serializeStream(new GZIPOutputStream(out)) - objOut.writeObject(rdd) - objOut.writeObject(func) - objOut.close() - val bytes = out.toByteArray - serializedInfoCache.put(stageId, bytes) - bytes - } - } - } - - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = - { - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val ser = SparkEnv.get.closureSerializer.newInstance() - val objIn = ser.deserializeStream(in) - val rdd = objIn.readObject().asInstanceOf[RDD[_]] - val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _] - (rdd, func) - } - - def removeStage(stageId: Int) { - serializedInfoCache.remove(stageId) - } - - def clearCache() { - synchronized { - serializedInfoCache.clear() - } - } -} - +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD /** * A task that sends back the output to the driver application. * - * See [[org.apache.spark.scheduler.Task]] for more information. + * See [[Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param rdd input to func + * @param rddBinary broadcast version of of the serialized RDD * @param func a function to apply on a partition of the RDD - * @param _partitionId index of the number in the RDD + * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). */ private[spark] class ResultTask[T, U]( stageId: Int, - var rdd: RDD[T], - var func: (TaskContext, Iterator[T]) => U, - _partitionId: Int, + val rddBinary: Broadcast[Array[Byte]], + val func: (TaskContext, Iterator[T]) => U, + val partition: Partition, @transient locs: Seq[TaskLocation], - var outputId: Int) - extends Task[U](stageId, _partitionId) with Externalizable { - - def this() = this(0, null, null, 0, null, 0) - - var split = if (rdd == null) null else rdd.partitions(partitionId) + val outputId: Int) + extends Task[U](stageId, partition.index) with Serializable { + + // TODO: Should we also broadcast func? For that we would need a place to + // keep a reference to it (perhaps in DAGScheduler's job object). + + def this( + stageId: Int, + rdd: RDD[T], + func: (TaskContext, Iterator[T]) => U, + partitionId: Int, + locs: Seq[TaskLocation], + outputId: Int) = { + this(stageId, rdd.broadcasted, func, rdd.partitions(partitionId), locs, outputId) + } - @transient private val preferredLocs: Seq[TaskLocation] = { + @transient private[this] val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } override def runTask(context: TaskContext): U = { + // Deserialize the RDD using the broadcast variable. + val ser = SparkEnv.get.closureSerializer.newInstance() + val rdd = ser.deserialize[RDD[T]](ByteBuffer.wrap(rddBinary.value), + Thread.currentThread.getContextClassLoader) metrics = Some(context.taskMetrics) try { - func(context, rdd.iterator(split, context)) + func(context, rdd.iterator(partition, context)) } finally { context.executeOnCompleteCallbacks() } } + // This is only callable on the driver side. override def preferredLocations: Seq[TaskLocation] = preferredLocs override def toString = "ResultTask(" + stageId + ", " + partitionId + ")" - - override def writeExternal(out: ObjectOutput) { - RDDCheckpointData.synchronized { - split = rdd.partitions(partitionId) - out.writeInt(stageId) - val bytes = ResultTask.serializeInfo( - stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _]) - out.writeInt(bytes.length) - out.write(bytes) - out.writeInt(partitionId) - out.writeInt(outputId) - out.writeLong(epoch) - out.writeObject(split) - } - } - - override def readExternal(in: ObjectInput) { - val stageId = in.readInt() - val numBytes = in.readInt() - val bytes = new Array[Byte](numBytes) - in.readFully(bytes) - val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes) - rdd = rdd_.asInstanceOf[RDD[T]] - func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] - partitionId = in.readInt() - outputId = in.readInt() - epoch = in.readLong() - split = in.readObject().asInstanceOf[Partition] - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index fdaf1de83f051..033c6e52861e0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,71 +17,13 @@ package org.apache.spark.scheduler -import scala.language.existentials - -import java.io._ -import java.util.zip.{GZIPInputStream, GZIPOutputStream} - -import scala.collection.mutable.HashMap +import java.nio.ByteBuffer import org.apache.spark._ -import org.apache.spark.rdd.{RDD, RDDCheckpointData} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.ShuffleWriter -private[spark] object ShuffleMapTask { - - // A simple map between the stage id to the serialized byte array of a task. - // Served as a cache for task serialization because serialization can be - // expensive on the master node if it needs to launch thousands of tasks. - private val serializedInfoCache = new HashMap[Int, Array[Byte]] - - def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_, _, _]): Array[Byte] = { - synchronized { - val old = serializedInfoCache.get(stageId).orNull - if (old != null) { - return old - } else { - val out = new ByteArrayOutputStream - val ser = SparkEnv.get.closureSerializer.newInstance() - val objOut = ser.serializeStream(new GZIPOutputStream(out)) - objOut.writeObject(rdd) - objOut.writeObject(dep) - objOut.close() - val bytes = out.toByteArray - serializedInfoCache.put(stageId, bytes) - bytes - } - } - } - - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_, _, _]) = { - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val ser = SparkEnv.get.closureSerializer.newInstance() - val objIn = ser.deserializeStream(in) - val rdd = objIn.readObject().asInstanceOf[RDD[_]] - val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_, _, _]] - (rdd, dep) - } - - // Since both the JarSet and FileSet have the same format this is used for both. - def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = { - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val objIn = new ObjectInputStream(in) - val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap - HashMap(set.toSeq: _*) - } - - def removeStage(stageId: Int) { - serializedInfoCache.remove(stageId) - } - - def clearCache() { - synchronized { - serializedInfoCache.clear() - } - } -} - /** * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner * specified in the ShuffleDependency). @@ -89,62 +31,47 @@ private[spark] object ShuffleMapTask { * See [[org.apache.spark.scheduler.Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param rdd the final RDD in this stage + * @param rddBinary broadcast version of of the serialized RDD * @param dep the ShuffleDependency - * @param _partitionId index of the number in the RDD + * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling */ private[spark] class ShuffleMapTask( stageId: Int, - var rdd: RDD[_], + var rddBinary: Broadcast[Array[Byte]], var dep: ShuffleDependency[_, _, _], - _partitionId: Int, + partition: Partition, @transient private var locs: Seq[TaskLocation]) - extends Task[MapStatus](stageId, _partitionId) - with Externalizable - with Logging { - - protected def this() = this(0, null, null, 0, null) + extends Task[MapStatus](stageId, partition.index) with Logging { + + // TODO: Should we also broadcast the ShuffleDependency? For that we would need a place to + // keep a reference to it (perhaps in Stage). + + def this( + stageId: Int, + rdd: RDD[_], + dep: ShuffleDependency[_, _, _], + partitionId: Int, + locs: Seq[TaskLocation]) = { + this(stageId, rdd.broadcasted, dep, rdd.partitions(partitionId), locs) + } @transient private val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } - var split = if (rdd == null) null else rdd.partitions(partitionId) - - override def writeExternal(out: ObjectOutput) { - RDDCheckpointData.synchronized { - split = rdd.partitions(partitionId) - out.writeInt(stageId) - val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep) - out.writeInt(bytes.length) - out.write(bytes) - out.writeInt(partitionId) - out.writeLong(epoch) - out.writeObject(split) - } - } - - override def readExternal(in: ObjectInput) { - val stageId = in.readInt() - val numBytes = in.readInt() - val bytes = new Array[Byte](numBytes) - in.readFully(bytes) - val (rdd_, dep_) = ShuffleMapTask.deserializeInfo(stageId, bytes) - rdd = rdd_ - dep = dep_ - partitionId = in.readInt() - epoch = in.readLong() - split = in.readObject().asInstanceOf[Partition] - } - override def runTask(context: TaskContext): MapStatus = { + // Deserialize the RDD using the broadcast variable. + val ser = SparkEnv.get.closureSerializer.newInstance() + val rdd = ser.deserialize[RDD[_]](ByteBuffer.wrap(rddBinary.value), + Thread.currentThread.getContextClassLoader) + metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null try { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) - writer.write(rdd.iterator(split, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) return writer.stop(success = true).get } catch { case e: Exception => diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 13b415cccb647..871f831531bee 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -52,9 +52,8 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } } - test("cleanup RDD") { - val rdd = newRDD.persist() + val rdd = newRDD().persist() val collected = rdd.collect().toList val tester = new CleanerTester(sc, rddIds = Seq(rdd.id)) @@ -67,7 +66,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup shuffle") { - val (rdd, shuffleDeps) = newRDDWithShuffleDependencies + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies() val collected = rdd.collect().toList val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) @@ -80,7 +79,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup broadcast") { - val broadcast = newBroadcast + val broadcast = newBroadcast() val tester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) // Explicit cleanup @@ -89,7 +88,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup RDD") { - var rdd = newRDD.persist() + var rdd = newRDD().persist() rdd.count() // Test that GC does not cause RDD cleanup due to a strong reference @@ -107,7 +106,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup shuffle") { - var rdd = newShuffleRDD + var rdd = newShuffleRDD() rdd.count() // Test that GC does not cause shuffle cleanup due to a strong reference @@ -125,7 +124,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup broadcast") { - var broadcast = newBroadcast + var broadcast = newBroadcast() // Test that GC does not cause broadcast cleanup due to a strong reference val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) @@ -141,11 +140,23 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo postGCTester.assertCleanup() } + test("automatically cleanup broadcast data for task dispatching") { + var rdd = newRDDWithShuffleDependencies()._1 + rdd.count() // This triggers an action that broadcasts the RDDs. + + // Test that GC causes broadcast task data cleanup after dereferencing the RDD. + val postGCTester = new CleanerTester(sc, + broadcastIds = Seq(rdd.broadcasted.id, rdd.firstParent.broadcasted.id)) + rdd = null + runGC() + postGCTester.assertCleanup() + } + test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId val broadcastIds = 0L until numBroadcasts @@ -175,8 +186,8 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo val numRdds = 10 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId val broadcastIds = 0L until numBroadcasts @@ -197,17 +208,18 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo //------ Helper functions ------ - def newRDD = sc.makeRDD(1 to 10) - def newPairRDD = newRDD.map(_ -> 1) - def newShuffleRDD = newPairRDD.reduceByKey(_ + _) - def newBroadcast = sc.broadcast(1 to 100) - def newRDDWithShuffleDependencies: (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { + private def newRDD() = sc.makeRDD(1 to 10) + private def newPairRDD() = newRDD().map(_ -> 1) + private def newShuffleRDD() = newPairRDD().reduceByKey(_ + _) + private def newBroadcast() = sc.broadcast(1 to 100) + + private def newRDDWithShuffleDependencies(): (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { rdd.dependencies ++ rdd.dependencies.flatMap { dep => getAllDependencies(dep.rdd) } } - val rdd = newShuffleRDD + val rdd = newShuffleRDD() // Get all the shuffle dependencies val shuffleDeps = getAllDependencies(rdd) @@ -216,34 +228,34 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo (rdd, shuffleDeps) } - def randomRdd = { + private def randomRdd() = { val rdd: RDD[_] = Random.nextInt(3) match { - case 0 => newRDD - case 1 => newShuffleRDD - case 2 => newPairRDD.join(newPairRDD) + case 0 => newRDD() + case 1 => newShuffleRDD() + case 2 => newPairRDD.join(newPairRDD()) } if (Random.nextBoolean()) rdd.persist() rdd.count() rdd } - def randomBroadcast = { + private def randomBroadcast() = { sc.broadcast(Random.nextInt(Int.MaxValue)) } /** Run GC and make sure it actually has run */ - def runGC() { + private def runGC() { val weakRef = new WeakReference(new Object()) val startTime = System.currentTimeMillis System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. // Wait until a weak reference object has been GCed - while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + while (System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { System.gc() Thread.sleep(200) } } - def cleaner = sc.cleaner.get + private def cleaner = sc.cleaner.get } From 805f329bb1535ef4b8441994705b94a3bf0d758a Mon Sep 17 00:00:00 2001 From: Lijie Xu Date: Sat, 19 Jul 2014 01:27:26 -0700 Subject: [PATCH 0012/1492] put 'curRequestSize = 0' after 'logDebug' it This is a minor change. We should first logDebug($curRequestSize) and then set it to 0. Author: Lijie Xu Closes #1477 from JerryLead/patch-1 and squashes the following commits: aed722d [Lijie Xu] put 'curRequestSize = 0' after 'logDebug' it --- .../scala/org/apache/spark/storage/BlockFetcherIterator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 408a797088059..2f0296c20f2e2 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -180,9 +180,9 @@ object BlockFetcherIterator { if (curRequestSize >= targetRequestSize) { // Add this FetchRequest remoteRequests += new FetchRequest(address, curBlocks) - curRequestSize = 0 curBlocks = new ArrayBuffer[(BlockId, Long)] logDebug(s"Creating fetch request of $curRequestSize at $address") + curRequestSize = 0 } } // Add in the final request From 2a732110d46712c535b75dd4f5a73761b6463aa8 Mon Sep 17 00:00:00 2001 From: chutium Date: Sat, 19 Jul 2014 11:04:41 -0500 Subject: [PATCH 0013/1492] SPARK-2407: Added Parser of SQL SUBSTR() follow-up of #1359 Author: chutium Closes #1442 from chutium/master and squashes the following commits: b49cc8a [chutium] SPARK-2407: Added Parser of SQL SUBSTRING() #1442 9a60ccf [chutium] SPARK-2407: Added Parser of SQL SUBSTR() #1442 06e933b [chutium] Merge https://github.com/apache/spark c870172 [chutium] Merge https://github.com/apache/spark 094f773 [chutium] Merge https://github.com/apache/spark 88cb37d [chutium] Merge https://github.com/apache/spark 1de83a7 [chutium] SPARK-2407: Added Parse of SQL SUBSTR() --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 9 ++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 15 +++++++++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index e5653c5b14ac1..a34b236c8ac6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -120,7 +120,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val WHERE = Keyword("WHERE") protected val INTERSECT = Keyword("INTERSECT") protected val EXCEPT = Keyword("EXCEPT") - + protected val SUBSTR = Keyword("SUBSTR") + protected val SUBSTRING = Keyword("SUBSTRING") // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -316,6 +317,12 @@ class SqlParser extends StandardTokenParsers with PackratParsers { IF ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { case c ~ "," ~ t ~ "," ~ f => If(c,t,f) } | + (SUBSTR | SUBSTRING) ~> "(" ~> expression ~ "," ~ expression <~ ")" ^^ { + case s ~ "," ~ p => Substring(s,p,Literal(Integer.MAX_VALUE)) + } | + (SUBSTR | SUBSTRING) ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { + case s ~ "," ~ p ~ "," ~ l => Substring(s,p,l) + } | ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ { case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0743cfe8cff0f..6736189c96d4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -36,6 +36,21 @@ class SQLQuerySuite extends QueryTest { "test") } + test("SPARK-2407 Added Parser of SQL SUBSTR()") { + checkAnswer( + sql("SELECT substr(tableName, 1, 2) FROM tableName"), + "te") + checkAnswer( + sql("SELECT substr(tableName, 3) FROM tableName"), + "st") + checkAnswer( + sql("SELECT substring(tableName, 1, 2) FROM tableName"), + "te") + checkAnswer( + sql("SELECT substring(tableName, 3) FROM tableName"), + "st") + } + test("index into array") { checkAnswer( sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), From 1efb3698b6cf39a80683b37124d2736ebf3c9d9a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 19 Jul 2014 16:56:22 -0700 Subject: [PATCH 0014/1492] Revert "[SPARK-2521] Broadcast RDD object (instead of sending it along with every task)." This reverts commit 7b8cd175254d42c8e82f0aa8eb4b7f3508d8fde2. --- .../scala/org/apache/spark/Dependency.scala | 28 ++-- .../scala/org/apache/spark/SparkContext.scala | 2 + .../main/scala/org/apache/spark/rdd/RDD.scala | 30 +--- .../apache/spark/rdd/RDDCheckpointData.scala | 9 +- .../apache/spark/scheduler/DAGScheduler.scala | 4 + .../apache/spark/scheduler/ResultTask.scala | 128 +++++++++++++----- .../spark/scheduler/ShuffleMapTask.scala | 125 +++++++++++++---- .../apache/spark/ContextCleanerSuite.scala | 62 ++++----- 8 files changed, 251 insertions(+), 137 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3935c8772252e..09a60571238ea 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -27,9 +27,7 @@ import org.apache.spark.shuffle.ShuffleHandle * Base class for dependencies. */ @DeveloperApi -abstract class Dependency[T] extends Serializable { - def rdd: RDD[T] -} +abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** @@ -38,24 +36,20 @@ abstract class Dependency[T] extends Serializable { * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ @DeveloperApi -abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { +abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. * @param partitionId a partition of the child RDD * @return the partitions of the parent RDD that the child partition depends upon */ def getParents(partitionId: Int): Seq[Int] - - override def rdd: RDD[T] = _rdd } /** * :: DeveloperApi :: - * Represents a dependency on the output of a shuffle stage. Note that in the case of shuffle, - * the RDD is transient since we don't need it on the executor side. - * - * @param _rdd the parent RDD + * Represents a dependency on the output of a shuffle stage. + * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will @@ -63,22 +57,20 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { */ @DeveloperApi class ShuffleDependency[K, V, C]( - @transient _rdd: RDD[_ <: Product2[K, V]], + @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, val mapSideCombine: Boolean = false) - extends Dependency[Product2[K, V]] { - - override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]] + extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { - val shuffleId: Int = _rdd.context.newShuffleId() + val shuffleId: Int = rdd.context.newShuffleId() - val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( - shuffleId, _rdd.partitions.size, this) + val shuffleHandle: ShuffleHandle = rdd.context.env.shuffleManager.registerShuffle( + shuffleId, rdd.partitions.size, this) - _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) + rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 48a09657fde26..8052499ab7526 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -997,6 +997,8 @@ class SparkContext(config: SparkConf) extends Logging { // TODO: Cache.stop()? env.stop() SparkEnv.set(null) + ShuffleMapTask.clearCache() + ResultTask.clearCache() listenerBus.stop() eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") 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 2ee9a8f1a8e0d..88a918aebf763 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,13 +35,12 @@ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD -import org.apache.spark.broadcast.Broadcast import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, CallSite, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} @@ -1196,36 +1195,21 @@ abstract class RDD[T: ClassTag]( /** * Return whether this RDD has been checkpointed or not */ - def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) + def isCheckpointed: Boolean = { + checkpointData.map(_.isCheckpointed).getOrElse(false) + } /** * Gets the name of the file to which this RDD was checkpointed */ - def getCheckpointFile: Option[String] = checkpointData.flatMap(_.getCheckpointFile) + def getCheckpointFile: Option[String] = { + checkpointData.flatMap(_.getCheckpointFile) + } // ======================================================================= // Other internal methods and fields // ======================================================================= - /** - * Broadcasted copy of this RDD, used to dispatch tasks to executors. Note that we broadcast - * the serialized copy of the RDD and for each task we will deserialize it, which means each - * task gets a different copy of the RDD. This provides stronger isolation between tasks that - * might modify state of objects referenced in their closures. This is necessary in Hadoop - * where the JobConf/Configuration object is not thread-safe. - */ - @transient private[spark] lazy val broadcasted: Broadcast[Array[Byte]] = { - val ser = SparkEnv.get.closureSerializer.newInstance() - val bytes = ser.serialize(this).array() - val size = Utils.bytesToString(bytes.length) - if (bytes.length > (1L << 20)) { - logWarning(s"Broadcasting RDD $id ($size), which contains large objects") - } else { - logDebug(s"Broadcasting RDD $id ($size)") - } - sc.broadcast(bytes) - } - private var storageLevel: StorageLevel = StorageLevel.NONE /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index f67e5f1857979..c3b2a33fb54d0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -106,6 +106,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) cpRDD = Some(newRDD) rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions cpState = Checkpointed + RDDCheckpointData.clearTaskCaches() } logInfo("Done checkpointing RDD " + rdd.id + " to " + path + ", new parent is RDD " + newRDD.id) } @@ -130,5 +131,9 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -// Used for synchronization -private[spark] object RDDCheckpointData +private[spark] object RDDCheckpointData { + def clearTaskCaches() { + ShuffleMapTask.clearCache() + ResultTask.clearCache() + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 88cb5feaaff2a..ede3c7d9f01ae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -376,6 +376,9 @@ class DAGScheduler( stageIdToStage -= stageId stageIdToJobIds -= stageId + ShuffleMapTask.removeStage(stageId) + ResultTask.removeStage(stageId) + logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } @@ -720,6 +723,7 @@ class DAGScheduler( } } + /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 62beb0d02a9c3..bbf9f7388b074 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,68 +17,134 @@ package org.apache.spark.scheduler -import java.nio.ByteBuffer +import scala.language.existentials import java.io._ +import java.util.zip.{GZIPInputStream, GZIPOutputStream} + +import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, RDDCheckpointData} + +private[spark] object ResultTask { + + // A simple map between the stage id to the serialized byte array of a task. + // Served as a cache for task serialization because serialization can be + // expensive on the master node if it needs to launch thousands of tasks. + private val serializedInfoCache = new HashMap[Int, Array[Byte]] + + def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = + { + synchronized { + val old = serializedInfoCache.get(stageId).orNull + if (old != null) { + old + } else { + val out = new ByteArrayOutputStream + val ser = SparkEnv.get.closureSerializer.newInstance() + val objOut = ser.serializeStream(new GZIPOutputStream(out)) + objOut.writeObject(rdd) + objOut.writeObject(func) + objOut.close() + val bytes = out.toByteArray + serializedInfoCache.put(stageId, bytes) + bytes + } + } + } + + def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = + { + val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) + val ser = SparkEnv.get.closureSerializer.newInstance() + val objIn = ser.deserializeStream(in) + val rdd = objIn.readObject().asInstanceOf[RDD[_]] + val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _] + (rdd, func) + } + + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + + def clearCache() { + synchronized { + serializedInfoCache.clear() + } + } +} + /** * A task that sends back the output to the driver application. * - * See [[Task]] for more information. + * See [[org.apache.spark.scheduler.Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param rddBinary broadcast version of of the serialized RDD + * @param rdd input to func * @param func a function to apply on a partition of the RDD - * @param partition partition of the RDD this task is associated with + * @param _partitionId index of the number in the RDD * @param locs preferred task execution locations for locality scheduling * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). */ private[spark] class ResultTask[T, U]( stageId: Int, - val rddBinary: Broadcast[Array[Byte]], - val func: (TaskContext, Iterator[T]) => U, - val partition: Partition, + var rdd: RDD[T], + var func: (TaskContext, Iterator[T]) => U, + _partitionId: Int, @transient locs: Seq[TaskLocation], - val outputId: Int) - extends Task[U](stageId, partition.index) with Serializable { - - // TODO: Should we also broadcast func? For that we would need a place to - // keep a reference to it (perhaps in DAGScheduler's job object). - - def this( - stageId: Int, - rdd: RDD[T], - func: (TaskContext, Iterator[T]) => U, - partitionId: Int, - locs: Seq[TaskLocation], - outputId: Int) = { - this(stageId, rdd.broadcasted, func, rdd.partitions(partitionId), locs, outputId) - } + var outputId: Int) + extends Task[U](stageId, _partitionId) with Externalizable { + + def this() = this(0, null, null, 0, null, 0) + + var split = if (rdd == null) null else rdd.partitions(partitionId) - @transient private[this] val preferredLocs: Seq[TaskLocation] = { + @transient private val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } override def runTask(context: TaskContext): U = { - // Deserialize the RDD using the broadcast variable. - val ser = SparkEnv.get.closureSerializer.newInstance() - val rdd = ser.deserialize[RDD[T]](ByteBuffer.wrap(rddBinary.value), - Thread.currentThread.getContextClassLoader) metrics = Some(context.taskMetrics) try { - func(context, rdd.iterator(partition, context)) + func(context, rdd.iterator(split, context)) } finally { context.executeOnCompleteCallbacks() } } - // This is only callable on the driver side. override def preferredLocations: Seq[TaskLocation] = preferredLocs override def toString = "ResultTask(" + stageId + ", " + partitionId + ")" + + override def writeExternal(out: ObjectOutput) { + RDDCheckpointData.synchronized { + split = rdd.partitions(partitionId) + out.writeInt(stageId) + val bytes = ResultTask.serializeInfo( + stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _]) + out.writeInt(bytes.length) + out.write(bytes) + out.writeInt(partitionId) + out.writeInt(outputId) + out.writeLong(epoch) + out.writeObject(split) + } + } + + override def readExternal(in: ObjectInput) { + val stageId = in.readInt() + val numBytes = in.readInt() + val bytes = new Array[Byte](numBytes) + in.readFully(bytes) + val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes) + rdd = rdd_.asInstanceOf[RDD[T]] + func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] + partitionId = in.readInt() + outputId = in.readInt() + epoch = in.readLong() + split = in.readObject().asInstanceOf[Partition] + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 033c6e52861e0..fdaf1de83f051 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,13 +17,71 @@ package org.apache.spark.scheduler -import java.nio.ByteBuffer +import scala.language.existentials + +import java.io._ +import java.util.zip.{GZIPInputStream, GZIPOutputStream} + +import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.shuffle.ShuffleWriter +private[spark] object ShuffleMapTask { + + // A simple map between the stage id to the serialized byte array of a task. + // Served as a cache for task serialization because serialization can be + // expensive on the master node if it needs to launch thousands of tasks. + private val serializedInfoCache = new HashMap[Int, Array[Byte]] + + def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_, _, _]): Array[Byte] = { + synchronized { + val old = serializedInfoCache.get(stageId).orNull + if (old != null) { + return old + } else { + val out = new ByteArrayOutputStream + val ser = SparkEnv.get.closureSerializer.newInstance() + val objOut = ser.serializeStream(new GZIPOutputStream(out)) + objOut.writeObject(rdd) + objOut.writeObject(dep) + objOut.close() + val bytes = out.toByteArray + serializedInfoCache.put(stageId, bytes) + bytes + } + } + } + + def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_, _, _]) = { + val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) + val ser = SparkEnv.get.closureSerializer.newInstance() + val objIn = ser.deserializeStream(in) + val rdd = objIn.readObject().asInstanceOf[RDD[_]] + val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_, _, _]] + (rdd, dep) + } + + // Since both the JarSet and FileSet have the same format this is used for both. + def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = { + val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) + val objIn = new ObjectInputStream(in) + val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap + HashMap(set.toSeq: _*) + } + + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + + def clearCache() { + synchronized { + serializedInfoCache.clear() + } + } +} + /** * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner * specified in the ShuffleDependency). @@ -31,47 +89,62 @@ import org.apache.spark.shuffle.ShuffleWriter * See [[org.apache.spark.scheduler.Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param rddBinary broadcast version of of the serialized RDD + * @param rdd the final RDD in this stage * @param dep the ShuffleDependency - * @param partition partition of the RDD this task is associated with + * @param _partitionId index of the number in the RDD * @param locs preferred task execution locations for locality scheduling */ private[spark] class ShuffleMapTask( stageId: Int, - var rddBinary: Broadcast[Array[Byte]], + var rdd: RDD[_], var dep: ShuffleDependency[_, _, _], - partition: Partition, + _partitionId: Int, @transient private var locs: Seq[TaskLocation]) - extends Task[MapStatus](stageId, partition.index) with Logging { - - // TODO: Should we also broadcast the ShuffleDependency? For that we would need a place to - // keep a reference to it (perhaps in Stage). - - def this( - stageId: Int, - rdd: RDD[_], - dep: ShuffleDependency[_, _, _], - partitionId: Int, - locs: Seq[TaskLocation]) = { - this(stageId, rdd.broadcasted, dep, rdd.partitions(partitionId), locs) - } + extends Task[MapStatus](stageId, _partitionId) + with Externalizable + with Logging { + + protected def this() = this(0, null, null, 0, null) @transient private val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } - override def runTask(context: TaskContext): MapStatus = { - // Deserialize the RDD using the broadcast variable. - val ser = SparkEnv.get.closureSerializer.newInstance() - val rdd = ser.deserialize[RDD[_]](ByteBuffer.wrap(rddBinary.value), - Thread.currentThread.getContextClassLoader) + var split = if (rdd == null) null else rdd.partitions(partitionId) + + override def writeExternal(out: ObjectOutput) { + RDDCheckpointData.synchronized { + split = rdd.partitions(partitionId) + out.writeInt(stageId) + val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep) + out.writeInt(bytes.length) + out.write(bytes) + out.writeInt(partitionId) + out.writeLong(epoch) + out.writeObject(split) + } + } + override def readExternal(in: ObjectInput) { + val stageId = in.readInt() + val numBytes = in.readInt() + val bytes = new Array[Byte](numBytes) + in.readFully(bytes) + val (rdd_, dep_) = ShuffleMapTask.deserializeInfo(stageId, bytes) + rdd = rdd_ + dep = dep_ + partitionId = in.readInt() + epoch = in.readLong() + split = in.readObject().asInstanceOf[Partition] + } + + override def runTask(context: TaskContext): MapStatus = { metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null try { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) - writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + writer.write(rdd.iterator(split, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) return writer.stop(success = true).get } catch { case e: Exception => diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 871f831531bee..13b415cccb647 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -52,8 +52,9 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } } + test("cleanup RDD") { - val rdd = newRDD().persist() + val rdd = newRDD.persist() val collected = rdd.collect().toList val tester = new CleanerTester(sc, rddIds = Seq(rdd.id)) @@ -66,7 +67,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup shuffle") { - val (rdd, shuffleDeps) = newRDDWithShuffleDependencies() + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies val collected = rdd.collect().toList val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) @@ -79,7 +80,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup broadcast") { - val broadcast = newBroadcast() + val broadcast = newBroadcast val tester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) // Explicit cleanup @@ -88,7 +89,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup RDD") { - var rdd = newRDD().persist() + var rdd = newRDD.persist() rdd.count() // Test that GC does not cause RDD cleanup due to a strong reference @@ -106,7 +107,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup shuffle") { - var rdd = newShuffleRDD() + var rdd = newShuffleRDD rdd.count() // Test that GC does not cause shuffle cleanup due to a strong reference @@ -124,7 +125,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup broadcast") { - var broadcast = newBroadcast() + var broadcast = newBroadcast // Test that GC does not cause broadcast cleanup due to a strong reference val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) @@ -140,23 +141,11 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo postGCTester.assertCleanup() } - test("automatically cleanup broadcast data for task dispatching") { - var rdd = newRDDWithShuffleDependencies()._1 - rdd.count() // This triggers an action that broadcasts the RDDs. - - // Test that GC causes broadcast task data cleanup after dereferencing the RDD. - val postGCTester = new CleanerTester(sc, - broadcastIds = Seq(rdd.broadcasted.id, rdd.firstParent.broadcasted.id)) - rdd = null - runGC() - postGCTester.assertCleanup() - } - test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId val broadcastIds = 0L until numBroadcasts @@ -186,8 +175,8 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo val numRdds = 10 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId val broadcastIds = 0L until numBroadcasts @@ -208,18 +197,17 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo //------ Helper functions ------ - private def newRDD() = sc.makeRDD(1 to 10) - private def newPairRDD() = newRDD().map(_ -> 1) - private def newShuffleRDD() = newPairRDD().reduceByKey(_ + _) - private def newBroadcast() = sc.broadcast(1 to 100) - - private def newRDDWithShuffleDependencies(): (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { + def newRDD = sc.makeRDD(1 to 10) + def newPairRDD = newRDD.map(_ -> 1) + def newShuffleRDD = newPairRDD.reduceByKey(_ + _) + def newBroadcast = sc.broadcast(1 to 100) + def newRDDWithShuffleDependencies: (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { rdd.dependencies ++ rdd.dependencies.flatMap { dep => getAllDependencies(dep.rdd) } } - val rdd = newShuffleRDD() + val rdd = newShuffleRDD // Get all the shuffle dependencies val shuffleDeps = getAllDependencies(rdd) @@ -228,34 +216,34 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo (rdd, shuffleDeps) } - private def randomRdd() = { + def randomRdd = { val rdd: RDD[_] = Random.nextInt(3) match { - case 0 => newRDD() - case 1 => newShuffleRDD() - case 2 => newPairRDD.join(newPairRDD()) + case 0 => newRDD + case 1 => newShuffleRDD + case 2 => newPairRDD.join(newPairRDD) } if (Random.nextBoolean()) rdd.persist() rdd.count() rdd } - private def randomBroadcast() = { + def randomBroadcast = { sc.broadcast(Random.nextInt(Int.MaxValue)) } /** Run GC and make sure it actually has run */ - private def runGC() { + def runGC() { val weakRef = new WeakReference(new Object()) val startTime = System.currentTimeMillis System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. // Wait until a weak reference object has been GCed - while (System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { System.gc() Thread.sleep(200) } } - private def cleaner = sc.cleaner.get + def cleaner = sc.cleaner.get } From 49e472744951d875627d78b0d6e93cd139232929 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 19 Jul 2014 18:19:08 -0700 Subject: [PATCH 0015/1492] SPARK-2596 A tool for mirroring github pull requests on JIRA. For a bunch of reasons we should automatically populate a JIRA with information about new pull requests when they arrive. I've written a small python script to do this that we can run from Jenkins every 5 or 10 minutes to keep things in sync. Author: Patrick Wendell Closes #1496 from pwendell/github-integration and squashes the following commits: 55ad226 [Patrick Wendell] Small fix afda547 [Patrick Wendell] Use sequence instead of dictiory for JIRA's 3e18cc1 [Patrick Wendell] Small edits 84c5606 [Patrick Wendell] SPARK-2596 A tool for mirroring github pull requests on JIRA. --- dev/github_jira_sync.py | 141 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 141 insertions(+) create mode 100755 dev/github_jira_sync.py diff --git a/dev/github_jira_sync.py b/dev/github_jira_sync.py new file mode 100755 index 0000000000000..4b0e266bbe60e --- /dev/null +++ b/dev/github_jira_sync.py @@ -0,0 +1,141 @@ +#!/usr/bin/env python + +# +# 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. +# +# Utility for updating JIRA's with information about Github pull requests + +import json +import os +import re +import sys +import urllib2 + +try: + import jira.client +except ImportError: + print "This tool requires the jira-python library" + print "Install using 'sudo pip install jira-python'" + sys.exit(-1) + +# User facing configs +GITHUB_API_BASE = os.environ.get("GITHUB_API_BASE", "https://api.github.com/repos/apache/spark") +JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "apachespark") +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "XXX") +# Maximum number of updates to perform in one run +MAX_UPDATES = int(os.environ.get("MAX_UPDATES", "100000")) +# Cut-off for oldest PR on which to comment. Useful for avoiding +# "notification overload" when running for the first time. +MIN_COMMENT_PR = int(os.environ.get("MIN_COMMENT_PR", "1496")) + +# File used as an opitimization to store maximum previously seen PR +# Used mostly because accessing ASF JIRA is slow, so we want to avoid checking +# the state of JIRA's that are tied to PR's we've already looked at. +MAX_FILE = ".github-jira-max" + +def get_url(url): + try: + return urllib2.urlopen(url) + except urllib2.HTTPError as e: + print "Unable to fetch URL, exiting: %s" % url + sys.exit(-1) + +def get_json(urllib_response): + return json.load(urllib_response) + +# Return a list of (JIRA id, JSON dict) tuples: +# e.g. [('SPARK-1234', {.. json ..}), ('SPARK-5687', {.. json ..})} +def get_jira_prs(): + result = [] + has_next_page = True + page_num = 0 + while has_next_page: + page = get_url(GITHUB_API_BASE + "/pulls?page=%s&per_page=100" % page_num) + page_json = get_json(page) + + for pull in page_json: + jiras = re.findall("SPARK-[0-9]{4,5}", pull['title']) + for jira in jiras: + result = result + [(jira, pull)] + + # Check if there is another page + link_header = filter(lambda k: k.startswith("Link"), page.info().headers)[0] + if not "next"in link_header: + has_next_page = False + else: + page_num = page_num + 1 + return result + +def set_max_pr(max_val): + f = open(MAX_FILE, 'w') + f.write("%s" % max_val) + f.close() + print "Writing largest PR number seen: %s" % max_val + +def get_max_pr(): + if os.path.exists(MAX_FILE): + result = int(open(MAX_FILE, 'r').read()) + print "Read largest PR number previously seen: %s" % result + return result + else: + return 0 + +jira_client = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + +jira_prs = get_jira_prs() + +previous_max = get_max_pr() +print "Retrieved %s JIRA PR's from Github" % len(jira_prs) +jira_prs = [(k, v) for k, v in jira_prs if int(v['number']) > previous_max] +print "%s PR's remain after excluding visted ones" % len(jira_prs) + +num_updates = 0 +considered = [] +for issue, pr in sorted(jira_prs, key=lambda (k, v): int(v['number'])): + if num_updates >= MAX_UPDATES: + break + pr_num = int(pr['number']) + + print "Checking issue %s" % issue + considered = considered + [pr_num] + + url = pr['html_url'] + title = "[Github] Pull Request #%s (%s)" % (pr['number'], pr['user']['login']) + + existing_links = map(lambda l: l.raw['object']['url'], jira_client.remote_links(issue)) + if url in existing_links: + continue + + icon = {"title": "Pull request #%s" % pr['number'], + "url16x16": "https://assets-cdn.github.com/favicon.ico"} + destination = {"title": title, "url": url, "icon": icon} + # For all possible fields see: + # https://developer.atlassian.com/display/JIRADEV/Fields+in+Remote+Issue+Links + # application = {"name": "Github pull requests", "type": "org.apache.spark.jira.github"} + jira_client.add_remote_link(issue, destination) + + comment = "User '%s' has created a pull request for this issue:" % pr['user']['login'] + comment = comment + ("\n%s" % pr['html_url']) + if pr_num >= MIN_COMMENT_PR: + jira_client.add_comment(issue, comment) + + print "Added link %s <-> PR #%s" % (issue, pr['number']) + num_updates = num_updates + 1 + +if len(considered) > 0: + set_max_pr(max(considered)) From d39e3b9673027bb9f4d1542e5a2386f73078eec0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 19 Jul 2014 18:24:21 -0700 Subject: [PATCH 0016/1492] SPARK-2596 HOTFIX: Deal with non-existent JIRAs. A small bug that was found in our JIRA sync script. --- dev/github_jira_sync.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/dev/github_jira_sync.py b/dev/github_jira_sync.py index 4b0e266bbe60e..8051080117062 100755 --- a/dev/github_jira_sync.py +++ b/dev/github_jira_sync.py @@ -116,8 +116,13 @@ def get_max_pr(): url = pr['html_url'] title = "[Github] Pull Request #%s (%s)" % (pr['number'], pr['user']['login']) - - existing_links = map(lambda l: l.raw['object']['url'], jira_client.remote_links(issue)) + try: + existing_links = map(lambda l: l.raw['object']['url'], jira_client.remote_links(issue)) + except: + print "Failure reading JIRA %s (does it exist?)" % issue + print sys.exc_info()[0] + continue + if url in existing_links: continue From 0d01e85f42f3c997df7fee942b05b509968bac4b Mon Sep 17 00:00:00 2001 From: Cesar Arevalo Date: Sat, 19 Jul 2014 20:20:07 -0700 Subject: [PATCH 0017/1492] Typo fix to the programming guide in the docs Typo fix to the programming guide in the docs. Changed the word "distibuted" to "distributed". Author: Cesar Arevalo Closes #1495 from cesararevalo/master and squashes the following commits: 0c2e3a7 [Cesar Arevalo] Typo fix to the programming guide in the docs --- docs/programming-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index b09d6347cd1b2..90c69713019f2 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -739,7 +739,7 @@ def doStuff(self, rdd): While most Spark operations work on RDDs containing any type of objects, a few special operations are only available on RDDs of key-value pairs. -The most common ones are distibuted "shuffle" operations, such as grouping or aggregating the elements +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements by a key. In Scala, these operations are automatically available on RDDs containing @@ -773,7 +773,7 @@ documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#ha While most Spark operations work on RDDs containing any type of objects, a few special operations are only available on RDDs of key-value pairs. -The most common ones are distibuted "shuffle" operations, such as grouping or aggregating the elements +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements by a key. In Java, key-value pairs are represented using the @@ -810,7 +810,7 @@ documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#ha While most Spark operations work on RDDs containing any type of objects, a few special operations are only available on RDDs of key-value pairs. -The most common ones are distibuted "shuffle" operations, such as grouping or aggregating the elements +The most common ones are distributed "shuffle" operations, such as grouping or aggregating the elements by a key. In Python, these operations work on RDDs containing built-in Python tuples such as `(1, 2)`. From c11949878d650ed776b190444b198dbdacb0c5d7 Mon Sep 17 00:00:00 2001 From: Mark Wagner Date: Sat, 19 Jul 2014 20:24:13 -0700 Subject: [PATCH 0018/1492] SPARK-2587: Fix error message in make-distribution.sh make-distribution.sh gives a slightly off error message when using --with-hive. Author: Mark Wagner Closes #1489 from wagnermarkd/SPARK-2587 and squashes the following commits: 7b5d3ff [Mark Wagner] SPARK-2587: Fix error message in make-distribution.sh --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index b5a90f0f3bfe9..c08093f46b61f 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -58,7 +58,7 @@ while (( "$#" )); do exit_with_usage ;; --with-hive) - echo "Error: '--with-hive' is no longer supported, use Maven option -Pyarn" + echo "Error: '--with-hive' is no longer supported, use Maven option -Phive" exit_with_usage ;; --skip-java-test) From 4da01e3813f0a0413fe691358c14278bbd5508ed Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Sat, 19 Jul 2014 20:46:59 -0700 Subject: [PATCH 0019/1492] [SPARK-2524] missing document about spark.deploy.retainedDrivers https://issues.apache.org/jira/browse/SPARK-2524 The configuration on spark.deploy.retainedDrivers is undocumented but actually used https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/master/Master.scala#L60 Author: lianhuiwang Author: Wang Lianhui Author: unknown Closes #1443 from lianhuiwang/SPARK-2524 and squashes the following commits: 64660fd [Wang Lianhui] address pwendell's comments 5f6bbb7 [Wang Lianhui] missing document about spark.deploy.retainedDrivers 44a3f50 [unknown] Merge remote-tracking branch 'upstream/master' eacf933 [lianhuiwang] Merge remote-tracking branch 'upstream/master' 8bbfe76 [lianhuiwang] Merge remote-tracking branch 'upstream/master' 480ce94 [lianhuiwang] address aarondav comments f2b5970 [lianhuiwang] bugfix worker DriverStateChanged state should match DriverState.FAILED --- docs/spark-standalone.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index f5c0f7cef83d2..ad8b6c0e51a78 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -156,6 +156,20 @@ SPARK_MASTER_OPTS supports the following system properties: + + + + + + + + + + From 98ab4112255d4e0fdb6e084bd3fe65807c5b209b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sun, 20 Jul 2014 01:24:32 -0700 Subject: [PATCH 0020/1492] SPARK-2519 part 2. Remove pattern matching on Tuple2 in critical section... ...s of CoGroupedRDD and PairRDDFunctions This also removes an unnecessary tuple creation in cogroup. Author: Sandy Ryza Closes #1447 from sryza/sandy-spark-2519-2 and squashes the following commits: b6d9699 [Sandy Ryza] Remove missed Tuple2 match in CoGroupedRDD a109828 [Sandy Ryza] Remove another pattern matching in MappedValuesRDD and revert some changes in PairRDDFunctions be10f8a [Sandy Ryza] SPARK-2519 part 2. Remove pattern matching on Tuple2 in critical sections of CoGroupedRDD and PairRDDFunctions --- .../org/apache/spark/rdd/CoGroupedRDD.scala | 4 +- .../apache/spark/rdd/MappedValuesRDD.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 60 +++++++++---------- 3 files changed, 33 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 5366c1a1cc1bd..aca235a62a6a8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -170,12 +170,12 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: val createCombiner: (CoGroupValue => CoGroupCombiner) = value => { val newCombiner = Array.fill(numRdds)(new CoGroup) - value match { case (v, depNum) => newCombiner(depNum) += v } + newCombiner(value._2) += value._1 newCombiner } val mergeValue: (CoGroupCombiner, CoGroupValue) => CoGroupCombiner = (combiner, value) => { - value match { case (v, depNum) => combiner(depNum) += v } + combiner(value._2) += value._1 combiner } val mergeCombiners: (CoGroupCombiner, CoGroupCombiner) => CoGroupCombiner = diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala index 2bc47eb9fcd74..a60952eee5901 100644 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala @@ -28,6 +28,6 @@ class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) override val partitioner = firstParent[Product2[K, U]].partitioner override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = { - firstParent[Product2[K, V]].iterator(split, context).map { case Product2(k ,v) => (k, f(v)) } + firstParent[Product2[K, V]].iterator(split, context).map { pair => (pair._1, f(pair._2)) } } } 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 29038b0359ccd..a6b920467283e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -216,17 +216,17 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val reducePartition = (iter: Iterator[(K, V)]) => { val map = new JHashMap[K, V] - iter.foreach { case (k, v) => - val old = map.get(k) - map.put(k, if (old == null) v else func(old, v)) + iter.foreach { pair => + val old = map.get(pair._1) + map.put(pair._1, if (old == null) pair._2 else func(old, pair._2)) } Iterator(map) } : Iterator[JHashMap[K, V]] val mergeMaps = (m1: JHashMap[K, V], m2: JHashMap[K, V]) => { - m2.foreach { case (k, v) => - val old = m1.get(k) - m1.put(k, if (old == null) v else func(old, v)) + m2.foreach { pair => + val old = m1.get(pair._1) + m1.put(pair._1, if (old == null) pair._2 else func(old, pair._2)) } m1 } : JHashMap[K, V] @@ -401,9 +401,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * (k, v2) is in `other`. Uses the given Partitioner to partition the output RDD. */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs; w <- ws) yield (v, w) - } + this.cogroup(other, partitioner).flatMapValues( pair => + for (v <- pair._1; w <- pair._2) yield (v, w) + ) } /** @@ -413,11 +413,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * partition the output RDD. */ def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - if (ws.isEmpty) { - vs.map(v => (v, None)) + this.cogroup(other, partitioner).flatMapValues { pair => + if (pair._2.isEmpty) { + pair._1.map(v => (v, None)) } else { - for (v <- vs; w <- ws) yield (v, Some(w)) + for (v <- pair._1; w <- pair._2) yield (v, Some(w)) } } } @@ -430,11 +430,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def rightOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Option[V], W))] = { - this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - if (vs.isEmpty) { - ws.map(w => (None, w)) + this.cogroup(other, partitioner).flatMapValues { pair => + if (pair._1.isEmpty) { + pair._2.map(w => (None, w)) } else { - for (v <- vs; w <- ws) yield (Some(v), w) + for (v <- pair._1; w <- pair._2) yield (Some(v), w) } } } @@ -535,7 +535,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val data = self.collect() val map = new mutable.HashMap[K, V] map.sizeHint(data.length) - data.foreach { case (k, v) => map.put(k, v) } + data.foreach { pair => map.put(pair._1, pair._2) } map } @@ -572,10 +572,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } val cg = new CoGroupedRDD[K](Seq(self, other1, other2, other3), partitioner) cg.mapValues { case Seq(vs, w1s, w2s, w3s) => - (vs.asInstanceOf[Seq[V]], - w1s.asInstanceOf[Seq[W1]], - w2s.asInstanceOf[Seq[W2]], - w3s.asInstanceOf[Seq[W3]]) + (vs.asInstanceOf[Seq[V]], + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]], + w3s.asInstanceOf[Seq[W3]]) } } @@ -589,8 +589,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) - cg.mapValues { case Seq(vs, ws) => - (vs.asInstanceOf[Seq[V]], ws.asInstanceOf[Seq[W]]) + cg.mapValues { case Seq(vs, w1s) => + (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W]]) } } @@ -606,8 +606,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => (vs.asInstanceOf[Seq[V]], - w1s.asInstanceOf[Seq[W1]], - w2s.asInstanceOf[Seq[W2]]) + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]]) } } @@ -712,8 +712,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val index = p.getPartition(key) val process = (it: Iterator[(K, V)]) => { val buf = new ArrayBuffer[V] - for ((k, v) <- it if k == key) { - buf += v + for (pair <- it if pair._1 == key) { + buf += pair._2 } buf } : Seq[V] @@ -858,8 +858,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] try { while (iter.hasNext) { - val (k, v) = iter.next() - writer.write(k, v) + val pair = iter.next() + writer.write(pair._1, pair._2) } } finally { writer.close(hadoopContext) From fa51b0fb5bee95a402c7b7f13dcf0b46cf5bb429 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 20 Jul 2014 11:06:06 -0700 Subject: [PATCH 0021/1492] [SPARK-2598] RangePartitioner's binary search does not use the given Ordering We should fix this in branch-1.0 as well. Author: Reynold Xin Closes #1500 from rxin/rangePartitioner and squashes the following commits: c0a94f5 [Reynold Xin] [SPARK-2598] RangePartitioner's binary search does not use the given Ordering. --- .../main/scala/org/apache/spark/Partitioner.scala | 4 ++-- ...ollectionsUtil.scala => CollectionsUtils.scala} | 7 ++++--- .../scala/org/apache/spark/PartitioningSuite.scala | 14 ++++++++++++++ 3 files changed, 20 insertions(+), 5 deletions(-) rename core/src/main/scala/org/apache/spark/util/{CollectionsUtil.scala => CollectionsUtils.scala} (88%) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index ec99648a8488a..52c018baa5f7b 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -134,8 +134,8 @@ class RangePartitioner[K : Ordering : ClassTag, V]( def getPartition(key: Any): Int = { val k = key.asInstanceOf[K] var partition = 0 - if (rangeBounds.length < 1000) { - // If we have less than 100 partitions naive search + if (rangeBounds.length <= 128) { + // If we have less than 128 partitions naive search while (partition < rangeBounds.length && ordering.gt(k, rangeBounds(partition))) { partition += 1 } diff --git a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala b/core/src/main/scala/org/apache/spark/util/CollectionsUtils.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala rename to core/src/main/scala/org/apache/spark/util/CollectionsUtils.scala index e4c254b9dd6b9..85da2842e8ddb 100644 --- a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala +++ b/core/src/main/scala/org/apache/spark/util/CollectionsUtils.scala @@ -19,11 +19,11 @@ package org.apache.spark.util import java.util -import scala.Array -import scala.reflect._ +import scala.reflect.{classTag, ClassTag} private[spark] object CollectionsUtils { def makeBinarySearch[K : Ordering : ClassTag] : (Array[K], K) => Int = { + // For primitive keys, we can use the natural ordering. Otherwise, use the Ordering comparator. classTag[K] match { case ClassTag.Float => (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Float]], x.asInstanceOf[Float]) @@ -40,7 +40,8 @@ private[spark] object CollectionsUtils { case ClassTag.Long => (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[Long]], x.asInstanceOf[Long]) case _ => - (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[AnyRef]], x) + val comparator = implicitly[Ordering[K]].asInstanceOf[java.util.Comparator[Any]] + (l, x) => util.Arrays.binarySearch(l.asInstanceOf[Array[AnyRef]], x, comparator) } } } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 7c30626a0c421..4658a08064280 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -91,6 +91,17 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet } } + test("RangePartitioner for keys that are not Comparable (but with Ordering)") { + // Row does not extend Comparable, but has an implicit Ordering defined. + implicit object RowOrdering extends Ordering[Row] { + override def compare(x: Row, y: Row) = x.value - y.value + } + + val rdd = sc.parallelize(1 to 4500).map(x => (Row(x), Row(x))) + val partitioner = new RangePartitioner(1500, rdd) + partitioner.getPartition(Row(100)) + } + test("HashPartitioner not equal to RangePartitioner") { val rdd = sc.parallelize(1 to 10).map(x => (x, x)) val rangeP2 = new RangePartitioner(2, rdd) @@ -177,3 +188,6 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet // Add other tests here for classes that should be able to handle empty partitions correctly } } + + +private sealed case class Row(value: Int) From 1b10b8114a396f94fc82b0f3af1a5f66dfa0945d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 20 Jul 2014 13:04:59 -0700 Subject: [PATCH 0022/1492] [SPARK-2495][MLLIB] remove private[mllib] from linear models' constructors This is part of SPARK-2495 to allow users construct linear models manually. Author: Xiangrui Meng Closes #1492 from mengxr/public-constructor and squashes the following commits: a48b766 [Xiangrui Meng] remove private[mllib] from linear models' constructors --- .../apache/spark/mllib/classification/LogisticRegression.scala | 2 +- .../main/scala/org/apache/spark/mllib/classification/SVM.scala | 2 +- .../main/scala/org/apache/spark/mllib/regression/Lasso.scala | 2 +- .../org/apache/spark/mllib/regression/LinearRegression.scala | 2 +- .../org/apache/spark/mllib/regression/RidgeRegression.scala | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 90aa8ac998ba9..2242329b7918e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LogisticRegressionModel private[mllib] ( +class LogisticRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 316ecd713b715..80f8a1b2f1e84 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -30,7 +30,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class SVMModel private[mllib] ( +class SVMModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index a05dfc045fb8e..cb0d39e759a9f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -28,7 +28,7 @@ import org.apache.spark.rdd.RDD * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LassoModel private[mllib] ( +class LassoModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 0ebad4eb58d88..8c078ec9f66e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.optimization._ * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class LinearRegressionModel private[mllib] ( +class LinearRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index bd983bac001a0..a826deb695ee1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.linalg.Vector * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -class RidgeRegressionModel private[mllib] ( +class RidgeRegressionModel ( override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) From 9564f8548917f563930d5e87911a304bf206d26e Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sun, 20 Jul 2014 14:45:34 -0700 Subject: [PATCH 0023/1492] SPARK-2564. ShuffleReadMetrics.totalBlocksRead is redundant Author: Sandy Ryza Closes #1474 from sryza/sandy-spark-2564 and squashes the following commits: 35b8388 [Sandy Ryza] Fix compile error on upmerge 7b985fb [Sandy Ryza] Fix test compile error 43f79e6 [Sandy Ryza] SPARK-2564. ShuffleReadMetrics.totalBlocksRead is redundant --- .../main/scala/org/apache/spark/executor/TaskMetrics.scala | 3 +-- .../apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala | 1 - .../scala/org/apache/spark/storage/BlockFetcherIterator.scala | 4 +--- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 2 -- .../test/scala/org/apache/spark/util/JsonProtocolSuite.scala | 3 --- 5 files changed, 2 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 5d59e00636ee6..21fe643b8d71f 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -99,7 +99,6 @@ class TaskMetrics extends Serializable { existingMetrics.fetchWaitTime += newMetrics.fetchWaitTime existingMetrics.localBlocksFetched += newMetrics.localBlocksFetched existingMetrics.remoteBlocksFetched += newMetrics.remoteBlocksFetched - existingMetrics.totalBlocksFetched += newMetrics.totalBlocksFetched existingMetrics.remoteBytesRead += newMetrics.remoteBytesRead case None => _shuffleReadMetrics = Some(newMetrics) @@ -149,7 +148,7 @@ class ShuffleReadMetrics extends Serializable { /** * Number of blocks fetched in this shuffle by this task (remote or local) */ - var totalBlocksFetched: Int = _ + def totalBlocksFetched: Int = remoteBlocksFetched + localBlocksFetched /** * Number of remote blocks fetched in this shuffle by this task diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 3795994cd920f..99788828981c7 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -81,7 +81,6 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleMetrics.shuffleFinishTime = System.currentTimeMillis shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead - shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks context.taskMetrics.updateShuffleReadMetrics(shuffleMetrics) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 2f0296c20f2e2..69905a960a2ca 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -46,7 +46,6 @@ import org.apache.spark.util.Utils private[storage] trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { def initialize() - def totalBlocks: Int def numLocalBlocks: Int def numRemoteBlocks: Int def fetchWaitTime: Long @@ -192,7 +191,7 @@ object BlockFetcherIterator { } } logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + - totalBlocks + " blocks") + (numLocal + numRemote) + " blocks") remoteRequests } @@ -235,7 +234,6 @@ object BlockFetcherIterator { logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") } - override def totalBlocks: Int = numLocal + numRemote override def numLocalBlocks: Int = numLocal override def numRemoteBlocks: Int = numRemote override def fetchWaitTime: Long = _fetchWaitTime diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 2ff8b25a56d10..3448aaaf5724c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -237,7 +237,6 @@ private[spark] object JsonProtocol { def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ - ("Total Blocks Fetched" -> shuffleReadMetrics.totalBlocksFetched) ~ ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ @@ -548,7 +547,6 @@ private[spark] object JsonProtocol { def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { val metrics = new ShuffleReadMetrics metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] - metrics.totalBlocksFetched = (json \ "Total Blocks Fetched").extract[Int] metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 11f70a6090d24..9305b6d9738e1 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -314,7 +314,6 @@ class JsonProtocolSuite extends FunSuite { private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { assert(metrics1.shuffleFinishTime === metrics2.shuffleFinishTime) - assert(metrics1.totalBlocksFetched === metrics2.totalBlocksFetched) assert(metrics1.remoteBlocksFetched === metrics2.remoteBlocksFetched) assert(metrics1.localBlocksFetched === metrics2.localBlocksFetched) assert(metrics1.fetchWaitTime === metrics2.fetchWaitTime) @@ -513,7 +512,6 @@ class JsonProtocolSuite extends FunSuite { } else { val sr = new ShuffleReadMetrics sr.shuffleFinishTime = b + c - sr.totalBlocksFetched = e + f sr.remoteBytesRead = b + d sr.localBlocksFetched = e sr.fetchWaitTime = a + d @@ -584,7 +582,6 @@ class JsonProtocolSuite extends FunSuite { | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, | "Shuffle Read Metrics":{ | "Shuffle Finish Time":900, - | "Total Blocks Fetched":1500, | "Remote Blocks Fetched":800, | "Local Blocks Fetched":700, | "Fetch Wait Time":900, From b86db517b6a2795f687211205b6a14c8685873eb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 20 Jul 2014 18:40:36 -0700 Subject: [PATCH 0024/1492] [SPARK-2552][MLLIB] stabilize logistic function in pyspark to avoid overflow in `exp(x)` if `x` is large. Author: Xiangrui Meng Closes #1493 from mengxr/py-logistic and squashes the following commits: 259e863 [Xiangrui Meng] stabilize logistic function in pyspark --- python/pyspark/mllib/classification.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 1c0c536c4fb3d..9e28dfbb9145d 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -63,7 +63,10 @@ class LogisticRegressionModel(LinearModel): def predict(self, x): _linear_predictor_typecheck(x, self._coeff) margin = _dot(x, self._coeff) + self._intercept - prob = 1/(1 + exp(-margin)) + if margin > 0: + prob = 1 / (1 + exp(-margin)) + else: + prob = 1 - 1 / (1 + exp(margin)) return 1 if prob > 0.5 else 0 From f6e7302cb49ee227aed537026d19f68528051dfd Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sun, 20 Jul 2014 20:18:18 -0700 Subject: [PATCH 0025/1492] Improve scheduler delay tooltip. As a result of shivaram's experience debugging long scheduler delay, I think we should improve the tooltip to point people in the right direction if scheduler delay is large. Author: Kay Ousterhout Closes #1488 from kayousterhout/better_tooltips and squashes the following commits: 22176fd [Kay Ousterhout] Improve scheduler delay tooltip. --- core/src/main/scala/org/apache/spark/ui/ToolTips.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 37708d75489c8..9ced9b8107ebf 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -20,9 +20,9 @@ package org.apache.spark.ui private[spark] object ToolTips { val SCHEDULER_DELAY = """Scheduler delay includes time to ship the task from the scheduler to - the executor, and time the time to send a message from the executor to the scheduler stating - that the task has completed. When the scheduler becomes overloaded, task completion messages - become queued up, and scheduler delay increases.""" + the executor, and time to send the task result from the executor to the scheduler. If + scheduler delay is large, consider decreasing the size of tasks or decreasing the size + of task results.""" val INPUT = "Bytes read from Hadoop or from Spark storage." From db56f2df1b8027171da1b8d2571d1f2ef1e103b6 Mon Sep 17 00:00:00 2001 From: Michael Giannakopoulos Date: Sun, 20 Jul 2014 20:48:44 -0700 Subject: [PATCH 0026/1492] [SPARK-1945][MLLIB] Documentation Improvements for Spark 1.0 Standalone application examples are added to 'mllib-linear-methods.md' file written in Java. This commit is related to the issue [Add full Java Examples in MLlib docs](https://issues.apache.org/jira/browse/SPARK-1945). Also I changed the name of the sigmoid function from 'logit' to 'f'. This is because the logit function is the inverse of sigmoid. Thanks, Michael Author: Michael Giannakopoulos Closes #1311 from miccagiann/master and squashes the following commits: 8ffe5ab [Michael Giannakopoulos] Update code so as to comply with code standards. f7ad5cc [Michael Giannakopoulos] Merge remote-tracking branch 'upstream/master' 38d92c7 [Michael Giannakopoulos] Adding PCA, SVD and LBFGS examples in Java. Performing minor updates in the already committed examples so as to eradicate the call of 'productElement' function whenever is possible. cc0a089 [Michael Giannakopoulos] Modyfied Java examples so as to comply with coding standards. b1141b2 [Michael Giannakopoulos] Added Java examples for Clustering and Collaborative Filtering [mllib-clustering.md & mllib-collaborative-filtering.md]. 837f7a8 [Michael Giannakopoulos] Merge remote-tracking branch 'upstream/master' 15f0eb4 [Michael Giannakopoulos] Java examples included in 'mllib-linear-methods.md' file. --- docs/mllib-clustering.md | 49 +++++++- docs/mllib-collaborative-filtering.md | 80 ++++++++++++- docs/mllib-dimensionality-reduction.md | 94 +++++++++++++++ docs/mllib-linear-methods.md | 154 ++++++++++++++++++++++++- docs/mllib-optimization.md | 96 ++++++++++++++- 5 files changed, 465 insertions(+), 8 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index c76ac010d3f81..561de48910132 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -69,7 +69,54 @@ println("Within Set Sum of Squared Errors = " + WSSSE) All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. A standalone application example +that is equivalent to the provided example in Scala is given bellow: + +{% highlight java %} +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.clustering.KMeans; +import org.apache.spark.mllib.clustering.KMeansModel; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.SparkConf; + +public class KMeansExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("K-means Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse data + String path = "data/mllib/kmeans_data.txt"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map( + new Function() { + public Vector call(String s) { + String[] sarray = s.split(" "); + double[] values = new double[sarray.length]; + for (int i = 0; i < sarray.length; i++) + values[i] = Double.parseDouble(sarray[i]); + return Vectors.dense(values); + } + } + ); + + // Cluster the data into two classes using KMeans + int numClusters = 2; + int numIterations = 20; + KMeansModel clusters = KMeans.train(parsedData.rdd(), numClusters, numIterations); + + // Evaluate clustering by computing Within Set Sum of Squared Errors + double WSSSE = clusters.computeCost(parsedData.rdd()); + System.out.println("Within Set Sum of Squared Errors = " + WSSSE); + } +} +{% endhighlight %} + +In order to run the above standalone application using Spark framework make +sure that you follow the instructions provided at section [Standalone +Applications](quick-start.html) of the quick-start guide. What is more, you +should include to your build file *spark-mllib* as a dependency.
diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 5cd71738722a9..0d28b5f7c89b3 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -99,7 +99,85 @@ val model = ALS.trainImplicit(ratings, rank, numIterations, alpha) All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. A standalone application example +that is equivalent to the provided example in Scala is given bellow: + +{% highlight java %} +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.recommendation.ALS; +import org.apache.spark.mllib.recommendation.MatrixFactorizationModel; +import org.apache.spark.mllib.recommendation.Rating; +import org.apache.spark.SparkConf; + +public class CollaborativeFiltering { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Collaborative Filtering Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse the data + String path = "data/mllib/als/test.data"; + JavaRDD data = sc.textFile(path); + JavaRDD ratings = data.map( + new Function() { + public Rating call(String s) { + String[] sarray = s.split(","); + return new Rating(Integer.parseInt(sarray[0]), Integer.parseInt(sarray[1]), + Double.parseDouble(sarray[2])); + } + } + ); + + // Build the recommendation model using ALS + int rank = 10; + int numIterations = 20; + MatrixFactorizationModel model = ALS.train(JavaRDD.toRDD(ratings), rank, numIterations, 0.01); + + // Evaluate the model on rating data + JavaRDD> userProducts = ratings.map( + new Function>() { + public Tuple2 call(Rating r) { + return new Tuple2(r.user(), r.product()); + } + } + ); + JavaPairRDD, Double> predictions = JavaPairRDD.fromJavaRDD( + model.predict(JavaRDD.toRDD(userProducts)).toJavaRDD().map( + new Function, Double>>() { + public Tuple2, Double> call(Rating r){ + return new Tuple2, Double>( + new Tuple2(r.user(), r.product()), r.rating()); + } + } + )); + JavaRDD> ratesAndPreds = + JavaPairRDD.fromJavaRDD(ratings.map( + new Function, Double>>() { + public Tuple2, Double> call(Rating r){ + return new Tuple2, Double>( + new Tuple2(r.user(), r.product()), r.rating()); + } + } + )).join(predictions).values(); + double MSE = JavaDoubleRDD.fromRDD(ratesAndPreds.map( + new Function, Object>() { + public Object call(Tuple2 pair) { + Double err = pair._1() - pair._2(); + return err * err; + } + } + ).rdd()).mean(); + System.out.println("Mean Squared Error = " + MSE); + } +} +{% endhighlight %} + +In order to run the above standalone application using Spark framework make +sure that you follow the instructions provided at section [Standalone +Applications](quick-start.html) of the quick-start guide. What is more, you +should include to your build file *spark-mllib* as a dependency.
diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index e3608075fbb13..8e434998c15ea 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -57,10 +57,57 @@ val U: RowMatrix = svd.U // The U factor is a RowMatrix. val s: Vector = svd.s // The singular values are stored in a local dense vector. val V: Matrix = svd.V // The V factor is a local dense matrix. {% endhighlight %} + +Same code applies to `IndexedRowMatrix`. +The only difference that the `U` matrix becomes an `IndexedRowMatrix`.
+
+In order to run the following standalone application using Spark framework make +sure that you follow the instructions provided at section [Standalone +Applications](quick-start.html) of the quick-start guide. What is more, you +should include to your build file *spark-mllib* as a dependency. + +{% highlight java %} +import java.util.LinkedList; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.SingularValueDecomposition; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.rdd.RDD; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class SVD { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("SVD Example"); + SparkContext sc = new SparkContext(conf); + + double[][] array = ... + LinkedList rowsList = new LinkedList(); + for (int i = 0; i < array.length; i++) { + Vector currentRow = Vectors.dense(array[i]); + rowsList.add(currentRow); + } + JavaRDD rows = JavaSparkContext.fromSparkContext(sc).parallelize(rowsList); + + // Create a RowMatrix from JavaRDD. + RowMatrix mat = new RowMatrix(rows.rdd()); + + // Compute the top 4 singular values and corresponding singular vectors. + SingularValueDecomposition svd = mat.computeSVD(4, true, 1.0E-9d); + RowMatrix U = svd.U(); + Vector s = svd.s(); + Matrix V = svd.V(); + } +} +{% endhighlight %} Same code applies to `IndexedRowMatrix`. The only difference that the `U` matrix becomes an `IndexedRowMatrix`.
+ ## Principal component analysis (PCA) @@ -91,4 +138,51 @@ val pc: Matrix = mat.computePrincipalComponents(10) // Principal components are val projected: RowMatrix = mat.multiply(pc) {% endhighlight %} + +
+ +The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +and use them to project the vectors into a low-dimensional space. +The number of columns should be small, e.g, less than 1000. + +{% highlight java %} +import java.util.LinkedList; + +import org.apache.spark.api.java.*; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.rdd.RDD; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class PCA { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("PCA Example"); + SparkContext sc = new SparkContext(conf); + + double[][] array = ... + LinkedList rowsList = new LinkedList(); + for (int i = 0; i < array.length; i++) { + Vector currentRow = Vectors.dense(array[i]); + rowsList.add(currentRow); + } + JavaRDD rows = JavaSparkContext.fromSparkContext(sc).parallelize(rowsList); + + // Create a RowMatrix from JavaRDD. + RowMatrix mat = new RowMatrix(rows.rdd()); + + // Compute the top 3 principal components. + Matrix pc = mat.computePrincipalComponents(3); + RowMatrix projected = mat.multiply(pc); + } +} +{% endhighlight %} + +In order to run the above standalone application using Spark framework make +sure that you follow the instructions provided at section [Standalone +Applications](quick-start.html) of the quick-start guide. What is more, you +should include to your build file *spark-mllib* as a dependency. +
diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index b4d22e0df5a85..254201147edc1 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -151,10 +151,10 @@ L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). Logistic regression algorithm outputs a logistic regression model, which makes predictions by applying the logistic function `\[ -\mathrm{logit}(z) = \frac{1}{1 + e^{-z}} +\mathrm{f}(z) = \frac{1}{1 + e^{-z}} \]` -$\wv^T \x$. -By default, if $\mathrm{logit}(\wv^T x) > 0.5$, the outcome is positive, or negative otherwise. +where $z = \wv^T \x$. +By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or negative otherwise. For the same reason mentioned above, quite often in practice, this default threshold is not a good choice. The threshold should be determined via model evaluation. @@ -242,7 +242,86 @@ Similarly, you can use replace `SVMWithSGD` by All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. A standalone application example +that is equivalent to the provided example in Scala is given bellow: + +{% highlight java %} +import java.util.Random; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.classification.*; +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class SVMClassifier { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("SVM Classifier Example"); + SparkContext sc = new SparkContext(conf); + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD training = data.sample(false, 0.6, 11L); + training.cache(); + JavaRDD test = data.subtract(training); + + // Run training algorithm to build the model. + int numIterations = 100; + final SVMModel model = SVMWithSGD.train(training.rdd(), numIterations); + + // Clear the default threshold. + model.clearThreshold(); + + // Compute raw scores on the test set. + JavaRDD> scoreAndLabels = test.map( + new Function>() { + public Tuple2 call(LabeledPoint p) { + Double score = model.predict(p.features()); + return new Tuple2(score, p.label()); + } + } + ); + + // Get evaluation metrics. + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(JavaRDD.toRDD(scoreAndLabels)); + double auROC = metrics.areaUnderROC(); + + System.out.println("Area under ROC = " + auROC); + } +} +{% endhighlight %} + +The `SVMWithSGD.train()` method by default performs L2 regularization with the +regularization parameter set to 1.0. If we want to configure this algorithm, we +can customize `SVMWithSGD` further by creating a new object directly and +calling setter methods. All other MLlib algorithms support customization in +this way as well. For example, the following code produces an L1 regularized +variant of SVMs with regularization parameter set to 0.1, and runs the training +algorithm for 200 iterations. + +{% highlight java %} +import org.apache.spark.mllib.optimization.L1Updater; + +SVMWithSGD svmAlg = new SVMWithSGD(); +svmAlg.optimizer() + .setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater()); +final SVMModel modelL1 = svmAlg.run(training.rdd()); +{% endhighlight %} + +In order to run the above standalone application using Spark framework make +sure that you follow the instructions provided at section [Standalone +Applications](quick-start.html) of the quick-start guide. What is more, you +should include to your build file *spark-mllib* as a dependency.
@@ -338,7 +417,72 @@ and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.Lass All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. +calling `.rdd()` on your `JavaRDD` object. The corresponding Java example to +the Scala snippet provided, is presented bellow: + +{% highlight java %} +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.regression.LinearRegressionModel; +import org.apache.spark.mllib.regression.LinearRegressionWithSGD; +import org.apache.spark.SparkConf; + +public class LinearRegression { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("Linear Regression Example"); + JavaSparkContext sc = new JavaSparkContext(conf); + + // Load and parse the data + String path = "data/mllib/ridge-data/lpsa.data"; + JavaRDD data = sc.textFile(path); + JavaRDD parsedData = data.map( + new Function() { + public LabeledPoint call(String line) { + String[] parts = line.split(","); + String[] features = parts[1].split(" "); + double[] v = new double[features.length]; + for (int i = 0; i < features.length - 1; i++) + v[i] = Double.parseDouble(features[i]); + return new LabeledPoint(Double.parseDouble(parts[0]), Vectors.dense(v)); + } + } + ); + + // Building the model + int numIterations = 100; + final LinearRegressionModel model = + LinearRegressionWithSGD.train(JavaRDD.toRDD(parsedData), numIterations); + + // Evaluate model on training examples and compute training error + JavaRDD> valuesAndPreds = parsedData.map( + new Function>() { + public Tuple2 call(LabeledPoint point) { + double prediction = model.predict(point.features()); + return new Tuple2(prediction, point.label()); + } + } + ); + JavaRDD MSE = new JavaDoubleRDD(valuesAndPreds.map( + new Function, Object>() { + public Object call(Tuple2 pair) { + return Math.pow(pair._1() - pair._2(), 2.0); + } + } + ).rdd()).mean(); + System.out.println("training Mean Squared Error = " + MSE); + } +} +{% endhighlight %} + +In order to run the above standalone application using Spark framework make +sure that you follow the instructions provided at section [Standalone +Applications](quick-start.html) of the quick-start guide. What is more, you +should include to your build file *spark-mllib* as a dependency.
diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 651958c7812f2..26ce5f3c501ff 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -207,6 +207,10 @@ the loss computed for every iteration. Here is an example to train binary logistic regression with L2 regularization using L-BFGS optimizer. + +
+ +
{% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics @@ -263,7 +267,97 @@ println("Loss of each step in training process") loss.foreach(println) println("Area under ROC = " + auROC) {% endhighlight %} - +
+ +
+{% highlight java %} +import java.util.Arrays; +import java.util.Random; + +import scala.Tuple2; + +import org.apache.spark.api.java.*; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.optimization.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; +import org.apache.spark.SparkContext; + +public class LBFGSExample { + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("L-BFGS Example"); + SparkContext sc = new SparkContext(conf); + String path = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc, path).toJavaRDD(); + int numFeatures = data.take(1).get(0).features().size(); + + // Split initial RDD into two... [60% training data, 40% testing data]. + JavaRDD trainingInit = data.sample(false, 0.6, 11L); + JavaRDD test = data.subtract(trainingInit); + + // Append 1 into the training data as intercept. + JavaRDD> training = data.map( + new Function>() { + public Tuple2 call(LabeledPoint p) { + return new Tuple2(p.label(), MLUtils.appendBias(p.features())); + } + }); + training.cache(); + + // Run training algorithm to build the model. + int numCorrections = 10; + double convergenceTol = 1e-4; + int maxNumIterations = 20; + double regParam = 0.1; + Vector initialWeightsWithIntercept = Vectors.dense(new double[numFeatures + 1]); + + Tuple2 result = LBFGS.runLBFGS( + training.rdd(), + new LogisticGradient(), + new SquaredL2Updater(), + numCorrections, + convergenceTol, + maxNumIterations, + regParam, + initialWeightsWithIntercept); + Vector weightsWithIntercept = result._1(); + double[] loss = result._2(); + + final LogisticRegressionModel model = new LogisticRegressionModel( + Vectors.dense(Arrays.copyOf(weightsWithIntercept.toArray(), weightsWithIntercept.size() - 1)), + (weightsWithIntercept.toArray())[weightsWithIntercept.size() - 1]); + + // Clear the default threshold. + model.clearThreshold(); + + // Compute raw scores on the test set. + JavaRDD> scoreAndLabels = test.map( + new Function>() { + public Tuple2 call(LabeledPoint p) { + Double score = model.predict(p.features()); + return new Tuple2(score, p.label()); + } + }); + + // Get evaluation metrics. + BinaryClassificationMetrics metrics = + new BinaryClassificationMetrics(scoreAndLabels.rdd()); + double auROC = metrics.areaUnderROC(); + + System.out.println("Loss of each step in training process"); + for (double l : loss) + System.out.println(l); + System.out.println("Area under ROC = " + auROC); + } +} +{% endhighlight %} +
+
#### Developer's note Since the Hessian is constructed approximately from previous gradient evaluations, the objective function can not be changed during the optimization process. From cd273a238144a9a436219cd01250369586f5638b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 21 Jul 2014 00:46:28 -0700 Subject: [PATCH 0027/1492] [SPARK-2190][SQL] Specialized ColumnType for Timestamp JIRA issue: [SPARK-2190](https://issues.apache.org/jira/browse/SPARK-2190) Added specialized in-memory column type for `Timestamp`. Whitelisted all timestamp related Hive tests except `timestamp_udf`, which is timezone sensitive. Author: Cheng Lian Closes #1440 from liancheng/timestamp-column-type and squashes the following commits: e682175 [Cheng Lian] Enabled more timezone sensitive Hive tests. 53a358f [Cheng Lian] Fixed failed test suites 01b592d [Cheng Lian] Fixed SimpleDateFormat thread safety issue 2a59343 [Cheng Lian] Removed timezone sensitive Hive timestamp tests 45dd05d [Cheng Lian] Added Timestamp specific in-memory columnar representation --- .gitignore | 1 + .../spark/sql/catalyst/expressions/Cast.scala | 23 +++++++++ .../ExpressionEvaluationSuite.scala | 22 ++++----- .../spark/sql/columnar/ColumnAccessor.scala | 24 ++++++---- .../spark/sql/columnar/ColumnBuilder.scala | 3 ++ .../spark/sql/columnar/ColumnStats.scala | 37 +++++++++++++-- .../spark/sql/columnar/ColumnType.scala | 47 ++++++++++++++----- .../spark/sql/columnar/ColumnStatsSuite.scala | 17 +++---- .../spark/sql/columnar/ColumnTypeSuite.scala | 34 +++++--------- .../sql/columnar/ColumnarTestUtils.scala | 25 ++++++---- .../apache/spark/sql/hive/HiveContext.scala | 3 ++ ...tamp_1-10-343c75daac6695917608c17db8bf473e | 1 + ...tamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a | 1 + ...tamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 | 0 ...tamp_1-13-90269c1e50c7ae8e75ca9cc297982135 | 1 + ...tamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 | 1 + ...tamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 | 1 + ...tamp_1-16-e7b398d2a8107a42419c83771bda41e6 | 1 + ...tamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf | 1 + ...tamp_1-18-67f274bf16de625cf4e85af0c6185cac | 1 + ...tamp_1-19-343c75daac6695917608c17db8bf473e | 1 + ...tamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a | 1 + ...tamp_1-21-d8fff1a6c464e50eb955babfafb0b98e | 0 ...tamp_1-22-90269c1e50c7ae8e75ca9cc297982135 | 1 + ...tamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 | 1 + ...tamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 | 1 + ...tamp_1-25-e7b398d2a8107a42419c83771bda41e6 | 1 + ...tamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf | 1 + ...tamp_1-27-67f274bf16de625cf4e85af0c6185cac | 1 + ...tamp_1-28-343c75daac6695917608c17db8bf473e | 1 + ...tamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a | 1 + ...stamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 | 0 ...tamp_1-30-273256141c33eb88194cad22eb940d21 | 0 ...tamp_1-31-90269c1e50c7ae8e75ca9cc297982135 | 1 + ...tamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 | 1 + ...tamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 | 1 + ...tamp_1-34-e7b398d2a8107a42419c83771bda41e6 | 1 + ...tamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf | 1 + ...tamp_1-36-67f274bf16de625cf4e85af0c6185cac | 1 + ...tamp_1-37-343c75daac6695917608c17db8bf473e | 1 + ...tamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a | 1 + ...tamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d | 0 ...stamp_1-4-90269c1e50c7ae8e75ca9cc297982135 | 1 + ...tamp_1-40-90269c1e50c7ae8e75ca9cc297982135 | 1 + ...tamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 | 1 + ...tamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 | 1 + ...tamp_1-43-e7b398d2a8107a42419c83771bda41e6 | 1 + ...tamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf | 1 + ...tamp_1-45-67f274bf16de625cf4e85af0c6185cac | 1 + ...tamp_1-46-343c75daac6695917608c17db8bf473e | 1 + ...tamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a | 1 + ...tamp_1-48-7029255241de8e8b9710801319990044 | 0 ...tamp_1-49-90269c1e50c7ae8e75ca9cc297982135 | 1 + ...stamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 | 1 + ...tamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 | 1 + ...tamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 | 1 + ...tamp_1-52-e7b398d2a8107a42419c83771bda41e6 | 1 + ...tamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf | 1 + ...tamp_1-54-67f274bf16de625cf4e85af0c6185cac | 1 + ...tamp_1-55-343c75daac6695917608c17db8bf473e | 1 + ...tamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a | 1 + ...tamp_1-57-d362501d0176855077e65f8faf067fa8 | 0 ...stamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 | 1 + ...stamp_1-7-e7b398d2a8107a42419c83771bda41e6 | 1 + ...stamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf | 1 + ...stamp_1-9-67f274bf16de625cf4e85af0c6185cac | 1 + ...tamp_2-10-5181279a0bf8939fe46ddacae015dad8 | 1 + ...tamp_2-11-240fce5f58794fa051824e8732c00c03 | 1 + ...tamp_2-12-7350308cbf49d6ebd6599d3802750acd | 0 ...tamp_2-13-25f6ec69328af6cba76899194e0dd84e | 1 + ...tamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 | 1 + ...tamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 | 1 + ...tamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 | 1 + ...tamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef | 1 + ...stamp_2-18-252aebfe7882335d31bfc53a8705b7a | 1 + ...tamp_2-19-5181279a0bf8939fe46ddacae015dad8 | 1 + ...tamp_2-20-240fce5f58794fa051824e8732c00c03 | 1 + ...stamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b | 0 ...tamp_2-22-25f6ec69328af6cba76899194e0dd84e | 1 + ...tamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 | 1 + ...tamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 | 1 + ...tamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 | 1 + ...tamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef | 1 + ...stamp_2-27-252aebfe7882335d31bfc53a8705b7a | 1 + ...tamp_2-28-5181279a0bf8939fe46ddacae015dad8 | 1 + ...tamp_2-29-240fce5f58794fa051824e8732c00c03 | 1 + ...stamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 | 0 ...tamp_2-30-ffe6b6ddaaba84152074f7781fba2243 | 0 ...tamp_2-31-25f6ec69328af6cba76899194e0dd84e | 1 + ...tamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 | 1 + ...tamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 | 1 + ...tamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 | 1 + ...tamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef | 1 + ...stamp_2-36-252aebfe7882335d31bfc53a8705b7a | 1 + ...tamp_2-37-5181279a0bf8939fe46ddacae015dad8 | 1 + ...tamp_2-38-240fce5f58794fa051824e8732c00c03 | 1 + ...tamp_2-39-8236608f28681eac5503195096a34181 | 0 ...stamp_2-4-25f6ec69328af6cba76899194e0dd84e | 1 + ...tamp_2-40-25f6ec69328af6cba76899194e0dd84e | 1 + ...tamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 | 1 + ...tamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 | 1 + ...tamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 | 1 + ...tamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef | 1 + ...stamp_2-45-252aebfe7882335d31bfc53a8705b7a | 1 + ...tamp_2-46-5181279a0bf8939fe46ddacae015dad8 | 1 + ...tamp_2-47-240fce5f58794fa051824e8732c00c03 | 1 + ...tamp_2-48-654e5533ec6dc911996abc7e47af8ccb | 0 ...tamp_2-49-25f6ec69328af6cba76899194e0dd84e | 1 + ...stamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 | 1 + ...tamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 | 1 + ...tamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 | 1 + ...tamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 | 1 + ...tamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef | 1 + ...stamp_2-54-252aebfe7882335d31bfc53a8705b7a | 1 + ...tamp_2-55-5181279a0bf8939fe46ddacae015dad8 | 1 + ...tamp_2-56-240fce5f58794fa051824e8732c00c03 | 1 + ...tamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 | 0 ...stamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 | 1 + ...stamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 | 1 + ...stamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef | 1 + ...estamp_2-9-252aebfe7882335d31bfc53a8705b7a | 1 + ...mp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 | 0 ...mp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 | 5 ++ ...mp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 | 5 ++ ...mestamp-0-d555c8cd733572bfa8cd3362da9480cb | 1 + ...mestamp-1-8a9dbadae706047715cf5f903ff4a724 | 2 + ...mestamp-2-28c40e51e55bed62693e626efda5d9c5 | 0 ...mestamp-3-732b21d386f2002b87eaf02d0b9951ed | 0 ...mestamp-4-b2e42ebb75cecf09961d36587797f6d0 | 1 + ...mestamp-5-31243f5cb64356425b9f95ba011ac9d6 | 1 + ...mestamp-6-9b0f20bde1aaf9102b67a5498b167f31 | 1 + ...mestamp-7-47f433ff6ccce4c666440cc1a228a96d | 1 + .../execution/HiveCompatibilitySuite.scala | 15 +++++- 133 files changed, 287 insertions(+), 77 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a create mode 100644 sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 create mode 100644 sql/hive/src/test/resources/golden/timestamp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 create mode 100644 sql/hive/src/test/resources/golden/timestamp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 create mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d diff --git a/.gitignore b/.gitignore index 4f177c82ae5e0..061c8946d23c1 100644 --- a/.gitignore +++ b/.gitignore @@ -19,6 +19,7 @@ conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties conf/spark-defaults.conf +conf/hive-site.xml docs/_site docs/api target/ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 1f9716e385e9e..0ad2b30cf9c1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp +import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.sql.catalyst.types._ @@ -41,6 +42,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // UDFToString private[this] def castToString: Any => Any = child.dataType match { case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) + case TimestampType => buildCast[Timestamp](_, timestampToString) case _ => buildCast[Any](_, _.toString) } @@ -126,6 +128,18 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { ts.getTime / 1000 + ts.getNanos.toDouble / 1000000000 } + // Converts Timestamp to string according to Hive TimestampWritable convention + private[this] def timestampToString(ts: Timestamp): String = { + val timestampString = ts.toString + val formatted = Cast.threadLocalDateFormat.get.format(ts) + + if (timestampString.length > 19 && timestampString.substring(19) != ".0") { + formatted + timestampString.substring(19) + } else { + formatted + } + } + private[this] def castToLong: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toLong catch { @@ -249,3 +263,12 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { if (evaluated == null) null else cast(evaluated) } } + +object Cast { + // `SimpleDateFormat` is not thread-safe. + private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { + override def initialValue() = { + new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") + } + } +} 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 73f546455b67f..db1ae29d400c6 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 @@ -158,7 +158,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) - + checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) } @@ -203,7 +203,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("data type casting") { - val sts = "1970-01-01 00:00:01.0" + val sts = "1970-01-01 00:00:01.1" val ts = Timestamp.valueOf(sts) checkEvaluation("abdef" cast StringType, "abdef") @@ -293,7 +293,7 @@ class ExpressionEvaluationSuite extends FunSuite { // A test for higher precision than millis checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) } - + test("null checking") { val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) val c1 = 'a.string.at(0) @@ -312,7 +312,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(IsNull(Literal(null, ShortType)), true) checkEvaluation(IsNotNull(Literal(null, ShortType)), false) - + checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row) checkEvaluation(Coalesce(Literal(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) @@ -323,11 +323,11 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(If(Literal(null, BooleanType), c2, c1), "^Ba*n", row) checkEvaluation(If(Literal(true, BooleanType), c1, c2), "^Ba*n", row) checkEvaluation(If(Literal(false, BooleanType), c2, c1), "^Ba*n", row) - checkEvaluation(If(Literal(false, BooleanType), + checkEvaluation(If(Literal(false, BooleanType), Literal("a", StringType), Literal("b", StringType)), "b", row) checkEvaluation(In(c1, c1 :: c2 :: Nil), true, row) - checkEvaluation(In(Literal("^Ba*n", StringType), + checkEvaluation(In(Literal("^Ba*n", StringType), Literal("^Ba*n", StringType) :: Nil), true, row) checkEvaluation(In(Literal("^Ba*n", StringType), Literal("^Ba*n", StringType) :: c2 :: Nil), true, row) @@ -378,7 +378,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("complex type") { val row = new GenericRow(Array[Any]( - "^Ba*n", // 0 + "^Ba*n", // 0 null.asInstanceOf[String], // 1 new GenericRow(Array[Any]("aa", "bb")), // 2 Map("aa"->"bb"), // 3 @@ -391,18 +391,18 @@ class ExpressionEvaluationSuite extends FunSuite { val typeMap = MapType(StringType, StringType) val typeArray = ArrayType(StringType) - checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), Literal("aa")), "bb", row) checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row) checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), Literal(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), Literal(1)), "bb", row) checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row) checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row) - checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), Literal(null, IntegerType)), null, row) checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 3c39e1d350fa8..42a5a9a84f362 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -90,6 +90,9 @@ private[sql] class FloatColumnAccessor(buffer: ByteBuffer) private[sql] class StringColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, STRING) +private[sql] class TimestampColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, TIMESTAMP) + private[sql] class BinaryColumnAccessor(buffer: ByteBuffer) extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer, BINARY) with NullableColumnAccessor @@ -105,16 +108,17 @@ private[sql] object ColumnAccessor { val columnTypeId = dup.getInt() columnTypeId match { - case INT.typeId => new IntColumnAccessor(dup) - case LONG.typeId => new LongColumnAccessor(dup) - case FLOAT.typeId => new FloatColumnAccessor(dup) - case DOUBLE.typeId => new DoubleColumnAccessor(dup) - case BOOLEAN.typeId => new BooleanColumnAccessor(dup) - case BYTE.typeId => new ByteColumnAccessor(dup) - case SHORT.typeId => new ShortColumnAccessor(dup) - case STRING.typeId => new StringColumnAccessor(dup) - case BINARY.typeId => new BinaryColumnAccessor(dup) - case GENERIC.typeId => new GenericColumnAccessor(dup) + case INT.typeId => new IntColumnAccessor(dup) + case LONG.typeId => new LongColumnAccessor(dup) + case FLOAT.typeId => new FloatColumnAccessor(dup) + case DOUBLE.typeId => new DoubleColumnAccessor(dup) + case BOOLEAN.typeId => new BooleanColumnAccessor(dup) + case BYTE.typeId => new ByteColumnAccessor(dup) + case SHORT.typeId => new ShortColumnAccessor(dup) + case STRING.typeId => new StringColumnAccessor(dup) + case TIMESTAMP.typeId => new TimestampColumnAccessor(dup) + case BINARY.typeId => new BinaryColumnAccessor(dup) + case GENERIC.typeId => new GenericColumnAccessor(dup) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 4be048cd742d6..74f5630fbddf1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -109,6 +109,9 @@ private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColum private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) +private[sql] class TimestampColumnBuilder + extends NativeColumnBuilder(new TimestampColumnStats, TIMESTAMP) + private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY) // TODO (lian) Add support for array, struct and map diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 95602d321dc6f..6502110e903fe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -344,21 +344,52 @@ private[sql] class StringColumnStats extends BasicColumnStats(STRING) { } override def contains(row: Row, ordinal: Int) = { - !(upperBound eq null) && { + (upperBound ne null) && { val field = columnType.getField(row, ordinal) lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 } } override def isAbove(row: Row, ordinal: Int) = { - !(upperBound eq null) && { + (upperBound ne null) && { val field = columnType.getField(row, ordinal) field.compareTo(upperBound) < 0 } } override def isBelow(row: Row, ordinal: Int) = { - !(lowerBound eq null) && { + (lowerBound ne null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) < 0 + } + } +} + +private[sql] class TimestampColumnStats extends BasicColumnStats(TIMESTAMP) { + override def initialBounds = (null, null) + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field + if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field + } + + override def contains(row: Row, ordinal: Int) = { + (upperBound ne null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 + } + } + + override def isAbove(row: Row, ordinal: Int) = { + (lowerBound ne null) && { + val field = columnType.getField(row, ordinal) + field.compareTo(upperBound) < 0 + } + } + + override def isBelow(row: Row, ordinal: Int) = { + (lowerBound ne null) && { val field = columnType.getField(row, ordinal) lowerBound.compareTo(field) < 0 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 4cd52d8288137..794bc60d0e315 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -21,6 +21,8 @@ import java.nio.ByteBuffer import scala.reflect.runtime.universe.TypeTag +import java.sql.Timestamp + import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types._ @@ -221,6 +223,26 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { override def getField(row: Row, ordinal: Int) = row.getString(ordinal) } +private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { + override def extract(buffer: ByteBuffer) = { + val timestamp = new Timestamp(buffer.getLong()) + timestamp.setNanos(buffer.getInt()) + timestamp + } + + override def append(v: Timestamp, buffer: ByteBuffer) { + buffer.putLong(v.getTime).putInt(v.getNanos) + } + + override def getField(row: Row, ordinal: Int) = { + row(ordinal).asInstanceOf[Timestamp] + } + + override def setField(row: MutableRow, ordinal: Int, value: Timestamp) { + row(ordinal) = value + } +} + private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( typeId: Int, defaultSize: Int) @@ -240,7 +262,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) { +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { row(ordinal) = value } @@ -251,7 +273,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) { // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) { +private[sql] object GENERIC extends ByteArrayColumnType[DataType](10, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } @@ -262,16 +284,17 @@ private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) { private[sql] object ColumnType { def apply(dataType: DataType): ColumnType[_, _] = { dataType match { - case IntegerType => INT - case LongType => LONG - case FloatType => FLOAT - case DoubleType => DOUBLE - case BooleanType => BOOLEAN - case ByteType => BYTE - case ShortType => SHORT - case StringType => STRING - case BinaryType => BINARY - case _ => GENERIC + case IntegerType => INT + case LongType => LONG + case FloatType => FLOAT + case DoubleType => DOUBLE + case BooleanType => BOOLEAN + case ByteType => BYTE + case ShortType => SHORT + case StringType => STRING + case BinaryType => BINARY + case TimestampType => TIMESTAMP + case _ => GENERIC } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 6f0d46d816266..5f61fb5e16ea3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -22,14 +22,15 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types._ class ColumnStatsSuite extends FunSuite { - testColumnStats(classOf[BooleanColumnStats], BOOLEAN) - testColumnStats(classOf[ByteColumnStats], BYTE) - testColumnStats(classOf[ShortColumnStats], SHORT) - testColumnStats(classOf[IntColumnStats], INT) - testColumnStats(classOf[LongColumnStats], LONG) - testColumnStats(classOf[FloatColumnStats], FLOAT) - testColumnStats(classOf[DoubleColumnStats], DOUBLE) - testColumnStats(classOf[StringColumnStats], STRING) + testColumnStats(classOf[BooleanColumnStats], BOOLEAN) + testColumnStats(classOf[ByteColumnStats], BYTE) + testColumnStats(classOf[ShortColumnStats], SHORT) + testColumnStats(classOf[IntColumnStats], INT) + testColumnStats(classOf[LongColumnStats], LONG) + testColumnStats(classOf[FloatColumnStats], FLOAT) + testColumnStats(classOf[DoubleColumnStats], DOUBLE) + testColumnStats(classOf[StringColumnStats], STRING) + testColumnStats(classOf[TimestampColumnStats], TIMESTAMP) def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]]( columnStatsClass: Class[U], diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 314b7d317ed75..829342215e691 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import java.sql.Timestamp import org.scalatest.FunSuite @@ -32,7 +33,7 @@ class ColumnTypeSuite extends FunSuite with Logging { test("defaultSize") { val checks = Map( INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, - BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16) + BOOLEAN -> 1, STRING -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -52,14 +53,15 @@ class ColumnTypeSuite extends FunSuite with Logging { } } - checkActualSize(INT, Int.MaxValue, 4) - checkActualSize(SHORT, Short.MaxValue, 2) - checkActualSize(LONG, Long.MaxValue, 8) - checkActualSize(BYTE, Byte.MaxValue, 1) - checkActualSize(DOUBLE, Double.MaxValue, 8) - checkActualSize(FLOAT, Float.MaxValue, 4) - checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(INT, Int.MaxValue, 4) + checkActualSize(SHORT, Short.MaxValue, 2) + checkActualSize(LONG, Long.MaxValue, 8) + checkActualSize(BYTE, Byte.MaxValue, 1) + checkActualSize(DOUBLE, Double.MaxValue, 8) + checkActualSize(FLOAT, Float.MaxValue, 4) + checkActualSize(BOOLEAN, true, 1) + checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: Byte) checkActualSize(BINARY, binary, 4 + 4) @@ -188,17 +190,7 @@ class ColumnTypeSuite extends FunSuite with Logging { } private def hexDump(value: Any): String = { - if (value.isInstanceOf[String]) { - val sb = new StringBuilder() - for (ch <- value.asInstanceOf[String].toCharArray) { - sb.append(Integer.toHexString(ch & 0xffff)).append(' ') - } - if (! sb.isEmpty) sb.setLength(sb.length - 1) - sb.toString() - } else { - // for now .. - hexDump(value.toString) - } + value.toString.map(ch => Integer.toHexString(ch & 0xffff)).mkString(" ") } private def dumpBuffer(buff: ByteBuffer): Any = { @@ -207,7 +199,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val b = buff.get() sb.append(Integer.toHexString(b & 0xff)).append(' ') } - if (! sb.isEmpty) sb.setLength(sb.length - 1) + if (sb.nonEmpty) sb.setLength(sb.length - 1) sb.toString() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index 04bdc43d95328..38b04dd959f70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql.columnar import scala.collection.immutable.HashSet import scala.util.Random +import java.sql.Timestamp + import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.{DataType, NativeType} @@ -39,15 +41,19 @@ object ColumnarTestUtils { } (columnType match { - case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte - case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort - case INT => Random.nextInt() - case LONG => Random.nextLong() - case FLOAT => Random.nextFloat() - case DOUBLE => Random.nextDouble() - case STRING => Random.nextString(Random.nextInt(32)) - case BOOLEAN => Random.nextBoolean() - case BINARY => randomBytes(Random.nextInt(32)) + case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte + case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort + case INT => Random.nextInt() + case LONG => Random.nextLong() + case FLOAT => Random.nextFloat() + case DOUBLE => Random.nextDouble() + case STRING => Random.nextString(Random.nextInt(32)) + case BOOLEAN => Random.nextBoolean() + case BINARY => randomBytes(Random.nextInt(32)) + case TIMESTAMP => + val timestamp = new Timestamp(Random.nextLong()) + timestamp.setNanos(Random.nextInt(999999999)) + timestamp case _ => // Using a random one-element map instead of an arbitrary object Map(Random.nextInt() -> Random.nextString(Random.nextInt(32))) @@ -96,5 +102,4 @@ object ColumnarTestUtils { (values, rows) } - } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 7aedfcd74189b..334462357eb86 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, File, InputStreamReader, PrintStream} +import java.sql.Timestamp import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ @@ -28,6 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.serde2.io.TimestampWritable import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD @@ -266,6 +268,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" + case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (other, tpe) if primitiveTypes contains tpe => other.toString } diff --git a/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 b/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e b/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..2158e77f78768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.2938724611E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..4bdd802b9cda9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.1 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 b/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 b/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d b/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 b/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e new file mode 100644 index 0000000000000..3eefb349894ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e @@ -0,0 +1 @@ +1.293872461001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a new file mode 100644 index 0000000000000..acce9d97b5eba --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a @@ -0,0 +1 @@ +2011-01-01 01:01:01.001000011 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd b/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b77c91ccee412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.293872461E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..bc2423dc08a1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b b/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..2158e77f78768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.2938724611E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..4bdd802b9cda9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.1 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 b/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 b/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 b/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..b71ff60863360 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.2938724610001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..8b014c4cd8d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.0001 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb b/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 new file mode 100644 index 0000000000000..987e7ca9a76df --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 @@ -0,0 +1 @@ +77 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 new file mode 100644 index 0000000000000..3eefb349894ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 @@ -0,0 +1 @@ +1.293872461001E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 new file mode 100644 index 0000000000000..acce9d97b5eba --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 @@ -0,0 +1 @@ +2011-01-01 01:01:01.001000011 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 new file mode 100644 index 0000000000000..8f3a49478b26b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 @@ -0,0 +1 @@ +-4787 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef new file mode 100644 index 0000000000000..211a320b9ca72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef @@ -0,0 +1 @@ +1293872461 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a new file mode 100644 index 0000000000000..502f94a71edbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a @@ -0,0 +1 @@ +1.29387251E9 diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 b/sql/hive/src/test/resources/golden/timestamp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 new file mode 100644 index 0000000000000..e6bfe0b1667ae --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_lazy-3-79e0c72c4fb3b259dfbffd245ccaa636 @@ -0,0 +1,5 @@ +2011-01-01 01:01:01 165 val_165 +2011-01-01 01:01:01 238 val_238 +2011-01-01 01:01:01 27 val_27 +2011-01-01 01:01:01 311 val_311 +2011-01-01 01:01:01 86 val_86 diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 b/sql/hive/src/test/resources/golden/timestamp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 new file mode 100644 index 0000000000000..e6bfe0b1667ae --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_lazy-4-b4c4417ce9f08baeb82ffde6ef1baa25 @@ -0,0 +1,5 @@ +2011-01-01 01:01:01 165 val_165 +2011-01-01 01:01:01 238 val_238 +2011-01-01 01:01:01 27 val_27 +2011-01-01 01:01:01 311 val_311 +2011-01-01 01:01:01 86 val_86 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb b/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb new file mode 100644 index 0000000000000..9913d42ffc1aa --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb @@ -0,0 +1 @@ +unix_timestamp([date[, pattern]]) - Returns the UNIX timestamp diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 new file mode 100644 index 0000000000000..ef4aa8e9595d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 @@ -0,0 +1,2 @@ +unix_timestamp([date[, pattern]]) - Returns the UNIX timestamp +Converts the current or specified time to number of seconds since 1970-01-01. diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 new file mode 100644 index 0000000000000..31aaa952b4cc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 @@ -0,0 +1 @@ +2009-03-20 11:30:01 1237573801 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 new file mode 100644 index 0000000000000..6d9ee690cea4f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 @@ -0,0 +1 @@ +2009-03-20 1237532400 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 new file mode 100644 index 0000000000000..e1d0cbb6c8495 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 @@ -0,0 +1 @@ +2009 Mar 20 11:30:01 am 1237573801 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d b/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d new file mode 100644 index 0000000000000..6b40e687afd5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d @@ -0,0 +1 @@ +random_string 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 63dbe57c4c772..fd44325925cdd 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.util.TimeZone import org.scalatest.BeforeAndAfter @@ -31,14 +32,20 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { lazy val hiveQueryDir = TestHive.getHiveFile("ql" + File.separator + "src" + File.separator + "test" + File.separator + "queries" + File.separator + "clientpositive") + var originalTimeZone: TimeZone = _ + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) override def beforeAll() { TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + originalTimeZone = TimeZone.getDefault + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) } override def afterAll() { TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) } /** A list of tests deemed out of scope currently and thus completely disregarded. */ @@ -92,8 +99,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_view_translate", "partitions_json", - // Timezone specific test answers. - "udf_unix_timestamp", + // This test is totally fine except that it includes wrong queries and expects errors, but error + // message format in Hive and Spark SQL differ. Should workaround this later. "udf_to_unix_timestamp", // Cant run without local map/reduce. @@ -659,8 +666,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats_publisher_error_1", "subq2", "tablename_with_select", + "timestamp_1", + "timestamp_2", "timestamp_3", "timestamp_comparison", + "timestamp_lazy", "timestamp_null", "timestamp_udf", "touch", @@ -802,6 +812,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_translate", "udf_trim", "udf_ucase", + "udf_unix_timestamp", "udf_upper", "udf_var_pop", "udf_var_samp", From f89cf65d7aced0bb387c05586f9f51cb29865022 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 21 Jul 2014 13:15:46 -0500 Subject: [PATCH 0028/1492] SPARK-1707. Remove unnecessary 3 second sleep in YarnClusterScheduler Author: Sandy Ryza Closes #634 from sryza/sandy-spark-1707 and squashes the following commits: 2f6e358 [Sandy Ryza] Default min registered executors ratio to .8 for YARN 354c630 [Sandy Ryza] Remove outdated comments c744ef3 [Sandy Ryza] Take out waitForInitialAllocations 2a4329b [Sandy Ryza] SPARK-1707. Remove unnecessary 3 second sleep in YarnClusterScheduler --- .../spark/deploy/yarn/ApplicationMaster.scala | 39 ----------------- .../cluster/YarnClientClusterScheduler.scala | 10 ----- .../cluster/YarnClientSchedulerBackend.scala | 5 +++ .../cluster/YarnClusterScheduler.scala | 8 +--- .../cluster/YarnClusterSchedulerBackend.scala | 5 +++ .../spark/deploy/yarn/ApplicationMaster.scala | 43 ------------------- 6 files changed, 11 insertions(+), 99 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 062f946a9fe93..3ec36487dcd26 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 @@ -255,10 +255,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkContext.getConf) } } - } finally { - // in case of exceptions, etc - ensure that count is atleast ALLOCATOR_LOOP_WAIT_COUNT : - // so that the loop (in ApplicationMaster.sparkContextInitialized) breaks - ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } } @@ -277,13 +273,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) - ApplicationMaster.incrementAllocatorLoop(1) Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) } - } finally { - // In case of exceptions, etc - ensure that count is at least ALLOCATOR_LOOP_WAIT_COUNT, - // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. - ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } logInfo("All executors have launched.") @@ -411,24 +402,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } object ApplicationMaster extends Logging { - // Number of times to wait for the allocator loop to complete. - // Each loop iteration waits for 100ms, so maximum of 3 seconds. - // This is to ensure that we have reasonable number of containers before we start // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. - private val ALLOCATOR_LOOP_WAIT_COUNT = 30 private val ALLOCATE_HEARTBEAT_INTERVAL = 100 - def incrementAllocatorLoop(by: Int) { - val count = yarnAllocatorLoop.getAndAdd(by) - if (count >= ALLOCATOR_LOOP_WAIT_COUNT) { - yarnAllocatorLoop.synchronized { - // to wake threads off wait ... - yarnAllocatorLoop.notifyAll() - } - } - } - private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() def register(master: ApplicationMaster) { @@ -437,7 +414,6 @@ object ApplicationMaster extends Logging { val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null /* initialValue */) - val yarnAllocatorLoop: AtomicInteger = new AtomicInteger(0) def sparkContextInitialized(sc: SparkContext): Boolean = { var modified = false @@ -472,21 +448,6 @@ object ApplicationMaster extends Logging { modified } - - /** - * Returns when we've either - * 1) received all the requested executors, - * 2) waited ALLOCATOR_LOOP_WAIT_COUNT * ALLOCATE_HEARTBEAT_INTERVAL ms, - * 3) hit an error that causes us to terminate trying to get containers. - */ - def waitForInitialAllocations() { - yarnAllocatorLoop.synchronized { - while (yarnAllocatorLoop.get() <= ALLOCATOR_LOOP_WAIT_COUNT) { - yarnAllocatorLoop.wait(1000L) - } - } - } - def main(argStrings: Array[String]) { SignalLogger.register(log) val args = new ApplicationMasterArguments(argStrings) 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 15e8c21aa5906..3474112ded5d7 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 @@ -37,14 +37,4 @@ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configur val retval = YarnAllocationHandler.lookupRack(conf, host) if (retval != null) Some(retval) else None } - - 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 1b37c4bb13f49..d8266f7b0c9a7 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 @@ -30,6 +30,11 @@ private[spark] class YarnClientSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with Logging { + if (conf.getOption("spark.scheduler.minRegisteredExecutorsRatio").isEmpty) { + minRegisteredRatio = 0.8 + ready = false + } + var client: Client = null var appId: ApplicationId = null 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 9ee53d797c8ea..9aeca4a637d38 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 @@ -47,14 +47,8 @@ private[spark] class YarnClusterScheduler(sc: SparkContext, conf: Configuration) } override def postStartHook() { - val sparkContextInitialized = ApplicationMaster.sparkContextInitialized(sc) + 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 index a04b08f43cc5a..0ad1794d19538 100644 --- 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 @@ -27,6 +27,11 @@ private[spark] class YarnClusterSchedulerBackend( sc: SparkContext) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { + if (conf.getOption("spark.scheduler.minRegisteredExecutorsRatio").isEmpty) { + minRegisteredRatio = 0.8 + ready = false + } + override def start() { super.start() var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS 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 1a24ec759b546..eaf594c8b49b9 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 @@ -234,10 +234,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkContext.getConf) } } - } finally { - // In case of exceptions, etc - ensure that the loop in - // ApplicationMaster#sparkContextInitialized() breaks. - ApplicationMaster.doneWithInitialAllocations() } } @@ -254,16 +250,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() - if (iters == ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) { - ApplicationMaster.doneWithInitialAllocations() - } Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) iters += 1 } - } finally { - // In case of exceptions, etc - ensure that the loop in - // ApplicationMaster#sparkContextInitialized() breaks. - ApplicationMaster.doneWithInitialAllocations() } logInfo("All executors have launched.") } @@ -365,12 +354,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } object ApplicationMaster extends Logging { - // Number of times to wait for the allocator loop to complete. - // Each loop iteration waits for 100ms, so maximum of 3 seconds. - // This is to ensure that we have reasonable number of containers before we start // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. - private val ALLOCATOR_LOOP_WAIT_COUNT = 30 private val ALLOCATE_HEARTBEAT_INTERVAL = 100 private val applicationMasters = new CopyOnWriteArrayList[ApplicationMaster]() @@ -378,20 +363,6 @@ object ApplicationMaster extends Logging { val sparkContextRef: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) - // Variable used to notify the YarnClusterScheduler that it should stop waiting - // for the initial set of executors to be started and get on with its business. - val doneWithInitialAllocationsMonitor = new Object() - - @volatile var isDoneWithInitialAllocations = false - - def doneWithInitialAllocations() { - isDoneWithInitialAllocations = true - doneWithInitialAllocationsMonitor.synchronized { - // to wake threads off wait ... - doneWithInitialAllocationsMonitor.notifyAll() - } - } - def register(master: ApplicationMaster) { applicationMasters.add(master) } @@ -434,20 +405,6 @@ object ApplicationMaster extends Logging { modified } - /** - * Returns when we've either - * 1) received all the requested executors, - * 2) waited ALLOCATOR_LOOP_WAIT_COUNT * ALLOCATE_HEARTBEAT_INTERVAL ms, - * 3) hit an error that causes us to terminate trying to get containers. - */ - def waitForInitialAllocations() { - doneWithInitialAllocationsMonitor.synchronized { - while (!isDoneWithInitialAllocations) { - doneWithInitialAllocationsMonitor.wait(1000L) - } - } - } - def getApplicationAttemptId(): ApplicationAttemptId = { val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) val containerId = ConverterUtils.toContainerId(containerIdString) From 872538c600a452ead52638c1ccba90643a9fa41c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 21 Jul 2014 11:59:54 -0700 Subject: [PATCH 0029/1492] [SPARK-2494] [PySpark] make hash of None consistant cross machines In CPython, hash of None is different cross machines, it will cause wrong result during shuffle. This PR will fix this. Author: Davies Liu Closes #1371 from davies/hash_of_none and squashes the following commits: d01745f [Davies Liu] add comments, remove outdated unit tests 5467141 [Davies Liu] disable hijack of hash, use it only for partitionBy() b7118aa [Davies Liu] use __builtin__ instead of __builtins__ 839e417 [Davies Liu] hijack hash to make hash of None consistant cross machines --- python/pyspark/rdd.py | 35 ++++++++++++++++++++++++++++++++--- 1 file changed, 32 insertions(+), 3 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 0c35c666805dd..94ba22306afbd 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -48,6 +48,35 @@ __all__ = ["RDD"] +# TODO: for Python 3.3+, PYTHONHASHSEED should be reset to disable randomized +# hash for string +def portable_hash(x): + """ + This function returns consistant hash code for builtin types, especially + for None and tuple with None. + + The algrithm is similar to that one used by CPython 2.7 + + >>> portable_hash(None) + 0 + >>> portable_hash((None, 1)) + 219750521 + """ + if x is None: + return 0 + if isinstance(x, tuple): + h = 0x345678 + for i in x: + h ^= portable_hash(i) + h *= 1000003 + h &= 0xffffffff + h ^= len(x) + if h == -1: + h = -2 + return h + return hash(x) + + def _extract_concise_traceback(): """ This function returns the traceback info for a callsite, returns a dict @@ -1164,7 +1193,9 @@ def rightOuterJoin(self, other, numPartitions=None): return python_right_outer_join(self, other, numPartitions) # TODO: add option to control map-side combining - def partitionBy(self, numPartitions, partitionFunc=None): + # portable_hash is used as default, because builtin hash of None is different + # cross machines. + def partitionBy(self, numPartitions, partitionFunc=portable_hash): """ Return a copy of the RDD partitioned using the specified partitioner. @@ -1176,8 +1207,6 @@ def partitionBy(self, numPartitions, partitionFunc=None): if numPartitions is None: numPartitions = self._defaultReducePartitions() - if partitionFunc is None: - partitionFunc = lambda x: 0 if x is None else hash(x) # Transferring O(n) objects to Java is too expensive. Instead, we'll # form the hash buckets in Python, transferring O(numPartitions) objects # to Java. Each object is a (splitNumber, [objects]) pair. From abeacffb7bcdfa3eeb1e969aa546029a7b464eaa Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 21 Jul 2014 14:35:15 -0700 Subject: [PATCH 0030/1492] Fix flakey HiveQuerySuite test Result may not be returned in the expected order, so relax that constraint. Author: Aaron Davidson Closes #1514 from aarondav/flakey and squashes the following commits: e5af823 [Aaron Davidson] Fix flakey HiveQuerySuite test --- .../sql/hive/execution/HiveQuerySuite.scala | 45 +++++++++---------- 1 file changed, 22 insertions(+), 23 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index d57e99db1858f..eb7df717284ce 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -421,64 +421,63 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - def rowsToPairs(rows: Array[Row]) = rows.map { case Row(key: String, value: String) => - key -> value - } + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { case Row(key: String, value: String) => key -> value }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. assert(hql("SET").collect().size == 0) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(hql(s"SET $testKey=$testVal").collect()) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(hql("SET").collect()) + assertResult(Set(testKey -> testVal)) { + collectResults(hql("SET")) } hql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - rowsToPairs(hql("SET").collect()) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(hql("SET")) } // "set key" - assertResult(Array(testKey -> testVal)) { - rowsToPairs(hql(s"SET $testKey").collect()) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey")) } - assertResult(Array(nonexistentKey -> "")) { - rowsToPairs(hql(s"SET $nonexistentKey").collect()) + assertResult(Set(nonexistentKey -> "")) { + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as hql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(sql(s"SET $testKey=$testVal").collect()) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(testKey -> testVal)) { - rowsToPairs(sql("SET").collect()) + assertResult(Set(testKey -> testVal)) { + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - rowsToPairs(sql("SET").collect()) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET")) } - assertResult(Array(testKey -> testVal)) { - rowsToPairs(sql(s"SET $testKey").collect()) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } - assertResult(Array(nonexistentKey -> "")) { - rowsToPairs(sql(s"SET $nonexistentKey").collect()) + assertResult(Set(nonexistentKey -> "")) { + collectResults(sql(s"SET $nonexistentKey")) } clear() From a4d60208ec7995146541451849c51670cdc56451 Mon Sep 17 00:00:00 2001 From: Burak Date: Mon, 21 Jul 2014 17:03:40 -0700 Subject: [PATCH 0031/1492] [SPARK-2434][MLlib]: Warning messages that point users to original MLlib implementations added to Examples [SPARK-2434][MLlib]: Warning messages that refer users to the original MLlib implementations of some popular example machine learning algorithms added both in the comments and the code. The following examples have been modified: Scala: * LocalALS * LocalFileLR * LocalKMeans * LocalLP * SparkALS * SparkHdfsLR * SparkKMeans * SparkLR Python: * kmeans.py * als.py * logistic_regression.py Author: Burak Closes #1515 from brkyvz/SPARK-2434 and squashes the following commits: 7505da9 [Burak] [SPARK-2434][MLlib]: Warning messages added, scalastyle errors fixed, and added missing punctuation b96b522 [Burak] [SPARK-2434][MLlib]: Warning messages added and scalastyle errors fixed 4762f39 [Burak] [SPARK-2434]: Warning messages added 17d3d83 [Burak] SPARK-2434: Added warning messages to the naive implementations of the example algorithms 2cb5301 [Burak] SPARK-2434: Warning messages redirecting to original implementaions added. --- examples/src/main/python/als.py | 9 +++++++++ examples/src/main/python/kmeans.py | 6 ++++++ examples/src/main/python/logistic_regression.py | 6 ++++++ .../org/apache/spark/examples/LocalALS.scala | 15 +++++++++++++++ .../org/apache/spark/examples/LocalFileLR.scala | 17 +++++++++++++++++ .../org/apache/spark/examples/LocalKMeans.scala | 14 ++++++++++++++ .../org/apache/spark/examples/LocalLR.scala | 15 ++++++++++++++- .../org/apache/spark/examples/SparkALS.scala | 16 ++++++++++++++++ .../org/apache/spark/examples/SparkHdfsLR.scala | 14 ++++++++++++++ .../org/apache/spark/examples/SparkKMeans.scala | 15 +++++++++++++++ .../org/apache/spark/examples/SparkLR.scala | 15 +++++++++++++++ 11 files changed, 141 insertions(+), 1 deletion(-) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 1a7c4c51f48cd..c862650b0aa1d 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -16,6 +16,9 @@ # """ +This is an example implementation of ALS for learning how to use Spark. Please refer to +ALS in pyspark.mllib.recommendation for more conventional use. + This example requires numpy (http://www.numpy.org/) """ from os.path import realpath @@ -49,9 +52,15 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": + """ Usage: als [M] [U] [F] [iterations] [slices]" """ + + print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an + example. Please use the ALS method found in pyspark.mllib.recommendation for more + conventional use.""" + sc = SparkContext(appName="PythonALS") M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 988fc45baf3bc..036bdf4c4f999 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -45,9 +45,15 @@ def closestPoint(p, centers): if __name__ == "__main__": + if len(sys.argv) != 4: print >> sys.stderr, "Usage: kmeans " exit(-1) + + print >> sys.stderr, """WARN: This is a naive implementation of KMeans Clustering and is given + as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on + how to use MLlib's KMeans implementation.""" + sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector).cache() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 6c33deabfd6ea..8456b272f9c05 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -47,9 +47,15 @@ def readPointBatch(iterator): return [matrix] if __name__ == "__main__": + if len(sys.argv) != 3: print >> sys.stderr, "Usage: logistic_regression " exit(-1) + + print >> sys.stderr, """WARN: This is a naive implementation of Logistic Regression and is + given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py + to see how MLlib's implementation is used.""" + sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() iterations = int(sys.argv[2]) diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 658f73d96a86a..1f576319b3ca8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -25,6 +25,9 @@ import cern.jet.math._ /** * Alternating least squares matrix factorization. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.recommendation.ALS */ object LocalALS { // Parameters set through command line arguments @@ -107,7 +110,16 @@ object LocalALS { solved2D.viewColumn(0) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of ALS and is given as an example! + |Please use the ALS method found in org.apache.spark.mllib.recommendation + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + args match { case Array(m, u, f, iters) => { M = m.toInt @@ -120,6 +132,9 @@ object LocalALS { System.exit(1) } } + + showWarning() + printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index 0ef3001ca4ccd..931faac5463c4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -21,6 +21,12 @@ import java.util.Random import breeze.linalg.{Vector, DenseVector} +/** + * Logistic regression based classification. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression + */ object LocalFileLR { val D = 10 // Numer of dimensions val rand = new Random(42) @@ -32,7 +38,18 @@ object LocalFileLR { DataPoint(new DenseVector(nums.slice(1, D + 1)), nums(0)) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + + showWarning() + val lines = scala.io.Source.fromFile(args(0)).getLines().toArray val points = lines.map(parsePoint _) val ITERATIONS = args(1).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala index e33a1b336d163..17624c20cff3d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalKMeans.scala @@ -28,6 +28,9 @@ import org.apache.spark.SparkContext._ /** * K-means clustering. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.clustering.KMeans */ object LocalKMeans { val N = 1000 @@ -61,7 +64,18 @@ object LocalKMeans { bestIndex } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of KMeans Clustering and is given as an example! + |Please use the KMeans method found in org.apache.spark.mllib.clustering + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + + showWarning() + val data = generateData var points = new HashSet[Vector[Double]] var kPoints = new HashMap[Int, Vector[Double]] diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 385b48089d572..2d75b9d2590f8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -23,6 +23,9 @@ import breeze.linalg.{Vector, DenseVector} /** * Logistic regression based classification. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression */ object LocalLR { val N = 10000 // Number of data points @@ -42,9 +45,19 @@ object LocalLR { Array.tabulate(N)(generatePoint) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { - val data = generateData + showWarning() + + val data = generateData // Initialize w to a random value var w = DenseVector.fill(D){2 * rand.nextDouble - 1} println("Initial w: " + w) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 5cbc966bf06ca..fde8ffeedf8b4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -27,6 +27,9 @@ import org.apache.spark._ /** * Alternating least squares matrix factorization. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.recommendation.ALS */ object SparkALS { // Parameters set through command line arguments @@ -87,7 +90,16 @@ object SparkALS { solved2D.viewColumn(0) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of ALS and is given as an example! + |Please use the ALS method found in org.apache.spark.mllib.recommendation + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + var slices = 0 val options = (0 to 4).map(i => if (i < args.length) Some(args(i)) else None) @@ -103,7 +115,11 @@ object SparkALS { System.err.println("Usage: SparkALS [M] [U] [F] [iters] [slices]") System.exit(1) } + + showWarning() + printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) + val sparkConf = new SparkConf().setAppName("SparkALS") val sc = new SparkContext(sparkConf) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 4906a696e90a7..d583cf421ed23 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -30,6 +30,9 @@ import org.apache.spark.scheduler.InputFormatInfo /** * Logistic regression based classification. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression */ object SparkHdfsLR { val D = 10 // Numer of dimensions @@ -48,12 +51,23 @@ object SparkHdfsLR { DataPoint(new DenseVector(x), y) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + if (args.length < 2) { System.err.println("Usage: SparkHdfsLR ") System.exit(1) } + showWarning() + val sparkConf = new SparkConf().setAppName("SparkHdfsLR") val inputPath = args(0) val conf = SparkHadoopUtil.get.newConfiguration() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 79cfedf332436..48e8d11cdf95b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -24,6 +24,9 @@ import org.apache.spark.SparkContext._ /** * K-means clustering. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.clustering.KMeans */ object SparkKMeans { @@ -46,11 +49,23 @@ object SparkKMeans { bestIndex } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of KMeans Clustering and is given as an example! + |Please use the KMeans method found in org.apache.spark.mllib.clustering + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + if (args.length < 3) { System.err.println("Usage: SparkKMeans ") System.exit(1) } + + showWarning() + val sparkConf = new SparkConf().setAppName("SparkKMeans") val sc = new SparkContext(sparkConf) val lines = sc.textFile(args(0)) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index 99ceb3089e9fe..fc23308fc4adf 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -28,6 +28,9 @@ import org.apache.spark._ /** * Logistic regression based classification. * Usage: SparkLR [slices] + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.mllib.classification.LogisticRegression */ object SparkLR { val N = 10000 // Number of data points @@ -47,7 +50,18 @@ object SparkLR { Array.tabulate(N)(generatePoint) } + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { + + showWarning() + val sparkConf = new SparkConf().setAppName("SparkLR") val sc = new SparkContext(sparkConf) val numSlices = if (args.length > 0) args(0).toInt else 2 @@ -66,6 +80,7 @@ object SparkLR { } println("Final w: " + w) + sc.stop() } } From 511a7314037219c23e824ea5363bf7f1df55bab3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 21 Jul 2014 18:18:17 -0700 Subject: [PATCH 0032/1492] [SPARK-2561][SQL] Fix apply schema We need to use the analyzed attributes otherwise we end up with a tree that will never resolve. Author: Michael Armbrust Closes #1470 from marmbrus/fixApplySchema and squashes the following commits: f968195 [Michael Armbrust] Use analyzed attributes when applying the schema. 4969015 [Michael Armbrust] Add test case. --- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- .../src/test/scala/org/apache/spark/sql/DslQuerySuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 993d085c75089..31d27bb4f0571 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -430,7 +430,7 @@ class SchemaRDD( * @group schema */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { - new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(logicalPlan.output, rdd))) + new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(queryExecution.analyzed.output, rdd))) } // ======================================================================= diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 68dae58728a2a..c8ea01c4e1b6a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -33,6 +33,12 @@ class DslQuerySuite extends QueryTest { testData.collect().toSeq) } + test("repartition") { + checkAnswer( + testData.select('key).repartition(10).select('key), + testData.select('key).collect().toSeq) + } + test("agg") { checkAnswer( testData2.groupBy('a)('a, Sum('b)), From c3462c65684885299cf037d56c88bd53c08c6348 Mon Sep 17 00:00:00 2001 From: Gregory Owen Date: Mon, 21 Jul 2014 18:55:01 -0700 Subject: [PATCH 0033/1492] [SPARK-2086] Improve output of toDebugString to make shuffle boundaries more clear Changes RDD.toDebugString() to show hierarchy and shuffle transformations more clearly New output: ``` (3) FlatMappedValuesRDD[325] at apply at Transformer.scala:22 | MappedValuesRDD[324] at apply at Transformer.scala:22 | CoGroupedRDD[323] at apply at Transformer.scala:22 +-(5) MappedRDD[320] at apply at Transformer.scala:22 | | MappedRDD[319] at apply at Transformer.scala:22 | | MappedValuesRDD[318] at apply at Transformer.scala:22 | | MapPartitionsRDD[317] at apply at Transformer.scala:22 | | ShuffledRDD[316] at apply at Transformer.scala:22 | +-(10) MappedRDD[315] at apply at Transformer.scala:22 | | ParallelCollectionRDD[314] at apply at Transformer.scala:22 +-(100) MappedRDD[322] at apply at Transformer.scala:22 | ParallelCollectionRDD[321] at apply at Transformer.scala:22 ``` Author: Gregory Owen Closes #1364 from GregOwen/to-debug-string and squashes the following commits: 08f5c78 [Gregory Owen] toDebugString: prettier debug printing to show shuffles and joins more clearly 1603f7b [Gregory Owen] toDebugString: prettier debug printing to show shuffles and joins more clearly --- .../main/scala/org/apache/spark/rdd/RDD.scala | 52 +++++++++++++++++-- project/MimaExcludes.scala | 8 +++ 2 files changed, 56 insertions(+), 4 deletions(-) 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 88a918aebf763..a1f2827248891 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1269,11 +1269,55 @@ abstract class RDD[T: ClassTag]( /** A description of this RDD and its recursive dependencies for debugging. */ def toDebugString: String = { - def debugString(rdd: RDD[_], prefix: String = ""): Seq[String] = { - Seq(prefix + rdd + " (" + rdd.partitions.size + " partitions)") ++ - rdd.dependencies.flatMap(d => debugString(d.rdd, prefix + " ")) + // Apply a different rule to the last child + def debugChildren(rdd: RDD[_], prefix: String): Seq[String] = { + val len = rdd.dependencies.length + len match { + case 0 => Seq.empty + case 1 => + val d = rdd.dependencies.head + debugString(d.rdd, prefix, d.isInstanceOf[ShuffleDependency[_,_,_]], true) + case _ => + val frontDeps = rdd.dependencies.take(len - 1) + val frontDepStrings = frontDeps.flatMap( + d => debugString(d.rdd, prefix, d.isInstanceOf[ShuffleDependency[_,_,_]])) + + val lastDep = rdd.dependencies.last + val lastDepStrings = + debugString(lastDep.rdd, prefix, lastDep.isInstanceOf[ShuffleDependency[_,_,_]], true) + + (frontDepStrings ++ lastDepStrings) + } + } + // The first RDD in the dependency stack has no parents, so no need for a +- + def firstDebugString(rdd: RDD[_]): Seq[String] = { + val partitionStr = "(" + rdd.partitions.size + ")" + val leftOffset = (partitionStr.length - 1) / 2 + val nextPrefix = (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset)) + Seq(partitionStr + " " + rdd) ++ debugChildren(rdd, nextPrefix) + } + def shuffleDebugString(rdd: RDD[_], prefix: String = "", isLastChild: Boolean): Seq[String] = { + val partitionStr = "(" + rdd.partitions.size + ")" + val leftOffset = (partitionStr.length - 1) / 2 + val thisPrefix = prefix.replaceAll("\\|\\s+$", "") + val nextPrefix = ( + thisPrefix + + (if (isLastChild) " " else "| ") + + (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset))) + Seq(thisPrefix + "+-" + partitionStr + " " + rdd) ++ debugChildren(rdd, nextPrefix) + } + def debugString(rdd: RDD[_], + prefix: String = "", + isShuffle: Boolean = true, + isLastChild: Boolean = false): Seq[String] = { + if (isShuffle) { + shuffleDebugString(rdd, prefix, isLastChild) + } + else { + Seq(prefix + rdd) ++ debugChildren(rdd, prefix) + } } - debugString(this).mkString("\n") + firstDebugString(this).mkString("\n") } override def toString: String = "%s%s[%d] at %s".format( diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index e0f433b26f7ff..4d86e1a0d8bbf 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -61,6 +61,14 @@ object MimaExcludes { "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$debugChildren$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$firstDebugString$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$shuffleDebugString$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$debugString$1"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$" + "createZero$1") From 5d16d5bbfd242c16ee0d6952c48dcd90651f8ae2 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Mon, 21 Jul 2014 22:30:53 -0700 Subject: [PATCH 0034/1492] [SPARK-2470] PEP8 fixes to PySpark This pull request aims to resolve all outstanding PEP8 violations in PySpark. Author: Nicholas Chammas Author: nchammas Closes #1505 from nchammas/master and squashes the following commits: 98171af [Nicholas Chammas] [SPARK-2470] revert PEP 8 fixes to cloudpickle cba7768 [Nicholas Chammas] [SPARK-2470] wrap expression list in parentheses e178dbe [Nicholas Chammas] [SPARK-2470] style - change position of line break 9127d2b [Nicholas Chammas] [SPARK-2470] wrap expression lists in parentheses 22132a4 [Nicholas Chammas] [SPARK-2470] wrap conditionals in parentheses 24639bc [Nicholas Chammas] [SPARK-2470] fix whitespace for doctest 7d557b7 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to tests.py 8f8e4c0 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to storagelevel.py b3b96cf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to statcounter.py d644477 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to worker.py aa3a7b6 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to sql.py 1916859 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to shell.py 95d1d95 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to serializers.py a0fec2e [Nicholas Chammas] [SPARK-2470] PEP8 fixes to mllib c85e1e5 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to join.py d14f2f1 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to __init__.py 81fcb20 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to resultiterable.py 1bde265 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to java_gateway.py 7fc849c [Nicholas Chammas] [SPARK-2470] PEP8 fixes to daemon.py ca2d28b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to context.py f4e0039 [Nicholas Chammas] [SPARK-2470] PEP8 fixes to conf.py a6d5e4b [Nicholas Chammas] [SPARK-2470] PEP8 fixes to cloudpickle.py f0a7ebf [Nicholas Chammas] [SPARK-2470] PEP8 fixes to rddsampler.py 4dd148f [nchammas] Merge pull request #5 from apache/master f7e4581 [Nicholas Chammas] unrelated pep8 fix a36eed0 [Nicholas Chammas] name ec2 instances and security groups consistently de7292a [nchammas] Merge pull request #4 from apache/master 2e4fe00 [nchammas] Merge pull request #3 from apache/master 89fde08 [nchammas] Merge pull request #2 from apache/master 69f6e22 [Nicholas Chammas] PEP8 fixes 2627247 [Nicholas Chammas] broke up lines before they hit 100 chars 6544b7e [Nicholas Chammas] [SPARK-2065] give launched instances names 69da6cf [nchammas] Merge pull request #1 from apache/master --- python/pyspark/__init__.py | 3 ++- python/pyspark/conf.py | 9 ++++--- python/pyspark/context.py | 45 ++++++++++++++++++-------------- python/pyspark/daemon.py | 12 ++++----- python/pyspark/java_gateway.py | 1 + python/pyspark/join.py | 4 ++- python/pyspark/mllib/_common.py | 4 ++- python/pyspark/mllib/linalg.py | 1 + python/pyspark/mllib/util.py | 2 -- python/pyspark/rddsampler.py | 24 ++++++++--------- python/pyspark/resultiterable.py | 3 +++ python/pyspark/serializers.py | 31 +++++++++++++--------- python/pyspark/shell.py | 3 ++- python/pyspark/sql.py | 38 +++++++++++++++------------ python/pyspark/statcounter.py | 25 +++++++++--------- python/pyspark/storagelevel.py | 5 ++-- python/pyspark/tests.py | 10 ++++--- python/pyspark/worker.py | 4 +-- 18 files changed, 127 insertions(+), 97 deletions(-) diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 07df8697bd1a8..312c75d112cbf 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -59,4 +59,5 @@ from pyspark.storagelevel import StorageLevel -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD", "SparkFiles", "StorageLevel", "Row"] +__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD", + "SparkFiles", "StorageLevel", "Row"] diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index 60fc6ba7c52c2..b50590ab3b444 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -50,7 +50,8 @@ spark.executorEnv.VAR4=value4 spark.home=/path >>> sorted(conf.getAll(), key=lambda p: p[0]) -[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] +[(u'spark.executorEnv.VAR1', u'value1'), (u'spark.executorEnv.VAR3', u'value3'), \ +(u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] """ @@ -118,9 +119,9 @@ def setExecutorEnv(self, key=None, value=None, pairs=None): """Set an environment variable to be passed to executors.""" 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: + elif key is not None: self._jconf.setExecutorEnv(key, value) - elif pairs != None: + elif pairs is not None: for (k, v) in pairs: self._jconf.setExecutorEnv(k, v) return self @@ -137,7 +138,7 @@ def setAll(self, pairs): def get(self, key, defaultValue=None): """Get the configured value for some key, or return a default otherwise.""" - if defaultValue == None: # Py4J doesn't call the right get() if we pass None + if defaultValue is None: # Py4J doesn't call the right get() if we pass None if not self._jconf.contains(key): return None return self._jconf.get(key) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 95c54e7a5ad63..e21be0e10a3f7 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer + PairDeserializer from pyspark.storagelevel import StorageLevel from pyspark import rdd from pyspark.rdd import RDD @@ -50,12 +50,11 @@ class SparkContext(object): _next_accum_id = 0 _active_spark_context = None _lock = Lock() - _python_includes = None # zip and egg files that need to be added to PYTHONPATH - + _python_includes = None # zip and egg files that need to be added to PYTHONPATH def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, - environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, - gateway=None): + environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, + gateway=None): """ Create a new SparkContext. At least the master and app name should be set, either through the named parameters here or through C{conf}. @@ -138,8 +137,8 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, self._accumulatorServer = accumulators._start_update_server() (host, port) = self._accumulatorServer.server_address self._javaAccumulator = self._jsc.accumulator( - self._jvm.java.util.ArrayList(), - self._jvm.PythonAccumulatorParam(host, port)) + self._jvm.java.util.ArrayList(), + self._jvm.PythonAccumulatorParam(host, port)) self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') @@ -165,7 +164,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, (dirname, filename) = os.path.split(path) self._python_includes.append(filename) sys.path.append(path) - if not dirname in sys.path: + if dirname not in sys.path: sys.path.append(dirname) # Create a temporary directory inside spark.local.dir: @@ -192,15 +191,19 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile if instance: - if SparkContext._active_spark_context and SparkContext._active_spark_context != instance: + if (SparkContext._active_spark_context and + SparkContext._active_spark_context != instance): currentMaster = SparkContext._active_spark_context.master currentAppName = SparkContext._active_spark_context.appName callsite = SparkContext._active_spark_context._callsite # Raise error if there is already a running Spark context - raise ValueError("Cannot run multiple SparkContexts at once; existing SparkContext(app=%s, master=%s)" \ - " created by %s at %s:%s " \ - % (currentAppName, currentMaster, callsite.function, callsite.file, callsite.linenum)) + raise ValueError( + "Cannot run multiple SparkContexts at once; " + "existing SparkContext(app=%s, master=%s)" + " created by %s at %s:%s " + % (currentAppName, currentMaster, + callsite.function, callsite.file, callsite.linenum)) else: SparkContext._active_spark_context = instance @@ -290,7 +293,7 @@ def textFile(self, name, minPartitions=None): Read a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings. - + >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: ... testFile.write("Hello world!") @@ -584,11 +587,12 @@ def addPyFile(self, path): HTTP, HTTPS or FTP URI. """ self.addFile(path) - (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix + (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'): self._python_includes.append(filename) - sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) # for tests in local mode + # for tests in local mode + sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) def setCheckpointDir(self, dirName): """ @@ -649,9 +653,9 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): Cancelled If interruptOnCancel is set to true for the job group, then job cancellation will result - in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure - that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, - where HDFS may respond to Thread.interrupt() by marking nodes as dead. + in Thread.interrupt() being called on the job's executor threads. This is useful to help + ensure that the tasks are actually stopped in a timely manner, but is off by default due + to HDFS-1208, where HDFS may respond to Thread.interrupt() by marking nodes as dead. """ self._jsc.setJobGroup(groupId, description, interruptOnCancel) @@ -688,7 +692,7 @@ def cancelAllJobs(self): """ self._jsc.sc().cancelAllJobs() - def runJob(self, rdd, partitionFunc, partitions = None, allowLocal = False): + def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): """ Executes the given partitionFunc on the specified set of partitions, returning the result as an array of elements. @@ -703,7 +707,7 @@ def runJob(self, rdd, partitionFunc, partitions = None, allowLocal = False): >>> sc.runJob(myRDD, lambda part: [x * x for x in part], [0, 2], True) [0, 1, 16, 25] """ - if partitions == None: + if partitions is None: partitions = range(rdd._jrdd.partitions().size()) javaPartitions = ListConverter().convert(partitions, self._gateway._gateway_client) @@ -714,6 +718,7 @@ def runJob(self, rdd, partitionFunc, partitions = None, allowLocal = False): it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal) return list(mappedRDD._collect_iterator_through_file(it)) + def _test(): import atexit import doctest diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 5eb1c63bf206b..8a5873ded2b8b 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -42,12 +42,12 @@ def should_exit(): def compute_real_exit_code(exit_code): - # SystemExit's code can be integer or string, but os._exit only accepts integers - import numbers - if isinstance(exit_code, numbers.Integral): - return exit_code - else: - return 1 + # SystemExit's code can be integer or string, but os._exit only accepts integers + import numbers + if isinstance(exit_code, numbers.Integral): + return exit_code + else: + return 1 def worker(listen_sock): diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2a17127a7e0f9..2c129679f47f3 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -24,6 +24,7 @@ from threading import Thread from py4j.java_gateway import java_import, JavaGateway, GatewayClient + def launch_gateway(): SPARK_HOME = os.environ["SPARK_HOME"] diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 5f3a7e71f7866..b0f1cc1927066 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -33,10 +33,11 @@ from pyspark.resultiterable import ResultIterable + def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x : dispatch(x.__iter__())) + return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x: dispatch(x.__iter__())) def python_join(rdd, other, numPartitions): @@ -85,6 +86,7 @@ def make_mapper(i): vrdds = [rdd.map(make_mapper(i)) for i, rdd in enumerate(rdds)] union_vrdds = reduce(lambda acc, other: acc.union(other), vrdds) rdd_len = len(vrdds) + def dispatch(seq): bufs = [[] for i in range(rdd_len)] for (n, v) in seq: diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index e609b60a0f968..43b491a9716fc 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -164,7 +164,7 @@ def _deserialize_double_vector(ba, offset=0): nb = len(ba) - offset if nb < 5: raise TypeError("_deserialize_double_vector called on a %d-byte array, " - "which is too short" % nb) + "which is too short" % nb) if ba[offset] == DENSE_VECTOR_MAGIC: return _deserialize_dense_vector(ba, offset) elif ba[offset] == SPARSE_VECTOR_MAGIC: @@ -272,6 +272,7 @@ def _serialize_labeled_point(p): header_float[0] = p.label return header + serialized_features + def _deserialize_labeled_point(ba, offset=0): """Deserialize a LabeledPoint from a mutually understood format.""" from pyspark.mllib.regression import LabeledPoint @@ -283,6 +284,7 @@ def _deserialize_labeled_point(ba, offset=0): features = _deserialize_double_vector(ba, offset + 9) return LabeledPoint(label, features) + def _copyto(array, buffer, offset, shape, dtype): """ Copy the contents of a vector to a destination bytearray at the diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index db39ed0acdb66..71f4ad1a8d44e 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -247,6 +247,7 @@ def stringify(vector): else: return "[" + ",".join([str(v) for v in vector]) + "]" + def _test(): import doctest (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index e24c144f458bd..a707a9dcd5b49 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -24,7 +24,6 @@ from pyspark.serializers import NoOpSerializer - class MLUtils: """ Helper methods to load, save and pre-process data used in MLlib. @@ -154,7 +153,6 @@ def saveAsLibSVMFile(data, dir): lines = data.map(lambda p: MLUtils._convert_labeled_point_to_libsvm(p)) lines.saveAsTextFile(dir) - @staticmethod def loadLabeledPoints(sc, path, minPartitions=None): """ diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 122bc38b03b0c..7ff1c316c7623 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -18,13 +18,16 @@ import sys import random + class RDDSampler(object): def __init__(self, withReplacement, fraction, seed=None): try: import numpy self._use_numpy = True except ImportError: - print >> sys.stderr, "NumPy does not appear to be installed. Falling back to default random generator for sampling." + print >> sys.stderr, ( + "NumPy does not appear to be installed. " + "Falling back to default random generator for sampling.") self._use_numpy = False self._seed = seed if seed is not None else random.randint(0, sys.maxint) @@ -61,7 +64,7 @@ def getUniformSample(self, split): def getPoissonSample(self, split, mean): if not self._rand_initialized or split != self._split: self.initRandomGenerator(split) - + if self._use_numpy: return self._random.poisson(mean) else: @@ -80,30 +83,27 @@ def getPoissonSample(self, split, mean): num_arrivals += 1 return (num_arrivals - 1) - + def shuffle(self, vals): if self._random is None: self.initRandomGenerator(0) # this should only ever called on the master so # the split does not matter - + if self._use_numpy: self._random.shuffle(vals) else: self._random.shuffle(vals, self._random.random) def func(self, split, iterator): - if self._withReplacement: + if self._withReplacement: for obj in iterator: - # For large datasets, the expected number of occurrences of each element in a sample with - # replacement is Poisson(frac). We use that to get a count for each element. - count = self.getPoissonSample(split, mean = self._fraction) + # For large datasets, the expected number of occurrences of each element in + # a sample with replacement is Poisson(frac). We use that to get a count for + # each element. + count = self.getPoissonSample(split, mean=self._fraction) for _ in range(0, count): yield obj else: for obj in iterator: if self.getUniformSample(split) <= self._fraction: yield obj - - - - diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index 7f418f8d2e29a..df34740fc8176 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -19,6 +19,7 @@ import collections + class ResultIterable(collections.Iterable): """ A special result iterable. This is used because the standard iterator can not be pickled @@ -27,7 +28,9 @@ def __init__(self, data): self.data = data self.index = 0 self.maxindex = len(data) + def __iter__(self): return iter(self.data) + def __len__(self): return len(self.data) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index b253807974a2e..9be78b39fbc21 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -91,7 +91,6 @@ def load_stream(self, stream): """ raise NotImplementedError - def _load_stream_without_unbatching(self, stream): return self.load_stream(stream) @@ -197,8 +196,8 @@ def _load_stream_without_unbatching(self, stream): return self.serializer.load_stream(stream) def __eq__(self, other): - return isinstance(other, BatchedSerializer) and \ - other.serializer == self.serializer + return (isinstance(other, BatchedSerializer) and + other.serializer == self.serializer) def __str__(self): return "BatchedSerializer<%s>" % str(self.serializer) @@ -229,8 +228,8 @@ def load_stream(self, stream): yield pair def __eq__(self, other): - return isinstance(other, CartesianDeserializer) and \ - self.key_ser == other.key_ser and self.val_ser == other.val_ser + return (isinstance(other, CartesianDeserializer) and + self.key_ser == other.key_ser and self.val_ser == other.val_ser) def __str__(self): return "CartesianDeserializer<%s, %s>" % \ @@ -252,18 +251,20 @@ def load_stream(self, stream): yield pair def __eq__(self, other): - return isinstance(other, PairDeserializer) and \ - self.key_ser == other.key_ser and self.val_ser == other.val_ser + return (isinstance(other, PairDeserializer) and + self.key_ser == other.key_ser and self.val_ser == other.val_ser) def __str__(self): - return "PairDeserializer<%s, %s>" % \ - (str(self.key_ser), str(self.val_ser)) + return "PairDeserializer<%s, %s>" % (str(self.key_ser), str(self.val_ser)) class NoOpSerializer(FramedSerializer): - def loads(self, obj): return obj - def dumps(self, obj): return obj + def loads(self, obj): + return obj + + def dumps(self, obj): + return obj class PickleSerializer(FramedSerializer): @@ -276,12 +277,16 @@ class PickleSerializer(FramedSerializer): not be as fast as more specialized serializers. """ - def dumps(self, obj): return cPickle.dumps(obj, 2) + def dumps(self, obj): + return cPickle.dumps(obj, 2) + loads = cPickle.loads + class CloudPickleSerializer(PickleSerializer): - def dumps(self, obj): return cloudpickle.dumps(obj, 2) + def dumps(self, obj): + return cloudpickle.dumps(obj, 2) class MarshalSerializer(FramedSerializer): diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 2ce5409cd67c2..e1e7cd954189f 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -35,7 +35,8 @@ 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") is not 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"]) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index ffe177576f363..cb83e89176823 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -30,7 +30,7 @@ class SQLContext: tables, execute SQL over tables, cache tables, and read parquet files. """ - def __init__(self, sparkContext, sqlContext = None): + def __init__(self, sparkContext, sqlContext=None): """Create a new SQLContext. @param sparkContext: The SparkContext to wrap. @@ -137,7 +137,6 @@ def parquetFile(self, path): jschema_rdd = self._ssql_ctx.parquetFile(path) return SchemaRDD(jschema_rdd, self) - def jsonFile(self, path): """Loads a text file storing one JSON object per line, returning the result as a L{SchemaRDD}. @@ -234,8 +233,8 @@ def _ssql_ctx(self): self._scala_HiveContext = self._get_hive_ctx() return self._scala_HiveContext except Py4JError as e: - raise Exception("You must build Spark with Hive. Export 'SPARK_HIVE=true' and run " \ - "sbt/sbt assembly" , e) + raise Exception("You must build Spark with Hive. Export 'SPARK_HIVE=true' and run " + "sbt/sbt assembly", e) def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) @@ -377,7 +376,7 @@ def registerAsTable(self, name): """ self._jschema_rdd.registerAsTable(name) - def insertInto(self, tableName, overwrite = False): + def insertInto(self, tableName, overwrite=False): """Inserts the contents of this SchemaRDD into the specified table. Optionally overwriting any existing data. @@ -420,7 +419,7 @@ def _toPython(self): # in Java land in the javaToPython function. May require a custom # pickle serializer in Pyrolite return RDD(jrdd, self._sc, BatchedSerializer( - PickleSerializer())).map(lambda d: Row(d)) + PickleSerializer())).map(lambda d: Row(d)) # We override the default cache/persist/checkpoint behavior as we want to cache the underlying # SchemaRDD object in the JVM, not the PythonRDD checkpointed by the super class @@ -483,6 +482,7 @@ def subtract(self, other, numPartitions=None): else: raise ValueError("Can only subtract another SchemaRDD") + def _test(): import doctest from array import array @@ -493,20 +493,25 @@ def _test(): sc = SparkContext('local[4]', 'PythonTest', batchSize=2) globs['sc'] = sc globs['sqlCtx'] = SQLContext(sc) - globs['rdd'] = sc.parallelize([{"field1" : 1, "field2" : "row1"}, - {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) - jsonStrings = ['{"field1": 1, "field2": "row1", "field3":{"field4":11}}', - '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]}, "field6":[{"field7": "row2"}]}', - '{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}'] + globs['rdd'] = sc.parallelize( + [{"field1": 1, "field2": "row1"}, + {"field1": 2, "field2": "row2"}, + {"field1": 3, "field2": "row3"}] + ) + jsonStrings = [ + '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', + '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]}, "field6":[{"field7": "row2"}]}', + '{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}' + ] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) globs['nestedRdd1'] = sc.parallelize([ - {"f1" : array('i', [1, 2]), "f2" : {"row1" : 1.0}}, - {"f1" : array('i', [2, 3]), "f2" : {"row2" : 2.0}}]) + {"f1": array('i', [1, 2]), "f2": {"row1": 1.0}}, + {"f1": array('i', [2, 3]), "f2": {"row2": 2.0}}]) globs['nestedRdd2'] = sc.parallelize([ - {"f1" : [[1, 2], [2, 3]], "f2" : set([1, 2]), "f3" : (1, 2)}, - {"f1" : [[2, 3], [3, 4]], "f2" : set([2, 3]), "f3" : (2, 3)}]) - (failure_count, test_count) = doctest.testmod(globs=globs,optionflags=doctest.ELLIPSIS) + {"f1": [[1, 2], [2, 3]], "f2": set([1, 2]), "f3": (1, 2)}, + {"f1": [[2, 3], [3, 4]], "f2": set([2, 3]), "f3": (2, 3)}]) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) @@ -514,4 +519,3 @@ def _test(): if __name__ == "__main__": _test() - diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 080325061a697..e287bd3da1f61 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -20,18 +20,19 @@ import copy import math + class StatCounter(object): - + def __init__(self, values=[]): self.n = 0L # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) self.maxValue = float("-inf") self.minValue = float("inf") - + for v in values: self.merge(v) - + # Add a value into this StatCounter, updating the internal statistics. def merge(self, value): delta = value - self.mu @@ -42,7 +43,7 @@ def merge(self, value): self.maxValue = value if self.minValue > value: self.minValue = value - + return self # Merge another StatCounter into this one, adding up the internal statistics. @@ -50,7 +51,7 @@ def mergeStats(self, other): if not isinstance(other, StatCounter): raise Exception("Can only merge Statcounters!") - if other is self: # reference equality holds + if other is self: # reference equality holds self.merge(copy.deepcopy(other)) # Avoid overwriting fields in a weird order else: if self.n == 0: @@ -59,8 +60,8 @@ def mergeStats(self, other): self.n = other.n self.maxValue = other.maxValue self.minValue = other.minValue - - elif other.n != 0: + + elif other.n != 0: delta = other.mu - self.mu if other.n * 10 < self.n: self.mu = self.mu + (delta * other.n) / (self.n + other.n) @@ -68,10 +69,10 @@ def mergeStats(self, other): self.mu = other.mu - (delta * self.n) / (self.n + other.n) else: self.mu = (self.mu * self.n + other.mu * other.n) / (self.n + other.n) - + self.maxValue = max(self.maxValue, other.maxValue) self.minValue = min(self.minValue, other.minValue) - + self.m2 += other.m2 + (delta * delta * self.n * other.n) / (self.n + other.n) self.n += other.n return self @@ -94,7 +95,7 @@ def min(self): def max(self): return self.maxValue - + # Return the variance of the values. def variance(self): if self.n == 0: @@ -124,5 +125,5 @@ def sampleStdev(self): return math.sqrt(self.sampleVariance()) def __repr__(self): - return "(count: %s, mean: %s, stdev: %s, max: %s, min: %s)" % (self.count(), self.mean(), self.stdev(), self.max(), self.min()) - + return ("(count: %s, mean: %s, stdev: %s, max: %s, min: %s)" % + (self.count(), self.mean(), self.stdev(), self.max(), self.min())) diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 3a18ea54eae4c..5d77a131f2856 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -17,6 +17,7 @@ __all__ = ["StorageLevel"] + class StorageLevel: """ Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, @@ -25,7 +26,7 @@ class StorageLevel: Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY. """ - def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication = 1): + def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication=1): self.useDisk = useDisk self.useMemory = useMemory self.useOffHeap = useOffHeap @@ -55,4 +56,4 @@ def __str__(self): StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2) StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False) StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2) -StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1) \ No newline at end of file +StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index c15bb457759ed..9c5ecd0bb02ab 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -52,12 +52,13 @@ class PySparkTestCase(unittest.TestCase): def setUp(self): self._old_sys_path = list(sys.path) class_name = self.__class__.__name__ - self.sc = SparkContext('local[4]', class_name , batchSize=2) + self.sc = SparkContext('local[4]', class_name, batchSize=2) def tearDown(self): self.sc.stop() sys.path = self._old_sys_path + class TestCheckpoint(PySparkTestCase): def setUp(self): @@ -190,6 +191,7 @@ def test_deleting_input_files(self): def testAggregateByKey(self): data = self.sc.parallelize([(1, 1), (1, 1), (3, 2), (5, 1), (5, 3)], 2) + def seqOp(x, y): x.add(y) return x @@ -197,17 +199,19 @@ def seqOp(x, y): def combOp(x, y): x |= y return x - + sets = dict(data.aggregateByKey(set(), seqOp, combOp).collect()) self.assertEqual(3, len(sets)) self.assertEqual(set([1]), sets[1]) self.assertEqual(set([2]), sets[3]) self.assertEqual(set([1, 3]), sets[5]) + class TestIO(PySparkTestCase): def test_stdout_redirection(self): import subprocess + def func(x): subprocess.check_call('ls', shell=True) self.sc.parallelize([1]).foreach(func) @@ -479,7 +483,7 @@ def test_module_dependency(self): | return x + 1 """) proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, script], - stdout=subprocess.PIPE) + stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) self.assertIn("[2, 3, 4]", out) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index f43210c6c0301..24d41b12d1b1a 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -57,8 +57,8 @@ def main(infile, outfile): SparkFiles._is_running_on_worker = True # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH - sys.path.append(spark_files_dir) # *.py files that were added will be copied here - num_python_includes = read_int(infile) + sys.path.append(spark_files_dir) # *.py files that were added will be copied here + num_python_includes = read_int(infile) for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) sys.path.append(os.path.join(spark_files_dir, filename)) From 81fec9922c5a1a44e086fba450c3eea03cddce63 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 22 Jul 2014 00:38:26 -0700 Subject: [PATCH 0035/1492] [SPARK-2452] Create a new valid for each instead of using lineId. Author: Prashant Sharma Closes #1441 from ScrapCodes/SPARK-2452/multi-statement and squashes the following commits: 26c5c72 [Prashant Sharma] Added a test case. 7e8d28d [Prashant Sharma] SPARK-2452, create a new valid for each instead of using lineId, because Line ids can be same sometimes. --- .../scala/org/apache/spark/repl/SparkImports.scala | 8 +++++++- .../test/scala/org/apache/spark/repl/ReplSuite.scala | 11 ++++++++++- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index bce5c74b9d0da..9099e052f5796 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -197,7 +197,7 @@ trait SparkImports { for (imv <- x.definedNames) { if (currentImps contains imv) addWrapper() val objName = req.lineRep.readPath - val valName = "$VAL" + req.lineRep.lineId + val valName = "$VAL" + newValId() if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported code.append("val " + valName + " = " + objName + ".INSTANCE;\n") @@ -222,4 +222,10 @@ trait SparkImports { private def membersAtPickler(sym: Symbol): List[Symbol] = beforePickler(sym.info.nonPrivateMembers.toList) + private var curValId = 0 + + private def newValId(): Int = { + curValId += 1 + curValId + } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index f2aa42dbcb4fc..e2d8d5ff38dbe 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -235,7 +235,7 @@ class ReplSuite extends FunSuite { assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) } - test("SPARK-1199-simple-reproduce") { + test("SPARK-1199 two instances of same class don't type check.") { val output = runInterpreter("local-cluster[1,1,512]", """ |case class Sum(exp: String, exp2: String) @@ -247,6 +247,15 @@ class ReplSuite extends FunSuite { assertDoesNotContain("Exception", output) } + test("SPARK-2452 compound statements.") { + val output = runInterpreter("local", + """ + |val x = 4 ; def f() = x + |f() + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", From 75db1742abf9e08111ddf8f330e6561c5520a86c Mon Sep 17 00:00:00 2001 From: "peng.zhang" Date: Tue, 22 Jul 2014 02:39:07 -0700 Subject: [PATCH 0036/1492] [SPARK-2612] [mllib] Fix data skew in ALS Author: peng.zhang Closes #1521 from renozhang/fix-als and squashes the following commits: b5727a4 [peng.zhang] Remove no need argument 1a4f7a0 [peng.zhang] Fix data skew in ALS --- .../org/apache/spark/mllib/recommendation/ALS.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index cc56fd6ef28d6..15e8855db6ca7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -252,14 +252,14 @@ class ALS private ( val YtY = Some(sc.broadcast(computeYtY(users))) val previousProducts = products products = updateFeatures(numProductBlocks, users, userOutLinks, productInLinks, - userPartitioner, rank, lambda, alpha, YtY) + rank, lambda, alpha, YtY) previousProducts.unpersist() logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) products.setName(s"products-$iter").persist() val XtX = Some(sc.broadcast(computeYtY(products))) val previousUsers = users users = updateFeatures(numUserBlocks, products, productOutLinks, userInLinks, - productPartitioner, rank, lambda, alpha, XtX) + rank, lambda, alpha, XtX) previousUsers.unpersist() } } else { @@ -267,11 +267,11 @@ class ALS private ( // perform ALS update logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) products = updateFeatures(numProductBlocks, users, userOutLinks, productInLinks, - userPartitioner, rank, lambda, alpha, YtY = None) + rank, lambda, alpha, YtY = None) products.setName(s"products-$iter") logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) users = updateFeatures(numUserBlocks, products, productOutLinks, userInLinks, - productPartitioner, rank, lambda, alpha, YtY = None) + rank, lambda, alpha, YtY = None) users.setName(s"users-$iter") } } @@ -464,7 +464,6 @@ class ALS private ( products: RDD[(Int, Array[Array[Double]])], productOutLinks: RDD[(Int, OutLinkBlock)], userInLinks: RDD[(Int, InLinkBlock)], - productPartitioner: Partitioner, rank: Int, lambda: Double, alpha: Double, @@ -477,7 +476,7 @@ class ALS private ( } } toSend.zipWithIndex.map{ case (buf, idx) => (idx, (bid, buf.toArray)) } - }.groupByKey(productPartitioner) + }.groupByKey(new HashPartitioner(numUserBlocks)) .join(userInLinks) .mapValues{ case (messages, inLinkBlock) => updateBlock(messages, inLinkBlock, rank, lambda, alpha, YtY) From 1407871733176c92c67ac547adf603c41a772f7f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 22 Jul 2014 11:45:37 -0700 Subject: [PATCH 0037/1492] [MLLIB] make Mima ignore updateFeatures (private) in ALS Fix Mima issues in #1521. Author: Xiangrui Meng Closes #1533 from mengxr/mima-als and squashes the following commits: 78386e1 [Xiangrui Meng] make Mima ignore updateFeatures (private) in ALS --- project/MimaExcludes.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4d86e1a0d8bbf..5e5ddd227aab6 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -82,7 +82,9 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( // The only public constructor is the one without arguments. "org.apache.spark.mllib.recommendation.ALS.this"), ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7") + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures") ) ++ MimaBuild.excludeSparkClass("mllib.linalg.distributed.ColumnStatisticsAggregator") ++ MimaBuild.excludeSparkClass("rdd.ZippedRDD") ++ From 85d3596e65512d481f4be54df100be6bdc9c8e29 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 22 Jul 2014 11:58:53 -0700 Subject: [PATCH 0038/1492] SPARK-2047: Introduce an in-mem Sorter, and use it to reduce mem usage ### Why and what? Currently, the AppendOnlyMap performs an "in-place" sort by converting its array of [key, value, key, value] pairs into a an array of [(key, value), (key, value)] pairs. However, this causes us to allocate many Tuple2 objects, which come at a nontrivial overhead. This patch adds a Sorter API, intended for in memory sorts, which simply ports the Android Timsort implementation (available under Apache v2) and abstracts the interface in a way which introduces no more than 1 virtual function invocation of overhead at each abstraction point. Please compare our port of the Android Timsort sort with the original implementation: http://www.diffchecker.com/wiwrykcl ### Memory implications An AppendOnlyMap contains N kv pairs, which results in roughly 2N elements within its underlying array. Each of these elements is 4 bytes wide in a [compressed OOPS](https://wikis.oracle.com/display/HotSpotInternals/CompressedOops) system, which is the default. Today's approach immediately allocates N Tuple2 objects, which take up 24N bytes in total (exposed via YourKit), and undergoes a Java sort. The Java 6 version immediately copies the entire array (4N bytes here), while the Java 7 version has a worst-case allocation of half the array (2N bytes). This results in a worst-case sorting overhead of 24N + 2N = 26N bytes (for Java 7). The Sorter does not require allocating any tuples, but since it uses Timsort, it may copy up to half the entire array in the worst case. This results in a worst-case sorting overhead of 4N bytes. Thus, we have reduced the worst-case overhead of the sort by roughly 22 bytes times the number of elements. ### Performance implications As the destructiveSortedIterator is used for spilling in an ExternalAppendOnlyMap, the purpose of this patch is to provide stability by reducing memory usage rather than improve performance. However, because it implements Timsort, it also brings a substantial performance boost over our prior implementation. Here are the results of a microbenchmark that sorted 25 million, randomly distributed (Float, Int) pairs. The Java Arrays.sort() tests were run **only on the keys**, and thus moved less data. Our current implementation is called "Tuple-sort using Arrays.sort()" while the new implementation is "KV-array using Sorter".
Property NameDefaultMeaning
spark.deploy.retainedApplications200 + The maximum number of completed applications to display. Older applications will be dropped from the UI to maintain this limit.
+
spark.deploy.retainedDrivers200 + The maximum number of completed drivers to display. Older drivers will be dropped from the UI to maintain this limit.
+
spark.deploy.spreadOut true
TestFirst run (JDK6)Average of 10 (JDK6)First run (JDK7)Average of 10 (JDK7)
primitive Arrays.sort()3216 ms1190 ms2724 ms131 ms (!!)
Arrays.sort()18564 ms2006 ms13201 ms878 ms
Tuple-sort using Arrays.sort()31813 ms3550 ms20990 ms1919 ms
KV-array using Sorter15020 ms834 ms
The results show that this Sorter performs exactly as expected (after the first run) -- it is as fast as the Java 7 Arrays.sort() (which shares the same algorithm), but is significantly faster than the Tuple-sort on Java 6 or 7. In short, this patch should significantly improve performance for users running either Java 6 or 7. Author: Aaron Davidson Closes #1502 from aarondav/sort and squashes the following commits: 652d936 [Aaron Davidson] Update license, move Sorter to java src a7b5b1c [Aaron Davidson] fix licenses 5c0efaf [Aaron Davidson] Update tmpLength ec395c8 [Aaron Davidson] Ignore benchmark (again) and fix docs 034bf10 [Aaron Davidson] Change to Apache v2 Timsort b97296c [Aaron Davidson] Don't try to run benchmark on Jenkins + private[spark] 6307338 [Aaron Davidson] SPARK-2047: Introduce an in-mem Sorter, and use it to reduce mem usage --- LICENSE | 20 +- .../apache/spark/util/collection/Sorter.java | 915 ++++++++++++++++++ .../spark/util/collection/AppendOnlyMap.scala | 15 +- .../collection/ExternalAppendOnlyMap.scala | 35 +- .../util/collection/SortDataFormat.scala | 94 ++ .../util/collection/AppendOnlyMapSuite.scala | 8 +- .../spark/util/collection/SorterSuite.scala | 167 ++++ 7 files changed, 1222 insertions(+), 32 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/util/collection/Sorter.java create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala diff --git a/LICENSE b/LICENSE index 383f079df8c8b..65e1f480d9b14 100644 --- a/LICENSE +++ b/LICENSE @@ -442,7 +442,7 @@ Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, An ======================================================================== -Fo SnapTree: +For SnapTree: ======================================================================== SNAPTREE LICENSE @@ -482,6 +482,24 @@ OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +======================================================================== +For Timsort (core/src/main/java/org/apache/spark/util/collection/Sorter.java): +======================================================================== +Copyright (C) 2008 The Android Open Source Project + +Licensed 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. + + ======================================================================== BSD-style licenses ======================================================================== diff --git a/core/src/main/java/org/apache/spark/util/collection/Sorter.java b/core/src/main/java/org/apache/spark/util/collection/Sorter.java new file mode 100644 index 0000000000000..64ad18c0e463a --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/collection/Sorter.java @@ -0,0 +1,915 @@ +/* + * 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.util.collection; + +import java.util.Comparator; + +/** + * A port of the Android Timsort class, which utilizes a "stable, adaptive, iterative mergesort." + * See the method comment on sort() for more details. + * + * This has been kept in Java with the original style in order to match very closely with the + * Anroid source code, and thus be easy to verify correctness. + * + * The purpose of the port is to generalize the interface to the sort to accept input data formats + * besides simple arrays where every element is sorted individually. For instance, the AppendOnlyMap + * uses this to sort an Array with alternating elements of the form [key, value, key, value]. + * This generalization comes with minimal overhead -- see SortDataFormat for more information. + */ +class Sorter { + + /** + * This is the minimum sized sequence that will be merged. Shorter + * sequences will be lengthened by calling binarySort. If the entire + * array is less than this length, no merges will be performed. + * + * This constant should be a power of two. It was 64 in Tim Peter's C + * implementation, but 32 was empirically determined to work better in + * this implementation. In the unlikely event that you set this constant + * to be a number that's not a power of two, you'll need to change the + * minRunLength computation. + * + * If you decrease this constant, you must change the stackLen + * computation in the TimSort constructor, or you risk an + * ArrayOutOfBounds exception. See listsort.txt for a discussion + * of the minimum stack length required as a function of the length + * of the array being sorted and the minimum merge sequence length. + */ + private static final int MIN_MERGE = 32; + + private final SortDataFormat s; + + public Sorter(SortDataFormat sortDataFormat) { + this.s = sortDataFormat; + } + + /** + * A stable, adaptive, iterative mergesort that requires far fewer than + * n lg(n) comparisons when running on partially sorted arrays, while + * offering performance comparable to a traditional mergesort when run + * on random arrays. Like all proper mergesorts, this sort is stable and + * runs O(n log n) time (worst case). In the worst case, this sort requires + * temporary storage space for n/2 object references; in the best case, + * it requires only a small constant amount of space. + * + * This implementation was adapted from Tim Peters's list sort for + * Python, which is described in detail here: + * + * http://svn.python.org/projects/python/trunk/Objects/listsort.txt + * + * Tim's C code may be found here: + * + * http://svn.python.org/projects/python/trunk/Objects/listobject.c + * + * The underlying techniques are described in this paper (and may have + * even earlier origins): + * + * "Optimistic Sorting and Information Theoretic Complexity" + * Peter McIlroy + * SODA (Fourth Annual ACM-SIAM Symposium on Discrete Algorithms), + * pp 467-474, Austin, Texas, 25-27 January 1993. + * + * While the API to this class consists solely of static methods, it is + * (privately) instantiable; a TimSort instance holds the state of an ongoing + * sort, assuming the input array is large enough to warrant the full-blown + * TimSort. Small arrays are sorted in place, using a binary insertion sort. + * + * @author Josh Bloch + */ + void sort(Buffer a, int lo, int hi, Comparator c) { + assert c != null; + + int nRemaining = hi - lo; + if (nRemaining < 2) + return; // Arrays of size 0 and 1 are always sorted + + // If array is small, do a "mini-TimSort" with no merges + if (nRemaining < MIN_MERGE) { + int initRunLen = countRunAndMakeAscending(a, lo, hi, c); + binarySort(a, lo, hi, lo + initRunLen, c); + return; + } + + /** + * March over the array once, left to right, finding natural runs, + * extending short natural runs to minRun elements, and merging runs + * to maintain stack invariant. + */ + SortState sortState = new SortState(a, c, hi - lo); + int minRun = minRunLength(nRemaining); + do { + // Identify next run + int runLen = countRunAndMakeAscending(a, lo, hi, c); + + // If run is short, extend to min(minRun, nRemaining) + if (runLen < minRun) { + int force = nRemaining <= minRun ? nRemaining : minRun; + binarySort(a, lo, lo + force, lo + runLen, c); + runLen = force; + } + + // Push run onto pending-run stack, and maybe merge + sortState.pushRun(lo, runLen); + sortState.mergeCollapse(); + + // Advance to find next run + lo += runLen; + nRemaining -= runLen; + } while (nRemaining != 0); + + // Merge all remaining runs to complete sort + assert lo == hi; + sortState.mergeForceCollapse(); + assert sortState.stackSize == 1; + } + + /** + * Sorts the specified portion of the specified array using a binary + * insertion sort. This is the best method for sorting small numbers + * of elements. It requires O(n log n) compares, but O(n^2) data + * movement (worst case). + * + * If the initial part of the specified range is already sorted, + * this method can take advantage of it: the method assumes that the + * elements from index {@code lo}, inclusive, to {@code start}, + * exclusive are already sorted. + * + * @param a the array in which a range is to be sorted + * @param lo the index of the first element in the range to be sorted + * @param hi the index after the last element in the range to be sorted + * @param start the index of the first element in the range that is + * not already known to be sorted ({@code lo <= start <= hi}) + * @param c comparator to used for the sort + */ + @SuppressWarnings("fallthrough") + private void binarySort(Buffer a, int lo, int hi, int start, Comparator c) { + assert lo <= start && start <= hi; + if (start == lo) + start++; + + Buffer pivotStore = s.allocate(1); + for ( ; start < hi; start++) { + s.copyElement(a, start, pivotStore, 0); + K pivot = s.getKey(pivotStore, 0); + + // Set left (and right) to the index where a[start] (pivot) belongs + int left = lo; + int right = start; + assert left <= right; + /* + * Invariants: + * pivot >= all in [lo, left). + * pivot < all in [right, start). + */ + while (left < right) { + int mid = (left + right) >>> 1; + if (c.compare(pivot, s.getKey(a, mid)) < 0) + right = mid; + else + left = mid + 1; + } + assert left == right; + + /* + * The invariants still hold: pivot >= all in [lo, left) and + * pivot < all in [left, start), so pivot belongs at left. Note + * that if there are elements equal to pivot, left points to the + * first slot after them -- that's why this sort is stable. + * Slide elements over to make room for pivot. + */ + int n = start - left; // The number of elements to move + // Switch is just an optimization for arraycopy in default case + switch (n) { + case 2: s.copyElement(a, left + 1, a, left + 2); + case 1: s.copyElement(a, left, a, left + 1); + break; + default: s.copyRange(a, left, a, left + 1, n); + } + s.copyElement(pivotStore, 0, a, left); + } + } + + /** + * Returns the length of the run beginning at the specified position in + * the specified array and reverses the run if it is descending (ensuring + * that the run will always be ascending when the method returns). + * + * A run is the longest ascending sequence with: + * + * a[lo] <= a[lo + 1] <= a[lo + 2] <= ... + * + * or the longest descending sequence with: + * + * a[lo] > a[lo + 1] > a[lo + 2] > ... + * + * For its intended use in a stable mergesort, the strictness of the + * definition of "descending" is needed so that the call can safely + * reverse a descending sequence without violating stability. + * + * @param a the array in which a run is to be counted and possibly reversed + * @param lo index of the first element in the run + * @param hi index after the last element that may be contained in the run. + It is required that {@code lo < hi}. + * @param c the comparator to used for the sort + * @return the length of the run beginning at the specified position in + * the specified array + */ + private int countRunAndMakeAscending(Buffer a, int lo, int hi, Comparator c) { + assert lo < hi; + int runHi = lo + 1; + if (runHi == hi) + return 1; + + // Find end of run, and reverse range if descending + if (c.compare(s.getKey(a, runHi++), s.getKey(a, lo)) < 0) { // Descending + while (runHi < hi && c.compare(s.getKey(a, runHi), s.getKey(a, runHi - 1)) < 0) + runHi++; + reverseRange(a, lo, runHi); + } else { // Ascending + while (runHi < hi && c.compare(s.getKey(a, runHi), s.getKey(a, runHi - 1)) >= 0) + runHi++; + } + + return runHi - lo; + } + + /** + * Reverse the specified range of the specified array. + * + * @param a the array in which a range is to be reversed + * @param lo the index of the first element in the range to be reversed + * @param hi the index after the last element in the range to be reversed + */ + private void reverseRange(Buffer a, int lo, int hi) { + hi--; + while (lo < hi) { + s.swap(a, lo, hi); + lo++; + hi--; + } + } + + /** + * Returns the minimum acceptable run length for an array of the specified + * length. Natural runs shorter than this will be extended with + * {@link #binarySort}. + * + * Roughly speaking, the computation is: + * + * If n < MIN_MERGE, return n (it's too small to bother with fancy stuff). + * Else if n is an exact power of 2, return MIN_MERGE/2. + * Else return an int k, MIN_MERGE/2 <= k <= MIN_MERGE, such that n/k + * is close to, but strictly less than, an exact power of 2. + * + * For the rationale, see listsort.txt. + * + * @param n the length of the array to be sorted + * @return the length of the minimum run to be merged + */ + private int minRunLength(int n) { + assert n >= 0; + int r = 0; // Becomes 1 if any 1 bits are shifted off + while (n >= MIN_MERGE) { + r |= (n & 1); + n >>= 1; + } + return n + r; + } + + private class SortState { + + /** + * The Buffer being sorted. + */ + private final Buffer a; + + /** + * Length of the sort Buffer. + */ + private final int aLength; + + /** + * The comparator for this sort. + */ + private final Comparator c; + + /** + * When we get into galloping mode, we stay there until both runs win less + * often than MIN_GALLOP consecutive times. + */ + private static final int MIN_GALLOP = 7; + + /** + * This controls when we get *into* galloping mode. It is initialized + * to MIN_GALLOP. The mergeLo and mergeHi methods nudge it higher for + * random data, and lower for highly structured data. + */ + private int minGallop = MIN_GALLOP; + + /** + * Maximum initial size of tmp array, which is used for merging. The array + * can grow to accommodate demand. + * + * Unlike Tim's original C version, we do not allocate this much storage + * when sorting smaller arrays. This change was required for performance. + */ + private static final int INITIAL_TMP_STORAGE_LENGTH = 256; + + /** + * Temp storage for merges. + */ + private Buffer tmp; // Actual runtime type will be Object[], regardless of T + + /** + * Length of the temp storage. + */ + private int tmpLength = 0; + + /** + * A stack of pending runs yet to be merged. Run i starts at + * address base[i] and extends for len[i] elements. It's always + * true (so long as the indices are in bounds) that: + * + * runBase[i] + runLen[i] == runBase[i + 1] + * + * so we could cut the storage for this, but it's a minor amount, + * and keeping all the info explicit simplifies the code. + */ + private int stackSize = 0; // Number of pending runs on stack + private final int[] runBase; + private final int[] runLen; + + /** + * Creates a TimSort instance to maintain the state of an ongoing sort. + * + * @param a the array to be sorted + * @param c the comparator to determine the order of the sort + */ + private SortState(Buffer a, Comparator c, int len) { + this.aLength = len; + this.a = a; + this.c = c; + + // Allocate temp storage (which may be increased later if necessary) + tmpLength = len < 2 * INITIAL_TMP_STORAGE_LENGTH ? len >>> 1 : INITIAL_TMP_STORAGE_LENGTH; + tmp = s.allocate(tmpLength); + + /* + * Allocate runs-to-be-merged stack (which cannot be expanded). The + * stack length requirements are described in listsort.txt. The C + * version always uses the same stack length (85), but this was + * measured to be too expensive when sorting "mid-sized" arrays (e.g., + * 100 elements) in Java. Therefore, we use smaller (but sufficiently + * large) stack lengths for smaller arrays. The "magic numbers" in the + * computation below must be changed if MIN_MERGE is decreased. See + * the MIN_MERGE declaration above for more information. + */ + int stackLen = (len < 120 ? 5 : + len < 1542 ? 10 : + len < 119151 ? 19 : 40); + runBase = new int[stackLen]; + runLen = new int[stackLen]; + } + + /** + * Pushes the specified run onto the pending-run stack. + * + * @param runBase index of the first element in the run + * @param runLen the number of elements in the run + */ + private void pushRun(int runBase, int runLen) { + this.runBase[stackSize] = runBase; + this.runLen[stackSize] = runLen; + stackSize++; + } + + /** + * Examines the stack of runs waiting to be merged and merges adjacent runs + * until the stack invariants are reestablished: + * + * 1. runLen[i - 3] > runLen[i - 2] + runLen[i - 1] + * 2. runLen[i - 2] > runLen[i - 1] + * + * This method is called each time a new run is pushed onto the stack, + * so the invariants are guaranteed to hold for i < stackSize upon + * entry to the method. + */ + private void mergeCollapse() { + while (stackSize > 1) { + int n = stackSize - 2; + if (n > 0 && runLen[n-1] <= runLen[n] + runLen[n+1]) { + if (runLen[n - 1] < runLen[n + 1]) + n--; + mergeAt(n); + } else if (runLen[n] <= runLen[n + 1]) { + mergeAt(n); + } else { + break; // Invariant is established + } + } + } + + /** + * Merges all runs on the stack until only one remains. This method is + * called once, to complete the sort. + */ + private void mergeForceCollapse() { + while (stackSize > 1) { + int n = stackSize - 2; + if (n > 0 && runLen[n - 1] < runLen[n + 1]) + n--; + mergeAt(n); + } + } + + /** + * Merges the two runs at stack indices i and i+1. Run i must be + * the penultimate or antepenultimate run on the stack. In other words, + * i must be equal to stackSize-2 or stackSize-3. + * + * @param i stack index of the first of the two runs to merge + */ + private void mergeAt(int i) { + assert stackSize >= 2; + assert i >= 0; + assert i == stackSize - 2 || i == stackSize - 3; + + int base1 = runBase[i]; + int len1 = runLen[i]; + int base2 = runBase[i + 1]; + int len2 = runLen[i + 1]; + assert len1 > 0 && len2 > 0; + assert base1 + len1 == base2; + + /* + * Record the length of the combined runs; if i is the 3rd-last + * run now, also slide over the last run (which isn't involved + * in this merge). The current run (i+1) goes away in any case. + */ + runLen[i] = len1 + len2; + if (i == stackSize - 3) { + runBase[i + 1] = runBase[i + 2]; + runLen[i + 1] = runLen[i + 2]; + } + stackSize--; + + /* + * Find where the first element of run2 goes in run1. Prior elements + * in run1 can be ignored (because they're already in place). + */ + int k = gallopRight(s.getKey(a, base2), a, base1, len1, 0, c); + assert k >= 0; + base1 += k; + len1 -= k; + if (len1 == 0) + return; + + /* + * Find where the last element of run1 goes in run2. Subsequent elements + * in run2 can be ignored (because they're already in place). + */ + len2 = gallopLeft(s.getKey(a, base1 + len1 - 1), a, base2, len2, len2 - 1, c); + assert len2 >= 0; + if (len2 == 0) + return; + + // Merge remaining runs, using tmp array with min(len1, len2) elements + if (len1 <= len2) + mergeLo(base1, len1, base2, len2); + else + mergeHi(base1, len1, base2, len2); + } + + /** + * Locates the position at which to insert the specified key into the + * specified sorted range; if the range contains an element equal to key, + * returns the index of the leftmost equal element. + * + * @param key the key whose insertion point to search for + * @param a the array in which to search + * @param base the index of the first element in the range + * @param len the length of the range; must be > 0 + * @param hint the index at which to begin the search, 0 <= hint < n. + * The closer hint is to the result, the faster this method will run. + * @param c the comparator used to order the range, and to search + * @return the int k, 0 <= k <= n such that a[b + k - 1] < key <= a[b + k], + * pretending that a[b - 1] is minus infinity and a[b + n] is infinity. + * In other words, key belongs at index b + k; or in other words, + * the first k elements of a should precede key, and the last n - k + * should follow it. + */ + private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator c) { + assert len > 0 && hint >= 0 && hint < len; + int lastOfs = 0; + int ofs = 1; + if (c.compare(key, s.getKey(a, base + hint)) > 0) { + // Gallop right until a[base+hint+lastOfs] < key <= a[base+hint+ofs] + int maxOfs = len - hint; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) > 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to base + lastOfs += hint; + ofs += hint; + } else { // key <= a[base + hint] + // Gallop left until a[base+hint-ofs] < key <= a[base+hint-lastOfs] + final int maxOfs = hint + 1; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) <= 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to base + int tmp = lastOfs; + lastOfs = hint - ofs; + ofs = hint - tmp; + } + assert -1 <= lastOfs && lastOfs < ofs && ofs <= len; + + /* + * Now a[base+lastOfs] < key <= a[base+ofs], so key belongs somewhere + * to the right of lastOfs but no farther right than ofs. Do a binary + * search, with invariant a[base + lastOfs - 1] < key <= a[base + ofs]. + */ + lastOfs++; + while (lastOfs < ofs) { + int m = lastOfs + ((ofs - lastOfs) >>> 1); + + if (c.compare(key, s.getKey(a, base + m)) > 0) + lastOfs = m + 1; // a[base + m] < key + else + ofs = m; // key <= a[base + m] + } + assert lastOfs == ofs; // so a[base + ofs - 1] < key <= a[base + ofs] + return ofs; + } + + /** + * Like gallopLeft, except that if the range contains an element equal to + * key, gallopRight returns the index after the rightmost equal element. + * + * @param key the key whose insertion point to search for + * @param a the array in which to search + * @param base the index of the first element in the range + * @param len the length of the range; must be > 0 + * @param hint the index at which to begin the search, 0 <= hint < n. + * The closer hint is to the result, the faster this method will run. + * @param c the comparator used to order the range, and to search + * @return the int k, 0 <= k <= n such that a[b + k - 1] <= key < a[b + k] + */ + private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator c) { + assert len > 0 && hint >= 0 && hint < len; + + int ofs = 1; + int lastOfs = 0; + if (c.compare(key, s.getKey(a, base + hint)) < 0) { + // Gallop left until a[b+hint - ofs] <= key < a[b+hint - lastOfs] + int maxOfs = hint + 1; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) < 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to b + int tmp = lastOfs; + lastOfs = hint - ofs; + ofs = hint - tmp; + } else { // a[b + hint] <= key + // Gallop right until a[b+hint + lastOfs] <= key < a[b+hint + ofs] + int maxOfs = len - hint; + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) >= 0) { + lastOfs = ofs; + ofs = (ofs << 1) + 1; + if (ofs <= 0) // int overflow + ofs = maxOfs; + } + if (ofs > maxOfs) + ofs = maxOfs; + + // Make offsets relative to b + lastOfs += hint; + ofs += hint; + } + assert -1 <= lastOfs && lastOfs < ofs && ofs <= len; + + /* + * Now a[b + lastOfs] <= key < a[b + ofs], so key belongs somewhere to + * the right of lastOfs but no farther right than ofs. Do a binary + * search, with invariant a[b + lastOfs - 1] <= key < a[b + ofs]. + */ + lastOfs++; + while (lastOfs < ofs) { + int m = lastOfs + ((ofs - lastOfs) >>> 1); + + if (c.compare(key, s.getKey(a, base + m)) < 0) + ofs = m; // key < a[b + m] + else + lastOfs = m + 1; // a[b + m] <= key + } + assert lastOfs == ofs; // so a[b + ofs - 1] <= key < a[b + ofs] + return ofs; + } + + /** + * Merges two adjacent runs in place, in a stable fashion. The first + * element of the first run must be greater than the first element of the + * second run (a[base1] > a[base2]), and the last element of the first run + * (a[base1 + len1-1]) must be greater than all elements of the second run. + * + * For performance, this method should be called only when len1 <= len2; + * its twin, mergeHi should be called if len1 >= len2. (Either method + * may be called if len1 == len2.) + * + * @param base1 index of first element in first run to be merged + * @param len1 length of first run to be merged (must be > 0) + * @param base2 index of first element in second run to be merged + * (must be aBase + aLen) + * @param len2 length of second run to be merged (must be > 0) + */ + private void mergeLo(int base1, int len1, int base2, int len2) { + assert len1 > 0 && len2 > 0 && base1 + len1 == base2; + + // Copy first run into temp array + Buffer a = this.a; // For performance + Buffer tmp = ensureCapacity(len1); + s.copyRange(a, base1, tmp, 0, len1); + + int cursor1 = 0; // Indexes into tmp array + int cursor2 = base2; // Indexes int a + int dest = base1; // Indexes int a + + // Move first element of second run and deal with degenerate cases + s.copyElement(a, cursor2++, a, dest++); + if (--len2 == 0) { + s.copyRange(tmp, cursor1, a, dest, len1); + return; + } + if (len1 == 1) { + s.copyRange(a, cursor2, a, dest, len2); + s.copyElement(tmp, cursor1, a, dest + len2); // Last elt of run 1 to end of merge + return; + } + + Comparator c = this.c; // Use local variable for performance + int minGallop = this.minGallop; // " " " " " + outer: + while (true) { + int count1 = 0; // Number of times in a row that first run won + int count2 = 0; // Number of times in a row that second run won + + /* + * Do the straightforward thing until (if ever) one run starts + * winning consistently. + */ + do { + assert len1 > 1 && len2 > 0; + if (c.compare(s.getKey(a, cursor2), s.getKey(tmp, cursor1)) < 0) { + s.copyElement(a, cursor2++, a, dest++); + count2++; + count1 = 0; + if (--len2 == 0) + break outer; + } else { + s.copyElement(tmp, cursor1++, a, dest++); + count1++; + count2 = 0; + if (--len1 == 1) + break outer; + } + } while ((count1 | count2) < minGallop); + + /* + * One run is winning so consistently that galloping may be a + * huge win. So try that, and continue galloping until (if ever) + * neither run appears to be winning consistently anymore. + */ + do { + assert len1 > 1 && len2 > 0; + count1 = gallopRight(s.getKey(a, cursor2), tmp, cursor1, len1, 0, c); + if (count1 != 0) { + s.copyRange(tmp, cursor1, a, dest, count1); + dest += count1; + cursor1 += count1; + len1 -= count1; + if (len1 <= 1) // len1 == 1 || len1 == 0 + break outer; + } + s.copyElement(a, cursor2++, a, dest++); + if (--len2 == 0) + break outer; + + count2 = gallopLeft(s.getKey(tmp, cursor1), a, cursor2, len2, 0, c); + if (count2 != 0) { + s.copyRange(a, cursor2, a, dest, count2); + dest += count2; + cursor2 += count2; + len2 -= count2; + if (len2 == 0) + break outer; + } + s.copyElement(tmp, cursor1++, a, dest++); + if (--len1 == 1) + break outer; + minGallop--; + } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP); + if (minGallop < 0) + minGallop = 0; + minGallop += 2; // Penalize for leaving gallop mode + } // End of "outer" loop + this.minGallop = minGallop < 1 ? 1 : minGallop; // Write back to field + + if (len1 == 1) { + assert len2 > 0; + s.copyRange(a, cursor2, a, dest, len2); + s.copyElement(tmp, cursor1, a, dest + len2); // Last elt of run 1 to end of merge + } else if (len1 == 0) { + throw new IllegalArgumentException( + "Comparison method violates its general contract!"); + } else { + assert len2 == 0; + assert len1 > 1; + s.copyRange(tmp, cursor1, a, dest, len1); + } + } + + /** + * Like mergeLo, except that this method should be called only if + * len1 >= len2; mergeLo should be called if len1 <= len2. (Either method + * may be called if len1 == len2.) + * + * @param base1 index of first element in first run to be merged + * @param len1 length of first run to be merged (must be > 0) + * @param base2 index of first element in second run to be merged + * (must be aBase + aLen) + * @param len2 length of second run to be merged (must be > 0) + */ + private void mergeHi(int base1, int len1, int base2, int len2) { + assert len1 > 0 && len2 > 0 && base1 + len1 == base2; + + // Copy second run into temp array + Buffer a = this.a; // For performance + Buffer tmp = ensureCapacity(len2); + s.copyRange(a, base2, tmp, 0, len2); + + int cursor1 = base1 + len1 - 1; // Indexes into a + int cursor2 = len2 - 1; // Indexes into tmp array + int dest = base2 + len2 - 1; // Indexes into a + + // Move last element of first run and deal with degenerate cases + s.copyElement(a, cursor1--, a, dest--); + if (--len1 == 0) { + s.copyRange(tmp, 0, a, dest - (len2 - 1), len2); + return; + } + if (len2 == 1) { + dest -= len1; + cursor1 -= len1; + s.copyRange(a, cursor1 + 1, a, dest + 1, len1); + s.copyElement(tmp, cursor2, a, dest); + return; + } + + Comparator c = this.c; // Use local variable for performance + int minGallop = this.minGallop; // " " " " " + outer: + while (true) { + int count1 = 0; // Number of times in a row that first run won + int count2 = 0; // Number of times in a row that second run won + + /* + * Do the straightforward thing until (if ever) one run + * appears to win consistently. + */ + do { + assert len1 > 0 && len2 > 1; + if (c.compare(s.getKey(tmp, cursor2), s.getKey(a, cursor1)) < 0) { + s.copyElement(a, cursor1--, a, dest--); + count1++; + count2 = 0; + if (--len1 == 0) + break outer; + } else { + s.copyElement(tmp, cursor2--, a, dest--); + count2++; + count1 = 0; + if (--len2 == 1) + break outer; + } + } while ((count1 | count2) < minGallop); + + /* + * One run is winning so consistently that galloping may be a + * huge win. So try that, and continue galloping until (if ever) + * neither run appears to be winning consistently anymore. + */ + do { + assert len1 > 0 && len2 > 1; + count1 = len1 - gallopRight(s.getKey(tmp, cursor2), a, base1, len1, len1 - 1, c); + if (count1 != 0) { + dest -= count1; + cursor1 -= count1; + len1 -= count1; + s.copyRange(a, cursor1 + 1, a, dest + 1, count1); + if (len1 == 0) + break outer; + } + s.copyElement(tmp, cursor2--, a, dest--); + if (--len2 == 1) + break outer; + + count2 = len2 - gallopLeft(s.getKey(a, cursor1), tmp, 0, len2, len2 - 1, c); + if (count2 != 0) { + dest -= count2; + cursor2 -= count2; + len2 -= count2; + s.copyRange(tmp, cursor2 + 1, a, dest + 1, count2); + if (len2 <= 1) // len2 == 1 || len2 == 0 + break outer; + } + s.copyElement(a, cursor1--, a, dest--); + if (--len1 == 0) + break outer; + minGallop--; + } while (count1 >= MIN_GALLOP | count2 >= MIN_GALLOP); + if (minGallop < 0) + minGallop = 0; + minGallop += 2; // Penalize for leaving gallop mode + } // End of "outer" loop + this.minGallop = minGallop < 1 ? 1 : minGallop; // Write back to field + + if (len2 == 1) { + assert len1 > 0; + dest -= len1; + cursor1 -= len1; + s.copyRange(a, cursor1 + 1, a, dest + 1, len1); + s.copyElement(tmp, cursor2, a, dest); // Move first elt of run2 to front of merge + } else if (len2 == 0) { + throw new IllegalArgumentException( + "Comparison method violates its general contract!"); + } else { + assert len1 == 0; + assert len2 > 0; + s.copyRange(tmp, 0, a, dest - (len2 - 1), len2); + } + } + + /** + * Ensures that the external array tmp has at least the specified + * number of elements, increasing its size if necessary. The size + * increases exponentially to ensure amortized linear time complexity. + * + * @param minCapacity the minimum required capacity of the tmp array + * @return tmp, whether or not it grew + */ + private Buffer ensureCapacity(int minCapacity) { + if (tmpLength < minCapacity) { + // Compute smallest power of 2 > minCapacity + int newSize = minCapacity; + newSize |= newSize >> 1; + newSize |= newSize >> 2; + newSize |= newSize >> 4; + newSize |= newSize >> 8; + newSize |= newSize >> 16; + newSize++; + + if (newSize < 0) // Not bloody likely! + newSize = minCapacity; + else + newSize = Math.min(newSize, aLength >>> 1); + + tmp = s.allocate(newSize); + tmpLength = newSize; + } + return tmp; + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 1a6f1c2b55799..290282c9c2e28 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -254,26 +254,21 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) * Return an iterator of the map in sorted order. This provides a way to sort the map without * using additional memory, at the expense of destroying the validity of the map. */ - def destructiveSortedIterator(cmp: Comparator[(K, V)]): Iterator[(K, V)] = { + def destructiveSortedIterator(keyComparator: Comparator[K]): Iterator[(K, V)] = { destroyed = true // Pack KV pairs into the front of the underlying array var keyIndex, newIndex = 0 while (keyIndex < capacity) { if (data(2 * keyIndex) != null) { - data(newIndex) = (data(2 * keyIndex), data(2 * keyIndex + 1)) + data(2 * newIndex) = data(2 * keyIndex) + data(2 * newIndex + 1) = data(2 * keyIndex + 1) newIndex += 1 } keyIndex += 1 } assert(curSize == newIndex + (if (haveNullValue) 1 else 0)) - // Sort by the given ordering - val rawOrdering = new Comparator[AnyRef] { - def compare(x: AnyRef, y: AnyRef): Int = { - cmp.compare(x.asInstanceOf[(K, V)], y.asInstanceOf[(K, V)]) - } - } - Arrays.sort(data, 0, newIndex, rawOrdering) + new Sorter(new KVArraySortDataFormat[K, AnyRef]).sort(data, 0, newIndex, keyComparator) new Iterator[(K, V)] { var i = 0 @@ -284,7 +279,7 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) nullValueReady = false (null.asInstanceOf[K], nullValue) } else { - val item = data(i).asInstanceOf[(K, V)] + val item = (data(2 * i).asInstanceOf[K], data(2 * i + 1).asInstanceOf[V]) i += 1 item } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 765254bf4c36e..71ab2a3e3bef4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -30,6 +30,7 @@ import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} +import org.apache.spark.util.collection.ExternalAppendOnlyMap.HashComparator /** * :: DeveloperApi :: @@ -66,8 +67,6 @@ class ExternalAppendOnlyMap[K, V, C]( blockManager: BlockManager = SparkEnv.get.blockManager) extends Iterable[(K, C)] with Serializable with Logging { - import ExternalAppendOnlyMap._ - private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] private val sparkConf = SparkEnv.get.conf @@ -105,7 +104,7 @@ class ExternalAppendOnlyMap[K, V, C]( private var _diskBytesSpilled = 0L private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 - private val comparator = new KCComparator[K, C] + private val keyComparator = new HashComparator[K] private val ser = serializer.newInstance() /** @@ -173,7 +172,7 @@ class ExternalAppendOnlyMap[K, V, C]( } try { - val it = currentMap.destructiveSortedIterator(comparator) + val it = currentMap.destructiveSortedIterator(keyComparator) while (it.hasNext) { val kv = it.next() writer.write(kv) @@ -231,7 +230,7 @@ class ExternalAppendOnlyMap[K, V, C]( // Input streams are derived both from the in-memory map and spilled maps on disk // The in-memory map is sorted in place, while the spilled maps are already in sorted order - private val sortedMap = currentMap.destructiveSortedIterator(comparator) + private val sortedMap = currentMap.destructiveSortedIterator(keyComparator) private val inputStreams = (Seq(sortedMap) ++ spilledMaps).map(it => it.buffered) inputStreams.foreach { it => @@ -252,7 +251,7 @@ class ExternalAppendOnlyMap[K, V, C]( if (it.hasNext) { var kc = it.next() kcPairs += kc - val minHash = getKeyHashCode(kc) + val minHash = hashKey(kc) while (it.hasNext && it.head._1.hashCode() == minHash) { kc = it.next() kcPairs += kc @@ -298,7 +297,7 @@ class ExternalAppendOnlyMap[K, V, C]( val minPair = minPairs.remove(0) val minKey = minPair._1 var minCombiner = minPair._2 - assert(getKeyHashCode(minPair) == minHash) + assert(hashKey(minPair) == minHash) // For all other streams that may have this key (i.e. have the same minimum key hash), // merge in the corresponding value (if any) from that stream @@ -339,7 +338,7 @@ class ExternalAppendOnlyMap[K, V, C]( // Invalid if there are no more pairs in this stream def minKeyHash: Int = { assert(pairs.length > 0) - getKeyHashCode(pairs.head) + hashKey(pairs.head) } override def compareTo(other: StreamBuffer): Int = { @@ -423,25 +422,27 @@ class ExternalAppendOnlyMap[K, V, C]( file.delete() } } + + /** Convenience function to hash the given (K, C) pair by the key. */ + private def hashKey(kc: (K, C)): Int = ExternalAppendOnlyMap.hash(kc._1) } private[spark] object ExternalAppendOnlyMap { /** - * Return the key hash code of the given (key, combiner) pair. - * If the key is null, return a special hash code. + * Return the hash code of the given object. If the object is null, return a special hash code. */ - private def getKeyHashCode[K, C](kc: (K, C)): Int = { - if (kc._1 == null) 0 else kc._1.hashCode() + private def hash[T](obj: T): Int = { + if (obj == null) 0 else obj.hashCode() } /** - * A comparator for (key, combiner) pairs based on their key hash codes. + * A comparator which sorts arbitrary keys based on their hash codes. */ - private class KCComparator[K, C] extends Comparator[(K, C)] { - def compare(kc1: (K, C), kc2: (K, C)): Int = { - val hash1 = getKeyHashCode(kc1) - val hash2 = getKeyHashCode(kc2) + private class HashComparator[K] extends Comparator[K] { + def compare(key1: K, key2: K): Int = { + val hash1 = hash(key1) + val hash2 = hash(key2) if (hash1 < hash2) -1 else if (hash1 == hash2) 0 else 1 } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala b/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala new file mode 100644 index 0000000000000..ac1528969f0be --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala @@ -0,0 +1,94 @@ +/* + * 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.util.collection + +import scala.reflect.ClassTag + +/** + * Abstraction for sorting an arbitrary input buffer of data. This interface requires determining + * the sort key for a given element index, as well as swapping elements and moving data from one + * buffer to another. + * + * Example format: an array of numbers, where each element is also the key. + * See [[KVArraySortDataFormat]] for a more exciting format. + * + * This trait extends Any to ensure it is universal (and thus compiled to a Java interface). + * + * @tparam K Type of the sort key of each element + * @tparam Buffer Internal data structure used by a particular format (e.g., Array[Int]). + */ +// TODO: Making Buffer a real trait would be a better abstraction, but adds some complexity. +private[spark] trait SortDataFormat[K, Buffer] extends Any { + /** Return the sort key for the element at the given index. */ + protected def getKey(data: Buffer, pos: Int): K + + /** Swap two elements. */ + protected def swap(data: Buffer, pos0: Int, pos1: Int): Unit + + /** Copy a single element from src(srcPos) to dst(dstPos). */ + protected def copyElement(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int): Unit + + /** + * Copy a range of elements starting at src(srcPos) to dst, starting at dstPos. + * Overlapping ranges are allowed. + */ + protected def copyRange(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int, length: Int): Unit + + /** + * Allocates a Buffer that can hold up to 'length' elements. + * All elements of the buffer should be considered invalid until data is explicitly copied in. + */ + protected def allocate(length: Int): Buffer +} + +/** + * Supports sorting an array of key-value pairs where the elements of the array alternate between + * keys and values, as used in [[AppendOnlyMap]]. + * + * @tparam K Type of the sort key of each element + * @tparam T Type of the Array we're sorting. Typically this must extend AnyRef, to support cases + * when the keys and values are not the same type. + */ +private[spark] +class KVArraySortDataFormat[K, T <: AnyRef : ClassTag] extends SortDataFormat[K, Array[T]] { + + override protected def getKey(data: Array[T], pos: Int): K = data(2 * pos).asInstanceOf[K] + + override protected def swap(data: Array[T], pos0: Int, pos1: Int) { + val tmpKey = data(2 * pos0) + val tmpVal = data(2 * pos0 + 1) + data(2 * pos0) = data(2 * pos1) + data(2 * pos0 + 1) = data(2 * pos1 + 1) + data(2 * pos1) = tmpKey + data(2 * pos1 + 1) = tmpVal + } + + override protected def copyElement(src: Array[T], srcPos: Int, dst: Array[T], dstPos: Int) { + dst(2 * dstPos) = src(2 * srcPos) + dst(2 * dstPos + 1) = src(2 * srcPos + 1) + } + + override protected def copyRange(src: Array[T], srcPos: Int, + dst: Array[T], dstPos: Int, length: Int) { + System.arraycopy(src, 2 * srcPos, dst, 2 * dstPos, 2 * length) + } + + override protected def allocate(length: Int): Array[T] = { + new Array[T](2 * length) + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala index 52c7288e18b69..cb99d14b27af4 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/AppendOnlyMapSuite.scala @@ -170,10 +170,10 @@ class AppendOnlyMapSuite extends FunSuite { case e: IllegalStateException => fail() } - val it = map.destructiveSortedIterator(new Comparator[(String, String)] { - def compare(kv1: (String, String), kv2: (String, String)): Int = { - val x = if (kv1 != null && kv1._1 != null) kv1._1.toInt else Int.MinValue - val y = if (kv2 != null && kv2._1 != null) kv2._1.toInt else Int.MinValue + val it = map.destructiveSortedIterator(new Comparator[String] { + def compare(key1: String, key2: String): Int = { + val x = if (key1 != null) key1.toInt else Int.MinValue + val y = if (key2 != null) key2.toInt else Int.MinValue x.compareTo(y) } }) diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala new file mode 100644 index 0000000000000..6fe1079c2719a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -0,0 +1,167 @@ +/* + * 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.util.collection + +import java.lang.{Float => JFloat} +import java.util.{Arrays, Comparator} + +import org.scalatest.FunSuite + +import org.apache.spark.util.random.XORShiftRandom + +class SorterSuite extends FunSuite { + + test("equivalent to Arrays.sort") { + val rand = new XORShiftRandom(123) + val data0 = Array.tabulate[Int](10000) { i => rand.nextInt() } + val data1 = data0.clone() + + Arrays.sort(data0) + new Sorter(new IntArraySortDataFormat).sort(data1, 0, data1.length, Ordering.Int) + + data0.zip(data1).foreach { case (x, y) => assert(x === y) } + } + + test("KVArraySorter") { + val rand = new XORShiftRandom(456) + + // Construct an array of keys (to Java sort) and an array where the keys and values + // alternate. Keys are random doubles, values are ordinals from 0 to length. + val keys = Array.tabulate[Double](5000) { i => rand.nextDouble() } + val keyValueArray = Array.tabulate[Number](10000) { i => + if (i % 2 == 0) keys(i / 2) else new Integer(i / 2) + } + + // Map from generated keys to values, to verify correctness later + val kvMap = + keyValueArray.grouped(2).map { case Array(k, v) => k.doubleValue() -> v.intValue() }.toMap + + Arrays.sort(keys) + new Sorter(new KVArraySortDataFormat[Double, Number]) + .sort(keyValueArray, 0, keys.length, Ordering.Double) + + keys.zipWithIndex.foreach { case (k, i) => + assert(k === keyValueArray(2 * i)) + assert(kvMap(k) === keyValueArray(2 * i + 1)) + } + } + + /** + * This provides a simple benchmark for comparing the Sorter with Java internal sorting. + * Ideally these would be executed one at a time, each in their own JVM, so their listing + * here is mainly to have the code. + * + * The goal of this code is to sort an array of key-value pairs, where the array physically + * has the keys and values alternating. The basic Java sorts work only on the keys, so the + * real Java solution is to make Tuple2s to store the keys and values and sort an array of + * those, while the Sorter approach can work directly on the input data format. + * + * Note that the Java implementation varies tremendously between Java 6 and Java 7, when + * the Java sort changed from merge sort to Timsort. + */ + ignore("Sorter benchmark") { + + /** Runs an experiment several times. */ + def runExperiment(name: String)(f: => Unit): Unit = { + val firstTry = org.apache.spark.util.Utils.timeIt(1)(f) + System.gc() + + var i = 0 + var next10: Long = 0 + while (i < 10) { + val time = org.apache.spark.util.Utils.timeIt(1)(f) + next10 += time + println(s"$name: Took $time ms") + i += 1 + } + + println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") + } + + val numElements = 25000000 // 25 mil + val rand = new XORShiftRandom(123) + + val keys = Array.tabulate[JFloat](numElements) { i => + new JFloat(rand.nextFloat()) + } + + // Test our key-value pairs where each element is a Tuple2[Float, Integer) + val kvTupleArray = Array.tabulate[AnyRef](numElements) { i => + (keys(i / 2): Float, i / 2: Int) + } + runExperiment("Tuple-sort using Arrays.sort()") { + Arrays.sort(kvTupleArray, new Comparator[AnyRef] { + override def compare(x: AnyRef, y: AnyRef): Int = + Ordering.Float.compare(x.asInstanceOf[(Float, _)]._1, y.asInstanceOf[(Float, _)]._1) + }) + } + + // Test our Sorter where each element alternates between Float and Integer, non-primitive + val keyValueArray = Array.tabulate[AnyRef](numElements * 2) { i => + if (i % 2 == 0) keys(i / 2) else new Integer(i / 2) + } + val sorter = new Sorter(new KVArraySortDataFormat[JFloat, AnyRef]) + runExperiment("KV-sort using Sorter") { + sorter.sort(keyValueArray, 0, keys.length, new Comparator[JFloat] { + override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) + }) + } + + // Test non-primitive sort on float array + runExperiment("Java Arrays.sort()") { + Arrays.sort(keys, new Comparator[JFloat] { + override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) + }) + } + + // Test primitive sort on float array + val primitiveKeys = Array.tabulate[Float](numElements) { i => rand.nextFloat() } + runExperiment("Java Arrays.sort() on primitive keys") { + Arrays.sort(primitiveKeys) + } + } +} + + +/** Format to sort a simple Array[Int]. Could be easily generified and specialized. */ +class IntArraySortDataFormat extends SortDataFormat[Int, Array[Int]] { + override protected def getKey(data: Array[Int], pos: Int): Int = { + data(pos) + } + + override protected def swap(data: Array[Int], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override protected def copyElement(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int) { + dst(dstPos) = src(srcPos) + } + + /** Copy a range of elements starting at src(srcPos) to dest, starting at destPos. */ + override protected def copyRange(src: Array[Int], srcPos: Int, + dst: Array[Int], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + /** Allocates a new structure that can hold up to 'length' elements. */ + override protected def allocate(length: Int): Array[Int] = { + new Array[Int](length) + } +} From 79fe7634f6817eb2443bc152c6790a4439721fda Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 22 Jul 2014 18:13:28 -0700 Subject: [PATCH 0039/1492] [SPARK-2615] [SQL] Add Equal Sign "==" Support for HiveQl Currently, the "==" in HiveQL expression will cause exception thrown, this patch will fix it. Author: Cheng Hao Closes #1522 from chenghao-intel/equal and squashes the following commits: f62a0ff [Cheng Hao] Add == Support for HiveQl --- .../main/scala/org/apache/spark/sql/hive/HiveQl.scala | 1 + ...relationoptimizer13-0-efd135a811fa94760736a761d220b82 | 0 ...elationoptimizer13-1-32a82500cc28465fac6f64dde0c431c6 | 0 ...elationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...elationoptimizer13-3-bb61d9292434f37bd386e5bff683764d | 0 ...rrelationoptimizer9-0-efd135a811fa94760736a761d220b82 | 0 ...relationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b | 1 + ...elationoptimizer9-10-1190d82f88f7fb1f91968f6e2e03772a | 0 ...elationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 | 9 +++++++++ ...elationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...elationoptimizer9-13-1190d82f88f7fb1f91968f6e2e03772a | 0 ...elationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 | 9 +++++++++ ...relationoptimizer9-2-32a82500cc28465fac6f64dde0c431c6 | 0 ...relationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f | 1 + ...orrelationoptimizer9-4-ec131bcf578dba99f20b16a7dc6b9b | 0 ...relationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 | 9 +++++++++ ...relationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 | 1 + ...relationoptimizer9-7-f952899d70bd718cbdbc44a5290938c9 | 0 ...relationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 | 9 +++++++++ ...relationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f | 1 + .../sql/hive/execution/HiveCompatibilitySuite.scala | 2 ++ 21 files changed, 45 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer13-0-efd135a811fa94760736a761d220b82 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer13-1-32a82500cc28465fac6f64dde0c431c6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer13-3-bb61d9292434f37bd386e5bff683764d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-0-efd135a811fa94760736a761d220b82 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-10-1190d82f88f7fb1f91968f6e2e03772a create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-13-1190d82f88f7fb1f91968f6e2e03772a create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-2-32a82500cc28465fac6f64dde0c431c6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-4-ec131bcf578dba99f20b16a7dc6b9b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-7-f952899d70bd718cbdbc44a5290938c9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f 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 300e249f5b2e1..53480a521dd14 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 @@ -932,6 +932,7 @@ private[hive] object HiveQl { /* Comparisons */ case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) + case Token("==", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-0-efd135a811fa94760736a761d220b82 b/sql/hive/src/test/resources/golden/correlationoptimizer13-0-efd135a811fa94760736a761d220b82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-1-32a82500cc28465fac6f64dde0c431c6 b/sql/hive/src/test/resources/golden/correlationoptimizer13-1-32a82500cc28465fac6f64dde0c431c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer13-2-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer13-3-bb61d9292434f37bd386e5bff683764d b/sql/hive/src/test/resources/golden/correlationoptimizer13-3-bb61d9292434f37bd386e5bff683764d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-0-efd135a811fa94760736a761d220b82 b/sql/hive/src/test/resources/golden/correlationoptimizer9-0-efd135a811fa94760736a761d220b82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-1-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-10-1190d82f88f7fb1f91968f6e2e03772a b/sql/hive/src/test/resources/golden/correlationoptimizer9-10-1190d82f88f7fb1f91968f6e2e03772a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 b/sql/hive/src/test/resources/golden/correlationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 new file mode 100644 index 0000000000000..17c838bb62b3b --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-11-bc2ae88b17ac2bdbd288e07194a40168 @@ -0,0 +1,9 @@ +103 val_103 103 val_103 4 4 +104 val_104 104 val_104 4 4 +105 val_105 105 val_105 1 1 +111 val_111 111 val_111 1 1 +113 val_113 113 val_113 4 4 +114 val_114 114 val_114 1 1 +116 val_116 116 val_116 1 1 +118 val_118 118 val_118 4 4 +119 val_119 119 val_119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-12-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-13-1190d82f88f7fb1f91968f6e2e03772a b/sql/hive/src/test/resources/golden/correlationoptimizer9-13-1190d82f88f7fb1f91968f6e2e03772a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 b/sql/hive/src/test/resources/golden/correlationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 new file mode 100644 index 0000000000000..17c838bb62b3b --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-14-bc2ae88b17ac2bdbd288e07194a40168 @@ -0,0 +1,9 @@ +103 val_103 103 val_103 4 4 +104 val_104 104 val_104 4 4 +105 val_105 105 val_105 1 1 +111 val_111 111 val_111 1 1 +113 val_113 113 val_113 4 4 +114 val_114 114 val_114 1 1 +116 val_116 116 val_116 1 1 +118 val_118 118 val_118 4 4 +119 val_119 119 val_119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-2-32a82500cc28465fac6f64dde0c431c6 b/sql/hive/src/test/resources/golden/correlationoptimizer9-2-32a82500cc28465fac6f64dde0c431c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-3-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-4-ec131bcf578dba99f20b16a7dc6b9b b/sql/hive/src/test/resources/golden/correlationoptimizer9-4-ec131bcf578dba99f20b16a7dc6b9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 b/sql/hive/src/test/resources/golden/correlationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 new file mode 100644 index 0000000000000..248a14f1f4a9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-5-b4e378104bb5ab8d8ba5f905aa1ff450 @@ -0,0 +1,9 @@ +103 103 4 4 +104 104 4 4 +105 105 1 1 +111 111 1 1 +113 113 4 4 +114 114 1 1 +116 116 1 1 +118 118 4 4 +119 119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-6-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-7-f952899d70bd718cbdbc44a5290938c9 b/sql/hive/src/test/resources/golden/correlationoptimizer9-7-f952899d70bd718cbdbc44a5290938c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 b/sql/hive/src/test/resources/golden/correlationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 new file mode 100644 index 0000000000000..248a14f1f4a9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-8-b4e378104bb5ab8d8ba5f905aa1ff450 @@ -0,0 +1,9 @@ +103 103 4 4 +104 104 4 4 +105 105 1 1 +111 111 1 1 +113 113 4 4 +114 114 1 1 +116 116 1 1 +118 118 4 4 +119 119 9 9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer9-9-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 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 fd44325925cdd..bd036faaa6354 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 @@ -291,6 +291,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "correlationoptimizer1", "correlationoptimizer10", "correlationoptimizer11", + "correlationoptimizer13", "correlationoptimizer14", "correlationoptimizer15", "correlationoptimizer2", @@ -299,6 +300,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "correlationoptimizer6", "correlationoptimizer7", "correlationoptimizer8", + "correlationoptimizer9", "count", "cp_mj_rc", "create_insert_outputformat", From 5f7b99168004f06191dcf121fe985e90bf7dcf74 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Tue, 22 Jul 2014 18:14:44 -0700 Subject: [PATCH 0040/1492] Graphx example fix examples Author: CrazyJvm Closes #1523 from CrazyJvm/graphx-example and squashes the following commits: 663457a [CrazyJvm] outDegrees does not take parameters 7cfff1d [CrazyJvm] fix example for joinVertices --- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 2 +- .../src/main/scala/org/apache/spark/graphx/GraphOps.scala | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 3507f358bfb40..fa4b891754c40 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -344,7 +344,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * * {{{ * val rawGraph: Graph[_, _] = Graph.textFile("webgraph") - * val outDeg: RDD[(VertexId, Int)] = rawGraph.outDegrees() + * val outDeg: RDD[(VertexId, Int)] = rawGraph.outDegrees * val graph = rawGraph.outerJoinVertices(outDeg) { * (vid, data, optDeg) => optDeg.getOrElse(0) * } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index edd5b79da1522..02afaa987d40d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -198,10 +198,10 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * * {{{ * val rawGraph: Graph[Int, Int] = GraphLoader.edgeListFile(sc, "webgraph") - * .mapVertices(v => 0) - * val outDeg: RDD[(Int, Int)] = rawGraph.outDegrees - * val graph = rawGraph.leftJoinVertices[Int,Int](outDeg, - * (v, deg) => deg ) + * .mapVertices((_, _) => 0) + * val outDeg = rawGraph.outDegrees + * val graph = rawGraph.joinVertices[Int](outDeg) + * ((_, _, outDeg) => outDeg) * }}} * */ From ddadf1b00470b9d7bf7386dacf198d41407a0a2b Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Tue, 22 Jul 2014 20:34:40 -0500 Subject: [PATCH 0041/1492] [YARN][SPARK-2606]:In some cases,the spark UI pages display incorrect The issue is caused by #1112 . Author: GuoQiang Li Closes #1501 from witgo/webui_style and squashes the following commits: 4b34998 [GuoQiang Li] In some cases, pages display incorrect in WebUI --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e07aa2ee3a5a2..715cc2f4df8dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -149,7 +149,7 @@ private[spark] object UIUtils extends Logging { def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource - val commonHeaderNodes = { + def commonHeaderNodes = { From 02e45729472a22a31629cdd17dc3836ba6810189 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Tue, 22 Jul 2014 21:05:12 -0500 Subject: [PATCH 0042/1492] [YARN] SPARK-2577: File upload to viewfs is broken due to mount point re... Opting to the option 2 defined in SPARK-2577, i.e., retrieve and pass the correct file system object to addResource. Author: Gera Shegalov Closes #1483 from gerashegalov/master and squashes the following commits: 90c9087 [Gera Shegalov] [YARN] SPARK-2577: File upload to viewfs is broken due to mount point resolution --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 556f49342977a..a1298e8f30b5c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -232,7 +232,8 @@ trait ClientBase extends Logging { if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { val setPermissions = if (destName.equals(ClientBase.APP_JAR)) true else false val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions) - distCacheMgr.addResource(fs, conf, destPath, localResources, LocalResourceType.FILE, + val destFs = FileSystem.get(destPath.toUri(), conf) + distCacheMgr.addResource(destFs, conf, destPath, localResources, LocalResourceType.FILE, destName, statCache) } else if (confKey != null) { sparkConf.set(confKey, localPath) From 6c2be93f081f33e9e97e1231b0084a6a0eb4fa22 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Tue, 22 Jul 2014 22:18:30 -0700 Subject: [PATCH 0043/1492] Remove GraphX MessageToPartition for compatibility with sort-based shuffle MessageToPartition was used in `Graph#partitionBy`. Unlike a Tuple2, it marked the key as transient to avoid sending it over the network. However, it was incompatible with sort-based shuffle (SPARK-2045) and represented only a minor optimization: for partitionBy, it improved performance by 6.3% (30.4 s to 28.5 s) and reduced communication by 5.6% (114.2 MB to 107.8 MB). Author: Ankur Dave Closes #1537 from ankurdave/remove-MessageToPartition and squashes the following commits: f9d0054 [Ankur Dave] Remove MessageToPartition ab71364 [Ankur Dave] Remove unused VertexBroadcastMsg --- .../spark/graphx/GraphKryoRegistrator.scala | 2 - .../apache/spark/graphx/impl/GraphImpl.scala | 7 +- .../graphx/impl/MessageToPartition.scala | 76 ------------------- .../spark/graphx/impl/Serializers.scala | 72 ------------------ .../apache/spark/graphx/SerializerSuite.scala | 73 ------------------ 5 files changed, 2 insertions(+), 228 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index f97f329c0e832..eea9fe9520caa 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -35,8 +35,6 @@ class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { kryo.register(classOf[Edge[Object]]) - kryo.register(classOf[MessageToPartition[Object]]) - kryo.register(classOf[VertexBroadcastMsg[Object]]) kryo.register(classOf[RoutingTableMessage]) kryo.register(classOf[(VertexId, Object)]) kryo.register(classOf[EdgePartition[Object, Object]]) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index ccdaa82eb9162..33f35cfb69a26 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -26,7 +26,6 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl._ -import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.graphx.util.BytecodeUtils @@ -83,15 +82,13 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( val vdTag = classTag[VD] val newEdges = edges.withPartitionsRDD(edges.map { e => val part: PartitionID = partitionStrategy.getPartition(e.srcId, e.dstId, numPartitions) - - // Should we be using 3-tuple or an optimized class - new MessageToPartition(part, (e.srcId, e.dstId, e.attr)) + (part, (e.srcId, e.dstId, e.attr)) } .partitionBy(new HashPartitioner(numPartitions)) .mapPartitionsWithIndex( { (pid, iter) => val builder = new EdgePartitionBuilder[ED, VD]()(edTag, vdTag) iter.foreach { message => - val data = message.data + val data = message._2 builder.add(data._1, data._2, data._3) } val edgePartition = builder.toEdgePartition diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index d85afa45b1264..5318b8da6412a 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -25,82 +25,6 @@ import org.apache.spark.graphx.{PartitionID, VertexId} import org.apache.spark.rdd.{ShuffledRDD, RDD} -private[graphx] -class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( - @transient var partition: PartitionID, - var vid: VertexId, - var data: T) - extends Product2[PartitionID, (VertexId, T)] with Serializable { - - override def _1 = partition - - override def _2 = (vid, data) - - override def canEqual(that: Any): Boolean = that.isInstanceOf[VertexBroadcastMsg[_]] -} - - -/** - * A message used to send a specific value to a partition. - * @param partition index of the target partition. - * @param data value to send - */ -private[graphx] -class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T]( - @transient var partition: PartitionID, - var data: T) - extends Product2[PartitionID, T] with Serializable { - - override def _1 = partition - - override def _2 = data - - override def canEqual(that: Any): Boolean = that.isInstanceOf[MessageToPartition[_]] -} - - -private[graphx] -class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T]]) { - def partitionBy(partitioner: Partitioner): RDD[VertexBroadcastMsg[T]] = { - val rdd = new ShuffledRDD[PartitionID, (VertexId, T), (VertexId, T), VertexBroadcastMsg[T]]( - self, partitioner) - - // Set a custom serializer if the data is of int or double type. - if (classTag[T] == ClassTag.Int) { - rdd.setSerializer(new IntVertexBroadcastMsgSerializer) - } else if (classTag[T] == ClassTag.Long) { - rdd.setSerializer(new LongVertexBroadcastMsgSerializer) - } else if (classTag[T] == ClassTag.Double) { - rdd.setSerializer(new DoubleVertexBroadcastMsgSerializer) - } - rdd - } -} - - -private[graphx] -class MsgRDDFunctions[T: ClassTag](self: RDD[MessageToPartition[T]]) { - - /** - * Return a copy of the RDD partitioned using the specified partitioner. - */ - def partitionBy(partitioner: Partitioner): RDD[MessageToPartition[T]] = { - new ShuffledRDD[PartitionID, T, T, MessageToPartition[T]](self, partitioner) - } - -} - -private[graphx] -object MsgRDDFunctions { - implicit def rdd2PartitionRDDFunctions[T: ClassTag](rdd: RDD[MessageToPartition[T]]) = { - new MsgRDDFunctions(rdd) - } - - implicit def rdd2vertexMessageRDDFunctions[T: ClassTag](rdd: RDD[VertexBroadcastMsg[T]]) = { - new VertexBroadcastMsgRDDFunctions(rdd) - } -} - private[graphx] class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 033237f597216..2d98c24d6970e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -76,78 +76,6 @@ class VertexIdMsgSerializer extends Serializer with Serializable { } } -/** A special shuffle serializer for VertexBroadcastMessage[Int]. */ -private[graphx] -class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[VertexBroadcastMsg[Int]] - writeVarLong(msg.vid, optimizePositive = false) - writeInt(msg.data) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readInt() - new VertexBroadcastMsg[Int](0, a, b).asInstanceOf[T] - } - } - } -} - -/** A special shuffle serializer for VertexBroadcastMessage[Long]. */ -private[graphx] -class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[VertexBroadcastMsg[Long]] - writeVarLong(msg.vid, optimizePositive = false) - writeLong(msg.data) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readLong() - new VertexBroadcastMsg[Long](0, a, b).asInstanceOf[T] - } - } - } -} - -/** A special shuffle serializer for VertexBroadcastMessage[Double]. */ -private[graphx] -class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[VertexBroadcastMsg[Double]] - writeVarLong(msg.vid, optimizePositive = false) - writeDouble(msg.data) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readDouble() - new VertexBroadcastMsg[Double](0, a, b).asInstanceOf[T] - } - } - } -} - /** A special shuffle serializer for AggregationMessage[Int]. */ private[graphx] class IntAggMsgSerializer extends Serializer with Serializable { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala index 91caa6b605a1e..864cb1fdf0022 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -26,75 +26,11 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.graphx.impl._ -import org.apache.spark.graphx.impl.MsgRDDFunctions._ import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { - test("IntVertexBroadcastMsgSerializer") { - val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) - val bout = new ByteArrayOutputStream - val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() - val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) - - intercept[EOFException] { - inStrm.readObject() - } - } - - test("LongVertexBroadcastMsgSerializer") { - val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) - val bout = new ByteArrayOutputStream - val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() - val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) - - intercept[EOFException] { - inStrm.readObject() - } - } - - test("DoubleVertexBroadcastMsgSerializer") { - val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) - val bout = new ByteArrayOutputStream - val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() - val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() - assert(outMsg.vid === inMsg1.vid) - assert(outMsg.vid === inMsg2.vid) - assert(outMsg.data === inMsg1.data) - assert(outMsg.data === inMsg2.data) - - intercept[EOFException] { - inStrm.readObject() - } - } - test("IntAggMsgSerializer") { val outMsg = (4: VertexId, 5) val bout = new ByteArrayOutputStream @@ -152,15 +88,6 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } } - test("TestShuffleVertexBroadcastMsg") { - withSpark { sc => - val bmsgs = sc.parallelize(0 until 100, 10).map { pid => - new VertexBroadcastMsg[Int](pid, pid, pid) - } - bmsgs.partitionBy(new HashPartitioner(3)).collect() - } - } - test("variable long encoding") { def testVarLongEncoding(v: Long, optimizePositive: Boolean) { val bout = new ByteArrayOutputStream From 4c7243e109c713bdfb87891748800109ffbaae07 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 23 Jul 2014 00:58:55 -0700 Subject: [PATCH 0044/1492] [SPARK-2617] Correct doc and usages of preservesPartitioning The name `preservesPartitioning` is ambiguous: 1) preserves the indices of partitions, 2) preserves the partitioner. The latter is correct and `preservesPartitioning` should really be called `preservesPartitioner` to avoid confusion. Unfortunately, this is already part of the API and we cannot change. We should be clear in the doc and fix wrong usages. This PR 1. adds notes in `maPartitions*`, 2. makes `RDD.sample` preserve partitioner, 3. changes `preservesPartitioning` to false in `RDD.zip` because the keys of the first RDD are no longer the keys of the zipped RDD, 4. fixes some wrong usages in MLlib. Author: Xiangrui Meng Closes #1526 from mengxr/preserve-partitioner and squashes the following commits: b361e65 [Xiangrui Meng] update doc based on pwendell's comments 3b1ba19 [Xiangrui Meng] update doc 357575c [Xiangrui Meng] fix unit test 20b4816 [Xiangrui Meng] Merge branch 'master' into preserve-partitioner d1caa65 [Xiangrui Meng] add doc to explain preservesPartitioning fix wrong usage of preservesPartitioning make sample preserse partitioning --- .../spark/rdd/PartitionwiseSampledRDD.scala | 4 ++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 17 +++++++++++++---- .../rdd/PartitionwiseSampledRDDSuite.scala | 4 ++-- .../scala/org/apache/spark/rdd/RDDSuite.scala | 9 +++++++++ .../BinaryClassificationMetrics.scala | 4 ++-- .../mllib/linalg/distributed/RowMatrix.scala | 8 ++++---- .../apache/spark/mllib/recommendation/ALS.scala | 2 +- .../org/apache/spark/mllib/util/MLUtils.scala | 4 ++-- 8 files changed, 37 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index b5b8a5706deb3..a637d6f15b7e5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -39,6 +39,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * * @param prev RDD to be sampled * @param sampler a random sampler + * @param preservesPartitioning whether the sampler preserves the partitioner of the parent RDD * @param seed random seed * @tparam T input RDD item type * @tparam U sampled RDD item type @@ -46,9 +47,12 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], + @transient preservesPartitioning: Boolean, @transient seed: Long = Utils.random.nextLong) extends RDD[U](prev) { + @transient override val partitioner = if (preservesPartitioning) prev.partitioner else None + override def getPartitions: Array[Partition] = { val random = new Random(seed) firstParent[T].partitions.map(x => new PartitionwiseSampledRDDPartition(x, random.nextLong())) 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 a1f2827248891..c1bafab3e7491 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -356,9 +356,9 @@ abstract class RDD[T: ClassTag]( seed: Long = Utils.random.nextLong): RDD[T] = { require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { - new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) + new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) } else { - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), seed) + new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](fraction), true, seed) } } @@ -374,7 +374,7 @@ abstract class RDD[T: ClassTag]( val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](x(0), x(1)), seed) + new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](x(0), x(1)), true, seed) }.toArray } @@ -586,6 +586,9 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD by applying a function to each partition of this RDD. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ def mapPartitions[U: ClassTag]( f: Iterator[T] => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -596,6 +599,9 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD by applying a function to each partition of this RDD, while tracking the index * of the original partition. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ def mapPartitionsWithIndex[U: ClassTag]( f: (Int, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -607,6 +613,9 @@ abstract class RDD[T: ClassTag]( * :: DeveloperApi :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. + * + * `preservesPartitioning` indicates whether the input function preserves the partitioner, which + * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( @@ -689,7 +698,7 @@ abstract class RDD[T: ClassTag]( * a map on the other). */ def zip[U: ClassTag](other: RDD[U]): RDD[(T, U)] = { - zipPartitions(other, true) { (thisIter, otherIter) => + zipPartitions(other, preservesPartitioning = false) { (thisIter, otherIter) => new Iterator[(T, U)] { def hasNext = (thisIter.hasNext, otherIter.hasNext) match { case (true, true) => true diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index 5dd8de319a654..a0483886f8db3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -43,7 +43,7 @@ class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { test("seed distribution") { val rdd = sc.makeRDD(Array(1L, 2L, 3L, 4L), 2) val sampler = new MockSampler - val sample = new PartitionwiseSampledRDD[Long, Long](rdd, sampler, 0L) + val sample = new PartitionwiseSampledRDD[Long, Long](rdd, sampler, false, 0L) assert(sample.distinct().count == 2, "Seeds must be different.") } @@ -52,7 +52,7 @@ class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { // We want to make sure there are no concurrency issues. val rdd = sc.parallelize(0 until 111, 10) for (sampler <- Seq(new BernoulliSampler[Int](0.5), new PoissonSampler[Int](0.5))) { - val sampled = new PartitionwiseSampledRDD[Int, Int](rdd, sampler) + val sampled = new PartitionwiseSampledRDD[Int, Int](rdd, sampler, true) sampled.zip(sampled).count() } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 2924de112934c..6654ec2d7c656 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -523,6 +523,15 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sortedTopK === nums.sorted(ord).take(5)) } + test("sample preserves partitioner") { + val partitioner = new HashPartitioner(2) + val rdd = sc.parallelize(Seq((0, 1), (2, 3))).partitionBy(partitioner) + for (withReplacement <- Seq(true, false)) { + val sampled = rdd.sample(withReplacement, 1.0) + assert(sampled.partitioner === rdd.partitioner) + } + } + test("takeSample") { val n = 1000000 val data = sc.parallelize(1 to n, 2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 079743742d86d..1af40de2c7fcf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -103,11 +103,11 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends mergeValue = (c: BinaryLabelCounter, label: Double) => c += label, mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2 ).sortByKey(ascending = false) - val agg = counts.values.mapPartitions({ iter => + val agg = counts.values.mapPartitions { iter => val agg = new BinaryLabelCounter() iter.foreach(agg += _) Iterator(agg) - }, preservesPartitioning = true).collect() + }.collect() val partitionwiseCumulativeCounts = agg.scanLeft(new BinaryLabelCounter())( (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index f4c403bc7861c..8c2b044ea73f2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -377,9 +377,9 @@ class RowMatrix( s"Only support dense matrix at this time but found ${B.getClass.getName}.") val Bb = rows.context.broadcast(B.toBreeze.asInstanceOf[BDM[Double]].toDenseVector.toArray) - val AB = rows.mapPartitions({ iter => + val AB = rows.mapPartitions { iter => val Bi = Bb.value - iter.map(row => { + iter.map { row => val v = BDV.zeros[Double](k) var i = 0 while (i < k) { @@ -387,8 +387,8 @@ class RowMatrix( i += 1 } Vectors.fromBreeze(v) - }) - }, preservesPartitioning = true) + } + } new RowMatrix(AB, nRows, B.numCols) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 15e8855db6ca7..5356790cb5339 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -430,7 +430,7 @@ class ALS private ( val inLinkBlock = makeInLinkBlock(numProductBlocks, ratings, productPartitioner) val outLinkBlock = makeOutLinkBlock(numProductBlocks, ratings, productPartitioner) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) - }, true) + }, preservesPartitioning = true) val inLinks = links.mapValues(_._1) val outLinks = links.mapValues(_._2) inLinks.persist(StorageLevel.MEMORY_AND_DISK) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index aaf92a1a8869a..30de24ad89f98 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -264,8 +264,8 @@ object MLUtils { (1 to numFolds).map { fold => val sampler = new BernoulliSampler[T]((fold - 1) / numFoldsF, fold / numFoldsF, complement = false) - val validation = new PartitionwiseSampledRDD(rdd, sampler, seed) - val training = new PartitionwiseSampledRDD(rdd, sampler.cloneComplement(), seed) + val validation = new PartitionwiseSampledRDD(rdd, sampler, true, seed) + val training = new PartitionwiseSampledRDD(rdd, sampler.cloneComplement(), true, seed) (training, validation) }.toArray } From 25921110fcd5afe568bf0d25fccd232787af7911 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Jul 2014 10:31:45 -0700 Subject: [PATCH 0045/1492] [SPARK-2609] Log thread ID when spilling ExternalAppendOnlyMap It's useful to know whether one thread is constantly spilling or multiple threads are spilling relatively infrequently. Right now everything looks a little jumbled and we can't tell which lines belong to the same thread. For instance: ``` 06:14:37 ExternalAppendOnlyMap: Spilling in-memory map of 4 MB to disk (194 times so far) 06:14:37 ExternalAppendOnlyMap: Spilling in-memory map of 4 MB to disk (198 times so far) 06:14:37 ExternalAppendOnlyMap: Spilling in-memory map of 4 MB to disk (198 times so far) 06:14:37 ExternalAppendOnlyMap: Spilling in-memory map of 10 MB to disk (197 times so far) 06:14:38 ExternalAppendOnlyMap: Spilling in-memory map of 9 MB to disk (45 times so far) 06:14:38 ExternalAppendOnlyMap: Spilling in-memory map of 23 MB to disk (198 times so far) 06:14:38 ExternalAppendOnlyMap: Spilling in-memory map of 38 MB to disk (25 times so far) 06:14:38 ExternalAppendOnlyMap: Spilling in-memory map of 161 MB to disk (25 times so far) 06:14:39 ExternalAppendOnlyMap: Spilling in-memory map of 0 MB to disk (199 times so far) 06:14:39 ExternalAppendOnlyMap: Spilling in-memory map of 4 MB to disk (166 times so far) 06:14:39 ExternalAppendOnlyMap: Spilling in-memory map of 4 MB to disk (199 times so far) 06:14:39 ExternalAppendOnlyMap: Spilling in-memory map of 4 MB to disk (200 times so far) ``` Author: Andrew Or Closes #1517 from andrewor14/external-log and squashes the following commits: 90e48bb [Andrew Or] Log thread ID when spilling --- .../spark/util/collection/ExternalAppendOnlyMap.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 71ab2a3e3bef4..be8f6529f7a1c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -106,6 +106,7 @@ class ExternalAppendOnlyMap[K, V, C]( private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 private val keyComparator = new HashComparator[K] private val ser = serializer.newInstance() + private val threadId = Thread.currentThread().getId /** * Insert the given key and value into the map. @@ -128,7 +129,6 @@ class ExternalAppendOnlyMap[K, V, C]( // Atomically check whether there is sufficient memory in the global pool for // this map to grow and, if possible, allocate the required amount shuffleMemoryMap.synchronized { - val threadId = Thread.currentThread().getId val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) val availableMemory = maxMemoryThreshold - (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) @@ -153,8 +153,8 @@ class ExternalAppendOnlyMap[K, V, C]( */ private def spill(mapSize: Long) { spillCount += 1 - logWarning("Spilling in-memory map of %d MB to disk (%d time%s so far)" - .format(mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) + logWarning("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" + .format(threadId, mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) var objectsWritten = 0 From f776bc98878428940b5130c0d7d9b7ee452c0bd3 Mon Sep 17 00:00:00 2001 From: woshilaiceshide Date: Wed, 23 Jul 2014 11:05:41 -0700 Subject: [PATCH 0046/1492] [CORE] SPARK-2640: In "local[N]", free cores of the only executor should be touched by "spark.task.cpus" for every finish/start-up of tasks. Make spark's "local[N]" better. In our company, we use "local[N]" in production. It works exellentlly. It's our best choice. Author: woshilaiceshide Closes #1544 from woshilaiceshide/localX and squashes the following commits: 6c85154 [woshilaiceshide] [CORE] SPARK-2640: In "local[N]", free cores of the only executor should be touched by "spark.task.cpus" for every finish/start-up of tasks. --- .../scala/org/apache/spark/scheduler/local/LocalBackend.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index e9f6273bfd9f0..5b897597fa285 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -57,7 +57,7 @@ private[spark] class LocalActor( case StatusUpdate(taskId, state, serializedData) => scheduler.statusUpdate(taskId, state, serializedData) if (TaskState.isFinished(state)) { - freeCores += 1 + freeCores += scheduler.CPUS_PER_TASK reviveOffers() } @@ -68,7 +68,7 @@ private[spark] class LocalActor( def reviveOffers() { val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) for (task <- scheduler.resourceOffers(offers).flatten) { - freeCores -= 1 + freeCores -= scheduler.CPUS_PER_TASK executor.launchTask(executorBackend, task.taskId, task.name, task.serializedTask) } } From 1b790cf7755cace0d89ac5777717e6df3be7356f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Wed, 23 Jul 2014 14:47:23 -0700 Subject: [PATCH 0047/1492] [SPARK-2588][SQL] Add some more DSLs. Author: Takuya UESHIN Closes #1491 from ueshin/issues/SPARK-2588 and squashes the following commits: 43d0a46 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-2588 1023ea0 [Takuya UESHIN] Modify tests to use DSLs. 2310bf1 [Takuya UESHIN] Add some more DSLs. --- .../spark/sql/catalyst/dsl/package.scala | 29 +++++++++ .../ExpressionEvaluationSuite.scala | 59 +++++++++++-------- .../org/apache/spark/sql/DslQuerySuite.scala | 15 +++-- 3 files changed, 70 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 1b503b957d146..15c98efbcabcf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -79,8 +79,24 @@ package object dsl { def === (other: Expression) = EqualTo(expr, other) def !== (other: Expression) = Not(EqualTo(expr, other)) + def in(list: Expression*) = In(expr, list) + def like(other: Expression) = Like(expr, other) def rlike(other: Expression) = RLike(expr, other) + def contains(other: Expression) = Contains(expr, other) + def startsWith(other: Expression) = StartsWith(expr, other) + def endsWith(other: Expression) = EndsWith(expr, other) + def substr(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + Substring(expr, pos, len) + def substring(pos: Expression, len: Expression = Literal(Int.MaxValue)) = + Substring(expr, pos, len) + + def isNull = IsNull(expr) + def isNotNull = IsNotNull(expr) + + def getItem(ordinal: Expression) = GetItem(expr, ordinal) + def getField(fieldName: String) = GetField(expr, fieldName) + def cast(to: DataType) = Cast(expr, to) def asc = SortOrder(expr, Ascending) @@ -112,6 +128,7 @@ package object dsl { def sumDistinct(e: Expression) = SumDistinct(e) def count(e: Expression) = Count(e) def countDistinct(e: Expression*) = CountDistinct(e) + def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd) def avg(e: Expression) = Average(e) def first(e: Expression) = First(e) def min(e: Expression) = Min(e) @@ -163,6 +180,18 @@ package object dsl { /** Creates a new AttributeReference of type binary */ def binary = AttributeReference(s, BinaryType, nullable = true)() + + /** Creates a new AttributeReference of type array */ + def array(dataType: DataType) = AttributeReference(s, ArrayType(dataType), nullable = true)() + + /** Creates a new AttributeReference of type map */ + def map(keyType: DataType, valueType: DataType): AttributeReference = + map(MapType(keyType, valueType)) + def map(mapType: MapType) = AttributeReference(s, mapType, nullable = true)() + + /** Creates a new AttributeReference of type struct */ + def struct(fields: StructField*): AttributeReference = struct(StructType(fields)) + def struct(structType: StructType) = AttributeReference(s, structType, nullable = true)() } implicit class DslAttribute(a: AttributeReference) { 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 db1ae29d400c6..c3f5c26fdbe59 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 @@ -301,17 +301,17 @@ class ExpressionEvaluationSuite extends FunSuite { val c3 = 'a.boolean.at(2) val c4 = 'a.boolean.at(3) - checkEvaluation(IsNull(c1), false, row) - checkEvaluation(IsNotNull(c1), true, row) + checkEvaluation(c1.isNull, false, row) + checkEvaluation(c1.isNotNull, true, row) - checkEvaluation(IsNull(c2), true, row) - checkEvaluation(IsNotNull(c2), false, row) + checkEvaluation(c2.isNull, true, row) + checkEvaluation(c2.isNotNull, false, row) - checkEvaluation(IsNull(Literal(1, ShortType)), false) - checkEvaluation(IsNotNull(Literal(1, ShortType)), true) + checkEvaluation(Literal(1, ShortType).isNull, false) + checkEvaluation(Literal(1, ShortType).isNotNull, true) - checkEvaluation(IsNull(Literal(null, ShortType)), true) - checkEvaluation(IsNotNull(Literal(null, ShortType)), false) + checkEvaluation(Literal(null, ShortType).isNull, true) + checkEvaluation(Literal(null, ShortType).isNotNull, false) checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row) checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row) @@ -326,11 +326,11 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(If(Literal(false, BooleanType), Literal("a", StringType), Literal("b", StringType)), "b", row) - checkEvaluation(In(c1, c1 :: c2 :: Nil), true, row) - checkEvaluation(In(Literal("^Ba*n", StringType), - Literal("^Ba*n", StringType) :: Nil), true, row) - checkEvaluation(In(Literal("^Ba*n", StringType), - Literal("^Ba*n", StringType) :: c2 :: Nil), true, row) + checkEvaluation(c1 in (c1, c2), true, row) + checkEvaluation( + Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType)), true, row) + checkEvaluation( + Literal("^Ba*n", StringType) in (Literal("^Ba*n", StringType), c2), true, row) } test("case when") { @@ -420,6 +420,10 @@ class ExpressionEvaluationSuite extends FunSuite { assert(GetField(Literal(null, typeS), "a").nullable === true) assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) + + checkEvaluation('c.map(typeMap).at(3).getItem("aa"), "bb", row) + checkEvaluation('c.array(typeArray.elementType).at(4).getItem(1), "bb", row) + checkEvaluation('c.struct(typeS).at(2).getField("a"), "aa", row) } test("arithmetic") { @@ -472,20 +476,20 @@ class ExpressionEvaluationSuite extends FunSuite { val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) - checkEvaluation(Contains(c1, "b"), true, row) - checkEvaluation(Contains(c1, "x"), false, row) - checkEvaluation(Contains(c2, "b"), null, row) - checkEvaluation(Contains(c1, Literal(null, StringType)), null, row) + checkEvaluation(c1 contains "b", true, row) + checkEvaluation(c1 contains "x", false, row) + checkEvaluation(c2 contains "b", null, row) + checkEvaluation(c1 contains Literal(null, StringType), null, row) - checkEvaluation(StartsWith(c1, "a"), true, row) - checkEvaluation(StartsWith(c1, "b"), false, row) - checkEvaluation(StartsWith(c2, "a"), null, row) - checkEvaluation(StartsWith(c1, Literal(null, StringType)), null, row) + checkEvaluation(c1 startsWith "a", true, row) + checkEvaluation(c1 startsWith "b", false, row) + checkEvaluation(c2 startsWith "a", null, row) + checkEvaluation(c1 startsWith Literal(null, StringType), null, row) - checkEvaluation(EndsWith(c1, "c"), true, row) - checkEvaluation(EndsWith(c1, "b"), false, row) - checkEvaluation(EndsWith(c2, "b"), null, row) - checkEvaluation(EndsWith(c1, Literal(null, StringType)), null, row) + checkEvaluation(c1 endsWith "c", true, row) + checkEvaluation(c1 endsWith "b", false, row) + checkEvaluation(c2 endsWith "b", null, row) + checkEvaluation(c1 endsWith Literal(null, StringType), null, row) } test("Substring") { @@ -542,5 +546,10 @@ class ExpressionEvaluationSuite extends FunSuite { 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) + + checkEvaluation(s.substr(0, 2), "ex", row) + checkEvaluation(s.substr(0), "example", row) + checkEvaluation(s.substring(0, 2), "ex", row) + checkEvaluation(s.substring(0), "example", row) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index c8ea01c4e1b6a..1a6a6c17473a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test._ /* Implicits */ @@ -41,15 +40,15 @@ class DslQuerySuite extends QueryTest { test("agg") { checkAnswer( - testData2.groupBy('a)('a, Sum('b)), + testData2.groupBy('a)('a, sum('b)), Seq((1,3),(2,3),(3,3)) ) checkAnswer( - testData2.groupBy('a)('a, Sum('b) as 'totB).aggregate(Sum('totB)), + testData2.groupBy('a)('a, sum('b) as 'totB).aggregate(sum('totB)), 9 ) checkAnswer( - testData2.aggregate(Sum('b)), + testData2.aggregate(sum('b)), 9 ) } @@ -104,19 +103,19 @@ class DslQuerySuite extends QueryTest { Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) checkAnswer( - arrayData.orderBy(GetItem('data, 0).asc), + arrayData.orderBy('data.getItem(0).asc), arrayData.collect().sortBy(_.data(0)).toSeq) checkAnswer( - arrayData.orderBy(GetItem('data, 0).desc), + arrayData.orderBy('data.getItem(0).desc), arrayData.collect().sortBy(_.data(0)).reverse.toSeq) checkAnswer( - mapData.orderBy(GetItem('data, 1).asc), + mapData.orderBy('data.getItem(1).asc), mapData.collect().sortBy(_.data(1)).toSeq) checkAnswer( - mapData.orderBy(GetItem('data, 1).desc), + mapData.orderBy('data.getItem(1).desc), mapData.collect().sortBy(_.data(1)).reverse.toSeq) } From 91903e0a50b0efb7217610021a628b3043004d82 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Wed, 23 Jul 2014 16:23:24 -0700 Subject: [PATCH 0048/1492] SPARK-2277: clear host->rack info properly Hi mridulm, I just think of this issue of [#1212](https://github.com/apache/spark/pull/1212): I added FakeRackUtil to hold the host -> rack mapping. It should be cleaned up after use so that it won't mess up with test cases others may add later. Really sorry about this. Author: Rui Li Closes #1454 from lirui-intel/SPARK-2277-fix-UT and squashes the following commits: f8ea25c [Rui Li] SPARK-2277: clear host->rack info properly --- .../scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 86b443b18f2a6..c52368b5514db 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -475,6 +475,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL and ANY assert(manager.myLocalityLevels.sameElements( Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) + FakeRackUtil.cleanUp() } test("test RACK_LOCAL tasks") { @@ -505,6 +506,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host2 // Task 1 can be scheduled with RACK_LOCAL assert(manager.resourceOffer("execB", "host2", RACK_LOCAL).get.index === 1) + FakeRackUtil.cleanUp() } test("do not emit warning when serialized task is small") { From e060d3ee2d910a5a802bb29630dca6f66cc0525d Mon Sep 17 00:00:00 2001 From: William Benton Date: Wed, 23 Jul 2014 16:25:32 -0700 Subject: [PATCH 0049/1492] SPARK-2226: [SQL] transform HAVING clauses with aggregate expressions that aren't in the aggregation list This change adds an analyzer rule to 1. find expressions in `HAVING` clause filters that depend on unresolved attributes, 2. push these expressions down to the underlying aggregates, and then 3. project them away above the filter. It also enables the `HAVING` queries in the Hive compatibility suite. Author: William Benton Closes #1497 from willb/spark-2226 and squashes the following commits: 92c9a93 [William Benton] Removed unnecessary import f1d4f34 [William Benton] Cleanups missed in prior commit 0e1624f [William Benton] Incorporated suggestions from @marmbrus; thanks! 541d4ee [William Benton] Cleanups from review 5a12647 [William Benton] Explanatory comments and stylistic cleanups. c7f2b2c [William Benton] Whitelist HAVING queries. 29a26e3 [William Benton] Added rule to handle unresolved attributes in HAVING clauses (SPARK-2226) --- .../sql/catalyst/analysis/Analyzer.scala | 27 +- .../having-0-57f3f26c0203c29c2a91a7cca557ce55 | 0 .../having-1-ef81808faeab6d212c3cf32abfc0d873 | 10 + .../having-2-a2b4f52cb92f730ddb912b063636d6c1 | 0 .../having-3-3fa6387b6a4ece110ac340c7b893964e | 308 ++++++++++++++++++ .../having-4-e9918bd385cb35db4ebcbd4e398547f4 | 0 .../having-5-4a0c4e521b8a6f6146151c13a2715ff | 199 +++++++++++ .../having-6-9f50df5b5f31c7166b0396ab434dc095 | 0 .../having-7-5ad96cb287df02080da1e2594f08d83e | 125 +++++++ .../having-8-4aa7197e20b5a64461ca670a79488103 | 0 .../having-9-a79743372d86d77b0ff53a71adcb1cff | 199 +++++++++++ .../execution/HiveCompatibilitySuite.scala | 2 + 12 files changed, 869 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/having-0-57f3f26c0203c29c2a91a7cca557ce55 create mode 100644 sql/hive/src/test/resources/golden/having-1-ef81808faeab6d212c3cf32abfc0d873 create mode 100644 sql/hive/src/test/resources/golden/having-2-a2b4f52cb92f730ddb912b063636d6c1 create mode 100644 sql/hive/src/test/resources/golden/having-3-3fa6387b6a4ece110ac340c7b893964e create mode 100644 sql/hive/src/test/resources/golden/having-4-e9918bd385cb35db4ebcbd4e398547f4 create mode 100644 sql/hive/src/test/resources/golden/having-5-4a0c4e521b8a6f6146151c13a2715ff create mode 100644 sql/hive/src/test/resources/golden/having-6-9f50df5b5f31c7166b0396ab434dc095 create mode 100644 sql/hive/src/test/resources/golden/having-7-5ad96cb287df02080da1e2594f08d83e create mode 100644 sql/hive/src/test/resources/golden/having-8-4aa7197e20b5a64461ca670a79488103 create mode 100644 sql/hive/src/test/resources/golden/having-9-a79743372d86d77b0ff53a71adcb1cff diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c7188469bfb86..02bdb64f308a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ - /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing * when all relations are already filled in and the analyser needs only to resolve attribute @@ -54,6 +53,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool StarExpansion :: ResolveFunctions :: GlobalAggregates :: + UnresolvedHavingClauseAttributes :: typeCoercionRules :_*), Batch("Check Analysis", Once, CheckResolution), @@ -151,6 +151,31 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } + /** + * This rule finds expressions in HAVING clause filters that depend on + * unresolved attributes. It pushes these expressions down to the underlying + * aggregates and then projects them away above the filter. + */ + object UnresolvedHavingClauseAttributes extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) + if !filter.resolved && aggregate.resolved && containsAggregate(havingCondition) => { + val evaluatedCondition = Alias(havingCondition, "havingCondition")() + val aggExprsWithHaving = evaluatedCondition +: originalAggExprs + + Project(aggregate.output, + Filter(evaluatedCondition.toAttribute, + aggregate.copy(aggregateExpressions = aggExprsWithHaving))) + } + + } + + protected def containsAggregate(condition: Expression): Boolean = + condition + .collect { case ae: AggregateExpression => ae } + .nonEmpty + } + /** * When a SELECT clause has only a single expression and that expression is a * [[catalyst.expressions.Generator Generator]] we convert the diff --git a/sql/hive/src/test/resources/golden/having-0-57f3f26c0203c29c2a91a7cca557ce55 b/sql/hive/src/test/resources/golden/having-0-57f3f26c0203c29c2a91a7cca557ce55 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-1-ef81808faeab6d212c3cf32abfc0d873 b/sql/hive/src/test/resources/golden/having-1-ef81808faeab6d212c3cf32abfc0d873 new file mode 100644 index 0000000000000..704f1e62f14c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-1-ef81808faeab6d212c3cf32abfc0d873 @@ -0,0 +1,10 @@ +4 +4 +5 +4 +5 +5 +4 +4 +5 +4 diff --git a/sql/hive/src/test/resources/golden/having-2-a2b4f52cb92f730ddb912b063636d6c1 b/sql/hive/src/test/resources/golden/having-2-a2b4f52cb92f730ddb912b063636d6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-3-3fa6387b6a4ece110ac340c7b893964e b/sql/hive/src/test/resources/golden/having-3-3fa6387b6a4ece110ac340c7b893964e new file mode 100644 index 0000000000000..b56757a60f780 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-3-3fa6387b6a4ece110ac340c7b893964e @@ -0,0 +1,308 @@ +0 val_0 +2 val_2 +4 val_4 +5 val_5 +8 val_8 +9 val_9 +10 val_10 +11 val_11 +12 val_12 +15 val_15 +17 val_17 +18 val_18 +19 val_19 +20 val_20 +24 val_24 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +100 val_100 +103 val_103 +104 val_104 +105 val_105 +111 val_111 +113 val_113 +114 val_114 +116 val_116 +118 val_118 +119 val_119 +120 val_120 +125 val_125 +126 val_126 +128 val_128 +129 val_129 +131 val_131 +133 val_133 +134 val_134 +136 val_136 +137 val_137 +138 val_138 +143 val_143 +145 val_145 +146 val_146 +149 val_149 +150 val_150 +152 val_152 +153 val_153 +155 val_155 +156 val_156 +157 val_157 +158 val_158 +160 val_160 +162 val_162 +163 val_163 +164 val_164 +165 val_165 +166 val_166 +167 val_167 +168 val_168 +169 val_169 +170 val_170 +172 val_172 +174 val_174 +175 val_175 +176 val_176 +177 val_177 +178 val_178 +179 val_179 +180 val_180 +181 val_181 +183 val_183 +186 val_186 +187 val_187 +189 val_189 +190 val_190 +191 val_191 +192 val_192 +193 val_193 +194 val_194 +195 val_195 +196 val_196 +197 val_197 +199 val_199 +200 val_200 +201 val_201 +202 val_202 +203 val_203 +205 val_205 +207 val_207 +208 val_208 +209 val_209 +213 val_213 +214 val_214 +216 val_216 +217 val_217 +218 val_218 +219 val_219 +221 val_221 +222 val_222 +223 val_223 +224 val_224 +226 val_226 +228 val_228 +229 val_229 +230 val_230 +233 val_233 +235 val_235 +237 val_237 +238 val_238 +239 val_239 +241 val_241 +242 val_242 +244 val_244 +247 val_247 +248 val_248 +249 val_249 +252 val_252 +255 val_255 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 diff --git a/sql/hive/src/test/resources/golden/having-4-e9918bd385cb35db4ebcbd4e398547f4 b/sql/hive/src/test/resources/golden/having-4-e9918bd385cb35db4ebcbd4e398547f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-5-4a0c4e521b8a6f6146151c13a2715ff b/sql/hive/src/test/resources/golden/having-5-4a0c4e521b8a6f6146151c13a2715ff new file mode 100644 index 0000000000000..2d7022e386303 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-5-4a0c4e521b8a6f6146151c13a2715ff @@ -0,0 +1,199 @@ +4 +5 +8 +9 +26 +27 +28 +30 +33 +34 +35 +37 +41 +42 +43 +44 +47 +51 +53 +54 +57 +58 +64 +65 +66 +67 +69 +70 +72 +74 +76 +77 +78 +80 +82 +83 +84 +85 +86 +87 +90 +92 +95 +96 +97 +98 +256 +257 +258 +260 +262 +263 +265 +266 +272 +273 +274 +275 +277 +278 +280 +281 +282 +283 +284 +285 +286 +287 +288 +289 +291 +292 +296 +298 +302 +305 +306 +307 +308 +309 +310 +311 +315 +316 +317 +318 +321 +322 +323 +325 +327 +331 +332 +333 +335 +336 +338 +339 +341 +342 +344 +345 +348 +351 +353 +356 +360 +362 +364 +365 +366 +367 +368 +369 +373 +374 +375 +377 +378 +379 +382 +384 +386 +389 +392 +393 +394 +395 +396 +397 +399 +400 +401 +402 +403 +404 +406 +407 +409 +411 +413 +414 +417 +418 +419 +421 +424 +427 +429 +430 +431 +432 +435 +436 +437 +438 +439 +443 +444 +446 +448 +449 +452 +453 +454 +455 +457 +458 +459 +460 +462 +463 +466 +467 +468 +469 +470 +472 +475 +477 +478 +479 +480 +481 +482 +483 +484 +485 +487 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 diff --git a/sql/hive/src/test/resources/golden/having-6-9f50df5b5f31c7166b0396ab434dc095 b/sql/hive/src/test/resources/golden/having-6-9f50df5b5f31c7166b0396ab434dc095 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-7-5ad96cb287df02080da1e2594f08d83e b/sql/hive/src/test/resources/golden/having-7-5ad96cb287df02080da1e2594f08d83e new file mode 100644 index 0000000000000..bd545ccf7430c --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-7-5ad96cb287df02080da1e2594f08d83e @@ -0,0 +1,125 @@ +302 +305 +306 +307 +308 +309 +310 +311 +315 +316 +317 +318 +321 +322 +323 +325 +327 +331 +332 +333 +335 +336 +338 +339 +341 +342 +344 +345 +348 +351 +353 +356 +360 +362 +364 +365 +366 +367 +368 +369 +373 +374 +375 +377 +378 +379 +382 +384 +386 +389 +392 +393 +394 +395 +396 +397 +399 +400 +401 +402 +403 +404 +406 +407 +409 +411 +413 +414 +417 +418 +419 +421 +424 +427 +429 +430 +431 +432 +435 +436 +437 +438 +439 +443 +444 +446 +448 +449 +452 +453 +454 +455 +457 +458 +459 +460 +462 +463 +466 +467 +468 +469 +470 +472 +475 +477 +478 +479 +480 +481 +482 +483 +484 +485 +487 +489 +490 +491 +492 +493 +494 +495 +496 +497 +498 diff --git a/sql/hive/src/test/resources/golden/having-8-4aa7197e20b5a64461ca670a79488103 b/sql/hive/src/test/resources/golden/having-8-4aa7197e20b5a64461ca670a79488103 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/having-9-a79743372d86d77b0ff53a71adcb1cff b/sql/hive/src/test/resources/golden/having-9-a79743372d86d77b0ff53a71adcb1cff new file mode 100644 index 0000000000000..d77586c12b6af --- /dev/null +++ b/sql/hive/src/test/resources/golden/having-9-a79743372d86d77b0ff53a71adcb1cff @@ -0,0 +1,199 @@ +4 val_4 +5 val_5 +8 val_8 +9 val_9 +26 val_26 +27 val_27 +28 val_28 +30 val_30 +33 val_33 +34 val_34 +35 val_35 +37 val_37 +41 val_41 +42 val_42 +43 val_43 +44 val_44 +47 val_47 +51 val_51 +53 val_53 +54 val_54 +57 val_57 +58 val_58 +64 val_64 +65 val_65 +66 val_66 +67 val_67 +69 val_69 +70 val_70 +72 val_72 +74 val_74 +76 val_76 +77 val_77 +78 val_78 +80 val_80 +82 val_82 +83 val_83 +84 val_84 +85 val_85 +86 val_86 +87 val_87 +90 val_90 +92 val_92 +95 val_95 +96 val_96 +97 val_97 +98 val_98 +256 val_256 +257 val_257 +258 val_258 +260 val_260 +262 val_262 +263 val_263 +265 val_265 +266 val_266 +272 val_272 +273 val_273 +274 val_274 +275 val_275 +277 val_277 +278 val_278 +280 val_280 +281 val_281 +282 val_282 +283 val_283 +284 val_284 +285 val_285 +286 val_286 +287 val_287 +288 val_288 +289 val_289 +291 val_291 +292 val_292 +296 val_296 +298 val_298 +302 val_302 +305 val_305 +306 val_306 +307 val_307 +308 val_308 +309 val_309 +310 val_310 +311 val_311 +315 val_315 +316 val_316 +317 val_317 +318 val_318 +321 val_321 +322 val_322 +323 val_323 +325 val_325 +327 val_327 +331 val_331 +332 val_332 +333 val_333 +335 val_335 +336 val_336 +338 val_338 +339 val_339 +341 val_341 +342 val_342 +344 val_344 +345 val_345 +348 val_348 +351 val_351 +353 val_353 +356 val_356 +360 val_360 +362 val_362 +364 val_364 +365 val_365 +366 val_366 +367 val_367 +368 val_368 +369 val_369 +373 val_373 +374 val_374 +375 val_375 +377 val_377 +378 val_378 +379 val_379 +382 val_382 +384 val_384 +386 val_386 +389 val_389 +392 val_392 +393 val_393 +394 val_394 +395 val_395 +396 val_396 +397 val_397 +399 val_399 +400 val_400 +401 val_401 +402 val_402 +403 val_403 +404 val_404 +406 val_406 +407 val_407 +409 val_409 +411 val_411 +413 val_413 +414 val_414 +417 val_417 +418 val_418 +419 val_419 +421 val_421 +424 val_424 +427 val_427 +429 val_429 +430 val_430 +431 val_431 +432 val_432 +435 val_435 +436 val_436 +437 val_437 +438 val_438 +439 val_439 +443 val_443 +444 val_444 +446 val_446 +448 val_448 +449 val_449 +452 val_452 +453 val_453 +454 val_454 +455 val_455 +457 val_457 +458 val_458 +459 val_459 +460 val_460 +462 val_462 +463 val_463 +466 val_466 +467 val_467 +468 val_468 +469 val_469 +470 val_470 +472 val_472 +475 val_475 +477 val_477 +478 val_478 +479 val_479 +480 val_480 +481 val_481 +482 val_482 +483 val_483 +484 val_484 +485 val_485 +487 val_487 +489 val_489 +490 val_490 +491 val_491 +492 val_492 +493 val_493 +494 val_494 +495 val_495 +496 val_496 +497 val_497 +498 val_498 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 bd036faaa6354..8b451973a47a1 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 @@ -391,6 +391,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_sort_8", "groupby_sort_9", "groupby_sort_test_1", + "having", + "having1", "implicit_cast1", "innerjoin", "inoutdriver", From 1871574a240e6f28adeb6bc8accc98c851cafae5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 23 Jul 2014 16:26:55 -0700 Subject: [PATCH 0050/1492] [SPARK-2569][SQL] Fix shipping of TEMPORARY hive UDFs. Instead of shipping just the name and then looking up the info on the workers, we now ship the whole classname. Also, I refactored the file as it was getting pretty large to move out the type conversion code to its own file. Author: Michael Armbrust Closes #1552 from marmbrus/fixTempUdfs and squashes the following commits: b695904 [Michael Armbrust] Make add jar execute with Hive. Ship the whole function class name since sometimes we cannot lookup temporary functions on the workers. --- .../spark/sql/hive/HiveInspectors.scala | 230 +++++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 4 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 262 ++---------------- 3 files changed, 261 insertions(+), 235 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala new file mode 100644 index 0000000000000..ad7dc0ecdb1bf --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.hive.serde2.{io => hiveIo} +import org.apache.hadoop.{io => hadoopIo} + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types +import org.apache.spark.sql.catalyst.types._ + +/* Implicit conversions */ +import scala.collection.JavaConversions._ + +private[hive] trait HiveInspectors { + + def javaClassToDataType(clz: Class[_]): DataType = clz match { + // writable + case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType + case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType + case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType + case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType + case c: Class[_] if c == classOf[hadoopIo.Text] => StringType + case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType + case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType + case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType + case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType + case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType + + // java class + case c: Class[_] if c == classOf[java.lang.String] => StringType + case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType + case c: Class[_] if c == classOf[HiveDecimal] => DecimalType + case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType + case c: Class[_] if c == classOf[Array[Byte]] => BinaryType + case c: Class[_] if c == classOf[java.lang.Short] => ShortType + case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType + case c: Class[_] if c == classOf[java.lang.Long] => LongType + case c: Class[_] if c == classOf[java.lang.Double] => DoubleType + case c: Class[_] if c == classOf[java.lang.Byte] => ByteType + case c: Class[_] if c == classOf[java.lang.Float] => FloatType + case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType + + // primitive type + case c: Class[_] if c == java.lang.Short.TYPE => ShortType + case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType + case c: Class[_] if c == java.lang.Long.TYPE => LongType + case c: Class[_] if c == java.lang.Double.TYPE => DoubleType + case c: Class[_] if c == java.lang.Byte.TYPE => ByteType + case c: Class[_] if c == java.lang.Float.TYPE => FloatType + case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType + + case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) + } + + /** Converts hive types to native catalyst types. */ + def unwrap(a: Any): Any = a match { + case null => null + case i: hadoopIo.IntWritable => i.get + case t: hadoopIo.Text => t.toString + case l: hadoopIo.LongWritable => l.get + case d: hadoopIo.DoubleWritable => d.get + case d: hiveIo.DoubleWritable => d.get + case s: hiveIo.ShortWritable => s.get + case b: hadoopIo.BooleanWritable => b.get + case b: hiveIo.ByteWritable => b.get + case b: hadoopIo.FloatWritable => b.get + case b: hadoopIo.BytesWritable => { + val bytes = new Array[Byte](b.getLength) + System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) + bytes + } + case t: hiveIo.TimestampWritable => t.getTimestamp + case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) + case list: java.util.List[_] => list.map(unwrap) + case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap + case array: Array[_] => array.map(unwrap).toSeq + case p: java.lang.Short => p + case p: java.lang.Long => p + case p: java.lang.Float => p + case p: java.lang.Integer => p + case p: java.lang.Double => p + case p: java.lang.Byte => p + case p: java.lang.Boolean => p + case str: String => str + case p: java.math.BigDecimal => p + case p: Array[Byte] => p + case p: java.sql.Timestamp => p + } + + def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { + case hvoi: HiveVarcharObjectInspector => + if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue + case hdoi: HiveDecimalObjectInspector => + if (data == null) null else BigDecimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) + case li: ListObjectInspector => + Option(li.getList(data)) + .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) + .orNull + case mi: MapObjectInspector => + Option(mi.getMap(data)).map( + _.map { + case (k,v) => + (unwrapData(k, mi.getMapKeyObjectInspector), + unwrapData(v, mi.getMapValueObjectInspector)) + }.toMap).orNull + case si: StructObjectInspector => + val allRefs = si.getAllStructFieldRefs + new GenericRow( + allRefs.map(r => + unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) + } + + /** Converts native catalyst types to the types expected by Hive */ + def wrap(a: Any): AnyRef = a match { + case s: String => new hadoopIo.Text(s) // TODO why should be Text? + case i: Int => i: java.lang.Integer + case b: Boolean => b: java.lang.Boolean + case f: Float => f: java.lang.Float + case d: Double => d: java.lang.Double + case l: Long => l: java.lang.Long + case l: Short => l: java.lang.Short + case l: Byte => l: java.lang.Byte + case b: BigDecimal => b.bigDecimal + case b: Array[Byte] => b + case t: java.sql.Timestamp => t + case s: Seq[_] => seqAsJavaList(s.map(wrap)) + case m: Map[_,_] => + mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) + case null => null + } + + def toInspector(dataType: DataType): ObjectInspector = dataType match { + case ArrayType(tpe) => ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) + case MapType(keyType, valueType) => + ObjectInspectorFactory.getStandardMapObjectInspector( + toInspector(keyType), toInspector(valueType)) + case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector + case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector + case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector + case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector + case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector + case FloatType => PrimitiveObjectInspectorFactory.javaFloatObjectInspector + case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector + case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector + case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector + case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector + case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector + case StructType(fields) => + ObjectInspectorFactory.getStandardStructObjectInspector( + fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) + } + + def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { + case s: StructObjectInspector => + StructType(s.getAllStructFieldRefs.map(f => { + types.StructField( + f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) + })) + case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) + case m: MapObjectInspector => + MapType( + inspectorToDataType(m.getMapKeyObjectInspector), + inspectorToDataType(m.getMapValueObjectInspector)) + case _: WritableStringObjectInspector => StringType + case _: JavaStringObjectInspector => StringType + case _: WritableIntObjectInspector => IntegerType + case _: JavaIntObjectInspector => IntegerType + case _: WritableDoubleObjectInspector => DoubleType + case _: JavaDoubleObjectInspector => DoubleType + case _: WritableBooleanObjectInspector => BooleanType + case _: JavaBooleanObjectInspector => BooleanType + case _: WritableLongObjectInspector => LongType + case _: JavaLongObjectInspector => LongType + case _: WritableShortObjectInspector => ShortType + case _: JavaShortObjectInspector => ShortType + case _: WritableByteObjectInspector => ByteType + case _: JavaByteObjectInspector => ByteType + case _: WritableFloatObjectInspector => FloatType + case _: JavaFloatObjectInspector => FloatType + case _: WritableBinaryObjectInspector => BinaryType + case _: JavaBinaryObjectInspector => BinaryType + case _: WritableHiveDecimalObjectInspector => DecimalType + case _: JavaHiveDecimalObjectInspector => DecimalType + case _: WritableTimestampObjectInspector => TimestampType + case _: JavaTimestampObjectInspector => TimestampType + } + + implicit class typeInfoConversions(dt: DataType) { + import org.apache.hadoop.hive.serde2.typeinfo._ + import TypeInfoFactory._ + + def toTypeInfo: TypeInfo = dt match { + case BinaryType => binaryTypeInfo + case BooleanType => booleanTypeInfo + case ByteType => byteTypeInfo + case DoubleType => doubleTypeInfo + case FloatType => floatTypeInfo + case IntegerType => intTypeInfo + case LongType => longTypeInfo + case ShortType => shortTypeInfo + case StringType => stringTypeInfo + case DecimalType => decimalTypeInfo + case TimestampType => timestampTypeInfo + case NullType => voidTypeInfo + } + } +} 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 53480a521dd14..c4ca9f362a04d 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 @@ -42,8 +42,6 @@ private[hive] case class ShellCommand(cmd: String) extends Command private[hive] case class SourceCommand(filePath: String) extends Command -private[hive] case class AddJar(jarPath: String) extends Command - private[hive] case class AddFile(filePath: String) extends Command /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ @@ -229,7 +227,7 @@ private[hive] object HiveQl { } else if (sql.trim.toLowerCase.startsWith("uncache table")) { CacheCommand(sql.trim.drop(14).trim, false) } else if (sql.trim.toLowerCase.startsWith("add jar")) { - AddJar(sql.trim.drop(8)) + NativeCommand(sql) } else if (sql.trim.toLowerCase.startsWith("add file")) { AddFile(sql.trim.drop(9)) } else if (sql.trim.toLowerCase.startsWith("dfs")) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index fc33c5b460d70..057eb60a02612 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -24,22 +24,19 @@ import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.primitive._ -import org.apache.hadoop.hive.serde2.{io => hiveIo} -import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.util.Utils.getContextOrSparkClassLoader /* Implicit conversions */ import scala.collection.JavaConversions._ -private[hive] object HiveFunctionRegistry - extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors { +private[hive] object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveInspectors { + + def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is @@ -47,111 +44,37 @@ private[hive] object HiveFunctionRegistry val functionInfo: FunctionInfo = Option(FunctionRegistry.getFunctionInfo(name)).getOrElse( sys.error(s"Couldn't find function $name")) + val functionClassName = functionInfo.getFunctionClass.getName() + if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - val function = createFunction[UDF](name) + val function = functionInfo.getFunctionClass.newInstance().asInstanceOf[UDF] val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) lazy val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) HiveSimpleUdf( - name, + functionClassName, children.zip(expectedDataTypes).map { case (e, t) => Cast(e, t) } ) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdf(name, children) + HiveGenericUdf(functionClassName, children) } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdaf(name, children) + HiveGenericUdaf(functionClassName, children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdtf(name, Nil, children) + HiveGenericUdtf(functionClassName, Nil, children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } } - - def javaClassToDataType(clz: Class[_]): DataType = clz match { - // writable - case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType - case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType - case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType - case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType - case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType - case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType - case c: Class[_] if c == classOf[hadoopIo.Text] => StringType - case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType - case c: Class[_] if c == classOf[hadoopIo.LongWritable] => LongType - case c: Class[_] if c == classOf[hadoopIo.FloatWritable] => FloatType - case c: Class[_] if c == classOf[hadoopIo.BooleanWritable] => BooleanType - case c: Class[_] if c == classOf[hadoopIo.BytesWritable] => BinaryType - - // java class - case c: Class[_] if c == classOf[java.lang.String] => StringType - case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType - case c: Class[_] if c == classOf[HiveDecimal] => DecimalType - case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType - case c: Class[_] if c == classOf[Array[Byte]] => BinaryType - case c: Class[_] if c == classOf[java.lang.Short] => ShortType - case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType - case c: Class[_] if c == classOf[java.lang.Long] => LongType - case c: Class[_] if c == classOf[java.lang.Double] => DoubleType - case c: Class[_] if c == classOf[java.lang.Byte] => ByteType - case c: Class[_] if c == classOf[java.lang.Float] => FloatType - case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType - - // primitive type - case c: Class[_] if c == java.lang.Short.TYPE => ShortType - case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType - case c: Class[_] if c == java.lang.Long.TYPE => LongType - case c: Class[_] if c == java.lang.Double.TYPE => DoubleType - case c: Class[_] if c == java.lang.Byte.TYPE => ByteType - case c: Class[_] if c == java.lang.Float.TYPE => FloatType - case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType - - case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) - } } private[hive] trait HiveFunctionFactory { - def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) - def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass - def createFunction[UDFType](name: String) = - getFunctionClass(name).newInstance.asInstanceOf[UDFType] - - /** Converts hive types to native catalyst types. */ - def unwrap(a: Any): Any = a match { - case null => null - case i: hadoopIo.IntWritable => i.get - case t: hadoopIo.Text => t.toString - case l: hadoopIo.LongWritable => l.get - case d: hadoopIo.DoubleWritable => d.get - case d: hiveIo.DoubleWritable => d.get - case s: hiveIo.ShortWritable => s.get - case b: hadoopIo.BooleanWritable => b.get - case b: hiveIo.ByteWritable => b.get - case b: hadoopIo.FloatWritable => b.get - case b: hadoopIo.BytesWritable => { - val bytes = new Array[Byte](b.getLength) - System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) - bytes - } - case t: hiveIo.TimestampWritable => t.getTimestamp - case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) - case list: java.util.List[_] => list.map(unwrap) - case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap - case array: Array[_] => array.map(unwrap).toSeq - case p: java.lang.Short => p - case p: java.lang.Long => p - case p: java.lang.Float => p - case p: java.lang.Integer => p - case p: java.lang.Double => p - case p: java.lang.Byte => p - case p: java.lang.Boolean => p - case str: String => str - case p: java.math.BigDecimal => p - case p: Array[Byte] => p - case p: java.sql.Timestamp => p - } + val functionClassName: String + + def createFunction[UDFType]() = + getContextOrSparkClassLoader.loadClass(functionClassName).newInstance.asInstanceOf[UDFType] } private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { @@ -160,19 +83,17 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu type UDFType type EvaluatedType = Any - val name: String - def nullable = true def references = children.flatMap(_.references).toSet - // FunctionInfo is not serializable so we must look it up here again. - lazy val functionInfo = getFunctionInfo(name) - lazy val function = createFunction[UDFType](name) + lazy val function = createFunction[UDFType]() - override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" } -private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { +private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[Expression]) + extends HiveUdf { + import org.apache.spark.sql.hive.HiveFunctionRegistry._ type UDFType = UDF @@ -226,7 +147,7 @@ private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) } } -private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) +private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq[Expression]) extends HiveUdf with HiveInspectors { import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ @@ -277,131 +198,8 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) } } -private[hive] trait HiveInspectors { - - def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { - case hvoi: HiveVarcharObjectInspector => - if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue - case hdoi: HiveDecimalObjectInspector => - if (data == null) null else BigDecimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) - case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) - case li: ListObjectInspector => - Option(li.getList(data)) - .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) - .orNull - case mi: MapObjectInspector => - Option(mi.getMap(data)).map( - _.map { - case (k,v) => - (unwrapData(k, mi.getMapKeyObjectInspector), - unwrapData(v, mi.getMapValueObjectInspector)) - }.toMap).orNull - case si: StructObjectInspector => - val allRefs = si.getAllStructFieldRefs - new GenericRow( - allRefs.map(r => - unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) - } - - /** Converts native catalyst types to the types expected by Hive */ - def wrap(a: Any): AnyRef = a match { - case s: String => new hadoopIo.Text(s) // TODO why should be Text? - case i: Int => i: java.lang.Integer - case b: Boolean => b: java.lang.Boolean - case f: Float => f: java.lang.Float - case d: Double => d: java.lang.Double - case l: Long => l: java.lang.Long - case l: Short => l: java.lang.Short - case l: Byte => l: java.lang.Byte - case b: BigDecimal => b.bigDecimal - case b: Array[Byte] => b - case t: java.sql.Timestamp => t - case s: Seq[_] => seqAsJavaList(s.map(wrap)) - case m: Map[_,_] => - mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) - case null => null - } - - def toInspector(dataType: DataType): ObjectInspector = dataType match { - case ArrayType(tpe) => ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) - case MapType(keyType, valueType) => - ObjectInspectorFactory.getStandardMapObjectInspector( - toInspector(keyType), toInspector(valueType)) - case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector - case IntegerType => PrimitiveObjectInspectorFactory.javaIntObjectInspector - case DoubleType => PrimitiveObjectInspectorFactory.javaDoubleObjectInspector - case BooleanType => PrimitiveObjectInspectorFactory.javaBooleanObjectInspector - case LongType => PrimitiveObjectInspectorFactory.javaLongObjectInspector - case FloatType => PrimitiveObjectInspectorFactory.javaFloatObjectInspector - case ShortType => PrimitiveObjectInspectorFactory.javaShortObjectInspector - case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector - case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector - case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector - case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector - case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector - case StructType(fields) => - ObjectInspectorFactory.getStandardStructObjectInspector( - fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) - } - - def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { - case s: StructObjectInspector => - StructType(s.getAllStructFieldRefs.map(f => { - types.StructField( - f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) - })) - case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) - case m: MapObjectInspector => - MapType( - inspectorToDataType(m.getMapKeyObjectInspector), - inspectorToDataType(m.getMapValueObjectInspector)) - case _: WritableStringObjectInspector => StringType - case _: JavaStringObjectInspector => StringType - case _: WritableIntObjectInspector => IntegerType - case _: JavaIntObjectInspector => IntegerType - case _: WritableDoubleObjectInspector => DoubleType - case _: JavaDoubleObjectInspector => DoubleType - case _: WritableBooleanObjectInspector => BooleanType - case _: JavaBooleanObjectInspector => BooleanType - case _: WritableLongObjectInspector => LongType - case _: JavaLongObjectInspector => LongType - case _: WritableShortObjectInspector => ShortType - case _: JavaShortObjectInspector => ShortType - case _: WritableByteObjectInspector => ByteType - case _: JavaByteObjectInspector => ByteType - case _: WritableFloatObjectInspector => FloatType - case _: JavaFloatObjectInspector => FloatType - case _: WritableBinaryObjectInspector => BinaryType - case _: JavaBinaryObjectInspector => BinaryType - case _: WritableHiveDecimalObjectInspector => DecimalType - case _: JavaHiveDecimalObjectInspector => DecimalType - case _: WritableTimestampObjectInspector => TimestampType - case _: JavaTimestampObjectInspector => TimestampType - } - - implicit class typeInfoConversions(dt: DataType) { - import org.apache.hadoop.hive.serde2.typeinfo._ - import TypeInfoFactory._ - - def toTypeInfo: TypeInfo = dt match { - case BinaryType => binaryTypeInfo - case BooleanType => booleanTypeInfo - case ByteType => byteTypeInfo - case DoubleType => doubleTypeInfo - case FloatType => floatTypeInfo - case IntegerType => intTypeInfo - case LongType => longTypeInfo - case ShortType => shortTypeInfo - case StringType => stringTypeInfo - case DecimalType => decimalTypeInfo - case TimestampType => timestampTypeInfo - case NullType => voidTypeInfo - } - } -} - private[hive] case class HiveGenericUdaf( - name: String, + functionClassName: String, children: Seq[Expression]) extends AggregateExpression with HiveInspectors with HiveFunctionFactory { @@ -409,7 +207,7 @@ private[hive] case class HiveGenericUdaf( type UDFType = AbstractGenericUDAFResolver @transient - protected lazy val resolver: AbstractGenericUDAFResolver = createFunction(name) + protected lazy val resolver: AbstractGenericUDAFResolver = createFunction() @transient protected lazy val objectInspector = { @@ -426,9 +224,9 @@ private[hive] case class HiveGenericUdaf( def references: Set[Attribute] = children.map(_.references).flatten.toSet - override def toString = s"$nodeName#$name(${children.mkString(",")})" + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" - def newInstance() = new HiveUdafFunction(name, children, this) + def newInstance() = new HiveUdafFunction(functionClassName, children, this) } /** @@ -443,7 +241,7 @@ private[hive] case class HiveGenericUdaf( * user defined aggregations, which have clean semantics even in a partitioned execution. */ private[hive] case class HiveGenericUdtf( - name: String, + functionClassName: String, aliasNames: Seq[String], children: Seq[Expression]) extends Generator with HiveInspectors with HiveFunctionFactory { @@ -451,7 +249,7 @@ private[hive] case class HiveGenericUdtf( override def references = children.flatMap(_.references).toSet @transient - protected lazy val function: GenericUDTF = createFunction(name) + protected lazy val function: GenericUDTF = createFunction() protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) @@ -506,11 +304,11 @@ private[hive] case class HiveGenericUdtf( } } - override def toString = s"$nodeName#$name(${children.mkString(",")})" + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" } private[hive] case class HiveUdafFunction( - functionName: String, + functionClassName: String, exprs: Seq[Expression], base: AggregateExpression) extends AggregateFunction @@ -519,7 +317,7 @@ private[hive] case class HiveUdafFunction( def this() = this(null, null, null) - private val resolver = createFunction[AbstractGenericUDAFResolver](functionName) + private val resolver = createFunction[AbstractGenericUDAFResolver]() private val inspectors = exprs.map(_.dataType).map(toInspector).toArray From efdaeb111917dd0314f1d00ee8524bed1e2e21ca Mon Sep 17 00:00:00 2001 From: Ian O Connell Date: Wed, 23 Jul 2014 16:30:06 -0700 Subject: [PATCH 0051/1492] [SPARK-2102][SQL][CORE] Add option for kryo registration required and use a resource pool in Spark SQL for Kryo instances. Author: Ian O Connell Closes #1377 from ianoc/feature/SPARK-2102 and squashes the following commits: 5498566 [Ian O Connell] Docs update suggested by Patrick 20e8555 [Ian O Connell] Slight style change f92c294 [Ian O Connell] Add docs for new KryoSerializer option f3735c8 [Ian O Connell] Add using a kryo resource pool for the SqlSerializer 4e5c342 [Ian O Connell] Register the SparkConf for kryo, it gets swept into serialization 665805a [Ian O Connell] Add a spark.kryo.registrationRequired option for configuring the Kryo Serializer --- .../spark/serializer/KryoSerializer.scala | 5 ++- docs/configuration.md | 19 ++++++-- .../sql/execution/SparkSqlSerializer.scala | 43 +++++++++++++------ 3 files changed, 50 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 1ce4243194798..c3a3e90a34901 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -48,6 +48,7 @@ class KryoSerializer(conf: SparkConf) private val bufferSize = conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) + private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) private val registrator = conf.getOption("spark.kryo.registrator") def newKryoOutput() = new KryoOutput(bufferSize) @@ -55,6 +56,7 @@ class KryoSerializer(conf: SparkConf) def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator val kryo = instantiator.newKryo() + kryo.setRegistrationRequired(registrationRequired) val classLoader = Thread.currentThread.getContextClassLoader // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. @@ -185,7 +187,8 @@ private[serializer] object KryoSerializer { classOf[MapStatus], classOf[BlockManagerId], classOf[Array[Byte]], - classOf[BoundedPriorityQueue[_]] + classOf[BoundedPriorityQueue[_]], + classOf[SparkConf] ) } diff --git a/docs/configuration.md b/docs/configuration.md index a70007c165442..02af461267c46 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -388,6 +388,17 @@ Apart from these, the following properties are also available, and may be useful case. + + spark.kryo.registrationRequired + false + + Whether to require registration with Kryo. If set to 'true', Kryo will throw an exception + if an unregistered class is serialized. If set to false (the default), Kryo will write + unregistered class names along with each object. Writing class names can cause + significant performance overhead, so enabling this option can enforce strictly that a + user has not omitted classes from registration. + + spark.kryoserializer.buffer.mb 2 @@ -497,9 +508,9 @@ Apart from these, the following properties are also available, and may be useful spark.hadoop.validateOutputSpecs true - If set to true, validates the output specification (e.g. checking if the output directory already exists) - used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing - output directories. We recommend that users do not disable this except if trying to achieve compatibility with + If set to true, validates the output specification (e.g. checking if the output directory already exists) + used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing + output directories. We recommend that users do not disable this except if trying to achieve compatibility with previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. @@ -861,7 +872,7 @@ Apart from these, the following properties are also available, and may be useful #### Cluster Managers -Each cluster manager in Spark has additional configuration options. Configurations +Each cluster manager in Spark has additional configuration options. Configurations can be found on the pages for each mode: * [YARN](running-on-yarn.html#configuration) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 34b355e906695..34654447a5f4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -24,10 +24,10 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} -import com.twitter.chill.AllScalaRegistrar +import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} -import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils @@ -48,22 +48,41 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co } } -private[sql] object SparkSqlSerializer { - // TODO (lian) Using KryoSerializer here is workaround, needs further investigation - // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization - // related error. - @transient lazy val ser: KryoSerializer = { +private[execution] class KryoResourcePool(size: Int) + extends ResourcePool[SerializerInstance](size) { + + val ser: KryoSerializer = { val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + // TODO (lian) Using KryoSerializer here is workaround, needs further investigation + // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization + // related error. new KryoSerializer(sparkConf) } - def serialize[T: ClassTag](o: T): Array[Byte] = { - ser.newInstance().serialize(o).array() - } + def newInstance() = ser.newInstance() +} - def deserialize[T: ClassTag](bytes: Array[Byte]): T = { - ser.newInstance().deserialize[T](ByteBuffer.wrap(bytes)) +private[sql] object SparkSqlSerializer { + @transient lazy val resourcePool = new KryoResourcePool(30) + + private[this] def acquireRelease[O](fn: SerializerInstance => O): O = { + val kryo = resourcePool.borrow + try { + fn(kryo) + } finally { + resourcePool.release(kryo) + } } + + def serialize[T: ClassTag](o: T): Array[Byte] = + acquireRelease { k => + k.serialize(o).array() + } + + def deserialize[T: ClassTag](bytes: Array[Byte]): T = + acquireRelease { k => + k.deserialize[T](ByteBuffer.wrap(bytes)) + } } private[sql] class BigDecimalSerializer extends Serializer[BigDecimal] { From 9b763329d968658d2dedee4485b7931646e6392e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 23 Jul 2014 17:12:28 -0700 Subject: [PATCH 0052/1492] [SPARK-2549] Functions defined inside of other functions trigger failures Author: Prashant Sharma Closes #1510 from ScrapCodes/SPARK-2549/fun-in-fun and squashes the following commits: 9458bc5 [Prashant Sharma] Tested by removing an inner function from excludes. bc03b1c [Prashant Sharma] SPARK-2549 Functions defined inside of other functions trigger failures --- project/MimaExcludes.scala | 183 ++++++++---------- .../spark/tools/GenerateMIMAIgnore.scala | 21 +- 2 files changed, 98 insertions(+), 106 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 5e5ddd227aab6..e9220db6b1f9a 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -32,108 +32,83 @@ import com.typesafe.tools.mima.core._ */ object MimaExcludes { - def excludes(version: String) = version match { - case v if v.startsWith("1.1") => - Seq( - MimaBuild.excludeSparkPackage("deploy"), - MimaBuild.excludeSparkPackage("graphx") - ) ++ - closures.map(method => ProblemFilters.exclude[MissingMethodProblem](method)) ++ - Seq( - // Adding new method to JavaRDLike trait - we should probably mark this as a developer API. - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitions"), - // We made a mistake earlier (ed06500d3) in the Java API to use default parameter values - // for countApproxDistinct* functions, which does not work in Java. We later removed - // them, and use the following to tell Mima to not care about them. - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDD.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.storage.MemoryStore.Entry"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$debugChildren$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$firstDebugString$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$shuffleDebugString$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$debugString$1"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$" - + "createZero$1") - ) ++ - Seq( - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.FlumeReceiver.this") - ) ++ - Seq( // Ignore some private methods in ALS. - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures"), - ProblemFilters.exclude[MissingMethodProblem]( // The only public constructor is the one without arguments. - "org.apache.spark.mllib.recommendation.ALS.this"), - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures") - ) ++ - MimaBuild.excludeSparkClass("mllib.linalg.distributed.ColumnStatisticsAggregator") ++ - MimaBuild.excludeSparkClass("rdd.ZippedRDD") ++ - MimaBuild.excludeSparkClass("rdd.ZippedPartition") ++ - MimaBuild.excludeSparkClass("util.SerializableHyperLogLog") ++ - MimaBuild.excludeSparkClass("storage.Values") ++ - MimaBuild.excludeSparkClass("storage.Entry") ++ - MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") ++ - Seq( - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.tree.impurity.Gini.calculate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.tree.impurity.Entropy.calculate"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]( - "org.apache.spark.mllib.tree.impurity.Variance.calculate") - ) - case v if v.startsWith("1.0") => - Seq( - MimaBuild.excludeSparkPackage("api.java"), - MimaBuild.excludeSparkPackage("mllib"), - MimaBuild.excludeSparkPackage("streaming") - ) ++ - MimaBuild.excludeSparkClass("rdd.ClassTags") ++ - MimaBuild.excludeSparkClass("util.XORShiftRandom") ++ - MimaBuild.excludeSparkClass("graphx.EdgeRDD") ++ - MimaBuild.excludeSparkClass("graphx.VertexRDD") ++ - MimaBuild.excludeSparkClass("graphx.impl.GraphImpl") ++ - MimaBuild.excludeSparkClass("graphx.impl.RoutingTable") ++ - MimaBuild.excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++ - MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++ - MimaBuild.excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ - MimaBuild.excludeSparkClass("mllib.optimization.SquaredGradient") ++ - MimaBuild.excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ - MimaBuild.excludeSparkClass("mllib.regression.LassoWithSGD") ++ - MimaBuild.excludeSparkClass("mllib.regression.LinearRegressionWithSGD") - case _ => Seq() - } - - private val closures = Seq( - "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$mergeMaps$1", - "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$countPartition$1", - "org.apache.spark.rdd.RDD.org$apache$spark$rdd$RDD$$distributePartition$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$mergeValue$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$writeToFile$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$reducePartition$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$writeShard$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$mergeCombiners$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$process$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$createCombiner$1", - "org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$rdd$PairRDDFunctions$$mergeMaps$1" - ) + def excludes(version: String) = + version match { + case v if v.startsWith("1.1") => + Seq( + MimaBuild.excludeSparkPackage("deploy"), + MimaBuild.excludeSparkPackage("graphx") + ) ++ + Seq( + // Adding new method to JavaRDLike trait - we should probably mark this as a developer API. + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.api.java.JavaRDDLike.partitions"), + // We made a mistake earlier (ed06500d3) in the Java API to use default parameter values + // for countApproxDistinct* functions, which does not work in Java. We later removed + // them, and use the following to tell Mima to not care about them. + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaPairRDD.countApproxDistinctByKey$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDD.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaDoubleRDD.countApproxDistinct$default$1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.MemoryStore.Entry") + ) ++ + Seq( + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.FlumeReceiver.this") + ) ++ + Seq( // Ignore some private methods in ALS. + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures"), + ProblemFilters.exclude[MissingMethodProblem]( // The only public constructor is the one without arguments. + "org.apache.spark.mllib.recommendation.ALS.this"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$$$default$7"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.recommendation.ALS.org$apache$spark$mllib$recommendation$ALS$^dateFeatures") + ) ++ + MimaBuild.excludeSparkClass("mllib.linalg.distributed.ColumnStatisticsAggregator") ++ + MimaBuild.excludeSparkClass("rdd.ZippedRDD") ++ + MimaBuild.excludeSparkClass("rdd.ZippedPartition") ++ + MimaBuild.excludeSparkClass("util.SerializableHyperLogLog") ++ + MimaBuild.excludeSparkClass("storage.Values") ++ + MimaBuild.excludeSparkClass("storage.Entry") ++ + MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") ++ + Seq( + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Gini.calculate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Entropy.calculate"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.mllib.tree.impurity.Variance.calculate") + ) + case v if v.startsWith("1.0") => + Seq( + MimaBuild.excludeSparkPackage("api.java"), + MimaBuild.excludeSparkPackage("mllib"), + MimaBuild.excludeSparkPackage("streaming") + ) ++ + MimaBuild.excludeSparkClass("rdd.ClassTags") ++ + MimaBuild.excludeSparkClass("util.XORShiftRandom") ++ + MimaBuild.excludeSparkClass("graphx.EdgeRDD") ++ + MimaBuild.excludeSparkClass("graphx.VertexRDD") ++ + MimaBuild.excludeSparkClass("graphx.impl.GraphImpl") ++ + MimaBuild.excludeSparkClass("graphx.impl.RoutingTable") ++ + MimaBuild.excludeSparkClass("graphx.util.collection.PrimitiveKeyOpenHashMap") ++ + MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") ++ + MimaBuild.excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ + MimaBuild.excludeSparkClass("mllib.optimization.SquaredGradient") ++ + MimaBuild.excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ + MimaBuild.excludeSparkClass("mllib.regression.LassoWithSGD") ++ + MimaBuild.excludeSparkClass("mllib.regression.LinearRegressionWithSGD") + case _ => Seq() + } } diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 03a73f92b275e..566983675bff5 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -99,9 +99,25 @@ object GenerateMIMAIgnore { (ignoredClasses.flatMap(c => Seq(c, c.replace("$", "#"))).toSet, ignoredMembers.toSet) } + /** Scala reflection does not let us see inner function even if they are upgraded + * to public for some reason. So had to resort to java reflection to get all inner + * functions with $$ in there name. + */ + def getInnerFunctions(classSymbol: unv.ClassSymbol): Seq[String] = { + try { + Class.forName(classSymbol.fullName, false, classLoader).getMethods.map(_.getName) + .filter(_.contains("$$")).map(classSymbol.fullName + "." + _) + } catch { + case t: Throwable => + println("[WARN] Unable to detect inner functions for class:" + classSymbol.fullName) + Seq.empty[String] + } + } + private def getAnnotatedOrPackagePrivateMembers(classSymbol: unv.ClassSymbol) = { classSymbol.typeSignature.members - .filter(x => isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x)).map(_.fullName) + .filter(x => isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x)).map(_.fullName) ++ + getInnerFunctions(classSymbol) } def main(args: Array[String]) { @@ -121,7 +137,8 @@ object GenerateMIMAIgnore { name.endsWith("$class") || name.contains("$sp") || name.contains("hive") || - name.contains("Hive") + name.contains("Hive") || + name.contains("repl") } /** From 60f0ae3d87c84fd96e1f4d0abf5be1f51870e7ab Mon Sep 17 00:00:00 2001 From: witgo Date: Wed, 23 Jul 2014 18:17:05 -0700 Subject: [PATCH 0053/1492] [SPARK-2484][SQL] Build should not run hivecompatibility tests by default. Author: witgo Closes #1403 from witgo/hive_compatibility and squashes the following commits: 4e5ecdb [witgo] The default does not run hive compatibility tests --- .../execution/HiveCompatibilitySuite.scala | 0 sql/hive/pom.xml | 30 +++++++++++++++++++ 2 files changed, 30 insertions(+) rename sql/hive/{ => compatibility}/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala (100%) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala similarity index 100% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala rename to sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index f30ae28b81e06..1699ffe06ce15 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -102,6 +102,36 @@ test + + + + hive + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + compatibility/src/test/scala + + + + + + + + + + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes From 2d25e34814f81f288587f3277324cb655a5fb38d Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 23 Jul 2014 20:11:28 -0700 Subject: [PATCH 0054/1492] Replace RoutingTableMessage with pair RoutingTableMessage was used to construct routing tables to enable joining VertexRDDs with partitioned edges. It stored three elements: the destination vertex ID, the source edge partition, and a byte specifying the position in which the edge partition referenced the vertex to enable join elimination. However, this was incompatible with sort-based shuffle (SPARK-2045). It was also slightly wasteful, because partition IDs are usually much smaller than 2^32, though this was mitigated by a custom serializer that used variable-length encoding. This commit replaces RoutingTableMessage with a pair of (VertexId, Int) where the Int encodes both the source partition ID (in the lower 30 bits) and the position (in the top 2 bits). Author: Ankur Dave Closes #1553 from ankurdave/remove-RoutingTableMessage and squashes the following commits: 697e17b [Ankur Dave] Replace RoutingTableMessage with pair --- .../spark/graphx/GraphKryoRegistrator.scala | 1 - .../graphx/impl/RoutingTablePartition.scala | 47 +++++++++++-------- .../spark/graphx/impl/Serializers.scala | 16 +++---- .../org/apache/spark/graphx/package.scala | 2 +- 4 files changed, 36 insertions(+), 30 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index eea9fe9520caa..1948c978c30bf 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -35,7 +35,6 @@ class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { kryo.register(classOf[Edge[Object]]) - kryo.register(classOf[RoutingTableMessage]) kryo.register(classOf[(VertexId, Object)]) kryo.register(classOf[EdgePartition[Object, Object]]) kryo.register(classOf[BitSet]) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index 502b112d31c2e..a565d3b28bf52 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -27,26 +27,13 @@ import org.apache.spark.util.collection.{BitSet, PrimitiveVector} import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap -/** - * A message from the edge partition `pid` to the vertex partition containing `vid` specifying that - * the edge partition references `vid` in the specified `position` (src, dst, or both). -*/ -private[graphx] -class RoutingTableMessage( - var vid: VertexId, - var pid: PartitionID, - var position: Byte) - extends Product2[VertexId, (PartitionID, Byte)] with Serializable { - override def _1 = vid - override def _2 = (pid, position) - override def canEqual(that: Any): Boolean = that.isInstanceOf[RoutingTableMessage] -} +import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage private[graphx] class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { - new ShuffledRDD[VertexId, (PartitionID, Byte), (PartitionID, Byte), RoutingTableMessage]( + new ShuffledRDD[VertexId, Int, Int, RoutingTableMessage]( self, partitioner).setSerializer(new RoutingTableMessageSerializer) } } @@ -62,6 +49,23 @@ object RoutingTableMessageRDDFunctions { private[graphx] object RoutingTablePartition { + /** + * A message from an edge partition to a vertex specifying the position in which the edge + * partition references the vertex (src, dst, or both). The edge partition is encoded in the lower + * 30 bytes of the Int, and the position is encoded in the upper 2 bytes of the Int. + */ + type RoutingTableMessage = (VertexId, Int) + + private def toMessage(vid: VertexId, pid: PartitionID, position: Byte): RoutingTableMessage = { + val positionUpper2 = position << 30 + val pidLower30 = pid & 0x3FFFFFFF + (vid, positionUpper2 | pidLower30) + } + + private def vidFromMessage(msg: RoutingTableMessage): VertexId = msg._1 + private def pidFromMessage(msg: RoutingTableMessage): PartitionID = msg._2 & 0x3FFFFFFF + private def positionFromMessage(msg: RoutingTableMessage): Byte = (msg._2 >> 30).toByte + val empty: RoutingTablePartition = new RoutingTablePartition(Array.empty) /** Generate a `RoutingTableMessage` for each vertex referenced in `edgePartition`. */ @@ -77,7 +81,9 @@ object RoutingTablePartition { map.changeValue(dstId, 0x2, (b: Byte) => (b | 0x2).toByte) } map.iterator.map { vidAndPosition => - new RoutingTableMessage(vidAndPosition._1, pid, vidAndPosition._2) + val vid = vidAndPosition._1 + val position = vidAndPosition._2 + toMessage(vid, pid, position) } } @@ -88,9 +94,12 @@ object RoutingTablePartition { val srcFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) val dstFlags = Array.fill(numEdgePartitions)(new PrimitiveVector[Boolean]) for (msg <- iter) { - pid2vid(msg.pid) += msg.vid - srcFlags(msg.pid) += (msg.position & 0x1) != 0 - dstFlags(msg.pid) += (msg.position & 0x2) != 0 + val vid = vidFromMessage(msg) + val pid = pidFromMessage(msg) + val position = positionFromMessage(msg) + pid2vid(pid) += vid + srcFlags(pid) += (position & 0x1) != 0 + dstFlags(pid) += (position & 0x2) != 0 } new RoutingTablePartition(pid2vid.zipWithIndex.map { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 2d98c24d6970e..3909efcdfc993 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -24,9 +24,11 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag -import org.apache.spark.graphx._ import org.apache.spark.serializer._ +import org.apache.spark.graphx._ +import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage + private[graphx] class RoutingTableMessageSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { @@ -35,10 +37,8 @@ class RoutingTableMessageSerializer extends Serializer with Serializable { new ShuffleSerializationStream(s) { def writeObject[T: ClassTag](t: T): SerializationStream = { val msg = t.asInstanceOf[RoutingTableMessage] - writeVarLong(msg.vid, optimizePositive = false) - writeUnsignedVarInt(msg.pid) - // TODO: Write only the bottom two bits of msg.position - s.write(msg.position) + writeVarLong(msg._1, optimizePositive = false) + writeInt(msg._2) this } } @@ -47,10 +47,8 @@ class RoutingTableMessageSerializer extends Serializer with Serializable { new ShuffleDeserializationStream(s) { override def readObject[T: ClassTag](): T = { val a = readVarLong(optimizePositive = false) - val b = readUnsignedVarInt() - val c = s.read() - if (c == -1) throw new EOFException - new RoutingTableMessage(a, b, c.toByte).asInstanceOf[T] + val b = readInt() + (a, b).asInstanceOf[T] } } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index ff17edeaf8f16..6aab28ff05355 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -30,7 +30,7 @@ package object graphx { */ type VertexId = Long - /** Integer identifer of a graph partition. */ + /** Integer identifer of a graph partition. Must be less than 2^30. */ // TODO: Consider using Char. type PartitionID = Int From 9e7725c86e70ffd3d2ff3a563460c2b7d0c9bbee Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Wed, 23 Jul 2014 22:50:39 -0700 Subject: [PATCH 0055/1492] SPARK-2662: Fix NPE for JsonProtocol Author: GuoQiang Li Closes #1511 from witgo/JsonProtocol and squashes the following commits: 2b6227f [GuoQiang Li] Fix NPE for JsonProtocol --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 3448aaaf5724c..bb6079154aafe 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -257,7 +257,8 @@ private[spark] object JsonProtocol { val reason = Utils.getFormattedClassName(taskEndReason) val json = taskEndReason match { case fetchFailed: FetchFailed => - val blockManagerAddress = blockManagerIdToJson(fetchFailed.bmAddress) + val blockManagerAddress = Option(fetchFailed.bmAddress). + map(blockManagerIdToJson).getOrElse(JNothing) ("Block Manager Address" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ ("Map ID" -> fetchFailed.mapId) ~ From 78d18fdbaa62d8ed235c29b2e37fd6607263c639 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 23 Jul 2014 22:52:49 -0700 Subject: [PATCH 0056/1492] [SPARK-2658][SQL] Add rule for true = 1. Author: Michael Armbrust Closes #1556 from marmbrus/fixBooleanEqualsOne and squashes the following commits: ad8edd4 [Michael Armbrust] Add rule for true = 1 and false = 0. --- .../sql/catalyst/analysis/HiveTypeCoercion.scala | 12 +++++++++++- ...olean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 | 1 + .../spark/sql/hive/execution/HiveQuerySuite.scala | 12 ++++++++++++ 3 files changed, 24 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/boolean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 76ddeba9cb312..9887856b9c1c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -231,10 +231,20 @@ trait HiveTypeCoercion { * Changes Boolean values to Bytes so that expressions like true < false can be Evaluated. */ object BooleanComparisons extends Rule[LogicalPlan] { + val trueValues = Seq(1, 1L, 1.toByte, 1.toShort, BigDecimal(1)).map(Literal(_)) + val falseValues = Seq(0, 0L, 0.toByte, 0.toShort, BigDecimal(0)).map(Literal(_)) + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - // No need to change EqualTo operators as that actually makes sense for boolean types. + + // Hive treats (true = 1) as true and (false = 0) as true. + case EqualTo(l @ BooleanType(), r) if trueValues.contains(r) => l + case EqualTo(l, r @ BooleanType()) if trueValues.contains(l) => r + case EqualTo(l @ BooleanType(), r) if falseValues.contains(r) => Not(l) + case EqualTo(l, r @ BooleanType()) if falseValues.contains(l) => Not(r) + + // No need to change other EqualTo operators as that actually makes sense for boolean types. case e: EqualTo => e // Otherwise turn them to Byte types so that there exists and ordering. case p: BinaryComparison diff --git a/sql/hive/src/test/resources/golden/boolean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 b/sql/hive/src/test/resources/golden/boolean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 new file mode 100644 index 0000000000000..4d1ebdcde2c71 --- /dev/null +++ b/sql/hive/src/test/resources/golden/boolean = number-0-6b6975fa1892cc48edd87dc0df48a7c0 @@ -0,0 +1 @@ +true true true true true true false false false false false false false false false false false false true true true true true true false false false false false false false false false false false false diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index eb7df717284ce..6f36a4f8cb905 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -30,6 +30,18 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("boolean = number", + """ + |SELECT + | 1 = true, 1L = true, 1Y = true, true = 1, true = 1L, true = 1Y, + | 0 = true, 0L = true, 0Y = true, true = 0, true = 0L, true = 0Y, + | 1 = false, 1L = false, 1Y = false, false = 1, false = 1L, false = 1Y, + | 0 = false, 0L = false, 0Y = false, false = 0, false = 0L, false = 0Y, + | 2 = true, 2L = true, 2Y = true, true = 2, true = 2L, true = 2Y, + | 2 = false, 2L = false, 2Y = false, false = 2, false = 2L, false = 2Y + |FROM src LIMIT 1 + """.stripMargin) + test("CREATE TABLE AS runs once") { hql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect() assert(hql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1, From e34922a221738bae1195d8ace90369c9ddc3a48d Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 23 Jul 2014 23:09:25 -0700 Subject: [PATCH 0057/1492] SPARK-2310. Support arbitrary Spark properties on the command line with ... ...spark-submit The PR allows invocations like spark-submit --class org.MyClass --spark.shuffle.spill false myjar.jar Author: Sandy Ryza Closes #1253 from sryza/sandy-spark-2310 and squashes the following commits: 1dc9855 [Sandy Ryza] More doc and cleanup 00edfb9 [Sandy Ryza] Review comments 91b244a [Sandy Ryza] Change format to --conf PROP=VALUE 8fabe77 [Sandy Ryza] SPARK-2310. Support arbitrary Spark properties on the command line with spark-submit --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 3 +++ .../apache/spark/deploy/SparkSubmitArguments.scala | 11 +++++++++++ .../org/apache/spark/deploy/SparkSubmitSuite.scala | 12 +++++++++++- docs/configuration.md | 8 +++++--- docs/submitting-applications.md | 2 ++ 5 files changed, 32 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3d8373d8175ee..3b5642b6caa36 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -269,6 +269,9 @@ object SparkSubmit { sysProps.getOrElseUpdate(k, v) } + // Spark properties included on command line take precedence + sysProps ++= args.sparkProperties + (childArgs, childClasspath, sysProps, childMainClass) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 57655aa4c32b1..3ab67a43a3b55 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -55,6 +55,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { var verbose: Boolean = false var isPython: Boolean = false var pyFiles: String = null + val sparkProperties: HashMap[String, String] = new HashMap[String, String]() parseOpts(args.toList) loadDefaults() @@ -177,6 +178,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | executorCores $executorCores | totalExecutorCores $totalExecutorCores | propertiesFile $propertiesFile + | extraSparkProperties $sparkProperties | driverMemory $driverMemory | driverCores $driverCores | driverExtraClassPath $driverExtraClassPath @@ -290,6 +292,13 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { jars = Utils.resolveURIs(value) parse(tail) + case ("--conf" | "-c") :: value :: tail => + value.split("=", 2).toSeq match { + case Seq(k, v) => sparkProperties(k) = v + case _ => SparkSubmit.printErrorAndExit(s"Spark config without '=': $value") + } + parse(tail) + case ("--help" | "-h") :: tail => printUsageAndExit(0) @@ -349,6 +358,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | on the PYTHONPATH for Python apps. | --files FILES Comma-separated list of files to be placed in the working | directory of each executor. + | + | --conf PROP=VALUE Arbitrary Spark configuration property. | --properties-file FILE Path to a file from which to load extra properties. If not | specified, this will look for conf/spark-defaults.conf. | diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 565c53e9529ff..f497a5e0a14f0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -120,6 +120,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "beauty", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -139,6 +140,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { mainClass should be ("org.apache.spark.deploy.yarn.Client") classpath should have length (0) sysProps("spark.app.name") should be ("beauty") + sysProps("spark.shuffle.spill") should be ("false") sysProps("SPARK_SUBMIT") should be ("true") } @@ -156,6 +158,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "--name", "trill", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -176,6 +179,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { sysProps("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") sysProps("spark.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") sysProps("SPARK_SUBMIT") should be ("true") + sysProps("spark.shuffle.spill") should be ("false") } test("handles standalone cluster mode") { @@ -186,6 +190,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--supervise", "--driver-memory", "4g", "--driver-cores", "5", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -195,9 +200,10 @@ class SparkSubmitSuite extends FunSuite with Matchers { childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have size (0) - sysProps should have size (2) + sysProps should have size (3) sysProps.keys should contain ("spark.jars") sysProps.keys should contain ("SPARK_SUBMIT") + sysProps("spark.shuffle.spill") should be ("false") } test("handles standalone client mode") { @@ -208,6 +214,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--total-executor-cores", "5", "--class", "org.SomeClass", "--driver-memory", "4g", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -218,6 +225,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") + sysProps("spark.shuffle.spill") should be ("false") } test("handles mesos client mode") { @@ -228,6 +236,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { "--total-executor-cores", "5", "--class", "org.SomeClass", "--driver-memory", "4g", + "--conf", "spark.shuffle.spill=false", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) @@ -238,6 +247,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { classpath(0) should endWith ("thejar.jar") sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") + sysProps("spark.shuffle.spill") should be ("false") } test("launch simple application with spark-submit") { diff --git a/docs/configuration.md b/docs/configuration.md index 02af461267c46..cb0c65e2d2200 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -42,13 +42,15 @@ val sc = new SparkContext(new SparkConf()) Then, you can supply configuration values at runtime: {% highlight bash %} -./bin/spark-submit --name "My fancy app" --master local[4] myApp.jar +./bin/spark-submit --name "My app" --master local[4] --conf spark.shuffle.spill=false + --conf "spark.executor.extraJavaOptions=-XX:+PrintGCDetails -XX:+PrintGCTimeStamps" myApp.jar {% endhighlight %} The Spark shell and [`spark-submit`](cluster-overview.html#launching-applications-with-spark-submit) tool support two ways to load configurations dynamically. The first are command line options, -such as `--master`, as shown above. Running `./bin/spark-submit --help` will show the entire list -of options. +such as `--master`, as shown above. `spark-submit` can accept any Spark property using the `--conf` +flag, but uses special flags for properties that play a part in launching the Spark application. +Running `./bin/spark-submit --help` will show the entire list of these options. `bin/spark-submit` will also read configuration options from `conf/spark-defaults.conf`, in which each line consists of a key and a value separated by whitespace. For example: diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index e05883072bfa8..45b70b1a5457a 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -33,6 +33,7 @@ dependencies, and can support different cluster managers and deploy modes that S --class --master \ --deploy-mode \ + --conf = \ ... # other options \ [application-arguments] @@ -43,6 +44,7 @@ Some of the commonly used options are: * `--class`: The entry point for your application (e.g. `org.apache.spark.examples.SparkPi`) * `--master`: The [master URL](#master-urls) for the cluster (e.g. `spark://23.195.26.187:7077`) * `--deploy-mode`: Whether to deploy your driver on the worker nodes (`cluster`) or locally as an external client (`client`) (default: `client`)* +* `--conf`: Arbitrary Spark configuration property in key=value format. For values that contain spaces wrap "key=value" in quotes (as shown). * `application-jar`: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. * `application-arguments`: Arguments passed to the main method of your main class, if any From 42dfab7d374cf64a39b692ebc089792a4ff7e42c Mon Sep 17 00:00:00 2001 From: Daoyuan Date: Thu, 24 Jul 2014 00:09:36 -0700 Subject: [PATCH 0058/1492] [SPARK-2661][bagel]unpersist old processed rdd Unpersist useless rdd during bagel iteration to make full use of memory. Author: Daoyuan Closes #1519 from adrian-wang/bagelunpersist and squashes the following commits: 182c9dd [Daoyuan] rename var nextUseless to lastRDD 87fd3a4 [Daoyuan] bagel unpersist old processed rdd --- bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 70a99b33d753c..ef0bb2ac13f08 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -72,6 +72,7 @@ object Bagel extends Logging { var verts = vertices var msgs = messages var noActivity = false + var lastRDD: RDD[(K, (V, Array[M]))] = null do { logInfo("Starting superstep " + superstep + ".") val startTime = System.currentTimeMillis @@ -83,6 +84,10 @@ object Bagel extends Logging { val superstep_ = superstep // Create a read-only copy of superstep for capture in closure val (processed, numMsgs, numActiveVerts) = comp[K, V, M, C](sc, grouped, compute(_, _, aggregated, superstep_), storageLevel) + if (lastRDD != null) { + lastRDD.unpersist(false) + } + lastRDD = processed val timeTaken = System.currentTimeMillis - startTime logInfo("Superstep %d took %d s".format(superstep, timeTaken / 1000)) From 46e224aaa26df4b232c5176e98472a902862b76c Mon Sep 17 00:00:00 2001 From: Rahul Singhal Date: Thu, 24 Jul 2014 09:31:04 -0500 Subject: [PATCH 0059/1492] SPARK-2150: Provide direct link to finished application UI in yarn resou... ...rce manager UI Use the event logger directory to provide a direct link to finished application UI in yarn resourcemanager UI. Author: Rahul Singhal Closes #1094 from rahulsinghaliitd/SPARK-2150 and squashes the following commits: 95f230c [Rahul Singhal] SPARK-2150: Provide direct link to finished application UI in yarn resource manager UI --- .../spark/deploy/history/FsHistoryProvider.scala | 3 ++- .../spark/deploy/history/HistoryPage.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 4 +++- .../org/apache/spark/deploy/master/Master.scala | 11 +++++++---- .../spark/scheduler/EventLoggingListener.scala | 7 +++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 4 +++- .../spark/deploy/yarn/ExecutorLauncher.scala | 2 +- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala | 16 ++++++++++++++++ .../cluster/YarnClientSchedulerBackend.scala | 3 ++- .../spark/deploy/yarn/ApplicationMaster.scala | 5 +++-- .../spark/deploy/yarn/ExecutorLauncher.scala | 2 +- 11 files changed, 46 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index a8c9ac072449f..01e7065c17b69 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -169,7 +169,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val ui: SparkUI = if (renderUI) { val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) - new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) + new SparkUI(conf, appSecManager, replayBus, appId, + HistoryServer.UI_PATH_PREFIX + s"/$appId") // Do not call ui.bind() to avoid creating a new server for each application } else { null diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index a958c837c2ff6..d7a3e3f120e67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -75,7 +75,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { "Last Updated") private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { - val uiAddress = "/history/" + info.id + val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" val startTime = UIUtils.formatDate(info.startTime) val endTime = UIUtils.formatDate(info.endTime) val duration = UIUtils.formatDuration(info.endTime - info.startTime) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 56b38ddfc9313..cacb9da8c947b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -114,7 +114,7 @@ class HistoryServer( attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) val contextHandler = new ServletContextHandler - contextHandler.setContextPath("/history") + contextHandler.setContextPath(HistoryServer.UI_PATH_PREFIX) contextHandler.addServlet(new ServletHolder(loaderServlet), "/*") attachHandler(contextHandler) } @@ -172,6 +172,8 @@ class HistoryServer( object HistoryServer extends Logging { private val conf = new SparkConf + val UI_PATH_PREFIX = "/history" + def main(argStrings: Array[String]) { SignalLogger.register(log) initSecurity() 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 bb1fcc8190fe4..21f8667819c44 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 @@ -35,6 +35,7 @@ import akka.serialization.SerializationExtension import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI @@ -664,9 +665,10 @@ private[spark] class Master( */ def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name + val notFoundBasePath = HistoryServer.UI_PATH_PREFIX + "/not-found" val eventLogDir = app.desc.eventLogDir.getOrElse { // Event logging is not enabled for this application - app.desc.appUiUrl = "/history/not-found" + app.desc.appUiUrl = notFoundBasePath return false } val fileSystem = Utils.getHadoopFileSystem(eventLogDir) @@ -681,13 +683,14 @@ private[spark] class Master( logWarning(msg) msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") - app.desc.appUiUrl = s"/history/not-found?msg=$msg&title=$title" + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&title=$title" return false } try { val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) + val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)", + HistoryServer.UI_PATH_PREFIX + s"/${app.id}") replayBus.replay() appIdToUI(app.id) = ui webUi.attachSparkUI(ui) @@ -702,7 +705,7 @@ private[spark] class Master( var msg = s"Exception in replaying log for application $appName!" logError(msg, e) msg = URLEncoder.encode(msg, "UTF-8") - app.desc.appUiUrl = s"/history/not-found?msg=$msg&exception=$exception&title=$title" + app.desc.appUiUrl = notFoundBasePath + s"?msg=$msg&exception=$exception&title=$title" false } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index a90b0d475c04e..ae6ca9f4e7bf5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -63,6 +63,13 @@ private[spark] class EventLoggingListener( // For testing. Keep track of all JSON serialized events that have been logged. private[scheduler] val loggedEvents = new ArrayBuffer[JValue] + /** + * Return only the unique application directory without the base directory. + */ + def getApplicationLogDir(): String = { + name + } + /** * Begin logging events. * If compression is used, log a file that indicates which compression library is used. 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 3ec36487dcd26..62b5c3bc5f0f3 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 @@ -60,6 +60,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var yarnAllocator: YarnAllocationHandler = _ private var isFinished: Boolean = false private var uiAddress: String = _ + private var uiHistoryAddress: String = _ private val maxAppAttempts: Int = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true @@ -237,6 +238,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, if (null != sparkContext) { uiAddress = sparkContext.ui.appUIHostPort + uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf) this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, resourceManager, @@ -360,7 +362,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, finishReq.setAppAttemptId(appAttemptId) finishReq.setFinishApplicationStatus(status) finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", "")) + finishReq.setTrackingUrl(uiHistoryAddress) resourceManager.finishApplicationMaster(finishReq) } } 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 a86ad256dfa39..d232c18d2f5a4 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 @@ -289,7 +289,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp .asInstanceOf[FinishApplicationMasterRequest] finishReq.setAppAttemptId(appAttemptId) finishReq.setFinishApplicationStatus(status) - finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", "")) + finishReq.setTrackingUrl(sparkConf.get("spark.driver.appUIHistoryAddress", "")) resourceManager.finishApplicationMaster(finishReq) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 718cb19f57261..e98308cdbd74e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -30,6 +30,9 @@ import org.apache.hadoop.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.SparkHadoopUtil /** @@ -132,4 +135,17 @@ object YarnSparkHadoopUtil { } } + def getUIHistoryAddress(sc: SparkContext, conf: SparkConf) : String = { + val eventLogDir = sc.eventLogger match { + case Some(logger) => logger.getApplicationLogDir() + case None => "" + } + val historyServerAddress = conf.get("spark.yarn.historyServer.address", "") + if (historyServerAddress != "" && eventLogDir != "") { + historyServerAddress + HistoryServer.UI_PATH_PREFIX + s"/$eventLogDir" + } else { + "" + } + } + } 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 d8266f7b0c9a7..77b91f8e260fe 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 @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher} +import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher, YarnSparkHadoopUtil} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -54,6 +54,7 @@ private[spark] class YarnClientSchedulerBackend( val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort) + conf.set("spark.driver.appUIHistoryAddress", YarnSparkHadoopUtil.getUIHistoryAddress(sc, conf)) val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( 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 eaf594c8b49b9..035356d390c80 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 @@ -59,6 +59,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var yarnAllocator: YarnAllocationHandler = _ private var isFinished: Boolean = false private var uiAddress: String = _ + private var uiHistoryAddress: String = _ private val maxAppAttempts: Int = conf.getInt( YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) private var isLastAMRetry: Boolean = true @@ -216,6 +217,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, if (sparkContext != null) { uiAddress = sparkContext.ui.appUIHostPort + uiHistoryAddress = YarnSparkHadoopUtil.getUIHistoryAddress(sparkContext, sparkConf) this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, amClient, @@ -312,8 +314,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, logInfo("Unregistering ApplicationMaster with " + status) if (registered) { - val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "") - amClient.unregisterApplicationMaster(status, diagnostics, trackingUrl) + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) } } } 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 5ac95f3798723..7158d9442a459 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 @@ -250,7 +250,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp def finishApplicationMaster(status: FinalApplicationStatus) { logInfo("Unregistering ApplicationMaster with " + status) - val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "") + val trackingUrl = sparkConf.get("spark.driver.appUIHistoryAddress", "") amClient.unregisterApplicationMaster(status, "" /* appMessage */ , trackingUrl) } From 9fd141477dca3beb5cc2f82305dcf9f93ba41e9a Mon Sep 17 00:00:00 2001 From: tzolov Date: Thu, 24 Jul 2014 11:12:25 -0700 Subject: [PATCH 0060/1492] [Build] SPARK-2619: Configurable filemode for the spark/bin folder in debian package Add a `744` property to the `assembly/pom.xml` that defaults to `744`. Use this property for ../bin folder . This patch doesn't change the current default modes but allows one override the modes at build time: `-Ddeb.bin.filemode=` Author: tzolov Closes #1531 from tzolov/SPARK-2619 and squashes the following commits: 6d95343 [tzolov] [Build] SPARK-2619: Configurable filemode for the spark/bin folder in the .deb package --- assembly/pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4f6aade133db7..567a8dd2a0d94 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -39,6 +39,7 @@ spark /usr/share/spark root + 744 @@ -276,7 +277,7 @@ ${deb.user} ${deb.user} ${deb.install.path}/bin - 744 + ${deb.bin.filemode} From b352ef175c234a2ea86b72c2f40da2ac69658b2e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 24 Jul 2014 11:19:19 -0700 Subject: [PATCH 0061/1492] [SPARK-2603][SQL] Remove unnecessary toMap and toList in converting Java collections to Scala collections JsonRDD.scala In JsonRDD.scalafy, we are using toMap/toList to convert a Java Map/List to a Scala one. These two operations are pretty expensive because they read elements from a Java Map/List and then load to a Scala Map/List. We can use Scala wrappers to wrap those Java collections instead of using toMap/toList. I did a quick test to see the performance. I had a 2.9GB cached RDD[String] storing one JSON object per record (twitter dataset). My simple test program is attached below. ```scala val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ val jsonData = sc.textFile("...") jsonData.cache.count val jsonSchemaRDD = sqlContext.jsonRDD(jsonData) jsonSchemaRDD.registerAsTable("jt") sqlContext.sql("select count(*) from jt").collect ``` Stages for the schema inference and the table scan both had 48 tasks. These tasks were executed sequentially. For the current implementation, scanning the JSON dataset will materialize values of all fields of a record. The inferred schema of the dataset can be accessed at https://gist.github.com/yhuai/05fe8a57c638c6666f8d. From the result, there was no significant difference on running `jsonRDD`. For the simple aggregation query, results are attached below. ``` Original: Run 1: 26.1s Run 2: 27.03s Run 3: 27.035s With this change: Run 1: 21.086s Run 2: 21.035s Run 3: 21.029s ``` JIRA: https://issues.apache.org/jira/browse/SPARK-2603 Author: Yin Huai Closes #1504 from yhuai/removeToMapToList and squashes the following commits: 6831b77 [Yin Huai] Fix failed tests. 09b9bca [Yin Huai] Merge remote-tracking branch 'upstream/master' into removeToMapToList d1abdb8 [Yin Huai] Remove unnecessary toMap and toList. --- .../scala/org/apache/spark/sql/json/JsonRDD.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index df80dfb98b93c..b48c70ee73a27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.json -import scala.collection.JavaConversions._ +import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal import com.fasterxml.jackson.databind.ObjectMapper @@ -210,12 +210,12 @@ private[sql] object JsonRDD extends Logging { case (k, dataType) => (s"$key.$k", dataType) } ++ Set((key, StructType(Nil))) } - case (key: String, array: List[_]) => { + case (key: String, array: Seq[_]) => { // The value associated with the key is an array. typeOfArray(array) match { case ArrayType(StructType(Nil)) => { // The elements of this arrays are structs. - array.asInstanceOf[List[Map[String, Any]]].flatMap { + array.asInstanceOf[Seq[Map[String, Any]]].flatMap { element => allKeysWithValueTypes(element) }.map { case (k, dataType) => (s"$key.$k", dataType) @@ -229,7 +229,7 @@ private[sql] object JsonRDD extends Logging { } /** - * Converts a Java Map/List to a Scala Map/List. + * Converts a Java Map/List to a Scala Map/Seq. * We do not use Jackson's scala module at here because * DefaultScalaModule in jackson-module-scala will make * the parsing very slow. @@ -239,9 +239,9 @@ private[sql] object JsonRDD extends Logging { // .map(identity) is used as a workaround of non-serializable Map // generated by .mapValues. // This issue is documented at https://issues.scala-lang.org/browse/SI-7005 - map.toMap.mapValues(scalafy).map(identity) + JMapWrapper(map).mapValues(scalafy).map(identity) case list: java.util.List[_] => - list.toList.map(scalafy) + JListWrapper(list).map(scalafy) case atom => atom } From c960b5051853f336fb01ea3f16567b9958baa1b6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 24 Jul 2014 12:37:02 -0700 Subject: [PATCH 0062/1492] [SPARK-2479 (partial)][MLLIB] fix binary metrics unit tests Allow small errors in comparison. @dbtsai , this unit test blocks https://github.com/apache/spark/pull/1562 . I may need to merge this one first. We can change it to use the tools in https://github.com/apache/spark/pull/1425 after that PR gets merged. Author: Xiangrui Meng Closes #1576 from mengxr/fix-binary-metrics-unit-tests and squashes the following commits: 5076a7f [Xiangrui Meng] fix binary metrics unit tests --- .../BinaryClassificationMetricsSuite.scala | 36 ++++++++++++++----- 1 file changed, 27 insertions(+), 9 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 9d16182f9d8c4..94db1dc183230 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -20,8 +20,26 @@ package org.apache.spark.mllib.evaluation import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils.DoubleWithAlmostEquals class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { + + // TODO: move utility functions to TestingUtils. + + def elementsAlmostEqual(actual: Seq[Double], expected: Seq[Double]): Boolean = { + actual.zip(expected).forall { case (x1, x2) => + x1.almostEquals(x2) + } + } + + def elementsAlmostEqual( + actual: Seq[(Double, Double)], + expected: Seq[(Double, Double)])(implicit dummy: DummyImplicit): Boolean = { + actual.zip(expected).forall { case ((x1, y1), (x2, y2)) => + x1.almostEquals(x2) && y1.almostEquals(y2) + } + } + test("binary evaluation metrics") { val scoreAndLabels = sc.parallelize( Seq((0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)), 2) @@ -41,14 +59,14 @@ class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { val prCurve = Seq((0.0, 1.0)) ++ pr val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r) } val f2 = pr.map { case (r, p) => 5.0 * (p * r) / (4.0 * p + r)} - assert(metrics.thresholds().collect().toSeq === threshold) - assert(metrics.roc().collect().toSeq === rocCurve) - assert(metrics.areaUnderROC() === AreaUnderCurve.of(rocCurve)) - assert(metrics.pr().collect().toSeq === prCurve) - assert(metrics.areaUnderPR() === AreaUnderCurve.of(prCurve)) - assert(metrics.fMeasureByThreshold().collect().toSeq === threshold.zip(f1)) - assert(metrics.fMeasureByThreshold(2.0).collect().toSeq === threshold.zip(f2)) - assert(metrics.precisionByThreshold().collect().toSeq === threshold.zip(precision)) - assert(metrics.recallByThreshold().collect().toSeq === threshold.zip(recall)) + assert(elementsAlmostEqual(metrics.thresholds().collect(), threshold)) + assert(elementsAlmostEqual(metrics.roc().collect(), rocCurve)) + assert(metrics.areaUnderROC().almostEquals(AreaUnderCurve.of(rocCurve))) + assert(elementsAlmostEqual(metrics.pr().collect(), prCurve)) + assert(metrics.areaUnderPR().almostEquals(AreaUnderCurve.of(prCurve))) + assert(elementsAlmostEqual(metrics.fMeasureByThreshold().collect(), threshold.zip(f1))) + assert(elementsAlmostEqual(metrics.fMeasureByThreshold(2.0).collect(), threshold.zip(f2))) + assert(elementsAlmostEqual(metrics.precisionByThreshold().collect(), threshold.zip(precision))) + assert(elementsAlmostEqual(metrics.recallByThreshold().collect(), threshold.zip(recall))) } } From 323a83c5235f9289cd9526491d62365df96a429b Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Thu, 24 Jul 2014 14:46:10 -0500 Subject: [PATCH 0063/1492] [SPARK-2037]: yarn client mode doesn't support spark.yarn.max.executor.failures Author: GuoQiang Li Closes #1180 from witgo/SPARK-2037 and squashes the following commits: 3d52411 [GuoQiang Li] review commit 7058f4d [GuoQiang Li] Correctly stop SparkContext 6d0561f [GuoQiang Li] Fix: yarn client mode doesn't support spark.yarn.max.executor.failures --- .../spark/deploy/yarn/ExecutorLauncher.scala | 80 ++++++++++++------- .../cluster/YarnClientSchedulerBackend.scala | 28 +++++++ .../spark/deploy/yarn/ExecutorLauncher.scala | 45 ++++++++--- 3 files changed, 115 insertions(+), 38 deletions(-) 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 d232c18d2f5a4..184e2ad6c82cd 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ -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 @@ -57,10 +56,17 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = _ - private var driverClosed:Boolean = false + + private var driverClosed: Boolean = false + private var isFinished: Boolean = false + private var registered: Boolean = false + + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) val securityManager = new SecurityManager(sparkConf) - val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, + val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityManager)._1 var actor: ActorRef = _ @@ -97,23 +103,26 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() - val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() - - // Compute number of threads for akka - val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() - - if (minimumMemory > 0) { - val mem = args.executorMemory + sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnAllocationHandler.MEMORY_OVERHEAD) - val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) - - if (numCore > 0) { - // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 - // TODO: Uncomment when hadoop is on a version which has this fixed. - // args.workerCores = numCore + synchronized { + if (!isFinished) { + val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + // Compute number of threads for akka + val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() + + if (minimumMemory > 0) { + val mem = args.executorMemory + sparkConf.getInt("spark.yarn.executor.memoryOverhead", + YarnAllocationHandler.MEMORY_OVERHEAD) + val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) + + if (numCore > 0) { + // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 + // TODO: Uncomment when hadoop is on a version which has this fixed. + // args.workerCores = numCore + } + } + registered = true } } - waitForSparkMaster() addAmIpFilter() // Allocate all containers @@ -243,11 +252,17 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) + checkNumExecutorsFailed() Thread.sleep(100) } logInfo("All executors have launched.") - + } + private def checkNumExecutorsFailed() { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of executor failures reached") + } } // TODO: We might want to extend this to allocate more containers in case they die ! @@ -257,6 +272,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp val t = new Thread { override def run() { while (!driverClosed) { + checkNumExecutorsFailed() val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning if (missingExecutorCount > 0) { logInfo("Allocating " + missingExecutorCount + @@ -282,15 +298,23 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp yarnAllocator.allocateContainers(0) } - def finishApplicationMaster(status: FinalApplicationStatus) { - - logInfo("finish ApplicationMaster with " + status) - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - finishReq.setTrackingUrl(sparkConf.get("spark.driver.appUIHistoryAddress", "")) - resourceManager.finishApplicationMaster(finishReq) + def finishApplicationMaster(status: FinalApplicationStatus, appMessage: String = "") { + synchronized { + if (isFinished) { + return + } + logInfo("Unregistering ApplicationMaster with " + status) + if (registered) { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + finishReq.setTrackingUrl(sparkConf.get("spark.yarn.historyServer.address", "")) + finishReq.setDiagnostics(appMessage) + resourceManager.finishApplicationMaster(finishReq) + } + isFinished = true + } } } 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 77b91f8e260fe..f8fb96b312f23 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 @@ -37,6 +37,8 @@ private[spark] class YarnClientSchedulerBackend( var client: Client = null var appId: ApplicationId = null + var checkerThread: Thread = null + var stopping: Boolean = false private[spark] def addArg(optionName: String, envVar: String, sysProp: String, arrayBuf: ArrayBuffer[String]) { @@ -86,6 +88,7 @@ private[spark] class YarnClientSchedulerBackend( client = new Client(args, conf) appId = client.runApp() waitForApp() + checkerThread = yarnApplicationStateCheckerThread() } def waitForApp() { @@ -116,7 +119,32 @@ private[spark] class YarnClientSchedulerBackend( } } + private def yarnApplicationStateCheckerThread(): Thread = { + val t = new Thread { + override def run() { + while (!stopping) { + val report = client.getApplicationReport(appId) + val state = report.getYarnApplicationState() + if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.KILLED + || state == YarnApplicationState.FAILED) { + logError(s"Yarn application already ended: $state") + sc.stop() + stopping = true + } + Thread.sleep(1000L) + } + checkerThread = null + Thread.currentThread().interrupt() + } + } + t.setName("Yarn Application State Checker") + t.setDaemon(true) + t.start() + t + } + override def stop() { + stopping = true super.stop() client.stop logInfo("Stopped") 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 7158d9442a459..fc7b8320d734d 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 @@ -19,15 +19,12 @@ package org.apache.spark.deploy.yarn import java.net.Socket import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.yarn.api._ +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ -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 @@ -57,10 +54,16 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp private val yarnConf: YarnConfiguration = new YarnConfiguration(conf) private var yarnAllocator: YarnAllocationHandler = _ - private var driverClosed:Boolean = false + private var driverClosed: Boolean = false + private var isFinished: Boolean = false + private var registered: Boolean = false private var amClient: AMRMClient[ContainerRequest] = _ + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) + val securityManager = new SecurityManager(sparkConf) val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, conf = sparkConf, securityManager = securityManager)._1 @@ -101,7 +104,12 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp amClient.start() appAttemptId = ApplicationMaster.getApplicationAttemptId() - registerApplicationMaster() + synchronized { + if (!isFinished) { + registerApplicationMaster() + registered = true + } + } waitForSparkMaster() addAmIpFilter() @@ -210,6 +218,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp yarnAllocator.addResourceRequests(args.numExecutors) yarnAllocator.allocateResources() while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() Thread.sleep(100) @@ -228,12 +237,20 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } } + private def checkNumExecutorsFailed() { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of executor failures reached") + } + } + private def launchReporterThread(_sleepTime: Long): Thread = { val sleepTime = if (_sleepTime <= 0) 0 else _sleepTime val t = new Thread { override def run() { while (!driverClosed) { + checkNumExecutorsFailed() allocateMissingExecutor() logDebug("Sending progress") yarnAllocator.allocateResources() @@ -248,10 +265,18 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp t } - def finishApplicationMaster(status: FinalApplicationStatus) { - logInfo("Unregistering ApplicationMaster with " + status) - val trackingUrl = sparkConf.get("spark.driver.appUIHistoryAddress", "") - amClient.unregisterApplicationMaster(status, "" /* appMessage */ , trackingUrl) + def finishApplicationMaster(status: FinalApplicationStatus, appMessage: String = "") { + synchronized { + if (isFinished) { + return + } + logInfo("Unregistering ApplicationMaster with " + status) + if (registered) { + val trackingUrl = sparkConf.get("spark.yarn.historyServer.address", "") + amClient.unregisterApplicationMaster(status, appMessage, trackingUrl) + } + isFinished = true + } } } From fec641b84d37848b329bba91857240ac5b87fc54 Mon Sep 17 00:00:00 2001 From: Neville Li Date: Thu, 24 Jul 2014 14:13:00 -0700 Subject: [PATCH 0064/1492] SPARK-2250: show stage RDDs in UI Author: Neville Li Closes #1188 from nevillelyh/neville/ui and squashes the following commits: d3ac425 [Neville Li] SPARK-2250: show persisted RDD in stage UI f075db9 [Neville Li] SPARK-2035: show call stack even when description is available --- .../org/apache/spark/ui/jobs/StageTable.scala | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5f45c0ced5ec5..f8b308c981548 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -18,6 +18,7 @@ package org.apache.spark.ui.jobs import scala.xml.Node +import scala.xml.Text import java.util.Date @@ -99,19 +100,30 @@ private[ui] class StageTableBase( {s.name} + val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) val details = if (s.details.nonEmpty) { - +show details - - + +details + ++ + } val stageDataOption = listener.stageIdToData.get(s.stageId) // Too many nested map/flatMaps with options are just annoying to read. Do this imperatively. if (stageDataOption.isDefined && stageDataOption.get.description.isDefined) { val desc = stageDataOption.get.description -
{desc}
{nameLink} {killLink}
+
{desc}
{killLink} {nameLink} {details}
} else {
{killLink} {nameLink} {details}
} From a45d5480f65d2e969fc7fbd8f358b1717fb99bef Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 24 Jul 2014 15:59:09 -0700 Subject: [PATCH 0065/1492] [SPARK-2464][Streaming] Fixed Twitter stream stopping bug Stopping the Twitter Receiver would call twitter4j's TwitterStream.shutdown, which in turn causes an Exception to be thrown to the listener. This exception caused the Receiver to be restarted. This patch check whether the receiver was stopped or not, and accordingly restarts on exception. Author: Tathagata Das Closes #1577 from tdas/twitter-stop and squashes the following commits: 011b525 [Tathagata Das] Fixed Twitter stream stopping bug. --- .../spark/streaming/twitter/TwitterInputDStream.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 5ea2e5549d7df..4eacc47da5699 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -63,7 +63,8 @@ class TwitterReceiver( storageLevel: StorageLevel ) extends Receiver[Status](storageLevel) with Logging { - private var twitterStream: TwitterStream = _ + @volatile private var twitterStream: TwitterStream = _ + @volatile private var stopped = false def onStart() { try { @@ -78,7 +79,9 @@ class TwitterReceiver( def onScrubGeo(l: Long, l1: Long) {} def onStallWarning(stallWarning: StallWarning) {} def onException(e: Exception) { - restart("Error receiving tweets", e) + if (!stopped) { + restart("Error receiving tweets", e) + } } }) @@ -91,12 +94,14 @@ class TwitterReceiver( } setTwitterStream(newTwitterStream) logInfo("Twitter receiver started") + stopped = false } catch { case e: Exception => restart("Error starting Twitter stream", e) } } def onStop() { + stopped = true setTwitterStream(null) logInfo("Twitter receiver stopped") } From eff9714e1c88e39e28317358ca9ec87677f121dc Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 24 Jul 2014 18:15:37 -0700 Subject: [PATCH 0066/1492] [SPARK-2014] Make PySpark store RDDs in MEMORY_ONLY_SER with compression by default Author: Prashant Sharma Closes #1051 from ScrapCodes/SPARK-2014/pyspark-cache and squashes the following commits: f192df7 [Prashant Sharma] Code Review 2a2f43f [Prashant Sharma] [SPARK-2014] Make PySpark store RDDs in MEMORY_ONLY_SER with compression by default --- python/pyspark/conf.py | 6 ++++++ python/pyspark/context.py | 2 +- python/pyspark/rdd.py | 4 ++-- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index b50590ab3b444..b4c82f519bd53 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -100,6 +100,12 @@ def set(self, key, value): self._jconf.set(key, unicode(value)) return self + def setIfMissing(self, key, value): + """Set a configuration property, if not already set.""" + if self.get(key) is None: + self.set(key, value) + return self + def setMaster(self, value): """Set master URL to connect to.""" self._jconf.setMaster(value) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index e21be0e10a3f7..024fb881877c9 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -101,7 +101,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, else: self.serializer = BatchedSerializer(self._unbatched_serializer, batchSize) - + self._conf.setIfMissing("spark.rdd.compress", "true") # Set any parameters passed directly to us on the conf if master: self._conf.setMaster(master) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 94ba22306afbd..a38dd0b9237c5 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -231,10 +231,10 @@ def context(self): def cache(self): """ - Persist this RDD with the default storage level (C{MEMORY_ONLY}). + Persist this RDD with the default storage level (C{MEMORY_ONLY_SER}). """ self.is_cached = True - self._jrdd.cache() + self.persist(StorageLevel.MEMORY_ONLY_SER) return self def persist(self, storageLevel): From 14174abd421318e71c16edd24224fd5094bdfed4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 24 Jul 2014 22:53:47 -0700 Subject: [PATCH 0067/1492] [SPARK-2538] [PySpark] Hash based disk spilling aggregation During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition). Author: Davies Liu Closes #1460 from davies/spill and squashes the following commits: cad91bf [Davies Liu] call gc.collect() after data.clear() to release memory as much as possible. 37d71f7 [Davies Liu] balance the partitions 902f036 [Davies Liu] add shuffle.py into run-tests dcf03a9 [Davies Liu] fix memory_info() of psutil 67e6eba [Davies Liu] comment for MAX_TOTAL_PARTITIONS f6bd5d6 [Davies Liu] rollback next_limit() again, the performance difference is huge: e74b785 [Davies Liu] fix code style and change next_limit to memory_limit 400be01 [Davies Liu] address all the comments 6178844 [Davies Liu] refactor and improve docs fdd0a49 [Davies Liu] add long doc string for ExternalMerger 1a97ce4 [Davies Liu] limit used memory and size of objects in partitionBy() e6cc7f9 [Davies Liu] Merge branch 'master' into spill 3652583 [Davies Liu] address comments e78a0a0 [Davies Liu] fix style 24cec6a [Davies Liu] get local directory by SPARK_LOCAL_DIR 57ee7ef [Davies Liu] update docs 286aaff [Davies Liu] let spilled aggregation in Python configurable e9a40f6 [Davies Liu] recursive merger 6edbd1f [Davies Liu] Hash based disk spilling aggregation --- .../apache/spark/api/python/PythonRDD.scala | 5 +- .../spark/storage/DiskBlockManager.scala | 2 +- docs/configuration.md | 9 + python/epydoc.conf | 2 +- python/pyspark/rdd.py | 92 +++- python/pyspark/serializers.py | 29 +- python/pyspark/shuffle.py | 439 ++++++++++++++++++ python/pyspark/tests.py | 57 +++ python/run-tests | 1 + 9 files changed, 611 insertions(+), 25 deletions(-) create mode 100644 python/pyspark/shuffle.py diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 462e09466bfa6..d6b0988641a97 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -57,7 +57,10 @@ private[spark] class PythonRDD[T: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis val env = SparkEnv.get - val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) + val localdir = env.blockManager.diskBlockManager.localDirs.map( + f => f.getPath()).mkString(",") + val worker: Socket = env.createPythonWorker(pythonExec, + envVars.toMap + ("SPARK_LOCAL_DIR" -> localdir)) // Start a thread to feed the process input from our parent's iterator val writerThread = new WriterThread(env, worker, split, context) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 673fc19c060a4..2e7ed7538e6e5 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -43,7 +43,7 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid * having really large inodes at the top level. */ - private val localDirs: Array[File] = createLocalDirs() + val localDirs: Array[File] = createLocalDirs() if (localDirs.isEmpty) { logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) diff --git a/docs/configuration.md b/docs/configuration.md index cb0c65e2d2200..dac8bb1d52468 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -197,6 +197,15 @@ Apart from these, the following properties are also available, and may be useful Spark's dependencies and user dependencies. It is currently an experimental feature. + + spark.python.worker.memory + 512m + + Amount of memory to use per python worker process during aggregation, in the same + format as JVM memory strings (e.g. 512m, 2g). If the memory + used during aggregation goes above this amount, it will spill the data into disks. + + #### Shuffle Behavior diff --git a/python/epydoc.conf b/python/epydoc.conf index b73860bad8263..51c0faf359939 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -35,4 +35,4 @@ private: no exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.java_gateway pyspark.examples pyspark.shell pyspark.tests pyspark.rddsampler pyspark.daemon pyspark.mllib._common - pyspark.mllib.tests + pyspark.mllib.tests pyspark.shuffle diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index a38dd0b9237c5..7ad6108261444 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -42,6 +42,8 @@ from pyspark.rddsampler import RDDSampler from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable +from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ + get_used_memory from py4j.java_collections import ListConverter, MapConverter @@ -197,6 +199,22 @@ def _replaceRoot(self, value): self._sink(1) +def _parse_memory(s): + """ + Parse a memory string in the format supported by Java (e.g. 1g, 200m) and + return the value in MB + + >>> _parse_memory("256m") + 256 + >>> _parse_memory("2g") + 2048 + """ + units = {'g': 1024, 'm': 1, 't': 1 << 20, 'k': 1.0 / 1024} + if s[-1] not in units: + raise ValueError("invalid format: " + s) + return int(float(s[:-1]) * units[s[-1].lower()]) + + class RDD(object): """ @@ -1207,20 +1225,49 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash): if numPartitions is None: numPartitions = self._defaultReducePartitions() - # Transferring O(n) objects to Java is too expensive. Instead, we'll - # form the hash buckets in Python, transferring O(numPartitions) objects - # to Java. Each object is a (splitNumber, [objects]) pair. + # Transferring O(n) objects to Java is too expensive. + # Instead, we'll form the hash buckets in Python, + # transferring O(numPartitions) objects to Java. + # Each object is a (splitNumber, [objects]) pair. + # In order to avoid too huge objects, the objects are + # grouped into chunks. outputSerializer = self.ctx._unbatched_serializer + limit = (_parse_memory(self.ctx._conf.get( + "spark.python.worker.memory", "512m")) / 2) + def add_shuffle_key(split, iterator): buckets = defaultdict(list) + c, batch = 0, min(10 * numPartitions, 1000) for (k, v) in iterator: buckets[partitionFunc(k) % numPartitions].append((k, v)) + c += 1 + + # check used memory and avg size of chunk of objects + if (c % 1000 == 0 and get_used_memory() > limit + or c > batch): + n, size = len(buckets), 0 + for split in buckets.keys(): + yield pack_long(split) + d = outputSerializer.dumps(buckets[split]) + del buckets[split] + yield d + size += len(d) + + avg = (size / n) >> 20 + # let 1M < avg < 10M + if avg < 1: + batch *= 1.5 + elif avg > 10: + batch = max(batch / 1.5, 1) + c = 0 + for (split, items) in buckets.iteritems(): yield pack_long(split) yield outputSerializer.dumps(items) + keyed = PipelinedRDD(self, add_shuffle_key) keyed._bypass_serializer = True with _JavaStackTrace(self.context) as st: @@ -1230,8 +1277,8 @@ def add_shuffle_key(split, iterator): id(partitionFunc)) jrdd = pairRDD.partitionBy(partitioner).values() rdd = RDD(jrdd, self.ctx, BatchedSerializer(outputSerializer)) - # This is required so that id(partitionFunc) remains unique, even if - # partitionFunc is a lambda: + # This is required so that id(partitionFunc) remains unique, + # even if partitionFunc is a lambda: rdd._partitionFunc = partitionFunc return rdd @@ -1265,26 +1312,28 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, if numPartitions is None: numPartitions = self._defaultReducePartitions() + serializer = self.ctx.serializer + spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() + == 'true') + memory = _parse_memory(self.ctx._conf.get( + "spark.python.worker.memory", "512m")) + agg = Aggregator(createCombiner, mergeValue, mergeCombiners) + def combineLocally(iterator): - combiners = {} - for x in iterator: - (k, v) = x - if k not in combiners: - combiners[k] = createCombiner(v) - else: - combiners[k] = mergeValue(combiners[k], v) - return combiners.iteritems() + merger = ExternalMerger(agg, memory * 0.9, serializer) \ + if spill else InMemoryMerger(agg) + merger.mergeValues(iterator) + return merger.iteritems() + locally_combined = self.mapPartitions(combineLocally) shuffled = locally_combined.partitionBy(numPartitions) def _mergeCombiners(iterator): - combiners = {} - for (k, v) in iterator: - if k not in combiners: - combiners[k] = v - else: - combiners[k] = mergeCombiners(combiners[k], v) - return combiners.iteritems() + merger = ExternalMerger(agg, memory, serializer) \ + if spill else InMemoryMerger(agg) + merger.mergeCombiners(iterator) + return merger.iteritems() + return shuffled.mapPartitions(_mergeCombiners) def aggregateByKey(self, zeroValue, seqFunc, combFunc, numPartitions=None): @@ -1343,7 +1392,8 @@ def mergeValue(xs, x): return xs def mergeCombiners(a, b): - return a + b + a.extend(b) + return a return self.combineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions).mapValues(lambda x: ResultIterable(x)) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 9be78b39fbc21..03b31ae9624c2 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -193,7 +193,7 @@ def load_stream(self, stream): return chain.from_iterable(self._load_stream_without_unbatching(stream)) def _load_stream_without_unbatching(self, stream): - return self.serializer.load_stream(stream) + return self.serializer.load_stream(stream) def __eq__(self, other): return (isinstance(other, BatchedSerializer) and @@ -302,6 +302,33 @@ class MarshalSerializer(FramedSerializer): loads = marshal.loads +class AutoSerializer(FramedSerializer): + """ + Choose marshal or cPickle as serialization protocol autumatically + """ + def __init__(self): + FramedSerializer.__init__(self) + self._type = None + + def dumps(self, obj): + if self._type is not None: + return 'P' + cPickle.dumps(obj, -1) + try: + return 'M' + marshal.dumps(obj) + except Exception: + self._type = 'P' + return 'P' + cPickle.dumps(obj, -1) + + def loads(self, obj): + _type = obj[0] + if _type == 'M': + return marshal.loads(obj[1:]) + elif _type == 'P': + return cPickle.loads(obj[1:]) + else: + raise ValueError("invalid sevialization type: %s" % _type) + + class UTF8Deserializer(Serializer): """ Deserializes streams written by String.getBytes. diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py new file mode 100644 index 0000000000000..e3923d1c36c57 --- /dev/null +++ b/python/pyspark/shuffle.py @@ -0,0 +1,439 @@ +# +# 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. +# + +import os +import sys +import platform +import shutil +import warnings +import gc + +from pyspark.serializers import BatchedSerializer, PickleSerializer + +try: + import psutil + + def get_used_memory(): + """ Return the used memory in MB """ + process = psutil.Process(os.getpid()) + if hasattr(process, "memory_info"): + info = process.memory_info() + else: + info = process.get_memory_info() + return info.rss >> 20 +except ImportError: + + def get_used_memory(): + """ Return the used memory in MB """ + if platform.system() == 'Linux': + for line in open('/proc/self/status'): + if line.startswith('VmRSS:'): + return int(line.split()[1]) >> 10 + else: + warnings.warn("Please install psutil to have better " + "support with spilling") + if platform.system() == "Darwin": + import resource + rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss + return rss >> 20 + # TODO: support windows + return 0 + + +class Aggregator(object): + + """ + Aggregator has tree functions to merge values into combiner. + + createCombiner: (value) -> combiner + mergeValue: (combine, value) -> combiner + mergeCombiners: (combiner, combiner) -> combiner + """ + + def __init__(self, createCombiner, mergeValue, mergeCombiners): + self.createCombiner = createCombiner + self.mergeValue = mergeValue + self.mergeCombiners = mergeCombiners + + +class SimpleAggregator(Aggregator): + + """ + SimpleAggregator is useful for the cases that combiners have + same type with values + """ + + def __init__(self, combiner): + Aggregator.__init__(self, lambda x: x, combiner, combiner) + + +class Merger(object): + + """ + Merge shuffled data together by aggregator + """ + + def __init__(self, aggregator): + self.agg = aggregator + + def mergeValues(self, iterator): + """ Combine the items by creator and combiner """ + raise NotImplementedError + + def mergeCombiners(self, iterator): + """ Merge the combined items by mergeCombiner """ + raise NotImplementedError + + def iteritems(self): + """ Return the merged items ad iterator """ + raise NotImplementedError + + +class InMemoryMerger(Merger): + + """ + In memory merger based on in-memory dict. + """ + + def __init__(self, aggregator): + Merger.__init__(self, aggregator) + self.data = {} + + def mergeValues(self, iterator): + """ Combine the items by creator and combiner """ + # speed up attributes lookup + d, creator = self.data, self.agg.createCombiner + comb = self.agg.mergeValue + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else creator(v) + + def mergeCombiners(self, iterator): + """ Merge the combined items by mergeCombiner """ + # speed up attributes lookup + d, comb = self.data, self.agg.mergeCombiners + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else v + + def iteritems(self): + """ Return the merged items ad iterator """ + return self.data.iteritems() + + +class ExternalMerger(Merger): + + """ + External merger will dump the aggregated data into disks when + memory usage goes above the limit, then merge them together. + + This class works as follows: + + - It repeatedly combine the items and save them in one dict in + memory. + + - When the used memory goes above memory limit, it will split + the combined data into partitions by hash code, dump them + into disk, one file per partition. + + - Then it goes through the rest of the iterator, combine items + into different dict by hash. Until the used memory goes over + memory limit, it dump all the dicts into disks, one file per + dict. Repeat this again until combine all the items. + + - Before return any items, it will load each partition and + combine them seperately. Yield them before loading next + partition. + + - During loading a partition, if the memory goes over limit, + it will partition the loaded data and dump them into disks + and load them partition by partition again. + + `data` and `pdata` are used to hold the merged items in memory. + At first, all the data are merged into `data`. Once the used + memory goes over limit, the items in `data` are dumped indo + disks, `data` will be cleared, all rest of items will be merged + into `pdata` and then dumped into disks. Before returning, all + the items in `pdata` will be dumped into disks. + + Finally, if any items were spilled into disks, each partition + will be merged into `data` and be yielded, then cleared. + + >>> agg = SimpleAggregator(lambda x, y: x + y) + >>> merger = ExternalMerger(agg, 10) + >>> N = 10000 + >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10) + >>> assert merger.spills > 0 + >>> sum(v for k,v in merger.iteritems()) + 499950000 + + >>> merger = ExternalMerger(agg, 10) + >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10) + >>> assert merger.spills > 0 + >>> sum(v for k,v in merger.iteritems()) + 499950000 + """ + + # the max total partitions created recursively + MAX_TOTAL_PARTITIONS = 4096 + + def __init__(self, aggregator, memory_limit=512, serializer=None, + localdirs=None, scale=1, partitions=59, batch=1000): + Merger.__init__(self, aggregator) + self.memory_limit = memory_limit + # default serializer is only used for tests + self.serializer = serializer or \ + BatchedSerializer(PickleSerializer(), 1024) + self.localdirs = localdirs or self._get_dirs() + # number of partitions when spill data into disks + self.partitions = partitions + # check the memory after # of items merged + self.batch = batch + # scale is used to scale down the hash of key for recursive hash map + self.scale = scale + # unpartitioned merged data + self.data = {} + # partitioned merged data, list of dicts + self.pdata = [] + # number of chunks dumped into disks + self.spills = 0 + # randomize the hash of key, id(o) is the address of o (aligned by 8) + self._seed = id(self) + 7 + + def _get_dirs(self): + """ Get all the directories """ + path = os.environ.get("SPARK_LOCAL_DIR", "/tmp") + dirs = path.split(",") + return [os.path.join(d, "python", str(os.getpid()), str(id(self))) + for d in dirs] + + def _get_spill_dir(self, n): + """ Choose one directory for spill by number n """ + return os.path.join(self.localdirs[n % len(self.localdirs)], str(n)) + + def _next_limit(self): + """ + Return the next memory limit. If the memory is not released + after spilling, it will dump the data only when the used memory + starts to increase. + """ + return max(self.memory_limit, get_used_memory() * 1.05) + + def mergeValues(self, iterator): + """ Combine the items by creator and combiner """ + iterator = iter(iterator) + # speedup attribute lookup + creator, comb = self.agg.createCombiner, self.agg.mergeValue + d, c, batch = self.data, 0, self.batch + + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else creator(v) + + c += 1 + if c % batch == 0 and get_used_memory() > self.memory_limit: + self._spill() + self._partitioned_mergeValues(iterator, self._next_limit()) + break + + def _partition(self, key): + """ Return the partition for key """ + return hash((key, self._seed)) % self.partitions + + def _partitioned_mergeValues(self, iterator, limit=0): + """ Partition the items by key, then combine them """ + # speedup attribute lookup + creator, comb = self.agg.createCombiner, self.agg.mergeValue + c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch + + for k, v in iterator: + d = pdata[hfun(k)] + d[k] = comb(d[k], v) if k in d else creator(v) + if not limit: + continue + + c += 1 + if c % batch == 0 and get_used_memory() > limit: + self._spill() + limit = self._next_limit() + + def mergeCombiners(self, iterator, check=True): + """ Merge (K,V) pair by mergeCombiner """ + iterator = iter(iterator) + # speedup attribute lookup + d, comb, batch = self.data, self.agg.mergeCombiners, self.batch + c = 0 + for k, v in iterator: + d[k] = comb(d[k], v) if k in d else v + if not check: + continue + + c += 1 + if c % batch == 0 and get_used_memory() > self.memory_limit: + self._spill() + self._partitioned_mergeCombiners(iterator, self._next_limit()) + break + + def _partitioned_mergeCombiners(self, iterator, limit=0): + """ Partition the items by key, then merge them """ + comb, pdata = self.agg.mergeCombiners, self.pdata + c, hfun = 0, self._partition + for k, v in iterator: + d = pdata[hfun(k)] + d[k] = comb(d[k], v) if k in d else v + if not limit: + continue + + c += 1 + if c % self.batch == 0 and get_used_memory() > limit: + self._spill() + limit = self._next_limit() + + def _spill(self): + """ + dump already partitioned data into disks. + + It will dump the data in batch for better performance. + """ + path = self._get_spill_dir(self.spills) + if not os.path.exists(path): + os.makedirs(path) + + if not self.pdata: + # The data has not been partitioned, it will iterator the + # dataset once, write them into different files, has no + # additional memory. It only called when the memory goes + # above limit at the first time. + + # open all the files for writing + streams = [open(os.path.join(path, str(i)), 'w') + for i in range(self.partitions)] + + for k, v in self.data.iteritems(): + h = self._partition(k) + # put one item in batch, make it compatitable with load_stream + # it will increase the memory if dump them in batch + self.serializer.dump_stream([(k, v)], streams[h]) + + for s in streams: + s.close() + + self.data.clear() + self.pdata = [{} for i in range(self.partitions)] + + else: + for i in range(self.partitions): + p = os.path.join(path, str(i)) + with open(p, "w") as f: + # dump items in batch + self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.pdata[i].clear() + + self.spills += 1 + gc.collect() # release the memory as much as possible + + def iteritems(self): + """ Return all merged items as iterator """ + if not self.pdata and not self.spills: + return self.data.iteritems() + return self._external_items() + + def _external_items(self): + """ Return all partitioned items as iterator """ + assert not self.data + if any(self.pdata): + self._spill() + hard_limit = self._next_limit() + + try: + for i in range(self.partitions): + self.data = {} + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(i)) + # do not check memory during merging + self.mergeCombiners(self.serializer.load_stream(open(p)), + False) + + # limit the total partitions + if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS + and j < self.spills - 1 + and get_used_memory() > hard_limit): + self.data.clear() # will read from disk again + gc.collect() # release the memory as much as possible + for v in self._recursive_merged_items(i): + yield v + return + + for v in self.data.iteritems(): + yield v + self.data.clear() + gc.collect() + + # remove the merged partition + for j in range(self.spills): + path = self._get_spill_dir(j) + os.remove(os.path.join(path, str(i))) + + finally: + self._cleanup() + + def _cleanup(self): + """ Clean up all the files in disks """ + for d in self.localdirs: + shutil.rmtree(d, True) + + def _recursive_merged_items(self, start): + """ + merge the partitioned items and return the as iterator + + If one partition can not be fit in memory, then them will be + partitioned and merged recursively. + """ + # make sure all the data are dumps into disks. + assert not self.data + if any(self.pdata): + self._spill() + assert self.spills > 0 + + for i in range(start, self.partitions): + subdirs = [os.path.join(d, "parts", str(i)) + for d in self.localdirs] + m = ExternalMerger(self.agg, self.memory_limit, self.serializer, + subdirs, self.scale * self.partitions) + m.pdata = [{} for _ in range(self.partitions)] + limit = self._next_limit() + + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(i)) + m._partitioned_mergeCombiners( + self.serializer.load_stream(open(p))) + + if get_used_memory() > limit: + m._spill() + limit = self._next_limit() + + for v in m._external_items(): + yield v + + # remove the merged partition + for j in range(self.spills): + path = self._get_spill_dir(j) + os.remove(os.path.join(path, str(i))) + + +if __name__ == "__main__": + import doctest + doctest.testmod() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 9c5ecd0bb02ab..a92abbf371f18 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -34,6 +34,7 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int +from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger _have_scipy = False try: @@ -47,6 +48,62 @@ SPARK_HOME = os.environ["SPARK_HOME"] +class TestMerger(unittest.TestCase): + + def setUp(self): + self.N = 1 << 16 + self.l = [i for i in xrange(self.N)] + self.data = zip(self.l, self.l) + self.agg = Aggregator(lambda x: [x], + lambda x, y: x.append(y) or x, + lambda x, y: x.extend(y) or x) + + def test_in_memory(self): + m = InMemoryMerger(self.agg) + m.mergeValues(self.data) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + m = InMemoryMerger(self.agg) + m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + def test_small_dataset(self): + m = ExternalMerger(self.agg, 1000) + m.mergeValues(self.data) + self.assertEqual(m.spills, 0) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + m = ExternalMerger(self.agg, 1000) + m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + self.assertEqual(m.spills, 0) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + def test_medium_dataset(self): + m = ExternalMerger(self.agg, 10) + m.mergeValues(self.data) + self.assertTrue(m.spills >= 1) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N))) + + m = ExternalMerger(self.agg, 10) + m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3)) + self.assertTrue(m.spills >= 1) + self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + sum(xrange(self.N)) * 3) + + def test_huge_dataset(self): + m = ExternalMerger(self.agg, 10) + m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) + self.assertTrue(m.spills >= 1) + self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), + self.N * 10) + m._cleanup() + + class PySparkTestCase(unittest.TestCase): def setUp(self): diff --git a/python/run-tests b/python/run-tests index 9282aa47e8375..29f755fc0dcd3 100755 --- a/python/run-tests +++ b/python/run-tests @@ -61,6 +61,7 @@ run_test "pyspark/broadcast.py" run_test "pyspark/accumulators.py" run_test "pyspark/serializers.py" unset PYSPARK_DOC_TEST +run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" run_test "pyspark/mllib/_common.py" run_test "pyspark/mllib/classification.py" From 2f75a4a30e1a3fdf384475b9660c6c43f093f68c Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 24 Jul 2014 23:42:08 -0700 Subject: [PATCH 0068/1492] [SPARK-2656] Python version of stratified sampling exact sample size not supported for now. Author: Doris Xin Closes #1554 from dorx/pystratified and squashes the following commits: 4ba927a [Doris Xin] use rel diff (+- 50%) instead of abs diff (+- 50) bdc3f8b [Doris Xin] updated unit to check sample holistically 7713c7b [Doris Xin] Python version of stratified sampling --- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- python/pyspark/rdd.py | 25 ++++++++++++++-- python/pyspark/rddsampler.py | 30 +++++++++++++++++-- 3 files changed, 51 insertions(+), 6 deletions(-) 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 c1bafab3e7491..edbf7eace9437 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -354,7 +354,7 @@ abstract class RDD[T: ClassTag]( def sample(withReplacement: Boolean, fraction: Double, seed: Long = Utils.random.nextLong): RDD[T] = { - require(fraction >= 0.0, "Invalid fraction value: " + fraction) + require(fraction >= 0.0, "Negative fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), true, seed) } else { diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 7ad6108261444..113a082e16721 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -39,7 +39,7 @@ from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter -from pyspark.rddsampler import RDDSampler +from pyspark.rddsampler import RDDSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ @@ -411,7 +411,7 @@ def sample(self, withReplacement, fraction, seed=None): >>> sc.parallelize(range(0, 100)).sample(False, 0.1, 2).collect() #doctest: +SKIP [2, 3, 20, 21, 24, 41, 42, 66, 67, 89, 90, 98] """ - assert fraction >= 0.0, "Invalid fraction value: %s" % fraction + assert fraction >= 0.0, "Negative fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) # this is ported from scala/spark/RDD.scala @@ -1456,6 +1456,27 @@ def cogroup(self, other, numPartitions=None): """ return python_cogroup((self, other), numPartitions) + def sampleByKey(self, withReplacement, fractions, seed=None): + """ + Return a subset of this RDD sampled by key (via stratified sampling). + Create a sample of this RDD using variable sampling rates for + different keys as specified by fractions, a key to sampling rate map. + + >>> fractions = {"a": 0.2, "b": 0.1} + >>> rdd = sc.parallelize(fractions.keys()).cartesian(sc.parallelize(range(0, 1000))) + >>> sample = dict(rdd.sampleByKey(False, fractions, 2).groupByKey().collect()) + >>> 100 < len(sample["a"]) < 300 and 50 < len(sample["b"]) < 150 + True + >>> max(sample["a"]) <= 999 and min(sample["a"]) >= 0 + True + >>> max(sample["b"]) <= 999 and min(sample["b"]) >= 0 + True + """ + for fraction in fractions.values(): + assert fraction >= 0.0, "Negative fraction value: %s" % fraction + return self.mapPartitionsWithIndex( \ + RDDStratifiedSampler(withReplacement, fractions, seed).func, True) + def subtractByKey(self, other, numPartitions=None): """ Return each (key, value) pair in C{self} that has no pair with matching diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 7ff1c316c7623..2df000fdb08ca 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -19,8 +19,8 @@ import random -class RDDSampler(object): - def __init__(self, withReplacement, fraction, seed=None): +class RDDSamplerBase(object): + def __init__(self, withReplacement, seed=None): try: import numpy self._use_numpy = True @@ -32,7 +32,6 @@ def __init__(self, withReplacement, fraction, seed=None): self._seed = seed if seed is not None else random.randint(0, sys.maxint) self._withReplacement = withReplacement - self._fraction = fraction self._random = None self._split = None self._rand_initialized = False @@ -94,6 +93,12 @@ def shuffle(self, vals): else: self._random.shuffle(vals, self._random.random) + +class RDDSampler(RDDSamplerBase): + def __init__(self, withReplacement, fraction, seed=None): + RDDSamplerBase.__init__(self, withReplacement, seed) + self._fraction = fraction + def func(self, split, iterator): if self._withReplacement: for obj in iterator: @@ -107,3 +112,22 @@ def func(self, split, iterator): for obj in iterator: if self.getUniformSample(split) <= self._fraction: yield obj + +class RDDStratifiedSampler(RDDSamplerBase): + def __init__(self, withReplacement, fractions, seed=None): + RDDSamplerBase.__init__(self, withReplacement, seed) + self._fractions = fractions + + def func(self, split, iterator): + if self._withReplacement: + for key, val in iterator: + # For large datasets, the expected number of occurrences of each element in + # a sample with replacement is Poisson(frac). We use that to get a count for + # each element. + count = self.getPoissonSample(split, mean=self._fractions[key]) + for _ in range(0, count): + yield key, val + else: + for key, val in iterator: + if self.getUniformSample(split) <= self._fractions[key]: + yield key, val From 8529ced35c6b77a384d10a26b654a8073d57e03d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 25 Jul 2014 00:32:32 -0700 Subject: [PATCH 0069/1492] SPARK-2657 Use more compact data structures than ArrayBuffer in groupBy & cogroup JIRA: https://issues.apache.org/jira/browse/SPARK-2657 Our current code uses ArrayBuffers for each group of values in groupBy, as well as for the key's elements in CoGroupedRDD. ArrayBuffers have a lot of overhead if there are few values in them, which is likely to happen in cases such as join. In particular, they have a pointer to an Object[] of size 16 by default, which is 24 bytes for the array header + 128 for the pointers in there, plus at least 32 for the ArrayBuffer data structure. This patch replaces the per-group buffers with a CompactBuffer class that can store up to 2 elements more efficiently (in fields of itself) and acts like an ArrayBuffer beyond that. For a key's elements in CoGroupedRDD, we use an Array of CompactBuffers instead of an ArrayBuffer of ArrayBuffers. There are some changes throughout the code to deal with CoGroupedRDD returning Array instead. We can also decide not to do that but CoGroupedRDD is a `DeveloperAPI` so I think it's okay to change it here. Author: Matei Zaharia Closes #1555 from mateiz/compact-groupby and squashes the following commits: 845a356 [Matei Zaharia] Lower initial size of CompactBuffer's vector to 8 07621a7 [Matei Zaharia] Review comments 0c1cd12 [Matei Zaharia] Don't use varargs in CompactBuffer.apply bdc8a39 [Matei Zaharia] Small tweak to +=, and typos f61f040 [Matei Zaharia] Fix line lengths 59da88b0 [Matei Zaharia] Fix line lengths 197cde8 [Matei Zaharia] Make CompactBuffer extend Seq to make its toSeq more efficient 775110f [Matei Zaharia] Change CoGroupedRDD to give (K, Array[Iterable[_]]) to avoid wrappers 9b4c6e8 [Matei Zaharia] Use CompactBuffer in CoGroupedRDD ed577ab [Matei Zaharia] Use CompactBuffer in groupByKey 10f0de1 [Matei Zaharia] A CompactBuffer that's more memory-efficient than ArrayBuffer for small buffers --- .../org/apache/spark/rdd/CoGroupedRDD.scala | 16 +- .../apache/spark/rdd/PairRDDFunctions.scala | 33 ++-- .../spark/serializer/KryoSerializer.scala | 2 + .../spark/util/collection/CompactBuffer.scala | 159 ++++++++++++++++++ .../org/apache/spark/CheckpointSuite.scala | 37 ++-- .../scala/org/apache/spark/ShuffleSuite.scala | 4 +- .../util/collection/CompactBufferSuite.scala | 105 ++++++++++++ .../correlation/SpearmanCorrelation.scala | 6 +- .../dstream/ReducedWindowedDStream.scala | 15 +- 9 files changed, 334 insertions(+), 43 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/CompactBufferSuite.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index aca235a62a6a8..7d96089e52ab9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -25,7 +25,7 @@ import scala.language.existentials import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} +import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap, CompactBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleHandle @@ -66,14 +66,14 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] */ @DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) - extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { + extends RDD[(K, Array[Iterable[_]])](rdds.head.context, Nil) { // For example, `(k, a) cogroup (k, b)` produces k -> Seq(ArrayBuffer as, ArrayBuffer bs). // Each ArrayBuffer is represented as a CoGroup, and the resulting Seq as a CoGroupCombiner. // CoGroupValue is the intermediate state of each value before being merged in compute. - private type CoGroup = ArrayBuffer[Any] + private type CoGroup = CompactBuffer[Any] private type CoGroupValue = (Any, Int) // Int is dependency number - private type CoGroupCombiner = Seq[CoGroup] + private type CoGroupCombiner = Array[CoGroup] private var serializer: Option[Serializer] = None @@ -114,7 +114,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: override val partitioner: Some[Partitioner] = Some(part) - override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { + override def compute(s: Partition, context: TaskContext): Iterator[(K, Array[Iterable[_]])] = { val sparkConf = SparkEnv.get.conf val externalSorting = sparkConf.getBoolean("spark.shuffle.spill", true) val split = s.asInstanceOf[CoGroupPartition] @@ -150,7 +150,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: getCombiner(kv._1)(depNum) += kv._2 } } - new InterruptibleIterator(context, map.iterator) + new InterruptibleIterator(context, + map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } else { val map = createExternalMap(numRdds) rddIterators.foreach { case (it, depNum) => @@ -161,7 +162,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: } context.taskMetrics.memoryBytesSpilled = map.memoryBytesSpilled context.taskMetrics.diskBytesSpilled = map.diskBytesSpilled - new InterruptibleIterator(context, map.iterator) + new InterruptibleIterator(context, + map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } } 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 a6b920467283e..c04d162a39616 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -46,6 +46,7 @@ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer +import org.apache.spark.util.collection.CompactBuffer /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -361,12 +362,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. - val createCombiner = (v: V) => ArrayBuffer(v) - val mergeValue = (buf: ArrayBuffer[V], v: V) => buf += v - val mergeCombiners = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => c1 ++ c2 - val bufs = combineByKey[ArrayBuffer[V]]( + val createCombiner = (v: V) => CompactBuffer(v) + val mergeValue = (buf: CompactBuffer[V], v: V) => buf += v + val mergeCombiners = (c1: CompactBuffer[V], c2: CompactBuffer[V]) => c1 ++= c2 + val bufs = combineByKey[CompactBuffer[V]]( createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine=false) - bufs.mapValues(_.toIterable) + bufs.asInstanceOf[RDD[(K, Iterable[V])]] } /** @@ -571,11 +572,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2, other3), partitioner) - cg.mapValues { case Seq(vs, w1s, w2s, w3s) => - (vs.asInstanceOf[Seq[V]], - w1s.asInstanceOf[Seq[W1]], - w2s.asInstanceOf[Seq[W2]], - w3s.asInstanceOf[Seq[W3]]) + cg.mapValues { case Array(vs, w1s, w2s, w3s) => + (vs.asInstanceOf[Iterable[V]], + w1s.asInstanceOf[Iterable[W1]], + w2s.asInstanceOf[Iterable[W2]], + w3s.asInstanceOf[Iterable[W3]]) } } @@ -589,8 +590,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) - cg.mapValues { case Seq(vs, w1s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W]]) + cg.mapValues { case Array(vs, w1s) => + (vs.asInstanceOf[Iterable[V]], w1s.asInstanceOf[Iterable[W]]) } } @@ -604,10 +605,10 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) - cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], - w1s.asInstanceOf[Seq[W1]], - w2s.asInstanceOf[Seq[W2]]) + cg.mapValues { case Array(vs, w1s, w2s) => + (vs.asInstanceOf[Iterable[V]], + w1s.asInstanceOf[Iterable[W1]], + w2s.asInstanceOf[Iterable[W2]]) } } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index c3a3e90a34901..fa79b25759153 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -31,6 +31,7 @@ import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} import org.apache.spark.util.BoundedPriorityQueue +import org.apache.spark.util.collection.CompactBuffer import scala.reflect.ClassTag @@ -185,6 +186,7 @@ private[serializer] object KryoSerializer { classOf[GotBlock], classOf[GetBlock], classOf[MapStatus], + classOf[CompactBuffer[_]], classOf[BlockManagerId], classOf[Array[Byte]], classOf[BoundedPriorityQueue[_]], diff --git a/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala new file mode 100644 index 0000000000000..d44e15e3c97ea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/CompactBuffer.scala @@ -0,0 +1,159 @@ +/* + * 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.util.collection + +/** + * An append-only buffer similar to ArrayBuffer, but more memory-efficient for small buffers. + * ArrayBuffer always allocates an Object array to store the data, with 16 entries by default, + * so it has about 80-100 bytes of overhead. In contrast, CompactBuffer can keep up to two + * elements in fields of the main object, and only allocates an Array[AnyRef] if there are more + * entries than that. This makes it more efficient for operations like groupBy where we expect + * some keys to have very few elements. + */ +private[spark] class CompactBuffer[T] extends Seq[T] with Serializable { + // First two elements + private var element0: T = _ + private var element1: T = _ + + // Number of elements, including our two in the main object + private var curSize = 0 + + // Array for extra elements + private var otherElements: Array[AnyRef] = null + + def apply(position: Int): T = { + if (position < 0 || position >= curSize) { + throw new IndexOutOfBoundsException + } + if (position == 0) { + element0 + } else if (position == 1) { + element1 + } else { + otherElements(position - 2).asInstanceOf[T] + } + } + + private def update(position: Int, value: T): Unit = { + if (position < 0 || position >= curSize) { + throw new IndexOutOfBoundsException + } + if (position == 0) { + element0 = value + } else if (position == 1) { + element1 = value + } else { + otherElements(position - 2) = value.asInstanceOf[AnyRef] + } + } + + def += (value: T): CompactBuffer[T] = { + val newIndex = curSize + if (newIndex == 0) { + element0 = value + curSize = 1 + } else if (newIndex == 1) { + element1 = value + curSize = 2 + } else { + growToSize(curSize + 1) + otherElements(newIndex - 2) = value.asInstanceOf[AnyRef] + } + this + } + + def ++= (values: TraversableOnce[T]): CompactBuffer[T] = { + values match { + // Optimize merging of CompactBuffers, used in cogroup and groupByKey + case compactBuf: CompactBuffer[T] => + val oldSize = curSize + // Copy the other buffer's size and elements to local variables in case it is equal to us + val itsSize = compactBuf.curSize + val itsElements = compactBuf.otherElements + growToSize(curSize + itsSize) + if (itsSize == 1) { + this(oldSize) = compactBuf.element0 + } else if (itsSize == 2) { + this(oldSize) = compactBuf.element0 + this(oldSize + 1) = compactBuf.element1 + } else if (itsSize > 2) { + this(oldSize) = compactBuf.element0 + this(oldSize + 1) = compactBuf.element1 + // At this point our size is also above 2, so just copy its array directly into ours. + // Note that since we added two elements above, the index in this.otherElements that we + // should copy to is oldSize. + System.arraycopy(itsElements, 0, otherElements, oldSize, itsSize - 2) + } + + case _ => + values.foreach(e => this += e) + } + this + } + + override def length: Int = curSize + + override def size: Int = curSize + + override def iterator: Iterator[T] = new Iterator[T] { + private var pos = 0 + override def hasNext: Boolean = pos < curSize + override def next(): T = { + if (!hasNext) { + throw new NoSuchElementException + } + pos += 1 + apply(pos - 1) + } + } + + /** Increase our size to newSize and grow the backing array if needed. */ + private def growToSize(newSize: Int): Unit = { + if (newSize < 0) { + throw new UnsupportedOperationException("Can't grow buffer past Int.MaxValue elements") + } + val capacity = if (otherElements != null) otherElements.length + 2 else 2 + if (newSize > capacity) { + var newArrayLen = 8 + while (newSize - 2 > newArrayLen) { + newArrayLen *= 2 + if (newArrayLen == Int.MinValue) { + // Prevent overflow if we double from 2^30 to 2^31, which will become Int.MinValue. + // Note that we set the new array length to Int.MaxValue - 2 so that our capacity + // calculation above still gives a positive integer. + newArrayLen = Int.MaxValue - 2 + } + } + val newArray = new Array[AnyRef](newArrayLen) + if (otherElements != null) { + System.arraycopy(otherElements, 0, newArray, 0, otherElements.length) + } + otherElements = newArray + } + curSize = newSize + } +} + +private[spark] object CompactBuffer { + def apply[T](): CompactBuffer[T] = new CompactBuffer[T] + + def apply[T](value: T): CompactBuffer[T] = { + val buf = new CompactBuffer[T] + buf += value + } +} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index fc00458083a33..d1cb2d9d3a53b 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -156,15 +156,20 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("CoGroupedRDD") { val longLineageRDD1 = generateFatPairRDD() + + // Collect the RDD as sequences instead of arrays to enable equality tests in testRDD + val seqCollectFunc = (rdd: RDD[(Int, Array[Iterable[Int]])]) => + rdd.map{case (p, a) => (p, a.toSeq)}.collect(): Any + testRDD(rdd => { CheckpointSuite.cogroup(longLineageRDD1, rdd.map(x => (x % 2, 1)), partitioner) - }) + }, seqCollectFunc) val longLineageRDD2 = generateFatPairRDD() testRDDPartitions(rdd => { CheckpointSuite.cogroup( longLineageRDD2, sc.makeRDD(1 to 2, 2).map(x => (x % 2, 1)), partitioner) - }) + }, seqCollectFunc) } test("ZippedPartitionsRDD") { @@ -235,12 +240,19 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { assert(rdd.partitions.size === 0) } + def defaultCollectFunc[T](rdd: RDD[T]): Any = rdd.collect() + /** * Test checkpointing of the RDD generated by the given operation. It tests whether the * serialized size of the RDD is reduce after checkpointing or not. This function should be called * on all RDDs that have a parent RDD (i.e., do not call on ParallelCollection, BlockRDD, etc.). + * + * @param op an operation to run on the RDD + * @param collectFunc a function for collecting the values in the RDD, in case there are + * non-comparable types like arrays that we want to convert to something that supports == */ - def testRDD[U: ClassTag](op: (RDD[Int]) => RDD[U]) { + def testRDD[U: ClassTag](op: (RDD[Int]) => RDD[U], + collectFunc: RDD[U] => Any = defaultCollectFunc[U] _) { // Generate the final RDD using given RDD operation val baseRDD = generateFatRDD() val operatedRDD = op(baseRDD) @@ -258,13 +270,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) operatedRDD.checkpoint() - val result = operatedRDD.collect() + val result = collectFunc(operatedRDD) operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) // Test whether the checkpoint file has been created - assert(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get).collect() === result) + assert(collectFunc(sc.checkpointFile[U](operatedRDD.getCheckpointFile.get)) === result) // Test whether dependencies have been changed from its earlier parent RDD assert(operatedRDD.dependencies.head.rdd != parentRDD) @@ -279,7 +291,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { assert(operatedRDD.partitions.length === numPartitions) // Test whether the data in the checkpointed RDD is same as original - assert(operatedRDD.collect() === result) + assert(collectFunc(operatedRDD) === result) // Test whether serialized size of the RDD has reduced. logInfo("Size of " + rddType + @@ -289,7 +301,6 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { "Size of " + rddType + " did not reduce after checkpointing " + " [" + rddSizeBeforeCheckpoint + " --> " + rddSizeAfterCheckpoint + "]" ) - } /** @@ -300,8 +311,12 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * This function should be called only those RDD whose partitions refer to parent RDD's * partitions (i.e., do not call it on simple RDD like MappedRDD). * + * @param op an operation to run on the RDD + * @param collectFunc a function for collecting the values in the RDD, in case there are + * non-comparable types like arrays that we want to convert to something that supports == */ - def testRDDPartitions[U: ClassTag](op: (RDD[Int]) => RDD[U]) { + def testRDDPartitions[U: ClassTag](op: (RDD[Int]) => RDD[U], + collectFunc: RDD[U] => Any = defaultCollectFunc[U] _) { // Generate the final RDD using given RDD operation val baseRDD = generateFatRDD() val operatedRDD = op(baseRDD) @@ -316,13 +331,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) val (rddSizeBeforeCheckpoint, partitionSizeBeforeCheckpoint) = getSerializedSizes(operatedRDD) parentRDDs.foreach(_.checkpoint()) // checkpoint the parent RDD, not the generated one - val result = operatedRDD.collect() // force checkpointing + val result = collectFunc(operatedRDD) // force checkpointing operatedRDD.collect() // force re-initialization of post-checkpoint lazy variables val (rddSizeAfterCheckpoint, partitionSizeAfterCheckpoint) = getSerializedSizes(operatedRDD) logInfo("RDD after checkpoint: " + operatedRDD + "\n" + operatedRDD.toDebugString) // Test whether the data in the checkpointed RDD is same as original - assert(operatedRDD.collect() === result) + assert(collectFunc(operatedRDD) === result) // Test whether serialized size of the partitions has reduced logInfo("Size of partitions of " + rddType + @@ -436,7 +451,7 @@ object CheckpointSuite { new CoGroupedRDD[K]( Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part - ).asInstanceOf[RDD[(K, Seq[Seq[V]])]] + ).asInstanceOf[RDD[(K, Array[Iterable[V]])]] } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 237e644b48e49..eae67c7747e82 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -176,7 +176,9 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) val pairs2: RDD[MutablePair[Int, String]] = sc.parallelize(data2, 2) - val results = new CoGroupedRDD[Int](Seq(pairs1, pairs2), new HashPartitioner(2)).collectAsMap() + val results = new CoGroupedRDD[Int](Seq(pairs1, pairs2), new HashPartitioner(2)) + .map(p => (p._1, p._2.map(_.toArray))) + .collectAsMap() assert(results(1)(0).length === 3) assert(results(1)(0).contains(1)) diff --git a/core/src/test/scala/org/apache/spark/util/collection/CompactBufferSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/CompactBufferSuite.scala new file mode 100644 index 0000000000000..6c956d93dc80d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/CompactBufferSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import org.scalatest.FunSuite + +class CompactBufferSuite extends FunSuite { + test("empty buffer") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + assert(b.size === 0) + assert(b.iterator.toList === Nil) + intercept[IndexOutOfBoundsException] { b(0) } + intercept[IndexOutOfBoundsException] { b(1) } + intercept[IndexOutOfBoundsException] { b(2) } + intercept[IndexOutOfBoundsException] { b(-1) } + } + + test("basic inserts") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + for (i <- 0 until 1000) { + b += i + assert(b.size === i + 1) + assert(b(i) === i) + } + assert(b.iterator.toList === (0 until 1000).toList) + assert(b.iterator.toList === (0 until 1000).toList) + assert(b.size === 1000) + } + + test("adding sequences") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + + // Add some simple lists and iterators + b ++= List(0) + assert(b.size === 1) + assert(b.iterator.toList === List(0)) + b ++= Iterator(1) + assert(b.size === 2) + assert(b.iterator.toList === List(0, 1)) + b ++= List(2) + assert(b.size === 3) + assert(b.iterator.toList === List(0, 1, 2)) + b ++= Iterator(3, 4, 5, 6, 7, 8, 9) + assert(b.size === 10) + assert(b.iterator.toList === (0 until 10).toList) + + // Add CompactBuffers + val b2 = new CompactBuffer[Int] + b2 ++= 0 until 10 + b ++= b2 + assert(b.iterator.toList === (1 to 2).flatMap(i => 0 until 10).toList) + b ++= b2 + assert(b.iterator.toList === (1 to 3).flatMap(i => 0 until 10).toList) + b ++= b2 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList) + + // Add some small CompactBuffers as well + val b3 = new CompactBuffer[Int] + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList) + b3 += 0 + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList ++ List(0)) + b3 += 1 + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList ++ List(0, 0, 1)) + b3 += 2 + b ++= b3 + assert(b.iterator.toList === (1 to 4).flatMap(i => 0 until 10).toList ++ List(0, 0, 1, 0, 1, 2)) + } + + test("adding the same buffer to itself") { + val b = new CompactBuffer[Int] + assert(b.size === 0) + assert(b.iterator.toList === Nil) + b += 1 + assert(b.toList === List(1)) + for (j <- 1 until 8) { + b ++= b + assert(b.size === (1 << j)) + assert(b.iterator.toList === (1 to (1 << j)).map(i => 1).toList) + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala index 88de2c82479b7..1f7de630e778c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -122,6 +122,10 @@ private[stat] object SpearmanCorrelation extends Correlation with Logging { private def makeRankMatrix(ranks: Array[RDD[(Long, Double)]], input: RDD[Vector]): RDD[Vector] = { val partitioner = new HashPartitioner(input.partitions.size) val cogrouped = new CoGroupedRDD[Long](ranks, partitioner) - cogrouped.map { case (_, values: Seq[Seq[Double]]) => new DenseVector(values.flatten.toArray) } + cogrouped.map { + case (_, values: Array[Iterable[_]]) => + val doubles = values.asInstanceOf[Array[Iterable[Double]]] + new DenseVector(doubles.flatten.toArray) + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index 40da31318942e..1a47089e513c4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -133,17 +133,17 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( val numOldValues = oldRDDs.size val numNewValues = newRDDs.size - val mergeValues = (seqOfValues: Seq[Seq[V]]) => { - if (seqOfValues.size != 1 + numOldValues + numNewValues) { + val mergeValues = (arrayOfValues: Array[Iterable[V]]) => { + if (arrayOfValues.size != 1 + numOldValues + numNewValues) { throw new Exception("Unexpected number of sequences of reduced values") } // Getting reduced values "old time steps" that will be removed from current window - val oldValues = (1 to numOldValues).map(i => seqOfValues(i)).filter(!_.isEmpty).map(_.head) + val oldValues = (1 to numOldValues).map(i => arrayOfValues(i)).filter(!_.isEmpty).map(_.head) // Getting reduced values "new time steps" val newValues = - (1 to numNewValues).map(i => seqOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head) + (1 to numNewValues).map(i => arrayOfValues(numOldValues + i)).filter(!_.isEmpty).map(_.head) - if (seqOfValues(0).isEmpty) { + if (arrayOfValues(0).isEmpty) { // If previous window's reduce value does not exist, then at least new values should exist if (newValues.isEmpty) { throw new Exception("Neither previous window has value for key, nor new values found. " + @@ -153,7 +153,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( newValues.reduce(reduceF) // return } else { // Get the previous window's reduced value - var tempValue = seqOfValues(0).head + var tempValue = arrayOfValues(0).head // If old values exists, then inverse reduce then from previous value if (!oldValues.isEmpty) { tempValue = invReduceF(tempValue, oldValues.reduce(reduceF)) @@ -166,7 +166,8 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( } } - val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K,Seq[Seq[V]])]].mapValues(mergeValues) + val mergedValuesRDD = cogroupedRDD.asInstanceOf[RDD[(K, Array[Iterable[V]])]] + .mapValues(mergeValues) if (filterFunc.isDefined) { Some(mergedValuesRDD.filter(filterFunc.get)) From eb82abd8e3d25c912fa75201cf4f429aab8d73c7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 25 Jul 2014 01:10:05 -0700 Subject: [PATCH 0070/1492] [SPARK-2529] Clean closures in foreach and foreachPartition. Author: Reynold Xin Closes #1583 from rxin/closureClean and squashes the following commits: 8982fe6 [Reynold Xin] [SPARK-2529] Clean closures in foreach and foreachPartition. --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 edbf7eace9437..b1c965a790472 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -754,14 +754,16 @@ abstract class RDD[T: ClassTag]( * Applies a function f to all elements of this RDD. */ def foreach(f: T => Unit) { - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: Iterator[T] => Unit) { - sc.runJob(this, (iter: Iterator[T]) => f(iter)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) } /** From 184aa1c6c0ddf26b703bcabf55397ade17497465 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 25 Jul 2014 01:30:22 -0700 Subject: [PATCH 0071/1492] [SPARK-2665] [SQL] Add EqualNS & Unit Tests Hive Supports the operator "<=>", which returns same result with EQUAL(=) operator for non-null operands, but returns TRUE if both are NULL, FALSE if one of the them is NULL. Author: Cheng Hao Closes #1570 from chenghao-intel/equalns and squashes the following commits: 8d6c789 [Cheng Hao] Remove the test case orc_predicate_pushdown 5b2ca88 [Cheng Hao] Add cases into whitelist 8e66cdd [Cheng Hao] Rename the EqualNSTo ==> EqualNullSafe 7af4b0b [Cheng Hao] Add EqualNS & Unit Tests --- .../catalyst/analysis/HiveTypeCoercion.scala | 2 + .../spark/sql/catalyst/dsl/package.scala | 1 + .../sql/catalyst/expressions/predicates.scala | 16 +++++++ .../sql/catalyst/optimizer/Optimizer.scala | 2 + .../ExpressionEvaluationSuite.scala | 10 ++++- .../execution/HiveCompatibilitySuite.scala | 7 +++- .../apache/spark/sql/hive/HiveContext.scala | 3 +- .../org/apache/spark/sql/hive/HiveQl.scala | 1 + ...ullsafe-0-869726b703f160eabdb7763700b53e60 | 1 + ...ullsafe-1-5644ab44e5ba9f2941216b8d5dc33a99 | 0 ...llsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 | 2 + ...llsafe-11-3aa243002a5363b84556736ef71613b1 | 0 ...llsafe-12-3cc55b14e8256d2c51361b61986c291e | 4 ++ ...llsafe-13-69d94e229191e7b9b1a3e7eae46eb993 | 12 ++++++ ...llsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c | 12 ++++++ ...llsafe-15-507d0fa6d7ce39e2d9921555cea6f8da | 13 ++++++ ...llsafe-16-1c714fc339304de4db630530e5d1ce97 | 11 +++++ ...llsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 | 11 +++++ ...llsafe-18-10b2051e65cac50ee1ea1c138ec192c8 | 0 ...llsafe-19-23ab7ac8229a53d391195be7ca092429 | 0 ...nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 | 0 ...llsafe-20-d6fc260320c577eec9a5db0d4135d224 | 0 ...llsafe-21-a60dae725ffc543f805242611d99de4e | 0 ...llsafe-22-24c80d0f9e3d72c48d947770fa184985 | 0 ...llsafe-23-3fe6ae20cab3417759dcc654a3a26746 | 0 ...llsafe-24-2db30531137611e06fdba478ca7a8412 | 1 + ...llsafe-25-e58b2754e8d9c56a473557a549d0d2b9 | 1 + ...llsafe-26-64cabe5164130a94f387288f37b62d71 | 1 + ...llsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 | 42 +++++++++++++++++++ ...llsafe-28-5a0c946cd7033857ca99e5fb800f8525 | 14 +++++++ ...llsafe-29-514043c2ddaf6ea8f16a764adc92d1cf | 42 +++++++++++++++++++ ...ullsafe-3-ae378fc0f875a21884e58fa35a6d52cd | 0 ...llsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 | 42 +++++++++++++++++++ ...llsafe-31-1cb03e1106f79d14f22bc89d386cedcf | 42 +++++++++++++++++++ ...llsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 | 40 ++++++++++++++++++ ...llsafe-33-63157d43422fcedadba408537ccecd5c | 40 ++++++++++++++++++ ...llsafe-34-9265f806b71c03061f93f9fbc88aa223 | 42 +++++++++++++++++++ ...llsafe-35-95815bafb81cccb8129c20d399a446fc | 42 +++++++++++++++++++ ...llsafe-36-c4762c60cc93236b7647ebd32a40ce57 | 42 +++++++++++++++++++ ...llsafe-37-a87893adfc73c9cc63ceab200bb56245 | 42 +++++++++++++++++++ ...llsafe-38-e3dfe0044b44c8a49414479521acf762 | 42 +++++++++++++++++++ ...llsafe-39-9a7e1f373b9c02e632d6c7c550b908ec | 42 +++++++++++++++++++ ...ullsafe-4-644c616d87ae426eb2f8c71638045185 | 11 +++++ ...llsafe-40-3c868718e4c120cb9a72ab7318c75be3 | 0 ...llsafe-41-1f7d8737c3e2d74d5ad865535d729811 | 9 ++++ ...ullsafe-5-1e393de94850e92b3b00536aacc9371f | 0 ...ullsafe-6-d66451815212e7d17744184e74c6b0a0 | 2 + ...ullsafe-7-a3ad3cc301d9884898d3e6ab6c792d4c | 0 ...ullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac | 29 +++++++++++++ ...ullsafe-9-88f6f40959b0d2faabd9d4b3cd853809 | 0 ...f_equal-0-36b6cdf7c5f68c91155569b1622f5876 | 1 + ...f_equal-1-2422b50b96502dde8b661acdfebd8892 | 2 + ...f_equal-2-e0faab0f5e736c24bcc5503aeac55053 | 1 + ...f_equal-3-39d8d6f197803de927f0af5409ec2f33 | 2 + ...f_equal-4-94ac2476006425e1b3bcddf29ad07b16 | 1 + ...f_equal-5-878650cf21e9360a07d204c8ffb0cde7 | 1 + ...f_equal-6-1635ef051fecdfc7891d9f5a9a3a545e | 1 + ...f_equal-7-78f1b96c199e307714fa1b804e5bae27 | 1 + 58 files changed, 683 insertions(+), 3 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-0-869726b703f160eabdb7763700b53e60 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-1-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-11-3aa243002a5363b84556736ef71613b1 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-12-3cc55b14e8256d2c51361b61986c291e create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-13-69d94e229191e7b9b1a3e7eae46eb993 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-15-507d0fa6d7ce39e2d9921555cea6f8da create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-16-1c714fc339304de4db630530e5d1ce97 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-18-10b2051e65cac50ee1ea1c138ec192c8 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-19-23ab7ac8229a53d391195be7ca092429 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-24-2db30531137611e06fdba478ca7a8412 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-25-e58b2754e8d9c56a473557a549d0d2b9 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-26-64cabe5164130a94f387288f37b62d71 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-28-5a0c946cd7033857ca99e5fb800f8525 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-29-514043c2ddaf6ea8f16a764adc92d1cf create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-3-ae378fc0f875a21884e58fa35a6d52cd create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-31-1cb03e1106f79d14f22bc89d386cedcf create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-33-63157d43422fcedadba408537ccecd5c create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-34-9265f806b71c03061f93f9fbc88aa223 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-35-95815bafb81cccb8129c20d399a446fc create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-36-c4762c60cc93236b7647ebd32a40ce57 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-37-a87893adfc73c9cc63ceab200bb56245 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-38-e3dfe0044b44c8a49414479521acf762 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-39-9a7e1f373b9c02e632d6c7c550b908ec create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-4-644c616d87ae426eb2f8c71638045185 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-40-3c868718e4c120cb9a72ab7318c75be3 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-41-1f7d8737c3e2d74d5ad865535d729811 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-5-1e393de94850e92b3b00536aacc9371f create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-6-d66451815212e7d17744184e74c6b0a0 create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-7-a3ad3cc301d9884898d3e6ab6c792d4c create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac create mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-9-88f6f40959b0d2faabd9d4b3cd853809 create mode 100644 sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 create mode 100644 sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 create mode 100644 sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 create mode 100644 sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 create mode 100644 sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 create mode 100644 sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 create mode 100644 sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e create mode 100644 sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 9887856b9c1c6..67a8ce9b88c3f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -246,6 +246,8 @@ trait HiveTypeCoercion { // No need to change other EqualTo operators as that actually makes sense for boolean types. case e: EqualTo => e + // No need to change the EqualNullSafe operators, too + case e: EqualNullSafe => e // Otherwise turn them to Byte types so that there exists and ordering. case p: BinaryComparison if p.left.dataType == BooleanType && p.right.dataType == BooleanType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 15c98efbcabcf..5c8c810d9135a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -77,6 +77,7 @@ package object dsl { def > (other: Expression) = GreaterThan(expr, other) def >= (other: Expression) = GreaterThanOrEqual(expr, other) def === (other: Expression) = EqualTo(expr, other) + def <=> (other: Expression) = EqualNullSafe(expr, other) def !== (other: Expression) = Not(EqualTo(expr, other)) def in(list: Expression*) = In(expr, list) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index b63406b94a4a3..06b94a98d3cd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -153,6 +153,22 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison } } +case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComparison { + def symbol = "<=>" + override def nullable = false + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) + if (l == null && r == null) { + true + } else if (l == null || r == null) { + false + } else { + l == r + } + } +} + case class LessThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<" override def eval(input: Row): Any = c2(input, left, right, _.lt(_, _)) 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 c65987b7120b2..5f86d6047cb9c 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 @@ -153,6 +153,8 @@ object NullPropagation extends Rule[LogicalPlan] { case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType) case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType) case e @ GetField(Literal(null, _), _) => Literal(null, e.dataType) + case e @ EqualNullSafe(Literal(null, _), r) => IsNull(r) + case e @ EqualNullSafe(l, Literal(null, _)) => IsNull(l) // For Coalesce, remove null literals. case e @ Coalesce(children) => 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 c3f5c26fdbe59..58f8c341e6676 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 @@ -451,11 +451,13 @@ class ExpressionEvaluationSuite extends FunSuite { } test("BinaryComparison") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) val c4 = 'a.int.at(3) + val c5 = 'a.int.at(4) + val c6 = 'a.int.at(5) checkEvaluation(LessThan(c1, c4), null, row) checkEvaluation(LessThan(c1, c2), true, row) @@ -469,6 +471,12 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 >= c2, false, row) checkEvaluation(c1 === c2, false, row) checkEvaluation(c1 !== c2, true, row) + checkEvaluation(c4 <=> c1, false, row) + checkEvaluation(c1 <=> c4, false, row) + checkEvaluation(c4 <=> c6, true, row) + checkEvaluation(c3 <=> c5, true, row) + checkEvaluation(Literal(true) <=> Literal(null, BooleanType), false, row) + checkEvaluation(Literal(null, BooleanType) <=> Literal(true), false, row) } test("StringComparison") { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 8b451973a47a1..c69e93ba2b9ba 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -196,7 +196,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Hive returns the results of describe as plain text. Comments with multiple lines // introduce extra lines in the Hive results, which make the result comparison fail. - "describe_comment_indent" + "describe_comment_indent", + + // Limit clause without a ordering, which causes failure. + "orc_predicate_pushdown" ) /** @@ -503,6 +506,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join_hive_626", "join_map_ppr", "join_nulls", + "join_nullsafe", "join_rc", "join_reorder2", "join_reorder3", @@ -734,6 +738,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_double", "udf_E", "udf_elt", + "udf_equal", "udf_exp", "udf_field", "udf_find_in_set", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 334462357eb86..201c85f3d501e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -253,7 +253,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DecimalType, TimestampType) + ShortType, DecimalType, TimestampType, BinaryType) protected def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => @@ -269,6 +269,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString + case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (other, tpe) if primitiveTypes contains tpe => other.toString } 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 c4ca9f362a04d..4395874526d51 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 @@ -931,6 +931,7 @@ private[hive] object HiveQl { /* Comparisons */ case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) case Token("==", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) + case Token("<=>", left :: right:: Nil) => EqualNullSafe(nodeToExpr(left), nodeToExpr(right)) case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-0-869726b703f160eabdb7763700b53e60 b/sql/hive/src/test/resources/golden/join_nullsafe-0-869726b703f160eabdb7763700b53e60 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-0-869726b703f160eabdb7763700b53e60 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-1-5644ab44e5ba9f2941216b8d5dc33a99 b/sql/hive/src/test/resources/golden/join_nullsafe-1-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 b/sql/hive/src/test/resources/golden/join_nullsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 new file mode 100644 index 0000000000000..31c409082cc2f --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-10-b6de4e85dcc1d1949c7431d39fa1b919 @@ -0,0 +1,2 @@ +NULL 10 10 NULL NULL 10 +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-11-3aa243002a5363b84556736ef71613b1 b/sql/hive/src/test/resources/golden/join_nullsafe-11-3aa243002a5363b84556736ef71613b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-12-3cc55b14e8256d2c51361b61986c291e b/sql/hive/src/test/resources/golden/join_nullsafe-12-3cc55b14e8256d2c51361b61986c291e new file mode 100644 index 0000000000000..9b77d13cbaab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-12-3cc55b14e8256d2c51361b61986c291e @@ -0,0 +1,4 @@ +NULL NULL NULL NULL NULL NULL +NULL 10 10 NULL NULL 10 +10 NULL NULL 10 10 NULL +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-13-69d94e229191e7b9b1a3e7eae46eb993 b/sql/hive/src/test/resources/golden/join_nullsafe-13-69d94e229191e7b9b1a3e7eae46eb993 new file mode 100644 index 0000000000000..47c0709d39851 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-13-69d94e229191e7b9b1a3e7eae46eb993 @@ -0,0 +1,12 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +48 NULL NULL NULL +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c b/sql/hive/src/test/resources/golden/join_nullsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c new file mode 100644 index 0000000000000..36ba48516b658 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-14-cf9ff6ee72a701a8e2f3e7fb0667903c @@ -0,0 +1,12 @@ +NULL NULL NULL NULL +NULL NULL NULL 35 +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-15-507d0fa6d7ce39e2d9921555cea6f8da b/sql/hive/src/test/resources/golden/join_nullsafe-15-507d0fa6d7ce39e2d9921555cea6f8da new file mode 100644 index 0000000000000..fc1fd198cf8be --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-15-507d0fa6d7ce39e2d9921555cea6f8da @@ -0,0 +1,13 @@ +NULL NULL NULL NULL +NULL NULL NULL 35 +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +48 NULL NULL NULL +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-16-1c714fc339304de4db630530e5d1ce97 b/sql/hive/src/test/resources/golden/join_nullsafe-16-1c714fc339304de4db630530e5d1ce97 new file mode 100644 index 0000000000000..1cc70524f9d6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-16-1c714fc339304de4db630530e5d1ce97 @@ -0,0 +1,11 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 b/sql/hive/src/test/resources/golden/join_nullsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 new file mode 100644 index 0000000000000..1cc70524f9d6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-17-8a4b0dc781a28ad11a0db9805fe03aa8 @@ -0,0 +1,11 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-18-10b2051e65cac50ee1ea1c138ec192c8 b/sql/hive/src/test/resources/golden/join_nullsafe-18-10b2051e65cac50ee1ea1c138ec192c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-19-23ab7ac8229a53d391195be7ca092429 b/sql/hive/src/test/resources/golden/join_nullsafe-19-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 b/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 b/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e b/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 b/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 b/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-24-2db30531137611e06fdba478ca7a8412 b/sql/hive/src/test/resources/golden/join_nullsafe-24-2db30531137611e06fdba478ca7a8412 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-24-2db30531137611e06fdba478ca7a8412 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-25-e58b2754e8d9c56a473557a549d0d2b9 b/sql/hive/src/test/resources/golden/join_nullsafe-25-e58b2754e8d9c56a473557a549d0d2b9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-25-e58b2754e8d9c56a473557a549d0d2b9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-26-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_nullsafe-26-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-26-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 b/sql/hive/src/test/resources/golden/join_nullsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-27-e8ed4a1b574a6ca70cbfb3f7b9980aa6 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-28-5a0c946cd7033857ca99e5fb800f8525 b/sql/hive/src/test/resources/golden/join_nullsafe-28-5a0c946cd7033857ca99e5fb800f8525 new file mode 100644 index 0000000000000..2efbef0484452 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-28-5a0c946cd7033857ca99e5fb800f8525 @@ -0,0 +1,14 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-29-514043c2ddaf6ea8f16a764adc92d1cf b/sql/hive/src/test/resources/golden/join_nullsafe-29-514043c2ddaf6ea8f16a764adc92d1cf new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-29-514043c2ddaf6ea8f16a764adc92d1cf @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-3-ae378fc0f875a21884e58fa35a6d52cd b/sql/hive/src/test/resources/golden/join_nullsafe-3-ae378fc0f875a21884e58fa35a6d52cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 b/sql/hive/src/test/resources/golden/join_nullsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-30-fcbf92cb1b85ab01102fbbc6caba9a88 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-31-1cb03e1106f79d14f22bc89d386cedcf b/sql/hive/src/test/resources/golden/join_nullsafe-31-1cb03e1106f79d14f22bc89d386cedcf new file mode 100644 index 0000000000000..66482299904bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-31-1cb03e1106f79d14f22bc89d386cedcf @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 10 +NULL NULL NULL 10 +NULL NULL NULL 35 +NULL NULL NULL 35 +NULL NULL NULL 110 +NULL NULL NULL 110 +NULL NULL NULL 135 +NULL NULL NULL 135 +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 NULL 10 +NULL 10 NULL 35 +NULL 10 NULL 110 +NULL 10 NULL 135 +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 NULL 10 +NULL 35 NULL 35 +NULL 35 NULL 110 +NULL 35 NULL 135 +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 NULL 10 +NULL 110 NULL 35 +NULL 110 NULL 110 +NULL 110 NULL 135 +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 NULL 10 +NULL 135 NULL 35 +NULL 135 NULL 110 +NULL 135 NULL 135 +10 NULL 10 NULL +48 NULL 48 NULL +100 100 100 100 +110 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 b/sql/hive/src/test/resources/golden/join_nullsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 new file mode 100644 index 0000000000000..ea001a222f357 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-32-6a0bf6127d4b042e67ae8ee15125fb87 @@ -0,0 +1,40 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +100 100 100 100 +110 NULL NULL 110 +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-33-63157d43422fcedadba408537ccecd5c b/sql/hive/src/test/resources/golden/join_nullsafe-33-63157d43422fcedadba408537ccecd5c new file mode 100644 index 0000000000000..ea001a222f357 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-33-63157d43422fcedadba408537ccecd5c @@ -0,0 +1,40 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +100 100 100 100 +110 NULL NULL 110 +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-34-9265f806b71c03061f93f9fbc88aa223 b/sql/hive/src/test/resources/golden/join_nullsafe-34-9265f806b71c03061f93f9fbc88aa223 new file mode 100644 index 0000000000000..1093bd89f6e3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-34-9265f806b71c03061f93f9fbc88aa223 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +48 NULL NULL NULL +100 100 100 100 +110 NULL NULL 110 +148 NULL NULL NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-35-95815bafb81cccb8129c20d399a446fc b/sql/hive/src/test/resources/golden/join_nullsafe-35-95815bafb81cccb8129c20d399a446fc new file mode 100644 index 0000000000000..9cf0036674d6e --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-35-95815bafb81cccb8129c20d399a446fc @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL 35 +NULL NULL NULL 135 +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 10 110 NULL +NULL 10 148 NULL +NULL 35 NULL NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +NULL 35 110 NULL +NULL 35 148 NULL +NULL 110 NULL NULL +NULL 110 NULL NULL +NULL 110 10 NULL +NULL 110 48 NULL +NULL 110 110 NULL +NULL 110 148 NULL +NULL 135 NULL NULL +NULL 135 NULL NULL +NULL 135 10 NULL +NULL 135 48 NULL +NULL 135 110 NULL +NULL 135 148 NULL +10 NULL NULL 10 +100 100 100 100 +110 NULL NULL 110 +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-36-c4762c60cc93236b7647ebd32a40ce57 b/sql/hive/src/test/resources/golden/join_nullsafe-36-c4762c60cc93236b7647ebd32a40ce57 new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-36-c4762c60cc93236b7647ebd32a40ce57 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-37-a87893adfc73c9cc63ceab200bb56245 b/sql/hive/src/test/resources/golden/join_nullsafe-37-a87893adfc73c9cc63ceab200bb56245 new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-37-a87893adfc73c9cc63ceab200bb56245 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-38-e3dfe0044b44c8a49414479521acf762 b/sql/hive/src/test/resources/golden/join_nullsafe-38-e3dfe0044b44c8a49414479521acf762 new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-38-e3dfe0044b44c8a49414479521acf762 @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-39-9a7e1f373b9c02e632d6c7c550b908ec b/sql/hive/src/test/resources/golden/join_nullsafe-39-9a7e1f373b9c02e632d6c7c550b908ec new file mode 100644 index 0000000000000..77f6a8ddd7c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-39-9a7e1f373b9c02e632d6c7c550b908ec @@ -0,0 +1,42 @@ +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL NULL 48 NULL +NULL NULL 110 NULL +NULL NULL 110 NULL +NULL NULL 148 NULL +NULL NULL 148 NULL +NULL 10 NULL 10 +NULL 35 NULL 35 +NULL 110 NULL 110 +NULL 135 NULL 135 +10 NULL NULL NULL +10 NULL NULL NULL +10 NULL 10 NULL +10 NULL 48 NULL +10 NULL 110 NULL +10 NULL 148 NULL +48 NULL NULL NULL +48 NULL NULL NULL +48 NULL 10 NULL +48 NULL 48 NULL +48 NULL 110 NULL +48 NULL 148 NULL +100 100 100 100 +110 NULL NULL NULL +110 NULL NULL NULL +110 NULL 10 NULL +110 NULL 48 NULL +110 NULL 110 NULL +110 NULL 148 NULL +148 NULL NULL NULL +148 NULL NULL NULL +148 NULL 10 NULL +148 NULL 48 NULL +148 NULL 110 NULL +148 NULL 148 NULL +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-4-644c616d87ae426eb2f8c71638045185 b/sql/hive/src/test/resources/golden/join_nullsafe-4-644c616d87ae426eb2f8c71638045185 new file mode 100644 index 0000000000000..1cc70524f9d6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-4-644c616d87ae426eb2f8c71638045185 @@ -0,0 +1,11 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL +10 NULL NULL 10 +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-40-3c868718e4c120cb9a72ab7318c75be3 b/sql/hive/src/test/resources/golden/join_nullsafe-40-3c868718e4c120cb9a72ab7318c75be3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-41-1f7d8737c3e2d74d5ad865535d729811 b/sql/hive/src/test/resources/golden/join_nullsafe-41-1f7d8737c3e2d74d5ad865535d729811 new file mode 100644 index 0000000000000..421049d6e509e --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-41-1f7d8737c3e2d74d5ad865535d729811 @@ -0,0 +1,9 @@ +NULL NULL NULL NULL +NULL NULL 10 NULL +NULL NULL 48 NULL +NULL 10 NULL NULL +NULL 10 10 NULL +NULL 10 48 NULL +NULL 35 NULL NULL +NULL 35 10 NULL +NULL 35 48 NULL diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-5-1e393de94850e92b3b00536aacc9371f b/sql/hive/src/test/resources/golden/join_nullsafe-5-1e393de94850e92b3b00536aacc9371f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-6-d66451815212e7d17744184e74c6b0a0 b/sql/hive/src/test/resources/golden/join_nullsafe-6-d66451815212e7d17744184e74c6b0a0 new file mode 100644 index 0000000000000..aec3122cae5f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-6-d66451815212e7d17744184e74c6b0a0 @@ -0,0 +1,2 @@ +10 NULL NULL 10 10 NULL +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-7-a3ad3cc301d9884898d3e6ab6c792d4c b/sql/hive/src/test/resources/golden/join_nullsafe-7-a3ad3cc301d9884898d3e6ab6c792d4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac b/sql/hive/src/test/resources/golden/join_nullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac new file mode 100644 index 0000000000000..30db79efa79b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nullsafe-8-cc7527bcf746ab7e2cd9f28db0ead0ac @@ -0,0 +1,29 @@ +NULL NULL NULL NULL NULL NULL +NULL NULL NULL NULL NULL 10 +NULL NULL NULL NULL NULL 35 +NULL NULL 10 NULL NULL NULL +NULL NULL 10 NULL NULL 10 +NULL NULL 10 NULL NULL 35 +NULL NULL 48 NULL NULL NULL +NULL NULL 48 NULL NULL 10 +NULL NULL 48 NULL NULL 35 +NULL 10 NULL NULL NULL NULL +NULL 10 NULL NULL NULL 10 +NULL 10 NULL NULL NULL 35 +NULL 10 10 NULL NULL NULL +NULL 10 10 NULL NULL 10 +NULL 10 10 NULL NULL 35 +NULL 10 48 NULL NULL NULL +NULL 10 48 NULL NULL 10 +NULL 10 48 NULL NULL 35 +NULL 35 NULL NULL NULL NULL +NULL 35 NULL NULL NULL 10 +NULL 35 NULL NULL NULL 35 +NULL 35 10 NULL NULL NULL +NULL 35 10 NULL NULL 10 +NULL 35 10 NULL NULL 35 +NULL 35 48 NULL NULL NULL +NULL 35 48 NULL NULL 10 +NULL 35 48 NULL NULL 35 +10 NULL NULL 10 10 NULL +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-9-88f6f40959b0d2faabd9d4b3cd853809 b/sql/hive/src/test/resources/golden/join_nullsafe-9-88f6f40959b0d2faabd9d4b3cd853809 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 b/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 new file mode 100644 index 0000000000000..9b9b6312a269a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 @@ -0,0 +1 @@ +a = b - Returns TRUE if a equals b and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 b/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 new file mode 100644 index 0000000000000..30fdf50f62e4e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 @@ -0,0 +1,2 @@ +a = b - Returns TRUE if a equals b and false otherwise +Synonyms: == diff --git a/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 b/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 new file mode 100644 index 0000000000000..d6b4c860778b7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 @@ -0,0 +1 @@ +a == b - Returns TRUE if a equals b and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 b/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 new file mode 100644 index 0000000000000..71e55d6d638a6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 @@ -0,0 +1,2 @@ +a == b - Returns TRUE if a equals b and false otherwise +Synonyms: = diff --git a/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 b/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 new file mode 100644 index 0000000000000..015c417bc68f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 @@ -0,0 +1 @@ +false false true true NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 b/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 new file mode 100644 index 0000000000000..aa7b4b51edea7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 @@ -0,0 +1 @@ +a <=> b - Returns same result with EQUAL(=) operator for non-null operands, but returns TRUE if both are NULL, FALSE if one of the them is NULL diff --git a/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e b/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e new file mode 100644 index 0000000000000..aa7b4b51edea7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e @@ -0,0 +1 @@ +a <=> b - Returns same result with EQUAL(=) operator for non-null operands, but returns TRUE if both are NULL, FALSE if one of the them is NULL diff --git a/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 b/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 new file mode 100644 index 0000000000000..05292fb23192d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 @@ -0,0 +1 @@ +false false true true true false false false false From a2715ccd9437fcdfa0b15e85ab4d0cec91aadf07 Mon Sep 17 00:00:00 2001 From: fireflyc Date: Fri, 25 Jul 2014 10:47:52 -0700 Subject: [PATCH 0072/1492] replace println to log4j Our program needs to receive a large amount of data and run for a long time. We set the log level to WARN but "Storing iterator" "received single" as such message written to the log file. (over yarn) Author: fireflyc Closes #1372 from fireflyc/fix-replace-stdout-log and squashes the following commits: e684140 [fireflyc] 'info' modified into the 'debug' fa22a38 [fireflyc] replace println to log4j --- .../spark/streaming/receiver/ActorReceiver.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index 743be58950c09..1868a1ebc7b4a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -68,13 +68,13 @@ object ActorSupervisorStrategy { * should be same. */ @DeveloperApi -trait ActorHelper { +trait ActorHelper extends Logging{ self: Actor => // to ensure that this can be added to Actor classes only /** Store an iterator of received data as a data block into Spark's memory. */ def store[T](iter: Iterator[T]) { - println("Storing iterator") + logDebug("Storing iterator") context.parent ! IteratorData(iter) } @@ -84,6 +84,7 @@ trait ActorHelper { * that Spark is configured to use. */ def store(bytes: ByteBuffer) { + logDebug("Storing Bytes") context.parent ! ByteBufferData(bytes) } @@ -93,7 +94,7 @@ trait ActorHelper { * being pushed into Spark's memory. */ def store[T](item: T) { - println("Storing item") + logDebug("Storing item") context.parent ! SingleItemData(item) } } @@ -157,15 +158,16 @@ private[streaming] class ActorReceiver[T: ClassTag]( def receive = { case IteratorData(iterator) => - println("received iterator") + logDebug("received iterator") store(iterator.asInstanceOf[Iterator[T]]) case SingleItemData(msg) => - println("received single") + logDebug("received single") store(msg.asInstanceOf[T]) n.incrementAndGet case ByteBufferData(bytes) => + logDebug("received bytes") store(bytes) case props: Props => From 32bcf9af94b39f2c509eb54f8565fb659c70ca97 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 25 Jul 2014 11:14:51 -0700 Subject: [PATCH 0073/1492] [SPARK-2683] unidoc failed because org.apache.spark.util.CallSite uses Java keywords as value names Renaming `short` to `shortForm` and `long` to `longForm`. JIRA: https://issues.apache.org/jira/browse/SPARK-2683 Author: Yin Huai Closes #1585 from yhuai/SPARK-2683 and squashes the following commits: 5ddb843 [Yin Huai] "short" and "long" are Java keyworks. In order to generate javadoc, renaming "short" to "shortForm" and "long" to "longForm". --- .../main/scala/org/apache/spark/SparkContext.scala | 12 +++++++----- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../org/apache/spark/scheduler/DAGScheduler.scala | 4 ++-- .../scala/org/apache/spark/scheduler/Stage.scala | 4 ++-- .../src/main/scala/org/apache/spark/util/Utils.scala | 6 +++--- .../org/apache/spark/SparkContextInfoSuite.scala | 2 +- 6 files changed, 16 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8052499ab7526..3e6addeaf04a8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1037,7 +1037,7 @@ class SparkContext(config: SparkConf) extends Logging { */ private[spark] def getCallSite(): CallSite = { Option(getLocalProperty("externalCallSite")) match { - case Some(callSite) => CallSite(callSite, long = "") + case Some(callSite) => CallSite(callSite, longForm = "") case None => Utils.getCallSite } } @@ -1059,11 +1059,12 @@ class SparkContext(config: SparkConf) extends Logging { } val callSite = getCallSite val cleanedFunc = clean(func) - logInfo("Starting job: " + callSite.short) + logInfo("Starting job: " + callSite.shortForm) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) - logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo( + "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() } @@ -1144,11 +1145,12 @@ class SparkContext(config: SparkConf) extends Logging { evaluator: ApproximateEvaluator[U, R], timeout: Long): PartialResult[R] = { val callSite = getCallSite - logInfo("Starting job: " + callSite.short) + logInfo("Starting job: " + callSite.shortForm) val start = System.nanoTime val result = dagScheduler.runApproximateJob(rdd, func, evaluator, callSite, timeout, localProperties.get) - logInfo("Job finished: " + callSite.short + ", took " + (System.nanoTime - start) / 1e9 + " s") + logInfo( + "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s") result } 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 b1c965a790472..a6abc49c5359e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1225,7 +1225,7 @@ abstract class RDD[T: ClassTag]( /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ @transient private[spark] val creationSite = Utils.getCallSite - private[spark] def getCreationSite: String = Option(creationSite).map(_.short).getOrElse("") + private[spark] def getCreationSite: String = Option(creationSite).map(_.shortForm).getOrElse("") private[spark] def elementClassTag: ClassTag[T] = classTag[T] diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ede3c7d9f01ae..acb4c4946eded 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -455,7 +455,7 @@ class DAGScheduler( waiter.awaitResult() match { case JobSucceeded => {} case JobFailed(exception: Exception) => - logInfo("Failed to run " + callSite.short) + logInfo("Failed to run " + callSite.shortForm) throw exception } } @@ -679,7 +679,7 @@ class DAGScheduler( val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) clearCacheLocs() logInfo("Got job %s (%s) with %d output partitions (allowLocal=%s)".format( - job.jobId, callSite.short, partitions.length, allowLocal)) + job.jobId, callSite.shortForm, partitions.length, allowLocal)) logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") logInfo("Parents of final stage: " + finalStage.parents) logInfo("Missing parents: " + getMissingParentStages(finalStage)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 8ec482a6f6d9c..798cbc598d36e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -108,8 +108,8 @@ private[spark] class Stage( def attemptId: Int = nextAttemptId - val name = callSite.short - val details = callSite.long + val name = callSite.shortForm + val details = callSite.longForm override def toString = "Stage " + id 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 5784e974fbb67..1a4f4eba98ea8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -44,7 +44,7 @@ import org.apache.spark.executor.ExecutorUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ -private[spark] case class CallSite(short: String, long: String) +private[spark] case class CallSite(shortForm: String, longForm: String) /** * Various utility methods used by Spark. @@ -848,8 +848,8 @@ private[spark] object Utils extends Logging { } val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt CallSite( - short = "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine), - long = callStack.take(callStackDepth).mkString("\n")) + shortForm = "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine), + longForm = callStack.take(callStackDepth).mkString("\n")) } /** Return a string containing part of a file from byte 'start' to 'end'. */ diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 1fde4badda949..fb18c3ebfe46f 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -70,7 +70,7 @@ package object testPackage extends Assertions { def runCallSiteTest(sc: SparkContext) { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) val rddCreationSite = rdd.getCreationSite - val curCallSite = sc.getCallSite().short // note: 2 lines after definition of "rdd" + val curCallSite = sc.getCallSite().shortForm // note: 2 lines after definition of "rdd" val rddCreationLine = rddCreationSite match { case CALL_SITE_REGEX(func, file, line) => { From 06dc0d2c6b69c5d59b4d194ced2ac85bfe2e05e2 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 25 Jul 2014 12:20:49 -0700 Subject: [PATCH 0074/1492] [SPARK-2410][SQL] Merging Hive Thrift/JDBC server JIRA issue: - Main: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) - Related: [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678) Cherry picked the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc). (Thanks chenghao-intel for his initial contribution of the Spark SQL CLI.) TODO - [x] Use `spark-submit` to launch the server, the CLI and beeline - [x] Migration guideline draft for Shark users ---- Hit by a bug in `SparkSubmitArguments` while working on this PR: all application options that are recognized by `SparkSubmitArguments` are stolen as `SparkSubmit` options. For example: ```bash $ spark-submit --class org.apache.hive.beeline.BeeLine spark-internal --help ``` This actually shows usage information of `SparkSubmit` rather than `BeeLine`. ~~Fixed this bug here since the `spark-internal` related stuff also touches `SparkSubmitArguments` and I'd like to avoid conflict.~~ **UPDATE** The bug mentioned above is now tracked by [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678). Decided to revert changes to this bug since it involves more subtle considerations and worth a separate PR. Author: Cheng Lian Closes #1399 from liancheng/thriftserver and squashes the following commits: 090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR 21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd] 199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver 1083e9d [Cheng Lian] Fixed failed test suites 7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic 9cc0f06 [Cheng Lian] Starts beeline with spark-submit cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile 061880f [Cheng Lian] Addressed all comments by @pwendell 7755062 [Cheng Lian] Adapts test suites to spark-submit settings 40bafef [Cheng Lian] Fixed more license header issues e214aab [Cheng Lian] Added missing license headers b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft 3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit 61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit 2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server --- .gitignore | 1 + assembly/pom.xml | 10 + bagel/pom.xml | 2 +- bin/beeline | 45 +++ bin/compute-classpath.sh | 1 + bin/spark-shell | 4 +- bin/spark-shell.cmd | 2 +- bin/spark-sql | 36 ++ core/pom.xml | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 14 +- .../spark/deploy/SparkSubmitArguments.scala | 5 +- dev/create-release/create-release.sh | 10 +- dev/run-tests | 2 +- dev/scalastyle | 2 +- docs/sql-programming-guide.md | 200 +++++++++- examples/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 7 +- project/SparkBuild.scala | 14 +- sbin/start-thriftserver.sh | 36 ++ sql/catalyst/pom.xml | 2 +- .../sql/catalyst/plans/logical/commands.scala | 3 +- sql/core/pom.xml | 2 +- .../scala/org/apache/spark/sql/SQLConf.scala | 20 +- .../apache/spark/sql/execution/commands.scala | 42 ++- .../org/apache/spark/sql/SQLConfSuite.scala | 13 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- sql/hive-thriftserver/pom.xml | 82 +++++ .../hive/thriftserver/HiveThriftServer2.scala | 97 +++++ .../hive/thriftserver/ReflectionUtils.scala | 58 +++ .../hive/thriftserver/SparkSQLCLIDriver.scala | 344 ++++++++++++++++++ .../thriftserver/SparkSQLCLIService.scala | 74 ++++ .../hive/thriftserver/SparkSQLDriver.scala | 93 +++++ .../sql/hive/thriftserver/SparkSQLEnv.scala | 58 +++ .../thriftserver/SparkSQLSessionManager.scala | 49 +++ .../server/SparkSQLOperationManager.scala | 151 ++++++++ .../test/resources/data/files/small_kv.txt | 5 + .../sql/hive/thriftserver/CliSuite.scala | 59 +++ .../thriftserver/HiveThriftServer2Suite.scala | 125 +++++++ .../sql/hive/thriftserver/TestUtils.scala | 108 ++++++ sql/hive/pom.xml | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 50 ++- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 54 files changed, 1772 insertions(+), 96 deletions(-) create mode 100755 bin/beeline create mode 100755 bin/spark-sql create mode 100755 sbin/start-thriftserver.sh create mode 100644 sql/hive-thriftserver/pom.xml create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala create mode 100755 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala create mode 100644 sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala diff --git a/.gitignore b/.gitignore index 061c8946d23c1..5b56a67c883e6 100644 --- a/.gitignore +++ b/.gitignore @@ -57,3 +57,4 @@ metastore_db/ metastore/ warehouse/ TempStatsStore/ +sql/hive-thriftserver/test_warehouses diff --git a/assembly/pom.xml b/assembly/pom.xml index 567a8dd2a0d94..703f15925bc44 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -165,6 +165,16 @@
+ + hive-thriftserver + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl diff --git a/bagel/pom.xml b/bagel/pom.xml index 90c4b095bb611..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-bagel_2.10 - bagel + bagel jar Spark Project Bagel diff --git a/bin/beeline b/bin/beeline new file mode 100755 index 0000000000000..09fe366c609fa --- /dev/null +++ b/bin/beeline @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +# Find the java binary +if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" +else + if [ `command -v java` ]; then + RUNNER="java" + else + echo "JAVA_HOME is not set" >&2 + exit 1 + fi +fi + +# Compute classpath using external script +classpath_output=$($FWDIR/bin/compute-classpath.sh) +if [[ "$?" != "0" ]]; then + echo "$classpath_output" + exit 1 +else + CLASSPATH=$classpath_output +fi + +CLASS="org.apache.hive.beeline.BeeLine" +exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index e81e8c060cb98..16b794a1592e8 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -52,6 +52,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" fi diff --git a/bin/spark-shell b/bin/spark-shell index 850e9507ec38f..756c8179d12b6 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 4b9708a8c03f3..b56d69801171c 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell --class org.apache.spark.repl.Main %* diff --git a/bin/spark-sql b/bin/spark-sql new file mode 100755 index 0000000000000..bba7f897b19bc --- /dev/null +++ b/bin/spark-sql @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# Shell script for starting the Spark SQL CLI + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/spark-sql [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + +CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/core/pom.xml b/core/pom.xml index 1054cec4d77bb..a24743495b0e1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-core_2.10 - core + core jar Spark Project Core diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3b5642b6caa36..c9cec33ebaa66 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -46,6 +46,10 @@ object SparkSubmit { private val CLUSTER = 2 private val ALL_DEPLOY_MODES = CLIENT | CLUSTER + // A special jar name that indicates the class being run is inside of Spark itself, and therefore + // no user jar is needed. + private val SPARK_INTERNAL = "spark-internal" + // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" @@ -257,7 +261,9 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (clusterManager == YARN && deployMode == CLUSTER) { childMainClass = "org.apache.spark.deploy.yarn.Client" - childArgs += ("--jar", args.primaryResource) + if (args.primaryResource != SPARK_INTERNAL) { + childArgs += ("--jar", args.primaryResource) + } childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } @@ -332,7 +338,7 @@ object SparkSubmit { * Return whether the given primary resource represents a user jar. */ private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) + !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) } /** @@ -349,6 +355,10 @@ object SparkSubmit { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } + private[spark] def isInternal(primaryResource: String): Boolean = { + primaryResource == SPARK_INTERNAL + } + /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 3ab67a43a3b55..01d0ae541a66b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -204,8 +204,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { - // Delineates parsing of Spark options from parsing of user options. var inSparkOpts = true + + // Delineates parsing of Spark options from parsing of user options. parse(opts) def parse(opts: Seq[String]): Unit = opts match { @@ -318,7 +319,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit(errMessage) case v => primaryResource = - if (!SparkSubmit.isShell(v)) { + if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) { Utils.resolveURI(v).toString } else { v diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 38830103d1e8d..33de24d1ae6d7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -53,7 +53,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare @@ -61,7 +61,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. @@ -111,10 +111,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" +make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" make_binary_release "hadoop2" \ - "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" # Copy data echo "Copying release tarballs" diff --git a/dev/run-tests b/dev/run-tests index 51e4def0f835a..98ec969dc1b37 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -65,7 +65,7 @@ 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" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive" sbt/sbt clean package \ + echo -e "q\n" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive -Phive-thriftserver" 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 | \ diff --git a/dev/scalastyle b/dev/scalastyle index a02d06912f238..d9f2b91a3a091 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 38728534a46e0..36d642f2923b2 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -136,7 +136,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD // Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) @@ -548,7 +548,6 @@ results = hiveContext.hql("FROM src SELECT key, value").collect() - # Writing Language-Integrated Relational Queries **Language-Integrated queries are currently only supported in Scala.** @@ -573,4 +572,199 @@ prefixed with a tick (`'`). Implicit conversions turn these symbols into expres evaluated by the SQL execution engine. A full list of the functions supported can be found in the [ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). - \ No newline at end of file + + +## Running the Thrift JDBC server + +The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] +(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test +the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive +you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` +for maven). + +To start the JDBC server, run the following in the Spark directory: + + ./sbin/start-thriftserver.sh + +The default port the server listens on is 10000. You may run +`./sbin/start-thriftserver.sh --help` for a complete list of all available +options. Now you can use beeline to test the Thrift JDBC server: + + ./bin/beeline + +Connect to the JDBC server in beeline with: + + beeline> !connect jdbc:hive2://localhost:10000 + +Beeline will ask you for a username and password. In non-secure mode, simply enter the username on +your machine and a blank password. For secure mode, please follow the instructions given in the +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. + +You may also use the beeline script comes with Hive. + +### Migration Guide for Shark Users + +#### Reducer number + +In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark +SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value +is 200. Users may customize this property via `SET`: + +``` +SET spark.sql.shuffle.partitions=10; +SELECT page, count(*) c FROM logs_last_month_cached +GROUP BY page ORDER BY c DESC LIMIT 10; +``` + +You may also put this property in `hive-site.xml` to override the default value. + +For now, the `mapred.reduce.tasks` property is still recognized, and is converted to +`spark.sql.shuffle.partitions` automatically. + +#### Caching + +The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no +longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to +let user control table caching explicitly: + +``` +CACHE TABLE logs_last_month; +UNCACHE TABLE logs_last_month; +``` + +**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", +but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be +cached, you may simply count the table immediately after executing `CACHE TABLE`: + +``` +CACHE TABLE logs_last_month; +SELECT COUNT(1) FROM logs_last_month; +``` + +Several caching related features are not supported yet: + +* User defined partition level cache eviction policy +* RDD reloading +* In-memory cache write through policy + +### Compatibility with Apache Hive + +#### Deploying in Exising Hive Warehouses + +Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +#### Supported Hive Features + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathemtatical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDe's) +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +#### Unsupported Hive Functionality + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + +**Esoteric Hive Features** + +* Tables with partitions using different input formats: In Spark SQL, all table partitions need to + have the same input format. +* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions + (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. +* `UNIONTYPE` +* Unique join +* Single query multi insert +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +not necessary due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block level bitmap indexes and virtual columns (used to build indexes) +* Automatically convert a join to map join: For joining a large table with multiple small tables, + Hive automatically converts the join into a map join. We are adding this auto conversion in the + next release. +* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you + need to control the degree of parallelism post-shuffle using "SET + spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the + next release. +* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available +options. diff --git a/examples/pom.xml b/examples/pom.xml index bd1c387c2eb91..c4ed0f5a6a02b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-examples_2.10 - examples + examples jar Spark Project Examples diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 61a6aff543aed..874b8a7959bb6 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-flume_2.10 - streaming-flume + streaming-flume jar Spark Project External Flume diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4762c50685a93..25a5c0a4d7d77 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-kafka_2.10 - streaming-kafka + streaming-kafka jar Spark Project External Kafka diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 32c530e600ce0..f31ed655f6779 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-mqtt_2.10 - streaming-mqtt + streaming-mqtt jar Spark Project External MQTT diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 637adb0f00da0..56bb24c2a072e 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-twitter_2.10 - streaming-twitter + streaming-twitter jar Spark Project External Twitter diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index e4d758a04a4cd..54b0242c54e78 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-zeromq_2.10 - streaming-zeromq + streaming-zeromq jar Spark Project External ZeroMQ diff --git a/graphx/pom.xml b/graphx/pom.xml index 7e3bcf29dcfbc..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-graphx_2.10 - graphx + graphx jar Spark Project GraphX diff --git a/mllib/pom.xml b/mllib/pom.xml index 92b07e2357db1..f27cf520dc9fa 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-mllib_2.10 - mllib + mllib jar Spark Project ML Library diff --git a/pom.xml b/pom.xml index 4e2d64a833640..3e9d388180d8e 100644 --- a/pom.xml +++ b/pom.xml @@ -95,6 +95,7 @@ sql/catalyst sql/core sql/hive + sql/hive-thriftserver repl assembly external/twitter @@ -252,9 +253,9 @@ 3.3.2 - commons-codec - commons-codec - 1.5 + commons-codec + commons-codec + 1.5 com.google.code.findbugs diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5461d25d72d7e..86d47734e77bb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -29,11 +29,11 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, mllib, repl, spark, sql, streaming, - streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = - Seq("bagel", "catalyst", "core", "graphx", "hive", "mllib", "repl", "spark", "sql", - "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", - "streaming-zeromq").map(ProjectRef(buildLocation, _)) + val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, + streaming, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = + Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", + "spark", "sql", "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", + "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") @@ -99,7 +99,7 @@ object SparkBuild extends PomBuild { Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) - case _ => + case _ => } override val userPropertiesMap = System.getProperties.toMap @@ -157,7 +157,7 @@ object SparkBuild extends PomBuild { /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl).exists(x => x == y)). + allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl).contains(x)). foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh new file mode 100755 index 0000000000000..8398e6f19b511 --- /dev/null +++ b/sbin/start-thriftserver.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# Shell script for starting the Spark SQL Thrift server + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/start-thriftserver [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + +CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" +exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6decde3fcd62d..531bfddbf237b 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -32,7 +32,7 @@ Spark Project Catalyst http://spark.apache.org/ - catalyst + catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 1d5f033f0d274..a357c6ffb8977 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -43,8 +43,7 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(0, AttributeReference("key", StringType, nullable = false)()), - BoundReference(1, AttributeReference("value", StringType, nullable = false)())) + BoundReference(1, AttributeReference("", StringType, nullable = false)())) } /** diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c309c43804d97..3a038a2db6173 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -32,7 +32,7 @@ Spark Project SQL http://spark.apache.org/ - sql + sql diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 2b787e14f3f15..41920c00b5a2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -30,12 +30,13 @@ import scala.collection.JavaConverters._ * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { + import SQLConf._ /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? /** Number of partitions to use for shuffle operators. */ - private[spark] def numShufflePartitions: Int = get("spark.sql.shuffle.partitions", "200").toInt + private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to @@ -43,11 +44,10 @@ trait SQLConf { * effectively disables auto conversion. * Hive setting: hive.auto.convert.join.noconditionaltask.size. */ - private[spark] def autoConvertJoinSize: Int = - get("spark.sql.auto.convert.join.size", "10000").toInt + private[spark] def autoConvertJoinSize: Int = get(AUTO_CONVERT_JOIN_SIZE, "10000").toInt /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get("spark.sql.join.broadcastTables", "") + private[spark] def joinBroadcastTables: String = get(JOIN_BROADCAST_TABLES, "") /** ********************** SQLConf functionality methods ************ */ @@ -61,7 +61,7 @@ trait SQLConf { def set(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for ${key}") + require(value != null, s"value cannot be null for $key") settings.put(key, value) } @@ -90,3 +90,13 @@ trait SQLConf { } } + +object SQLConf { + val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" + val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" + val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" + + object Deprecated { + val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 98d2f89c8ae71..9293239131d52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.execution +import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SQLConf, SQLContext} trait Command { /** @@ -44,28 +45,53 @@ trait Command { case class SetCommand( key: Option[String], value: Option[String], output: Seq[Attribute])( @transient context: SQLContext) - extends LeafNode with Command { + extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => - context.set(k, v) - Array(k -> v) + if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") + context.set(SQLConf.SHUFFLE_PARTITIONS, v) + Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") + } else { + context.set(k, v) + Array(s"$k=$v") + } // Query the value bound to key k. case (Some(k), _) => - Array(k -> context.getOption(k).getOrElse("")) + // TODO (lian) This is just a workaround to make the Simba ODBC driver work. + // Should remove this once we get the ODBC driver updated. + if (k == "-v") { + val hiveJars = Seq( + "hive-exec-0.12.0.jar", + "hive-service-0.12.0.jar", + "hive-common-0.12.0.jar", + "hive-hwi-0.12.0.jar", + "hive-0.12.0.jar").mkString(":") + + Array( + "system:java.class.path=" + hiveJars, + "system:sun.java.command=shark.SharkServer2") + } + else { + Array(s"$k=${context.getOption(k).getOrElse("")}") + } // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => - context.getAll + context.getAll.map { case (k, v) => + s"$k=$v" + } case _ => throw new IllegalArgumentException() } def execute(): RDD[Row] = { - val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) } + val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } context.sparkContext.parallelize(rows, 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 08293f7f0ca30..1a58d73d9e7f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -54,10 +54,10 @@ class SQLConfSuite extends QueryTest { assert(get(testKey, testVal + "_") == testVal) assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - sql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - sql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + sql("set some.property=20") + assert(get("some.property", "0") == "20") + sql("set some.property = 40") + assert(get("some.property", "0") == "40") val key = "spark.sql.key" val vs = "val0,val_1,val2.3,my_table" @@ -70,4 +70,9 @@ class SQLConfSuite extends QueryTest { clear() } + test("deprecated property") { + clear() + sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") + assert(get(SQLConf.SHUFFLE_PARTITIONS) == "10") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6736189c96d4b..de9e8aa4f62ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -424,25 +424,25 @@ class SQLQuerySuite extends QueryTest { sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - Seq(Seq(testKey, testVal)) + Seq(Seq(s"$testKey=$testVal")) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), Seq( - Seq(testKey, testVal), - Seq(testKey + testKey, testVal + testVal)) + Seq(s"$testKey=$testVal"), + Seq(s"${testKey + testKey}=${testVal + testVal}")) ) // "set key" checkAnswer( sql(s"SET $testKey"), - Seq(Seq(testKey, testVal)) + Seq(Seq(s"$testKey=$testVal")) ) checkAnswer( sql(s"SET $nonexistentKey"), - Seq(Seq(nonexistentKey, "")) + Seq(Seq(s"$nonexistentKey=")) ) clear() } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml new file mode 100644 index 0000000000000..7fac90fdc596d --- /dev/null +++ b/sql/hive-thriftserver/pom.xml @@ -0,0 +1,82 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive-thriftserver_2.10 + jar + Spark Project Hive + http://spark.apache.org/ + + hive-thriftserver + + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + org.spark-project.hive + hive-cli + ${hive.version} + + + org.spark-project.hive + hive-jdbc + ${hive.version} + + + org.spark-project.hive + hive-beeline + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala new file mode 100644 index 0000000000000..ddbc2a79fb512 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService +import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a + * `HiveThriftServer2` thrift server. + */ +private[hive] object HiveThriftServer2 extends Logging { + var LOG = LogFactory.getLog(classOf[HiveServer2]) + + def main(args: Array[String]) { + val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") + + if (!optionsProcessor.process(args)) { + logger.warn("Error starting HiveThriftServer2 with given arguments") + System.exit(-1) + } + + val ss = new SessionState(new HiveConf(classOf[SessionState])) + + // Set all properties specified via command line. + val hiveConf: HiveConf = ss.getConf + hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => + logger.debug(s"HiveConf var: $k=$v") + } + + SessionState.start(ss) + + logger.info("Starting SparkContext") + SparkSQLEnv.init() + SessionState.start(ss) + + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.sparkContext.stop() + } + } + ) + + try { + val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) + server.init(hiveConf) + server.start() + logger.info("HiveThriftServer2 started") + } catch { + case e: Exception => + logger.error("Error starting HiveThriftServer2", e) + System.exit(-1) + } + } +} + +private[hive] class HiveThriftServer2(hiveContext: HiveContext) + extends HiveServer2 + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + val sparkSqlCliService = new SparkSQLCLIService(hiveContext) + setSuperField(this, "cliService", sparkSqlCliService) + addService(sparkSqlCliService) + + val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala new file mode 100644 index 0000000000000..599294dfbb7d7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +private[hive] object ReflectionUtils { + def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { + setAncestorField(obj, 1, fieldName, fieldValue) + } + + def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { + val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.set(obj, fieldValue) + } + + def getSuperField[T](obj: AnyRef, fieldName: String): T = { + getAncestorField[T](obj, 1, fieldName) + } + + def getAncestorField[T](clazz: Object, level: Int, fieldName: String): T = { + val ancestor = Iterator.iterate[Class[_]](clazz.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.get(clazz).asInstanceOf[T] + } + + def invokeStatic(clazz: Class[_], methodName: String, args: (Class[_], AnyRef)*): AnyRef = { + invoke(clazz, null, methodName, args: _*) + } + + def invoke( + clazz: Class[_], + obj: AnyRef, + methodName: String, + args: (Class[_], AnyRef)*): AnyRef = { + + val (types, values) = args.unzip + val method = clazz.getDeclaredMethod(methodName, types: _*) + method.setAccessible(true) + method.invoke(obj, values.toSeq: _*) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala new file mode 100755 index 0000000000000..27268ecb923e9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io._ +import java.util.{ArrayList => JArrayList} + +import jline.{ConsoleReader, History} +import org.apache.commons.lang.StringUtils +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException +import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.thrift.transport.TSocket + +import org.apache.spark.sql.Logging + +private[hive] object SparkSQLCLIDriver { + private var prompt = "spark-sql" + private var continuedPrompt = "".padTo(prompt.length, ' ') + private var transport:TSocket = _ + + installSignalHandler() + + /** + * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), + * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while + * a command is being processed by the current thread. + */ + def installSignalHandler() { + HiveInterruptUtils.add(new HiveInterruptCallback { + override def interrupt() { + // Handle remote execution mode + if (SparkSQLEnv.sparkContext != null) { + SparkSQLEnv.sparkContext.cancelAllJobs() + } else { + if (transport != null) { + // Force closing of TCP connection upon session termination + transport.getSocket.close() + } + } + } + }) + } + + def main(args: Array[String]) { + val oproc = new OptionsProcessor() + if (!oproc.process_stage1(args)) { + System.exit(1) + } + + // NOTE: It is critical to do this here so that log4j is reinitialized + // before any of the other core hive classes are loaded + var logInitFailed = false + var logInitDetailMessage: String = null + try { + logInitDetailMessage = LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => + logInitFailed = true + logInitDetailMessage = e.getMessage + } + + val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) + + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + if (!oproc.process_stage2(sessionState)) { + System.exit(2) + } + + if (!sessionState.getIsSilent) { + if (logInitFailed) System.err.println(logInitDetailMessage) + else SessionState.getConsole.printInfo(logInitDetailMessage) + } + + // Set all properties specified via command line. + val conf: HiveConf = sessionState.getConf + sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => + conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + sessionState.getOverriddenConfigurations.put( + item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + } + + SessionState.start(sessionState) + + // Clean up after we exit + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.stop() + } + } + ) + + // "-h" option has been passed, so connect to Hive thrift server. + if (sessionState.getHost != null) { + sessionState.connect() + if (sessionState.isRemoteMode) { + prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt + continuedPrompt = "".padTo(prompt.length, ' ') + } + } + + if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { + // Hadoop-20 and above - we need to augment classpath using hiveconf + // components. + // See also: code in ExecDriver.java + var loader = conf.getClassLoader + val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) + if (StringUtils.isNotBlank(auxJars)) { + loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")) + } + conf.setClassLoader(loader) + Thread.currentThread().setContextClassLoader(loader) + } + + val cli = new SparkSQLCLIDriver + cli.setHiveVariables(oproc.getHiveVariables) + + // TODO work around for set the log output to console, because the HiveContext + // will set the output into an invalid buffer. + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + // Execute -i init files (always in silent mode) + cli.processInitFiles(sessionState) + + if (sessionState.execString != null) { + System.exit(cli.processLine(sessionState.execString)) + } + + try { + if (sessionState.fileName != null) { + System.exit(cli.processFile(sessionState.fileName)) + } + } catch { + case e: FileNotFoundException => + System.err.println(s"Could not open input file for reading. (${e.getMessage})") + System.exit(3) + } + + val reader = new ConsoleReader() + reader.setBellEnabled(false) + // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) + CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e)) + + val historyDirectory = System.getProperty("user.home") + + try { + if (new File(historyDirectory).exists()) { + val historyFile = historyDirectory + File.separator + ".hivehistory" + reader.setHistory(new History(new File(historyFile))) + } else { + System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + + " does not exist. History will not be available during this session.") + } + } catch { + case e: Exception => + System.err.println("WARNING: Encountered an error while trying to initialize Hive's " + + "history file. History will not be available during this session.") + System.err.println(e.getMessage) + } + + val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") + clientTransportTSocketField.setAccessible(true) + + transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket] + + var ret = 0 + var prefix = "" + val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", + classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) + + def promptWithCurrentDB = s"$prompt$currentDB" + def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( + classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) + + var currentPrompt = promptWithCurrentDB + var line = reader.readLine(currentPrompt + "> ") + + while (line != null) { + if (prefix.nonEmpty) { + prefix += '\n' + } + + if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { + line = prefix + line + ret = cli.processLine(line, true) + prefix = "" + currentPrompt = promptWithCurrentDB + } else { + prefix = prefix + line + currentPrompt = continuedPromptWithDBSpaces + } + + line = reader.readLine(currentPrompt + "> ") + } + + sessionState.close() + + System.exit(ret) + } +} + +private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { + private val sessionState = SessionState.get().asInstanceOf[CliSessionState] + + private val LOG = LogFactory.getLog("CliDriver") + + private val console = new SessionState.LogHelper(LOG) + + private val conf: Configuration = + if (sessionState != null) sessionState.getConf else new Configuration() + + // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver + // because the Hive unit tests do not go through the main() code path. + if (!sessionState.isRemoteMode) { + SparkSQLEnv.init() + } + + override def processCmd(cmd: String): Int = { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + if (cmd_trimmed.toLowerCase.equals("quit") || + cmd_trimmed.toLowerCase.equals("exit") || + tokens(0).equalsIgnoreCase("source") || + cmd_trimmed.startsWith("!") || + tokens(0).toLowerCase.equals("list") || + sessionState.isRemoteMode) { + val start = System.currentTimeMillis() + super.processCmd(cmd) + val end = System.currentTimeMillis() + val timeTaken: Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds") + 0 + } else { + var ret = 0 + val hconf = conf.asInstanceOf[HiveConf] + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) + + if (proc != null) { + if (proc.isInstanceOf[Driver]) { + val driver = new SparkSQLDriver + + driver.init() + val out = sessionState.out + val start:Long = System.currentTimeMillis() + if (sessionState.getIsVerbose) { + out.println(cmd) + } + + ret = driver.run(cmd).getResponseCode + if (ret != 0) { + driver.close() + return ret + } + + val res = new JArrayList[String]() + + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { + // Print the column names. + Option(driver.getSchema.getFieldSchemas).map { fields => + out.println(fields.map(_.getName).mkString("\t")) + } + } + + try { + while (!out.checkError() && driver.getResults(res)) { + res.foreach(out.println) + res.clear() + } + } catch { + case e:IOException => + console.printError( + s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} + |${org.apache.hadoop.util.StringUtils.stringifyException(e)} + """.stripMargin) + ret = 1 + } + + val cret = driver.close() + if (ret == 0) { + ret = cret + } + + val end = System.currentTimeMillis() + if (end > start) { + val timeTaken:Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds", null) + } + + // Destroy the driver to release all the locks. + driver.destroy() + } else { + if (sessionState.getIsVerbose) { + sessionState.out.println(tokens(0) + " " + cmd_1) + } + ret = proc.run(cmd_1).getResponseCode + } + } + ret + } + } +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala new file mode 100644 index 0000000000000..42cbf363b274f --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io.IOException +import java.util.{List => JList} +import javax.security.auth.login.LoginException + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hive.service.Service.STATE +import org.apache.hive.service.auth.HiveAuthFactory +import org.apache.hive.service.cli.CLIService +import org.apache.hive.service.{AbstractService, Service, ServiceException} + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +private[hive] class SparkSQLCLIService(hiveContext: HiveContext) + extends CLIService + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) + setSuperField(this, "sessionManager", sparkSqlSessionManager) + addService(sparkSqlSessionManager) + + try { + HiveAuthFactory.loginFromKeytab(hiveConf) + val serverUserName = ShimLoader.getHadoopShims + .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) + setSuperField(this, "serverUserName", serverUserName) + } catch { + case e @ (_: IOException | _: LoginException) => + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + } + + initCompositeService(hiveConf) + } +} + +private[thriftserver] trait ReflectedCompositeService { this: AbstractService => + def initCompositeService(hiveConf: HiveConf) { + // Emulating `CompositeService.init(hiveConf)` + val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") + serviceList.foreach(_.init(hiveConf)) + + // Emulating `AbstractService.init(hiveConf)` + invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) + setAncestorField(this, 3, "hiveConf", hiveConf) + invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) + getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala new file mode 100644 index 0000000000000..5202aa9903e03 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.util.{ArrayList => JArrayList} + +import org.apache.commons.lang.exception.ExceptionUtils +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} + +private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) + extends Driver with Logging { + + private var tableSchema: Schema = _ + private var hiveResponse: Seq[String] = _ + + override def init(): Unit = { + } + + private def getResultSetSchema(query: context.QueryExecution): Schema = { + val analyzed = query.analyzed + logger.debug(s"Result Schema: ${analyzed.output}") + if (analyzed.output.size == 0) { + new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) + } else { + val fieldSchemas = analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + + new Schema(fieldSchemas, null) + } + } + + override def run(command: String): CommandProcessorResponse = { + val execution = context.executePlan(context.hql(command).logicalPlan) + + // TODO unify the error code + try { + hiveResponse = execution.stringResult() + tableSchema = getResultSetSchema(execution) + new CommandProcessorResponse(0) + } catch { + case cause: Throwable => + logger.error(s"Failed in [$command]", cause) + new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) + } + } + + override def close(): Int = { + hiveResponse = null + tableSchema = null + 0 + } + + override def getSchema: Schema = tableSchema + + override def getResults(res: JArrayList[String]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.addAll(hiveResponse) + hiveResponse = null + true + } + } + + override def destroy() { + super.destroy() + hiveResponse = null + tableSchema = null + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala new file mode 100644 index 0000000000000..451c3bd7b9352 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} + +/** A singleton object for the master program. The slaves should not access this. */ +private[hive] object SparkSQLEnv extends Logging { + logger.debug("Initializing SparkSQLEnv") + + var hiveContext: HiveContext = _ + var sparkContext: SparkContext = _ + + def init() { + if (hiveContext == null) { + sparkContext = new SparkContext(new SparkConf() + .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) + + sparkContext.addSparkListener(new StatsReportListener()) + + hiveContext = new HiveContext(sparkContext) { + @transient override lazy val sessionState = SessionState.get() + @transient override lazy val hiveconf = sessionState.getConf + } + } + } + + /** Cleans up and shuts down the Spark SQL environments. */ + def stop() { + logger.debug("Shutting down Spark SQL Environment") + // Stop the SparkContext + if (SparkSQLEnv.sparkContext != null) { + sparkContext.stop() + sparkContext = null + hiveContext = null + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala new file mode 100644 index 0000000000000..6b3275b4eaf04 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.concurrent.Executors + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.session.SessionManager + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala new file mode 100644 index 0000000000000..a4e1f3e762e89 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver.server + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.math.{random, round} + +import java.sql.Timestamp +import java.util.{Map => JMap} + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} + +/** + * Executes queries using Spark SQL, and maintains a list of handles to active queries. + */ +class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { + val handleToOperation = ReflectionUtils + .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") + + override def newExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String], + async: Boolean): ExecuteStatementOperation = synchronized { + + val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logger.debug("CLOSING") + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + if (!iter.hasNext) { + new RowSet() + } else { + val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. + var curRow = 0 + var rowSet = new ArrayBuffer[Row](maxRows) + + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = new Row() + var curCol = 0 + + while (curCol < sparkRow.length) { + dataTypes(curCol) match { + case StringType => + row.addString(sparkRow(curCol).asInstanceOf[String]) + case IntegerType => + row.addColumnValue(ColumnValue.intValue(sparkRow.getInt(curCol))) + case BooleanType => + row.addColumnValue(ColumnValue.booleanValue(sparkRow.getBoolean(curCol))) + case DoubleType => + row.addColumnValue(ColumnValue.doubleValue(sparkRow.getDouble(curCol))) + case FloatType => + row.addColumnValue(ColumnValue.floatValue(sparkRow.getFloat(curCol))) + case DecimalType => + val hiveDecimal = sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal + row.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) + case LongType => + row.addColumnValue(ColumnValue.longValue(sparkRow.getLong(curCol))) + case ByteType => + row.addColumnValue(ColumnValue.byteValue(sparkRow.getByte(curCol))) + case ShortType => + row.addColumnValue(ColumnValue.intValue(sparkRow.getShort(curCol))) + case TimestampType => + row.addColumnValue( + ColumnValue.timestampValue(sparkRow.get(curCol).asInstanceOf[Timestamp])) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((sparkRow.get(curCol), dataTypes(curCol))) + row.addColumnValue(ColumnValue.stringValue(hiveString)) + } + curCol += 1 + } + rowSet += row + curRow += 1 + } + new RowSet(rowSet, 0) + } + } + + def getResultSetSchema: TableSchema = { + logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + def run(): Unit = { + logger.info(s"Running query '$statement'") + setState(OperationState.RUNNING) + try { + result = hiveContext.hql(statement) + logger.debug(result.queryExecution.toString()) + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + iter = result.queryExecution.toRdd.toLocalIterator + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + setHasResultSet(true) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logger.error("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + setState(OperationState.FINISHED) + } + } + + handleToOperation.put(operation.getHandle, operation) + operation + } +} diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt new file mode 100644 index 0000000000000..850f8014b6f05 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt @@ -0,0 +1,5 @@ +238val_238 +86val_86 +311val_311 +27val_27 +165val_165 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala new file mode 100644 index 0000000000000..b90670a796b81 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, InputStreamReader, PrintWriter} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.sql.hive.test.TestHive + +class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { + val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") + val METASTORE_PATH = TestUtils.getMetastorePath("cli") + + override def beforeAll() { + val pb = new ProcessBuilder( + "../../bin/spark-sql", + "--master", + "local", + "--hiveconf", + s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", + "--hiveconf", + "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + process = pb.start() + outputWriter = new PrintWriter(process.getOutputStream, true) + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "spark-sql>") + } + + override def afterAll() { + process.destroy() + process.waitFor() + } + + test("simple commands") { + val dataFilePath = getDataFile("data/files/small_kv.txt") + executeQuery("create table hive_test1(key int, val string);") + executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") + executeQuery("cache table hive_test1", "Time taken") + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala new file mode 100644 index 0000000000000..59f4952b78bc6 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent._ + +import java.io.{BufferedReader, InputStreamReader} +import java.sql.{Connection, DriverManager, Statement} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.util.getTempFilePath + +/** + * Test for the HiveThriftServer2 using JDBC. + */ +class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { + + val WAREHOUSE_PATH = getTempFilePath("warehouse") + val METASTORE_PATH = getTempFilePath("metastore") + + val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" + val TABLE = "test" + // use a different port, than the hive standard 10000, + // for tests to avoid issues with the port being taken on some machines + val PORT = "10000" + + // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. + val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean + + Class.forName(DRIVER_NAME) + + override def beforeAll() { launchServer() } + + override def afterAll() { stopServer() } + + private def launchServer(args: Seq[String] = Seq.empty) { + // Forking a new process to start the Hive Thrift server. The reason to do this is it is + // hard to clean up Hive resources entirely, so we just start a new process and kill + // that process for cleanup. + val defaultArgs = Seq( + "../../sbin/start-thriftserver.sh", + "--master local", + "--hiveconf", + "hive.root.logger=INFO,console", + "--hiveconf", + s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", + "--hiveconf", + s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH") + val pb = new ProcessBuilder(defaultArgs ++ args) + process = pb.start() + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "ThriftBinaryCLIService listening on") + + // Spawn a thread to read the output from the forked process. + // Note that this is necessary since in some configurations, log4j could be blocked + // if its output to stderr are not read, and eventually blocking the entire test suite. + future { + while (true) { + val stdout = readFrom(inputReader) + val stderr = readFrom(errorReader) + if (VERBOSE && stdout.length > 0) { + println(stdout) + } + if (VERBOSE && stderr.length > 0) { + println(stderr) + } + Thread.sleep(50) + } + } + } + + private def stopServer() { + process.destroy() + process.waitFor() + } + + test("test query execution against a Hive Thrift server") { + Thread.sleep(5 * 1000) + val dataFilePath = getDataFile("data/files/small_kv.txt") + val stmt = createStatement() + stmt.execute("DROP TABLE IF EXISTS test") + stmt.execute("DROP TABLE IF EXISTS test_cached") + stmt.execute("CREATE TABLE test(key int, val string)") + stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") + stmt.execute("CREATE TABLE test_cached as select * from test limit 4") + stmt.execute("CACHE TABLE test_cached") + + var rs = stmt.executeQuery("select count(*) from test") + rs.next() + assert(rs.getInt(1) === 5) + + rs = stmt.executeQuery("select count(*) from test_cached") + rs.next() + assert(rs.getInt(1) === 4) + + stmt.close() + } + + def getConnection: Connection = { + val connectURI = s"jdbc:hive2://localhost:$PORT/" + DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") + } + + def createStatement(): Statement = getConnection.createStatement() +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala new file mode 100644 index 0000000000000..bb2242618fbef --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, PrintWriter} +import java.text.SimpleDateFormat +import java.util.Date + +import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException + +object TestUtils { + val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") + + def getWarehousePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + + timestamp.format(new Date) + } + + def getMetastorePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + + timestamp.format(new Date) + } + + // Dummy function for initialize the log4j properties. + def init() { } + + // initialize log4j + try { + LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => // Ignore the error. + } +} + +trait TestUtils { + var process : Process = null + var outputWriter : PrintWriter = null + var inputReader : BufferedReader = null + var errorReader : BufferedReader = null + + def executeQuery( + cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { + println("Executing: " + cmd + ", expecting output: " + outputMessage) + outputWriter.write(cmd + "\n") + outputWriter.flush() + waitForQuery(timeout, outputMessage) + } + + protected def waitForQuery(timeout: Long, message: String): String = { + if (waitForOutput(errorReader, message, timeout)) { + Thread.sleep(500) + readOutput() + } else { + assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) + null + } + } + + // Wait for the specified str to appear in the output. + protected def waitForOutput( + reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { + val startTime = System.currentTimeMillis + var out = "" + while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { + out += readFrom(reader) + } + out.contains(str) + } + + // Read stdout output and filter out garbage collection messages. + protected def readOutput(): String = { + val output = readFrom(inputReader) + // Remove GC Messages + val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) + .mkString("\n") + filteredOutput + } + + protected def readFrom(reader: BufferedReader): String = { + var out = "" + var c = 0 + while (reader.ready) { + c = reader.read() + out += c.asInstanceOf[Char] + } + out + } + + protected def getDataFile(name: String) = { + Thread.currentThread().getContextClassLoader.getResource(name) + } +} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1699ffe06ce15..93d00f7c37c9b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -32,7 +32,7 @@ Spark Project Hive http://spark.apache.org/ - hive + hive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 201c85f3d501e..84d43eaeea51d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -255,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DecimalType, TimestampType, BinaryType) - protected def toHiveString(a: (Any, DataType)): String = a match { + protected[sql] def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 6f36a4f8cb905..8489f2a34e63c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -416,10 +416,10 @@ class HiveQuerySuite extends HiveComparisonTest { hql(s"set $testKey=$testVal") assert(get(testKey, testVal + "_") == testVal) - hql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - hql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + hql("set some.property=20") + assert(get("some.property", "0") == "20") + hql("set some.property = 40") + assert(get("some.property", "0") == "40") hql(s"set $testKey=$testVal") assert(get(testKey, "0") == testVal) @@ -433,63 +433,61 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - def collectResults(rdd: SchemaRDD): Set[(String, String)] = - rdd.collect().map { case Row(key: String, value: String) => key -> value }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. assert(hql("SET").collect().size == 0) - assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey=$testVal")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(hql("SET")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } hql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(hql("SET")) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + hql(s"SET").collect().map(_.getString(0)) } // "set key" - assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Set(nonexistentKey -> "")) { - collectResults(hql(s"SET $nonexistentKey")) + assertResult(Array(s"$nonexistentKey=")) { + hql(s"SET $nonexistentKey").collect().map(_.getString(0)) } // Assert that sql() should have the same effects as hql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey=$testVal")) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(sql("SET")) + assertResult(Array(s"$testKey=$testVal")) { + sql("SET").collect().map(_.getString(0)) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(sql("SET")) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + sql("SET").collect().map(_.getString(0)) } - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey")) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Set(nonexistentKey -> "")) { - collectResults(sql(s"SET $nonexistentKey")) + assertResult(Array(s"$nonexistentKey=")) { + sql(s"SET $nonexistentKey").collect().map(_.getString(0)) } clear() diff --git a/streaming/pom.xml b/streaming/pom.xml index f60697ce745b7..b99f306b8f2cc 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming_2.10 - streaming + streaming jar Spark Project Streaming diff --git a/tools/pom.xml b/tools/pom.xml index c0ee8faa7a615..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,7 +27,7 @@ org.apache.spark spark-tools_2.10 - tools + tools jar Spark Project Tools diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 5b13a1f002d6e..51744ece0412d 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-alpha + yarn-alpha org.apache.spark diff --git a/yarn/pom.xml b/yarn/pom.xml index efb473aa1b261..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -29,7 +29,7 @@ pom Spark Project YARN Parent POM - yarn + yarn diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index ceaf9f9d71001..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-stable + yarn-stable org.apache.spark From a19d8c89d90eea7eb8295378cec3d27444e0336d Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 25 Jul 2014 13:00:13 -0700 Subject: [PATCH 0075/1492] [SPARK-2682] Javadoc generated from Scala source code is not in javadoc's index Add genjavadocSettings back to SparkBuild. It requires #1585 . https://issues.apache.org/jira/browse/SPARK-2682 Author: Yin Huai Closes #1584 from yhuai/SPARK-2682 and squashes the following commits: 2e89461 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2682 54e3b66 [Yin Huai] Add genjavadocSettings back. --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 86d47734e77bb..1629bc2cba8ba 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConversions._ import sbt._ import sbt.Classpaths.publishTask import sbt.Keys._ +import sbtunidoc.Plugin.genjavadocSettings import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} import net.virtualvoid.sbt.graph.Plugin.graphSettings @@ -107,7 +108,7 @@ object SparkBuild extends PomBuild { 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 ( + lazy val sharedSettings = graphSettings ++ ScalaStyleSettings ++ genjavadocSettings ++ Seq ( javaHome := Properties.envOrNone("JAVA_HOME").map(file), incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, From ab3c6a455c0b50e3fcfea3bbb3b9035aba8f06e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 25 Jul 2014 13:59:45 -0700 Subject: [PATCH 0076/1492] [SQL]Update HiveMetastoreCatalog.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit I think it's better to defined hiveQlTable as a val Author: baishuo(白硕) Closes #1569 from baishuo/patch-1 and squashes the following commits: dc2f895 [baishuo(白硕)] Update HiveMetastoreCatalog.scala a7b32a2 [baishuo(白硕)] Update HiveMetastoreCatalog.scala --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 8db60d32767b5..156b090712df2 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 @@ -258,7 +258,7 @@ private[hive] case class MetastoreRelation // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException // which indicates the SerDe we used is not Serializable. - def hiveQlTable = new Table(table) + @transient lazy val hiveQlTable = new Table(table) def hiveQlPartitions = partitions.map { p => new Partition(hiveQlTable, p) From 47b6b38ca8d9c5de794183cc91cbf6559ef27390 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 25 Jul 2014 14:34:38 -0700 Subject: [PATCH 0077/1492] [SPARK-2125] Add sort flag and move sort into shuffle implementations This patch adds a sort flag into ShuffleDependecy and moves sort into hash shuffle implementation. Moving sort into shuffle implementation can give space for other shuffle implementations (like sort-based shuffle) to better optimize sort through shuffle. Author: jerryshao Closes #1210 from jerryshao/SPARK-2125 and squashes the following commits: 2feaf7b [jerryshao] revert MimaExcludes ceddf75 [jerryshao] add MimaExeclude f674ff4 [jerryshao] Add missing Scope restriction b9fe0dd [jerryshao] Fix some style issues according to comments ef6b729 [jerryshao] Change sort flag into Option 3f6eeed [jerryshao] Fix issues related to unit test 2f552a5 [jerryshao] Minor changes about naming and order c92a281 [jerryshao] Move sort into shuffle implementations --- .../scala/org/apache/spark/Dependency.scala | 4 +++- .../apache/spark/rdd/OrderedRDDFunctions.scala | 17 ++++++++--------- .../org/apache/spark/rdd/ShuffledRDD.scala | 12 +++++++++++- .../spark/shuffle/hash/HashShuffleReader.scala | 14 +++++++++++++- 4 files changed, 35 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 09a60571238ea..f010c03223ef4 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.SortOrder.SortOrder import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleHandle @@ -62,7 +63,8 @@ class ShuffleDependency[K, V, C]( val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, - val mapSideCombine: Boolean = false) + val mapSideCombine: Boolean = false, + val sortOrder: Option[SortOrder] = None) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index f1f4b4324edfd..afd7075f686b9 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -57,14 +57,13 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, */ def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = { val part = new RangePartitioner(numPartitions, self, ascending) - val shuffled = new ShuffledRDD[K, V, V, P](self, part).setKeyOrdering(ordering) - shuffled.mapPartitions(iter => { - val buf = iter.toArray - if (ascending) { - buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator - } else { - buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator - } - }, preservesPartitioning = true) + new ShuffledRDD[K, V, V, P](self, part) + .setKeyOrdering(ordering) + .setSortOrder(if (ascending) SortOrder.ASCENDING else SortOrder.DESCENDING) } } + +private[spark] object SortOrder extends Enumeration { + type SortOrder = Value + val ASCENDING, DESCENDING = Value +} diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index bf02f68d0d3d3..da4a8c3dc22b1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -21,6 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.SortOrder.SortOrder import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -51,6 +52,8 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( private var mapSideCombine: Boolean = false + private var sortOrder: Option[SortOrder] = None + /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C, P] = { this.serializer = Option(serializer) @@ -75,8 +78,15 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( this } + /** Set sort order for RDD's sorting. */ + def setSortOrder(sortOrder: SortOrder): ShuffledRDD[K, V, C, P] = { + this.sortOrder = Option(sortOrder) + this + } + override def getDependencies: Seq[Dependency[_]] = { - List(new ShuffleDependency(prev, part, serializer, keyOrdering, aggregator, mapSideCombine)) + List(new ShuffleDependency(prev, part, serializer, + keyOrdering, aggregator, mapSideCombine, sortOrder)) } override val partitioner = Some(part) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index d45258c0a492b..76cdb8f4f8e8a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -18,6 +18,7 @@ package org.apache.spark.shuffle.hash import org.apache.spark.{InterruptibleIterator, TaskContext} +import org.apache.spark.rdd.SortOrder import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} @@ -38,7 +39,7 @@ class HashShuffleReader[K, C]( val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, Serializer.getSerializer(dep.serializer)) - if (dep.aggregator.isDefined) { + val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { new InterruptibleIterator(context, dep.aggregator.get.combineCombinersByKey(iter, context)) } else { @@ -49,6 +50,17 @@ class HashShuffleReader[K, C]( } else { iter } + + val sortedIter = for (sortOrder <- dep.sortOrder; ordering <- dep.keyOrdering) yield { + val buf = aggregatedIter.toArray + if (sortOrder == SortOrder.ASCENDING) { + buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator + } else { + buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator + } + } + + sortedIter.getOrElse(aggregatedIter) } /** Close this reader */ From 37ad3b724590dcf42bcdbfaf91b7a11914501945 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 25 Jul 2014 15:14:13 -0700 Subject: [PATCH 0078/1492] [SPARK-1726] [SPARK-2567] Eliminate zombie stages in UI. Due to problems with when we update runningStages (in DAGScheduler.scala) and how we decide to send a SparkListenerStageCompleted message to SparkListeners, sometimes stages can be shown as "running" in the UI forever (even after they have failed). This issue can manifest when stages are resubmitted with 0 tasks, or when the DAGScheduler catches non-serializable tasks. The problem also resulted in a (small) memory leak in the DAGScheduler, where stages can stay in runningStages forever. This commit fixes that problem and adds a unit test. Thanks tsudukim for helping to look into this issue! cc markhamstra rxin Author: Kay Ousterhout Closes #1566 from kayousterhout/dag_fix and squashes the following commits: 217d74b [Kay Ousterhout] [SPARK-1726] [SPARK-2567] Eliminate zombie stages in UI. --- .../apache/spark/scheduler/DAGScheduler.scala | 12 +- .../spark/scheduler/DAGSchedulerSuite.scala | 129 ++++++++++-------- 2 files changed, 76 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index acb4c4946eded..00b8af27a7b39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -710,7 +710,6 @@ class DAGScheduler( if (missing == Nil) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) - runningStages += stage } else { for (parent <- missing) { submitStage(parent) @@ -753,11 +752,14 @@ class DAGScheduler( null } - // must be run listener before possible NotSerializableException - // should be "StageSubmitted" first and then "JobEnded" - listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) - if (tasks.size > 0) { + runningStages += stage + // SparkListenerStageSubmitted should be posted before testing whether tasks are + // serializable. If tasks are not serializable, a SparkListenerStageCompleted event + // will be posted, which should always come after a corresponding SparkListenerStageSubmitted + // event. + listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) + // Preemptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception // down the road, where we have several different implementations for local scheduler and diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9f498d579a095..44dd1e092ad67 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -37,6 +37,29 @@ class BuggyDAGEventProcessActor extends Actor { } } +/** + * An RDD for passing to DAGScheduler. These RDDs will use the dependencies and + * preferredLocations (if any) that are passed to them. They are deliberately not executable + * so we can test that DAGScheduler does not try to execute RDDs locally. + */ +class MyRDD( + sc: SparkContext, + numPartitions: Int, + dependencies: List[Dependency[_]], + locations: Seq[Seq[String]] = Nil) extends RDD[(Int, Int)](sc, dependencies) with Serializable { + override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = + throw new RuntimeException("should not be reached") + override def getPartitions = (0 until numPartitions).map(i => new Partition { + override def index = i + }).toArray + override def getPreferredLocations(split: Partition): Seq[String] = + if (locations.isDefinedAt(split.index)) + locations(split.index) + else + Nil + override def toString: String = "DAGSchedulerSuiteRDD " + id +} + class DAGSchedulerSuiteDummyException extends Exception class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike @@ -148,34 +171,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F * Type of RDD we use for testing. Note that we should never call the real RDD compute methods. * This is a pair RDD type so it can always be used in ShuffleDependencies. */ - type MyRDD = RDD[(Int, Int)] - - /** - * Create an RDD for passing to DAGScheduler. These RDDs will use the dependencies and - * preferredLocations (if any) that are passed to them. They are deliberately not executable - * so we can test that DAGScheduler does not try to execute RDDs locally. - */ - private def makeRdd( - numPartitions: Int, - dependencies: List[Dependency[_]], - locations: Seq[Seq[String]] = Nil - ): MyRDD = { - val maxPartition = numPartitions - 1 - val newRDD = new MyRDD(sc, dependencies) { - override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = - throw new RuntimeException("should not be reached") - override def getPartitions = (0 to maxPartition).map(i => new Partition { - override def index = i - }).toArray - override def getPreferredLocations(split: Partition): Seq[String] = - if (locations.isDefinedAt(split.index)) - locations(split.index) - else - Nil - override def toString: String = "DAGSchedulerSuiteRDD " + id - } - newRDD - } + type PairOfIntsRDD = RDD[(Int, Int)] /** * Process the supplied event as if it were the top of the DAGScheduler event queue, expecting @@ -234,19 +230,19 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(makeRdd(0, Nil), Array(), listener = fakeListener) + submit(new MyRDD(sc, 0, Nil), Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { - submit(makeRdd(1, Nil), Array(0)) + submit(new MyRDD(sc, 1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } test("local job") { - val rdd = new MyRDD(sc, Nil) { + val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = Array(42 -> 0).iterator override def getPartitions = Array( new Partition { override def index = 0 } ) @@ -260,7 +256,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("local job oom") { - val rdd = new MyRDD(sc, Nil) { + val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new java.lang.OutOfMemoryError("test local job oom") override def getPartitions = Array( new Partition { override def index = 0 } ) @@ -274,8 +270,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial job w/ dependency") { - val baseRdd = makeRdd(1, Nil) - val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + val baseRdd = new MyRDD(sc, 1, Nil) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -283,8 +279,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("cache location preferences w/ dependency") { - val baseRdd = makeRdd(1, Nil) - val finalRdd = makeRdd(1, List(new OneToOneDependency(baseRdd))) + val baseRdd = new MyRDD(sc, 1, Nil) + val finalRdd = new MyRDD(sc, 1, List(new OneToOneDependency(baseRdd))) cacheLocations(baseRdd.id -> 0) = Seq(makeBlockManagerId("hostA"), makeBlockManagerId("hostB")) submit(finalRdd, Array(0)) @@ -295,8 +291,22 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("unserializable task") { + val unserializableRdd = new MyRDD(sc, 1, Nil) { + class UnserializableClass + val unserializable = new UnserializableClass + } + submit(unserializableRdd, Array(0)) + assert(failure.getMessage.startsWith( + "Job aborted due to stage failure: Task not serializable:")) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) + assertDataStructuresEmpty + } + test("trivial job failure") { - submit(makeRdd(1, Nil), Array(0)) + submit(new MyRDD(sc, 1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) @@ -306,7 +316,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("trivial job cancellation") { - val rdd = makeRdd(1, Nil) + val rdd = new MyRDD(sc, 1, Nil) val jobId = submit(rdd, Array(0)) cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled ") @@ -347,8 +357,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( Props(classOf[DAGSchedulerEventProcessActor], noKillScheduler))(system) - val rdd = makeRdd(1, Nil) - val jobId = submit(rdd, Array(0)) + val jobId = submit(new MyRDD(sc, 1, Nil), Array(0)) cancel(jobId) // Because the job wasn't actually cancelled, we shouldn't have received a failure message. assert(failure === null) @@ -364,10 +373,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(1, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) submit(reduceRdd, Array(0)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), @@ -380,10 +389,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle with fetch failure") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) complete(taskSets(0), Seq( (Success, makeMapStatus("hostA", 1)), @@ -406,10 +415,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("ignore late map task completions") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch @@ -435,9 +444,9 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run shuffle with map stage failure") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) - val reduceRdd = makeRdd(2, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) submit(reduceRdd, Array(0, 1)) // Fail the map stage. This should cause the entire job to fail. @@ -472,13 +481,13 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F * without shuffleMapRdd1. */ test("failure of stage used by two jobs") { - val shuffleMapRdd1 = makeRdd(2, Nil) + val shuffleMapRdd1 = new MyRDD(sc, 2, Nil) val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, null) - val shuffleMapRdd2 = makeRdd(2, Nil) + val shuffleMapRdd2 = new MyRDD(sc, 2, Nil) val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, null) - val reduceRdd1 = makeRdd(2, List(shuffleDep1)) - val reduceRdd2 = makeRdd(2, List(shuffleDep1, shuffleDep2)) + val reduceRdd1 = new MyRDD(sc, 2, List(shuffleDep1)) + val reduceRdd2 = new MyRDD(sc, 2, List(shuffleDep1, shuffleDep2)) // We need to make our own listeners for this test, since by default submit uses the same // listener for all jobs, and here we want to capture the failure for each job separately. @@ -511,10 +520,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("run trivial shuffle with out-of-band failure and retry") { - val shuffleMapRdd = makeRdd(2, Nil) + val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) val shuffleId = shuffleDep.shuffleId - val reduceRdd = makeRdd(1, List(shuffleDep)) + val reduceRdd = new MyRDD(sc, 1, List(shuffleDep)) submit(reduceRdd, Array(0)) // blockManagerMaster.removeExecutor("exec-hostA") // pretend we were told hostA went away @@ -534,11 +543,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("recursive shuffle failures") { - val shuffleOneRdd = makeRdd(2, Nil) + val shuffleOneRdd = new MyRDD(sc, 2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) // have the first stage complete normally complete(taskSets(0), Seq( @@ -563,11 +572,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } test("cached post-shuffle") { - val shuffleOneRdd = makeRdd(2, Nil) + val shuffleOneRdd = new MyRDD(sc, 2, Nil) val shuffleDepOne = new ShuffleDependency(shuffleOneRdd, null) - val shuffleTwoRdd = makeRdd(2, List(shuffleDepOne)) + val shuffleTwoRdd = new MyRDD(sc, 2, List(shuffleDepOne)) val shuffleDepTwo = new ShuffleDependency(shuffleTwoRdd, null) - val finalRdd = makeRdd(1, List(shuffleDepTwo)) + val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo)) submit(finalRdd, Array(0)) cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) From afd757a241f41d7f8c458ef8f1f9ce8ed12986e5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 25 Jul 2014 15:36:57 -0700 Subject: [PATCH 0079/1492] Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server" This reverts commit 06dc0d2c6b69c5d59b4d194ced2ac85bfe2e05e2. #1399 is making Jenkins fail. We should investigate and put this back after its passing tests. Author: Michael Armbrust Closes #1594 from marmbrus/revertJDBC and squashes the following commits: 59748da [Michael Armbrust] Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server" --- .gitignore | 1 - assembly/pom.xml | 10 - bagel/pom.xml | 2 +- bin/beeline | 45 --- bin/compute-classpath.sh | 1 - bin/spark-shell | 4 +- bin/spark-shell.cmd | 2 +- bin/spark-sql | 36 -- core/pom.xml | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 14 +- .../spark/deploy/SparkSubmitArguments.scala | 5 +- dev/create-release/create-release.sh | 10 +- dev/run-tests | 2 +- dev/scalastyle | 2 +- docs/sql-programming-guide.md | 200 +--------- examples/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 7 +- project/SparkBuild.scala | 14 +- sbin/start-thriftserver.sh | 36 -- sql/catalyst/pom.xml | 2 +- .../sql/catalyst/plans/logical/commands.scala | 3 +- sql/core/pom.xml | 2 +- .../scala/org/apache/spark/sql/SQLConf.scala | 20 +- .../apache/spark/sql/execution/commands.scala | 42 +-- .../org/apache/spark/sql/SQLConfSuite.scala | 13 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- sql/hive-thriftserver/pom.xml | 82 ----- .../hive/thriftserver/HiveThriftServer2.scala | 97 ----- .../hive/thriftserver/ReflectionUtils.scala | 58 --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 344 ------------------ .../thriftserver/SparkSQLCLIService.scala | 74 ---- .../hive/thriftserver/SparkSQLDriver.scala | 93 ----- .../sql/hive/thriftserver/SparkSQLEnv.scala | 58 --- .../thriftserver/SparkSQLSessionManager.scala | 49 --- .../server/SparkSQLOperationManager.scala | 151 -------- .../test/resources/data/files/small_kv.txt | 5 - .../sql/hive/thriftserver/CliSuite.scala | 59 --- .../thriftserver/HiveThriftServer2Suite.scala | 125 ------- .../sql/hive/thriftserver/TestUtils.scala | 108 ------ sql/hive/pom.xml | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 50 +-- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 54 files changed, 96 insertions(+), 1772 deletions(-) delete mode 100755 bin/beeline delete mode 100755 bin/spark-sql delete mode 100755 sbin/start-thriftserver.sh delete mode 100644 sql/hive-thriftserver/pom.xml delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala delete mode 100755 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala delete mode 100644 sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala diff --git a/.gitignore b/.gitignore index 5b56a67c883e6..061c8946d23c1 100644 --- a/.gitignore +++ b/.gitignore @@ -57,4 +57,3 @@ metastore_db/ metastore/ warehouse/ TempStatsStore/ -sql/hive-thriftserver/test_warehouses diff --git a/assembly/pom.xml b/assembly/pom.xml index 703f15925bc44..567a8dd2a0d94 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -165,16 +165,6 @@ - - hive-thriftserver - - - org.apache.spark - spark-hive-thriftserver_${scala.binary.version} - ${project.version} - - - spark-ganglia-lgpl diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..90c4b095bb611 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-bagel_2.10 - bagel + bagel jar Spark Project Bagel diff --git a/bin/beeline b/bin/beeline deleted file mode 100755 index 09fe366c609fa..0000000000000 --- a/bin/beeline +++ /dev/null @@ -1,45 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -# Find the java binary -if [ -n "${JAVA_HOME}" ]; then - RUNNER="${JAVA_HOME}/bin/java" -else - if [ `command -v java` ]; then - RUNNER="java" - else - echo "JAVA_HOME is not set" >&2 - exit 1 - fi -fi - -# Compute classpath using external script -classpath_output=$($FWDIR/bin/compute-classpath.sh) -if [[ "$?" != "0" ]]; then - echo "$classpath_output" - exit 1 -else - CLASSPATH=$classpath_output -fi - -CLASS="org.apache.hive.beeline.BeeLine" -exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 16b794a1592e8..e81e8c060cb98 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -52,7 +52,6 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" fi diff --git a/bin/spark-shell b/bin/spark-shell index 756c8179d12b6..850e9507ec38f 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index b56d69801171c..4b9708a8c03f3 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell --class org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main diff --git a/bin/spark-sql b/bin/spark-sql deleted file mode 100755 index bba7f897b19bc..0000000000000 --- a/bin/spark-sql +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# -# Shell script for starting the Spark SQL CLI - -# Enter posix mode for bash -set -o posix - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/spark-sql [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 -fi - -CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/core/pom.xml b/core/pom.xml index a24743495b0e1..1054cec4d77bb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-core_2.10 - core + core jar Spark Project Core diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c9cec33ebaa66..3b5642b6caa36 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -46,10 +46,6 @@ object SparkSubmit { private val CLUSTER = 2 private val ALL_DEPLOY_MODES = CLIENT | CLUSTER - // A special jar name that indicates the class being run is inside of Spark itself, and therefore - // no user jar is needed. - private val SPARK_INTERNAL = "spark-internal" - // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" @@ -261,9 +257,7 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (clusterManager == YARN && deployMode == CLUSTER) { childMainClass = "org.apache.spark.deploy.yarn.Client" - if (args.primaryResource != SPARK_INTERNAL) { - childArgs += ("--jar", args.primaryResource) - } + childArgs += ("--jar", args.primaryResource) childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } @@ -338,7 +332,7 @@ object SparkSubmit { * Return whether the given primary resource represents a user jar. */ private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) + !isShell(primaryResource) && !isPython(primaryResource) } /** @@ -355,10 +349,6 @@ object SparkSubmit { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } - private[spark] def isInternal(primaryResource: String): Boolean = { - primaryResource == SPARK_INTERNAL - } - /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 01d0ae541a66b..3ab67a43a3b55 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -204,9 +204,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { - var inSparkOpts = true - // Delineates parsing of Spark options from parsing of user options. + var inSparkOpts = true parse(opts) def parse(opts: Seq[String]): Unit = opts match { @@ -319,7 +318,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit(errMessage) case v => primaryResource = - if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) { + if (!SparkSubmit.isShell(v)) { Utils.resolveURI(v).toString } else { v diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 33de24d1ae6d7..38830103d1e8d 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -53,7 +53,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare @@ -61,7 +61,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. @@ -111,10 +111,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" +make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" make_binary_release "hadoop2" \ - "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" # Copy data echo "Copying release tarballs" diff --git a/dev/run-tests b/dev/run-tests index 98ec969dc1b37..51e4def0f835a 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -65,7 +65,7 @@ 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" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive -Phive-thriftserver" sbt/sbt clean package \ + 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 | \ diff --git a/dev/scalastyle b/dev/scalastyle index d9f2b91a3a091..a02d06912f238 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 36d642f2923b2..38728534a46e0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -136,7 +136,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD // Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) @@ -548,6 +548,7 @@ results = hiveContext.hql("FROM src SELECT key, value").collect() + # Writing Language-Integrated Relational Queries **Language-Integrated queries are currently only supported in Scala.** @@ -572,199 +573,4 @@ prefixed with a tick (`'`). Implicit conversions turn these symbols into expres evaluated by the SQL execution engine. A full list of the functions supported can be found in the [ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). - - -## Running the Thrift JDBC server - -The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] -(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test -the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive -you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` -for maven). - -To start the JDBC server, run the following in the Spark directory: - - ./sbin/start-thriftserver.sh - -The default port the server listens on is 10000. You may run -`./sbin/start-thriftserver.sh --help` for a complete list of all available -options. Now you can use beeline to test the Thrift JDBC server: - - ./bin/beeline - -Connect to the JDBC server in beeline with: - - beeline> !connect jdbc:hive2://localhost:10000 - -Beeline will ask you for a username and password. In non-secure mode, simply enter the username on -your machine and a blank password. For secure mode, please follow the instructions given in the -[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) - -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. - -You may also use the beeline script comes with Hive. - -### Migration Guide for Shark Users - -#### Reducer number - -In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark -SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value -is 200. Users may customize this property via `SET`: - -``` -SET spark.sql.shuffle.partitions=10; -SELECT page, count(*) c FROM logs_last_month_cached -GROUP BY page ORDER BY c DESC LIMIT 10; -``` - -You may also put this property in `hive-site.xml` to override the default value. - -For now, the `mapred.reduce.tasks` property is still recognized, and is converted to -`spark.sql.shuffle.partitions` automatically. - -#### Caching - -The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no -longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to -let user control table caching explicitly: - -``` -CACHE TABLE logs_last_month; -UNCACHE TABLE logs_last_month; -``` - -**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", -but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be -cached, you may simply count the table immediately after executing `CACHE TABLE`: - -``` -CACHE TABLE logs_last_month; -SELECT COUNT(1) FROM logs_last_month; -``` - -Several caching related features are not supported yet: - -* User defined partition level cache eviction policy -* RDD reloading -* In-memory cache write through policy - -### Compatibility with Apache Hive - -#### Deploying in Exising Hive Warehouses - -Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -#### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathemtatical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDe's) -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -#### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - -**Esoteric Hive Features** - -* Tables with partitions using different input formats: In Spark SQL, all table partitions need to - have the same input format. -* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions - (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. -* `UNIONTYPE` -* Unique join -* Single query multi insert -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -not necessary due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block level bitmap indexes and virtual columns (used to build indexes) -* Automatically convert a join to map join: For joining a large table with multiple small tables, - Hive automatically converts the join into a map join. We are adding this auto conversion in the - next release. -* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you - need to control the degree of parallelism post-shuffle using "SET - spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the - next release. -* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -## Running the Spark SQL CLI - -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. - -To start the Spark SQL CLI, run the following in the Spark directory: - - ./bin/spark-sql - -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available -options. + \ No newline at end of file diff --git a/examples/pom.xml b/examples/pom.xml index c4ed0f5a6a02b..bd1c387c2eb91 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-examples_2.10 - examples + examples jar Spark Project Examples diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 874b8a7959bb6..61a6aff543aed 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-flume_2.10 - streaming-flume + streaming-flume jar Spark Project External Flume diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 25a5c0a4d7d77..4762c50685a93 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-kafka_2.10 - streaming-kafka + streaming-kafka jar Spark Project External Kafka diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index f31ed655f6779..32c530e600ce0 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-mqtt_2.10 - streaming-mqtt + streaming-mqtt jar Spark Project External MQTT diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 56bb24c2a072e..637adb0f00da0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-twitter_2.10 - streaming-twitter + streaming-twitter jar Spark Project External Twitter diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 54b0242c54e78..e4d758a04a4cd 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-zeromq_2.10 - streaming-zeromq + streaming-zeromq jar Spark Project External ZeroMQ diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..7e3bcf29dcfbc 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-graphx_2.10 - graphx + graphx jar Spark Project GraphX diff --git a/mllib/pom.xml b/mllib/pom.xml index f27cf520dc9fa..92b07e2357db1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-mllib_2.10 - mllib + mllib jar Spark Project ML Library diff --git a/pom.xml b/pom.xml index 3e9d388180d8e..4e2d64a833640 100644 --- a/pom.xml +++ b/pom.xml @@ -95,7 +95,6 @@ sql/catalyst sql/core sql/hive - sql/hive-thriftserver repl assembly external/twitter @@ -253,9 +252,9 @@ 3.3.2 - commons-codec - commons-codec - 1.5 + commons-codec + commons-codec + 1.5 com.google.code.findbugs diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1629bc2cba8ba..62576f84dd031 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,11 +30,11 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, - streaming, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = - Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "spark", "sql", "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", - "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) + val allProjects@Seq(bagel, catalyst, core, graphx, hive, mllib, repl, spark, sql, streaming, + streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = + Seq("bagel", "catalyst", "core", "graphx", "hive", "mllib", "repl", "spark", "sql", + "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", + "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") @@ -100,7 +100,7 @@ object SparkBuild extends PomBuild { Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) - case _ => + case _ => } override val userPropertiesMap = System.getProperties.toMap @@ -158,7 +158,7 @@ object SparkBuild extends PomBuild { /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl).contains(x)). + allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl).exists(x => x == y)). foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh deleted file mode 100755 index 8398e6f19b511..0000000000000 --- a/sbin/start-thriftserver.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# -# Shell script for starting the Spark SQL Thrift server - -# Enter posix mode for bash -set -o posix - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/start-thriftserver [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 -fi - -CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 531bfddbf237b..6decde3fcd62d 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -32,7 +32,7 @@ Spark Project Catalyst http://spark.apache.org/ - catalyst + catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index a357c6ffb8977..1d5f033f0d274 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -43,7 +43,8 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(1, AttributeReference("", StringType, nullable = false)())) + BoundReference(0, AttributeReference("key", StringType, nullable = false)()), + BoundReference(1, AttributeReference("value", StringType, nullable = false)())) } /** diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3a038a2db6173..c309c43804d97 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -32,7 +32,7 @@ Spark Project SQL http://spark.apache.org/ - sql + sql diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 41920c00b5a2c..2b787e14f3f15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -30,13 +30,12 @@ import scala.collection.JavaConverters._ * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { - import SQLConf._ /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? /** Number of partitions to use for shuffle operators. */ - private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt + private[spark] def numShufflePartitions: Int = get("spark.sql.shuffle.partitions", "200").toInt /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to @@ -44,10 +43,11 @@ trait SQLConf { * effectively disables auto conversion. * Hive setting: hive.auto.convert.join.noconditionaltask.size. */ - private[spark] def autoConvertJoinSize: Int = get(AUTO_CONVERT_JOIN_SIZE, "10000").toInt + private[spark] def autoConvertJoinSize: Int = + get("spark.sql.auto.convert.join.size", "10000").toInt /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get(JOIN_BROADCAST_TABLES, "") + private[spark] def joinBroadcastTables: String = get("spark.sql.join.broadcastTables", "") /** ********************** SQLConf functionality methods ************ */ @@ -61,7 +61,7 @@ trait SQLConf { def set(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for $key") + require(value != null, s"value cannot be null for ${key}") settings.put(key, value) } @@ -90,13 +90,3 @@ trait SQLConf { } } - -object SQLConf { - val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" - val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" - val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" - - object Deprecated { - val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 9293239131d52..98d2f89c8ae71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.execution -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLConf, SQLContext} +import org.apache.spark.sql.{Row, SQLContext} trait Command { /** @@ -45,53 +44,28 @@ trait Command { case class SetCommand( key: Option[String], value: Option[String], output: Seq[Attribute])( @transient context: SQLContext) - extends LeafNode with Command with Logging { + extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => - if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { - logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + - s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - context.set(SQLConf.SHUFFLE_PARTITIONS, v) - Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") - } else { - context.set(k, v) - Array(s"$k=$v") - } + context.set(k, v) + Array(k -> v) // Query the value bound to key k. case (Some(k), _) => - // TODO (lian) This is just a workaround to make the Simba ODBC driver work. - // Should remove this once we get the ODBC driver updated. - if (k == "-v") { - val hiveJars = Seq( - "hive-exec-0.12.0.jar", - "hive-service-0.12.0.jar", - "hive-common-0.12.0.jar", - "hive-hwi-0.12.0.jar", - "hive-0.12.0.jar").mkString(":") - - Array( - "system:java.class.path=" + hiveJars, - "system:sun.java.command=shark.SharkServer2") - } - else { - Array(s"$k=${context.getOption(k).getOrElse("")}") - } + Array(k -> context.getOption(k).getOrElse("")) // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => - context.getAll.map { case (k, v) => - s"$k=$v" - } + context.getAll case _ => throw new IllegalArgumentException() } def execute(): RDD[Row] = { - val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } + val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) } context.sparkContext.parallelize(rows, 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 1a58d73d9e7f4..08293f7f0ca30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -54,10 +54,10 @@ class SQLConfSuite extends QueryTest { assert(get(testKey, testVal + "_") == testVal) assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - sql("set some.property=20") - assert(get("some.property", "0") == "20") - sql("set some.property = 40") - assert(get("some.property", "0") == "40") + sql("set mapred.reduce.tasks=20") + assert(get("mapred.reduce.tasks", "0") == "20") + sql("set mapred.reduce.tasks = 40") + assert(get("mapred.reduce.tasks", "0") == "40") val key = "spark.sql.key" val vs = "val0,val_1,val2.3,my_table" @@ -70,9 +70,4 @@ class SQLConfSuite extends QueryTest { clear() } - test("deprecated property") { - clear() - sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") - assert(get(SQLConf.SHUFFLE_PARTITIONS) == "10") - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index de9e8aa4f62ed..6736189c96d4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -424,25 +424,25 @@ class SQLQuerySuite extends QueryTest { sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - Seq(Seq(s"$testKey=$testVal")) + Seq(Seq(testKey, testVal)) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), Seq( - Seq(s"$testKey=$testVal"), - Seq(s"${testKey + testKey}=${testVal + testVal}")) + Seq(testKey, testVal), + Seq(testKey + testKey, testVal + testVal)) ) // "set key" checkAnswer( sql(s"SET $testKey"), - Seq(Seq(s"$testKey=$testVal")) + Seq(Seq(testKey, testVal)) ) checkAnswer( sql(s"SET $nonexistentKey"), - Seq(Seq(s"$nonexistentKey=")) + Seq(Seq(nonexistentKey, "")) ) clear() } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml deleted file mode 100644 index 7fac90fdc596d..0000000000000 --- a/sql/hive-thriftserver/pom.xml +++ /dev/null @@ -1,82 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent - 1.1.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-hive-thriftserver_2.10 - jar - Spark Project Hive - http://spark.apache.org/ - - hive-thriftserver - - - - - org.apache.spark - spark-hive_${scala.binary.version} - ${project.version} - - - org.spark-project.hive - hive-cli - ${hive.version} - - - org.spark-project.hive - hive-jdbc - ${hive.version} - - - org.spark-project.hive - hive-beeline - ${hive.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala deleted file mode 100644 index ddbc2a79fb512..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService -import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ - -/** - * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a - * `HiveThriftServer2` thrift server. - */ -private[hive] object HiveThriftServer2 extends Logging { - var LOG = LogFactory.getLog(classOf[HiveServer2]) - - def main(args: Array[String]) { - val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") - - if (!optionsProcessor.process(args)) { - logger.warn("Error starting HiveThriftServer2 with given arguments") - System.exit(-1) - } - - val ss = new SessionState(new HiveConf(classOf[SessionState])) - - // Set all properties specified via command line. - val hiveConf: HiveConf = ss.getConf - hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logger.debug(s"HiveConf var: $k=$v") - } - - SessionState.start(ss) - - logger.info("Starting SparkContext") - SparkSQLEnv.init() - SessionState.start(ss) - - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.sparkContext.stop() - } - } - ) - - try { - val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(hiveConf) - server.start() - logger.info("HiveThriftServer2 started") - } catch { - case e: Exception => - logger.error("Error starting HiveThriftServer2", e) - System.exit(-1) - } - } -} - -private[hive] class HiveThriftServer2(hiveContext: HiveContext) - extends HiveServer2 - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - val sparkSqlCliService = new SparkSQLCLIService(hiveContext) - setSuperField(this, "cliService", sparkSqlCliService) - addService(sparkSqlCliService) - - val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) - - initCompositeService(hiveConf) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala deleted file mode 100644 index 599294dfbb7d7..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -private[hive] object ReflectionUtils { - def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { - setAncestorField(obj, 1, fieldName, fieldValue) - } - - def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { - val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() - val field = ancestor.getDeclaredField(fieldName) - field.setAccessible(true) - field.set(obj, fieldValue) - } - - def getSuperField[T](obj: AnyRef, fieldName: String): T = { - getAncestorField[T](obj, 1, fieldName) - } - - def getAncestorField[T](clazz: Object, level: Int, fieldName: String): T = { - val ancestor = Iterator.iterate[Class[_]](clazz.getClass)(_.getSuperclass).drop(level).next() - val field = ancestor.getDeclaredField(fieldName) - field.setAccessible(true) - field.get(clazz).asInstanceOf[T] - } - - def invokeStatic(clazz: Class[_], methodName: String, args: (Class[_], AnyRef)*): AnyRef = { - invoke(clazz, null, methodName, args: _*) - } - - def invoke( - clazz: Class[_], - obj: AnyRef, - methodName: String, - args: (Class[_], AnyRef)*): AnyRef = { - - val (types, values) = args.unzip - val method = clazz.getDeclaredMethod(methodName, types: _*) - method.setAccessible(true) - method.invoke(obj, values.toSeq: _*) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala deleted file mode 100755 index 27268ecb923e9..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ /dev/null @@ -1,344 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.io._ -import java.util.{ArrayList => JArrayList} - -import jline.{ConsoleReader, History} -import org.apache.commons.lang.StringUtils -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException -import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.thrift.transport.TSocket - -import org.apache.spark.sql.Logging - -private[hive] object SparkSQLCLIDriver { - private var prompt = "spark-sql" - private var continuedPrompt = "".padTo(prompt.length, ' ') - private var transport:TSocket = _ - - installSignalHandler() - - /** - * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), - * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while - * a command is being processed by the current thread. - */ - def installSignalHandler() { - HiveInterruptUtils.add(new HiveInterruptCallback { - override def interrupt() { - // Handle remote execution mode - if (SparkSQLEnv.sparkContext != null) { - SparkSQLEnv.sparkContext.cancelAllJobs() - } else { - if (transport != null) { - // Force closing of TCP connection upon session termination - transport.getSocket.close() - } - } - } - }) - } - - def main(args: Array[String]) { - val oproc = new OptionsProcessor() - if (!oproc.process_stage1(args)) { - System.exit(1) - } - - // NOTE: It is critical to do this here so that log4j is reinitialized - // before any of the other core hive classes are loaded - var logInitFailed = false - var logInitDetailMessage: String = null - try { - logInitDetailMessage = LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => - logInitFailed = true - logInitDetailMessage = e.getMessage - } - - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) - - sessionState.in = System.in - try { - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.info = new PrintStream(System.err, true, "UTF-8") - sessionState.err = new PrintStream(System.err, true, "UTF-8") - } catch { - case e: UnsupportedEncodingException => System.exit(3) - } - - if (!oproc.process_stage2(sessionState)) { - System.exit(2) - } - - if (!sessionState.getIsSilent) { - if (logInitFailed) System.err.println(logInitDetailMessage) - else SessionState.getConsole.printInfo(logInitDetailMessage) - } - - // Set all properties specified via command line. - val conf: HiveConf = sessionState.getConf - sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => - conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - sessionState.getOverriddenConfigurations.put( - item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - } - - SessionState.start(sessionState) - - // Clean up after we exit - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.stop() - } - } - ) - - // "-h" option has been passed, so connect to Hive thrift server. - if (sessionState.getHost != null) { - sessionState.connect() - if (sessionState.isRemoteMode) { - prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt - continuedPrompt = "".padTo(prompt.length, ' ') - } - } - - if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { - // Hadoop-20 and above - we need to augment classpath using hiveconf - // components. - // See also: code in ExecDriver.java - var loader = conf.getClassLoader - val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) - if (StringUtils.isNotBlank(auxJars)) { - loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")) - } - conf.setClassLoader(loader) - Thread.currentThread().setContextClassLoader(loader) - } - - val cli = new SparkSQLCLIDriver - cli.setHiveVariables(oproc.getHiveVariables) - - // TODO work around for set the log output to console, because the HiveContext - // will set the output into an invalid buffer. - sessionState.in = System.in - try { - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.info = new PrintStream(System.err, true, "UTF-8") - sessionState.err = new PrintStream(System.err, true, "UTF-8") - } catch { - case e: UnsupportedEncodingException => System.exit(3) - } - - // Execute -i init files (always in silent mode) - cli.processInitFiles(sessionState) - - if (sessionState.execString != null) { - System.exit(cli.processLine(sessionState.execString)) - } - - try { - if (sessionState.fileName != null) { - System.exit(cli.processFile(sessionState.fileName)) - } - } catch { - case e: FileNotFoundException => - System.err.println(s"Could not open input file for reading. (${e.getMessage})") - System.exit(3) - } - - val reader = new ConsoleReader() - reader.setBellEnabled(false) - // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) - CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e)) - - val historyDirectory = System.getProperty("user.home") - - try { - if (new File(historyDirectory).exists()) { - val historyFile = historyDirectory + File.separator + ".hivehistory" - reader.setHistory(new History(new File(historyFile))) - } else { - System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + - " does not exist. History will not be available during this session.") - } - } catch { - case e: Exception => - System.err.println("WARNING: Encountered an error while trying to initialize Hive's " + - "history file. History will not be available during this session.") - System.err.println(e.getMessage) - } - - val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") - clientTransportTSocketField.setAccessible(true) - - transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket] - - var ret = 0 - var prefix = "" - val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", - classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) - - def promptWithCurrentDB = s"$prompt$currentDB" - def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( - classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) - - var currentPrompt = promptWithCurrentDB - var line = reader.readLine(currentPrompt + "> ") - - while (line != null) { - if (prefix.nonEmpty) { - prefix += '\n' - } - - if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { - line = prefix + line - ret = cli.processLine(line, true) - prefix = "" - currentPrompt = promptWithCurrentDB - } else { - prefix = prefix + line - currentPrompt = continuedPromptWithDBSpaces - } - - line = reader.readLine(currentPrompt + "> ") - } - - sessionState.close() - - System.exit(ret) - } -} - -private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { - private val sessionState = SessionState.get().asInstanceOf[CliSessionState] - - private val LOG = LogFactory.getLog("CliDriver") - - private val console = new SessionState.LogHelper(LOG) - - private val conf: Configuration = - if (sessionState != null) sessionState.getConf else new Configuration() - - // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver - // because the Hive unit tests do not go through the main() code path. - if (!sessionState.isRemoteMode) { - SparkSQLEnv.init() - } - - override def processCmd(cmd: String): Int = { - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - if (cmd_trimmed.toLowerCase.equals("quit") || - cmd_trimmed.toLowerCase.equals("exit") || - tokens(0).equalsIgnoreCase("source") || - cmd_trimmed.startsWith("!") || - tokens(0).toLowerCase.equals("list") || - sessionState.isRemoteMode) { - val start = System.currentTimeMillis() - super.processCmd(cmd) - val end = System.currentTimeMillis() - val timeTaken: Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds") - 0 - } else { - var ret = 0 - val hconf = conf.asInstanceOf[HiveConf] - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) - - if (proc != null) { - if (proc.isInstanceOf[Driver]) { - val driver = new SparkSQLDriver - - driver.init() - val out = sessionState.out - val start:Long = System.currentTimeMillis() - if (sessionState.getIsVerbose) { - out.println(cmd) - } - - ret = driver.run(cmd).getResponseCode - if (ret != 0) { - driver.close() - return ret - } - - val res = new JArrayList[String]() - - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { - // Print the column names. - Option(driver.getSchema.getFieldSchemas).map { fields => - out.println(fields.map(_.getName).mkString("\t")) - } - } - - try { - while (!out.checkError() && driver.getResults(res)) { - res.foreach(out.println) - res.clear() - } - } catch { - case e:IOException => - console.printError( - s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} - |${org.apache.hadoop.util.StringUtils.stringifyException(e)} - """.stripMargin) - ret = 1 - } - - val cret = driver.close() - if (ret == 0) { - ret = cret - } - - val end = System.currentTimeMillis() - if (end > start) { - val timeTaken:Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds", null) - } - - // Destroy the driver to release all the locks. - driver.destroy() - } else { - if (sessionState.getIsVerbose) { - sessionState.out.println(tokens(0) + " " + cmd_1) - } - ret = proc.run(cmd_1).getResponseCode - } - } - ret - } - } -} - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala deleted file mode 100644 index 42cbf363b274f..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.io.IOException -import java.util.{List => JList} -import javax.security.auth.login.LoginException - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.hive.service.Service.STATE -import org.apache.hive.service.auth.HiveAuthFactory -import org.apache.hive.service.cli.CLIService -import org.apache.hive.service.{AbstractService, Service, ServiceException} - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ - -private[hive] class SparkSQLCLIService(hiveContext: HiveContext) - extends CLIService - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) - setSuperField(this, "sessionManager", sparkSqlSessionManager) - addService(sparkSqlSessionManager) - - try { - HiveAuthFactory.loginFromKeytab(hiveConf) - val serverUserName = ShimLoader.getHadoopShims - .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) - setSuperField(this, "serverUserName", serverUserName) - } catch { - case e @ (_: IOException | _: LoginException) => - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) - } - - initCompositeService(hiveConf) - } -} - -private[thriftserver] trait ReflectedCompositeService { this: AbstractService => - def initCompositeService(hiveConf: HiveConf) { - // Emulating `CompositeService.init(hiveConf)` - val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") - serviceList.foreach(_.init(hiveConf)) - - // Emulating `AbstractService.init(hiveConf)` - invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) - setAncestorField(this, 3, "hiveConf", hiveConf) - invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) - getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala deleted file mode 100644 index 5202aa9903e03..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.util.{ArrayList => JArrayList} - -import org.apache.commons.lang.exception.ExceptionUtils -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} -import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} - -private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) - extends Driver with Logging { - - private var tableSchema: Schema = _ - private var hiveResponse: Seq[String] = _ - - override def init(): Unit = { - } - - private def getResultSetSchema(query: context.QueryExecution): Schema = { - val analyzed = query.analyzed - logger.debug(s"Result Schema: ${analyzed.output}") - if (analyzed.output.size == 0) { - new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) - } else { - val fieldSchemas = analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - - new Schema(fieldSchemas, null) - } - } - - override def run(command: String): CommandProcessorResponse = { - val execution = context.executePlan(context.hql(command).logicalPlan) - - // TODO unify the error code - try { - hiveResponse = execution.stringResult() - tableSchema = getResultSetSchema(execution) - new CommandProcessorResponse(0) - } catch { - case cause: Throwable => - logger.error(s"Failed in [$command]", cause) - new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) - } - } - - override def close(): Int = { - hiveResponse = null - tableSchema = null - 0 - } - - override def getSchema: Schema = tableSchema - - override def getResults(res: JArrayList[String]): Boolean = { - if (hiveResponse == null) { - false - } else { - res.addAll(hiveResponse) - hiveResponse = null - true - } - } - - override def destroy() { - super.destroy() - hiveResponse = null - tableSchema = null - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala deleted file mode 100644 index 451c3bd7b9352..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import org.apache.hadoop.hive.ql.session.SessionState - -import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} - -/** A singleton object for the master program. The slaves should not access this. */ -private[hive] object SparkSQLEnv extends Logging { - logger.debug("Initializing SparkSQLEnv") - - var hiveContext: HiveContext = _ - var sparkContext: SparkContext = _ - - def init() { - if (hiveContext == null) { - sparkContext = new SparkContext(new SparkConf() - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) - - sparkContext.addSparkListener(new StatsReportListener()) - - hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = SessionState.get() - @transient override lazy val hiveconf = sessionState.getConf - } - } - } - - /** Cleans up and shuts down the Spark SQL environments. */ - def stop() { - logger.debug("Shutting down Spark SQL Environment") - // Stop the SparkContext - if (SparkSQLEnv.sparkContext != null) { - sparkContext.stop() - sparkContext = null - hiveContext = null - } - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala deleted file mode 100644 index 6b3275b4eaf04..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.util.concurrent.Executors - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.session.SessionManager - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager - -private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) - extends SessionManager - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) - setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) - getAncestorField[Log](this, 3, "LOG").info( - s"HiveServer2: Async execution pool size $backgroundPoolSize") - - val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) - setSuperField(this, "operationManager", sparkSqlOperationManager) - addService(sparkSqlOperationManager) - - initCompositeService(hiveConf) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala deleted file mode 100644 index a4e1f3e762e89..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver.server - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.math.{random, round} - -import java.sql.Timestamp -import java.util.{Map => JMap} - -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} -import org.apache.hive.service.cli.session.HiveSession - -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} - -/** - * Executes queries using Spark SQL, and maintains a list of handles to active queries. - */ -class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { - val handleToOperation = ReflectionUtils - .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") - - override def newExecuteStatementOperation( - parentSession: HiveSession, - statement: String, - confOverlay: JMap[String, String], - async: Boolean): ExecuteStatementOperation = synchronized { - - val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { - private var result: SchemaRDD = _ - private var iter: Iterator[SparkRow] = _ - private var dataTypes: Array[DataType] = _ - - def close(): Unit = { - // RDDs will be cleaned automatically upon garbage collection. - logger.debug("CLOSING") - } - - def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { - if (!iter.hasNext) { - new RowSet() - } else { - val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. - var curRow = 0 - var rowSet = new ArrayBuffer[Row](maxRows) - - while (curRow < maxRows && iter.hasNext) { - val sparkRow = iter.next() - val row = new Row() - var curCol = 0 - - while (curCol < sparkRow.length) { - dataTypes(curCol) match { - case StringType => - row.addString(sparkRow(curCol).asInstanceOf[String]) - case IntegerType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getInt(curCol))) - case BooleanType => - row.addColumnValue(ColumnValue.booleanValue(sparkRow.getBoolean(curCol))) - case DoubleType => - row.addColumnValue(ColumnValue.doubleValue(sparkRow.getDouble(curCol))) - case FloatType => - row.addColumnValue(ColumnValue.floatValue(sparkRow.getFloat(curCol))) - case DecimalType => - val hiveDecimal = sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal - row.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) - case LongType => - row.addColumnValue(ColumnValue.longValue(sparkRow.getLong(curCol))) - case ByteType => - row.addColumnValue(ColumnValue.byteValue(sparkRow.getByte(curCol))) - case ShortType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getShort(curCol))) - case TimestampType => - row.addColumnValue( - ColumnValue.timestampValue(sparkRow.get(curCol).asInstanceOf[Timestamp])) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = result - .queryExecution - .asInstanceOf[HiveContext#QueryExecution] - .toHiveString((sparkRow.get(curCol), dataTypes(curCol))) - row.addColumnValue(ColumnValue.stringValue(hiveString)) - } - curCol += 1 - } - rowSet += row - curRow += 1 - } - new RowSet(rowSet, 0) - } - } - - def getResultSetSchema: TableSchema = { - logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") - if (result.queryExecution.analyzed.output.size == 0) { - new TableSchema(new FieldSchema("Result", "string", "") :: Nil) - } else { - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - new TableSchema(schema) - } - } - - def run(): Unit = { - logger.info(s"Running query '$statement'") - setState(OperationState.RUNNING) - try { - result = hiveContext.hql(statement) - logger.debug(result.queryExecution.toString()) - val groupId = round(random * 1000000).toString - hiveContext.sparkContext.setJobGroup(groupId, statement) - iter = result.queryExecution.toRdd.toLocalIterator - dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray - setHasResultSet(true) - } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - logger.error("Error executing query:",e) - throw new HiveSQLException(e.toString) - } - setState(OperationState.FINISHED) - } - } - - handleToOperation.put(operation.getHandle, operation) - operation - } -} diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt deleted file mode 100644 index 850f8014b6f05..0000000000000 --- a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt +++ /dev/null @@ -1,5 +0,0 @@ -238val_238 -86val_86 -311val_311 -27val_27 -165val_165 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala deleted file mode 100644 index b90670a796b81..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.io.{BufferedReader, InputStreamReader, PrintWriter} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.sql.hive.test.TestHive - -class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { - val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") - val METASTORE_PATH = TestUtils.getMetastorePath("cli") - - override def beforeAll() { - val pb = new ProcessBuilder( - "../../bin/spark-sql", - "--master", - "local", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - process = pb.start() - outputWriter = new PrintWriter(process.getOutputStream, true) - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "spark-sql>") - } - - override def afterAll() { - process.destroy() - process.waitFor() - } - - test("simple commands") { - val dataFilePath = getDataFile("data/files/small_kv.txt") - executeQuery("create table hive_test1(key int, val string);") - executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") - executeQuery("cache table hive_test1", "Time taken") - } -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala deleted file mode 100644 index 59f4952b78bc6..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ /dev/null @@ -1,125 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent._ - -import java.io.{BufferedReader, InputStreamReader} -import java.sql.{Connection, DriverManager, Statement} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.catalyst.util.getTempFilePath - -/** - * Test for the HiveThriftServer2 using JDBC. - */ -class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { - - val WAREHOUSE_PATH = getTempFilePath("warehouse") - val METASTORE_PATH = getTempFilePath("metastore") - - val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" - val TABLE = "test" - // use a different port, than the hive standard 10000, - // for tests to avoid issues with the port being taken on some machines - val PORT = "10000" - - // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. - val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean - - Class.forName(DRIVER_NAME) - - override def beforeAll() { launchServer() } - - override def afterAll() { stopServer() } - - private def launchServer(args: Seq[String] = Seq.empty) { - // Forking a new process to start the Hive Thrift server. The reason to do this is it is - // hard to clean up Hive resources entirely, so we just start a new process and kill - // that process for cleanup. - val defaultArgs = Seq( - "../../sbin/start-thriftserver.sh", - "--master local", - "--hiveconf", - "hive.root.logger=INFO,console", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH") - val pb = new ProcessBuilder(defaultArgs ++ args) - process = pb.start() - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on") - - // Spawn a thread to read the output from the forked process. - // Note that this is necessary since in some configurations, log4j could be blocked - // if its output to stderr are not read, and eventually blocking the entire test suite. - future { - while (true) { - val stdout = readFrom(inputReader) - val stderr = readFrom(errorReader) - if (VERBOSE && stdout.length > 0) { - println(stdout) - } - if (VERBOSE && stderr.length > 0) { - println(stderr) - } - Thread.sleep(50) - } - } - } - - private def stopServer() { - process.destroy() - process.waitFor() - } - - test("test query execution against a Hive Thrift server") { - Thread.sleep(5 * 1000) - val dataFilePath = getDataFile("data/files/small_kv.txt") - val stmt = createStatement() - stmt.execute("DROP TABLE IF EXISTS test") - stmt.execute("DROP TABLE IF EXISTS test_cached") - stmt.execute("CREATE TABLE test(key int, val string)") - stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") - stmt.execute("CREATE TABLE test_cached as select * from test limit 4") - stmt.execute("CACHE TABLE test_cached") - - var rs = stmt.executeQuery("select count(*) from test") - rs.next() - assert(rs.getInt(1) === 5) - - rs = stmt.executeQuery("select count(*) from test_cached") - rs.next() - assert(rs.getInt(1) === 4) - - stmt.close() - } - - def getConnection: Connection = { - val connectURI = s"jdbc:hive2://localhost:$PORT/" - DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") - } - - def createStatement(): Statement = getConnection.createStatement() -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala deleted file mode 100644 index bb2242618fbef..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.io.{BufferedReader, PrintWriter} -import java.text.SimpleDateFormat -import java.util.Date - -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException - -object TestUtils { - val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") - - def getWarehousePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + - timestamp.format(new Date) - } - - def getMetastorePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + - timestamp.format(new Date) - } - - // Dummy function for initialize the log4j properties. - def init() { } - - // initialize log4j - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } -} - -trait TestUtils { - var process : Process = null - var outputWriter : PrintWriter = null - var inputReader : BufferedReader = null - var errorReader : BufferedReader = null - - def executeQuery( - cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { - println("Executing: " + cmd + ", expecting output: " + outputMessage) - outputWriter.write(cmd + "\n") - outputWriter.flush() - waitForQuery(timeout, outputMessage) - } - - protected def waitForQuery(timeout: Long, message: String): String = { - if (waitForOutput(errorReader, message, timeout)) { - Thread.sleep(500) - readOutput() - } else { - assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) - null - } - } - - // Wait for the specified str to appear in the output. - protected def waitForOutput( - reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { - val startTime = System.currentTimeMillis - var out = "" - while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { - out += readFrom(reader) - } - out.contains(str) - } - - // Read stdout output and filter out garbage collection messages. - protected def readOutput(): String = { - val output = readFrom(inputReader) - // Remove GC Messages - val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) - .mkString("\n") - filteredOutput - } - - protected def readFrom(reader: BufferedReader): String = { - var out = "" - var c = 0 - while (reader.ready) { - c = reader.read() - out += c.asInstanceOf[Char] - } - out - } - - protected def getDataFile(name: String) = { - Thread.currentThread().getContextClassLoader.getResource(name) - } -} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 93d00f7c37c9b..1699ffe06ce15 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -32,7 +32,7 @@ Spark Project Hive http://spark.apache.org/ - hive + hive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 84d43eaeea51d..201c85f3d501e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -255,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DecimalType, TimestampType, BinaryType) - protected[sql] def toHiveString(a: (Any, DataType)): String = a match { + protected def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 8489f2a34e63c..6f36a4f8cb905 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -416,10 +416,10 @@ class HiveQuerySuite extends HiveComparisonTest { hql(s"set $testKey=$testVal") assert(get(testKey, testVal + "_") == testVal) - hql("set some.property=20") - assert(get("some.property", "0") == "20") - hql("set some.property = 40") - assert(get("some.property", "0") == "40") + hql("set mapred.reduce.tasks=20") + assert(get("mapred.reduce.tasks", "0") == "20") + hql("set mapred.reduce.tasks = 40") + assert(get("mapred.reduce.tasks", "0") == "40") hql(s"set $testKey=$testVal") assert(get(testKey, "0") == testVal) @@ -433,61 +433,63 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { case Row(key: String, value: String) => key -> value }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. assert(hql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql("SET")) } hql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - hql(s"SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(hql("SET")) } // "set key" - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - hql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as hql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET")) } - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(sql(s"SET $nonexistentKey")) } clear() diff --git a/streaming/pom.xml b/streaming/pom.xml index b99f306b8f2cc..f60697ce745b7 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming_2.10 - streaming + streaming jar Spark Project Streaming diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..c0ee8faa7a615 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,7 +27,7 @@ org.apache.spark spark-tools_2.10 - tools + tools jar Spark Project Tools diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 51744ece0412d..5b13a1f002d6e 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-alpha + yarn-alpha org.apache.spark diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..efb473aa1b261 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -29,7 +29,7 @@ pom Spark Project YARN Parent POM - yarn + yarn diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..ceaf9f9d71001 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-stable + yarn-stable org.apache.spark From 9d8666cac84fc4fc867f6a5e80097dbe5cb65301 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 25 Jul 2014 18:45:02 -0700 Subject: [PATCH 0080/1492] Part of [SPARK-2456] Removed some HashMaps from DAGScheduler by storing information in Stage. This is part of the scheduler cleanup/refactoring effort to make the scheduler code easier to maintain. @kayousterhout @markhamstra please take a look ... Author: Reynold Xin Closes #1561 from rxin/dagSchedulerHashMaps and squashes the following commits: 1c44e15 [Reynold Xin] Clear pending tasks in submitMissingTasks. 620a0d1 [Reynold Xin] Use filterKeys. 5b54404 [Reynold Xin] Code review feedback. c1e9a1c [Reynold Xin] Removed some HashMaps from DAGScheduler by storing information in Stage. --- .../apache/spark/scheduler/DAGScheduler.scala | 143 +++++++----------- .../org/apache/spark/scheduler/Stage.scala | 19 ++- .../spark/scheduler/DAGSchedulerSuite.scala | 4 - 3 files changed, 69 insertions(+), 97 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 00b8af27a7b39..dc6142ab79d03 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -85,12 +85,9 @@ class DAGScheduler( private val nextStageId = new AtomicInteger(0) private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new HashMap[Int, HashSet[Int]] private[scheduler] val stageIdToStage = new HashMap[Int, Stage] private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] - private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[scheduler] val stageToInfos = new HashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] @@ -101,9 +98,6 @@ class DAGScheduler( // Stages that must be resubmitted due to fetch failures private[scheduler] val failedStages = new HashSet[Stage] - // Missing tasks from each stage - private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] - private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on @@ -223,7 +217,6 @@ class DAGScheduler( new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) - stageToInfos(stage) = StageInfo.fromStage(stage) stage } @@ -315,13 +308,12 @@ class DAGScheduler( */ private def updateJobIdStageIdMaps(jobId: Int, stage: Stage) { def updateJobIdStageIdMapsList(stages: List[Stage]) { - if (!stages.isEmpty) { + if (stages.nonEmpty) { val s = stages.head - stageIdToJobIds.getOrElseUpdate(s.id, new HashSet[Int]()) += jobId + s.jobIds += jobId jobIdToStageIds.getOrElseUpdate(jobId, new HashSet[Int]()) += s.id - val parents = getParentStages(s.rdd, jobId) - val parentsWithoutThisJobId = parents.filter(p => - !stageIdToJobIds.get(p.id).exists(_.contains(jobId))) + val parents: List[Stage] = getParentStages(s.rdd, jobId) + val parentsWithoutThisJobId = parents.filter { ! _.jobIds.contains(jobId) } updateJobIdStageIdMapsList(parentsWithoutThisJobId ++ stages.tail) } } @@ -333,16 +325,15 @@ class DAGScheduler( * handle cancelling tasks or notifying the SparkListener about finished jobs/stages/tasks. * * @param job The job whose state to cleanup. - * @param resultStage Specifies the result stage for the job; if set to None, this method - * searches resultStagesToJob to find and cleanup the appropriate result stage. */ - private def cleanupStateForJobAndIndependentStages(job: ActiveJob, resultStage: Option[Stage]) { + private def cleanupStateForJobAndIndependentStages(job: ActiveJob) { val registeredStages = jobIdToStageIds.get(job.jobId) if (registeredStages.isEmpty || registeredStages.get.isEmpty) { logError("No stages registered for job " + job.jobId) } else { - stageIdToJobIds.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { - case (stageId, jobSet) => + stageIdToStage.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { + case (stageId, stage) => + val jobSet = stage.jobIds if (!jobSet.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" @@ -355,14 +346,9 @@ class DAGScheduler( logDebug("Removing running stage %d".format(stageId)) runningStages -= stage } - stageToInfos -= stage for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { shuffleToMapStage.remove(k) } - if (pendingTasks.contains(stage) && !pendingTasks(stage).isEmpty) { - logDebug("Removing pending status for stage %d".format(stageId)) - } - pendingTasks -= stage if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) waitingStages -= stage @@ -374,7 +360,6 @@ class DAGScheduler( } // data structures based on StageId stageIdToStage -= stageId - stageIdToJobIds -= stageId ShuffleMapTask.removeStage(stageId) ResultTask.removeStage(stageId) @@ -393,19 +378,7 @@ class DAGScheduler( jobIdToStageIds -= job.jobId jobIdToActiveJob -= job.jobId activeJobs -= job - - if (resultStage.isEmpty) { - // Clean up result stages. - val resultStagesForJob = resultStageToJob.keySet.filter( - stage => resultStageToJob(stage).jobId == job.jobId) - if (resultStagesForJob.size != 1) { - logWarning( - s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") - } - resultStageToJob --= resultStagesForJob - } else { - resultStageToJob -= resultStage.get - } + job.finalStage.resultOfJob = None } /** @@ -591,9 +564,10 @@ class DAGScheduler( job.listener.jobFailed(exception) } finally { val s = job.finalStage - stageIdToJobIds -= s.id // clean up data structures that were populated for a local job, - stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through - stageToInfos -= s // completion events or stage abort + // clean up data structures that were populated for a local job, + // but that won't get cleaned up via the normal paths through + // completion events or stage abort + stageIdToStage -= s.id jobIdToStageIds -= job.jobId listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) } @@ -605,12 +579,8 @@ class DAGScheduler( // That should take care of at least part of the priority inversion problem with // cross-job dependencies. private def activeJobForStage(stage: Stage): Option[Int] = { - if (stageIdToJobIds.contains(stage.id)) { - val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted - jobsThatUseStage.find(jobIdToActiveJob.contains) - } else { - None - } + val jobsThatUseStage: Array[Int] = stage.jobIds.toArray.sorted + jobsThatUseStage.find(jobIdToActiveJob.contains) } private[scheduler] def handleJobGroupCancelled(groupId: String) { @@ -642,9 +612,8 @@ class DAGScheduler( // is in the process of getting stopped. val stageFailedMessage = "Stage cancelled because SparkContext was shut down" runningStages.foreach { stage => - val info = stageToInfos(stage) - info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(info)) + stage.info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(stage.info)) } listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } @@ -690,7 +659,7 @@ class DAGScheduler( } else { jobIdToActiveJob(jobId) = job activeJobs += job - resultStageToJob(finalStage) = job + finalStage.resultOfJob = Some(job) listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) @@ -727,8 +696,7 @@ class DAGScheduler( private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry - val myPending = pendingTasks.getOrElseUpdate(stage, new HashSet) - myPending.clear() + stage.pendingTasks.clear() var tasks = ArrayBuffer[Task[_]]() if (stage.isShuffleMap) { for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { @@ -737,7 +705,7 @@ class DAGScheduler( } } else { // This is a final stage; figure out its job's missing partitions - val job = resultStageToJob(stage) + val job = stage.resultOfJob.get for (id <- 0 until job.numPartitions if !job.finished(id)) { val partition = job.partitions(id) val locs = getPreferredLocs(stage.rdd, partition) @@ -758,7 +726,7 @@ class DAGScheduler( // serializable. If tasks are not serializable, a SparkListenerStageCompleted event // will be posted, which should always come after a corresponding SparkListenerStageSubmitted // event. - listenerBus.post(SparkListenerStageSubmitted(stageToInfos(stage), properties)) + listenerBus.post(SparkListenerStageSubmitted(stage.info, properties)) // Preemptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception @@ -778,11 +746,11 @@ class DAGScheduler( } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") - myPending ++= tasks - logDebug("New pending tasks: " + myPending) + stage.pendingTasks ++= tasks + logDebug("New pending tasks: " + stage.pendingTasks) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stageToInfos(stage).submissionTime = Some(clock.getTime()) + stage.info.submissionTime = Some(clock.getTime()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) @@ -807,13 +775,13 @@ class DAGScheduler( val stage = stageIdToStage(task.stageId) def markStageAsFinished(stage: Stage) = { - val serviceTime = stageToInfos(stage).submissionTime match { + val serviceTime = stage.info.submissionTime match { case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) case _ => "Unknown" } logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stageToInfos(stage).completionTime = Some(clock.getTime()) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + stage.info.completionTime = Some(clock.getTime()) + listenerBus.post(SparkListenerStageCompleted(stage.info)) runningStages -= stage } event.reason match { @@ -822,10 +790,10 @@ class DAGScheduler( // TODO: fail the stage if the accumulator update fails... Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } - pendingTasks(stage) -= task + stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => - resultStageToJob.get(stage) match { + stage.resultOfJob match { case Some(job) => if (!job.finished(rt.outputId)) { job.finished(rt.outputId) = true @@ -833,7 +801,7 @@ class DAGScheduler( // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { markStageAsFinished(stage) - cleanupStateForJobAndIndependentStages(job, Some(stage)) + cleanupStateForJobAndIndependentStages(job) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } @@ -860,7 +828,7 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) } - if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { + if (runningStages.contains(stage) && stage.pendingTasks.isEmpty) { markStageAsFinished(stage) logInfo("looking for newly runnable stages") logInfo("running: " + runningStages) @@ -909,7 +877,7 @@ class DAGScheduler( case Resubmitted => logInfo("Resubmitted " + task + ", so marking it as still running") - pendingTasks(stage) += task + stage.pendingTasks += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable @@ -994,13 +962,14 @@ class DAGScheduler( } private[scheduler] def handleStageCancellation(stageId: Int) { - if (stageIdToJobIds.contains(stageId)) { - val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray - jobsThatUseStage.foreach(jobId => { - handleJobCancellation(jobId, "because Stage %s was cancelled".format(stageId)) - }) - } else { - logInfo("No active jobs to kill for Stage " + stageId) + stageIdToStage.get(stageId) match { + case Some(stage) => + val jobsThatUseStage: Array[Int] = stage.jobIds.toArray + jobsThatUseStage.foreach { jobId => + handleJobCancellation(jobId, s"because Stage $stageId was cancelled") + } + case None => + logInfo("No active jobs to kill for Stage " + stageId) } submitWaitingStages() } @@ -1009,8 +978,8 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), - "Job %d cancelled %s".format(jobId, reason), None) + failJobAndIndependentStages( + jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason)) } submitWaitingStages() } @@ -1024,26 +993,21 @@ class DAGScheduler( // Skip all the actions if the stage has been removed. return } - val dependentStages = resultStageToJob.keys.filter(x => stageDependsOn(x, failedStage)).toSeq - stageToInfos(failedStage).completionTime = Some(clock.getTime()) - for (resultStage <- dependentStages) { - val job = resultStageToJob(resultStage) - failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", - Some(resultStage)) + val dependentJobs: Seq[ActiveJob] = + activeJobs.filter(job => stageDependsOn(job.finalStage, failedStage)).toSeq + failedStage.info.completionTime = Some(clock.getTime()) + for (job <- dependentJobs) { + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } - if (dependentStages.isEmpty) { + if (dependentJobs.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") } } /** * Fails a job and all stages that are only used by that job, and cleans up relevant state. - * - * @param resultStage The result stage for the job, if known. Used to cleanup state for the job - * slightly more efficiently than when not specified. */ - private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, - resultStage: Option[Stage]) { + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { val error = new SparkException(failureReason) var ableToCancelStages = true @@ -1057,7 +1021,7 @@ class DAGScheduler( logError("No stages registered for job " + job.jobId) } stages.foreach { stageId => - val jobsForStage = stageIdToJobIds.get(stageId) + val jobsForStage: Option[HashSet[Int]] = stageIdToStage.get(stageId).map(_.jobIds) if (jobsForStage.isEmpty || !jobsForStage.get.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" @@ -1071,9 +1035,8 @@ class DAGScheduler( if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask taskScheduler.cancelTasks(stageId, shouldInterruptThread) - val stageInfo = stageToInfos(stage) - stageInfo.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + stage.info.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stage.info)) } catch { case e: UnsupportedOperationException => logInfo(s"Could not cancel tasks for stage $stageId", e) @@ -1086,7 +1049,7 @@ class DAGScheduler( if (ableToCancelStages) { job.listener.jobFailed(error) - cleanupStateForJobAndIndependentStages(job, resultStage) + cleanupStateForJobAndIndependentStages(job) listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 798cbc598d36e..800905413d145 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashSet + import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.BlockManagerId @@ -56,8 +58,22 @@ private[spark] class Stage( val numPartitions = rdd.partitions.size val outputLocs = Array.fill[List[MapStatus]](numPartitions)(Nil) var numAvailableOutputs = 0 + + /** Set of jobs that this stage belongs to. */ + val jobIds = new HashSet[Int] + + /** For stages that are the final (consists of only ResultTasks), link to the ActiveJob. */ + var resultOfJob: Option[ActiveJob] = None + var pendingTasks = new HashSet[Task[_]] + private var nextAttemptId = 0 + val name = callSite.shortForm + val details = callSite.longForm + + /** Pointer to the [StageInfo] object, set by DAGScheduler. */ + var info: StageInfo = StageInfo.fromStage(this) + def isAvailable: Boolean = { if (!isShuffleMap) { true @@ -108,9 +124,6 @@ private[spark] class Stage( def attemptId: Int = nextAttemptId - val name = callSite.shortForm - val details = callSite.longForm - override def toString = "Stage " + id override def hashCode(): Int = id diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 44dd1e092ad67..9021662bcf712 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -686,15 +686,11 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F BlockManagerId("exec-" + host, host, 12345, 0) private def assertDataStructuresEmpty = { - assert(scheduler.pendingTasks.isEmpty) assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) assert(scheduler.jobIdToActiveJob.isEmpty) assert(scheduler.jobIdToStageIds.isEmpty) - assert(scheduler.stageIdToJobIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) - assert(scheduler.stageToInfos.isEmpty) - assert(scheduler.resultStageToJob.isEmpty) assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) assert(scheduler.waitingStages.isEmpty) From 8904791230a0fae336db93e5a80f65c4d9d584dc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 25 Jul 2014 19:17:49 -0700 Subject: [PATCH 0081/1492] [SPARK-2659][SQL] Fix division semantics for hive Author: Michael Armbrust Closes #1557 from marmbrus/fixDivision and squashes the following commits: b85077f [Michael Armbrust] Fix unit tests. af98f29 [Michael Armbrust] Change DIV to long type 0c29ae8 [Michael Armbrust] Fix division semantics for hive --- .../catalyst/analysis/HiveTypeCoercion.scala | 18 ++++++++++++++++++ .../optimizer/ConstantFoldingSuite.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 3 ++- .../div-0-3760f9b354ddacd7c7b01b28791d4585 | 1 + ...division-0-63b19f8a22471c8ba0415c1d3bc276f7 | 1 + .../hive/execution/HiveComparisonTest.scala | 6 ------ .../sql/hive/execution/HiveQuerySuite.scala | 5 ++++- 7 files changed, 27 insertions(+), 9 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 create mode 100644 sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 67a8ce9b88c3f..47c7ad076ad07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -50,6 +50,7 @@ trait HiveTypeCoercion { StringToIntegralCasts :: FunctionArgumentConversion :: CastNulls :: + Division :: Nil /** @@ -317,6 +318,23 @@ trait HiveTypeCoercion { } } + /** + * Hive only performs integral division with the DIV operator. The arguments to / are always + * converted to fractional types. + */ + object Division extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes who's children have not been resolved yet. + case e if !e.childrenResolved => e + + // Decimal and Double remain the same + case d: Divide if d.dataType == DoubleType => d + case d: Divide if d.dataType == DecimalType => d + + case Divide(l, r) => Divide(Cast(l, DoubleType), Cast(r, DoubleType)) + } + } + /** * Ensures that NullType gets casted to some other types under certain circumstances. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index d607eed1bea89..0a27cce337482 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -83,7 +83,7 @@ class ConstantFoldingSuite extends PlanTest { Literal(10) as Symbol("2*3+4"), Literal(14) as Symbol("2*(3+4)")) .where(Literal(true)) - .groupBy(Literal(3))(Literal(3) as Symbol("9/3")) + .groupBy(Literal(3.0))(Literal(3.0) as Symbol("9/3")) .analyze comparePlans(optimized, correctAnswer) 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 4395874526d51..e6ab68b563f8d 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 @@ -925,7 +925,8 @@ private[hive] object HiveQl { case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) case Token("/", left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) - case Token(DIV(), left :: right:: Nil) => Divide(nodeToExpr(left), nodeToExpr(right)) + case Token(DIV(), left :: right:: Nil) => + Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) /* Comparisons */ diff --git a/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 b/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 new file mode 100644 index 0000000000000..17ba0bea723c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/div-0-3760f9b354ddacd7c7b01b28791d4585 @@ -0,0 +1 @@ +0 0 0 1 2 diff --git a/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 new file mode 100644 index 0000000000000..7b7a9175114ce --- /dev/null +++ b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 @@ -0,0 +1 @@ +2.0 0.5 0.3333333333333333 0.002 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 08ef4d9b6bb93..b4dbf2b115799 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -350,12 +350,6 @@ abstract class HiveComparisonTest val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") - println("hive output") - hive.foreach(println) - - println("catalyst printout") - catalyst.foreach(println) - if (recomputeCache) { logger.warn(s"Clearing cache files for failed test $testCaseName") hiveCacheFiles.foreach(_.delete()) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 6f36a4f8cb905..a8623b64c656f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -52,7 +52,10 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT * FROM src WHERE key Between 1 and 2") createQueryTest("div", - "SELECT 1 DIV 2, 1 div 2, 1 dIv 2 FROM src LIMIT 1") + "SELECT 1 DIV 2, 1 div 2, 1 dIv 2, 100 DIV 51, 100 DIV 49 FROM src LIMIT 1") + + createQueryTest("division", + "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1") test("Query expressed in SQL") { assert(sql("SELECT 1").collect() === Array(Seq(1))) From cf3e9fd84dc64f8a57ecbcfdd6b22f5492d41bd7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 26 Jul 2014 00:54:05 -0700 Subject: [PATCH 0082/1492] [SPARK-1458] [PySpark] Expose sc.version in Java and PySpark Author: Josh Rosen Closes #1596 from JoshRosen/spark-1458 and squashes the following commits: fdbb0bf [Josh Rosen] Add SparkContext.version to Python & Java [SPARK-1458] --- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 3 +++ python/pyspark/context.py | 7 +++++++ 2 files changed, 10 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 1e0493c4855e0..a678355a1cfa0 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -112,6 +112,9 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def startTime: java.lang.Long = sc.startTime + /** The version of Spark on which this application is running. */ + def version: String = sc.version + /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ def defaultParallelism: java.lang.Integer = sc.defaultParallelism diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 024fb881877c9..bdf14ea0ee27a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -216,6 +216,13 @@ def setSystemProperty(cls, key, value): SparkContext._ensure_initialized() SparkContext._jvm.java.lang.System.setProperty(key, value) + @property + def version(self): + """ + The version of Spark on which this application is running. + """ + return self._jsc.version() + @property def defaultParallelism(self): """ From 66f26a4610aede57322cb7e193a50aecb6c57d22 Mon Sep 17 00:00:00 2001 From: Hossein Date: Sat, 26 Jul 2014 01:04:56 -0700 Subject: [PATCH 0083/1492] [SPARK-2696] Reduce default value of spark.serializer.objectStreamReset The current default value of spark.serializer.objectStreamReset is 10,000. When trying to re-partition (e.g., to 64 partitions) a large file (e.g., 500MB), containing 1MB records, the serializer will cache 10000 x 1MB x 64 ~= 640 GB which will cause out of memory errors. This patch sets the default value to a more reasonable default value (100). Author: Hossein Closes #1595 from falaki/objectStreamReset and squashes the following commits: 650a935 [Hossein] Updated documentation 1aa0df8 [Hossein] Reduce default value of spark.serializer.objectStreamReset --- .../scala/org/apache/spark/serializer/JavaSerializer.scala | 2 +- docs/configuration.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 0a7e1ec539679..a7fa057ee05f7 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -108,7 +108,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize */ @DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { - private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset) diff --git a/docs/configuration.md b/docs/configuration.md index dac8bb1d52468..4e4b78153a105 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -380,13 +380,13 @@ Apart from these, the following properties are also available, and may be useful spark.serializer.objectStreamReset - 10000 + 100 When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches objects to prevent writing redundant data, however that stops garbage collection of those objects. By calling 'reset' you flush that info from the serializer, and allow old objects to be collected. To turn off this periodic reset set it to a value <= 0. - By default it will reset the serializer every 10,000 objects. + By default it will reset the serializer every 100 objects. From 75663b57f90bb173f0c6c288944ec568c4719b2a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 26 Jul 2014 01:07:08 -0700 Subject: [PATCH 0084/1492] [SPARK-2652] [PySpark] Turning some default configs for PySpark Add several default configs for PySpark, related to serialization in JVM. spark.serializer = org.apache.spark.serializer.KryoSerializer spark.serializer.objectStreamReset = 100 spark.rdd.compress = True This will help to reduce the memory usage during RDD.partitionBy() Author: Davies Liu Closes #1568 from davies/conf and squashes the following commits: cd316f1 [Davies Liu] remove duplicated line f71a355 [Davies Liu] rebase to master, add spark.rdd.compress = True 8f63f45 [Davies Liu] Merge branch 'master' into conf 8bc9f08 [Davies Liu] fix unittest c04a83d [Davies Liu] some default configs for PySpark --- python/pyspark/context.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index bdf14ea0ee27a..e8ac9895cf54a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -37,6 +37,15 @@ from py4j.java_collections import ListConverter +# These are special default configs for PySpark, they will overwrite +# the default ones for Spark if they are not configured by user. +DEFAULT_CONFIGS = { + "spark.serializer": "org.apache.spark.serializer.KryoSerializer", + "spark.serializer.objectStreamReset": 100, + "spark.rdd.compress": True, +} + + class SparkContext(object): """ Main entry point for Spark functionality. A SparkContext represents the @@ -101,7 +110,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, else: self.serializer = BatchedSerializer(self._unbatched_serializer, batchSize) - self._conf.setIfMissing("spark.rdd.compress", "true") + # Set any parameters passed directly to us on the conf if master: self._conf.setMaster(master) @@ -112,6 +121,8 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, if environment: for key, value in environment.iteritems(): self._conf.setExecutorEnv(key, value) + for key, value in DEFAULT_CONFIGS.items(): + self._conf.setIfMissing(key, value) # Check that we have at least the required parameters if not self._conf.contains("spark.master"): From c183b92c3c70ad2d36a2d60bdb10c02b65bc0212 Mon Sep 17 00:00:00 2001 From: bpaulin Date: Sat, 26 Jul 2014 10:27:09 -0700 Subject: [PATCH 0085/1492] [SPARK-2279] Added emptyRDD method to Java API Added emptyRDD method to Java API with tests. Author: bpaulin Closes #1597 from bobpaulin/SPARK-2279 and squashes the following commits: 5ad57c2 [bpaulin] [SPARK-2279] Added emptyRDD method to Java API --- .../org/apache/spark/api/java/JavaSparkContext.scala | 9 ++++++++- core/src/test/java/org/apache/spark/JavaAPISuite.java | 9 +++++++-- 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index a678355a1cfa0..8a5f8088a05ca 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{EmptyRDD, RDD} /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns @@ -135,6 +135,13 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork sc.parallelize(JavaConversions.asScalaBuffer(list), numSlices) } + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T]: JavaRDD[T] = { + implicit val ctag: ClassTag[T] = fakeClassTag + JavaRDD.fromRDD(new EmptyRDD[T](sc)) + } + + /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T]): JavaRDD[T] = parallelize(list, sc.defaultParallelism) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b2868b59ce6c6..f882a8623fd84 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -118,8 +118,7 @@ public void intersection() { JavaRDD intersections = s1.intersection(s2); Assert.assertEquals(3, intersections.count()); - List list = new ArrayList(); - JavaRDD empty = sc.parallelize(list); + JavaRDD empty = sc.emptyRDD(); JavaRDD emptyIntersection = empty.intersection(s2); Assert.assertEquals(0, emptyIntersection.count()); @@ -184,6 +183,12 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @Test + public void emptyRDD() { + JavaRDD rdd = sc.emptyRDD(); + Assert.assertEquals("Empty RDD shouldn't have any values", 0, rdd.count()); + } + @Test public void sortBy() { List> pairs = new ArrayList>(); From 12901643b7e808aa75cf0b19e2d0c3d40b1a978d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 26 Jul 2014 15:00:32 -0700 Subject: [PATCH 0086/1492] [SPARK-2704] Name threads in ConnectionManager and mark them as daemon. handleMessageExecutor, handleReadWriteExecutor, and handleConnectExecutor are not marked as daemon and not named. I think there exists some condition in which Spark programs won't terminate because of this. Stack dump attached in https://issues.apache.org/jira/browse/SPARK-2704 Author: Reynold Xin Closes #1604 from rxin/daemon and squashes the following commits: 98d6a6c [Reynold Xin] [SPARK-2704] Name threads in ConnectionManager and mark them as daemon. --- .../spark/network/ConnectionManager.scala | 9 ++++--- .../scala/org/apache/spark/util/Utils.scala | 27 ++++++++++++------- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 8a1cdb812962e..566e8a4aaa1d2 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -62,13 +62,15 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, conf.getInt("spark.core.connection.handler.threads.min", 20), conf.getInt("spark.core.connection.handler.threads.max", 60), conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-message-executor")) private val handleReadWriteExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.io.threads.min", 4), conf.getInt("spark.core.connection.io.threads.max", 32), conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-read-write-executor")) // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : // which should be executed asap @@ -76,7 +78,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, conf.getInt("spark.core.connection.connect.threads.min", 1), conf.getInt("spark.core.connection.connect.threads.max", 8), conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, - new LinkedBlockingDeque[Runnable]()) + new LinkedBlockingDeque[Runnable](), + Utils.namedThreadFactory("handle-connect-executor")) private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation 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 1a4f4eba98ea8..8cbb9050f393b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,7 +21,7 @@ import java.io._ import java.net.{InetAddress, Inet4Address, NetworkInterface, URI, URL, URLConnection} import java.nio.ByteBuffer import java.util.{Locale, Random, UUID} -import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map @@ -553,19 +553,19 @@ private[spark] object Utils extends Logging { new ThreadFactoryBuilder().setDaemon(true) /** - * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a - * unique, sequentially assigned integer. + * Create a thread factory that names threads with a prefix and also sets the threads to daemon. */ - def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { - val threadFactory = daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() - Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] + def namedThreadFactory(prefix: String): ThreadFactory = { + daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() } /** - * Return the string to tell how long has passed in milliseconds. + * Wrapper over newCachedThreadPool. Thread names are formatted as prefix-ID, where ID is a + * unique, sequentially assigned integer. */ - def getUsedTimeMs(startTimeMs: Long): String = { - " " + (System.currentTimeMillis - startTimeMs) + " ms" + def newDaemonCachedThreadPool(prefix: String): ThreadPoolExecutor = { + val threadFactory = namedThreadFactory(prefix) + Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] } /** @@ -573,10 +573,17 @@ private[spark] object Utils extends Logging { * unique, sequentially assigned integer. */ def newDaemonFixedThreadPool(nThreads: Int, prefix: String): ThreadPoolExecutor = { - val threadFactory = daemonThreadFactoryBuilder.setNameFormat(prefix + "-%d").build() + val threadFactory = namedThreadFactory(prefix) Executors.newFixedThreadPool(nThreads, threadFactory).asInstanceOf[ThreadPoolExecutor] } + /** + * Return the string to tell how long has passed in milliseconds. + */ + def getUsedTimeMs(startTimeMs: Long): String = { + " " + (System.currentTimeMillis - startTimeMs) + " ms" + } + private def listFilesSafely(file: File): Seq[File] = { val files = file.listFiles() if (files == null) { From ba46bbed5d32aec0f11f0b71c82bba8dbe19f05a Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 26 Jul 2014 17:37:05 -0700 Subject: [PATCH 0087/1492] [SPARK-2601] [PySpark] Fix Py4J error when transforming pickleFiles MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Similar to SPARK-1034, the problem was that Py4J didn’t cope well with the fake ClassTags used in the Java API. It doesn’t look like there’s any reason why PythonRDD needs to take a ClassTag, since it just ignores the type of the previous RDD, so I removed the type parameter and we no longer pass ClassTags from Python. Author: Josh Rosen Closes #1605 from JoshRosen/spark-2601 and squashes the following commits: b68e118 [Josh Rosen] Fix Py4J error when transforming pickleFiles [SPARK-2601] --- .../scala/org/apache/spark/api/python/PythonRDD.scala | 4 ++-- python/pyspark/rdd.py | 4 +--- python/pyspark/tests.py | 9 +++++++++ 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index d6b0988641a97..d87783efd2d01 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -37,8 +37,8 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils -private[spark] class PythonRDD[T: ClassTag]( - parent: RDD[T], +private[spark] class PythonRDD( + parent: RDD[_], command: Array[Byte], envVars: JMap[String, String], pythonIncludes: JList[String], diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 113a082e16721..b84d976114f0d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1687,7 +1687,6 @@ def _jrdd(self): [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) self.ctx._pickled_broadcast_vars.clear() - class_tag = self._prev_jrdd.classTag() env = MapConverter().convert(self.ctx.environment, self.ctx._gateway._gateway_client) includes = ListConverter().convert(self.ctx._python_includes, @@ -1696,8 +1695,7 @@ def _jrdd(self): bytearray(pickled_command), env, includes, self.preservesPartitioning, self.ctx.pythonExec, - broadcast_vars, self.ctx._javaAccumulator, - class_tag) + broadcast_vars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index a92abbf371f18..8ba51461d106d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -226,6 +226,15 @@ def test_transforming_cartesian_result(self): cart = rdd1.cartesian(rdd2) result = cart.map(lambda (x, y): x + y).collect() + def test_transforming_pickle_file(self): + # Regression test for SPARK-2601 + data = self.sc.parallelize(["Hello", "World!"]) + tempFile = tempfile.NamedTemporaryFile(delete=True) + tempFile.close() + data.saveAsPickleFile(tempFile.name) + pickled_file = self.sc.pickleFile(tempFile.name) + pickled_file.map(lambda x: x).collect() + def test_cartesian_on_textfile(self): # Regression test for path = os.path.join(SPARK_HOME, "python/test_support/hello.txt") From b547f69bdb5f4a6d5f471a2d998c2df6fb2a9347 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 26 Jul 2014 22:44:17 -0700 Subject: [PATCH 0088/1492] SPARK-2680: Lower spark.shuffle.memoryFraction to 0.2 by default Author: Matei Zaharia Closes #1593 from mateiz/spark-2680 and squashes the following commits: 3c949c4 [Matei Zaharia] Lower spark.shuffle.memoryFraction to 0.2 by default --- .../apache/spark/util/collection/ExternalAppendOnlyMap.scala | 2 +- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index be8f6529f7a1c..c22bb8d9c60a9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -74,7 +74,7 @@ class ExternalAppendOnlyMap[K, V, C]( // Collective memory threshold shared across all running tasks private val maxMemoryThreshold = { - val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.3) + val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.2) val safetyFraction = sparkConf.getDouble("spark.shuffle.safetyFraction", 0.8) (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } diff --git a/docs/configuration.md b/docs/configuration.md index 4e4b78153a105..46e3dd914b5ac 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -239,7 +239,7 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.memoryFraction - 0.3 + 0.2 Fraction of Java heap to use for aggregation and cogroups during shuffles, if spark.shuffle.spill is true. At any given time, the collective size of From aaf2b735fddbebccd28012006ee4647af3b3624f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 26 Jul 2014 22:56:07 -0700 Subject: [PATCH 0089/1492] [SPARK-2361][MLLIB] Use broadcast instead of serializing data directly into task closure We saw task serialization problems with large feature dimension, which could be avoid if we don't serialize data directly into task but use broadcast variables. This PR uses broadcast in both training and prediction and adds tests to make sure the task size is small. Author: Xiangrui Meng Closes #1427 from mengxr/broadcast-new and squashes the following commits: b9a1228 [Xiangrui Meng] style update b97c184 [Xiangrui Meng] minimal change to LBFGS 9ebadcc [Xiangrui Meng] add task size test to RowMatrix 9427bf0 [Xiangrui Meng] add task size tests to linear methods e0a5cf2 [Xiangrui Meng] add task size test to GD 28a8411 [Xiangrui Meng] add test for NaiveBayes 380778c [Xiangrui Meng] update KMeans test bccab92 [Xiangrui Meng] add task size test to LBFGS 02103ba [Xiangrui Meng] remove print e73d68e [Xiangrui Meng] update tests for k-means 174cb15 [Xiangrui Meng] use local-cluster for test with a small akka.frameSize 1928a5a [Xiangrui Meng] add test for KMeans task size e00c2da [Xiangrui Meng] use broadcast in GD, KMeans 010d076 [Xiangrui Meng] modify NaiveBayesModel and GLM to use broadcast --- .../mllib/classification/NaiveBayes.scala | 8 +- .../spark/mllib/clustering/KMeans.scala | 19 +++-- .../spark/mllib/clustering/KMeansModel.scala | 6 +- .../mllib/optimization/GradientDescent.scala | 6 +- .../spark/mllib/optimization/LBFGS.scala | 7 +- .../GeneralizedLinearAlgorithm.scala | 7 +- .../JavaLogisticRegressionSuite.java | 2 - .../LogisticRegressionSuite.scala | 18 ++++- .../classification/NaiveBayesSuite.scala | 20 ++++- .../spark/mllib/classification/SVMSuite.scala | 25 +++++-- .../spark/mllib/clustering/KMeansSuite.scala | 75 ++++++++++++------- .../linalg/distributed/RowMatrixSuite.scala | 29 ++++++- .../optimization/GradientDescentSuite.scala | 34 +++++++-- .../spark/mllib/optimization/LBFGSSuite.scala | 30 +++++++- .../spark/mllib/regression/LassoSuite.scala | 21 +++++- .../regression/LinearRegressionSuite.scala | 21 +++++- .../regression/RidgeRegressionSuite.scala | 23 +++++- .../mllib/util/LocalClusterSparkContext.scala | 42 +++++++++++ .../spark/mllib/util/LocalSparkContext.scala | 7 +- 19 files changed, 330 insertions(+), 70 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index b6e0c4a80e27b..6c7be0a4f1dcb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -54,7 +54,13 @@ class NaiveBayesModel private[mllib] ( } } - override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) + override def predict(testData: RDD[Vector]): RDD[Double] = { + val bcModel = testData.context.broadcast(this) + testData.mapPartitions { iter => + val model = bcModel.value + iter.map(model.predict) + } + } override def predict(testData: Vector): Double = { labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index de22fbb6ffc10..db425d866bbad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -165,18 +165,21 @@ class KMeans private ( val activeCenters = activeRuns.map(r => centers(r)).toArray val costAccums = activeRuns.map(_ => sc.accumulator(0.0)) + val bcActiveCenters = sc.broadcast(activeCenters) + // Find the sum and count of points mapping to each center val totalContribs = data.mapPartitions { points => - val runs = activeCenters.length - val k = activeCenters(0).length - val dims = activeCenters(0)(0).vector.length + val thisActiveCenters = bcActiveCenters.value + val runs = thisActiveCenters.length + val k = thisActiveCenters(0).length + val dims = thisActiveCenters(0)(0).vector.length val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) val counts = Array.fill(runs, k)(0L) points.foreach { point => (0 until runs).foreach { i => - val (bestCenter, cost) = KMeans.findClosest(activeCenters(i), point) + val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) costAccums(i) += cost sums(i)(bestCenter) += point.vector counts(i)(bestCenter) += 1 @@ -264,16 +267,17 @@ class KMeans private ( // to their squared distance from that run's current centers var step = 0 while (step < initializationSteps) { + val bcCenters = data.context.broadcast(centers) val sumCosts = data.flatMap { point => (0 until runs).map { r => - (r, KMeans.pointCost(centers(r), point)) + (r, KMeans.pointCost(bcCenters.value(r), point)) } }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) points.flatMap { p => (0 until runs).filter { r => - rand.nextDouble() < 2.0 * KMeans.pointCost(centers(r), p) * k / sumCosts(r) + rand.nextDouble() < 2.0 * KMeans.pointCost(bcCenters.value(r), p) * k / sumCosts(r) }.map((_, p)) } }.collect() @@ -286,9 +290,10 @@ class KMeans private ( // Finally, we might have a set of more than k candidate centers for each run; weigh each // candidate by the number of points in the dataset mapping to it and run a local k-means++ // on the weighted centers to pick just k of them + val bcCenters = data.context.broadcast(centers) val weightMap = data.flatMap { p => (0 until runs).map { r => - ((r, KMeans.findClosest(centers(r), p)._1), 1.0) + ((r, KMeans.findClosest(bcCenters.value(r), p)._1), 1.0) } }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index fba21aefaaacd..5823cb6e52e7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -38,7 +38,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser /** Maps given points to their cluster indices. */ def predict(points: RDD[Vector]): RDD[Int] = { val centersWithNorm = clusterCentersWithNorm - points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) + val bcCentersWithNorm = points.context.broadcast(centersWithNorm) + points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))._1) } /** Maps given points to their cluster indices. */ @@ -51,7 +52,8 @@ class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Ser */ def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm - data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() + val bcCentersWithNorm = data.context.broadcast(centersWithNorm) + data.map(p => KMeans.pointCost(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))).sum() } private def clusterCentersWithNorm: Iterable[BreezeVectorWithNorm] = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 7030eeabe400a..9fd760bf78083 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -163,6 +163,7 @@ object GradientDescent extends Logging { // Initialize weights as a column vector var weights = Vectors.dense(initialWeights.toArray) + val n = weights.size /** * For the first iteration, the regVal will be initialized as sum of weight squares @@ -172,12 +173,13 @@ object GradientDescent extends Logging { weights, Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 for (i <- 1 to numIterations) { + val bcWeights = data.context.broadcast(weights) // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) - .aggregate((BDV.zeros[Double](weights.size), 0.0))( + .aggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = gradient.compute(features, label, weights, Vectors.fromBreeze(grad)) + val l = gradient.compute(features, label, bcWeights.value, Vectors.fromBreeze(grad)) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 7bbed9c8fdbef..179cd4a3f1625 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -195,13 +195,14 @@ object LBFGS extends Logging { override def calculate(weights: BDV[Double]) = { // Have a local copy to avoid the serialization of CostFun object which is not serializable. - val localData = data val localGradient = gradient + val n = weights.length + val bcWeights = data.context.broadcast(weights) - val (gradientSum, lossSum) = localData.aggregate((BDV.zeros[Double](weights.size), 0.0))( + val (gradientSum, lossSum) = data.aggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => val l = localGradient.compute( - features, label, Vectors.fromBreeze(weights), Vectors.fromBreeze(grad)) + features, label, Vectors.fromBreeze(bcWeights.value), Vectors.fromBreeze(grad)) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => 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 fe41863bce985..54854252d7477 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 @@ -56,9 +56,12 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. val localWeights = weights + val bcWeights = testData.context.broadcast(localWeights) val localIntercept = intercept - - testData.map(v => predictPoint(v, localWeights, localIntercept)) + testData.mapPartitions { iter => + val w = bcWeights.value + iter.map(v => predictPoint(v, w, localIntercept)) + } } /** diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java index faa675b59cd50..862221d48798a 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java @@ -92,8 +92,6 @@ public void runLRUsingStaticMethods() { testRDD.rdd(), 100, 1.0, 1.0); int numAccurate = validatePrediction(validationData, model); - System.out.println(numAccurate); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 44b757b6a1fb7..3f6ff859374c7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.Matchers import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object LogisticRegressionSuite { @@ -126,3 +126,19 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LogisticRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 516895d04222d..06cdd04f5fdae 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -23,7 +23,7 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object NaiveBayesSuite { @@ -96,3 +96,21 @@ class NaiveBayesSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map { i => + LabeledPoint(random.nextInt(2), Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + } + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = NaiveBayes.train(examples) + val predictions = model.predict(examples.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index 886c71dde3af7..65e5df58db4c7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.mllib.classification -import scala.util.Random import scala.collection.JavaConversions._ - -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite import org.apache.spark.SparkException -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object SVMSuite { @@ -193,3 +192,19 @@ class SVMSuite extends FunSuite with LocalSparkContext { new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } } + +class SVMClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = SVMWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 76a3bdf9b11c8..34bc4537a7b3a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.mllib.clustering +import scala.util.Random + import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class KMeansSuite extends FunSuite with LocalSparkContext { - import KMeans.{RANDOM, K_MEANS_PARALLEL} + import org.apache.spark.mllib.clustering.KMeans.{K_MEANS_PARALLEL, RANDOM} test("single cluster") { val data = sc.parallelize(Array( @@ -38,26 +40,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) model = KMeans.train( - data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) } @@ -100,26 +102,27 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.dense(1.0, 3.0, 4.0) - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.size === 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) } @@ -145,25 +148,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) - var model = KMeans.train(data, k=1, maxIterations=1) + var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=2) + model = KMeans.train(data, k = 1, maxIterations = 2) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=5) + model = KMeans.train(data, k = 1, maxIterations = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=5) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) assert(model.clusterCenters.head === center) - model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, + initializationMode = K_MEANS_PARALLEL) assert(model.clusterCenters.head === center) data.unpersist() @@ -183,15 +187,15 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // it will make at least five passes, and it will give non-zero probability to each // unselected point as long as it hasn't yet selected all of them - var model = KMeans.train(rdd, k=5, maxIterations=1) + var model = KMeans.train(rdd, k = 5, maxIterations = 1) assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Iterations of Lloyd's should not change the answer either - model = KMeans.train(rdd, k=5, maxIterations=10) + model = KMeans.train(rdd, k = 5, maxIterations = 10) assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Neither should more runs - model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) + model = KMeans.train(rdd, k = 5, maxIterations = 10, runs = 5) assert(Set(model.clusterCenters: _*) === Set(points: _*)) } @@ -220,3 +224,22 @@ class KMeansSuite extends FunSuite with LocalSparkContext { } } } + +class KMeansClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble))) + }.cache() + for (initMode <- Seq(KMeans.RANDOM, KMeans.K_MEANS_PARALLEL)) { + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = KMeans.train(points, 2, 2, 1, initMode) + val predictions = model.predict(points).collect() + val cost = model.computeCost(points) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index a961f89456a18..325b817980f68 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.linalg.distributed -import org.scalatest.FunSuite +import scala.util.Random import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class RowMatrixSuite extends FunSuite with LocalSparkContext { @@ -193,3 +194,27 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } } + +class RowMatrixClusterSuite extends FunSuite with LocalClusterSparkContext { + + var mat: RowMatrix = _ + + override def beforeAll() { + super.beforeAll() + val m = 4 + val n = 200000 + val rows = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => Vectors.dense(Array.fill(n)(random.nextDouble()))) + } + mat = new RowMatrix(rows) + } + + test("task size should be small in svd") { + val svd = mat.computeSVD(1, computeU = true) + } + + test("task size should be small in summarize") { + val summary = mat.computeColumnSummaryStatistics() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 951b4f7c6e6f4..dfb2eb7f0d14e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -17,15 +17,14 @@ package org.apache.spark.mllib.optimization -import scala.util.Random import scala.collection.JavaConversions._ +import scala.util.Random -import org.scalatest.FunSuite -import org.scalatest.Matchers +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} object GradientDescentSuite { @@ -46,7 +45,7 @@ object GradientDescentSuite { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) - val unifRand = new scala.util.Random(45) + val unifRand = new Random(45) val rLogis = (0 until nPoints).map { i => val u = unifRand.nextDouble() math.log(u) - math.log(1.0-u) @@ -144,3 +143,26 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with Matchers "should be initialWeightsWithIntercept.") } } + +class GradientDescentClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val (weights, loss) = GradientDescent.runMiniBatchSGD( + points, + new LogisticGradient, + new SquaredL2Updater, + 0.1, + 2, + 1.0, + 1.0, + Vectors.dense(new Array[Double](n))) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index fe7a9033cd5f4..ff414742e8393 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.optimization -import org.scalatest.FunSuite -import org.scalatest.Matchers +import scala.util.Random + +import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { @@ -230,3 +231,24 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { "The weight differences between LBFGS and GD should be within 2%.") } } + +class LBFGSClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small") { + val m = 10 + val n = 200000 + val examples = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble)))) + }.cache() + val lbfgs = new LBFGS(new LogisticGradient, new SquaredL2Updater) + .setNumCorrections(1) + .setConvergenceTol(1e-12) + .setMaxNumIterations(1) + .setRegParam(1.0) + val random = new Random(0) + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val weights = lbfgs.optimize(examples, Vectors.dense(Array.fill(n)(random.nextDouble))) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index bfa42959c8ead..7aa96421aed87 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class LassoSuite extends FunSuite with LocalSparkContext { @@ -113,3 +116,19 @@ class LassoSuite extends FunSuite with LocalSparkContext { validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } } + +class LassoClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LassoWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 7aaad7d7a3e39..4f89112b650c5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -17,10 +17,13 @@ package org.apache.spark.mllib.regression +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class LinearRegressionSuite extends FunSuite with LocalSparkContext { @@ -122,3 +125,19 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) } } + +class LinearRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = LinearRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 67768e17fbe6d..727bbd051ff15 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.regression -import org.scalatest.FunSuite +import scala.util.Random import org.jblas.DoubleMatrix +import org.scalatest.FunSuite -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LocalClusterSparkContext, LinearDataGenerator, + LocalSparkContext} class RidgeRegressionSuite extends FunSuite with LocalSparkContext { @@ -73,3 +76,19 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } } + +class RidgeRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { + + test("task size should be small in both training and prediction") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = RidgeRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala new file mode 100644 index 0000000000000..5e9101cdd3804 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala @@ -0,0 +1,42 @@ +/* + * 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.util + +import org.scalatest.{Suite, BeforeAndAfterAll} + +import org.apache.spark.{SparkConf, SparkContext} + +trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => + @transient var sc: SparkContext = _ + + override def beforeAll() { + val conf = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("test-cluster") + .set("spark.akka.frameSize", "1") // set to 1MB to detect direct serialization of data + sc = new SparkContext(conf) + super.beforeAll() + } + + override def afterAll() { + if (sc != null) { + sc.stop() + } + super.afterAll() + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala index 0d4868f3d9e42..7857d9e5ee5c4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala @@ -20,13 +20,16 @@ package org.apache.spark.mllib.util import org.scalatest.Suite import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkContext +import org.apache.spark.{SparkConf, SparkContext} trait LocalSparkContext extends BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ override def beforeAll() { - sc = new SparkContext("local", "test") + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + sc = new SparkContext(conf) super.beforeAll() } From 3a69c72e5cbe270b76f6ab6a84a2e334e87cce8c Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Sun, 27 Jul 2014 07:21:07 -0700 Subject: [PATCH 0090/1492] [SPARK-2679] [MLLib] Ser/De for Double Added a set of serializer/deserializer for Double in _common.py and PythonMLLibAPI in MLLib. Author: Doris Xin Closes #1581 from dorx/doubleSerDe and squashes the following commits: 86a85b3 [Doris Xin] Merge branch 'master' into doubleSerDe 2bfe7a4 [Doris Xin] Removed magic byte ad4d0d9 [Doris Xin] removed a space in unit a9020bc [Doris Xin] units passed 7dad9af [Doris Xin] WIP --- .../mllib/api/python/PythonMLLibAPI.scala | 23 +++++++++ .../api/python/PythonMLLibAPISuite.scala | 8 ++++ python/pyspark/mllib/_common.py | 48 +++++++++++++++++-- 3 files changed, 76 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index c44173793b39a..954621ee8b933 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -54,6 +54,13 @@ class PythonMLLibAPI extends Serializable { } } + private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { + require(bytes.length - offset == 8, "Wrong size byte array for Double") + val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) + bb.order(ByteOrder.nativeOrder()) + bb.getDouble + } + private def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { val packetLength = bytes.length - offset require(packetLength >= 5, "Byte array too short") @@ -89,6 +96,22 @@ class PythonMLLibAPI extends Serializable { Vectors.sparse(size, indices, values) } + /** + * Returns an 8-byte array for the input Double. + * + * Note: we currently do not use a magic byte for double for storage efficiency. + * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. + * The corresponding deserializer, deserializeDouble, needs to be modified as well if the + * serialization scheme changes. + */ + private[python] def serializeDouble(double: Double): Array[Byte] = { + val bytes = new Array[Byte](8) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.putDouble(double) + bytes + } + private def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { val len = doubles.length val bytes = new Array[Byte](5 + 8 * len) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 642843f90204c..d94cfa2fcec81 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -57,4 +57,12 @@ class PythonMLLibAPISuite extends FunSuite { assert(q.features === p.features) } } + + test("double serialization") { + for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue)) { + val bytes = py.serializeDouble(x) + val deser = py.deserializeDouble(bytes) + assert(x === deser) + } + } } diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 43b491a9716fc..8e3ad6b783b6c 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -72,9 +72,9 @@ # Python interpreter must agree on what endian the machine is. -DENSE_VECTOR_MAGIC = 1 +DENSE_VECTOR_MAGIC = 1 SPARSE_VECTOR_MAGIC = 2 -DENSE_MATRIX_MAGIC = 3 +DENSE_MATRIX_MAGIC = 3 LABELED_POINT_MAGIC = 4 @@ -97,8 +97,28 @@ def _deserialize_numpy_array(shape, ba, offset, dtype=float64): return ar.copy() +def _serialize_double(d): + """ + Serialize a double (float or numpy.float64) into a mutually understood format. + """ + if type(d) == float or type(d) == float64: + d = float64(d) + ba = bytearray(8) + _copyto(d, buffer=ba, offset=0, shape=[1], dtype=float64) + return ba + else: + raise TypeError("_serialize_double called on non-float input") + + def _serialize_double_vector(v): - """Serialize a double vector into a mutually understood format. + """ + Serialize a double vector into a mutually understood format. + + Note: we currently do not use a magic byte for double for storage + efficiency. This should be reconsidered when we add Ser/De for other + 8-byte types (e.g. Long), for safety. The corresponding deserializer, + _deserialize_double, needs to be modified as well if the serialization + scheme changes. >>> x = array([1,2,3]) >>> y = _deserialize_double_vector(_serialize_double_vector(x)) @@ -148,6 +168,28 @@ def _serialize_sparse_vector(v): return ba +def _deserialize_double(ba, offset=0): + """Deserialize a double from a mutually understood format. + + >>> import sys + >>> _deserialize_double(_serialize_double(123.0)) == 123.0 + True + >>> _deserialize_double(_serialize_double(float64(0.0))) == 0.0 + True + >>> x = sys.float_info.max + >>> _deserialize_double(_serialize_double(sys.float_info.max)) == x + True + >>> y = float64(sys.float_info.max) + >>> _deserialize_double(_serialize_double(sys.float_info.max)) == y + True + """ + if type(ba) != bytearray: + raise TypeError("_deserialize_double called on a %s; wanted bytearray" % type(ba)) + if len(ba) - offset != 8: + raise TypeError("_deserialize_double called on a %d-byte array; wanted 8 bytes." % nb) + return struct.unpack("d", ba[offset:])[0] + + def _deserialize_double_vector(ba, offset=0): """Deserialize a double vector from a mutually understood format. From 985705301e5e55de14b00ad8ce3143e91aae185d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sun, 27 Jul 2014 11:20:20 -0700 Subject: [PATCH 0091/1492] SPARK-2684: Update ExternalAppendOnlyMap to take an iterator as input This will decrease object allocation from the "update" closure used in map.changeValue. Author: Matei Zaharia Closes #1607 from mateiz/spark-2684 and squashes the following commits: b7d89e6 [Matei Zaharia] Add insertAll for Iterables too, and fix some code style 561fc97 [Matei Zaharia] Update ExternalAppendOnlyMap to take an iterator as input --- .../scala/org/apache/spark/Aggregator.scala | 5 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 7 +- .../collection/ExternalAppendOnlyMap.scala | 77 +++++++++++++------ .../ExternalAppendOnlyMapSuite.scala | 17 ++-- 4 files changed, 64 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 1d640579efe77..ff0ca11749d42 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -55,10 +55,7 @@ case class Aggregator[K, V, C] ( combiners.iterator } else { val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) - while (iter.hasNext) { - val pair = iter.next() - combiners.insert(pair._1, pair._2) - } + combiners.insertAll(iter) // TODO: Make this non optional in a future release Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled) Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 7d96089e52ab9..6388ef82cc5db 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -154,11 +154,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } else { val map = createExternalMap(numRdds) - rddIterators.foreach { case (it, depNum) => - while (it.hasNext) { - val kv = it.next() - map.insert(kv._1, new CoGroupValue(kv._2, depNum)) - } + for ((it, depNum) <- rddIterators) { + map.insertAll(it.map(pair => (pair._1, new CoGroupValue(pair._2, depNum)))) } context.taskMetrics.memoryBytesSpilled = map.memoryBytesSpilled context.taskMetrics.diskBytesSpilled = map.diskBytesSpilled diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index c22bb8d9c60a9..6f263c39d1435 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -110,42 +110,69 @@ class ExternalAppendOnlyMap[K, V, C]( /** * Insert the given key and value into the map. + */ + def insert(key: K, value: V): Unit = { + insertAll(Iterator((key, value))) + } + + /** + * Insert the given iterator of keys and values into the map. * - * If the underlying map is about to grow, check if the global pool of shuffle memory has + * When the underlying map needs to grow, check if the global pool of shuffle memory has * enough room for this to happen. If so, allocate the memory required to grow the map; * otherwise, spill the in-memory map to disk. * * The shuffle memory usage of the first trackMemoryThreshold entries is not tracked. */ - def insert(key: K, value: V) { + def insertAll(entries: Iterator[Product2[K, V]]): Unit = { + // An update function for the map that we reuse across entries to avoid allocating + // a new closure each time + var curEntry: Product2[K, V] = null val update: (Boolean, C) => C = (hadVal, oldVal) => { - if (hadVal) mergeValue(oldVal, value) else createCombiner(value) + if (hadVal) mergeValue(oldVal, curEntry._2) else createCombiner(curEntry._2) } - if (numPairsInMemory > trackMemoryThreshold && currentMap.atGrowThreshold) { - val mapSize = currentMap.estimateSize() - var shouldSpill = false - val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap - - // Atomically check whether there is sufficient memory in the global pool for - // this map to grow and, if possible, allocate the required amount - shuffleMemoryMap.synchronized { - val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) - val availableMemory = maxMemoryThreshold - - (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) - - // Assume map growth factor is 2x - shouldSpill = availableMemory < mapSize * 2 - if (!shouldSpill) { - shuffleMemoryMap(threadId) = mapSize * 2 + + while (entries.hasNext) { + curEntry = entries.next() + if (numPairsInMemory > trackMemoryThreshold && currentMap.atGrowThreshold) { + val mapSize = currentMap.estimateSize() + var shouldSpill = false + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + + // Atomically check whether there is sufficient memory in the global pool for + // this map to grow and, if possible, allocate the required amount + shuffleMemoryMap.synchronized { + val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) + val availableMemory = maxMemoryThreshold - + (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) + + // Assume map growth factor is 2x + shouldSpill = availableMemory < mapSize * 2 + if (!shouldSpill) { + shuffleMemoryMap(threadId) = mapSize * 2 + } + } + // Do not synchronize spills + if (shouldSpill) { + spill(mapSize) } } - // Do not synchronize spills - if (shouldSpill) { - spill(mapSize) - } + currentMap.changeValue(curEntry._1, update) + numPairsInMemory += 1 } - currentMap.changeValue(key, update) - numPairsInMemory += 1 + } + + /** + * Insert the given iterable of keys and values into the map. + * + * When the underlying map needs to grow, check if the global pool of shuffle memory has + * enough room for this to happen. If so, allocate the memory required to grow the map; + * otherwise, spill the in-memory map to disk. + * + * The shuffle memory usage of the first trackMemoryThreshold entries is not tracked. + */ + def insertAll(entries: Iterable[Product2[K, V]]): Unit = { + insertAll(entries.iterator) } /** diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 428822949c085..0b7ad184a46d2 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -63,12 +63,13 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) - map.insert(1, 10) - map.insert(2, 20) - map.insert(3, 30) - map.insert(1, 100) - map.insert(2, 200) - map.insert(1, 1000) + map.insertAll(Seq( + (1, 10), + (2, 20), + (3, 30), + (1, 100), + (2, 200), + (1, 1000))) val it = map.iterator assert(it.hasNext) val result = it.toSet[(Int, ArrayBuffer[Int])].map(kv => (kv._1, kv._2.toSet)) @@ -282,7 +283,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(w1.hashCode === w2.hashCode) } - (1 to 100000).map(_.toString).foreach { i => map.insert(i, i) } + map.insertAll((1 to 100000).iterator.map(_.toString).map(i => (i, i))) collisionPairs.foreach { case (w1, w2) => map.insert(w1, w2) map.insert(w2, w1) @@ -355,7 +356,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]]( createCombiner, mergeValue, mergeCombiners) - (1 to 100000).foreach { i => map.insert(i, i) } + map.insertAll((1 to 100000).iterator.map(i => (i, i))) map.insert(null.asInstanceOf[Int], 1) map.insert(1, null.asInstanceOf[Int]) map.insert(null.asInstanceOf[Int], null.asInstanceOf[Int]) From 2bbf235376f40a4b95d7e6e42e1bed893c124ecb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 27 Jul 2014 12:35:21 -0700 Subject: [PATCH 0092/1492] [SPARK-2705][CORE] Fixed stage description in stage info page Stage description should be a `String`, but was changed to an `Option[String]` by mistake: ![stage-desc-small](https://cloud.githubusercontent.com/assets/230655/3655611/f6d0b0f6-117b-11e4-83ed-71000dcd5009.png) Author: Cheng Lian Closes #1524 from liancheng/fix-stage-desc and squashes the following commits: 3c69327 [Cheng Lian] Fixed stage description object type in Web UI stage table --- .../org/apache/spark/ui/jobs/StageTable.scala | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index f8b308c981548..3dcfaf76e4aba 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -119,14 +119,14 @@ private[ui] class StageTableBase( } - val stageDataOption = listener.stageIdToData.get(s.stageId) - // Too many nested map/flatMaps with options are just annoying to read. Do this imperatively. - if (stageDataOption.isDefined && stageDataOption.get.description.isDefined) { - val desc = stageDataOption.get.description -
{desc}
{killLink} {nameLink} {details}
- } else { -
{killLink} {nameLink} {details}
+ val stageDesc = for { + stageData <- listener.stageIdToData.get(s.stageId) + desc <- stageData.description + } yield { +
{desc}
} + +
{stageDesc.getOrElse("")} {killLink} {nameLink} {details}
} protected def stageRow(s: StageInfo): Seq[Node] = { From f6ff2a61d00d12481bfb211ae13d6992daacdcc2 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 27 Jul 2014 13:03:38 -0700 Subject: [PATCH 0093/1492] [SPARK-2410][SQL] Merging Hive Thrift/JDBC server (This is a replacement of #1399, trying to fix potential `HiveThriftServer2` port collision between parallel builds. Please refer to [these comments](https://github.com/apache/spark/pull/1399#issuecomment-50212572) for details.) JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) Merging the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc). Thanks chenghao-intel for his initial contribution of the Spark SQL CLI. Author: Cheng Lian Closes #1600 from liancheng/jdbc and squashes the following commits: ac4618b [Cheng Lian] Uses random port for HiveThriftServer2 to avoid collision with parallel builds 090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR 21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd] 199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver 1083e9d [Cheng Lian] Fixed failed test suites 7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic 9cc0f06 [Cheng Lian] Starts beeline with spark-submit cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile 061880f [Cheng Lian] Addressed all comments by @pwendell 7755062 [Cheng Lian] Adapts test suites to spark-submit settings 40bafef [Cheng Lian] Fixed more license header issues e214aab [Cheng Lian] Added missing license headers b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft 3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit 61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit 2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server --- .gitignore | 1 + assembly/pom.xml | 10 + bagel/pom.xml | 2 +- bin/beeline | 45 +++ bin/compute-classpath.sh | 1 + bin/spark-shell | 4 +- bin/spark-shell.cmd | 2 +- bin/spark-sql | 36 ++ core/pom.xml | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 14 +- .../spark/deploy/SparkSubmitArguments.scala | 5 +- dev/create-release/create-release.sh | 10 +- dev/run-tests | 2 +- dev/scalastyle | 2 +- docs/sql-programming-guide.md | 201 +++++++++- examples/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 7 +- project/SparkBuild.scala | 14 +- sbin/start-thriftserver.sh | 36 ++ sql/catalyst/pom.xml | 2 +- .../sql/catalyst/plans/logical/commands.scala | 3 +- sql/core/pom.xml | 2 +- .../scala/org/apache/spark/sql/SQLConf.scala | 20 +- .../apache/spark/sql/execution/commands.scala | 42 ++- .../org/apache/spark/sql/SQLConfSuite.scala | 13 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- sql/hive-thriftserver/pom.xml | 82 +++++ .../hive/thriftserver/HiveThriftServer2.scala | 97 +++++ .../hive/thriftserver/ReflectionUtils.scala | 58 +++ .../hive/thriftserver/SparkSQLCLIDriver.scala | 344 ++++++++++++++++++ .../thriftserver/SparkSQLCLIService.scala | 74 ++++ .../hive/thriftserver/SparkSQLDriver.scala | 93 +++++ .../sql/hive/thriftserver/SparkSQLEnv.scala | 58 +++ .../thriftserver/SparkSQLSessionManager.scala | 49 +++ .../server/SparkSQLOperationManager.scala | 151 ++++++++ .../test/resources/data/files/small_kv.txt | 5 + .../sql/hive/thriftserver/CliSuite.scala | 57 +++ .../thriftserver/HiveThriftServer2Suite.scala | 135 +++++++ .../sql/hive/thriftserver/TestUtils.scala | 108 ++++++ sql/hive/pom.xml | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 50 ++- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 54 files changed, 1781 insertions(+), 96 deletions(-) create mode 100755 bin/beeline create mode 100755 bin/spark-sql create mode 100755 sbin/start-thriftserver.sh create mode 100644 sql/hive-thriftserver/pom.xml create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala create mode 100755 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala create mode 100644 sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala diff --git a/.gitignore b/.gitignore index 061c8946d23c1..5b56a67c883e6 100644 --- a/.gitignore +++ b/.gitignore @@ -57,3 +57,4 @@ metastore_db/ metastore/ warehouse/ TempStatsStore/ +sql/hive-thriftserver/test_warehouses diff --git a/assembly/pom.xml b/assembly/pom.xml index 567a8dd2a0d94..703f15925bc44 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -165,6 +165,16 @@
+ + hive-thriftserver + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl diff --git a/bagel/pom.xml b/bagel/pom.xml index 90c4b095bb611..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-bagel_2.10 - bagel + bagel jar Spark Project Bagel diff --git a/bin/beeline b/bin/beeline new file mode 100755 index 0000000000000..09fe366c609fa --- /dev/null +++ b/bin/beeline @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +# Find the java binary +if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" +else + if [ `command -v java` ]; then + RUNNER="java" + else + echo "JAVA_HOME is not set" >&2 + exit 1 + fi +fi + +# Compute classpath using external script +classpath_output=$($FWDIR/bin/compute-classpath.sh) +if [[ "$?" != "0" ]]; then + echo "$classpath_output" + exit 1 +else + CLASSPATH=$classpath_output +fi + +CLASS="org.apache.hive.beeline.BeeLine" +exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index e81e8c060cb98..16b794a1592e8 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -52,6 +52,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" fi diff --git a/bin/spark-shell b/bin/spark-shell index 850e9507ec38f..756c8179d12b6 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 4b9708a8c03f3..b56d69801171c 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell --class org.apache.spark.repl.Main %* diff --git a/bin/spark-sql b/bin/spark-sql new file mode 100755 index 0000000000000..bba7f897b19bc --- /dev/null +++ b/bin/spark-sql @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# Shell script for starting the Spark SQL CLI + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/spark-sql [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + +CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/core/pom.xml b/core/pom.xml index 1054cec4d77bb..a24743495b0e1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-core_2.10 - core + core jar Spark Project Core diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3b5642b6caa36..c9cec33ebaa66 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -46,6 +46,10 @@ object SparkSubmit { private val CLUSTER = 2 private val ALL_DEPLOY_MODES = CLIENT | CLUSTER + // A special jar name that indicates the class being run is inside of Spark itself, and therefore + // no user jar is needed. + private val SPARK_INTERNAL = "spark-internal" + // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" @@ -257,7 +261,9 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (clusterManager == YARN && deployMode == CLUSTER) { childMainClass = "org.apache.spark.deploy.yarn.Client" - childArgs += ("--jar", args.primaryResource) + if (args.primaryResource != SPARK_INTERNAL) { + childArgs += ("--jar", args.primaryResource) + } childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } @@ -332,7 +338,7 @@ object SparkSubmit { * Return whether the given primary resource represents a user jar. */ private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) + !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) } /** @@ -349,6 +355,10 @@ object SparkSubmit { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } + private[spark] def isInternal(primaryResource: String): Boolean = { + primaryResource == SPARK_INTERNAL + } + /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 3ab67a43a3b55..01d0ae541a66b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -204,8 +204,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { - // Delineates parsing of Spark options from parsing of user options. var inSparkOpts = true + + // Delineates parsing of Spark options from parsing of user options. parse(opts) def parse(opts: Seq[String]): Unit = opts match { @@ -318,7 +319,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit(errMessage) case v => primaryResource = - if (!SparkSubmit.isShell(v)) { + if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) { Utils.resolveURI(v).toString } else { v diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 38830103d1e8d..33de24d1ae6d7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -53,7 +53,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare @@ -61,7 +61,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. @@ -111,10 +111,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" +make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" make_binary_release "hadoop2" \ - "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" # Copy data echo "Copying release tarballs" diff --git a/dev/run-tests b/dev/run-tests index 51e4def0f835a..98ec969dc1b37 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -65,7 +65,7 @@ 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" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive" sbt/sbt clean package \ + echo -e "q\n" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive -Phive-thriftserver" 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 | \ diff --git a/dev/scalastyle b/dev/scalastyle index a02d06912f238..d9f2b91a3a091 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 38728534a46e0..156e0aebdebe6 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -136,7 +136,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD // Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) @@ -548,7 +548,6 @@ results = hiveContext.hql("FROM src SELECT key, value").collect() - # Writing Language-Integrated Relational Queries **Language-Integrated queries are currently only supported in Scala.** @@ -573,4 +572,200 @@ prefixed with a tick (`'`). Implicit conversions turn these symbols into expres evaluated by the SQL execution engine. A full list of the functions supported can be found in the [ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). - \ No newline at end of file + + +## Running the Thrift JDBC server + +The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] +(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test +the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive +you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` +for maven). + +To start the JDBC server, run the following in the Spark directory: + + ./sbin/start-thriftserver.sh + +The default port the server listens on is 10000. To listen on customized host and port, please set +the `HIVE_SERVER2_THRIFT_PORT` and `HIVE_SERVER2_THRIFT_BIND_HOST` environment variables. You may +run `./sbin/start-thriftserver.sh --help` for a complete list of all available options. Now you can +use beeline to test the Thrift JDBC server: + + ./bin/beeline + +Connect to the JDBC server in beeline with: + + beeline> !connect jdbc:hive2://localhost:10000 + +Beeline will ask you for a username and password. In non-secure mode, simply enter the username on +your machine and a blank password. For secure mode, please follow the instructions given in the +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. + +You may also use the beeline script comes with Hive. + +### Migration Guide for Shark Users + +#### Reducer number + +In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark +SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value +is 200. Users may customize this property via `SET`: + +``` +SET spark.sql.shuffle.partitions=10; +SELECT page, count(*) c FROM logs_last_month_cached +GROUP BY page ORDER BY c DESC LIMIT 10; +``` + +You may also put this property in `hive-site.xml` to override the default value. + +For now, the `mapred.reduce.tasks` property is still recognized, and is converted to +`spark.sql.shuffle.partitions` automatically. + +#### Caching + +The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no +longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to +let user control table caching explicitly: + +``` +CACHE TABLE logs_last_month; +UNCACHE TABLE logs_last_month; +``` + +**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", +but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be +cached, you may simply count the table immediately after executing `CACHE TABLE`: + +``` +CACHE TABLE logs_last_month; +SELECT COUNT(1) FROM logs_last_month; +``` + +Several caching related features are not supported yet: + +* User defined partition level cache eviction policy +* RDD reloading +* In-memory cache write through policy + +### Compatibility with Apache Hive + +#### Deploying in Exising Hive Warehouses + +Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +#### Supported Hive Features + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathemtatical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDe's) +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +#### Unsupported Hive Functionality + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + +**Esoteric Hive Features** + +* Tables with partitions using different input formats: In Spark SQL, all table partitions need to + have the same input format. +* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions + (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. +* `UNIONTYPE` +* Unique join +* Single query multi insert +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +not necessary due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block level bitmap indexes and virtual columns (used to build indexes) +* Automatically convert a join to map join: For joining a large table with multiple small tables, + Hive automatically converts the join into a map join. We are adding this auto conversion in the + next release. +* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you + need to control the degree of parallelism post-shuffle using "SET + spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the + next release. +* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available +options. diff --git a/examples/pom.xml b/examples/pom.xml index bd1c387c2eb91..c4ed0f5a6a02b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-examples_2.10 - examples + examples jar Spark Project Examples diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 61a6aff543aed..874b8a7959bb6 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-flume_2.10 - streaming-flume + streaming-flume jar Spark Project External Flume diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4762c50685a93..25a5c0a4d7d77 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-kafka_2.10 - streaming-kafka + streaming-kafka jar Spark Project External Kafka diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 32c530e600ce0..f31ed655f6779 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-mqtt_2.10 - streaming-mqtt + streaming-mqtt jar Spark Project External MQTT diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 637adb0f00da0..56bb24c2a072e 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-twitter_2.10 - streaming-twitter + streaming-twitter jar Spark Project External Twitter diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index e4d758a04a4cd..54b0242c54e78 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-zeromq_2.10 - streaming-zeromq + streaming-zeromq jar Spark Project External ZeroMQ diff --git a/graphx/pom.xml b/graphx/pom.xml index 7e3bcf29dcfbc..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-graphx_2.10 - graphx + graphx jar Spark Project GraphX diff --git a/mllib/pom.xml b/mllib/pom.xml index 92b07e2357db1..f27cf520dc9fa 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-mllib_2.10 - mllib + mllib jar Spark Project ML Library diff --git a/pom.xml b/pom.xml index 4e2d64a833640..3e9d388180d8e 100644 --- a/pom.xml +++ b/pom.xml @@ -95,6 +95,7 @@ sql/catalyst sql/core sql/hive + sql/hive-thriftserver repl assembly external/twitter @@ -252,9 +253,9 @@ 3.3.2 - commons-codec - commons-codec - 1.5 + commons-codec + commons-codec + 1.5 com.google.code.findbugs diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 62576f84dd031..1629bc2cba8ba 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,11 +30,11 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, mllib, repl, spark, sql, streaming, - streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = - Seq("bagel", "catalyst", "core", "graphx", "hive", "mllib", "repl", "spark", "sql", - "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", - "streaming-zeromq").map(ProjectRef(buildLocation, _)) + val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, + streaming, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = + Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", + "spark", "sql", "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", + "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") @@ -100,7 +100,7 @@ object SparkBuild extends PomBuild { Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) - case _ => + case _ => } override val userPropertiesMap = System.getProperties.toMap @@ -158,7 +158,7 @@ object SparkBuild extends PomBuild { /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl).exists(x => x == y)). + allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl).contains(x)). foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh new file mode 100755 index 0000000000000..8398e6f19b511 --- /dev/null +++ b/sbin/start-thriftserver.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# Shell script for starting the Spark SQL Thrift server + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/start-thriftserver [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + +CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" +exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6decde3fcd62d..531bfddbf237b 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -32,7 +32,7 @@ Spark Project Catalyst http://spark.apache.org/ - catalyst + catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 1d5f033f0d274..a357c6ffb8977 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -43,8 +43,7 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(0, AttributeReference("key", StringType, nullable = false)()), - BoundReference(1, AttributeReference("value", StringType, nullable = false)())) + BoundReference(1, AttributeReference("", StringType, nullable = false)())) } /** diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c309c43804d97..3a038a2db6173 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -32,7 +32,7 @@ Spark Project SQL http://spark.apache.org/ - sql + sql diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 2b787e14f3f15..41920c00b5a2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -30,12 +30,13 @@ import scala.collection.JavaConverters._ * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { + import SQLConf._ /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? /** Number of partitions to use for shuffle operators. */ - private[spark] def numShufflePartitions: Int = get("spark.sql.shuffle.partitions", "200").toInt + private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to @@ -43,11 +44,10 @@ trait SQLConf { * effectively disables auto conversion. * Hive setting: hive.auto.convert.join.noconditionaltask.size. */ - private[spark] def autoConvertJoinSize: Int = - get("spark.sql.auto.convert.join.size", "10000").toInt + private[spark] def autoConvertJoinSize: Int = get(AUTO_CONVERT_JOIN_SIZE, "10000").toInt /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get("spark.sql.join.broadcastTables", "") + private[spark] def joinBroadcastTables: String = get(JOIN_BROADCAST_TABLES, "") /** ********************** SQLConf functionality methods ************ */ @@ -61,7 +61,7 @@ trait SQLConf { def set(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for ${key}") + require(value != null, s"value cannot be null for $key") settings.put(key, value) } @@ -90,3 +90,13 @@ trait SQLConf { } } + +object SQLConf { + val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" + val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" + val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" + + object Deprecated { + val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 98d2f89c8ae71..9293239131d52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.execution +import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SQLConf, SQLContext} trait Command { /** @@ -44,28 +45,53 @@ trait Command { case class SetCommand( key: Option[String], value: Option[String], output: Seq[Attribute])( @transient context: SQLContext) - extends LeafNode with Command { + extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => - context.set(k, v) - Array(k -> v) + if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") + context.set(SQLConf.SHUFFLE_PARTITIONS, v) + Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") + } else { + context.set(k, v) + Array(s"$k=$v") + } // Query the value bound to key k. case (Some(k), _) => - Array(k -> context.getOption(k).getOrElse("")) + // TODO (lian) This is just a workaround to make the Simba ODBC driver work. + // Should remove this once we get the ODBC driver updated. + if (k == "-v") { + val hiveJars = Seq( + "hive-exec-0.12.0.jar", + "hive-service-0.12.0.jar", + "hive-common-0.12.0.jar", + "hive-hwi-0.12.0.jar", + "hive-0.12.0.jar").mkString(":") + + Array( + "system:java.class.path=" + hiveJars, + "system:sun.java.command=shark.SharkServer2") + } + else { + Array(s"$k=${context.getOption(k).getOrElse("")}") + } // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => - context.getAll + context.getAll.map { case (k, v) => + s"$k=$v" + } case _ => throw new IllegalArgumentException() } def execute(): RDD[Row] = { - val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) } + val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } context.sparkContext.parallelize(rows, 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 08293f7f0ca30..1a58d73d9e7f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -54,10 +54,10 @@ class SQLConfSuite extends QueryTest { assert(get(testKey, testVal + "_") == testVal) assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - sql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - sql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + sql("set some.property=20") + assert(get("some.property", "0") == "20") + sql("set some.property = 40") + assert(get("some.property", "0") == "40") val key = "spark.sql.key" val vs = "val0,val_1,val2.3,my_table" @@ -70,4 +70,9 @@ class SQLConfSuite extends QueryTest { clear() } + test("deprecated property") { + clear() + sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") + assert(get(SQLConf.SHUFFLE_PARTITIONS) == "10") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6736189c96d4b..de9e8aa4f62ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -424,25 +424,25 @@ class SQLQuerySuite extends QueryTest { sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - Seq(Seq(testKey, testVal)) + Seq(Seq(s"$testKey=$testVal")) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), Seq( - Seq(testKey, testVal), - Seq(testKey + testKey, testVal + testVal)) + Seq(s"$testKey=$testVal"), + Seq(s"${testKey + testKey}=${testVal + testVal}")) ) // "set key" checkAnswer( sql(s"SET $testKey"), - Seq(Seq(testKey, testVal)) + Seq(Seq(s"$testKey=$testVal")) ) checkAnswer( sql(s"SET $nonexistentKey"), - Seq(Seq(nonexistentKey, "")) + Seq(Seq(s"$nonexistentKey=")) ) clear() } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml new file mode 100644 index 0000000000000..7fac90fdc596d --- /dev/null +++ b/sql/hive-thriftserver/pom.xml @@ -0,0 +1,82 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive-thriftserver_2.10 + jar + Spark Project Hive + http://spark.apache.org/ + + hive-thriftserver + + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + org.spark-project.hive + hive-cli + ${hive.version} + + + org.spark-project.hive + hive-jdbc + ${hive.version} + + + org.spark-project.hive + hive-beeline + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala new file mode 100644 index 0000000000000..ddbc2a79fb512 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService +import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a + * `HiveThriftServer2` thrift server. + */ +private[hive] object HiveThriftServer2 extends Logging { + var LOG = LogFactory.getLog(classOf[HiveServer2]) + + def main(args: Array[String]) { + val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") + + if (!optionsProcessor.process(args)) { + logger.warn("Error starting HiveThriftServer2 with given arguments") + System.exit(-1) + } + + val ss = new SessionState(new HiveConf(classOf[SessionState])) + + // Set all properties specified via command line. + val hiveConf: HiveConf = ss.getConf + hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => + logger.debug(s"HiveConf var: $k=$v") + } + + SessionState.start(ss) + + logger.info("Starting SparkContext") + SparkSQLEnv.init() + SessionState.start(ss) + + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.sparkContext.stop() + } + } + ) + + try { + val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) + server.init(hiveConf) + server.start() + logger.info("HiveThriftServer2 started") + } catch { + case e: Exception => + logger.error("Error starting HiveThriftServer2", e) + System.exit(-1) + } + } +} + +private[hive] class HiveThriftServer2(hiveContext: HiveContext) + extends HiveServer2 + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + val sparkSqlCliService = new SparkSQLCLIService(hiveContext) + setSuperField(this, "cliService", sparkSqlCliService) + addService(sparkSqlCliService) + + val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala new file mode 100644 index 0000000000000..599294dfbb7d7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +private[hive] object ReflectionUtils { + def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { + setAncestorField(obj, 1, fieldName, fieldValue) + } + + def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { + val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.set(obj, fieldValue) + } + + def getSuperField[T](obj: AnyRef, fieldName: String): T = { + getAncestorField[T](obj, 1, fieldName) + } + + def getAncestorField[T](clazz: Object, level: Int, fieldName: String): T = { + val ancestor = Iterator.iterate[Class[_]](clazz.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.get(clazz).asInstanceOf[T] + } + + def invokeStatic(clazz: Class[_], methodName: String, args: (Class[_], AnyRef)*): AnyRef = { + invoke(clazz, null, methodName, args: _*) + } + + def invoke( + clazz: Class[_], + obj: AnyRef, + methodName: String, + args: (Class[_], AnyRef)*): AnyRef = { + + val (types, values) = args.unzip + val method = clazz.getDeclaredMethod(methodName, types: _*) + method.setAccessible(true) + method.invoke(obj, values.toSeq: _*) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala new file mode 100755 index 0000000000000..27268ecb923e9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io._ +import java.util.{ArrayList => JArrayList} + +import jline.{ConsoleReader, History} +import org.apache.commons.lang.StringUtils +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException +import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.thrift.transport.TSocket + +import org.apache.spark.sql.Logging + +private[hive] object SparkSQLCLIDriver { + private var prompt = "spark-sql" + private var continuedPrompt = "".padTo(prompt.length, ' ') + private var transport:TSocket = _ + + installSignalHandler() + + /** + * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), + * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while + * a command is being processed by the current thread. + */ + def installSignalHandler() { + HiveInterruptUtils.add(new HiveInterruptCallback { + override def interrupt() { + // Handle remote execution mode + if (SparkSQLEnv.sparkContext != null) { + SparkSQLEnv.sparkContext.cancelAllJobs() + } else { + if (transport != null) { + // Force closing of TCP connection upon session termination + transport.getSocket.close() + } + } + } + }) + } + + def main(args: Array[String]) { + val oproc = new OptionsProcessor() + if (!oproc.process_stage1(args)) { + System.exit(1) + } + + // NOTE: It is critical to do this here so that log4j is reinitialized + // before any of the other core hive classes are loaded + var logInitFailed = false + var logInitDetailMessage: String = null + try { + logInitDetailMessage = LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => + logInitFailed = true + logInitDetailMessage = e.getMessage + } + + val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) + + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + if (!oproc.process_stage2(sessionState)) { + System.exit(2) + } + + if (!sessionState.getIsSilent) { + if (logInitFailed) System.err.println(logInitDetailMessage) + else SessionState.getConsole.printInfo(logInitDetailMessage) + } + + // Set all properties specified via command line. + val conf: HiveConf = sessionState.getConf + sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => + conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + sessionState.getOverriddenConfigurations.put( + item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + } + + SessionState.start(sessionState) + + // Clean up after we exit + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.stop() + } + } + ) + + // "-h" option has been passed, so connect to Hive thrift server. + if (sessionState.getHost != null) { + sessionState.connect() + if (sessionState.isRemoteMode) { + prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt + continuedPrompt = "".padTo(prompt.length, ' ') + } + } + + if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { + // Hadoop-20 and above - we need to augment classpath using hiveconf + // components. + // See also: code in ExecDriver.java + var loader = conf.getClassLoader + val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) + if (StringUtils.isNotBlank(auxJars)) { + loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")) + } + conf.setClassLoader(loader) + Thread.currentThread().setContextClassLoader(loader) + } + + val cli = new SparkSQLCLIDriver + cli.setHiveVariables(oproc.getHiveVariables) + + // TODO work around for set the log output to console, because the HiveContext + // will set the output into an invalid buffer. + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + // Execute -i init files (always in silent mode) + cli.processInitFiles(sessionState) + + if (sessionState.execString != null) { + System.exit(cli.processLine(sessionState.execString)) + } + + try { + if (sessionState.fileName != null) { + System.exit(cli.processFile(sessionState.fileName)) + } + } catch { + case e: FileNotFoundException => + System.err.println(s"Could not open input file for reading. (${e.getMessage})") + System.exit(3) + } + + val reader = new ConsoleReader() + reader.setBellEnabled(false) + // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) + CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e)) + + val historyDirectory = System.getProperty("user.home") + + try { + if (new File(historyDirectory).exists()) { + val historyFile = historyDirectory + File.separator + ".hivehistory" + reader.setHistory(new History(new File(historyFile))) + } else { + System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + + " does not exist. History will not be available during this session.") + } + } catch { + case e: Exception => + System.err.println("WARNING: Encountered an error while trying to initialize Hive's " + + "history file. History will not be available during this session.") + System.err.println(e.getMessage) + } + + val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") + clientTransportTSocketField.setAccessible(true) + + transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket] + + var ret = 0 + var prefix = "" + val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", + classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) + + def promptWithCurrentDB = s"$prompt$currentDB" + def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( + classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) + + var currentPrompt = promptWithCurrentDB + var line = reader.readLine(currentPrompt + "> ") + + while (line != null) { + if (prefix.nonEmpty) { + prefix += '\n' + } + + if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { + line = prefix + line + ret = cli.processLine(line, true) + prefix = "" + currentPrompt = promptWithCurrentDB + } else { + prefix = prefix + line + currentPrompt = continuedPromptWithDBSpaces + } + + line = reader.readLine(currentPrompt + "> ") + } + + sessionState.close() + + System.exit(ret) + } +} + +private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { + private val sessionState = SessionState.get().asInstanceOf[CliSessionState] + + private val LOG = LogFactory.getLog("CliDriver") + + private val console = new SessionState.LogHelper(LOG) + + private val conf: Configuration = + if (sessionState != null) sessionState.getConf else new Configuration() + + // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver + // because the Hive unit tests do not go through the main() code path. + if (!sessionState.isRemoteMode) { + SparkSQLEnv.init() + } + + override def processCmd(cmd: String): Int = { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + if (cmd_trimmed.toLowerCase.equals("quit") || + cmd_trimmed.toLowerCase.equals("exit") || + tokens(0).equalsIgnoreCase("source") || + cmd_trimmed.startsWith("!") || + tokens(0).toLowerCase.equals("list") || + sessionState.isRemoteMode) { + val start = System.currentTimeMillis() + super.processCmd(cmd) + val end = System.currentTimeMillis() + val timeTaken: Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds") + 0 + } else { + var ret = 0 + val hconf = conf.asInstanceOf[HiveConf] + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) + + if (proc != null) { + if (proc.isInstanceOf[Driver]) { + val driver = new SparkSQLDriver + + driver.init() + val out = sessionState.out + val start:Long = System.currentTimeMillis() + if (sessionState.getIsVerbose) { + out.println(cmd) + } + + ret = driver.run(cmd).getResponseCode + if (ret != 0) { + driver.close() + return ret + } + + val res = new JArrayList[String]() + + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { + // Print the column names. + Option(driver.getSchema.getFieldSchemas).map { fields => + out.println(fields.map(_.getName).mkString("\t")) + } + } + + try { + while (!out.checkError() && driver.getResults(res)) { + res.foreach(out.println) + res.clear() + } + } catch { + case e:IOException => + console.printError( + s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} + |${org.apache.hadoop.util.StringUtils.stringifyException(e)} + """.stripMargin) + ret = 1 + } + + val cret = driver.close() + if (ret == 0) { + ret = cret + } + + val end = System.currentTimeMillis() + if (end > start) { + val timeTaken:Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds", null) + } + + // Destroy the driver to release all the locks. + driver.destroy() + } else { + if (sessionState.getIsVerbose) { + sessionState.out.println(tokens(0) + " " + cmd_1) + } + ret = proc.run(cmd_1).getResponseCode + } + } + ret + } + } +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala new file mode 100644 index 0000000000000..42cbf363b274f --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io.IOException +import java.util.{List => JList} +import javax.security.auth.login.LoginException + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hive.service.Service.STATE +import org.apache.hive.service.auth.HiveAuthFactory +import org.apache.hive.service.cli.CLIService +import org.apache.hive.service.{AbstractService, Service, ServiceException} + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +private[hive] class SparkSQLCLIService(hiveContext: HiveContext) + extends CLIService + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) + setSuperField(this, "sessionManager", sparkSqlSessionManager) + addService(sparkSqlSessionManager) + + try { + HiveAuthFactory.loginFromKeytab(hiveConf) + val serverUserName = ShimLoader.getHadoopShims + .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) + setSuperField(this, "serverUserName", serverUserName) + } catch { + case e @ (_: IOException | _: LoginException) => + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + } + + initCompositeService(hiveConf) + } +} + +private[thriftserver] trait ReflectedCompositeService { this: AbstractService => + def initCompositeService(hiveConf: HiveConf) { + // Emulating `CompositeService.init(hiveConf)` + val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") + serviceList.foreach(_.init(hiveConf)) + + // Emulating `AbstractService.init(hiveConf)` + invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) + setAncestorField(this, 3, "hiveConf", hiveConf) + invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) + getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala new file mode 100644 index 0000000000000..5202aa9903e03 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.util.{ArrayList => JArrayList} + +import org.apache.commons.lang.exception.ExceptionUtils +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} + +private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) + extends Driver with Logging { + + private var tableSchema: Schema = _ + private var hiveResponse: Seq[String] = _ + + override def init(): Unit = { + } + + private def getResultSetSchema(query: context.QueryExecution): Schema = { + val analyzed = query.analyzed + logger.debug(s"Result Schema: ${analyzed.output}") + if (analyzed.output.size == 0) { + new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) + } else { + val fieldSchemas = analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + + new Schema(fieldSchemas, null) + } + } + + override def run(command: String): CommandProcessorResponse = { + val execution = context.executePlan(context.hql(command).logicalPlan) + + // TODO unify the error code + try { + hiveResponse = execution.stringResult() + tableSchema = getResultSetSchema(execution) + new CommandProcessorResponse(0) + } catch { + case cause: Throwable => + logger.error(s"Failed in [$command]", cause) + new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) + } + } + + override def close(): Int = { + hiveResponse = null + tableSchema = null + 0 + } + + override def getSchema: Schema = tableSchema + + override def getResults(res: JArrayList[String]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.addAll(hiveResponse) + hiveResponse = null + true + } + } + + override def destroy() { + super.destroy() + hiveResponse = null + tableSchema = null + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala new file mode 100644 index 0000000000000..451c3bd7b9352 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} + +/** A singleton object for the master program. The slaves should not access this. */ +private[hive] object SparkSQLEnv extends Logging { + logger.debug("Initializing SparkSQLEnv") + + var hiveContext: HiveContext = _ + var sparkContext: SparkContext = _ + + def init() { + if (hiveContext == null) { + sparkContext = new SparkContext(new SparkConf() + .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) + + sparkContext.addSparkListener(new StatsReportListener()) + + hiveContext = new HiveContext(sparkContext) { + @transient override lazy val sessionState = SessionState.get() + @transient override lazy val hiveconf = sessionState.getConf + } + } + } + + /** Cleans up and shuts down the Spark SQL environments. */ + def stop() { + logger.debug("Shutting down Spark SQL Environment") + // Stop the SparkContext + if (SparkSQLEnv.sparkContext != null) { + sparkContext.stop() + sparkContext = null + hiveContext = null + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala new file mode 100644 index 0000000000000..6b3275b4eaf04 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.concurrent.Executors + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.session.SessionManager + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala new file mode 100644 index 0000000000000..a4e1f3e762e89 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver.server + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.math.{random, round} + +import java.sql.Timestamp +import java.util.{Map => JMap} + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} + +/** + * Executes queries using Spark SQL, and maintains a list of handles to active queries. + */ +class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { + val handleToOperation = ReflectionUtils + .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") + + override def newExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String], + async: Boolean): ExecuteStatementOperation = synchronized { + + val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logger.debug("CLOSING") + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + if (!iter.hasNext) { + new RowSet() + } else { + val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. + var curRow = 0 + var rowSet = new ArrayBuffer[Row](maxRows) + + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = new Row() + var curCol = 0 + + while (curCol < sparkRow.length) { + dataTypes(curCol) match { + case StringType => + row.addString(sparkRow(curCol).asInstanceOf[String]) + case IntegerType => + row.addColumnValue(ColumnValue.intValue(sparkRow.getInt(curCol))) + case BooleanType => + row.addColumnValue(ColumnValue.booleanValue(sparkRow.getBoolean(curCol))) + case DoubleType => + row.addColumnValue(ColumnValue.doubleValue(sparkRow.getDouble(curCol))) + case FloatType => + row.addColumnValue(ColumnValue.floatValue(sparkRow.getFloat(curCol))) + case DecimalType => + val hiveDecimal = sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal + row.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) + case LongType => + row.addColumnValue(ColumnValue.longValue(sparkRow.getLong(curCol))) + case ByteType => + row.addColumnValue(ColumnValue.byteValue(sparkRow.getByte(curCol))) + case ShortType => + row.addColumnValue(ColumnValue.intValue(sparkRow.getShort(curCol))) + case TimestampType => + row.addColumnValue( + ColumnValue.timestampValue(sparkRow.get(curCol).asInstanceOf[Timestamp])) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((sparkRow.get(curCol), dataTypes(curCol))) + row.addColumnValue(ColumnValue.stringValue(hiveString)) + } + curCol += 1 + } + rowSet += row + curRow += 1 + } + new RowSet(rowSet, 0) + } + } + + def getResultSetSchema: TableSchema = { + logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + def run(): Unit = { + logger.info(s"Running query '$statement'") + setState(OperationState.RUNNING) + try { + result = hiveContext.hql(statement) + logger.debug(result.queryExecution.toString()) + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + iter = result.queryExecution.toRdd.toLocalIterator + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + setHasResultSet(true) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logger.error("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + setState(OperationState.FINISHED) + } + } + + handleToOperation.put(operation.getHandle, operation) + operation + } +} diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt new file mode 100644 index 0000000000000..850f8014b6f05 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt @@ -0,0 +1,5 @@ +238val_238 +86val_86 +311val_311 +27val_27 +165val_165 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala new file mode 100644 index 0000000000000..69f19f826a802 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, InputStreamReader, PrintWriter} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { + val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") + val METASTORE_PATH = TestUtils.getMetastorePath("cli") + + override def beforeAll() { + val pb = new ProcessBuilder( + "../../bin/spark-sql", + "--master", + "local", + "--hiveconf", + s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", + "--hiveconf", + "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + process = pb.start() + outputWriter = new PrintWriter(process.getOutputStream, true) + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "spark-sql>") + } + + override def afterAll() { + process.destroy() + process.waitFor() + } + + test("simple commands") { + val dataFilePath = getDataFile("data/files/small_kv.txt") + executeQuery("create table hive_test1(key int, val string);") + executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") + executeQuery("cache table hive_test1", "Time taken") + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala new file mode 100644 index 0000000000000..fe3403b3292ec --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent._ + +import java.io.{BufferedReader, InputStreamReader} +import java.net.ServerSocket +import java.sql.{Connection, DriverManager, Statement} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.util.getTempFilePath + +/** + * Test for the HiveThriftServer2 using JDBC. + */ +class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { + + val WAREHOUSE_PATH = getTempFilePath("warehouse") + val METASTORE_PATH = getTempFilePath("metastore") + + val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" + val TABLE = "test" + val HOST = "localhost" + val PORT = { + // Let the system to choose a random available port to avoid collision with other parallel + // builds. + val socket = new ServerSocket(0) + val port = socket.getLocalPort + socket.close() + port + } + + // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. + val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean + + Class.forName(DRIVER_NAME) + + override def beforeAll() { launchServer() } + + override def afterAll() { stopServer() } + + private def launchServer(args: Seq[String] = Seq.empty) { + // Forking a new process to start the Hive Thrift server. The reason to do this is it is + // hard to clean up Hive resources entirely, so we just start a new process and kill + // that process for cleanup. + val defaultArgs = Seq( + "../../sbin/start-thriftserver.sh", + "--master local", + "--hiveconf", + "hive.root.logger=INFO,console", + "--hiveconf", + s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", + "--hiveconf", + s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH") + val pb = new ProcessBuilder(defaultArgs ++ args) + val environment = pb.environment() + environment.put("HIVE_SERVER2_THRIFT_PORT", PORT.toString) + environment.put("HIVE_SERVER2_THRIFT_BIND_HOST", HOST) + process = pb.start() + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "ThriftBinaryCLIService listening on") + + // Spawn a thread to read the output from the forked process. + // Note that this is necessary since in some configurations, log4j could be blocked + // if its output to stderr are not read, and eventually blocking the entire test suite. + future { + while (true) { + val stdout = readFrom(inputReader) + val stderr = readFrom(errorReader) + if (VERBOSE && stdout.length > 0) { + println(stdout) + } + if (VERBOSE && stderr.length > 0) { + println(stderr) + } + Thread.sleep(50) + } + } + } + + private def stopServer() { + process.destroy() + process.waitFor() + } + + test("test query execution against a Hive Thrift server") { + Thread.sleep(5 * 1000) + val dataFilePath = getDataFile("data/files/small_kv.txt") + val stmt = createStatement() + stmt.execute("DROP TABLE IF EXISTS test") + stmt.execute("DROP TABLE IF EXISTS test_cached") + stmt.execute("CREATE TABLE test(key int, val string)") + stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") + stmt.execute("CREATE TABLE test_cached as select * from test limit 4") + stmt.execute("CACHE TABLE test_cached") + + var rs = stmt.executeQuery("select count(*) from test") + rs.next() + assert(rs.getInt(1) === 5) + + rs = stmt.executeQuery("select count(*) from test_cached") + rs.next() + assert(rs.getInt(1) === 4) + + stmt.close() + } + + def getConnection: Connection = { + val connectURI = s"jdbc:hive2://localhost:$PORT/" + DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") + } + + def createStatement(): Statement = getConnection.createStatement() +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala new file mode 100644 index 0000000000000..bb2242618fbef --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, PrintWriter} +import java.text.SimpleDateFormat +import java.util.Date + +import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException + +object TestUtils { + val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") + + def getWarehousePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + + timestamp.format(new Date) + } + + def getMetastorePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + + timestamp.format(new Date) + } + + // Dummy function for initialize the log4j properties. + def init() { } + + // initialize log4j + try { + LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => // Ignore the error. + } +} + +trait TestUtils { + var process : Process = null + var outputWriter : PrintWriter = null + var inputReader : BufferedReader = null + var errorReader : BufferedReader = null + + def executeQuery( + cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { + println("Executing: " + cmd + ", expecting output: " + outputMessage) + outputWriter.write(cmd + "\n") + outputWriter.flush() + waitForQuery(timeout, outputMessage) + } + + protected def waitForQuery(timeout: Long, message: String): String = { + if (waitForOutput(errorReader, message, timeout)) { + Thread.sleep(500) + readOutput() + } else { + assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) + null + } + } + + // Wait for the specified str to appear in the output. + protected def waitForOutput( + reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { + val startTime = System.currentTimeMillis + var out = "" + while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { + out += readFrom(reader) + } + out.contains(str) + } + + // Read stdout output and filter out garbage collection messages. + protected def readOutput(): String = { + val output = readFrom(inputReader) + // Remove GC Messages + val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) + .mkString("\n") + filteredOutput + } + + protected def readFrom(reader: BufferedReader): String = { + var out = "" + var c = 0 + while (reader.ready) { + c = reader.read() + out += c.asInstanceOf[Char] + } + out + } + + protected def getDataFile(name: String) = { + Thread.currentThread().getContextClassLoader.getResource(name) + } +} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1699ffe06ce15..93d00f7c37c9b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -32,7 +32,7 @@ Spark Project Hive http://spark.apache.org/ - hive + hive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 201c85f3d501e..84d43eaeea51d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -255,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DecimalType, TimestampType, BinaryType) - protected def toHiveString(a: (Any, DataType)): String = a match { + protected[sql] def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a8623b64c656f..a022a1e2dc70e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -419,10 +419,10 @@ class HiveQuerySuite extends HiveComparisonTest { hql(s"set $testKey=$testVal") assert(get(testKey, testVal + "_") == testVal) - hql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - hql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + hql("set some.property=20") + assert(get("some.property", "0") == "20") + hql("set some.property = 40") + assert(get("some.property", "0") == "40") hql(s"set $testKey=$testVal") assert(get(testKey, "0") == testVal) @@ -436,63 +436,61 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - def collectResults(rdd: SchemaRDD): Set[(String, String)] = - rdd.collect().map { case Row(key: String, value: String) => key -> value }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. assert(hql("SET").collect().size == 0) - assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey=$testVal")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(hql("SET")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } hql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(hql("SET")) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + hql(s"SET").collect().map(_.getString(0)) } // "set key" - assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Set(nonexistentKey -> "")) { - collectResults(hql(s"SET $nonexistentKey")) + assertResult(Array(s"$nonexistentKey=")) { + hql(s"SET $nonexistentKey").collect().map(_.getString(0)) } // Assert that sql() should have the same effects as hql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey=$testVal")) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(sql("SET")) + assertResult(Array(s"$testKey=$testVal")) { + sql("SET").collect().map(_.getString(0)) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(sql("SET")) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + sql("SET").collect().map(_.getString(0)) } - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey")) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Set(nonexistentKey -> "")) { - collectResults(sql(s"SET $nonexistentKey")) + assertResult(Array(s"$nonexistentKey=")) { + sql(s"SET $nonexistentKey").collect().map(_.getString(0)) } clear() diff --git a/streaming/pom.xml b/streaming/pom.xml index f60697ce745b7..b99f306b8f2cc 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming_2.10 - streaming + streaming jar Spark Project Streaming diff --git a/tools/pom.xml b/tools/pom.xml index c0ee8faa7a615..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,7 +27,7 @@ org.apache.spark spark-tools_2.10 - tools + tools jar Spark Project Tools diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 5b13a1f002d6e..51744ece0412d 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-alpha + yarn-alpha org.apache.spark diff --git a/yarn/pom.xml b/yarn/pom.xml index efb473aa1b261..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -29,7 +29,7 @@ pom Spark Project YARN Parent POM - yarn + yarn diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index ceaf9f9d71001..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-stable + yarn-stable org.apache.spark From ecf30ee7e78ea59c462c54db0fde5328f997466c Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 27 Jul 2014 16:08:16 -0700 Subject: [PATCH 0094/1492] [SPARK-1777] Prevent OOMs from single partitions **Problem.** When caching, we currently unroll the entire RDD partition before making sure we have enough free memory. This is a common cause for OOMs especially when (1) the BlockManager has little free space left in memory, and (2) the partition is large. **Solution.** We maintain a global memory pool of `M` bytes shared across all threads, similar to the way we currently manage memory for shuffle aggregation. Then, while we unroll each partition, periodically check if there is enough space to continue. If not, drop enough RDD blocks to ensure we have at least `M` bytes to work with, then try again. If we still don't have enough space to unroll the partition, give up and drop the block to disk directly if applicable. **New configurations.** - `spark.storage.bufferFraction` - the value of `M` as a fraction of the storage memory. (default: 0.2) - `spark.storage.safetyFraction` - a margin of safety in case size estimation is slightly off. This is the equivalent of the existing `spark.shuffle.safetyFraction`. (default 0.9) For more detail, see the [design document](https://issues.apache.org/jira/secure/attachment/12651793/spark-1777-design-doc.pdf). Tests pending for performance and memory usage patterns. Author: Andrew Or Closes #1165 from andrewor14/them-rdd-memories and squashes the following commits: e77f451 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories c7c8832 [Andrew Or] Simplify logic + update a few comments 269d07b [Andrew Or] Very minor changes to tests 6645a8a [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories b7e165c [Andrew Or] Add new tests for unrolling blocks f12916d [Andrew Or] Slightly clean up tests 71672a7 [Andrew Or] Update unrollSafely tests 369ad07 [Andrew Or] Correct ensureFreeSpace and requestMemory behavior f4d035c [Andrew Or] Allow one thread to unroll multiple blocks a66fbd2 [Andrew Or] Rename a few things + update comments 68730b3 [Andrew Or] Fix weird scalatest behavior e40c60d [Andrew Or] Fix MIMA excludes ff77aa1 [Andrew Or] Fix tests 1a43c06 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories b9a6eee [Andrew Or] Simplify locking behavior on unrollMemoryMap ed6cda4 [Andrew Or] Formatting fix (super minor) f9ff82e [Andrew Or] putValues -> putIterator + putArray beb368f [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 8448c9b [Andrew Or] Fix tests a49ba4d [Andrew Or] Do not expose unroll memory check period 69bc0a5 [Andrew Or] Always synchronize on putLock before unrollMemoryMap 3f5a083 [Andrew Or] Simplify signature of ensureFreeSpace dce55c8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 8288228 [Andrew Or] Synchronize put and unroll properly 4f18a3d [Andrew Or] bufferFraction -> unrollFraction 28edfa3 [Andrew Or] Update a few comments / log messages 728323b [Andrew Or] Do not synchronize every 1000 elements 5ab2329 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 129c441 [Andrew Or] Fix bug: Use toArray rather than array 9a65245 [Andrew Or] Update a few comments + minor control flow changes 57f8d85 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories abeae4f [Andrew Or] Add comment clarifying the MEMORY_AND_DISK case 3dd96aa [Andrew Or] AppendOnlyBuffer -> Vector (+ a few small changes) f920531 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 0871835 [Andrew Or] Add an effective storage level interface to BlockManager 64e7d4c [Andrew Or] Add/modify a few comments (minor) 8af2f35 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 4f4834e [Andrew Or] Use original storage level for blocks dropped to disk ecc8c2d [Andrew Or] Fix binary incompatibility 24185ea [Andrew Or] Avoid dropping a block back to disk if reading from disk 2b7ee66 [Andrew Or] Fix bug in SizeTracking* 9b9a273 [Andrew Or] Fix tests 20eb3e5 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 649bdb3 [Andrew Or] Document spark.storage.bufferFraction a10b0e7 [Andrew Or] Add initial memory request threshold + rename a few things e9c3cb0 [Andrew Or] cacheMemoryMap -> unrollMemoryMap 198e374 [Andrew Or] Unfold -> unroll 0d50155 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories d9d02a8 [Andrew Or] Remove unused param in unfoldSafely ec728d8 [Andrew Or] Add tests for safe unfolding of blocks 22b2209 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 078eb83 [Andrew Or] Add check for hasNext in PrimitiveVector.iterator 0871535 [Andrew Or] Fix tests in BlockManagerSuite d68f31e [Andrew Or] Safely unfold blocks for all memory puts 5961f50 [Andrew Or] Fix tests 195abd7 [Andrew Or] Refactor: move unfold logic to MemoryStore 1e82d00 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 3ce413e [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories d5dd3b4 [Andrew Or] Free buffer memory in finally ea02eec [Andrew Or] Fix tests b8e1d9c [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories a8704c1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories e1b8b25 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories 87aa75c [Andrew Or] Fix mima excludes again (typo) 11eb921 [Andrew Or] Clarify comment (minor) 50cae44 [Andrew Or] Remove now duplicate mima exclude 7de5ef9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories df47265 [Andrew Or] Fix binary incompatibility 6d05a81 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories f94f5af [Andrew Or] Update a few comments (minor) 776aec9 [Andrew Or] Prevent OOM if a single RDD partition is too large bbd3eea [Andrew Or] Fix CacheManagerSuite to use Array 97ea499 [Andrew Or] Change BlockManager interface to use Arrays c12f093 [Andrew Or] Add SizeTrackingAppendOnlyBuffer and tests --- .../scala/org/apache/spark/CacheManager.scala | 72 ++- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 4 +- .../apache/spark/storage/BlockManager.scala | 110 ++-- .../org/apache/spark/storage/BlockStore.scala | 6 +- .../org/apache/spark/storage/DiskStore.scala | 12 +- .../apache/spark/storage/MemoryStore.scala | 256 +++++++- .../apache/spark/storage/TachyonStore.scala | 12 +- .../apache/spark/storage/ThreadingTest.scala | 2 +- .../org/apache/spark/util/SizeEstimator.scala | 2 +- .../util/collection/PrimitiveVector.scala | 15 +- .../spark/util/collection/SizeTracker.scala | 105 ++++ .../SizeTrackingAppendOnlyMap.scala | 71 +-- .../util/collection/SizeTrackingVector.scala | 46 ++ .../org/apache/spark/CacheManagerSuite.scala | 25 +- .../spark/storage/BlockManagerSuite.scala | 594 ++++++++++++------ .../util/SizeTrackingAppendOnlyMapSuite.scala | 120 ---- .../util/collection/SizeTrackerSuite.scala | 204 ++++++ docs/configuration.md | 9 + project/MimaExcludes.scala | 10 +- .../receiver/ReceiverSupervisorImpl.scala | 5 +- 21 files changed, 1165 insertions(+), 517 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala delete mode 100644 core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 8f867686a0443..5ddda4d6953fa 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -17,9 +17,9 @@ package org.apache.spark -import scala.collection.mutable.{ArrayBuffer, HashSet} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer -import org.apache.spark.executor.InputMetrics import org.apache.spark.rdd.RDD import org.apache.spark.storage._ @@ -30,7 +30,7 @@ import org.apache.spark.storage._ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD partitions that are being computed/loaded. */ - private val loading = new HashSet[RDDBlockId]() + private val loading = new mutable.HashSet[RDDBlockId] /** Gets or computes an RDD partition. Used by RDD.iterator() when an RDD is cached. */ def getOrCompute[T]( @@ -118,21 +118,29 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { } /** - * Cache the values of a partition, keeping track of any updates in the storage statuses - * of other blocks along the way. + * Cache the values of a partition, keeping track of any updates in the storage statuses of + * other blocks along the way. + * + * The effective storage level refers to the level that actually specifies BlockManager put + * behavior, not the level originally specified by the user. This is mainly for forcing a + * MEMORY_AND_DISK partition to disk if there is not enough room to unroll the partition, + * while preserving the the original semantics of the RDD as specified by the application. */ private def putInBlockManager[T]( key: BlockId, values: Iterator[T], - storageLevel: StorageLevel, - updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)]): Iterator[T] = { - - if (!storageLevel.useMemory) { - /* This RDD is not to be cached in memory, so we can just pass the computed values - * as an iterator directly to the BlockManager, rather than first fully unrolling - * it in memory. The latter option potentially uses much more memory and risks OOM - * exceptions that can be avoided. */ - updatedBlocks ++= blockManager.put(key, values, storageLevel, tellMaster = true) + level: StorageLevel, + updatedBlocks: ArrayBuffer[(BlockId, BlockStatus)], + effectiveStorageLevel: Option[StorageLevel] = None): Iterator[T] = { + + val putLevel = effectiveStorageLevel.getOrElse(level) + if (!putLevel.useMemory) { + /* + * This RDD is not to be cached in memory, so we can just pass the computed values as an + * iterator directly to the BlockManager rather than first fully unrolling it in memory. + */ + updatedBlocks ++= + blockManager.putIterator(key, values, level, tellMaster = true, effectiveStorageLevel) blockManager.get(key) match { case Some(v) => v.data.asInstanceOf[Iterator[T]] case None => @@ -140,14 +148,36 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { throw new BlockException(key, s"Block manager failed to return cached value for $key!") } } else { - /* This RDD is to be cached in memory. In this case we cannot pass the computed values + /* + * This RDD is to be cached in memory. In this case we cannot pass the computed values * to the BlockManager as an iterator and expect to read it back later. This is because - * we may end up dropping a partition from memory store before getting it back, e.g. - * when the entirety of the RDD does not fit in memory. */ - val elements = new ArrayBuffer[Any] - elements ++= values - updatedBlocks ++= blockManager.put(key, elements, storageLevel, tellMaster = true) - elements.iterator.asInstanceOf[Iterator[T]] + * we may end up dropping a partition from memory store before getting it back. + * + * In addition, we must be careful to not unroll the entire partition in memory at once. + * Otherwise, we may cause an OOM exception if the JVM does not have enough space for this + * single partition. Instead, we unroll the values cautiously, potentially aborting and + * dropping the partition to disk if applicable. + */ + blockManager.memoryStore.unrollSafely(key, values, updatedBlocks) match { + case Left(arr) => + // We have successfully unrolled the entire partition, so cache it in memory + updatedBlocks ++= + blockManager.putArray(key, arr, level, tellMaster = true, effectiveStorageLevel) + arr.iterator.asInstanceOf[Iterator[T]] + case Right(it) => + // There is not enough space to cache this partition in memory + logWarning(s"Not enough space to cache partition $key in memory! " + + s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") + val returnValues = it.asInstanceOf[Iterator[T]] + if (putLevel.useDisk) { + logWarning(s"Persisting partition $key to disk instead.") + val diskOnlyLevel = StorageLevel(useDisk = true, useMemory = false, + useOffHeap = false, deserialized = false, putLevel.replication) + putInBlockManager[T](key, returnValues, level, updatedBlocks, Some(diskOnlyLevel)) + } else { + returnValues + } + } } } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 8f70744d804d9..6ee731b22c03c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -67,7 +67,7 @@ class SparkEnv ( val metricsSystem: MetricsSystem, val conf: SparkConf) extends Logging { - // A mapping of thread ID to amount of memory used for shuffle in bytes + // A mapping of thread ID to amount of memory, in bytes, used for shuffle aggregations // All accesses should be manually synchronized val shuffleMemoryMap = mutable.HashMap[Long, Long]() diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index b16133b20cc02..3b69bc4ca4142 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -266,11 +266,13 @@ private[spark] class Executor( } } } finally { - // TODO: Unregister shuffle memory only for ResultTask + // Release memory used by this thread for shuffles val shuffleMemoryMap = env.shuffleMemoryMap shuffleMemoryMap.synchronized { shuffleMemoryMap.remove(Thread.currentThread().getId) } + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() runningTasks.remove(taskId) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0db0a5bc7341b..d746526639e58 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -38,7 +38,7 @@ import org.apache.spark.util._ private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues -private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends BlockValues +private[spark] case class ArrayValues(buffer: Array[Any]) extends BlockValues /* Class for returning a fetched block and associated metrics. */ private[spark] class BlockResult( @@ -71,9 +71,9 @@ private[spark] class BlockManager( // Actual storage of where blocks are kept private var tachyonInitialized = false - private[storage] val memoryStore = new MemoryStore(this, maxMemory) - private[storage] val diskStore = new DiskStore(this, diskBlockManager) - private[storage] lazy val tachyonStore: TachyonStore = { + private[spark] val memoryStore = new MemoryStore(this, maxMemory) + private[spark] val diskStore = new DiskStore(this, diskBlockManager) + private[spark] lazy val tachyonStore: TachyonStore = { val storeDir = conf.get("spark.tachyonStore.baseDir", "/tmp_spark_tachyon") val appFolderName = conf.get("spark.tachyonStore.folderName") val tachyonStorePath = s"$storeDir/$appFolderName/${this.executorId}" @@ -463,16 +463,17 @@ private[spark] class BlockManager( val values = dataDeserialize(blockId, bytes) if (level.deserialized) { // Cache the values before returning them - // TODO: Consider creating a putValues that also takes in a iterator? - val valuesBuffer = new ArrayBuffer[Any] - valuesBuffer ++= values - memoryStore.putValues(blockId, valuesBuffer, level, returnValues = true).data - match { - case Left(values2) => - return Some(new BlockResult(values2, DataReadMethod.Disk, info.size)) - case _ => - throw new SparkException("Memory store did not return back an iterator") - } + val putResult = memoryStore.putIterator( + blockId, values, level, returnValues = true, allowPersistToDisk = false) + // The put may or may not have succeeded, depending on whether there was enough + // space to unroll the block. Either way, the put here should return an iterator. + putResult.data match { + case Left(it) => + return Some(new BlockResult(it, DataReadMethod.Disk, info.size)) + case _ => + // This only happens if we dropped the values back to disk (which is never) + throw new SparkException("Memory store did not return an iterator!") + } } else { return Some(new BlockResult(values, DataReadMethod.Disk, info.size)) } @@ -561,13 +562,14 @@ private[spark] class BlockManager( iter } - def put( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, - tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, IteratorValues(values), level, tellMaster) + doPut(blockId, IteratorValues(values), level, tellMaster, effectiveStorageLevel) } /** @@ -589,13 +591,14 @@ private[spark] class BlockManager( * Put a new block of values to the block manager. * Return a list of blocks updated as a result of this put. */ - def put( + def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") - doPut(blockId, ArrayBufferValues(values), level, tellMaster) + doPut(blockId, ArrayValues(values), level, tellMaster, effectiveStorageLevel) } /** @@ -606,19 +609,33 @@ private[spark] class BlockManager( blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None): Seq[(BlockId, BlockStatus)] = { require(bytes != null, "Bytes is null") - doPut(blockId, ByteBufferValues(bytes), level, tellMaster) + doPut(blockId, ByteBufferValues(bytes), level, tellMaster, effectiveStorageLevel) } + /** + * Put the given block according to the given level in one of the block stores, replicating + * the values if necessary. + * + * The effective storage level refers to the level according to which the block will actually be + * handled. This allows the caller to specify an alternate behavior of doPut while preserving + * the original level specified by the user. + */ private def doPut( blockId: BlockId, data: BlockValues, level: StorageLevel, - tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + tellMaster: Boolean = true, + effectiveStorageLevel: Option[StorageLevel] = None) + : Seq[(BlockId, BlockStatus)] = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + effectiveStorageLevel.foreach { level => + require(level != null && level.isValid, "Effective StorageLevel is null or invalid") + } // Return value val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -657,13 +674,16 @@ private[spark] class BlockManager( // Size of the block in bytes var size = 0L + // The level we actually use to put the block + val putLevel = effectiveStorageLevel.getOrElse(level) + // If we're storing bytes, then initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. val replicationFuture = data match { - case b: ByteBufferValues if level.replication > 1 => + case b: ByteBufferValues if putLevel.replication > 1 => // Duplicate doesn't copy the bytes, but just creates a wrapper val bufferView = b.buffer.duplicate() - Future { replicate(blockId, bufferView, level) } + Future { replicate(blockId, bufferView, putLevel) } case _ => null } @@ -676,18 +696,18 @@ private[spark] class BlockManager( // returnValues - Whether to return the values put // blockStore - The type of storage to put these values into val (returnValues, blockStore: BlockStore) = { - if (level.useMemory) { + if (putLevel.useMemory) { // Put it in memory first, even if it also has useDisk set to true; // We will drop it to disk later if the memory store can't hold it. (true, memoryStore) - } else if (level.useOffHeap) { + } else if (putLevel.useOffHeap) { // Use tachyon for off-heap storage (false, tachyonStore) - } else if (level.useDisk) { + } else if (putLevel.useDisk) { // Don't get back the bytes from put unless we replicate them - (level.replication > 1, diskStore) + (putLevel.replication > 1, diskStore) } else { - assert(level == StorageLevel.NONE) + assert(putLevel == StorageLevel.NONE) throw new BlockException( blockId, s"Attempted to put block $blockId without specifying storage level!") } @@ -696,22 +716,22 @@ private[spark] class BlockManager( // Actually put the values val result = data match { case IteratorValues(iterator) => - blockStore.putValues(blockId, iterator, level, returnValues) - case ArrayBufferValues(array) => - blockStore.putValues(blockId, array, level, returnValues) + blockStore.putIterator(blockId, iterator, putLevel, returnValues) + case ArrayValues(array) => + blockStore.putArray(blockId, array, putLevel, returnValues) case ByteBufferValues(bytes) => bytes.rewind() - blockStore.putBytes(blockId, bytes, level) + blockStore.putBytes(blockId, bytes, putLevel) } size = result.size result.data match { - case Left (newIterator) if level.useMemory => valuesAfterPut = newIterator + case Left (newIterator) if putLevel.useMemory => valuesAfterPut = newIterator case Right (newBytes) => bytesAfterPut = newBytes case _ => } // Keep track of which blocks are dropped from memory - if (level.useMemory) { + if (putLevel.useMemory) { result.droppedBlocks.foreach { updatedBlocks += _ } } @@ -742,7 +762,7 @@ private[spark] class BlockManager( // Either we're storing bytes and we asynchronously started replication, or we're storing // values and need to serialize and replicate them now: - if (level.replication > 1) { + if (putLevel.replication > 1) { data match { case ByteBufferValues(bytes) => if (replicationFuture != null) { @@ -758,7 +778,7 @@ private[spark] class BlockManager( } bytesAfterPut = dataSerialize(blockId, valuesAfterPut) } - replicate(blockId, bytesAfterPut, level) + replicate(blockId, bytesAfterPut, putLevel) logDebug("Put block %s remotely took %s" .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) } @@ -766,7 +786,7 @@ private[spark] class BlockManager( BlockManager.dispose(bytesAfterPut) - if (level.replication > 1) { + if (putLevel.replication > 1) { logDebug("Putting block %s with replication took %s" .format(blockId, Utils.getUsedTimeMs(startTimeMs))) } else { @@ -818,7 +838,7 @@ private[spark] class BlockManager( value: Any, level: StorageLevel, tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { - put(blockId, Iterator(value), level, tellMaster) + putIterator(blockId, Iterator(value), level, tellMaster) } /** @@ -829,7 +849,7 @@ private[spark] class BlockManager( */ def dropFromMemory( blockId: BlockId, - data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { + data: Either[Array[Any], ByteBuffer]): Option[BlockStatus] = { logInfo(s"Dropping block $blockId from memory") val info = blockInfo.get(blockId).orNull @@ -853,7 +873,7 @@ private[spark] class BlockManager( logInfo(s"Writing block $blockId to disk") data match { case Left(elements) => - diskStore.putValues(blockId, elements, level, returnValues = false) + diskStore.putArray(blockId, elements, level, returnValues = false) case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } @@ -1068,9 +1088,11 @@ private[spark] class BlockManager( private[spark] object BlockManager extends Logging { private val ID_GENERATOR = new IdGenerator + /** Return the total amount of storage memory available. */ private def getMaxMemory(conf: SparkConf): Long = { val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) - (Runtime.getRuntime.maxMemory * memoryFraction).toLong + val safetyFraction = conf.getDouble("spark.storage.safetyFraction", 0.9) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } def getHeartBeatFrequency(conf: SparkConf): Long = diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index b9b53b1a2f118..69985c9759e2d 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -37,15 +37,15 @@ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ - def putValues( + def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult - def putValues( + def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index ebff0cb5ba153..c83261dd91b36 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -21,8 +21,6 @@ import java.io.{FileOutputStream, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode -import scala.collection.mutable.ArrayBuffer - import org.apache.spark.Logging import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -30,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on disk. */ -private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) +private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManager) extends BlockStore(blockManager) with Logging { val minMemoryMapBytes = blockManager.conf.getLong("spark.storage.memoryMapThreshold", 2 * 4096L) @@ -57,15 +55,15 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage PutResult(bytes.limit(), Right(bytes.duplicate())) } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 71f66c826c5b3..28f675c2bbb1e 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -20,27 +20,45 @@ package org.apache.spark.storage import java.nio.ByteBuffer import java.util.LinkedHashMap +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.spark.util.{SizeEstimator, Utils} +import org.apache.spark.util.collection.SizeTrackingVector private case class MemoryEntry(value: Any, size: Long, deserialized: Boolean) /** - * Stores blocks in memory, either as ArrayBuffers of deserialized Java objects or as + * Stores blocks in memory, either as Arrays of deserialized Java objects or as * serialized ByteBuffers. */ -private class MemoryStore(blockManager: BlockManager, maxMemory: Long) +private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) extends BlockStore(blockManager) { + private val conf = blockManager.conf private val entries = new LinkedHashMap[BlockId, MemoryEntry](32, 0.75f, true) + @volatile private var currentMemory = 0L - // Object used to ensure that only one thread is putting blocks and if necessary, dropping - // blocks from the memory store. - private val putLock = new Object() + + // Ensure only one thread is putting, and if necessary, dropping blocks at any given time + private val accountingLock = new Object + + // A mapping from thread ID to amount of memory used for unrolling a block (in bytes) + // All accesses of this map are assumed to have manually synchronized on `accountingLock` + private val unrollMemoryMap = mutable.HashMap[Long, Long]() + + /** + * The amount of space ensured for unrolling values in memory, shared across all cores. + * This space is not reserved in advance, but allocated dynamically by dropping existing blocks. + */ + private val maxUnrollMemory: Long = { + val unrollFraction = conf.getDouble("spark.storage.unrollFraction", 0.2) + (maxMemory * unrollFraction).toLong + } logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) + /** Free memory not occupied by existing blocks. Note that this does not include unroll memory. */ def freeMemory: Long = maxMemory - currentMemory override def getSize(blockId: BlockId): Long = { @@ -55,20 +73,16 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) bytes.rewind() if (level.deserialized) { val values = blockManager.dataDeserialize(blockId, bytes) - val elements = new ArrayBuffer[Any] - elements ++= values - val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) - val putAttempt = tryToPut(blockId, elements, sizeEstimate, deserialized = true) - PutResult(sizeEstimate, Left(values.toIterator), putAttempt.droppedBlocks) + putIterator(blockId, values, level, returnValues = true) } else { val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { if (level.deserialized) { @@ -82,14 +96,52 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - val valueEntries = new ArrayBuffer[Any]() - valueEntries ++= values - putValues(blockId, valueEntries, level, returnValues) + putIterator(blockId, values, level, returnValues, allowPersistToDisk = true) + } + + /** + * Attempt to put the given block in memory store. + * + * There may not be enough space to fully unroll the iterator in memory, in which case we + * optionally drop the values to disk if + * (1) the block's storage level specifies useDisk, and + * (2) `allowPersistToDisk` is true. + * + * One scenario in which `allowPersistToDisk` is false is when the BlockManager reads a block + * back from disk and attempts to cache it in memory. In this case, we should not persist the + * block back on disk again, as it is already in disk store. + */ + private[storage] def putIterator( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean, + allowPersistToDisk: Boolean): PutResult = { + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + val unrolledValues = unrollSafely(blockId, values, droppedBlocks) + unrolledValues match { + case Left(arrayValues) => + // Values are fully unrolled in memory, so store them as an array + val res = putArray(blockId, arrayValues, level, returnValues) + droppedBlocks ++= res.droppedBlocks + PutResult(res.size, res.data, droppedBlocks) + case Right(iteratorValues) => + // Not enough space to unroll this block; drop to disk if applicable + logWarning(s"Not enough space to store block $blockId in memory! " + + s"Free memory is $freeMemory bytes.") + if (level.useDisk && allowPersistToDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + val res = blockManager.diskStore.putIterator(blockId, iteratorValues, level, returnValues) + PutResult(res.size, res.data, droppedBlocks) + } else { + PutResult(0, Left(iteratorValues), droppedBlocks) + } + } } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { @@ -99,7 +151,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[ArrayBuffer[Any]].iterator)) + Some(blockManager.dataSerialize(blockId, entry.value.asInstanceOf[Array[Any]].iterator)) } else { Some(entry.value.asInstanceOf[ByteBuffer].duplicate()) // Doesn't actually copy the data } @@ -112,7 +164,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) if (entry == null) { None } else if (entry.deserialized) { - Some(entry.value.asInstanceOf[ArrayBuffer[Any]].iterator) + Some(entry.value.asInstanceOf[Array[Any]].iterator) } else { val buffer = entry.value.asInstanceOf[ByteBuffer].duplicate() // Doesn't actually copy data Some(blockManager.dataDeserialize(blockId, buffer)) @@ -140,6 +192,93 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("MemoryStore cleared") } + /** + * Unroll the given block in memory safely. + * + * The safety of this operation refers to avoiding potential OOM exceptions caused by + * unrolling the entirety of the block in memory at once. This is achieved by periodically + * checking whether the memory restrictions for unrolling blocks are still satisfied, + * stopping immediately if not. This check is a safeguard against the scenario in which + * there is not enough free memory to accommodate the entirety of a single block. + * + * This method returns either an array with the contents of the entire block or an iterator + * containing the values of the block (if the array would have exceeded available memory). + */ + def unrollSafely( + blockId: BlockId, + values: Iterator[Any], + droppedBlocks: ArrayBuffer[(BlockId, BlockStatus)]) + : Either[Array[Any], Iterator[Any]] = { + + // Number of elements unrolled so far + var elementsUnrolled = 0 + // Whether there is still enough memory for us to continue unrolling this block + var keepUnrolling = true + // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing. + val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + // How often to check whether we need to request more memory + val memoryCheckPeriod = 16 + // Memory currently reserved by this thread for this particular unrolling operation + var memoryThreshold = initialMemoryThreshold + // Memory to request as a multiple of current vector size + val memoryGrowthFactor = 1.5 + // Previous unroll memory held by this thread, for releasing later (only at the very end) + val previousMemoryReserved = currentUnrollMemoryForThisThread + // Underlying vector for unrolling the block + var vector = new SizeTrackingVector[Any] + + // Request enough memory to begin unrolling + keepUnrolling = reserveUnrollMemoryForThisThread(initialMemoryThreshold) + + // Unroll this block safely, checking whether we have exceeded our threshold periodically + try { + while (values.hasNext && keepUnrolling) { + vector += values.next() + if (elementsUnrolled % memoryCheckPeriod == 0) { + // If our vector's size has exceeded the threshold, request more memory + val currentSize = vector.estimateSize() + if (currentSize >= memoryThreshold) { + val amountToRequest = (currentSize * (memoryGrowthFactor - 1)).toLong + // Hold the accounting lock, in case another thread concurrently puts a block that + // takes up the unrolling space we just ensured here + accountingLock.synchronized { + if (!reserveUnrollMemoryForThisThread(amountToRequest)) { + // If the first request is not granted, try again after ensuring free space + // If there is still not enough space, give up and drop the partition + val spaceToEnsure = maxUnrollMemory - currentUnrollMemory + if (spaceToEnsure > 0) { + val result = ensureFreeSpace(blockId, spaceToEnsure) + droppedBlocks ++= result.droppedBlocks + } + keepUnrolling = reserveUnrollMemoryForThisThread(amountToRequest) + } + } + // New threshold is currentSize * memoryGrowthFactor + memoryThreshold = currentSize + amountToRequest + } + } + elementsUnrolled += 1 + } + + if (keepUnrolling) { + // We successfully unrolled the entirety of this block + Left(vector.toArray) + } else { + // We ran out of space while unrolling the values for this block + Right(vector.iterator ++ values) + } + + } finally { + // If we return an array, the values returned do not depend on the underlying vector and + // we can immediately free up space for other threads. Otherwise, if we return an iterator, + // we release the memory claimed by this thread later on when the task finishes. + if (keepUnrolling) { + val amountToRelease = currentUnrollMemoryForThisThread - previousMemoryReserved + releaseUnrollMemoryForThisThread(amountToRelease) + } + } + } + /** * Return the RDD ID that a given block ID is from, or None if it is not an RDD block. */ @@ -149,10 +288,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Try to put in a set of values, if we can free up enough space. The value should either be - * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) - * size must also be passed by the caller. + * an Array if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) size + * must also be passed by the caller. * - * Lock on the object putLock to ensure that all the put requests and its associated block + * Synchronize on `accountingLock` to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. @@ -174,7 +313,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) var putSuccess = false val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] - putLock.synchronized { + accountingLock.synchronized { val freeSpaceResult = ensureFreeSpace(blockId, size) val enoughFreeSpace = freeSpaceResult.success droppedBlocks ++= freeSpaceResult.droppedBlocks @@ -193,7 +332,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. val data = if (deserialized) { - Left(value.asInstanceOf[ArrayBuffer[Any]]) + Left(value.asInstanceOf[Array[Any]]) } else { Right(value.asInstanceOf[ByteBuffer].duplicate()) } @@ -210,12 +349,14 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assume that a lock is held by the caller to ensure only one thread is dropping blocks. - * Otherwise, the freed space may fill up before the caller puts in their new value. + * Assume that `accountingLock` is held by the caller to ensure only one thread is dropping + * blocks. Otherwise, the freed space may fill up before the caller puts in their new value. * * Return whether there is enough free space, along with the blocks dropped in the process. */ - private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { + private def ensureFreeSpace( + blockIdToAdd: BlockId, + space: Long): ResultWithDroppedBlocks = { logInfo(s"ensureFreeSpace($space) called with curMem=$currentMemory, maxMem=$maxMemory") val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] @@ -225,9 +366,12 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) return ResultWithDroppedBlocks(success = false, droppedBlocks) } - if (maxMemory - currentMemory < space) { + // Take into account the amount of memory currently occupied by unrolling blocks + val actualFreeMemory = freeMemory - currentUnrollMemory + + if (actualFreeMemory < space) { val rddToAdd = getRddId(blockIdToAdd) - val selectedBlocks = new ArrayBuffer[BlockId]() + val selectedBlocks = new ArrayBuffer[BlockId] var selectedMemory = 0L // This is synchronized to ensure that the set of entries is not changed @@ -235,7 +379,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // can lead to exceptions. entries.synchronized { val iterator = entries.entrySet().iterator() - while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) { + while (actualFreeMemory + selectedMemory < space && iterator.hasNext) { val pair = iterator.next() val blockId = pair.getKey if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) { @@ -245,7 +389,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - if (maxMemory - (currentMemory - selectedMemory) >= space) { + if (actualFreeMemory + selectedMemory >= space) { logInfo(s"${selectedBlocks.size} blocks selected for dropping") for (blockId <- selectedBlocks) { val entry = entries.synchronized { entries.get(blockId) } @@ -254,7 +398,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) // future safety. if (entry != null) { val data = if (entry.deserialized) { - Left(entry.value.asInstanceOf[ArrayBuffer[Any]]) + Left(entry.value.asInstanceOf[Array[Any]]) } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } @@ -275,8 +419,56 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) override def contains(blockId: BlockId): Boolean = { entries.synchronized { entries.containsKey(blockId) } } + + /** + * Reserve additional memory for unrolling blocks used by this thread. + * Return whether the request is granted. + */ + private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { + accountingLock.synchronized { + val granted = freeMemory > currentUnrollMemory + memory + if (granted) { + val threadId = Thread.currentThread().getId + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, 0L) + memory + } + granted + } + } + + /** + * Release memory used by this thread for unrolling blocks. + * If the amount is not specified, remove the current thread's allocation altogether. + */ + private[spark] def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { + val threadId = Thread.currentThread().getId + accountingLock.synchronized { + if (memory < 0) { + unrollMemoryMap.remove(threadId) + } else { + unrollMemoryMap(threadId) = unrollMemoryMap.getOrElse(threadId, memory) - memory + // If this thread claims no more unroll memory, release it completely + if (unrollMemoryMap(threadId) <= 0) { + unrollMemoryMap.remove(threadId) + } + } + } + } + + /** + * Return the amount of memory currently occupied for unrolling blocks across all threads. + */ + private[spark] def currentUnrollMemory: Long = accountingLock.synchronized { + unrollMemoryMap.values.sum + } + + /** + * Return the amount of memory currently occupied for unrolling blocks by this thread. + */ + private[spark] def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { + unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) + } } -private case class ResultWithDroppedBlocks( +private[spark] case class ResultWithDroppedBlocks( success: Boolean, droppedBlocks: Seq[(BlockId, BlockStatus)]) diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index d8ff4ff6bd42c..932b5616043b4 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -20,8 +20,6 @@ package org.apache.spark.storage import java.io.IOException import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer - import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -30,7 +28,7 @@ import org.apache.spark.util.Utils /** * Stores BlockManager blocks on Tachyon. */ -private class TachyonStore( +private[spark] class TachyonStore( blockManager: BlockManager, tachyonManager: TachyonBlockManager) extends BlockStore(blockManager: BlockManager) with Logging { @@ -45,15 +43,15 @@ private class TachyonStore( putIntoTachyonStore(blockId, bytes, returnValues = true) } - override def putValues( + override def putArray( blockId: BlockId, - values: ArrayBuffer[Any], + values: Array[Any], level: StorageLevel, returnValues: Boolean): PutResult = { - putValues(blockId, values.toIterator, level, returnValues) + putIterator(blockId, values.toIterator, level, returnValues) } - override def putValues( + override def putIterator( blockId: BlockId, values: Iterator[Any], level: StorageLevel, diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 328be158db680..75c2e09a6bbb8 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -48,7 +48,7 @@ private[spark] object ThreadingTest { val block = (1 to blockSize).map(_ => Random.nextInt()) val level = randomLevel() val startTime = System.currentTimeMillis() - manager.put(blockId, block.iterator, level, tellMaster = true) + manager.putIterator(blockId, block.iterator, level, tellMaster = true) println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") queue.add((blockId, block)) } diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 08465575309c6..bce3b3afe9aba 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -180,7 +180,7 @@ private[spark] object SizeEstimator extends Logging { } } - // Estimat the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. + // Estimate the size of arrays larger than ARRAY_SIZE_FOR_SAMPLING by sampling. private val ARRAY_SIZE_FOR_SAMPLING = 200 private val ARRAY_SAMPLE_SIZE = 100 // should be lower than ARRAY_SIZE_FOR_SAMPLING diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala index b84eb65c62bc7..7e76d060d6000 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala @@ -36,7 +36,7 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: _array(index) } - def +=(value: V) { + def +=(value: V): Unit = { if (_numElements == _array.length) { resize(_array.length * 2) } @@ -50,6 +50,19 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassTag](initialSize: def size: Int = _numElements + def iterator: Iterator[V] = new Iterator[V] { + var index = 0 + override def hasNext: Boolean = index < _numElements + override def next(): V = { + if (!hasNext) { + throw new NoSuchElementException + } + val value = _array(index) + index += 1 + value + } + } + /** Gets the underlying array backing this vector. */ def array: Array[V] = _array diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala new file mode 100644 index 0000000000000..3eb1010dc1e8d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTracker.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.collection.mutable + +import org.apache.spark.util.SizeEstimator + +/** + * A general interface for collections to keep track of their estimated sizes in bytes. + * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, + * as each call to SizeEstimator is somewhat expensive (order of a few milliseconds). + */ +private[spark] trait SizeTracker { + + import SizeTracker._ + + /** + * Controls the base of the exponential which governs the rate of sampling. + * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. + */ + private val SAMPLE_GROWTH_RATE = 1.1 + + /** Samples taken since last resetSamples(). Only the last two are kept for extrapolation. */ + private val samples = new mutable.Queue[Sample] + + /** The average number of bytes per update between our last two samples. */ + private var bytesPerUpdate: Double = _ + + /** Total number of insertions and updates into the map since the last resetSamples(). */ + private var numUpdates: Long = _ + + /** The value of 'numUpdates' at which we will take our next sample. */ + private var nextSampleNum: Long = _ + + resetSamples() + + /** + * Reset samples collected so far. + * This should be called after the collection undergoes a dramatic change in size. + */ + protected def resetSamples(): Unit = { + numUpdates = 1 + nextSampleNum = 1 + samples.clear() + takeSample() + } + + /** + * Callback to be invoked after every update. + */ + protected def afterUpdate(): Unit = { + numUpdates += 1 + if (nextSampleNum == numUpdates) { + takeSample() + } + } + + /** + * Take a new sample of the current collection's size. + */ + private def takeSample(): Unit = { + samples.enqueue(Sample(SizeEstimator.estimate(this), numUpdates)) + // Only use the last two samples to extrapolate + if (samples.size > 2) { + samples.dequeue() + } + val bytesDelta = samples.toList.reverse match { + case latest :: previous :: tail => + (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) + // If fewer than 2 samples, assume no change + case _ => 0 + } + bytesPerUpdate = math.max(0, bytesDelta) + nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong + } + + /** + * Estimate the current size of the collection in bytes. O(1) time. + */ + def estimateSize(): Long = { + assert(samples.nonEmpty) + val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) + (samples.last.size + extrapolatedDelta).toLong + } +} + +private object SizeTracker { + case class Sample(size: Long, numUpdates: Long) +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index 204330dad48b9..de61e1d17fe10 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -17,85 +17,24 @@ package org.apache.spark.util.collection -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.util.SizeEstimator -import org.apache.spark.util.collection.SizeTrackingAppendOnlyMap.Sample - /** - * Append-only map that keeps track of its estimated size in bytes. - * We sample with a slow exponential back-off using the SizeEstimator to amortize the time, - * as each call to SizeEstimator can take a sizable amount of time (order of a few milliseconds). + * An append-only map that keeps track of its estimated size in bytes. */ -private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] { - - /** - * Controls the base of the exponential which governs the rate of sampling. - * E.g., a value of 2 would mean we sample at 1, 2, 4, 8, ... elements. - */ - private val SAMPLE_GROWTH_RATE = 1.1 - - /** All samples taken since last resetSamples(). Only the last two are used for extrapolation. */ - private val samples = new ArrayBuffer[Sample]() - - /** Total number of insertions and updates into the map since the last resetSamples(). */ - private var numUpdates: Long = _ - - /** The value of 'numUpdates' at which we will take our next sample. */ - private var nextSampleNum: Long = _ - - /** The average number of bytes per update between our last two samples. */ - private var bytesPerUpdate: Double = _ - - resetSamples() - - /** Called after the map grows in size, as this can be a dramatic change for small objects. */ - def resetSamples() { - numUpdates = 1 - nextSampleNum = 1 - samples.clear() - takeSample() - } +private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] with SizeTracker { override def update(key: K, value: V): Unit = { super.update(key, value) - numUpdates += 1 - if (nextSampleNum == numUpdates) { takeSample() } + super.afterUpdate() } override def changeValue(key: K, updateFunc: (Boolean, V) => V): V = { val newValue = super.changeValue(key, updateFunc) - numUpdates += 1 - if (nextSampleNum == numUpdates) { takeSample() } + super.afterUpdate() newValue } - /** Takes a new sample of the current map's size. */ - def takeSample() { - samples += Sample(SizeEstimator.estimate(this), numUpdates) - // Only use the last two samples to extrapolate. If fewer than 2 samples, assume no change. - bytesPerUpdate = math.max(0, samples.toSeq.reverse match { - case latest :: previous :: tail => - (latest.size - previous.size).toDouble / (latest.numUpdates - previous.numUpdates) - case _ => - 0 - }) - nextSampleNum = math.ceil(numUpdates * SAMPLE_GROWTH_RATE).toLong - } - - override protected def growTable() { + override protected def growTable(): Unit = { super.growTable() resetSamples() } - - /** Estimates the current size of the map in bytes. O(1) time. */ - def estimateSize(): Long = { - assert(samples.nonEmpty) - val extrapolatedDelta = bytesPerUpdate * (numUpdates - samples.last.numUpdates) - (samples.last.size + extrapolatedDelta).toLong - } -} - -private object SizeTrackingAppendOnlyMap { - case class Sample(size: Long, numUpdates: Long) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala new file mode 100644 index 0000000000000..65a7b4e0d497b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingVector.scala @@ -0,0 +1,46 @@ +/* + * 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.util.collection + +import scala.reflect.ClassTag + +/** + * An append-only buffer that keeps track of its estimated size in bytes. + */ +private[spark] class SizeTrackingVector[T: ClassTag] + extends PrimitiveVector[T] + with SizeTracker { + + override def +=(value: T): Unit = { + super.+=(value) + super.afterUpdate() + } + + override def resize(newLength: Int): PrimitiveVector[T] = { + super.resize(newLength) + resetSamples() + this + } + + /** + * Return a trimmed version of the underlying array. + */ + def toArray: Array[T] = { + super.iterator.toArray + } +} diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 7f5d0b061e8b0..9c5f394d3899d 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.collection.mutable.ArrayBuffer - import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar @@ -52,22 +50,21 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } test("get uncached rdd") { - expecting { - blockManager.get(RDDBlockId(0, 0)).andReturn(None) - blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, - true).andStubReturn(Seq[(BlockId, BlockStatus)]()) - } - - whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0) - val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) - assert(value.toList === List(1, 2, 3, 4)) - } + // Do not mock this test, because attempting to match Array[Any], which is not covariant, + // in blockManager.put is a losing battle. You have been warned. + blockManager = sc.env.blockManager + cacheManager = sc.env.cacheManager + val context = new TaskContext(0, 0, 0) + val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) + val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) + assert(computeValue.toList === List(1, 2, 3, 4)) + assert(getValue.isDefined, "Block cached from getOrCompute is not found!") + assert(getValue.get.data.toList === List(1, 2, 3, 4)) } test("get cached rdd") { expecting { - val result = new BlockResult(ArrayBuffer(5, 6, 7).iterator, DataReadMethod.Memory, 12) + val result = new BlockResult(Array(5, 6, 7).iterator, DataReadMethod.Memory, 12) blockManager.get(RDDBlockId(0, 0)).andReturn(Some(result)) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 23cb6905bfdeb..dd4fd535d3577 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -31,7 +31,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.SpanSugar._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} @@ -43,6 +43,7 @@ import scala.language.postfixOps class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { + private val conf = new SparkConf(false) var store: BlockManager = null var store2: BlockManager = null @@ -61,21 +62,29 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) + private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + new BlockManager( + name, actorSystem, master, serializer, maxMem, conf, securityMgr, mapOutputTracker) + } + before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf, - securityManager = securityMgr) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) this.actorSystem = actorSystem - conf.set("spark.driver.port", boundPort.toString) - - master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") conf.set("spark.storage.disableBlockManagerHeartBeat", "true") + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.storage.unrollFraction", "0.4") + conf.set("spark.storage.unrollMemoryThreshold", "512") + + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) + val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() } @@ -138,11 +147,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -169,10 +177,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000, "exec1") + store2 = makeBlockManager(2000, "exec2") val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -187,11 +193,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory and telling master only about a1 and a2 store.putSingle("a1-to-remove", a1, StorageLevel.MEMORY_ONLY) @@ -200,8 +205,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Checking whether blocks are in memory and memory size val memStatus = master.getMemoryStatus.head._2 - assert(memStatus._1 == 2000L, "total memory " + memStatus._1 + " should equal 2000") - assert(memStatus._2 <= 1200L, "remaining memory " + memStatus._2 + " should <= 1200") + assert(memStatus._1 == 20000L, "total memory " + memStatus._1 + " should equal 20000") + assert(memStatus._2 <= 12000L, "remaining memory " + memStatus._2 + " should <= 12000") assert(store.getSingle("a1-to-remove").isDefined, "a1 was not in store") assert(store.getSingle("a2-to-remove").isDefined, "a2 was not in store") assert(store.getSingle("a3-to-remove").isDefined, "a3 was not in store") @@ -230,17 +235,16 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { val memStatus = master.getMemoryStatus.head._2 - memStatus._1 should equal (2000L) - memStatus._2 should equal (2000L) + memStatus._1 should equal (20000L) + memStatus._2 should equal (20000L) } } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(20000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) // Putting a1, a2 and a3 in memory. store.putSingle(rdd(0, 0), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 1), a2, StorageLevel.MEMORY_ONLY) @@ -270,11 +274,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("removing broadcast") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val driverStore = store - val executorStore = new BlockManager("executor", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + val executorStore = makeBlockManager(2000, "executor") val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -343,8 +345,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -380,8 +381,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -390,7 +390,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master.removeExecutor(store.blockManagerId.executorId) val t1 = new Thread { override def run() { - store.put("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("a2", a2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } } val t2 = new Thread { @@ -418,19 +418,14 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("correct BlockResult returned from get() calls") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr, - mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list1ForSizeEstimate = new ArrayBuffer[Any] - list1ForSizeEstimate ++= list1.iterator - val list1SizeEstimate = SizeEstimator.estimate(list1ForSizeEstimate) - val list2 = List(new Array[Byte](50), new Array[Byte](100), new Array[Byte](150)) - val list2ForSizeEstimate = new ArrayBuffer[Any] - list2ForSizeEstimate ++= list2.iterator - val list2SizeEstimate = SizeEstimator.estimate(list2ForSizeEstimate) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) + val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) + val list2SizeEstimate = SizeEstimator.estimate(list2.iterator.toArray) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2memory", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2disk", list2.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val list1Get = store.get("list1") assert(list1Get.isDefined, "list1 expected to be in store") assert(list1Get.get.data.size === 2) @@ -451,11 +446,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY) @@ -471,11 +465,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_ONLY_SER) @@ -491,11 +484,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle(rdd(0, 1), a1, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), a2, StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 3), a3, StorageLevel.MEMORY_ONLY) @@ -511,11 +503,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(12000) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // At this point rdd_1_1 should've replaced rdd_0_1 assert(store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was not in store") assert(!store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was in store") @@ -523,8 +514,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Do a get() on rdd_0_2 so that it is the most recently used item assert(store.getSingle(rdd(0, 2)).isDefined, "rdd_0_2 was not in store") // Put in more partitions from RDD 0; they should replace rdd_1_1 - store.putSingle(rdd(0, 3), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(0, 4), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 3), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 4), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Now rdd_1_1 should be dropped to add rdd_0_3, but then rdd_0_2 should *not* be dropped // when we try to add rdd_0_4. assert(!store.memoryStore.contains(rdd(1, 1)), "rdd_1_1 was in store") @@ -538,8 +529,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -555,8 +545,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -569,11 +558,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -585,11 +573,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK) @@ -601,11 +588,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -617,11 +603,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) store.putSingle("a1", a1, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_AND_DISK_SER) store.putSingle("a3", a3, StorageLevel.MEMORY_AND_DISK_SER) @@ -633,12 +618,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val a1 = new Array[Byte](400) - val a2 = new Array[Byte](400) - val a3 = new Array[Byte](400) - val a4 = new Array[Byte](400) + store = makeBlockManager(12000) + val a1 = new Array[Byte](4000) + val a2 = new Array[Byte](4000) + val a3 = new Array[Byte](4000) + val a4 = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) @@ -656,14 +640,13 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) assert(store.get("list3").isDefined, "list3 was not in store") @@ -672,7 +655,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) // At this point list2 was gotten last, so LRU will getSingle rid of list3 - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.get("list1").isDefined, "list1 was not in store") assert(store.get("list1").get.data.size === 2) assert(store.get("list2").isDefined, "list2 was not in store") @@ -681,16 +664,15 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list1 = List(new Array[Byte](200), new Array[Byte](200)) - val list2 = List(new Array[Byte](200), new Array[Byte](200)) - val list3 = List(new Array[Byte](200), new Array[Byte](200)) - val list4 = List(new Array[Byte](200), new Array[Byte](200)) + store = makeBlockManager(12000) + val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) + val list4 = List(new Array[Byte](2000), new Array[Byte](2000)) // First store list1 and list2, both in memory, and list3, on disk only - store.put("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) - store.put("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) + store.putIterator("list1", list1.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list2", list2.iterator, StorageLevel.MEMORY_ONLY_SER, tellMaster = true) + store.putIterator("list3", list3.iterator, StorageLevel.DISK_ONLY, tellMaster = true) val listForSizeEstimate = new ArrayBuffer[Any] listForSizeEstimate ++= list1.iterator val listSize = SizeEstimator.estimate(listForSizeEstimate) @@ -708,7 +690,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.get("list3").isDefined, "list3 was not in store") assert(store.get("list3").get.data.size === 2) // Now let's add in list4, which uses both disk and memory; list1 should drop out - store.put("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) + store.putIterator("list4", list4.iterator, StorageLevel.MEMORY_AND_DISK_SER, tellMaster = true) assert(store.get("list1") === None, "list1 was in store") assert(store.get("list2").isDefined, "list2 was not in store") assert(store.get("list2").get.data.size === 2) @@ -731,11 +713,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf, - securityMgr, mapOutputTracker) - store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(5000) + store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") - store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) + store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) assert(store.memoryStore.getValues("a2") === None, "a2 was in memory store") assert(store.getSingle("a2").isDefined, "a2 was not in store") } @@ -743,8 +724,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(20000, "exec1") store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -752,52 +732,46 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, + store = makeBlockManager(20000, "exec2") + store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") store.stop() store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, + store = makeBlockManager(20000, "exec3") + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") store.stop() store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") + store = makeBlockManager(20000, "exec4") + store.putSingle(BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") + store = makeBlockManager(20000, "exec5") + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) - assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") + store = makeBlockManager(20000, "exec6") + store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) + assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) - store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) - assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") + store = makeBlockManager(20000, "exec7") + store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) + assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") store.stop() store = null } finally { @@ -871,30 +845,29 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(Arrays.equals(mappedAsArray, bytes)) assert(Arrays.equals(notMappedAsArray, bytes)) } - + test("updated block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) - val bigList = List.fill(8)(new Array[Byte](200)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](2000)) + val bigList = List.fill(8)(new Array[Byte](2000)) // 1 updated block (i.e. list1) val updatedBlocks1 = - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks1.size === 1) assert(updatedBlocks1.head._1 === TestBlockId("list1")) assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 1 updated block (i.e. list2) val updatedBlocks2 = - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) assert(updatedBlocks2.size === 1) assert(updatedBlocks2.head._1 === TestBlockId("list2")) assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) // 2 updated blocks - list1 is kicked out of memory while list3 is added val updatedBlocks3 = - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks3.size === 2) updatedBlocks3.foreach { case (id, status) => id match { @@ -903,11 +876,11 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list1 nor list3") } } - assert(store.get("list3").isDefined, "list3 was not in store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added val updatedBlocks4 = - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks4.size === 2) updatedBlocks4.foreach { case (id, status) => id match { @@ -916,26 +889,37 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter case _ => fail("Updated block is neither list2 nor list4") } } - assert(store.get("list4").isDefined, "list4 was not in store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") - // No updated blocks - nothing is kicked out of memory because list5 is too big to be added + // No updated blocks - list5 is too big to fit in store and nothing is kicked out val updatedBlocks5 = - store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(updatedBlocks5.size === 0) - assert(store.get("list2").isDefined, "list2 was not in store") - assert(store.get("list4").isDefined, "list4 was not in store") - assert(!store.get("list5").isDefined, "list5 was in store") + + // memory store contains only list3 and list4 + assert(!store.memoryStore.contains("list1"), "list1 was in memory store") + assert(!store.memoryStore.contains("list2"), "list2 was in memory store") + assert(store.memoryStore.contains("list3"), "list3 was not in memory store") + assert(store.memoryStore.contains("list4"), "list4 was not in memory store") + assert(!store.memoryStore.contains("list5"), "list5 was in memory store") + + // disk store contains only list2 + assert(!store.diskStore.contains("list1"), "list1 was in disk store") + assert(store.diskStore.contains("list2"), "list2 was not in disk store") + assert(!store.diskStore.contains("list3"), "list3 was in disk store") + assert(!store.diskStore.contains("list4"), "list4 was in disk store") + assert(!store.diskStore.contains("list5"), "list5 was in disk store") } test("query block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](200)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. - store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getLocations("list1").size === 0) @@ -949,9 +933,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. - store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) - store.put("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.putIterator("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) // getLocations should return nothing because the master is not informed // getBlockStatus without asking slaves should have the same result @@ -968,23 +952,22 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("get matching blocks") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - val list = List.fill(2)(new Array[Byte](10)) + store = makeBlockManager(12000) + val list = List.fill(2)(new Array[Byte](100)) // insert some blocks - store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) // insert some more blocks - store.put("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - store.put("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - store.put("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.putIterator("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) @@ -992,7 +975,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => - store.put(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.putIterator(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) } val matchedBlockIds = store.master.getMatchingBlockIds(_ match { case RDDBlockId(1, _) => true @@ -1002,17 +985,240 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) - store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) - store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store = makeBlockManager(12000) + store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. assert(store.getSingle(rdd(1, 0)).isDefined, "rdd_1_0 was not in store") // According to the same-RDD rule, rdd_1_0 should be replaced here. - store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // rdd_1_0 should have been replaced, even it's not least recently used. assert(store.memoryStore.contains(rdd(0, 0)), "rdd_0_0 was not in store") assert(store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was not in store") assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } + + test("reserve/release unroll memory") { + store = makeBlockManager(12000) + val memoryStore = store.memoryStore + assert(memoryStore.currentUnrollMemory === 0) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Reserve + memoryStore.reserveUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 100) + memoryStore.reserveUnrollMemoryForThisThread(200) + assert(memoryStore.currentUnrollMemoryForThisThread === 300) + memoryStore.reserveUnrollMemoryForThisThread(500) + assert(memoryStore.currentUnrollMemoryForThisThread === 800) + memoryStore.reserveUnrollMemoryForThisThread(1000000) + assert(memoryStore.currentUnrollMemoryForThisThread === 800) // not granted + // Release + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 700) + memoryStore.releaseUnrollMemoryForThisThread(100) + assert(memoryStore.currentUnrollMemoryForThisThread === 600) + // Reserve again + memoryStore.reserveUnrollMemoryForThisThread(4400) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) + memoryStore.reserveUnrollMemoryForThisThread(20000) + assert(memoryStore.currentUnrollMemoryForThisThread === 5000) // not granted + // Release again + memoryStore.releaseUnrollMemoryForThisThread(1000) + assert(memoryStore.currentUnrollMemoryForThisThread === 4000) + memoryStore.releaseUnrollMemoryForThisThread() // release all + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + } + + /** + * Verify the result of MemoryStore#unrollSafely is as expected. + */ + private def verifyUnroll( + expected: Iterator[Any], + result: Either[Array[Any], Iterator[Any]], + shouldBeArray: Boolean): Unit = { + val actual: Iterator[Any] = result match { + case Left(arr: Array[Any]) => + assert(shouldBeArray, "expected iterator from unroll!") + arr.iterator + case Right(it: Iterator[Any]) => + assert(!shouldBeArray, "expected array from unroll!") + it + case _ => + fail("unroll returned neither an iterator nor an array...") + } + expected.zip(actual).foreach { case (e, a) => + assert(e === a, "unroll did not return original values!") + } + } + + test("safely unroll blocks") { + store = makeBlockManager(12000) + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + val memoryStore = store.memoryStore + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with all the space in the world. This should succeed and return an array. + var unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with not enough space. This should succeed after kicking out someBlock1. + store.putIterator("someBlock1", smallList.iterator, StorageLevel.MEMORY_ONLY) + store.putIterator("someBlock2", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", smallList.iterator, droppedBlocks) + verifyUnroll(smallList.iterator, unrollResult, shouldBeArray = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock1")) + droppedBlocks.clear() + + // Unroll huge block with not enough space. Even after ensuring free space of 12000 * 0.4 = + // 4800 bytes, there is still not enough room to unroll this block. This returns an iterator. + // In the mean time, however, we kicked out someBlock2 before giving up. + store.putIterator("someBlock3", smallList.iterator, StorageLevel.MEMORY_ONLY) + unrollResult = memoryStore.unrollSafely("unroll", bigList.iterator, droppedBlocks) + verifyUnroll(bigList.iterator, unrollResult, shouldBeArray = false) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + assert(droppedBlocks.size === 1) + assert(droppedBlocks.head._1 === TestBlockId("someBlock2")) + droppedBlocks.clear() + } + + test("safely unroll blocks through putIterator") { + store = makeBlockManager(12000) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll with plenty of space. This should succeed and cache both blocks. + val result1 = memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + val result2 = memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(result1.size > 0) // unroll was successful + assert(result2.size > 0) + assert(result1.data.isLeft) // unroll did not drop this block to disk + assert(result2.data.isLeft) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Re-put these two blocks so block manager knows about them too. Otherwise, block manager + // would not know how to drop them from memory later. + memoryStore.remove("b1") + memoryStore.remove("b2") + store.putIterator("b1", smallIterator, memOnly) + store.putIterator("b2", smallIterator, memOnly) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + val result3 = memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + assert(result3.size > 0) + assert(result3.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, memOnly) + + // Unroll huge block with not enough space. This should fail and kick out b2 in the process. + val result4 = memoryStore.putIterator("b4", bigIterator, memOnly, returnValues = true) + assert(result4.size === 0) // unroll was unsuccessful + assert(result4.data.isLeft) + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + } + + /** + * This test is essentially identical to the preceding one, except that it uses MEMORY_AND_DISK. + */ + test("safely unroll blocks through putIterator (disk)") { + store = makeBlockManager(12000) + val memAndDisk = StorageLevel.MEMORY_AND_DISK + val memoryStore = store.memoryStore + val diskStore = store.diskStore + val smallList = List.fill(40)(new Array[Byte](100)) + val bigList = List.fill(40)(new Array[Byte](1000)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + store.putIterator("b1", smallIterator, memAndDisk) + store.putIterator("b2", smallIterator, memAndDisk) + + // Unroll with not enough space. This should succeed but kick out b1 in the process. + // Memory store should contain b2 and b3, while disk store should contain only b1 + val result3 = memoryStore.putIterator("b3", smallIterator, memAndDisk, returnValues = true) + assert(result3.size > 0) + assert(!memoryStore.contains("b1")) + assert(memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(diskStore.contains("b1")) + assert(!diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + memoryStore.remove("b3") + store.putIterator("b3", smallIterator, StorageLevel.MEMORY_ONLY) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll huge block with not enough space. This should fail and drop the new block to disk + // directly in addition to kicking out b2 in the process. Memory store should contain only + // b3, while disk store should contain b1, b2 and b4. + val result4 = memoryStore.putIterator("b4", bigIterator, memAndDisk, returnValues = true) + assert(result4.size > 0) + assert(result4.data.isRight) // unroll returned bytes from disk + assert(!memoryStore.contains("b1")) + assert(!memoryStore.contains("b2")) + assert(memoryStore.contains("b3")) + assert(!memoryStore.contains("b4")) + assert(diskStore.contains("b1")) + assert(diskStore.contains("b2")) + assert(!diskStore.contains("b3")) + assert(diskStore.contains("b4")) + assert(memoryStore.currentUnrollMemoryForThisThread > 0) // we returned an iterator + } + + test("multiple unrolls by the same thread") { + store = makeBlockManager(12000) + val memOnly = StorageLevel.MEMORY_ONLY + val memoryStore = store.memoryStore + val smallList = List.fill(40)(new Array[Byte](100)) + def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // All unroll memory used is released because unrollSafely returned an array + memoryStore.putIterator("b1", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + memoryStore.putIterator("b2", smallIterator, memOnly, returnValues = true) + assert(memoryStore.currentUnrollMemoryForThisThread === 0) + + // Unroll memory is not released because unrollSafely returned an iterator + // that still depends on the underlying vector used in the process + memoryStore.putIterator("b3", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB3 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB3 > 0) + + // The unroll memory owned by this thread builds on top of its value after the previous unrolls + memoryStore.putIterator("b4", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB4 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB4 > unrollMemoryAfterB3) + + // ... but only to a certain extent (until we run out of free space to grant new unroll memory) + memoryStore.putIterator("b5", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB5 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b6", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB6 = memoryStore.currentUnrollMemoryForThisThread + memoryStore.putIterator("b7", smallIterator, memOnly, returnValues = true) + val unrollMemoryAfterB7 = memoryStore.currentUnrollMemoryForThisThread + assert(unrollMemoryAfterB5 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB6 === unrollMemoryAfterB4) + assert(unrollMemoryAfterB7 === unrollMemoryAfterB4) + } } diff --git a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala deleted file mode 100644 index 93f0c6a8e6408..0000000000000 --- a/core/src/test/scala/org/apache/spark/util/SizeTrackingAppendOnlyMapSuite.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import scala.util.Random - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.util.SizeTrackingAppendOnlyMapSuite.LargeDummyClass -import org.apache.spark.util.collection.{AppendOnlyMap, SizeTrackingAppendOnlyMap} - -class SizeTrackingAppendOnlyMapSuite extends FunSuite with BeforeAndAfterAll { - val NORMAL_ERROR = 0.20 - val HIGH_ERROR = 0.30 - - test("fixed size insertions") { - testWith[Int, Long](10000, i => (i, i.toLong)) - testWith[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) - testWith[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass())) - } - - test("variable size insertions") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[Int, String](10000, i => (i, randString(0, 10))) - testWith[Int, String](10000, i => (i, randString(0, 100))) - testWith[Int, String](10000, i => (i, randString(90, 100))) - } - - test("updates") { - val rand = new Random(123456789) - def randString(minLen: Int, maxLen: Int): String = { - "a" * (rand.nextInt(maxLen - minLen) + minLen) - } - testWith[String, Int](10000, i => (randString(0, 10000), i)) - } - - def testWith[K, V](numElements: Int, makeElement: (Int) => (K, V)) { - val map = new SizeTrackingAppendOnlyMap[K, V]() - for (i <- 0 until numElements) { - val (k, v) = makeElement(i) - map(k) = v - expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) - } - } - - def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { - val betterEstimatedSize = SizeEstimator.estimate(obj) - assert(betterEstimatedSize * (1 - error) < estimatedSize, - s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") - assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, - s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") - } -} - -object SizeTrackingAppendOnlyMapSuite { - // Speed test, for reproducibility of results. - // These could be highly non-deterministic in general, however. - // Results: - // AppendOnlyMap: 31 ms - // SizeTracker: 54 ms - // SizeEstimator: 1500 ms - def main(args: Array[String]) { - val numElements = 100000 - - val baseTimes = for (i <- 0 until 10) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - } - } - - val sampledTimes = for (i <- 0 until 10) yield time { - val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - map.estimateSize() - } - } - - val unsampledTimes = for (i <- 0 until 3) yield time { - val map = new AppendOnlyMap[Int, LargeDummyClass]() - for (i <- 0 until numElements) { - map(i) = new LargeDummyClass() - SizeEstimator.estimate(map) - } - } - - println("Base: " + baseTimes) - println("SizeTracker (sampled): " + sampledTimes) - println("SizeEstimator (unsampled): " + unsampledTimes) - } - - def time(f: => Unit): Long = { - val start = System.currentTimeMillis() - f - System.currentTimeMillis() - start - } - - private class LargeDummyClass { - val arr = new Array[Int](100) - } -} diff --git a/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala new file mode 100644 index 0000000000000..1f33967249654 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/SizeTrackerSuite.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.util.SizeEstimator + +class SizeTrackerSuite extends FunSuite { + val NORMAL_ERROR = 0.20 + val HIGH_ERROR = 0.30 + + import SizeTrackerSuite._ + + test("vector fixed size insertions") { + testVector[Long](10000, i => i.toLong) + testVector[(Long, Long)](10000, i => (i.toLong, i.toLong)) + testVector[LargeDummyClass](10000, i => new LargeDummyClass) + } + + test("vector variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testVector[String](10000, i => randString(0, 10)) + testVector[String](10000, i => randString(0, 100)) + testVector[String](10000, i => randString(90, 100)) + } + + test("map fixed size insertions") { + testMap[Int, Long](10000, i => (i, i.toLong)) + testMap[Int, (Long, Long)](10000, i => (i, (i.toLong, i.toLong))) + testMap[Int, LargeDummyClass](10000, i => (i, new LargeDummyClass)) + } + + test("map variable size insertions") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[Int, String](10000, i => (i, randString(0, 10))) + testMap[Int, String](10000, i => (i, randString(0, 100))) + testMap[Int, String](10000, i => (i, randString(90, 100))) + } + + test("map updates") { + val rand = new Random(123456789) + def randString(minLen: Int, maxLen: Int): String = { + "a" * (rand.nextInt(maxLen - minLen) + minLen) + } + testMap[String, Int](10000, i => (randString(0, 10000), i)) + } + + def testVector[T: ClassTag](numElements: Int, makeElement: Int => T) { + val vector = new SizeTrackingVector[T] + for (i <- 0 until numElements) { + val item = makeElement(i) + vector += item + expectWithinError(vector, vector.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def testMap[K, V](numElements: Int, makeElement: (Int) => (K, V)) { + val map = new SizeTrackingAppendOnlyMap[K, V] + for (i <- 0 until numElements) { + val (k, v) = makeElement(i) + map(k) = v + expectWithinError(map, map.estimateSize(), if (i < 32) HIGH_ERROR else NORMAL_ERROR) + } + } + + def expectWithinError(obj: AnyRef, estimatedSize: Long, error: Double) { + val betterEstimatedSize = SizeEstimator.estimate(obj) + assert(betterEstimatedSize * (1 - error) < estimatedSize, + s"Estimated size $estimatedSize was less than expected size $betterEstimatedSize") + assert(betterEstimatedSize * (1 + 2 * error) > estimatedSize, + s"Estimated size $estimatedSize was greater than expected size $betterEstimatedSize") + } +} + +private object SizeTrackerSuite { + + /** + * Run speed tests for size tracking collections. + */ + def main(args: Array[String]): Unit = { + if (args.size < 1) { + println("Usage: SizeTrackerSuite [num elements]") + System.exit(1) + } + val numElements = args(0).toInt + vectorSpeedTest(numElements) + mapSpeedTest(numElements) + } + + /** + * Speed test for SizeTrackingVector. + * + * Results for 100000 elements (possibly non-deterministic): + * PrimitiveVector 15 ms + * SizeTracker 51 ms + * SizeEstimator 2000 ms + */ + def vectorSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val vector = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val vector = new SizeTrackingVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + vector.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val vector = new PrimitiveVector[LargeDummyClass] + for (i <- 0 until numElements) { + vector += new LargeDummyClass + SizeEstimator.estimate(vector) + } + } + printSpeedTestResult("SizeTrackingVector", baseTimes, sampledTimes, unsampledTimes) + } + + /** + * Speed test for SizeTrackingAppendOnlyMap. + * + * Results for 100000 elements (possibly non-deterministic): + * AppendOnlyMap 30 ms + * SizeTracker 41 ms + * SizeEstimator 1666 ms + */ + def mapSpeedTest(numElements: Int): Unit = { + val baseTimes = for (i <- 0 until 10) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + } + } + val sampledTimes = for (i <- 0 until 10) yield time { + val map = new SizeTrackingAppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + map.estimateSize() + } + } + val unsampledTimes = for (i <- 0 until 3) yield time { + val map = new AppendOnlyMap[Int, LargeDummyClass] + for (i <- 0 until numElements) { + map(i) = new LargeDummyClass + SizeEstimator.estimate(map) + } + } + printSpeedTestResult("SizeTrackingAppendOnlyMap", baseTimes, sampledTimes, unsampledTimes) + } + + def printSpeedTestResult( + testName: String, + baseTimes: Seq[Long], + sampledTimes: Seq[Long], + unsampledTimes: Seq[Long]): Unit = { + println(s"Average times for $testName (ms):") + println(" Base - " + averageTime(baseTimes)) + println(" SizeTracker (sampled) - " + averageTime(sampledTimes)) + println(" SizeEstimator (unsampled) - " + averageTime(unsampledTimes)) + println() + } + + def time(f: => Unit): Long = { + val start = System.currentTimeMillis() + f + System.currentTimeMillis() - start + } + + def averageTime(v: Seq[Long]): Long = { + v.sum / v.size + } + + private class LargeDummyClass { + val arr = new Array[Int](100) + } +} diff --git a/docs/configuration.md b/docs/configuration.md index 46e3dd914b5ac..2e6c85cc2bcca 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -480,6 +480,15 @@ Apart from these, the following properties are also available, and may be useful increase it if you configure your own old generation size. + + spark.storage.unrollFraction + 0.2 + + Fraction of spark.storage.memoryFraction to use for unrolling blocks in memory. + This is dynamically allocated by dropping existing blocks when there is not enough free + storage space to unroll the new block in its entirety. + + spark.tachyonStore.baseDir System.getProperty("java.io.tmpdir") diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index e9220db6b1f9a..5ff88f0dd1cac 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -31,7 +31,6 @@ import com.typesafe.tools.mima.core._ * MimaBuild.excludeSparkClass("graphx.util.collection.GraphXPrimitiveKeyOpenHashMap") */ object MimaExcludes { - def excludes(version: String) = version match { case v if v.startsWith("1.1") => @@ -62,6 +61,15 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry") ) ++ + Seq( + // Renamed putValues -> putArray + putIterator + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.MemoryStore.putValues"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.DiskStore.putValues"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.storage.TachyonStore.putValues") + ) ++ Seq( ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.FlumeReceiver.this") ) ++ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index ce8316bb14891..d934b9cbfc3e8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -110,8 +110,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], - storageLevel, tellMaster = true) + blockManager.putArray(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) } @@ -124,7 +123,7 @@ private[streaming] class ReceiverSupervisorImpl( ) { val blockId = optionalBlockId.getOrElse(nextBlockId) val time = System.currentTimeMillis - blockManager.put(blockId, iterator, storageLevel, tellMaster = true) + blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") reportPushedBlock(blockId, -1, optionalMetadata) } From 81fcdd22c8ef52889ed51b3ec5c2747708505fc2 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Sun, 27 Jul 2014 16:16:39 -0700 Subject: [PATCH 0095/1492] [SPARK-2514] [mllib] Random RDD generator Utilities for generating random RDDs. RandomRDD and RandomVectorRDD are created instead of using `sc.parallelize(range:Range)` because `Range` objects in Scala can only have `size <= Int.MaxValue`. The object `RandomRDDGenerators` can be transformed into a generator class to reduce the number of auxiliary methods for optional arguments. Author: Doris Xin Closes #1520 from dorx/randomRDD and squashes the following commits: 01121ac [Doris Xin] reviewer comments 6bf27d8 [Doris Xin] Merge branch 'master' into randomRDD a8ea92d [Doris Xin] Reviewer comments 063ea0b [Doris Xin] Merge branch 'master' into randomRDD aec68eb [Doris Xin] newline bc90234 [Doris Xin] units passed. d56cacb [Doris Xin] impl with RandomRDD 92d6f1c [Doris Xin] solution for Cloneable df5bcff [Doris Xin] Merge branch 'generator' into randomRDD f46d928 [Doris Xin] WIP 49ed20d [Doris Xin] alternative poisson distribution generator 7cb0e40 [Doris Xin] fix for data inconsistency 8881444 [Doris Xin] RandomRDDGenerator: initial design --- .../mllib/random/DistributionGenerator.scala | 101 ++++ .../mllib/random/RandomRDDGenerators.scala | 473 ++++++++++++++++++ .../apache/spark/mllib/rdd/RandomRDD.scala | 118 +++++ .../random/DistributionGeneratorSuite.scala | 90 ++++ .../random/RandomRDDGeneratorsSuite.scala | 158 ++++++ 5 files changed, 940 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala new file mode 100644 index 0000000000000..7ecb409c4a91a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand + +import org.apache.spark.annotation.Experimental +import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} + +/** + * :: Experimental :: + * Trait for random number generators that generate i.i.d. values from a distribution. + */ +@Experimental +trait DistributionGenerator extends Pseudorandom with Serializable { + + /** + * Returns an i.i.d. sample as a Double from an underlying distribution. + */ + def nextValue(): Double + + /** + * Returns a copy of the DistributionGenerator with a new instance of the rng object used in the + * class when applicable for non-locking concurrent usage. + */ + def copy(): DistributionGenerator +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from U[0.0, 1.0] + */ +@Experimental +class UniformGenerator extends DistributionGenerator { + + // XORShiftRandom for better performance. Thread safety isn't necessary here. + private val random = new XORShiftRandom() + + override def nextValue(): Double = { + random.nextDouble() + } + + override def setSeed(seed: Long) = random.setSeed(seed) + + override def copy(): UniformGenerator = new UniformGenerator() +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from the standard normal distribution. + */ +@Experimental +class StandardNormalGenerator extends DistributionGenerator { + + // XORShiftRandom for better performance. Thread safety isn't necessary here. + private val random = new XORShiftRandom() + + override def nextValue(): Double = { + random.nextGaussian() + } + + override def setSeed(seed: Long) = random.setSeed(seed) + + override def copy(): StandardNormalGenerator = new StandardNormalGenerator() +} + +/** + * :: Experimental :: + * Generates i.i.d. samples from the Poisson distribution with the given mean. + * + * @param mean mean for the Poisson distribution. + */ +@Experimental +class PoissonGenerator(val mean: Double) extends DistributionGenerator { + + private var rng = new Poisson(mean, new DRand) + + override def nextValue(): Double = rng.nextDouble() + + override def setSeed(seed: Long) { + rng = new Poisson(mean, new DRand(seed.toInt)) + } + + override def copy(): PoissonGenerator = new PoissonGenerator(mean) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala new file mode 100644 index 0000000000000..d7ee2d3f46846 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala @@ -0,0 +1,473 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +/** + * :: Experimental :: + * Generator methods for creating RDDs comprised of i.i.d samples from some distribution. + */ +@Experimental +object RandomRDDGenerators { + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + val uniform = new UniformGenerator() + randomRDD(sc, uniform, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { + uniformRDD(sc, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformRDD(sc: SparkContext, size: Long): RDD[Double] = { + uniformRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + val normal = new StandardNormalGenerator() + randomRDD(sc, normal, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { + normalRDD(sc, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + */ + @Experimental + def normalRDD(sc: SparkContext, size: Long): RDD[Double] = { + normalRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, + mean: Double, + size: Long, + numPartitions: Int, + seed: Long): RDD[Double] = { + val poisson = new PoissonGenerator(mean) + randomRDD(sc, poisson, size, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, mean: Double, size: Long, numPartitions: Int): RDD[Double] = { + poissonRDD(sc, mean, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + */ + @Experimental + def poissonRDD(sc: SparkContext, mean: Double, size: Long): RDD[Double] = { + poissonRDD(sc, mean, size, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD(sc: SparkContext, + generator: DistributionGenerator, + size: Long, + numPartitions: Int, + seed: Long): RDD[Double] = { + new RandomRDD(sc, size, numPartitions, generator, seed) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD(sc: SparkContext, + generator: DistributionGenerator, + size: Long, + numPartitions: Int): RDD[Double] = { + randomRDD(sc, generator, size, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param size Size of the RDD. + * @return RDD[Double] comprised of i.i.d. samples produced by generator. + */ + @Experimental + def randomRDD(sc: SparkContext, + generator: DistributionGenerator, + size: Long): RDD[Double] = { + randomRDD(sc, generator, size, sc.defaultParallelism, Utils.random.nextLong) + } + + // TODO Generate RDD[Vector] from multivariate distributions. + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * uniform distribution on [0.0 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val uniform = new UniformGenerator() + randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * uniform distribution on [0.0 1.0]. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + uniformVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * uniform distribution on [0.0 1.0]. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + */ + @Experimental + def uniformVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { + uniformVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val uniform = new StandardNormalGenerator() + randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * standard normal distribution. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + normalVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * standard normal distribution. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + */ + @Experimental + def normalVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { + normalVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + val poisson = new PoissonGenerator(mean) + randomVectorRDD(sc, poisson, numRows, numCols, numPartitions, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Poisson distribution with the input mean. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + poissonVectorRDD(sc, mean, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Poisson distribution with the input mean. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param mean Mean, or lambda, for the Poisson distribution. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + */ + @Experimental + def poissonVectorRDD(sc: SparkContext, + mean: Double, + numRows: Long, + numCols: Int): RDD[Vector] = { + poissonVectorRDD(sc, mean, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: DistributionGenerator, + numRows: Long, + numCols: Int, + numPartitions: Int, + seed: Long): RDD[Vector] = { + new RandomVectorRDD(sc, numRows, numCols, numPartitions, generator, seed) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * input DistributionGenerator. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @param numPartitions Number of partitions in the RDD. + * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: DistributionGenerator, + numRows: Long, + numCols: Int, + numPartitions: Int): RDD[Vector] = { + randomVectorRDD(sc, generator, numRows, numCols, numPartitions, Utils.random.nextLong) + } + + /** + * :: Experimental :: + * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * input DistributionGenerator. + * sc.defaultParallelism used for the number of partitions in the RDD. + * + * @param sc SparkContext used to create the RDD. + * @param generator DistributionGenerator used to populate the RDD. + * @param numRows Number of Vectors in the RDD. + * @param numCols Number of elements in each Vector. + * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + */ + @Experimental + def randomVectorRDD(sc: SparkContext, + generator: DistributionGenerator, + numRows: Long, + numCols: Int): RDD[Vector] = { + randomVectorRDD(sc, generator, numRows, numCols, + sc.defaultParallelism, Utils.random.nextLong) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala new file mode 100644 index 0000000000000..f13282d07ff92 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala @@ -0,0 +1,118 @@ +/* + * 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.rdd + +import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.mllib.linalg.{DenseVector, Vector} +import org.apache.spark.mllib.random.DistributionGenerator +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +import scala.util.Random + +private[mllib] class RandomRDDPartition(override val index: Int, + val size: Int, + val generator: DistributionGenerator, + val seed: Long) extends Partition { + + require(size >= 0, "Non-negative partition size required.") +} + +// These two classes are necessary since Range objects in Scala cannot have size > Int.MaxValue +private[mllib] class RandomRDD(@transient sc: SparkContext, + size: Long, + numPartitions: Int, + @transient rng: DistributionGenerator, + @transient seed: Long = Utils.random.nextLong) extends RDD[Double](sc, Nil) { + + require(size > 0, "Positive RDD size required.") + require(numPartitions > 0, "Positive number of partitions required") + require(math.ceil(size.toDouble / numPartitions) <= Int.MaxValue, + "Partition size cannot exceed Int.MaxValue") + + override def compute(splitIn: Partition, context: TaskContext): Iterator[Double] = { + val split = splitIn.asInstanceOf[RandomRDDPartition] + RandomRDD.getPointIterator(split) + } + + override def getPartitions: Array[Partition] = { + RandomRDD.getPartitions(size, numPartitions, rng, seed) + } +} + +private[mllib] class RandomVectorRDD(@transient sc: SparkContext, + size: Long, + vectorSize: Int, + numPartitions: Int, + @transient rng: DistributionGenerator, + @transient seed: Long = Utils.random.nextLong) extends RDD[Vector](sc, Nil) { + + require(size > 0, "Positive RDD size required.") + require(numPartitions > 0, "Positive number of partitions required") + require(vectorSize > 0, "Positive vector size required.") + require(math.ceil(size.toDouble / numPartitions) <= Int.MaxValue, + "Partition size cannot exceed Int.MaxValue") + + override def compute(splitIn: Partition, context: TaskContext): Iterator[Vector] = { + val split = splitIn.asInstanceOf[RandomRDDPartition] + RandomRDD.getVectorIterator(split, vectorSize) + } + + override protected def getPartitions: Array[Partition] = { + RandomRDD.getPartitions(size, numPartitions, rng, seed) + } +} + +private[mllib] object RandomRDD { + + def getPartitions(size: Long, + numPartitions: Int, + rng: DistributionGenerator, + seed: Long): Array[Partition] = { + + val partitions = new Array[RandomRDDPartition](numPartitions) + var i = 0 + var start: Long = 0 + var end: Long = 0 + val random = new Random(seed) + while (i < numPartitions) { + end = ((i + 1) * size) / numPartitions + partitions(i) = new RandomRDDPartition(i, (end - start).toInt, rng, random.nextLong()) + start = end + i += 1 + } + partitions.asInstanceOf[Array[Partition]] + } + + // The RNG has to be reset every time the iterator is requested to guarantee same data + // every time the content of the RDD is examined. + def getPointIterator(partition: RandomRDDPartition): Iterator[Double] = { + val generator = partition.generator.copy() + generator.setSeed(partition.seed) + Array.fill(partition.size)(generator.nextValue()).toIterator + } + + // The RNG has to be reset every time the iterator is requested to guarantee same data + // every time the content of the RDD is examined. + def getVectorIterator(partition: RandomRDDPartition, vectorSize: Int): Iterator[Vector] = { + val generator = partition.generator.copy() + generator.setSeed(partition.seed) + Array.fill(partition.size)(new DenseVector( + (0 until vectorSize).map { _ => generator.nextValue() }.toArray)).toIterator + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala new file mode 100644 index 0000000000000..974dec4c0b5ee --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.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.random + +import org.scalatest.FunSuite + +import org.apache.spark.util.StatCounter + +// TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged +class DistributionGeneratorSuite extends FunSuite { + + def apiChecks(gen: DistributionGenerator) { + + // resetting seed should generate the same sequence of random numbers + gen.setSeed(42L) + val array1 = (0 until 1000).map(_ => gen.nextValue()) + gen.setSeed(42L) + val array2 = (0 until 1000).map(_ => gen.nextValue()) + assert(array1.equals(array2)) + + // newInstance should contain a difference instance of the rng + // i.e. setting difference seeds for difference instances produces different sequences of + // random numbers. + val gen2 = gen.copy() + gen.setSeed(0L) + val array3 = (0 until 1000).map(_ => gen.nextValue()) + gen2.setSeed(1L) + val array4 = (0 until 1000).map(_ => gen2.nextValue()) + // Compare arrays instead of elements since individual elements can coincide by chance but the + // sequences should differ given two different seeds. + assert(!array3.equals(array4)) + + // test that setting the same seed in the copied instance produces the same sequence of numbers + gen.setSeed(0L) + val array5 = (0 until 1000).map(_ => gen.nextValue()) + gen2.setSeed(0L) + val array6 = (0 until 1000).map(_ => gen2.nextValue()) + assert(array5.equals(array6)) + } + + def distributionChecks(gen: DistributionGenerator, + mean: Double = 0.0, + stddev: Double = 1.0, + epsilon: Double = 0.01) { + for (seed <- 0 until 5) { + gen.setSeed(seed.toLong) + val sample = (0 until 100000).map { _ => gen.nextValue()} + val stats = new StatCounter(sample) + assert(math.abs(stats.mean - mean) < epsilon) + assert(math.abs(stats.stdev - stddev) < epsilon) + } + } + + test("UniformGenerator") { + val uniform = new UniformGenerator() + apiChecks(uniform) + // Stddev of uniform distribution = (ub - lb) / math.sqrt(12) + distributionChecks(uniform, 0.5, 1 / math.sqrt(12)) + } + + test("StandardNormalGenerator") { + val normal = new StandardNormalGenerator() + apiChecks(normal) + distributionChecks(normal, 0.0, 1.0) + } + + test("PoissonGenerator") { + // mean = 0.0 will not pass the API checks since 0.0 is always deterministically produced. + for (mean <- List(1.0, 5.0, 100.0)) { + val poisson = new PoissonGenerator(mean) + apiChecks(poisson) + distributionChecks(poisson, mean, math.sqrt(mean), 0.1) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala new file mode 100644 index 0000000000000..6aa4f803df0f7 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.rdd.{RandomRDDPartition, RandomRDD} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.util.StatCounter + +/* + * Note: avoid including APIs that do not set the seed for the RNG in unit tests + * in order to guarantee deterministic behavior. + * + * TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged + */ +class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Serializable { + + def testGeneratedRDD(rdd: RDD[Double], + expectedSize: Long, + expectedNumPartitions: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double = 0.01) { + val stats = rdd.stats() + assert(expectedSize === stats.count) + assert(expectedNumPartitions === rdd.partitions.size) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + // assume test RDDs are small + def testGeneratedVectorRDD(rdd: RDD[Vector], + expectedRows: Long, + expectedColumns: Int, + expectedNumPartitions: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double = 0.01) { + assert(expectedNumPartitions === rdd.partitions.size) + val values = new ArrayBuffer[Double]() + rdd.collect.foreach { vector => { + assert(vector.size === expectedColumns) + values ++= vector.toArray + }} + assert(expectedRows === values.size / expectedColumns) + val stats = new StatCounter(values) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + test("RandomRDD sizes") { + + // some cases where size % numParts != 0 to test getPartitions behaves correctly + for ((size, numPartitions) <- List((10000, 6), (12345, 1), (1000, 101))) { + val rdd = new RandomRDD(sc, size, numPartitions, new UniformGenerator, 0L) + assert(rdd.count() === size) + assert(rdd.partitions.size === numPartitions) + + // check that partition sizes are balanced + val partSizes = rdd.partitions.map(p => p.asInstanceOf[RandomRDDPartition].size.toDouble) + val partStats = new StatCounter(partSizes) + assert(partStats.max - partStats.min <= 1) + } + + // size > Int.MaxValue + val size = Int.MaxValue.toLong * 100L + val numPartitions = 101 + val rdd = new RandomRDD(sc, size, numPartitions, new UniformGenerator, 0L) + assert(rdd.partitions.size === numPartitions) + val count = rdd.partitions.foldLeft(0L) { (count, part) => + count + part.asInstanceOf[RandomRDDPartition].size + } + assert(count === size) + + // size needs to be positive + intercept[IllegalArgumentException] { new RandomRDD(sc, 0, 10, new UniformGenerator, 0L) } + + // numPartitions needs to be positive + intercept[IllegalArgumentException] { new RandomRDD(sc, 100, 0, new UniformGenerator, 0L) } + + // partition size needs to be <= Int.MaxValue + intercept[IllegalArgumentException] { + new RandomRDD(sc, Int.MaxValue.toLong * 100L, 99, new UniformGenerator, 0L) + } + } + + test("randomRDD for different distributions") { + val size = 100000L + val numPartitions = 10 + val poissonMean = 100.0 + + for (seed <- 0 until 5) { + val uniform = RandomRDDGenerators.uniformRDD(sc, size, numPartitions, seed) + testGeneratedRDD(uniform, size, numPartitions, 0.5, 1 / math.sqrt(12)) + + val normal = RandomRDDGenerators.normalRDD(sc, size, numPartitions, seed) + testGeneratedRDD(normal, size, numPartitions, 0.0, 1.0) + + val poisson = RandomRDDGenerators.poissonRDD(sc, poissonMean, size, numPartitions, seed) + testGeneratedRDD(poisson, size, numPartitions, poissonMean, math.sqrt(poissonMean), 0.1) + } + + // mock distribution to check that partitions have unique seeds + val random = RandomRDDGenerators.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L) + assert(random.collect.size === random.collect.distinct.size) + } + + test("randomVectorRDD for different distributions") { + val rows = 1000L + val cols = 100 + val parts = 10 + val poissonMean = 100.0 + + for (seed <- 0 until 5) { + val uniform = RandomRDDGenerators.uniformVectorRDD(sc, rows, cols, parts, seed) + testGeneratedVectorRDD(uniform, rows, cols, parts, 0.5, 1 / math.sqrt(12)) + + val normal = RandomRDDGenerators.normalVectorRDD(sc, rows, cols, parts, seed) + testGeneratedVectorRDD(normal, rows, cols, parts, 0.0, 1.0) + + val poisson = RandomRDDGenerators.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) + testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) + } + } +} + +private[random] class MockDistro extends DistributionGenerator { + + var seed = 0L + + // This allows us to check that each partition has a different seed + override def nextValue(): Double = seed.toDouble + + override def setSeed(seed: Long) = this.seed = seed + + override def copy(): MockDistro = new MockDistro +} From e5bbce9a60eb99c059315edbf18a1a923d93d9d5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 27 Jul 2014 18:46:58 -0700 Subject: [PATCH 0096/1492] Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server" This reverts commit f6ff2a61d00d12481bfb211ae13d6992daacdcc2. --- .gitignore | 1 - assembly/pom.xml | 10 - bagel/pom.xml | 2 +- bin/beeline | 45 --- bin/compute-classpath.sh | 1 - bin/spark-shell | 4 +- bin/spark-shell.cmd | 2 +- bin/spark-sql | 36 -- core/pom.xml | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 14 +- .../spark/deploy/SparkSubmitArguments.scala | 5 +- dev/create-release/create-release.sh | 10 +- dev/run-tests | 2 +- dev/scalastyle | 2 +- docs/sql-programming-guide.md | 201 +--------- examples/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 7 +- project/SparkBuild.scala | 14 +- sbin/start-thriftserver.sh | 36 -- sql/catalyst/pom.xml | 2 +- .../sql/catalyst/plans/logical/commands.scala | 3 +- sql/core/pom.xml | 2 +- .../scala/org/apache/spark/sql/SQLConf.scala | 20 +- .../apache/spark/sql/execution/commands.scala | 42 +-- .../org/apache/spark/sql/SQLConfSuite.scala | 13 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- sql/hive-thriftserver/pom.xml | 82 ----- .../hive/thriftserver/HiveThriftServer2.scala | 97 ----- .../hive/thriftserver/ReflectionUtils.scala | 58 --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 344 ------------------ .../thriftserver/SparkSQLCLIService.scala | 74 ---- .../hive/thriftserver/SparkSQLDriver.scala | 93 ----- .../sql/hive/thriftserver/SparkSQLEnv.scala | 58 --- .../thriftserver/SparkSQLSessionManager.scala | 49 --- .../server/SparkSQLOperationManager.scala | 151 -------- .../test/resources/data/files/small_kv.txt | 5 - .../sql/hive/thriftserver/CliSuite.scala | 57 --- .../thriftserver/HiveThriftServer2Suite.scala | 135 ------- .../sql/hive/thriftserver/TestUtils.scala | 108 ------ sql/hive/pom.xml | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 50 +-- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 54 files changed, 96 insertions(+), 1781 deletions(-) delete mode 100755 bin/beeline delete mode 100755 bin/spark-sql delete mode 100755 sbin/start-thriftserver.sh delete mode 100644 sql/hive-thriftserver/pom.xml delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala delete mode 100755 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala delete mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala delete mode 100644 sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala diff --git a/.gitignore b/.gitignore index 5b56a67c883e6..061c8946d23c1 100644 --- a/.gitignore +++ b/.gitignore @@ -57,4 +57,3 @@ metastore_db/ metastore/ warehouse/ TempStatsStore/ -sql/hive-thriftserver/test_warehouses diff --git a/assembly/pom.xml b/assembly/pom.xml index 703f15925bc44..567a8dd2a0d94 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -165,16 +165,6 @@ - - hive-thriftserver - - - org.apache.spark - spark-hive-thriftserver_${scala.binary.version} - ${project.version} - - - spark-ganglia-lgpl diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..90c4b095bb611 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-bagel_2.10 - bagel + bagel jar Spark Project Bagel diff --git a/bin/beeline b/bin/beeline deleted file mode 100755 index 09fe366c609fa..0000000000000 --- a/bin/beeline +++ /dev/null @@ -1,45 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -# Find the java binary -if [ -n "${JAVA_HOME}" ]; then - RUNNER="${JAVA_HOME}/bin/java" -else - if [ `command -v java` ]; then - RUNNER="java" - else - echo "JAVA_HOME is not set" >&2 - exit 1 - fi -fi - -# Compute classpath using external script -classpath_output=$($FWDIR/bin/compute-classpath.sh) -if [[ "$?" != "0" ]]; then - echo "$classpath_output" - exit 1 -else - CLASSPATH=$classpath_output -fi - -CLASS="org.apache.hive.beeline.BeeLine" -exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 16b794a1592e8..e81e8c060cb98 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -52,7 +52,6 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" fi diff --git a/bin/spark-shell b/bin/spark-shell index 756c8179d12b6..850e9507ec38f 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index b56d69801171c..4b9708a8c03f3 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell --class org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main diff --git a/bin/spark-sql b/bin/spark-sql deleted file mode 100755 index bba7f897b19bc..0000000000000 --- a/bin/spark-sql +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# -# Shell script for starting the Spark SQL CLI - -# Enter posix mode for bash -set -o posix - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/spark-sql [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 -fi - -CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/core/pom.xml b/core/pom.xml index a24743495b0e1..1054cec4d77bb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-core_2.10 - core + core jar Spark Project Core diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c9cec33ebaa66..3b5642b6caa36 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -46,10 +46,6 @@ object SparkSubmit { private val CLUSTER = 2 private val ALL_DEPLOY_MODES = CLIENT | CLUSTER - // A special jar name that indicates the class being run is inside of Spark itself, and therefore - // no user jar is needed. - private val SPARK_INTERNAL = "spark-internal" - // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" @@ -261,9 +257,7 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (clusterManager == YARN && deployMode == CLUSTER) { childMainClass = "org.apache.spark.deploy.yarn.Client" - if (args.primaryResource != SPARK_INTERNAL) { - childArgs += ("--jar", args.primaryResource) - } + childArgs += ("--jar", args.primaryResource) childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } @@ -338,7 +332,7 @@ object SparkSubmit { * Return whether the given primary resource represents a user jar. */ private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) + !isShell(primaryResource) && !isPython(primaryResource) } /** @@ -355,10 +349,6 @@ object SparkSubmit { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } - private[spark] def isInternal(primaryResource: String): Boolean = { - primaryResource == SPARK_INTERNAL - } - /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 01d0ae541a66b..3ab67a43a3b55 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -204,9 +204,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { - var inSparkOpts = true - // Delineates parsing of Spark options from parsing of user options. + var inSparkOpts = true parse(opts) def parse(opts: Seq[String]): Unit = opts match { @@ -319,7 +318,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit(errMessage) case v => primaryResource = - if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) { + if (!SparkSubmit.isShell(v)) { Utils.resolveURI(v).toString } else { v diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 33de24d1ae6d7..38830103d1e8d 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -53,7 +53,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare @@ -61,7 +61,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. @@ -111,10 +111,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" +make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" make_binary_release "hadoop2" \ - "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" # Copy data echo "Copying release tarballs" diff --git a/dev/run-tests b/dev/run-tests index 98ec969dc1b37..51e4def0f835a 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -65,7 +65,7 @@ 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" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive -Phive-thriftserver" sbt/sbt clean package \ + 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 | \ diff --git a/dev/scalastyle b/dev/scalastyle index d9f2b91a3a091..a02d06912f238 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 156e0aebdebe6..38728534a46e0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -136,7 +136,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD // Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) @@ -548,6 +548,7 @@ results = hiveContext.hql("FROM src SELECT key, value").collect() + # Writing Language-Integrated Relational Queries **Language-Integrated queries are currently only supported in Scala.** @@ -572,200 +573,4 @@ prefixed with a tick (`'`). Implicit conversions turn these symbols into expres evaluated by the SQL execution engine. A full list of the functions supported can be found in the [ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). - - -## Running the Thrift JDBC server - -The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] -(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test -the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive -you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` -for maven). - -To start the JDBC server, run the following in the Spark directory: - - ./sbin/start-thriftserver.sh - -The default port the server listens on is 10000. To listen on customized host and port, please set -the `HIVE_SERVER2_THRIFT_PORT` and `HIVE_SERVER2_THRIFT_BIND_HOST` environment variables. You may -run `./sbin/start-thriftserver.sh --help` for a complete list of all available options. Now you can -use beeline to test the Thrift JDBC server: - - ./bin/beeline - -Connect to the JDBC server in beeline with: - - beeline> !connect jdbc:hive2://localhost:10000 - -Beeline will ask you for a username and password. In non-secure mode, simply enter the username on -your machine and a blank password. For secure mode, please follow the instructions given in the -[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) - -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. - -You may also use the beeline script comes with Hive. - -### Migration Guide for Shark Users - -#### Reducer number - -In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark -SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value -is 200. Users may customize this property via `SET`: - -``` -SET spark.sql.shuffle.partitions=10; -SELECT page, count(*) c FROM logs_last_month_cached -GROUP BY page ORDER BY c DESC LIMIT 10; -``` - -You may also put this property in `hive-site.xml` to override the default value. - -For now, the `mapred.reduce.tasks` property is still recognized, and is converted to -`spark.sql.shuffle.partitions` automatically. - -#### Caching - -The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no -longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to -let user control table caching explicitly: - -``` -CACHE TABLE logs_last_month; -UNCACHE TABLE logs_last_month; -``` - -**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", -but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be -cached, you may simply count the table immediately after executing `CACHE TABLE`: - -``` -CACHE TABLE logs_last_month; -SELECT COUNT(1) FROM logs_last_month; -``` - -Several caching related features are not supported yet: - -* User defined partition level cache eviction policy -* RDD reloading -* In-memory cache write through policy - -### Compatibility with Apache Hive - -#### Deploying in Exising Hive Warehouses - -Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -#### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathemtatical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDe's) -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -#### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - -**Esoteric Hive Features** - -* Tables with partitions using different input formats: In Spark SQL, all table partitions need to - have the same input format. -* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions - (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. -* `UNIONTYPE` -* Unique join -* Single query multi insert -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -not necessary due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block level bitmap indexes and virtual columns (used to build indexes) -* Automatically convert a join to map join: For joining a large table with multiple small tables, - Hive automatically converts the join into a map join. We are adding this auto conversion in the - next release. -* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you - need to control the degree of parallelism post-shuffle using "SET - spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the - next release. -* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -## Running the Spark SQL CLI - -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. - -To start the Spark SQL CLI, run the following in the Spark directory: - - ./bin/spark-sql - -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available -options. + \ No newline at end of file diff --git a/examples/pom.xml b/examples/pom.xml index c4ed0f5a6a02b..bd1c387c2eb91 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-examples_2.10 - examples + examples jar Spark Project Examples diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 874b8a7959bb6..61a6aff543aed 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-flume_2.10 - streaming-flume + streaming-flume jar Spark Project External Flume diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 25a5c0a4d7d77..4762c50685a93 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-kafka_2.10 - streaming-kafka + streaming-kafka jar Spark Project External Kafka diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index f31ed655f6779..32c530e600ce0 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-mqtt_2.10 - streaming-mqtt + streaming-mqtt jar Spark Project External MQTT diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 56bb24c2a072e..637adb0f00da0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-twitter_2.10 - streaming-twitter + streaming-twitter jar Spark Project External Twitter diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 54b0242c54e78..e4d758a04a4cd 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-zeromq_2.10 - streaming-zeromq + streaming-zeromq jar Spark Project External ZeroMQ diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..7e3bcf29dcfbc 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-graphx_2.10 - graphx + graphx jar Spark Project GraphX diff --git a/mllib/pom.xml b/mllib/pom.xml index f27cf520dc9fa..92b07e2357db1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-mllib_2.10 - mllib + mllib jar Spark Project ML Library diff --git a/pom.xml b/pom.xml index 3e9d388180d8e..4e2d64a833640 100644 --- a/pom.xml +++ b/pom.xml @@ -95,7 +95,6 @@ sql/catalyst sql/core sql/hive - sql/hive-thriftserver repl assembly external/twitter @@ -253,9 +252,9 @@ 3.3.2 - commons-codec - commons-codec - 1.5 + commons-codec + commons-codec + 1.5 com.google.code.findbugs diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1629bc2cba8ba..62576f84dd031 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,11 +30,11 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, - streaming, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = - Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "spark", "sql", "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", - "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) + val allProjects@Seq(bagel, catalyst, core, graphx, hive, mllib, repl, spark, sql, streaming, + streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = + Seq("bagel", "catalyst", "core", "graphx", "hive", "mllib", "repl", "spark", "sql", + "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", + "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") @@ -100,7 +100,7 @@ object SparkBuild extends PomBuild { Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) - case _ => + case _ => } override val userPropertiesMap = System.getProperties.toMap @@ -158,7 +158,7 @@ object SparkBuild extends PomBuild { /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl).contains(x)). + allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl).exists(x => x == y)). foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh deleted file mode 100755 index 8398e6f19b511..0000000000000 --- a/sbin/start-thriftserver.sh +++ /dev/null @@ -1,36 +0,0 @@ -#!/usr/bin/env bash - -# -# 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. -# - -# -# Shell script for starting the Spark SQL Thrift server - -# Enter posix mode for bash -set -o posix - -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" - -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/start-thriftserver [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 -fi - -CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 531bfddbf237b..6decde3fcd62d 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -32,7 +32,7 @@ Spark Project Catalyst http://spark.apache.org/ - catalyst + catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index a357c6ffb8977..1d5f033f0d274 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -43,7 +43,8 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(1, AttributeReference("", StringType, nullable = false)())) + BoundReference(0, AttributeReference("key", StringType, nullable = false)()), + BoundReference(1, AttributeReference("value", StringType, nullable = false)())) } /** diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3a038a2db6173..c309c43804d97 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -32,7 +32,7 @@ Spark Project SQL http://spark.apache.org/ - sql + sql diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 41920c00b5a2c..2b787e14f3f15 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -30,13 +30,12 @@ import scala.collection.JavaConverters._ * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { - import SQLConf._ /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? /** Number of partitions to use for shuffle operators. */ - private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt + private[spark] def numShufflePartitions: Int = get("spark.sql.shuffle.partitions", "200").toInt /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to @@ -44,10 +43,11 @@ trait SQLConf { * effectively disables auto conversion. * Hive setting: hive.auto.convert.join.noconditionaltask.size. */ - private[spark] def autoConvertJoinSize: Int = get(AUTO_CONVERT_JOIN_SIZE, "10000").toInt + private[spark] def autoConvertJoinSize: Int = + get("spark.sql.auto.convert.join.size", "10000").toInt /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get(JOIN_BROADCAST_TABLES, "") + private[spark] def joinBroadcastTables: String = get("spark.sql.join.broadcastTables", "") /** ********************** SQLConf functionality methods ************ */ @@ -61,7 +61,7 @@ trait SQLConf { def set(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for $key") + require(value != null, s"value cannot be null for ${key}") settings.put(key, value) } @@ -90,13 +90,3 @@ trait SQLConf { } } - -object SQLConf { - val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" - val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" - val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" - - object Deprecated { - val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 9293239131d52..98d2f89c8ae71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -17,13 +17,12 @@ package org.apache.spark.sql.execution -import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLConf, SQLContext} +import org.apache.spark.sql.{Row, SQLContext} trait Command { /** @@ -45,53 +44,28 @@ trait Command { case class SetCommand( key: Option[String], value: Option[String], output: Seq[Attribute])( @transient context: SQLContext) - extends LeafNode with Command with Logging { + extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => - if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { - logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + - s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - context.set(SQLConf.SHUFFLE_PARTITIONS, v) - Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") - } else { - context.set(k, v) - Array(s"$k=$v") - } + context.set(k, v) + Array(k -> v) // Query the value bound to key k. case (Some(k), _) => - // TODO (lian) This is just a workaround to make the Simba ODBC driver work. - // Should remove this once we get the ODBC driver updated. - if (k == "-v") { - val hiveJars = Seq( - "hive-exec-0.12.0.jar", - "hive-service-0.12.0.jar", - "hive-common-0.12.0.jar", - "hive-hwi-0.12.0.jar", - "hive-0.12.0.jar").mkString(":") - - Array( - "system:java.class.path=" + hiveJars, - "system:sun.java.command=shark.SharkServer2") - } - else { - Array(s"$k=${context.getOption(k).getOrElse("")}") - } + Array(k -> context.getOption(k).getOrElse("")) // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => - context.getAll.map { case (k, v) => - s"$k=$v" - } + context.getAll case _ => throw new IllegalArgumentException() } def execute(): RDD[Row] = { - val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } + val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) } context.sparkContext.parallelize(rows, 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 1a58d73d9e7f4..08293f7f0ca30 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -54,10 +54,10 @@ class SQLConfSuite extends QueryTest { assert(get(testKey, testVal + "_") == testVal) assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - sql("set some.property=20") - assert(get("some.property", "0") == "20") - sql("set some.property = 40") - assert(get("some.property", "0") == "40") + sql("set mapred.reduce.tasks=20") + assert(get("mapred.reduce.tasks", "0") == "20") + sql("set mapred.reduce.tasks = 40") + assert(get("mapred.reduce.tasks", "0") == "40") val key = "spark.sql.key" val vs = "val0,val_1,val2.3,my_table" @@ -70,9 +70,4 @@ class SQLConfSuite extends QueryTest { clear() } - test("deprecated property") { - clear() - sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") - assert(get(SQLConf.SHUFFLE_PARTITIONS) == "10") - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index de9e8aa4f62ed..6736189c96d4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -424,25 +424,25 @@ class SQLQuerySuite extends QueryTest { sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - Seq(Seq(s"$testKey=$testVal")) + Seq(Seq(testKey, testVal)) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), Seq( - Seq(s"$testKey=$testVal"), - Seq(s"${testKey + testKey}=${testVal + testVal}")) + Seq(testKey, testVal), + Seq(testKey + testKey, testVal + testVal)) ) // "set key" checkAnswer( sql(s"SET $testKey"), - Seq(Seq(s"$testKey=$testVal")) + Seq(Seq(testKey, testVal)) ) checkAnswer( sql(s"SET $nonexistentKey"), - Seq(Seq(s"$nonexistentKey=")) + Seq(Seq(nonexistentKey, "")) ) clear() } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml deleted file mode 100644 index 7fac90fdc596d..0000000000000 --- a/sql/hive-thriftserver/pom.xml +++ /dev/null @@ -1,82 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent - 1.1.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-hive-thriftserver_2.10 - jar - Spark Project Hive - http://spark.apache.org/ - - hive-thriftserver - - - - - org.apache.spark - spark-hive_${scala.binary.version} - ${project.version} - - - org.spark-project.hive - hive-cli - ${hive.version} - - - org.spark-project.hive - hive-jdbc - ${hive.version} - - - org.spark-project.hive - hive-beeline - ${hive.version} - - - org.scalatest - scalatest_${scala.binary.version} - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.scalatest - scalatest-maven-plugin - - - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala deleted file mode 100644 index ddbc2a79fb512..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService -import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ - -/** - * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a - * `HiveThriftServer2` thrift server. - */ -private[hive] object HiveThriftServer2 extends Logging { - var LOG = LogFactory.getLog(classOf[HiveServer2]) - - def main(args: Array[String]) { - val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") - - if (!optionsProcessor.process(args)) { - logger.warn("Error starting HiveThriftServer2 with given arguments") - System.exit(-1) - } - - val ss = new SessionState(new HiveConf(classOf[SessionState])) - - // Set all properties specified via command line. - val hiveConf: HiveConf = ss.getConf - hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logger.debug(s"HiveConf var: $k=$v") - } - - SessionState.start(ss) - - logger.info("Starting SparkContext") - SparkSQLEnv.init() - SessionState.start(ss) - - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.sparkContext.stop() - } - } - ) - - try { - val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(hiveConf) - server.start() - logger.info("HiveThriftServer2 started") - } catch { - case e: Exception => - logger.error("Error starting HiveThriftServer2", e) - System.exit(-1) - } - } -} - -private[hive] class HiveThriftServer2(hiveContext: HiveContext) - extends HiveServer2 - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - val sparkSqlCliService = new SparkSQLCLIService(hiveContext) - setSuperField(this, "cliService", sparkSqlCliService) - addService(sparkSqlCliService) - - val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) - setSuperField(this, "thriftCLIService", thriftCliService) - addService(thriftCliService) - - initCompositeService(hiveConf) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala deleted file mode 100644 index 599294dfbb7d7..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -private[hive] object ReflectionUtils { - def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { - setAncestorField(obj, 1, fieldName, fieldValue) - } - - def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { - val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() - val field = ancestor.getDeclaredField(fieldName) - field.setAccessible(true) - field.set(obj, fieldValue) - } - - def getSuperField[T](obj: AnyRef, fieldName: String): T = { - getAncestorField[T](obj, 1, fieldName) - } - - def getAncestorField[T](clazz: Object, level: Int, fieldName: String): T = { - val ancestor = Iterator.iterate[Class[_]](clazz.getClass)(_.getSuperclass).drop(level).next() - val field = ancestor.getDeclaredField(fieldName) - field.setAccessible(true) - field.get(clazz).asInstanceOf[T] - } - - def invokeStatic(clazz: Class[_], methodName: String, args: (Class[_], AnyRef)*): AnyRef = { - invoke(clazz, null, methodName, args: _*) - } - - def invoke( - clazz: Class[_], - obj: AnyRef, - methodName: String, - args: (Class[_], AnyRef)*): AnyRef = { - - val (types, values) = args.unzip - val method = clazz.getDeclaredMethod(methodName, types: _*) - method.setAccessible(true) - method.invoke(obj, values.toSeq: _*) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala deleted file mode 100755 index 27268ecb923e9..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ /dev/null @@ -1,344 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.io._ -import java.util.{ArrayList => JArrayList} - -import jline.{ConsoleReader, History} -import org.apache.commons.lang.StringUtils -import org.apache.commons.logging.LogFactory -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException -import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.thrift.transport.TSocket - -import org.apache.spark.sql.Logging - -private[hive] object SparkSQLCLIDriver { - private var prompt = "spark-sql" - private var continuedPrompt = "".padTo(prompt.length, ' ') - private var transport:TSocket = _ - - installSignalHandler() - - /** - * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), - * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while - * a command is being processed by the current thread. - */ - def installSignalHandler() { - HiveInterruptUtils.add(new HiveInterruptCallback { - override def interrupt() { - // Handle remote execution mode - if (SparkSQLEnv.sparkContext != null) { - SparkSQLEnv.sparkContext.cancelAllJobs() - } else { - if (transport != null) { - // Force closing of TCP connection upon session termination - transport.getSocket.close() - } - } - } - }) - } - - def main(args: Array[String]) { - val oproc = new OptionsProcessor() - if (!oproc.process_stage1(args)) { - System.exit(1) - } - - // NOTE: It is critical to do this here so that log4j is reinitialized - // before any of the other core hive classes are loaded - var logInitFailed = false - var logInitDetailMessage: String = null - try { - logInitDetailMessage = LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => - logInitFailed = true - logInitDetailMessage = e.getMessage - } - - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) - - sessionState.in = System.in - try { - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.info = new PrintStream(System.err, true, "UTF-8") - sessionState.err = new PrintStream(System.err, true, "UTF-8") - } catch { - case e: UnsupportedEncodingException => System.exit(3) - } - - if (!oproc.process_stage2(sessionState)) { - System.exit(2) - } - - if (!sessionState.getIsSilent) { - if (logInitFailed) System.err.println(logInitDetailMessage) - else SessionState.getConsole.printInfo(logInitDetailMessage) - } - - // Set all properties specified via command line. - val conf: HiveConf = sessionState.getConf - sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => - conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - sessionState.getOverriddenConfigurations.put( - item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) - } - - SessionState.start(sessionState) - - // Clean up after we exit - Runtime.getRuntime.addShutdownHook( - new Thread() { - override def run() { - SparkSQLEnv.stop() - } - } - ) - - // "-h" option has been passed, so connect to Hive thrift server. - if (sessionState.getHost != null) { - sessionState.connect() - if (sessionState.isRemoteMode) { - prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt - continuedPrompt = "".padTo(prompt.length, ' ') - } - } - - if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { - // Hadoop-20 and above - we need to augment classpath using hiveconf - // components. - // See also: code in ExecDriver.java - var loader = conf.getClassLoader - val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) - if (StringUtils.isNotBlank(auxJars)) { - loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")) - } - conf.setClassLoader(loader) - Thread.currentThread().setContextClassLoader(loader) - } - - val cli = new SparkSQLCLIDriver - cli.setHiveVariables(oproc.getHiveVariables) - - // TODO work around for set the log output to console, because the HiveContext - // will set the output into an invalid buffer. - sessionState.in = System.in - try { - sessionState.out = new PrintStream(System.out, true, "UTF-8") - sessionState.info = new PrintStream(System.err, true, "UTF-8") - sessionState.err = new PrintStream(System.err, true, "UTF-8") - } catch { - case e: UnsupportedEncodingException => System.exit(3) - } - - // Execute -i init files (always in silent mode) - cli.processInitFiles(sessionState) - - if (sessionState.execString != null) { - System.exit(cli.processLine(sessionState.execString)) - } - - try { - if (sessionState.fileName != null) { - System.exit(cli.processFile(sessionState.fileName)) - } - } catch { - case e: FileNotFoundException => - System.err.println(s"Could not open input file for reading. (${e.getMessage})") - System.exit(3) - } - - val reader = new ConsoleReader() - reader.setBellEnabled(false) - // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) - CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e)) - - val historyDirectory = System.getProperty("user.home") - - try { - if (new File(historyDirectory).exists()) { - val historyFile = historyDirectory + File.separator + ".hivehistory" - reader.setHistory(new History(new File(historyFile))) - } else { - System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + - " does not exist. History will not be available during this session.") - } - } catch { - case e: Exception => - System.err.println("WARNING: Encountered an error while trying to initialize Hive's " + - "history file. History will not be available during this session.") - System.err.println(e.getMessage) - } - - val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") - clientTransportTSocketField.setAccessible(true) - - transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket] - - var ret = 0 - var prefix = "" - val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", - classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) - - def promptWithCurrentDB = s"$prompt$currentDB" - def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( - classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) - - var currentPrompt = promptWithCurrentDB - var line = reader.readLine(currentPrompt + "> ") - - while (line != null) { - if (prefix.nonEmpty) { - prefix += '\n' - } - - if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { - line = prefix + line - ret = cli.processLine(line, true) - prefix = "" - currentPrompt = promptWithCurrentDB - } else { - prefix = prefix + line - currentPrompt = continuedPromptWithDBSpaces - } - - line = reader.readLine(currentPrompt + "> ") - } - - sessionState.close() - - System.exit(ret) - } -} - -private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { - private val sessionState = SessionState.get().asInstanceOf[CliSessionState] - - private val LOG = LogFactory.getLog("CliDriver") - - private val console = new SessionState.LogHelper(LOG) - - private val conf: Configuration = - if (sessionState != null) sessionState.getConf else new Configuration() - - // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver - // because the Hive unit tests do not go through the main() code path. - if (!sessionState.isRemoteMode) { - SparkSQLEnv.init() - } - - override def processCmd(cmd: String): Int = { - val cmd_trimmed: String = cmd.trim() - val tokens: Array[String] = cmd_trimmed.split("\\s+") - val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - if (cmd_trimmed.toLowerCase.equals("quit") || - cmd_trimmed.toLowerCase.equals("exit") || - tokens(0).equalsIgnoreCase("source") || - cmd_trimmed.startsWith("!") || - tokens(0).toLowerCase.equals("list") || - sessionState.isRemoteMode) { - val start = System.currentTimeMillis() - super.processCmd(cmd) - val end = System.currentTimeMillis() - val timeTaken: Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds") - 0 - } else { - var ret = 0 - val hconf = conf.asInstanceOf[HiveConf] - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) - - if (proc != null) { - if (proc.isInstanceOf[Driver]) { - val driver = new SparkSQLDriver - - driver.init() - val out = sessionState.out - val start:Long = System.currentTimeMillis() - if (sessionState.getIsVerbose) { - out.println(cmd) - } - - ret = driver.run(cmd).getResponseCode - if (ret != 0) { - driver.close() - return ret - } - - val res = new JArrayList[String]() - - if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { - // Print the column names. - Option(driver.getSchema.getFieldSchemas).map { fields => - out.println(fields.map(_.getName).mkString("\t")) - } - } - - try { - while (!out.checkError() && driver.getResults(res)) { - res.foreach(out.println) - res.clear() - } - } catch { - case e:IOException => - console.printError( - s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} - |${org.apache.hadoop.util.StringUtils.stringifyException(e)} - """.stripMargin) - ret = 1 - } - - val cret = driver.close() - if (ret == 0) { - ret = cret - } - - val end = System.currentTimeMillis() - if (end > start) { - val timeTaken:Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds", null) - } - - // Destroy the driver to release all the locks. - driver.destroy() - } else { - if (sessionState.getIsVerbose) { - sessionState.out.println(tokens(0) + " " + cmd_1) - } - ret = proc.run(cmd_1).getResponseCode - } - } - ret - } - } -} - diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala deleted file mode 100644 index 42cbf363b274f..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.io.IOException -import java.util.{List => JList} -import javax.security.auth.login.LoginException - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.shims.ShimLoader -import org.apache.hive.service.Service.STATE -import org.apache.hive.service.auth.HiveAuthFactory -import org.apache.hive.service.cli.CLIService -import org.apache.hive.service.{AbstractService, Service, ServiceException} - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ - -private[hive] class SparkSQLCLIService(hiveContext: HiveContext) - extends CLIService - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) - setSuperField(this, "sessionManager", sparkSqlSessionManager) - addService(sparkSqlSessionManager) - - try { - HiveAuthFactory.loginFromKeytab(hiveConf) - val serverUserName = ShimLoader.getHadoopShims - .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) - setSuperField(this, "serverUserName", serverUserName) - } catch { - case e @ (_: IOException | _: LoginException) => - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) - } - - initCompositeService(hiveConf) - } -} - -private[thriftserver] trait ReflectedCompositeService { this: AbstractService => - def initCompositeService(hiveConf: HiveConf) { - // Emulating `CompositeService.init(hiveConf)` - val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") - serviceList.foreach(_.init(hiveConf)) - - // Emulating `AbstractService.init(hiveConf)` - invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) - setAncestorField(this, 3, "hiveConf", hiveConf) - invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) - getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala deleted file mode 100644 index 5202aa9903e03..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ - -import java.util.{ArrayList => JArrayList} - -import org.apache.commons.lang.exception.ExceptionUtils -import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} -import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} - -private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) - extends Driver with Logging { - - private var tableSchema: Schema = _ - private var hiveResponse: Seq[String] = _ - - override def init(): Unit = { - } - - private def getResultSetSchema(query: context.QueryExecution): Schema = { - val analyzed = query.analyzed - logger.debug(s"Result Schema: ${analyzed.output}") - if (analyzed.output.size == 0) { - new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) - } else { - val fieldSchemas = analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - - new Schema(fieldSchemas, null) - } - } - - override def run(command: String): CommandProcessorResponse = { - val execution = context.executePlan(context.hql(command).logicalPlan) - - // TODO unify the error code - try { - hiveResponse = execution.stringResult() - tableSchema = getResultSetSchema(execution) - new CommandProcessorResponse(0) - } catch { - case cause: Throwable => - logger.error(s"Failed in [$command]", cause) - new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) - } - } - - override def close(): Int = { - hiveResponse = null - tableSchema = null - 0 - } - - override def getSchema: Schema = tableSchema - - override def getResults(res: JArrayList[String]): Boolean = { - if (hiveResponse == null) { - false - } else { - res.addAll(hiveResponse) - hiveResponse = null - true - } - } - - override def destroy() { - super.destroy() - hiveResponse = null - tableSchema = null - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala deleted file mode 100644 index 451c3bd7b9352..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import org.apache.hadoop.hive.ql.session.SessionState - -import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.sql.Logging -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} - -/** A singleton object for the master program. The slaves should not access this. */ -private[hive] object SparkSQLEnv extends Logging { - logger.debug("Initializing SparkSQLEnv") - - var hiveContext: HiveContext = _ - var sparkContext: SparkContext = _ - - def init() { - if (hiveContext == null) { - sparkContext = new SparkContext(new SparkConf() - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) - - sparkContext.addSparkListener(new StatsReportListener()) - - hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = SessionState.get() - @transient override lazy val hiveconf = sessionState.getConf - } - } - } - - /** Cleans up and shuts down the Spark SQL environments. */ - def stop() { - logger.debug("Shutting down Spark SQL Environment") - // Stop the SparkContext - if (SparkSQLEnv.sparkContext != null) { - sparkContext.stop() - sparkContext = null - hiveContext = null - } - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala deleted file mode 100644 index 6b3275b4eaf04..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.util.concurrent.Executors - -import org.apache.commons.logging.Log -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hive.service.cli.session.SessionManager - -import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager - -private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) - extends SessionManager - with ReflectedCompositeService { - - override def init(hiveConf: HiveConf) { - setSuperField(this, "hiveConf", hiveConf) - - val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) - setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) - getAncestorField[Log](this, 3, "LOG").info( - s"HiveServer2: Async execution pool size $backgroundPoolSize") - - val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) - setSuperField(this, "operationManager", sparkSqlOperationManager) - addService(sparkSqlOperationManager) - - initCompositeService(hiveConf) - } -} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala deleted file mode 100644 index a4e1f3e762e89..0000000000000 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ /dev/null @@ -1,151 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver.server - -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.math.{random, round} - -import java.sql.Timestamp -import java.util.{Map => JMap} - -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} -import org.apache.hive.service.cli.session.HiveSession - -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} - -/** - * Executes queries using Spark SQL, and maintains a list of handles to active queries. - */ -class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { - val handleToOperation = ReflectionUtils - .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") - - override def newExecuteStatementOperation( - parentSession: HiveSession, - statement: String, - confOverlay: JMap[String, String], - async: Boolean): ExecuteStatementOperation = synchronized { - - val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { - private var result: SchemaRDD = _ - private var iter: Iterator[SparkRow] = _ - private var dataTypes: Array[DataType] = _ - - def close(): Unit = { - // RDDs will be cleaned automatically upon garbage collection. - logger.debug("CLOSING") - } - - def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { - if (!iter.hasNext) { - new RowSet() - } else { - val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. - var curRow = 0 - var rowSet = new ArrayBuffer[Row](maxRows) - - while (curRow < maxRows && iter.hasNext) { - val sparkRow = iter.next() - val row = new Row() - var curCol = 0 - - while (curCol < sparkRow.length) { - dataTypes(curCol) match { - case StringType => - row.addString(sparkRow(curCol).asInstanceOf[String]) - case IntegerType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getInt(curCol))) - case BooleanType => - row.addColumnValue(ColumnValue.booleanValue(sparkRow.getBoolean(curCol))) - case DoubleType => - row.addColumnValue(ColumnValue.doubleValue(sparkRow.getDouble(curCol))) - case FloatType => - row.addColumnValue(ColumnValue.floatValue(sparkRow.getFloat(curCol))) - case DecimalType => - val hiveDecimal = sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal - row.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) - case LongType => - row.addColumnValue(ColumnValue.longValue(sparkRow.getLong(curCol))) - case ByteType => - row.addColumnValue(ColumnValue.byteValue(sparkRow.getByte(curCol))) - case ShortType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getShort(curCol))) - case TimestampType => - row.addColumnValue( - ColumnValue.timestampValue(sparkRow.get(curCol).asInstanceOf[Timestamp])) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = result - .queryExecution - .asInstanceOf[HiveContext#QueryExecution] - .toHiveString((sparkRow.get(curCol), dataTypes(curCol))) - row.addColumnValue(ColumnValue.stringValue(hiveString)) - } - curCol += 1 - } - rowSet += row - curRow += 1 - } - new RowSet(rowSet, 0) - } - } - - def getResultSetSchema: TableSchema = { - logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") - if (result.queryExecution.analyzed.output.size == 0) { - new TableSchema(new FieldSchema("Result", "string", "") :: Nil) - } else { - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - new TableSchema(schema) - } - } - - def run(): Unit = { - logger.info(s"Running query '$statement'") - setState(OperationState.RUNNING) - try { - result = hiveContext.hql(statement) - logger.debug(result.queryExecution.toString()) - val groupId = round(random * 1000000).toString - hiveContext.sparkContext.setJobGroup(groupId, statement) - iter = result.queryExecution.toRdd.toLocalIterator - dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray - setHasResultSet(true) - } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - logger.error("Error executing query:",e) - throw new HiveSQLException(e.toString) - } - setState(OperationState.FINISHED) - } - } - - handleToOperation.put(operation.getHandle, operation) - operation - } -} diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt deleted file mode 100644 index 850f8014b6f05..0000000000000 --- a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt +++ /dev/null @@ -1,5 +0,0 @@ -238val_238 -86val_86 -311val_311 -27val_27 -165val_165 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala deleted file mode 100644 index 69f19f826a802..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.io.{BufferedReader, InputStreamReader, PrintWriter} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { - val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") - val METASTORE_PATH = TestUtils.getMetastorePath("cli") - - override def beforeAll() { - val pb = new ProcessBuilder( - "../../bin/spark-sql", - "--master", - "local", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - - process = pb.start() - outputWriter = new PrintWriter(process.getOutputStream, true) - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "spark-sql>") - } - - override def afterAll() { - process.destroy() - process.waitFor() - } - - test("simple commands") { - val dataFilePath = getDataFile("data/files/small_kv.txt") - executeQuery("create table hive_test1(key int, val string);") - executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") - executeQuery("cache table hive_test1", "Time taken") - } -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala deleted file mode 100644 index fe3403b3292ec..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import scala.collection.JavaConversions._ -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent._ - -import java.io.{BufferedReader, InputStreamReader} -import java.net.ServerSocket -import java.sql.{Connection, DriverManager, Statement} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.sql.Logging -import org.apache.spark.sql.catalyst.util.getTempFilePath - -/** - * Test for the HiveThriftServer2 using JDBC. - */ -class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { - - val WAREHOUSE_PATH = getTempFilePath("warehouse") - val METASTORE_PATH = getTempFilePath("metastore") - - val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" - val TABLE = "test" - val HOST = "localhost" - val PORT = { - // Let the system to choose a random available port to avoid collision with other parallel - // builds. - val socket = new ServerSocket(0) - val port = socket.getLocalPort - socket.close() - port - } - - // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. - val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean - - Class.forName(DRIVER_NAME) - - override def beforeAll() { launchServer() } - - override def afterAll() { stopServer() } - - private def launchServer(args: Seq[String] = Seq.empty) { - // Forking a new process to start the Hive Thrift server. The reason to do this is it is - // hard to clean up Hive resources entirely, so we just start a new process and kill - // that process for cleanup. - val defaultArgs = Seq( - "../../sbin/start-thriftserver.sh", - "--master local", - "--hiveconf", - "hive.root.logger=INFO,console", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH") - val pb = new ProcessBuilder(defaultArgs ++ args) - val environment = pb.environment() - environment.put("HIVE_SERVER2_THRIFT_PORT", PORT.toString) - environment.put("HIVE_SERVER2_THRIFT_BIND_HOST", HOST) - process = pb.start() - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on") - - // Spawn a thread to read the output from the forked process. - // Note that this is necessary since in some configurations, log4j could be blocked - // if its output to stderr are not read, and eventually blocking the entire test suite. - future { - while (true) { - val stdout = readFrom(inputReader) - val stderr = readFrom(errorReader) - if (VERBOSE && stdout.length > 0) { - println(stdout) - } - if (VERBOSE && stderr.length > 0) { - println(stderr) - } - Thread.sleep(50) - } - } - } - - private def stopServer() { - process.destroy() - process.waitFor() - } - - test("test query execution against a Hive Thrift server") { - Thread.sleep(5 * 1000) - val dataFilePath = getDataFile("data/files/small_kv.txt") - val stmt = createStatement() - stmt.execute("DROP TABLE IF EXISTS test") - stmt.execute("DROP TABLE IF EXISTS test_cached") - stmt.execute("CREATE TABLE test(key int, val string)") - stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") - stmt.execute("CREATE TABLE test_cached as select * from test limit 4") - stmt.execute("CACHE TABLE test_cached") - - var rs = stmt.executeQuery("select count(*) from test") - rs.next() - assert(rs.getInt(1) === 5) - - rs = stmt.executeQuery("select count(*) from test_cached") - rs.next() - assert(rs.getInt(1) === 4) - - stmt.close() - } - - def getConnection: Connection = { - val connectURI = s"jdbc:hive2://localhost:$PORT/" - DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") - } - - def createStatement(): Statement = getConnection.createStatement() -} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala deleted file mode 100644 index bb2242618fbef..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.io.{BufferedReader, PrintWriter} -import java.text.SimpleDateFormat -import java.util.Date - -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException - -object TestUtils { - val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") - - def getWarehousePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + - timestamp.format(new Date) - } - - def getMetastorePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + - timestamp.format(new Date) - } - - // Dummy function for initialize the log4j properties. - def init() { } - - // initialize log4j - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } -} - -trait TestUtils { - var process : Process = null - var outputWriter : PrintWriter = null - var inputReader : BufferedReader = null - var errorReader : BufferedReader = null - - def executeQuery( - cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { - println("Executing: " + cmd + ", expecting output: " + outputMessage) - outputWriter.write(cmd + "\n") - outputWriter.flush() - waitForQuery(timeout, outputMessage) - } - - protected def waitForQuery(timeout: Long, message: String): String = { - if (waitForOutput(errorReader, message, timeout)) { - Thread.sleep(500) - readOutput() - } else { - assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) - null - } - } - - // Wait for the specified str to appear in the output. - protected def waitForOutput( - reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { - val startTime = System.currentTimeMillis - var out = "" - while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { - out += readFrom(reader) - } - out.contains(str) - } - - // Read stdout output and filter out garbage collection messages. - protected def readOutput(): String = { - val output = readFrom(inputReader) - // Remove GC Messages - val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) - .mkString("\n") - filteredOutput - } - - protected def readFrom(reader: BufferedReader): String = { - var out = "" - var c = 0 - while (reader.ready) { - c = reader.read() - out += c.asInstanceOf[Char] - } - out - } - - protected def getDataFile(name: String) = { - Thread.currentThread().getContextClassLoader.getResource(name) - } -} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 93d00f7c37c9b..1699ffe06ce15 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -32,7 +32,7 @@ Spark Project Hive http://spark.apache.org/ - hive + hive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 84d43eaeea51d..201c85f3d501e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -255,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DecimalType, TimestampType, BinaryType) - protected[sql] def toHiveString(a: (Any, DataType)): String = a match { + protected def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a022a1e2dc70e..a8623b64c656f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -419,10 +419,10 @@ class HiveQuerySuite extends HiveComparisonTest { hql(s"set $testKey=$testVal") assert(get(testKey, testVal + "_") == testVal) - hql("set some.property=20") - assert(get("some.property", "0") == "20") - hql("set some.property = 40") - assert(get("some.property", "0") == "40") + hql("set mapred.reduce.tasks=20") + assert(get("mapred.reduce.tasks", "0") == "20") + hql("set mapred.reduce.tasks = 40") + assert(get("mapred.reduce.tasks", "0") == "40") hql(s"set $testKey=$testVal") assert(get(testKey, "0") == testVal) @@ -436,61 +436,63 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { case Row(key: String, value: String) => key -> value }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. assert(hql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql("SET")) } hql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - hql(s"SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(hql("SET")) } // "set key" - assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - hql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as hql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET")) } - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(sql(s"SET $nonexistentKey")) } clear() diff --git a/streaming/pom.xml b/streaming/pom.xml index b99f306b8f2cc..f60697ce745b7 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming_2.10 - streaming + streaming jar Spark Project Streaming diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..c0ee8faa7a615 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,7 +27,7 @@ org.apache.spark spark-tools_2.10 - tools + tools jar Spark Project Tools diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 51744ece0412d..5b13a1f002d6e 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-alpha + yarn-alpha org.apache.spark diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..efb473aa1b261 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -29,7 +29,7 @@ pom Spark Project YARN Parent POM - yarn + yarn diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..ceaf9f9d71001 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-stable + yarn-stable org.apache.spark From d7eac4c3db7462e60e0c456dc93780167f5fcb2c Mon Sep 17 00:00:00 2001 From: Rahul Singhal Date: Sun, 27 Jul 2014 18:50:32 -0700 Subject: [PATCH 0097/1492] SPARK-2651: Add maven scalastyle plugin Can be run as: "mvn scalastyle:check" Author: Rahul Singhal Closes #1550 from rahulsinghaliitd/SPARK-2651 and squashes the following commits: 53748dd [Rahul Singhal] SPARK-2651: Add maven scalastyle plugin --- .gitignore | 1 + pom.xml | 24 ++++++++++++++++++++++++ 2 files changed, 25 insertions(+) diff --git a/.gitignore b/.gitignore index 061c8946d23c1..a4ec12ca6b53f 100644 --- a/.gitignore +++ b/.gitignore @@ -51,6 +51,7 @@ unit-tests.log rat-results.txt scalastyle.txt conf/*.conf +scalastyle-output.xml # For Hive metastore_db/ diff --git a/pom.xml b/pom.xml index 4e2d64a833640..d2e6b3c0ed5a4 100644 --- a/pom.xml +++ b/pom.xml @@ -957,6 +957,30 @@ org.apache.maven.plugins maven-source-plugin + + org.scalastyle + scalastyle-maven-plugin + 0.4.0 + + false + true + false + false + ${basedir}/src/main/scala + ${basedir}/src/test/scala + scalastyle-config.xml + scalastyle-output.xml + UTF-8 + + + + package + + check + + + +
From a7d145e98c55fa66a541293930f25d9cdc25f3b4 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 27 Jul 2014 22:54:43 -0700 Subject: [PATCH 0098/1492] [SPARK-1550] [PySpark] Allow SparkContext creation after failed attempts This addresses a PySpark issue where a failed attempt to construct SparkContext would prevent any future SparkContext creation. Author: Josh Rosen Closes #1606 from JoshRosen/SPARK-1550 and squashes the following commits: ec7fadc [Josh Rosen] [SPARK-1550] [PySpark] Allow SparkContext creation after failed attempts --- python/pyspark/context.py | 18 ++++++++++++------ python/pyspark/tests.py | 6 ++++++ 2 files changed, 18 insertions(+), 6 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index e8ac9895cf54a..830a6ee03f2a6 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -100,7 +100,16 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, tempNamedTuple = namedtuple("Callsite", "function file linenum") self._callsite = tempNamedTuple(function=None, file=None, linenum=None) SparkContext._ensure_initialized(self, gateway=gateway) - + try: + self._do_init(master, appName, sparkHome, pyFiles, environment, batchSize, serializer, + conf) + except: + # If an error occurs, clean up in order to allow future SparkContext creation: + self.stop() + raise + + def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, serializer, + conf): self.environment = environment or {} self._conf = conf or SparkConf(_jvm=self._jvm) self._batchSize = batchSize # -1 represents an unlimited batch size @@ -249,17 +258,14 @@ def defaultMinPartitions(self): """ return self._jsc.sc().defaultMinPartitions() - def __del__(self): - self.stop() - def stop(self): """ Shut down the SparkContext. """ - if self._jsc: + if getattr(self, "_jsc", None): self._jsc.stop() self._jsc = None - if self._accumulatorServer: + if getattr(self, "_accumulatorServer", None): self._accumulatorServer.shutdown() self._accumulatorServer = None with SparkContext._lock: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 8ba51461d106d..63cc5e9ad96fa 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -209,6 +209,12 @@ def func(): class TestRDDFunctions(PySparkTestCase): + def test_failed_sparkcontext_creation(self): + # Regression test for SPARK-1550 + self.sc.stop() + self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name")) + self.sc = SparkContext("local") + def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" From 2b8d89e30ebfe2272229a1eddd7542d7437c9924 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 28 Jul 2014 10:59:53 -0700 Subject: [PATCH 0099/1492] [SPARK-2523] [SQL] Hadoop table scan bug fixing In HiveTableScan.scala, ObjectInspector was created for all of the partition based records, which probably causes ClassCastException if the object inspector is not identical among table & partitions. This is the follow up with: https://github.com/apache/spark/pull/1408 https://github.com/apache/spark/pull/1390 I've run a micro benchmark in my local with 15000000 records totally, and got the result as below: With This Patch | Partition-Based Table | Non-Partition-Based Table ------------ | ------------- | ------------- No | 1927 ms | 1885 ms Yes | 1541 ms | 1524 ms It showed this patch will also improve the performance. PS: the benchmark code is also attached. (thanks liancheng ) ``` package org.apache.spark.sql.hive import org.apache.spark.SparkContext import org.apache.spark.SparkConf import org.apache.spark.sql._ object HiveTableScanPrepare extends App { case class Record(key: String, value: String) val sparkContext = new SparkContext( new SparkConf() .setMaster("local") .setAppName(getClass.getSimpleName.stripSuffix("$"))) val hiveContext = new LocalHiveContext(sparkContext) val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"$i", s"val_$i"))) import hiveContext._ hql("SHOW TABLES") hql("DROP TABLE if exists part_scan_test") hql("DROP TABLE if exists scan_test") hql("DROP TABLE if exists records") rdd.registerAsTable("records") hql("""CREATE TABLE part_scan_test (key STRING, value STRING) PARTITIONED BY (part1 string, part2 STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) hql("""CREATE TABLE scan_test (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) for (part1 <- 2000 until 2001) { for (part2 <- 1 to 5) { hql(s"""from records | insert into table part_scan_test PARTITION (part1='$part1', part2='2010-01-$part2') | select key, value """.stripMargin) hql(s"""from records | insert into table scan_test select key, value """.stripMargin) } } } object HiveTableScanTest extends App { val sparkContext = new SparkContext( new SparkConf() .setMaster("local") .setAppName(getClass.getSimpleName.stripSuffix("$"))) val hiveContext = new LocalHiveContext(sparkContext) import hiveContext._ hql("SHOW TABLES") val part_scan_test = hql("select key, value from part_scan_test") val scan_test = hql("select key, value from scan_test") val r_part_scan_test = (0 to 5).map(i => benchmark(part_scan_test)) val r_scan_test = (0 to 5).map(i => benchmark(scan_test)) println("Scanning Partition-Based Table") r_part_scan_test.foreach(printResult) println("Scanning Non-Partition-Based Table") r_scan_test.foreach(printResult) def printResult(result: (Long, Long)) { println(s"Duration: ${result._1} ms Result: ${result._2}") } def benchmark(srdd: SchemaRDD) = { val begin = System.currentTimeMillis() val result = srdd.count() val end = System.currentTimeMillis() ((end - begin), result) } } ``` Author: Cheng Hao Closes #1439 from chenghao-intel/hadoop_table_scan and squashes the following commits: 888968f [Cheng Hao] Fix issues in code style 27540ba [Cheng Hao] Fix the TableScan Bug while partition serde differs 40a24a7 [Cheng Hao] Add Unit Test --- .../apache/spark/sql/hive/TableReader.scala | 113 +++++++++++++----- .../sql/hive/execution/HiveTableScan.scala | 90 ++------------ ...t_serde-0-8caed2a6e80250a6d38a59388679c298 | 2 + .../hive/execution/HiveTableScanSuite.scala | 48 ++++++++ 4 files changed, 138 insertions(+), 115 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-8caed2a6e80250a6d38a59388679c298 create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala 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 c3942578d6b5a..82c88280d7754 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 @@ -24,6 +24,8 @@ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector + import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} @@ -31,13 +33,16 @@ import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row, GenericMutableRow, Literal, Cast} +import org.apache.spark.sql.catalyst.types.DataType + /** * A trait for subclasses that handle table scans. */ private[hive] sealed trait TableReader { - def makeRDDForTable(hiveTable: HiveTable): RDD[_] + def makeRDDForTable(hiveTable: HiveTable): RDD[Row] - def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] + def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row] } @@ -46,7 +51,10 @@ private[hive] sealed trait TableReader { * data warehouse directory. */ private[hive] -class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveContext) +class HadoopTableReader( + @transient attributes: Seq[Attribute], + @transient relation: MetastoreRelation, + @transient sc: HiveContext) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless @@ -63,10 +71,10 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon def hiveConf = _broadcastedHiveConf.value.value - override def makeRDDForTable(hiveTable: HiveTable): RDD[_] = + override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] = makeRDDForTable( hiveTable, - _tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + relation.tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], filterOpt = None) /** @@ -81,14 +89,14 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon def makeRDDForTable( hiveTable: HiveTable, deserializerClass: Class[_ <: Deserializer], - filterOpt: Option[PathFilter]): RDD[_] = { + filterOpt: Option[PathFilter]): RDD[Row] = { assert(!hiveTable.isPartitioned, """makeRDDForTable() cannot be called on a partitioned table, since input formats may differ across partitions. Use makeRDDForTablePartitions() instead.""") // Create local references to member variables, so that the entire `this` object won't be // serialized in the closure below. - val tableDesc = _tableDesc + val tableDesc = relation.tableDesc val broadcastedHiveConf = _broadcastedHiveConf val tablePath = hiveTable.getPath @@ -99,23 +107,20 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) + val attrsWithIndex = attributes.zipWithIndex + val mutableRow = new GenericMutableRow(attrsWithIndex.length) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = deserializerClass.newInstance() deserializer.initialize(hconf, tableDesc.getProperties) - // Deserialize each Writable to get the row value. - iter.map { - case v: Writable => deserializer.deserialize(v) - case value => - sys.error(s"Unable to deserialize non-Writable: $value of ${value.getClass.getName}") - } + HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow) } deserializedHadoopRDD } - override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] = { + override def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[Row] = { val partitionToDeserializer = partitions.map(part => (part, part.getDeserializer.getClass.asInstanceOf[Class[Deserializer]])).toMap makeRDDForPartitionedTable(partitionToDeserializer, filterOpt = None) @@ -132,9 +137,9 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon * subdirectory of each partition being read. If None, then all files are accepted. */ def makeRDDForPartitionedTable( - partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], - filterOpt: Option[PathFilter]): RDD[_] = { - + partitionToDeserializer: Map[HivePartition, + Class[_ <: Deserializer]], + filterOpt: Option[PathFilter]): RDD[Row] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) val partPath = partition.getPartitionPath @@ -156,33 +161,42 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon } // Create local references so that the outer object isn't serialized. - val tableDesc = _tableDesc + val tableDesc = relation.tableDesc val broadcastedHiveConf = _broadcastedHiveConf val localDeserializer = partDeserializer + val mutableRow = new GenericMutableRow(attributes.length) + + // split the attributes (output schema) into 2 categories: + // (partition keys, ordinal), (normal attributes, ordinal), the ordinal mean the + // index of the attribute in the output Row. + val (partitionKeys, attrs) = attributes.zipWithIndex.partition(attr => { + relation.partitionKeys.indexOf(attr._1) >= 0 + }) + + def fillPartitionKeys(parts: Array[String], row: GenericMutableRow) = { + partitionKeys.foreach { case (attr, ordinal) => + // get partition key ordinal for a given attribute + val partOridinal = relation.partitionKeys.indexOf(attr) + row(ordinal) = Cast(Literal(parts(partOridinal)), attr.dataType).eval(null) + } + } + // fill the partition key for the given MutableRow Object + fillPartitionKeys(partValues, mutableRow) val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) 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 deserializedRow = deserializer.deserialize(value) - rowWithPartArr.update(0, deserializedRow) - rowWithPartArr.asInstanceOf[Object] - } + // fill the non partition key attributes + HadoopTableReader.fillObject(iter, deserializer, attrs, mutableRow) } }.toSeq // Even if we don't use any partitions, we still need an empty RDD if (hivePartitionRDDs.size == 0) { - new EmptyRDD[Object](sc.sparkContext) + new EmptyRDD[Row](sc.sparkContext) } else { new UnionRDD(hivePartitionRDDs(0).context, hivePartitionRDDs) } @@ -225,10 +239,9 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) } - } -private[hive] object HadoopTableReader { +private[hive] object HadoopTableReader extends HiveInspectors { /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to * instantiate a HadoopRDD. @@ -241,4 +254,40 @@ private[hive] object HadoopTableReader { val bufferSize = System.getProperty("spark.buffer.size", "65536") jobConf.set("io.file.buffer.size", bufferSize) } + + /** + * Transform the raw data(Writable object) into the Row object for an iterable input + * @param iter Iterable input which represented as Writable object + * @param deserializer Deserializer associated with the input writable object + * @param attrs Represents the row attribute names and its zero-based position in the MutableRow + * @param row reusable MutableRow object + * + * @return Iterable Row object that transformed from the given iterable input. + */ + def fillObject( + iter: Iterator[Writable], + deserializer: Deserializer, + attrs: Seq[(Attribute, Int)], + row: GenericMutableRow): Iterator[Row] = { + val soi = deserializer.getObjectInspector().asInstanceOf[StructObjectInspector] + // get the field references according to the attributes(output of the reader) required + val fieldRefs = attrs.map { case (attr, idx) => (soi.getStructFieldRef(attr.name), idx) } + + // Map each tuple to a row object + iter.map { value => + val raw = deserializer.deserialize(value) + var idx = 0; + while (idx < fieldRefs.length) { + val fieldRef = fieldRefs(idx)._1 + val fieldIdx = fieldRefs(idx)._2 + val fieldValue = soi.getStructFieldData(raw, fieldRef) + + row(fieldIdx) = unwrapData(fieldValue, fieldRef.getFieldObjectInspector()) + + idx += 1 + } + + row: Row + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index e7016fa16eea9..8920e2a76a27f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive._ -import org.apache.spark.util.MutablePair /** * :: DeveloperApi :: @@ -50,8 +49,7 @@ case class HiveTableScan( relation: MetastoreRelation, partitionPruningPred: Option[Expression])( @transient val context: HiveContext) - extends LeafNode - with HiveInspectors { + extends LeafNode { require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, "Partition pruning predicates only supported for partitioned tables.") @@ -67,42 +65,7 @@ case class HiveTableScan( } @transient - private[this] val hadoopReader = new HadoopTableReader(relation.tableDesc, context) - - /** - * The hive object inspector for this table, which can be used to extract values from the - * serialized row representation. - */ - @transient - private[this] lazy val objectInspector = - relation.tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] - - /** - * Functions that extract the requested attributes from the hive output. Partitioned values are - * casted from string to its declared data type. - */ - @transient - protected lazy val attributeFunctions: Seq[(Any, Array[String]) => Any] = { - attributes.map { a => - val ordinal = relation.partitionKeys.indexOf(a) - if (ordinal >= 0) { - val dataType = relation.partitionKeys(ordinal).dataType - (_: Any, partitionKeys: Array[String]) => { - castFromString(partitionKeys(ordinal), dataType) - } - } else { - val ref = objectInspector.getAllStructFieldRefs - .find(_.getFieldName == a.name) - .getOrElse(sys.error(s"Can't find attribute $a")) - val fieldObjectInspector = ref.getFieldObjectInspector - - (row: Any, _: Array[String]) => { - val data = objectInspector.getStructFieldData(row, ref) - unwrapData(data, fieldObjectInspector) - } - } - } - } + private[this] val hadoopReader = new HadoopTableReader(attributes, relation, context) private[this] def castFromString(value: String, dataType: DataType) = { Cast(Literal(value), dataType).eval(null) @@ -114,6 +77,7 @@ case class HiveTableScan( val columnInternalNames = neededColumnIDs.map(HiveConf.getColumnInternalName(_)).mkString(",") if (attributes.size == relation.output.size) { + // SQLContext#pruneFilterProject guarantees no duplicated value in `attributes` ColumnProjectionUtils.setFullyReadColumns(hiveConf) } else { ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) @@ -140,12 +104,6 @@ case class HiveTableScan( addColumnMetadataToConf(context.hiveconf) - private def inputRdd = if (!relation.hiveQlTable.isPartitioned) { - hadoopReader.makeRDDForTable(relation.hiveQlTable) - } else { - hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) - } - /** * Prunes partitions not involve the query plan. * @@ -169,44 +127,10 @@ case class HiveTableScan( } } - override def execute() = { - inputRdd.mapPartitions { iterator => - if (iterator.isEmpty) { - Iterator.empty - } else { - val mutableRow = new GenericMutableRow(attributes.length) - val mutablePair = new MutablePair[Any, Array[String]]() - val buffered = iterator.buffered - - // NOTE (lian): Critical path of Hive table scan, unnecessary FP style code and pattern - // matching are avoided intentionally. - val rowsAndPartitionKeys = buffered.head match { - // With partition keys - case _: Array[Any] => - buffered.map { case array: Array[Any] => - val deserializedRow = array(0) - val partitionKeys = array(1).asInstanceOf[Array[String]] - mutablePair.update(deserializedRow, partitionKeys) - } - - // Without partition keys - case _ => - val emptyPartitionKeys = Array.empty[String] - buffered.map { deserializedRow => - mutablePair.update(deserializedRow, emptyPartitionKeys) - } - } - - rowsAndPartitionKeys.map { pair => - var i = 0 - while (i < attributes.length) { - mutableRow(i) = attributeFunctions(i)(pair._1, pair._2) - i += 1 - } - mutableRow: Row - } - } - } + override def execute() = if (!relation.hiveQlTable.isPartitioned) { + hadoopReader.makeRDDForTable(relation.hiveQlTable) + } else { + hadoopReader.makeRDDForPartitionedTable(prunePartitions(relation.hiveQlPartitions)) } override def output = attributes diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-8caed2a6e80250a6d38a59388679c298 b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-8caed2a6e80250a6d38a59388679c298 new file mode 100644 index 0000000000000..f369f21e1833f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-8caed2a6e80250a6d38a59388679c298 @@ -0,0 +1,2 @@ +100 100 2010-01-01 +200 200 2010-01-02 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala new file mode 100644 index 0000000000000..bcb00f871d185 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.hive.test.TestHive + +class HiveTableScanSuite extends HiveComparisonTest { + // MINOR HACK: You must run a query before calling reset the first time. + TestHive.hql("SHOW TABLES") + TestHive.reset() + + TestHive.hql("""CREATE TABLE part_scan_test (key STRING, value STRING) PARTITIONED BY (ds STRING) + | ROW FORMAT SERDE + | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' + | STORED AS RCFILE + """.stripMargin) + TestHive.hql("""FROM src + | INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-01') + | SELECT 100,100 LIMIT 1 + """.stripMargin) + TestHive.hql("""ALTER TABLE part_scan_test SET SERDE + | 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' + """.stripMargin) + TestHive.hql("""FROM src INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-02') + | SELECT 200,200 LIMIT 1 + """.stripMargin) + + createQueryTest("partition_based_table_scan_with_different_serde", + "SELECT * from part_scan_test", false) +} From 255b56f9f530e8594a7e6055ae07690454c66799 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 28 Jul 2014 11:34:19 -0700 Subject: [PATCH 0100/1492] [SPARK-2479][MLlib] Comparing floating-point numbers using relative error in UnitTests Floating point math is not exact, and most floating-point numbers end up being slightly imprecise due to rounding errors. Simple values like 0.1 cannot be precisely represented using binary floating point numbers, and the limited precision of floating point numbers means that slight changes in the order of operations or the precision of intermediates can change the result. That means that comparing two floats to see if they are equal is usually not what we want. As long as this imprecision stays small, it can usually be ignored. Based on discussion in the community, we have implemented two different APIs for relative tolerance, and absolute tolerance. It makes sense that test writers should know which one they need depending on their circumstances. Developers also need to explicitly specify the eps, and there is no default value which will sometimes cause confusion. When comparing against zero using relative tolerance, a exception will be raised to warn users that it's meaningless. For relative tolerance, users can now write assert(23.1 ~== 23.52 relTol 0.02) assert(23.1 ~== 22.74 relTol 0.02) assert(23.1 ~= 23.52 relTol 0.02) assert(23.1 ~= 22.74 relTol 0.02) assert(!(23.1 !~= 23.52 relTol 0.02)) assert(!(23.1 !~= 22.74 relTol 0.02)) // This will throw exception with the following message. // "Did not expect 23.1 and 23.52 to be within 0.02 using relative tolerance." assert(23.1 !~== 23.52 relTol 0.02) // "Expected 23.1 and 22.34 to be within 0.02 using relative tolerance." assert(23.1 ~== 22.34 relTol 0.02) For absolute error, assert(17.8 ~== 17.99 absTol 0.2) assert(17.8 ~== 17.61 absTol 0.2) assert(17.8 ~= 17.99 absTol 0.2) assert(17.8 ~= 17.61 absTol 0.2) assert(!(17.8 !~= 17.99 absTol 0.2)) assert(!(17.8 !~= 17.61 absTol 0.2)) // This will throw exception with the following message. // "Did not expect 17.8 and 17.99 to be within 0.2 using absolute error." assert(17.8 !~== 17.99 absTol 0.2) // "Expected 17.8 and 17.59 to be within 0.2 using absolute error." assert(17.8 ~== 17.59 absTol 0.2) Authors: DB Tsai Marek Kolodziej Author: DB Tsai Closes #1425 from dbtsai/SPARK-2479_comparing_floating_point and squashes the following commits: 8c7cbcc [DB Tsai] Alpine Data Labs --- .../LogisticRegressionSuite.scala | 12 +- .../spark/mllib/clustering/KMeansSuite.scala | 63 +++--- .../evaluation/AreaUnderCurveSuite.scala | 13 +- .../BinaryClassificationMetricsSuite.scala | 40 ++-- .../optimization/GradientDescentSuite.scala | 16 +- .../spark/mllib/optimization/LBFGSSuite.scala | 17 +- .../spark/mllib/optimization/NNLSSuite.scala | 6 +- .../MultivariateOnlineSummarizerSuite.scala | 68 +++---- .../spark/mllib/util/TestingUtils.scala | 151 +++++++++++++-- .../spark/mllib/util/TestingUtilsSuite.scala | 182 ++++++++++++++++++ 10 files changed, 438 insertions(+), 130 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 3f6ff859374c7..da7c633bbd2af 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.Matchers import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ object LogisticRegressionSuite { @@ -81,9 +82,8 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match val model = lr.run(testRDD) // Test the weights - val weight0 = model.weights(0) - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + assert(model.weights(0) ~== -1.52 relTol 0.01) + assert(model.intercept ~== 2.00 relTol 0.01) val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) val validationRDD = sc.parallelize(validationData, 2) @@ -113,9 +113,9 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match val model = lr.run(testRDD, initialWeights) - val weight0 = model.weights(0) - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") + // Test the weights + assert(model.weights(0) ~== -1.50 relTol 0.01) + assert(model.intercept ~== 1.97 relTol 0.01) val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) val validationRDD = sc.parallelize(validationData, 2) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 34bc4537a7b3a..afa1f79b95a12 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -21,8 +21,9 @@ import scala.util.Random import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ class KMeansSuite extends FunSuite with LocalSparkContext { @@ -41,26 +42,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // centered at the mean of the points var model = KMeans.train(data, k = 1, maxIterations = 1) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 2) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train( data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) } test("no distinct points") { @@ -104,26 +105,26 @@ class KMeansSuite extends FunSuite with LocalSparkContext { var model = KMeans.train(data, k = 1, maxIterations = 1) assert(model.clusterCenters.size === 1) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 2) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) } test("single cluster with sparse data") { @@ -149,31 +150,39 @@ class KMeansSuite extends FunSuite with LocalSparkContext { val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) var model = KMeans.train(data, k = 1, maxIterations = 1) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 2) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 5) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = RANDOM) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) model = KMeans.train(data, k = 1, maxIterations = 1, runs = 1, initializationMode = K_MEANS_PARALLEL) - assert(model.clusterCenters.head === center) + assert(model.clusterCenters.head ~== center absTol 1E-5) data.unpersist() } test("k-means|| initialization") { + + case class VectorWithCompare(x: Vector) extends Ordered[VectorWithCompare] { + @Override def compare(that: VectorWithCompare): Int = { + if(this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > + that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) -1 else 1 + } + } + val points = Seq( Vectors.dense(1.0, 2.0, 6.0), Vectors.dense(1.0, 3.0, 0.0), @@ -188,15 +197,19 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // unselected point as long as it hasn't yet selected all of them var model = KMeans.train(rdd, k = 5, maxIterations = 1) - assert(Set(model.clusterCenters: _*) === Set(points: _*)) + + assert(model.clusterCenters.sortBy(VectorWithCompare(_)) + .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) // Iterations of Lloyd's should not change the answer either model = KMeans.train(rdd, k = 5, maxIterations = 10) - assert(Set(model.clusterCenters: _*) === Set(points: _*)) + assert(model.clusterCenters.sortBy(VectorWithCompare(_)) + .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) // Neither should more runs model = KMeans.train(rdd, k = 5, maxIterations = 10, runs = 5) - assert(Set(model.clusterCenters: _*) === Set(points: _*)) + assert(model.clusterCenters.sortBy(VectorWithCompare(_)) + .zip(points.sortBy(VectorWithCompare(_))).forall(x => x._1 ~== (x._2) absTol 1E-5)) } test("two clusters") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala index 1c9844f289fe0..994e0feb8629e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala @@ -20,27 +20,28 @@ package org.apache.spark.mllib.evaluation import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class AreaUnderCurveSuite extends FunSuite with LocalSparkContext { test("auc computation") { val curve = Seq((0.0, 0.0), (1.0, 1.0), (2.0, 3.0), (3.0, 0.0)) val auc = 4.0 - assert(AreaUnderCurve.of(curve) === auc) + assert(AreaUnderCurve.of(curve) ~== auc absTol 1E-5) val rddCurve = sc.parallelize(curve, 2) - assert(AreaUnderCurve.of(rddCurve) == auc) + assert(AreaUnderCurve.of(rddCurve) ~== auc absTol 1E-5) } test("auc of an empty curve") { val curve = Seq.empty[(Double, Double)] - assert(AreaUnderCurve.of(curve) === 0.0) + assert(AreaUnderCurve.of(curve) ~== 0.0 absTol 1E-5) val rddCurve = sc.parallelize(curve, 2) - assert(AreaUnderCurve.of(rddCurve) === 0.0) + assert(AreaUnderCurve.of(rddCurve) ~== 0.0 absTol 1E-5) } test("auc of a curve with a single point") { val curve = Seq((1.0, 1.0)) - assert(AreaUnderCurve.of(curve) === 0.0) + assert(AreaUnderCurve.of(curve) ~== 0.0 absTol 1E-5) val rddCurve = sc.parallelize(curve, 2) - assert(AreaUnderCurve.of(rddCurve) === 0.0) + assert(AreaUnderCurve.of(rddCurve) ~== 0.0 absTol 1E-5) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index 94db1dc183230..a733f88b60b80 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -20,25 +20,14 @@ package org.apache.spark.mllib.evaluation import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.util.TestingUtils.DoubleWithAlmostEquals +import org.apache.spark.mllib.util.TestingUtils._ class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { - // TODO: move utility functions to TestingUtils. + def cond1(x: (Double, Double)): Boolean = x._1 ~= (x._2) absTol 1E-5 - def elementsAlmostEqual(actual: Seq[Double], expected: Seq[Double]): Boolean = { - actual.zip(expected).forall { case (x1, x2) => - x1.almostEquals(x2) - } - } - - def elementsAlmostEqual( - actual: Seq[(Double, Double)], - expected: Seq[(Double, Double)])(implicit dummy: DummyImplicit): Boolean = { - actual.zip(expected).forall { case ((x1, y1), (x2, y2)) => - x1.almostEquals(x2) && y1.almostEquals(y2) - } - } + def cond2(x: ((Double, Double), (Double, Double))): Boolean = + (x._1._1 ~= x._2._1 absTol 1E-5) && (x._1._2 ~= x._2._2 absTol 1E-5) test("binary evaluation metrics") { val scoreAndLabels = sc.parallelize( @@ -57,16 +46,17 @@ class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { val rocCurve = Seq((0.0, 0.0)) ++ fpr.zip(recall) ++ Seq((1.0, 1.0)) val pr = recall.zip(precision) val prCurve = Seq((0.0, 1.0)) ++ pr - val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r) } + val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r)} val f2 = pr.map { case (r, p) => 5.0 * (p * r) / (4.0 * p + r)} - assert(elementsAlmostEqual(metrics.thresholds().collect(), threshold)) - assert(elementsAlmostEqual(metrics.roc().collect(), rocCurve)) - assert(metrics.areaUnderROC().almostEquals(AreaUnderCurve.of(rocCurve))) - assert(elementsAlmostEqual(metrics.pr().collect(), prCurve)) - assert(metrics.areaUnderPR().almostEquals(AreaUnderCurve.of(prCurve))) - assert(elementsAlmostEqual(metrics.fMeasureByThreshold().collect(), threshold.zip(f1))) - assert(elementsAlmostEqual(metrics.fMeasureByThreshold(2.0).collect(), threshold.zip(f2))) - assert(elementsAlmostEqual(metrics.precisionByThreshold().collect(), threshold.zip(precision))) - assert(elementsAlmostEqual(metrics.recallByThreshold().collect(), threshold.zip(recall))) + + assert(metrics.thresholds().collect().zip(threshold).forall(cond1)) + assert(metrics.roc().collect().zip(rocCurve).forall(cond2)) + assert(metrics.areaUnderROC() ~== AreaUnderCurve.of(rocCurve) absTol 1E-5) + assert(metrics.pr().collect().zip(prCurve).forall(cond2)) + assert(metrics.areaUnderPR() ~== AreaUnderCurve.of(prCurve) absTol 1E-5) + assert(metrics.fMeasureByThreshold().collect().zip(threshold.zip(f1)).forall(cond2)) + assert(metrics.fMeasureByThreshold(2.0).collect().zip(threshold.zip(f2)).forall(cond2)) + assert(metrics.precisionByThreshold().collect().zip(threshold.zip(precision)).forall(cond2)) + assert(metrics.recallByThreshold().collect().zip(threshold.zip(recall)).forall(cond2)) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index dfb2eb7f0d14e..bf040110e228b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.{FunSuite, Matchers} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ object GradientDescentSuite { @@ -126,19 +127,14 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with Matchers val (newWeights1, loss1) = GradientDescent.runMiniBatchSGD( dataRDD, gradient, updater, 1, 1, regParam1, 1.0, initialWeightsWithIntercept) - def compareDouble(x: Double, y: Double, tol: Double = 1E-3): Boolean = { - math.abs(x - y) / (math.abs(y) + 1e-15) < tol - } - - assert(compareDouble( - loss1(0), - loss0(0) + (math.pow(initialWeightsWithIntercept(0), 2) + - math.pow(initialWeightsWithIntercept(1), 2)) / 2), + assert( + loss1(0) ~= (loss0(0) + (math.pow(initialWeightsWithIntercept(0), 2) + + math.pow(initialWeightsWithIntercept(1), 2)) / 2) absTol 1E-5, """For non-zero weights, the regVal should be \frac{1}{2}\sum_i w_i^2.""") assert( - compareDouble(newWeights1(0) , newWeights0(0) - initialWeightsWithIntercept(0)) && - compareDouble(newWeights1(1) , newWeights0(1) - initialWeightsWithIntercept(1)), + (newWeights1(0) ~= (newWeights0(0) - initialWeightsWithIntercept(0)) absTol 1E-5) && + (newWeights1(1) ~= (newWeights0(1) - initialWeightsWithIntercept(1)) absTol 1E-5), "The different between newWeights with/without regularization " + "should be initialWeightsWithIntercept.") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index ff414742e8393..5f4c24115ac80 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -24,6 +24,7 @@ import org.scalatest.{FunSuite, Matchers} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} +import org.apache.spark.mllib.util.TestingUtils._ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { @@ -49,10 +50,6 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { lazy val dataRDD = sc.parallelize(data, 2).cache() - def compareDouble(x: Double, y: Double, tol: Double = 1E-3): Boolean = { - math.abs(x - y) / (math.abs(y) + 1e-15) < tol - } - test("LBFGS loss should be decreasing and match the result of Gradient Descent.") { val regParam = 0 @@ -126,15 +123,15 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { miniBatchFrac, initialWeightsWithIntercept) - assert(compareDouble(lossGD(0), lossLBFGS(0)), + assert(lossGD(0) ~= lossLBFGS(0) absTol 1E-5, "The first losses of LBFGS and GD should be the same.") // The 2% difference here is based on observation, but is not theoretically guaranteed. - assert(compareDouble(lossGD.last, lossLBFGS.last, 0.02), + assert(lossGD.last ~= lossLBFGS.last relTol 0.02, "The last losses of LBFGS and GD should be within 2% difference.") - assert(compareDouble(weightLBFGS(0), weightGD(0), 0.02) && - compareDouble(weightLBFGS(1), weightGD(1), 0.02), + assert( + (weightLBFGS(0) ~= weightGD(0) relTol 0.02) && (weightLBFGS(1) ~= weightGD(1) relTol 0.02), "The weight differences between LBFGS and GD should be within 2%.") } @@ -226,8 +223,8 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { initialWeightsWithIntercept) // for class LBFGS and the optimize method, we only look at the weights - assert(compareDouble(weightLBFGS(0), weightGD(0), 0.02) && - compareDouble(weightLBFGS(1), weightGD(1), 0.02), + assert( + (weightLBFGS(0) ~= weightGD(0) relTol 0.02) && (weightLBFGS(1) ~= weightGD(1) relTol 0.02), "The weight differences between LBFGS and GD should be within 2%.") } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index bbf385229081a..b781a6aed9a8c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -21,7 +21,9 @@ import scala.util.Random import org.scalatest.FunSuite -import org.jblas.{DoubleMatrix, SimpleBlas, NativeBlas} +import org.jblas.{DoubleMatrix, SimpleBlas} + +import org.apache.spark.mllib.util.TestingUtils._ class NNLSSuite extends FunSuite { /** Generate an NNLS problem whose optimal solution is the all-ones vector. */ @@ -73,7 +75,7 @@ class NNLSSuite extends FunSuite { val ws = NNLS.createWorkspace(n) val x = NNLS.solve(ata, atb, ws) for (i <- 0 until n) { - assert(Math.abs(x(i) - goodx(i)) < 1e-3) + assert(x(i) ~== goodx(i) absTol 1E-3) assert(x(i) >= 0) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala index 4b7b019d820b4..db13f142df517 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala @@ -89,15 +89,15 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { .add(Vectors.dense(-1.0, 0.0, 6.0)) .add(Vectors.dense(3.0, -3.0, 0.0)) - assert(summarizer.mean.almostEquals(Vectors.dense(1.0, -1.5, 3.0)), "mean mismatch") + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") - assert(summarizer.min.almostEquals(Vectors.dense(-1.0, -3, 0.0)), "min mismatch") + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") - assert(summarizer.max.almostEquals(Vectors.dense(3.0, 0.0, 6.0)), "max mismatch") + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") - assert(summarizer.numNonzeros.almostEquals(Vectors.dense(2, 1, 1)), "numNonzeros mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") - assert(summarizer.variance.almostEquals(Vectors.dense(8.0, 4.5, 18.0)), "variance mismatch") + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") assert(summarizer.count === 2) } @@ -107,15 +107,15 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { .add(Vectors.sparse(3, Seq((0, -1.0), (2, 6.0)))) .add(Vectors.sparse(3, Seq((0, 3.0), (1, -3.0)))) - assert(summarizer.mean.almostEquals(Vectors.dense(1.0, -1.5, 3.0)), "mean mismatch") + assert(summarizer.mean ~== Vectors.dense(1.0, -1.5, 3.0) absTol 1E-5, "mean mismatch") - assert(summarizer.min.almostEquals(Vectors.dense(-1.0, -3, 0.0)), "min mismatch") + assert(summarizer.min ~== Vectors.dense(-1.0, -3, 0.0) absTol 1E-5, "min mismatch") - assert(summarizer.max.almostEquals(Vectors.dense(3.0, 0.0, 6.0)), "max mismatch") + assert(summarizer.max ~== Vectors.dense(3.0, 0.0, 6.0) absTol 1E-5, "max mismatch") - assert(summarizer.numNonzeros.almostEquals(Vectors.dense(2, 1, 1)), "numNonzeros mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(2, 1, 1) absTol 1E-5, "numNonzeros mismatch") - assert(summarizer.variance.almostEquals(Vectors.dense(8.0, 4.5, 18.0)), "variance mismatch") + assert(summarizer.variance ~== Vectors.dense(8.0, 4.5, 18.0) absTol 1E-5, "variance mismatch") assert(summarizer.count === 2) } @@ -129,17 +129,17 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { .add(Vectors.dense(1.7, -0.6, 0.0)) .add(Vectors.sparse(3, Seq((1, 1.9), (2, 0.0)))) - assert(summarizer.mean.almostEquals( - Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333)), "mean mismatch") + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") - assert(summarizer.min.almostEquals(Vectors.dense(-2.0, -5.1, -3)), "min mismatch") + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") - assert(summarizer.max.almostEquals(Vectors.dense(3.8, 2.3, 1.9)), "max mismatch") + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") - assert(summarizer.numNonzeros.almostEquals(Vectors.dense(3, 5, 2)), "numNonzeros mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") - assert(summarizer.variance.almostEquals( - Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666)), "variance mismatch") + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") assert(summarizer.count === 6) } @@ -157,17 +157,17 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { val summarizer = summarizer1.merge(summarizer2) - assert(summarizer.mean.almostEquals( - Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333)), "mean mismatch") + assert(summarizer.mean ~== + Vectors.dense(0.583333333333, -0.416666666666, -0.183333333333) absTol 1E-5, "mean mismatch") - assert(summarizer.min.almostEquals(Vectors.dense(-2.0, -5.1, -3)), "min mismatch") + assert(summarizer.min ~== Vectors.dense(-2.0, -5.1, -3) absTol 1E-5, "min mismatch") - assert(summarizer.max.almostEquals(Vectors.dense(3.8, 2.3, 1.9)), "max mismatch") + assert(summarizer.max ~== Vectors.dense(3.8, 2.3, 1.9) absTol 1E-5, "max mismatch") - assert(summarizer.numNonzeros.almostEquals(Vectors.dense(3, 5, 2)), "numNonzeros mismatch") + assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") - assert(summarizer.variance.almostEquals( - Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666)), "variance mismatch") + assert(summarizer.variance ~== + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") assert(summarizer.count === 6) } @@ -186,24 +186,24 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { val summarizer3 = (new MultivariateOnlineSummarizer).merge(new MultivariateOnlineSummarizer) assert(summarizer3.count === 0) - assert(summarizer1.mean.almostEquals(Vectors.dense(0.0, -1.0, -3.0)), "mean mismatch") + assert(summarizer1.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") - assert(summarizer2.mean.almostEquals(Vectors.dense(0.0, -1.0, -3.0)), "mean mismatch") + assert(summarizer2.mean ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "mean mismatch") - assert(summarizer1.min.almostEquals(Vectors.dense(0.0, -1.0, -3.0)), "min mismatch") + assert(summarizer1.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") - assert(summarizer2.min.almostEquals(Vectors.dense(0.0, -1.0, -3.0)), "min mismatch") + assert(summarizer2.min ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "min mismatch") - assert(summarizer1.max.almostEquals(Vectors.dense(0.0, -1.0, -3.0)), "max mismatch") + assert(summarizer1.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") - assert(summarizer2.max.almostEquals(Vectors.dense(0.0, -1.0, -3.0)), "max mismatch") + assert(summarizer2.max ~== Vectors.dense(0.0, -1.0, -3.0) absTol 1E-5, "max mismatch") - assert(summarizer1.numNonzeros.almostEquals(Vectors.dense(0, 1, 1)), "numNonzeros mismatch") + assert(summarizer1.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") - assert(summarizer2.numNonzeros.almostEquals(Vectors.dense(0, 1, 1)), "numNonzeros mismatch") + assert(summarizer2.numNonzeros ~== Vectors.dense(0, 1, 1) absTol 1E-5, "numNonzeros mismatch") - assert(summarizer1.variance.almostEquals(Vectors.dense(0, 0, 0)), "variance mismatch") + assert(summarizer1.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") - assert(summarizer2.variance.almostEquals(Vectors.dense(0, 0, 0)), "variance mismatch") + assert(summarizer2.variance ~== Vectors.dense(0, 0, 0) absTol 1E-5, "variance mismatch") } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index 64b1ba7527183..29cc42d8cbea7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -18,28 +18,155 @@ package org.apache.spark.mllib.util import org.apache.spark.mllib.linalg.Vector +import org.scalatest.exceptions.TestFailedException object TestingUtils { + val ABS_TOL_MSG = " using absolute tolerance" + val REL_TOL_MSG = " using relative tolerance" + + /** + * Private helper function for comparing two values using relative tolerance. + * Note that if x or y is extremely close to zero, i.e., smaller than Double.MinPositiveValue, + * the relative tolerance is meaningless, so the exception will be raised to warn users. + */ + private def RelativeErrorComparison(x: Double, y: Double, eps: Double): Boolean = { + val absX = math.abs(x) + val absY = math.abs(y) + val diff = math.abs(x - y) + if (x == y) { + true + } else if (absX < Double.MinPositiveValue || absY < Double.MinPositiveValue) { + throw new TestFailedException( + s"$x or $y is extremely close to zero, so the relative tolerance is meaningless.", 0) + } else { + diff < eps * math.min(absX, absY) + } + } + + /** + * Private helper function for comparing two values using absolute tolerance. + */ + private def AbsoluteErrorComparison(x: Double, y: Double, eps: Double): Boolean = { + math.abs(x - y) < eps + } + + case class CompareDoubleRightSide( + fun: (Double, Double, Double) => Boolean, y: Double, eps: Double, method: String) + + /** + * Implicit class for comparing two double values using relative tolerance or absolute tolerance. + */ implicit class DoubleWithAlmostEquals(val x: Double) { - // An improved version of AlmostEquals would always divide by the larger number. - // This will avoid the problem of diving by zero. - def almostEquals(y: Double, epsilon: Double = 1E-10): Boolean = { - if(x == y) { - true - } else if(math.abs(x) > math.abs(y)) { - math.abs(x - y) / math.abs(x) < epsilon - } else { - math.abs(x - y) / math.abs(y) < epsilon + + /** + * When the difference of two values are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareDoubleRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two values are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareDoubleRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two values are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareDoubleRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected $x and ${r.y} to be within ${r.eps}${r.method}.", 0) } + true } + + /** + * Throws exception when the difference of two values are within eps; otherwise, returns true. + */ + def !~==(r: CompareDoubleRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect $x and ${r.y} to be within ${r.eps}${r.method}.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(AbsoluteErrorComparison, + x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. + */ + def relTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(RelativeErrorComparison, + x, eps, REL_TOL_MSG) + + override def toString = x.toString } + case class CompareVectorRightSide( + fun: (Vector, Vector, Double) => Boolean, y: Vector, eps: Double, method: String) + + /** + * Implicit class for comparing two vectors using relative tolerance or absolute tolerance. + */ implicit class VectorWithAlmostEquals(val x: Vector) { - def almostEquals(y: Vector, epsilon: Double = 1E-10): Boolean = { - x.toArray.corresponds(y.toArray) { - _.almostEquals(_, epsilon) + + /** + * When the difference of two vectors are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareVectorRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two vectors are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareVectorRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two vectors are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareVectorRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected $x and ${r.y} to be within ${r.eps}${r.method} for all elements.", 0) } + true } + + /** + * Throws exception when the difference of two vectors are within eps; otherwise, returns true. + */ + def !~==(r: CompareVectorRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect $x and ${r.y} to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( + (x: Vector, y: Vector, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + }, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. Note that comparing against sparse vector + * with elements having value of zero will raise exception because it involves with + * comparing against zero. + */ + def relTol(eps: Double): CompareVectorRightSide = CompareVectorRightSide( + (x: Vector, y: Vector, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + }, x, eps, REL_TOL_MSG) + + override def toString = x.toString } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala new file mode 100644 index 0000000000000..b0ecb33c28483 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -0,0 +1,182 @@ +/* + * 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.util + +import org.apache.spark.mllib.linalg.Vectors +import org.scalatest.FunSuite +import org.apache.spark.mllib.util.TestingUtils._ +import org.scalatest.exceptions.TestFailedException + +class TestingUtilsSuite extends FunSuite { + + test("Comparing doubles using relative error.") { + + assert(23.1 ~== 23.52 relTol 0.02) + assert(23.1 ~== 22.74 relTol 0.02) + assert(23.1 ~= 23.52 relTol 0.02) + assert(23.1 ~= 22.74 relTol 0.02) + assert(!(23.1 !~= 23.52 relTol 0.02)) + assert(!(23.1 !~= 22.74 relTol 0.02)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](23.1 !~== 23.52 relTol 0.02) + intercept[TestFailedException](23.1 !~== 22.74 relTol 0.02) + intercept[TestFailedException](23.1 ~== 23.63 relTol 0.02) + intercept[TestFailedException](23.1 ~== 22.34 relTol 0.02) + + assert(23.1 !~== 23.63 relTol 0.02) + assert(23.1 !~== 22.34 relTol 0.02) + assert(23.1 !~= 23.63 relTol 0.02) + assert(23.1 !~= 22.34 relTol 0.02) + assert(!(23.1 ~= 23.63 relTol 0.02)) + assert(!(23.1 ~= 22.34 relTol 0.02)) + + // Comparing against zero should fail the test and throw exception with message + // saying that the relative error is meaningless in this situation. + intercept[TestFailedException](0.1 ~== 0.0 relTol 0.032) + intercept[TestFailedException](0.1 ~= 0.0 relTol 0.032) + intercept[TestFailedException](0.1 !~== 0.0 relTol 0.032) + intercept[TestFailedException](0.1 !~= 0.0 relTol 0.032) + intercept[TestFailedException](0.0 ~== 0.1 relTol 0.032) + intercept[TestFailedException](0.0 ~= 0.1 relTol 0.032) + intercept[TestFailedException](0.0 !~== 0.1 relTol 0.032) + intercept[TestFailedException](0.0 !~= 0.1 relTol 0.032) + + // Comparisons of numbers very close to zero. + assert(10 * Double.MinPositiveValue ~== 9.5 * Double.MinPositiveValue relTol 0.01) + assert(10 * Double.MinPositiveValue !~== 11 * Double.MinPositiveValue relTol 0.01) + + assert(-Double.MinPositiveValue ~== 1.18 * -Double.MinPositiveValue relTol 0.012) + assert(-Double.MinPositiveValue ~== 1.38 * -Double.MinPositiveValue relTol 0.012) + } + + test("Comparing doubles using absolute error.") { + + assert(17.8 ~== 17.99 absTol 0.2) + assert(17.8 ~== 17.61 absTol 0.2) + assert(17.8 ~= 17.99 absTol 0.2) + assert(17.8 ~= 17.61 absTol 0.2) + assert(!(17.8 !~= 17.99 absTol 0.2)) + assert(!(17.8 !~= 17.61 absTol 0.2)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](17.8 !~== 17.99 absTol 0.2) + intercept[TestFailedException](17.8 !~== 17.61 absTol 0.2) + intercept[TestFailedException](17.8 ~== 18.01 absTol 0.2) + intercept[TestFailedException](17.8 ~== 17.59 absTol 0.2) + + assert(17.8 !~== 18.01 absTol 0.2) + assert(17.8 !~== 17.59 absTol 0.2) + assert(17.8 !~= 18.01 absTol 0.2) + assert(17.8 !~= 17.59 absTol 0.2) + assert(!(17.8 ~= 18.01 absTol 0.2)) + assert(!(17.8 ~= 17.59 absTol 0.2)) + + // Comparisons of numbers very close to zero, and both side of zeros + assert(Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert(Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + + assert(-Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert(Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + } + + test("Comparing vectors using relative error.") { + + //Comparisons of two dense vectors + assert(Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + assert(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + assert(!(Vectors.dense(Array(3.1, 3.5)) !~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01)) + assert(!(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.135, 3.534)) relTol 0.01)) + + // Should throw exception with message when test fails. + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) + + // Comparing against zero should fail the test and throw exception with message + // saying that the relative error is meaningless in this situation. + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 0.01)) ~== Vectors.dense(Array(3.13, 0.0)) relTol 0.01) + + intercept[TestFailedException]( + Vectors.dense(Array(3.1, 0.01)) ~== Vectors.sparse(2, Array(0), Array(3.13)) relTol 0.01) + + // Comparisons of two sparse vectors + assert(Vectors.dense(Array(3.1, 3.5)) ~== + Vectors.sparse(2, Array(0, 1), Array(3.130, 3.534)) relTol 0.01) + + assert(Vectors.dense(Array(3.1, 3.5)) !~== + Vectors.sparse(2, Array(0, 1), Array(3.135, 3.534)) relTol 0.01) + } + + test("Comparing vectors using absolute error.") { + + //Comparisons of two dense vectors + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~== + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) !~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~= + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1, 3.5, 0.0)) !~= + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + assert(!(Vectors.dense(Array(3.1, 3.5, 0.0)) !~= + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6)) + + assert(!(Vectors.dense(Array(3.1, 3.5, 0.0)) ~= + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6)) + + // Should throw exception with message when test fails. + intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) !~== + Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) + + intercept[TestFailedException](Vectors.dense(Array(3.1, 3.5, 0.0)) ~== + Vectors.dense(Array(3.1 + 1E-5, 3.5 + 2E-7, 1 + 1E-3)) absTol 1E-6) + + // Comparisons of two sparse vectors + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-8, 2.4 + 1E-7)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-8, 2.4 + 1E-7)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-3, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1 + 1E-3, 2.4)) !~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + // Comparisons of a dense vector and a sparse vector + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) ~== + Vectors.dense(Array(3.1 + 1E-8, 0, 2.4 + 1E-7)) absTol 1E-6) + + assert(Vectors.dense(Array(3.1 + 1E-8, 0, 2.4 + 1E-7)) ~== + Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) absTol 1E-6) + + assert(Vectors.sparse(3, Array(0, 2), Array(3.1, 2.4)) !~== + Vectors.dense(Array(3.1, 1E-3, 2.4)) absTol 1E-6) + } +} From a7a9d14479ea6421513a962ff0f45cb969368bab Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 28 Jul 2014 12:07:30 -0700 Subject: [PATCH 0101/1492] [SPARK-2410][SQL] Merging Hive Thrift/JDBC server (with Maven profile fix) JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) Another try for #1399 & #1600. Those two PR breaks Jenkins builds because we made a separate profile `hive-thriftserver` in sub-project `assembly`, but the `hive-thriftserver` module is defined outside the `hive-thriftserver` profile. Thus every time a pull request that doesn't touch SQL code will also execute test suites defined in `hive-thriftserver`, but tests fail because related .class files are not included in the assembly jar. In the most recent commit, module `hive-thriftserver` is moved into its own profile to fix this problem. All previous commits are squashed for clarity. Author: Cheng Lian Closes #1620 from liancheng/jdbc-with-maven-fix and squashes the following commits: 629988e [Cheng Lian] Moved hive-thriftserver module definition into its own profile ec3c7a7 [Cheng Lian] Cherry picked the Hive Thrift server --- .gitignore | 1 + assembly/pom.xml | 10 + bagel/pom.xml | 2 +- bin/beeline | 45 +++ bin/compute-classpath.sh | 1 + bin/spark-shell | 4 +- bin/spark-shell.cmd | 2 +- bin/spark-sql | 36 ++ core/pom.xml | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 14 +- .../spark/deploy/SparkSubmitArguments.scala | 5 +- dev/create-release/create-release.sh | 10 +- dev/run-tests | 2 +- dev/scalastyle | 2 +- docs/sql-programming-guide.md | 201 +++++++++- examples/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 16 +- project/SparkBuild.scala | 14 +- sbin/start-thriftserver.sh | 36 ++ sql/catalyst/pom.xml | 2 +- .../sql/catalyst/plans/logical/commands.scala | 3 +- sql/core/pom.xml | 2 +- .../scala/org/apache/spark/sql/SQLConf.scala | 20 +- .../apache/spark/sql/execution/commands.scala | 42 ++- .../org/apache/spark/sql/SQLConfSuite.scala | 13 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- sql/hive-thriftserver/pom.xml | 82 +++++ .../hive/thriftserver/HiveThriftServer2.scala | 97 +++++ .../hive/thriftserver/ReflectionUtils.scala | 58 +++ .../hive/thriftserver/SparkSQLCLIDriver.scala | 344 ++++++++++++++++++ .../thriftserver/SparkSQLCLIService.scala | 74 ++++ .../hive/thriftserver/SparkSQLDriver.scala | 93 +++++ .../sql/hive/thriftserver/SparkSQLEnv.scala | 58 +++ .../thriftserver/SparkSQLSessionManager.scala | 49 +++ .../server/SparkSQLOperationManager.scala | 151 ++++++++ .../test/resources/data/files/small_kv.txt | 5 + .../sql/hive/thriftserver/CliSuite.scala | 57 +++ .../thriftserver/HiveThriftServer2Suite.scala | 135 +++++++ .../sql/hive/thriftserver/TestUtils.scala | 108 ++++++ sql/hive/pom.xml | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 50 ++- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 54 files changed, 1790 insertions(+), 96 deletions(-) create mode 100755 bin/beeline create mode 100755 bin/spark-sql create mode 100755 sbin/start-thriftserver.sh create mode 100644 sql/hive-thriftserver/pom.xml create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala create mode 100755 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala create mode 100644 sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala create mode 100644 sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala create mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala diff --git a/.gitignore b/.gitignore index a4ec12ca6b53f..7ec8d45e12c6b 100644 --- a/.gitignore +++ b/.gitignore @@ -58,3 +58,4 @@ metastore_db/ metastore/ warehouse/ TempStatsStore/ +sql/hive-thriftserver/test_warehouses diff --git a/assembly/pom.xml b/assembly/pom.xml index 567a8dd2a0d94..703f15925bc44 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -165,6 +165,16 @@ + + hive-thriftserver + + + org.apache.spark + spark-hive-thriftserver_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl diff --git a/bagel/pom.xml b/bagel/pom.xml index 90c4b095bb611..bd51b112e26fa 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-bagel_2.10 - bagel + bagel jar Spark Project Bagel diff --git a/bin/beeline b/bin/beeline new file mode 100755 index 0000000000000..09fe366c609fa --- /dev/null +++ b/bin/beeline @@ -0,0 +1,45 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +# Find the java binary +if [ -n "${JAVA_HOME}" ]; then + RUNNER="${JAVA_HOME}/bin/java" +else + if [ `command -v java` ]; then + RUNNER="java" + else + echo "JAVA_HOME is not set" >&2 + exit 1 + fi +fi + +# Compute classpath using external script +classpath_output=$($FWDIR/bin/compute-classpath.sh) +if [[ "$?" != "0" ]]; then + echo "$classpath_output" + exit 1 +else + CLASSPATH=$classpath_output +fi + +CLASS="org.apache.hive.beeline.BeeLine" +exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index e81e8c060cb98..16b794a1592e8 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -52,6 +52,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" fi diff --git a/bin/spark-shell b/bin/spark-shell index 850e9507ec38f..756c8179d12b6 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -46,11 +46,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit spark-shell "$@" --class org.apache.spark.repl.Main + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" fi } diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 4b9708a8c03f3..b56d69801171c 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell %* --class org.apache.spark.repl.Main +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell --class org.apache.spark.repl.Main %* diff --git a/bin/spark-sql b/bin/spark-sql new file mode 100755 index 0000000000000..bba7f897b19bc --- /dev/null +++ b/bin/spark-sql @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# Shell script for starting the Spark SQL CLI + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/spark-sql [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + +CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/core/pom.xml b/core/pom.xml index 1054cec4d77bb..a24743495b0e1 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-core_2.10 - core + core jar Spark Project Core diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3b5642b6caa36..c9cec33ebaa66 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -46,6 +46,10 @@ object SparkSubmit { private val CLUSTER = 2 private val ALL_DEPLOY_MODES = CLIENT | CLUSTER + // A special jar name that indicates the class being run is inside of Spark itself, and therefore + // no user jar is needed. + private val SPARK_INTERNAL = "spark-internal" + // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" @@ -257,7 +261,9 @@ object SparkSubmit { // In yarn-cluster mode, use yarn.Client as a wrapper around the user class if (clusterManager == YARN && deployMode == CLUSTER) { childMainClass = "org.apache.spark.deploy.yarn.Client" - childArgs += ("--jar", args.primaryResource) + if (args.primaryResource != SPARK_INTERNAL) { + childArgs += ("--jar", args.primaryResource) + } childArgs += ("--class", args.mainClass) if (args.childArgs != null) { args.childArgs.foreach { arg => childArgs += ("--arg", arg) } @@ -332,7 +338,7 @@ object SparkSubmit { * Return whether the given primary resource represents a user jar. */ private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) + !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) } /** @@ -349,6 +355,10 @@ object SparkSubmit { primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL } + private[spark] def isInternal(primaryResource: String): Boolean = { + primaryResource == SPARK_INTERNAL + } + /** * Merge a sequence of comma-separated file lists, some of which may be null to indicate * no files, into a single comma-separated string. diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 3ab67a43a3b55..01d0ae541a66b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -204,8 +204,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { - // Delineates parsing of Spark options from parsing of user options. var inSparkOpts = true + + // Delineates parsing of Spark options from parsing of user options. parse(opts) def parse(opts: Seq[String]): Unit = opts match { @@ -318,7 +319,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { SparkSubmit.printErrorAndExit(errMessage) case v => primaryResource = - if (!SparkSubmit.isShell(v)) { + if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) { Utils.resolveURI(v).toString } else { v diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 38830103d1e8d..33de24d1ae6d7 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -53,7 +53,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare @@ -61,7 +61,7 @@ if [[ ! "$@" =~ --package-only ]]; then -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ release:perform cd .. @@ -111,10 +111,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" +make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" make_binary_release "hadoop2" \ - "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" # Copy data echo "Copying release tarballs" diff --git a/dev/run-tests b/dev/run-tests index 51e4def0f835a..98ec969dc1b37 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -65,7 +65,7 @@ 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" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive" sbt/sbt clean package \ + echo -e "q\n" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive -Phive-thriftserver" 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 | \ diff --git a/dev/scalastyle b/dev/scalastyle index a02d06912f238..d9f2b91a3a091 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 38728534a46e0..156e0aebdebe6 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -136,7 +136,7 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD // Define the schema using a case class. -// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, // you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) @@ -548,7 +548,6 @@ results = hiveContext.hql("FROM src SELECT key, value").collect() - # Writing Language-Integrated Relational Queries **Language-Integrated queries are currently only supported in Scala.** @@ -573,4 +572,200 @@ prefixed with a tick (`'`). Implicit conversions turn these symbols into expres evaluated by the SQL execution engine. A full list of the functions supported can be found in the [ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). - \ No newline at end of file + + +## Running the Thrift JDBC server + +The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] +(https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test +the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive +you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` +for maven). + +To start the JDBC server, run the following in the Spark directory: + + ./sbin/start-thriftserver.sh + +The default port the server listens on is 10000. To listen on customized host and port, please set +the `HIVE_SERVER2_THRIFT_PORT` and `HIVE_SERVER2_THRIFT_BIND_HOST` environment variables. You may +run `./sbin/start-thriftserver.sh --help` for a complete list of all available options. Now you can +use beeline to test the Thrift JDBC server: + + ./bin/beeline + +Connect to the JDBC server in beeline with: + + beeline> !connect jdbc:hive2://localhost:10000 + +Beeline will ask you for a username and password. In non-secure mode, simply enter the username on +your machine and a blank password. For secure mode, please follow the instructions given in the +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients) + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. + +You may also use the beeline script comes with Hive. + +### Migration Guide for Shark Users + +#### Reducer number + +In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark +SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value +is 200. Users may customize this property via `SET`: + +``` +SET spark.sql.shuffle.partitions=10; +SELECT page, count(*) c FROM logs_last_month_cached +GROUP BY page ORDER BY c DESC LIMIT 10; +``` + +You may also put this property in `hive-site.xml` to override the default value. + +For now, the `mapred.reduce.tasks` property is still recognized, and is converted to +`spark.sql.shuffle.partitions` automatically. + +#### Caching + +The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no +longer automcatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to +let user control table caching explicitly: + +``` +CACHE TABLE logs_last_month; +UNCACHE TABLE logs_last_month; +``` + +**NOTE** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", +but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be +cached, you may simply count the table immediately after executing `CACHE TABLE`: + +``` +CACHE TABLE logs_last_month; +SELECT COUNT(1) FROM logs_last_month; +``` + +Several caching related features are not supported yet: + +* User defined partition level cache eviction policy +* RDD reloading +* In-memory cache write through policy + +### Compatibility with Apache Hive + +#### Deploying in Exising Hive Warehouses + +Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +#### Supported Hive Features + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arthimatic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathemtatical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDe's) +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +#### Unsupported Hive Functionality + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + +**Esoteric Hive Features** + +* Tables with partitions using different input formats: In Spark SQL, all table partitions need to + have the same input format. +* Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions + (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. +* `UNIONTYPE` +* Unique join +* Single query multi insert +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +not necessary due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block level bitmap indexes and virtual columns (used to build indexes) +* Automatically convert a join to map join: For joining a large table with multiple small tables, + Hive automatically converts the join into a map join. We are adding this auto conversion in the + next release. +* Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you + need to control the degree of parallelism post-shuffle using "SET + spark.sql.shuffle.partitions=[num_tasks];". We are going to add auto-setting of parallelism in the + next release. +* Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available +options. diff --git a/examples/pom.xml b/examples/pom.xml index bd1c387c2eb91..c4ed0f5a6a02b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-examples_2.10 - examples + examples jar Spark Project Examples diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 61a6aff543aed..874b8a7959bb6 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-flume_2.10 - streaming-flume + streaming-flume jar Spark Project External Flume diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4762c50685a93..25a5c0a4d7d77 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-kafka_2.10 - streaming-kafka + streaming-kafka jar Spark Project External Kafka diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 32c530e600ce0..f31ed655f6779 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-mqtt_2.10 - streaming-mqtt + streaming-mqtt jar Spark Project External MQTT diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 637adb0f00da0..56bb24c2a072e 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-twitter_2.10 - streaming-twitter + streaming-twitter jar Spark Project External Twitter diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index e4d758a04a4cd..54b0242c54e78 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-zeromq_2.10 - streaming-zeromq + streaming-zeromq jar Spark Project External ZeroMQ diff --git a/graphx/pom.xml b/graphx/pom.xml index 7e3bcf29dcfbc..6dd52fc618b1e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-graphx_2.10 - graphx + graphx jar Spark Project GraphX diff --git a/mllib/pom.xml b/mllib/pom.xml index 92b07e2357db1..f27cf520dc9fa 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-mllib_2.10 - mllib + mllib jar Spark Project ML Library diff --git a/pom.xml b/pom.xml index d2e6b3c0ed5a4..93ef3b91b5bce 100644 --- a/pom.xml +++ b/pom.xml @@ -252,9 +252,9 @@ 3.3.2 - commons-codec - commons-codec - 1.5 + commons-codec + commons-codec + 1.5 com.google.code.findbugs @@ -1139,5 +1139,15 @@ + + hive-thriftserver + + false + + + sql/hive-thriftserver + + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 62576f84dd031..1629bc2cba8ba 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,11 +30,11 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, mllib, repl, spark, sql, streaming, - streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = - Seq("bagel", "catalyst", "core", "graphx", "hive", "mllib", "repl", "spark", "sql", - "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", - "streaming-zeromq").map(ProjectRef(buildLocation, _)) + val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, + streaming, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = + Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", + "spark", "sql", "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", + "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") @@ -100,7 +100,7 @@ object SparkBuild extends PomBuild { Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { case Some(v) => v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.split("=")).foreach(x => System.setProperty(x(0), x(1))) - case _ => + case _ => } override val userPropertiesMap = System.getProperties.toMap @@ -158,7 +158,7 @@ object SparkBuild extends PomBuild { /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(y => Seq(spark, sql, hive, catalyst, repl).exists(x => x == y)). + allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl).contains(x)). foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh new file mode 100755 index 0000000000000..8398e6f19b511 --- /dev/null +++ b/sbin/start-thriftserver.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# +# Shell script for starting the Spark SQL Thrift server + +# Enter posix mode for bash +set -o posix + +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/start-thriftserver [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + +CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" +exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6decde3fcd62d..531bfddbf237b 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -32,7 +32,7 @@ Spark Project Catalyst http://spark.apache.org/ - catalyst + catalyst diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 1d5f033f0d274..a357c6ffb8977 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -43,8 +43,7 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(0, AttributeReference("key", StringType, nullable = false)()), - BoundReference(1, AttributeReference("value", StringType, nullable = false)())) + BoundReference(1, AttributeReference("", StringType, nullable = false)())) } /** diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c309c43804d97..3a038a2db6173 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -32,7 +32,7 @@ Spark Project SQL http://spark.apache.org/ - sql + sql diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 2b787e14f3f15..41920c00b5a2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -30,12 +30,13 @@ import scala.collection.JavaConverters._ * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). */ trait SQLConf { + import SQLConf._ /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? /** Number of partitions to use for shuffle operators. */ - private[spark] def numShufflePartitions: Int = get("spark.sql.shuffle.partitions", "200").toInt + private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to @@ -43,11 +44,10 @@ trait SQLConf { * effectively disables auto conversion. * Hive setting: hive.auto.convert.join.noconditionaltask.size. */ - private[spark] def autoConvertJoinSize: Int = - get("spark.sql.auto.convert.join.size", "10000").toInt + private[spark] def autoConvertJoinSize: Int = get(AUTO_CONVERT_JOIN_SIZE, "10000").toInt /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get("spark.sql.join.broadcastTables", "") + private[spark] def joinBroadcastTables: String = get(JOIN_BROADCAST_TABLES, "") /** ********************** SQLConf functionality methods ************ */ @@ -61,7 +61,7 @@ trait SQLConf { def set(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for ${key}") + require(value != null, s"value cannot be null for $key") settings.put(key, value) } @@ -90,3 +90,13 @@ trait SQLConf { } } + +object SQLConf { + val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" + val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" + val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" + + object Deprecated { + val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 98d2f89c8ae71..9293239131d52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.execution +import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.{Row, SQLConf, SQLContext} trait Command { /** @@ -44,28 +45,53 @@ trait Command { case class SetCommand( key: Option[String], value: Option[String], output: Seq[Attribute])( @transient context: SQLContext) - extends LeafNode with Command { + extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[(String, String)] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => - context.set(k, v) - Array(k -> v) + if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") + context.set(SQLConf.SHUFFLE_PARTITIONS, v) + Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") + } else { + context.set(k, v) + Array(s"$k=$v") + } // Query the value bound to key k. case (Some(k), _) => - Array(k -> context.getOption(k).getOrElse("")) + // TODO (lian) This is just a workaround to make the Simba ODBC driver work. + // Should remove this once we get the ODBC driver updated. + if (k == "-v") { + val hiveJars = Seq( + "hive-exec-0.12.0.jar", + "hive-service-0.12.0.jar", + "hive-common-0.12.0.jar", + "hive-hwi-0.12.0.jar", + "hive-0.12.0.jar").mkString(":") + + Array( + "system:java.class.path=" + hiveJars, + "system:sun.java.command=shark.SharkServer2") + } + else { + Array(s"$k=${context.getOption(k).getOrElse("")}") + } // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => - context.getAll + context.getAll.map { case (k, v) => + s"$k=$v" + } case _ => throw new IllegalArgumentException() } def execute(): RDD[Row] = { - val rows = sideEffectResult.map { case (k, v) => new GenericRow(Array[Any](k, v)) } + val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } context.sparkContext.parallelize(rows, 1) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 08293f7f0ca30..1a58d73d9e7f4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -54,10 +54,10 @@ class SQLConfSuite extends QueryTest { assert(get(testKey, testVal + "_") == testVal) assert(TestSQLContext.get(testKey, testVal + "_") == testVal) - sql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - sql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + sql("set some.property=20") + assert(get("some.property", "0") == "20") + sql("set some.property = 40") + assert(get("some.property", "0") == "40") val key = "spark.sql.key" val vs = "val0,val_1,val2.3,my_table" @@ -70,4 +70,9 @@ class SQLConfSuite extends QueryTest { clear() } + test("deprecated property") { + clear() + sql(s"set ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS}=10") + assert(get(SQLConf.SHUFFLE_PARTITIONS) == "10") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6736189c96d4b..de9e8aa4f62ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -424,25 +424,25 @@ class SQLQuerySuite extends QueryTest { sql(s"SET $testKey=$testVal") checkAnswer( sql("SET"), - Seq(Seq(testKey, testVal)) + Seq(Seq(s"$testKey=$testVal")) ) sql(s"SET ${testKey + testKey}=${testVal + testVal}") checkAnswer( sql("set"), Seq( - Seq(testKey, testVal), - Seq(testKey + testKey, testVal + testVal)) + Seq(s"$testKey=$testVal"), + Seq(s"${testKey + testKey}=${testVal + testVal}")) ) // "set key" checkAnswer( sql(s"SET $testKey"), - Seq(Seq(testKey, testVal)) + Seq(Seq(s"$testKey=$testVal")) ) checkAnswer( sql(s"SET $nonexistentKey"), - Seq(Seq(nonexistentKey, "")) + Seq(Seq(s"$nonexistentKey=")) ) clear() } diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml new file mode 100644 index 0000000000000..7fac90fdc596d --- /dev/null +++ b/sql/hive-thriftserver/pom.xml @@ -0,0 +1,82 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-hive-thriftserver_2.10 + jar + Spark Project Hive + http://spark.apache.org/ + + hive-thriftserver + + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + org.spark-project.hive + hive-cli + ${hive.version} + + + org.spark-project.hive + hive-jdbc + ${hive.version} + + + org.spark-project.hive + hive-beeline + ${hive.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala new file mode 100644 index 0000000000000..ddbc2a79fb512 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService +import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a + * `HiveThriftServer2` thrift server. + */ +private[hive] object HiveThriftServer2 extends Logging { + var LOG = LogFactory.getLog(classOf[HiveServer2]) + + def main(args: Array[String]) { + val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") + + if (!optionsProcessor.process(args)) { + logger.warn("Error starting HiveThriftServer2 with given arguments") + System.exit(-1) + } + + val ss = new SessionState(new HiveConf(classOf[SessionState])) + + // Set all properties specified via command line. + val hiveConf: HiveConf = ss.getConf + hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => + logger.debug(s"HiveConf var: $k=$v") + } + + SessionState.start(ss) + + logger.info("Starting SparkContext") + SparkSQLEnv.init() + SessionState.start(ss) + + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.sparkContext.stop() + } + } + ) + + try { + val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) + server.init(hiveConf) + server.start() + logger.info("HiveThriftServer2 started") + } catch { + case e: Exception => + logger.error("Error starting HiveThriftServer2", e) + System.exit(-1) + } + } +} + +private[hive] class HiveThriftServer2(hiveContext: HiveContext) + extends HiveServer2 + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + val sparkSqlCliService = new SparkSQLCLIService(hiveContext) + setSuperField(this, "cliService", sparkSqlCliService) + addService(sparkSqlCliService) + + val thriftCliService = new ThriftBinaryCLIService(sparkSqlCliService) + setSuperField(this, "thriftCLIService", thriftCliService) + addService(thriftCliService) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala new file mode 100644 index 0000000000000..599294dfbb7d7 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ReflectionUtils.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +private[hive] object ReflectionUtils { + def setSuperField(obj : Object, fieldName: String, fieldValue: Object) { + setAncestorField(obj, 1, fieldName, fieldValue) + } + + def setAncestorField(obj: AnyRef, level: Int, fieldName: String, fieldValue: AnyRef) { + val ancestor = Iterator.iterate[Class[_]](obj.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.set(obj, fieldValue) + } + + def getSuperField[T](obj: AnyRef, fieldName: String): T = { + getAncestorField[T](obj, 1, fieldName) + } + + def getAncestorField[T](clazz: Object, level: Int, fieldName: String): T = { + val ancestor = Iterator.iterate[Class[_]](clazz.getClass)(_.getSuperclass).drop(level).next() + val field = ancestor.getDeclaredField(fieldName) + field.setAccessible(true) + field.get(clazz).asInstanceOf[T] + } + + def invokeStatic(clazz: Class[_], methodName: String, args: (Class[_], AnyRef)*): AnyRef = { + invoke(clazz, null, methodName, args: _*) + } + + def invoke( + clazz: Class[_], + obj: AnyRef, + methodName: String, + args: (Class[_], AnyRef)*): AnyRef = { + + val (types, values) = args.unzip + val method = clazz.getDeclaredMethod(methodName, types: _*) + method.setAccessible(true) + method.invoke(obj, values.toSeq: _*) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala new file mode 100755 index 0000000000000..27268ecb923e9 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io._ +import java.util.{ArrayList => JArrayList} + +import jline.{ConsoleReader, History} +import org.apache.commons.lang.StringUtils +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException +import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.exec.Utilities +import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.thrift.transport.TSocket + +import org.apache.spark.sql.Logging + +private[hive] object SparkSQLCLIDriver { + private var prompt = "spark-sql" + private var continuedPrompt = "".padTo(prompt.length, ' ') + private var transport:TSocket = _ + + installSignalHandler() + + /** + * Install an interrupt callback to cancel all Spark jobs. In Hive's CliDriver#processLine(), + * a signal handler will invoke this registered callback if a Ctrl+C signal is detected while + * a command is being processed by the current thread. + */ + def installSignalHandler() { + HiveInterruptUtils.add(new HiveInterruptCallback { + override def interrupt() { + // Handle remote execution mode + if (SparkSQLEnv.sparkContext != null) { + SparkSQLEnv.sparkContext.cancelAllJobs() + } else { + if (transport != null) { + // Force closing of TCP connection upon session termination + transport.getSocket.close() + } + } + } + }) + } + + def main(args: Array[String]) { + val oproc = new OptionsProcessor() + if (!oproc.process_stage1(args)) { + System.exit(1) + } + + // NOTE: It is critical to do this here so that log4j is reinitialized + // before any of the other core hive classes are loaded + var logInitFailed = false + var logInitDetailMessage: String = null + try { + logInitDetailMessage = LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => + logInitFailed = true + logInitDetailMessage = e.getMessage + } + + val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) + + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + if (!oproc.process_stage2(sessionState)) { + System.exit(2) + } + + if (!sessionState.getIsSilent) { + if (logInitFailed) System.err.println(logInitDetailMessage) + else SessionState.getConsole.printInfo(logInitDetailMessage) + } + + // Set all properties specified via command line. + val conf: HiveConf = sessionState.getConf + sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => + conf.set(item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + sessionState.getOverriddenConfigurations.put( + item.getKey.asInstanceOf[String], item.getValue.asInstanceOf[String]) + } + + SessionState.start(sessionState) + + // Clean up after we exit + Runtime.getRuntime.addShutdownHook( + new Thread() { + override def run() { + SparkSQLEnv.stop() + } + } + ) + + // "-h" option has been passed, so connect to Hive thrift server. + if (sessionState.getHost != null) { + sessionState.connect() + if (sessionState.isRemoteMode) { + prompt = s"[${sessionState.getHost}:${sessionState.getPort}]" + prompt + continuedPrompt = "".padTo(prompt.length, ' ') + } + } + + if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { + // Hadoop-20 and above - we need to augment classpath using hiveconf + // components. + // See also: code in ExecDriver.java + var loader = conf.getClassLoader + val auxJars = HiveConf.getVar(conf, HiveConf.ConfVars.HIVEAUXJARS) + if (StringUtils.isNotBlank(auxJars)) { + loader = Utilities.addToClassPath(loader, StringUtils.split(auxJars, ",")) + } + conf.setClassLoader(loader) + Thread.currentThread().setContextClassLoader(loader) + } + + val cli = new SparkSQLCLIDriver + cli.setHiveVariables(oproc.getHiveVariables) + + // TODO work around for set the log output to console, because the HiveContext + // will set the output into an invalid buffer. + sessionState.in = System.in + try { + sessionState.out = new PrintStream(System.out, true, "UTF-8") + sessionState.info = new PrintStream(System.err, true, "UTF-8") + sessionState.err = new PrintStream(System.err, true, "UTF-8") + } catch { + case e: UnsupportedEncodingException => System.exit(3) + } + + // Execute -i init files (always in silent mode) + cli.processInitFiles(sessionState) + + if (sessionState.execString != null) { + System.exit(cli.processLine(sessionState.execString)) + } + + try { + if (sessionState.fileName != null) { + System.exit(cli.processFile(sessionState.fileName)) + } + } catch { + case e: FileNotFoundException => + System.err.println(s"Could not open input file for reading. (${e.getMessage})") + System.exit(3) + } + + val reader = new ConsoleReader() + reader.setBellEnabled(false) + // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) + CliDriver.getCommandCompletor.foreach((e) => reader.addCompletor(e)) + + val historyDirectory = System.getProperty("user.home") + + try { + if (new File(historyDirectory).exists()) { + val historyFile = historyDirectory + File.separator + ".hivehistory" + reader.setHistory(new History(new File(historyFile))) + } else { + System.err.println("WARNING: Directory for Hive history file: " + historyDirectory + + " does not exist. History will not be available during this session.") + } + } catch { + case e: Exception => + System.err.println("WARNING: Encountered an error while trying to initialize Hive's " + + "history file. History will not be available during this session.") + System.err.println(e.getMessage) + } + + val clientTransportTSocketField = classOf[CliSessionState].getDeclaredField("transport") + clientTransportTSocketField.setAccessible(true) + + transport = clientTransportTSocketField.get(sessionState).asInstanceOf[TSocket] + + var ret = 0 + var prefix = "" + val currentDB = ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", + classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) + + def promptWithCurrentDB = s"$prompt$currentDB" + def continuedPromptWithDBSpaces = continuedPrompt + ReflectionUtils.invokeStatic( + classOf[CliDriver], "spacesForString", classOf[String] -> currentDB) + + var currentPrompt = promptWithCurrentDB + var line = reader.readLine(currentPrompt + "> ") + + while (line != null) { + if (prefix.nonEmpty) { + prefix += '\n' + } + + if (line.trim().endsWith(";") && !line.trim().endsWith("\\;")) { + line = prefix + line + ret = cli.processLine(line, true) + prefix = "" + currentPrompt = promptWithCurrentDB + } else { + prefix = prefix + line + currentPrompt = continuedPromptWithDBSpaces + } + + line = reader.readLine(currentPrompt + "> ") + } + + sessionState.close() + + System.exit(ret) + } +} + +private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { + private val sessionState = SessionState.get().asInstanceOf[CliSessionState] + + private val LOG = LogFactory.getLog("CliDriver") + + private val console = new SessionState.LogHelper(LOG) + + private val conf: Configuration = + if (sessionState != null) sessionState.getConf else new Configuration() + + // Force initializing SparkSQLEnv. This is put here but not object SparkSQLCliDriver + // because the Hive unit tests do not go through the main() code path. + if (!sessionState.isRemoteMode) { + SparkSQLEnv.init() + } + + override def processCmd(cmd: String): Int = { + val cmd_trimmed: String = cmd.trim() + val tokens: Array[String] = cmd_trimmed.split("\\s+") + val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() + if (cmd_trimmed.toLowerCase.equals("quit") || + cmd_trimmed.toLowerCase.equals("exit") || + tokens(0).equalsIgnoreCase("source") || + cmd_trimmed.startsWith("!") || + tokens(0).toLowerCase.equals("list") || + sessionState.isRemoteMode) { + val start = System.currentTimeMillis() + super.processCmd(cmd) + val end = System.currentTimeMillis() + val timeTaken: Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds") + 0 + } else { + var ret = 0 + val hconf = conf.asInstanceOf[HiveConf] + val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) + + if (proc != null) { + if (proc.isInstanceOf[Driver]) { + val driver = new SparkSQLDriver + + driver.init() + val out = sessionState.out + val start:Long = System.currentTimeMillis() + if (sessionState.getIsVerbose) { + out.println(cmd) + } + + ret = driver.run(cmd).getResponseCode + if (ret != 0) { + driver.close() + return ret + } + + val res = new JArrayList[String]() + + if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER)) { + // Print the column names. + Option(driver.getSchema.getFieldSchemas).map { fields => + out.println(fields.map(_.getName).mkString("\t")) + } + } + + try { + while (!out.checkError() && driver.getResults(res)) { + res.foreach(out.println) + res.clear() + } + } catch { + case e:IOException => + console.printError( + s"""Failed with exception ${e.getClass.getName}: ${e.getMessage} + |${org.apache.hadoop.util.StringUtils.stringifyException(e)} + """.stripMargin) + ret = 1 + } + + val cret = driver.close() + if (ret == 0) { + ret = cret + } + + val end = System.currentTimeMillis() + if (end > start) { + val timeTaken:Double = (end - start) / 1000.0 + console.printInfo(s"Time taken: $timeTaken seconds", null) + } + + // Destroy the driver to release all the locks. + driver.destroy() + } else { + if (sessionState.getIsVerbose) { + sessionState.out.println(tokens(0) + " " + cmd_1) + } + ret = proc.run(cmd_1).getResponseCode + } + } + ret + } + } +} + diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala new file mode 100644 index 0000000000000..42cbf363b274f --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.io.IOException +import java.util.{List => JList} +import javax.security.auth.login.LoginException + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hive.service.Service.STATE +import org.apache.hive.service.auth.HiveAuthFactory +import org.apache.hive.service.cli.CLIService +import org.apache.hive.service.{AbstractService, Service, ServiceException} + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +private[hive] class SparkSQLCLIService(hiveContext: HiveContext) + extends CLIService + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) + setSuperField(this, "sessionManager", sparkSqlSessionManager) + addService(sparkSqlSessionManager) + + try { + HiveAuthFactory.loginFromKeytab(hiveConf) + val serverUserName = ShimLoader.getHadoopShims + .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) + setSuperField(this, "serverUserName", serverUserName) + } catch { + case e @ (_: IOException | _: LoginException) => + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + } + + initCompositeService(hiveConf) + } +} + +private[thriftserver] trait ReflectedCompositeService { this: AbstractService => + def initCompositeService(hiveConf: HiveConf) { + // Emulating `CompositeService.init(hiveConf)` + val serviceList = getAncestorField[JList[Service]](this, 2, "serviceList") + serviceList.foreach(_.init(hiveConf)) + + // Emulating `AbstractService.init(hiveConf)` + invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) + setAncestorField(this, 3, "hiveConf", hiveConf) + invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) + getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala new file mode 100644 index 0000000000000..5202aa9903e03 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ + +import java.util.{ArrayList => JArrayList} + +import org.apache.commons.lang.exception.ExceptionUtils +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} + +private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) + extends Driver with Logging { + + private var tableSchema: Schema = _ + private var hiveResponse: Seq[String] = _ + + override def init(): Unit = { + } + + private def getResultSetSchema(query: context.QueryExecution): Schema = { + val analyzed = query.analyzed + logger.debug(s"Result Schema: ${analyzed.output}") + if (analyzed.output.size == 0) { + new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) + } else { + val fieldSchemas = analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + + new Schema(fieldSchemas, null) + } + } + + override def run(command: String): CommandProcessorResponse = { + val execution = context.executePlan(context.hql(command).logicalPlan) + + // TODO unify the error code + try { + hiveResponse = execution.stringResult() + tableSchema = getResultSetSchema(execution) + new CommandProcessorResponse(0) + } catch { + case cause: Throwable => + logger.error(s"Failed in [$command]", cause) + new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) + } + } + + override def close(): Int = { + hiveResponse = null + tableSchema = null + 0 + } + + override def getSchema: Schema = tableSchema + + override def getResults(res: JArrayList[String]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.addAll(hiveResponse) + hiveResponse = null + true + } + } + + override def destroy() { + super.destroy() + hiveResponse = null + tableSchema = null + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala new file mode 100644 index 0000000000000..451c3bd7b9352 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hadoop.hive.ql.session.SessionState + +import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} +import org.apache.spark.sql.Logging +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.{SparkConf, SparkContext} + +/** A singleton object for the master program. The slaves should not access this. */ +private[hive] object SparkSQLEnv extends Logging { + logger.debug("Initializing SparkSQLEnv") + + var hiveContext: HiveContext = _ + var sparkContext: SparkContext = _ + + def init() { + if (hiveContext == null) { + sparkContext = new SparkContext(new SparkConf() + .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) + + sparkContext.addSparkListener(new StatsReportListener()) + + hiveContext = new HiveContext(sparkContext) { + @transient override lazy val sessionState = SessionState.get() + @transient override lazy val hiveconf = sessionState.getConf + } + } + } + + /** Cleans up and shuts down the Spark SQL environments. */ + def stop() { + logger.debug("Shutting down Spark SQL Environment") + // Stop the SparkContext + if (SparkSQLEnv.sparkContext != null) { + sparkContext.stop() + sparkContext = null + hiveContext = null + } + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala new file mode 100644 index 0000000000000..6b3275b4eaf04 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.concurrent.Executors + +import org.apache.commons.logging.Log +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hive.service.cli.session.SessionManager + +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager + +private[hive] class SparkSQLSessionManager(hiveContext: HiveContext) + extends SessionManager + with ReflectedCompositeService { + + override def init(hiveConf: HiveConf) { + setSuperField(this, "hiveConf", hiveConf) + + val backgroundPoolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS) + setSuperField(this, "backgroundOperationPool", Executors.newFixedThreadPool(backgroundPoolSize)) + getAncestorField[Log](this, 3, "LOG").info( + s"HiveServer2: Async execution pool size $backgroundPoolSize") + + val sparkSqlOperationManager = new SparkSQLOperationManager(hiveContext) + setSuperField(this, "operationManager", sparkSqlOperationManager) + addService(sparkSqlOperationManager) + + initCompositeService(hiveConf) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala new file mode 100644 index 0000000000000..a4e1f3e762e89 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver.server + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.math.{random, round} + +import java.sql.Timestamp +import java.util.{Map => JMap} + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} + +/** + * Executes queries using Spark SQL, and maintains a list of handles to active queries. + */ +class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { + val handleToOperation = ReflectionUtils + .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") + + override def newExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String], + async: Boolean): ExecuteStatementOperation = synchronized { + + val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logger.debug("CLOSING") + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + if (!iter.hasNext) { + new RowSet() + } else { + val maxRows = maxRowsL.toInt // Do you really want a row batch larger than Int Max? No. + var curRow = 0 + var rowSet = new ArrayBuffer[Row](maxRows) + + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = new Row() + var curCol = 0 + + while (curCol < sparkRow.length) { + dataTypes(curCol) match { + case StringType => + row.addString(sparkRow(curCol).asInstanceOf[String]) + case IntegerType => + row.addColumnValue(ColumnValue.intValue(sparkRow.getInt(curCol))) + case BooleanType => + row.addColumnValue(ColumnValue.booleanValue(sparkRow.getBoolean(curCol))) + case DoubleType => + row.addColumnValue(ColumnValue.doubleValue(sparkRow.getDouble(curCol))) + case FloatType => + row.addColumnValue(ColumnValue.floatValue(sparkRow.getFloat(curCol))) + case DecimalType => + val hiveDecimal = sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal + row.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) + case LongType => + row.addColumnValue(ColumnValue.longValue(sparkRow.getLong(curCol))) + case ByteType => + row.addColumnValue(ColumnValue.byteValue(sparkRow.getByte(curCol))) + case ShortType => + row.addColumnValue(ColumnValue.intValue(sparkRow.getShort(curCol))) + case TimestampType => + row.addColumnValue( + ColumnValue.timestampValue(sparkRow.get(curCol).asInstanceOf[Timestamp])) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((sparkRow.get(curCol), dataTypes(curCol))) + row.addColumnValue(ColumnValue.stringValue(hiveString)) + } + curCol += 1 + } + rowSet += row + curRow += 1 + } + new RowSet(rowSet, 0) + } + } + + def getResultSetSchema: TableSchema = { + logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + def run(): Unit = { + logger.info(s"Running query '$statement'") + setState(OperationState.RUNNING) + try { + result = hiveContext.hql(statement) + logger.debug(result.queryExecution.toString()) + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + iter = result.queryExecution.toRdd.toLocalIterator + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + setHasResultSet(true) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logger.error("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + setState(OperationState.FINISHED) + } + } + + handleToOperation.put(operation.getHandle, operation) + operation + } +} diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt new file mode 100644 index 0000000000000..850f8014b6f05 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/data/files/small_kv.txt @@ -0,0 +1,5 @@ +238val_238 +86val_86 +311val_311 +27val_27 +165val_165 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala new file mode 100644 index 0000000000000..69f19f826a802 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, InputStreamReader, PrintWriter} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { + val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") + val METASTORE_PATH = TestUtils.getMetastorePath("cli") + + override def beforeAll() { + val pb = new ProcessBuilder( + "../../bin/spark-sql", + "--master", + "local", + "--hiveconf", + s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", + "--hiveconf", + "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) + + process = pb.start() + outputWriter = new PrintWriter(process.getOutputStream, true) + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "spark-sql>") + } + + override def afterAll() { + process.destroy() + process.waitFor() + } + + test("simple commands") { + val dataFilePath = getDataFile("data/files/small_kv.txt") + executeQuery("create table hive_test1(key int, val string);") + executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") + executeQuery("cache table hive_test1", "Time taken") + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala new file mode 100644 index 0000000000000..fe3403b3292ec --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import scala.collection.JavaConversions._ +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent._ + +import java.io.{BufferedReader, InputStreamReader} +import java.net.ServerSocket +import java.sql.{Connection, DriverManager, Statement} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.util.getTempFilePath + +/** + * Test for the HiveThriftServer2 using JDBC. + */ +class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { + + val WAREHOUSE_PATH = getTempFilePath("warehouse") + val METASTORE_PATH = getTempFilePath("metastore") + + val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" + val TABLE = "test" + val HOST = "localhost" + val PORT = { + // Let the system to choose a random available port to avoid collision with other parallel + // builds. + val socket = new ServerSocket(0) + val port = socket.getLocalPort + socket.close() + port + } + + // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. + val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean + + Class.forName(DRIVER_NAME) + + override def beforeAll() { launchServer() } + + override def afterAll() { stopServer() } + + private def launchServer(args: Seq[String] = Seq.empty) { + // Forking a new process to start the Hive Thrift server. The reason to do this is it is + // hard to clean up Hive resources entirely, so we just start a new process and kill + // that process for cleanup. + val defaultArgs = Seq( + "../../sbin/start-thriftserver.sh", + "--master local", + "--hiveconf", + "hive.root.logger=INFO,console", + "--hiveconf", + s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", + "--hiveconf", + s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH") + val pb = new ProcessBuilder(defaultArgs ++ args) + val environment = pb.environment() + environment.put("HIVE_SERVER2_THRIFT_PORT", PORT.toString) + environment.put("HIVE_SERVER2_THRIFT_BIND_HOST", HOST) + process = pb.start() + inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) + errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) + waitForOutput(inputReader, "ThriftBinaryCLIService listening on") + + // Spawn a thread to read the output from the forked process. + // Note that this is necessary since in some configurations, log4j could be blocked + // if its output to stderr are not read, and eventually blocking the entire test suite. + future { + while (true) { + val stdout = readFrom(inputReader) + val stderr = readFrom(errorReader) + if (VERBOSE && stdout.length > 0) { + println(stdout) + } + if (VERBOSE && stderr.length > 0) { + println(stderr) + } + Thread.sleep(50) + } + } + } + + private def stopServer() { + process.destroy() + process.waitFor() + } + + test("test query execution against a Hive Thrift server") { + Thread.sleep(5 * 1000) + val dataFilePath = getDataFile("data/files/small_kv.txt") + val stmt = createStatement() + stmt.execute("DROP TABLE IF EXISTS test") + stmt.execute("DROP TABLE IF EXISTS test_cached") + stmt.execute("CREATE TABLE test(key int, val string)") + stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") + stmt.execute("CREATE TABLE test_cached as select * from test limit 4") + stmt.execute("CACHE TABLE test_cached") + + var rs = stmt.executeQuery("select count(*) from test") + rs.next() + assert(rs.getInt(1) === 5) + + rs = stmt.executeQuery("select count(*) from test_cached") + rs.next() + assert(rs.getInt(1) === 4) + + stmt.close() + } + + def getConnection: Connection = { + val connectURI = s"jdbc:hive2://localhost:$PORT/" + DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") + } + + def createStatement(): Statement = getConnection.createStatement() +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala new file mode 100644 index 0000000000000..bb2242618fbef --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.io.{BufferedReader, PrintWriter} +import java.text.SimpleDateFormat +import java.util.Date + +import org.apache.hadoop.hive.common.LogUtils +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException + +object TestUtils { + val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") + + def getWarehousePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + + timestamp.format(new Date) + } + + def getMetastorePath(prefix: String): String = { + System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + + timestamp.format(new Date) + } + + // Dummy function for initialize the log4j properties. + def init() { } + + // initialize log4j + try { + LogUtils.initHiveLog4j() + } catch { + case e: LogInitializationException => // Ignore the error. + } +} + +trait TestUtils { + var process : Process = null + var outputWriter : PrintWriter = null + var inputReader : BufferedReader = null + var errorReader : BufferedReader = null + + def executeQuery( + cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { + println("Executing: " + cmd + ", expecting output: " + outputMessage) + outputWriter.write(cmd + "\n") + outputWriter.flush() + waitForQuery(timeout, outputMessage) + } + + protected def waitForQuery(timeout: Long, message: String): String = { + if (waitForOutput(errorReader, message, timeout)) { + Thread.sleep(500) + readOutput() + } else { + assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) + null + } + } + + // Wait for the specified str to appear in the output. + protected def waitForOutput( + reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { + val startTime = System.currentTimeMillis + var out = "" + while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { + out += readFrom(reader) + } + out.contains(str) + } + + // Read stdout output and filter out garbage collection messages. + protected def readOutput(): String = { + val output = readFrom(inputReader) + // Remove GC Messages + val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) + .mkString("\n") + filteredOutput + } + + protected def readFrom(reader: BufferedReader): String = { + var out = "" + var c = 0 + while (reader.ready) { + c = reader.read() + out += c.asInstanceOf[Char] + } + out + } + + protected def getDataFile(name: String) = { + Thread.currentThread().getContextClassLoader.getResource(name) + } +} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 1699ffe06ce15..93d00f7c37c9b 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -32,7 +32,7 @@ Spark Project Hive http://spark.apache.org/ - hive + hive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 201c85f3d501e..84d43eaeea51d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -255,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DecimalType, TimestampType, BinaryType) - protected def toHiveString(a: (Any, DataType)): String = a match { + protected[sql] def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a8623b64c656f..a022a1e2dc70e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -419,10 +419,10 @@ class HiveQuerySuite extends HiveComparisonTest { hql(s"set $testKey=$testVal") assert(get(testKey, testVal + "_") == testVal) - hql("set mapred.reduce.tasks=20") - assert(get("mapred.reduce.tasks", "0") == "20") - hql("set mapred.reduce.tasks = 40") - assert(get("mapred.reduce.tasks", "0") == "40") + hql("set some.property=20") + assert(get("some.property", "0") == "20") + hql("set some.property = 40") + assert(get("some.property", "0") == "40") hql(s"set $testKey=$testVal") assert(get(testKey, "0") == testVal) @@ -436,63 +436,61 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - def collectResults(rdd: SchemaRDD): Set[(String, String)] = - rdd.collect().map { case Row(key: String, value: String) => key -> value }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. assert(hql("SET").collect().size == 0) - assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey=$testVal")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(hql("SET")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } hql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(hql("SET")) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + hql(s"SET").collect().map(_.getString(0)) } // "set key" - assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey")) + assertResult(Array(s"$testKey=$testVal")) { + hql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Set(nonexistentKey -> "")) { - collectResults(hql(s"SET $nonexistentKey")) + assertResult(Array(s"$nonexistentKey=")) { + hql(s"SET $nonexistentKey").collect().map(_.getString(0)) } // Assert that sql() should have the same effects as hql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey=$testVal")) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(sql("SET")) + assertResult(Array(s"$testKey=$testVal")) { + sql("SET").collect().map(_.getString(0)) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(sql("SET")) + assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { + sql("SET").collect().map(_.getString(0)) } - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey")) + assertResult(Array(s"$testKey=$testVal")) { + sql(s"SET $testKey").collect().map(_.getString(0)) } - assertResult(Set(nonexistentKey -> "")) { - collectResults(sql(s"SET $nonexistentKey")) + assertResult(Array(s"$nonexistentKey=")) { + sql(s"SET $nonexistentKey").collect().map(_.getString(0)) } clear() diff --git a/streaming/pom.xml b/streaming/pom.xml index f60697ce745b7..b99f306b8f2cc 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming_2.10 - streaming + streaming jar Spark Project Streaming diff --git a/tools/pom.xml b/tools/pom.xml index c0ee8faa7a615..97abb6b2b63e0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -27,7 +27,7 @@ org.apache.spark spark-tools_2.10 - tools + tools jar Spark Project Tools diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 5b13a1f002d6e..51744ece0412d 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-alpha + yarn-alpha org.apache.spark diff --git a/yarn/pom.xml b/yarn/pom.xml index efb473aa1b261..3faaf053634d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -29,7 +29,7 @@ pom Spark Project YARN Parent POM - yarn + yarn diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index ceaf9f9d71001..b6c8456d06684 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,7 +24,7 @@ ../pom.xml - yarn-stable + yarn-stable org.apache.spark From 39ab87b924ad65b6b9b7aa6831f3e9ddc2b76dd7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 28 Jul 2014 13:37:44 -0700 Subject: [PATCH 0102/1492] Use commons-lang3 in SignalLogger rather than commons-lang Spark only transitively depends on the latter, based on the Hadoop version. Author: Aaron Davidson Closes #1621 from aarondav/lang3 and squashes the following commits: 93c93bf [Aaron Davidson] Use commons-lang3 in SignalLogger rather than commons-lang --- core/src/main/scala/org/apache/spark/util/SignalLogger.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala index d769b54fa2fae..f77488ef3d449 100644 --- a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import org.apache.commons.lang.SystemUtils +import org.apache.commons.lang3.SystemUtils import org.slf4j.Logger import sun.misc.{Signal, SignalHandler} From 16ef4d110f15dfe66852802fdadfe2ed7574ddc2 Mon Sep 17 00:00:00 2001 From: Yadong Qi Date: Mon, 28 Jul 2014 21:39:02 -0700 Subject: [PATCH 0103/1492] Excess judgment Author: Yadong Qi Closes #1629 from watermen/bug-fix2 and squashes the following commits: 59b7237 [Yadong Qi] Update HiveQl.scala --- sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e6ab68b563f8d..d18ccf8167487 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 @@ -610,7 +610,7 @@ private[hive] object HiveQl { // TOK_DESTINATION means to overwrite the table. val resultDestination = (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) - val overwrite = if (intoClause.isEmpty) true else false + val overwrite = intoClause.isEmpty nodeToDest( resultDestination, withLimit, From ccd5ab5f82812abc2eb518448832cc20fb903345 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 29 Jul 2014 00:15:45 -0700 Subject: [PATCH 0104/1492] [SPARK-2580] [PySpark] keep silent in worker if JVM close the socket During rdd.take(n), JVM will close the socket if it had got enough data, the Python worker should keep silent in this case. In the same time, the worker should not print the trackback into stderr if it send the traceback to JVM successfully. Author: Davies Liu Closes #1625 from davies/error and squashes the following commits: 4fbcc6d [Davies Liu] disable log4j during testing when exception is expected. cc14202 [Davies Liu] keep silent in worker if JVM close the socket --- python/pyspark/tests.py | 6 ++++++ python/pyspark/worker.py | 21 +++++++++++++-------- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 63cc5e9ad96fa..6dee7dc66cee6 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -165,11 +165,17 @@ class TestAddFile(PySparkTestCase): def test_add_py_file(self): # To ensure that we're actually testing addPyFile's effects, check that # this job fails due to `userlibrary` not being on the Python path: + # disable logging in log4j temporarily + log4j = self.sc._jvm.org.apache.log4j + old_level = log4j.LogManager.getRootLogger().getLevel() + log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) def func(x): from userlibrary import UserClass return UserClass().hello() self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) + log4j.LogManager.getRootLogger().setLevel(old_level) + # Add the file, so the job should now succeed: path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") self.sc.addPyFile(path) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 24d41b12d1b1a..2770f63059853 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -75,14 +75,19 @@ def main(infile, outfile): init_time = time.time() iterator = deserializer.load_stream(infile) serializer.dump_stream(func(split_index, iterator), outfile) - except Exception as e: - # Write the error to stderr in addition to trying to pass it back to - # Java, in case it happened while serializing a record - print >> sys.stderr, "PySpark worker failed with exception:" - print >> sys.stderr, traceback.format_exc() - write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc(), outfile) - sys.exit(-1) + except Exception: + try: + write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) + write_with_length(traceback.format_exc(), outfile) + outfile.flush() + except IOError: + # JVM close the socket + pass + except Exception: + # Write the error to stderr if it happened while serializing + print >> sys.stderr, "PySpark worker failed with exception:" + print >> sys.stderr, traceback.format_exc() + exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) # Mark the beginning of the accumulators section of the output From 92ef02626e793ea853cced4cbfee316f0b748ed7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 29 Jul 2014 01:02:18 -0700 Subject: [PATCH 0105/1492] [SPARK-791] [PySpark] fix pickle itemgetter with cloudpickle fix the problem with pickle operator.itemgetter with multiple index. Author: Davies Liu Closes #1627 from davies/itemgetter and squashes the following commits: aabd7fa [Davies Liu] fix pickle itemgetter with cloudpickle --- python/pyspark/cloudpickle.py | 5 +++-- python/pyspark/tests.py | 6 ++++++ 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 4fda2a9b950b8..68062483dedaa 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -560,8 +560,9 @@ class ItemGetterType(ctypes.Structure): ] - itemgetter_obj = ctypes.cast(ctypes.c_void_p(id(obj)), ctypes.POINTER(ItemGetterType)).contents - return self.save_reduce(operator.itemgetter, (itemgetter_obj.item,)) + obj = ctypes.cast(ctypes.c_void_p(id(obj)), ctypes.POINTER(ItemGetterType)).contents + return self.save_reduce(operator.itemgetter, + obj.item if obj.nitems > 1 else (obj.item,)) if PyObject_HEAD: dispatch[operator.itemgetter] = save_itemgetter diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 6dee7dc66cee6..8486c8595b5a4 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -284,6 +284,12 @@ def combOp(x, y): self.assertEqual(set([2]), sets[3]) self.assertEqual(set([1, 3]), sets[5]) + def test_itemgetter(self): + rdd = self.sc.parallelize([range(10)]) + from operator import itemgetter + self.assertEqual([1], rdd.map(itemgetter(1)).collect()) + self.assertEqual([(2, 3)], rdd.map(itemgetter(2, 3)).collect()) + class TestIO(PySparkTestCase): From 96ba04bbf917bcb971dd0d8cd1e1766dbe9366e8 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 29 Jul 2014 01:12:44 -0700 Subject: [PATCH 0106/1492] [SPARK-2726] and [SPARK-2727] Remove SortOrder and do in-place sort. The pull request includes two changes: 1. Removes SortOrder introduced by SPARK-2125. The key ordering already includes the SortOrder information since an Ordering can be reverse. This is similar to Java's Comparator interface. Rarely does an API accept both a Comparator as well as a SortOrder. 2. Replaces the sortWith call in HashShuffleReader with an in-place quick sort. Author: Reynold Xin Closes #1631 from rxin/sortOrder and squashes the following commits: c9d37e1 [Reynold Xin] [SPARK-2726] and [SPARK-2727] Remove SortOrder and do in-place sort. --- .../scala/org/apache/spark/Dependency.scala | 4 +-- .../spark/rdd/OrderedRDDFunctions.scala | 8 +----- .../org/apache/spark/rdd/ShuffledRDD.scala | 12 +-------- .../shuffle/hash/HashShuffleReader.scala | 25 +++++++++++-------- 4 files changed, 18 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index f010c03223ef4..09a60571238ea 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -19,7 +19,6 @@ package org.apache.spark import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.SortOrder.SortOrder import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleHandle @@ -63,8 +62,7 @@ class ShuffleDependency[K, V, C]( val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, - val mapSideCombine: Boolean = false, - val sortOrder: Option[SortOrder] = None) + val mapSideCombine: Boolean = false) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index afd7075f686b9..d85f962783931 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -58,12 +58,6 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = { val part = new RangePartitioner(numPartitions, self, ascending) new ShuffledRDD[K, V, V, P](self, part) - .setKeyOrdering(ordering) - .setSortOrder(if (ascending) SortOrder.ASCENDING else SortOrder.DESCENDING) + .setKeyOrdering(if (ascending) ordering else ordering.reverse) } } - -private[spark] object SortOrder extends Enumeration { - type SortOrder = Value - val ASCENDING, DESCENDING = Value -} diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index da4a8c3dc22b1..bf02f68d0d3d3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -21,7 +21,6 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.rdd.SortOrder.SortOrder import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -52,8 +51,6 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( private var mapSideCombine: Boolean = false - private var sortOrder: Option[SortOrder] = None - /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C, P] = { this.serializer = Option(serializer) @@ -78,15 +75,8 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( this } - /** Set sort order for RDD's sorting. */ - def setSortOrder(sortOrder: SortOrder): ShuffledRDD[K, V, C, P] = { - this.sortOrder = Option(sortOrder) - this - } - override def getDependencies: Seq[Dependency[_]] = { - List(new ShuffleDependency(prev, part, serializer, - keyOrdering, aggregator, mapSideCombine, sortOrder)) + List(new ShuffleDependency(prev, part, serializer, keyOrdering, aggregator, mapSideCombine)) } override val partitioner = Some(part) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 76cdb8f4f8e8a..c8059496a1bdf 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -18,7 +18,6 @@ package org.apache.spark.shuffle.hash import org.apache.spark.{InterruptibleIterator, TaskContext} -import org.apache.spark.rdd.SortOrder import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} @@ -51,16 +50,22 @@ class HashShuffleReader[K, C]( iter } - val sortedIter = for (sortOrder <- dep.sortOrder; ordering <- dep.keyOrdering) yield { - val buf = aggregatedIter.toArray - if (sortOrder == SortOrder.ASCENDING) { - buf.sortWith((x, y) => ordering.lt(x._1, y._1)).iterator - } else { - buf.sortWith((x, y) => ordering.gt(x._1, y._1)).iterator - } + // Sort the output if there is a sort ordering defined. + dep.keyOrdering match { + case Some(keyOrd: Ordering[K]) => + // Define a Comparator for the whole record based on the key Ordering. + val cmp = new Ordering[Product2[K, C]] { + override def compare(o1: Product2[K, C], o2: Product2[K, C]): Int = { + keyOrd.compare(o1._1, o2._1) + } + } + val sortBuffer: Array[Product2[K, C]] = aggregatedIter.toArray + // TODO: do external sort. + scala.util.Sorting.quickSort(sortBuffer)(cmp) + sortBuffer.iterator + case None => + aggregatedIter } - - sortedIter.getOrElse(aggregatedIter) } /** Close this reader */ From 20424dad30e6c89ba42b07eb329070bdcb3494cb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 29 Jul 2014 01:16:41 -0700 Subject: [PATCH 0107/1492] [SPARK-2174][MLLIB] treeReduce and treeAggregate In `reduce` and `aggregate`, the driver node spends linear time on the number of partitions. It becomes a bottleneck when there are many partitions and the data from each partition is big. SPARK-1485 (#506) tracks the progress of implementing AllReduce on Spark. I did several implementations including butterfly, reduce + broadcast, and treeReduce + broadcast. treeReduce + BT broadcast seems to be right way to go for Spark. Using binary tree may introduce some overhead in communication, because the driver still need to coordinate on data shuffling. In my experiments, n -> sqrt(n) -> 1 gives the best performance in general, which is why I set "depth = 2" in MLlib algorithms. But it certainly needs more testing. I left `treeReduce` and `treeAggregate` public for easy testing. Some numbers from a test on 32-node m3.2xlarge cluster. code: ~~~ import breeze.linalg._ import org.apache.log4j._ Logger.getRootLogger.setLevel(Level.OFF) for (n <- Seq(1, 10, 100, 1000, 10000, 100000, 1000000)) { val vv = sc.parallelize(0 until 1024, 1024).map(i => DenseVector.zeros[Double](n)) var start = System.nanoTime(); vv.treeReduce(_ + _, 2); println((System.nanoTime() - start) / 1e9) start = System.nanoTime(); vv.reduce(_ + _); println((System.nanoTime() - start) / 1e9) } ~~~ out: | n | treeReduce(,2) | reduce | |---|---------------------|-----------| | 10 | 0.215538731 | 0.204206899 | | 100 | 0.278405907 | 0.205732582 | | 1000 | 0.208972182 | 0.214298272 | | 10000 | 0.194792071 | 0.349353687 | | 100000 | 0.347683285 | 6.086671892 | | 1000000 | 2.589350682 | 66.572906702 | CC: @pwendell This is clearly more scalable than the default implementation. My question is whether we should use this implementation in `reduce` and `aggregate` or put them as separate methods. The concern is that users may use `reduce` and `aggregate` as collect, where having multiple stages doesn't reduce the data size. However, in this case, `collect` is more appropriate. Author: Xiangrui Meng Closes #1110 from mengxr/tree and squashes the following commits: c6cd267 [Xiangrui Meng] make depth default to 2 b04b96a [Xiangrui Meng] address comments 9bcc5d3 [Xiangrui Meng] add depth for readability 7495681 [Xiangrui Meng] fix compile error 142a857 [Xiangrui Meng] merge master d58a087 [Xiangrui Meng] move treeReduce and treeAggregate to mllib 8a2a59c [Xiangrui Meng] Merge branch 'master' into tree be6a88a [Xiangrui Meng] use treeAggregate in mllib 0f94490 [Xiangrui Meng] add docs eb71c33 [Xiangrui Meng] add treeReduce fe42a5e [Xiangrui Meng] add treeAggregate --- .../mllib/linalg/distributed/RowMatrix.scala | 23 +++---- .../mllib/optimization/GradientDescent.scala | 3 +- .../spark/mllib/optimization/LBFGS.scala | 3 +- .../apache/spark/mllib/rdd/RDDFunctions.scala | 66 +++++++++++++++++++ .../spark/mllib/rdd/RDDFunctionsSuite.scala | 18 +++++ 5 files changed, 98 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 8c2b044ea73f2..58c1322757a43 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -28,6 +28,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging +import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} /** @@ -79,7 +80,7 @@ class RowMatrix( private[mllib] def multiplyGramianMatrixBy(v: BDV[Double]): BDV[Double] = { val n = numCols().toInt val vbr = rows.context.broadcast(v) - rows.aggregate(BDV.zeros[Double](n))( + rows.treeAggregate(BDV.zeros[Double](n))( seqOp = (U, r) => { val rBrz = r.toBreeze val a = rBrz.dot(vbr.value) @@ -91,9 +92,7 @@ class RowMatrix( s"Do not support vector operation from type ${rBrz.getClass.getName}.") } U - }, - combOp = (U1, U2) => U1 += U2 - ) + }, combOp = (U1, U2) => U1 += U2) } /** @@ -104,13 +103,11 @@ class RowMatrix( val nt: Int = n * (n + 1) / 2 // Compute the upper triangular part of the gram matrix. - val GU = rows.aggregate(new BDV[Double](new Array[Double](nt)))( + val GU = rows.treeAggregate(new BDV[Double](new Array[Double](nt)))( seqOp = (U, v) => { RowMatrix.dspr(1.0, v, U.data) U - }, - combOp = (U1, U2) => U1 += U2 - ) + }, combOp = (U1, U2) => U1 += U2) RowMatrix.triuToFull(n, GU.data) } @@ -290,9 +287,10 @@ class RowMatrix( s"We need at least $mem bytes of memory.") } - val (m, mean) = rows.aggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( + val (m, mean) = rows.treeAggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), - combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) + combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => + (s1._1 + s2._1, s1._2 += s2._2) ) updateNumRows(m) @@ -353,10 +351,9 @@ class RowMatrix( * Computes column-wise summary statistics. */ def computeColumnSummaryStatistics(): MultivariateStatisticalSummary = { - val summary = rows.aggregate[MultivariateOnlineSummarizer](new MultivariateOnlineSummarizer)( + val summary = rows.treeAggregate(new MultivariateOnlineSummarizer)( (aggregator, data) => aggregator.add(data), - (aggregator1, aggregator2) => aggregator1.merge(aggregator2) - ) + (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) updateNumRows(summary.count) summary } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 9fd760bf78083..356aa949afcf5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -25,6 +25,7 @@ import org.apache.spark.annotation.{Experimental, DeveloperApi} import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.rdd.RDDFunctions._ /** * Class used to solve an optimization problem using Gradient Descent. @@ -177,7 +178,7 @@ object GradientDescent extends Logging { // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) - .aggregate((BDV.zeros[Double](n), 0.0))( + .treeAggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => val l = gradient.compute(features, label, bcWeights.value, Vectors.fromBreeze(grad)) (grad, loss + l) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 179cd4a3f1625..26a2b62e76ed0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -26,6 +26,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.rdd.RDDFunctions._ /** * :: DeveloperApi :: @@ -199,7 +200,7 @@ object LBFGS extends Logging { val n = weights.length val bcWeights = data.context.broadcast(weights) - val (gradientSum, lossSum) = data.aggregate((BDV.zeros[Double](n), 0.0))( + val (gradientSum, lossSum) = data.treeAggregate((BDV.zeros[Double](n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => val l = localGradient.compute( features, label, Vectors.fromBreeze(bcWeights.value), Vectors.fromBreeze(grad)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index 365b5e75d7f75..b5e403bc8c14d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -20,7 +20,10 @@ package org.apache.spark.mllib.rdd import scala.language.implicitConversions import scala.reflect.ClassTag +import org.apache.spark.HashPartitioner +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * Machine learning specific RDD functions. @@ -44,6 +47,69 @@ class RDDFunctions[T: ClassTag](self: RDD[T]) { new SlidingRDD[T](self, windowSize) } } + + /** + * Reduces the elements of this RDD in a multi-level tree pattern. + * + * @param depth suggested depth of the tree (default: 2) + * @see [[org.apache.spark.rdd.RDD#reduce]] + */ + def treeReduce(f: (T, T) => T, depth: Int = 2): T = { + require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") + val cleanF = self.context.clean(f) + val reducePartition: Iterator[T] => Option[T] = iter => { + if (iter.hasNext) { + Some(iter.reduceLeft(cleanF)) + } else { + None + } + } + val partiallyReduced = self.mapPartitions(it => Iterator(reducePartition(it))) + val op: (Option[T], Option[T]) => Option[T] = (c, x) => { + if (c.isDefined && x.isDefined) { + Some(cleanF(c.get, x.get)) + } else if (c.isDefined) { + c + } else if (x.isDefined) { + x + } else { + None + } + } + RDDFunctions.fromRDD(partiallyReduced).treeAggregate(Option.empty[T])(op, op, depth) + .getOrElse(throw new UnsupportedOperationException("empty collection")) + } + + /** + * Aggregates the elements of this RDD in a multi-level tree pattern. + * + * @param depth suggested depth of the tree (default: 2) + * @see [[org.apache.spark.rdd.RDD#aggregate]] + */ + def treeAggregate[U: ClassTag](zeroValue: U)( + seqOp: (U, T) => U, + combOp: (U, U) => U, + depth: Int = 2): U = { + require(depth >= 1, s"Depth must be greater than or equal to 1 but got $depth.") + if (self.partitions.size == 0) { + return Utils.clone(zeroValue, self.context.env.closureSerializer.newInstance()) + } + val cleanSeqOp = self.context.clean(seqOp) + val cleanCombOp = self.context.clean(combOp) + val aggregatePartition = (it: Iterator[T]) => it.aggregate(zeroValue)(cleanSeqOp, cleanCombOp) + var partiallyAggregated = self.mapPartitions(it => Iterator(aggregatePartition(it))) + var numPartitions = partiallyAggregated.partitions.size + val scale = math.max(math.ceil(math.pow(numPartitions, 1.0 / depth)).toInt, 2) + // If creating an extra level doesn't help reduce the wall-clock time, we stop tree aggregation. + while (numPartitions > scale + numPartitions / scale) { + numPartitions /= scale + val curNumPartitions = numPartitions + partiallyAggregated = partiallyAggregated.mapPartitionsWithIndex { (i, iter) => + iter.map((i % curNumPartitions, _)) + }.reduceByKey(new HashPartitioner(curNumPartitions), cleanCombOp).values + } + partiallyAggregated.reduce(cleanCombOp) + } } private[mllib] diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala index 3f3b10dfff35e..27a19f793242b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala @@ -46,4 +46,22 @@ class RDDFunctionsSuite extends FunSuite with LocalSparkContext { val expected = data.flatMap(x => x).sliding(3).toList assert(sliding.collect().toList === expected) } + + test("treeAggregate") { + val rdd = sc.makeRDD(-1000 until 1000, 10) + def seqOp = (c: Long, x: Int) => c + x + def combOp = (c1: Long, c2: Long) => c1 + c2 + for (depth <- 1 until 10) { + val sum = rdd.treeAggregate(0L)(seqOp, combOp, depth) + assert(sum === -1000L) + } + } + + test("treeReduce") { + val rdd = sc.makeRDD(-1000 until 1000, 10) + for (depth <- 1 until 10) { + val sum = rdd.treeReduce(_ + _, depth) + assert(sum === -1000) + } + } } From fc4d05700026f4ee9cc5544cf493d900039c38f3 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Tue, 29 Jul 2014 01:35:26 -0700 Subject: [PATCH 0108/1492] Minor indentation and comment typo fixes. Author: Aaron Staple Closes #1630 from staple/minor and squashes the following commits: 6f295a2 [Aaron Staple] Fix typos in comment about ExprId. 8566467 [Aaron Staple] Fix off by one column indentation in SqlParser. --- .../apache/spark/sql/catalyst/SqlParser.scala | 22 +++++++++---------- .../expressions/namedExpressions.scala | 4 ++-- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a34b236c8ac6a..2c73a80f64ebf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -210,21 +210,21 @@ class SqlParser extends StandardTokenParsers with PackratParsers { } | "(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) } - protected lazy val joinedRelation: Parser[LogicalPlan] = - relationFactor ~ opt(joinType) ~ JOIN ~ relationFactor ~ opt(joinConditions) ^^ { + protected lazy val joinedRelation: Parser[LogicalPlan] = + relationFactor ~ opt(joinType) ~ JOIN ~ relationFactor ~ opt(joinConditions) ^^ { case r1 ~ jt ~ _ ~ r2 ~ cond => Join(r1, r2, joinType = jt.getOrElse(Inner), cond) - } + } - protected lazy val joinConditions: Parser[Expression] = - ON ~> expression + protected lazy val joinConditions: Parser[Expression] = + ON ~> expression - protected lazy val joinType: Parser[JoinType] = - INNER ^^^ Inner | - LEFT ~ SEMI ^^^ LeftSemi | - LEFT ~ opt(OUTER) ^^^ LeftOuter | - RIGHT ~ opt(OUTER) ^^^ RightOuter | - FULL ~ opt(OUTER) ^^^ FullOuter + protected lazy val joinType: Parser[JoinType] = + INNER ^^^ Inner | + LEFT ~ SEMI ^^^ LeftSemi | + LEFT ~ opt(OUTER) ^^^ LeftOuter | + RIGHT ~ opt(OUTER) ^^^ RightOuter | + FULL ~ opt(OUTER) ^^^ FullOuter protected lazy val filter: Parser[Expression] = WHERE ~ expression ^^ { case _ ~ e => e } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 934bad8c27294..ed69928ae9eb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -28,8 +28,8 @@ object NamedExpression { } /** - * A globally (within this JVM) id for a given named expression. - * Used to identify with attribute output by a relation is being + * A globally unique (within this JVM) id for a given named expression. + * Used to identify which attribute output by a relation is being * referenced in a subsequent computation. */ case class ExprId(id: Long) From 800ecff4b1127d9042d5a8a746348fb4d45aa34b Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Tue, 29 Jul 2014 11:11:29 -0700 Subject: [PATCH 0109/1492] [STREAMING] SPARK-1729. Make Flume pull data from source, rather than the current pu... ...sh model Currently Spark uses Flume's internal Avro Protocol to ingest data from Flume. If the executor running the receiver fails, it currently has to be restarted on the same node to be able to receive data. This commit adds a new Sink which can be deployed to a Flume agent. This sink can be polled by a new DStream that is also included in this commit. This model ensures that data can be pulled into Spark from Flume even if the receiver is restarted on a new node. This also allows the receiver to receive data on multiple threads for better performance. Author: Hari Shreedharan Author: Hari Shreedharan Author: Tathagata Das Author: harishreedharan Closes #807 from harishreedharan/master and squashes the following commits: e7f70a3 [Hari Shreedharan] Merge remote-tracking branch 'asf-git/master' 96cfb6f [Hari Shreedharan] Merge remote-tracking branch 'asf/master' e48d785 [Hari Shreedharan] Documenting flume-sink being ignored for Mima checks. 5f212ce [Hari Shreedharan] Ignore Spark Sink from mima. 981bf62 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' 7a1bc6e [Hari Shreedharan] Fix SparkBuild.scala a082eb3 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' 1f47364 [Hari Shreedharan] Minor fixes. 73d6f6d [Hari Shreedharan] Cleaned up tests a bit. Added some docs in multiple places. 65b76b4 [Hari Shreedharan] Fixing the unit test. e59cc20 [Hari Shreedharan] Use SparkFlumeEvent instead of the new type. Also, Flume Polling Receiver now uses the store(ArrayBuffer) method. f3c99d1 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' 3572180 [Hari Shreedharan] Adding a license header, making Jenkins happy. 799509f [Hari Shreedharan] Fix a compile issue. 3c5194c [Hari Shreedharan] Merge remote-tracking branch 'asf/master' d248d22 [harishreedharan] Merge pull request #1 from tdas/flume-polling 10b6214 [Tathagata Das] Changed public API, changed sink package, and added java unit test to make sure Java API is callable from Java. 1edc806 [Hari Shreedharan] SPARK-1729. Update logging in Spark Sink. 8c00289 [Hari Shreedharan] More debug messages 393bd94 [Hari Shreedharan] SPARK-1729. Use LinkedBlockingQueue instead of ArrayBuffer to keep track of connections. 120e2a1 [Hari Shreedharan] SPARK-1729. Some test changes and changes to utils classes. 9fd0da7 [Hari Shreedharan] SPARK-1729. Use foreach instead of map for all Options. 8136aa6 [Hari Shreedharan] Adding TransactionProcessor to map on returning batch of data 86aa274 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' 205034d [Hari Shreedharan] Merging master in 4b0c7fc [Hari Shreedharan] FLUME-1729. New Flume-Spark integration. bda01fc [Hari Shreedharan] FLUME-1729. Flume-Spark integration. 0d69604 [Hari Shreedharan] FLUME-1729. Better Flume-Spark integration. 3c23c18 [Hari Shreedharan] SPARK-1729. New Spark-Flume integration. 70bcc2a [Hari Shreedharan] SPARK-1729. New Flume-Spark integration. d6fa3aa [Hari Shreedharan] SPARK-1729. New Flume-Spark integration. e7da512 [Hari Shreedharan] SPARK-1729. Fixing import order 9741683 [Hari Shreedharan] SPARK-1729. Fixes based on review. c604a3c [Hari Shreedharan] SPARK-1729. Optimize imports. 0f10788 [Hari Shreedharan] SPARK-1729. Make Flume pull data from source, rather than the current push model 87775aa [Hari Shreedharan] SPARK-1729. Make Flume pull data from source, rather than the current push model 8df37e4 [Hari Shreedharan] SPARK-1729. Make Flume pull data from source, rather than the current push model 03d6c1c [Hari Shreedharan] SPARK-1729. Make Flume pull data from source, rather than the current push model 08176ad [Hari Shreedharan] SPARK-1729. Make Flume pull data from source, rather than the current push model d24d9d4 [Hari Shreedharan] SPARK-1729. Make Flume pull data from source, rather than the current push model 6d6776a [Hari Shreedharan] SPARK-1729. Make Flume pull data from source, rather than the current push model --- .../streaming/FlumePollingEventCount.scala | 67 +++++ external/flume-sink/pom.xml | 100 ++++++++ .../flume-sink/src/main/avro/sparkflume.avdl | 40 +++ .../spark/streaming/flume/sink/Logging.scala | 125 ++++++++++ .../flume/sink/SparkAvroCallbackHandler.scala | 131 ++++++++++ .../streaming/flume/sink/SparkSink.scala | 154 ++++++++++++ .../streaming/flume/sink/SparkSinkUtils.scala | 28 +++ .../flume/sink/TransactionProcessor.scala | 228 ++++++++++++++++++ external/flume/pom.xml | 5 + .../streaming/flume/EventTransformer.scala | 72 ++++++ .../streaming/flume/FlumeInputDStream.scala | 3 - .../flume/FlumePollingInputDStream.scala | 178 ++++++++++++++ .../spark/streaming/flume/FlumeUtils.scala | 144 ++++++++++- .../flume/JavaFlumePollingStreamSuite.java | 44 ++++ .../flume/FlumePollingStreamSuite.scala | 195 +++++++++++++++ pom.xml | 1 + project/SparkBuild.scala | 20 +- project/plugins.sbt | 2 + 18 files changed, 1524 insertions(+), 13 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala create mode 100644 external/flume-sink/pom.xml create mode 100644 external/flume-sink/src/main/avro/sparkflume.avdl create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala create mode 100644 external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala create mode 100644 external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java create mode 100644 external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala new file mode 100644 index 0000000000000..1cc8c8d5c23b6 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming + +import org.apache.spark.SparkConf +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ +import org.apache.spark.streaming.flume._ +import org.apache.spark.util.IntParam +import java.net.InetSocketAddress + +/** + * Produces a count of events received from Flume. + * + * This should be used in conjunction with the Spark Sink running in a Flume agent. See + * the Spark Streaming programming guide for more details. + * + * Usage: FlumePollingEventCount + * `host` is the host on which the Spark Sink is running. + * `port` is the port at which the Spark Sink is listening. + * + * To run this example: + * `$ bin/run-example org.apache.spark.examples.streaming.FlumePollingEventCount [host] [port] ` + */ +object FlumePollingEventCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println( + "Usage: FlumePollingEventCount ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + val Array(host, IntParam(port)) = args + + val batchInterval = Milliseconds(2000) + + // Create the context and set the batch size + val sparkConf = new SparkConf().setAppName("FlumePollingEventCount") + val ssc = new StreamingContext(sparkConf, batchInterval) + + // Create a flume stream that polls the Spark Sink running in a Flume agent + val stream = FlumeUtils.createPollingStream(ssc, host, port) + + // Print out the count of events received from this server in each batch + stream.count().map(cnt => "Received " + cnt + " flume events." ).print() + + ssc.start() + ssc.awaitTermination() + } +} diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml new file mode 100644 index 0000000000000..d11129ce8d89d --- /dev/null +++ b/external/flume-sink/pom.xml @@ -0,0 +1,100 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + spark-streaming-flume-sink_2.10 + + streaming-flume-sink + + + jar + Spark Project External Flume Sink + http://spark.apache.org/ + + + org.apache.flume + flume-ng-sdk + 1.4.0 + + + io.netty + netty + + + org.apache.thrift + libthrift + + + + + org.apache.flume + flume-ng-core + 1.4.0 + + + io.netty + netty + + + org.apache.thrift + libthrift + + + + + org.scala-lang + scala-library + 2.10.4 + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + org.apache.avro + avro-maven-plugin + 1.7.3 + + + ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro + + + + generate-sources + + idl-protocol + + + + + + + diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl new file mode 100644 index 0000000000000..8806e863ac7c6 --- /dev/null +++ b/external/flume-sink/src/main/avro/sparkflume.avdl @@ -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. + */ + +@namespace("org.apache.spark.streaming.flume.sink") + +protocol SparkFlumeProtocol { + + record SparkSinkEvent { + map headers; + bytes body; + } + + record EventBatch { + string errorMsg = ""; // If this is empty it is a valid message, else it represents an error + string sequenceNumber; + array events; + } + + EventBatch getEventBatch (int n); + + void ack (string sequenceNumber); + + void nack (string sequenceNumber); +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala new file mode 100644 index 0000000000000..17cbc6707b5ea --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import org.slf4j.{Logger, LoggerFactory} + +/** + * Copy of the org.apache.spark.Logging for being used in the Spark Sink. + * The org.apache.spark.Logging is not used so that all of Spark is not brought + * in as a dependency. + */ +private[sink] trait Logging { + // Make the log field transient so that objects with Logging can + // be serialized and used on another machine + @transient private var log_ : Logger = null + + // Method to get or create the logger for this object + protected def log: Logger = { + if (log_ == null) { + initializeIfNecessary() + var className = this.getClass.getName + // Ignore trailing $'s in the class names for Scala objects + if (className.endsWith("$")) { + className = className.substring(0, className.length - 1) + } + log_ = LoggerFactory.getLogger(className) + } + log_ + } + + // Log methods that take only a String + protected def logInfo(msg: => String) { + if (log.isInfoEnabled) log.info(msg) + } + + protected def logDebug(msg: => String) { + if (log.isDebugEnabled) log.debug(msg) + } + + protected def logTrace(msg: => String) { + if (log.isTraceEnabled) log.trace(msg) + } + + protected def logWarning(msg: => String) { + if (log.isWarnEnabled) log.warn(msg) + } + + protected def logError(msg: => String) { + if (log.isErrorEnabled) log.error(msg) + } + + // Log methods that take Throwables (Exceptions/Errors) too + protected def logInfo(msg: => String, throwable: Throwable) { + if (log.isInfoEnabled) log.info(msg, throwable) + } + + protected def logDebug(msg: => String, throwable: Throwable) { + if (log.isDebugEnabled) log.debug(msg, throwable) + } + + protected def logTrace(msg: => String, throwable: Throwable) { + if (log.isTraceEnabled) log.trace(msg, throwable) + } + + protected def logWarning(msg: => String, throwable: Throwable) { + if (log.isWarnEnabled) log.warn(msg, throwable) + } + + protected def logError(msg: => String, throwable: Throwable) { + if (log.isErrorEnabled) log.error(msg, throwable) + } + + protected def isTraceEnabled(): Boolean = { + log.isTraceEnabled + } + + private def initializeIfNecessary() { + if (!Logging.initialized) { + Logging.initLock.synchronized { + if (!Logging.initialized) { + initializeLogging() + } + } + } + } + + private def initializeLogging() { + Logging.initialized = true + + // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads + // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html + log + } +} + +private[sink] object Logging { + @volatile private var initialized = false + val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala new file mode 100644 index 0000000000000..7da8eb3e35912 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.util.concurrent.{ConcurrentHashMap, Executors} +import java.util.concurrent.atomic.AtomicLong + +import org.apache.flume.Channel +import org.apache.commons.lang.RandomStringUtils +import com.google.common.util.concurrent.ThreadFactoryBuilder + +/** + * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process + * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. + * @param threads Number of threads to use to process requests. + * @param channel The channel that the sink pulls events from + * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark + * is rolled back. + */ +// Flume forces transactions to be thread-local. So each transaction *must* be committed, or +// rolled back from the thread it was originally created in. So each getEvents call from Spark +// creates a TransactionProcessor which runs in a new thread, in which the transaction is created +// and events are pulled off the channel. Once the events are sent to spark, +// that thread is blocked and the TransactionProcessor is saved in a map, +// until an ACK or NACK comes back or the transaction times out (after the specified timeout). +// When the response comes or a timeout is hit, the TransactionProcessor is retrieved and then +// unblocked, at which point the transaction is committed or rolled back. + +private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, + val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { + val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat("Spark Sink Processor Thread - %d").build())) + private val processorMap = new ConcurrentHashMap[CharSequence, TransactionProcessor]() + // This sink will not persist sequence numbers and reuses them if it gets restarted. + // So it is possible to commit a transaction which may have been meant for the sink before the + // restart. + // Since the new txn may not have the same sequence number we must guard against accidentally + // committing a new transaction. To reduce the probability of that happening a random string is + // prepended to the sequence number. Does not change for life of sink + private val seqBase = RandomStringUtils.randomAlphanumeric(8) + private val seqCounter = new AtomicLong(0) + + /** + * Returns a bunch of events to Spark over Avro RPC. + * @param n Maximum number of events to return in a batch + * @return [[EventBatch]] instance that has a sequence number and an array of at most n events + */ + override def getEventBatch(n: Int): EventBatch = { + logDebug("Got getEventBatch call from Spark.") + val sequenceNumber = seqBase + seqCounter.incrementAndGet() + val processor = new TransactionProcessor(channel, sequenceNumber, + n, transactionTimeout, backOffInterval, this) + transactionExecutorOpt.foreach(executor => { + executor.submit(processor) + }) + // Wait until a batch is available - will be an error if error message is non-empty + val batch = processor.getEventBatch + if (!SparkSinkUtils.isErrorBatch(batch)) { + processorMap.put(sequenceNumber.toString, processor) + logDebug("Sending event batch with sequence number: " + sequenceNumber) + } + batch + } + + /** + * Called by Spark to indicate successful commit of a batch + * @param sequenceNumber The sequence number of the event batch that was successful + */ + override def ack(sequenceNumber: CharSequence): Void = { + logDebug("Received Ack for batch with sequence number: " + sequenceNumber) + completeTransaction(sequenceNumber, success = true) + null + } + + /** + * Called by Spark to indicate failed commit of a batch + * @param sequenceNumber The sequence number of the event batch that failed + * @return + */ + override def nack(sequenceNumber: CharSequence): Void = { + completeTransaction(sequenceNumber, success = false) + logInfo("Spark failed to commit transaction. Will reattempt events.") + null + } + + /** + * Helper method to commit or rollback a transaction. + * @param sequenceNumber The sequence number of the batch that was completed + * @param success Whether the batch was successful or not. + */ + private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { + Option(removeAndGetProcessor(sequenceNumber)).foreach(processor => { + processor.batchProcessed(success) + }) + } + + /** + * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. + * @param sequenceNumber + * @return The transaction processor for the corresponding batch. Note that this instance is no + * longer tracked and the caller is responsible for that txn processor. + */ + private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { + processorMap.remove(sequenceNumber.toString) // The toString is required! + } + + /** + * Shuts down the executor used to process transactions. + */ + def shutdown() { + logInfo("Shutting down Spark Avro Callback Handler") + transactionExecutorOpt.foreach(executor => { + executor.shutdownNow() + }) + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala new file mode 100644 index 0000000000000..7b735133e3d14 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.net.InetSocketAddress +import java.util.concurrent._ + +import org.apache.avro.ipc.NettyServer +import org.apache.avro.ipc.specific.SpecificResponder +import org.apache.flume.Context +import org.apache.flume.Sink.Status +import org.apache.flume.conf.{Configurable, ConfigurationException} +import org.apache.flume.sink.AbstractSink + +/** + * A sink that uses Avro RPC to run a server that can be polled by Spark's + * FlumePollingInputDStream. This sink has the following configuration parameters: + * + * hostname - The hostname to bind to. Default: 0.0.0.0 + * port - The port to bind to. (No default - mandatory) + * timeout - Time in seconds after which a transaction is rolled back, + * if an ACK is not received from Spark within that time + * threads - Number of threads to use to receive requests from Spark (Default: 10) + * + * This sink is unlike other Flume sinks in the sense that it does not push data, + * instead the process method in this sink simply blocks the SinkRunner the first time it is + * called. This sink starts up an Avro IPC server that uses the SparkFlumeProtocol. + * + * Each time a getEventBatch call comes, creates a transaction and reads events + * from the channel. When enough events are read, the events are sent to the Spark receiver and + * the thread itself is blocked and a reference to it saved off. + * + * When the ack for that batch is received, + * the thread which created the transaction is is retrieved and it commits the transaction with the + * channel from the same thread it was originally created in (since Flume transactions are + * thread local). If a nack is received instead, the sink rolls back the transaction. If no ack + * is received within the specified timeout, the transaction is rolled back too. If an ack comes + * after that, it is simply ignored and the events get re-sent. + * + */ + +private[flume] +class SparkSink extends AbstractSink with Logging with Configurable { + + // Size of the pool to use for holding transaction processors. + private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS + + // Timeout for each transaction. If spark does not respond in this much time, + // rollback the transaction + private var transactionTimeout = SparkSinkConfig.DEFAULT_TRANSACTION_TIMEOUT + + // Address info to bind on + private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME + private var port: Int = 0 + + private var backOffInterval: Int = 200 + + // Handle to the server + private var serverOpt: Option[NettyServer] = None + + // The handler that handles the callback from Avro + private var handler: Option[SparkAvroCallbackHandler] = None + + // Latch that blocks off the Flume framework from wasting 1 thread. + private val blockingLatch = new CountDownLatch(1) + + override def start() { + logInfo("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + + hostname + " with " + "pool size: " + poolSize + " and transaction timeout: " + + transactionTimeout + ".") + handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout, + backOffInterval)) + val responder = new SpecificResponder(classOf[SparkFlumeProtocol], handler.get) + // Using the constructor that takes specific thread-pools requires bringing in netty + // dependencies which are being excluded in the build. In practice, + // Netty dependencies are already available on the JVM as Flume would have pulled them in. + serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) + serverOpt.foreach(server => { + logInfo("Starting Avro server for sink: " + getName) + server.start() + }) + super.start() + } + + override def stop() { + logInfo("Stopping Spark Sink: " + getName) + handler.foreach(callbackHandler => { + callbackHandler.shutdown() + }) + serverOpt.foreach(server => { + logInfo("Stopping Avro Server for sink: " + getName) + server.close() + server.join() + }) + blockingLatch.countDown() + super.stop() + } + + override def configure(ctx: Context) { + import SparkSinkConfig._ + hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) + port = Option(ctx.getInteger(CONF_PORT)). + getOrElse(throw new ConfigurationException("The port to bind to must be specified")) + poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) + transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) + backOffInterval = ctx.getInteger(CONF_BACKOFF_INTERVAL, DEFAULT_BACKOFF_INTERVAL) + logInfo("Configured Spark Sink with hostname: " + hostname + ", port: " + port + ", " + + "poolSize: " + poolSize + ", transactionTimeout: " + transactionTimeout + ", " + + "backoffInterval: " + backOffInterval) + } + + override def process(): Status = { + // This method is called in a loop by the Flume framework - block it until the sink is + // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is + // being shut down. + logInfo("Blocking Sink Runner, sink will continue to run..") + blockingLatch.await() + Status.BACKOFF + } +} + +/** + * Configuration parameters and their defaults. + */ +private[flume] +object SparkSinkConfig { + val THREADS = "threads" + val DEFAULT_THREADS = 10 + + val CONF_TRANSACTION_TIMEOUT = "timeout" + val DEFAULT_TRANSACTION_TIMEOUT = 60 + + val CONF_HOSTNAME = "hostname" + val DEFAULT_HOSTNAME = "0.0.0.0" + + val CONF_PORT = "port" + + val CONF_BACKOFF_INTERVAL = "backoffInterval" + val DEFAULT_BACKOFF_INTERVAL = 200 +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala new file mode 100644 index 0000000000000..47c0e294d6b52 --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +private[flume] object SparkSinkUtils { + /** + * This method determines if this batch represents an error or not. + * @param batch - The batch to check + * @return - true if the batch represents an error + */ + def isErrorBatch(batch: EventBatch): Boolean = { + !batch.getErrorMsg.toString.equals("") // If there is an error message, it is an error batch. + } +} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala new file mode 100644 index 0000000000000..b9e3c786ebb3b --- /dev/null +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -0,0 +1,228 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.nio.ByteBuffer +import java.util +import java.util.concurrent.{Callable, CountDownLatch, TimeUnit} + +import scala.util.control.Breaks + +import org.apache.flume.{Transaction, Channel} + +// Flume forces transactions to be thread-local (horrible, I know!) +// So the sink basically spawns a new thread to pull the events out within a transaction. +// The thread fills in the event batch object that is set before the thread is scheduled. +// After filling it in, the thread waits on a condition - which is released only +// when the success message comes back for the specific sequence number for that event batch. +/** + * This class represents a transaction on the Flume channel. This class runs a separate thread + * which owns the transaction. The thread is blocked until the success call for that transaction + * comes back with an ACK or NACK. + * @param channel The channel from which to pull events + * @param seqNum The sequence number to use for the transaction. Must be unique + * @param maxBatchSize The maximum number of events to process per batch + * @param transactionTimeout Time in seconds after which a transaction must be rolled back + * without waiting for an ACK from Spark + * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts + */ +private class TransactionProcessor(val channel: Channel, val seqNum: String, + var maxBatchSize: Int, val transactionTimeout: Int, val backOffInterval: Int, + val parent: SparkAvroCallbackHandler) extends Callable[Void] with Logging { + + // If a real batch is not returned, we always have to return an error batch. + @volatile private var eventBatch: EventBatch = new EventBatch("Unknown Error", "", + util.Collections.emptyList()) + + // Synchronization primitives + val batchGeneratedLatch = new CountDownLatch(1) + val batchAckLatch = new CountDownLatch(1) + + // Sanity check to ensure we don't loop like crazy + val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 + + // OK to use volatile, since the change would only make this true (otherwise it will be + // changed to false - we never apply a negation operation to this) - which means the transaction + // succeeded. + @volatile private var batchSuccess = false + + // The transaction that this processor would handle + var txOpt: Option[Transaction] = None + + /** + * Get an event batch from the channel. This method will block until a batch of events is + * available from the channel. If no events are available after a large number of attempts of + * polling the channel, this method will return an [[EventBatch]] with a non-empty error message + * + * @return An [[EventBatch]] instance with sequence number set to seqNum, filled with a + * maximum of maxBatchSize events + */ + def getEventBatch: EventBatch = { + batchGeneratedLatch.await() + eventBatch + } + + /** + * This method is to be called by the sink when it receives an ACK or NACK from Spark. This + * method is a no-op if it is called after transactionTimeout has expired since + * getEventBatch returned a batch of events. + * @param success True if an ACK was received and the transaction should be committed, else false. + */ + def batchProcessed(success: Boolean) { + logDebug("Batch processed for sequence number: " + seqNum) + batchSuccess = success + batchAckLatch.countDown() + } + + /** + * Populates events into the event batch. If the batch cannot be populated, + * this method will not set the events into the event batch, but it sets an error message. + */ + private def populateEvents() { + try { + txOpt = Option(channel.getTransaction) + if(txOpt.isEmpty) { + eventBatch.setErrorMsg("Something went wrong. Channel was " + + "unable to create a transaction!") + } + txOpt.foreach(tx => { + tx.begin() + val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) + val loop = new Breaks + var gotEventsInThisTxn = false + var loopCounter: Int = 0 + loop.breakable { + while (events.size() < maxBatchSize + && loopCounter < totalAttemptsToRemoveFromChannel) { + loopCounter += 1 + Option(channel.take()) match { + case Some(event) => + events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), + ByteBuffer.wrap(event.getBody))) + gotEventsInThisTxn = true + case None => + if (!gotEventsInThisTxn) { + logDebug("Sleeping for " + backOffInterval + " millis as no events were read in" + + " the current transaction") + TimeUnit.MILLISECONDS.sleep(backOffInterval) + } else { + loop.break() + } + } + } + } + if (!gotEventsInThisTxn) { + val msg = "Tried several times, " + + "but did not get any events from the channel!" + logWarning(msg) + eventBatch.setErrorMsg(msg) + } else { + // At this point, the events are available, so fill them into the event batch + eventBatch = new EventBatch("",seqNum, events) + } + }) + } catch { + case e: Exception => + logWarning("Error while processing transaction.", e) + eventBatch.setErrorMsg(e.getMessage) + try { + txOpt.foreach(tx => { + rollbackAndClose(tx, close = true) + }) + } finally { + txOpt = None + } + } finally { + batchGeneratedLatch.countDown() + } + } + + /** + * Waits for upto transactionTimeout seconds for an ACK. If an ACK comes in + * this method commits the transaction with the channel. If the ACK does not come in within + * that time or a NACK comes in, this method rolls back the transaction. + */ + private def processAckOrNack() { + batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) + txOpt.foreach(tx => { + if (batchSuccess) { + try { + logDebug("Committing transaction") + tx.commit() + } catch { + case e: Exception => + logWarning("Error while attempting to commit transaction. Transaction will be rolled " + + "back", e) + rollbackAndClose(tx, close = false) // tx will be closed later anyway + } finally { + tx.close() + } + } else { + logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") + rollbackAndClose(tx, close = true) + // This might have been due to timeout or a NACK. Either way the following call does not + // cause issues. This is required to ensure the TransactionProcessor instance is not leaked + parent.removeAndGetProcessor(seqNum) + } + }) + } + + /** + * Helper method to rollback and optionally close a transaction + * @param tx The transaction to rollback + * @param close Whether the transaction should be closed or not after rolling back + */ + private def rollbackAndClose(tx: Transaction, close: Boolean) { + try { + logWarning("Spark was unable to successfully process the events. Transaction is being " + + "rolled back.") + tx.rollback() + } catch { + case e: Exception => + logError("Error rolling back transaction. Rollback may have failed!", e) + } finally { + if (close) { + tx.close() + } + } + } + + /** + * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] + * @param inMap The map to be converted + * @return The converted map + */ + private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, + CharSequence] = { + val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) + charSeqMap.putAll(inMap) + charSeqMap + } + + /** + * When the thread is started it sets as many events as the batch size or less (if enough + * events aren't available) into the eventBatch and object and lets any threads waiting on the + * [[getEventBatch]] method to proceed. Then this thread waits for acks or nacks to come in, + * or for a specified timeout and commits or rolls back the transaction. + * @return + */ + override def call(): Void = { + populateEvents() + processAckOrNack() + null + } +} diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 874b8a7959bb6..9f680b27c3308 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -77,6 +77,11 @@ junit-interface test + + org.apache.spark + spark-streaming-flume-sink_2.10 + ${project.version} + target/scala-${scala.binary.version}/classes diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala new file mode 100644 index 0000000000000..dc629df4f4ac2 --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume + +import java.io.{ObjectOutput, ObjectInput} + +import scala.collection.JavaConversions._ + +import org.apache.spark.util.Utils +import org.apache.spark.Logging + +/** + * A simple object that provides the implementation of readExternal and writeExternal for both + * the wrapper classes for Flume-style Events. + */ +private[streaming] object EventTransformer extends Logging { + def readExternal(in: ObjectInput): (java.util.HashMap[CharSequence, CharSequence], + Array[Byte]) = { + val bodyLength = in.readInt() + val bodyBuff = new Array[Byte](bodyLength) + in.readFully(bodyBuff) + + val numHeaders = in.readInt() + val headers = new java.util.HashMap[CharSequence, CharSequence] + + for (i <- 0 until numHeaders) { + val keyLength = in.readInt() + val keyBuff = new Array[Byte](keyLength) + in.readFully(keyBuff) + val key: String = Utils.deserialize(keyBuff) + + val valLength = in.readInt() + val valBuff = new Array[Byte](valLength) + in.readFully(valBuff) + val value: String = Utils.deserialize(valBuff) + + headers.put(key, value) + } + (headers, bodyBuff) + } + + def writeExternal(out: ObjectOutput, headers: java.util.Map[CharSequence, CharSequence], + body: Array[Byte]) { + out.writeInt(body.length) + out.write(body) + val numHeaders = headers.size() + out.writeInt(numHeaders) + for ((k,v) <- headers) { + val keyBuff = Utils.serialize(k.toString) + out.writeInt(keyBuff.length) + out.write(keyBuff) + val valBuff = Utils.serialize(v.toString) + out.writeInt(valBuff.length) + out.write(valBuff) + } + } +} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 56d2886b26878..4b2ea45fb81d0 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -39,11 +39,8 @@ import org.apache.spark.streaming.receiver.Receiver import org.jboss.netty.channel.ChannelPipelineFactory import org.jboss.netty.channel.Channels -import org.jboss.netty.channel.ChannelPipeline -import org.jboss.netty.channel.ChannelFactory import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.codec.compression._ -import org.jboss.netty.handler.execution.ExecutionHandler private[streaming] class FlumeInputDStream[T: ClassTag]( diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala new file mode 100644 index 0000000000000..148262bb6771e --- /dev/null +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume + + +import java.net.InetSocketAddress +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, Executors} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.flume.sink._ + +/** + * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running + * [[org.apache.spark.streaming.flume.sink.SparkSink]]s. + * @param _ssc Streaming context that will execute this input stream + * @param addresses List of addresses at which SparkSinks are listening + * @param maxBatchSize Maximum size of a batch + * @param parallelism Number of parallel connections to open + * @param storageLevel The storage level to use. + * @tparam T Class type of the object of this stream + */ +private[streaming] class FlumePollingInputDStream[T: ClassTag]( + @transient _ssc: StreamingContext, + val addresses: Seq[InetSocketAddress], + val maxBatchSize: Int, + val parallelism: Int, + storageLevel: StorageLevel + ) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { + + override def getReceiver(): Receiver[SparkFlumeEvent] = { + new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) + } +} + +private[streaming] class FlumePollingReceiver( + addresses: Seq[InetSocketAddress], + maxBatchSize: Int, + parallelism: Int, + storageLevel: StorageLevel + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { + + lazy val channelFactoryExecutor = + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). + setNameFormat("Flume Receiver Channel Thread - %d").build()) + + lazy val channelFactory = + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + + lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) + + private lazy val connections = new LinkedBlockingQueue[FlumeConnection]() + + override def onStart(): Unit = { + // Create the connections to each Flume agent. + addresses.foreach(host => { + val transceiver = new NettyTransceiver(host, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + connections.add(new FlumeConnection(transceiver, client)) + }) + for (i <- 0 until parallelism) { + logInfo("Starting Flume Polling Receiver worker threads starting..") + // Threads that pull data from Flume. + receiverExecutor.submit(new Runnable { + override def run(): Unit = { + while (true) { + val connection = connections.poll() + val client = connection.client + try { + val eventBatch = client.getEventBatch(maxBatchSize) + if (!SparkSinkUtils.isErrorBatch(eventBatch)) { + // No error, proceed with processing data + val seq = eventBatch.getSequenceNumber + val events: java.util.List[SparkSinkEvent] = eventBatch.getEvents + logDebug( + "Received batch of " + events.size() + " events with sequence number: " + seq) + try { + // Convert each Flume event to a serializable SparkFlumeEvent + val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) + var j = 0 + while (j < events.size()) { + buffer += toSparkFlumeEvent(events(j)) + j += 1 + } + store(buffer) + logDebug("Sending ack for sequence number: " + seq) + // Send an ack to Flume so that Flume discards the events from its channels. + client.ack(seq) + logDebug("Ack sent for sequence number: " + seq) + } catch { + case e: Exception => + try { + // Let Flume know that the events need to be pushed back into the channel. + logDebug("Sending nack for sequence number: " + seq) + client.nack(seq) // If the agent is down, even this could fail and throw + logDebug("Nack sent for sequence number: " + seq) + } catch { + case e: Exception => logError( + "Sending Nack also failed. A Flume agent is down.") + } + TimeUnit.SECONDS.sleep(2L) // for now just leave this as a fixed 2 seconds. + logWarning("Error while attempting to store events", e) + } + } else { + logWarning("Did not receive events from Flume agent due to error on the Flume " + + "agent: " + eventBatch.getErrorMsg) + } + } catch { + case e: Exception => + logWarning("Error while reading data from Flume", e) + } finally { + connections.add(connection) + } + } + } + }) + } + } + + override def onStop(): Unit = { + logInfo("Shutting down Flume Polling Receiver") + receiverExecutor.shutdownNow() + connections.foreach(connection => { + connection.transceiver.close() + }) + channelFactory.releaseExternalResources() + } + + /** + * Utility method to convert [[SparkSinkEvent]] to [[SparkFlumeEvent]] + * @param event - Event to convert to SparkFlumeEvent + * @return - The SparkFlumeEvent generated from SparkSinkEvent + */ + private def toSparkFlumeEvent(event: SparkSinkEvent): SparkFlumeEvent = { + val sparkFlumeEvent = new SparkFlumeEvent() + sparkFlumeEvent.event.setBody(event.getBody) + sparkFlumeEvent.event.setHeaders(event.getHeaders) + sparkFlumeEvent + } +} + +/** + * A wrapper around the transceiver and the Avro IPC API. + * @param transceiver The transceiver to use for communication with Flume + * @param client The client that the callbacks are received on. + */ +private class FlumeConnection(val transceiver: NettyTransceiver, + val client: SparkFlumeProtocol.Callback) + + + diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 716db9fa76031..4b732c1592ab2 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -17,12 +17,19 @@ package org.apache.spark.streaming.flume +import java.net.InetSocketAddress + +import org.apache.spark.annotation.Experimental import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaInputDStream, JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.ReceiverInputDStream + object FlumeUtils { + private val DEFAULT_POLLING_PARALLELISM = 5 + private val DEFAULT_POLLING_BATCH_SIZE = 1000 + /** * Create a input stream from a Flume source. * @param ssc StreamingContext object @@ -56,7 +63,7 @@ object FlumeUtils { ): ReceiverInputDStream[SparkFlumeEvent] = { val inputStream = new FlumeInputDStream[SparkFlumeEvent]( ssc, hostname, port, storageLevel, enableDecompression) - + inputStream } @@ -105,4 +112,135 @@ object FlumeUtils { ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port, storageLevel, enableDecompression) } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Address of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + ssc: StreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(ssc, Seq(new InetSocketAddress(hostname, port)), storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel + ): ReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(ssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param addresses List of InetSocketAddresses representing the hosts to connect to. + * @param maxBatchSize Maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + ssc: StreamingContext, + addresses: Seq[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): ReceiverInputDStream[SparkFlumeEvent] = { + new FlumePollingInputDStream[SparkFlumeEvent](ssc, addresses, maxBatchSize, + parallelism, storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, hostname, port, StorageLevel.MEMORY_AND_DISK_SER_2) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param hostname Hostname of the host on which the Spark Sink is running + * @param port Port of the host at which the Spark Sink is listening + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + hostname: String, + port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, Array(new InetSocketAddress(hostname, port)), storageLevel) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * This stream will use a batch size of 1000 events and run 5 threads to pull data. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running. + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc, addresses, storageLevel, + DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) + } + + /** + * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. + * This stream will poll the sink for data and will pull events as they are available. + * @param addresses List of InetSocketAddresses on which the Spark Sink is running + * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a + * single RPC call + * @param parallelism Number of concurrent requests this stream should send to the sink. Note + * that having a higher number of requests concurrently being pulled will + * result in this stream using more threads + * @param storageLevel Storage level to use for storing the received objects + */ + @Experimental + def createPollingStream( + jssc: JavaStreamingContext, + addresses: Array[InetSocketAddress], + storageLevel: StorageLevel, + maxBatchSize: Int, + parallelism: Int + ): JavaReceiverInputDStream[SparkFlumeEvent] = { + createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) + } } diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java new file mode 100644 index 0000000000000..79c5b91654b42 --- /dev/null +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.flume; + +import java.net.InetSocketAddress; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.LocalJavaStreamingContext; + +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; +import org.junit.Test; + +public class JavaFlumePollingStreamSuite extends LocalJavaStreamingContext { + @Test + public void testFlumeStream() { + // tests the API, does not actually test data receiving + InetSocketAddress[] addresses = new InetSocketAddress[] { + new InetSocketAddress("localhost", 12345) + }; + JavaReceiverInputDStream test1 = + FlumeUtils.createPollingStream(ssc, "localhost", 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( + ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2()); + JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( + ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2(), 100, 5); + } +} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala new file mode 100644 index 0000000000000..47071d0cc4714 --- /dev/null +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.spark.streaming.flume + +import java.net.InetSocketAddress +import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} + +import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.conf.Configurables +import org.apache.flume.event.EventBuilder + +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} +import org.apache.spark.streaming.flume.sink._ + +class FlumePollingStreamSuite extends TestSuiteBase { + + val testPort = 9999 + val batchCount = 5 + val eventsPerBatch = 100 + val totalEventsPerChannel = batchCount * eventsPerBatch + val channelCapacity = 5000 + + test("flume polling test") { + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), + StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + + // Start the channel and sink. + val context = new Context() + context.put("capacity", channelCapacity.toString) + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + ssc.start() + + writeAndVerify(Seq(channel), ssc, outputBuffer) + assertChannelIsEmpty(channel) + sink.stop() + channel.stop() + } + + test("flume polling test multiple hosts") { + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _)) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, + eventsPerBatch, 5) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + + // Start the channel and sink. + val context = new Context() + context.put("capacity", channelCapacity.toString) + context.put("transactionCapacity", "1000") + context.put("keep-alive", "0") + val channel = new MemoryChannel() + Configurables.configure(channel, context) + + val channel2 = new MemoryChannel() + Configurables.configure(channel2, context) + + val sink = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + Configurables.configure(sink, context) + sink.setChannel(channel) + sink.start() + + val sink2 = new SparkSink() + context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort + 1)) + Configurables.configure(sink2, context) + sink2.setChannel(channel2) + sink2.start() + ssc.start() + writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) + assertChannelIsEmpty(channel) + assertChannelIsEmpty(channel2) + sink.stop() + channel.stop() + } + + def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, + outputBuffer: ArrayBuffer[Seq[SparkFlumeEvent]]) { + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val executor = Executors.newCachedThreadPool() + val executorCompletion = new ExecutorCompletionService[Void](executor) + channels.map(channel => { + executorCompletion.submit(new TxnSubmitter(channel, clock)) + }) + for (i <- 0 until channels.size) { + executorCompletion.take() + } + val startTime = System.currentTimeMillis() + while (outputBuffer.size < batchCount * channels.size && + System.currentTimeMillis() - startTime < 15000) { + logInfo("output.size = " + outputBuffer.size) + Thread.sleep(100) + } + val timeTaken = System.currentTimeMillis() - startTime + assert(timeTaken < 15000, "Operation timed out after " + timeTaken + " ms") + logInfo("Stopping context") + ssc.stop() + + val flattenedBuffer = outputBuffer.flatten + assert(flattenedBuffer.size === totalEventsPerChannel * channels.size) + var counter = 0 + for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { + val eventToVerify = EventBuilder.withBody((channels(k).getName + " - " + + String.valueOf(i)).getBytes("utf-8"), + Map[String, String]("test-" + i.toString -> "header")) + var found = false + var j = 0 + while (j < flattenedBuffer.size && !found) { + val strToCompare = new String(flattenedBuffer(j).event.getBody.array(), "utf-8") + if (new String(eventToVerify.getBody, "utf-8") == strToCompare && + eventToVerify.getHeaders.get("test-" + i.toString) + .equals(flattenedBuffer(j).event.getHeaders.get("test-" + i.toString))) { + found = true + counter += 1 + } + j += 1 + } + } + assert(counter === totalEventsPerChannel * channels.size) + } + + def assertChannelIsEmpty(channel: MemoryChannel) = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining"); + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + assert(m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] === 5000) + } + + private class TxnSubmitter(channel: MemoryChannel, clock: ManualClock) extends Callable[Void] { + override def call(): Void = { + var t = 0 + for (i <- 0 until batchCount) { + val tx = channel.getTransaction + tx.begin() + for (j <- 0 until eventsPerBatch) { + channel.put(EventBuilder.withBody((channel.getName + " - " + String.valueOf(t)).getBytes( + "utf-8"), + Map[String, String]("test-" + t.toString -> "header"))) + t += 1 + } + tx.commit() + tx.close() + Thread.sleep(500) // Allow some time for the events to reach + clock.addToTime(batchDuration.milliseconds) + } + null + } + } +} diff --git a/pom.xml b/pom.xml index 93ef3b91b5bce..8b1435cfe5d19 100644 --- a/pom.xml +++ b/pom.xml @@ -100,6 +100,7 @@ external/twitter external/kafka external/flume + external/flume-sink external/zeromq external/mqtt examples diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1629bc2cba8ba..0a6326e72297a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,11 +30,12 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, sql, - streaming, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = + val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, + sql, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, + streamingTwitter, streamingZeromq) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "spark", "sql", "streaming", "streaming-flume", "streaming-kafka", "streaming-mqtt", - "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) + "spark", "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", + "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") @@ -156,10 +157,9 @@ object SparkBuild extends PomBuild { /* Enable tests settings for all projects except examples, assembly and tools */ (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) - /* Enable Mima for all projects except spark, hive, catalyst, sql and repl */ // TODO: Add Sql to mima checks - allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl).contains(x)). - foreach (x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) + allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, + streamingFlumeSink).contains(x)).foreach(x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) @@ -173,6 +173,8 @@ object SparkBuild extends PomBuild { /* Hive console settings */ enable(Hive.settings)(hive) + enable(Flume.settings)(streamingFlumeSink) + // TODO: move this to its upstream project. override def projectDefinitions(baseDirectory: File): Seq[Project] = { super.projectDefinitions(baseDirectory).map { x => @@ -183,6 +185,10 @@ object SparkBuild extends PomBuild { } +object Flume { + lazy val settings = sbtavro.SbtAvro.avroSettings +} + object SQL { lazy val settings = Seq( diff --git a/project/plugins.sbt b/project/plugins.sbt index d3ac4bf335e87..06d18e193076e 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -24,3 +24,5 @@ addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") + +addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") From 0c5c6a63d19bed2a813a09309c46971ecdd173f0 Mon Sep 17 00:00:00 2001 From: Daoyuan Date: Tue, 29 Jul 2014 12:22:48 -0700 Subject: [PATCH 0110/1492] [SQL]change some test lists 1. there's no `hook_context.q` but a `hook_context_cs.q` in query folder 2. there's no `compute_stats_table.q` in query folder 3. there's no `having1.q` in query folder 4. `udf_E` and `udf_PI` appear twice in white list Author: Daoyuan Closes #1634 from adrian-wang/testcases and squashes the following commits: d7482ce [Daoyuan] change some test lists --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index c69e93ba2b9ba..4fef071161719 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -52,7 +52,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { override def blackList = Seq( // These tests use hooks that are not on the classpath and thus break all subsequent execution. "hook_order", - "hook_context", + "hook_context_cs", "mapjoin_hook", "multi_sahooks", "overridden_confs", @@ -289,7 +289,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "compute_stats_empty_table", "compute_stats_long", "compute_stats_string", - "compute_stats_table", "convert_enum_to_string", "correlationoptimizer1", "correlationoptimizer10", @@ -395,7 +394,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_sort_9", "groupby_sort_test_1", "having", - "having1", "implicit_cast1", "innerjoin", "inoutdriver", @@ -697,8 +695,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf7", "udf8", "udf9", - "udf_E", - "udf_PI", "udf_abs", "udf_acos", "udf_add", From e3643485de8fdaf5c52b266fead1b13214f29d5e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 29 Jul 2014 12:23:34 -0700 Subject: [PATCH 0111/1492] [SPARK-2730][SQL] When retrieving a value from a Map, GetItem evaluates key twice JIRA: https://issues.apache.org/jira/browse/SPARK-2730 Author: Yin Huai Closes #1637 from yhuai/SPARK-2730 and squashes the following commits: 1a9f24e [Yin Huai] Remove unnecessary key evaluation. --- .../org/apache/spark/sql/catalyst/expressions/complexTypes.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 5d3bb25ad568c..0acb29012f314 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -61,7 +61,6 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { } } else { val baseValue = value.asInstanceOf[Map[Any, _]] - val key = ordinal.eval(input) baseValue.get(key).orNull } } From f0d880e288eba97c86dceb1b5edab4f3a935943b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 29 Jul 2014 12:31:39 -0700 Subject: [PATCH 0112/1492] [SPARK-2674] [SQL] [PySpark] support datetime type for SchemaRDD Datetime and time in Python will be converted into java.util.Calendar after serialization, it will be converted into java.sql.Timestamp during inferSchema(). In javaToPython(), Timestamp will be converted into Calendar, then be converted into datetime in Python after pickling. Author: Davies Liu Closes #1601 from davies/date and squashes the following commits: f0599b0 [Davies Liu] remove tests for sets and tuple in sql, fix list of list c9d607a [Davies Liu] convert datetype for runtime 709d40d [Davies Liu] remove brackets 96db384 [Davies Liu] support datetime type for SchemaRDD --- .../apache/spark/api/python/PythonRDD.scala | 4 +- python/pyspark/sql.py | 22 +++++---- .../org/apache/spark/sql/SQLContext.scala | 40 ++++++++++++++-- .../org/apache/spark/sql/SchemaRDD.scala | 46 +++++++------------ 4 files changed, 68 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index d87783efd2d01..0d8453fb184a3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -550,11 +550,11 @@ private[spark] object PythonRDD extends Logging { def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { pyRDD.rdd.mapPartitions { iter => val unpickle = new Unpickler - // TODO: Figure out why flatMap is necessay for pyspark iter.flatMap { row => unpickle.loads(row) match { + // in case of objects are pickled in batch mode case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) - // Incase the partition doesn't have a collection + // not in batch mode case obj: JMap[String @unchecked, _] => Seq(obj.toMap) } } diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index cb83e89176823..a6b3277db3266 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -47,12 +47,14 @@ def __init__(self, sparkContext, sqlContext=None): ... ValueError:... - >>> allTypes = sc.parallelize([{"int" : 1, "string" : "string", "double" : 1.0, "long": 1L, - ... "boolean" : True}]) + >>> from datetime import datetime + >>> allTypes = sc.parallelize([{"int": 1, "string": "string", "double": 1.0, "long": 1L, + ... "boolean": True, "time": datetime(2010, 1, 1, 1, 1, 1), "dict": {"a": 1}, + ... "list": [1, 2, 3]}]) >>> srdd = sqlCtx.inferSchema(allTypes).map(lambda x: (x.int, x.string, x.double, x.long, - ... x.boolean)) + ... x.boolean, x.time, x.dict["a"], x.list)) >>> srdd.collect()[0] - (1, u'string', 1.0, 1, True) + (1, u'string', 1.0, 1, True, datetime.datetime(2010, 1, 1, 1, 1, 1), 1, [1, 2, 3]) """ self._sc = sparkContext self._jsc = self._sc._jsc @@ -88,13 +90,13 @@ def inferSchema(self, rdd): >>> from array import array >>> srdd = sqlCtx.inferSchema(nestedRdd1) - >>> srdd.collect() == [{"f1" : array('i', [1, 2]), "f2" : {"row1" : 1.0}}, - ... {"f1" : array('i', [2, 3]), "f2" : {"row2" : 2.0}}] + >>> srdd.collect() == [{"f1" : [1, 2], "f2" : {"row1" : 1.0}}, + ... {"f1" : [2, 3], "f2" : {"row2" : 2.0}}] True >>> srdd = sqlCtx.inferSchema(nestedRdd2) - >>> srdd.collect() == [{"f1" : [[1, 2], [2, 3]], "f2" : set([1, 2]), "f3" : (1, 2)}, - ... {"f1" : [[2, 3], [3, 4]], "f2" : set([2, 3]), "f3" : (2, 3)}] + >>> srdd.collect() == [{"f1" : [[1, 2], [2, 3]], "f2" : [1, 2]}, + ... {"f1" : [[2, 3], [3, 4]], "f2" : [2, 3]}] True """ if (rdd.__class__ is SchemaRDD): @@ -509,8 +511,8 @@ def _test(): {"f1": array('i', [1, 2]), "f2": {"row1": 1.0}}, {"f1": array('i', [2, 3]), "f2": {"row2": 2.0}}]) globs['nestedRdd2'] = sc.parallelize([ - {"f1": [[1, 2], [2, 3]], "f2": set([1, 2]), "f3": (1, 2)}, - {"f1": [[2, 3], [3, 4]], "f2": set([2, 3]), "f3": (2, 3)}]) + {"f1": [[1, 2], [2, 3]], "f2": [1, 2]}, + {"f1": [[2, 3], [3, 4]], "f2": [2, 3]}]) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4abd89955bd27..c178dad662532 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -352,8 +352,10 @@ class SQLContext(@transient val sparkContext: SparkContext) case c: java.lang.Long => LongType case c: java.lang.Double => DoubleType case c: java.lang.Boolean => BooleanType + case c: java.math.BigDecimal => DecimalType + case c: java.sql.Timestamp => TimestampType + case c: java.util.Calendar => TimestampType case c: java.util.List[_] => ArrayType(typeFor(c.head)) - case c: java.util.Set[_] => ArrayType(typeFor(c.head)) case c: java.util.Map[_, _] => val (key, value) = c.head MapType(typeFor(key), typeFor(value)) @@ -362,11 +364,43 @@ class SQLContext(@transient val sparkContext: SparkContext) ArrayType(typeFor(elem)) case c => throw new Exception(s"Object of type $c cannot be used") } - val schema = rdd.first().map { case (fieldName, obj) => + val firstRow = rdd.first() + val schema = firstRow.map { case (fieldName, obj) => AttributeReference(fieldName, typeFor(obj), true)() }.toSeq - val rowRdd = rdd.mapPartitions { iter => + def needTransform(obj: Any): Boolean = obj match { + case c: java.util.List[_] => true + case c: java.util.Map[_, _] => true + case c if c.getClass.isArray => true + case c: java.util.Calendar => true + case c => false + } + + // convert JList, JArray into Seq, convert JMap into Map + // convert Calendar into Timestamp + def transform(obj: Any): Any = obj match { + case c: java.util.List[_] => c.map(transform).toSeq + case c: java.util.Map[_, _] => c.map { + case (key, value) => (key, transform(value)) + }.toMap + case c if c.getClass.isArray => + c.asInstanceOf[Array[_]].map(transform).toSeq + case c: java.util.Calendar => + new java.sql.Timestamp(c.getTime().getTime()) + case c => c + } + + val need = firstRow.exists {case (key, value) => needTransform(value)} + val transformed = if (need) { + rdd.mapPartitions { iter => + iter.map { + m => m.map {case (key, value) => (key, transform(value))} + } + } + } else rdd + + val rowRdd = transformed.mapPartitions { iter => iter.map { map => new GenericRow(map.values.toArray.asInstanceOf[Array[Any]]): Row } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 31d27bb4f0571..019ff9d300a18 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.types.{ArrayType, BooleanType, StructType} +import org.apache.spark.sql.catalyst.types.{DataType, ArrayType, BooleanType, StructType, MapType} import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD @@ -376,39 +376,27 @@ class SchemaRDD( * Converts a JavaRDD to a PythonRDD. It is used by pyspark. */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { + def toJava(obj: Any, dataType: DataType): Any = dataType match { + case struct: StructType => rowToMap(obj.asInstanceOf[Row], struct) + case array: ArrayType => obj match { + case seq: Seq[Any] => seq.map(x => toJava(x, array.elementType)).asJava + case list: JList[_] => list.map(x => toJava(x, array.elementType)).asJava + case arr if arr != null && arr.getClass.isArray => + arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) + case other => other + } + case mt: MapType => obj.asInstanceOf[Map[_, _]].map { + case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type + }.asJava + // Pyrolite can handle Timestamp + case other => obj + } def rowToMap(row: Row, structType: StructType): JMap[String, Any] = { val fields = structType.fields.map(field => (field.name, field.dataType)) val map: JMap[String, Any] = new java.util.HashMap row.zip(fields).foreach { - case (obj, (attrName, dataType)) => - dataType match { - case struct: StructType => map.put(attrName, rowToMap(obj.asInstanceOf[Row], struct)) - case array @ ArrayType(struct: StructType) => - val arrayValues = obj match { - case seq: Seq[Any] => - seq.map(element => rowToMap(element.asInstanceOf[Row], struct)).asJava - case list: JList[_] => - list.map(element => rowToMap(element.asInstanceOf[Row], struct)) - case set: JSet[_] => - set.map(element => rowToMap(element.asInstanceOf[Row], struct)) - case arr if arr != null && arr.getClass.isArray => - arr.asInstanceOf[Array[Any]].map { - element => rowToMap(element.asInstanceOf[Row], struct) - } - case other => other - } - map.put(attrName, arrayValues) - case array: ArrayType => { - val arrayValues = obj match { - case seq: Seq[Any] => seq.asJava - case other => other - } - map.put(attrName, arrayValues) - } - case other => map.put(attrName, obj) - } + case (obj, (attrName, dataType)) => map.put(attrName, toJava(obj, dataType)) } - map } From dc9653641f8806960d79652afa043c3fb84f25d2 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 29 Jul 2014 12:49:44 -0700 Subject: [PATCH 0113/1492] [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size Implemented stratified sampling that guarantees exact sample size using ScaRSR with two passes over the RDD for sampling without replacement and three passes for sampling with replacement. Author: Doris Xin Author: Xiangrui Meng Closes #1025 from dorx/stratified and squashes the following commits: 245439e [Doris Xin] moved minSamplingRate to getUpperBound eaf5771 [Doris Xin] bug fixes. 17a381b [Doris Xin] fixed a merge issue and a failed unit ea7d27f [Doris Xin] merge master b223529 [Xiangrui Meng] use approx bounds for poisson fix poisson mean for waitlisting add unit tests for Java b3013a4 [Xiangrui Meng] move math3 back to test scope eecee5f [Doris Xin] Merge branch 'master' into stratified f4c21f3 [Doris Xin] Reviewer comments a10e68d [Doris Xin] style fix a2bf756 [Doris Xin] Merge branch 'master' into stratified 680b677 [Doris Xin] use mapPartitionWithIndex instead 9884a9f [Doris Xin] style fix bbfb8c9 [Doris Xin] Merge branch 'master' into stratified ee9d260 [Doris Xin] addressed reviewer comments 6b5b10b [Doris Xin] Merge branch 'master' into stratified 254e03c [Doris Xin] minor fixes and Java API. 4ad516b [Doris Xin] remove unused imports from PairRDDFunctions bd9dc6e [Doris Xin] unit bug and style violation fixed 1fe1cff [Doris Xin] Changed fractionByKey to a map to enable arg check 944a10c [Doris Xin] [SPARK-2145] Add lower bound on sampling rate 0214a76 [Doris Xin] cleanUp 90d94c0 [Doris Xin] merge master 9e74ab5 [Doris Xin] Separated out most of the logic in sampleByKey 7327611 [Doris Xin] merge master 50581fc [Doris Xin] added a TODO for logging in python 46f6c8c [Doris Xin] fixed the NPE caused by closures being cleaned before being passed into the aggregate function 7e1a481 [Doris Xin] changed the permission on SamplingUtil 1d413ce [Doris Xin] fixed checkstyle issues 9ee94ee [Doris Xin] [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size e3fd6a6 [Doris Xin] Merge branch 'master' into takeSample 7cab53a [Doris Xin] fixed import bug in rdd.py ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD 1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS --- .../apache/spark/api/java/JavaPairRDD.scala | 69 +++- .../apache/spark/rdd/PairRDDFunctions.scala | 54 ++- .../spark/util/random/SamplingUtils.scala | 74 +++- .../util/random/StratifiedSamplingUtils.scala | 316 ++++++++++++++++++ .../java/org/apache/spark/JavaAPISuite.java | 37 ++ .../spark/rdd/PairRDDFunctionsSuite.scala | 116 +++++++ pom.xml | 6 + 7 files changed, 656 insertions(+), 16 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 4f3081433a542..31bf8dced2638 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, List => JList, Map => JMap} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ @@ -129,6 +129,73 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed)) + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * If `exact` is set to false, create the sample via simple random sampling, with one pass + * over the RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over + * the RDD to create a sample size that's exactly equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double], + exact: Boolean, + seed: Long): JavaPairRDD[K, V] = + new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, exact, seed)) + + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * If `exact` is set to false, create the sample via simple random sampling, with one pass + * over the RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over + * the RDD to create a sample size that's exactly equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. + * + * Use Utils.random.nextLong as the default seed for the random number generator + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double], + exact: Boolean): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, exact, Utils.random.nextLong) + + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * Produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via + * simple random sampling. + */ + def sampleByKey(withReplacement: Boolean, + fractions: JMap[K, Double], + seed: Long): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, false, seed) + + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * Produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via + * simple random sampling. + * + * Use Utils.random.nextLong as the default seed for the random number generator + */ + def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, false, Utils.random.nextLong) + /** * Return the union of this RDD and another one. Any identical elements will appear multiple * times (use `.distinct()` to eliminate them). 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 c04d162a39616..1af4e5f0b6d08 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -19,12 +19,10 @@ package org.apache.spark.rdd import java.nio.ByteBuffer import java.text.SimpleDateFormat -import java.util.Date -import java.util.{HashMap => JHashMap} +import java.util.{Date, HashMap => JHashMap} +import scala.collection.{Map, mutable} import scala.collection.JavaConversions._ -import scala.collection.Map -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag @@ -34,19 +32,19 @@ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, +import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ -import org.apache.spark.annotation.Experimental -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils import org.apache.spark.util.collection.CompactBuffer +import org.apache.spark.util.random.StratifiedSamplingUtils /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. @@ -195,6 +193,41 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) foldByKey(zeroValue, defaultPartitioner(self))(func) } + /** + * Return a subset of this RDD sampled by key (via stratified sampling). + * + * Create a sample of this RDD using variable sampling rates for different keys as specified by + * `fractions`, a key to sampling rate map. + * + * If `exact` is set to false, create the sample via simple random sampling, with one pass + * over the RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values; otherwise, use + * additional passes over the RDD to create a sample size that's exactly equal to the sum of + * math.ceil(numItems * samplingRate) over all key values with a 99.99% confidence. When sampling + * without replacement, we need one additional pass over the RDD to guarantee sample size; + * when sampling with replacement, we need two additional passes. + * + * @param withReplacement whether to sample with or without replacement + * @param fractions map of specific keys to sampling rates + * @param seed seed for the random number generator + * @param exact whether sample size needs to be exactly math.ceil(fraction * size) per key + * @return RDD containing the sampled subset + */ + def sampleByKey(withReplacement: Boolean, + fractions: Map[K, Double], + exact: Boolean = false, + seed: Long = Utils.random.nextLong): RDD[(K, V)]= { + + require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") + + val samplingFunc = if (withReplacement) { + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, exact, seed) + } else { + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, exact, seed) + } + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) + } + /** * Merge the values for each key using an associative reduce function. This will also perform * the merging locally on each mapper before sending results to a reducer, similarly to a @@ -531,6 +564,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) /** * Return the key-value pairs in this RDD to the master as a Map. + * + * Warning: this doesn't return a multimap (so if you have multiple values to the same key, only + * one value per key is preserved in the map returned) */ def collectAsMap(): Map[K, V] = { val data = self.collect() diff --git a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala index d10141b90e621..c9a864ae62778 100644 --- a/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/SamplingUtils.scala @@ -81,6 +81,9 @@ private[spark] object SamplingUtils { * ~ Binomial(total, fraction) and our choice of q guarantees 1-delta, or 0.9999 success * rate, where success rate is defined the same as in sampling with replacement. * + * The smallest sampling rate supported is 1e-10 (in order to avoid running into the limit of the + * RNG's resolution). + * * @param sampleSizeLowerBound sample size * @param total size of RDD * @param withReplacement whether sampling with replacement @@ -88,14 +91,73 @@ private[spark] object SamplingUtils { */ def computeFractionForSampleSize(sampleSizeLowerBound: Int, total: Long, withReplacement: Boolean): Double = { - val fraction = sampleSizeLowerBound.toDouble / total if (withReplacement) { - val numStDev = if (sampleSizeLowerBound < 12) 9 else 5 - fraction + numStDev * math.sqrt(fraction / total) + PoissonBounds.getUpperBound(sampleSizeLowerBound) / total } else { - val delta = 1e-4 - val gamma = - math.log(delta) / total - math.min(1, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction)) + val fraction = sampleSizeLowerBound.toDouble / total + BinomialBounds.getUpperBound(1e-4, total, fraction) } } } + +/** + * Utility functions that help us determine bounds on adjusted sampling rate to guarantee exact + * sample sizes with high confidence when sampling with replacement. + */ +private[spark] object PoissonBounds { + + /** + * Returns a lambda such that Pr[X > s] is very small, where X ~ Pois(lambda). + */ + def getLowerBound(s: Double): Double = { + math.max(s - numStd(s) * math.sqrt(s), 1e-15) + } + + /** + * Returns a lambda such that Pr[X < s] is very small, where X ~ Pois(lambda). + * + * @param s sample size + */ + def getUpperBound(s: Double): Double = { + math.max(s + numStd(s) * math.sqrt(s), 1e-10) + } + + private def numStd(s: Double): Double = { + // TODO: Make it tighter. + if (s < 6.0) { + 12.0 + } else if (s < 16.0) { + 9.0 + } else { + 6.0 + } + } +} + +/** + * Utility functions that help us determine bounds on adjusted sampling rate to guarantee exact + * sample size with high confidence when sampling without replacement. + */ +private[spark] object BinomialBounds { + + val minSamplingRate = 1e-10 + + /** + * Returns a threshold `p` such that if we conduct n Bernoulli trials with success rate = `p`, + * it is very unlikely to have more than `fraction * n` successes. + */ + def getLowerBound(delta: Double, n: Long, fraction: Double): Double = { + val gamma = - math.log(delta) / n * (2.0 / 3.0) + fraction + gamma - math.sqrt(gamma * gamma + 3 * gamma * fraction) + } + + /** + * Returns a threshold `p` such that if we conduct n Bernoulli trials with success rate = `p`, + * it is very unlikely to have less than `fraction * n` successes. + */ + def getUpperBound(delta: Double, n: Long, fraction: Double): Double = { + val gamma = - math.log(delta) / n + math.min(1, + math.max(minSamplingRate, fraction + gamma + math.sqrt(gamma * gamma + 2 * gamma * fraction))) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala new file mode 100644 index 0000000000000..8f95d7c6b799b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -0,0 +1,316 @@ +/* + * 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.util.random + +import scala.collection.Map +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD + +/** + * Auxiliary functions and data structures for the sampleByKey method in PairRDDFunctions. + * + * Essentially, when exact sample size is necessary, we make additional passes over the RDD to + * compute the exact threshold value to use for each stratum to guarantee exact sample size with + * high probability. This is achieved by maintaining a waitlist of size O(log(s)), where s is the + * desired sample size for each stratum. + * + * Like in simple random sampling, we generate a random value for each item from the + * uniform distribution [0.0, 1.0]. All items with values <= min(values of items in the waitlist) + * are accepted into the sample instantly. The threshold for instant accept is designed so that + * s - numAccepted = O(sqrt(s)), where s is again the desired sample size. Thus, by maintaining a + * waitlist size = O(sqrt(s)), we will be able to create a sample of the exact size s by adding + * a portion of the waitlist to the set of items that are instantly accepted. The exact threshold + * is computed by sorting the values in the waitlist and picking the value at (s - numAccepted). + * + * Note that since we use the same seed for the RNG when computing the thresholds and the actual + * sample, our computed thresholds are guaranteed to produce the desired sample size. + * + * For more theoretical background on the sampling techniques used here, please refer to + * http://jmlr.org/proceedings/papers/v28/meng13a.html + */ + +private[spark] object StratifiedSamplingUtils extends Logging { + + /** + * Count the number of items instantly accepted and generate the waitlist for each stratum. + * + * This is only invoked when exact sample size is required. + */ + def getAcceptanceResults[K, V](rdd: RDD[(K, V)], + withReplacement: Boolean, + fractions: Map[K, Double], + counts: Option[Map[K, Long]], + seed: Long): mutable.Map[K, AcceptanceResult] = { + val combOp = getCombOp[K] + val mappedPartitionRDD = rdd.mapPartitionsWithIndex { case (partition, iter) => + val zeroU: mutable.Map[K, AcceptanceResult] = new mutable.HashMap[K, AcceptanceResult]() + val rng = new RandomDataGenerator() + rng.reSeed(seed + partition) + val seqOp = getSeqOp(withReplacement, fractions, rng, counts) + Iterator(iter.aggregate(zeroU)(seqOp, combOp)) + } + mappedPartitionRDD.reduce(combOp) + } + + /** + * Returns the function used by aggregate to collect sampling statistics for each partition. + */ + def getSeqOp[K, V](withReplacement: Boolean, + fractions: Map[K, Double], + rng: RandomDataGenerator, + counts: Option[Map[K, Long]]): + (mutable.Map[K, AcceptanceResult], (K, V)) => mutable.Map[K, AcceptanceResult] = { + val delta = 5e-5 + (result: mutable.Map[K, AcceptanceResult], item: (K, V)) => { + val key = item._1 + val fraction = fractions(key) + if (!result.contains(key)) { + result += (key -> new AcceptanceResult()) + } + val acceptResult = result(key) + + if (withReplacement) { + // compute acceptBound and waitListBound only if they haven't been computed already + // since they don't change from iteration to iteration. + // TODO change this to the streaming version + if (acceptResult.areBoundsEmpty) { + val n = counts.get(key) + val sampleSize = math.ceil(n * fraction).toLong + val lmbd1 = PoissonBounds.getLowerBound(sampleSize) + val lmbd2 = PoissonBounds.getUpperBound(sampleSize) + acceptResult.acceptBound = lmbd1 / n + acceptResult.waitListBound = (lmbd2 - lmbd1) / n + } + val acceptBound = acceptResult.acceptBound + val copiesAccepted = if (acceptBound == 0.0) 0L else rng.nextPoisson(acceptBound) + if (copiesAccepted > 0) { + acceptResult.numAccepted += copiesAccepted + } + val copiesWaitlisted = rng.nextPoisson(acceptResult.waitListBound) + if (copiesWaitlisted > 0) { + acceptResult.waitList ++= ArrayBuffer.fill(copiesWaitlisted)(rng.nextUniform()) + } + } else { + // We use the streaming version of the algorithm for sampling without replacement to avoid + // using an extra pass over the RDD for computing the count. + // Hence, acceptBound and waitListBound change on every iteration. + acceptResult.acceptBound = + BinomialBounds.getLowerBound(delta, acceptResult.numItems, fraction) + acceptResult.waitListBound = + BinomialBounds.getUpperBound(delta, acceptResult.numItems, fraction) + + val x = rng.nextUniform() + if (x < acceptResult.acceptBound) { + acceptResult.numAccepted += 1 + } else if (x < acceptResult.waitListBound) { + acceptResult.waitList += x + } + } + acceptResult.numItems += 1 + result + } + } + + /** + * Returns the function used combine results returned by seqOp from different partitions. + */ + def getCombOp[K]: (mutable.Map[K, AcceptanceResult], mutable.Map[K, AcceptanceResult]) + => mutable.Map[K, AcceptanceResult] = { + (result1: mutable.Map[K, AcceptanceResult], result2: mutable.Map[K, AcceptanceResult]) => { + // take union of both key sets in case one partition doesn't contain all keys + result1.keySet.union(result2.keySet).foreach { key => + // Use result2 to keep the combined result since r1 is usual empty + val entry1 = result1.get(key) + if (result2.contains(key)) { + result2(key).merge(entry1) + } else { + if (entry1.isDefined) { + result2 += (key -> entry1.get) + } + } + } + result2 + } + } + + /** + * Given the result returned by getCounts, determine the threshold for accepting items to + * generate exact sample size. + * + * To do so, we compute sampleSize = math.ceil(size * samplingRate) for each stratum and compare + * it to the number of items that were accepted instantly and the number of items in the waitlist + * for that stratum. Most of the time, numAccepted <= sampleSize <= (numAccepted + numWaitlisted), + * which means we need to sort the elements in the waitlist by their associated values in order + * to find the value T s.t. |{elements in the stratum whose associated values <= T}| = sampleSize. + * Note that all elements in the waitlist have values >= bound for instant accept, so a T value + * in the waitlist range would allow all elements that were instantly accepted on the first pass + * to be included in the sample. + */ + def computeThresholdByKey[K](finalResult: Map[K, AcceptanceResult], + fractions: Map[K, Double]): Map[K, Double] = { + val thresholdByKey = new mutable.HashMap[K, Double]() + for ((key, acceptResult) <- finalResult) { + val sampleSize = math.ceil(acceptResult.numItems * fractions(key)).toLong + if (acceptResult.numAccepted > sampleSize) { + logWarning("Pre-accepted too many") + thresholdByKey += (key -> acceptResult.acceptBound) + } else { + val numWaitListAccepted = (sampleSize - acceptResult.numAccepted).toInt + if (numWaitListAccepted >= acceptResult.waitList.size) { + logWarning("WaitList too short") + thresholdByKey += (key -> acceptResult.waitListBound) + } else { + thresholdByKey += (key -> acceptResult.waitList.sorted.apply(numWaitListAccepted)) + } + } + } + thresholdByKey + } + + /** + * Return the per partition sampling function used for sampling without replacement. + * + * When exact sample size is required, we make an additional pass over the RDD to determine the + * exact sampling rate that guarantees sample size with high confidence. + * + * The sampling function has a unique seed per partition. + */ + def getBernoulliSamplingFunction[K, V](rdd: RDD[(K, V)], + fractions: Map[K, Double], + exact: Boolean, + seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { + var samplingRateByKey = fractions + if (exact) { + // determine threshold for each stratum and resample + val finalResult = getAcceptanceResults(rdd, false, fractions, None, seed) + samplingRateByKey = computeThresholdByKey(finalResult, fractions) + } + (idx: Int, iter: Iterator[(K, V)]) => { + val rng = new RandomDataGenerator + rng.reSeed(seed + idx) + // Must use the same invoke pattern on the rng as in getSeqOp for without replacement + // in order to generate the same sequence of random numbers when creating the sample + iter.filter(t => rng.nextUniform() < samplingRateByKey(t._1)) + } + } + + /** + * Return the per partition sampling function used for sampling with replacement. + * + * When exact sample size is required, we make two additional passed over the RDD to determine + * the exact sampling rate that guarantees sample size with high confidence. The first pass + * counts the number of items in each stratum (group of items with the same key) in the RDD, and + * the second pass uses the counts to determine exact sampling rates. + * + * The sampling function has a unique seed per partition. + */ + def getPoissonSamplingFunction[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)], + fractions: Map[K, Double], + exact: Boolean, + seed: Long): (Int, Iterator[(K, V)]) => Iterator[(K, V)] = { + // TODO implement the streaming version of sampling w/ replacement that doesn't require counts + if (exact) { + val counts = Some(rdd.countByKey()) + val finalResult = getAcceptanceResults(rdd, true, fractions, counts, seed) + val thresholdByKey = computeThresholdByKey(finalResult, fractions) + (idx: Int, iter: Iterator[(K, V)]) => { + val rng = new RandomDataGenerator() + rng.reSeed(seed + idx) + iter.flatMap { item => + val key = item._1 + val acceptBound = finalResult(key).acceptBound + // Must use the same invoke pattern on the rng as in getSeqOp for with replacement + // in order to generate the same sequence of random numbers when creating the sample + val copiesAccepted = if (acceptBound == 0) 0L else rng.nextPoisson(acceptBound) + val copiesWailisted = rng.nextPoisson(finalResult(key).waitListBound) + val copiesInSample = copiesAccepted + + (0 until copiesWailisted).count(i => rng.nextUniform() < thresholdByKey(key)) + if (copiesInSample > 0) { + Iterator.fill(copiesInSample.toInt)(item) + } else { + Iterator.empty + } + } + } + } else { + (idx: Int, iter: Iterator[(K, V)]) => { + val rng = new RandomDataGenerator() + rng.reSeed(seed + idx) + iter.flatMap { item => + val count = rng.nextPoisson(fractions(item._1)) + if (count > 0) { + Iterator.fill(count)(item) + } else { + Iterator.empty + } + } + } + } + } + + /** A random data generator that generates both uniform values and Poisson values. */ + private class RandomDataGenerator { + val uniform = new XORShiftRandom() + var poisson = new Poisson(1.0, new DRand) + + def reSeed(seed: Long) { + uniform.setSeed(seed) + poisson = new Poisson(1.0, new DRand(seed.toInt)) + } + + def nextPoisson(mean: Double): Int = { + poisson.nextInt(mean) + } + + def nextUniform(): Double = { + uniform.nextDouble() + } + } +} + +/** + * Object used by seqOp to keep track of the number of items accepted and items waitlisted per + * stratum, as well as the bounds for accepting and waitlisting items. + * + * `[random]` here is necessary since it's in the return type signature of seqOp defined above + */ +private[random] class AcceptanceResult(var numItems: Long = 0L, var numAccepted: Long = 0L) + extends Serializable { + + val waitList = new ArrayBuffer[Double] + var acceptBound: Double = Double.NaN // upper bound for accepting item instantly + var waitListBound: Double = Double.NaN // upper bound for adding item to waitlist + + def areBoundsEmpty = acceptBound.isNaN || waitListBound.isNaN + + def merge(other: Option[AcceptanceResult]): Unit = { + if (other.isDefined) { + waitList ++= other.get.waitList + numAccepted += other.get.numAccepted + numItems += other.get.numItems + } + } +} diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index f882a8623fd84..e8bd65f8e4507 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -29,6 +29,7 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -1208,4 +1209,40 @@ public Tuple2 call(Integer x) { pairRDD.collect(); // Works fine pairRDD.collectAsMap(); // Used to crash with ClassCastException } + + @Test + @SuppressWarnings("unchecked") + public void sampleByKey() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i % 2, 1); + } + }); + Map fractions = Maps.newHashMap(); + fractions.put(0, 0.5); + fractions.put(1, 1.0); + JavaPairRDD wr = rdd2.sampleByKey(true, fractions, 1L); + Map wrCounts = (Map) (Object) wr.countByKey(); + Assert.assertTrue(wrCounts.size() == 2); + Assert.assertTrue(wrCounts.get(0) > 0); + Assert.assertTrue(wrCounts.get(1) > 0); + JavaPairRDD wor = rdd2.sampleByKey(false, fractions, 1L); + Map worCounts = (Map) (Object) wor.countByKey(); + Assert.assertTrue(worCounts.size() == 2); + Assert.assertTrue(worCounts.get(0) > 0); + Assert.assertTrue(worCounts.get(1) > 0); + JavaPairRDD wrExact = rdd2.sampleByKey(true, fractions, true, 1L); + Map wrExactCounts = (Map) (Object) wrExact.countByKey(); + Assert.assertTrue(wrExactCounts.size() == 2); + Assert.assertTrue(wrExactCounts.get(0) == 2); + Assert.assertTrue(wrExactCounts.get(1) == 4); + JavaPairRDD worExact = rdd2.sampleByKey(false, fractions, true, 1L); + Map worExactCounts = (Map) (Object) worExact.countByKey(); + Assert.assertTrue(worExactCounts.size() == 2); + Assert.assertTrue(worExactCounts.get(0) == 2); + Assert.assertTrue(worExactCounts.get(1) == 4); + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 447e38ec9dbd0..4f49d4a1d4d34 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -83,6 +83,122 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(valuesFor2.toList.sorted === List(1)) } + test("sampleByKey") { + def stratifier (fractionPositive: Double) = { + (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" + } + + def checkSize(exact: Boolean, + withReplacement: Boolean, + expected: Long, + actual: Long, + p: Double): Boolean = { + if (exact) { + return expected == actual + } + val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) + // Very forgiving margin since we're dealing with very small sample sizes most of the time + math.abs(actual - expected) <= 6 * stdev + } + + // Without replacement validation + def takeSampleAndValidateBernoulli(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey() + .mapValues(count => math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = stratifiedData.sampleByKey(false, fractions, exact, seed) + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case(k, v) => + assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } + assert(takeSample.size === takeSample.toSet.size) + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + } + + // With replacement validation + def takeSampleAndValidatePoisson(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey().mapValues(count => + math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = stratifiedData.sampleByKey(true, fractions, exact, seed) + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case(k, v) => + assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) } + val groupedByKey = takeSample.groupBy(_._1) + for ((key, v) <- groupedByKey) { + if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { + // sample large enough for there to be repeats with high likelihood + assert(v.toSet.size < expectedSampleSize(key)) + } else { + if (exact) { + assert(v.toSet.size <= expectedSampleSize(key)) + } else { + assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) + } + } + } + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + } + + def checkAllCombos(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { + takeSampleAndValidateBernoulli(stratifiedData, true, samplingRate, seed, n) + takeSampleAndValidateBernoulli(stratifiedData, false, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, true, samplingRate, seed, n) + takeSampleAndValidatePoisson(stratifiedData, false, samplingRate, seed, n) + } + + val defaultSeed = 1L + + // vary RDD size + for (n <- List(100, 1000, 1000000)) { + val data = sc.parallelize(1 to n, 2) + val fractionPositive = 0.3 + val stratifiedData = data.keyBy(stratifier(fractionPositive)) + + val samplingRate = 0.1 + checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + } + + // vary fractionPositive + for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(stratifier(fractionPositive)) + + val samplingRate = 0.1 + checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + } + + // Use the same data for the rest of the tests + val fractionPositive = 0.3 + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(stratifier(fractionPositive)) + + // vary seed + for (seed <- defaultSeed to defaultSeed + 5L) { + val samplingRate = 0.1 + checkAllCombos(stratifiedData, samplingRate, seed, n) + } + + // vary sampling rate + for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { + checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + } + } + test("reduceByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) val sums = pairs.reduceByKey(_+_).collect() diff --git a/pom.xml b/pom.xml index 8b1435cfe5d19..39538f9660623 100644 --- a/pom.xml +++ b/pom.xml @@ -257,6 +257,12 @@ commons-codec 1.5 + + org.apache.commons + commons-math3 + 3.3 + test + com.google.code.findbugs jsr305 From c7db274be79f448fda566208946cb50958ea9b1a Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Tue, 29 Jul 2014 15:32:50 -0700 Subject: [PATCH 0114/1492] [SPARK-2393][SQL] Cost estimation optimization framework for Catalyst logical plans & sample usage. The idea is that every Catalyst logical plan gets hold of a Statistics class, the usage of which provides useful estimations on various statistics. See the implementations of `MetastoreRelation`. This patch also includes several usages of the estimation interface in the planner. For instance, we now use physical table sizes from the estimate interface to convert an equi-join to a broadcast join (when doing so is beneficial, as determined by a size threshold). Finally, there are a couple minor accompanying changes including: - Remove the not-in-use `BaseRelation`. - Make SparkLogicalPlan take a `SQLContext` in the second param list. Author: Zongheng Yang Closes #1238 from concretevitamin/estimates and squashes the following commits: 329071d [Zongheng Yang] Address review comments; turn config name from string to field in SQLConf. 8663e84 [Zongheng Yang] Use BigInt for stat; for logical leaves, by default throw an exception. 2f2fb89 [Zongheng Yang] Fix statistics for SparkLogicalPlan. 9951305 [Zongheng Yang] Remove childrenStats. 16fc60a [Zongheng Yang] Avoid calling statistics on plans if auto join conversion is disabled. 8bd2816 [Zongheng Yang] Add a note on performance of statistics. 6e594b8 [Zongheng Yang] Get size info from metastore for MetastoreRelation. 01b7a3e [Zongheng Yang] Update scaladoc for a field and move it to @param section. 549061c [Zongheng Yang] Remove numTuples in Statistics for now. 729a8e2 [Zongheng Yang] Update docs to be more explicit. 573e644 [Zongheng Yang] Remove singleton SQLConf and move back `settings` to the trait. 2d99eb5 [Zongheng Yang] {Cleanup, use synchronized in, enrich} StatisticsSuite. ca5b825 [Zongheng Yang] Inject SQLContext into SparkLogicalPlan, removing SQLConf mixin from it. 43d38a6 [Zongheng Yang] Revert optimization for BroadcastNestedLoopJoin (this fixes tests). 0ef9e5b [Zongheng Yang] Use multiplication instead of sum for default estimates. 4ef0d26 [Zongheng Yang] Make Statistics a case class. 3ba8f3e [Zongheng Yang] Add comment. e5bcf5b [Zongheng Yang] Fix optimization conditions & update scala docs to explain. 7d9216a [Zongheng Yang] Apply estimation to planning ShuffleHashJoin & BroadcastNestedLoopJoin. 73cde01 [Zongheng Yang] Move SQLConf back. Assign default sizeInBytes to SparkLogicalPlan. 73412be [Zongheng Yang] Move SQLConf to Catalyst & add default val for sizeInBytes. 7a60ab7 [Zongheng Yang] s/Estimates/Statistics, s/cardinality/numTuples. de3ae13 [Zongheng Yang] Add parquetAfter() properly in test. dcff9bd [Zongheng Yang] Cleanups. 84301a4 [Zongheng Yang] Refactors. 5bf5586 [Zongheng Yang] Typo. 56a8e6e [Zongheng Yang] Prototype impl of estimations for Catalyst logical plans. --- .../sql/catalyst/analysis/unresolved.scala | 4 +- .../catalyst/plans/logical/BaseRelation.scala | 24 ----- .../catalyst/plans/logical/LogicalPlan.scala | 22 +++++ .../scala/org/apache/spark/sql/SQLConf.scala | 61 +++++++----- .../org/apache/spark/sql/SQLContext.scala | 20 ++-- .../org/apache/spark/sql/SchemaRDD.scala | 3 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 2 +- .../spark/sql/api/java/JavaSQLContext.scala | 4 +- .../spark/sql/execution/SparkPlan.scala | 18 ++-- .../spark/sql/execution/SparkStrategies.scala | 57 ++++++----- .../org/apache/spark/sql/json/JsonRDD.scala | 11 ++- .../spark/sql/parquet/ParquetRelation.scala | 4 +- .../org/apache/spark/sql/JoinSuite.scala | 2 - .../spark/sql/hive/HiveMetastoreCatalog.scala | 47 ++++++--- .../spark/sql/hive/StatisticsSuite.scala | 95 +++++++++++++++++++ .../hive/execution/HiveComparisonTest.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../spark/sql/parquet/HiveParquetSuite.scala | 2 +- 18 files changed, 256 insertions(+), 124 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 7abeb032964e1..a0e25775da6dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.{errors, trees} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.BaseRelation +import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.trees.TreeNode /** @@ -36,7 +36,7 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str case class UnresolvedRelation( databaseName: Option[String], tableName: String, - alias: Option[String] = None) extends BaseRelation { + alias: Option[String] = None) extends LeafNode { override def output = Nil override lazy val resolved = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala deleted file mode 100644 index 582334aa42590..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.plans.logical - -abstract class BaseRelation extends LeafNode { - self: Product => - - def tableName: String -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index edc37e3877c0e..ac85f95b52a2f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -26,6 +26,25 @@ import org.apache.spark.sql.catalyst.trees abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => + /** + * Estimates of various statistics. The default estimation logic simply lazily multiplies the + * corresponding statistic produced by the children. To override this behavior, override + * `statistics` and assign it an overriden version of `Statistics`. + * + * '''NOTE''': concrete and/or overriden versions of statistics fields should pay attention to the + * performance of the implementations. The reason is that estimations might get triggered in + * performance-critical processes, such as query plan planning. + * + * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it + * defaults to the product of children's `sizeInBytes`. + */ + case class Statistics( + sizeInBytes: BigInt + ) + lazy val statistics: Statistics = Statistics( + sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product + ) + /** * Returns the set of attributes that are referenced by this node * during evaluation. @@ -92,6 +111,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { self: Product => + override lazy val statistics: Statistics = + throw new UnsupportedOperationException("default leaf nodes don't have meaningful Statistics") + // Leaf nodes by definition cannot reference any input attributes. override def references = Set.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 41920c00b5a2c..be8d4e15ec4b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -21,17 +21,31 @@ import java.util.Properties import scala.collection.JavaConverters._ +object SQLConf { + val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold" + val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" + val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" + + object Deprecated { + val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" + } +} + /** - * SQLConf holds mutable config parameters and hints. These can be set and - * queried either by passing SET commands into Spark SQL's DSL - * functions (sql(), hql(), etc.), or by programmatically using setters and - * getters of this class. + * A trait that enables the setting and getting of mutable config parameters/hints. + * + * In the presence of a SQLContext, these can be set and queried by passing SET commands + * into Spark SQL's query functions (sql(), hql(), etc.). Otherwise, users of this trait can + * modify the hints by programmatically calling the setters and getters of this trait. * - * SQLConf is thread-safe (internally synchronized so safe to be used in multiple threads). + * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ trait SQLConf { import SQLConf._ + @transient protected[spark] val settings = java.util.Collections.synchronizedMap( + new java.util.HashMap[String, String]()) + /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? @@ -40,28 +54,33 @@ trait SQLConf { /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to - * a broadcast value during the physical executions of join operations. Setting this to 0 + * a broadcast value during the physical executions of join operations. Setting this to -1 * effectively disables auto conversion. - * Hive setting: hive.auto.convert.join.noconditionaltask.size. + * + * Hive setting: hive.auto.convert.join.noconditionaltask.size, whose default value is also 10000. */ - private[spark] def autoConvertJoinSize: Int = get(AUTO_CONVERT_JOIN_SIZE, "10000").toInt + private[spark] def autoBroadcastJoinThreshold: Int = + get(AUTO_BROADCASTJOIN_THRESHOLD, "10000").toInt - /** A comma-separated list of table names marked to be broadcasted during joins. */ - private[spark] def joinBroadcastTables: String = get(JOIN_BROADCAST_TABLES, "") + /** + * The default size in bytes to assign to a logical operator's estimation statistics. By default, + * it is set to a larger value than `autoConvertJoinSize`, hence any logical operator without a + * properly implemented estimation of this statistic will not be incorrectly broadcasted in joins. + */ + private[spark] def defaultSizeInBytes: Long = + getOption(DEFAULT_SIZE_IN_BYTES).map(_.toLong).getOrElse(autoBroadcastJoinThreshold + 1) /** ********************** SQLConf functionality methods ************ */ - @transient - private val settings = java.util.Collections.synchronizedMap( - new java.util.HashMap[String, String]()) - def set(props: Properties): Unit = { - props.asScala.foreach { case (k, v) => this.settings.put(k, v) } + settings.synchronized { + props.asScala.foreach { case (k, v) => settings.put(k, v) } + } } def set(key: String, value: String): Unit = { require(key != null, "key cannot be null") - require(value != null, s"value cannot be null for $key") + require(value != null, s"value cannot be null for key: $key") settings.put(key, value) } @@ -90,13 +109,3 @@ trait SQLConf { } } - -object SQLConf { - val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" - val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" - val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" - - object Deprecated { - val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c178dad662532..a136c7b3ffef5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -24,14 +24,14 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.ExpressionConversions -import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies @@ -86,7 +86,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))) + new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))(self)) /** * Loads a Parquet file, returning the result as a [[SchemaRDD]]. @@ -127,7 +127,7 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = - new SchemaRDD(this, JsonRDD.inferSchema(json, samplingRatio)) + new SchemaRDD(this, JsonRDD.inferSchema(self, json, samplingRatio)) /** * :: Experimental :: @@ -170,11 +170,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - val name = tableName - val newPlan = rdd.logicalPlan transform { - case s @ SparkLogicalPlan(ExistingRdd(_, _), _) => s.copy(tableName = name) - } - catalog.registerTable(None, tableName, newPlan) + catalog.registerTable(None, tableName, rdd.logicalPlan) } /** @@ -212,7 +208,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case inMem @ InMemoryRelation(_, _, e: ExistingRdd) => inMem.cachedColumnBuffers.unpersist() catalog.unregisterTable(None, tableName) - catalog.registerTable(None, tableName, SparkLogicalPlan(e)) + catalog.registerTable(None, tableName, SparkLogicalPlan(e)(self)) case inMem: InMemoryRelation => inMem.cachedColumnBuffers.unpersist() catalog.unregisterTable(None, tableName) @@ -405,7 +401,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new GenericRow(map.values.toArray.asInstanceOf[Array[Any]]): Row } } - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema, rowRdd))) + new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema, rowRdd))(self)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 019ff9d300a18..172b6e0e7f26b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -418,7 +418,8 @@ class SchemaRDD( * @group schema */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { - new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(queryExecution.analyzed.output, rdd))) + new SchemaRDD(sqlContext, + SparkLogicalPlan(ExistingRdd(queryExecution.analyzed.output, rdd))(sqlContext)) } // ======================================================================= diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index fe81721943202..fd751031b26e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -56,7 +56,7 @@ private[sql] trait SchemaRDDLike { // happen right away to let these side effects take place eagerly. case _: Command | _: InsertIntoTable | _: InsertIntoCreatedTable | _: WriteToFile => queryExecution.toRdd - SparkLogicalPlan(queryExecution.executedPlan) + SparkLogicalPlan(queryExecution.executedPlan)(sqlContext) case _ => baseLogicalPlan } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 790d9ef22cf16..806097c917b91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -92,7 +92,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow } } - new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))) + new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))(sqlContext)) } /** @@ -120,7 +120,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { * @group userf */ def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = - new JavaSchemaRDD(sqlContext, JsonRDD.inferSchema(json, 1.0)) + new JavaSchemaRDD(sqlContext, JsonRDD.inferSchema(sqlContext, json, 1.0)) /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 27dc091b85812..77c874d0315ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -19,12 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.sql.{Logging, Row, SQLContext} import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.BaseRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ /** @@ -66,8 +66,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { * linking. */ @DeveloperApi -case class SparkLogicalPlan(alreadyPlanned: SparkPlan, tableName: String = "SparkLogicalPlan") - extends BaseRelation with MultiInstanceRelation { +case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQLContext) + extends LogicalPlan with MultiInstanceRelation { def output = alreadyPlanned.output override def references = Set.empty @@ -78,9 +78,15 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan, tableName: String = "Spar alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) case _ => sys.error("Multiple instance of the same relation detected.") - }, tableName) - .asInstanceOf[this.type] + })(sqlContext).asInstanceOf[this.type] } + + @transient override lazy val statistics = Statistics( + // TODO: Instead of returning a default value here, find a way to return a meaningful size + // estimate for RDDs. See PR 1238 for more discussions. + sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + ) + } private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index c078e71fe0290..404d48ae05b45 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.execution +import scala.util.Try + import org.apache.spark.sql.{SQLContext, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ @@ -47,9 +49,18 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { /** * Uses the ExtractEquiJoinKeys pattern to find joins where at least some of the predicates can be * evaluated by matching hash keys. + * + * This strategy applies a simple optimization based on the estimates of the physical sizes of + * the two join sides. When planning a [[execution.BroadcastHashJoin]], if one side has an + * estimated physical size smaller than the user-settable threshold + * [[org.apache.spark.sql.SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]], the planner would mark it as the + * ''build'' relation and mark the other relation as the ''stream'' side. The build table will be + * ''broadcasted'' to all of the executors involved in the join, as a + * [[org.apache.spark.broadcast.Broadcast]] object. If both estimates exceed the threshold, they + * will instead be used to decide the build side in a [[execution.ShuffledHashJoin]]. */ object HashJoin extends Strategy with PredicateHelper { - private[this] def broadcastHashJoin( + private[this] def makeBroadcastHashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], left: LogicalPlan, @@ -61,33 +72,27 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { condition.map(Filter(_, broadcastHashJoin)).getOrElse(broadcastHashJoin) :: Nil } - def broadcastTables: Seq[String] = sqlContext.joinBroadcastTables.split(",").toBuffer - def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ExtractEquiJoinKeys( - Inner, - leftKeys, - rightKeys, - condition, - left, - right @ PhysicalOperation(_, _, b: BaseRelation)) - if broadcastTables.contains(b.tableName) => - broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if Try(sqlContext.autoBroadcastJoinThreshold > 0 && + right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold).getOrElse(false) => + makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) - case ExtractEquiJoinKeys( - Inner, - leftKeys, - rightKeys, - condition, - left @ PhysicalOperation(_, _, b: BaseRelation), - right) - if broadcastTables.contains(b.tableName) => - broadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if Try(sqlContext.autoBroadcastJoinThreshold > 0 && + left.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold).getOrElse(false) => + makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => + val buildSide = + if (Try(right.statistics.sizeInBytes <= left.statistics.sizeInBytes).getOrElse(false)) { + BuildRight + } else { + BuildLeft + } val hashJoin = execution.ShuffledHashJoin( - leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) + leftKeys, rightKeys, buildSide, planLater(left), planLater(right)) condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil case _ => Nil @@ -273,8 +278,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Limit(limit, planLater(child))(sqlContext) :: Nil case Unions(unionChildren) => execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil - case logical.Except(left,right) => - execution.Except(planLater(left),planLater(right)) :: Nil + case logical.Except(left,right) => + execution.Except(planLater(left),planLater(right)) :: Nil case logical.Intersect(left, right) => execution.Intersect(planLater(left), planLater(right)) :: Nil case logical.Generate(generator, join, outer, _, child) => @@ -283,7 +288,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ExistingRdd(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil - case SparkLogicalPlan(existingPlan, _) => existingPlan :: Nil + case SparkLogicalPlan(existingPlan) => existingPlan :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index b48c70ee73a27..6c2b553bb908e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -28,11 +28,12 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} -import org.apache.spark.sql.Logging +import org.apache.spark.sql.{SQLContext, Logging} private[sql] object JsonRDD extends Logging { private[sql] def inferSchema( + sqlContext: SQLContext, json: RDD[String], samplingRatio: Double = 1.0): LogicalPlan = { require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") @@ -40,15 +41,17 @@ private[sql] object JsonRDD extends Logging { val allKeys = parseJson(schemaData).map(allKeysWithValueTypes).reduce(_ ++ _) val baseSchema = createSchema(allKeys) - createLogicalPlan(json, baseSchema) + createLogicalPlan(json, baseSchema, sqlContext) } private def createLogicalPlan( json: RDD[String], - baseSchema: StructType): LogicalPlan = { + baseSchema: StructType, + sqlContext: SQLContext): LogicalPlan = { val schema = nullTypeToStringType(baseSchema) - SparkLogicalPlan(ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema)))) + SparkLogicalPlan( + ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema))))(sqlContext) } private def createSchema(allKeys: Set[(String, DataType)]): StructType = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 9c4771d1a9846..8c7dbd5eb4a09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -27,6 +27,7 @@ import parquet.hadoop.ParquetOutputFormat import parquet.hadoop.metadata.CompressionCodecName import parquet.schema.MessageType +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} @@ -45,7 +46,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} */ private[sql] case class ParquetRelation( path: String, - @transient conf: Option[Configuration] = None) extends LeafNode with MultiInstanceRelation { + @transient conf: Option[Configuration] = None) + extends LeafNode with MultiInstanceRelation { self: Product => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index e17ecc87fd52a..025c396ef0629 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner} -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ class JoinSuite extends QueryTest { 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 156b090712df2..dff1d6a4b93bb 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 @@ -19,15 +19,16 @@ package org.apache.spark.sql.hive import scala.util.parsing.combinator.RegexParsers +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.Logging +import org.apache.spark.sql.{SQLContext, Logging} import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical @@ -64,9 +65,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Since HiveQL is case insensitive for table names we make them all lowercase. MetastoreRelation( - databaseName, - tblName, - alias)(table.getTTable, partitions.map(part => part.getTPartition)) + databaseName, tblName, alias)( + table.getTTable, partitions.map(part => part.getTPartition))(hive) } def createTable( @@ -251,7 +251,11 @@ object HiveMetastoreTypes extends RegexParsers { private[hive] case class MetastoreRelation (databaseName: String, tableName: String, alias: Option[String]) (val table: TTable, val partitions: Seq[TPartition]) - extends BaseRelation { + (@transient sqlContext: SQLContext) + extends LeafNode { + + self: Product => + // TODO: Can we use org.apache.hadoop.hive.ql.metadata.Table as the type of table and // use org.apache.hadoop.hive.ql.metadata.Partition as the type of elements of partitions. // Right now, using org.apache.hadoop.hive.ql.metadata.Table and @@ -264,6 +268,21 @@ private[hive] case class MetastoreRelation new Partition(hiveQlTable, p) } + @transient override lazy val statistics = Statistics( + sizeInBytes = { + // TODO: check if this estimate is valid for tables after partition pruning. + // NOTE: getting `totalSize` directly from params is kind of hacky, but this should be + // relatively cheap if parameters for the table are populated into the metastore. An + // alternative would be going through Hadoop's FileSystem API, which can be expensive if a lot + // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, + // `rawDataSize` keys that we can look at in the future. + BigInt( + Option(hiveQlTable.getParameters.get("totalSize")) + .map(_.toLong) + .getOrElse(sqlContext.defaultSizeInBytes)) + } + ) + val tableDesc = new TableDesc( Class.forName(hiveQlTable.getSerializationLib).asInstanceOf[Class[Deserializer]], hiveQlTable.getInputFormatClass, @@ -275,14 +294,14 @@ private[hive] case class MetastoreRelation hiveQlTable.getMetadata ) - implicit class SchemaAttribute(f: FieldSchema) { - def toAttribute = AttributeReference( - f.getName, - HiveMetastoreTypes.toDataType(f.getType), - // Since data can be dumped in randomly with no validation, everything is nullable. - nullable = true - )(qualifiers = tableName +: alias.toSeq) - } + implicit class SchemaAttribute(f: FieldSchema) { + def toAttribute = AttributeReference( + f.getName, + HiveMetastoreTypes.toDataType(f.getType), + // Since data can be dumped in randomly with no validation, everything is nullable. + nullable = true + )(qualifiers = tableName +: alias.toSeq) + } // Must be a stable value since new attributes are born here. val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala new file mode 100644 index 0000000000000..a61fd9df95c94 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import scala.reflect.ClassTag + +import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ + +class StatisticsSuite extends QueryTest { + + test("estimates the size of a test MetastoreRelation") { + val rdd = hql("""SELECT * FROM src""") + val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => + mr.statistics.sizeInBytes + } + assert(sizes.size === 1) + assert(sizes(0).equals(BigInt(5812)), + s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") + } + + test("auto converts to broadcast hash join, by size estimate of a relation") { + def mkTest( + before: () => Unit, + after: () => Unit, + query: String, + expectedAnswer: Seq[Any], + ct: ClassTag[_]) = { + before() + + var rdd = hql(query) + + // Assert src has a size smaller than the threshold. + val sizes = rdd.queryExecution.analyzed.collect { + case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes + } + assert(sizes.size === 2 && sizes(0) <= autoBroadcastJoinThreshold, + s"query should contain two relations, each of which has size smaller than autoConvertSize") + + // Using `sparkPlan` because for relevant patterns in HashJoin to be + // matched, other strategies need to be applied. + var bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + assert(bhj.size === 1, + s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") + + checkAnswer(rdd, expectedAnswer) // check correctness of output + + TestHive.settings.synchronized { + val tmp = autoBroadcastJoinThreshold + + hql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""") + rdd = hql(query) + bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } + assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") + + val shj = rdd.queryExecution.sparkPlan.collect { case j: ShuffledHashJoin => j } + assert(shj.size === 1, + "ShuffledHashJoin should be planned when BroadcastHashJoin is turned off") + + hql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""") + } + + after() + } + + /** Tests for MetastoreRelation */ + val metastoreQuery = """SELECT * FROM src a JOIN src b ON a.key = 238 AND a.key = b.key""" + val metastoreAnswer = Seq.fill(4)((238, "val_238", 238, "val_238")) + mkTest( + () => (), + () => (), + metastoreQuery, + metastoreAnswer, + implicitly[ClassTag[MetastoreRelation]] + ) + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index b4dbf2b115799..6c8fe4b196dea 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -132,7 +132,7 @@ abstract class HiveComparisonTest answer: Seq[String]): Seq[String] = { def isSorted(plan: LogicalPlan): Boolean = plan match { - case _: Join | _: Aggregate | _: BaseRelation | _: Generate | _: Sample | _: Distinct => false + case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false case PhysicalOperation(_, _, Sort(_, _)) => true case _ => plan.children.iterator.exists(isSorted) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a022a1e2dc70e..50f85289fdad8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -21,7 +21,7 @@ import scala.util.Try import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.{SchemaRDD, Row} +import org.apache.spark.sql.{Row, SchemaRDD} case class TestData(a: Int, b: String) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 91ad59d7f82c0..3bfe49a760be5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -35,7 +35,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft override def beforeAll() { // write test data - ParquetTestData.writeFile + ParquetTestData.writeFile() testRDD = parquetFile(ParquetTestData.testDir.toString) testRDD.registerAsTable("testsource") } From 2c356665c986564482ccfb3f880f0a2c023a7cb7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 29 Jul 2014 17:52:48 -0700 Subject: [PATCH 0115/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #740 (close requested by 'rxin') Closes #647 (close requested by 'rxin') Closes #1383 (close requested by 'rxin') Closes #1485 (close requested by 'pwendell') Closes #693 (close requested by 'rxin') Closes #478 (close requested by 'JoshRosen') From 39b8193102ebf32ef6b40631a949318b281d44a1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 29 Jul 2014 18:14:20 -0700 Subject: [PATCH 0116/1492] [SPARK-2716][SQL] Don't check resolved for having filters. For queries like `... HAVING COUNT(*) > 9` the expression is always resolved since it contains no attributes. This was causing us to avoid doing the Having clause aggregation rewrite. Author: Michael Armbrust Closes #1640 from marmbrus/havingNoRef and squashes the following commits: 92d3901 [Michael Armbrust] Don't check resolved for having filters. --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- ...erences-0-d2de3ba23759d25ef77cdfbab72cbb63 | 136 ++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 3 + 3 files changed, 140 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/having no references-0-d2de3ba23759d25ef77cdfbab72cbb63 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 02bdb64f308a5..74c0104e5b17f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -159,7 +159,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool object UnresolvedHavingClauseAttributes extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) - if !filter.resolved && aggregate.resolved && containsAggregate(havingCondition) => { + if aggregate.resolved && containsAggregate(havingCondition) => { val evaluatedCondition = Alias(havingCondition, "havingCondition")() val aggExprsWithHaving = evaluatedCondition +: originalAggExprs diff --git a/sql/hive/src/test/resources/golden/having no references-0-d2de3ba23759d25ef77cdfbab72cbb63 b/sql/hive/src/test/resources/golden/having no references-0-d2de3ba23759d25ef77cdfbab72cbb63 new file mode 100644 index 0000000000000..3f2cab688ccc2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/having no references-0-d2de3ba23759d25ef77cdfbab72cbb63 @@ -0,0 +1,136 @@ +0 +5 +12 +15 +18 +24 +26 +35 +37 +42 +51 +58 +67 +70 +72 +76 +83 +84 +90 +95 +97 +98 +100 +103 +104 +113 +118 +119 +120 +125 +128 +129 +134 +137 +138 +146 +149 +152 +164 +165 +167 +169 +172 +174 +175 +176 +179 +187 +191 +193 +195 +197 +199 +200 +203 +205 +207 +208 +209 +213 +216 +217 +219 +221 +223 +224 +229 +230 +233 +237 +238 +239 +242 +255 +256 +265 +272 +273 +277 +278 +280 +281 +282 +288 +298 +307 +309 +311 +316 +317 +318 +321 +322 +325 +327 +331 +333 +342 +344 +348 +353 +367 +369 +382 +384 +395 +396 +397 +399 +401 +403 +404 +406 +409 +413 +414 +417 +424 +429 +430 +431 +438 +439 +454 +458 +459 +462 +463 +466 +468 +469 +478 +480 +489 +492 +498 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 50f85289fdad8..aadfd2e900151 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -30,6 +30,9 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("having no references", + "SELECT key FROM src GROUP BY key HAVING COUNT(*) > 1") + createQueryTest("boolean = number", """ |SELECT From 86534d0f5255362618c05a07b0171ec35c915822 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 29 Jul 2014 18:20:51 -0700 Subject: [PATCH 0117/1492] [SPARK-2631][SQL] Use SQLConf to configure in-memory columnar caching Author: Michael Armbrust Closes #1638 from marmbrus/cachedConfig and squashes the following commits: 2362082 [Michael Armbrust] Use SQLConf to configure in-memory columnar caching --- sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala | 4 ++++ sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 -- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index be8d4e15ec4b0..5d85a0fd4eebb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -22,6 +22,7 @@ import java.util.Properties import scala.collection.JavaConverters._ object SQLConf { + val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed" val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold" val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" @@ -49,6 +50,9 @@ trait SQLConf { /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? + /** When true tables cached using the in-memory columnar caching will be compressed. */ + private[spark] def useCompression: Boolean = get(COMPRESS_CACHED, "false").toBoolean + /** Number of partitions to use for shuffle operators. */ private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a136c7b3ffef5..c2bdef732372c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -192,8 +192,6 @@ class SQLContext(@transient val sparkContext: SparkContext) currentTable.logicalPlan case _ => - val useCompression = - sparkContext.conf.getBoolean("spark.sql.inMemoryColumnarStorage.compressed", false) InMemoryRelation(useCompression, executePlan(currentTable).executedPlan) } From 22649b6cde8e18f043f122bce46f446174d00f6c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 29 Jul 2014 19:02:06 -0700 Subject: [PATCH 0118/1492] [SPARK-2305] [PySpark] Update Py4J to version 0.8.2.1 Author: Josh Rosen Closes #1626 from JoshRosen/SPARK-2305 and squashes the following commits: 03fb283 [Josh Rosen] Update Py4J to version 0.8.2.1. --- LICENSE | 4 ++-- bin/pyspark | 2 +- bin/pyspark2.cmd | 2 +- core/pom.xml | 2 +- .../apache/spark/api/python/PythonUtils.scala | 2 +- python/lib/py4j-0.8.1-src.zip | Bin 37662 -> 0 bytes python/lib/py4j-0.8.2.1-src.zip | Bin 0 -> 37562 bytes sbin/spark-config.sh | 2 +- sbin/spark-executor | 2 +- 9 files changed, 8 insertions(+), 8 deletions(-) delete mode 100644 python/lib/py4j-0.8.1-src.zip create mode 100644 python/lib/py4j-0.8.2.1-src.zip diff --git a/LICENSE b/LICENSE index 65e1f480d9b14..76a3601c66918 100644 --- a/LICENSE +++ b/LICENSE @@ -272,7 +272,7 @@ SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. ======================================================================== -For Py4J (python/lib/py4j0.7.egg and files in assembly/lib/net/sf/py4j): +For Py4J (python/lib/py4j-0.8.2.1-src.zip) ======================================================================== Copyright (c) 2009-2011, Barthelemy Dagenais All rights reserved. @@ -532,7 +532,7 @@ The following components are provided under a BSD-style license. See project lin (New BSD license) Protocol Buffer Java API (org.spark-project.protobuf:protobuf-java:2.4.1-shaded - http://code.google.com/p/protobuf) (The BSD License) Fortran to Java ARPACK (net.sourceforge.f2j:arpack_combined_all:0.1 - http://f2j.sourceforge.net) (The BSD License) xmlenc Library (xmlenc:xmlenc:0.52 - http://xmlenc.sourceforge.net) - (The New BSD License) Py4J (net.sf.py4j:py4j:0.8.1 - http://py4j.sourceforge.net/) + (The New BSD License) Py4J (net.sf.py4j:py4j:0.8.2.1 - http://py4j.sourceforge.net/) (Two-clause BSD-style license) JUnit-Interface (com.novocode:junit-interface:0.10 - http://github.com/szeiger/junit-interface/) (ISC/BSD License) jbcrypt (org.mindrot:jbcrypt:0.3m - http://www.mindrot.org/) diff --git a/bin/pyspark b/bin/pyspark index 69b056fe28f2c..39a20e2a24a3c 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -52,7 +52,7 @@ export PYSPARK_PYTHON # Add the PySpark classes to the Python path: export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH +export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP=$PYTHONSTARTUP diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 0ef9eea95342e..2c4b08af8d4c3 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -45,7 +45,7 @@ rem Figure out which Python to use. if [%PYSPARK_PYTHON%] == [] set PYSPARK_PYTHON=python set PYTHONPATH=%FWDIR%python;%PYTHONPATH% -set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.1-src.zip;%PYTHONPATH% +set PYTHONPATH=%FWDIR%python\lib\py4j-0.8.2.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%FWDIR%python\pyspark\shell.py diff --git a/core/pom.xml b/core/pom.xml index a24743495b0e1..4f061099a477d 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -275,7 +275,7 @@ net.sf.py4j py4j - 0.8.1 + 0.8.2.1 diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 6d3e257c4d5df..52c70712eea3d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -29,7 +29,7 @@ private[spark] object PythonUtils { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.1-src.zip").mkString(File.separator) + pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.8.2.1-src.zip").mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) diff --git a/python/lib/py4j-0.8.1-src.zip b/python/lib/py4j-0.8.1-src.zip deleted file mode 100644 index 2069a328d1f2e6a94df057c6a3930048ae3f3832..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 37662 zcmb4qW00lml5Uw@R(IL9ZQHhO+qP|2mu=g&ZCkhZnK^f6pF0yXcSYnE>(7_#ja+X& zPf}I_7z7I7uTNqGf#5&B{QCnG029F0jnSM&Nf8nNc*I~>{`HSIxk3W~f}8*Y0Q}=2 z``^9}{m<7OjT{^u{zHka;eV+F`OmLH08DZd@;A3k22KCb9|R2mfbn-F=6cS0I#znN zI)CY;wsrf*$>`sYWdD+XIoYN1woL&my!UbS&1{Hoq5$NkR~hx(5#pM4b#ocBQj`e}AF+Qsz!IK%djA7H>1%Ch;U{;tu|W`SF9aXjcT zl;^-?i^*str1(K=^1}5Z765nW{w5InPGw{M`jK;zH~e|@G6vbQ6Y@gPeFyMyqV~$< z;S`V+a}(ml@^Uw+;%P@1)!NL)RppgRHb%d~sPpF{)u*|W^6TiRu(vNeuv$r6fmz&f zW1?oEd#1=siOQIaM6fMPaYl8nsc!}!#*uZT`tR}L4W()~lF4`BT8-2owFt^^SyI0Q z=zDi!QJ))ZH;|xv?=joZkcKjoDz#Px48PH#T^}znus&CaTh@vn{LSf5{EO)1$fhqf zo^w6$L8`cz=`iy$GB6l*a%Cdmv#7aqK=Gi^PGVwGQb;-BW)_bJDOIUwhZX8!0*oN$ zdgn%Hs!i%~Q!xa9q|)bDcN_&44`wzu#a9?iq#G$n;fX}@qkk#d2@4|WdA&^%KS81k z*yFptZ$T*2{l2JvLLwcC1Q04N+GEHLz(L{9%PjuL-kJ<8Y?MksFRuzBAkzSLT{uX- zoeV~`MeCfo2r#9FIZACrZ1681Mk+!=}aULvzNXL^eYp#FUxJL19RH+ z3UgA;mOiBb-GH{}^N5!t&y0KSGAYD+E1a|gXHVI08CxKv6o(2lLo6UAJL=gXX7O;a za#soIbnV2FxmN+QXpJpbs^}^)EV0=N-&)@^DTeEmFPh*Wv41Zy7L_ny3I%Jr9`Y+0iU!4K%Ayw0uV)2sUK;hXe7Tbn4$YSEHi&>au>e%jr`iC(k;U+;_A zzS@0}4UUFu=i)wj@(?@AbP3KlvaEZr4g99G4?M~g7clMHF-GLN*_(O@JNN}Ej5z8r zeXovwqXHin%7sn^ucGmCkMITub(APK`6{4t$V=FL8qtk@PLqI57Pt_IlOcEQ0_Ihh zU_%q98TAetPtpB8X%>WXBF`$10-xcydo5cX$heJ=3i$Gs}C1hyVZ%dH?`8 zf7ka9%rVh(G;+~%`y1{wakt3-CI5;$oafH#L$lwn$QY;KJ&F9Fv2`d}1=-~EDC3Kl zNMV^EMlKD1<3vT{Q z7>^DVqb)d8jYmb^`&A0Om}ugV+qfiQF|~l7*ES-No$y1owPBr%I92Ki?C zA$!e5cOASf)j>1{YPJG!ujp-y43K7CQDBA%mKW9)^Dl&k{HjRm9o92(An~ zvBe}D^`aO?ZzRe}BPNyQQq+pqGXmdzIJ!~kr|G4lE(raJA)7VwytF<*oJS64GKF9VETBXd z2n4hHIk`S%aFVkSkae~`h6feunlA7^uqoPNS1edh(~ZL4ksPTP#)%}n zPKA#U1LqSwEo>>#iMk@)dotg^xR^^Swft8Zv=K~(Xp`Ot00nP2QR(k^3HZHEw??*U z<=>5j;Y!xzTZHU*>`(j;Um?H4an%PteW=Syt~-pf(IWTcUY>Cucw-6>28N2JS=%n_ z!AVO!i%lFed{l)CsY1TT9VSchYG-`Kz!eh_OnyfI<6@xOV~L8V88n8^)(f&9{1BH( z*3VxrTLBx;Nt3^OPy#iZ8i^6{hT^>2#SvVlY>y0}kc5%rNo!l(Eoc*9%%jYyfI8uw zSK8^fnP)AHBQOB?;Oq7OjHCG#>52A*Ll6^(1Bo3!Is%6<6EX|QS@;3!9`(kcmpw)P z!($xtXbTM!3&~i@Ea43Rc|rJqu4zW9NJ)w(EcG0-tat{B6c`vs8FvjTvFg{vaBEl< z)&A*L-jcQNP*O9@Uh+raZ7Aa=7*zWXCwz)7EI)52YXolFO{jH7WST$<{D5Yv2`|9C zZ8QL2G=FHu&kRsIyfUk0&>M#H9igP7Zc-Mu3GLYZK?18?SH6L}0DOkNaJDwi-+dcA z@D3dY7f#89@9Kz3)hhyYHC3&K{uO&?KcTwEY*(fFj68q=XT-rO(Z2>Vz~%fww^rd{ zf@YRuledQA0&m;!gsX#_;K53sR$`j)XZ6UG#b&dJ)5pdzd=}2yd7Z_zYCr^UYln{r zZ8MF2)`cbWQx7z-Q#DQHJ+(ov;G>^*!7I;`_FsaIDk145L=GTkUb{v7Z)#W1!&vHV~VM(xxk^d33`o{LP|9 z9C1*kmuv-?g?T{SiH;6mKRq1JL0~Ig!2N;lo&QXWexmb!Cwn&=@sR{*1rDM)pC9th zW#p^XQNT$*AHBlSctV0ouRZc0RikE@wYqou*`e#! z)ODeo0gR5e?cI$nYY#il-`E#!FFcnSO5}?a6U`75do&YQEDiI*;up4$%l1Th)kF&J2P;8k+P&4GSm_S0`Tc>5m1%Y{Qr= z)^$`V&ClY~q9(#(4{V_+HFuj>lB(DpNN7n(5kOUpN2{xft*$^92~@cfh#X0pzZ;s= zIXhysrlVARUheYZw7Zp}?69*)YMDi1*RVnQ$9C&ME-^@4?XQ7)kd5R#q|qKtZOjr`-nf zSfqzZn|q`)^kFo0LMo0EN1X#N#9{T~WN_fe-lSHiWEM%oX^N~X%dW3k<9k#sDeU(p ziwO)Z5l;c=Nklst5SVzj#&xQ>7*_9~7A2p=erUE$BqbOWR)BKActViP)?0~wS1)m* z`T4k9p}}TwR~jlU5#Od-ZX!3P_%%&zop0$qD_z&H!RlzrAz09|S<-mie55z#7Zh5B+r z3$k~pB50ZuK>2Hswl9 zT{?G_k+TPE)b90IoU`9F1{1f3%X81Yh^E6H25jThCZ9W58O>zcL!vO^&0QCWA`t28 z!bHstmoaBYCdnk`T)K~O7T5nB2nT7B)Tc&1=#M{yI~sUsi`eCaj;XO2s+^ZE*tz3) zwu=7)n>yB_NX&6{S}0XUoRyy8OeS`bmM7PtI1w|sg;+~2DAH^wTNzR)-Pm;NeV}`R z&?+f_lnuSLeTHqh@4mOY6*_CLIZ$JijcQRCNTJpTlFdTG9&}!vMo)uVv!^GA`9!Vp zcSE*QB6{SBm=$<|5~0h;6{EUdTkrwbpor^Z>5!Jle zIdf2PtjRDdAGb)5tN4Zp_XggC8hPNwy`_rHY8h|ztjK&7!qC)0ERcMPSe?@+!sn1& z+H%5C_n)EqVqW7{H^CI$h7l#NvVmF--zBg-OVP(@qy3SuAM|8PV%T&0U>H2Tv8sGbyj;) zI2pNbhl0pCR)?N{1K|iC&V>csn3ca@tE>`|F2+WLFfm>pW<@bjJFcT%o7QahX*rmZ z;ZS#HCO?IZXF0MbMc*S;Q^yi$T!5Ti6((F-+<~F4@6yPWu_-_qO;M`4Bl)wd*w_i@ zy9r<0tpfMmVZjW$ifLC(*H4%QX1Z>G#>!91O(SOA!>X{GEh1cKFh34R?Hxn%DHAl8 zIjH>e$KQB7}kHJ zbvTkF^7Nq`Zg?94#i~T4<1>K2unW$}L!_VW(oHJ{{pKv+Ps0$D06Y_{AE>KP;Sia& z1qi$duLzl+j$!+GuMowqtLf3QGtQIxD)t3HLLa??OseWy6RxjpY`9pAS!};VfQ`Ya z`Em=-`92q<3)mm>QB+fzAs1lg)F@45V!iK#Ie_F7BA>&$NAQS7X%Mi@nhYwk%R7)r-T9Jfu)uHVwC}2Yk2w9GX9=a-;U^;AaY;z}}K+ zrTl#+zi<&-y4&zdyt^=HA4Rqh@R-IYlr~5nGfX}YH{vKvBtt3Tn2c!(M*@yX zRA-nT0q6l~e4q1P+(Z##51SC zQ&)GUjpt;dv$I!a*hY=sEZnFXnD4R39a|z@UaBPXesbknx?OVrR*U8?3z;xePV;9( zgayTPIa@}9%+Mjv#r%0i@6Ca^X;ILj#t~JQN-~f~`xqh<>G2T8Jg?G8NFKC0(sk@O z?hddwE_dIC<-P|`%;0+yv30}E3AqoNLCUu0!ZOHtQ`f|DU9?0V#e49xx;3e*=!$NR z^Sd=QNQ8maSAp_{qu$Y@BN66a{Fg9RqA;i&1|wVDiYy=y>%7%g?J=!$^YUj|VF5kV z1~r}Zh(YvN&vds&e8B2mWtY73BUi+pzZ)0@3DWRjvoLvK`B3#oq8FwoS%)a1snvCY zrvuP4k^HSaP4Nhg;Z98R*3)j-x)mDkMp-I;<0#zl0G0fRb0|g=vhfe{PkGn)({#?& zvqp_}x0X4DK7KwY33#bMB>zg83j6u*WyAQ3bmGutH$KODG!|ns%;OZXZY6Gi(!TIs zsOEvv5GAOt7nEIZ(N3Ws!kT=semfN_J@gz$+#im*OO;RcS;GzCsi8x03Eh_$7v;PK z?J!XdE&*(h^qUq+&eW$E2wD3+CBof@;&M>gSv zhT<9RuScF-Iq)!Dmj1QqJ&tUWi0TKfZMhQLX|*-I658o8C3S-#cX!>6kpl4|dGS=S z=%9xdf$>spy0^$nIYdGGF&|0?)$aFMcMeucdkvZs&#F!<%-UJowU=y8Uh7*3d%u;U zD`)1v!T+Opf(4fcsw*uMG4oE%o&bEdEx#%Jn;z{Y(C}dZlJ% zv(60vb*0nWfQsQ}ljQjN-^4p+<4Va5z{KYeB$-*xVZ`8%Xn{)l^AeDPSl7|B|fN>C)!feoT& zZG%hNW5PvmqSpwD;Q~4T*6co8PLiP$l&;<@Oj(UR&GF2h_WnI3jI3h-tsfFClhb4{ zQDuFy>he3`DiuoKpL+vbzdSKpjsr&JFp6A-Hyq>aa;W!(3w9HJiL+TkPy&dEK?DSD z!W*wvqeE0NNk)hC&%A9Q7pGe-cwQ0RD9q9vhgOhfDX=S;C~o|SC;Y)V0gTQ2_4^rAsrr<3Jkar1t!QJB*apGLP9)W#1VfRBY8!#e~#l_7(VL+*=k!5;ja1fN8 zRp5Gx_+)bRy?+)&YHCJjxTM6?o{|6fdOEn^_9s?%Tuj2OE6`j)Z6c1(Pgs!x;4)G^ zFDwMqpIJy-)d9WKfmm$DC^C_}oa}|0>ud|L{`+utNvvq>#GY7TANq(o0r_q zSDEmslK+imq4B&Yt?IV@QM7MX^s5IpS*dC)QmI1N@&IX;%FmSdgFC{Y6hsJ*QPyb_ zfIRb@Tr>^Y!AQ!Y0Awry(28!AOiJyYCL5CtVT1lLktk3a9-ikA5DHTmDY-0ZoN5~C z2!lHm3wpr1axy7zbHxWT4X#b)D2s4kS=UCiaudMKivKObz~}RU?S@9E1vJChn6y~) zeAQ7vQ=#XkjZwY`s*emmBZ0_-vTo8UO_sTpg325})SfIFE17$hb#D^B1i#>S?6G<| zQ1@Er7{ZXQUF~grTDZM^%x`H7Cl1@ya3y;JEx$W+&*d&kD~CMsaAa|cSJ5s_qZJa# z&V0U)ku*dL-#A+8Qgbq?hc#_L&g5IN+Eqz?TSeHixJejOrHkh1z>$;}B%{1-f5`-U zFheC46$_4G`J11SR_S7P-iGS&D8R6hOi2W#)tGA;qxilGF5!i3SxcXfd7S%?DK8es z>&sWo$-pPVKlflaYI0IH!ig}{q-0f;S(S*-Sv&gzZ{Wg??is9T3LWII-|vJS(Pigk z4ml*UAJUzbmH>rH4uS)dv+ZfuzHM8`LcG2q5Je!$8xB$hD23lDJ7F`$T9*NyReE&$dd%<)Mdv7M4|XLpsZb^Mk9 ziOM9?B*v;{?fLXUCKnk)^A0t^JpD)+Q+}X$#qH;k-P$H23YwVGbGq=rM3WRHGEuWN zJSpRqMELgg1E$w*NcG52KYe1Ul`C}YHsKOZFyvR(&ER81evR6iD)kptQjL2TK#1kd zc-O!}gaWhYSe;FI%^(Edo9tJ$M5djxn0xRci>S38Ih zoG~NVuyoD?!BlC@5;Iv!p;Kfb0tSl#GnH~>3swoO(md8ppNKNodk03qf1mR85z6cO zxblIRtm0j@R!C{T+#_vq7~x$nk;;g|l+qJj3L=7Te0dvXK?=qM22A{XTSSKos_fTI z?4(e>G8~Cc>j(S5lXqWHa+B+4(2yi!VvWSlk_D`$nFXLhGRZCe=Ehj_Z^U=>_<}RC` zI-SU6&%0qQfu~D~%@#@iB1-Pk2x0S)2m@%T{W$*NoWx*7)F;1+LHZ(j;Q+N-hX;9l zb=7-Zu6B20WU??2KtMq@d=xvWgZyUXbdSo-)b z+9vr|!+-2iNxq={2buoWS&FB+G5GjHrsre;f2j1|JK+D;!v04m$kNQg@oz->-?g9B zwHz{7k-eU)WwXF-BYs6*Ixp34tuor(*QOCGv}auzaz^SO{6r%_6-4}gPZ|gTh1hFS z+X$JI91?xbt|+J=zyk@-Z;%E8fvf|3>LFA2-Qs{r4ZhW=cqEA&fdVJ61qZIrs zaea2TPYZ4hi>NI8DeU6i70imUu+}IB0X0AKMDd;_Td+%O(vo8hiC z*H+&mep3j8)X`%tY{k)Hn4N&z<2MxXA@55|uU7>bp=5;cg{`TcGOQC_nX$jx3dY|e zsN51B8imuTTwzo|1OX}49F)B?^c?jXR9eU04e0Xwet_jT5a z+A%W|HE5{$G_ZGH8h_aZe?GPCL0q88owcZdqBIqcXc6o;dzJBi`MriogN2*`Zvz3EK8{f$vj<7OkCvu}X*Zp}@=) zvX>M32^1HuiWFW+!gJ2_nD5&qv9DGk>i>5+?q5AhPAMS$!X=UiHWpH5-vxC zJ0$@eRR6+zLrQ6cBH}%$)BX^0w}oMN5MU*WlV;X+w)qPN27c9DL(wV%lBW2C zeX1xL{4&3ffjpR@aApni-Sa@d!^4BwP2NtgEzbvnvtMvge{*s!{z!nj>>lfMdXN!B z<4y{>bfnk%q$y^nW2(klV#mnOF!bUH^ktPV5fjX=BJ*Bqc=*RJMXn6sq=gHlyr1zhk6HIgXh<4ncxSJz zzJD?YeHMazex9?{LAL8wkh*6&QK3VYl1ezAUvkTnj80o(o6n;+Q8bxigY zQh<)3<5Y2_xl_)kE1d-?Y+2|5O@ z>cT3SJzNO`Mk{?-?OEsIqdtWq9m9M|j-QPM1A(fTey37!aGoizn}p9%P*Frg27BC{ zm7KM?d00umu`)tJIy%7;4opR;m=uh$rD_U8SuLIYkg04XLs%J^J`Z!F|D`o`gqJ79 zxZRL7B$kR&F$HETvuZCmh$>87P>In)11~qAPA&$Xwq)8HHe4ClGVkQA70 zHRbR<0Dy2vGupkjpx2rU#xxMtO@={SSQE5hE;CCmHFKu)n(UeF=W8KwS54B$H53Qf zA+S!m#YAS6w;IFV=l(ihq7Jfdg=FgJsgifrTj&l{S_{Y1hX+kLY|qDohk_z9fsYlb zO^v3MM`!S#%I;7J&h%qqWWkHRE72-}MTa1&8yPi^@_7^`@*8Q1FlOehdaC@QKj`w; z_fhLNLMP@Hw1iwZ!Q^15u8id%l8!Ljst2{IcvB|X9Ggo9JF9&qF4#I!(TjAjN``2* zG5SLxRa``m>}rJgo@=X(LmoSBDr(N2>^n?vRrHLHLKnB5wX>J6M*3p0RFTQ+9|iW* ze@ium&l$uH!SXpXkaxOuB}@>tRyL52<`OvoV&}qm^WGUBHVIh9Br zRoeR0c4c;d>oTcze19jK;tR;oX{B}-4k8uoM`%&?H(W5~at*}mOyJqExn0&V40?%h zgJ)-++gpx};k2>z`3McLsUDEk7KLwMnqgQ$MTKvZ6UKcd4FoGRI+?v;Y;3CVg_aV|=oTs|QIw?1gy; zNg!^sQo}fLO;>@J@%8HAIyfSiA| zDqttI+EF?%x{c`&6pTttP3JNn>w?eP_XgoC`kdg{dVK*vOuRARIPlHKK&3x0T*FL=(1LVU6S)*n}HC$psrvOC_0LwN$t zxeveF9UQjgJ{WgUlC{w37rWvr-D=GpJ^A3?HdWv}HYaL@)fGs$A&B{~HuM_Md$W*n zokzo$fk051YtIBNKApM&|3*H|_^HzLu~keaKV|>;shLF9Yee4lhIn-_`~?ihZAfmO z)a5Wk?d;USTjJPkeavy7!R_h2Ih#pTrDK5-6)&ePAjKmJ{xk?SZ5jG(o(atSb5)!R<_<=x9%Rpuc3P&X7RG+lh1ucicBI=F#M;e>I2eVZxJ7GL#u)o2hwH$}lI zJlgGOYd=?ACc%fXTW1&7AaA-iKo)uNi7clKgis3C*_BM`b*Rf5%ipiz-kP1b*PpS% z!5;tsSbx8UR!)|VW{#%zMtX+-i^0N0m38ZNR(P*vm71#n!X6ZR4aeCaKPN0Ev$F-h z-^gmnfW4vyA!I~Qgog`0-2!rTMl4e4Lg7}RE2yJv4r~nRabDZc!ceJxkt6<+d4%oN z?}j!eF2na<#!0PlXE3eCV;jD{o~N%6sbpzL;wDLD$g<+D+Tuo{N;iteyr3+(sWV=E zRgK6=fMi7ECOwE<)49ySeMO9T8fFaB*C)U9&%G_lv>URAOH%qX; zfZG~1sNt*&6px4)sFZ8;DfM64?a%pT8R+{~Mm`92(M`DKE0 zf*&kXF+H+VdLvB(IOSi)?&JX$b5V2sZ=ytQ)I)w1Nrvt?OrKpnid`$I?@br;Z&Y=f zI-Zg`v-H;?a~cJY=ewY6StKVBSy47N>BNtZOW3pG;gCSOv9R%kBgIROggo!liA5XO zJ9GaUoF{YpuF@vHW#l_`RUen35iVfm{*97kI8#i+Ti`$&@c!q-wu(8R)8c=D===%Hn)_h?KJm)F8tkff(VI zx5DZGiM`SfA}y1HTLqe|0x|`Cda;l$_mYB~3M20^!I6?YRN5fAH7}o%abJmwI3L$# z-ul#0>eloNLQtc1blu95RKVM!1fNXsUB$BL?<N2#3(@Jt1Zu_s z&V+)_O#0~TAQrS}MiTwwOQo^*Y@`PhTPt=!T2`9ZpiXdCLF;u04qrp2UaA2wIi7wlUfewHq zIb&{qH9Oveab0DS#<$xpr3Mli*739^{SDE^<1bl?I=p5NZ};Z(j+McWND|937dg90 z&9{skFv_{JjcN|o4onGOzz@oI_^!CjD)q;-z`s|Kn+PB=RLRP`#qZ#}I}bcIV5|HOpM3gsdEFC_BGD3`(p9FPsC>%4 z`(y%XqM$M2uAjqf`pb11tP0Q_tUfBx8_f5VQPf}mC0P?Z@T&#-G~aiLaiq&6fApio0-C+JPf*LDnh+Qebi|1u{3h#Em*KN;u_47(d0 zxq^<5D%#&SkrXPxe@d=Y1|Tj5Ph2ZY809b20Wt;*>Th=n|3q$(O`glvH^R`LSb>Xh z3sm?czfprTnK9;djx?FxOA3~N9HMt+SrN)mEbC%wd()*!&oQdgYPE{6b9hAxyByD{ zn6pO#t32_8y}RT&AP4x3^~MIKU$rk>G)N)uCL{^}! z+!{Hle&*nXrJ->}m{XkT4~!k>H2S-$92@Sv6??bwd?Jx@L^dT?{66|MbEWkHqgHoJ zl3*RbMh>k}K*aV-ZAMXXm8bLIfdg|`sMMc>>!p~3c_56xAp#)gFZKtjJS7(8kA0|1 zl_kpk;7N1kew)g&XdVZPhG9wV;mW%cVC!o9Y*dak>w+~##Y7Kv7`#17LLa%pyQ7wu zQY{9c0^xV-Rv5a>7GM>0)-G)obJ}r(S+PR#!VyyM)SkA7x~#wY067%`JfC8}M>k3e z|5S~ntC%IuznwgUgk(ScCe*}Ef$vQ=wxd++>OPy~IX*qQFT0bTtlPhMeKwLnI%FA@ zqsUV+{em=2J@rCrd$HG{XBa(J{ls%(8;>Ewha*q)!ZB3xzZ3`AvaK{8^f_2^Vgs36 z=A07|Ru^Mun(G^OLMhon!`ZEW(p+DIP?k*hd8Q$fLdp5KKSQK1&-7#+<-ffONrfi-;G;V@vcVu>j~O&1-QuGTI?V0kJ%C%nYaqp!PEQvyF`GL_pH3<1J>-SG!j1whTto*-1< zzu*+C6PFgmlRg<<#=Yr8QlceaDWFw{bS9s`ZsIHtVIQ; z(>>j*T4{ZuT_=KRlStF8$*e-lo1r7{-(wDRrJSdUKV}sAV@CAqx2`$xz zQaA)IgsA?29A5n}wQ*%?30t%>);OK5*zE_kRA(4R2v$+)Y6v6LzjLsx>wu)$5SkoMq~ zvX7KE!SRv)P2FXC;+UX5&sPa2eC12A0VLI2(uf9>Hi3;b6i}yyvw5TB6Px#VmYrhp>xagj z>a`Wyq+b=QrrCRNGHs&y>N=q2CU`;1x|BdX1G#Ii?ooL=QFJ(I4phkY+=OtfD*WbC z0bx#8RGq&m8>Uhw9cCHRMl7u3qD4=$*)=1n8NPegyVSFo+Z2Ay!J88ga}zvvFv+*4 z22w}1rKrpL!2ZhCL&_!c`8k!^&G0Px9gr>eG}GFb8T^s+_dlub{hbi>uYDhc|ED5q zY-X)zY36QZ|G&xJ7F9Ma{$y`H&pLQ*AuE)rE68AeJ>pH=KiYbixT5e3UpLs3DkbC5 z+Pb|-eAXY=3_qU)^v|Wc+ut`sE>>{Tj2HZ2ch@Q67O&++7Eti7nwu<9OgnQ1jW$E#kTCs6|b4cHoKmbjO-3`z+f|!MRDG7 zYH#fg-_@O+Mf7oyM-^0$Ylas`S(QI~x*oZj@6@6credEos&idUG7Me;c<;s2Ygl5F zcVD)5J?amu5IA#nz-)-up3q6Ylq7^*rL5=SQkDs?!IvIkr4;o5dK@1V$pz^bmZb6* z>ut}Cbxz#iJH=ZuduG*>lgFC(VM;6d5r6fHWdeq#O@8d@BcgDY9?P}$YR3)4J`DO+ z;gVRgM-)WaSOS|sB`bzl;)l_r|| zr>gMxeE|Q%wbm%6_{k@DHue9wRyyo|z3S#*^!HxF$B%ZozvN%o^-@JDW=|Nt^Iat| z6}V}jY3RZjjshLFAT@$BT zyVF{rdEH4EDyWyJ5YiI?>h0QNru(TH>sTZ%L#mv+M=`IDD3Ei4n{xiQU7y*+`=n$y2J6E9{v36l>L5Tw!uv_zcO%S_+)7yQ_WQ zrC%rb!~5sf6K@2Hedp*Ty_Y!Vu-YC}171Aq7xkXxoPZ*SV%2jm;2UOMeNgP&=TdDE zJJ(5vn&HFZh|a)@ObI~qbeWngb+8(7K@hE6x?`6|<)KjWt=Aw_KME%|cb_X1kBq`kHem+S z^G%8@wc80Q7br1ONW(xkJWwh3)C|W&;sa~`z4glCkqc`=#;aa5tbDf-%_w`Jp^9+) zw5sJ&h_3+qu8pd zxjQ{zn)Tv8bBd~!fy|9+SxRebQx+$y&S+@EVTZ)=7s@A-Kcp(-x76J?uR1)8?=6!3 z<)A)}o_#&LSdI>iq4~OB0JuvW?B_u^-S>H(L>@xQ3qHYjmD65`MQU z(1*t76N%~e5bfjqs$Mymv%#iYTEVI7Rii+I@|x4nV~06_R>*z$k5@P7zWE>)~7 z7I@*mvUTVd;I>A!WDyd1dzHx`x_p;p@RvIc?N^=aF`QXCv-f;loLO24pXD`#kgd42 zi0|Jf7#KFVQn^$4%jv1nAU!#`(zqSsPj+^2-3#i}aaa7+Mre-_VQ5)*;~~F5nF{@X z0i#P574AC7PbL*VgR-D9v4IdX%<&VS5{+r}le*`{sZH>Tx%M-1X7%ic#9#}{^_gps zX7$j`H}U!%LTT55iVsD}kS&)7u2NQ8RoXEFYb%RuloFL7+7h#Z8uv;1PDUX%4yw(B z`WI1=)=)Gud48)Gp-m2(KOS&OE2+t~PMX-Oq?3m87nN+i0gu~7^@>hT(>3lB0=bj+ z6R@u5IfHHRHN%@v2?l6bK>$=Nx%MKif_5I3$`uD1Sy*;lBgD`Zw=pj!Z=e!NrNKKS zYe(*nROFx{(}C=9D{DimqLM;7(Cg2o^Vu#90w*c#<0_(llxUVT>b6B#>=FOiT7nnv zBBd1VAdFypn}CGOR&c4I>#||5=7wFg^-BtR9kckQ+*B^rNix9Mp>ITFyXVOyh8fh; zwPYYOyq|qw$`4g6+c?5YE`e9GCY*}g%0zlcys!lZ?kol4b(06-#DYy)N!-B)zlgaX zR|Hxb9%|kiUt7@lyv~FPyL-+;bI%;&S6pB><8igFjxQyC1j?AvQ)k)aOe?+#abvkB zktRn5!-o;yWz%L2W(6l1Z@7$nmR+cyomQ3q|4ku&UxQDGt zy4&OwC_?jsif-Wp*)e98?igi)be+orDCx4;&%rN3)0#J{EdM)LceXtDl8e;m1-_{E z_l0?aWheZlGnV^JkF=BuC69xaxGYbPbkg-$aneCCr) zc?7BFoQr~H!5+-$?ie;2OptN2rWdFg`;*p{Sff_4m91IJ+7s;@uQFSsdKy`~M?sdJ zZ+XRtUKM0c0t6(s=nZ0uW{tU%dQaP9-(R#}F3Z(A%M*2|Mu zOK{dp*u@w4qf!uib26M537bCH8`PF4>~{SXJUwOGV(ymlIR57Pje_>BON3rP??TcTuG6SXCiz4#%;~ovTgKy?AN*>fZX5uxZ+xrUeYXvk zR{@`VG=qnpi$20hRn=8hRYgUGjv%-WmSBQ@`k_n6?BvO0FV9{^=jZ2Vy?$GIL@9MX zWwlV6^w|kn#dX2nQ$gwiN=MY(Nyr0T)RvJ^r}*)#8hh)sJ;LOeK0=X4%6Z$!XUy%^ zR(r$5+4!5#4=aUZ40s)RWSlD z7ed0x^ykTW3eC5f%{8so;WziU@vDP-?K$hKNQO=mF%S?tt9Xq4Q^sRsDDzD zS2bzMM1+;gE}s+SDwgvn@y5QX(7C}gsZ=O8kJqJ!inBPYQ3`FvP|MJSwDqYo%WwDR z;^vk15xCFBRBZuCDt(9*!WE@nk_~jd{_H=Y*2_2Yqo2GMeY$;4O~$iZAx`^b`wLe&WfL zE20WHiWPgN(6)7bdD+?Jc5m#Oq?z-z4sY&t zf0y&+eQo%1baHe_%Er#timt8Ge?C=F{f*}h`Tbm8(5~3lxkT21NHNg9X;dB;J{Hf zz!%^HVpqhi4IZV3H#isvHpMKB8h`N%Z>D0V8@_Mei@BA=ddx7@zTPL^b>PP!<2d}! z#;VZkJ#HsP?Zs72?+hg;IJsu`d@(s5=f>Zm-Dhb>i=i{!@!s*HoWRmh{Z%*7NY6+~ z!beaKkb}Nr-5gk7AoDKH&W_&knK)T61w4D_2M<3+iCVkdF1*|mYF(O;IY`N;^O>|eha#2vm3Gg8>uoo@F${U5^4X-O0yx}sy-wr$_B zZQHhO+qP}nwr$+8J(HV-)6m?vJwK6uT@B>Xw*62JD@;w zV==ZU6!m*ZRo9h~KH3Ar1wh}13n0_z3WMC*hlw7|dY$5j0V(Y-QdP{W+t-YYb3G2( zk;YoETfX%BY*V5D4-l5W1K9IQ!Gu+9%K&#Z_io#x=_>r{n$=*EDuX8B;bVkKxEEw0 z!pSW>)no+$m8fD17cj~kbWl(!gOl~9`L2LV z&?^FGJz^@r_W~iJ0TKqrX3PP}Xfd!G(F&ks7eG7c)$qukvQ1b}6?L-Y&&iJL&vIC? zutnD~C`yBbvDTWY9GLmn-j!1;0j z8VlVj3qqm5HR5d`u$t=1S#@Y_{ol}J@It>S6!=tZo6bRLr`-c#b?t<39uk9fqHjL3o(}-kG1{!GjG zJ*>w=IVY@kFyNz8^E>>6Y7wN&5}?b6e2K)^GKb&1vV06B-@rQKrt0|N1xx|`dA0qq z`CODkj=Hsw3*YdT!`4WPs3u#QcsMxx4Z!LfZ6@~w4?!laD4%edl@#T0!#XD9>LPJv zhpsMbB4iP!H#uZdjXWx1*9EGRfys;vnGA5oz7=vl47z0_m3aBAqI`|nKdVhnftO|h zWb9UH(}pwJ3HMW~*6iqplQadIv@K9?gGa@vMM((7fSNx#!nCH+@@R4rF@HHHB?i%G zCJ2?C^*81e?t*k@cLRNe>yS{cL<_Yc>pzBz^h2^G<8}YUjF!G}B#xHp;}H_Zo@nA8 z4CVqRAR(J_GHA_MJc}9CuR@QSG6-3aiKJ#`2fp*%BZ6I8JeK}B1shI$gtYG^s#~H_ zi;l^xx5E1BAo?kJ19w9qD@^`&dkfP*v6Bt6lk+~w)bnL0+3_s0w6Tvv&Iis$*P39% z@J+?L^52u7RGm@?0C;>O;>6DF0c%od9h&Aq9ByjYzWttpzxJ+4LU-gausGOe5?!dP z``J{0p1DZAaOpD%CNY7|N2}5)eN-j&9WsCVqr1Y!Jd2cIRk7Kc`hj?0kh*y?HR;9m$K@~#msfOp6hesZ8X2pg5ipdVHdqR+EsnTz7(;qI739eWX7vmlsjmO+1HRU)Hj`~ za2Y@~CVn!(1-7+**2L4=uPu$rLULOi!U!OU9%Ya2Tcy^0aYBJbq7O+#R9j9mboBll zvG!v{=Zq>;YKJLt(A3UR`l@OaB?jQ}{Vb{zvq|%g7P5+(dA_7J28^ys)^?d2A}yPw z^F&ekDGp?Fac5d{xQJ%N?b&JJ1&C^dmA2w^7n1^ctztthyhZQK0-^Fj#*p5C5dM?p z%7BCxa!ojZ`r0vTDp(cZRa~El;R3du%E|7#NDU%qy^G!o6M=)S{8f%IkYKuujI(PsK$_QK}tee0+4q4d?#m z<4|=TprS9Zo;?j0Dlb>YOQgRw(89D3`nVLOXCkDXZ+hE)EXu$1Z>ZwIy=upqXGD=# z3z5K|;BEbp%tKO39h(LUTF-NB^`owZh*3!R_4M6m0c&a$c5Tv zeO%7+S8Hr`z?Ihlv8?AzVf)nO+}v#ldpVXD$bL5RnEnVsA=i;8hbn9t{uR29`jN|& z=~50ERjqx-RoqioE<+wpcHclU;$Vv31aQ`=E)45-H%w#xqDqN)xq)eeI0KsDNR@6R zSG5LeiJOMp06y|Z1ktxL<)`UaQ@0Tk)#L}C%0~YIr9zP-;QUYwkvij7$M91KqC$5` zp`E}_%fB>17R+Iz8!0>C$ujP5EuLzKH!Z^n-eRTn2~T{O$Gc=yt9G_lk8fGzc66V? zae3N&AcI!Ju$|3_*c)`}af}u#iiZ;B(4a-LiYz-vYmFyM41CbYAwdb8hxj93vIto* z!6bX&3YV`h4pyg0JswjxBcCL!k1*dliu}+xso1YBU?_zS`qgfuoRp*5Bmy&f&7Q%q z(l}lPBZ0xb;D29?0c|j&T0(;*RpXo*NOZp3&Jyp3uiW3Ckxr{lsZNND`|HGjdi>2g zk47{@-K6HE?~PsjB0Wz#s{j(U(mk7sAuKX;5NM3+5r8f07PwVuuLOY$f1Yku*GUk4 zyh>6{?!U4MZ-h~9&jv8E-{*rO8c}?}xCv$Da!ZBQt%l-;|D&Yp#h}Twf?j8L9v`Q! zdsfjR_rVppF>uTIL4G9lth6DLpWTF$=dRHB;}{lSo{uu)HK>*<>vc(aSW zxON%WyLgub+hj`gr@tbek;qisoEaKckAb*wnZ*jsa4u|T+kOXV1qdw*O?H?j#G?Pc zL?9d+HbcO|dU~TuqJ~6bP?Z4kb=d4^GF5MB^UgRf)gBG+=MDdF&m1^m>grFvqo1BAHcw+`>;k;d!z*W+10IECnyt;>Y5sK~bFh_KOESB+Ln%6swh zPLKUX3BKugg4d(X*zHDjw@mfM@qQn;w^}D&*wtWGhsV7YDz_Scn+GK<)VYR7h*dVf z#}zcs3pf4fR)9}*^f_g%;wH_nV$_ zCTqSu*pR2Kiyx&&X15da6u9Ipp#$ z-}f6hp-3OhcFxcNfuQ#F&j;9qQ7tB8oSueiB~-&8hl`@{cXKK>ns#Q3t0-76RtpeZ z>BW@ho5>XgirR70FOigEl`s>VB^Mqkp!c|f4XpnCa^P7#S6NDOMledfFnXrIGQ*SE{fu&4%l3xpwY+oq|co52*I`t=V(N`5SH*KjcCc>5xVB4 zxVHtW0PzSqKu|)qt0?9a2C^w&Fc+z)EJ`b5%PKO86&JugKtK5RFYC|ysYB2F0FU|7 z6+iGdu1e!*PD0wcEjmQqO!2DR8A=8%H)Z{l;Egrl5ttC=$0*|E<#YE}BkqyrMm`Ytb6wibrgEU`c&sU)6nKoZaN$csA$~lT8iYv1z*T+(I=Q z(bPS`U@v#gnNPrx*|Lh9Lyl9vY!EeUVwuW zk#=nr!{tk1i{i3w>FB0roLs#tb|qM4d#mLQp=_%LOT^IQm^4_Qw4Td5Ms6p%>b8k} z){A+IHs)K-|M~Hyh1mxCBBJh->R?5qiM@V?(UvM=6|`x-Ey%r-FEq(Vdn6V>sg^uA zFdfpEGu=)%F0j%&+NKzj>pZdq$P43y&fXwL=gTG6%Z&x=$K}W_YZNckez~Qw(iG%( z!jbcN}+u6|UdTYU(t zji?bv@#oek_%{t>{3amOsLd%1dzL{SJL|YW6YMYlh-l!NLb0rPaLab}5X3D!ysnRZ zRaXy(Blyg9NHo#4htz@BgO+NwZz&Fl%xJd$k|Wg2l7A54_$9F0FMlLm>hoq0DYp}K zzJys@&>M=)a?+iPT1ss22%ed~kAbjt*%C{hrjNrLi($*LKEeTlUu`302pYoaXRf-b zQSSYEN=)-RK9(kbn0#WU=*TIZX!yNw!S&XLgw#4znNjO{4afAU>SJQII_p^ZVnLlP zT9mrT+RL_$9P%hzY!Umst}c(oN?JC6w#?1r!w(tM5<3RLl%2|W`A9ECHX3im_!@;` zDRlCp1$#R@S#H6M=m~`qXnylx&ezZwnj2YUA;uaYpgo_t{uTI9-wqKOjhQ|cOcvJ1 zy2pNf#|jB}{@zffQR{wdq0u7j>DMItP-I7Cw-kyfSQbtzL_D zSrFL0-LgLOHs!if(PzociS^P1!j>mAsox+Y#`~+r4RakeE&|>nb(5@(ru2+Bs zCyb{CW3;wsYn5uiPy4L2tNw}IJ0(4cH3tq~$p+@^b94Yt=pRlwrW*s50KXcZ#l|2t zIm&}Uh<1kRFu&`nJm;)xr4T^cx$)d*1r!BvoN8nBZDqFro~q$j(hPI?X|->J$Q@)|wG-eGpOtk5rWS^g@X#U0NuRK9raI@)LZa-mfLty+C!) zX{6uNOt9}~p}Ut3Ch%_gIb7O+M9pq!(5maCELo7e(TqyA47jdTSdG9-k_2C2kt3EQ zg(=?XAXIu7qB6N=%?WW@#~GrP75h4}pZ^yCMJ+V`mnw-6WGsE1jeC&CG>*#_bbeuz z-EqJDMOi_wiw_Yjh2T#0p`B0IOvhjnJS?-Hjr8GkzqvxQp!M?4573Nmz40YX$3f@q zZExnhEW<)2mOB}S&5H1`MFy&?Avzdaz=a5QCs8ODt*Y|_SaMhyQ&kIfLZR3RXW}|* za*l#e+|G~xMMZSM7#o6IJbRi96*~fU*kv~0e#FeQ$0)-Jhq;II*dq!__KcXH4p_aD zB0yeEDEuccIVFEoX~U<3p11E&f}YiQ*YQpBquIfdGEQE}w)II=KtakQ=5T^*z|P8L z@ceIK1|(;;w(THqd4M0(K;W&1_dSiinu*qx;>~&5QZvB?1%xz%?P7&k9 zuKdP~1pbwa?C4UrA=$fLVA+q%>P*$e(r9ZRjGvWaI?2N;sPx!HRmR;bKPMrr(SPym zNZgJ*=~p^WtP~^!YI~jcOF{zr%)%G_u{n09DG0E~^F^w6rXU2|m=SP^zT@@BL(+H% zJw(5x&BAS_pW&_Q06XKykvMJ4aQK+iYdckY!TG(d$>aIJsl9uqXP%ot!~MzTNzxRq zf3JTbp-N{D2Oqw!4!bge<|z^?2UcuPFPo$&k3OuRpXImxkCdNmwfi3~tQwEM zSxBTghWSk&TR^CL)ohfA3m@jrTP8$y_Wi6T-ZY3BI8#I)bj%QI{*<4WzMM z<~2q27MVp3>kAA*niCGG8}?ryWX3>1&M$<~V<|~)%5~#8uhrD&t|qq(cgwe$1iptd z#!v@TIjY+2h=s>GGk(z`&L|)L^?ISm-h^pI4{wBG+ip8i& z!_<`*n@fZ(LlIdfdfyQtq*K$mrN1h`i6$J{jyxA1luerRL*8xOoS+OTkK6ZIP zyamOL8gMVZqh6$VI7X69Vs|W_g;efyp^)vOf+NlFpm^pA8@@c^%uuK8S~E+DM(M@E z@b7>%g!t-5uP~4gFr}|1oZVJQn1c}fRuTiRCGB;t>1-Nx$^4{B%E3E@0MamQaBdhVGOah$x_YNsdP6?(Rz1`rzg=zXH`steA zBow7H#oi6;E|%A@zNoDHRWOJY1XmcE?DXA|#0gs4B4e2r`{%;=LoBJ6!SdKs(*ea$ zx2v_mHfJZXkcXtePx2N*fyxd$?OV;`@2y19WF@(c;VC%f^dkkr##!<(5rUPTWUnX@ zVboo~5Yd6XccMt(5ptBL{D7aLTS!ZWATlmqMuq6owGd>O zWkd!PoyDNDm8e-PLJB&8gkJK~rkU*&AH;LPlS^U~GLf2OfBoy4R7#j-uWG@?C+tk} zjl=qy9@2zY+v7ehWGX7aYtT2_Q6nr!sa(Vd5g!^sJq$yGG?%#pUTH|v$5uON1)e3o zcN%+kJ6mpYoBG>#mofyLqm3ADv^=Uw0k|GRgKv%itpFUZFEWvrbO{2gv z=d2az$JXy7Iw7Zxy9c5js=B-+TkSL#Gv=eZgUUkwS&!Wmc&bvEbOLFR)z1;Y7NkpL z`mzh#VCj>0#q4vQ9Bdyn8Np?QfG<(Xe`%LXT`~llHHo51Q&$&!L>Y^?>W4O+j+;x` zW0$~{LggDeIm@U)O`EI*B?TYjbL5i4yf=l=_t)e4!08 zaB@fKl~<1~Q%OH%c_2#QSmw`57SK z8V8#yyIBR9TUpdp>&Y}nKFW!>yRR4%o@j3+sG!r~5k8AC^I-(yZ4(E zv+k34OA?rrL~pb0RC*p&8}G*#)R}E~qR73G)@Dn|M|A$-lPzy(mG^kqa-aYaTE-XUlrt(=GhT`uFwv*76( zp+yxC)g71b(r23-<;{VHq1V7RFB8im*4OWXU+{(+rS9Mrb(^hTL1Q4ES*tZt4&k%| z;IG$j6G-bZ!zi6??Jh~#K&>g4477M2+5&^eUlqW2!d_bE1VqkTQWzrxHshcH89EyN zwu?=@J-+{Ypd0P&WCIWq^tt6q>m3O#y>UwTlwet`yGc1P7Ip4Ucel7`qFgNr@Fnh+ zvWQ58Bh(Uyu-`tH4fMoOS@lLUi0y35Ch)d;54W8-s9tf|*B@By1t+mrtd)|x?2|=c z9xDx)pjM@b5h1k-H`4S_6CQ;UM|9$hjixdW#+?a6AV-_C+Bt!MFKINl!023(ukm*; z0*Ag`U=8)6X7Nhr4z%rIlFBiJa=GGy&ccLZxS$FPqXMZ7DAG)1!z$8V7ST@t87MV2 zhg??&4-zZPT_ELTn}4XBO0wgSLiz!6qdRYlCV z=eER-TjAIp@3t`kQ*C@}3e4V?Ac|ZP!a4(|k1LGWULv+;;rbXCw8uPq*n|<1LoBEx z5aX0(et_@DuY;{s)0&!*wxCwZ*&rsH;;cd$=6RB~PWy1p5idL|@-*+d&JeI^N_?xe z-XTu&sf~9RVvER~pX*Yv;WDh&*5pXQoY*#AgqAXsH}P^6mc^xP)xV+!b|7X5jJc zd1?l|uan!*Q9z|H5X`VQ&}3Bx9d7=Qf3YqE+^aY?iH`@L40;Rx!sWJSxTHyQ!cO5P zH@gU$K0V$>_=CENYMKnV*xLkyuS`~xL$i33&BMg>9yNWj;97y=!W(ZaYeOUnoH(bZ znPGYETGS;$L%l8)5hLx2zE>FW8J>k0JuXNSnb>GN*CZl(oy-L9sj%+@Zgq0RW*P1z zCSY&2R7i2gR-^F7M%yfS4R|vv2^gAuZT_kWxsc+-p5eudUiPayo#yy}uQczTO1b-$ zFs1VcgG4{tch^iM(L+Wn>PACOL(p?j_K9>KMj%Hn0h z;BS;TrMFg8xO;P0X4KGF zGYsA0MF1jEiI`b!Y(TL^z9<%Wh6jqy3^wlzoQPeV0pMlpH&M)->W$owSxpS~QC2OMPP zvchHndw;z~eXRh{qromXiK^aBC9Sm3({(Sh8=9Ek4ish03w(Dkz-~Dx@ye6EN94+3 zs-fGoIy*;j;iUgIM&JFC-<#x6hz@Ydsl6ZTVizTOkUGKi3A6qK?|Sk7mD?Im7kU&i zQ6@UDyMFd6_ZDPG@-6p`rSY;~shW?1oym22rq0C}6j0_wtRT2ky3rE*a70^~%5=t; zk8MV7MqAPD6#0b-E$QGD73@SyqP02b+RNPjy0jy~A@VYogcyZWUJd=Uz$&>UZdXPG z;JQQ!hpoYqmhUt0Z>X6){2i88KpEu7y_^(00o4Q|(alOQ0KmqNz{}I67MwAsBf-o$ zM+)Z|i+qsu)N!nJ0_oeBlnyQ|wW})860uEkM%$JKB8WZSA1_pG7T}9id6A6^7BU85 zOimqN6)9P%8W7QV$Y%w-rI8PM|5&zHKr9epR_^fY2E0+>@biN@x# z5GMwGr$EI+g2PU^RN=PMU$MHDD zqPMonw8D+tWHivzojcefxyZrB``3jg4L1j1T;;y6BO_{aN=@r>@C3W`Y+T|rtKnKd z*4Zqe)NH?@p(yzU6mxwcFB9rTb`gkSo0B^#8D2xd7Hifk>Nl zgs~$0R<9f9%IZku3Dxb(VQ62hhs#5WEsTBtKpoaE&`V97cCUMuz74p#BWrgD-r=Sl zmz$o=4DeO6dsYB$y^!q}eR6NEDdREs_hH804BolL=bFyJ#6un`ABkx3#}y_`+TT2G z!p19p)5%1QMGg=E@j`T}9p^ye`{u<;1I=WQ06K8j+v+938N(=rMycd1st%R>y0W8Em=8mgZs;IEPe# z9+UU#bJyj0VMlB8RSq8q;QFp~cf&3}NHT5H{Rc-^As~j0e~8RwDh>3^)s}3R+cev} z)%^AIm9yO&K?oU&E8t?W#%B0lXZP2n^?Kk}pXoMGkfE{%*b%n0Pu8rGi76e+Au^#?)>c*z)=lYcXf!k1k)34K%awI(iFN&+DQz28qvYs9U+%_H zrksS)nIAc~cTy*=ZtxgepLm(o=X_o_W6P#9J=-ITwzrHX8@8g1PfTckCryZ*l!ZTq zEuSM)XMyAxg z$|@zng@(V7XJ=*5P{;L?C^uJFO>H-9Cv!5#)tS`ejHrkq>=MTxXbqvHEg}scA2{eA zr&`W;1v)ZS)O)x(J-i>b5Z~@m71XXbD)DH6q-BP-PyMgkYw(1_o6FdwYaV;IWPan< z&~xMF7CnLHTc~QM%bu|Wr>pOn^fmTXi-ouluIB2R8&G0p3w5!*7#&9wP-%MO*q6ts zb^%q#_Zug=TuOBIHt7cYXevzbYRwbI7x=tQ<(W6%GPR*aSyhk6*6u#A$|X`-mrU%_ zn_G0SJABkF1aXMh<3ZzoS|YA=*1jt?8pbo*Ztd@T#kxMg@~_lj>rj)?a%MdSW4!f% zc1MPKEC*DBTJ%dcR3qLJsEXytE>puayb%+UIkgZv=j-lVi#l@Tjy>t^R?A23rgN&Lm^2-H6TgP%JNMtyz+T>hu=f3NMEV2&zb1V2vD z-e87L0GP~c;6B6(fiVrbiHu-KgmHntuk+4fIwn~P+q5)^zGBNKGc!uF%|wIv<@?tM zWgVpjRGXX)ZI&r8UZsQo24@Bb-4s!labPxa7T)d^%0W%8H?yW9dIl$RZ3rEh=qhWV zx}3;}GWALWF4a?TNE>QzNX!fCqvY(WR&7ZxQ`y>N%2)zRAg`ytLEObV@L?61EBAT5 z1zLmT`_sk5d63S8Hb?+XRHN`25-K}{b4RxsYD*;D1Jq2?Qg!?Fx-OMMKBwLE+v}bk zCUrL*m1vx=3(@Lg0rrV0bwVUc2rm-G2pU<<#u~1SEL|+ulu%z3se6|%_OgMV*QOh1 z=P2P)sw@TtaCr_J~@=Dhd6KNz3(7jORr2*Mj3UWUWoS^rgS21R^%0oK zE}p%kd6(LFjMGg^s^Ah&D9ZQeH0=0eY)Q%lFIzWTuWCyw5~@08th$(=L=j~=>vbR# zU!NrR4#FQuphQ%SeQ^|ESE#JrfQEaZC?Q|+i~N5)s{e$u?U|Kgb^ke3jQ_Ce|Aw>e z9qpX$jO?uc3#a}+?AjJAE4wX@hb~`G`cHw?Q7aOg`mQ@0Gnl~I2PSV($o)U>|v`$sf|RMmsNMa7fs{~b$(`X$gd(2qr65vG?W9DBx@n& zZIddMeQ=uDajFC38N*SFw<*`LHi^mVaFZ-!32&^>{z@6Nu4o>{UUGsAQE#|LSV&4e zW~AFf&?8C<+H^Sq256EUvPLum;d@1<6EIiJ-m=Wix~Z zxQXYM?SSTp{kB41vI+$xvWyUM#Ph&nOmN={4$RMEJM(U^UfPn{bkS;=uXrG<_gCZ- zK%L;}!Leo;^;0)_c%NBwuyhG&(f+?r)iS!-*}3YfX*d2Tw~5e`0*Ub=UXBt2JS^h0 z*gU2qN#X9hmTA` z-v=M$pKjfrFeWMY3q;<1{=9LHl=Jcj52n;UoZd6B=sl}&0+5FDk)AYL@;`J2Q+LpH zd8+z_`EyquVQ-u~aT#Z8=XK{rgRU@bWVkm*Ds#*LX3-NR_?ko!IF+aQjkiufp23|L zw_IzxJz!#E6{2Mi+b?wUQNBXO?+>t~=GHW|uvOr;lht82AXX_)DG2#meiPgdy!(%0 zsc!TezC9wo3*1^L7Ja4|>`}YX*DmT5s9e;k5=Tkc|J&2yu(!&_cUbyadmD=#vT?{( z-V|mKRooSlcd7Ew8T!f4(^VePL^qs)2OuFu5QN?CCc|$4$YLKBLs0Gf5CMH^9*l$* z79_n+W$cT_-%6DSc1;N9#QtR!?rKxmFhxsFkV4J|C>n@qj}2mBdWVR|h$|B;3HLcY z>~kbrYKx4p#;<7=BdOHOe)Wpwp#bX?mCyl5w1~hS#vh1O)MY?CE_v}yrGKy}AJXkR z8G!XICMcans3`K5Dgw!_qok87 z;Q@f4nA6Ic9eWRwFASO)new4&wo_*j%A#3a@u5QIMTbe+@U+<_htzKpRXE^IcnB9+ z{IE_ILWI+f^B89epx4MXCbm{OGhF|B9Vt#0wn00G<)HHL3Bj{N+)^+s@)p5T!Jd_H zGH_!_LPmsMi0FN0)B#La+=-fY; z?$L*go9bK(X+&Lc6!M9`JHP0zQ)?eNLA$v2e8V^?05`ihpWg$_#SvZHVi1XeFXU|o z(^-k}#hvG|7&Z#nt)0hws<*Fg%O3q;jIPeHW76P$cisUINh*HbHQDo0Rq>GZoE?2z z*-o~gyY|Tm6!*`kaFGoJU4yjiygwS3a;It|WmIu~0#kg2`AE6eG(Hq7mgeZUww;XU zYEzVxBj~89p#m{~di@BLEQ$Bkx6`O*)j*`boH>_{&`$k@Q9Y%L(@x|ZHqkTz8RjR9 zaHWy?d+c0<9d78%hS1W33lULfmNzTiFBx6kp>N-G%A*u=GChB6Y4&nEe8qz_0xN*q z03%?#Yh{;bSv*mnY$|VdM8U@^QmA>ltNhjQ6(k*x+47!0J1FJ7#pFEdXyG$CeU&td z8;R9z;|hJ>5Ng$B2~@V85a$!c$Yscz3U4LS>Ne%V@=>U&Pi7)9YXk_oUILhsa{3oB z9w;F>pDHb^#5_;C{xl**zeb!mOSFms4LEBJyyx(uCdXkU-8|j1gMw6&8=hl8;BrzE zw%t^c+-GVyHH@Gt;tkzBBT~}b87#7cp3sCtsbc`6qIpvO5SLB%iw%WsZ>gLz7HuBF zBxg2;qTo{=I0hUsn%PCRFw9uWzfD5Z${1SS#vCBh1l>%(4MxM9(KzS0j%*(C;>0f| z-g=Qn70MtO1jY^pv2b`rO5uOw`6eQqbLDuqOovhX@{z5IRr(g*OB;tWO>!%9{^)0- zW^WZ+6WC>u9CVa_HwS!3z_>9}<0@W2u;l8)B|u^zL7P!R$cYR=_}MkM4KH^m9JF%P zkFpY*6lg9)Tgm_Za*mCBCUwGjAT#c53R3d=$vI6*Z{blvRg(};_Kepz#i*}l&!~DJ zP-q7TwpZ3P$c9R!Yr+;Xn!tdb?{kWeW*p#VJNnn4_KpalPK0e>F!$q|0;f8MFP1AV z;hz=r#Qvf1XPo{r0IW~2Cus8#=?y|cXMvGZ8dn#s~S!dqy@(r*1B`DdqlH}aEsIGm9@kQ7dkF?* zFOPKhV(xs^>3M#mo-|3^Rj)BEG(F^V2HJTgZV ziwepzE)cKTXlnnkxdd!_iN)ad22Y>1d7|PbN z65f*}6u!Yt_Ni)rGC;Wbb2GvO?E}zn^-sOS%2RZK_ZWXKGK~sqM-uo0{dZ?8o2R(} z{$?ui$BN%9tKSSQkDm*6IYl1F_YILdz%&r2IM_G#;SvOZwQaXW*)E+R^k1r?y z9e9&V8f`v8ZD4T{r%iu@^v?z!lo23krI->EOO<_^AG7@MhojUnhQmKL`LIfdM$b61 z&Wq_|{8->&Qk;dV*-&66xU_^!)Hfbfxj6W_w;dHM4xw`bC#$2QzOs~sUhz@;%w!B^ zPSQ&y4B3m7o`w?f(AeR;av*5E`L#`&X1~wk07}_*zYRMSC6FfH;EsLQclzC_b1G3g z*y-uO1ABQH6*#MQe)``&;Wr^~ttUj)KPIWNkjHbJ2(Uuc49na7$sl|Ug$>2Z>vX?R zaA6Z>IU3Vo3lWT?i4aN+v#E;Z!ub!zGS2MXAnY zW^`S4cD1b(cV;yp<*)LGMloxZRW9c)ll#VFlzm>uSXE)qKY>&{qVyBZ{t9?7at`B7 z{;Fc4eymYKf;rmFwDD}}Fd8q}cVZI~k6&zM>W3*!Gj8|Pk-@iSXjKN;<-Mfj&pCya zt_{B`Z2j;&(C6Di#n1Q5Bi2JnXXzvk3I<1Qaq$F3jsE)wM&DDZfvMd*q05UxtU=ok zP77^8;9ib;I99d5Def`ED5wT5J*FQ=iU@OlB$#tsGW~FZU;Yhr2Gp=b>&S=#Pu5Dq zs3~Golp;*`;GqO_L*n}$HOV2=6jMtsS?Pyq9}dA6{HtS#E>)1r!NE%a2>pYXA3)9r~fCfUTa_4VROR$-t-Bk`QEj= z#1wbo!wEF;vt90%q=_MRLDO`OD7T)tZ{A1S;YoOl{Js)bjEO%+9+7O($ODU5M-)>; zenljkenj_p|0K#BX&~s_G`$|N)Bi@$=GeAvamJW=P^c3vs^F9ifb`BCfrJ-PCm8ql z?>;&OS0=J9Fg}3E#nJ3hZj(HK^5nT&>*f+LkrVIbo1)nDsr7XobtutX;ADQWD`6 zvxImc(P!MwJ<`Yp-!|!YL?%U_C;%v-`C=Fo9T;qyc_)q<;w;1b&pxt5_4|21k>LL=8D>}`{K+}hXx)GCg?M1>Dhwv2l^ht%$jSc0pHoIVQC z&$E8d>mM3q=mkR$q$6E`VCH%-A4O6(vS*bIQB%|8LE&tz)mg9S#sxHu}SpU^k59f}SH z#!p+1dsD0jULC>L5xR?xF=B(%A2nb=UwTF56l4hFlirpF1%!+$TjDXu#n6WP5Z%?_nJkMUBmhx2D!1@L zc$;>>m4b^F zTbhlfLo>Bnr=?oOW_6YtM0|^1{pZUy98fhebkwm1vRn92%7b?rrH^AC==@;g-?dmH zPkx~x&^#L32*AWspmV_pt*a%YQEH7;Z1?#ug{KEVj3NhwlYa}+KD&(q}e< z70Z9-^}cd}j9R+@Xk0)c5o(_~g73oLuzEp8Ce%k%cU$s>Mw^^n1b$*=9 zOh3@kN~I7tUB$XmKR0+J3ol?VM)ywl-#9?`AY8a^Trx;KZh$iAHmGjIY{Z`6wgh!B zdQ=>&S|2hJdjVCrkq2r0!L5WJLdx>Dbcky2AP)k9<%a!{W{hg*3|ePR+od8gIdq&# z)GEWwy9@N{-kL#faS8g7a1BOBEa#KrFJWnqT*(rBf^Z1SFeo0VwP-oY4lUAwe{-m) zDQfb)0|&J-rIK{VCeWF9;`*^JQI{V#HU0juZ~@k9E4<^)49Dcm-))@n%*eWD;&ul} z?g8RJ@bPmW6}2ASG_HYD0JH*-RJrJ#xB7#mlYG-{S zOEhzfN{M%GCA6@!)nj4HZ3Ul9+bZzQQU34yItpeb?|KP!aWgfVAc&ZzxxE zO!VZ14k+7s>KnZ!XihaGM2L}(i4B3XFopIylT34z$kx8L&;mRB^q>`#CZ==Pj#LVMEbCa*x?Sl3Lt2<}8gk+EO` zFHoM7d<9oxK%PmC7Yq>5>D$7nhc`6lCbE>tD$A> zrY?RA95_Fb>^BQmS1Sp9umt%QIVHdv;gjm)%4eCzXKUKeTdT}HPS^x=gFwI$+LNt@ zCzc61mWWGU7MY4M{Rp6_eWJoA=HGEUS}KqCn^JoK06c%2C?7Y(+sYRi> zLpp$4QJXHNhTdQ529TA@TeI|{FJsDZw+LJT+W>X9V>Moy9a%4)^ z2UrbkaKN+RB~*I)=!^&HXiUwStU}JNUV!?$?%fGMlkf0mjW#}oNnvY$wzSf?oD{k8 zjx#N_?0}>Vh6pc#fqC zJfd$?VZqvsD|Hs_kaKI`~K&B&pGq^wH=KrsF2cf48opJC;VytWLh^`OXej~ zW47;s8G(N>67H+^U7DrXwmLbkeR;#d-ZB(Wm`m@Ssa<)#Q$kc{}kr#*RuV+;c;AD(*ni(O&4zieLm-!z(i#ziIERHl! zk#gs5$Da?hdgI=k>7dDY7E_-T_d>TLk)bZ?VCy*03;d7Xh<|UB!Vjpt9*omqSP4?~ zb*j4mgTjpFdw*|9t@fk#7hjh|>HVA8OI%JCEC3fH^sWD{E=aw)k7 zwhr7VEwna*51W@h(z$v~DeM+4WRx%ML=f%EGE<)DGlFwC3)rMUbq+J=kbqM_#;}tCT z!B2$96AjB3N5$Xw{JxeK$>2zNVfMc~d+D3fM%>nsx{W0Xv|7@_bR4PHh;Qb!PH#gT zsXROMtXGIgW~+_@ex{B8pm1?V%HJk=c+Mn3wYR=%HKn93<#KUbjQ<{lqfBg*CRM6s zjts{;FUm5vKr9_4QO%USk2YoWi2;(h%u{e>3v|}?v@fvrS=srdBvhc8C!6pQ7Q1{M z7At2cw_QsEsiD1{7rMhz(im`=1Qcb{;V{^DEAY+)ypNOL3i=nG1KzP|h)vTHsu>qm zoP_;~woxYYo)N@keR>m!k(Z9_<{f^5ppM~!FV@&{IfoPvyG_w#aI{pW_ zZIS=o)8Jtc4rO&h^+UcbEyyp-*?L$0prYMlz(@U9|Km&PV?2$=x6Jk4xD&?Zfl@on z;mYayzUZA++Xp_Gsy@$Q&#Vc_i9v37?$*zt-Xrt>-5WzE(=UpjPkwk)F?UZT;@-}? z7X+O21Ng%q+G+o`KW?TF`AToh3vd)x37Z=g)(QMi#a2EtQ8J_5UZnCt_-s=PO@=h7 zke)t1Q*ZZHg;FqL@hZaCwq7}7f|9GV+4&jj6p#M#RJrOH`rv6x?a~;(+LX`T1B2W6;my^UZg`ZYb9=oplJo z=1rKNn9|pp7#lcj^AqRC?4xrdC_V_3@d^~nR%~NHQQ%^381o@_S$RW56QDX6!N(K} zWyJ=JbXexaa46uCmKfg2=l`O-K*Y>Z&o-aH5wlC2iLFM=B*+G=Tb3HniWnk$wh=Zf z_$EN~>h@P zJwJe^tYe7^W?`lY>JSHJ*(AG`1?m^~^<*&_ppH3U1{)O!!@B-C-^Ky8LjiNN{Q%)u z&m863D4^~PV2T2D2*px&uz59nonoy{J(z-93qrAhYCXWTj$%D~uS2EW!AzFgYni~p z?q6>Sv(KSY=wN(;&RYD{vHS0dpmN?|B2irk(W;;+WEy_4j*yP^_XSjc6-=PAdzoN0_@!U{jk{{73!w^r^E9IVR< zD!AsY#Ji60BYPU4Z#{vL%e^npSDS^)(u0lyR|9B}<)u1m#WyxcI@LKO4cKX^Z4^8f$< diff --git a/python/lib/py4j-0.8.2.1-src.zip b/python/lib/py4j-0.8.2.1-src.zip new file mode 100644 index 0000000000000000000000000000000000000000..5203b84d9119ea5415114939624411ba7d3daa8a GIT binary patch literal 37562 zcmaI+V~{R9xHOE8ZQHi(HMZ`tZQHhO8*6Oaw!Ox-zh}R7-XG`GK3`Hv>3k#AtH2BfiHX`GLH?61ie-V> ziuG;V0CDeXPq33i?y^aBwK?GwAdx%52f3mbTgnWq9>RSpPK)CXMx1NOGw5R^s$@WO z{a0h&$0gWUh5he2sn!_zskHjPvGDvm=vhs$ucLz6U`Pv5S{Yzd2-Ay(W2UWZSiG(i z%PY&$C+w$lDDZ0Zk06_y@6kW83+W4inEq*lg{0Mdodn(V%>nR+oStwYgIdfv;_>hG zT+1AH)?M@56v1D&$SmHn9q6#>j@ZT9+9;jDM|?~_^w6lHZc^2?#l z8k6M3Pi56CpvF_8cF>%fnM6IY3+GM>Hsjgk0wMoj0-0Ib8QNHSnL7P19J*Gw zwO$rN`pMI0UPj!V&{afE5gJgVhUyJiRUldGF>%^(Yr=Ko?8!UycX#9LAbVBT7DKZY z(4~0%m}X_&=Fbqw5UFBez=ZbU40cS4> z!Un~WD=j&2R-Q>Mdj;pjVCM#-U|kfUI47Uf9-{IpOw^nflJXd0duIVfXpfD#e@`vwA+}lLu%jbur zr|HEUjaTPmpfQ6T3aIJ9YG6=7JF^%qW(hm2j)11B91*Eve3k2zw-|IrTOU%u?XDt8 z_fpYGc9TVOe@sI|=c%~YGxyr_)CnN0I0UAKzE4_XtF>0i^-vD;bsL$II~@A3OW}p~ z_pTZ%j2-12TZq7vDYlFY(Z~?jZ;EH2b+A)CP%Q64K)T7n``#5oxpERv)lzf|Bdy>a zC6b38*gRn)EO)^c`?PF^_GSBL zhlS;avik>Ll3eyG;M~e7KG_Y)?{LE${vP}=twHv+_Rs~x6YA$T`UudE5`$=?pR)PpaFciZ3APz*(0Lq(ss;Ni_a@7GbgVp^L5n>6IYC10%HHw+ zNB8jVZ_<^ZKtMN0|8Kg-`oFrjGITT4H@3I2F*SCvw6}Bq-{h}zX^+c|@O#rIlJ0lc z?iyRtMTj8WB*byKTbeF`+yzV5IilKn;<0%jV^1LEBli1BS~({97Ax!U2;Hi$mIs+a}FWeiH8FLijssH@Kow@EUmi+Puw zUN@OBGn##5!gb-vEf&3l(-Pj*CTLKv`?y-U*|jTgnsde?-!SicAk&Ueq+7e<^rI#r zC}9irL}ARloqMEH2)S)C=!i;=IZ*;q!SKT}AvrMIH1|mwHOzz7S9T#YFZ8X_RCtUD zCYOJ=0>?OON_?|tD5^ZWBv`sL53h_rpGXk6q*6NP0UXt=3i=bs9-Ty)d*Lm^{m=`y zNp1rj1(ownC8=YKnKSkILzi{@vz`f;aT0ik(|SGstkjSfT{KT8c}xeXVNbr6Xvayb z0ic*OPpeOPBDj^KlU^7%t6)&-5%;z!JZ^1l0Be`TT%scet6IfBok8n#MlK;#M@=7v z8syu&=l2f{GW9}W1~HH?K(X>XSd5}*7(1}b2MNSBdXR_q8~GZMNCo_O5tE>pkjW1C z7GZVMb)rRDVSr|UN&%;XnnM>6J^C`DA9T;Li)Tli8bbcHW)Qz!d!YrE2797Y9?~AVLjqoAcnyuJ)%KYXUZeL zcB0u3^JHGWDpWeRmt#go5<;fdMk>xT6=5O|<$Pg2x4+^Mh8#c^G<>AoArM_;Z6>Hx z9`*=(i2{yYro~dxkc^nd<`!WJ7nI$W1qX(XE??p^%)`<__z>UK(EN8(QT;#*sjjff=O-(Y5aV-MF6WMg^fDZLU)TE%6JM)qxEskgPa{~{C6xi zsFPn9NOX@Tw!&};RG2()f7aDgFsQXhD!2OrmLf6&p+->yBPhQ`=%3w3K?*>4(LHM4 zy5<+JHztKaZ;;GF7>#rZ8_AZ2?6Uc58(%lj5uFXCh+hEm+ns9|C{Y#Y^bnSJf(P7m ztjMhuaZ=;*a`pvDC>vOn!JM~Gd78`dwVsKf@yN-v$#5DuTlNG?uX1USqFiLqps zTfX)V`%RkkTteP$vap#T3Rabs|!(BDnv zS9Io}q9x>B#C6)(g{hWWO>Zsg+g-I?gK^QOnMhByQN|Y1x&mu z^lGE5yK~H%-r7N4Nh!vW2u)@u9G8=lFHu>~Jn2#cq6jFfa2P(AwHO8J4sG%Qmhm83 zs@ejdph4{{nPk1O2}~Bg_{yyNy%68F`N^{O&;M zJzzX2K_T9w;?|>^#x+PPpjIHVYFGWU)&Q^!vTufsdW+f*v8EDJ764_thoWsC4y0xyF1c>G5o`)%XdCe%x*XMYrFW&QxOkad_G9 zCw0CFzEUrMAQ8dfxK^-xIHvKRq2ebkGPBOVukgyigQ-{w-?c(WG$ef=V6Ay68>*EZ z6FvE11FH632F7oRT2l>)krI?+5<`$|EMdJaWYb)w@_%1j=s_KSdN2x0RRe|Km4QBE zK+h&YBGqb|H>hcu2a^LoPOz8HSH#=l>Ib2z^miDcG<< zmuZ<2vm^6XPy%4w*|C&$L_W65b0L#X@Ox6#@!)5Gc=pwf~XQ@P#=f+5- zDN)AJd_Aqecvuc20yhCL}k&SIjm`$kB3`w*kyOPS^%^gFzt>J5a1f zB$bOel}buq7Mn@1{0O6|f1)EM72NSUS*ecqo6&j#0X=`4svb8a*eMn@sUQ~%@&;gz zLpwrR)0!=&h23B31yWQf^P+;kyL;l|G6N0{UzXRC)9LDh);2(AxvTIhvzuIsyTAQ0a^0cC!H;9ztdYLME6 zDu6Vmrn2XaxE$L3Q6Tib;t5bHb8(!!i*svPeI}2LI~iHu+EXz7=1>j->8_Yiabn3Z z09p-da3ru4AXa($=u80XXiUqUtVYeLS%A4%_vr+pD{y?X!5E*yrm}N5U0P{ePL5i6 z$D5W}c0|#EKt`0p!oK+=oLbVo%#|PiIRq`Jz0}u6-FYqs>2(LUsl(Rm%g8xLUJ~s{ z88NV}v}A8k)0Vfj!k;^(-jTV^q7ag7E2RJC5S?4kCIS}Uj_Tm4&H}~)k{xKoZ#h4*T5baw z8P9R-Vi8&fxQOnPlEn4st!=?NV#ujYcl_(thIw+B7jS&4n(XG9&$56M&!g9o5lEdRT#B&Fhi}OHGk(-cpOI>%d{s^U*h;voDt$ ze`AYg!dv2E_&n*Cw1@i*19zrH$X39>A0dJitGV*E!7C#skR#jWn5V_Ds8jOW>0Myx z{!6gc9yT%u;aWptU1_>f)Uwv}<{89p+k`K5K-gkyYwMj}{Fr>S(5^G7oXySk>uC3L zwEZPKJM#;(X7As6!3?Lbg0*knq>3+?N`@^W0kcD=y=z50W1--POpEATP}asjCCnzD z$CZr@?ARKh^C!qv+ObZB(A@A?gpiaN3k(pofATI(SzEpfCe<3Jv!Z33Us zBHh7gn;!b4yFdR-hSc1jbR&1IM1K}dZ=FfHS(Kj4D(yKn!V|1qS@%QMAq1?Le6w0M z?$&t9E3Uk{SASI{_fS6=bMK?AaplA?tkfd~TTKif@GYiEW*H zHIGN_RL<;!PbhO#_9V7QR=)f?6rO%XvM~D>bSp<*xo*URIkXp!S^T*chbUN_W?qrs zV3_g`y0@!iAG-^~Wtb%YU&OkE$#y^WKGe?5+dBXb35+iYDChWSZ*J#Fw@?T93`;f7kDKgp4J zLb8irq>E=N`_Er*EJ9rjiur#F^S;>!>14vl={0d1X$@Sxgy~*AI77h^i1U%RMbgS2 z0Bc{OYZnwp=xCt7n4$@NmZ(E8T zObx{;(v$Kwb{!{T?)U7VL?HNpMJ9pIJ(YOBVqxW@hGd%{ z6Mm9;UELjBU0r~#?lTF`+aXaF+7ewKZ;YL|J;D$2-qrQ(Z9e+aijJr&vZ*Kiq>HkI zf~GGadYfRRl94me)COya5$4)8K6bL7u< zbg$3*L;lMP!V>yZ6<>@V=(;yEX0)qI(rzT*3=Lij|AkM|wAynOniPs@v?(xD%>$gc zCfOt0n3LXf(NinUziKTq9iw<_`KWiiFibC%$|{xnN9n5x!pFlK1g}~@40LtvzvUyM zfQrj!6h#_9p)`@?cXb9|L{>GRddpN(RH%`k znV7mga2d@H*J;($;Qs2s7-UXkmZ)Lk3Rld1i5#1%@(!O*9bGN^Jp2_AMsy_aB+6z;>tWN)ZdiACp z;*{XDmzY?Xm{_^{z|nKp^L2Wc!AsZ&0BaO3D6C9xk5y&-TLwDn*O^{@p8% zdr^1gxb2&%>Wr??=3gkOAnejod+IyO`nM7>-R=l?uPP)k|uT<{R}k< zIN8sg=L@*(_Uh{Fi;P>42UpyGcx~bwJ4wdL_i^*{;}#*Stk4FAo%c7-eVw|@>Z54tq$n?l`uoGFjkXe%nqsi?c_v1P&4K8TT_W)k*BrYU~T!f zTxS1wNn-`Uho(y4=jK<{q~nKqMp_5ygR}mE`Pg{~ zsiX5?uleLcfF8W1yG1GZ%mk3ev;|3SQ(kY}!QN&7>rG{&WR&K6CH8q;i;9*(l;>lB z1GZqpvO%Y8Fd|_942bd9?n)5zc^jb+Cqktd@a&wVe6=ziYj1{NOq5B{Fs<*~GyYrb ze{hU8)rr&nYY^s?6$usrIbQ*3FI0dWmbRx1+P^=#=YVUBfnmts)K6uKknT)E5H5Q` zHjIX%gaO%*6#~JcDHtVhPIT$0l+cZ)78d@e7LQz@p&JtxtwVLg0#I36?7{K+HBZND zf%n_1szV@RV>MM7u~Yyo`ad@~?V?-i8}IIlS;!Iwjsp1Z`;6Y?)BZ7YFnm1gou0(;=I& zh|`jzjS6X*V4VjE?LMRze)|XmrFn!&dR{wYKHO9@K-E)rDah55a&Ze`9@mE0>r*j- zP1zp)-ND>*1{Xf~p2dYzaU5VYA*&4B@)mzQE`IFM7r%rkYxygL*@(RgX;dLgbXQJG zE~=PYV#dw57IO9j$!^2}eIC_l+e3(lC)5C{vC(FFPxKIM+KTpxkX1!BPCKk?N~w`{ zt1xnX!H_14a<02TeqK>k{-QyPEM~{Tl*5EnQcp+;7bhqj%S`^}{ zyBVR_9pow~;Rs6R!9n*#;+48zX4jfR3;a>&fkxM<0-e-fzOar)D4{HeK21h8+@MoK zs{0Lp&OH#K) zr=Am=Rd0>+)j{%8iUsA4Mp2aV?T!_piEb|+Zm;0OoTKl@N=EHdjAZ9vE$}5op2Ldo09R^VQ_JS{Z(2qGzD-O>n{wrxsM zto~ey_GM+Zs_Il4$0^B|)S<6O8rl)GWBBS7A?D4DJz+WxnpX$(Z5#kcFEP_ zYFlnjQiN2xInc{QMzC3b2y8=ISy7ePKxqFw(^H!@(4kpHn^;eUWht<-2?Dsv<~bGN zD{{2xnHpb2r+|Udo-^c1!>sx{iy0HSmU*%l-_V<5RYK7j$PjE#Di{82>o}MLKZnHO z7hzG`;D2{ZpVmY!qMT2F{xz;Sj;RaU`$Lc%rgigCrKeV@&@eD8@aZs%f^-lV+K9%s zd0@fE8FWtBl>eA(;ONc)3OZR#aa4eJOq*6_)T{w$;s2Qx;v99+zj26)lsSa zcJBYLz}NH#+kAuH9BaBzeYrBiK)tAi6{UwVz;EZDiQ~hOpLzfKcRX6EA zC5gIH@&@gJM4X7CETL}Z_sxN7jA}ReTSXv_S;mkxH45g-r#3xTSlFqfEnNw=VD57Q zrs%w&*m6NAuenomqh3(Pf5My(VWl>I#gqmn4`b@sS@GojaWNnm^A{fKtbWiqUBJ8G zLyz!dZErUwsXCePZyqyFKi<^VlRVkOHegZo3ioX$%&+}5$RMTSj^_c{hSPc7p)B|B zHWBV=s!$bdf*GoBE4DgpLQ+y?f0JRI5+52JINjMA}G?R zxVlH4yO)48gPuPp9|3teWwV&~ok4PdFJm}zpxjEv_?6~>d2&Al%LX{xzo;B|vGVK0 z7KJeTY^Tdvp&HH24uptW5VE!0DRf35gi<11zVZ{(&GPs8@LW)PtqYM08K{ z6~ZaeMx}oq#>mILHbNB3i{*+16=}J1OArIW-L zj$X1=Gpsa^0byh@<>vh!%1>$aX4T3#vE*tzGQ!9%Rtiqec)h~ukW|fq3 zJM)Bib=|Ydm3a@`sEt8eE)R;&^wDz0=z%tqpLhL_#pj9WK0*kr{AL5qPLSh%x|U0A zTX5ql{Ry2Y?sth!$u=qMxbL9lB3_9Z#Dyw!sy@Q85z0$7*b)4MZjU9du&S^cd(93c zX4JBvag6XHeG3WLDWj1&Jz2}rNn|!1Sq=(59DiNj*=ynLZx?uTLc;}r!awF8g(FH` znJT6UQ!yN`k#!XU_M5|tYS42;$?(Yg0%ys7odlJv}U4kjrWywfRT;g|W#!D%7nvohS46FN~S#2g=?Tm4SAZotOBG?mh=H^0yADt5qcGy%3)k;vhBoTY)n@5 zQ|o1wdBlbP3Iiobdzkn`Hm{ec_v3ncr-YJ)f6m>z%~Nmf=5cBwlOfC0ddwr2U?|BA zE%z|uRnM16$96<8SKOSb38*eXZjXi`*nbun1h1Uziwd?LOyU*E+70zkO1yz0@F#Ihdx@viHwH>-!h4i`rL^SR4c8)oK7 zfRqfEOGT*c^kQnsO;qZF#36)fS7?&4$|o_cQVY+NP<#BrdU7@n*!Y$=0J}SPB=hZ+ zacfQNwV!Iy0%2iqwp(PcS$$^G1!ZnQinnVM0B7vKa6vLesWt~U(482eGRaP^+9Ec@ zY{};4Y5JjVfj>z?fT;!~ZEG3t<5FM(WgB|R3HL0H{W0I%Go>baU*4)NWtEjKJYz`K z6;yk>Tc|NJCvTMCKFEzsafD)b@u{<4#)kIc?>Q%sxreK(qv=1u=a>_4lTqo-mciEI zRiMsl1)t4){d=t#PqW&)jwX#uyN;%AM6+_Z>0LWd=m@L{qZw=Gz2YBwM{^8g-ErN5 ziNjiocqCp0nQrX?8eoD#u8?i$b^uClQ4oR>7Aq+Lpe&~fDX+{VQBsKT0NWVQzr5cP zP>+`N4H@^XBY6<`qQ&Z1K~3o~ErrQ)n9g6d&z%fWY0mjB!yjkJFElACgzN6<6Z8+L zN!%wnKv{lSz_t*N)>;t;bpaq+5eq4~@1Y$|#vU5hk&f1WxBQ!5!Jl=n$O1zb@Z^>U zcs2UqOLZizn5$!|V#U%You9Tbqt(S;1B%TIt6pdYeAN!=!1+!E*;0S03{myM-kWOmR6?7FaA%ld%r`91@pi!X{?CH#>tLLp>v~8a zq`Ws5z|Z?O7B`01in3uQqC(Em6ok0|t=lL*_biwplG;5Ymjm2>sI`Z;;QUX^N{jvR z7sgiSJ(|7Qaf)<`_~chW`prKw=xW!5%Z=B){on?qrNL5AbxLTbWL@%Z(|g%5vOX=_ z75d=&V4xb&&b19w;-;XZCt#iDH;*JX6?!{6)9vAoe(+wJ&WA#cz75;1)i5C?Y8*{Rd)_RL;kNKL zv76>QDbHTdCVoywS`)Wjr%VGHT)*z5 zer`Q*UTPzDfBwq^bPdmkjh*{Z@Y+$f{Xz(^eI(Sr@eq;ZR>}Qm2us`$rIOkoBSZj< z*WXQzvpf=p?C>IJA*1T(CrZn_as73_0y+Qoj0aU$l@?5qvI#WY;yU@t<>(h0D>Jt` zM_AsKKS$QWxw zSa!}X&q|~1k>JOO6llX5tZ!Rer0)pV+Rqv2zTx%B|B1+z552134!!p}7LFxg>drK% z7mFMRvmchh#}qd*D~m^rc#7dQx9Kjh=B_BK8bsQ0eBa_QkOFm_tZ#05u%)+-!iK9b z+Q&>5QQR?c)%hksU%C|4$>ZjH&*PcxnG3Xg?zPzkHpUE(Dn4b1VluMKR@>r32f)3< zfOfts+csn0Hhx3Y^mb--ce04(Q(DOHNu*O@c0f0Phkh(@65VUTv^tT4+7_;uOupf5 zrh&SO7W!Dn5a;%o>97YEY~|kPkn5~jos*bMc}+vQA(14ROCd*gbS}r>Eg=EP8X`tz z{Rlrz>CX@eM7S#I-q&?w|CgH3?&_(31aQ)H4w*VASGO4!w8SGFIqHT+i>E0u$dMIWRV5xcpk zU$6odYYa&5GcoZyJm{8|Z$mnKCD%(X~e-{pmm`H|1gN*tU#CASX2erHdtamOFcxCCa9^U)vTai!s={2wUP7I_~w1?zcmb!8?LDTduz7KLPF1($z`sy-=eSZnR$YE{tL4 zAWiJ&dGVb)7s^Jnxv`f@d59~)AdfF<@Z*o03^R}-wCQLZlz~RIw5>X(58@9-#SQ;J z(6B9{U`3XdN6xHgpt3P*F1+B97hmi$B=wjBrnHq)lcUvA5$_a$`?XTWBzJTJm-V)! zCA4qt>-bB5luv+pBuPk_{4KjDQHB%-v%TK?C1nVIYU#IjYKhH=$Uk0t)}P_IZB z8MBZ7TL--mkYa8)i8r0u(D&!BM8z-7^q^gEcChJH?*R^-#Yxwet%3W4LI^Wl1g?%; zSdiFt%MxNMwL;9*K#`lT@1cKS2j{yWXr14W8(eq6x=xOcsJcCJ+Rk++sTN{p`Li#5l3;i}F zc46h~<+TMv-|6hS%!N^q>w#ZvnB+XoH_rsfNoH~EwaV5?RQ0FyfJ0c9^KDN{^_L90 zN5XJWc^w!d98csG1OLQ_e+bAdhswL%6P+$NeViz~+*R55OlJv+; z3c)L?z=o8&5t)EU!t0Xu{Cerc*Qa^f-}&d8o2S+m^}{x0u+i6;X)D@SjlDk$%Gf;V zP(VoV=jSZja6Zk@8F-_GW7hO_YB$t|l7K$Dx>ZUXFfCHo;V|pD9w-@OZJjRuAEKzIqQ(r@VeIPq8gMv*Ik)Wh7 z(#~lZT(}g;h230hpb*hEep;Lz_XPbE`zg_Od8Tf~GVc#+A;+jBd_)~__?6IQ;Z5uU zhzg6AwfEt9(fy`_*+_32yH@z)4pu+m0AYQ(DPo6S{pV5dVw!?lY2{Np{^1>FM)Krkcr3 zBL92xtAA^UA?TKD(xJo~!o`tMEStA)^+P!}u{KiFyy9ueLGO*gDFBS)Y!lg(`*HpW zjj~SG-WKESv)cEl;taE-TZ1m-NGp_;Lffs?h2EM)nx~9NhnxRX3J;Ak;(BDaL~;sd zeYPvle?qV3RFJogKu)s7F*2_v379KM2EBJEbq5Me#H686a&kBfhJrZ_b?lNh=gJ{A zRnWX=9x0G5*t5_z(uyglZ0Du$8nCD-fjM|Vz#ni-l zkge!1M6<1|Fg+%EMi;i(Y9Oa=a>Vp)> zv`e1JJ4k`BoLm+`M971QuvX{<4W>(47QCZSKRdOphsGBWq8j&u8F?Ts`nq-&G&!9&;Umnhp6 za@006ELcU-WocQ&P%APjc2nh-KSF4{-h4}ta zhg^5w7!p@SX2j6wiuxSHARQc=2zHAVbM*u;f%LTl~x%juHMmzvql%2 z>bf)tYj&ZMuA|IwBTIhsH4zlcYeN%BA(e`*mv`EvmDNxxl{b`Rv9jh~zF}b&A6a6b zxminRAYiaXIRW&F1a>YiS7SxSE+sYyO>7K8MT6DMWWrgsAAsezm?6!6!}BANESJ$* zyC1W5S%neZ8}r08iE#+0I7`ESn-zKIgIf=g6QY28U4zxSRi+KF*V#qb_If#?(~diY z?i)^Zh9*(ke#XbdwT2C){LOXdIj;^Q{*4R!ID?RPiRZ;v-US-%S$6R9^JE2i|Nh#! zvtsMsB$}}c3aY?0Y0XLaoSpc{dS^{FZ)+zfkyK|o|HOrh-})~_3N+Lk59IJ_#s-0C zeMed;F90pIHg-9k*0Kd=U-XH^FQu9G1O*BLKZ4M8%P-8}@e@f?HB)ci?uIT~L7Zcx z^zlZrowk)VK8(!O&qHZhva#fX#qxiLAB(4zoTYgGR+R3 zm#9L9wJVSK!&r3G(a$$ERiTjz^p?0tyOb-A_s2C3X}^6g2c(&+>Z%aq0G@}Xo7(5n z0~tutphlHMWS)qcopvHSW$#+56e=wC-#8kQXA4c}hNILLF{HQrai`?Hor%2Xdqt4i zx{_(Y_i~bD?Q!*+q8_~4S}|RPS*1Uz({yua8Eukw>1Nk{$u^58LFORZ#j=a6GPd2} zJODmMG*TOQq^anNqs?Jf(@_`|Br7hrUF4(-99s0526F~>wVLk}n;+wyf=p3@ZANHz zyzo{d8@|adMSV;Id!*#?blaFzOC@y~XsT?d|N68z?3yX65}xBvNh<0$P*(|PaI+ZD zR?gunyDf3PKraqKY#4OvjvLG(7Zkt{}nRwQDc_tPdH2E4udK{kjPBAhq8W|$)< zH>jT~Y@QDKiB%h`nh!I$TYj;RR!gm?7ATJ?`C_cB9lIf-1Y02%)zK(U!%}#``$+Hp z<-1N6tNz3dT_xI9s?5}57n>&?I*R1FGut~l>t(I>PHTz=&%md852IjA={XKZ;CYm4 zpoMJ9Zr{fpfdHg>(oP*I&R>6y$kE!XCtuxtYt&b9L}6a}@~JLj z)Ztzy>Al*=mySeLrO{`Fw)KV}Ei2*zZS}#Vnjal8TtGV{ZbJOgg&XYDzs{MjDrJ%u z8}VAA9R3trLk)ImKFsnZYX=YwO>ZZyb+}L9hT3WN^iPpw_kBLIxJF(oT0c6;wWsxH z*B`T^E0xRB>a`KNEUu_Y8d*+8McnPvoJb~d!Lf#S5bb0-xL@z@v z%u(Ur#K{>;VqH8D4pAH3ulfgSrb^|#_}t}Z7a_6rW+woG=xXv}FjCf(O2d$$QkP`W zCA^gNsWIZ@!Ei069mTVdMS4|#L@R|JLhWf?FiN$%vY$-By{Q|LCjK32t2z?iGX)`Q zMu;gXum5Z3%}E1CrPx}$ zYC$igCetmr0*{>cVLq1Te2t-z<(WoRMmTv^K?Z}xBtGHvHbGWhIOTv#f;%^pa-3rH5!K_ix2W8 z^gU_L(Pgipm1$Hw9%AeigEk&w+D8@rDFllB@@Nf9=l`1ecy*nWuV}WgKmWk%hR4G{ zH!QQw49}9^9M2Ql@W$&36`*x-t&_gt2|?>q{8Ez7;nPhDFKw9?u);s>kRkzi+t!0z z;QKaq2CmG+7_Kt`vh7?oZjj0oXc(x2m>2%Ju$cBqG!lcxK!V|vELZ!? z6azw+FvTpn+5Sd`tz-&6UEf`wAkd%-sH$r1w&|#IAQ~(=aBl7MPb&NwNirdXDwx+A zng182RXd3-&?hcT3Q6HFE%f~5gF4_RHqZ58}xGUC{3fSeF)x8Hx2pcx7k;d7{TEvr@Y$7 zsV;S4qB}`=PQM5XFle)f`10!A{d}(9j*Iz!V)Ff{!zEcVA}iyU_5|6lG_6OS&XijH zunclSDklpPW-$1vym%=h>a1Eh8q~(;_sa(FdRuY6W5ori;@RNk6x>8h(zRJwh6}Ob ziZr9aY(+UEVydDk?}mO-P(YsBO>cZ4o=2o;)EElQ~y508~8*FqnT z2ib4Qk2i}B1SnH*Mr(oSx zL8lWVs`E;%Jj$pPKs?xj(#mTQx}erMyqI*{WGVO9xh1r4(6JBWswK`ZC}&-{L)+aR zBVmi*982J50Z%hFz*|i^?a5c8DxAE@k)d{5yCg7ZcdhOjA;fiJ%`x-Ia^WVd#$3ObWo+4e z^Yah2lVx(jcxm%m6w6*M@#>RbXG+mD{PB5T78}woARsB1Kc(5?4rF<1TI{mXhYgCY zPMnctyUTQ~SYm!bKwELrt+JD#H?<&#R(n?o6mp{SdA1(%{k=NRmWzoy z74UMO1KOl(@=HE`s^Zm7G)v#o4vyl(Lan_kGrW|H9$+?`24Q&*N=z^R?1T<*X`v>;K?y>UgY4vTb3M5kzjp9k2Ee3Xp6kIB-7MNFrYe zUK86cACb#88bYG1hFE`YfZ+n#@&fG%g~E|HHc0apV>pOl){Np~#~ud%G;j}%P^^o$ zH}zP3@i%D**iV%-r#he75L;CQvW%!)zWn#4Cd@UrPB&A@vu%?me^~Xm$CFb@0F829 z-`aj3$sM|RAQNnTk>vipmhoo^bza#h5N1{w z8d)nbbtYO#S+jtuxsuE2tpqM=|Bn2l=)(e5zNJoMm?i>mzh{k#D8BS|0?0m#jL&|k zqFlJ-z$70%GhH?vCvGk;GN6LvzG#NVsQ%cu_j3J$qt> zl90SoMC#0utlr7nD%H%Mo`~$_tilrVtTcB%%q*;27Tbr-$G2g18lCEaTM}zH(fkuf z@R{@dMiQx!>sI>Y$^YCrM(oH9OiW2z&9T^Wy2;eOX?mMxq1bHW;3MUCv~ z#E!S4lS^qnbZK2cqr0h<-eI`9>u5JVp3|S9>Oxs zkr&6dwiK%~M|p!&3Kp%gQ{eAp(|uR#6qfPfQ?e z;kXkJSxLw<%jOfLI7%$6`^ft ze^R6xstgY@9A+AD2@^bnW$9P4&Zb|S^hH7D$z_>)W$j`@gJPN6#<0kD*U_r!~hl42x{dkLwRf~b~WD?DN zQoQiEBRW%qaV3E%<@k^;0;t9vNF0l9Bx&j&XY3)X2C4N#nwJ%KzZXs9b9H`Zamdd? z5~JJ(Jv5YkmLzK-<}H&-l|68pnK7z;<7vYYi?>PF(N>9xs&JDmV+n7p(7p;8w9aTA z#vXEl3{h{m23SZ+J!Yia0?tR5p)Zr8>SNp z2r0t?QeWQ5`URLJ{`Uo~PfxHvAV4-oHskg(`2#`#l4B0^$AuePH?zN7KS?$UKuP%1 zA!Y$X6|Jtd#nrdWq8(*l={H`{6!_X8;%(|gj^zL&;Wv3H$bHLOWsYb7Z_O+DhJSs6 z`-0AG_?6#CN?p&3? zf7~ojb8CNeS`o*s+Syi1y;iufr+eJB3hv?C7`d!XVzkFqF?(oy=9AJ{M zlS-D`^jmC2a9^RPZ?$+PN!)hp0vMFuNb}k9XJ=X5<7}+h*j3?|+rZv}fv&E#jC6nN zbzcS!qM`gclZg@va<$uDubeN|jqsls>vXZB*KR8Py|$7yFH`>mVwu^Vv}3J&W&xYM z1B_}g=oa!xoqtsirM?F~z(3WpHEv8&<`;;(^Zaq+94Y7J4<1aZeK56aV$pM2?gStW z=OaB~w&;K045sd&>+)3j1H*e)7GZCkJboExYv*<6MT4#|Wn{QJN-A^20A|r0CHR^| z5jdHv`GvPeK%T*!8@E(ryVY-EV-=!h58Een^QUZ?ir*h#QO&J!a(*-4Z9A*ou3xM| zo>CC;jlYE|7wno4&WZigD%{nkpnj5;oFIjq4Nx=?(;ged z!t@Rij}ccUSQ74IYRKnMw!{`0VU=IgDn?SNhyCgm%R>RyDJr2IkZ1vcJ&ZpPsqpW~ z)0pJ>7nT10f_zAq??eFBmzbb*7NMfZTdD{oyN;4hj+EP6aGS6&=t9=C=a(AzGw_jK zy}cCzdaGRV5Lg$;{Shqcel;M8n@C$Cn8{S^M9{Db@7V}Z`%Y{MIx9Fua7IUW1dq<+ zah5;QPMaR_a7vjE1qS+(AIUkIBq@dPG>B3%kXDLxUyzww5}|hDxw0u#hLJKsnt}$= zYOJ}|YWFzkN#eJ;zrq6mK@q2wGduP!BwrXbGcx5v<4lLn0+dCQy5d8*%!>|_wBbpU z%WqP@2~^>LJK;fGWbuPqSqKqMJI*7VNq`CABbuQjANS=f5*-z@tT2Tur| z?c$b#VUf28mJ0T)gcE`5iz1f-?@^wW5mQvsX9AFtChEvHbZEFlkA^QW*Zfm(^Y4ZN z_3AFc@bCyh`BNk+M3C?+b6p{_YVn+0T$yz?;y!26fT0910lwivr@=9MKplL6JGp}F zO#yh!Fd|W8D@JGj!E}#4WZYC|T1dm{f+LVm{9Sp4cO6=L$O+m-HD~L_NddUoMS1)l zU@ng6;uZr)416JP)0oamj4$pyk43Oiz;10k=94|Wt(*4f`=fNVj_nf$_d9bAfJjpD zYp%(jm#T^ftY_@#TgrB_`CTtd> zT2uH?tXP^OU)pvuo-2(}PL7}>riKc{{ONVWP_iW6S6@yenw9;L{&MDAIzroZ=SFpu zE>7E#v)Dva1Z0>WFv1l^=I^ny5q7wt)9XTu4=zMRnOWYfbU$QtwFkbvQz?&9$jS8l zu_f6{ZSds}(g>^oZvBjaZLSrao~7|by|SslvLo{Uyds5~r@P8u4P8Of@t7^``m=*l z-djw}p^g+hlhap9qqvb+-8L-K_YR^~UKT@T>j`l_Q4C*(ys7-EKw8 z;*7>Q!*yiykQXO@G4a-mG%8mH!5}bp$d84?D^v=e6*GP#iw&@EXn}CRbqKA%JP}%Cd6n3KA^y#_>Zp$g({iNb zGf2G_?@>la4b1%H*`pfrjxDs9MYE?O*6B6oZ7<*O+8=`AJu69`4slDHd(Rtv->3)@ z;AkH-9)ShK+cpO7JV2)dj|1ig1EpV9GtuKza*hqe%!=7X<ob2)f7Gj1< z3W;j6-Qg?`r92L#tObT9QR=*2n9EDeBng$`HREo}M3`cTX!~{4)Q#CSEkDh|SMdrs zH~EU77E0bGRTZ=*7z&mSErvJ+u#vas$nloDgjp2zgp}!k&#@grtX=HxdAp$qu zpyJh7nAn(RkZb5-ANP;?&syraRJ7zHb20&;Z#jZ-+kf_-`qxli5nU@6N8rPs^TK`I z=Y7g_RYwVcnauyF>t-<4?7TZ-@ehWRl69m|&!_?q;CG@UJ2)r4ea9ZIg5}-DALH}t zXMcGAIev_y#|@ACEs8}2Wf>QU*JL!gchFP}Hnqs&b{R7DgYA*e7vMeL9mm)E`9R!o zT7cfLdi;$)`3VeVYgqyBNfHWQ?v0*r6za zG^r_w?@k?)iQ2(VPx~I&OG~K0Sv7N0F69$%Lf%?Wh^o9Msj`qqvz!R9LRAb) zTYbqOeDwwOMapY*KTvRC<78$QNkY=cH@Qv%a4=l-T&V-mC2rU^Y74}Y06w}9B3czp ze7?e|XmkgQ9IJ{_9m&k-y6o&~nqEx3fG)8?Onc2MNR10BsdNIWp~?aGPbi!5lV9? z6sS%r4`c~BB$0CyOlu9r~d~6AliTb z;{O*15bwYLhM|YEiJp_QqlK-R-aohVFSgAg%9FO63=%U9-RZAa=;y0gND3)*%khTWcg zUYE=QrZR*@Q&bf+l9UexEU6$o9EDVZ>oq}p>9x+X!Lc2z$Dm-e5*m8f1-MrNw*GgB z7qOSbr?#6*02-P?WQ-L;?Lai+DD5IReV?;ikkwD+J(1q1czG14k`z+oU65RE_q+(B zqXMA|Hq?^Sg>Zoc3VYdYHITiDCfur1=&l1qy`JE3rH`Qm!_pi@E|^@(>x}Dl4@^|U z2fMVvi+DU}l{PmZy~ZGxqq@)=Kp(9lrVUX(2~;-CbHdXoIf^UR&KG*tqXfet=aB%<1G99Q5iRFps*=%q={afl?`# z@s5Yw34ElTOWnCc#QlJ?vzfenu=BMQkiOnSzrB4|jIuN0KSt97!-n)ip+PeyHu!5x zab|)W$rVkR4mNg}1MLXZ{?v?z5cg6So+F^&kGBuDmjzq zk%JIPl4 zeR2NgF1J%7?$4mjP(3a|?rhQoXln^$6|#V0RJgQEkm$Ls`%)jQnIIC@)Z*vnuTFlt zQpl)@MXZh`m(jQD>JdnnJM18aAj)gkoCTdEuZ2z%Whm;JTCHM(@uk6fl8W0{w(qe+ z(RG(_=yg%{tA6(?D&`Ca`YpkTI*qx{HJUZj_tP6tqxn8p04-o6%0Ju0pO?~UZ~}pqj?uOsOYnRJFMoxZ@EoitE%Bm#)G`+w6PT=Oj!7BDN#H$9X&ppeBySyfl zE7O2V!+&O321j{77R|MxE)|xr5WN5D~!+1Yd5$0&`> zUpSIx_*mYkSjpgp^rVcvJrV_-}M}5Yx@qUzX6OJ|e_t$Ge#^mC5H_sry z<&iRfkfWO=f@{-&RlqF_A1UT>9Imp|HBQF{s34fd+8|N?cWox z5eT)s$IIjPeB>|1ci))!Ec&;5-hiVf?wf$fviRw4c$!{;Yy)O|ExI@I-ZKhsRA75n zI9{_HS2?Cb#V?10z7q11s((9Zp?a{%cq&t^gEKWmhkk(n$gAPwg9P+1xDN#JKgkRC zzsu{t@O=i3js_n8PG2!9o3a6ra(es2!(+ zJ%Qo&W24uwh|t6b2c}ZOg#^#3RLTJ)B@oE!*I0wd< z2a_ua5buGCUCwe^wZxNvpz@Ibfmu)YI+rkcPm5w>y|1etA{j1UlHx zN>AHx4ShC<-<`(KjXORWnOZ{0o_oIM3MAg?_bVfaopiZ0@kk+QD$}rLC#0pJGiAny zMacEYb-h-#xS%pSh7Ha4w{a7i$S!QOMtV0x&@wfl9j=IAk5Kq^*&#Hr$rG{CfBsj^vU z((Z{v9;z46%&jvHh}3zl%OWnR_Hr3IbYuw!m5wpIS%x#555iIf5d^V#bvRVxEw!w8 z>PJ_mDpe7HK#`|VWiHRIeHtnrfh%)Bpy)}Ar>}+4tQu|6180hcgB9*He1DRHHFiVz zK&L39UIIW1BIwbpG;*CI#3AgeTh=b&vgZN2W`pX3FQVD4JL?E_-FWi}axMaRIm7jc zWs(y4r5?>#IZsw_KYa`d$#woiqK%h^(4T7RK%?B#doj&-dVca){vbQuaCr6pY9fVv z%s!?-ov&v84QZZ!?t|R%=BUTSJbtSFMd-pg8AnM3PnGP0Z>$n{EeWz?U2Qt-ceLWd z2{OIPvmhp_DZ$0MFfi(ZTDpghzu)+xy}1FQ>JBMJv!rI@}geIJv`Oc=CtAjh7 z^o;(Gv;-DW+W`Jb>k0fnNsHiLNXy8;+S<^-$m-vz>smug4x1Cj_ivTyEZ*2^P^bEs)j^Vez{_1WgLI)NGKN_o_?UxfeKe>8 zj&M*sbcCU(&JE&dX(Ggxm`!jg;;WHFc5p$0Jy<=x1x1@wL3o45h;M2Y%R;UKkBkKp zod>KvZ;)>RI+LRWa3T`aRU)OU9rJ*^JQ?o89$PAZF^NJbxpr53y$)Wmx+x7%BPXM) z-5sv$iGH6I(h? zlR~iBP(+Q<`MuYQISUVyP(pB&Kky#GlR;8AH&N9}Cs?4F1=;3skDZP!l}eUP)B6VO zej$n8W0{#!kd%q&3&lGVgivmWy5bC5ttR!-t8d-+B7(N4W5PkYe~2pZnlG4t)Sf}< zCiDuJ(^y}C*`5ex=4kvLgR|l`W+tt(gG0cJ?EL5=kZN?Pd zP)@gI0XJW`OPhSq&o9VjI{0FUe*hxlV0hvxFdE*TLfe=?t74DuhZdJjg+Dd0=h-?e z3OB#SX2d1S@VJCp%XY;qJ=!y~wcby&A%*~#r%qq7L%vWJUN&{@N*`T2H>ceo`Mq92 zs&`~#3y`t_#WRQqq(${>CQW_b*lHbxuAFYFt-udIDVZX5^Rw{5EiuFr!8@VH`qphx zwF)$mi`Po-w)+M5LIUI?Mds6$`tm4P4KDE4nGb;~2(5`GU>rmMHN#stEav#(`zvU(J09#2#$jQJu7VOA*h=Z9XuWVnK&GRg@w5oDeI^% z?kKRJeyx=`3Aobk#vNHGsD@fb0-H0VV$q%II}HiQ!z&dK|ry)(A6wlqm}FZQ+1DT^gmDaQLP6QvLX)_3E_&gQ>p2Hge8U_goOJ660+Ksvbf@xxk%6l4&r+F z5_>5zHDX1F6KiVmwX$Bj%cL}j`c?mwk}OC_Kv@k^u!j5cp;T4#s6>&{2`_r<*>@%b z2=Z<+Q@Y=6d6y^-nNC2W!F0=favMi1!~ru4W)7zwSXh$a1Qt2~?R-h!P%ffPaS++M{*zg>PbCiBwR9t1U}C&7kk)J}AmGwxfG8HP!@5c#5)p#c- z^fj&$C2}0A(nggN4q3yaCx_g3`d!ohxsSnyAUne9)0{uQ4yN>q0E-1F4gdwja^vpF$6sMSH zChKowp^btNQM{IMKS7#0yzO4rc>e6XOho^F;7xVDUvg>yY#`%@Y2_ScAxcmQ-{q;= zj9ZN*K_L@Pua}=%#*7z~@2h{(M=J^QFBixyxY^_8gLmlPn@m%dBcm96bFu4>?$?a_ z%d=H#st~63qwW^3RBx`?z&xPmR--oeqg?2@(dAWLXi#RNm?^!K^I7-$31ED`Rt?h~ zqYY}9U(9#b?{+_$O1{OKcoa#2$jwEAX4A=VZ;4tawQ(g83l^y-5Z5(={6S1MMr|%P z>0e_ZMW7X4Mg%)i@Jk0~qhkNQkElWth@S!c)})mpL`lZ6TCD=6_oB3VVU7V)xs}F* zThdei@M${d04$kuU_&-&XD(CGe}n$Vp|`p|e1-qI^8)UFa%lX2;n4pR+V*b^n&;AW zb7cPK9R>Rwyf0Z0RK5ZCcVP|{6Y69$U9NcNIirt#>YidSED#NVr0GuC>^%C%-*j-_ zfn<1MN=fs9_>rM4H#0Z0*C5^cheI4k|I)Kv+}Owb`VO`Axq0RaCoE>ndNRl_k7*pk zo7eE>-XEpXN41W~Qvr8jGM4DmDWfP>vGkDsz_k}^)LKE)%^@9{AOrNz!y#L>#^F_Sbxo`t~H&|fWZQYVA;G{5h);CVIVH3 zY&c}t>lXsCOfB?c9a7YeoZuxUGh@oc5`# z$T(;)vas1hxPJTy$BXulk*{O(nrIt%E#%EkpaApx2c{dym0n0ah z?%y-^_KybkMtXYN(PNDDa35d3W-J)7OK6o&UvWXLFPlm5`&)`#k&F`c)oM}Dmw5u&70Tvx)!1Lr%(8iEn&akABUcs>Bd-0n1LR`^p(u; z))1j@)yv%CIBZYYLWOwce%gZCxOp2IPigk$bd}9dbnpT42}>Iv%Tq&WA<3&;$7_4P zC3W=@CMyv+J3HP@7>Ej~A6g&XZpY51{k3Ui_AREtzc1{e0n$)^$-q141FMv=_IU`j`$NZI?R zkESwW-tp<@!7#@}aAi*2o1-KkLO4>9fUEQF-3vBsudk2e3DAVfbHb_%e-h*fNwbZ8 zgiovDWq=aiBD4U?h7oi3PlP1{OKvfXK)Wu6G8~PxREtezJr}VN^oSsKn~ZL7!8F;D zF#usYbHVO&A}FF{f$t|=rl+g^5d09x;qyPhdgzBIEQH3p#f?*F*Gt>)3qh2QwSvG_ z5cE3&gFKe@8P(@~D_ipWD#<^fF(UDPL;gv7?}LenCx8ZzyU~HD+%UY78||qWZVJ-B zQ=bnRlHD(|Ql-3+n`@V^O{IA(w#tw~kd=F=?yombeq*`-bh@y5S_^D(g*2UYm&xWF zRK7t5N9cc49~9;BLAcgG)IL0ZZF8`c?|W? zWr!W!-wm1r=8YXiz*hTpc-As|lOO+(k+W!XZ-AOmgY!K*%UT*bBq&6NE3Y7$-AD)L z?`=-uOh9Q8yG$iIi+rBIOu?xU8ev;%h>hT9B1yTRhFZjF#E18RdAAnkhoJL#j8O$c)=N);wtcwUptT!0`U2O zCh%;4_>T^g0i2Eif|EfGAmG#>ClCk+6rru9HQG=}QHu_G-)z;OnM&qQ)z%S%muT!Q zo;ma=G?6L}N;{(J%>1$e>d9w7`XR1S?IX$hYGj$`_|0Sh>v1AcN8(?p^H>>5ZHzwM)rmgUq_^HEmb{LxDST>6CEHaoAkfQWPGpC%WMjE1v2uodJ40Vt|jE=G085;Ys6lyqlkb5V$7?s z^_Ov=@<&{m4$ipL3O%TrWVFW9#)a(CTVvA}=eiC*Qra2C$y$H$$#x%eW=4j~>xW&M4G8m~0t}Yaw4X~JL z=Ab~Z-6)yuL|!rxSl=G4yBM*-!tZLv=z#g_*~)d8*KiaTU~dF9kY)#tm(hClwgdd; ziG89*H_VQ(p#y$~g{aCn)~LFl0L97qzx3iT{Ht9{oLv5!6Re7>58y|ik9oZY3K(u# z#98t54;uiP`i$DFk`5X^M1#N}^$(TUk>4fr?I6PA`BVsrq)`1cB%`B_5`_rVe&De5 zZdy!M60e2RKwgV5^&p@sl$FnH(8!`Hz!*(DS$7r!{?@%_ElfYj)`uBl78 z0eZz)#&s^Ko~xoEJ{3C7p}t%S+ag=4+!j9*xdjp#Ik#3FdA@H(oB}UWqK`_u5{V>8Pu1RQ=76ekqagc($?9>Z0;y`Tx|Uw$jcK((QYU0 zEy--@Mqi54f0j<2`@QwO*kO5p_sf-+UH4~yGECL;OOZx zLHS@v_E%6|i0nKnKB?Q+fw->4TIr~3IVgQ`=kZSDVr(39(%Xmmp*{O$1vGdRfwrD} z?oI>f;~}57y88fPZqp7|D6%^qUI8_Ftb_FLcVH_2g+FmB(06Z`WgTO>xQx{7FxW|D z>h&aIRJoU|Bv^cElx*{3G31{T7Gzw1-}%%JQM>*^CQqBiQUYE^%pJ(b<|a(A&!|_+ zA@v)#Ew-gPucdig`9-_Bl&qLGDK)zKFX?0;6(?&%wExYK*sJkoOn01?iguoOxZOF0 zntW7zRmv6X4MDMhd^7%Aui9e=^!vJ9kIVW`TcV0a){s$yjM7pNPLl~<7mxe0pTk`R zFPxsX0wOj8=as^z%Z``TP$fIqCHvX*TNt+3IKS^S*C<#R>awy6-P~I;HvI;^|{7ehXD!4^}Lk z0F~hSqvwXFWFrPCI;4mwb4vd%zY^Ic{H=H8RWe<$yf(+=W(|qW-k+4_hf1vy_0qt7 z@X$AY-gHReNV)J!9aIKYrEq^9XkpJTbwq6}VyHkZs z1mnr5_DDT4SlQ~&;i}~Y!rgn$7vn@Qcy#er#S+fz^upy0E@Ox3ms16?Iv;ikOJ#HE_8`(r(IT&WaB+cb!VVllg$BvU)u5Gn<5h=3ES?j zF0`1lyUuZ_*lKeh3${tS%^H?^vHvh>VUpXnB~i+J&cSzo0mRubESq;D>-FA4W~X0| z`~8p{%3LgeG8L4Bpo#fx=YTg16xkW~0v;pL7*`C)QS=yAmS<5NR%>WODPb6$7CS$t%ahdz4lg z+bHD9X%}dM$&+A_qUrREaeBf5=O3$W^W^yYRQG4(V=V(7X%2(#`3%y?K7w~3It-Ty zNDK+Pq%c(xRkfKgur%X6sP7*P!H?@}q}PT(WAGl+licj!WKCDqyJU4dVUs;1$T_RL z!r`mexY^Y1swj;B3}m4Zd#+BS*L?;dJv{ha__lwz`O5nczCaf9F8U5svA)9|ZP=s0 z4Nsp>LC<}JjZ;*U%I~3=Ny4^hw73g?KWPqQbK{eGwRmvZ3X6iL>!fdZSm$L!f8Xb} z=J)-8Q)n8-`b8atbI7DGz6DmjY?-4pR^G(lc!r^+I9tx|KAdFE?@_5Y9!0QZD)2N4 zPShUY0G1;B40&x?s&c@>mxVnP!p>-2rO2yrZ!_?rly7^0h^8H4znvf;ARN7K8T?UcV|q#T(Q^1}oCm-FRgduK<^hK7L$*vHgeYN>#3 z9g_&GWT(Rzi#)og!8y?C2)D6gggZb!1bhSAmizA9(E}eXC3a0uXPvI7`boQA7nJ>f z2*W|O!NB%cs;)6~E!#`>@C>44Hm*O~P-tSChvPKcM~f|1AIX0#SV_dL~|G@&H`AaCeHdO$-hR3dm5S+5{s97KdrJpZ{1P$@-on3r}{7 zb!>m&KC;9{RhI8k+4PH>9ruNDn=K6*g6+ab+4C9i5P=cXv5fKGsa!c_@H*l9dNOdT zeXP?OYMMw3AC8Uh+WI1?7R>L4imr6-2YR;OxL9(hJ;j92-19XjOW`UT6GzxT41y7T zl@b`9?95e(t1s@zz~l|XWv^gU2lX3Ncf(#LENKsXlNqcou41V|?t^(v~UyVt#i4xVpE>usFb8CHY= z(uM&QVe8McW=&dbS^;tMOxh<9_8zGi3U~g14a{bMud9Z9`n=1ER+|-NE#T{$fl|GY zH=My=HnMaPm={TJ2Wzi7W`Gev-7j6lHh6z}Qkv2oIe_$feL+iC_YR+7yxYE=*)jsw zS>G-XE}*7n$oEbbpT~{4K+*Bt9oRBv%3h5pzDjfDWXY10@W#LootsoP&UlH8G6WP@n5F+%q^ zh~7ZFnq-&?gGx#+7BtR5LMbAq7K6vk^k5S@i&Dfaa&)}?;(#{jT?Is{l}y#TWRHdB zZcuz&u+E@(3Djf=OOsx6XD58;T4wXjI$4x2I7yu+Y!^0U$+2xkrOeK&F{h?7rpbAo z5g$~9@X-Q`9*t>a5mV?O)-;DQux0}x6z+>62qC~qm8UIi8|({~jEn+mdq!f_grv=h zNCwnVwFKp=PJz7Gpz!C73Ow^ce+p^8j zeOz&YF3li6zDES|AX(Gq%8EV+C53KE&vUG~iB>C>>El6YoaBEj;kEvdSy0EBy|@_a zon9|P-9#bX+!pWmP#pM{W}aBh)f!S{W|FOzRzC`6;4oJ^l?a*4l+9#2AUN(#t5Lx^~mdG_M{O3_-apD@!LtwWDJy85>QUJ9{Dh5;Or0j+ENNBvSfQ6|8Gt$xx-U z7HVkfzF1yo=T-Ra4K8DI*&vfc4k!MIf}S&}1hwxWNvQG^ouD8=Z^n(%U86B5$HK;6 zSJtR>gePmp=4gUswCq)RIjmY{Xj($g2fVvtDq5E~U!_km@@PErpKG?m~3D zS+KRVgd`U>T}4F3$|_pLhoy>`l!-O9R!3*1Xk>I0J)fs(hNPr4zq=0W1ND#7(*44R!k-LzuOxElY$@In(K^2hFV=NQdooZaW9`pb z4u+Uadz&z0v>y$0d$cDLcja-QcA#Q&^1fhLFC-&_Sk-IS%R{%57+T0xU!iPj1wO*6 zo^729yf_S?UA*v=;b^?+|G zXs$j1?FZ(4>~Z8}G>+r#MH1#X1Vlg!m^aAA05Bih1SBy{(41d06c1KTsq(TzTh(L* zNl554`W6=+Y}4uo0iFq!Pe=8f0g|m|y$jN}cBJ_CpM+C}Ya*Up*%H1oPb^E@XO zed_f6IKvJM9t_w@RldMH&^=bxB6tTrfe$N=<`S54ITf9pf+%D|QM6Ia3gF=~&K!Q;0Pf>+)W@-t z!Wj|OblLNj6eSgQ(!u@q%w~dDAiI4c_`@1TH^5a@pk|_`lIu}@=u5)q^9_)mbg%UN zG7Aqrk;Dqhq)j=}cf3|=${cGVlw;sI9|~P2j|+SUx!~T05%M5nM|-=#Bj}$di23t_M-7Am z`+)B^caMmHM*tlk-KoH1Bp)8uT!T6o0WNi{_v2@4G!Q2v@jMA7EbwgGvP>MVPAy0; z)7kPKC@7Hegfb&Am2AX$Vf9<{e zd5TiX-qK!E;f0M730RfYfrR90KpsoS$v0CWXpSgdfJ>h6lLh@)iO%a66J7>qiCao1 ztQ3#CkLPtM&P0+^5;=@?OOiAwO)4w-)$KI(u;cUO9>u=yIE>}KpXfJN&aX_2RCa3Fa6cN{3&MP!YO5^J$DEM|N^ZNjUSya#zhEazcshz- zp%^CO znUtdHxFpr7?>72!%ps5wph;Q zb5?;9e#rXuR$2eebepN6!8Jf;7I|rbp2J(D{?CsvbZWLaYKY^v`30eWp!m=5Mac2m1f@G2vwL z@BetK>BJqdq4>owO4SA%;hmXj5ujK4>$W*P*P zWQOs``gi^$n?|H2N!GVU#ojJBLLkdj4s&GD6)$B>#8%ce$Ghi5D`Fd;f+vfaaWpWL zkqEuSCA+Hf=40XTqV#*VQ7I|{**Ug3T0p0GDreW`i+0zcr6eck%_-$6)|$Pz{KSIz zIfAdT1UOM|ce}oGpI*q7FWS~C?;R>*=g5>YS8>!KwGGH794EB*kp#IZ+%jz9`4myJeM6B<`?S&I^)YKW#6@vB_9p=APw?;V3vBx?^|6fj~o+gz4`%r z(73obhMa-fMTlQAy*t5(!b)%FkH$drHW06RrG@rvL9@A$7AP}B4{&B7KUW4UP=gK$ z6hQ*btaPY4g_upxHz_l|^(U#!AtdoAOaomA+Ey4RqOr!3H)KRT+m`4)+}I8=UUwp4 z=6FnMh3X0S6$RpB&seAe4Zuioa_(dy)3j@s?lOA0NHX?ef0z8o{%Ba16JL+^&Vw^0 zg~~A}v$JU8#Dz!L?}j?>$*B+@4i2sDKbi!lH_bmn#1h4;0y-6Rzxl_}NR@)bkHtT9 z5sl(286oQrPU357798sXR(B^jx&f+G1+zAEf2(Y4%vha$UDHvmbnCtc-OB@LN@woprEEGjgZuMxUncj3+TFpO| z8WaGCzZXZ?r!c_(@%g{9&NeRA&KAz*jwS}i|EAQktY#y(DUR@cqgOA(x0b3H)v6@N zE7HNzCXa%Tx?a_+zB_`+pAIgn}agMv7qDc+e_VeX1jQpa~OMWy@)}l z_(hEcO5_u_SA7~<8#@m@{2C{=#Gga8nN4i@`FozcL8g+YAd4F(l%Xg{dFYFoim2SF znDYX$<)zPl8LVnXP5`)Ia3FMZ%_D8&gdlyv^|yx#{fvpBN^CINk6LZMoRw4`EU2QqPgn1__FV(Viwe zF7spEQ@+C=24ZpZxf#xmQXHu3);Xw!6ReA)h|W0ss%kG&Guu&Zwk6AP3yYI~_759N zH?SPHtEXSPV;hXT!H1meC;0M(4hp4d>7ahb(S>GM8fiQ7cz0su>6_uPf^pvSdN#0( z|DIjYuX@?^F5m8?zG;4gV=Bv~kH$KuurJn93X+U90{%Y1%No-@Y+v-B$2@hWq0~>nIu5xwQwj>Tmy|rs6HnNwwV_&!7sd$ZQgTtb($34ayN==#c)_#_M zoAeGj5>>lISHl#MNOHbMT9sH~1pN z!aeO?CxzFWhW}={k>|tIljSw)&~vv3@wA|$bvtLb98O3C79@q$0lRL}ex}8sIXgZS zg>%?>$=Wr<(FgDCx`$ltmsC@9qZXoRc1?=$lz0=O-&~3pZpjtRRJVXE_fAh36t59< zJMnO3(f6}?7^Wl2?j3pU{R~_5)zb*pcm{rsnoQ9*I;pZ&=)WjB6P$6CsVo6Z-WaOv zTwHombY*?;LO*5{^C$9Tu@ z>GF(rIbrog#fpv=b-U;LYY6FOV*YchK{T?Y5mB9tGN_9C;1xyKSn;kiOdojfR%i3L zWJvOJ$r}&i2$yNKL7t5R1J3=(wZ03oaLqsO_J>?1H4V$@odb2VA9 z@=4Xd+F&9%4-p;r-#r$|-aug!D zR-MSm!Z}WL#Yt2nh=wahGD*Iid2?xHPvM8UZEtX2{9`s|LQP15QW2`ccY{2`9O8cHOH>UXtt0X`#9wRv@MnUSB=G% zQ?bQ!^8Ev{QtyS&3rdw$bW~pLW&G4>V<(fktUFMjVQ>5{0-v0C{wcECf3SDN`Dym; zNT0ZX@sN%hrMc5rJa2_5Q7%Wq4Wm=cA_#Ncb{$pba0^1Ar?9*m3T;O9(K4BrvmwiD zxtoAR&LSDGk?ud*i0iA3F1iK;diwe9;aQ0($I)`qZRGaE6ss}|%!!UC9m-wG<`Zzh z^i0gS;butz;5PigfkE$sAE09e&S1Ne3I-U^K7ILU&1i3(Y_&mvUY8jbKH7`jI{xW$ z1n6>7l=9chA*$W)N#C*?arp@sDqx2k1{jPJLh}Dga`yAVJAP5@(&lhnS~e;M+FA+% z2IGcs%7KS^`w#61FSjj88wK7dnID3>4-CL;qqgTNfYrCWU7dq~stNoTB6w{tUI_8= zelQWd-7@X+fz{Vg<70r@+b$nub+GSAdwT&Js9ZDJ>!yE`zG}CB3~~?zAw{WRAdnyQ z2(YO3FoYKhf_8(U6MeL)-~kOmNOAqb zc2gl^owT9g&K4A!8Mzw@85yAs1!sFuPcV&vK=>6wFo*$Yy Date: Tue, 29 Jul 2014 20:58:05 -0700 Subject: [PATCH 0119/1492] [SPARK-2054][SQL] Code Generation for Expression Evaluation Adds a new method for evaluating expressions using code that is generated though Scala reflection. This functionality is configured by the SQLConf option `spark.sql.codegen` and is currently turned off by default. Evaluation can be done in several specialized ways: - *Projection* - Given an input row, produce a new row from a set of expressions that define each column in terms of the input row. This can either produce a new Row object or perform the projection in-place on an existing Row (MutableProjection). - *Ordering* - Compares two rows based on a list of `SortOrder` expressions - *Condition* - Returns `true` or `false` given an input row. For each of the above operations there is both a Generated and Interpreted version. When generation for a given expression type is undefined, the code generator falls back on calling the `eval` function of the expression class. Even without custom code, there is still a potential speed up, as loops are unrolled and code can still be inlined by JIT. This PR also contains a new type of Aggregation operator, `GeneratedAggregate`, that performs aggregation by using generated `Projection` code. Currently the required expression rewriting only works for simple aggregations like `SUM` and `COUNT`. This functionality will be extended in a future PR. This PR also performs several clean ups that simplified the implementation: - The notion of `Binding` all expressions in a tree automatically before query execution has been removed. Instead it is the responsibly of an operator to provide the input schema when creating one of the specialized evaluators defined above. In cases when the standard eval method is going to be called, binding can still be done manually using `BindReferences`. There are a few reasons for this change: First, there were many operators where it just didn't work before. For example, operators with more than one child, and operators like aggregation that do significant rewriting of the expression. Second, the semantics of equality with `BoundReferences` are broken. Specifically, we have had a few bugs where partitioning breaks because of the binding. - A copy of the current `SQLContext` is automatically propagated to all `SparkPlan` nodes by the query planner. Before this was done ad-hoc for the nodes that needed this. However, this required a lot of boilerplate as one had to always remember to make it `transient` and also had to modify the `otherCopyArgs`. Author: Michael Armbrust Closes #993 from marmbrus/newCodeGen and squashes the following commits: 96ef82c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into newCodeGen f34122d [Michael Armbrust] Merge remote-tracking branch 'apache/master' into newCodeGen 67b1c48 [Michael Armbrust] Use conf variable in SQLConf object 4bdc42c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen 41a40c9 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen de22aac [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen fed3634 [Michael Armbrust] Inspectors are not serializable. ef8d42b [Michael Armbrust] comments 533fdfd [Michael Armbrust] More logging of expression rewriting for GeneratedAggregate. 3cd773e [Michael Armbrust] Allow codegen for Generate. 64b2ee1 [Michael Armbrust] Implement copy 3587460 [Michael Armbrust] Drop unused string builder function. 9cce346 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen 1a61293 [Michael Armbrust] Address review comments. 0672e8a [Michael Armbrust] Address comments. 1ec2d6e [Michael Armbrust] Address comments 033abc6 [Michael Armbrust] off by default 4771fab [Michael Armbrust] Docs, more test coverage. d30fee2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen d2ad5c5 [Michael Armbrust] Refactor putting SQLContext into SparkPlan. Fix ordering, other test cases. be2cd6b [Michael Armbrust] WIP: Remove old method for reference binding, more work on configuration. bc88ecd [Michael Armbrust] Style 6cc97ca [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen 4220f1e [Michael Armbrust] Better config, docs, etc. ca6cc6b [Michael Armbrust] WIP 9d67d85 [Michael Armbrust] Fix hive planner fc522d5 [Michael Armbrust] Hook generated aggregation in to the planner. e742640 [Michael Armbrust] Remove unneeded changes and code. 675e679 [Michael Armbrust] Upgrade paradise. 0093376 [Michael Armbrust] Comment / indenting cleanup. d81f998 [Michael Armbrust] include schema for binding. 0e889e8 [Michael Armbrust] Use typeOf instead tq f623ffd [Michael Armbrust] Quiet logging from test suite. efad14f [Michael Armbrust] Remove some half finished functions. 92e74a4 [Michael Armbrust] add overrides a2b5408 [Michael Armbrust] WIP: Code generation with scala reflection. --- pom.xml | 10 + project/SparkBuild.scala | 11 +- sql/catalyst/pom.xml | 9 + .../spark/sql/catalyst/dsl/package.scala | 2 +- .../catalyst/expressions/BoundAttribute.scala | 50 +- .../sql/catalyst/expressions/Projection.scala | 39 +- .../spark/sql/catalyst/expressions/Row.scala | 40 +- .../sql/catalyst/expressions/ScalaUdf.scala | 1 + .../expressions/codegen/CodeGenerator.scala | 468 ++++++++++++++++++ .../codegen/GenerateMutableProjection.scala | 76 +++ .../codegen/GenerateOrdering.scala | 98 ++++ .../codegen/GeneratePredicate.scala | 48 ++ .../codegen/GenerateProjection.scala | 219 ++++++++ .../expressions/codegen/package.scala | 80 +++ .../sql/catalyst/expressions/package.scala | 28 +- .../sql/catalyst/expressions/predicates.scala | 3 + .../apache/spark/sql/catalyst/package.scala | 27 + .../sql/catalyst/planning/patterns.scala | 71 +++ .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../sql/catalyst/plans/logical/commands.scala | 12 +- .../sql/catalyst/rules/RuleExecutor.scala | 5 +- .../spark/sql/catalyst/types/dataTypes.scala | 18 +- .../ExpressionEvaluationSuite.scala | 55 +- .../GeneratedEvaluationSuite.scala | 69 +++ .../GeneratedMutableEvaluationSuite.scala | 61 +++ .../optimizer/CombiningLimitsSuite.scala | 4 +- .../scala/org/apache/spark/sql/SQLConf.scala | 19 +- .../org/apache/spark/sql/SQLContext.scala | 25 +- .../spark/sql/api/java/JavaSQLContext.scala | 4 +- .../spark/sql/execution/Aggregate.scala | 13 +- .../apache/spark/sql/execution/Exchange.scala | 8 +- .../apache/spark/sql/execution/Generate.scala | 13 +- .../sql/execution/GeneratedAggregate.scala | 200 ++++++++ .../spark/sql/execution/SparkPlan.scala | 81 ++- .../spark/sql/execution/SparkStrategies.scala | 138 +++--- .../spark/sql/execution/basicOperators.scala | 44 +- .../spark/sql/execution/debug/package.scala | 8 +- .../apache/spark/sql/execution/joins.scala | 44 +- .../spark/sql/parquet/ParquetRelation.scala | 18 +- .../sql/parquet/ParquetTableOperations.scala | 14 +- .../spark/sql/parquet/ParquetTestData.scala | 9 +- .../org/apache/spark/sql/QueryTest.scala | 1 + .../spark/sql/execution/PlannerSuite.scala | 8 +- .../apache/spark/sql/execution/TgfSuite.scala | 2 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 5 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 2 +- .../hive/execution/ScriptTransformation.scala | 2 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 6 +- ...se null-0-8ef2f741400830ef889a9dd0c817fe3d | 1 + ...le case-0-f513687d17dcb18546fefa75000a52f2 | 1 + ...le case-0-c264e319c52f1840a32959d552b99e73 | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 11 + 53 files changed, 1889 insertions(+), 297 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala create mode 100644 sql/hive/src/test/resources/golden/case else null-0-8ef2f741400830ef889a9dd0c817fe3d create mode 100644 sql/hive/src/test/resources/golden/double case-0-f513687d17dcb18546fefa75000a52f2 create mode 100644 sql/hive/src/test/resources/golden/single case-0-c264e319c52f1840a32959d552b99e73 diff --git a/pom.xml b/pom.xml index 39538f9660623..ae97bf03c53a2 100644 --- a/pom.xml +++ b/pom.xml @@ -114,6 +114,7 @@ spark 2.10.4 2.10 + 2.0.1 0.18.1 shaded-protobuf org.spark-project.akka @@ -825,6 +826,15 @@ -target ${java.version} + + + + org.scalamacros + paradise_${scala.version} + ${scala.macros.version} + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0a6326e72297a..490fac3cc3646 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -167,6 +167,9 @@ object SparkBuild extends PomBuild { /* Enable unidoc only for the root spark project */ enable(Unidoc.settings)(spark) + /* Catalyst macro settings */ + enable(Catalyst.settings)(catalyst) + /* Spark SQL Core console settings */ enable(SQL.settings)(sql) @@ -189,10 +192,13 @@ object Flume { lazy val settings = sbtavro.SbtAvro.avroSettings } -object SQL { - +object Catalyst { lazy val settings = Seq( + addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full)) +} +object SQL { + lazy val settings = Seq( initialCommands in console := """ |import org.apache.spark.sql.catalyst.analysis._ @@ -207,7 +213,6 @@ object SQL { |import org.apache.spark.sql.test.TestSQLContext._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin ) - } object Hive { diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 531bfddbf237b..54fa96baa1e18 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -36,10 +36,19 @@ + + org.scala-lang + scala-compiler + org.scala-lang scala-reflect + + org.scalamacros + quasiquotes_${scala.binary.version} + ${scala.macros.version} + org.apache.spark spark-core_${scala.binary.version} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 5c8c810d9135a..f44521d6381c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -202,7 +202,7 @@ package object dsl { // Protobuf terminology def required = a.withNullability(false) - def at(ordinal: Int) = BoundReference(ordinal, a) + def at(ordinal: Int) = BoundReference(ordinal, a.dataType, a.nullable) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 9ce1f01056462..a3ebec8082cbd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.trees + import org.apache.spark.sql.Logging /** @@ -28,61 +30,27 @@ import org.apache.spark.sql.Logging * to be retrieved more efficiently. However, since operations like column pruning can change * the layout of intermediate tuples, BindReferences should be run after all such transformations. */ -case class BoundReference(ordinal: Int, baseReference: Attribute) - extends Attribute with trees.LeafNode[Expression] { +case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) + extends Expression with trees.LeafNode[Expression] { type EvaluatedType = Any - override def nullable = baseReference.nullable - override def dataType = baseReference.dataType - override def exprId = baseReference.exprId - override def qualifiers = baseReference.qualifiers - override def name = baseReference.name + override def references = Set.empty - override def newInstance = BoundReference(ordinal, baseReference.newInstance) - override def withNullability(newNullability: Boolean) = - BoundReference(ordinal, baseReference.withNullability(newNullability)) - override def withQualifiers(newQualifiers: Seq[String]) = - BoundReference(ordinal, baseReference.withQualifiers(newQualifiers)) - - override def toString = s"$baseReference:$ordinal" + override def toString = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) } -/** - * Used to denote operators that do their own binding of attributes internally. - */ -trait NoBind { self: trees.TreeNode[_] => } - -class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { - import BindReferences._ - - def apply(plan: TreeNode): TreeNode = { - plan.transform { - case n: NoBind => n.asInstanceOf[TreeNode] - case leafNode if leafNode.children.isEmpty => leafNode - case unaryNode if unaryNode.children.size == 1 => unaryNode.transformExpressions { case e => - bindReference(e, unaryNode.children.head.output) - } - } - } -} - object BindReferences extends Logging { def bindReference[A <: Expression](expression: A, input: Seq[Attribute]): A = { expression.transform { case a: AttributeReference => attachTree(a, "Binding attribute") { val ordinal = input.indexWhere(_.exprId == a.exprId) if (ordinal == -1) { - // TODO: This fallback is required because some operators (such as ScriptTransform) - // produce new attributes that can't be bound. Likely the right thing to do is remove - // this rule and require all operators to explicitly bind to the input schema that - // they specify. - logger.debug(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") - a + sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") } else { - BoundReference(ordinal, a) + BoundReference(ordinal, a.dataType, a.nullable) } } }.asInstanceOf[A] // Kind of a hack, but safe. TODO: Tighten return type when possible. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 2c71d2c7b3563..8fc5896974438 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -17,12 +17,13 @@ package org.apache.spark.sql.catalyst.expressions + /** - * Converts a [[Row]] to another Row given a sequence of expression that define each column of the - * new row. If the schema of the input row is specified, then the given expression will be bound to - * that schema. + * A [[Projection]] that is calculated by calling the `eval` of each of the specified expressions. + * @param expressions a sequence of expressions that determine the value of each column of the + * output row. */ -class Projection(expressions: Seq[Expression]) extends (Row => Row) { +class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) @@ -40,25 +41,25 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { } /** - * Converts a [[Row]] to another Row given a sequence of expression that define each column of th - * new row. If the schema of the input row is specified, then the given expression will be bound to - * that schema. - * - * In contrast to a normal projection, a MutableProjection reuses the same underlying row object - * each time an input row is added. This significantly reduces the cost of calculating the - * projection, but means that it is not safe to hold on to a reference to a [[Row]] after `next()` - * has been called on the [[Iterator]] that produced it. Instead, the user must call `Row.copy()` - * and hold on to the returned [[Row]] before calling `next()`. + * A [[MutableProjection]] that is calculated by calling `eval` on each of the specified + * expressions. + * @param expressions a sequence of expressions that determine the value of each column of the + * output row. */ -case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) { +case class InterpretedMutableProjection(expressions: Seq[Expression]) extends MutableProjection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) private[this] val exprArray = expressions.toArray - private[this] val mutableRow = new GenericMutableRow(exprArray.size) + private[this] var mutableRow: MutableRow = new GenericMutableRow(exprArray.size) def currentValue: Row = mutableRow - def apply(input: Row): Row = { + override def target(row: MutableRow): MutableProjection = { + mutableRow = row + this + } + + override def apply(input: Row): Row = { var i = 0 while (i < exprArray.length) { mutableRow(i) = exprArray(i).eval(input) @@ -76,6 +77,12 @@ class JoinedRow extends Row { private[this] var row1: Row = _ private[this] var row2: Row = _ + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ def apply(r1: Row, r2: Row): Row = { row1 = r1 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 74ae723686cfe..7470cb861b83b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -88,15 +88,6 @@ trait MutableRow extends Row { def setByte(ordinal: Int, value: Byte) def setFloat(ordinal: Int, value: Float) def setString(ordinal: Int, value: String) - - /** - * Experimental - * - * Returns a mutable string builder for the specified column. A given row should return the - * result of any mutations made to the returned buffer next time getString is called for the same - * column. - */ - def getStringBuilder(ordinal: Int): StringBuilder } /** @@ -180,6 +171,35 @@ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { values(i).asInstanceOf[String] } + // Custom hashCode function that matches the efficient code generated version. + override def hashCode(): Int = { + var result: Int = 37 + + var i = 0 + while (i < values.length) { + val update: Int = + if (isNullAt(i)) { + 0 + } else { + apply(i) match { + case b: Boolean => if (b) 0 else 1 + case b: Byte => b.toInt + case s: Short => s.toInt + case i: Int => i + case l: Long => (l ^ (l >>> 32)).toInt + case f: Float => java.lang.Float.floatToIntBits(f) + case d: Double => + val b = java.lang.Double.doubleToLongBits(d) + (b ^ (b >>> 32)).toInt + case other => other.hashCode() + } + } + result = 37 * result + update + i += 1 + } + result + } + def copy() = this } @@ -187,8 +207,6 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { /** No-arg constructor for serialization. */ def this() = this(0) - def getStringBuilder(ordinal: Int): StringBuilder = ??? - override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value } override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 5e089f7618e0a..acddf5e9c7004 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -29,6 +29,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi override def eval(input: Row): Any = { children.size match { + case 0 => function.asInstanceOf[() => Any]() case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) case 2 => function.asInstanceOf[(Any, Any) => Any]( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala new file mode 100644 index 0000000000000..5b398695bf560 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -0,0 +1,468 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import com.google.common.cache.{CacheLoader, CacheBuilder} + +import scala.language.existentials + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ + +/** + * A base class for generators of byte code to perform expression evaluation. Includes a set of + * helpers for referring to Catalyst types and building trees that perform evaluation of individual + * expressions. + */ +abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Logging { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + import scala.tools.reflect.ToolBox + + protected val toolBox = runtimeMirror(getClass.getClassLoader).mkToolBox() + + protected val rowType = typeOf[Row] + protected val mutableRowType = typeOf[MutableRow] + protected val genericRowType = typeOf[GenericRow] + protected val genericMutableRowType = typeOf[GenericMutableRow] + + protected val projectionType = typeOf[Projection] + protected val mutableProjectionType = typeOf[MutableProjection] + + private val curId = new java.util.concurrent.atomic.AtomicInteger() + private val javaSeparator = "$" + + /** + * Generates a class for a given input expression. Called when there is not cached code + * already available. + */ + protected def create(in: InType): OutType + + /** + * Canonicalizes an input expression. Used to avoid double caching expressions that differ only + * cosmetically. + */ + protected def canonicalize(in: InType): InType + + /** Binds an input expression to a given input schema */ + protected def bind(in: InType, inputSchema: Seq[Attribute]): InType + + /** + * A cache of generated classes. + * + * From the Guava Docs: A Cache is similar to ConcurrentMap, but not quite the same. The most + * fundamental difference is that a ConcurrentMap persists all elements that are added to it until + * they are explicitly removed. A Cache on the other hand is generally configured to evict entries + * automatically, in order to constrain its memory footprint + */ + protected val cache = CacheBuilder.newBuilder() + .maximumSize(1000) + .build( + new CacheLoader[InType, OutType]() { + override def load(in: InType): OutType = globalLock.synchronized { + create(in) + } + }) + + /** Generates the requested evaluator binding the given expression(s) to the inputSchema. */ + def apply(expressions: InType, inputSchema: Seq[Attribute]): OutType = + apply(bind(expressions, inputSchema)) + + /** Generates the requested evaluator given already bound expression(s). */ + def apply(expressions: InType): OutType = cache.get(canonicalize(expressions)) + + /** + * Returns a term name that is unique within this instance of a `CodeGenerator`. + * + * (Since we aren't in a macro context we do not seem to have access to the built in `freshName` + * function.) + */ + protected def freshName(prefix: String): TermName = { + newTermName(s"$prefix$javaSeparator${curId.getAndIncrement}") + } + + /** + * Scala ASTs for evaluating an [[Expression]] given a [[Row]] of input. + * + * @param code The sequence of statements required to evaluate the expression. + * @param nullTerm A term that holds a boolean value representing whether the expression evaluated + * to null. + * @param primitiveTerm A term for a possible primitive value of the result of the evaluation. Not + * valid if `nullTerm` is set to `false`. + * @param objectTerm A possibly boxed version of the result of evaluating this expression. + */ + protected case class EvaluatedExpression( + code: Seq[Tree], + nullTerm: TermName, + primitiveTerm: TermName, + objectTerm: TermName) + + /** + * Given an expression tree returns an [[EvaluatedExpression]], which contains Scala trees that + * can be used to determine the result of evaluating the expression on an input row. + */ + def expressionEvaluator(e: Expression): EvaluatedExpression = { + val primitiveTerm = freshName("primitiveTerm") + val nullTerm = freshName("nullTerm") + val objectTerm = freshName("objectTerm") + + implicit class Evaluate1(e: Expression) { + def castOrNull(f: TermName => Tree, dataType: DataType): Seq[Tree] = { + val eval = expressionEvaluator(e) + eval.code ++ + q""" + val $nullTerm = ${eval.nullTerm} + val $primitiveTerm = + if($nullTerm) + ${defaultPrimitive(dataType)} + else + ${f(eval.primitiveTerm)} + """.children + } + } + + implicit class Evaluate2(expressions: (Expression, Expression)) { + + /** + * Short hand for generating binary evaluation code, which depends on two sub-evaluations of + * the same type. If either of the sub-expressions is null, the result of this computation + * is assumed to be null. + * + * @param f a function from two primitive term names to a tree that evaluates them. + */ + def evaluate(f: (TermName, TermName) => Tree): Seq[Tree] = + evaluateAs(expressions._1.dataType)(f) + + def evaluateAs(resultType: DataType)(f: (TermName, TermName) => Tree): Seq[Tree] = { + // TODO: Right now some timestamp tests fail if we enforce this... + if (expressions._1.dataType != expressions._2.dataType) { + log.warn(s"${expressions._1.dataType} != ${expressions._2.dataType}") + } + + val eval1 = expressionEvaluator(expressions._1) + val eval2 = expressionEvaluator(expressions._2) + val resultCode = f(eval1.primitiveTerm, eval2.primitiveTerm) + + eval1.code ++ eval2.code ++ + q""" + val $nullTerm = ${eval1.nullTerm} || ${eval2.nullTerm} + val $primitiveTerm: ${termForType(resultType)} = + if($nullTerm) { + ${defaultPrimitive(resultType)} + } else { + $resultCode.asInstanceOf[${termForType(resultType)}] + } + """.children : Seq[Tree] + } + } + + val inputTuple = newTermName(s"i") + + // TODO: Skip generation of null handling code when expression are not nullable. + val primitiveEvaluation: PartialFunction[Expression, Seq[Tree]] = { + case b @ BoundReference(ordinal, dataType, nullable) => + val nullValue = q"$inputTuple.isNullAt($ordinal)" + q""" + val $nullTerm: Boolean = $nullValue + val $primitiveTerm: ${termForType(dataType)} = + if($nullTerm) + ${defaultPrimitive(dataType)} + else + ${getColumn(inputTuple, dataType, ordinal)} + """.children + + case expressions.Literal(null, dataType) => + q""" + val $nullTerm = true + val $primitiveTerm: ${termForType(dataType)} = null.asInstanceOf[${termForType(dataType)}] + """.children + + case expressions.Literal(value: Boolean, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case expressions.Literal(value: String, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case expressions.Literal(value: Int, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case expressions.Literal(value: Long, dataType) => + q""" + val $nullTerm = ${value == null} + val $primitiveTerm: ${termForType(dataType)} = $value + """.children + + case Cast(e @ BinaryType(), StringType) => + val eval = expressionEvaluator(e) + eval.code ++ + q""" + val $nullTerm = ${eval.nullTerm} + val $primitiveTerm = + if($nullTerm) + ${defaultPrimitive(StringType)} + else + new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) + """.children + + case Cast(child @ NumericType(), IntegerType) => + child.castOrNull(c => q"$c.toInt", IntegerType) + + case Cast(child @ NumericType(), LongType) => + child.castOrNull(c => q"$c.toLong", LongType) + + case Cast(child @ NumericType(), DoubleType) => + child.castOrNull(c => q"$c.toDouble", DoubleType) + + case Cast(child @ NumericType(), FloatType) => + child.castOrNull(c => q"$c.toFloat", IntegerType) + + // Special handling required for timestamps in hive test cases since the toString function + // does not match the expected output. + case Cast(e, StringType) if e.dataType != TimestampType => + val eval = expressionEvaluator(e) + eval.code ++ + q""" + val $nullTerm = ${eval.nullTerm} + val $primitiveTerm = + if($nullTerm) + ${defaultPrimitive(StringType)} + else + ${eval.primitiveTerm}.toString + """.children + + case EqualTo(e1, e2) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 == $eval2" } + + /* TODO: Fix null semantics. + case In(e1, list) if !list.exists(!_.isInstanceOf[expressions.Literal]) => + val eval = expressionEvaluator(e1) + + val checks = list.map { + case expressions.Literal(v: String, dataType) => + q"if(${eval.primitiveTerm} == $v) return true" + case expressions.Literal(v: Int, dataType) => + q"if(${eval.primitiveTerm} == $v) return true" + } + + val funcName = newTermName(s"isIn${curId.getAndIncrement()}") + + q""" + def $funcName: Boolean = { + ..${eval.code} + if(${eval.nullTerm}) return false + ..$checks + return false + } + val $nullTerm = false + val $primitiveTerm = $funcName + """.children + */ + + case GreaterThan(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 > $eval2" } + case GreaterThanOrEqual(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 >= $eval2" } + case LessThan(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 < $eval2" } + case LessThanOrEqual(e1 @ NumericType(), e2 @ NumericType()) => + (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 <= $eval2" } + + case And(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = false + + if ((!${eval1.nullTerm} && !${eval1.primitiveTerm}) || + (!${eval2.nullTerm} && !${eval2.primitiveTerm})) { + $nullTerm = false + $primitiveTerm = false + } else if (${eval1.nullTerm} || ${eval2.nullTerm} ) { + $nullTerm = true + } else { + $nullTerm = false + $primitiveTerm = true + } + """.children + + case Or(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = false + + if ((!${eval1.nullTerm} && ${eval1.primitiveTerm}) || + (!${eval2.nullTerm} && ${eval2.primitiveTerm})) { + $nullTerm = false + $primitiveTerm = true + } else if (${eval1.nullTerm} || ${eval2.nullTerm} ) { + $nullTerm = true + } else { + $nullTerm = false + $primitiveTerm = false + } + """.children + + case Not(child) => + // Uh, bad function name... + child.castOrNull(c => q"!$c", BooleanType) + + case Add(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 + $eval2" } + case Subtract(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 - $eval2" } + case Multiply(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 * $eval2" } + case Divide(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 / $eval2" } + + case IsNotNull(e) => + val eval = expressionEvaluator(e) + q""" + ..${eval.code} + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = !${eval.nullTerm} + """.children + + case IsNull(e) => + val eval = expressionEvaluator(e) + q""" + ..${eval.code} + var $nullTerm = false + var $primitiveTerm: ${termForType(BooleanType)} = ${eval.nullTerm} + """.children + + case c @ Coalesce(children) => + q""" + var $nullTerm = true + var $primitiveTerm: ${termForType(c.dataType)} = ${defaultPrimitive(c.dataType)} + """.children ++ + children.map { c => + val eval = expressionEvaluator(c) + q""" + if($nullTerm) { + ..${eval.code} + if(!${eval.nullTerm}) { + $nullTerm = false + $primitiveTerm = ${eval.primitiveTerm} + } + } + """ + } + + case i @ expressions.If(condition, trueValue, falseValue) => + val condEval = expressionEvaluator(condition) + val trueEval = expressionEvaluator(trueValue) + val falseEval = expressionEvaluator(falseValue) + + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(i.dataType)} = ${defaultPrimitive(i.dataType)} + ..${condEval.code} + if(!${condEval.nullTerm} && ${condEval.primitiveTerm}) { + ..${trueEval.code} + $nullTerm = ${trueEval.nullTerm} + $primitiveTerm = ${trueEval.primitiveTerm} + } else { + ..${falseEval.code} + $nullTerm = ${falseEval.nullTerm} + $primitiveTerm = ${falseEval.primitiveTerm} + } + """.children + } + + // If there was no match in the partial function above, we fall back on calling the interpreted + // expression evaluator. + val code: Seq[Tree] = + primitiveEvaluation.lift.apply(e).getOrElse { + log.debug(s"No rules to generate $e") + val tree = reify { e } + q""" + val $objectTerm = $tree.eval(i) + val $nullTerm = $objectTerm == null + val $primitiveTerm = $objectTerm.asInstanceOf[${termForType(e.dataType)}] + """.children + } + + EvaluatedExpression(code, nullTerm, primitiveTerm, objectTerm) + } + + protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { + dataType match { + case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)" + case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]" + } + } + + protected def setColumn( + destinationRow: TermName, + dataType: DataType, + ordinal: Int, + value: TermName) = { + dataType match { + case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" + case _ => q"$destinationRow.update($ordinal, $value)" + } + } + + protected def accessorForType(dt: DataType) = newTermName(s"get${primitiveForType(dt)}") + protected def mutatorForType(dt: DataType) = newTermName(s"set${primitiveForType(dt)}") + + protected def primitiveForType(dt: DataType) = dt match { + case IntegerType => "Int" + case LongType => "Long" + case ShortType => "Short" + case ByteType => "Byte" + case DoubleType => "Double" + case FloatType => "Float" + case BooleanType => "Boolean" + case StringType => "String" + } + + protected def defaultPrimitive(dt: DataType) = dt match { + case BooleanType => ru.Literal(Constant(false)) + case FloatType => ru.Literal(Constant(-1.0.toFloat)) + case StringType => ru.Literal(Constant("")) + case ShortType => ru.Literal(Constant(-1.toShort)) + case LongType => ru.Literal(Constant(1L)) + case ByteType => ru.Literal(Constant(-1.toByte)) + case DoubleType => ru.Literal(Constant(-1.toDouble)) + case DecimalType => ru.Literal(Constant(-1)) // Will get implicity converted as needed. + case IntegerType => ru.Literal(Constant(-1)) + case _ => ru.Literal(Constant(null)) + } + + protected def termForType(dt: DataType) = dt match { + case n: NativeType => n.tag + case _ => typeTag[Any] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala new file mode 100644 index 0000000000000..a419fd7ecb39b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.sql.catalyst.expressions._ + +/** + * Generates byte code that produces a [[MutableRow]] object that can update itself based on a new + * input [[Row]] for a fixed set of [[Expression Expressions]]. + */ +object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => MutableProjection] { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + val mutableRowName = newTermName("mutableRow") + + protected def canonicalize(in: Seq[Expression]): Seq[Expression] = + in.map(ExpressionCanonicalizer(_)) + + protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = + in.map(BindReferences.bindReference(_, inputSchema)) + + protected def create(expressions: Seq[Expression]): (() => MutableProjection) = { + val projectionCode = expressions.zipWithIndex.flatMap { case (e, i) => + val evaluationCode = expressionEvaluator(e) + + evaluationCode.code :+ + q""" + if(${evaluationCode.nullTerm}) + mutableRow.setNullAt($i) + else + ${setColumn(mutableRowName, e.dataType, i, evaluationCode.primitiveTerm)} + """ + } + + val code = + q""" + () => { new $mutableProjectionType { + + private[this] var $mutableRowName: $mutableRowType = + new $genericMutableRowType(${expressions.size}) + + def target(row: $mutableRowType): $mutableProjectionType = { + $mutableRowName = row + this + } + + /* Provide immutable access to the last projected row. */ + def currentValue: $rowType = mutableRow + + def apply(i: $rowType): $rowType = { + ..$projectionCode + mutableRow + } + } } + """ + + log.debug(s"code for ${expressions.mkString(",")}:\n$code") + toolBox.eval(code).asInstanceOf[() => MutableProjection] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala new file mode 100644 index 0000000000000..4211998f7511a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import com.typesafe.scalalogging.slf4j.Logging +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types.{StringType, NumericType} + +/** + * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of + * [[Expression Expressions]]. + */ +object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] with Logging { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] = + in.map(ExpressionCanonicalizer(_).asInstanceOf[SortOrder]) + + protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] = + in.map(BindReferences.bindReference(_, inputSchema)) + + protected def create(ordering: Seq[SortOrder]): Ordering[Row] = { + val a = newTermName("a") + val b = newTermName("b") + val comparisons = ordering.zipWithIndex.map { case (order, i) => + val evalA = expressionEvaluator(order.child) + val evalB = expressionEvaluator(order.child) + + val compare = order.child.dataType match { + case _: NumericType => + q""" + val comp = ${evalA.primitiveTerm} - ${evalB.primitiveTerm} + if(comp != 0) { + return ${if (order.direction == Ascending) q"comp.toInt" else q"-comp.toInt"} + } + """ + case StringType => + if (order.direction == Ascending) { + q"""return ${evalA.primitiveTerm}.compare(${evalB.primitiveTerm})""" + } else { + q"""return ${evalB.primitiveTerm}.compare(${evalA.primitiveTerm})""" + } + } + + q""" + i = $a + ..${evalA.code} + i = $b + ..${evalB.code} + if (${evalA.nullTerm} && ${evalB.nullTerm}) { + // Nothing + } else if (${evalA.nullTerm}) { + return ${if (order.direction == Ascending) q"-1" else q"1"} + } else if (${evalB.nullTerm}) { + return ${if (order.direction == Ascending) q"1" else q"-1"} + } else { + $compare + } + """ + } + + val q"class $orderingName extends $orderingType { ..$body }" = reify { + class SpecificOrdering extends Ordering[Row] { + val o = ordering + } + }.tree.children.head + + val code = q""" + class $orderingName extends $orderingType { + ..$body + def compare(a: $rowType, b: $rowType): Int = { + var i: $rowType = null // Holds current row being evaluated. + ..$comparisons + return 0 + } + } + new $orderingName() + """ + logger.debug(s"Generated Ordering: $code") + toolBox.eval(code).asInstanceOf[Ordering[Row]] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala new file mode 100644 index 0000000000000..2a0935c790cf3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.sql.catalyst.expressions._ + +/** + * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[Row]]. + */ +object GeneratePredicate extends CodeGenerator[Expression, (Row) => Boolean] { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer(in) + + protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = + BindReferences.bindReference(in, inputSchema) + + protected def create(predicate: Expression): ((Row) => Boolean) = { + val cEval = expressionEvaluator(predicate) + + val code = + q""" + (i: $rowType) => { + ..${cEval.code} + if (${cEval.nullTerm}) false else ${cEval.primitiveTerm} + } + """ + + log.debug(s"Generated predicate '$predicate':\n$code") + toolBox.eval(code).asInstanceOf[Row => Boolean] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala new file mode 100644 index 0000000000000..77fa02c13de30 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -0,0 +1,219 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ + + +/** + * Generates bytecode that produces a new [[Row]] object based on a fixed set of input + * [[Expression Expressions]] and a given input [[Row]]. The returned [[Row]] object is custom + * generated based on the output types of the [[Expression]] to avoid boxing of primitive values. + */ +object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { + import scala.reflect.runtime.{universe => ru} + import scala.reflect.runtime.universe._ + + protected def canonicalize(in: Seq[Expression]): Seq[Expression] = + in.map(ExpressionCanonicalizer(_)) + + protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = + in.map(BindReferences.bindReference(_, inputSchema)) + + // Make Mutablility optional... + protected def create(expressions: Seq[Expression]): Projection = { + val tupleLength = ru.Literal(Constant(expressions.length)) + val lengthDef = q"final val length = $tupleLength" + + /* TODO: Configurable... + val nullFunctions = + q""" + private final val nullSet = new org.apache.spark.util.collection.BitSet(length) + final def setNullAt(i: Int) = nullSet.set(i) + final def isNullAt(i: Int) = nullSet.get(i) + """ + */ + + val nullFunctions = + q""" + private[this] var nullBits = new Array[Boolean](${expressions.size}) + final def setNullAt(i: Int) = { nullBits(i) = true } + final def isNullAt(i: Int) = nullBits(i) + """.children + + val tupleElements = expressions.zipWithIndex.flatMap { + case (e, i) => + val elementName = newTermName(s"c$i") + val evaluatedExpression = expressionEvaluator(e) + val iLit = ru.Literal(Constant(i)) + + q""" + var ${newTermName(s"c$i")}: ${termForType(e.dataType)} = _ + { + ..${evaluatedExpression.code} + if(${evaluatedExpression.nullTerm}) + setNullAt($iLit) + else + $elementName = ${evaluatedExpression.primitiveTerm} + } + """.children : Seq[Tree] + } + + val iteratorFunction = { + val allColumns = (0 until expressions.size).map { i => + val iLit = ru.Literal(Constant(i)) + q"if(isNullAt($iLit)) { null } else { ${newTermName(s"c$i")} }" + } + q"final def iterator = Iterator[Any](..$allColumns)" + } + + val accessorFailure = q"""scala.sys.error("Invalid ordinal:" + i)""" + val applyFunction = { + val cases = (0 until expressions.size).map { i => + val ordinal = ru.Literal(Constant(i)) + val elementName = newTermName(s"c$i") + val iLit = ru.Literal(Constant(i)) + + q"if(i == $ordinal) { if(isNullAt($i)) return null else return $elementName }" + } + q"final def apply(i: Int): Any = { ..$cases; $accessorFailure }" + } + + val updateFunction = { + val cases = expressions.zipWithIndex.map {case (e, i) => + val ordinal = ru.Literal(Constant(i)) + val elementName = newTermName(s"c$i") + val iLit = ru.Literal(Constant(i)) + + q""" + if(i == $ordinal) { + if(value == null) { + setNullAt(i) + } else { + $elementName = value.asInstanceOf[${termForType(e.dataType)}] + return + } + }""" + } + q"final def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }" + } + + val specificAccessorFunctions = NativeType.all.map { dataType => + val ifStatements = expressions.zipWithIndex.flatMap { + case (e, i) if e.dataType == dataType => + val elementName = newTermName(s"c$i") + // TODO: The string of ifs gets pretty inefficient as the row grows in size. + // TODO: Optional null checks? + q"if(i == $i) return $elementName" :: Nil + case _ => Nil + } + + q""" + final def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = { + ..$ifStatements; + $accessorFailure + }""" + } + + val specificMutatorFunctions = NativeType.all.map { dataType => + val ifStatements = expressions.zipWithIndex.flatMap { + case (e, i) if e.dataType == dataType => + val elementName = newTermName(s"c$i") + // TODO: The string of ifs gets pretty inefficient as the row grows in size. + // TODO: Optional null checks? + q"if(i == $i) { $elementName = value; return }" :: Nil + case _ => Nil + } + + q""" + final def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = { + ..$ifStatements; + $accessorFailure + }""" + } + + val hashValues = expressions.zipWithIndex.map { case (e,i) => + val elementName = newTermName(s"c$i") + val nonNull = e.dataType match { + case BooleanType => q"if ($elementName) 0 else 1" + case ByteType | ShortType | IntegerType => q"$elementName.toInt" + case LongType => q"($elementName ^ ($elementName >>> 32)).toInt" + case FloatType => q"java.lang.Float.floatToIntBits($elementName)" + case DoubleType => + q"{ val b = java.lang.Double.doubleToLongBits($elementName); (b ^ (b >>>32)).toInt }" + case _ => q"$elementName.hashCode" + } + q"if (isNullAt($i)) 0 else $nonNull" + } + + val hashUpdates: Seq[Tree] = hashValues.map(v => q"""result = 37 * result + $v""": Tree) + + val hashCodeFunction = + q""" + override def hashCode(): Int = { + var result: Int = 37 + ..$hashUpdates + result + } + """ + + val columnChecks = (0 until expressions.size).map { i => + val elementName = newTermName(s"c$i") + q"if (this.$elementName != specificType.$elementName) return false" + } + + val equalsFunction = + q""" + override def equals(other: Any): Boolean = other match { + case specificType: SpecificRow => + ..$columnChecks + return true + case other => super.equals(other) + } + """ + + val copyFunction = + q""" + final def copy() = new $genericRowType(this.toArray) + """ + + val classBody = + nullFunctions ++ ( + lengthDef +: + iteratorFunction +: + applyFunction +: + updateFunction +: + equalsFunction +: + hashCodeFunction +: + copyFunction +: + (tupleElements ++ specificAccessorFunctions ++ specificMutatorFunctions)) + + val code = q""" + final class SpecificRow(i: $rowType) extends $mutableRowType { + ..$classBody + } + + new $projectionType { def apply(r: $rowType) = new SpecificRow(r) } + """ + + log.debug( + s"MutableRow, initExprs: ${expressions.mkString(",")} code:\n${toolBox.typeCheck(code)}") + toolBox.eval(code).asInstanceOf[Projection] + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala new file mode 100644 index 0000000000000..80c7dfd376c96 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/package.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.rules +import org.apache.spark.sql.catalyst.util + +/** + * A collection of generators that build custom bytecode at runtime for performing the evaluation + * of catalyst expression. + */ +package object codegen { + + /** + * A lock to protect invoking the scala compiler at runtime, since it is not thread safe in Scala + * 2.10. + */ + protected[codegen] val globalLock = org.apache.spark.sql.catalyst.ScalaReflectionLock + + /** Canonicalizes an expression so those that differ only by names can reuse the same code. */ + object ExpressionCanonicalizer extends rules.RuleExecutor[Expression] { + val batches = + Batch("CleanExpressions", FixedPoint(20), CleanExpressions) :: Nil + + object CleanExpressions extends rules.Rule[Expression] { + def apply(e: Expression): Expression = e transform { + case Alias(c, _) => c + } + } + } + + /** + * :: DeveloperApi :: + * Dumps the bytecode from a class to the screen using javap. + */ + @DeveloperApi + object DumpByteCode { + import scala.sys.process._ + val dumpDirectory = util.getTempFilePath("sparkSqlByteCode") + dumpDirectory.mkdir() + + def apply(obj: Any): Unit = { + val generatedClass = obj.getClass + val classLoader = + generatedClass + .getClassLoader + .asInstanceOf[scala.tools.nsc.interpreter.AbstractFileClassLoader] + val generatedBytes = classLoader.classBytes(generatedClass.getName) + + val packageDir = new java.io.File(dumpDirectory, generatedClass.getPackage.getName) + if (!packageDir.exists()) { packageDir.mkdir() } + + val classFile = + new java.io.File(packageDir, generatedClass.getName.split("\\.").last + ".class") + + val outfile = new java.io.FileOutputStream(classFile) + outfile.write(generatedBytes) + outfile.close() + + println( + s"javap -p -v -classpath ${dumpDirectory.getCanonicalPath} ${generatedClass.getName}".!!) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index b6f2451b52e1f..55d95991c5f11 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -47,4 +47,30 @@ package org.apache.spark.sql.catalyst * ==Evaluation== * The result of expressions can be evaluated using the `Expression.apply(Row)` method. */ -package object expressions +package object expressions { + + /** + * Converts a [[Row]] to another Row given a sequence of expression that define each column of the + * new row. If the schema of the input row is specified, then the given expression will be bound + * to that schema. + */ + abstract class Projection extends (Row => Row) + + /** + * Converts a [[Row]] to another Row given a sequence of expression that define each column of the + * new row. If the schema of the input row is specified, then the given expression will be bound + * to that schema. + * + * In contrast to a normal projection, a MutableProjection reuses the same underlying row object + * each time an input row is added. This significantly reduces the cost of calculating the + * projection, but means that it is not safe to hold on to a reference to a [[Row]] after `next()` + * has been called on the [[Iterator]] that produced it. Instead, the user must call `Row.copy()` + * and hold on to the returned [[Row]] before calling `next()`. + */ + abstract class MutableProjection extends Projection { + def currentValue: Row + + /** Uses the given row to store the output of the projection. */ + def target(row: MutableRow): MutableProjection + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 06b94a98d3cd0..5976b0ddf3e03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -23,6 +23,9 @@ import org.apache.spark.sql.catalyst.types.BooleanType object InterpretedPredicate { + def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = + apply(BindReferences.bindReference(expression, inputSchema)) + def apply(expression: Expression): (Row => Boolean) = { (r: Row) => expression.eval(r).asInstanceOf[Boolean] } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala new file mode 100644 index 0000000000000..3b3e206055cfc --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +package object catalyst { + /** + * 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 +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 026692abe067d..418f8686bfe5c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -104,6 +104,77 @@ object PhysicalOperation extends PredicateHelper { } } +/** + * Matches a logical aggregation that can be performed on distributed data in two steps. The first + * operates on the data in each partition performing partial aggregation for each group. The second + * occurs after the shuffle and completes the aggregation. + * + * This pattern will only match if all aggregate expressions can be computed partially and will + * return the rewritten aggregation expressions for both phases. + * + * The returned values for this match are as follows: + * - Grouping attributes for the final aggregation. + * - Aggregates for the final aggregation. + * - Grouping expressions for the partial aggregation. + * - Partial aggregate expressions. + * - Input to the aggregation. + */ +object PartialAggregation { + type ReturnType = + (Seq[Attribute], Seq[NamedExpression], Seq[Expression], Seq[NamedExpression], LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { + case logical.Aggregate(groupingExpressions, aggregateExpressions, child) => + // Collect all aggregate expressions. + val allAggregates = + aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a}) + // Collect all aggregate expressions that can be computed partially. + val partialAggregates = + aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p}) + + // Only do partial aggregation if supported by all aggregate expressions. + if (allAggregates.size == partialAggregates.size) { + // Create a map of expressions to their partial evaluations for all aggregate expressions. + val partialEvaluations: Map[Long, SplitEvaluation] = + partialAggregates.map(a => (a.id, a.asPartial)).toMap + + // We need to pass all grouping expressions though so the grouping can happen a second + // time. However some of them might be unnamed so we alias them allowing them to be + // referenced in the second aggregation. + val namedGroupingExpressions: Map[Expression, NamedExpression] = groupingExpressions.map { + case n: NamedExpression => (n, n) + case other => (other, Alias(other, "PartialGroup")()) + }.toMap + + // Replace aggregations with a new expression that computes the result from the already + // computed partial evaluations and grouping values. + val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp { + case e: Expression if partialEvaluations.contains(e.id) => + partialEvaluations(e.id).finalEvaluation + case e: Expression if namedGroupingExpressions.contains(e) => + namedGroupingExpressions(e).toAttribute + }).asInstanceOf[Seq[NamedExpression]] + + val partialComputation = + (namedGroupingExpressions.values ++ + partialEvaluations.values.flatMap(_.partialEvaluations)).toSeq + + val namedGroupingAttributes = namedGroupingExpressions.values.map(_.toAttribute).toSeq + + Some( + (namedGroupingAttributes, + rewrittenAggregateExpressions, + groupingExpressions, + partialComputation, + child)) + } else { + None + } + case _ => None + } +} + + /** * A pattern that finds joins with equality conditions that can be evaluated using equi-join. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ac85f95b52a2f..888cb08e95f06 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -112,7 +112,7 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { self: Product => override lazy val statistics: Statistics = - throw new UnsupportedOperationException("default leaf nodes don't have meaningful Statistics") + throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") // Leaf nodes by definition cannot reference any input attributes. override def references = Set.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index a357c6ffb8977..481a5a4f212b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -35,7 +35,7 @@ abstract class Command extends LeafNode { */ case class NativeCommand(cmd: String) extends Command { override def output = - Seq(BoundReference(0, AttributeReference("result", StringType, nullable = false)())) + Seq(AttributeReference("result", StringType, nullable = false)()) } /** @@ -43,7 +43,7 @@ case class NativeCommand(cmd: String) extends Command { */ case class SetCommand(key: Option[String], value: Option[String]) extends Command { override def output = Seq( - BoundReference(1, AttributeReference("", StringType, nullable = false)())) + AttributeReference("", StringType, nullable = false)()) } /** @@ -52,7 +52,7 @@ case class SetCommand(key: Option[String], value: Option[String]) extends Comman */ case class ExplainCommand(plan: LogicalPlan) extends Command { override def output = - Seq(BoundReference(0, AttributeReference("plan", StringType, nullable = false)())) + Seq(AttributeReference("plan", StringType, nullable = false)()) } /** @@ -71,7 +71,7 @@ case class DescribeCommand( isExtended: Boolean) extends Command { override def output = Seq( // Column names are based on Hive. - BoundReference(0, AttributeReference("col_name", StringType, nullable = false)()), - BoundReference(1, AttributeReference("data_type", StringType, nullable = false)()), - BoundReference(2, AttributeReference("comment", StringType, nullable = false)())) + AttributeReference("col_name", StringType, nullable = false)(), + AttributeReference("data_type", StringType, nullable = false)(), + AttributeReference("comment", StringType, nullable = false)()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index e32adb76fe146..e300bdbececbd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -72,7 +72,10 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { } iteration += 1 if (iteration > batch.strategy.maxIterations) { - logger.info(s"Max iterations ($iteration) reached for batch ${batch.name}") + // Only log if this is a rule that is supposed to run more than once. + if (iteration != 2) { + logger.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") + } continue = false } 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 cd4b5e9c1b529..71808f76d632b 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 @@ -23,16 +23,13 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} import scala.util.parsing.combinator.RegexParsers +import org.apache.spark.sql.catalyst.ScalaReflectionLock 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. + * Utility functions for working with DataTypes. */ -protected[catalyst] object ScalaReflectionLock - object DataType extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "StringType" ^^^ StringType | @@ -99,6 +96,13 @@ abstract class DataType { case object NullType extends DataType +object NativeType { + def all = Seq( + IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) + + def unapply(dt: DataType): Boolean = all.contains(dt) +} + trait PrimitiveType extends DataType { override def isPrimitive = true } @@ -149,6 +153,10 @@ abstract class NumericType extends NativeType with PrimitiveType { val numeric: Numeric[JvmType] } +object NumericType { + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[NumericType] +} + /** Matcher for any expressions that evaluate to [[IntegralType]]s */ object IntegralType { def unapply(a: Expression): Boolean = a match { 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 58f8c341e6676..999c9fff38d60 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 @@ -29,7 +29,11 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ class ExpressionEvaluationSuite extends FunSuite { test("literals") { - assert((Literal(1) + Literal(1)).eval(null) === 2) + checkEvaluation(Literal(1), 1) + checkEvaluation(Literal(true), true) + checkEvaluation(Literal(0L), 0L) + checkEvaluation(Literal("test"), "test") + checkEvaluation(Literal(1) + Literal(1), 2) } /** @@ -61,10 +65,8 @@ class ExpressionEvaluationSuite extends FunSuite { test("3VL Not") { notTrueTable.foreach { case (v, answer) => - val expr = ! Literal(v, BooleanType) - val result = expr.eval(null) - if (result != answer) - fail(s"$expr should not evaluate to $result, expected: $answer") } + checkEvaluation(!Literal(v, BooleanType), answer) + } } booleanLogicTest("AND", _ && _, @@ -127,6 +129,13 @@ class ExpressionEvaluationSuite extends FunSuite { } } + test("IN") { + checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) + checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) + checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal(null, StringType).like("a"), null) checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) @@ -232,21 +241,21 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(false) cast IntegerType, 0) checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1) checkEvaluation(Cast(Literal(0) cast BooleanType, IntegerType), 0) - checkEvaluation("23" cast DoubleType, 23) + checkEvaluation("23" cast DoubleType, 23d) checkEvaluation("23" cast IntegerType, 23) - checkEvaluation("23" cast FloatType, 23) - checkEvaluation("23" cast DecimalType, 23) - checkEvaluation("23" cast ByteType, 23) - checkEvaluation("23" cast ShortType, 23) + checkEvaluation("23" cast FloatType, 23f) + checkEvaluation("23" cast DecimalType, 23: BigDecimal) + checkEvaluation("23" cast ByteType, 23.toByte) + checkEvaluation("23" cast ShortType, 23.toShort) checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) - checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24) + checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24d) checkEvaluation(Literal(23) + Cast(true, IntegerType), 24) - checkEvaluation(Literal(23f) + Cast(true, FloatType), 24) - checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24) - checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24) - checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24) + checkEvaluation(Literal(23f) + Cast(true, FloatType), 24f) + checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24: BigDecimal) + checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24.toByte) + checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24.toShort) intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} @@ -391,21 +400,21 @@ class ExpressionEvaluationSuite extends FunSuite { val typeMap = MapType(StringType, StringType) val typeArray = ArrayType(StringType) - checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal("aa")), "bb", row) checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row) checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()), + checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal(null, StringType)), null, row) - checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(1)), "bb", row) checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row) checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row) - checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()), + checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(null, IntegerType)), null, row) - checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row) + checkEvaluation(GetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) checkEvaluation(GetField(Literal(null, typeS), "a"), null, row) val typeS_notNullable = StructType( @@ -413,10 +422,8 @@ class ExpressionEvaluationSuite extends FunSuite { :: StructField("b", StringType, nullable = false) :: Nil ) - assert(GetField(BoundReference(2, - AttributeReference("c", typeS)()), "a").nullable === true) - assert(GetField(BoundReference(2, - AttributeReference("c", typeS_notNullable, nullable = false)()), "a").nullable === false) + assert(GetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) + assert(GetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) assert(GetField(Literal(null, typeS), "a").nullable === true) assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala new file mode 100644 index 0000000000000..245a2e148030c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ + +/** + * Overrides our expression evaluation tests to use code generation for evaluation. + */ +class GeneratedEvaluationSuite extends ExpressionEvaluationSuite { + override def checkEvaluation( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + val plan = try { + GenerateMutableProjection(Alias(expression, s"Optimized($expression)")() :: Nil)() + } catch { + case e: Throwable => + val evaluated = GenerateProjection.expressionEvaluator(expression) + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code.mkString("\n")} + |$e + """.stripMargin) + } + + val actual = plan(inputRow).apply(0) + if(actual != expected) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + + test("multithreaded eval") { + import scala.concurrent._ + import ExecutionContext.Implicits.global + import scala.concurrent.duration._ + + val futures = (1 to 20).map { _ => + future { + GeneratePredicate(EqualTo(Literal(1), Literal(1))) + GenerateProjection(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateMutableProjection(EqualTo(Literal(1), Literal(1)) :: Nil) + GenerateOrdering(Add(Literal(1), Literal(1)).asc :: Nil) + } + } + + futures.foreach(Await.result(_, 10.seconds)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala new file mode 100644 index 0000000000000..887aabb1d5fb4 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ + +/** + * Overrides our expression evaluation tests to use generated code on mutable rows. + */ +class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { + override def checkEvaluation( + expression: Expression, + expected: Any, + inputRow: Row = EmptyRow): Unit = { + lazy val evaluated = GenerateProjection.expressionEvaluator(expression) + + val plan = try { + GenerateProjection(Alias(expression, s"Optimized($expression)")() :: Nil) + } catch { + case e: Throwable => + fail( + s""" + |Code generation of $expression failed: + |${evaluated.code.mkString("\n")} + |$e + """.stripMargin) + } + + val actual = plan(inputRow) + val expectedRow = new GenericRow(Array[Any](expected)) + if (actual.hashCode() != expectedRow.hashCode()) { + fail( + s""" + |Mismatched hashCodes for values: $actual, $expectedRow + |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()} + |${evaluated.code.mkString("\n")} + """.stripMargin) + } + if (actual != expectedRow) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index 4896f1b955f01..e2ae0d25db1a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -27,9 +27,9 @@ class CombiningLimitsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Combine Limit", FixedPoint(2), + Batch("Combine Limit", FixedPoint(10), CombineLimits) :: - Batch("Constant Folding", FixedPoint(3), + Batch("Constant Folding", FixedPoint(10), NullPropagation, ConstantFolding, BooleanSimplification) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 5d85a0fd4eebb..2d407077be303 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -24,8 +24,11 @@ import scala.collection.JavaConverters._ object SQLConf { val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed" val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold" - val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" + val AUTO_CONVERT_JOIN_SIZE = "spark.sql.auto.convert.join.size" + val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" + val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" + val CODEGEN_ENABLED = "spark.sql.codegen" object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" @@ -56,6 +59,18 @@ trait SQLConf { /** Number of partitions to use for shuffle operators. */ private[spark] def numShufflePartitions: Int = get(SHUFFLE_PARTITIONS, "200").toInt + /** + * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode + * that evaluates expressions found in queries. In general this custom code runs much faster + * than interpreted evaluation, but there are significant start-up costs due to compilation. + * As a result codegen is only benificial when queries run for a long time, or when the same + * expressions are used multiple times. + * + * Defaults to false as this feature is currently experimental. + */ + private[spark] def codegenEnabled: Boolean = + if (get(CODEGEN_ENABLED, "false") == "true") true else false + /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to * a broadcast value during the physical executions of join operations. Setting this to -1 @@ -111,5 +126,5 @@ trait SQLConf { private[spark] def clear() { settings.clear() } - } + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c2bdef732372c..e4b6810180994 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -94,7 +94,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def parquetFile(path: String): SchemaRDD = - new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration))) + new SchemaRDD(this, parquet.ParquetRelation(path, Some(sparkContext.hadoopConfiguration), this)) /** * Loads a JSON file (one object per line), returning the result as a [[SchemaRDD]]. @@ -160,7 +160,8 @@ class SQLContext(@transient val sparkContext: SparkContext) conf: Configuration = new Configuration()): SchemaRDD = { new SchemaRDD( this, - ParquetRelation.createEmpty(path, ScalaReflection.attributesFor[A], allowExisting, conf)) + ParquetRelation.createEmpty( + path, ScalaReflection.attributesFor[A], allowExisting, conf, this)) } /** @@ -228,12 +229,14 @@ class SQLContext(@transient val sparkContext: SparkContext) val sqlContext: SQLContext = self + def codegenEnabled = self.codegenEnabled + def numPartitions = self.numShufflePartitions val strategies: Seq[Strategy] = CommandStrategy(self) :: TakeOrdered :: - PartialAggregation :: + HashAggregation :: LeftSemiJoin :: HashJoin :: InMemoryScans :: @@ -291,27 +294,30 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] lazy val emptyResult = sparkContext.parallelize(Seq.empty[Row], 1) /** - * Prepares a planned SparkPlan for execution by binding references to specific ordinals, and - * inserting shuffle operations as needed. + * Prepares a planned SparkPlan for execution by inserting shuffle operations as needed. */ @transient protected[sql] val prepareForExecution = new RuleExecutor[SparkPlan] { val batches = - Batch("Add exchange", Once, AddExchange(self)) :: - Batch("Prepare Expressions", Once, new BindReferences[SparkPlan]) :: Nil + Batch("Add exchange", Once, AddExchange(self)) :: Nil } /** + * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy * access to the intermediate phases of query execution for developers. */ + @DeveloperApi protected abstract class QueryExecution { def logical: LogicalPlan lazy val analyzed = analyzer(logical) lazy val optimizedPlan = optimizer(analyzed) // TODO: Don't just pick the first one... - lazy val sparkPlan = planner(optimizedPlan).next() + lazy val sparkPlan = { + SparkPlan.currentContext.set(self) + planner(optimizedPlan).next() + } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan) @@ -331,6 +337,9 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(optimizedPlan)} |== Physical Plan == |${stringOrError(executedPlan)} + |Code Generation: ${executedPlan.codegenEnabled} + |== RDD == + |${stringOrError(toRdd.toDebugString)} """.stripMargin.trim } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 806097c917b91..85726bae54911 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -72,7 +72,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { conf: Configuration = new Configuration()): JavaSchemaRDD = { new JavaSchemaRDD( sqlContext, - ParquetRelation.createEmpty(path, getSchema(beanClass), allowExisting, conf)) + ParquetRelation.createEmpty(path, getSchema(beanClass), allowExisting, conf, sqlContext)) } /** @@ -101,7 +101,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { def parquetFile(path: String): JavaSchemaRDD = new JavaSchemaRDD( sqlContext, - ParquetRelation(path, Some(sqlContext.sparkContext.hadoopConfiguration))) + ParquetRelation(path, Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext)) /** * Loads a JSON file (one object per line), returning the result as a [[JavaSchemaRDD]]. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index c1ced8bfa404a..463a1d32d7fd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -42,8 +42,8 @@ case class Aggregate( partial: Boolean, groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], - child: SparkPlan)(@transient sqlContext: SQLContext) - extends UnaryNode with NoBind { + child: SparkPlan) + extends UnaryNode { override def requiredChildDistribution = if (partial) { @@ -56,8 +56,6 @@ case class Aggregate( } } - override def otherCopyArgs = sqlContext :: Nil - // HACK: Generators don't correctly preserve their output through serializations so we grab // out child's output attributes statically here. private[this] val childOutput = child.output @@ -138,7 +136,7 @@ case class Aggregate( i += 1 } } - val resultProjection = new Projection(resultExpressions, computedSchema) + val resultProjection = new InterpretedProjection(resultExpressions, computedSchema) val aggregateResults = new GenericMutableRow(computedAggregates.length) var i = 0 @@ -152,7 +150,7 @@ case class Aggregate( } else { child.execute().mapPartitions { iter => val hashTable = new HashMap[Row, Array[AggregateFunction]] - val groupingProjection = new MutableProjection(groupingExpressions, childOutput) + val groupingProjection = new InterpretedMutableProjection(groupingExpressions, childOutput) var currentRow: Row = null while (iter.hasNext) { @@ -175,7 +173,8 @@ case class Aggregate( private[this] val hashTableIter = hashTable.entrySet().iterator() private[this] val aggregateResults = new GenericMutableRow(computedAggregates.length) private[this] val resultProjection = - new MutableProjection(resultExpressions, computedSchema ++ namedGroups.map(_._2)) + new InterpretedMutableProjection( + resultExpressions, computedSchema ++ namedGroups.map(_._2)) private[this] val joinedRow = new JoinedRow override final def hasNext: Boolean = hashTableIter.hasNext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 00010ef6e798a..392a7f3be3904 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -22,7 +22,7 @@ import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions.{NoBind, MutableProjection, RowOrdering} +import org.apache.spark.sql.catalyst.expressions.RowOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair @@ -31,7 +31,7 @@ import org.apache.spark.util.MutablePair * :: DeveloperApi :: */ @DeveloperApi -case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode with NoBind { +case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning @@ -42,7 +42,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().mapPartitions { iter => - val hashExpressions = new MutableProjection(expressions, child.output) + @transient val hashExpressions = + newMutableProjection(expressions, child.output)() + val mutablePair = new MutablePair[Row, Row]() iter.map(r => mutablePair.update(hashExpressions(r), r)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index 47b3d00262dbb..c386fd121c5de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -47,23 +47,26 @@ case class Generate( } } - override def output = + // This must be a val since the generator output expr ids are not preserved by serialization. + override val output = if (join) child.output ++ generatorOutput else generatorOutput + val boundGenerator = BindReferences.bindReference(generator, child.output) + override def execute() = { if (join) { child.execute().mapPartitions { iter => val nullValues = Seq.fill(generator.output.size)(Literal(null)) // Used to produce rows with no matches when outer = true. val outerProjection = - new Projection(child.output ++ nullValues, child.output) + newProjection(child.output ++ nullValues, child.output) val joinProjection = - new Projection(child.output ++ generator.output, child.output ++ generator.output) + newProjection(child.output ++ generator.output, child.output ++ generator.output) val joinedRow = new JoinedRow iter.flatMap {row => - val outputRows = generator.eval(row) + val outputRows = boundGenerator.eval(row) if (outer && outputRows.isEmpty) { outerProjection(row) :: Nil } else { @@ -72,7 +75,7 @@ case class Generate( } } } else { - child.execute().mapPartitions(iter => iter.flatMap(row => generator.eval(row))) + child.execute().mapPartitions(iter => iter.flatMap(row => boundGenerator.eval(row))) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala new file mode 100644 index 0000000000000..4a26934c49c93 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -0,0 +1,200 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.types._ + +case class AggregateEvaluation( + schema: Seq[Attribute], + initialValues: Seq[Expression], + update: Seq[Expression], + result: Expression) + +/** + * :: DeveloperApi :: + * Alternate version of aggregation that leverages projection and thus code generation. + * Aggregations are converted into a set of projections from a aggregation buffer tuple back onto + * itself. Currently only used for simple aggregations like SUM, COUNT, or AVERAGE are supported. + * + * @param partial if true then aggregation is done partially on local data without shuffling to + * ensure all values where `groupingExpressions` are equal are present. + * @param groupingExpressions expressions that are evaluated to determine grouping. + * @param aggregateExpressions expressions that are computed for each group. + * @param child the input data source. + */ +@DeveloperApi +case class GeneratedAggregate( + partial: Boolean, + groupingExpressions: Seq[Expression], + aggregateExpressions: Seq[NamedExpression], + child: SparkPlan) + extends UnaryNode { + + override def requiredChildDistribution = + if (partial) { + UnspecifiedDistribution :: Nil + } else { + if (groupingExpressions == Nil) { + AllTuples :: Nil + } else { + ClusteredDistribution(groupingExpressions) :: Nil + } + } + + override def output = aggregateExpressions.map(_.toAttribute) + + override def execute() = { + val aggregatesToCompute = aggregateExpressions.flatMap { a => + a.collect { case agg: AggregateExpression => agg} + } + + val computeFunctions = aggregatesToCompute.map { + case c @ Count(expr) => + val currentCount = AttributeReference("currentCount", LongType, nullable = false)() + val initialValue = Literal(0L) + val updateFunction = If(IsNotNull(expr), Add(currentCount, Literal(1L)), currentCount) + val result = currentCount + + AggregateEvaluation(currentCount :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + + case Sum(expr) => + val currentSum = AttributeReference("currentSum", expr.dataType, nullable = false)() + val initialValue = Cast(Literal(0L), expr.dataType) + + // Coalasce avoids double calculation... + // but really, common sub expression elimination would be better.... + val updateFunction = Coalesce(Add(expr, currentSum) :: currentSum :: Nil) + val result = currentSum + + AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + + case a @ Average(expr) => + val currentCount = AttributeReference("currentCount", LongType, nullable = false)() + val currentSum = AttributeReference("currentSum", expr.dataType, nullable = false)() + val initialCount = Literal(0L) + val initialSum = Cast(Literal(0L), expr.dataType) + val updateCount = If(IsNotNull(expr), Add(currentCount, Literal(1L)), currentCount) + val updateSum = Coalesce(Add(expr, currentSum) :: currentSum :: Nil) + + val result = Divide(Cast(currentSum, DoubleType), Cast(currentCount, DoubleType)) + + AggregateEvaluation( + currentCount :: currentSum :: Nil, + initialCount :: initialSum :: Nil, + updateCount :: updateSum :: Nil, + result + ) + } + + val computationSchema = computeFunctions.flatMap(_.schema) + + val resultMap: Map[Long, Expression] = aggregatesToCompute.zip(computeFunctions).map { + case (agg, func) => agg.id -> func.result + }.toMap + + val namedGroups = groupingExpressions.zipWithIndex.map { + case (ne: NamedExpression, _) => (ne, ne) + case (e, i) => (e, Alias(e, s"GroupingExpr$i")()) + } + + val groupMap: Map[Expression, Attribute] = + namedGroups.map { case (k, v) => k -> v.toAttribute}.toMap + + // The set of expressions that produce the final output given the aggregation buffer and the + // grouping expressions. + val resultExpressions = aggregateExpressions.map(_.transform { + case e: Expression if resultMap.contains(e.id) => resultMap(e.id) + case e: Expression if groupMap.contains(e) => groupMap(e) + }) + + child.execute().mapPartitions { iter => + // Builds a new custom class for holding the results of aggregation for a group. + val initialValues = computeFunctions.flatMap(_.initialValues) + val newAggregationBuffer = newProjection(initialValues, child.output) + log.info(s"Initial values: ${initialValues.mkString(",")}") + + // A projection that computes the group given an input tuple. + val groupProjection = newProjection(groupingExpressions, child.output) + log.info(s"Grouping Projection: ${groupingExpressions.mkString(",")}") + + // A projection that is used to update the aggregate values for a group given a new tuple. + // This projection should be targeted at the current values for the group and then applied + // to a joined row of the current values with the new input row. + val updateExpressions = computeFunctions.flatMap(_.update) + val updateSchema = computeFunctions.flatMap(_.schema) ++ child.output + val updateProjection = newMutableProjection(updateExpressions, updateSchema)() + log.info(s"Update Expressions: ${updateExpressions.mkString(",")}") + + // A projection that produces the final result, given a computation. + val resultProjectionBuilder = + newMutableProjection( + resultExpressions, + (namedGroups.map(_._2.toAttribute) ++ computationSchema).toSeq) + log.info(s"Result Projection: ${resultExpressions.mkString(",")}") + + val joinedRow = new JoinedRow + + if (groupingExpressions.isEmpty) { + // TODO: Codegening anything other than the updateProjection is probably over kill. + val buffer = newAggregationBuffer(EmptyRow).asInstanceOf[MutableRow] + var currentRow: Row = null + updateProjection.target(buffer) + + while (iter.hasNext) { + currentRow = iter.next() + updateProjection(joinedRow(buffer, currentRow)) + } + + val resultProjection = resultProjectionBuilder() + Iterator(resultProjection(buffer)) + } else { + val buffers = new java.util.HashMap[Row, MutableRow]() + + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + val currentGroup = groupProjection(currentRow) + var currentBuffer = buffers.get(currentGroup) + if (currentBuffer == null) { + currentBuffer = newAggregationBuffer(EmptyRow).asInstanceOf[MutableRow] + buffers.put(currentGroup, currentBuffer) + } + // Target the projection at the current aggregation buffer and then project the updated + // values. + updateProjection.target(currentBuffer)(joinedRow(currentBuffer, currentRow)) + } + + new Iterator[Row] { + private[this] val resultIterator = buffers.entrySet.iterator() + private[this] val resultProjection = resultProjectionBuilder() + + def hasNext = resultIterator.hasNext + + def next() = { + val currentGroup = resultIterator.next() + resultProjection(joinedRow(currentGroup.getKey, currentGroup.getValue)) + } + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 77c874d0315ee..21cbbc9772a00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -18,22 +18,55 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Logging, Row, SQLContext} + + +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ + +object SparkPlan { + protected[sql] val currentContext = new ThreadLocal[SQLContext]() +} + /** * :: DeveloperApi :: */ @DeveloperApi -abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { +abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializable { self: Product => + /** + * A handle to the SQL Context that was used to create this plan. Since many operators need + * access to the sqlContext for RDD operations or configuration this field is automatically + * populated by the query planning infrastructure. + */ + @transient + protected val sqlContext = SparkPlan.currentContext.get() + + protected def sparkContext = sqlContext.sparkContext + + // sqlContext will be null when we are being deserialized on the slaves. In this instance + // the value of codegenEnabled will be set by the desserializer after the constructor has run. + val codegenEnabled: Boolean = if (sqlContext != null) { + sqlContext.codegenEnabled + } else { + false + } + + /** Overridden make copy also propogates sqlContext to copied plan. */ + override def makeCopy(newArgs: Array[AnyRef]): this.type = { + SparkPlan.currentContext.set(sqlContext) + super.makeCopy(newArgs) + } + // TODO: Move to `DistributedPlan` /** Specifies how data is partitioned across different nodes in the cluster. */ def outputPartitioning: Partitioning = UnknownPartitioning(0) // TODO: WRONG WIDTH! @@ -51,8 +84,46 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { */ def executeCollect(): Array[Row] = execute().map(_.copy()).collect() - protected def buildRow(values: Seq[Any]): Row = - new GenericRow(values.toArray) + protected def newProjection( + expressions: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { + log.debug( + s"Creating Projection: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") + if (codegenEnabled) { + GenerateProjection(expressions, inputSchema) + } else { + new InterpretedProjection(expressions, inputSchema) + } + } + + protected def newMutableProjection( + expressions: Seq[Expression], + inputSchema: Seq[Attribute]): () => MutableProjection = { + log.debug( + s"Creating MutableProj: $expressions, inputSchema: $inputSchema, codegen:$codegenEnabled") + if(codegenEnabled) { + GenerateMutableProjection(expressions, inputSchema) + } else { + () => new InterpretedMutableProjection(expressions, inputSchema) + } + } + + + protected def newPredicate( + expression: Expression, inputSchema: Seq[Attribute]): (Row) => Boolean = { + if (codegenEnabled) { + GeneratePredicate(expression, inputSchema) + } else { + InterpretedPredicate(expression, inputSchema) + } + } + + protected def newOrdering(order: Seq[SortOrder], inputSchema: Seq[Attribute]): Ordering[Row] = { + if (codegenEnabled) { + GenerateOrdering(order, inputSchema) + } else { + new RowOrdering(order, inputSchema) + } + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 404d48ae05b45..5f1fe99f75c9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution -import scala.util.Try - import org.apache.spark.sql.{SQLContext, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ @@ -41,7 +39,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // no predicate can be evaluated by matching hash keys case logical.Join(left, right, LeftSemi, condition) => execution.LeftSemiJoinBNL( - planLater(left), planLater(right), condition)(sqlContext) :: Nil + planLater(left), planLater(right), condition) :: Nil case _ => Nil } } @@ -60,6 +58,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * will instead be used to decide the build side in a [[execution.ShuffledHashJoin]]. */ object HashJoin extends Strategy with PredicateHelper { + private[this] def makeBroadcastHashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -68,24 +67,24 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { condition: Option[Expression], side: BuildSide) = { val broadcastHashJoin = execution.BroadcastHashJoin( - leftKeys, rightKeys, side, planLater(left), planLater(right))(sqlContext) + leftKeys, rightKeys, side, planLater(left), planLater(right)) condition.map(Filter(_, broadcastHashJoin)).getOrElse(broadcastHashJoin) :: Nil } def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) - if Try(sqlContext.autoBroadcastJoinThreshold > 0 && - right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold).getOrElse(false) => + if sqlContext.autoBroadcastJoinThreshold > 0 && + right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) - if Try(sqlContext.autoBroadcastJoinThreshold > 0 && - left.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold).getOrElse(false) => + if sqlContext.autoBroadcastJoinThreshold > 0 && + left.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => val buildSide = - if (Try(right.statistics.sizeInBytes <= left.statistics.sizeInBytes).getOrElse(false)) { + if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { BuildRight } else { BuildLeft @@ -99,65 +98,65 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } - object PartialAggregation extends Strategy { + object HashAggregation extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.Aggregate(groupingExpressions, aggregateExpressions, child) => - // Collect all aggregate expressions. - val allAggregates = - aggregateExpressions.flatMap(_ collect { case a: AggregateExpression => a }) - // Collect all aggregate expressions that can be computed partially. - val partialAggregates = - aggregateExpressions.flatMap(_ collect { case p: PartialAggregate => p }) - - // Only do partial aggregation if supported by all aggregate expressions. - if (allAggregates.size == partialAggregates.size) { - // Create a map of expressions to their partial evaluations for all aggregate expressions. - val partialEvaluations: Map[Long, SplitEvaluation] = - partialAggregates.map(a => (a.id, a.asPartial)).toMap - - // We need to pass all grouping expressions though so the grouping can happen a second - // time. However some of them might be unnamed so we alias them allowing them to be - // referenced in the second aggregation. - val namedGroupingExpressions: Map[Expression, NamedExpression] = groupingExpressions.map { - case n: NamedExpression => (n, n) - case other => (other, Alias(other, "PartialGroup")()) - }.toMap + // Aggregations that can be performed in two phases, before and after the shuffle. - // Replace aggregations with a new expression that computes the result from the already - // computed partial evaluations and grouping values. - val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp { - case e: Expression if partialEvaluations.contains(e.id) => - partialEvaluations(e.id).finalEvaluation - case e: Expression if namedGroupingExpressions.contains(e) => - namedGroupingExpressions(e).toAttribute - }).asInstanceOf[Seq[NamedExpression]] - - val partialComputation = - (namedGroupingExpressions.values ++ - partialEvaluations.values.flatMap(_.partialEvaluations)).toSeq - - // Construct two phased aggregation. - execution.Aggregate( + // Cases where all aggregates can be codegened. + case PartialAggregation( + namedGroupingAttributes, + rewrittenAggregateExpressions, + groupingExpressions, + partialComputation, + child) + if canBeCodeGened( + allAggregates(partialComputation) ++ + allAggregates(rewrittenAggregateExpressions)) && + codegenEnabled => + execution.GeneratedAggregate( partial = false, - namedGroupingExpressions.values.map(_.toAttribute).toSeq, + namedGroupingAttributes, rewrittenAggregateExpressions, - execution.Aggregate( + execution.GeneratedAggregate( partial = true, groupingExpressions, partialComputation, - planLater(child))(sqlContext))(sqlContext) :: Nil - } else { - Nil - } + planLater(child))) :: Nil + + // Cases where some aggregate can not be codegened + case PartialAggregation( + namedGroupingAttributes, + rewrittenAggregateExpressions, + groupingExpressions, + partialComputation, + child) => + execution.Aggregate( + partial = false, + namedGroupingAttributes, + rewrittenAggregateExpressions, + execution.Aggregate( + partial = true, + groupingExpressions, + partialComputation, + planLater(child))) :: Nil + case _ => Nil } + + def canBeCodeGened(aggs: Seq[AggregateExpression]) = !aggs.exists { + case _: Sum | _: Count => false + case _ => true + } + + def allAggregates(exprs: Seq[Expression]) = + exprs.flatMap(_.collect { case a: AggregateExpression => a }) } object BroadcastNestedLoopJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, joinType, condition) => execution.BroadcastNestedLoopJoin( - planLater(left), planLater(right), joinType, condition)(sqlContext) :: Nil + planLater(left), planLater(right), joinType, condition) :: Nil case _ => Nil } } @@ -176,16 +175,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { protected lazy val singleRowRdd = sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) - def convertToCatalyst(a: Any): Any = a match { - case s: Seq[Any] => s.map(convertToCatalyst) - case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) - case other => other - } - object TakeOrdered extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) => - execution.TakeOrdered(limit, order, planLater(child))(sqlContext) :: Nil + execution.TakeOrdered(limit, order, planLater(child)) :: Nil case _ => Nil } } @@ -195,11 +188,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // TODO: need to support writing to other types of files. Unify the below code paths. case logical.WriteToFile(path, child) => val relation = - ParquetRelation.create(path, child, sparkContext.hadoopConfiguration) + ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) // Note: overwrite=false because otherwise the metadata we just created will be deleted - InsertIntoParquetTable(relation, planLater(child), overwrite=false)(sqlContext) :: Nil + InsertIntoParquetTable(relation, planLater(child), overwrite = false) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => - InsertIntoParquetTable(table, planLater(child), overwrite)(sqlContext) :: Nil + InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = if (sparkContext.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { @@ -228,7 +221,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { projectList, filters, prunePushedDownFilters, - ParquetTableScan(_, relation, filters)(sqlContext)) :: Nil + ParquetTableScan(_, relation, filters)) :: Nil case _ => Nil } @@ -266,20 +259,19 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Filter(condition, child) => execution.Filter(condition, planLater(child)) :: Nil case logical.Aggregate(group, agg, child) => - execution.Aggregate(partial = false, group, agg, planLater(child))(sqlContext) :: Nil + execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case logical.LocalRelation(output, data) => - val dataAsRdd = - sparkContext.parallelize(data.map(r => - new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row)) - execution.ExistingRdd(output, dataAsRdd) :: Nil + ExistingRdd( + output, + ExistingRdd.productToRowRdd(sparkContext.parallelize(data, numPartitions))) :: Nil case logical.Limit(IntegerLiteral(limit), child) => - execution.Limit(limit, planLater(child))(sqlContext) :: Nil + execution.Limit(limit, planLater(child)) :: Nil case Unions(unionChildren) => - execution.Union(unionChildren.map(planLater))(sqlContext) :: Nil - case logical.Except(left,right) => - execution.Except(planLater(left),planLater(right)) :: Nil + execution.Union(unionChildren.map(planLater)) :: Nil + case logical.Except(left, right) => + execution.Except(planLater(left), planLater(right)) :: Nil case logical.Intersect(left, right) => execution.Intersect(planLater(left), planLater(right)) :: Nil case logical.Generate(generator, join, outer, _, child) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 966d8f95fc83c..174eda8f1a72c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -37,9 +37,11 @@ import org.apache.spark.util.MutablePair case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { override def output = projectList.map(_.toAttribute) - override def execute() = child.execute().mapPartitions { iter => - @transient val reusableProjection = new MutableProjection(projectList) - iter.map(reusableProjection) + @transient lazy val buildProjection = newMutableProjection(projectList, child.output) + + def execute() = child.execute().mapPartitions { iter => + val resuableProjection = buildProjection() + iter.map(resuableProjection) } } @@ -50,8 +52,10 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output - override def execute() = child.execute().mapPartitions { iter => - iter.filter(condition.eval(_).asInstanceOf[Boolean]) + @transient lazy val conditionEvaluator = newPredicate(condition, child.output) + + def execute() = child.execute().mapPartitions { iter => + iter.filter(conditionEvaluator) } } @@ -72,12 +76,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: * :: DeveloperApi :: */ @DeveloperApi -case class Union(children: Seq[SparkPlan])(@transient sqlContext: SQLContext) extends SparkPlan { +case class Union(children: Seq[SparkPlan]) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes override def output = children.head.output - override def execute() = sqlContext.sparkContext.union(children.map(_.execute())) - - override def otherCopyArgs = sqlContext :: Nil + override def execute() = sparkContext.union(children.map(_.execute())) } /** @@ -89,13 +91,11 @@ case class Union(children: Seq[SparkPlan])(@transient sqlContext: SQLContext) ex * repartition all the data to a single partition to compute the global limit. */ @DeveloperApi -case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext) +case class Limit(limit: Int, child: SparkPlan) extends UnaryNode { // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again - override def otherCopyArgs = sqlContext :: Nil - override def output = child.output /** @@ -161,20 +161,18 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sqlContext: SQLContext * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion. */ @DeveloperApi -case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) - (@transient sqlContext: SQLContext) extends UnaryNode { - override def otherCopyArgs = sqlContext :: Nil +case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) extends UnaryNode { override def output = child.output - @transient - lazy val ordering = new RowOrdering(sortOrder) + val ordering = new RowOrdering(sortOrder, child.output) + // TODO: Is this copying for no reason? override def executeCollect() = child.execute().map(_.copy()).takeOrdered(limit)(ordering) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. - override def execute() = sqlContext.sparkContext.makeRDD(executeCollect(), 1) + override def execute() = sparkContext.makeRDD(executeCollect(), 1) } /** @@ -189,15 +187,13 @@ case class Sort( override def requiredChildDistribution = if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil - @transient - lazy val ordering = new RowOrdering(sortOrder) override def execute() = attachTree(this, "sort") { - // TODO: Optimize sorting operation? child.execute() - .mapPartitions( - iterator => iterator.map(_.copy()).toArray.sorted(ordering).iterator, - preservesPartitioning = true) + .mapPartitions( { iterator => + val ordering = newOrdering(sortOrder, child.output) + iterator.map(_.copy()).toArray.sorted(ordering).iterator + }, preservesPartitioning = true) } override def output = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index c6fbd6d2f6930..5ef46c32d44bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -41,13 +41,13 @@ package object debug { */ @DeveloperApi implicit class DebugQuery(query: SchemaRDD) { - def debug(implicit sc: SparkContext): Unit = { + def debug(): Unit = { val plan = query.queryExecution.executedPlan val visited = new collection.mutable.HashSet[Long]() val debugPlan = plan transform { case s: SparkPlan if !visited.contains(s.id) => visited += s.id - DebugNode(sc, s) + DebugNode(s) } println(s"Results returned: ${debugPlan.execute().count()}") debugPlan.foreach { @@ -57,9 +57,7 @@ package object debug { } } - private[sql] case class DebugNode( - @transient sparkContext: SparkContext, - child: SparkPlan) extends UnaryNode { + private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { def references = Set.empty def output = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 7d1f11caae838..2750ddbce896f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -38,6 +38,8 @@ case object BuildLeft extends BuildSide case object BuildRight extends BuildSide trait HashJoin { + self: SparkPlan => + val leftKeys: Seq[Expression] val rightKeys: Seq[Expression] val buildSide: BuildSide @@ -56,9 +58,9 @@ trait HashJoin { def output = left.output ++ right.output - @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, buildPlan.output) + @transient lazy val buildSideKeyGenerator = newProjection(buildKeys, buildPlan.output) @transient lazy val streamSideKeyGenerator = - () => new MutableProjection(streamedKeys, streamedPlan.output) + newMutableProjection(streamedKeys, streamedPlan.output) def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = { // TODO: Use Spark's HashMap implementation. @@ -217,9 +219,8 @@ case class BroadcastHashJoin( rightKeys: Seq[Expression], buildSide: BuildSide, left: SparkPlan, - right: SparkPlan)(@transient sqlContext: SQLContext) extends BinaryNode with HashJoin { + right: SparkPlan) extends BinaryNode with HashJoin { - override def otherCopyArgs = sqlContext :: Nil override def outputPartitioning: Partitioning = left.outputPartitioning @@ -228,7 +229,7 @@ case class BroadcastHashJoin( @transient lazy val broadcastFuture = future { - sqlContext.sparkContext.broadcast(buildPlan.executeCollect()) + sparkContext.broadcast(buildPlan.executeCollect()) } def execute() = { @@ -248,14 +249,11 @@ case class BroadcastHashJoin( @DeveloperApi case class LeftSemiJoinBNL( streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) - (@transient sqlContext: SQLContext) extends BinaryNode { // TODO: Override requiredChildDistribution. override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def otherCopyArgs = sqlContext :: Nil - def output = left.output /** The Streamed Relation */ @@ -271,7 +269,7 @@ case class LeftSemiJoinBNL( def execute() = { val broadcastedRelation = - sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) streamed.execute().mapPartitions { streamedIter => val joinedRow = new JoinedRow @@ -300,8 +298,14 @@ case class LeftSemiJoinBNL( case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { def output = left.output ++ right.output - def execute() = left.execute().map(_.copy()).cartesian(right.execute().map(_.copy())).map { - case (l: Row, r: Row) => buildRow(l ++ r) + def execute() = { + val leftResults = left.execute().map(_.copy()) + val rightResults = right.execute().map(_.copy()) + + leftResults.cartesian(rightResults).mapPartitions { iter => + val joinedRow = new JoinedRow + iter.map(r => joinedRow(r._1, r._2)) + } } } @@ -311,14 +315,11 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod @DeveloperApi case class BroadcastNestedLoopJoin( streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) - (@transient sqlContext: SQLContext) extends BinaryNode { // TODO: Override requiredChildDistribution. override def outputPartitioning: Partitioning = streamed.outputPartitioning - override def otherCopyArgs = sqlContext :: Nil - override def output = { joinType match { case LeftOuter => @@ -345,13 +346,14 @@ case class BroadcastNestedLoopJoin( def execute() = { val broadcastedRelation = - sqlContext.sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter => val matchedRows = new ArrayBuffer[Row] // TODO: Use Spark's BitSet. val includedBroadcastTuples = new BitSet(broadcastedRelation.value.size) val joinedRow = new JoinedRow + val rightNulls = new GenericMutableRow(right.output.size) streamedIter.foreach { streamedRow => var i = 0 @@ -361,7 +363,7 @@ case class BroadcastNestedLoopJoin( // TODO: One bitset per partition instead of per row. val broadcastedRow = broadcastedRelation.value(i) if (boundCondition(joinedRow(streamedRow, broadcastedRow))) { - matchedRows += buildRow(streamedRow ++ broadcastedRow) + matchedRows += joinedRow(streamedRow, broadcastedRow).copy() matched = true includedBroadcastTuples += i } @@ -369,7 +371,7 @@ case class BroadcastNestedLoopJoin( } if (!matched && (joinType == LeftOuter || joinType == FullOuter)) { - matchedRows += buildRow(streamedRow ++ Array.fill(right.output.size)(null)) + matchedRows += joinedRow(streamedRow, rightNulls).copy() } } Iterator((matchedRows, includedBroadcastTuples)) @@ -383,20 +385,20 @@ case class BroadcastNestedLoopJoin( streamedPlusMatches.map(_._2).reduce(_ ++ _) } + val leftNulls = new GenericMutableRow(left.output.size) val rightOuterMatches: Seq[Row] = if (joinType == RightOuter || joinType == FullOuter) { broadcastedRelation.value.zipWithIndex.filter { case (row, i) => !allIncludedBroadcastTuples.contains(i) }.map { - // TODO: Use projection. - case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row) + case (row, _) => new JoinedRow(leftNulls, row) } } else { Vector() } // TODO: Breaks lineage. - sqlContext.sparkContext.union( - streamedPlusMatches.flatMap(_._1), sqlContext.sparkContext.makeRDD(rightOuterMatches)) + sparkContext.union( + streamedPlusMatches.flatMap(_._1), sparkContext.makeRDD(rightOuterMatches)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 8c7dbd5eb4a09..b3bae5db0edbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -46,7 +46,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} */ private[sql] case class ParquetRelation( path: String, - @transient conf: Option[Configuration] = None) + @transient conf: Option[Configuration], + @transient sqlContext: SQLContext) extends LeafNode with MultiInstanceRelation { self: Product => @@ -61,7 +62,7 @@ private[sql] case class ParquetRelation( /** Attributes */ override val output = ParquetTypesConverter.readSchemaFromFile(new Path(path), conf) - override def newInstance = ParquetRelation(path).asInstanceOf[this.type] + override def newInstance = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] // Equals must also take into account the output attributes so that we can distinguish between // different instances of the same relation, @@ -70,6 +71,9 @@ private[sql] case class ParquetRelation( p.path == path && p.output == output case _ => false } + + // TODO: Use data from the footers. + override lazy val statistics = Statistics(sizeInBytes = sqlContext.defaultSizeInBytes) } private[sql] object ParquetRelation { @@ -106,13 +110,14 @@ private[sql] object ParquetRelation { */ def create(pathString: String, child: LogicalPlan, - conf: Configuration): ParquetRelation = { + conf: Configuration, + sqlContext: SQLContext): ParquetRelation = { if (!child.resolved) { throw new UnresolvedException[LogicalPlan]( child, "Attempt to create Parquet table from unresolved child (when schema is not available)") } - createEmpty(pathString, child.output, false, conf) + createEmpty(pathString, child.output, false, conf, sqlContext) } /** @@ -127,14 +132,15 @@ private[sql] object ParquetRelation { def createEmpty(pathString: String, attributes: Seq[Attribute], allowExisting: Boolean, - conf: Configuration): ParquetRelation = { + conf: Configuration, + sqlContext: SQLContext): ParquetRelation = { val path = checkPath(pathString, allowExisting, conf) if (conf.get(ParquetOutputFormat.COMPRESSION) == null) { conf.set(ParquetOutputFormat.COMPRESSION, ParquetRelation.defaultCompression.name()) } ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) - new ParquetRelation(path.toString, Some(conf)) { + new ParquetRelation(path.toString, Some(conf), sqlContext) { override val output = attributes } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index ea74320d06c86..912a9f002b7d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -55,8 +55,7 @@ case class ParquetTableScan( // https://issues.apache.org/jira/browse/SPARK-1367 output: Seq[Attribute], relation: ParquetRelation, - columnPruningPred: Seq[Expression])( - @transient val sqlContext: SQLContext) + columnPruningPred: Seq[Expression]) extends LeafNode { override def execute(): RDD[Row] = { @@ -99,8 +98,6 @@ case class ParquetTableScan( .filter(_ != null) // Parquet's record filters may produce null values } - override def otherCopyArgs = sqlContext :: Nil - /** * Applies a (candidate) projection. * @@ -110,7 +107,7 @@ case class ParquetTableScan( def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = { val success = validateProjection(prunedAttributes) if (success) { - ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sqlContext) + ParquetTableScan(prunedAttributes, relation, columnPruningPred) } else { sys.error("Warning: Could not validate Parquet schema projection in pruneColumns") this @@ -150,8 +147,7 @@ case class ParquetTableScan( case class InsertIntoParquetTable( relation: ParquetRelation, child: SparkPlan, - overwrite: Boolean = false)( - @transient val sqlContext: SQLContext) + overwrite: Boolean = false) extends UnaryNode with SparkHadoopMapReduceUtil { /** @@ -171,7 +167,7 @@ case class InsertIntoParquetTable( val writeSupport = if (child.output.map(_.dataType).forall(_.isPrimitive)) { - logger.debug("Initializing MutableRowWriteSupport") + log.debug("Initializing MutableRowWriteSupport") classOf[org.apache.spark.sql.parquet.MutableRowWriteSupport] } else { classOf[org.apache.spark.sql.parquet.RowWriteSupport] @@ -203,8 +199,6 @@ case class InsertIntoParquetTable( override def output = child.output - override def otherCopyArgs = sqlContext :: Nil - /** * Stores the given Row RDD as a Hadoop file. * 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 d4599da711254..837ea7695dbb3 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 @@ -22,6 +22,7 @@ import java.io.File import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.mapreduce.Job +import org.apache.spark.sql.test.TestSQLContext import parquet.example.data.{GroupWriter, Group} import parquet.example.data.simple.SimpleGroup @@ -103,7 +104,7 @@ private[sql] object ParquetTestData { val testDir = Utils.createTempDir() val testFilterDir = Utils.createTempDir() - lazy val testData = new ParquetRelation(testDir.toURI.toString) + lazy val testData = new ParquetRelation(testDir.toURI.toString, None, TestSQLContext) val testNestedSchema1 = // based on blogpost example, source: @@ -202,8 +203,10 @@ private[sql] object ParquetTestData { val testNestedDir3 = Utils.createTempDir() val testNestedDir4 = Utils.createTempDir() - lazy val testNestedData1 = new ParquetRelation(testNestedDir1.toURI.toString) - lazy val testNestedData2 = new ParquetRelation(testNestedDir2.toURI.toString) + lazy val testNestedData1 = + new ParquetRelation(testNestedDir1.toURI.toString, None, TestSQLContext) + lazy val testNestedData2 = + new ParquetRelation(testNestedDir2.toURI.toString, None, TestSQLContext) def writeFile() = { testDir.delete() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 8e1e1971d968b..1fd8d27b34c59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -45,6 +45,7 @@ class QueryTest extends PlanTest { |${rdd.queryExecution} |== Exception == |$e + |${org.apache.spark.sql.catalyst.util.stackTraceToString(e)} """.stripMargin) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 215618e852eb2..76b1724471442 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -39,22 +39,22 @@ class PlannerSuite extends FunSuite { test("count is partially aggregated") { val query = testData.groupBy('value)(Count('key)).queryExecution.analyzed - val planned = PartialAggregation(query).head - val aggregations = planned.collect { case a: Aggregate => a } + val planned = HashAggregation(query).head + val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n } assert(aggregations.size === 2) } test("count distinct is not partially aggregated") { val query = testData.groupBy('value)(CountDistinct('key :: Nil)).queryExecution.analyzed - val planned = PartialAggregation(query) + val planned = HashAggregation(query) assert(planned.isEmpty) } test("mixed aggregates are not partially aggregated") { val query = testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).queryExecution.analyzed - val planned = PartialAggregation(query) + val planned = HashAggregation(query) assert(planned.isEmpty) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index e55648b8ed15a..2cab5e0c44d92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.test.TestSQLContext._ * Note: this is only a rough example of how TGFs can be expressed, the final version will likely * involve a lot more sugar for cleaner use in Scala/Java/etc. */ -case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generator { +case class ExampleTGF(input: Seq[Expression] = Seq('name, 'age)) extends Generator { def children = input protected def makeOutput() = 'nameAndAge.string :: Nil 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 3c911e9a4e7b1..561f5b4a49965 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 @@ -25,6 +25,7 @@ import parquet.schema.MessageTypeParser import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job + import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser} @@ -32,6 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, IntegerType} import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.util.Utils @@ -207,10 +209,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection of simple Parquet file") { + SparkPlan.currentContext.set(TestSQLContext) val scanner = new ParquetTableScan( ParquetTestData.testData.output, ParquetTestData.testData, - Seq())(TestSQLContext) + Seq()) val projected = scanner.pruneColumns(ParquetTypesConverter .convertToAttributes(MessageTypeParser .parseMessageType(ParquetTestData.subTestSchema))) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 84d43eaeea51d..f0a61270daf05 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -231,7 +231,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { HiveTableScans, DataSinks, Scripts, - PartialAggregation, + HashAggregation, LeftSemiJoin, HashJoin, BasicOperators, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index c2b0b00aa5852..39033bdeac4b0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -131,7 +131,7 @@ case class InsertIntoHiveTable( conf, SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) - logger.debug("Saving as hadoop file of type " + valueClass.getSimpleName) + log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) writer.preSetup() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 8258ee5fef0eb..0c8f676e9c5c8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -67,7 +67,7 @@ case class ScriptTransformation( } } readerThread.start() - val outputProjection = new Projection(input) + val outputProjection = new InterpretedProjection(input, child.output) iter .map(outputProjection) // TODO: Use SerDe diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 057eb60a02612..7582b4743d404 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -251,8 +251,10 @@ private[hive] case class HiveGenericUdtf( @transient protected lazy val function: GenericUDTF = createFunction() + @transient protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) + @transient protected lazy val outputInspectors = { val structInspector = function.initialize(inputInspectors.toArray) structInspector.getAllStructFieldRefs.map(_.getFieldObjectInspector) @@ -278,7 +280,7 @@ private[hive] case class HiveGenericUdtf( override def eval(input: Row): TraversableOnce[Row] = { outputInspectors // Make sure initialized. - val inputProjection = new Projection(children) + val inputProjection = new InterpretedProjection(children) val collector = new UDTFCollector function.setCollector(collector) @@ -332,7 +334,7 @@ private[hive] case class HiveUdafFunction( override def eval(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) @transient - val inputProjection = new Projection(exprs) + val inputProjection = new InterpretedProjection(exprs) def update(input: Row): Unit = { val inputs = inputProjection(input).asInstanceOf[Seq[AnyRef]].toArray diff --git a/sql/hive/src/test/resources/golden/case else null-0-8ef2f741400830ef889a9dd0c817fe3d b/sql/hive/src/test/resources/golden/case else null-0-8ef2f741400830ef889a9dd0c817fe3d new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/case else null-0-8ef2f741400830ef889a9dd0c817fe3d @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/double case-0-f513687d17dcb18546fefa75000a52f2 b/sql/hive/src/test/resources/golden/double case-0-f513687d17dcb18546fefa75000a52f2 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/double case-0-f513687d17dcb18546fefa75000a52f2 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/single case-0-c264e319c52f1840a32959d552b99e73 b/sql/hive/src/test/resources/golden/single case-0-c264e319c52f1840a32959d552b99e73 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/single case-0-c264e319c52f1840a32959d552b99e73 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index aadfd2e900151..89cc589fb8001 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive.execution import scala.util.Try +import org.apache.spark.sql.{SchemaRDD, Row} +import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.{Row, SchemaRDD} @@ -30,6 +32,15 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("single case", + """SELECT case when true then 1 else 2 end FROM src LIMIT 1""") + + createQueryTest("double case", + """SELECT case when 1 = 2 then 1 when 2 = 2 then 3 else 2 end FROM src LIMIT 1""") + + createQueryTest("case else null", + """SELECT case when 1 = 2 then 1 when 2 = 2 then 3 else null end FROM src LIMIT 1""") + createQueryTest("having no references", "SELECT key FROM src GROUP BY key HAVING COUNT(*) > 1") From 2e6efcacea19bddbdae1d655ef54186f2e52747f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 29 Jul 2014 22:16:20 -0700 Subject: [PATCH 0120/1492] [SPARK-2568] RangePartitioner should run only one job if data is balanced As of Spark 1.0, RangePartitioner goes through data twice: once to compute the count and once to do sampling. As a result, to do sortByKey, Spark goes through data 3 times (once to count, once to sample, and once to sort). `RangePartitioner` should go through data only once, collecting samples from input partitions as well as counting. If the data is balanced, this should give us a good sketch. If we see big partitions, we re-sample from them in order to collect enough items. The downside is that we need to collect more from each partition in the first pass. An alternative solution is caching the intermediate result and decide whether to fetch the data after. Author: Xiangrui Meng Author: Reynold Xin Closes #1562 from mengxr/range-partitioner and squashes the following commits: 6cc2551 [Xiangrui Meng] change foreach to for eb39b08 [Xiangrui Meng] Merge branch 'master' into range-partitioner eb95dd8 [Xiangrui Meng] separate sketching and determining bounds impl c436d30 [Xiangrui Meng] fix binary metrics unit tests db58a55 [Xiangrui Meng] add unit tests a6e35d6 [Xiangrui Meng] minor update 60be09e [Xiangrui Meng] remove importance sampler 9ee9992 [Xiangrui Meng] update range partitioner to run only one job on roughly balanced data cc12f47 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into range-part 06ac2ec [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into range-part 17bcbf3 [Reynold Xin] Added seed. badf20d [Reynold Xin] Renamed the method. 6940010 [Reynold Xin] Reservoir sampling implementation. --- .../scala/org/apache/spark/Partitioner.scala | 121 +++++++++++++++--- .../org/apache/spark/PartitioningSuite.scala | 64 ++++++++- .../scala/org/apache/spark/rdd/RDDSuite.scala | 5 + 3 files changed, 171 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 52c018baa5f7b..37053bb6f37ad 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -19,11 +19,15 @@ package org.apache.spark import java.io.{IOException, ObjectInputStream, ObjectOutputStream} -import scala.reflect.ClassTag +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.reflect.{ClassTag, classTag} +import scala.util.hashing.byteswap32 -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{CollectionsUtils, Utils} +import org.apache.spark.util.random.{XORShiftRandom, SamplingUtils} /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. @@ -103,26 +107,49 @@ class RangePartitioner[K : Ordering : ClassTag, V]( private var ascending: Boolean = true) extends Partitioner { + // We allow partitions = 0, which happens when sorting an empty RDD under the default settings. + require(partitions >= 0, s"Number of partitions cannot be negative but found $partitions.") + private var ordering = implicitly[Ordering[K]] // An array of upper bounds for the first (partitions - 1) partitions private var rangeBounds: Array[K] = { - if (partitions == 1) { - Array() + if (partitions <= 1) { + Array.empty } else { - val rddSize = rdd.count() - val maxSampleSize = partitions * 20.0 - val frac = math.min(maxSampleSize / math.max(rddSize, 1), 1.0) - val rddSample = rdd.sample(false, frac, 1).map(_._1).collect().sorted - if (rddSample.length == 0) { - Array() + // This is the sample size we need to have roughly balanced output partitions, capped at 1M. + val sampleSize = math.min(20.0 * partitions, 1e6) + // Assume the input partitions are roughly balanced and over-sample a little bit. + val sampleSizePerPartition = math.ceil(3.0 * sampleSize / rdd.partitions.size).toInt + val (numItems, sketched) = RangePartitioner.sketch(rdd.map(_._1), sampleSizePerPartition) + if (numItems == 0L) { + Array.empty } else { - val bounds = new Array[K](partitions - 1) - for (i <- 0 until partitions - 1) { - val index = (rddSample.length - 1) * (i + 1) / partitions - bounds(i) = rddSample(index) + // If a partition contains much more than the average number of items, we re-sample from it + // to ensure that enough items are collected from that partition. + val fraction = math.min(sampleSize / math.max(numItems, 1L), 1.0) + val candidates = ArrayBuffer.empty[(K, Float)] + val imbalancedPartitions = mutable.Set.empty[Int] + sketched.foreach { case (idx, n, sample) => + if (fraction * n > sampleSizePerPartition) { + imbalancedPartitions += idx + } else { + // The weight is 1 over the sampling probability. + val weight = (n.toDouble / sample.size).toFloat + for (key <- sample) { + candidates += ((key, weight)) + } + } + } + if (imbalancedPartitions.nonEmpty) { + // Re-sample imbalanced partitions with the desired sampling probability. + val imbalanced = new PartitionPruningRDD(rdd.map(_._1), imbalancedPartitions.contains) + val seed = byteswap32(-rdd.id - 1) + val reSampled = imbalanced.sample(withReplacement = false, fraction, seed).collect() + val weight = (1.0 / fraction).toFloat + candidates ++= reSampled.map(x => (x, weight)) } - bounds + RangePartitioner.determineBounds(candidates, partitions) } } } @@ -212,3 +239,67 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } } } + +private[spark] object RangePartitioner { + + /** + * Sketches the input RDD via reservoir sampling on each partition. + * + * @param rdd the input RDD to sketch + * @param sampleSizePerPartition max sample size per partition + * @return (total number of items, an array of (partitionId, number of items, sample)) + */ + def sketch[K:ClassTag]( + rdd: RDD[K], + sampleSizePerPartition: Int): (Long, Array[(Int, Int, Array[K])]) = { + val shift = rdd.id + // val classTagK = classTag[K] // to avoid serializing the entire partitioner object + val sketched = rdd.mapPartitionsWithIndex { (idx, iter) => + val seed = byteswap32(idx ^ (shift << 16)) + val (sample, n) = SamplingUtils.reservoirSampleAndCount( + iter, sampleSizePerPartition, seed) + Iterator((idx, n, sample)) + }.collect() + val numItems = sketched.map(_._2.toLong).sum + (numItems, sketched) + } + + /** + * Determines the bounds for range partitioning from candidates with weights indicating how many + * items each represents. Usually this is 1 over the probability used to sample this candidate. + * + * @param candidates unordered candidates with weights + * @param partitions number of partitions + * @return selected bounds + */ + def determineBounds[K:Ordering:ClassTag]( + candidates: ArrayBuffer[(K, Float)], + partitions: Int): Array[K] = { + val ordering = implicitly[Ordering[K]] + val ordered = candidates.sortBy(_._1) + val numCandidates = ordered.size + val sumWeights = ordered.map(_._2.toDouble).sum + val step = sumWeights / partitions + var cumWeight = 0.0 + var target = step + val bounds = ArrayBuffer.empty[K] + var i = 0 + var j = 0 + var previousBound = Option.empty[K] + while ((i < numCandidates) && (j < partitions - 1)) { + val (key, weight) = ordered(i) + cumWeight += weight + if (cumWeight > target) { + // Skip duplicate values. + if (previousBound.isEmpty || ordering.gt(key, previousBound.get)) { + bounds += key + target += step + j += 1 + previousBound = Some(key) + } + } + i += 1 + } + bounds.toArray + } +} diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 4658a08064280..fc0cee3e8749d 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark +import scala.collection.mutable.ArrayBuffer import scala.math.abs import org.scalatest.{FunSuite, PrivateMethodTester} @@ -52,14 +53,12 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(p2 === p2) assert(p4 === p4) - assert(p2 != p4) - assert(p4 != p2) + assert(p2 === p4) assert(p4 === anotherP4) assert(anotherP4 === p4) assert(descendingP2 === descendingP2) assert(descendingP4 === descendingP4) - assert(descendingP2 != descendingP4) - assert(descendingP4 != descendingP2) + assert(descendingP2 === descendingP4) assert(p2 != descendingP2) assert(p4 != descendingP4) assert(descendingP2 != p2) @@ -102,6 +101,63 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet partitioner.getPartition(Row(100)) } + test("RangPartitioner.sketch") { + val rdd = sc.makeRDD(0 until 20, 20).flatMap { i => + val random = new java.util.Random(i) + Iterator.fill(i)(random.nextDouble()) + }.cache() + val sampleSizePerPartition = 10 + val (count, sketched) = RangePartitioner.sketch(rdd, sampleSizePerPartition) + assert(count === rdd.count()) + sketched.foreach { case (idx, n, sample) => + assert(n === idx) + assert(sample.size === math.min(n, sampleSizePerPartition)) + } + } + + test("RangePartitioner.determineBounds") { + assert(RangePartitioner.determineBounds(ArrayBuffer.empty[(Int, Float)], 10).isEmpty, + "Bounds on an empty candidates set should be empty.") + val candidates = ArrayBuffer( + (0.7, 2.0f), (0.1, 1.0f), (0.4, 1.0f), (0.3, 1.0f), (0.2, 1.0f), (0.5, 1.0f), (1.0, 3.0f)) + assert(RangePartitioner.determineBounds(candidates, 3) === Array(0.4, 0.7)) + } + + test("RangePartitioner should run only one job if data is roughly balanced") { + val rdd = sc.makeRDD(0 until 20, 20).flatMap { i => + val random = new java.util.Random(i) + Iterator.fill(5000 * i)((random.nextDouble() + i, i)) + }.cache() + for (numPartitions <- Seq(10, 20, 40)) { + val partitioner = new RangePartitioner(numPartitions, rdd) + assert(partitioner.numPartitions === numPartitions) + val counts = rdd.keys.map(key => partitioner.getPartition(key)).countByValue().values + assert(counts.max < 3.0 * counts.min) + } + } + + test("RangePartitioner should work well on unbalanced data") { + val rdd = sc.makeRDD(0 until 20, 20).flatMap { i => + val random = new java.util.Random(i) + Iterator.fill(20 * i * i * i)((random.nextDouble() + i, i)) + }.cache() + for (numPartitions <- Seq(2, 4, 8)) { + val partitioner = new RangePartitioner(numPartitions, rdd) + assert(partitioner.numPartitions === numPartitions) + val counts = rdd.keys.map(key => partitioner.getPartition(key)).countByValue().values + assert(counts.max < 3.0 * counts.min) + } + } + + test("RangePartitioner should return a single partition for empty RDDs") { + val empty1 = sc.emptyRDD[(Int, Double)] + val partitioner1 = new RangePartitioner(0, empty1) + assert(partitioner1.numPartitions === 1) + val empty2 = sc.makeRDD(0 until 2, 2).flatMap(i => Seq.empty[(Int, Double)]) + val partitioner2 = new RangePartitioner(2, empty2) + assert(partitioner2.numPartitions === 1) + } + test("HashPartitioner not equal to RangePartitioner") { val rdd = sc.parallelize(1 to 10).map(x => (x, x)) val rangeP2 = new RangePartitioner(2, rdd) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 6654ec2d7c656..fdc83bc0a5f8e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -613,6 +613,11 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + test("sort an empty RDD") { + val data = sc.emptyRDD[Int] + assert(data.sortBy(x => x).collect() === Array.empty) + } + test("sortByKey") { val data = sc.parallelize(Seq("5|50|A","4|60|C", "6|40|B")) From 077f633b4720422c5efbf0382e869ead3dc49612 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 29 Jul 2014 22:42:54 -0700 Subject: [PATCH 0121/1492] [SQL] Handle null values in debug() Author: Michael Armbrust Closes #1646 from marmbrus/nullDebug and squashes the following commits: 49050a8 [Michael Armbrust] Handle null values in debug() --- .../scala/org/apache/spark/sql/execution/debug/package.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 5ef46c32d44bc..f31df051824d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -105,7 +105,9 @@ package object debug { var i = 0 while (i < numColumns) { val value = currentRow(i) - columnStats(i).elementTypes += HashSet(value.getClass.getName) + if (value != null) { + columnStats(i).elementTypes += HashSet(value.getClass.getName) + } i += 1 } currentRow From 4ce92ccaf761e48a10fc4fe4927dbfca858ca22b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 29 Jul 2014 23:52:09 -0700 Subject: [PATCH 0122/1492] [SPARK-2260] Fix standalone-cluster mode, which was broken The main thing was that spark configs were not propagated to the driver, and so applications that do not specify `master` or `appName` automatically failed. This PR fixes that and a couple of miscellaneous things that are related. One thing that may or may not be an issue is that the jars must be available on the driver node. In `standalone-cluster` mode, this effectively means these jars must be available on all the worker machines, since the driver is launched on one of them. The semantics here are not the same as `yarn-cluster` mode, where all the relevant jars are uploaded to a distributed cache automatically and shipped to the containers. This is probably not a concern, but still worth a mention. Author: Andrew Or Closes #1538 from andrewor14/standalone-cluster and squashes the following commits: 8c11a0d [Andrew Or] Clean up imports / comments (minor) 2678d13 [Andrew Or] Handle extraJavaOpts properly 7660547 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-cluster 6f64a9b [Andrew Or] Revert changes in YARN 2f2908b [Andrew Or] Fix tests ed01491 [Andrew Or] Don't go overboard with escaping 8e105e1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into standalone-cluster b890949 [Andrew Or] Abstract usages of converting spark opts to java opts 79f63a3 [Andrew Or] Move sparkProps into javaOpts 78752f8 [Andrew Or] Fix tests 5a9c6c7 [Andrew Or] Fix line too long c141a00 [Andrew Or] Don't display "unknown app" on driver log pages d7e2728 [Andrew Or] Avoid deprecation warning in standalone Client 6ceb14f [Andrew Or] Allow relevant configs to propagate to standalone Driver 7f854bc [Andrew Or] Fix test 855256e [Andrew Or] Fix standalone-cluster mode fd9da51 [Andrew Or] Formatting changes (minor) --- .../scala/org/apache/spark/SparkConf.scala | 22 ++++++++++++++++++- .../org/apache/spark/deploy/Client.scala | 21 +++++++++--------- .../org/apache/spark/deploy/Command.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 12 +++++----- .../spark/deploy/client/TestClient.scala | 6 ++--- .../spark/deploy/worker/CommandUtils.scala | 7 +++--- .../spark/deploy/worker/DriverRunner.scala | 3 ++- .../spark/deploy/worker/ExecutorRunner.scala | 14 +++++++----- .../spark/deploy/worker/ui/LogPage.scala | 11 +++++----- .../CoarseGrainedExecutorBackend.scala | 9 ++++++-- .../cluster/SparkDeploySchedulerBackend.scala | 11 ++++++---- .../scala/org/apache/spark/util/Utils.scala | 9 ++++++++ .../spark/deploy/JsonProtocolSuite.scala | 6 ++--- .../spark/deploy/SparkSubmitSuite.scala | 7 ++++-- .../deploy/worker/DriverRunnerTest.scala | 2 +- .../deploy/worker/ExecutorRunnerTest.scala | 2 +- 16 files changed, 93 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 8ce4b91cae8ae..38700847c80f4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -40,6 +40,8 @@ import scala.collection.mutable.HashMap */ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { + import SparkConf._ + /** Create a SparkConf that loads defaults from system properties and the classpath */ def this() = this(true) @@ -198,7 +200,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { * * E.g. spark.akka.option.x.y.x = "value" */ - getAll.filter {case (k, v) => k.startsWith("akka.")} + getAll.filter { case (k, _) => isAkkaConf(k) } /** Does the configuration contain a given parameter? */ def contains(key: String): Boolean = settings.contains(key) @@ -292,3 +294,21 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { settings.toArray.sorted.map{case (k, v) => k + "=" + v}.mkString("\n") } } + +private[spark] object SparkConf { + /** + * Return whether the given config is an akka config (e.g. akka.actor.provider). + * Note that this does not include spark-specific akka configs (e.g. spark.akka.timeout). + */ + def isAkkaConf(name: String): Boolean = name.startsWith("akka.") + + /** + * Return whether the given config should be passed to an executor on start-up. + * + * Certain akka and authentication configs are required of the executor when it connects to + * the scheduler, while the rest of the spark configs can be inherited from the driver later. + */ + def isExecutorStartupConf(name: String): Boolean = { + isAkkaConf(name) || name.startsWith("spark.akka") || name.startsWith("spark.auth") + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index c371dc3a51c73..17c507af2652d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import scala.collection.JavaConversions._ -import scala.collection.mutable.Map import scala.concurrent._ import akka.actor._ @@ -50,9 +48,6 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends // TODO: We could add an env variable here and intercept it in `sc.addJar` that would // truncate filesystem paths similar to what YARN does. For now, we just require // people call `addJar` assuming the jar is in the same directory. - val env = Map[String, String]() - System.getenv().foreach{case (k, v) => env(k) = v} - val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" val classPathConf = "spark.driver.extraClassPath" @@ -65,10 +60,13 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends cp.split(java.io.File.pathSeparator) } - val javaOptionsConf = "spark.driver.extraJavaOptions" - val javaOpts = sys.props.get(javaOptionsConf) + val extraJavaOptsConf = "spark.driver.extraJavaOptions" + val extraJavaOpts = sys.props.get(extraJavaOptsConf) + .map(Utils.splitCommandString).getOrElse(Seq.empty) + val sparkJavaOpts = Utils.sparkJavaOpts(conf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) + driverArgs.driverOptions, sys.env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, @@ -109,6 +107,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends // Exception, if present statusResponse.exception.map { e => println(s"Exception from cluster was: $e") + e.printStackTrace() System.exit(-1) } System.exit(0) @@ -141,8 +140,10 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends */ object Client { def main(args: Array[String]) { - println("WARNING: This client is deprecated and will be removed in a future version of Spark.") - println("Use ./bin/spark-submit with \"--master spark://host:port\"") + if (!sys.props.contains("SPARK_SUBMIT")) { + println("WARNING: This client is deprecated and will be removed in a future version of Spark") + println("Use ./bin/spark-submit with \"--master spark://host:port\"") + } val conf = new SparkConf() val driverArgs = new ClientArguments(args) diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index 32f3ba385084f..a2b263544c6a2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -25,5 +25,5 @@ private[spark] case class Command( environment: Map[String, String], classPathEntries: Seq[String], libraryPathEntries: Seq[String], - extraJavaOptions: Option[String] = None) { + javaOpts: Seq[String]) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index c9cec33ebaa66..3df811c4ac5df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -136,8 +136,6 @@ object SparkSubmit { (clusterManager, deployMode) match { case (MESOS, CLUSTER) => printErrorAndExit("Cluster deploy mode is currently not supported for Mesos clusters.") - case (STANDALONE, CLUSTER) => - printErrorAndExit("Cluster deploy mode is currently not supported for Standalone clusters.") case (_, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python applications.") case (_, CLUSTER) if isShell(args.primaryResource) => @@ -170,9 +168,9 @@ object SparkSubmit { val options = List[OptionAssigner]( // All cluster managers - OptionAssigner(args.master, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.master"), - OptionAssigner(args.name, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.app.name"), - OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"), + OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"), + OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), // Standalone cluster only OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"), @@ -203,9 +201,9 @@ object SparkSubmit { sysProp = "spark.driver.extraJavaOptions"), OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, CLUSTER, sysProp = "spark.driver.extraLibraryPath"), - OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, CLIENT, + OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), - OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, CLIENT, + OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.files") diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index e15a87bd38fda..b8ffa9afb69cb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -46,11 +46,11 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val conf = new SparkConf - val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), - Seq()), Some("dummy-spark-home"), "ignored") + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), + Seq(), Seq(), Seq()), Some("dummy-spark-home"), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 4af5bc3afad6c..687e492a0d6fc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -47,7 +47,6 @@ object CommandUtils extends Logging { */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") - val extraOpts = command.extraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq()) // Exists for backwards compatibility with older Spark versions val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")).map(Utils.splitCommandString) @@ -62,7 +61,7 @@ object CommandUtils extends Logging { val joined = command.libraryPathEntries.mkString(File.pathSeparator) Seq(s"-Djava.library.path=$joined") } else { - Seq() + Seq() } val permGenOpt = Seq("-XX:MaxPermSize=128m") @@ -71,11 +70,11 @@ object CommandUtils extends Logging { val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), - extraEnvironment=command.environment) + extraEnvironment = command.environment) val userClassPath = command.classPathEntries ++ Seq(classPath) Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - permGenOpt ++ libraryOpts ++ extraOpts ++ workerLocalOpts ++ memoryOpts + permGenOpt ++ libraryOpts ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 662d37871e7a6..5caaf6bea3575 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -36,6 +36,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState /** * Manages the execution of one driver, including automatically restarting the driver on failure. + * This is currently only used in standalone cluster deploy mode. */ private[spark] class DriverRunner( val driverId: String, @@ -81,7 +82,7 @@ private[spark] class DriverRunner( driverDesc.command.environment, classPath, driverDesc.command.libraryPathEntries, - driverDesc.command.extraJavaOptions) + driverDesc.command.javaOpts) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 467317dd9b44c..7be89f9aff0f3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -30,6 +30,7 @@ import org.apache.spark.util.logging.FileAppender /** * Manages the execution of one executor process. + * This is currently only used in standalone mode. */ private[spark] class ExecutorRunner( val appId: String, @@ -72,7 +73,7 @@ private[spark] class ExecutorRunner( } /** - * kill executor process, wait for exit and notify worker to update resource status + * Kill executor process, wait for exit and notify worker to update resource status. * * @param message the exception message which caused the executor's death */ @@ -114,10 +115,13 @@ private[spark] class ExecutorRunner( } def getCommandSeq = { - val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, - appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, - appDesc.command.extraJavaOptions) + val command = Command( + appDesc.command.mainClass, + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), + appDesc.command.environment, + appDesc.command.classPathEntries, + appDesc.command.libraryPathEntries, + appDesc.command.javaOpts) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index b389cb546de6c..ecb358c399819 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.worker.ui -import java.io.File import javax.servlet.http.HttpServletRequest import scala.xml.Node @@ -25,7 +24,7 @@ import scala.xml.Node import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils import org.apache.spark.Logging -import org.apache.spark.util.logging.{FileAppender, RollingFileAppender} +import org.apache.spark.util.logging.RollingFileAppender private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with Logging { private val worker = parent.worker @@ -64,11 +63,11 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") w val offset = Option(request.getParameter("offset")).map(_.toLong) val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - val (logDir, params) = (appId, executorId, driverId) match { + val (logDir, params, pageName) = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => - (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e") + (s"${workDir.getPath}/$a/$e/", s"appId=$a&executorId=$e", s"$a/$e") case (None, None, Some(d)) => - (s"${workDir.getPath}/$d/", s"driverId=$d") + (s"${workDir.getPath}/$d/", s"driverId=$d", d) case _ => throw new Exception("Request must specify either application or driver identifiers") } @@ -120,7 +119,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") w - UIUtils.basicSparkPage(content, logType + " log page for " + appId.getOrElse("unknown app")) + UIUtils.basicSparkPage(content, logType + " log page for " + pageName) } /** Get the part of the log files given the offset and desired length of bytes */ diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index b455c9fcf4bd6..860b47e056451 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -98,8 +98,13 @@ private[spark] class CoarseGrainedExecutorBackend( } private[spark] object CoarseGrainedExecutorBackend extends Logging { - def run(driverUrl: String, executorId: String, hostname: String, cores: Int, - workerUrl: Option[String]) { + + private def run( + driverUrl: String, + executorId: String, + hostname: String, + cores: Int, + workerUrl: Option[String]) { SignalLogger.register(log) 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 bf2dc88e29048..48aaaa54bdb35 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} @@ -46,6 +46,7 @@ private[spark] class SparkDeploySchedulerBackend( CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + .map(Utils.splitCommandString).getOrElse(Seq.empty) val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } @@ -54,9 +55,11 @@ private[spark] class SparkDeploySchedulerBackend( cp.split(java.io.File.pathSeparator) } - val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, - classPathEntries, libraryPathEntries, extraJavaOpts) + // Start executors with a few necessary configs for registering with the scheduler + val sparkJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) + val javaOpts = sparkJavaOpts ++ extraJavaOpts + val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", + args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) 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 8cbb9050f393b..69f65b4bdccb1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1313,4 +1313,13 @@ private[spark] object Utils extends Logging { s"$className: $desc\n$st" } + /** + * Convert all spark properties set in the given SparkConf to a sequence of java options. + */ + def sparkJavaOpts(conf: SparkConf, filterKey: (String => Boolean) = _ => true): Seq[String] = { + conf.getAll + .filter { case (k, _) => filterKey(k) } + .map { case (k, v) => s"-D$k=$v" } + } + } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 01ab2d549325c..093394ad6d142 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -88,7 +88,7 @@ class JsonProtocolSuite extends FunSuite { } def createAppDesc(): ApplicationDescription = { - val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq()) + val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq(), Seq()) new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") } @@ -101,7 +101,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Some("-Dfoo") + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, @@ -170,7 +170,7 @@ object JsonConstants { """ |{"name":"name","cores":4,"memoryperslave":1234, |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),None)"} + |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),List())"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index f497a5e0a14f0..a301cbd48a0c3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -200,9 +200,12 @@ class SparkSubmitSuite extends FunSuite with Matchers { childArgsStr should include regex ("launch spark://h:p .*thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have size (0) - sysProps should have size (3) - sysProps.keys should contain ("spark.jars") + sysProps should have size (5) sysProps.keys should contain ("SPARK_SUBMIT") + sysProps.keys should contain ("spark.master") + sysProps.keys should contain ("spark.app.name") + sysProps.keys should contain ("spark.jars") + sysProps.keys should contain ("spark.shuffle.spill") sysProps("spark.shuffle.spill") should be ("false") } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index 4633bc3f7f25e..c930839b47f11 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.{Command, DriverDescription} class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { - val command = new Command("mainClass", Seq(), Map(), Seq(), Seq()) + val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, null, "akka://1.2.3.4/worker/") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index e5f748d55500d..ca4d987619c91 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -29,7 +29,7 @@ class ExecutorRunnerTest extends FunSuite { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) val appDesc = new ApplicationDescription("app name", Some(8), 500, - Command("foo", Seq(), Map(), Seq(), Seq()), + Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), From 7003c163dbb46bb7313aab130a33486a356435a8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 30 Jul 2014 00:15:31 -0700 Subject: [PATCH 0123/1492] [SPARK-2179][SQL] Public API for DataTypes and Schema The current PR contains the following changes: * Expose `DataType`s in the sql package (internal details are private to sql). * Users can create Rows. * Introduce `applySchema` to create a `SchemaRDD` by applying a `schema: StructType` to an `RDD[Row]`. * Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`. * `ScalaReflection.typeOfObject` provides a way to infer the Catalyst data type based on an object. Also, we can compose `typeOfObject` with some custom logics to form a new function to infer the data type (for different use cases). * `JsonRDD` has been refactored to use changes introduced by this PR. * Add a field `containsNull` to `ArrayType`. So, we can explicitly mark if an `ArrayType` can contain null values. The default value of `containsNull` is `false`. New APIs are introduced in the sql package object and SQLContext. You can find the scaladoc at [sql package object](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.package) and [SQLContext](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.SQLContext). An example of using `applySchema` is shown below. ```scala import org.apache.spark.sql._ val sqlContext = new org.apache.spark.sql.SQLContext(sc) val schema = StructType( StructField("name", StringType, false) :: StructField("age", IntegerType, true) :: Nil) val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Row(p(0), p(1).trim.toInt)) val peopleSchemaRDD = sqlContext. applySchema(people, schema) peopleSchemaRDD.printSchema // root // |-- name: string (nullable = false) // |-- age: integer (nullable = true) peopleSchemaRDD.registerAsTable("people") sqlContext.sql("select name from people").collect.foreach(println) ``` I will add new contents to the SQL programming guide later. JIRA: https://issues.apache.org/jira/browse/SPARK-2179 Author: Yin Huai Closes #1346 from yhuai/dataTypeAndSchema and squashes the following commits: 1d45977 [Yin Huai] Clean up. a6e08b4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema c712fbf [Yin Huai] Converts types of values based on defined schema. 4ceeb66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema e5f8df5 [Yin Huai] Scaladoc. 122d1e7 [Yin Huai] Address comments. 03bfd95 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 2476ed0 [Yin Huai] Minor updates. ab71f21 [Yin Huai] Format. fc2bed1 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema bd40a33 [Yin Huai] Address comments. 991f860 [Yin Huai] Move "asJavaDataType" and "asScalaDataType" to DataTypeConversions.scala. 1cb35fe [Yin Huai] Add "valueContainsNull" to MapType. 3edb3ae [Yin Huai] Python doc. 692c0b9 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 1d93395 [Yin Huai] Python APIs. 246da96 [Yin Huai] Add java data type APIs to javadoc index. 1db9531 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema d48fc7b [Yin Huai] Minor updates. 33c4fec [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema b9f3071 [Yin Huai] Java API for applySchema. 1c9f33c [Yin Huai] Java APIs for DataTypes and Row. 624765c [Yin Huai] Tests for applySchema. aa92e84 [Yin Huai] Update data type tests. 8da1a17 [Yin Huai] Add Row.fromSeq. 9c99bc0 [Yin Huai] Several minor updates. 1d9c13a [Yin Huai] Update applySchema API. 85e9b51 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema e495e4e [Yin Huai] More comments. 42d47a3 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema c3f4a02 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 2e58dbd [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema b8b7db4 [Yin Huai] 1. Move sql package object and package-info to sql-core. 2. Minor updates on APIs. 3. Update scala doc. 68525a2 [Yin Huai] Update JSON unit test. 3209108 [Yin Huai] Add unit tests. dcaf22f [Yin Huai] Add a field containsNull to ArrayType to indicate if an array can contain null values or not. If an ArrayType is constructed by "ArrayType(elementType)" (the existing constructor), the value of containsNull is false. 9168b83 [Yin Huai] Update comments. fc649d7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema eca7d04 [Yin Huai] Add two apply methods which will be used to extract StructField(s) from a StructType. 949d6bb [Yin Huai] When creating a SchemaRDD for a JSON dataset, users can apply an existing schema. 7a6a7e5 [Yin Huai] Fix bug introduced by the change made on SQLContext.inferSchema. 43a45e1 [Yin Huai] Remove sql.util.package introduced in a previous commit. 0266761 [Yin Huai] Format 03eec4c [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 90460ac [Yin Huai] Infer the Catalyst data type from an object and cast a data value to the expected type. 3fa0df5 [Yin Huai] Provide easier ways to construct a StructType. 16be3e5 [Yin Huai] This commit contains three changes: * Expose `DataType`s in the sql package (internal details are private to sql). * Introduce `createSchemaRDD` to create a `SchemaRDD` from an `RDD` with a provided schema (represented by a `StructType`) and a provided function to construct `Row`, * Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`. --- .../apache/spark/api/python/PythonRDD.scala | 3 +- project/SparkBuild.scala | 2 +- python/pyspark/sql.py | 567 +++++++++++++++++- .../spark/sql/catalyst/ScalaReflection.scala | 20 + .../catalyst/expressions/BoundAttribute.scala | 5 +- .../spark/sql/catalyst/expressions/Row.scala | 10 + .../catalyst/expressions/WrapDynamic.scala | 15 +- .../catalyst/expressions/complexTypes.scala | 4 +- .../sql/catalyst/expressions/generators.scala | 8 +- .../apache/spark/sql/catalyst/package.scala | 2 + .../sql/catalyst/planning/QueryPlanner.scala | 2 +- .../sql/catalyst/planning/patterns.scala | 3 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 45 +- .../plans/logical/basicOperators.scala | 2 +- .../spark/sql/catalyst/rules/Rule.scala | 2 +- .../sql/catalyst/rules/RuleExecutor.scala | 5 +- .../spark/sql/catalyst/trees/package.scala | 5 +- .../spark/sql/catalyst/types/dataTypes.scala | 268 +++++++-- .../sql/catalyst/ScalaReflectionSuite.scala | 66 +- .../spark/sql/api/java/types/ArrayType.java | 68 +++ .../spark/sql/api/java/types/BinaryType.java} | 19 +- .../spark/sql/api/java/types/BooleanType.java | 27 + .../spark/sql/api/java/types/ByteType.java | 27 + .../spark/sql/api/java/types/DataType.java | 190 ++++++ .../spark/sql/api/java/types/DecimalType.java | 27 + .../spark/sql/api/java/types/DoubleType.java | 27 + .../spark/sql/api/java/types/FloatType.java | 27 + .../spark/sql/api/java/types/IntegerType.java | 27 + .../spark/sql/api/java/types/LongType.java | 27 + .../spark/sql/api/java/types/MapType.java | 78 +++ .../spark/sql/api/java/types/ShortType.java | 27 + .../spark/sql/api/java/types/StringType.java | 27 + .../spark/sql/api/java/types/StructField.java | 76 +++ .../spark/sql/api/java/types/StructType.java | 59 ++ .../sql/api/java/types/TimestampType.java | 27 + .../sql/api/java/types/package-info.java | 22 + .../org/apache/spark/sql/SQLContext.scala | 230 +++++-- .../org/apache/spark/sql/SchemaRDD.scala | 10 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 12 +- .../spark/sql/api/java/JavaSQLContext.scala | 65 +- .../spark/sql/api/java/JavaSchemaRDD.scala | 7 + .../org/apache/spark/sql/api/java/Row.scala | 59 +- .../org/apache/spark/sql/json/JsonRDD.scala | 118 ++-- .../org/apache/spark/sql/package-info.java | 0 .../scala/org/apache/spark/sql/package.scala | 409 +++++++++++++ .../spark/sql/parquet/ParquetConverter.scala | 8 +- .../sql/parquet/ParquetTableSupport.scala | 4 +- .../spark/sql/parquet/ParquetTypes.scala | 18 +- .../sql/types/util/DataTypeConversions.scala | 110 ++++ .../sql/api/java/JavaApplySchemaSuite.java | 166 +++++ .../spark/sql/api/java/JavaRowSuite.java | 170 ++++++ .../java/JavaSideDataTypeConversionSuite.java | 150 +++++ .../org/apache/spark/sql/DataTypeSuite.scala | 58 ++ .../scala/org/apache/spark/sql/RowSuite.scala | 46 ++ .../org/apache/spark/sql/SQLQuerySuite.scala | 64 +- .../scala/org/apache/spark/sql/TestData.scala | 7 + .../ScalaSideDataTypeConversionSuite.scala | 81 +++ .../org/apache/spark/sql/json/JsonSuite.scala | 198 +++--- .../apache/spark/sql/hive/HiveContext.scala | 9 +- .../spark/sql/hive/HiveInspectors.scala | 5 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 +- 61 files changed, 3442 insertions(+), 386 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/ArrayType.java rename sql/{catalyst/src/main/scala/org/apache/spark/sql/package.scala => core/src/main/java/org/apache/spark/sql/api/java/types/BinaryType.java} (59%) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/BooleanType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/ByteType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/DataType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/DecimalType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/DoubleType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/FloatType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/IntegerType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/LongType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/MapType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/ShortType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/StringType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructField.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/TimestampType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java rename sql/{catalyst => core}/src/main/scala/org/apache/spark/sql/package-info.java (100%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/package.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala create mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java create mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java create mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 0d8453fb184a3..f551a59ee3fe8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -544,7 +544,8 @@ private[spark] object PythonRDD extends Logging { } /** - * Convert an RDD of serialized Python dictionaries to Scala Maps + * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions). + * It is only used by pyspark.sql. * TODO: Support more Python types. */ def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 490fac3cc3646..e2dab0f9f79ea 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -312,7 +312,7 @@ object Unidoc { "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", "mllib.tree.impurity", "mllib.tree.model", "mllib.util" ), - "-group", "Spark SQL", packageList("sql.api.java", "sql.hive.api.java"), + "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"), "-noqualifier", "java.lang" ) ) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index a6b3277db3266..13f0ed4e35490 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -20,7 +20,451 @@ from py4j.protocol import Py4JError -__all__ = ["SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", "SchemaRDD", "Row"] +__all__ = [ + "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", + "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", + "ShortType", "ArrayType", "MapType", "StructField", "StructType", + "SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", "SchemaRDD", "Row"] + + +class PrimitiveTypeSingleton(type): + _instances = {} + + def __call__(cls): + if cls not in cls._instances: + cls._instances[cls] = super(PrimitiveTypeSingleton, cls).__call__() + return cls._instances[cls] + + +class StringType(object): + """Spark SQL StringType + + The data type representing string values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "StringType" + + +class BinaryType(object): + """Spark SQL BinaryType + + The data type representing bytearray values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "BinaryType" + + +class BooleanType(object): + """Spark SQL BooleanType + + The data type representing bool values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "BooleanType" + + +class TimestampType(object): + """Spark SQL TimestampType + + The data type representing datetime.datetime values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "TimestampType" + + +class DecimalType(object): + """Spark SQL DecimalType + + The data type representing decimal.Decimal values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "DecimalType" + + +class DoubleType(object): + """Spark SQL DoubleType + + The data type representing float values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "DoubleType" + + +class FloatType(object): + """Spark SQL FloatType + + The data type representing single precision floating-point values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "FloatType" + + +class ByteType(object): + """Spark SQL ByteType + + The data type representing int values with 1 singed byte. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "ByteType" + + +class IntegerType(object): + """Spark SQL IntegerType + + The data type representing int values. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "IntegerType" + + +class LongType(object): + """Spark SQL LongType + + The data type representing long values. If the any value is beyond the range of + [-9223372036854775808, 9223372036854775807], please use DecimalType. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "LongType" + + +class ShortType(object): + """Spark SQL ShortType + + The data type representing int values with 2 signed bytes. + + """ + __metaclass__ = PrimitiveTypeSingleton + + def __repr__(self): + return "ShortType" + + +class ArrayType(object): + """Spark SQL ArrayType + + The data type representing list values. + An ArrayType object comprises two fields, elementType (a DataType) and containsNull (a bool). + The field of elementType is used to specify the type of array elements. + The field of containsNull is used to specify if the array has None values. + + """ + def __init__(self, elementType, containsNull=False): + """Creates an ArrayType + + :param elementType: the data type of elements. + :param containsNull: indicates whether the list contains None values. + + >>> ArrayType(StringType) == ArrayType(StringType, False) + True + >>> ArrayType(StringType, True) == ArrayType(StringType) + False + """ + self.elementType = elementType + self.containsNull = containsNull + + def __repr__(self): + return "ArrayType(" + self.elementType.__repr__() + "," + \ + str(self.containsNull).lower() + ")" + + def __eq__(self, other): + return (isinstance(other, self.__class__) and + self.elementType == other.elementType and + self.containsNull == other.containsNull) + + def __ne__(self, other): + return not self.__eq__(other) + + +class MapType(object): + """Spark SQL MapType + + The data type representing dict values. + A MapType object comprises three fields, + keyType (a DataType), valueType (a DataType) and valueContainsNull (a bool). + The field of keyType is used to specify the type of keys in the map. + The field of valueType is used to specify the type of values in the map. + The field of valueContainsNull is used to specify if values of this map has None values. + For values of a MapType column, keys are not allowed to have None values. + + """ + def __init__(self, keyType, valueType, valueContainsNull=True): + """Creates a MapType + :param keyType: the data type of keys. + :param valueType: the data type of values. + :param valueContainsNull: indicates whether values contains null values. + + >>> MapType(StringType, IntegerType) == MapType(StringType, IntegerType, True) + True + >>> MapType(StringType, IntegerType, False) == MapType(StringType, FloatType) + False + """ + self.keyType = keyType + self.valueType = valueType + self.valueContainsNull = valueContainsNull + + def __repr__(self): + return "MapType(" + self.keyType.__repr__() + "," + \ + self.valueType.__repr__() + "," + \ + str(self.valueContainsNull).lower() + ")" + + def __eq__(self, other): + return (isinstance(other, self.__class__) and + self.keyType == other.keyType and + self.valueType == other.valueType and + self.valueContainsNull == other.valueContainsNull) + + def __ne__(self, other): + return not self.__eq__(other) + + +class StructField(object): + """Spark SQL StructField + + Represents a field in a StructType. + A StructField object comprises three fields, name (a string), dataType (a DataType), + and nullable (a bool). The field of name is the name of a StructField. The field of + dataType specifies the data type of a StructField. + The field of nullable specifies if values of a StructField can contain None values. + + """ + def __init__(self, name, dataType, nullable): + """Creates a StructField + :param name: the name of this field. + :param dataType: the data type of this field. + :param nullable: indicates whether values of this field can be null. + + >>> StructField("f1", StringType, True) == StructField("f1", StringType, True) + True + >>> StructField("f1", StringType, True) == StructField("f2", StringType, True) + False + """ + self.name = name + self.dataType = dataType + self.nullable = nullable + + def __repr__(self): + return "StructField(" + self.name + "," + \ + self.dataType.__repr__() + "," + \ + str(self.nullable).lower() + ")" + + def __eq__(self, other): + return (isinstance(other, self.__class__) and + self.name == other.name and + self.dataType == other.dataType and + self.nullable == other.nullable) + + def __ne__(self, other): + return not self.__eq__(other) + + +class StructType(object): + """Spark SQL StructType + + The data type representing namedtuple values. + A StructType object comprises a list of L{StructField}s. + + """ + def __init__(self, fields): + """Creates a StructType + + >>> struct1 = StructType([StructField("f1", StringType, True)]) + >>> struct2 = StructType([StructField("f1", StringType, True)]) + >>> struct1 == struct2 + True + >>> struct1 = StructType([StructField("f1", StringType, True)]) + >>> struct2 = StructType([StructField("f1", StringType, True), + ... [StructField("f2", IntegerType, False)]]) + >>> struct1 == struct2 + False + """ + self.fields = fields + + def __repr__(self): + return "StructType(List(" + \ + ",".join([field.__repr__() for field in self.fields]) + "))" + + def __eq__(self, other): + return (isinstance(other, self.__class__) and + self.fields == other.fields) + + def __ne__(self, other): + return not self.__eq__(other) + + +def _parse_datatype_list(datatype_list_string): + """Parses a list of comma separated data types.""" + index = 0 + datatype_list = [] + start = 0 + depth = 0 + while index < len(datatype_list_string): + if depth == 0 and datatype_list_string[index] == ",": + datatype_string = datatype_list_string[start:index].strip() + datatype_list.append(_parse_datatype_string(datatype_string)) + start = index + 1 + elif datatype_list_string[index] == "(": + depth += 1 + elif datatype_list_string[index] == ")": + depth -= 1 + + index += 1 + + # Handle the last data type + datatype_string = datatype_list_string[start:index].strip() + datatype_list.append(_parse_datatype_string(datatype_string)) + return datatype_list + + +def _parse_datatype_string(datatype_string): + """Parses the given data type string. + + >>> def check_datatype(datatype): + ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.__repr__()) + ... python_datatype = _parse_datatype_string(scala_datatype.toString()) + ... return datatype == python_datatype + >>> check_datatype(StringType()) + True + >>> check_datatype(BinaryType()) + True + >>> check_datatype(BooleanType()) + True + >>> check_datatype(TimestampType()) + True + >>> check_datatype(DecimalType()) + True + >>> check_datatype(DoubleType()) + True + >>> check_datatype(FloatType()) + True + >>> check_datatype(ByteType()) + True + >>> check_datatype(IntegerType()) + True + >>> check_datatype(LongType()) + True + >>> check_datatype(ShortType()) + True + >>> # Simple ArrayType. + >>> simple_arraytype = ArrayType(StringType(), True) + >>> check_datatype(simple_arraytype) + True + >>> # Simple MapType. + >>> simple_maptype = MapType(StringType(), LongType()) + >>> check_datatype(simple_maptype) + True + >>> # Simple StructType. + >>> simple_structtype = StructType([ + ... StructField("a", DecimalType(), False), + ... StructField("b", BooleanType(), True), + ... StructField("c", LongType(), True), + ... StructField("d", BinaryType(), False)]) + >>> check_datatype(simple_structtype) + True + >>> # Complex StructType. + >>> complex_structtype = StructType([ + ... StructField("simpleArray", simple_arraytype, True), + ... StructField("simpleMap", simple_maptype, True), + ... StructField("simpleStruct", simple_structtype, True), + ... StructField("boolean", BooleanType(), False)]) + >>> check_datatype(complex_structtype) + True + >>> # Complex ArrayType. + >>> complex_arraytype = ArrayType(complex_structtype, True) + >>> check_datatype(complex_arraytype) + True + >>> # Complex MapType. + >>> complex_maptype = MapType(complex_structtype, complex_arraytype, False) + >>> check_datatype(complex_maptype) + True + """ + left_bracket_index = datatype_string.find("(") + if left_bracket_index == -1: + # It is a primitive type. + left_bracket_index = len(datatype_string) + type_or_field = datatype_string[:left_bracket_index] + rest_part = datatype_string[left_bracket_index+1:len(datatype_string)-1].strip() + if type_or_field == "StringType": + return StringType() + elif type_or_field == "BinaryType": + return BinaryType() + elif type_or_field == "BooleanType": + return BooleanType() + elif type_or_field == "TimestampType": + return TimestampType() + elif type_or_field == "DecimalType": + return DecimalType() + elif type_or_field == "DoubleType": + return DoubleType() + elif type_or_field == "FloatType": + return FloatType() + elif type_or_field == "ByteType": + return ByteType() + elif type_or_field == "IntegerType": + return IntegerType() + elif type_or_field == "LongType": + return LongType() + elif type_or_field == "ShortType": + return ShortType() + elif type_or_field == "ArrayType": + last_comma_index = rest_part.rfind(",") + containsNull = True + if rest_part[last_comma_index+1:].strip().lower() == "false": + containsNull = False + elementType = _parse_datatype_string(rest_part[:last_comma_index].strip()) + return ArrayType(elementType, containsNull) + elif type_or_field == "MapType": + last_comma_index = rest_part.rfind(",") + valueContainsNull = True + if rest_part[last_comma_index+1:].strip().lower() == "false": + valueContainsNull = False + keyType, valueType = _parse_datatype_list(rest_part[:last_comma_index].strip()) + return MapType(keyType, valueType, valueContainsNull) + elif type_or_field == "StructField": + first_comma_index = rest_part.find(",") + name = rest_part[:first_comma_index].strip() + last_comma_index = rest_part.rfind(",") + nullable = True + if rest_part[last_comma_index+1:].strip().lower() == "false": + nullable = False + dataType = _parse_datatype_string( + rest_part[first_comma_index+1:last_comma_index].strip()) + return StructField(name, dataType, nullable) + elif type_or_field == "StructType": + # rest_part should be in the format like + # List(StructField(field1,IntegerType,false)). + field_list_string = rest_part[rest_part.find("(")+1:-1] + fields = _parse_datatype_list(field_list_string) + return StructType(fields) class SQLContext: @@ -109,6 +553,40 @@ def inferSchema(self, rdd): srdd = self._ssql_ctx.inferSchema(jrdd.rdd()) return SchemaRDD(srdd, self) + def applySchema(self, rdd, schema): + """Applies the given schema to the given RDD of L{dict}s. + + >>> schema = StructType([StructField("field1", IntegerType(), False), + ... StructField("field2", StringType(), False)]) + >>> srdd = sqlCtx.applySchema(rdd, schema) + >>> sqlCtx.registerRDDAsTable(srdd, "table1") + >>> srdd2 = sqlCtx.sql("SELECT * from table1") + >>> srdd2.collect() == [{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, + ... {"field1" : 3, "field2": "row3"}] + True + >>> from datetime import datetime + >>> rdd = sc.parallelize([{"byte": 127, "short": -32768, "float": 1.0, + ... "time": datetime(2010, 1, 1, 1, 1, 1), "map": {"a": 1}, "struct": {"b": 2}, + ... "list": [1, 2, 3]}]) + >>> schema = StructType([ + ... StructField("byte", ByteType(), False), + ... StructField("short", ShortType(), False), + ... StructField("float", FloatType(), False), + ... StructField("time", TimestampType(), False), + ... StructField("map", MapType(StringType(), IntegerType(), False), False), + ... StructField("struct", StructType([StructField("b", ShortType(), False)]), False), + ... StructField("list", ArrayType(ByteType(), False), False), + ... StructField("null", DoubleType(), True)]) + >>> srdd = sqlCtx.applySchema(rdd, schema).map( + ... lambda x: ( + ... x.byte, x.short, x.float, x.time, x.map["a"], x.struct["b"], x.list, x.null)) + >>> srdd.collect()[0] + (127, -32768, 1.0, datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + """ + jrdd = self._pythonToJavaMap(rdd._jrdd) + srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.__repr__()) + return SchemaRDD(srdd, self) + def registerRDDAsTable(self, rdd, tableName): """Registers the given RDD as a temporary table in the catalog. @@ -139,10 +617,11 @@ def parquetFile(self, path): jschema_rdd = self._ssql_ctx.parquetFile(path) return SchemaRDD(jschema_rdd, self) - def jsonFile(self, path): - """Loads a text file storing one JSON object per line, - returning the result as a L{SchemaRDD}. - It goes through the entire dataset once to determine the schema. + def jsonFile(self, path, schema=None): + """Loads a text file storing one JSON object per line as a L{SchemaRDD}. + + If the schema is provided, applies the given schema to this JSON dataset. + Otherwise, it goes through the entire dataset once to determine the schema. >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() @@ -151,8 +630,8 @@ def jsonFile(self, path): >>> for json in jsonStrings: ... print>>ofn, json >>> ofn.close() - >>> srdd = sqlCtx.jsonFile(jsonFile) - >>> sqlCtx.registerRDDAsTable(srdd, "table1") + >>> srdd1 = sqlCtx.jsonFile(jsonFile) + >>> sqlCtx.registerRDDAsTable(srdd1, "table1") >>> srdd2 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") >>> srdd2.collect() == [ @@ -160,16 +639,45 @@ def jsonFile(self, path): ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] True + >>> srdd3 = sqlCtx.jsonFile(jsonFile, srdd1.schema()) + >>> sqlCtx.registerRDDAsTable(srdd3, "table2") + >>> srdd4 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table2") + >>> srdd4.collect() == [ + ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, + ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, + ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] + True + >>> schema = StructType([ + ... StructField("field2", StringType(), True), + ... StructField("field3", + ... StructType([ + ... StructField("field5", ArrayType(IntegerType(), False), True)]), False)]) + >>> srdd5 = sqlCtx.jsonFile(jsonFile, schema) + >>> sqlCtx.registerRDDAsTable(srdd5, "table3") + >>> srdd6 = sqlCtx.sql( + ... "SELECT field2 AS f1, field3.field5 as f2, field3.field5[0] as f3 from table3") + >>> srdd6.collect() == [ + ... {"f1": "row1", "f2": None, "f3": None}, + ... {"f1": None, "f2": [10, 11], "f3": 10}, + ... {"f1": "row3", "f2": [], "f3": None}] + True """ - jschema_rdd = self._ssql_ctx.jsonFile(path) + if schema is None: + jschema_rdd = self._ssql_ctx.jsonFile(path) + else: + scala_datatype = self._ssql_ctx.parseDataType(schema.__repr__()) + jschema_rdd = self._ssql_ctx.jsonFile(path, scala_datatype) return SchemaRDD(jschema_rdd, self) - def jsonRDD(self, rdd): - """Loads an RDD storing one JSON object per string, returning the result as a L{SchemaRDD}. - It goes through the entire dataset once to determine the schema. + def jsonRDD(self, rdd, schema=None): + """Loads an RDD storing one JSON object per string as a L{SchemaRDD}. - >>> srdd = sqlCtx.jsonRDD(json) - >>> sqlCtx.registerRDDAsTable(srdd, "table1") + If the schema is provided, applies the given schema to this JSON dataset. + Otherwise, it goes through the entire dataset once to determine the schema. + + >>> srdd1 = sqlCtx.jsonRDD(json) + >>> sqlCtx.registerRDDAsTable(srdd1, "table1") >>> srdd2 = sqlCtx.sql( ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") >>> srdd2.collect() == [ @@ -177,6 +685,29 @@ def jsonRDD(self, rdd): ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] True + >>> srdd3 = sqlCtx.jsonRDD(json, srdd1.schema()) + >>> sqlCtx.registerRDDAsTable(srdd3, "table2") + >>> srdd4 = sqlCtx.sql( + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table2") + >>> srdd4.collect() == [ + ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, + ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, + ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] + True + >>> schema = StructType([ + ... StructField("field2", StringType(), True), + ... StructField("field3", + ... StructType([ + ... StructField("field5", ArrayType(IntegerType(), False), True)]), False)]) + >>> srdd5 = sqlCtx.jsonRDD(json, schema) + >>> sqlCtx.registerRDDAsTable(srdd5, "table3") + >>> srdd6 = sqlCtx.sql( + ... "SELECT field2 AS f1, field3.field5 as f2, field3.field5[0] as f3 from table3") + >>> srdd6.collect() == [ + ... {"f1": "row1", "f2": None, "f3": None}, + ... {"f1": None, "f2": [10, 11], "f3": 10}, + ... {"f1": "row3", "f2": [], "f3": None}] + True """ def func(split, iterator): for x in iterator: @@ -186,7 +717,11 @@ def func(split, iterator): keyed = PipelinedRDD(rdd, func) keyed._bypass_serializer = True jrdd = keyed._jrdd.map(self._jvm.BytesToString()) - jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + if schema is None: + jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + else: + scala_datatype = self._ssql_ctx.parseDataType(schema.__repr__()) + jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) return SchemaRDD(jschema_rdd, self) def sql(self, sqlQuery): @@ -389,6 +924,10 @@ def saveAsTable(self, tableName): """Creates a new table with the contents of this SchemaRDD.""" self._jschema_rdd.saveAsTable(tableName) + def schema(self): + """Returns the schema of this SchemaRDD (represented by a L{StructType}).""" + return _parse_datatype_string(self._jschema_rdd.schema().toString()) + def schemaString(self): """Returns the output schema in the tree format.""" return self._jschema_rdd.schemaString() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 5a55be1e51558..0d26b52a84695 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -85,6 +85,26 @@ object ScalaReflection { case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) } + def typeOfObject: PartialFunction[Any, DataType] = { + // The data type can be determined without ambiguity. + case obj: BooleanType.JvmType => BooleanType + case obj: BinaryType.JvmType => BinaryType + case obj: StringType.JvmType => StringType + case obj: ByteType.JvmType => ByteType + case obj: ShortType.JvmType => ShortType + case obj: IntegerType.JvmType => IntegerType + case obj: LongType.JvmType => LongType + case obj: FloatType.JvmType => FloatType + case obj: DoubleType.JvmType => DoubleType + case obj: DecimalType.JvmType => DecimalType + case obj: TimestampType.JvmType => TimestampType + case null => NullType + // For other cases, there is no obvious mapping from the type of the given object to a + // Catalyst data type. A user should provide his/her specific rules + // (in a user-defined PartialFunction) to infer the Catalyst data type for other types of + // objects and then compose the user-defined PartialFunction with this one. + } + implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index a3ebec8082cbd..f38f99569f207 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -17,14 +17,11 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.Logging - /** * A bound reference points to a specific slot in the input tuple, allowing the actual value * to be retrieved more efficiently. However, since operations like column pruning can change diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index 7470cb861b83b..c9a63e201ef60 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -32,6 +32,16 @@ object Row { * }}} */ def unapplySeq(row: Row): Some[Seq[Any]] = Some(row) + + /** + * This method can be used to construct a [[Row]] with the given values. + */ + def apply(values: Any*): Row = new GenericRow(values.toArray) + + /** + * This method can be used to construct a [[Row]] from a [[Seq]] of values. + */ + def fromSeq(values: Seq[Any]): Row = new GenericRow(values.toArray) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index e787c59e75723..eb8898900d6a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -21,8 +21,16 @@ import scala.language.dynamics import org.apache.spark.sql.catalyst.types.DataType -case object DynamicType extends DataType +/** + * The data type representing [[DynamicRow]] values. + */ +case object DynamicType extends DataType { + def simpleString: String = "dynamic" +} +/** + * Wrap a [[Row]] as a [[DynamicRow]]. + */ case class WrapDynamic(children: Seq[Attribute]) extends Expression { type EvaluatedType = DynamicRow @@ -37,6 +45,11 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { } } +/** + * DynamicRows use scala's Dynamic trait to emulate an ORM of in a dynamically typed language. + * Since the type of the column is not known at compile time, all attributes are converted to + * strings before being passed to the function. + */ class DynamicRow(val schema: Seq[Attribute], values: Array[Any]) extends GenericRow(values) with Dynamic { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 0acb29012f314..72add5e20e8b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -31,8 +31,8 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def foldable = child.foldable && ordinal.foldable override def references = children.flatMap(_.references).toSet def dataType = child.dataType match { - case ArrayType(dt) => dt - case MapType(_, vt) => vt + case ArrayType(dt, _) => dt + case MapType(_, vt, _) => vt } override lazy val resolved = childrenResolved && diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index dd78614754e12..422839dab770d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -84,8 +84,8 @@ case class Explode(attributeNames: Seq[String], child: Expression) (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) private lazy val elementTypes = child.dataType match { - case ArrayType(et) => et :: Nil - case MapType(kt,vt) => kt :: vt :: Nil + case ArrayType(et, _) => et :: Nil + case MapType(kt,vt, _) => kt :: vt :: Nil } // TODO: Move this pattern into Generator. @@ -102,10 +102,10 @@ case class Explode(attributeNames: Seq[String], child: Expression) override def eval(input: Row): TraversableOnce[Row] = { child.dataType match { - case ArrayType(_) => + case ArrayType(_, _) => val inputArray = child.eval(input).asInstanceOf[Seq[Any]] if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v))) - case MapType(_, _) => + case MapType(_, _, _) => val inputMap = child.eval(input).asInstanceOf[Map[Any,Any]] if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Array(k,v)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala index 3b3e206055cfc..ca9642954eb27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -24,4 +24,6 @@ package object catalyst { * 2.10.* builds. See SI-6240 for more details. */ protected[catalyst] object ScalaReflectionLock + + protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 67833664b35ae..781ba489b44c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.planning -import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 418f8686bfe5c..bc763a4e06e67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec -import org.apache.spark.sql.Logging - import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 7b82e19b2e714..0988b0c6d990c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -125,51 +125,10 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy }.toSeq } - protected def generateSchemaString(schema: Seq[Attribute]): String = { - val builder = new StringBuilder - builder.append("root\n") - val prefix = " |" - schema.foreach { attribute => - val name = attribute.name - val dataType = attribute.dataType - dataType match { - case fields: StructType => - builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaString(fields, s"$prefix |", builder) - case ArrayType(fields: StructType) => - builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaString(fields, s"$prefix |", builder) - case ArrayType(elementType: DataType) => - builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") - case _ => builder.append(s"$prefix-- $name: $dataType\n") - } - } - - builder.toString() - } - - protected def generateSchemaString( - schema: StructType, - prefix: String, - builder: StringBuilder): StringBuilder = { - schema.fields.foreach { - case StructField(name, fields: StructType, _) => - builder.append(s"$prefix-- $name: $StructType\n") - generateSchemaString(fields, s"$prefix |", builder) - case StructField(name, ArrayType(fields: StructType), _) => - builder.append(s"$prefix-- $name: $ArrayType[$StructType]\n") - generateSchemaString(fields, s"$prefix |", builder) - case StructField(name, ArrayType(elementType: DataType), _) => - builder.append(s"$prefix-- $name: $ArrayType[$elementType]\n") - case StructField(name, fieldType: DataType, _) => - builder.append(s"$prefix-- $name: $fieldType\n") - } - - builder - } + def schema: StructType = StructType.fromAttributes(output) /** Returns the output schema in the tree format. */ - def schemaString: String = generateSchemaString(output) + def schemaString: String = schema.treeString /** Prints out the schema in the tree format */ def printSchema(): Unit = println(schemaString) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 1537de259c5b4..3cb407217c4c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -177,7 +177,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { case StructType(fields) => StructType(fields.map(f => StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable))) - case ArrayType(elemType) => ArrayType(lowerCaseSchema(elemType)) + case ArrayType(elemType, containsNull) => ArrayType(lowerCaseSchema(elemType), containsNull) case otherType => otherType } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index 1076537bc7602..f8960b3fe7a17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index e300bdbececbd..6aa407c836aec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package rules +package org.apache.spark.sql.catalyst.rules +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.sideBySide diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index d159ecdd5d781..9a28d035a10a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.sql.Logger - /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are * granted the following interface: @@ -35,5 +33,6 @@ import org.apache.spark.sql.Logger */ package object trees { // Since we want tree nodes to be lightweight, we create one logger for all treenode instances. - protected val logger = Logger("catalyst.trees") + protected val logger = + com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("catalyst.trees")) } 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 71808f76d632b..b52ee6d3378a3 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 @@ -45,11 +45,13 @@ object DataType extends RegexParsers { "TimestampType" ^^^ TimestampType protected lazy val arrayType: Parser[DataType] = - "ArrayType" ~> "(" ~> dataType <~ ")" ^^ ArrayType + "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { + case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) + } protected lazy val mapType: Parser[DataType] = - "MapType" ~> "(" ~> dataType ~ "," ~ dataType <~ ")" ^^ { - case t1 ~ _ ~ t2 => MapType(t1, t2) + "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { + case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) } protected lazy val structField: Parser[StructField] = @@ -82,6 +84,21 @@ object DataType extends RegexParsers { case Success(result, _) => result case failure: NoSuccess => sys.error(s"Unsupported dataType: $asString, $failure") } + + protected[types] def buildFormattedString( + dataType: DataType, + prefix: String, + builder: StringBuilder): Unit = { + dataType match { + case array: ArrayType => + array.buildFormattedString(prefix, builder) + case struct: StructType => + struct.buildFormattedString(prefix, builder) + case map: MapType => + map.buildFormattedString(prefix, builder) + case _ => + } + } } abstract class DataType { @@ -92,9 +109,13 @@ abstract class DataType { } def isPrimitive: Boolean = false + + def simpleString: String } -case object NullType extends DataType +case object NullType extends DataType { + def simpleString: String = "null" +} object NativeType { def all = Seq( @@ -108,40 +129,45 @@ trait PrimitiveType extends DataType { } abstract class NativeType extends DataType { - type JvmType - @transient val tag: TypeTag[JvmType] - val ordering: Ordering[JvmType] + private[sql] type JvmType + @transient private[sql] val tag: TypeTag[JvmType] + private[sql] val ordering: Ordering[JvmType] - @transient val classTag = ScalaReflectionLock.synchronized { + @transient private[sql] val classTag = ScalaReflectionLock.synchronized { val mirror = runtimeMirror(Utils.getSparkClassLoader) ClassTag[JvmType](mirror.runtimeClass(tag.tpe)) } } case object StringType extends NativeType with PrimitiveType { - type JvmType = String - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = String + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "string" } case object BinaryType extends DataType with PrimitiveType { - type JvmType = Array[Byte] + private[sql] type JvmType = Array[Byte] + def simpleString: String = "binary" } case object BooleanType extends NativeType with PrimitiveType { - type JvmType = Boolean - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Boolean + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "boolean" } case object TimestampType extends NativeType { - type JvmType = Timestamp + private[sql] type JvmType = Timestamp - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val ordering = new Ordering[JvmType] { + private[sql] val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) } + + def simpleString: String = "timestamp" } abstract class NumericType extends NativeType with PrimitiveType { @@ -150,7 +176,7 @@ abstract class NumericType extends NativeType with PrimitiveType { // type parameter and and add a numeric annotation (i.e., [JvmType : Numeric]). This gets // desugared by the compiler into an argument to the objects constructor. This means there is no // longer an no argument constructor and thus the JVM cannot serialize the object anymore. - val numeric: Numeric[JvmType] + private[sql] val numeric: Numeric[JvmType] } object NumericType { @@ -166,39 +192,43 @@ object IntegralType { } abstract class IntegralType extends NumericType { - val integral: Integral[JvmType] + private[sql] val integral: Integral[JvmType] } case object LongType extends IntegralType { - type JvmType = Long - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val numeric = implicitly[Numeric[Long]] - val integral = implicitly[Integral[Long]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Long + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Long]] + private[sql] val integral = implicitly[Integral[Long]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "long" } case object IntegerType extends IntegralType { - type JvmType = Int - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val numeric = implicitly[Numeric[Int]] - val integral = implicitly[Integral[Int]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Int + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Int]] + private[sql] val integral = implicitly[Integral[Int]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "integer" } case object ShortType extends IntegralType { - type JvmType = Short - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val numeric = implicitly[Numeric[Short]] - val integral = implicitly[Integral[Short]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Short + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Short]] + private[sql] val integral = implicitly[Integral[Short]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "short" } case object ByteType extends IntegralType { - type JvmType = Byte - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val numeric = implicitly[Numeric[Byte]] - val integral = implicitly[Integral[Byte]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Byte + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Byte]] + private[sql] val integral = implicitly[Integral[Byte]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "byte" } /** Matcher for any expressions that evaluate to [[FractionalType]]s */ @@ -209,47 +239,159 @@ object FractionalType { } } abstract class FractionalType extends NumericType { - val fractional: Fractional[JvmType] + private[sql] val fractional: Fractional[JvmType] } case object DecimalType extends FractionalType { - type JvmType = BigDecimal - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val numeric = implicitly[Numeric[BigDecimal]] - val fractional = implicitly[Fractional[BigDecimal]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = BigDecimal + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[BigDecimal]] + private[sql] val fractional = implicitly[Fractional[BigDecimal]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "decimal" } case object DoubleType extends FractionalType { - type JvmType = Double - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val numeric = implicitly[Numeric[Double]] - val fractional = implicitly[Fractional[Double]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Double + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Double]] + private[sql] val fractional = implicitly[Fractional[Double]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "double" } case object FloatType extends FractionalType { - type JvmType = Float - @transient lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - val numeric = implicitly[Numeric[Float]] - val fractional = implicitly[Fractional[Float]] - val ordering = implicitly[Ordering[JvmType]] + private[sql] type JvmType = Float + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val numeric = implicitly[Numeric[Float]] + private[sql] val fractional = implicitly[Fractional[Float]] + private[sql] val ordering = implicitly[Ordering[JvmType]] + def simpleString: String = "float" } -case class ArrayType(elementType: DataType) extends DataType +object ArrayType { + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is false. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, false) +} -case class StructField(name: String, dataType: DataType, nullable: Boolean) +/** + * The data type for collections of multiple values. + * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * + * @param elementType The data type of values. + * @param containsNull Indicates if values have `null` values + */ +case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append( + s"${prefix}-- element: ${elementType.simpleString} (containsNull = ${containsNull})\n") + DataType.buildFormattedString(elementType, s"$prefix |", builder) + } + + def simpleString: String = "array" +} + +/** + * A field inside a StructType. + * @param name The name of this field. + * @param dataType The data type of this field. + * @param nullable Indicates if values of this field can be `null` values. + */ +case class StructField(name: String, dataType: DataType, nullable: Boolean) { + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"${prefix}-- ${name}: ${dataType.simpleString} (nullable = ${nullable})\n") + DataType.buildFormattedString(dataType, s"$prefix |", builder) + } +} object StructType { - def fromAttributes(attributes: Seq[Attribute]): StructType = { + protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) - } - // def apply(fields: Seq[StructField]) = new StructType(fields.toIndexedSeq) + private def validateFields(fields: Seq[StructField]): Boolean = + fields.map(field => field.name).distinct.size == fields.size } case class StructType(fields: Seq[StructField]) extends DataType { - def toAttributes = fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) + require(StructType.validateFields(fields), "Found fields with the same name.") + + /** + * Returns all field names in a [[Seq]]. + */ + lazy val fieldNames: Seq[String] = fields.map(_.name) + private lazy val fieldNamesSet: Set[String] = fieldNames.toSet + private lazy val nameToField: Map[String, StructField] = fields.map(f => f.name -> f).toMap + /** + * Extracts a [[StructField]] of the given name. If the [[StructType]] object does not + * have a name matching the given name, `null` will be returned. + */ + def apply(name: String): StructField = { + nameToField.get(name).getOrElse( + throw new IllegalArgumentException(s"Field ${name} does not exist.")) + } + + /** + * Returns a [[StructType]] containing [[StructField]]s of the given names. + * Those names which do not have matching fields will be ignored. + */ + def apply(names: Set[String]): StructType = { + val nonExistFields = names -- fieldNamesSet + if (!nonExistFields.isEmpty) { + throw new IllegalArgumentException( + s"Field ${nonExistFields.mkString(",")} does not exist.") + } + // Preserve the original order of fields. + StructType(fields.filter(f => names.contains(f.name))) + } + + protected[sql] def toAttributes = + fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) + + def treeString: String = { + val builder = new StringBuilder + builder.append("root\n") + val prefix = " |" + fields.foreach(field => field.buildFormattedString(prefix, builder)) + + builder.toString() + } + + def printTreeString(): Unit = println(treeString) + + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + fields.foreach(field => field.buildFormattedString(prefix, builder)) + } + + def simpleString: String = "struct" +} + +object MapType { + /** + * Construct a [[MapType]] object with the given key type and value type. + * The `valueContainsNull` is true. + */ + def apply(keyType: DataType, valueType: DataType): MapType = + MapType(keyType: DataType, valueType: DataType, true) } -case class MapType(keyType: DataType, valueType: DataType) extends DataType +/** + * The data type for Maps. Keys in a map are not allowed to have `null` values. + * @param keyType The data type of map keys. + * @param valueType The data type of map values. + * @param valueContainsNull Indicates if map values have `null` values. + */ +case class MapType( + keyType: DataType, + valueType: DataType, + valueContainsNull: Boolean) extends DataType { + private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { + builder.append(s"${prefix}-- key: ${keyType.simpleString}\n") + builder.append(s"${prefix}-- value: ${valueType.simpleString} " + + s"(valueContainsNull = ${valueContainsNull})\n") + DataType.buildFormattedString(keyType, s"$prefix |", builder) + DataType.buildFormattedString(valueType, s"$prefix |", builder) + } + + def simpleString: String = "map" +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index c0438dbe52a47..e030d6e13d472 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.catalyst +import java.math.BigInteger import java.sql.Timestamp import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ case class PrimitiveData( @@ -148,4 +148,68 @@ class ScalaReflectionSuite extends FunSuite { StructField("_2", StringType, nullable = true))), nullable = true)) } + + test("get data type of a value") { + // BooleanType + assert(BooleanType === typeOfObject(true)) + assert(BooleanType === typeOfObject(false)) + + // BinaryType + assert(BinaryType === typeOfObject("string".getBytes)) + + // StringType + assert(StringType === typeOfObject("string")) + + // ByteType + assert(ByteType === typeOfObject(127.toByte)) + + // ShortType + assert(ShortType === typeOfObject(32767.toShort)) + + // IntegerType + assert(IntegerType === typeOfObject(2147483647)) + + // LongType + assert(LongType === typeOfObject(9223372036854775807L)) + + // FloatType + assert(FloatType === typeOfObject(3.4028235E38.toFloat)) + + // DoubleType + assert(DoubleType === typeOfObject(1.7976931348623157E308)) + + // DecimalType + assert(DecimalType === typeOfObject(BigDecimal("1.7976931348623157E318"))) + + // TimestampType + assert(TimestampType === typeOfObject(java.sql.Timestamp.valueOf("2014-7-25 10:26:00"))) + + // NullType + assert(NullType === typeOfObject(null)) + + def typeOfObject1: PartialFunction[Any, DataType] = typeOfObject orElse { + case value: java.math.BigInteger => DecimalType + case value: java.math.BigDecimal => DecimalType + case _ => StringType + } + + assert(DecimalType === typeOfObject1( + new BigInteger("92233720368547758070"))) + assert(DecimalType === typeOfObject1( + new java.math.BigDecimal("1.7976931348623157E318"))) + assert(StringType === typeOfObject1(BigInt("92233720368547758070"))) + + def typeOfObject2: PartialFunction[Any, DataType] = typeOfObject orElse { + case value: java.math.BigInteger => DecimalType + } + + intercept[MatchError](typeOfObject2(BigInt("92233720368547758070"))) + + def typeOfObject3: PartialFunction[Any, DataType] = typeOfObject orElse { + case c: Seq[_] => ArrayType(typeOfObject3(c.head)) + } + + assert(ArrayType(IntegerType) === typeOfObject3(Seq(1, 2, 3))) + assert(ArrayType(ArrayType(IntegerType)) === typeOfObject3(Seq(Seq(1,2,3)))) + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ArrayType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ArrayType.java new file mode 100644 index 0000000000000..17334ca31b2b7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ArrayType.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing Lists. + * An ArrayType object comprises two fields, {@code DataType elementType} and + * {@code boolean containsNull}. The field of {@code elementType} is used to specify the type of + * array elements. The field of {@code containsNull} is used to specify if the array has + * {@code null} values. + * + * To create an {@link ArrayType}, + * {@link org.apache.spark.sql.api.java.types.DataType#createArrayType(DataType)} or + * {@link org.apache.spark.sql.api.java.types.DataType#createArrayType(DataType, boolean)} + * should be used. + */ +public class ArrayType extends DataType { + private DataType elementType; + private boolean containsNull; + + protected ArrayType(DataType elementType, boolean containsNull) { + this.elementType = elementType; + this.containsNull = containsNull; + } + + public DataType getElementType() { + return elementType; + } + + public boolean isContainsNull() { + return containsNull; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ArrayType arrayType = (ArrayType) o; + + if (containsNull != arrayType.containsNull) return false; + if (!elementType.equals(arrayType.elementType)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = elementType.hashCode(); + result = 31 * result + (containsNull ? 1 : 0); + return result; + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BinaryType.java similarity index 59% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala rename to sql/core/src/main/java/org/apache/spark/sql/api/java/types/BinaryType.java index 4589129cd1c90..61703179850e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BinaryType.java @@ -15,22 +15,13 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.sql.api.java.types; /** - * Allows the execution of relational queries, including those expressed in SQL using Spark. + * The data type representing byte[] values. * - * Note that this package is located in catalyst instead of in core so that all subprojects can - * inherit the settings from this package object. + * {@code BinaryType} is represented by the singleton object {@link DataType#BinaryType}. */ -package object sql { - - protected[sql] def Logger(name: String) = - com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger(name)) - - protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging - - type Row = catalyst.expressions.Row - - val Row = catalyst.expressions.Row +public class BinaryType extends DataType { + protected BinaryType() {} } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BooleanType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BooleanType.java new file mode 100644 index 0000000000000..8fa24d85d1238 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BooleanType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing boolean and Boolean values. + * + * {@code BooleanType} is represented by the singleton object {@link DataType#BooleanType}. + */ +public class BooleanType extends DataType { + protected BooleanType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ByteType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ByteType.java new file mode 100644 index 0000000000000..2de32978e2705 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ByteType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing byte and Byte values. + * + * {@code ByteType} is represented by the singleton object {@link DataType#ByteType}. + */ +public class ByteType extends DataType { + protected ByteType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DataType.java new file mode 100644 index 0000000000000..f84e5a490a905 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DataType.java @@ -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.sql.api.java.types; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * The base type of all Spark SQL data types. + * + * To get/create specific data type, users should use singleton objects and factory methods + * provided by this class. + */ +public abstract class DataType { + + /** + * Gets the StringType object. + */ + public static final StringType StringType = new StringType(); + + /** + * Gets the BinaryType object. + */ + public static final BinaryType BinaryType = new BinaryType(); + + /** + * Gets the BooleanType object. + */ + public static final BooleanType BooleanType = new BooleanType(); + + /** + * Gets the TimestampType object. + */ + public static final TimestampType TimestampType = new TimestampType(); + + /** + * Gets the DecimalType object. + */ + public static final DecimalType DecimalType = new DecimalType(); + + /** + * Gets the DoubleType object. + */ + public static final DoubleType DoubleType = new DoubleType(); + + /** + * Gets the FloatType object. + */ + public static final FloatType FloatType = new FloatType(); + + /** + * Gets the ByteType object. + */ + public static final ByteType ByteType = new ByteType(); + + /** + * Gets the IntegerType object. + */ + public static final IntegerType IntegerType = new IntegerType(); + + /** + * Gets the LongType object. + */ + public static final LongType LongType = new LongType(); + + /** + * Gets the ShortType object. + */ + public static final ShortType ShortType = new ShortType(); + + /** + * Creates an ArrayType by specifying the data type of elements ({@code elementType}). + * The field of {@code containsNull} is set to {@code false}. + */ + public static ArrayType createArrayType(DataType elementType) { + if (elementType == null) { + throw new IllegalArgumentException("elementType should not be null."); + } + + return new ArrayType(elementType, false); + } + + /** + * Creates an ArrayType by specifying the data type of elements ({@code elementType}) and + * whether the array contains null values ({@code containsNull}). + */ + public static ArrayType createArrayType(DataType elementType, boolean containsNull) { + if (elementType == null) { + throw new IllegalArgumentException("elementType should not be null."); + } + + return new ArrayType(elementType, containsNull); + } + + /** + * Creates a MapType by specifying the data type of keys ({@code keyType}) and values + * ({@code keyType}). The field of {@code valueContainsNull} is set to {@code true}. + */ + public static MapType createMapType(DataType keyType, DataType valueType) { + if (keyType == null) { + throw new IllegalArgumentException("keyType should not be null."); + } + if (valueType == null) { + throw new IllegalArgumentException("valueType should not be null."); + } + + return new MapType(keyType, valueType, true); + } + + /** + * Creates a MapType by specifying the data type of keys ({@code keyType}), the data type of + * values ({@code keyType}), and whether values contain any null value + * ({@code valueContainsNull}). + */ + public static MapType createMapType( + DataType keyType, + DataType valueType, + boolean valueContainsNull) { + if (keyType == null) { + throw new IllegalArgumentException("keyType should not be null."); + } + if (valueType == null) { + throw new IllegalArgumentException("valueType should not be null."); + } + + return new MapType(keyType, valueType, valueContainsNull); + } + + /** + * Creates a StructField by specifying the name ({@code name}), data type ({@code dataType}) and + * whether values of this field can be null values ({@code nullable}). + */ + public static StructField createStructField(String name, DataType dataType, boolean nullable) { + if (name == null) { + throw new IllegalArgumentException("name should not be null."); + } + if (dataType == null) { + throw new IllegalArgumentException("dataType should not be null."); + } + + return new StructField(name, dataType, nullable); + } + + /** + * Creates a StructType with the given list of StructFields ({@code fields}). + */ + public static StructType createStructType(List fields) { + return createStructType(fields.toArray(new StructField[0])); + } + + /** + * Creates a StructType with the given StructField array ({@code fields}). + */ + public static StructType createStructType(StructField[] fields) { + if (fields == null) { + throw new IllegalArgumentException("fields should not be null."); + } + Set distinctNames = new HashSet(); + for (StructField field: fields) { + if (field == null) { + throw new IllegalArgumentException( + "fields should not contain any null."); + } + + distinctNames.add(field.getName()); + } + if (distinctNames.size() != fields.length) { + throw new IllegalArgumentException("fields should have distinct names."); + } + + return new StructType(fields); + } + +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DecimalType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DecimalType.java new file mode 100644 index 0000000000000..9250491a2d2ca --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DecimalType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing java.math.BigDecimal values. + * + * {@code DecimalType} is represented by the singleton object {@link DataType#DecimalType}. + */ +public class DecimalType extends DataType { + protected DecimalType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DoubleType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DoubleType.java new file mode 100644 index 0000000000000..3e86917fddc4b --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DoubleType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing double and Double values. + * + * {@code DoubleType} is represented by the singleton object {@link DataType#DoubleType}. + */ +public class DoubleType extends DataType { + protected DoubleType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/FloatType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/FloatType.java new file mode 100644 index 0000000000000..fa860d40176ef --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/FloatType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing float and Float values. + * + * {@code FloatType} is represented by the singleton object {@link DataType#FloatType}. + */ +public class FloatType extends DataType { + protected FloatType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/IntegerType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/IntegerType.java new file mode 100644 index 0000000000000..bd973eca2c3ce --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/IntegerType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing int and Integer values. + * + * {@code IntegerType} is represented by the singleton object {@link DataType#IntegerType}. + */ +public class IntegerType extends DataType { + protected IntegerType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/LongType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/LongType.java new file mode 100644 index 0000000000000..e00233304cefa --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/LongType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing long and Long values. + * + * {@code LongType} is represented by the singleton object {@link DataType#LongType}. + */ +public class LongType extends DataType { + protected LongType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/MapType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/MapType.java new file mode 100644 index 0000000000000..94936e2e4ee7a --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/MapType.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing Maps. A MapType object comprises two fields, + * {@code DataType keyType}, {@code DataType valueType}, and {@code boolean valueContainsNull}. + * The field of {@code keyType} is used to specify the type of keys in the map. + * The field of {@code valueType} is used to specify the type of values in the map. + * The field of {@code valueContainsNull} is used to specify if map values have + * {@code null} values. + * For values of a MapType column, keys are not allowed to have {@code null} values. + * + * To create a {@link MapType}, + * {@link org.apache.spark.sql.api.java.types.DataType#createMapType(DataType, DataType)} or + * {@link org.apache.spark.sql.api.java.types.DataType#createMapType(DataType, DataType, boolean)} + * should be used. + */ +public class MapType extends DataType { + private DataType keyType; + private DataType valueType; + private boolean valueContainsNull; + + protected MapType(DataType keyType, DataType valueType, boolean valueContainsNull) { + this.keyType = keyType; + this.valueType = valueType; + this.valueContainsNull = valueContainsNull; + } + + public DataType getKeyType() { + return keyType; + } + + public DataType getValueType() { + return valueType; + } + + public boolean isValueContainsNull() { + return valueContainsNull; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + MapType mapType = (MapType) o; + + if (valueContainsNull != mapType.valueContainsNull) return false; + if (!keyType.equals(mapType.keyType)) return false; + if (!valueType.equals(mapType.valueType)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = keyType.hashCode(); + result = 31 * result + valueType.hashCode(); + result = 31 * result + (valueContainsNull ? 1 : 0); + return result; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ShortType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ShortType.java new file mode 100644 index 0000000000000..98f9507acf121 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ShortType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing short and Short values. + * + * {@code ShortType} is represented by the singleton object {@link DataType#ShortType}. + */ +public class ShortType extends DataType { + protected ShortType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StringType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StringType.java new file mode 100644 index 0000000000000..b8e7dbe646071 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StringType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing String values. + * + * {@code StringType} is represented by the singleton object {@link DataType#StringType}. + */ +public class StringType extends DataType { + protected StringType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructField.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructField.java new file mode 100644 index 0000000000000..54e9c11ea415e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructField.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * A StructField object represents a field in a StructType object. + * A StructField object comprises three fields, {@code String name}, {@code DataType dataType}, + * and {@code boolean nullable}. The field of {@code name} is the name of a StructField. + * The field of {@code dataType} specifies the data type of a StructField. + * The field of {@code nullable} specifies if values of a StructField can contain {@code null} + * values. + * + * To create a {@link StructField}, + * {@link org.apache.spark.sql.api.java.types.DataType#createStructField(String, DataType, boolean)} + * should be used. + */ +public class StructField { + private String name; + private DataType dataType; + private boolean nullable; + + protected StructField(String name, DataType dataType, boolean nullable) { + this.name = name; + this.dataType = dataType; + this.nullable = nullable; + } + + public String getName() { + return name; + } + + public DataType getDataType() { + return dataType; + } + + public boolean isNullable() { + return nullable; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StructField that = (StructField) o; + + if (nullable != that.nullable) return false; + if (!dataType.equals(that.dataType)) return false; + if (!name.equals(that.name)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + dataType.hashCode(); + result = 31 * result + (nullable ? 1 : 0); + return result; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructType.java new file mode 100644 index 0000000000000..33a42f4b16265 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructType.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +import java.util.Arrays; +import java.util.List; + +/** + * The data type representing Rows. + * A StructType object comprises an array of StructFields. + * + * To create an {@link StructType}, + * {@link org.apache.spark.sql.api.java.types.DataType#createStructType(java.util.List)} or + * {@link org.apache.spark.sql.api.java.types.DataType#createStructType(StructField[])} + * should be used. + */ +public class StructType extends DataType { + private StructField[] fields; + + protected StructType(StructField[] fields) { + this.fields = fields; + } + + public StructField[] getFields() { + return fields; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StructType that = (StructType) o; + + if (!Arrays.equals(fields, that.fields)) return false; + + return true; + } + + @Override + public int hashCode() { + return Arrays.hashCode(fields); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/TimestampType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/TimestampType.java new file mode 100644 index 0000000000000..65295779f71ec --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/TimestampType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java.types; + +/** + * The data type representing java.sql.Timestamp values. + * + * {@code TimestampType} is represented by the singleton object {@link DataType#TimestampType}. + */ +public class TimestampType extends DataType { + protected TimestampType() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java new file mode 100644 index 0000000000000..f169ac65e226f --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + + +/** + * Allows users to get and create Spark SQL data types. + */ +package org.apache.spark.sql.api.java.types; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index e4b6810180994..86338752a21c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies @@ -88,6 +87,44 @@ class SQLContext(@transient val sparkContext: SparkContext) implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))(self)) + /** + * :: DeveloperApi :: + * Creates a [[SchemaRDD]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD. + * It is important to make sure that the structure of every [[Row]] of the provided RDD matches + * the provided schema. Otherwise, there will be runtime exception. + * Example: + * {{{ + * import org.apache.spark.sql._ + * val sqlContext = new org.apache.spark.sql.SQLContext(sc) + * + * val schema = + * StructType( + * StructField("name", StringType, false) :: + * StructField("age", IntegerType, true) :: Nil) + * + * val people = + * sc.textFile("examples/src/main/resources/people.txt").map( + * _.split(",")).map(p => Row(p(0), p(1).trim.toInt)) + * val peopleSchemaRDD = sqlContext. applySchema(people, schema) + * peopleSchemaRDD.printSchema + * // root + * // |-- name: string (nullable = false) + * // |-- age: integer (nullable = true) + * + * peopleSchemaRDD.registerAsTable("people") + * sqlContext.sql("select name from people").collect.foreach(println) + * }}} + * + * @group userf + */ + @DeveloperApi + def applySchema(rowRDD: RDD[Row], schema: StructType): SchemaRDD = { + // TODO: use MutableProjection when rowRDD is another SchemaRDD and the applied + // schema differs from the existing schema on any field data type. + val logicalPlan = SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRDD))(self) + new SchemaRDD(this, logicalPlan) + } + /** * Loads a Parquet file, returning the result as a [[SchemaRDD]]. * @@ -104,6 +141,19 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def jsonFile(path: String): SchemaRDD = jsonFile(path, 1.0) + /** + * :: Experimental :: + * Loads a JSON file (one object per line) and applies the given schema, + * returning the result as a [[SchemaRDD]]. + * + * @group userf + */ + @Experimental + def jsonFile(path: String, schema: StructType): SchemaRDD = { + val json = sparkContext.textFile(path) + jsonRDD(json, schema) + } + /** * :: Experimental :: */ @@ -122,12 +172,30 @@ class SQLContext(@transient val sparkContext: SparkContext) */ def jsonRDD(json: RDD[String]): SchemaRDD = jsonRDD(json, 1.0) + /** + * :: Experimental :: + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, + * returning the result as a [[SchemaRDD]]. + * + * @group userf + */ + @Experimental + def jsonRDD(json: RDD[String], schema: StructType): SchemaRDD = { + val appliedSchema = + Option(schema).getOrElse(JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, 1.0))) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema) + applySchema(rowRDD, appliedSchema) + } + /** * :: Experimental :: */ @Experimental - def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = - new SchemaRDD(this, JsonRDD.inferSchema(self, json, samplingRatio)) + def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = { + val appliedSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, samplingRatio)) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema) + applySchema(rowRDD, appliedSchema) + } /** * :: Experimental :: @@ -345,70 +413,138 @@ class SQLContext(@transient val sparkContext: SparkContext) /** * Peek at the first row of the RDD and infer its schema. - * TODO: consolidate this with the type system developed in SPARK-2060. + * It is only used by PySpark. */ private[sql] def inferSchema(rdd: RDD[Map[String, _]]): SchemaRDD = { import scala.collection.JavaConversions._ - def typeFor(obj: Any): DataType = obj match { - case c: java.lang.String => StringType - case c: java.lang.Integer => IntegerType - case c: java.lang.Long => LongType - case c: java.lang.Double => DoubleType - case c: java.lang.Boolean => BooleanType - case c: java.math.BigDecimal => DecimalType - case c: java.sql.Timestamp => TimestampType + + def typeOfComplexValue: PartialFunction[Any, DataType] = { case c: java.util.Calendar => TimestampType - case c: java.util.List[_] => ArrayType(typeFor(c.head)) + case c: java.util.List[_] => + ArrayType(typeOfObject(c.head)) case c: java.util.Map[_, _] => val (key, value) = c.head - MapType(typeFor(key), typeFor(value)) + MapType(typeOfObject(key), typeOfObject(value)) case c if c.getClass.isArray => val elem = c.asInstanceOf[Array[_]].head - ArrayType(typeFor(elem)) + ArrayType(typeOfObject(elem)) case c => throw new Exception(s"Object of type $c cannot be used") } + def typeOfObject = ScalaReflection.typeOfObject orElse typeOfComplexValue + val firstRow = rdd.first() - val schema = firstRow.map { case (fieldName, obj) => - AttributeReference(fieldName, typeFor(obj), true)() + val fields = firstRow.map { + case (fieldName, obj) => StructField(fieldName, typeOfObject(obj), true) }.toSeq - def needTransform(obj: Any): Boolean = obj match { - case c: java.util.List[_] => true - case c: java.util.Map[_, _] => true - case c if c.getClass.isArray => true - case c: java.util.Calendar => true - case c => false + applySchemaToPythonRDD(rdd, StructType(fields)) + } + + /** + * Parses the data type in our internal string representation. The data type string should + * have the same format as the one generated by `toString` in scala. + * It is only used by PySpark. + */ + private[sql] def parseDataType(dataTypeString: String): DataType = { + val parser = org.apache.spark.sql.catalyst.types.DataType + parser(dataTypeString) + } + + /** + * Apply a schema defined by the schemaString to an RDD. It is only used by PySpark. + */ + private[sql] def applySchemaToPythonRDD( + rdd: RDD[Map[String, _]], + schemaString: String): SchemaRDD = { + val schema = parseDataType(schemaString).asInstanceOf[StructType] + applySchemaToPythonRDD(rdd, schema) + } + + /** + * Apply a schema defined by the schema to an RDD. It is only used by PySpark. + */ + private[sql] def applySchemaToPythonRDD( + rdd: RDD[Map[String, _]], + schema: StructType): SchemaRDD = { + // TODO: We should have a better implementation once we do not turn a Python side record + // to a Map. + import scala.collection.JavaConversions._ + import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} + + def needsConversion(dataType: DataType): Boolean = dataType match { + case ByteType => true + case ShortType => true + case FloatType => true + case TimestampType => true + case ArrayType(_, _) => true + case MapType(_, _, _) => true + case StructType(_) => true + case other => false } - // convert JList, JArray into Seq, convert JMap into Map - // convert Calendar into Timestamp - def transform(obj: Any): Any = obj match { - case c: java.util.List[_] => c.map(transform).toSeq - case c: java.util.Map[_, _] => c.map { - case (key, value) => (key, transform(value)) - }.toMap - case c if c.getClass.isArray => - c.asInstanceOf[Array[_]].map(transform).toSeq - case c: java.util.Calendar => - new java.sql.Timestamp(c.getTime().getTime()) - case c => c + // Converts value to the type specified by the data type. + // Because Python does not have data types for TimestampType, FloatType, ShortType, and + // ByteType, we need to explicitly convert values in columns of these data types to the desired + // JVM data types. + def convert(obj: Any, dataType: DataType): Any = (obj, dataType) match { + // TODO: We should check nullable + case (null, _) => null + + case (c: java.util.List[_], ArrayType(elementType, _)) => + val converted = c.map { e => convert(e, elementType)} + JListWrapper(converted) + + case (c: java.util.Map[_, _], struct: StructType) => + val row = new GenericMutableRow(struct.fields.length) + struct.fields.zipWithIndex.foreach { + case (field, i) => + val value = convert(c.get(field.name), field.dataType) + row.update(i, value) + } + row + + case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => + val converted = c.map { + case (key, value) => + (convert(key, keyType), convert(value, valueType)) + } + JMapWrapper(converted) + + case (c, ArrayType(elementType, _)) if c.getClass.isArray => + val converted = c.asInstanceOf[Array[_]].map(e => convert(e, elementType)) + converted: Seq[Any] + + case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) + case (c: Int, ByteType) => c.toByte + case (c: Int, ShortType) => c.toShort + case (c: Double, FloatType) => c.toFloat + + case (c, _) => c + } + + val convertedRdd = if (schema.fields.exists(f => needsConversion(f.dataType))) { + rdd.map(m => m.map { case (key, value) => (key, convert(value, schema(key).dataType)) }) + } else { + rdd } - val need = firstRow.exists {case (key, value) => needTransform(value)} - val transformed = if (need) { - rdd.mapPartitions { iter => - iter.map { - m => m.map {case (key, value) => (key, transform(value))} + val rowRdd = convertedRdd.mapPartitions { iter => + val row = new GenericMutableRow(schema.fields.length) + val fieldsWithIndex = schema.fields.zipWithIndex + iter.map { m => + // We cannot use m.values because the order of values returned by m.values may not + // match fields order. + fieldsWithIndex.foreach { + case (field, i) => + val value = + m.get(field.name).flatMap(v => Option(v)).map(v => convert(v, field.dataType)).orNull + row.update(i, value) } - } - } else rdd - val rowRdd = transformed.mapPartitions { iter => - iter.map { map => - new GenericRow(map.values.toArray.asInstanceOf[Array[Any]]): Row + row: Row } } - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema, rowRdd))(self)) - } + new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRdd))(self)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 172b6e0e7f26b..420f21fb9c1ae 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.util.{Map => JMap, List => JList, Set => JSet} +import java.util.{Map => JMap, List => JList} import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.catalyst.types.{DataType, ArrayType, BooleanType, StructType, MapType} import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} import org.apache.spark.api.java.JavaRDD @@ -120,6 +119,11 @@ class SchemaRDD( override protected def getDependencies: Seq[Dependency[_]] = List(new OneToOneDependency(queryExecution.toRdd)) + /** Returns the schema of this SchemaRDD (represented by a [[StructType]]). + * + * @group schema + */ + def schema: StructType = queryExecution.analyzed.schema // ======================================================================= // Query DSL @@ -376,6 +380,8 @@ class SchemaRDD( * Converts a JavaRDD to a PythonRDD. It is used by pyspark. */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { + import scala.collection.Map + def toJava(obj: Any, dataType: DataType): Any = dataType match { case struct: StructType => rowToMap(obj.asInstanceOf[Row], struct) case array: ArrayType => obj match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index fd751031b26e5..6a20def475822 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -123,9 +123,15 @@ private[sql] trait SchemaRDDLike { def saveAsTable(tableName: String): Unit = sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd - /** Returns the output schema in the tree format. */ - def schemaString: String = queryExecution.analyzed.schemaString + /** Returns the schema as a string in the tree format. + * + * @group schema + */ + def schemaString: String = baseSchemaRDD.schema.treeString - /** Prints out the schema in the tree format. */ + /** Prints out the schema. + * + * @group schema + */ def printSchema(): Unit = println(schemaString) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 85726bae54911..c1c18a0cd0ed6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -21,14 +21,16 @@ import java.beans.Introspector import org.apache.hadoop.conf.Configuration -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.sql.api.java.types.{StructType => JStructType} import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.sql.types.util.DataTypeConversions +import DataTypeConversions.asScalaDataType; import org.apache.spark.util.Utils /** @@ -95,6 +97,21 @@ class JavaSQLContext(val sqlContext: SQLContext) { new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))(sqlContext)) } + /** + * :: DeveloperApi :: + * Creates a JavaSchemaRDD from an RDD containing Rows by applying a schema to this RDD. + * It is important to make sure that the structure of every Row of the provided RDD matches the + * provided schema. Otherwise, there will be runtime exception. + */ + @DeveloperApi + def applySchema(rowRDD: JavaRDD[Row], schema: JStructType): JavaSchemaRDD = { + val scalaRowRDD = rowRDD.rdd.map(r => r.row) + val scalaSchema = asScalaDataType(schema).asInstanceOf[StructType] + val logicalPlan = + SparkLogicalPlan(ExistingRdd(scalaSchema.toAttributes, scalaRowRDD))(sqlContext) + new JavaSchemaRDD(sqlContext, logicalPlan) + } + /** * Loads a parquet file, returning the result as a [[JavaSchemaRDD]]. */ @@ -104,23 +121,49 @@ class JavaSQLContext(val sqlContext: SQLContext) { ParquetRelation(path, Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext)) /** - * Loads a JSON file (one object per line), returning the result as a [[JavaSchemaRDD]]. + * Loads a JSON file (one object per line), returning the result as a JavaSchemaRDD. * It goes through the entire dataset once to determine the schema. - * - * @group userf */ def jsonFile(path: String): JavaSchemaRDD = jsonRDD(sqlContext.sparkContext.textFile(path)) + /** + * :: Experimental :: + * Loads a JSON file (one object per line) and applies the given schema, + * returning the result as a JavaSchemaRDD. + */ + @Experimental + def jsonFile(path: String, schema: JStructType): JavaSchemaRDD = + jsonRDD(sqlContext.sparkContext.textFile(path), schema) + /** * Loads an RDD[String] storing JSON objects (one object per record), returning the result as a - * [[JavaSchemaRDD]]. + * JavaSchemaRDD. * It goes through the entire dataset once to determine the schema. - * - * @group userf */ - def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = - new JavaSchemaRDD(sqlContext, JsonRDD.inferSchema(sqlContext, json, 1.0)) + def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = { + val appliedScalaSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0)) + val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) + val logicalPlan = + SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + new JavaSchemaRDD(sqlContext, logicalPlan) + } + + /** + * :: Experimental :: + * Loads an RDD[String] storing JSON objects (one object per record) and applies the given schema, + * returning the result as a JavaSchemaRDD. + */ + @Experimental + def jsonRDD(json: JavaRDD[String], schema: JStructType): JavaSchemaRDD = { + val appliedScalaSchema = + Option(asScalaDataType(schema)).getOrElse( + JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[StructType] + val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) + val logicalPlan = + SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + new JavaSchemaRDD(sqlContext, logicalPlan) + } /** * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only 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 8fbf13b8b0150..824574149858c 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 @@ -22,8 +22,11 @@ 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} +import org.apache.spark.sql.api.java.types.StructType +import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import DataTypeConversions._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -53,6 +56,10 @@ class JavaSchemaRDD( override def toString: String = baseSchemaRDD.toString + /** Returns the schema of this JavaSchemaRDD (represented by a StructType). */ + def schema: StructType = + asJavaDataType(baseSchemaRDD.schema).asInstanceOf[StructType] + // ======================================================================= // Base RDD functions that do NOT change schema // ======================================================================= diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 9b0dd2176149b..6c67934bda5b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -17,6 +17,11 @@ package org.apache.spark.sql.api.java +import scala.annotation.varargs +import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} +import scala.collection.JavaConversions +import scala.math.BigDecimal + import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** @@ -29,7 +34,7 @@ class Row(private[spark] val row: ScalaRow) extends Serializable { /** Returns the value of column `i`. */ def get(i: Int): Any = - row(i) + Row.toJavaValue(row(i)) /** Returns true if value at column `i` is NULL. */ def isNullAt(i: Int) = get(i) == null @@ -89,5 +94,57 @@ class Row(private[spark] val row: ScalaRow) extends Serializable { */ def getString(i: Int): String = row.getString(i) + + def canEqual(other: Any): Boolean = other.isInstanceOf[Row] + + override def equals(other: Any): Boolean = other match { + case that: Row => + (that canEqual this) && + row == that.row + case _ => false + } + + override def hashCode(): Int = row.hashCode() } +object Row { + + private def toJavaValue(value: Any): Any = value match { + // For values of this ScalaRow, we will do the conversion when + // they are actually accessed. + case row: ScalaRow => new Row(row) + case map: scala.collection.Map[_, _] => + JavaConversions.mapAsJavaMap( + map.map { + case (key, value) => (toJavaValue(key), toJavaValue(value)) + } + ) + case seq: scala.collection.Seq[_] => + JavaConversions.seqAsJavaList(seq.map(toJavaValue)) + case decimal: BigDecimal => decimal.underlying() + case other => other + } + + // TODO: Consolidate the toScalaValue at here with the scalafy in JsonRDD? + private def toScalaValue(value: Any): Any = value match { + // Values of this row have been converted to Scala values. + case row: Row => row.row + case map: java.util.Map[_, _] => + JMapWrapper(map).map { + case (key, value) => (toScalaValue(key), toScalaValue(value)) + } + case list: java.util.List[_] => + JListWrapper(list).map(toScalaValue) + case decimal: java.math.BigDecimal => BigDecimal(decimal) + case other => other + } + + /** + * Creates a Row with the given values. + */ + @varargs def create(values: Any*): Row = { + // Right now, we cannot use @varargs to annotate the constructor of + // org.apache.spark.sql.api.java.Row. See https://issues.scala-lang.org/browse/SI-8383. + new Row(ScalaRow(values.map(toScalaValue):_*)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 6c2b553bb908e..bd29ee421bbc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -25,33 +25,25 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} -import org.apache.spark.sql.{SQLContext, Logging} +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.Logging private[sql] object JsonRDD extends Logging { + private[sql] def jsonStringToRow( + json: RDD[String], + schema: StructType): RDD[Row] = { + parseJson(json).map(parsed => asRow(parsed, schema)) + } + private[sql] def inferSchema( - sqlContext: SQLContext, json: RDD[String], - samplingRatio: Double = 1.0): LogicalPlan = { + samplingRatio: Double = 1.0): StructType = { require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) val allKeys = parseJson(schemaData).map(allKeysWithValueTypes).reduce(_ ++ _) - val baseSchema = createSchema(allKeys) - - createLogicalPlan(json, baseSchema, sqlContext) - } - - private def createLogicalPlan( - json: RDD[String], - baseSchema: StructType, - sqlContext: SQLContext): LogicalPlan = { - val schema = nullTypeToStringType(baseSchema) - - SparkLogicalPlan( - ExistingRdd(asAttributes(schema), parseJson(json).map(asRow(_, schema))))(sqlContext) + createSchema(allKeys) } private def createSchema(allKeys: Set[(String, DataType)]): StructType = { @@ -75,8 +67,8 @@ private[sql] object JsonRDD extends Logging { val (topLevel, structLike) = values.partition(_.size == 1) val topLevelFields = topLevel.filter { name => resolved.get(prefix ++ name).get match { - case ArrayType(StructType(Nil)) => false - case ArrayType(_) => true + case ArrayType(StructType(Nil), _) => false + case ArrayType(_, _) => true case struct: StructType => false case _ => true } @@ -90,7 +82,8 @@ private[sql] object JsonRDD extends Logging { val structType = makeStruct(nestedFields, prefix :+ name) val dataType = resolved.get(prefix :+ name).get dataType match { - case array: ArrayType => Some(StructField(name, ArrayType(structType), nullable = true)) + case array: ArrayType => + Some(StructField(name, ArrayType(structType, array.containsNull), nullable = true)) case struct: StructType => Some(StructField(name, structType, nullable = true)) // dataType is StringType means that we have resolved type conflicts involving // primitive types and complex types. So, the type of name has been relaxed to @@ -109,6 +102,22 @@ private[sql] object JsonRDD extends Logging { makeStruct(resolved.keySet.toSeq, Nil) } + private[sql] def nullTypeToStringType(struct: StructType): StructType = { + val fields = struct.fields.map { + case StructField(fieldName, dataType, nullable) => { + val newType = dataType match { + case NullType => StringType + case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull) + case struct: StructType => nullTypeToStringType(struct) + case other: DataType => other + } + StructField(fieldName, newType, nullable) + } + } + + StructType(fields) + } + /** * Returns the most general data type for two given data types. */ @@ -139,8 +148,8 @@ private[sql] object JsonRDD extends Logging { case StructField(name, _, _) => name }) } - case (ArrayType(elementType1), ArrayType(elementType2)) => - ArrayType(compatibleType(elementType1, elementType2)) + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => + ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) // TODO: We should use JsonObjectStringType to mark that values of field will be // strings and every string is a Json object. case (_, _) => StringType @@ -148,18 +157,13 @@ private[sql] object JsonRDD extends Logging { } } - private def typeOfPrimitiveValue(value: Any): DataType = { - value match { - case value: java.lang.String => StringType - case value: java.lang.Integer => IntegerType - case value: java.lang.Long => LongType + private def typeOfPrimitiveValue: PartialFunction[Any, DataType] = { + ScalaReflection.typeOfObject orElse { // Since we do not have a data type backed by BigInteger, // when we see a Java BigInteger, we use DecimalType. case value: java.math.BigInteger => DecimalType - case value: java.lang.Double => DoubleType + // DecimalType's JVMType is scala BigDecimal. case value: java.math.BigDecimal => DecimalType - case value: java.lang.Boolean => BooleanType - case null => NullType // Unexpected data type. case _ => StringType } @@ -172,12 +176,13 @@ private[sql] object JsonRDD extends Logging { * treat the element as String. */ private def typeOfArray(l: Seq[Any]): ArrayType = { + val containsNull = l.exists(v => v == null) val elements = l.flatMap(v => Option(v)) if (elements.isEmpty) { // If this JSON array is empty, we use NullType as a placeholder. // If this array is not empty in other JSON objects, we can resolve // the type after we have passed through all JSON objects. - ArrayType(NullType) + ArrayType(NullType, containsNull) } else { val elementType = elements.map { e => e match { @@ -189,7 +194,7 @@ private[sql] object JsonRDD extends Logging { } }.reduce((type1: DataType, type2: DataType) => compatibleType(type1, type2)) - ArrayType(elementType) + ArrayType(elementType, containsNull) } } @@ -216,15 +221,16 @@ private[sql] object JsonRDD extends Logging { case (key: String, array: Seq[_]) => { // The value associated with the key is an array. typeOfArray(array) match { - case ArrayType(StructType(Nil)) => { + case ArrayType(StructType(Nil), containsNull) => { // The elements of this arrays are structs. array.asInstanceOf[Seq[Map[String, Any]]].flatMap { element => allKeysWithValueTypes(element) }.map { case (k, dataType) => (s"$key.$k", dataType) - } :+ (key, ArrayType(StructType(Nil))) + } :+ (key, ArrayType(StructType(Nil), containsNull)) } - case ArrayType(elementType) => (key, ArrayType(elementType)) :: Nil + case ArrayType(elementType, containsNull) => + (key, ArrayType(elementType, containsNull)) :: Nil } } case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil @@ -262,8 +268,11 @@ private[sql] object JsonRDD extends Logging { // the ObjectMapper will take the last value associated with this duplicate key. // For example: for {"key": 1, "key":2}, we will get "key"->2. val mapper = new ObjectMapper() - iter.map(record => mapper.readValue(record, classOf[java.util.Map[String, Any]])) - }).map(scalafy).map(_.asInstanceOf[Map[String, Any]]) + iter.map { record => + val parsed = scalafy(mapper.readValue(record, classOf[java.util.Map[String, Any]])) + parsed.asInstanceOf[Map[String, Any]] + } + }) } private def toLong(value: Any): Long = { @@ -334,7 +343,7 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case ArrayType(elementType) => + case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case StringType => toString(value) case IntegerType => value.asInstanceOf[IntegerType.JvmType] @@ -348,6 +357,7 @@ private[sql] object JsonRDD extends Logging { } private def asRow(json: Map[String,Any], schema: StructType): Row = { + // TODO: Reuse the row instead of creating a new one for every record. val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { // StructType @@ -356,7 +366,7 @@ private[sql] object JsonRDD extends Logging { v => asRow(v.asInstanceOf[Map[String, Any]], fields)).orNull) // ArrayType(StructType) - case (StructField(name, ArrayType(structType: StructType), _), i) => + case (StructField(name, ArrayType(structType: StructType, _), _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( v => v.asInstanceOf[Seq[Any]].map( @@ -370,32 +380,4 @@ private[sql] object JsonRDD extends Logging { row } - - private def nullTypeToStringType(struct: StructType): StructType = { - val fields = struct.fields.map { - case StructField(fieldName, dataType, nullable) => { - val newType = dataType match { - case NullType => StringType - case ArrayType(NullType) => ArrayType(StringType) - case struct: StructType => nullTypeToStringType(struct) - case other: DataType => other - } - StructField(fieldName, newType, nullable) - } - } - - StructType(fields) - } - - private def asAttributes(struct: StructType): Seq[AttributeReference] = { - struct.fields.map(f => AttributeReference(f.name, f.dataType, nullable = true)()) - } - - private def asStruct(attributes: Seq[AttributeReference]): StructType = { - val fields = attributes.map { - case AttributeReference(name, dataType, nullable) => StructField(name, dataType, nullable) - } - - StructType(fields) - } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java b/sql/core/src/main/scala/org/apache/spark/sql/package-info.java similarity index 100% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/package-info.java rename to sql/core/src/main/scala/org/apache/spark/sql/package-info.java diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala new file mode 100644 index 0000000000000..0995a4eb6299f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -0,0 +1,409 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.apache.spark.annotation.DeveloperApi + +/** + * Allows the execution of relational queries, including those expressed in SQL using Spark. + * + * @groupname dataType Data types + * @groupdesc Spark SQL data types. + * @groupprio dataType -3 + * @groupname field Field + * @groupprio field -2 + * @groupname row Row + * @groupprio row -1 + */ +package object sql { + + protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging + + /** + * :: DeveloperApi :: + * + * Represents one row of output from a relational operator. + * @group row + */ + @DeveloperApi + type Row = catalyst.expressions.Row + + /** + * :: DeveloperApi :: + * + * A [[Row]] object can be constructed by providing field values. Example: + * {{{ + * import org.apache.spark.sql._ + * + * // Create a Row from values. + * Row(value1, value2, value3, ...) + * // Create a Row from a Seq of values. + * Row.fromSeq(Seq(value1, value2, ...)) + * }}} + * + * A value of a row can be accessed through both generic access by ordinal, + * which will incur boxing overhead for primitives, as well as native primitive access. + * An example of generic access by ordinal: + * {{{ + * import org.apache.spark.sql._ + * + * val row = Row(1, true, "a string", null) + * // row: Row = [1,true,a string,null] + * val firstValue = row(0) + * // firstValue: Any = 1 + * val fourthValue = row(3) + * // fourthValue: Any = null + * }}} + * + * For native primitive access, it is invalid to use the native primitive interface to retrieve + * a value that is null, instead a user must check `isNullAt` before attempting to retrieve a + * value that might be null. + * An example of native primitive access: + * {{{ + * // using the row from the previous example. + * val firstValue = row.getInt(0) + * // firstValue: Int = 1 + * val isNull = row.isNullAt(3) + * // isNull: Boolean = true + * }}} + * + * Interfaces related to native primitive access are: + * + * `isNullAt(i: Int): Boolean` + * + * `getInt(i: Int): Int` + * + * `getLong(i: Int): Long` + * + * `getDouble(i: Int): Double` + * + * `getFloat(i: Int): Float` + * + * `getBoolean(i: Int): Boolean` + * + * `getShort(i: Int): Short` + * + * `getByte(i: Int): Byte` + * + * `getString(i: Int): String` + * + * Fields in a [[Row]] object can be extracted in a pattern match. Example: + * {{{ + * import org.apache.spark.sql._ + * + * val pairs = sql("SELECT key, value FROM src").rdd.map { + * case Row(key: Int, value: String) => + * key -> value + * } + * }}} + * + * @group row + */ + @DeveloperApi + val Row = catalyst.expressions.Row + + /** + * :: DeveloperApi :: + * + * The base type of all Spark SQL data types. + * + * @group dataType + */ + @DeveloperApi + type DataType = catalyst.types.DataType + + /** + * :: DeveloperApi :: + * + * The data type representing `String` values + * + * @group dataType + */ + @DeveloperApi + val StringType = catalyst.types.StringType + + /** + * :: DeveloperApi :: + * + * The data type representing `Array[Byte]` values. + * + * @group dataType + */ + @DeveloperApi + val BinaryType = catalyst.types.BinaryType + + /** + * :: DeveloperApi :: + * + * The data type representing `Boolean` values. + * + *@group dataType + */ + @DeveloperApi + val BooleanType = catalyst.types.BooleanType + + /** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Timestamp` values. + * + * @group dataType + */ + @DeveloperApi + val TimestampType = catalyst.types.TimestampType + + /** + * :: DeveloperApi :: + * + * The data type representing `scala.math.BigDecimal` values. + * + * @group dataType + */ + @DeveloperApi + val DecimalType = catalyst.types.DecimalType + + /** + * :: DeveloperApi :: + * + * The data type representing `Double` values. + * + * @group dataType + */ + @DeveloperApi + val DoubleType = catalyst.types.DoubleType + + /** + * :: DeveloperApi :: + * + * The data type representing `Float` values. + * + * @group dataType + */ + @DeveloperApi + val FloatType = catalyst.types.FloatType + + /** + * :: DeveloperApi :: + * + * The data type representing `Byte` values. + * + * @group dataType + */ + @DeveloperApi + val ByteType = catalyst.types.ByteType + + /** + * :: DeveloperApi :: + * + * The data type representing `Int` values. + * + * @group dataType + */ + @DeveloperApi + val IntegerType = catalyst.types.IntegerType + + /** + * :: DeveloperApi :: + * + * The data type representing `Long` values. + * + * @group dataType + */ + @DeveloperApi + val LongType = catalyst.types.LongType + + /** + * :: DeveloperApi :: + * + * The data type representing `Short` values. + * + * @group dataType + */ + @DeveloperApi + val ShortType = catalyst.types.ShortType + + /** + * :: DeveloperApi :: + * + * The data type for collections of multiple values. + * Internally these are represented as columns that contain a ``scala.collection.Seq``. + * + * An [[ArrayType]] object comprises two fields, `elementType: [[DataType]]` and + * `containsNull: Boolean`. The field of `elementType` is used to specify the type of + * array elements. The field of `containsNull` is used to specify if the array has `null` values. + * + * @group dataType + */ + @DeveloperApi + type ArrayType = catalyst.types.ArrayType + + /** + * :: DeveloperApi :: + * + * An [[ArrayType]] object can be constructed with two ways, + * {{{ + * ArrayType(elementType: DataType, containsNull: Boolean) + * }}} and + * {{{ + * ArrayType(elementType: DataType) + * }}} + * For `ArrayType(elementType)`, the field of `containsNull` is set to `false`. + * + * @group dataType + */ + @DeveloperApi + val ArrayType = catalyst.types.ArrayType + + /** + * :: DeveloperApi :: + * + * The data type representing `Map`s. A [[MapType]] object comprises three fields, + * `keyType: [[DataType]]`, `valueType: [[DataType]]` and `valueContainsNull: Boolean`. + * The field of `keyType` is used to specify the type of keys in the map. + * The field of `valueType` is used to specify the type of values in the map. + * The field of `valueContainsNull` is used to specify if values of this map has `null` values. + * For values of a MapType column, keys are not allowed to have `null` values. + * + * @group dataType + */ + @DeveloperApi + type MapType = catalyst.types.MapType + + /** + * :: DeveloperApi :: + * + * A [[MapType]] object can be constructed with two ways, + * {{{ + * MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) + * }}} and + * {{{ + * MapType(keyType: DataType, valueType: DataType) + * }}} + * For `MapType(keyType: DataType, valueType: DataType)`, + * the field of `valueContainsNull` is set to `true`. + * + * @group dataType + */ + @DeveloperApi + val MapType = catalyst.types.MapType + + /** + * :: DeveloperApi :: + * + * The data type representing [[Row]]s. + * A [[StructType]] object comprises a [[Seq]] of [[StructField]]s. + * + * @group dataType + */ + @DeveloperApi + type StructType = catalyst.types.StructType + + /** + * :: DeveloperApi :: + * + * A [[StructType]] object can be constructed by + * {{{ + * StructType(fields: Seq[StructField]) + * }}} + * For a [[StructType]] object, one or multiple [[StructField]]s can be extracted by names. + * If multiple [[StructField]]s are extracted, a [[StructType]] object will be returned. + * If a provided name does not have a matching field, it will be ignored. For the case + * of extracting a single StructField, a `null` will be returned. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val struct = + * StructType( + * StructField("a", IntegerType, true) :: + * StructField("b", LongType, false) :: + * StructField("c", BooleanType, false) :: Nil) + * + * // Extract a single StructField. + * val singleField = struct("b") + * // singleField: StructField = StructField(b,LongType,false) + * + * // This struct does not have a field called "d". null will be returned. + * val nonExisting = struct("d") + * // nonExisting: StructField = null + * + * // Extract multiple StructFields. Field names are provided in a set. + * // A StructType object will be returned. + * val twoFields = struct(Set("b", "c")) + * // twoFields: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * + * // Those names do not have matching fields will be ignored. + * // For the case shown below, "d" will be ignored and + * // it is treated as struct(Set("b", "c")). + * val ignoreNonExisting = struct(Set("b", "c", "d")) + * // ignoreNonExisting: StructType = + * // StructType(List(StructField(b,LongType,false), StructField(c,BooleanType,false))) + * }}} + * + * A [[Row]] object is used as a value of the StructType. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val innerStruct = + * StructType( + * StructField("f1", IntegerType, true) :: + * StructField("f2", LongType, false) :: + * StructField("f3", BooleanType, false) :: Nil) + * + * val struct = StructType( + * StructField("a", innerStruct, true) :: Nil) + * + * // Create a Row with the schema defined by struct + * val row = Row(Row(1, 2, true)) + * // row: Row = [[1,2,true]] + * }}} + * + * @group dataType + */ + @DeveloperApi + val StructType = catalyst.types.StructType + + /** + * :: DeveloperApi :: + * + * A [[StructField]] object represents a field in a [[StructType]] object. + * A [[StructField]] object comprises three fields, `name: [[String]]`, `dataType: [[DataType]]`, + * and `nullable: Boolean`. The field of `name` is the name of a `StructField`. The field of + * `dataType` specifies the data type of a `StructField`. + * The field of `nullable` specifies if values of a `StructField` can contain `null` values. + * + * @group field + */ + @DeveloperApi + type StructField = catalyst.types.StructField + + /** + * :: DeveloperApi :: + * + * A [[StructField]] object can be constructed by + * {{{ + * StructField(name: String, dataType: DataType, nullable: Boolean) + * }}} + * + * @group dataType + */ + @DeveloperApi + val StructField = catalyst.types.StructField +} 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 de8fe2dae38f6..0a3b59cbc233a 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 @@ -75,21 +75,21 @@ private[sql] object CatalystConverter { val fieldType: DataType = field.dataType fieldType match { // For native JVM types we use a converter with native arrays - case ArrayType(elementType: NativeType) => { + case ArrayType(elementType: NativeType, false) => { new CatalystNativeArrayConverter(elementType, fieldIndex, parent) } // This is for other types of arrays, including those with nested fields - case ArrayType(elementType: DataType) => { + case ArrayType(elementType: DataType, false) => { new CatalystArrayConverter(elementType, fieldIndex, parent) } case StructType(fields: Seq[StructField]) => { new CatalystStructConverter(fields.toArray, fieldIndex, parent) } - case MapType(keyType: DataType, valueType: DataType) => { + case MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) => { new CatalystMapConverter( Array( new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false), - new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, true)), + new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, valueContainsNull)), fieldIndex, parent) } 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 39294a3f4bf5a..6d4ce32ac5bfa 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 @@ -172,10 +172,10 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeValue(schema: DataType, value: Any): Unit = { if (value != null) { schema match { - case t @ ArrayType(_) => writeArray( + case t @ ArrayType(_, false) => writeArray( t, value.asInstanceOf[CatalystConverter.ArrayScalaType[_]]) - case t @ MapType(_, _) => writeMap( + case t @ MapType(_, _, _) => writeMap( t, value.asInstanceOf[CatalystConverter.MapScalaType[_, _]]) case t @ StructType(_) => writeStruct( 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 58370b955a5ec..aaef1a1d474fe 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 @@ -116,7 +116,7 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetOriginalType.LIST => { // TODO: check enums! assert(groupType.getFieldCount == 1) val field = groupType.getFields.apply(0) - new ArrayType(toDataType(field)) + ArrayType(toDataType(field), containsNull = false) } case ParquetOriginalType.MAP => { assert( @@ -130,7 +130,9 @@ private[parquet] object ParquetTypesConverter extends Logging { assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) val valueType = toDataType(keyValueGroup.getFields.apply(1)) assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - new MapType(keyType, valueType) + // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true + // at here. + MapType(keyType, valueType) } case _ => { // Note: the order of these checks is important! @@ -140,10 +142,12 @@ private[parquet] object ParquetTypesConverter extends Logging { assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) val valueType = toDataType(keyValueGroup.getFields.apply(1)) assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - new MapType(keyType, valueType) + // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true + // at here. + MapType(keyType, valueType) } else if (correspondsToArray(groupType)) { // ArrayType val elementType = toDataType(groupType.getFields.apply(0)) - new ArrayType(elementType) + ArrayType(elementType, containsNull = false) } else { // everything else: StructType val fields = groupType .getFields @@ -151,7 +155,7 @@ private[parquet] object ParquetTypesConverter extends Logging { ptype.getName, toDataType(ptype), ptype.getRepetition != Repetition.REQUIRED)) - new StructType(fields) + StructType(fields) } } } @@ -234,7 +238,7 @@ private[parquet] object ParquetTypesConverter extends Logging { new ParquetPrimitiveType(repetition, primitiveType, name, originalType.orNull) }.getOrElse { ctype match { - case ArrayType(elementType) => { + case ArrayType(elementType, false) => { val parquetElementType = fromDataType( elementType, CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, @@ -248,7 +252,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } new ParquetGroupType(repetition, name, fields) } - case MapType(keyType, valueType) => { + case MapType(keyType, valueType, _) => { val parquetKeyType = fromDataType( keyType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala new file mode 100644 index 0000000000000..d1aa3c8d53757 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.types.util + +import org.apache.spark.sql._ +import org.apache.spark.sql.api.java.types.{DataType => JDataType, StructField => JStructField} + +import scala.collection.JavaConverters._ + +protected[sql] object DataTypeConversions { + + /** + * Returns the equivalent StructField in Scala for the given StructField in Java. + */ + def asJavaStructField(scalaStructField: StructField): JStructField = { + JDataType.createStructField( + scalaStructField.name, + asJavaDataType(scalaStructField.dataType), + scalaStructField.nullable) + } + + /** + * Returns the equivalent DataType in Java for the given DataType in Scala. + */ + def asJavaDataType(scalaDataType: DataType): JDataType = scalaDataType match { + case StringType => JDataType.StringType + case BinaryType => JDataType.BinaryType + case BooleanType => JDataType.BooleanType + case TimestampType => JDataType.TimestampType + case DecimalType => JDataType.DecimalType + case DoubleType => JDataType.DoubleType + case FloatType => JDataType.FloatType + case ByteType => JDataType.ByteType + case IntegerType => JDataType.IntegerType + case LongType => JDataType.LongType + case ShortType => JDataType.ShortType + + case arrayType: ArrayType => JDataType.createArrayType( + asJavaDataType(arrayType.elementType), arrayType.containsNull) + case mapType: MapType => JDataType.createMapType( + asJavaDataType(mapType.keyType), + asJavaDataType(mapType.valueType), + mapType.valueContainsNull) + case structType: StructType => JDataType.createStructType( + structType.fields.map(asJavaStructField).asJava) + } + + /** + * Returns the equivalent StructField in Scala for the given StructField in Java. + */ + def asScalaStructField(javaStructField: JStructField): StructField = { + StructField( + javaStructField.getName, + asScalaDataType(javaStructField.getDataType), + javaStructField.isNullable) + } + + /** + * Returns the equivalent DataType in Scala for the given DataType in Java. + */ + def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match { + case stringType: org.apache.spark.sql.api.java.types.StringType => + StringType + case binaryType: org.apache.spark.sql.api.java.types.BinaryType => + BinaryType + case booleanType: org.apache.spark.sql.api.java.types.BooleanType => + BooleanType + case timestampType: org.apache.spark.sql.api.java.types.TimestampType => + TimestampType + case decimalType: org.apache.spark.sql.api.java.types.DecimalType => + DecimalType + case doubleType: org.apache.spark.sql.api.java.types.DoubleType => + DoubleType + case floatType: org.apache.spark.sql.api.java.types.FloatType => + FloatType + case byteType: org.apache.spark.sql.api.java.types.ByteType => + ByteType + case integerType: org.apache.spark.sql.api.java.types.IntegerType => + IntegerType + case longType: org.apache.spark.sql.api.java.types.LongType => + LongType + case shortType: org.apache.spark.sql.api.java.types.ShortType => + ShortType + + case arrayType: org.apache.spark.sql.api.java.types.ArrayType => + ArrayType(asScalaDataType(arrayType.getElementType), arrayType.isContainsNull) + case mapType: org.apache.spark.sql.api.java.types.MapType => + MapType( + asScalaDataType(mapType.getKeyType), + asScalaDataType(mapType.getValueType), + mapType.isValueContainsNull) + case structType: org.apache.spark.sql.api.java.types.StructType => + StructType(structType.getFields.map(asScalaStructField)) + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java new file mode 100644 index 0000000000000..8ee4591105010 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.sql.api.java.types.DataType; +import org.apache.spark.sql.api.java.types.StructField; +import org.apache.spark.sql.api.java.types.StructType; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaApplySchemaSuite implements Serializable { + private transient JavaSparkContext javaCtx; + private transient JavaSQLContext javaSqlCtx; + + @Before + public void setUp() { + javaCtx = new JavaSparkContext("local", "JavaApplySchemaSuite"); + javaSqlCtx = new JavaSQLContext(javaCtx); + } + + @After + public void tearDown() { + javaCtx.stop(); + javaCtx = null; + javaSqlCtx = null; + } + + public static class Person implements Serializable { + private String name; + private int age; + + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public int getAge() { + return age; + } + + public void setAge(int age) { + this.age = age; + } + } + + @Test + public void applySchema() { + List personList = new ArrayList(2); + Person person1 = new Person(); + person1.setName("Michael"); + person1.setAge(29); + personList.add(person1); + Person person2 = new Person(); + person2.setName("Yin"); + person2.setAge(28); + personList.add(person2); + + JavaRDD rowRDD = javaCtx.parallelize(personList).map( + new Function() { + public Row call(Person person) throws Exception { + return Row.create(person.getName(), person.getAge()); + } + }); + + List fields = new ArrayList(2); + fields.add(DataType.createStructField("name", DataType.StringType, false)); + fields.add(DataType.createStructField("age", DataType.IntegerType, false)); + StructType schema = DataType.createStructType(fields); + + JavaSchemaRDD schemaRDD = javaSqlCtx.applySchema(rowRDD, schema); + schemaRDD.registerAsTable("people"); + List actual = javaSqlCtx.sql("SELECT * FROM people").collect(); + + List expected = new ArrayList(2); + expected.add(Row.create("Michael", 29)); + expected.add(Row.create("Yin", 28)); + + Assert.assertEquals(expected, actual); + } + + @Test + public void applySchemaToJSON() { + JavaRDD jsonRDD = javaCtx.parallelize(Arrays.asList( + "{\"string\":\"this is a simple string.\", \"integer\":10, \"long\":21474836470, " + + "\"bigInteger\":92233720368547758070, \"double\":1.7976931348623157E308, " + + "\"boolean\":true, \"null\":null}", + "{\"string\":\"this is another simple string.\", \"integer\":11, \"long\":21474836469, " + + "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " + + "\"boolean\":false, \"null\":null}")); + List fields = new ArrayList(7); + fields.add(DataType.createStructField("bigInteger", DataType.DecimalType, true)); + fields.add(DataType.createStructField("boolean", DataType.BooleanType, true)); + fields.add(DataType.createStructField("double", DataType.DoubleType, true)); + fields.add(DataType.createStructField("integer", DataType.IntegerType, true)); + fields.add(DataType.createStructField("long", DataType.LongType, true)); + fields.add(DataType.createStructField("null", DataType.StringType, true)); + fields.add(DataType.createStructField("string", DataType.StringType, true)); + StructType expectedSchema = DataType.createStructType(fields); + List expectedResult = new ArrayList(2); + expectedResult.add( + Row.create( + new BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.")); + expectedResult.add( + Row.create( + new BigDecimal("92233720368547758069"), + false, + 1.7976931348623157E305, + 11, + 21474836469L, + null, + "this is another simple string.")); + + JavaSchemaRDD schemaRDD1 = javaSqlCtx.jsonRDD(jsonRDD); + StructType actualSchema1 = schemaRDD1.schema(); + Assert.assertEquals(expectedSchema, actualSchema1); + schemaRDD1.registerAsTable("jsonTable1"); + List actual1 = javaSqlCtx.sql("select * from jsonTable1").collect(); + Assert.assertEquals(expectedResult, actual1); + + JavaSchemaRDD schemaRDD2 = javaSqlCtx.jsonRDD(jsonRDD, expectedSchema); + StructType actualSchema2 = schemaRDD2.schema(); + Assert.assertEquals(expectedSchema, actualSchema2); + schemaRDD1.registerAsTable("jsonTable2"); + List actual2 = javaSqlCtx.sql("select * from jsonTable2").collect(); + Assert.assertEquals(expectedResult, actual2); + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java new file mode 100644 index 0000000000000..52d07b5425cc3 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.math.BigDecimal; +import java.sql.Timestamp; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class JavaRowSuite { + private byte byteValue; + private short shortValue; + private int intValue; + private long longValue; + private float floatValue; + private double doubleValue; + private BigDecimal decimalValue; + private boolean booleanValue; + private String stringValue; + private byte[] binaryValue; + private Timestamp timestampValue; + + @Before + public void setUp() { + byteValue = (byte)127; + shortValue = (short)32767; + intValue = 2147483647; + longValue = 9223372036854775807L; + floatValue = (float)3.4028235E38; + doubleValue = 1.7976931348623157E308; + decimalValue = new BigDecimal("1.7976931348623157E328"); + booleanValue = true; + stringValue = "this is a string"; + binaryValue = stringValue.getBytes(); + timestampValue = Timestamp.valueOf("2014-06-30 09:20:00.0"); + } + + @Test + public void constructSimpleRow() { + Row simpleRow = Row.create( + byteValue, // ByteType + new Byte(byteValue), + shortValue, // ShortType + new Short(shortValue), + intValue, // IntegerType + new Integer(intValue), + longValue, // LongType + new Long(longValue), + floatValue, // FloatType + new Float(floatValue), + doubleValue, // DoubleType + new Double(doubleValue), + decimalValue, // DecimalType + booleanValue, // BooleanType + new Boolean(booleanValue), + stringValue, // StringType + binaryValue, // BinaryType + timestampValue, // TimestampType + null // null + ); + + Assert.assertEquals(byteValue, simpleRow.getByte(0)); + Assert.assertEquals(byteValue, simpleRow.get(0)); + Assert.assertEquals(byteValue, simpleRow.getByte(1)); + Assert.assertEquals(byteValue, simpleRow.get(1)); + Assert.assertEquals(shortValue, simpleRow.getShort(2)); + Assert.assertEquals(shortValue, simpleRow.get(2)); + Assert.assertEquals(shortValue, simpleRow.getShort(3)); + Assert.assertEquals(shortValue, simpleRow.get(3)); + Assert.assertEquals(intValue, simpleRow.getInt(4)); + Assert.assertEquals(intValue, simpleRow.get(4)); + Assert.assertEquals(intValue, simpleRow.getInt(5)); + Assert.assertEquals(intValue, simpleRow.get(5)); + Assert.assertEquals(longValue, simpleRow.getLong(6)); + Assert.assertEquals(longValue, simpleRow.get(6)); + Assert.assertEquals(longValue, simpleRow.getLong(7)); + Assert.assertEquals(longValue, simpleRow.get(7)); + // When we create the row, we do not do any conversion + // for a float/double value, so we just set the delta to 0. + Assert.assertEquals(floatValue, simpleRow.getFloat(8), 0); + Assert.assertEquals(floatValue, simpleRow.get(8)); + Assert.assertEquals(floatValue, simpleRow.getFloat(9), 0); + Assert.assertEquals(floatValue, simpleRow.get(9)); + Assert.assertEquals(doubleValue, simpleRow.getDouble(10), 0); + Assert.assertEquals(doubleValue, simpleRow.get(10)); + Assert.assertEquals(doubleValue, simpleRow.getDouble(11), 0); + Assert.assertEquals(doubleValue, simpleRow.get(11)); + Assert.assertEquals(decimalValue, simpleRow.get(12)); + Assert.assertEquals(booleanValue, simpleRow.getBoolean(13)); + Assert.assertEquals(booleanValue, simpleRow.get(13)); + Assert.assertEquals(booleanValue, simpleRow.getBoolean(14)); + Assert.assertEquals(booleanValue, simpleRow.get(14)); + Assert.assertEquals(stringValue, simpleRow.getString(15)); + Assert.assertEquals(stringValue, simpleRow.get(15)); + Assert.assertEquals(binaryValue, simpleRow.get(16)); + Assert.assertEquals(timestampValue, simpleRow.get(17)); + Assert.assertEquals(true, simpleRow.isNullAt(18)); + Assert.assertEquals(null, simpleRow.get(18)); + } + + @Test + public void constructComplexRow() { + // Simple array + List simpleStringArray = Arrays.asList( + stringValue + " (1)", stringValue + " (2)", stringValue + "(3)"); + + // Simple map + Map simpleMap = new HashMap(); + simpleMap.put(stringValue + " (1)", longValue); + simpleMap.put(stringValue + " (2)", longValue - 1); + simpleMap.put(stringValue + " (3)", longValue - 2); + + // Simple struct + Row simpleStruct = Row.create( + doubleValue, stringValue, timestampValue, null); + + // Complex array + List> arrayOfMaps = Arrays.asList(simpleMap); + List arrayOfRows = Arrays.asList(simpleStruct); + + // Complex map + Map, Row> complexMap = new HashMap, Row>(); + complexMap.put(arrayOfRows, simpleStruct); + + // Complex struct + Row complexStruct = Row.create( + simpleStringArray, + simpleMap, + simpleStruct, + arrayOfMaps, + arrayOfRows, + complexMap, + null); + Assert.assertEquals(simpleStringArray, complexStruct.get(0)); + Assert.assertEquals(simpleMap, complexStruct.get(1)); + Assert.assertEquals(simpleStruct, complexStruct.get(2)); + Assert.assertEquals(arrayOfMaps, complexStruct.get(3)); + Assert.assertEquals(arrayOfRows, complexStruct.get(4)); + Assert.assertEquals(complexMap, complexStruct.get(5)); + Assert.assertEquals(null, complexStruct.get(6)); + + // A very complex row + Row complexRow = Row.create(arrayOfMaps, arrayOfRows, complexMap, complexStruct); + Assert.assertEquals(arrayOfMaps, complexRow.get(0)); + Assert.assertEquals(arrayOfRows, complexRow.get(1)); + Assert.assertEquals(complexMap, complexRow.get(2)); + Assert.assertEquals(complexStruct, complexRow.get(3)); + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java new file mode 100644 index 0000000000000..96a503962f7d1 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.util.List; +import java.util.ArrayList; + +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.sql.types.util.DataTypeConversions; +import org.apache.spark.sql.api.java.types.DataType; +import org.apache.spark.sql.api.java.types.StructField; + +public class JavaSideDataTypeConversionSuite { + public void checkDataType(DataType javaDataType) { + org.apache.spark.sql.catalyst.types.DataType scalaDataType = + DataTypeConversions.asScalaDataType(javaDataType); + DataType actual = DataTypeConversions.asJavaDataType(scalaDataType); + Assert.assertEquals(javaDataType, actual); + } + + @Test + public void createDataTypes() { + // Simple DataTypes. + checkDataType(DataType.StringType); + checkDataType(DataType.BinaryType); + checkDataType(DataType.BooleanType); + checkDataType(DataType.TimestampType); + checkDataType(DataType.DecimalType); + checkDataType(DataType.DoubleType); + checkDataType(DataType.FloatType); + checkDataType(DataType.ByteType); + checkDataType(DataType.IntegerType); + checkDataType(DataType.LongType); + checkDataType(DataType.ShortType); + + // Simple ArrayType. + DataType simpleJavaArrayType = DataType.createArrayType(DataType.StringType, true); + checkDataType(simpleJavaArrayType); + + // Simple MapType. + DataType simpleJavaMapType = DataType.createMapType(DataType.StringType, DataType.LongType); + checkDataType(simpleJavaMapType); + + // Simple StructType. + List simpleFields = new ArrayList(); + simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); + simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); + simpleFields.add(DataType.createStructField("d", DataType.BinaryType, false)); + DataType simpleJavaStructType = DataType.createStructType(simpleFields); + checkDataType(simpleJavaStructType); + + // Complex StructType. + List complexFields = new ArrayList(); + complexFields.add(DataType.createStructField("simpleArray", simpleJavaArrayType, true)); + complexFields.add(DataType.createStructField("simpleMap", simpleJavaMapType, true)); + complexFields.add(DataType.createStructField("simpleStruct", simpleJavaStructType, true)); + complexFields.add(DataType.createStructField("boolean", DataType.BooleanType, false)); + DataType complexJavaStructType = DataType.createStructType(complexFields); + checkDataType(complexJavaStructType); + + // Complex ArrayType. + DataType complexJavaArrayType = DataType.createArrayType(complexJavaStructType, true); + checkDataType(complexJavaArrayType); + + // Complex MapType. + DataType complexJavaMapType = + DataType.createMapType(complexJavaStructType, complexJavaArrayType, false); + checkDataType(complexJavaMapType); + } + + @Test + public void illegalArgument() { + // ArrayType + try { + DataType.createArrayType(null, true); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + + // MapType + try { + DataType.createMapType(null, DataType.StringType); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createMapType(DataType.StringType, null); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createMapType(null, null); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + + // StructField + try { + DataType.createStructField(null, DataType.StringType, true); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createStructField("name", null, true); + } catch (IllegalArgumentException expectedException) { + } + try { + DataType.createStructField(null, null, true); + } catch (IllegalArgumentException expectedException) { + } + + // StructType + try { + List simpleFields = new ArrayList(); + simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); + simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); + simpleFields.add(null); + DataType.createStructType(simpleFields); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + try { + List simpleFields = new ArrayList(); + simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("a", DataType.BooleanType, true)); + simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); + DataType.createStructType(simpleFields); + Assert.fail(); + } catch (IllegalArgumentException expectedException) { + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala new file mode 100644 index 0000000000000..cf7d79f42db1d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -0,0 +1,58 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import org.scalatest.FunSuite + +class DataTypeSuite extends FunSuite { + + test("construct an ArrayType") { + val array = ArrayType(StringType) + + assert(ArrayType(StringType, false) === array) + } + + test("construct an MapType") { + val map = MapType(StringType, IntegerType) + + assert(MapType(StringType, IntegerType, true) === map) + } + + test("extract fields from a StructType") { + val struct = StructType( + StructField("a", IntegerType, true) :: + StructField("b", LongType, false) :: + StructField("c", StringType, true) :: + StructField("d", FloatType, true) :: Nil) + + assert(StructField("b", LongType, false) === struct("b")) + + intercept[IllegalArgumentException] { + struct("e") + } + + val expectedStruct = StructType( + StructField("b", LongType, false) :: + StructField("d", FloatType, true) :: Nil) + + assert(expectedStruct === struct(Set("b", "d"))) + intercept[IllegalArgumentException] { + struct(Set("b", "d", "e", "f")) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala new file mode 100644 index 0000000000000..651cb735ab7d9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -0,0 +1,46 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + +class RowSuite extends FunSuite { + + test("create row") { + val expected = new GenericMutableRow(4) + expected.update(0, 2147483647) + expected.update(1, "this is a string") + expected.update(2, false) + expected.update(3, null) + val actual1 = Row(2147483647, "this is a string", false, null) + assert(expected.size === actual1.size) + assert(expected.getInt(0) === actual1.getInt(0)) + assert(expected.getString(1) === actual1.getString(1)) + assert(expected.getBoolean(2) === actual1.getBoolean(2)) + assert(expected(3) === actual1(3)) + + val actual2 = Row.fromSeq(Seq(2147483647, "this is a string", false, null)) + assert(expected.size === actual2.size) + assert(expected.getInt(0) === actual2.getInt(0)) + assert(expected.getString(1) === actual2.getString(1)) + assert(expected.getBoolean(2) === actual2.getBoolean(2)) + assert(expected(3) === actual2(3)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index de9e8aa4f62ed..bebb490645420 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,9 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.test._ /* Implicits */ @@ -446,4 +444,66 @@ class SQLQuerySuite extends QueryTest { ) clear() } + + test("apply schema") { + val schema1 = StructType( + StructField("f1", IntegerType, false) :: + StructField("f2", StringType, false) :: + StructField("f3", BooleanType, false) :: + StructField("f4", IntegerType, true) :: Nil) + + val rowRDD1 = unparsedStrings.map { r => + val values = r.split(",").map(_.trim) + val v4 = try values(3).toInt catch { + case _: NumberFormatException => null + } + Row(values(0).toInt, values(1), values(2).toBoolean, v4) + } + + val schemaRDD1 = applySchema(rowRDD1, schema1) + schemaRDD1.registerAsTable("applySchema1") + checkAnswer( + sql("SELECT * FROM applySchema1"), + (1, "A1", true, null) :: + (2, "B2", false, null) :: + (3, "C3", true, null) :: + (4, "D4", true, 2147483644) :: Nil) + + checkAnswer( + sql("SELECT f1, f4 FROM applySchema1"), + (1, null) :: + (2, null) :: + (3, null) :: + (4, 2147483644) :: Nil) + + val schema2 = StructType( + StructField("f1", StructType( + StructField("f11", IntegerType, false) :: + StructField("f12", BooleanType, false) :: Nil), false) :: + StructField("f2", MapType(StringType, IntegerType, true), false) :: Nil) + + val rowRDD2 = unparsedStrings.map { r => + val values = r.split(",").map(_.trim) + val v4 = try values(3).toInt catch { + case _: NumberFormatException => null + } + Row(Row(values(0).toInt, values(2).toBoolean), Map(values(1) -> v4)) + } + + val schemaRDD2 = applySchema(rowRDD2, schema2) + schemaRDD2.registerAsTable("applySchema2") + checkAnswer( + sql("SELECT * FROM applySchema2"), + (Seq(1, true), Map("A1" -> null)) :: + (Seq(2, false), Map("B2" -> null)) :: + (Seq(3, true), Map("C3" -> null)) :: + (Seq(4, true), Map("D4" -> 2147483644)) :: Nil) + + checkAnswer( + sql("SELECT f1.f11, f2['D4'] FROM applySchema2"), + (1, null) :: + (2, null) :: + (3, null) :: + (4, 2147483644) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 330b20b315d63..213190e812026 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -128,4 +128,11 @@ object TestData { case class TableName(tableName: String) TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).registerAsTable("tableName") + + val unparsedStrings = + TestSQLContext.sparkContext.parallelize( + "1, A1, true, null" :: + "2, B2, false, null" :: + "3, C3, true, null" :: + "4, D4, true, 2147483644" :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala new file mode 100644 index 0000000000000..46de6fe239228 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java + +import org.apache.spark.sql.types.util.DataTypeConversions +import org.scalatest.FunSuite + +import org.apache.spark.sql._ +import DataTypeConversions._ + +class ScalaSideDataTypeConversionSuite extends FunSuite { + + def checkDataType(scalaDataType: DataType) { + val javaDataType = asJavaDataType(scalaDataType) + val actual = asScalaDataType(javaDataType) + assert(scalaDataType === actual, s"Converted data type ${actual} " + + s"does not equal the expected data type ${scalaDataType}") + } + + test("convert data types") { + // Simple DataTypes. + checkDataType(StringType) + checkDataType(BinaryType) + checkDataType(BooleanType) + checkDataType(TimestampType) + checkDataType(DecimalType) + checkDataType(DoubleType) + checkDataType(FloatType) + checkDataType(ByteType) + checkDataType(IntegerType) + checkDataType(LongType) + checkDataType(ShortType) + + // Simple ArrayType. + val simpleScalaArrayType = ArrayType(StringType, true) + checkDataType(simpleScalaArrayType) + + // Simple MapType. + val simpleScalaMapType = MapType(StringType, LongType) + checkDataType(simpleScalaMapType) + + // Simple StructType. + val simpleScalaStructType = StructType( + StructField("a", DecimalType, false) :: + StructField("b", BooleanType, true) :: + StructField("c", LongType, true) :: + StructField("d", BinaryType, false) :: Nil) + checkDataType(simpleScalaStructType) + + // Complex StructType. + val complexScalaStructType = StructType( + StructField("simpleArray", simpleScalaArrayType, true) :: + StructField("simpleMap", simpleScalaMapType, true) :: + StructField("simpleStruct", simpleScalaStructType, true) :: + StructField("boolean", BooleanType, false) :: Nil) + checkDataType(complexScalaStructType) + + // Complex ArrayType. + val complexScalaArrayType = ArrayType(complexScalaStructType, true) + checkDataType(complexScalaArrayType) + + // Complex MapType. + val complexScalaMapType = MapType(complexScalaStructType, complexScalaArrayType, false) + checkDataType(complexScalaMapType) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index e765cfc83a397..9d9cfdd7c92e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -17,16 +17,12 @@ package org.apache.spark.sql.json -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.LeafNode import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.TestSQLContext._ -protected case class Schema(output: Seq[Attribute]) extends LeafNode - class JsonSuite extends QueryTest { import TestJsonData._ TestJsonData @@ -127,6 +123,18 @@ class JsonSuite extends QueryTest { checkDataType(ArrayType(IntegerType), ArrayType(LongType), ArrayType(LongType)) checkDataType(ArrayType(IntegerType), ArrayType(StringType), ArrayType(StringType)) checkDataType(ArrayType(IntegerType), StructType(Nil), StringType) + checkDataType( + ArrayType(IntegerType, true), ArrayType(IntegerType), ArrayType(IntegerType, true)) + checkDataType( + ArrayType(IntegerType, true), ArrayType(IntegerType, false), ArrayType(IntegerType, true)) + checkDataType( + ArrayType(IntegerType, true), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) + checkDataType( + ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, false)) + checkDataType( + ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType, false)) + checkDataType( + ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType)) // StructType checkDataType(StructType(Nil), StructType(Nil), StructType(Nil)) @@ -164,16 +172,16 @@ class JsonSuite extends QueryTest { test("Primitive field and type inferring") { val jsonSchemaRDD = jsonRDD(primitiveFieldAndType) - val expectedSchema = - AttributeReference("bigInteger", DecimalType, true)() :: - AttributeReference("boolean", BooleanType, true)() :: - AttributeReference("double", DoubleType, true)() :: - AttributeReference("integer", IntegerType, true)() :: - AttributeReference("long", LongType, true)() :: - AttributeReference("null", StringType, true)() :: - AttributeReference("string", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("bigInteger", DecimalType, true) :: + StructField("boolean", BooleanType, true) :: + StructField("double", DoubleType, true) :: + StructField("integer", IntegerType, true) :: + StructField("long", LongType, true) :: + StructField("null", StringType, true) :: + StructField("string", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) jsonSchemaRDD.registerAsTable("jsonTable") @@ -192,27 +200,28 @@ class JsonSuite extends QueryTest { test("Complex field and type inferring") { val jsonSchemaRDD = jsonRDD(complexFieldAndType) - val expectedSchema = - AttributeReference("arrayOfArray1", ArrayType(ArrayType(StringType)), true)() :: - AttributeReference("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true)() :: - AttributeReference("arrayOfBigInteger", ArrayType(DecimalType), true)() :: - AttributeReference("arrayOfBoolean", ArrayType(BooleanType), true)() :: - AttributeReference("arrayOfDouble", ArrayType(DoubleType), true)() :: - AttributeReference("arrayOfInteger", ArrayType(IntegerType), true)() :: - AttributeReference("arrayOfLong", ArrayType(LongType), true)() :: - AttributeReference("arrayOfNull", ArrayType(StringType), true)() :: - AttributeReference("arrayOfString", ArrayType(StringType), true)() :: - AttributeReference("arrayOfStruct", ArrayType( - StructType(StructField("field1", BooleanType, true) :: - StructField("field2", StringType, true) :: Nil)), true)() :: - AttributeReference("struct", StructType( - StructField("field1", BooleanType, true) :: - StructField("field2", DecimalType, true) :: Nil), true)() :: - AttributeReference("structWithArrayFields", StructType( + val expectedSchema = StructType( + StructField("arrayOfArray1", ArrayType(ArrayType(StringType)), true) :: + StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType), true) :: + StructField("arrayOfInteger", ArrayType(IntegerType), true) :: + StructField("arrayOfLong", ArrayType(LongType), true) :: + StructField("arrayOfNull", ArrayType(StringType, true), true) :: + StructField("arrayOfString", ArrayType(StringType), true) :: + StructField("arrayOfStruct", ArrayType( + StructType( + StructField("field1", BooleanType, true) :: + StructField("field2", StringType, true) :: Nil)), true) :: + StructField("struct", StructType( + StructField("field1", BooleanType, true) :: + StructField("field2", DecimalType, true) :: Nil), true) :: + StructField("structWithArrayFields", StructType( StructField("field1", ArrayType(IntegerType), true) :: - StructField("field2", ArrayType(StringType), true) :: Nil), true)() :: Nil + StructField("field2", ArrayType(StringType), true) :: Nil), true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) jsonSchemaRDD.registerAsTable("jsonTable") @@ -301,15 +310,15 @@ class JsonSuite extends QueryTest { test("Type conflict in primitive field values") { val jsonSchemaRDD = jsonRDD(primitiveFieldValueTypeConflict) - val expectedSchema = - AttributeReference("num_bool", StringType, true)() :: - AttributeReference("num_num_1", LongType, true)() :: - AttributeReference("num_num_2", DecimalType, true)() :: - AttributeReference("num_num_3", DoubleType, true)() :: - AttributeReference("num_str", StringType, true)() :: - AttributeReference("str_bool", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("num_bool", StringType, true) :: + StructField("num_num_1", LongType, true) :: + StructField("num_num_2", DecimalType, true) :: + StructField("num_num_3", DoubleType, true) :: + StructField("num_str", StringType, true) :: + StructField("str_bool", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) jsonSchemaRDD.registerAsTable("jsonTable") @@ -426,15 +435,15 @@ class JsonSuite extends QueryTest { test("Type conflict in complex field values") { val jsonSchemaRDD = jsonRDD(complexFieldValueTypeConflict) - val expectedSchema = - AttributeReference("array", ArrayType(IntegerType), true)() :: - AttributeReference("num_struct", StringType, true)() :: - AttributeReference("str_array", StringType, true)() :: - AttributeReference("struct", StructType( - StructField("field", StringType, true) :: Nil), true)() :: - AttributeReference("struct_array", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("array", ArrayType(IntegerType), true) :: + StructField("num_struct", StringType, true) :: + StructField("str_array", StringType, true) :: + StructField("struct", StructType( + StructField("field", StringType, true) :: Nil), true) :: + StructField("struct_array", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) jsonSchemaRDD.registerAsTable("jsonTable") @@ -450,12 +459,12 @@ class JsonSuite extends QueryTest { test("Type conflict in array elements") { val jsonSchemaRDD = jsonRDD(arrayElementTypeConflict) - val expectedSchema = - AttributeReference("array1", ArrayType(StringType), true)() :: - AttributeReference("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil)), true)() :: Nil + val expectedSchema = StructType( + StructField("array1", ArrayType(StringType, true), true) :: + StructField("array2", ArrayType(StructType( + StructField("field", LongType, true) :: Nil)), true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) jsonSchemaRDD.registerAsTable("jsonTable") @@ -475,15 +484,15 @@ class JsonSuite extends QueryTest { test("Handling missing fields") { val jsonSchemaRDD = jsonRDD(missingFields) - val expectedSchema = - AttributeReference("a", BooleanType, true)() :: - AttributeReference("b", LongType, true)() :: - AttributeReference("c", ArrayType(IntegerType), true)() :: - AttributeReference("d", StructType( - StructField("field", BooleanType, true) :: Nil), true)() :: - AttributeReference("e", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("a", BooleanType, true) :: + StructField("b", LongType, true) :: + StructField("c", ArrayType(IntegerType), true) :: + StructField("d", StructType( + StructField("field", BooleanType, true) :: Nil), true) :: + StructField("e", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) jsonSchemaRDD.registerAsTable("jsonTable") } @@ -494,16 +503,16 @@ class JsonSuite extends QueryTest { primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) val jsonSchemaRDD = jsonFile(path) - val expectedSchema = - AttributeReference("bigInteger", DecimalType, true)() :: - AttributeReference("boolean", BooleanType, true)() :: - AttributeReference("double", DoubleType, true)() :: - AttributeReference("integer", IntegerType, true)() :: - AttributeReference("long", LongType, true)() :: - AttributeReference("null", StringType, true)() :: - AttributeReference("string", StringType, true)() :: Nil + val expectedSchema = StructType( + StructField("bigInteger", DecimalType, true) :: + StructField("boolean", BooleanType, true) :: + StructField("double", DoubleType, true) :: + StructField("integer", IntegerType, true) :: + StructField("long", LongType, true) :: + StructField("null", StringType, true) :: + StructField("string", StringType, true) :: Nil) - comparePlans(Schema(expectedSchema), Schema(jsonSchemaRDD.logicalPlan.output)) + assert(expectedSchema === jsonSchemaRDD.schema) jsonSchemaRDD.registerAsTable("jsonTable") @@ -518,4 +527,53 @@ class JsonSuite extends QueryTest { "this is a simple string.") :: Nil ) } + + test("Applying schemas") { + val file = getTempFilePath("json") + val path = file.toString + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) + + val schema = StructType( + StructField("bigInteger", DecimalType, true) :: + StructField("boolean", BooleanType, true) :: + StructField("double", DoubleType, true) :: + StructField("integer", IntegerType, true) :: + StructField("long", LongType, true) :: + StructField("null", StringType, true) :: + StructField("string", StringType, true) :: Nil) + + val jsonSchemaRDD1 = jsonFile(path, schema) + + assert(schema === jsonSchemaRDD1.schema) + + jsonSchemaRDD1.registerAsTable("jsonTable1") + + checkAnswer( + sql("select * from jsonTable1"), + (BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.") :: Nil + ) + + val jsonSchemaRDD2 = jsonRDD(primitiveFieldAndType, schema) + + assert(schema === jsonSchemaRDD2.schema) + + jsonSchemaRDD2.registerAsTable("jsonTable2") + + checkAnswer( + sql("select * from jsonTable2"), + (BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.") :: Nil + ) + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index f0a61270daf05..b413373345eea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -37,7 +37,6 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand @@ -260,9 +259,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ)) => + case (seq: Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType)) => + case (map: Map[_,_], MapType(kType, vType, _)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) @@ -279,9 +278,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { struct.zip(fields).map { case (v, t) => s""""${t.name}":${toHiveStructString(v, t.dataType)}""" }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ)) => + case (seq: Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType)) => + case (map: Map[_,_], MapType(kType, vType, _)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index ad7dc0ecdb1bf..354fcd53f303b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -152,8 +152,9 @@ private[hive] trait HiveInspectors { } def toInspector(dataType: DataType): ObjectInspector = dataType match { - case ArrayType(tpe) => ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) - case MapType(keyType, valueType) => + case ArrayType(tpe, _) => + ObjectInspectorFactory.getStandardListObjectInspector(toInspector(tpe)) + case MapType(keyType, valueType, _) => ObjectInspectorFactory.getStandardMapObjectInspector( toInspector(keyType), toInspector(valueType)) case StringType => PrimitiveObjectInspectorFactory.javaStringObjectInspector 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 dff1d6a4b93bb..fa4e78439c26c 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 @@ -200,7 +200,9 @@ object HiveMetastoreTypes extends RegexParsers { "varchar\\((\\d+)\\)".r ^^^ StringType protected lazy val arrayType: Parser[DataType] = - "array" ~> "<" ~> dataType <~ ">" ^^ ArrayType + "array" ~> "<" ~> dataType <~ ">" ^^ { + case tpe => ArrayType(tpe) + } protected lazy val mapType: Parser[DataType] = "map" ~> "<" ~> dataType ~ "," ~ dataType <~ ">" ^^ { @@ -229,10 +231,10 @@ object HiveMetastoreTypes extends RegexParsers { } def toMetastoreType(dt: DataType): String = dt match { - case ArrayType(elementType) => s"array<${toMetastoreType(elementType)}>" + case ArrayType(elementType, _) => s"array<${toMetastoreType(elementType)}>" case StructType(fields) => s"struct<${fields.map(f => s"${f.name}:${toMetastoreType(f.dataType)}").mkString(",")}>" - case MapType(keyType, valueType) => + case MapType(keyType, valueType, _) => s"map<${toMetastoreType(keyType)},${toMetastoreType(valueType)}>" case StringType => "string" case FloatType => "float" From 7c5fc28af42daaa6725af083d78c2372f3d0a338 Mon Sep 17 00:00:00 2001 From: Koert Kuipers Date: Wed, 30 Jul 2014 00:18:59 -0700 Subject: [PATCH 0124/1492] SPARK-2543: Allow user to set maximum Kryo buffer size Author: Koert Kuipers Closes #735 from koertkuipers/feat-kryo-max-buffersize and squashes the following commits: 15f6d81 [Koert Kuipers] change default for spark.kryoserializer.buffer.max.mb to 64mb and add some documentation 1bcc22c [Koert Kuipers] Merge branch 'master' into feat-kryo-max-buffersize 0c9f8eb [Koert Kuipers] make default for kryo max buffer size 16MB 143ec4d [Koert Kuipers] test resizable buffer in kryo Output 0732445 [Koert Kuipers] support setting maxCapacity to something different than capacity in kryo Output --- .../spark/serializer/KryoSerializer.scala | 3 +- .../serializer/KryoSerializerSuite.scala | 30 +++++++++++++++++++ docs/configuration.md | 16 +++++++--- 3 files changed, 44 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index fa79b25759153..e60b802a86a14 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -48,11 +48,12 @@ class KryoSerializer(conf: SparkConf) with Serializable { private val bufferSize = conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 + private val maxBufferSize = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) * 1024 * 1024 private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) private val registrator = conf.getOption("spark.kryo.registrator") - def newKryoOutput() = new KryoOutput(bufferSize) + def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 79280d1a06653..789b773bae316 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -209,6 +209,36 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } } +class KryoSerializerResizableOutputSuite extends FunSuite { + import org.apache.spark.SparkConf + import org.apache.spark.SparkContext + import org.apache.spark.LocalSparkContext + import org.apache.spark.SparkException + + // trial and error showed this will not serialize with 1mb buffer + val x = (1 to 400000).toArray + + test("kryo without resizable output buffer should fail on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "1") + val sc = new SparkContext("local", "test", conf) + intercept[SparkException](sc.parallelize(x).collect) + LocalSparkContext.stop(sc) + } + + test("kryo with resizable output buffer should succeed on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "2") + val sc = new SparkContext("local", "test", conf) + assert(sc.parallelize(x).collect === x) + LocalSparkContext.stop(sc) + } +} + object KryoTest { case class CaseClass(i: Int, s: String) {} diff --git a/docs/configuration.md b/docs/configuration.md index 2e6c85cc2bcca..ea69057b5be10 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -414,10 +414,18 @@ Apart from these, the following properties are also available, and may be useful spark.kryoserializer.buffer.mb 2 - Maximum object size to allow within Kryo (the library needs to create a buffer at least as - large as the largest single object you'll serialize). Increase this if you get a "buffer limit - exceeded" exception inside Kryo. Note that there will be one buffer per core on each - worker. + Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer + per core on each worker. This buffer will grow up to + spark.kryoserializer.buffer.max.mb if needed. + + + + spark.kryoserializer.buffer.max.mb + 64 + + Maximum allowable size of Kryo serialization buffer, in megabytes. This must be larger than any + object you attempt to serialize. Increase this if you get a "buffer limit exceeded" exception + inside Kryo. From ee07541e99f0d262bf662b669b6542cf302ff39c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 30 Jul 2014 08:55:15 -0700 Subject: [PATCH 0125/1492] SPARK-2748 [MLLIB] [GRAPHX] Loss of precision for small arguments to Math.exp, Math.log In a few places in MLlib, an expression of the form `log(1.0 + p)` is evaluated. When p is so small that `1.0 + p == 1.0`, the result is 0.0. However the correct answer is very near `p`. This is why `Math.log1p` exists. Similarly for one instance of `exp(m) - 1` in GraphX; there's a special `Math.expm1` method. While the errors occur only for very small arguments, given their use in machine learning algorithms, this is entirely possible. Also note the related PR for Python: https://github.com/apache/spark/pull/1652 Author: Sean Owen Closes #1659 from srowen/SPARK-2748 and squashes the following commits: c5926d4 [Sean Owen] Use log1p, expm1 for better precision for tiny arguments --- .../org/apache/spark/graphx/util/GraphGenerators.scala | 6 ++++-- .../org/apache/spark/mllib/optimization/Gradient.scala | 8 ++++---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 635514f09ece0..60149548ab852 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -100,8 +100,10 @@ object GraphGenerators { */ private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { val rand = new Random() - val m = math.exp(mu + (sigma * sigma) / 2.0) - val s = math.sqrt((math.exp(sigma*sigma) - 1) * math.exp(2*mu + sigma*sigma)) + val sigmaSq = sigma * sigma + val m = math.exp(mu + sigmaSq / 2.0) + // expm1 is exp(m)-1 with better accuracy for tiny m + val s = math.sqrt(math.expm1(sigmaSq) * math.exp(2*mu + sigmaSq)) // Z ~ N(0, 1) var X: Double = maxVal diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 679842f831c2a..9d82f011e674a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -68,9 +68,9 @@ class LogisticGradient extends Gradient { val gradient = brzData * gradientMultiplier val loss = if (label > 0) { - math.log(1 + math.exp(margin)) + math.log1p(math.exp(margin)) // log1p is log(1+p) but more accurate for small p } else { - math.log(1 + math.exp(margin)) - margin + math.log1p(math.exp(margin)) - margin } (Vectors.fromBreeze(gradient), loss) @@ -89,9 +89,9 @@ class LogisticGradient extends Gradient { brzAxpy(gradientMultiplier, brzData, cumGradient.toBreeze) if (label > 0) { - math.log(1 + math.exp(margin)) + math.log1p(math.exp(margin)) } else { - math.log(1 + math.exp(margin)) - margin + math.log1p(math.exp(margin)) - margin } } } From 774142f5556ac37fddf03cfa46eb23ca1bde2492 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 09:27:43 -0700 Subject: [PATCH 0126/1492] [SPARK-2521] Broadcast RDD object (instead of sending it along with every task) This is a resubmission of #1452. It was reverted because it broke the build. Currently (as of Spark 1.0.1), Spark sends RDD object (which contains closures) using Akka along with the task itself to the executors. This is inefficient because all tasks in the same stage use the same RDD object, but we have to send RDD object multiple times to the executors. This is especially bad when a closure references some variable that is very large. The current design led to users having to explicitly broadcast large variables. The patch uses broadcast to send RDD objects and the closures to executors, and use Akka to only send a reference to the broadcast RDD/closure along with the partition specific information for the task. For those of you who know more about the internals, Spark already relies on broadcast to send the Hadoop JobConf every time it uses the Hadoop input, because the JobConf is large. The user-facing impact of the change include: 1. Users won't need to decide what to broadcast anymore, unless they would want to use a large object multiple times in different operations 2. Task size will get smaller, resulting in faster scheduling and higher task dispatch throughput. In addition, the change will simplify some internals of Spark, eliminating the need to maintain task caches and the complex logic to broadcast JobConf (which also led to a deadlock recently). A simple way to test this: ```scala val a = new Array[Byte](1000*1000); scala.util.Random.nextBytes(a); sc.parallelize(1 to 1000, 1000).map { x => a; x }.groupBy { x => a; x }.count ``` Numbers on 3 r3.8xlarge instances on EC2 ``` master branch: 5.648436068 s, 4.715361895 s, 5.360161877 s with this change: 3.416348793 s, 1.477846558 s, 1.553432156 s ``` Author: Reynold Xin Closes #1498 from rxin/broadcast-task and squashes the following commits: f7364db [Reynold Xin] Code review feedback. f8535dc [Reynold Xin] Fixed the style violation. 252238d [Reynold Xin] Serialize the final task closure as well as ShuffleDependency in taskBinary. 111007d [Reynold Xin] Fix broadcast tests. 797c247 [Reynold Xin] Properly send SparkListenerStageSubmitted and SparkListenerStageCompleted. bab1d8b [Reynold Xin] Check for NotSerializableException in submitMissingTasks. cf38450 [Reynold Xin] Use TorrentBroadcastFactory. 991c002 [Reynold Xin] Use HttpBroadcast. de779f8 [Reynold Xin] Fix TaskContextSuite. cc152fc [Reynold Xin] Don't cache the RDD broadcast variable. d256b45 [Reynold Xin] Fixed unit test failures. One more to go. cae0af3 [Reynold Xin] [SPARK-2521] Broadcast RDD object (instead of sending it along with every task). --- .../scala/org/apache/spark/Dependency.scala | 28 ++-- .../scala/org/apache/spark/SparkContext.scala | 2 - .../main/scala/org/apache/spark/rdd/RDD.scala | 11 +- .../apache/spark/rdd/RDDCheckpointData.scala | 9 +- .../apache/spark/scheduler/DAGScheduler.scala | 87 ++++++++---- .../apache/spark/scheduler/ResultTask.scala | 118 +++------------- .../spark/scheduler/ShuffleMapTask.scala | 129 ++++-------------- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../apache/spark/ContextCleanerSuite.scala | 71 ++++++---- .../scala/org/apache/spark/rdd/RDDSuite.scala | 8 +- .../spark/scheduler/TaskContextSuite.scala | 24 ++-- .../ui/jobs/JobProgressListenerSuite.scala | 11 +- 12 files changed, 198 insertions(+), 302 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 09a60571238ea..3935c8772252e 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -27,7 +27,9 @@ import org.apache.spark.shuffle.ShuffleHandle * Base class for dependencies. */ @DeveloperApi -abstract class Dependency[T](val rdd: RDD[T]) extends Serializable +abstract class Dependency[T] extends Serializable { + def rdd: RDD[T] +} /** @@ -36,20 +38,24 @@ abstract class Dependency[T](val rdd: RDD[T]) extends Serializable * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ @DeveloperApi -abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { +abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { /** * Get the parent partitions for a child partition. * @param partitionId a partition of the child RDD * @return the partitions of the parent RDD that the child partition depends upon */ def getParents(partitionId: Int): Seq[Int] + + override def rdd: RDD[T] = _rdd } /** * :: DeveloperApi :: - * Represents a dependency on the output of a shuffle stage. - * @param rdd the parent RDD + * Represents a dependency on the output of a shuffle stage. Note that in the case of shuffle, + * the RDD is transient since we don't need it on the executor side. + * + * @param _rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to None, * the default serializer, as specified by `spark.serializer` config option, will @@ -57,20 +63,22 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { */ @DeveloperApi class ShuffleDependency[K, V, C]( - @transient rdd: RDD[_ <: Product2[K, V]], + @transient _rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, val serializer: Option[Serializer] = None, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, val mapSideCombine: Boolean = false) - extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { + extends Dependency[Product2[K, V]] { + + override def rdd = _rdd.asInstanceOf[RDD[Product2[K, V]]] - val shuffleId: Int = rdd.context.newShuffleId() + val shuffleId: Int = _rdd.context.newShuffleId() - val shuffleHandle: ShuffleHandle = rdd.context.env.shuffleManager.registerShuffle( - shuffleId, rdd.partitions.size, this) + val shuffleHandle: ShuffleHandle = _rdd.context.env.shuffleManager.registerShuffle( + shuffleId, _rdd.partitions.size, this) - rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) + _rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3e6addeaf04a8..fb4c86716bb8d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -997,8 +997,6 @@ class SparkContext(config: SparkConf) extends Logging { // TODO: Cache.stop()? env.stop() SparkEnv.set(null) - ShuffleMapTask.clearCache() - ResultTask.clearCache() listenerBus.stop() eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") 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 a6abc49c5359e..726b3f2bbeea7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,12 +35,13 @@ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD +import org.apache.spark.broadcast.Broadcast import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, CallSite, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} @@ -1206,16 +1207,12 @@ abstract class RDD[T: ClassTag]( /** * Return whether this RDD has been checkpointed or not */ - def isCheckpointed: Boolean = { - checkpointData.map(_.isCheckpointed).getOrElse(false) - } + def isCheckpointed: Boolean = checkpointData.exists(_.isCheckpointed) /** * Gets the name of the file to which this RDD was checkpointed */ - def getCheckpointFile: Option[String] = { - checkpointData.flatMap(_.getCheckpointFile) - } + def getCheckpointFile: Option[String] = checkpointData.flatMap(_.getCheckpointFile) // ======================================================================= // Other internal methods and fields diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index c3b2a33fb54d0..f67e5f1857979 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -106,7 +106,6 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) cpRDD = Some(newRDD) rdd.markCheckpointed(newRDD) // Update the RDD's dependencies and partitions cpState = Checkpointed - RDDCheckpointData.clearTaskCaches() } logInfo("Done checkpointing RDD " + rdd.id + " to " + path + ", new parent is RDD " + newRDD.id) } @@ -131,9 +130,5 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -private[spark] object RDDCheckpointData { - def clearTaskCaches() { - ShuffleMapTask.clearCache() - ResultTask.clearCache() - } -} +// Used for synchronization +private[spark] object RDDCheckpointData diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index dc6142ab79d03..50186d097a632 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.io.{NotSerializableException, PrintWriter, StringWriter} +import java.io.NotSerializableException import java.util.Properties import java.util.concurrent.atomic.AtomicInteger @@ -35,6 +35,7 @@ import akka.pattern.ask import akka.util.Timeout import org.apache.spark._ +import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD @@ -114,6 +115,10 @@ class DAGScheduler( private val dagSchedulerActorSupervisor = env.actorSystem.actorOf(Props(new DAGSchedulerActorSupervisor(this))) + // A closure serializer that we reuse. + // This is only safe because DAGScheduler runs in a single thread. + private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() + private[scheduler] var eventProcessActor: ActorRef = _ private def initializeEventProcessActor() { @@ -361,9 +366,6 @@ class DAGScheduler( // data structures based on StageId stageIdToStage -= stageId - ShuffleMapTask.removeStage(stageId) - ResultTask.removeStage(stageId) - logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } @@ -691,49 +693,83 @@ class DAGScheduler( } } - /** Called when stage's parents are available and we can now do its task. */ private def submitMissingTasks(stage: Stage, jobId: Int) { logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry stage.pendingTasks.clear() var tasks = ArrayBuffer[Task[_]]() + + val properties = if (jobIdToActiveJob.contains(jobId)) { + jobIdToActiveJob(stage.jobId).properties + } else { + // this stage will be assigned to "default" pool + null + } + + runningStages += stage + // SparkListenerStageSubmitted should be posted before testing whether tasks are + // serializable. If tasks are not serializable, a SparkListenerStageCompleted event + // will be posted, which should always come after a corresponding SparkListenerStageSubmitted + // event. + listenerBus.post(SparkListenerStageSubmitted(stage.info, properties)) + + // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. + // Broadcasted binary for the task, used to dispatch tasks to executors. Note that we broadcast + // the serialized copy of the RDD and for each task we will deserialize it, which means each + // task gets a different copy of the RDD. This provides stronger isolation between tasks that + // might modify state of objects referenced in their closures. This is necessary in Hadoop + // where the JobConf/Configuration object is not thread-safe. + var taskBinary: Broadcast[Array[Byte]] = null + try { + // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). + // For ResultTask, serialize and broadcast (rdd, func). + val taskBinaryBytes: Array[Byte] = + if (stage.isShuffleMap) { + closureSerializer.serialize((stage.rdd, stage.shuffleDep.get) : AnyRef).array() + } else { + closureSerializer.serialize((stage.rdd, stage.resultOfJob.get.func) : AnyRef).array() + } + taskBinary = sc.broadcast(taskBinaryBytes) + } catch { + // In the case of a failure during serialization, abort the stage. + case e: NotSerializableException => + abortStage(stage, "Task not serializable: " + e.toString) + runningStages -= stage + return + case NonFatal(e) => + abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}") + runningStages -= stage + return + } + if (stage.isShuffleMap) { for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { val locs = getPreferredLocs(stage.rdd, p) - tasks += new ShuffleMapTask(stage.id, stage.rdd, stage.shuffleDep.get, p, locs) + val part = stage.rdd.partitions(p) + tasks += new ShuffleMapTask(stage.id, taskBinary, part, locs) } } else { // This is a final stage; figure out its job's missing partitions val job = stage.resultOfJob.get for (id <- 0 until job.numPartitions if !job.finished(id)) { - val partition = job.partitions(id) - val locs = getPreferredLocs(stage.rdd, partition) - tasks += new ResultTask(stage.id, stage.rdd, job.func, partition, locs, id) + val p: Int = job.partitions(id) + val part = stage.rdd.partitions(p) + val locs = getPreferredLocs(stage.rdd, p) + tasks += new ResultTask(stage.id, taskBinary, part, locs, id) } } - val properties = if (jobIdToActiveJob.contains(jobId)) { - jobIdToActiveJob(stage.jobId).properties - } else { - // this stage will be assigned to "default" pool - null - } - if (tasks.size > 0) { - runningStages += stage - // SparkListenerStageSubmitted should be posted before testing whether tasks are - // serializable. If tasks are not serializable, a SparkListenerStageCompleted event - // will be posted, which should always come after a corresponding SparkListenerStageSubmitted - // event. - listenerBus.post(SparkListenerStageSubmitted(stage.info, properties)) - // Preemptively serialize a task to make sure it can be serialized. We are catching this // exception here because it would be fairly hard to catch the non-serializable exception // down the road, where we have several different implementations for local scheduler and // cluster schedulers. + // + // We've already serialized RDDs and closures in taskBinary, but here we check for all other + // objects such as Partition. try { - SparkEnv.get.closureSerializer.newInstance().serialize(tasks.head) + closureSerializer.serialize(tasks.head) } catch { case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) @@ -752,6 +788,9 @@ class DAGScheduler( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stage.info.submissionTime = Some(clock.getTime()) } else { + // Because we posted SparkListenerStageSubmitted earlier, we should post + // SparkListenerStageCompleted here in case there are no tasks to run. + listenerBus.post(SparkListenerStageCompleted(stage.info)) logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) runningStages -= stage diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index bbf9f7388b074..d09fd7aa57642 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -17,134 +17,56 @@ package org.apache.spark.scheduler -import scala.language.existentials +import java.nio.ByteBuffer import java.io._ -import java.util.zip.{GZIPInputStream, GZIPOutputStream} - -import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.rdd.{RDD, RDDCheckpointData} - -private[spark] object ResultTask { - - // A simple map between the stage id to the serialized byte array of a task. - // Served as a cache for task serialization because serialization can be - // expensive on the master node if it needs to launch thousands of tasks. - private val serializedInfoCache = new HashMap[Int, Array[Byte]] - - def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = - { - synchronized { - val old = serializedInfoCache.get(stageId).orNull - if (old != null) { - old - } else { - val out = new ByteArrayOutputStream - val ser = SparkEnv.get.closureSerializer.newInstance() - val objOut = ser.serializeStream(new GZIPOutputStream(out)) - objOut.writeObject(rdd) - objOut.writeObject(func) - objOut.close() - val bytes = out.toByteArray - serializedInfoCache.put(stageId, bytes) - bytes - } - } - } - - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = - { - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val ser = SparkEnv.get.closureSerializer.newInstance() - val objIn = ser.deserializeStream(in) - val rdd = objIn.readObject().asInstanceOf[RDD[_]] - val func = objIn.readObject().asInstanceOf[(TaskContext, Iterator[_]) => _] - (rdd, func) - } - - def removeStage(stageId: Int) { - serializedInfoCache.remove(stageId) - } - - def clearCache() { - synchronized { - serializedInfoCache.clear() - } - } -} - +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD /** * A task that sends back the output to the driver application. * - * See [[org.apache.spark.scheduler.Task]] for more information. + * See [[Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param rdd input to func - * @param func a function to apply on a partition of the RDD - * @param _partitionId index of the number in the RDD + * @param taskBinary broadcasted version of the serialized RDD and the function to apply on each + * partition of the given RDD. Once deserialized, the type should be + * (RDD[T], (TaskContext, Iterator[T]) => U). + * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling * @param outputId index of the task in this job (a job can launch tasks on only a subset of the * input RDD's partitions). */ private[spark] class ResultTask[T, U]( stageId: Int, - var rdd: RDD[T], - var func: (TaskContext, Iterator[T]) => U, - _partitionId: Int, + taskBinary: Broadcast[Array[Byte]], + partition: Partition, @transient locs: Seq[TaskLocation], - var outputId: Int) - extends Task[U](stageId, _partitionId) with Externalizable { + val outputId: Int) + extends Task[U](stageId, partition.index) with Serializable { - def this() = this(0, null, null, 0, null, 0) - - var split = if (rdd == null) null else rdd.partitions(partitionId) - - @transient private val preferredLocs: Seq[TaskLocation] = { + @transient private[this] val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } override def runTask(context: TaskContext): U = { + // Deserialize the RDD and the func using the broadcast variables. + val ser = SparkEnv.get.closureSerializer.newInstance() + val (rdd, func) = ser.deserialize[(RDD[T], (TaskContext, Iterator[T]) => U)]( + ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + metrics = Some(context.taskMetrics) try { - func(context, rdd.iterator(split, context)) + func(context, rdd.iterator(partition, context)) } finally { context.executeOnCompleteCallbacks() } } + // This is only callable on the driver side. override def preferredLocations: Seq[TaskLocation] = preferredLocs override def toString = "ResultTask(" + stageId + ", " + partitionId + ")" - - override def writeExternal(out: ObjectOutput) { - RDDCheckpointData.synchronized { - split = rdd.partitions(partitionId) - out.writeInt(stageId) - val bytes = ResultTask.serializeInfo( - stageId, rdd, func.asInstanceOf[(TaskContext, Iterator[_]) => _]) - out.writeInt(bytes.length) - out.write(bytes) - out.writeInt(partitionId) - out.writeInt(outputId) - out.writeLong(epoch) - out.writeObject(split) - } - } - - override def readExternal(in: ObjectInput) { - val stageId = in.readInt() - val numBytes = in.readInt() - val bytes = new Array[Byte](numBytes) - in.readFully(bytes) - val (rdd_, func_) = ResultTask.deserializeInfo(stageId, bytes) - rdd = rdd_.asInstanceOf[RDD[T]] - func = func_.asInstanceOf[(TaskContext, Iterator[T]) => U] - partitionId = in.readInt() - outputId = in.readInt() - epoch = in.readLong() - split = in.readObject().asInstanceOf[Partition] - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index fdaf1de83f051..11255c07469d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -17,134 +17,55 @@ package org.apache.spark.scheduler -import scala.language.existentials - -import java.io._ -import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import java.nio.ByteBuffer -import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ -import org.apache.spark.rdd.{RDD, RDDCheckpointData} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD import org.apache.spark.shuffle.ShuffleWriter -private[spark] object ShuffleMapTask { - - // A simple map between the stage id to the serialized byte array of a task. - // Served as a cache for task serialization because serialization can be - // expensive on the master node if it needs to launch thousands of tasks. - private val serializedInfoCache = new HashMap[Int, Array[Byte]] - - def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_, _, _]): Array[Byte] = { - synchronized { - val old = serializedInfoCache.get(stageId).orNull - if (old != null) { - return old - } else { - val out = new ByteArrayOutputStream - val ser = SparkEnv.get.closureSerializer.newInstance() - val objOut = ser.serializeStream(new GZIPOutputStream(out)) - objOut.writeObject(rdd) - objOut.writeObject(dep) - objOut.close() - val bytes = out.toByteArray - serializedInfoCache.put(stageId, bytes) - bytes - } - } - } - - def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], ShuffleDependency[_, _, _]) = { - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val ser = SparkEnv.get.closureSerializer.newInstance() - val objIn = ser.deserializeStream(in) - val rdd = objIn.readObject().asInstanceOf[RDD[_]] - val dep = objIn.readObject().asInstanceOf[ShuffleDependency[_, _, _]] - (rdd, dep) - } - - // Since both the JarSet and FileSet have the same format this is used for both. - def deserializeFileSet(bytes: Array[Byte]): HashMap[String, Long] = { - val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) - val objIn = new ObjectInputStream(in) - val set = objIn.readObject().asInstanceOf[Array[(String, Long)]].toMap - HashMap(set.toSeq: _*) - } - - def removeStage(stageId: Int) { - serializedInfoCache.remove(stageId) - } - - def clearCache() { - synchronized { - serializedInfoCache.clear() - } - } -} - /** - * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner - * specified in the ShuffleDependency). - * - * See [[org.apache.spark.scheduler.Task]] for more information. - * +* A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner +* specified in the ShuffleDependency). +* +* See [[org.apache.spark.scheduler.Task]] for more information. +* * @param stageId id of the stage this task belongs to - * @param rdd the final RDD in this stage - * @param dep the ShuffleDependency - * @param _partitionId index of the number in the RDD + * @param taskBinary broadcast version of of the RDD and the ShuffleDependency. Once deserialized, + * the type should be (RDD[_], ShuffleDependency[_, _, _]). + * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling */ private[spark] class ShuffleMapTask( stageId: Int, - var rdd: RDD[_], - var dep: ShuffleDependency[_, _, _], - _partitionId: Int, + taskBinary: Broadcast[Array[Byte]], + partition: Partition, @transient private var locs: Seq[TaskLocation]) - extends Task[MapStatus](stageId, _partitionId) - with Externalizable - with Logging { + extends Task[MapStatus](stageId, partition.index) with Logging { - protected def this() = this(0, null, null, 0, null) + /** A constructor used only in test suites. This does not require passing in an RDD. */ + def this(partitionId: Int) { + this(0, null, new Partition { override def index = 0 }, null) + } @transient private val preferredLocs: Seq[TaskLocation] = { if (locs == null) Nil else locs.toSet.toSeq } - var split = if (rdd == null) null else rdd.partitions(partitionId) - - override def writeExternal(out: ObjectOutput) { - RDDCheckpointData.synchronized { - split = rdd.partitions(partitionId) - out.writeInt(stageId) - val bytes = ShuffleMapTask.serializeInfo(stageId, rdd, dep) - out.writeInt(bytes.length) - out.write(bytes) - out.writeInt(partitionId) - out.writeLong(epoch) - out.writeObject(split) - } - } - - override def readExternal(in: ObjectInput) { - val stageId = in.readInt() - val numBytes = in.readInt() - val bytes = new Array[Byte](numBytes) - in.readFully(bytes) - val (rdd_, dep_) = ShuffleMapTask.deserializeInfo(stageId, bytes) - rdd = rdd_ - dep = dep_ - partitionId = in.readInt() - epoch = in.readLong() - split = in.readObject().asInstanceOf[Partition] - } - override def runTask(context: TaskContext): MapStatus = { + // Deserialize the RDD using the broadcast variable. + val ser = SparkEnv.get.closureSerializer.newInstance() + val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( + ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) + metrics = Some(context.taskMetrics) var writer: ShuffleWriter[Any, Any] = null try { val manager = SparkEnv.get.shuffleManager writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) - writer.write(rdd.iterator(split, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) return writer.stop(success = true).get } catch { case e: Exception => 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 69f65b4bdccb1..f8fbb3ad6d4a1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -38,7 +38,7 @@ import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} +import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.ExecutorUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 13b415cccb647..ad20f9b937ac1 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark import java.lang.ref.WeakReference +import org.apache.spark.broadcast.Broadcast + +import scala.collection.mutable import scala.collection.mutable.{HashSet, SynchronizedSet} import scala.language.existentials import scala.language.postfixOps @@ -52,9 +55,8 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } } - test("cleanup RDD") { - val rdd = newRDD.persist() + val rdd = newRDD().persist() val collected = rdd.collect().toList val tester = new CleanerTester(sc, rddIds = Seq(rdd.id)) @@ -67,7 +69,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup shuffle") { - val (rdd, shuffleDeps) = newRDDWithShuffleDependencies + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies() val collected = rdd.collect().toList val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) @@ -80,7 +82,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("cleanup broadcast") { - val broadcast = newBroadcast + val broadcast = newBroadcast() val tester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) // Explicit cleanup @@ -89,7 +91,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup RDD") { - var rdd = newRDD.persist() + var rdd = newRDD().persist() rdd.count() // Test that GC does not cause RDD cleanup due to a strong reference @@ -107,7 +109,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup shuffle") { - var rdd = newShuffleRDD + var rdd = newShuffleRDD() rdd.count() // Test that GC does not cause shuffle cleanup due to a strong reference @@ -125,7 +127,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } test("automatically cleanup broadcast") { - var broadcast = newBroadcast + var broadcast = newBroadcast() // Test that GC does not cause broadcast cleanup due to a strong reference val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) @@ -144,11 +146,11 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId - val broadcastIds = 0L until numBroadcasts + val broadcastIds = broadcastBuffer.map(_.id) val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) runGC() @@ -162,6 +164,13 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo rddBuffer.clear() runGC() postGCTester.assertCleanup() + + // Make sure the broadcasted task closure no longer exists after GC. + val taskClosureBroadcastId = broadcastIds.max + 1 + assert(sc.env.blockManager.master.getMatchingBlockIds({ + case BroadcastBlockId(`taskClosureBroadcastId`, _) => true + case _ => false + }, askSlaves = true).isEmpty) } test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { @@ -175,11 +184,11 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo val numRdds = 10 val numBroadcasts = 4 // Broadcasts are more costly - val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId - val broadcastIds = 0L until numBroadcasts + val broadcastIds = broadcastBuffer.map(_.id) val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) runGC() @@ -193,21 +202,29 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo rddBuffer.clear() runGC() postGCTester.assertCleanup() + + // Make sure the broadcasted task closure no longer exists after GC. + val taskClosureBroadcastId = broadcastIds.max + 1 + assert(sc.env.blockManager.master.getMatchingBlockIds({ + case BroadcastBlockId(`taskClosureBroadcastId`, _) => true + case _ => false + }, askSlaves = true).isEmpty) } //------ Helper functions ------ - def newRDD = sc.makeRDD(1 to 10) - def newPairRDD = newRDD.map(_ -> 1) - def newShuffleRDD = newPairRDD.reduceByKey(_ + _) - def newBroadcast = sc.broadcast(1 to 100) - def newRDDWithShuffleDependencies: (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { + private def newRDD() = sc.makeRDD(1 to 10) + private def newPairRDD() = newRDD().map(_ -> 1) + private def newShuffleRDD() = newPairRDD().reduceByKey(_ + _) + private def newBroadcast() = sc.broadcast(1 to 100) + + private def newRDDWithShuffleDependencies(): (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { rdd.dependencies ++ rdd.dependencies.flatMap { dep => getAllDependencies(dep.rdd) } } - val rdd = newShuffleRDD + val rdd = newShuffleRDD() // Get all the shuffle dependencies val shuffleDeps = getAllDependencies(rdd) @@ -216,34 +233,34 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo (rdd, shuffleDeps) } - def randomRdd = { + private def randomRdd() = { val rdd: RDD[_] = Random.nextInt(3) match { - case 0 => newRDD - case 1 => newShuffleRDD - case 2 => newPairRDD.join(newPairRDD) + case 0 => newRDD() + case 1 => newShuffleRDD() + case 2 => newPairRDD.join(newPairRDD()) } if (Random.nextBoolean()) rdd.persist() rdd.count() rdd } - def randomBroadcast = { + private def randomBroadcast() = { sc.broadcast(Random.nextInt(Int.MaxValue)) } /** Run GC and make sure it actually has run */ - def runGC() { + private def runGC() { val weakRef = new WeakReference(new Object()) val startTime = System.currentTimeMillis System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. // Wait until a weak reference object has been GCed - while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + while (System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { System.gc() Thread.sleep(200) } } - def cleaner = sc.cleaner.get + private def cleaner = sc.cleaner.get } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index fdc83bc0a5f8e..4953d565ae83a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -155,19 +155,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { override def getPartitions: Array[Partition] = Array(onlySplit) override val getDependencies = List[Dependency[_]]() override def compute(split: Partition, context: TaskContext): Iterator[Int] = { - if (shouldFail) { - throw new Exception("injected failure") - } else { - Array(1, 2, 3, 4).iterator - } + throw new Exception("injected failure") } }.cache() val thrown = intercept[Exception]{ rdd.collect() } assert(thrown.getMessage.contains("injected failure")) - shouldFail = false - assert(rdd.collect().toList === List(1, 2, 3, 4)) } test("empty RDD") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 8bb5317cd2875..270f7e661045a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -20,31 +20,35 @@ package org.apache.spark.scheduler import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter -import org.apache.spark.LocalSparkContext -import org.apache.spark.Partition -import org.apache.spark.SparkContext -import org.apache.spark.TaskContext +import org.apache.spark._ import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { test("Calls executeOnCompleteCallbacks after failure") { - var completed = false + TaskContextSuite.completed = false sc = new SparkContext("local", "test") val rdd = new RDD[String](sc, List()) { override def getPartitions = Array[Partition](StubPartition(0)) override def compute(split: Partition, context: TaskContext) = { - context.addOnCompleteCallback(() => completed = true) + context.addOnCompleteCallback(() => TaskContextSuite.completed = true) sys.error("failed") } } - val func = (c: TaskContext, i: Iterator[String]) => i.next - val task = new ResultTask[String, String](0, rdd, func, 0, Seq(), 0) + val closureSerializer = SparkEnv.get.closureSerializer.newInstance() + val func = (c: TaskContext, i: Iterator[String]) => i.next() + val task = new ResultTask[String, String]( + 0, sc.broadcast(closureSerializer.serialize((rdd, func)).array), rdd.partitions(0), Seq(), 0) intercept[RuntimeException] { task.run(0) } - assert(completed === true) + assert(TaskContextSuite.completed === true) } +} - case class StubPartition(val index: Int) extends Partition +private object TaskContextSuite { + @volatile var completed = false } + +private case class StubPartition(index: Int) extends Partition diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index b52f81877d557..86a271eb67000 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matchers { + test("test LRU eviction of stages") { val conf = new SparkConf() conf.set("spark.ui.retainedStages", 5.toString) @@ -66,7 +67,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.updateShuffleReadMetrics(shuffleReadMetrics) var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - var task = new ShuffleMapTask(0, null, null, 0, null) + var task = new ShuffleMapTask(0) val taskType = Utils.getFormattedClassName(task) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) @@ -76,14 +77,14 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskInfo = new TaskInfo(1234L, 0, 1, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL, true) taskInfo.finishTime = 1 - task = new ShuffleMapTask(0, null, null, 0, null) + task = new ShuffleMapTask(0) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToData.size === 1) // finish this task, should get updated duration taskInfo = new TaskInfo(1235L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - task = new ShuffleMapTask(0, null, null, 0, null) + task = new ShuffleMapTask(0) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) .shuffleRead === 2000) @@ -91,7 +92,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc // finish this task, should get updated duration taskInfo = new TaskInfo(1236L, 0, 2, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - task = new ShuffleMapTask(0, null, null, 0, null) + task = new ShuffleMapTask(0) listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-2", fail()) .shuffleRead === 1000) @@ -103,7 +104,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val metrics = new TaskMetrics() val taskInfo = new TaskInfo(1234L, 0, 3, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 - val task = new ShuffleMapTask(0, null, null, 0, null) + val task = new ShuffleMapTask(0) val taskType = Utils.getFormattedClassName(task) // Go through all the failure cases to make sure we are counting them as failures. From 3bc3f1801e3347e02cbecdd8e941003430155da2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 09:28:53 -0700 Subject: [PATCH 0127/1492] [SPARK-2747] git diff --dirstat can miss sql changes and not run Hive tests dev/run-tests use "git diff --dirstat master" to check whether sql is changed. However, --dirstat won't show sql if sql's change is negligible (e.g. 1k loc change in core, and only 1 loc change in hive). We should use "git diff --name-only master" instead. Author: Reynold Xin Closes #1656 from rxin/hiveTest and squashes the following commits: f5eab9f [Reynold Xin] [SPARK-2747] git diff --dirstat can miss sql changes and not run Hive tests. --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 98ec969dc1b37..795d16a4d983d 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -37,7 +37,7 @@ JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..* # Partial solution for SPARK-1455. Only run Hive tests if there are sql changes. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - diffs=`git diff --dirstat master | awk '{ print $2; }' | grep "^sql/"` + diffs=`git diff --name-only master | grep "^sql/"` if [ -n "$diffs" ]; then echo "Detected changes in SQL. Will run Hive test suite." export _RUN_SQL_TESTS=true # exported for PySpark tests From e3d85b7e40073b05e2588583e9d8db11366c2f7b Mon Sep 17 00:00:00 2001 From: Naftali Harris Date: Wed, 30 Jul 2014 09:56:59 -0700 Subject: [PATCH 0128/1492] Avoid numerical instability This avoids basically doing 1 - 1, for example: ```python >>> from math import exp >>> margin = -40 >>> 1 - 1 / (1 + exp(margin)) 0.0 >>> exp(margin) / (1 + exp(margin)) 4.248354255291589e-18 >>> ``` Author: Naftali Harris Closes #1652 from naftaliharris/patch-2 and squashes the following commits: 0d55a9f [Naftali Harris] Avoid numerical instability --- python/pyspark/mllib/classification.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 9e28dfbb9145d..2bbb9c3fca315 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -66,7 +66,8 @@ def predict(self, x): if margin > 0: prob = 1 / (1 + exp(-margin)) else: - prob = 1 - 1 / (1 + exp(margin)) + exp_margin = exp(margin) + prob = exp_margin / (1 + exp_margin) return 1 if prob > 0.5 else 0 From fc47bb6967e0df40870413e09d37aa9b90248f43 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Wed, 30 Jul 2014 11:00:11 -0700 Subject: [PATCH 0129/1492] [SPARK-2544][MLLIB] Improve ALS algorithm resource usage Author: GuoQiang Li Author: witgo Closes #929 from witgo/improve_als and squashes the following commits: ea25033 [GuoQiang Li] checkpoint products 3,6,9 ... 154dccf [GuoQiang Li] checkpoint products only c5779ff [witgo] Improve ALS algorithm resource usage --- .../scala/org/apache/spark/mllib/recommendation/ALS.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 5356790cb5339..d208cfb917f3d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -255,6 +255,9 @@ class ALS private ( rank, lambda, alpha, YtY) previousProducts.unpersist() logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) + if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { + products.checkpoint() + } products.setName(s"products-$iter").persist() val XtX = Some(sc.broadcast(computeYtY(products))) val previousUsers = users @@ -268,6 +271,9 @@ class ALS private ( logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) products = updateFeatures(numProductBlocks, users, userOutLinks, productInLinks, rank, lambda, alpha, YtY = None) + if (sc.checkpointDir.isDefined && (iter % 3 == 0)) { + products.checkpoint() + } products.setName(s"products-$iter") logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) users = updateFeatures(numUserBlocks, products, productOutLinks, userInLinks, From ff511bacf223e19244f5f6114d60af7dcadeda4d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 11:45:24 -0700 Subject: [PATCH 0130/1492] [SPARK-2746] Set SBT_MAVEN_PROFILES only when it is not set explicitly by the user. Author: Reynold Xin Closes #1655 from rxin/SBT_MAVEN_PROFILES and squashes the following commits: b268c4b [Reynold Xin] [SPARK-2746] Set SBT_MAVEN_PROFILES only when it is not set explicitly by the user. --- dev/run-tests | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 795d16a4d983d..c95ef8a5743fc 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,7 +21,10 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd $FWDIR -export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" +if [ -z "$SBT_MAVEN_PROFILES" ]; then + export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" +fi +echo "SBT_MAVEN_PROFILES=\"$SBT_MAVEN_PROFILES\"" # Remove work directory rm -rf ./work From f2eb84fe737e6b06f5625640b209cf02f80732cf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 12:24:35 -0700 Subject: [PATCH 0131/1492] Wrap FWDIR in quotes. --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index c95ef8a5743fc..f2b523b996617 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -19,7 +19,7 @@ # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" -cd $FWDIR +cd "$FWDIR" if [ -z "$SBT_MAVEN_PROFILES" ]; then export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" From 95cf203936c412bc689bd2345fec7f9ad3648c25 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 12:33:42 -0700 Subject: [PATCH 0132/1492] Wrap FWDIR in quotes in dev/check-license. --- dev/check-license | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/check-license b/dev/check-license index fbd2dd465bb18..7a603bf0180ad 100755 --- a/dev/check-license +++ b/dev/check-license @@ -51,7 +51,7 @@ acquire_rat_jar () { # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" -cd $FWDIR +cd "$FWDIR" if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" From 0feb349ea07361f0363117404ffc9797c2c80dd1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 13:04:20 -0700 Subject: [PATCH 0133/1492] More wrapping FWDIR in quotes. --- dev/mima | 2 +- dev/run-tests-jenkins | 2 +- make-distribution.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/mima b/dev/mima index 7857294f61caf..4c3e65039b160 100755 --- a/dev/mima +++ b/dev/mima @@ -22,7 +22,7 @@ set -e # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" -cd $FWDIR +cd "$FWDIR" echo -e "q\n" | sbt/sbt oldDeps/update diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 8dda671e976ce..3076eb847b420 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -22,7 +22,7 @@ # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" -cd $FWDIR +cd "$FWDIR" COMMENTS_URL="https://api.github.com/repos/apache/spark/issues/$ghprbPullId/comments" diff --git a/make-distribution.sh b/make-distribution.sh index c08093f46b61f..0a3283ecec6f8 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -150,7 +150,7 @@ else fi # Build uber fat JAR -cd $FWDIR +cd "$FWDIR" export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" From 2248891a43d93cf2c05580211faf1e4f8dc7932d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 30 Jul 2014 13:11:09 -0700 Subject: [PATCH 0134/1492] [SQL] Fix compiling of catalyst docs. Author: Michael Armbrust Closes #1653 from marmbrus/fixDocs and squashes the following commits: 0aa1feb [Michael Armbrust] Fix compiling of catalyst docs. --- project/SparkBuild.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e2dab0f9f79ea..672343fbbed2e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -194,7 +194,10 @@ object Flume { object Catalyst { lazy val settings = Seq( - addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full)) + addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full), + // Quasiquotes break compiling scala doc... + // TODO: Investigate fixing this. + sources in (Compile, doc) ~= (_ filter (_.getName contains "codegen"))) } object SQL { From 437dc8c5b54f0dcf9564c1fb07e8dce9e771c8cd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 13:17:14 -0700 Subject: [PATCH 0135/1492] dev/check-license wrap folders in quotes. --- dev/check-license | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/check-license b/dev/check-license index 7a603bf0180ad..00bb20c133b7d 100755 --- a/dev/check-license +++ b/dev/check-license @@ -27,7 +27,7 @@ acquire_rat_jar () { if [[ ! -f "$rat_jar" ]]; then # Download rat launch jar if it hasn't been downloaded yet - if [ ! -f ${JAR} ]; then + if [ ! -f "$JAR" ]; then # Download printf "Attempting to fetch rat\n" JAR_DL=${JAR}.part @@ -40,10 +40,10 @@ acquire_rat_jar () { exit -1 fi fi - if [ ! -f ${JAR} ]; then - # We failed to download - printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" - exit -1 + if [ ! -f "$JAR" ]; then + # We failed to download + printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + exit -1 fi printf "Launching rat from ${JAR}\n" fi From 94d1f46fc43c0cb85125f757fb40db9271caf1f4 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Wed, 30 Jul 2014 13:19:05 -0700 Subject: [PATCH 0136/1492] [SPARK-2024] Add saveAsSequenceFile to PySpark JIRA issue: https://issues.apache.org/jira/browse/SPARK-2024 This PR is a followup to #455 and adds capabilities for saving PySpark RDDs using SequenceFile or any Hadoop OutputFormats. * Added RDD methods ```saveAsSequenceFile```, ```saveAsHadoopFile``` and ```saveAsHadoopDataset```, for both old and new MapReduce APIs. * Default converter for converting common data types to Writables. Users may specify custom converters to convert to desired data types. * No out-of-box support for reading/writing arrays, since ArrayWritable itself doesn't have a no-arg constructor for creating an empty instance upon reading. Users need to provide ArrayWritable subtypes. Custom converters for converting arrays to suitable ArrayWritable subtypes are also needed when writing. When reading, the default converter will convert any custom ArrayWritable subtypes to ```Object[]``` and they get pickled to Python tuples. * Added HBase and Cassandra output examples to show how custom output formats and converters can be used. cc MLnick mateiz ahirreddy pwendell Author: Kan Zhang Closes #1338 from kanzhang/SPARK-2024 and squashes the following commits: c01e3ef [Kan Zhang] [SPARK-2024] code formatting 6591e37 [Kan Zhang] [SPARK-2024] renaming pickled -> pickledRDD d998ad6 [Kan Zhang] [SPARK-2024] refectoring to get method params below 10 57a7a5e [Kan Zhang] [SPARK-2024] correcting typo 75ca5bd [Kan Zhang] [SPARK-2024] Better type checking for batch serialized RDD 0bdec55 [Kan Zhang] [SPARK-2024] Refactoring newly added tests 9f39ff4 [Kan Zhang] [SPARK-2024] Adding 2 saveAsHadoopDataset tests 0c134f3 [Kan Zhang] [SPARK-2024] Test refactoring and adding couple unbatched cases 7a176df [Kan Zhang] [SPARK-2024] Add saveAsSequenceFile to PySpark --- .../spark/api/python/PythonHadoopUtil.scala | 82 ++++- .../apache/spark/api/python/PythonRDD.scala | 247 +++++++++++--- .../apache/spark/api/python/SerDeUtil.scala | 61 +++- .../WriteInputFormatTestDataGenerator.scala | 69 +++- docs/programming-guide.md | 52 ++- .../src/main/python/cassandra_outputformat.py | 83 +++++ examples/src/main/python/hbase_inputformat.py | 3 +- .../src/main/python/hbase_outputformat.py | 65 ++++ .../CassandraConverters.scala | 24 +- .../pythonconverters/HBaseConverter.scala | 33 -- .../pythonconverters/HBaseConverters.scala | 70 ++++ python/pyspark/context.py | 51 ++- python/pyspark/rdd.py | 114 +++++++ python/pyspark/tests.py | 317 +++++++++++++++++- 14 files changed, 1085 insertions(+), 186 deletions(-) create mode 100644 examples/src/main/python/cassandra_outputformat.py create mode 100644 examples/src/main/python/hbase_outputformat.py delete mode 100644 examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index adaa1ef6cf9ff..f3b05e1243045 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -17,8 +17,9 @@ package org.apache.spark.api.python +import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD -import org.apache.spark.Logging +import org.apache.spark.{Logging, SerializableWritable, SparkException} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io._ import scala.util.{Failure, Success, Try} @@ -31,13 +32,14 @@ import org.apache.spark.annotation.Experimental * transformation code by overriding the convert method. */ @Experimental -trait Converter[T, U] extends Serializable { +trait Converter[T, + U] extends Serializable { def convert(obj: T): U } private[python] object Converter extends Logging { - def getInstance(converterClass: Option[String]): Converter[Any, Any] = { + def getInstance(converterClass: Option[String], + defaultConverter: Converter[Any, Any]): Converter[Any, Any] = { converterClass.map { cc => Try { val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]] @@ -49,7 +51,7 @@ private[python] object Converter extends Logging { logError(s"Failed to load converter: $cc") throw err } - }.getOrElse { new DefaultConverter } + }.getOrElse { defaultConverter } } } @@ -57,7 +59,9 @@ private[python] object Converter extends Logging { * A converter that handles conversion of common [[org.apache.hadoop.io.Writable]] objects. * Other objects are passed through without conversion. */ -private[python] class DefaultConverter extends Converter[Any, Any] { +private[python] class WritableToJavaConverter( + conf: Broadcast[SerializableWritable[Configuration]], + batchSize: Int) extends Converter[Any, Any] { /** * Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or @@ -72,17 +76,30 @@ private[python] class DefaultConverter extends Converter[Any, Any] { case fw: FloatWritable => fw.get() case t: Text => t.toString case bw: BooleanWritable => bw.get() - case byw: BytesWritable => byw.getBytes + case byw: BytesWritable => + val bytes = new Array[Byte](byw.getLength) + System.arraycopy(byw.getBytes(), 0, bytes, 0, byw.getLength) + bytes case n: NullWritable => null - case aw: ArrayWritable => aw.get().map(convertWritable(_)) - case mw: MapWritable => mapAsJavaMap(mw.map { case (k, v) => - (convertWritable(k), convertWritable(v)) - }.toMap) + case aw: ArrayWritable => + // Due to erasure, all arrays appear as Object[] and they get pickled to Python tuples. + // Since we can't determine element types for empty arrays, we will not attempt to + // convert to primitive arrays (which get pickled to Python arrays). Users may want + // write custom converters for arrays if they know the element types a priori. + aw.get().map(convertWritable(_)) + case mw: MapWritable => + val map = new java.util.HashMap[Any, Any]() + mw.foreach { case (k, v) => + map.put(convertWritable(k), convertWritable(v)) + } + map + case w: Writable => + if (batchSize > 1) WritableUtils.clone(w, conf.value.value) else w case other => other } } - def convert(obj: Any): Any = { + override def convert(obj: Any): Any = { obj match { case writable: Writable => convertWritable(writable) @@ -92,6 +109,47 @@ private[python] class DefaultConverter extends Converter[Any, Any] { } } +/** + * A converter that converts common types to [[org.apache.hadoop.io.Writable]]. Note that array + * types are not supported since the user needs to subclass [[org.apache.hadoop.io.ArrayWritable]] + * to set the type properly. See [[org.apache.spark.api.python.DoubleArrayWritable]] and + * [[org.apache.spark.api.python.DoubleArrayToWritableConverter]] for an example. They are used in + * PySpark RDD `saveAsNewAPIHadoopFile` doctest. + */ +private[python] class JavaToWritableConverter extends Converter[Any, Writable] { + + /** + * Converts common data types to [[org.apache.hadoop.io.Writable]]. Note that array types are not + * supported out-of-the-box. + */ + private def convertToWritable(obj: Any): Writable = { + import collection.JavaConversions._ + obj match { + case i: java.lang.Integer => new IntWritable(i) + case d: java.lang.Double => new DoubleWritable(d) + case l: java.lang.Long => new LongWritable(l) + case f: java.lang.Float => new FloatWritable(f) + case s: java.lang.String => new Text(s) + case b: java.lang.Boolean => new BooleanWritable(b) + case aob: Array[Byte] => new BytesWritable(aob) + case null => NullWritable.get() + case map: java.util.Map[_, _] => + val mapWritable = new MapWritable() + map.foreach { case (k, v) => + mapWritable.put(convertToWritable(k), convertToWritable(v)) + } + mapWritable + case other => throw new SparkException( + s"Data of type ${other.getClass.getName} cannot be used") + } + } + + override def convert(obj: Any): Writable = obj match { + case writable: Writable => writable + case other => convertToWritable(other) + } +} + /** Utilities for working with Python objects <-> Hadoop-related objects */ private[python] object PythonHadoopUtil { @@ -118,7 +176,7 @@ private[python] object PythonHadoopUtil { /** * Converts an RDD of key-value pairs, where key and/or value could be instances of - * [[org.apache.hadoop.io.Writable]], into an RDD[(K, V)] + * [[org.apache.hadoop.io.Writable]], into an RDD of base types, or vice versa. */ def convertRDD[K, V](rdd: RDD[(K, V)], keyConverter: Converter[Any, Any], diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index f551a59ee3fe8..a9d758bf998c3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -23,15 +23,18 @@ import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ +import scala.language.existentials import scala.reflect.ClassTag import scala.util.Try import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapred.{InputFormat, JobConf} -import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} +import org.apache.hadoop.io.compress.CompressionCodec +import org.apache.hadoop.mapred.{InputFormat, OutputFormat, JobConf} +import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, OutputFormat => NewOutputFormat} import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -365,19 +368,17 @@ private[spark] object PythonRDD extends Logging { valueClassMaybeNull: String, keyConverterClass: String, valueConverterClass: String, - minSplits: Int) = { + minSplits: Int, + batchSize: Int) = { val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") - implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] - implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] - val kc = kcm.runtimeClass.asInstanceOf[Class[K]] - val vc = vcm.runtimeClass.asInstanceOf[Class[V]] - + val kc = Class.forName(keyClass).asInstanceOf[Class[K]] + val vc = Class.forName(valueClass).asInstanceOf[Class[V]] val rdd = sc.sc.sequenceFile[K, V](path, kc, vc, minSplits) - val keyConverter = Converter.getInstance(Option(keyConverterClass)) - val valueConverter = Converter.getInstance(Option(valueConverterClass)) - val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) - JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(sc.hadoopConfiguration())) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } /** @@ -394,17 +395,16 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { - val conf = PythonHadoopUtil.mapToConf(confAsMap) - val baseConf = sc.hadoopConfiguration() - val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf) + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { + val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = newAPIHadoopRDDFromClassNames[K, V, F](sc, Some(path), inputFormatClass, keyClass, valueClass, mergedConf) - val keyConverter = Converter.getInstance(Option(keyConverterClass)) - val valueConverter = Converter.getInstance(Option(valueConverterClass)) - val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) - JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } /** @@ -421,15 +421,16 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val rdd = newAPIHadoopRDDFromClassNames[K, V, F](sc, None, inputFormatClass, keyClass, valueClass, conf) - val keyConverter = Converter.getInstance(Option(keyConverterClass)) - val valueConverter = Converter.getInstance(Option(valueConverterClass)) - val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) - JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } private def newAPIHadoopRDDFromClassNames[K, V, F <: NewInputFormat[K, V]]( @@ -439,18 +440,14 @@ private[spark] object PythonRDD extends Logging { keyClass: String, valueClass: String, conf: Configuration) = { - implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] - implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] - implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]] - val kc = kcm.runtimeClass.asInstanceOf[Class[K]] - val vc = vcm.runtimeClass.asInstanceOf[Class[V]] - val fc = fcm.runtimeClass.asInstanceOf[Class[F]] - val rdd = if (path.isDefined) { + val kc = Class.forName(keyClass).asInstanceOf[Class[K]] + val vc = Class.forName(valueClass).asInstanceOf[Class[V]] + val fc = Class.forName(inputFormatClass).asInstanceOf[Class[F]] + if (path.isDefined) { sc.sc.newAPIHadoopFile[K, V, F](path.get, fc, kc, vc, conf) } else { sc.sc.newAPIHadoopRDD[K, V, F](conf, fc, kc, vc) } - rdd } /** @@ -467,17 +464,16 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { - val conf = PythonHadoopUtil.mapToConf(confAsMap) - val baseConf = sc.hadoopConfiguration() - val mergedConf = PythonHadoopUtil.mergeConfs(baseConf, conf) + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { + val mergedConf = getMergedConf(confAsMap, sc.hadoopConfiguration()) val rdd = hadoopRDDFromClassNames[K, V, F](sc, Some(path), inputFormatClass, keyClass, valueClass, mergedConf) - val keyConverter = Converter.getInstance(Option(keyConverterClass)) - val valueConverter = Converter.getInstance(Option(valueConverterClass)) - val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) - JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } /** @@ -494,15 +490,16 @@ private[spark] object PythonRDD extends Logging { valueClass: String, keyConverterClass: String, valueConverterClass: String, - confAsMap: java.util.HashMap[String, String]) = { + confAsMap: java.util.HashMap[String, String], + batchSize: Int) = { val conf = PythonHadoopUtil.mapToConf(confAsMap) val rdd = hadoopRDDFromClassNames[K, V, F](sc, None, inputFormatClass, keyClass, valueClass, conf) - val keyConverter = Converter.getInstance(Option(keyConverterClass)) - val valueConverter = Converter.getInstance(Option(valueConverterClass)) - val converted = PythonHadoopUtil.convertRDD[K, V](rdd, keyConverter, valueConverter) - JavaRDD.fromRDD(SerDeUtil.rddToPython(converted)) + val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf)) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new WritableToJavaConverter(confBroadcasted, batchSize)) + JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } private def hadoopRDDFromClassNames[K, V, F <: InputFormat[K, V]]( @@ -512,18 +509,14 @@ private[spark] object PythonRDD extends Logging { keyClass: String, valueClass: String, conf: Configuration) = { - implicit val kcm = ClassTag(Class.forName(keyClass)).asInstanceOf[ClassTag[K]] - implicit val vcm = ClassTag(Class.forName(valueClass)).asInstanceOf[ClassTag[V]] - implicit val fcm = ClassTag(Class.forName(inputFormatClass)).asInstanceOf[ClassTag[F]] - val kc = kcm.runtimeClass.asInstanceOf[Class[K]] - val vc = vcm.runtimeClass.asInstanceOf[Class[V]] - val fc = fcm.runtimeClass.asInstanceOf[Class[F]] - val rdd = if (path.isDefined) { + val kc = Class.forName(keyClass).asInstanceOf[Class[K]] + val vc = Class.forName(valueClass).asInstanceOf[Class[V]] + val fc = Class.forName(inputFormatClass).asInstanceOf[Class[F]] + if (path.isDefined) { sc.sc.hadoopFile(path.get, fc, kc, vc) } else { sc.sc.hadoopRDD(new JobConf(conf), fc, kc, vc) } - rdd } def writeUTF(str: String, dataOut: DataOutputStream) { @@ -562,6 +555,152 @@ private[spark] object PythonRDD extends Logging { } } + private def getMergedConf(confAsMap: java.util.HashMap[String, String], + baseConf: Configuration): Configuration = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + PythonHadoopUtil.mergeConfs(baseConf, conf) + } + + private def inferKeyValueTypes[K, V](rdd: RDD[(K, V)], keyConverterClass: String = null, + valueConverterClass: String = null): (Class[_], Class[_]) = { + // Peek at an element to figure out key/value types. Since Writables are not serializable, + // we cannot call first() on the converted RDD. Instead, we call first() on the original RDD + // and then convert locally. + val (key, value) = rdd.first() + val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass, + new JavaToWritableConverter) + (kc.convert(key).getClass, vc.convert(value).getClass) + } + + private def getKeyValueTypes(keyClass: String, valueClass: String): + Option[(Class[_], Class[_])] = { + for { + k <- Option(keyClass) + v <- Option(valueClass) + } yield (Class.forName(k), Class.forName(v)) + } + + private def getKeyValueConverters(keyConverterClass: String, valueConverterClass: String, + defaultConverter: Converter[Any, Any]): (Converter[Any, Any], Converter[Any, Any]) = { + val keyConverter = Converter.getInstance(Option(keyConverterClass), defaultConverter) + val valueConverter = Converter.getInstance(Option(valueConverterClass), defaultConverter) + (keyConverter, valueConverter) + } + + /** + * Convert an RDD of key-value pairs from internal types to serializable types suitable for + * output, or vice versa. + */ + private def convertRDD[K, V](rdd: RDD[(K, V)], + keyConverterClass: String, + valueConverterClass: String, + defaultConverter: Converter[Any, Any]): RDD[(Any, Any)] = { + val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass, + defaultConverter) + PythonHadoopUtil.convertRDD(rdd, kc, vc) + } + + /** + * Output a Python RDD of key-value pairs as a Hadoop SequenceFile using the Writable types + * we convert from the RDD's key and value types. Note that keys and values can't be + * [[org.apache.hadoop.io.Writable]] types already, since Writables are not Java + * `Serializable` and we can't peek at them. The `path` can be on any Hadoop file system. + */ + def saveAsSequenceFile[K, V, C <: CompressionCodec]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + path: String, + compressionCodecClass: String) = { + saveAsHadoopFile( + pyRDD, batchSerialized, path, "org.apache.hadoop.mapred.SequenceFileOutputFormat", + null, null, null, null, new java.util.HashMap(), compressionCodecClass) + } + + /** + * Output a Python RDD of key-value pairs to any Hadoop file system, using old Hadoop + * `OutputFormat` in mapred package. Keys and values are converted to suitable output + * types using either user specified converters or, if not specified, + * [[org.apache.spark.api.python.JavaToWritableConverter]]. Post-conversion types + * `keyClass` and `valueClass` are automatically inferred if not specified. The passed-in + * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of + * this RDD. + */ + def saveAsHadoopFile[K, V, F <: OutputFormat[_, _], C <: CompressionCodec]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + path: String, + outputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String], + compressionCodecClass: String) = { + val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) + val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( + inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) + val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration) + val codec = Option(compressionCodecClass).map(Class.forName(_).asInstanceOf[Class[C]]) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new JavaToWritableConverter) + val fc = Class.forName(outputFormatClass).asInstanceOf[Class[F]] + converted.saveAsHadoopFile(path, kc, vc, fc, new JobConf(mergedConf), codec=codec) + } + + /** + * Output a Python RDD of key-value pairs to any Hadoop file system, using new Hadoop + * `OutputFormat` in mapreduce package. Keys and values are converted to suitable output + * types using either user specified converters or, if not specified, + * [[org.apache.spark.api.python.JavaToWritableConverter]]. Post-conversion types + * `keyClass` and `valueClass` are automatically inferred if not specified. The passed-in + * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of + * this RDD. + */ + def saveAsNewAPIHadoopFile[K, V, F <: NewOutputFormat[_, _]]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + path: String, + outputFormatClass: String, + keyClass: String, + valueClass: String, + keyConverterClass: String, + valueConverterClass: String, + confAsMap: java.util.HashMap[String, String]) = { + val rdd = SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized) + val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( + inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) + val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration) + val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, + new JavaToWritableConverter) + val fc = Class.forName(outputFormatClass).asInstanceOf[Class[F]] + converted.saveAsNewAPIHadoopFile(path, kc, vc, fc, mergedConf) + } + + /** + * Output a Python RDD of key-value pairs to any Hadoop file system, using a Hadoop conf + * converted from the passed-in `confAsMap`. The conf should set relevant output params ( + * e.g., output path, output format, etc), in the same way as it would be configured for + * a Hadoop MapReduce job. Both old and new Hadoop OutputFormat APIs are supported + * (mapred vs. mapreduce). Keys/values are converted for output using either user specified + * converters or, by default, [[org.apache.spark.api.python.JavaToWritableConverter]]. + */ + def saveAsHadoopDataset[K, V]( + pyRDD: JavaRDD[Array[Byte]], + batchSerialized: Boolean, + confAsMap: java.util.HashMap[String, String], + keyConverterClass: String, + valueConverterClass: String, + useNewAPI: Boolean) = { + val conf = PythonHadoopUtil.mapToConf(confAsMap) + val converted = convertRDD(SerDeUtil.pythonToPairRDD(pyRDD, batchSerialized), + keyConverterClass, valueConverterClass, new JavaToWritableConverter) + if (useNewAPI) { + converted.saveAsNewAPIHadoopDataset(conf) + } else { + converted.saveAsHadoopDataset(new JobConf(conf)) + } + } + /** * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by * PySpark. diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 9a012e7254901..efc9009c088a8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -17,13 +17,14 @@ package org.apache.spark.api.python -import scala.util.Try -import org.apache.spark.rdd.RDD -import org.apache.spark.Logging -import scala.util.Success +import scala.collection.JavaConversions._ import scala.util.Failure -import net.razorvine.pickle.Pickler +import scala.util.Try +import net.razorvine.pickle.{Unpickler, Pickler} + +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.rdd.RDD /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ private[python] object SerDeUtil extends Logging { @@ -65,20 +66,52 @@ private[python] object SerDeUtil extends Logging { * by PySpark. By default, if serialization fails, toString is called and the string * representation is serialized */ - def rddToPython(rdd: RDD[(Any, Any)]): RDD[Array[Byte]] = { + def pairRDDToPython(rdd: RDD[(Any, Any)], batchSize: Int): RDD[Array[Byte]] = { val (keyFailed, valueFailed) = checkPickle(rdd.first()) rdd.mapPartitions { iter => val pickle = new Pickler - iter.map { case (k, v) => - if (keyFailed && valueFailed) { - pickle.dumps(Array(k.toString, v.toString)) - } else if (keyFailed) { - pickle.dumps(Array(k.toString, v)) - } else if (!keyFailed && valueFailed) { - pickle.dumps(Array(k, v.toString)) + val cleaned = iter.map { case (k, v) => + val key = if (keyFailed) k.toString else k + val value = if (valueFailed) v.toString else v + Array[Any](key, value) + } + if (batchSize > 1) { + cleaned.grouped(batchSize).map(batched => pickle.dumps(seqAsJavaList(batched))) + } else { + cleaned.map(pickle.dumps(_)) + } + } + } + + /** + * Convert an RDD of serialized Python tuple (K, V) to RDD[(K, V)]. + */ + def pythonToPairRDD[K, V](pyRDD: RDD[Array[Byte]], batchSerialized: Boolean): RDD[(K, V)] = { + def isPair(obj: Any): Boolean = { + Option(obj.getClass.getComponentType).map(!_.isPrimitive).getOrElse(false) && + obj.asInstanceOf[Array[_]].length == 2 + } + pyRDD.mapPartitions { iter => + val unpickle = new Unpickler + val unpickled = + if (batchSerialized) { + iter.flatMap { batch => + unpickle.loads(batch) match { + case objs: java.util.List[_] => collectionAsScalaIterable(objs) + case other => throw new SparkException( + s"Unexpected type ${other.getClass.getName} for batch serialized Python RDD") + } + } } else { - pickle.dumps(Array(k, v)) + iter.map(unpickle.loads(_)) } + unpickled.map { + case obj if isPair(obj) => + // we only accept (K, V) + val arr = obj.asInstanceOf[Array[_]] + (arr.head.asInstanceOf[K], arr.last.asInstanceOf[V]) + case other => throw new SparkException( + s"RDD element of type ${other.getClass.getName} cannot be used") } } } diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index f0e3fb9aff5a0..d11db978b842e 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -17,15 +17,16 @@ package org.apache.spark.api.python -import org.apache.spark.SparkContext -import org.apache.hadoop.io._ -import scala.Array import java.io.{DataOutput, DataInput} +import java.nio.charset.Charset + +import org.apache.hadoop.io._ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.{SparkContext, SparkException} /** - * A class to test MsgPack serialization on the Scala side, that will be deserialized + * A class to test Pyrolite serialization on the Scala side, that will be deserialized * in Python * @param str * @param int @@ -54,7 +55,13 @@ case class TestWritable(var str: String, var int: Int, var double: Double) exten } } -class TestConverter extends Converter[Any, Any] { +private[python] class TestInputKeyConverter extends Converter[Any, Any] { + override def convert(obj: Any) = { + obj.asInstanceOf[IntWritable].get().toChar + } +} + +private[python] class TestInputValueConverter extends Converter[Any, Any] { import collection.JavaConversions._ override def convert(obj: Any) = { val m = obj.asInstanceOf[MapWritable] @@ -62,6 +69,38 @@ class TestConverter extends Converter[Any, Any] { } } +private[python] class TestOutputKeyConverter extends Converter[Any, Any] { + override def convert(obj: Any) = { + new Text(obj.asInstanceOf[Int].toString) + } +} + +private[python] class TestOutputValueConverter extends Converter[Any, Any] { + import collection.JavaConversions._ + override def convert(obj: Any) = { + new DoubleWritable(obj.asInstanceOf[java.util.Map[Double, _]].keySet().head) + } +} + +private[python] class DoubleArrayWritable extends ArrayWritable(classOf[DoubleWritable]) + +private[python] class DoubleArrayToWritableConverter extends Converter[Any, Writable] { + override def convert(obj: Any) = obj match { + case arr if arr.getClass.isArray && arr.getClass.getComponentType == classOf[Double] => + val daw = new DoubleArrayWritable + daw.set(arr.asInstanceOf[Array[Double]].map(new DoubleWritable(_))) + daw + case other => throw new SparkException(s"Data of type $other is not supported") + } +} + +private[python] class WritableToDoubleArrayConverter extends Converter[Any, Array[Double]] { + override def convert(obj: Any): Array[Double] = obj match { + case daw : DoubleArrayWritable => daw.get().map(_.asInstanceOf[DoubleWritable].get()) + case other => throw new SparkException(s"Data of type $other is not supported") + } +} + /** * This object contains method to generate SequenceFile test data and write it to a * given directory (probably a temp directory) @@ -97,7 +136,8 @@ object WriteInputFormatTestDataGenerator { sc.parallelize(intKeys).saveAsSequenceFile(intPath) sc.parallelize(intKeys.map{ case (k, v) => (k.toDouble, v) }).saveAsSequenceFile(doublePath) sc.parallelize(intKeys.map{ case (k, v) => (k.toString, v) }).saveAsSequenceFile(textPath) - sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes) }).saveAsSequenceFile(bytesPath) + sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(Charset.forName("UTF-8"))) } + ).saveAsSequenceFile(bytesPath) val bools = Seq((1, true), (2, true), (2, false), (3, true), (2, false), (1, false)) sc.parallelize(bools).saveAsSequenceFile(boolPath) sc.parallelize(intKeys).map{ case (k, v) => @@ -106,19 +146,20 @@ object WriteInputFormatTestDataGenerator { // Create test data for ArrayWritable val data = Seq( - (1, Array(1.0, 2.0, 3.0)), + (1, Array()), (2, Array(3.0, 4.0, 5.0)), (3, Array(4.0, 5.0, 6.0)) ) sc.parallelize(data, numSlices = 2) .map{ case (k, v) => - (new IntWritable(k), new ArrayWritable(classOf[DoubleWritable], v.map(new DoubleWritable(_)))) - }.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, ArrayWritable]](arrPath) + val va = new DoubleArrayWritable + va.set(v.map(new DoubleWritable(_))) + (new IntWritable(k), va) + }.saveAsNewAPIHadoopFile[SequenceFileOutputFormat[IntWritable, DoubleArrayWritable]](arrPath) // Create test data for MapWritable, with keys DoubleWritable and values Text val mapData = Seq( - (1, Map(2.0 -> "aa")), - (2, Map(3.0 -> "bb")), + (1, Map()), (2, Map(1.0 -> "cc")), (3, Map(2.0 -> "dd")), (2, Map(1.0 -> "aa")), @@ -126,9 +167,9 @@ object WriteInputFormatTestDataGenerator { ) sc.parallelize(mapData, numSlices = 2).map{ case (i, m) => val mw = new MapWritable() - val k = m.keys.head - val v = m.values.head - mw.put(new DoubleWritable(k), new Text(v)) + m.foreach { case (k, v) => + mw.put(new DoubleWritable(k), new Text(v)) + } (new IntWritable(i), mw) }.saveAsSequenceFile(mapPath) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 90c69713019f2..a88bf27add883 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -383,16 +383,16 @@ Apart from text files, Spark's Python API also supports several other data forma * `RDD.saveAsPickleFile` and `SparkContext.pickleFile` support saving an RDD in a simple format consisting of pickled Python objects. Batching is used on pickle serialization, with default batch size 10. -* Details on reading `SequenceFile` and arbitrary Hadoop `InputFormat` are given below. - -### SequenceFile and Hadoop InputFormats +* SequenceFile and Hadoop Input/Output Formats **Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on SparkSQL, in which case SparkSQL is the preferred approach. -#### Writable Support +**Writable Support** -PySpark SequenceFile support loads an RDD within Java, and pickles the resulting Java objects using -[Pyrolite](https://github.com/irmen/Pyrolite/). The following Writables are automatically converted: +PySpark SequenceFile support loads an RDD of key-value pairs within Java, converts Writables to base Java types, and pickles the +resulting Java objects using [Pyrolite](https://github.com/irmen/Pyrolite/). When saving an RDD of key-value pairs to SequenceFile, +PySpark does the reverse. It unpickles Python objects into Java objects and then converts them to Writables. The following +Writables are automatically converted: @@ -403,32 +403,30 @@ PySpark SequenceFile support loads an RDD within Java, and pickles the resulting - - -
Writable TypePython Type
BooleanWritablebool
BytesWritablebytearray
NullWritableNone
ArrayWritablelist of primitives, or tuple of objects
MapWritabledict
Custom Class conforming to Java Bean conventionsdict of public properties (via JavaBean getters and setters) + __class__ for the class type
-#### Loading SequenceFiles +Arrays are not handled out-of-the-box. Users need to specify custom `ArrayWritable` subtypes when reading or writing. When writing, +users also need to specify custom converters that convert arrays to custom `ArrayWritable` subtypes. When reading, the default +converter will convert custom `ArrayWritable` subtypes to Java `Object[]`, which then get pickled to Python tuples. To get +Python `array.array` for arrays of primitive types, users need to specify custom converters. + +**Saving and Loading SequenceFiles** -Similarly to text files, SequenceFiles can be loaded by specifying the path. The key and value +Similarly to text files, SequenceFiles can be saved and loaded by specifying the path. The key and value classes can be specified, but for standard Writables this is not required. {% highlight python %} ->>> rdd = sc.sequenceFile("path/to/sequencefile/of/doubles") ->>> rdd.collect() # this example has DoubleWritable keys and Text values -[(1.0, u'aa'), - (2.0, u'bb'), - (2.0, u'aa'), - (3.0, u'cc'), - (2.0, u'bb'), - (1.0, u'aa')] +>>> rdd = sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x )) +>>> rdd.saveAsSequenceFile("path/to/file") +>>> sorted(sc.sequenceFile("path/to/file").collect()) +[(1, u'a'), (2, u'aa'), (3, u'aaa')] {% endhighlight %} -#### Loading Other Hadoop InputFormats +**Saving and Loading Other Hadoop Input/Output Formats** -PySpark can also read any Hadoop InputFormat, for both 'new' and 'old' Hadoop APIs. If required, -a Hadoop configuration can be passed in as a Python dict. Here is an example using the +PySpark can also read any Hadoop InputFormat or write any Hadoop OutputFormat, for both 'new' and 'old' Hadoop MapReduce APIs. +If required, a Hadoop configuration can be passed in as a Python dict. Here is an example using the Elasticsearch ESInputFormat: {% highlight python %} @@ -447,8 +445,7 @@ Note that, if the InputFormat simply depends on a Hadoop configuration and/or in the key and value classes can easily be converted according to the above table, then this approach should work well for such cases. -If you have custom serialized binary data (such as loading data from Cassandra / HBase) or custom -classes that don't conform to the JavaBean requirements, then you will first need to +If you have custom serialized binary data (such as loading data from Cassandra / HBase), then you will first need to transform that data on the Scala/Java side to something which can be handled by Pyrolite's pickler. A [Converter](api/scala/index.html#org.apache.spark.api.python.Converter) trait is provided for this. Simply extend this trait and implement your transformation code in the ```convert``` @@ -456,11 +453,8 @@ method. Remember to ensure that this class, along with any dependencies required classpath. See the [Python examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python) and -the [Converter examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/pythonconverters) -for examples of using HBase and Cassandra ```InputFormat```. - -Future support for writing data out as ```SequenceFileOutputFormat``` and other ```OutputFormats```, -is forthcoming. +the [Converter examples]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/pythonconverters) +for examples of using Cassandra / HBase ```InputFormat``` and ```OutputFormat``` with custom converters. diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py new file mode 100644 index 0000000000000..1dfbf98604425 --- /dev/null +++ b/examples/src/main/python/cassandra_outputformat.py @@ -0,0 +1,83 @@ +# +# 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. +# + +import sys + +from pyspark import SparkContext + +""" +Create data in Cassandra fist +(following: https://wiki.apache.org/cassandra/GettingStarted) + +cqlsh> CREATE KEYSPACE test + ... WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }; +cqlsh> use test; +cqlsh:test> CREATE TABLE users ( + ... user_id int PRIMARY KEY, + ... fname text, + ... lname text + ... ); + +> cassandra_outputformat test users 1745 john smith +> cassandra_outputformat test users 1744 john doe +> cassandra_outputformat test users 1746 john smith + +cqlsh:test> SELECT * FROM users; + + user_id | fname | lname +---------+-------+------- + 1745 | john | smith + 1744 | john | doe + 1746 | john | smith +""" +if __name__ == "__main__": + if len(sys.argv) != 7: + print >> sys.stderr, """ + Usage: cassandra_outputformat + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/cassandra_outputformat.py + Assumes you have created the following table in Cassandra already, + running on , in . + + cqlsh:> CREATE TABLE ( + ... user_id int PRIMARY KEY, + ... fname text, + ... lname text + ... ); + """ + exit(-1) + + host = sys.argv[1] + keyspace = sys.argv[2] + cf = sys.argv[3] + sc = SparkContext(appName="CassandraOutputFormat") + + conf = {"cassandra.output.thrift.address":host, + "cassandra.output.thrift.port":"9160", + "cassandra.output.keyspace":keyspace, + "cassandra.output.partitioner.class":"Murmur3Partitioner", + "cassandra.output.cql":"UPDATE " + keyspace + "." + cf + " SET fname = ?, lname = ?", + "mapreduce.output.basename":cf, + "mapreduce.outputformat.class":"org.apache.cassandra.hadoop.cql3.CqlOutputFormat", + "mapreduce.job.output.key.class":"java.util.Map", + "mapreduce.job.output.value.class":"java.util.List"} + key = {"user_id" : int(sys.argv[4])} + sc.parallelize([(key, sys.argv[5:])]).saveAsNewAPIHadoopDataset( + conf=conf, + keyConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLKeyConverter", + valueConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLValueConverter") diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index 3289d9880a0f5..c9fa8e171c2a1 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -65,7 +65,8 @@ "org.apache.hadoop.hbase.mapreduce.TableInputFormat", "org.apache.hadoop.hbase.io.ImmutableBytesWritable", "org.apache.hadoop.hbase.client.Result", - valueConverter="org.apache.spark.examples.pythonconverters.HBaseConverter", + keyConverter="org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter", + valueConverter="org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter", conf=conf) output = hbase_rdd.collect() for (k, v) in output: diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py new file mode 100644 index 0000000000000..5e11548fd13f7 --- /dev/null +++ b/examples/src/main/python/hbase_outputformat.py @@ -0,0 +1,65 @@ +# +# 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. +# + +import sys + +from pyspark import SparkContext + +""" +Create test table in HBase first: + +hbase(main):001:0> create 'test', 'f1' +0 row(s) in 0.7840 seconds + +> hbase_outputformat test row1 f1 q1 value1 +> hbase_outputformat test row2 f1 q1 value2 +> hbase_outputformat test row3 f1 q1 value3 +> hbase_outputformat test row4 f1 q1 value4 + +hbase(main):002:0> scan 'test' +ROW COLUMN+CELL + row1 column=f1:q1, timestamp=1405659615726, value=value1 + row2 column=f1:q1, timestamp=1405659626803, value=value2 + row3 column=f1:q1, timestamp=1405659640106, value=value3 + row4 column=f1:q1, timestamp=1405659650292, value=value4 +4 row(s) in 0.0780 seconds +""" +if __name__ == "__main__": + if len(sys.argv) != 7: + print >> sys.stderr, """ + Usage: hbase_outputformat + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/hbase_outputformat.py + Assumes you have created
with column family in HBase running on already + """ + exit(-1) + + host = sys.argv[1] + table = sys.argv[2] + sc = SparkContext(appName="HBaseOutputFormat") + + conf = {"hbase.zookeeper.quorum": host, + "hbase.mapred.outputtable": table, + "mapreduce.outputformat.class" : "org.apache.hadoop.hbase.mapreduce.TableOutputFormat", + "mapreduce.job.output.key.class" : "org.apache.hadoop.hbase.io.ImmutableBytesWritable", + "mapreduce.job.output.value.class" : "org.apache.hadoop.io.Writable"} + + sc.parallelize([sys.argv[3:]]).map(lambda x: (x[0], x)).saveAsNewAPIHadoopDataset( + conf=conf, + keyConverter="org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter", + valueConverter="org.apache.spark.examples.pythonconverters.StringListToPutConverter") diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala index 29a65c7a5f295..83feb5703b908 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/CassandraConverters.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.pythonconverters import org.apache.spark.api.python.Converter import java.nio.ByteBuffer import org.apache.cassandra.utils.ByteBufferUtil -import collection.JavaConversions.{mapAsJavaMap, mapAsScalaMap} +import collection.JavaConversions._ /** @@ -44,3 +44,25 @@ class CassandraCQLValueConverter extends Converter[Any, java.util.Map[String, St mapAsJavaMap(result.mapValues(bb => ByteBufferUtil.string(bb))) } } + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * Map[String, Int] to Cassandra key + */ +class ToCassandraCQLKeyConverter extends Converter[Any, java.util.Map[String, ByteBuffer]] { + override def convert(obj: Any): java.util.Map[String, ByteBuffer] = { + val input = obj.asInstanceOf[java.util.Map[String, Int]] + mapAsJavaMap(input.mapValues(i => ByteBufferUtil.bytes(i))) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * List[String] to Cassandra value + */ +class ToCassandraCQLValueConverter extends Converter[Any, java.util.List[ByteBuffer]] { + override def convert(obj: Any): java.util.List[ByteBuffer] = { + val input = obj.asInstanceOf[java.util.List[String]] + seqAsJavaList(input.map(s => ByteBufferUtil.bytes(s))) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala deleted file mode 100644 index 42ae960bd64a1..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverter.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.examples.pythonconverters - -import org.apache.spark.api.python.Converter -import org.apache.hadoop.hbase.client.Result -import org.apache.hadoop.hbase.util.Bytes - -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts a HBase Result - * to a String - */ -class HBaseConverter extends Converter[Any, String] { - override def convert(obj: Any): String = { - val result = obj.asInstanceOf[Result] - Bytes.toStringBinary(result.value()) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala new file mode 100644 index 0000000000000..273bee0a8b30f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/HBaseConverters.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.pythonconverters + +import scala.collection.JavaConversions._ + +import org.apache.spark.api.python.Converter +import org.apache.hadoop.hbase.client.{Put, Result} +import org.apache.hadoop.hbase.io.ImmutableBytesWritable +import org.apache.hadoop.hbase.util.Bytes + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts an + * HBase Result to a String + */ +class HBaseResultToStringConverter extends Converter[Any, String] { + override def convert(obj: Any): String = { + val result = obj.asInstanceOf[Result] + Bytes.toStringBinary(result.value()) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts an + * ImmutableBytesWritable to a String + */ +class ImmutableBytesWritableToStringConverter extends Converter[Any, String] { + override def convert(obj: Any): String = { + val key = obj.asInstanceOf[ImmutableBytesWritable] + Bytes.toStringBinary(key.get()) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * String to an ImmutableBytesWritable + */ +class StringToImmutableBytesWritableConverter extends Converter[Any, ImmutableBytesWritable] { + override def convert(obj: Any): ImmutableBytesWritable = { + val bytes = Bytes.toBytes(obj.asInstanceOf[String]) + new ImmutableBytesWritable(bytes) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts a + * list of Strings to HBase Put + */ +class StringListToPutConverter extends Converter[Any, Put] { + override def convert(obj: Any): Put = { + val output = obj.asInstanceOf[java.util.ArrayList[String]].map(Bytes.toBytes(_)).toArray + val put = new Put(output(0)) + put.add(output(1), output(2), output(3)) + } +} diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 830a6ee03f2a6..7b0f8d83aedc5 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -60,6 +60,7 @@ class SparkContext(object): _active_spark_context = None _lock = Lock() _python_includes = None # zip and egg files that need to be added to PYTHONPATH + _default_batch_size_for_serialized_input = 10 def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, @@ -378,7 +379,7 @@ def _dictToJavaMap(self, d): return jm def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, - valueConverter=None, minSplits=None): + valueConverter=None, minSplits=None, batchSize=None): """ Read a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -398,14 +399,18 @@ def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, @param valueConverter: @param minSplits: minimum splits in dataset (default min(2, sc.defaultParallelism)) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ minSplits = minSplits or min(self.defaultParallelism, 2) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.sequenceFile(self._jsc, path, keyClass, valueClass, - keyConverter, valueConverter, minSplits) - return RDD(jrdd, self, PickleSerializer()) + keyConverter, valueConverter, minSplits, batchSize) + return RDD(jrdd, self, ser) def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -425,14 +430,18 @@ def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConv @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopFile(self._jsc, path, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + valueClass, keyConverter, valueConverter, jconf, batchSize) + return RDD(jrdd, self, ser) def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -449,14 +458,18 @@ def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=N @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopRDD(self._jsc, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + valueClass, keyConverter, valueConverter, jconf, batchSize) + return RDD(jrdd, self, ser) def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -476,14 +489,18 @@ def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter= @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.hadoopFile(self._jsc, path, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + valueClass, keyConverter, valueConverter, jconf, batchSize) + return RDD(jrdd, self, ser) def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None): + valueConverter=None, conf=None, batchSize=None): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -500,11 +517,15 @@ def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, @param valueConverter: (None by default) @param conf: Hadoop configuration, passed in as a dict (None by default) + @param batchSize: The number of Python objects represented as a single + Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) + batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) + ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.hadoopRDD(self._jsc, inputFormatClass, keyClass, valueClass, - keyConverter, valueConverter, jconf) - return RDD(jrdd, self, PickleSerializer()) + keyConverter, valueConverter, jconf, batchSize) + return RDD(jrdd, self, ser) def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index b84d976114f0d..e8fcc900efb24 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -231,6 +231,13 @@ def __init__(self, jrdd, ctx, jrdd_deserializer): self._jrdd_deserializer = jrdd_deserializer self._id = jrdd.id() + def _toPickleSerialization(self): + if (self._jrdd_deserializer == PickleSerializer() or + self._jrdd_deserializer == BatchedSerializer(PickleSerializer())): + return self + else: + return self._reserialize(BatchedSerializer(PickleSerializer(), 10)) + def id(self): """ A unique ID for this RDD (within its SparkContext). @@ -1030,6 +1037,113 @@ def first(self): """ return self.take(1)[0] + def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are + converted for output using either user specified converters or, by default, + L{org.apache.spark.api.python.JavaToWritableConverter}. + + @param conf: Hadoop job configuration, passed in as a dict + @param keyConverter: (None by default) + @param valueConverter: (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf, + keyConverter, valueConverter, True) + + def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None, + keyConverter=None, valueConverter=None, conf=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the new Hadoop OutputFormat API (mapreduce package). Key and value types + will be inferred if not specified. Keys and values are converted for output using either + user specified converters or L{org.apache.spark.api.python.JavaToWritableConverter}. The + C{conf} is applied on top of the base Hadoop conf associated with the SparkContext + of this RDD to create a merged Hadoop MapReduce job configuration for saving the data. + + @param path: path to Hadoop file + @param outputFormatClass: fully qualified classname of Hadoop OutputFormat + (e.g. "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") + @param keyClass: fully qualified classname of key Writable class + (e.g. "org.apache.hadoop.io.IntWritable", None by default) + @param valueClass: fully qualified classname of value Writable class + (e.g. "org.apache.hadoop.io.Text", None by default) + @param keyConverter: (None by default) + @param valueConverter: (None by default) + @param conf: Hadoop job configuration, passed in as a dict (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsNewAPIHadoopFile(pickledRDD._jrdd, batched, path, + outputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf) + + def saveAsHadoopDataset(self, conf, keyConverter=None, valueConverter=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the old Hadoop OutputFormat API (mapred package). Keys/values are + converted for output using either user specified converters or, by default, + L{org.apache.spark.api.python.JavaToWritableConverter}. + + @param conf: Hadoop job configuration, passed in as a dict + @param keyConverter: (None by default) + @param valueConverter: (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf, + keyConverter, valueConverter, False) + + def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None, + keyConverter=None, valueConverter=None, conf=None, + compressionCodecClass=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the old Hadoop OutputFormat API (mapred package). Key and value types + will be inferred if not specified. Keys and values are converted for output using either + user specified converters or L{org.apache.spark.api.python.JavaToWritableConverter}. The + C{conf} is applied on top of the base Hadoop conf associated with the SparkContext + of this RDD to create a merged Hadoop MapReduce job configuration for saving the data. + + @param path: path to Hadoop file + @param outputFormatClass: fully qualified classname of Hadoop OutputFormat + (e.g. "org.apache.hadoop.mapred.SequenceFileOutputFormat") + @param keyClass: fully qualified classname of key Writable class + (e.g. "org.apache.hadoop.io.IntWritable", None by default) + @param valueClass: fully qualified classname of value Writable class + (e.g. "org.apache.hadoop.io.Text", None by default) + @param keyConverter: (None by default) + @param valueConverter: (None by default) + @param conf: (None by default) + @param compressionCodecClass: (None by default) + """ + jconf = self.ctx._dictToJavaMap(conf) + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsHadoopFile(pickledRDD._jrdd, batched, path, + outputFormatClass, keyClass, valueClass, keyConverter, valueConverter, + jconf, compressionCodecClass) + + def saveAsSequenceFile(self, path, compressionCodecClass=None): + """ + Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file + system, using the L{org.apache.hadoop.io.Writable} types that we convert from the + RDD's key and value types. The mechanism is as follows: + 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects. + 2. Keys and values of this Java RDD are converted to Writables and written out. + + @param path: path to sequence file + @param compressionCodecClass: (None by default) + """ + pickledRDD = self._toPickleSerialization() + batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) + self.ctx._jvm.PythonRDD.saveAsSequenceFile(pickledRDD._jrdd, batched, + path, compressionCodecClass) + def saveAsPickleFile(self, path, batchSize=10): """ Save this RDD as a SequenceFile of serialized objects. The serializer diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 8486c8595b5a4..c29deb9574ea2 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -19,6 +19,7 @@ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. """ +from array import array from fileinput import input from glob import glob import os @@ -327,6 +328,17 @@ def test_sequencefiles(self): ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')] self.assertEqual(doubles, ed) + bytes = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbytes/", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.BytesWritable").collect()) + ebs = [(1, bytearray('aa', 'utf-8')), + (1, bytearray('aa', 'utf-8')), + (2, bytearray('aa', 'utf-8')), + (2, bytearray('bb', 'utf-8')), + (2, bytearray('bb', 'utf-8')), + (3, bytearray('cc', 'utf-8'))] + self.assertEqual(bytes, ebs) + text = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sftext/", "org.apache.hadoop.io.Text", "org.apache.hadoop.io.Text").collect()) @@ -353,14 +365,34 @@ def test_sequencefiles(self): maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable").collect()) - em = [(1, {2.0: u'aa'}), + em = [(1, {}), (1, {3.0: u'bb'}), (2, {1.0: u'aa'}), (2, {1.0: u'cc'}), - (2, {3.0: u'bb'}), (3, {2.0: u'dd'})] self.assertEqual(maps, em) + # arrays get pickled to tuples by default + tuples = sorted(self.sc.sequenceFile( + basepath + "/sftestdata/sfarray/", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable").collect()) + et = [(1, ()), + (2, (3.0, 4.0, 5.0)), + (3, (4.0, 5.0, 6.0))] + self.assertEqual(tuples, et) + + # with custom converters, primitive arrays can stay as arrays + arrays = sorted(self.sc.sequenceFile( + basepath + "/sftestdata/sfarray/", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect()) + ea = [(1, array('d')), + (2, array('d', [3.0, 4.0, 5.0])), + (3, array('d', [4.0, 5.0, 6.0]))] + self.assertEqual(arrays, ea) + clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", "org.apache.hadoop.io.Text", "org.apache.spark.api.python.TestWritable").collect()) @@ -369,6 +401,12 @@ def test_sequencefiles(self): u'double': 54.0, u'int': 123, u'str': u'test1'}) self.assertEqual(clazz[0], ec) + unbatched_clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", + "org.apache.hadoop.io.Text", + "org.apache.spark.api.python.TestWritable", + batchSize=1).collect()) + self.assertEqual(unbatched_clazz[0], ec) + def test_oldhadoop(self): basepath = self.tempdir.name ints = sorted(self.sc.hadoopFile(basepath + "/sftestdata/sfint/", @@ -379,10 +417,11 @@ def test_oldhadoop(self): self.assertEqual(ints, ei) hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt") - hello = self.sc.hadoopFile(hellopath, - "org.apache.hadoop.mapred.TextInputFormat", - "org.apache.hadoop.io.LongWritable", - "org.apache.hadoop.io.Text").collect() + oldconf = {"mapred.input.dir" : hellopath} + hello = self.sc.hadoopRDD("org.apache.hadoop.mapred.TextInputFormat", + "org.apache.hadoop.io.LongWritable", + "org.apache.hadoop.io.Text", + conf=oldconf).collect() result = [(0, u'Hello World!')] self.assertEqual(hello, result) @@ -397,10 +436,11 @@ def test_newhadoop(self): self.assertEqual(ints, ei) hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt") - hello = self.sc.newAPIHadoopFile(hellopath, - "org.apache.hadoop.mapreduce.lib.input.TextInputFormat", - "org.apache.hadoop.io.LongWritable", - "org.apache.hadoop.io.Text").collect() + newconf = {"mapred.input.dir" : hellopath} + hello = self.sc.newAPIHadoopRDD("org.apache.hadoop.mapreduce.lib.input.TextInputFormat", + "org.apache.hadoop.io.LongWritable", + "org.apache.hadoop.io.Text", + conf=newconf).collect() result = [(0, u'Hello World!')] self.assertEqual(hello, result) @@ -435,16 +475,267 @@ def test_bad_inputs(self): "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.Text")) - def test_converter(self): + def test_converters(self): + # use of custom converters basepath = self.tempdir.name maps = sorted(self.sc.sequenceFile( basepath + "/sftestdata/sfmap/", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable", - valueConverter="org.apache.spark.api.python.TestConverter").collect()) - em = [(1, [2.0]), (1, [3.0]), (2, [1.0]), (2, [1.0]), (2, [3.0]), (3, [2.0])] + keyConverter="org.apache.spark.api.python.TestInputKeyConverter", + valueConverter="org.apache.spark.api.python.TestInputValueConverter").collect()) + em = [(u'\x01', []), + (u'\x01', [3.0]), + (u'\x02', [1.0]), + (u'\x02', [1.0]), + (u'\x03', [2.0])] + self.assertEqual(maps, em) + +class TestOutputFormat(PySparkTestCase): + + def setUp(self): + PySparkTestCase.setUp(self) + self.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(self.tempdir.name) + + def tearDown(self): + PySparkTestCase.tearDown(self) + shutil.rmtree(self.tempdir.name, ignore_errors=True) + + def test_sequencefiles(self): + basepath = self.tempdir.name + ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] + self.sc.parallelize(ei).saveAsSequenceFile(basepath + "/sfint/") + ints = sorted(self.sc.sequenceFile(basepath + "/sfint/").collect()) + self.assertEqual(ints, ei) + + ed = [(1.0, u'aa'), (1.0, u'aa'), (2.0, u'aa'), (2.0, u'bb'), (2.0, u'bb'), (3.0, u'cc')] + self.sc.parallelize(ed).saveAsSequenceFile(basepath + "/sfdouble/") + doubles = sorted(self.sc.sequenceFile(basepath + "/sfdouble/").collect()) + self.assertEqual(doubles, ed) + + ebs = [(1, bytearray(b'\x00\x07spam\x08')), (2, bytearray(b'\x00\x07spam\x08'))] + self.sc.parallelize(ebs).saveAsSequenceFile(basepath + "/sfbytes/") + bytes = sorted(self.sc.sequenceFile(basepath + "/sfbytes/").collect()) + self.assertEqual(bytes, ebs) + + et = [(u'1', u'aa'), + (u'2', u'bb'), + (u'3', u'cc')] + self.sc.parallelize(et).saveAsSequenceFile(basepath + "/sftext/") + text = sorted(self.sc.sequenceFile(basepath + "/sftext/").collect()) + self.assertEqual(text, et) + + eb = [(1, False), (1, True), (2, False), (2, False), (2, True), (3, True)] + self.sc.parallelize(eb).saveAsSequenceFile(basepath + "/sfbool/") + bools = sorted(self.sc.sequenceFile(basepath + "/sfbool/").collect()) + self.assertEqual(bools, eb) + + en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] + self.sc.parallelize(en).saveAsSequenceFile(basepath + "/sfnull/") + nulls = sorted(self.sc.sequenceFile(basepath + "/sfnull/").collect()) + self.assertEqual(nulls, en) + + em = [(1, {}), + (1, {3.0: u'bb'}), + (2, {1.0: u'aa'}), + (2, {1.0: u'cc'}), + (3, {2.0: u'dd'})] + self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/") + maps = sorted(self.sc.sequenceFile(basepath + "/sfmap/").collect()) self.assertEqual(maps, em) + def test_oldhadoop(self): + basepath = self.tempdir.name + dict_data = [(1, {}), + (1, {"row1" : 1.0}), + (2, {"row2" : 2.0})] + self.sc.parallelize(dict_data).saveAsHadoopFile( + basepath + "/oldhadoop/", + "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable") + result = sorted(self.sc.hadoopFile( + basepath + "/oldhadoop/", + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable").collect()) + self.assertEqual(result, dict_data) + + conf = { + "mapred.output.format.class" : "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class" : "org.apache.hadoop.io.MapWritable", + "mapred.output.dir" : basepath + "/olddataset/"} + self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) + input_conf = {"mapred.input.dir" : basepath + "/olddataset/"} + old_dataset = sorted(self.sc.hadoopRDD( + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable", + conf=input_conf).collect()) + self.assertEqual(old_dataset, dict_data) + + def test_newhadoop(self): + basepath = self.tempdir.name + # use custom ArrayWritable types and converters to handle arrays + array_data = [(1, array('d')), + (1, array('d', [1.0, 2.0, 3.0])), + (2, array('d', [3.0, 4.0, 5.0]))] + self.sc.parallelize(array_data).saveAsNewAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter") + result = sorted(self.sc.newAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect()) + self.assertEqual(result, array_data) + + conf = {"mapreduce.outputformat.class" : + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class" : "org.apache.spark.api.python.DoubleArrayWritable", + "mapred.output.dir" : basepath + "/newdataset/"} + self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(conf, + valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter") + input_conf = {"mapred.input.dir" : basepath + "/newdataset/"} + new_dataset = sorted(self.sc.newAPIHadoopRDD( + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.spark.api.python.DoubleArrayWritable", + valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter", + conf=input_conf).collect()) + self.assertEqual(new_dataset, array_data) + + def test_newolderror(self): + basepath = self.tempdir.name + rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x )) + self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( + basepath + "/newolderror/saveAsHadoopFile/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")) + self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile( + basepath + "/newolderror/saveAsNewAPIHadoopFile/", + "org.apache.hadoop.mapred.SequenceFileOutputFormat")) + + def test_bad_inputs(self): + basepath = self.tempdir.name + rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x )) + self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( + basepath + "/badinputs/saveAsHadoopFile/", + "org.apache.hadoop.mapred.NotValidOutputFormat")) + self.assertRaises(Exception, lambda: rdd.saveAsNewAPIHadoopFile( + basepath + "/badinputs/saveAsNewAPIHadoopFile/", + "org.apache.hadoop.mapreduce.lib.output.NotValidOutputFormat")) + + def test_converters(self): + # use of custom converters + basepath = self.tempdir.name + data = [(1, {3.0: u'bb'}), + (2, {1.0: u'aa'}), + (3, {2.0: u'dd'})] + self.sc.parallelize(data).saveAsNewAPIHadoopFile( + basepath + "/converters/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + keyConverter="org.apache.spark.api.python.TestOutputKeyConverter", + valueConverter="org.apache.spark.api.python.TestOutputValueConverter") + converted = sorted(self.sc.sequenceFile(basepath + "/converters/").collect()) + expected = [(u'1', 3.0), + (u'2', 1.0), + (u'3', 2.0)] + self.assertEqual(converted, expected) + + def test_reserialization(self): + basepath = self.tempdir.name + x = range(1, 5) + y = range(1001, 1005) + data = zip(x, y) + rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y)) + rdd.saveAsSequenceFile(basepath + "/reserialize/sequence") + result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) + self.assertEqual(result1, data) + + rdd.saveAsHadoopFile(basepath + "/reserialize/hadoop", + "org.apache.hadoop.mapred.SequenceFileOutputFormat") + result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect()) + self.assertEqual(result2, data) + + rdd.saveAsNewAPIHadoopFile(basepath + "/reserialize/newhadoop", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") + result3 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newhadoop").collect()) + self.assertEqual(result3, data) + + conf4 = { + "mapred.output.format.class" : "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class" : "org.apache.hadoop.io.IntWritable", + "mapred.output.dir" : basepath + "/reserialize/dataset"} + rdd.saveAsHadoopDataset(conf4) + result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect()) + self.assertEqual(result4, data) + + conf5 = {"mapreduce.outputformat.class" : + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class" : "org.apache.hadoop.io.IntWritable", + "mapred.output.dir" : basepath + "/reserialize/newdataset"} + rdd.saveAsNewAPIHadoopDataset(conf5) + result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect()) + self.assertEqual(result5, data) + + def test_unbatched_save_and_read(self): + basepath = self.tempdir.name + ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] + self.sc.parallelize(ei, numSlices=len(ei)).saveAsSequenceFile( + basepath + "/unbatched/") + + unbatched_sequence = sorted(self.sc.sequenceFile(basepath + "/unbatched/", + batchSize=1).collect()) + self.assertEqual(unbatched_sequence, ei) + + unbatched_hadoopFile = sorted(self.sc.hadoopFile(basepath + "/unbatched/", + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + batchSize=1).collect()) + self.assertEqual(unbatched_hadoopFile, ei) + + unbatched_newAPIHadoopFile = sorted(self.sc.newAPIHadoopFile(basepath + "/unbatched/", + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + batchSize=1).collect()) + self.assertEqual(unbatched_newAPIHadoopFile, ei) + + oldconf = {"mapred.input.dir" : basepath + "/unbatched/"} + unbatched_hadoopRDD = sorted(self.sc.hadoopRDD( + "org.apache.hadoop.mapred.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + conf=oldconf, + batchSize=1).collect()) + self.assertEqual(unbatched_hadoopRDD, ei) + + newconf = {"mapred.input.dir" : basepath + "/unbatched/"} + unbatched_newAPIHadoopRDD = sorted(self.sc.newAPIHadoopRDD( + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + conf=newconf, + batchSize=1).collect()) + self.assertEqual(unbatched_newAPIHadoopRDD, ei) + + def test_malformed_RDD(self): + basepath = self.tempdir.name + # non-batch-serialized RDD[[(K, V)]] should be rejected + data = [[(1, "a")], [(2, "aa")], [(3, "aaa")]] + rdd = self.sc.parallelize(data, numSlices=len(data)) + self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile( + basepath + "/malformed/sequence")) class TestDaemon(unittest.TestCase): def connect(self, port): From 7c7ce54522015315c909e111d6c2cff83e9fb501 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 13:42:43 -0700 Subject: [PATCH 0137/1492] Wrap JAR_DL in dev/check-license. --- dev/check-license | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/check-license b/dev/check-license index 00bb20c133b7d..625ec161bc571 100755 --- a/dev/check-license +++ b/dev/check-license @@ -32,9 +32,9 @@ acquire_rat_jar () { printf "Attempting to fetch rat\n" JAR_DL=${JAR}.part if hash curl 2>/dev/null; then - (curl --progress-bar ${URL1} > ${JAR_DL} || curl --progress-bar ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (curl --progress-bar ${URL1} > "$JAR_DL" || curl --progress-bar ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then - (wget --progress=bar ${URL1} -O ${JAR_DL} || wget --progress=bar ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (wget --progress=bar ${URL1} -O "$JAR_DL" || wget --progress=bar ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" else printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 From 1097327538ec3870544f406775efcfe7722e48be Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 14:08:24 -0700 Subject: [PATCH 0138/1492] Set AMPLAB_JENKINS_BUILD_PROFILE. --- dev/run-tests | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/dev/run-tests b/dev/run-tests index f2b523b996617..fb50fb380b15e 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,6 +21,18 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd "$FWDIR" +if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then + if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then + export SBT_MAVEN_PROFILES="-Dhadoop.version=1.0.4" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then + export SBT_MAVEN_PROFILES="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then + export SBT_MAVEN_PROFILES="-Pyarn -Dhadoop.version=2.2.0" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then + export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" + fi +fi + if [ -z "$SBT_MAVEN_PROFILES" ]; then export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" fi From 2f4b17056fdcba26fd3a7503b858364b883ab0b0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 14:31:20 -0700 Subject: [PATCH 0139/1492] Properly pass SBT_MAVEN_PROFILES into sbt. --- dev/run-tests | 27 +++++++++++++-------------- 1 file changed, 13 insertions(+), 14 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index fb50fb380b15e..daa85bc750c07 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -23,20 +23,20 @@ cd "$FWDIR" if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then - export SBT_MAVEN_PROFILES="-Dhadoop.version=1.0.4" + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=1.0.4" elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then - export SBT_MAVEN_PROFILES="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then - export SBT_MAVEN_PROFILES="-Pyarn -Dhadoop.version=2.2.0" + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Dhadoop.version=2.2.0" elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then - export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" fi fi -if [ -z "$SBT_MAVEN_PROFILES" ]; then - export SBT_MAVEN_PROFILES="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" +if [ -z "$SBT_MAVEN_PROFILES_ARGS" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" fi -echo "SBT_MAVEN_PROFILES=\"$SBT_MAVEN_PROFILES\"" +echo "SBT_MAVEN_PROFILES_ARGS=\"$SBT_MAVEN_PROFILES_ARGS\"" # Remove work directory rm -rf ./work @@ -76,16 +76,15 @@ dev/scalastyle echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" + +if [ -n "$_RUN_SQL_TESTS" ]; then + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" +fi # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. -if [ -n "$_RUN_SQL_TESTS" ]; then - echo -e "q\n" | SBT_MAVEN_PROFILES="$SBT_MAVEN_PROFILES -Phive -Phive-thriftserver" 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" -fi +echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS clean package assembly/assembly test | \ + grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" echo "=========================================================================" echo "Running PySpark tests" From 6ab96a6fd0db7731c8c5d6478d9e28b619581687 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 30 Jul 2014 15:04:33 -0700 Subject: [PATCH 0140/1492] SPARK-2749 [BUILD]. Spark SQL Java tests aren't compiling in Jenkins' Maven builds; missing junit:junit dep The Maven-based builds in the build matrix have been failing for a few days: https://amplab.cs.berkeley.edu/jenkins/view/Spark/ On inspection, it looks like the Spark SQL Java tests don't compile: https://amplab.cs.berkeley.edu/jenkins/view/Spark/job/Spark-Master-Maven-pre-YARN/hadoop.version=1.0.4,label=centos/244/consoleFull I confirmed it by repeating the command vs master: `mvn -Dhadoop.version=1.0.4 -Dlabel=centos -DskipTests clean package` The problem is that this module doesn't depend on JUnit. In fact, none of the modules do, but `com.novocode:junit-interface` (the SBT-JUnit bridge) pulls it in, in most places. However this module doesn't depend on `com.novocode:junit-interface` Adding the `junit:junit` dependency fixes the compile problem. In fact, the other modules with Java tests should probably depend on it explicitly instead of happening to get it via `com.novocode:junit-interface`, since that is a bit SBT/Scala-specific (and I am not even sure it's needed). Author: Sean Owen Closes #1660 from srowen/SPARK-2749 and squashes the following commits: 858ff7c [Sean Owen] Add explicit junit dep to other modules with Java tests for robustness 9636794 [Sean Owen] Add junit dep so that Spark SQL Java tests compile --- core/pom.xml | 5 +++++ external/flume/pom.xml | 5 +++++ external/kafka/pom.xml | 5 +++++ external/mqtt/pom.xml | 5 +++++ external/twitter/pom.xml | 5 +++++ external/zeromq/pom.xml | 5 +++++ extras/java8-tests/pom.xml | 5 +++++ mllib/pom.xml | 5 +++++ sql/core/pom.xml | 5 +++++ streaming/pom.xml | 5 +++++ 10 files changed, 50 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 4f061099a477d..04d4b9cc1068e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -262,6 +262,11 @@ asm test + + junit + junit + test + com.novocode junit-interface diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 9f680b27c3308..c532705f3950c 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -72,6 +72,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 25a5c0a4d7d77..daf03360bc5f5 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -80,6 +80,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index f31ed655f6779..dc48a08c93de2 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -67,6 +67,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 56bb24c2a072e..b93ad016f84f0 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -62,6 +62,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 54b0242c54e78..22c1fff23d9a2 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -62,6 +62,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 3eade411b38b7..5308bb4e440ea 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -50,6 +50,11 @@ ${project.version} test-jar + + junit + junit + test + com.novocode junit-interface diff --git a/mllib/pom.xml b/mllib/pom.xml index f27cf520dc9fa..cb0fa7b97cb15 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -72,6 +72,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 3a038a2db6173..c8016e41256d5 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -68,6 +68,11 @@ jackson-databind 2.3.0 + + junit + junit + test + org.scalatest scalatest_${scala.binary.version} diff --git a/streaming/pom.xml b/streaming/pom.xml index b99f306b8f2cc..1072f74aea0d9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -58,6 +58,11 @@ scalacheck_${scala.binary.version} test + + junit + junit + test + com.novocode junit-interface From 2ac37db7ac8f7ec5c99f3bfe459f8e2ac240961f Mon Sep 17 00:00:00 2001 From: Brock Noland Date: Wed, 30 Jul 2014 17:04:30 -0700 Subject: [PATCH 0141/1492] SPARK-2741 - Publish version of spark assembly which does not contain Hive Provide a version of the Spark tarball which does not package Hive. This is meant for HIve + Spark users. Author: Brock Noland Closes #1667 from brockn/master and squashes the following commits: 5beafb2 [Brock Noland] SPARK-2741 - Publish version of spark assembly which does not contain Hive --- dev/create-release/create-release.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 33de24d1ae6d7..af46572e6602b 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -115,6 +115,8 @@ make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4 make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" make_binary_release "hadoop2" \ "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" +make_binary_release "hadoop2-without-hive" \ + "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" # Copy data echo "Copying release tarballs" From 88a519db90d66ee5a1455ef4fcc1ad2a687e3d0b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 30 Jul 2014 17:30:51 -0700 Subject: [PATCH 0142/1492] [SPARK-2734][SQL] Remove tables from cache when DROP TABLE is run. Author: Michael Armbrust Closes #1650 from marmbrus/dropCached and squashes the following commits: e6ab80b [Michael Armbrust] Support if exists. 83426c6 [Michael Armbrust] Remove tables from cache when DROP TABLE is run. --- .../org/apache/spark/sql/hive/HiveQl.scala | 9 +++- .../spark/sql/hive/HiveStrategies.scala | 2 + .../spark/sql/hive/execution/DropTable.scala | 48 +++++++++++++++++++ .../spark/sql/hive/CachedTableSuite.scala | 16 +++++++ 4 files changed, 74 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DropTable.scala 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 d18ccf8167487..3d2eb1eefaeda 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 @@ -44,6 +44,8 @@ private[hive] case class SourceCommand(filePath: String) extends Command private[hive] case class AddFile(filePath: String) extends Command +private[hive] case class DropTable(tableName: String, ifExists: Boolean) extends Command + /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { protected val nativeCommands = Seq( @@ -96,7 +98,6 @@ private[hive] object HiveQl { "TOK_CREATEINDEX", "TOK_DROPDATABASE", "TOK_DROPINDEX", - "TOK_DROPTABLE", "TOK_MSCK", // TODO(marmbrus): Figure out how view are expanded by hive, as we might need to handle this. @@ -377,6 +378,12 @@ private[hive] object HiveQl { } protected def nodeToPlan(node: Node): LogicalPlan = node match { + // Special drop table that also uncaches. + case Token("TOK_DROPTABLE", + Token("TOK_TABNAME", tableNameParts) :: + ifExists) => + val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") + DropTable(tableName, ifExists.nonEmpty) // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if noExplainCommands.contains(explainArgs.head.getText) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 4d0fab4140b21..2175c5f3835a6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -81,6 +81,8 @@ private[hive] trait HiveStrategies { case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil + case DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil + case describe: logical.DescribeCommand => val resolvedTable = context.executePlan(describe.table).analyzed resolvedTable match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DropTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DropTable.scala new file mode 100644 index 0000000000000..9cd0c86c6c796 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DropTable.scala @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.execution.{Command, LeafNode} +import org.apache.spark.sql.hive.HiveContext + +/** + * :: DeveloperApi :: + * Drops a table from the metastore and removes it if it is cached. + */ +@DeveloperApi +case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Command { + + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + def output = Seq.empty + + override protected[sql] lazy val sideEffectResult: Seq[Any] = { + val ifExistsClause = if (ifExists) "IF EXISTS " else "" + hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") + hiveContext.catalog.unregisterTable(None, tableName) + Seq.empty + } + + override def execute(): RDD[Row] = { + sideEffectResult + sparkContext.emptyRDD[Row] + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 3132d0112c708..08da6405a17c6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -23,6 +23,8 @@ import org.apache.spark.sql.hive.execution.HiveComparisonTest import org.apache.spark.sql.hive.test.TestHive class CachedTableSuite extends HiveComparisonTest { + import TestHive._ + TestHive.loadTestTable("src") test("cache table") { @@ -32,6 +34,20 @@ class CachedTableSuite extends HiveComparisonTest { createQueryTest("read from cached table", "SELECT * FROM src LIMIT 1", reset = false) + test("Drop cached table") { + hql("CREATE TABLE test(a INT)") + cacheTable("test") + hql("SELECT * FROM test").collect() + hql("DROP TABLE test") + intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] { + hql("SELECT * FROM test").collect() + } + } + + test("DROP nonexistant table") { + hql("DROP TABLE IF EXISTS nonexistantTable") + } + test("check that table is cached and uncache") { TestHive.table("src").queryExecution.analyzed match { case _ : InMemoryRelation => // Found evidence of caching From e9b275b7697e7ad3b52b157d3274acc17ca8d828 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 30 Jul 2014 17:34:32 -0700 Subject: [PATCH 0143/1492] SPARK-2341 [MLLIB] loadLibSVMFile doesn't handle regression datasets Per discussion at https://issues.apache.org/jira/browse/SPARK-2341 , this is a look at deprecating the multiclass parameter. Thoughts welcome of course. Author: Sean Owen Closes #1663 from srowen/SPARK-2341 and squashes the following commits: 8a3abd7 [Sean Owen] Suppress MIMA error for removed package private classes 18a8c8e [Sean Owen] Updates from review 83d0092 [Sean Owen] Deprecated methods with multiclass, and instead always parse target as a double (ie. multiclass = true) --- .../examples/mllib/LinearRegression.scala | 2 +- .../examples/mllib/SparseNaiveBayes.scala | 4 +- .../spark/mllib/util/LabelParsers.scala | 56 ------------------- .../org/apache/spark/mllib/util/MLUtils.scala | 52 ++++++----------- .../spark/mllib/util/LabelParsersSuite.scala | 41 -------------- .../spark/mllib/util/MLUtilsSuite.scala | 14 ++--- project/MimaExcludes.scala | 8 +++ python/pyspark/mllib/util.py | 23 ++++---- 8 files changed, 46 insertions(+), 154 deletions(-) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 4811bb70e4b28..05b7d66f8dffd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -91,7 +91,7 @@ object LinearRegression extends App { Logger.getRootLogger.setLevel(Level.WARN) - val examples = MLUtils.loadLibSVMFile(sc, params.input, multiclass = true).cache() + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() val splits = examples.randomSplit(Array(0.8, 0.2)) val training = splits(0).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index 537e68a0991aa..88acd9dbb0878 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -22,7 +22,7 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.classification.NaiveBayes -import org.apache.spark.mllib.util.{MLUtils, MulticlassLabelParser} +import org.apache.spark.mllib.util.MLUtils /** * An example naive Bayes app. Run with @@ -76,7 +76,7 @@ object SparseNaiveBayes { if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions val examples = - MLUtils.loadLibSVMFile(sc, params.input, multiclass = true, params.numFeatures, minPartitions) + MLUtils.loadLibSVMFile(sc, params.input, params.numFeatures, minPartitions) // Cache examples because it will be used in both training and evaluation. examples.cache() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala deleted file mode 100644 index e25bf18b780bf..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.util - -/** Trait for label parsers. */ -private trait LabelParser extends Serializable { - /** Parses a string label into a double label. */ - def parse(labelString: String): Double -} - -/** Factory methods for label parsers. */ -private object LabelParser { - def getInstance(multiclass: Boolean): LabelParser = { - if (multiclass) MulticlassLabelParser else BinaryLabelParser - } -} - -/** - * Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5, - * or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling. - */ -private object BinaryLabelParser extends LabelParser { - /** Gets the default instance of BinaryLabelParser. */ - def getInstance(): LabelParser = this - - /** - * Parses the input label into positive (1.0) if the value is greater than 0.5, - * or negative (0.0) otherwise. - */ - override def parse(labelString: String): Double = if (labelString.toDouble > 0.5) 1.0 else 0.0 -} - -/** - * Label parser for multiclass labels, which converts the input label to double. - */ -private object MulticlassLabelParser extends LabelParser { - /** Gets the default instance of MulticlassLabelParser. */ - def getInstance(): LabelParser = this - - override def parse(labelString: String): Double = labelString.toDouble -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 30de24ad89f98..dc10a194783ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -55,7 +55,6 @@ object MLUtils { * * @param sc Spark context * @param path file or directory path in any Hadoop-supported file system URI - * @param labelParser parser for labels * @param numFeatures number of features, which will be determined from the input data if a * nonpositive value is given. This is useful when the dataset is already split * into multiple files and you want to load them separately, because some @@ -64,10 +63,9 @@ object MLUtils { * @param minPartitions min number of partitions * @return labeled data stored as an RDD[LabeledPoint] */ - private def loadLibSVMFile( + def loadLibSVMFile( sc: SparkContext, path: String, - labelParser: LabelParser, numFeatures: Int, minPartitions: Int): RDD[LabeledPoint] = { val parsed = sc.textFile(path, minPartitions) @@ -75,7 +73,7 @@ object MLUtils { .filter(line => !(line.isEmpty || line.startsWith("#"))) .map { line => val items = line.split(' ') - val label = labelParser.parse(items.head) + val label = items.head.toDouble val (indices, values) = items.tail.map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. @@ -102,64 +100,46 @@ object MLUtils { // Convenient methods for `loadLibSVMFile`. - /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. - * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. - * Each line represents a labeled sparse feature vector using the following format: - * {{{label index1:value1 index2:value2 ...}}} - * where the indices are one-based and in ascending order. - * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], - * where the feature indices are converted to zero-based. - * - * @param sc Spark context - * @param path file or directory path in any Hadoop-supported file system URI - * @param multiclass whether the input labels contain more than two classes. If false, any label - * with value greater than 0.5 will be mapped to 1.0, or 0.0 otherwise. So it - * works for both +1/-1 and 1/0 cases. If true, the double value parsed directly - * from the label string will be used as the label value. - * @param numFeatures number of features, which will be determined from the input data if a - * nonpositive value is given. This is useful when the dataset is already split - * into multiple files and you want to load them separately, because some - * features may not present in certain files, which leads to inconsistent - * feature dimensions. - * @param minPartitions min number of partitions - * @return labeled data stored as an RDD[LabeledPoint] - */ - def loadLibSVMFile( + @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") + def loadLibSVMFile( sc: SparkContext, path: String, multiclass: Boolean, numFeatures: Int, minPartitions: Int): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, LabelParser.getInstance(multiclass), numFeatures, minPartitions) + loadLibSVMFile(sc, path, numFeatures, minPartitions) /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the default number of * partitions. */ + def loadLibSVMFile( + sc: SparkContext, + path: String, + numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMFile(sc, path, numFeatures, sc.defaultMinPartitions) + + @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") def loadLibSVMFile( sc: SparkContext, path: String, multiclass: Boolean, numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, multiclass, numFeatures, sc.defaultMinPartitions) + loadLibSVMFile(sc, path, numFeatures) - /** - * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the number of features - * determined automatically and the default number of partitions. - */ + @deprecated("use method without multiclass argument, which no longer has effect", "1.1.0") def loadLibSVMFile( sc: SparkContext, path: String, multiclass: Boolean): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, multiclass, -1, sc.defaultMinPartitions) + loadLibSVMFile(sc, path) /** * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], with number of * features determined automatically and the default number of partitions. */ def loadLibSVMFile(sc: SparkContext, path: String): RDD[LabeledPoint] = - loadLibSVMFile(sc, path, multiclass = false, -1, sc.defaultMinPartitions) + loadLibSVMFile(sc, path, -1) /** * Save labeled data in LIBSVM format. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala deleted file mode 100644 index ac85677f2f014..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala +++ /dev/null @@ -1,41 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.util - -import org.scalatest.FunSuite - -class LabelParsersSuite extends FunSuite { - test("binary label parser") { - for (parser <- Seq(BinaryLabelParser, BinaryLabelParser.getInstance())) { - assert(parser.parse("+1") === 1.0) - assert(parser.parse("1") === 1.0) - assert(parser.parse("0") === 0.0) - assert(parser.parse("-1") === 0.0) - } - } - - test("multiclass label parser") { - for (parser <- Seq(MulticlassLabelParser, MulticlassLabelParser.getInstance())) { - assert(parser.parse("0") == 0.0) - assert(parser.parse("+1") === 1.0) - assert(parser.parse("1") === 1.0) - assert(parser.parse("2") === 2.0) - assert(parser.parse("3") === 3.0) - } - } -} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index c14870fb969a8..8ef2bb1bf6a78 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -63,9 +63,9 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { test("loadLibSVMFile") { val lines = """ - |+1 1:1.0 3:2.0 5:3.0 - |-1 - |-1 2:4.0 4:5.0 6:6.0 + |1 1:1.0 3:2.0 5:3.0 + |0 + |0 2:4.0 4:5.0 6:6.0 """.stripMargin val tempDir = Files.createTempDir() tempDir.deleteOnExit() @@ -73,7 +73,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString - val pointsWithNumFeatures = loadLibSVMFile(sc, path, multiclass = false, 6).collect() + val pointsWithNumFeatures = loadLibSVMFile(sc, path, 6).collect() val pointsWithoutNumFeatures = loadLibSVMFile(sc, path).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { @@ -86,11 +86,11 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) } - val multiclassPoints = loadLibSVMFile(sc, path, multiclass = true).collect() + val multiclassPoints = loadLibSVMFile(sc, path).collect() assert(multiclassPoints.length === 3) assert(multiclassPoints(0).label === 1.0) - assert(multiclassPoints(1).label === -1.0) - assert(multiclassPoints(2).label === -1.0) + assert(multiclassPoints(1).label === 0.0) + assert(multiclassPoints(2).label === 0.0) Utils.deleteRecursively(tempDir) } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 5ff88f0dd1cac..5a835f58207cf 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -97,6 +97,14 @@ object MimaExcludes { "org.apache.spark.mllib.tree.impurity.Entropy.calculate"), ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.mllib.tree.impurity.Variance.calculate") + ) ++ + Seq ( // Package-private classes removed in SPARK-2341 + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") ) case v if v.startsWith("1.0") => Seq( diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index a707a9dcd5b49..d94900cefdb77 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -29,15 +29,18 @@ class MLUtils: Helper methods to load, save and pre-process data used in MLlib. """ + @deprecated @staticmethod def _parse_libsvm_line(line, multiclass): + return _parse_libsvm_line(line) + + @staticmethod + def _parse_libsvm_line(line): """ Parses a line in LIBSVM format into (label, indices, values). """ items = line.split(None) label = float(items[0]) - if not multiclass: - label = 1.0 if label > 0.5 else 0.0 nnz = len(items) - 1 indices = np.zeros(nnz, dtype=np.int32) values = np.zeros(nnz) @@ -64,8 +67,13 @@ def _convert_labeled_point_to_libsvm(p): " but got " % type(v)) return " ".join(items) + @deprecated @staticmethod def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): + return loadLibSVMFile(sc, path, numFeatures, minPartitions) + + @staticmethod + def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): """ Loads labeled data in the LIBSVM format into an RDD of LabeledPoint. The LIBSVM format is a text-based format used by @@ -81,13 +89,6 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non @param sc: Spark context @param path: file or directory path in any Hadoop-supported file system URI - @param multiclass: whether the input labels contain more than - two classes. If false, any label with value - greater than 0.5 will be mapped to 1.0, or - 0.0 otherwise. So it works for both +1/-1 and - 1/0 cases. If true, the double value parsed - directly from the label string will be used - as the label value. @param numFeatures: number of features, which will be determined from the input data if a nonpositive value is given. This is useful when the dataset is @@ -105,7 +106,7 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() - >>> multiclass_examples = MLUtils.loadLibSVMFile(sc, tempFile.name, True).collect() + >>> multiclass_examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() >>> tempFile.close() >>> type(examples[0]) == LabeledPoint True @@ -124,7 +125,7 @@ def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=Non """ lines = sc.textFile(path, minPartitions) - parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l, multiclass)) + parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l)) if numFeatures <= 0: parsed.cache() numFeatures = parsed.map(lambda x: 0 if x[1].size == 0 else x[1][-1]).reduce(max) + 1 From da501766834453c9ac7095c7e8c930151f87cf11 Mon Sep 17 00:00:00 2001 From: strat0sphere Date: Wed, 30 Jul 2014 17:57:50 -0700 Subject: [PATCH 0144/1492] Update DecisionTreeRunner.scala Author: strat0sphere Closes #1676 from strat0sphere/patch-1 and squashes the following commits: 044d2fa [strat0sphere] Update DecisionTreeRunner.scala --- .../org/apache/spark/examples/mllib/DecisionTreeRunner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 43f13fe24f0d0..6db9bf3cf5be6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -33,7 +33,7 @@ import org.apache.spark.rdd.RDD /** * An example runner for decision tree. Run with * {{{ - * ./bin/spark-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] + * ./bin/run-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] * }}} * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ From e966284409f9355e1169960e73a2215617c8cb22 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 30 Jul 2014 18:07:59 -0700 Subject: [PATCH 0145/1492] SPARK-2045 Sort-based shuffle This adds a new ShuffleManager based on sorting, as described in https://issues.apache.org/jira/browse/SPARK-2045. The bulk of the code is in an ExternalSorter class that is similar to ExternalAppendOnlyMap, but sorts key-value pairs by partition ID and can be used to create a single sorted file with a map task's output. (Longer-term I think this can take on the remaining functionality in ExternalAppendOnlyMap and replace it so we don't have code duplication.) The main TODOs still left are: - [x] enabling ExternalSorter to merge across spilled files - [x] with an Ordering - [x] without an Ordering, using the keys' hash codes - [x] adding more tests (e.g. a version of our shuffle suite that runs on this) - [x] rebasing on top of the size-tracking refactoring in #1165 when that is merged - [x] disabling spilling if spark.shuffle.spill is set to false Despite this though, this seems to work pretty well (running successfully in cases where the hash shuffle would OOM, such as 1000 reduce tasks on executors with only 1G memory), and it seems to be comparable in speed or faster than hash-based shuffle (it will create much fewer files for the OS to keep track of). So I'm posting it to get some early feedback. After these TODOs are done, I'd also like to enable ExternalSorter to sort data within each partition by a key as well, which will allow us to use it to implement external spilling in reduce tasks in `sortByKey`. Author: Matei Zaharia Closes #1499 from mateiz/sort-based-shuffle and squashes the following commits: bd841f9 [Matei Zaharia] Various review comments d1c137fd [Matei Zaharia] Various review comments a611159 [Matei Zaharia] Compile fixes due to rebase 62c56c8 [Matei Zaharia] Fix ShuffledRDD sometimes not returning Tuple2s. f617432 [Matei Zaharia] Fix a failing test (seems to be due to change in SizeTracker logic) 9464d5f [Matei Zaharia] Simplify code and fix conflicts after latest rebase 0174149 [Matei Zaharia] Add cleanup behavior and cleanup tests for sort-based shuffle eb4ee0d [Matei Zaharia] Remove customizable element type in ShuffledRDD fa2e8db [Matei Zaharia] Allow nextBatchStream to be called after we're done looking at all streams a34b352 [Matei Zaharia] Fix tracking of indices within a partition in SpillReader, and add test 03e1006 [Matei Zaharia] Add a SortShuffleSuite that runs ShuffleSuite with sort-based shuffle 3c7ff1f [Matei Zaharia] Obey the spark.shuffle.spill setting in ExternalSorter ad65fbd [Matei Zaharia] Rebase on top of Aaron's Sorter change, and use Sorter in our buffer 44d2a93 [Matei Zaharia] Use estimateSize instead of atGrowThreshold to test collection sizes 5686f71 [Matei Zaharia] Optimize merging phase for in-memory only data: 5461cbb [Matei Zaharia] Review comments and more tests (e.g. tests with 1 element per partition) e9ad356 [Matei Zaharia] Update ContextCleanerSuite to make sure shuffle cleanup tests use hash shuffle (since they were written for it) c72362a [Matei Zaharia] Added bug fix and test for when iterators are empty de1fb40 [Matei Zaharia] Make trait SizeTrackingCollection private[spark] 4988d16 [Matei Zaharia] tweak c1b7572 [Matei Zaharia] Small optimization ba7db7f [Matei Zaharia] Handle null keys in hash-based comparator, and add tests for collisions ef4e397 [Matei Zaharia] Support for partial aggregation even without an Ordering 4b7a5ce [Matei Zaharia] More tests, and ability to sort data if a total ordering is given e1f84be [Matei Zaharia] Fix disk block manager test 5a40a1c [Matei Zaharia] More tests 614f1b4 [Matei Zaharia] Add spill metrics to map tasks cc52caf [Matei Zaharia] Add more error handling and tests for error cases bbf359d [Matei Zaharia] More work 3a56341 [Matei Zaharia] More partial work towards sort-based shuffle 7a0895d [Matei Zaharia] Some more partial work towards sort-based shuffle b615476 [Matei Zaharia] Scaffolding for sort-based shuffle --- .../scala/org/apache/spark/Aggregator.scala | 24 +- .../scala/org/apache/spark/SparkContext.scala | 8 +- .../apache/spark/api/java/JavaPairRDD.scala | 2 +- .../org/apache/spark/rdd/CoGroupedRDD.scala | 7 +- .../spark/rdd/OrderedRDDFunctions.scala | 14 +- .../apache/spark/rdd/PairRDDFunctions.scala | 4 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 8 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 17 +- .../shuffle/hash/HashShuffleManager.scala | 2 +- .../shuffle/hash/HashShuffleReader.scala | 5 +- .../shuffle/hash/HashShuffleWriter.scala | 6 +- .../shuffle/sort/SortShuffleManager.scala | 80 +++ .../shuffle/sort/SortShuffleWriter.scala | 165 +++++ .../org/apache/spark/storage/BlockId.scala | 11 +- .../spark/storage/DiskBlockManager.scala | 38 +- .../spark/storage/ShuffleBlockManager.scala | 29 +- .../collection/ExternalAppendOnlyMap.scala | 36 +- .../util/collection/ExternalSorter.scala | 662 ++++++++++++++++++ .../SizeTrackingAppendOnlyMap.scala | 5 +- .../collection/SizeTrackingPairBuffer.scala | 86 +++ .../SizeTrackingPairCollection.scala | 34 + .../org/apache/spark/CheckpointSuite.scala | 2 +- .../apache/spark/ContextCleanerSuite.scala | 186 +++-- .../org/apache/spark/ShuffleNettySuite.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 26 +- .../org/apache/spark/SortShuffleSuite.scala | 34 + .../scala/org/apache/spark/rdd/RDDSuite.scala | 6 +- .../ExternalAppendOnlyMapSuite.scala | 25 +- .../util/collection/ExternalSorterSuite.scala | 566 +++++++++++++++ .../util/collection/FixedHashObject.scala | 25 + .../graphx/impl/MessageToPartition.scala | 2 +- .../graphx/impl/RoutingTablePartition.scala | 2 +- project/SparkBuild.scala | 1 + .../apache/spark/sql/execution/Exchange.scala | 6 +- .../spark/sql/execution/basicOperators.scala | 2 +- 35 files changed, 1969 insertions(+), 159 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala create mode 100644 core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairBuffer.scala create mode 100644 core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairCollection.scala create mode 100644 core/src/test/scala/org/apache/spark/SortShuffleSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/collection/FixedHashObject.scala diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ff0ca11749d42..79c9c451d273d 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -56,18 +56,23 @@ case class Aggregator[K, V, C] ( } else { val combiners = new ExternalAppendOnlyMap[K, V, C](createCombiner, mergeValue, mergeCombiners) combiners.insertAll(iter) - // TODO: Make this non optional in a future release - Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled) - Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled) + // Update task metrics if context is not null + // TODO: Make context non optional in a future release + Option(context).foreach { c => + c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled + c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled + } combiners.iterator } } @deprecated("use combineCombinersByKey with TaskContext argument", "0.9.0") - def combineCombinersByKey(iter: Iterator[(K, C)]) : Iterator[(K, C)] = + def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]]) : Iterator[(K, C)] = combineCombinersByKey(iter, null) - def combineCombinersByKey(iter: Iterator[(K, C)], context: TaskContext) : Iterator[(K, C)] = { + def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]], context: TaskContext) + : Iterator[(K, C)] = + { if (!externalSorting) { val combiners = new AppendOnlyMap[K,C] var kc: Product2[K, C] = null @@ -85,9 +90,12 @@ case class Aggregator[K, V, C] ( val pair = iter.next() combiners.insert(pair._1, pair._2) } - // TODO: Make this non optional in a future release - Option(context).foreach(c => c.taskMetrics.memoryBytesSpilled = combiners.memoryBytesSpilled) - Option(context).foreach(c => c.taskMetrics.diskBytesSpilled = combiners.diskBytesSpilled) + // Update task metrics if context is not null + // TODO: Make context non-optional in a future release + Option(context).foreach { c => + c.taskMetrics.memoryBytesSpilled += combiners.memoryBytesSpilled + c.taskMetrics.diskBytesSpilled += combiners.diskBytesSpilled + } combiners.iterator } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fb4c86716bb8d..b25f081761a64 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -289,7 +289,7 @@ class SparkContext(config: SparkConf) extends Logging { value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { executorEnvs(envKey) = value } - Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => + Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => executorEnvs("SPARK_PREPEND_CLASSES") = v } // The Mesos scheduler backend relies on this environment variable to set executor memory. @@ -1203,10 +1203,10 @@ class SparkContext(config: SparkConf) extends Logging { /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) - * If checkSerializable is set, clean will also proactively - * check to see if f is serializable and throw a SparkException + * If checkSerializable is set, clean will also proactively + * check to see if f is serializable and throw a SparkException * if not. - * + * * @param f the closure to clean * @param checkSerializable whether or not to immediately check f for serializability * @throws SparkException if checkSerializable is set but f is not diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 31bf8dced2638..47708cb2e78bd 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -122,7 +122,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) */ def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] = sample(withReplacement, fraction, Utils.random.nextLong) - + /** * Return a sampled subset of this RDD. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 6388ef82cc5db..fabb882cdd4b3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -17,10 +17,11 @@ package org.apache.spark.rdd +import scala.language.existentials + import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer -import scala.language.existentials import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} @@ -157,8 +158,8 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: for ((it, depNum) <- rddIterators) { map.insertAll(it.map(pair => (pair._1, new CoGroupValue(pair._2, depNum)))) } - context.taskMetrics.memoryBytesSpilled = map.memoryBytesSpilled - context.taskMetrics.diskBytesSpilled = map.diskBytesSpilled + context.taskMetrics.memoryBytesSpilled += map.memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += map.diskBytesSpilled new InterruptibleIterator(context, map.iterator.asInstanceOf[Iterator[(K, Array[Iterable[_]])]]) } diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index d85f962783931..e98bad2026e32 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Logging, RangePartitioner} +import org.apache.spark.annotation.DeveloperApi /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through @@ -43,10 +44,10 @@ import org.apache.spark.{Logging, RangePartitioner} */ class OrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag, - P <: Product2[K, V] : ClassTag]( + P <: Product2[K, V] : ClassTag] @DeveloperApi() ( self: RDD[P]) - extends Logging with Serializable { - + extends Logging with Serializable +{ private val ordering = implicitly[Ordering[K]] /** @@ -55,9 +56,12 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in * order of the keys). */ - def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size): RDD[P] = { + // TODO: this currently doesn't work on P other than Tuple2! + def sortByKey(ascending: Boolean = true, numPartitions: Int = self.partitions.size) + : RDD[(K, V)] = + { val part = new RangePartitioner(numPartitions, self, ascending) - new ShuffledRDD[K, V, V, P](self, part) + new ShuffledRDD[K, V, V](self, part) .setKeyOrdering(if (ascending) ordering else ordering.reverse) } } 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 1af4e5f0b6d08..93af50c0a9cd1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -90,7 +90,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) } else { - new ShuffledRDD[K, V, C, (K, C)](self, partitioner) + new ShuffledRDD[K, V, C](self, partitioner) .setSerializer(serializer) .setAggregator(aggregator) .setMapSideCombine(mapSideCombine) @@ -425,7 +425,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) if (self.partitioner == Some(partitioner)) { self } else { - new ShuffledRDD[K, V, V, (K, V)](self, partitioner) + new ShuffledRDD[K, V, V](self, partitioner) } } 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 726b3f2bbeea7..74ac97091fd0b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -332,7 +332,7 @@ abstract class RDD[T: ClassTag]( val distributePartition = (index: Int, items: Iterator[T]) => { var position = (new Random(index)).nextInt(numPartitions) items.map { t => - // Note that the hash code of the key will just be the key itself. The HashPartitioner + // Note that the hash code of the key will just be the key itself. The HashPartitioner // will mod it with the number of total partitions. position = position + 1 (position, t) @@ -341,7 +341,7 @@ abstract class RDD[T: ClassTag]( // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( - new ShuffledRDD[Int, T, T, (Int, T)](mapPartitionsWithIndex(distributePartition), + new ShuffledRDD[Int, T, T](mapPartitionsWithIndex(distributePartition), new HashPartitioner(numPartitions)), numPartitions).values } else { @@ -352,8 +352,8 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, - fraction: Double, + def sample(withReplacement: Boolean, + fraction: Double, seed: Long = Utils.random.nextLong): RDD[T] = { require(fraction >= 0.0, "Negative fraction value: " + fraction) if (withReplacement) { diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index bf02f68d0d3d3..d9fe6847254fa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -37,11 +37,12 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { * @tparam V the value class. * @tparam C the combiner class. */ +// TODO: Make this return RDD[Product2[K, C]] or have some way to configure mutable pairs @DeveloperApi -class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( +class ShuffledRDD[K, V, C]( @transient var prev: RDD[_ <: Product2[K, V]], part: Partitioner) - extends RDD[P](prev.context, Nil) { + extends RDD[(K, C)](prev.context, Nil) { private var serializer: Option[Serializer] = None @@ -52,25 +53,25 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( private var mapSideCombine: Boolean = false /** Set a serializer for this RDD's shuffle, or null to use the default (spark.serializer) */ - def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C, P] = { + def setSerializer(serializer: Serializer): ShuffledRDD[K, V, C] = { this.serializer = Option(serializer) this } /** Set key ordering for RDD's shuffle. */ - def setKeyOrdering(keyOrdering: Ordering[K]): ShuffledRDD[K, V, C, P] = { + def setKeyOrdering(keyOrdering: Ordering[K]): ShuffledRDD[K, V, C] = { this.keyOrdering = Option(keyOrdering) this } /** Set aggregator for RDD's shuffle. */ - def setAggregator(aggregator: Aggregator[K, V, C]): ShuffledRDD[K, V, C, P] = { + def setAggregator(aggregator: Aggregator[K, V, C]): ShuffledRDD[K, V, C] = { this.aggregator = Option(aggregator) this } /** Set mapSideCombine flag for RDD's shuffle. */ - def setMapSideCombine(mapSideCombine: Boolean): ShuffledRDD[K, V, C, P] = { + def setMapSideCombine(mapSideCombine: Boolean): ShuffledRDD[K, V, C] = { this.mapSideCombine = mapSideCombine this } @@ -85,11 +86,11 @@ class ShuffledRDD[K, V, C, P <: Product2[K, C] : ClassTag]( Array.tabulate[Partition](part.numPartitions)(i => new ShuffledRDDPartition(i)) } - override def compute(split: Partition, context: TaskContext): Iterator[P] = { + override def compute(split: Partition, context: TaskContext): Iterator[(K, C)] = { val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]] SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context) .read() - .asInstanceOf[Iterator[P]] + .asInstanceOf[Iterator[(K, C)]] } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala index 5b0940ecce29d..df98d18fa8193 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleManager.scala @@ -24,7 +24,7 @@ import org.apache.spark.shuffle._ * A ShuffleManager using hashing, that creates one output file per reduce partition on each * mapper (possibly reusing these across waves of tasks). */ -class HashShuffleManager(conf: SparkConf) extends ShuffleManager { +private[spark] class HashShuffleManager(conf: SparkConf) extends ShuffleManager { /* Register a shuffle with the manager and obtain a handle for it to pass to tasks. */ override def registerShuffle[K, V, C]( shuffleId: Int, diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index c8059496a1bdf..e32ad9c036ad4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -21,7 +21,7 @@ import org.apache.spark.{InterruptibleIterator, TaskContext} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} -class HashShuffleReader[K, C]( +private[spark] class HashShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], startPartition: Int, endPartition: Int, @@ -47,7 +47,8 @@ class HashShuffleReader[K, C]( } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { throw new IllegalStateException("Aggregator is empty for map-side combine") } else { - iter + // Convert the Product2s to pairs since this is what downstream RDDs currently expect + iter.asInstanceOf[Iterator[Product2[K, C]]].map(pair => (pair._1, pair._2)) } // Sort the output if there is a sort ordering defined. diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 9b78228519da4..1923f7c71a48f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -24,7 +24,7 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus -class HashShuffleWriter[K, V]( +private[spark] class HashShuffleWriter[K, V]( handle: BaseShuffleHandle[K, V, _], mapId: Int, context: TaskContext) @@ -33,6 +33,10 @@ class HashShuffleWriter[K, V]( private val dep = handle.dependency private val numOutputSplits = dep.partitioner.numPartitions private val metrics = context.taskMetrics + + // Are we in the process of stopping? Because map tasks can call stop() with success = true + // and then call stop() with success = false if they get an exception, we want to make sure + // we don't try deleting files, etc twice. private var stopping = false private val blockManager = SparkEnv.get.blockManager diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala new file mode 100644 index 0000000000000..6dcca47ea7c0c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle.sort + +import java.io.{DataInputStream, FileInputStream} + +import org.apache.spark.shuffle._ +import org.apache.spark.{TaskContext, ShuffleDependency} +import org.apache.spark.shuffle.hash.HashShuffleReader +import org.apache.spark.storage.{DiskBlockManager, FileSegment, ShuffleBlockId} + +private[spark] class SortShuffleManager extends ShuffleManager { + /** + * Register a shuffle with the manager and obtain a handle for it to pass to tasks. + */ + override def registerShuffle[K, V, C]( + shuffleId: Int, + numMaps: Int, + dependency: ShuffleDependency[K, V, C]): ShuffleHandle = { + new BaseShuffleHandle(shuffleId, numMaps, dependency) + } + + /** + * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). + * Called on executors by reduce tasks. + */ + override def getReader[K, C]( + handle: ShuffleHandle, + startPartition: Int, + endPartition: Int, + context: TaskContext): ShuffleReader[K, C] = { + // We currently use the same block store shuffle fetcher as the hash-based shuffle. + new HashShuffleReader( + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context) + } + + /** Get a writer for a given partition. Called on executors by map tasks. */ + override def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext) + : ShuffleWriter[K, V] = { + new SortShuffleWriter(handle.asInstanceOf[BaseShuffleHandle[K, V, _]], mapId, context) + } + + /** Remove a shuffle's metadata from the ShuffleManager. */ + override def unregisterShuffle(shuffleId: Int): Unit = {} + + /** Shut down this ShuffleManager. */ + override def stop(): Unit = {} + + /** Get the location of a block in a map output file. Uses the index file we create for it. */ + def getBlockLocation(blockId: ShuffleBlockId, diskManager: DiskBlockManager): FileSegment = { + // The block is actually going to be a range of a single map output file for this map, so + // figure out the ID of the consolidated file, then the offset within that from our index + val consolidatedId = blockId.copy(reduceId = 0) + val indexFile = diskManager.getFile(consolidatedId.name + ".index") + val in = new DataInputStream(new FileInputStream(indexFile)) + try { + in.skip(blockId.reduceId * 8) + val offset = in.readLong() + val nextOffset = in.readLong() + new FileSegment(diskManager.getFile(consolidatedId), offset, nextOffset - offset) + } finally { + in.close() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala new file mode 100644 index 0000000000000..42fcd07fa18bc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -0,0 +1,165 @@ +/* + * 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.shuffle.sort + +import java.io.{BufferedOutputStream, File, FileOutputStream, DataOutputStream} + +import org.apache.spark.{MapOutputTracker, SparkEnv, Logging, TaskContext} +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{ShuffleWriter, BaseShuffleHandle} +import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.util.collection.ExternalSorter + +private[spark] class SortShuffleWriter[K, V, C]( + handle: BaseShuffleHandle[K, V, C], + mapId: Int, + context: TaskContext) + extends ShuffleWriter[K, V] with Logging { + + private val dep = handle.dependency + private val numPartitions = dep.partitioner.numPartitions + + private val blockManager = SparkEnv.get.blockManager + private val ser = Serializer.getSerializer(dep.serializer.orNull) + + private val conf = SparkEnv.get.conf + private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 + + private var sorter: ExternalSorter[K, V, _] = null + private var outputFile: File = null + + // Are we in the process of stopping? Because map tasks can call stop() with success = true + // and then call stop() with success = false if they get an exception, we want to make sure + // we don't try deleting files, etc twice. + private var stopping = false + + private var mapStatus: MapStatus = null + + /** Write a bunch of records to this task's output */ + override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { + // Get an iterator with the elements for each partition ID + val partitions: Iterator[(Int, Iterator[Product2[K, _]])] = { + if (dep.mapSideCombine) { + if (!dep.aggregator.isDefined) { + throw new IllegalStateException("Aggregator is empty for map-side combine") + } + sorter = new ExternalSorter[K, V, C]( + dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) + sorter.write(records) + sorter.partitionedIterator + } else { + // In this case we pass neither an aggregator nor an ordering to the sorter, because we + // don't care whether the keys get sorted in each partition; that will be done on the + // reduce side if the operation being run is sortByKey. + sorter = new ExternalSorter[K, V, V]( + None, Some(dep.partitioner), None, dep.serializer) + sorter.write(records) + sorter.partitionedIterator + } + } + + // Create a single shuffle file with reduce ID 0 that we'll write all results to. We'll later + // serve different ranges of this file using an index file that we create at the end. + val blockId = ShuffleBlockId(dep.shuffleId, mapId, 0) + outputFile = blockManager.diskBlockManager.getFile(blockId) + + // Track location of each range in the output file + val offsets = new Array[Long](numPartitions + 1) + val lengths = new Array[Long](numPartitions) + + // Statistics + var totalBytes = 0L + var totalTime = 0L + + for ((id, elements) <- partitions) { + if (elements.hasNext) { + val writer = blockManager.getDiskWriter(blockId, outputFile, ser, fileBufferSize) + for (elem <- elements) { + writer.write(elem) + } + writer.commit() + writer.close() + val segment = writer.fileSegment() + offsets(id + 1) = segment.offset + segment.length + lengths(id) = segment.length + totalTime += writer.timeWriting() + totalBytes += segment.length + } else { + // The partition is empty; don't create a new writer to avoid writing headers, etc + offsets(id + 1) = offsets(id) + } + } + + val shuffleMetrics = new ShuffleWriteMetrics + shuffleMetrics.shuffleBytesWritten = totalBytes + shuffleMetrics.shuffleWriteTime = totalTime + context.taskMetrics.shuffleWriteMetrics = Some(shuffleMetrics) + context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled + + // Write an index file with the offsets of each block, plus a final offset at the end for the + // end of the output file. This will be used by SortShuffleManager.getBlockLocation to figure + // out where each block begins and ends. + + val diskBlockManager = blockManager.diskBlockManager + val indexFile = diskBlockManager.getFile(blockId.name + ".index") + val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) + try { + var i = 0 + while (i < numPartitions + 1) { + out.writeLong(offsets(i)) + i += 1 + } + } finally { + out.close() + } + + // Register our map output with the ShuffleBlockManager, which handles cleaning it over time + blockManager.shuffleBlockManager.addCompletedMap(dep.shuffleId, mapId, numPartitions) + + mapStatus = new MapStatus(blockManager.blockManagerId, + lengths.map(MapOutputTracker.compressSize)) + } + + /** Close this writer, passing along whether the map completed */ + override def stop(success: Boolean): Option[MapStatus] = { + try { + if (stopping) { + return None + } + stopping = true + if (success) { + return Option(mapStatus) + } else { + // The map task failed, so delete our output file if we created one + if (outputFile != null) { + outputFile.delete() + } + return None + } + } finally { + // Clean up our sorter, which may have its own intermediate files + if (sorter != null) { + sorter.stop() + sorter = null + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 42ec181b00bb3..c1756ac905417 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -54,11 +54,15 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { } @DeveloperApi -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) - extends BlockId { +case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } +@DeveloperApi +case class ShuffleIndexBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { + def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId + ".index" +} + @DeveloperApi case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) @@ -88,6 +92,7 @@ private[spark] case class TestBlockId(id: String) extends BlockId { object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r + val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r @@ -99,6 +104,8 @@ object BlockId { RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) + case SHUFFLE_INDEX(shuffleId, mapId, reduceId) => + ShuffleIndexBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) case BROADCAST(broadcastId, field) => BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 2e7ed7538e6e5..4d66ccea211fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -21,10 +21,11 @@ import java.io.File import java.text.SimpleDateFormat import java.util.{Date, Random, UUID} -import org.apache.spark.Logging +import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.network.netty.{PathResolver, ShuffleSender} import org.apache.spark.util.Utils +import org.apache.spark.shuffle.sort.SortShuffleManager /** * Creates and maintains the logical mapping between logical blocks and physical on-disk @@ -34,11 +35,13 @@ import org.apache.spark.util.Utils * * @param rootDirs The directories to use for storing block files. Data will be hashed among these. */ -private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootDirs: String) +private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, rootDirs: String) extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = shuffleManager.conf.getInt("spark.diskStore.subDirectories", 64) + + private val subDirsPerLocalDir = + shuffleBlockManager.conf.getInt("spark.diskStore.subDirectories", 64) /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid @@ -54,13 +57,19 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD addShutdownHook() /** - * Returns the physical file segment in which the given BlockId is located. - * If the BlockId has been mapped to a specific FileSegment, that will be returned. - * Otherwise, we assume the Block is mapped to a whole file identified by the BlockId directly. + * Returns the physical file segment in which the given BlockId is located. If the BlockId has + * been mapped to a specific FileSegment by the shuffle layer, that will be returned. + * Otherwise, we assume the Block is mapped to the whole file identified by the BlockId. */ def getBlockLocation(blockId: BlockId): FileSegment = { - if (blockId.isShuffle && shuffleManager.consolidateShuffleFiles) { - shuffleManager.getBlockLocation(blockId.asInstanceOf[ShuffleBlockId]) + val env = SparkEnv.get // NOTE: can be null in unit tests + if (blockId.isShuffle && env != null && env.shuffleManager.isInstanceOf[SortShuffleManager]) { + // For sort-based shuffle, let it figure out its blocks + val sortShuffleManager = env.shuffleManager.asInstanceOf[SortShuffleManager] + sortShuffleManager.getBlockLocation(blockId.asInstanceOf[ShuffleBlockId], this) + } else if (blockId.isShuffle && shuffleBlockManager.consolidateShuffleFiles) { + // For hash-based shuffle with consolidated files, ShuffleBlockManager takes care of this + shuffleBlockManager.getBlockLocation(blockId.asInstanceOf[ShuffleBlockId]) } else { val file = getFile(blockId.name) new FileSegment(file, 0, file.length()) @@ -99,13 +108,18 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD getBlockLocation(blockId).file.exists() } - /** List all the blocks currently stored on disk by the disk manager. */ - def getAllBlocks(): Seq[BlockId] = { + /** List all the files currently stored on disk by the disk manager. */ + def getAllFiles(): Seq[File] = { // Get all the files inside the array of array of directories subDirs.flatten.filter(_ != null).flatMap { dir => - val files = dir.list() + val files = dir.listFiles() if (files != null) files else Seq.empty - }.map(BlockId.apply) + } + } + + /** List all the blocks currently stored on disk by the disk manager. */ + def getAllBlocks(): Seq[BlockId] = { + getAllFiles().map(f => BlockId(f.getName)) } /** Produces a unique block id and File suitable for intermediate results. */ diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 35910e552fe86..7beb55c411e71 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -28,6 +28,7 @@ import org.apache.spark.serializer.Serializer import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} +import org.apache.spark.shuffle.sort.SortShuffleManager /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -58,6 +59,7 @@ private[spark] trait ShuffleWriterGroup { * each block stored in each file. In order to find the location of a shuffle block, we search the * files within a ShuffleFileGroups associated with the block's reducer. */ +// TODO: Factor this into a separate class for each ShuffleManager implementation private[spark] class ShuffleBlockManager(blockManager: BlockManager) extends Logging { def conf = blockManager.conf @@ -67,6 +69,10 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { val consolidateShuffleFiles = conf.getBoolean("spark.shuffle.consolidateFiles", false) + // Are we using sort-based shuffle? + val sortBasedShuffle = + conf.get("spark.shuffle.manager", "") == classOf[SortShuffleManager].getName + private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 /** @@ -91,6 +97,20 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SHUFFLE_BLOCK_MANAGER, this.cleanup, conf) + /** + * Register a completed map without getting a ShuffleWriterGroup. Used by sort-based shuffle + * because it just writes a single file by itself. + */ + def addCompletedMap(shuffleId: Int, mapId: Int, numBuckets: Int): Unit = { + shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) + val shuffleState = shuffleStates(shuffleId) + shuffleState.completedMapTasks.add(mapId) + } + + /** + * Get a ShuffleWriterGroup for the given map task, which will register it as complete + * when the writers are closed successfully + */ def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = { new ShuffleWriterGroup { shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) @@ -182,7 +202,14 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { shuffleStates.get(shuffleId) match { case Some(state) => - if (consolidateShuffleFiles) { + if (sortBasedShuffle) { + // There's a single block ID for each map, plus an index file for it + for (mapId <- state.completedMapTasks) { + val blockId = new ShuffleBlockId(shuffleId, mapId, 0) + blockManager.diskBlockManager.getFile(blockId).delete() + blockManager.diskBlockManager.getFile(blockId.name + ".index").delete() + } + } else if (consolidateShuffleFiles) { for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { file.delete() } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 6f263c39d1435..b34512ef9eb60 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -79,12 +79,16 @@ class ExternalAppendOnlyMap[K, V, C]( (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } - // Number of pairs in the in-memory map - private var numPairsInMemory = 0L + // Number of pairs inserted since last spill; note that we count them even if a value is merged + // with a previous key in case we're doing something like groupBy where the result grows + private var elementsRead = 0L // Number of in-memory pairs inserted before tracking the map's shuffle memory usage private val trackMemoryThreshold = 1000 + // How much of the shared memory pool this collection has claimed + private var myMemoryThreshold = 0L + /** * Size of object batches when reading/writing from serializers. * @@ -106,7 +110,6 @@ class ExternalAppendOnlyMap[K, V, C]( private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 private val keyComparator = new HashComparator[K] private val ser = serializer.newInstance() - private val threadId = Thread.currentThread().getId /** * Insert the given key and value into the map. @@ -134,31 +137,35 @@ class ExternalAppendOnlyMap[K, V, C]( while (entries.hasNext) { curEntry = entries.next() - if (numPairsInMemory > trackMemoryThreshold && currentMap.atGrowThreshold) { - val mapSize = currentMap.estimateSize() + if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && + currentMap.estimateSize() >= myMemoryThreshold) + { + val currentSize = currentMap.estimateSize() var shouldSpill = false val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap // Atomically check whether there is sufficient memory in the global pool for // this map to grow and, if possible, allocate the required amount shuffleMemoryMap.synchronized { + val threadId = Thread.currentThread().getId val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) val availableMemory = maxMemoryThreshold - (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) - // Assume map growth factor is 2x - shouldSpill = availableMemory < mapSize * 2 + // Try to allocate at least 2x more memory, otherwise spill + shouldSpill = availableMemory < currentSize * 2 if (!shouldSpill) { - shuffleMemoryMap(threadId) = mapSize * 2 + shuffleMemoryMap(threadId) = currentSize * 2 + myMemoryThreshold = currentSize * 2 } } // Do not synchronize spills if (shouldSpill) { - spill(mapSize) + spill(currentSize) } } currentMap.changeValue(curEntry._1, update) - numPairsInMemory += 1 + elementsRead += 1 } } @@ -178,9 +185,10 @@ class ExternalAppendOnlyMap[K, V, C]( /** * Sort the existing contents of the in-memory map and spill them to a temporary file on disk. */ - private def spill(mapSize: Long) { + private def spill(mapSize: Long): Unit = { spillCount += 1 - logWarning("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" + val threadId = Thread.currentThread().getId + logInfo("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" .format(threadId, mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) @@ -227,7 +235,9 @@ class ExternalAppendOnlyMap[K, V, C]( shuffleMemoryMap.synchronized { shuffleMemoryMap(Thread.currentThread().getId) = 0 } - numPairsInMemory = 0 + myMemoryThreshold = 0 + + elementsRead = 0 _memoryBytesSpilled += mapSize } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala new file mode 100644 index 0000000000000..54c3310744136 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -0,0 +1,662 @@ +/* + * 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.util.collection + +import java.io._ +import java.util.Comparator + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable + +import com.google.common.io.ByteStreams + +import org.apache.spark.{Aggregator, SparkEnv, Logging, Partitioner} +import org.apache.spark.serializer.Serializer +import org.apache.spark.storage.BlockId + +/** + * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner + * pairs of type (K, C). Uses a Partitioner to first group the keys into partitions, and then + * optionally sorts keys within each partition using a custom Comparator. Can output a single + * partitioned file with a different byte range for each partition, suitable for shuffle fetches. + * + * If combining is disabled, the type C must equal V -- we'll cast the objects at the end. + * + * @param aggregator optional Aggregator with combine functions to use for merging data + * @param partitioner optional Partitioner; if given, sort by partition ID and then key + * @param ordering optional Ordering to sort keys within each partition; should be a total ordering + * @param serializer serializer to use when spilling to disk + * + * Note that if an Ordering is given, we'll always sort using it, so only provide it if you really + * want the output keys to be sorted. In a map task without map-side combine for example, you + * probably want to pass None as the ordering to avoid extra sorting. On the other hand, if you do + * want to do combining, having an Ordering is more efficient than not having it. + * + * At a high level, this class works as follows: + * + * - We repeatedly fill up buffers of in-memory data, using either a SizeTrackingAppendOnlyMap if + * we want to combine by key, or an simple SizeTrackingBuffer if we don't. Inside these buffers, + * we sort elements of type ((Int, K), C) where the Int is the partition ID. This is done to + * avoid calling the partitioner multiple times on the same key (e.g. for RangePartitioner). + * + * - When each buffer reaches our memory limit, we spill it to a file. This file is sorted first + * by partition ID and possibly second by key or by hash code of the key, if we want to do + * aggregation. For each file, we track how many objects were in each partition in memory, so we + * don't have to write out the partition ID for every element. + * + * - When the user requests an iterator, the spilled files are merged, along with any remaining + * in-memory data, using the same sort order defined above (unless both sorting and aggregation + * are disabled). If we need to aggregate by key, we either use a total ordering from the + * ordering parameter, or read the keys with the same hash code and compare them with each other + * for equality to merge values. + * + * - Users are expected to call stop() at the end to delete all the intermediate files. + */ +private[spark] class ExternalSorter[K, V, C]( + aggregator: Option[Aggregator[K, V, C]] = None, + partitioner: Option[Partitioner] = None, + ordering: Option[Ordering[K]] = None, + serializer: Option[Serializer] = None) extends Logging { + + private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1) + private val shouldPartition = numPartitions > 1 + + private val blockManager = SparkEnv.get.blockManager + private val diskBlockManager = blockManager.diskBlockManager + private val ser = Serializer.getSerializer(serializer) + private val serInstance = ser.newInstance() + + private val conf = SparkEnv.get.conf + private val spillingEnabled = conf.getBoolean("spark.shuffle.spill", true) + private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 100) * 1024 + + // Size of object batches when reading/writing from serializers. + // + // Objects are written in batches, with each batch using its own serialization stream. This + // cuts down on the size of reference-tracking maps constructed when deserializing a stream. + // + // NOTE: Setting this too low can cause excessive copying when serializing, since some serializers + // grow internal data structures by growing + copying every time the number of objects doubles. + private val serializerBatchSize = conf.getLong("spark.shuffle.spill.batchSize", 10000) + + private def getPartition(key: K): Int = { + if (shouldPartition) partitioner.get.getPartition(key) else 0 + } + + // Data structures to store in-memory objects before we spill. Depending on whether we have an + // Aggregator set, we either put objects into an AppendOnlyMap where we combine them, or we + // store them in an array buffer. + private var map = new SizeTrackingAppendOnlyMap[(Int, K), C] + private var buffer = new SizeTrackingPairBuffer[(Int, K), C] + + // Number of pairs read from input since last spill; note that we count them even if a value is + // merged with a previous key in case we're doing something like groupBy where the result grows + private var elementsRead = 0L + + // What threshold of elementsRead we start estimating map size at. + private val trackMemoryThreshold = 1000 + + // Spilling statistics + private var spillCount = 0 + private var _memoryBytesSpilled = 0L + private var _diskBytesSpilled = 0L + + // Collective memory threshold shared across all running tasks + private val maxMemoryThreshold = { + val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) + val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong + } + + // How much of the shared memory pool this collection has claimed + private var myMemoryThreshold = 0L + + // A comparator for keys K that orders them within a partition to allow aggregation or sorting. + // Can be a partial ordering by hash code if a total ordering is not provided through by the + // user. (A partial ordering means that equal keys have comparator.compare(k, k) = 0, but some + // non-equal keys also have this, so we need to do a later pass to find truly equal keys). + // Note that we ignore this if no aggregator and no ordering are given. + private val keyComparator: Comparator[K] = ordering.getOrElse(new Comparator[K] { + override def compare(a: K, b: K): Int = { + val h1 = if (a == null) 0 else a.hashCode() + val h2 = if (b == null) 0 else b.hashCode() + h1 - h2 + } + }) + + // A comparator for (Int, K) elements that orders them by partition and then possibly by key + private val partitionKeyComparator: Comparator[(Int, K)] = { + if (ordering.isDefined || aggregator.isDefined) { + // Sort by partition ID then key comparator + new Comparator[(Int, K)] { + override def compare(a: (Int, K), b: (Int, K)): Int = { + val partitionDiff = a._1 - b._1 + if (partitionDiff != 0) { + partitionDiff + } else { + keyComparator.compare(a._2, b._2) + } + } + } + } else { + // Just sort it by partition ID + new Comparator[(Int, K)] { + override def compare(a: (Int, K), b: (Int, K)): Int = { + a._1 - b._1 + } + } + } + } + + // Information about a spilled file. Includes sizes in bytes of "batches" written by the + // serializer as we periodically reset its stream, as well as number of elements in each + // partition, used to efficiently keep track of partitions when merging. + private[this] case class SpilledFile( + file: File, + blockId: BlockId, + serializerBatchSizes: Array[Long], + elementsPerPartition: Array[Long]) + private val spills = new ArrayBuffer[SpilledFile] + + def write(records: Iterator[_ <: Product2[K, V]]): Unit = { + // TODO: stop combining if we find that the reduction factor isn't high + val shouldCombine = aggregator.isDefined + + if (shouldCombine) { + // Combine values in-memory first using our AppendOnlyMap + val mergeValue = aggregator.get.mergeValue + val createCombiner = aggregator.get.createCombiner + var kv: Product2[K, V] = null + val update = (hadValue: Boolean, oldValue: C) => { + if (hadValue) mergeValue(oldValue, kv._2) else createCombiner(kv._2) + } + while (records.hasNext) { + elementsRead += 1 + kv = records.next() + map.changeValue((getPartition(kv._1), kv._1), update) + maybeSpill(usingMap = true) + } + } else { + // Stick values into our buffer + while (records.hasNext) { + elementsRead += 1 + val kv = records.next() + buffer.insert((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) + maybeSpill(usingMap = false) + } + } + } + + /** + * Spill the current in-memory collection to disk if needed. + * + * @param usingMap whether we're using a map or buffer as our current in-memory collection + */ + private def maybeSpill(usingMap: Boolean): Unit = { + if (!spillingEnabled) { + return + } + + val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer + + // TODO: factor this out of both here and ExternalAppendOnlyMap + if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && + collection.estimateSize() >= myMemoryThreshold) + { + // TODO: This logic doesn't work if there are two external collections being used in the same + // task (e.g. to read shuffle output and write it out into another shuffle) [SPARK-2711] + + val currentSize = collection.estimateSize() + var shouldSpill = false + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + + // Atomically check whether there is sufficient memory in the global pool for + // us to double our threshold + shuffleMemoryMap.synchronized { + val threadId = Thread.currentThread().getId + val previouslyClaimedMemory = shuffleMemoryMap.get(threadId) + val availableMemory = maxMemoryThreshold - + (shuffleMemoryMap.values.sum - previouslyClaimedMemory.getOrElse(0L)) + + // Try to allocate at least 2x more memory, otherwise spill + shouldSpill = availableMemory < currentSize * 2 + if (!shouldSpill) { + shuffleMemoryMap(threadId) = currentSize * 2 + myMemoryThreshold = currentSize * 2 + } + } + // Do not hold lock during spills + if (shouldSpill) { + spill(currentSize, usingMap) + } + } + } + + /** + * Spill the current in-memory collection to disk, adding a new file to spills, and clear it. + * + * @param usingMap whether we're using a map or buffer as our current in-memory collection + */ + private def spill(memorySize: Long, usingMap: Boolean): Unit = { + val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer + val memorySize = collection.estimateSize() + + spillCount += 1 + val threadId = Thread.currentThread().getId + logInfo("Thread %d spilling in-memory batch of %d MB to disk (%d spill%s so far)" + .format(threadId, memorySize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) + val (blockId, file) = diskBlockManager.createTempBlock() + var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize) + var objectsWritten = 0 // Objects written since the last flush + + // List of batch sizes (bytes) in the order they are written to disk + val batchSizes = new ArrayBuffer[Long] + + // How many elements we have in each partition + val elementsPerPartition = new Array[Long](numPartitions) + + // Flush the disk writer's contents to disk, and update relevant variables + def flush() = { + writer.commit() + val bytesWritten = writer.bytesWritten + batchSizes.append(bytesWritten) + _diskBytesSpilled += bytesWritten + objectsWritten = 0 + } + + try { + val it = collection.destructiveSortedIterator(partitionKeyComparator) + while (it.hasNext) { + val elem = it.next() + val partitionId = elem._1._1 + val key = elem._1._2 + val value = elem._2 + writer.write(key) + writer.write(value) + elementsPerPartition(partitionId) += 1 + objectsWritten += 1 + + if (objectsWritten == serializerBatchSize) { + flush() + writer.close() + writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize) + } + } + if (objectsWritten > 0) { + flush() + } + writer.close() + } catch { + case e: Exception => + writer.close() + file.delete() + throw e + } + + if (usingMap) { + map = new SizeTrackingAppendOnlyMap[(Int, K), C] + } else { + buffer = new SizeTrackingPairBuffer[(Int, K), C] + } + + // Reset the amount of shuffle memory used by this map in the global pool + val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap + shuffleMemoryMap.synchronized { + shuffleMemoryMap(Thread.currentThread().getId) = 0 + } + myMemoryThreshold = 0 + + spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition)) + _memoryBytesSpilled += memorySize + } + + /** + * Merge a sequence of sorted files, giving an iterator over partitions and then over elements + * inside each partition. This can be used to either write out a new file or return data to + * the user. + * + * Returns an iterator over all the data written to this object, grouped by partition. For each + * partition we then have an iterator over its contents, and these are expected to be accessed + * in order (you can't "skip ahead" to one partition without reading the previous one). + * Guaranteed to return a key-value pair for each partition, in order of partition ID. + */ + private def merge(spills: Seq[SpilledFile], inMemory: Iterator[((Int, K), C)]) + : Iterator[(Int, Iterator[Product2[K, C]])] = { + val readers = spills.map(new SpillReader(_)) + val inMemBuffered = inMemory.buffered + (0 until numPartitions).iterator.map { p => + val inMemIterator = new IteratorForPartition(p, inMemBuffered) + val iterators = readers.map(_.readNextPartition()) ++ Seq(inMemIterator) + if (aggregator.isDefined) { + // Perform partial aggregation across partitions + (p, mergeWithAggregation( + iterators, aggregator.get.mergeCombiners, keyComparator, ordering.isDefined)) + } else if (ordering.isDefined) { + // No aggregator given, but we have an ordering (e.g. used by reduce tasks in sortByKey); + // sort the elements without trying to merge them + (p, mergeSort(iterators, ordering.get)) + } else { + (p, iterators.iterator.flatten) + } + } + } + + /** + * Merge-sort a sequence of (K, C) iterators using a given a comparator for the keys. + */ + private def mergeSort(iterators: Seq[Iterator[Product2[K, C]]], comparator: Comparator[K]) + : Iterator[Product2[K, C]] = + { + val bufferedIters = iterators.filter(_.hasNext).map(_.buffered) + type Iter = BufferedIterator[Product2[K, C]] + val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] { + // Use the reverse of comparator.compare because PriorityQueue dequeues the max + override def compare(x: Iter, y: Iter): Int = -comparator.compare(x.head._1, y.head._1) + }) + heap.enqueue(bufferedIters: _*) // Will contain only the iterators with hasNext = true + new Iterator[Product2[K, C]] { + override def hasNext: Boolean = !heap.isEmpty + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val firstBuf = heap.dequeue() + val firstPair = firstBuf.next() + if (firstBuf.hasNext) { + heap.enqueue(firstBuf) + } + firstPair + } + } + } + + /** + * Merge a sequence of (K, C) iterators by aggregating values for each key, assuming that each + * iterator is sorted by key with a given comparator. If the comparator is not a total ordering + * (e.g. when we sort objects by hash code and different keys may compare as equal although + * they're not), we still merge them by doing equality tests for all keys that compare as equal. + */ + private def mergeWithAggregation( + iterators: Seq[Iterator[Product2[K, C]]], + mergeCombiners: (C, C) => C, + comparator: Comparator[K], + totalOrder: Boolean) + : Iterator[Product2[K, C]] = + { + if (!totalOrder) { + // We only have a partial ordering, e.g. comparing the keys by hash code, which means that + // multiple distinct keys might be treated as equal by the ordering. To deal with this, we + // need to read all keys considered equal by the ordering at once and compare them. + new Iterator[Iterator[Product2[K, C]]] { + val sorted = mergeSort(iterators, comparator).buffered + + // Buffers reused across elements to decrease memory allocation + val keys = new ArrayBuffer[K] + val combiners = new ArrayBuffer[C] + + override def hasNext: Boolean = sorted.hasNext + + override def next(): Iterator[Product2[K, C]] = { + if (!hasNext) { + throw new NoSuchElementException + } + keys.clear() + combiners.clear() + val firstPair = sorted.next() + keys += firstPair._1 + combiners += firstPair._2 + val key = firstPair._1 + while (sorted.hasNext && comparator.compare(sorted.head._1, key) == 0) { + val pair = sorted.next() + var i = 0 + var foundKey = false + while (i < keys.size && !foundKey) { + if (keys(i) == pair._1) { + combiners(i) = mergeCombiners(combiners(i), pair._2) + foundKey = true + } + i += 1 + } + if (!foundKey) { + keys += pair._1 + combiners += pair._2 + } + } + + // Note that we return an iterator of elements since we could've had many keys marked + // equal by the partial order; we flatten this below to get a flat iterator of (K, C). + keys.iterator.zip(combiners.iterator) + } + }.flatMap(i => i) + } else { + // We have a total ordering, so the objects with the same key are sequential. + new Iterator[Product2[K, C]] { + val sorted = mergeSort(iterators, comparator).buffered + + override def hasNext: Boolean = sorted.hasNext + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val elem = sorted.next() + val k = elem._1 + var c = elem._2 + while (sorted.hasNext && sorted.head._1 == k) { + c = mergeCombiners(c, sorted.head._2) + } + (k, c) + } + } + } + } + + /** + * An internal class for reading a spilled file partition by partition. Expects all the + * partitions to be requested in order. + */ + private[this] class SpillReader(spill: SpilledFile) { + val fileStream = new FileInputStream(spill.file) + val bufferedStream = new BufferedInputStream(fileStream, fileBufferSize) + + // Track which partition and which batch stream we're in. These will be the indices of + // the next element we will read. We'll also store the last partition read so that + // readNextPartition() can figure out what partition that was from. + var partitionId = 0 + var indexInPartition = 0L + var batchStreamsRead = 0 + var indexInBatch = 0 + var lastPartitionId = 0 + + skipToNextPartition() + + // An intermediate stream that reads from exactly one batch + // This guards against pre-fetching and other arbitrary behavior of higher level streams + var batchStream = nextBatchStream() + var compressedStream = blockManager.wrapForCompression(spill.blockId, batchStream) + var deserStream = serInstance.deserializeStream(compressedStream) + var nextItem: (K, C) = null + var finished = false + + /** Construct a stream that only reads from the next batch */ + def nextBatchStream(): InputStream = { + if (batchStreamsRead < spill.serializerBatchSizes.length) { + batchStreamsRead += 1 + ByteStreams.limit(bufferedStream, spill.serializerBatchSizes(batchStreamsRead - 1)) + } else { + // No more batches left; give an empty stream + bufferedStream + } + } + + /** + * Update partitionId if we have reached the end of our current partition, possibly skipping + * empty partitions on the way. + */ + private def skipToNextPartition() { + while (partitionId < numPartitions && + indexInPartition == spill.elementsPerPartition(partitionId)) { + partitionId += 1 + indexInPartition = 0L + } + } + + /** + * Return the next (K, C) pair from the deserialization stream and update partitionId, + * indexInPartition, indexInBatch and such to match its location. + * + * If the current batch is drained, construct a stream for the next batch and read from it. + * If no more pairs are left, return null. + */ + private def readNextItem(): (K, C) = { + if (finished) { + return null + } + val k = deserStream.readObject().asInstanceOf[K] + val c = deserStream.readObject().asInstanceOf[C] + lastPartitionId = partitionId + // Start reading the next batch if we're done with this one + indexInBatch += 1 + if (indexInBatch == serializerBatchSize) { + batchStream = nextBatchStream() + compressedStream = blockManager.wrapForCompression(spill.blockId, batchStream) + deserStream = serInstance.deserializeStream(compressedStream) + indexInBatch = 0 + } + // Update the partition location of the element we're reading + indexInPartition += 1 + skipToNextPartition() + // If we've finished reading the last partition, remember that we're done + if (partitionId == numPartitions) { + finished = true + deserStream.close() + } + (k, c) + } + + var nextPartitionToRead = 0 + + def readNextPartition(): Iterator[Product2[K, C]] = new Iterator[Product2[K, C]] { + val myPartition = nextPartitionToRead + nextPartitionToRead += 1 + + override def hasNext: Boolean = { + if (nextItem == null) { + nextItem = readNextItem() + if (nextItem == null) { + return false + } + } + assert(lastPartitionId >= myPartition) + // Check that we're still in the right partition; note that readNextItem will have returned + // null at EOF above so we would've returned false there + lastPartitionId == myPartition + } + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val item = nextItem + nextItem = null + item + } + } + } + + /** + * Return an iterator over all the data written to this object, grouped by partition and + * aggregated by the requested aggregator. For each partition we then have an iterator over its + * contents, and these are expected to be accessed in order (you can't "skip ahead" to one + * partition without reading the previous one). Guaranteed to return a key-value pair for each + * partition, in order of partition ID. + * + * For now, we just merge all the spilled files in once pass, but this can be modified to + * support hierarchical merging. + */ + def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = { + val usingMap = aggregator.isDefined + val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer + if (spills.isEmpty) { + // Special case: if we have only in-memory data, we don't need to merge streams, and perhaps + // we don't even need to sort by anything other than partition ID + if (!ordering.isDefined) { + // The user isn't requested sorted keys, so only sort by partition ID, not key + val partitionComparator = new Comparator[(Int, K)] { + override def compare(a: (Int, K), b: (Int, K)): Int = { + a._1 - b._1 + } + } + groupByPartition(collection.destructiveSortedIterator(partitionComparator)) + } else { + // We do need to sort by both partition ID and key + groupByPartition(collection.destructiveSortedIterator(partitionKeyComparator)) + } + } else { + // General case: merge spilled and in-memory data + merge(spills, collection.destructiveSortedIterator(partitionKeyComparator)) + } + } + + /** + * Return an iterator over all the data written to this object, aggregated by our aggregator. + */ + def iterator: Iterator[Product2[K, C]] = partitionedIterator.flatMap(pair => pair._2) + + def stop(): Unit = { + spills.foreach(s => s.file.delete()) + spills.clear() + } + + def memoryBytesSpilled: Long = _memoryBytesSpilled + + def diskBytesSpilled: Long = _diskBytesSpilled + + /** + * Given a stream of ((partition, key), combiner) pairs *assumed to be sorted by partition ID*, + * group together the pairs for each partition into a sub-iterator. + * + * @param data an iterator of elements, assumed to already be sorted by partition ID + */ + private def groupByPartition(data: Iterator[((Int, K), C)]) + : Iterator[(Int, Iterator[Product2[K, C]])] = + { + val buffered = data.buffered + (0 until numPartitions).iterator.map(p => (p, new IteratorForPartition(p, buffered))) + } + + /** + * An iterator that reads only the elements for a given partition ID from an underlying buffered + * stream, assuming this partition is the next one to be read. Used to make it easier to return + * partitioned iterators from our in-memory collection. + */ + private[this] class IteratorForPartition(partitionId: Int, data: BufferedIterator[((Int, K), C)]) + extends Iterator[Product2[K, C]] + { + override def hasNext: Boolean = data.hasNext && data.head._1._1 == partitionId + + override def next(): Product2[K, C] = { + if (!hasNext) { + throw new NoSuchElementException + } + val elem = data.next() + (elem._1._2, elem._2) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala index de61e1d17fe10..eb4de413867a0 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingAppendOnlyMap.scala @@ -20,8 +20,9 @@ package org.apache.spark.util.collection /** * An append-only map that keeps track of its estimated size in bytes. */ -private[spark] class SizeTrackingAppendOnlyMap[K, V] extends AppendOnlyMap[K, V] with SizeTracker { - +private[spark] class SizeTrackingAppendOnlyMap[K, V] + extends AppendOnlyMap[K, V] with SizeTracker with SizeTrackingPairCollection[K, V] +{ override def update(key: K, value: V): Unit = { super.update(key, value) super.afterUpdate() diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairBuffer.scala new file mode 100644 index 0000000000000..9e9c16c5a2962 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairBuffer.scala @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.collection + +import java.util.Comparator + +/** + * Append-only buffer of key-value pairs that keeps track of its estimated size in bytes. + */ +private[spark] class SizeTrackingPairBuffer[K, V](initialCapacity: Int = 64) + extends SizeTracker with SizeTrackingPairCollection[K, V] +{ + require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") + require(initialCapacity >= 1, "Invalid initial capacity") + + // Basic growable array data structure. We use a single array of AnyRef to hold both the keys + // and the values, so that we can sort them efficiently with KVArraySortDataFormat. + private var capacity = initialCapacity + private var curSize = 0 + private var data = new Array[AnyRef](2 * initialCapacity) + + /** Add an element into the buffer */ + def insert(key: K, value: V): Unit = { + if (curSize == capacity) { + growArray() + } + data(2 * curSize) = key.asInstanceOf[AnyRef] + data(2 * curSize + 1) = value.asInstanceOf[AnyRef] + curSize += 1 + afterUpdate() + } + + /** Total number of elements in buffer */ + override def size: Int = curSize + + /** Iterate over the elements of the buffer */ + override def iterator: Iterator[(K, V)] = new Iterator[(K, V)] { + var pos = 0 + + override def hasNext: Boolean = pos < curSize + + override def next(): (K, V) = { + if (!hasNext) { + throw new NoSuchElementException + } + val pair = (data(2 * pos).asInstanceOf[K], data(2 * pos + 1).asInstanceOf[V]) + pos += 1 + pair + } + } + + /** Double the size of the array because we've reached capacity */ + private def growArray(): Unit = { + if (capacity == (1 << 29)) { + // Doubling the capacity would create an array bigger than Int.MaxValue, so don't + throw new Exception("Can't grow buffer beyond 2^29 elements") + } + val newCapacity = capacity * 2 + val newArray = new Array[AnyRef](2 * newCapacity) + System.arraycopy(data, 0, newArray, 0, 2 * capacity) + data = newArray + capacity = newCapacity + resetSamples() + } + + /** Iterate through the data in a given order. For this class this is not really destructive. */ + override def destructiveSortedIterator(keyComparator: Comparator[K]): Iterator[(K, V)] = { + new Sorter(new KVArraySortDataFormat[K, AnyRef]).sort(data, 0, curSize, keyComparator) + iterator + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairCollection.scala b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairCollection.scala new file mode 100644 index 0000000000000..faa4e2b12ddb6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/SizeTrackingPairCollection.scala @@ -0,0 +1,34 @@ +/* + * 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.util.collection + +import java.util.Comparator + +/** + * A common interface for our size-tracking collections of key-value pairs, which are used in + * external operations. These all support estimating the size and obtaining a memory-efficient + * sorted iterator. + */ +// TODO: should extend Iterable[Product2[K, V]] instead of (K, V) +private[spark] trait SizeTrackingPairCollection[K, V] extends Iterable[(K, V)] { + /** Estimate the collection's current memory usage in bytes. */ + def estimateSize(): Long + + /** Iterate through the data in a given key order. This may destroy the underlying collection. */ + def destructiveSortedIterator(keyComparator: Comparator[K]): Iterator[(K, V)] +} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d1cb2d9d3a53b..a41914a1a9d0c 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -99,7 +99,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { test("ShuffledRDD") { testRDD(rdd => { // Creating ShuffledRDD directly as PairRDDFunctions.combineByKey produces a MapPartitionedRDD - new ShuffledRDD[Int, Int, Int, (Int, Int)](rdd.map(x => (x % 2, 1)), partitioner) + new ShuffledRDD[Int, Int, Int](rdd.map(x => (x % 2, 1)), partitioner) }) } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index ad20f9b937ac1..4bc4346c0a288 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark import java.lang.ref.WeakReference -import org.apache.spark.broadcast.Broadcast - -import scala.collection.mutable import scala.collection.mutable.{HashSet, SynchronizedSet} import scala.language.existentials import scala.language.postfixOps @@ -34,15 +31,28 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockId, BroadcastBlockId, RDDBlockId, ShuffleBlockId} - -class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - +import org.apache.spark.storage._ +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.storage.BroadcastBlockId +import org.apache.spark.storage.RDDBlockId +import org.apache.spark.storage.ShuffleBlockId +import org.apache.spark.storage.ShuffleIndexBlockId + +/** + * An abstract base class for context cleaner tests, which sets up a context with a config + * suitable for cleaner tests and provides some utility functions. Subclasses can use different + * config options, in particular, a different shuffle manager class + */ +abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[HashShuffleManager]) + extends FunSuite with BeforeAndAfter with LocalSparkContext +{ implicit val defaultTimeout = timeout(10000 millis) val conf = new SparkConf() .setMaster("local[2]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.shuffle.manager", shuffleManager.getName) before { sc = new SparkContext(conf) @@ -55,6 +65,59 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo } } + //------ Helper functions ------ + + protected def newRDD() = sc.makeRDD(1 to 10) + protected def newPairRDD() = newRDD().map(_ -> 1) + protected def newShuffleRDD() = newPairRDD().reduceByKey(_ + _) + protected def newBroadcast() = sc.broadcast(1 to 100) + + protected def newRDDWithShuffleDependencies(): (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { + def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { + rdd.dependencies ++ rdd.dependencies.flatMap { dep => + getAllDependencies(dep.rdd) + } + } + val rdd = newShuffleRDD() + + // Get all the shuffle dependencies + val shuffleDeps = getAllDependencies(rdd) + .filter(_.isInstanceOf[ShuffleDependency[_, _, _]]) + .map(_.asInstanceOf[ShuffleDependency[_, _, _]]) + (rdd, shuffleDeps) + } + + protected def randomRdd() = { + val rdd: RDD[_] = Random.nextInt(3) match { + case 0 => newRDD() + case 1 => newShuffleRDD() + case 2 => newPairRDD.join(newPairRDD()) + } + if (Random.nextBoolean()) rdd.persist() + rdd.count() + rdd + } + + /** Run GC and make sure it actually has run */ + protected def runGC() { + val weakRef = new WeakReference(new Object()) + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + // Wait until a weak reference object has been GCed + while (System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + Thread.sleep(200) + } + } + + protected def cleaner = sc.cleaner.get +} + + +/** + * Basic ContextCleanerSuite, which uses sort-based shuffle + */ +class ContextCleanerSuite extends ContextCleanerSuiteBase { test("cleanup RDD") { val rdd = newRDD().persist() val collected = rdd.collect().toList @@ -147,7 +210,7 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId val broadcastIds = broadcastBuffer.map(_.id) @@ -180,12 +243,13 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) val numRdds = 10 val numBroadcasts = 4 // Broadcasts are more costly val rddBuffer = (1 to numRdds).map(i => randomRdd()).toBuffer - val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast()).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast()).toBuffer val rddIds = sc.persistentRdds.keys.toSeq val shuffleIds = 0 until sc.newShuffleId val broadcastIds = broadcastBuffer.map(_.id) @@ -210,57 +274,82 @@ class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkCo case _ => false }, askSlaves = true).isEmpty) } +} - //------ Helper functions ------ - private def newRDD() = sc.makeRDD(1 to 10) - private def newPairRDD() = newRDD().map(_ -> 1) - private def newShuffleRDD() = newPairRDD().reduceByKey(_ + _) - private def newBroadcast() = sc.broadcast(1 to 100) +/** + * A copy of the shuffle tests for sort-based shuffle + */ +class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[SortShuffleManager]) { + test("cleanup shuffle") { + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies() + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) - private def newRDDWithShuffleDependencies(): (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { - def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { - rdd.dependencies ++ rdd.dependencies.flatMap { dep => - getAllDependencies(dep.rdd) - } - } - val rdd = newShuffleRDD() + // Explicit cleanup + shuffleDeps.foreach(s => cleaner.doCleanupShuffle(s.shuffleId, blocking = true)) + tester.assertCleanup() - // Get all the shuffle dependencies - val shuffleDeps = getAllDependencies(rdd) - .filter(_.isInstanceOf[ShuffleDependency[_, _, _]]) - .map(_.asInstanceOf[ShuffleDependency[_, _, _]]) - (rdd, shuffleDeps) + // Verify that shuffles can be re-executed after cleaning up + assert(rdd.collect().toList.equals(collected)) } - private def randomRdd() = { - val rdd: RDD[_] = Random.nextInt(3) match { - case 0 => newRDD() - case 1 => newShuffleRDD() - case 2 => newPairRDD.join(newPairRDD()) - } - if (Random.nextBoolean()) rdd.persist() + test("automatically cleanup shuffle") { + var rdd = newShuffleRDD() rdd.count() - rdd - } - private def randomBroadcast() = { - sc.broadcast(Random.nextInt(Int.MaxValue)) + // Test that GC does not cause shuffle cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes shuffle cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + rdd = null // Make RDD out of scope, so that corresponding shuffle goes out of scope + runGC() + postGCTester.assertCleanup() } - /** Run GC and make sure it actually has run */ - private def runGC() { - val weakRef = new WeakReference(new Object()) - val startTime = System.currentTimeMillis - System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. - // Wait until a weak reference object has been GCed - while (System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { - System.gc() - Thread.sleep(200) + test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { + sc.stop() + + val conf2 = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.shuffle.manager", shuffleManager.getName) + sc = new SparkContext(conf2) + + val numRdds = 10 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => newBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId() + val broadcastIds = broadcastBuffer.map(_.id) + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) } - } - private def cleaner = sc.cleaner.get + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + + // Make sure the broadcasted task closure no longer exists after GC. + val taskClosureBroadcastId = broadcastIds.max + 1 + assert(sc.env.blockManager.master.getMatchingBlockIds({ + case BroadcastBlockId(`taskClosureBroadcastId`, _) => true + case _ => false + }, askSlaves = true).isEmpty) + } } @@ -418,6 +507,7 @@ class CleanerTester( private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { blockManager.master.getMatchingBlockIds( _ match { case ShuffleBlockId(`shuffleId`, _, _) => true + case ShuffleIndexBlockId(`shuffleId`, _, _) => true case _ => false }, askSlaves = true) } diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index 47df00050c1e2..d7b2d2e1e330f 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -28,6 +28,6 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { } override def afterAll() { - System.setProperty("spark.shuffle.use.netty", "false") + System.clearProperty("spark.shuffle.use.netty") } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index eae67c7747e82..b13ddf96bc77c 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -58,8 +58,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, - NonJavaSerializableClass, - (Int, NonJavaSerializableClass)](b, new HashPartitioner(NUM_BLOCKS)) + NonJavaSerializableClass](b, new HashPartitioner(NUM_BLOCKS)) c.setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId @@ -83,8 +82,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, - NonJavaSerializableClass, - (Int, NonJavaSerializableClass)](b, new HashPartitioner(3)) + NonJavaSerializableClass](b, new HashPartitioner(3)) c.setSerializer(new KryoSerializer(conf)) assert(c.count === 10) } @@ -100,7 +98,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo - val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) .setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId @@ -126,7 +124,7 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer should create zero-sized blocks - val c = new ShuffledRDD[Int, Int, Int, (Int, Int)](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) @@ -141,19 +139,19 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { assert(nonEmptyBlocks.size <= 4) } - test("shuffle using mutable pairs") { + test("shuffle on mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test") def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) - val results = new ShuffledRDD[Int, Int, Int, MutablePair[Int, Int]](pairs, + val results = new ShuffledRDD[Int, Int, Int](pairs, new HashPartitioner(2)).collect() - data.foreach { pair => results should contain (pair) } + data.foreach { pair => results should contain ((pair._1, pair._2)) } } - test("sorting using mutable pairs") { + test("sorting on mutable pairs") { // This is not in SortingSuite because of the local cluster setup. // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test") @@ -162,10 +160,10 @@ class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs) .sortByKey().collect() - results(0) should be (p(1, 11)) - results(1) should be (p(2, 22)) - results(2) should be (p(3, 33)) - results(3) should be (p(100, 100)) + results(0) should be ((1, 11)) + results(1) should be ((2, 22)) + results(2) should be ((3, 33)) + results(3) should be ((100, 100)) } test("cogroup using mutable pairs") { diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala new file mode 100644 index 0000000000000..5c02c00586ef4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.scalatest.BeforeAndAfterAll + +class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { + + // This test suite should run all tests in ShuffleSuite with sort-based shuffle. + + override def beforeAll() { + System.setProperty("spark.shuffle.manager", + "org.apache.spark.shuffle.sort.SortShuffleManager") + } + + override def afterAll() { + System.clearProperty("spark.shuffle.manager") + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 4953d565ae83a..8966eedd80ebc 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -270,7 +270,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // we can optionally shuffle to keep the upstream parallel val coalesced5 = data.coalesce(1, shuffle = true) val isEquals = coalesced5.dependencies.head.rdd.dependencies.head.rdd. - asInstanceOf[ShuffledRDD[_, _, _, _]] != null + asInstanceOf[ShuffledRDD[_, _, _]] != null assert(isEquals) // when shuffling, we can increase the number of partitions @@ -730,9 +730,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Any ancestors before the shuffle are not considered assert(ancestors4.size === 0) - assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 0) + assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 0) assert(ancestors5.size === 3) - assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 0) assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 0b7ad184a46d2..7de5df6e1c8bd 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -208,11 +208,8 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val resultA = rddA.reduceByKey(math.max).collect() assert(resultA.length == 50000) resultA.foreach { case(k, v) => - k match { - case 0 => assert(v == 1) - case 25000 => assert(v == 50001) - case 49999 => assert(v == 99999) - case _ => + if (v != k * 2 + 1) { + fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}") } } @@ -221,11 +218,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { val resultB = rddB.groupByKey().collect() assert(resultB.length == 25000) resultB.foreach { case(i, seq) => - i match { - case 0 => assert(seq.toSet == Set[Int](0, 1, 2, 3)) - case 12500 => assert(seq.toSet == Set[Int](50000, 50001, 50002, 50003)) - case 24999 => assert(seq.toSet == Set[Int](99996, 99997, 99998, 99999)) - case _ => + val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3) + if (seq.toSet != expected) { + fail(s"Value for ${i} was wrong: expected ${expected}, got ${seq.toSet}") } } @@ -239,6 +234,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { case 0 => assert(seq1.toSet == Set[Int](0)) assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) + case 1 => + assert(seq1.toSet == Set[Int](1)) + assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) case 5000 => assert(seq1.toSet == Set[Int](5000)) assert(seq2.toSet == Set[Int]()) @@ -369,10 +367,3 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } } - -/** - * A dummy class that always returns the same hash code, to easily test hash collisions - */ -case class FixedHashObject(v: Int, h: Int) extends Serializable { - override def hashCode(): Int = h -} diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala new file mode 100644 index 0000000000000..ddb5df40360e9 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -0,0 +1,566 @@ +/* + * 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.util.collection + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +import org.apache.spark._ +import org.apache.spark.SparkContext._ + +class ExternalSorterSuite extends FunSuite with LocalSparkContext { + test("empty data stream") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + + // Both aggregator and ordering + val sorter = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(3)), Some(ord), None) + assert(sorter.iterator.toSeq === Seq()) + sorter.stop() + + // Only aggregator + val sorter2 = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(3)), None, None) + assert(sorter2.iterator.toSeq === Seq()) + sorter2.stop() + + // Only ordering + val sorter3 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assert(sorter3.iterator.toSeq === Seq()) + sorter3.stop() + + // Neither aggregator nor ordering + val sorter4 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), None, None) + assert(sorter4.iterator.toSeq === Seq()) + sorter4.stop() + } + + test("few elements per partition") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + val elements = Set((1, 1), (2, 2), (5, 5)) + val expected = Set( + (0, Set()), (1, Set((1, 1))), (2, Set((2, 2))), (3, Set()), (4, Set()), + (5, Set((5, 5))), (6, Set())) + + // Both aggregator and ordering + val sorter = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(7)), Some(ord), None) + sorter.write(elements.iterator) + assert(sorter.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter.stop() + + // Only aggregator + val sorter2 = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(7)), None, None) + sorter2.write(elements.iterator) + assert(sorter2.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter2.stop() + + // Only ordering + val sorter3 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), Some(ord), None) + sorter3.write(elements.iterator) + assert(sorter3.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter3.stop() + + // Neither aggregator nor ordering + val sorter4 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), None, None) + sorter4.write(elements.iterator) + assert(sorter4.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) + sorter4.stop() + } + + test("empty partitions with spilling") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + val elements = Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) + + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), None, None) + sorter.write(elements) + assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0) // Make sure it spilled + val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList)) + assert(iter.next() === (0, Nil)) + assert(iter.next() === (1, List((1, 1)))) + assert(iter.next() === (2, (0 until 100000).map(x => (2, 2)).toList)) + assert(iter.next() === (3, Nil)) + assert(iter.next() === (4, Nil)) + assert(iter.next() === (5, List((5, 5)))) + assert(iter.next() === (6, Nil)) + sorter.stop() + } + + test("spilling in local cluster") { + val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + // reduceByKey - should spill ~8 times + val rddA = sc.parallelize(0 until 100000).map(i => (i/2, i)) + val resultA = rddA.reduceByKey(math.max).collect() + assert(resultA.length == 50000) + resultA.foreach { case(k, v) => + if (v != k * 2 + 1) { + fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}") + } + } + + // groupByKey - should spill ~17 times + val rddB = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultB = rddB.groupByKey().collect() + assert(resultB.length == 25000) + resultB.foreach { case(i, seq) => + val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3) + if (seq.toSet != expected) { + fail(s"Value for ${i} was wrong: expected ${expected}, got ${seq.toSet}") + } + } + + // cogroup - should spill ~7 times + val rddC1 = sc.parallelize(0 until 10000).map(i => (i, i)) + val rddC2 = sc.parallelize(0 until 10000).map(i => (i%1000, i)) + val resultC = rddC1.cogroup(rddC2).collect() + assert(resultC.length == 10000) + resultC.foreach { case(i, (seq1, seq2)) => + i match { + case 0 => + assert(seq1.toSet == Set[Int](0)) + assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) + case 1 => + assert(seq1.toSet == Set[Int](1)) + assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) + case 5000 => + assert(seq1.toSet == Set[Int](5000)) + assert(seq2.toSet == Set[Int]()) + case 9999 => + assert(seq1.toSet == Set[Int](9999)) + assert(seq2.toSet == Set[Int]()) + case _ => + } + } + + // larger cogroup - should spill ~7 times + val rddD1 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val rddD2 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val resultD = rddD1.cogroup(rddD2).collect() + assert(resultD.length == 5000) + resultD.foreach { case(i, (seq1, seq2)) => + val expected = Set(i * 2, i * 2 + 1) + if (seq1.toSet != expected) { + fail(s"Value 1 for ${i} was wrong: expected ${expected}, got ${seq1.toSet}") + } + if (seq2.toSet != expected) { + fail(s"Value 2 for ${i} was wrong: expected ${expected}, got ${seq2.toSet}") + } + } + } + + test("spilling in local cluster with many reduce tasks") { + val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + + // reduceByKey - should spill ~4 times per executor + val rddA = sc.parallelize(0 until 100000).map(i => (i/2, i)) + val resultA = rddA.reduceByKey(math.max _, 100).collect() + assert(resultA.length == 50000) + resultA.foreach { case(k, v) => + if (v != k * 2 + 1) { + fail(s"Value for ${k} was wrong: expected ${k * 2 + 1}, got ${v}") + } + } + + // groupByKey - should spill ~8 times per executor + val rddB = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultB = rddB.groupByKey(100).collect() + assert(resultB.length == 25000) + resultB.foreach { case(i, seq) => + val expected = Set(i * 4, i * 4 + 1, i * 4 + 2, i * 4 + 3) + if (seq.toSet != expected) { + fail(s"Value for ${i} was wrong: expected ${expected}, got ${seq.toSet}") + } + } + + // cogroup - should spill ~4 times per executor + val rddC1 = sc.parallelize(0 until 10000).map(i => (i, i)) + val rddC2 = sc.parallelize(0 until 10000).map(i => (i%1000, i)) + val resultC = rddC1.cogroup(rddC2, 100).collect() + assert(resultC.length == 10000) + resultC.foreach { case(i, (seq1, seq2)) => + i match { + case 0 => + assert(seq1.toSet == Set[Int](0)) + assert(seq2.toSet == Set[Int](0, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000)) + case 1 => + assert(seq1.toSet == Set[Int](1)) + assert(seq2.toSet == Set[Int](1, 1001, 2001, 3001, 4001, 5001, 6001, 7001, 8001, 9001)) + case 5000 => + assert(seq1.toSet == Set[Int](5000)) + assert(seq2.toSet == Set[Int]()) + case 9999 => + assert(seq1.toSet == Set[Int](9999)) + assert(seq2.toSet == Set[Int]()) + case _ => + } + } + + // larger cogroup - should spill ~4 times per executor + val rddD1 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val rddD2 = sc.parallelize(0 until 10000).map(i => (i/2, i)) + val resultD = rddD1.cogroup(rddD2).collect() + assert(resultD.length == 5000) + resultD.foreach { case(i, (seq1, seq2)) => + val expected = Set(i * 2, i * 2 + 1) + if (seq1.toSet != expected) { + fail(s"Value 1 for ${i} was wrong: expected ${expected}, got ${seq1.toSet}") + } + if (seq2.toSet != expected) { + fail(s"Value 2 for ${i} was wrong: expected ${expected}, got ${seq2.toSet}") + } + } + } + + test("cleanup of intermediate files in sorter") { + val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + sorter.write((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + + val sorter2 = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + sorter2.write((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + assert(sorter2.iterator.toSet === (0 until 100000).map(i => (i, i)).toSet) + sorter2.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in sorter if there are errors") { + val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + intercept[SparkException] { + sorter.write((0 until 100000).iterator.map(i => { + if (i == 99990) { + throw new SparkException("Intentional failure") + } + (i, i) + })) + } + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in shuffle") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val data = sc.parallelize(0 until 100000, 2).map(i => (i, i)) + assert(data.reduceByKey(_ + _).count() === 100000) + + // After the shuffle, there should be only 4 files on disk: our two map output files and + // their index files. All other intermediate files should've been deleted. + assert(diskBlockManager.getAllFiles().length === 4) + } + + test("cleanup of intermediate files in shuffle with errors") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + + val data = sc.parallelize(0 until 100000, 2).map(i => { + if (i == 99990) { + throw new Exception("Intentional failure") + } + (i, i) + }) + intercept[SparkException] { + data.reduceByKey(_ + _).count() + } + + // After the shuffle, there should be only 2 files on disk: the output of task 1 and its index. + // All other files (map 2's output and intermediate merge files) should've been deleted. + assert(diskBlockManager.getAllFiles().length === 2) + } + + test("no partial aggregation or sorting") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + sorter.write((0 until 100000).iterator.map(i => (i / 4, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 100000).map(i => (i / 4, i)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("partial aggregation without spill") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), None, None) + sorter.write((0 until 100).iterator.map(i => (i / 2, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 50).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("partial aggregation with spill, no ordering") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), None, None) + sorter.write((0 until 100000).iterator.map(i => (i / 2, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 50000).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("partial aggregation with spill, with ordering") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), Some(ord), None) + sorter.write((0 until 100000).iterator.map(i => (i / 2, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet + val expected = (0 until 3).map(p => { + (p, (0 until 50000).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) + }).toSet + assert(results === expected) + } + + test("sorting without aggregation, no spill") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val ord = implicitly[Ordering[Int]] + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + sorter.write((0 until 100).iterator.map(i => (i, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSeq)}.toSeq + val expected = (0 until 3).map(p => { + (p, (0 until 100).map(i => (i, i)).filter(_._1 % 3 == p).toSeq) + }).toSeq + assert(results === expected) + } + + test("sorting without aggregation, with spill") { + val conf = new SparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val ord = implicitly[Ordering[Int]] + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + sorter.write((0 until 100000).iterator.map(i => (i, i))) + val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSeq)}.toSeq + val expected = (0 until 3).map(p => { + (p, (0 until 100000).map(i => (i, i)).filter(_._1 % 3 == p).toSeq) + }).toSeq + assert(results === expected) + } + + test("spilling with hash collisions") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + def createCombiner(i: String) = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i + def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) = + buffer1 ++= buffer2 + + val agg = new Aggregator[String, String, ArrayBuffer[String]]( + createCombiner _, mergeValue _, mergeCombiners _) + + val sorter = new ExternalSorter[String, String, ArrayBuffer[String]]( + Some(agg), None, None, None) + + val collisionPairs = Seq( + ("Aa", "BB"), // 2112 + ("to", "v1"), // 3707 + ("variants", "gelato"), // -1249574770 + ("Teheran", "Siblings"), // 231609873 + ("misused", "horsemints"), // 1069518484 + ("isohel", "epistolaries"), // -1179291542 + ("righto", "buzzards"), // -931102253 + ("hierarch", "crinolines"), // -1732884796 + ("inwork", "hypercatalexes"), // -1183663690 + ("wainages", "presentencing"), // 240183619 + ("trichothecenes", "locular"), // 339006536 + ("pomatoes", "eructation") // 568647356 + ) + + collisionPairs.foreach { case (w1, w2) => + // String.hashCode is documented to use a specific algorithm, but check just in case + assert(w1.hashCode === w2.hashCode) + } + + val toInsert = (1 to 100000).iterator.map(_.toString).map(s => (s, s)) ++ + collisionPairs.iterator ++ collisionPairs.iterator.map(_.swap) + + sorter.write(toInsert) + + // A map of collision pairs in both directions + val collisionPairsMap = (collisionPairs ++ collisionPairs.map(_.swap)).toMap + + // Avoid map.size or map.iterator.length because this destructively sorts the underlying map + var count = 0 + + val it = sorter.iterator + while (it.hasNext) { + val kv = it.next() + val expectedValue = ArrayBuffer[String](collisionPairsMap.getOrElse(kv._1, kv._1)) + assert(kv._2.equals(expectedValue)) + count += 1 + } + assert(count === 100000 + collisionPairs.size * 2) + } + + test("spilling with many hash collisions") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.0001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + val agg = new Aggregator[FixedHashObject, Int, Int](_ => 1, _ + _, _ + _) + val sorter = new ExternalSorter[FixedHashObject, Int, Int](Some(agg), None, None, None) + + // Insert 10 copies each of lots of objects whose hash codes are either 0 or 1. This causes + // problems if the map fails to group together the objects with the same code (SPARK-2043). + val toInsert = for (i <- 1 to 10; j <- 1 to 10000) yield (FixedHashObject(j, j % 2), 1) + sorter.write(toInsert.iterator) + + val it = sorter.iterator + var count = 0 + while (it.hasNext) { + val kv = it.next() + assert(kv._2 === 10) + count += 1 + } + assert(count === 10000) + } + + test("spilling with hash collisions using the Int.MaxValue key") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + def createCombiner(i: Int) = ArrayBuffer[Int](i) + def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i + def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + + val agg = new Aggregator[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) + val sorter = new ExternalSorter[Int, Int, ArrayBuffer[Int]](Some(agg), None, None, None) + + sorter.write((1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) + + val it = sorter.iterator + while (it.hasNext) { + // Should not throw NoSuchElementException + it.next() + } + } + + test("spilling with null keys and values") { + val conf = new SparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.001") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + def createCombiner(i: String) = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i + def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]) = buf1 ++= buf2 + + val agg = new Aggregator[String, String, ArrayBuffer[String]]( + createCombiner, mergeValue, mergeCombiners) + + val sorter = new ExternalSorter[String, String, ArrayBuffer[String]]( + Some(agg), None, None, None) + + sorter.write((1 to 100000).iterator.map(i => (i.toString, i.toString)) ++ Iterator( + (null.asInstanceOf[String], "1"), + ("1", null.asInstanceOf[String]), + (null.asInstanceOf[String], null.asInstanceOf[String]) + )) + + val it = sorter.iterator + while (it.hasNext) { + // Should not throw NullPointerException + it.next() + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/collection/FixedHashObject.scala b/core/src/test/scala/org/apache/spark/util/collection/FixedHashObject.scala new file mode 100644 index 0000000000000..c787b5f066e00 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/collection/FixedHashObject.scala @@ -0,0 +1,25 @@ +/* + * 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.util.collection + +/** + * A dummy class that always returns the same hash code, to easily test hash collisions + */ +case class FixedHashObject(v: Int, h: Int) extends Serializable { + override def hashCode(): Int = h +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index 5318b8da6412a..714f3b81c9dad 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -28,7 +28,7 @@ import org.apache.spark.rdd.{ShuffledRDD, RDD} private[graphx] class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { - val rdd = new ShuffledRDD[VertexId, VD, VD, (VertexId, VD)](self, partitioner) + val rdd = new ShuffledRDD[VertexId, VD, VD](self, partitioner) // Set a custom serializer if the data is of int or double type. if (classTag[VD] == ClassTag.Int) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index a565d3b28bf52..b27485953f719 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -33,7 +33,7 @@ private[graphx] class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { - new ShuffledRDD[VertexId, Int, Int, RoutingTableMessage]( + new ShuffledRDD[VertexId, Int, Int]( self, partitioner).setSerializer(new RoutingTableMessageSerializer) } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 672343fbbed2e..a8bbd55861954 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -295,6 +295,7 @@ object Unidoc { .map(_.filterNot(_.getCanonicalPath.contains("akka"))) .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) .map(_.filterNot(_.getCanonicalPath.contains("network"))) + .map(_.filterNot(_.getCanonicalPath.contains("shuffle"))) .map(_.filterNot(_.getCanonicalPath.contains("executor"))) .map(_.filterNot(_.getCanonicalPath.contains("python"))) .map(_.filterNot(_.getCanonicalPath.contains("collection"))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 392a7f3be3904..30712f03cab4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -49,7 +49,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(r => mutablePair.update(hashExpressions(r), r)) } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, Row, MutablePair[Row, Row]](rdd, part) + val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) @@ -62,7 +62,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(row => mutablePair.update(row, null)) } val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, Null, MutablePair[Row, Null]](rdd, part) + val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._1) @@ -73,7 +73,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una iter.map(r => mutablePair.update(null, r)) } val partitioner = new HashPartitioner(1) - val shuffled = new ShuffledRDD[Null, Row, Row, MutablePair[Null, Row]](rdd, partitioner) + val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 174eda8f1a72c..0027f3cf1fc79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -148,7 +148,7 @@ case class Limit(limit: Int, child: SparkPlan) iter.take(limit).map(row => mutablePair.update(false, row)) } val part = new HashPartitioner(1) - val shuffled = new ShuffledRDD[Boolean, Row, Row, MutablePair[Boolean, Row]](rdd, part) + val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.mapPartitions(_.take(limit).map(_._2)) } From 894d48ffb8c91e347ab60c58de983e1aaf181188 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 30 Jul 2014 21:30:13 -0700 Subject: [PATCH 0146/1492] [SPARK-2758] UnionRDD's UnionPartition should not reference parent RDDs Author: Reynold Xin Closes #1675 from rxin/unionrdd and squashes the following commits: 941d316 [Reynold Xin] Clear RDDs for checkpointing. c9f05f2 [Reynold Xin] [SPARK-2758] UnionRDD's UnionPartition should not reference parent RDDs --- .../scala/org/apache/spark/rdd/UnionRDD.scala | 41 ++++++++++++++----- .../scala/org/apache/spark/rdd/RDDSuite.scala | 12 ++++++ 2 files changed, 42 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 21c6e07d69f90..197167ecad0bd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -25,21 +25,32 @@ import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi -private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) +/** + * Partition for UnionRDD. + * + * @param idx index of the partition + * @param rdd the parent RDD this partition refers to + * @param parentRddIndex index of the parent RDD this partition refers to + * @param parentRddPartitionIndex index of the partition within the parent RDD + * this partition refers to + */ +private[spark] class UnionPartition[T: ClassTag]( + idx: Int, + @transient rdd: RDD[T], + val parentRddIndex: Int, + @transient parentRddPartitionIndex: Int) extends Partition { - var split: Partition = rdd.partitions(splitIndex) - - def iterator(context: TaskContext) = rdd.iterator(split, context) + var parentPartition: Partition = rdd.partitions(parentRddPartitionIndex) - def preferredLocations() = rdd.preferredLocations(split) + def preferredLocations() = rdd.preferredLocations(parentPartition) override val index: Int = idx @throws(classOf[IOException]) private def writeObject(oos: ObjectOutputStream) { // Update the reference to parent split at the time of task serialization - split = rdd.partitions(splitIndex) + parentPartition = rdd.partitions(parentRddPartitionIndex) oos.defaultWriteObject() } } @@ -47,14 +58,14 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd @DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, - @transient var rdds: Seq[RDD[T]]) + var rdds: Seq[RDD[T]]) extends RDD[T](sc, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { val array = new Array[Partition](rdds.map(_.partitions.size).sum) var pos = 0 - for (rdd <- rdds; split <- rdd.partitions) { - array(pos) = new UnionPartition(pos, rdd, split.index) + for ((rdd, rddIndex) <- rdds.zipWithIndex; split <- rdd.partitions) { + array(pos) = new UnionPartition(pos, rdd, rddIndex, split.index) pos += 1 } array @@ -70,9 +81,17 @@ class UnionRDD[T: ClassTag]( deps } - override def compute(s: Partition, context: TaskContext): Iterator[T] = - s.asInstanceOf[UnionPartition[T]].iterator(context) + override def compute(s: Partition, context: TaskContext): Iterator[T] = { + val part = s.asInstanceOf[UnionPartition[T]] + val parentRdd = dependencies(part.parentRddIndex).rdd.asInstanceOf[RDD[T]] + parentRdd.iterator(part.parentPartition, context) + } override def getPreferredLocations(s: Partition): Seq[String] = s.asInstanceOf[UnionPartition[T]].preferredLocations() + + override def clearDependencies() { + super.clearDependencies() + rdds = null + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 8966eedd80ebc..ae6e52587584f 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -121,6 +121,18 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(union.partitioner === nums1.partitioner) } + test("UnionRDD partition serialized size should be small") { + val largeVariable = new Array[Byte](1000 * 1000) + val rdd1 = sc.parallelize(1 to 10, 2).map(i => largeVariable.length) + val rdd2 = sc.parallelize(1 to 10, 3) + + val ser = SparkEnv.get.closureSerializer.newInstance() + val union = rdd1.union(rdd2) + // The UnionRDD itself should be large, but each individual partition should be small. + assert(ser.serialize(union).limit() > 2000) + assert(ser.serialize(union.partitions.head).limit() < 2000) + } + test("aggregate") { val pairs = sc.makeRDD(Array(("a", 1), ("b", 2), ("a", 2), ("c", 5), ("a", 3))) type StringMap = HashMap[String, Int] From 118c1c422d3dfbfb2277995062678f0a808af6c3 Mon Sep 17 00:00:00 2001 From: derek ma Date: Wed, 30 Jul 2014 21:37:59 -0700 Subject: [PATCH 0147/1492] Required AM memory is "amMem", not "args.amMemory" "ERROR yarn.Client: Required AM memory (1024) is above the max threshold (1048) of this cluster" appears if this code is not changed. obviously, 1024 is less than 1048, so change this Author: derek ma Closes #1494 from maji2014/master and squashes the following commits: b0f6640 [derek ma] Required AM memory is "amMem", not "args.amMemory" --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index a1298e8f30b5c..b7e8636e02eb2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -109,7 +109,7 @@ trait ClientBase extends Logging { if (amMem > maxMem) { val errorMessage = "Required AM memory (%d) is above the max threshold (%d) of this cluster." - .format(args.amMemory, maxMem) + .format(amMem, maxMem) logError(errorMessage) throw new IllegalArgumentException(errorMessage) } From a7c305b86b3b83645ae5ff5d3dfeafc20c443204 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 30 Jul 2014 21:57:32 -0700 Subject: [PATCH 0148/1492] [SPARK-2340] Resolve event logging and History Server paths properly We resolve relative paths to the local `file:/` system for `--jars` and `--files` in spark submit (#853). We should do the same for the history server. Author: Andrew Or Closes #1280 from andrewor14/hist-serv-fix and squashes the following commits: 13ff406 [Andrew Or] Merge branch 'master' of github.com:apache/spark into hist-serv-fix b393e17 [Andrew Or] Strip trailing "/" from logging directory 622a471 [Andrew Or] Fix test in EventLoggingListenerSuite 0e20f71 [Andrew Or] Shift responsibility of resolving paths up one level b037c0c [Andrew Or] Use resolved paths for everything in history server c7e36ee [Andrew Or] Resolve paths for event logging too 40e3933 [Andrew Or] Resolve history server file paths --- .../deploy/history/FsHistoryProvider.scala | 34 ++++++++++--------- .../spark/deploy/history/HistoryPage.scala | 2 +- .../spark/deploy/history/HistoryServer.scala | 6 ++-- .../history/HistoryServerArguments.scala | 5 +-- .../scheduler/EventLoggingListener.scala | 6 ++-- .../org/apache/spark/util/FileLogger.scala | 2 +- .../scheduler/EventLoggingListenerSuite.scala | 2 +- 7 files changed, 28 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 01e7065c17b69..6d2d4cef1ee46 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -36,11 +36,11 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis conf.getInt("spark.history.updateInterval", 10)) * 1000 private val logDir = conf.get("spark.history.fs.logDirectory", null) - if (logDir == null) { - throw new IllegalArgumentException("Logging directory must be specified.") - } + private val resolvedLogDir = Option(logDir) + .map { d => Utils.resolveURI(d) } + .getOrElse { throw new IllegalArgumentException("Logging directory must be specified.") } - private val fs = Utils.getHadoopFileSystem(logDir) + private val fs = Utils.getHadoopFileSystem(resolvedLogDir) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTimeMs = -1L @@ -76,14 +76,14 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def initialize() { // Validate the log directory. - val path = new Path(logDir) + val path = new Path(resolvedLogDir) if (!fs.exists(path)) { throw new IllegalArgumentException( - "Logging directory specified does not exist: %s".format(logDir)) + "Logging directory specified does not exist: %s".format(resolvedLogDir)) } if (!fs.getFileStatus(path).isDir) { throw new IllegalArgumentException( - "Logging directory specified is not a directory: %s".format(logDir)) + "Logging directory specified is not a directory: %s".format(resolvedLogDir)) } checkForLogs() @@ -95,15 +95,16 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis override def getAppUI(appId: String): SparkUI = { try { - val appLogDir = fs.getFileStatus(new Path(logDir, appId)) - loadAppInfo(appLogDir, true)._2 + val appLogDir = fs.getFileStatus(new Path(resolvedLogDir.toString, appId)) + val (_, ui) = loadAppInfo(appLogDir, renderUI = true) + ui } catch { case e: FileNotFoundException => null } } override def getConfig(): Map[String, String] = - Map(("Event Log Location" -> logDir)) + Map("Event Log Location" -> resolvedLogDir.toString) /** * Builds the application list based on the current contents of the log directory. @@ -114,14 +115,14 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis lastLogCheckTimeMs = getMonotonicTimeMs() logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) try { - val logStatus = fs.listStatus(new Path(logDir)) + val logStatus = fs.listStatus(new Path(resolvedLogDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - val logInfos = logDirs.filter { - dir => fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE)) + val logInfos = logDirs.filter { dir => + fs.isFile(new Path(dir.getPath, EventLoggingListener.APPLICATION_COMPLETE)) } val currentApps = Map[String, ApplicationHistoryInfo]( - appList.map(app => (app.id -> app)):_*) + appList.map(app => app.id -> app):_*) // For any application that either (i) is not listed or (ii) has changed since the last time // the listing was created (defined by the log dir's modification time), load the app's info. @@ -131,7 +132,8 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val curr = currentApps.getOrElse(dir.getPath().getName(), null) if (curr == null || curr.lastUpdated < getModificationTime(dir)) { try { - newApps += loadAppInfo(dir, false)._1 + val (app, _) = loadAppInfo(dir, renderUI = false) + newApps += app } catch { case e: Exception => logError(s"Failed to load app info from directory $dir.") } @@ -159,9 +161,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false. */ private def loadAppInfo(logDir: FileStatus, renderUI: Boolean) = { - val elogInfo = EventLoggingListener.parseLoggingInfo(logDir.getPath(), fs) val path = logDir.getPath val appId = path.getName + val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs) val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec) val appListener = new ApplicationEventListener replayBus.addListener(appListener) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index d7a3e3f120e67..c4ef8b63b0071 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -45,7 +45,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
    - { providerConfig.map(e =>
  • {e._1}: {e._2}
  • ) } + {providerConfig.map { case (k, v) =>
  • {k}: {v}
  • }}
{ if (allApps.size > 0) { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index cacb9da8c947b..d1a64c1912cb8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -25,9 +25,9 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.ui.{WebUI, SparkUI, UIUtils} +import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{SignalLogger, Utils} +import org.apache.spark.util.SignalLogger /** * A web server that renders SparkUIs of completed applications. @@ -177,7 +177,7 @@ object HistoryServer extends Logging { def main(argStrings: Array[String]) { SignalLogger.register(log) initSecurity() - val args = new HistoryServerArguments(conf, argStrings) + new HistoryServerArguments(conf, argStrings) val securityManager = new SecurityManager(conf) val providerName = conf.getOption("spark.history.provider") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index be9361b754fc3..25fc76c23e0fb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.history import org.apache.spark.SparkConf -import org.apache.spark.util.Utils /** * Command-line parser for the master. @@ -32,6 +31,7 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] args match { case ("--dir" | "-d") :: value :: tail => logDir = value + conf.set("spark.history.fs.logDirectory", value) parse(tail) case ("--help" | "-h") :: tail => @@ -42,9 +42,6 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] case _ => printUsageAndExit(1) } - if (logDir != null) { - conf.set("spark.history.fs.logDirectory", logDir) - } } private def printUsageAndExit(exitCode: Int) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index ae6ca9f4e7bf5..406147f167bf3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -29,7 +29,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{FileLogger, JsonProtocol} +import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} /** * A SparkListener that logs events to persistent storage. @@ -55,7 +55,7 @@ private[spark] class EventLoggingListener( private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis - val logDir = logBaseDir + "/" + name + val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) @@ -215,7 +215,7 @@ private[spark] object EventLoggingListener extends Logging { } catch { case e: Exception => logError("Exception in parsing logging info from directory %s".format(logDir), e) - EventLoggingInfo.empty + EventLoggingInfo.empty } } 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 9dcdafdd6350e..2e8fbf5a91ee7 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -52,7 +52,7 @@ private[spark] class FileLogger( override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } - private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + private val fileSystem = Utils.getHadoopFileSystem(logDir) var fileIndex = 0 // Only used if compression is enabled diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 21e3db34b8b7a..10d8b299317ea 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -259,7 +259,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { assert(sc.eventLogger.isDefined) val eventLogger = sc.eventLogger.get val expectedLogDir = logDirPath.toString - assert(eventLogger.logDir.startsWith(expectedLogDir)) + assert(eventLogger.logDir.contains(expectedLogDir)) // Begin listening for events that trigger asserts val eventExistenceListener = new EventExistenceListener(eventLogger) From 4fb259353f616822c32537e3f031944a6d2a09a8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 30 Jul 2014 22:40:57 -0700 Subject: [PATCH 0149/1492] [SPARK-2737] Add retag() method for changing RDDs' ClassTags. The Java API's use of fake ClassTags doesn't seem to cause any problems for Java users, but it can lead to issues when passing JavaRDDs' underlying RDDs to Scala code (e.g. in the MLlib Java API wrapper code). If we call collect() on a Scala RDD with an incorrect ClassTag, this causes ClassCastExceptions when we try to allocate an array of the wrong type (for example, see SPARK-2197). There are a few possible fixes here. An API-breaking fix would be to completely remove the fake ClassTags and require Java API users to pass java.lang.Class instances to all parallelize() calls and add returnClass fields to all Function implementations. This would be extremely verbose. Instead, this patch adds internal APIs to "repair" a Scala RDD with an incorrect ClassTag by wrapping it and overriding its ClassTag. This should be okay for cases where the Scala code that calls collect() knows what type of array should be allocated, which is the case in the MLlib wrappers. Author: Josh Rosen Closes #1639 from JoshRosen/SPARK-2737 and squashes the following commits: 572b4c8 [Josh Rosen] Replace newRDD[T] with mapPartitions(). 469d941 [Josh Rosen] Preserve partitioner in retag(). af78816 [Josh Rosen] Allow retag() to get classTag implicitly. d1d54e6 [Josh Rosen] [SPARK-2737] Add retag() method for changing RDDs' ClassTags. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 17 +++++++++++++++++ .../java/org/apache/spark/JavaAPISuite.java | 17 +++++++++++++++++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 8 ++++++++ 3 files changed, 42 insertions(+) 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 74ac97091fd0b..e1c49e35abecd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1236,6 +1236,23 @@ abstract class RDD[T: ClassTag]( /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context = sc + /** + * Private API for changing an RDD's ClassTag. + * Used for internal Java <-> Scala API compatibility. + */ + private[spark] def retag(cls: Class[T]): RDD[T] = { + val classTag: ClassTag[T] = ClassTag.apply(cls) + this.retag(classTag) + } + + /** + * Private API for changing an RDD's ClassTag. + * Used for internal Java <-> Scala API compatibility. + */ + private[spark] def retag(implicit classTag: ClassTag[T]): RDD[T] = { + this.mapPartitions(identity, preservesPartitioning = true)(classTag) + } + // Avoid handling doCheckpoint multiple times to prevent excessive recursion @transient private var doCheckpointCalled = false diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e8bd65f8e4507..fab64a54e2479 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1245,4 +1245,21 @@ public Tuple2 call(Integer i) { Assert.assertTrue(worExactCounts.get(0) == 2); Assert.assertTrue(worExactCounts.get(1) == 4); } + + private static class SomeCustomClass implements Serializable { + public SomeCustomClass() { + // Intentionally left blank + } + } + + @Test + public void collectUnderlyingScalaRDD() { + List data = new ArrayList(); + for (int i = 0; i < 100; i++) { + data.add(new SomeCustomClass()); + } + JavaRDD rdd = sc.parallelize(data); + SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); + Assert.assertEquals(data.size(), collected.length); + } } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index ae6e52587584f..b31e3a09e5b9c 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.rdd import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import org.scalatest.FunSuite @@ -26,6 +27,7 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ class RDDSuite extends FunSuite with SharedSparkContext { @@ -718,6 +720,12 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(ids.length === n) } + test("retag with implicit ClassTag") { + val jsc: JavaSparkContext = new JavaSparkContext(sc) + val jrdd: JavaRDD[String] = jsc.parallelize(Seq("A", "B", "C").asJava) + jrdd.rdd.retag.collect() + } + test("getNarrowAncestors") { val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1) From 5a110da25f15694773d6f7c6ee63c5b08ada4eb0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 30 Jul 2014 22:46:30 -0700 Subject: [PATCH 0150/1492] [SPARK-2497] Included checks for module symbols too. Author: Prashant Sharma Closes #1463 from ScrapCodes/SPARK-2497/mima-exclude-all and squashes the following commits: 72077b1 [Prashant Sharma] Check separately for module symbols. cd96192 [Prashant Sharma] SPARK-2497 Produce "member excludes" irrespective of the fact that class itself is excluded or not. --- .../spark/tools/GenerateMIMAIgnore.scala | 20 +++++++++---------- 1 file changed, 9 insertions(+), 11 deletions(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 566983675bff5..16ff89a8a9809 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -68,12 +68,11 @@ object GenerateMIMAIgnore { for (className <- classes) { try { val classSymbol = mirror.classSymbol(Class.forName(className, false, classLoader)) - val moduleSymbol = mirror.staticModule(className) // TODO: see if it is necessary. + val moduleSymbol = mirror.staticModule(className) val directlyPrivateSpark = isPackagePrivate(classSymbol) || isPackagePrivateModule(moduleSymbol) - val developerApi = isDeveloperApi(classSymbol) - val experimental = isExperimental(classSymbol) - + val developerApi = isDeveloperApi(classSymbol) || isDeveloperApi(moduleSymbol) + val experimental = isExperimental(classSymbol) || isExperimental(moduleSymbol) /* Inner classes defined within a private[spark] class or object are effectively invisible, so we account for them as package private. */ lazy val indirectlyPrivateSpark = { @@ -87,10 +86,9 @@ object GenerateMIMAIgnore { } if (directlyPrivateSpark || indirectlyPrivateSpark || developerApi || experimental) { ignoredClasses += className - } else { - // check if this class has package-private/annotated members. - ignoredMembers ++= getAnnotatedOrPackagePrivateMembers(classSymbol) } + // check if this class has package-private/annotated members. + ignoredMembers ++= getAnnotatedOrPackagePrivateMembers(classSymbol) } catch { case _: Throwable => println("Error instrumenting class:" + className) @@ -115,8 +113,9 @@ object GenerateMIMAIgnore { } private def getAnnotatedOrPackagePrivateMembers(classSymbol: unv.ClassSymbol) = { - classSymbol.typeSignature.members - .filter(x => isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x)).map(_.fullName) ++ + classSymbol.typeSignature.members.filterNot(x => + x.fullName.startsWith("java") || x.fullName.startsWith("scala")) + .filter(x => isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x)).map(_.fullName) ++ getInnerFunctions(classSymbol) } @@ -137,8 +136,7 @@ object GenerateMIMAIgnore { name.endsWith("$class") || name.contains("$sp") || name.contains("hive") || - name.contains("Hive") || - name.contains("repl") + name.contains("Hive") } /** From 669e3f05895d9dfa37abf60f60aecebb03988e50 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Wed, 30 Jul 2014 23:37:25 -0700 Subject: [PATCH 0151/1492] automatically set master according to `spark.master` in `spark-defaults.... automatically set master according to `spark.master` in `spark-defaults.conf` Author: CrazyJvm Closes #1644 from CrazyJvm/standalone-guide and squashes the following commits: bb12b95 [CrazyJvm] automatically set master according to `spark.master` in `spark-defaults.conf` --- docs/spark-standalone.md | 3 --- 1 file changed, 3 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index ad8b6c0e51a78..2fb30765f35e8 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -242,9 +242,6 @@ To run an interactive Spark shell against the cluster, run the following command ./bin/spark-shell --master spark://IP:PORT -Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will -automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`. - You can also pass an option `--cores ` to control the number of cores that spark-shell uses on the cluster. # Launching Compiled Spark Applications From 92ca910eb866701e01b987a4f5003564b4785959 Mon Sep 17 00:00:00 2001 From: Timothy Hunter Date: Thu, 31 Jul 2014 10:25:40 -0700 Subject: [PATCH 0152/1492] [SPARK-2762] SparkILoop leaks memory in multi-repl configurations This pull request is a small refactor so that a partial function (hence a closure) is not created. Instead, a regular function is used. The behavior of the code is not changed. Author: Timothy Hunter Closes #1674 from thunterdb/closure_issue and squashes the following commits: e1e664d [Timothy Hunter] simplify closure --- .../org/apache/spark/repl/SparkILoop.scala | 39 ++++++++++--------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index e1db4d5395ab9..6f9fa0d9f2b25 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -557,29 +557,27 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, if (isReplPower) powerCommands else Nil )*/ - val replayQuestionMessage = + private val replayQuestionMessage = """|That entry seems to have slain the compiler. Shall I replay |your session? I can re-run each line except the last one. |[y/n] """.trim.stripMargin - private val crashRecovery: PartialFunction[Throwable, Boolean] = { - case ex: Throwable => - echo(intp.global.throwableAsString(ex)) - - ex match { - case _: NoSuchMethodError | _: NoClassDefFoundError => - echo("\nUnrecoverable error.") - throw ex - case _ => - def fn(): Boolean = - try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) - catch { case _: RuntimeException => false } - - if (fn()) replay() - else echo("\nAbandoning crashed session.") - } - true + private def crashRecovery(ex: Throwable): Boolean = { + echo(ex.toString) + ex match { + case _: NoSuchMethodError | _: NoClassDefFoundError => + echo("\nUnrecoverable error.") + throw ex + case _ => + def fn(): Boolean = + try in.readYesOrNo(replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + catch { case _: RuntimeException => false } + + if (fn()) replay() + else echo("\nAbandoning crashed session.") + } + true } /** The main read-eval-print loop for the repl. It calls @@ -605,7 +603,10 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } } def innerLoop() { - if ( try processLine(readOneLine()) catch crashRecovery ) + val shouldContinue = try { + processLine(readOneLine()) + } catch {case t: Throwable => crashRecovery(t)} + if (shouldContinue) innerLoop() } innerLoop() From 3072b96026fa3e63e8eef780f2b04dd81f11ea27 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 31 Jul 2014 11:15:25 -0700 Subject: [PATCH 0153/1492] [SPARK-2743][SQL] Resolve original attributes in ParquetTableScan Author: Michael Armbrust Closes #1647 from marmbrus/parquetCase and squashes the following commits: a1799b7 [Michael Armbrust] move comment 2a2a68b [Michael Armbrust] Merge remote-tracking branch 'apache/master' into parquetCase bb35d5b [Michael Armbrust] Fix test case that produced an invalid plan. e6870bf [Michael Armbrust] Better error message. 539a2e1 [Michael Armbrust] Resolve original attributes in ParquetTableScan --- .../sql/parquet/ParquetTableOperations.scala | 14 ++++++++++---- .../spark/sql/parquet/ParquetQuerySuite.scala | 14 +------------- .../spark/sql/parquet/HiveParquetSuite.scala | 17 +++++++++++++++++ 3 files changed, 28 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 912a9f002b7d1..759a2a586b926 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -51,13 +51,20 @@ import org.apache.spark.{Logging, SerializableWritable, TaskContext} * [[org.apache.spark.sql.parquet.ParquetRelation]] as a ``RDD[Row]``. */ case class ParquetTableScan( - // note: output cannot be transient, see - // https://issues.apache.org/jira/browse/SPARK-1367 - output: Seq[Attribute], + attributes: Seq[Attribute], relation: ParquetRelation, columnPruningPred: Seq[Expression]) extends LeafNode { + // The resolution of Parquet attributes is case sensitive, so we resolve the original attributes + // by exprId. note: output cannot be transient, see + // https://issues.apache.org/jira/browse/SPARK-1367 + val output = attributes.map { a => + relation.output + .find(o => o.exprId == a.exprId) + .getOrElse(sys.error(s"Invalid parquet attribute $a in ${relation.output.mkString(",")}")) + } + override def execute(): RDD[Row] = { val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) @@ -110,7 +117,6 @@ case class ParquetTableScan( ParquetTableScan(prunedAttributes, relation, columnPruningPred) } else { sys.error("Warning: Could not validate Parquet schema projection in pruneColumns") - this } } 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 561f5b4a49965..8955455ec98c7 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 @@ -209,19 +209,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection of simple Parquet file") { - SparkPlan.currentContext.set(TestSQLContext) - val scanner = new ParquetTableScan( - ParquetTestData.testData.output, - ParquetTestData.testData, - Seq()) - val projected = scanner.pruneColumns(ParquetTypesConverter - .convertToAttributes(MessageTypeParser - .parseMessageType(ParquetTestData.subTestSchema))) - assert(projected.output.size === 2) - val result = projected - .execute() - .map(_.copy()) - .collect() + val result = ParquetTestData.testData.select('myboolean, 'mylong).collect() result.zipWithIndex.foreach { case (row, index) => { if (index % 3 == 0) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 3bfe49a760be5..47526e3596e44 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import java.io.File + import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} @@ -27,6 +29,8 @@ import org.apache.spark.util.Utils // Implicits import org.apache.spark.sql.hive.test.TestHive._ +case class Cases(lower: String, UPPER: String) + class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { val dirname = Utils.createTempDir() @@ -55,6 +59,19 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft Utils.deleteRecursively(dirname) } + test("Case insensitive attribute names") { + val tempFile = File.createTempFile("parquet", "") + tempFile.delete() + sparkContext.parallelize(1 to 10) + .map(_.toString) + .map(i => Cases(i, i)) + .saveAsParquetFile(tempFile.getCanonicalPath) + + parquetFile(tempFile.getCanonicalPath).registerAsTable("cases") + hql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) + hql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) + } + test("SELECT on Parquet table") { val rdd = hql("SELECT * FROM testsource").collect() assert(rdd != null) From 72cfb13987bab07461266905930f84619b3a0068 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 31 Jul 2014 11:26:43 -0700 Subject: [PATCH 0154/1492] [SPARK-2397][SQL] Deprecate LocalHiveContext LocalHiveContext is redundant with HiveContext. The only difference is it creates `./metastore` instead of `./metastore_db`. Author: Michael Armbrust Closes #1641 from marmbrus/localHiveContext and squashes the following commits: e5ec497 [Michael Armbrust] Add deprecation version 626e056 [Michael Armbrust] Don't remove from imports yet 905cc5f [Michael Armbrust] Merge remote-tracking branch 'apache/master' into localHiveContext 1c2727e [Michael Armbrust] Deprecate LocalHiveContext --- .../sbt_app_hive/src/main/scala/HiveApp.scala | 4 ++-- docs/sql-programming-guide.md | 6 +++--- .../spark/examples/sql/hive/HiveFromSpark.scala | 4 ++-- python/pyspark/sql.py | 6 ++++++ .../org/apache/spark/sql/hive/HiveContext.scala | 7 +++++-- .../org/apache/spark/sql/hive/TestHive.scala | 15 ++++++++++++--- 6 files changed, 30 insertions(+), 12 deletions(-) diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala index 7257d17d10116..a21410f3b9813 100644 --- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala +++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.{ListBuffer, Queue} import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.sql.hive.LocalHiveContext +import org.apache.spark.sql.hive.HiveContext case class Person(name: String, age: Int) @@ -34,7 +34,7 @@ object SparkSqlExample { case None => new SparkConf().setAppName("Simple Sql App") } val sc = new SparkContext(conf) - val hiveContext = new LocalHiveContext(sc) + val hiveContext = new HiveContext(sc) import hiveContext._ hql("DROP TABLE IF EXISTS src") diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 156e0aebdebe6..a047d32b6ee6c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -487,9 +487,9 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do -not have an existing Hive deployment can also experiment with the `LocalHiveContext`, -which is similar to `HiveContext`, but creates a local copy of the `metastore` and `warehouse` -automatically. +not have an existing Hive deployment can still create a HiveContext. When not configured by the +hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current +directory. {% highlight scala %} // sc is an existing SparkContext. diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 66a23fac39999..dc5290fb4f10e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -19,7 +19,7 @@ package org.apache.spark.examples.sql.hive import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql._ -import org.apache.spark.sql.hive.LocalHiveContext +import org.apache.spark.sql.hive.HiveContext object HiveFromSpark { case class Record(key: Int, value: String) @@ -31,7 +31,7 @@ object HiveFromSpark { // A local hive context creates an instance of the Hive Metastore in process, storing the // the warehouse data in the current directory. This location can be overridden by // specifying a second parameter to the constructor. - val hiveContext = new LocalHiveContext(sc) + val hiveContext = new HiveContext(sc) import hiveContext._ hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 13f0ed4e35490..9388ead5eaad3 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -15,6 +15,8 @@ # limitations under the License. # +import warnings + from pyspark.rdd import RDD, PipelinedRDD from pyspark.serializers import BatchedSerializer, PickleSerializer @@ -813,6 +815,10 @@ class LocalHiveContext(HiveContext): 130091 """ + def __init__(self, sparkContext, sqlContext=None): + HiveContext.__init__(self, sparkContext, sqlContext) + warnings.warn("LocalHiveContext is deprecated. Use HiveContext instead.", DeprecationWarning) + def _get_hive_ctx(self): return self._jvm.LocalHiveContext(self._jsc.sc()) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index b413373345eea..27b444daba2d4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -42,9 +42,12 @@ import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand /** - * Starts up an instance of hive where metadata is stored locally. An in-process metadata data is - * created with data stored in ./metadata. Warehouse data is stored in in ./warehouse. + * DEPRECATED: Use HiveContext instead. */ +@deprecated(""" + Use HiveContext instead. It will still create a local metastore if one is not specified. + However, note that the default directory is ./metastore_db, not ./metastore + """, "1.1") class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { lazy val metastorePath = new File("metastore").getCanonicalPath diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 9386008d02d51..c50e8c4b5c5d3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -53,15 +53,24 @@ object TestHive * hive metastore seems to lead to weird non-deterministic failures. Therefore, the execution of * test cases that rely on TestHive must be serialized. */ -class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { +class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { self => // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") - override lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath - override lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath + lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath + lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath + + /** Sets up the system initially or after a RESET command */ + protected def configure() { + set("javax.jdo.option.ConnectionURL", + s"jdbc:derby:;databaseName=$metastorePath;create=true") + set("hive.metastore.warehouse.dir", warehousePath) + } + + configure() // Must be called before initializing the catalog below. /** The location of the compiled hive distribution */ lazy val hiveHome = envVarToFile("HIVE_HOME") From f1933123525e7c806f5fc0b0a46a78a7546f8b61 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 31 Jul 2014 11:35:38 -0700 Subject: [PATCH 0155/1492] SPARK-2028: Expose mapPartitionsWithInputSplit in HadoopRDD This allows users to gain access to the InputSplit which backs each partition. An alternative solution would have been to have a .withInputSplit() method which returns a new RDD[(InputSplit, (K, V))], but this is confusing because you could not cache this RDD or shuffle it, as InputSplit is not inherently serializable. Author: Aaron Davidson Closes #973 from aarondav/hadoop and squashes the following commits: 9c9112b [Aaron Davidson] Add JavaAPISuite test 9942cd7 [Aaron Davidson] Add Java API 1284a3a [Aaron Davidson] SPARK-2028: Expose mapPartitionsWithInputSplit in HadoopRDD --- .../apache/spark/api/java/JavaHadoopRDD.scala | 43 +++++++++++++++++++ .../spark/api/java/JavaNewHadoopRDD.scala | 43 +++++++++++++++++++ .../spark/api/java/JavaSparkContext.scala | 21 +++++---- .../org/apache/spark/rdd/HadoopRDD.scala | 32 ++++++++++++++ .../org/apache/spark/rdd/NewHadoopRDD.scala | 34 +++++++++++++++ .../java/org/apache/spark/JavaAPISuite.java | 26 ++++++++++- .../scala/org/apache/spark/FileSuite.scala | 34 ++++++++++++++- 7 files changed, 222 insertions(+), 11 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala create mode 100644 core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala new file mode 100644 index 0000000000000..0ae0b4ec042e2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/java/JavaHadoopRDD.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java + +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag + +import org.apache.hadoop.mapred.InputSplit + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaSparkContext._ +import org.apache.spark.api.java.function.{Function2 => JFunction2} +import org.apache.spark.rdd.HadoopRDD + +@DeveloperApi +class JavaHadoopRDD[K, V](rdd: HadoopRDD[K, V]) + (implicit override val kClassTag: ClassTag[K], implicit override val vClassTag: ClassTag[V]) + extends JavaPairRDD[K, V](rdd) { + + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[R]( + f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]], + preservesPartitioning: Boolean = false): JavaRDD[R] = { + new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)), + preservesPartitioning)(fakeClassTag))(fakeClassTag) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala new file mode 100644 index 0000000000000..ec4f3964d75e0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/java/JavaNewHadoopRDD.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java + +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag + +import org.apache.hadoop.mapreduce.InputSplit + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaSparkContext._ +import org.apache.spark.api.java.function.{Function2 => JFunction2} +import org.apache.spark.rdd.NewHadoopRDD + +@DeveloperApi +class JavaNewHadoopRDD[K, V](rdd: NewHadoopRDD[K, V]) + (implicit override val kClassTag: ClassTag[K], implicit override val vClassTag: ClassTag[V]) + extends JavaPairRDD[K, V](rdd) { + + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[R]( + f: JFunction2[InputSplit, java.util.Iterator[(K, V)], java.util.Iterator[R]], + preservesPartitioning: Boolean = false): JavaRDD[R] = { + new JavaRDD(rdd.mapPartitionsWithInputSplit((a, b) => f.call(a, asJavaIterator(b)), + preservesPartitioning)(fakeClassTag))(fakeClassTag) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8a5f8088a05ca..d9d1c5955ca99 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -34,7 +34,7 @@ import org.apache.spark._ import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.{EmptyRDD, RDD} +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns @@ -294,7 +294,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minPartitions)) + val rdd = sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass, minPartitions) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** @@ -314,7 +315,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass)) + val rdd = sc.hadoopRDD(conf, inputFormatClass, keyClass, valueClass) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** Get an RDD for a Hadoop file with an arbitrary InputFormat. @@ -333,7 +335,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions)) + val rdd = sc.hadoopFile(path, inputFormatClass, keyClass, valueClass, minPartitions) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** Get an RDD for a Hadoop file with an arbitrary InputFormat @@ -351,8 +354,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork ): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(keyClass) implicit val ctagV: ClassTag[V] = ClassTag(valueClass) - new JavaPairRDD(sc.hadoopFile(path, - inputFormatClass, keyClass, valueClass)) + val rdd = sc.hadoopFile(path, inputFormatClass, keyClass, valueClass) + new JavaHadoopRDD(rdd.asInstanceOf[HadoopRDD[K, V]]) } /** @@ -372,7 +375,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork conf: Configuration): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(kClass) implicit val ctagV: ClassTag[V] = ClassTag(vClass) - new JavaPairRDD(sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf)) + val rdd = sc.newAPIHadoopFile(path, fClass, kClass, vClass, conf) + new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]]) } /** @@ -391,7 +395,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork vClass: Class[V]): JavaPairRDD[K, V] = { implicit val ctagK: ClassTag[K] = ClassTag(kClass) implicit val ctagV: ClassTag[V] = ClassTag(vClass) - new JavaPairRDD(sc.newAPIHadoopRDD(conf, fClass, kClass, vClass)) + val rdd = sc.newAPIHadoopRDD(conf, fClass, kClass, vClass) + new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]]) } /** Build the union of two or more RDDs. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index e521612ffc27c..8d92ea01d9a3f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -20,7 +20,9 @@ package org.apache.spark.rdd import java.text.SimpleDateFormat import java.util.Date import java.io.EOFException + import scala.collection.immutable.Map +import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.mapred.FileSplit @@ -39,6 +41,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.NextIterator /** @@ -232,6 +235,14 @@ class HadoopRDD[K, V]( new InterruptibleIterator[(K, V)](context, iter) } + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[U: ClassTag]( + f: (InputSplit, Iterator[(K, V)]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = { + new HadoopMapPartitionsWithSplitRDD(this, f, preservesPartitioning) + } + override def getPreferredLocations(split: Partition): Seq[String] = { // TODO: Filtering out "localhost" in case of file:// URLs val hadoopSplit = split.asInstanceOf[HadoopPartition] @@ -272,4 +283,25 @@ private[spark] object HadoopRDD { conf.setInt("mapred.task.partition", splitId) conf.set("mapred.job.id", jobID.toString) } + + /** + * Analogous to [[org.apache.spark.rdd.MapPartitionsRDD]], but passes in an InputSplit to + * the given function rather than the index of the partition. + */ + private[spark] class HadoopMapPartitionsWithSplitRDD[U: ClassTag, T: ClassTag]( + prev: RDD[T], + f: (InputSplit, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false) + extends RDD[U](prev) { + + override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None + + override def getPartitions: Array[Partition] = firstParent[T].partitions + + override def compute(split: Partition, context: TaskContext) = { + val partition = split.asInstanceOf[HadoopPartition] + val inputSplit = partition.inputSplit.value + f(inputSplit, firstParent[T].iterator(split, context)) + } + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index f2b3a64bf1345..7dfec9a18ec67 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -20,6 +20,8 @@ package org.apache.spark.rdd import java.text.SimpleDateFormat import java.util.Date +import scala.reflect.ClassTag + import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -32,6 +34,7 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD private[spark] class NewHadoopPartition( rddId: Int, @@ -157,6 +160,14 @@ class NewHadoopRDD[K, V]( new InterruptibleIterator(context, iter) } + /** Maps over a partition, providing the InputSplit that was used as the base of the partition. */ + @DeveloperApi + def mapPartitionsWithInputSplit[U: ClassTag]( + f: (InputSplit, Iterator[(K, V)]) => Iterator[U], + preservesPartitioning: Boolean = false): RDD[U] = { + new NewHadoopMapPartitionsWithSplitRDD(this, f, preservesPartitioning) + } + override def getPreferredLocations(split: Partition): Seq[String] = { val theSplit = split.asInstanceOf[NewHadoopPartition] theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") @@ -165,6 +176,29 @@ class NewHadoopRDD[K, V]( def getConf: Configuration = confBroadcast.value.value } +private[spark] object NewHadoopRDD { + /** + * Analogous to [[org.apache.spark.rdd.MapPartitionsRDD]], but passes in an InputSplit to + * the given function rather than the index of the partition. + */ + private[spark] class NewHadoopMapPartitionsWithSplitRDD[U: ClassTag, T: ClassTag]( + prev: RDD[T], + f: (InputSplit, Iterator[T]) => Iterator[U], + preservesPartitioning: Boolean = false) + extends RDD[U](prev) { + + override val partitioner = if (preservesPartitioning) firstParent[T].partitioner else None + + override def getPartitions: Array[Partition] = firstParent[T].partitions + + override def compute(split: Partition, context: TaskContext) = { + val partition = split.asInstanceOf[NewHadoopPartition] + val inputSplit = partition.serializableHadoopSplit.value + f(inputSplit, firstParent[T].iterator(split, context)) + } + } +} + private[spark] class WholeTextFileRDD( sc : SparkContext, inputFormatClass: Class[_ <: WholeTextFileInputFormat], diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index fab64a54e2479..56150caa5d6ba 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -25,19 +25,23 @@ import scala.Tuple3; import scala.Tuple4; - import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.mapred.FileSplit; +import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapreduce.Job; import org.junit.After; import org.junit.Assert; @@ -45,6 +49,7 @@ import org.junit.Test; import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaHadoopRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -1262,4 +1267,23 @@ public void collectUnderlyingScalaRDD() { SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); Assert.assertEquals(data.size(), collected.length); } + + public void getHadoopInputSplits() { + String outDir = new File(tempDir, "output").getAbsolutePath(); + sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2).saveAsTextFile(outDir); + + JavaHadoopRDD hadoopRDD = (JavaHadoopRDD) + sc.hadoopFile(outDir, TextInputFormat.class, LongWritable.class, Text.class); + List inputPaths = hadoopRDD.mapPartitionsWithInputSplit( + new Function2>, Iterator>() { + @Override + public Iterator call(InputSplit split, Iterator> it) + throws Exception { + FileSplit fileSplit = (FileSplit) split; + return Lists.newArrayList(fileSplit.getPath().toUri().getPath()).iterator(); + } + }, true).collect(); + Assert.assertEquals(Sets.newHashSet(inputPaths), + Sets.newHashSet(outDir + "/part-00000", outDir + "/part-00001")); + } } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index c70e22cf09433..4a53d25012ad9 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -24,12 +24,14 @@ import scala.io.Source import com.google.common.io.Files import org.apache.hadoop.io._ import org.apache.hadoop.io.compress.DefaultCodec -import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, TextOutputFormat} -import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, FileSplit, TextInputFormat, TextOutputFormat} import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInputFormat => NewTextInputFormat} +import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{NewHadoopRDD, HadoopRDD} import org.apache.spark.util.Utils class FileSuite extends FunSuite with LocalSparkContext { @@ -318,4 +320,32 @@ class FileSuite extends FunSuite with LocalSparkContext { randomRDD.saveAsNewAPIHadoopDataset(job.getConfiguration) assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) } + + test("Get input files via old Hadoop API") { + sc = new SparkContext("local", "test") + val outDir = new File(tempDir, "output").getAbsolutePath + sc.makeRDD(1 to 4, 2).saveAsTextFile(outDir) + + val inputPaths = + sc.hadoopFile(outDir, classOf[TextInputFormat], classOf[LongWritable], classOf[Text]) + .asInstanceOf[HadoopRDD[_, _]] + .mapPartitionsWithInputSplit { (split, part) => + Iterator(split.asInstanceOf[FileSplit].getPath.toUri.getPath) + }.collect() + assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001")) + } + + test("Get input files via new Hadoop API") { + sc = new SparkContext("local", "test") + val outDir = new File(tempDir, "output").getAbsolutePath + sc.makeRDD(1 to 4, 2).saveAsTextFile(outDir) + + val inputPaths = + sc.newAPIHadoopFile(outDir, classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]) + .asInstanceOf[NewHadoopRDD[_, _]] + .mapPartitionsWithInputSplit { (split, part) => + Iterator(split.asInstanceOf[NewFileSplit].getPath.toUri.getPath) + }.collect() + assert(inputPaths.toSet === Set(s"$outDir/part-00000", s"$outDir/part-00001")) + } } From f68105df52902a1c65207d4f51bfdeb55cccf767 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 31 Jul 2014 11:51:20 -0700 Subject: [PATCH 0156/1492] SPARK-2664. Deal with `--conf` options in spark-submit that relate to fl... ...ags Author: Sandy Ryza Closes #1665 from sryza/sandy-spark-2664 and squashes the following commits: 0518c63 [Sandy Ryza] SPARK-2664. Deal with `--conf` options in spark-submit that relate to flags --- .../org/apache/spark/deploy/SparkSubmit.scala | 11 +++++--- .../spark/deploy/SparkSubmitArguments.scala | 26 +++++++++++-------- .../spark/deploy/SparkSubmitSuite.scala | 16 ++++++++++++ 3 files changed, 38 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3df811c4ac5df..318509a67a36f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -184,7 +184,7 @@ object SparkSubmit { OptionAssigner(args.archives, YARN, CLIENT, sysProp = "spark.yarn.dist.archives"), // Yarn cluster only - OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name", sysProp = "spark.app.name"), + OptionAssigner(args.name, YARN, CLUSTER, clOption = "--name"), OptionAssigner(args.driverMemory, YARN, CLUSTER, clOption = "--driver-memory"), OptionAssigner(args.queue, YARN, CLUSTER, clOption = "--queue"), OptionAssigner(args.numExecutors, YARN, CLUSTER, clOption = "--num-executors"), @@ -268,14 +268,17 @@ object SparkSubmit { } } + // Properties given with --conf are superceded by other options, but take precedence over + // properties in the defaults file. + for ((k, v) <- args.sparkProperties) { + sysProps.getOrElseUpdate(k, v) + } + // Read from default spark properties, if any for ((k, v) <- args.getDefaultSparkProperties) { sysProps.getOrElseUpdate(k, v) } - // Spark properties included on command line take precedence - sysProps ++= args.sparkProperties - (childArgs, childClasspath, sysProps, childMainClass) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 01d0ae541a66b..dd044e6298760 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -58,7 +58,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { val sparkProperties: HashMap[String, String] = new HashMap[String, String]() parseOpts(args.toList) - loadDefaults() + mergeSparkProperties() checkRequiredArguments() /** Return default present in the currently defined defaults file. */ @@ -79,9 +79,11 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { defaultProperties } - /** Fill in any undefined values based on the current properties file or built-in defaults. */ - private def loadDefaults(): Unit = { - + /** + * Fill in any undefined values based on the default properties file or options passed in through + * the '--conf' flag. + */ + private def mergeSparkProperties(): Unit = { // Use common defaults file, if not specified by user if (propertiesFile == null) { sys.env.get("SPARK_HOME").foreach { sparkHome => @@ -94,18 +96,20 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } } - val defaultProperties = getDefaultSparkProperties + val properties = getDefaultSparkProperties + properties.putAll(sparkProperties) + // Use properties file as fallback for values which have a direct analog to // arguments in this script. - master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull) + master = Option(master).getOrElse(properties.get("spark.master").orNull) executorMemory = Option(executorMemory) - .getOrElse(defaultProperties.get("spark.executor.memory").orNull) + .getOrElse(properties.get("spark.executor.memory").orNull) executorCores = Option(executorCores) - .getOrElse(defaultProperties.get("spark.executor.cores").orNull) + .getOrElse(properties.get("spark.executor.cores").orNull) totalExecutorCores = Option(totalExecutorCores) - .getOrElse(defaultProperties.get("spark.cores.max").orNull) - name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull) - jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull) + .getOrElse(properties.get("spark.cores.max").orNull) + name = Option(name).getOrElse(properties.get("spark.app.name").orNull) + jars = Option(jars).getOrElse(properties.get("spark.jars").orNull) // This supports env vars in older versions of Spark master = Option(master).getOrElse(System.getenv("MASTER")) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index a301cbd48a0c3..9190b05e2dba2 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -253,6 +253,22 @@ class SparkSubmitSuite extends FunSuite with Matchers { sysProps("spark.shuffle.spill") should be ("false") } + test("handles confs with flag equivalents") { + val clArgs = Seq( + "--deploy-mode", "cluster", + "--executor-memory", "5g", + "--class", "org.SomeClass", + "--conf", "spark.executor.memory=4g", + "--conf", "spark.master=yarn", + "thejar.jar", + "arg1", "arg2") + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, sysProps, mainClass) = createLaunchEnv(appArgs) + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.master") should be ("yarn-cluster") + mainClass should be ("org.apache.spark.deploy.yarn.Client") + } + test("launch simple application with spark-submit") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val args = Seq( From 4dbabb39a7bf248ac4f9b7f5eb2fe69e5047dcb3 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 31 Jul 2014 12:18:40 -0700 Subject: [PATCH 0157/1492] SPARK-2749 [BUILD] Part 2. Fix a follow-on scalastyle error The test compile error is fixed, but the build still fails because of one scalastyle error. https://amplab.cs.berkeley.edu/jenkins/view/Spark/job/Spark-Master-Maven-pre-YARN/lastFailedBuild/hadoop.version=1.0.4,label=centos/console Author: Sean Owen Closes #1690 from srowen/SPARK-2749 and squashes the following commits: 1c9e7a6 [Sean Owen] Also: fix scalastyle error by wrapping a long line --- .../scala/org/apache/spark/tools/GenerateMIMAIgnore.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 16ff89a8a9809..bcf6d43ab34eb 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -114,9 +114,10 @@ object GenerateMIMAIgnore { private def getAnnotatedOrPackagePrivateMembers(classSymbol: unv.ClassSymbol) = { classSymbol.typeSignature.members.filterNot(x => - x.fullName.startsWith("java") || x.fullName.startsWith("scala")) - .filter(x => isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x)).map(_.fullName) ++ - getInnerFunctions(classSymbol) + x.fullName.startsWith("java") || x.fullName.startsWith("scala") + ).filter(x => + isPackagePrivate(x) || isDeveloperApi(x) || isExperimental(x) + ).map(_.fullName) ++ getInnerFunctions(classSymbol) } def main(args: Array[String]) { From e5749a1342327263dc6b94ba470e392fbea703fa Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 31 Jul 2014 12:26:36 -0700 Subject: [PATCH 0158/1492] SPARK-2646. log4j initialization not quite compatible with log4j 2.x The logging code that handles log4j initialization leads to an stack overflow error when used with log4j 2.x, which has just been released. This occurs even a downstream project has correctly adjusted SLF4J bindings, and that is the right thing to do for log4j 2.x, since it is effectively a separate project from 1.x. Here is the relevant bit of Logging.scala: ``` private def initializeLogging() { // If Log4j is being used, but is not initialized, load a default properties file val binder = StaticLoggerBinder.getSingleton val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory") val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized && usingLog4j) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } Logging.initialized = true // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html log } ``` The first minor issue is that there is a call to a logger inside this method, which is initializing logging. In this situation, it ends up causing the initialization to be called recursively until the stack overflow. It would be slightly tidier to log this only after Logging.initialized = true. Or not at all. But it's not the root problem, or else, it would not work at all now. The calls to log4j classes here always reference log4j 1.2 no matter what. For example, there is not getAllAppenders in log4j 2.x. That's fine. Really, "usingLog4j" means "using log4j 1.2" and "log4jInitialized" means "log4j 1.2 is initialized". usingLog4j should be false for log4j 2.x, because the initialization only matters for log4j 1.2. But, it's true, and that's the real issue. And log4jInitialized is always false, since calls to the log4j 1.2 API are stubs and no-ops in this setup, where the caller has swapped in log4j 2.x. Hence the loop. This is fixed, I believe, if "usingLog4j" can be false for log4j 2.x. The SLF4J static binding class has the same name for both versions, unfortunately, which causes the issue. However they're in different packages. For example, if the test included "... and begins with org.slf4j", it should work, as the SLF4J binding for log4j 2.x is provided by log4j 2.x at the moment, and is in package org.apache.logging.slf4j. Of course, I assume that SLF4J will eventually offer its own binding. I hope to goodness they at least name the binding class differently, or else this will again not work. But then some other check can probably be made. Author: Sean Owen Closes #1547 from srowen/SPARK-2646 and squashes the following commits: 92a9898 [Sean Owen] System.out -> System.err 94be4c7 [Sean Owen] Add back log message as System.out, with informational comment a7f8876 [Sean Owen] Updates from review 6f3c1d3 [Sean Owen] Remove log statement in logging initialization, and distinguish log4j 1.2 from 2.0, to avoid stack overflow in initialization --- .../main/scala/org/apache/spark/Logging.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 50d8e93e1f0d7..807ef3e9c9d60 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -45,10 +45,7 @@ trait Logging { initializeIfNecessary() var className = this.getClass.getName // Ignore trailing $'s in the class names for Scala objects - if (className.endsWith("$")) { - className = className.substring(0, className.length - 1) - } - log_ = LoggerFactory.getLogger(className) + log_ = LoggerFactory.getLogger(className.stripSuffix("$")) } log_ } @@ -110,23 +107,27 @@ trait Logging { } private def initializeLogging() { - // If Log4j is being used, but is not initialized, load a default properties file - val binder = StaticLoggerBinder.getSingleton - val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory") - val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements - if (!log4jInitialized && usingLog4j) { + // Don't use a logger in here, as this is itself occurring during initialization of a logger + // If Log4j 1.2 is being used, but is not initialized, load a default properties file + val binderClass = StaticLoggerBinder.getSingleton.getLoggerFactoryClassStr + // This distinguishes the log4j 1.2 binding, currently + // org.slf4j.impl.Log4jLoggerFactory, from the log4j 2.0 binding, currently + // org.apache.logging.slf4j.Log4jLoggerFactory + val usingLog4j12 = "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass) + val log4j12Initialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements + if (!log4j12Initialized && usingLog4j12) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => PropertyConfigurator.configure(url) - log.info(s"Using Spark's default log4j profile: $defaultLogProps") + System.err.println(s"Using Spark's default log4j profile: $defaultLogProps") case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } Logging.initialized = true - // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads + // Force a call into slf4j to initialize it. Avoids this happening from multiple threads // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html log } From dc0865bc7e119fe507061c27069c17523b87dfea Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 31 Jul 2014 12:55:00 -0700 Subject: [PATCH 0159/1492] [SPARK-2511][MLLIB] add HashingTF and IDF This is roughly the TF-IDF implementation used in the Databricks Cloud Demo: http://databricks.com/cloud/ . Both `HashingTF` and `IDF` are implemented as transformers, similar to scikit-learn. Author: Xiangrui Meng Closes #1671 from mengxr/tfidf and squashes the following commits: 7d65888 [Xiangrui Meng] use JavaConverters._ 5fe9ec4 [Xiangrui Meng] fix unit test 6e214ec [Xiangrui Meng] add apache header cfd9aed [Xiangrui Meng] add Java-friendly methods move classes to mllib.feature 3814440 [Xiangrui Meng] add HashingTF and IDF --- .../spark/mllib/feature/HashingTF.scala | 79 +++++++ .../org/apache/spark/mllib/feature/IDF.scala | 194 ++++++++++++++++++ .../spark/mllib/feature/JavaTfIdfSuite.java | 66 ++++++ .../spark/mllib/feature/HashingTFSuite.scala | 52 +++++ .../apache/spark/mllib/feature/IDFSuite.scala | 63 ++++++ 5 files changed, 454 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala create mode 100644 mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala new file mode 100644 index 0000000000000..0f6d5809e098f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -0,0 +1,79 @@ +/* + * 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.feature + +import java.lang.{Iterable => JavaIterable} + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +/** + * :: Experimental :: + * Maps a sequence of terms to their term frequencies using the hashing trick. + * + * @param numFeatures number of features (default: 1000000) + */ +@Experimental +class HashingTF(val numFeatures: Int) extends Serializable { + + def this() = this(1000000) + + /** + * Returns the index of the input term. + */ + def indexOf(term: Any): Int = Utils.nonNegativeMod(term.##, numFeatures) + + /** + * Transforms the input document into a sparse term frequency vector. + */ + def transform(document: Iterable[_]): Vector = { + val termFrequencies = mutable.HashMap.empty[Int, Double] + document.foreach { term => + val i = indexOf(term) + termFrequencies.put(i, termFrequencies.getOrElse(i, 0.0) + 1.0) + } + Vectors.sparse(numFeatures, termFrequencies.toSeq) + } + + /** + * Transforms the input document into a sparse term frequency vector (Java version). + */ + def transform(document: JavaIterable[_]): Vector = { + transform(document.asScala) + } + + /** + * Transforms the input document to term frequency vectors. + */ + def transform[D <: Iterable[_]](dataset: RDD[D]): RDD[Vector] = { + dataset.map(this.transform) + } + + /** + * Transforms the input document to term frequency vectors (Java version). + */ + def transform[D <: JavaIterable[_]](dataset: JavaRDD[D]): JavaRDD[Vector] = { + dataset.rdd.map(this.transform).toJavaRDD() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala new file mode 100644 index 0000000000000..7ed611a857acc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -0,0 +1,194 @@ +/* + * 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.feature + +import breeze.linalg.{DenseVector => BDV} + +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * Inverse document frequency (IDF). + * The standard formulation is used: `idf = log((m + 1) / (d(t) + 1))`, where `m` is the total + * number of documents and `d(t)` is the number of documents that contain term `t`. + */ +@Experimental +class IDF { + + // TODO: Allow different IDF formulations. + + private var brzIdf: BDV[Double] = _ + + /** + * Computes the inverse document frequency. + * @param dataset an RDD of term frequency vectors + */ + def fit(dataset: RDD[Vector]): this.type = { + brzIdf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator)( + seqOp = (df, v) => df.add(v), + combOp = (df1, df2) => df1.merge(df2) + ).idf() + this + } + + /** + * Computes the inverse document frequency. + * @param dataset a JavaRDD of term frequency vectors + */ + def fit(dataset: JavaRDD[Vector]): this.type = { + fit(dataset.rdd) + } + + /** + * Transforms term frequency (TF) vectors to TF-IDF vectors. + * @param dataset an RDD of term frequency vectors + * @return an RDD of TF-IDF vectors + */ + def transform(dataset: RDD[Vector]): RDD[Vector] = { + if (!initialized) { + throw new IllegalStateException("Haven't learned IDF yet. Call fit first.") + } + val theIdf = brzIdf + val bcIdf = dataset.context.broadcast(theIdf) + dataset.mapPartitions { iter => + val thisIdf = bcIdf.value + iter.map { v => + val n = v.size + v match { + case sv: SparseVector => + val nnz = sv.indices.size + val newValues = new Array[Double](nnz) + var k = 0 + while (k < nnz) { + newValues(k) = sv.values(k) * thisIdf(sv.indices(k)) + k += 1 + } + Vectors.sparse(n, sv.indices, newValues) + case dv: DenseVector => + val newValues = new Array[Double](n) + var j = 0 + while (j < n) { + newValues(j) = dv.values(j) * thisIdf(j) + j += 1 + } + Vectors.dense(newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } + } + } + + /** + * Transforms term frequency (TF) vectors to TF-IDF vectors (Java version). + * @param dataset a JavaRDD of term frequency vectors + * @return a JavaRDD of TF-IDF vectors + */ + def transform(dataset: JavaRDD[Vector]): JavaRDD[Vector] = { + transform(dataset.rdd).toJavaRDD() + } + + /** Returns the IDF vector. */ + def idf(): Vector = { + if (!initialized) { + throw new IllegalStateException("Haven't learned IDF yet. Call fit first.") + } + Vectors.fromBreeze(brzIdf) + } + + private def initialized: Boolean = brzIdf != null +} + +private object IDF { + + /** Document frequency aggregator. */ + class DocumentFrequencyAggregator extends Serializable { + + /** number of documents */ + private var m = 0L + /** document frequency vector */ + private var df: BDV[Long] = _ + + /** Adds a new document. */ + def add(doc: Vector): this.type = { + if (isEmpty) { + df = BDV.zeros(doc.size) + } + doc match { + case sv: SparseVector => + val nnz = sv.indices.size + var k = 0 + while (k < nnz) { + if (sv.values(k) > 0) { + df(sv.indices(k)) += 1L + } + k += 1 + } + case dv: DenseVector => + val n = dv.size + var j = 0 + while (j < n) { + if (dv.values(j) > 0.0) { + df(j) += 1L + } + j += 1 + } + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + m += 1L + this + } + + /** Merges another. */ + def merge(other: DocumentFrequencyAggregator): this.type = { + if (!other.isEmpty) { + m += other.m + if (df == null) { + df = other.df.copy + } else { + df += other.df + } + } + this + } + + private def isEmpty: Boolean = m == 0L + + /** Returns the current IDF vector. */ + def idf(): BDV[Double] = { + if (isEmpty) { + throw new IllegalStateException("Haven't seen any document yet.") + } + val n = df.length + val inv = BDV.zeros[Double](n) + var j = 0 + while (j < n) { + inv(j) = math.log((m + 1.0)/ (df(j) + 1.0)) + j += 1 + } + inv + } + } +} diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java new file mode 100644 index 0000000000000..e8d99f4ae43ae --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -0,0 +1,66 @@ +/* + * 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.feature; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; + +public class JavaTfIdfSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaTfIdfSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void tfIdf() { + // The tests are to check Java compatibility. + HashingTF tf = new HashingTF(); + JavaRDD> documents = sc.parallelize(Lists.newArrayList( + Lists.newArrayList("this is a sentence".split(" ")), + Lists.newArrayList("this is another sentence".split(" ")), + Lists.newArrayList("this is still a sentence".split(" "))), 2); + JavaRDD termFreqs = tf.transform(documents); + termFreqs.collect(); + IDF idf = new IDF(); + JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); + List localTfIdfs = tfIdfs.collect(); + int indexOfThis = tf.indexOf("this"); + for (Vector v: localTfIdfs) { + Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala new file mode 100644 index 0000000000000..a599e0d938569 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/HashingTFSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext + +class HashingTFSuite extends FunSuite with LocalSparkContext { + + test("hashing tf on a single doc") { + val hashingTF = new HashingTF(1000) + val doc = "a a b b c d".split(" ") + val n = hashingTF.numFeatures + val termFreqs = Seq( + (hashingTF.indexOf("a"), 2.0), + (hashingTF.indexOf("b"), 2.0), + (hashingTF.indexOf("c"), 1.0), + (hashingTF.indexOf("d"), 1.0)) + assert(termFreqs.map(_._1).forall(i => i >= 0 && i < n), + "index must be in range [0, #features)") + assert(termFreqs.map(_._1).toSet.size === 4, "expecting perfect hashing") + val expected = Vectors.sparse(n, termFreqs) + assert(hashingTF.transform(doc) === expected) + } + + test("hashing tf on an RDD") { + val hashingTF = new HashingTF + val localDocs: Seq[Seq[String]] = Seq( + "a a b b b c d".split(" "), + "a b c d a b c".split(" "), + "c b a c b a a".split(" ")) + val docs = sc.parallelize(localDocs, 2) + assert(hashingTF.transform(docs).collect().toSet === localDocs.map(hashingTF.transform).toSet) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala new file mode 100644 index 0000000000000..78a2804ff204b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala @@ -0,0 +1,63 @@ +/* + * 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.feature + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class IDFSuite extends FunSuite with LocalSparkContext { + + test("idf") { + val n = 4 + val localTermFrequencies = Seq( + Vectors.sparse(n, Array(1, 3), Array(1.0, 2.0)), + Vectors.dense(0.0, 1.0, 2.0, 3.0), + Vectors.sparse(n, Array(1), Array(1.0)) + ) + val m = localTermFrequencies.size + val termFrequencies = sc.parallelize(localTermFrequencies, 2) + val idf = new IDF + intercept[IllegalStateException] { + idf.idf() + } + intercept[IllegalStateException] { + idf.transform(termFrequencies) + } + idf.fit(termFrequencies) + val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => + math.log((m.toDouble + 1.0) / (x + 1.0)) + }) + assert(idf.idf() ~== expected absTol 1e-12) + val tfidf = idf.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() + assert(tfidf.size === 3) + val tfidf0 = tfidf(0L).asInstanceOf[SparseVector] + assert(tfidf0.indices === Array(1, 3)) + assert(Vectors.dense(tfidf0.values) ~== + Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) + val tfidf1 = tfidf(1L).asInstanceOf[DenseVector] + assert(Vectors.dense(tfidf1.values) ~== + Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) + val tfidf2 = tfidf(2L).asInstanceOf[SparseVector] + assert(tfidf2.indices === Array(1)) + assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + } +} From 49b361298b09d415de1857846367913495aecfa6 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 31 Jul 2014 13:05:24 -0700 Subject: [PATCH 0160/1492] [SPARK-2523] [SQL] Hadoop table scan bug fixing (fix failing Jenkins maven test) This PR tries to resolve the broken Jenkins maven test issue introduced by #1439. Now, we create a single query test to run both the setup work and the test query. Author: Yin Huai Closes #1669 from yhuai/SPARK-2523-fixTest and squashes the following commits: 358af1a [Yin Huai] Make partition_based_table_scan_with_different_serde run atomically. --- ...t_serde-0-1436cccda63b78dd6e43a399da6cc474 | 0 ...t_serde-1-8d9bf54373f45bc35f8cb6e82771b154 | 0 ...t_serde-2-7816c17905012cf381abf93d230faa8d | 0 ...t_serde-3-90089a6db3c3d8ee5ff5ea6b9153b3cc | 0 ..._serde-4-8caed2a6e80250a6d38a59388679c298} | 0 .../hive/execution/HiveTableScanSuite.scala | 45 ++++++++----------- 6 files changed, 19 insertions(+), 26 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-1436cccda63b78dd6e43a399da6cc474 create mode 100644 sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-1-8d9bf54373f45bc35f8cb6e82771b154 create mode 100644 sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-2-7816c17905012cf381abf93d230faa8d create mode 100644 sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-3-90089a6db3c3d8ee5ff5ea6b9153b3cc rename sql/hive/src/test/resources/golden/{partition_based_table_scan_with_different_serde-0-8caed2a6e80250a6d38a59388679c298 => partition_based_table_scan_with_different_serde-4-8caed2a6e80250a6d38a59388679c298} (100%) diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-1436cccda63b78dd6e43a399da6cc474 b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-1436cccda63b78dd6e43a399da6cc474 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-1-8d9bf54373f45bc35f8cb6e82771b154 b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-1-8d9bf54373f45bc35f8cb6e82771b154 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-2-7816c17905012cf381abf93d230faa8d b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-2-7816c17905012cf381abf93d230faa8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-3-90089a6db3c3d8ee5ff5ea6b9153b3cc b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-3-90089a6db3c3d8ee5ff5ea6b9153b3cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-8caed2a6e80250a6d38a59388679c298 b/sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-4-8caed2a6e80250a6d38a59388679c298 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-0-8caed2a6e80250a6d38a59388679c298 rename to sql/hive/src/test/resources/golden/partition_based_table_scan_with_different_serde-4-8caed2a6e80250a6d38a59388679c298 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index bcb00f871d185..c5736723b47c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -17,32 +17,25 @@ package org.apache.spark.sql.hive.execution -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.hive.test.TestHive - class HiveTableScanSuite extends HiveComparisonTest { - // MINOR HACK: You must run a query before calling reset the first time. - TestHive.hql("SHOW TABLES") - TestHive.reset() - - TestHive.hql("""CREATE TABLE part_scan_test (key STRING, value STRING) PARTITIONED BY (ds STRING) - | ROW FORMAT SERDE - | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' - | STORED AS RCFILE - """.stripMargin) - TestHive.hql("""FROM src - | INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-01') - | SELECT 100,100 LIMIT 1 - """.stripMargin) - TestHive.hql("""ALTER TABLE part_scan_test SET SERDE - | 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' - """.stripMargin) - TestHive.hql("""FROM src INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-02') - | SELECT 200,200 LIMIT 1 - """.stripMargin) - createQueryTest("partition_based_table_scan_with_different_serde", - "SELECT * from part_scan_test", false) + createQueryTest("partition_based_table_scan_with_different_serde", + """ + |CREATE TABLE part_scan_test (key STRING, value STRING) PARTITIONED BY (ds STRING) + |ROW FORMAT SERDE + |'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' + |STORED AS RCFILE; + | + |FROM src + |INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-01') + |SELECT 100,100 LIMIT 1; + | + |ALTER TABLE part_scan_test SET SERDE + |'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + | + |FROM src INSERT INTO TABLE part_scan_test PARTITION (ds='2010-01-02') + |SELECT 200,200 LIMIT 1; + | + |SELECT * from part_scan_test; + """.stripMargin) } From e02136214a6c2635e88c36b1f530a97e975d83e3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 31 Jul 2014 14:35:09 -0700 Subject: [PATCH 0161/1492] Improvements to merge_spark_pr.py This commit fixes a couple of issues in the merge_spark_pr.py developer script: - Allow recovery from failed cherry-picks. - Fix detection of pull requests that have already been merged. Both of these fixes are useful when backporting changes. Author: Josh Rosen Closes #1668 from JoshRosen/pr-script-improvements and squashes the following commits: ff4f33a [Josh Rosen] Default SPARK_HOME to cwd(); detect missing JIRA credentials. ed5bc57 [Josh Rosen] Improvements for backporting using merge_spark_pr: --- dev/merge_spark_pr.py | 53 +++++++++++++++++++++++++++---------------- 1 file changed, 34 insertions(+), 19 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index c44320239bbbf..53df9b5a3f1d5 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -29,7 +29,6 @@ import re import subprocess import sys -import tempfile import urllib2 try: @@ -39,15 +38,15 @@ JIRA_IMPORTED = False # Location of your Spark git development area -SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") +SPARK_HOME = os.environ.get("SPARK_HOME", os.getcwd()) # Remote name which points to the Gihub site PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") # ASF JIRA username -JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "") # ASF JIRA password -JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234") +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "") GITHUB_BASE = "https://github.com/apache/spark/pull" GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" @@ -129,7 +128,7 @@ def merge_pr(pr_num, target_ref): merge_message_flags = [] merge_message_flags += ["-m", title] - if body != None: + if body is not None: # We remove @ symbols from the body to avoid triggering e-mails # to people every time someone creates a public fork of Spark. merge_message_flags += ["-m", body.replace("@", "")] @@ -179,7 +178,14 @@ def cherry_pick(pr_num, merge_hash, default_branch): run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, pick_ref, pick_branch_name)) run_cmd("git checkout %s" % pick_branch_name) - run_cmd("git cherry-pick -sx %s" % merge_hash) + + try: + run_cmd("git cherry-pick -sx %s" % merge_hash) + except Exception as e: + msg = "Error cherry-picking: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and finish the cherry-pick. Finished?" + continue_maybe(msg) continue_maybe("Pick complete (local ref %s). Push to %s?" % ( pick_branch_name, PUSH_REMOTE_NAME)) @@ -280,6 +286,7 @@ def get_version_json(version_str): pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) +pr_events = get_json("%s/issues/%s/events" % (GITHUB_API_BASE, pr_num)) url = pr["url"] title = pr["title"] @@ -289,19 +296,23 @@ def get_version_json(version_str): base_ref = pr["head"]["ref"] pr_repo_desc = "%s/%s" % (user_login, base_ref) -if pr["merged"] is True: +# Merged pull requests don't appear as merged in the GitHub API; +# Instead, they're closed by asfgit. +merge_commits = \ + [e for e in pr_events if e["actor"]["login"] == "asfgit" and e["event"] == "closed"] + +if merge_commits: + merge_hash = merge_commits[0]["commit_id"] + message = get_json("%s/commits/%s" % (GITHUB_API_BASE, merge_hash))["commit"]["message"] + print "Pull request %s has already been merged, assuming you want to backport" % pr_num - merge_commit_desc = run_cmd([ - 'git', 'log', '--merges', '--first-parent', - '--grep=pull request #%s' % pr_num, '--oneline']).split("\n")[0] - if merge_commit_desc == "": + commit_is_downloaded = run_cmd(['git', 'rev-parse', '--quiet', '--verify', + "%s^{commit}" % merge_hash]).strip() != "" + if not commit_is_downloaded: fail("Couldn't find any merge commit for #%s, you may need to update HEAD." % pr_num) - merge_hash = merge_commit_desc[:7] - message = merge_commit_desc[8:] - - print "Found: %s" % message - maybe_cherry_pick(pr_num, merge_hash, latest_branch) + print "Found commit %s:\n%s" % (merge_hash, message) + cherry_pick(pr_num, merge_hash, latest_branch) sys.exit(0) if not bool(pr["mergeable"]): @@ -323,9 +334,13 @@ def get_version_json(version_str): merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] if JIRA_IMPORTED: - continue_maybe("Would you like to update an associated JIRA?") - jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) - resolve_jira(title, merged_refs, jira_comment) + if JIRA_USERNAME and JIRA_PASSWORD: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira(title, merged_refs, jira_comment) + else: + print "JIRA_USERNAME and JIRA_PASSWORD not set" + print "Exiting without trying to close the associated JIRA." else: print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." print "Exiting without trying to close the associated JIRA." From cc820502fb08f71b03237103153c34487b2600b4 Mon Sep 17 00:00:00 2001 From: kballou Date: Thu, 31 Jul 2014 14:58:52 -0700 Subject: [PATCH 0162/1492] Docs: monitoring, streaming programming guide Fix several awkward wordings and grammatical issues in the following documents: * docs/monitoring.md * docs/streaming-programming-guide.md Author: kballou Closes #1662 from kennyballou/grammar_fixes and squashes the following commits: e1b8ad6 [kballou] Docs: monitoring, streaming programming guide --- docs/monitoring.md | 4 ++-- docs/streaming-programming-guide.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/monitoring.md b/docs/monitoring.md index 84073fe4d949a..d07ec4a57a2cc 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -33,7 +33,7 @@ application's UI after the application has finished. If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished application through Spark's history server, provided that the application's event logs exist. -You can start a the history server by executing: +You can start the history server by executing: ./sbin/start-history-server.sh @@ -106,7 +106,7 @@ follows:
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 90a0eef60c200..7b8b7933434c4 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -939,7 +939,7 @@ Receiving multiple data streams can therefore be achieved by creating multiple i and configuring them to receive different partitions of the data stream from the source(s). For example, a single Kafka input stream receiving two topics of data can be split into two Kafka input streams, each receiving only one topic. This would run two receivers on two workers, -thus allowing data to received in parallel, and increasing overall throughput. +thus allowing data to be received in parallel, and increasing overall throughput. Another parameter that should be considered is the receiver's blocking interval. For most receivers, the received data is coalesced together into large blocks of data before storing inside Spark's memory. @@ -980,7 +980,7 @@ If the number of tasks launched per second is high (say, 50 or more per second), of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: -* **Task Serialization**: Using Kryo serialization for serializing tasks can reduced the task +* **Task Serialization**: Using Kryo serialization for serializing tasks can reduce the task sizes, and therefore reduce the time taken to send them to the slaves. * **Execution mode**: Running Spark in Standalone mode or coarse-grained Mesos mode leads to From 492a195c5c4d68c85b8b1b48e3aa85165bbb5dc3 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Thu, 31 Jul 2014 15:07:26 -0700 Subject: [PATCH 0163/1492] SPARK-2740: allow user to specify ascending and numPartitions for sortBy... It should be more convenient if user can specify ascending and numPartitions when calling sortByKey. Author: Rui Li Closes #1645 from lirui-intel/spark-2740 and squashes the following commits: fb5d52e [Rui Li] SPARK-2740: allow user to specify ascending and numPartitions for sortByKey --- .../scala/org/apache/spark/api/java/JavaPairRDD.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 47708cb2e78bd..76d4193e96aea 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -783,6 +783,17 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) sortByKey(comp, ascending) } + /** + * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling + * `collect` or `save` on the resulting RDD will return or output an ordered list of records + * (in the `save` case, they will be written to multiple `part-X` files in the filesystem, in + * order of the keys). + */ + def sortByKey(ascending: Boolean, numPartitions: Int): JavaPairRDD[K, V] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]] + sortByKey(comp, ascending, numPartitions) + } + /** * Sort the RDD by key, so that each partition contains a sorted range of the elements. Calling * `collect` or `save` on the resulting RDD will return or output an ordered list of records From ef4ff00f87a4e8d38866f163f01741c2673e41da Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 31 Jul 2014 15:31:53 -0700 Subject: [PATCH 0164/1492] SPARK-2282: Reuse Socket for sending accumulator updates to Pyspark Prior to this change, every PySpark task completion opened a new socket to the accumulator server, passed its updates through, and then quit. I'm not entirely sure why PySpark always sends accumulator updates, but regardless this causes a very rapid buildup of ephemeral TCP connections that remain in the TCP_WAIT state for around a minute before being cleaned up. Rather than trying to allow these sockets to be cleaned up faster, this patch simply reuses the connection between tasks completions (since they're fed updates in a single-threaded manner by the DAGScheduler anyway). The only tricky part here was making sure that the AccumulatorServer was able to shutdown in a timely manner (i.e., stop polling for new data), and this was accomplished via minor feats of magic. I have confirmed that this patch eliminates the buildup of ephemeral sockets due to the accumulator updates. However, I did note that there were still significant sockets being created against the PySpark daemon port, but my machine was not able to create enough sockets fast enough to fail. This may not be the last time we've seen this issue, though. Author: Aaron Davidson Closes #1503 from aarondav/accum and squashes the following commits: b3e12f7 [Aaron Davidson] SPARK-2282: Reuse Socket for sending accumulator updates to Pyspark --- .../apache/spark/api/python/PythonRDD.scala | 20 ++++++++--- python/pyspark/accumulators.py | 34 +++++++++++++++---- 2 files changed, 42 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index a9d758bf998c3..94d666aa92025 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -731,19 +731,30 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536) + /** + * We try to reuse a single Socket to transfer accumulator updates, as they are all added + * by the DAGScheduler's single-threaded actor anyway. + */ + @transient var socket: Socket = _ + + def openSocket(): Socket = synchronized { + if (socket == null || socket.isClosed) { + socket = new Socket(serverHost, serverPort) + } + socket + } + override def zero(value: JList[Array[Byte]]): JList[Array[Byte]] = new JArrayList override def addInPlace(val1: JList[Array[Byte]], val2: JList[Array[Byte]]) - : JList[Array[Byte]] = { + : JList[Array[Byte]] = synchronized { if (serverHost == null) { // This happens on the worker node, where we just want to remember all the updates val1.addAll(val2) val1 } else { // This happens on the master, where we pass the updates to Python through a socket - val socket = new Socket(serverHost, serverPort) - // SPARK-2282: Immediately reuse closed sockets because we create one per task. - socket.setReuseAddress(true) + val socket = openSocket() val in = socket.getInputStream val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream, bufferSize)) out.writeInt(val2.size) @@ -757,7 +768,6 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: if (byteRead == -1) { throw new SparkException("EOF reached before Python server acknowledged") } - socket.close() null } } diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 2204e9c9ca701..45d36e5d0e764 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -86,6 +86,7 @@ Exception:... """ +import select import struct import SocketServer import threading @@ -209,19 +210,38 @@ def addInPlace(self, value1, value2): class _UpdateRequestHandler(SocketServer.StreamRequestHandler): + """ + This handler will keep polling updates from the same socket until the + server is shutdown. + """ + def handle(self): from pyspark.accumulators import _accumulatorRegistry - num_updates = read_int(self.rfile) - for _ in range(num_updates): - (aid, update) = pickleSer._read_with_length(self.rfile) - _accumulatorRegistry[aid] += update - # Write a byte in acknowledgement - self.wfile.write(struct.pack("!b", 1)) + while not self.server.server_shutdown: + # Poll every 1 second for new data -- don't block in case of shutdown. + r, _, _ = select.select([self.rfile], [], [], 1) + if self.rfile in r: + num_updates = read_int(self.rfile) + for _ in range(num_updates): + (aid, update) = pickleSer._read_with_length(self.rfile) + _accumulatorRegistry[aid] += update + # Write a byte in acknowledgement + self.wfile.write(struct.pack("!b", 1)) + +class AccumulatorServer(SocketServer.TCPServer): + """ + A simple TCP server that intercepts shutdown() in order to interrupt + our continuous polling on the handler. + """ + server_shutdown = False + def shutdown(self): + self.server_shutdown = True + SocketServer.TCPServer.shutdown(self) def _start_update_server(): """Start a TCP server to receive accumulator updates in a daemon thread, and returns it""" - server = SocketServer.TCPServer(("localhost", 0), _UpdateRequestHandler) + server = AccumulatorServer(("localhost", 0), _UpdateRequestHandler) thread = threading.Thread(target=server.serve_forever) thread.daemon = True thread.start() From 8f51491ea78d8e88fc664c2eac3b4ac14226d98f Mon Sep 17 00:00:00 2001 From: Zongheng Yang Date: Thu, 31 Jul 2014 19:32:16 -0700 Subject: [PATCH 0165/1492] [SPARK-2531 & SPARK-2436] [SQL] Optimize the BuildSide when planning BroadcastNestedLoopJoin. This PR resolves the following two tickets: - [SPARK-2531](https://issues.apache.org/jira/browse/SPARK-2531): BNLJ currently assumes the build side is the right relation. This patch refactors some of its logic to take into account a BuildSide properly. - [SPARK-2436](https://issues.apache.org/jira/browse/SPARK-2436): building on top of the above, we simply use the physical size statistics (if available) of both relations, and make the smaller relation the build side in the planner. Author: Zongheng Yang Closes #1448 from concretevitamin/bnlj-buildSide and squashes the following commits: 1780351 [Zongheng Yang] Use size estimation to decide optimal build side of BNLJ. 68e6c5b [Zongheng Yang] Consolidate two adjacent pattern matchings. 96d312a [Zongheng Yang] Use a while loop instead of collection methods chaining. 4bc525e [Zongheng Yang] Make BroadcastNestedLoopJoin take a BuildSide. --- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../apache/spark/sql/execution/joins.scala | 79 ++++++++++++------- 2 files changed, 55 insertions(+), 28 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5f1fe99f75c9d..d57b6eaf40b09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -155,8 +155,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object BroadcastNestedLoopJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, joinType, condition) => + val buildSide = + if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) BuildRight else BuildLeft execution.BroadcastNestedLoopJoin( - planLater(left), planLater(right), joinType, condition) :: Nil + planLater(left), planLater(right), buildSide, joinType, condition) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 2750ddbce896f..b068579db75cd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -314,10 +314,19 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod */ @DeveloperApi case class BroadcastNestedLoopJoin( - streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) - extends BinaryNode { + left: SparkPlan, + right: SparkPlan, + buildSide: BuildSide, + joinType: JoinType, + condition: Option[Expression]) extends BinaryNode { // TODO: Override requiredChildDistribution. + /** BuildRight means the right relation <=> the broadcast relation. */ + val (streamed, broadcast) = buildSide match { + case BuildRight => (left, right) + case BuildLeft => (right, left) + } + override def outputPartitioning: Partitioning = streamed.outputPartitioning override def output = { @@ -333,11 +342,6 @@ case class BroadcastNestedLoopJoin( } } - /** The Streamed Relation */ - def left = streamed - /** The Broadcast relation */ - def right = broadcast - @transient lazy val boundCondition = InterpretedPredicate( condition @@ -348,57 +352,78 @@ case class BroadcastNestedLoopJoin( val broadcastedRelation = sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) - val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter => + /** All rows that either match both-way, or rows from streamed joined with nulls. */ + val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => val matchedRows = new ArrayBuffer[Row] // TODO: Use Spark's BitSet. - val includedBroadcastTuples = new BitSet(broadcastedRelation.value.size) + val includedBroadcastTuples = + new scala.collection.mutable.BitSet(broadcastedRelation.value.size) val joinedRow = new JoinedRow + val leftNulls = new GenericMutableRow(left.output.size) val rightNulls = new GenericMutableRow(right.output.size) streamedIter.foreach { streamedRow => var i = 0 - var matched = false + var streamRowMatched = false while (i < broadcastedRelation.value.size) { // TODO: One bitset per partition instead of per row. val broadcastedRow = broadcastedRelation.value(i) - if (boundCondition(joinedRow(streamedRow, broadcastedRow))) { - matchedRows += joinedRow(streamedRow, broadcastedRow).copy() - matched = true - includedBroadcastTuples += i + buildSide match { + case BuildRight if boundCondition(joinedRow(streamedRow, broadcastedRow)) => + matchedRows += joinedRow(streamedRow, broadcastedRow).copy() + streamRowMatched = true + includedBroadcastTuples += i + case BuildLeft if boundCondition(joinedRow(broadcastedRow, streamedRow)) => + matchedRows += joinedRow(broadcastedRow, streamedRow).copy() + streamRowMatched = true + includedBroadcastTuples += i + case _ => } i += 1 } - if (!matched && (joinType == LeftOuter || joinType == FullOuter)) { - matchedRows += joinedRow(streamedRow, rightNulls).copy() + (streamRowMatched, joinType, buildSide) match { + case (false, LeftOuter | FullOuter, BuildRight) => + matchedRows += joinedRow(streamedRow, rightNulls).copy() + case (false, RightOuter | FullOuter, BuildLeft) => + matchedRows += joinedRow(leftNulls, streamedRow).copy() + case _ => } } Iterator((matchedRows, includedBroadcastTuples)) } - val includedBroadcastTuples = streamedPlusMatches.map(_._2) + val includedBroadcastTuples = matchesOrStreamedRowsWithNulls.map(_._2) val allIncludedBroadcastTuples = if (includedBroadcastTuples.count == 0) { new scala.collection.mutable.BitSet(broadcastedRelation.value.size) } else { - streamedPlusMatches.map(_._2).reduce(_ ++ _) + includedBroadcastTuples.reduce(_ ++ _) } val leftNulls = new GenericMutableRow(left.output.size) - val rightOuterMatches: Seq[Row] = - if (joinType == RightOuter || joinType == FullOuter) { - broadcastedRelation.value.zipWithIndex.filter { - case (row, i) => !allIncludedBroadcastTuples.contains(i) - }.map { - case (row, _) => new JoinedRow(leftNulls, row) + val rightNulls = new GenericMutableRow(right.output.size) + /** Rows from broadcasted joined with nulls. */ + val broadcastRowsWithNulls: Seq[Row] = { + val arrBuf: collection.mutable.ArrayBuffer[Row] = collection.mutable.ArrayBuffer() + var i = 0 + val rel = broadcastedRelation.value + while (i < rel.length) { + if (!allIncludedBroadcastTuples.contains(i)) { + (joinType, buildSide) match { + case (RightOuter | FullOuter, BuildRight) => arrBuf += new JoinedRow(leftNulls, rel(i)) + case (LeftOuter | FullOuter, BuildLeft) => arrBuf += new JoinedRow(rel(i), rightNulls) + case _ => + } } - } else { - Vector() + i += 1 } + arrBuf.toSeq + } // TODO: Breaks lineage. sparkContext.union( - streamedPlusMatches.flatMap(_._1), sparkContext.makeRDD(rightOuterMatches)) + matchesOrStreamedRowsWithNulls.flatMap(_._1), sparkContext.makeRDD(broadcastRowsWithNulls)) } } From d8430148ee1f6ba02569db0538eeae473a32c78e Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 31 Jul 2014 20:32:57 -0700 Subject: [PATCH 0166/1492] [SPARK-2724] Python version of RandomRDDGenerators RandomRDDGenerators but without support for randomRDD and randomVectorRDD, which take in arbitrary DistributionGenerator. `randomRDD.py` is named to avoid collision with the built-in Python `random` package. Author: Doris Xin Closes #1628 from dorx/pythonRDD and squashes the following commits: 55c6de8 [Doris Xin] review comments. all python units passed. f831d9b [Doris Xin] moved default args logic into PythonMLLibAPI 2d73917 [Doris Xin] fix for linalg.py 8663e6a [Doris Xin] reverting back to a single python file for random f47c481 [Doris Xin] docs update 687aac0 [Doris Xin] add RandomRDDGenerators.py to run-tests 4338f40 [Doris Xin] renamed randomRDD to rand and import as random 29d205e [Doris Xin] created mllib.random package bd2df13 [Doris Xin] typos 07ddff2 [Doris Xin] units passed. 23b2ecd [Doris Xin] WIP --- .../mllib/api/python/PythonMLLibAPI.scala | 97 ++++++++++ .../mllib/random/RandomRDDGenerators.scala | 90 +++++---- python/pyspark/__init__.py | 10 + python/pyspark/mllib/linalg.py | 4 + python/pyspark/mllib/random.py | 182 ++++++++++++++++++ python/run-tests | 1 + 6 files changed, 348 insertions(+), 36 deletions(-) create mode 100644 python/pyspark/mllib/random.py diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 954621ee8b933..d2e8ccf208970 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -24,10 +24,12 @@ import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.random.{RandomRDDGenerators => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -453,4 +455,99 @@ class PythonMLLibAPI extends Serializable { val ratings = ratingsBytesJRDD.rdd.map(unpackRating) ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) } + + // Used by the *RDD methods to get default seed if not passed in from pyspark + private def getSeedOrDefault(seed: java.lang.Long): Long = { + if (seed == null) Utils.random.nextLong else seed + } + + // Used by *RDD methods to get default numPartitions if not passed in from pyspark + private def getNumPartitionsOrDefault(numPartitions: java.lang.Integer, + jsc: JavaSparkContext): Int = { + if (numPartitions == null) { + jsc.sc.defaultParallelism + } else { + numPartitions + } + } + + // Note: for the following methods, numPartitions and seed are boxed to allow nulls to be passed + // in for either argument from pyspark + + /** + * Java stub for Python mllib RandomRDDGenerators.uniformRDD() + */ + def uniformRDD(jsc: JavaSparkContext, + size: Long, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.uniformRDD(jsc.sc, size, parts, s).map(serializeDouble) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.normalRDD() + */ + def normalRDD(jsc: JavaSparkContext, + size: Long, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.normalRDD(jsc.sc, size, parts, s).map(serializeDouble) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.poissonRDD() + */ + def poissonRDD(jsc: JavaSparkContext, + mean: Double, + size: Long, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.poissonRDD(jsc.sc, mean, size, parts, s).map(serializeDouble) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.uniformVectorRDD() + */ + def uniformVectorRDD(jsc: JavaSparkContext, + numRows: Long, + numCols: Int, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s).map(serializeDoubleVector) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.normalVectorRDD() + */ + def normalVectorRDD(jsc: JavaSparkContext, + numRows: Long, + numCols: Int, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s).map(serializeDoubleVector) + } + + /** + * Java stub for Python mllib RandomRDDGenerators.poissonVectorRDD() + */ + def poissonVectorRDD(jsc: JavaSparkContext, + mean: Double, + numRows: Long, + numCols: Int, + numPartitions: java.lang.Integer, + seed: java.lang.Long): JavaRDD[Array[Byte]] = { + val parts = getNumPartitionsOrDefault(numPartitions, jsc) + val s = getSeedOrDefault(seed) + RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(serializeDoubleVector) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala index d7ee2d3f46846..021d651d4dbaa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala @@ -26,14 +26,17 @@ import org.apache.spark.util.Utils /** * :: Experimental :: - * Generator methods for creating RDDs comprised of i.i.d samples from some distribution. + * Generator methods for creating RDDs comprised of i.i.d. samples from some distribution. */ @Experimental object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. + * + * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use + * `RandomRDDGenerators.uniformRDD(sc, n, p, seed).map(v => a + (b - a) * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. @@ -49,7 +52,10 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. + * + * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use + * `RandomRDDGenerators.uniformRDD(sc, n, p).map(v => a + (b - a) * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. @@ -63,9 +69,12 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the uniform distribution on [0.0, 1.0]. + * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. * sc.defaultParallelism used for the number of partitions in the RDD. * + * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use + * `RandomRDDGenerators.uniformRDD(sc, n).map(v => a + (b - a) * v)`. + * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. @@ -77,7 +86,10 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. + * + * To transform the distribution in the generated RDD from standard normal to some other normal + * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p, seed).map(v => mean + sigma * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. @@ -93,7 +105,10 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. + * + * To transform the distribution in the generated RDD from standard normal to some other normal + * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p).map(v => mean + sigma * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. @@ -107,9 +122,12 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the standard normal distribution. + * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. * sc.defaultParallelism used for the number of partitions in the RDD. * + * To transform the distribution in the generated RDD from standard normal to some other normal + * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n).map(v => mean + sigma * v)`. + * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). @@ -121,7 +139,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. * * @param sc SparkContext used to create the RDD. * @param mean Mean, or lambda, for the Poisson distribution. @@ -142,7 +160,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. * * @param sc SparkContext used to create the RDD. * @param mean Mean, or lambda, for the Poisson distribution. @@ -157,7 +175,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples from the Poisson distribution with the input mean. + * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. * sc.defaultParallelism used for the number of partitions in the RDD. * * @param sc SparkContext used to create the RDD. @@ -172,7 +190,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. * * @param sc SparkContext used to create the RDD. * @param generator DistributionGenerator used to populate the RDD. @@ -192,7 +210,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. * * @param sc SparkContext used to create the RDD. * @param generator DistributionGenerator used to populate the RDD. @@ -210,7 +228,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD comprised of i.i.d samples produced by the input DistributionGenerator. + * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. * sc.defaultParallelism used for the number of partitions in the RDD. * * @param sc SparkContext used to create the RDD. @@ -229,7 +247,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * uniform distribution on [0.0 1.0]. * * @param sc SparkContext used to create the RDD. @@ -251,14 +269,14 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * uniform distribution on [0.0 1.0]. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. */ @Experimental def uniformVectorRDD(sc: SparkContext, @@ -270,14 +288,14 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * uniform distribution on [0.0 1.0]. * sc.defaultParallelism used for the number of partitions in the RDD. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. */ @Experimental def uniformVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { @@ -286,7 +304,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * standard normal distribution. * * @param sc SparkContext used to create the RDD. @@ -294,7 +312,7 @@ object RandomRDDGenerators { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). */ @Experimental def normalVectorRDD(sc: SparkContext, @@ -308,14 +326,14 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * standard normal distribution. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). */ @Experimental def normalVectorRDD(sc: SparkContext, @@ -327,14 +345,14 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * standard normal distribution. * sc.defaultParallelism used for the number of partitions in the RDD. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d samples ~ N(0.0, 1.0). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). */ @Experimental def normalVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { @@ -343,7 +361,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * Poisson distribution with the input mean. * * @param sc SparkContext used to create the RDD. @@ -352,7 +370,7 @@ object RandomRDDGenerators { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). */ @Experimental def poissonVectorRDD(sc: SparkContext, @@ -367,7 +385,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * Poisson distribution with the input mean. * * @param sc SparkContext used to create the RDD. @@ -375,7 +393,7 @@ object RandomRDDGenerators { * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). */ @Experimental def poissonVectorRDD(sc: SparkContext, @@ -388,7 +406,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples drawn from the + * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * Poisson distribution with the input mean. * sc.defaultParallelism used for the number of partitions in the RDD. * @@ -396,7 +414,7 @@ object RandomRDDGenerators { * @param mean Mean, or lambda, for the Poisson distribution. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d samples ~ Pois(mean). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). */ @Experimental def poissonVectorRDD(sc: SparkContext, @@ -408,7 +426,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the * input DistributionGenerator. * * @param sc SparkContext used to create the RDD. @@ -417,7 +435,7 @@ object RandomRDDGenerators { * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. */ @Experimental def randomVectorRDD(sc: SparkContext, @@ -431,7 +449,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the * input DistributionGenerator. * * @param sc SparkContext used to create the RDD. @@ -439,7 +457,7 @@ object RandomRDDGenerators { * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. */ @Experimental def randomVectorRDD(sc: SparkContext, @@ -452,7 +470,7 @@ object RandomRDDGenerators { /** * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d samples produced by the + * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the * input DistributionGenerator. * sc.defaultParallelism used for the number of partitions in the RDD. * @@ -460,7 +478,7 @@ object RandomRDDGenerators { * @param generator DistributionGenerator used to populate the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d samples produced by generator. + * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. */ @Experimental def randomVectorRDD(sc: SparkContext, diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 312c75d112cbf..c58555fc9d2c5 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -49,6 +49,16 @@ Main entry point for accessing data stored in Apache Hive.. """ +# The following block allows us to import python's random instead of mllib.random for scripts in +# mllib that depend on top level pyspark packages, which transitively depend on python's random. +# Since Python's import logic looks for modules in the current package first, we eliminate +# mllib.random as a candidate for C{import random} by removing the first search path, the script's +# location, in order to force the loader to look in Python's top-level modules for C{random}. +import sys +s = sys.path.pop(0) +import random +sys.path.insert(0, s) + from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.sql import SQLContext diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 71f4ad1a8d44e..54720c2324ca6 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -255,4 +255,8 @@ def _test(): exit(-1) if __name__ == "__main__": + # remove current path from list of search paths to avoid importing mllib.random + # for C{import random}, which is done in an external dependency of pyspark during doctests. + import sys + sys.path.pop(0) _test() diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py new file mode 100644 index 0000000000000..36e710dbae7a8 --- /dev/null +++ b/python/pyspark/mllib/random.py @@ -0,0 +1,182 @@ +# +# 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. +# + +""" +Python package for random data generation. +""" + + +from pyspark.rdd import RDD +from pyspark.mllib._common import _deserialize_double, _deserialize_double_vector +from pyspark.serializers import NoOpSerializer + +class RandomRDDGenerators: + """ + Generator methods for creating RDDs comprised of i.i.d samples from + some distribution. + """ + + @staticmethod + def uniformRDD(sc, size, numPartitions=None, seed=None): + """ + Generates an RDD comprised of i.i.d. samples from the + uniform distribution on [0.0, 1.0]. + + To transform the distribution in the generated RDD from U[0.0, 1.0] + to U[a, b], use + C{RandomRDDGenerators.uniformRDD(sc, n, p, seed)\ + .map(lambda v: a + (b - a) * v)} + + >>> x = RandomRDDGenerators.uniformRDD(sc, 100).collect() + >>> len(x) + 100 + >>> max(x) <= 1.0 and min(x) >= 0.0 + True + >>> RandomRDDGenerators.uniformRDD(sc, 100, 4).getNumPartitions() + 4 + >>> parts = RandomRDDGenerators.uniformRDD(sc, 100, seed=4).getNumPartitions() + >>> parts == sc.defaultParallelism + True + """ + jrdd = sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) + uniform = RDD(jrdd, sc, NoOpSerializer()) + return uniform.map(lambda bytes: _deserialize_double(bytearray(bytes))) + + @staticmethod + def normalRDD(sc, size, numPartitions=None, seed=None): + """ + Generates an RDD comprised of i.i.d samples from the standard normal + distribution. + + To transform the distribution in the generated RDD from standard normal + to some other normal N(mean, sigma), use + C{RandomRDDGenerators.normal(sc, n, p, seed)\ + .map(lambda v: mean + sigma * v)} + + >>> x = RandomRDDGenerators.normalRDD(sc, 1000, seed=1L) + >>> stats = x.stats() + >>> stats.count() + 1000L + >>> abs(stats.mean() - 0.0) < 0.1 + True + >>> abs(stats.stdev() - 1.0) < 0.1 + True + """ + jrdd = sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) + normal = RDD(jrdd, sc, NoOpSerializer()) + return normal.map(lambda bytes: _deserialize_double(bytearray(bytes))) + + @staticmethod + def poissonRDD(sc, mean, size, numPartitions=None, seed=None): + """ + Generates an RDD comprised of i.i.d samples from the Poisson + distribution with the input mean. + + >>> mean = 100.0 + >>> x = RandomRDDGenerators.poissonRDD(sc, mean, 1000, seed=1L) + >>> stats = x.stats() + >>> stats.count() + 1000L + >>> abs(stats.mean() - mean) < 0.5 + True + >>> from math import sqrt + >>> abs(stats.stdev() - sqrt(mean)) < 0.5 + True + """ + jrdd = sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) + poisson = RDD(jrdd, sc, NoOpSerializer()) + return poisson.map(lambda bytes: _deserialize_double(bytearray(bytes))) + + @staticmethod + def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): + """ + Generates an RDD comprised of vectors containing i.i.d samples drawn + from the uniform distribution on [0.0 1.0]. + + >>> import numpy as np + >>> mat = np.matrix(RandomRDDGenerators.uniformVectorRDD(sc, 10, 10).collect()) + >>> mat.shape + (10, 10) + >>> mat.max() <= 1.0 and mat.min() >= 0.0 + True + >>> RandomRDDGenerators.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() + 4 + """ + jrdd = sc._jvm.PythonMLLibAPI() \ + .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + uniform = RDD(jrdd, sc, NoOpSerializer()) + return uniform.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) + + @staticmethod + def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): + """ + Generates an RDD comprised of vectors containing i.i.d samples drawn + from the standard normal distribution. + + >>> import numpy as np + >>> mat = np.matrix(RandomRDDGenerators.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat.shape + (100, 100) + >>> abs(mat.mean() - 0.0) < 0.1 + True + >>> abs(mat.std() - 1.0) < 0.1 + True + """ + jrdd = sc._jvm.PythonMLLibAPI() \ + .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + normal = RDD(jrdd, sc, NoOpSerializer()) + return normal.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) + + @staticmethod + def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): + """ + Generates an RDD comprised of vectors containing i.i.d samples drawn + from the Poisson distribution with the input mean. + + >>> import numpy as np + >>> mean = 100.0 + >>> rdd = RandomRDDGenerators.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> mat = np.mat(rdd.collect()) + >>> mat.shape + (100, 100) + >>> abs(mat.mean() - mean) < 0.5 + True + >>> from math import sqrt + >>> abs(mat.std() - sqrt(mean)) < 0.5 + True + """ + jrdd = sc._jvm.PythonMLLibAPI() \ + .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) + poisson = RDD(jrdd, sc, NoOpSerializer()) + return poisson.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) + + +def _test(): + import doctest + from pyspark.context import SparkContext + globs = globals().copy() + # The small batch size here ensures that we see multiple batches, + # even in these small test examples: + globs['sc'] = SparkContext('local[2]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/run-tests b/python/run-tests index 29f755fc0dcd3..5049e15ce5f8a 100755 --- a/python/run-tests +++ b/python/run-tests @@ -67,6 +67,7 @@ run_test "pyspark/mllib/_common.py" run_test "pyspark/mllib/classification.py" run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/linalg.py" +run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" run_test "pyspark/mllib/tests.py" From b124de584a45b7ebde9fbe10128db429c56aeaee Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 31 Jul 2014 20:51:48 -0700 Subject: [PATCH 0167/1492] [SPARK-2756] [mllib] Decision tree bug fixes MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit (1) Inconsistent aggregate (agg) indexing for unordered features. (2) Fixed gain calculations for edge cases. (3) One-off error in choosing thresholds for continuous features for small datasets. (4) (not a bug) Changed meaning of tree depth by 1 to fit scikit-learn and rpart. (Depth 1 used to mean 1 leaf node; depth 0 now means 1 leaf node.) Other updates, to help with tests: * Updated DecisionTreeRunner to print more info. * Added utility functions to DecisionTreeModel: toString, depth, numNodes * Improved internal DecisionTree documentation Bug fix details: (1) Indexing was inconsistent for aggregate calculations for unordered features (in multiclass classification with categorical features, where the features had few enough values such that they could be considered unordered, i.e., isSpaceSufficientForAllCategoricalSplits=true). * updateBinForUnorderedFeature indexed agg as (node, feature, featureValue, binIndex), where ** featureValue was from arr (so it was a feature value) ** binIndex was in [0,…, 2^(maxFeatureValue-1)-1) * The rest of the code indexed agg as (node, feature, binIndex, label). * Corrected this bug by changing updateBinForUnorderedFeature to use the second indexing pattern. Unit tests in DecisionTreeSuite * Updated a few tests to train a model and test its training accuracy, which catches the indexing bug from updateBinForUnorderedFeature() discussed above. * Added new test (“stump with categorical variables for multiclass classification, with just enough bins”) to test bin extremes. (2) Bug fix: calculateGainForSplit (for classification): * It used to return dummy prediction values when either the right or left children had 0 weight. These were incorrect for multiclass classification. It has been corrected. Updated impurities to allow for count = 0. This was related to the above bug fix for calculateGainForSplit (for classification). Small updates to documentation and coding style. (3) Bug fix: Off-by-1 when finding thresholds for splits for continuous features. * Exhibited bug in new test in DecisionTreeSuite: “stump with 1 continuous variable for binary classification, to check off-by-1 error” * Description: When finding thresholds for possible splits for continuous features in DecisionTree.findSplitsBins, the thresholds were set according to individual training examples’ feature values. * Fix: The threshold is set to be the average of 2 consecutive (sorted) examples’ feature values. E.g.: If the old code set the threshold using example i, the new code sets the threshold using exam * Note: In 4 DecisionTreeSuite tests with all labels identical, removed check of threshold since it is somewhat arbitrary. CC: mengxr manishamde Please let me know if I missed something! Author: Joseph K. Bradley Closes #1673 from jkbradley/decisiontree-bugfix and squashes the following commits: 2b20c61 [Joseph K. Bradley] Small doc and style updates dab0b67 [Joseph K. Bradley] Added documentation for DecisionTree internals 8bb8aa0 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix 978cfcf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix 6eed482 [Joseph K. Bradley] In DecisionTree: Changed from using procedural syntax for functions returning Unit to explicitly writing Unit return type. 376dca2 [Joseph K. Bradley] Updated meaning of maxDepth by 1 to fit scikit-learn and rpart. * In code, replaced usages of maxDepth <-- maxDepth + 1 * In params, replace settings of maxDepth <-- maxDepth - 1 59750f8 [Joseph K. Bradley] * Updated Strategy to check numClassesForClassification only if algo=Classification. * Updates based on comments: ** DecisionTreeRunner *** Made dataFormat arg default to libsvm ** Small cleanups ** tree.Node: Made recursive helper methods private, and renamed them. 52e17c5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix da50db7 [Joseph K. Bradley] Added one more test to DecisionTreeSuite: stump with 2 continuous variables for binary classification. Caused problems in past, but fixed now. 8ea8750 [Joseph K. Bradley] Bug fix: Off-by-1 when finding thresholds for splits for continuous features. 2283df8 [Joseph K. Bradley] 2 bug fixes. 73fbea2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix 5f920a1 [Joseph K. Bradley] Demonstration of bug before submitting fix: Updated DecisionTreeSuite so that 3 tests fail. Will describe bug in next commit. --- .../examples/mllib/DecisionTreeRunner.scala | 92 +++- .../spark/mllib/tree/DecisionTree.scala | 408 +++++++++++------- .../mllib/tree/configuration/Strategy.scala | 7 +- .../spark/mllib/tree/impurity/Entropy.scala | 6 +- .../spark/mllib/tree/impurity/Gini.scala | 6 +- .../spark/mllib/tree/impurity/Impurity.scala | 4 +- .../spark/mllib/tree/impurity/Variance.scala | 6 +- .../mllib/tree/model/DecisionTreeModel.scala | 31 +- .../apache/spark/mllib/tree/model/Node.scala | 56 +++ .../spark/mllib/tree/DecisionTreeSuite.scala | 115 ++++- 10 files changed, 538 insertions(+), 193 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 6db9bf3cf5be6..cf3d2cca81ff6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -21,7 +21,6 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.{DecisionTree, impurity} import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} @@ -36,6 +35,9 @@ import org.apache.spark.rdd.RDD * ./bin/run-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] * }}} * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + * + * Note: This script treats all features as real-valued (not categorical). + * To include categorical features, modify categoricalFeaturesInfo. */ object DecisionTreeRunner { @@ -48,11 +50,12 @@ object DecisionTreeRunner { case class Params( input: String = null, + dataFormat: String = "libsvm", algo: Algo = Classification, - numClassesForClassification: Int = 2, - maxDepth: Int = 5, + maxDepth: Int = 4, impurity: ImpurityType = Gini, - maxBins: Int = 100) + maxBins: Int = 100, + fracTest: Double = 0.2) def main(args: Array[String]) { val defaultParams = Params() @@ -69,25 +72,31 @@ object DecisionTreeRunner { opt[Int]("maxDepth") .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") .action((x, c) => c.copy(maxDepth = x)) - opt[Int]("numClassesForClassification") - .text(s"number of classes for classification, " - + s"default: ${defaultParams.numClassesForClassification}") - .action((x, c) => c.copy(numClassesForClassification = x)) opt[Int]("maxBins") .text(s"max number of bins, default: ${defaultParams.maxBins}") .action((x, c) => c.copy(maxBins = x)) + opt[Double]("fracTest") + .text(s"fraction of data to hold out for testing, default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) arg[String]("") .text("input paths to labeled examples in dense format (label,f0 f1 f2 ...)") .required() .action((x, c) => c.copy(input = x)) checkConfig { params => - if (params.algo == Classification && - (params.impurity == Gini || params.impurity == Entropy)) { - success - } else if (params.algo == Regression && params.impurity == Variance) { - success + if (params.fracTest < 0 || params.fracTest > 1) { + failure(s"fracTest ${params.fracTest} value incorrect; should be in [0,1].") } else { - failure(s"Algo ${params.algo} is not compatible with impurity ${params.impurity}.") + if (params.algo == Classification && + (params.impurity == Gini || params.impurity == Entropy)) { + success + } else if (params.algo == Regression && params.impurity == Variance) { + success + } else { + failure(s"Algo ${params.algo} is not compatible with impurity ${params.impurity}.") + } } } } @@ -100,16 +109,57 @@ object DecisionTreeRunner { } def run(params: Params) { + val conf = new SparkConf().setAppName("DecisionTreeRunner") val sc = new SparkContext(conf) // Load training data and cache it. - val examples = MLUtils.loadLabeledPoints(sc, params.input).cache() + val origExamples = params.dataFormat match { + case "dense" => MLUtils.loadLabeledPoints(sc, params.input).cache() + case "libsvm" => MLUtils.loadLibSVMFile(sc, params.input).cache() + } + // For classification, re-index classes if needed. + val (examples, numClasses) = params.algo match { + case Classification => { + // classCounts: class --> # examples in class + val classCounts = origExamples.map(_.label).countByValue() + val sortedClasses = classCounts.keys.toList.sorted + val numClasses = classCounts.size + // classIndexMap: class --> index in 0,...,numClasses-1 + val classIndexMap = { + if (classCounts.keySet != Set(0.0, 1.0)) { + sortedClasses.zipWithIndex.toMap + } else { + Map[Double, Int]() + } + } + val examples = { + if (classIndexMap.isEmpty) { + origExamples + } else { + origExamples.map(lp => LabeledPoint(classIndexMap(lp.label), lp.features)) + } + } + val numExamples = examples.count() + println(s"numClasses = $numClasses.") + println(s"Per-class example fractions, counts:") + println(s"Class\tFrac\tCount") + sortedClasses.foreach { c => + val frac = classCounts(c) / numExamples.toDouble + println(s"$c\t$frac\t${classCounts(c)}") + } + (examples, numClasses) + } + case Regression => + (origExamples, 0) + case _ => + throw new IllegalArgumentException("Algo ${params.algo} not supported.") + } - val splits = examples.randomSplit(Array(0.8, 0.2)) + // Split into training, test. + val splits = examples.randomSplit(Array(1.0 - params.fracTest, params.fracTest)) val training = splits(0).cache() val test = splits(1).cache() - val numTraining = training.count() val numTest = test.count() @@ -129,17 +179,19 @@ object DecisionTreeRunner { impurity = impurityCalculator, maxDepth = params.maxDepth, maxBins = params.maxBins, - numClassesForClassification = params.numClassesForClassification) + numClassesForClassification = numClasses) val model = DecisionTree.train(training, strategy) + println(model) + if (params.algo == Classification) { val accuracy = accuracyScore(model, test) - println(s"Test accuracy = $accuracy.") + println(s"Test accuracy = $accuracy") } if (params.algo == Regression) { val mse = meanSquaredError(model, test) - println(s"Test mean squared error = $mse.") + println(s"Test mean squared error = $mse") } sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index ad32e3f4560fe..7d123dd6ae996 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -31,8 +31,8 @@ import org.apache.spark.util.random.XORShiftRandom /** * :: Experimental :: - * A class that implements a decision tree algorithm for classification and regression. It - * supports both continuous and categorical features. + * A class which implements a decision tree learning algorithm for classification and regression. + * It supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. @@ -42,8 +42,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo /** * Method to train a decision tree model over an RDD - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * @return a DecisionTreeModel that can be used for prediction + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @return DecisionTreeModel that can be used for prediction */ def train(input: RDD[LabeledPoint]): DecisionTreeModel = { @@ -60,7 +60,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // depth of the decision tree val maxDepth = strategy.maxDepth // the max number of nodes possible given the depth of the tree - val maxNumNodes = math.pow(2, maxDepth).toInt - 1 + val maxNumNodes = math.pow(2, maxDepth + 1).toInt - 1 // Initialize an array to hold filters applied to points for each node. val filters = new Array[List[Filter]](maxNumNodes) // The filter at the top node is an empty list. @@ -100,7 +100,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo var level = 0 var break = false - while (level < maxDepth && !break) { + while (level <= maxDepth && !break) { logDebug("#####################################") logDebug("level = " + level) @@ -152,7 +152,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val split = nodeSplitStats._1 val stats = nodeSplitStats._2 val nodeIndex = math.pow(2, level).toInt - 1 + index - val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth - 1) + val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) logDebug("Node = " + node) nodes(nodeIndex) = node @@ -173,7 +173,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo while (i <= 1) { // Calculate the index of the node from the node level and the index at the current level. val nodeIndex = math.pow(2, level + 1).toInt - 1 + 2 * index + i - if (level < maxDepth - 1) { + if (level < maxDepth) { val impurity = if (i == 0) { nodeSplitStats._2.leftImpurity } else { @@ -197,17 +197,16 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo object DecisionTree extends Serializable with Logging { /** - * Method to train a decision tree model where the instances are represented as an RDD of - * (label, features) pairs. The method supports binary classification and regression. For the - * binary classification, the label for each instance should either be 0 or 1 to denote the two - * classes. The parameters for the algorithm are specified using the strategy parameter. + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. * - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. - * @return a DecisionTreeModel that can be used for prediction + * @return DecisionTreeModel that can be used for prediction */ def train(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeModel = { new DecisionTree(strategy).train(input) @@ -219,12 +218,14 @@ object DecisionTree extends Serializable with Logging { * binary classification, the label for each instance should either be 0 or 1 to denote the two * classes. * - * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as - * training data + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. * @param algo algorithm, classification or regression * @param impurity impurity criterion used for information gain calculation - * @param maxDepth maxDepth maximum depth of the tree - * @return a DecisionTreeModel that can be used for prediction + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * @return DecisionTreeModel that can be used for prediction */ def train( input: RDD[LabeledPoint], @@ -241,13 +242,15 @@ object DecisionTree extends Serializable with Logging { * binary classification, the label for each instance should either be 0 or 1 to denote the two * classes. * - * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as - * training data + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. * @param algo algorithm, classification or regression * @param impurity impurity criterion used for information gain calculation - * @param maxDepth maxDepth maximum depth of the tree + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. * @param numClassesForClassification number of classes for classification. Default value of 2. - * @return a DecisionTreeModel that can be used for prediction + * @return DecisionTreeModel that can be used for prediction */ def train( input: RDD[LabeledPoint], @@ -266,11 +269,13 @@ object DecisionTree extends Serializable with Logging { * 1 to denote the two classes. The method also supports categorical features inputs where the * number of categories can specified using the categoricalFeaturesInfo option. * - * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as - * training data for DecisionTree + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. * @param algo classification or regression * @param impurity criterion used for information gain calculation - * @param maxDepth maximum depth of the tree + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. * @param numClassesForClassification number of classes for classification. Default value of 2. * @param maxBins maximum number of bins used for splitting features * @param quantileCalculationStrategy algorithm for calculating quantiles @@ -279,7 +284,7 @@ object DecisionTree extends Serializable with Logging { * an entry (n -> k) implies the feature n is categorical with k * categories 0, 1, 2, ... , k-1. It's important to note that * features are zero-indexed. - * @return a DecisionTreeModel that can be used for prediction + * @return DecisionTreeModel that can be used for prediction */ def train( input: RDD[LabeledPoint], @@ -301,11 +306,10 @@ object DecisionTree extends Serializable with Logging { * Returns an array of optimal splits for all nodes at a given level. Splits the task into * multiple groups if the level-wise training task could lead to memory overflow. * - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] * @param parentImpurities Impurities for all parent nodes for the current level * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree + * parameters for constructing the DecisionTree * @param level Level of the tree * @param filters Filters for all nodes at a given level * @param splits possible splits for all features @@ -348,11 +352,10 @@ object DecisionTree extends Serializable with Logging { /** * Returns an array of optimal splits for a group of nodes at a given level * - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] * @param parentImpurities Impurities for all parent nodes for the current level * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree + * parameters for constructing the DecisionTree * @param level Level of the tree * @param filters Filters for all nodes at a given level * @param splits possible splits for all features @@ -373,7 +376,7 @@ object DecisionTree extends Serializable with Logging { groupIndex: Int = 0): Array[(Split, InformationGainStats)] = { /* - * The high-level description for the best split optimizations are noted here. + * The high-level descriptions of the best split optimizations are noted here. * * *Level-wise training* * We perform bin calculations for all nodes at the given level to avoid making multiple @@ -396,18 +399,27 @@ object DecisionTree extends Serializable with Logging { * drastically reduce the communication overhead. */ - // common calculations for multiple nested methods + // Common calculations for multiple nested methods: + + // numNodes: Number of nodes in this (level of tree, group), + // where nodes at deeper (larger) levels may be divided into groups. val numNodes = math.pow(2, level).toInt / numGroups logDebug("numNodes = " + numNodes) + // Find the number of features by looking at the first sample. val numFeatures = input.first().features.size logDebug("numFeatures = " + numFeatures) + + // numBins: Number of bins = 1 + number of possible splits val numBins = bins(0).length logDebug("numBins = " + numBins) + val numClasses = strategy.numClassesForClassification logDebug("numClasses = " + numClasses) + val isMulticlassClassification = strategy.isMulticlassClassification logDebug("isMulticlassClassification = " + isMulticlassClassification) + val isMulticlassClassificationWithCategoricalFeatures = strategy.isMulticlassWithCategoricalFeatures logDebug("isMultiClassWithCategoricalFeatures = " + @@ -465,10 +477,13 @@ object DecisionTree extends Serializable with Logging { } /** - * Find bin for one feature. + * Find bin for one (labeledPoint, feature). */ - def findBin(featureIndex: Int, labeledPoint: LabeledPoint, - isFeatureContinuous: Boolean, isSpaceSufficientForAllCategoricalSplits: Boolean): Int = { + def findBin( + featureIndex: Int, + labeledPoint: LabeledPoint, + isFeatureContinuous: Boolean, + isSpaceSufficientForAllCategoricalSplits: Boolean): Int = { val binForFeatures = bins(featureIndex) val feature = labeledPoint.features(featureIndex) @@ -535,7 +550,9 @@ object DecisionTree extends Serializable with Logging { } else { // Perform sequential search to find bin for categorical features. val binIndex = { - if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { + val isUnorderedFeature = + isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits + if (isUnorderedFeature) { sequentialBinSearchForUnorderedCategoricalFeatureInClassification() } else { sequentialBinSearchForOrderedCategoricalFeatureInClassification() @@ -555,6 +572,14 @@ object DecisionTree extends Serializable with Logging { * where b_ij is an integer between 0 and numBins - 1 for regressions and binary * classification and the categorical feature value in multiclass classification. * Invalid sample is denoted by noting bin for feature 1 as -1. + * + * For unordered features, the "bin index" returned is actually the feature value (category). + * + * @return Array of size 1 + numFeatures * numNodes, where + * arr(0) = label for labeledPoint, and + * arr(1 + numFeatures * nodeIndex + featureIndex) = + * bin index for this labeledPoint + * (or InvalidBinIndex if labeledPoint is not handled by this node) */ def findBinsForLevel(labeledPoint: LabeledPoint): Array[Double] = { // Calculate bin index and label per feature per node. @@ -598,9 +623,21 @@ object DecisionTree extends Serializable with Logging { // Find feature bins for all nodes at a level. val binMappedRDD = input.map(x => findBinsForLevel(x)) - def updateBinForOrderedFeature(arr: Array[Double], agg: Array[Double], nodeIndex: Int, - label: Double, featureIndex: Int) = { - + /** + * Increment aggregate in location for (node, feature, bin, label). + * + * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. + * Array of size 1 + (numFeatures * numNodes). + * @param agg Array storing aggregate calculation, of size: + * numClasses * numBins * numFeatures * numNodes. + * Indexed by (node, feature, bin, label) where label is the least significant bit. + */ + def updateBinForOrderedFeature( + arr: Array[Double], + agg: Array[Double], + nodeIndex: Int, + label: Double, + featureIndex: Int): Unit = { // Find the bin index for this feature. val arrShift = 1 + numFeatures * nodeIndex val arrIndex = arrShift + featureIndex @@ -612,44 +649,58 @@ object DecisionTree extends Serializable with Logging { agg(aggIndex + labelInt) = agg(aggIndex + labelInt) + 1 } - def updateBinForUnorderedFeature(nodeIndex: Int, featureIndex: Int, arr: Array[Double], - label: Double, agg: Array[Double], rightChildShift: Int) = { + /** + * Increment aggregate in location for (nodeIndex, featureIndex, [bins], label), + * where [bins] ranges over all bins. + * Updates left or right side of aggregate depending on split. + * + * @param arr arr(0) = label. + * arr(1 + featureIndex + nodeIndex * numFeatures) = feature value (category) + * @param agg Indexed by (left/right, node, feature, bin, label) + * where label is the least significant bit. + * The left/right specifier is a 0/1 index indicating left/right child info. + * @param rightChildShift Offset for right side of agg. + */ + def updateBinForUnorderedFeature( + nodeIndex: Int, + featureIndex: Int, + arr: Array[Double], + label: Double, + agg: Array[Double], + rightChildShift: Int): Unit = { // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex + val arrIndex = 1 + numFeatures * nodeIndex + featureIndex + val featureValue = arr(arrIndex).toInt // Update the left or right count for one bin. - val aggShift = numClasses * numBins * numFeatures * nodeIndex - val aggIndex - = aggShift + numClasses * featureIndex * numBins + arr(arrIndex).toInt * numClasses + val aggShift = + numClasses * numBins * numFeatures * nodeIndex + + numClasses * numBins * featureIndex + + label.toInt // Find all matching bins and increment their values val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) val numCategoricalBins = math.pow(2.0, featureCategories - 1).toInt - 1 var binIndex = 0 while (binIndex < numCategoricalBins) { - val labelInt = label.toInt - if (bins(featureIndex)(binIndex).highSplit.categories.contains(labelInt)) { - agg(aggIndex + binIndex) - = agg(aggIndex + binIndex) + 1 + val aggIndex = aggShift + binIndex * numClasses + if (bins(featureIndex)(binIndex).highSplit.categories.contains(featureValue)) { + agg(aggIndex) += 1 } else { - agg(rightChildShift + aggIndex + binIndex) - = agg(rightChildShift + aggIndex + binIndex) + 1 + agg(rightChildShift + aggIndex) += 1 } binIndex += 1 } } /** - * Performs a sequential aggregation over a partition for classification. For l nodes, - * k features, either the left count or the right count of one of the p bins is - * incremented based upon whether the feature is classified as 0 or 1. + * Helper for binSeqOp. * - * @param agg Array[Double] storing aggregate calculation of size - * numClasses * numSplits * numFeatures*numNodes for classification - * @param arr Array[Double] of size 1 + (numFeatures * numNodes) - * @return Array[Double] storing aggregate calculation of size - * 2 * numSplits * numFeatures * numNodes for classification + * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. + * Array of size 1 + (numFeatures * numNodes). + * @param agg Array storing aggregate calculation, of size: + * numClasses * numBins * numFeatures * numNodes. + * Indexed by (node, feature, bin, label) where label is the least significant bit. */ - def orderedClassificationBinSeqOp(arr: Array[Double], agg: Array[Double]) = { + def binaryOrNotCategoricalBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { // Iterate over all nodes. var nodeIndex = 0 while (nodeIndex < numNodes) { @@ -671,17 +722,21 @@ object DecisionTree extends Serializable with Logging { } /** - * Performs a sequential aggregation over a partition for classification. For l nodes, - * k features, either the left count or the right count of one of the p bins is - * incremented based upon whether the feature is classified as 0 or 1. + * Helper for binSeqOp. * - * @param agg Array[Double] storing aggregate calculation of size - * numClasses * numSplits * numFeatures*numNodes for classification - * @param arr Array[Double] of size 1 + (numFeatures * numNodes) - * @return Array[Double] storing aggregate calculation of size - * 2 * numClasses * numSplits * numFeatures * numNodes for classification + * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. + * Array of size 1 + (numFeatures * numNodes). + * For ordered features, + * arr(1 + featureIndex + nodeIndex * numFeatures) = bin index. + * For unordered features, + * arr(1 + featureIndex + nodeIndex * numFeatures) = feature value (category). + * @param agg Array storing aggregate calculation. + * For ordered features, this is of size: + * numClasses * numBins * numFeatures * numNodes. + * For unordered features, this is of size: + * 2 * numClasses * numBins * numFeatures * numNodes. */ - def unorderedClassificationBinSeqOp(arr: Array[Double], agg: Array[Double]) = { + def multiclassWithCategoricalBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { // Iterate over all nodes. var nodeIndex = 0 while (nodeIndex < numNodes) { @@ -717,16 +772,17 @@ object DecisionTree extends Serializable with Logging { } /** - * Performs a sequential aggregation over a partition for regression. For l nodes, k features, + * Performs a sequential aggregation over a partition for regression. + * For l nodes, k features, * the count, sum, sum of squares of one of the p bins is incremented. * - * @param agg Array[Double] storing aggregate calculation of size - * 3 * numSplits * numFeatures * numNodes for classification - * @param arr Array[Double] of size 1 + (numFeatures * numNodes) - * @return Array[Double] storing aggregate calculation of size - * 3 * numSplits * numFeatures * numNodes for regression + * @param agg Array storing aggregate calculation, updated by this function. + * Size: 3 * numBins * numFeatures * numNodes + * @param arr Bin mapping from findBinsForLevel. + * Array of size 1 + (numFeatures * numNodes). + * @return agg */ - def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]) = { + def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { // Iterate over all nodes. var nodeIndex = 0 while (nodeIndex < numNodes) { @@ -757,14 +813,30 @@ object DecisionTree extends Serializable with Logging { /** * Performs a sequential aggregation over a partition. + * For l nodes, k features, + * For classification: + * Either the left count or the right count of one of the bins is + * incremented based upon whether the feature is classified as 0 or 1. + * For regression: + * The count, sum, sum of squares of one of the bins is incremented. + * + * @param agg Array storing aggregate calculation, updated by this function. + * Size for classification: + * numClasses * numBins * numFeatures * numNodes for ordered features, or + * 2 * numClasses * numBins * numFeatures * numNodes for unordered features. + * Size for regression: + * 3 * numBins * numFeatures * numNodes. + * @param arr Bin mapping from findBinsForLevel. + * Array of size 1 + (numFeatures * numNodes). + * @return agg */ def binSeqOp(agg: Array[Double], arr: Array[Double]): Array[Double] = { strategy.algo match { case Classification => if(isMulticlassClassificationWithCategoricalFeatures) { - unorderedClassificationBinSeqOp(arr, agg) + multiclassWithCategoricalBinSeqOp(arr, agg) } else { - orderedClassificationBinSeqOp(arr, agg) + binaryOrNotCategoricalBinSeqOp(arr, agg) } case Regression => regressionBinSeqOp(arr, agg) } @@ -815,20 +887,10 @@ object DecisionTree extends Serializable with Logging { topImpurity: Double): InformationGainStats = { strategy.algo match { case Classification => - var classIndex = 0 - val leftCounts: Array[Double] = new Array[Double](numClasses) - val rightCounts: Array[Double] = new Array[Double](numClasses) - var leftTotalCount = 0.0 - var rightTotalCount = 0.0 - while (classIndex < numClasses) { - val leftClassCount = leftNodeAgg(featureIndex)(splitIndex)(classIndex) - val rightClassCount = rightNodeAgg(featureIndex)(splitIndex)(classIndex) - leftCounts(classIndex) = leftClassCount - leftTotalCount += leftClassCount - rightCounts(classIndex) = rightClassCount - rightTotalCount += rightClassCount - classIndex += 1 - } + val leftCounts: Array[Double] = leftNodeAgg(featureIndex)(splitIndex) + val rightCounts: Array[Double] = rightNodeAgg(featureIndex)(splitIndex) + val leftTotalCount = leftCounts.sum + val rightTotalCount = rightCounts.sum val impurity = { if (level > 0) { @@ -845,33 +907,17 @@ object DecisionTree extends Serializable with Logging { } } - if (leftTotalCount == 0) { - return new InformationGainStats(0, topImpurity, topImpurity, Double.MinValue, 1) - } - if (rightTotalCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, 1) - } - - val leftImpurity = strategy.impurity.calculate(leftCounts, leftTotalCount) - val rightImpurity = strategy.impurity.calculate(rightCounts, rightTotalCount) - - val leftWeight = leftTotalCount / (leftTotalCount + rightTotalCount) - val rightWeight = rightTotalCount / (leftTotalCount + rightTotalCount) - - val gain = { - if (level > 0) { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } else { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } - } - val totalCount = leftTotalCount + rightTotalCount + if (totalCount == 0) { + // Return arbitrary prediction. + return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + } // Sum of count for each label - val leftRightCounts: Array[Double] - = leftCounts.zip(rightCounts) - .map{case (leftCount, rightCount) => leftCount + rightCount} + val leftRightCounts: Array[Double] = + leftCounts.zip(rightCounts).map { case (leftCount, rightCount) => + leftCount + rightCount + } def indexOfLargestArrayElement(array: Array[Double]): Int = { val result = array.foldLeft(-1, Double.MinValue, 0) { @@ -885,6 +931,22 @@ object DecisionTree extends Serializable with Logging { val predict = indexOfLargestArrayElement(leftRightCounts) val prob = leftRightCounts(predict) / totalCount + val leftImpurity = if (leftTotalCount == 0) { + topImpurity + } else { + strategy.impurity.calculate(leftCounts, leftTotalCount) + } + val rightImpurity = if (rightTotalCount == 0) { + topImpurity + } else { + strategy.impurity.calculate(rightCounts, rightTotalCount) + } + + val leftWeight = leftTotalCount / totalCount + val rightWeight = rightTotalCount / totalCount + + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) case Regression => val leftCount = leftNodeAgg(featureIndex)(splitIndex)(0) @@ -937,10 +999,18 @@ object DecisionTree extends Serializable with Logging { /** * Extracts left and right split aggregates. - * @param binData Array[Double] of size 2*numFeatures*numSplits - * @return (leftNodeAgg, rightNodeAgg) tuple of type (Array[Array[Array[Double\]\]\], - * Array[Array[Array[Double\]\]\]) where each array is of size(numFeature, - * (numBins - 1), numClasses) + * @param binData Aggregate array slice from getBinDataForNode. + * For classification: + * For unordered features, this is leftChildData ++ rightChildData, + * each of which is indexed by (feature, split/bin, class), + * with class being the least significant bit. + * For ordered features, this is of size numClasses * numBins * numFeatures. + * For regression: + * This is of size 2 * numFeatures * numBins. + * @return (leftNodeAgg, rightNodeAgg) pair of arrays. + * For classification, each array is of size (numFeatures, (numBins - 1), numClasses). + * For regression, each array is of size (numFeatures, (numBins - 1), 3). + * */ def extractLeftRightNodeAggregates( binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { @@ -983,6 +1053,11 @@ object DecisionTree extends Serializable with Logging { } } + /** + * Reshape binData for this feature. + * Indexes binData as (feature, split, class) with class as the least significant bit. + * @param leftNodeAgg leftNodeAgg(featureIndex)(splitIndex)(classIndex) = aggregate value + */ def findAggForUnorderedFeatureClassification( leftNodeAgg: Array[Array[Array[Double]]], rightNodeAgg: Array[Array[Array[Double]]], @@ -1107,7 +1182,7 @@ object DecisionTree extends Serializable with Logging { /** * Find the best split for a node. - * @param binData Array[Double] of size 2 * numSplits * numFeatures + * @param binData Bin data slice for this node, given by getBinDataForNode. * @param nodeImpurity impurity of the top node * @return tuple of split and information gain */ @@ -1133,7 +1208,7 @@ object DecisionTree extends Serializable with Logging { while (featureIndex < numFeatures) { // Iterate over all splits. var splitIndex = 0 - val maxSplitIndex : Double = { + val maxSplitIndex: Double = { val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty if (isFeatureContinuous) { numBins - 1 @@ -1162,8 +1237,8 @@ object DecisionTree extends Serializable with Logging { (bestFeatureIndex, bestSplitIndex, bestGainStats) } + logDebug("best split = " + splits(bestFeatureIndex)(bestSplitIndex)) logDebug("best split bin = " + bins(bestFeatureIndex)(bestSplitIndex)) - logDebug("best split bin = " + splits(bestFeatureIndex)(bestSplitIndex)) (splits(bestFeatureIndex)(bestSplitIndex), gainStats) } @@ -1214,8 +1289,17 @@ object DecisionTree extends Serializable with Logging { bestSplits } - private def getElementsPerNode(numFeatures: Int, numBins: Int, numClasses: Int, - isMulticlassClassificationWithCategoricalFeatures: Boolean, algo: Algo): Int = { + /** + * Get the number of values to be stored per node in the bin aggregates. + * + * @param numBins Number of bins = 1 + number of possible splits. + */ + private def getElementsPerNode( + numFeatures: Int, + numBins: Int, + numClasses: Int, + isMulticlassClassificationWithCategoricalFeatures: Boolean, + algo: Algo): Int = { algo match { case Classification => if (isMulticlassClassificationWithCategoricalFeatures) { @@ -1228,18 +1312,40 @@ object DecisionTree extends Serializable with Logging { } /** - * Returns split and bins for decision tree calculation. - * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data - * for DecisionTree + * Returns splits and bins for decision tree calculation. + * Continuous and categorical features are handled differently. + * + * Continuous features: + * For each feature, there are numBins - 1 possible splits representing the possible binary + * decisions at each node in the tree. + * + * Categorical features: + * For each feature, there is 1 bin per split. + * Splits and bins are handled in 2 ways: + * (a) For multiclass classification with a low-arity feature + * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), + * the feature is split based on subsets of categories. + * There are 2^(maxFeatureValue - 1) - 1 splits. + * (b) For regression and binary classification, + * and for multiclass classification with a high-arity feature, + * there is one split per category. + + * Categorical case (a) features are called unordered features. + * Other cases are called ordered features. + * + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree - * @return a tuple of (splits,bins) where splits is an Array of [org.apache.spark.mllib.tree - * .model.Split] of size (numFeatures, numSplits-1) and bins is an Array of [org.apache - * .spark.mllib.tree.model.Bin] of size (numFeatures, numSplits1) + * parameters for construction the DecisionTree + * @return A tuple of (splits,bins). + * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] + * of size (numFeatures, numBins - 1). + * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] + * of size (numFeatures, numBins). */ protected[tree] def findSplitsBins( input: RDD[LabeledPoint], strategy: Strategy): (Array[Array[Split]], Array[Array[Bin]]) = { + val count = input.count() // Find the number of features by looking at the first sample @@ -1271,7 +1377,8 @@ object DecisionTree extends Serializable with Logging { logDebug("fraction of data used for calculating quantiles = " + fraction) // sampled input for RDD calculation - val sampledInput = input.sample(false, fraction, new XORShiftRandom().nextInt()).collect() + val sampledInput = + input.sample(withReplacement = false, fraction, new XORShiftRandom().nextInt()).collect() val numSamples = sampledInput.length val stride: Double = numSamples.toDouble / numBins @@ -1294,8 +1401,10 @@ object DecisionTree extends Serializable with Logging { val stride: Double = numSamples.toDouble / numBins logDebug("stride = " + stride) for (index <- 0 until numBins - 1) { - val sampleIndex = (index + 1) * stride.toInt - val split = new Split(featureIndex, featureSamples(sampleIndex), Continuous, List()) + val sampleIndex = index * stride.toInt + // Set threshold halfway in between 2 samples. + val threshold = (featureSamples(sampleIndex) + featureSamples(sampleIndex + 1)) / 2.0 + val split = new Split(featureIndex, threshold, Continuous, List()) splits(featureIndex)(index) = split } } else { // Categorical feature @@ -1304,8 +1413,10 @@ object DecisionTree extends Serializable with Logging { = numBins > math.pow(2, featureCategories.toInt - 1) - 1 // Use different bin/split calculation strategy for categorical features in multiclass - // classification that satisfy the space constraint - if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { + // classification that satisfy the space constraint. + val isUnorderedFeature = + isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits + if (isUnorderedFeature) { // 2^(maxFeatureValue- 1) - 1 combinations var index = 0 while (index < math.pow(2.0, featureCategories - 1).toInt - 1) { @@ -1330,8 +1441,13 @@ object DecisionTree extends Serializable with Logging { } index += 1 } - } else { - + } else { // ordered feature + /* For a given categorical feature, use a subsample of the data + * to choose how to arrange possible splits. + * This examines each category and computes a centroid. + * These centroids are later used to sort the possible splits. + * centroidForCategories is a mapping: category (for the given feature) --> centroid + */ val centroidForCategories = { if (isMulticlassClassification) { // For categorical variables in multiclass classification, @@ -1341,7 +1457,7 @@ object DecisionTree extends Serializable with Logging { .groupBy(_._1) .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) .map(x => (x._1, x._2.values.toArray)) - .map(x => (x._1, strategy.impurity.calculate(x._2,x._2.sum))) + .map(x => (x._1, strategy.impurity.calculate(x._2, x._2.sum))) } else { // regression or binary classification // For categorical variables in regression and binary classification, // each bin is a category. The bins are sorted and they @@ -1352,7 +1468,7 @@ object DecisionTree extends Serializable with Logging { } } - logDebug("centriod for categories = " + centroidForCategories.mkString(",")) + logDebug("centroid for categories = " + centroidForCategories.mkString(",")) // Check for missing categorical variables and putting them last in the sorted list. val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() @@ -1367,7 +1483,7 @@ object DecisionTree extends Serializable with Logging { // bins sorted by centroids val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) - logDebug("centriod for categorical variable = " + categoriesSortedByCentroid) + logDebug("centroid for categorical variable = " + categoriesSortedByCentroid) var categoriesForSplit = List[Double]() categoriesSortedByCentroid.iterator.zipWithIndex.foreach { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 7c027ac2fda6b..5c65b537b6867 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -27,7 +27,8 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * Stores all the configuration options for tree construction * @param algo classification or regression * @param impurity criterion used for information gain calculation - * @param maxDepth maximum depth of the tree + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. * @param numClassesForClassification number of classes for classification. Default value is 2 * leads to binary classification * @param maxBins maximum number of bins used for splitting features @@ -52,7 +53,9 @@ class Strategy ( val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), val maxMemoryInMB: Int = 128) extends Serializable { - require(numClassesForClassification >= 2) + if (algo == Classification) { + require(numClassesForClassification >= 2) + } val isMulticlassClassification = numClassesForClassification > 2 val isMulticlassWithCategoricalFeatures = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index a0e2d91762782..9297c20596527 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -34,10 +34,13 @@ object Entropy extends Impurity { * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels - * @return information value + * @return information value, or 0 if totalCount = 0 */ @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = { + if (totalCount == 0) { + return 0 + } val numClasses = counts.length var impurity = 0.0 var classIndex = 0 @@ -58,6 +61,7 @@ object Entropy extends Impurity { * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels + * @return information value, or 0 if count = 0 */ @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 48144b5e6d1e4..2874bcf496484 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -33,10 +33,13 @@ object Gini extends Impurity { * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels - * @return information value + * @return information value, or 0 if totalCount = 0 */ @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = { + if (totalCount == 0) { + return 0 + } val numClasses = counts.length var impurity = 1.0 var classIndex = 0 @@ -54,6 +57,7 @@ object Gini extends Impurity { * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels + * @return information value, or 0 if count = 0 */ @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 7b2a9320cc21d..92b0c7b4a6fbc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -31,7 +31,7 @@ trait Impurity extends Serializable { * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels - * @return information value + * @return information value, or 0 if totalCount = 0 */ @DeveloperApi def calculate(counts: Array[Double], totalCount: Double): Double @@ -42,7 +42,7 @@ trait Impurity extends Serializable { * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels - * @return information value + * @return information value, or 0 if count = 0 */ @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 97149a99ead59..698a1a2a8e899 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -31,7 +31,7 @@ object Variance extends Impurity { * information calculation for multiclass classification * @param counts Array[Double] with counts for each label * @param totalCount sum of counts for all labels - * @return information value + * @return information value, or 0 if totalCount = 0 */ @DeveloperApi override def calculate(counts: Array[Double], totalCount: Double): Double = @@ -43,9 +43,13 @@ object Variance extends Impurity { * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels + * @return information value, or 0 if count = 0 */ @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { + if (count == 0) { + return 0 + } val squaredLoss = sumSquares - (sum * sum) / count squaredLoss / count } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index bf692ca8c4bd7..3d3406b5d5f22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -24,7 +24,8 @@ import org.apache.spark.mllib.linalg.Vector /** * :: Experimental :: - * Model to store the decision tree parameters + * Decision tree model for classification or regression. + * This model stores the decision tree structure and parameters. * @param topNode root node * @param algo algorithm type -- classification or regression */ @@ -50,4 +51,32 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable def predict(features: RDD[Vector]): RDD[Double] = { features.map(x => predict(x)) } + + /** + * Get number of nodes in tree, including leaf nodes. + */ + def numNodes: Int = { + 1 + topNode.numDescendants + } + + /** + * Get depth of tree. + * E.g.: Depth 0 means 1 leaf node. Depth 1 means 1 internal node and 2 leaf nodes. + */ + def depth: Int = { + topNode.subtreeDepth + } + + /** + * Print full model. + */ + override def toString: String = algo match { + case Classification => + s"DecisionTreeModel classifier\n" + topNode.subtreeToString(2) + case Regression => + s"DecisionTreeModel regressor\n" + topNode.subtreeToString(2) + case _ => throw new IllegalArgumentException( + s"DecisionTreeModel given unknown algo parameter: $algo.") + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 682f213f411a7..944f11c2c2e4f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -91,4 +91,60 @@ class Node ( } } } + + /** + * Get the number of nodes in tree below this node, including leaf nodes. + * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. + */ + private[tree] def numDescendants: Int = { + if (isLeaf) { + 0 + } else { + 2 + leftNode.get.numDescendants + rightNode.get.numDescendants + } + } + + /** + * Get depth of tree from this node. + * E.g.: Depth 0 means this is a leaf node. + */ + private[tree] def subtreeDepth: Int = { + if (isLeaf) { + 0 + } else { + 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) + } + } + + /** + * Recursive print function. + * @param indentFactor The number of spaces to add to each level of indentation. + */ + private[tree] def subtreeToString(indentFactor: Int = 0): String = { + + def splitToString(split: Split, left: Boolean): String = { + split.featureType match { + case Continuous => if (left) { + s"(feature ${split.feature} <= ${split.threshold})" + } else { + s"(feature ${split.feature} > ${split.threshold})" + } + case Categorical => if (left) { + s"(feature ${split.feature} in ${split.categories.mkString("{",",","}")})" + } else { + s"(feature ${split.feature} not in ${split.categories.mkString("{",",","}")})" + } + } + } + val prefix: String = " " * indentFactor + if (isLeaf) { + prefix + s"Predict: $predict\n" + } else { + prefix + s"If ${splitToString(split.get, left=true)}\n" + + leftNode.get.subtreeToString(indentFactor + 1) + + prefix + s"Else ${splitToString(split.get, left=false)}\n" + + rightNode.get.subtreeToString(indentFactor + 1) + } + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 5961a618c59d9..10462db700628 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -20,8 +20,7 @@ package org.apache.spark.mllib.tree import org.scalatest.FunSuite import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.Filter -import org.apache.spark.mllib.tree.model.Split +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Filter, Split} import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ @@ -31,6 +30,18 @@ import org.apache.spark.mllib.regression.LabeledPoint class DecisionTreeSuite extends FunSuite with LocalSparkContext { + def validateClassifier( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredAccuracy: Double) { + val predictions = input.map(x => model.predict(x.features)) + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + prediction != expected.label + } + val accuracy = (input.length - numOffPredictions).toDouble / input.length + assert(accuracy >= requiredAccuracy) + } + test("split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) @@ -50,7 +61,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, - maxDepth = 3, + maxDepth = 2, numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) @@ -130,7 +141,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, - maxDepth = 3, + maxDepth = 2, numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) @@ -236,7 +247,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("extract categories from a number for multiclass classification") { val l = DecisionTree.extractMultiClassCategories(13, 10) assert(l.length === 3) - assert(List(3.0, 2.0, 0.0).toSeq == l.toSeq) + assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) } test("split and bin calculations for unordered categorical variables with multiclass " + @@ -247,7 +258,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, - maxDepth = 3, + maxDepth = 2, numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) @@ -341,7 +352,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Classification, Gini, - maxDepth = 3, + maxDepth = 2, numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) @@ -397,7 +408,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { Classification, Gini, numClassesForClassification = 2, - maxDepth = 3, + maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) @@ -413,7 +424,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = bestSplits(0)._2 assert(stats.gain > 0) assert(stats.predict === 1) - assert(stats.prob == 0.6) + assert(stats.prob === 0.6) assert(stats.impurity > 0.2) } @@ -424,7 +435,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategy = new Strategy( Regression, Variance, - maxDepth = 3, + maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy) @@ -439,7 +450,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = bestSplits(0)._2 assert(stats.gain > 0) - assert(stats.predict == 0.6) + assert(stats.predict === 0.6) assert(stats.impurity > 0.2) } @@ -460,7 +471,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) @@ -483,7 +493,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) @@ -507,7 +516,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) @@ -531,7 +539,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._1.threshold === 10) assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) @@ -587,7 +594,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with categorical variables for multiclass classification") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val input = sc.parallelize(arr) - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) @@ -602,12 +609,78 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.featureType === Categorical) } + test("stump with 1 continuous variable for binary classification, to check off-by-1 error") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) + arr(2) = new LabeledPoint(1.0, Vectors.dense(2.0)) + arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 2) + + val model = DecisionTree.train(input, strategy) + validateClassifier(model, arr, 1.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + } + + test("stump with 2 continuous variables for binary classification") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) + arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 2) + + val model = DecisionTree.train(input, strategy) + validateClassifier(model, arr, 1.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + assert(model.topNode.split.get.feature === 1) + } + + test("stump with categorical variables for multiclass classification, with just enough bins") { + val maxBins = math.pow(2, 3 - 1).toInt // just enough bins to allow unordered features + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + assert(strategy.isMulticlassClassification) + + val model = DecisionTree.train(input, strategy) + validateClassifier(model, arr, 1.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + Array[List[Filter]](), splits, bins, 10) + + assert(bestSplits.length === 1) + val bestSplit = bestSplits(0)._1 + assert(bestSplit.feature === 0) + assert(bestSplit.categories.length === 1) + assert(bestSplit.categories.contains(1)) + assert(bestSplit.featureType === Categorical) + val gain = bestSplits(0)._2 + assert(gain.leftImpurity === 0) + assert(gain.rightImpurity === 0) + } + test("stump with continuous variables for multiclass classification") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val input = sc.parallelize(arr) - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3) assert(strategy.isMulticlassClassification) + + val model = DecisionTree.train(input, strategy) + validateClassifier(model, arr, 0.9) + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, Array[List[Filter]](), splits, bins, 10) @@ -625,9 +698,13 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with continuous + categorical variables for multiclass classification") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val input = sc.parallelize(arr) - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) + + val model = DecisionTree.train(input, strategy) + validateClassifier(model, arr, 0.9) + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, Array[List[Filter]](), splits, bins, 10) @@ -644,7 +721,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with categorical variables for ordered multiclass classification") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val input = sc.parallelize(arr) - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) From 9632719c9ef16ad95af4f3b85ae72d54b02b0f90 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 31 Jul 2014 21:02:11 -0700 Subject: [PATCH 0168/1492] [SPARK-2779] [SQL] asInstanceOf[Map[...]] should use scala.collection.Map instead of scala.collection.immutable.Map Since we let users create Rows. It makes sense to accept mutable Maps as values of MapType columns. JIRA: https://issues.apache.org/jira/browse/SPARK-2779 Author: Yin Huai Closes #1705 from yhuai/SPARK-2779 and squashes the following commits: 00d72fd [Yin Huai] Use scala.collection.Map. --- .../catalyst/expressions/complexTypes.scala | 2 ++ .../sql/catalyst/expressions/generators.scala | 2 ++ .../org/apache/spark/sql/json/JsonRDD.scala | 1 + .../org/apache/spark/sql/SQLQuerySuite.scala | 19 +++++++++++++++++++ 4 files changed, 24 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 72add5e20e8b4..c1154eb81c319 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.Map + import org.apache.spark.sql.catalyst.types._ /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 422839dab770d..3d41acb79e5fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.Map + import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.types._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index bd29ee421bbc4..70db1ebd3a3e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.json +import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index bebb490645420..5c571d35d1bb9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -505,5 +505,24 @@ class SQLQuerySuite extends QueryTest { (2, null) :: (3, null) :: (4, 2147483644) :: Nil) + + // The value of a MapType column can be a mutable map. + val rowRDD3 = unparsedStrings.map { r => + val values = r.split(",").map(_.trim) + val v4 = try values(3).toInt catch { + case _: NumberFormatException => null + } + Row(Row(values(0).toInt, values(2).toBoolean), scala.collection.mutable.Map(values(1) -> v4)) + } + + val schemaRDD3 = applySchema(rowRDD3, schema2) + schemaRDD3.registerAsTable("applySchema3") + + checkAnswer( + sql("SELECT f1.f11, f2['D4'] FROM applySchema3"), + (1, null) :: + (2, null) :: + (3, null) :: + (4, 2147483644) :: Nil) } } From 9998efab96a4fdc927818eaae53c04f946c4cf13 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Thu, 31 Jul 2014 21:06:57 -0700 Subject: [PATCH 0169/1492] SPARK-2766: ScalaReflectionSuite throw an llegalArgumentException in JDK 6 Author: GuoQiang Li Closes #1683 from witgo/SPARK-2766 and squashes the following commits: d0db00c [GuoQiang Li] ScalaReflectionSuite throw an llegalArgumentException in JDK 6 --- .../org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index e030d6e13d472..e75373d5a74a7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -182,7 +182,7 @@ class ScalaReflectionSuite extends FunSuite { assert(DecimalType === typeOfObject(BigDecimal("1.7976931348623157E318"))) // TimestampType - assert(TimestampType === typeOfObject(java.sql.Timestamp.valueOf("2014-7-25 10:26:00"))) + assert(TimestampType === typeOfObject(java.sql.Timestamp.valueOf("2014-07-25 10:26:00"))) // NullType assert(NullType === typeOfObject(null)) From b19008320bdf7064e764db04c43ef003a3ce0ecd Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 31 Jul 2014 21:14:08 -0700 Subject: [PATCH 0170/1492] [SPARK-2777][MLLIB] change ALS factors storage level to MEMORY_AND_DISK Now the factors are persisted in memory only. If they get kicked off by later jobs, we might have to start the computation from very beginning. A better solution is changing the storage level to `MEMORY_AND_DISK`. srowen Author: Xiangrui Meng Closes #1700 from mengxr/als-level and squashes the following commits: c103d76 [Xiangrui Meng] change ALS factors storage level to MEMORY_AND_DISK --- .../scala/org/apache/spark/mllib/recommendation/ALS.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index d208cfb917f3d..36d262fed425a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -290,8 +290,8 @@ class ALS private ( val usersOut = unblockFactors(users, userOutLinks) val productsOut = unblockFactors(products, productOutLinks) - usersOut.setName("usersOut").persist() - productsOut.setName("productsOut").persist() + usersOut.setName("usersOut").persist(StorageLevel.MEMORY_AND_DISK) + productsOut.setName("productsOut").persist(StorageLevel.MEMORY_AND_DISK) // Materialize usersOut and productsOut. usersOut.count() From c4755403e7d670176d81211813b6515dec76bee2 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Thu, 31 Jul 2014 21:23:35 -0700 Subject: [PATCH 0171/1492] [SPARK-2782][mllib] Bug fix for getRanks in SpearmanCorrelation getRanks computes the wrong rank when numPartition >= size in the input RDDs before this patch. added units to address this bug. Author: Doris Xin Closes #1710 from dorx/correlationBug and squashes the following commits: 733def4 [Doris Xin] bugs and reviewer comments. 31db920 [Doris Xin] revert unnecessary change 043ff83 [Doris Xin] bug fix for spearman corner case --- .../apache/spark/mllib/stat/Statistics.scala | 22 ++++++++++------ .../correlation/SpearmanCorrelation.scala | 18 ++++++------- .../spark/mllib/stat/CorrelationSuite.scala | 25 +++++++++++++++++++ 3 files changed, 47 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index 68f3867ba6c11..9d6de9b6e1f60 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -30,7 +30,7 @@ object Statistics { /** * Compute the Pearson correlation matrix for the input RDD of Vectors. - * Returns NaN if either vector has 0 variance. + * Columns with 0 covariance produce NaN entries in the correlation matrix. * * @param X an RDD[Vector] for which the correlation matrix is to be computed. * @return Pearson correlation matrix comparing columns in X. @@ -39,7 +39,7 @@ object Statistics { /** * Compute the correlation matrix for the input RDD of Vectors using the specified method. - * Methods currently supported: `pearson` (default), `spearman` + * Methods currently supported: `pearson` (default), `spearman`. * * Note that for Spearman, a rank correlation, we need to create an RDD[Double] for each column * and sort it in order to retrieve the ranks and then join the columns back into an RDD[Vector], @@ -55,20 +55,26 @@ object Statistics { /** * Compute the Pearson correlation for the input RDDs. - * Columns with 0 covariance produce NaN entries in the correlation matrix. + * Returns NaN if either vector has 0 variance. + * + * Note: the two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. * - * @param x RDD[Double] of the same cardinality as y - * @param y RDD[Double] of the same cardinality as x + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. * @return A Double containing the Pearson correlation between the two input RDD[Double]s */ def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) /** * Compute the correlation for the input RDDs using the specified method. - * Methods currently supported: pearson (default), spearman + * Methods currently supported: `pearson` (default), `spearman`. + * + * Note: the two input RDDs need to have the same number of partitions and the same number of + * elements in each partition. * - * @param x RDD[Double] of the same cardinality as y - * @param y RDD[Double] of the same cardinality as x + * @param x RDD[Double] of the same cardinality as y. + * @param y RDD[Double] of the same cardinality as x. * @param method String specifying the method to use for computing correlation. * Supported: `pearson` (default), `spearman` *@return A Double containing the correlation between the two input RDD[Double]s using the diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala index 1f7de630e778c..9bd0c2cd05de4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -89,20 +89,18 @@ private[stat] object SpearmanCorrelation extends Correlation with Logging { val ranks: RDD[(Long, Double)] = sorted.mapPartitions { iter => // add an extra element to signify the end of the list so that flatMap can flush the last // batch of duplicates - val padded = iter ++ - Iterator[((Double, Long), Long)](((Double.NaN, -1L), -1L)) - var lastVal = 0.0 - var firstRank = 0.0 - val idBuffer = new ArrayBuffer[Long]() + val end = -1L + val padded = iter ++ Iterator[((Double, Long), Long)](((Double.NaN, end), end)) + val firstEntry = padded.next() + var lastVal = firstEntry._1._1 + var firstRank = firstEntry._2.toDouble + val idBuffer = ArrayBuffer(firstEntry._1._2) padded.flatMap { case ((v, id), rank) => - if (v == lastVal && id != Long.MinValue) { + if (v == lastVal && id != end) { idBuffer += id Iterator.empty } else { - val entries = if (idBuffer.size == 0) { - // edge case for the first value matching the initial value of lastVal - Iterator.empty - } else if (idBuffer.size == 1) { + val entries = if (idBuffer.size == 1) { Iterator((idBuffer(0), firstRank)) } else { val averageRank = firstRank + (idBuffer.size - 1.0) / 2.0 diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala index bce4251426df7..a3f76f77a5dcc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -31,6 +31,7 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { // test input data val xData = Array(1.0, 0.0, -2.0) val yData = Array(4.0, 5.0, 3.0) + val zeros = new Array[Double](3) val data = Seq( Vectors.dense(1.0, 0.0, 0.0, -2.0), Vectors.dense(4.0, 5.0, 0.0, 3.0), @@ -46,6 +47,18 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { val p1 = Statistics.corr(x, y, "pearson") assert(approxEqual(expected, default)) assert(approxEqual(expected, p1)) + + // numPartitions >= size for input RDDs + for (numParts <- List(xData.size, xData.size * 2)) { + val x1 = sc.parallelize(xData, numParts) + val y1 = sc.parallelize(yData, numParts) + val p2 = Statistics.corr(x1, y1) + assert(approxEqual(expected, p2)) + } + + // RDD of zero variance + val z = sc.parallelize(zeros) + assert(Statistics.corr(x, z).isNaN()) } test("corr(x, y) spearman") { @@ -54,6 +67,18 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { val expected = 0.5 val s1 = Statistics.corr(x, y, "spearman") assert(approxEqual(expected, s1)) + + // numPartitions >= size for input RDDs + for (numParts <- List(xData.size, xData.size * 2)) { + val x1 = sc.parallelize(xData, numParts) + val y1 = sc.parallelize(yData, numParts) + val s2 = Statistics.corr(x1, y1, "spearman") + assert(approxEqual(expected, s2)) + } + + // RDD of zero variance => zero variance in ranks + val z = sc.parallelize(zeros) + assert(Statistics.corr(x, z, "spearman").isNaN()) } test("corr(X) default, pearson") { From 2cdc3e5c6f5601086590a0cebf40a48f7560d02e Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Thu, 31 Jul 2014 22:53:42 -0700 Subject: [PATCH 0172/1492] [SPARK-2702][Core] Upgrade Tachyon dependency to 0.5.0 Author: Haoyuan Li Closes #1651 from haoyuan/upgrade-tachyon and squashes the following commits: 6f3f98f [Haoyuan Li] upgrade tachyon to 0.5.0 --- core/pom.xml | 4 ++-- make-distribution.sh | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 04d4b9cc1068e..7c60cf10c3dc2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -192,8 +192,8 @@ org.tachyonproject - tachyon - 0.4.1-thrift + tachyon-client + 0.5.0 org.apache.hadoop diff --git a/make-distribution.sh b/make-distribution.sh index 0a3283ecec6f8..1441497b3995a 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -128,7 +128,7 @@ if [[ ! "$JAVA_VERSION" =~ "1.6" && -z "$SKIP_JAVA_TEST" ]]; then if [[ ! $REPLY =~ ^[Yy]$ ]]; then echo "Okay, exiting." exit 1 - fi + fi fi if [ "$NAME" == "none" ]; then @@ -173,7 +173,7 @@ cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy example sources (needed for python and SQL) mkdir -p "$DISTDIR/examples/src/main" -cp -r $FWDIR/examples/src/main "$DISTDIR/examples/src/" +cp -r $FWDIR/examples/src/main "$DISTDIR/examples/src/" if [ "$SPARK_HIVE" == "true" ]; then cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" @@ -199,7 +199,7 @@ cp -r "$FWDIR/ec2" "$DISTDIR" # Download and copy in tachyon, if requested if [ "$SPARK_TACHYON" == "true" ]; then - TACHYON_VERSION="0.4.1" + TACHYON_VERSION="0.5.0" TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/tachyon-${TACHYON_VERSION}-bin.tar.gz" TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` From 149910111331133d52e0cb01b256f7f731b436ad Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 31 Jul 2014 22:57:13 -0700 Subject: [PATCH 0173/1492] SPARK-2632, SPARK-2576. Fixed by only importing what is necessary during class definition. Without this patch, it imports everything available in the scope. ```scala scala> val a = 10l val a = 10l a: Long = 10 scala> import a._ import a._ import a._ scala> case class A(a: Int) // show case class A(a: Int) // show class $read extends Serializable { def () = { super.; () }; class $iwC extends Serializable { def () = { super.; () }; class $iwC extends Serializable { def () = { super.; () }; import org.apache.spark.SparkContext._; class $iwC extends Serializable { def () = { super.; () }; val $VAL5 = $line5.$read.INSTANCE; import $VAL5.$iw.$iw.$iw.$iw.a; class $iwC extends Serializable { def () = { super.; () }; import a._; class $iwC extends Serializable { def () = { super.; () }; class $iwC extends Serializable { def () = { super.; () }; case class A extends scala.Product with scala.Serializable { val a: Int = _; def (a: Int) = { super.; () } } }; val $iw = new $iwC. }; val $iw = new $iwC. }; val $iw = new $iwC. }; val $iw = new $iwC. }; val $iw = new $iwC. }; val $iw = new $iwC. } object $read extends scala.AnyRef { def () = { super.; () }; val INSTANCE = new $read. } defined class A ``` With this patch, it just imports only the necessary. ```scala scala> val a = 10l val a = 10l a: Long = 10 scala> import a._ import a._ import a._ scala> case class A(a: Int) // show case class A(a: Int) // show class $read extends Serializable { def () = { super.; () }; class $iwC extends Serializable { def () = { super.; () }; class $iwC extends Serializable { def () = { super.; () }; case class A extends scala.Product with scala.Serializable { val a: Int = _; def (a: Int) = { super.; () } } }; val $iw = new $iwC. }; val $iw = new $iwC. } object $read extends scala.AnyRef { def () = { super.; () }; val INSTANCE = new $read. } defined class A scala> ``` This patch also adds a `:fallback` mode on being enabled it will restore the spark-shell's 1.0.0 behaviour. Author: Prashant Sharma Author: Yin Huai Author: Prashant Sharma Closes #1635 from ScrapCodes/repl-fix-necessary-imports and squashes the following commits: b1968d2 [Prashant Sharma] Added toschemaRDD to test case. 0b712bb [Yin Huai] Add a REPL test to test importing a method. 02ad8ff [Yin Huai] Add a REPL test for importing SQLContext.createSchemaRDD. ed6d0c7 [Prashant Sharma] Added a fallback mode, incase users run into issues while using repl. b63d3b2 [Prashant Sharma] SPARK-2632, SPARK-2576. Fixed by only importing what is necessary during class definition. --- repl/pom.xml | 6 +++++ .../org/apache/spark/repl/SparkILoop.scala | 17 ++++++++++++ .../org/apache/spark/repl/SparkIMain.scala | 7 ++++- .../org/apache/spark/repl/SparkImports.scala | 15 ++++++++--- .../org/apache/spark/repl/ReplSuite.scala | 27 +++++++++++++++++++ 5 files changed, 67 insertions(+), 5 deletions(-) diff --git a/repl/pom.xml b/repl/pom.xml index 4ebb1b82f0e8c..68f4504450778 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -55,6 +55,12 @@ ${project.version} runtime + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test + org.eclipse.jetty jetty-server diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 6f9fa0d9f2b25..42c7e511dc3f5 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -230,6 +230,20 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, case xs => xs find (_.name == cmd) } } + private var fallbackMode = false + + private def toggleFallbackMode() { + val old = fallbackMode + fallbackMode = !old + System.setProperty("spark.repl.fallback", fallbackMode.toString) + echo(s""" + |Switched ${if (old) "off" else "on"} fallback mode without restarting. + | If you have defined classes in the repl, it would + |be good to redefine them incase you plan to use them. If you still run + |into issues it would be good to restart the repl and turn on `:fallback` + |mode as first command. + """.stripMargin) + } /** Show the history */ lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { @@ -299,6 +313,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), shCommand, nullary("silent", "disable/enable automatic printing of results", verbosity), + nullary("fallback", """ + |disable/enable advanced repl changes, these fix some issues but may introduce others. + |This mode will be removed once these fixes stablize""".stripMargin, toggleFallbackMode), cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) ) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 3842c291d0b7b..f60bbb4662af1 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -892,11 +892,16 @@ import org.apache.spark.util.Utils def definedTypeSymbol(name: String) = definedSymbols(newTypeName(name)) def definedTermSymbol(name: String) = definedSymbols(newTermName(name)) + val definedClasses = handlers.exists { + case _: ClassHandler => true + case _ => false + } + /** Code to import bound names from previous lines - accessPath is code to * append to objectName to access anything bound by request. */ val SparkComputedImports(importsPreamble, importsTrailer, accessPath) = - importsCode(referencedNames.toSet) + importsCode(referencedNames.toSet, definedClasses) /** Code to access a variable with the specified name */ def fullPath(vname: String) = { diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index 9099e052f5796..193a42dcded12 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -108,8 +108,9 @@ trait SparkImports { * last one imported is actually usable. */ case class SparkComputedImports(prepend: String, append: String, access: String) + def fallback = System.getProperty("spark.repl.fallback", "false").toBoolean - protected def importsCode(wanted: Set[Name]): SparkComputedImports = { + protected def importsCode(wanted: Set[Name], definedClass: Boolean): SparkComputedImports = { /** Narrow down the list of requests from which imports * should be taken. Removes requests which cannot contribute * useful imports for the specified set of wanted names. @@ -124,8 +125,14 @@ trait SparkImports { // Single symbol imports might be implicits! See bug #1752. Rather than // try to finesse this, we will mimic all imports for now. def keepHandler(handler: MemberHandler) = handler match { - case _: ImportHandler => true - case x => x.definesImplicit || (x.definedNames exists wanted) + /* This case clause tries to "precisely" import only what is required. And in this + * it may miss out on some implicits, because implicits are not known in `wanted`. Thus + * it is suitable for defining classes. AFAIK while defining classes implicits are not + * needed.*/ + case h: ImportHandler if definedClass && !fallback => + h.importedNames.exists(x => wanted.contains(x)) + case _: ImportHandler => true + case x => x.definesImplicit || (x.definedNames exists wanted) } reqs match { @@ -182,7 +189,7 @@ trait SparkImports { // ambiguity errors will not be generated. Also, quote // the name of the variable, so that we don't need to // handle quoting keywords separately. - case x: ClassHandler => + case x: ClassHandler if !fallback => // I am trying to guess if the import is a defined class // This is an ugly hack, I am not 100% sure of the consequences. // Here we, let everything but "defined classes" use the import with val. diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index e2d8d5ff38dbe..c8763eb277052 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -256,6 +256,33 @@ class ReplSuite extends FunSuite { assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) } + + test("SPARK-2576 importing SQLContext.createSchemaRDD.") { + // We need to use local-cluster to test this case. + val output = runInterpreter("local-cluster[1,1,512]", + """ + |val sqlContext = new org.apache.spark.sql.SQLContext(sc) + |import sqlContext.createSchemaRDD + |case class TestCaseClass(value: Int) + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).toSchemaRDD.collect + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } + + test("SPARK-2632 importing a method from non serializable class and not using it.") { + val output = runInterpreter("local", + """ + |class TestClass() { def testMethod = 3 } + |val t = new TestClass + |import t.testMethod + |case class TestCaseClass(value: Int) + |sc.parallelize(1 to 10).map(x => TestCaseClass(x)).collect + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } + if (System.getenv("MESOS_NATIVE_LIBRARY") != null) { test("running on Mesos") { val output = runInterpreter("localquiet", From cb9e7d5aff2ce9cb501a2825651224311263ce20 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 31 Jul 2014 23:12:38 -0700 Subject: [PATCH 0174/1492] SPARK-2738. Remove redundant imports in BlockManagerSuite Author: Sandy Ryza Closes #1642 from sryza/sandy-spark-2738 and squashes the following commits: a923e4e [Sandy Ryza] SPARK-2738. Remove redundant imports in BlockManagerSuite --- .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index dd4fd535d3577..58ea0cc30e954 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,9 +21,6 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import akka.actor._ -import org.apache.spark.SparkConf -import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} -import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ From 8ff4417f70198ba2d848157f9da4e1e7e18f4fca Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 1 Aug 2014 00:01:30 -0700 Subject: [PATCH 0175/1492] [SPARK-2670] FetchFailedException should be thrown when local fetch has failed Author: Kousuke Saruta Closes #1578 from sarutak/SPARK-2670 and squashes the following commits: 85c8938 [Kousuke Saruta] Removed useless results.put for fail fast e8713cc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670 d353984 [Kousuke Saruta] Refined assertion messages in BlockFetcherIteratorSuite.scala 03bcb02 [Kousuke Saruta] Merge branch 'SPARK-2670' of github.com:sarutak/spark into SPARK-2670 5d05855 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670 4fca130 [Kousuke Saruta] Added test cases for BasicBlockFetcherIterator b7b8250 [Kousuke Saruta] Modified BasicBlockFetchIterator to fail fast when local fetch error has been occurred a3a9be1 [Kousuke Saruta] Modified BlockFetcherIterator for SPARK-2670 460dc01 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2670 e310c0b [Kousuke Saruta] Modified BlockFetcherIterator to handle local fetch failure as fatch fail --- .../spark/storage/BlockFetcherIterator.scala | 19 ++- .../storage/BlockFetcherIteratorSuite.scala | 140 ++++++++++++++++++ 2 files changed, 151 insertions(+), 8 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 69905a960a2ca..ccf830e118ee7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -200,14 +200,17 @@ object BlockFetcherIterator { // these all at once because they will just memory-map some files, so they won't consume // any memory that might exceed our maxBytesInFlight for (id <- localBlocksToFetch) { - getLocalFromDisk(id, serializer) match { - case Some(iter) => { - // Pass 0 as size since it's not in flight - results.put(new FetchResult(id, 0, () => iter)) - logDebug("Got local block " + id) - } - case None => { - throw new BlockException(id, "Could not get block " + id + " from local machine") + try { + // getLocalFromDisk never return None but throws BlockException + val iter = getLocalFromDisk(id, serializer).get + // Pass 0 as size since it's not in flight + results.put(new FetchResult(id, 0, () => iter)) + logDebug("Got local block " + id) + } catch { + case e: Exception => { + logError(s"Error occurred while fetching local blocks", e) + results.put(new FetchResult(id, -1, null)) + return } } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala new file mode 100644 index 0000000000000..8dca2ebb312f5 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.scalatest.{FunSuite, Matchers} +import org.scalatest.PrivateMethodTester._ + +import org.mockito.Mockito._ +import org.mockito.Matchers.{any, eq => meq} +import org.mockito.stubbing.Answer +import org.mockito.invocation.InvocationOnMock + +import org.apache.spark._ +import org.apache.spark.storage.BlockFetcherIterator._ +import org.apache.spark.network.{ConnectionManager, ConnectionManagerId, + Message} + +class BlockFetcherIteratorSuite extends FunSuite with Matchers { + + test("block fetch from local fails using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + doReturn(connManager).when(blockManager).connectionManager + doReturn(BlockManagerId("test-client", "test-client", 1, 0)).when(blockManager).blockManagerId + + doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + val answer = new Answer[Option[Iterator[Any]]] { + override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { + throw new Exception + } + } + + // 3rd block is going to fail + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(1)), any()) + doAnswer(answer).when(blockManager).getLocalFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, + blocksByAddress, null) + + iterator.initialize() + + // 3rd getLocalFromDisk invocation should be failed + verify(blockManager, times(3)).getLocalFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + // the 2nd element of the tuple returned by iterator.next should be defined when fetching successfully + assert(iterator.next._2.isDefined, "1st element should be defined but is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next._2.isDefined, "2nd element should be defined but is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + // 3rd fetch should be failed + assert(!iterator.next._2.isDefined, "3rd element should not be defined but is actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") + // Don't call next() after fetching non-defined element even if thare are rest of elements in the iterator. + // Otherwise, BasicBlockFetcherIterator hangs up. + } + + + test("block fetch from local succeed using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + doReturn(connManager).when(blockManager).connectionManager + doReturn(BlockManagerId("test-client", "test-client", 1, 0)).when(blockManager).blockManagerId + + doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + + // All blocks should be fetched successfully + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(1)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, + blocksByAddress, null) + + iterator.initialize() + + // getLocalFromDis should be invoked for all of 5 blocks + verify(blockManager, times(5)).getLocalFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 1st element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next._2.isDefined, "All elements should be defined but 2nd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 3rd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 4th element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") + assert(iterator.next._2.isDefined, "All elements should be defined but 5th element is not actually defined") + } + +} From 72e33699732496fa71e8c8b0de2203b908423fb2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 1 Aug 2014 00:16:18 -0700 Subject: [PATCH 0176/1492] SPARK-983. Support external sorting in sortByKey() This patch simply uses the ExternalSorter class from sort-based shuffle. Closes #931 and Closes #1090 Author: Matei Zaharia Closes #1677 from mateiz/spark-983 and squashes the following commits: 96b3fda [Matei Zaharia] SPARK-983. Support external sorting in sortByKey() --- .../shuffle/hash/HashShuffleReader.scala | 22 +++++++++---------- .../util/collection/ExternalSorterSuite.scala | 10 +++++++++ 2 files changed, 20 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index e32ad9c036ad4..7c9dc8e5f88ef 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -20,6 +20,7 @@ package org.apache.spark.shuffle.hash import org.apache.spark.{InterruptibleIterator, TaskContext} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{BaseShuffleHandle, ShuffleReader} +import org.apache.spark.util.collection.ExternalSorter private[spark] class HashShuffleReader[K, C]( handle: BaseShuffleHandle[K, _, C], @@ -35,8 +36,8 @@ private[spark] class HashShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, - Serializer.getSerializer(dep.serializer)) + val ser = Serializer.getSerializer(dep.serializer) + val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser) val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { @@ -54,16 +55,13 @@ private[spark] class HashShuffleReader[K, C]( // Sort the output if there is a sort ordering defined. dep.keyOrdering match { case Some(keyOrd: Ordering[K]) => - // Define a Comparator for the whole record based on the key Ordering. - val cmp = new Ordering[Product2[K, C]] { - override def compare(o1: Product2[K, C], o2: Product2[K, C]): Int = { - keyOrd.compare(o1._1, o2._1) - } - } - val sortBuffer: Array[Product2[K, C]] = aggregatedIter.toArray - // TODO: do external sort. - scala.util.Sorting.quickSort(sortBuffer)(cmp) - sortBuffer.iterator + // Create an ExternalSorter to sort the data. Note that if spark.shuffle.spill is disabled, + // the ExternalSorter won't spill to disk. + val sorter = new ExternalSorter[K, C, C](ordering = Some(keyOrd), serializer = Some(ser)) + sorter.write(aggregatedIter) + context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled + sorter.iterator case None => aggregatedIter } diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index ddb5df40360e9..65a71e5a83698 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -190,6 +190,11 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { fail(s"Value 2 for ${i} was wrong: expected ${expected}, got ${seq2.toSet}") } } + + // sortByKey - should spill ~17 times + val rddE = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultE = rddE.sortByKey().collect().toSeq + assert(resultE === (0 until 100000).map(i => (i/4, i)).toSeq) } test("spilling in local cluster with many reduce tasks") { @@ -256,6 +261,11 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { fail(s"Value 2 for ${i} was wrong: expected ${expected}, got ${seq2.toSet}") } } + + // sortByKey - should spill ~8 times per executor + val rddE = sc.parallelize(0 until 100000).map(i => (i/4, i)) + val resultE = rddE.sortByKey().collect().toSeq + assert(resultE === (0 until 100000).map(i => (i/4, i)).toSeq) } test("cleanup of intermediate files in sorter") { From f1957e11652a537efd40771f843591a4c9341014 Mon Sep 17 00:00:00 2001 From: Rahul Singhal Date: Fri, 1 Aug 2014 00:33:15 -0700 Subject: [PATCH 0177/1492] SPARK-2134: Report metrics before application finishes Author: Rahul Singhal Closes #1076 from rahulsinghaliitd/SPARK-2134 and squashes the following commits: 15f18b6 [Rahul Singhal] SPARK-2134: Report metrics before application finishes --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../main/scala/org/apache/spark/deploy/master/Master.scala | 2 ++ .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 1 + .../apache/spark/executor/CoarseGrainedExecutorBackend.scala | 1 + core/src/main/scala/org/apache/spark/executor/Executor.scala | 4 ++++ .../main/scala/org/apache/spark/metrics/MetricsSystem.scala | 4 ++++ .../scala/org/apache/spark/metrics/sink/ConsoleSink.scala | 4 ++++ .../main/scala/org/apache/spark/metrics/sink/CsvSink.scala | 4 ++++ .../scala/org/apache/spark/metrics/sink/GraphiteSink.scala | 4 ++++ .../main/scala/org/apache/spark/metrics/sink/JmxSink.scala | 2 ++ .../scala/org/apache/spark/metrics/sink/MetricsServlet.scala | 2 ++ core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala | 1 + .../scala/org/apache/spark/metrics/sink/GangliaSink.scala | 4 ++++ 13 files changed, 34 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b25f081761a64..f5a0549834a0d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -990,6 +990,7 @@ class SparkContext(config: SparkConf) extends Logging { val dagSchedulerCopy = dagScheduler dagScheduler = null if (dagSchedulerCopy != null) { + env.metricsSystem.report() metadataCleaner.cancel() cleaner.foreach(_.stop()) dagSchedulerCopy.stop() 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 21f8667819c44..a70ecdb375373 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 @@ -154,6 +154,8 @@ private[spark] class Master( } override def postStop() { + masterMetricsSystem.report() + applicationMetricsSystem.report() // prevent the CompleteRecovery message sending to restarted master if (recoveryCompletionTask != null) { recoveryCompletionTask.cancel() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index ce425443051b0..fb5252da96519 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -357,6 +357,7 @@ private[spark] class Worker( } override def postStop() { + metricsSystem.report() registrationRetryTimer.foreach(_.cancel()) executors.values.foreach(_.kill()) drivers.values.foreach(_.kill()) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 860b47e056451..af736de405397 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -88,6 +88,7 @@ private[spark] class CoarseGrainedExecutorBackend( case StopExecutor => logInfo("Driver commanded a shutdown") + executor.stop() context.stop(self) context.system.shutdown() } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 3b69bc4ca4142..99d650a3636e2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -121,6 +121,10 @@ private[spark] class Executor( } } + def stop(): Unit = { + env.metricsSystem.report() + } + /** Get the Yarn approved local directories. */ private def getYarnLocalDirs(): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 651511da1b7fe..6ef817d0e587e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -91,6 +91,10 @@ private[spark] class MetricsSystem private (val instance: String, sinks.foreach(_.stop) } + def report(): Unit = { + sinks.foreach(_.report()) + } + def registerSource(source: Source) { sources += source try { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 05852f1f98993..81b9056b40fb8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -57,5 +57,9 @@ private[spark] class ConsoleSink(val property: Properties, val registry: MetricR override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 542dce65366b2..9d5f2ae9328ad 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -66,5 +66,9 @@ private[spark] class CsvSink(val property: Properties, val registry: MetricRegis override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index aeb4ad44a0647..d7b5f5c40efae 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -81,4 +81,8 @@ private[spark] class GraphiteSink(val property: Properties, val registry: Metric override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index ed27234b4e760..2588fe2c9edb8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -35,4 +35,6 @@ private[spark] class JmxSink(val property: Properties, val registry: MetricRegis reporter.stop() } + override def report() { } + } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 571539ba5e467..2f65bc8b46609 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -57,4 +57,6 @@ private[spark] class MetricsServlet(val property: Properties, val registry: Metr override def start() { } override def stop() { } + + override def report() { } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 6f2b5a06027ea..0d83d8c425ca4 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -20,4 +20,5 @@ package org.apache.spark.metrics.sink private[spark] trait Sink { def start: Unit def stop: Unit + def report(): Unit } diff --git a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index d03d7774e8c80..3b1880e143513 100644 --- a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -82,5 +82,9 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, override def stop() { reporter.stop() } + + override def report() { + reporter.report() + } } From 284771efbef2d6b22212afd49dd62732a2cf52a8 Mon Sep 17 00:00:00 2001 From: Ye Xianjin Date: Fri, 1 Aug 2014 00:34:39 -0700 Subject: [PATCH 0178/1492] [Spark 2557] fix LOCAL_N_REGEX in createTaskScheduler and make local-n and local-n-failures consistent [SPARK-2557](https://issues.apache.org/jira/browse/SPARK-2557) Author: Ye Xianjin Closes #1464 from advancedxy/SPARK-2557 and squashes the following commits: d844d67 [Ye Xianjin] add local-*-n-failures, bad-local-n, bad-local-n-failures test case 3bbc668 [Ye Xianjin] fix LOCAL_N_REGEX regular expression and make local_n_failures accept * as all cores on the computer --- .../scala/org/apache/spark/SparkContext.scala | 10 +++++--- .../SparkContextSchedulerCreationSuite.scala | 23 +++++++++++++++++++ 2 files changed, 30 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f5a0549834a0d..0e513568b0243 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1452,9 +1452,9 @@ object SparkContext extends Logging { /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { // Regular expression used for local[N] and local[*] master formats - val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r + val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks - val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r + val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+|\*)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally val LOCAL_CLUSTER_REGEX = """local-cluster\[\s*([0-9]+)\s*,\s*([0-9]+)\s*,\s*([0-9]+)\s*]""".r // Regular expression for connecting to Spark deploy clusters @@ -1484,8 +1484,12 @@ object SparkContext extends Logging { scheduler case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => + def localCpuCount = Runtime.getRuntime.availableProcessors() + // local[*, M] means the number of cores on the computer with M failures + // local[N, M] means exactly N threads with M failures + val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) - val backend = new LocalBackend(scheduler, threads.toInt) + val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) scheduler diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 67e3be21c3c93..4b727e50dbe67 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -68,6 +68,15 @@ class SparkContextSchedulerCreationSuite } } + test("local-*-n-failures") { + val sched = createTaskScheduler("local[* ,2]") + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } + } + test("local-n-failures") { val sched = createTaskScheduler("local[4, 2]") assert(sched.maxTaskFailures === 2) @@ -77,6 +86,20 @@ class SparkContextSchedulerCreationSuite } } + test("bad-local-n") { + val e = intercept[SparkException] { + createTaskScheduler("local[2*]") + } + assert(e.getMessage.contains("Could not parse Master URL")) + } + + test("bad-local-n-failures") { + val e = intercept[SparkException] { + createTaskScheduler("local[2*,4]") + } + assert(e.getMessage.contains("Could not parse Master URL")) + } + test("local-default-parallelism") { val defaultParallelism = System.getProperty("spark.default.parallelism") System.setProperty("spark.default.parallelism", "16") From a32f0fb73a739c56208cafcd9f08618fb6dd8859 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Fri, 1 Aug 2014 04:32:46 -0700 Subject: [PATCH 0179/1492] [SPARK-2103][Streaming] Change to ClassTag for KafkaInputDStream and fix reflection issue This PR updates previous Manifest for KafkaInputDStream's Decoder to ClassTag, also fix the problem addressed in [SPARK-2103](https://issues.apache.org/jira/browse/SPARK-2103). Previous Java interface cannot actually get the type of Decoder, so when using this Manifest to reconstruct the decode object will meet reflection exception. Also for other two Java interfaces, ClassTag[String] is useless because calling Scala API will get the right implicit ClassTag. Current Kafka unit test cannot actually verify the interface. I've tested these interfaces in my local and distribute settings. Author: jerryshao Closes #1508 from jerryshao/SPARK-2103 and squashes the following commits: e90c37b [jerryshao] Add Mima excludes 7529810 [jerryshao] Change Manifest to ClassTag for KafkaInputDStream's Decoder and fix Decoder construct issue when using Java API --- .../streaming/kafka/KafkaInputDStream.scala | 14 +++++++------- .../spark/streaming/kafka/KafkaUtils.scala | 16 +++++----------- project/MimaExcludes.scala | 7 ++++++- 3 files changed, 18 insertions(+), 19 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index 38095e88dcea9..e20e2c8f26991 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.kafka import scala.collection.Map -import scala.reflect.ClassTag +import scala.reflect.{classTag, ClassTag} import java.util.Properties import java.util.concurrent.Executors @@ -48,8 +48,8 @@ private[streaming] class KafkaInputDStream[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: Manifest, - T <: Decoder[_]: Manifest]( + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( @transient ssc_ : StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], @@ -66,8 +66,8 @@ private[streaming] class KafkaReceiver[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: Manifest, - T <: Decoder[_]: Manifest]( + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag]( kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel @@ -103,10 +103,10 @@ class KafkaReceiver[ tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id")) } - val keyDecoder = manifest[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) .newInstance(consumerConfig.props) .asInstanceOf[Decoder[K]] - val valueDecoder = manifest[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) .newInstance(consumerConfig.props) .asInstanceOf[Decoder[V]] diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 86bb91f362d29..48668f763e41e 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -65,7 +65,7 @@ object KafkaUtils { * in its own thread. * @param storageLevel Storage level to use for storing the received objects */ - def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: Manifest, T <: Decoder[_]: Manifest]( + def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( ssc: StreamingContext, kafkaParams: Map[String, String], topics: Map[String, Int], @@ -89,8 +89,6 @@ object KafkaUtils { groupId: String, topics: JMap[String, JInt] ): JavaPairReceiverInputDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) } @@ -111,8 +109,6 @@ object KafkaUtils { topics: JMap[String, JInt], storageLevel: StorageLevel ): JavaPairReceiverInputDStream[String, String] = { - implicit val cmt: ClassTag[String] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } @@ -140,13 +136,11 @@ object KafkaUtils { topics: JMap[String, JInt], storageLevel: StorageLevel ): JavaPairReceiverInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val valueCmt: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) - implicit val keyCmd: Manifest[U] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[U]] - implicit val valueCmd: Manifest[T] = implicitly[Manifest[AnyRef]].asInstanceOf[Manifest[T]] + implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) + implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) createStream[K, V, U, T]( jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 5a835f58207cf..537ca0dcf267d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -71,7 +71,12 @@ object MimaExcludes { "org.apache.spark.storage.TachyonStore.putValues") ) ++ Seq( - ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.flume.FlumeReceiver.this") + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.streaming.flume.FlumeReceiver.this"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.streaming.kafka.KafkaUtils.createStream"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]( + "org.apache.spark.streaming.kafka.KafkaReceiver.this") ) ++ Seq( // Ignore some private methods in ALS. ProblemFilters.exclude[MissingMethodProblem]( From 82d209d43fb543c174e640667de15b00c7fb5d35 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 1 Aug 2014 07:32:53 -0700 Subject: [PATCH 0180/1492] SPARK-2768 [MLLIB] Add product, user recommend method to MatrixFactorizationModel MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Right now, `MatrixFactorizationModel` can only predict a score for one or more `(user,product)` tuples. As a comment in the file notes, it would be more useful to expose a recommend method, that computes top N scoring products for a user (or vice versa – users for a product). (This also corrects some long lines in the Java ALS test suite.) As you can see, it's a little messy to access the class from Java. Should there be a Java-friendly wrapper for it? with a pointer about where that should go, I could add that. Author: Sean Owen Closes #1687 from srowen/SPARK-2768 and squashes the following commits: b349675 [Sean Owen] Additional review changes c9edb04 [Sean Owen] Updates from code review 7bc35f9 [Sean Owen] Add recommend methods to MatrixFactorizationModel --- .../MatrixFactorizationModel.scala | 44 ++++++++++- .../mllib/recommendation/JavaALSSuite.java | 75 ++++++++++++++----- 2 files changed, 100 insertions(+), 19 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 899286d235a9d..a1a76fcbe9f9c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -65,6 +65,48 @@ class MatrixFactorizationModel private[mllib] ( } } + /** + * Recommends products to a user. + * + * @param user the user to recommend products to + * @param num how many products to return. The number returned may be less than this. + * @return [[Rating]] objects, each of which contains the given user ID, a product ID, and a + * "score" in the rating field. Each represents one recommended product, and they are sorted + * by score, decreasing. The first returned is the one predicted to be most strongly + * recommended to the user. The score is an opaque value that indicates how strongly + * recommended the product is. + */ + def recommendProducts(user: Int, num: Int): Array[Rating] = + recommend(userFeatures.lookup(user).head, productFeatures, num) + .map(t => Rating(user, t._1, t._2)) + + /** + * Recommends users to a product. That is, this returns users who are most likely to be + * interested in a product. + * + * @param product the product to recommend users to + * @param num how many users to return. The number returned may be less than this. + * @return [[Rating]] objects, each of which contains a user ID, the given product ID, and a + * "score" in the rating field. Each represents one recommended user, and they are sorted + * by score, decreasing. The first returned is the one predicted to be most strongly + * recommended to the product. The score is an opaque value that indicates how strongly + * recommended the user is. + */ + def recommendUsers(product: Int, num: Int): Array[Rating] = + recommend(productFeatures.lookup(product).head, userFeatures, num) + .map(t => Rating(t._1, product, t._2)) + + private def recommend( + recommendToFeatures: Array[Double], + recommendableFeatures: RDD[(Int, Array[Double])], + num: Int): Array[(Int, Double)] = { + val recommendToVector = new DoubleMatrix(recommendToFeatures) + val scored = recommendableFeatures.map { case (id,features) => + (id, recommendToVector.dot(new DoubleMatrix(features))) + } + scored.top(num)(Ordering.by(_._2)) + } + /** * :: DeveloperApi :: * Predict the rating of many users for many products. @@ -80,6 +122,4 @@ class MatrixFactorizationModel private[mllib] ( predict(usersProducts).map(rate => pythonAPI.serializeRating(rate)) } - // TODO: Figure out what other good bulk prediction methods would look like. - // Probably want a way to get the top users for a product or vice-versa. } diff --git a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java index bf2365f82044c..f6ca9643227f8 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/recommendation/JavaALSSuite.java @@ -20,6 +20,11 @@ import java.io.Serializable; import java.util.List; +import scala.Tuple2; +import scala.Tuple3; + +import org.jblas.DoubleMatrix; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -28,8 +33,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; -import org.jblas.DoubleMatrix; - public class JavaALSSuite implements Serializable { private transient JavaSparkContext sc; @@ -44,21 +47,28 @@ public void tearDown() { sc = null; } - static void validatePrediction(MatrixFactorizationModel model, int users, int products, int features, - DoubleMatrix trueRatings, double matchThreshold, boolean implicitPrefs, DoubleMatrix truePrefs) { + static void validatePrediction( + MatrixFactorizationModel model, + int users, + int products, + int features, + DoubleMatrix trueRatings, + double matchThreshold, + boolean implicitPrefs, + DoubleMatrix truePrefs) { DoubleMatrix predictedU = new DoubleMatrix(users, features); - List> userFeatures = model.userFeatures().toJavaRDD().collect(); + List> userFeatures = model.userFeatures().toJavaRDD().collect(); for (int i = 0; i < features; ++i) { - for (scala.Tuple2 userFeature : userFeatures) { + for (Tuple2 userFeature : userFeatures) { predictedU.put((Integer)userFeature._1(), i, userFeature._2()[i]); } } DoubleMatrix predictedP = new DoubleMatrix(products, features); - List> productFeatures = + List> productFeatures = model.productFeatures().toJavaRDD().collect(); for (int i = 0; i < features; ++i) { - for (scala.Tuple2 productFeature : productFeatures) { + for (Tuple2 productFeature : productFeatures) { predictedP.put((Integer)productFeature._1(), i, productFeature._2()[i]); } } @@ -75,7 +85,8 @@ static void validatePrediction(MatrixFactorizationModel model, int users, int pr } } } else { - // For implicit prefs we use the confidence-weighted RMSE to test (ref Mahout's implicit ALS tests) + // For implicit prefs we use the confidence-weighted RMSE to test + // (ref Mahout's implicit ALS tests) double sqErr = 0.0; double denom = 0.0; for (int u = 0; u < users; ++u) { @@ -100,7 +111,7 @@ public void runALSUsingStaticMethods() { int iterations = 15; int users = 50; int products = 100; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, false, false); JavaRDD data = sc.parallelize(testData._1()); @@ -114,14 +125,14 @@ public void runALSUsingConstructor() { int iterations = 15; int users = 100; int products = 200; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, false, false); JavaRDD data = sc.parallelize(testData._1()); MatrixFactorizationModel model = new ALS().setRank(features) - .setIterations(iterations) - .run(data.rdd()); + .setIterations(iterations) + .run(data.rdd()); validatePrediction(model, users, products, features, testData._2(), 0.3, false, testData._3()); } @@ -131,7 +142,7 @@ public void runImplicitALSUsingStaticMethods() { int iterations = 15; int users = 80; int products = 160; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, true, false); JavaRDD data = sc.parallelize(testData._1()); @@ -145,7 +156,7 @@ public void runImplicitALSUsingConstructor() { int iterations = 15; int users = 100; int products = 200; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, true, false); JavaRDD data = sc.parallelize(testData._1()); @@ -163,12 +174,42 @@ public void runImplicitALSWithNegativeWeight() { int iterations = 15; int users = 80; int products = 160; - scala.Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( users, products, features, 0.7, true, true); JavaRDD data = sc.parallelize(testData._1()); - MatrixFactorizationModel model = ALS.trainImplicit(data.rdd(), features, iterations); + MatrixFactorizationModel model = new ALS().setRank(features) + .setIterations(iterations) + .setImplicitPrefs(true) + .setSeed(8675309L) + .run(data.rdd()); validatePrediction(model, users, products, features, testData._2(), 0.4, true, testData._3()); } + @Test + public void runRecommend() { + int features = 5; + int iterations = 10; + int users = 200; + int products = 50; + Tuple3, DoubleMatrix, DoubleMatrix> testData = ALSSuite.generateRatingsAsJavaList( + users, products, features, 0.7, true, false); + JavaRDD data = sc.parallelize(testData._1()); + MatrixFactorizationModel model = new ALS().setRank(features) + .setIterations(iterations) + .setImplicitPrefs(true) + .setSeed(8675309L) + .run(data.rdd()); + validateRecommendations(model.recommendProducts(1, 10), 10); + validateRecommendations(model.recommendUsers(1, 20), 20); + } + + private static void validateRecommendations(Rating[] recommendations, int howMany) { + Assert.assertEquals(howMany, recommendations.length); + for (int i = 1; i < recommendations.length; i++) { + Assert.assertTrue(recommendations[i-1].rating() >= recommendations[i].rating()); + } + Assert.assertTrue(recommendations[0].rating() > 0.7); + } + } From 0dacb1adb5e6118bd218537bee71926344cd9fb0 Mon Sep 17 00:00:00 2001 From: witgo Date: Fri, 1 Aug 2014 07:47:44 -0700 Subject: [PATCH 0181/1492] [SPARK-1997] update breeze to version 0.8.1 `breeze 0.8.1` dependent on `scala-logging-slf4j 2.1.1` The relevant code on #1369 Author: witgo Closes #940 from witgo/breeze-8.0.1 and squashes the following commits: 65cc65e [witgo] update breeze to version 0.8.1 --- mllib/pom.xml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index cb0fa7b97cb15..9e15ca6ab836c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -52,7 +52,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.7 + 0.8.1 @@ -60,6 +60,10 @@ junit junit + + org.apache.commons + commons-math3 + From 5328c0aaa09911c848f9b3e1e1f2397bef932d0f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 1 Aug 2014 10:00:46 -0700 Subject: [PATCH 0182/1492] [HOTFIX] downgrade breeze version to 0.7 breeze-0.8.1 causes dependency issues, as discussed in #940 . Author: Xiangrui Meng Closes #1718 from mengxr/revert-breeze and squashes the following commits: 99c4681 [Xiangrui Meng] downgrade breeze version to 0.7 --- mllib/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 9e15ca6ab836c..45046eca5b18c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -52,7 +52,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.8.1 + 0.7 From 8d338f64c4eda45d22ae33f61ef7928011cc2846 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 1 Aug 2014 11:08:39 -0700 Subject: [PATCH 0183/1492] SPARK-2099. Report progress while task is running. This is a sketch of a patch that allows the UI to show metrics for tasks that have not yet completed. It adds a heartbeat every 2 seconds from the executors to the driver, reporting metrics for all of the executor's tasks. It still needs unit tests, polish, and cluster testing, but I wanted to put it up to get feedback on the approach. Author: Sandy Ryza Closes #1056 from sryza/sandy-spark-2099 and squashes the following commits: 93b9fdb [Sandy Ryza] Up heartbeat interval to 10 seconds and other tidying 132aec7 [Sandy Ryza] Heartbeat and HeartbeatResponse are already Serializable as case classes 38dffde [Sandy Ryza] Additional review feedback and restore test that was removed in BlockManagerSuite 51fa396 [Sandy Ryza] Remove hostname race, add better comments about threading, and some stylistic improvements 3084f10 [Sandy Ryza] Make TaskUIData a case class again 3bda974 [Sandy Ryza] Stylistic fixes 0dae734 [Sandy Ryza] SPARK-2099. Report progress while task is running. --- .../org/apache/spark/HeartbeatReceiver.scala | 46 +++++++ .../scala/org/apache/spark/SparkContext.scala | 4 + .../scala/org/apache/spark/SparkEnv.scala | 8 +- .../org/apache/spark/executor/Executor.scala | 55 +++++++- .../apache/spark/executor/TaskMetrics.scala | 10 +- .../apache/spark/scheduler/DAGScheduler.scala | 21 +++- .../spark/scheduler/SparkListener.scala | 11 ++ .../spark/scheduler/SparkListenerBus.scala | 2 + .../org/apache/spark/scheduler/Task.scala | 3 + .../spark/scheduler/TaskScheduler.scala | 10 ++ .../spark/scheduler/TaskSchedulerImpl.scala | 23 ++++ .../spark/scheduler/local/LocalBackend.scala | 9 +- .../apache/spark/storage/BlockManager.scala | 25 +--- .../spark/storage/BlockManagerMaster.scala | 43 +------ .../storage/BlockManagerMasterActor.scala | 29 +++-- .../spark/storage/BlockManagerMessages.scala | 6 +- .../spark/ui/jobs/JobProgressListener.scala | 117 +++++++++++------- .../org/apache/spark/ui/jobs/UIData.scala | 9 +- .../org/apache/spark/util/AkkaUtils.scala | 66 +++++++++- .../SparkContextSchedulerCreationSuite.scala | 6 +- .../spark/scheduler/DAGSchedulerSuite.scala | 5 + .../spark/storage/BlockManagerSuite.scala | 23 ++-- .../ui/jobs/JobProgressListenerSuite.scala | 86 ++++++++++++- docs/configuration.md | 7 ++ 24 files changed, 467 insertions(+), 157 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala new file mode 100644 index 0000000000000..24ccce21b62ca --- /dev/null +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import akka.actor.Actor +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.scheduler.TaskScheduler + +/** + * A heartbeat from executors to the driver. This is a shared message used by several internal + * components to convey liveness or execution information for in-progress tasks. + */ +private[spark] case class Heartbeat( + executorId: String, + taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics + blockManagerId: BlockManagerId) + +private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) + +/** + * Lives in the driver to receive heartbeats from executors.. + */ +private[spark] class HeartbeatReceiver(scheduler: TaskScheduler) extends Actor { + override def receive = { + case Heartbeat(executorId, taskMetrics, blockManagerId) => + val response = HeartbeatResponse( + !scheduler.executorHeartbeatReceived(executorId, taskMetrics, blockManagerId)) + sender ! response + } +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0e513568b0243..5f75c1dd2cb68 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf, Sequence import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHadoopJob} import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import akka.actor.Props import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast @@ -307,6 +308,8 @@ class SparkContext(config: SparkConf) extends Logging { // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) + private val heartbeatReceiver = env.actorSystem.actorOf( + Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver") @volatile private[spark] var dagScheduler: DAGScheduler = _ try { dagScheduler = new DAGScheduler(this) @@ -992,6 +995,7 @@ class SparkContext(config: SparkConf) extends Logging { if (dagSchedulerCopy != null) { env.metricsSystem.report() metadataCleaner.cancel() + env.actorSystem.stop(heartbeatReceiver) cleaner.foreach(_.stop()) dagSchedulerCopy.stop() taskScheduler = null diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6ee731b22c03c..92c809d854167 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -193,13 +193,7 @@ object SparkEnv extends Logging { logInfo("Registering " + name) actorSystem.actorOf(Props(newActor), name = name) } else { - val driverHost: String = conf.get("spark.driver.host", "localhost") - val driverPort: Int = conf.getInt("spark.driver.port", 7077) - Utils.checkHost(driverHost, "Expected hostname") - val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" - val timeout = AkkaUtils.lookupTimeout(conf) - logInfo(s"Connecting to $name: $url") - Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) + AkkaUtils.makeDriverRef(name, conf, actorSystem) } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 99d650a3636e2..1bb1b4aae91bb 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import java.util.concurrent._ import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark._ import org.apache.spark.scheduler._ @@ -48,6 +48,8 @@ private[spark] class Executor( private val EMPTY_BYTE_BUFFER = ByteBuffer.wrap(new Array[Byte](0)) + @volatile private var isStopped = false + // No ip or host:port - just hostname Utils.checkHost(slaveHostname, "Expected executed slave to be a hostname") // must not have port specified. @@ -107,6 +109,8 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] + startDriverHeartbeater() + def launchTask( context: ExecutorBackend, taskId: Long, taskName: String, serializedTask: ByteBuffer) { val tr = new TaskRunner(context, taskId, taskName, serializedTask) @@ -121,8 +125,10 @@ private[spark] class Executor( } } - def stop(): Unit = { + def stop() { env.metricsSystem.report() + isStopped = true + threadPool.shutdown() } /** Get the Yarn approved local directories. */ @@ -141,11 +147,12 @@ private[spark] class Executor( } class TaskRunner( - execBackend: ExecutorBackend, taskId: Long, taskName: String, serializedTask: ByteBuffer) + execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) extends Runnable { @volatile private var killed = false - @volatile private var task: Task[Any] = _ + @volatile var task: Task[Any] = _ + @volatile var attemptedTask: Option[Task[Any]] = None def kill(interruptThread: Boolean) { logInfo(s"Executor is trying to kill $taskName (TID $taskId)") @@ -162,7 +169,6 @@ private[spark] class Executor( val ser = SparkEnv.get.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) - var attemptedTask: Option[Task[Any]] = None var taskStart: Long = 0 def gcTime = ManagementFactory.getGarbageCollectorMXBeans.map(_.getCollectionTime).sum val startGCTime = gcTime @@ -204,7 +210,6 @@ private[spark] class Executor( val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { - m.hostname = Utils.localHostName() m.executorDeserializeTime = taskStart - startTime m.executorRunTime = taskFinish - taskStart m.jvmGCTime = gcTime - startGCTime @@ -354,4 +359,42 @@ private[spark] class Executor( } } } + + def startDriverHeartbeater() { + val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val timeout = AkkaUtils.lookupTimeout(conf) + val retryAttempts = AkkaUtils.numRetries(conf) + val retryIntervalMs = AkkaUtils.retryWaitMs(conf) + val heartbeatReceiverRef = AkkaUtils.makeDriverRef("HeartbeatReceiver", conf, env.actorSystem) + + val t = new Thread() { + override def run() { + // Sleep a random interval so the heartbeats don't end up in sync + Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) + + while (!isStopped) { + val tasksMetrics = new ArrayBuffer[(Long, TaskMetrics)]() + for (taskRunner <- runningTasks.values()) { + if (!taskRunner.attemptedTask.isEmpty) { + Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => + tasksMetrics += ((taskRunner.taskId, metrics)) + } + } + } + + val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) + val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, + retryAttempts, retryIntervalMs, timeout) + if (response.reregisterBlockManager) { + logWarning("Told to re-register on heartbeat") + env.blockManager.reregister() + } + Thread.sleep(interval) + } + } + } + t.setDaemon(true) + t.setName("Driver Heartbeater") + t.start() + } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 21fe643b8d71f..56cd8723a3a22 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -23,6 +23,14 @@ import org.apache.spark.storage.{BlockId, BlockStatus} /** * :: DeveloperApi :: * Metrics tracked during the execution of a task. + * + * This class is used to house metrics both for in-progress and completed tasks. In executors, + * both the task thread and the heartbeat thread write to the TaskMetrics. The heartbeat thread + * reads it to send in-progress metrics, and the task thread reads it to send metrics along with + * the completed task. + * + * So, when adding new fields, take into consideration that the whole object can be serialized for + * shipping off at any time to consumers of the SparkListener interface. */ @DeveloperApi class TaskMetrics extends Serializable { @@ -143,7 +151,7 @@ class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data */ - var shuffleFinishTime: Long = _ + var shuffleFinishTime: Long = -1 /** * Number of blocks fetched in this shuffle by this task (remote or local) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 50186d097a632..c7e3d7c5f8530 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -29,7 +29,6 @@ import scala.reflect.ClassTag import scala.util.control.NonFatal import akka.actor._ -import akka.actor.OneForOneStrategy import akka.actor.SupervisorStrategy.Stop import akka.pattern.ask import akka.util.Timeout @@ -39,8 +38,9 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} +import org.apache.spark.storage._ import org.apache.spark.util.{CallSite, SystemClock, Clock, Utils} +import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -154,6 +154,23 @@ class DAGScheduler( eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) } + /** + * Update metrics for in-progress tasks and let the master know that the BlockManager is still + * alive. Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + def executorHeartbeatReceived( + execId: String, + taskMetrics: Array[(Long, Int, TaskMetrics)], // (taskId, stageId, metrics) + blockManagerId: BlockManagerId): Boolean = { + listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) + implicit val timeout = Timeout(600 seconds) + + Await.result( + blockManagerMaster.driverActor ? BlockManagerHeartbeat(blockManagerId), + timeout.duration).asInstanceOf[Boolean] + } + // Called by TaskScheduler when an executor fails. def executorLost(execId: String) { eventProcessActor ! ExecutorLost(execId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 82163eadd56e9..d01d318633877 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -75,6 +75,12 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +@DeveloperApi +case class SparkListenerExecutorMetricsUpdate( + execId: String, + taskMetrics: Seq[(Long, Int, TaskMetrics)]) + extends SparkListenerEvent + @DeveloperApi case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) extends SparkListenerEvent @@ -158,6 +164,11 @@ trait SparkListener { * Called when the application ends */ def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } + + /** + * Called when the driver receives task metrics from an executor in a heartbeat. + */ + def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index ed9fb24bc8ce8..e79ffd7a3587d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -68,6 +68,8 @@ private[spark] trait SparkListenerBus extends Logging { foreachListener(_.onApplicationStart(applicationStart)) case applicationEnd: SparkListenerApplicationEnd => foreachListener(_.onApplicationEnd(applicationEnd)) + case metricsUpdate: SparkListenerExecutorMetricsUpdate => + foreachListener(_.onExecutorMetricsUpdate(metricsUpdate)) case SparkListenerShutdown => } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 5871edeb856ad..5c5e421404a21 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -26,6 +26,8 @@ import org.apache.spark.TaskContext import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.Utils + /** * A unit of execution. We have two kinds of Task's in Spark: @@ -44,6 +46,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(attemptId: Long): T = { context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + context.taskMetrics.hostname = Utils.localHostName(); taskThread = Thread.currentThread() if (_killed) { kill(interruptThread = false) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 819c35257b5a7..1a0b877c8a5e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -18,6 +18,8 @@ package org.apache.spark.scheduler import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId /** * Low-level task scheduler interface, currently implemented exclusively by TaskSchedulerImpl. @@ -54,4 +56,12 @@ private[spark] trait TaskScheduler { // Get the default level of parallelism to use in the cluster, as a hint for sizing jobs. def defaultParallelism(): Int + + /** + * Update metrics for in-progress tasks and let the master know that the BlockManager is still + * alive. Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean } 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 be3673c48eda8..d2f764fc22f54 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -32,6 +32,9 @@ import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId +import akka.actor.Props /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -320,6 +323,26 @@ private[spark] class TaskSchedulerImpl( } } + /** + * Update metrics for in-progress tasks and let the master know that the BlockManager is still + * alive. Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + override def executorHeartbeatReceived( + execId: String, + taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics + blockManagerId: BlockManagerId): Boolean = { + val metricsWithStageIds = taskMetrics.flatMap { + case (id, metrics) => { + taskIdToTaskSetId.get(id) + .flatMap(activeTaskSets.get) + .map(_.stageId) + .map(x => (id, x, metrics)) + } + } + dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) + } + def handleTaskGettingResult(taskSetManager: TaskSetManager, tid: Long) { taskSetManager.handleTaskGettingResult(tid) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 5b897597fa285..3d1cf312ccc97 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -23,8 +23,9 @@ import akka.actor.{Actor, ActorRef, Props} import org.apache.spark.{Logging, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.executor.{TaskMetrics, Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.storage.BlockManagerId private case class ReviveOffers() @@ -32,6 +33,8 @@ private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: private case class KillTask(taskId: Long, interruptThread: Boolean) +private case class StopExecutor() + /** * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on * LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend @@ -63,6 +66,9 @@ private[spark] class LocalActor( case KillTask(taskId, interruptThread) => executor.killTask(taskId, interruptThread) + + case StopExecutor => + executor.stop() } def reviveOffers() { @@ -91,6 +97,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: } override def stop() { + localActor ! StopExecutor } override def reviveOffers() { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index d746526639e58..c0a06017945f0 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -116,15 +116,6 @@ private[spark] class BlockManager( private var asyncReregisterTask: Future[Unit] = null private val asyncReregisterLock = new Object - private def heartBeat(): Unit = { - if (!master.sendHeartBeat(blockManagerId)) { - reregister() - } - } - - private val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) - private var heartBeatTask: Cancellable = null - private val metadataCleaner = new MetadataCleaner( MetadataCleanerType.BLOCK_MANAGER, this.dropOldNonBroadcastBlocks, conf) private val broadcastCleaner = new MetadataCleaner( @@ -161,11 +152,6 @@ private[spark] class BlockManager( private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) BlockManagerWorker.startBlockManagerWorker(this) - if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { - heartBeatTask = actorSystem.scheduler.schedule(0.seconds, heartBeatFrequency.milliseconds) { - Utils.tryOrExit { heartBeat() } - } - } } /** @@ -195,7 +181,7 @@ private[spark] class BlockManager( * * Note that this method must be called without any BlockInfo locks held. */ - private def reregister(): Unit = { + def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo("BlockManager re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveActor) @@ -1065,9 +1051,6 @@ private[spark] class BlockManager( } def stop(): Unit = { - if (heartBeatTask != null) { - heartBeatTask.cancel() - } connectionManager.stop() shuffleBlockManager.stop() diskBlockManager.stop() @@ -1095,12 +1078,6 @@ private[spark] object BlockManager extends Logging { (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong } - def getHeartBeatFrequency(conf: SparkConf): Long = - conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) / 4 - - def getDisableHeartBeatsForTesting(conf: SparkConf): Boolean = - conf.getBoolean("spark.test.disableBlockManagerHeartBeat", false) - /** * Attempt to clean up a ByteBuffer if it is memory-mapped. This uses an *unsafe* Sun API that * might cause errors if one attempts to read from the unmapped buffer, but it's better than diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 7897fade2df2b..669307765d1fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -21,7 +21,6 @@ import scala.concurrent.{Await, Future} import scala.concurrent.ExecutionContext.Implicits.global import akka.actor._ -import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.storage.BlockManagerMessages._ @@ -29,8 +28,8 @@ import org.apache.spark.util.AkkaUtils private[spark] class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { - val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) - val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) + private val AKKA_RETRY_ATTEMPTS: Int = AkkaUtils.numRetries(conf) + private val AKKA_RETRY_INTERVAL_MS: Int = AkkaUtils.retryWaitMs(conf) val DRIVER_AKKA_ACTOR_NAME = "BlockManagerMaster" @@ -42,15 +41,6 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log logInfo("Removed " + execId + " successfully in removeExecutor") } - /** - * Send the driver actor a heart beat from the slave. Returns true if everything works out, - * false if the driver does not know about the given block manager, which means the block - * manager should re-register. - */ - def sendHeartBeat(blockManagerId: BlockManagerId): Boolean = { - askDriverWithReply[Boolean](HeartBeat(blockManagerId)) - } - /** Register the BlockManager's id with the driver. */ def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") @@ -223,33 +213,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log * throw a SparkException if this fails. */ private def askDriverWithReply[T](message: Any): T = { - // TODO: Consider removing multiple attempts - if (driverActor == null) { - throw new SparkException("Error sending message to BlockManager as driverActor is null " + - "[message = " + message + "]") - } - var attempts = 0 - var lastException: Exception = null - while (attempts < AKKA_RETRY_ATTEMPTS) { - attempts += 1 - try { - val future = driverActor.ask(message)(timeout) - val result = Await.result(future, timeout) - if (result == null) { - throw new SparkException("BlockManagerMaster returned null") - } - return result.asInstanceOf[T] - } catch { - case ie: InterruptedException => throw ie - case e: Exception => - lastException = e - logWarning("Error sending message to BlockManagerMaster in " + attempts + " attempts", e) - } - Thread.sleep(AKKA_RETRY_INTERVAL_MS) - } - - throw new SparkException( - "Error sending message to BlockManagerMaster [message = " + message + "]", lastException) + AkkaUtils.askWithReply(message, driverActor, AKKA_RETRY_ATTEMPTS, AKKA_RETRY_INTERVAL_MS, + timeout) } } 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 de1cc5539fb48..94f5a4bb2e9cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -52,25 +52,24 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus private val akkaTimeout = AkkaUtils.askTimeout(conf) - val slaveTimeout = conf.get("spark.storage.blockManagerSlaveTimeoutMs", - "" + (BlockManager.getHeartBeatFrequency(conf) * 3)).toLong + val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", + math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000)) - val checkTimeoutInterval = conf.get("spark.storage.blockManagerTimeoutIntervalMs", - "60000").toLong + val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", + 60000) var timeoutCheckingTask: Cancellable = null override def preStart() { - if (!BlockManager.getDisableHeartBeatsForTesting(conf)) { - import context.dispatcher - timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, - checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) - } + import context.dispatcher + timeoutCheckingTask = context.system.scheduler.schedule(0.seconds, + checkTimeoutInterval.milliseconds, self, ExpireDeadHosts) super.preStart() } def receive = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => + logInfo("received a register") register(blockManagerId, maxMemSize, slaveActor) sender ! true @@ -129,8 +128,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case ExpireDeadHosts => expireDeadHosts() - case HeartBeat(blockManagerId) => - sender ! heartBeat(blockManagerId) + case BlockManagerHeartbeat(blockManagerId) => + sender ! heartbeatReceived(blockManagerId) case other => logWarning("Got unknown message: " + other) @@ -216,7 +215,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val minSeenTime = now - slaveTimeout val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { - if (info.lastSeenMs < minSeenTime) { + if (info.lastSeenMs < minSeenTime && info.blockManagerId.executorId != "") { logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId @@ -230,7 +229,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockManagerIdByExecutor.get(execId).foreach(removeBlockManager) } - private def heartBeat(blockManagerId: BlockManagerId): Boolean = { + /** + * Return true if the driver knows about the given block manager. Otherwise, return false, + * indicating that the block manager should re-register. + */ + private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { blockManagerId.executorId == "" && !isLocal } else { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 2b53bf33b5fba..10b65286fb7db 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -21,7 +21,7 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef -private[storage] object BlockManagerMessages { +private[spark] object BlockManagerMessages { ////////////////////////////////////////////////////////////////////////////////// // Messages from the master to slaves. ////////////////////////////////////////////////////////////////////////////////// @@ -53,8 +53,6 @@ private[storage] object BlockManagerMessages { sender: ActorRef) extends ToBlockManagerMaster - case class HeartBeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - class UpdateBlockInfo( var blockManagerId: BlockManagerId, var blockId: BlockId, @@ -124,5 +122,7 @@ private[storage] object BlockManagerMessages { case class GetMatchingBlockIds(filter: BlockId => Boolean, askSlaves: Boolean = true) extends ToBlockManagerMaster + case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + case object ExpireDeadHosts extends ToBlockManagerMaster } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index efb527b4f03e6..da2f5d3172fe2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -130,32 +130,16 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { new StageUIData }) - // create executor summary map if necessary - val executorSummaryMap = stageData.executorSummary - executorSummaryMap.getOrElseUpdate(key = info.executorId, op = new ExecutorSummary) - - executorSummaryMap.get(info.executorId).foreach { y => - // first update failed-task, succeed-task - taskEnd.reason match { - case Success => - y.succeededTasks += 1 - case _ => - y.failedTasks += 1 - } - - // update duration - y.taskTime += info.duration - - val metrics = taskEnd.taskMetrics - if (metrics != null) { - metrics.inputMetrics.foreach { y.inputBytes += _.bytesRead } - metrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } - metrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } - y.memoryBytesSpilled += metrics.memoryBytesSpilled - y.diskBytesSpilled += metrics.diskBytesSpilled - } + val execSummaryMap = stageData.executorSummary + val execSummary = execSummaryMap.getOrElseUpdate(info.executorId, new ExecutorSummary) + + taskEnd.reason match { + case Success => + execSummary.succeededTasks += 1 + case _ => + execSummary.failedTasks += 1 } - + execSummary.taskTime += info.duration stageData.numActiveTasks -= 1 val (errorMessage, metrics): (Option[String], Option[TaskMetrics]) = @@ -171,28 +155,75 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { (Some(e.toErrorString), None) } + if (!metrics.isEmpty) { + val oldMetrics = stageData.taskData.get(info.taskId).flatMap(_.taskMetrics) + updateAggregateMetrics(stageData, info.executorId, metrics.get, oldMetrics) + } - val taskRunTime = metrics.map(_.executorRunTime).getOrElse(0L) - stageData.executorRunTime += taskRunTime - val inputBytes = metrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L) - stageData.inputBytes += inputBytes - - val shuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L) - stageData.shuffleReadBytes += shuffleRead - - val shuffleWrite = - metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L) - stageData.shuffleWriteBytes += shuffleWrite - - val memoryBytesSpilled = metrics.map(_.memoryBytesSpilled).getOrElse(0L) - stageData.memoryBytesSpilled += memoryBytesSpilled + val taskData = stageData.taskData.getOrElseUpdate(info.taskId, new TaskUIData(info)) + taskData.taskInfo = info + taskData.taskMetrics = metrics + taskData.errorMessage = errorMessage + } + } - val diskBytesSpilled = metrics.map(_.diskBytesSpilled).getOrElse(0L) - stageData.diskBytesSpilled += diskBytesSpilled + /** + * Upon receiving new metrics for a task, updates the per-stage and per-executor-per-stage + * aggregate metrics by calculating deltas between the currently recorded metrics and the new + * metrics. + */ + def updateAggregateMetrics( + stageData: StageUIData, + execId: String, + taskMetrics: TaskMetrics, + oldMetrics: Option[TaskMetrics]) { + val execSummary = stageData.executorSummary.getOrElseUpdate(execId, new ExecutorSummary) + + val shuffleWriteDelta = + (taskMetrics.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L)) + stageData.shuffleWriteBytes += shuffleWriteDelta + execSummary.shuffleWrite += shuffleWriteDelta + + val shuffleReadDelta = + (taskMetrics.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L) + - oldMetrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L)) + stageData.shuffleReadBytes += shuffleReadDelta + execSummary.shuffleRead += shuffleReadDelta + + val diskSpillDelta = + taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) + stageData.diskBytesSpilled += diskSpillDelta + execSummary.diskBytesSpilled += diskSpillDelta + + val memorySpillDelta = + taskMetrics.memoryBytesSpilled - oldMetrics.map(_.memoryBytesSpilled).getOrElse(0L) + stageData.memoryBytesSpilled += memorySpillDelta + execSummary.memoryBytesSpilled += memorySpillDelta + + val timeDelta = + taskMetrics.executorRunTime - oldMetrics.map(_.executorRunTime).getOrElse(0L) + stageData.executorRunTime += timeDelta + } - stageData.taskData(info.taskId) = new TaskUIData(info, metrics, errorMessage) + override def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { + for ((taskId, sid, taskMetrics) <- executorMetricsUpdate.taskMetrics) { + val stageData = stageIdToData.getOrElseUpdate(sid, { + logWarning("Metrics update for task in unknown stage " + sid) + new StageUIData + }) + val taskData = stageData.taskData.get(taskId) + taskData.map { t => + if (!t.taskInfo.finished) { + updateAggregateMetrics(stageData, executorMetricsUpdate.execId, taskMetrics, + t.taskMetrics) + + // Overwrite task metrics + t.taskMetrics = Some(taskMetrics) + } + } } - } // end of onTaskEnd + } override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index be11a11695b01..2f96f7909c199 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -55,8 +55,11 @@ private[jobs] object UIData { var executorSummary = new HashMap[String, ExecutorSummary] } + /** + * These are kept mutable and reused throughout a task's lifetime to avoid excessive reallocation. + */ case class TaskUIData( - taskInfo: TaskInfo, - taskMetrics: Option[TaskMetrics] = None, - errorMessage: Option[String] = None) + var taskInfo: TaskInfo, + var taskMetrics: Option[TaskMetrics] = None, + var errorMessage: Option[String] = None) } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 9930c717492f2..feafd654e9e71 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -18,13 +18,16 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap +import scala.concurrent.Await import scala.concurrent.duration.{Duration, FiniteDuration} -import akka.actor.{ActorSystem, ExtendedActorSystem} +import akka.actor.{Actor, ActorRef, ActorSystem, ExtendedActorSystem} +import akka.pattern.ask + import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} /** * Various utility classes for working with Akka. @@ -124,4 +127,63 @@ private[spark] object AkkaUtils extends Logging { /** Space reserved for extra data in an Akka message besides serialized task or task result. */ val reservedSizeBytes = 200 * 1024 + + /** Returns the configured number of times to retry connecting */ + def numRetries(conf: SparkConf): Int = { + conf.getInt("spark.akka.num.retries", 3) + } + + /** Returns the configured number of milliseconds to wait on each retry */ + def retryWaitMs(conf: SparkConf): Int = { + conf.getInt("spark.akka.retry.wait", 3000) + } + + /** + * Send a message to the given actor and get its result within a default timeout, or + * throw a SparkException if this fails. + */ + def askWithReply[T]( + message: Any, + actor: ActorRef, + retryAttempts: Int, + retryInterval: Int, + timeout: FiniteDuration): T = { + // TODO: Consider removing multiple attempts + if (actor == null) { + throw new SparkException("Error sending message as driverActor is null " + + "[message = " + message + "]") + } + var attempts = 0 + var lastException: Exception = null + while (attempts < retryAttempts) { + attempts += 1 + try { + val future = actor.ask(message)(timeout) + val result = Await.result(future, timeout) + if (result == null) { + throw new SparkException("Actor returned null") + } + return result.asInstanceOf[T] + } catch { + case ie: InterruptedException => throw ie + case e: Exception => + lastException = e + logWarning("Error sending message in " + attempts + " attempts", e) + } + Thread.sleep(retryInterval) + } + + throw new SparkException( + "Error sending message [message = " + message + "]", lastException) + } + + def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { + val driverHost: String = conf.get("spark.driver.host", "localhost") + val driverPort: Int = conf.getInt("spark.driver.port", 7077) + Utils.checkHost(driverHost, "Expected hostname") + val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" + val timeout = AkkaUtils.lookupTimeout(conf) + logInfo(s"Connecting to $name: $url") + Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) + } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 4b727e50dbe67..495a0d48633a4 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.scalatest.{FunSuite, PrivateMethodTester} +import org.scalatest.{BeforeAndAfterEach, FunSuite, PrivateMethodTester} import org.apache.spark.scheduler.{TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} @@ -25,12 +25,12 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite - extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging { + extends FunSuite with PrivateMethodTester with Logging with BeforeAndAfterEach { def createTaskScheduler(master: String): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test") + val sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 9021662bcf712..0ce13d015df05 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} import org.apache.spark.util.CallSite +import org.apache.spark.executor.TaskMetrics class BuggyDAGEventProcessActor extends Actor { val state = 0 @@ -77,6 +78,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F override def schedulingMode: SchedulingMode = SchedulingMode.NONE override def start() = {} override def stop() = {} + override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean = true override def submitTasks(taskSet: TaskSet) = { // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) @@ -342,6 +345,8 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 + override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean = true } val noKillScheduler = new DAGScheduler( sc, diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 58ea0cc30e954..0ac0269d7cfc1 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -19,22 +19,28 @@ package org.apache.spark.storage import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays +import java.util.concurrent.TimeUnit import akka.actor._ +import akka.pattern.ask +import akka.util.Timeout + import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers -import org.scalatest.time.SpanSugar._ import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Await +import scala.concurrent.duration._ import scala.language.implicitConversions import scala.language.postfixOps @@ -73,7 +79,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter oldArch = System.setProperty("os.arch", "amd64") conf.set("os.arch", "amd64") conf.set("spark.test.useCompressedOops", "true") - conf.set("spark.storage.disableBlockManagerHeartBeat", "true") conf.set("spark.driver.port", boundPort.toString) conf.set("spark.storage.unrollFraction", "0.4") conf.set("spark.storage.unrollMemoryThreshold", "512") @@ -341,7 +346,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("reregistration on heart beat") { - val heartBeat = PrivateMethod[Unit]('heartBeat) store = makeBlockManager(2000) val a1 = new Array[Byte](400) @@ -353,13 +357,15 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master.removeExecutor(store.blockManagerId.executorId) assert(master.getLocations("a1").size == 0, "a1 was not removed from master") - store invokePrivate heartBeat() - assert(master.getLocations("a1").size > 0, "a1 was not reregistered with master") + implicit val timeout = Timeout(30, TimeUnit.SECONDS) + val reregister = !Await.result( + master.driverActor ? BlockManagerHeartbeat(store.blockManagerId), + timeout.duration).asInstanceOf[Boolean] + assert(reregister == true) } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, - securityMgr, mapOutputTracker) + store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -377,7 +383,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("reregistration doesn't dead lock") { - val heartBeat = PrivateMethod[Unit]('heartBeat) store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -397,7 +402,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } val t3 = new Thread { override def run() { - store invokePrivate heartBeat() + store.reregister() } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 86a271eb67000..cb8252515238e 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -21,7 +21,8 @@ import org.scalatest.FunSuite import org.scalatest.Matchers import org.apache.spark._ -import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.{LocalSparkContext, SparkConf, Success} +import org.apache.spark.executor.{ShuffleWriteMetrics, ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -129,4 +130,87 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(listener.stageIdToData(task.stageId).numCompleteTasks === 1) assert(listener.stageIdToData(task.stageId).numFailedTasks === failCount) } + + test("test update metrics") { + val conf = new SparkConf() + val listener = new JobProgressListener(conf) + + val taskType = Utils.getFormattedClassName(new ShuffleMapTask(0)) + val execId = "exe-1" + + def makeTaskMetrics(base: Int) = { + val taskMetrics = new TaskMetrics() + val shuffleReadMetrics = new ShuffleReadMetrics() + val shuffleWriteMetrics = new ShuffleWriteMetrics() + taskMetrics.updateShuffleReadMetrics(shuffleReadMetrics) + taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) + shuffleReadMetrics.remoteBytesRead = base + 1 + shuffleReadMetrics.remoteBlocksFetched = base + 2 + shuffleWriteMetrics.shuffleBytesWritten = base + 3 + taskMetrics.executorRunTime = base + 4 + taskMetrics.diskBytesSpilled = base + 5 + taskMetrics.memoryBytesSpilled = base + 6 + taskMetrics + } + + def makeTaskInfo(taskId: Long, finishTime: Int = 0) = { + val taskInfo = new TaskInfo(taskId, 0, 1, 0L, execId, "host1", TaskLocality.NODE_LOCAL, + false) + taskInfo.finishTime = finishTime + taskInfo + } + + listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1234L))) + listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1235L))) + listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1236L))) + listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1237L))) + + listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, Array( + (1234L, 0, makeTaskMetrics(0)), + (1235L, 0, makeTaskMetrics(100)), + (1236L, 1, makeTaskMetrics(200))))) + + var stage0Data = listener.stageIdToData.get(0).get + var stage1Data = listener.stageIdToData.get(1).get + assert(stage0Data.shuffleReadBytes == 102) + assert(stage1Data.shuffleReadBytes == 201) + assert(stage0Data.shuffleWriteBytes == 106) + assert(stage1Data.shuffleWriteBytes == 203) + assert(stage0Data.executorRunTime == 108) + assert(stage1Data.executorRunTime == 204) + assert(stage0Data.diskBytesSpilled == 110) + assert(stage1Data.diskBytesSpilled == 205) + assert(stage0Data.memoryBytesSpilled == 112) + assert(stage1Data.memoryBytesSpilled == 206) + assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 2) + assert(stage0Data.taskData.get(1235L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 102) + assert(stage1Data.taskData.get(1236L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 202) + + // task that was included in a heartbeat + listener.onTaskEnd(SparkListenerTaskEnd(0, taskType, Success, makeTaskInfo(1234L, 1), + makeTaskMetrics(300))) + // task that wasn't included in a heartbeat + listener.onTaskEnd(SparkListenerTaskEnd(1, taskType, Success, makeTaskInfo(1237L, 1), + makeTaskMetrics(400))) + + stage0Data = listener.stageIdToData.get(0).get + stage1Data = listener.stageIdToData.get(1).get + assert(stage0Data.shuffleReadBytes == 402) + assert(stage1Data.shuffleReadBytes == 602) + assert(stage0Data.shuffleWriteBytes == 406) + assert(stage1Data.shuffleWriteBytes == 606) + assert(stage0Data.executorRunTime == 408) + assert(stage1Data.executorRunTime == 608) + assert(stage0Data.diskBytesSpilled == 410) + assert(stage1Data.diskBytesSpilled == 610) + assert(stage0Data.memoryBytesSpilled == 412) + assert(stage1Data.memoryBytesSpilled == 612) + assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 302) + assert(stage1Data.taskData.get(1237L).get.taskMetrics.get.shuffleReadMetrics.get + .totalBlocksFetched == 402) + } } diff --git a/docs/configuration.md b/docs/configuration.md index ea69057b5be10..2a71d7b820e5f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -541,6 +541,13 @@ Apart from these, the following properties are also available, and may be useful output directories. We recommend that users do not disable this except if trying to achieve compatibility with previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + + + + +
Indicates whether the history server should use kerberos to login. This is useful if the history server is accessing HDFS files on a secure Hadoop cluster. If this is - true it looks uses the configs spark.history.kerberos.principal and + true, it uses the configs spark.history.kerberos.principal and spark.history.kerberos.keytab.
spark.executor.heartbeatInterval10000Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + the driver know that the executor is still alive and update it with metrics for in-progress + tasks.
#### Networking From c41fdf04f4beebe36379396b0c4fff3ab7ad3cf4 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 1 Aug 2014 11:14:53 -0700 Subject: [PATCH 0184/1492] [SPARK-2179][SQL] A minor refactoring Java data type APIs (2179 follow-up). It is a follow-up PR of SPARK-2179 (https://issues.apache.org/jira/browse/SPARK-2179). It makes package names of data type APIs more consistent across languages (Scala: `org.apache.spark.sql`, Java: `org.apache.spark.sql.api.java`, Python: `pyspark.sql`). Author: Yin Huai Closes #1712 from yhuai/javaDataType and squashes the following commits: 62eb705 [Yin Huai] Move package-info. add4bcb [Yin Huai] Make the package names of data type classes consistent across languages by moving all Java data type classes to package sql.api.java. --- .../sql/api/java/{types => }/ArrayType.java | 6 +- .../sql/api/java/{types => }/BinaryType.java | 2 +- .../sql/api/java/{types => }/BooleanType.java | 2 +- .../sql/api/java/{types => }/ByteType.java | 2 +- .../sql/api/java/{types => }/DataType.java | 2 +- .../sql/api/java/{types => }/DecimalType.java | 2 +- .../sql/api/java/{types => }/DoubleType.java | 2 +- .../sql/api/java/{types => }/FloatType.java | 2 +- .../sql/api/java/{types => }/IntegerType.java | 2 +- .../sql/api/java/{types => }/LongType.java | 2 +- .../sql/api/java/{types => }/MapType.java | 6 +- .../sql/api/java/{types => }/ShortType.java | 2 +- .../sql/api/java/{types => }/StringType.java | 2 +- .../sql/api/java/{types => }/StructField.java | 4 +- .../sql/api/java/{types => }/StructType.java | 7 +-- .../api/java/{types => }/TimestampType.java | 2 +- .../spark/sql/api/java}/package-info.java | 2 +- .../sql/api/java/types/package-info.java | 22 ------- .../spark/sql/api/java/JavaSQLContext.scala | 60 ++++++++++++------- .../spark/sql/api/java/JavaSchemaRDD.scala | 1 - .../sql/types/util/DataTypeConversions.scala | 30 +++++----- .../sql/api/java/JavaApplySchemaSuite.java | 3 - .../java/JavaSideDataTypeConversionSuite.java | 2 - .../ScalaSideDataTypeConversionSuite.scala | 59 +++++++++--------- 24 files changed, 108 insertions(+), 118 deletions(-) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/ArrayType.java (90%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/BinaryType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/BooleanType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/ByteType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/DataType.java (99%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/DecimalType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/DoubleType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/FloatType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/IntegerType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/LongType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/MapType.java (91%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/ShortType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/StringType.java (95%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/StructField.java (94%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/StructType.java (86%) rename sql/core/src/main/java/org/apache/spark/sql/api/java/{types => }/TimestampType.java (95%) rename sql/core/src/main/{scala/org/apache/spark/sql => java/org/apache/spark/sql/api/java}/package-info.java (95%) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ArrayType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java similarity index 90% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/ArrayType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java index 17334ca31b2b7..b73a371e93001 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ArrayType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/ArrayType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing Lists. @@ -25,8 +25,8 @@ * {@code null} values. * * To create an {@link ArrayType}, - * {@link org.apache.spark.sql.api.java.types.DataType#createArrayType(DataType)} or - * {@link org.apache.spark.sql.api.java.types.DataType#createArrayType(DataType, boolean)} + * {@link DataType#createArrayType(DataType)} or + * {@link DataType#createArrayType(DataType, boolean)} * should be used. */ public class ArrayType extends DataType { diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BinaryType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/BinaryType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java index 61703179850e9..7daad60f62a0b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BinaryType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/BinaryType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing byte[] values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BooleanType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/BooleanType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java index 8fa24d85d1238..5a1f52725631b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/BooleanType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/BooleanType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing boolean and Boolean values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ByteType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/ByteType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java index 2de32978e2705..e5cdf06b21bbe 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ByteType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/ByteType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing byte and Byte values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java similarity index 99% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/DataType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index f84e5a490a905..3eccddef88134 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; import java.util.HashSet; import java.util.List; diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DecimalType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/DecimalType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java index 9250491a2d2ca..bc54c078d7a4e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DecimalType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing java.math.BigDecimal values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DoubleType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/DoubleType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java index 3e86917fddc4b..f0060d0bcf9f5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/DoubleType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DoubleType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing double and Double values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/FloatType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/FloatType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java index fa860d40176ef..4a6a37f69176a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/FloatType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/FloatType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing float and Float values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/IntegerType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/IntegerType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java index bd973eca2c3ce..bfd70490bbbbb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/IntegerType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/IntegerType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing int and Integer values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/LongType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/LongType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java index e00233304cefa..af13a46eb165c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/LongType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/LongType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing long and Long values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/MapType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java similarity index 91% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/MapType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java index 94936e2e4ee7a..063e6b34abc48 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/MapType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/MapType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing Maps. A MapType object comprises two fields, @@ -27,8 +27,8 @@ * For values of a MapType column, keys are not allowed to have {@code null} values. * * To create a {@link MapType}, - * {@link org.apache.spark.sql.api.java.types.DataType#createMapType(DataType, DataType)} or - * {@link org.apache.spark.sql.api.java.types.DataType#createMapType(DataType, DataType, boolean)} + * {@link DataType#createMapType(DataType, DataType)} or + * {@link DataType#createMapType(DataType, DataType, boolean)} * should be used. */ public class MapType extends DataType { diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ShortType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/ShortType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java index 98f9507acf121..7d7604b4e3d2d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/ShortType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/ShortType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing short and Short values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StringType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/StringType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java index b8e7dbe646071..f4ba0c07c9c6e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StringType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/StringType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing String values. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructField.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java similarity index 94% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructField.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java index 54e9c11ea415e..b48e2a2c5f953 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructField.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * A StructField object represents a field in a StructType object. @@ -26,7 +26,7 @@ * values. * * To create a {@link StructField}, - * {@link org.apache.spark.sql.api.java.types.DataType#createStructField(String, DataType, boolean)} + * {@link DataType#createStructField(String, DataType, boolean)} * should be used. */ public class StructField { diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java similarity index 86% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java index 33a42f4b16265..a4b501efd9a10 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/StructType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructType.java @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; import java.util.Arrays; -import java.util.List; /** * The data type representing Rows. * A StructType object comprises an array of StructFields. * * To create an {@link StructType}, - * {@link org.apache.spark.sql.api.java.types.DataType#createStructType(java.util.List)} or - * {@link org.apache.spark.sql.api.java.types.DataType#createStructType(StructField[])} + * {@link DataType#createStructType(java.util.List)} or + * {@link DataType#createStructType(StructField[])} * should be used. */ public class StructType extends DataType { diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/TimestampType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/api/java/types/TimestampType.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java index 65295779f71ec..06d44c731cdfe 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/TimestampType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/TimestampType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.api.java.types; +package org.apache.spark.sql.api.java; /** * The data type representing java.sql.Timestamp values. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package-info.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/package-info.java similarity index 95% rename from sql/core/src/main/scala/org/apache/spark/sql/package-info.java rename to sql/core/src/main/java/org/apache/spark/sql/api/java/package-info.java index 53603614518f5..67007a9f0d1a3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package-info.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/package-info.java @@ -18,4 +18,4 @@ /** * Allows the execution of relational queries, including those expressed in SQL using Spark. */ -package org.apache.spark.sql; \ No newline at end of file +package org.apache.spark.sql.api.java; diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java deleted file mode 100644 index f169ac65e226f..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/types/package-info.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -/** - * Allows users to get and create Spark SQL data types. - */ -package org.apache.spark.sql.api.java.types; diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index c1c18a0cd0ed6..809dd038f94aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -23,9 +23,8 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} -import org.apache.spark.sql.api.java.types.{StructType => JStructType} import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql._ +import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} @@ -104,9 +103,9 @@ class JavaSQLContext(val sqlContext: SQLContext) { * provided schema. Otherwise, there will be runtime exception. */ @DeveloperApi - def applySchema(rowRDD: JavaRDD[Row], schema: JStructType): JavaSchemaRDD = { + def applySchema(rowRDD: JavaRDD[Row], schema: StructType): JavaSchemaRDD = { val scalaRowRDD = rowRDD.rdd.map(r => r.row) - val scalaSchema = asScalaDataType(schema).asInstanceOf[StructType] + val scalaSchema = asScalaDataType(schema).asInstanceOf[SStructType] val logicalPlan = SparkLogicalPlan(ExistingRdd(scalaSchema.toAttributes, scalaRowRDD))(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) @@ -133,7 +132,7 @@ class JavaSQLContext(val sqlContext: SQLContext) { * returning the result as a JavaSchemaRDD. */ @Experimental - def jsonFile(path: String, schema: JStructType): JavaSchemaRDD = + def jsonFile(path: String, schema: StructType): JavaSchemaRDD = jsonRDD(sqlContext.sparkContext.textFile(path), schema) /** @@ -155,10 +154,10 @@ class JavaSQLContext(val sqlContext: SQLContext) { * returning the result as a JavaSchemaRDD. */ @Experimental - def jsonRDD(json: JavaRDD[String], schema: JStructType): JavaSchemaRDD = { + def jsonRDD(json: JavaRDD[String], schema: StructType): JavaSchemaRDD = { val appliedScalaSchema = Option(asScalaDataType(schema)).getOrElse( - JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[StructType] + JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[SStructType] val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) val logicalPlan = SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) @@ -181,22 +180,37 @@ class JavaSQLContext(val sqlContext: SQLContext) { val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") fields.map { property => val (dataType, nullable) = property.getPropertyType match { - case c: Class[_] if c == classOf[java.lang.String] => (StringType, true) - case c: Class[_] if c == java.lang.Short.TYPE => (ShortType, false) - case c: Class[_] if c == java.lang.Integer.TYPE => (IntegerType, false) - case c: Class[_] if c == java.lang.Long.TYPE => (LongType, false) - case c: Class[_] if c == java.lang.Double.TYPE => (DoubleType, false) - case c: Class[_] if c == java.lang.Byte.TYPE => (ByteType, false) - case c: Class[_] if c == java.lang.Float.TYPE => (FloatType, false) - case c: Class[_] if c == java.lang.Boolean.TYPE => (BooleanType, false) - - case c: Class[_] if c == classOf[java.lang.Short] => (ShortType, true) - case c: Class[_] if c == classOf[java.lang.Integer] => (IntegerType, true) - case c: Class[_] if c == classOf[java.lang.Long] => (LongType, true) - case c: Class[_] if c == classOf[java.lang.Double] => (DoubleType, true) - case c: Class[_] if c == classOf[java.lang.Byte] => (ByteType, true) - case c: Class[_] if c == classOf[java.lang.Float] => (FloatType, true) - case c: Class[_] if c == classOf[java.lang.Boolean] => (BooleanType, true) + case c: Class[_] if c == classOf[java.lang.String] => + (org.apache.spark.sql.StringType, true) + case c: Class[_] if c == java.lang.Short.TYPE => + (org.apache.spark.sql.ShortType, false) + case c: Class[_] if c == java.lang.Integer.TYPE => + (org.apache.spark.sql.IntegerType, false) + case c: Class[_] if c == java.lang.Long.TYPE => + (org.apache.spark.sql.LongType, false) + case c: Class[_] if c == java.lang.Double.TYPE => + (org.apache.spark.sql.DoubleType, false) + case c: Class[_] if c == java.lang.Byte.TYPE => + (org.apache.spark.sql.ByteType, false) + case c: Class[_] if c == java.lang.Float.TYPE => + (org.apache.spark.sql.FloatType, false) + case c: Class[_] if c == java.lang.Boolean.TYPE => + (org.apache.spark.sql.BooleanType, false) + + case c: Class[_] if c == classOf[java.lang.Short] => + (org.apache.spark.sql.ShortType, true) + case c: Class[_] if c == classOf[java.lang.Integer] => + (org.apache.spark.sql.IntegerType, true) + case c: Class[_] if c == classOf[java.lang.Long] => + (org.apache.spark.sql.LongType, true) + case c: Class[_] if c == classOf[java.lang.Double] => + (org.apache.spark.sql.DoubleType, true) + case c: Class[_] if c == classOf[java.lang.Byte] => + (org.apache.spark.sql.ByteType, true) + case c: Class[_] if c == classOf[java.lang.Float] => + (org.apache.spark.sql.FloatType, true) + case c: Class[_] if c == classOf[java.lang.Boolean] => + (org.apache.spark.sql.BooleanType, true) } AttributeReference(property.getName, dataType, nullable)() } 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 824574149858c..4d799b4038fdd 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 @@ -22,7 +22,6 @@ 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} -import org.apache.spark.sql.api.java.types.StructType import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index d1aa3c8d53757..77353f4eb0227 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.types.util import org.apache.spark.sql._ -import org.apache.spark.sql.api.java.types.{DataType => JDataType, StructField => JStructField} +import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField} import scala.collection.JavaConverters._ @@ -74,37 +74,37 @@ protected[sql] object DataTypeConversions { * Returns the equivalent DataType in Scala for the given DataType in Java. */ def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match { - case stringType: org.apache.spark.sql.api.java.types.StringType => + case stringType: org.apache.spark.sql.api.java.StringType => StringType - case binaryType: org.apache.spark.sql.api.java.types.BinaryType => + case binaryType: org.apache.spark.sql.api.java.BinaryType => BinaryType - case booleanType: org.apache.spark.sql.api.java.types.BooleanType => + case booleanType: org.apache.spark.sql.api.java.BooleanType => BooleanType - case timestampType: org.apache.spark.sql.api.java.types.TimestampType => + case timestampType: org.apache.spark.sql.api.java.TimestampType => TimestampType - case decimalType: org.apache.spark.sql.api.java.types.DecimalType => + case decimalType: org.apache.spark.sql.api.java.DecimalType => DecimalType - case doubleType: org.apache.spark.sql.api.java.types.DoubleType => + case doubleType: org.apache.spark.sql.api.java.DoubleType => DoubleType - case floatType: org.apache.spark.sql.api.java.types.FloatType => + case floatType: org.apache.spark.sql.api.java.FloatType => FloatType - case byteType: org.apache.spark.sql.api.java.types.ByteType => + case byteType: org.apache.spark.sql.api.java.ByteType => ByteType - case integerType: org.apache.spark.sql.api.java.types.IntegerType => + case integerType: org.apache.spark.sql.api.java.IntegerType => IntegerType - case longType: org.apache.spark.sql.api.java.types.LongType => + case longType: org.apache.spark.sql.api.java.LongType => LongType - case shortType: org.apache.spark.sql.api.java.types.ShortType => + case shortType: org.apache.spark.sql.api.java.ShortType => ShortType - case arrayType: org.apache.spark.sql.api.java.types.ArrayType => + case arrayType: org.apache.spark.sql.api.java.ArrayType => ArrayType(asScalaDataType(arrayType.getElementType), arrayType.isContainsNull) - case mapType: org.apache.spark.sql.api.java.types.MapType => + case mapType: org.apache.spark.sql.api.java.MapType => MapType( asScalaDataType(mapType.getKeyType), asScalaDataType(mapType.getValueType), mapType.isValueContainsNull) - case structType: org.apache.spark.sql.api.java.types.StructType => + case structType: org.apache.spark.sql.api.java.StructType => StructType(structType.getFields.map(asScalaStructField)) } } diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index 8ee4591105010..3c92906d82864 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -28,9 +28,6 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.sql.api.java.types.DataType; -import org.apache.spark.sql.api.java.types.StructField; -import org.apache.spark.sql.api.java.types.StructType; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java index 96a503962f7d1..d099a48a1f4b6 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java @@ -24,8 +24,6 @@ import org.junit.Test; import org.apache.spark.sql.types.util.DataTypeConversions; -import org.apache.spark.sql.api.java.types.DataType; -import org.apache.spark.sql.api.java.types.StructField; public class JavaSideDataTypeConversionSuite { public void checkDataType(DataType javaDataType) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala index 46de6fe239228..ff1debff0f8c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.api.java import org.apache.spark.sql.types.util.DataTypeConversions import org.scalatest.FunSuite -import org.apache.spark.sql._ +import org.apache.spark.sql.{DataType => SDataType, StructField => SStructField} +import org.apache.spark.sql.{StructType => SStructType} import DataTypeConversions._ class ScalaSideDataTypeConversionSuite extends FunSuite { - def checkDataType(scalaDataType: DataType) { + def checkDataType(scalaDataType: SDataType) { val javaDataType = asJavaDataType(scalaDataType) val actual = asScalaDataType(javaDataType) assert(scalaDataType === actual, s"Converted data type ${actual} " + @@ -34,48 +35,52 @@ class ScalaSideDataTypeConversionSuite extends FunSuite { test("convert data types") { // Simple DataTypes. - checkDataType(StringType) - checkDataType(BinaryType) - checkDataType(BooleanType) - checkDataType(TimestampType) - checkDataType(DecimalType) - checkDataType(DoubleType) - checkDataType(FloatType) - checkDataType(ByteType) - checkDataType(IntegerType) - checkDataType(LongType) - checkDataType(ShortType) + checkDataType(org.apache.spark.sql.StringType) + checkDataType(org.apache.spark.sql.BinaryType) + checkDataType(org.apache.spark.sql.BooleanType) + checkDataType(org.apache.spark.sql.TimestampType) + checkDataType(org.apache.spark.sql.DecimalType) + checkDataType(org.apache.spark.sql.DoubleType) + checkDataType(org.apache.spark.sql.FloatType) + checkDataType(org.apache.spark.sql.ByteType) + checkDataType(org.apache.spark.sql.IntegerType) + checkDataType(org.apache.spark.sql.LongType) + checkDataType(org.apache.spark.sql.ShortType) // Simple ArrayType. - val simpleScalaArrayType = ArrayType(StringType, true) + val simpleScalaArrayType = + org.apache.spark.sql.ArrayType(org.apache.spark.sql.StringType, true) checkDataType(simpleScalaArrayType) // Simple MapType. - val simpleScalaMapType = MapType(StringType, LongType) + val simpleScalaMapType = + org.apache.spark.sql.MapType(org.apache.spark.sql.StringType, org.apache.spark.sql.LongType) checkDataType(simpleScalaMapType) // Simple StructType. - val simpleScalaStructType = StructType( - StructField("a", DecimalType, false) :: - StructField("b", BooleanType, true) :: - StructField("c", LongType, true) :: - StructField("d", BinaryType, false) :: Nil) + val simpleScalaStructType = SStructType( + SStructField("a", org.apache.spark.sql.DecimalType, false) :: + SStructField("b", org.apache.spark.sql.BooleanType, true) :: + SStructField("c", org.apache.spark.sql.LongType, true) :: + SStructField("d", org.apache.spark.sql.BinaryType, false) :: Nil) checkDataType(simpleScalaStructType) // Complex StructType. - val complexScalaStructType = StructType( - StructField("simpleArray", simpleScalaArrayType, true) :: - StructField("simpleMap", simpleScalaMapType, true) :: - StructField("simpleStruct", simpleScalaStructType, true) :: - StructField("boolean", BooleanType, false) :: Nil) + val complexScalaStructType = SStructType( + SStructField("simpleArray", simpleScalaArrayType, true) :: + SStructField("simpleMap", simpleScalaMapType, true) :: + SStructField("simpleStruct", simpleScalaStructType, true) :: + SStructField("boolean", org.apache.spark.sql.BooleanType, false) :: Nil) checkDataType(complexScalaStructType) // Complex ArrayType. - val complexScalaArrayType = ArrayType(complexScalaStructType, true) + val complexScalaArrayType = + org.apache.spark.sql.ArrayType(complexScalaStructType, true) checkDataType(complexScalaArrayType) // Complex MapType. - val complexScalaMapType = MapType(complexScalaStructType, complexScalaArrayType, false) + val complexScalaMapType = + org.apache.spark.sql.MapType(complexScalaStructType, complexScalaArrayType, false) checkDataType(complexScalaMapType) } } From 4415722e9199d04c2c18bfbd29113ebc40f732f5 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 1 Aug 2014 11:27:12 -0700 Subject: [PATCH 0185/1492] [SQL][SPARK-2212]Hash Outer Join This patch is to support the hash based outer join. Currently, outer join for big relations are resort to `BoradcastNestedLoopJoin`, which is super slow. This PR will create 2 hash tables for both relations in the same partition, which greatly reduce the table scans. Here is the testing code that I used: ``` package org.apache.spark.sql.hive import org.apache.spark.SparkContext import org.apache.spark.SparkConf import org.apache.spark.sql._ case class Record(key: String, value: String) object JoinTablePrepare extends App { import TestHive2._ val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"${i % 828193}", s"val_$i"))) runSqlHive("SHOW TABLES") runSqlHive("DROP TABLE if exists a") runSqlHive("DROP TABLE if exists b") runSqlHive("DROP TABLE if exists result") rdd.registerAsTable("records") runSqlHive("""CREATE TABLE a (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) runSqlHive("""CREATE TABLE b (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) runSqlHive("""CREATE TABLE result (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) hql(s"""from records | insert into table a | select key, value """.stripMargin) hql(s"""from records | insert into table b select key + 100000, value """.stripMargin) } object JoinTablePerformanceTest extends App { import TestHive2._ hql("SHOW TABLES") hql("set spark.sql.shuffle.partitions=20") val leftOuterJoin = "insert overwrite table result select a.key, b.value from a left outer join b on a.key=b.key" val rightOuterJoin = "insert overwrite table result select a.key, b.value from a right outer join b on a.key=b.key" val fullOuterJoin = "insert overwrite table result select a.key, b.value from a full outer join b on a.key=b.key" val results = ("LeftOuterJoin", benchmark(leftOuterJoin)) :: ("LeftOuterJoin", benchmark(leftOuterJoin)) :: ("RightOuterJoin", benchmark(rightOuterJoin)) :: ("RightOuterJoin", benchmark(rightOuterJoin)) :: ("FullOuterJoin", benchmark(fullOuterJoin)) :: ("FullOuterJoin", benchmark(fullOuterJoin)) :: Nil val explains = hql(s"explain $leftOuterJoin").collect ++ hql(s"explain $rightOuterJoin").collect ++ hql(s"explain $fullOuterJoin").collect println(explains.mkString(",\n")) results.foreach { case (prompt, result) => { println(s"$prompt: took ${result._1} ms (${result._2} records)") } } def benchmark(cmd: String) = { val begin = System.currentTimeMillis() val result = hql(cmd) val end = System.currentTimeMillis() val count = hql("select count(1) from result").collect.mkString("") ((end - begin), count) } } ``` And the result as shown below: ``` [Physical execution plan:], [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true], [ Project [key#95,value#98]], [ HashOuterJoin [key#95], [key#97], LeftOuter, None], [ Exchange (HashPartitioning [key#95], 20)], [ HiveTableScan [key#95], (MetastoreRelation default, a, None), None], [ Exchange (HashPartitioning [key#97], 20)], [ HiveTableScan [key#97,value#98], (MetastoreRelation default, b, None), None], [Physical execution plan:], [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true], [ Project [key#102,value#105]], [ HashOuterJoin [key#102], [key#104], RightOuter, None], [ Exchange (HashPartitioning [key#102], 20)], [ HiveTableScan [key#102], (MetastoreRelation default, a, None), None], [ Exchange (HashPartitioning [key#104], 20)], [ HiveTableScan [key#104,value#105], (MetastoreRelation default, b, None), None], [Physical execution plan:], [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true], [ Project [key#109,value#112]], [ HashOuterJoin [key#109], [key#111], FullOuter, None], [ Exchange (HashPartitioning [key#109], 20)], [ HiveTableScan [key#109], (MetastoreRelation default, a, None), None], [ Exchange (HashPartitioning [key#111], 20)], [ HiveTableScan [key#111,value#112], (MetastoreRelation default, b, None), None] LeftOuterJoin: took 16072 ms ([3000000] records) LeftOuterJoin: took 14394 ms ([3000000] records) RightOuterJoin: took 14802 ms ([3000000] records) RightOuterJoin: took 14747 ms ([3000000] records) FullOuterJoin: took 17715 ms ([6000000] records) FullOuterJoin: took 17629 ms ([6000000] records) ``` Without this PR, the benchmark will run seems never end. Author: Cheng Hao Closes #1147 from chenghao-intel/hash_based_outer_join and squashes the following commits: 65c599e [Cheng Hao] Fix issues with the community comments 72b1394 [Cheng Hao] Fix bug of stale value in joinedRow 55baef7 [Cheng Hao] Add HashOuterJoin --- .../spark/sql/execution/SparkStrategies.scala | 4 + .../apache/spark/sql/execution/joins.scala | 183 +++++++++++++++++- .../org/apache/spark/sql/JoinSuite.scala | 138 ++++++++++++- 3 files changed, 319 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index d57b6eaf40b09..8bec015c7b465 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -94,6 +94,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { leftKeys, rightKeys, buildSide, planLater(left), planLater(right)) condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) => + execution.HashOuterJoin( + leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil + case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index b068579db75cd..82f0a74b630bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -72,7 +72,7 @@ trait HashJoin { while (buildIter.hasNext) { currentRow = buildIter.next() val rowKey = buildSideKeyGenerator(currentRow) - if(!rowKey.anyNull) { + if (!rowKey.anyNull) { val existingMatchList = hashTable.get(rowKey) val matchList = if (existingMatchList == null) { val newMatchList = new ArrayBuffer[Row]() @@ -136,6 +136,185 @@ trait HashJoin { } } +/** + * Constant Value for Binary Join Node + */ +object HashOuterJoin { + val DUMMY_LIST = Seq[Row](null) + val EMPTY_LIST = Seq[Row]() +} + +/** + * :: DeveloperApi :: + * Performs a hash based outer join for two child relations by shuffling the data using + * the join keys. This operator requires loading the associated partition in both side into memory. + */ +@DeveloperApi +case class HashOuterJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + def output = left.output ++ right.output + + // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala + // iterator for performance purpose. + + private[this] def leftOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val rightNullRow = new GenericRow(right.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + leftIter.iterator.flatMap { l => + joinedRow.withLeft(l) + var matched = false + (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => + matched = true + joinedRow.copy + } else { + Nil + }) ++ HashOuterJoin.DUMMY_LIST.filter(_ => !matched).map( _ => { + // HashOuterJoin.DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all of the + // records in right side. + // If we didn't get any proper row, then append a single row with empty right + joinedRow.withRight(rightNullRow).copy + }) + } + } + + private[this] def rightOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val leftNullRow = new GenericRow(left.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + rightIter.iterator.flatMap { r => + joinedRow.withRight(r) + var matched = false + (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => + matched = true + joinedRow.copy + } else { + Nil + }) ++ HashOuterJoin.DUMMY_LIST.filter(_ => !matched).map( _ => { + // HashOuterJoin.DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all of the + // records in left side. + // If we didn't get any proper row, then append a single row with empty left. + joinedRow.withLeft(leftNullRow).copy + }) + } + } + + private[this] def fullOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val leftNullRow = new GenericRow(left.output.length) + val rightNullRow = new GenericRow(right.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + if (!key.anyNull) { + // Store the positions of records in right, if one of its associated row satisfy + // the join condition. + val rightMatchedSet = scala.collection.mutable.Set[Int]() + leftIter.iterator.flatMap[Row] { l => + joinedRow.withLeft(l) + var matched = false + rightIter.zipWithIndex.collect { + // 1. For those matched (satisfy the join condition) records with both sides filled, + // append them directly + + case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> { + matched = true + // if the row satisfy the join condition, add its index into the matched set + rightMatchedSet.add(idx) + joinedRow.copy + } + } ++ HashOuterJoin.DUMMY_LIST.filter(_ => !matched).map( _ => { + // 2. For those unmatched records in left, append additional records with empty right. + + // HashOuterJoin.DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all + // of the records in right side. + // If we didn't get any proper row, then append a single row with empty right. + joinedRow.withRight(rightNullRow).copy + }) + } ++ rightIter.zipWithIndex.collect { + // 3. For those unmatched records in right, append additional records with empty left. + + // Re-visiting the records in right, and append additional row with empty left, if its not + // in the matched set. + case (r, idx) if (!rightMatchedSet.contains(idx)) => { + joinedRow(leftNullRow, r).copy + } + } + } else { + leftIter.iterator.map[Row] { l => + joinedRow(l, rightNullRow).copy + } ++ rightIter.iterator.map[Row] { r => + joinedRow(leftNullRow, r).copy + } + } + } + + private[this] def buildHashTable( + iter: Iterator[Row], keyGenerator: Projection): Map[Row, ArrayBuffer[Row]] = { + // TODO: Use Spark's HashMap implementation. + val hashTable = scala.collection.mutable.Map[Row, ArrayBuffer[Row]]() + while (iter.hasNext) { + val currentRow = iter.next() + val rowKey = keyGenerator(currentRow) + + val existingMatchList = hashTable.getOrElseUpdate(rowKey, {new ArrayBuffer[Row]()}) + existingMatchList += currentRow.copy() + } + + hashTable.toMap[Row, ArrayBuffer[Row]] + } + + def execute() = { + left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => + // TODO this probably can be replaced by external sort (sort merged join?) + // Build HashMap for current partition in left relation + val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) + // Build HashMap for current partition in right relation + val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + joinType match { + case LeftOuter => leftHashTable.keysIterator.flatMap { key => + leftOuterIterator(key, leftHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST), + rightHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST)) + } + case RightOuter => rightHashTable.keysIterator.flatMap { key => + rightOuterIterator(key, leftHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST), + rightHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST)) + } + case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => + fullOuterIterator(key, + leftHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST), + rightHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST)) + } + case x => throw new Exception(s"Need to add implementation for $x") + } + } + } +} + /** * :: DeveloperApi :: * Performs an inner hash join of two child relations by first shuffling the data using the join @@ -189,7 +368,7 @@ case class LeftSemiJoinHash( while (buildIter.hasNext) { currentRow = buildIter.next() val rowKey = buildSideKeyGenerator(currentRow) - if(!rowKey.anyNull) { + if (!rowKey.anyNull) { val keyExists = hashSet.contains(rowKey) if (!keyExists) { hashSet.add(rowKey) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 025c396ef0629..037890682f7b1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -17,11 +17,17 @@ package org.apache.spark.sql +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner} +import org.apache.spark.sql.catalyst.plans.JoinType +import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner, LeftSemi} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ -class JoinSuite extends QueryTest { +class JoinSuite extends QueryTest with BeforeAndAfterEach { // Ensures tables are loaded. TestData @@ -34,6 +40,56 @@ class JoinSuite extends QueryTest { assert(planned.size === 1) } + test("join operator selection") { + def assertJoin(sqlString: String, c: Class[_]): Any = { + val rdd = sql(sqlString) + val physical = rdd.queryExecution.sparkPlan + val operators = physical.collect { + case j: ShuffledHashJoin => j + case j: HashOuterJoin => j + case j: LeftSemiJoinHash => j + case j: BroadcastHashJoin => j + case j: LeftSemiJoinBNL => j + case j: CartesianProduct => j + case j: BroadcastNestedLoopJoin => j + } + + assert(operators.size === 1) + if (operators(0).getClass() != c) { + fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") + } + } + + val cases1 = Seq( + ("SELECT * FROM testData left semi join testData2 ON key = a", classOf[LeftSemiJoinHash]), + ("SELECT * FROM testData left semi join testData2", classOf[LeftSemiJoinBNL]), + ("SELECT * FROM testData join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData left join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData right join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData full outer join testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData left join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData right join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData full outer join testData2 where key=2", classOf[CartesianProduct]), + ("SELECT * FROM testData join testData2 where key>a", classOf[CartesianProduct]), + ("SELECT * FROM testData full outer join testData2 where key>a", classOf[CartesianProduct]), + ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData left join testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData right join testData2 ON key = a where key=2", + classOf[HashOuterJoin]), + ("SELECT * FROM testData right join testData2 ON key = a and key=2", + classOf[HashOuterJoin]), + ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]) + // TODO add BroadcastNestedLoopJoin + ) + cases1.foreach { c => assertJoin(c._1, c._2) } + } + test("multiple-key equi-join is hash-join") { val x = testData2.as('x) val y = testData2.as('y) @@ -114,6 +170,33 @@ class JoinSuite extends QueryTest { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) + + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'n > 1)), + (1, "A", null, null) :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) + + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'N > 1)), + (1, "A", null, null) :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) + + checkAnswer( + upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'l > 'L)), + (1, "A", 1, "a") :: + (2, "B", 2, "b") :: + (3, "C", 3, "c") :: + (4, "D", 4, "d") :: + (5, "E", null, null) :: + (6, "F", null, null) :: Nil) } test("right outer join") { @@ -125,11 +208,38 @@ class JoinSuite extends QueryTest { (4, "d", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'n > 1)), + (null, null, 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'N > 1)), + (null, null, 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + checkAnswer( + lowerCaseData.join(upperCaseData, RightOuter, Some('n === 'N && 'l > 'L)), + (1, "a", 1, "A") :: + (2, "b", 2, "B") :: + (3, "c", 3, "C") :: + (4, "d", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) } test("full outer join") { - val left = upperCaseData.where('N <= 4).as('left) - val right = upperCaseData.where('N >= 3).as('right) + upperCaseData.where('N <= 4).registerAsTable("left") + upperCaseData.where('N >= 3).registerAsTable("right") + + val left = UnresolvedRelation(None, "left", None) + val right = UnresolvedRelation(None, "right", None) checkAnswer( left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)), @@ -139,5 +249,25 @@ class JoinSuite extends QueryTest { (4, "D", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) + + checkAnswer( + left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("left.N".attr !== 3))), + (1, "A", null, null) :: + (2, "B", null, null) :: + (3, "C", null, null) :: + (null, null, 3, "C") :: + (4, "D", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) + + checkAnswer( + left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("right.N".attr !== 3))), + (1, "A", null, null) :: + (2, "B", null, null) :: + (3, "C", null, null) :: + (null, null, 3, "C") :: + (4, "D", 4, "D") :: + (null, null, 5, "E") :: + (null, null, 6, "F") :: Nil) } } From 580c7011cab6bc93294b6486e778557216bedb10 Mon Sep 17 00:00:00 2001 From: chutium Date: Fri, 1 Aug 2014 11:31:44 -0700 Subject: [PATCH 0186/1492] [SPARK-2729] [SQL] Forgot to match Timestamp type in ColumnBuilder just a match forgot, found after SPARK-2710 , TimestampType can be used by a SchemaRDD generated from JDBC ResultSet Author: chutium Closes #1636 from chutium/SPARK-2729 and squashes the following commits: 71af77a [chutium] [SPARK-2729] [SQL] added Timestamp in NullableColumnAccessorSuite 39cf9f8 [chutium] [SPARK-2729] add Timestamp Type into ColumnBuilder TestSuite, ref. #1636 ab6ff97 [chutium] [SPARK-2729] Forgot to match Timestamp type in ColumnBuilder --- .../scala/org/apache/spark/sql/columnar/ColumnBuilder.scala | 1 + .../apache/spark/sql/columnar/NullableColumnAccessorSuite.scala | 2 +- .../apache/spark/sql/columnar/NullableColumnBuilderSuite.scala | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 74f5630fbddf1..c416a745739b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -154,6 +154,7 @@ private[sql] object ColumnBuilder { case STRING.typeId => new StringColumnBuilder case BINARY.typeId => new BinaryColumnBuilder case GENERIC.typeId => new GenericColumnBuilder + case TIMESTAMP.typeId => new TimestampColumnBuilder }).asInstanceOf[ColumnBuilder] builder.initialize(initialSize, columnName, useCompression) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 35ab14cbc353d..3baa6f8ec0c83 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -41,7 +41,7 @@ object TestNullableColumnAccessor { class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { + Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { testNullableColumnAccessor(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index d8898527baa39..dc813fe146c47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -37,7 +37,7 @@ object TestNullableColumnBuilder { class NullableColumnBuilderSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { + Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { testNullableColumnBuilder(_) } From c0b47bada3c9f0e9e0f14ab41ffb91012a357211 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 1 Aug 2014 11:42:05 -0700 Subject: [PATCH 0187/1492] [SPARK-2767] [SQL] SparkSQL CLI doens't output error message if query failed. Author: Cheng Hao Closes #1686 from chenghao-intel/spark_sql_cli and squashes the following commits: eb664cc [Cheng Hao] Output detailed failure message in console 93b0382 [Cheng Hao] Fix Bug of no output in cli if exception thrown internally --- .../spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala | 4 +++- .../spark/sql/hive/thriftserver/SparkSQLDriver.scala | 3 +-- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 7 ++++--- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 27268ecb923e9..cb17d7ce58ea0 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -288,8 +288,10 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { out.println(cmd) } - ret = driver.run(cmd).getResponseCode + val rc = driver.run(cmd) + ret = rc.getResponseCode if (ret != 0) { + console.printError(rc.getErrorMessage()) driver.close() return ret } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 5202aa9903e03..a56b19a4bcda0 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -53,10 +53,9 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo } override def run(command: String): CommandProcessorResponse = { - val execution = context.executePlan(context.hql(command).logicalPlan) - // TODO unify the error code try { + val execution = context.executePlan(context.hql(command).logicalPlan) hiveResponse = execution.stringResult() tableSchema = getResultSetSchema(execution) new CommandProcessorResponse(0) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 27b444daba2d4..7e3b8727bebed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -131,12 +131,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @transient protected[hive] lazy val sessionState = { val ss = new SessionState(hiveconf) set(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. + + ss.err = new PrintStream(outputBuffer, true, "UTF-8") + ss.out = new PrintStream(outputBuffer, true, "UTF-8") + ss } - sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") - sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") - override def set(key: String, value: String): Unit = { super.set(key, value) runSqlHive(s"SET $key=$value") From c82fe4781cd0356bcfdd25c7eadf1da624bb2228 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Fri, 1 Aug 2014 11:46:13 -0700 Subject: [PATCH 0188/1492] [SQL] Documentation: Explain cacheTable command add the `cacheTable` specification Author: CrazyJvm Closes #1681 from CrazyJvm/sql-programming-guide-cache and squashes the following commits: 0a231e0 [CrazyJvm] grammar fixes a04020e [CrazyJvm] modify title to Cached tables 18b6594 [CrazyJvm] fix format 2cbbf58 [CrazyJvm] add cacheTable guide --- docs/sql-programming-guide.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a047d32b6ee6c..7261badd411a9 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -769,3 +769,13 @@ To start the Spark SQL CLI, run the following in the Spark directory: Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may run `./bin/spark-sql --help` for a complete list of all available options. + +# Cached tables + +Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. +Then Spark SQL will scan only required columns and will automatically tune compression to minimize +memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. + +Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in +in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to +cache tables. From eb5bdcaf6c7834558cb76b7132f68b8d94230356 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Fri, 1 Aug 2014 12:04:04 -0700 Subject: [PATCH 0189/1492] [SPARK-695] In DAGScheduler's getPreferredLocs, track set of visited partitions. getPreferredLocs traverses a dependency graph of partitions using depth first search. Given a complex dependency graph, the old implementation may explore a set of paths in the graph that is exponential in the number of nodes. By maintaining a set of visited nodes the new implementation avoids revisiting nodes, preventing exponential blowup. Some comment and whitespace cleanups are also included. Author: Aaron Staple Closes #1362 from staple/SPARK-695 and squashes the following commits: ecea0f3 [Aaron Staple] address review comments 751c661 [Aaron Staple] [SPARK-695] Add a unit test. 5adf326 [Aaron Staple] Replace getPreferredLocsInternal's HashMap argument with a simpler HashSet. 58e37d0 [Aaron Staple] Replace comment documenting NarrowDependency. 6751ced [Aaron Staple] Revert "Remove unused variable." 04c7097 [Aaron Staple] Fix indentation. 0030884 [Aaron Staple] Remove unused variable. 33f67c6 [Aaron Staple] Clarify comment. 4e42b46 [Aaron Staple] Remove apparently incorrect comment describing NarrowDependency. 65c2d3d [Aaron Staple] [SPARK-695] In DAGScheduler's getPreferredLocs, track set of visited partitions. --- .../scala/org/apache/spark/Dependency.scala | 4 ++-- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../org/apache/spark/rdd/CoalescedRDD.scala | 4 ++-- .../apache/spark/scheduler/DAGScheduler.scala | 18 +++++++++++++++++- .../spark/scheduler/DAGSchedulerSuite.scala | 16 +++++++++++++++- 5 files changed, 37 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3935c8772252e..ab2594cfc02eb 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -34,8 +34,8 @@ abstract class Dependency[T] extends Serializable { /** * :: DeveloperApi :: - * Base class for dependencies where each partition of the parent RDD is used by at most one - * partition of the child RDD. Narrow dependencies allow for pipelined execution. + * Base class for dependencies where each partition of the child RDD depends on a small number + * of partitions of the parent RDD. Narrow dependencies allow for pipelined execution. */ @DeveloperApi abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5f75c1dd2cb68..368835a867493 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -458,7 +458,7 @@ class SparkContext(config: SparkConf) extends Logging { /** Distribute a local Scala collection to form an RDD, with one or more * location preferences (hostnames of Spark nodes) for each object. * Create a new partition for each collection item. */ - def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = { + def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = { val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index e7221e3032c11..11ebafbf6d457 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -49,8 +49,8 @@ private[spark] case class CoalescedRDDPartition( } /** - * Computes how many of the parents partitions have getPreferredLocation - * as one of their preferredLocations + * Computes the fraction of the parents' partitions containing preferredLocation within + * their getPreferredLocs. * @return locality of this coalesced partition between 0 and 1 */ def localFraction: Double = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c7e3d7c5f8530..5110785de357c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1148,6 +1148,22 @@ class DAGScheduler( */ private[spark] def getPreferredLocs(rdd: RDD[_], partition: Int): Seq[TaskLocation] = synchronized { + getPreferredLocsInternal(rdd, partition, new HashSet) + } + + /** Recursive implementation for getPreferredLocs. */ + private def getPreferredLocsInternal( + rdd: RDD[_], + partition: Int, + visited: HashSet[(RDD[_],Int)]) + : Seq[TaskLocation] = + { + // If the partition has already been visited, no need to re-visit. + // This avoids exponential path exploration. SPARK-695 + if (!visited.add((rdd,partition))) { + // Nil has already been returned for previously visited partitions. + return Nil + } // If the partition is cached, return the cache locations val cached = getCacheLocs(rdd)(partition) if (!cached.isEmpty) { @@ -1164,7 +1180,7 @@ class DAGScheduler( rdd.dependencies.foreach { case n: NarrowDependency[_] => for (inPart <- n.getParents(partition)) { - val locs = getPreferredLocs(n.rdd, inPart) + val locs = getPreferredLocsInternal(n.rdd, inPart, visited) if (locs != Nil) { return locs } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 0ce13d015df05..36e238b4c9434 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -23,6 +23,8 @@ import scala.language.reflectiveCalls import akka.actor._ import akka.testkit.{ImplicitSender, TestKit, TestActorRef} import org.scalatest.{BeforeAndAfter, FunSuiteLike} +import org.scalatest.concurrent.Timeouts +import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.rdd.RDD @@ -64,7 +66,7 @@ class MyRDD( class DAGSchedulerSuiteDummyException extends Exception class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuiteLike - with ImplicitSender with BeforeAndAfter with LocalSparkContext { + with ImplicitSender with BeforeAndAfter with LocalSparkContext with Timeouts { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -294,6 +296,18 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("avoid exponential blowup when getting preferred locs list") { + // Build up a complex dependency graph with repeated zip operations, without preferred locations. + var rdd: RDD[_] = new MyRDD(sc, 1, Nil) + (1 to 30).foreach(_ => rdd = rdd.zip(rdd)) + // getPreferredLocs runs quickly, indicating that exponential graph traversal is avoided. + failAfter(10 seconds) { + val preferredLocs = scheduler.getPreferredLocs(rdd,0) + // No preferred locations are returned. + assert(preferredLocs.length === 0) + } + } + test("unserializable task") { val unserializableRdd = new MyRDD(sc, 1, Nil) { class UnserializableClass From baf9ce1a4ecb7acf5accf7a7029f29604b4360c2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 1 Aug 2014 12:12:30 -0700 Subject: [PATCH 0190/1492] [SPARK-2490] Change recursive visiting on RDD dependencies to iterative approach When performing some transformations on RDDs after many iterations, the dependencies of RDDs could be very long. It can easily cause StackOverflowError when recursively visiting these dependencies in Spark core. For example: var rdd = sc.makeRDD(Array(1)) for (i <- 1 to 1000) { rdd = rdd.coalesce(1).cache() rdd.collect() } This PR changes recursive visiting on rdd's dependencies to iterative approach to avoid StackOverflowError. In addition to the recursive visiting, since the Java serializer has a known [bug](http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4152790) that causes StackOverflowError too when serializing/deserializing a large graph of objects. So applying this PR only solves part of the problem. Using KryoSerializer to replace Java serializer might be helpful. However, since KryoSerializer is not supported for `spark.closure.serializer` now, I can not test if KryoSerializer can solve Java serializer's problem completely. Author: Liang-Chi Hsieh Closes #1418 from viirya/remove_recursive_visit and squashes the following commits: 6b2c615 [Liang-Chi Hsieh] change function name; comply with code style. 5f072a7 [Liang-Chi Hsieh] add comments to explain Stack usage. 8742dbb [Liang-Chi Hsieh] comply with code style. 900538b [Liang-Chi Hsieh] change recursive visiting on rdd's dependencies to iterative approach to avoid stackoverflowerror. --- .../apache/spark/scheduler/DAGScheduler.scala | 83 +++++++++++++++++-- 1 file changed, 75 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5110785de357c..d87c3048985fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -21,7 +21,7 @@ import java.io.NotSerializableException import java.util.Properties import java.util.concurrent.atomic.AtomicInteger -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map, Stack} import scala.concurrent.Await import scala.concurrent.duration._ import scala.language.postfixOps @@ -211,11 +211,15 @@ class DAGScheduler( shuffleToMapStage.get(shuffleDep.shuffleId) match { case Some(stage) => stage case None => + // We are going to register ancestor shuffle dependencies + registerShuffleDependencies(shuffleDep, jobId) + // Then register current shuffleDep val stage = newOrUsedStage( shuffleDep.rdd, shuffleDep.rdd.partitions.size, shuffleDep, jobId, shuffleDep.rdd.creationSite) shuffleToMapStage(shuffleDep.shuffleId) = stage + stage } } @@ -280,6 +284,9 @@ class DAGScheduler( private def getParentStages(rdd: RDD[_], jobId: Int): List[Stage] = { val parents = new HashSet[Stage] val visited = new HashSet[RDD[_]] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] def visit(r: RDD[_]) { if (!visited(r)) { visited += r @@ -290,18 +297,69 @@ class DAGScheduler( case shufDep: ShuffleDependency[_, _, _] => parents += getShuffleMapStage(shufDep, jobId) case _ => - visit(dep.rdd) + waitingForVisit.push(dep.rdd) } } } } - visit(rdd) + waitingForVisit.push(rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } parents.toList } + // Find ancestor missing shuffle dependencies and register into shuffleToMapStage + private def registerShuffleDependencies(shuffleDep: ShuffleDependency[_, _, _], jobId: Int) = { + val parentsWithNoMapStage = getAncestorShuffleDependencies(shuffleDep.rdd) + while (!parentsWithNoMapStage.isEmpty) { + val currentShufDep = parentsWithNoMapStage.pop() + val stage = + newOrUsedStage( + currentShufDep.rdd, currentShufDep.rdd.partitions.size, currentShufDep, jobId, + currentShufDep.rdd.creationSite) + shuffleToMapStage(currentShufDep.shuffleId) = stage + } + } + + // Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet + private def getAncestorShuffleDependencies(rdd: RDD[_]): Stack[ShuffleDependency[_, _, _]] = { + val parents = new Stack[ShuffleDependency[_, _, _]] + val visited = new HashSet[RDD[_]] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] + def visit(r: RDD[_]) { + if (!visited(r)) { + visited += r + for (dep <- r.dependencies) { + dep match { + case shufDep: ShuffleDependency[_, _, _] => + if (!shuffleToMapStage.contains(shufDep.shuffleId)) { + parents.push(shufDep) + } + + waitingForVisit.push(shufDep.rdd) + case _ => + waitingForVisit.push(dep.rdd) + } + } + } + } + + waitingForVisit.push(rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } + parents + } + private def getMissingParentStages(stage: Stage): List[Stage] = { val missing = new HashSet[Stage] val visited = new HashSet[RDD[_]] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] def visit(rdd: RDD[_]) { if (!visited(rdd)) { visited += rdd @@ -314,13 +372,16 @@ class DAGScheduler( missing += mapStage } case narrowDep: NarrowDependency[_] => - visit(narrowDep.rdd) + waitingForVisit.push(narrowDep.rdd) } } } } } - visit(stage.rdd) + waitingForVisit.push(stage.rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } missing.toList } @@ -1119,6 +1180,9 @@ class DAGScheduler( } val visitedRdds = new HashSet[RDD[_]] val visitedStages = new HashSet[Stage] + // We are manually maintaining a stack here to prevent StackOverflowError + // caused by recursively visiting + val waitingForVisit = new Stack[RDD[_]] def visit(rdd: RDD[_]) { if (!visitedRdds(rdd)) { visitedRdds += rdd @@ -1128,15 +1192,18 @@ class DAGScheduler( val mapStage = getShuffleMapStage(shufDep, stage.jobId) if (!mapStage.isAvailable) { visitedStages += mapStage - visit(mapStage.rdd) + waitingForVisit.push(mapStage.rdd) } // Otherwise there's no need to follow the dependency back case narrowDep: NarrowDependency[_] => - visit(narrowDep.rdd) + waitingForVisit.push(narrowDep.rdd) } } } } - visit(stage.rdd) + waitingForVisit.push(stage.rdd) + while (!waitingForVisit.isEmpty) { + visit(waitingForVisit.pop()) + } visitedRdds.contains(target.rdd) } From f5d9bea20e0db22c09c1191ca44a6471de765739 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 1 Aug 2014 13:25:04 -0700 Subject: [PATCH 0191/1492] SPARK-1612: Fix potential resource leaks JIRA: https://issues.apache.org/jira/browse/SPARK-1612 Move the "close" statements into a "finally" block. Author: zsxwing Closes #535 from zsxwing/SPARK-1612 and squashes the following commits: ae52f50 [zsxwing] Update to follow the code style 549ba13 [zsxwing] SPARK-1612: Fix potential resource leaks --- .../scala/org/apache/spark/util/Utils.scala | 35 ++++++++++++------- 1 file changed, 22 insertions(+), 13 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 f8fbb3ad6d4a1..30073a82857d2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -286,17 +286,23 @@ private[spark] object Utils extends Logging { out: OutputStream, closeStreams: Boolean = false) { - val buf = new Array[Byte](8192) - var n = 0 - while (n != -1) { - n = in.read(buf) - if (n != -1) { - out.write(buf, 0, n) + try { + val buf = new Array[Byte](8192) + var n = 0 + while (n != -1) { + n = in.read(buf) + if (n != -1) { + out.write(buf, 0, n) + } + } + } finally { + if (closeStreams) { + try { + in.close() + } finally { + out.close() + } } - } - if (closeStreams) { - in.close() - out.close() } } @@ -868,9 +874,12 @@ private[spark] object Utils extends Logging { val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) val stream = new FileInputStream(file) - stream.skip(effectiveStart) - stream.read(buff) - stream.close() + try { + stream.skip(effectiveStart) + stream.read(buff) + } finally { + stream.close() + } Source.fromBytes(buff).mkString } From b270309d7608fb749e402cd5afd36087446be398 Mon Sep 17 00:00:00 2001 From: joyyoj Date: Fri, 1 Aug 2014 13:41:55 -0700 Subject: [PATCH 0192/1492] [SPARK-2379] Fix the bug that streaming's receiver may fall into a dead loop Author: joyyoj Closes #1694 from joyyoj/SPARK-2379 and squashes the following commits: d73790d [joyyoj] SPARK-2379 Fix the bug that streaming's receiver may fall into a dead loop 22e7821 [joyyoj] Merge remote-tracking branch 'apache/master' 3f4a602 [joyyoj] Merge remote-tracking branch 'remotes/apache/master' f4660c5 [joyyoj] [SPARK-1998] SparkFlumeEvent with body bigger than 1020 bytes are not read properly --- .../apache/spark/streaming/receiver/ReceiverSupervisor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 09be3a50d2dfa..1f0244c251eba 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -138,7 +138,7 @@ private[streaming] abstract class ReceiverSupervisor( onReceiverStop(message, error) } catch { case t: Throwable => - stop("Error stopping receiver " + streamId, Some(t)) + logError("Error stopping receiver " + streamId + t.getStackTraceString) } } From 78f2af582286b81e6dc9fa9d455ed2b369d933bd Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 1 Aug 2014 13:57:19 -0700 Subject: [PATCH 0193/1492] SPARK-2791: Fix committing, reverting and state tracking in shuffle file consolidation All changes from this PR are by mridulm and are drawn from his work in #1609. This patch is intended to fix all major issues related to shuffle file consolidation that mridulm found, while minimizing changes to the code, with the hope that it may be more easily merged into 1.1. This patch is **not** intended as a replacement for #1609, which provides many additional benefits, including fixes to ExternalAppendOnlyMap, improvements to DiskBlockObjectWriter's API, and several new unit tests. If it is feasible to merge #1609 for the 1.1 deadline, that is a preferable option. Author: Aaron Davidson Closes #1678 from aarondav/consol and squashes the following commits: 53b3f6d [Aaron Davidson] Correct behavior when writing unopened file 701d045 [Aaron Davidson] Rebase with sort-based shuffle 9160149 [Aaron Davidson] SPARK-2532: Minimal shuffle consolidation fixes --- .../shuffle/hash/HashShuffleWriter.scala | 14 +-- .../shuffle/sort/SortShuffleWriter.scala | 3 +- .../spark/storage/BlockObjectWriter.scala | 53 ++++++----- .../spark/storage/ShuffleBlockManager.scala | 28 +++--- .../collection/ExternalAppendOnlyMap.scala | 2 +- .../util/collection/ExternalSorter.scala | 6 +- .../spark/storage/DiskBlockManagerSuite.scala | 87 ++++++++++++++++++- .../spark/tools/StoragePerfTester.scala | 5 +- 8 files changed, 146 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 1923f7c71a48f..45d3b8b9b8725 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -65,7 +65,8 @@ private[spark] class HashShuffleWriter[K, V]( } /** Close this writer, passing along whether the map completed */ - override def stop(success: Boolean): Option[MapStatus] = { + override def stop(initiallySuccess: Boolean): Option[MapStatus] = { + var success = initiallySuccess try { if (stopping) { return None @@ -73,15 +74,16 @@ private[spark] class HashShuffleWriter[K, V]( stopping = true if (success) { try { - return Some(commitWritesAndBuildStatus()) + Some(commitWritesAndBuildStatus()) } catch { case e: Exception => + success = false revertWrites() throw e } } else { revertWrites() - return None + None } } finally { // Release the writers back to the shuffle block manager. @@ -100,8 +102,7 @@ private[spark] class HashShuffleWriter[K, V]( var totalBytes = 0L var totalTime = 0L val compressedSizes = shuffle.writers.map { writer: BlockObjectWriter => - writer.commit() - writer.close() + writer.commitAndClose() val size = writer.fileSegment().length totalBytes += size totalTime += writer.timeWriting() @@ -120,8 +121,7 @@ private[spark] class HashShuffleWriter[K, V]( private def revertWrites(): Unit = { if (shuffle != null && shuffle.writers != null) { for (writer <- shuffle.writers) { - writer.revertPartialWrites() - writer.close() + writer.revertPartialWritesAndClose() } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 42fcd07fa18bc..9a356d0dbaf17 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -94,8 +94,7 @@ private[spark] class SortShuffleWriter[K, V, C]( for (elem <- elements) { writer.write(elem) } - writer.commit() - writer.close() + writer.commitAndClose() val segment = writer.fileSegment() offsets(id + 1) = segment.offset + segment.length lengths(id) = segment.length diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index a2687e6be4e34..01d46e1ffc960 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -39,16 +39,16 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { def isOpen: Boolean /** - * Flush the partial writes and commit them as a single atomic block. Return the - * number of bytes written for this commit. + * Flush the partial writes and commit them as a single atomic block. */ - def commit(): Long + def commitAndClose(): Unit /** * Reverts writes that haven't been flushed yet. Callers should invoke this function - * when there are runtime exceptions. + * when there are runtime exceptions. This method will not throw, though it may be + * unsuccessful in truncating written data. */ - def revertPartialWrites() + def revertPartialWritesAndClose() /** * Writes an object. @@ -57,6 +57,7 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { /** * Returns the file segment of committed data that this Writer has written. + * This is only valid after commitAndClose() has been called. */ def fileSegment(): FileSegment @@ -108,7 +109,7 @@ private[spark] class DiskBlockObjectWriter( private var ts: TimeTrackingOutputStream = null private var objOut: SerializationStream = null private val initialPosition = file.length() - private var lastValidPosition = initialPosition + private var finalPosition: Long = -1 private var initialized = false private var _timeWriting = 0L @@ -116,7 +117,6 @@ private[spark] class DiskBlockObjectWriter( fos = new FileOutputStream(file, true) ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() - lastValidPosition = initialPosition bs = compressStream(new BufferedOutputStream(ts, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true @@ -147,28 +147,36 @@ private[spark] class DiskBlockObjectWriter( override def isOpen: Boolean = objOut != null - override def commit(): Long = { + override def commitAndClose(): Unit = { if (initialized) { // NOTE: Because Kryo doesn't flush the underlying stream we explicitly flush both the // serializer stream and the lower level stream. objOut.flush() bs.flush() - val prevPos = lastValidPosition - lastValidPosition = channel.position() - lastValidPosition - prevPos - } else { - // lastValidPosition is zero if stream is uninitialized - lastValidPosition + close() } + finalPosition = file.length() } - override def revertPartialWrites() { - if (initialized) { - // Discard current writes. We do this by flushing the outstanding writes and - // truncate the file to the last valid position. - objOut.flush() - bs.flush() - channel.truncate(lastValidPosition) + // Discard current writes. We do this by flushing the outstanding writes and then + // truncating the file to its initial position. + override def revertPartialWritesAndClose() { + try { + if (initialized) { + objOut.flush() + bs.flush() + close() + } + + val truncateStream = new FileOutputStream(file, true) + try { + truncateStream.getChannel.truncate(initialPosition) + } finally { + truncateStream.close() + } + } catch { + case e: Exception => + logError("Uncaught exception while reverting partial writes to file " + file, e) } } @@ -188,6 +196,7 @@ private[spark] class DiskBlockObjectWriter( // Only valid if called after commit() override def bytesWritten: Long = { - lastValidPosition - initialPosition + assert(finalPosition != -1, "bytesWritten is only valid after successful commit()") + finalPosition - initialPosition } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 7beb55c411e71..28aa35bc7e147 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -144,7 +144,8 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { if (consolidateShuffleFiles) { if (success) { val offsets = writers.map(_.fileSegment().offset) - fileGroup.recordMapOutput(mapId, offsets) + val lengths = writers.map(_.fileSegment().length) + fileGroup.recordMapOutput(mapId, offsets, lengths) } recycleFileGroup(fileGroup) } else { @@ -247,6 +248,8 @@ object ShuffleBlockManager { * A particular mapper will be assigned a single ShuffleFileGroup to write its output to. */ private class ShuffleFileGroup(val shuffleId: Int, val fileId: Int, val files: Array[File]) { + private var numBlocks: Int = 0 + /** * Stores the absolute index of each mapId in the files of this group. For instance, * if mapId 5 is the first block in each file, mapIdToIndex(5) = 0. @@ -254,23 +257,27 @@ object ShuffleBlockManager { private val mapIdToIndex = new PrimitiveKeyOpenHashMap[Int, Int]() /** - * Stores consecutive offsets of blocks into each reducer file, ordered by position in the file. - * This ordering allows us to compute block lengths by examining the following block offset. + * Stores consecutive offsets and lengths of blocks into each reducer file, ordered by + * position in the file. * Note: mapIdToIndex(mapId) returns the index of the mapper into the vector for every * reducer. */ private val blockOffsetsByReducer = Array.fill[PrimitiveVector[Long]](files.length) { new PrimitiveVector[Long]() } - - def numBlocks = mapIdToIndex.size + private val blockLengthsByReducer = Array.fill[PrimitiveVector[Long]](files.length) { + new PrimitiveVector[Long]() + } def apply(bucketId: Int) = files(bucketId) - def recordMapOutput(mapId: Int, offsets: Array[Long]) { + def recordMapOutput(mapId: Int, offsets: Array[Long], lengths: Array[Long]) { + assert(offsets.length == lengths.length) mapIdToIndex(mapId) = numBlocks + numBlocks += 1 for (i <- 0 until offsets.length) { blockOffsetsByReducer(i) += offsets(i) + blockLengthsByReducer(i) += lengths(i) } } @@ -278,16 +285,11 @@ object ShuffleBlockManager { def getFileSegmentFor(mapId: Int, reducerId: Int): Option[FileSegment] = { val file = files(reducerId) val blockOffsets = blockOffsetsByReducer(reducerId) + val blockLengths = blockLengthsByReducer(reducerId) val index = mapIdToIndex.getOrElse(mapId, -1) if (index >= 0) { val offset = blockOffsets(index) - val length = - if (index + 1 < numBlocks) { - blockOffsets(index + 1) - offset - } else { - file.length() - offset - } - assert(length >= 0) + val length = blockLengths(index) Some(new FileSegment(file, offset, length)) } else { None diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index b34512ef9eb60..cb67a1c039f20 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -199,7 +199,7 @@ class ExternalAppendOnlyMap[K, V, C]( // Flush the disk writer's contents to disk, and update relevant variables def flush() = { - writer.commit() + writer.commitAndClose() val bytesWritten = writer.bytesWritten batchSizes.append(bytesWritten) _diskBytesSpilled += bytesWritten diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 54c3310744136..6e415a2bd8ce2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -270,9 +270,10 @@ private[spark] class ExternalSorter[K, V, C]( // How many elements we have in each partition val elementsPerPartition = new Array[Long](numPartitions) - // Flush the disk writer's contents to disk, and update relevant variables + // Flush the disk writer's contents to disk, and update relevant variables. + // The writer is closed at the end of this process, and cannot be reused. def flush() = { - writer.commit() + writer.commitAndClose() val bytesWritten = writer.bytesWritten batchSizes.append(bytesWritten) _diskBytesSpilled += bytesWritten @@ -293,7 +294,6 @@ private[spark] class ExternalSorter[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() - writer.close() writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize) } } diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index aaa7714049732..985ac9394738c 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -22,11 +22,14 @@ import java.io.{File, FileWriter} import scala.collection.mutable import scala.language.reflectiveCalls +import akka.actor.Props import com.google.common.io.Files import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.SparkConf -import org.apache.spark.util.Utils +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{AkkaUtils, Utils} class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) @@ -121,6 +124,88 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before newFile.delete() } + private def checkSegments(segment1: FileSegment, segment2: FileSegment) { + assert (segment1.file.getCanonicalPath === segment2.file.getCanonicalPath) + assert (segment1.offset === segment2.offset) + assert (segment1.length === segment2.length) + } + + test("consolidated shuffle can write to shuffle group without messing existing offsets/lengths") { + + val serializer = new JavaSerializer(testConf) + val confCopy = testConf.clone + // reset after EACH object write. This is to ensure that there are bytes appended after + // an object is written. So if the codepaths assume writeObject is end of data, this should + // flush those bugs out. This was common bug in ExternalAppendOnlyMap, etc. + confCopy.set("spark.serializer.objectStreamReset", "1") + + val securityManager = new org.apache.spark.SecurityManager(confCopy) + // Do not use the shuffleBlockManager above ! + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, confCopy, + securityManager) + val master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, confCopy, new LiveListenerBus))), + confCopy) + val store = new BlockManager("", actorSystem, master , serializer, confCopy, + securityManager, null) + + try { + + val shuffleManager = store.shuffleBlockManager + + val shuffle1 = shuffleManager.forMapTask(1, 1, 1, serializer) + for (writer <- shuffle1.writers) { + writer.write("test1") + writer.write("test2") + } + for (writer <- shuffle1.writers) { + writer.commitAndClose() + } + + val shuffle1Segment = shuffle1.writers(0).fileSegment() + shuffle1.releaseWriters(success = true) + + val shuffle2 = shuffleManager.forMapTask(1, 2, 1, new JavaSerializer(testConf)) + + for (writer <- shuffle2.writers) { + writer.write("test3") + writer.write("test4") + } + for (writer <- shuffle2.writers) { + writer.commitAndClose() + } + val shuffle2Segment = shuffle2.writers(0).fileSegment() + shuffle2.releaseWriters(success = true) + + // Now comes the test : + // Write to shuffle 3; and close it, but before registering it, check if the file lengths for + // previous task (forof shuffle1) is the same as 'segments'. Earlier, we were inferring length + // of block based on remaining data in file : which could mess things up when there is concurrent read + // and writes happening to the same shuffle group. + + val shuffle3 = shuffleManager.forMapTask(1, 3, 1, new JavaSerializer(testConf)) + for (writer <- shuffle3.writers) { + writer.write("test3") + writer.write("test4") + } + for (writer <- shuffle3.writers) { + writer.commitAndClose() + } + // check before we register. + checkSegments(shuffle2Segment, shuffleManager.getBlockLocation(ShuffleBlockId(1, 2, 0))) + shuffle3.releaseWriters(success = true) + checkSegments(shuffle2Segment, shuffleManager.getBlockLocation(ShuffleBlockId(1, 2, 0))) + shuffleManager.removeShuffle(1) + } finally { + + if (store != null) { + store.stop() + } + actorSystem.shutdown() + actorSystem.awaitTermination() + } + } + def assertSegmentEquals(blockId: BlockId, filename: String, offset: Int, length: Int) { val segment = diskBlockManager.getBlockLocation(blockId) assert(segment.file.getName === filename) diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 8e8c35615a711..8a05fcb449aa6 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -61,10 +61,9 @@ object StoragePerfTester { for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeData) } - writers.map {w => - w.commit() + writers.map { w => + w.commitAndClose() total.addAndGet(w.fileSegment().length) - w.close() } shuffle.releaseWriters(true) From d88e69561367d65e1a2b94527b80a1f65a2cba90 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Fri, 1 Aug 2014 15:02:17 -0700 Subject: [PATCH 0194/1492] [SPARK-2786][mllib] Python correlations Author: Doris Xin Closes #1713 from dorx/pythonCorrelation and squashes the following commits: 5f1e60c [Doris Xin] reviewer comments. 46ff6eb [Doris Xin] reviewer comments. ad44085 [Doris Xin] style fix e69d446 [Doris Xin] fixed missed conflicts. eb5bf56 [Doris Xin] merge master cc9f725 [Doris Xin] units passed. 9141a63 [Doris Xin] WIP2 d199f1f [Doris Xin] Moved correlation names into a public object cd163d6 [Doris Xin] WIP --- .../mllib/api/python/PythonMLLibAPI.scala | 39 ++++++- .../apache/spark/mllib/stat/Statistics.scala | 10 +- .../mllib/stat/correlation/Correlation.scala | 49 +++++---- .../api/python/PythonMLLibAPISuite.scala | 21 +++- python/pyspark/mllib/_common.py | 6 +- python/pyspark/mllib/stat.py | 104 ++++++++++++++++++ 6 files changed, 199 insertions(+), 30 deletions(-) create mode 100644 python/pyspark/mllib/stat.py diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index d2e8ccf208970..122925d096e98 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -20,13 +20,15 @@ package org.apache.spark.mllib.api.python import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.api.java.{JavaSparkContext, JavaRDD} +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.random.{RandomRDDGenerators => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -227,7 +229,7 @@ class PythonMLLibAPI extends Serializable { jsc: JavaSparkContext, path: String, minPartitions: Int): JavaRDD[Array[Byte]] = - MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(serializeLabeledPoint).toJavaRDD() + MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(serializeLabeledPoint) private def trainRegressionModel( trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, @@ -456,6 +458,37 @@ class PythonMLLibAPI extends Serializable { ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) } + /** + * Java stub for mllib Statistics.corr(X: RDD[Vector], method: String). + * Returns the correlation matrix serialized into a byte array understood by deserializers in + * pyspark. + */ + def corr(X: JavaRDD[Array[Byte]], method: String): Array[Byte] = { + val inputMatrix = X.rdd.map(deserializeDoubleVector(_)) + val result = Statistics.corr(inputMatrix, getCorrNameOrDefault(method)) + serializeDoubleMatrix(to2dArray(result)) + } + + /** + * Java stub for mllib Statistics.corr(x: RDD[Double], y: RDD[Double], method: String). + */ + def corr(x: JavaRDD[Array[Byte]], y: JavaRDD[Array[Byte]], method: String): Double = { + val xDeser = x.rdd.map(deserializeDouble(_)) + val yDeser = y.rdd.map(deserializeDouble(_)) + Statistics.corr(xDeser, yDeser, getCorrNameOrDefault(method)) + } + + // used by the corr methods to retrieve the name of the correlation method passed in via pyspark + private def getCorrNameOrDefault(method: String) = { + if (method == null) CorrelationNames.defaultCorrName else method + } + + // Reformat a Matrix into Array[Array[Double]] for serialization + private[python] def to2dArray(matrix: Matrix): Array[Array[Double]] = { + val values = matrix.toArray + Array.tabulate(matrix.numRows, matrix.numCols)((i, j) => values(i + j * matrix.numRows)) + } + // Used by the *RDD methods to get default seed if not passed in from pyspark private def getSeedOrDefault(seed: java.lang.Long): Long = { if (seed == null) Utils.random.nextLong else seed diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index 9d6de9b6e1f60..f416a9fbb323d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -23,21 +23,24 @@ import org.apache.spark.mllib.stat.correlation.Correlations import org.apache.spark.rdd.RDD /** - * API for statistical functions in MLlib + * API for statistical functions in MLlib. */ @Experimental object Statistics { /** + * :: Experimental :: * Compute the Pearson correlation matrix for the input RDD of Vectors. * Columns with 0 covariance produce NaN entries in the correlation matrix. * * @param X an RDD[Vector] for which the correlation matrix is to be computed. * @return Pearson correlation matrix comparing columns in X. */ + @Experimental def corr(X: RDD[Vector]): Matrix = Correlations.corrMatrix(X) /** + * :: Experimental :: * Compute the correlation matrix for the input RDD of Vectors using the specified method. * Methods currently supported: `pearson` (default), `spearman`. * @@ -51,9 +54,11 @@ object Statistics { * Supported: `pearson` (default), `spearman` * @return Correlation matrix comparing columns in X. */ + @Experimental def corr(X: RDD[Vector], method: String): Matrix = Correlations.corrMatrix(X, method) /** + * :: Experimental :: * Compute the Pearson correlation for the input RDDs. * Returns NaN if either vector has 0 variance. * @@ -64,9 +69,11 @@ object Statistics { * @param y RDD[Double] of the same cardinality as x. * @return A Double containing the Pearson correlation between the two input RDD[Double]s */ + @Experimental def corr(x: RDD[Double], y: RDD[Double]): Double = Correlations.corr(x, y) /** + * :: Experimental :: * Compute the correlation for the input RDDs using the specified method. * Methods currently supported: `pearson` (default), `spearman`. * @@ -80,5 +87,6 @@ object Statistics { *@return A Double containing the correlation between the two input RDD[Double]s using the * specified method. */ + @Experimental def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala index f23393d3da257..1fb8d7b3d4f32 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/Correlation.scala @@ -49,43 +49,48 @@ private[stat] trait Correlation { } /** - * Delegates computation to the specific correlation object based on the input method name - * - * Currently supported correlations: pearson, spearman. - * After new correlation algorithms are added, please update the documentation here and in - * Statistics.scala for the correlation APIs. - * - * Maintains the default correlation type, pearson + * Delegates computation to the specific correlation object based on the input method name. */ private[stat] object Correlations { - // Note: after new types of correlations are implemented, please update this map - val nameToObjectMap = Map(("pearson", PearsonCorrelation), ("spearman", SpearmanCorrelation)) - val defaultCorrName: String = "pearson" - val defaultCorr: Correlation = nameToObjectMap(defaultCorrName) - - def corr(x: RDD[Double], y: RDD[Double], method: String = defaultCorrName): Double = { + def corr(x: RDD[Double], + y: RDD[Double], + method: String = CorrelationNames.defaultCorrName): Double = { val correlation = getCorrelationFromName(method) correlation.computeCorrelation(x, y) } - def corrMatrix(X: RDD[Vector], method: String = defaultCorrName): Matrix = { + def corrMatrix(X: RDD[Vector], + method: String = CorrelationNames.defaultCorrName): Matrix = { val correlation = getCorrelationFromName(method) correlation.computeCorrelationMatrix(X) } - /** - * Match input correlation name with a known name via simple string matching - * - * private to stat for ease of unit testing - */ - private[stat] def getCorrelationFromName(method: String): Correlation = { + // Match input correlation name with a known name via simple string matching. + def getCorrelationFromName(method: String): Correlation = { try { - nameToObjectMap(method) + CorrelationNames.nameToObjectMap(method) } catch { case nse: NoSuchElementException => throw new IllegalArgumentException("Unrecognized method name. Supported correlations: " - + nameToObjectMap.keys.mkString(", ")) + + CorrelationNames.nameToObjectMap.keys.mkString(", ")) } } } + +/** + * Maintains supported and default correlation names. + * + * Currently supported correlations: `pearson`, `spearman`. + * Current default correlation: `pearson`. + * + * After new correlation algorithms are added, please update the documentation here and in + * Statistics.scala for the correlation APIs. + */ +private[mllib] object CorrelationNames { + + // Note: after new types of correlations are implemented, please update this map. + val nameToObjectMap = Map(("pearson", PearsonCorrelation), ("spearman", SpearmanCorrelation)) + val defaultCorrName: String = "pearson" + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index d94cfa2fcec81..bd413a80f5107 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.api.python import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint class PythonMLLibAPISuite extends FunSuite { @@ -59,10 +59,25 @@ class PythonMLLibAPISuite extends FunSuite { } test("double serialization") { - for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue)) { + for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { val bytes = py.serializeDouble(x) val deser = py.deserializeDouble(bytes) - assert(x === deser) + // We use `equals` here for comparison because we cannot use `==` for NaN + assert(x.equals(deser)) } } + + test("matrix to 2D array") { + val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) + val matrix = Matrices.dense(2, 3, values) + val arr = py.to2dArray(matrix) + val expected = Array(Array[Double](0, 3, 7), Array[Double](1.2, 4.56, 8)) + assert(arr === expected) + + // Test conversion for empty matrix + val empty = Array[Double]() + val emptyMatrix = Matrices.dense(0, 0, empty) + val empty2D = py.to2dArray(emptyMatrix) + assert(empty2D === Array[Array[Double]]()) + } } diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 8e3ad6b783b6c..c6ca6a75df746 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -101,7 +101,7 @@ def _serialize_double(d): """ Serialize a double (float or numpy.float64) into a mutually understood format. """ - if type(d) == float or type(d) == float64: + if type(d) == float or type(d) == float64 or type(d) == int or type(d) == long: d = float64(d) ba = bytearray(8) _copyto(d, buffer=ba, offset=0, shape=[1], dtype=float64) @@ -176,6 +176,10 @@ def _deserialize_double(ba, offset=0): True >>> _deserialize_double(_serialize_double(float64(0.0))) == 0.0 True + >>> _deserialize_double(_serialize_double(1)) == 1.0 + True + >>> _deserialize_double(_serialize_double(1L)) == 1.0 + True >>> x = sys.float_info.max >>> _deserialize_double(_serialize_double(sys.float_info.max)) == x True diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py new file mode 100644 index 0000000000000..0a08a562d1f1f --- /dev/null +++ b/python/pyspark/mllib/stat.py @@ -0,0 +1,104 @@ +# +# 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. +# + +""" +Python package for statistical functions in MLlib. +""" + +from pyspark.mllib._common import \ + _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ + _serialize_double, _serialize_double_vector, \ + _deserialize_double, _deserialize_double_matrix + +class Statistics(object): + + @staticmethod + def corr(x, y=None, method=None): + """ + Compute the correlation (matrix) for the input RDD(s) using the + specified method. + Methods currently supported: I{pearson (default), spearman}. + + If a single RDD of Vectors is passed in, a correlation matrix + comparing the columns in the input RDD is returned. Use C{method=} + to specify the method to be used for single RDD inout. + If two RDDs of floats are passed in, a single float is returned. + + >>> x = sc.parallelize([1.0, 0.0, -2.0], 2) + >>> y = sc.parallelize([4.0, 5.0, 3.0], 2) + >>> zeros = sc.parallelize([0.0, 0.0, 0.0], 2) + >>> abs(Statistics.corr(x, y) - 0.6546537) < 1e-7 + True + >>> Statistics.corr(x, y) == Statistics.corr(x, y, "pearson") + True + >>> Statistics.corr(x, y, "spearman") + 0.5 + >>> from math import isnan + >>> isnan(Statistics.corr(x, zeros)) + True + >>> from linalg import Vectors + >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), + ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) + >>> Statistics.corr(rdd) + array([[ 1. , 0.05564149, nan, 0.40047142], + [ 0.05564149, 1. , nan, 0.91359586], + [ nan, nan, 1. , nan], + [ 0.40047142, 0.91359586, nan, 1. ]]) + >>> Statistics.corr(rdd, method="spearman") + array([[ 1. , 0.10540926, nan, 0.4 ], + [ 0.10540926, 1. , nan, 0.9486833 ], + [ nan, nan, 1. , nan], + [ 0.4 , 0.9486833 , nan, 1. ]]) + >>> try: + ... Statistics.corr(rdd, "spearman") + ... print "Method name as second argument without 'method=' shouldn't be allowed." + ... except TypeError: + ... pass + """ + sc = x.ctx + # Check inputs to determine whether a single value or a matrix is needed for output. + # Since it's legal for users to use the method name as the second argument, we need to + # check if y is used to specify the method name instead. + if type(y) == str: + raise TypeError("Use 'method=' to specify method name.") + if not y: + try: + Xser = _get_unmangled_double_vector_rdd(x) + except TypeError: + raise TypeError("corr called on a single RDD not consisted of Vectors.") + resultMat = sc._jvm.PythonMLLibAPI().corr(Xser._jrdd, method) + return _deserialize_double_matrix(resultMat) + else: + xSer = _get_unmangled_rdd(x, _serialize_double) + ySer = _get_unmangled_rdd(y, _serialize_double) + result = sc._jvm.PythonMLLibAPI().corr(xSer._jrdd, ySer._jrdd, method) + return result + + +def _test(): + import doctest + from pyspark import SparkContext + globs = globals().copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() From 7058a5393bccc2f917189fa9b4cf7f314410b0de Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 1 Aug 2014 15:52:21 -0700 Subject: [PATCH 0195/1492] [SPARK-2796] [mllib] DecisionTree bug fix: ordered categorical features Bug: In DecisionTree, the method sequentialBinSearchForOrderedCategoricalFeatureInClassification() indexed bins from 0 to (math.pow(2, featureCategories.toInt - 1) - 1). This upper bound is the bound for unordered categorical features, not ordered ones. The upper bound should be the arity (i.e., max value) of the feature. Added new test to DecisionTreeSuite to catch this: "regression stump with categorical variables of arity 2" Bug fix: Modified upper bound discussed above. Also: Small improvements to coding style in DecisionTree. CC mengxr manishamde Author: Joseph K. Bradley Closes #1720 from jkbradley/decisiontree-bugfix2 and squashes the following commits: 225822f [Joseph K. Bradley] Bug: In DecisionTree, the method sequentialBinSearchForOrderedCategoricalFeatureInClassification() indexed bins from 0 to (math.pow(2, featureCategories.toInt - 1) - 1). This upper bound is the bound for unordered categorical features, not ordered ones. The upper bound should be the arity (i.e., max value) of the feature. --- .../spark/mllib/tree/DecisionTree.scala | 45 +++++++++++-------- .../spark/mllib/tree/DecisionTreeSuite.scala | 29 ++++++++++++ 2 files changed, 56 insertions(+), 18 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 7d123dd6ae996..382e76a9b7cba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -498,7 +498,7 @@ object DecisionTree extends Serializable with Logging { val bin = binForFeatures(mid) val lowThreshold = bin.lowSplit.threshold val highThreshold = bin.highSplit.threshold - if ((lowThreshold < feature) && (highThreshold >= feature)){ + if ((lowThreshold < feature) && (highThreshold >= feature)) { return mid } else if (lowThreshold >= feature) { @@ -522,28 +522,36 @@ object DecisionTree extends Serializable with Logging { } /** - * Sequential search helper method to find bin for categorical feature. + * Sequential search helper method to find bin for categorical feature + * (for classification and regression). */ - def sequentialBinSearchForOrderedCategoricalFeatureInClassification(): Int = { + def sequentialBinSearchForOrderedCategoricalFeature(): Int = { val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val numCategoricalBins = math.pow(2.0, featureCategories - 1).toInt - 1 + val featureValue = labeledPoint.features(featureIndex) var binIndex = 0 - while (binIndex < numCategoricalBins) { + while (binIndex < featureCategories) { val bin = bins(featureIndex)(binIndex) val categories = bin.highSplit.categories - val features = labeledPoint.features - if (categories.contains(features(featureIndex))) { + if (categories.contains(featureValue)) { return binIndex } binIndex += 1 } + if (featureValue < 0 || featureValue >= featureCategories) { + throw new IllegalArgumentException( + s"DecisionTree given invalid data:" + + s" Feature $featureIndex is categorical with values in" + + s" {0,...,${featureCategories - 1}," + + s" but a data point gives it value $featureValue.\n" + + " Bad data point: " + labeledPoint.toString) + } -1 } if (isFeatureContinuous) { // Perform binary search for finding bin for continuous features. val binIndex = binarySearchForBins() - if (binIndex == -1){ + if (binIndex == -1) { throw new UnknownError("no bin was found for continuous variable.") } binIndex @@ -555,10 +563,10 @@ object DecisionTree extends Serializable with Logging { if (isUnorderedFeature) { sequentialBinSearchForUnorderedCategoricalFeatureInClassification() } else { - sequentialBinSearchForOrderedCategoricalFeatureInClassification() + sequentialBinSearchForOrderedCategoricalFeature() } } - if (binIndex == -1){ + if (binIndex == -1) { throw new UnknownError("no bin was found for categorical variable.") } binIndex @@ -642,11 +650,12 @@ object DecisionTree extends Serializable with Logging { val arrShift = 1 + numFeatures * nodeIndex val arrIndex = arrShift + featureIndex // Update the left or right count for one bin. - val aggShift = numClasses * numBins * numFeatures * nodeIndex - val aggIndex - = aggShift + numClasses * featureIndex * numBins + arr(arrIndex).toInt * numClasses - val labelInt = label.toInt - agg(aggIndex + labelInt) = agg(aggIndex + labelInt) + 1 + val aggIndex = + numClasses * numBins * numFeatures * nodeIndex + + numClasses * numBins * featureIndex + + numClasses * arr(arrIndex).toInt + + label.toInt + agg(aggIndex) += 1 } /** @@ -1127,7 +1136,7 @@ object DecisionTree extends Serializable with Logging { val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) var featureIndex = 0 while (featureIndex < numFeatures) { - if (isMulticlassClassificationWithCategoricalFeatures){ + if (isMulticlassClassificationWithCategoricalFeatures) { val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty if (isFeatureContinuous) { findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) @@ -1393,7 +1402,7 @@ object DecisionTree extends Serializable with Logging { // Iterate over all features. var featureIndex = 0 - while (featureIndex < numFeatures){ + while (featureIndex < numFeatures) { // Check whether the feature is continuous. val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty if (isFeatureContinuous) { @@ -1513,7 +1522,7 @@ object DecisionTree extends Serializable with Logging { if (isFeatureContinuous) { // Bins for categorical variables are already assigned. bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), splits(featureIndex)(0), Continuous, Double.MinValue) - for (index <- 1 until numBins - 1){ + for (index <- 1 until numBins - 1) { val bin = new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), Continuous, Double.MinValue) bins(featureIndex)(index) = bin diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 10462db700628..546a132559326 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -42,6 +42,18 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(accuracy >= requiredAccuracy) } + def validateRegressor( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredMSE: Double) { + val predictions = input.map(x => model.predict(x.features)) + val squaredError = predictions.zip(input).map { case (prediction, expected) => + (prediction - expected.label) * (prediction - expected.label) + }.sum + val mse = squaredError / input.length + assert(mse <= requiredMSE) + } + test("split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) @@ -454,6 +466,23 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.impurity > 0.2) } + test("regression stump with categorical variables of arity 2") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Regression, + Variance, + maxDepth = 2, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + + val model = DecisionTree.train(rdd, strategy) + validateRegressor(model, arr, 0.0) + assert(model.numNodes === 3) + assert(model.depth === 1) + } + test("stump with fixed label 0 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) From 880eabec37c69ce4e9594d7babfac291b0f93f50 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 1 Aug 2014 18:47:41 -0700 Subject: [PATCH 0196/1492] [SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDD Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes. This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance. root |-- field1: integer (nullable = true) |-- field2: string (nullable = true) |-- field3: struct (nullable = true) | |-- field4: integer (nullable = true) | |-- field5: array (nullable = true) | | |-- element: integer (containsNull = false) |-- field6: array (nullable = true) | |-- element: struct (containsNull = false) | | |-- field7: string (nullable = true) Then we can access them by row.field3.field5[0] or row.field6[5].field7 It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType. You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as: ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1])) Or you could use Row to create a class just like namedtuple, for example: Person = Row("name", "age") ctx.inferSchema(rdd.map(lambda x: Person(*x))) Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details. schema = StructType([StructField("name, StringType, True), StructType("age", IntegerType, True)]) ctx.applySchema(rdd, schema) PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable. Author: Davies Liu Closes #1598 from davies/nested and squashes the following commits: f1d15b6 [Davies Liu] verify schema with the first few rows 8852aaf [Davies Liu] check type of schema abe9e6e [Davies Liu] address comments 61b2292 [Davies Liu] add @deprecated to pythonToJavaMap 1e5b801 [Davies Liu] improve cache of classes 51aa135 [Davies Liu] use Row to infer schema e9c0d5c [Davies Liu] remove string typed schema 353a3f2 [Davies Liu] fix code style 63de8f8 [Davies Liu] fix typo c79ca67 [Davies Liu] fix serialization of nested data 6b258b5 [Davies Liu] fix pep8 9d8447c [Davies Liu] apply schema provided by string of names f5df97f [Davies Liu] refactor, address comments 9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python 84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested 0eaaf56 [Davies Liu] fix doc tests b3559b4 [Davies Liu] use generated Row instead of namedtuple c4ddc30 [Davies Liu] fix conflict between name of fields and variables 7f6f251 [Davies Liu] address all comments d69d397 [Davies Liu] refactor 2cc2d45 [Davies Liu] refactor 182fb46 [Davies Liu] refactor bc6e9e1 [Davies Liu] switch to new Schema API 547bf3e [Davies Liu] Merge branch 'master' into nested a435b5a [Davies Liu] add docs and code refactor 2c8debc [Davies Liu] Merge branch 'master' into nested 644665a [Davies Liu] use tuple and namedtuple for schemardd --- .../apache/spark/api/python/PythonRDD.scala | 69 +- python/pyspark/rdd.py | 8 +- python/pyspark/sql.py | 1258 ++++++++++++----- .../org/apache/spark/sql/SQLContext.scala | 87 +- .../org/apache/spark/sql/SchemaRDD.scala | 18 +- 5 files changed, 996 insertions(+), 444 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 94d666aa92025..fe9a9e50ef21d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -25,7 +25,7 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.language.existentials import scala.reflect.ClassTag -import scala.util.Try +import scala.util.{Try, Success, Failure} import net.razorvine.pickle.{Pickler, Unpickler} @@ -536,25 +536,6 @@ private[spark] object PythonRDD extends Logging { file.close() } - /** - * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions). - * It is only used by pyspark.sql. - * TODO: Support more Python types. - */ - def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { - pyRDD.rdd.mapPartitions { iter => - val unpickle = new Unpickler - iter.flatMap { row => - unpickle.loads(row) match { - // in case of objects are pickled in batch mode - case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) - // not in batch mode - case obj: JMap[String @unchecked, _] => Seq(obj.toMap) - } - } - } - } - private def getMergedConf(confAsMap: java.util.HashMap[String, String], baseConf: Configuration): Configuration = { val conf = PythonHadoopUtil.mapToConf(confAsMap) @@ -701,6 +682,54 @@ private[spark] object PythonRDD extends Logging { } } + + /** + * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions). + * This function is outdated, PySpark does not use it anymore + */ + @deprecated + def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + iter.flatMap { row => + unpickle.loads(row) match { + // in case of objects are pickled in batch mode + case objs: JArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) + // not in batch mode + case obj: JMap[String @unchecked, _] => Seq(obj.toMap) + } + } + } + } + + /** + * Convert an RDD of serialized Python tuple to Array (no recursive conversions). + * It is only used by pyspark.sql. + */ + def pythonToJavaArray(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Array[_]] = { + + def toArray(obj: Any): Array[_] = { + obj match { + case objs: JArrayList[_] => + objs.toArray + case obj if obj.getClass.isArray => + obj.asInstanceOf[Array[_]].toArray + } + } + + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj.asInstanceOf[JArrayList[_]].map(toArray) + } else { + Seq(toArray(obj)) + } + } + }.toJavaRDD() + } + /** * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by * PySpark. diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index e8fcc900efb24..309f5a9b6038d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -318,9 +318,9 @@ def map(self, f, preservesPartitioning=False): >>> sorted(rdd.map(lambda x: (x, 1)).collect()) [('a', 1), ('b', 1), ('c', 1)] """ - def func(split, iterator): + def func(_, iterator): return imap(f, iterator) - return PipelinedRDD(self, func, preservesPartitioning) + return self.mapPartitionsWithIndex(func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): """ @@ -1184,7 +1184,7 @@ def func(split, iterator): if not isinstance(x, basestring): x = unicode(x) yield x.encode("utf-8") - keyed = PipelinedRDD(self, func) + keyed = self.mapPartitionsWithIndex(func) keyed._bypass_serializer = True keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) @@ -1382,7 +1382,7 @@ def add_shuffle_key(split, iterator): yield pack_long(split) yield outputSerializer.dumps(items) - keyed = PipelinedRDD(self, add_shuffle_key) + keyed = self.mapPartitionsWithIndex(add_shuffle_key) keyed._bypass_serializer = True with _JavaStackTrace(self.context) as st: pairRDD = self.ctx._jvm.PairwiseRDD( diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 9388ead5eaad3..f840475ffaf70 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -15,7 +15,17 @@ # limitations under the License. # + +import sys +import types +import itertools +import warnings +import decimal +import datetime +import keyword import warnings +from array import array +from operator import itemgetter from pyspark.rdd import RDD, PipelinedRDD from pyspark.serializers import BatchedSerializer, PickleSerializer @@ -26,10 +36,30 @@ "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", - "SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", "SchemaRDD", "Row"] + "SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", + "SchemaRDD", "Row"] + + +class DataType(object): + """Spark SQL DataType""" + + def __repr__(self): + return self.__class__.__name__ + + def __hash__(self): + return hash(str(self)) + + def __eq__(self, other): + return (isinstance(other, self.__class__) and + self.__dict__ == other.__dict__) + + def __ne__(self, other): + return not self.__eq__(other) class PrimitiveTypeSingleton(type): + """Metaclass for PrimitiveType""" + _instances = {} def __call__(cls): @@ -38,148 +68,105 @@ def __call__(cls): return cls._instances[cls] -class StringType(object): +class PrimitiveType(DataType): + """Spark SQL PrimitiveType""" + + __metaclass__ = PrimitiveTypeSingleton + + def __eq__(self, other): + # because they should be the same object + return self is other + + +class StringType(PrimitiveType): """Spark SQL StringType The data type representing string values. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "StringType" -class BinaryType(object): +class BinaryType(PrimitiveType): """Spark SQL BinaryType The data type representing bytearray values. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "BinaryType" -class BooleanType(object): +class BooleanType(PrimitiveType): """Spark SQL BooleanType The data type representing bool values. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "BooleanType" -class TimestampType(object): +class TimestampType(PrimitiveType): """Spark SQL TimestampType The data type representing datetime.datetime values. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "TimestampType" -class DecimalType(object): +class DecimalType(PrimitiveType): """Spark SQL DecimalType The data type representing decimal.Decimal values. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "DecimalType" -class DoubleType(object): +class DoubleType(PrimitiveType): """Spark SQL DoubleType The data type representing float values. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "DoubleType" -class FloatType(object): +class FloatType(PrimitiveType): """Spark SQL FloatType The data type representing single precision floating-point values. - """ - __metaclass__ = PrimitiveTypeSingleton - def __repr__(self): - return "FloatType" - -class ByteType(object): +class ByteType(PrimitiveType): """Spark SQL ByteType The data type representing int values with 1 singed byte. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "ByteType" -class IntegerType(object): +class IntegerType(PrimitiveType): """Spark SQL IntegerType The data type representing int values. - """ - __metaclass__ = PrimitiveTypeSingleton - def __repr__(self): - return "IntegerType" - -class LongType(object): +class LongType(PrimitiveType): """Spark SQL LongType - The data type representing long values. If the any value is beyond the range of - [-9223372036854775808, 9223372036854775807], please use DecimalType. - + The data type representing long values. If the any value is + beyond the range of [-9223372036854775808, 9223372036854775807], + please use DecimalType. """ - __metaclass__ = PrimitiveTypeSingleton - def __repr__(self): - return "LongType" - -class ShortType(object): +class ShortType(PrimitiveType): """Spark SQL ShortType The data type representing int values with 2 signed bytes. - """ - __metaclass__ = PrimitiveTypeSingleton - - def __repr__(self): - return "ShortType" -class ArrayType(object): +class ArrayType(DataType): """Spark SQL ArrayType - The data type representing list values. - An ArrayType object comprises two fields, elementType (a DataType) and containsNull (a bool). + The data type representing list values. An ArrayType object + comprises two fields, elementType (a DataType) and containsNull (a bool). The field of elementType is used to specify the type of array elements. The field of containsNull is used to specify if the array has None values. """ + def __init__(self, elementType, containsNull=False): """Creates an ArrayType @@ -194,40 +181,39 @@ def __init__(self, elementType, containsNull=False): self.elementType = elementType self.containsNull = containsNull - def __repr__(self): - return "ArrayType(" + self.elementType.__repr__() + "," + \ - str(self.containsNull).lower() + ")" - - def __eq__(self, other): - return (isinstance(other, self.__class__) and - self.elementType == other.elementType and - self.containsNull == other.containsNull) - - def __ne__(self, other): - return not self.__eq__(other) + def __str__(self): + return "ArrayType(%s,%s)" % (self.elementType, + str(self.containsNull).lower()) -class MapType(object): +class MapType(DataType): """Spark SQL MapType - The data type representing dict values. - A MapType object comprises three fields, - keyType (a DataType), valueType (a DataType) and valueContainsNull (a bool). + The data type representing dict values. A MapType object comprises + three fields, keyType (a DataType), valueType (a DataType) and + valueContainsNull (a bool). + The field of keyType is used to specify the type of keys in the map. The field of valueType is used to specify the type of values in the map. - The field of valueContainsNull is used to specify if values of this map has None values. + The field of valueContainsNull is used to specify if values of this + map has None values. + For values of a MapType column, keys are not allowed to have None values. """ + def __init__(self, keyType, valueType, valueContainsNull=True): """Creates a MapType :param keyType: the data type of keys. :param valueType: the data type of values. - :param valueContainsNull: indicates whether values contains null values. + :param valueContainsNull: indicates whether values contains + null values. - >>> MapType(StringType, IntegerType) == MapType(StringType, IntegerType, True) + >>> (MapType(StringType, IntegerType) + ... == MapType(StringType, IntegerType, True)) True - >>> MapType(StringType, IntegerType, False) == MapType(StringType, FloatType) + >>> (MapType(StringType, IntegerType, False) + ... == MapType(StringType, FloatType)) False """ self.keyType = keyType @@ -235,39 +221,36 @@ def __init__(self, keyType, valueType, valueContainsNull=True): self.valueContainsNull = valueContainsNull def __repr__(self): - return "MapType(" + self.keyType.__repr__() + "," + \ - self.valueType.__repr__() + "," + \ - str(self.valueContainsNull).lower() + ")" + return "MapType(%s,%s,%s)" % (self.keyType, self.valueType, + str(self.valueContainsNull).lower()) - def __eq__(self, other): - return (isinstance(other, self.__class__) and - self.keyType == other.keyType and - self.valueType == other.valueType and - self.valueContainsNull == other.valueContainsNull) - def __ne__(self, other): - return not self.__eq__(other) - - -class StructField(object): +class StructField(DataType): """Spark SQL StructField Represents a field in a StructType. - A StructField object comprises three fields, name (a string), dataType (a DataType), - and nullable (a bool). The field of name is the name of a StructField. The field of - dataType specifies the data type of a StructField. - The field of nullable specifies if values of a StructField can contain None values. + A StructField object comprises three fields, name (a string), + dataType (a DataType) and nullable (a bool). The field of name + is the name of a StructField. The field of dataType specifies + the data type of a StructField. + + The field of nullable specifies if values of a StructField can + contain None values. """ + def __init__(self, name, dataType, nullable): """Creates a StructField :param name: the name of this field. :param dataType: the data type of this field. - :param nullable: indicates whether values of this field can be null. + :param nullable: indicates whether values of this field + can be null. - >>> StructField("f1", StringType, True) == StructField("f1", StringType, True) + >>> (StructField("f1", StringType, True) + ... == StructField("f1", StringType, True)) True - >>> StructField("f1", StringType, True) == StructField("f2", StringType, True) + >>> (StructField("f1", StringType, True) + ... == StructField("f2", StringType, True)) False """ self.name = name @@ -275,27 +258,18 @@ def __init__(self, name, dataType, nullable): self.nullable = nullable def __repr__(self): - return "StructField(" + self.name + "," + \ - self.dataType.__repr__() + "," + \ - str(self.nullable).lower() + ")" + return "StructField(%s,%s,%s)" % (self.name, self.dataType, + str(self.nullable).lower()) - def __eq__(self, other): - return (isinstance(other, self.__class__) and - self.name == other.name and - self.dataType == other.dataType and - self.nullable == other.nullable) - def __ne__(self, other): - return not self.__eq__(other) - - -class StructType(object): +class StructType(DataType): """Spark SQL StructType - The data type representing namedtuple values. + The data type representing rows. A StructType object comprises a list of L{StructField}s. """ + def __init__(self, fields): """Creates a StructType @@ -312,15 +286,8 @@ def __init__(self, fields): self.fields = fields def __repr__(self): - return "StructType(List(" + \ - ",".join([field.__repr__() for field in self.fields]) + "))" - - def __eq__(self, other): - return (isinstance(other, self.__class__) and - self.fields == other.fields) - - def __ne__(self, other): - return not self.__eq__(other) + return ("StructType(List(%s))" % + ",".join(str(field) for field in self.fields)) def _parse_datatype_list(datatype_list_string): @@ -347,34 +314,19 @@ def _parse_datatype_list(datatype_list_string): return datatype_list +_all_primitive_types = dict((k, v) for k, v in globals().iteritems() + if type(v) is PrimitiveTypeSingleton and v.__base__ == PrimitiveType) + + def _parse_datatype_string(datatype_string): """Parses the given data type string. >>> def check_datatype(datatype): - ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.__repr__()) - ... python_datatype = _parse_datatype_string(scala_datatype.toString()) + ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(str(datatype)) + ... python_datatype = _parse_datatype_string( + ... scala_datatype.toString()) ... return datatype == python_datatype - >>> check_datatype(StringType()) - True - >>> check_datatype(BinaryType()) - True - >>> check_datatype(BooleanType()) - True - >>> check_datatype(TimestampType()) - True - >>> check_datatype(DecimalType()) - True - >>> check_datatype(DoubleType()) - True - >>> check_datatype(FloatType()) - True - >>> check_datatype(ByteType()) - True - >>> check_datatype(IntegerType()) - True - >>> check_datatype(LongType()) - True - >>> check_datatype(ShortType()) + >>> all(check_datatype(cls()) for cls in _all_primitive_types.values()) True >>> # Simple ArrayType. >>> simple_arraytype = ArrayType(StringType(), True) @@ -405,70 +357,525 @@ def _parse_datatype_string(datatype_string): >>> check_datatype(complex_arraytype) True >>> # Complex MapType. - >>> complex_maptype = MapType(complex_structtype, complex_arraytype, False) + >>> complex_maptype = MapType(complex_structtype, + ... complex_arraytype, False) >>> check_datatype(complex_maptype) True """ - left_bracket_index = datatype_string.find("(") - if left_bracket_index == -1: + index = datatype_string.find("(") + if index == -1: # It is a primitive type. - left_bracket_index = len(datatype_string) - type_or_field = datatype_string[:left_bracket_index] - rest_part = datatype_string[left_bracket_index+1:len(datatype_string)-1].strip() - if type_or_field == "StringType": - return StringType() - elif type_or_field == "BinaryType": - return BinaryType() - elif type_or_field == "BooleanType": - return BooleanType() - elif type_or_field == "TimestampType": - return TimestampType() - elif type_or_field == "DecimalType": - return DecimalType() - elif type_or_field == "DoubleType": - return DoubleType() - elif type_or_field == "FloatType": - return FloatType() - elif type_or_field == "ByteType": - return ByteType() - elif type_or_field == "IntegerType": - return IntegerType() - elif type_or_field == "LongType": - return LongType() - elif type_or_field == "ShortType": - return ShortType() + index = len(datatype_string) + type_or_field = datatype_string[:index] + rest_part = datatype_string[index + 1:len(datatype_string) - 1].strip() + + if type_or_field in _all_primitive_types: + return _all_primitive_types[type_or_field]() + elif type_or_field == "ArrayType": last_comma_index = rest_part.rfind(",") containsNull = True - if rest_part[last_comma_index+1:].strip().lower() == "false": + if rest_part[last_comma_index + 1:].strip().lower() == "false": containsNull = False - elementType = _parse_datatype_string(rest_part[:last_comma_index].strip()) + elementType = _parse_datatype_string( + rest_part[:last_comma_index].strip()) return ArrayType(elementType, containsNull) + elif type_or_field == "MapType": last_comma_index = rest_part.rfind(",") valueContainsNull = True - if rest_part[last_comma_index+1:].strip().lower() == "false": + if rest_part[last_comma_index + 1:].strip().lower() == "false": valueContainsNull = False - keyType, valueType = _parse_datatype_list(rest_part[:last_comma_index].strip()) + keyType, valueType = _parse_datatype_list( + rest_part[:last_comma_index].strip()) return MapType(keyType, valueType, valueContainsNull) + elif type_or_field == "StructField": first_comma_index = rest_part.find(",") name = rest_part[:first_comma_index].strip() last_comma_index = rest_part.rfind(",") nullable = True - if rest_part[last_comma_index+1:].strip().lower() == "false": + if rest_part[last_comma_index + 1:].strip().lower() == "false": nullable = False dataType = _parse_datatype_string( - rest_part[first_comma_index+1:last_comma_index].strip()) + rest_part[first_comma_index + 1:last_comma_index].strip()) return StructField(name, dataType, nullable) + elif type_or_field == "StructType": # rest_part should be in the format like # List(StructField(field1,IntegerType,false)). - field_list_string = rest_part[rest_part.find("(")+1:-1] + field_list_string = rest_part[rest_part.find("(") + 1:-1] fields = _parse_datatype_list(field_list_string) return StructType(fields) +# Mapping Python types to Spark SQL DateType +_type_mappings = { + bool: BooleanType, + int: IntegerType, + long: LongType, + float: DoubleType, + str: StringType, + unicode: StringType, + decimal.Decimal: DecimalType, + datetime.datetime: TimestampType, + datetime.date: TimestampType, + datetime.time: TimestampType, +} + + +def _infer_type(obj): + """Infer the DataType from obj""" + if obj is None: + raise ValueError("Can not infer type for None") + + dataType = _type_mappings.get(type(obj)) + if dataType is not None: + return dataType() + + if isinstance(obj, dict): + if not obj: + raise ValueError("Can not infer type for empty dict") + key, value = obj.iteritems().next() + return MapType(_infer_type(key), _infer_type(value), True) + elif isinstance(obj, (list, array)): + if not obj: + raise ValueError("Can not infer type for empty list/array") + return ArrayType(_infer_type(obj[0]), True) + else: + try: + return _infer_schema(obj) + except ValueError: + raise ValueError("not supported type: %s" % type(obj)) + + +def _infer_schema(row): + """Infer the schema from dict/namedtuple/object""" + if isinstance(row, dict): + items = sorted(row.items()) + + elif isinstance(row, tuple): + if hasattr(row, "_fields"): # namedtuple + items = zip(row._fields, tuple(row)) + elif hasattr(row, "__FIELDS__"): # Row + items = zip(row.__FIELDS__, tuple(row)) + elif all(isinstance(x, tuple) and len(x) == 2 for x in row): + items = row + else: + raise ValueError("Can't infer schema from tuple") + + elif hasattr(row, "__dict__"): # object + items = sorted(row.__dict__.items()) + + else: + raise ValueError("Can not infer schema for type: %s" % type(row)) + + fields = [StructField(k, _infer_type(v), True) for k, v in items] + return StructType(fields) + + +def _create_converter(obj, dataType): + """Create an converter to drop the names of fields in obj """ + if not _has_struct(dataType): + return lambda x: x + + elif isinstance(dataType, ArrayType): + conv = _create_converter(obj[0], dataType.elementType) + return lambda row: map(conv, row) + + elif isinstance(dataType, MapType): + value = obj.values()[0] + conv = _create_converter(value, dataType.valueType) + return lambda row: dict((k, conv(v)) for k, v in row.iteritems()) + + # dataType must be StructType + names = [f.name for f in dataType.fields] + + if isinstance(obj, dict): + conv = lambda o: tuple(o.get(n) for n in names) + + elif isinstance(obj, tuple): + if hasattr(obj, "_fields"): # namedtuple + conv = tuple + elif hasattr(obj, "__FIELDS__"): + conv = tuple + elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): + conv = lambda o: tuple(v for k, v in o) + else: + raise ValueError("unexpected tuple") + + elif hasattr(obj, "__dict__"): # object + conv = lambda o: [o.__dict__.get(n, None) for n in names] + + nested = any(_has_struct(f.dataType) for f in dataType.fields) + if not nested: + return conv + + row = conv(obj) + convs = [_create_converter(v, f.dataType) + for v, f in zip(row, dataType.fields)] + + def nested_conv(row): + return tuple(f(v) for f, v in zip(convs, conv(row))) + + return nested_conv + + +def _drop_schema(rows, schema): + """ all the names of fields, becoming tuples""" + iterator = iter(rows) + row = iterator.next() + converter = _create_converter(row, schema) + yield converter(row) + for i in iterator: + yield converter(i) + + +_BRACKETS = {'(': ')', '[': ']', '{': '}'} + + +def _split_schema_abstract(s): + """ + split the schema abstract into fields + + >>> _split_schema_abstract("a b c") + ['a', 'b', 'c'] + >>> _split_schema_abstract("a(a b)") + ['a(a b)'] + >>> _split_schema_abstract("a b[] c{a b}") + ['a', 'b[]', 'c{a b}'] + >>> _split_schema_abstract(" ") + [] + """ + + r = [] + w = '' + brackets = [] + for c in s: + if c == ' ' and not brackets: + if w: + r.append(w) + w = '' + else: + w += c + if c in _BRACKETS: + brackets.append(c) + elif c in _BRACKETS.values(): + if not brackets or c != _BRACKETS[brackets.pop()]: + raise ValueError("unexpected " + c) + + if brackets: + raise ValueError("brackets not closed: %s" % brackets) + if w: + r.append(w) + return r + + +def _parse_field_abstract(s): + """ + Parse a field in schema abstract + + >>> _parse_field_abstract("a") + StructField(a,None,true) + >>> _parse_field_abstract("b(c d)") + StructField(b,StructType(...c,None,true),StructField(d... + >>> _parse_field_abstract("a[]") + StructField(a,ArrayType(None,true),true) + >>> _parse_field_abstract("a{[]}") + StructField(a,MapType(None,ArrayType(None,true),true),true) + """ + if set(_BRACKETS.keys()) & set(s): + idx = min((s.index(c) for c in _BRACKETS if c in s)) + name = s[:idx] + return StructField(name, _parse_schema_abstract(s[idx:]), True) + else: + return StructField(s, None, True) + + +def _parse_schema_abstract(s): + """ + parse abstract into schema + + >>> _parse_schema_abstract("a b c") + StructType...a...b...c... + >>> _parse_schema_abstract("a[b c] b{}") + StructType...a,ArrayType...b...c...b,MapType... + >>> _parse_schema_abstract("c{} d{a b}") + StructType...c,MapType...d,MapType...a...b... + >>> _parse_schema_abstract("a b(t)").fields[1] + StructField(b,StructType(List(StructField(t,None,true))),true) + """ + s = s.strip() + if not s: + return + + elif s.startswith('('): + return _parse_schema_abstract(s[1:-1]) + + elif s.startswith('['): + return ArrayType(_parse_schema_abstract(s[1:-1]), True) + + elif s.startswith('{'): + return MapType(None, _parse_schema_abstract(s[1:-1])) + + parts = _split_schema_abstract(s) + fields = [_parse_field_abstract(p) for p in parts] + return StructType(fields) + + +def _infer_schema_type(obj, dataType): + """ + Fill the dataType with types infered from obj + + >>> schema = _parse_schema_abstract("a b c") + >>> row = (1, 1.0, "str") + >>> _infer_schema_type(row, schema) + StructType...IntegerType...DoubleType...StringType... + >>> row = [[1], {"key": (1, 2.0)}] + >>> schema = _parse_schema_abstract("a[] b{c d}") + >>> _infer_schema_type(row, schema) + StructType...a,ArrayType...b,MapType(StringType,...c,IntegerType... + """ + if dataType is None: + return _infer_type(obj) + + if not obj: + raise ValueError("Can not infer type from empty value") + + if isinstance(dataType, ArrayType): + eType = _infer_schema_type(obj[0], dataType.elementType) + return ArrayType(eType, True) + + elif isinstance(dataType, MapType): + k, v = obj.iteritems().next() + return MapType(_infer_type(k), + _infer_schema_type(v, dataType.valueType)) + + elif isinstance(dataType, StructType): + fs = dataType.fields + assert len(fs) == len(obj), \ + "Obj(%s) have different length with fields(%s)" % (obj, fs) + fields = [StructField(f.name, _infer_schema_type(o, f.dataType), True) + for o, f in zip(obj, fs)] + return StructType(fields) + + else: + raise ValueError("Unexpected dataType: %s" % dataType) + + +_acceptable_types = { + BooleanType: (bool,), + ByteType: (int, long), + ShortType: (int, long), + IntegerType: (int, long), + LongType: (int, long), + FloatType: (float,), + DoubleType: (float,), + DecimalType: (decimal.Decimal,), + StringType: (str, unicode), + TimestampType: (datetime.datetime, datetime.time, datetime.date), + ArrayType: (list, tuple, array), + MapType: (dict,), + StructType: (tuple, list), +} + +def _verify_type(obj, dataType): + """ + Verify the type of obj against dataType, raise an exception if + they do not match. + + >>> _verify_type(None, StructType([])) + >>> _verify_type("", StringType()) + >>> _verify_type(0, IntegerType()) + >>> _verify_type(range(3), ArrayType(ShortType())) + >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + TypeError:... + >>> _verify_type({}, MapType(StringType(), IntegerType())) + >>> _verify_type((), StructType([])) + >>> _verify_type([], StructType([])) + >>> _verify_type([1], StructType([])) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + ValueError:... + """ + # all objects are nullable + if obj is None: + return + + _type = type(dataType) + if _type not in _acceptable_types: + return + + if type(obj) not in _acceptable_types[_type]: + raise TypeError("%s can not accept abject in type %s" + % (dataType, type(obj))) + + if isinstance(dataType, ArrayType): + for i in obj: + _verify_type(i, dataType.elementType) + + elif isinstance(dataType, MapType): + for k, v in obj.iteritems(): + _verify_type(k, dataType.keyType) + _verify_type(v, dataType.valueType) + + elif isinstance(dataType, StructType): + if len(obj) != len(dataType.fields): + raise ValueError("Length of object (%d) does not match with" + "length of fields (%d)" % (len(obj), len(dataType.fields))) + for v, f in zip(obj, dataType.fields): + _verify_type(v, f.dataType) + + +_cached_cls = {} + + +def _restore_object(dataType, obj): + """ Restore object during unpickling. """ + # use id(dataType) as key to speed up lookup in dict + # Because of batched pickling, dataType will be the + # same object in mose cases. + k = id(dataType) + cls = _cached_cls.get(k) + if cls is None: + # use dataType as key to avoid create multiple class + cls = _cached_cls.get(dataType) + if cls is None: + cls = _create_cls(dataType) + _cached_cls[dataType] = cls + _cached_cls[k] = cls + return cls(obj) + + +def _create_object(cls, v): + """ Create an customized object with class `cls`. """ + return cls(v) if v is not None else v + + +def _create_getter(dt, i): + """ Create a getter for item `i` with schema """ + cls = _create_cls(dt) + + def getter(self): + return _create_object(cls, self[i]) + + return getter + + +def _has_struct(dt): + """Return whether `dt` is or has StructType in it""" + if isinstance(dt, StructType): + return True + elif isinstance(dt, ArrayType): + return _has_struct(dt.elementType) + elif isinstance(dt, MapType): + return _has_struct(dt.valueType) + return False + + +def _create_properties(fields): + """Create properties according to fields""" + ps = {} + for i, f in enumerate(fields): + name = f.name + if (name.startswith("__") and name.endswith("__") + or keyword.iskeyword(name)): + warnings.warn("field name %s can not be accessed in Python," + "use position to access it instead" % name) + if _has_struct(f.dataType): + # delay creating object until accessing it + getter = _create_getter(f.dataType, i) + else: + getter = itemgetter(i) + ps[name] = property(getter) + return ps + + +def _create_cls(dataType): + """ + Create an class by dataType + + The created class is similar to namedtuple, but can have nested schema. + + >>> schema = _parse_schema_abstract("a b c") + >>> row = (1, 1.0, "str") + >>> schema = _infer_schema_type(row, schema) + >>> obj = _create_cls(schema)(row) + >>> import pickle + >>> pickle.loads(pickle.dumps(obj)) + Row(a=1, b=1.0, c='str') + + >>> row = [[1], {"key": (1, 2.0)}] + >>> schema = _parse_schema_abstract("a[] b{c d}") + >>> schema = _infer_schema_type(row, schema) + >>> obj = _create_cls(schema)(row) + >>> pickle.loads(pickle.dumps(obj)) + Row(a=[1], b={'key': Row(c=1, d=2.0)}) + """ + + if isinstance(dataType, ArrayType): + cls = _create_cls(dataType.elementType) + + class List(list): + + def __getitem__(self, i): + # create object with datetype + return _create_object(cls, list.__getitem__(self, i)) + + def __repr__(self): + # call collect __repr__ for nested objects + return "[%s]" % (", ".join(repr(self[i]) + for i in range(len(self)))) + + def __reduce__(self): + return list.__reduce__(self) + + return List + + elif isinstance(dataType, MapType): + vcls = _create_cls(dataType.valueType) + + class Dict(dict): + + def __getitem__(self, k): + # create object with datetype + return _create_object(vcls, dict.__getitem__(self, k)) + + def __repr__(self): + # call collect __repr__ for nested objects + return "{%s}" % (", ".join("%r: %r" % (k, self[k]) + for k in self)) + + def __reduce__(self): + return dict.__reduce__(self) + + return Dict + + elif not isinstance(dataType, StructType): + raise Exception("unexpected data type: %s" % dataType) + + class Row(tuple): + """ Row in SchemaRDD """ + __DATATYPE__ = dataType + __FIELDS__ = tuple(f.name for f in dataType.fields) + __slots__ = () + + # create property for fast access + locals().update(_create_properties(dataType.fields)) + + def __repr__(self): + # call collect __repr__ for nested objects + return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) + for n in self.__FIELDS__)) + + def __reduce__(self): + return (_restore_object, (self.__DATATYPE__, tuple(self))) + + return Row + + class SQLContext: """Main entry point for SparkSQL functionality. @@ -485,7 +892,7 @@ def __init__(self, sparkContext, sqlContext=None): >>> sqlCtx.inferSchema(srdd) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... - ValueError:... + TypeError:... >>> bad_rdd = sc.parallelize([1,2,3]) >>> sqlCtx.inferSchema(bad_rdd) # doctest: +IGNORE_EXCEPTION_DETAIL @@ -494,18 +901,22 @@ def __init__(self, sparkContext, sqlContext=None): ValueError:... >>> from datetime import datetime - >>> allTypes = sc.parallelize([{"int": 1, "string": "string", "double": 1.0, "long": 1L, - ... "boolean": True, "time": datetime(2010, 1, 1, 1, 1, 1), "dict": {"a": 1}, - ... "list": [1, 2, 3]}]) - >>> srdd = sqlCtx.inferSchema(allTypes).map(lambda x: (x.int, x.string, x.double, x.long, - ... x.boolean, x.time, x.dict["a"], x.list)) - >>> srdd.collect()[0] - (1, u'string', 1.0, 1, True, datetime.datetime(2010, 1, 1, 1, 1, 1), 1, [1, 2, 3]) + >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, + ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), + ... time=datetime(2014, 8, 1, 14, 1, 5))]) + >>> srdd = sqlCtx.inferSchema(allTypes) + >>> srdd.registerAsTable("allTypes") + >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' + ... 'from allTypes where b and i > 0').collect() + [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] + >>> srdd.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, + ... x.row.a, x.list)).collect() + [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ self._sc = sparkContext self._jsc = self._sc._jsc self._jvm = self._sc._jvm - self._pythonToJavaMap = self._jvm.PythonRDD.pythonToJavaMap + self._pythonToJava = self._jvm.PythonRDD.pythonToJavaArray if sqlContext: self._scala_SQLContext = sqlContext @@ -522,71 +933,123 @@ def _ssql_ctx(self): return self._scala_SQLContext def inferSchema(self, rdd): - """Infer and apply a schema to an RDD of L{dict}s. + """Infer and apply a schema to an RDD of L{Row}s. + + We peek at the first row of the RDD to determine the fields' names + and types. Nested collections are supported, which include array, + dict, list, Row, tuple, namedtuple, or object. - We peek at the first row of the RDD to determine the fields names - and types, and then use that to extract all the dictionaries. Nested - collections are supported, which include array, dict, list, set, and - tuple. + All the rows in `rdd` should have the same type with the first one, + or it will cause runtime exceptions. + Each row could be L{pyspark.sql.Row} object or namedtuple or objects, + using dict is deprecated. + + >>> rdd = sc.parallelize( + ... [Row(field1=1, field2="row1"), + ... Row(field1=2, field2="row2"), + ... Row(field1=3, field2="row3")]) >>> srdd = sqlCtx.inferSchema(rdd) - >>> srdd.collect() == [{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, - ... {"field1" : 3, "field2": "row3"}] - True + >>> srdd.collect()[0] + Row(field1=1, field2=u'row1') - >>> from array import array + >>> NestedRow = Row("f1", "f2") + >>> nestedRdd1 = sc.parallelize([ + ... NestedRow(array('i', [1, 2]), {"row1": 1.0}), + ... NestedRow(array('i', [2, 3]), {"row2": 2.0})]) >>> srdd = sqlCtx.inferSchema(nestedRdd1) - >>> srdd.collect() == [{"f1" : [1, 2], "f2" : {"row1" : 1.0}}, - ... {"f1" : [2, 3], "f2" : {"row2" : 2.0}}] - True + >>> srdd.collect() + [Row(f1=[1, 2], f2={u'row1': 1.0}), ..., f2={u'row2': 2.0})] + >>> nestedRdd2 = sc.parallelize([ + ... NestedRow([[1, 2], [2, 3]], [1, 2]), + ... NestedRow([[2, 3], [3, 4]], [2, 3])]) >>> srdd = sqlCtx.inferSchema(nestedRdd2) - >>> srdd.collect() == [{"f1" : [[1, 2], [2, 3]], "f2" : [1, 2]}, - ... {"f1" : [[2, 3], [3, 4]], "f2" : [2, 3]}] - True + >>> srdd.collect() + [Row(f1=[[1, 2], [2, 3]], f2=[1, 2]), ..., f2=[2, 3])] """ - if (rdd.__class__ is SchemaRDD): - raise ValueError("Cannot apply schema to %s" % SchemaRDD.__name__) - elif not isinstance(rdd.first(), dict): - raise ValueError("Only RDDs with dictionaries can be converted to %s: %s" % - (SchemaRDD.__name__, rdd.first())) - jrdd = self._pythonToJavaMap(rdd._jrdd) - srdd = self._ssql_ctx.inferSchema(jrdd.rdd()) - return SchemaRDD(srdd, self) + if isinstance(rdd, SchemaRDD): + raise TypeError("Cannot apply schema to SchemaRDD") + + first = rdd.first() + if not first: + raise ValueError("The first row in RDD is empty, " + "can not infer schema") + if type(first) is dict: + warnings.warn("Using RDD of dict to inferSchema is deprecated") + + schema = _infer_schema(first) + rdd = rdd.mapPartitions(lambda rows: _drop_schema(rows, schema)) + return self.applySchema(rdd, schema) def applySchema(self, rdd, schema): - """Applies the given schema to the given RDD of L{dict}s. + """ + Applies the given schema to the given RDD of L{tuple} or L{list}s. + + These tuples or lists can contain complex nested structures like + lists, maps or nested rows. + + The schema should be a StructType. + It is important that the schema matches the types of the objects + in each row or exceptions could be thrown at runtime. + + >>> rdd2 = sc.parallelize([(1, "row1"), (2, "row2"), (3, "row3")]) >>> schema = StructType([StructField("field1", IntegerType(), False), ... StructField("field2", StringType(), False)]) - >>> srdd = sqlCtx.applySchema(rdd, schema) + >>> srdd = sqlCtx.applySchema(rdd2, schema) >>> sqlCtx.registerRDDAsTable(srdd, "table1") >>> srdd2 = sqlCtx.sql("SELECT * from table1") - >>> srdd2.collect() == [{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, - ... {"field1" : 3, "field2": "row3"}] - True + >>> srdd2.collect() + [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] + >>> from datetime import datetime - >>> rdd = sc.parallelize([{"byte": 127, "short": -32768, "float": 1.0, - ... "time": datetime(2010, 1, 1, 1, 1, 1), "map": {"a": 1}, "struct": {"b": 2}, - ... "list": [1, 2, 3]}]) + >>> rdd = sc.parallelize([(127, -32768, 1.0, + ... datetime(2010, 1, 1, 1, 1, 1), + ... {"a": 1}, (2,), [1, 2, 3], None)]) >>> schema = StructType([ ... StructField("byte", ByteType(), False), ... StructField("short", ShortType(), False), ... StructField("float", FloatType(), False), ... StructField("time", TimestampType(), False), - ... StructField("map", MapType(StringType(), IntegerType(), False), False), - ... StructField("struct", StructType([StructField("b", ShortType(), False)]), False), + ... StructField("map", + ... MapType(StringType(), IntegerType(), False), False), + ... StructField("struct", + ... StructType([StructField("b", ShortType(), False)]), False), ... StructField("list", ArrayType(ByteType(), False), False), ... StructField("null", DoubleType(), True)]) >>> srdd = sqlCtx.applySchema(rdd, schema).map( - ... lambda x: ( - ... x.byte, x.short, x.float, x.time, x.map["a"], x.struct["b"], x.list, x.null)) + ... lambda x: (x.byte, x.short, x.float, x.time, + ... x.map["a"], x.struct.b, x.list, x.null)) >>> srdd.collect()[0] - (127, -32768, 1.0, datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + (127, -32768, 1.0, ...(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + + >>> rdd = sc.parallelize([(127, -32768, 1.0, + ... datetime(2010, 1, 1, 1, 1, 1), + ... {"a": 1}, (2,), [1, 2, 3])]) + >>> abstract = "byte short float time map{} struct(b) list[]" + >>> schema = _parse_schema_abstract(abstract) + >>> typedSchema = _infer_schema_type(rdd.first(), schema) + >>> srdd = sqlCtx.applySchema(rdd, typedSchema) + >>> srdd.collect() + [Row(byte=127, short=-32768, float=1.0, time=..., list=[1, 2, 3])] """ - jrdd = self._pythonToJavaMap(rdd._jrdd) - srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.__repr__()) + + if isinstance(rdd, SchemaRDD): + raise TypeError("Cannot apply schema to SchemaRDD") + + if not isinstance(schema, StructType): + raise TypeError("schema should be StructType") + + # take the first few rows to verify schema + rows = rdd.take(10) + for row in rows: + _verify_type(row, schema) + + batched = isinstance(rdd._jrdd_deserializer, BatchedSerializer) + jrdd = self._pythonToJava(rdd._jrdd, batched) + srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), str(schema)) return SchemaRDD(srdd, self) def registerRDDAsTable(self, rdd, tableName): @@ -620,10 +1083,15 @@ def parquetFile(self, path): return SchemaRDD(jschema_rdd, self) def jsonFile(self, path, schema=None): - """Loads a text file storing one JSON object per line as a L{SchemaRDD}. + """ + Loads a text file storing one JSON object per line as a + L{SchemaRDD}. - If the schema is provided, applies the given schema to this JSON dataset. - Otherwise, it goes through the entire dataset once to determine the schema. + If the schema is provided, applies the given schema to this + JSON dataset. + + Otherwise, it goes through the entire dataset once to determine + the schema. >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() @@ -635,94 +1103,100 @@ def jsonFile(self, path, schema=None): >>> srdd1 = sqlCtx.jsonFile(jsonFile) >>> sqlCtx.registerRDDAsTable(srdd1, "table1") >>> srdd2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") - >>> srdd2.collect() == [ - ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, - ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, - ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] - True + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table1") + >>> for r in srdd2.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) >>> srdd3 = sqlCtx.jsonFile(jsonFile, srdd1.schema()) >>> sqlCtx.registerRDDAsTable(srdd3, "table2") >>> srdd4 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table2") - >>> srdd4.collect() == [ - ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, - ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, - ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] - True + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table2") + >>> for r in srdd4.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) >>> schema = StructType([ ... StructField("field2", StringType(), True), ... StructField("field3", ... StructType([ - ... StructField("field5", ArrayType(IntegerType(), False), True)]), False)]) + ... StructField("field5", + ... ArrayType(IntegerType(), False), True)]), False)]) >>> srdd5 = sqlCtx.jsonFile(jsonFile, schema) >>> sqlCtx.registerRDDAsTable(srdd5, "table3") >>> srdd6 = sqlCtx.sql( - ... "SELECT field2 AS f1, field3.field5 as f2, field3.field5[0] as f3 from table3") - >>> srdd6.collect() == [ - ... {"f1": "row1", "f2": None, "f3": None}, - ... {"f1": None, "f2": [10, 11], "f3": 10}, - ... {"f1": "row3", "f2": [], "f3": None}] - True + ... "SELECT field2 AS f1, field3.field5 as f2, " + ... "field3.field5[0] as f3 from table3") + >>> srdd6.collect() + [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] """ if schema is None: jschema_rdd = self._ssql_ctx.jsonFile(path) else: - scala_datatype = self._ssql_ctx.parseDataType(schema.__repr__()) + scala_datatype = self._ssql_ctx.parseDataType(str(schema)) jschema_rdd = self._ssql_ctx.jsonFile(path, scala_datatype) return SchemaRDD(jschema_rdd, self) def jsonRDD(self, rdd, schema=None): """Loads an RDD storing one JSON object per string as a L{SchemaRDD}. - If the schema is provided, applies the given schema to this JSON dataset. - Otherwise, it goes through the entire dataset once to determine the schema. + If the schema is provided, applies the given schema to this + JSON dataset. + + Otherwise, it goes through the entire dataset once to determine + the schema. >>> srdd1 = sqlCtx.jsonRDD(json) >>> sqlCtx.registerRDDAsTable(srdd1, "table1") >>> srdd2 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table1") - >>> srdd2.collect() == [ - ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, - ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, - ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] - True + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table1") + >>> for r in srdd2.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) >>> srdd3 = sqlCtx.jsonRDD(json, srdd1.schema()) >>> sqlCtx.registerRDDAsTable(srdd3, "table2") >>> srdd4 = sqlCtx.sql( - ... "SELECT field1 AS f1, field2 as f2, field3 as f3, field6 as f4 from table2") - >>> srdd4.collect() == [ - ... {"f1":1, "f2":"row1", "f3":{"field4":11, "field5": None}, "f4":None}, - ... {"f1":2, "f2":None, "f3":{"field4":22, "field5": [10, 11]}, "f4":[{"field7": "row2"}]}, - ... {"f1":None, "f2":"row3", "f3":{"field4":33, "field5": []}, "f4":None}] - True + ... "SELECT field1 AS f1, field2 as f2, field3 as f3, " + ... "field6 as f4 from table2") + >>> for r in srdd4.collect(): + ... print r + Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) + Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) + Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) >>> schema = StructType([ ... StructField("field2", StringType(), True), ... StructField("field3", ... StructType([ - ... StructField("field5", ArrayType(IntegerType(), False), True)]), False)]) + ... StructField("field5", + ... ArrayType(IntegerType(), False), True)]), False)]) >>> srdd5 = sqlCtx.jsonRDD(json, schema) >>> sqlCtx.registerRDDAsTable(srdd5, "table3") >>> srdd6 = sqlCtx.sql( - ... "SELECT field2 AS f1, field3.field5 as f2, field3.field5[0] as f3 from table3") - >>> srdd6.collect() == [ - ... {"f1": "row1", "f2": None, "f3": None}, - ... {"f1": None, "f2": [10, 11], "f3": 10}, - ... {"f1": "row3", "f2": [], "f3": None}] - True + ... "SELECT field2 AS f1, field3.field5 as f2, " + ... "field3.field5[0] as f3 from table3") + >>> srdd6.collect() + [Row(f1=u'row1', f2=None,...Row(f1=u'row3', f2=[], f3=None)] """ - def func(split, iterator): + + def func(iterator): for x in iterator: if not isinstance(x, basestring): x = unicode(x) yield x.encode("utf-8") - keyed = PipelinedRDD(rdd, func) + keyed = rdd.mapPartitions(func) keyed._bypass_serializer = True jrdd = keyed._jrdd.map(self._jvm.BytesToString()) if schema is None: jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) else: - scala_datatype = self._ssql_ctx.parseDataType(schema.__repr__()) + scala_datatype = self._ssql_ctx.parseDataType(str(schema)) jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) return SchemaRDD(jschema_rdd, self) @@ -732,9 +1206,8 @@ def sql(self, sqlQuery): >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.registerRDDAsTable(srdd, "table1") >>> srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") - >>> srdd2.collect() == [{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, - ... {"f1" : 3, "f2": "row3"}] - True + >>> srdd2.collect() + [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] """ return SchemaRDD(self._ssql_ctx.sql(sqlQuery), self) @@ -772,7 +1245,8 @@ def _ssql_ctx(self): self._scala_HiveContext = self._get_hive_ctx() return self._scala_HiveContext except Py4JError as e: - raise Exception("You must build Spark with Hive. Export 'SPARK_HIVE=true' and run " + raise Exception("You must build Spark with Hive. " + "Export 'SPARK_HIVE=true' and run " "sbt/sbt assembly", e) def _get_hive_ctx(self): @@ -780,13 +1254,15 @@ def _get_hive_ctx(self): def hiveql(self, hqlQuery): """ - Runs a query expressed in HiveQL, returning the result as a L{SchemaRDD}. + Runs a query expressed in HiveQL, returning the result as + a L{SchemaRDD}. """ return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery), self) def hql(self, hqlQuery): """ - Runs a query expressed in HiveQL, returning the result as a L{SchemaRDD}. + Runs a query expressed in HiveQL, returning the result as + a L{SchemaRDD}. """ return self.hiveql(hqlQuery) @@ -803,10 +1279,14 @@ class LocalHiveContext(HiveContext): ... supress = hiveCtx.hql("DROP TABLE src") ... except Exception: ... pass - >>> kv1 = os.path.join(os.environ["SPARK_HOME"], 'examples/src/main/resources/kv1.txt') - >>> supress = hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - >>> supress = hiveCtx.hql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" % kv1) - >>> results = hiveCtx.hql("FROM src SELECT value").map(lambda r: int(r.value.split('_')[1])) + >>> kv1 = os.path.join(os.environ["SPARK_HOME"], + ... 'examples/src/main/resources/kv1.txt') + >>> supress = hiveCtx.hql( + ... "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + >>> supress = hiveCtx.hql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" + ... % kv1) + >>> results = hiveCtx.hql("FROM src SELECT value" + ... ).map(lambda r: int(r.value.split('_')[1])) >>> num = results.count() >>> reduce_sum = results.reduce(lambda x, y: x + y) >>> num @@ -816,8 +1296,9 @@ class LocalHiveContext(HiveContext): """ def __init__(self, sparkContext, sqlContext=None): - HiveContext.__init__(self, sparkContext, sqlContext) - warnings.warn("LocalHiveContext is deprecated. Use HiveContext instead.", DeprecationWarning) + HiveContext.__init__(self, sparkContext, sqlContext) + warnings.warn("LocalHiveContext is deprecated. " + "Use HiveContext instead.", DeprecationWarning) def _get_hive_ctx(self): return self._jvm.LocalHiveContext(self._jsc.sc()) @@ -829,25 +1310,83 @@ def _get_hive_ctx(self): return self._jvm.TestHiveContext(self._jsc.sc()) -# TODO: Investigate if it is more efficient to use a namedtuple. One problem is that named tuples -# are custom classes that must be generated per Schema. -class Row(dict): - """A row in L{SchemaRDD}. +def _create_row(fields, values): + row = Row(*values) + row.__FIELDS__ = fields + return row + + +class Row(tuple): + """ + A row in L{SchemaRDD}. The fields in it can be accessed like attributes. + + Row can be used to create a row object by using named arguments, + the fields will be sorted by names. + + >>> row = Row(name="Alice", age=11) + >>> row + Row(age=11, name='Alice') + >>> row.name, row.age + ('Alice', 11) - An extended L{dict} that takes a L{dict} in its constructor, and - exposes those items as fields. + Row also can be used to create another Row like class, then it + could be used to create Row objects, such as - >>> r = Row({"hello" : "world", "foo" : "bar"}) - >>> r.hello - 'world' - >>> r.foo - 'bar' + >>> Person = Row("name", "age") + >>> Person + + >>> Person("Alice", 11) + Row(name='Alice', age=11) """ - def __init__(self, d): - d.update(self.__dict__) - self.__dict__ = d - dict.__init__(self, d) + def __new__(self, *args, **kwargs): + if args and kwargs: + raise ValueError("Can not use both args " + "and kwargs to create Row") + if args: + # create row class or objects + return tuple.__new__(self, args) + + elif kwargs: + # create row objects + names = sorted(kwargs.keys()) + values = tuple(kwargs[n] for n in names) + row = tuple.__new__(self, values) + row.__FIELDS__ = names + return row + + else: + raise ValueError("No args or kwargs") + + + # let obect acs like class + def __call__(self, *args): + """create new Row object""" + return _create_row(self, args) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + try: + # it will be slow when it has many fields, + # but this will not be used in normal cases + idx = self.__FIELDS__.index(item) + return self[idx] + except IndexError: + raise AttributeError(item) + + def __reduce__(self): + if hasattr(self, "__FIELDS__"): + return (_create_row, (self.__FIELDS__, tuple(self))) + else: + return tuple.__reduce__(self) + + def __repr__(self): + if hasattr(self, "__FIELDS__"): + return "Row(%s)" % ", ".join("%s=%r" % (k, v) + for k, v in zip(self.__FIELDS__, self)) + else: + return "" % ", ".join(self) class SchemaRDD(RDD): @@ -861,6 +1400,10 @@ class SchemaRDD(RDD): implementation is an RDD composed of Java objects. Instead it is converted to a PythonRDD in the JVM, on which Python operations can be done. + + This class receives raw tuples from Java but assigns a class to it in + all its data-collection methods (mapPartitionsWithIndex, collect, take, + etc) so that PySpark sees them as Row objects with named fields. """ def __init__(self, jschema_rdd, sql_ctx): @@ -871,7 +1414,8 @@ def __init__(self, jschema_rdd, sql_ctx): self.is_cached = False self.is_checkpointed = False self.ctx = self.sql_ctx._sc - self._jrdd_deserializer = self.ctx.serializer + # the _jrdd is created by javaToPython(), serialized by pickle + self._jrdd_deserializer = BatchedSerializer(PickleSerializer()) @property def _jrdd(self): @@ -881,7 +1425,7 @@ def _jrdd(self): L{pyspark.rdd.RDD} super class (map, filter, etc.). """ if not hasattr(self, '_lazy_jrdd'): - self._lazy_jrdd = self._toPython()._jrdd + self._lazy_jrdd = self._jschema_rdd.javaToPython() return self._lazy_jrdd @property @@ -931,7 +1475,8 @@ def saveAsTable(self, tableName): self._jschema_rdd.saveAsTable(tableName) def schema(self): - """Returns the schema of this SchemaRDD (represented by a L{StructType}).""" + """Returns the schema of this SchemaRDD (represented by + a L{StructType}).""" return _parse_datatype_string(self._jschema_rdd.schema().toString()) def schemaString(self): @@ -957,19 +1502,45 @@ def count(self): """ return self._jschema_rdd.count() - def _toPython(self): - # We have to import the Row class explicitly, so that the reference Pickler has is - # pyspark.sql.Row instead of __main__.Row - from pyspark.sql import Row - jrdd = self._jschema_rdd.javaToPython() - # TODO: This is inefficient, we should construct the Python Row object - # in Java land in the javaToPython function. May require a custom - # pickle serializer in Pyrolite - return RDD(jrdd, self._sc, BatchedSerializer( - PickleSerializer())).map(lambda d: Row(d)) - - # We override the default cache/persist/checkpoint behavior as we want to cache the underlying - # SchemaRDD object in the JVM, not the PythonRDD checkpointed by the super class + def collect(self): + """ + Return a list that contains all of the rows in this RDD. + + Each object in the list is on Row, the fields can be accessed as + attributes. + """ + rows = RDD.collect(self) + cls = _create_cls(self.schema()) + return map(cls, rows) + + # Convert each object in the RDD to a Row with the right class + # for this SchemaRDD, so that fields can be accessed as attributes. + def mapPartitionsWithIndex(self, f, preservesPartitioning=False): + """ + Return a new RDD by applying a function to each partition of this RDD, + while tracking the index of the original partition. + + >>> rdd = sc.parallelize([1, 2, 3, 4], 4) + >>> def f(splitIndex, iterator): yield splitIndex + >>> rdd.mapPartitionsWithIndex(f).sum() + 6 + """ + rdd = RDD(self._jrdd, self._sc, self._jrdd_deserializer) + + schema = self.schema() + import pickle + pickle.loads(pickle.dumps(schema)) + + def applySchema(_, it): + cls = _create_cls(schema) + return itertools.imap(cls, it) + + objrdd = rdd.mapPartitionsWithIndex(applySchema, preservesPartitioning) + return objrdd.mapPartitionsWithIndex(f, preservesPartitioning) + + # We override the default cache/persist/checkpoint behavior + # as we want to cache the underlying SchemaRDD object in the JVM, + # not the PythonRDD checkpointed by the super class def cache(self): self.is_cached = True self._jschema_rdd.cache() @@ -1024,7 +1595,8 @@ def subtract(self, other, numPartitions=None): if numPartitions is None: rdd = self._jschema_rdd.subtract(other._jschema_rdd) else: - rdd = self._jschema_rdd.subtract(other._jschema_rdd, numPartitions) + rdd = self._jschema_rdd.subtract(other._jschema_rdd, + numPartitions) return SchemaRDD(rdd, self.sql_ctx) else: raise ValueError("Can only subtract another SchemaRDD") @@ -1034,31 +1606,31 @@ def _test(): import doctest from array import array from pyspark.context import SparkContext - globs = globals().copy() + # let doctest run in pyspark.sql, so DataTypes can be picklable + import pyspark.sql + from pyspark.sql import Row, SQLContext + globs = pyspark.sql.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext('local[4]', 'PythonTest', batchSize=2) globs['sc'] = sc globs['sqlCtx'] = SQLContext(sc) globs['rdd'] = sc.parallelize( - [{"field1": 1, "field2": "row1"}, - {"field1": 2, "field2": "row2"}, - {"field1": 3, "field2": "row3"}] + [Row(field1=1, field2="row1"), + Row(field1=2, field2="row2"), + Row(field1=3, field2="row3")] ) jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', - '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]}, "field6":[{"field7": "row2"}]}', - '{"field1" : null, "field2": "row3", "field3":{"field4":33, "field5": []}}' + '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' + '"field6":[{"field7": "row2"}]}', + '{"field1" : null, "field2": "row3", ' + '"field3":{"field4":33, "field5": []}}' ] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) - globs['nestedRdd1'] = sc.parallelize([ - {"f1": array('i', [1, 2]), "f2": {"row1": 1.0}}, - {"f1": array('i', [2, 3]), "f2": {"row2": 2.0}}]) - globs['nestedRdd2'] = sc.parallelize([ - {"f1": [[1, 2], [2, 3]], "f2": [1, 2]}, - {"f1": [[2, 3], [3, 4]], "f2": [2, 3]}]) - (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + (failure_count, test_count) = doctest.testmod( + pyspark.sql, globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 86338752a21c1..dad71079c29b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -411,35 +411,6 @@ class SQLContext(@transient val sparkContext: SparkContext) """.stripMargin.trim } - /** - * Peek at the first row of the RDD and infer its schema. - * It is only used by PySpark. - */ - private[sql] def inferSchema(rdd: RDD[Map[String, _]]): SchemaRDD = { - import scala.collection.JavaConversions._ - - def typeOfComplexValue: PartialFunction[Any, DataType] = { - case c: java.util.Calendar => TimestampType - case c: java.util.List[_] => - ArrayType(typeOfObject(c.head)) - case c: java.util.Map[_, _] => - val (key, value) = c.head - MapType(typeOfObject(key), typeOfObject(value)) - case c if c.getClass.isArray => - val elem = c.asInstanceOf[Array[_]].head - ArrayType(typeOfObject(elem)) - case c => throw new Exception(s"Object of type $c cannot be used") - } - def typeOfObject = ScalaReflection.typeOfObject orElse typeOfComplexValue - - val firstRow = rdd.first() - val fields = firstRow.map { - case (fieldName, obj) => StructField(fieldName, typeOfObject(obj), true) - }.toSeq - - applySchemaToPythonRDD(rdd, StructType(fields)) - } - /** * Parses the data type in our internal string representation. The data type string should * have the same format as the one generated by `toString` in scala. @@ -454,7 +425,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * Apply a schema defined by the schemaString to an RDD. It is only used by PySpark. */ private[sql] def applySchemaToPythonRDD( - rdd: RDD[Map[String, _]], + rdd: RDD[Array[Any]], schemaString: String): SchemaRDD = { val schema = parseDataType(schemaString).asInstanceOf[StructType] applySchemaToPythonRDD(rdd, schema) @@ -464,10 +435,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * Apply a schema defined by the schema to an RDD. It is only used by PySpark. */ private[sql] def applySchemaToPythonRDD( - rdd: RDD[Map[String, _]], + rdd: RDD[Array[Any]], schema: StructType): SchemaRDD = { - // TODO: We should have a better implementation once we do not turn a Python side record - // to a Map. import scala.collection.JavaConversions._ import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} @@ -494,55 +463,39 @@ class SQLContext(@transient val sparkContext: SparkContext) val converted = c.map { e => convert(e, elementType)} JListWrapper(converted) - case (c: java.util.Map[_, _], struct: StructType) => - val row = new GenericMutableRow(struct.fields.length) - struct.fields.zipWithIndex.foreach { - case (field, i) => - val value = convert(c.get(field.name), field.dataType) - row.update(i, value) - } - row - - case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => - val converted = c.map { - case (key, value) => - (convert(key, keyType), convert(value, valueType)) - } - JMapWrapper(converted) - case (c, ArrayType(elementType, _)) if c.getClass.isArray => - val converted = c.asInstanceOf[Array[_]].map(e => convert(e, elementType)) - converted: Seq[Any] + c.asInstanceOf[Array[_]].map(e => convert(e, elementType)): Seq[Any] + + case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { + case (key, value) => (convert(key, keyType), convert(value, valueType)) + }.toMap + + case (c, StructType(fields)) if c.getClass.isArray => + new GenericRow(c.asInstanceOf[Array[_]].zip(fields).map { + case (e, f) => convert(e, f.dataType) + }): Row + + case (c: java.util.Calendar, TimestampType) => + new java.sql.Timestamp(c.getTime().getTime()) - case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) case (c: Int, ByteType) => c.toByte case (c: Int, ShortType) => c.toShort case (c: Double, FloatType) => c.toFloat + case (c, StringType) if !c.isInstanceOf[String] => c.toString case (c, _) => c } val convertedRdd = if (schema.fields.exists(f => needsConversion(f.dataType))) { - rdd.map(m => m.map { case (key, value) => (key, convert(value, schema(key).dataType)) }) + rdd.map(m => m.zip(schema.fields).map { + case (value, field) => convert(value, field.dataType) + }) } else { rdd } val rowRdd = convertedRdd.mapPartitions { iter => - val row = new GenericMutableRow(schema.fields.length) - val fieldsWithIndex = schema.fields.zipWithIndex - iter.map { m => - // We cannot use m.values because the order of values returned by m.values may not - // match fields order. - fieldsWithIndex.foreach { - case (field, i) => - val value = - m.get(field.name).flatMap(v => Option(v)).map(v => convert(v, field.dataType)).orNull - row.update(i, value) - } - - row: Row - } + iter.map { m => new GenericRow(m): Row} } new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRdd))(self)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 420f21fb9c1ae..d34f62dc8865e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -383,7 +383,7 @@ class SchemaRDD( import scala.collection.Map def toJava(obj: Any, dataType: DataType): Any = dataType match { - case struct: StructType => rowToMap(obj.asInstanceOf[Row], struct) + case struct: StructType => rowToArray(obj.asInstanceOf[Row], struct) case array: ArrayType => obj match { case seq: Seq[Any] => seq.map(x => toJava(x, array.elementType)).asJava case list: JList[_] => list.map(x => toJava(x, array.elementType)).asJava @@ -397,21 +397,19 @@ class SchemaRDD( // Pyrolite can handle Timestamp case other => obj } - def rowToMap(row: Row, structType: StructType): JMap[String, Any] = { - val fields = structType.fields.map(field => (field.name, field.dataType)) - val map: JMap[String, Any] = new java.util.HashMap - row.zip(fields).foreach { - case (obj, (attrName, dataType)) => map.put(attrName, toJava(obj, dataType)) - } - map + def rowToArray(row: Row, structType: StructType): Array[Any] = { + val fields = structType.fields.map(field => field.dataType) + row.zip(fields).map { + case (obj, dataType) => toJava(obj, dataType) + }.toArray } val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => - rowToMap(row, rowSchema) - }.grouped(10).map(batched => pickle.dumps(batched.toArray)) + rowToArray(row, rowSchema) + }.grouped(100).map(batched => pickle.dumps(batched.toArray)) } } From 3822f33f3ce1428703a4796d7a119b40a6b32259 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 1 Aug 2014 18:52:01 -0700 Subject: [PATCH 0197/1492] [SPARK-2212][SQL] Hash Outer Join (follow-up bug fix). We need to carefully set the ouputPartitioning of the HashOuterJoin Operator. Otherwise, we may not correctly handle nulls. Author: Yin Huai Closes #1721 from yhuai/SPARK-2212-BugFix and squashes the following commits: ed5eef7 [Yin Huai] Correctly choosing outputPartitioning for the HashOuterJoin operator. --- .../apache/spark/sql/execution/joins.scala | 9 +- .../org/apache/spark/sql/JoinSuite.scala | 99 +++++++++++++++++++ .../scala/org/apache/spark/sql/TestData.scala | 8 ++ 3 files changed, 114 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 82f0a74b630bf..cc138c749949d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -158,7 +158,12 @@ case class HashOuterJoin( left: SparkPlan, right: SparkPlan) extends BinaryNode { - override def outputPartitioning: Partitioning = left.outputPartitioning + override def outputPartitioning: Partitioning = joinType match { + case LeftOuter => left.outputPartitioning + case RightOuter => right.outputPartitioning + case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) + case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } override def requiredChildDistribution = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil @@ -309,7 +314,7 @@ case class HashOuterJoin( leftHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST), rightHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST)) } - case x => throw new Exception(s"Need to add implementation for $x") + case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 037890682f7b1..2fc80588182d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -197,6 +197,31 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) + + // Make sure we are choosing left.outputPartitioning as the + // outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ + |SELECT l.N, count(*) + |FROM upperCaseData l LEFT OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY l.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.a, count(*) + |FROM upperCaseData l LEFT OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY r.a + """.stripMargin), + (null, 6) :: Nil) } test("right outer join") { @@ -232,6 +257,31 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "d", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) + + // Make sure we are choosing right.outputPartitioning as the + // outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ + |SELECT l.a, count(*) + |FROM allNulls l RIGHT OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY l.a + """.stripMargin), + (null, 6) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.N, count(*) + |FROM allNulls l RIGHT OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY r.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: Nil) } test("full outer join") { @@ -269,5 +319,54 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) + + // Make sure we are UnknownPartitioning as the outputPartitioning for the outer join operator. + checkAnswer( + sql( + """ + |SELECT l.a, count(*) + |FROM allNulls l FULL OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY l.a + """.stripMargin), + (null, 10) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.N, count(*) + |FROM allNulls l FULL OUTER JOIN upperCaseData r ON (l.a = r.N) + |GROUP BY r.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: + (null, 4) :: Nil) + + checkAnswer( + sql( + """ + |SELECT l.N, count(*) + |FROM upperCaseData l FULL OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY l.N + """.stripMargin), + (1, 1) :: + (2, 1) :: + (3, 1) :: + (4, 1) :: + (5, 1) :: + (6, 1) :: + (null, 4) :: Nil) + + checkAnswer( + sql( + """ + |SELECT r.a, count(*) + |FROM upperCaseData l FULL OUTER JOIN allNulls r ON (l.N = r.a) + |GROUP BY r.a + """.stripMargin), + (null, 10) :: Nil) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 213190e812026..58cee21e8ad4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -118,6 +118,14 @@ object TestData { ) nullInts.registerAsTable("nullInts") + val allNulls = + TestSQLContext.sparkContext.parallelize( + NullInts(null) :: + NullInts(null) :: + NullInts(null) :: + NullInts(null) :: Nil) + allNulls.registerAsTable("allNulls") + case class NullStrings(n: Int, s: String) val nullStrings = TestSQLContext.sparkContext.parallelize( From 0da07da53e5466ec44c8050020cbc4b9957cb949 Mon Sep 17 00:00:00 2001 From: Albert Chu Date: Fri, 1 Aug 2014 19:00:38 -0700 Subject: [PATCH 0198/1492] [SPARK-2116] Load spark-defaults.conf from SPARK_CONF_DIR if set If SPARK_CONF_DIR environment variable is set, search it for spark-defaults.conf. Author: Albert Chu Closes #1059 from chu11/SPARK-2116 and squashes the following commits: 9f3ac94 [Albert Chu] SPARK-2116: If SPARK_CONF_DIR environment variable is set, search it for spark-defaults.conf. --- .../apache/spark/deploy/SparkSubmitArguments.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index dd044e6298760..9391f24e71ed7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -85,6 +85,17 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { */ private def mergeSparkProperties(): Unit = { // Use common defaults file, if not specified by user + if (propertiesFile == null) { + sys.env.get("SPARK_CONF_DIR").foreach { sparkConfDir => + val sep = File.separator + val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } + } + if (propertiesFile == null) { sys.env.get("SPARK_HOME").foreach { sparkHome => val sep = File.separator From a38d3c9efcc0386b52ac4f041920985ae7300e28 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Fri, 1 Aug 2014 19:35:16 -0700 Subject: [PATCH 0199/1492] [SPARK-2800]: Exclude scalastyle-output.xml Apache RAT checks Author: GuoQiang Li Closes #1729 from witgo/SPARK-2800 and squashes the following commits: 13ca966 [GuoQiang Li] Add scalastyle-output.xml to .rat-excludes file --- .rat-excludes | 1 + 1 file changed, 1 insertion(+) diff --git a/.rat-excludes b/.rat-excludes index 372bc2587ccc3..bccb043c2bb55 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -55,3 +55,4 @@ dist/* .*ipr .*iws logs +.*scalastyle-output.xml From e8e0fd691a06a2887fdcffb2217b96805ace0cb0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 1 Aug 2014 19:38:21 -0700 Subject: [PATCH 0200/1492] [SPARK-2764] Simplify daemon.py process structure MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Curently, daemon.py forks a pool of numProcessors subprocesses, and those processes fork themselves again to create the actual Python worker processes that handle data. I think that this extra layer of indirection is unnecessary and adds a lot of complexity. This commit attempts to remove this middle layer of subprocesses by launching the workers directly from daemon.py. See https://github.com/mesos/spark/pull/563 for the original PR that added daemon.py, where I raise some issues with the current design. Author: Josh Rosen Closes #1680 from JoshRosen/pyspark-daemon and squashes the following commits: 5abbcb9 [Josh Rosen] Replace magic number: 4 -> EINTR 5495dff [Josh Rosen] Throw IllegalStateException if worker launch fails. b79254d [Josh Rosen] Detect failed fork() calls; improve error logging. 282c2c4 [Josh Rosen] Remove daemon.py exit logging, since it caused problems: 8554536 [Josh Rosen] Fix daemon’s shutdown(); log shutdown reason. 4e0fab8 [Josh Rosen] Remove shared-memory exit_flag; don't die on worker death. e9892b4 [Josh Rosen] [WIP] [SPARK-2764] Simplify daemon.py process structure. --- .../api/python/PythonWorkerFactory.scala | 10 +- python/pyspark/daemon.py | 179 +++++++----------- 2 files changed, 79 insertions(+), 110 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 759cbe2c46c52..15fe8a9be6bfe 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -64,10 +64,16 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // Attempt to connect, restart and retry once if it fails try { - new Socket(daemonHost, daemonPort) + val socket = new Socket(daemonHost, daemonPort) + val launchStatus = new DataInputStream(socket.getInputStream).readInt() + if (launchStatus != 0) { + throw new IllegalStateException("Python daemon failed to launch worker") + } + socket } catch { case exc: SocketException => - logWarning("Python daemon unexpectedly quit, attempting to restart") + logWarning("Failed to open socket to Python daemon:", exc) + logWarning("Assuming that daemon unexpectedly quit, attempting to restart") stopDaemon() startDaemon() new Socket(daemonHost, daemonPort) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 8a5873ded2b8b..9fde0dde0f4b4 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -15,64 +15,39 @@ # limitations under the License. # +import numbers import os import signal +import select import socket import sys import traceback -import multiprocessing -from ctypes import c_bool from errno import EINTR, ECHILD from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN from pyspark.worker import main as worker_main from pyspark.serializers import write_int -try: - POOLSIZE = multiprocessing.cpu_count() -except NotImplementedError: - POOLSIZE = 4 - -exit_flag = multiprocessing.Value(c_bool, False) - - -def should_exit(): - global exit_flag - return exit_flag.value - def compute_real_exit_code(exit_code): # SystemExit's code can be integer or string, but os._exit only accepts integers - import numbers if isinstance(exit_code, numbers.Integral): return exit_code else: return 1 -def worker(listen_sock): +def worker(sock): + """ + Called by a worker process after the fork(). + """ # Redirect stdout to stderr os.dup2(2, 1) sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 - # Manager sends SIGHUP to request termination of workers in the pool - def handle_sighup(*args): - assert should_exit() - signal.signal(SIGHUP, handle_sighup) - - # Cleanup zombie children - def handle_sigchld(*args): - pid = status = None - try: - while (pid, status) != (0, 0): - pid, status = os.waitpid(0, os.WNOHANG) - except EnvironmentError as err: - if err.errno == EINTR: - # retry - handle_sigchld() - elif err.errno != ECHILD: - raise - signal.signal(SIGCHLD, handle_sigchld) + signal.signal(SIGHUP, SIG_DFL) + signal.signal(SIGCHLD, SIG_DFL) + signal.signal(SIGTERM, SIG_DFL) # Blocks until the socket is closed by draining the input stream # until it raises an exception or returns EOF. @@ -85,55 +60,23 @@ def waitSocketClose(sock): except: pass - # Handle clients - while not should_exit(): - # Wait until a client arrives or we have to exit - sock = None - while not should_exit() and sock is None: - try: - sock, addr = listen_sock.accept() - except EnvironmentError as err: - if err.errno != EINTR: - raise - - if sock is not None: - # Fork a child to handle the client. - # The client is handled in the child so that the manager - # never receives SIGCHLD unless a worker crashes. - if os.fork() == 0: - # Leave the worker pool - signal.signal(SIGHUP, SIG_DFL) - signal.signal(SIGCHLD, SIG_DFL) - listen_sock.close() - # Read the socket using fdopen instead of socket.makefile() because the latter - # seems to be very slow; note that we need to dup() the file descriptor because - # otherwise writes also cause a seek that makes us miss data on the read side. - infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - exit_code = 0 - try: - worker_main(infile, outfile) - except SystemExit as exc: - exit_code = exc.code - finally: - outfile.flush() - # The Scala side will close the socket upon task completion. - waitSocketClose(sock) - os._exit(compute_real_exit_code(exit_code)) - else: - sock.close() - - -def launch_worker(listen_sock): - if os.fork() == 0: - try: - worker(listen_sock) - except Exception as err: - traceback.print_exc() - os._exit(1) - else: - assert should_exit() - os._exit(0) + # Read the socket using fdopen instead of socket.makefile() because the latter + # seems to be very slow; note that we need to dup() the file descriptor because + # otherwise writes also cause a seek that makes us miss data on the read side. + infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + exit_code = 0 + try: + write_int(0, outfile) # Acknowledge that the fork was successful + outfile.flush() + worker_main(infile, outfile) + except SystemExit as exc: + exit_code = exc.code + finally: + outfile.flush() + # The Scala side will close the socket upon task completion. + waitSocketClose(sock) + os._exit(compute_real_exit_code(exit_code)) def manager(): @@ -143,29 +86,28 @@ def manager(): # Create a listening socket on the AF_INET loopback interface listen_sock = socket.socket(AF_INET, SOCK_STREAM) listen_sock.bind(('127.0.0.1', 0)) - listen_sock.listen(max(1024, 2 * POOLSIZE, SOMAXCONN)) + listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() write_int(listen_port, sys.stdout) - # Launch initial worker pool - for idx in range(POOLSIZE): - launch_worker(listen_sock) - listen_sock.close() - - def shutdown(): - global exit_flag - exit_flag.value = True + def shutdown(code): + signal.signal(SIGTERM, SIG_DFL) + # Send SIGHUP to notify workers of shutdown + os.kill(0, SIGHUP) + exit(code) - # Gracefully exit on SIGTERM, don't die on SIGHUP - signal.signal(SIGTERM, lambda signum, frame: shutdown()) - signal.signal(SIGHUP, SIG_IGN) + def handle_sigterm(*args): + shutdown(1) + signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM + signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP # Cleanup zombie children def handle_sigchld(*args): try: pid, status = os.waitpid(0, os.WNOHANG) - if status != 0 and not should_exit(): - raise RuntimeError("worker crashed: %s, %s" % (pid, status)) + if status != 0: + msg = "worker %s crashed abruptly with exit status %s" % (pid, status) + print >> sys.stderr, msg except EnvironmentError as err: if err.errno not in (ECHILD, EINTR): raise @@ -174,20 +116,41 @@ def handle_sigchld(*args): # Initialization complete sys.stdout.close() try: - while not should_exit(): + while True: try: - # Spark tells us to exit by closing stdin - if os.read(0, 512) == '': - shutdown() - except EnvironmentError as err: - if err.errno != EINTR: - shutdown() + ready_fds = select.select([0, listen_sock], [], [])[0] + except select.error as ex: + if ex[0] == EINTR: + continue + else: raise + if 0 in ready_fds: + # Spark told us to exit by closing stdin + shutdown(0) + if listen_sock in ready_fds: + sock, addr = listen_sock.accept() + # Launch a worker process + try: + fork_return_code = os.fork() + if fork_return_code == 0: + listen_sock.close() + try: + worker(sock) + except: + traceback.print_exc() + os._exit(1) + else: + os._exit(0) + else: + sock.close() + except OSError as e: + print >> sys.stderr, "Daemon failed to fork PySpark worker: %s" % e + outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + write_int(-1, outfile) # Signal that the fork failed + outfile.flush() + sock.close() finally: - signal.signal(SIGTERM, SIG_DFL) - exit_flag.value = True - # Send SIGHUP to notify workers of shutdown - os.kill(0, SIGHUP) + shutdown(1) if __name__ == '__main__': From f6a1899306c5ad766fea122d3ab4b83436d9f6fd Mon Sep 17 00:00:00 2001 From: Jeremy Freeman Date: Fri, 1 Aug 2014 20:10:26 -0700 Subject: [PATCH 0201/1492] Streaming mllib [SPARK-2438][MLLIB] This PR implements a streaming linear regression analysis, in which a linear regression model is trained online as new data arrive. The design is based on discussions with tdas and mengxr, in which we determined how to add this functionality in a general way, with minimal changes to existing libraries. __Summary of additions:__ _StreamingLinearAlgorithm_ - An abstract class for fitting generalized linear models online to streaming data, including training on (and updating) a model, and making predictions. _StreamingLinearRegressionWithSGD_ - Class and companion object for running streaming linear regression _StreamingLinearRegressionTestSuite_ - Unit tests _StreamingLinearRegression_ - Example use case: fitting a model online to data from one stream, and making predictions on other data __Notes__ - If this looks good, I can use the StreamingLinearAlgorithm class to easily implement other analyses that follow the same logic (Ridge, Lasso, Logistic, SVM). Author: Jeremy Freeman Author: freeman Closes #1361 from freeman-lab/streaming-mllib and squashes the following commits: 775ea29 [Jeremy Freeman] Throw error if user doesn't initialize weights 4086fee [Jeremy Freeman] Fixed current weight formatting 8b95b27 [Jeremy Freeman] Restored broadcasting 29f27ec [Jeremy Freeman] Formatting 8711c41 [Jeremy Freeman] Used return to avoid indentation 777b596 [Jeremy Freeman] Restored treeAggregate 74cf440 [Jeremy Freeman] Removed static methods d28cf9a [Jeremy Freeman] Added usage notes c3326e7 [Jeremy Freeman] Improved documentation 9541a41 [Jeremy Freeman] Merge remote-tracking branch 'upstream/master' into streaming-mllib 66eba5e [Jeremy Freeman] Fixed line lengths 2fe0720 [Jeremy Freeman] Minor cleanup 7d51378 [Jeremy Freeman] Moved streaming loader to MLUtils b9b69f6 [Jeremy Freeman] Added setter methods c3f8b5a [Jeremy Freeman] Modified logging 00aafdc [Jeremy Freeman] Add modifiers 14b801e [Jeremy Freeman] Name changes c7d38a3 [Jeremy Freeman] Move check for empty data to GradientDescent 4b0a5d3 [Jeremy Freeman] Cleaned up tests 74188d6 [Jeremy Freeman] Eliminate dependency on commons 50dd237 [Jeremy Freeman] Removed experimental tag 6bfe1e6 [Jeremy Freeman] Fixed imports a2a63ad [freeman] Makes convergence test more robust 86220bc [freeman] Streaming linear regression unit tests fb4683a [freeman] Minor changes for scalastyle consistency fd31e03 [freeman] Changed logging behavior 453974e [freeman] Fixed indentation c4b1143 [freeman] Streaming linear regression 604f4d7 [freeman] Expanded private class to include mllib d99aa85 [freeman] Helper methods for streaming MLlib apps 0898add [freeman] Added dependency on streaming --- .../mllib/StreamingLinearRegression.scala | 73 ++++++++++ mllib/pom.xml | 5 + .../mllib/optimization/GradientDescent.scala | 9 ++ .../mllib/regression/LinearRegression.scala | 4 +- .../regression/StreamingLinearAlgorithm.scala | 106 ++++++++++++++ .../StreamingLinearRegressionWithSGD.scala | 88 ++++++++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 15 ++ .../StreamingLinearRegressionSuite.scala | 135 ++++++++++++++++++ 8 files changed, 433 insertions(+), 2 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala new file mode 100644 index 0000000000000..1fd37edfa7427 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD +import org.apache.spark.SparkConf +import org.apache.spark.streaming.{Seconds, StreamingContext} + +/** + * Train a linear regression model on one stream of data and make predictions + * on another stream, where the data streams arrive as text files + * into two different directories. + * + * The rows of the text files must be labeled data points in the form + * `(y,[x1,x2,x3,...,xn])` + * Where n is the number of features. n must be the same for train and test. + * + * Usage: StreamingLinearRegression + * + * To run on your local machine using the two directories `trainingDir` and `testDir`, + * with updates every 5 seconds, and 2 features per data point, call: + * $ bin/run-example \ + * org.apache.spark.examples.mllib.StreamingLinearRegression trainingDir testDir 5 2 + * + * As you add text files to `trainingDir` the model will continuously update. + * Anytime you add text files to `testDir`, you'll see predictions from the current model. + * + */ +object StreamingLinearRegression { + + def main(args: Array[String]) { + + if (args.length != 4) { + System.err.println( + "Usage: StreamingLinearRegression ") + System.exit(1) + } + + val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") + val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) + + val trainingData = MLUtils.loadStreamingLabeledPoints(ssc, args(0)) + val testData = MLUtils.loadStreamingLabeledPoints(ssc, args(1)) + + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(Array.fill[Double](args(3).toInt)(0))) + + model.trainOn(trainingData) + model.predictOn(testData).print() + + ssc.start() + ssc.awaitTermination() + + } + +} diff --git a/mllib/pom.xml b/mllib/pom.xml index 45046eca5b18c..9a33bd1cf6ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -40,6 +40,11 @@ spark-core_${scala.binary.version} ${project.version}
+ + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + org.eclipse.jetty jetty-server diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 356aa949afcf5..a6912056395d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -162,6 +162,14 @@ object GradientDescent extends Logging { val numExamples = data.count() val miniBatchSize = numExamples * miniBatchFraction + // if no data, return initial weights to avoid NaNs + if (numExamples == 0) { + + logInfo("GradientDescent.runMiniBatchSGD returning initial weights, no data found") + return (initialWeights, stochasticLossHistory.toArray) + + } + // Initialize weights as a column vector var weights = Vectors.dense(initialWeights.toArray) val n = weights.size @@ -202,5 +210,6 @@ object GradientDescent extends Logging { stochasticLossHistory.takeRight(10).mkString(", "))) (weights, stochasticLossHistory.toArray) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 8c078ec9f66e9..81b6598377ff5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -49,7 +49,7 @@ class LinearRegressionModel ( * its corresponding right hand side label y. * See also the documentation for the precise formulation. */ -class LinearRegressionWithSGD private ( +class LinearRegressionWithSGD private[mllib] ( private var stepSize: Double, private var numIterations: Int, private var miniBatchFraction: Double) @@ -68,7 +68,7 @@ class LinearRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0) - override protected def createModel(weights: Vector, intercept: Double) = { + override protected[mllib] def createModel(weights: Vector, intercept: Double) = { new LinearRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala new file mode 100644 index 0000000000000..b8b0b42611775 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.regression + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.Logging +import org.apache.spark.streaming.dstream.DStream + +/** + * :: DeveloperApi :: + * StreamingLinearAlgorithm implements methods for continuously + * training a generalized linear model model on streaming data, + * and using it for prediction on (possibly different) streaming data. + * + * This class takes as type parameters a GeneralizedLinearModel, + * and a GeneralizedLinearAlgorithm, making it easy to extend to construct + * streaming versions of any analyses using GLMs. + * Initial weights must be set before calling trainOn or predictOn. + * Only weights will be updated, not an intercept. If the model needs + * an intercept, it should be manually appended to the input data. + * + * For example usage, see `StreamingLinearRegressionWithSGD`. + * + * NOTE(Freeman): In some use cases, the order in which trainOn and predictOn + * are called in an application will affect the results. When called on + * the same DStream, if trainOn is called before predictOn, when new data + * arrive the model will update and the prediction will be based on the new + * model. Whereas if predictOn is called first, the prediction will use the model + * from the previous update. + * + * NOTE(Freeman): It is ok to call predictOn repeatedly on multiple streams; this + * will generate predictions for each one all using the current model. + * It is also ok to call trainOn on different streams; this will update + * the model using each of the different sources, in sequence. + * + */ +@DeveloperApi +abstract class StreamingLinearAlgorithm[ + M <: GeneralizedLinearModel, + A <: GeneralizedLinearAlgorithm[M]] extends Logging { + + /** The model to be updated and used for prediction. */ + protected var model: M + + /** The algorithm to use for updating. */ + protected val algorithm: A + + /** Return the latest model. */ + def latestModel(): M = { + model + } + + /** + * Update the model by training on batches of data from a DStream. + * This operation registers a DStream for training the model, + * and updates the model based on every subsequent + * batch of data from the stream. + * + * @param data DStream containing labeled data + */ + def trainOn(data: DStream[LabeledPoint]) { + if (Option(model.weights) == None) { + logError("Initial weights must be set before starting training") + throw new IllegalArgumentException + } + data.foreachRDD { (rdd, time) => + model = algorithm.run(rdd, model.weights) + logInfo("Model updated at time %s".format(time.toString)) + val display = model.weights.size match { + case x if x > 100 => model.weights.toArray.take(100).mkString("[", ",", "...") + case _ => model.weights.toArray.mkString("[", ",", "]") + } + logInfo("Current model: weights, %s".format (display)) + } + } + + /** + * Use the model to make predictions on batches of data from a DStream + * + * @param data DStream containing labeled data + * @return DStream containing predictions + */ + def predictOn(data: DStream[LabeledPoint]): DStream[Double] = { + if (Option(model.weights) == None) { + logError("Initial weights must be set before starting prediction") + throw new IllegalArgumentException + } + data.map(x => model.predict(x.features)) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala new file mode 100644 index 0000000000000..8851097050318 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -0,0 +1,88 @@ +/* + * 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.regression + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +/** + * Train or predict a linear regression model on streaming data. Training uses + * Stochastic Gradient Descent to update the model based on each new batch of + * incoming data from a DStream (see `LinearRegressionWithSGD` for model equation) + * + * Each batch of data is assumed to be an RDD of LabeledPoints. + * The number of data points per batch can vary, but the number + * of features must be constant. An initial weight + * vector must be provided. + * + * Use a builder pattern to construct a streaming linear regression + * analysis in an application, like: + * + * val model = new StreamingLinearRegressionWithSGD() + * .setStepSize(0.5) + * .setNumIterations(10) + * .setInitialWeights(Vectors.dense(...)) + * .trainOn(DStream) + * + */ +@Experimental +class StreamingLinearRegressionWithSGD ( + private var stepSize: Double, + private var numIterations: Int, + private var miniBatchFraction: Double, + private var initialWeights: Vector) + extends StreamingLinearAlgorithm[ + LinearRegressionModel, LinearRegressionWithSGD] with Serializable { + + /** + * Construct a StreamingLinearRegression object with default parameters: + * {stepSize: 0.1, numIterations: 50, miniBatchFraction: 1.0}. + * Initial weights must be set before using trainOn or predictOn + * (see `StreamingLinearAlgorithm`) + */ + def this() = this(0.1, 50, 1.0, null) + + val algorithm = new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + + var model = algorithm.createModel(initialWeights, 0.0) + + /** Set the step size for gradient descent. Default: 0.1. */ + def setStepSize(stepSize: Double): this.type = { + this.algorithm.optimizer.setStepSize(stepSize) + this + } + + /** Set the number of iterations of gradient descent to run per update. Default: 50. */ + def setNumIterations(numIterations: Int): this.type = { + this.algorithm.optimizer.setNumIterations(numIterations) + this + } + + /** Set the fraction of each batch to use for updates. Default: 1.0. */ + def setMiniBatchFraction(miniBatchFraction: Double): this.type = { + this.algorithm.optimizer.setMiniBatchFraction(miniBatchFraction) + this + } + + /** Set the initial weights. Default: [0.0, 0.0]. */ + def setInitialWeights(initialWeights: Vector): this.type = { + this.model = algorithm.createModel(initialWeights, 0.0) + this + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index dc10a194783ed..f4cce86a65ba7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -30,6 +30,8 @@ import org.apache.spark.util.random.BernoulliSampler import org.apache.spark.mllib.regression.{LabeledPointParser, LabeledPoint} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream.DStream /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -192,6 +194,19 @@ object MLUtils { def loadLabeledPoints(sc: SparkContext, dir: String): RDD[LabeledPoint] = loadLabeledPoints(sc, dir, sc.defaultMinPartitions) + /** + * Loads streaming labeled points from a stream of text files + * where points are in the same format as used in `RDD[LabeledPoint].saveAsTextFile`. + * See `StreamingContext.textFileStream` for more details on how to + * generate a stream from files + * + * @param ssc Streaming context + * @param dir Directory path in any Hadoop-supported file system URI + * @return Labeled points stored as a DStream[LabeledPoint] + */ + def loadStreamingLabeledPoints(ssc: StreamingContext, dir: String): DStream[LabeledPoint] = + ssc.textFileStream(dir).map(LabeledPointParser.parse) + /** * Load labeled data from a file. The data format used here is * , ... diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala new file mode 100644 index 0000000000000..ed21f84472c9a --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -0,0 +1,135 @@ +/* + * 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.regression + +import java.io.File +import java.nio.charset.Charset + +import scala.collection.mutable.ArrayBuffer + +import com.google.common.io.Files +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext, MLUtils} +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.util.Utils + +class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { + + // Assert that two values are equal within tolerance epsilon + def assertEqual(v1: Double, v2: Double, epsilon: Double) { + def errorMessage = v1.toString + " did not equal " + v2.toString + assert(math.abs(v1-v2) <= epsilon, errorMessage) + } + + // Assert that model predictions are correct + def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + // A prediction is off if the prediction is more than 0.5 away from expected value. + math.abs(prediction - expected.label) > 0.5 + } + // At least 80% of the predictions should be on. + assert(numOffPredictions < input.length / 5) + } + + // Test if we can accurately learn Y = 10*X1 + 10*X2 on streaming data + test("streaming linear regression parameter accuracy") { + + val testDir = Files.createTempDir() + val numBatches = 10 + val batchDuration = Milliseconds(1000) + val ssc = new StreamingContext(sc, batchDuration) + val data = MLUtils.loadStreamingLabeledPoints(ssc, testDir.toString) + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0, 0.0)) + .setStepSize(0.1) + .setNumIterations(50) + + model.trainOn(data) + + ssc.start() + + // write data to a file stream + for (i <- 0 until numBatches) { + val samples = LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 42 * (i + 1)) + val file = new File(testDir, i.toString) + Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) + Thread.sleep(batchDuration.milliseconds) + } + + ssc.stop(stopSparkContext=false) + + System.clearProperty("spark.driver.port") + Utils.deleteRecursively(testDir) + + // check accuracy of final parameter estimates + assertEqual(model.latestModel().intercept, 0.0, 0.1) + assertEqual(model.latestModel().weights(0), 10.0, 0.1) + assertEqual(model.latestModel().weights(1), 10.0, 0.1) + + // check accuracy of predictions + val validationData = LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 17) + validatePrediction(validationData.map(row => model.latestModel().predict(row.features)), + validationData) + } + + // Test that parameter estimates improve when learning Y = 10*X1 on streaming data + test("streaming linear regression parameter convergence") { + + val testDir = Files.createTempDir() + val batchDuration = Milliseconds(2000) + val ssc = new StreamingContext(sc, batchDuration) + val numBatches = 5 + val data = MLUtils.loadStreamingLabeledPoints(ssc, testDir.toString) + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(0.0)) + .setStepSize(0.1) + .setNumIterations(50) + + model.trainOn(data) + + ssc.start() + + // write data to a file stream + val history = new ArrayBuffer[Double](numBatches) + for (i <- 0 until numBatches) { + val samples = LinearDataGenerator.generateLinearInput(0.0, Array(10.0), 100, 42 * (i + 1)) + val file = new File(testDir, i.toString) + Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) + Thread.sleep(batchDuration.milliseconds) + // wait an extra few seconds to make sure the update finishes before new data arrive + Thread.sleep(4000) + history.append(math.abs(model.latestModel().weights(0) - 10.0)) + } + + ssc.stop(stopSparkContext=false) + + System.clearProperty("spark.driver.port") + Utils.deleteRecursively(testDir) + + val deltas = history.drop(1).zip(history.dropRight(1)) + // check error stability (it always either shrinks, or increases with small tol) + assert(deltas.forall(x => (x._1 - x._2) <= 0.1)) + // check that error shrunk on at least 2 batches + assert(deltas.map(x => if ((x._1 - x._2) < 0) 1 else 0).sum > 1) + + } + +} From c281189222e645d2c87277c269e2102c3c8ccc95 Mon Sep 17 00:00:00 2001 From: Michael Giannakopoulos Date: Fri, 1 Aug 2014 21:00:31 -0700 Subject: [PATCH 0202/1492] [SPARK-2550][MLLIB][APACHE SPARK] Support regularization and intercept in pyspark's linear methods. Related to issue: [SPARK-2550](https://issues.apache.org/jira/browse/SPARK-2550?jql=project%20%3D%20SPARK%20AND%20resolution%20%3D%20Unresolved%20AND%20priority%20%3D%20Major%20ORDER%20BY%20key%20DESC). Author: Michael Giannakopoulos Closes #1624 from miccagiann/new-branch and squashes the following commits: c02e5f5 [Michael Giannakopoulos] Merge cleanly with upstream/master. 8dcb888 [Michael Giannakopoulos] Putting the if/else if statements in brackets. fed8eaa [Michael Giannakopoulos] Adding a space in the message related to the IllegalArgumentException. 44e6ff0 [Michael Giannakopoulos] Adding a blank line before python class LinearRegressionWithSGD. 8eba9c5 [Michael Giannakopoulos] Change function signatures. Exception is thrown from the scala component and not from the python one. 638be47 [Michael Giannakopoulos] Modified code to comply with code standards. ec50ee9 [Michael Giannakopoulos] Shorten the if-elif-else statement in regression.py file b962744 [Michael Giannakopoulos] Replaced the enum classes, with strings-keywords for defining the values of 'regType' parameter. 78853ec [Michael Giannakopoulos] Providing intercept and regualizer functionallity for linear methods in only one function. 3ac8874 [Michael Giannakopoulos] Added support for regularizer and intercection parameters for linear regression method. --- .../mllib/api/python/PythonMLLibAPI.scala | 28 ++++++++++++---- python/pyspark/mllib/regression.py | 32 ++++++++++++++++--- 2 files changed, 49 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 122925d096e98..7d912737b8f0b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -23,6 +23,8 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ +import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.random.{RandomRDDGenerators => RG} import org.apache.spark.mllib.recommendation._ @@ -252,15 +254,27 @@ class PythonMLLibAPI extends Serializable { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + initialWeightsBA: Array[Byte], + regParam: Double, + regType: String, + intercept: Boolean): java.util.List[java.lang.Object] = { + val lrAlg = new LinearRegressionWithSGD() + lrAlg.setIntercept(intercept) + lrAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + if (regType == "l2") { + lrAlg.optimizer.setUpdater(new SquaredL2Updater) + } else if (regType == "l1") { + lrAlg.optimizer.setUpdater(new L1Updater) + } else if (regType != "none") { + throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." + + " Can only be initialized using the following string values: [l1, l2, none].") + } trainRegressionModel( (data, initialWeights) => - LinearRegressionWithSGD.train( - data, - numIterations, - stepSize, - miniBatchFraction, - initialWeights), + lrAlg.run(data, initialWeights), dataBytesJRDD, initialWeightsBA) } diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index b84bc531dec8c..041b119269427 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -112,12 +112,36 @@ class LinearRegressionModel(LinearRegressionModelBase): class LinearRegressionWithSGD(object): @classmethod - def train(cls, data, iterations=100, step=1.0, - miniBatchFraction=1.0, initialWeights=None): - """Train a linear regression model on the given data.""" + def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, + initialWeights=None, regParam=1.0, regType=None, intercept=False): + """ + Train a linear regression model on the given data. + + @param data: The training data. + @param iterations: The number of iterations (default: 100). + @param step: The step parameter used in SGD + (default: 1.0). + @param miniBatchFraction: Fraction of data to be used for each SGD + iteration. + @param initialWeights: The initial weights (default: None). + @param regParam: The regularizer parameter (default: 1.0). + @param regType: The type of regularizer used for training + our model. + Allowed values: "l1" for using L1Updater, + "l2" for using + SquaredL2Updater, + "none" for no regularizer. + (default: "none") + @param intercept: Boolean parameter which indicates the use + or not of the augmented representation for + training data (i.e. whether bias features + are activated or not). + """ sc = data.context + if regType is None: + regType = "none" train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i) + d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) return _regression_train_wrapper(sc, train_f, LinearRegressionModel, data, initialWeights) From e25ec06171e3ba95920cbfe9df3cd3d990f1a3a3 Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Fri, 1 Aug 2014 21:25:02 -0700 Subject: [PATCH 0203/1492] [SPARK-1580][MLLIB] Estimate ALS communication and computation costs. Continue the work from #493. Closes #493 and Closes #593 Author: Tor Myklebust Author: Xiangrui Meng Closes #1731 from mengxr/tmyklebu-alscost and squashes the following commits: 9b56a8b [Xiangrui Meng] updated API and added a simple test 68a3229 [Xiangrui Meng] merge master 217bd1d [Tor Myklebust] Documentation and choleskies -> subproblems. 8cbb718 [Tor Myklebust] Braces get spaces. 0455cd4 [Tor Myklebust] Parens for collectAsMap. 2b2febe [Tor Myklebust] Use `makeLinkRDDs` when estimating costs. 2ab7a5d [Tor Myklebust] Reindent estimateCost's declaration and make it return Seqs. 8b21e6d [Tor Myklebust] Fix overlong lines. 8cbebf1 [Tor Myklebust] Rename and clean up the return format of cost estimator. 6615ed5 [Tor Myklebust] It's more useful to give per-partition estimates. Do that. 5530678 [Tor Myklebust] Merge branch 'master' of https://github.com/apache/spark into alscost 6c31324 [Tor Myklebust] Make it actually build... a1184d1 [Tor Myklebust] Mark ALS.evaluatePartitioner DeveloperApi. 657a71b [Tor Myklebust] Simple-minded estimates of computation and communication costs in ALS. dcf583a [Tor Myklebust] Remove the partitioner member variable; instead, thread that needle everywhere it needs to go. 23d6f91 [Tor Myklebust] Stop making the partitioner configurable. 495784f [Tor Myklebust] Merge branch 'master' of https://github.com/apache/spark 674933a [Tor Myklebust] Fix style. 40edc23 [Tor Myklebust] Fix missing space. f841345 [Tor Myklebust] Fix daft bug creating 'pairs', also for -> foreach. 5ec9e6c [Tor Myklebust] Clean a couple of things up using 'map'. 36a0f43 [Tor Myklebust] Make the partitioner private. d872b09 [Tor Myklebust] Add negative id ALS test. df27697 [Tor Myklebust] Support custom partitioners. Currently we use the same partitioner for users and products. c90b6d8 [Tor Myklebust] Scramble user and product ids before bucketing. c774d7d [Tor Myklebust] Make the partitioner a member variable and use it instead of modding directly. --- .../spark/mllib/recommendation/ALS.scala | 126 +++++++++++++++++- .../spark/mllib/recommendation/ALSSuite.scala | 26 +++- 2 files changed, 144 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 36d262fed425a..8ebc7e27ed4dd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -17,7 +17,8 @@ package org.apache.spark.mllib.recommendation -import scala.collection.mutable.{ArrayBuffer, BitSet} +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting @@ -25,7 +26,7 @@ import scala.util.hashing.byteswap32 import org.jblas.{DoubleMatrix, SimpleBlas, Solve} -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.{Logging, HashPartitioner, Partitioner} import org.apache.spark.storage.StorageLevel @@ -39,7 +40,8 @@ import org.apache.spark.mllib.optimization.NNLS * of the elements within this block, and the list of destination blocks that each user or * product will need to send its feature vector to. */ -private[recommendation] case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[BitSet]) +private[recommendation] +case class OutLinkBlock(elementIds: Array[Int], shouldSend: Array[mutable.BitSet]) /** @@ -382,7 +384,7 @@ class ALS private ( val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap - val shouldSend = Array.fill(numUsers)(new BitSet(numProductBlocks)) + val shouldSend = Array.fill(numUsers)(new mutable.BitSet(numProductBlocks)) for (r <- ratings) { shouldSend(userIdToPos(r.user))(productPartitioner.getPartition(r.product)) = true } @@ -797,4 +799,120 @@ object ALS { : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } + + /** + * :: DeveloperApi :: + * Statistics of a block in ALS computation. + * + * @param category type of this block, "user" or "product" + * @param index index of this block + * @param count number of users or products inside this block, the same as the number of + * least-squares problems to solve on this block in each iteration + * @param numRatings total number of ratings inside this block, the same as the number of outer + * products we need to make on this block in each iteration + * @param numInLinks total number of incoming links, the same as the number of vectors to retrieve + * before each iteration + * @param numOutLinks total number of outgoing links, the same as the number of vectors to send + * for the next iteration + */ + @DeveloperApi + case class BlockStats( + category: String, + index: Int, + count: Long, + numRatings: Long, + numInLinks: Long, + numOutLinks: Long) + + /** + * :: DeveloperApi :: + * Given an RDD of ratings, number of user blocks, and number of product blocks, computes the + * statistics of each block in ALS computation. This is useful for estimating cost and diagnosing + * load balance. + * + * @param ratings an RDD of ratings + * @param numUserBlocks number of user blocks + * @param numProductBlocks number of product blocks + * @return statistics of user blocks and product blocks + */ + @DeveloperApi + def analyzeBlocks( + ratings: RDD[Rating], + numUserBlocks: Int, + numProductBlocks: Int): Array[BlockStats] = { + + val userPartitioner = new ALSPartitioner(numUserBlocks) + val productPartitioner = new ALSPartitioner(numProductBlocks) + + val ratingsByUserBlock = ratings.map { rating => + (userPartitioner.getPartition(rating.user), rating) + } + val ratingsByProductBlock = ratings.map { rating => + (productPartitioner.getPartition(rating.product), + Rating(rating.product, rating.user, rating.rating)) + } + + val als = new ALS() + val (userIn, userOut) = + als.makeLinkRDDs(numUserBlocks, numProductBlocks, ratingsByUserBlock, userPartitioner) + val (prodIn, prodOut) = + als.makeLinkRDDs(numProductBlocks, numUserBlocks, ratingsByProductBlock, productPartitioner) + + def sendGrid(outLinks: RDD[(Int, OutLinkBlock)]): Map[(Int, Int), Long] = { + outLinks.map { x => + val grid = new mutable.HashMap[(Int, Int), Long]() + val uPartition = x._1 + x._2.shouldSend.foreach { ss => + ss.foreach { pPartition => + val pair = (uPartition, pPartition) + grid.put(pair, grid.getOrElse(pair, 0L) + 1L) + } + } + grid + }.reduce { (grid1, grid2) => + grid2.foreach { x => + grid1.put(x._1, grid1.getOrElse(x._1, 0L) + x._2) + } + grid1 + }.toMap + } + + val userSendGrid = sendGrid(userOut) + val prodSendGrid = sendGrid(prodOut) + + val userInbound = new Array[Long](numUserBlocks) + val prodInbound = new Array[Long](numProductBlocks) + val userOutbound = new Array[Long](numUserBlocks) + val prodOutbound = new Array[Long](numProductBlocks) + + for (u <- 0 until numUserBlocks; p <- 0 until numProductBlocks) { + userOutbound(u) += userSendGrid.getOrElse((u, p), 0L) + prodInbound(p) += userSendGrid.getOrElse((u, p), 0L) + userInbound(u) += prodSendGrid.getOrElse((p, u), 0L) + prodOutbound(p) += prodSendGrid.getOrElse((p, u), 0L) + } + + val userCounts = userOut.mapValues(x => x.elementIds.length).collectAsMap() + val prodCounts = prodOut.mapValues(x => x.elementIds.length).collectAsMap() + + val userRatings = countRatings(userIn) + val prodRatings = countRatings(prodIn) + + val userStats = Array.tabulate(numUserBlocks)( + u => BlockStats("user", u, userCounts(u), userRatings(u), userInbound(u), userOutbound(u))) + val productStatus = Array.tabulate(numProductBlocks)( + p => BlockStats("product", p, prodCounts(p), prodRatings(p), prodInbound(p), prodOutbound(p))) + + (userStats ++ productStatus).toArray + } + + private def countRatings(inLinks: RDD[(Int, InLinkBlock)]): Map[Int, Long] = { + inLinks.mapValues { ilb => + var numRatings = 0L + ilb.ratingsForBlock.foreach { ar => + ar.foreach { p => numRatings += p._1.length } + } + numRatings + }.collectAsMap().toMap + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 81bebec8c7a39..017c39edb185f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -22,11 +22,11 @@ import scala.math.abs import scala.util.Random import org.scalatest.FunSuite - import org.jblas.DoubleMatrix -import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.recommendation.ALS.BlockStats object ALSSuite { @@ -67,8 +67,10 @@ object ALSSuite { case true => // Generate raw values from [0,9], or if negativeWeights, from [-2,7] val raw = new DoubleMatrix(users, products, - Array.fill(users * products)((if (negativeWeights) -2 else 0) + rand.nextInt(10).toDouble): _*) - val prefs = new DoubleMatrix(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0): _*) + Array.fill(users * products)( + (if (negativeWeights) -2 else 0) + rand.nextInt(10).toDouble): _*) + val prefs = + new DoubleMatrix(users, products, raw.data.map(v => if (v > 0) 1.0 else 0.0): _*) (raw, prefs) case false => (userMatrix.mmul(productMatrix), null) } @@ -160,6 +162,22 @@ class ALSSuite extends FunSuite with LocalSparkContext { testALS(100, 200, 2, 15, 0.7, 0.4, false, false, false, -1, -1, false) } + test("analyze one user block and one product block") { + val localRatings = Seq( + Rating(0, 100, 1.0), + Rating(0, 101, 2.0), + Rating(0, 102, 3.0), + Rating(1, 102, 4.0), + Rating(2, 103, 5.0)) + val ratings = sc.makeRDD(localRatings, 2) + val stats = ALS.analyzeBlocks(ratings, 1, 1) + assert(stats.size === 2) + assert(stats(0) === BlockStats("user", 0, 3, 5, 4, 3)) + assert(stats(1) === BlockStats("product", 0, 4, 5, 3, 4)) + } + + // TODO: add tests for analyzing multiple user/product blocks + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * From fda475987f3b8b37d563033b0e45706ce433824a Mon Sep 17 00:00:00 2001 From: Burak Date: Fri, 1 Aug 2014 22:32:12 -0700 Subject: [PATCH 0204/1492] [SPARK-2801][MLlib]: DistributionGenerator renamed to RandomDataGenerator. RandomRDD is now of generic type The RandomRDDGenerators used to only output RDD[Double]. Now RandomRDDGenerators.randomRDD can be used to generate a random RDD[T] via a class that extends RandomDataGenerator, by supplying a type T and overriding the nextValue() function as they wish. Author: Burak Closes #1732 from brkyvz/SPARK-2801 and squashes the following commits: c94a694 [Burak] [SPARK-2801][MLlib] Missing ClassTags added 22d96fe [Burak] [SPARK-2801][MLlib]: DistributionGenerator renamed to RandomDataGenerator, generic types added for RandomRDD instead of Double --- ...erator.scala => RandomDataGenerator.scala} | 18 +++++----- .../mllib/random/RandomRDDGenerators.scala | 32 +++++++++-------- .../apache/spark/mllib/rdd/RandomRDD.scala | 34 ++++++++++--------- ...e.scala => RandomDataGeneratorSuite.scala} | 6 ++-- .../random/RandomRDDGeneratorsSuite.scala | 8 +++-- 5 files changed, 52 insertions(+), 46 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/random/{DistributionGenerator.scala => RandomDataGenerator.scala} (80%) rename mllib/src/test/scala/org/apache/spark/mllib/random/{DistributionGeneratorSuite.scala => RandomDataGeneratorSuite.scala} (95%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala similarity index 80% rename from mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala rename to mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 7ecb409c4a91a..9cab49f6ed1f0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/DistributionGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -25,21 +25,21 @@ import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} /** * :: Experimental :: - * Trait for random number generators that generate i.i.d. values from a distribution. + * Trait for random data generators that generate i.i.d. data. */ @Experimental -trait DistributionGenerator extends Pseudorandom with Serializable { +trait RandomDataGenerator[T] extends Pseudorandom with Serializable { /** - * Returns an i.i.d. sample as a Double from an underlying distribution. + * Returns an i.i.d. sample as a generic type from an underlying distribution. */ - def nextValue(): Double + def nextValue(): T /** - * Returns a copy of the DistributionGenerator with a new instance of the rng object used in the + * Returns a copy of the RandomDataGenerator with a new instance of the rng object used in the * class when applicable for non-locking concurrent usage. */ - def copy(): DistributionGenerator + def copy(): RandomDataGenerator[T] } /** @@ -47,7 +47,7 @@ trait DistributionGenerator extends Pseudorandom with Serializable { * Generates i.i.d. samples from U[0.0, 1.0] */ @Experimental -class UniformGenerator extends DistributionGenerator { +class UniformGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. private val random = new XORShiftRandom() @@ -66,7 +66,7 @@ class UniformGenerator extends DistributionGenerator { * Generates i.i.d. samples from the standard normal distribution. */ @Experimental -class StandardNormalGenerator extends DistributionGenerator { +class StandardNormalGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. private val random = new XORShiftRandom() @@ -87,7 +87,7 @@ class StandardNormalGenerator extends DistributionGenerator { * @param mean mean for the Poisson distribution. */ @Experimental -class PoissonGenerator(val mean: Double) extends DistributionGenerator { +class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { private var rng = new Poisson(mean, new DRand) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala index 021d651d4dbaa..b0a0593223910 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala @@ -24,6 +24,8 @@ import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils +import scala.reflect.ClassTag + /** * :: Experimental :: * Generator methods for creating RDDs comprised of i.i.d. samples from some distribution. @@ -200,12 +202,12 @@ object RandomRDDGenerators { * @return RDD[Double] comprised of i.i.d. samples produced by generator. */ @Experimental - def randomRDD(sc: SparkContext, - generator: DistributionGenerator, + def randomRDD[T: ClassTag](sc: SparkContext, + generator: RandomDataGenerator[T], size: Long, numPartitions: Int, - seed: Long): RDD[Double] = { - new RandomRDD(sc, size, numPartitions, generator, seed) + seed: Long): RDD[T] = { + new RandomRDD[T](sc, size, numPartitions, generator, seed) } /** @@ -219,11 +221,11 @@ object RandomRDDGenerators { * @return RDD[Double] comprised of i.i.d. samples produced by generator. */ @Experimental - def randomRDD(sc: SparkContext, - generator: DistributionGenerator, + def randomRDD[T: ClassTag](sc: SparkContext, + generator: RandomDataGenerator[T], size: Long, - numPartitions: Int): RDD[Double] = { - randomRDD(sc, generator, size, numPartitions, Utils.random.nextLong) + numPartitions: Int): RDD[T] = { + randomRDD[T](sc, generator, size, numPartitions, Utils.random.nextLong) } /** @@ -237,10 +239,10 @@ object RandomRDDGenerators { * @return RDD[Double] comprised of i.i.d. samples produced by generator. */ @Experimental - def randomRDD(sc: SparkContext, - generator: DistributionGenerator, - size: Long): RDD[Double] = { - randomRDD(sc, generator, size, sc.defaultParallelism, Utils.random.nextLong) + def randomRDD[T: ClassTag](sc: SparkContext, + generator: RandomDataGenerator[T], + size: Long): RDD[T] = { + randomRDD[T](sc, generator, size, sc.defaultParallelism, Utils.random.nextLong) } // TODO Generate RDD[Vector] from multivariate distributions. @@ -439,7 +441,7 @@ object RandomRDDGenerators { */ @Experimental def randomVectorRDD(sc: SparkContext, - generator: DistributionGenerator, + generator: RandomDataGenerator[Double], numRows: Long, numCols: Int, numPartitions: Int, @@ -461,7 +463,7 @@ object RandomRDDGenerators { */ @Experimental def randomVectorRDD(sc: SparkContext, - generator: DistributionGenerator, + generator: RandomDataGenerator[Double], numRows: Long, numCols: Int, numPartitions: Int): RDD[Vector] = { @@ -482,7 +484,7 @@ object RandomRDDGenerators { */ @Experimental def randomVectorRDD(sc: SparkContext, - generator: DistributionGenerator, + generator: RandomDataGenerator[Double], numRows: Long, numCols: Int): RDD[Vector] = { randomVectorRDD(sc, generator, numRows, numCols, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala index f13282d07ff92..c8db3910c6eab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala @@ -19,35 +19,36 @@ package org.apache.spark.mllib.rdd import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.mllib.linalg.{DenseVector, Vector} -import org.apache.spark.mllib.random.DistributionGenerator +import org.apache.spark.mllib.random.RandomDataGenerator import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils +import scala.reflect.ClassTag import scala.util.Random -private[mllib] class RandomRDDPartition(override val index: Int, +private[mllib] class RandomRDDPartition[T](override val index: Int, val size: Int, - val generator: DistributionGenerator, + val generator: RandomDataGenerator[T], val seed: Long) extends Partition { require(size >= 0, "Non-negative partition size required.") } // These two classes are necessary since Range objects in Scala cannot have size > Int.MaxValue -private[mllib] class RandomRDD(@transient sc: SparkContext, +private[mllib] class RandomRDD[T: ClassTag](@transient sc: SparkContext, size: Long, numPartitions: Int, - @transient rng: DistributionGenerator, - @transient seed: Long = Utils.random.nextLong) extends RDD[Double](sc, Nil) { + @transient rng: RandomDataGenerator[T], + @transient seed: Long = Utils.random.nextLong) extends RDD[T](sc, Nil) { require(size > 0, "Positive RDD size required.") require(numPartitions > 0, "Positive number of partitions required") require(math.ceil(size.toDouble / numPartitions) <= Int.MaxValue, "Partition size cannot exceed Int.MaxValue") - override def compute(splitIn: Partition, context: TaskContext): Iterator[Double] = { - val split = splitIn.asInstanceOf[RandomRDDPartition] - RandomRDD.getPointIterator(split) + override def compute(splitIn: Partition, context: TaskContext): Iterator[T] = { + val split = splitIn.asInstanceOf[RandomRDDPartition[T]] + RandomRDD.getPointIterator[T](split) } override def getPartitions: Array[Partition] = { @@ -59,7 +60,7 @@ private[mllib] class RandomVectorRDD(@transient sc: SparkContext, size: Long, vectorSize: Int, numPartitions: Int, - @transient rng: DistributionGenerator, + @transient rng: RandomDataGenerator[Double], @transient seed: Long = Utils.random.nextLong) extends RDD[Vector](sc, Nil) { require(size > 0, "Positive RDD size required.") @@ -69,7 +70,7 @@ private[mllib] class RandomVectorRDD(@transient sc: SparkContext, "Partition size cannot exceed Int.MaxValue") override def compute(splitIn: Partition, context: TaskContext): Iterator[Vector] = { - val split = splitIn.asInstanceOf[RandomRDDPartition] + val split = splitIn.asInstanceOf[RandomRDDPartition[Double]] RandomRDD.getVectorIterator(split, vectorSize) } @@ -80,12 +81,12 @@ private[mllib] class RandomVectorRDD(@transient sc: SparkContext, private[mllib] object RandomRDD { - def getPartitions(size: Long, + def getPartitions[T](size: Long, numPartitions: Int, - rng: DistributionGenerator, + rng: RandomDataGenerator[T], seed: Long): Array[Partition] = { - val partitions = new Array[RandomRDDPartition](numPartitions) + val partitions = new Array[RandomRDDPartition[T]](numPartitions) var i = 0 var start: Long = 0 var end: Long = 0 @@ -101,7 +102,7 @@ private[mllib] object RandomRDD { // The RNG has to be reset every time the iterator is requested to guarantee same data // every time the content of the RDD is examined. - def getPointIterator(partition: RandomRDDPartition): Iterator[Double] = { + def getPointIterator[T: ClassTag](partition: RandomRDDPartition[T]): Iterator[T] = { val generator = partition.generator.copy() generator.setSeed(partition.seed) Array.fill(partition.size)(generator.nextValue()).toIterator @@ -109,7 +110,8 @@ private[mllib] object RandomRDD { // The RNG has to be reset every time the iterator is requested to guarantee same data // every time the content of the RDD is examined. - def getVectorIterator(partition: RandomRDDPartition, vectorSize: Int): Iterator[Vector] = { + def getVectorIterator(partition: RandomRDDPartition[Double], + vectorSize: Int): Iterator[Vector] = { val generator = partition.generator.copy() generator.setSeed(partition.seed) Array.fill(partition.size)(new DenseVector( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala similarity index 95% rename from mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala index 974dec4c0b5ee..3df7c128af5ab 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/DistributionGeneratorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala @@ -22,9 +22,9 @@ import org.scalatest.FunSuite import org.apache.spark.util.StatCounter // TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged -class DistributionGeneratorSuite extends FunSuite { +class RandomDataGeneratorSuite extends FunSuite { - def apiChecks(gen: DistributionGenerator) { + def apiChecks(gen: RandomDataGenerator[Double]) { // resetting seed should generate the same sequence of random numbers gen.setSeed(42L) @@ -53,7 +53,7 @@ class DistributionGeneratorSuite extends FunSuite { assert(array5.equals(array6)) } - def distributionChecks(gen: DistributionGenerator, + def distributionChecks(gen: RandomDataGenerator[Double], mean: Double = 0.0, stddev: Double = 1.0, epsilon: Double = 0.01) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala index 6aa4f803df0f7..96e0bc63b0fa4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala @@ -78,7 +78,9 @@ class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Seri assert(rdd.partitions.size === numPartitions) // check that partition sizes are balanced - val partSizes = rdd.partitions.map(p => p.asInstanceOf[RandomRDDPartition].size.toDouble) + val partSizes = rdd.partitions.map(p => + p.asInstanceOf[RandomRDDPartition[Double]].size.toDouble) + val partStats = new StatCounter(partSizes) assert(partStats.max - partStats.min <= 1) } @@ -89,7 +91,7 @@ class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Seri val rdd = new RandomRDD(sc, size, numPartitions, new UniformGenerator, 0L) assert(rdd.partitions.size === numPartitions) val count = rdd.partitions.foldLeft(0L) { (count, part) => - count + part.asInstanceOf[RandomRDDPartition].size + count + part.asInstanceOf[RandomRDDPartition[Double]].size } assert(count === size) @@ -145,7 +147,7 @@ class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Seri } } -private[random] class MockDistro extends DistributionGenerator { +private[random] class MockDistro extends RandomDataGenerator[Double] { var seed = 0L From 4bc3bb29a4b6ab24b6b7e1f8df26414c41c80ace Mon Sep 17 00:00:00 2001 From: Jeremy Freeman Date: Fri, 1 Aug 2014 22:33:25 -0700 Subject: [PATCH 0205/1492] StatCounter on NumPy arrays [PYSPARK][SPARK-2012] These changes allow StatCounters to work properly on NumPy arrays, to fix the issue reported here (https://issues.apache.org/jira/browse/SPARK-2012). If NumPy is installed, the NumPy functions ``maximum``, ``minimum``, and ``sqrt``, which work on arrays, are used to merge statistics. If not, we fall back on scalar operators, so it will work on arrays with NumPy, but will also work without NumPy. New unit tests added, along with a check for NumPy in the tests. Author: Jeremy Freeman Closes #1725 from freeman-lab/numpy-max-statcounter and squashes the following commits: fe973b1 [Jeremy Freeman] Avoid duplicate array import in tests 7f0e397 [Jeremy Freeman] Refactored check for numpy 8e764dd [Jeremy Freeman] Explicit numpy imports 875414c [Jeremy Freeman] Fixed indents 1c8a832 [Jeremy Freeman] Unit tests for StatCounter with NumPy arrays 176a127 [Jeremy Freeman] Use numpy arrays in StatCounter --- python/pyspark/statcounter.py | 21 +++++++++++++-------- python/pyspark/tests.py | 24 ++++++++++++++++++++++++ 2 files changed, 37 insertions(+), 8 deletions(-) diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index e287bd3da1f61..1e597d64e03fe 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -20,6 +20,13 @@ import copy import math +try: + from numpy import maximum, minimum, sqrt +except ImportError: + maximum = max + minimum = min + sqrt = math.sqrt + class StatCounter(object): @@ -39,10 +46,8 @@ def merge(self, value): self.n += 1 self.mu += delta / self.n self.m2 += delta * (value - self.mu) - if self.maxValue < value: - self.maxValue = value - if self.minValue > value: - self.minValue = value + self.maxValue = maximum(self.maxValue, value) + self.minValue = minimum(self.minValue, value) return self @@ -70,8 +75,8 @@ def mergeStats(self, other): else: self.mu = (self.mu * self.n + other.mu * other.n) / (self.n + other.n) - self.maxValue = max(self.maxValue, other.maxValue) - self.minValue = min(self.minValue, other.minValue) + self.maxValue = maximum(self.maxValue, other.maxValue) + self.minValue = minimum(self.minValue, other.minValue) self.m2 += other.m2 + (delta * delta * self.n * other.n) / (self.n + other.n) self.n += other.n @@ -115,14 +120,14 @@ def sampleVariance(self): # Return the standard deviation of the values. def stdev(self): - return math.sqrt(self.variance()) + return sqrt(self.variance()) # # Return the sample standard deviation of the values, which corrects for bias in estimating the # variance by dividing by N-1 instead of N. # def sampleStdev(self): - return math.sqrt(self.sampleVariance()) + return sqrt(self.sampleVariance()) def __repr__(self): return ("(count: %s, mean: %s, stdev: %s, max: %s, min: %s)" % diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index c29deb9574ea2..16fb5a9256220 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -38,12 +38,19 @@ from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger _have_scipy = False +_have_numpy = False try: import scipy.sparse _have_scipy = True except: # No SciPy, but that's okay, we'll skip those tests pass +try: + import numpy as np + _have_numpy = True +except: + # No NumPy, but that's okay, we'll skip those tests + pass SPARK_HOME = os.environ["SPARK_HOME"] @@ -914,9 +921,26 @@ def test_serialize(self): self.assertEqual(expected, observed) +@unittest.skipIf(not _have_numpy, "NumPy not installed") +class NumPyTests(PySparkTestCase): + """General PySpark tests that depend on numpy """ + + def test_statcounter_array(self): + x = self.sc.parallelize([np.array([1.0,1.0]), np.array([2.0,2.0]), np.array([3.0,3.0])]) + s = x.stats() + self.assertSequenceEqual([2.0,2.0], s.mean().tolist()) + self.assertSequenceEqual([1.0,1.0], s.min().tolist()) + self.assertSequenceEqual([3.0,3.0], s.max().tolist()) + self.assertSequenceEqual([1.0,1.0], s.sampleStdev().tolist()) + + if __name__ == "__main__": if not _have_scipy: print "NOTE: Skipping SciPy tests as it does not seem to be installed" + if not _have_numpy: + print "NOTE: Skipping NumPy tests as it does not seem to be installed" unittest.main() if not _have_scipy: print "NOTE: SciPy tests were skipped as it does not seem to be installed" + if not _have_numpy: + print "NOTE: NumPy tests were skipped as it does not seem to be installed" From adc8303294e26efb4ed15e5f5ba1062f7988625d Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Fri, 1 Aug 2014 23:55:11 -0700 Subject: [PATCH 0206/1492] [SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api Author: GuoQiang Li Closes #1369 from witgo/SPARK-1470_new and squashes the following commits: 66a1641 [GuoQiang Li] IncompatibleResultTypeProblem 73a89ba [GuoQiang Li] Use the scala-logging wrapper instead of the directly sfl4j api. --- core/pom.xml | 4 + .../main/scala/org/apache/spark/Logging.scala | 39 +++++--- .../org/apache/spark/util/SignalLogger.scala | 2 +- mllib/pom.xml | 4 + pom.xml | 5 + project/MimaExcludes.scala | 91 ++++++++++++++++++- sql/catalyst/pom.xml | 5 - .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../catalyst/analysis/HiveTypeCoercion.scala | 8 +- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../codegen/GenerateOrdering.scala | 4 +- .../apache/spark/sql/catalyst/package.scala | 1 - .../sql/catalyst/planning/QueryPlanner.scala | 2 +- .../sql/catalyst/planning/patterns.scala | 6 +- .../spark/sql/catalyst/rules/Rule.scala | 2 +- .../sql/catalyst/rules/RuleExecutor.scala | 12 +-- .../spark/sql/catalyst/trees/package.scala | 8 +- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../CompressibleColumnBuilder.scala | 5 +- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../org/apache/spark/sql/json/JsonRDD.scala | 2 +- .../scala/org/apache/spark/sql/package.scala | 2 - .../spark/sql/columnar/ColumnTypeSuite.scala | 4 +- .../hive/thriftserver/HiveThriftServer2.scala | 12 +-- .../hive/thriftserver/SparkSQLCLIDriver.scala | 2 +- .../hive/thriftserver/SparkSQLDriver.scala | 6 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 6 +- .../server/SparkSQLOperationManager.scala | 13 +-- .../thriftserver/HiveThriftServer2Suite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 3 +- .../org/apache/spark/sql/hive/TestHive.scala | 10 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 4 +- .../hive/execution/HiveComparisonTest.scala | 22 ++--- .../hive/execution/HiveQueryFileTest.scala | 2 +- 35 files changed, 203 insertions(+), 97 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 7c60cf10c3dc2..47766ae5fbb3d 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -98,6 +98,10 @@ org.slf4j jcl-over-slf4j + + com.typesafe.scala-logging + scala-logging-slf4j_${scala.binary.version} + log4j log4j diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 807ef3e9c9d60..6e61c00b8dbbf 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -18,8 +18,9 @@ package org.apache.spark import org.apache.log4j.{LogManager, PropertyConfigurator} -import org.slf4j.{Logger, LoggerFactory} +import org.slf4j.LoggerFactory import org.slf4j.impl.StaticLoggerBinder +import com.typesafe.scalalogging.slf4j.Logger import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -39,61 +40,69 @@ trait Logging { // be serialized and used on another machine @transient private var log_ : Logger = null + // Method to get the logger name for this object + protected def logName = { + var className = this.getClass.getName + // Ignore trailing $'s in the class names for Scala objects + if (className.endsWith("$")) { + className = className.substring(0, className.length - 1) + } + className + } + // Method to get or create the logger for this object protected def log: Logger = { if (log_ == null) { initializeIfNecessary() - var className = this.getClass.getName - // Ignore trailing $'s in the class names for Scala objects - log_ = LoggerFactory.getLogger(className.stripSuffix("$")) + log_ = Logger(LoggerFactory.getLogger(logName)) } log_ } // Log methods that take only a String protected def logInfo(msg: => String) { - if (log.isInfoEnabled) log.info(msg) + log.info(msg) } protected def logDebug(msg: => String) { - if (log.isDebugEnabled) log.debug(msg) + log.debug(msg) } protected def logTrace(msg: => String) { - if (log.isTraceEnabled) log.trace(msg) + log.trace(msg) } protected def logWarning(msg: => String) { - if (log.isWarnEnabled) log.warn(msg) + log.warn(msg) } protected def logError(msg: => String) { - if (log.isErrorEnabled) log.error(msg) + log.error(msg) } // Log methods that take Throwables (Exceptions/Errors) too protected def logInfo(msg: => String, throwable: Throwable) { - if (log.isInfoEnabled) log.info(msg, throwable) + log.info(msg, throwable) } protected def logDebug(msg: => String, throwable: Throwable) { - if (log.isDebugEnabled) log.debug(msg, throwable) + log.debug(msg, throwable) } protected def logTrace(msg: => String, throwable: Throwable) { - if (log.isTraceEnabled) log.trace(msg, throwable) + log.trace(msg, throwable) } protected def logWarning(msg: => String, throwable: Throwable) { - if (log.isWarnEnabled) log.warn(msg, throwable) + log.warn(msg, throwable) } protected def logError(msg: => String, throwable: Throwable) { - if (log.isErrorEnabled) log.error(msg, throwable) + log.error(msg, throwable) } protected def isTraceEnabled(): Boolean = { - log.isTraceEnabled + log.underlying.isTraceEnabled } private def initializeIfNecessary() { diff --git a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala index f77488ef3d449..e84a6b951f65e 100644 --- a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import org.apache.commons.lang3.SystemUtils -import org.slf4j.Logger +import com.typesafe.scalalogging.slf4j.Logger import sun.misc.{Signal, SignalHandler} /** diff --git a/mllib/pom.xml b/mllib/pom.xml index 9a33bd1cf6ad1..3007681a44f1c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -59,6 +59,10 @@ breeze_${scala.binary.version} 0.7 + + com.typesafe + scalalogging-slf4j_${scala.binary.version} + diff --git a/pom.xml b/pom.xml index ae97bf03c53a2..9d62cea68995f 100644 --- a/pom.xml +++ b/pom.xml @@ -279,6 +279,11 @@ slf4j-log4j12 ${slf4j.version} + + com.typesafe.scala-logging + scala-logging-slf4j_${scala.binary.version} + 2.1.2 + org.slf4j jul-to-slf4j diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 537ca0dcf267d..a0cee1d765c7f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -103,14 +103,101 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.mllib.tree.impurity.Variance.calculate") ) ++ - Seq ( // Package-private classes removed in SPARK-2341 + Seq( // Package-private classes removed in SPARK-2341 ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") - ) + ) ++ + Seq( + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.bagel.Bagel.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.StreamingContext.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.dstream.DStream.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.mllib.recommendation.ALS.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.mllib.clustering.KMeans.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.mllib.classification.NaiveBayes.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.kafka.KafkaReceiver.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.SparkContext.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.rdd.PairRDDFunctions.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.rdd.OrderedRDDFunctions.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.rdd.SequenceFileRDDFunctions.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.rdd.DoubleRDDFunctions.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.twitter.TwitterReceiver.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.flume.FlumeReceiver.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.rdd.RDD.log"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.SparkConf.log"), + + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.SparkConf.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.StreamingContext.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.dstream.DStream.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.mllib.recommendation.ALS.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.mllib.clustering.KMeans.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.mllib.classification.NaiveBayes.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.SparkContext.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.rdd.RDD.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.rdd.SequenceFileRDDFunctions.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.rdd.OrderedRDDFunctions.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.rdd.DoubleRDDFunctions.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.flume.FlumeReceiver.org$apache$spark$Logging$$log__="), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log_"), + ProblemFilters.exclude[IncompatibleMethTypeProblem] + ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log_"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log_"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.org$apache$spark$Logging$$log_"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log_"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log_"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.flume.FlumeReceiver.org$apache$spark$Logging$$log_"), + ProblemFilters.exclude[IncompatibleResultTypeProblem] + ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log_") + ) case v if v.startsWith("1.0") => Seq( MimaBuild.excludeSparkPackage("api.java"), diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 54fa96baa1e18..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -54,11 +54,6 @@ spark-core_${scala.binary.version} ${project.version} - - com.typesafe - scalalogging-slf4j_${scala.binary.version} - 1.0.1 - org.scalatest scalatest_${scala.binary.version} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 74c0104e5b17f..2b36582215f24 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -109,12 +109,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case q: LogicalPlan if q.childrenResolved => - logger.trace(s"Attempting to resolve ${q.simpleString}") + log.trace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = q.resolve(name).getOrElse(u) - logger.debug(s"Resolving $u to $result") + log.debug(s"Resolving $u to $result") result } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 47c7ad076ad07..eafbb70dc3fdd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -75,7 +75,7 @@ trait HiveTypeCoercion { // Leave the same if the dataTypes match. case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => - logger.debug(s"Promoting $a to $newType in ${q.simpleString}}") + log.debug(s"Promoting $a to $newType in ${q.simpleString}}") newType } } @@ -154,7 +154,7 @@ trait HiveTypeCoercion { (Alias(Cast(l, StringType), l.name)(), r) case (l, r) if l.dataType != r.dataType => - logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") + log.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") findTightestCommonType(l.dataType, r.dataType).map { widestType => val newLeft = if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)() @@ -170,7 +170,7 @@ trait HiveTypeCoercion { val newLeft = if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union $castedLeft ${left.output}") + log.debug(s"Widening numeric types in union $castedLeft ${left.output}") Project(castedLeft, left) } else { left @@ -178,7 +178,7 @@ trait HiveTypeCoercion { val newRight = if (castedRight.map(_.dataType) != right.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union $castedRight ${right.output}") + log.debug(s"Widening numeric types in union $castedRight ${right.output}") Project(castedRight, right) } else { right diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index f38f99569f207..0913f15888780 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 4211998f7511a..e2552d432cb71 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import com.typesafe.scalalogging.slf4j.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{StringType, NumericType} @@ -92,7 +92,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit } new $orderingName() """ - logger.debug(s"Generated Ordering: $code") + log.debug(s"Generated Ordering: $code") toolBox.eval(code).asInstanceOf[Ordering[Row]] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala index ca9642954eb27..bdd07bbeb2230 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -25,5 +25,4 @@ package object catalyst { */ protected[catalyst] object ScalaReflectionLock - protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 781ba489b44c6..5839c9f7c43ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.planning -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index bc763a4e06e67..06c5ffe92abc8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -184,7 +184,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { case join @ Join(left, right, joinType, condition) => - logger.debug(s"Considering join on: $condition") + log.debug(s"Considering join on: $condition") // Find equi-join predicates that can be evaluated before the join, and thus can be used // as join keys. val (joinPredicates, otherPredicates) = @@ -202,7 +202,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { val rightKeys = joinKeys.map(_._2) if (joinKeys.nonEmpty) { - logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") + log.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) } else { None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index f8960b3fe7a17..03414b2301e81 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 6aa407c836aec..20bf8eed7ddf3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.sideBySide @@ -60,7 +60,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { case (plan, rule) => val result = rule(plan) if (!result.fastEquals(plan)) { - logger.trace( + log.trace( s""" |=== Applying Rule ${rule.ruleName} === |${sideBySide(plan.treeString, result.treeString).mkString("\n")} @@ -73,26 +73,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { if (iteration > batch.strategy.maxIterations) { // Only log if this is a rule that is supposed to run more than once. if (iteration != 2) { - logger.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") + log.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") } continue = false } if (curPlan.fastEquals(lastPlan)) { - logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") + log.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") continue = false } lastPlan = curPlan } if (!batchStartPlan.fastEquals(curPlan)) { - logger.debug( + log.debug( s""" |=== Result of Batch ${batch.name} === |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")} """.stripMargin) } else { - logger.trace(s"Batch ${batch.name} has no effect.") + log.trace(s"Batch ${batch.name} has no effect.") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index 9a28d035a10a3..d725a92c06f7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.Logging + /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are * granted the following interface: @@ -31,8 +33,8 @@ package org.apache.spark.sql.catalyst *
  • debugging support - pretty printing, easy splicing of trees, etc.
  • * */ -package object trees { +package object trees extends Logging { // Since we want tree nodes to be lightweight, we create one logger for all treenode instances. - protected val logger = - com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("catalyst.trees")) + protected override def logName = "catalyst.trees" + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index dad71079c29b9..00dd34aabc389 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} /** * :: AlphaComponent :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 4c6675c3c87bf..828a8896ff60a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.columnar.compression import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.Logging +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} @@ -101,7 +102,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] copyColumnHeader(rawBuffer, compressedBuffer) - logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + log.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") encoder.compress(rawBuffer, compressedBuffer, columnType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 30712f03cab4c..0c3d537ccb494 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -101,7 +101,7 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl !operator.requiredChildDistribution.zip(operator.children).map { case (required, child) => val valid = child.outputPartitioning.satisfies(required) - logger.debug( + log.debug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 70db1ebd3a3e1..a3d2a1c7a51f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.Logging +import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 0995a4eb6299f..f513eae9c2d13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -32,8 +32,6 @@ import org.apache.spark.annotation.DeveloperApi */ package object sql { - protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging - /** * :: DeveloperApi :: * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 829342215e691..a165531573a20 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -22,7 +22,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -166,7 +166,7 @@ class ColumnTypeSuite extends FunSuite with Logging { buffer.rewind() seq.foreach { expected => - logger.info("buffer = " + buffer + ", expected = " + expected) + log.info("buffer = " + buffer + ", expected = " + expected) val extracted = columnType.extract(buffer) assert( expected === extracted, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index ddbc2a79fb512..5959ba3d23f8e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ @@ -40,7 +40,7 @@ private[hive] object HiveThriftServer2 extends Logging { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") if (!optionsProcessor.process(args)) { - logger.warn("Error starting HiveThriftServer2 with given arguments") + log.warn("Error starting HiveThriftServer2 with given arguments") System.exit(-1) } @@ -49,12 +49,12 @@ private[hive] object HiveThriftServer2 extends Logging { // Set all properties specified via command line. val hiveConf: HiveConf = ss.getConf hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logger.debug(s"HiveConf var: $k=$v") + log.debug(s"HiveConf var: $k=$v") } SessionState.start(ss) - logger.info("Starting SparkContext") + log.info("Starting SparkContext") SparkSQLEnv.init() SessionState.start(ss) @@ -70,10 +70,10 @@ private[hive] object HiveThriftServer2 extends Logging { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) server.init(hiveConf) server.start() - logger.info("HiveThriftServer2 started") + log.info("HiveThriftServer2 started") } catch { case e: Exception => - logger.error("Error starting HiveThriftServer2", e) + log.error("Error starting HiveThriftServer2", e) System.exit(-1) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index cb17d7ce58ea0..4d0c506c5a397 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket -import org.apache.spark.sql.Logging +import org.apache.spark.Logging private[hive] object SparkSQLCLIDriver { private var prompt = "spark-sql" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index a56b19a4bcda0..276723990b2ad 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) @@ -40,7 +40,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo private def getResultSetSchema(query: context.QueryExecution): Schema = { val analyzed = query.analyzed - logger.debug(s"Result Schema: ${analyzed.output}") + log.debug(s"Result Schema: ${analyzed.output}") if (analyzed.output.size == 0) { new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) } else { @@ -61,7 +61,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo new CommandProcessorResponse(0) } catch { case cause: Throwable => - logger.error(s"Failed in [$command]", cause) + log.error(s"Failed in [$command]", cause) new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 451c3bd7b9352..dfc93b19d019c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.{SparkConf, SparkContext} /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { - logger.debug("Initializing SparkSQLEnv") + log.debug("Initializing SparkSQLEnv") var hiveContext: HiveContext = _ var sparkContext: SparkContext = _ @@ -47,7 +47,7 @@ private[hive] object SparkSQLEnv extends Logging { /** Cleans up and shuts down the Spark SQL environments. */ def stop() { - logger.debug("Shutting down Spark SQL Environment") + log.debug("Shutting down Spark SQL Environment") // Stop the SparkContext if (SparkSQLEnv.sparkContext != null) { sparkContext.stop() diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index a4e1f3e762e89..2c6e24e80d6dd 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -30,10 +30,11 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -55,7 +56,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. - logger.debug("CLOSING") + log.debug("CLOSING") } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { @@ -112,7 +113,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def getResultSetSchema: TableSchema = { - logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + log.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { new TableSchema(new FieldSchema("Result", "string", "") :: Nil) } else { @@ -124,11 +125,11 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def run(): Unit = { - logger.info(s"Running query '$statement'") + log.info(s"Running query '$statement'") setState(OperationState.RUNNING) try { result = hiveContext.hql(statement) - logger.debug(result.queryExecution.toString()) + log.debug(result.queryExecution.toString()) val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) iter = result.queryExecution.toRdd.toLocalIterator @@ -138,7 +139,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => - logger.error("Error executing query:",e) + log.error("Error executing query:",e) throw new HiveSQLException(e.toString) } setState(OperationState.FINISHED) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index fe3403b3292ec..b7b7c9957ac34 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -27,7 +27,7 @@ import java.sql.{Connection, DriverManager, Statement} import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 7e3b8727bebed..1f31d35eaa10d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -207,7 +207,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } catch { case e: Exception => - logger.error( + log.error( s""" |====================== |HIVE FAILURE OUTPUT 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 fa4e78439c26c..df3604439e483 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 @@ -28,7 +28,8 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.{SQLContext, Logging} +import org.apache.spark.Logging +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index c50e8c4b5c5d3..7376fb5dc83f8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -148,7 +148,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { describedTables ++ logical.collect { case UnresolvedRelation(databaseName, name, _) => name } val referencedTestTables = referencedTables.filter(testTables.contains) - logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + log.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. analyzer(logical) @@ -273,7 +273,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. loadedTables += name - logger.info(s"Loading test table $name") + log.info(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -312,7 +312,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadedTables.clear() catalog.client.getAllTables("default").foreach { t => - logger.debug(s"Deleting table $t") + log.debug(s"Deleting table $t") val table = catalog.client.getTable("default", t) catalog.client.getIndexes("default", t, 255).foreach { index => @@ -325,7 +325,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { } catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - logger.debug(s"Dropping Database: $db") + log.debug(s"Dropping Database: $db") catalog.client.dropDatabase(db, true, false, true) } @@ -347,7 +347,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadTestTable("srcpart") } catch { case e: Exception => - logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") + log.error(s"FATAL ERROR: Failed to reset TestDB state. $e") // At this point there is really no reason to continue, but the test framework traps exits. // So instead we just pause forever so that at least the developer can see where things // started to go wrong. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 7582b4743d404..4d8eaa18d7844 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ @@ -119,7 +119,7 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) (a: Any) => { - logger.debug( + log.debug( s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") // We must make sure that primitives get boxed java style. if (a == null) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 6c8fe4b196dea..52cb1cf986f16 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -21,7 +21,7 @@ import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand => LogicalNativeCommand} @@ -197,7 +197,7 @@ abstract class HiveComparisonTest // If test sharding is enable, skip tests that are not in the correct shard. shardInfo.foreach { case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return - case (shardId, _) => logger.debug(s"Shard $shardId includes test '$testCaseName'") + case (shardId, _) => log.debug(s"Shard $shardId includes test '$testCaseName'") } // Skip tests found in directories specified by user. @@ -213,13 +213,13 @@ abstract class HiveComparisonTest .map(new File(_, testCaseName)) .filter(_.exists) if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { - logger.debug( + log.debug( s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") return } test(testCaseName) { - logger.debug(s"=== HIVE TEST: $testCaseName ===") + log.debug(s"=== HIVE TEST: $testCaseName ===") // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) @@ -235,7 +235,7 @@ abstract class HiveComparisonTest .filterNot(_ contains "hive.outerjoin.supports.filters") if (allQueries != queryList) - logger.warn(s"Simplifications made on unsupported operations for test $testCaseName") + log.warn(s"Simplifications made on unsupported operations for test $testCaseName") lazy val consoleTestCase = { val quotes = "\"\"\"" @@ -257,11 +257,11 @@ abstract class HiveComparisonTest } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - logger.debug(s"Looking for cached answer file $cachedAnswerFile.") + log.debug(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else { - logger.debug(s"File $cachedAnswerFile not found") + log.debug(s"File $cachedAnswerFile not found") None } }.map { @@ -272,7 +272,7 @@ abstract class HiveComparisonTest val hiveResults: Seq[Seq[String]] = if (hiveCachedResults.size == queryList.size) { - logger.info(s"Using answer cache for test: $testCaseName") + log.info(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { @@ -287,7 +287,7 @@ abstract class HiveComparisonTest if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) sys.error("hive exec hooks not supported for tests.") - logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") + log.warn(s"Running query ${i+1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. val answer = hiveQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. @@ -351,7 +351,7 @@ abstract class HiveComparisonTest val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") if (recomputeCache) { - logger.warn(s"Clearing cache files for failed test $testCaseName") + log.warn(s"Clearing cache files for failed test $testCaseName") hiveCacheFiles.foreach(_.delete()) } @@ -380,7 +380,7 @@ abstract class HiveComparisonTest TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") + log.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") // The testing setup traps exits so wait here for a long time so the developer can see when things started // to go wrong. Thread.sleep(1000000) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 50ab71a9003d3..9ca5575c1be8a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -53,7 +53,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { testCases.sorted.foreach { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { - logger.debug(s"Blacklisted test skipped $testCaseName") + log.debug(s"Blacklisted test skipped $testCaseName") } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) From dab37966b0cfd290919ca5c005f59dde00615c0e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 1 Aug 2014 23:55:30 -0700 Subject: [PATCH 0207/1492] Revert "[SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api" This reverts commit adc8303294e26efb4ed15e5f5ba1062f7988625d. --- core/pom.xml | 4 - .../main/scala/org/apache/spark/Logging.scala | 39 +++----- .../org/apache/spark/util/SignalLogger.scala | 2 +- mllib/pom.xml | 4 - pom.xml | 5 - project/MimaExcludes.scala | 91 +------------------ sql/catalyst/pom.xml | 5 + .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../catalyst/analysis/HiveTypeCoercion.scala | 8 +- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../codegen/GenerateOrdering.scala | 4 +- .../apache/spark/sql/catalyst/package.scala | 1 + .../sql/catalyst/planning/QueryPlanner.scala | 2 +- .../sql/catalyst/planning/patterns.scala | 6 +- .../spark/sql/catalyst/rules/Rule.scala | 2 +- .../sql/catalyst/rules/RuleExecutor.scala | 12 +-- .../spark/sql/catalyst/trees/package.scala | 8 +- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../CompressibleColumnBuilder.scala | 5 +- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../org/apache/spark/sql/json/JsonRDD.scala | 2 +- .../scala/org/apache/spark/sql/package.scala | 2 + .../spark/sql/columnar/ColumnTypeSuite.scala | 4 +- .../hive/thriftserver/HiveThriftServer2.scala | 12 +-- .../hive/thriftserver/SparkSQLCLIDriver.scala | 2 +- .../hive/thriftserver/SparkSQLDriver.scala | 6 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 6 +- .../server/SparkSQLOperationManager.scala | 13 ++- .../thriftserver/HiveThriftServer2Suite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 3 +- .../org/apache/spark/sql/hive/TestHive.scala | 10 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 4 +- .../hive/execution/HiveComparisonTest.scala | 22 ++--- .../hive/execution/HiveQueryFileTest.scala | 2 +- 35 files changed, 97 insertions(+), 203 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 47766ae5fbb3d..7c60cf10c3dc2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -98,10 +98,6 @@ org.slf4j jcl-over-slf4j
    - - com.typesafe.scala-logging - scala-logging-slf4j_${scala.binary.version} - log4j log4j diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 6e61c00b8dbbf..807ef3e9c9d60 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -18,9 +18,8 @@ package org.apache.spark import org.apache.log4j.{LogManager, PropertyConfigurator} -import org.slf4j.LoggerFactory +import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder -import com.typesafe.scalalogging.slf4j.Logger import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -40,69 +39,61 @@ trait Logging { // be serialized and used on another machine @transient private var log_ : Logger = null - // Method to get the logger name for this object - protected def logName = { - var className = this.getClass.getName - // Ignore trailing $'s in the class names for Scala objects - if (className.endsWith("$")) { - className = className.substring(0, className.length - 1) - } - className - } - // Method to get or create the logger for this object protected def log: Logger = { if (log_ == null) { initializeIfNecessary() - log_ = Logger(LoggerFactory.getLogger(logName)) + var className = this.getClass.getName + // Ignore trailing $'s in the class names for Scala objects + log_ = LoggerFactory.getLogger(className.stripSuffix("$")) } log_ } // Log methods that take only a String protected def logInfo(msg: => String) { - log.info(msg) + if (log.isInfoEnabled) log.info(msg) } protected def logDebug(msg: => String) { - log.debug(msg) + if (log.isDebugEnabled) log.debug(msg) } protected def logTrace(msg: => String) { - log.trace(msg) + if (log.isTraceEnabled) log.trace(msg) } protected def logWarning(msg: => String) { - log.warn(msg) + if (log.isWarnEnabled) log.warn(msg) } protected def logError(msg: => String) { - log.error(msg) + if (log.isErrorEnabled) log.error(msg) } // Log methods that take Throwables (Exceptions/Errors) too protected def logInfo(msg: => String, throwable: Throwable) { - log.info(msg, throwable) + if (log.isInfoEnabled) log.info(msg, throwable) } protected def logDebug(msg: => String, throwable: Throwable) { - log.debug(msg, throwable) + if (log.isDebugEnabled) log.debug(msg, throwable) } protected def logTrace(msg: => String, throwable: Throwable) { - log.trace(msg, throwable) + if (log.isTraceEnabled) log.trace(msg, throwable) } protected def logWarning(msg: => String, throwable: Throwable) { - log.warn(msg, throwable) + if (log.isWarnEnabled) log.warn(msg, throwable) } protected def logError(msg: => String, throwable: Throwable) { - log.error(msg, throwable) + if (log.isErrorEnabled) log.error(msg, throwable) } protected def isTraceEnabled(): Boolean = { - log.underlying.isTraceEnabled + log.isTraceEnabled } private def initializeIfNecessary() { diff --git a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala index e84a6b951f65e..f77488ef3d449 100644 --- a/core/src/main/scala/org/apache/spark/util/SignalLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/SignalLogger.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import org.apache.commons.lang3.SystemUtils -import com.typesafe.scalalogging.slf4j.Logger +import org.slf4j.Logger import sun.misc.{Signal, SignalHandler} /** diff --git a/mllib/pom.xml b/mllib/pom.xml index 3007681a44f1c..9a33bd1cf6ad1 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -59,10 +59,6 @@ breeze_${scala.binary.version} 0.7 - - com.typesafe - scalalogging-slf4j_${scala.binary.version} - diff --git a/pom.xml b/pom.xml index 9d62cea68995f..ae97bf03c53a2 100644 --- a/pom.xml +++ b/pom.xml @@ -279,11 +279,6 @@ slf4j-log4j12 ${slf4j.version} - - com.typesafe.scala-logging - scala-logging-slf4j_${scala.binary.version} - 2.1.2 - org.slf4j jul-to-slf4j diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a0cee1d765c7f..537ca0dcf267d 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -103,101 +103,14 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.mllib.tree.impurity.Variance.calculate") ) ++ - Seq( // Package-private classes removed in SPARK-2341 + Seq ( // Package-private classes removed in SPARK-2341 ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") - ) ++ - Seq( - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.bagel.Bagel.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.StreamingContext.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.dstream.DStream.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.mllib.recommendation.ALS.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.mllib.clustering.KMeans.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.mllib.classification.NaiveBayes.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.kafka.KafkaReceiver.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.SparkContext.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.rdd.PairRDDFunctions.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.rdd.OrderedRDDFunctions.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.rdd.SequenceFileRDDFunctions.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.rdd.DoubleRDDFunctions.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.twitter.TwitterReceiver.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.flume.FlumeReceiver.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.rdd.RDD.log"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.SparkConf.log"), - - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.SparkConf.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.StreamingContext.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.dstream.DStream.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.mllib.recommendation.ALS.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.mllib.clustering.KMeans.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.mllib.classification.NaiveBayes.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.SparkContext.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.rdd.RDD.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.rdd.SequenceFileRDDFunctions.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.rdd.OrderedRDDFunctions.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.rdd.PairRDDFunctions.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.rdd.DoubleRDDFunctions.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.flume.FlumeReceiver.org$apache$spark$Logging$$log__="), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log_"), - ProblemFilters.exclude[IncompatibleMethTypeProblem] - ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log_"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.twitter.TwitterReceiver.org$apache$spark$Logging$$log_"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.zeromq.ZeroMQReceiver.org$apache$spark$Logging$$log_"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log_"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.bagel.Bagel.org$apache$spark$Logging$$log_"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.flume.FlumeReceiver.org$apache$spark$Logging$$log_"), - ProblemFilters.exclude[IncompatibleResultTypeProblem] - ("org.apache.spark.streaming.kafka.KafkaReceiver.org$apache$spark$Logging$$log_") - ) + ) case v if v.startsWith("1.0") => Seq( MimaBuild.excludeSparkPackage("api.java"), diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..54fa96baa1e18 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -54,6 +54,11 @@ spark-core_${scala.binary.version} ${project.version} + + com.typesafe + scalalogging-slf4j_${scala.binary.version} + 1.0.1 + org.scalatest scalatest_${scala.binary.version} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 2b36582215f24..74c0104e5b17f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -109,12 +109,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case q: LogicalPlan if q.childrenResolved => - log.trace(s"Attempting to resolve ${q.simpleString}") + logger.trace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = q.resolve(name).getOrElse(u) - log.debug(s"Resolving $u to $result") + logger.debug(s"Resolving $u to $result") result } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index eafbb70dc3fdd..47c7ad076ad07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -75,7 +75,7 @@ trait HiveTypeCoercion { // Leave the same if the dataTypes match. case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => - log.debug(s"Promoting $a to $newType in ${q.simpleString}}") + logger.debug(s"Promoting $a to $newType in ${q.simpleString}}") newType } } @@ -154,7 +154,7 @@ trait HiveTypeCoercion { (Alias(Cast(l, StringType), l.name)(), r) case (l, r) if l.dataType != r.dataType => - log.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") + logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") findTightestCommonType(l.dataType, r.dataType).map { widestType => val newLeft = if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)() @@ -170,7 +170,7 @@ trait HiveTypeCoercion { val newLeft = if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) { - log.debug(s"Widening numeric types in union $castedLeft ${left.output}") + logger.debug(s"Widening numeric types in union $castedLeft ${left.output}") Project(castedLeft, left) } else { left @@ -178,7 +178,7 @@ trait HiveTypeCoercion { val newRight = if (castedRight.map(_.dataType) != right.output.map(_.dataType)) { - log.debug(s"Widening numeric types in union $castedRight ${right.output}") + logger.debug(s"Widening numeric types in union $castedRight ${right.output}") Project(castedRight, right) } else { right diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 0913f15888780..f38f99569f207 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index e2552d432cb71..4211998f7511a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import org.apache.spark.Logging +import com.typesafe.scalalogging.slf4j.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{StringType, NumericType} @@ -92,7 +92,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit } new $orderingName() """ - log.debug(s"Generated Ordering: $code") + logger.debug(s"Generated Ordering: $code") toolBox.eval(code).asInstanceOf[Ordering[Row]] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala index bdd07bbeb2230..ca9642954eb27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -25,4 +25,5 @@ package object catalyst { */ protected[catalyst] object ScalaReflectionLock + protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 5839c9f7c43ef..781ba489b44c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.planning -import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 06c5ffe92abc8..bc763a4e06e67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -184,7 +184,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { case join @ Join(left, right, joinType, condition) => - log.debug(s"Considering join on: $condition") + logger.debug(s"Considering join on: $condition") // Find equi-join predicates that can be evaluated before the join, and thus can be used // as join keys. val (joinPredicates, otherPredicates) = @@ -202,7 +202,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { val rightKeys = joinKeys.map(_._2) if (joinKeys.nonEmpty) { - log.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") + logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) } else { None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index 03414b2301e81..f8960b3fe7a17 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 20bf8eed7ddf3..6aa407c836aec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.Logging import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.sideBySide @@ -60,7 +60,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { case (plan, rule) => val result = rule(plan) if (!result.fastEquals(plan)) { - log.trace( + logger.trace( s""" |=== Applying Rule ${rule.ruleName} === |${sideBySide(plan.treeString, result.treeString).mkString("\n")} @@ -73,26 +73,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { if (iteration > batch.strategy.maxIterations) { // Only log if this is a rule that is supposed to run more than once. if (iteration != 2) { - log.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") + logger.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") } continue = false } if (curPlan.fastEquals(lastPlan)) { - log.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") + logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") continue = false } lastPlan = curPlan } if (!batchStartPlan.fastEquals(curPlan)) { - log.debug( + logger.debug( s""" |=== Result of Batch ${batch.name} === |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")} """.stripMargin) } else { - log.trace(s"Batch ${batch.name} has no effect.") + logger.trace(s"Batch ${batch.name} has no effect.") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index d725a92c06f7b..9a28d035a10a3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.Logging - /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are * granted the following interface: @@ -33,8 +31,8 @@ import org.apache.spark.Logging *
  • debugging support - pretty printing, easy splicing of trees, etc.
  • * */ -package object trees extends Logging { +package object trees { // Since we want tree nodes to be lightweight, we create one logger for all treenode instances. - protected override def logName = "catalyst.trees" - + protected val logger = + com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("catalyst.trees")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 00dd34aabc389..dad71079c29b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext /** * :: AlphaComponent :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 828a8896ff60a..4c6675c3c87bf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -19,8 +19,7 @@ package org.apache.spark.sql.columnar.compression import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.Logging -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Logging, Row} import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} @@ -102,7 +101,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] copyColumnHeader(rawBuffer, compressedBuffer) - log.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") encoder.compress(rawBuffer, compressedBuffer, columnType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 0c3d537ccb494..30712f03cab4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -101,7 +101,7 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl !operator.requiredChildDistribution.zip(operator.children).map { case (required, child) => val valid = child.outputPartitioning.satisfies(required) - log.debug( + logger.debug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index a3d2a1c7a51f8..70db1ebd3a3e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.Logging +import org.apache.spark.sql.Logging private[sql] object JsonRDD extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index f513eae9c2d13..0995a4eb6299f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -32,6 +32,8 @@ import org.apache.spark.annotation.DeveloperApi */ package object sql { + protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging + /** * :: DeveloperApi :: * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index a165531573a20..829342215e691 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -22,7 +22,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite -import org.apache.spark.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -166,7 +166,7 @@ class ColumnTypeSuite extends FunSuite with Logging { buffer.rewind() seq.foreach { expected => - log.info("buffer = " + buffer + ", expected = " + expected) + logger.info("buffer = " + buffer + ", expected = " + expected) val extracted = columnType.extract(buffer) assert( expected === extracted, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 5959ba3d23f8e..ddbc2a79fb512 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} -import org.apache.spark.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ @@ -40,7 +40,7 @@ private[hive] object HiveThriftServer2 extends Logging { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") if (!optionsProcessor.process(args)) { - log.warn("Error starting HiveThriftServer2 with given arguments") + logger.warn("Error starting HiveThriftServer2 with given arguments") System.exit(-1) } @@ -49,12 +49,12 @@ private[hive] object HiveThriftServer2 extends Logging { // Set all properties specified via command line. val hiveConf: HiveConf = ss.getConf hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - log.debug(s"HiveConf var: $k=$v") + logger.debug(s"HiveConf var: $k=$v") } SessionState.start(ss) - log.info("Starting SparkContext") + logger.info("Starting SparkContext") SparkSQLEnv.init() SessionState.start(ss) @@ -70,10 +70,10 @@ private[hive] object HiveThriftServer2 extends Logging { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) server.init(hiveConf) server.start() - log.info("HiveThriftServer2 started") + logger.info("HiveThriftServer2 started") } catch { case e: Exception => - log.error("Error starting HiveThriftServer2", e) + logger.error("Error starting HiveThriftServer2", e) System.exit(-1) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 4d0c506c5a397..cb17d7ce58ea0 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket -import org.apache.spark.Logging +import org.apache.spark.sql.Logging private[hive] object SparkSQLCLIDriver { private var prompt = "spark-sql" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 276723990b2ad..a56b19a4bcda0 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse -import org.apache.spark.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) @@ -40,7 +40,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo private def getResultSetSchema(query: context.QueryExecution): Schema = { val analyzed = query.analyzed - log.debug(s"Result Schema: ${analyzed.output}") + logger.debug(s"Result Schema: ${analyzed.output}") if (analyzed.output.size == 0) { new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) } else { @@ -61,7 +61,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo new CommandProcessorResponse(0) } catch { case cause: Throwable => - log.error(s"Failed in [$command]", cause) + logger.error(s"Failed in [$command]", cause) new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index dfc93b19d019c..451c3bd7b9352 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.{SparkConf, SparkContext} /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { - log.debug("Initializing SparkSQLEnv") + logger.debug("Initializing SparkSQLEnv") var hiveContext: HiveContext = _ var sparkContext: SparkContext = _ @@ -47,7 +47,7 @@ private[hive] object SparkSQLEnv extends Logging { /** Cleans up and shuts down the Spark SQL environments. */ def stop() { - log.debug("Shutting down Spark SQL Environment") + logger.debug("Shutting down Spark SQL Environment") // Stop the SparkContext if (SparkSQLEnv.sparkContext != null) { sparkContext.stop() diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 2c6e24e80d6dd..a4e1f3e762e89 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -30,11 +30,10 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession -import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -56,7 +55,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. - log.debug("CLOSING") + logger.debug("CLOSING") } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { @@ -113,7 +112,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def getResultSetSchema: TableSchema = { - log.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { new TableSchema(new FieldSchema("Result", "string", "") :: Nil) } else { @@ -125,11 +124,11 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def run(): Unit = { - log.info(s"Running query '$statement'") + logger.info(s"Running query '$statement'") setState(OperationState.RUNNING) try { result = hiveContext.hql(statement) - log.debug(result.queryExecution.toString()) + logger.debug(result.queryExecution.toString()) val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) iter = result.queryExecution.toRdd.toLocalIterator @@ -139,7 +138,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => - log.error("Error executing query:",e) + logger.error("Error executing query:",e) throw new HiveSQLException(e.toString) } setState(OperationState.FINISHED) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index b7b7c9957ac34..fe3403b3292ec 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -27,7 +27,7 @@ import java.sql.{Connection, DriverManager, Statement} import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.apache.spark.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 1f31d35eaa10d..7e3b8727bebed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -207,7 +207,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } catch { case e: Exception => - log.error( + logger.error( s""" |====================== |HIVE FAILURE OUTPUT 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 df3604439e483..fa4e78439c26c 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 @@ -28,8 +28,7 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.Logging -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SQLContext, Logging} import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 7376fb5dc83f8..c50e8c4b5c5d3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -148,7 +148,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { describedTables ++ logical.collect { case UnresolvedRelation(databaseName, name, _) => name } val referencedTestTables = referencedTables.filter(testTables.contains) - log.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. analyzer(logical) @@ -273,7 +273,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. loadedTables += name - log.info(s"Loading test table $name") + logger.info(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -312,7 +312,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadedTables.clear() catalog.client.getAllTables("default").foreach { t => - log.debug(s"Deleting table $t") + logger.debug(s"Deleting table $t") val table = catalog.client.getTable("default", t) catalog.client.getIndexes("default", t, 255).foreach { index => @@ -325,7 +325,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { } catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - log.debug(s"Dropping Database: $db") + logger.debug(s"Dropping Database: $db") catalog.client.dropDatabase(db, true, false, true) } @@ -347,7 +347,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadTestTable("srcpart") } catch { case e: Exception => - log.error(s"FATAL ERROR: Failed to reset TestDB state. $e") + logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") // At this point there is really no reason to continue, but the test framework traps exits. // So instead we just pause forever so that at least the developer can see where things // started to go wrong. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 4d8eaa18d7844..7582b4743d404 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.spark.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ @@ -119,7 +119,7 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) (a: Any) => { - log.debug( + logger.debug( s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") // We must make sure that primitives get boxed java style. if (a == null) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 52cb1cf986f16..6c8fe4b196dea 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -21,7 +21,7 @@ import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import org.apache.spark.Logging +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand => LogicalNativeCommand} @@ -197,7 +197,7 @@ abstract class HiveComparisonTest // If test sharding is enable, skip tests that are not in the correct shard. shardInfo.foreach { case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return - case (shardId, _) => log.debug(s"Shard $shardId includes test '$testCaseName'") + case (shardId, _) => logger.debug(s"Shard $shardId includes test '$testCaseName'") } // Skip tests found in directories specified by user. @@ -213,13 +213,13 @@ abstract class HiveComparisonTest .map(new File(_, testCaseName)) .filter(_.exists) if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { - log.debug( + logger.debug( s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") return } test(testCaseName) { - log.debug(s"=== HIVE TEST: $testCaseName ===") + logger.debug(s"=== HIVE TEST: $testCaseName ===") // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) @@ -235,7 +235,7 @@ abstract class HiveComparisonTest .filterNot(_ contains "hive.outerjoin.supports.filters") if (allQueries != queryList) - log.warn(s"Simplifications made on unsupported operations for test $testCaseName") + logger.warn(s"Simplifications made on unsupported operations for test $testCaseName") lazy val consoleTestCase = { val quotes = "\"\"\"" @@ -257,11 +257,11 @@ abstract class HiveComparisonTest } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - log.debug(s"Looking for cached answer file $cachedAnswerFile.") + logger.debug(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else { - log.debug(s"File $cachedAnswerFile not found") + logger.debug(s"File $cachedAnswerFile not found") None } }.map { @@ -272,7 +272,7 @@ abstract class HiveComparisonTest val hiveResults: Seq[Seq[String]] = if (hiveCachedResults.size == queryList.size) { - log.info(s"Using answer cache for test: $testCaseName") + logger.info(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { @@ -287,7 +287,7 @@ abstract class HiveComparisonTest if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) sys.error("hive exec hooks not supported for tests.") - log.warn(s"Running query ${i+1}/${queryList.size} with hive.") + logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. val answer = hiveQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. @@ -351,7 +351,7 @@ abstract class HiveComparisonTest val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") if (recomputeCache) { - log.warn(s"Clearing cache files for failed test $testCaseName") + logger.warn(s"Clearing cache files for failed test $testCaseName") hiveCacheFiles.foreach(_.delete()) } @@ -380,7 +380,7 @@ abstract class HiveComparisonTest TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - log.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") + logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") // The testing setup traps exits so wait here for a long time so the developer can see when things started // to go wrong. Thread.sleep(1000000) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 9ca5575c1be8a..50ab71a9003d3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -53,7 +53,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { testCases.sorted.foreach { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { - log.debug(s"Blacklisted test skipped $testCaseName") + logger.debug(s"Blacklisted test skipped $testCaseName") } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) From d934801d53fc2f1d57d3534ae4e1e9384c7dda99 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 1 Aug 2014 23:56:24 -0700 Subject: [PATCH 0208/1492] [SPARK-2316] Avoid O(blocks) operations in listeners The existing code in `StorageUtils` is not the most efficient. Every time we want to update an `RDDInfo` we end up iterating through all blocks on all block managers just to discard most of them. The symptoms manifest themselves in the bountiful UI bugs observed in the wild. Many of these bugs are caused by the slow consumption of events in `LiveListenerBus`, which frequently leads to the event queue overflowing and `SparkListenerEvent`s being dropped on the floor. The changes made in this PR avoid this by first filtering out only the blocks relevant to us before computing storage information from them. It's worth a mention that this corner of the Spark code is also not very well-tested at all. The bulk of the changes in this PR (more than 60%) is actually test cases for the various logic in `StorageUtils.scala` as well as `StorageTab.scala`. These will eventually be extended to cover the various listeners that constitute the `SparkUI`. Author: Andrew Or Closes #1679 from andrewor14/fix-drop-events and squashes the following commits: f80c1fa [Andrew Or] Rewrite fold and reduceOption as sum e132d69 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events 14fa1c3 [Andrew Or] Simplify some code + update a few comments a91be46 [Andrew Or] Make ExecutorsPage blazingly fast bf6f09b [Andrew Or] Minor changes 8981de1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events af19bc0 [Andrew Or] *UsedByRDD -> *UsedByRdd (minor) 6970bc8 [Andrew Or] Add extensive tests for StorageListener and the new code in StorageUtils e080b9e [Andrew Or] Reduce run time of StorageUtils.updateRddInfo to near constant 2c3ef6a [Andrew Or] Actually filter out only the relevant RDDs 6fef86a [Andrew Or] Add extensive tests for new code in StorageStatus b66b6b0 [Andrew Or] Use more efficient underlying data structures for blocks 6a7b7c0 [Andrew Or] Avoid chained operations on TraversableLike a9ec384 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events b12fcd7 [Andrew Or] Fix tests + simplify sc.getRDDStorageInfo da8e322 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-drop-events 8e91921 [Andrew Or] Iterate through a filtered set of blocks when updating RDDInfo 7b2c4aa [Andrew Or] Rewrite blockLocationsFromStorageStatus + clean up method signatures 41fa50d [Andrew Or] Add a legacy constructor for StorageStatus 53af15d [Andrew Or] Refactor StorageStatus + add a bunch of tests --- .../scala/org/apache/spark/SparkContext.scala | 6 +- .../storage/BlockManagerMasterActor.scala | 14 +- .../spark/storage/BlockManagerSource.scala | 14 +- .../org/apache/spark/storage/RDDInfo.scala | 2 + .../spark/storage/StorageStatusListener.scala | 12 +- .../apache/spark/storage/StorageUtils.scala | 316 +++++++++++----- .../apache/spark/ui/exec/ExecutorsPage.scala | 12 +- .../org/apache/spark/ui/storage/RDDPage.scala | 17 +- .../apache/spark/ui/storage/StorageTab.scala | 13 +- .../apache/spark/SparkContextInfoSuite.scala | 22 +- .../storage/StorageStatusListenerSuite.scala | 72 ++-- .../apache/spark/storage/StorageSuite.scala | 354 ++++++++++++++++++ .../spark/ui/storage/StorageTabSuite.scala | 165 ++++++++ 13 files changed, 843 insertions(+), 176 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/StorageSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 368835a867493..9ba21cfcde01a 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -48,7 +48,7 @@ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend -import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} +import org.apache.spark.storage._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} @@ -843,7 +843,9 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def getRDDStorageInfo: Array[RDDInfo] = { - StorageUtils.rddInfoFromStorageStatus(getExecutorStorageStatus, this) + val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray + StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) + rddInfos.filter(_.isCached) } /** 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 94f5a4bb2e9cd..bd31e3c5a187f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -267,9 +267,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } private def storageStatus: Array[StorageStatus] = { - blockManagerInfo.map { case(blockManagerId, info) => - val blockMap = mutable.Map[BlockId, BlockStatus](info.blocks.toSeq: _*) - new StorageStatus(blockManagerId, info.maxMem, blockMap) + blockManagerInfo.map { case (blockManagerId, info) => + new StorageStatus(blockManagerId, info.maxMem, info.blocks) }.toArray } @@ -424,7 +423,14 @@ case class BlockStatus( storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) + tachyonSize: Long) { + def isCached: Boolean = memSize + diskSize + tachyonSize > 0 +} + +@DeveloperApi +object BlockStatus { + def empty: BlockStatus = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) +} private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 687586490abfe..e939318a029dd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -30,7 +30,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum maxMem / 1024 / 1024 } }) @@ -38,7 +38,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "remainingMem_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + val remainingMem = storageStatusList.map(_.memRemaining).sum remainingMem / 1024 / 1024 } }) @@ -46,8 +46,8 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).reduce(_ + _) - val remainingMem = storageStatusList.map(_.memRemaining).reduce(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum + val remainingMem = storageStatusList.map(_.memRemaining).sum (maxMem - remainingMem) / 1024 / 1024 } }) @@ -55,11 +55,7 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("disk", "diskSpaceUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val diskSpaceUsed = storageStatusList - .flatMap(_.blocks.values.map(_.diskSize)) - .reduceOption(_ + _) - .getOrElse(0L) - + val diskSpaceUsed = storageStatusList.map(_.diskUsed).sum diskSpaceUsed / 1024 / 1024 } }) diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 5a72e216872a6..120c327a7e580 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -34,6 +34,8 @@ class RDDInfo( var diskSize = 0L var tachyonSize = 0L + def isCached: Boolean = (memSize + diskSize + tachyonSize > 0) && numCachedPartitions > 0 + override def toString = { import Utils.bytesToString ("RDD \"%s\" (%d) StorageLevel: %s; CachedPartitions: %d; TotalPartitions: %d; " + diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 41c960c867e2e..d9066f766476e 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -35,13 +35,12 @@ class StorageStatusListener extends SparkListener { /** Update storage status list to reflect updated block statuses */ private def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { - val filteredStatus = executorIdToStorageStatus.get(execId) - filteredStatus.foreach { storageStatus => + executorIdToStorageStatus.get(execId).foreach { storageStatus => updatedBlocks.foreach { case (blockId, updatedStatus) => if (updatedStatus.storageLevel == StorageLevel.NONE) { - storageStatus.blocks.remove(blockId) + storageStatus.removeBlock(blockId) } else { - storageStatus.blocks(blockId) = updatedStatus + storageStatus.updateBlock(blockId, updatedStatus) } } } @@ -50,9 +49,8 @@ class StorageStatusListener extends SparkListener { /** Update storage status list to reflect the removal of an RDD from the cache */ private def updateStorageStatus(unpersistedRDDId: Int) { storageStatusList.foreach { storageStatus => - val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) - unpersistedBlocksIds.foreach { blockId => - storageStatus.blocks.remove(blockId) + storageStatus.rddBlocksById(unpersistedRDDId).foreach { case (blockId, _) => + storageStatus.removeBlock(blockId) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 177281f663367..0a0a448baa2ef 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -20,122 +20,258 @@ package org.apache.spark.storage import scala.collection.Map import scala.collection.mutable -import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: * Storage information for each BlockManager. + * + * This class assumes BlockId and BlockStatus are immutable, such that the consumers of this + * class cannot mutate the source of the information. Accesses are not thread-safe. */ @DeveloperApi -class StorageStatus( - val blockManagerId: BlockManagerId, - val maxMem: Long, - val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { +class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { - def memUsed = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + /** + * Internal representation of the blocks stored in this block manager. + * + * We store RDD blocks and non-RDD blocks separately to allow quick retrievals of RDD blocks. + * These collections should only be mutated through the add/update/removeBlock methods. + */ + private val _rddBlocks = new mutable.HashMap[Int, mutable.Map[BlockId, BlockStatus]] + private val _nonRddBlocks = new mutable.HashMap[BlockId, BlockStatus] - def memUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + /** + * Storage information of the blocks that entails memory, disk, and off-heap memory usage. + * + * As with the block maps, we store the storage information separately for RDD blocks and + * non-RDD blocks for the same reason. In particular, RDD storage information is stored + * in a map indexed by the RDD ID to the following 4-tuple: + * + * (memory size, disk size, off-heap size, storage level) + * + * We assume that all the blocks that belong to the same RDD have the same storage level. + * This field is not relevant to non-RDD blocks, however, so the storage information for + * non-RDD blocks contains only the first 3 fields (in the same order). + */ + private val _rddStorageInfo = new mutable.HashMap[Int, (Long, Long, Long, StorageLevel)] + private var _nonRddStorageInfo: (Long, Long, Long) = (0L, 0L, 0L) - def diskUsed = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + /** Create a storage status with an initial set of blocks, leaving the source unmodified. */ + def this(bmid: BlockManagerId, maxMem: Long, initialBlocks: Map[BlockId, BlockStatus]) { + this(bmid, maxMem) + initialBlocks.foreach { case (bid, bstatus) => addBlock(bid, bstatus) } + } - def diskUsedByRDD(rddId: Int) = - rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + /** + * Return the blocks stored in this block manager. + * + * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * concatenating them together. Much faster alternatives exist for common operations such as + * contains, get, and size. + */ + def blocks: Map[BlockId, BlockStatus] = _nonRddBlocks ++ rddBlocks - def memRemaining: Long = maxMem - memUsed + /** + * Return the RDD blocks stored in this block manager. + * + * Note that this is somewhat expensive, as it involves cloning the underlying maps and then + * concatenating them together. Much faster alternatives exist for common operations such as + * getting the memory, disk, and off-heap memory sizes occupied by this RDD. + */ + def rddBlocks: Map[BlockId, BlockStatus] = _rddBlocks.flatMap { case (_, blocks) => blocks } - def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } -} + /** Return the blocks that belong to the given RDD stored in this block manager. */ + def rddBlocksById(rddId: Int): Map[BlockId, BlockStatus] = { + _rddBlocks.get(rddId).getOrElse(Map.empty) + } -/** Helper methods for storage-related objects. */ -private[spark] object StorageUtils { + /** Add the given block to this storage status. If it already exists, overwrite it. */ + private[spark] def addBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + updateStorageInfo(blockId, blockStatus) + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.getOrElseUpdate(rddId, new mutable.HashMap)(blockId) = blockStatus + case _ => + _nonRddBlocks(blockId) = blockStatus + } + } + + /** Update the given block in this storage status. If it doesn't already exist, add it. */ + private[spark] def updateBlock(blockId: BlockId, blockStatus: BlockStatus): Unit = { + addBlock(blockId, blockStatus) + } + + /** Remove the given block from this storage status. */ + private[spark] def removeBlock(blockId: BlockId): Option[BlockStatus] = { + updateStorageInfo(blockId, BlockStatus.empty) + blockId match { + case RDDBlockId(rddId, _) => + // Actually remove the block, if it exists + if (_rddBlocks.contains(rddId)) { + val removed = _rddBlocks(rddId).remove(blockId) + // If the given RDD has no more blocks left, remove the RDD + if (_rddBlocks(rddId).isEmpty) { + _rddBlocks.remove(rddId) + } + removed + } else { + None + } + case _ => + _nonRddBlocks.remove(blockId) + } + } /** - * Returns basic information of all RDDs persisted in the given SparkContext. This does not - * include storage information. + * Return whether the given block is stored in this block manager in O(1) time. + * Note that this is much faster than `this.blocks.contains`, which is O(blocks) time. */ - def rddInfoFromSparkContext(sc: SparkContext): Array[RDDInfo] = { - sc.persistentRdds.values.map { rdd => - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - val rddNumPartitions = rdd.partitions.size - val rddStorageLevel = rdd.getStorageLevel - val rddInfo = new RDDInfo(rdd.id, rddName, rddNumPartitions, rddStorageLevel) - rddInfo - }.toArray + def containsBlock(blockId: BlockId): Boolean = { + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.get(rddId).exists(_.contains(blockId)) + case _ => + _nonRddBlocks.contains(blockId) + } } - /** Returns storage information of all RDDs persisted in the given SparkContext. */ - def rddInfoFromStorageStatus( - storageStatuses: Seq[StorageStatus], - sc: SparkContext): Array[RDDInfo] = { - rddInfoFromStorageStatus(storageStatuses, rddInfoFromSparkContext(sc)) + /** + * Return the given block stored in this block manager in O(1) time. + * Note that this is much faster than `this.blocks.get`, which is O(blocks) time. + */ + def getBlock(blockId: BlockId): Option[BlockStatus] = { + blockId match { + case RDDBlockId(rddId, _) => + _rddBlocks.get(rddId).map(_.get(blockId)).flatten + case _ => + _nonRddBlocks.get(blockId) + } } - /** Returns storage information of all RDDs in the given list. */ - def rddInfoFromStorageStatus( - storageStatuses: Seq[StorageStatus], - rddInfos: Seq[RDDInfo], - updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty): Array[RDDInfo] = { - - // Mapping from a block ID -> its status - val blockMap = mutable.Map(storageStatuses.flatMap(_.rddBlocks): _*) - - // Record updated blocks, if any - updatedBlocks - .collect { case (id: RDDBlockId, status) => (id, status) } - .foreach { case (id, status) => blockMap(id) = status } - - // Mapping from RDD ID -> an array of associated BlockStatuses - val rddBlockMap = blockMap - .groupBy { case (k, _) => k.rddId } - .mapValues(_.values.toArray) - - // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information) - val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap - - val rddStorageInfos = rddBlockMap.flatMap { case (rddId, blocks) => - // Add up memory, disk and Tachyon sizes - val persistedBlocks = - blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } - val _storageLevel = - if (persistedBlocks.length > 0) persistedBlocks(0).storageLevel else StorageLevel.NONE - val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) - val tachyonSize = persistedBlocks.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) - rddInfoMap.get(rddId).map { rddInfo => - rddInfo.storageLevel = _storageLevel - rddInfo.numCachedPartitions = persistedBlocks.length - rddInfo.memSize = memSize - rddInfo.diskSize = diskSize - rddInfo.tachyonSize = tachyonSize - rddInfo - } - }.toArray + /** + * Return the number of blocks stored in this block manager in O(RDDs) time. + * Note that this is much faster than `this.blocks.size`, which is O(blocks) time. + */ + def numBlocks: Int = _nonRddBlocks.size + numRddBlocks + + /** + * Return the number of RDD blocks stored in this block manager in O(RDDs) time. + * Note that this is much faster than `this.rddBlocks.size`, which is O(RDD blocks) time. + */ + def numRddBlocks: Int = _rddBlocks.values.map(_.size).sum - scala.util.Sorting.quickSort(rddStorageInfos) - rddStorageInfos + /** + * Return the number of blocks that belong to the given RDD in O(1) time. + * Note that this is much faster than `this.rddBlocksById(rddId).size`, which is + * O(blocks in this RDD) time. + */ + def numRddBlocksById(rddId: Int): Int = _rddBlocks.get(rddId).map(_.size).getOrElse(0) + + /** Return the memory remaining in this block manager. */ + def memRemaining: Long = maxMem - memUsed + + /** Return the memory used by this block manager. */ + def memUsed: Long = + _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum + + /** Return the disk space used by this block manager. */ + def diskUsed: Long = + _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum + + /** Return the off-heap space used by this block manager. */ + def offHeapUsed: Long = + _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).sum + + /** Return the memory used by the given RDD in this block manager in O(1) time. */ + def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) + + /** Return the disk space used by the given RDD in this block manager in O(1) time. */ + def diskUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._2).getOrElse(0L) + + /** Return the off-heap space used by the given RDD in this block manager in O(1) time. */ + def offHeapUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._3).getOrElse(0L) + + /** Return the storage level, if any, used by the given RDD in this block manager. */ + def rddStorageLevel(rddId: Int): Option[StorageLevel] = _rddStorageInfo.get(rddId).map(_._4) + + /** + * Update the relevant storage info, taking into account any existing status for this block. + */ + private def updateStorageInfo(blockId: BlockId, newBlockStatus: BlockStatus): Unit = { + val oldBlockStatus = getBlock(blockId).getOrElse(BlockStatus.empty) + val changeInMem = newBlockStatus.memSize - oldBlockStatus.memSize + val changeInDisk = newBlockStatus.diskSize - oldBlockStatus.diskSize + val changeInTachyon = newBlockStatus.tachyonSize - oldBlockStatus.tachyonSize + val level = newBlockStatus.storageLevel + + // Compute new info from old info + val (oldMem, oldDisk, oldTachyon) = blockId match { + case RDDBlockId(rddId, _) => + _rddStorageInfo.get(rddId) + .map { case (mem, disk, tachyon, _) => (mem, disk, tachyon) } + .getOrElse((0L, 0L, 0L)) + case _ => + _nonRddStorageInfo + } + val newMem = math.max(oldMem + changeInMem, 0L) + val newDisk = math.max(oldDisk + changeInDisk, 0L) + val newTachyon = math.max(oldTachyon + changeInTachyon, 0L) + + // Set the correct info + blockId match { + case RDDBlockId(rddId, _) => + // If this RDD is no longer persisted, remove it + if (newMem + newDisk + newTachyon == 0) { + _rddStorageInfo.remove(rddId) + } else { + _rddStorageInfo(rddId) = (newMem, newDisk, newTachyon, level) + } + case _ => + _nonRddStorageInfo = (newMem, newDisk, newTachyon) + } } - /** Returns a mapping from BlockId to the locations of the associated block. */ - def blockLocationsFromStorageStatus( - storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { - val blockLocationPairs = storageStatuses.flatMap { storageStatus => - storageStatus.blocks.map { case (bid, _) => (bid, storageStatus.blockManagerId.hostPort) } +} + +/** Helper methods for storage-related objects. */ +private[spark] object StorageUtils { + + /** + * Update the given list of RDDInfo with the given list of storage statuses. + * This method overwrites the old values stored in the RDDInfo's. + */ + def updateRddInfo(rddInfos: Seq[RDDInfo], statuses: Seq[StorageStatus]): Unit = { + rddInfos.foreach { rddInfo => + val rddId = rddInfo.id + // Assume all blocks belonging to the same RDD have the same storage level + val storageLevel = statuses + .map(_.rddStorageLevel(rddId)).flatMap(s => s).headOption.getOrElse(StorageLevel.NONE) + val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).sum + val memSize = statuses.map(_.memUsedByRdd(rddId)).sum + val diskSize = statuses.map(_.diskUsedByRdd(rddId)).sum + val tachyonSize = statuses.map(_.offHeapUsedByRdd(rddId)).sum + + rddInfo.storageLevel = storageLevel + rddInfo.numCachedPartitions = numCachedPartitions + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo.tachyonSize = tachyonSize } - blockLocationPairs.toMap - .groupBy { case (blockId, _) => blockId } - .mapValues(_.values.toSeq) } - /** Filters the given list of StorageStatus by the given RDD ID. */ - def filterStorageStatusByRDD( - storageStatuses: Seq[StorageStatus], - rddId: Int): Array[StorageStatus] = { - storageStatuses.map { status => - val filteredBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toSeq - val filteredBlockMap = mutable.Map[BlockId, BlockStatus](filteredBlocks: _*) - new StorageStatus(status.blockManagerId, status.maxMem, filteredBlockMap) - }.toArray + /** + * Return a mapping from block ID to its locations for each block that belongs to the given RDD. + */ + def getRddBlockLocations(rddId: Int, statuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { + val blockLocations = new mutable.HashMap[BlockId, mutable.ListBuffer[String]] + statuses.foreach { status => + status.rddBlocksById(rddId).foreach { case (bid, _) => + val location = status.blockManagerId.hostPort + blockLocations.getOrElseUpdate(bid, mutable.ListBuffer.empty) += location + } + } + blockLocations } + } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index b358c855e1c88..b814b0e6b8509 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -49,9 +49,9 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList - val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) - val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) + val maxMem = storageStatusList.map(_.maxMem).sum + val memUsed = storageStatusList.map(_.memUsed).sum + val diskUsed = storageStatusList.map(_.diskUsed).sum val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.id) @@ -80,7 +80,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { - {execInfoSorted.map(execRow(_))} + {execInfoSorted.map(execRow)} @@ -91,7 +91,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") {
  • Memory: {Utils.bytesToString(memUsed)} Used ({Utils.bytesToString(maxMem)} Total)
  • -
  • Disk: {Utils.bytesToString(diskSpaceUsed)} Used
  • +
  • Disk: {Utils.bytesToString(diskUsed)} Used
  • @@ -145,7 +145,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size + val rddBlocks = status.numBlocks val memUsed = status.memUsed val maxMem = status.maxMem val diskUsed = status.diskUsed diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 2155633b8096f..84ac53da47552 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -45,12 +45,13 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) // Block table - val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, rddId) - val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).sortWith(_._1.name < _._1.name) - val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) - val blocks = blockStatuses.map { case (blockId, status) => - (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) - } + val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) + val blocks = storageStatusList + .flatMap(_.rddBlocksById(rddId)) + .sortWith(_._1.name < _._1.name) + .map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) + } val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) val content = @@ -119,10 +120,10 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.bytesToString(status.memUsedByRDD(rddId))} + {Utils.bytesToString(status.memUsedByRdd(rddId))} ({Utils.bytesToString(status.memRemaining)} Remaining) - {Utils.bytesToString(status.diskUsedByRDD(rddId))} + {Utils.bytesToString(status.diskUsedByRdd(rddId))} } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 0cc0cf3117173..5f6740d495521 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -41,19 +41,18 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" */ @DeveloperApi class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { - private val _rddInfoMap = mutable.Map[Int, RDDInfo]() + private[ui] val _rddInfoMap = mutable.Map[Int, RDDInfo]() // exposed for testing def storageStatusList = storageStatusListener.storageStatusList /** Filter RDD info to include only those with cached partitions */ def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq - /** Update each RDD's info to reflect any updates to the RDD's storage status */ - private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) { - val rddInfos = _rddInfoMap.values.toSeq - val updatedRddInfos = - StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos, updatedBlocks) - updatedRddInfos.foreach { info => _rddInfoMap(info.id) = info } + /** Update the storage info of the RDDs whose blocks are among the given updated blocks */ + private def updateRDDInfo(updatedBlocks: Seq[(BlockId, BlockStatus)]): Unit = { + val rddIdsToUpdate = updatedBlocks.flatMap { case (bid, _) => bid.asRDDId.map(_.rddId) }.toSet + val rddInfosToUpdate = _rddInfoMap.values.toSeq.filter { s => rddIdsToUpdate.contains(s.id) } + StorageUtils.updateRddInfo(rddInfosToUpdate, storageStatusList) } /** diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index fb18c3ebfe46f..e6ab538d77bcc 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.scalatest.{Assertions, FunSuite} +import org.apache.spark.storage.StorageLevel class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs only returns RDDs that are marked as cached") { @@ -35,26 +36,33 @@ class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs returns an immutable map") { sc = new SparkContext("local", "test") val rdd1 = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - val myRdds = sc.getPersistentRDDs assert(myRdds.size === 1) - assert(myRdds.values.head === rdd1) + assert(myRdds(0) === rdd1) + assert(myRdds(0).getStorageLevel === StorageLevel.MEMORY_ONLY) + // myRdds2 should have 2 RDDs, but myRdds should not change val rdd2 = sc.makeRDD(Array(5, 6, 7, 8), 1).cache() - - // getPersistentRDDs should have 2 RDDs, but myRdds should not change - assert(sc.getPersistentRDDs.size === 2) + val myRdds2 = sc.getPersistentRDDs + assert(myRdds2.size === 2) + assert(myRdds2(0) === rdd1) + assert(myRdds2(1) === rdd2) + assert(myRdds2(0).getStorageLevel === StorageLevel.MEMORY_ONLY) + assert(myRdds2(1).getStorageLevel === StorageLevel.MEMORY_ONLY) assert(myRdds.size === 1) + assert(myRdds(0) === rdd1) + assert(myRdds(0).getStorageLevel === StorageLevel.MEMORY_ONLY) } test("getRDDStorageInfo only reports on RDDs that actually persist data") { sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - assert(sc.getRDDStorageInfo.size === 0) - rdd.collect() assert(sc.getRDDStorageInfo.size === 1) + assert(sc.getRDDStorageInfo.head.isCached) + assert(sc.getRDDStorageInfo.head.memSize > 0) + assert(sc.getRDDStorageInfo.head.storageLevel === StorageLevel.MEMORY_ONLY) } test("call sites report correct locations") { diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 2179c6dd3302e..51fb646a3cb61 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -41,13 +41,13 @@ class StorageStatusListenerSuite extends FunSuite { assert(listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus("big").blockManagerId === bm1) assert(listener.executorIdToStorageStatus("big").maxMem === 1000L) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) assert(listener.executorIdToStorageStatus.size === 2) assert(listener.executorIdToStorageStatus.get("fat").isDefined) assert(listener.executorIdToStorageStatus("fat").blockManagerId === bm2) assert(listener.executorIdToStorageStatus("fat").maxMem === 2000L) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) // Block manager remove listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm1)) @@ -67,14 +67,14 @@ class StorageStatusListenerSuite extends FunSuite { val taskMetrics = new TaskMetrics // Task end with no updated blocks - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } test("task end with updated blocks") { @@ -90,20 +90,20 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics2.updatedBlocks = Some(Seq(block3)) // Task end with new blocks - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) // Task end with dropped blocks val droppedBlock1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) @@ -112,17 +112,17 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 1) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 1) - assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.contains(RDDBlockId(4, 0))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 1) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) + assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 1) - assert(listener.executorIdToStorageStatus("fat").blocks.size === 0) - assert(!listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) - assert(listener.executorIdToStorageStatus("fat").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 1) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) + assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } test("unpersist RDD") { @@ -137,16 +137,16 @@ class StorageStatusListenerSuite extends FunSuite { taskMetrics2.updatedBlocks = Some(Seq(block3)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics2)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + assert(listener.executorIdToStorageStatus("big").numBlocks === 3) // Unpersist RDD listener.onUnpersistRDD(SparkListenerUnpersistRDD(9090)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 3) + assert(listener.executorIdToStorageStatus("big").numBlocks === 3) listener.onUnpersistRDD(SparkListenerUnpersistRDD(4)) - assert(listener.executorIdToStorageStatus("big").blocks.size === 2) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 1))) - assert(listener.executorIdToStorageStatus("big").blocks.contains(RDDBlockId(1, 2))) + assert(listener.executorIdToStorageStatus("big").numBlocks === 2) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) + assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) listener.onUnpersistRDD(SparkListenerUnpersistRDD(1)) - assert(listener.executorIdToStorageStatus("big").blocks.isEmpty) + assert(listener.executorIdToStorageStatus("big").numBlocks === 0) } } diff --git a/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala new file mode 100644 index 0000000000000..38678bbd1dd28 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/StorageSuite.scala @@ -0,0 +1,354 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.scalatest.FunSuite + +/** + * Test various functionalities in StorageUtils and StorageStatus. + */ +class StorageSuite extends FunSuite { + private val memAndDisk = StorageLevel.MEMORY_AND_DISK + + // For testing add, update, and remove (for non-RDD blocks) + private def storageStatus1: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + assert(status.blocks.isEmpty) + assert(status.rddBlocks.isEmpty) + assert(status.memUsed === 0L) + assert(status.memRemaining === 1000L) + assert(status.diskUsed === 0L) + assert(status.offHeapUsed === 0L) + status.addBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(TestBlockId("faa"), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status + } + + test("storage status add non-RDD blocks") { + val status = storageStatus1 + assert(status.blocks.size === 3) + assert(status.blocks.contains(TestBlockId("foo"))) + assert(status.blocks.contains(TestBlockId("fee"))) + assert(status.blocks.contains(TestBlockId("faa"))) + assert(status.rddBlocks.isEmpty) + assert(status.memUsed === 30L) + assert(status.memRemaining === 970L) + assert(status.diskUsed === 60L) + assert(status.offHeapUsed === 3L) + } + + test("storage status update non-RDD blocks") { + val status = storageStatus1 + status.updateBlock(TestBlockId("foo"), BlockStatus(memAndDisk, 50L, 100L, 1L)) + status.updateBlock(TestBlockId("fee"), BlockStatus(memAndDisk, 100L, 20L, 0L)) + assert(status.blocks.size === 3) + assert(status.memUsed === 160L) + assert(status.memRemaining === 840L) + assert(status.diskUsed === 140L) + assert(status.offHeapUsed === 2L) + } + + test("storage status remove non-RDD blocks") { + val status = storageStatus1 + status.removeBlock(TestBlockId("foo")) + status.removeBlock(TestBlockId("faa")) + assert(status.blocks.size === 1) + assert(status.blocks.contains(TestBlockId("fee"))) + assert(status.memUsed === 10L) + assert(status.memRemaining === 990L) + assert(status.diskUsed === 20L) + assert(status.offHeapUsed === 1L) + } + + // For testing add, update, remove, get, and contains etc. for both RDD and non-RDD blocks + private def storageStatus2: StorageStatus = { + val status = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + assert(status.rddBlocks.isEmpty) + status.addBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(TestBlockId("man"), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 100L, 200L, 1L)) + status.addBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 10L, 20L, 1L)) + status.addBlock(RDDBlockId(2, 3), BlockStatus(memAndDisk, 10L, 20L, 0L)) + status.addBlock(RDDBlockId(2, 4), BlockStatus(memAndDisk, 10L, 40L, 0L)) + status + } + + test("storage status add RDD blocks") { + val status = storageStatus2 + assert(status.blocks.size === 7) + assert(status.rddBlocks.size === 5) + assert(status.rddBlocks.contains(RDDBlockId(0, 0))) + assert(status.rddBlocks.contains(RDDBlockId(1, 1))) + assert(status.rddBlocks.contains(RDDBlockId(2, 2))) + assert(status.rddBlocks.contains(RDDBlockId(2, 3))) + assert(status.rddBlocks.contains(RDDBlockId(2, 4))) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(0).contains(RDDBlockId(0, 0))) + assert(status.rddBlocksById(1).size === 1) + assert(status.rddBlocksById(1).contains(RDDBlockId(1, 1))) + assert(status.rddBlocksById(2).size === 3) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 2))) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 4))) + assert(status.memUsedByRdd(0) === 10L) + assert(status.memUsedByRdd(1) === 100L) + assert(status.memUsedByRdd(2) === 30L) + assert(status.diskUsedByRdd(0) === 20L) + assert(status.diskUsedByRdd(1) === 200L) + assert(status.diskUsedByRdd(2) === 80L) + assert(status.offHeapUsedByRdd(0) === 1L) + assert(status.offHeapUsedByRdd(1) === 1L) + assert(status.offHeapUsedByRdd(2) === 1L) + assert(status.rddStorageLevel(0) === Some(memAndDisk)) + assert(status.rddStorageLevel(1) === Some(memAndDisk)) + assert(status.rddStorageLevel(2) === Some(memAndDisk)) + + // Verify default values for RDDs that don't exist + assert(status.rddBlocksById(10).isEmpty) + assert(status.memUsedByRdd(10) === 0L) + assert(status.diskUsedByRdd(10) === 0L) + assert(status.offHeapUsedByRdd(10) === 0L) + assert(status.rddStorageLevel(10) === None) + } + + test("storage status update RDD blocks") { + val status = storageStatus2 + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 5000L, 0L, 0L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 0L, 0L, 0L)) + status.updateBlock(RDDBlockId(2, 2), BlockStatus(memAndDisk, 0L, 1000L, 0L)) + assert(status.blocks.size === 7) + assert(status.rddBlocks.size === 5) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(1).size === 1) + assert(status.rddBlocksById(2).size === 3) + assert(status.memUsedByRdd(0) === 0L) + assert(status.memUsedByRdd(1) === 100L) + assert(status.memUsedByRdd(2) === 20L) + assert(status.diskUsedByRdd(0) === 0L) + assert(status.diskUsedByRdd(1) === 200L) + assert(status.diskUsedByRdd(2) === 1060L) + assert(status.offHeapUsedByRdd(0) === 0L) + assert(status.offHeapUsedByRdd(1) === 1L) + assert(status.offHeapUsedByRdd(2) === 0L) + } + + test("storage status remove RDD blocks") { + val status = storageStatus2 + status.removeBlock(TestBlockId("man")) + status.removeBlock(RDDBlockId(1, 1)) + status.removeBlock(RDDBlockId(2, 2)) + status.removeBlock(RDDBlockId(2, 4)) + assert(status.blocks.size === 3) + assert(status.rddBlocks.size === 2) + assert(status.rddBlocks.contains(RDDBlockId(0, 0))) + assert(status.rddBlocks.contains(RDDBlockId(2, 3))) + assert(status.rddBlocksById(0).size === 1) + assert(status.rddBlocksById(0).contains(RDDBlockId(0, 0))) + assert(status.rddBlocksById(1).size === 0) + assert(status.rddBlocksById(2).size === 1) + assert(status.rddBlocksById(2).contains(RDDBlockId(2, 3))) + assert(status.memUsedByRdd(0) === 10L) + assert(status.memUsedByRdd(1) === 0L) + assert(status.memUsedByRdd(2) === 10L) + assert(status.diskUsedByRdd(0) === 20L) + assert(status.diskUsedByRdd(1) === 0L) + assert(status.diskUsedByRdd(2) === 20L) + assert(status.offHeapUsedByRdd(0) === 1L) + assert(status.offHeapUsedByRdd(1) === 0L) + assert(status.offHeapUsedByRdd(2) === 0L) + } + + test("storage status containsBlock") { + val status = storageStatus2 + // blocks that actually exist + assert(status.blocks.contains(TestBlockId("dan")) === status.containsBlock(TestBlockId("dan"))) + assert(status.blocks.contains(TestBlockId("man")) === status.containsBlock(TestBlockId("man"))) + assert(status.blocks.contains(RDDBlockId(0, 0)) === status.containsBlock(RDDBlockId(0, 0))) + assert(status.blocks.contains(RDDBlockId(1, 1)) === status.containsBlock(RDDBlockId(1, 1))) + assert(status.blocks.contains(RDDBlockId(2, 2)) === status.containsBlock(RDDBlockId(2, 2))) + assert(status.blocks.contains(RDDBlockId(2, 3)) === status.containsBlock(RDDBlockId(2, 3))) + assert(status.blocks.contains(RDDBlockId(2, 4)) === status.containsBlock(RDDBlockId(2, 4))) + // blocks that don't exist + assert(status.blocks.contains(TestBlockId("fan")) === status.containsBlock(TestBlockId("fan"))) + assert(status.blocks.contains(RDDBlockId(100, 0)) === status.containsBlock(RDDBlockId(100, 0))) + } + + test("storage status getBlock") { + val status = storageStatus2 + // blocks that actually exist + assert(status.blocks.get(TestBlockId("dan")) === status.getBlock(TestBlockId("dan"))) + assert(status.blocks.get(TestBlockId("man")) === status.getBlock(TestBlockId("man"))) + assert(status.blocks.get(RDDBlockId(0, 0)) === status.getBlock(RDDBlockId(0, 0))) + assert(status.blocks.get(RDDBlockId(1, 1)) === status.getBlock(RDDBlockId(1, 1))) + assert(status.blocks.get(RDDBlockId(2, 2)) === status.getBlock(RDDBlockId(2, 2))) + assert(status.blocks.get(RDDBlockId(2, 3)) === status.getBlock(RDDBlockId(2, 3))) + assert(status.blocks.get(RDDBlockId(2, 4)) === status.getBlock(RDDBlockId(2, 4))) + // blocks that don't exist + assert(status.blocks.get(TestBlockId("fan")) === status.getBlock(TestBlockId("fan"))) + assert(status.blocks.get(RDDBlockId(100, 0)) === status.getBlock(RDDBlockId(100, 0))) + } + + test("storage status num[Rdd]Blocks") { + val status = storageStatus2 + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + status.addBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.addBlock(RDDBlockId(4, 4), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.addBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + status.updateBlock(TestBlockId("Foo"), BlockStatus(memAndDisk, 0L, 10L, 400L)) + status.updateBlock(RDDBlockId(4, 0), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.updateBlock(RDDBlockId(4, 8), BlockStatus(memAndDisk, 0L, 0L, 100L)) + status.updateBlock(RDDBlockId(10, 10), BlockStatus(memAndDisk, 0L, 0L, 100L)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + assert(status.rddBlocksById(100).size === status.numRddBlocksById(100)) + status.removeBlock(RDDBlockId(4, 0)) + status.removeBlock(RDDBlockId(10, 10)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + // remove a block that doesn't exist + status.removeBlock(RDDBlockId(1000, 999)) + assert(status.blocks.size === status.numBlocks) + assert(status.rddBlocks.size === status.numRddBlocks) + assert(status.rddBlocksById(4).size === status.numRddBlocksById(4)) + assert(status.rddBlocksById(10).size === status.numRddBlocksById(10)) + assert(status.rddBlocksById(1000).size === status.numRddBlocksById(1000)) + } + + test("storage status memUsed, diskUsed, tachyonUsed") { + val status = storageStatus2 + def actualMemUsed: Long = status.blocks.values.map(_.memSize).sum + def actualDiskUsed: Long = status.blocks.values.map(_.diskSize).sum + def actualOffHeapUsed: Long = status.blocks.values.map(_.tachyonSize).sum + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.addBlock(TestBlockId("fire"), BlockStatus(memAndDisk, 4000L, 5000L, 6000L)) + status.addBlock(TestBlockId("wire"), BlockStatus(memAndDisk, 400L, 500L, 600L)) + status.addBlock(RDDBlockId(25, 25), BlockStatus(memAndDisk, 40L, 50L, 60L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.updateBlock(TestBlockId("dan"), BlockStatus(memAndDisk, 4L, 5L, 6L)) + status.updateBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 4L, 5L, 6L)) + status.updateBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 4L, 5L, 6L)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + status.removeBlock(TestBlockId("fire")) + status.removeBlock(TestBlockId("man")) + status.removeBlock(RDDBlockId(2, 2)) + status.removeBlock(RDDBlockId(2, 3)) + assert(status.memUsed === actualMemUsed) + assert(status.diskUsed === actualDiskUsed) + assert(status.offHeapUsed === actualOffHeapUsed) + } + + // For testing StorageUtils.updateRddInfo and StorageUtils.getRddBlockLocations + private def stockStorageStatuses: Seq[StorageStatus] = { + val status1 = new StorageStatus(BlockManagerId("big", "dog", 1, 1), 1000L) + val status2 = new StorageStatus(BlockManagerId("fat", "duck", 2, 2), 2000L) + val status3 = new StorageStatus(BlockManagerId("fat", "cat", 3, 3), 3000L) + status1.addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status1.addBlock(RDDBlockId(0, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(0, 3), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status2.addBlock(RDDBlockId(1, 1), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) + status3.addBlock(RDDBlockId(1, 2), BlockStatus(memAndDisk, 1L, 2L, 0L)) + Seq(status1, status2, status3) + } + + // For testing StorageUtils.updateRddInfo + private def stockRDDInfos: Seq[RDDInfo] = { + val info0 = new RDDInfo(0, "0", 10, memAndDisk) + val info1 = new RDDInfo(1, "1", 3, memAndDisk) + Seq(info0, info1) + } + + test("StorageUtils.updateRddInfo") { + val storageStatuses = stockStorageStatuses + val rddInfos = stockRDDInfos + StorageUtils.updateRddInfo(rddInfos, storageStatuses) + assert(rddInfos(0).storageLevel === memAndDisk) + assert(rddInfos(0).numCachedPartitions === 5) + assert(rddInfos(0).memSize === 5L) + assert(rddInfos(0).diskSize === 10L) + assert(rddInfos(0).tachyonSize === 0L) + assert(rddInfos(1).storageLevel === memAndDisk) + assert(rddInfos(1).numCachedPartitions === 3) + assert(rddInfos(1).memSize === 3L) + assert(rddInfos(1).diskSize === 6L) + assert(rddInfos(1).tachyonSize === 0L) + } + + test("StorageUtils.getRddBlockLocations") { + val storageStatuses = stockStorageStatuses + val blockLocations0 = StorageUtils.getRddBlockLocations(0, storageStatuses) + val blockLocations1 = StorageUtils.getRddBlockLocations(1, storageStatuses) + assert(blockLocations0.size === 5) + assert(blockLocations1.size === 3) + assert(blockLocations0.contains(RDDBlockId(0, 0))) + assert(blockLocations0.contains(RDDBlockId(0, 1))) + assert(blockLocations0.contains(RDDBlockId(0, 2))) + assert(blockLocations0.contains(RDDBlockId(0, 3))) + assert(blockLocations0.contains(RDDBlockId(0, 4))) + assert(blockLocations1.contains(RDDBlockId(1, 0))) + assert(blockLocations1.contains(RDDBlockId(1, 1))) + assert(blockLocations1.contains(RDDBlockId(1, 2))) + assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 4)) === Seq("cat:3")) + assert(blockLocations1(RDDBlockId(1, 0)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) + } + + test("StorageUtils.getRddBlockLocations with multiple locations") { + val storageStatuses = stockStorageStatuses + storageStatuses(0).addBlock(RDDBlockId(1, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + storageStatuses(0).addBlock(RDDBlockId(0, 4), BlockStatus(memAndDisk, 1L, 2L, 0L)) + storageStatuses(2).addBlock(RDDBlockId(0, 0), BlockStatus(memAndDisk, 1L, 2L, 0L)) + val blockLocations0 = StorageUtils.getRddBlockLocations(0, storageStatuses) + val blockLocations1 = StorageUtils.getRddBlockLocations(1, storageStatuses) + assert(blockLocations0.size === 5) + assert(blockLocations1.size === 3) + assert(blockLocations0(RDDBlockId(0, 0)) === Seq("dog:1", "cat:3")) + assert(blockLocations0(RDDBlockId(0, 1)) === Seq("dog:1")) + assert(blockLocations0(RDDBlockId(0, 2)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 3)) === Seq("duck:2")) + assert(blockLocations0(RDDBlockId(0, 4)) === Seq("dog:1", "cat:3")) + assert(blockLocations1(RDDBlockId(1, 0)) === Seq("dog:1", "duck:2")) + assert(blockLocations1(RDDBlockId(1, 1)) === Seq("duck:2")) + assert(blockLocations1(RDDBlockId(1, 2)) === Seq("cat:3")) + } + +} diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala new file mode 100644 index 0000000000000..6e68dcb3425aa --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -0,0 +1,165 @@ +/* + * 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.ui.storage + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.Success +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ + +/** + * Test various functionality in the StorageListener that supports the StorageTab. + */ +class StorageTabSuite extends FunSuite with BeforeAndAfter { + private var bus: LiveListenerBus = _ + private var storageStatusListener: StorageStatusListener = _ + private var storageListener: StorageListener = _ + private val memAndDisk = StorageLevel.MEMORY_AND_DISK + private val memOnly = StorageLevel.MEMORY_ONLY + private val none = StorageLevel.NONE + private val taskInfo = new TaskInfo(0, 0, 0, 0, "big", "dog", TaskLocality.ANY, false) + private def rddInfo0 = new RDDInfo(0, "freedom", 100, memOnly) + private def rddInfo1 = new RDDInfo(1, "hostage", 200, memOnly) + private def rddInfo2 = new RDDInfo(2, "sanity", 300, memAndDisk) + private def rddInfo3 = new RDDInfo(3, "grace", 400, memAndDisk) + private val bm1 = BlockManagerId("big", "dog", 1, 1) + + before { + bus = new LiveListenerBus + storageStatusListener = new StorageStatusListener + storageListener = new StorageListener(storageStatusListener) + bus.addListener(storageStatusListener) + bus.addListener(storageListener) + } + + test("stage submitted / completed") { + assert(storageListener._rddInfoMap.isEmpty) + assert(storageListener.rddInfoList.isEmpty) + + // 2 RDDs are known, but none are cached + val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0, rddInfo1), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.isEmpty) + + // 4 RDDs are known, but only 2 are cached + val rddInfo2Cached = rddInfo2 + val rddInfo3Cached = rddInfo3 + rddInfo2Cached.numCachedPartitions = 1 + rddInfo3Cached.numCachedPartitions = 1 + val stageInfo1 = new StageInfo(1, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) + assert(storageListener._rddInfoMap.size === 4) + assert(storageListener.rddInfoList.size === 2) + + // Submitting RDDInfos with duplicate IDs does nothing + val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY) + rddInfo0Cached.numCachedPartitions = 1 + val stageInfo0Cached = new StageInfo(0, "0", 100, Seq(rddInfo0), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) + assert(storageListener._rddInfoMap.size === 4) + assert(storageListener.rddInfoList.size === 2) + + // We only keep around the RDDs that are cached + bus.postToAll(SparkListenerStageCompleted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.size === 2) + } + + test("unpersist") { + val rddInfo0Cached = rddInfo0 + val rddInfo1Cached = rddInfo1 + rddInfo0Cached.numCachedPartitions = 1 + rddInfo1Cached.numCachedPartitions = 1 + val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 2) + assert(storageListener.rddInfoList.size === 2) + bus.postToAll(SparkListenerUnpersistRDD(0)) + assert(storageListener._rddInfoMap.size === 1) + assert(storageListener.rddInfoList.size === 1) + bus.postToAll(SparkListenerUnpersistRDD(4)) // doesn't exist + assert(storageListener._rddInfoMap.size === 1) + assert(storageListener.rddInfoList.size === 1) + bus.postToAll(SparkListenerUnpersistRDD(1)) + assert(storageListener._rddInfoMap.size === 0) + assert(storageListener.rddInfoList.size === 0) + } + + test("task end") { + val myRddInfo0 = rddInfo0 + val myRddInfo1 = rddInfo1 + val myRddInfo2 = rddInfo2 + val stageInfo0 = new StageInfo(0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) + bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) + assert(storageListener._rddInfoMap.size === 3) + assert(storageListener.rddInfoList.size === 0) // not cached + assert(!storageListener._rddInfoMap(0).isCached) + assert(!storageListener._rddInfoMap(1).isCached) + assert(!storageListener._rddInfoMap(2).isCached) + + // Task end with no updated blocks. This should not change anything. + bus.postToAll(SparkListenerTaskEnd(0, "obliteration", Success, taskInfo, new TaskMetrics)) + assert(storageListener._rddInfoMap.size === 3) + assert(storageListener.rddInfoList.size === 0) + + // Task end with a few new persisted blocks, some from the same RDD + val metrics1 = new TaskMetrics + metrics1.updatedBlocks = Some(Seq( + (RDDBlockId(0, 100), BlockStatus(memAndDisk, 400L, 0L, 0L)), + (RDDBlockId(0, 101), BlockStatus(memAndDisk, 0L, 400L, 0L)), + (RDDBlockId(0, 102), BlockStatus(memAndDisk, 400L, 0L, 200L)), + (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L, 0L)) + )) + bus.postToAll(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo, metrics1)) + assert(storageListener._rddInfoMap(0).memSize === 800L) + assert(storageListener._rddInfoMap(0).diskSize === 400L) + assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).numCachedPartitions === 3) + assert(storageListener._rddInfoMap(0).isCached) + assert(storageListener._rddInfoMap(1).memSize === 0L) + assert(storageListener._rddInfoMap(1).diskSize === 240L) + assert(storageListener._rddInfoMap(1).tachyonSize === 0L) + assert(storageListener._rddInfoMap(1).numCachedPartitions === 1) + assert(storageListener._rddInfoMap(1).isCached) + assert(!storageListener._rddInfoMap(2).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + + // Task end with a few dropped blocks + val metrics2 = new TaskMetrics + metrics2.updatedBlocks = Some(Seq( + (RDDBlockId(0, 100), BlockStatus(none, 0L, 0L, 0L)), + (RDDBlockId(1, 20), BlockStatus(none, 0L, 0L, 0L)), + (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L, 0L)), // doesn't actually exist + (RDDBlockId(4, 80), BlockStatus(none, 0L, 0L, 0L)) // doesn't actually exist + )) + bus.postToAll(SparkListenerTaskEnd(2, "obliteration", Success, taskInfo, metrics2)) + assert(storageListener._rddInfoMap(0).memSize === 400L) + assert(storageListener._rddInfoMap(0).diskSize === 400L) + assert(storageListener._rddInfoMap(0).tachyonSize === 200L) + assert(storageListener._rddInfoMap(0).numCachedPartitions === 2) + assert(storageListener._rddInfoMap(0).isCached) + assert(!storageListener._rddInfoMap(1).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + assert(!storageListener._rddInfoMap(2).isCached) + assert(storageListener._rddInfoMap(2).numCachedPartitions === 0) + } + +} From 148af6082cdb44840bbd61c7a4f67a95badad10b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 2 Aug 2014 00:45:38 -0700 Subject: [PATCH 0209/1492] [SPARK-2454] Do not ship spark home to Workers When standalone Workers launch executors, they inherit the Spark home set by the driver. This means if the worker machines do not share the same directory structure as the driver node, the Workers will attempt to run scripts (e.g. bin/compute-classpath.sh) that do not exist locally and fail. This is a common scenario if the driver is launched from outside of the cluster. The solution is to simply not pass the driver's Spark home to the Workers. This PR further makes an attempt to avoid overloading the usages of `spark.home`, which is now only used for setting executor Spark home on Mesos and in python. This is based on top of #1392 and originally reported by YanTangZhai. Tested on standalone cluster. Author: Andrew Or Closes #1734 from andrewor14/spark-home-reprise and squashes the following commits: f71f391 [Andrew Or] Revert changes in python 1c2532c [Andrew Or] Merge branch 'master' of github.com:apache/spark into spark-home-reprise 188fc5d [Andrew Or] Avoid using spark.home where possible 09272b7 [Andrew Or] Always use Worker's working directory as spark home --- .../org/apache/spark/deploy/ApplicationDescription.scala | 1 - .../main/scala/org/apache/spark/deploy/JsonProtocol.scala | 1 - .../scala/org/apache/spark/deploy/client/TestClient.scala | 5 ++--- .../main/scala/org/apache/spark/deploy/worker/Worker.scala | 7 +++---- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 3 +-- core/src/test/scala/org/apache/spark/DriverSuite.scala | 2 +- .../scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 5 ++--- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- .../apache/spark/deploy/worker/ExecutorRunnerTest.scala | 7 +++---- project/SparkBuild.scala | 2 +- python/pyspark/context.py | 2 +- repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala | 3 --- .../main/scala/org/apache/spark/streaming/Checkpoint.scala | 1 - 13 files changed, 15 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 86305d2ea8a09..65a1a8fd7e929 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -22,7 +22,6 @@ private[spark] class ApplicationDescription( val maxCores: Option[Int], val memoryPerSlave: Int, val command: Command, - val sparkHome: Option[String], var appUiUrl: String, val eventLogDir: Option[String] = None) extends Serializable { diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index c4f5e294a393e..696f32a6f5730 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -56,7 +56,6 @@ private[spark] object JsonProtocol { ("cores" -> obj.maxCores) ~ ("memoryperslave" -> obj.memoryPerSlave) ~ ("user" -> obj.user) ~ - ("sparkhome" -> obj.sparkHome) ~ ("command" -> obj.command.toString) } diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index b8ffa9afb69cb..88a0862b96afe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -48,9 +48,8 @@ private[spark] object TestClient { val conf = new SparkConf val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) - val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), - Seq(), Seq(), Seq()), Some("dummy-spark-home"), "ignored") + val desc = new ApplicationDescription("TestClient", Some(1), 512, + Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), Seq(), Seq()), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index fb5252da96519..c6ea42fceb659 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -81,7 +81,8 @@ private[spark] class Worker( @volatile var registered = false @volatile var connected = false val workerId = generateWorkerId() - val sparkHome = new File(Option(System.getenv("SPARK_HOME")).getOrElse(".")) + val sparkHome = + new File(sys.props.get("spark.test.home").orElse(sys.env.get("SPARK_HOME")).getOrElse(".")) var workDir: File = null val executors = new HashMap[String, ExecutorRunner] val finishedExecutors = new HashMap[String, ExecutorRunner] @@ -233,9 +234,7 @@ private[spark] class Worker( try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, - appDesc.sparkHome.map(userSparkHome => new File(userSparkHome)).getOrElse(sparkHome), - workDir, akkaUrl, conf, ExecutorState.RUNNING) + self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.RUNNING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ 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 48aaaa54bdb35..a28446f6c8a6b 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 @@ -60,9 +60,8 @@ private[spark] class SparkDeploySchedulerBackend( val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) - val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) + sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index de4bd90c8f7e5..e36902ec81e08 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -34,7 +34,7 @@ import scala.language.postfixOps class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + val sparkHome = sys.props("spark.test.home") // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 093394ad6d142..31aa7ec837f43 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -89,7 +89,7 @@ class JsonProtocolSuite extends FunSuite { def createAppDesc(): ApplicationDescription = { val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq(), Seq()) - new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") + new ApplicationDescription("name", Some(4), 1234, cmd, "appUiUrl") } def createAppInfo() : ApplicationInfo = { @@ -169,8 +169,7 @@ object JsonConstants { val appDescJsonStr = """ |{"name":"name","cores":4,"memoryperslave":1234, - |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),List())"} + |"user":"%s","command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),List())"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 9190b05e2dba2..8126ef1bb23aa 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -295,7 +295,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. def runSparkSubmit(args: Seq[String]): String = { - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + val sparkHome = sys.props("spark.test.home") Utils.executeAndGetOutput( Seq("./bin/spark-submit") ++ args, new File(sparkHome), diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index ca4d987619c91..149a2b3d95b86 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,12 +27,11 @@ import org.apache.spark.SparkConf class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) + val sparkHome = sys.props("spark.test.home") val appDesc = new ApplicationDescription("app name", Some(8), 500, - Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), - sparkHome, "appUiUrl") + Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), "appUiUrl") val appId = "12345-worker321-9876" - val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), + val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), f("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) assert(er.getCommandSeq.last === appId) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a8bbd55861954..1d7cc6dd6aef3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -328,7 +328,7 @@ object TestSettings { lazy val settings = Seq ( // Fork new JVMs for tests and set Java options for those fork := true, - javaOptions in Test += "-Dspark.home=" + sparkHome, + javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 7b0f8d83aedc5..2e80eb50f2207 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -84,7 +84,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, @param serializer: The serializer for RDDs. @param conf: A L{SparkConf} object setting Spark properties. @param gateway: Use an existing gateway and JVM, otherwise a new JVM - will be instatiated. + will be instantiated. >>> from pyspark.context import SparkContext diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 42c7e511dc3f5..65788f4646d91 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -969,9 +969,6 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, if (execUri != null) { conf.set("spark.executor.uri", execUri) } - if (System.getenv("SPARK_HOME") != null) { - conf.setSparkHome(System.getenv("SPARK_HOME")) - } sparkContext = new SparkContext(conf) logInfo("Created spark context..") sparkContext diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index ac56ff709c1c4..b780282bdac37 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -35,7 +35,6 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) extends Logging with Serializable { val master = ssc.sc.master val framework = ssc.sc.appName - val sparkHome = ssc.sc.getSparkHome.getOrElse(null) val jars = ssc.sc.jars val graph = ssc.graph val checkpointDir = ssc.checkpointDir From 08c095b6647033285e8f6703922bdacecce3fc71 Mon Sep 17 00:00:00 2001 From: Anand Avati Date: Sat, 2 Aug 2014 00:48:17 -0700 Subject: [PATCH 0210/1492] [SPARK-1812] sql/catalyst - Provide explicit type information For Scala 2.11 compatibility. Without the explicit type specification, withNullability return type is inferred to be Attribute, and thus calling at() on the returned object fails in these tests: [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:370: value at is not a [ERROR] val c4_notNull = 'a.boolean.notNull.at(3) [ERROR] ^ [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:371: value at is not a [ERROR] val c5_notNull = 'a.boolean.notNull.at(4) [ERROR] ^ [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:372: value at is not a [ERROR] val c6_notNull = 'a.boolean.notNull.at(5) [ERROR] ^ [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:558: value at is not a [ERROR] val s_notNull = 'a.string.notNull.at(0) Signed-off-by: Anand Avati Author: Anand Avati Closes #1709 from avati/SPARK-1812-notnull and squashes the following commits: 0470eb3 [Anand Avati] SPARK-1812: sql/catalyst - Provide explicit type information --- .../spark/sql/catalyst/expressions/namedExpressions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index ed69928ae9eb8..02d04762629f5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -134,7 +134,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea /** * Returns a copy of this [[AttributeReference]] with changed nullability. */ - override def withNullability(newNullability: Boolean) = { + override def withNullability(newNullability: Boolean): AttributeReference = { if (nullable == newNullability) { this } else { From 25cad6adf6479fb00265df06d5f77599f8defd26 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 2 Aug 2014 00:57:47 -0700 Subject: [PATCH 0211/1492] HOTFIX: Fixing test error in maven for flume-sink. We needed to add an explicit dependency on scalatest since this module will not get it from spark core like others do. --- external/flume-sink/pom.xml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d11129ce8d89d..d0bf1cf1ea796 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -67,7 +67,10 @@ org.scala-lang scala-library - 2.10.4 + + + org.scalatest + scalatest_${scala.binary.version}
    From 44460ba594fbfe5a6ee66e5121ead914bf16f9f6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 2 Aug 2014 01:11:03 -0700 Subject: [PATCH 0212/1492] HOTFIX: Fix concurrency issue in FlumePollingStreamSuite. This has been failing on master. One possible cause is that the port gets contended if multiple test runs happen concurrently and they hit this test at the same time. Since this test takes a long time (60 seconds) that's very plausible. This patch randomizes the port used in this test to avoid contention. --- .../spark/streaming/flume/FlumePollingStreamSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 47071d0cc4714..27bf2ac962721 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.flume import java.net.InetSocketAddress import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} +import java.util.Random import scala.collection.JavaConversions._ import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} @@ -37,13 +38,16 @@ import org.apache.spark.streaming.flume.sink._ class FlumePollingStreamSuite extends TestSuiteBase { - val testPort = 9999 + val random = new Random() + /** Return a port in the ephemeral range. */ + def getTestPort = random.nextInt(16382) + 49152 val batchCount = 5 val eventsPerBatch = 100 val totalEventsPerChannel = batchCount * eventsPerBatch val channelCapacity = 5000 test("flume polling test") { + val testPort = getTestPort // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = @@ -77,6 +81,7 @@ class FlumePollingStreamSuite extends TestSuiteBase { } test("flume polling test multiple hosts") { + val testPort = getTestPort // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _)) From 87738bfa4051771ddfb8c4a4c1eb142fd77e3a46 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 2 Aug 2014 01:26:16 -0700 Subject: [PATCH 0213/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #706 (close requested by 'pwendell') Closes #453 (close requested by 'pwendell') Closes #557 (close requested by 'tdas') Closes #495 (close requested by 'tdas') Closes #1232 (close requested by 'pwendell') Closes #82 (close requested by 'pwendell') Closes #600 (close requested by 'pwendell') Closes #473 (close requested by 'pwendell') Closes #351 (close requested by 'pwendell') From e09e18b3123c20e9b9497cf606473da500349d4d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 2 Aug 2014 12:11:50 -0700 Subject: [PATCH 0214/1492] [HOTFIX] Do not throw NPE if spark.test.home is not set `spark.test.home` was introduced in #1734. This is fine for SBT but is failing maven tests. Either way it shouldn't throw an NPE. Author: Andrew Or Closes #1739 from andrewor14/fix-spark-test-home and squashes the following commits: ce2624c [Andrew Or] Do not throw NPE if spark.test.home is not set --- .../scala/org/apache/spark/deploy/worker/Worker.scala | 9 +++++++-- core/src/test/scala/org/apache/spark/DriverSuite.scala | 2 +- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- .../apache/spark/deploy/worker/ExecutorRunnerTest.scala | 2 +- pom.xml | 8 ++++---- 5 files changed, 14 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index c6ea42fceb659..458d9947bd873 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -71,7 +71,7 @@ private[spark] class Worker( // TTL for app folders/data; after TTL expires it will be cleaned up val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) - + val testing: Boolean = sys.props.contains("spark.testing") val masterLock: Object = new Object() var master: ActorSelection = null var masterAddress: Address = null @@ -82,7 +82,12 @@ private[spark] class Worker( @volatile var connected = false val workerId = generateWorkerId() val sparkHome = - new File(sys.props.get("spark.test.home").orElse(sys.env.get("SPARK_HOME")).getOrElse(".")) + if (testing) { + assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!") + new File(sys.props("spark.test.home")) + } else { + new File(sys.env.get("SPARK_HOME").getOrElse(".")) + } var workDir: File = null val executors = new HashMap[String, ExecutorRunner] val finishedExecutors = new HashMap[String, ExecutorRunner] diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index e36902ec81e08..a73e1ef0288a5 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -34,7 +34,7 @@ import scala.language.postfixOps class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { - val sparkHome = sys.props("spark.test.home") + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" val masters = Table(("master"), ("local"), ("local-cluster[2,1,512]")) forAll(masters) { (master: String) => diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 8126ef1bb23aa..a5cdcfb5de03b 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -295,7 +295,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. def runSparkSubmit(args: Seq[String]): String = { - val sparkHome = sys.props("spark.test.home") + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) Utils.executeAndGetOutput( Seq("./bin/spark-submit") ++ args, new File(sparkHome), diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 149a2b3d95b86..39ab53cf0b5b1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkConf class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) - val sparkHome = sys.props("spark.test.home") + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), "appUiUrl") val appId = "12345-worker321-9876" diff --git a/pom.xml b/pom.xml index ae97bf03c53a2..99ae4b8b33f94 100644 --- a/pom.xml +++ b/pom.xml @@ -868,10 +868,10 @@ ${project.build.directory}/SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m - - ${session.executionRootDirectory} - 1 - + + ${session.executionRootDirectory} + 1 + From 3f67382e7c9c3f6a8f6ce124ab3fcb1a9c1a264f Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sat, 2 Aug 2014 13:07:17 -0700 Subject: [PATCH 0215/1492] [SPARK-2478] [mllib] DecisionTree Python API Added experimental Python API for Decision Trees. API: * class DecisionTreeModel ** predict() for single examples and RDDs, taking both feature vectors and LabeledPoints ** numNodes() ** depth() ** __str__() * class DecisionTree ** trainClassifier() ** trainRegressor() ** train() Examples and testing: * Added example testing classification and regression with batch prediction: examples/src/main/python/mllib/tree.py * Have also tested example usage in doc of python/pyspark/mllib/tree.py which tests single-example prediction with dense and sparse vectors Also: Small bug fix in python/pyspark/mllib/_common.py: In _linear_predictor_typecheck, changed check for RDD to use isinstance() instead of type() in order to catch RDD subclasses. CC mengxr manishamde Author: Joseph K. Bradley Closes #1727 from jkbradley/decisiontree-python-new and squashes the following commits: 3744488 [Joseph K. Bradley] Renamed test tree.py to decision_tree_runner.py Small updates based on github review. 6b86a9d [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new affceb9 [Joseph K. Bradley] * Fixed bug in doc tests in pyspark/mllib/util.py caused by change in loadLibSVMFile behavior. (It used to threshold labels at 0 to make them 0/1, but it now leaves them as they are.) * Fixed small bug in loadLibSVMFile: If a data file had no features, then loadLibSVMFile would create a single all-zero feature. 67a29bc [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new cf46ad7 [Joseph K. Bradley] Python DecisionTreeModel * predict(empty RDD) returns an empty RDD instead of an error. * Removed support for calling predict() on LabeledPoint and RDD[LabeledPoint] * predict() does not cache serialized RDD any more. aa29873 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new bf21be4 [Joseph K. Bradley] removed old run() func from DecisionTree fa10ea7 [Joseph K. Bradley] Small style update 7968692 [Joseph K. Bradley] small braces typo fix e34c263 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new 4801b40 [Joseph K. Bradley] Small style update to DecisionTreeSuite db0eab2 [Joseph K. Bradley] Merge branch 'decisiontree-bugfix2' into decisiontree-python-new 6873fa9 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new 225822f [Joseph K. Bradley] Bug: In DecisionTree, the method sequentialBinSearchForOrderedCategoricalFeatureInClassification() indexed bins from 0 to (math.pow(2, featureCategories.toInt - 1) - 1). This upper bound is the bound for unordered categorical features, not ordered ones. The upper bound should be the arity (i.e., max value) of the feature. 93953f1 [Joseph K. Bradley] Likely done with Python API. 6df89a9 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new 4562c08 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new 665ba78 [Joseph K. Bradley] Small updates towards Python DecisionTree API 188cb0d [Joseph K. Bradley] Merge branch 'decisiontree-bugfix' into decisiontree-python-new 6622247 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new b8fac57 [Joseph K. Bradley] Finished Python DecisionTree API and example but need to test a bit more. 2b20c61 [Joseph K. Bradley] Small doc and style updates 1b29c13 [Joseph K. Bradley] Merge branch 'decisiontree-bugfix' into decisiontree-python-new 584449a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new dab0b67 [Joseph K. Bradley] Added documentation for DecisionTree internals 8bb8aa0 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix 978cfcf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix 6eed482 [Joseph K. Bradley] In DecisionTree: Changed from using procedural syntax for functions returning Unit to explicitly writing Unit return type. 376dca2 [Joseph K. Bradley] Updated meaning of maxDepth by 1 to fit scikit-learn and rpart. * In code, replaced usages of maxDepth <-- maxDepth + 1 * In params, replace settings of maxDepth <-- maxDepth - 1 e06e423 [Joseph K. Bradley] Merge branch 'decisiontree-bugfix' into decisiontree-python-new bab3f19 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new 59750f8 [Joseph K. Bradley] * Updated Strategy to check numClassesForClassification only if algo=Classification. * Updates based on comments: ** DecisionTreeRunner *** Made dataFormat arg default to libsvm ** Small cleanups ** tree.Node: Made recursive helper methods private, and renamed them. 52e17c5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix f5a036c [Joseph K. Bradley] Merge branch 'decisiontree-bugfix' into decisiontree-python-new da50db7 [Joseph K. Bradley] Added one more test to DecisionTreeSuite: stump with 2 continuous variables for binary classification. Caused problems in past, but fixed now. 8e227ea [Joseph K. Bradley] Changed Strategy so it only requires numClassesForClassification >= 2 for classification cd1d933 [Joseph K. Bradley] Merge branch 'decisiontree-bugfix' into decisiontree-python-new 8ea8750 [Joseph K. Bradley] Bug fix: Off-by-1 when finding thresholds for splits for continuous features. 8a758db [Joseph K. Bradley] Merge branch 'decisiontree-bugfix' into decisiontree-python-new 5fe44ed [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-python-new 2283df8 [Joseph K. Bradley] 2 bug fixes. 73fbea2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into decisiontree-bugfix 5f920a1 [Joseph K. Bradley] Demonstration of bug before submitting fix: Updated DecisionTreeSuite so that 3 tests fail. Will describe bug in next commit. f825352 [Joseph K. Bradley] Wrote Python API and example for DecisionTree. Also added toString, depth, and numNodes methods to DecisionTreeModel. --- .../main/python/mllib/decision_tree_runner.py | 133 +++++++++++ .../main/python/mllib/logistic_regression.py | 4 +- .../mllib/api/python/PythonMLLibAPI.scala | 78 ++++++ .../mllib/tree/configuration/Strategy.scala | 3 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 3 +- python/pyspark/mllib/_common.py | 33 ++- python/pyspark/mllib/tests.py | 36 +++ python/pyspark/mllib/tree.py | 225 ++++++++++++++++++ python/pyspark/mllib/util.py | 14 +- python/run-tests | 1 + 10 files changed, 509 insertions(+), 21 deletions(-) create mode 100755 examples/src/main/python/mllib/decision_tree_runner.py create mode 100644 python/pyspark/mllib/tree.py diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py new file mode 100755 index 0000000000000..8efadb5223f56 --- /dev/null +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -0,0 +1,133 @@ +# +# 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. +# + +""" +Decision tree classification and regression using MLlib. +""" + +import numpy, os, sys + +from operator import add + +from pyspark import SparkContext +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + + +def getAccuracy(dtModel, data): + """ + Return accuracy of DecisionTreeModel on the given RDD[LabeledPoint]. + """ + seqOp = (lambda acc, x: acc + (x[0] == x[1])) + predictions = dtModel.predict(data.map(lambda x: x.features)) + truth = data.map(lambda p: p.label) + trainCorrect = predictions.zip(truth).aggregate(0, seqOp, add) + if data.count() == 0: + return 0 + return trainCorrect / (0.0 + data.count()) + + +def getMSE(dtModel, data): + """ + Return mean squared error (MSE) of DecisionTreeModel on the given + RDD[LabeledPoint]. + """ + seqOp = (lambda acc, x: acc + numpy.square(x[0] - x[1])) + predictions = dtModel.predict(data.map(lambda x: x.features)) + truth = data.map(lambda p: p.label) + trainMSE = predictions.zip(truth).aggregate(0, seqOp, add) + if data.count() == 0: + return 0 + return trainMSE / (0.0 + data.count()) + + +def reindexClassLabels(data): + """ + Re-index class labels in a dataset to the range {0,...,numClasses-1}. + If all labels in that range already appear at least once, + then the returned RDD is the same one (without a mapping). + Note: If a label simply does not appear in the data, + the index will not include it. + Be aware of this when reindexing subsampled data. + :param data: RDD of LabeledPoint where labels are integer values + denoting labels for a classification problem. + :return: Pair (reindexedData, origToNewLabels) where + reindexedData is an RDD of LabeledPoint with labels in + the range {0,...,numClasses-1}, and + origToNewLabels is a dictionary mapping original labels + to new labels. + """ + # classCounts: class --> # examples in class + classCounts = data.map(lambda x: x.label).countByValue() + numExamples = sum(classCounts.values()) + sortedClasses = sorted(classCounts.keys()) + numClasses = len(classCounts) + # origToNewLabels: class --> index in 0,...,numClasses-1 + if (numClasses < 2): + print >> sys.stderr, \ + "Dataset for classification should have at least 2 classes." + \ + " The given dataset had only %d classes." % numClasses + exit(1) + origToNewLabels = dict([(sortedClasses[i], i) for i in range(0, numClasses)]) + + print "numClasses = %d" % numClasses + print "Per-class example fractions, counts:" + print "Class\tFrac\tCount" + for c in sortedClasses: + frac = classCounts[c] / (numExamples + 0.0) + print "%g\t%g\t%d" % (c, frac, classCounts[c]) + + if (sortedClasses[0] == 0 and sortedClasses[-1] == numClasses - 1): + return (data, origToNewLabels) + else: + reindexedData = \ + data.map(lambda x: LabeledPoint(origToNewLabels[x.label], x.features)) + return (reindexedData, origToNewLabels) + + +def usage(): + print >> sys.stderr, \ + "Usage: decision_tree_runner [libsvm format data filepath]\n" + \ + " Note: This only supports binary classification." + exit(1) + + +if __name__ == "__main__": + if len(sys.argv) > 2: + usage() + sc = SparkContext(appName="PythonDT") + + # Load data. + dataPath = 'data/mllib/sample_libsvm_data.txt' + if len(sys.argv) == 2: + dataPath = sys.argv[1] + if not os.path.isfile(dataPath): + usage() + points = MLUtils.loadLibSVMFile(sc, dataPath) + + # Re-index class labels if needed. + (reindexedData, origToNewLabels) = reindexClassLabels(points) + + # Train a classifier. + model = DecisionTree.trainClassifier(reindexedData, numClasses=2) + # Print learned tree and stats. + print "Trained DecisionTree for classification:" + print " Model numNodes: %d\n" % model.numNodes() + print " Model depth: %d\n" % model.depth() + print " Training accuracy: %g\n" % getAccuracy(model, reindexedData) + print model diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 6e0f7a4ee5a81..9d547ff77c984 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -30,8 +30,10 @@ from pyspark.mllib.classification import LogisticRegressionWithSGD -# Parse a line of text into an MLlib LabeledPoint object def parsePoint(line): + """ + Parse a line of text into an MLlib LabeledPoint object. + """ values = [float(s) for s in line.split(' ')] if values[0] == -1: # Convert -1 labels to 0 for MLlib values[0] = 0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 7d912737b8f0b..1d5d3762ed8e9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.api.python import java.nio.{ByteBuffer, ByteOrder} +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ @@ -29,6 +31,11 @@ import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.random.{RandomRDDGenerators => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} +import org.apache.spark.mllib.tree.model.DecisionTreeModel import org.apache.spark.mllib.stat.Statistics import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.util.MLUtils @@ -472,6 +479,76 @@ class PythonMLLibAPI extends Serializable { ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) } + /** + * Java stub for Python mllib DecisionTree.train(). + * This stub returns a handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on exit; + * see the Py4J documentation. + * @param dataBytesJRDD Training data + * @param categoricalFeaturesInfoJMap Categorical features info, as Java map + */ + def trainDecisionTreeModel( + dataBytesJRDD: JavaRDD[Array[Byte]], + algoStr: String, + numClasses: Int, + categoricalFeaturesInfoJMap: java.util.Map[Int, Int], + impurityStr: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + + val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint) + + val algo: Algo = algoStr match { + case "classification" => Classification + case "regression" => Regression + case _ => throw new IllegalArgumentException(s"Bad algoStr parameter: $algoStr") + } + val impurity: Impurity = impurityStr match { + case "gini" => Gini + case "entropy" => Entropy + case "variance" => Variance + case _ => throw new IllegalArgumentException(s"Bad impurityStr parameter: $impurityStr") + } + + val strategy = new Strategy( + algo = algo, + impurity = impurity, + maxDepth = maxDepth, + numClassesForClassification = numClasses, + maxBins = maxBins, + categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap) + + DecisionTree.train(data, strategy) + } + + /** + * Predict the label of the given data point. + * This is a Java stub for python DecisionTreeModel.predict() + * + * @param featuresBytes Serialized feature vector for data point + * @return predicted label + */ + def predictDecisionTreeModel( + model: DecisionTreeModel, + featuresBytes: Array[Byte]): Double = { + val features: Vector = deserializeDoubleVector(featuresBytes) + model.predict(features) + } + + /** + * Predict the labels of the given data points. + * This is a Java stub for python DecisionTreeModel.predict() + * + * @param dataJRDD A JavaRDD with serialized feature vectors + * @return JavaRDD of serialized predictions + */ + def predictDecisionTreeModel( + model: DecisionTreeModel, + dataJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { + val data = dataJRDD.rdd.map(xBytes => deserializeDoubleVector(xBytes)) + model.predict(data).map(serializeDouble) + } + /** * Java stub for mllib Statistics.corr(X: RDD[Vector], method: String). * Returns the correlation matrix serialized into a byte array understood by deserializers in @@ -597,4 +674,5 @@ class PythonMLLibAPI extends Serializable { val s = getSeedOrDefault(seed) RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(serializeDoubleVector) } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 5c65b537b6867..fdad4f029aa99 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -56,7 +56,8 @@ class Strategy ( if (algo == Classification) { require(numClassesForClassification >= 2) } - val isMulticlassClassification = numClassesForClassification > 2 + val isMulticlassClassification = + algo == Classification && numClassesForClassification > 2 val isMulticlassWithCategoricalFeatures = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 546a132559326..8665a00f3b356 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -48,7 +48,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { requiredMSE: Double) { val predictions = input.map(x => model.predict(x.features)) val squaredError = predictions.zip(input).map { case (prediction, expected) => - (prediction - expected.label) * (prediction - expected.label) + val err = prediction - expected.label + err * err }.sum val mse = squaredError / input.length assert(mse <= requiredMSE) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index c6ca6a75df746..9c1565affbdac 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -343,22 +343,35 @@ def _copyto(array, buffer, offset, shape, dtype): temp_array[...] = array -def _get_unmangled_rdd(data, serializer): +def _get_unmangled_rdd(data, serializer, cache=True): + """ + :param cache: If True, the serialized RDD is cached. (default = True) + WARNING: Users should unpersist() this later! + """ dataBytes = data.map(serializer) dataBytes._bypass_serializer = True - dataBytes.cache() # TODO: users should unpersist() this later! + if cache: + dataBytes.cache() return dataBytes -# Map a pickled Python RDD of Python dense or sparse vectors to a Java RDD of -# _serialized_double_vectors -def _get_unmangled_double_vector_rdd(data): - return _get_unmangled_rdd(data, _serialize_double_vector) +def _get_unmangled_double_vector_rdd(data, cache=True): + """ + Map a pickled Python RDD of Python dense or sparse vectors to a Java RDD of + _serialized_double_vectors. + :param cache: If True, the serialized RDD is cached. (default = True) + WARNING: Users should unpersist() this later! + """ + return _get_unmangled_rdd(data, _serialize_double_vector, cache) -# Map a pickled Python RDD of LabeledPoint to a Java RDD of _serialized_labeled_points -def _get_unmangled_labeled_point_rdd(data): - return _get_unmangled_rdd(data, _serialize_labeled_point) +def _get_unmangled_labeled_point_rdd(data, cache=True): + """ + Map a pickled Python RDD of LabeledPoint to a Java RDD of _serialized_labeled_points. + :param cache: If True, the serialized RDD is cached. (default = True) + WARNING: Users should unpersist() this later! + """ + return _get_unmangled_rdd(data, _serialize_labeled_point, cache) # Common functions for dealing with and training linear models @@ -380,7 +393,7 @@ def _linear_predictor_typecheck(x, coeffs): if x.size != coeffs.shape[0]: raise RuntimeError("Got sparse vector of size %d; wanted %d" % ( x.size, coeffs.shape[0])) - elif (type(x) == RDD): + elif isinstance(x, RDD): raise RuntimeError("Bulk predict not yet supported.") else: raise TypeError("Argument of type " + type(x).__name__ + " unsupported") diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 37ccf1d590743..9d1e5be637a9a 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -100,6 +100,7 @@ def test_clustering(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(0.0, [1, 0, 0]), LabeledPoint(1.0, [0, 1, 1]), @@ -127,9 +128,19 @@ def test_classification(self): self.assertTrue(nb_model.predict(features[2]) <= 0) self.assertTrue(nb_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories + dt_model = \ + DecisionTree.trainClassifier(rdd, numClasses=2, + categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + def test_regression(self): from pyspark.mllib.regression import LinearRegressionWithSGD, LassoWithSGD, \ RidgeRegressionWithSGD + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(-1.0, [0, -1]), LabeledPoint(1.0, [0, 1]), @@ -157,6 +168,14 @@ def test_regression(self): self.assertTrue(rr_model.predict(features[2]) <= 0) self.assertTrue(rr_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories + dt_model = \ + DecisionTree.trainRegressor(rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): @@ -229,6 +248,7 @@ def test_clustering(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(0.0, self.scipy_matrix(2, {0: 1.0})), LabeledPoint(1.0, self.scipy_matrix(2, {1: 1.0})), @@ -256,9 +276,18 @@ def test_classification(self): self.assertTrue(nb_model.predict(features[2]) <= 0) self.assertTrue(nb_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories + dt_model = DecisionTree.trainClassifier(rdd, numClasses=2, + categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + def test_regression(self): from pyspark.mllib.regression import LinearRegressionWithSGD, LassoWithSGD, \ RidgeRegressionWithSGD + from pyspark.mllib.tree import DecisionTree data = [ LabeledPoint(-1.0, self.scipy_matrix(2, {1: -1.0})), LabeledPoint(1.0, self.scipy_matrix(2, {1: 1.0})), @@ -286,6 +315,13 @@ def test_regression(self): self.assertTrue(rr_model.predict(features[2]) <= 0) self.assertTrue(rr_model.predict(features[3]) > 0) + categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories + dt_model = DecisionTree.trainRegressor(rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) + self.assertTrue(dt_model.predict(features[0]) <= 0) + self.assertTrue(dt_model.predict(features[1]) > 0) + self.assertTrue(dt_model.predict(features[2]) <= 0) + self.assertTrue(dt_model.predict(features[3]) > 0) + if __name__ == "__main__": if not _have_scipy: diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py new file mode 100644 index 0000000000000..1e0006df75ac6 --- /dev/null +++ b/python/pyspark/mllib/tree.py @@ -0,0 +1,225 @@ +# +# 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. +# + +from py4j.java_collections import MapConverter + +from pyspark import SparkContext, RDD +from pyspark.mllib._common import \ + _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _serialize_double_vector, \ + _deserialize_labeled_point, _get_unmangled_labeled_point_rdd, \ + _deserialize_double +from pyspark.mllib.regression import LabeledPoint +from pyspark.serializers import NoOpSerializer + +class DecisionTreeModel(object): + """ + A decision tree model for classification or regression. + + EXPERIMENTAL: This is an experimental API. + It will probably be modified for Spark v1.2. + """ + + def __init__(self, sc, java_model): + """ + :param sc: Spark context + :param java_model: Handle to Java model object + """ + self._sc = sc + self._java_model = java_model + + def __del__(self): + self._sc._gateway.detach(self._java_model) + + def predict(self, x): + """ + Predict the label of one or more examples. + :param x: Data point (feature vector), + or an RDD of data points (feature vectors). + """ + pythonAPI = self._sc._jvm.PythonMLLibAPI() + if isinstance(x, RDD): + # Bulk prediction + if x.count() == 0: + return self._sc.parallelize([]) + dataBytes = _get_unmangled_double_vector_rdd(x, cache=False) + jSerializedPreds = \ + pythonAPI.predictDecisionTreeModel(self._java_model, + dataBytes._jrdd) + serializedPreds = RDD(jSerializedPreds, self._sc, NoOpSerializer()) + return serializedPreds.map(lambda bytes: _deserialize_double(bytearray(bytes))) + else: + # Assume x is a single data point. + x_ = _serialize_double_vector(x) + return pythonAPI.predictDecisionTreeModel(self._java_model, x_) + + def numNodes(self): + return self._java_model.numNodes() + + def depth(self): + return self._java_model.depth() + + def __str__(self): + return self._java_model.toString() + + +class DecisionTree(object): + """ + Learning algorithm for a decision tree model + for classification or regression. + + EXPERIMENTAL: This is an experimental API. + It will probably be modified for Spark v1.2. + + Example usage: + >>> from numpy import array, ndarray + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> from pyspark.mllib.linalg import SparseVector + >>> + >>> data = [ + ... LabeledPoint(0.0, [0.0]), + ... LabeledPoint(1.0, [1.0]), + ... LabeledPoint(1.0, [2.0]), + ... LabeledPoint(1.0, [3.0]) + ... ] + >>> + >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2) + >>> print(model) + DecisionTreeModel classifier + If (feature 0 <= 0.5) + Predict: 0.0 + Else (feature 0 > 0.5) + Predict: 1.0 + + >>> model.predict(array([1.0])) > 0 + True + >>> model.predict(array([0.0])) == 0 + True + >>> sparse_data = [ + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) + ... ] + >>> + >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data)) + >>> model.predict(array([0.0, 1.0])) == 1 + True + >>> model.predict(array([0.0, 0.0])) == 0 + True + >>> model.predict(SparseVector(2, {1: 1.0})) == 1 + True + >>> model.predict(SparseVector(2, {1: 0.0})) == 0 + True + """ + + @staticmethod + def trainClassifier(data, numClasses, categoricalFeaturesInfo={}, + impurity="gini", maxDepth=4, maxBins=100): + """ + Train a DecisionTreeModel for classification. + + :param data: Training data: RDD of LabeledPoint. + Labels are integers {0,1,...,numClasses}. + :param numClasses: Number of classes for classification. + :param categoricalFeaturesInfo: Map from categorical feature index + to number of categories. + Any feature not in this map + is treated as continuous. + :param impurity: Supported values: "entropy" or "gini" + :param maxDepth: Max depth of tree. + E.g., depth 0 means 1 leaf node. + Depth 1 means 1 internal node + 2 leaf nodes. + :param maxBins: Number of bins used for finding splits at each node. + :return: DecisionTreeModel + """ + return DecisionTree.train(data, "classification", numClasses, + categoricalFeaturesInfo, + impurity, maxDepth, maxBins) + + @staticmethod + def trainRegressor(data, categoricalFeaturesInfo={}, + impurity="variance", maxDepth=4, maxBins=100): + """ + Train a DecisionTreeModel for regression. + + :param data: Training data: RDD of LabeledPoint. + Labels are real numbers. + :param categoricalFeaturesInfo: Map from categorical feature index + to number of categories. + Any feature not in this map + is treated as continuous. + :param impurity: Supported values: "variance" + :param maxDepth: Max depth of tree. + E.g., depth 0 means 1 leaf node. + Depth 1 means 1 internal node + 2 leaf nodes. + :param maxBins: Number of bins used for finding splits at each node. + :return: DecisionTreeModel + """ + return DecisionTree.train(data, "regression", 0, + categoricalFeaturesInfo, + impurity, maxDepth, maxBins) + + + @staticmethod + def train(data, algo, numClasses, categoricalFeaturesInfo, + impurity, maxDepth, maxBins=100): + """ + Train a DecisionTreeModel for classification or regression. + + :param data: Training data: RDD of LabeledPoint. + For classification, labels are integers + {0,1,...,numClasses}. + For regression, labels are real numbers. + :param algo: "classification" or "regression" + :param numClasses: Number of classes for classification. + :param categoricalFeaturesInfo: Map from categorical feature index + to number of categories. + Any feature not in this map + is treated as continuous. + :param impurity: For classification: "entropy" or "gini". + For regression: "variance". + :param maxDepth: Max depth of tree. + E.g., depth 0 means 1 leaf node. + Depth 1 means 1 internal node + 2 leaf nodes. + :param maxBins: Number of bins used for finding splits at each node. + :return: DecisionTreeModel + """ + sc = data.context + dataBytes = _get_unmangled_labeled_point_rdd(data) + categoricalFeaturesInfoJMap = \ + MapConverter().convert(categoricalFeaturesInfo, + sc._gateway._gateway_client) + model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( + dataBytes._jrdd, algo, + numClasses, categoricalFeaturesInfoJMap, + impurity, maxDepth, maxBins) + dataBytes.unpersist() + return DecisionTreeModel(sc, model) + + +def _test(): + import doctest + globs = globals().copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index d94900cefdb77..639cda6350229 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -16,6 +16,7 @@ # import numpy as np +import warnings from pyspark.mllib.linalg import Vectors, SparseVector from pyspark.mllib.regression import LabeledPoint @@ -29,9 +30,9 @@ class MLUtils: Helper methods to load, save and pre-process data used in MLlib. """ - @deprecated @staticmethod def _parse_libsvm_line(line, multiclass): + warnings.warn("deprecated", DeprecationWarning) return _parse_libsvm_line(line) @staticmethod @@ -67,9 +68,9 @@ def _convert_labeled_point_to_libsvm(p): " but got " % type(v)) return " ".join(items) - @deprecated @staticmethod def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): + warnings.warn("deprecated", DeprecationWarning) return loadLibSVMFile(sc, path, numFeatures, minPartitions) @staticmethod @@ -106,7 +107,6 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() - >>> multiclass_examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() >>> tempFile.close() >>> type(examples[0]) == LabeledPoint True @@ -115,20 +115,18 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): >>> type(examples[1]) == LabeledPoint True >>> print examples[1] - (0.0,(6,[],[])) + (-1.0,(6,[],[])) >>> type(examples[2]) == LabeledPoint True >>> print examples[2] - (0.0,(6,[1,3,5],[4.0,5.0,6.0])) - >>> multiclass_examples[1].label - -1.0 + (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) """ lines = sc.textFile(path, minPartitions) parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l)) if numFeatures <= 0: parsed.cache() - numFeatures = parsed.map(lambda x: 0 if x[1].size == 0 else x[1][-1]).reduce(max) + 1 + numFeatures = parsed.map(lambda x: -1 if x[1].size == 0 else x[1][-1]).reduce(max) + 1 return parsed.map(lambda x: LabeledPoint(x[0], Vectors.sparse(numFeatures, x[1], x[2]))) @staticmethod diff --git a/python/run-tests b/python/run-tests index 5049e15ce5f8a..48feba2f5bd63 100755 --- a/python/run-tests +++ b/python/run-tests @@ -71,6 +71,7 @@ run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" run_test "pyspark/mllib/tests.py" +run_test "pyspark/mllib/util.py" if [[ $FAILED == 0 ]]; then echo -en "\033[32m" # Green From 67bd8e3c217a80c3117a6e3853aa60fe13d08c91 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 2 Aug 2014 13:16:41 -0700 Subject: [PATCH 0216/1492] [SQL] Set outputPartitioning of BroadcastHashJoin correctly. I think we will not generate the plan triggering this bug at this moment. But, let me explain it... Right now, we are using `left.outputPartitioning` as the `outputPartitioning` of a `BroadcastHashJoin`. We may have a wrong physical plan for cases like... ```sql SELECT l.key, count(*) FROM (SELECT key, count(*) as cnt FROM src GROUP BY key) l // This is buildPlan JOIN r // This is the streamedPlan ON (l.cnt = r.value) GROUP BY l.key ``` Let's say we have a `BroadcastHashJoin` on `l` and `r`. For this case, we will pick `l`'s `outputPartitioning` for the `outputPartitioning`of the `BroadcastHashJoin` on `l` and `r`. Also, because the last `GROUP BY` is using `l.key` as the key, we will not introduce an `Exchange` for this aggregation. However, `r`'s outputPartitioning may not match the required distribution of the last `GROUP BY` and we fail to group data correctly. JIRA is being reindexed. I will create a JIRA ticket once it is back online. Author: Yin Huai Closes #1735 from yhuai/BroadcastHashJoin and squashes the following commits: 96d9cb3 [Yin Huai] Set outputPartitioning correctly. --- .../src/main/scala/org/apache/spark/sql/execution/joins.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index cc138c749949d..51bb61530744c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -405,8 +405,7 @@ case class BroadcastHashJoin( left: SparkPlan, right: SparkPlan) extends BinaryNode with HashJoin { - - override def outputPartitioning: Partitioning = left.outputPartitioning + override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning override def requiredChildDistribution = UnspecifiedDistribution :: UnspecifiedDistribution :: Nil From 91f9504e6086fac05b40545099f9818949c24bca Mon Sep 17 00:00:00 2001 From: Chris Fregly Date: Sat, 2 Aug 2014 13:35:35 -0700 Subject: [PATCH 0217/1492] [SPARK-1981] Add AWS Kinesis streaming support Author: Chris Fregly Closes #1434 from cfregly/master and squashes the following commits: 4774581 [Chris Fregly] updated docs, renamed retry to retryRandom to be more clear, removed retries around store() method 0393795 [Chris Fregly] moved Kinesis examples out of examples/ and back into extras/kinesis-asl 691a6be [Chris Fregly] fixed tests and formatting, fixed a bug with JavaKinesisWordCount during union of streams 0e1c67b [Chris Fregly] Merge remote-tracking branch 'upstream/master' 74e5c7c [Chris Fregly] updated per TD's feedback. simplified examples, updated docs e33cbeb [Chris Fregly] Merge remote-tracking branch 'upstream/master' bf614e9 [Chris Fregly] per matei's feedback: moved the kinesis examples into the examples/ dir d17ca6d [Chris Fregly] per TD's feedback: updated docs, simplified the KinesisUtils api 912640c [Chris Fregly] changed the foundKinesis class to be a publically-avail class db3eefd [Chris Fregly] Merge remote-tracking branch 'upstream/master' 21de67f [Chris Fregly] Merge remote-tracking branch 'upstream/master' 6c39561 [Chris Fregly] parameterized the versions of the aws java sdk and kinesis client 338997e [Chris Fregly] improve build docs for kinesis 828f8ae [Chris Fregly] more cleanup e7c8978 [Chris Fregly] Merge remote-tracking branch 'upstream/master' cd68c0d [Chris Fregly] fixed typos and backward compatibility d18e680 [Chris Fregly] Merge remote-tracking branch 'upstream/master' b3b0ff1 [Chris Fregly] [SPARK-1981] Add AWS Kinesis streaming support --- bin/run-example | 3 +- bin/run-example2.cmd | 3 +- dev/audit-release/audit_release.py | 4 +- .../src/main/scala/SparkApp.scala | 7 + dev/audit-release/sbt_app_kinesis/build.sbt | 28 ++ .../src/main/scala/SparkApp.scala | 33 +++ dev/create-release/create-release.sh | 4 +- dev/run-tests | 3 + docs/streaming-custom-receivers.md | 4 +- docs/streaming-kinesis.md | 58 ++++ docs/streaming-programming-guide.md | 12 +- examples/pom.xml | 13 + extras/kinesis-asl/pom.xml | 96 ++++++ .../streaming/JavaKinesisWordCountASL.java | 180 ++++++++++++ .../src/main/resources/log4j.properties | 37 +++ .../streaming/KinesisWordCountASL.scala | 251 ++++++++++++++++ .../kinesis/KinesisCheckpointState.scala | 56 ++++ .../streaming/kinesis/KinesisReceiver.scala | 149 ++++++++++ .../kinesis/KinesisRecordProcessor.scala | 212 ++++++++++++++ .../streaming/kinesis/KinesisUtils.scala | 96 ++++++ .../kinesis/JavaKinesisStreamSuite.java | 41 +++ .../src/test/resources/log4j.properties | 26 ++ .../kinesis/KinesisReceiverSuite.scala | 275 ++++++++++++++++++ pom.xml | 10 + project/SparkBuild.scala | 6 +- 25 files changed, 1592 insertions(+), 15 deletions(-) create mode 100644 dev/audit-release/sbt_app_kinesis/build.sbt create mode 100644 dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala create mode 100644 docs/streaming-kinesis.md create mode 100644 extras/kinesis-asl/pom.xml create mode 100644 extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java create mode 100644 extras/kinesis-asl/src/main/resources/log4j.properties create mode 100644 extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala create mode 100644 extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala create mode 100644 extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala create mode 100644 extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala create mode 100644 extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala create mode 100644 extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java create mode 100644 extras/kinesis-asl/src/test/resources/log4j.properties create mode 100644 extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala diff --git a/bin/run-example b/bin/run-example index 942706d733122..68a35702eddd3 100755 --- a/bin/run-example +++ b/bin/run-example @@ -29,7 +29,8 @@ if [ -n "$1" ]; then else echo "Usage: ./bin/run-example [example-args]" 1>&2 echo " - set MASTER=XX to use a specific master" 1>&2 - echo " - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression)" 1>&2 + echo " - can use abbreviated example class name relative to com.apache.spark.examples" 1>&2 + echo " (e.g. SparkPi, mllib.LinearRegression, streaming.KinesisWordCountASL)" 1>&2 exit 1 fi diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index eadedd7fa61ff..b29bf90c64e90 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -32,7 +32,8 @@ rem Test that an argument was given if not "x%1"=="x" goto arg_given echo Usage: run-example ^ [example-args] echo - set MASTER=XX to use a specific master - echo - can use abbreviated example class name (e.g. SparkPi, mllib.LinearRegression) + echo - can use abbreviated example class name relative to com.apache.spark.examples + echo (e.g. SparkPi, mllib.LinearRegression, streaming.KinesisWordCountASL) goto exit :arg_given diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 230e900ecd4de..16ea1a71290dc 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -105,7 +105,7 @@ def get_url(url): "spark-core", "spark-bagel", "spark-mllib", "spark-streaming", "spark-repl", "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq", - "spark-catalyst", "spark-sql", "spark-hive" + "spark-catalyst", "spark-sql", "spark-hive", "spark-streaming-kinesis-asl" ] modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules) @@ -136,7 +136,7 @@ def ensure_path_not_present(x): os.chdir(original_dir) # SBT application tests -for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive"]: +for app in ["sbt_app_core", "sbt_app_graphx", "sbt_app_streaming", "sbt_app_sql", "sbt_app_hive", "sbt_app_kinesis"]: os.chdir(app) ret = run_cmd("sbt clean run", exit_on_failure=False) test(ret == 0, "sbt application (%s)" % app) diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 77bbd167b199a..fc03fec9866a6 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -50,5 +50,12 @@ object SimpleApp { println("Ganglia sink was loaded via spark-core") System.exit(-1) } + + // Remove kinesis from default build due to ASL license issue + val foundKinesis = Try(Class.forName("org.apache.spark.streaming.kinesis.KinesisUtils")).isSuccess + if (foundKinesis) { + println("Kinesis was loaded via spark-core") + System.exit(-1) + } } } diff --git a/dev/audit-release/sbt_app_kinesis/build.sbt b/dev/audit-release/sbt_app_kinesis/build.sbt new file mode 100644 index 0000000000000..981bc7957b5ed --- /dev/null +++ b/dev/audit-release/sbt_app_kinesis/build.sbt @@ -0,0 +1,28 @@ +// +// 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. +// + +name := "Kinesis Test" + +version := "1.0" + +scalaVersion := System.getenv.get("SCALA_VERSION") + +libraryDependencies += "org.apache.spark" %% "spark-streaming-kinesis-asl" % System.getenv.get("SPARK_VERSION") + +resolvers ++= Seq( + "Spark Release Repository" at System.getenv.get("SPARK_RELEASE_REPOSITORY"), + "Spray Repository" at "http://repo.spray.cc/") diff --git a/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala new file mode 100644 index 0000000000000..9f85066501472 --- /dev/null +++ b/dev/audit-release/sbt_app_kinesis/src/main/scala/SparkApp.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package main.scala + +import scala.util.Try + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ + +object SimpleApp { + def main(args: Array[String]) { + val foundKinesis = Try(Class.forName("org.apache.spark.streaming.kinesis.KinesisUtils")).isSuccess + if (!foundKinesis) { + println("Kinesis not loaded via kinesis-asl") + System.exit(-1) + } + } +} diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index af46572e6602b..42473629d4f15 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -53,15 +53,15 @@ if [[ ! "$@" =~ --package-only ]]; then -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ release:perform cd .. diff --git a/dev/run-tests b/dev/run-tests index daa85bc750c07..d401c90f41d7b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -36,6 +36,9 @@ fi if [ -z "$SBT_MAVEN_PROFILES_ARGS" ]; then export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" fi + +export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" + echo "SBT_MAVEN_PROFILES_ARGS=\"$SBT_MAVEN_PROFILES_ARGS\"" # Remove work directory diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index a2dc3a8961dfc..1e045a3dd0ca9 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -4,7 +4,7 @@ title: Spark Streaming Custom Receivers --- Spark Streaming can receive streaming data from any arbitrary data source beyond -the one's for which it has in-built support (that is, beyond Flume, Kafka, files, sockets, etc.). +the one's for which it has in-built support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). This requires the developer to implement a *receiver* that is customized for receiving data from the concerned data source. This guide walks through the process of implementing a custom receiver and using it in a Spark Streaming application. @@ -174,7 +174,7 @@ val words = lines.flatMap(_.split(" ")) ... {% endhighlight %} -The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala). +The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala).
    diff --git a/docs/streaming-kinesis.md b/docs/streaming-kinesis.md new file mode 100644 index 0000000000000..801c905c88df8 --- /dev/null +++ b/docs/streaming-kinesis.md @@ -0,0 +1,58 @@ +--- +layout: global +title: Spark Streaming Kinesis Receiver +--- + +### Kinesis +Build notes: +
  • Spark supports a Kinesis Streaming Receiver which is not included in the default build due to licensing restrictions.
  • +
  • _**Note that by embedding this library you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • +
  • The Spark Kinesis Streaming Receiver source code, examples, tests, and artifacts live in $SPARK_HOME/extras/kinesis-asl.
  • +
  • To build with Kinesis, you must run the maven or sbt builds with -Pkinesis-asl`.
  • +
  • Applications will need to link to the 'spark-streaming-kinesis-asl` artifact.
  • + +Kinesis examples notes: +
  • To build the Kinesis examples, you must run the maven or sbt builds with -Pkinesis-asl`.
  • +
  • These examples automatically determine the number of local threads and KinesisReceivers to spin up based on the number of shards for the stream.
  • +
  • KinesisWordCountProducerASL will generate random data to put onto the Kinesis stream for testing.
  • +
  • Checkpointing is disabled (no checkpoint dir is set). The examples as written will not recover from a driver failure.
  • + +Deployment and runtime notes: +
  • A single KinesisReceiver can process many shards of a stream.
  • +
  • Each shard of a stream is processed by one or more KinesisReceiver's managed by the Kinesis Client Library (KCL) Worker.
  • +
  • You never need more KinesisReceivers than the number of shards in your stream.
  • +
  • You can horizontally scale the receiving by creating more KinesisReceiver/DStreams (up to the number of shards for a given stream)
  • +
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the Kinesis Client Library.
  • +
  • This code uses the DefaultAWSCredentialsProviderChain and searches for credentials in the following order of precedence:
    + 1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    + 2) Java System Properties - aws.accessKeyId and aws.secretKey
    + 3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    + 4) Instance profile credentials - delivered through the Amazon EC2 metadata service
    +
  • +
  • You need to setup a Kinesis stream with 1 or more shards per the following:
    + http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • +
  • Valid Kinesis endpoint urls can be found here: Valid endpoint urls: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • +
  • When you first start up the KinesisReceiver, the Kinesis Client Library (KCL) needs ~30s to establish connectivity with the AWS Kinesis service, +retrieve any checkpoint data, and negotiate with other KCL's reading from the same stream.
  • +
  • Be careful when changing the app name. Kinesis maintains a mapping table in DynamoDB based on this app name (http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization). +Changing the app name could lead to Kinesis errors as only 1 logical application can process a stream. In order to start fresh, +it's always best to delete the DynamoDB table that matches your app name. This DynamoDB table lives in us-east-1 regardless of the Kinesis endpoint URL.
  • + +Failure recovery notes: +
  • The combination of Spark Streaming and Kinesis creates 3 different checkpoints as follows:
    + 1) RDD data checkpoint (Spark Streaming) - frequency is configurable with DStream.checkpoint(Duration)
    + 2) RDD metadata checkpoint (Spark Streaming) - frequency is every DStream batch
    + 3) Kinesis checkpointing (Kinesis) - frequency is controlled by the developer calling ICheckpointer.checkpoint() directly
    +
  • +
  • Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling
  • +
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last checkpoint sequence number recorded per shard.
  • +
  • If no checkpoint info exists, the worker will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) +or from the tip/latest (InitialPostitionInStream.LATEST). This is configurable.
  • +
  • When pulling from the stream tip (InitialPositionInStream.LATEST), only new stream data will be picked up after the KinesisReceiver starts.
  • +
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running.
  • +
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data +depending on the checkpoint frequency.
  • +
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records depending on the checkpoint frequency.
  • +
  • Record processing should be idempotent when possible.
  • +
  • Failed or latent KinesisReceivers will be detected and automatically shutdown/load-balanced by the KCL.
  • +
  • If possible, explicitly shutdown the worker if a failure occurs in order to trigger the final checkpoint.
  • diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 7b8b7933434c4..9f331ed50d2a4 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -9,7 +9,7 @@ title: Spark Streaming Programming Guide # Overview Spark Streaming is an extension of the core Spark API that allows enables high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources -like Kafka, Flume, Twitter, ZeroMQ or plain old TCP sockets and be processed using complex +like Kafka, Flume, Twitter, ZeroMQ, Kinesis or plain old TCP sockets and be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark's in-built @@ -38,7 +38,7 @@ stream of results in batches. Spark Streaming provides a high-level abstraction called *discretized stream* or *DStream*, which represents a continuous stream of data. DStreams can be created either from input data -stream from sources such as Kafka and Flume, or by applying high-level +stream from sources such as Kafka, Flume, and Kinesis, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of [RDDs](api/scala/index.html#org.apache.spark.rdd.RDD). @@ -313,7 +313,7 @@ To write your own Spark Streaming program, you will have to add the following de artifactId = spark-streaming_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} -For ingesting data from sources like Kafka and Flume that are not present in the Spark +For ingesting data from sources like Kafka, Flume, and Kinesis that are not present in the Spark Streaming core API, you will have to add the corresponding artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, @@ -327,6 +327,7 @@ some of the common ones are as follows. Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}} ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}} MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}} + Kinesis
    (built separately) kinesis-asl_{{site.SCALA_BINARY_VERSION}} @@ -442,7 +443,7 @@ see the API documentations of the relevant functions in Scala and [JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) for Java. -Additional functionality for creating DStreams from sources such as Kafka, Flume, and Twitter +Additional functionality for creating DStreams from sources such as Kafka, Flume, Kinesis, and Twitter can be imported by adding the right dependencies as explained in an [earlier](#linking) section. To take the case of Kafka, after adding the artifact `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` to the @@ -467,6 +468,9 @@ For more details on these additional sources, see the corresponding [API documen Furthermore, you can also implement your own custom receiver for your sources. See the [Custom Receiver Guide](streaming-custom-receivers.html). +### Kinesis +[Kinesis](streaming-kinesis.html) + ## Operations There are two kinds of DStream operations - _transformations_ and _output operations_. Similar to RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams diff --git a/examples/pom.xml b/examples/pom.xml index c4ed0f5a6a02b..8c4c128bb484d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -34,6 +34,19 @@ Spark Project Examples http://spark.apache.org/ + + + kinesis-asl + + + org.apache.spark + spark-streaming-kinesis-asl_${scala.binary.version} + ${project.version} + + + + + org.apache.spark diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml new file mode 100644 index 0000000000000..a54b34235dfb4 --- /dev/null +++ b/extras/kinesis-asl/pom.xml @@ -0,0 +1,96 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.1.0-SNAPSHOT + ../../pom.xml + + + + org.apache.spark + spark-streaming-kinesis-asl_2.10 + jar + Spark Kinesis Integration + + + kinesis-asl + + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + + + com.amazonaws + amazon-kinesis-client + ${aws.kinesis.client.version} + + + com.amazonaws + aws-java-sdk + ${aws.java.sdk.version} + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.mockito + mockito-all + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.easymock + easymockclassextension + test + + + com.novocode + junit-interface + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.scalatest + scalatest-maven-plugin + + + + diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java new file mode 100644 index 0000000000000..a8b907b241893 --- /dev/null +++ b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -0,0 +1,180 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.examples.streaming; + +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Pattern; + +import org.apache.log4j.Logger; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.FlatMapFunction; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.kinesis.KinesisUtils; + +import scala.Tuple2; + +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.google.common.collect.Lists; + +/** + * Java-friendly Kinesis Spark Streaming WordCount example + * + * See http://spark.apache.org/docs/latest/streaming-kinesis.html for more details + * on the Kinesis Spark Streaming integration. + * + * This example spins up 1 Kinesis Worker (Spark Streaming Receiver) per shard + * for the given stream. + * It then starts pulling from the last checkpointed sequence number of the given + * and . + * + * Valid endpoint urls: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region + * + * This code uses the DefaultAWSCredentialsProviderChain and searches for credentials + * in the following order of precedence: + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Java System Properties - aws.accessKeyId and aws.secretKey + * Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs + * Instance profile credentials - delivered through the Amazon EC2 metadata service + * + * Usage: JavaKinesisWordCountASL + * is the name of the Kinesis stream (ie. mySparkStream) + * is the endpoint of the Kinesis service + * (ie. https://kinesis.us-east-1.amazonaws.com) + * + * Example: + * $ export AWS_ACCESS_KEY_ID= + * $ export AWS_SECRET_KEY= + * $ $SPARK_HOME/bin/run-example \ + * org.apache.spark.examples.streaming.JavaKinesisWordCountASL mySparkStream \ + * https://kinesis.us-east-1.amazonaws.com + * + * There is a companion helper class called KinesisWordCountProducerASL which puts dummy data + * onto the Kinesis stream. + * Usage instructions for KinesisWordCountProducerASL are provided in the class definition. + */ +public final class JavaKinesisWordCountASL { + private static final Pattern WORD_SEPARATOR = Pattern.compile(" "); + private static final Logger logger = Logger.getLogger(JavaKinesisWordCountASL.class); + + /* Make the constructor private to enforce singleton */ + private JavaKinesisWordCountASL() { + } + + public static void main(String[] args) { + /* Check that all required args were passed in. */ + if (args.length < 2) { + System.err.println( + "|Usage: KinesisWordCount \n" + + "| is the name of the Kinesis stream\n" + + "| is the endpoint of the Kinesis service\n" + + "| (e.g. https://kinesis.us-east-1.amazonaws.com)\n"); + System.exit(1); + } + + StreamingExamples.setStreamingLogLevels(); + + /* Populate the appropriate variables from the given args */ + String streamName = args[0]; + String endpointUrl = args[1]; + /* Set the batch interval to a fixed 2000 millis (2 seconds) */ + Duration batchInterval = new Duration(2000); + + /* Create a Kinesis client in order to determine the number of shards for the given stream */ + AmazonKinesisClient kinesisClient = new AmazonKinesisClient( + new DefaultAWSCredentialsProviderChain()); + kinesisClient.setEndpoint(endpointUrl); + + /* Determine the number of shards from the stream */ + int numShards = kinesisClient.describeStream(streamName) + .getStreamDescription().getShards().size(); + + /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard */ + int numStreams = numShards; + + /* Must add 1 more thread than the number of receivers or the output won't show properly from the driver */ + int numSparkThreads = numStreams + 1; + + /* Setup the Spark config. */ + SparkConf sparkConfig = new SparkConf().setAppName("KinesisWordCount").setMaster( + "local[" + numSparkThreads + "]"); + + /* Kinesis checkpoint interval. Same as batchInterval for this example. */ + Duration checkpointInterval = batchInterval; + + /* Setup the StreamingContext */ + JavaStreamingContext jssc = new JavaStreamingContext(sparkConfig, batchInterval); + + /* Create the same number of Kinesis DStreams/Receivers as Kinesis stream's shards */ + List> streamsList = new ArrayList>(numStreams); + for (int i = 0; i < numStreams; i++) { + streamsList.add( + KinesisUtils.createStream(jssc, streamName, endpointUrl, checkpointInterval, + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2()) + ); + } + + /* Union all the streams if there is more than 1 stream */ + JavaDStream unionStreams; + if (streamsList.size() > 1) { + unionStreams = jssc.union(streamsList.get(0), streamsList.subList(1, streamsList.size())); + } else { + /* Otherwise, just use the 1 stream */ + unionStreams = streamsList.get(0); + } + + /* + * Split each line of the union'd DStreams into multiple words using flatMap to produce the collection. + * Convert lines of byte[] to multiple Strings by first converting to String, then splitting on WORD_SEPARATOR. + */ + JavaDStream words = unionStreams.flatMap(new FlatMapFunction() { + @Override + public Iterable call(byte[] line) { + return Lists.newArrayList(WORD_SEPARATOR.split(new String(line))); + } + }); + + /* Map each word to a (word, 1) tuple, then reduce/aggregate by word. */ + JavaPairDStream wordCounts = words.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(String s) { + return new Tuple2(s, 1); + } + }).reduceByKey(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + }); + + /* Print the first 10 wordCounts */ + wordCounts.print(); + + /* Start the streaming context and await termination */ + jssc.start(); + jssc.awaitTermination(); + } +} diff --git a/extras/kinesis-asl/src/main/resources/log4j.properties b/extras/kinesis-asl/src/main/resources/log4j.properties new file mode 100644 index 0000000000000..97348fb5b6123 --- /dev/null +++ b/extras/kinesis-asl/src/main/resources/log4j.properties @@ -0,0 +1,37 @@ +# +# 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. +# + +log4j.rootCategory=WARN, console + +# File appender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Console appender +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.out +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO \ No newline at end of file diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala new file mode 100644 index 0000000000000..d03edf8b30a9f --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.streaming + +import java.nio.ByteBuffer +import scala.util.Random +import org.apache.spark.Logging +import org.apache.spark.SparkConf +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Milliseconds +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions +import org.apache.spark.streaming.kinesis.KinesisUtils +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain +import com.amazonaws.services.kinesis.AmazonKinesisClient +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.model.PutRecordRequest +import org.apache.log4j.Logger +import org.apache.log4j.Level + +/** + * Kinesis Spark Streaming WordCount example. + * + * See http://spark.apache.org/docs/latest/streaming-kinesis.html for more details on + * the Kinesis Spark Streaming integration. + * + * This example spins up 1 Kinesis Worker (Spark Streaming Receiver) per shard + * for the given stream. + * It then starts pulling from the last checkpointed sequence number of the given + * and . + * + * Valid endpoint urls: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region + * + * This code uses the DefaultAWSCredentialsProviderChain and searches for credentials + * in the following order of precedence: + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Java System Properties - aws.accessKeyId and aws.secretKey + * Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs + * Instance profile credentials - delivered through the Amazon EC2 metadata service + * + * Usage: KinesisWordCountASL + * is the name of the Kinesis stream (ie. mySparkStream) + * is the endpoint of the Kinesis service + * (ie. https://kinesis.us-east-1.amazonaws.com) + * + * Example: + * $ export AWS_ACCESS_KEY_ID= + * $ export AWS_SECRET_KEY= + * $ $SPARK_HOME/bin/run-example \ + * org.apache.spark.examples.streaming.KinesisWordCountASL mySparkStream \ + * https://kinesis.us-east-1.amazonaws.com + * + * There is a companion helper class below called KinesisWordCountProducerASL which puts + * dummy data onto the Kinesis stream. + * Usage instructions for KinesisWordCountProducerASL are provided in that class definition. + */ +object KinesisWordCountASL extends Logging { + def main(args: Array[String]) { + /* Check that all required args were passed in. */ + if (args.length < 2) { + System.err.println( + """ + |Usage: KinesisWordCount + | is the name of the Kinesis stream + | is the endpoint of the Kinesis service + | (e.g. https://kinesis.us-east-1.amazonaws.com) + """.stripMargin) + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + /* Populate the appropriate variables from the given args */ + val Array(streamName, endpointUrl) = args + + /* Determine the number of shards from the stream */ + val kinesisClient = new AmazonKinesisClient(new DefaultAWSCredentialsProviderChain()) + kinesisClient.setEndpoint(endpointUrl) + val numShards = kinesisClient.describeStream(streamName).getStreamDescription().getShards() + .size() + + /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard. */ + val numStreams = numShards + + /* + * numSparkThreads should be 1 more thread than the number of receivers. + * This leaves one thread available for actually processing the data. + */ + val numSparkThreads = numStreams + 1 + + /* Setup the and SparkConfig and StreamingContext */ + /* Spark Streaming batch interval */ + val batchInterval = Milliseconds(2000) + val sparkConfig = new SparkConf().setAppName("KinesisWordCount") + .setMaster(s"local[$numSparkThreads]") + val ssc = new StreamingContext(sparkConfig, batchInterval) + + /* Kinesis checkpoint interval. Same as batchInterval for this example. */ + val kinesisCheckpointInterval = batchInterval + + /* Create the same number of Kinesis DStreams/Receivers as Kinesis stream's shards */ + val kinesisStreams = (0 until numStreams).map { i => + KinesisUtils.createStream(ssc, streamName, endpointUrl, kinesisCheckpointInterval, + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2) + } + + /* Union all the streams */ + val unionStreams = ssc.union(kinesisStreams) + + /* Convert each line of Array[Byte] to String, split into words, and count them */ + val words = unionStreams.flatMap(byteArray => new String(byteArray) + .split(" ")) + + /* Map each word to a (word, 1) tuple so we can reduce/aggregate by key. */ + val wordCounts = words.map(word => (word, 1)).reduceByKey(_ + _) + + /* Print the first 10 wordCounts */ + wordCounts.print() + + /* Start the streaming context and await termination */ + ssc.start() + ssc.awaitTermination() + } +} + +/** + * Usage: KinesisWordCountProducerASL + * + * is the name of the Kinesis stream (ie. mySparkStream) + * is the endpoint of the Kinesis service + * (ie. https://kinesis.us-east-1.amazonaws.com) + * is the rate of records per second to put onto the stream + * is the rate of records per second to put onto the stream + * + * Example: + * $ export AWS_ACCESS_KEY_ID= + * $ export AWS_SECRET_KEY= + * $ $SPARK_HOME/bin/run-example \ + * org.apache.spark.examples.streaming.KinesisWordCountProducerASL mySparkStream \ + * https://kinesis.us-east-1.amazonaws.com 10 5 + */ +object KinesisWordCountProducerASL { + def main(args: Array[String]) { + if (args.length < 4) { + System.err.println("Usage: KinesisWordCountProducerASL " + + " ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + /* Populate the appropriate variables from the given args */ + val Array(stream, endpoint, recordsPerSecond, wordsPerRecord) = args + + /* Generate the records and return the totals */ + val totals = generate(stream, endpoint, recordsPerSecond.toInt, wordsPerRecord.toInt) + + /* Print the array of (index, total) tuples */ + println("Totals") + totals.foreach(total => println(total.toString())) + } + + def generate(stream: String, + endpoint: String, + recordsPerSecond: Int, + wordsPerRecord: Int): Seq[(Int, Int)] = { + + val MaxRandomInts = 10 + + /* Create the Kinesis client */ + val kinesisClient = new AmazonKinesisClient(new DefaultAWSCredentialsProviderChain()) + kinesisClient.setEndpoint(endpoint) + + println(s"Putting records onto stream $stream and endpoint $endpoint at a rate of" + + s" $recordsPerSecond records per second and $wordsPerRecord words per record"); + + val totals = new Array[Int](MaxRandomInts) + /* Put String records onto the stream per the given recordPerSec and wordsPerRecord */ + for (i <- 1 to 5) { + + /* Generate recordsPerSec records to put onto the stream */ + val records = (1 to recordsPerSecond.toInt).map { recordNum => + /* + * Randomly generate each wordsPerRec words between 0 (inclusive) + * and MAX_RANDOM_INTS (exclusive) + */ + val data = (1 to wordsPerRecord.toInt).map(x => { + /* Generate the random int */ + val randomInt = Random.nextInt(MaxRandomInts) + + /* Keep track of the totals */ + totals(randomInt) += 1 + + randomInt.toString() + }).mkString(" ") + + /* Create a partitionKey based on recordNum */ + val partitionKey = s"partitionKey-$recordNum" + + /* Create a PutRecordRequest with an Array[Byte] version of the data */ + val putRecordRequest = new PutRecordRequest().withStreamName(stream) + .withPartitionKey(partitionKey) + .withData(ByteBuffer.wrap(data.getBytes())); + + /* Put the record onto the stream and capture the PutRecordResult */ + val putRecordResult = kinesisClient.putRecord(putRecordRequest); + } + + /* Sleep for a second */ + Thread.sleep(1000) + println("Sent " + recordsPerSecond + " records") + } + + /* Convert the totals to (index, total) tuple */ + (0 to (MaxRandomInts - 1)).zip(totals) + } +} + +/** + * Utility functions for Spark Streaming examples. + * This has been lifted from the examples/ project to remove the circular dependency. + */ +object StreamingExamples extends Logging { + + /** Set reasonable logging levels for streaming if the user has not configured log4j. */ + def setStreamingLogLevels() { + val log4jInitialized = Logger.getRootLogger.getAllAppenders.hasMoreElements + if (!log4jInitialized) { + // We first log something to initialize Spark's default logging, then we override the + // logging level. + logInfo("Setting log level to [WARN] for streaming example." + + " To override add a custom log4j.properties to the classpath.") + Logger.getRootLogger.setLevel(Level.WARN) + } + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala new file mode 100644 index 0000000000000..0b80b611cdce7 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointState.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import org.apache.spark.Logging +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.util.Clock +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.util.SystemClock + +/** + * This is a helper class for managing checkpoint clocks. + * + * @param checkpointInterval + * @param currentClock. Default to current SystemClock if none is passed in (mocking purposes) + */ +private[kinesis] class KinesisCheckpointState( + checkpointInterval: Duration, + currentClock: Clock = new SystemClock()) + extends Logging { + + /* Initialize the checkpoint clock using the given currentClock + checkpointInterval millis */ + val checkpointClock = new ManualClock() + checkpointClock.setTime(currentClock.currentTime() + checkpointInterval.milliseconds) + + /** + * Check if it's time to checkpoint based on the current time and the derived time + * for the next checkpoint + * + * @return true if it's time to checkpoint + */ + def shouldCheckpoint(): Boolean = { + new SystemClock().currentTime() > checkpointClock.currentTime() + } + + /** + * Advance the checkpoint clock by the checkpoint interval. + */ + def advanceCheckpoint() = { + checkpointClock.addToTime(checkpointInterval.milliseconds) + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala new file mode 100644 index 0000000000000..1bd1f324298e7 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import java.net.InetAddress +import java.util.UUID + +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.receiver.Receiver + +import com.amazonaws.auth.AWSCredentialsProvider +import com.amazonaws.auth.DefaultAWSCredentialsProviderChain +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorFactory +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.KinesisClientLibConfiguration +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.Worker + +/** + * Custom AWS Kinesis-specific implementation of Spark Streaming's Receiver. + * This implementation relies on the Kinesis Client Library (KCL) Worker as described here: + * https://github.com/awslabs/amazon-kinesis-client + * This is a custom receiver used with StreamingContext.receiverStream(Receiver) + * as described here: + * http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * Instances of this class will get shipped to the Spark Streaming Workers + * to run within a Spark Executor. + * + * @param appName Kinesis application name. Kinesis Apps are mapped to Kinesis Streams + * by the Kinesis Client Library. If you change the App name or Stream name, + * the KCL will throw errors. This usually requires deleting the backing + * DynamoDB table with the same name this Kinesis application. + * @param streamName Kinesis stream name + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the + * worker's initial starting position in the stream. + * The values are either the beginning of the stream + * per Kinesis' limit of 24 hours + * (InitialPositionInStream.TRIM_HORIZON) or + * the tip of the stream (InitialPositionInStream.LATEST). + * @param storageLevel Storage level to use for storing the received objects + * + * @return ReceiverInputDStream[Array[Byte]] + */ +private[kinesis] class KinesisReceiver( + appName: String, + streamName: String, + endpointUrl: String, + checkpointInterval: Duration, + initialPositionInStream: InitialPositionInStream, + storageLevel: StorageLevel) + extends Receiver[Array[Byte]](storageLevel) with Logging { receiver => + + /* + * The following vars are built in the onStart() method which executes in the Spark Worker after + * this code is serialized and shipped remotely. + */ + + /* + * workerId should be based on the ip address of the actual Spark Worker where this code runs + * (not the Driver's ip address.) + */ + var workerId: String = null + + /* + * This impl uses the DefaultAWSCredentialsProviderChain and searches for credentials + * in the following order of precedence: + * Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY + * Java System Properties - aws.accessKeyId and aws.secretKey + * Credential profiles file at the default location (~/.aws/credentials) shared by all + * AWS SDKs and the AWS CLI + * Instance profile credentials delivered through the Amazon EC2 metadata service + */ + var credentialsProvider: AWSCredentialsProvider = null + + /* KCL config instance. */ + var kinesisClientLibConfiguration: KinesisClientLibConfiguration = null + + /* + * RecordProcessorFactory creates impls of IRecordProcessor. + * IRecordProcessor adapts the KCL to our Spark KinesisReceiver via the + * IRecordProcessor.processRecords() method. + * We're using our custom KinesisRecordProcessor in this case. + */ + var recordProcessorFactory: IRecordProcessorFactory = null + + /* + * Create a Kinesis Worker. + * This is the core client abstraction from the Kinesis Client Library (KCL). + * We pass the RecordProcessorFactory from above as well as the KCL config instance. + * A Kinesis Worker can process 1..* shards from the given stream - each with its + * own RecordProcessor. + */ + var worker: Worker = null + + /** + * This is called when the KinesisReceiver starts and must be non-blocking. + * The KCL creates and manages the receiving/processing thread pool through the Worker.run() + * method. + */ + override def onStart() { + workerId = InetAddress.getLocalHost.getHostAddress() + ":" + UUID.randomUUID() + credentialsProvider = new DefaultAWSCredentialsProviderChain() + kinesisClientLibConfiguration = new KinesisClientLibConfiguration(appName, streamName, + credentialsProvider, workerId).withKinesisEndpoint(endpointUrl) + .withInitialPositionInStream(initialPositionInStream).withTaskBackoffTimeMillis(500) + recordProcessorFactory = new IRecordProcessorFactory { + override def createProcessor: IRecordProcessor = new KinesisRecordProcessor(receiver, + workerId, new KinesisCheckpointState(checkpointInterval)) + } + worker = new Worker(recordProcessorFactory, kinesisClientLibConfiguration) + worker.run() + logInfo(s"Started receiver with workerId $workerId") + } + + /** + * This is called when the KinesisReceiver stops. + * The KCL worker.shutdown() method stops the receiving/processing threads. + * The KCL will do its best to drain and checkpoint any in-flight records upon shutdown. + */ + override def onStop() { + worker.shutdown() + logInfo(s"Shut down receiver with workerId $workerId") + workerId = null + credentialsProvider = null + kinesisClientLibConfiguration = null + recordProcessorFactory = null + worker = null + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala new file mode 100644 index 0000000000000..8ecc2d90160b1 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisRecordProcessor.scala @@ -0,0 +1,212 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import java.util.List + +import scala.collection.JavaConversions.asScalaBuffer +import scala.util.Random + +import org.apache.spark.Logging + +import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessor +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason +import com.amazonaws.services.kinesis.model.Record + +/** + * Kinesis-specific implementation of the Kinesis Client Library (KCL) IRecordProcessor. + * This implementation operates on the Array[Byte] from the KinesisReceiver. + * The Kinesis Worker creates an instance of this KinesisRecordProcessor upon startup. + * + * @param receiver Kinesis receiver + * @param workerId for logging purposes + * @param checkpointState represents the checkpoint state including the next checkpoint time. + * It's injected here for mocking purposes. + */ +private[kinesis] class KinesisRecordProcessor( + receiver: KinesisReceiver, + workerId: String, + checkpointState: KinesisCheckpointState) extends IRecordProcessor with Logging { + + /* shardId to be populated during initialize() */ + var shardId: String = _ + + /** + * The Kinesis Client Library calls this method during IRecordProcessor initialization. + * + * @param shardId assigned by the KCL to this particular RecordProcessor. + */ + override def initialize(shardId: String) { + logInfo(s"Initialize: Initializing workerId $workerId with shardId $shardId") + this.shardId = shardId + } + + /** + * This method is called by the KCL when a batch of records is pulled from the Kinesis stream. + * This is the record-processing bridge between the KCL's IRecordProcessor.processRecords() + * and Spark Streaming's Receiver.store(). + * + * @param batch list of records from the Kinesis stream shard + * @param checkpointer used to update Kinesis when this batch has been processed/stored + * in the DStream + */ + override def processRecords(batch: List[Record], checkpointer: IRecordProcessorCheckpointer) { + if (!receiver.isStopped()) { + try { + /* + * Note: If we try to store the raw ByteBuffer from record.getData(), the Spark Streaming + * Receiver.store(ByteBuffer) attempts to deserialize the ByteBuffer using the + * internally-configured Spark serializer (kryo, etc). + * This is not desirable, so we instead store a raw Array[Byte] and decouple + * ourselves from Spark's internal serialization strategy. + */ + batch.foreach(record => receiver.store(record.getData().array())) + + logDebug(s"Stored: Worker $workerId stored ${batch.size} records for shardId $shardId") + + /* + * Checkpoint the sequence number of the last record successfully processed/stored + * in the batch. + * In this implementation, we're checkpointing after the given checkpointIntervalMillis. + * Note that this logic requires that processRecords() be called AND that it's time to + * checkpoint. I point this out because there is no background thread running the + * checkpointer. Checkpointing is tested and trigger only when a new batch comes in. + * If the worker is shutdown cleanly, checkpoint will happen (see shutdown() below). + * However, if the worker dies unexpectedly, a checkpoint may not happen. + * This could lead to records being processed more than once. + */ + if (checkpointState.shouldCheckpoint()) { + /* Perform the checkpoint */ + KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100) + + /* Update the next checkpoint time */ + checkpointState.advanceCheckpoint() + + logDebug(s"Checkpoint: WorkerId $workerId completed checkpoint of ${batch.size}" + + s" records for shardId $shardId") + logDebug(s"Checkpoint: Next checkpoint is at " + + s" ${checkpointState.checkpointClock.currentTime()} for shardId $shardId") + } + } catch { + case e: Throwable => { + /* + * If there is a failure within the batch, the batch will not be checkpointed. + * This will potentially cause records since the last checkpoint to be processed + * more than once. + */ + logError(s"Exception: WorkerId $workerId encountered and exception while storing " + + " or checkpointing a batch for workerId $workerId and shardId $shardId.", e) + + /* Rethrow the exception to the Kinesis Worker that is managing this RecordProcessor.*/ + throw e + } + } + } else { + /* RecordProcessor has been stopped. */ + logInfo(s"Stopped: The Spark KinesisReceiver has stopped for workerId $workerId" + + s" and shardId $shardId. No more records will be processed.") + } + } + + /** + * Kinesis Client Library is shutting down this Worker for 1 of 2 reasons: + * 1) the stream is resharding by splitting or merging adjacent shards + * (ShutdownReason.TERMINATE) + * 2) the failed or latent Worker has stopped sending heartbeats for whatever reason + * (ShutdownReason.ZOMBIE) + * + * @param checkpointer used to perform a Kinesis checkpoint for ShutdownReason.TERMINATE + * @param reason for shutdown (ShutdownReason.TERMINATE or ShutdownReason.ZOMBIE) + */ + override def shutdown(checkpointer: IRecordProcessorCheckpointer, reason: ShutdownReason) { + logInfo(s"Shutdown: Shutting down workerId $workerId with reason $reason") + reason match { + /* + * TERMINATE Use Case. Checkpoint. + * Checkpoint to indicate that all records from the shard have been drained and processed. + * It's now OK to read from the new shards that resulted from a resharding event. + */ + case ShutdownReason.TERMINATE => + KinesisRecordProcessor.retryRandom(checkpointer.checkpoint(), 4, 100) + + /* + * ZOMBIE Use Case. NoOp. + * No checkpoint because other workers may have taken over and already started processing + * the same records. + * This may lead to records being processed more than once. + */ + case ShutdownReason.ZOMBIE => + + /* Unknown reason. NoOp */ + case _ => + } + } +} + +private[kinesis] object KinesisRecordProcessor extends Logging { + /** + * Retry the given amount of times with a random backoff time (millis) less than the + * given maxBackOffMillis + * + * @param expression expression to evalute + * @param numRetriesLeft number of retries left + * @param maxBackOffMillis: max millis between retries + * + * @return evaluation of the given expression + * @throws Unretryable exception, unexpected exception, + * or any exception that persists after numRetriesLeft reaches 0 + */ + @annotation.tailrec + def retryRandom[T](expression: => T, numRetriesLeft: Int, maxBackOffMillis: Int): T = { + util.Try { expression } match { + /* If the function succeeded, evaluate to x. */ + case util.Success(x) => x + /* If the function failed, either retry or throw the exception */ + case util.Failure(e) => e match { + /* Retry: Throttling or other Retryable exception has occurred */ + case _: ThrottlingException | _: KinesisClientLibDependencyException if numRetriesLeft > 1 + => { + val backOffMillis = Random.nextInt(maxBackOffMillis) + Thread.sleep(backOffMillis) + logError(s"Retryable Exception: Random backOffMillis=${backOffMillis}", e) + retryRandom(expression, numRetriesLeft - 1, maxBackOffMillis) + } + /* Throw: Shutdown has been requested by the Kinesis Client Library.*/ + case _: ShutdownException => { + logError(s"ShutdownException: Caught shutdown exception, skipping checkpoint.", e) + throw e + } + /* Throw: Non-retryable exception has occurred with the Kinesis Client Library */ + case _: InvalidStateException => { + logError(s"InvalidStateException: Cannot save checkpoint to the DynamoDB table used" + + s" by the Amazon Kinesis Client Library. Table likely doesn't exist.", e) + throw e + } + /* Throw: Unexpected exception has occurred */ + case _ => { + logError(s"Unexpected, non-retryable exception.", e) + throw e + } + } + } + } +} diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala new file mode 100644 index 0000000000000..713cac0e293c0 --- /dev/null +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import org.apache.spark.annotation.Experimental +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Duration +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream +import org.apache.spark.streaming.api.java.JavaStreamingContext +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + +/** + * Helper class to create Amazon Kinesis Input Stream + * :: Experimental :: + */ +@Experimental +object KinesisUtils { + /** + * Create an InputDStream that pulls messages from a Kinesis stream. + * + * @param ssc StreamingContext object + * @param streamName Kinesis stream name + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the + * worker's initial starting position in the stream. + * The values are either the beginning of the stream + * per Kinesis' limit of 24 hours + * (InitialPositionInStream.TRIM_HORIZON) or + * the tip of the stream (InitialPositionInStream.LATEST). + * @param storageLevel Storage level to use for storing the received objects + * + * @return ReceiverInputDStream[Array[Byte]] + */ + def createStream( + ssc: StreamingContext, + streamName: String, + endpointUrl: String, + checkpointInterval: Duration, + initialPositionInStream: InitialPositionInStream, + storageLevel: StorageLevel): ReceiverInputDStream[Array[Byte]] = { + ssc.receiverStream(new KinesisReceiver(ssc.sc.appName, streamName, endpointUrl, + checkpointInterval, initialPositionInStream, storageLevel)) + } + + /** + * Create a Java-friendly InputDStream that pulls messages from a Kinesis stream. + * + * @param jssc Java StreamingContext object + * @param ssc StreamingContext object + * @param streamName Kinesis stream name + * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) + * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. + * See the Kinesis Spark Streaming documentation for more + * details on the different types of checkpoints. + * @param initialPositionInStream In the absence of Kinesis checkpoint info, this is the + * worker's initial starting position in the stream. + * The values are either the beginning of the stream + * per Kinesis' limit of 24 hours + * (InitialPositionInStream.TRIM_HORIZON) or + * the tip of the stream (InitialPositionInStream.LATEST). + * @param storageLevel Storage level to use for storing the received objects + * + * @return JavaReceiverInputDStream[Array[Byte]] + */ + def createStream( + jssc: JavaStreamingContext, + streamName: String, + endpointUrl: String, + checkpointInterval: Duration, + initialPositionInStream: InitialPositionInStream, + storageLevel: StorageLevel): JavaReceiverInputDStream[Array[Byte]] = { + jssc.receiverStream(new KinesisReceiver(jssc.ssc.sc.appName, streamName, + endpointUrl, checkpointInterval, initialPositionInStream, storageLevel)) + } +} diff --git a/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java b/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java new file mode 100644 index 0000000000000..87954a31f60ce --- /dev/null +++ b/extras/kinesis-asl/src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kinesis; + +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.junit.Test; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; + +/** + * Demonstrate the use of the KinesisUtils Java API + */ +public class JavaKinesisStreamSuite extends LocalJavaStreamingContext { + @Test + public void testKinesisStream() { + // Tests the API, does not actually test data receiving + JavaDStream kinesisStream = KinesisUtils.createStream(ssc, "mySparkStream", + "https://kinesis.us-west-2.amazonaws.com", new Duration(2000), + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2()); + + ssc.stop(); + } +} diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..e01e049595475 --- /dev/null +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -0,0 +1,26 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN diff --git a/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala new file mode 100644 index 0000000000000..41dbd64c2b1fa --- /dev/null +++ b/extras/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -0,0 +1,275 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.kinesis + +import java.nio.ByteBuffer + +import scala.collection.JavaConversions.seqAsJavaList + +import org.apache.spark.annotation.Experimental +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.Milliseconds +import org.apache.spark.streaming.Seconds +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.TestSuiteBase +import org.apache.spark.streaming.util.Clock +import org.apache.spark.streaming.util.ManualClock +import org.scalatest.BeforeAndAfter +import org.scalatest.Matchers +import org.scalatest.mock.EasyMockSugar + +import com.amazonaws.services.kinesis.clientlibrary.exceptions.InvalidStateException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.KinesisClientLibDependencyException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ShutdownException +import com.amazonaws.services.kinesis.clientlibrary.exceptions.ThrottlingException +import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream +import com.amazonaws.services.kinesis.clientlibrary.types.ShutdownReason +import com.amazonaws.services.kinesis.model.Record + +/** + * Suite of Kinesis streaming receiver tests focusing mostly on the KinesisRecordProcessor + */ +class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAfter + with EasyMockSugar { + + val app = "TestKinesisReceiver" + val stream = "mySparkStream" + val endpoint = "endpoint-url" + val workerId = "dummyWorkerId" + val shardId = "dummyShardId" + + val record1 = new Record() + record1.setData(ByteBuffer.wrap("Spark In Action".getBytes())) + val record2 = new Record() + record2.setData(ByteBuffer.wrap("Learning Spark".getBytes())) + val batch = List[Record](record1, record2) + + var receiverMock: KinesisReceiver = _ + var checkpointerMock: IRecordProcessorCheckpointer = _ + var checkpointClockMock: ManualClock = _ + var checkpointStateMock: KinesisCheckpointState = _ + var currentClockMock: Clock = _ + + override def beforeFunction() = { + receiverMock = mock[KinesisReceiver] + checkpointerMock = mock[IRecordProcessorCheckpointer] + checkpointClockMock = mock[ManualClock] + checkpointStateMock = mock[KinesisCheckpointState] + currentClockMock = mock[Clock] + } + + test("kinesis utils api") { + val ssc = new StreamingContext(master, framework, batchDuration) + // Tests the API, does not actually test data receiving + val kinesisStream = KinesisUtils.createStream(ssc, "mySparkStream", + "https://kinesis.us-west-2.amazonaws.com", Seconds(2), + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2); + ssc.stop() + } + + test("process records including store and checkpoint") { + val expectedCheckpointIntervalMillis = 10 + expecting { + receiverMock.isStopped().andReturn(false).once() + receiverMock.store(record1.getData().array()).once() + receiverMock.store(record2.getData().array()).once() + checkpointStateMock.shouldCheckpoint().andReturn(true).once() + checkpointerMock.checkpoint().once() + checkpointStateMock.advanceCheckpoint().once() + } + whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + } + } + + test("shouldn't store and checkpoint when receiver is stopped") { + expecting { + receiverMock.isStopped().andReturn(true).once() + } + whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + } + } + + test("shouldn't checkpoint when exception occurs during store") { + expecting { + receiverMock.isStopped().andReturn(false).once() + receiverMock.store(record1.getData().array()).andThrow(new RuntimeException()).once() + } + whenExecuting(receiverMock, checkpointerMock, checkpointStateMock) { + intercept[RuntimeException] { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.processRecords(batch, checkpointerMock) + } + } + } + + test("should set checkpoint time to currentTime + checkpoint interval upon instantiation") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointIntervalMillis = 10 + val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) + assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) + } + } + + test("should checkpoint if we have exceeded the checkpoint interval") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MinValue), currentClockMock) + assert(checkpointState.shouldCheckpoint()) + } + } + + test("shouldn't checkpoint if we have not exceeded the checkpoint interval") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointState = new KinesisCheckpointState(Milliseconds(Long.MaxValue), currentClockMock) + assert(!checkpointState.shouldCheckpoint()) + } + } + + test("should add to time when advancing checkpoint") { + expecting { + currentClockMock.currentTime().andReturn(0).once() + } + whenExecuting(currentClockMock) { + val checkpointIntervalMillis = 10 + val checkpointState = new KinesisCheckpointState(Milliseconds(checkpointIntervalMillis), currentClockMock) + assert(checkpointState.checkpointClock.currentTime() == checkpointIntervalMillis) + checkpointState.advanceCheckpoint() + assert(checkpointState.checkpointClock.currentTime() == (2 * checkpointIntervalMillis)) + } + } + + test("shutdown should checkpoint if the reason is TERMINATE") { + expecting { + checkpointerMock.checkpoint().once() + } + whenExecuting(checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + val reason = ShutdownReason.TERMINATE + recordProcessor.shutdown(checkpointerMock, reason) + } + } + + test("shutdown should not checkpoint if the reason is something other than TERMINATE") { + expecting { + } + whenExecuting(checkpointerMock, checkpointStateMock) { + val recordProcessor = new KinesisRecordProcessor(receiverMock, workerId, + checkpointStateMock) + recordProcessor.shutdown(checkpointerMock, ShutdownReason.ZOMBIE) + recordProcessor.shutdown(checkpointerMock, null) + } + } + + test("retry success on first attempt") { + val expectedIsStopped = false + expecting { + receiverMock.isStopped().andReturn(expectedIsStopped).once() + } + whenExecuting(receiverMock) { + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + } + } + + test("retry success on second attempt after a Kinesis throttling exception") { + val expectedIsStopped = false + expecting { + receiverMock.isStopped().andThrow(new ThrottlingException("error message")) + .andReturn(expectedIsStopped).once() + } + whenExecuting(receiverMock) { + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + } + } + + test("retry success on second attempt after a Kinesis dependency exception") { + val expectedIsStopped = false + expecting { + receiverMock.isStopped().andThrow(new KinesisClientLibDependencyException("error message")) + .andReturn(expectedIsStopped).once() + } + whenExecuting(receiverMock) { + val actualVal = KinesisRecordProcessor.retryRandom(receiverMock.isStopped(), 2, 100) + assert(actualVal == expectedIsStopped) + } + } + + test("retry failed after a shutdown exception") { + expecting { + checkpointerMock.checkpoint().andThrow(new ShutdownException("error message")).once() + } + whenExecuting(checkpointerMock) { + intercept[ShutdownException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + } + } + + test("retry failed after an invalid state exception") { + expecting { + checkpointerMock.checkpoint().andThrow(new InvalidStateException("error message")).once() + } + whenExecuting(checkpointerMock) { + intercept[InvalidStateException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + } + } + + test("retry failed after unexpected exception") { + expecting { + checkpointerMock.checkpoint().andThrow(new RuntimeException("error message")).once() + } + whenExecuting(checkpointerMock) { + intercept[RuntimeException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + } + } + + test("retry failed after exhausing all retries") { + val expectedErrorMessage = "final try error message" + expecting { + checkpointerMock.checkpoint().andThrow(new ThrottlingException("error message")) + .andThrow(new ThrottlingException(expectedErrorMessage)).once() + } + whenExecuting(checkpointerMock) { + val exception = intercept[RuntimeException] { + KinesisRecordProcessor.retryRandom(checkpointerMock.checkpoint(), 2, 100) + } + exception.getMessage().shouldBe(expectedErrorMessage) + } + } +} diff --git a/pom.xml b/pom.xml index 99ae4b8b33f94..a42759169149b 100644 --- a/pom.xml +++ b/pom.xml @@ -134,6 +134,8 @@ 3.0.0 1.7.6 0.7.1 + 1.8.3 + 1.1.0 64m 512m @@ -1011,6 +1013,14 @@ + + + kinesis-asl + + extras/kinesis-asl + + + java8-tests diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1d7cc6dd6aef3..aac621fe53938 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -37,8 +37,8 @@ object BuildCommons { "spark", "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) - val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl) = - Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl") + val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = + Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl", "kinesis-asl") .map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(assembly, examples) = Seq("assembly", "examples") @@ -62,7 +62,7 @@ object SparkBuild extends PomBuild { var isAlphaYarn = false var profiles: mutable.Seq[String] = mutable.Seq.empty if (Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined) { - println("NOTE: SPARK_GANGLIA_LGPL is deprecated, please use -Pganglia-lgpl flag.") + println("NOTE: SPARK_GANGLIA_LGPL is deprecated, please use -Pspark-ganglia-lgpl flag.") profiles ++= Seq("spark-ganglia-lgpl") } if (Properties.envOrNone("SPARK_HIVE").isDefined) { From 4c477117bb1ffef463776c86f925d35036f96b7a Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 2 Aug 2014 13:55:28 -0700 Subject: [PATCH 0218/1492] SPARK-2804: Remove scalalogging-slf4j dependency This also Closes #1701. Author: GuoQiang Li Closes #1208 from witgo/SPARK-1470 and squashes the following commits: 422646b [GuoQiang Li] Remove scalalogging-slf4j dependency --- .../main/scala/org/apache/spark/Logging.scala | 10 ++++++--- sql/catalyst/pom.xml | 5 ----- .../sql/catalyst/analysis/Analyzer.scala | 4 ++-- .../catalyst/analysis/HiveTypeCoercion.scala | 8 +++---- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../codegen/GenerateOrdering.scala | 4 ++-- .../apache/spark/sql/catalyst/package.scala | 1 - .../sql/catalyst/planning/QueryPlanner.scala | 2 +- .../sql/catalyst/planning/patterns.scala | 6 ++--- .../spark/sql/catalyst/rules/Rule.scala | 2 +- .../sql/catalyst/rules/RuleExecutor.scala | 12 +++++----- .../spark/sql/catalyst/trees/package.scala | 8 ++++--- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../CompressibleColumnBuilder.scala | 5 +++-- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../org/apache/spark/sql/json/JsonRDD.scala | 2 +- .../scala/org/apache/spark/sql/package.scala | 2 -- .../spark/sql/columnar/ColumnTypeSuite.scala | 4 ++-- .../hive/thriftserver/HiveThriftServer2.scala | 12 +++++----- .../hive/thriftserver/SparkSQLCLIDriver.scala | 2 +- .../hive/thriftserver/SparkSQLDriver.scala | 6 ++--- .../sql/hive/thriftserver/SparkSQLEnv.scala | 6 ++--- .../server/SparkSQLOperationManager.scala | 13 ++++++----- .../thriftserver/HiveThriftServer2Suite.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 3 ++- .../org/apache/spark/sql/hive/TestHive.scala | 10 ++++----- .../org/apache/spark/sql/hive/hiveUdfs.scala | 4 ++-- .../hive/execution/HiveComparisonTest.scala | 22 +++++++++---------- .../hive/execution/HiveQueryFileTest.scala | 2 +- 30 files changed, 83 insertions(+), 82 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 807ef3e9c9d60..d4f2624061e35 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -39,13 +39,17 @@ trait Logging { // be serialized and used on another machine @transient private var log_ : Logger = null + // Method to get the logger name for this object + protected def logName = { + // Ignore trailing $'s in the class names for Scala objects + this.getClass.getName.stripSuffix("$") + } + // Method to get or create the logger for this object protected def log: Logger = { if (log_ == null) { initializeIfNecessary() - var className = this.getClass.getName - // Ignore trailing $'s in the class names for Scala objects - log_ = LoggerFactory.getLogger(className.stripSuffix("$")) + log_ = LoggerFactory.getLogger(logName) } log_ } diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 54fa96baa1e18..58d44e7923bee 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -54,11 +54,6 @@ spark-core_${scala.binary.version} ${project.version} - - com.typesafe - scalalogging-slf4j_${scala.binary.version} - 1.0.1 - org.scalatest scalatest_${scala.binary.version} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 74c0104e5b17f..2ba68cab115fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -109,12 +109,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case q: LogicalPlan if q.childrenResolved => - logger.trace(s"Attempting to resolve ${q.simpleString}") + logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = q.resolve(name).getOrElse(u) - logger.debug(s"Resolving $u to $result") + logDebug(s"Resolving $u to $result") result } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 47c7ad076ad07..e94f2a3bea63e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -75,7 +75,7 @@ trait HiveTypeCoercion { // Leave the same if the dataTypes match. case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => - logger.debug(s"Promoting $a to $newType in ${q.simpleString}}") + logDebug(s"Promoting $a to $newType in ${q.simpleString}}") newType } } @@ -154,7 +154,7 @@ trait HiveTypeCoercion { (Alias(Cast(l, StringType), l.name)(), r) case (l, r) if l.dataType != r.dataType => - logger.debug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") + logDebug(s"Resolving mismatched union input ${l.dataType}, ${r.dataType}") findTightestCommonType(l.dataType, r.dataType).map { widestType => val newLeft = if (l.dataType == widestType) l else Alias(Cast(l, widestType), l.name)() @@ -170,7 +170,7 @@ trait HiveTypeCoercion { val newLeft = if (castedLeft.map(_.dataType) != left.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union $castedLeft ${left.output}") + logDebug(s"Widening numeric types in union $castedLeft ${left.output}") Project(castedLeft, left) } else { left @@ -178,7 +178,7 @@ trait HiveTypeCoercion { val newRight = if (castedRight.map(_.dataType) != right.output.map(_.dataType)) { - logger.debug(s"Widening numeric types in union $castedRight ${right.output}") + logDebug(s"Widening numeric types in union $castedRight ${right.output}") Project(castedRight, right) } else { right diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index f38f99569f207..0913f15888780 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 4211998f7511a..094ff14552283 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import com.typesafe.scalalogging.slf4j.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{StringType, NumericType} @@ -92,7 +92,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit } new $orderingName() """ - logger.debug(s"Generated Ordering: $code") + logDebug(s"Generated Ordering: $code") toolBox.eval(code).asInstanceOf[Ordering[Row]] } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala index ca9642954eb27..bdd07bbeb2230 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -25,5 +25,4 @@ package object catalyst { */ protected[catalyst] object ScalaReflectionLock - protected[catalyst] type Logging = com.typesafe.scalalogging.slf4j.Logging } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 781ba489b44c6..5839c9f7c43ef 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.planning -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index bc763a4e06e67..90923fe31a063 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -184,7 +184,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { case join @ Join(left, right, joinType, condition) => - logger.debug(s"Considering join on: $condition") + logDebug(s"Considering join on: $condition") // Find equi-join predicates that can be evaluated before the join, and thus can be used // as join keys. val (joinPredicates, otherPredicates) = @@ -202,7 +202,7 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { val rightKeys = joinKeys.map(_._2) if (joinKeys.nonEmpty) { - logger.debug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") + logDebug(s"leftKeys:${leftKeys} | rightKeys:${rightKeys}") Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) } else { None diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala index f8960b3fe7a17..03414b2301e81 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 6aa407c836aec..d192b151ac1c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.rules -import org.apache.spark.sql.catalyst.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.sideBySide @@ -60,7 +60,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { case (plan, rule) => val result = rule(plan) if (!result.fastEquals(plan)) { - logger.trace( + logTrace( s""" |=== Applying Rule ${rule.ruleName} === |${sideBySide(plan.treeString, result.treeString).mkString("\n")} @@ -73,26 +73,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { if (iteration > batch.strategy.maxIterations) { // Only log if this is a rule that is supposed to run more than once. if (iteration != 2) { - logger.info(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") + logInfo(s"Max iterations (${iteration - 1}) reached for batch ${batch.name}") } continue = false } if (curPlan.fastEquals(lastPlan)) { - logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") + logTrace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.") continue = false } lastPlan = curPlan } if (!batchStartPlan.fastEquals(curPlan)) { - logger.debug( + logDebug( s""" |=== Result of Batch ${batch.name} === |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")} """.stripMargin) } else { - logger.trace(s"Batch ${batch.name} has no effect.") + logTrace(s"Batch ${batch.name} has no effect.") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala index 9a28d035a10a3..d725a92c06f7b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst +import org.apache.spark.Logging + /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are * granted the following interface: @@ -31,8 +33,8 @@ package org.apache.spark.sql.catalyst *
  • debugging support - pretty printing, easy splicing of trees, etc.
  • * */ -package object trees { +package object trees extends Logging { // Since we want tree nodes to be lightweight, we create one logger for all treenode instances. - protected val logger = - com.typesafe.scalalogging.slf4j.Logger(org.slf4j.LoggerFactory.getLogger("catalyst.trees")) + protected override def logName = "catalyst.trees" + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index dad71079c29b9..00dd34aabc389 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.SparkStrategies import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} /** * :: AlphaComponent :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 4c6675c3c87bf..6ad12a0dcb64d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.columnar.compression import java.nio.{ByteBuffer, ByteOrder} -import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.Logging +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} @@ -101,7 +102,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] copyColumnHeader(rawBuffer, compressedBuffer) - logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + logInfo(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") encoder.compress(rawBuffer, compressedBuffer, columnType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 30712f03cab4c..77dc2ad733215 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -101,7 +101,7 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl !operator.requiredChildDistribution.zip(operator.children).map { case (required, child) => val valid = child.outputPartitioning.satisfies(required) - logger.debug( + logDebug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 70db1ebd3a3e1..a3d2a1c7a51f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.Logging +import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 0995a4eb6299f..f513eae9c2d13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -32,8 +32,6 @@ import org.apache.spark.annotation.DeveloperApi */ package object sql { - protected[sql] type Logging = com.typesafe.scalalogging.slf4j.Logging - /** * :: DeveloperApi :: * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 829342215e691..75f653f3280bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -22,7 +22,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -166,7 +166,7 @@ class ColumnTypeSuite extends FunSuite with Logging { buffer.rewind() seq.foreach { expected => - logger.info("buffer = " + buffer + ", expected = " + expected) + logInfo("buffer = " + buffer + ", expected = " + expected) val extracted = columnType.extract(buffer) assert( expected === extracted, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index ddbc2a79fb512..08d3f983d9e71 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ @@ -40,7 +40,7 @@ private[hive] object HiveThriftServer2 extends Logging { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") if (!optionsProcessor.process(args)) { - logger.warn("Error starting HiveThriftServer2 with given arguments") + logWarning("Error starting HiveThriftServer2 with given arguments") System.exit(-1) } @@ -49,12 +49,12 @@ private[hive] object HiveThriftServer2 extends Logging { // Set all properties specified via command line. val hiveConf: HiveConf = ss.getConf hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logger.debug(s"HiveConf var: $k=$v") + logDebug(s"HiveConf var: $k=$v") } SessionState.start(ss) - logger.info("Starting SparkContext") + logInfo("Starting SparkContext") SparkSQLEnv.init() SessionState.start(ss) @@ -70,10 +70,10 @@ private[hive] object HiveThriftServer2 extends Logging { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) server.init(hiveConf) server.start() - logger.info("HiveThriftServer2 started") + logInfo("HiveThriftServer2 started") } catch { case e: Exception => - logger.error("Error starting HiveThriftServer2", e) + logError("Error starting HiveThriftServer2", e) System.exit(-1) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index cb17d7ce58ea0..4d0c506c5a397 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket -import org.apache.spark.sql.Logging +import org.apache.spark.Logging private[hive] object SparkSQLCLIDriver { private var prompt = "spark-sql" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index a56b19a4bcda0..d362d599d08ca 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) @@ -40,7 +40,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo private def getResultSetSchema(query: context.QueryExecution): Schema = { val analyzed = query.analyzed - logger.debug(s"Result Schema: ${analyzed.output}") + logDebug(s"Result Schema: ${analyzed.output}") if (analyzed.output.size == 0) { new Schema(new FieldSchema("Response code", "string", "") :: Nil, null) } else { @@ -61,7 +61,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo new CommandProcessorResponse(0) } catch { case cause: Throwable => - logger.error(s"Failed in [$command]", cause) + logError(s"Failed in [$command]", cause) new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 451c3bd7b9352..582264eb59f83 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -20,13 +20,13 @@ package org.apache.spark.sql.hive.thriftserver import org.apache.hadoop.hive.ql.session.SessionState import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.hive.HiveContext import org.apache.spark.{SparkConf, SparkContext} /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { - logger.debug("Initializing SparkSQLEnv") + logDebug("Initializing SparkSQLEnv") var hiveContext: HiveContext = _ var sparkContext: SparkContext = _ @@ -47,7 +47,7 @@ private[hive] object SparkSQLEnv extends Logging { /** Cleans up and shuts down the Spark SQL environments. */ def stop() { - logger.debug("Shutting down Spark SQL Environment") + logDebug("Shutting down Spark SQL Environment") // Stop the SparkContext if (SparkSQLEnv.sparkContext != null) { sparkContext.stop() diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index a4e1f3e762e89..d4dadfd21d13f 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -30,10 +30,11 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{Logging, SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -55,7 +56,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. - logger.debug("CLOSING") + logDebug("CLOSING") } def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { @@ -112,7 +113,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def getResultSetSchema: TableSchema = { - logger.warn(s"Result Schema: ${result.queryExecution.analyzed.output}") + logWarning(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { new TableSchema(new FieldSchema("Result", "string", "") :: Nil) } else { @@ -124,11 +125,11 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def run(): Unit = { - logger.info(s"Running query '$statement'") + logInfo(s"Running query '$statement'") setState(OperationState.RUNNING) try { result = hiveContext.hql(statement) - logger.debug(result.queryExecution.toString()) + logDebug(result.queryExecution.toString()) val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) iter = result.queryExecution.toRdd.toLocalIterator @@ -138,7 +139,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => - logger.error("Error executing query:",e) + logError("Error executing query:",e) throw new HiveSQLException(e.toString) } setState(OperationState.FINISHED) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index fe3403b3292ec..b7b7c9957ac34 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -27,7 +27,7 @@ import java.sql.{Connection, DriverManager, Statement} import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 7e3b8727bebed..2c7270d9f83a9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -207,7 +207,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } catch { case e: Exception => - logger.error( + logError( s""" |====================== |HIVE FAILURE OUTPUT 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 fa4e78439c26c..df3604439e483 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 @@ -28,7 +28,8 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.{SQLContext, Logging} +import org.apache.spark.Logging +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index c50e8c4b5c5d3..728452a25a00e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -148,7 +148,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { describedTables ++ logical.collect { case UnresolvedRelation(databaseName, name, _) => name } val referencedTestTables = referencedTables.filter(testTables.contains) - logger.debug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") + logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(loadTestTable) // Proceed with analysis. analyzer(logical) @@ -273,7 +273,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. loadedTables += name - logger.info(s"Loading test table $name") + logInfo(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) @@ -312,7 +312,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadedTables.clear() catalog.client.getAllTables("default").foreach { t => - logger.debug(s"Deleting table $t") + logDebug(s"Deleting table $t") val table = catalog.client.getTable("default", t) catalog.client.getIndexes("default", t, 255).foreach { index => @@ -325,7 +325,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { } catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => - logger.debug(s"Dropping Database: $db") + logDebug(s"Dropping Database: $db") catalog.client.dropDatabase(db, true, false, true) } @@ -347,7 +347,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadTestTable("srcpart") } catch { case e: Exception => - logger.error(s"FATAL ERROR: Failed to reset TestDB state. $e") + logError(s"FATAL ERROR: Failed to reset TestDB state. $e") // At this point there is really no reason to continue, but the test framework traps exits. // So instead we just pause forever so that at least the developer can see where things // started to go wrong. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 7582b4743d404..d181921269b56 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ @@ -119,7 +119,7 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) (a: Any) => { - logger.debug( + logDebug( s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") // We must make sure that primitives get boxed java style. if (a == null) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 6c8fe4b196dea..83cfbc6b4a002 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -21,7 +21,7 @@ import java.io._ import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import org.apache.spark.sql.Logging +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand => LogicalNativeCommand} @@ -197,7 +197,7 @@ abstract class HiveComparisonTest // If test sharding is enable, skip tests that are not in the correct shard. shardInfo.foreach { case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return - case (shardId, _) => logger.debug(s"Shard $shardId includes test '$testCaseName'") + case (shardId, _) => logDebug(s"Shard $shardId includes test '$testCaseName'") } // Skip tests found in directories specified by user. @@ -213,13 +213,13 @@ abstract class HiveComparisonTest .map(new File(_, testCaseName)) .filter(_.exists) if (runOnlyDirectories.nonEmpty && runIndicators.isEmpty) { - logger.debug( + logDebug( s"Skipping test '$testCaseName' not found in ${runOnlyDirectories.map(_.getCanonicalPath)}") return } test(testCaseName) { - logger.debug(s"=== HIVE TEST: $testCaseName ===") + logDebug(s"=== HIVE TEST: $testCaseName ===") // Clear old output for this testcase. outputDirectories.map(new File(_, testCaseName)).filter(_.exists()).foreach(_.delete()) @@ -235,7 +235,7 @@ abstract class HiveComparisonTest .filterNot(_ contains "hive.outerjoin.supports.filters") if (allQueries != queryList) - logger.warn(s"Simplifications made on unsupported operations for test $testCaseName") + logWarning(s"Simplifications made on unsupported operations for test $testCaseName") lazy val consoleTestCase = { val quotes = "\"\"\"" @@ -257,11 +257,11 @@ abstract class HiveComparisonTest } val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => - logger.debug(s"Looking for cached answer file $cachedAnswerFile.") + logDebug(s"Looking for cached answer file $cachedAnswerFile.") if (cachedAnswerFile.exists) { Some(fileToString(cachedAnswerFile)) } else { - logger.debug(s"File $cachedAnswerFile not found") + logDebug(s"File $cachedAnswerFile not found") None } }.map { @@ -272,7 +272,7 @@ abstract class HiveComparisonTest val hiveResults: Seq[Seq[String]] = if (hiveCachedResults.size == queryList.size) { - logger.info(s"Using answer cache for test: $testCaseName") + logInfo(s"Using answer cache for test: $testCaseName") hiveCachedResults } else { @@ -287,7 +287,7 @@ abstract class HiveComparisonTest if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) sys.error("hive exec hooks not supported for tests.") - logger.warn(s"Running query ${i+1}/${queryList.size} with hive.") + logWarning(s"Running query ${i+1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. val answer = hiveQuery.analyzed match { case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. @@ -351,7 +351,7 @@ abstract class HiveComparisonTest val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") if (recomputeCache) { - logger.warn(s"Clearing cache files for failed test $testCaseName") + logWarning(s"Clearing cache files for failed test $testCaseName") hiveCacheFiles.foreach(_.delete()) } @@ -380,7 +380,7 @@ abstract class HiveComparisonTest TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - logger.error(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") + logError(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") // The testing setup traps exits so wait here for a long time so the developer can see when things started // to go wrong. Thread.sleep(1000000) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 50ab71a9003d3..02518d516261b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -53,7 +53,7 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { testCases.sorted.foreach { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { - logger.debug(s"Blacklisted test skipped $testCaseName") + logDebug(s"Blacklisted test skipped $testCaseName") } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) From 158ad0bba9382fd494b4789b5628a9cec00cfa19 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 16:33:48 -0700 Subject: [PATCH 0219/1492] [SPARK-2097][SQL] UDF Support This patch adds the ability to register lambda functions written in Python, Java or Scala as UDFs for use in SQL or HiveQL. Scala: ```scala registerFunction("strLenScala", (_: String).length) sql("SELECT strLenScala('test')") ``` Python: ```python sqlCtx.registerFunction("strLenPython", lambda x: len(x), IntegerType()) sqlCtx.sql("SELECT strLenPython('test')") ``` Java: ```java sqlContext.registerFunction("stringLengthJava", new UDF1() { Override public Integer call(String str) throws Exception { return str.length(); } }, DataType.IntegerType); sqlContext.sql("SELECT stringLengthJava('test')"); ``` Author: Michael Armbrust Closes #1063 from marmbrus/udfs and squashes the following commits: 9eda0fe [Michael Armbrust] newline 747c05e [Michael Armbrust] Add some scala UDF tests. d92727d [Michael Armbrust] Merge remote-tracking branch 'apache/master' into udfs 005d684 [Michael Armbrust] Fix naming and formatting. d14dac8 [Michael Armbrust] Fix last line of autogened java files. 8135c48 [Michael Armbrust] Move UDF unit tests to pyspark. 40b0ffd [Michael Armbrust] Merge remote-tracking branch 'apache/master' into udfs 6a36890 [Michael Armbrust] Switch logging so that SQLContext can be serializable. 7a83101 [Michael Armbrust] Drop toString 795fd15 [Michael Armbrust] Try to avoid capturing SQLContext. e54fb45 [Michael Armbrust] Docs and tests. 437cbe3 [Michael Armbrust] Update use of dataTypes, fix some python tests, address review comments. 01517d6 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udfs 8e6c932 [Michael Armbrust] WIP 3f96a52 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udfs 6237c8d [Michael Armbrust] WIP 2766f0b [Michael Armbrust] Move udfs support to SQL from hive. Add support for Java UDFs. 0f7d50c [Michael Armbrust] Draft of native Spark SQL UDFs for Scala and Python. --- python/pyspark/sql.py | 39 ++- .../catalyst/analysis/FunctionRegistry.scala | 32 ++ .../sql/catalyst/expressions/ScalaUdf.scala | 307 ++++++++++++++++++ .../org/apache/spark/sql/api/java/UDF1.java | 32 ++ .../org/apache/spark/sql/api/java/UDF10.java | 32 ++ .../org/apache/spark/sql/api/java/UDF11.java | 32 ++ .../org/apache/spark/sql/api/java/UDF12.java | 32 ++ .../org/apache/spark/sql/api/java/UDF13.java | 32 ++ .../org/apache/spark/sql/api/java/UDF14.java | 32 ++ .../org/apache/spark/sql/api/java/UDF15.java | 32 ++ .../org/apache/spark/sql/api/java/UDF16.java | 32 ++ .../org/apache/spark/sql/api/java/UDF17.java | 32 ++ .../org/apache/spark/sql/api/java/UDF18.java | 32 ++ .../org/apache/spark/sql/api/java/UDF19.java | 32 ++ .../org/apache/spark/sql/api/java/UDF2.java | 32 ++ .../org/apache/spark/sql/api/java/UDF20.java | 32 ++ .../org/apache/spark/sql/api/java/UDF21.java | 32 ++ .../org/apache/spark/sql/api/java/UDF22.java | 32 ++ .../org/apache/spark/sql/api/java/UDF3.java | 32 ++ .../org/apache/spark/sql/api/java/UDF4.java | 32 ++ .../org/apache/spark/sql/api/java/UDF5.java | 32 ++ .../org/apache/spark/sql/api/java/UDF6.java | 32 ++ .../org/apache/spark/sql/api/java/UDF7.java | 32 ++ .../org/apache/spark/sql/api/java/UDF8.java | 32 ++ .../org/apache/spark/sql/api/java/UDF9.java | 32 ++ .../org/apache/spark/sql/SQLContext.scala | 11 +- .../apache/spark/sql/UdfRegistration.scala | 196 +++++++++++ .../spark/sql/api/java/JavaSQLContext.scala | 5 +- .../spark/sql/api/java/UDFRegistration.scala | 252 ++++++++++++++ .../spark/sql/execution/SparkStrategies.scala | 2 + .../spark/sql/execution/pythonUdfs.scala | 177 ++++++++++ .../spark/sql/api/java/JavaAPISuite.java | 90 +++++ .../apache/spark/sql/InsertIntoSuite.scala | 2 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 36 ++ .../apache/spark/sql/hive/HiveContext.scala | 13 +- .../org/apache/spark/sql/hive/TestHive.scala | 4 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 6 +- .../org/apache/spark/sql/QueryTest.scala | 4 +- 38 files changed, 1861 insertions(+), 19 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala create mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index f840475ffaf70..e7c35ac1ffe02 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -28,9 +28,13 @@ from operator import itemgetter from pyspark.rdd import RDD, PipelinedRDD -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer + +from itertools import chain, ifilter, imap from py4j.protocol import Py4JError +from py4j.java_collections import ListConverter, MapConverter + __all__ = [ "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", @@ -932,6 +936,39 @@ def _ssql_ctx(self): self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext + def registerFunction(self, name, f, returnType=StringType()): + """Registers a lambda function as a UDF so it can be used in SQL statements. + + In addition to a name and the function itself, the return type can be optionally specified. + When the return type is not given it default to a string and conversion will automatically + be done. For any other return type, the produced object must match the specified type. + + >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) + >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + [Row(c0=u'4')] + >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + [Row(c0=4)] + >>> sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + >>> sqlCtx.sql("SELECT twoArgs('test', 1)").collect() + [Row(c0=5)] + """ + func = lambda _, it: imap(lambda x: f(*x), it) + command = (func, + BatchedSerializer(PickleSerializer(), 1024), + BatchedSerializer(PickleSerializer(), 1024)) + env = MapConverter().convert(self._sc.environment, + self._sc._gateway._gateway_client) + includes = ListConverter().convert(self._sc._python_includes, + self._sc._gateway._gateway_client) + self._ssql_ctx.registerPython(name, + bytearray(CloudPickleSerializer().dumps(command)), + env, + includes, + self._sc.pythonExec, + self._sc._javaAccumulator, + str(returnType)) + def inferSchema(self, rdd): """Infer and apply a schema to an RDD of L{Row}s. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c0255701b7ba5..760c49fbca4a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -18,17 +18,49 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Expression +import scala.collection.mutable /** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ trait FunctionRegistry { + type FunctionBuilder = Seq[Expression] => Expression + + def registerFunction(name: String, builder: FunctionBuilder): Unit + def lookupFunction(name: String, children: Seq[Expression]): Expression } +trait OverrideFunctionRegistry extends FunctionRegistry { + + val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() + + def registerFunction(name: String, builder: FunctionBuilder) = { + functionBuilders.put(name, builder) + } + + abstract override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + functionBuilders.get(name).map(_(children)).getOrElse(super.lookupFunction(name,children)) + } +} + +class SimpleFunctionRegistry extends FunctionRegistry { + val functionBuilders = new mutable.HashMap[String, FunctionBuilder]() + + def registerFunction(name: String, builder: FunctionBuilder) = { + functionBuilders.put(name, builder) + } + + override def lookupFunction(name: String, children: Seq[Expression]): Expression = { + functionBuilders(name)(children) + } +} + /** * A trivial catalog that returns an error when a function is requested. Used for testing when all * functions are already filled in and the analyser needs only to resolve attribute references. */ object EmptyFunctionRegistry extends FunctionRegistry { + def registerFunction(name: String, builder: FunctionBuilder) = ??? + def lookupFunction(name: String, children: Seq[Expression]): Expression = { throw new UnsupportedOperationException } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index acddf5e9c7004..95633dd0c9870 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -27,6 +27,22 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi def references = children.flatMap(_.references).toSet def nullable = true + /** This method has been generated by this script + + (1 to 22).map { x => + val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) + val evals = (0 to x - 1).map(x => s"children($x).eval(input)").reduce(_ + ",\n " + _) + + s""" + case $x => + function.asInstanceOf[($anys) => Any]( + $evals) + """ + } + + */ + + // scalastyle:off override def eval(input: Row): Any = { children.size match { case 0 => function.asInstanceOf[() => Any]() @@ -35,6 +51,297 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi function.asInstanceOf[(Any, Any) => Any]( children(0).eval(input), children(1).eval(input)) + case 3 => + function.asInstanceOf[(Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input)) + case 4 => + function.asInstanceOf[(Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input)) + case 5 => + function.asInstanceOf[(Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input)) + case 6 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input)) + case 7 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input)) + case 8 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input)) + case 9 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input)) + case 10 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input)) + case 11 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input)) + case 12 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input)) + case 13 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input)) + case 14 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input)) + case 15 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input)) + case 16 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input)) + case 17 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input)) + case 18 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input)) + case 19 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input)) + case 20 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input)) + case 21 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input), + children(20).eval(input)) + case 22 => + function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( + children(0).eval(input), + children(1).eval(input), + children(2).eval(input), + children(3).eval(input), + children(4).eval(input), + children(5).eval(input), + children(6).eval(input), + children(7).eval(input), + children(8).eval(input), + children(9).eval(input), + children(10).eval(input), + children(11).eval(input), + children(12).eval(input), + children(13).eval(input), + children(14).eval(input), + children(15).eval(input), + children(16).eval(input), + children(17).eval(input), + children(18).eval(input), + children(19).eval(input), + children(20).eval(input), + children(21).eval(input)) } + // scalastyle:on } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java new file mode 100644 index 0000000000000..ef959e35e1027 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 1 arguments. + */ +public interface UDF1 extends Serializable { + public R call(T1 t1) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java new file mode 100644 index 0000000000000..96ab3a96c3d5e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 10 arguments. + */ +public interface UDF10 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java new file mode 100644 index 0000000000000..58ae8edd6d817 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 11 arguments. + */ +public interface UDF11 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java new file mode 100644 index 0000000000000..d9da0f6eddd94 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 12 arguments. + */ +public interface UDF12 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java new file mode 100644 index 0000000000000..095fc1a8076b5 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 13 arguments. + */ +public interface UDF13 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java new file mode 100644 index 0000000000000..eb27eaa180086 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 14 arguments. + */ +public interface UDF14 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java new file mode 100644 index 0000000000000..1fbcff56332b6 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 15 arguments. + */ +public interface UDF15 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java new file mode 100644 index 0000000000000..1133561787a69 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 16 arguments. + */ +public interface UDF16 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java new file mode 100644 index 0000000000000..dfae7922c9b63 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 17 arguments. + */ +public interface UDF17 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java new file mode 100644 index 0000000000000..e9d1c6d52d4ea --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 18 arguments. + */ +public interface UDF18 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java new file mode 100644 index 0000000000000..46b9d2d3c9457 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 19 arguments. + */ +public interface UDF19 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java new file mode 100644 index 0000000000000..cd3fde8da419e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 2 arguments. + */ +public interface UDF2 extends Serializable { + public R call(T1 t1, T2 t2) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java new file mode 100644 index 0000000000000..113d3d26be4a7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 20 arguments. + */ +public interface UDF20 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java new file mode 100644 index 0000000000000..74118f2cf8da7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 21 arguments. + */ +public interface UDF21 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java new file mode 100644 index 0000000000000..0e7cc40be45ec --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 22 arguments. + */ +public interface UDF22 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java new file mode 100644 index 0000000000000..6a880f16be47a --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 3 arguments. + */ +public interface UDF3 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java new file mode 100644 index 0000000000000..fcad2febb18e6 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 4 arguments. + */ +public interface UDF4 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java new file mode 100644 index 0000000000000..ce0cef43a2144 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 5 arguments. + */ +public interface UDF5 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java new file mode 100644 index 0000000000000..f56b806684e61 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 6 arguments. + */ +public interface UDF6 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java new file mode 100644 index 0000000000000..25bd6d3241bd4 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 7 arguments. + */ +public interface UDF7 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java new file mode 100644 index 0000000000000..a3b7ac5f94ce7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 8 arguments. + */ +public interface UDF8 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java new file mode 100644 index 0000000000000..205e72a1522fc --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +// ************************************************** +// THIS FILE IS AUTOGENERATED BY CODE IN +// org.apache.spark.sql.api.java.FunctionRegistration +// ************************************************** + +/** + * A Spark SQL UDF that has 9 arguments. + */ +public interface UDF9 extends Serializable { + public R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 00dd34aabc389..33931e5d996f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -48,18 +48,23 @@ import org.apache.spark.{Logging, SparkContext} */ @AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) - extends Logging + extends org.apache.spark.Logging with SQLConf with ExpressionConversions + with UDFRegistration with Serializable { self => @transient protected[sql] lazy val catalog: Catalog = new SimpleCatalog(true) + + @transient + protected[sql] lazy val functionRegistry: FunctionRegistry = new SimpleFunctionRegistry + @transient protected[sql] lazy val analyzer: Analyzer = - new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) + new Analyzer(catalog, functionRegistry, caseSensitive = true) @transient protected[sql] val optimizer = Optimizer @transient @@ -379,7 +384,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected abstract class QueryExecution { def logical: LogicalPlan - lazy val analyzed = analyzer(logical) + lazy val analyzed = ExtractPythonUdfs(analyzer(logical)) lazy val optimizedPlan = optimizer(analyzed) // TODO: Don't just pick the first one... lazy val sparkPlan = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala new file mode 100644 index 0000000000000..0b48e9e659faa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.util.{List => JList, Map => JMap} + +import org.apache.spark.Accumulator +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} +import org.apache.spark.sql.execution.PythonUDF + +import scala.reflect.runtime.universe.{TypeTag, typeTag} + +/** + * Functions for registering scala lambda functions as UDFs in a SQLContext. + */ +protected[sql] trait UDFRegistration { + self: SQLContext => + + private[spark] def registerPython( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + accumulator: Accumulator[JList[Array[Byte]]], + stringDataType: String): Unit = { + log.debug( + s""" + | Registering new PythonUDF: + | name: $name + | command: ${command.toSeq} + | envVars: $envVars + | pythonIncludes: $pythonIncludes + | pythonExec: $pythonExec + | dataType: $stringDataType + """.stripMargin) + + + val dataType = parseDataType(stringDataType) + + def builder(e: Seq[Expression]) = + PythonUDF( + name, + command, + envVars, + pythonIncludes, + pythonExec, + accumulator, + dataType, + e) + + functionRegistry.registerFunction(name, builder) + } + + /** registerFunction 1-22 were generated by this script + + (1 to 22).map { x => + val types = (1 to x).map(x => "_").reduce(_ + ", " + _) + s""" + def registerFunction[T: TypeTag](name: String, func: Function$x[$types, T]): Unit = { + def builder(e: Seq[Expression]) = + ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + """ + } + */ + + // scalastyle:off + def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function2[_, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function3[_, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function4[_, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function5[_, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function6[_, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function7[_, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function8[_, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function9[_, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function10[_, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + + def registerFunction[T: TypeTag](name: String, func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + functionRegistry.registerFunction(name, builder) + } + // scalastyle:on +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 809dd038f94aa..ae45193ed15d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -28,14 +28,13 @@ import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} -import org.apache.spark.sql.types.util.DataTypeConversions -import DataTypeConversions.asScalaDataType; +import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaSQLContext(val sqlContext: SQLContext) { +class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala new file mode 100644 index 0000000000000..158f26e3d445f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDFRegistration.scala @@ -0,0 +1,252 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.api.java + +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} +import org.apache.spark.sql.types.util.DataTypeConversions._ + +/** + * A collection of functions that allow Java users to register UDFs. In order to handle functions + * of varying airities with minimal boilerplate for our users, we generate classes and functions + * for each airity up to 22. The code for this generation can be found in comments in this trait. + */ +private[java] trait UDFRegistration { + self: JavaSQLContext => + + /* The following functions and required interfaces are generated with these code fragments: + + (1 to 22).foreach { i => + val extTypeArgs = (1 to i).map(_ => "_").mkString(", ") + val anyTypeArgs = (1 to i).map(_ => "Any").mkString(", ") + val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs, Any]]" + val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") + println(s""" + |def registerFunction( + | name: String, f: UDF$i[$extTypeArgs, _], @transient dataType: DataType) = { + | val scalaType = asScalaDataType(dataType) + | sqlContext.functionRegistry.registerFunction( + | name, + | (e: Seq[Expression]) => ScalaUdf(f$anyCast.call($anyParams), scalaType, e)) + |} + """.stripMargin) + } + + import java.io.File + import org.apache.spark.sql.catalyst.util.stringToFile + val directory = new File("sql/core/src/main/java/org/apache/spark/sql/api/java/") + (1 to 22).foreach { i => + val typeArgs = (1 to i).map(i => s"T$i").mkString(", ") + val args = (1 to i).map(i => s"T$i t$i").mkString(", ") + + val contents = + s"""/* + | * Licensed to the Apache Software Foundation (ASF) under one or more + | * contributor license agreements. See the NOTICE file distributed with + | * this work for additional information regarding copyright ownership. + | * The ASF licenses this file to You under the Apache License, Version 2.0 + | * (the "License"); you may not use this file except in compliance with + | * the License. You may obtain a copy of the License at + | * + | * http://www.apache.org/licenses/LICENSE-2.0 + | * + | * Unless required by applicable law or agreed to in writing, software + | * distributed under the License is distributed on an "AS IS" BASIS, + | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + | * See the License for the specific language governing permissions and + | * limitations under the License. + | */ + | + |package org.apache.spark.sql.api.java; + | + |import java.io.Serializable; + | + |// ************************************************** + |// THIS FILE IS AUTOGENERATED BY CODE IN + |// org.apache.spark.sql.api.java.FunctionRegistration + |// ************************************************** + | + |/** + | * A Spark SQL UDF that has $i arguments. + | */ + |public interface UDF$i<$typeArgs, R> extends Serializable { + | public R call($args) throws Exception; + |} + |""".stripMargin + + stringToFile(new File(directory, s"UDF$i.java"), contents) + } + + */ + + // scalastyle:off + def registerFunction(name: String, f: UDF1[_, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF1[Any, Any]].call(_: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF2[_, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF3[_, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF4[_, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF5[_, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF6[_, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF7[_, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + def registerFunction(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], dataType: DataType) = { + val scalaType = asScalaDataType(dataType) + sqlContext.functionRegistry.registerFunction( + name, + (e: Seq[Expression]) => ScalaUdf(f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any), scalaType, e)) + } + + // scalastyle:on +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 8bec015c7b465..f0c958fdb537f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -286,6 +286,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.ExistingRdd(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil + case e @ EvaluatePython(udf, child) => + BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case SparkLogicalPlan(existingPlan) => existingPlan :: Nil case _ => Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala new file mode 100644 index 0000000000000..b92091b560b1c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -0,0 +1,177 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.execution + +import java.util.{List => JList, Map => JMap} + +import net.razorvine.pickle.{Pickler, Unpickler} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.python.PythonRDD +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.{Accumulator, Logging => SparkLogging} + +import scala.collection.JavaConversions._ + +/** + * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. + */ +private[spark] case class PythonUDF( + name: String, + command: Array[Byte], + envVars: JMap[String, String], + pythonIncludes: JList[String], + pythonExec: String, + accumulator: Accumulator[JList[Array[Byte]]], + dataType: DataType, + children: Seq[Expression]) extends Expression with SparkLogging { + + override def toString = s"PythonUDF#$name(${children.mkString(",")})" + + def nullable: Boolean = true + def references: Set[Attribute] = children.flatMap(_.references).toSet + + override def eval(input: Row) = sys.error("PythonUDFs can not be directly evaluated.") +} + +/** + * Extracts PythonUDFs from operators, rewriting the query plan so that the UDF can be evaluated + * alone in a batch. + * + * This has the limitation that the input to the Python UDF is not allowed include attributes from + * multiple child operators. + */ +private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan) = plan transform { + // Skip EvaluatePython nodes. + case p: EvaluatePython => p + + case l: LogicalPlan => + // Extract any PythonUDFs from the current operator. + val udfs = l.expressions.flatMap(_.collect { case udf: PythonUDF => udf}) + if (udfs.isEmpty) { + // If there aren't any, we are done. + l + } else { + // Pick the UDF we are going to evaluate (TODO: Support evaluating multiple UDFs at a time) + // If there is more than one, we will add another evaluation operator in a subsequent pass. + val udf = udfs.head + + var evaluation: EvaluatePython = null + + // Rewrite the child that has the input required for the UDF + val newChildren = l.children.map { child => + // Check to make sure that the UDF can be evaluated with only the input of this child. + // Other cases are disallowed as they are ambiguous or would require a cartisian product. + if (udf.references.subsetOf(child.outputSet)) { + evaluation = EvaluatePython(udf, child) + evaluation + } else if (udf.references.intersect(child.outputSet).nonEmpty) { + sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") + } else { + child + } + } + + assert(evaluation != null, "Unable to evaluate PythonUDF. Missing input attributes.") + + // Trim away the new UDF value if it was only used for filtering or something. + logical.Project( + l.output, + l.transformExpressions { + case p: PythonUDF if p.id == udf.id => evaluation.resultAttribute + }.withNewChildren(newChildren)) + } + } +} + +/** + * :: DeveloperApi :: + * Evaluates a [[PythonUDF]], appending the result to the end of the input tuple. + */ +@DeveloperApi +case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { + val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() + + def references = Set.empty + def output = child.output :+ resultAttribute +} + +/** + * :: DeveloperApi :: + * Uses PythonRDD to evaluate a [[PythonUDF]], one partition of tuples at a time. The input + * data is cached and zipped with the result of the udf evaluation. + */ +@DeveloperApi +case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: SparkPlan) + extends SparkPlan { + def children = child :: Nil + + def execute() = { + // TODO: Clean up after ourselves? + val childResults = child.execute().map(_.copy()).cache() + + val parent = childResults.mapPartitions { iter => + val pickle = new Pickler + val currentRow = newMutableProjection(udf.children, child.output)() + iter.grouped(1000).map { inputRows => + val toBePickled = inputRows.map(currentRow(_).toArray).toArray + pickle.dumps(toBePickled) + } + } + + val pyRDD = new PythonRDD( + parent, + udf.command, + udf.envVars, + udf.pythonIncludes, + false, + udf.pythonExec, + Seq[Broadcast[Array[Byte]]](), + udf.accumulator + ).mapPartitions { iter => + val pickle = new Unpickler + iter.flatMap { pickedResult => + val unpickledBatch = pickle.loads(pickedResult) + unpickledBatch.asInstanceOf[java.util.ArrayList[Any]] + } + }.mapPartitions { iter => + val row = new GenericMutableRow(1) + iter.map { result => + row(0) = udf.dataType match { + case StringType => result.toString + case other => result + } + row: Row + } + } + + childResults.zip(pyRDD).mapPartitions { iter => + val joinedRow = new JoinedRow() + iter.map { + case (row, udfResult) => + joinedRow(row, udfResult) + } + } + } +} diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java new file mode 100644 index 0000000000000..a9a11285def54 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaAPISuite.java @@ -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.sql.api.java; + +import java.io.Serializable; + +import org.apache.spark.sql.api.java.UDF1; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Suite; +import org.junit.runner.RunWith; + +import org.apache.spark.api.java.JavaSparkContext; + +// The test suite itself is Serializable so that anonymous Function implementations can be +// serialized, as an alternative to converting these anonymous classes to static inner classes; +// see http://stackoverflow.com/questions/758570/. +public class JavaAPISuite implements Serializable { + private transient JavaSparkContext sc; + private transient JavaSQLContext sqlContext; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaAPISuite"); + sqlContext = new JavaSQLContext(sc); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @SuppressWarnings("unchecked") + @Test + public void udf1Test() { + // With Java 8 lambdas: + // sqlContext.registerFunction( + // "stringLengthTest", (String str) -> str.length(), DataType.IntegerType); + + sqlContext.registerFunction("stringLengthTest", new UDF1() { + @Override + public Integer call(String str) throws Exception { + return str.length(); + } + }, DataType.IntegerType); + + // TODO: Why do we need this cast? + Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test')").first(); + assert(result.getInt(0) == 4); + } + + @SuppressWarnings("unchecked") + @Test + public void udf2Test() { + // With Java 8 lambdas: + // sqlContext.registerFunction( + // "stringLengthTest", + // (String str1, String str2) -> str1.length() + str2.length, + // DataType.IntegerType); + + sqlContext.registerFunction("stringLengthTest", new UDF2() { + @Override + public Integer call(String str1, String str2) throws Exception { + return str1.length() + str2.length(); + } + }, DataType.IntegerType); + + // TODO: Why do we need this cast? + Row result = (Row) sqlContext.sql("SELECT stringLengthTest('test', 'test2')").first(); + assert(result.getInt(0) == 9); + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala index 4f0b85f26254b..23a711d08c58b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.File +import _root_.java.io.File /* Implicits */ import org.apache.spark.sql.test.TestSQLContext._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala new file mode 100644 index 0000000000000..76aa9b0081d7e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.test._ + +/* Implicits */ +import TestSQLContext._ + +class UDFSuite extends QueryTest { + + test("Simple UDF") { + registerFunction("strLenScala", (_: String).length) + assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4) + } + + test("TwoArgument UDF") { + registerFunction("strLenScala", (_: String).length + (_:Int)) + assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 2c7270d9f83a9..3c70b3f0921a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -23,7 +23,7 @@ import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag +import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver @@ -35,8 +35,9 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{OverrideFunctionRegistry, Analyzer, OverrideCatalog} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand @@ -155,10 +156,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } } + // Note that HiveUDFs will be overridden by functions registered in this context. + override protected[sql] lazy val functionRegistry = + new HiveFunctionRegistry with OverrideFunctionRegistry + /* An analyzer that uses the Hive metastore. */ @transient override protected[sql] lazy val analyzer = - new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + new Analyzer(catalog, functionRegistry, caseSensitive = false) /** * Runs the specified SQL query using Hive. @@ -250,7 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected[sql] abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = - optimizer(catalog.PreInsertionCasts(catalog.CreateTables(analyzed))) + optimizer(ExtractPythonUdfs(catalog.PreInsertionCasts(catalog.CreateTables(analyzed)))) override lazy val toRdd: RDD[Row] = executedPlan.execute().map(_.copy()) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 728452a25a00e..c605e8adcfb0f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -297,8 +297,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { def reset() { try { // HACK: Hive is too noisy by default. - org.apache.log4j.LogManager.getCurrentLoggers.foreach { logger => - logger.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) + org.apache.log4j.LogManager.getCurrentLoggers.foreach { log => + log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } // It is important that we RESET first as broken hooks that might have been set could break diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index d181921269b56..179aac5cbd5cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -34,7 +34,8 @@ import org.apache.spark.util.Utils.getContextOrSparkClassLoader /* Implicit conversions */ import scala.collection.JavaConversions._ -private[hive] object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveInspectors { +private[hive] abstract class HiveFunctionRegistry + extends analysis.FunctionRegistry with HiveInspectors { def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) @@ -92,9 +93,8 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu } private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[Expression]) - extends HiveUdf { + extends HiveUdf with HiveInspectors { - import org.apache.spark.sql.hive.HiveFunctionRegistry._ type UDFType = UDF @transient diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 11d8b1f0a3d96..95921c3d7ae09 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -51,9 +51,9 @@ class QueryTest extends FunSuite { fail( s""" |Exception thrown while executing query: - |${rdd.logicalPlan} + |${rdd.queryExecution} |== Exception == - |$e + |${stackTraceToString(e)} """.stripMargin) } From 198df11f1a9f419f820f47eba0e9f2ab371a824b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 16:48:07 -0700 Subject: [PATCH 0220/1492] [SPARK-2785][SQL] Remove assertions that throw when users try unsupported Hive commands. Author: Michael Armbrust Closes #1742 from marmbrus/asserts and squashes the following commits: 5182d54 [Michael Armbrust] Remove assertions that throw when users try unsupported Hive commands. --- .../scala/org/apache/spark/sql/hive/HiveQl.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 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 3d2eb1eefaeda..bc2fefafd58c8 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 @@ -297,8 +297,11 @@ private[hive] object HiveQl { matches.headOption } - assert(remainingNodes.isEmpty, - s"Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}") + if (remainingNodes.nonEmpty) { + sys.error( + s"""Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}. + |You are likely trying to use an unsupported Hive feature."""".stripMargin) + } clauses } @@ -748,7 +751,10 @@ private[hive] object HiveQl { case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => - assert(other.size <= 1, s"Unhandled join child $other") + if (!(other.size <= 1)) { + sys.error(s"Unsupported join operation: $other") + } + val joinType = joinToken match { case "TOK_JOIN" => Inner case "TOK_RIGHTOUTERJOIN" => RightOuter @@ -756,7 +762,6 @@ private[hive] object HiveQl { case "TOK_FULLOUTERJOIN" => FullOuter case "TOK_LEFTSEMIJOIN" => LeftSemi } - assert(other.size <= 1, "Unhandled join clauses.") Join(nodeToRelation(relation1), nodeToRelation(relation2), joinType, From 866cf1f822cfda22294054be026ef2d96307eb75 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 2 Aug 2014 17:12:49 -0700 Subject: [PATCH 0221/1492] [SPARK-2729][SQL] Added test case for SPARK-2729 This is a follow up of #1636. Author: Cheng Lian Closes #1738 from liancheng/test-for-spark-2729 and squashes the following commits: b13692a [Cheng Lian] Added test case for SPARK-2729 --- .../test/scala/org/apache/spark/sql/TestData.scala | 12 ++++++++++-- .../sql/columnar/InMemoryColumnarQuerySuite.scala | 12 ++++++++++++ 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 58cee21e8ad4c..088e6e3c843aa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql +import java.sql.Timestamp + import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.test._ /* Implicits */ -import TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext._ case class TestData(key: Int, value: String) @@ -40,7 +42,7 @@ object TestData { LargeAndSmallInts(2147483646, 1) :: LargeAndSmallInts(3, 2) :: Nil) largeAndSmallInts.registerAsTable("largeAndSmallInts") - + case class TestData2(a: Int, b: Int) val testData2: SchemaRDD = TestSQLContext.sparkContext.parallelize( @@ -143,4 +145,10 @@ object TestData { "2, B2, false, null" :: "3, C3, true, null" :: "4, D4, true, 2147483644" :: Nil) + + case class TimestampField(time: Timestamp) + val timestamps = TestSQLContext.sparkContext.parallelize((1 to 3).map { i => + TimestampField(new Timestamp(i)) + }) + timestamps.registerAsTable("timestamps") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 86727b93f3659..b561b44ad7ee2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -73,4 +73,16 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM nullableRepeatedData"), nullableRepeatedData.collect().toSeq) } + + test("SPARK-2729 regression: timestamp data type") { + checkAnswer( + sql("SELECT time FROM timestamps"), + timestamps.collect().toSeq) + + TestSQLContext.cacheTable("timestamps") + + checkAnswer( + sql("SELECT time FROM timestamps"), + timestamps.collect().toSeq) + } } From d210022e96804e59e42ab902e53637e50884a9ab Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 2 Aug 2014 17:55:22 -0700 Subject: [PATCH 0222/1492] [SPARK-2797] [SQL] SchemaRDDs don't support unpersist() The cause is explained in https://issues.apache.org/jira/browse/SPARK-2797. Author: Yin Huai Closes #1745 from yhuai/SPARK-2797 and squashes the following commits: 7b1627d [Yin Huai] The unpersist method of the Scala RDD cannot be called without the input parameter (blocking) from PySpark. --- python/pyspark/sql.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index e7c35ac1ffe02..36e50e49c9a9c 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1589,9 +1589,9 @@ def persist(self, storageLevel): self._jschema_rdd.persist(javaStorageLevel) return self - def unpersist(self): + def unpersist(self, blocking=True): self.is_cached = False - self._jschema_rdd.unpersist() + self._jschema_rdd.unpersist(blocking) return self def checkpoint(self): From 1a8043739dc1d9435def6ea3c6341498ba52b708 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 2 Aug 2014 18:27:04 -0700 Subject: [PATCH 0223/1492] [SPARK-2739][SQL] Rename registerAsTable to registerTempTable There have been user complaints that the difference between `registerAsTable` and `saveAsTable` is too subtle. This PR addresses this by renaming `registerAsTable` to `registerTempTable`, which more clearly reflects what is happening. `registerAsTable` remains, but will cause a deprecation warning. Author: Michael Armbrust Closes #1743 from marmbrus/registerTempTable and squashes the following commits: d031348 [Michael Armbrust] Merge remote-tracking branch 'apache/master' into registerTempTable 4dff086 [Michael Armbrust] Fix .java files too 89a2f12 [Michael Armbrust] Merge remote-tracking branch 'apache/master' into registerTempTable 0b7b71e [Michael Armbrust] Rename registerAsTable to registerTempTable --- .../sbt_app_sql/src/main/scala/SqlApp.scala | 2 +- docs/sql-programming-guide.md | 18 ++++++------ .../spark/examples/sql/JavaSparkSQL.java | 8 +++--- .../spark/examples/sql/RDDRelation.scala | 4 +-- .../examples/sql/hive/HiveFromSpark.scala | 2 +- python/pyspark/sql.py | 12 +++++--- .../org/apache/spark/sql/SQLContext.scala | 4 +-- .../org/apache/spark/sql/SchemaRDD.scala | 2 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 5 +++- .../spark/sql/api/java/JavaSQLContext.scala | 2 +- .../sql/api/java/JavaApplySchemaSuite.java | 6 ++-- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../apache/spark/sql/InsertIntoSuite.scala | 4 +-- .../org/apache/spark/sql/JoinSuite.scala | 4 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 ++-- .../sql/ScalaReflectionRelationSuite.scala | 8 +++--- .../scala/org/apache/spark/sql/TestData.scala | 28 +++++++++---------- .../spark/sql/api/java/JavaSQLSuite.scala | 10 +++---- .../org/apache/spark/sql/json/JsonSuite.scala | 22 +++++++-------- .../spark/sql/parquet/ParquetQuerySuite.scala | 26 ++++++++--------- .../sql/hive/InsertIntoHiveTableSuite.scala | 2 +- .../sql/hive/api/java/JavaHiveQLSuite.scala | 4 +-- .../sql/hive/execution/HiveQuerySuite.scala | 6 ++-- .../hive/execution/HiveResolutionSuite.scala | 4 +-- .../spark/sql/parquet/HiveParquetSuite.scala | 8 +++--- 25 files changed, 103 insertions(+), 96 deletions(-) diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala index 50af90c213b5a..d888de929fdda 100644 --- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -38,7 +38,7 @@ object SparkSqlExample { import sqlContext._ val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) - people.registerAsTable("people") + people.registerTempTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() teenagerNames.foreach(println) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 7261badd411a9..0465468084cee 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -142,7 +142,7 @@ case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) -people.registerAsTable("people") +people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -210,7 +210,7 @@ JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").m // Apply a schema to an RDD of JavaBeans and register it as a table. JavaSchemaRDD schemaPeople = sqlContext.applySchema(people, Person.class); -schemaPeople.registerAsTable("people"); +schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -248,7 +248,7 @@ people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) # In future versions of PySpark we would like to add support for registering RDDs with other # datatypes as tables schemaPeople = sqlContext.inferSchema(people) -schemaPeople.registerAsTable("people") +schemaPeople.registerTempTable("people") # SQL can be run over SchemaRDDs that have been registered as a table. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -292,7 +292,7 @@ people.saveAsParquetFile("people.parquet") val parquetFile = sqlContext.parquetFile("people.parquet") //Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerAsTable("parquetFile") +parquetFile.registerTempTable("parquetFile") val teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} @@ -314,7 +314,7 @@ schemaPeople.saveAsParquetFile("people.parquet"); JavaSchemaRDD parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerAsTable("parquetFile"); +parquetFile.registerTempTable("parquetFile"); JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); List teenagerNames = teenagers.map(new Function() { public String call(Row row) { @@ -340,7 +340,7 @@ schemaPeople.saveAsParquetFile("people.parquet") parquetFile = sqlContext.parquetFile("people.parquet") # Parquet files can also be registered as tables and then used in SQL statements. -parquetFile.registerAsTable("parquetFile"); +parquetFile.registerTempTable("parquetFile"); teenagers = sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") teenNames = teenagers.map(lambda p: "Name: " + p.name) for teenName in teenNames.collect(): @@ -378,7 +378,7 @@ people.printSchema() // |-- name: StringType // Register this SchemaRDD as a table. -people.registerAsTable("people") +people.registerTempTable("people") // SQL statements can be run by using the sql methods provided by sqlContext. val teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") @@ -416,7 +416,7 @@ people.printSchema(); // |-- name: StringType // Register this JavaSchemaRDD as a table. -people.registerAsTable("people"); +people.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlContext. JavaSchemaRDD teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -455,7 +455,7 @@ people.printSchema() # |-- name: StringType # Register this SchemaRDD as a table. -people.registerAsTable("people") +people.registerTempTable("people") # SQL statements can be run by using the sql methods provided by sqlContext. teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index 607df3eddd550..898297dc658ba 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -74,7 +74,7 @@ public Person call(String line) throws Exception { // Apply a schema to an RDD of Java Beans and register it as a table. JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class); - schemaPeople.registerAsTable("people"); + schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -100,7 +100,7 @@ public String call(Row row) { JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. - parquetFile.registerAsTable("parquetFile"); + parquetFile.registerTempTable("parquetFile"); JavaSchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.map(new Function() { @@ -128,7 +128,7 @@ public String call(Row row) { // |-- name: StringType // Register this JavaSchemaRDD as a table. - peopleFromJsonFile.registerAsTable("people"); + peopleFromJsonFile.registerTempTable("people"); // SQL statements can be run by using the sql methods provided by sqlCtx. JavaSchemaRDD teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); @@ -158,7 +158,7 @@ public String call(Row row) { // | |-- state: StringType // |-- name: StringType - peopleFromJsonRDD.registerAsTable("people2"); + peopleFromJsonRDD.registerTempTable("people2"); JavaSchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.map(new Function() { diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index 63db688bfb8c0..d56d64c564200 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -36,7 +36,7 @@ object RDDRelation { val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) // Any RDD containing case classes can be registered as a table. The schema of the table is // automatically inferred using scala reflection. - rdd.registerAsTable("records") + rdd.registerTempTable("records") // Once tables have been registered, you can run SQL queries over them. println("Result of SELECT *:") @@ -66,7 +66,7 @@ object RDDRelation { parquetFile.where('key === 1).select('value as 'a).collect().foreach(println) // These files can also be registered as tables. - parquetFile.registerAsTable("parquetFile") + parquetFile.registerTempTable("parquetFile") sql("SELECT * FROM parquetFile").collect().foreach(println) } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index dc5290fb4f10e..12530c8490b09 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -56,7 +56,7 @@ object HiveFromSpark { // You can also register RDDs as temporary tables within a HiveContext. val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) - rdd.registerAsTable("records") + rdd.registerTempTable("records") // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 36e50e49c9a9c..42b738e112809 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -909,7 +909,7 @@ def __init__(self, sparkContext, sqlContext=None): ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> srdd = sqlCtx.inferSchema(allTypes) - >>> srdd.registerAsTable("allTypes") + >>> srdd.registerTempTable("allTypes") >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] @@ -1486,19 +1486,23 @@ def saveAsParquetFile(self, path): """ self._jschema_rdd.saveAsParquetFile(path) - def registerAsTable(self, name): + def registerTempTable(self, name): """Registers this RDD as a temporary table using the given name. The lifetime of this temporary table is tied to the L{SQLContext} that was used to create this SchemaRDD. >>> srdd = sqlCtx.inferSchema(rdd) - >>> srdd.registerAsTable("test") + >>> srdd.registerTempTable("test") >>> srdd2 = sqlCtx.sql("select * from test") >>> sorted(srdd.collect()) == sorted(srdd2.collect()) True """ - self._jschema_rdd.registerAsTable(name) + self._jschema_rdd.registerTempTable(name) + + def registerAsTable(self, name): + warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) + self.registerTempTable(name) def insertInto(self, tableName, overwrite=False): """Inserts the contents of this SchemaRDD into the specified table. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 33931e5d996f5..567f4dca991b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -116,7 +116,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * // |-- name: string (nullable = false) * // |-- age: integer (nullable = true) * - * peopleSchemaRDD.registerAsTable("people") + * peopleSchemaRDD.registerTempTable("people") * sqlContext.sql("select name from people").collect.foreach(println) * }}} * @@ -212,7 +212,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * import sqlContext._ * * case class Person(name: String, age: Int) - * createParquetFile[Person]("path/to/file.parquet").registerAsTable("people") + * createParquetFile[Person]("path/to/file.parquet").registerTempTable("people") * sql("INSERT INTO people SELECT 'michael', 29") * }}} * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index d34f62dc8865e..57df79321b35d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -67,7 +67,7 @@ import org.apache.spark.api.java.JavaRDD * val rdd = sc.parallelize((1 to 100).map(i => Record(i, s"val_$i"))) * // Any RDD containing case classes can be registered as a table. The schema of the table is * // automatically inferred using scala reflection. - * rdd.registerAsTable("records") + * rdd.registerTempTable("records") * * val results: SchemaRDD = sql("SELECT * FROM records") * }}} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 6a20def475822..2f3033a5f94f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -83,10 +83,13 @@ private[sql] trait SchemaRDDLike { * * @group schema */ - def registerAsTable(tableName: String): Unit = { + def registerTempTable(tableName: String): Unit = { sqlContext.registerRDDAsTable(baseSchemaRDD, tableName) } + @deprecated("Use registerTempTable instead of registerAsTable.", "1.1") + def registerAsTable(tableName: String): Unit = registerTempTable(tableName) + /** * :: Experimental :: * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index ae45193ed15d3..dbaa16e8b0c68 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -52,7 +52,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { * {{{ * JavaSQLContext sqlCtx = new JavaSQLContext(...) * - * sqlCtx.createParquetFile(Person.class, "path/to/file.parquet").registerAsTable("people") + * sqlCtx.createParquetFile(Person.class, "path/to/file.parquet").registerTempTable("people") * sqlCtx.sql("INSERT INTO people SELECT 'michael', 29") * }}} * diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index 3c92906d82864..33e5020bc636a 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -98,7 +98,7 @@ public Row call(Person person) throws Exception { StructType schema = DataType.createStructType(fields); JavaSchemaRDD schemaRDD = javaSqlCtx.applySchema(rowRDD, schema); - schemaRDD.registerAsTable("people"); + schemaRDD.registerTempTable("people"); List actual = javaSqlCtx.sql("SELECT * FROM people").collect(); List expected = new ArrayList(2); @@ -149,14 +149,14 @@ public void applySchemaToJSON() { JavaSchemaRDD schemaRDD1 = javaSqlCtx.jsonRDD(jsonRDD); StructType actualSchema1 = schemaRDD1.schema(); Assert.assertEquals(expectedSchema, actualSchema1); - schemaRDD1.registerAsTable("jsonTable1"); + schemaRDD1.registerTempTable("jsonTable1"); List actual1 = javaSqlCtx.sql("select * from jsonTable1").collect(); Assert.assertEquals(expectedResult, actual1); JavaSchemaRDD schemaRDD2 = javaSqlCtx.jsonRDD(jsonRDD, expectedSchema); StructType actualSchema2 = schemaRDD2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); - schemaRDD1.registerAsTable("jsonTable2"); + schemaRDD1.registerTempTable("jsonTable2"); List actual2 = javaSqlCtx.sql("select * from jsonTable2").collect(); Assert.assertEquals(expectedResult, actual2); } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index c3c0dcb1aa00b..fbf9bd9dbcdea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -78,7 +78,7 @@ class CachedTableSuite extends QueryTest { } test("SELECT Star Cached Table") { - TestSQLContext.sql("SELECT * FROM testData").registerAsTable("selectStar") + TestSQLContext.sql("SELECT * FROM testData").registerTempTable("selectStar") TestSQLContext.cacheTable("selectStar") TestSQLContext.sql("SELECT * FROM selectStar WHERE key = 1").collect() TestSQLContext.uncacheTable("selectStar") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala index 23a711d08c58b..c87d762751e6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/InsertIntoSuite.scala @@ -31,7 +31,7 @@ class InsertIntoSuite extends QueryTest { testFilePath.delete() testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - testFile.registerAsTable("createAndInsertTest") + testFile.registerTempTable("createAndInsertTest") // Add some data. testData.insertInto("createAndInsertTest") @@ -86,7 +86,7 @@ class InsertIntoSuite extends QueryTest { testFilePath.delete() testFilePath.deleteOnExit() val testFile = createParquetFile[TestData](testFilePath.getCanonicalPath) - testFile.registerAsTable("createAndInsertSQLTest") + testFile.registerTempTable("createAndInsertSQLTest") sql("INSERT INTO createAndInsertSQLTest SELECT * FROM testData") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 2fc80588182d9..6c7697ece8c56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -285,8 +285,8 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { } test("full outer join") { - upperCaseData.where('N <= 4).registerAsTable("left") - upperCaseData.where('N >= 3).registerAsTable("right") + upperCaseData.where('N <= 4).registerTempTable("left") + upperCaseData.where('N >= 3).registerTempTable("right") val left = UnresolvedRelation(None, "left", None) val right = UnresolvedRelation(None, "right", None) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 5c571d35d1bb9..9b2a36d33fca7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -461,7 +461,7 @@ class SQLQuerySuite extends QueryTest { } val schemaRDD1 = applySchema(rowRDD1, schema1) - schemaRDD1.registerAsTable("applySchema1") + schemaRDD1.registerTempTable("applySchema1") checkAnswer( sql("SELECT * FROM applySchema1"), (1, "A1", true, null) :: @@ -491,7 +491,7 @@ class SQLQuerySuite extends QueryTest { } val schemaRDD2 = applySchema(rowRDD2, schema2) - schemaRDD2.registerAsTable("applySchema2") + schemaRDD2.registerTempTable("applySchema2") checkAnswer( sql("SELECT * FROM applySchema2"), (Seq(1, true), Map("A1" -> null)) :: @@ -516,7 +516,7 @@ class SQLQuerySuite extends QueryTest { } val schemaRDD3 = applySchema(rowRDD3, schema2) - schemaRDD3.registerAsTable("applySchema3") + schemaRDD3.registerTempTable("applySchema3") checkAnswer( sql("SELECT f1.f11, f2['D4'] FROM applySchema3"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index f2934da9a031d..5b84c658db942 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -61,7 +61,7 @@ class ScalaReflectionRelationSuite extends FunSuite { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, BigDecimal(1), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerAsTable("reflectData") + rdd.registerTempTable("reflectData") assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq) } @@ -69,7 +69,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with nulls") { val data = NullReflectData(null, null, null, null, null, null, null) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerAsTable("reflectNullData") + rdd.registerTempTable("reflectNullData") assert(sql("SELECT * FROM reflectNullData").collect().head === Seq.fill(7)(null)) } @@ -77,7 +77,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD with Nones") { val data = OptionalReflectData(None, None, None, None, None, None, None) val rdd = sparkContext.parallelize(data :: Nil) - rdd.registerAsTable("reflectOptionalData") + rdd.registerTempTable("reflectOptionalData") assert(sql("SELECT * FROM reflectOptionalData").collect().head === Seq.fill(7)(null)) } @@ -85,7 +85,7 @@ class ScalaReflectionRelationSuite extends FunSuite { // Equality is broken for Arrays, so we test that separately. test("query binary data") { val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil) - rdd.registerAsTable("reflectBinary") + rdd.registerTempTable("reflectBinary") val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 088e6e3c843aa..c3ec82fb69778 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -30,7 +30,7 @@ case class TestData(key: Int, value: String) object TestData { val testData: SchemaRDD = TestSQLContext.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))) - testData.registerAsTable("testData") + testData.registerTempTable("testData") case class LargeAndSmallInts(a: Int, b: Int) val largeAndSmallInts: SchemaRDD = @@ -41,7 +41,7 @@ object TestData { LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: LargeAndSmallInts(3, 2) :: Nil) - largeAndSmallInts.registerAsTable("largeAndSmallInts") + largeAndSmallInts.registerTempTable("largeAndSmallInts") case class TestData2(a: Int, b: Int) val testData2: SchemaRDD = @@ -52,7 +52,7 @@ object TestData { TestData2(2, 2) :: TestData2(3, 1) :: TestData2(3, 2) :: Nil) - testData2.registerAsTable("testData2") + testData2.registerTempTable("testData2") // TODO: There is no way to express null primitives as case classes currently... val testData3 = @@ -71,7 +71,7 @@ object TestData { UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: UpperCaseData(6, "F") :: Nil) - upperCaseData.registerAsTable("upperCaseData") + upperCaseData.registerTempTable("upperCaseData") case class LowerCaseData(n: Int, l: String) val lowerCaseData = @@ -80,14 +80,14 @@ object TestData { LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil) - lowerCaseData.registerAsTable("lowerCaseData") + lowerCaseData.registerTempTable("lowerCaseData") case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) val arrayData = TestSQLContext.sparkContext.parallelize( ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) - arrayData.registerAsTable("arrayData") + arrayData.registerTempTable("arrayData") case class MapData(data: Map[Int, String]) val mapData = @@ -97,18 +97,18 @@ object TestData { MapData(Map(1 -> "a3", 2 -> "b3", 3 -> "c3")) :: MapData(Map(1 -> "a4", 2 -> "b4")) :: MapData(Map(1 -> "a5")) :: Nil) - mapData.registerAsTable("mapData") + mapData.registerTempTable("mapData") case class StringData(s: String) val repeatedData = TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test"))) - repeatedData.registerAsTable("repeatedData") + repeatedData.registerTempTable("repeatedData") val nullableRepeatedData = TestSQLContext.sparkContext.parallelize( List.fill(2)(StringData(null)) ++ List.fill(2)(StringData("test"))) - nullableRepeatedData.registerAsTable("nullableRepeatedData") + nullableRepeatedData.registerTempTable("nullableRepeatedData") case class NullInts(a: Integer) val nullInts = @@ -118,7 +118,7 @@ object TestData { NullInts(3) :: NullInts(null) :: Nil ) - nullInts.registerAsTable("nullInts") + nullInts.registerTempTable("nullInts") val allNulls = TestSQLContext.sparkContext.parallelize( @@ -126,7 +126,7 @@ object TestData { NullInts(null) :: NullInts(null) :: NullInts(null) :: Nil) - allNulls.registerAsTable("allNulls") + allNulls.registerTempTable("allNulls") case class NullStrings(n: Int, s: String) val nullStrings = @@ -134,10 +134,10 @@ object TestData { NullStrings(1, "abc") :: NullStrings(2, "ABC") :: NullStrings(3, null) :: Nil) - nullStrings.registerAsTable("nullStrings") + nullStrings.registerTempTable("nullStrings") case class TableName(tableName: String) - TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).registerAsTable("tableName") + TestSQLContext.sparkContext.parallelize(TableName("test") :: Nil).registerTempTable("tableName") val unparsedStrings = TestSQLContext.sparkContext.parallelize( @@ -150,5 +150,5 @@ object TestData { val timestamps = TestSQLContext.sparkContext.parallelize((1 to 3).map { i => TimestampField(new Timestamp(i)) }) - timestamps.registerAsTable("timestamps") + timestamps.registerTempTable("timestamps") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index 020baf0c7ec6f..203ff847e94cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -59,7 +59,7 @@ class JavaSQLSuite extends FunSuite { val rdd = javaCtx.parallelize(person :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[PersonBean]) - schemaRDD.registerAsTable("people") + schemaRDD.registerTempTable("people") javaSqlCtx.sql("SELECT * FROM people").collect() } @@ -76,7 +76,7 @@ class JavaSQLSuite extends FunSuite { val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) - schemaRDD.registerAsTable("allTypes") + schemaRDD.registerTempTable("allTypes") assert( javaSqlCtx.sql( @@ -101,7 +101,7 @@ class JavaSQLSuite extends FunSuite { val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) - schemaRDD.registerAsTable("allTypes") + schemaRDD.registerTempTable("allTypes") assert( javaSqlCtx.sql( @@ -127,7 +127,7 @@ class JavaSQLSuite extends FunSuite { var schemaRDD = javaSqlCtx.jsonRDD(rdd) - schemaRDD.registerAsTable("jsonTable1") + schemaRDD.registerTempTable("jsonTable1") assert( javaSqlCtx.sql("select * from jsonTable1").collect.head.row === @@ -144,7 +144,7 @@ class JavaSQLSuite extends FunSuite { rdd.saveAsTextFile(path) schemaRDD = javaSqlCtx.jsonFile(path) - schemaRDD.registerAsTable("jsonTable2") + schemaRDD.registerTempTable("jsonTable2") assert( javaSqlCtx.sql("select * from jsonTable2").collect.head.row === diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 9d9cfdd7c92e3..75c0589eb208e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -183,7 +183,7 @@ class JsonSuite extends QueryTest { assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -223,7 +223,7 @@ class JsonSuite extends QueryTest { assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") // Access elements of a primitive array. checkAnswer( @@ -291,7 +291,7 @@ class JsonSuite extends QueryTest { ignore("Complex field and type inferring (Ignored)") { val jsonSchemaRDD = jsonRDD(complexFieldAndType) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") // Right now, "field1" and "field2" are treated as aliases. We should fix it. checkAnswer( @@ -320,7 +320,7 @@ class JsonSuite extends QueryTest { assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -374,7 +374,7 @@ class JsonSuite extends QueryTest { ignore("Type conflict in primitive field values (Ignored)") { val jsonSchemaRDD = jsonRDD(primitiveFieldValueTypeConflict) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") // Right now, the analyzer does not promote strings in a boolean expreesion. // Number and Boolean conflict: resolve the type as boolean in this query. @@ -445,7 +445,7 @@ class JsonSuite extends QueryTest { assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -466,7 +466,7 @@ class JsonSuite extends QueryTest { assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -494,7 +494,7 @@ class JsonSuite extends QueryTest { assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") } test("Loading a JSON dataset from a text file") { @@ -514,7 +514,7 @@ class JsonSuite extends QueryTest { assert(expectedSchema === jsonSchemaRDD.schema) - jsonSchemaRDD.registerAsTable("jsonTable") + jsonSchemaRDD.registerTempTable("jsonTable") checkAnswer( sql("select * from jsonTable"), @@ -546,7 +546,7 @@ class JsonSuite extends QueryTest { assert(schema === jsonSchemaRDD1.schema) - jsonSchemaRDD1.registerAsTable("jsonTable1") + jsonSchemaRDD1.registerTempTable("jsonTable1") checkAnswer( sql("select * from jsonTable1"), @@ -563,7 +563,7 @@ class JsonSuite extends QueryTest { assert(schema === jsonSchemaRDD2.schema) - jsonSchemaRDD2.registerAsTable("jsonTable2") + jsonSchemaRDD2.registerTempTable("jsonTable2") checkAnswer( sql("select * from jsonTable2"), 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 8955455ec98c7..9933575038bd3 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 @@ -101,9 +101,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA ParquetTestData.writeNestedFile3() ParquetTestData.writeNestedFile4() testRDD = parquetFile(ParquetTestData.testDir.toString) - testRDD.registerAsTable("testsource") + testRDD.registerTempTable("testsource") parquetFile(ParquetTestData.testFilterDir.toString) - .registerAsTable("testfiltersource") + .registerTempTable("testfiltersource") } override def afterAll() { @@ -247,7 +247,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Creating case class RDD table") { TestSQLContext.sparkContext.parallelize((1 to 100)) .map(i => TestRDDEntry(i, s"val_$i")) - .registerAsTable("tmp") + .registerTempTable("tmp") val rdd = sql("SELECT * FROM tmp").collect().sortBy(_.getInt(0)) var counter = 1 rdd.foreach { @@ -266,7 +266,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .map(i => TestRDDEntry(i, s"val_$i")) rdd.saveAsParquetFile(path) val readFile = parquetFile(path) - readFile.registerAsTable("tmpx") + readFile.registerTempTable("tmpx") val rdd_copy = sql("SELECT * FROM tmpx").collect() val rdd_orig = rdd.collect() for(i <- 0 to 99) { @@ -280,9 +280,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val dirname = Utils.createTempDir() val source_rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) .map(i => TestRDDEntry(i, s"val_$i")) - source_rdd.registerAsTable("source") + source_rdd.registerTempTable("source") val dest_rdd = createParquetFile[TestRDDEntry](dirname.toString) - dest_rdd.registerAsTable("dest") + dest_rdd.registerTempTable("dest") sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() assert(rdd_copy1.size === 100) @@ -547,7 +547,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD - data.registerAsTable("data") + data.registerTempTable("data") val query = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM data") val tmp = query.collect() assert(tmp.size === 2) @@ -562,7 +562,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD - data.registerAsTable("data") + data.registerTempTable("data") val result1 = nestedParserSqlContext.sql("SELECT entries[0].value FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) @@ -589,7 +589,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD - data.registerAsTable("data") + data.registerTempTable("data") val result1 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) @@ -608,7 +608,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = TestSQLContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD - data.registerAsTable("mapTable") + data.registerTempTable("mapTable") val result1 = sql("SELECT data1 FROM mapTable").collect() assert(result1.size === 1) assert(result1(0)(0) @@ -625,7 +625,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val data = nestedParserSqlContext .parquetFile(ParquetTestData.testNestedDir4.toString) .toSchemaRDD - data.registerAsTable("mapTable") + data.registerTempTable("mapTable") val result1 = nestedParserSqlContext.sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) val entry1 = result1(0)(0) @@ -658,7 +658,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA nestedParserSqlContext .parquetFile(tmpdir.toString) .toSchemaRDD - .registerAsTable("tmpcopy") + .registerTempTable("tmpcopy") val tmpdata = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() assert(tmpdata.size === 2) assert(tmpdata(0).size === 2) @@ -679,7 +679,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA nestedParserSqlContext .parquetFile(tmpdir.toString) .toSchemaRDD - .registerAsTable("tmpmapcopy") + .registerTempTable("tmpmapcopy") val result1 = nestedParserSqlContext.sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() assert(result1.size === 1) assert(result1(0)(0) === 2) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 833f3502154f3..7e323146f9da2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -28,7 +28,7 @@ case class TestData(key: Int, value: String) class InsertIntoHiveTableSuite extends QueryTest { val testData = TestHive.sparkContext.parallelize( (1 to 100).map(i => TestData(i, i.toString))) - testData.registerAsTable("testData") + testData.registerTempTable("testData") test("insertInto() HiveTable") { createTable[TestData]("createAndInsertTest") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala index 10c8069a624e6..578f27574ad2f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala @@ -63,7 +63,7 @@ class JavaHiveQLSuite extends FunSuite { javaHiveCtx.hql(s"CREATE TABLE $tableName(key INT, value STRING)").count() } - javaHiveCtx.hql("SHOW TABLES").registerAsTable("show_tables") + javaHiveCtx.hql("SHOW TABLES").registerTempTable("show_tables") assert( javaHiveCtx @@ -73,7 +73,7 @@ class JavaHiveQLSuite extends FunSuite { .contains(tableName)) assertResult(Array(Array("key", "int", "None"), Array("value", "string", "None"))) { - javaHiveCtx.hql(s"DESCRIBE $tableName").registerAsTable("describe_table") + javaHiveCtx.hql(s"DESCRIBE $tableName").registerTempTable("describe_table") javaHiveCtx .hql("SELECT result FROM describe_table") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 89cc589fb8001..4ed41550cf530 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -247,7 +247,7 @@ class HiveQuerySuite extends HiveComparisonTest { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) - testData.registerAsTable("REGisteredTABle") + testData.registerTempTable("REGisteredTABle") assertResult(Array(Array(2, "str2"))) { hql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + @@ -272,7 +272,7 @@ class HiveQuerySuite extends HiveComparisonTest { test("SPARK-2180: HAVING support in GROUP BY clauses (positive)") { val fixture = List(("foo", 2), ("bar", 1), ("foo", 4), ("bar", 3)) .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)} - TestHive.sparkContext.parallelize(fixture).registerAsTable("having_test") + TestHive.sparkContext.parallelize(fixture).registerTempTable("having_test") val results = hql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() @@ -401,7 +401,7 @@ class HiveQuerySuite extends HiveComparisonTest { TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) - testData.registerAsTable("test_describe_commands2") + testData.registerTempTable("test_describe_commands2") assertResult( Array( 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 fb03db12a0b01..2455c18925dfa 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 @@ -54,14 +54,14 @@ class HiveResolutionSuite extends HiveComparisonTest { test("case insensitivity with scala reflection") { // Test resolution with Scala Reflection TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) - .registerAsTable("caseSensitivityTest") + .registerTempTable("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") + .registerTempTable("nestedRepeatedTest") assert(hql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 47526e3596e44..6545e8d7dcb69 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -41,7 +41,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft // write test data ParquetTestData.writeFile() testRDD = parquetFile(ParquetTestData.testDir.toString) - testRDD.registerAsTable("testsource") + testRDD.registerTempTable("testsource") } override def afterAll() { @@ -67,7 +67,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft .map(i => Cases(i, i)) .saveAsParquetFile(tempFile.getCanonicalPath) - parquetFile(tempFile.getCanonicalPath).registerAsTable("cases") + parquetFile(tempFile.getCanonicalPath).registerTempTable("cases") hql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) hql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) } @@ -86,7 +86,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft test("Converting Hive to Parquet Table via saveAsParquetFile") { hql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) - parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") + parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") val rddOne = hql("SELECT * FROM src").collect().sortBy(_.getInt(0)) val rddTwo = hql("SELECT * from ptable").collect().sortBy(_.getInt(0)) compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) @@ -94,7 +94,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft test("INSERT OVERWRITE TABLE Parquet table") { hql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) - parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") + parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") // let's do three overwrites for good measure hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() From 33f167d762483b55d5d874dcc1e3075f661d4375 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 2 Aug 2014 21:44:19 -0700 Subject: [PATCH 0224/1492] SPARK-2602 [BUILD] Tests steal focus under Java 6 As per https://issues.apache.org/jira/browse/SPARK-2602 , this may be resolved for Java 6 with the java.awt.headless system property, which never hurt anyone running a command line app. I tested it and seemed to get rid of focus stealing. Author: Sean Owen Closes #1747 from srowen/SPARK-2602 and squashes the following commits: b141018 [Sean Owen] Set java.awt.headless during tests --- pom.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/pom.xml b/pom.xml index a42759169149b..cc9377cec2a07 100644 --- a/pom.xml +++ b/pom.xml @@ -871,6 +871,7 @@ -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m + true ${session.executionRootDirectory} 1 From 9cf429aaf529e91f619910c33cfe46bf33a66982 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 2 Aug 2014 21:55:56 -0700 Subject: [PATCH 0225/1492] SPARK-2414 [BUILD] Add LICENSE entry for jquery The JIRA concerned removing jquery, and this does not remove jquery. While it is distributed by Spark it should have an accompanying line in LICENSE, very technically, as per http://www.apache.org/dev/licensing-howto.html Author: Sean Owen Closes #1748 from srowen/SPARK-2414 and squashes the following commits: 2fdb03c [Sean Owen] Add LICENSE entry for jquery --- LICENSE | 1 + 1 file changed, 1 insertion(+) diff --git a/LICENSE b/LICENSE index 76a3601c66918..e9a1153fdc5db 100644 --- a/LICENSE +++ b/LICENSE @@ -549,3 +549,4 @@ The following components are provided under the MIT License. See project link fo (MIT License) pyrolite (org.spark-project:pyrolite:2.0.1 - http://pythonhosted.org/Pyro4/) (MIT License) scopt (com.github.scopt:scopt_2.10:3.2.0 - https://github.com/scopt/scopt) (The MIT License) Mockito (org.mockito:mockito-all:1.8.5 - http://www.mockito.org) + (MIT License) jquery (https://jquery.org/license/) From 3dc55fdf450b4237f7c592fce56d1467fd206366 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 2 Aug 2014 22:00:46 -0700 Subject: [PATCH 0226/1492] [Minor] Fixes on top of #1679 Minor fixes on top of #1679. Author: Andrew Or Closes #1736 from andrewor14/amend-#1679 and squashes the following commits: 3b46f5e [Andrew Or] Minor fixes --- .../org/apache/spark/storage/BlockManagerSource.scala | 5 ++--- .../scala/org/apache/spark/storage/StorageUtils.scala | 11 ++++------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index e939318a029dd..3f14c40ec61cb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -46,9 +46,8 @@ private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: Spar metricRegistry.register(MetricRegistry.name("memory", "memUsed_MB"), new Gauge[Long] { override def getValue: Long = { val storageStatusList = blockManager.master.getStorageStatus - val maxMem = storageStatusList.map(_.maxMem).sum - val remainingMem = storageStatusList.map(_.memRemaining).sum - (maxMem - remainingMem) / 1024 / 1024 + val memUsed = storageStatusList.map(_.memUsed).sum + memUsed / 1024 / 1024 } }) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 0a0a448baa2ef..2bd6b749be261 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -172,16 +172,13 @@ class StorageStatus(val blockManagerId: BlockManagerId, val maxMem: Long) { def memRemaining: Long = maxMem - memUsed /** Return the memory used by this block manager. */ - def memUsed: Long = - _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum + def memUsed: Long = _nonRddStorageInfo._1 + _rddBlocks.keys.toSeq.map(memUsedByRdd).sum /** Return the disk space used by this block manager. */ - def diskUsed: Long = - _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum + def diskUsed: Long = _nonRddStorageInfo._2 + _rddBlocks.keys.toSeq.map(diskUsedByRdd).sum /** Return the off-heap space used by this block manager. */ - def offHeapUsed: Long = - _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).sum + def offHeapUsed: Long = _nonRddStorageInfo._3 + _rddBlocks.keys.toSeq.map(offHeapUsedByRdd).sum /** Return the memory used by the given RDD in this block manager in O(1) time. */ def memUsedByRdd(rddId: Int): Long = _rddStorageInfo.get(rddId).map(_._1).getOrElse(0L) @@ -246,7 +243,7 @@ private[spark] object StorageUtils { val rddId = rddInfo.id // Assume all blocks belonging to the same RDD have the same storage level val storageLevel = statuses - .map(_.rddStorageLevel(rddId)).flatMap(s => s).headOption.getOrElse(StorageLevel.NONE) + .flatMap(_.rddStorageLevel(rddId)).headOption.getOrElse(StorageLevel.NONE) val numCachedPartitions = statuses.map(_.numRddBlocksById(rddId)).sum val memSize = statuses.map(_.memUsedByRdd(rddId)).sum val diskSize = statuses.map(_.diskUsedByRdd(rddId)).sum From f8cd143b6b1b4d8aac87c229e5af263b0319b3ea Mon Sep 17 00:00:00 2001 From: Stephen Boesch Date: Sun, 3 Aug 2014 10:19:04 -0700 Subject: [PATCH 0227/1492] SPARK-2712 - Add a small note to maven doc that mvn package must happen ... Per request by Reynold adding small note about proper sequencing of build then test. Author: Stephen Boesch Closes #1615 from javadba/docs and squashes the following commits: 6c3183e [Stephen Boesch] Moved updated testing blurb per PWendell 5764757 [Stephen Boesch] SPARK-2712 - Add a small note to maven doc that mvn package must happen before test --- docs/building-with-maven.md | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 55a9e37dfed83..672d0ef114f6d 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -98,7 +98,12 @@ mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -Dski # Spark Tests in Maven -Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. You can then run the tests with `mvn -Dhadoop.version=... test`. +Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). + +Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: + + mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive clean package + mvn -Pyarn -Phadoop-2.3 -Phive test The ScalaTest plugin also supports running only a specific test suite as follows: From a0bcbc159e89be868ccc96175dbf1439461557e1 Mon Sep 17 00:00:00 2001 From: "Allan Douglas R. de Oliveira" Date: Sun, 3 Aug 2014 10:25:59 -0700 Subject: [PATCH 0228/1492] SPARK-2246: Add user-data option to EC2 scripts Author: Allan Douglas R. de Oliveira Closes #1186 from douglaz/spark_ec2_user_data and squashes the following commits: 94a36f9 [Allan Douglas R. de Oliveira] Added user data option to EC2 script --- ec2/spark_ec2.py | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 02cfe4ec39c7d..0c2f85a3868f4 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -135,6 +135,10 @@ def parse_args(): "--master-opts", type="string", default="", help="Extra options to give to master through SPARK_MASTER_OPTS variable " + "(e.g -Dspark.worker.timeout=180)") + parser.add_option( + "--user-data", type="string", default="", + help="Path to a user-data file (most AMI's interpret this as an initialization script)") + (opts, args) = parser.parse_args() if len(args) != 2: @@ -274,6 +278,12 @@ def launch_cluster(conn, opts, cluster_name): if opts.key_pair is None: print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." sys.exit(1) + + user_data_content = None + if opts.user_data: + with open(opts.user_data) as user_data_file: + user_data_content = user_data_file.read() + print "Setting up security groups..." master_group = get_or_make_group(conn, cluster_name + "-master") slave_group = get_or_make_group(conn, cluster_name + "-slaves") @@ -347,7 +357,8 @@ def launch_cluster(conn, opts, cluster_name): key_name=opts.key_pair, security_groups=[slave_group], instance_type=opts.instance_type, - block_device_map=block_map) + block_device_map=block_map, + user_data=user_data_content) my_req_ids += [req.id for req in slave_reqs] i += 1 @@ -398,7 +409,8 @@ def launch_cluster(conn, opts, cluster_name): placement=zone, min_count=num_slaves_this_zone, max_count=num_slaves_this_zone, - block_device_map=block_map) + block_device_map=block_map, + user_data=user_data_content) slave_nodes += slave_res.instances print "Launched %d slaves in %s, regid = %s" % (num_slaves_this_zone, zone, slave_res.id) From 2998e38a942351974da36cb619e863c6f0316e7a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 3 Aug 2014 10:36:52 -0700 Subject: [PATCH 0229/1492] [SPARK-2197] [mllib] Java DecisionTree bug fix and easy-of-use Bug fix: Before, when an RDD was created in Java and passed to DecisionTree.train(), the fake class tag caused problems. * Fix: DecisionTree: Used new RDD.retag() method to allow passing RDDs from Java. Other improvements to Decision Trees for easy-of-use with Java: * impurity classes: Added instance() methods to help with Java interface. * Strategy: Added Java-friendly constructor --> Note: I removed quantileCalculationStrategy from the Java-friendly constructor since (a) it is a special class and (b) there is only 1 option currently. I suspect we will redo the API before the other options are included. CC: mengxr Author: Joseph K. Bradley Closes #1740 from jkbradley/dt-java-new and squashes the following commits: 0805dc6 [Joseph K. Bradley] Changed Strategy to use JavaConverters instead of JavaConversions 519b1b7 [Joseph K. Bradley] * Organized imports in JavaDecisionTreeSuite.java * Using JavaConverters instead of JavaConversions in DecisionTreeSuite.scala f7b5ca1 [Joseph K. Bradley] Improvements to make it easier to run DecisionTree from Java. * DecisionTree: Used new RDD.retag() method to allow passing RDDs from Java. * impurity classes: Added instance() methods to help with Java interface. * Strategy: Added Java-friendly constructor ** Note: I removed quantileCalculationStrategy from the Java-friendly constructor since (a) it is a special class and (b) there is only 1 option currently. I suspect we will redo the API before the other options are included. d78ada6 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-java 320853f [Joseph K. Bradley] Added JavaDecisionTreeSuite, partly written 13a585e [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-java f1a8283 [Joseph K. Bradley] Added old JavaDecisionTreeSuite, to be updated later 225822f [Joseph K. Bradley] Bug: In DecisionTree, the method sequentialBinSearchForOrderedCategoricalFeatureInClassification() indexed bins from 0 to (math.pow(2, featureCategories.toInt - 1) - 1). This upper bound is the bound for unordered categorical features, not ordered ones. The upper bound should be the arity (i.e., max value) of the feature. --- .../spark/mllib/tree/DecisionTree.scala | 8 +- .../mllib/tree/configuration/Strategy.scala | 29 +++++ .../spark/mllib/tree/impurity/Entropy.scala | 7 ++ .../spark/mllib/tree/impurity/Gini.scala | 7 ++ .../spark/mllib/tree/impurity/Variance.scala | 7 ++ .../mllib/tree/JavaDecisionTreeSuite.java | 102 ++++++++++++++++++ .../spark/mllib/tree/DecisionTreeSuite.scala | 6 ++ 7 files changed, 162 insertions(+), 4 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 382e76a9b7cba..1d03e6e3b36cf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -48,12 +48,12 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo def train(input: RDD[LabeledPoint]): DecisionTreeModel = { // Cache input RDD for speedup during multiple passes. - input.cache() + val retaggedInput = input.retag(classOf[LabeledPoint]).cache() logDebug("algo = " + strategy.algo) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, strategy) val numBins = bins(0).length logDebug("numBins = " + numBins) @@ -70,7 +70,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // dummy value for top node (updated during first split calculation) val nodes = new Array[Node](maxNumNodes) // num features - val numFeatures = input.take(1)(0).features.size + val numFeatures = retaggedInput.take(1)(0).features.size // Calculate level for single group construction @@ -107,7 +107,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("#####################################") // Find best split for all nodes at a level. - val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, + val splitsStatsForLevel = DecisionTree.findBestSplits(retaggedInput, parentImpurities, strategy, level, filters, splits, bins, maxLevelForSingleGroup) for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index fdad4f029aa99..4ee4bcd0bcbc7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.tree.configuration +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.mllib.tree.configuration.Algo._ @@ -61,4 +63,31 @@ class Strategy ( val isMulticlassWithCategoricalFeatures = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) + /** + * Java-friendly constructor. + * + * @param algo classification or regression + * @param impurity criterion used for information gain calculation + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * @param numClassesForClassification number of classes for classification. Default value is 2 + * leads to binary classification + * @param maxBins maximum number of bins used for splitting features + * @param categoricalFeaturesInfo A map storing information about the categorical variables and + * the number of discrete values they take. For example, an entry + * (n -> k) implies the feature n is categorical with k categories + * 0, 1, 2, ... , k-1. It's important to note that features are + * zero-indexed. + */ + def this( + algo: Algo, + impurity: Impurity, + maxDepth: Int, + numClassesForClassification: Int, + maxBins: Int, + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]) { + this(algo, impurity, maxDepth, numClassesForClassification, maxBins, Sort, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 9297c20596527..96d2471e1f88c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -66,4 +66,11 @@ object Entropy extends Impurity { @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") + + /** + * Get this impurity instance. + * This is useful for passing impurity parameters to a Strategy in Java. + */ + def instance = this + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 2874bcf496484..d586f449048bb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -62,4 +62,11 @@ object Gini extends Impurity { @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") + + /** + * Get this impurity instance. + * This is useful for passing impurity parameters to a Strategy in Java. + */ + def instance = this + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 698a1a2a8e899..f7d99a40eb380 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -53,4 +53,11 @@ object Variance extends Impurity { val squaredLoss = sumSquares - (sum * sum) / count squaredLoss / count } + + /** + * Get this impurity instance. + * This is useful for passing impurity parameters to a Strategy in Java. + */ + def instance = this + } diff --git a/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java new file mode 100644 index 0000000000000..2c281a1ee7157 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.configuration.Algo; +import org.apache.spark.mllib.tree.configuration.Strategy; +import org.apache.spark.mllib.tree.impurity.Gini; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; + + +public class JavaDecisionTreeSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaDecisionTreeSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + int validatePrediction(List validationData, DecisionTreeModel model) { + int numCorrect = 0; + for (LabeledPoint point: validationData) { + Double prediction = model.predict(point.features()); + if (prediction == point.label()) { + numCorrect++; + } + } + return numCorrect; + } + + @Test + public void runDTUsingConstructor() { + List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); + JavaRDD rdd = sc.parallelize(arr); + HashMap categoricalFeaturesInfo = new HashMap(); + categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories + + int maxDepth = 4; + int numClasses = 2; + int maxBins = 100; + Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, + maxBins, categoricalFeaturesInfo); + + DecisionTree learner = new DecisionTree(strategy); + DecisionTreeModel model = learner.train(rdd.rdd()); + + int numCorrect = validatePrediction(arr, model); + Assert.assertTrue(numCorrect == rdd.count()); + } + + @Test + public void runDTUsingStaticMethods() { + List arr = DecisionTreeSuite.generateCategoricalDataPointsAsJavaList(); + JavaRDD rdd = sc.parallelize(arr); + HashMap categoricalFeaturesInfo = new HashMap(); + categoricalFeaturesInfo.put(1, 2); // feature 1 has 2 categories + + int maxDepth = 4; + int numClasses = 2; + int maxBins = 100; + Strategy strategy = new Strategy(Algo.Classification(), Gini.instance(), maxDepth, numClasses, + maxBins, categoricalFeaturesInfo); + + DecisionTreeModel model = DecisionTree$.MODULE$.train(rdd.rdd(), strategy); + + int numCorrect = validatePrediction(arr, model); + Assert.assertTrue(numCorrect == rdd.count()); + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 8665a00f3b356..70ca7c8a266f2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.tree +import scala.collection.JavaConverters._ + import org.scalatest.FunSuite import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} @@ -815,6 +817,10 @@ object DecisionTreeSuite { arr } + def generateCategoricalDataPointsAsJavaList(): java.util.List[LabeledPoint] = { + generateCategoricalDataPoints().toList.asJava + } + def generateCategoricalDataPointsForMulticlass(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](3000) for (i <- 0 until 3000) { From 236dfac6769016e433b2f6517cda2d308dea74bc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 3 Aug 2014 12:28:29 -0700 Subject: [PATCH 0230/1492] [SPARK-2784][SQL] Deprecate hql() method in favor of a config option, 'spark.sql.dialect' Many users have reported being confused by the distinction between the `sql` and `hql` methods. Specifically, many users think that `sql(...)` cannot be used to read hive tables. In this PR I introduce a new configuration option `spark.sql.dialect` that picks which dialect with be used for parsing. For SQLContext this must be set to `sql`. In `HiveContext` it defaults to `hiveql` but can also be set to `sql`. The `hql` and `hiveql` methods continue to act the same but are now marked as deprecated. **This is a possibly breaking change for some users unless they set the dialect manually, though this is unlikely.** For example: `hiveContex.sql("SELECT 1")` will now throw a parsing exception by default. Author: Michael Armbrust Closes #1746 from marmbrus/sqlLanguageConf and squashes the following commits: ad375cc [Michael Armbrust] Merge remote-tracking branch 'apache/master' into sqlLanguageConf 20c43f8 [Michael Armbrust] override function instead of just setting the value 7e4ae93 [Michael Armbrust] Deprecate hql() method in favor of a config option, 'spark.sql.dialect' --- .../sbt_app_hive/src/main/scala/HiveApp.scala | 8 +- docs/sql-programming-guide.md | 18 ++-- .../examples/sql/hive/HiveFromSpark.scala | 12 +-- python/pyspark/sql.py | 20 ++-- .../scala/org/apache/spark/sql/SQLConf.scala | 17 +++- .../org/apache/spark/sql/SQLContext.scala | 11 ++- .../spark/sql/api/java/JavaSQLContext.scala | 14 ++- .../hive/thriftserver/SparkSQLDriver.scala | 2 +- .../server/SparkSQLOperationManager.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 26 ++++-- .../sql/hive/api/java/JavaHiveContext.scala | 15 ++- .../spark/sql/hive/CachedTableSuite.scala | 14 +-- .../spark/sql/hive/StatisticsSuite.scala | 10 +- .../sql/hive/api/java/JavaHiveQLSuite.scala | 19 ++-- .../hive/execution/HiveComparisonTest.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 93 ++++++++++--------- .../hive/execution/HiveResolutionSuite.scala | 6 +- .../execution/HiveTypeCoercionSuite.scala | 2 +- .../sql/hive/execution/HiveUdfSuite.scala | 10 +- .../sql/hive/execution/PruningSuite.scala | 2 +- .../spark/sql/parquet/HiveParquetSuite.scala | 27 +++--- 21 files changed, 199 insertions(+), 133 deletions(-) diff --git a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala index a21410f3b9813..5111bc0adb772 100644 --- a/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala +++ b/dev/audit-release/sbt_app_hive/src/main/scala/HiveApp.scala @@ -37,10 +37,10 @@ object SparkSqlExample { val hiveContext = new HiveContext(sc) import hiveContext._ - hql("DROP TABLE IF EXISTS src") - hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - hql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src") - val results = hql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect() + sql("DROP TABLE IF EXISTS src") + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("LOAD DATA LOCAL INPATH 'data.txt' INTO TABLE src") + val results = sql("FROM src SELECT key, value WHERE key >= 0 AND KEY < 5").collect() results.foreach(println) def test(f: => Boolean, failureMsg: String) = { diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 0465468084cee..cd6543945c385 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -495,11 +495,11 @@ directory. // sc is an existing SparkContext. val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) -hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -hiveContext.hql("FROM src SELECT key, value").collect().foreach(println) +hiveContext.sql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
    @@ -515,11 +515,11 @@ expressed in HiveQL. // sc is an existing JavaSparkContext. JavaHiveContext hiveContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); -hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); +hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = hiveContext.hql("FROM src SELECT key, value").collect(); +Row[] results = hiveContext.sql("FROM src SELECT key, value").collect(); {% endhighlight %} @@ -537,11 +537,11 @@ expressed in HiveQL. from pyspark.sql import HiveContext hiveContext = HiveContext(sc) -hiveContext.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = hiveContext.hql("FROM src SELECT key, value").collect() +results = hiveContext.sql("FROM src SELECT key, value").collect() {% endhighlight %} diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 12530c8490b09..3423fac0ad303 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -34,20 +34,20 @@ object HiveFromSpark { val hiveContext = new HiveContext(sc) import hiveContext._ - hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - hql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL println("Result of 'SELECT *': ") - hql("SELECT * FROM src").collect.foreach(println) + sql("SELECT * FROM src").collect.foreach(println) // Aggregation queries are also supported. - val count = hql("SELECT COUNT(*) FROM src").collect().head.getLong(0) + val count = sql("SELECT COUNT(*) FROM src").collect().head.getLong(0) println(s"COUNT(*): $count") // The results of SQL queries are themselves RDDs and support all normal RDD functions. The // items in the RDD are of type Row, which allows you to access each column by ordinal. - val rddFromSql = hql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") + val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") println("Result of RDD.map:") val rddAsStrings = rddFromSql.map { @@ -60,6 +60,6 @@ object HiveFromSpark { // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") - hql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) + sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) } } diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 42b738e112809..1a829c6fafe03 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1291,16 +1291,20 @@ def _get_hive_ctx(self): def hiveql(self, hqlQuery): """ - Runs a query expressed in HiveQL, returning the result as - a L{SchemaRDD}. + DEPRECATED: Use sql() """ + warnings.warn("hiveql() is deprecated as the sql function now parses using HiveQL by" + + "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'", + DeprecationWarning) return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery), self) def hql(self, hqlQuery): """ - Runs a query expressed in HiveQL, returning the result as - a L{SchemaRDD}. + DEPRECATED: Use sql() """ + warnings.warn("hql() is deprecated as the sql function now parses using HiveQL by" + + "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'", + DeprecationWarning) return self.hiveql(hqlQuery) @@ -1313,16 +1317,16 @@ class LocalHiveContext(HiveContext): >>> import os >>> hiveCtx = LocalHiveContext(sc) >>> try: - ... supress = hiveCtx.hql("DROP TABLE src") + ... supress = hiveCtx.sql("DROP TABLE src") ... except Exception: ... pass >>> kv1 = os.path.join(os.environ["SPARK_HOME"], ... 'examples/src/main/resources/kv1.txt') - >>> supress = hiveCtx.hql( + >>> supress = hiveCtx.sql( ... "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - >>> supress = hiveCtx.hql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" + >>> supress = hiveCtx.sql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" ... % kv1) - >>> results = hiveCtx.hql("FROM src SELECT value" + >>> results = hiveCtx.sql("FROM src SELECT value" ... ).map(lambda r: int(r.value.split('_')[1])) >>> num = results.count() >>> reduce_sum = results.reduce(lambda x, y: x + y) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 2d407077be303..40bfd55e95a12 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -29,6 +29,7 @@ object SQLConf { val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" val JOIN_BROADCAST_TABLES = "spark.sql.join.broadcastTables" val CODEGEN_ENABLED = "spark.sql.codegen" + val DIALECT = "spark.sql.dialect" object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" @@ -39,7 +40,7 @@ object SQLConf { * A trait that enables the setting and getting of mutable config parameters/hints. * * In the presence of a SQLContext, these can be set and queried by passing SET commands - * into Spark SQL's query functions (sql(), hql(), etc.). Otherwise, users of this trait can + * into Spark SQL's query functions (i.e. sql()). Otherwise, users of this trait can * modify the hints by programmatically calling the setters and getters of this trait. * * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). @@ -53,6 +54,20 @@ trait SQLConf { /** ************************ Spark SQL Params/Hints ******************* */ // TODO: refactor so that these hints accessors don't pollute the name space of SQLContext? + /** + * The SQL dialect that is used when parsing queries. This defaults to 'sql' which uses + * a simple SQL parser provided by Spark SQL. This is currently the only option for users of + * SQLContext. + * + * When using a HiveContext, this value defaults to 'hiveql', which uses the Hive 0.12.0 HiveQL + * parser. Users can change this to 'sql' if they want to run queries that aren't supported by + * HiveQL (e.g., SELECT 1). + * + * Note that the choice of dialect does not affect things like what tables are available or + * how query execution is performed. + */ + private[spark] def dialect: String = get(DIALECT, "sql") + /** When true tables cached using the in-memory columnar caching will be compressed. */ private[spark] def useCompression: Boolean = get(COMPRESS_CACHED, "false").toBoolean diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 567f4dca991b2..ecd5fbaa0b094 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -248,11 +248,18 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Executes a SQL query using Spark, returning the result as a SchemaRDD. + * Executes a SQL query using Spark, returning the result as a SchemaRDD. The dialect that is + * used for SQL parsing can be configured with 'spark.sql.dialect'. * * @group userf */ - def sql(sqlText: String): SchemaRDD = new SchemaRDD(this, parseSql(sqlText)) + def sql(sqlText: String): SchemaRDD = { + if (dialect == "sql") { + new SchemaRDD(this, parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: $dialect") + } + } /** Returns the specified table as a SchemaRDD */ def table(tableName: String): SchemaRDD = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index dbaa16e8b0c68..150ff8a42063d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -39,10 +39,18 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) /** - * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD + * Executes a SQL query using Spark, returning the result as a SchemaRDD. The dialect that is + * used for SQL parsing can be configured with 'spark.sql.dialect'. + * + * @group userf */ - def sql(sqlQuery: String): JavaSchemaRDD = - new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery)) + def sql(sqlText: String): JavaSchemaRDD = { + if (sqlContext.dialect == "sql") { + new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: $sqlContext.dialect") + } + } /** * :: Experimental :: diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index d362d599d08ca..7463df1f47d43 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -55,7 +55,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo override def run(command: String): CommandProcessorResponse = { // TODO unify the error code try { - val execution = context.executePlan(context.hql(command).logicalPlan) + val execution = context.executePlan(context.sql(command).logicalPlan) hiveResponse = execution.stringResult() tableSchema = getResultSetSchema(execution) new CommandProcessorResponse(0) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index d4dadfd21d13f..dee092159dd4c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -128,7 +128,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage logInfo(s"Running query '$statement'") setState(OperationState.RUNNING) try { - result = hiveContext.hql(statement) + result = hiveContext.sql(statement) logDebug(result.queryExecution.toString()) val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 3c70b3f0921a5..7db0159512610 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -71,15 +71,29 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => + // Change the default SQL dialect to HiveQL + override private[spark] def dialect: String = get(SQLConf.DIALECT, "hiveql") + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } - /** - * Executes a query expressed in HiveQL using Spark, returning the result as a SchemaRDD. - */ + override def sql(sqlText: String): SchemaRDD = { + // TODO: Create a framework for registering parsers instead of just hardcoding if statements. + if (dialect == "sql") { + super.sql(sqlText) + } else if (dialect == "hiveql") { + new SchemaRDD(this, HiveQl.parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: $dialect. Try 'sql' or 'hiveql'") + } + } + + @deprecated("hiveql() is deprecated as the sql function now parses using HiveQL by default. " + + s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") def hiveql(hqlQuery: String): SchemaRDD = new SchemaRDD(this, HiveQl.parseSql(hqlQuery)) - /** An alias for `hiveql`. */ + @deprecated("hql() is deprecated as the sql function now parses using HiveQL by default. " + + s"The SQL dialect for parsing can be set using ${SQLConf.DIALECT}", "1.1") def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery) /** @@ -95,7 +109,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient - protected val outputBuffer = new java.io.OutputStream { + protected lazy val outputBuffer = new java.io.OutputStream { var pos: Int = 0 var buffer = new Array[Int](10240) def write(i: Int): Unit = { @@ -125,7 +139,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /** * SQLConf and HiveConf contracts: when the hive session is first initialized, params in * HiveConf will get picked up by the SQLConf. Additionally, any properties set by - * set() or a SET command inside hql() or sql() will be set in the SQLConf *as well as* + * set() or a SET command inside sql() will be set in the SQLConf *as well as* * in the HiveConf. */ @transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState]) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala index c9ee162191c96..a201d2349a2ef 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive.api.java import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD} +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.hive.{HiveContext, HiveQl} /** @@ -28,9 +29,21 @@ class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(spa override val sqlContext = new HiveContext(sparkContext) + override def sql(sqlText: String): JavaSchemaRDD = { + // TODO: Create a framework for registering parsers instead of just hardcoding if statements. + if (sqlContext.dialect == "sql") { + super.sql(sqlText) + } else if (sqlContext.dialect == "hiveql") { + new JavaSchemaRDD(sqlContext, HiveQl.parseSql(sqlText)) + } else { + sys.error(s"Unsupported SQL dialect: ${sqlContext.dialect}. Try 'sql' or 'hiveql'") + } + } + /** - * Executes a query expressed in HiveQL, returning the result as a JavaSchemaRDD. + * DEPRECATED: Use sql(...) Instead */ + @Deprecated def hql(hqlQuery: String): JavaSchemaRDD = new JavaSchemaRDD(sqlContext, HiveQl.parseSql(hqlQuery)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 08da6405a17c6..188579edd7bdd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -35,17 +35,17 @@ class CachedTableSuite extends HiveComparisonTest { "SELECT * FROM src LIMIT 1", reset = false) test("Drop cached table") { - hql("CREATE TABLE test(a INT)") + sql("CREATE TABLE test(a INT)") cacheTable("test") - hql("SELECT * FROM test").collect() - hql("DROP TABLE test") + sql("SELECT * FROM test").collect() + sql("DROP TABLE test") intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] { - hql("SELECT * FROM test").collect() + sql("SELECT * FROM test").collect() } } test("DROP nonexistant table") { - hql("DROP TABLE IF EXISTS nonexistantTable") + sql("DROP TABLE IF EXISTS nonexistantTable") } test("check that table is cached and uncache") { @@ -74,14 +74,14 @@ class CachedTableSuite extends HiveComparisonTest { } test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") { - TestHive.hql("CACHE TABLE src") + TestHive.sql("CACHE TABLE src") TestHive.table("src").queryExecution.executedPlan match { case _: InMemoryColumnarTableScan => // Found evidence of caching case _ => fail(s"Table 'src' should be cached") } assert(TestHive.isCached("src"), "Table 'src' should be cached") - TestHive.hql("UNCACHE TABLE src") + TestHive.sql("UNCACHE TABLE src") TestHive.table("src").queryExecution.executedPlan match { case _: InMemoryColumnarTableScan => fail(s"Table 'src' should not be cached") case _ => // Found evidence of uncaching diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index a61fd9df95c94..d8c77d6021d63 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.hive.test.TestHive._ class StatisticsSuite extends QueryTest { test("estimates the size of a test MetastoreRelation") { - val rdd = hql("""SELECT * FROM src""") + val rdd = sql("""SELECT * FROM src""") val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => mr.statistics.sizeInBytes } @@ -45,7 +45,7 @@ class StatisticsSuite extends QueryTest { ct: ClassTag[_]) = { before() - var rdd = hql(query) + var rdd = sql(query) // Assert src has a size smaller than the threshold. val sizes = rdd.queryExecution.analyzed.collect { @@ -65,8 +65,8 @@ class StatisticsSuite extends QueryTest { TestHive.settings.synchronized { val tmp = autoBroadcastJoinThreshold - hql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""") - rdd = hql(query) + sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=-1""") + rdd = sql(query) bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") @@ -74,7 +74,7 @@ class StatisticsSuite extends QueryTest { assert(shj.size === 1, "ShuffledHashJoin should be planned when BroadcastHashJoin is turned off") - hql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""") + sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""") } after() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala index 578f27574ad2f..9644b707eb1a0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala @@ -40,7 +40,7 @@ class JavaHiveQLSuite extends FunSuite { ignore("SELECT * FROM src") { assert( - javaHiveCtx.hql("SELECT * FROM src").collect().map(_.getInt(0)) === + javaHiveCtx.sql("SELECT * FROM src").collect().map(_.getInt(0)) === TestHive.sql("SELECT * FROM src").collect().map(_.getInt(0)).toSeq) } @@ -56,33 +56,34 @@ class JavaHiveQLSuite extends FunSuite { val tableName = "test_native_commands" assertResult(0) { - javaHiveCtx.hql(s"DROP TABLE IF EXISTS $tableName").count() + javaHiveCtx.sql(s"DROP TABLE IF EXISTS $tableName").count() } assertResult(0) { - javaHiveCtx.hql(s"CREATE TABLE $tableName(key INT, value STRING)").count() + javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)").count() } - javaHiveCtx.hql("SHOW TABLES").registerTempTable("show_tables") + javaHiveCtx.sql("SHOW TABLES").registerTempTable("show_tables") assert( javaHiveCtx - .hql("SELECT result FROM show_tables") + .sql("SELECT result FROM show_tables") .collect() .map(_.getString(0)) .contains(tableName)) assertResult(Array(Array("key", "int", "None"), Array("value", "string", "None"))) { - javaHiveCtx.hql(s"DESCRIBE $tableName").registerTempTable("describe_table") + javaHiveCtx.sql(s"DESCRIBE $tableName").registerTempTable("describe_table") + javaHiveCtx - .hql("SELECT result FROM describe_table") + .sql("SELECT result FROM describe_table") .collect() .map(_.getString(0).split("\t").map(_.trim)) .toArray } - assert(isExplanation(javaHiveCtx.hql( + assert(isExplanation(javaHiveCtx.sql( s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key"))) TestHive.reset() @@ -90,7 +91,7 @@ class JavaHiveQLSuite extends FunSuite { ignore("Exactly once semantics for DDL and command statements") { val tableName = "test_exactly_once" - val q0 = javaHiveCtx.hql(s"CREATE TABLE $tableName(key INT, value STRING)") + val q0 = javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)") // If the table was not created, the following assertion would fail assert(Try(TestHive.table(tableName)).isSuccess) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 83cfbc6b4a002..0ebaf6ffd5458 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -241,13 +241,13 @@ abstract class HiveComparisonTest val quotes = "\"\"\"" queryList.zipWithIndex.map { case (query, i) => - s"""val q$i = hql($quotes$query$quotes); q$i.collect()""" + s"""val q$i = sql($quotes$query$quotes); q$i.collect()""" }.mkString("\n== Console version of this test ==\n", "\n", "\n") } try { // MINOR HACK: You must run a query before calling reset the first time. - TestHive.hql("SHOW TABLES") + TestHive.sql("SHOW TABLES") if (reset) { TestHive.reset() } val hiveCacheFiles = queryList.zipWithIndex.map { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 4ed41550cf530..aa810a291231a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -57,8 +57,8 @@ class HiveQuerySuite extends HiveComparisonTest { """.stripMargin) test("CREATE TABLE AS runs once") { - hql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect() - assert(hql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1, + sql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect() + assert(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1, "Incorrect number of rows in created table") } @@ -72,12 +72,14 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1") test("Query expressed in SQL") { + set("spark.sql.dialect", "sql") assert(sql("SELECT 1").collect() === Array(Seq(1))) + set("spark.sql.dialect", "hiveql") + } test("Query expressed in HiveQL") { - hql("FROM src SELECT key").collect() - hiveql("FROM src SELECT key").collect() + sql("FROM src SELECT key").collect() } createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT", @@ -193,12 +195,12 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v") test("sampling") { - hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") + sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") } test("SchemaRDD toString") { - hql("SHOW TABLES").toString - hql("SELECT * FROM src").toString + sql("SHOW TABLES").toString + sql("SELECT * FROM src").toString } createQueryTest("case statements with key #1", @@ -226,8 +228,8 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15") test("implement identity function using case statement") { - val actual = hql("SELECT (CASE key WHEN key THEN key END) FROM src").collect().toSet - val expected = hql("SELECT key FROM src").collect().toSet + val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src").collect().toSet + val expected = sql("SELECT key FROM src").collect().toSet assert(actual === expected) } @@ -235,7 +237,7 @@ class HiveQuerySuite extends HiveComparisonTest { // See https://github.com/apache/spark/pull/1055#issuecomment-45820167 for a discussion. ignore("non-boolean conditions in a CaseWhen are illegal") { intercept[Exception] { - hql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect() + sql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect() } } @@ -250,7 +252,7 @@ class HiveQuerySuite extends HiveComparisonTest { testData.registerTempTable("REGisteredTABle") assertResult(Array(Array(2, "str2"))) { - hql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + + sql("SELECT tablealias.A, TABLEALIAS.b FROM reGisteredTABle TableAlias " + "WHERE TableAliaS.a > 1").collect() } } @@ -261,9 +263,9 @@ class HiveQuerySuite extends HiveComparisonTest { } test("SPARK-1704: Explain commands as a SchemaRDD") { - hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - val rdd = hql("explain select key, count(value) from src group by key") + val rdd = sql("explain select key, count(value) from src group by key") assert(isExplanation(rdd)) TestHive.reset() @@ -274,7 +276,7 @@ class HiveQuerySuite extends HiveComparisonTest { .zipWithIndex.map {case Pair(Pair(value, attr), key) => HavingRow(key, value, attr)} TestHive.sparkContext.parallelize(fixture).registerTempTable("having_test") val results = - hql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") + sql("SELECT value, max(attr) AS attr FROM having_test GROUP BY value HAVING attr > 3") .collect() .map(x => Pair(x.getString(0), x.getInt(1))) @@ -283,39 +285,39 @@ class HiveQuerySuite extends HiveComparisonTest { } test("SPARK-2180: HAVING with non-boolean clause raises no exceptions") { - hql("select key, count(*) c from src group by key having c").collect() + sql("select key, count(*) c from src group by key having c").collect() } test("SPARK-2225: turn HAVING without GROUP BY into a simple filter") { - assert(hql("select key from src having key > 490").collect().size < 100) + assert(sql("select key from src having key > 490").collect().size < 100) } test("Query Hive native command execution result") { val tableName = "test_native_commands" assertResult(0) { - hql(s"DROP TABLE IF EXISTS $tableName").count() + sql(s"DROP TABLE IF EXISTS $tableName").count() } assertResult(0) { - hql(s"CREATE TABLE $tableName(key INT, value STRING)").count() + sql(s"CREATE TABLE $tableName(key INT, value STRING)").count() } assert( - hql("SHOW TABLES") + sql("SHOW TABLES") .select('result) .collect() .map(_.getString(0)) .contains(tableName)) - assert(isExplanation(hql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key"))) + assert(isExplanation(sql(s"EXPLAIN SELECT key, COUNT(*) FROM $tableName GROUP BY key"))) TestHive.reset() } test("Exactly once semantics for DDL and command statements") { val tableName = "test_exactly_once" - val q0 = hql(s"CREATE TABLE $tableName(key INT, value STRING)") + val q0 = sql(s"CREATE TABLE $tableName(key INT, value STRING)") // If the table was not created, the following assertion would fail assert(Try(table(tableName)).isSuccess) @@ -325,9 +327,9 @@ class HiveQuerySuite extends HiveComparisonTest { } test("DESCRIBE commands") { - hql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)") + sql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)") - hql( + sql( """FROM src INSERT OVERWRITE TABLE test_describe_commands1 PARTITION (dt='2008-06-08') |SELECT key, value """.stripMargin) @@ -342,7 +344,7 @@ class HiveQuerySuite extends HiveComparisonTest { Array("# col_name", "data_type", "comment"), Array("dt", "string", null)) ) { - hql("DESCRIBE test_describe_commands1") + sql("DESCRIBE test_describe_commands1") .select('col_name, 'data_type, 'comment) .collect() } @@ -357,14 +359,14 @@ class HiveQuerySuite extends HiveComparisonTest { Array("# col_name", "data_type", "comment"), Array("dt", "string", null)) ) { - hql("DESCRIBE default.test_describe_commands1") + sql("DESCRIBE default.test_describe_commands1") .select('col_name, 'data_type, 'comment) .collect() } // Describe a column is a native command assertResult(Array(Array("value", "string", "from deserializer"))) { - hql("DESCRIBE test_describe_commands1 value") + sql("DESCRIBE test_describe_commands1 value") .select('result) .collect() .map(_.getString(0).split("\t").map(_.trim)) @@ -372,7 +374,7 @@ class HiveQuerySuite extends HiveComparisonTest { // Describe a column is a native command assertResult(Array(Array("value", "string", "from deserializer"))) { - hql("DESCRIBE default.test_describe_commands1 value") + sql("DESCRIBE default.test_describe_commands1 value") .select('result) .collect() .map(_.getString(0).split("\t").map(_.trim)) @@ -390,7 +392,7 @@ class HiveQuerySuite extends HiveComparisonTest { Array("", "", ""), Array("dt", "string", "None")) ) { - hql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") + sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") .select('result) .collect() .map(_.getString(0).split("\t").map(_.trim)) @@ -409,16 +411,16 @@ class HiveQuerySuite extends HiveComparisonTest { Array("a", "IntegerType", null), Array("b", "StringType", null)) ) { - hql("DESCRIBE test_describe_commands2") + sql("DESCRIBE test_describe_commands2") .select('col_name, 'data_type, 'comment) .collect() } } test("SPARK-2263: Insert Map values") { - hql("CREATE TABLE m(value MAP)") - hql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") - hql("SELECT * FROM m").collect().zip(hql("SELECT * FROM src LIMIT 10").collect()).map { + sql("CREATE TABLE m(value MAP)") + sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") + sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).map { case (Row(map: Map[_, _]), Row(key: Int, value: String)) => assert(map.size === 1) assert(map.head === (key, value)) @@ -430,18 +432,18 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "val0,val_1,val2.3,my_table" - hql(s"set $testKey=$testVal") + sql(s"set $testKey=$testVal") assert(get(testKey, testVal + "_") == testVal) - hql("set some.property=20") + sql("set some.property=20") assert(get("some.property", "0") == "20") - hql("set some.property = 40") + sql("set some.property = 40") assert(get("some.property", "0") == "40") - hql(s"set $testKey=$testVal") + sql(s"set $testKey=$testVal") assert(get(testKey, "0") == testVal) - hql(s"set $testKey=") + sql(s"set $testKey=") assert(get(testKey, "0") == "") } @@ -454,33 +456,34 @@ class HiveQuerySuite extends HiveComparisonTest { clear() // "set" itself returns all config variables currently specified in SQLConf. - assert(hql("SET").collect().size == 0) + // TODO: Should we be listing the default here always? probably... + assert(sql("SET").collect().size == 0) assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } assert(hiveconf.get(testKey, "") == testVal) assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) } - hql(s"SET ${testKey + testKey}=${testVal + testVal}") + sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - hql(s"SET").collect().map(_.getString(0)) + sql(s"SET").collect().map(_.getString(0)) } // "set key" assertResult(Array(s"$testKey=$testVal")) { - hql(s"SET $testKey").collect().map(_.getString(0)) + sql(s"SET $testKey").collect().map(_.getString(0)) } assertResult(Array(s"$nonexistentKey=")) { - hql(s"SET $nonexistentKey").collect().map(_.getString(0)) + sql(s"SET $nonexistentKey").collect().map(_.getString(0)) } - // Assert that sql() should have the same effects as hql() by repeating the above using sql(). + // Assert that sql() should have the same effects as sql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) 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 2455c18925dfa..6b3ffd1c0ffe2 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 @@ -56,13 +56,13 @@ class HiveResolutionSuite extends HiveComparisonTest { TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerTempTable("caseSensitivityTest") - hql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + sql("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) - .registerTempTable("nestedRepeatedTest") - assert(hql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) + .registerTempTable("nestedRepeatedTest") + assert(sql("SELECT nestedArray[0].a FROM nestedRepeatedTest").collect().head(0) === 1) } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index 7436de264a1e1..c3c18cf8ccac3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -35,7 +35,7 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { test("[SPARK-2210] boolean cast on boolean value should be removed") { val q = "select cast(cast(key=0 as boolean) as boolean) from src" - val project = TestHive.hql(q).queryExecution.executedPlan.collect { case e: Project => e }.head + val project = TestHive.sql(q).queryExecution.executedPlan.collect { case e: Project => e }.head // No cast expression introduced project.transformAllExpressions { case c: Cast => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index f944d010660eb..b6b8592344ef5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject */ class HiveUdfSuite extends HiveComparisonTest { - TestHive.hql( + TestHive.sql( """ |CREATE EXTERNAL TABLE hiveUdfTestTable ( | pair STRUCT @@ -48,16 +48,16 @@ class HiveUdfSuite extends HiveComparisonTest { """.stripMargin.format(classOf[PairSerDe].getName) ) - TestHive.hql( + TestHive.sql( "ALTER TABLE hiveUdfTestTable ADD IF NOT EXISTS PARTITION(partition='testUdf') LOCATION '%s'" .format(this.getClass.getClassLoader.getResource("data/files/testUdf").getFile) ) - TestHive.hql("CREATE TEMPORARY FUNCTION testUdf AS '%s'".format(classOf[PairUdf].getName)) + TestHive.sql("CREATE TEMPORARY FUNCTION testUdf AS '%s'".format(classOf[PairUdf].getName)) - TestHive.hql("SELECT testUdf(pair) FROM hiveUdfTestTable") + TestHive.sql("SELECT testUdf(pair) FROM hiveUdfTestTable") - TestHive.hql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") + TestHive.sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") } class TestPair(x: Int, y: Int) extends Writable with Serializable { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 34d8a061ccc83..1a6dbc0ce0c0d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -27,7 +27,7 @@ import scala.collection.JavaConversions._ */ class PruningSuite extends HiveComparisonTest { // MINOR HACK: You must run a query before calling reset the first time. - TestHive.hql("SHOW TABLES") + TestHive.sql("SHOW TABLES") // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset // the environment to ensure all referenced tables in this suites are not cached in-memory. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 6545e8d7dcb69..6f57fe8958387 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -68,39 +68,40 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft .saveAsParquetFile(tempFile.getCanonicalPath) parquetFile(tempFile.getCanonicalPath).registerTempTable("cases") - hql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) - hql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) + sql("SELECT upper FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) + sql("SELECT LOWER FROM cases").collect().map(_.getString(0)) === (1 to 10).map(_.toString) } test("SELECT on Parquet table") { - val rdd = hql("SELECT * FROM testsource").collect() + val rdd = sql("SELECT * FROM testsource").collect() assert(rdd != null) assert(rdd.forall(_.size == 6)) } test("Simple column projection + filter on Parquet table") { - val rdd = hql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() + val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() assert(rdd.size === 5, "Filter returned incorrect number of rows") assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value") } test("Converting Hive to Parquet Table via saveAsParquetFile") { - hql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) + sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") - val rddOne = hql("SELECT * FROM src").collect().sortBy(_.getInt(0)) - val rddTwo = hql("SELECT * from ptable").collect().sortBy(_.getInt(0)) + val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0)) + val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0)) + compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) } test("INSERT OVERWRITE TABLE Parquet table") { - hql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) + sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) parquetFile(dirname.getAbsolutePath).registerTempTable("ptable") // let's do three overwrites for good measure - hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - val rddCopy = hql("SELECT * FROM ptable").collect() - val rddOrig = hql("SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + val rddCopy = sql("SELECT * FROM ptable").collect() + val rddOrig = sql("SELECT * FROM testsource").collect() assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames) } From ac33cbbf33bd1ab29bc8165c9be02fb8934b1fdf Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 3 Aug 2014 12:34:46 -0700 Subject: [PATCH 0231/1492] [SPARK-2814][SQL] HiveThriftServer2 throws NPE when executing native commands JIRA issue: [SPARK-2814](https://issues.apache.org/jira/browse/SPARK-2814) Author: Cheng Lian Closes #1753 from liancheng/spark-2814 and squashes the following commits: c74a3b2 [Cheng Lian] Fixed SPARK-2814 --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 7db0159512610..acad681f68b14 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -146,13 +146,12 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @transient protected[hive] lazy val sessionState = { val ss = new SessionState(hiveconf) set(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. - - ss.err = new PrintStream(outputBuffer, true, "UTF-8") - ss.out = new PrintStream(outputBuffer, true, "UTF-8") - ss } + sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") + sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") + override def set(key: String, value: String): Unit = { super.set(key, value) runSqlHive(s"SET $key=$value") From e139e2be60ef23281327744e1b3e74904dfdf63f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 3 Aug 2014 14:54:41 -0700 Subject: [PATCH 0232/1492] [SPARK-2783][SQL] Basic support for analyze in HiveContext JIRA: https://issues.apache.org/jira/browse/SPARK-2783 Author: Yin Huai Closes #1741 from yhuai/analyzeTable and squashes the following commits: 7bb5f02 [Yin Huai] Use sql instead of hql. 4d09325 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable e3ebcd4 [Yin Huai] Renaming. c170f4e [Yin Huai] Do not use getContentSummary. 62393b6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable db233a6 [Yin Huai] Trying to debug jenkins... fee84f0 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable f0501f3 [Yin Huai] Fix compilation error. 24ad391 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable 8918140 [Yin Huai] Wording. 23df227 [Yin Huai] Add a simple analyze method to get the size of a table and update the "totalSize" property of this table in the Hive metastore. --- .../apache/spark/sql/hive/HiveContext.scala | 79 +++++++++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 5 +- .../spark/sql/hive/StatisticsSuite.scala | 54 +++++++++++++ 3 files changed, 136 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index acad681f68b14..d8e7a5943daa5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -25,10 +25,14 @@ import scala.collection.JavaConversions._ import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.io.TimestampWritable import org.apache.spark.SparkContext @@ -107,6 +111,81 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { catalog.createTable("default", tableName, ScalaReflection.attributesFor[A], allowExisting) } + /** + * Analyzes the given table in the current database to generate statistics, which will be + * used in query optimizations. + * + * Right now, it only supports Hive tables and it only updates the size of a Hive table + * in the Hive metastore. + */ + def analyze(tableName: String) { + val relation = catalog.lookupRelation(None, tableName) match { + case LowerCaseSchema(r) => r + case o => o + } + + relation match { + case relation: MetastoreRelation => { + // This method is mainly based on + // org.apache.hadoop.hive.ql.stats.StatsUtils.getFileSizeForTable(HiveConf, Table) + // in Hive 0.13 (except that we do not use fs.getContentSummary). + // TODO: Generalize statistics collection. + // TODO: Why fs.getContentSummary returns wrong size on Jenkins? + // Can we use fs.getContentSummary in future? + // Seems fs.getContentSummary returns wrong table size on Jenkins. So we use + // countFileSize to count the table size. + def calculateTableSize(fs: FileSystem, path: Path): Long = { + val fileStatus = fs.getFileStatus(path) + val size = if (fileStatus.isDir) { + fs.listStatus(path).map(status => calculateTableSize(fs, status.getPath)).sum + } else { + fileStatus.getLen + } + + size + } + + def getFileSizeForTable(conf: HiveConf, table: Table): Long = { + val path = table.getPath() + var size: Long = 0L + try { + val fs = path.getFileSystem(conf) + size = calculateTableSize(fs, path) + } catch { + case e: Exception => + logWarning( + s"Failed to get the size of table ${table.getTableName} in the " + + s"database ${table.getDbName} because of ${e.toString}", e) + size = 0L + } + + size + } + + val tableParameters = relation.hiveQlTable.getParameters + val oldTotalSize = + Option(tableParameters.get(StatsSetupConst.TOTAL_SIZE)).map(_.toLong).getOrElse(0L) + val newTotalSize = getFileSizeForTable(hiveconf, relation.hiveQlTable) + // Update the Hive metastore if the total size of the table is different than the size + // recorded in the Hive metastore. + // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). + if (newTotalSize > 0 && newTotalSize != oldTotalSize) { + tableParameters.put(StatsSetupConst.TOTAL_SIZE, newTotalSize.toString) + val hiveTTable = relation.hiveQlTable.getTTable + hiveTTable.setParameters(tableParameters) + val tableFullName = + relation.hiveQlTable.getDbName() + "." + relation.hiveQlTable.getTableName() + + catalog.client.alterTable(tableFullName, new Table(hiveTTable)) + } + } + case otherRelation => + throw new NotImplementedError( + s"Analyze has only implemented for Hive tables, " + + s"but ${tableName} is a ${otherRelation.nodeName}") + } + } + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient protected lazy val outputBuffer = new java.io.OutputStream { 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 df3604439e483..301cf51c00e2b 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 @@ -25,6 +25,7 @@ import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, Ser import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.annotation.DeveloperApi @@ -278,9 +279,9 @@ private[hive] case class MetastoreRelation // relatively cheap if parameters for the table are populated into the metastore. An // alternative would be going through Hadoop's FileSystem API, which can be expensive if a lot // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, - // `rawDataSize` keys that we can look at in the future. + // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. BigInt( - Option(hiveQlTable.getParameters.get("totalSize")) + Option(hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)) .map(_.toLong) .getOrElse(sqlContext.defaultSizeInBytes)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index d8c77d6021d63..bf5931bbf97ee 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -26,6 +26,60 @@ import org.apache.spark.sql.hive.test.TestHive._ class StatisticsSuite extends QueryTest { + test("analyze MetastoreRelations") { + def queryTotalSize(tableName: String): BigInt = + catalog.lookupRelation(None, tableName).statistics.sizeInBytes + + // Non-partitioned table + sql("CREATE TABLE analyzeTable (key STRING, value STRING)").collect() + sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() + sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() + + assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) + + analyze("analyzeTable") + + assert(queryTotalSize("analyzeTable") === BigInt(11624)) + + sql("DROP TABLE analyzeTable").collect() + + // Partitioned table + sql( + """ + |CREATE TABLE analyzeTable_part (key STRING, value STRING) PARTITIONED BY (ds STRING) + """.stripMargin).collect() + sql( + """ + |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-01') + |SELECT * FROM src + """.stripMargin).collect() + sql( + """ + |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-02') + |SELECT * FROM src + """.stripMargin).collect() + sql( + """ + |INSERT INTO TABLE analyzeTable_part PARTITION (ds='2010-01-03') + |SELECT * FROM src + """.stripMargin).collect() + + assert(queryTotalSize("analyzeTable_part") === defaultSizeInBytes) + + analyze("analyzeTable_part") + + assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) + + sql("DROP TABLE analyzeTable_part").collect() + + // Try to analyze a temp table + sql("""SELECT * FROM src""").registerTempTable("tempTable") + intercept[NotImplementedError] { + analyze("tempTable") + } + catalog.unregisterTable(None, "tempTable") + } + test("estimates the size of a test MetastoreRelation") { val rdd = sql("""SELECT * FROM src""") val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => From 55349f9fe81ba5af5e4a5e4908ebf174e63c6cc9 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 3 Aug 2014 15:52:00 -0700 Subject: [PATCH 0233/1492] [SPARK-1740] [PySpark] kill the python worker Kill only the python worker related to cancelled tasks. The daemon will start a background thread to monitor all the opened sockets for all workers. If the socket is closed by JVM, this thread will kill the worker. When an task is cancelled, the socket to worker will be closed, then the worker will be killed by deamon. Author: Davies Liu Closes #1643 from davies/kill and squashes the following commits: 8ffe9f3 [Davies Liu] kill worker by deamon, because runtime.exec() is too heavy 46ca150 [Davies Liu] address comment acd751c [Davies Liu] kill the worker when task is canceled --- .../scala/org/apache/spark/SparkEnv.scala | 5 +- .../apache/spark/api/python/PythonRDD.scala | 9 ++- .../api/python/PythonWorkerFactory.scala | 64 ++++++++++++++----- python/pyspark/daemon.py | 24 +++++-- python/pyspark/tests.py | 51 +++++++++++++++ 5 files changed, 125 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 92c809d854167..0bce531aaba3e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -18,6 +18,7 @@ package org.apache.spark import java.io.File +import java.net.Socket import scala.collection.JavaConversions._ import scala.collection.mutable @@ -102,10 +103,10 @@ class SparkEnv ( } private[spark] - def destroyPythonWorker(pythonExec: String, envVars: Map[String, String]) { + def destroyPythonWorker(pythonExec: String, envVars: Map[String, String], worker: Socket) { synchronized { val key = (pythonExec, envVars) - pythonWorkers(key).stop() + pythonWorkers.get(key).foreach(_.stopWorker(worker)) } } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index fe9a9e50ef21d..0b5322c6fb965 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -62,8 +62,8 @@ private[spark] class PythonRDD( val env = SparkEnv.get val localdir = env.blockManager.diskBlockManager.localDirs.map( f => f.getPath()).mkString(",") - val worker: Socket = env.createPythonWorker(pythonExec, - envVars.toMap + ("SPARK_LOCAL_DIR" -> localdir)) + envVars += ("SPARK_LOCAL_DIR" -> localdir) // it's also used in monitor thread + val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator val writerThread = new WriterThread(env, worker, split, context) @@ -241,7 +241,7 @@ private[spark] class PythonRDD( if (!context.completed) { try { logWarning("Incomplete task interrupted: Attempting to kill Python Worker") - env.destroyPythonWorker(pythonExec, envVars.toMap) + env.destroyPythonWorker(pythonExec, envVars.toMap, worker) } catch { case e: Exception => logError("Exception when trying to kill worker", e) @@ -685,9 +685,8 @@ private[spark] object PythonRDD extends Logging { /** * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions). - * This function is outdated, PySpark does not use it anymore */ - @deprecated + @deprecated("PySpark does not use it anymore", "1.1") def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { pyRDD.rdd.mapPartitions { iter => val unpickle = new Unpickler diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 15fe8a9be6bfe..7af260d0b7f26 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,9 +17,11 @@ package org.apache.spark.api.python -import java.io.{DataInputStream, InputStream, OutputStreamWriter} +import java.lang.Runtime +import java.io.{DataOutputStream, DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} +import scala.collection.mutable import scala.collection.JavaConversions._ import org.apache.spark._ @@ -39,6 +41,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 + var daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + + var simpleWorkers = new mutable.WeakHashMap[Socket, Process]() val pythonPath = PythonUtils.mergePythonPaths( PythonUtils.sparkPythonPath, @@ -58,25 +63,31 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String * to avoid the high cost of forking from Java. This currently only works on UNIX-based systems. */ private def createThroughDaemon(): Socket = { + + def createSocket(): Socket = { + val socket = new Socket(daemonHost, daemonPort) + val pid = new DataInputStream(socket.getInputStream).readInt() + if (pid < 0) { + throw new IllegalStateException("Python daemon failed to launch worker") + } + daemonWorkers.put(socket, pid) + socket + } + synchronized { // Start the daemon if it hasn't been started startDaemon() // Attempt to connect, restart and retry once if it fails try { - val socket = new Socket(daemonHost, daemonPort) - val launchStatus = new DataInputStream(socket.getInputStream).readInt() - if (launchStatus != 0) { - throw new IllegalStateException("Python daemon failed to launch worker") - } - socket + createSocket() } catch { case exc: SocketException => logWarning("Failed to open socket to Python daemon:", exc) logWarning("Assuming that daemon unexpectedly quit, attempting to restart") stopDaemon() startDaemon() - new Socket(daemonHost, daemonPort) + createSocket() } } } @@ -107,7 +118,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // Wait for it to connect to our socket serverSocket.setSoTimeout(10000) try { - return serverSocket.accept() + val socket = serverSocket.accept() + simpleWorkers.put(socket, worker) + return socket } catch { case e: Exception => throw new SparkException("Python worker did not connect back in time", e) @@ -189,19 +202,40 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String private def stopDaemon() { synchronized { - // Request shutdown of existing daemon by sending SIGTERM - if (daemon != null) { - daemon.destroy() - } + if (useDaemon) { + // Request shutdown of existing daemon by sending SIGTERM + if (daemon != null) { + daemon.destroy() + } - daemon = null - daemonPort = 0 + daemon = null + daemonPort = 0 + } else { + simpleWorkers.mapValues(_.destroy()) + } } } def stop() { stopDaemon() } + + def stopWorker(worker: Socket) { + if (useDaemon) { + if (daemon != null) { + daemonWorkers.get(worker).foreach { pid => + // tell daemon to kill worker by pid + val output = new DataOutputStream(daemon.getOutputStream) + output.writeInt(pid) + output.flush() + daemon.getOutputStream.flush() + } + } + } else { + simpleWorkers.get(worker).foreach(_.destroy()) + } + worker.close() + } } private object PythonWorkerFactory { diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 9fde0dde0f4b4..b00da833d06f1 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -26,7 +26,7 @@ from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN from pyspark.worker import main as worker_main -from pyspark.serializers import write_int +from pyspark.serializers import read_int, write_int def compute_real_exit_code(exit_code): @@ -67,7 +67,8 @@ def waitSocketClose(sock): outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) exit_code = 0 try: - write_int(0, outfile) # Acknowledge that the fork was successful + # Acknowledge that the fork was successful + write_int(os.getpid(), outfile) outfile.flush() worker_main(infile, outfile) except SystemExit as exc: @@ -125,14 +126,23 @@ def handle_sigchld(*args): else: raise if 0 in ready_fds: - # Spark told us to exit by closing stdin - shutdown(0) + try: + worker_pid = read_int(sys.stdin) + except EOFError: + # Spark told us to exit by closing stdin + shutdown(0) + try: + os.kill(worker_pid, signal.SIGKILL) + except OSError: + pass # process already died + + if listen_sock in ready_fds: sock, addr = listen_sock.accept() # Launch a worker process try: - fork_return_code = os.fork() - if fork_return_code == 0: + pid = os.fork() + if pid == 0: listen_sock.close() try: worker(sock) @@ -143,11 +153,13 @@ def handle_sigchld(*args): os._exit(0) else: sock.close() + except OSError as e: print >> sys.stderr, "Daemon failed to fork PySpark worker: %s" % e outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) write_int(-1, outfile) # Signal that the fork failed outfile.flush() + outfile.close() sock.close() finally: shutdown(1) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 16fb5a9256220..acc3c30371621 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -790,6 +790,57 @@ def test_termination_sigterm(self): self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) +class TestWorker(PySparkTestCase): + def test_cancel_task(self): + temp = tempfile.NamedTemporaryFile(delete=True) + temp.close() + path = temp.name + def sleep(x): + import os, time + with open(path, 'w') as f: + f.write("%d %d" % (os.getppid(), os.getpid())) + time.sleep(100) + + # start job in background thread + def run(): + self.sc.parallelize(range(1)).foreach(sleep) + import threading + t = threading.Thread(target=run) + t.daemon = True + t.start() + + daemon_pid, worker_pid = 0, 0 + while True: + if os.path.exists(path): + data = open(path).read().split(' ') + daemon_pid, worker_pid = map(int, data) + break + time.sleep(0.1) + + # cancel jobs + self.sc.cancelAllJobs() + t.join() + + for i in range(50): + try: + os.kill(worker_pid, 0) + time.sleep(0.1) + except OSError: + break # worker was killed + else: + self.fail("worker has not been killed after 5 seconds") + + try: + os.kill(daemon_pid, 0) + except OSError: + self.fail("daemon had been killed") + + def test_fd_leak(self): + N = 1100 # fd limit is 1024 by default + rdd = self.sc.parallelize(range(N), N) + self.assertEquals(N, rdd.count()) + + class TestSparkSubmit(unittest.TestCase): def setUp(self): self.programDir = tempfile.mkdtemp() From 6ba6c3ebfe9a47351a50e45271e241140b09bf10 Mon Sep 17 00:00:00 2001 From: Anand Avati Date: Sun, 3 Aug 2014 17:47:49 -0700 Subject: [PATCH 0234/1492] [SPARK-2810] upgrade to scala-maven-plugin 3.2.0 Needed for Scala 2.11 compiler-interface Signed-off-by: Anand Avati Author: Anand Avati Closes #1711 from avati/SPARK-1812-scala-maven-plugin and squashes the following commits: 9a22fc8 [Anand Avati] SPARK-1812: upgrade to scala-maven-plugin 3.2.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index cc9377cec2a07..4ab027bad55c0 100644 --- a/pom.xml +++ b/pom.xml @@ -782,7 +782,7 @@ net.alchim31.maven scala-maven-plugin - 3.1.6 + 3.2.0 scala-compile-first From 5507dd8e18fbb52d5e0c64a767103b2418cb09c6 Mon Sep 17 00:00:00 2001 From: Sarah Gerweck Date: Sun, 3 Aug 2014 19:47:05 -0700 Subject: [PATCH 0235/1492] Fix some bugs with spaces in directory name. Any time you use the directory name (`FWDIR`) it needs to be surrounded in quotes. If you're also using wildcards, you can safely put the quotes around just `$FWDIR`. Author: Sarah Gerweck Closes #1756 from sarahgerweck/folderSpaces and squashes the following commits: 732629d [Sarah Gerweck] Fix some bugs with spaces in directory name. --- make-distribution.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index 1441497b3995a..f7a6a9d838bb6 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -168,22 +168,22 @@ mkdir -p "$DISTDIR/lib" echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars -cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" -cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" +cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy example sources (needed for python and SQL) mkdir -p "$DISTDIR/examples/src/main" -cp -r $FWDIR/examples/src/main "$DISTDIR/examples/src/" +cp -r "$FWDIR"/examples/src/main "$DISTDIR/examples/src/" if [ "$SPARK_HIVE" == "true" ]; then - cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" + cp "$FWDIR"/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/" fi # Copy license and ASF files cp "$FWDIR/LICENSE" "$DISTDIR" cp "$FWDIR/NOTICE" "$DISTDIR" -if [ -e $FWDIR/CHANGES.txt ]; then +if [ -e "$FWDIR"/CHANGES.txt ]; then cp "$FWDIR/CHANGES.txt" "$DISTDIR" fi From ae58aea2d1435b5bb011e68127e1bcddc2edf5b2 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Sun, 3 Aug 2014 21:39:21 -0700 Subject: [PATCH 0236/1492] SPARK-2272 [MLlib] Feature scaling which standardizes the range of independent variables or features of data Feature scaling is a method used to standardize the range of independent variables or features of data. In data processing, it is generally performed during the data preprocessing step. In this work, a trait called `VectorTransformer` is defined for generic transformation on a vector. It contains one method to be implemented, `transform` which applies transformation on a vector. There are two implementations of `VectorTransformer` now, and they all can be easily extended with PMML transformation support. 1) `StandardScaler` - Standardizes features by removing the mean and scaling to unit variance using column summary statistics on the samples in the training set. 2) `Normalizer` - Normalizes samples individually to unit L^n norm Author: DB Tsai Closes #1207 from dbtsai/dbtsai-feature-scaling and squashes the following commits: 78c15d3 [DB Tsai] Alpine Data Labs --- .../spark/mllib/feature/Normalizer.scala | 76 +++++++ .../spark/mllib/feature/StandardScaler.scala | 119 +++++++++++ .../mllib/feature/VectorTransformer.scala | 51 +++++ .../mllib/linalg/distributed/RowMatrix.scala | 2 +- .../spark/mllib/feature/NormalizerSuite.scala | 120 +++++++++++ .../mllib/feature/StandardScalerSuite.scala | 200 ++++++++++++++++++ 6 files changed, 567 insertions(+), 1 deletion(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala new file mode 100644 index 0000000000000..ea9fd0a80d8e0 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -0,0 +1,76 @@ +/* + * 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.feature + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +/** + * :: DeveloperApi :: + * Normalizes samples individually to unit L^p^ norm + * + * For any 1 <= p < Double.PositiveInfinity, normalizes samples using + * sum(abs(vector).^p^)^(1/p)^ as norm. + * + * For p = Double.PositiveInfinity, max(abs(vector)) will be used as norm for normalization. + * + * @param p Normalization in L^p^ space, p = 2 by default. + */ +@DeveloperApi +class Normalizer(p: Double) extends VectorTransformer { + + def this() = this(2) + + require(p >= 1.0) + + /** + * Applies unit length normalization on a vector. + * + * @param vector vector to be normalized. + * @return normalized vector. If the norm of the input is zero, it will return the input vector. + */ + override def transform(vector: Vector): Vector = { + var norm = vector.toBreeze.norm(p) + + if (norm != 0.0) { + // For dense vector, we've to allocate new memory for new output vector. + // However, for sparse vector, the `index` array will not be changed, + // so we can re-use it to save memory. + vector.toBreeze match { + case dv: BDV[Double] => Vectors.fromBreeze(dv :/ norm) + case sv: BSV[Double] => + val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + var i = 0 + while (i < output.data.length) { + output.data(i) /= norm + i += 1 + } + Vectors.fromBreeze(output) + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + } else { + // Since the norm is zero, return the input vector object itself. + // Note that it's safe since we always assume that the data in RDD + // should be immutable. + vector + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala new file mode 100644 index 0000000000000..cc2d7579c2901 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -0,0 +1,119 @@ +/* + * 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.feature + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.rdd.RDD + +/** + * :: DeveloperApi :: + * Standardizes features by removing the mean and scaling to unit variance using column summary + * statistics on the samples in the training set. + * + * @param withMean False by default. Centers the data with mean before scaling. It will build a + * dense output, so this does not work on sparse input and will raise an exception. + * @param withStd True by default. Scales the data to unit standard deviation. + */ +@DeveloperApi +class StandardScaler(withMean: Boolean, withStd: Boolean) extends VectorTransformer { + + def this() = this(false, true) + + require(withMean || withStd, s"withMean and withStd both equal to false. Doing nothing.") + + private var mean: BV[Double] = _ + private var factor: BV[Double] = _ + + /** + * Computes the mean and variance and stores as a model to be used for later scaling. + * + * @param data The data used to compute the mean and variance to build the transformation model. + * @return This StandardScalar object. + */ + def fit(data: RDD[Vector]): this.type = { + val summary = data.treeAggregate(new MultivariateOnlineSummarizer)( + (aggregator, data) => aggregator.add(data), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) + + mean = summary.mean.toBreeze + factor = summary.variance.toBreeze + require(mean.length == factor.length) + + var i = 0 + while (i < factor.length) { + factor(i) = if (factor(i) != 0.0) 1.0 / math.sqrt(factor(i)) else 0.0 + i += 1 + } + + this + } + + /** + * Applies standardization transformation on a vector. + * + * @param vector Vector to be standardized. + * @return Standardized vector. If the variance of a column is zero, it will return default `0.0` + * for the column with zero variance. + */ + override def transform(vector: Vector): Vector = { + if (mean == null || factor == null) { + throw new IllegalStateException( + "Haven't learned column summary statistics yet. Call fit first.") + } + + require(vector.size == mean.length) + + if (withMean) { + vector.toBreeze match { + case dv: BDV[Double] => + val output = vector.toBreeze.copy + var i = 0 + while (i < output.length) { + output(i) = (output(i) - mean(i)) * (if (withStd) factor(i) else 1.0) + i += 1 + } + Vectors.fromBreeze(output) + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + } else if (withStd) { + vector.toBreeze match { + case dv: BDV[Double] => Vectors.fromBreeze(dv :* factor) + case sv: BSV[Double] => + // For sparse vector, the `index` array inside sparse vector object will not be changed, + // so we can re-use it to save memory. + val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + var i = 0 + while (i < output.data.length) { + output.data(i) *= factor(output.index(i)) + i += 1 + } + Vectors.fromBreeze(output) + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + } else { + // Note that it's safe since we always assume that the data in RDD should be immutable. + vector + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala new file mode 100644 index 0000000000000..415a845332d45 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -0,0 +1,51 @@ +/* + * 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.feature + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.rdd.RDD + +/** + * :: DeveloperApi :: + * Trait for transformation of a vector + */ +@DeveloperApi +trait VectorTransformer extends Serializable { + + /** + * Applies transformation on a vector. + * + * @param vector vector to be transformed. + * @return transformed vector. + */ + def transform(vector: Vector): Vector + + /** + * Applies transformation on an RDD[Vector]. + * + * @param data RDD[Vector] to be transformed. + * @return transformed RDD[Vector]. + */ + def transform(data: RDD[Vector]): RDD[Vector] = { + // Later in #1498 , all RDD objects are sent via broadcasting instead of akka. + // So it should be no longer necessary to explicitly broadcast `this` object. + data.map(x => this.transform(x)) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 58c1322757a43..45486b2c7d82d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import java.util.Arrays -import breeze.linalg.{Vector => BV, DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV} +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV} import breeze.linalg.{svd => brzSvd, axpy => brzAxpy} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala new file mode 100644 index 0000000000000..fb76dccfdf79e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala @@ -0,0 +1,120 @@ +/* + * 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.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class NormalizerSuite extends FunSuite with LocalSparkContext { + + val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.dense(0.0, 0.0, 0.0), + Vectors.dense(0.6, -1.1, -3.0), + Vectors.sparse(3, Seq((1, 0.91), (2, 3.2))), + Vectors.sparse(3, Seq((0, 5.7), (1, 0.72), (2, 2.7))), + Vectors.sparse(3, Seq()) + ) + + lazy val dataRDD = sc.parallelize(data, 3) + + test("Normalization using L1 distance") { + val l1Normalizer = new Normalizer(1) + + val data1 = data.map(l1Normalizer.transform) + val data1RDD = l1Normalizer.transform(dataRDD) + + assert((data, data1, data1RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + + assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(data1(0).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + assert(data1(2).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + assert(data1(3).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + assert(data1(4).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + + assert(data1(0) ~== Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))) absTol 1E-5) + assert(data1(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(data1(2) ~== Vectors.dense(0.12765957, -0.23404255, -0.63829787) absTol 1E-5) + assert(data1(3) ~== Vectors.sparse(3, Seq((1, 0.22141119), (2, 0.7785888))) absTol 1E-5) + assert(data1(4) ~== Vectors.dense(0.625, 0.07894737, 0.29605263) absTol 1E-5) + assert(data1(5) ~== Vectors.sparse(3, Seq()) absTol 1E-5) + } + + test("Normalization using L2 distance") { + val l2Normalizer = new Normalizer() + + val data2 = data.map(l2Normalizer.transform) + val data2RDD = l2Normalizer.transform(dataRDD) + + assert((data, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(data2(0).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + assert(data2(2).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + assert(data2(3).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + assert(data2(4).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + + assert(data2(0) ~== Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))) absTol 1E-5) + assert(data2(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(data2(2) ~== Vectors.dense(0.184549876, -0.3383414, -0.922749378) absTol 1E-5) + assert(data2(3) ~== Vectors.sparse(3, Seq((1, 0.27352993), (2, 0.96186349))) absTol 1E-5) + assert(data2(4) ~== Vectors.dense(0.897906166, 0.113419726, 0.42532397) absTol 1E-5) + assert(data2(5) ~== Vectors.sparse(3, Seq()) absTol 1E-5) + } + + test("Normalization using L^Inf distance.") { + val lInfNormalizer = new Normalizer(Double.PositiveInfinity) + + val dataInf = data.map(lInfNormalizer.transform) + val dataInfRDD = lInfNormalizer.transform(dataRDD) + + assert((data, dataInf, dataInfRDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after normalization.") + + assert((dataInf, dataInfRDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(dataInf(0).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + assert(dataInf(2).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + assert(dataInf(3).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + assert(dataInf(4).toArray.map(Math.abs).max ~== 1.0 absTol 1E-5) + + assert(dataInf(0) ~== Vectors.sparse(3, Seq((0, -0.86956522), (1, 1.0))) absTol 1E-5) + assert(dataInf(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(dataInf(2) ~== Vectors.dense(0.2, -0.36666667, -1.0) absTol 1E-5) + assert(dataInf(3) ~== Vectors.sparse(3, Seq((1, 0.284375), (2, 1.0))) absTol 1E-5) + assert(dataInf(4) ~== Vectors.dense(1.0, 0.12631579, 0.473684211) absTol 1E-5) + assert(dataInf(5) ~== Vectors.sparse(3, Seq()) absTol 1E-5) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala new file mode 100644 index 0000000000000..5a9be923a8625 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala @@ -0,0 +1,200 @@ +/* + * 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.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, MultivariateOnlineSummarizer} +import org.apache.spark.rdd.RDD + +class StandardScalerSuite extends FunSuite with LocalSparkContext { + + private def computeSummary(data: RDD[Vector]): MultivariateStatisticalSummary = { + data.treeAggregate(new MultivariateOnlineSummarizer)( + (aggregator, data) => aggregator.add(data), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) + } + + test("Standardization with dense input") { + val data = Array( + Vectors.dense(-2.0, 2.3, 0), + Vectors.dense(0.0, -1.0, -3.0), + Vectors.dense(0.0, -5.1, 0.0), + Vectors.dense(3.8, 0.0, 1.9), + Vectors.dense(1.7, -0.6, 0.0), + Vectors.dense(0.0, 1.9, 0.0) + ) + + val dataRDD = sc.parallelize(data, 3) + + val standardizer1 = new StandardScaler(withMean = true, withStd = true) + val standardizer2 = new StandardScaler() + val standardizer3 = new StandardScaler(withMean = true, withStd = false) + + withClue("Using a standardizer before fitting the model should throw exception.") { + intercept[IllegalStateException] { + data.map(standardizer1.transform) + } + } + + standardizer1.fit(dataRDD) + standardizer2.fit(dataRDD) + standardizer3.fit(dataRDD) + + val data1 = data.map(standardizer1.transform) + val data2 = data.map(standardizer2.transform) + val data3 = data.map(standardizer3.transform) + + val data1RDD = standardizer1.transform(dataRDD) + val data2RDD = standardizer2.transform(dataRDD) + val data3RDD = standardizer3.transform(dataRDD) + + val summary = computeSummary(dataRDD) + val summary1 = computeSummary(data1RDD) + val summary2 = computeSummary(data2RDD) + val summary3 = computeSummary(data3RDD) + + assert((data, data1, data1RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data, data3, data3RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + assert((data3, data3RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(summary1.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary1.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + + assert(summary2.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary2.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + + assert(summary3.mean ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary3.variance ~== summary.variance absTol 1E-5) + + assert(data1(0) ~== Vectors.dense(-1.31527964, 1.023470449, 0.11637768424) absTol 1E-5) + assert(data1(3) ~== Vectors.dense(1.637735298, 0.156973995, 1.32247368462) absTol 1E-5) + assert(data2(4) ~== Vectors.dense(0.865538862, -0.22604255, 0.0) absTol 1E-5) + assert(data2(5) ~== Vectors.dense(0.0, 0.71580142, 0.0) absTol 1E-5) + assert(data3(1) ~== Vectors.dense(-0.58333333, -0.58333333, -2.8166666666) absTol 1E-5) + assert(data3(5) ~== Vectors.dense(-0.58333333, 2.316666666, 0.18333333333) absTol 1E-5) + } + + + test("Standardization with sparse input") { + val data = Array( + Vectors.sparse(3, Seq((0, -2.0), (1, 2.3))), + Vectors.sparse(3, Seq((1, -1.0), (2, -3.0))), + Vectors.sparse(3, Seq((1, -5.1))), + Vectors.sparse(3, Seq((0, 3.8), (2, 1.9))), + Vectors.sparse(3, Seq((0, 1.7), (1, -0.6))), + Vectors.sparse(3, Seq((1, 1.9))) + ) + + val dataRDD = sc.parallelize(data, 3) + + val standardizer1 = new StandardScaler(withMean = true, withStd = true) + val standardizer2 = new StandardScaler() + val standardizer3 = new StandardScaler(withMean = true, withStd = false) + + standardizer1.fit(dataRDD) + standardizer2.fit(dataRDD) + standardizer3.fit(dataRDD) + + val data2 = data.map(standardizer2.transform) + + withClue("Standardization with mean can not be applied on sparse input.") { + intercept[IllegalArgumentException] { + data.map(standardizer1.transform) + } + } + + withClue("Standardization with mean can not be applied on sparse input.") { + intercept[IllegalArgumentException] { + data.map(standardizer3.transform) + } + } + + val data2RDD = standardizer2.transform(dataRDD) + + val summary2 = computeSummary(data2RDD) + + assert((data, data2, data2RDD.collect()).zipped.forall { + case (v1: DenseVector, v2: DenseVector, v3: DenseVector) => true + case (v1: SparseVector, v2: SparseVector, v3: SparseVector) => true + case _ => false + }, "The vector type should be preserved after standardization.") + + assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) + + assert(summary2.mean !~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) + assert(summary2.variance ~== Vectors.dense(1.0, 1.0, 1.0) absTol 1E-5) + + assert(data2(4) ~== Vectors.sparse(3, Seq((0, 0.865538862), (1, -0.22604255))) absTol 1E-5) + assert(data2(5) ~== Vectors.sparse(3, Seq((1, 0.71580142))) absTol 1E-5) + } + + test("Standardization with constant input") { + // When the input data is all constant, the variance is zero. The standardization against + // zero variance is not well-defined, but we decide to just set it into zero here. + val data = Array( + Vectors.dense(2.0), + Vectors.dense(2.0), + Vectors.dense(2.0) + ) + + val dataRDD = sc.parallelize(data, 2) + + val standardizer1 = new StandardScaler(withMean = true, withStd = true) + val standardizer2 = new StandardScaler(withMean = true, withStd = false) + val standardizer3 = new StandardScaler(withMean = false, withStd = true) + + standardizer1.fit(dataRDD) + standardizer2.fit(dataRDD) + standardizer3.fit(dataRDD) + + val data1 = data.map(standardizer1.transform) + val data2 = data.map(standardizer2.transform) + val data3 = data.map(standardizer3.transform) + + assert(data1.forall(_.toArray.forall(_ == 0.0)), + "The variance is zero, so the transformed result should be 0.0") + assert(data2.forall(_.toArray.forall(_ == 0.0)), + "The variance is zero, so the transformed result should be 0.0") + assert(data3.forall(_.toArray.forall(_ == 0.0)), + "The variance is zero, so the transformed result should be 0.0") + } + +} From e053c55819363fab7068bb9165e3379f0c2f570c Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Sun, 3 Aug 2014 23:55:58 -0700 Subject: [PATCH 0237/1492] [MLlib] [SPARK-2510]Word2Vec: Distributed Representation of Words This is a pull request regarding SPARK-2510 at https://issues.apache.org/jira/browse/SPARK-2510. Word2Vec creates vector representation of words in a text corpus. The algorithm first constructs a vocabulary from the corpus and then learns vector representation of words in the vocabulary. The vector representation can be used as features in natural language processing and machine learning algorithms. To make our implementation more scalable, we train each partition separately and merge the model of each partition after each iteration. To make the model more accurate, multiple iterations may be needed. To investigate the vector representations is to find the closest words for a query word. For example, the top 20 closest words to "china" are for 1 partition and 1 iteration : taiwan 0.8077646146334014 korea 0.740913304563621 japan 0.7240667798885471 republic 0.7107151279078352 thailand 0.6953217332072862 tibet 0.6916782118129544 mongolia 0.6800858715972612 macau 0.6794925677480378 singapore 0.6594048695593799 manchuria 0.658989931844148 laos 0.6512978726001666 nepal 0.6380792327845325 mainland 0.6365469459587788 myanmar 0.6358614338840394 macedonia 0.6322366180313249 xinjiang 0.6285291551708028 russia 0.6279951236068411 india 0.6272874944023487 shanghai 0.6234544135576999 macao 0.6220588462925876 The result with 10 partitions and 5 iterations is: taiwan 0.8310495079388313 india 0.7737171315919039 japan 0.756777901233668 korea 0.7429767187102452 indonesia 0.7407557427278356 pakistan 0.712883426985585 mainland 0.7053379963140822 thailand 0.696298191073948 mongolia 0.693690656871415 laos 0.6913069680735292 macau 0.6903427690029617 republic 0.6766381604813666 malaysia 0.676460699141784 singapore 0.6728790997360923 malaya 0.672345232966194 manchuria 0.6703732292753156 macedonia 0.6637955686322028 myanmar 0.6589462882439646 kazakhstan 0.657017801081494 cambodia 0.6542383836451932 Author: Liquan Pei Author: Xiangrui Meng Author: Liquan Pei Closes #1719 from Ishiihara/master and squashes the following commits: 2ba9483 [Liquan Pei] minor fix for Word2Vec test e248441 [Liquan Pei] minor style change 26a948d [Liquan Pei] Merge pull request #1 from mengxr/Ishiihara-master c14da41 [Xiangrui Meng] fix styles 384c771 [Xiangrui Meng] remove minCount and window from constructor change model to use float instead of double e93e726 [Liquan Pei] use treeAggregate instead of aggregate 1a8fb41 [Liquan Pei] use weighted sum in combOp 7efbb6f [Liquan Pei] use broadcast version of vocab in aggregate 6bcc8be [Liquan Pei] add multiple iteration support 720b5a3 [Liquan Pei] Add test for Word2Vec algorithm, minor fixes 2e92b59 [Liquan Pei] modify according to feedback 57dc50d [Liquan Pei] code formatting e4a04d3 [Liquan Pei] minor fix 0aafb1b [Liquan Pei] Add comments, minor fixes 8d6befe [Liquan Pei] initial commit --- .../apache/spark/mllib/feature/Word2Vec.scala | 424 ++++++++++++++++++ .../spark/mllib/feature/Word2VecSuite.scala | 61 +++ 2 files changed, 485 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala new file mode 100644 index 0000000000000..87c81e7b0bd2f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -0,0 +1,424 @@ +/* + * 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.feature + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.{HashPartitioner, Logging} +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.rdd._ +import org.apache.spark.storage.StorageLevel + +/** + * Entry in vocabulary + */ +private case class VocabWord( + var word: String, + var cn: Int, + var point: Array[Int], + var code: Array[Int], + var codeLen:Int +) + +/** + * :: Experimental :: + * Word2Vec creates vector representation of words in a text corpus. + * The algorithm first constructs a vocabulary from the corpus + * and then learns vector representation of words in the vocabulary. + * The vector representation can be used as features in + * natural language processing and machine learning algorithms. + * + * We used skip-gram model in our implementation and hierarchical softmax + * method to train the model. The variable names in the implementation + * matches the original C implementation. + * + * For original C implementation, see https://code.google.com/p/word2vec/ + * For research papers, see + * Efficient Estimation of Word Representations in Vector Space + * and + * Distributed Representations of Words and Phrases and their Compositionality. + * @param size vector dimension + * @param startingAlpha initial learning rate + * @param parallelism number of partitions to run Word2Vec (using a small number for accuracy) + * @param numIterations number of iterations to run, should be smaller than or equal to parallelism + */ +@Experimental +class Word2Vec( + val size: Int, + val startingAlpha: Double, + val parallelism: Int, + val numIterations: Int) extends Serializable with Logging { + + /** + * Word2Vec with a single thread. + */ + def this(size: Int, startingAlpha: Int) = this(size, startingAlpha, 1, 1) + + private val EXP_TABLE_SIZE = 1000 + private val MAX_EXP = 6 + private val MAX_CODE_LENGTH = 40 + private val MAX_SENTENCE_LENGTH = 1000 + private val layer1Size = size + private val modelPartitionNum = 100 + + /** context words from [-window, window] */ + private val window = 5 + + /** minimum frequency to consider a vocabulary word */ + private val minCount = 5 + + private var trainWordsCount = 0 + private var vocabSize = 0 + private var vocab: Array[VocabWord] = null + private var vocabHash = mutable.HashMap.empty[String, Int] + private var alpha = startingAlpha + + private def learnVocab(words:RDD[String]): Unit = { + vocab = words.map(w => (w, 1)) + .reduceByKey(_ + _) + .map(x => VocabWord( + x._1, + x._2, + new Array[Int](MAX_CODE_LENGTH), + new Array[Int](MAX_CODE_LENGTH), + 0)) + .filter(_.cn >= minCount) + .collect() + .sortWith((a, b) => a.cn > b.cn) + + vocabSize = vocab.length + var a = 0 + while (a < vocabSize) { + vocabHash += vocab(a).word -> a + trainWordsCount += vocab(a).cn + a += 1 + } + logInfo("trainWordsCount = " + trainWordsCount) + } + + private def createExpTable(): Array[Float] = { + val expTable = new Array[Float](EXP_TABLE_SIZE) + var i = 0 + while (i < EXP_TABLE_SIZE) { + val tmp = math.exp((2.0 * i / EXP_TABLE_SIZE - 1.0) * MAX_EXP) + expTable(i) = (tmp / (tmp + 1.0)).toFloat + i += 1 + } + expTable + } + + private def createBinaryTree(): Unit = { + val count = new Array[Long](vocabSize * 2 + 1) + val binary = new Array[Int](vocabSize * 2 + 1) + val parentNode = new Array[Int](vocabSize * 2 + 1) + val code = new Array[Int](MAX_CODE_LENGTH) + val point = new Array[Int](MAX_CODE_LENGTH) + var a = 0 + while (a < vocabSize) { + count(a) = vocab(a).cn + a += 1 + } + while (a < 2 * vocabSize) { + count(a) = 1e9.toInt + a += 1 + } + var pos1 = vocabSize - 1 + var pos2 = vocabSize + + var min1i = 0 + var min2i = 0 + + a = 0 + while (a < vocabSize - 1) { + if (pos1 >= 0) { + if (count(pos1) < count(pos2)) { + min1i = pos1 + pos1 -= 1 + } else { + min1i = pos2 + pos2 += 1 + } + } else { + min1i = pos2 + pos2 += 1 + } + if (pos1 >= 0) { + if (count(pos1) < count(pos2)) { + min2i = pos1 + pos1 -= 1 + } else { + min2i = pos2 + pos2 += 1 + } + } else { + min2i = pos2 + pos2 += 1 + } + count(vocabSize + a) = count(min1i) + count(min2i) + parentNode(min1i) = vocabSize + a + parentNode(min2i) = vocabSize + a + binary(min2i) = 1 + a += 1 + } + // Now assign binary code to each vocabulary word + var i = 0 + a = 0 + while (a < vocabSize) { + var b = a + i = 0 + while (b != vocabSize * 2 - 2) { + code(i) = binary(b) + point(i) = b + i += 1 + b = parentNode(b) + } + vocab(a).codeLen = i + vocab(a).point(0) = vocabSize - 2 + b = 0 + while (b < i) { + vocab(a).code(i - b - 1) = code(b) + vocab(a).point(i - b) = point(b) - vocabSize + b += 1 + } + a += 1 + } + } + + /** + * Computes the vector representation of each word in vocabulary. + * @param dataset an RDD of words + * @return a Word2VecModel + */ + def fit[S <: Iterable[String]](dataset: RDD[S]): Word2VecModel = { + + val words = dataset.flatMap(x => x) + + learnVocab(words) + + createBinaryTree() + + val sc = dataset.context + + val expTable = sc.broadcast(createExpTable()) + val bcVocab = sc.broadcast(vocab) + val bcVocabHash = sc.broadcast(vocabHash) + + val sentences: RDD[Array[Int]] = words.mapPartitions { iter => + new Iterator[Array[Int]] { + def hasNext: Boolean = iter.hasNext + + def next(): Array[Int] = { + var sentence = new ArrayBuffer[Int] + var sentenceLength = 0 + while (iter.hasNext && sentenceLength < MAX_SENTENCE_LENGTH) { + val word = bcVocabHash.value.get(iter.next()) + word match { + case Some(w) => + sentence += w + sentenceLength += 1 + case None => + } + } + sentence.toArray + } + } + } + + val newSentences = sentences.repartition(parallelism).cache() + var syn0Global = + Array.fill[Float](vocabSize * layer1Size)((Random.nextFloat() - 0.5f) / layer1Size) + var syn1Global = new Array[Float](vocabSize * layer1Size) + + for(iter <- 1 to numIterations) { + val (aggSyn0, aggSyn1, _, _) = + // TODO: broadcast temp instead of serializing it directly + // or initialize the model in each executor + newSentences.treeAggregate((syn0Global, syn1Global, 0, 0))( + seqOp = (c, v) => (c, v) match { + case ((syn0, syn1, lastWordCount, wordCount), sentence) => + var lwc = lastWordCount + var wc = wordCount + if (wordCount - lastWordCount > 10000) { + lwc = wordCount + alpha = startingAlpha * (1 - parallelism * wordCount.toDouble / (trainWordsCount + 1)) + if (alpha < startingAlpha * 0.0001) alpha = startingAlpha * 0.0001 + logInfo("wordCount = " + wordCount + ", alpha = " + alpha) + } + wc += sentence.size + var pos = 0 + while (pos < sentence.size) { + val word = sentence(pos) + // TODO: fix random seed + val b = Random.nextInt(window) + // Train Skip-gram + var a = b + while (a < window * 2 + 1 - b) { + if (a != window) { + val c = pos - window + a + if (c >= 0 && c < sentence.size) { + val lastWord = sentence(c) + val l1 = lastWord * layer1Size + val neu1e = new Array[Float](layer1Size) + // Hierarchical softmax + var d = 0 + while (d < bcVocab.value(word).codeLen) { + val l2 = bcVocab.value(word).point(d) * layer1Size + // Propagate hidden -> output + var f = blas.sdot(layer1Size, syn0, l1, 1, syn1, l2, 1) + if (f > -MAX_EXP && f < MAX_EXP) { + val ind = ((f + MAX_EXP) * (EXP_TABLE_SIZE / MAX_EXP / 2.0)).toInt + f = expTable.value(ind) + val g = ((1 - bcVocab.value(word).code(d) - f) * alpha).toFloat + blas.saxpy(layer1Size, g, syn1, l2, 1, neu1e, 0, 1) + blas.saxpy(layer1Size, g, syn0, l1, 1, syn1, l2, 1) + } + d += 1 + } + blas.saxpy(layer1Size, 1.0f, neu1e, 0, 1, syn0, l1, 1) + } + } + a += 1 + } + pos += 1 + } + (syn0, syn1, lwc, wc) + }, + combOp = (c1, c2) => (c1, c2) match { + case ((syn0_1, syn1_1, lwc_1, wc_1), (syn0_2, syn1_2, lwc_2, wc_2)) => + val n = syn0_1.length + val weight1 = 1.0f * wc_1 / (wc_1 + wc_2) + val weight2 = 1.0f * wc_2 / (wc_1 + wc_2) + blas.sscal(n, weight1, syn0_1, 1) + blas.sscal(n, weight1, syn1_1, 1) + blas.saxpy(n, weight2, syn0_2, 1, syn0_1, 1) + blas.saxpy(n, weight2, syn1_2, 1, syn1_1, 1) + (syn0_1, syn1_1, lwc_1 + lwc_2, wc_1 + wc_2) + }) + syn0Global = aggSyn0 + syn1Global = aggSyn1 + } + newSentences.unpersist() + + val wordMap = new Array[(String, Array[Float])](vocabSize) + var i = 0 + while (i < vocabSize) { + val word = bcVocab.value(i).word + val vector = new Array[Float](layer1Size) + Array.copy(syn0Global, i * layer1Size, vector, 0, layer1Size) + wordMap(i) = (word, vector) + i += 1 + } + val modelRDD = sc.parallelize(wordMap, modelPartitionNum) + .partitionBy(new HashPartitioner(modelPartitionNum)) + .persist(StorageLevel.MEMORY_AND_DISK) + + new Word2VecModel(modelRDD) + } +} + +/** +* Word2Vec model +*/ +class Word2VecModel(private val model: RDD[(String, Array[Float])]) extends Serializable { + + private def cosineSimilarity(v1: Array[Float], v2: Array[Float]): Double = { + require(v1.length == v2.length, "Vectors should have the same length") + val n = v1.length + val norm1 = blas.snrm2(n, v1, 1) + val norm2 = blas.snrm2(n, v2, 1) + if (norm1 == 0 || norm2 == 0) return 0.0 + blas.sdot(n, v1, 1, v2,1) / norm1 / norm2 + } + + /** + * Transforms a word to its vector representation + * @param word a word + * @return vector representation of word + */ + def transform(word: String): Vector = { + val result = model.lookup(word) + if (result.isEmpty) { + throw new IllegalStateException(s"$word not in vocabulary") + } + else Vectors.dense(result(0).map(_.toDouble)) + } + + /** + * Transforms an RDD to its vector representation + * @param dataset a an RDD of words + * @return RDD of vector representation + */ + def transform(dataset: RDD[String]): RDD[Vector] = { + dataset.map(word => transform(word)) + } + + /** + * Find synonyms of a word + * @param word a word + * @param num number of synonyms to find + * @return array of (word, similarity) + */ + def findSynonyms(word: String, num: Int): Array[(String, Double)] = { + val vector = transform(word) + findSynonyms(vector,num) + } + + /** + * Find synonyms of the vector representation of a word + * @param vector vector representation of a word + * @param num number of synonyms to find + * @return array of (word, cosineSimilarity) + */ + def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { + require(num > 0, "Number of similar words should > 0") + val topK = model.map { case(w, vec) => + (cosineSimilarity(vector.toArray.map(_.toFloat), vec), w) } + .sortByKey(ascending = false) + .take(num + 1) + .map(_.swap) + .tail + + topK + } +} + +object Word2Vec{ + /** + * Train Word2Vec model + * @param input RDD of words + * @param size vector dimension + * @param startingAlpha initial learning rate + * @param parallelism number of partitions to run Word2Vec (using a small number for accuracy) + * @param numIterations number of iterations, should be smaller than or equal to parallelism + * @return Word2Vec model + */ + def train[S <: Iterable[String]]( + input: RDD[S], + size: Int, + startingAlpha: Double, + parallelism: Int = 1, + numIterations:Int = 1): Word2VecModel = { + new Word2Vec(size,startingAlpha, parallelism, numIterations).fit[S](input) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala new file mode 100644 index 0000000000000..b5db39b68a223 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.feature + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext + +class Word2VecSuite extends FunSuite with LocalSparkContext { + + // TODO: add more tests + + test("Word2Vec") { + val sentence = "a b " * 100 + "a c " * 10 + val localDoc = Seq(sentence, sentence) + val doc = sc.parallelize(localDoc) + .map(line => line.split(" ").toSeq) + val size = 10 + val startingAlpha = 0.025 + val window = 2 + val minCount = 2 + val num = 2 + + val model = Word2Vec.train(doc, size, startingAlpha) + val syms = model.findSynonyms("a", 2) + assert(syms.length == num) + assert(syms(0)._1 == "b") + assert(syms(1)._1 == "c") + } + + + test("Word2VecModel") { + val num = 2 + val localModel = Seq( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val model = new Word2VecModel(sc.parallelize(localModel, 2)) + val syms = model.findSynonyms("china", num) + assert(syms.length == num) + assert(syms(0)._1 == "taiwan") + assert(syms(1)._1 == "japan") + } +} From 59f84a9531f7974a053fd4963ce9afd88273ea4c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 4 Aug 2014 12:13:41 -0700 Subject: [PATCH 0238/1492] [SPARK-1687] [PySpark] pickable namedtuple Add an hook to replace original namedtuple with an pickable one, then namedtuple could be used in RDDs. PS: pyspark should be import BEFORE "from collections import namedtuple" Author: Davies Liu Closes #1623 from davies/namedtuple and squashes the following commits: 045dad8 [Davies Liu] remove unrelated code changes 4132f32 [Davies Liu] address comment 55b1c1a [Davies Liu] fix tests 61f86eb [Davies Liu] replace all the reference of namedtuple to new hacked one 98df6c6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into namedtuple f7b1bde [Davies Liu] add hack for CloudPickleSerializer 0c5c849 [Davies Liu] Merge branch 'master' of github.com:apache/spark into namedtuple 21991e6 [Davies Liu] hack namedtuple in __main__ module, make it picklable. 93b03b8 [Davies Liu] pickable namedtuple --- python/pyspark/serializers.py | 60 +++++++++++++++++++++++++++++++++++ python/pyspark/tests.py | 19 +++++++++++ 2 files changed, 79 insertions(+) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 03b31ae9624c2..1b52c144df087 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -65,6 +65,9 @@ import marshal import struct import sys +import types +import collections + from pyspark import cloudpickle @@ -267,6 +270,63 @@ def dumps(self, obj): return obj +# Hook namedtuple, make it picklable + +__cls = {} + + +def _restore(name, fields, value): + """ Restore an object of namedtuple""" + k = (name, fields) + cls = __cls.get(k) + if cls is None: + cls = collections.namedtuple(name, fields) + __cls[k] = cls + return cls(*value) + + +def _hack_namedtuple(cls): + """ Make class generated by namedtuple picklable """ + name = cls.__name__ + fields = cls._fields + def __reduce__(self): + return (_restore, (name, fields, tuple(self))) + cls.__reduce__ = __reduce__ + return cls + + +def _hijack_namedtuple(): + """ Hack namedtuple() to make it picklable """ + global _old_namedtuple # or it will put in closure + + def _copy_func(f): + return types.FunctionType(f.func_code, f.func_globals, f.func_name, + f.func_defaults, f.func_closure) + + _old_namedtuple = _copy_func(collections.namedtuple) + + def namedtuple(name, fields, verbose=False, rename=False): + cls = _old_namedtuple(name, fields, verbose, rename) + return _hack_namedtuple(cls) + + # replace namedtuple with new one + collections.namedtuple.func_globals["_old_namedtuple"] = _old_namedtuple + collections.namedtuple.func_globals["_hack_namedtuple"] = _hack_namedtuple + collections.namedtuple.func_code = namedtuple.func_code + + # hack the cls already generated by namedtuple + # those created in other module can be pickled as normal, + # so only hack those in __main__ module + for n, o in sys.modules["__main__"].__dict__.iteritems(): + if (type(o) is type and o.__base__ is tuple + and hasattr(o, "_fields") + and "__reduce__" not in o.__dict__): + _hack_namedtuple(o) # hack inplace + + +_hijack_namedtuple() + + class PickleSerializer(FramedSerializer): """ Serializes objects using Python's cPickle serializer: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index acc3c30371621..4ac94ba729d35 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -112,6 +112,17 @@ def test_huge_dataset(self): m._cleanup() +class SerializationTestCase(unittest.TestCase): + + def test_namedtuple(self): + from collections import namedtuple + from cPickle import dumps, loads + P = namedtuple("P", "x y") + p1 = P(1, 3) + p2 = loads(dumps(p1, 2)) + self.assertEquals(p1, p2) + + class PySparkTestCase(unittest.TestCase): def setUp(self): @@ -298,6 +309,14 @@ def test_itemgetter(self): self.assertEqual([1], rdd.map(itemgetter(1)).collect()) self.assertEqual([(2, 3)], rdd.map(itemgetter(2, 3)).collect()) + def test_namedtuple_in_rdd(self): + from collections import namedtuple + Person = namedtuple("Person", "id firstName lastName") + jon = Person(1, "Jon", "Doe") + jane = Person(2, "Jane", "Doe") + theDoes = self.sc.parallelize([jon, jane]) + self.assertEquals([jon, jane], theDoes.collect()) + class TestIO(PySparkTestCase): From 8e7d5ba1a20a8a1f409e9d6472ae3e6c4bc948b4 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 4 Aug 2014 12:59:18 -0700 Subject: [PATCH 0239/1492] SPARK-2792. Fix reading too much or too little data from each stream in ExternalMap / Sorter All these changes are from mridulm's work in #1609, but extracted here to fix this specific issue and make it easier to merge not 1.1. This particular set of changes is to make sure that we read exactly the right range of bytes from each spill file in EAOM: some serializers can write bytes after the last object (e.g. the TC_RESET flag in Java serialization) and that would confuse the previous code into reading it as part of the next batch. There are also improvements to cleanup to make sure files are closed. In addition to bringing in the changes to ExternalAppendOnlyMap, I also copied them to the corresponding code in ExternalSorter and updated its test suite to test for the same issues. Author: Matei Zaharia Closes #1722 from mateiz/spark-2792 and squashes the following commits: 5d4bfb5 [Matei Zaharia] Make objectStreamReset counter count the last object written too 18fe865 [Matei Zaharia] Update docs on objectStreamReset 576ee83 [Matei Zaharia] Allow objectStreamReset to be 0 0374217 [Matei Zaharia] Remove super paranoid code to close file handles bda37bb [Matei Zaharia] Implement Mridul's ExternalAppendOnlyMap fixes in ExternalSorter too 0d6dad7 [Matei Zaharia] Added Mridul's test changes for ExternalAppendOnlyMap 9a78e4b [Matei Zaharia] Add @mridulm's fixes to ExternalAppendOnlyMap for batch sizes --- .../spark/serializer/JavaSerializer.scala | 5 +- .../collection/ExternalAppendOnlyMap.scala | 86 +++++++++++---- .../util/collection/ExternalSorter.scala | 104 +++++++++++++----- .../ExternalAppendOnlyMapSuite.scala | 33 ++++-- .../util/collection/ExternalSorterSuite.scala | 47 +++++--- docs/configuration.md | 2 +- 6 files changed, 194 insertions(+), 83 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index a7fa057ee05f7..34bc3124097bb 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -35,16 +35,15 @@ private[spark] class JavaSerializationStream(out: OutputStream, counterReset: In /** * Calling reset to avoid memory leak: * http://stackoverflow.com/questions/1281549/memory-leak-traps-in-the-java-standard-api - * But only call it every 10,000th time to avoid bloated serialization streams (when + * But only call it every 100th time to avoid bloated serialization streams (when * the stream 'resets' object class descriptions have to be re-written) */ def writeObject[T: ClassTag](t: T): SerializationStream = { objOut.writeObject(t) + counter += 1 if (counterReset > 0 && counter >= counterReset) { objOut.reset() counter = 0 - } else { - counter += 1 } this } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index cb67a1c039f20..5d10a1f84493c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import java.io.{InputStream, BufferedInputStream, FileInputStream, File, Serializable, EOFException} +import java.io._ import java.util.Comparator import scala.collection.BufferedIterator @@ -28,7 +28,7 @@ import com.google.common.io.ByteStreams import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.serializer.Serializer +import org.apache.spark.serializer.{DeserializationStream, Serializer} import org.apache.spark.storage.{BlockId, BlockManager} import org.apache.spark.util.collection.ExternalAppendOnlyMap.HashComparator @@ -199,13 +199,16 @@ class ExternalAppendOnlyMap[K, V, C]( // Flush the disk writer's contents to disk, and update relevant variables def flush() = { - writer.commitAndClose() - val bytesWritten = writer.bytesWritten + val w = writer + writer = null + w.commitAndClose() + val bytesWritten = w.bytesWritten batchSizes.append(bytesWritten) _diskBytesSpilled += bytesWritten objectsWritten = 0 } + var success = false try { val it = currentMap.destructiveSortedIterator(keyComparator) while (it.hasNext) { @@ -215,16 +218,28 @@ class ExternalAppendOnlyMap[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() - writer.close() writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) } } if (objectsWritten > 0) { flush() + } else if (writer != null) { + val w = writer + writer = null + w.revertPartialWritesAndClose() } + success = true } finally { - // Partial failures cannot be tolerated; do not revert partial writes - writer.close() + if (!success) { + // This code path only happens if an exception was thrown above before we set success; + // close our stuff and let the exception be thrown further + if (writer != null) { + writer.revertPartialWritesAndClose() + } + if (file.exists()) { + file.delete() + } + } } currentMap = new SizeTrackingAppendOnlyMap[K, C] @@ -389,27 +404,51 @@ class ExternalAppendOnlyMap[K, V, C]( * An iterator that returns (K, C) pairs in sorted order from an on-disk map */ private class DiskMapIterator(file: File, blockId: BlockId, batchSizes: ArrayBuffer[Long]) - extends Iterator[(K, C)] { - private val fileStream = new FileInputStream(file) - private val bufferedStream = new BufferedInputStream(fileStream, fileBufferSize) + extends Iterator[(K, C)] + { + private val batchOffsets = batchSizes.scanLeft(0L)(_ + _) // Size will be batchSize.length + 1 + assert(file.length() == batchOffsets(batchOffsets.length - 1)) + + private var batchIndex = 0 // Which batch we're in + private var fileStream: FileInputStream = null // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams - private var batchStream = nextBatchStream() - private var compressedStream = blockManager.wrapForCompression(blockId, batchStream) - private var deserializeStream = ser.deserializeStream(compressedStream) + private var deserializeStream = nextBatchStream() private var nextItem: (K, C) = null private var objectsRead = 0 /** * Construct a stream that reads only from the next batch. */ - private def nextBatchStream(): InputStream = { - if (batchSizes.length > 0) { - ByteStreams.limit(bufferedStream, batchSizes.remove(0)) + private def nextBatchStream(): DeserializationStream = { + // Note that batchOffsets.length = numBatches + 1 since we did a scan above; check whether + // we're still in a valid batch. + if (batchIndex < batchOffsets.length - 1) { + if (deserializeStream != null) { + deserializeStream.close() + fileStream.close() + deserializeStream = null + fileStream = null + } + + val start = batchOffsets(batchIndex) + fileStream = new FileInputStream(file) + fileStream.getChannel.position(start) + batchIndex += 1 + + val end = batchOffsets(batchIndex) + + assert(end >= start, "start = " + start + ", end = " + end + + ", batchOffsets = " + batchOffsets.mkString("[", ", ", "]")) + + val bufferedStream = new BufferedInputStream(ByteStreams.limit(fileStream, end - start)) + val compressedStream = blockManager.wrapForCompression(blockId, bufferedStream) + ser.deserializeStream(compressedStream) } else { // No more batches left - bufferedStream + cleanup() + null } } @@ -424,10 +463,8 @@ class ExternalAppendOnlyMap[K, V, C]( val item = deserializeStream.readObject().asInstanceOf[(K, C)] objectsRead += 1 if (objectsRead == serializerBatchSize) { - batchStream = nextBatchStream() - compressedStream = blockManager.wrapForCompression(blockId, batchStream) - deserializeStream = ser.deserializeStream(compressedStream) objectsRead = 0 + deserializeStream = nextBatchStream() } item } catch { @@ -439,6 +476,9 @@ class ExternalAppendOnlyMap[K, V, C]( override def hasNext: Boolean = { if (nextItem == null) { + if (deserializeStream == null) { + return false + } nextItem = readNextItem() } nextItem != null @@ -455,7 +495,11 @@ class ExternalAppendOnlyMap[K, V, C]( // TODO: Ensure this gets called even if the iterator isn't drained. private def cleanup() { - deserializeStream.close() + batchIndex = batchOffsets.length // Prevent reading any other batch + val ds = deserializeStream + deserializeStream = null + fileStream = null + ds.close() file.delete() } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 6e415a2bd8ce2..b04c50bd3e196 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -26,7 +26,7 @@ import scala.collection.mutable import com.google.common.io.ByteStreams import org.apache.spark.{Aggregator, SparkEnv, Logging, Partitioner} -import org.apache.spark.serializer.Serializer +import org.apache.spark.serializer.{DeserializationStream, Serializer} import org.apache.spark.storage.BlockId /** @@ -273,13 +273,16 @@ private[spark] class ExternalSorter[K, V, C]( // Flush the disk writer's contents to disk, and update relevant variables. // The writer is closed at the end of this process, and cannot be reused. def flush() = { - writer.commitAndClose() - val bytesWritten = writer.bytesWritten + val w = writer + writer = null + w.commitAndClose() + val bytesWritten = w.bytesWritten batchSizes.append(bytesWritten) _diskBytesSpilled += bytesWritten objectsWritten = 0 } + var success = false try { val it = collection.destructiveSortedIterator(partitionKeyComparator) while (it.hasNext) { @@ -299,13 +302,23 @@ private[spark] class ExternalSorter[K, V, C]( } if (objectsWritten > 0) { flush() + } else if (writer != null) { + val w = writer + writer = null + w.revertPartialWritesAndClose() + } + success = true + } finally { + if (!success) { + // This code path only happens if an exception was thrown above before we set success; + // close our stuff and let the exception be thrown further + if (writer != null) { + writer.revertPartialWritesAndClose() + } + if (file.exists()) { + file.delete() + } } - writer.close() - } catch { - case e: Exception => - writer.close() - file.delete() - throw e } if (usingMap) { @@ -472,36 +485,58 @@ private[spark] class ExternalSorter[K, V, C]( * partitions to be requested in order. */ private[this] class SpillReader(spill: SpilledFile) { - val fileStream = new FileInputStream(spill.file) - val bufferedStream = new BufferedInputStream(fileStream, fileBufferSize) + // Serializer batch offsets; size will be batchSize.length + 1 + val batchOffsets = spill.serializerBatchSizes.scanLeft(0L)(_ + _) // Track which partition and which batch stream we're in. These will be the indices of // the next element we will read. We'll also store the last partition read so that // readNextPartition() can figure out what partition that was from. var partitionId = 0 var indexInPartition = 0L - var batchStreamsRead = 0 + var batchId = 0 var indexInBatch = 0 var lastPartitionId = 0 skipToNextPartition() - // An intermediate stream that reads from exactly one batch + + // Intermediate file and deserializer streams that read from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams - var batchStream = nextBatchStream() - var compressedStream = blockManager.wrapForCompression(spill.blockId, batchStream) - var deserStream = serInstance.deserializeStream(compressedStream) + var fileStream: FileInputStream = null + var deserializeStream = nextBatchStream() // Also sets fileStream + var nextItem: (K, C) = null var finished = false /** Construct a stream that only reads from the next batch */ - def nextBatchStream(): InputStream = { - if (batchStreamsRead < spill.serializerBatchSizes.length) { - batchStreamsRead += 1 - ByteStreams.limit(bufferedStream, spill.serializerBatchSizes(batchStreamsRead - 1)) + def nextBatchStream(): DeserializationStream = { + // Note that batchOffsets.length = numBatches + 1 since we did a scan above; check whether + // we're still in a valid batch. + if (batchId < batchOffsets.length - 1) { + if (deserializeStream != null) { + deserializeStream.close() + fileStream.close() + deserializeStream = null + fileStream = null + } + + val start = batchOffsets(batchId) + fileStream = new FileInputStream(spill.file) + fileStream.getChannel.position(start) + batchId += 1 + + val end = batchOffsets(batchId) + + assert(end >= start, "start = " + start + ", end = " + end + + ", batchOffsets = " + batchOffsets.mkString("[", ", ", "]")) + + val bufferedStream = new BufferedInputStream(ByteStreams.limit(fileStream, end - start)) + val compressedStream = blockManager.wrapForCompression(spill.blockId, bufferedStream) + serInstance.deserializeStream(compressedStream) } else { - // No more batches left; give an empty stream - bufferedStream + // No more batches left + cleanup() + null } } @@ -525,19 +560,17 @@ private[spark] class ExternalSorter[K, V, C]( * If no more pairs are left, return null. */ private def readNextItem(): (K, C) = { - if (finished) { + if (finished || deserializeStream == null) { return null } - val k = deserStream.readObject().asInstanceOf[K] - val c = deserStream.readObject().asInstanceOf[C] + val k = deserializeStream.readObject().asInstanceOf[K] + val c = deserializeStream.readObject().asInstanceOf[C] lastPartitionId = partitionId // Start reading the next batch if we're done with this one indexInBatch += 1 if (indexInBatch == serializerBatchSize) { - batchStream = nextBatchStream() - compressedStream = blockManager.wrapForCompression(spill.blockId, batchStream) - deserStream = serInstance.deserializeStream(compressedStream) indexInBatch = 0 + deserializeStream = nextBatchStream() } // Update the partition location of the element we're reading indexInPartition += 1 @@ -545,7 +578,9 @@ private[spark] class ExternalSorter[K, V, C]( // If we've finished reading the last partition, remember that we're done if (partitionId == numPartitions) { finished = true - deserStream.close() + if (deserializeStream != null) { + deserializeStream.close() + } } (k, c) } @@ -578,6 +613,17 @@ private[spark] class ExternalSorter[K, V, C]( item } } + + // Clean up our open streams and put us in a state where we can't read any more data + def cleanup() { + batchId = batchOffsets.length // Prevent reading any other batch + val ds = deserializeStream + deserializeStream = null + fileStream = null + ds.close() + // NOTE: We don't do file.delete() here because that is done in ExternalSorter.stop(). + // This should also be fixed in ExternalAppendOnlyMap. + } } /** diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 7de5df6e1c8bd..04d7338488628 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -30,8 +30,19 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { private def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i private def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + private def createSparkConf(loadDefaults: Boolean): SparkConf = { + val conf = new SparkConf(loadDefaults) + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + conf.set("spark.serializer.objectStreamReset", "1") + conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + // Ensure that we actually have multiple batches per spill file + conf.set("spark.shuffle.spill.batchSize", "10") + conf + } + test("simple insert") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, @@ -57,7 +68,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("insert with collision") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, @@ -80,7 +91,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("ordering") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map1 = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, @@ -125,7 +136,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("null keys and values") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val map = new ExternalAppendOnlyMap[Int, Int, ArrayBuffer[Int]](createCombiner, @@ -166,7 +177,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("simple aggregator") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) // reduceByKey @@ -181,7 +192,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("simple cogroup") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) sc = new SparkContext("local", "test", conf) val rdd1 = sc.parallelize(1 to 4).map(i => (i, i)) val rdd2 = sc.parallelize(1 to 4).map(i => (i%2, i)) @@ -199,7 +210,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling") { - val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -249,7 +260,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with hash collisions") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -304,7 +315,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with many hash collisions") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.0001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -329,7 +340,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with hash collisions using the Int.MaxValue key") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -347,7 +358,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { } test("spilling with null keys and values") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 65a71e5a83698..57dcb4ffabac1 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -25,6 +25,17 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ class ExternalSorterSuite extends FunSuite with LocalSparkContext { + private def createSparkConf(loadDefaults: Boolean): SparkConf = { + val conf = new SparkConf(loadDefaults) + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + conf.set("spark.serializer.objectStreamReset", "1") + conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") + // Ensure that we actually have multiple batches per spill file + conf.set("spark.shuffle.spill.batchSize", "10") + conf + } + test("empty data stream") { val conf = new SparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") @@ -60,7 +71,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("few elements per partition") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -102,7 +113,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("empty partitions with spilling") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -127,7 +138,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("spilling in local cluster") { - val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -198,7 +209,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("spilling in local cluster with many reduce tasks") { - val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local-cluster[2,1,512]", "test", conf) @@ -269,7 +280,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("cleanup of intermediate files in sorter") { - val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -290,7 +301,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("cleanup of intermediate files in sorter if there are errors") { - val conf = new SparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -311,7 +322,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("cleanup of intermediate files in shuffle") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -326,7 +337,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("cleanup of intermediate files in shuffle with errors") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -348,7 +359,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("no partial aggregation or sorting") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -363,7 +374,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("partial aggregation without spill") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -379,7 +390,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("partial aggregation with spill, no ordering") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -395,7 +406,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("partial aggregation with spill, with ordering") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -412,7 +423,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("sorting without aggregation, no spill") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -429,7 +440,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("sorting without aggregation, with spill") { - val conf = new SparkConf(false) + val conf = createSparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) @@ -446,7 +457,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("spilling with hash collisions") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -503,7 +514,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("spilling with many hash collisions") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.0001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -526,7 +537,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("spilling with hash collisions using the Int.MaxValue key") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) @@ -547,7 +558,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { } test("spilling with null keys and values") { - val conf = new SparkConf(true) + val conf = createSparkConf(true) conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) diff --git a/docs/configuration.md b/docs/configuration.md index 2a71d7b820e5f..870343f1c0bd2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -385,7 +385,7 @@ Apart from these, the following properties are also available, and may be useful When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches objects to prevent writing redundant data, however that stops garbage collection of those objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value <= 0. + objects to be collected. To turn off this periodic reset set it to -1. By default it will reset the serializer every 100 objects. From 9fd82dbbcb8b10debbe95f1acab53ae8b340f38e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 4 Aug 2014 15:54:52 -0700 Subject: [PATCH 0240/1492] [SPARK-1687] [PySpark] fix unit tests related to pickable namedtuple serializer is imported multiple times during doctests, so it's better to make _hijack_namedtuple() safe to be called multiple times. Author: Davies Liu Closes #1771 from davies/fix and squashes the following commits: 1a9e336 [Davies Liu] fix unit tests --- python/pyspark/serializers.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 1b52c144df087..a10f85b55ad30 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -297,8 +297,11 @@ def __reduce__(self): def _hijack_namedtuple(): """ Hack namedtuple() to make it picklable """ - global _old_namedtuple # or it will put in closure + # hijack only one time + if hasattr(collections.namedtuple, "__hijack"): + return + global _old_namedtuple # or it will put in closure def _copy_func(f): return types.FunctionType(f.func_code, f.func_globals, f.func_name, f.func_defaults, f.func_closure) @@ -313,6 +316,7 @@ def namedtuple(name, fields, verbose=False, rename=False): collections.namedtuple.func_globals["_old_namedtuple"] = _old_namedtuple collections.namedtuple.func_globals["_hack_namedtuple"] = _hack_namedtuple collections.namedtuple.func_code = namedtuple.func_code + collections.namedtuple.__hijack = 1 # hack the cls already generated by namedtuple # those created in other module can be pickled as normal, From 05bf4e4aff0d052a53d3e64c43688f07e27fec50 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 4 Aug 2014 20:39:18 -0700 Subject: [PATCH 0241/1492] [SPARK-2323] Exception in accumulator update should not crash DAGScheduler & SparkContext Author: Reynold Xin Closes #1772 from rxin/accumulator-dagscheduler and squashes the following commits: 6a58520 [Reynold Xin] [SPARK-2323] Exception in accumulator update should not crash DAGScheduler & SparkContext. --- .../org/apache/spark/scheduler/DAGScheduler.scala | 9 +++++++-- .../apache/spark/scheduler/DAGSchedulerSuite.scala | 11 +++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d87c3048985fc..9fa3a4e9c71ae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -904,8 +904,13 @@ class DAGScheduler( event.reason match { case Success => if (event.accumUpdates != null) { - // TODO: fail the stage if the accumulator update fails... - Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted + try { + Accumulators.add(event.accumUpdates) + } catch { + // If we see an exception during accumulator update, just log the error and move on. + case e: Exception => + logError(s"Failed to update accumulators for $task", e) + } } stage.pendingTasks -= task task match { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 36e238b4c9434..8c1b0fed11f72 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -622,8 +622,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } - // TODO: Fix this and un-ignore the test. - ignore("misbehaved accumulator should not crash DAGScheduler and SparkContext") { + test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { val acc = new Accumulator[Int](0, new AccumulatorParam[Int] { override def addAccumulator(t1: Int, t2: Int): Int = t1 + t2 override def zero(initialValue: Int): Int = 0 @@ -633,14 +632,10 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F }) // Run this on executors - intercept[SparkDriverExecutionException] { - sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) } - } + sc.parallelize(1 to 10, 2).foreach { item => acc.add(1) } // Run this within a local thread - intercept[SparkDriverExecutionException] { - sc.parallelize(1 to 10, 2).map { item => acc.add(1) }.take(1) - } + sc.parallelize(1 to 10, 2).map { item => acc.add(1) }.take(1) // Make sure we can still run local commands as well as cluster commands. assert(sc.parallelize(1 to 10, 2).count() === 10) From 066765d60d21b6b9943862b788e4a4bd07396e6c Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 4 Aug 2014 23:27:53 -0700 Subject: [PATCH 0242/1492] SPARK-2685. Update ExternalAppendOnlyMap to avoid buffer.remove() Replaces this with an O(1) operation that does not have to shift over the whole tail of the array into the gap produced by the element removed. Author: Matei Zaharia Closes #1773 from mateiz/SPARK-2685 and squashes the following commits: 1ea028a [Matei Zaharia] Update comments in StreamBuffer and EAOM, and reuse ArrayBuffers eb1abfd [Matei Zaharia] Update ExternalAppendOnlyMap to avoid buffer.remove() --- .../collection/ExternalAppendOnlyMap.scala | 50 +++++++++++++------ 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 5d10a1f84493c..1f7d2dc838ebc 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -286,30 +286,32 @@ class ExternalAppendOnlyMap[K, V, C]( private val inputStreams = (Seq(sortedMap) ++ spilledMaps).map(it => it.buffered) inputStreams.foreach { it => - val kcPairs = getMorePairs(it) + val kcPairs = new ArrayBuffer[(K, C)] + readNextHashCode(it, kcPairs) if (kcPairs.length > 0) { mergeHeap.enqueue(new StreamBuffer(it, kcPairs)) } } /** - * Fetch from the given iterator until a key of different hash is retrieved. + * Fill a buffer with the next set of keys with the same hash code from a given iterator. We + * read streams one hash code at a time to ensure we don't miss elements when they are merged. + * + * Assumes the given iterator is in sorted order of hash code. * - * In the event of key hash collisions, this ensures no pairs are hidden from being merged. - * Assume the given iterator is in sorted order. + * @param it iterator to read from + * @param buf buffer to write the results into */ - private def getMorePairs(it: BufferedIterator[(K, C)]): ArrayBuffer[(K, C)] = { - val kcPairs = new ArrayBuffer[(K, C)] + private def readNextHashCode(it: BufferedIterator[(K, C)], buf: ArrayBuffer[(K, C)]): Unit = { if (it.hasNext) { var kc = it.next() - kcPairs += kc + buf += kc val minHash = hashKey(kc) while (it.hasNext && it.head._1.hashCode() == minHash) { kc = it.next() - kcPairs += kc + buf += kc } } - kcPairs } /** @@ -321,7 +323,9 @@ class ExternalAppendOnlyMap[K, V, C]( while (i < buffer.pairs.length) { val pair = buffer.pairs(i) if (pair._1 == key) { - buffer.pairs.remove(i) + // Note that there's at most one pair in the buffer with a given key, since we always + // merge stuff in a map before spilling, so it's safe to return after the first we find + removeFromBuffer(buffer.pairs, i) return mergeCombiners(baseCombiner, pair._2) } i += 1 @@ -329,6 +333,19 @@ class ExternalAppendOnlyMap[K, V, C]( baseCombiner } + /** + * Remove the index'th element from an ArrayBuffer in constant time, swapping another element + * into its place. This is more efficient than the ArrayBuffer.remove method because it does + * not have to shift all the elements in the array over. It works for our array buffers because + * we don't care about the order of elements inside, we just want to search them for a key. + */ + private def removeFromBuffer[T](buffer: ArrayBuffer[T], index: Int): T = { + val elem = buffer(index) + buffer(index) = buffer(buffer.size - 1) // This also works if index == buffer.size - 1 + buffer.reduceToSize(buffer.size - 1) + elem + } + /** * Return true if there exists an input stream that still has unvisited pairs. */ @@ -346,7 +363,7 @@ class ExternalAppendOnlyMap[K, V, C]( val minBuffer = mergeHeap.dequeue() val minPairs = minBuffer.pairs val minHash = minBuffer.minKeyHash - val minPair = minPairs.remove(0) + val minPair = removeFromBuffer(minPairs, 0) val minKey = minPair._1 var minCombiner = minPair._2 assert(hashKey(minPair) == minHash) @@ -363,7 +380,7 @@ class ExternalAppendOnlyMap[K, V, C]( // Repopulate each visited stream buffer and add it back to the queue if it is non-empty mergedBuffers.foreach { buffer => if (buffer.isEmpty) { - buffer.pairs ++= getMorePairs(buffer.iterator) + readNextHashCode(buffer.iterator, buffer.pairs) } if (!buffer.isEmpty) { mergeHeap.enqueue(buffer) @@ -375,10 +392,13 @@ class ExternalAppendOnlyMap[K, V, C]( /** * A buffer for streaming from a map iterator (in-memory or on-disk) sorted by key hash. - * Each buffer maintains the lowest-ordered keys in the corresponding iterator. Due to - * hash collisions, it is possible for multiple keys to be "tied" for being the lowest. + * Each buffer maintains all of the key-value pairs with what is currently the lowest hash + * code among keys in the stream. There may be multiple keys if there are hash collisions. + * Note that because when we spill data out, we only spill one value for each key, there is + * at most one element for each key. * - * StreamBuffers are ordered by the minimum key hash found across all of their own pairs. + * StreamBuffers are ordered by the minimum key hash currently available in their stream so + * that we can put them into a heap and sort that. */ private class StreamBuffer( val iterator: BufferedIterator[(K, C)], From 4fde28c2063f673ec7f51d514ba62a73321960a1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 4 Aug 2014 23:41:03 -0700 Subject: [PATCH 0243/1492] SPARK-2711. Create a ShuffleMemoryManager to track memory for all spilling collections This tracks memory properly if there are multiple spilling collections in the same task (which was a problem before), and also implements an algorithm that lets each thread grow up to 1 / 2N of the memory pool (where N is the number of threads) before spilling, which avoids an inefficiency with small spills we had before (some threads would spill many times at 0-1 MB because the pool was allocated elsewhere). Author: Matei Zaharia Closes #1707 from mateiz/spark-2711 and squashes the following commits: debf75b [Matei Zaharia] Review comments 24f28f3 [Matei Zaharia] Small rename c8f3a8b [Matei Zaharia] Update ShuffleMemoryManager to be able to partially grant requests 315e3a5 [Matei Zaharia] Some review comments b810120 [Matei Zaharia] Create central manager to track memory for all spilling collections --- .../scala/org/apache/spark/SparkEnv.scala | 10 +- .../org/apache/spark/executor/Executor.scala | 5 +- .../spark/shuffle/ShuffleMemoryManager.scala | 125 ++++++++ .../collection/ExternalAppendOnlyMap.scala | 48 +-- .../util/collection/ExternalSorter.scala | 49 +-- .../shuffle/ShuffleMemoryManagerSuite.scala | 294 ++++++++++++++++++ 6 files changed, 450 insertions(+), 81 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala create mode 100644 core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 0bce531aaba3e..dd8e4ac66dc66 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -35,7 +35,7 @@ import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.ConnectionManager import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.ShuffleManager +import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -66,12 +66,9 @@ class SparkEnv ( val httpFileServer: HttpFileServer, val sparkFilesDir: String, val metricsSystem: MetricsSystem, + val shuffleMemoryManager: ShuffleMemoryManager, val conf: SparkConf) extends Logging { - // A mapping of thread ID to amount of memory, in bytes, used for shuffle aggregations - // All accesses should be manually synchronized - val shuffleMemoryMap = mutable.HashMap[Long, Long]() - private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation @@ -252,6 +249,8 @@ object SparkEnv extends Logging { val shuffleManager = instantiateClass[ShuffleManager]( "spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") + val shuffleMemoryManager = new ShuffleMemoryManager(conf) + // Warn about deprecated spark.cache.class property if (conf.contains("spark.cache.class")) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + @@ -273,6 +272,7 @@ object SparkEnv extends Logging { httpFileServer, sparkFilesDir, metricsSystem, + shuffleMemoryManager, conf) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 1bb1b4aae91bb..c2b9c660ddaec 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -276,10 +276,7 @@ private[spark] class Executor( } } finally { // Release memory used by this thread for shuffles - val shuffleMemoryMap = env.shuffleMemoryMap - shuffleMemoryMap.synchronized { - shuffleMemoryMap.remove(Thread.currentThread().getId) - } + env.shuffleMemoryManager.releaseMemoryForThisThread() // Release memory used by this thread for unrolling blocks env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() runningTasks.remove(taskId) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala new file mode 100644 index 0000000000000..ee91a368b76ea --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.shuffle + +import scala.collection.mutable + +import org.apache.spark.{Logging, SparkException, SparkConf} + +/** + * Allocates a pool of memory to task threads for use in shuffle operations. Each disk-spilling + * collection (ExternalAppendOnlyMap or ExternalSorter) used by these tasks can acquire memory + * from this pool and release it as it spills data out. When a task ends, all its memory will be + * released by the Executor. + * + * This class tries to ensure that each thread gets a reasonable share of memory, instead of some + * thread ramping up to a large amount first and then causing others to spill to disk repeatedly. + * If there are N threads, it ensures that each thread can acquire at least 1 / 2N of the memory + * before it has to spill, and at most 1 / N. Because N varies dynamically, we keep track of the + * set of active threads and redo the calculations of 1 / 2N and 1 / N in waiting threads whenever + * this set changes. This is all done by synchronizing access on "this" to mutate state and using + * wait() and notifyAll() to signal changes. + */ +private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { + private val threadMemory = new mutable.HashMap[Long, Long]() // threadId -> memory bytes + + def this(conf: SparkConf) = this(ShuffleMemoryManager.getMaxMemory(conf)) + + /** + * Try to acquire up to numBytes memory for the current thread, and return the number of bytes + * obtained, or 0 if none can be allocated. This call may block until there is enough free memory + * in some situations, to make sure each thread has a chance to ramp up to at least 1 / 2N of the + * total memory pool (where N is the # of active threads) before it is forced to spill. This can + * happen if the number of threads increases but an older thread had a lot of memory already. + */ + def tryToAcquire(numBytes: Long): Long = synchronized { + val threadId = Thread.currentThread().getId + assert(numBytes > 0, "invalid number of bytes requested: " + numBytes) + + // Add this thread to the threadMemory map just so we can keep an accurate count of the number + // of active threads, to let other threads ramp down their memory in calls to tryToAcquire + if (!threadMemory.contains(threadId)) { + threadMemory(threadId) = 0L + notifyAll() // Will later cause waiting threads to wake up and check numThreads again + } + + // Keep looping until we're either sure that we don't want to grant this request (because this + // thread would have more than 1 / numActiveThreads of the memory) or we have enough free + // memory to give it (we always let each thread get at least 1 / (2 * numActiveThreads)). + while (true) { + val numActiveThreads = threadMemory.keys.size + val curMem = threadMemory(threadId) + val freeMemory = maxMemory - threadMemory.values.sum + + // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads + val maxToGrant = math.min(numBytes, (maxMemory / numActiveThreads) - curMem) + + if (curMem < maxMemory / (2 * numActiveThreads)) { + // We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking; + // if we can't give it this much now, wait for other threads to free up memory + // (this happens if older threads allocated lots of memory before N grew) + if (freeMemory >= math.min(maxToGrant, maxMemory / (2 * numActiveThreads) - curMem)) { + val toGrant = math.min(maxToGrant, freeMemory) + threadMemory(threadId) += toGrant + return toGrant + } else { + logInfo(s"Thread $threadId waiting for at least 1/2N of shuffle memory pool to be free") + wait() + } + } else { + // Only give it as much memory as is free, which might be none if it reached 1 / numThreads + val toGrant = math.min(maxToGrant, freeMemory) + threadMemory(threadId) += toGrant + return toGrant + } + } + 0L // Never reached + } + + /** Release numBytes bytes for the current thread. */ + def release(numBytes: Long): Unit = synchronized { + val threadId = Thread.currentThread().getId + val curMem = threadMemory.getOrElse(threadId, 0L) + if (curMem < numBytes) { + throw new SparkException( + s"Internal error: release called on ${numBytes} bytes but thread only has ${curMem}") + } + threadMemory(threadId) -= numBytes + notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed + } + + /** Release all memory for the current thread and mark it as inactive (e.g. when a task ends). */ + def releaseMemoryForThisThread(): Unit = synchronized { + val threadId = Thread.currentThread().getId + threadMemory.remove(threadId) + notifyAll() // Notify waiters who locked "this" in tryToAcquire that memory has been freed + } +} + +private object ShuffleMemoryManager { + /** + * Figure out the shuffle memory limit from a SparkConf. We currently have both a fraction + * of the memory pool and a safety factor since collections can sometimes grow bigger than + * the size we target before we estimate their sizes again. + */ + def getMaxMemory(conf: SparkConf): Long = { + val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) + val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8) + (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong + } +} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 1f7d2dc838ebc..cc0423856cefb 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -71,13 +71,7 @@ class ExternalAppendOnlyMap[K, V, C]( private val spilledMaps = new ArrayBuffer[DiskMapIterator] private val sparkConf = SparkEnv.get.conf private val diskBlockManager = blockManager.diskBlockManager - - // Collective memory threshold shared across all running tasks - private val maxMemoryThreshold = { - val memoryFraction = sparkConf.getDouble("spark.shuffle.memoryFraction", 0.2) - val safetyFraction = sparkConf.getDouble("spark.shuffle.safetyFraction", 0.8) - (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong - } + private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager // Number of pairs inserted since last spill; note that we count them even if a value is merged // with a previous key in case we're doing something like groupBy where the result grows @@ -140,28 +134,15 @@ class ExternalAppendOnlyMap[K, V, C]( if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && currentMap.estimateSize() >= myMemoryThreshold) { - val currentSize = currentMap.estimateSize() - var shouldSpill = false - val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap - - // Atomically check whether there is sufficient memory in the global pool for - // this map to grow and, if possible, allocate the required amount - shuffleMemoryMap.synchronized { - val threadId = Thread.currentThread().getId - val previouslyOccupiedMemory = shuffleMemoryMap.get(threadId) - val availableMemory = maxMemoryThreshold - - (shuffleMemoryMap.values.sum - previouslyOccupiedMemory.getOrElse(0L)) - - // Try to allocate at least 2x more memory, otherwise spill - shouldSpill = availableMemory < currentSize * 2 - if (!shouldSpill) { - shuffleMemoryMap(threadId) = currentSize * 2 - myMemoryThreshold = currentSize * 2 - } - } - // Do not synchronize spills - if (shouldSpill) { - spill(currentSize) + // Claim up to double our current memory from the shuffle memory pool + val currentMemory = currentMap.estimateSize() + val amountToRequest = 2 * currentMemory - myMemoryThreshold + val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) + myMemoryThreshold += granted + if (myMemoryThreshold <= currentMemory) { + // We were granted too little memory to grow further (either tryToAcquire returned 0, + // or we already had more memory than myMemoryThreshold); spill the current collection + spill(currentMemory) // Will also release memory back to ShuffleMemoryManager } } currentMap.changeValue(curEntry._1, update) @@ -245,12 +226,9 @@ class ExternalAppendOnlyMap[K, V, C]( currentMap = new SizeTrackingAppendOnlyMap[K, C] spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes)) - // Reset the amount of shuffle memory used by this map in the global pool - val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap - shuffleMemoryMap.synchronized { - shuffleMemoryMap(Thread.currentThread().getId) = 0 - } - myMemoryThreshold = 0 + // Release our memory back to the shuffle pool so that other threads can grab it + shuffleMemoryManager.release(myMemoryThreshold) + myMemoryThreshold = 0L elementsRead = 0 _memoryBytesSpilled += mapSize diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index b04c50bd3e196..101c83b264f63 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -78,6 +78,7 @@ private[spark] class ExternalSorter[K, V, C]( private val blockManager = SparkEnv.get.blockManager private val diskBlockManager = blockManager.diskBlockManager + private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager private val ser = Serializer.getSerializer(serializer) private val serInstance = ser.newInstance() @@ -116,13 +117,6 @@ private[spark] class ExternalSorter[K, V, C]( private var _memoryBytesSpilled = 0L private var _diskBytesSpilled = 0L - // Collective memory threshold shared across all running tasks - private val maxMemoryThreshold = { - val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) - val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8) - (Runtime.getRuntime.maxMemory * memoryFraction * safetyFraction).toLong - } - // How much of the shared memory pool this collection has claimed private var myMemoryThreshold = 0L @@ -218,31 +212,15 @@ private[spark] class ExternalSorter[K, V, C]( if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && collection.estimateSize() >= myMemoryThreshold) { - // TODO: This logic doesn't work if there are two external collections being used in the same - // task (e.g. to read shuffle output and write it out into another shuffle) [SPARK-2711] - - val currentSize = collection.estimateSize() - var shouldSpill = false - val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap - - // Atomically check whether there is sufficient memory in the global pool for - // us to double our threshold - shuffleMemoryMap.synchronized { - val threadId = Thread.currentThread().getId - val previouslyClaimedMemory = shuffleMemoryMap.get(threadId) - val availableMemory = maxMemoryThreshold - - (shuffleMemoryMap.values.sum - previouslyClaimedMemory.getOrElse(0L)) - - // Try to allocate at least 2x more memory, otherwise spill - shouldSpill = availableMemory < currentSize * 2 - if (!shouldSpill) { - shuffleMemoryMap(threadId) = currentSize * 2 - myMemoryThreshold = currentSize * 2 - } - } - // Do not hold lock during spills - if (shouldSpill) { - spill(currentSize, usingMap) + // Claim up to double our current memory from the shuffle memory pool + val currentMemory = collection.estimateSize() + val amountToRequest = 2 * currentMemory - myMemoryThreshold + val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) + myMemoryThreshold += granted + if (myMemoryThreshold <= currentMemory) { + // We were granted too little memory to grow further (either tryToAcquire returned 0, + // or we already had more memory than myMemoryThreshold); spill the current collection + spill(currentMemory, usingMap) // Will also release memory back to ShuffleMemoryManager } } } @@ -327,11 +305,8 @@ private[spark] class ExternalSorter[K, V, C]( buffer = new SizeTrackingPairBuffer[(Int, K), C] } - // Reset the amount of shuffle memory used by this map in the global pool - val shuffleMemoryMap = SparkEnv.get.shuffleMemoryMap - shuffleMemoryMap.synchronized { - shuffleMemoryMap(Thread.currentThread().getId) = 0 - } + // Release our memory back to the shuffle pool so that other threads can grab it + shuffleMemoryManager.release(myMemoryThreshold) myMemoryThreshold = 0 spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition)) diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala new file mode 100644 index 0000000000000..d31bc22ee74f7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala @@ -0,0 +1,294 @@ +/* + * 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.shuffle + +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts +import org.scalatest.time.SpanSugar._ +import java.util.concurrent.atomic.AtomicBoolean +import java.util.concurrent.CountDownLatch + +class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { + /** Launch a thread with the given body block and return it. */ + private def startThread(name: String)(body: => Unit): Thread = { + val thread = new Thread("ShuffleMemorySuite " + name) { + override def run() { + body + } + } + thread.start() + thread + } + + test("single thread requesting memory") { + val manager = new ShuffleMemoryManager(1000L) + + assert(manager.tryToAcquire(100L) === 100L) + assert(manager.tryToAcquire(400L) === 400L) + assert(manager.tryToAcquire(400L) === 400L) + assert(manager.tryToAcquire(200L) === 100L) + assert(manager.tryToAcquire(100L) === 0L) + assert(manager.tryToAcquire(100L) === 0L) + + manager.release(500L) + assert(manager.tryToAcquire(300L) === 300L) + assert(manager.tryToAcquire(300L) === 200L) + + manager.releaseMemoryForThisThread() + assert(manager.tryToAcquire(1000L) === 1000L) + assert(manager.tryToAcquire(100L) === 0L) + } + + test("two threads requesting full memory") { + // Two threads request 500 bytes first, wait for each other to get it, and then request + // 500 more; we should immediately return 0 as both are now at 1 / N + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Result1 = -1L + var t2Result1 = -1L + var t1Result2 = -1L + var t2Result2 = -1L + } + val state = new State + + val t1 = startThread("t1") { + val r1 = manager.tryToAcquire(500L) + state.synchronized { + state.t1Result1 = r1 + state.notifyAll() + while (state.t2Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t1Result2 = r2 } + } + + val t2 = startThread("t2") { + val r1 = manager.tryToAcquire(500L) + state.synchronized { + state.t2Result1 = r1 + state.notifyAll() + while (state.t1Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t2Result2 = r2 } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + assert(state.t1Result1 === 500L) + assert(state.t2Result1 === 500L) + assert(state.t1Result2 === 0L) + assert(state.t2Result2 === 0L) + } + + + test("threads cannot grow past 1 / N") { + // Two threads request 250 bytes first, wait for each other to get it, and then request + // 500 more; we should only grant 250 bytes to each of them on this second request + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Result1 = -1L + var t2Result1 = -1L + var t1Result2 = -1L + var t2Result2 = -1L + } + val state = new State + + val t1 = startThread("t1") { + val r1 = manager.tryToAcquire(250L) + state.synchronized { + state.t1Result1 = r1 + state.notifyAll() + while (state.t2Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t1Result2 = r2 } + } + + val t2 = startThread("t2") { + val r1 = manager.tryToAcquire(250L) + state.synchronized { + state.t2Result1 = r1 + state.notifyAll() + while (state.t1Result1 === -1L) { + state.wait() + } + } + val r2 = manager.tryToAcquire(500L) + state.synchronized { state.t2Result2 = r2 } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + assert(state.t1Result1 === 250L) + assert(state.t2Result1 === 250L) + assert(state.t1Result2 === 250L) + assert(state.t2Result2 === 250L) + } + + test("threads can block to get at least 1 / 2N memory") { + // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps + // for a bit and releases 250 bytes, which should then be greanted to t2. Further requests + // by t2 will return false right away because it now has 1 / 2N of the memory. + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Requested = false + var t2Requested = false + var t1Result = -1L + var t2Result = -1L + var t2Result2 = -1L + var t2WaitTime = 0L + } + val state = new State + + val t1 = startThread("t1") { + state.synchronized { + state.t1Result = manager.tryToAcquire(1000L) + state.t1Requested = true + state.notifyAll() + while (!state.t2Requested) { + state.wait() + } + } + // Sleep a bit before releasing our memory; this is hacky but it would be difficult to make + // sure the other thread blocks for some time otherwise + Thread.sleep(300) + manager.release(250L) + } + + val t2 = startThread("t2") { + state.synchronized { + while (!state.t1Requested) { + state.wait() + } + state.t2Requested = true + state.notifyAll() + } + val startTime = System.currentTimeMillis() + val result = manager.tryToAcquire(250L) + val endTime = System.currentTimeMillis() + state.synchronized { + state.t2Result = result + // A second call should return 0 because we're now already at 1 / 2N + state.t2Result2 = manager.tryToAcquire(100L) + state.t2WaitTime = endTime - startTime + } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + // Both threads should've been able to acquire their memory; the second one will have waited + // until the first one acquired 1000 bytes and then released 250 + state.synchronized { + assert(state.t1Result === 1000L, "t1 could not allocate memory") + assert(state.t2Result === 250L, "t2 could not allocate memory") + assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})") + assert(state.t2Result2 === 0L, "t1 got extra memory the second time") + } + } + + test("releaseMemoryForThisThread") { + // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps + // for a bit and releases all its memory. t2 should now be able to grab all the memory. + + val manager = new ShuffleMemoryManager(1000L) + + class State { + var t1Requested = false + var t2Requested = false + var t1Result = -1L + var t2Result1 = -1L + var t2Result2 = -1L + var t2Result3 = -1L + var t2WaitTime = 0L + } + val state = new State + + val t1 = startThread("t1") { + state.synchronized { + state.t1Result = manager.tryToAcquire(1000L) + state.t1Requested = true + state.notifyAll() + while (!state.t2Requested) { + state.wait() + } + } + // Sleep a bit before releasing our memory; this is hacky but it would be difficult to make + // sure the other thread blocks for some time otherwise + Thread.sleep(300) + manager.releaseMemoryForThisThread() + } + + val t2 = startThread("t2") { + state.synchronized { + while (!state.t1Requested) { + state.wait() + } + state.t2Requested = true + state.notifyAll() + } + val startTime = System.currentTimeMillis() + val r1 = manager.tryToAcquire(500L) + val endTime = System.currentTimeMillis() + val r2 = manager.tryToAcquire(500L) + val r3 = manager.tryToAcquire(500L) + state.synchronized { + state.t2Result1 = r1 + state.t2Result2 = r2 + state.t2Result3 = r3 + state.t2WaitTime = endTime - startTime + } + } + + failAfter(20 seconds) { + t1.join() + t2.join() + } + + // Both threads should've been able to acquire their memory; the second one will have waited + // until the first one acquired 1000 bytes and then released all of it + state.synchronized { + assert(state.t1Result === 1000L, "t1 could not allocate memory") + assert(state.t2Result1 === 500L, "t2 didn't get 500 bytes the first time") + assert(state.t2Result2 === 500L, "t2 didn't get 500 bytes the second time") + assert(state.t2Result3 === 0L, s"t2 got more bytes a third time (${state.t2Result3})") + assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})") + } + } +} From a646a365e3beb8d0cd7e492e625ce68ee9439a07 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 5 Aug 2014 00:39:07 -0700 Subject: [PATCH 0244/1492] [SPARK-2857] Correct properties to set Master / Worker ports `master.ui.port` and `worker.ui.port` were never picked up by SparkConf, simply because they are not prefixed with "spark." Unfortunately, this is also currently the documented way of setting these values. Author: Andrew Or Closes #1779 from andrewor14/master-worker-port and squashes the following commits: 8475e95 [Andrew Or] Update docs to reflect changes in configs 4db3d5d [Andrew Or] Stop using configs that don't actually work --- .../org/apache/spark/deploy/master/MasterArguments.scala | 4 ++-- .../scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- docs/spark-standalone.md | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index a87781fb93850..4b0dbbe543d3f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -38,8 +38,8 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } - if (conf.contains("master.ui.port")) { - webUiPort = conf.get("master.ui.port").toInt + if (conf.contains("spark.master.ui.port")) { + webUiPort = conf.get("spark.master.ui.port").toInt } parse(args.toList) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 0ad2edba2227f..a9f531e9e4cae 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -58,6 +58,6 @@ private[spark] object WorkerWebUI { val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { - requestedPort.getOrElse(conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) + requestedPort.getOrElse(conf.getInt("spark.worker.ui.port", WorkerWebUI.DEFAULT_PORT)) } } diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 2fb30765f35e8..293a7ac9bc9aa 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -314,7 +314,7 @@ configure those ports. Standalone Cluster Master 8080 Web UI - master.ui.port + spark.master.ui.port Jetty-based @@ -338,7 +338,7 @@ configure those ports. Worker 8081 Web UI - worker.ui.port + spark.worker.ui.port Jetty-based From 9862c614c06507aa7624208f1d7ed5bc027ca52e Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 5 Aug 2014 00:51:07 -0700 Subject: [PATCH 0245/1492] [SPARK-1779] Throw an exception if memory fractions are not between 0 and 1 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Author: wangfei Author: wangfei Closes #714 from scwf/memoryFraction and squashes the following commits: 6e385b9 [wangfei] Update SparkConf.scala da6ee59 [wangfei] add configs 829a195 [wangfei] add indent 717c0ca [wangfei] updated to make more concise fc45476 [wangfei] validate memoryfraction in sparkconf 2e79b3d [wangfei] && => || 43621bd [wangfei] && => || cf38bcf [wangfei] throw IllegalArgumentException 14d18ac [wangfei] throw IllegalArgumentException dff1f0f [wangfei] Update BlockManager.scala 764965f [wangfei] Update ExternalAppendOnlyMap.scala a59d76b [wangfei] Throw exception when memoryFracton is out of range 7b899c2 [wangfei] 【SPARK-1779】 --- .../main/scala/org/apache/spark/SparkConf.scala | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 38700847c80f4..cce7a23d3b9fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -238,6 +238,20 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { } } + // Validate memory fractions + val memoryKeys = Seq( + "spark.storage.memoryFraction", + "spark.shuffle.memoryFraction", + "spark.shuffle.safetyFraction", + "spark.storage.unrollFraction", + "spark.storage.safetyFraction") + for (key <- memoryKeys) { + val value = getDouble(key, 0.5) + if (value > 1 || value < 0) { + throw new IllegalArgumentException("$key should be between 0 and 1 (was '$value').") + } + } + // Check for legacy configs sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = From 184048f80b6fa160c89d5bb47b937a0a89534a95 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 5 Aug 2014 01:30:46 -0700 Subject: [PATCH 0246/1492] [SPARK-2856] Decrease initial buffer size for Kryo to 64KB. Author: Reynold Xin Closes #1780 from rxin/kryo-init-size and squashes the following commits: 551b935 [Reynold Xin] [SPARK-2856] Decrease initial buffer size for Kryo to 64KB. --- .../scala/org/apache/spark/serializer/KryoSerializer.scala | 4 +++- docs/configuration.md | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index e60b802a86a14..407cb9db6ee9a 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -47,7 +47,9 @@ class KryoSerializer(conf: SparkConf) with Logging with Serializable { - private val bufferSize = conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 + private val bufferSize = + (conf.getDouble("spark.kryoserializer.buffer.mb", 0.064) * 1024 * 1024).toInt + private val maxBufferSize = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) * 1024 * 1024 private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) diff --git a/docs/configuration.md b/docs/configuration.md index 870343f1c0bd2..b3dee3f131411 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -412,7 +412,7 @@ Apart from these, the following properties are also available, and may be useful spark.kryoserializer.buffer.mb - 2 + 0.064 Initial size of Kryo's serialization buffer, in megabytes. Note that there will be one buffer per core on each worker. This buffer will grow up to From e87075df977a539e4a1684045a7bd66c36285174 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 5 Aug 2014 10:40:28 -0700 Subject: [PATCH 0247/1492] [SPARK-1022][Streaming] Add Kafka real unit test This PR is a updated version of (https://github.com/apache/spark/pull/557) to actually test sending and receiving data through Kafka, and fix previous flaky issues. @tdas, would you mind reviewing this PR? Thanks a lot. Author: jerryshao Closes #1751 from jerryshao/kafka-unit-test and squashes the following commits: b6a505f [jerryshao] code refactor according to comments 5222330 [jerryshao] Change JavaKafkaStreamSuite to better test it 5525f10 [jerryshao] Fix flaky issue of Kafka real unit test 4559310 [jerryshao] Minor changes for Kafka unit test 860f649 [jerryshao] Minor style changes, and tests ignored due to flakiness 796d4ca [jerryshao] Add real Kafka streaming test --- external/kafka/pom.xml | 6 + .../streaming/kafka/JavaKafkaStreamSuite.java | 125 +++++++++-- .../streaming/kafka/KafkaStreamSuite.scala | 197 ++++++++++++++++-- 3 files changed, 293 insertions(+), 35 deletions(-) diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index daf03360bc5f5..2aee99949223a 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -70,6 +70,12 @@ + + net.sf.jopt-simple + jopt-simple + 3.2 + test + org.scalatest scalatest_${scala.binary.version} diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 9f8046bf00f8f..0571454c01dae 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -17,31 +17,118 @@ package org.apache.spark.streaming.kafka; +import java.io.Serializable; import java.util.HashMap; +import java.util.List; + +import scala.Predef; +import scala.Tuple2; +import scala.collection.JavaConverters; + +import junit.framework.Assert; -import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; -import org.junit.Test; -import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; + +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; import org.apache.spark.streaming.LocalJavaStreamingContext; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +import org.junit.Test; +import org.junit.After; +import org.junit.Before; + +public class JavaKafkaStreamSuite extends LocalJavaStreamingContext implements Serializable { + private transient KafkaStreamSuite testSuite = new KafkaStreamSuite(); + + @Before + @Override + public void setUp() { + testSuite.beforeFunction(); + System.clearProperty("spark.driver.port"); + //System.setProperty("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock"); + ssc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); + } + + @After + @Override + public void tearDown() { + ssc.stop(); + ssc = null; + System.clearProperty("spark.driver.port"); + testSuite.afterFunction(); + } -public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { @Test - public void testKafkaStream() { - HashMap topics = Maps.newHashMap(); - - // tests the API, does not actually test data receiving - JavaPairReceiverInputDStream test1 = - KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); - JavaPairReceiverInputDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, - StorageLevel.MEMORY_AND_DISK_SER_2()); - - HashMap kafkaParams = Maps.newHashMap(); - kafkaParams.put("zookeeper.connect", "localhost:12345"); - kafkaParams.put("group.id","consumer-group"); - JavaPairReceiverInputDStream test3 = KafkaUtils.createStream(ssc, - String.class, String.class, StringDecoder.class, StringDecoder.class, - kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); + public void testKafkaStream() throws InterruptedException { + String topic = "topic1"; + HashMap topics = new HashMap(); + topics.put(topic, 1); + + HashMap sent = new HashMap(); + sent.put("a", 5); + sent.put("b", 3); + sent.put("c", 10); + + testSuite.createTopic(topic); + HashMap tmp = new HashMap(sent); + testSuite.produceAndSendMessage(topic, + JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( + Predef.>conforms())); + + HashMap kafkaParams = new HashMap(); + kafkaParams.put("zookeeper.connect", testSuite.zkConnect()); + kafkaParams.put("group.id", "test-consumer-" + KafkaTestUtils.random().nextInt(10000)); + kafkaParams.put("auto.offset.reset", "smallest"); + + JavaPairDStream stream = KafkaUtils.createStream(ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY_SER()); + + final HashMap result = new HashMap(); + + JavaDStream words = stream.map( + new Function, String>() { + @Override + public String call(Tuple2 tuple2) throws Exception { + return tuple2._2(); + } + } + ); + + words.countByValue().foreachRDD( + new Function, Void>() { + @Override + public Void call(JavaPairRDD rdd) throws Exception { + List> ret = rdd.collect(); + for (Tuple2 r : ret) { + if (result.containsKey(r._1())) { + result.put(r._1(), result.get(r._1()) + r._2()); + } else { + result.put(r._1(), r._2()); + } + } + + return null; + } + } + ); + + ssc.start(); + ssc.awaitTermination(3000); + + Assert.assertEquals(sent.size(), result.size()); + for (String k : sent.keySet()) { + Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); + } } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index e6f2c4a5cf5d1..c0b55e9340253 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -17,28 +17,193 @@ package org.apache.spark.streaming.kafka -import kafka.serializer.StringDecoder +import java.io.File +import java.net.InetSocketAddress +import java.util.{Properties, Random} + +import scala.collection.mutable + +import kafka.admin.CreateTopicCommand +import kafka.common.TopicAndPartition +import kafka.producer.{KeyedMessage, ProducerConfig, Producer} +import kafka.utils.ZKStringSerializer +import kafka.serializer.{StringDecoder, StringEncoder} +import kafka.server.{KafkaConfig, KafkaServer} + +import org.I0Itec.zkclient.ZkClient + +import org.apache.zookeeper.server.ZooKeeperServer +import org.apache.zookeeper.server.NIOServerCnxnFactory + import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.ReceiverInputDStream +import org.apache.spark.util.Utils class KafkaStreamSuite extends TestSuiteBase { + import KafkaTestUtils._ + + val zkConnect = "localhost:2181" + val zkConnectionTimeout = 6000 + val zkSessionTimeout = 6000 + + val brokerPort = 9092 + val brokerProps = getBrokerConfig(brokerPort, zkConnect) + val brokerConf = new KafkaConfig(brokerProps) + + protected var zookeeper: EmbeddedZookeeper = _ + protected var zkClient: ZkClient = _ + protected var server: KafkaServer = _ + protected var producer: Producer[String, String] = _ + + override def useManualClock = false + + override def beforeFunction() { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(zkConnect) + logInfo("==================== 0 ====================") + zkClient = new ZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) + logInfo("==================== 1 ====================") - test("kafka input stream") { + // Kafka broker startup + server = new KafkaServer(brokerConf) + logInfo("==================== 2 ====================") + server.startup() + logInfo("==================== 3 ====================") + Thread.sleep(2000) + logInfo("==================== 4 ====================") + super.beforeFunction() + } + + override def afterFunction() { + producer.close() + server.shutdown() + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + zkClient.close() + zookeeper.shutdown() + + super.afterFunction() + } + + test("Kafka input stream") { val ssc = new StreamingContext(master, framework, batchDuration) - val topics = Map("my-topic" -> 1) - - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[(String, String)] = - KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) - val test2: ReceiverInputDStream[(String, String)] = - KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) - val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3: ReceiverInputDStream[(String, String)] = - KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) - - // TODO: Actually test receiving data + val topic = "topic1" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + createTopic(topic) + produceAndSendMessage(topic, sent) + + val kafkaParams = Map("zookeeper.connect" -> zkConnect, + "group.id" -> s"test-consumer-${random.nextInt(10000)}", + "auto.offset.reset" -> "smallest") + + val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + ssc, + kafkaParams, + Map(topic -> 1), + StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() + stream.map { case (k, v) => v } + .countByValue() + .foreachRDD { r => + val ret = r.collect() + ret.toMap.foreach { kv => + val count = result.getOrElseUpdate(kv._1, 0) + kv._2 + result.put(kv._1, count) + } + } + ssc.start() + ssc.awaitTermination(3000) + + assert(sent.size === result.size) + sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } + ssc.stop() } + + private def createTestMessage(topic: String, sent: Map[String, Int]) + : Seq[KeyedMessage[String, String]] = { + val messages = for ((s, freq) <- sent; i <- 0 until freq) yield { + new KeyedMessage[String, String](topic, s) + } + messages.toSeq + } + + def createTopic(topic: String) { + CreateTopicCommand.createTopic(zkClient, topic, 1, 1, "0") + logInfo("==================== 5 ====================") + // wait until metadata is propagated + waitUntilMetadataIsPropagated(Seq(server), topic, 0, 1000) + } + + def produceAndSendMessage(topic: String, sent: Map[String, Int]) { + val brokerAddr = brokerConf.hostName + ":" + brokerConf.port + producer = new Producer[String, String](new ProducerConfig(getProducerConfig(brokerAddr))) + producer.send(createTestMessage(topic, sent): _*) + logInfo("==================== 6 ====================") + } +} + +object KafkaTestUtils { + val random = new Random() + + def getBrokerConfig(port: Int, zkConnect: String): Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", port.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkConnect) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + def getProducerConfig(brokerList: String): Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerList) + props.put("serializer.class", classOf[StringEncoder].getName) + props + } + + def waitUntilTrue(condition: () => Boolean, waitTime: Long): Boolean = { + val startTime = System.currentTimeMillis() + while (true) { + if (condition()) + return true + if (System.currentTimeMillis() > startTime + waitTime) + return false + Thread.sleep(waitTime.min(100L)) + } + // Should never go to here + throw new RuntimeException("unexpected error") + } + + def waitUntilMetadataIsPropagated(servers: Seq[KafkaServer], topic: String, partition: Int, + timeout: Long) { + assert(waitUntilTrue(() => + servers.foldLeft(true)(_ && _.apis.leaderCache.keySet.contains( + TopicAndPartition(topic, partition))), timeout), + s"Partition [$topic, $partition] metadata not propagated after timeout") + } + + class EmbeddedZookeeper(val zkConnect: String) { + val random = new Random() + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } } From 2c0f705e26ca3dfc43a1e9a0722c0e57f67c970a Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 5 Aug 2014 12:48:26 -0500 Subject: [PATCH 0248/1492] SPARK-1528 - spark on yarn, add support for accessing remote HDFS Add a config (spark.yarn.access.namenodes) to allow applications running on yarn to access other secure HDFS cluster. User just specifies the namenodes of the other clusters and we get Tokens for those and ship them with the spark application. Author: Thomas Graves Closes #1159 from tgravescs/spark-1528 and squashes the following commits: ddbcd16 [Thomas Graves] review comments 0ac8501 [Thomas Graves] SPARK-1528 - add support for accessing remote HDFS --- docs/running-on-yarn.md | 7 +++ .../apache/spark/deploy/yarn/ClientBase.scala | 56 +++++++++++++------ .../spark/deploy/yarn/ClientBaseSuite.scala | 55 +++++++++++++++++- 3 files changed, 101 insertions(+), 17 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 0362f5a223319..573930dbf4e54 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -106,6 +106,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes set this configuration to "hdfs:///some/path". + + spark.yarn.access.namenodes + (none) + + A list of secure HDFS namenodes your Spark application is going to access. For example, `spark.yarn.access.namenodes=hdfs://nn1.com:8032,hdfs://nn2.com:8032`. The Spark application must have acess to the namenodes listed and Kerberos must be properly configured to be able to access them (either in the same realm or in a trusted realm). Spark acquires security tokens for each of the namenodes so that the Spark application can access those remote HDFS clusters. + + # Launching Spark on YARN diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index b7e8636e02eb2..ed8f56ab8b75e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -191,23 +191,11 @@ trait ClientBase extends Logging { // Upload Spark and the application JAR to the remote file system if necessary. Add them as // local resources to the application master. val fs = FileSystem.get(conf) - - val delegTokenRenewer = Master.getMasterPrincipal(conf) - if (UserGroupInformation.isSecurityEnabled()) { - if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { - val errorMessage = "Can't get Master Kerberos principal for use as renewer" - logError(errorMessage) - throw new SparkException(errorMessage) - } - } val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort - - if (UserGroupInformation.isSecurityEnabled()) { - val dstFs = dst.getFileSystem(conf) - dstFs.addDelegationTokens(delegTokenRenewer, credentials) - } + val nns = ClientBase.getNameNodesToAccess(sparkConf) + dst + ClientBase.obtainTokensForNamenodes(nns, conf, credentials) + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort val localResources = HashMap[String, LocalResource]() FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) @@ -614,4 +602,40 @@ object ClientBase extends Logging { YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, File.pathSeparator) + /** + * Get the list of namenodes the user may access. + */ + private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "").split(",").map(_.trim()).filter(!_.isEmpty) + .map(new Path(_)).toSet + } + + private[yarn] def getTokenRenewer(conf: Configuration): String = { + val delegTokenRenewer = Master.getMasterPrincipal(conf) + logDebug("delegation token renewer is: " + delegTokenRenewer) + if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { + val errorMessage = "Can't get Master Kerberos principal for use as renewer" + logError(errorMessage) + throw new SparkException(errorMessage) + } + delegTokenRenewer + } + + /** + * Obtains tokens for the namenodes passed in and adds them to the credentials. + */ + private[yarn] def obtainTokensForNamenodes(paths: Set[Path], conf: Configuration, + creds: Credentials) { + if (UserGroupInformation.isSecurityEnabled()) { + val delegTokenRenewer = getTokenRenewer(conf) + + paths.foreach { + dst => + val dstFs = dst.getFileSystem(conf) + logDebug("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) + } + } + } + } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 686714dc36488..68cc2890f3a22 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -31,6 +31,8 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ import org.mockito.Mockito._ + + import org.scalatest.FunSuite import org.scalatest.Matchers @@ -38,7 +40,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ HashMap => MutableHashMap } import scala.util.Try -import org.apache.spark.SparkConf +import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.util.Utils class ClientBaseSuite extends FunSuite with Matchers { @@ -138,6 +140,57 @@ class ClientBaseSuite extends FunSuite with Matchers { } } + test("check access nns empty") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns unset") { + val sparkConf = new SparkConf() + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set()) + } + + test("check access nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access nns space") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032, ") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"))) + } + + test("check access two nns") { + val sparkConf = new SparkConf() + sparkConf.set("spark.yarn.access.namenodes", "hdfs://nn1:8032,hdfs://nn2:8032") + val nns = ClientBase.getNameNodesToAccess(sparkConf) + nns should be(Set(new Path("hdfs://nn1:8032"), new Path("hdfs://nn2:8032"))) + } + + test("check token renewer") { + val hadoopConf = new Configuration() + hadoopConf.set("yarn.resourcemanager.address", "myrm:8033") + hadoopConf.set("yarn.resourcemanager.principal", "yarn/myrm:8032@SPARKTEST.COM") + val renewer = ClientBase.getTokenRenewer(hadoopConf) + renewer should be ("yarn/myrm:8032@SPARKTEST.COM") + } + + test("check token renewer default") { + val hadoopConf = new Configuration() + val caught = + intercept[SparkException] { + ClientBase.getTokenRenewer(hadoopConf) + } + assert(caught.getMessage === "Can't get Master Kerberos principal for use as renewer") + } + object Fixtures { val knownDefYarnAppCP: Seq[String] = From 1c5555a23d3aa40423d658cfbf2c956ad415a6b1 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 5 Aug 2014 12:52:52 -0500 Subject: [PATCH 0249/1492] SPARK-1890 and SPARK-1891- add admin and modify acls It was easier to combine these 2 jira since they touch many of the same places. This pr adds the following: - adds modify acls - adds admin acls (list of admins/users that get added to both view and modify acls) - modify Kill button on UI to take modify acls into account - changes config name of spark.ui.acls.enable to spark.acls.enable since I choose poorly in original name. We keep backwards compatibility so people can still use spark.ui.acls.enable. The acls should apply to any web ui as well as any CLI interfaces. - send view and modify acls information on to YARN so that YARN interfaces can use (yarn cli for killing applications for example). Author: Thomas Graves Closes #1196 from tgravescs/SPARK-1890 and squashes the following commits: 8292eb1 [Thomas Graves] review comments b92ec89 [Thomas Graves] remove unneeded variable from applistener 4c765f4 [Thomas Graves] Add in admin acls 72eb0ac [Thomas Graves] Add modify acls --- .../org/apache/spark/SecurityManager.scala | 107 +++++++++++++++--- .../deploy/history/FsHistoryProvider.scala | 4 +- .../scheduler/ApplicationEventListener.scala | 4 +- .../apache/spark/ui/jobs/JobProgressTab.scala | 2 +- .../apache/spark/SecurityManagerSuite.scala | 83 ++++++++++++-- docs/configuration.md | 27 ++++- docs/security.md | 7 +- .../apache/spark/deploy/yarn/ClientBase.scala | 9 +- 8 files changed, 206 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 74aa441619bd2..25c2c9fc6af7c 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -41,10 +41,19 @@ import org.apache.spark.deploy.SparkHadoopUtil * secure the UI if it has data that other users should not be allowed to see. The javax * servlet filter specified by the user can authenticate the user and then once the user * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' + * authorized to view the UI. The configs 'spark.acls.enable' and 'spark.ui.view.acls' * control the behavior of the acls. Note that the person who started the application * always has view access to the UI. * + * Spark has a set of modify acls (`spark.modify.acls`) that controls which users have permission + * to modify a single application. This would include things like killing the application. By + * default the person who started the application has modify access. For modify access through + * the UI, you must have a filter that does authentication in place for the modify acls to work + * properly. + * + * Spark also has a set of admin acls (`spark.admin.acls`) which is a set of users/administrators + * who always have permission to view or modify the Spark application. + * * Spark does not currently support encryption after authentication. * * At this point spark has multiple communication protocols that need to be secured and @@ -137,18 +146,32 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { private val sparkSecretLookupKey = "sparkCookie" private val authOn = sparkConf.getBoolean("spark.authenticate", false) - private var uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + // keep spark.ui.acls.enable for backwards compatibility with 1.0 + private var aclsOn = sparkConf.getOption("spark.acls.enable").getOrElse( + sparkConf.get("spark.ui.acls.enable", "false")).toBoolean + + // admin acls should be set before view or modify acls + private var adminAcls: Set[String] = + stringToSet(sparkConf.get("spark.admin.acls", "")) private var viewAcls: Set[String] = _ + + // list of users who have permission to modify the application. This should + // apply to both UI and CLI for things like killing the application. + private var modifyAcls: Set[String] = _ + // always add the current user and SPARK_USER to the viewAcls - private val defaultAclUsers = Seq[String](System.getProperty("user.name", ""), + private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Option(System.getenv("SPARK_USER")).getOrElse("")) + setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) + setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) private val secretKey = generateSecretKey() logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + - "; ui acls " + (if (uiAclsOn) "enabled" else "disabled") + - "; users with view permissions: " + viewAcls.toString()) + "; ui acls " + (if (aclsOn) "enabled" else "disabled") + + "; users with view permissions: " + viewAcls.toString() + + "; users with modify permissions: " + modifyAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. // This is needed by the HTTP client fetching from the HttpServer. Put here so its @@ -169,18 +192,51 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { ) } - private[spark] def setViewAcls(defaultUsers: Seq[String], allowedUsers: String) { - viewAcls = (defaultUsers ++ allowedUsers.split(',')).map(_.trim()).filter(!_.isEmpty).toSet + /** + * Split a comma separated String, filter out any empty items, and return a Set of strings + */ + private def stringToSet(list: String): Set[String] = { + list.split(',').map(_.trim).filter(!_.isEmpty).toSet + } + + /** + * Admin acls should be set before the view or modify acls. If you modify the admin + * acls you should also set the view and modify acls again to pick up the changes. + */ + def setViewAcls(defaultUsers: Set[String], allowedUsers: String) { + viewAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) logInfo("Changing view acls to: " + viewAcls.mkString(",")) } - private[spark] def setViewAcls(defaultUser: String, allowedUsers: String) { - setViewAcls(Seq[String](defaultUser), allowedUsers) + def setViewAcls(defaultUser: String, allowedUsers: String) { + setViewAcls(Set[String](defaultUser), allowedUsers) + } + + def getViewAcls: String = viewAcls.mkString(",") + + /** + * Admin acls should be set before the view or modify acls. If you modify the admin + * acls you should also set the view and modify acls again to pick up the changes. + */ + def setModifyAcls(defaultUsers: Set[String], allowedUsers: String) { + modifyAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) + } + + def getModifyAcls: String = modifyAcls.mkString(",") + + /** + * Admin acls should be set before the view or modify acls. If you modify the admin + * acls you should also set the view and modify acls again to pick up the changes. + */ + def setAdminAcls(adminUsers: String) { + adminAcls = stringToSet(adminUsers) + logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } - private[spark] def setUIAcls(aclSetting: Boolean) { - uiAclsOn = aclSetting - logInfo("Changing acls enabled to: " + uiAclsOn) + def setAcls(aclSetting: Boolean) { + aclsOn = aclSetting + logInfo("Changing acls enabled to: " + aclsOn) } /** @@ -224,22 +280,39 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { * Check to see if Acls for the UI are enabled * @return true if UI authentication is enabled, otherwise false */ - def uiAclsEnabled(): Boolean = uiAclsOn + def aclsEnabled(): Boolean = aclsOn /** * Checks the given user against the view acl list to see if they have - * authorization to view the UI. If the UI acls must are disabled - * via spark.ui.acls.enable, all users have view access. + * authorization to view the UI. If the UI acls are disabled + * via spark.acls.enable, all users have view access. If the user is null + * it is assumed authentication is off and all users have access. * * @param user to see if is authorized * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { - logDebug("user=" + user + " uiAclsEnabled=" + uiAclsEnabled() + " viewAcls=" + + logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" + viewAcls.mkString(",")) - if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true + if (aclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true } + /** + * Checks the given user against the modify acl list to see if they have + * authorization to modify the application. If the UI acls are disabled + * via spark.acls.enable, all users have modify access. If the user is null + * it is assumed authentication isn't turned on and all users have access. + * + * @param user to see if is authorized + * @return true is the user has permission, otherwise false + */ + def checkModifyPermissions(user: String): Boolean = { + logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" + + modifyAcls.mkString(",")) + if (aclsEnabled() && (user != null) && (!modifyAcls.contains(user))) false else true + } + + /** * Check to see if authentication for the Spark communication protocols is enabled * @return true if authentication is enabled, otherwise false diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 6d2d4cef1ee46..cc06540ee0647 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -189,7 +189,9 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis if (ui != null) { val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) - ui.getSecurityManager.setUIAcls(uiAclsEnabled) + ui.getSecurityManager.setAcls(uiAclsEnabled) + // make sure to set admin acls before view acls so properly picked up + ui.getSecurityManager.setAdminAcls(appListener.adminAcls) ui.getSecurityManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) } (appInfo, ui) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index cd5d44ad4a7e6..162158babc35b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -29,7 +29,7 @@ private[spark] class ApplicationEventListener extends SparkListener { var startTime = -1L var endTime = -1L var viewAcls = "" - var enableViewAcls = false + var adminAcls = "" def applicationStarted = startTime != -1 @@ -55,7 +55,7 @@ private[spark] class ApplicationEventListener extends SparkListener { val environmentDetails = environmentUpdate.environmentDetails val allProperties = environmentDetails("Spark Properties").toMap viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") - enableViewAcls = allProperties.getOrElse("spark.ui.acls.enable", "false").toBoolean + adminAcls = allProperties.getOrElse("spark.admin.acls", "") } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 3308c8c8a3d37..8a01ec80c9dd6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -41,7 +41,7 @@ private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stag def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) def handleKillRequest(request: HttpServletRequest) = { - if (killEnabled) { + if ((killEnabled) && (parent.securityManager.checkModifyPermissions(request.getRemoteUser))) { val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index e39093e24d68a..fcca0867b8072 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -31,7 +31,7 @@ class SecurityManagerSuite extends FunSuite { conf.set("spark.ui.view.acls", "user1,user2") val securityManager = new SecurityManager(conf); assert(securityManager.isAuthenticationEnabled() === true) - assert(securityManager.uiAclsEnabled() === true) + assert(securityManager.aclsEnabled() === true) assert(securityManager.checkUIViewPermissions("user1") === true) assert(securityManager.checkUIViewPermissions("user2") === true) assert(securityManager.checkUIViewPermissions("user3") === false) @@ -41,17 +41,17 @@ class SecurityManagerSuite extends FunSuite { val conf = new SparkConf conf.set("spark.ui.view.acls", "user1,user2") val securityManager = new SecurityManager(conf); - securityManager.setUIAcls(true) - assert(securityManager.uiAclsEnabled() === true) - securityManager.setUIAcls(false) - assert(securityManager.uiAclsEnabled() === false) + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setAcls(false) + assert(securityManager.aclsEnabled() === false) // acls are off so doesn't matter what view acls set to assert(securityManager.checkUIViewPermissions("user4") === true) - securityManager.setUIAcls(true) - assert(securityManager.uiAclsEnabled() === true) - securityManager.setViewAcls(ArrayBuffer[String]("user5"), "user6,user7") + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setViewAcls(Set[String]("user5"), "user6,user7") assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user5") === true) assert(securityManager.checkUIViewPermissions("user6") === true) @@ -59,5 +59,72 @@ class SecurityManagerSuite extends FunSuite { assert(securityManager.checkUIViewPermissions("user8") === false) assert(securityManager.checkUIViewPermissions(null) === true) } + + test("set security modify acls") { + val conf = new SparkConf + conf.set("spark.modify.acls", "user1,user2") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setAcls(false) + assert(securityManager.aclsEnabled() === false) + + // acls are off so doesn't matter what view acls set to + assert(securityManager.checkModifyPermissions("user4") === true) + + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + securityManager.setModifyAcls(Set("user5"), "user6,user7") + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user5") === true) + assert(securityManager.checkModifyPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user8") === false) + assert(securityManager.checkModifyPermissions(null) === true) + } + + test("set security admin acls") { + val conf = new SparkConf + conf.set("spark.admin.acls", "user1,user2") + conf.set("spark.ui.view.acls", "user3") + conf.set("spark.modify.acls", "user4") + + val securityManager = new SecurityManager(conf); + securityManager.setAcls(true) + assert(securityManager.aclsEnabled() === true) + + assert(securityManager.checkModifyPermissions("user1") === true) + assert(securityManager.checkModifyPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user4") === true) + assert(securityManager.checkModifyPermissions("user3") === false) + assert(securityManager.checkModifyPermissions("user5") === false) + assert(securityManager.checkModifyPermissions(null) === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user3") === true) + assert(securityManager.checkUIViewPermissions("user4") === false) + assert(securityManager.checkUIViewPermissions("user5") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + + securityManager.setAdminAcls("user6") + securityManager.setViewAcls(Set[String]("user8"), "user9") + securityManager.setModifyAcls(Set("user11"), "user9") + assert(securityManager.checkModifyPermissions("user6") === true) + assert(securityManager.checkModifyPermissions("user11") === true) + assert(securityManager.checkModifyPermissions("user9") === true) + assert(securityManager.checkModifyPermissions("user1") === false) + assert(securityManager.checkModifyPermissions("user4") === false) + assert(securityManager.checkModifyPermissions(null) === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkUIViewPermissions("user8") === true) + assert(securityManager.checkUIViewPermissions("user9") === true) + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user3") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + + } + + } diff --git a/docs/configuration.md b/docs/configuration.md index b3dee3f131411..25adea210cba0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -815,13 +815,13 @@ Apart from these, the following properties are also available, and may be useful - spark.ui.acls.enable + spark.acls.enable false - Whether Spark web ui acls should are enabled. If enabled, this checks to see if the user has - access permissions to view the web ui. See spark.ui.view.acls for more details. - Also note this requires the user to be known, if the user comes across as null no checks - are done. Filters can be used to authenticate and set the user. + Whether Spark acls should are enabled. If enabled, this checks to see if the user has + access permissions to view or modify the job. Note this requires the user to be known, + so if the user comes across as null no checks are done. Filters can be used with the UI + to authenticate and set the user. @@ -832,6 +832,23 @@ Apart from these, the following properties are also available, and may be useful user that started the Spark job has view access. + + spark.modify.acls + Empty + + Comma separated list of users that have modify access to the Spark job. By default only the + user that started the Spark job has access to modify it (kill it for example). + + + + spark.admin.acls + Empty + + Comma separated list of users/administrators that have view and modify access to all Spark jobs. + This can be used if you run on a shared cluster and have a set of administrators or devs who + help debug when things work. + + #### Spark Streaming diff --git a/docs/security.md b/docs/security.md index 90ba678033b19..8312f8d017e1f 100644 --- a/docs/security.md +++ b/docs/security.md @@ -8,8 +8,11 @@ Spark currently supports authentication via a shared secret. Authentication can * For Spark on [YARN](running-on-yarn.html) deployments, configuring `spark.authenticate` to `true` will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. * For other types of Spark deployments, the Spark parameter `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. -The Spark UI can also be secured by using [javax servlet filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html) via the `spark.ui.filters` setting. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.ui.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. -On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. +The Spark UI can also be secured by using [javax servlet filters](http://docs.oracle.com/javaee/6/api/javax/servlet/Filter.html) via the `spark.ui.filters` setting. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view ACLs to make sure they are authorized to view the UI. The configs `spark.acls.enable` and `spark.ui.view.acls` control the behavior of the ACLs. Note that the user who started the application always has view access to the UI. On YARN, the Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. + +Spark also supports modify ACLs to control who has access to modify a running Spark application. This includes things like killing the application or a task. This is controlled by the configs `spark.acls.enable` and `spark.modify.acls`. Note that if you are authenticating the web UI, in order to use the kill button on the web UI it might be necessary to add the users in the modify acls to the view acls also. On YARN, the modify acls are passed in and control who has modify access via YARN interfaces. + +Spark allows for a set of administrators to be specified in the acls who always have view and modify permissions to all the applications. is controlled by the config `spark.admin.acls`. This is useful on a shared cluster where you might have administrators or support staff who help users debug applications. If your applications are using event logging, the directory where the event logs go (`spark.eventLog.dir`) should be manually created and have the proper permissions set on it. If you want those log files secured, the permissions should be set to `drwxrwxrwxt` for that directory. The owner of the directory should be the super user who is running the history server and the group permissions should be restricted to super user group. This will allow all users to write to the directory but will prevent unprivileged users from removing or renaming a file unless they own the file or directory. The event log files will be created by Spark with permissions such that only the user and group have read and write access. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index ed8f56ab8b75e..44e025b8f60ba 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records -import org.apache.spark.{SparkException, Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The @@ -405,6 +405,13 @@ trait ClientBase extends Logging { amContainer.setCommands(printableCommands) setupSecurityToken(amContainer) + + // send the acl settings into YARN to control who has access via YARN interfaces + val securityManager = new SecurityManager(sparkConf) + val acls = Map[ApplicationAccessType, String] ( + ApplicationAccessType.VIEW_APP -> securityManager.getViewAcls, + ApplicationAccessType.MODIFY_APP -> securityManager.getModifyAcls) + amContainer.setApplicationACLs(acls) amContainer } } From 6e821e3d1ae1ed23459bc7f1098510b968130152 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 5 Aug 2014 11:17:50 -0700 Subject: [PATCH 0250/1492] [SPARK-2860][SQL] Fix coercion of CASE WHEN. Author: Michael Armbrust Closes #1785 from marmbrus/caseNull and squashes the following commits: 126006d [Michael Armbrust] better error message 2fe357f [Michael Armbrust] Fix coercion of CASE WHEN. --- .../catalyst/analysis/HiveTypeCoercion.scala | 56 +++++++++++-------- ...ll case-0-581cdfe70091e546414b202da2cebdcb | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 3 + 3 files changed, 36 insertions(+), 24 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/null case-0-581cdfe70091e546414b202da2cebdcb diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index e94f2a3bea63e..15eb5982a4a91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -49,10 +49,21 @@ trait HiveTypeCoercion { BooleanCasts :: StringToIntegralCasts :: FunctionArgumentConversion :: - CastNulls :: + CaseWhenCoercion :: Division :: Nil + trait TypeWidening { + def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + } + } + /** * Applies any changes to [[AttributeReference]] data types that are made by other rules to * instances higher in the query tree. @@ -133,16 +144,7 @@ trait HiveTypeCoercion { * - LongType to FloatType * - LongType to DoubleType */ - object WidenTypes extends Rule[LogicalPlan] { - - def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - } + object WidenTypes extends Rule[LogicalPlan] with TypeWidening { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case u @ Union(left, right) if u.childrenResolved && !u.resolved => @@ -336,28 +338,34 @@ trait HiveTypeCoercion { } /** - * Ensures that NullType gets casted to some other types under certain circumstances. + * Coerces the type of different branches of a CASE WHEN statement to a common type. */ - object CastNulls extends Rule[LogicalPlan] { + object CaseWhenCoercion extends Rule[LogicalPlan] with TypeWidening { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case cw @ CaseWhen(branches) => + case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => val valueTypes = branches.sliding(2, 2).map { - case Seq(_, value) if value.resolved => Some(value.dataType) - case Seq(elseVal) if elseVal.resolved => Some(elseVal.dataType) - case _ => None + case Seq(_, value) => value.dataType + case Seq(elseVal) => elseVal.dataType }.toSeq - if (valueTypes.distinct.size == 2 && valueTypes.exists(_ == Some(NullType))) { - val otherType = valueTypes.filterNot(_ == Some(NullType))(0).get + + logDebug(s"Input values for null casting ${valueTypes.mkString(",")}") + + if (valueTypes.distinct.size > 1) { + val commonType = valueTypes.reduce { (v1, v2) => + findTightestCommonType(v1, v2) + .getOrElse(sys.error( + s"Types in CASE WHEN must be the same or coercible to a common type: $v1 != $v2")) + } val transformedBranches = branches.sliding(2, 2).map { - case Seq(cond, value) if value.resolved && value.dataType == NullType => - Seq(cond, Cast(value, otherType)) - case Seq(elseVal) if elseVal.resolved && elseVal.dataType == NullType => - Seq(Cast(elseVal, otherType)) + case Seq(cond, value) if value.dataType != commonType => + Seq(cond, Cast(value, commonType)) + case Seq(elseVal) if elseVal.dataType != commonType => + Seq(Cast(elseVal, commonType)) case s => s }.reduce(_ ++ _) CaseWhen(transformedBranches) } else { - // It is possible to have more types due to the possibility of short-circuiting. + // Types match up. Hopefully some other rule fixes whatever is wrong with resolution. cw } } diff --git a/sql/hive/src/test/resources/golden/null case-0-581cdfe70091e546414b202da2cebdcb b/sql/hive/src/test/resources/golden/null case-0-581cdfe70091e546414b202da2cebdcb new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/null case-0-581cdfe70091e546414b202da2cebdcb @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index aa810a291231a..2f0be49b6a6d7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -32,6 +32,9 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("null case", + "SELECT case when(true) then 1 else null end FROM src LIMIT 1") + createQueryTest("single case", """SELECT case when true then 1 else 2 end FROM src LIMIT 1""") From ac3440f4f3c4b79070ffec7db0b08ad062b4df90 Mon Sep 17 00:00:00 2001 From: "Guancheng (G.C.) Chen" Date: Tue, 5 Aug 2014 11:50:08 -0700 Subject: [PATCH 0251/1492] [SPARK-2859] Update url of Kryo project in related docs JIRA Issue: https://issues.apache.org/jira/browse/SPARK-2859 Kryo project has been migrated from googlecode to github, hence we need to update its URL in related docs such as tuning.md. Author: Guancheng (G.C.) Chen Closes #1782 from gchen/kryo-docs and squashes the following commits: b62543c [Guancheng (G.C.) Chen] update url of Kryo project --- docs/tuning.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 4917c11bc1147..8fb2a0433b1a8 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -32,7 +32,7 @@ in your operations) and performance. It provides two serialization libraries: [`java.io.Externalizable`](http://docs.oracle.com/javase/6/docs/api/java/io/Externalizable.html). Java serialization is flexible but often quite slow, and leads to large serialized formats for many classes. -* [Kryo serialization](http://code.google.com/p/kryo/): Spark can also use +* [Kryo serialization](https://github.com/EsotericSoftware/kryo): Spark can also use the Kryo library (version 2) to serialize objects more quickly. Kryo is significantly faster and more compact than Java serialization (often as much as 10x), but does not support all `Serializable` types and requires you to *register* the classes you'll use in the program in advance @@ -68,7 +68,7 @@ conf.set("spark.kryo.registrator", "mypackage.MyRegistrator") val sc = new SparkContext(conf) {% endhighlight %} -The [Kryo documentation](http://code.google.com/p/kryo/) describes more advanced +The [Kryo documentation](https://github.com/EsotericSoftware/kryo) describes more advanced registration options, such as adding custom serialization code. If your objects are large, you may also need to increase the `spark.kryoserializer.buffer.mb` From 74f82c71b03d265a7d0c98ce196ca8c44de002e8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 5 Aug 2014 13:08:23 -0700 Subject: [PATCH 0252/1492] SPARK-2380: Support displaying accumulator values in the web UI This patch adds support for giving accumulators user-visible names and displaying accumulator values in the web UI. This allows users to create custom counters that can display in the UI. The current approach displays both the accumulator deltas caused by each task and a "current" value of the accumulator totals for each stage, which gets update as tasks finish. Currently in Spark developers have been extending the `TaskMetrics` functionality to provide custom instrumentation for RDD's. This provides a potentially nicer alternative of going through the existing accumulator framework (actually `TaskMetrics` and accumulators are on an awkward collision course as we add more features to the former). The current patch demo's how we can use the feature to provide instrumentation for RDD input sizes. The nice thing about going through accumulators is that users can read the current value of the data being tracked in their programs. This could be useful to e.g. decide to short-circuit a Spark stage depending on how things are going. ![counters](https://cloud.githubusercontent.com/assets/320616/3488815/6ee7bc34-0505-11e4-84ce-e36d9886e2cf.png) Author: Patrick Wendell Closes #1309 from pwendell/metrics and squashes the following commits: 8815308 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into HEAD 93fbe0f [Patrick Wendell] Other minor fixes cc43f68 [Patrick Wendell] Updating unit tests c991b1b [Patrick Wendell] Moving some code into the Accumulators class 9a9ba3c [Patrick Wendell] More merge fixes c5ace9e [Patrick Wendell] More merge conflicts 1da15e3 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into metrics 9860c55 [Patrick Wendell] Potential solution to posting listener events 0bb0e33 [Patrick Wendell] Remove "display" variable and assume display = name.isDefined 0ec4ac7 [Patrick Wendell] Java API's e95bf69 [Patrick Wendell] Stash be97261 [Patrick Wendell] Style fix 8407308 [Patrick Wendell] Removing examples in Hadoop and RDD class 64d405f [Patrick Wendell] Adding missing file 5d8b156 [Patrick Wendell] Changes based on Kay's review. 9f18bad [Patrick Wendell] Minor style changes and tests 7a63abc [Patrick Wendell] Adding Json serialization and responding to Reynold's feedback ad85076 [Patrick Wendell] Example of using named accumulators for custom RDD metrics. 0b72660 [Patrick Wendell] Initial WIP example of supporing globally named accumulators. --- .../scala/org/apache/spark/Accumulators.scala | 19 ++++-- .../scala/org/apache/spark/SparkContext.scala | 19 ++++++ .../spark/api/java/JavaSparkContext.scala | 59 ++++++++++++++++++ .../spark/scheduler/AccumulableInfo.scala | 46 ++++++++++++++ .../apache/spark/scheduler/DAGScheduler.scala | 24 ++++++- .../apache/spark/scheduler/StageInfo.scala | 4 ++ .../org/apache/spark/scheduler/TaskInfo.scala | 9 +++ .../spark/ui/jobs/JobProgressListener.scala | 10 ++- .../org/apache/spark/ui/jobs/StagePage.scala | 21 ++++++- .../org/apache/spark/ui/jobs/UIData.scala | 3 +- .../org/apache/spark/util/JsonProtocol.scala | 39 +++++++++++- .../apache/spark/util/JsonProtocolSuite.scala | 62 +++++++++++++++---- docs/programming-guide.md | 6 +- 13 files changed, 294 insertions(+), 27 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 9c55bfbb47626..12f2fe031cb1d 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -36,15 +36,21 @@ import org.apache.spark.serializer.JavaSerializer * * @param initialValue initial value of accumulator * @param param helper object defining how to add elements of type `R` and `T` + * @param name human-readable name for use in Spark's web UI * @tparam R the full accumulated data (result type) * @tparam T partial data that can be added in */ class Accumulable[R, T] ( @transient initialValue: R, - param: AccumulableParam[R, T]) + param: AccumulableParam[R, T], + val name: Option[String]) extends Serializable { - val id = Accumulators.newId + def this(@transient initialValue: R, param: AccumulableParam[R, T]) = + this(initialValue, param, None) + + val id: Long = Accumulators.newId + @transient private var value_ = initialValue // Current value on master val zero = param.zero(initialValue) // Zero value to be passed to workers private var deserialized = false @@ -219,8 +225,10 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa * @param param helper object defining how to add elements of type `T` * @tparam T result type */ -class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T]) - extends Accumulable[T,T](initialValue, param) +class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T], name: Option[String]) + extends Accumulable[T,T](initialValue, param, name) { + def this(initialValue: T, param: AccumulatorParam[T]) = this(initialValue, param, None) +} /** * A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add @@ -281,4 +289,7 @@ private object Accumulators { } } } + + def stringifyPartialValue(partialValue: Any) = "%s".format(partialValue) + def stringifyValue(value: Any) = "%s".format(value) } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9ba21cfcde01a..e132955f0f850 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -760,6 +760,15 @@ class SparkContext(config: SparkConf) extends Logging { def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]) = new Accumulator(initialValue, param) + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, with a name for display + * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the + * driver can access the accumulator's `value`. + */ + def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) = { + new Accumulator(initialValue, param, Some(name)) + } + /** * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values * with `+=`. Only the driver can access the accumuable's `value`. @@ -769,6 +778,16 @@ class SparkContext(config: SparkConf) extends Logging { def accumulable[T, R](initialValue: T)(implicit param: AccumulableParam[T, R]) = new Accumulable(initialValue, param) + /** + * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the + * Spark UI. Tasks can add values to the accumuable using the `+=` operator. Only the driver can + * access the accumuable's `value`. + * @tparam T accumulator type + * @tparam R type that can be added to the accumulator + */ + def accumulable[T, R](initialValue: T, name: String)(implicit param: AccumulableParam[T, R]) = + new Accumulable(initialValue, param, Some(name)) + /** * Create an accumulator from a "mutable collection" type. * diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index d9d1c5955ca99..e0a4815940db3 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -429,6 +429,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] = sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]] + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def intAccumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = + sc.accumulator(initialValue, name)(IntAccumulatorParam) + .asInstanceOf[Accumulator[java.lang.Integer]] + /** * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. @@ -436,12 +446,31 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def doubleAccumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = + sc.accumulator(initialValue, name)(DoubleAccumulatorParam) + .asInstanceOf[Accumulator[java.lang.Double]] + /** * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. */ def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue) + /** + * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = + intAccumulator(initialValue, name) + /** * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values * to using the `add` method. Only the master can access the accumulator's `value`. @@ -449,6 +478,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def accumulator(initialValue: Double): Accumulator[java.lang.Double] = doubleAccumulator(initialValue) + + /** + * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values + * to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = + doubleAccumulator(initialValue, name) + /** * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" * values to using the `add` method. Only the master can access the accumulator's `value`. @@ -456,6 +495,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = sc.accumulator(initialValue)(accumulatorParam) + /** + * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" + * values to using the `add` method. Only the master can access the accumulator's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulator[T](initialValue: T, name: String, accumulatorParam: AccumulatorParam[T]) + : Accumulator[T] = + sc.accumulator(initialValue, name)(accumulatorParam) + /** * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks * can "add" values with `add`. Only the master can access the accumuable's `value`. @@ -463,6 +512,16 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = sc.accumulable(initialValue)(param) + /** + * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks + * can "add" values with `add`. Only the master can access the accumuable's `value`. + * + * This version supports naming the accumulator for display in Spark's web UI. + */ + def accumulable[T, R](initialValue: T, name: String, param: AccumulableParam[T, R]) + : Accumulable[T, R] = + sc.accumulable(initialValue, name)(param) + /** * Broadcast a read-only variable to the cluster, returning a * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala new file mode 100644 index 0000000000000..fa83372bb4d11 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -0,0 +1,46 @@ +/* + * 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 + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Information about an [[org.apache.spark.Accumulable]] modified during a task or stage. + */ +@DeveloperApi +class AccumulableInfo ( + val id: Long, + val name: String, + val update: Option[String], // represents a partial update within a task + val value: String) { + + override def equals(other: Any): Boolean = other match { + case acc: AccumulableInfo => + this.id == acc.id && this.name == acc.name && + this.update == acc.update && this.value == acc.value + case _ => false + } +} + +object AccumulableInfo { + def apply(id: Long, name: String, update: Option[String], value: String) = + new AccumulableInfo(id, name, update, value) + + def apply(id: Long, name: String, value: String) = new AccumulableInfo(id, name, None, value) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 9fa3a4e9c71ae..430e45ada5808 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -883,8 +883,14 @@ class DAGScheduler( val task = event.task val stageId = task.stageId val taskType = Utils.getFormattedClassName(task) - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, - event.taskMetrics)) + + // The success case is dealt with separately below, since we need to compute accumulator + // updates before posting. + if (event.reason != Success) { + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) + } + if (!stageIdToStage.contains(task.stageId)) { // Skip all the actions if the stage has been cancelled. return @@ -906,12 +912,26 @@ class DAGScheduler( if (event.accumUpdates != null) { try { Accumulators.add(event.accumUpdates) + event.accumUpdates.foreach { case (id, partialValue) => + val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] + // To avoid UI cruft, ignore cases where value wasn't updated + if (acc.name.isDefined && partialValue != acc.zero) { + val name = acc.name.get + val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) + val stringValue = Accumulators.stringifyValue(acc.value) + stage.info.accumulables(id) = AccumulableInfo(id, name, stringValue) + event.taskInfo.accumulables += + AccumulableInfo(id, name, Some(stringPartialValue), stringValue) + } + } } catch { // If we see an exception during accumulator update, just log the error and move on. case e: Exception => logError(s"Failed to update accumulators for $task", e) } } + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 480891550eb60..2a407e47a05bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.HashMap + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo @@ -37,6 +39,8 @@ class StageInfo( var completionTime: Option[Long] = None /** If the stage failed, the reason why. */ var failureReason: Option[String] = None + /** Terminal values of accumulables updated during this stage. */ + val accumulables = HashMap[Long, AccumulableInfo]() def stageFailed(reason: String) { failureReason = Some(reason) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index ca0595f35143e..6fa1f2c880f7a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.ListBuffer + import org.apache.spark.annotation.DeveloperApi /** @@ -41,6 +43,13 @@ class TaskInfo( */ var gettingResultTime: Long = 0 + /** + * Intermediate updates to accumulables during this task. Note that it is valid for the same + * accumulable to be updated multiple times in a single task or for two accumulables with the + * same name but different IDs to exist in a task. + */ + val accumulables = ListBuffer[AccumulableInfo]() + /** * The time when the task has completed successfully (including the time to remotely fetch * results, if necessary). diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index da2f5d3172fe2..a57a354620163 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{HashMap, ListBuffer} +import scala.collection.mutable.{HashMap, ListBuffer, Map} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi @@ -65,6 +65,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { new StageUIData }) + for ((id, info) <- stageCompleted.stageInfo.accumulables) { + stageData.accumulables(id) = info + } + poolToActiveStages.get(stageData.schedulingPool).foreach(_.remove(stageId)) activeStages.remove(stageId) if (stage.failureReason.isEmpty) { @@ -130,6 +134,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { new StageUIData }) + for (accumulableInfo <- info.accumulables) { + stageData.accumulables(accumulableInfo.id) = accumulableInfo + } + val execSummaryMap = stageData.executorSummary val execSummary = execSummaryMap.getOrElseUpdate(info.executorId, new ExecutorSummary) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index cab26b9e2f7d3..8bc1ba758cf77 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -20,11 +20,12 @@ package org.apache.spark.ui.jobs import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.xml.Node +import scala.xml.{Node, Unparsed} import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.util.{Utils, Distribution} +import org.apache.spark.scheduler.AccumulableInfo /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { @@ -51,6 +52,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val tasks = stageData.taskData.values.toSeq.sortBy(_.taskInfo.launchTime) val numCompleted = tasks.count(_.taskInfo.finished) + val accumulables = listener.stageIdToData(stageId).accumulables val hasInput = stageData.inputBytes > 0 val hasShuffleRead = stageData.shuffleReadBytes > 0 val hasShuffleWrite = stageData.shuffleWriteBytes > 0 @@ -95,10 +97,15 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { // scalastyle:on + val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") + def accumulableRow(acc: AccumulableInfo) = {acc.name}{acc.value} + val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow, + accumulables.values.toSeq) + val taskHeaders: Seq[String] = Seq( "Index", "ID", "Attempt", "Status", "Locality Level", "Executor", - "Launch Time", "Duration", "GC Time") ++ + "Launch Time", "Duration", "GC Time", "Accumulators") ++ {if (hasInput) Seq("Input") else Nil} ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ @@ -208,11 +215,16 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } val executorTable = new ExecutorTable(stageId, parent) + + val maybeAccumulableTable: Seq[Node] = + if (accumulables.size > 0) {

    Accumulators

    ++ accumulableTable } else Seq() + val content = summary ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++ + maybeAccumulableTable ++

    Tasks

    ++ taskTable UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), @@ -279,6 +291,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} + + {Unparsed( + info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
    ") + )} + - - Browser - Standalone Cluster Master - 8080 - Web UI - spark.master.ui.port - Jetty-based - - - Browser - Driver - 4040 - Web UI - spark.ui.port - Jetty-based - - - Browser - History Server - 18080 - Web UI - spark.history.ui.port - Jetty-based - - - Browser - Worker - 8081 - Web UI - spark.worker.ui.port - Jetty-based - - - - Application - Standalone Cluster Master - 7077 - Submit job to cluster - spark.driver.port - Akka-based. Set to "0" to choose a port randomly - - - Worker - Standalone Cluster Master - 7077 - Join cluster - spark.driver.port - Akka-based. Set to "0" to choose a port randomly - - - Application - Worker - (random) - Join cluster - SPARK_WORKER_PORT (standalone cluster) - Akka-based - - - - - Driver and other Workers - Worker - (random) - -
      -
    • File server for file and jars
    • -
    • Http Broadcast
    • -
    • Class file server (Spark Shell only)
    • -
    - - None - Jetty-based. Each of these services starts on a random port that cannot be configured - - - +Spark makes heavy use of the network, and some environments have strict requirements for using +tight firewall settings. For a complete list of ports to configure, see the +[security page](security.html#configuring-ports-for-network-security). # High Availability By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below. -## Standby Masters with ZooKeeper +# Standby Masters with ZooKeeper **Overview** @@ -429,7 +347,7 @@ There's an important distinction to be made between "registering with a Master" Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. -## Single-Node Recovery with Local File System +# Single-Node Recovery with Local File System **Overview** diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aac621fe53938..40b588512ff08 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -330,6 +330,8 @@ object TestSettings { fork := true, javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", + javaOptions in Test += "-Dspark.ports.maxRetries=100", + javaOptions in Test += "-Dspark.ui.port=0", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index f60bbb4662af1..84b57cd2dc1af 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -102,7 +102,8 @@ import org.apache.spark.util.Utils val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles /** Jetty server that will serve our classes to worker nodes */ - val classServer = new HttpServer(outputDir, new SecurityManager(conf)) + val classServerPort = conf.getInt("spark.replClassServer.port", 0) + val classServer = new HttpServer(outputDir, new SecurityManager(conf), classServerPort, "HTTP class server") private var currentSettings: Settings = initialSettings var printResults = true // whether to print result lines var totalSilence = false // whether to print anything From 48789117c2dd6d38e0bd8d21cdbcb989913205a6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 6 Aug 2014 11:08:12 -0700 Subject: [PATCH 0269/1492] [SPARK-2875] [PySpark] [SQL] handle null in schemaRDD() Handle null in schemaRDD during converting them into Python. Author: Davies Liu Closes #1802 from davies/json and squashes the following commits: 88e6b1f [Davies Liu] handle null in schemaRDD() --- python/pyspark/sql.py | 7 +++++ .../org/apache/spark/sql/SchemaRDD.scala | 27 +++++++++++-------- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index f1093701ddc89..adc56e7ec0e2b 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1231,6 +1231,13 @@ def jsonRDD(self, rdd, schema=None): ... "field3.field5[0] as f3 from table3") >>> srdd6.collect() [Row(f1=u'row1', f2=None,...Row(f1=u'row3', f2=[], f3=None)] + + >>> sqlCtx.jsonRDD(sc.parallelize(['{}', + ... '{"key0": {"key1": "value1"}}'])).collect() + [Row(key0=None), Row(key0=Row(key1=u'value1'))] + >>> sqlCtx.jsonRDD(sc.parallelize(['{"key0": null}', + ... '{"key0": {"key1": "value1"}}'])).collect() + [Row(key0=None), Row(key0=Row(key1=u'value1'))] """ def func(iterator): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 57df79321b35d..33b2ed1b3a399 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -382,21 +382,26 @@ class SchemaRDD( private[sql] def javaToPython: JavaRDD[Array[Byte]] = { import scala.collection.Map - def toJava(obj: Any, dataType: DataType): Any = dataType match { - case struct: StructType => rowToArray(obj.asInstanceOf[Row], struct) - case array: ArrayType => obj match { - case seq: Seq[Any] => seq.map(x => toJava(x, array.elementType)).asJava - case list: JList[_] => list.map(x => toJava(x, array.elementType)).asJava - case arr if arr != null && arr.getClass.isArray => - arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) - case other => other - } - case mt: MapType => obj.asInstanceOf[Map[_, _]].map { + def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + case (null, _) => null + + case (obj: Row, struct: StructType) => rowToArray(obj, struct) + + case (seq: Seq[Any], array: ArrayType) => + seq.map(x => toJava(x, array.elementType)).asJava + case (list: JList[_], array: ArrayType) => + list.map(x => toJava(x, array.elementType)).asJava + case (arr, array: ArrayType) if arr.getClass.isArray => + arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) + + case (obj: Map[_, _], mt: MapType) => obj.map { case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type }.asJava + // Pyrolite can handle Timestamp - case other => obj + case (other, _) => other } + def rowToArray(row: Row, structType: StructType): Array[Any] = { val fields = structType.fields.map(field => field.dataType) row.zip(fields).map { From a6cd31108f0d73ce6823daafe8447677e03cfd13 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 6 Aug 2014 12:28:35 -0700 Subject: [PATCH 0270/1492] [SPARK-2678][Core][SQL] A workaround for SPARK-2678 JIRA issues: - Main: [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678) - Related: [SPARK-2874](https://issues.apache.org/jira/browse/SPARK-2874) Related PR: - #1715 This PR is both a fix for SPARK-2874 and a workaround for SPARK-2678. Fixing SPARK-2678 completely requires some API level changes that need further discussion, and we decided not to include it in Spark 1.1 release. As currently SPARK-2678 only affects Spark SQL scripts, this workaround is enough for Spark 1.1. Command line option handling logic in bash scripts looks somewhat dirty and duplicated, but it helps to provide a cleaner user interface as well as retain full downward compatibility for now. Author: Cheng Lian Closes #1801 from liancheng/spark-2874 and squashes the following commits: 8045d7a [Cheng Lian] Make sure test suites pass 8493a9e [Cheng Lian] Using eval to retain quoted arguments aed523f [Cheng Lian] Fixed typo in bin/spark-sql f12a0b1 [Cheng Lian] Worked arount SPARK-2678 daee105 [Cheng Lian] Fixed usage messages of all Spark SQL related scripts --- bin/beeline | 29 ++------ bin/spark-sql | 66 +++++++++++++++++-- .../spark/deploy/SparkSubmitArguments.scala | 39 ++++------- .../spark/deploy/SparkSubmitSuite.scala | 12 ++++ sbin/start-thriftserver.sh | 50 ++++++++++++-- .../hive/thriftserver/HiveThriftServer2.scala | 1 - .../sql/hive/thriftserver/CliSuite.scala | 19 +++--- .../thriftserver/HiveThriftServer2Suite.scala | 23 ++++--- 8 files changed, 164 insertions(+), 75 deletions(-) diff --git a/bin/beeline b/bin/beeline index 09fe366c609fa..1bda4dba50605 100755 --- a/bin/beeline +++ b/bin/beeline @@ -17,29 +17,14 @@ # limitations under the License. # -# Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +# +# Shell script for starting BeeLine -# Find the java binary -if [ -n "${JAVA_HOME}" ]; then - RUNNER="${JAVA_HOME}/bin/java" -else - if [ `command -v java` ]; then - RUNNER="java" - else - echo "JAVA_HOME is not set" >&2 - exit 1 - fi -fi +# Enter posix mode for bash +set -o posix -# Compute classpath using external script -classpath_output=$($FWDIR/bin/compute-classpath.sh) -if [[ "$?" != "0" ]]; then - echo "$classpath_output" - exit 1 -else - CLASSPATH=$classpath_output -fi +# Figure out where Spark is installed +FWDIR="$(cd `dirname $0`/..; pwd)" CLASS="org.apache.hive.beeline.BeeLine" -exec "$RUNNER" -cp "$CLASSPATH" $CLASS "$@" +exec "$FWDIR/bin/spark-class" $CLASS "$@" diff --git a/bin/spark-sql b/bin/spark-sql index bba7f897b19bc..61ebd8ab6dec8 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -23,14 +23,72 @@ # Enter posix mode for bash set -o posix +CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" + # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/spark-sql [options]" +function usage { + echo "Usage: ./sbin/spark-sql [options] [cli option]" + pattern="usage" + pattern+="\|Spark assembly has been built with Hive" + pattern+="\|NOTE: SPARK_PREPEND_CLASSES is set" + pattern+="\|Spark Command: " + pattern+="\|--help" + pattern+="\|=======" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + echo + echo "CLI options:" + $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 +} + +function ensure_arg_number { + arg_number=$1 + at_least=$2 + + if [[ $arg_number -lt $at_least ]]; then + usage + exit 1 + fi +} + +if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then + usage exit 0 fi -CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ +CLI_ARGS=() +SUBMISSION_ARGS=() + +while (($#)); do + case $1 in + -d | --define | --database | -f | -h | --hiveconf | --hivevar | -i | -p) + ensure_arg_number $# 2 + CLI_ARGS+=($1); shift + CLI_ARGS+=($1); shift + ;; + + -e) + ensure_arg_number $# 2 + CLI_ARGS+=($1); shift + CLI_ARGS+=(\"$1\"); shift + ;; + + -s | --silent) + CLI_ARGS+=($1); shift + ;; + + -v | --verbose) + # Both SparkSubmit and SparkSQLCLIDriver recognizes -v | --verbose + CLI_ARGS+=($1) + SUBMISSION_ARGS+=($1); shift + ;; + + *) + SUBMISSION_ARGS+=($1); shift + ;; + esac +done + +eval exec "$FWDIR"/bin/spark-submit --class $CLASS ${SUBMISSION_ARGS[*]} spark-internal ${CLI_ARGS[*]} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 9391f24e71ed7..087dd4d633db0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -220,6 +220,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { var inSparkOpts = true + val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r // Delineates parsing of Spark options from parsing of user options. parse(opts) @@ -322,33 +323,21 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { verbose = true parse(tail) + case EQ_SEPARATED_OPT(opt, value) :: tail => + parse(opt :: value :: tail) + + case value :: tail if value.startsWith("-") => + SparkSubmit.printErrorAndExit(s"Unrecognized option '$value'.") + case value :: tail => - if (inSparkOpts) { - value match { - // convert --foo=bar to --foo bar - case v if v.startsWith("--") && v.contains("=") && v.split("=").size == 2 => - val parts = v.split("=") - parse(Seq(parts(0), parts(1)) ++ tail) - case v if v.startsWith("-") => - val errMessage = s"Unrecognized option '$value'." - SparkSubmit.printErrorAndExit(errMessage) - case v => - primaryResource = - if (!SparkSubmit.isShell(v) && !SparkSubmit.isInternal(v)) { - Utils.resolveURI(v).toString - } else { - v - } - inSparkOpts = false - isPython = SparkSubmit.isPython(v) - parse(tail) + primaryResource = + if (!SparkSubmit.isShell(value) && !SparkSubmit.isInternal(value)) { + Utils.resolveURI(value).toString + } else { + value } - } else { - if (!value.isEmpty) { - childArgs += value - } - parse(tail) - } + isPython = SparkSubmit.isPython(value) + childArgs ++= tail case Nil => } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index a5cdcfb5de03b..7e1ef80c84561 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -106,6 +106,18 @@ class SparkSubmitSuite extends FunSuite with Matchers { appArgs.childArgs should be (Seq("some", "--weird", "args")) } + test("handles arguments to user program with name collision") { + val clArgs = Seq( + "--name", "myApp", + "--class", "Foo", + "userjar.jar", + "--master", "local", + "some", + "--weird", "args") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.childArgs should be (Seq("--master", "local", "some", "--weird", "args")) + } + test("handles YARN cluster mode") { val clArgs = Seq( "--deploy-mode", "cluster", diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 8398e6f19b511..603f50ae13240 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -26,11 +26,53 @@ set -o posix # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/start-thriftserver [options]" +CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" + +function usage { + echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" + pattern="usage" + pattern+="\|Spark assembly has been built with Hive" + pattern+="\|NOTE: SPARK_PREPEND_CLASSES is set" + pattern+="\|Spark Command: " + pattern+="\|=======" + pattern+="\|--help" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + echo + echo "Thrift server options:" + $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 +} + +function ensure_arg_number { + arg_number=$1 + at_least=$2 + + if [[ $arg_number -lt $at_least ]]; then + usage + exit 1 + fi +} + +if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then + usage exit 0 fi -CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -exec "$FWDIR"/bin/spark-submit --class $CLASS spark-internal $@ +THRIFT_SERVER_ARGS=() +SUBMISSION_ARGS=() + +while (($#)); do + case $1 in + --hiveconf) + ensure_arg_number $# 2 + THRIFT_SERVER_ARGS+=($1); shift + THRIFT_SERVER_ARGS+=($1); shift + ;; + + *) + SUBMISSION_ARGS+=($1); shift + ;; + esac +done + +eval exec "$FWDIR"/bin/spark-submit --class $CLASS ${SUBMISSION_ARGS[*]} spark-internal ${THRIFT_SERVER_ARGS[*]} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 08d3f983d9e71..6f7942aba314a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -40,7 +40,6 @@ private[hive] object HiveThriftServer2 extends Logging { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") if (!optionsProcessor.process(args)) { - logWarning("Error starting HiveThriftServer2 with given arguments") System.exit(-1) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 69f19f826a802..2bf8cfdcacd22 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.{BufferedReader, InputStreamReader, PrintWriter} +import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { @@ -27,15 +28,15 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { val METASTORE_PATH = TestUtils.getMetastorePath("cli") override def beforeAll() { - val pb = new ProcessBuilder( - "../../bin/spark-sql", - "--master", - "local", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - "hive.metastore.warehouse.dir=" + WAREHOUSE_PATH) - + val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" + val commands = + s"""../../bin/spark-sql + | --master local + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$WAREHOUSE_PATH + """.stripMargin.split("\\s+") + + val pb = new ProcessBuilder(commands: _*) process = pb.start() outputWriter = new PrintWriter(process.getOutputStream, true) inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index b7b7c9957ac34..78bffa2607349 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -25,6 +25,7 @@ import java.io.{BufferedReader, InputStreamReader} import java.net.ServerSocket import java.sql.{Connection, DriverManager, Statement} +import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging @@ -63,16 +64,18 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt // Forking a new process to start the Hive Thrift server. The reason to do this is it is // hard to clean up Hive resources entirely, so we just start a new process and kill // that process for cleanup. - val defaultArgs = Seq( - "../../sbin/start-thriftserver.sh", - "--master local", - "--hiveconf", - "hive.root.logger=INFO,console", - "--hiveconf", - s"javax.jdo.option.ConnectionURL=jdbc:derby:;databaseName=$METASTORE_PATH;create=true", - "--hiveconf", - s"hive.metastore.warehouse.dir=$WAREHOUSE_PATH") - val pb = new ProcessBuilder(defaultArgs ++ args) + val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" + val command = + s"""../../sbin/start-thriftserver.sh + | --master local + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$METASTORE_PATH + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$HOST + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$PORT + """.stripMargin.split("\\s+") + + val pb = new ProcessBuilder(command ++ args: _*) val environment = pb.environment() environment.put("HIVE_SERVER2_THRIFT_PORT", PORT.toString) environment.put("HIVE_SERVER2_THRIFT_BIND_HOST", HOST) From d614967b0bad1e6c5277d612602ec0a653a00258 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 6 Aug 2014 12:58:24 -0700 Subject: [PATCH 0271/1492] [SPARK-2627] [PySpark] have the build enforce PEP 8 automatically As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that. Notes: * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server. * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request. * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete. * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo. Author: Nicholas Chammas Author: nchammas Closes #1744 from nchammas/master and squashes the following commits: 274b238 [Nicholas Chammas] [SPARK-2627] [PySpark] minor indentation changes 983d963 [nchammas] Merge pull request #5 from apache/master 1db5314 [nchammas] Merge pull request #4 from apache/master 0e0245f [Nicholas Chammas] [SPARK-2627] undo erroneous whitespace fixes bf30942 [Nicholas Chammas] [SPARK-2627] PEP8: comment spacing 6db9a44 [nchammas] Merge pull request #3 from apache/master 7b4750e [Nicholas Chammas] merge upstream changes 91b7584 [Nicholas Chammas] [SPARK-2627] undo unnecessary line breaks 44e3e56 [Nicholas Chammas] [SPARK-2627] use tox.ini to exclude files b09fae2 [Nicholas Chammas] don't wrap comments unnecessarily bfb9f9f [Nicholas Chammas] [SPARK-2627] keep up with the PEP 8 fixes 9da347f [nchammas] Merge pull request #2 from apache/master aa5b4b5 [Nicholas Chammas] [SPARK-2627] follow Spark bash style for if blocks d0a83b9 [Nicholas Chammas] [SPARK-2627] check that pep8 downloaded fine dffb5dd [Nicholas Chammas] [SPARK-2627] download pep8 at runtime a1ce7ae [Nicholas Chammas] [SPARK-2627] space out test report sections 21da538 [Nicholas Chammas] [SPARK-2627] it's PEP 8, not PEP8 6f4900b [Nicholas Chammas] [SPARK-2627] more misc PEP 8 fixes fe57ed0 [Nicholas Chammas] removing merge conflict backups 9c01d4c [nchammas] Merge pull request #1 from apache/master 9a66cb0 [Nicholas Chammas] resolving merge conflicts a31ccc4 [Nicholas Chammas] [SPARK-2627] miscellaneous PEP 8 fixes beaa9ac [Nicholas Chammas] [SPARK-2627] fail check on non-zero status 723ed39 [Nicholas Chammas] always delete the report file 0541ebb [Nicholas Chammas] [SPARK-2627] call Python linter from run-tests 12440fa [Nicholas Chammas] [SPARK-2627] add Scala linter 61c07b9 [Nicholas Chammas] [SPARK-2627] add Python linter 75ad552 [Nicholas Chammas] make check output style consistent --- dev/lint-python | 60 +++++++++++ dev/lint-scala | 23 ++++ dev/run-tests | 13 ++- dev/scalastyle | 2 +- python/pyspark/accumulators.py | 7 ++ python/pyspark/broadcast.py | 1 + python/pyspark/conf.py | 1 + python/pyspark/context.py | 25 +++-- python/pyspark/daemon.py | 5 +- python/pyspark/files.py | 1 + python/pyspark/java_gateway.py | 1 + python/pyspark/mllib/_common.py | 5 +- python/pyspark/mllib/classification.py | 8 ++ python/pyspark/mllib/clustering.py | 3 + python/pyspark/mllib/linalg.py | 2 + python/pyspark/mllib/random.py | 14 +-- python/pyspark/mllib/recommendation.py | 2 + python/pyspark/mllib/regression.py | 12 +++ python/pyspark/mllib/stat.py | 1 + python/pyspark/mllib/tests.py | 11 +- python/pyspark/mllib/tree.py | 4 +- python/pyspark/mllib/util.py | 1 + python/pyspark/rdd.py | 22 ++-- python/pyspark/rddsampler.py | 4 + python/pyspark/resultiterable.py | 2 + python/pyspark/serializers.py | 21 +++- python/pyspark/shuffle.py | 20 ++-- python/pyspark/sql.py | 66 ++++++++---- python/pyspark/storagelevel.py | 1 + python/pyspark/tests.py | 143 ++++++++++++++----------- python/test_support/userlibrary.py | 2 + tox.ini | 1 + 32 files changed, 348 insertions(+), 136 deletions(-) create mode 100755 dev/lint-python create mode 100755 dev/lint-scala diff --git a/dev/lint-python b/dev/lint-python new file mode 100755 index 0000000000000..4efddad839387 --- /dev/null +++ b/dev/lint-python @@ -0,0 +1,60 @@ +#!/usr/bin/env bash + +# +# 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. +# + +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" +SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" +PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" + +cd $SPARK_ROOT_DIR + +# Get pep8 at runtime so that we don't rely on it being installed on the build server. +#+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 +#+ TODOs: +#+ - Dynamically determine latest release version of pep8 and use that. +#+ - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?)) +PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py" + +curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" +curl_status=$? + +if [ $curl_status -ne 0 ]; then + echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"." + exit $curl_status +fi + + +# There is no need to write this output to a file +#+ first, but we do so so that the check status can +#+ be output before the report, like with the +#+ scalastyle and RAT checks. +python $PEP8_SCRIPT_PATH ./python > "$PEP8_REPORT_PATH" +pep8_status=${PIPESTATUS[0]} #$? + +if [ $pep8_status -ne 0 ]; then + echo "PEP 8 checks failed." + cat "$PEP8_REPORT_PATH" +else + echo "PEP 8 checks passed." +fi + +rm -f "$PEP8_REPORT_PATH" +rm "$PEP8_SCRIPT_PATH" + +exit $pep8_status diff --git a/dev/lint-scala b/dev/lint-scala new file mode 100755 index 0000000000000..c676dfdf4f44e --- /dev/null +++ b/dev/lint-scala @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +# +# 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. +# + +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" +SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" + +"$SCRIPT_DIR/scalastyle" diff --git a/dev/run-tests b/dev/run-tests index d401c90f41d7b..0e24515d1376c 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -66,16 +66,25 @@ fi set -e set -o pipefail +echo "" echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" dev/check-license +echo "" echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" -dev/scalastyle +dev/lint-scala +echo "" +echo "=========================================================================" +echo "Running Python style checks" +echo "=========================================================================" +dev/lint-python + +echo "" echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" @@ -89,11 +98,13 @@ fi echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS clean package assembly/assembly test | \ grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +echo "" echo "=========================================================================" echo "Running PySpark tests" echo "=========================================================================" ./python/run-tests +echo "" echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" diff --git a/dev/scalastyle b/dev/scalastyle index d9f2b91a3a091..b53053a04ff42 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -30,5 +30,5 @@ if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" exit 1 else - echo -e "Scalastyle checks passed.\n" + echo -e "Scalastyle checks passed." fi diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 45d36e5d0e764..f133cf6f7befc 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -110,6 +110,7 @@ def _deserialize_accumulator(aid, zero_value, accum_param): class Accumulator(object): + """ A shared variable that can be accumulated, i.e., has a commutative and associative "add" operation. Worker tasks on a Spark cluster can add values to an Accumulator with the C{+=} @@ -166,6 +167,7 @@ def __repr__(self): class AccumulatorParam(object): + """ Helper object that defines how to accumulate values of a given type. """ @@ -186,6 +188,7 @@ def addInPlace(self, value1, value2): class AddingAccumulatorParam(AccumulatorParam): + """ An AccumulatorParam that uses the + operators to add values. Designed for simple types such as integers, floats, and lists. Requires the zero value for the underlying type @@ -210,6 +213,7 @@ def addInPlace(self, value1, value2): class _UpdateRequestHandler(SocketServer.StreamRequestHandler): + """ This handler will keep polling updates from the same socket until the server is shutdown. @@ -228,7 +232,9 @@ def handle(self): # Write a byte in acknowledgement self.wfile.write(struct.pack("!b", 1)) + class AccumulatorServer(SocketServer.TCPServer): + """ A simple TCP server that intercepts shutdown() in order to interrupt our continuous polling on the handler. @@ -239,6 +245,7 @@ def shutdown(self): self.server_shutdown = True SocketServer.TCPServer.shutdown(self) + def _start_update_server(): """Start a TCP server to receive accumulator updates in a daemon thread, and returns it""" server = AccumulatorServer(("localhost", 0), _UpdateRequestHandler) diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 43f40f8783bfd..f3e64989ed564 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -45,6 +45,7 @@ def _from_id(bid): class Broadcast(object): + """ A broadcast variable created with L{SparkContext.broadcast()}. diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index b4c82f519bd53..fb716f6753a45 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -56,6 +56,7 @@ class SparkConf(object): + """ Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2e80eb50f2207..4001ecab5ea00 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -47,6 +47,7 @@ class SparkContext(object): + """ Main entry point for Spark functionality. A SparkContext represents the connection to a Spark cluster, and can be used to create L{RDD}s and @@ -213,7 +214,7 @@ def _ensure_initialized(cls, instance=None, gateway=None): if instance: if (SparkContext._active_spark_context and - SparkContext._active_spark_context != instance): + SparkContext._active_spark_context != instance): currentMaster = SparkContext._active_spark_context.master currentAppName = SparkContext._active_spark_context.appName callsite = SparkContext._active_spark_context._callsite @@ -406,7 +407,7 @@ def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.sequenceFile(self._jsc, path, keyClass, valueClass, - keyConverter, valueConverter, minSplits, batchSize) + keyConverter, valueConverter, minSplits, batchSize) return RDD(jrdd, self, ser) def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, @@ -437,7 +438,8 @@ def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConv batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopFile(self._jsc, path, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf, batchSize) + valueClass, keyConverter, valueConverter, + jconf, batchSize) return RDD(jrdd, self, ser) def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, @@ -465,7 +467,8 @@ def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=N batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopRDD(self._jsc, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf, batchSize) + valueClass, keyConverter, valueConverter, + jconf, batchSize) return RDD(jrdd, self, ser) def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, @@ -496,7 +499,8 @@ def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter= batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.hadoopFile(self._jsc, path, inputFormatClass, keyClass, - valueClass, keyConverter, valueConverter, jconf, batchSize) + valueClass, keyConverter, valueConverter, + jconf, batchSize) return RDD(jrdd, self, ser) def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, @@ -523,8 +527,9 @@ def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, jconf = self._dictToJavaMap(conf) batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() - jrdd = self._jvm.PythonRDD.hadoopRDD(self._jsc, inputFormatClass, keyClass, valueClass, - keyConverter, valueConverter, jconf, batchSize) + jrdd = self._jvm.PythonRDD.hadoopRDD(self._jsc, inputFormatClass, keyClass, + valueClass, keyConverter, valueConverter, + jconf, batchSize) return RDD(jrdd, self, ser) def _checkpointFile(self, name, input_deserializer): @@ -555,8 +560,7 @@ def union(self, rdds): first = rdds[0]._jrdd rest = [x._jrdd for x in rdds[1:]] rest = ListConverter().convert(rest, self._gateway._gateway_client) - return RDD(self._jsc.union(first, rest), self, - rdds[0]._jrdd_deserializer) + return RDD(self._jsc.union(first, rest), self, rdds[0]._jrdd_deserializer) def broadcast(self, value): """ @@ -568,8 +572,7 @@ def broadcast(self, value): pickleSer = PickleSerializer() pickled = pickleSer.dumps(value) jbroadcast = self._jsc.broadcast(bytearray(pickled)) - return Broadcast(jbroadcast.id(), value, jbroadcast, - self._pickled_broadcast_vars) + return Broadcast(jbroadcast.id(), value, jbroadcast, self._pickled_broadcast_vars) def accumulator(self, value, accum_param=None): """ diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index b00da833d06f1..e73538baf0b93 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -43,7 +43,7 @@ def worker(sock): """ # Redirect stdout to stderr os.dup2(2, 1) - sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 + sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 signal.signal(SIGHUP, SIG_DFL) signal.signal(SIGCHLD, SIG_DFL) @@ -134,8 +134,7 @@ def handle_sigchld(*args): try: os.kill(worker_pid, signal.SIGKILL) except OSError: - pass # process already died - + pass # process already died if listen_sock in ready_fds: sock, addr = listen_sock.accept() diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 57ee14eeb7776..331de9a9b2212 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -19,6 +19,7 @@ class SparkFiles(object): + """ Resolves paths to files added through L{SparkContext.addFile()}. diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2c129679f47f3..37386ab0d7d49 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -65,6 +65,7 @@ def preexec_func(): # Create a thread to echo output from the GatewayServer, which is required # for Java log output to show up: class EchoOutputThread(Thread): + def __init__(self, stream): Thread.__init__(self) self.daemon = True diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 9c1565affbdac..db341da85f865 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -72,9 +72,9 @@ # Python interpreter must agree on what endian the machine is. -DENSE_VECTOR_MAGIC = 1 +DENSE_VECTOR_MAGIC = 1 SPARSE_VECTOR_MAGIC = 2 -DENSE_MATRIX_MAGIC = 3 +DENSE_MATRIX_MAGIC = 3 LABELED_POINT_MAGIC = 4 @@ -443,6 +443,7 @@ def _serialize_rating(r): class RatingDeserializer(Serializer): + def loads(self, stream): length = struct.unpack("!i", stream.read(4))[0] ba = stream.read(length) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 5ec1a8084d269..ffdda7ee19302 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -31,6 +31,7 @@ class LogisticRegressionModel(LinearModel): + """A linear binary classification model derived from logistic regression. >>> data = [ @@ -60,6 +61,7 @@ class LogisticRegressionModel(LinearModel): >>> lrm.predict(SparseVector(2, {1: 0.0})) <= 0 True """ + def predict(self, x): _linear_predictor_typecheck(x, self._coeff) margin = _dot(x, self._coeff) + self._intercept @@ -72,6 +74,7 @@ def predict(self, x): class LogisticRegressionWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, initialWeights=None, regParam=1.0, regType=None, intercept=False): @@ -108,6 +111,7 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, class SVMModel(LinearModel): + """A support vector machine. >>> data = [ @@ -131,6 +135,7 @@ class SVMModel(LinearModel): >>> svm.predict(SparseVector(2, {0: -1.0})) <= 0 True """ + def predict(self, x): _linear_predictor_typecheck(x, self._coeff) margin = _dot(x, self._coeff) + self._intercept @@ -138,6 +143,7 @@ def predict(self, x): class SVMWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None, regType=None, intercept=False): @@ -173,6 +179,7 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, class NaiveBayesModel(object): + """ Model for Naive Bayes classifiers. @@ -213,6 +220,7 @@ def predict(self, x): class NaiveBayes(object): + @classmethod def train(cls, data, lambda_=1.0): """ diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index b380e8f6c8725..a0630d1d5c58b 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -27,6 +27,7 @@ class KMeansModel(object): + """A clustering model derived from the k-means method. >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2) @@ -55,6 +56,7 @@ class KMeansModel(object): >>> type(model.clusterCenters) """ + def __init__(self, centers): self.centers = centers @@ -76,6 +78,7 @@ def predict(self, x): class KMeans(object): + @classmethod def train(cls, data, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 54720c2324ca6..9a239abfbbeb1 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -27,6 +27,7 @@ class SparseVector(object): + """ A simple sparse vector class for passing data to MLlib. Users may alternatively pass SciPy's {scipy.sparse} data types. @@ -192,6 +193,7 @@ def __ne__(self, other): class Vectors(object): + """ Factory methods for working with vectors. Note that dense vectors are simply represented as NumPy array objects, so there is no need diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 36e710dbae7a8..eb496688b6eef 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -24,7 +24,9 @@ from pyspark.mllib._common import _deserialize_double, _deserialize_double_vector from pyspark.serializers import NoOpSerializer + class RandomRDDGenerators: + """ Generator methods for creating RDDs comprised of i.i.d samples from some distribution. @@ -53,7 +55,7 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): True """ jrdd = sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) + uniform = RDD(jrdd, sc, NoOpSerializer()) return uniform.map(lambda bytes: _deserialize_double(bytearray(bytes))) @staticmethod @@ -77,7 +79,7 @@ def normalRDD(sc, size, numPartitions=None, seed=None): True """ jrdd = sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) + normal = RDD(jrdd, sc, NoOpSerializer()) return normal.map(lambda bytes: _deserialize_double(bytearray(bytes))) @staticmethod @@ -98,7 +100,7 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): True """ jrdd = sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) + poisson = RDD(jrdd, sc, NoOpSerializer()) return poisson.map(lambda bytes: _deserialize_double(bytearray(bytes))) @staticmethod @@ -118,7 +120,7 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ jrdd = sc._jvm.PythonMLLibAPI() \ .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) + uniform = RDD(jrdd, sc, NoOpSerializer()) return uniform.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod @@ -138,7 +140,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ jrdd = sc._jvm.PythonMLLibAPI() \ .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) + normal = RDD(jrdd, sc, NoOpSerializer()) return normal.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod @@ -161,7 +163,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ jrdd = sc._jvm.PythonMLLibAPI() \ .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) + poisson = RDD(jrdd, sc, NoOpSerializer()) return poisson.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 6c385042ffa5f..e863fc249ec36 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -26,6 +26,7 @@ class MatrixFactorizationModel(object): + """A matrix factorisation model trained by regularized alternating least-squares. @@ -58,6 +59,7 @@ def predictAll(self, usersProducts): class ALS(object): + @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): sc = ratings.context diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 041b119269427..d8792cf44872f 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -27,6 +27,7 @@ class LabeledPoint(object): + """ The features and labels of a data point. @@ -34,6 +35,7 @@ class LabeledPoint(object): @param features: Vector of features for this point (NumPy array, list, pyspark.mllib.linalg.SparseVector, or scipy.sparse column matrix) """ + def __init__(self, label, features): self.label = label if (type(features) == ndarray or type(features) == SparseVector @@ -49,7 +51,9 @@ def __str__(self): class LinearModel(object): + """A linear model that has a vector of coefficients and an intercept.""" + def __init__(self, weights, intercept): self._coeff = weights self._intercept = intercept @@ -64,6 +68,7 @@ def intercept(self): class LinearRegressionModelBase(LinearModel): + """A linear regression model. >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1) @@ -72,6 +77,7 @@ class LinearRegressionModelBase(LinearModel): >>> abs(lrmb.predict(SparseVector(2, {0: -1.03, 1: 7.777})) - 14.624) < 1e-6 True """ + def predict(self, x): """Predict the value of the dependent variable given a vector x""" """containing values for the independent variables.""" @@ -80,6 +86,7 @@ def predict(self, x): class LinearRegressionModel(LinearRegressionModelBase): + """A linear regression model derived from a least-squares fit. >>> from pyspark.mllib.regression import LabeledPoint @@ -111,6 +118,7 @@ class LinearRegressionModel(LinearRegressionModelBase): class LinearRegressionWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, initialWeights=None, regParam=1.0, regType=None, intercept=False): @@ -146,6 +154,7 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, class LassoModel(LinearRegressionModelBase): + """A linear regression model derived from a least-squares fit with an l_1 penalty term. @@ -178,6 +187,7 @@ class LassoModel(LinearRegressionModelBase): class LassoWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): @@ -189,6 +199,7 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, class RidgeRegressionModel(LinearRegressionModelBase): + """A linear regression model derived from a least-squares fit with an l_2 penalty term. @@ -221,6 +232,7 @@ class RidgeRegressionModel(LinearRegressionModelBase): class RidgeRegressionWithSGD(object): + @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 0a08a562d1f1f..982906b9d09f0 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -24,6 +24,7 @@ _serialize_double, _serialize_double_vector, \ _deserialize_double, _deserialize_double_matrix + class Statistics(object): @staticmethod diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 9d1e5be637a9a..6f3ec8ac94bac 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -39,6 +39,7 @@ class VectorTests(unittest.TestCase): + def test_serialize(self): sv = SparseVector(4, {1: 1, 3: 2}) dv = array([1., 2., 3., 4.]) @@ -81,6 +82,7 @@ def test_squared_distance(self): class ListTests(PySparkTestCase): + """ Test MLlib algorithms on plain lists, to make sure they're passed through as NumPy arrays. @@ -128,7 +130,7 @@ def test_classification(self): self.assertTrue(nb_model.predict(features[2]) <= 0) self.assertTrue(nb_model.predict(features[3]) > 0) - categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories + categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories dt_model = \ DecisionTree.trainClassifier(rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo) @@ -168,7 +170,7 @@ def test_regression(self): self.assertTrue(rr_model.predict(features[2]) <= 0) self.assertTrue(rr_model.predict(features[3]) > 0) - categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories + categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories dt_model = \ DecisionTree.trainRegressor(rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) self.assertTrue(dt_model.predict(features[0]) <= 0) @@ -179,6 +181,7 @@ def test_regression(self): @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): + """ Test both vector operations and MLlib algorithms with SciPy sparse matrices, if SciPy is available. @@ -276,7 +279,7 @@ def test_classification(self): self.assertTrue(nb_model.predict(features[2]) <= 0) self.assertTrue(nb_model.predict(features[3]) > 0) - categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories + categoricalFeaturesInfo = {0: 3} # feature 0 has 3 categories dt_model = DecisionTree.trainClassifier(rdd, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo) self.assertTrue(dt_model.predict(features[0]) <= 0) @@ -315,7 +318,7 @@ def test_regression(self): self.assertTrue(rr_model.predict(features[2]) <= 0) self.assertTrue(rr_model.predict(features[3]) > 0) - categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories + categoricalFeaturesInfo = {0: 2} # feature 0 has 2 categories dt_model = DecisionTree.trainRegressor(rdd, categoricalFeaturesInfo=categoricalFeaturesInfo) self.assertTrue(dt_model.predict(features[0]) <= 0) self.assertTrue(dt_model.predict(features[1]) > 0) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 1e0006df75ac6..2518001ea0b93 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -25,7 +25,9 @@ from pyspark.mllib.regression import LabeledPoint from pyspark.serializers import NoOpSerializer + class DecisionTreeModel(object): + """ A decision tree model for classification or regression. @@ -77,6 +79,7 @@ def __str__(self): class DecisionTree(object): + """ Learning algorithm for a decision tree model for classification or regression. @@ -174,7 +177,6 @@ def trainRegressor(data, categoricalFeaturesInfo={}, categoricalFeaturesInfo, impurity, maxDepth, maxBins) - @staticmethod def train(data, algo, numClasses, categoricalFeaturesInfo, impurity, maxDepth, maxBins=100): diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 639cda6350229..4962d05491c03 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -26,6 +26,7 @@ class MLUtils: + """ Helper methods to load, save and pre-process data used in MLlib. """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 309f5a9b6038d..30b834d2085cd 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -233,7 +233,7 @@ def __init__(self, jrdd, ctx, jrdd_deserializer): def _toPickleSerialization(self): if (self._jrdd_deserializer == PickleSerializer() or - self._jrdd_deserializer == BatchedSerializer(PickleSerializer())): + self._jrdd_deserializer == BatchedSerializer(PickleSerializer())): return self else: return self._reserialize(BatchedSerializer(PickleSerializer(), 10)) @@ -1079,7 +1079,9 @@ def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueCl pickledRDD = self._toPickleSerialization() batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) self.ctx._jvm.PythonRDD.saveAsNewAPIHadoopFile(pickledRDD._jrdd, batched, path, - outputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf) + outputFormatClass, + keyClass, valueClass, + keyConverter, valueConverter, jconf) def saveAsHadoopDataset(self, conf, keyConverter=None, valueConverter=None): """ @@ -1125,8 +1127,10 @@ def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=No pickledRDD = self._toPickleSerialization() batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) self.ctx._jvm.PythonRDD.saveAsHadoopFile(pickledRDD._jrdd, batched, path, - outputFormatClass, keyClass, valueClass, keyConverter, valueConverter, - jconf, compressionCodecClass) + outputFormatClass, + keyClass, valueClass, + keyConverter, valueConverter, + jconf, compressionCodecClass) def saveAsSequenceFile(self, path, compressionCodecClass=None): """ @@ -1348,7 +1352,7 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash): outputSerializer = self.ctx._unbatched_serializer limit = (_parse_memory(self.ctx._conf.get( - "spark.python.worker.memory", "512m")) / 2) + "spark.python.worker.memory", "512m")) / 2) def add_shuffle_key(split, iterator): @@ -1430,12 +1434,12 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == 'true') memory = _parse_memory(self.ctx._conf.get( - "spark.python.worker.memory", "512m")) + "spark.python.worker.memory", "512m")) agg = Aggregator(createCombiner, mergeValue, mergeCombiners) def combineLocally(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ - if spill else InMemoryMerger(agg) + if spill else InMemoryMerger(agg) merger.mergeValues(iterator) return merger.iteritems() @@ -1444,7 +1448,7 @@ def combineLocally(iterator): def _mergeCombiners(iterator): merger = ExternalMerger(agg, memory, serializer) \ - if spill else InMemoryMerger(agg) + if spill else InMemoryMerger(agg) merger.mergeCombiners(iterator) return merger.iteritems() @@ -1588,7 +1592,7 @@ def sampleByKey(self, withReplacement, fractions, seed=None): """ for fraction in fractions.values(): assert fraction >= 0.0, "Negative fraction value: %s" % fraction - return self.mapPartitionsWithIndex( \ + return self.mapPartitionsWithIndex( RDDStratifiedSampler(withReplacement, fractions, seed).func, True) def subtractByKey(self, other, numPartitions=None): diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 2df000fdb08ca..55e247da0e4dc 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -20,6 +20,7 @@ class RDDSamplerBase(object): + def __init__(self, withReplacement, seed=None): try: import numpy @@ -95,6 +96,7 @@ def shuffle(self, vals): class RDDSampler(RDDSamplerBase): + def __init__(self, withReplacement, fraction, seed=None): RDDSamplerBase.__init__(self, withReplacement, seed) self._fraction = fraction @@ -113,7 +115,9 @@ def func(self, split, iterator): if self.getUniformSample(split) <= self._fraction: yield obj + class RDDStratifiedSampler(RDDSamplerBase): + def __init__(self, withReplacement, fractions, seed=None): RDDSamplerBase.__init__(self, withReplacement, seed) self._fractions = fractions diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index df34740fc8176..ef04c82866e6c 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -21,9 +21,11 @@ class ResultIterable(collections.Iterable): + """ A special result iterable. This is used because the standard iterator can not be pickled """ + def __init__(self, data): self.data = data self.index = 0 diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index a10f85b55ad30..b35558db3e007 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -111,6 +111,7 @@ def __ne__(self, other): class FramedSerializer(Serializer): + """ Serializer that writes objects as a stream of (length, data) pairs, where C{length} is a 32-bit integer and data is C{length} bytes. @@ -162,6 +163,7 @@ def loads(self, obj): class BatchedSerializer(Serializer): + """ Serializes a stream of objects in batches by calling its wrapped Serializer with streams of objects. @@ -207,6 +209,7 @@ def __str__(self): class CartesianDeserializer(FramedSerializer): + """ Deserializes the JavaRDD cartesian() of two PythonRDDs. """ @@ -240,6 +243,7 @@ def __str__(self): class PairDeserializer(CartesianDeserializer): + """ Deserializes the JavaRDD zip() of two PythonRDDs. """ @@ -289,6 +293,7 @@ def _hack_namedtuple(cls): """ Make class generated by namedtuple picklable """ name = cls.__name__ fields = cls._fields + def __reduce__(self): return (_restore, (name, fields, tuple(self))) cls.__reduce__ = __reduce__ @@ -301,10 +306,11 @@ def _hijack_namedtuple(): if hasattr(collections.namedtuple, "__hijack"): return - global _old_namedtuple # or it will put in closure + global _old_namedtuple # or it will put in closure + def _copy_func(f): return types.FunctionType(f.func_code, f.func_globals, f.func_name, - f.func_defaults, f.func_closure) + f.func_defaults, f.func_closure) _old_namedtuple = _copy_func(collections.namedtuple) @@ -323,15 +329,16 @@ def namedtuple(name, fields, verbose=False, rename=False): # so only hack those in __main__ module for n, o in sys.modules["__main__"].__dict__.iteritems(): if (type(o) is type and o.__base__ is tuple - and hasattr(o, "_fields") - and "__reduce__" not in o.__dict__): - _hack_namedtuple(o) # hack inplace + and hasattr(o, "_fields") + and "__reduce__" not in o.__dict__): + _hack_namedtuple(o) # hack inplace _hijack_namedtuple() class PickleSerializer(FramedSerializer): + """ Serializes objects using Python's cPickle serializer: @@ -354,6 +361,7 @@ def dumps(self, obj): class MarshalSerializer(FramedSerializer): + """ Serializes objects using Python's Marshal serializer: @@ -367,9 +375,11 @@ class MarshalSerializer(FramedSerializer): class AutoSerializer(FramedSerializer): + """ Choose marshal or cPickle as serialization protocol autumatically """ + def __init__(self): FramedSerializer.__init__(self) self._type = None @@ -394,6 +404,7 @@ def loads(self, obj): class UTF8Deserializer(Serializer): + """ Deserializes streams written by String.getBytes. """ diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index e3923d1c36c57..2c68cd4921deb 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -45,7 +45,7 @@ def get_used_memory(): return int(line.split()[1]) >> 10 else: warnings.warn("Please install psutil to have better " - "support with spilling") + "support with spilling") if platform.system() == "Darwin": import resource rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss @@ -141,7 +141,7 @@ class ExternalMerger(Merger): This class works as follows: - - It repeatedly combine the items and save them in one dict in + - It repeatedly combine the items and save them in one dict in memory. - When the used memory goes above memory limit, it will split @@ -190,12 +190,12 @@ class ExternalMerger(Merger): MAX_TOTAL_PARTITIONS = 4096 def __init__(self, aggregator, memory_limit=512, serializer=None, - localdirs=None, scale=1, partitions=59, batch=1000): + localdirs=None, scale=1, partitions=59, batch=1000): Merger.__init__(self, aggregator) self.memory_limit = memory_limit # default serializer is only used for tests self.serializer = serializer or \ - BatchedSerializer(PickleSerializer(), 1024) + BatchedSerializer(PickleSerializer(), 1024) self.localdirs = localdirs or self._get_dirs() # number of partitions when spill data into disks self.partitions = partitions @@ -341,7 +341,7 @@ def _spill(self): self.pdata[i].clear() self.spills += 1 - gc.collect() # release the memory as much as possible + gc.collect() # release the memory as much as possible def iteritems(self): """ Return all merged items as iterator """ @@ -370,8 +370,8 @@ def _external_items(self): if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS and j < self.spills - 1 and get_used_memory() > hard_limit): - self.data.clear() # will read from disk again - gc.collect() # release the memory as much as possible + self.data.clear() # will read from disk again + gc.collect() # release the memory as much as possible for v in self._recursive_merged_items(i): yield v return @@ -409,9 +409,9 @@ def _recursive_merged_items(self, start): for i in range(start, self.partitions): subdirs = [os.path.join(d, "parts", str(i)) - for d in self.localdirs] + for d in self.localdirs] m = ExternalMerger(self.agg, self.memory_limit, self.serializer, - subdirs, self.scale * self.partitions) + subdirs, self.scale * self.partitions) m.pdata = [{} for _ in range(self.partitions)] limit = self._next_limit() @@ -419,7 +419,7 @@ def _recursive_merged_items(self, start): path = self._get_spill_dir(j) p = os.path.join(path, str(i)) m._partitioned_mergeCombiners( - self.serializer.load_stream(open(p))) + self.serializer.load_stream(open(p))) if get_used_memory() > limit: m._spill() diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index adc56e7ec0e2b..950e275adbf01 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -45,6 +45,7 @@ class DataType(object): + """Spark SQL DataType""" def __repr__(self): @@ -62,6 +63,7 @@ def __ne__(self, other): class PrimitiveTypeSingleton(type): + """Metaclass for PrimitiveType""" _instances = {} @@ -73,6 +75,7 @@ def __call__(cls): class PrimitiveType(DataType): + """Spark SQL PrimitiveType""" __metaclass__ = PrimitiveTypeSingleton @@ -83,6 +86,7 @@ def __eq__(self, other): class StringType(PrimitiveType): + """Spark SQL StringType The data type representing string values. @@ -90,6 +94,7 @@ class StringType(PrimitiveType): class BinaryType(PrimitiveType): + """Spark SQL BinaryType The data type representing bytearray values. @@ -97,6 +102,7 @@ class BinaryType(PrimitiveType): class BooleanType(PrimitiveType): + """Spark SQL BooleanType The data type representing bool values. @@ -104,6 +110,7 @@ class BooleanType(PrimitiveType): class TimestampType(PrimitiveType): + """Spark SQL TimestampType The data type representing datetime.datetime values. @@ -111,6 +118,7 @@ class TimestampType(PrimitiveType): class DecimalType(PrimitiveType): + """Spark SQL DecimalType The data type representing decimal.Decimal values. @@ -118,6 +126,7 @@ class DecimalType(PrimitiveType): class DoubleType(PrimitiveType): + """Spark SQL DoubleType The data type representing float values. @@ -125,6 +134,7 @@ class DoubleType(PrimitiveType): class FloatType(PrimitiveType): + """Spark SQL FloatType The data type representing single precision floating-point values. @@ -132,6 +142,7 @@ class FloatType(PrimitiveType): class ByteType(PrimitiveType): + """Spark SQL ByteType The data type representing int values with 1 singed byte. @@ -139,6 +150,7 @@ class ByteType(PrimitiveType): class IntegerType(PrimitiveType): + """Spark SQL IntegerType The data type representing int values. @@ -146,6 +158,7 @@ class IntegerType(PrimitiveType): class LongType(PrimitiveType): + """Spark SQL LongType The data type representing long values. If the any value is @@ -155,6 +168,7 @@ class LongType(PrimitiveType): class ShortType(PrimitiveType): + """Spark SQL ShortType The data type representing int values with 2 signed bytes. @@ -162,6 +176,7 @@ class ShortType(PrimitiveType): class ArrayType(DataType): + """Spark SQL ArrayType The data type representing list values. An ArrayType object @@ -187,10 +202,11 @@ def __init__(self, elementType, containsNull=False): def __str__(self): return "ArrayType(%s,%s)" % (self.elementType, - str(self.containsNull).lower()) + str(self.containsNull).lower()) class MapType(DataType): + """Spark SQL MapType The data type representing dict values. A MapType object comprises @@ -226,10 +242,11 @@ def __init__(self, keyType, valueType, valueContainsNull=True): def __repr__(self): return "MapType(%s,%s,%s)" % (self.keyType, self.valueType, - str(self.valueContainsNull).lower()) + str(self.valueContainsNull).lower()) class StructField(DataType): + """Spark SQL StructField Represents a field in a StructType. @@ -263,10 +280,11 @@ def __init__(self, name, dataType, nullable): def __repr__(self): return "StructField(%s,%s,%s)" % (self.name, self.dataType, - str(self.nullable).lower()) + str(self.nullable).lower()) class StructType(DataType): + """Spark SQL StructType The data type representing rows. @@ -291,7 +309,7 @@ def __init__(self, fields): def __repr__(self): return ("StructType(List(%s))" % - ",".join(str(field) for field in self.fields)) + ",".join(str(field) for field in self.fields)) def _parse_datatype_list(datatype_list_string): @@ -319,7 +337,7 @@ def _parse_datatype_list(datatype_list_string): _all_primitive_types = dict((k, v) for k, v in globals().iteritems() - if type(v) is PrimitiveTypeSingleton and v.__base__ == PrimitiveType) + if type(v) is PrimitiveTypeSingleton and v.__base__ == PrimitiveType) def _parse_datatype_string(datatype_string): @@ -459,16 +477,16 @@ def _infer_schema(row): items = sorted(row.items()) elif isinstance(row, tuple): - if hasattr(row, "_fields"): # namedtuple + if hasattr(row, "_fields"): # namedtuple items = zip(row._fields, tuple(row)) - elif hasattr(row, "__FIELDS__"): # Row + elif hasattr(row, "__FIELDS__"): # Row items = zip(row.__FIELDS__, tuple(row)) elif all(isinstance(x, tuple) and len(x) == 2 for x in row): items = row else: raise ValueError("Can't infer schema from tuple") - elif hasattr(row, "__dict__"): # object + elif hasattr(row, "__dict__"): # object items = sorted(row.__dict__.items()) else: @@ -499,7 +517,7 @@ def _create_converter(obj, dataType): conv = lambda o: tuple(o.get(n) for n in names) elif isinstance(obj, tuple): - if hasattr(obj, "_fields"): # namedtuple + if hasattr(obj, "_fields"): # namedtuple conv = tuple elif hasattr(obj, "__FIELDS__"): conv = tuple @@ -508,7 +526,7 @@ def _create_converter(obj, dataType): else: raise ValueError("unexpected tuple") - elif hasattr(obj, "__dict__"): # object + elif hasattr(obj, "__dict__"): # object conv = lambda o: [o.__dict__.get(n, None) for n in names] nested = any(_has_struct(f.dataType) for f in dataType.fields) @@ -660,7 +678,7 @@ def _infer_schema_type(obj, dataType): assert len(fs) == len(obj), \ "Obj(%s) have different length with fields(%s)" % (obj, fs) fields = [StructField(f.name, _infer_schema_type(o, f.dataType), True) - for o, f in zip(obj, fs)] + for o, f in zip(obj, fs)] return StructType(fields) else: @@ -683,6 +701,7 @@ def _infer_schema_type(obj, dataType): StructType: (tuple, list), } + def _verify_type(obj, dataType): """ Verify the type of obj against dataType, raise an exception if @@ -728,7 +747,7 @@ def _verify_type(obj, dataType): elif isinstance(dataType, StructType): if len(obj) != len(dataType.fields): raise ValueError("Length of object (%d) does not match with" - "length of fields (%d)" % (len(obj), len(dataType.fields))) + "length of fields (%d)" % (len(obj), len(dataType.fields))) for v, f in zip(obj, dataType.fields): _verify_type(v, f.dataType) @@ -861,6 +880,7 @@ def __reduce__(self): raise Exception("unexpected data type: %s" % dataType) class Row(tuple): + """ Row in SchemaRDD """ __DATATYPE__ = dataType __FIELDS__ = tuple(f.name for f in dataType.fields) @@ -872,7 +892,7 @@ class Row(tuple): def __repr__(self): # call collect __repr__ for nested objects return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) - for n in self.__FIELDS__)) + for n in self.__FIELDS__)) def __reduce__(self): return (_restore_object, (self.__DATATYPE__, tuple(self))) @@ -881,6 +901,7 @@ def __reduce__(self): class SQLContext: + """Main entry point for SparkSQL functionality. A SQLContext can be used create L{SchemaRDD}s, register L{SchemaRDD}s as @@ -960,7 +981,7 @@ def registerFunction(self, name, f, returnType=StringType()): env = MapConverter().convert(self._sc.environment, self._sc._gateway._gateway_client) includes = ListConverter().convert(self._sc._python_includes, - self._sc._gateway._gateway_client) + self._sc._gateway._gateway_client) self._ssql_ctx.registerPython(name, bytearray(CloudPickleSerializer().dumps(command)), env, @@ -1012,7 +1033,7 @@ def inferSchema(self, rdd): first = rdd.first() if not first: raise ValueError("The first row in RDD is empty, " - "can not infer schema") + "can not infer schema") if type(first) is dict: warnings.warn("Using RDD of dict to inferSchema is deprecated") @@ -1287,6 +1308,7 @@ def uncacheTable(self, tableName): class HiveContext(SQLContext): + """A variant of Spark SQL that integrates with data stored in Hive. Configuration for Hive is read from hive-site.xml on the classpath. @@ -1327,6 +1349,7 @@ def hql(self, hqlQuery): class LocalHiveContext(HiveContext): + """Starts up an instance of hive where metadata is stored locally. An in-process metadata data is created with data stored in ./metadata. @@ -1357,7 +1380,7 @@ class LocalHiveContext(HiveContext): def __init__(self, sparkContext, sqlContext=None): HiveContext.__init__(self, sparkContext, sqlContext) warnings.warn("LocalHiveContext is deprecated. " - "Use HiveContext instead.", DeprecationWarning) + "Use HiveContext instead.", DeprecationWarning) def _get_hive_ctx(self): return self._jvm.LocalHiveContext(self._jsc.sc()) @@ -1376,6 +1399,7 @@ def _create_row(fields, values): class Row(tuple): + """ A row in L{SchemaRDD}. The fields in it can be accessed like attributes. @@ -1417,7 +1441,6 @@ def __new__(self, *args, **kwargs): else: raise ValueError("No args or kwargs") - # let obect acs like class def __call__(self, *args): """create new Row object""" @@ -1443,12 +1466,13 @@ def __reduce__(self): def __repr__(self): if hasattr(self, "__FIELDS__"): return "Row(%s)" % ", ".join("%s=%r" % (k, v) - for k, v in zip(self.__FIELDS__, self)) + for k, v in zip(self.__FIELDS__, self)) else: return "" % ", ".join(self) class SchemaRDD(RDD): + """An RDD of L{Row} objects that has an associated schema. The underlying JVM object is a SchemaRDD, not a PythonRDD, so we can @@ -1659,7 +1683,7 @@ def subtract(self, other, numPartitions=None): rdd = self._jschema_rdd.subtract(other._jschema_rdd) else: rdd = self._jschema_rdd.subtract(other._jschema_rdd, - numPartitions) + numPartitions) return SchemaRDD(rdd, self.sql_ctx) else: raise ValueError("Can only subtract another SchemaRDD") @@ -1686,9 +1710,9 @@ def _test(): jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' - '"field6":[{"field7": "row2"}]}', + '"field6":[{"field7": "row2"}]}', '{"field1" : null, "field2": "row3", ' - '"field3":{"field4":33, "field5": []}}' + '"field3":{"field4":33, "field5": []}}' ] globs['jsonStrings'] = jsonStrings globs['json'] = sc.parallelize(jsonStrings) diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 5d77a131f2856..2aa0fb9d2c1ed 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -19,6 +19,7 @@ class StorageLevel: + """ Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 4ac94ba729d35..88a61176e51ab 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -62,53 +62,53 @@ def setUp(self): self.N = 1 << 16 self.l = [i for i in xrange(self.N)] self.data = zip(self.l, self.l) - self.agg = Aggregator(lambda x: [x], - lambda x, y: x.append(y) or x, - lambda x, y: x.extend(y) or x) + self.agg = Aggregator(lambda x: [x], + lambda x, y: x.append(y) or x, + lambda x, y: x.extend(y) or x) def test_in_memory(self): m = InMemoryMerger(self.agg) m.mergeValues(self.data) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), - sum(xrange(self.N))) + sum(xrange(self.N))) m = InMemoryMerger(self.agg) m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), - sum(xrange(self.N))) + sum(xrange(self.N))) def test_small_dataset(self): m = ExternalMerger(self.agg, 1000) m.mergeValues(self.data) self.assertEqual(m.spills, 0) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), - sum(xrange(self.N))) + sum(xrange(self.N))) m = ExternalMerger(self.agg, 1000) m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) self.assertEqual(m.spills, 0) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), - sum(xrange(self.N))) + sum(xrange(self.N))) def test_medium_dataset(self): m = ExternalMerger(self.agg, 10) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), - sum(xrange(self.N))) + sum(xrange(self.N))) m = ExternalMerger(self.agg, 10) m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3)) self.assertTrue(m.spills >= 1) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), - sum(xrange(self.N)) * 3) + sum(xrange(self.N)) * 3) def test_huge_dataset(self): m = ExternalMerger(self.agg, 10) m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) self.assertTrue(m.spills >= 1) self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), - self.N * 10) + self.N * 10) m._cleanup() @@ -188,6 +188,7 @@ def test_add_py_file(self): log4j = self.sc._jvm.org.apache.log4j old_level = log4j.LogManager.getRootLogger().getLevel() log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) + def func(x): from userlibrary import UserClass return UserClass().hello() @@ -355,8 +356,8 @@ def test_sequencefiles(self): self.assertEqual(doubles, ed) bytes = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfbytes/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.BytesWritable").collect()) + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.BytesWritable").collect()) ebs = [(1, bytearray('aa', 'utf-8')), (1, bytearray('aa', 'utf-8')), (2, bytearray('aa', 'utf-8')), @@ -428,9 +429,9 @@ def test_sequencefiles(self): self.assertEqual(clazz[0], ec) unbatched_clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", - "org.apache.hadoop.io.Text", - "org.apache.spark.api.python.TestWritable", - batchSize=1).collect()) + "org.apache.hadoop.io.Text", + "org.apache.spark.api.python.TestWritable", + batchSize=1).collect()) self.assertEqual(unbatched_clazz[0], ec) def test_oldhadoop(self): @@ -443,7 +444,7 @@ def test_oldhadoop(self): self.assertEqual(ints, ei) hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt") - oldconf = {"mapred.input.dir" : hellopath} + oldconf = {"mapred.input.dir": hellopath} hello = self.sc.hadoopRDD("org.apache.hadoop.mapred.TextInputFormat", "org.apache.hadoop.io.LongWritable", "org.apache.hadoop.io.Text", @@ -462,7 +463,7 @@ def test_newhadoop(self): self.assertEqual(ints, ei) hellopath = os.path.join(SPARK_HOME, "python/test_support/hello.txt") - newconf = {"mapred.input.dir" : hellopath} + newconf = {"mapred.input.dir": hellopath} hello = self.sc.newAPIHadoopRDD("org.apache.hadoop.mapreduce.lib.input.TextInputFormat", "org.apache.hadoop.io.LongWritable", "org.apache.hadoop.io.Text", @@ -517,6 +518,7 @@ def test_converters(self): (u'\x03', [2.0])] self.assertEqual(maps, em) + class TestOutputFormat(PySparkTestCase): def setUp(self): @@ -574,8 +576,8 @@ def test_sequencefiles(self): def test_oldhadoop(self): basepath = self.tempdir.name dict_data = [(1, {}), - (1, {"row1" : 1.0}), - (2, {"row2" : 2.0})] + (1, {"row1": 1.0}), + (2, {"row2": 2.0})] self.sc.parallelize(dict_data).saveAsHadoopFile( basepath + "/oldhadoop/", "org.apache.hadoop.mapred.SequenceFileOutputFormat", @@ -589,12 +591,13 @@ def test_oldhadoop(self): self.assertEqual(result, dict_data) conf = { - "mapred.output.format.class" : "org.apache.hadoop.mapred.SequenceFileOutputFormat", - "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", - "mapred.output.value.class" : "org.apache.hadoop.io.MapWritable", - "mapred.output.dir" : basepath + "/olddataset/"} + "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.MapWritable", + "mapred.output.dir": basepath + "/olddataset/" + } self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) - input_conf = {"mapred.input.dir" : basepath + "/olddataset/"} + input_conf = {"mapred.input.dir": basepath + "/olddataset/"} old_dataset = sorted(self.sc.hadoopRDD( "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", @@ -622,14 +625,17 @@ def test_newhadoop(self): valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect()) self.assertEqual(result, array_data) - conf = {"mapreduce.outputformat.class" : - "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", - "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", - "mapred.output.value.class" : "org.apache.spark.api.python.DoubleArrayWritable", - "mapred.output.dir" : basepath + "/newdataset/"} - self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(conf, + conf = { + "mapreduce.outputformat.class": + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable", + "mapred.output.dir": basepath + "/newdataset/" + } + self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset( + conf, valueConverter="org.apache.spark.api.python.DoubleArrayToWritableConverter") - input_conf = {"mapred.input.dir" : basepath + "/newdataset/"} + input_conf = {"mapred.input.dir": basepath + "/newdataset/"} new_dataset = sorted(self.sc.newAPIHadoopRDD( "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", @@ -640,7 +646,7 @@ def test_newhadoop(self): def test_newolderror(self): basepath = self.tempdir.name - rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x )) + rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( basepath + "/newolderror/saveAsHadoopFile/", "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")) @@ -650,7 +656,7 @@ def test_newolderror(self): def test_bad_inputs(self): basepath = self.tempdir.name - rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x )) + rdd = self.sc.parallelize(range(1, 4)).map(lambda x: (x, "a" * x)) self.assertRaises(Exception, lambda: rdd.saveAsHadoopFile( basepath + "/badinputs/saveAsHadoopFile/", "org.apache.hadoop.mapred.NotValidOutputFormat")) @@ -685,30 +691,32 @@ def test_reserialization(self): result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) self.assertEqual(result1, data) - rdd.saveAsHadoopFile(basepath + "/reserialize/hadoop", - "org.apache.hadoop.mapred.SequenceFileOutputFormat") + rdd.saveAsHadoopFile( + basepath + "/reserialize/hadoop", + "org.apache.hadoop.mapred.SequenceFileOutputFormat") result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect()) self.assertEqual(result2, data) - rdd.saveAsNewAPIHadoopFile(basepath + "/reserialize/newhadoop", - "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") + rdd.saveAsNewAPIHadoopFile( + basepath + "/reserialize/newhadoop", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") result3 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newhadoop").collect()) self.assertEqual(result3, data) conf4 = { - "mapred.output.format.class" : "org.apache.hadoop.mapred.SequenceFileOutputFormat", - "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", - "mapred.output.value.class" : "org.apache.hadoop.io.IntWritable", - "mapred.output.dir" : basepath + "/reserialize/dataset"} + "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.dir": basepath + "/reserialize/dataset"} rdd.saveAsHadoopDataset(conf4) result4 = sorted(self.sc.sequenceFile(basepath + "/reserialize/dataset").collect()) self.assertEqual(result4, data) - conf5 = {"mapreduce.outputformat.class" : - "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", - "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable", - "mapred.output.value.class" : "org.apache.hadoop.io.IntWritable", - "mapred.output.dir" : basepath + "/reserialize/newdataset"} + conf5 = {"mapreduce.outputformat.class": + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.dir": basepath + "/reserialize/newdataset"} rdd.saveAsNewAPIHadoopDataset(conf5) result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect()) self.assertEqual(result5, data) @@ -719,25 +727,28 @@ def test_unbatched_save_and_read(self): self.sc.parallelize(ei, numSlices=len(ei)).saveAsSequenceFile( basepath + "/unbatched/") - unbatched_sequence = sorted(self.sc.sequenceFile(basepath + "/unbatched/", + unbatched_sequence = sorted(self.sc.sequenceFile( + basepath + "/unbatched/", batchSize=1).collect()) self.assertEqual(unbatched_sequence, ei) - unbatched_hadoopFile = sorted(self.sc.hadoopFile(basepath + "/unbatched/", + unbatched_hadoopFile = sorted(self.sc.hadoopFile( + basepath + "/unbatched/", "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.Text", batchSize=1).collect()) self.assertEqual(unbatched_hadoopFile, ei) - unbatched_newAPIHadoopFile = sorted(self.sc.newAPIHadoopFile(basepath + "/unbatched/", + unbatched_newAPIHadoopFile = sorted(self.sc.newAPIHadoopFile( + basepath + "/unbatched/", "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.Text", batchSize=1).collect()) self.assertEqual(unbatched_newAPIHadoopFile, ei) - oldconf = {"mapred.input.dir" : basepath + "/unbatched/"} + oldconf = {"mapred.input.dir": basepath + "/unbatched/"} unbatched_hadoopRDD = sorted(self.sc.hadoopRDD( "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", @@ -746,7 +757,7 @@ def test_unbatched_save_and_read(self): batchSize=1).collect()) self.assertEqual(unbatched_hadoopRDD, ei) - newconf = {"mapred.input.dir" : basepath + "/unbatched/"} + newconf = {"mapred.input.dir": basepath + "/unbatched/"} unbatched_newAPIHadoopRDD = sorted(self.sc.newAPIHadoopRDD( "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", @@ -763,7 +774,9 @@ def test_malformed_RDD(self): self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile( basepath + "/malformed/sequence")) + class TestDaemon(unittest.TestCase): + def connect(self, port): from socket import socket, AF_INET, SOCK_STREAM sock = socket(AF_INET, SOCK_STREAM) @@ -810,12 +823,15 @@ def test_termination_sigterm(self): class TestWorker(PySparkTestCase): + def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) temp.close() path = temp.name + def sleep(x): - import os, time + import os + import time with open(path, 'w') as f: f.write("%d %d" % (os.getppid(), os.getpid())) time.sleep(100) @@ -845,7 +861,7 @@ def run(): os.kill(worker_pid, 0) time.sleep(0.1) except OSError: - break # worker was killed + break # worker was killed else: self.fail("worker has not been killed after 5 seconds") @@ -855,12 +871,13 @@ def run(): self.fail("daemon had been killed") def test_fd_leak(self): - N = 1100 # fd limit is 1024 by default + N = 1100 # fd limit is 1024 by default rdd = self.sc.parallelize(range(N), N) self.assertEquals(N, rdd.count()) class TestSparkSubmit(unittest.TestCase): + def setUp(self): self.programDir = tempfile.mkdtemp() self.sparkSubmit = os.path.join(os.environ.get("SPARK_HOME"), "bin", "spark-submit") @@ -953,9 +970,9 @@ def test_module_dependency_on_cluster(self): |def myfunc(x): | return x + 1 """) - proc = subprocess.Popen( - [self.sparkSubmit, "--py-files", zip, "--master", "local-cluster[1,1,512]", script], - stdout=subprocess.PIPE) + proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, "--master", + "local-cluster[1,1,512]", script], + stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) self.assertIn("[2, 3, 4]", out) @@ -981,6 +998,7 @@ def test_single_script_on_cluster(self): @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): + """General PySpark tests that depend on scipy """ def test_serialize(self): @@ -993,15 +1011,16 @@ def test_serialize(self): @unittest.skipIf(not _have_numpy, "NumPy not installed") class NumPyTests(PySparkTestCase): + """General PySpark tests that depend on numpy """ def test_statcounter_array(self): - x = self.sc.parallelize([np.array([1.0,1.0]), np.array([2.0,2.0]), np.array([3.0,3.0])]) + x = self.sc.parallelize([np.array([1.0, 1.0]), np.array([2.0, 2.0]), np.array([3.0, 3.0])]) s = x.stats() - self.assertSequenceEqual([2.0,2.0], s.mean().tolist()) - self.assertSequenceEqual([1.0,1.0], s.min().tolist()) - self.assertSequenceEqual([3.0,3.0], s.max().tolist()) - self.assertSequenceEqual([1.0,1.0], s.sampleStdev().tolist()) + self.assertSequenceEqual([2.0, 2.0], s.mean().tolist()) + self.assertSequenceEqual([1.0, 1.0], s.min().tolist()) + self.assertSequenceEqual([3.0, 3.0], s.max().tolist()) + self.assertSequenceEqual([1.0, 1.0], s.sampleStdev().tolist()) if __name__ == "__main__": diff --git a/python/test_support/userlibrary.py b/python/test_support/userlibrary.py index 8e4a6292bc17c..73fd26e71f10d 100755 --- a/python/test_support/userlibrary.py +++ b/python/test_support/userlibrary.py @@ -19,6 +19,8 @@ Used to test shipping of code depenencies with SparkContext.addPyFile(). """ + class UserClass(object): + def hello(self): return "Hello World!" diff --git a/tox.ini b/tox.ini index 44766e529bf7f..a1fefdd0e176f 100644 --- a/tox.ini +++ b/tox.ini @@ -15,3 +15,4 @@ [pep8] max-line-length=100 +exclude=cloudpickle.py From 4e982364426c7d65032e8006c63ca4f9a0d40470 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 6 Aug 2014 13:10:33 -0700 Subject: [PATCH 0272/1492] SPARK-2566. Update ShuffleWriteMetrics incrementally I haven't tested this out on a cluster yet, but wanted to make sure the approach (passing ShuffleWriteMetrics down to DiskBlockObjectWriter) was ok Author: Sandy Ryza Closes #1481 from sryza/sandy-spark-2566 and squashes the following commits: 8090d88 [Sandy Ryza] Fix ExternalSorter b2a62ed [Sandy Ryza] Fix more test failures 8be6218 [Sandy Ryza] Fix test failures and mark a couple variables private c5e68e5 [Sandy Ryza] SPARK-2566. Update ShuffleWriteMetrics incrementally --- .../apache/spark/executor/TaskMetrics.scala | 4 +- .../shuffle/hash/HashShuffleWriter.scala | 16 ++-- .../shuffle/sort/SortShuffleWriter.scala | 16 ++-- .../apache/spark/storage/BlockManager.scala | 12 +-- .../spark/storage/BlockObjectWriter.scala | 77 ++++++++++--------- .../spark/storage/ShuffleBlockManager.scala | 9 ++- .../collection/ExternalAppendOnlyMap.scala | 18 +++-- .../util/collection/ExternalSorter.scala | 17 ++-- .../storage/BlockObjectWriterSuite.scala | 65 ++++++++++++++++ .../spark/storage/DiskBlockManagerSuite.scala | 9 ++- .../spark/tools/StoragePerfTester.scala | 3 +- 11 files changed, 164 insertions(+), 82 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 56cd8723a3a22..11a6e10243211 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -190,10 +190,10 @@ class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task */ - var shuffleBytesWritten: Long = _ + @volatile var shuffleBytesWritten: Long = _ /** * Time the task spent blocking on writes to disk or buffer cache, in nanoseconds */ - var shuffleWriteTime: Long = _ + @volatile var shuffleWriteTime: Long = _ } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 45d3b8b9b8725..51e454d9313c9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -39,10 +39,14 @@ private[spark] class HashShuffleWriter[K, V]( // we don't try deleting files, etc twice. private var stopping = false + private val writeMetrics = new ShuffleWriteMetrics() + metrics.shuffleWriteMetrics = Some(writeMetrics) + private val blockManager = SparkEnv.get.blockManager private val shuffleBlockManager = blockManager.shuffleBlockManager private val ser = Serializer.getSerializer(dep.serializer.getOrElse(null)) - private val shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser) + private val shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, mapId, numOutputSplits, ser, + writeMetrics) /** Write a bunch of records to this task's output */ override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { @@ -99,22 +103,12 @@ private[spark] class HashShuffleWriter[K, V]( private def commitWritesAndBuildStatus(): MapStatus = { // Commit the writes. Get the size of each bucket block (total block size). - var totalBytes = 0L - var totalTime = 0L val compressedSizes = shuffle.writers.map { writer: BlockObjectWriter => writer.commitAndClose() val size = writer.fileSegment().length - totalBytes += size - totalTime += writer.timeWriting() MapOutputTracker.compressSize(size) } - // Update shuffle metrics. - val shuffleMetrics = new ShuffleWriteMetrics - shuffleMetrics.shuffleBytesWritten = totalBytes - shuffleMetrics.shuffleWriteTime = totalTime - metrics.shuffleWriteMetrics = Some(shuffleMetrics) - new MapStatus(blockManager.blockManagerId, compressedSizes) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 24db2f287a47b..e54e6383d2ccc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -52,6 +52,9 @@ private[spark] class SortShuffleWriter[K, V, C]( private var mapStatus: MapStatus = null + private val writeMetrics = new ShuffleWriteMetrics() + context.taskMetrics.shuffleWriteMetrics = Some(writeMetrics) + /** Write a bunch of records to this task's output */ override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { // Get an iterator with the elements for each partition ID @@ -84,13 +87,10 @@ private[spark] class SortShuffleWriter[K, V, C]( val offsets = new Array[Long](numPartitions + 1) val lengths = new Array[Long](numPartitions) - // Statistics - var totalBytes = 0L - var totalTime = 0L - for ((id, elements) <- partitions) { if (elements.hasNext) { - val writer = blockManager.getDiskWriter(blockId, outputFile, ser, fileBufferSize) + val writer = blockManager.getDiskWriter(blockId, outputFile, ser, fileBufferSize, + writeMetrics) for (elem <- elements) { writer.write(elem) } @@ -98,18 +98,12 @@ private[spark] class SortShuffleWriter[K, V, C]( val segment = writer.fileSegment() offsets(id + 1) = segment.offset + segment.length lengths(id) = segment.length - totalTime += writer.timeWriting() - totalBytes += segment.length } else { // The partition is empty; don't create a new writer to avoid writing headers, etc offsets(id + 1) = offsets(id) } } - val shuffleMetrics = new ShuffleWriteMetrics - shuffleMetrics.shuffleBytesWritten = totalBytes - shuffleMetrics.shuffleWriteTime = totalTime - context.taskMetrics.shuffleWriteMetrics = Some(shuffleMetrics) context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 3876cf43e2a7d..8d21b02b747ff 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,7 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ -import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.executor.{DataReadMethod, InputMetrics, ShuffleWriteMetrics} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -562,17 +562,19 @@ private[spark] class BlockManager( /** * A short circuited method to get a block writer that can write data directly to disk. - * The Block will be appended to the File specified by filename. This is currently used for - * writing shuffle files out. Callers should handle error cases. + * The Block will be appended to the File specified by filename. Callers should handle error + * cases. */ def getDiskWriter( blockId: BlockId, file: File, serializer: Serializer, - bufferSize: Int): BlockObjectWriter = { + bufferSize: Int, + writeMetrics: ShuffleWriteMetrics): BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) - new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) + new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites, + writeMetrics) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 01d46e1ffc960..adda971fd7b47 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -22,6 +22,7 @@ import java.nio.channels.FileChannel import org.apache.spark.Logging import org.apache.spark.serializer.{SerializationStream, Serializer} +import org.apache.spark.executor.ShuffleWriteMetrics /** * An interface for writing JVM objects to some underlying storage. This interface allows @@ -60,41 +61,26 @@ private[spark] abstract class BlockObjectWriter(val blockId: BlockId) { * This is only valid after commitAndClose() has been called. */ def fileSegment(): FileSegment - - /** - * Cumulative time spent performing blocking writes, in ns. - */ - def timeWriting(): Long - - /** - * Number of bytes written so far - */ - def bytesWritten: Long } -/** BlockObjectWriter which writes directly to a file on disk. Appends to the given file. */ +/** + * BlockObjectWriter which writes directly to a file on disk. Appends to the given file. + * The given write metrics will be updated incrementally, but will not necessarily be current until + * commitAndClose is called. + */ private[spark] class DiskBlockObjectWriter( blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int, compressStream: OutputStream => OutputStream, - syncWrites: Boolean) + syncWrites: Boolean, + writeMetrics: ShuffleWriteMetrics) extends BlockObjectWriter(blockId) with Logging { - /** Intercepts write calls and tracks total time spent writing. Not thread safe. */ private class TimeTrackingOutputStream(out: OutputStream) extends OutputStream { - def timeWriting = _timeWriting - private var _timeWriting = 0L - - private def callWithTiming(f: => Unit) = { - val start = System.nanoTime() - f - _timeWriting += (System.nanoTime() - start) - } - def write(i: Int): Unit = callWithTiming(out.write(i)) override def write(b: Array[Byte]) = callWithTiming(out.write(b)) override def write(b: Array[Byte], off: Int, len: Int) = callWithTiming(out.write(b, off, len)) @@ -111,7 +97,11 @@ private[spark] class DiskBlockObjectWriter( private val initialPosition = file.length() private var finalPosition: Long = -1 private var initialized = false - private var _timeWriting = 0L + + /** Calling channel.position() to update the write metrics can be a little bit expensive, so we + * only call it every N writes */ + private var writesSinceMetricsUpdate = 0 + private var lastPosition = initialPosition override def open(): BlockObjectWriter = { fos = new FileOutputStream(file, true) @@ -128,14 +118,11 @@ private[spark] class DiskBlockObjectWriter( if (syncWrites) { // Force outstanding writes to disk and track how long it takes objOut.flush() - val start = System.nanoTime() - fos.getFD.sync() - _timeWriting += System.nanoTime() - start + def sync = fos.getFD.sync() + callWithTiming(sync) } objOut.close() - _timeWriting += ts.timeWriting - channel = null bs = null fos = null @@ -153,6 +140,7 @@ private[spark] class DiskBlockObjectWriter( // serializer stream and the lower level stream. objOut.flush() bs.flush() + updateBytesWritten() close() } finalPosition = file.length() @@ -162,6 +150,8 @@ private[spark] class DiskBlockObjectWriter( // truncating the file to its initial position. override def revertPartialWritesAndClose() { try { + writeMetrics.shuffleBytesWritten -= (lastPosition - initialPosition) + if (initialized) { objOut.flush() bs.flush() @@ -184,19 +174,36 @@ private[spark] class DiskBlockObjectWriter( if (!initialized) { open() } + objOut.writeObject(value) + + if (writesSinceMetricsUpdate == 32) { + writesSinceMetricsUpdate = 0 + updateBytesWritten() + } else { + writesSinceMetricsUpdate += 1 + } } override def fileSegment(): FileSegment = { - new FileSegment(file, initialPosition, bytesWritten) + new FileSegment(file, initialPosition, finalPosition - initialPosition) } - // Only valid if called after close() - override def timeWriting() = _timeWriting + private def updateBytesWritten() { + val pos = channel.position() + writeMetrics.shuffleBytesWritten += (pos - lastPosition) + lastPosition = pos + } + + private def callWithTiming(f: => Unit) = { + val start = System.nanoTime() + f + writeMetrics.shuffleWriteTime += (System.nanoTime() - start) + } - // Only valid if called after commit() - override def bytesWritten: Long = { - assert(finalPosition != -1, "bytesWritten is only valid after successful commit()") - finalPosition - initialPosition + // For testing + private[spark] def flush() { + objOut.flush() + bs.flush() } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index f9fdffae8bd8f..3565719b54545 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -29,6 +29,7 @@ import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.executor.ShuffleWriteMetrics /** A group of writers for a ShuffleMapTask, one writer per reducer. */ private[spark] trait ShuffleWriterGroup { @@ -111,7 +112,8 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { * Get a ShuffleWriterGroup for the given map task, which will register it as complete * when the writers are closed successfully */ - def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer) = { + def forMapTask(shuffleId: Int, mapId: Int, numBuckets: Int, serializer: Serializer, + writeMetrics: ShuffleWriteMetrics) = { new ShuffleWriterGroup { shuffleStates.putIfAbsent(shuffleId, new ShuffleState(numBuckets)) private val shuffleState = shuffleStates(shuffleId) @@ -121,7 +123,8 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { fileGroup = getUnusedFileGroup() Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => val blockId = ShuffleBlockId(shuffleId, mapId, bucketId) - blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize) + blockManager.getDiskWriter(blockId, fileGroup(bucketId), serializer, bufferSize, + writeMetrics) } } else { Array.tabulate[BlockObjectWriter](numBuckets) { bucketId => @@ -136,7 +139,7 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { logWarning(s"Failed to remove existing shuffle file $blockFile") } } - blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize) + blockManager.getDiskWriter(blockId, blockFile, serializer, bufferSize, writeMetrics) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 260a5c3888aa7..9f85b94a70800 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -31,6 +31,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.{DeserializationStream, Serializer} import org.apache.spark.storage.{BlockId, BlockManager} import org.apache.spark.util.collection.ExternalAppendOnlyMap.HashComparator +import org.apache.spark.executor.ShuffleWriteMetrics /** * :: DeveloperApi :: @@ -102,6 +103,10 @@ class ExternalAppendOnlyMap[K, V, C]( private var _diskBytesSpilled = 0L private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + + // Write metrics for current spill + private var curWriteMetrics: ShuffleWriteMetrics = _ + private val keyComparator = new HashComparator[K] private val ser = serializer.newInstance() @@ -172,7 +177,9 @@ class ExternalAppendOnlyMap[K, V, C]( logInfo("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" .format(threadId, mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() - var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) + curWriteMetrics = new ShuffleWriteMetrics() + var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, + curWriteMetrics) var objectsWritten = 0 // List of batch sizes (bytes) in the order they are written to disk @@ -183,9 +190,8 @@ class ExternalAppendOnlyMap[K, V, C]( val w = writer writer = null w.commitAndClose() - val bytesWritten = w.bytesWritten - batchSizes.append(bytesWritten) - _diskBytesSpilled += bytesWritten + _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten + batchSizes.append(curWriteMetrics.shuffleBytesWritten) objectsWritten = 0 } @@ -199,7 +205,9 @@ class ExternalAppendOnlyMap[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() - writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize) + curWriteMetrics = new ShuffleWriteMetrics() + writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, + curWriteMetrics) } } if (objectsWritten > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 3f93afd57b3ad..eb4849ebc6e52 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -28,6 +28,7 @@ import com.google.common.io.ByteStreams import org.apache.spark.{Aggregator, SparkEnv, Logging, Partitioner} import org.apache.spark.serializer.{DeserializationStream, Serializer} import org.apache.spark.storage.BlockId +import org.apache.spark.executor.ShuffleWriteMetrics /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -112,11 +113,14 @@ private[spark] class ExternalSorter[K, V, C]( // What threshold of elementsRead we start estimating map size at. private val trackMemoryThreshold = 1000 - // Spilling statistics + // Total spilling statistics private var spillCount = 0 private var _memoryBytesSpilled = 0L private var _diskBytesSpilled = 0L + // Write metrics for current spill + private var curWriteMetrics: ShuffleWriteMetrics = _ + // How much of the shared memory pool this collection has claimed private var myMemoryThreshold = 0L @@ -239,7 +243,8 @@ private[spark] class ExternalSorter[K, V, C]( logInfo("Thread %d spilling in-memory batch of %d MB to disk (%d spill%s so far)" .format(threadId, memorySize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) val (blockId, file) = diskBlockManager.createTempBlock() - var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize) + curWriteMetrics = new ShuffleWriteMetrics() + var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) var objectsWritten = 0 // Objects written since the last flush // List of batch sizes (bytes) in the order they are written to disk @@ -254,9 +259,8 @@ private[spark] class ExternalSorter[K, V, C]( val w = writer writer = null w.commitAndClose() - val bytesWritten = w.bytesWritten - batchSizes.append(bytesWritten) - _diskBytesSpilled += bytesWritten + _diskBytesSpilled += curWriteMetrics.shuffleBytesWritten + batchSizes.append(curWriteMetrics.shuffleBytesWritten) objectsWritten = 0 } @@ -275,7 +279,8 @@ private[spark] class ExternalSorter[K, V, C]( if (objectsWritten == serializerBatchSize) { flush() - writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize) + curWriteMetrics = new ShuffleWriteMetrics() + writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) } } if (objectsWritten > 0) { diff --git a/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala new file mode 100644 index 0000000000000..bbc7e1357b90d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockObjectWriterSuite.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.storage + +import org.scalatest.FunSuite +import java.io.File +import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.SparkConf + +class BlockObjectWriterSuite extends FunSuite { + test("verify write metrics") { + val file = new File("somefile") + file.deleteOnExit() + val writeMetrics = new ShuffleWriteMetrics() + val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, + new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + + writer.write(Long.box(20)) + // Metrics don't update on every write + assert(writeMetrics.shuffleBytesWritten == 0) + // After 32 writes, metrics should update + for (i <- 0 until 32) { + writer.flush() + writer.write(Long.box(i)) + } + assert(writeMetrics.shuffleBytesWritten > 0) + writer.commitAndClose() + assert(file.length() == writeMetrics.shuffleBytesWritten) + } + + test("verify write metrics on revert") { + val file = new File("somefile") + file.deleteOnExit() + val writeMetrics = new ShuffleWriteMetrics() + val writer = new DiskBlockObjectWriter(new TestBlockId("0"), file, + new JavaSerializer(new SparkConf()), 1024, os => os, true, writeMetrics) + + writer.write(Long.box(20)) + // Metrics don't update on every write + assert(writeMetrics.shuffleBytesWritten == 0) + // After 32 writes, metrics should update + for (i <- 0 until 32) { + writer.flush() + writer.write(Long.box(i)) + } + assert(writeMetrics.shuffleBytesWritten > 0) + writer.revertPartialWritesAndClose() + assert(writeMetrics.shuffleBytesWritten == 0) + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 985ac9394738c..b8299e2ea187f 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.SparkConf import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.executor.ShuffleWriteMetrics class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) @@ -153,7 +154,7 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before val shuffleManager = store.shuffleBlockManager - val shuffle1 = shuffleManager.forMapTask(1, 1, 1, serializer) + val shuffle1 = shuffleManager.forMapTask(1, 1, 1, serializer, new ShuffleWriteMetrics) for (writer <- shuffle1.writers) { writer.write("test1") writer.write("test2") @@ -165,7 +166,8 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before val shuffle1Segment = shuffle1.writers(0).fileSegment() shuffle1.releaseWriters(success = true) - val shuffle2 = shuffleManager.forMapTask(1, 2, 1, new JavaSerializer(testConf)) + val shuffle2 = shuffleManager.forMapTask(1, 2, 1, new JavaSerializer(testConf), + new ShuffleWriteMetrics) for (writer <- shuffle2.writers) { writer.write("test3") @@ -183,7 +185,8 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // of block based on remaining data in file : which could mess things up when there is concurrent read // and writes happening to the same shuffle group. - val shuffle3 = shuffleManager.forMapTask(1, 3, 1, new JavaSerializer(testConf)) + val shuffle3 = shuffleManager.forMapTask(1, 3, 1, new JavaSerializer(testConf), + new ShuffleWriteMetrics) for (writer <- shuffle3.writers) { writer.write("test3") writer.write("test4") diff --git a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 8a05fcb449aa6..17bf7c2541d13 100644 --- a/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -23,6 +23,7 @@ import java.util.concurrent.atomic.AtomicLong import org.apache.spark.SparkContext import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils +import org.apache.spark.executor.ShuffleWriteMetrics /** * Internal utility for micro-benchmarking shuffle write performance. @@ -56,7 +57,7 @@ object StoragePerfTester { def writeOutputBytes(mapId: Int, total: AtomicLong) = { val shuffle = blockManager.shuffleBlockManager.forMapTask(1, mapId, numOutputSplits, - new KryoSerializer(sc.conf)) + new KryoSerializer(sc.conf), new ShuffleWriteMetrics()) val writers = shuffle.writers for (i <- 1 to recordsPerMap) { writers(i % numOutputSplits).write(writeData) From 25cff1019da9d6cfc486a31d035b372ea5fbdfd2 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 6 Aug 2014 14:07:51 -0700 Subject: [PATCH 0273/1492] [SPARK-2852][MLLIB] API consistency for `mllib.feature` This is part of SPARK-2828: 1. added a Java-friendly fit method to Word2Vec with tests 2. change DeveloperApi to Experimental for Normalizer & StandardScaler 3. change default feature dimension to 2^20 in HashingTF Author: Xiangrui Meng Closes #1807 from mengxr/feature-api-check and squashes the following commits: 773c1a9 [Xiangrui Meng] change default numFeatures to 2^20 in HashingTF change annotation from DeveloperApi to Experimental in Normalizer and StandardScaler 883e122 [Xiangrui Meng] add @Experimental to Word2VecModel add a Java-friendly method to Word2Vec.fit with tests --- .../spark/mllib/feature/HashingTF.scala | 4 +- .../spark/mllib/feature/Normalizer.scala | 6 +- .../spark/mllib/feature/StandardScaler.scala | 6 +- .../apache/spark/mllib/feature/Word2Vec.scala | 19 +++++- .../mllib/feature/JavaWord2VecSuite.java | 66 +++++++++++++++++++ 5 files changed, 91 insertions(+), 10 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index 0f6d5809e098f..c53475818395f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -32,12 +32,12 @@ import org.apache.spark.util.Utils * :: Experimental :: * Maps a sequence of terms to their term frequencies using the hashing trick. * - * @param numFeatures number of features (default: 1000000) + * @param numFeatures number of features (default: 2^20^) */ @Experimental class HashingTF(val numFeatures: Int) extends Serializable { - def this() = this(1000000) + def this() = this(1 << 20) /** * Returns the index of the input term. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index ea9fd0a80d8e0..3afb47767281c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -19,11 +19,11 @@ package org.apache.spark.mllib.feature import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{Vector, Vectors} /** - * :: DeveloperApi :: + * :: Experimental :: * Normalizes samples individually to unit L^p^ norm * * For any 1 <= p < Double.PositiveInfinity, normalizes samples using @@ -33,7 +33,7 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} * * @param p Normalization in L^p^ space, p = 2 by default. */ -@DeveloperApi +@Experimental class Normalizer(p: Double) extends VectorTransformer { def this() = this(2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index cc2d7579c2901..e6c9f8f67df63 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -19,14 +19,14 @@ package org.apache.spark.mllib.feature import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD /** - * :: DeveloperApi :: + * :: Experimental :: * Standardizes features by removing the mean and scaling to unit variance using column summary * statistics on the samples in the training set. * @@ -34,7 +34,7 @@ import org.apache.spark.rdd.RDD * dense output, so this does not work on sparse input and will raise an exception. * @param withStd True by default. Scales the data to unit standard deviation. */ -@DeveloperApi +@Experimental class StandardScaler(withMean: Boolean, withStd: Boolean) extends VectorTransformer { def this() = this(false, true) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 3bf44ad7c44e3..395037e1ec47c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -17,6 +17,9 @@ package org.apache.spark.mllib.feature +import java.lang.{Iterable => JavaIterable} + +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -25,6 +28,7 @@ import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd._ @@ -239,7 +243,7 @@ class Word2Vec extends Serializable with Logging { a += 1 } } - + /** * Computes the vector representation of each word in vocabulary. * @param dataset an RDD of words @@ -369,11 +373,22 @@ class Word2Vec extends Serializable with Logging { new Word2VecModel(word2VecMap.toMap) } + + /** + * Computes the vector representation of each word in vocabulary (Java version). + * @param dataset a JavaRDD of words + * @return a Word2VecModel + */ + def fit[S <: JavaIterable[String]](dataset: JavaRDD[S]): Word2VecModel = { + fit(dataset.rdd.map(_.asScala)) + } } /** -* Word2Vec model + * :: Experimental :: + * Word2Vec model */ +@Experimental class Word2VecModel private[mllib] ( private val model: Map[String, Array[Float]]) extends Serializable { diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java new file mode 100644 index 0000000000000..fb7afe8c6434b --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaWord2VecSuite.java @@ -0,0 +1,66 @@ +/* + * 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.feature; + +import java.io.Serializable; +import java.util.List; + +import scala.Tuple2; + +import com.google.common.collect.Lists; +import com.google.common.base.Strings; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +public class JavaWord2VecSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaWord2VecSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + @SuppressWarnings("unchecked") + public void word2Vec() { + // The tests are to check Java compatibility. + String sentence = Strings.repeat("a b ", 100) + Strings.repeat("a c ", 10); + List words = Lists.newArrayList(sentence.split(" ")); + List> localDoc = Lists.newArrayList(words, words); + JavaRDD> doc = sc.parallelize(localDoc); + Word2Vec word2vec = new Word2Vec() + .setVectorSize(10) + .setSeed(42L); + Word2VecModel model = word2vec.fit(doc); + Tuple2[] syms = model.findSynonyms("a", 2); + Assert.assertEquals(2, syms.length); + Assert.assertEquals("b", syms[0]._1()); + Assert.assertEquals("c", syms[1]._1()); + } +} From e537b33c63d3fb373fe41deaa607d72e76e3906b Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Wed, 6 Aug 2014 14:12:21 -0700 Subject: [PATCH 0274/1492] [PySpark] Add blanklines to Python docstrings so example code renders correctly Author: RJ Nowling Closes #1808 from rnowling/pyspark_docs and squashes the following commits: c06d774 [RJ Nowling] Add blanklines to Python docstrings so example code renders correctly --- python/pyspark/rdd.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 30b834d2085cd..756e8f35fb03d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -134,6 +134,7 @@ class MaxHeapQ(object): """ An implementation of MaxHeap. + >>> import pyspark.rdd >>> heap = pyspark.rdd.MaxHeapQ(5) >>> [heap.insert(i) for i in range(10)] @@ -381,6 +382,7 @@ def mapPartitionsWithSplit(self, f, preservesPartitioning=False): def getNumPartitions(self): """ Returns the number of partitions in RDD + >>> rdd = sc.parallelize([1, 2, 3, 4], 2) >>> rdd.getNumPartitions() 2 @@ -570,6 +572,7 @@ 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)] @@ -1209,6 +1212,7 @@ def collectAsMap(self): def keys(self): """ Return an RDD with the keys of each tuple. + >>> m = sc.parallelize([(1, 2), (3, 4)]).keys() >>> m.collect() [1, 3] @@ -1218,6 +1222,7 @@ def keys(self): def values(self): """ Return an RDD with the values of each tuple. + >>> m = sc.parallelize([(1, 2), (3, 4)]).values() >>> m.collect() [2, 4] @@ -1642,6 +1647,7 @@ def repartition(self, numPartitions): Internally, this uses a shuffle to redistribute data. If you are decreasing the number of partitions in this RDD, consider using `coalesce`, which can avoid performing a shuffle. + >>> rdd = sc.parallelize([1,2,3,4,5,6,7], 4) >>> sorted(rdd.glom().collect()) [[1], [2, 3], [4, 5], [6, 7]] @@ -1656,6 +1662,7 @@ def repartition(self, numPartitions): def coalesce(self, numPartitions, shuffle=False): """ Return a new RDD that is reduced into `numPartitions` partitions. + >>> sc.parallelize([1, 2, 3, 4, 5], 3).glom().collect() [[1], [2, 3], [4, 5]] >>> sc.parallelize([1, 2, 3, 4, 5], 3).coalesce(1).glom().collect() @@ -1694,6 +1701,7 @@ def name(self): def setName(self, name): """ Assign a name to this RDD. + >>> rdd1 = sc.parallelize([1,2]) >>> rdd1.setName('RDD1') >>> rdd1.name() @@ -1753,6 +1761,7 @@ class PipelinedRDD(RDD): """ Pipelined maps: + >>> rdd = sc.parallelize([1, 2, 3, 4]) >>> rdd.map(lambda x: 2 * x).cache().map(lambda x: 2 * x).collect() [4, 8, 12, 16] From c6889d2cb9cd99f7e3e0ee14a4fdf301f1f9810e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 6 Aug 2014 16:34:53 -0700 Subject: [PATCH 0275/1492] [HOTFIX][Streaming] Handle port collisions in flume polling test This is failing my tests in #1777. @tdas Author: Andrew Or Closes #1803 from andrewor14/fix-flaky-streaming-test and squashes the following commits: ea11a03 [Andrew Or] Catch all exceptions caused by BindExceptions 54a0ca0 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-flaky-streaming-test 664095c [Andrew Or] Tone down bind exception message af3ddc9 [Andrew Or] Handle port collisions in flume polling test --- .../flume/FlumePollingStreamSuite.scala | 32 ++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 27bf2ac962721..a69baa16981a1 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -35,6 +35,7 @@ import org.apache.spark.streaming.dstream.ReceiverInputDStream import org.apache.spark.streaming.util.ManualClock import org.apache.spark.streaming.{TestSuiteBase, TestOutputStream, StreamingContext} import org.apache.spark.streaming.flume.sink._ +import org.apache.spark.util.Utils class FlumePollingStreamSuite extends TestSuiteBase { @@ -45,8 +46,37 @@ class FlumePollingStreamSuite extends TestSuiteBase { val eventsPerBatch = 100 val totalEventsPerChannel = batchCount * eventsPerBatch val channelCapacity = 5000 + val maxAttempts = 5 test("flume polling test") { + testMultipleTimes(testFlumePolling) + } + + test("flume polling test multiple hosts") { + testMultipleTimes(testFlumePollingMultipleHost) + } + + /** + * Run the given test until no more java.net.BindException's are thrown. + * Do this only up to a certain attempt limit. + */ + private def testMultipleTimes(test: () => Unit): Unit = { + var testPassed = false + var attempt = 0 + while (!testPassed && attempt < maxAttempts) { + try { + test() + testPassed = true + } catch { + case e: Exception if Utils.isBindCollision(e) => + logWarning("Exception when running flume polling test: " + e) + attempt += 1 + } + } + assert(testPassed, s"Test failed after $attempt attempts!") + } + + private def testFlumePolling(): Unit = { val testPort = getTestPort // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) @@ -80,7 +110,7 @@ class FlumePollingStreamSuite extends TestSuiteBase { channel.stop() } - test("flume polling test multiple hosts") { + private def testFlumePollingMultipleHost(): Unit = { val testPort = getTestPort // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) From 4e008334ee0fb60f9fe8820afa06f7b7f0fa7a6c Mon Sep 17 00:00:00 2001 From: Gregory Owen Date: Wed, 6 Aug 2014 16:52:00 -0700 Subject: [PATCH 0276/1492] SPARK-2882: Spark build now checks local maven cache for dependencies Fixes [SPARK-2882](https://issues.apache.org/jira/browse/SPARK-2882) Author: Gregory Owen Closes #1818 from GregOwen/spark-2882 and squashes the following commits: 294446d [Gregory Owen] SPARK-2882: Spark build now checks local maven cache for dependencies --- project/SparkBuild.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 40b588512ff08..ed587783d5606 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -115,7 +115,8 @@ object SparkBuild extends PomBuild { retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", publishMavenStyle := true, - + + resolvers += Resolver.mavenLocal, 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) From 17caae48b3608552dd6e3ae652043831f932ce95 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 6 Aug 2014 17:27:55 -0700 Subject: [PATCH 0277/1492] [SPARK-2583] ConnectionManager error reporting This patch modifies the ConnectionManager so that error messages are sent in reply when uncaught exceptions occur during message processing. This prevents message senders from hanging while waiting for an acknowledgment if the remote message processing failed. This is an updated version of sarutak's PR, #1490. The main change is to use Futures / Promises to signal errors. Author: Kousuke Saruta Author: Josh Rosen Closes #1758 from JoshRosen/connection-manager-fixes and squashes the following commits: 68620cb [Josh Rosen] Fix test in BlockFetcherIteratorSuite: 83673de [Josh Rosen] Error ACKs should trigger IOExceptions, so catch only those exceptions in the test. b8bb4d4 [Josh Rosen] Fix manager.id vs managerServer.id typo that broke security tests. 659521f [Josh Rosen] Include previous exception when throwing new one a2f745c [Josh Rosen] Remove sendMessageReliablySync; callers can wait themselves. c01c450 [Josh Rosen] Return Try[Message] from sendMessageReliablySync. f1cd1bb [Josh Rosen] Clean up @sarutak's PR #1490 for [SPARK-2583]: ConnectionManager error reporting 7399c6b [Josh Rosen] Merge remote-tracking branch 'origin/pr/1490' into connection-manager-fixes ee91bb7 [Kousuke Saruta] Modified BufferMessage.scala to keep the spark code style 9dfd0d8 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583 e7d9aa6 [Kousuke Saruta] rebase to master 326a17f [Kousuke Saruta] Add test cases to ConnectionManagerSuite.scala for SPARK-2583 2a18d6b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583 22d7ebd [Kousuke Saruta] Add test cases to BlockManagerSuite for SPARK-2583 e579302 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583 281589c [Kousuke Saruta] Add a test case to BlockFetcherIteratorSuite.scala for fetching block from remote from successfully 0654128 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583 ffaa83d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583 12d3de8 [Kousuke Saruta] Added BlockFetcherIteratorSuite.scala 4117b8f [Kousuke Saruta] Modified ConnectionManager to be alble to handle error during processing message 717c9c3 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583 6635467 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2583 e2b8c4a [Kousuke Saruta] Modify to propagete error using ConnectionManager --- .../apache/spark/network/BufferMessage.scala | 7 +- .../spark/network/ConnectionManager.scala | 143 ++++++++++-------- .../org/apache/spark/network/Message.scala | 2 + .../spark/network/MessageChunkHeader.scala | 7 +- .../org/apache/spark/network/SenderTest.scala | 7 +- .../spark/storage/BlockFetcherIterator.scala | 9 +- .../spark/storage/BlockManagerWorker.scala | 30 ++-- .../network/ConnectionManagerSuite.scala | 38 ++++- .../storage/BlockFetcherIteratorSuite.scala | 98 +++++++++++- .../spark/storage/BlockManagerSuite.scala | 110 +++++++++++++- 10 files changed, 362 insertions(+), 89 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala index 04df2f3b0d696..af35f1fc3e459 100644 --- a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala @@ -48,7 +48,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: val security = if (isSecurityNeg) 1 else 0 if (size == 0 && !gotChunkForSendingOnce) { val newChunk = new MessageChunk( - new MessageChunkHeader(typ, id, 0, 0, ackId, security, senderAddress), null) + new MessageChunkHeader(typ, id, 0, 0, ackId, hasError, security, senderAddress), null) gotChunkForSendingOnce = true return Some(newChunk) } @@ -66,7 +66,8 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: } buffer.position(buffer.position + newBuffer.remaining) val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, security, senderAddress), newBuffer) + typ, id, size, newBuffer.remaining, ackId, + hasError, security, senderAddress), newBuffer) gotChunkForSendingOnce = true return Some(newChunk) } @@ -88,7 +89,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer] buffer.position(buffer.position + newBuffer.remaining) val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, security, senderAddress), newBuffer) + typ, id, size, newBuffer.remaining, ackId, hasError, security, senderAddress), newBuffer) return Some(newChunk) } None diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 4c00225280cce..95f96b8463a01 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -17,6 +17,7 @@ package org.apache.spark.network +import java.io.IOException import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ @@ -45,16 +46,26 @@ private[spark] class ConnectionManager( name: String = "Connection manager") extends Logging { + /** + * Used by sendMessageReliably to track messages being sent. + * @param message the message that was sent + * @param connectionManagerId the connection manager that sent this message + * @param completionHandler callback that's invoked when the send has completed or failed + */ class MessageStatus( val message: Message, val connectionManagerId: ConnectionManagerId, completionHandler: MessageStatus => Unit) { + /** This is non-None if message has been ack'd */ var ackMessage: Option[Message] = None - var attempted = false - var acked = false - def markDone() { completionHandler(this) } + def markDone(ackMessage: Option[Message]) { + this.synchronized { + this.ackMessage = ackMessage + completionHandler(this) + } + } } private val selector = SelectorProvider.provider.openSelector() @@ -442,11 +453,7 @@ private[spark] class ConnectionManager( messageStatuses.values.filter(_.connectionManagerId == sendingConnectionManagerId) .foreach(status => { logInfo("Notifying " + status) - status.synchronized { - status.attempted = true - status.acked = false - status.markDone() - } + status.markDone(None) }) messageStatuses.retain((i, status) => { @@ -475,11 +482,7 @@ private[spark] class ConnectionManager( for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) { logInfo("Notifying " + s) - s.synchronized { - s.attempted = true - s.acked = false - s.markDone() - } + s.markDone(None) } messageStatuses.retain((i, status) => { @@ -547,13 +550,13 @@ private[spark] class ConnectionManager( val securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId.toString) val message = securityMsgResp.toBufferMessage - if (message == null) throw new Exception("Error creating security message") + if (message == null) throw new IOException("Error creating security message") sendSecurityMessage(waitingConn.getRemoteConnectionManagerId(), message) } catch { case e: Exception => { logError("Error handling sasl client authentication", e) waitingConn.close() - throw new Exception("Error evaluating sasl response: " + e) + throw new IOException("Error evaluating sasl response: ", e) } } } @@ -661,34 +664,39 @@ private[spark] class ConnectionManager( } } } - sentMessageStatus.synchronized { - sentMessageStatus.ackMessage = Some(message) - sentMessageStatus.attempted = true - sentMessageStatus.acked = true - sentMessageStatus.markDone() - } + sentMessageStatus.markDone(Some(message)) } else { - val ackMessage = if (onReceiveCallback != null) { - logDebug("Calling back") - onReceiveCallback(bufferMessage, connectionManagerId) - } else { - logDebug("Not calling back as callback is null") - None - } + var ackMessage : Option[Message] = None + try { + ackMessage = if (onReceiveCallback != null) { + logDebug("Calling back") + onReceiveCallback(bufferMessage, connectionManagerId) + } else { + logDebug("Not calling back as callback is null") + None + } - if (ackMessage.isDefined) { - if (!ackMessage.get.isInstanceOf[BufferMessage]) { - logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " - + ackMessage.get.getClass) - } else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) { - logDebug("Response to " + bufferMessage + " does not have ack id set") - ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id + if (ackMessage.isDefined) { + if (!ackMessage.get.isInstanceOf[BufferMessage]) { + logDebug("Response to " + bufferMessage + " is not a buffer message, it is of type " + + ackMessage.get.getClass) + } else if (!ackMessage.get.asInstanceOf[BufferMessage].hasAckId) { + logDebug("Response to " + bufferMessage + " does not have ack id set") + ackMessage.get.asInstanceOf[BufferMessage].ackId = bufferMessage.id + } + } + } catch { + case e: Exception => { + logError(s"Exception was thrown while processing message", e) + val m = Message.createBufferMessage(bufferMessage.id) + m.hasError = true + ackMessage = Some(m) } + } finally { + sendMessage(connectionManagerId, ackMessage.getOrElse { + Message.createBufferMessage(bufferMessage.id) + }) } - - sendMessage(connectionManagerId, ackMessage.getOrElse { - Message.createBufferMessage(bufferMessage.id) - }) } } case _ => throw new Exception("Unknown type message received") @@ -800,11 +808,7 @@ private[spark] class ConnectionManager( case Some(msgStatus) => { messageStatuses -= message.id logInfo("Notifying " + msgStatus.connectionManagerId) - msgStatus.synchronized { - msgStatus.attempted = true - msgStatus.acked = false - msgStatus.markDone() - } + msgStatus.markDone(None) } case None => { logError("no messageStatus for failed message id: " + message.id) @@ -823,11 +827,28 @@ private[spark] class ConnectionManager( selector.wakeup() } + /** + * Send a message and block until an acknowldgment is received or an error occurs. + * @param connectionManagerId the message's destination + * @param message the message being sent + * @return a Future that either returns the acknowledgment message or captures an exception. + */ def sendMessageReliably(connectionManagerId: ConnectionManagerId, message: Message) - : Future[Option[Message]] = { - val promise = Promise[Option[Message]] - val status = new MessageStatus( - message, connectionManagerId, s => promise.success(s.ackMessage)) + : Future[Message] = { + val promise = Promise[Message]() + val status = new MessageStatus(message, connectionManagerId, s => { + s.ackMessage match { + case None => // Indicates a failure where we either never sent or never got ACK'd + promise.failure(new IOException("sendMessageReliably failed without being ACK'd")) + case Some(ackMessage) => + if (ackMessage.hasError) { + promise.failure( + new IOException("sendMessageReliably failed with ACK that signalled a remote error")) + } else { + promise.success(ackMessage) + } + } + }) messageStatuses.synchronized { messageStatuses += ((message.id, status)) } @@ -835,11 +856,6 @@ private[spark] class ConnectionManager( promise.future } - def sendMessageReliablySync(connectionManagerId: ConnectionManagerId, - message: Message): Option[Message] = { - Await.result(sendMessageReliably(connectionManagerId, message), Duration.Inf) - } - def onReceiveMessage(callback: (Message, ConnectionManagerId) => Option[Message]) { onReceiveCallback = callback } @@ -862,6 +878,7 @@ private[spark] class ConnectionManager( private[spark] object ConnectionManager { + import ExecutionContext.Implicits.global def main(args: Array[String]) { val conf = new SparkConf @@ -896,7 +913,7 @@ private[spark] object ConnectionManager { (0 until count).map(i => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(manager.id, bufferMessage) + Await.result(manager.sendMessageReliably(manager.id, bufferMessage), Duration.Inf) }) println("--------------------------") println() @@ -917,8 +934,10 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 1 second) - if (!g.isDefined) println("Failed") + f.onFailure { + case e => println("Failed due to " + e) + } + Await.ready(f, 1 second) }) val finishTime = System.currentTimeMillis @@ -952,8 +971,10 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffers(count - 1 - i).duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 1 second) - if (!g.isDefined) println("Failed") + f.onFailure { + case e => println("Failed due to " + e) + } + Await.ready(f, 1 second) }) val finishTime = System.currentTimeMillis @@ -982,8 +1003,10 @@ private[spark] object ConnectionManager { val bufferMessage = Message.createBufferMessage(buffer.duplicate) manager.sendMessageReliably(manager.id, bufferMessage) }).foreach(f => { - val g = Await.result(f, 1 second) - if (!g.isDefined) println("Failed") + f.onFailure { + case e => println("Failed due to " + e) + } + Await.ready(f, 1 second) }) val finishTime = System.currentTimeMillis Thread.sleep(1000) diff --git a/core/src/main/scala/org/apache/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/Message.scala index 7caccfdbb44f9..04ea50f62918c 100644 --- a/core/src/main/scala/org/apache/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/Message.scala @@ -28,6 +28,7 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { var startTime = -1L var finishTime = -1L var isSecurityNeg = false + var hasError = false def size: Int @@ -87,6 +88,7 @@ private[spark] object Message { case BUFFER_MESSAGE => new BufferMessage(header.id, ArrayBuffer(ByteBuffer.allocate(header.totalSize)), header.other) } + newMessage.hasError = header.hasError newMessage.senderAddress = header.address newMessage } diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala index ead663ede7a1c..f3ecca5f992e0 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala @@ -27,6 +27,7 @@ private[spark] class MessageChunkHeader( val totalSize: Int, val chunkSize: Int, val other: Int, + val hasError: Boolean, val securityNeg: Int, val address: InetSocketAddress) { lazy val buffer = { @@ -41,6 +42,7 @@ private[spark] class MessageChunkHeader( putInt(totalSize). putInt(chunkSize). putInt(other). + put(if (hasError) 1.asInstanceOf[Byte] else 0.asInstanceOf[Byte]). putInt(securityNeg). putInt(ip.size). put(ip). @@ -56,7 +58,7 @@ private[spark] class MessageChunkHeader( private[spark] object MessageChunkHeader { - val HEADER_SIZE = 44 + val HEADER_SIZE = 45 def create(buffer: ByteBuffer): MessageChunkHeader = { if (buffer.remaining != HEADER_SIZE) { @@ -67,13 +69,14 @@ private[spark] object MessageChunkHeader { val totalSize = buffer.getInt() val chunkSize = buffer.getInt() val other = buffer.getInt() + val hasError = buffer.get() != 0 val securityNeg = buffer.getInt() val ipSize = buffer.getInt() val ipBytes = new Array[Byte](ipSize) buffer.get(ipBytes) val ip = InetAddress.getByAddress(ipBytes) val port = buffer.getInt() - new MessageChunkHeader(typ, id, totalSize, chunkSize, other, securityNeg, + new MessageChunkHeader(typ, id, totalSize, chunkSize, other, hasError, securityNeg, new InetSocketAddress(ip, port)) } } diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index b8ea7c2cff9a2..ea2ad104ecae1 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -20,6 +20,10 @@ package org.apache.spark.network import java.nio.ByteBuffer import org.apache.spark.{SecurityManager, SparkConf} +import scala.concurrent.Await +import scala.concurrent.duration.Duration +import scala.util.Try + private[spark] object SenderTest { def main(args: Array[String]) { @@ -51,7 +55,8 @@ private[spark] object SenderTest { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis /* println("Started timer at " + startTime) */ - val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) + val promise = manager.sendMessageReliably(targetConnectionManagerId, dataMessage) + val responseStr: String = Try(Await.result(promise, Duration.Inf)) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) new String(buffer.array, "utf-8") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index ccf830e118ee7..938af6f5b923a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -22,6 +22,7 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue +import scala.util.{Failure, Success} import io.netty.buffer.ByteBuf @@ -118,8 +119,8 @@ object BlockFetcherIterator { bytesInFlight += req.size val sizeMap = req.blocks.toMap // so we can look up the size of each blockID val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onSuccess { - case Some(message) => { + future.onComplete { + case Success(message) => { val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { @@ -135,8 +136,8 @@ object BlockFetcherIterator { logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } } - case None => { - logError("Could not get block(s) from " + cmId) + case Failure(exception) => { + logError("Could not get block(s) from " + cmId, exception) for ((blockId, size) <- req.blocks) { results.put(new FetchResult(blockId, -1, null)) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala index c7766a3a65671..bf002a42d5dc5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala @@ -23,6 +23,10 @@ import org.apache.spark.Logging import org.apache.spark.network._ import org.apache.spark.util.Utils +import scala.concurrent.Await +import scala.concurrent.duration.Duration +import scala.util.{Try, Failure, Success} + /** * A network interface for BlockManager. Each slave should have one * BlockManagerWorker. @@ -44,13 +48,19 @@ private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) Some(new BlockMessageArray(responseMessages).toBufferMessage) } catch { - case e: Exception => logError("Exception handling buffer message", e) - None + case e: Exception => { + logError("Exception handling buffer message", e) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } } } case otherMessage: Any => { logError("Unknown type message received: " + otherMessage) - None + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) } } } @@ -109,9 +119,9 @@ private[spark] object BlockManagerWorker extends Logging { val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromPutBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) - val resultMessage = connectionManager.sendMessageReliablySync( - toConnManagerId, blockMessageArray.toBufferMessage) - resultMessage.isDefined + val resultMessage = Try(Await.result(connectionManager.sendMessageReliably( + toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) + resultMessage.isSuccess } def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = { @@ -119,10 +129,10 @@ private[spark] object BlockManagerWorker extends Logging { val connectionManager = blockManager.connectionManager val blockMessage = BlockMessage.fromGetBlock(msg) val blockMessageArray = new BlockMessageArray(blockMessage) - val responseMessage = connectionManager.sendMessageReliablySync( - toConnManagerId, blockMessageArray.toBufferMessage) + val responseMessage = Try(Await.result(connectionManager.sendMessageReliably( + toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) responseMessage match { - case Some(message) => { + case Success(message) => { val bufferMessage = message.asInstanceOf[BufferMessage] logDebug("Response message received " + bufferMessage) BlockMessageArray.fromBufferMessage(bufferMessage).foreach(blockMessage => { @@ -130,7 +140,7 @@ private[spark] object BlockManagerWorker extends Logging { return blockMessage.getData }) } - case None => logDebug("No response message received") + case Failure(exception) => logDebug("No response message received") } null } diff --git a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala index 415ad8c432c12..846537df003df 100644 --- a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.network +import java.io.IOException import java.nio._ import org.apache.spark.{SecurityManager, SparkConf} @@ -25,6 +26,7 @@ import org.scalatest.FunSuite import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ import scala.language.postfixOps +import scala.util.Try /** * Test the ConnectionManager with various security settings. @@ -46,7 +48,7 @@ class ConnectionManagerSuite extends FunSuite { buffer.flip val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(manager.id, bufferMessage) + Await.result(manager.sendMessageReliably(manager.id, bufferMessage), 10 seconds) assert(receivedMessage == true) @@ -79,7 +81,7 @@ class ConnectionManagerSuite extends FunSuite { (0 until count).map(i => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(managerServer.id, bufferMessage) + Await.result(manager.sendMessageReliably(managerServer.id, bufferMessage), 10 seconds) }) assert(numReceivedServerMessages == 10) @@ -118,7 +120,10 @@ class ConnectionManagerSuite extends FunSuite { val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip val bufferMessage = Message.createBufferMessage(buffer.duplicate) - manager.sendMessageReliablySync(managerServer.id, bufferMessage) + // Expect managerServer to close connection, which we'll report as an error: + intercept[IOException] { + Await.result(manager.sendMessageReliably(managerServer.id, bufferMessage), 10 seconds) + } assert(numReceivedServerMessages == 0) assert(numReceivedMessages == 0) @@ -163,6 +168,8 @@ class ConnectionManagerSuite extends FunSuite { val g = Await.result(f, 1 second) assert(false) } catch { + case i: IOException => + assert(true) case e: TimeoutException => { // we should timeout here since the client can't do the negotiation assert(true) @@ -209,7 +216,6 @@ class ConnectionManagerSuite extends FunSuite { }).foreach(f => { try { val g = Await.result(f, 1 second) - if (!g.isDefined) assert(false) else assert(true) } catch { case e: Exception => { assert(false) @@ -223,7 +229,31 @@ class ConnectionManagerSuite extends FunSuite { managerServer.stop() } + test("Ack error message") { + val conf = new SparkConf + conf.set("spark.authenticate", "false") + val securityManager = new SecurityManager(conf) + val manager = new ConnectionManager(0, conf, securityManager) + val managerServer = new ConnectionManager(0, conf, securityManager) + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + throw new Exception + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer) + + val future = manager.sendMessageReliably(managerServer.id, bufferMessage) + + intercept[IOException] { + Await.result(future, 1 second) + } + manager.stop() + managerServer.stop() + + } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala index 8dca2ebb312f5..1538995a6b404 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala @@ -17,18 +17,22 @@ package org.apache.spark.storage +import java.io.IOException +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.future +import scala.concurrent.ExecutionContext.Implicits.global + import org.scalatest.{FunSuite, Matchers} -import org.scalatest.PrivateMethodTester._ import org.mockito.Mockito._ import org.mockito.Matchers.{any, eq => meq} import org.mockito.stubbing.Answer import org.mockito.invocation.InvocationOnMock -import org.apache.spark._ import org.apache.spark.storage.BlockFetcherIterator._ -import org.apache.spark.network.{ConnectionManager, ConnectionManagerId, - Message} +import org.apache.spark.network.{ConnectionManager, Message} class BlockFetcherIteratorSuite extends FunSuite with Matchers { @@ -137,4 +141,90 @@ class BlockFetcherIteratorSuite extends FunSuite with Matchers { assert(iterator.next._2.isDefined, "All elements should be defined but 5th element is not actually defined") } + test("block fetch from remote fails using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + when(blockManager.connectionManager).thenReturn(connManager) + + val f = future { + throw new IOException("Send failed or we received an error ACK") + } + when(connManager.sendMessageReliably(any(), + any())).thenReturn(f) + when(blockManager.futureExecContext).thenReturn(global) + + when(blockManager.blockManagerId).thenReturn( + BlockManagerId("test-client", "test-client", 1, 0)) + when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) + + val blId1 = ShuffleBlockId(0,0,0) + val blId2 = ShuffleBlockId(0,1,0) + val bmId = BlockManagerId("test-server", "test-server",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, Seq((blId1, 1L), (blId2, 1L))) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, + blocksByAddress, null) + + iterator.initialize() + iterator.foreach{ + case (_, r) => { + (!r.isDefined) should be(true) + } + } + } + + test("block fetch from remote succeed using BasicBlockFetcherIterator") { + val blockManager = mock(classOf[BlockManager]) + val connManager = mock(classOf[ConnectionManager]) + when(blockManager.connectionManager).thenReturn(connManager) + + val blId1 = ShuffleBlockId(0,0,0) + val blId2 = ShuffleBlockId(0,1,0) + val buf1 = ByteBuffer.allocate(4) + val buf2 = ByteBuffer.allocate(4) + buf1.putInt(1) + buf1.flip() + buf2.putInt(1) + buf2.flip() + val blockMessage1 = BlockMessage.fromGotBlock(GotBlock(blId1, buf1)) + val blockMessage2 = BlockMessage.fromGotBlock(GotBlock(blId2, buf2)) + val blockMessageArray = new BlockMessageArray( + Seq(blockMessage1, blockMessage2)) + + val bufferMessage = blockMessageArray.toBufferMessage + val buffer = ByteBuffer.allocate(bufferMessage.size) + val arrayBuffer = new ArrayBuffer[ByteBuffer] + bufferMessage.buffers.foreach{ b => + buffer.put(b) + } + buffer.flip() + arrayBuffer += buffer + + val f = future { + Message.createBufferMessage(arrayBuffer) + } + when(connManager.sendMessageReliably(any(), + any())).thenReturn(f) + when(blockManager.futureExecContext).thenReturn(global) + + when(blockManager.blockManagerId).thenReturn( + BlockManagerId("test-client", "test-client", 1, 0)) + when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) + + val bmId = BlockManagerId("test-server", "test-server",1 , 0) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, Seq((blId1, 1L), (blId2, 1L))) + ) + + val iterator = new BasicBlockFetcherIterator(blockManager, + blocksByAddress, null) + iterator.initialize() + iterator.foreach{ + case (_, r) => { + (r.isDefined) should be(true) + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 0ac0269d7cfc1..94bb2c445d2e9 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -25,7 +25,11 @@ import akka.actor._ import akka.pattern.ask import akka.util.Timeout -import org.mockito.Mockito.{mock, when} +import org.mockito.invocation.InvocationOnMock +import org.mockito.Matchers.any +import org.mockito.Mockito.{doAnswer, mock, spy, when} +import org.mockito.stubbing.Answer + import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ @@ -33,6 +37,7 @@ import org.scalatest.Matchers import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod +import org.apache.spark.network.{Message, ConnectionManagerId} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat @@ -1000,6 +1005,109 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } + test("return error message when error occurred in BlockManagerWorker#onBlockMessageReceive") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + + val worker = spy(new BlockManagerWorker(store)) + val connManagerId = mock(classOf[ConnectionManagerId]) + + // setup request block messages + val reqBlId1 = ShuffleBlockId(0,0,0) + val reqBlId2 = ShuffleBlockId(0,1,0) + val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) + val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) + val reqBlockMessages = new BlockMessageArray( + Seq(reqBlockMessage1, reqBlockMessage2)) + val reqBufferMessage = reqBlockMessages.toBufferMessage + + val answer = new Answer[Option[BlockMessage]] { + override def answer(invocation: InvocationOnMock) + :Option[BlockMessage]= { + throw new Exception + } + } + + doAnswer(answer).when(worker).processBlockMessage(any()) + + // Test when exception was thrown during processing block messages + var ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) + + assert(ackMessage.isDefined, "When Exception was thrown in " + + "BlockManagerWorker#processBlockMessage, " + + "ackMessage should be defined") + assert(ackMessage.get.hasError, "When Exception was thown in " + + "BlockManagerWorker#processBlockMessage, " + + "ackMessage should have error") + + val notBufferMessage = mock(classOf[Message]) + + // Test when not BufferMessage was received + ackMessage = worker.onBlockMessageReceive(notBufferMessage, connManagerId) + assert(ackMessage.isDefined, "When not BufferMessage was passed to " + + "BlockManagerWorker#onBlockMessageReceive, " + + "ackMessage should be defined") + assert(ackMessage.get.hasError, "When not BufferMessage was passed to " + + "BlockManagerWorker#onBlockMessageReceive, " + + "ackMessage should have error") + } + + test("return ack message when no error occurred in BlocManagerWorker#onBlockMessageReceive") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + + val worker = spy(new BlockManagerWorker(store)) + val connManagerId = mock(classOf[ConnectionManagerId]) + + // setup request block messages + val reqBlId1 = ShuffleBlockId(0,0,0) + val reqBlId2 = ShuffleBlockId(0,1,0) + val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) + val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) + val reqBlockMessages = new BlockMessageArray( + Seq(reqBlockMessage1, reqBlockMessage2)) + + val tmpBufferMessage = reqBlockMessages.toBufferMessage + val buffer = ByteBuffer.allocate(tmpBufferMessage.size) + val arrayBuffer = new ArrayBuffer[ByteBuffer] + tmpBufferMessage.buffers.foreach{ b => + buffer.put(b) + } + buffer.flip() + arrayBuffer += buffer + val reqBufferMessage = Message.createBufferMessage(arrayBuffer) + + // setup ack block messages + val buf1 = ByteBuffer.allocate(4) + val buf2 = ByteBuffer.allocate(4) + buf1.putInt(1) + buf1.flip() + buf2.putInt(1) + buf2.flip() + val ackBlockMessage1 = BlockMessage.fromGotBlock(GotBlock(reqBlId1, buf1)) + val ackBlockMessage2 = BlockMessage.fromGotBlock(GotBlock(reqBlId2, buf2)) + + val answer = new Answer[Option[BlockMessage]] { + override def answer(invocation: InvocationOnMock) + :Option[BlockMessage]= { + if (invocation.getArguments()(0).asInstanceOf[BlockMessage].eq( + reqBlockMessage1)) { + return Some(ackBlockMessage1) + } else { + return Some(ackBlockMessage2) + } + } + } + + doAnswer(answer).when(worker).processBlockMessage(any()) + + val ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) + assert(ackMessage.isDefined, "When BlockManagerWorker#onBlockMessageReceive " + + "was executed successfully, ackMessage should be defined") + assert(!ackMessage.get.hasError, "When BlockManagerWorker#onBlockMessageReceive " + + "was executed successfully, ackMessage should not have error") + } + test("reserve/release unroll memory") { store = makeBlockManager(12000) val memoryStore = store.memoryStore From 4201d2711cd20a2892c40eb11102f73c2f826b2e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 6 Aug 2014 18:13:35 -0700 Subject: [PATCH 0278/1492] SPARK-2879 [BUILD] Use HTTPS to access Maven Central and other repos Maven Central has just now enabled HTTPS access for everyone to Maven Central (http://central.sonatype.org/articles/2014/Aug/03/https-support-launching-now/) This is timely, as a reminder of how easily an attacker can slip malicious code into a build that's downloading artifacts over HTTP (http://blog.ontoillogical.com/blog/2014/07/28/how-to-take-over-any-java-developer/). In the meantime, it looks like the Spring repo also now supports HTTPS, so can be used this way too. I propose to use HTTPS to access these repos. Author: Sean Owen Closes #1805 from srowen/SPARK-2879 and squashes the following commits: 7043a8e [Sean Owen] Use HTTPS for Maven Central libs and plugins; use id 'central' to override parent properly; use HTTPS for Spring repo --- pom.xml | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 4ab027bad55c0..76bf6d8f902a8 100644 --- a/pom.xml +++ b/pom.xml @@ -143,11 +143,11 @@ - maven-repo + central Maven Repository - http://repo.maven.apache.org/maven2 + https://repo.maven.apache.org/maven2 true @@ -213,7 +213,7 @@ spring-releases Spring Release Repository - http://repo.spring.io/libs-release + https://repo.spring.io/libs-release true @@ -222,6 +222,15 @@ + + + central + https://repo1.maven.org/maven2 + + true + + + From a263a7e9f060b3017142cdae5f1270db9458d8d3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 6 Aug 2014 18:45:03 -0700 Subject: [PATCH 0279/1492] HOTFIX: Support custom Java 7 location --- dev/create-release/create-release.sh | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 42473629d4f15..1867cf4ec46ca 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -35,6 +35,12 @@ RELEASE_VERSION=${RELEASE_VERSION:-1.0.0} RC_NAME=${RC_NAME:-rc2} USER_NAME=${USER_NAME:-pwendell} +if [ -z "$JAVA_HOME" ]; then + echo "Error: JAVA_HOME is not set, cannot proceed." + exit -1 +fi +JAVA_7_HOME=${JAVA_7_HOME:-$JAVA_HOME} + set -e GIT_TAG=v$RELEASE_VERSION-$RC_NAME @@ -130,7 +136,8 @@ scp spark-* \ cd spark sbt/sbt clean cd docs -PRODUCTION=1 jekyll build +# Compile docs with Java 7 to use nicer format +JAVA_HOME=$JAVA_7_HOME PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs ssh $USER_NAME@people.apache.org \ From ffd1f59a62a9dd9a4d5a7b09490b9d01ff1cd42d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 6 Aug 2014 21:22:13 -0700 Subject: [PATCH 0280/1492] [SPARK-2887] fix bug of countApproxDistinct() when have more than one partition fix bug of countApproxDistinct() when have more than one partition Author: Davies Liu Closes #1812 from davies/approx and squashes the following commits: bf757ce [Davies Liu] fix bug of countApproxDistinct() when have more than one partition --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) 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 e1c49e35abecd..0159003c88e06 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1004,7 +1004,7 @@ abstract class RDD[T: ClassTag]( }, (h1: HyperLogLogPlus, h2: HyperLogLogPlus) => { h1.addAll(h2) - h2 + h1 }).cardinality() } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index b31e3a09e5b9c..4a7dc8dca25e2 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -81,11 +81,11 @@ class RDDSuite extends FunSuite with SharedSparkContext { def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble - val size = 100 - val uniformDistro = for (i <- 1 to 100000) yield i % size - val simpleRdd = sc.makeRDD(uniformDistro) - assert(error(simpleRdd.countApproxDistinct(4, 0), size) < 0.4) - assert(error(simpleRdd.countApproxDistinct(8, 0), size) < 0.1) + val size = 1000 + val uniformDistro = for (i <- 1 to 5000) yield i % size + val simpleRdd = sc.makeRDD(uniformDistro, 10) + assert(error(simpleRdd.countApproxDistinct(8, 0), size) < 0.2) + assert(error(simpleRdd.countApproxDistinct(12, 0), size) < 0.1) } test("SparkContext.union") { From 47ccd5e71be49b723476f3ff8d5768f0f45c2ea6 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 6 Aug 2014 22:58:59 -0700 Subject: [PATCH 0281/1492] [SPARK-2851] [mllib] DecisionTree Python consistency update Added 6 static train methods to match Python API, but without default arguments (but with Python default args noted in docs). Added factory classes for Algo and Impurity, but made private[mllib]. CC: mengxr dorx Please let me know if there are other changes which would help with API consistency---thanks! Author: Joseph K. Bradley Closes #1798 from jkbradley/dt-python-consistency and squashes the following commits: 6f7edf8 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-python-consistency a0d7dbe [Joseph K. Bradley] DecisionTree: In Java-friendly train* methods, changed to use JavaRDD instead of RDD. ee1d236 [Joseph K. Bradley] DecisionTree API updates: * Removed train() function in Python API (tree.py) ** Removed corresponding function in Scala/Java API (the ones taking basic types) 00f820e [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-python-consistency fe6dbfa [Joseph K. Bradley] removed unnecessary imports e358661 [Joseph K. Bradley] DecisionTree API change: * Added 6 static train methods to match Python API, but without default arguments (but with Python default args noted in docs). c699850 [Joseph K. Bradley] a few doc comments eaf84c0 [Joseph K. Bradley] Added DecisionTree static train() methods API to match Python, but without default parameters --- .../mllib/api/python/PythonMLLibAPI.scala | 19 +-- .../spark/mllib/tree/DecisionTree.scala | 151 ++++++++++++++---- .../spark/mllib/tree/configuration/Algo.scala | 6 + .../mllib/tree/impurity/Impurities.scala | 32 ++++ python/pyspark/mllib/tree.py | 50 ++---- 5 files changed, 181 insertions(+), 77 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurities.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index fd0b9556c7d54..ba7ccd8ce4b8b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -25,16 +25,14 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ -import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} import org.apache.spark.mllib.random.{RandomRDDGenerators => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} +import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model.DecisionTreeModel import org.apache.spark.mllib.stat.Statistics import org.apache.spark.mllib.stat.correlation.CorrelationNames @@ -523,17 +521,8 @@ class PythonMLLibAPI extends Serializable { val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint) - val algo: Algo = algoStr match { - case "classification" => Classification - case "regression" => Regression - case _ => throw new IllegalArgumentException(s"Bad algoStr parameter: $algoStr") - } - val impurity: Impurity = impurityStr match { - case "gini" => Gini - case "entropy" => Entropy - case "variance" => Variance - case _ => throw new IllegalArgumentException(s"Bad impurityStr parameter: $impurityStr") - } + val algo = Algo.fromString(algoStr) + val impurity = Impurities.fromString(impurityStr) val strategy = new Strategy( algo = algo, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 1d03e6e3b36cf..c8a865659682f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -17,14 +17,18 @@ package org.apache.spark.mllib.tree +import org.apache.spark.api.java.JavaRDD + +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.mllib.tree.impurity.{Impurities, Gini, Entropy, Impurity} import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom @@ -200,6 +204,10 @@ object DecisionTree extends Serializable with Logging { * Method to train a decision tree model. * The method supports binary and multiclass classification and regression. * + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. + * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. * For regression, labels are real numbers. @@ -213,10 +221,12 @@ object DecisionTree extends Serializable with Logging { } /** - * Method to train a decision tree model where the instances are represented as an RDD of - * (label, features) pairs. The method supports binary classification and regression. For the - * binary classification, the label for each instance should either be 0 or 1 to denote the two - * classes. + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. + * + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. @@ -237,10 +247,12 @@ object DecisionTree extends Serializable with Logging { } /** - * Method to train a decision tree model where the instances are represented as an RDD of - * (label, features) pairs. The method supports binary classification and regression. For the - * binary classification, the label for each instance should either be 0 or 1 to denote the two - * classes. + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. + * + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. @@ -263,11 +275,12 @@ object DecisionTree extends Serializable with Logging { } /** - * Method to train a decision tree model where the instances are represented as an RDD of - * (label, features) pairs. The decision tree method supports binary classification and - * regression. For the binary classification, the label for each instance should either be 0 or - * 1 to denote the two classes. The method also supports categorical features inputs where the - * number of categories can specified using the categoricalFeaturesInfo option. + * Method to train a decision tree model. + * The method supports binary and multiclass classification and regression. + * + * Note: Using [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + * and [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + * is recommended to clearly separate classification and regression. * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * For classification, labels should take values {0, 1, ..., numClasses-1}. @@ -279,11 +292,9 @@ object DecisionTree extends Serializable with Logging { * @param numClassesForClassification number of classes for classification. Default value of 2. * @param maxBins maximum number of bins used for splitting features * @param quantileCalculationStrategy algorithm for calculating quantiles - * @param categoricalFeaturesInfo A map storing information about the categorical variables and - * the number of discrete values they take. For example, - * an entry (n -> k) implies the feature n is categorical with k - * categories 0, 1, 2, ... , k-1. It's important to note that - * features are zero-indexed. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. * @return DecisionTreeModel that can be used for prediction */ def train( @@ -300,6 +311,93 @@ object DecisionTree extends Serializable with Logging { new DecisionTree(strategy).train(input) } + /** + * Method to train a decision tree model for binary or multiclass classification. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels should take values {0, 1, ..., numClasses-1}. + * @param numClassesForClassification number of classes for classification. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param impurity Criterion used for information gain calculation. + * Supported values: "gini" (recommended) or "entropy". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @return DecisionTreeModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: Map[Int, Int], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + val impurityType = Impurities.fromString(impurity) + train(input, Classification, impurityType, maxDepth, numClassesForClassification, maxBins, Sort, + categoricalFeaturesInfo) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainClassifier]] + */ + def trainClassifier( + input: JavaRDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + trainClassifier(input.rdd, numClassesForClassification, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + impurity, maxDepth, maxBins) + } + + /** + * Method to train a decision tree model for regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels are real numbers. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param impurity Criterion used for information gain calculation. + * Supported values: "variance". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @return DecisionTreeModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + categoricalFeaturesInfo: Map[Int, Int], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + val impurityType = Impurities.fromString(impurity) + train(input, Regression, impurityType, maxDepth, 0, maxBins, Sort, categoricalFeaturesInfo) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.DecisionTree$#trainRegressor]] + */ + def trainRegressor( + input: JavaRDD[LabeledPoint], + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + impurity: String, + maxDepth: Int, + maxBins: Int): DecisionTreeModel = { + trainRegressor(input.rdd, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + impurity, maxDepth, maxBins) + } + + private val InvalidBinIndex = -1 /** @@ -1331,16 +1429,15 @@ object DecisionTree extends Serializable with Logging { * Categorical features: * For each feature, there is 1 bin per split. * Splits and bins are handled in 2 ways: - * (a) For multiclass classification with a low-arity feature + * (a) "unordered features" + * For multiclass classification with a low-arity feature * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), * the feature is split based on subsets of categories. - * There are 2^(maxFeatureValue - 1) - 1 splits. - * (b) For regression and binary classification, + * There are math.pow(2, maxFeatureValue - 1) - 1 splits. + * (b) "ordered features" + * For regression and binary classification, * and for multiclass classification with a high-arity feature, - * there is one split per category. - - * Categorical case (a) features are called unordered features. - * Other cases are called ordered features. + * there is one bin per category. * * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 79a01f58319e8..0ef9c6181a0a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -27,4 +27,10 @@ import org.apache.spark.annotation.Experimental object Algo extends Enumeration { type Algo = Value val Classification, Regression = Value + + private[mllib] def fromString(name: String): Algo = name match { + case "classification" => Classification + case "regression" => Regression + case _ => throw new IllegalArgumentException(s"Did not recognize Algo name: $name") + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurities.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurities.scala new file mode 100644 index 0000000000000..9a6452aa13a61 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurities.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impurity + +/** + * Factory for Impurity instances. + */ +private[mllib] object Impurities { + + def fromString(name: String): Impurity = name match { + case "gini" => Gini + case "entropy" => Entropy + case "variance" => Variance + case _ => throw new IllegalArgumentException(s"Did not recognize Impurity name: $name") + } + +} diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 2518001ea0b93..e1a4671709b7d 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -131,7 +131,7 @@ class DecisionTree(object): """ @staticmethod - def trainClassifier(data, numClasses, categoricalFeaturesInfo={}, + def trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity="gini", maxDepth=4, maxBins=100): """ Train a DecisionTreeModel for classification. @@ -150,12 +150,20 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo={}, :param maxBins: Number of bins used for finding splits at each node. :return: DecisionTreeModel """ - return DecisionTree.train(data, "classification", numClasses, - categoricalFeaturesInfo, - impurity, maxDepth, maxBins) + sc = data.context + dataBytes = _get_unmangled_labeled_point_rdd(data) + categoricalFeaturesInfoJMap = \ + MapConverter().convert(categoricalFeaturesInfo, + sc._gateway._gateway_client) + model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( + dataBytes._jrdd, "classification", + numClasses, categoricalFeaturesInfoJMap, + impurity, maxDepth, maxBins) + dataBytes.unpersist() + return DecisionTreeModel(sc, model) @staticmethod - def trainRegressor(data, categoricalFeaturesInfo={}, + def trainRegressor(data, categoricalFeaturesInfo, impurity="variance", maxDepth=4, maxBins=100): """ Train a DecisionTreeModel for regression. @@ -173,42 +181,14 @@ def trainRegressor(data, categoricalFeaturesInfo={}, :param maxBins: Number of bins used for finding splits at each node. :return: DecisionTreeModel """ - return DecisionTree.train(data, "regression", 0, - categoricalFeaturesInfo, - impurity, maxDepth, maxBins) - - @staticmethod - def train(data, algo, numClasses, categoricalFeaturesInfo, - impurity, maxDepth, maxBins=100): - """ - Train a DecisionTreeModel for classification or regression. - - :param data: Training data: RDD of LabeledPoint. - For classification, labels are integers - {0,1,...,numClasses}. - For regression, labels are real numbers. - :param algo: "classification" or "regression" - :param numClasses: Number of classes for classification. - :param categoricalFeaturesInfo: Map from categorical feature index - to number of categories. - Any feature not in this map - is treated as continuous. - :param impurity: For classification: "entropy" or "gini". - For regression: "variance". - :param maxDepth: Max depth of tree. - E.g., depth 0 means 1 leaf node. - Depth 1 means 1 internal node + 2 leaf nodes. - :param maxBins: Number of bins used for finding splits at each node. - :return: DecisionTreeModel - """ sc = data.context dataBytes = _get_unmangled_labeled_point_rdd(data) categoricalFeaturesInfoJMap = \ MapConverter().convert(categoricalFeaturesInfo, sc._gateway._gateway_client) model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - dataBytes._jrdd, algo, - numClasses, categoricalFeaturesInfoJMap, + dataBytes._jrdd, "regression", + 0, categoricalFeaturesInfoJMap, impurity, maxDepth, maxBins) dataBytes.unpersist() return DecisionTreeModel(sc, model) From 75993a65173172da32bbe98751e8c0f55c17a52e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 7 Aug 2014 00:04:18 -0700 Subject: [PATCH 0282/1492] SPARK-2879 part 2 [BUILD] Use HTTPS to access Maven Central and other repos .. and use canonical repo1.maven.org Maven Central repo. (And make sure snapshots are disabled for plugins from Maven Central.) Author: Sean Owen Closes #1828 from srowen/SPARK-2879.2 and squashes the following commits: 639f495 [Sean Owen] .. and use canonical repo1.maven.org Maven Central repo. (And make sure snapshots are disabled for plugins from Maven Central.) --- pom.xml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 76bf6d8f902a8..920912353fe9c 100644 --- a/pom.xml +++ b/pom.xml @@ -146,8 +146,7 @@ central Maven Repository - - https://repo.maven.apache.org/maven2 + https://repo1.maven.org/maven2 true @@ -229,6 +228,9 @@ true + + false + From 8d1dec4fa4798bb48b8947446d306ec9ba6bddb5 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 7 Aug 2014 00:20:38 -0700 Subject: [PATCH 0283/1492] [mllib] DecisionTree Strategy parameter checks Added some checks to Strategy to print out meaningful error messages when given invalid DecisionTree parameters. CC mengxr Author: Joseph K. Bradley Closes #1821 from jkbradley/dt-robustness and squashes the following commits: 4dc449a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-robustness 7a61f7b [Joseph K. Bradley] Added some checks to Strategy to print out meaningful error messages when given invalid DecisionTree parameters --- .../spark/mllib/tree/DecisionTree.scala | 10 ++++-- .../mllib/tree/configuration/Strategy.scala | 31 ++++++++++++++++++- 2 files changed, 38 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index c8a865659682f..bb50f07be5d7b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -44,6 +44,8 @@ import org.apache.spark.util.random.XORShiftRandom @Experimental class DecisionTree (private val strategy: Strategy) extends Serializable with Logging { + strategy.assertValid() + /** * Method to train a decision tree model over an RDD * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] @@ -1465,10 +1467,14 @@ object DecisionTree extends Serializable with Logging { /* - * Ensure #bins is always greater than the categories. For multiclass classification, - * #bins should be greater than 2^(maxCategories - 1) - 1. + * Ensure numBins is always greater than the categories. For multiclass classification, + * numBins should be greater than 2^(maxCategories - 1) - 1. * It's a limitation of the current implementation but a reasonable trade-off since features * with large number of categories get favored over continuous features. + * + * This needs to be checked here instead of in Strategy since numBins can be determined + * by the number of training examples. + * TODO: Allow this case, where we simply will know nothing about some categories. */ if (strategy.categoricalFeaturesInfo.size > 0) { val maxCategoriesForFeatures = strategy.categoricalFeaturesInfo.maxBy(_._2)._2 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 4ee4bcd0bcbc7..f31a503608b22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.configuration import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.mllib.tree.impurity.{Variance, Entropy, Gini, Impurity} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ @@ -90,4 +90,33 @@ class Strategy ( categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) } + private[tree] def assertValid(): Unit = { + algo match { + case Classification => + require(numClassesForClassification >= 2, + s"DecisionTree Strategy for Classification must have numClassesForClassification >= 2," + + s" but numClassesForClassification = $numClassesForClassification.") + require(Set(Gini, Entropy).contains(impurity), + s"DecisionTree Strategy given invalid impurity for Classification: $impurity." + + s" Valid settings: Gini, Entropy") + case Regression => + require(impurity == Variance, + s"DecisionTree Strategy given invalid impurity for Regression: $impurity." + + s" Valid settings: Variance") + case _ => + throw new IllegalArgumentException( + s"DecisionTree Strategy given invalid algo parameter: $algo." + + s" Valid settings are: Classification, Regression.") + } + require(maxDepth >= 0, s"DecisionTree Strategy given invalid maxDepth parameter: $maxDepth." + + s" Valid values are integers >= 0.") + require(maxBins >= 2, s"DecisionTree Strategy given invalid maxBins parameter: $maxBins." + + s" Valid values are integers >= 2.") + categoricalFeaturesInfo.foreach { case (feature, arity) => + require(arity >= 2, + s"DecisionTree Strategy given invalid categoricalFeaturesInfo setting:" + + s" feature $feature has $arity categories. The number of categories should be >= 2.") + } + } + } From b9e9e53773a618e4322b845c40deae22f2ba52ac Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 7 Aug 2014 11:28:12 -0700 Subject: [PATCH 0284/1492] [SPARK-2852][MLLIB] Separate model from IDF/StandardScaler algorithms This is part of SPARK-2828: 1. separate IDF model from IDF algorithm (which generates a model) 2. separate StandardScaler model from StandardScaler CC: dbtsai Author: Xiangrui Meng Closes #1814 from mengxr/feature-api-update and squashes the following commits: 40d863b [Xiangrui Meng] move mean and variance to model 48a0fff [Xiangrui Meng] separate Model from StandardScaler algorithm 89f3486 [Xiangrui Meng] update IDF to separate Model from Algorithm --- .../org/apache/spark/mllib/feature/IDF.scala | 130 ++++++++---------- .../spark/mllib/feature/StandardScaler.scala | 58 ++++---- .../apache/spark/mllib/feature/IDFSuite.scala | 12 +- .../mllib/feature/StandardScalerSuite.scala | 50 +++---- 4 files changed, 121 insertions(+), 129 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index 7ed611a857acc..d40d5553c1d21 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -36,87 +36,25 @@ class IDF { // TODO: Allow different IDF formulations. - private var brzIdf: BDV[Double] = _ - /** * Computes the inverse document frequency. * @param dataset an RDD of term frequency vectors */ - def fit(dataset: RDD[Vector]): this.type = { - brzIdf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator)( + def fit(dataset: RDD[Vector]): IDFModel = { + val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator)( seqOp = (df, v) => df.add(v), combOp = (df1, df2) => df1.merge(df2) ).idf() - this + new IDFModel(idf) } /** * Computes the inverse document frequency. * @param dataset a JavaRDD of term frequency vectors */ - def fit(dataset: JavaRDD[Vector]): this.type = { + def fit(dataset: JavaRDD[Vector]): IDFModel = { fit(dataset.rdd) } - - /** - * Transforms term frequency (TF) vectors to TF-IDF vectors. - * @param dataset an RDD of term frequency vectors - * @return an RDD of TF-IDF vectors - */ - def transform(dataset: RDD[Vector]): RDD[Vector] = { - if (!initialized) { - throw new IllegalStateException("Haven't learned IDF yet. Call fit first.") - } - val theIdf = brzIdf - val bcIdf = dataset.context.broadcast(theIdf) - dataset.mapPartitions { iter => - val thisIdf = bcIdf.value - iter.map { v => - val n = v.size - v match { - case sv: SparseVector => - val nnz = sv.indices.size - val newValues = new Array[Double](nnz) - var k = 0 - while (k < nnz) { - newValues(k) = sv.values(k) * thisIdf(sv.indices(k)) - k += 1 - } - Vectors.sparse(n, sv.indices, newValues) - case dv: DenseVector => - val newValues = new Array[Double](n) - var j = 0 - while (j < n) { - newValues(j) = dv.values(j) * thisIdf(j) - j += 1 - } - Vectors.dense(newValues) - case other => - throw new UnsupportedOperationException( - s"Only sparse and dense vectors are supported but got ${other.getClass}.") - } - } - } - } - - /** - * Transforms term frequency (TF) vectors to TF-IDF vectors (Java version). - * @param dataset a JavaRDD of term frequency vectors - * @return a JavaRDD of TF-IDF vectors - */ - def transform(dataset: JavaRDD[Vector]): JavaRDD[Vector] = { - transform(dataset.rdd).toJavaRDD() - } - - /** Returns the IDF vector. */ - def idf(): Vector = { - if (!initialized) { - throw new IllegalStateException("Haven't learned IDF yet. Call fit first.") - } - Vectors.fromBreeze(brzIdf) - } - - private def initialized: Boolean = brzIdf != null } private object IDF { @@ -177,18 +115,72 @@ private object IDF { private def isEmpty: Boolean = m == 0L /** Returns the current IDF vector. */ - def idf(): BDV[Double] = { + def idf(): Vector = { if (isEmpty) { throw new IllegalStateException("Haven't seen any document yet.") } val n = df.length - val inv = BDV.zeros[Double](n) + val inv = new Array[Double](n) var j = 0 while (j < n) { inv(j) = math.log((m + 1.0)/ (df(j) + 1.0)) j += 1 } - inv + Vectors.dense(inv) } } } + +/** + * :: Experimental :: + * Represents an IDF model that can transform term frequency vectors. + */ +@Experimental +class IDFModel private[mllib] (val idf: Vector) extends Serializable { + + /** + * Transforms term frequency (TF) vectors to TF-IDF vectors. + * @param dataset an RDD of term frequency vectors + * @return an RDD of TF-IDF vectors + */ + def transform(dataset: RDD[Vector]): RDD[Vector] = { + val bcIdf = dataset.context.broadcast(idf) + dataset.mapPartitions { iter => + val thisIdf = bcIdf.value + iter.map { v => + val n = v.size + v match { + case sv: SparseVector => + val nnz = sv.indices.size + val newValues = new Array[Double](nnz) + var k = 0 + while (k < nnz) { + newValues(k) = sv.values(k) * thisIdf(sv.indices(k)) + k += 1 + } + Vectors.sparse(n, sv.indices, newValues) + case dv: DenseVector => + val newValues = new Array[Double](n) + var j = 0 + while (j < n) { + newValues(j) = dv.values(j) * thisIdf(j) + j += 1 + } + Vectors.dense(newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } + } + } + + /** + * Transforms term frequency (TF) vectors to TF-IDF vectors (Java version). + * @param dataset a JavaRDD of term frequency vectors + * @return a JavaRDD of TF-IDF vectors + */ + def transform(dataset: JavaRDD[Vector]): JavaRDD[Vector] = { + transform(dataset.rdd).toJavaRDD() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index e6c9f8f67df63..4dfd1f0ab8134 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -17,8 +17,9 @@ package org.apache.spark.mllib.feature -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} +import org.apache.spark.Logging import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.rdd.RDDFunctions._ @@ -35,37 +36,55 @@ import org.apache.spark.rdd.RDD * @param withStd True by default. Scales the data to unit standard deviation. */ @Experimental -class StandardScaler(withMean: Boolean, withStd: Boolean) extends VectorTransformer { +class StandardScaler(withMean: Boolean, withStd: Boolean) extends Logging { def this() = this(false, true) - require(withMean || withStd, s"withMean and withStd both equal to false. Doing nothing.") - - private var mean: BV[Double] = _ - private var factor: BV[Double] = _ + if (!(withMean || withStd)) { + logWarning("Both withMean and withStd are false. The model does nothing.") + } /** * Computes the mean and variance and stores as a model to be used for later scaling. * * @param data The data used to compute the mean and variance to build the transformation model. - * @return This StandardScalar object. + * @return a StandardScalarModel */ - def fit(data: RDD[Vector]): this.type = { + def fit(data: RDD[Vector]): StandardScalerModel = { + // TODO: skip computation if both withMean and withStd are false val summary = data.treeAggregate(new MultivariateOnlineSummarizer)( (aggregator, data) => aggregator.add(data), (aggregator1, aggregator2) => aggregator1.merge(aggregator2)) + new StandardScalerModel(withMean, withStd, summary.mean, summary.variance) + } +} - mean = summary.mean.toBreeze - factor = summary.variance.toBreeze - require(mean.length == factor.length) +/** + * :: Experimental :: + * Represents a StandardScaler model that can transform vectors. + * + * @param withMean whether to center the data before scaling + * @param withStd whether to scale the data to have unit standard deviation + * @param mean column mean values + * @param variance column variance values + */ +@Experimental +class StandardScalerModel private[mllib] ( + val withMean: Boolean, + val withStd: Boolean, + val mean: Vector, + val variance: Vector) extends VectorTransformer { + + require(mean.size == variance.size) + private lazy val factor: BDV[Double] = { + val f = BDV.zeros[Double](variance.size) var i = 0 - while (i < factor.length) { - factor(i) = if (factor(i) != 0.0) 1.0 / math.sqrt(factor(i)) else 0.0 + while (i < f.size) { + f(i) = if (variance(i) != 0.0) 1.0 / math.sqrt(variance(i)) else 0.0 i += 1 } - - this + f } /** @@ -76,13 +95,7 @@ class StandardScaler(withMean: Boolean, withStd: Boolean) extends VectorTransfor * for the column with zero variance. */ override def transform(vector: Vector): Vector = { - if (mean == null || factor == null) { - throw new IllegalStateException( - "Haven't learned column summary statistics yet. Call fit first.") - } - - require(vector.size == mean.length) - + require(mean.size == vector.size) if (withMean) { vector.toBreeze match { case dv: BDV[Double] => @@ -115,5 +128,4 @@ class StandardScaler(withMean: Boolean, withStd: Boolean) extends VectorTransfor vector } } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala index 78a2804ff204b..53d9c0c640b98 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala @@ -36,18 +36,12 @@ class IDFSuite extends FunSuite with LocalSparkContext { val m = localTermFrequencies.size val termFrequencies = sc.parallelize(localTermFrequencies, 2) val idf = new IDF - intercept[IllegalStateException] { - idf.idf() - } - intercept[IllegalStateException] { - idf.transform(termFrequencies) - } - idf.fit(termFrequencies) + val model = idf.fit(termFrequencies) val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => math.log((m.toDouble + 1.0) / (x + 1.0)) }) - assert(idf.idf() ~== expected absTol 1e-12) - val tfidf = idf.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() + assert(model.idf ~== expected absTol 1e-12) + val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() assert(tfidf.size === 3) val tfidf0 = tfidf(0L).asInstanceOf[SparseVector] assert(tfidf0.indices === Array(1, 3)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala index 5a9be923a8625..e217b93cebbdb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/StandardScalerSuite.scala @@ -50,23 +50,17 @@ class StandardScalerSuite extends FunSuite with LocalSparkContext { val standardizer2 = new StandardScaler() val standardizer3 = new StandardScaler(withMean = true, withStd = false) - withClue("Using a standardizer before fitting the model should throw exception.") { - intercept[IllegalStateException] { - data.map(standardizer1.transform) - } - } - - standardizer1.fit(dataRDD) - standardizer2.fit(dataRDD) - standardizer3.fit(dataRDD) + val model1 = standardizer1.fit(dataRDD) + val model2 = standardizer2.fit(dataRDD) + val model3 = standardizer3.fit(dataRDD) - val data1 = data.map(standardizer1.transform) - val data2 = data.map(standardizer2.transform) - val data3 = data.map(standardizer3.transform) + val data1 = data.map(model1.transform) + val data2 = data.map(model2.transform) + val data3 = data.map(model3.transform) - val data1RDD = standardizer1.transform(dataRDD) - val data2RDD = standardizer2.transform(dataRDD) - val data3RDD = standardizer3.transform(dataRDD) + val data1RDD = model1.transform(dataRDD) + val data2RDD = model2.transform(dataRDD) + val data3RDD = model3.transform(dataRDD) val summary = computeSummary(dataRDD) val summary1 = computeSummary(data1RDD) @@ -129,25 +123,25 @@ class StandardScalerSuite extends FunSuite with LocalSparkContext { val standardizer2 = new StandardScaler() val standardizer3 = new StandardScaler(withMean = true, withStd = false) - standardizer1.fit(dataRDD) - standardizer2.fit(dataRDD) - standardizer3.fit(dataRDD) + val model1 = standardizer1.fit(dataRDD) + val model2 = standardizer2.fit(dataRDD) + val model3 = standardizer3.fit(dataRDD) - val data2 = data.map(standardizer2.transform) + val data2 = data.map(model2.transform) withClue("Standardization with mean can not be applied on sparse input.") { intercept[IllegalArgumentException] { - data.map(standardizer1.transform) + data.map(model1.transform) } } withClue("Standardization with mean can not be applied on sparse input.") { intercept[IllegalArgumentException] { - data.map(standardizer3.transform) + data.map(model3.transform) } } - val data2RDD = standardizer2.transform(dataRDD) + val data2RDD = model2.transform(dataRDD) val summary2 = computeSummary(data2RDD) @@ -181,13 +175,13 @@ class StandardScalerSuite extends FunSuite with LocalSparkContext { val standardizer2 = new StandardScaler(withMean = true, withStd = false) val standardizer3 = new StandardScaler(withMean = false, withStd = true) - standardizer1.fit(dataRDD) - standardizer2.fit(dataRDD) - standardizer3.fit(dataRDD) + val model1 = standardizer1.fit(dataRDD) + val model2 = standardizer2.fit(dataRDD) + val model3 = standardizer3.fit(dataRDD) - val data1 = data.map(standardizer1.transform) - val data2 = data.map(standardizer2.transform) - val data3 = data.map(standardizer3.transform) + val data1 = data.map(model1.transform) + val data2 = data.map(model2.transform) + val data3 = data.map(model3.transform) assert(data1.forall(_.toArray.forall(_ == 0.0)), "The variance is zero, so the transformed result should be 0.0") From 80ec5bad1311651fe56e1d5178090dc63753233b Mon Sep 17 00:00:00 2001 From: Oleg Danilov Date: Thu, 7 Aug 2014 15:48:44 -0700 Subject: [PATCH 0285/1492] SPARK-2905 Fixed path sbin => bin Author: Oleg Danilov Closes #1835 from dosoft/SPARK-2905 and squashes the following commits: 4df423c [Oleg Danilov] SPARK-2905 Fixed path sbin => bin --- bin/spark-sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/spark-sql b/bin/spark-sql index 61ebd8ab6dec8..7813ccc361415 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -29,7 +29,7 @@ CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" FWDIR="$(cd `dirname $0`/..; pwd)" function usage { - echo "Usage: ./sbin/spark-sql [options] [cli option]" + echo "Usage: ./bin/spark-sql [options] [cli option]" pattern="usage" pattern+="\|Spark assembly has been built with Hive" pattern+="\|NOTE: SPARK_PREPEND_CLASSES is set" From 32096c2aed9978cfb9a904b4f56bb61800d17e9e Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 7 Aug 2014 16:24:22 -0700 Subject: [PATCH 0286/1492] SPARK-2899 Doc generation is back to working in new SBT Build. The reason for this bug was introduciton of OldDeps project. It had to be excluded to prevent unidocs from trying to put it on "docs compile" classpath. Author: Prashant Sharma Closes #1830 from ScrapCodes/doc-fix and squashes the following commits: e5d52e6 [Prashant Sharma] SPARK-2899 Doc generation is back to working in new SBT Build. --- project/SparkBuild.scala | 60 ++++++++++++++++++++++------------------ project/plugins.sbt | 2 +- 2 files changed, 34 insertions(+), 28 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ed587783d5606..63a285b81a60c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,11 +30,11 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile - val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, spark, + val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, sql, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "spark", "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", + "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = @@ -44,8 +44,9 @@ object BuildCommons { val assemblyProjects@Seq(assembly, examples) = Seq("assembly", "examples") .map(ProjectRef(buildLocation, _)) - val tools = "tools" - + val tools = ProjectRef(buildLocation, "tools") + // Root project. + val spark = ProjectRef(buildLocation, "spark") val sparkHome = buildLocation } @@ -126,26 +127,6 @@ object SparkBuild extends PomBuild { publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn ) - /** Following project only exists to pull previous artifacts of Spark for generating - Mima ignores. For more information see: SPARK 2071 */ - lazy val oldDeps = Project("oldDeps", file("dev"), settings = oldDepsSettings) - - def versionArtifact(id: String): Option[sbt.ModuleID] = { - val fullId = id + "_2.10" - Some("org.apache.spark" % fullId % "1.0.0") - } - - def oldDepsSettings() = Defaults.defaultSettings ++ Seq( - name := "old-deps", - scalaVersion := "2.10.4", - retrieveManaged := true, - retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", - libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", - "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-twitter", - "spark-streaming", "spark-mllib", "spark-bagel", "spark-graphx", - "spark-core").map(versionArtifact(_).get intransitive()) - ) - def enable(settings: Seq[Setting[_]])(projectRef: ProjectRef) = { val existingSettings = projectsMap.getOrElse(projectRef.project, Seq[Setting[_]]()) projectsMap += (projectRef.project -> (existingSettings ++ settings)) @@ -184,7 +165,7 @@ object SparkBuild extends PomBuild { super.projectDefinitions(baseDirectory).map { x => if (projectsMap.exists(_._1 == x.id)) x.settings(projectsMap(x.id): _*) else x.settings(Seq[Setting[_]](): _*) - } ++ Seq[Project](oldDeps) + } ++ Seq[Project](OldDeps.project) } } @@ -193,6 +174,31 @@ object Flume { lazy val settings = sbtavro.SbtAvro.avroSettings } +/** + * Following project only exists to pull previous artifacts of Spark for generating + * Mima ignores. For more information see: SPARK 2071 + */ +object OldDeps { + + lazy val project = Project("oldDeps", file("dev"), settings = oldDepsSettings) + + def versionArtifact(id: String): Option[sbt.ModuleID] = { + val fullId = id + "_2.10" + Some("org.apache.spark" % fullId % "1.0.0") + } + + def oldDepsSettings() = Defaults.defaultSettings ++ Seq( + name := "old-deps", + scalaVersion := "2.10.4", + retrieveManaged := true, + retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", + libraryDependencies := Seq("spark-streaming-mqtt", "spark-streaming-zeromq", + "spark-streaming-flume", "spark-streaming-kafka", "spark-streaming-twitter", + "spark-streaming", "spark-mllib", "spark-bagel", "spark-graphx", + "spark-core").map(versionArtifact(_).get intransitive()) + ) +} + object Catalyst { lazy val settings = Seq( addCompilerPlugin("org.scalamacros" % "paradise" % "2.0.1" cross CrossVersion.full), @@ -285,9 +291,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(repl, examples, tools, catalyst, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, yarn, yarnAlpha), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(repl, bagel, graphx, examples, tools, catalyst, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, bagel, graphx, examples, tools, catalyst, yarn, yarnAlpha), // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { diff --git a/project/plugins.sbt b/project/plugins.sbt index 06d18e193076e..2a61f56c2ea60 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -23,6 +23,6 @@ addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") -addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") +addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.1") addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") From 6906b69cf568015f20c7d7c77cbcba650e5431a9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 7 Aug 2014 18:04:49 -0700 Subject: [PATCH 0287/1492] SPARK-2787: Make sort-based shuffle write files directly when there's no sorting/aggregation and # partitions is small As described in https://issues.apache.org/jira/browse/SPARK-2787, right now sort-based shuffle is more expensive than hash-based for map operations that do no partial aggregation or sorting, such as groupByKey. This is because it has to serialize each data item twice (once when spilling to intermediate files, and then again when merging these files object-by-object). This patch adds a code path to just write separate files directly if the # of output partitions is small, and concatenate them at the end to produce a sorted file. On the unit test side, I added some tests that force or don't force this bypass path to be used, and checked that our tests for other features (e.g. all the operations) cover both cases. Author: Matei Zaharia Closes #1799 from mateiz/SPARK-2787 and squashes the following commits: 88cf26a [Matei Zaharia] Fix rebase 10233af [Matei Zaharia] Review comments 398cb95 [Matei Zaharia] Fix looking up shuffle manager in conf ca3efd9 [Matei Zaharia] Add docs for shuffle manager properties, and allow short names for them d0ae3c5 [Matei Zaharia] Fix some comments 90d084f [Matei Zaharia] Add code path to bypass merge-sort in ExternalSorter, and tests 31e5d7c [Matei Zaharia] Move existing logic for writing partitioned files into ExternalSorter --- .../scala/org/apache/spark/SparkEnv.scala | 27 +- .../shuffle/hash/HashShuffleReader.scala | 2 +- .../shuffle/sort/SortShuffleWriter.scala | 80 ++---- .../util/collection/ExternalSorter.scala | 233 +++++++++++++++--- .../util/collection/ExternalSorterSuite.scala | 165 +++++++++++-- docs/configuration.md | 18 ++ 6 files changed, 407 insertions(+), 118 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 9d4edeb6d96cf..22d8d1cb1ddcf 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,11 +156,9 @@ object SparkEnv extends Logging { conf.set("spark.driver.port", boundPort.toString) } - // Create an instance of the class named by the given Java system property, or by - // defaultClassName if the property is not set, and return it as a T - def instantiateClass[T](propertyName: String, defaultClassName: String): T = { - val name = conf.get(propertyName, defaultClassName) - val cls = Class.forName(name, true, Utils.getContextOrSparkClassLoader) + // Create an instance of the class with the given name, possibly initializing it with our conf + def instantiateClass[T](className: String): T = { + val cls = Class.forName(className, true, Utils.getContextOrSparkClassLoader) // Look for a constructor taking a SparkConf and a boolean isDriver, then one taking just // SparkConf, then one taking no arguments try { @@ -178,11 +176,17 @@ object SparkEnv extends Logging { } } - val serializer = instantiateClass[Serializer]( + // Create an instance of the class named by the given SparkConf property, or defaultClassName + // if the property is not set, possibly initializing it with our conf + def instantiateClassFromConf[T](propertyName: String, defaultClassName: String): T = { + instantiateClass[T](conf.get(propertyName, defaultClassName)) + } + + val serializer = instantiateClassFromConf[Serializer]( "spark.serializer", "org.apache.spark.serializer.JavaSerializer") logDebug(s"Using serializer: ${serializer.getClass}") - val closureSerializer = instantiateClass[Serializer]( + val closureSerializer = instantiateClassFromConf[Serializer]( "spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer") def registerOrLookup(name: String, newActor: => Actor): ActorRef = { @@ -246,8 +250,13 @@ object SparkEnv extends Logging { "." } - val shuffleManager = instantiateClass[ShuffleManager]( - "spark.shuffle.manager", "org.apache.spark.shuffle.hash.HashShuffleManager") + // Let the user specify short names for shuffle managers + val shortShuffleMgrNames = Map( + "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", + "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") + val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") + val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) + val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) val shuffleMemoryManager = new ShuffleMemoryManager(conf) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 7c9dc8e5f88ef..88a5f1e5ddf58 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -58,7 +58,7 @@ private[spark] class HashShuffleReader[K, C]( // Create an ExternalSorter to sort the data. Note that if spark.shuffle.spill is disabled, // the ExternalSorter won't spill to disk. val sorter = new ExternalSorter[K, C, C](ordering = Some(keyOrd), serializer = Some(ser)) - sorter.write(aggregatedIter) + sorter.insertAll(aggregatedIter) context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled sorter.iterator diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index e54e6383d2ccc..22f656fa371ea 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -44,6 +44,7 @@ private[spark] class SortShuffleWriter[K, V, C]( private var sorter: ExternalSorter[K, V, _] = null private var outputFile: File = null + private var indexFile: File = null // Are we in the process of stopping? Because map tasks can call stop() with success = true // and then call stop() with success = false if they get an exception, we want to make sure @@ -57,78 +58,36 @@ private[spark] class SortShuffleWriter[K, V, C]( /** Write a bunch of records to this task's output */ override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { - // Get an iterator with the elements for each partition ID - val partitions: Iterator[(Int, Iterator[Product2[K, _]])] = { - if (dep.mapSideCombine) { - if (!dep.aggregator.isDefined) { - throw new IllegalStateException("Aggregator is empty for map-side combine") - } - sorter = new ExternalSorter[K, V, C]( - dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) - sorter.write(records) - sorter.partitionedIterator - } else { - // In this case we pass neither an aggregator nor an ordering to the sorter, because we - // don't care whether the keys get sorted in each partition; that will be done on the - // reduce side if the operation being run is sortByKey. - sorter = new ExternalSorter[K, V, V]( - None, Some(dep.partitioner), None, dep.serializer) - sorter.write(records) - sorter.partitionedIterator + if (dep.mapSideCombine) { + if (!dep.aggregator.isDefined) { + throw new IllegalStateException("Aggregator is empty for map-side combine") } + sorter = new ExternalSorter[K, V, C]( + dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) + sorter.insertAll(records) + } else { + // In this case we pass neither an aggregator nor an ordering to the sorter, because we don't + // care whether the keys get sorted in each partition; that will be done on the reduce side + // if the operation being run is sortByKey. + sorter = new ExternalSorter[K, V, V]( + None, Some(dep.partitioner), None, dep.serializer) + sorter.insertAll(records) } // Create a single shuffle file with reduce ID 0 that we'll write all results to. We'll later // serve different ranges of this file using an index file that we create at the end. val blockId = ShuffleBlockId(dep.shuffleId, mapId, 0) - outputFile = blockManager.diskBlockManager.getFile(blockId) - - // Track location of each range in the output file - val offsets = new Array[Long](numPartitions + 1) - val lengths = new Array[Long](numPartitions) - - for ((id, elements) <- partitions) { - if (elements.hasNext) { - val writer = blockManager.getDiskWriter(blockId, outputFile, ser, fileBufferSize, - writeMetrics) - for (elem <- elements) { - writer.write(elem) - } - writer.commitAndClose() - val segment = writer.fileSegment() - offsets(id + 1) = segment.offset + segment.length - lengths(id) = segment.length - } else { - // The partition is empty; don't create a new writer to avoid writing headers, etc - offsets(id + 1) = offsets(id) - } - } - - context.taskMetrics.memoryBytesSpilled += sorter.memoryBytesSpilled - context.taskMetrics.diskBytesSpilled += sorter.diskBytesSpilled - // Write an index file with the offsets of each block, plus a final offset at the end for the - // end of the output file. This will be used by SortShuffleManager.getBlockLocation to figure - // out where each block begins and ends. + outputFile = blockManager.diskBlockManager.getFile(blockId) + indexFile = blockManager.diskBlockManager.getFile(blockId.name + ".index") - val diskBlockManager = blockManager.diskBlockManager - val indexFile = diskBlockManager.getFile(blockId.name + ".index") - val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) - try { - var i = 0 - while (i < numPartitions + 1) { - out.writeLong(offsets(i)) - i += 1 - } - } finally { - out.close() - } + val partitionLengths = sorter.writePartitionedFile(blockId, context) // Register our map output with the ShuffleBlockManager, which handles cleaning it over time blockManager.shuffleBlockManager.addCompletedMap(dep.shuffleId, mapId, numPartitions) mapStatus = new MapStatus(blockManager.blockManagerId, - lengths.map(MapOutputTracker.compressSize)) + partitionLengths.map(MapOutputTracker.compressSize)) } /** Close this writer, passing along whether the map completed */ @@ -145,6 +104,9 @@ private[spark] class SortShuffleWriter[K, V, C]( if (outputFile != null) { outputFile.delete() } + if (indexFile != null) { + indexFile.delete() + } return None } } finally { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index eb4849ebc6e52..b73d5e0cf1714 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -25,10 +25,10 @@ import scala.collection.mutable import com.google.common.io.ByteStreams -import org.apache.spark.{Aggregator, SparkEnv, Logging, Partitioner} +import org.apache.spark._ import org.apache.spark.serializer.{DeserializationStream, Serializer} -import org.apache.spark.storage.BlockId import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.storage.{BlockObjectWriter, BlockId} /** * Sorts and potentially merges a number of key-value pairs of type (K, V) to produce key-combiner @@ -67,6 +67,13 @@ import org.apache.spark.executor.ShuffleWriteMetrics * for equality to merge values. * * - Users are expected to call stop() at the end to delete all the intermediate files. + * + * As a special case, if no Ordering and no Aggregator is given, and the number of partitions is + * less than spark.shuffle.sort.bypassMergeThreshold, we bypass the merge-sort and just write to + * separate files for each partition each time we spill, similar to the HashShuffleWriter. We can + * then concatenate these files to produce a single sorted file, without having to serialize and + * de-serialize each item twice (as is needed during the merge). This speeds up the map side of + * groupBy, sort, etc operations since they do no partial aggregation. */ private[spark] class ExternalSorter[K, V, C]( aggregator: Option[Aggregator[K, V, C]] = None, @@ -124,6 +131,18 @@ private[spark] class ExternalSorter[K, V, C]( // How much of the shared memory pool this collection has claimed private var myMemoryThreshold = 0L + // If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't need + // local aggregation and sorting, write numPartitions files directly and just concatenate them + // at the end. This avoids doing serialization and deserialization twice to merge together the + // spilled files, which would happen with the normal code path. The downside is having multiple + // files open at a time and thus more memory allocated to buffers. + private val bypassMergeThreshold = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + private val bypassMergeSort = + (numPartitions <= bypassMergeThreshold && aggregator.isEmpty && ordering.isEmpty) + + // Array of file writers for each partition, used if bypassMergeSort is true and we've spilled + private var partitionWriters: Array[BlockObjectWriter] = null + // A comparator for keys K that orders them within a partition to allow aggregation or sorting. // Can be a partial ordering by hash code if a total ordering is not provided through by the // user. (A partial ordering means that equal keys have comparator.compare(k, k) = 0, but some @@ -137,7 +156,14 @@ private[spark] class ExternalSorter[K, V, C]( } }) - // A comparator for (Int, K) elements that orders them by partition and then possibly by key + // A comparator for (Int, K) pairs that orders them by only their partition ID + private val partitionComparator: Comparator[(Int, K)] = new Comparator[(Int, K)] { + override def compare(a: (Int, K), b: (Int, K)): Int = { + a._1 - b._1 + } + } + + // A comparator that orders (Int, K) pairs by partition ID and then possibly by key private val partitionKeyComparator: Comparator[(Int, K)] = { if (ordering.isDefined || aggregator.isDefined) { // Sort by partition ID then key comparator @@ -153,11 +179,7 @@ private[spark] class ExternalSorter[K, V, C]( } } else { // Just sort it by partition ID - new Comparator[(Int, K)] { - override def compare(a: (Int, K), b: (Int, K)): Int = { - a._1 - b._1 - } - } + partitionComparator } } @@ -171,7 +193,7 @@ private[spark] class ExternalSorter[K, V, C]( elementsPerPartition: Array[Long]) private val spills = new ArrayBuffer[SpilledFile] - def write(records: Iterator[_ <: Product2[K, V]]): Unit = { + def insertAll(records: Iterator[_ <: Product2[K, V]]): Unit = { // TODO: stop combining if we find that the reduction factor isn't high val shouldCombine = aggregator.isDefined @@ -242,6 +264,38 @@ private[spark] class ExternalSorter[K, V, C]( val threadId = Thread.currentThread().getId logInfo("Thread %d spilling in-memory batch of %d MB to disk (%d spill%s so far)" .format(threadId, memorySize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) + + if (bypassMergeSort) { + spillToPartitionFiles(collection) + } else { + spillToMergeableFile(collection) + } + + if (usingMap) { + map = new SizeTrackingAppendOnlyMap[(Int, K), C] + } else { + buffer = new SizeTrackingPairBuffer[(Int, K), C] + } + + // Release our memory back to the shuffle pool so that other threads can grab it + shuffleMemoryManager.release(myMemoryThreshold) + myMemoryThreshold = 0 + + _memoryBytesSpilled += memorySize + } + + /** + * Spill our in-memory collection to a sorted file that we can merge later (normal code path). + * We add this file into spilledFiles to find it later. + * + * Alternatively, if bypassMergeSort is true, we spill to separate files for each partition. + * See spillToPartitionedFiles() for that code path. + * + * @param collection whichever collection we're using (map or buffer) + */ + private def spillToMergeableFile(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { + assert(!bypassMergeSort) + val (blockId, file) = diskBlockManager.createTempBlock() curWriteMetrics = new ShuffleWriteMetrics() var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) @@ -304,18 +358,36 @@ private[spark] class ExternalSorter[K, V, C]( } } - if (usingMap) { - map = new SizeTrackingAppendOnlyMap[(Int, K), C] - } else { - buffer = new SizeTrackingPairBuffer[(Int, K), C] - } + spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition)) + } - // Release our memory back to the shuffle pool so that other threads can grab it - shuffleMemoryManager.release(myMemoryThreshold) - myMemoryThreshold = 0 + /** + * Spill our in-memory collection to separate files, one for each partition. This is used when + * there's no aggregator and ordering and the number of partitions is small, because it allows + * writePartitionedFile to just concatenate files without deserializing data. + * + * @param collection whichever collection we're using (map or buffer) + */ + private def spillToPartitionFiles(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { + assert(bypassMergeSort) + + // Create our file writers if we haven't done so yet + if (partitionWriters == null) { + curWriteMetrics = new ShuffleWriteMetrics() + partitionWriters = Array.fill(numPartitions) { + val (blockId, file) = diskBlockManager.createTempBlock() + blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics).open() + } + } - spills.append(SpilledFile(file, blockId, batchSizes.toArray, elementsPerPartition)) - _memoryBytesSpilled += memorySize + val it = collection.iterator // No need to sort stuff, just write each element out + while (it.hasNext) { + val elem = it.next() + val partitionId = elem._1._1 + val key = elem._1._2 + val value = elem._2 + partitionWriters(partitionId).write((key, value)) + } } /** @@ -479,7 +551,6 @@ private[spark] class ExternalSorter[K, V, C]( skipToNextPartition() - // Intermediate file and deserializer streams that read from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams var fileStream: FileInputStream = null @@ -619,23 +690,25 @@ private[spark] class ExternalSorter[K, V, C]( def partitionedIterator: Iterator[(Int, Iterator[Product2[K, C]])] = { val usingMap = aggregator.isDefined val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer - if (spills.isEmpty) { + if (spills.isEmpty && partitionWriters == null) { // Special case: if we have only in-memory data, we don't need to merge streams, and perhaps // we don't even need to sort by anything other than partition ID if (!ordering.isDefined) { - // The user isn't requested sorted keys, so only sort by partition ID, not key - val partitionComparator = new Comparator[(Int, K)] { - override def compare(a: (Int, K), b: (Int, K)): Int = { - a._1 - b._1 - } - } + // The user hasn't requested sorted keys, so only sort by partition ID, not key groupByPartition(collection.destructiveSortedIterator(partitionComparator)) } else { // We do need to sort by both partition ID and key groupByPartition(collection.destructiveSortedIterator(partitionKeyComparator)) } + } else if (bypassMergeSort) { + // Read data from each partition file and merge it together with the data in memory; + // note that there's no ordering or aggregator in this case -- we just partition objects + val collIter = groupByPartition(collection.destructiveSortedIterator(partitionComparator)) + collIter.map { case (partitionId, values) => + (partitionId, values ++ readPartitionFile(partitionWriters(partitionId))) + } } else { - // General case: merge spilled and in-memory data + // Merge spilled and in-memory data merge(spills, collection.destructiveSortedIterator(partitionKeyComparator)) } } @@ -645,9 +718,113 @@ private[spark] class ExternalSorter[K, V, C]( */ def iterator: Iterator[Product2[K, C]] = partitionedIterator.flatMap(pair => pair._2) + /** + * Write all the data added into this ExternalSorter into a file in the disk store, creating + * an .index file for it as well with the offsets of each partition. This is called by the + * SortShuffleWriter and can go through an efficient path of just concatenating binary files + * if we decided to avoid merge-sorting. + * + * @param blockId block ID to write to. The index file will be blockId.name + ".index". + * @param context a TaskContext for a running Spark task, for us to update shuffle metrics. + * @return array of lengths, in bytes, of each partition of the file (used by map output tracker) + */ + def writePartitionedFile(blockId: BlockId, context: TaskContext): Array[Long] = { + val outputFile = blockManager.diskBlockManager.getFile(blockId) + + // Track location of each range in the output file + val offsets = new Array[Long](numPartitions + 1) + val lengths = new Array[Long](numPartitions) + + if (bypassMergeSort && partitionWriters != null) { + // We decided to write separate files for each partition, so just concatenate them. To keep + // this simple we spill out the current in-memory collection so that everything is in files. + spillToPartitionFiles(if (aggregator.isDefined) map else buffer) + partitionWriters.foreach(_.commitAndClose()) + var out: FileOutputStream = null + var in: FileInputStream = null + try { + out = new FileOutputStream(outputFile) + for (i <- 0 until numPartitions) { + val file = partitionWriters(i).fileSegment().file + in = new FileInputStream(file) + org.apache.spark.util.Utils.copyStream(in, out) + in.close() + in = null + lengths(i) = file.length() + offsets(i + 1) = offsets(i) + lengths(i) + } + } finally { + if (out != null) { + out.close() + } + if (in != null) { + in.close() + } + } + } else { + // Either we're not bypassing merge-sort or we have only in-memory data; get an iterator by + // partition and just write everything directly. + for ((id, elements) <- this.partitionedIterator) { + if (elements.hasNext) { + val writer = blockManager.getDiskWriter( + blockId, outputFile, ser, fileBufferSize, context.taskMetrics.shuffleWriteMetrics.get) + for (elem <- elements) { + writer.write(elem) + } + writer.commitAndClose() + val segment = writer.fileSegment() + offsets(id + 1) = segment.offset + segment.length + lengths(id) = segment.length + } else { + // The partition is empty; don't create a new writer to avoid writing headers, etc + offsets(id + 1) = offsets(id) + } + } + } + + context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += diskBytesSpilled + + // Write an index file with the offsets of each block, plus a final offset at the end for the + // end of the output file. This will be used by SortShuffleManager.getBlockLocation to figure + // out where each block begins and ends. + + val diskBlockManager = blockManager.diskBlockManager + val indexFile = diskBlockManager.getFile(blockId.name + ".index") + val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexFile))) + try { + var i = 0 + while (i < numPartitions + 1) { + out.writeLong(offsets(i)) + i += 1 + } + } finally { + out.close() + } + + lengths + } + + /** + * Read a partition file back as an iterator (used in our iterator method) + */ + def readPartitionFile(writer: BlockObjectWriter): Iterator[Product2[K, C]] = { + if (writer.isOpen) { + writer.commitAndClose() + } + blockManager.getLocalFromDisk(writer.blockId, ser).get.asInstanceOf[Iterator[Product2[K, C]]] + } + def stop(): Unit = { spills.foreach(s => s.file.delete()) spills.clear() + if (partitionWriters != null) { + partitionWriters.foreach { w => + w.revertPartialWritesAndClose() + diskBlockManager.getFile(w.blockId).delete() + } + partitionWriters = null + } } def memoryBytesSpilled: Long = _memoryBytesSpilled diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 57dcb4ffabac1..706faed980f31 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -19,12 +19,12 @@ package org.apache.spark.util.collection import scala.collection.mutable.ArrayBuffer -import org.scalatest.FunSuite +import org.scalatest.{PrivateMethodTester, FunSuite} import org.apache.spark._ import org.apache.spark.SparkContext._ -class ExternalSorterSuite extends FunSuite with LocalSparkContext { +class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMethodTester { private def createSparkConf(loadDefaults: Boolean): SparkConf = { val conf = new SparkConf(loadDefaults) // Make the Java serializer write a reset instruction (TC_RESET) after each object to test @@ -36,6 +36,16 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { conf } + private def assertBypassedMergeSort(sorter: ExternalSorter[_, _, _]): Unit = { + val bypassMergeSort = PrivateMethod[Boolean]('bypassMergeSort) + assert(sorter.invokePrivate(bypassMergeSort()), "sorter did not bypass merge-sort") + } + + private def assertDidNotBypassMergeSort(sorter: ExternalSorter[_, _, _]): Unit = { + val bypassMergeSort = PrivateMethod[Boolean]('bypassMergeSort) + assert(!sorter.invokePrivate(bypassMergeSort()), "sorter bypassed merge-sort") + } + test("empty data stream") { val conf = new SparkConf(false) conf.set("spark.shuffle.memoryFraction", "0.001") @@ -86,28 +96,28 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { // Both aggregator and ordering val sorter = new ExternalSorter[Int, Int, Int]( Some(agg), Some(new HashPartitioner(7)), Some(ord), None) - sorter.write(elements.iterator) + sorter.insertAll(elements.iterator) assert(sorter.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) sorter.stop() // Only aggregator val sorter2 = new ExternalSorter[Int, Int, Int]( Some(agg), Some(new HashPartitioner(7)), None, None) - sorter2.write(elements.iterator) + sorter2.insertAll(elements.iterator) assert(sorter2.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) sorter2.stop() // Only ordering val sorter3 = new ExternalSorter[Int, Int, Int]( None, Some(new HashPartitioner(7)), Some(ord), None) - sorter3.write(elements.iterator) + sorter3.insertAll(elements.iterator) assert(sorter3.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) sorter3.stop() // Neither aggregator nor ordering val sorter4 = new ExternalSorter[Int, Int, Int]( None, Some(new HashPartitioner(7)), None, None) - sorter4.write(elements.iterator) + sorter4.insertAll(elements.iterator) assert(sorter4.partitionedIterator.map(p => (p._1, p._2.toSet)).toSet === expected) sorter4.stop() } @@ -118,13 +128,37 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") sc = new SparkContext("local", "test", conf) - val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) val ord = implicitly[Ordering[Int]] val elements = Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(7)), Some(ord), None) + assertDidNotBypassMergeSort(sorter) + sorter.insertAll(elements) + assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0) // Make sure it spilled + val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList)) + assert(iter.next() === (0, Nil)) + assert(iter.next() === (1, List((1, 1)))) + assert(iter.next() === (2, (0 until 100000).map(x => (2, 2)).toList)) + assert(iter.next() === (3, Nil)) + assert(iter.next() === (4, Nil)) + assert(iter.next() === (5, List((5, 5)))) + assert(iter.next() === (6, Nil)) + sorter.stop() + } + + test("empty partitions with spilling, bypass merge-sort") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val elements = Iterator((1, 1), (5, 5)) ++ (0 until 100000).iterator.map(x => (2, 2)) + val sorter = new ExternalSorter[Int, Int, Int]( None, Some(new HashPartitioner(7)), None, None) - sorter.write(elements) + assertBypassedMergeSort(sorter) + sorter.insertAll(elements) assert(sc.env.blockManager.diskBlockManager.getAllFiles().length > 0) // Make sure it spilled val iter = sorter.partitionedIterator.map(p => (p._1, p._2.toList)) assert(iter.next() === (0, Nil)) @@ -286,14 +320,43 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test", conf) val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + val ord = implicitly[Ordering[Int]] + + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assertDidNotBypassMergeSort(sorter) + sorter.insertAll((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + + val sorter2 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assertDidNotBypassMergeSort(sorter2) + sorter2.insertAll((0 until 100000).iterator.map(i => (i, i))) + assert(diskBlockManager.getAllFiles().length > 0) + assert(sorter2.iterator.toSet === (0 until 100000).map(i => (i, i)).toSet) + sorter2.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in sorter, bypass merge-sort") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) - sorter.write((0 until 100000).iterator.map(i => (i, i))) + assertBypassedMergeSort(sorter) + sorter.insertAll((0 until 100000).iterator.map(i => (i, i))) assert(diskBlockManager.getAllFiles().length > 0) sorter.stop() assert(diskBlockManager.getAllBlocks().length === 0) val sorter2 = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) - sorter2.write((0 until 100000).iterator.map(i => (i, i))) + assertBypassedMergeSort(sorter2) + sorter2.insertAll((0 until 100000).iterator.map(i => (i, i))) assert(diskBlockManager.getAllFiles().length > 0) assert(sorter2.iterator.toSet === (0 until 100000).map(i => (i, i)).toSet) sorter2.stop() @@ -307,9 +370,35 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test", conf) val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + val ord = implicitly[Ordering[Int]] + + val sorter = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(3)), Some(ord), None) + assertDidNotBypassMergeSort(sorter) + intercept[SparkException] { + sorter.insertAll((0 until 100000).iterator.map(i => { + if (i == 99990) { + throw new SparkException("Intentional failure") + } + (i, i) + })) + } + assert(diskBlockManager.getAllFiles().length > 0) + sorter.stop() + assert(diskBlockManager.getAllBlocks().length === 0) + } + + test("cleanup of intermediate files in sorter if there are errors, bypass merge-sort") { + val conf = createSparkConf(true) // Load defaults, otherwise SPARK_HOME is not found + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + val diskBlockManager = SparkEnv.get.blockManager.diskBlockManager + val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) + assertBypassedMergeSort(sorter) intercept[SparkException] { - sorter.write((0 until 100000).iterator.map(i => { + sorter.insertAll((0 until 100000).iterator.map(i => { if (i == 99990) { throw new SparkException("Intentional failure") } @@ -365,7 +454,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test", conf) val sorter = new ExternalSorter[Int, Int, Int](None, Some(new HashPartitioner(3)), None, None) - sorter.write((0 until 100000).iterator.map(i => (i / 4, i))) + sorter.insertAll((0 until 100000).iterator.map(i => (i / 4, i))) val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet val expected = (0 until 3).map(p => { (p, (0 until 100000).map(i => (i / 4, i)).filter(_._1 % 3 == p).toSet) @@ -381,7 +470,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), None, None) - sorter.write((0 until 100).iterator.map(i => (i / 2, i))) + sorter.insertAll((0 until 100).iterator.map(i => (i / 2, i))) val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet val expected = (0 until 3).map(p => { (p, (0 until 50).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) @@ -397,7 +486,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), None, None) - sorter.write((0 until 100000).iterator.map(i => (i / 2, i))) + sorter.insertAll((0 until 100000).iterator.map(i => (i / 2, i))) val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet val expected = (0 until 3).map(p => { (p, (0 until 50000).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) @@ -414,7 +503,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) val ord = implicitly[Ordering[Int]] val sorter = new ExternalSorter(Some(agg), Some(new HashPartitioner(3)), Some(ord), None) - sorter.write((0 until 100000).iterator.map(i => (i / 2, i))) + sorter.insertAll((0 until 100000).iterator.map(i => (i / 2, i))) val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSet)}.toSet val expected = (0 until 3).map(p => { (p, (0 until 50000).map(i => (i, i * 4 + 1)).filter(_._1 % 3 == p).toSet) @@ -431,7 +520,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val ord = implicitly[Ordering[Int]] val sorter = new ExternalSorter[Int, Int, Int]( None, Some(new HashPartitioner(3)), Some(ord), None) - sorter.write((0 until 100).iterator.map(i => (i, i))) + sorter.insertAll((0 until 100).iterator.map(i => (i, i))) val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSeq)}.toSeq val expected = (0 until 3).map(p => { (p, (0 until 100).map(i => (i, i)).filter(_._1 % 3 == p).toSeq) @@ -448,7 +537,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val ord = implicitly[Ordering[Int]] val sorter = new ExternalSorter[Int, Int, Int]( None, Some(new HashPartitioner(3)), Some(ord), None) - sorter.write((0 until 100000).iterator.map(i => (i, i))) + sorter.insertAll((0 until 100000).iterator.map(i => (i, i))) val results = sorter.partitionedIterator.map{case (p, vs) => (p, vs.toSeq)}.toSeq val expected = (0 until 3).map(p => { (p, (0 until 100000).map(i => (i, i)).filter(_._1 % 3 == p).toSeq) @@ -495,7 +584,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val toInsert = (1 to 100000).iterator.map(_.toString).map(s => (s, s)) ++ collisionPairs.iterator ++ collisionPairs.iterator.map(_.swap) - sorter.write(toInsert) + sorter.insertAll(toInsert) // A map of collision pairs in both directions val collisionPairsMap = (collisionPairs ++ collisionPairs.map(_.swap)).toMap @@ -524,7 +613,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { // Insert 10 copies each of lots of objects whose hash codes are either 0 or 1. This causes // problems if the map fails to group together the objects with the same code (SPARK-2043). val toInsert = for (i <- 1 to 10; j <- 1 to 10000) yield (FixedHashObject(j, j % 2), 1) - sorter.write(toInsert.iterator) + sorter.insertAll(toInsert.iterator) val it = sorter.iterator var count = 0 @@ -548,7 +637,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val agg = new Aggregator[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) val sorter = new ExternalSorter[Int, Int, ArrayBuffer[Int]](Some(agg), None, None, None) - sorter.write((1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) + sorter.insertAll((1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) val it = sorter.iterator while (it.hasNext) { @@ -572,7 +661,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { val sorter = new ExternalSorter[String, String, ArrayBuffer[String]]( Some(agg), None, None, None) - sorter.write((1 to 100000).iterator.map(i => (i.toString, i.toString)) ++ Iterator( + sorter.insertAll((1 to 100000).iterator.map(i => (i.toString, i.toString)) ++ Iterator( (null.asInstanceOf[String], "1"), ("1", null.asInstanceOf[String]), (null.asInstanceOf[String], null.asInstanceOf[String]) @@ -584,4 +673,38 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext { it.next() } } + + test("conditions for bypassing merge-sort") { + val conf = createSparkConf(false) + conf.set("spark.shuffle.memoryFraction", "0.001") + conf.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.SortShuffleManager") + sc = new SparkContext("local", "test", conf) + + val agg = new Aggregator[Int, Int, Int](i => i, (i, j) => i + j, (i, j) => i + j) + val ord = implicitly[Ordering[Int]] + + // Numbers of partitions that are above and below the default bypassMergeThreshold + val FEW_PARTITIONS = 50 + val MANY_PARTITIONS = 10000 + + // Sorters with no ordering or aggregator: should bypass unless # of partitions is high + + val sorter1 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(FEW_PARTITIONS)), None, None) + assertBypassedMergeSort(sorter1) + + val sorter2 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(MANY_PARTITIONS)), None, None) + assertDidNotBypassMergeSort(sorter2) + + // Sorters with an ordering or aggregator: should not bypass even if they have few partitions + + val sorter3 = new ExternalSorter[Int, Int, Int]( + None, Some(new HashPartitioner(FEW_PARTITIONS)), Some(ord), None) + assertDidNotBypassMergeSort(sorter3) + + val sorter4 = new ExternalSorter[Int, Int, Int]( + Some(agg), Some(new HashPartitioner(FEW_PARTITIONS)), None, None) + assertDidNotBypassMergeSort(sorter4) + } } diff --git a/docs/configuration.md b/docs/configuration.md index 5e3eb0f0871af..4d27c5a918fe0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -281,6 +281,24 @@ Apart from these, the following properties are also available, and may be useful overhead per reduce task, so keep it small unless you have a large amount of memory. + + spark.shuffle.manager + HASH + + Implementation to use for shuffling data. A hash-based shuffle manager is the default, but + starting in Spark 1.1 there is an experimental sort-based shuffle manager that is more + memory-efficient in environments with small executors, such as YARN. To use that, change + this value to SORT. + + + + spark.shuffle.sort.bypassMergeThreshold + 200 + + (Advanced) In the sort-based shuffle manager, avoid merge-sorting data if there is no + map-side aggregation and there are at most this many reduce partitions. + + #### Spark UI From 4c51098f320f164eb66f92ff0f26b0b595a58f38 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 7 Aug 2014 18:09:03 -0700 Subject: [PATCH 0288/1492] SPARK-2565. Update ShuffleReadMetrics as blocks are fetched Author: Sandy Ryza Closes #1507 from sryza/sandy-spark-2565 and squashes the following commits: 74dad41 [Sandy Ryza] SPARK-2565. Update ShuffleReadMetrics as blocks are fetched --- .../org/apache/spark/executor/Executor.scala | 1 + .../apache/spark/executor/TaskMetrics.scala | 55 ++++++++++++++----- .../hash/BlockStoreShuffleFetcher.scala | 13 ++--- .../shuffle/hash/HashShuffleReader.scala | 4 +- .../spark/storage/BlockFetcherIterator.scala | 40 +++++--------- .../apache/spark/storage/BlockManager.scala | 11 ++-- .../org/apache/spark/util/JsonProtocol.scala | 5 +- .../storage/BlockFetcherIteratorSuite.scala | 13 +++-- .../ui/jobs/JobProgressListenerSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 2 +- 10 files changed, 84 insertions(+), 64 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c2b9c660ddaec..eac1f2326a29d 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -374,6 +374,7 @@ private[spark] class Executor( for (taskRunner <- runningTasks.values()) { if (!taskRunner.attemptedTask.isEmpty) { Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => + metrics.updateShuffleReadMetrics tasksMetrics += ((taskRunner.taskId, metrics)) } } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 11a6e10243211..99a88c13456df 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} @@ -81,12 +83,27 @@ class TaskMetrics extends Serializable { var inputMetrics: Option[InputMetrics] = None /** - * If this task reads from shuffle output, metrics on getting shuffle data will be collected here + * If this task reads from shuffle output, metrics on getting shuffle data will be collected here. + * This includes read metrics aggregated over all the task's shuffle dependencies. */ private var _shuffleReadMetrics: Option[ShuffleReadMetrics] = None def shuffleReadMetrics = _shuffleReadMetrics + /** + * This should only be used when recreating TaskMetrics, not when updating read metrics in + * executors. + */ + private[spark] def setShuffleReadMetrics(shuffleReadMetrics: Option[ShuffleReadMetrics]) { + _shuffleReadMetrics = shuffleReadMetrics + } + + /** + * ShuffleReadMetrics per dependency for collecting independently while task is in progress. + */ + @transient private lazy val depsShuffleReadMetrics: ArrayBuffer[ShuffleReadMetrics] = + new ArrayBuffer[ShuffleReadMetrics]() + /** * If this task writes to shuffle output, metrics on the written shuffle data will be collected * here @@ -98,19 +115,31 @@ class TaskMetrics extends Serializable { */ var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None - /** Adds the given ShuffleReadMetrics to any existing shuffle metrics for this task. */ - def updateShuffleReadMetrics(newMetrics: ShuffleReadMetrics) = synchronized { - _shuffleReadMetrics match { - case Some(existingMetrics) => - existingMetrics.shuffleFinishTime = math.max( - existingMetrics.shuffleFinishTime, newMetrics.shuffleFinishTime) - existingMetrics.fetchWaitTime += newMetrics.fetchWaitTime - existingMetrics.localBlocksFetched += newMetrics.localBlocksFetched - existingMetrics.remoteBlocksFetched += newMetrics.remoteBlocksFetched - existingMetrics.remoteBytesRead += newMetrics.remoteBytesRead - case None => - _shuffleReadMetrics = Some(newMetrics) + /** + * A task may have multiple shuffle readers for multiple dependencies. To avoid synchronization + * issues from readers in different threads, in-progress tasks use a ShuffleReadMetrics for each + * dependency, and merge these metrics before reporting them to the driver. This method returns + * a ShuffleReadMetrics for a dependency and registers it for merging later. + */ + private [spark] def createShuffleReadMetricsForDependency(): ShuffleReadMetrics = synchronized { + val readMetrics = new ShuffleReadMetrics() + depsShuffleReadMetrics += readMetrics + readMetrics + } + + /** + * Aggregates shuffle read metrics for all registered dependencies into shuffleReadMetrics. + */ + private[spark] def updateShuffleReadMetrics() = synchronized { + val merged = new ShuffleReadMetrics() + for (depMetrics <- depsShuffleReadMetrics) { + merged.fetchWaitTime += depMetrics.fetchWaitTime + merged.localBlocksFetched += depMetrics.localBlocksFetched + merged.remoteBlocksFetched += depMetrics.remoteBlocksFetched + merged.remoteBytesRead += depMetrics.remoteBytesRead + merged.shuffleFinishTime = math.max(merged.shuffleFinishTime, depMetrics.shuffleFinishTime) } + _shuffleReadMetrics = Some(merged) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 99788828981c7..12b475658e29d 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -32,7 +32,8 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleId: Int, reduceId: Int, context: TaskContext, - serializer: Serializer) + serializer: Serializer, + shuffleMetrics: ShuffleReadMetrics) : Iterator[T] = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) @@ -73,17 +74,11 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer) + val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer, shuffleMetrics) val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { - val shuffleMetrics = new ShuffleReadMetrics - shuffleMetrics.shuffleFinishTime = System.currentTimeMillis - shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime - shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead - shuffleMetrics.localBlocksFetched = blockFetcherItr.numLocalBlocks - shuffleMetrics.remoteBlocksFetched = blockFetcherItr.numRemoteBlocks - context.taskMetrics.updateShuffleReadMetrics(shuffleMetrics) + context.taskMetrics.updateShuffleReadMetrics() }) new InterruptibleIterator[T](context, completionIter) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 88a5f1e5ddf58..7bed97a63f0f6 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -36,8 +36,10 @@ private[spark] class HashShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { + val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() val ser = Serializer.getSerializer(dep.serializer) - val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser) + val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser, + readMetrics) val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 938af6f5b923a..5f44f5f3197fd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -27,6 +27,7 @@ import scala.util.{Failure, Success} import io.netty.buffer.ByteBuf import org.apache.spark.{Logging, SparkException} +import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId import org.apache.spark.network.netty.ShuffleCopier @@ -47,10 +48,6 @@ import org.apache.spark.util.Utils private[storage] trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { def initialize() - def numLocalBlocks: Int - def numRemoteBlocks: Int - def fetchWaitTime: Long - def remoteBytesRead: Long } @@ -72,14 +69,12 @@ object BlockFetcherIterator { class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer) + serializer: Serializer, + readMetrics: ShuffleReadMetrics) extends BlockFetcherIterator { import blockManager._ - private var _remoteBytesRead = 0L - private var _fetchWaitTime = 0L - if (blocksByAddress == null) { throw new IllegalArgumentException("BlocksByAddress is null") } @@ -89,13 +84,9 @@ object BlockFetcherIterator { protected var startTime = System.currentTimeMillis - // This represents the number of local blocks, also counting zero-sized blocks - private var numLocal = 0 // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks protected val localBlocksToFetch = new ArrayBuffer[BlockId]() - // This represents the number of remote blocks, also counting zero-sized blocks - private var numRemote = 0 // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks protected val remoteBlocksToFetch = new HashSet[BlockId]() @@ -132,7 +123,10 @@ object BlockFetcherIterator { val networkSize = blockMessage.getData.limit() results.put(new FetchResult(blockId, sizeMap(blockId), () => dataDeserialize(blockId, blockMessage.getData, serializer))) - _remoteBytesRead += networkSize + // TODO: NettyBlockFetcherIterator has some race conditions where multiple threads can + // be incrementing bytes read at the same time (SPARK-2625). + readMetrics.remoteBytesRead += networkSize + readMetrics.remoteBlocksFetched += 1 logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } } @@ -155,14 +149,14 @@ object BlockFetcherIterator { // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] + var totalBlocks = 0 for ((address, blockInfos) <- blocksByAddress) { + totalBlocks += blockInfos.size if (address == blockManagerId) { - numLocal = blockInfos.size // Filter out zero-sized blocks localBlocksToFetch ++= blockInfos.filter(_._2 != 0).map(_._1) _numBlocksToFetch += localBlocksToFetch.size } else { - numRemote += blockInfos.size val iterator = blockInfos.iterator var curRequestSize = 0L var curBlocks = new ArrayBuffer[(BlockId, Long)] @@ -192,7 +186,7 @@ object BlockFetcherIterator { } } logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + - (numLocal + numRemote) + " blocks") + totalBlocks + " blocks") remoteRequests } @@ -205,6 +199,7 @@ object BlockFetcherIterator { // getLocalFromDisk never return None but throws BlockException val iter = getLocalFromDisk(id, serializer).get // Pass 0 as size since it's not in flight + readMetrics.localBlocksFetched += 1 results.put(new FetchResult(id, 0, () => iter)) logDebug("Got local block " + id) } catch { @@ -238,12 +233,6 @@ object BlockFetcherIterator { logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") } - override def numLocalBlocks: Int = numLocal - override def numRemoteBlocks: Int = numRemote - override def fetchWaitTime: Long = _fetchWaitTime - override def remoteBytesRead: Long = _remoteBytesRead - - // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue // as they arrive. @volatile protected var resultsGotten = 0 @@ -255,7 +244,7 @@ object BlockFetcherIterator { val startFetchWait = System.currentTimeMillis() val result = results.take() val stopFetchWait = System.currentTimeMillis() - _fetchWaitTime += (stopFetchWait - startFetchWait) + readMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) if (! result.failed) bytesInFlight -= result.size while (!fetchRequests.isEmpty && (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { @@ -269,8 +258,9 @@ object BlockFetcherIterator { class NettyBlockFetcherIterator( blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer) - extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer) { + serializer: Serializer, + readMetrics: ShuffleReadMetrics) + extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer, readMetrics) { import blockManager._ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 8d21b02b747ff..e8bbd298c631a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,7 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ -import org.apache.spark.executor.{DataReadMethod, InputMetrics, ShuffleWriteMetrics} +import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -539,12 +539,15 @@ private[spark] class BlockManager( */ def getMultiple( blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer): BlockFetcherIterator = { + serializer: Serializer, + readMetrics: ShuffleReadMetrics): BlockFetcherIterator = { val iter = if (conf.getBoolean("spark.shuffle.use.netty", false)) { - new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) + new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer, + readMetrics) } else { - new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer) + new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer, + readMetrics) } iter.initialize() iter diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b112b359368cd..6f8eb1ee12634 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -560,9 +560,8 @@ private[spark] object JsonProtocol { metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] - Utils.jsonOption(json \ "Shuffle Read Metrics").map { shuffleReadMetrics => - metrics.updateShuffleReadMetrics(shuffleReadMetricsFromJson(shuffleReadMetrics)) - } + metrics.setShuffleReadMetrics( + Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson)) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) metrics.inputMetrics = diff --git a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala index 1538995a6b404..bcbfe8baf36ad 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala @@ -33,6 +33,7 @@ import org.mockito.invocation.InvocationOnMock import org.apache.spark.storage.BlockFetcherIterator._ import org.apache.spark.network.{ConnectionManager, Message} +import org.apache.spark.executor.ShuffleReadMetrics class BlockFetcherIteratorSuite extends FunSuite with Matchers { @@ -70,8 +71,8 @@ class BlockFetcherIteratorSuite extends FunSuite with Matchers { (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) ) - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null) + val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, + new ShuffleReadMetrics()) iterator.initialize() @@ -121,8 +122,8 @@ class BlockFetcherIteratorSuite extends FunSuite with Matchers { (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) ) - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null) + val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, + new ShuffleReadMetrics()) iterator.initialize() @@ -165,7 +166,7 @@ class BlockFetcherIteratorSuite extends FunSuite with Matchers { ) val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null) + blocksByAddress, null, new ShuffleReadMetrics()) iterator.initialize() iterator.foreach{ @@ -219,7 +220,7 @@ class BlockFetcherIteratorSuite extends FunSuite with Matchers { ) val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null) + blocksByAddress, null, new ShuffleReadMetrics()) iterator.initialize() iterator.foreach{ case (_, r) => { diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index cb8252515238e..f5ba31c309277 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -65,7 +65,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc // finish this task, should get updated shuffleRead shuffleReadMetrics.remoteBytesRead = 1000 - taskMetrics.updateShuffleReadMetrics(shuffleReadMetrics) + taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) var taskInfo = new TaskInfo(1234L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 var task = new ShuffleMapTask(0) @@ -142,7 +142,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() val shuffleWriteMetrics = new ShuffleWriteMetrics() - taskMetrics.updateShuffleReadMetrics(shuffleReadMetrics) + taskMetrics.setShuffleReadMetrics(Some(shuffleReadMetrics)) taskMetrics.shuffleWriteMetrics = Some(shuffleWriteMetrics) shuffleReadMetrics.remoteBytesRead = base + 1 shuffleReadMetrics.remoteBlocksFetched = base + 2 diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 2002a817d9168..97ffb07662482 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -539,7 +539,7 @@ class JsonProtocolSuite extends FunSuite { sr.localBlocksFetched = e sr.fetchWaitTime = a + d sr.remoteBlocksFetched = f - t.updateShuffleReadMetrics(sr) + t.setShuffleReadMetrics(Some(sr)) } sw.shuffleBytesWritten = a + b + c sw.shuffleWriteTime = b + c + d From 9de6a42bb34ea8963225ce90f1a45adcfee38b58 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 7 Aug 2014 18:53:15 -0700 Subject: [PATCH 0289/1492] [SPARK-2904] Remove non-used local variable in SparkSubmitArguments Author: Kousuke Saruta Closes #1834 from sarutak/SPARK-2904 and squashes the following commits: 38e7d45 [Kousuke Saruta] Removed non-used variable in SparkSubmitArguments --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 087dd4d633db0..c21f1529a1837 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -219,7 +219,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { /** Fill in values by parsing user options. */ private def parseOpts(opts: Seq[String]): Unit = { - var inSparkOpts = true val EQ_SEPARATED_OPT="""(--[^=]+)=(.+)""".r // Delineates parsing of Spark options from parsing of user options. From 9a54de16ed9de536e0436d532c587384e1ea0af6 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Thu, 7 Aug 2014 23:45:16 -0700 Subject: [PATCH 0290/1492] [SPARK-2911]: provide rdd.parent[T](j) to obtain jth parent RDD Author: Erik Erlandson Closes #1841 from erikerlandson/spark-2911-pr and squashes the following commits: 4699e2f [Erik Erlandson] [SPARK-2911]: provide rdd.parent[T](j) to obtain jth parent RDD --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 5 +++++ .../src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 10 ++++++++++ 2 files changed, 15 insertions(+) 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 0159003c88e06..19e10bd04681b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1233,6 +1233,11 @@ abstract class RDD[T: ClassTag]( dependencies.head.rdd.asInstanceOf[RDD[U]] } + /** Returns the jth parent RDD: e.g. rdd.parent[T](0) is equivalent to rdd.firstParent[T] */ + protected[spark] def parent[U: ClassTag](j: Int) = { + dependencies(j).rdd.asInstanceOf[RDD[U]] + } + /** The [[org.apache.spark.SparkContext]] that this RDD was created on. */ def context = sc diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 4a7dc8dca25e2..926d4fecb5b91 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -726,6 +726,16 @@ class RDDSuite extends FunSuite with SharedSparkContext { jrdd.rdd.retag.collect() } + test("parent method") { + val rdd1 = sc.parallelize(1 to 10, 2) + val rdd2 = rdd1.filter(_ % 2 == 0) + val rdd3 = rdd2.map(_ + 1) + val rdd4 = new UnionRDD(sc, List(rdd1, rdd2, rdd3)) + assert(rdd4.parent(0).isInstanceOf[ParallelCollectionRDD[_]]) + assert(rdd4.parent(1).isInstanceOf[FilteredRDD[_]]) + assert(rdd4.parent(2).isInstanceOf[MappedRDD[_, _]]) + } + test("getNarrowAncestors") { val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1) From 9016af3f2729101027e33593e094332f05f48d92 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 8 Aug 2014 11:01:51 -0700 Subject: [PATCH 0291/1492] [SPARK-2888] [SQL] Fix addColumnMetadataToConf in HiveTableScan JIRA: https://issues.apache.org/jira/browse/SPARK-2888 Author: Yin Huai Closes #1817 from yhuai/fixAddColumnMetadataToConf and squashes the following commits: fba728c [Yin Huai] Fix addColumnMetadataToConf. --- .../sql/hive/execution/HiveTableScan.scala | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 8920e2a76a27f..577ca928b43b6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -72,17 +72,12 @@ case class HiveTableScan( } private def addColumnMetadataToConf(hiveConf: HiveConf) { - // Specifies IDs and internal names of columns to be scanned. - val neededColumnIDs = attributes.map(a => relation.output.indexWhere(_.name == a.name): Integer) - val columnInternalNames = neededColumnIDs.map(HiveConf.getColumnInternalName(_)).mkString(",") - - if (attributes.size == relation.output.size) { - // SQLContext#pruneFilterProject guarantees no duplicated value in `attributes` - ColumnProjectionUtils.setFullyReadColumns(hiveConf) - } else { - ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) - } + // Specifies needed column IDs for those non-partitioning columns. + val neededColumnIDs = + attributes.map(a => + relation.attributes.indexWhere(_.name == a.name): Integer).filter(index => index >= 0) + ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) // Specifies types and object inspectors of columns to be scanned. @@ -99,7 +94,7 @@ case class HiveTableScan( .mkString(",") hiveConf.set(serdeConstants.LIST_COLUMN_TYPES, columnTypeNames) - hiveConf.set(serdeConstants.LIST_COLUMNS, columnInternalNames) + hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(",")) } addColumnMetadataToConf(context.hiveconf) From 0489cee6b24ca34f1adab03a75d157e04a9e06b7 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 8 Aug 2014 11:10:11 -0700 Subject: [PATCH 0292/1492] [SPARK-2908] [SQL] JsonRDD.nullTypeToStringType does not convert all NullType to StringType JIRA: https://issues.apache.org/jira/browse/SPARK-2908 Author: Yin Huai Closes #1840 from yhuai/SPARK-2908 and squashes the following commits: 86e833e [Yin Huai] Update test. cb11759 [Yin Huai] nullTypeToStringType should check columns with the type of array of structs. --- .../scala/org/apache/spark/sql/json/JsonRDD.scala | 4 +++- .../scala/org/apache/spark/sql/json/JsonSuite.scala | 11 ++++++++--- .../org/apache/spark/sql/json/TestJsonData.scala | 2 +- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index a3d2a1c7a51f8..1c0b03c684f10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -109,7 +109,9 @@ private[sql] object JsonRDD extends Logging { val newType = dataType match { case NullType => StringType case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull) - case struct: StructType => nullTypeToStringType(struct) + case ArrayType(struct: StructType, containsNull) => + ArrayType(nullTypeToStringType(struct), containsNull) + case struct: StructType =>nullTypeToStringType(struct) case other: DataType => other } StructField(fieldName, newType, nullable) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 75c0589eb208e..58b1e23891a3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -213,7 +213,8 @@ class JsonSuite extends QueryTest { StructField("arrayOfStruct", ArrayType( StructType( StructField("field1", BooleanType, true) :: - StructField("field2", StringType, true) :: Nil)), true) :: + StructField("field2", StringType, true) :: + StructField("field3", StringType, true) :: Nil)), true) :: StructField("struct", StructType( StructField("field1", BooleanType, true) :: StructField("field2", DecimalType, true) :: Nil), true) :: @@ -263,8 +264,12 @@ class JsonSuite extends QueryTest { // Access elements of an array of structs. checkAnswer( - sql("select arrayOfStruct[0], arrayOfStruct[1], arrayOfStruct[2] from jsonTable"), - (true :: "str1" :: Nil, false :: null :: Nil, null) :: Nil + sql("select arrayOfStruct[0], arrayOfStruct[1], arrayOfStruct[2], arrayOfStruct[3] " + + "from jsonTable"), + (true :: "str1" :: null :: Nil, + false :: null :: null :: Nil, + null :: null :: null :: Nil, + null) :: Nil ) // Access a struct and fields inside of it. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index d0180f3754f22..a88310b5f1b46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -43,7 +43,7 @@ object TestJsonData { "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], - "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}], + "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" :: Nil) From c874723fa844b49f057bb2434a12228b2f717e99 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 8 Aug 2014 11:15:16 -0700 Subject: [PATCH 0293/1492] [SPARK-2877] [SQL] MetastoreRelation should use SparkClassLoader when creating the tableDesc JIRA: https://issues.apache.org/jira/browse/SPARK-2877 Author: Yin Huai Closes #1806 from yhuai/SPARK-2877 and squashes the following commits: 4142bcb [Yin Huai] Use Spark's classloader. --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 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 301cf51c00e2b..82e9c1a248626 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 @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive import scala.util.parsing.combinator.RegexParsers -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, SerDeInfo} import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} @@ -39,6 +37,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.util.Utils /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -288,7 +287,10 @@ private[hive] case class MetastoreRelation ) val tableDesc = new TableDesc( - Class.forName(hiveQlTable.getSerializationLib).asInstanceOf[Class[Deserializer]], + Class.forName( + hiveQlTable.getSerializationLib, + true, + Utils.getContextOrSparkClassLoader).asInstanceOf[Class[Deserializer]], hiveQlTable.getInputFormatClass, // The class of table should be org.apache.hadoop.hive.ql.metadata.Table because // getOutputFormatClass will use HiveFileFormatUtils.getOutputFormatSubstitute to From 45d8f4deab50ae069ecde2201bd486d464a4501e Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 8 Aug 2014 11:23:58 -0700 Subject: [PATCH 0294/1492] [SPARK-2919] [SQL] Basic support for analyze command in HiveQl The command we will support is ``` ANALYZE TABLE tablename COMPUTE STATISTICS noscan ``` Other cases shown in https://cwiki.apache.org/confluence/display/Hive/StatsDev#StatsDev-ExistingTables will still be treated as Hive native commands. JIRA: https://issues.apache.org/jira/browse/SPARK-2919 Author: Yin Huai Closes #1848 from yhuai/sqlAnalyze and squashes the following commits: 0b79d36 [Yin Huai] Typo and format. c59d94b [Yin Huai] Support "ANALYZE TABLE tableName COMPUTE STATISTICS noscan". --- .../org/apache/spark/sql/hive/HiveQl.scala | 21 +++++++-- .../spark/sql/hive/HiveStrategies.scala | 2 + .../{DropTable.scala => commands.scala} | 26 +++++++++++ .../spark/sql/hive/StatisticsSuite.scala | 45 ++++++++++++++++++- 4 files changed, 89 insertions(+), 5 deletions(-) rename sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/{DropTable.scala => commands.scala} (72%) 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 bc2fefafd58c8..05b2f5f6cd3f7 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 @@ -46,6 +46,8 @@ private[hive] case class AddFile(filePath: String) extends Command private[hive] case class DropTable(tableName: String, ifExists: Boolean) extends Command +private[hive] case class AnalyzeTable(tableName: String) extends Command + /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ private[hive] object HiveQl { protected val nativeCommands = Seq( @@ -74,7 +76,6 @@ private[hive] object HiveQl { "TOK_CREATEFUNCTION", "TOK_DROPFUNCTION", - "TOK_ANALYZE", "TOK_ALTERDATABASE_PROPERTIES", "TOK_ALTERINDEX_PROPERTIES", "TOK_ALTERINDEX_REBUILD", @@ -92,7 +93,6 @@ private[hive] object HiveQl { "TOK_ALTERTABLE_SKEWED", "TOK_ALTERTABLE_TOUCH", "TOK_ALTERTABLE_UNARCHIVE", - "TOK_ANALYZE", "TOK_CREATEDATABASE", "TOK_CREATEFUNCTION", "TOK_CREATEINDEX", @@ -239,7 +239,6 @@ private[hive] object HiveQl { ShellCommand(sql.drop(1)) } else { val tree = getAst(sql) - if (nativeCommands contains tree.getText) { NativeCommand(sql) } else { @@ -387,6 +386,22 @@ private[hive] object HiveQl { ifExists) => val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") DropTable(tableName, ifExists.nonEmpty) + // Support "ANALYZE TABLE tableNmae COMPUTE STATISTICS noscan" + case Token("TOK_ANALYZE", + Token("TOK_TAB", Token("TOK_TABNAME", tableNameParts) :: partitionSpec) :: + isNoscan) => + // Reference: + // https://cwiki.apache.org/confluence/display/Hive/StatsDev#StatsDev-ExistingTables + if (partitionSpec.nonEmpty) { + // Analyze partitions will be treated as a Hive native command. + NativePlaceholder + } else if (isNoscan.isEmpty) { + // If users do not specify "noscan", it will be treated as a Hive native command. + NativePlaceholder + } else { + val tableName = tableNameParts.map { case Token(p, Nil) => p }.mkString(".") + AnalyzeTable(tableName) + } // Just fake explain for any of the native commands. case Token("TOK_EXPLAIN", explainArgs) if noExplainCommands.contains(explainArgs.head.getText) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 2175c5f3835a6..85d2496a34cfb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -83,6 +83,8 @@ private[hive] trait HiveStrategies { case DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil + case AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil + case describe: logical.DescribeCommand => val resolvedTable = context.executePlan(describe.table).analyzed resolvedTable match { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DropTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala similarity index 72% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DropTable.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 9cd0c86c6c796..2985169da033c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DropTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -23,6 +23,32 @@ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{Command, LeafNode} import org.apache.spark.sql.hive.HiveContext +/** + * :: DeveloperApi :: + * Analyzes the given table in the current database to generate statistics, which will be + * used in query optimizations. + * + * Right now, it only supports Hive tables and it only updates the size of a Hive table + * in the Hive metastore. + */ +@DeveloperApi +case class AnalyzeTable(tableName: String) extends LeafNode with Command { + + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + def output = Seq.empty + + override protected[sql] lazy val sideEffectResult = { + hiveContext.analyze(tableName) + Seq.empty[Any] + } + + override def execute(): RDD[Row] = { + sideEffectResult + sparkContext.emptyRDD[Row] + } +} + /** * :: DeveloperApi :: * Drops a table from the metastore and removes it if it is cached. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index bf5931bbf97ee..7c82964b5ecdc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -19,13 +19,54 @@ package org.apache.spark.sql.hive import scala.reflect.ClassTag + import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.catalyst.plans.logical.NativeCommand import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ class StatisticsSuite extends QueryTest { + test("parse analyze commands") { + def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { + val parsed = HiveQl.parseSql(analyzeCommand) + val operators = parsed.collect { + case a: AnalyzeTable => a + case o => o + } + + assert(operators.size === 1) + if (operators(0).getClass() != c) { + fail( + s"""$analyzeCommand expected command: $c, but got ${operators(0)} + |parsed command: + |$parsed + """.stripMargin) + } + } + + assertAnalyzeCommand( + "ANALYZE TABLE Table1 COMPUTE STATISTICS", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds='2008-04-09', hr=11) COMPUTE STATISTICS noscan", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS", + classOf[NativeCommand]) + assertAnalyzeCommand( + "ANALYZE TABLE Table1 PARTITION(ds, hr) COMPUTE STATISTICS noscan", + classOf[NativeCommand]) + + assertAnalyzeCommand( + "ANALYZE TABLE Table1 COMPUTE STATISTICS nOscAn", + classOf[AnalyzeTable]) + } + test("analyze MetastoreRelations") { def queryTotalSize(tableName: String): BigInt = catalog.lookupRelation(None, tableName).statistics.sizeInBytes @@ -37,7 +78,7 @@ class StatisticsSuite extends QueryTest { assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) - analyze("analyzeTable") + sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") assert(queryTotalSize("analyzeTable") === BigInt(11624)) @@ -66,7 +107,7 @@ class StatisticsSuite extends QueryTest { assert(queryTotalSize("analyzeTable_part") === defaultSizeInBytes) - analyze("analyzeTable_part") + sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) From b7c89a7f0ca73153dce36e0f01b81a3947ee1189 Mon Sep 17 00:00:00 2001 From: chutium Date: Fri, 8 Aug 2014 13:31:08 -0700 Subject: [PATCH 0295/1492] [SPARK-2700] [SQL] Hidden files (such as .impala_insert_staging) should be filtered out by sqlContext.parquetFile Author: chutium Closes #1691 from chutium/SPARK-2700 and squashes the following commits: b76ae8c [chutium] [SPARK-2700] [SQL] fixed styling issue d75a8bd [chutium] [SPARK-2700] [SQL] Hidden files (such as .impala_insert_staging) should be filtered out by sqlContext.parquetFile --- .../scala/org/apache/spark/sql/parquet/ParquetTypes.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 aaef1a1d474fe..2867dc0a8b1f9 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 @@ -373,8 +373,9 @@ private[parquet] object ParquetTypesConverter extends Logging { } ParquetRelation.enableLogForwarding() - val children = fs.listStatus(path).filterNot { - _.getPath.getName == FileOutputCommitter.SUCCEEDED_FILE_NAME + val children = fs.listStatus(path).filterNot { status => + val name = status.getPath.getName + name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME } // NOTE (lian): Parquet "_metadata" file can be very slow if the file consists of lots of row From 74d6f62264babfc6045c21545552f0a2e6958155 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 8 Aug 2014 15:07:31 -0700 Subject: [PATCH 0296/1492] [SPARK-1997][MLLIB] update breeze to 0.9 0.9 dependences (this version doesn't depend on scalalogging and I excluded commons-math3 from its transitive dependencies): ~~~ +-org.scalanlp:breeze_2.10:0.9 [S] +-com.github.fommil.netlib:core:1.1.2 +-com.github.rwl:jtransforms:2.4.0 +-net.sf.opencsv:opencsv:2.3 +-net.sourceforge.f2j:arpack_combined_all:0.1 +-org.scalanlp:breeze-macros_2.10:0.3.1 [S] | +-org.scalamacros:quasiquotes_2.10:2.0.0 [S] | +-org.slf4j:slf4j-api:1.7.5 +-org.spire-math:spire_2.10:0.7.4 [S] +-org.scalamacros:quasiquotes_2.10:2.0.0 [S] | +-org.spire-math:spire-macros_2.10:0.7.4 [S] +-org.scalamacros:quasiquotes_2.10:2.0.0 [S] ~~~ Closes #1749 CC: witgo avati Author: Xiangrui Meng Closes #1857 from mengxr/breeze-0.9 and squashes the following commits: 7fc16b6 [Xiangrui Meng] don't know why but exclude a private method for mima dcc502e [Xiangrui Meng] update breeze to 0.9 --- mllib/pom.xml | 2 +- .../org/apache/spark/mllib/linalg/distributed/RowMatrix.scala | 4 ++-- .../spark/mllib/linalg/distributed/RowMatrixSuite.scala | 2 +- project/MimaExcludes.scala | 4 ++++ 4 files changed, 8 insertions(+), 4 deletions(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index 9a33bd1cf6ad1..fc1ecfbea708f 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -57,7 +57,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.7 + 0.9 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 45486b2c7d82d..e76bc9fefff01 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -222,7 +222,7 @@ class RowMatrix( EigenValueDecomposition.symmetricEigs(v => G * v, n, k, tol, maxIter) case SVDMode.LocalLAPACK => val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] - val (uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G) + val brzSvd.SVD(uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G) (sigmaSquaresFull, uFull) case SVDMode.DistARPACK => require(k < n, s"k must be smaller than n in dist-eigs mode but got k=$k and n=$n.") @@ -338,7 +338,7 @@ class RowMatrix( val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] - val (u: BDM[Double], _, _) = brzSvd(Cov) + val brzSvd.SVD(u: BDM[Double], _, _) = brzSvd(Cov) if (k == n) { Matrices.dense(n, k, u.data) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 325b817980f68..1d3a3221365cc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -99,7 +99,7 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { for (mat <- Seq(denseMat, sparseMat)) { for (mode <- Seq("auto", "local-svd", "local-eigs", "dist-eigs")) { val localMat = mat.toBreeze() - val (localU, localSigma, localVt) = brzSvd(localMat) + val brzSvd.SVD(localU, localSigma, localVt) = brzSvd(localMat) val localV: BDM[Double] = localVt.t.toDenseMatrix for (k <- 1 to n) { val skip = (mode == "local-eigs" || mode == "dist-eigs") && k == n diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 537ca0dcf267d..b4653c72c10b5 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -110,6 +110,10 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") + ) ++ + Seq ( // package-private classes removed in MLlib + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.regression.GeneralizedLinearAlgorithm.org$apache$spark$mllib$regression$GeneralizedLinearAlgorithm$$prependOne") ) case v if v.startsWith("1.0") => Seq( From ec79063fad44751a6689f5e58d47886babeaecff Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Fri, 8 Aug 2014 16:57:26 -0700 Subject: [PATCH 0297/1492] [SPARK-2897][SPARK-2920]TorrentBroadcast does use the serializer class specified in the spark option "spark.serializer" Author: GuoQiang Li Closes #1836 from witgo/SPARK-2897 and squashes the following commits: 23cdc5b [GuoQiang Li] review commit ada4fba [GuoQiang Li] TorrentBroadcast does not support broadcast compression fb91792 [GuoQiang Li] org.apache.spark.broadcast.TorrentBroadcast does use the serializer class specified in the spark option "spark.serializer" --- .../spark/broadcast/TorrentBroadcast.scala | 31 +++++++++++++++---- .../spark/broadcast/BroadcastSuite.scala | 10 ++++-- 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 86731b684f441..fe73456ef8fad 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -17,14 +17,15 @@ package org.apache.spark.broadcast -import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} +import java.io.{ByteArrayOutputStream, ByteArrayInputStream, InputStream, + ObjectInputStream, ObjectOutputStream, OutputStream} import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} +import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} -import org.apache.spark.util.Utils /** * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like @@ -214,11 +215,15 @@ private[broadcast] object TorrentBroadcast extends Logging { private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 private var initialized = false private var conf: SparkConf = null + private var compress: Boolean = false + private var compressionCodec: CompressionCodec = null def initialize(_isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests synchronized { if (!initialized) { + compress = conf.getBoolean("spark.broadcast.compress", true) + compressionCodec = CompressionCodec.createCodec(conf) initialized = true } } @@ -228,8 +233,13 @@ private[broadcast] object TorrentBroadcast extends Logging { initialized = false } - def blockifyObject[T](obj: T): TorrentInfo = { - val byteArray = Utils.serialize[T](obj) + def blockifyObject[T: ClassTag](obj: T): TorrentInfo = { + val bos = new ByteArrayOutputStream() + val out: OutputStream = if (compress) compressionCodec.compressedOutputStream(bos) else bos + val ser = SparkEnv.get.serializer.newInstance() + val serOut = ser.serializeStream(out) + serOut.writeObject[T](obj).close() + val byteArray = bos.toByteArray val bais = new ByteArrayInputStream(byteArray) var blockNum = byteArray.length / BLOCK_SIZE @@ -255,7 +265,7 @@ private[broadcast] object TorrentBroadcast extends Logging { info } - def unBlockifyObject[T]( + def unBlockifyObject[T: ClassTag]( arrayOfBlocks: Array[TorrentBlock], totalBytes: Int, totalBlocks: Int): T = { @@ -264,7 +274,16 @@ private[broadcast] object TorrentBroadcast extends Logging { System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray, i * BLOCK_SIZE, arrayOfBlocks(i).byteArray.length) } - Utils.deserialize[T](retByteArray, Thread.currentThread.getContextClassLoader) + + val in: InputStream = { + val arrIn = new ByteArrayInputStream(retByteArray) + if (compress) compressionCodec.compressedInputStream(arrIn) else arrIn + } + val ser = SparkEnv.get.serializer.newInstance() + val serIn = ser.deserializeStream(in) + val obj = serIn.readObject[T]() + serIn.close() + obj } /** diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 7c3d0208b195a..17c64455b2429 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -44,7 +44,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { test("Accessing HttpBroadcast variables in a local cluster") { val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", httpConf) + val conf = httpConf.clone + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.broadcast.compress", "true") + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) @@ -69,7 +72,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { test("Accessing TorrentBroadcast variables in a local cluster") { val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", torrentConf) + val conf = torrentConf.clone + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.broadcast.compress", "true") + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) From 1c84dba9881118204687c81003bded6d49e27255 Mon Sep 17 00:00:00 2001 From: WangTao Date: Fri, 8 Aug 2014 20:53:21 -0700 Subject: [PATCH 0298/1492] [Web UI]Make decision order of Worker's WebUI port consistent with Master's The decision order of Worker's WebUI port is "--webui-port", SPARK_WORKER_WEBUI_POR, 8081(default), spark.worker.ui.port. But in Master, the order is "--webui-port", spark.master.ui.port, SPARK_MASTER_WEBUI_PORT and 8080(default). So we change the order in Worker's to keep it consistent with Master. Author: WangTao Closes #1838 from WangTaoTheTonic/reOrder and squashes the following commits: 460f4d4 [WangTao] Make decision order of Worker's WebUI consistent with Master's --- .../scala/org/apache/spark/deploy/worker/Worker.scala | 5 +++-- .../org/apache/spark/deploy/worker/WorkerArguments.scala | 6 +++++- .../org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 9 ++------- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 458d9947bd873..bacb514ed6335 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -136,7 +136,7 @@ private[spark] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) + webUi = new WorkerWebUI(this, workDir, webUiPort) webUi.bind() registerWithMaster() @@ -373,7 +373,8 @@ private[spark] class Worker( private[spark] object Worker extends Logging { def main(argStrings: Array[String]) { SignalLogger.register(log) - val args = new WorkerArguments(argStrings) + val conf = new SparkConf + val args = new WorkerArguments(argStrings, conf) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, args.memory, args.masters, args.workDir) actorSystem.awaitTermination() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index dc5158102054e..1e295aaa48c30 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -20,11 +20,12 @@ package org.apache.spark.deploy.worker import java.lang.management.ManagementFactory import org.apache.spark.util.{IntParam, MemoryParam, Utils} +import org.apache.spark.SparkConf /** * Command-line parser for the worker. */ -private[spark] class WorkerArguments(args: Array[String]) { +private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 0 var webUiPort = 8081 @@ -46,6 +47,9 @@ private[spark] class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt } + if (conf.contains("spark.worker.ui.port")) { + webUiPort = conf.get("spark.worker.ui.port").toInt + } if (System.getenv("SPARK_WORKER_DIR") != null) { workDir = System.getenv("SPARK_WORKER_DIR") } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 47fbda600bea7..b07942a9ca729 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -34,8 +34,8 @@ private[spark] class WorkerWebUI( val worker: Worker, val workDir: File, - port: Option[Int] = None) - extends WebUI(worker.securityMgr, getUIPort(port, worker.conf), worker.conf, name = "WorkerUI") + requestedPort: Int) + extends WebUI(worker.securityMgr, requestedPort, worker.conf, name = "WorkerUI") with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) @@ -55,10 +55,5 @@ class WorkerWebUI( } private[spark] object WorkerWebUI { - val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - - def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { - requestedPort.getOrElse(conf.getInt("spark.worker.ui.port", WorkerWebUI.DEFAULT_PORT)) - } } From 43af2817007eaa2cce2567bd83f5cde1ee28d1f7 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Fri, 8 Aug 2014 20:58:44 -0700 Subject: [PATCH 0299/1492] [SPARK-2911] apply parent[T](j) to clarify UnionRDD code References to dependencies(j) for actually obtaining RDD parents are less common than I originally estimated. It does clarify UnionRDD (also will clarify some of my other PRs) Use of firstParent[T] is ubiquitous, but not as sure that benefits from being replaced with parent(0)[T]. Author: Erik Erlandson Closes #1858 from erikerlandson/spark-2911-pr2 and squashes the following commits: 7ffea74 [Erik Erlandson] [SPARK-2911] apply parent[T](j) to clarify UnionRDD code --- core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 197167ecad0bd..0c97eb0aaa51f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -83,8 +83,7 @@ class UnionRDD[T: ClassTag]( override def compute(s: Partition, context: TaskContext): Iterator[T] = { val part = s.asInstanceOf[UnionPartition[T]] - val parentRdd = dependencies(part.parentRddIndex).rdd.asInstanceOf[RDD[T]] - parentRdd.iterator(part.parentPartition, context) + parent[T](part.parentRddIndex).iterator(part.parentPartition, context) } override def getPreferredLocations(s: Partition): Seq[String] = From 28dbae85aaf6842e22cd7465cb11cb34d58fc56d Mon Sep 17 00:00:00 2001 From: li-zhihui Date: Fri, 8 Aug 2014 22:52:56 -0700 Subject: [PATCH 0300/1492] [SPARK-2635] Fix race condition at SchedulerBackend.isReady in standalone mode In SPARK-1946(PR #900), configuration spark.scheduler.minRegisteredExecutorsRatio was introduced. However, in standalone mode, there is a race condition where isReady() can return true because totalExpectedExecutors has not been correctly set. Because expected executors is uncertain in standalone mode, the PR try to use CPU cores(--total-executor-cores) as expected resources to judge whether SchedulerBackend is ready. Author: li-zhihui Author: Li Zhihui Closes #1525 from li-zhihui/fixre4s and squashes the following commits: e9a630b [Li Zhihui] Rename variable totalExecutors and clean codes abf4860 [Li Zhihui] Push down variable totalExpectedResources to children classes ca54bd9 [li-zhihui] Format log with String interpolation 88c7dc6 [li-zhihui] Few codes and docs refactor 41cf47e [li-zhihui] Fix race condition at SchedulerBackend.isReady in standalone mode --- .../CoarseGrainedSchedulerBackend.scala | 30 +++++++++---------- .../cluster/SparkDeploySchedulerBackend.scala | 6 +++- docs/configuration.md | 13 ++++---- .../cluster/YarnClientSchedulerBackend.scala | 9 ++++-- .../cluster/YarnClusterSchedulerBackend.scala | 17 +++++++---- 5 files changed, 43 insertions(+), 32 deletions(-) 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 9f085eef46720..33500d967ebb1 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 @@ -47,19 +47,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) + var totalRegisteredExecutors = 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) + // Submit tasks only after (registered resources / total expected resources) // 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). + var minRegisteredRatio = + math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) + // Submit tasks after maxRegisteredWaitingTime milliseconds + // if minRegisteredRatio has not yet been reached val maxRegisteredWaitingTime = - conf.getInt("spark.scheduler.maxRegisteredExecutorsWaitingTime", 30000) + conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 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] @@ -94,12 +94,7 @@ 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) - } + totalRegisteredExecutors.addAndGet(1) makeOffers() } @@ -268,14 +263,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } } + def sufficientResourcesRegistered(): Boolean = true + override def isReady(): Boolean = { - if (ready) { + if (sufficientResourcesRegistered) { + logInfo("SchedulerBackend is ready for scheduling beginning after " + + s"reached minRegisteredResourcesRatio: $minRegisteredRatio") return true } if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { - ready = true logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - "maxRegisteredExecutorsWaitingTime: " + maxRegisteredWaitingTime) + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)") return true } false 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 a28446f6c8a6b..589dba2e40d20 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 @@ -36,6 +36,7 @@ private[spark] class SparkDeploySchedulerBackend( var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ val maxCores = conf.getOption("spark.cores.max").map(_.toInt) + val totalExpectedCores = maxCores.getOrElse(0) override def start() { super.start() @@ -97,7 +98,6 @@ 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))) } @@ -110,4 +110,8 @@ private[spark] class SparkDeploySchedulerBackend( logInfo("Executor %s removed: %s".format(fullId, message)) removeExecutor(fullId.split("/")(1), reason.toString) } + + override def sufficientResourcesRegistered(): Boolean = { + totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio + } } diff --git a/docs/configuration.md b/docs/configuration.md index 4d27c5a918fe0..617a72a021f6e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -825,21 +825,22 @@ Apart from these, the following properties are also available, and may be useful - spark.scheduler.minRegisteredExecutorsRatio + spark.scheduler.minRegisteredResourcesRatio 0 - The minimum ratio of registered executors (registered executors / total expected executors) + The minimum ratio of registered resources (registered resources / total expected resources) + (resources are executors in yarn mode, CPU cores in standalone mode) 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, + Regardless of whether the minimum ratio of resources has been reached, the maximum amount of time it will wait before scheduling begins is controlled by config - spark.scheduler.maxRegisteredExecutorsWaitingTime + spark.scheduler.maxRegisteredResourcesWaitingTime - spark.scheduler.maxRegisteredExecutorsWaitingTime + spark.scheduler.maxRegisteredResourcesWaitingTime 30000 - Maximum amount of time to wait for executors to register before scheduling begins + Maximum amount of time to wait for resources to register before scheduling begins (in milliseconds). 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 f8fb96b312f23..833e249f9f612 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 @@ -30,15 +30,15 @@ private[spark] class YarnClientSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with Logging { - if (conf.getOption("spark.scheduler.minRegisteredExecutorsRatio").isEmpty) { + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { minRegisteredRatio = 0.8 - ready = false } var client: Client = null var appId: ApplicationId = null var checkerThread: Thread = null var stopping: Boolean = false + var totalExpectedExecutors = 0 private[spark] def addArg(optionName: String, envVar: String, sysProp: String, arrayBuf: ArrayBuffer[String]) { @@ -84,7 +84,7 @@ private[spark] class YarnClientSchedulerBackend( logDebug("ClientArguments called with: " + argsArrayBuf) val args = new ClientArguments(argsArrayBuf.toArray, conf) - totalExpectedExecutors.set(args.numExecutors) + totalExpectedExecutors = args.numExecutors client = new Client(args, conf) appId = client.runApp() waitForApp() @@ -150,4 +150,7 @@ private[spark] class YarnClientSchedulerBackend( logInfo("Stopped") } + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio + } } 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 index 0ad1794d19538..55665220a6f96 100644 --- 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 @@ -27,19 +27,24 @@ private[spark] class YarnClusterSchedulerBackend( sc: SparkContext) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { - if (conf.getOption("spark.scheduler.minRegisteredExecutorsRatio").isEmpty) { + var totalExpectedExecutors = 0 + + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { minRegisteredRatio = 0.8 - ready = false } override def start() { super.start() - var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS + totalExpectedExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) { - numExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES")).getOrElse(numExecutors) + totalExpectedExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES")) + .getOrElse(totalExpectedExecutors) } // System property can override environment variable. - numExecutors = sc.getConf.getInt("spark.executor.instances", numExecutors) - totalExpectedExecutors.set(numExecutors) + totalExpectedExecutors = sc.getConf.getInt("spark.executor.instances", totalExpectedExecutors) + } + + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } } From b431e6747f410aaf9624585920adc1f303159861 Mon Sep 17 00:00:00 2001 From: Chandan Kumar Date: Sat, 9 Aug 2014 00:45:54 -0700 Subject: [PATCH 0301/1492] [SPARK-2861] Fix Doc comment of histogram method Tested and ready to merge. Author: Chandan Kumar Closes #1786 from nrchandan/spark-2861 and squashes the following commits: cb0bc1e [Chandan Kumar] [SPARK-2861] Fix a typo in the histogram doc comment 6a2a71b [Chandan Kumar] SPARK-2861. Fix Doc comment of histogram method --- .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 9ca971c8a4c27..f233544d128f5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -119,11 +119,11 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** * Compute a histogram using the provided buckets. The buckets are all open - * to the left except for the last which is closed + * to the right except for the last which is closed * e.g. for the array * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] - * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 + * e.g 1<=x<10 , 10<=x<20, 20<=x<=50 + * And on the input of 1 and 50 we would have a histogram of 1, 0, 1 * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets From e45daf226d780f4a7aaabc2de9f04367bee16f26 Mon Sep 17 00:00:00 2001 From: Chris Cope Date: Sat, 9 Aug 2014 20:58:56 -0700 Subject: [PATCH 0302/1492] [SPARK-1766] sorted functions to meet pedantic requirements Pedantry is underrated Author: Chris Cope Closes #1859 from copester/master and squashes the following commits: 0fb4499 [Chris Cope] [SPARK-1766] sorted functions to meet pedantic requirements --- .../apache/spark/rdd/PairRDDFunctions.scala | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 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 93af50c0a9cd1..5dd6472b0776c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -237,6 +237,25 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) combineByKey[V]((v: V) => v, func, func, partitioner) } + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. + */ + def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { + reduceByKey(new HashPartitioner(numPartitions), func) + } + + /** + * Merge the values for each key using an associative reduce function. This will also perform + * the merging locally on each mapper before sending results to a reducer, similarly to a + * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ + * parallelism level. + */ + def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { + reduceByKey(defaultPartitioner(self), func) + } + /** * Merge the values for each key using an associative reduce function, but return the results * immediately to the master as a Map. This will also perform the merging locally on each mapper @@ -374,15 +393,6 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) countApproxDistinctByKey(relativeSD, defaultPartitioner(self)) } - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with numPartitions partitions. - */ - def reduceByKey(func: (V, V) => V, numPartitions: Int): RDD[(K, V)] = { - reduceByKey(new HashPartitioner(numPartitions), func) - } - /** * 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. @@ -482,16 +492,6 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) combineByKey(createCombiner, mergeValue, mergeCombiners, defaultPartitioner(self)) } - /** - * Merge the values for each key using an associative reduce function. This will also perform - * the merging locally on each mapper before sending results to a reducer, similarly to a - * "combiner" in MapReduce. Output will be hash-partitioned with the existing partitioner/ - * parallelism level. - */ - def reduceByKey(func: (V, V) => V): RDD[(K, V)] = { - reduceByKey(defaultPartitioner(self), func) - } - /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. From 4f4a9884d9268ba9808744b3d612ac23c75f105a Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 9 Aug 2014 21:10:43 -0700 Subject: [PATCH 0303/1492] [SPARK-2894] spark-shell doesn't accept flags As sryza reported, spark-shell doesn't accept any flags. The root cause is wrong usage of spark-submit in spark-shell and it come to the surface by #1801 Author: Kousuke Saruta Author: Cheng Lian Closes #1715, Closes #1864, and Closes #1861 Closes #1825 from sarutak/SPARK-2894 and squashes the following commits: 47f3510 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2894 2c899ed [Kousuke Saruta] Removed useless code from java_gateway.py 98287ed [Kousuke Saruta] Removed useless code from java_gateway.py 513ad2e [Kousuke Saruta] Modified util.sh to enable to use option including white spaces 28a374e [Kousuke Saruta] Modified java_gateway.py to recognize arguments 5afc584 [Cheng Lian] Filter out spark-submit options when starting Python gateway e630d19 [Cheng Lian] Fixing pyspark and spark-shell CLI options --- bin/pyspark | 18 ++++-- bin/spark-shell | 20 +++++-- bin/utils.sh | 59 +++++++++++++++++++ .../spark/deploy/SparkSubmitArguments.scala | 4 ++ dev/merge_spark_pr.py | 2 + python/pyspark/java_gateway.py | 2 +- 6 files changed, 94 insertions(+), 11 deletions(-) create mode 100644 bin/utils.sh diff --git a/bin/pyspark b/bin/pyspark index 39a20e2a24a3c..01d42025c978e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -23,12 +23,18 @@ FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" +source $FWDIR/bin/utils.sh + SCALA_VERSION=2.10 -if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then +function usage() { echo "Usage: ./bin/pyspark [options]" 1>&2 $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 +} + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage fi # Exit if the user hasn't compiled Spark @@ -66,10 +72,11 @@ fi # Build up arguments list manually to preserve quotes and backslashes. # We export Spark submit arguments as an environment variable because shell.py must run as a # PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. - +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" PYSPARK_SUBMIT_ARGS="" whitespace="[[:space:]]" -for i in "$@"; do +for i in "${SUBMISSION_OPTS[@]}"; do if [[ $i =~ \" ]]; then i=$(echo $i | sed 's/\"/\\\"/g'); fi if [[ $i =~ $whitespace ]]; then i=\"$i\"; fi PYSPARK_SUBMIT_ARGS="$PYSPARK_SUBMIT_ARGS $i" @@ -90,7 +97,10 @@ fi if [[ "$1" =~ \.py$ ]]; then echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 echo -e "Use ./bin/spark-submit \n" 1>&2 - exec $FWDIR/bin/spark-submit "$@" + primary=$1 + shift + gatherSparkSubmitOpts "$@" + exec $FWDIR/bin/spark-submit "${SUBMISSION_OPTS[@]}" $primary "${APPLICATION_OPTS[@]}" else # Only use ipython if no command line arguments were provided [SPARK-1134] if [[ "$IPYTHON" = "1" ]]; then diff --git a/bin/spark-shell b/bin/spark-shell index 756c8179d12b6..8b7ccd7439551 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -31,13 +31,21 @@ set -o posix ## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" +function usage() { + echo "Usage: ./bin/spark-shell [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +} + if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./bin/spark-shell [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + usage fi -function main(){ +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" + +function main() { if $cygwin; then # Workaround for issue involving JLine and Cygwin # (see http://sourceforge.net/p/jline/bugs/40/). @@ -46,11 +54,11 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main spark-shell "$@" + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" fi } diff --git a/bin/utils.sh b/bin/utils.sh new file mode 100644 index 0000000000000..0804b1ed9f231 --- /dev/null +++ b/bin/utils.sh @@ -0,0 +1,59 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Gather all all spark-submit options into SUBMISSION_OPTS +function gatherSparkSubmitOpts() { + + if [ -z "$SUBMIT_USAGE_FUNCTION" ]; then + echo "Function for printing usage of $0 is not set." 1>&2 + echo "Please set usage function to shell variable 'SUBMIT_USAGE_FUNCTION' in $0" 1>&2 + exit 1 + fi + + # NOTE: If you add or remove spark-sumbmit options, + # modify NOT ONLY this script but also SparkSubmitArgument.scala + SUBMISSION_OPTS=() + APPLICATION_OPTS=() + while (($#)); do + case "$1" in + --master | --deploy-mode | --class | --name | --jars | --py-files | --files | \ + --conf | --properties-file | --driver-memory | --driver-java-options | \ + --driver-library-path | --driver-class-path | --executor-memory | --driver-cores | \ + --total-executor-cores | --executor-cores | --queue | --num-executors | --archives) + if [[ $# -lt 2 ]]; then + "$SUBMIT_USAGE_FUNCTION" + exit 1; + fi + SUBMISSION_OPTS+=("$1"); shift + SUBMISSION_OPTS+=("$1"); shift + ;; + + --verbose | -v | --supervise) + SUBMISSION_OPTS+=("$1"); shift + ;; + + *) + APPLICATION_OPTS+=("$1"); shift + ;; + esac + done + + export SUBMISSION_OPTS + export APPLICATION_OPTS +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index c21f1529a1837..d545f58c5da7e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -224,6 +224,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { // Delineates parsing of Spark options from parsing of user options. parse(opts) + /** + * NOTE: If you add or remove spark-submit options, + * modify NOT ONLY this file but also utils.sh + */ def parse(opts: Seq[String]): Unit = opts match { case ("--name") :: value :: tail => name = value diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 53df9b5a3f1d5..d48c8bde12905 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -74,8 +74,10 @@ def fail(msg): def run_cmd(cmd): if isinstance(cmd, list): + print " ".join(cmd) return subprocess.check_output(cmd) else: + print cmd return subprocess.check_output(cmd.split(" ")) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 37386ab0d7d49..c7f7c1fe591b0 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -39,7 +39,7 @@ def launch_gateway(): submit_args = os.environ.get("PYSPARK_SUBMIT_ARGS") submit_args = submit_args if submit_args is not None else "" submit_args = shlex.split(submit_args) - command = [os.path.join(SPARK_HOME, script), "pyspark-shell"] + submit_args + command = [os.path.join(SPARK_HOME, script)] + submit_args + ["pyspark-shell"] if not on_windows: # Don't send ctrl-c / SIGINT to the Java gateway: def preexec_func(): From 5b6585de6b939837d5bdc4b1a44634301949add6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 9 Aug 2014 22:05:36 -0700 Subject: [PATCH 0304/1492] Updated Spark SQL README to include the hive-thriftserver module Author: Reynold Xin Closes #1867 from rxin/sql-readme and squashes the following commits: 42a5307 [Reynold Xin] Updated Spark SQL README to include the hive-thriftserver module --- sql/README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/README.md b/sql/README.md index 14d5555f0c713..31f9152344086 100644 --- a/sql/README.md +++ b/sql/README.md @@ -3,10 +3,11 @@ Spark SQL This module provides support for executing relational queries expressed in either SQL or a LINQ-like Scala DSL. -Spark SQL is broken up into three subprojects: +Spark SQL is broken up into four subprojects: - Catalyst (sql/catalyst) - An implementation-agnostic framework for manipulating trees of relational operators and expressions. - Execution (sql/core) - A query planner / execution engine for translating Catalyst’s logical query plans into Spark RDDs. This component also includes a new public interface, SQLContext, that allows users to execute SQL or LINQ statements against existing RDDs and Parquet files. - Hive Support (sql/hive) - Includes an extension of SQLContext called HiveContext that allows users to write queries using a subset of HiveQL and access data from a Hive Metastore using Hive SerDes. There are also wrappers that allows users to run queries that include Hive UDFs, UDAFs, and UDTFs. + - HiveServer and CLI support (sql/hive-thriftserver) - Includes support for the SQL CLI (bin/spark-sql) and a HiveServer2 (for JDBC/ODBC) compatible server. Other dependencies for developers From 482c5afbf6f3f12ac23851300a33249b26ddff3c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 9 Aug 2014 23:06:54 -0700 Subject: [PATCH 0305/1492] Turn UpdateBlockInfo into case class. This helps us log UpdateBlockInfo properly once #1870 is merged. Author: Reynold Xin Closes #1872 from rxin/UpdateBlockInfo and squashes the following commits: 0cee1c2 [Reynold Xin] Turn UpdateBlockInfo into case class. --- .../spark/storage/BlockManagerMessages.scala | 20 +------------------ 1 file changed, 1 insertion(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 10b65286fb7db..2ba16b8476600 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -53,7 +53,7 @@ private[spark] object BlockManagerMessages { sender: ActorRef) extends ToBlockManagerMaster - class UpdateBlockInfo( + case class UpdateBlockInfo( var blockManagerId: BlockManagerId, var blockId: BlockId, var storageLevel: StorageLevel, @@ -84,24 +84,6 @@ private[spark] object BlockManagerMessages { } } - object UpdateBlockInfo { - def apply( - blockManagerId: BlockManagerId, - blockId: BlockId, - storageLevel: StorageLevel, - memSize: Long, - diskSize: Long, - tachyonSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize) - } - - // For pattern-matching - def unapply(h: UpdateBlockInfo) - : Option[(BlockManagerId, BlockId, StorageLevel, Long, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize, h.tachyonSize)) - } - } - case class GetLocations(blockId: BlockId) extends ToBlockManagerMaster case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster From 3570119c34ab8d61507e7703a171b742fb0957d4 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sun, 10 Aug 2014 12:12:22 -0700 Subject: [PATCH 0306/1492] Remove extra semicolon in Task.scala Author: GuoQiang Li Closes #1876 from witgo/remove_semicolon_in_Task_scala and squashes the following commits: c6ea732 [GuoQiang Li] Remove extra semicolon in Task.scala --- core/src/main/scala/org/apache/spark/scheduler/Task.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 5c5e421404a21..cbe0bc0bcb0a5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -46,7 +46,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(attemptId: Long): T = { context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) - context.taskMetrics.hostname = Utils.localHostName(); + context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() if (_killed) { kill(interruptThread = false) From 1d03a26a4895c24ebfab1a3cf6656af75cb53003 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Sun, 10 Aug 2014 12:44:17 -0700 Subject: [PATCH 0307/1492] [SPARK-2950] Add gc time and shuffle write time to JobLogger The JobLogger is very useful for performing offline performance profiling of Spark jobs. GC Time and Shuffle Write time are available in TaskMetrics but are currently missed from the JobLogger output. This patch adds these two fields. ~~Since this is a small change, I didn't create a JIRA. Let me know if I should do that.~~ cc kayousterhout Author: Shivaram Venkataraman Closes #1869 from shivaram/job-logger and squashes the following commits: 1b709fc [Shivaram Venkataraman] Add a space before GC_TIME c418105 [Shivaram Venkataraman] Add gc time and shuffle write time to JobLogger --- .../scala/org/apache/spark/scheduler/JobLogger.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 47dd112f68325..4d6b5c81883b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -162,6 +162,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime + val gcTime = " GC_TIME=" + taskMetrics.jvmGCTime val inputMetrics = taskMetrics.inputMetrics match { case Some(metrics) => " READ_METHOD=" + metrics.readMethod.toString + @@ -179,11 +180,13 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener case None => "" } val writeMetrics = taskMetrics.shuffleWriteMetrics match { - case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten + case Some(metrics) => + " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten + + " SHUFFLE_WRITE_TIME=" + metrics.shuffleWriteTime case None => "" } - stageLogInfo(stageId, status + info + executorRunTime + inputMetrics + shuffleReadMetrics + - writeMetrics) + stageLogInfo(stageId, status + info + executorRunTime + gcTime + inputMetrics + + shuffleReadMetrics + writeMetrics) } /** From 28dcbb531ae57dc50f15ad9df6c31022731669c9 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 10 Aug 2014 13:00:38 -0700 Subject: [PATCH 0308/1492] [SPARK-2898] [PySpark] fix bugs in deamon.py 1. do not use signal handler for SIGCHILD, it's easy to cause deadlock 2. handle EINTR during accept() 3. pass errno into JVM 4. handle EAGAIN during fork() Now, it can pass 50k tasks tests in 180 seconds. Author: Davies Liu Closes #1842 from davies/qa and squashes the following commits: f0ea451 [Davies Liu] fix lint 03a2e8c [Davies Liu] cleanup dead children every seconds 32cb829 [Davies Liu] fix lint 0cd0817 [Davies Liu] fix bugs in deamon.py --- .../api/python/PythonWorkerFactory.scala | 2 +- python/pyspark/daemon.py | 78 +++++++++++-------- 2 files changed, 48 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 7af260d0b7f26..bf716a8ab025b 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -68,7 +68,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String val socket = new Socket(daemonHost, daemonPort) val pid = new DataInputStream(socket.getInputStream).readInt() if (pid < 0) { - throw new IllegalStateException("Python daemon failed to launch worker") + throw new IllegalStateException("Python daemon failed to launch worker with code " + pid) } daemonWorkers.put(socket, pid) socket diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index e73538baf0b93..22ab8d30c0ae3 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -22,7 +22,8 @@ import socket import sys import traceback -from errno import EINTR, ECHILD +import time +from errno import EINTR, ECHILD, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN from pyspark.worker import main as worker_main @@ -80,6 +81,17 @@ def waitSocketClose(sock): os._exit(compute_real_exit_code(exit_code)) +# Cleanup zombie children +def cleanup_dead_children(): + try: + while True: + pid, _ = os.waitpid(0, os.WNOHANG) + if not pid: + break + except: + pass + + def manager(): # Create a new process group to corral our children os.setpgid(0, 0) @@ -102,29 +114,21 @@ def handle_sigterm(*args): signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP - # Cleanup zombie children - def handle_sigchld(*args): - try: - pid, status = os.waitpid(0, os.WNOHANG) - if status != 0: - msg = "worker %s crashed abruptly with exit status %s" % (pid, status) - print >> sys.stderr, msg - except EnvironmentError as err: - if err.errno not in (ECHILD, EINTR): - raise - signal.signal(SIGCHLD, handle_sigchld) - # Initialization complete sys.stdout.close() try: while True: try: - ready_fds = select.select([0, listen_sock], [], [])[0] + ready_fds = select.select([0, listen_sock], [], [], 1)[0] except select.error as ex: if ex[0] == EINTR: continue else: raise + + # cleanup in signal handler will cause deadlock + cleanup_dead_children() + if 0 in ready_fds: try: worker_pid = read_int(sys.stdin) @@ -137,29 +141,41 @@ def handle_sigchld(*args): pass # process already died if listen_sock in ready_fds: - sock, addr = listen_sock.accept() + try: + sock, _ = listen_sock.accept() + except OSError as e: + if e.errno == EINTR: + continue + raise + # Launch a worker process try: pid = os.fork() - if pid == 0: - listen_sock.close() - try: - worker(sock) - except: - traceback.print_exc() - os._exit(1) - else: - os._exit(0) + except OSError as e: + if e.errno in (EAGAIN, EINTR): + time.sleep(1) + pid = os.fork() # error here will shutdown daemon else: + outfile = sock.makefile('w') + write_int(e.errno, outfile) # Signal that the fork failed + outfile.flush() + outfile.close() sock.close() - - except OSError as e: - print >> sys.stderr, "Daemon failed to fork PySpark worker: %s" % e - outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - write_int(-1, outfile) # Signal that the fork failed - outfile.flush() - outfile.close() + continue + + if pid == 0: + # in child process + listen_sock.close() + try: + worker(sock) + except: + traceback.print_exc() + os._exit(1) + else: + os._exit(0) + else: sock.close() + finally: shutdown(1) From b715aa0c8090cd57158ead2a1b35632cb98a6277 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Sun, 10 Aug 2014 16:31:07 -0700 Subject: [PATCH 0309/1492] [SPARK-2937] Separate out samplyByKeyExact as its own API in PairRDDFunction To enable Python consistency and `Experimental` label of the `sampleByKeyExact` API. Author: Doris Xin Author: Xiangrui Meng Closes #1866 from dorx/stratified and squashes the following commits: 0ad97b2 [Doris Xin] reviewer comments. 2948aae [Doris Xin] remove unrelated changes e990325 [Doris Xin] Merge branch 'master' into stratified 555a3f9 [Doris Xin] separate out sampleByKeyExact as its own API 616e55c [Doris Xin] merge master 245439e [Doris Xin] moved minSamplingRate to getUpperBound eaf5771 [Doris Xin] bug fixes. 17a381b [Doris Xin] fixed a merge issue and a failed unit ea7d27f [Doris Xin] merge master b223529 [Xiangrui Meng] use approx bounds for poisson fix poisson mean for waitlisting add unit tests for Java b3013a4 [Xiangrui Meng] move math3 back to test scope eecee5f [Doris Xin] Merge branch 'master' into stratified f4c21f3 [Doris Xin] Reviewer comments a10e68d [Doris Xin] style fix a2bf756 [Doris Xin] Merge branch 'master' into stratified 680b677 [Doris Xin] use mapPartitionWithIndex instead 9884a9f [Doris Xin] style fix bbfb8c9 [Doris Xin] Merge branch 'master' into stratified ee9d260 [Doris Xin] addressed reviewer comments 6b5b10b [Doris Xin] Merge branch 'master' into stratified 254e03c [Doris Xin] minor fixes and Java API. 4ad516b [Doris Xin] remove unused imports from PairRDDFunctions bd9dc6e [Doris Xin] unit bug and style violation fixed 1fe1cff [Doris Xin] Changed fractionByKey to a map to enable arg check 944a10c [Doris Xin] [SPARK-2145] Add lower bound on sampling rate 0214a76 [Doris Xin] cleanUp 90d94c0 [Doris Xin] merge master 9e74ab5 [Doris Xin] Separated out most of the logic in sampleByKey 7327611 [Doris Xin] merge master 50581fc [Doris Xin] added a TODO for logging in python 46f6c8c [Doris Xin] fixed the NPE caused by closures being cleaned before being passed into the aggregate function 7e1a481 [Doris Xin] changed the permission on SamplingUtil 1d413ce [Doris Xin] fixed checkstyle issues 9ee94ee [Doris Xin] [SPARK-2082] stratified sampling in PairRDDFunctions that guarantees exact sample size e3fd6a6 [Doris Xin] Merge branch 'master' into takeSample 7cab53a [Doris Xin] fixed import bug in rdd.py ffea61a [Doris Xin] SPARK-1939: Refactor takeSample method in RDD 1441977 [Doris Xin] SPARK-1939 Refactor takeSample method in RDD to use ScaSRS --- .../apache/spark/api/java/JavaPairRDD.scala | 68 +++--- .../apache/spark/rdd/PairRDDFunctions.scala | 51 +++-- .../java/org/apache/spark/JavaAPISuite.java | 20 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 205 +++++++++++------- 4 files changed, 216 insertions(+), 128 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 76d4193e96aea..feeb6c02caa78 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -133,68 +133,62 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return a subset of this RDD sampled by key (via stratified sampling). * * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. - * - * If `exact` is set to false, create the sample via simple random sampling, with one pass - * over the RDD, to produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over - * the RDD to create a sample size that's exactly equal to the sum of + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of * math.ceil(numItems * samplingRate) over all key values. */ def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double], - exact: Boolean, seed: Long): JavaPairRDD[K, V] = - new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, exact, seed)) + new JavaPairRDD[K, V](rdd.sampleByKey(withReplacement, fractions, seed)) /** * Return a subset of this RDD sampled by key (via stratified sampling). * * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. - * - * If `exact` is set to false, create the sample via simple random sampling, with one pass - * over the RDD, to produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values; otherwise, use additional passes over - * the RDD to create a sample size that's exactly equal to the sum of + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of * math.ceil(numItems * samplingRate) over all key values. * - * Use Utils.random.nextLong as the default seed for the random number generator + * Use Utils.random.nextLong as the default seed for the random number generator. */ def sampleByKey(withReplacement: Boolean, - fractions: JMap[K, Double], - exact: Boolean): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, exact, Utils.random.nextLong) + fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKey(withReplacement, fractions, Utils.random.nextLong) /** - * Return a subset of this RDD sampled by key (via stratified sampling). - * - * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. + * ::Experimental:: + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * - * Produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via - * simple random sampling. + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. */ - def sampleByKey(withReplacement: Boolean, + @Experimental + def sampleByKeyExact(withReplacement: Boolean, fractions: JMap[K, Double], seed: Long): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, false, seed) + new JavaPairRDD[K, V](rdd.sampleByKeyExact(withReplacement, fractions, seed)) /** - * Return a subset of this RDD sampled by key (via stratified sampling). + * ::Experimental:: + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). * - * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. - * - * Produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values with one pass over the RDD via - * simple random sampling. + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. * - * Use Utils.random.nextLong as the default seed for the random number generator + * Use Utils.random.nextLong as the default seed for the random number generator. */ - def sampleByKey(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = - sampleByKey(withReplacement, fractions, false, Utils.random.nextLong) + @Experimental + def sampleByKeyExact(withReplacement: Boolean, fractions: JMap[K, Double]): JavaPairRDD[K, V] = + sampleByKeyExact(withReplacement, fractions, Utils.random.nextLong) /** * Return the union of this RDD and another one. Any identical elements will appear multiple 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 5dd6472b0776c..f6d9d12fe9006 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -197,33 +197,56 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * Return a subset of this RDD sampled by key (via stratified sampling). * * Create a sample of this RDD using variable sampling rates for different keys as specified by - * `fractions`, a key to sampling rate map. - * - * If `exact` is set to false, create the sample via simple random sampling, with one pass - * over the RDD, to produce a sample of size that's approximately equal to the sum of - * math.ceil(numItems * samplingRate) over all key values; otherwise, use - * additional passes over the RDD to create a sample size that's exactly equal to the sum of - * math.ceil(numItems * samplingRate) over all key values with a 99.99% confidence. When sampling - * without replacement, we need one additional pass over the RDD to guarantee sample size; - * when sampling with replacement, we need two additional passes. + * `fractions`, a key to sampling rate map, via simple random sampling with one pass over the + * RDD, to produce a sample of size that's approximately equal to the sum of + * math.ceil(numItems * samplingRate) over all key values. * * @param withReplacement whether to sample with or without replacement * @param fractions map of specific keys to sampling rates * @param seed seed for the random number generator - * @param exact whether sample size needs to be exactly math.ceil(fraction * size) per key * @return RDD containing the sampled subset */ def sampleByKey(withReplacement: Boolean, fractions: Map[K, Double], - exact: Boolean = false, - seed: Long = Utils.random.nextLong): RDD[(K, V)]= { + seed: Long = Utils.random.nextLong): RDD[(K, V)] = { + + require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") + + val samplingFunc = if (withReplacement) { + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, false, seed) + } else { + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, false, seed) + } + self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) + } + + /** + * ::Experimental:: + * Return a subset of this RDD sampled by key (via stratified sampling) containing exactly + * math.ceil(numItems * samplingRate) for each stratum (group of pairs with the same key). + * + * This method differs from [[sampleByKey]] in that we make additional passes over the RDD to + * create a sample size that's exactly equal to the sum of math.ceil(numItems * samplingRate) + * over all key values with a 99.99% confidence. When sampling without replacement, we need one + * additional pass over the RDD to guarantee sample size; when sampling with replacement, we need + * two additional passes. + * + * @param withReplacement whether to sample with or without replacement + * @param fractions map of specific keys to sampling rates + * @param seed seed for the random number generator + * @return RDD containing the sampled subset + */ + @Experimental + def sampleByKeyExact(withReplacement: Boolean, + fractions: Map[K, Double], + seed: Long = Utils.random.nextLong): RDD[(K, V)] = { require(fractions.values.forall(v => v >= 0.0), "Negative sampling rates.") val samplingFunc = if (withReplacement) { - StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, exact, seed) + StratifiedSamplingUtils.getPoissonSamplingFunction(self, fractions, true, seed) } else { - StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, exact, seed) + StratifiedSamplingUtils.getBernoulliSamplingFunction(self, fractions, true, seed) } self.mapPartitionsWithIndex(samplingFunc, preservesPartitioning = true) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 56150caa5d6ba..e1c13de04a0be 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1239,12 +1239,28 @@ public Tuple2 call(Integer i) { Assert.assertTrue(worCounts.size() == 2); Assert.assertTrue(worCounts.get(0) > 0); Assert.assertTrue(worCounts.get(1) > 0); - JavaPairRDD wrExact = rdd2.sampleByKey(true, fractions, true, 1L); + } + + @Test + @SuppressWarnings("unchecked") + public void sampleByKeyExact() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8), 3); + JavaPairRDD rdd2 = rdd1.mapToPair( + new PairFunction() { + @Override + public Tuple2 call(Integer i) { + return new Tuple2(i % 2, 1); + } + }); + Map fractions = Maps.newHashMap(); + fractions.put(0, 0.5); + fractions.put(1, 1.0); + JavaPairRDD wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); Map wrExactCounts = (Map) (Object) wrExact.countByKey(); Assert.assertTrue(wrExactCounts.size() == 2); Assert.assertTrue(wrExactCounts.get(0) == 2); Assert.assertTrue(wrExactCounts.get(1) == 4); - JavaPairRDD worExact = rdd2.sampleByKey(false, fractions, true, 1L); + JavaPairRDD worExact = rdd2.sampleByKeyExact(false, fractions, 1L); Map worExactCounts = (Map) (Object) worExact.countByKey(); Assert.assertTrue(worExactCounts.size() == 2); Assert.assertTrue(worExactCounts.get(0) == 2); diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 4f49d4a1d4d34..63d3ddb4af98a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -84,118 +84,81 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } test("sampleByKey") { - def stratifier (fractionPositive: Double) = { - (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" - } - def checkSize(exact: Boolean, - withReplacement: Boolean, - expected: Long, - actual: Long, - p: Double): Boolean = { - if (exact) { - return expected == actual - } - val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) - // Very forgiving margin since we're dealing with very small sample sizes most of the time - math.abs(actual - expected) <= 6 * stdev + val defaultSeed = 1L + + // vary RDD size + for (n <- List(100, 1000, 1000000)) { + val data = sc.parallelize(1 to n, 2) + val fractionPositive = 0.3 + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) } - // Without replacement validation - def takeSampleAndValidateBernoulli(stratifiedData: RDD[(String, Int)], - exact: Boolean, - samplingRate: Double, - seed: Long, - n: Long) = { - val expectedSampleSize = stratifiedData.countByKey() - .mapValues(count => math.ceil(count * samplingRate).toInt) - val fractions = Map("1" -> samplingRate, "0" -> samplingRate) - val sample = stratifiedData.sampleByKey(false, fractions, exact, seed) - val sampleCounts = sample.countByKey() - val takeSample = sample.collect() - sampleCounts.foreach { case(k, v) => - assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } - assert(takeSample.size === takeSample.toSet.size) - takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + // vary fractionPositive + for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) } - // With replacement validation - def takeSampleAndValidatePoisson(stratifiedData: RDD[(String, Int)], - exact: Boolean, - samplingRate: Double, - seed: Long, - n: Long) = { - val expectedSampleSize = stratifiedData.countByKey().mapValues(count => - math.ceil(count * samplingRate).toInt) - val fractions = Map("1" -> samplingRate, "0" -> samplingRate) - val sample = stratifiedData.sampleByKey(true, fractions, exact, seed) - val sampleCounts = sample.countByKey() - val takeSample = sample.collect() - sampleCounts.foreach { case(k, v) => - assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) } - val groupedByKey = takeSample.groupBy(_._1) - for ((key, v) <- groupedByKey) { - if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { - // sample large enough for there to be repeats with high likelihood - assert(v.toSet.size < expectedSampleSize(key)) - } else { - if (exact) { - assert(v.toSet.size <= expectedSampleSize(key)) - } else { - assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) - } - } - } - takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + // Use the same data for the rest of the tests + val fractionPositive = 0.3 + val n = 100 + val data = sc.parallelize(1 to n, 2) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) + + // vary seed + for (seed <- defaultSeed to defaultSeed + 5L) { + val samplingRate = 0.1 + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, seed, n) } - def checkAllCombos(stratifiedData: RDD[(String, Int)], - samplingRate: Double, - seed: Long, - n: Long) = { - takeSampleAndValidateBernoulli(stratifiedData, true, samplingRate, seed, n) - takeSampleAndValidateBernoulli(stratifiedData, false, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, true, samplingRate, seed, n) - takeSampleAndValidatePoisson(stratifiedData, false, samplingRate, seed, n) + // vary sampling rate + for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { + StratifiedAuxiliary.testSample(stratifiedData, samplingRate, defaultSeed, n) } + } + test("sampleByKeyExact") { val defaultSeed = 1L // vary RDD size for (n <- List(100, 1000, 1000000)) { val data = sc.parallelize(1 to n, 2) val fractionPositive = 0.3 - val stratifiedData = data.keyBy(stratifier(fractionPositive)) - + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) val samplingRate = 0.1 - checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) } // vary fractionPositive for (fractionPositive <- List(0.1, 0.3, 0.5, 0.7, 0.9)) { val n = 100 val data = sc.parallelize(1 to n, 2) - val stratifiedData = data.keyBy(stratifier(fractionPositive)) - + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) val samplingRate = 0.1 - checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) } // Use the same data for the rest of the tests val fractionPositive = 0.3 val n = 100 val data = sc.parallelize(1 to n, 2) - val stratifiedData = data.keyBy(stratifier(fractionPositive)) + val stratifiedData = data.keyBy(StratifiedAuxiliary.stratifier(fractionPositive)) // vary seed for (seed <- defaultSeed to defaultSeed + 5L) { val samplingRate = 0.1 - checkAllCombos(stratifiedData, samplingRate, seed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, seed, n) } // vary sampling rate for (samplingRate <- List(0.01, 0.05, 0.1, 0.5)) { - checkAllCombos(stratifiedData, samplingRate, defaultSeed, n) + StratifiedAuxiliary.testSampleExact(stratifiedData, samplingRate, defaultSeed, n) } } @@ -556,6 +519,98 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { intercept[IllegalArgumentException] {shuffled.lookup(-1)} } + private object StratifiedAuxiliary { + def stratifier (fractionPositive: Double) = { + (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" + } + + def checkSize(exact: Boolean, + withReplacement: Boolean, + expected: Long, + actual: Long, + p: Double): Boolean = { + if (exact) { + return expected == actual + } + val stdev = if (withReplacement) math.sqrt(expected) else math.sqrt(expected * p * (1 - p)) + // Very forgiving margin since we're dealing with very small sample sizes most of the time + math.abs(actual - expected) <= 6 * stdev + } + + def testSampleExact(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { + testBernoulli(stratifiedData, true, samplingRate, seed, n) + testPoisson(stratifiedData, true, samplingRate, seed, n) + } + + def testSample(stratifiedData: RDD[(String, Int)], + samplingRate: Double, + seed: Long, + n: Long) = { + testBernoulli(stratifiedData, false, samplingRate, seed, n) + testPoisson(stratifiedData, false, samplingRate, seed, n) + } + + // Without replacement validation + def testBernoulli(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey() + .mapValues(count => math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = if (exact) { + stratifiedData.sampleByKeyExact(false, fractions, seed) + } else { + stratifiedData.sampleByKey(false, fractions, seed) + } + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case(k, v) => + assert(checkSize(exact, false, expectedSampleSize(k), v, samplingRate)) } + assert(takeSample.size === takeSample.toSet.size) + takeSample.foreach { x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]") } + } + + // With replacement validation + def testPoisson(stratifiedData: RDD[(String, Int)], + exact: Boolean, + samplingRate: Double, + seed: Long, + n: Long) = { + val expectedSampleSize = stratifiedData.countByKey().mapValues(count => + math.ceil(count * samplingRate).toInt) + val fractions = Map("1" -> samplingRate, "0" -> samplingRate) + val sample = if (exact) { + stratifiedData.sampleByKeyExact(true, fractions, seed) + } else { + stratifiedData.sampleByKey(true, fractions, seed) + } + val sampleCounts = sample.countByKey() + val takeSample = sample.collect() + sampleCounts.foreach { case (k, v) => + assert(checkSize(exact, true, expectedSampleSize(k), v, samplingRate)) + } + val groupedByKey = takeSample.groupBy(_._1) + for ((key, v) <- groupedByKey) { + if (expectedSampleSize(key) >= 100 && samplingRate >= 0.1) { + // sample large enough for there to be repeats with high likelihood + assert(v.toSet.size < expectedSampleSize(key)) + } else { + if (exact) { + assert(v.toSet.size <= expectedSampleSize(key)) + } else { + assert(checkSize(false, true, expectedSampleSize(key), v.toSet.size, samplingRate)) + } + } + } + takeSample.foreach(x => assert(1 <= x._2 && x._2 <= n, s"elements not in [1, $n]")) + } + } + } /* From ba28a8fcbc3ba432e7ea4d6f0b535450a6ec96c6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 10 Aug 2014 20:36:54 -0700 Subject: [PATCH 0310/1492] [SPARK-2936] Migrate Netty network module from Java to Scala The Netty network module was originally written when Scala 2.9.x had a bug that prevents a pure Scala implementation, and a subset of the files were done in Java. We have since upgraded to Scala 2.10, and can migrate all Java files now to Scala. https://github.com/netty/netty/issues/781 https://github.com/mesos/spark/pull/522 Author: Reynold Xin Closes #1865 from rxin/netty and squashes the following commits: 332422f [Reynold Xin] Code review feedback ca9eeee [Reynold Xin] Minor update. 7f1434b [Reynold Xin] [SPARK-2936] Migrate Netty network module from Java to Scala --- .../spark/network/netty/FileClient.java | 100 ---------------- .../spark/network/netty/FileServer.java | 111 ------------------ .../network/netty/FileServerHandler.java | 83 ------------- .../spark/network/netty/FileClient.scala | 85 ++++++++++++++ .../netty/FileClientChannelInitializer.scala} | 24 ++-- .../network/netty/FileClientHandler.scala} | 47 ++++---- .../spark/network/netty/FileHeader.scala | 5 +- .../spark/network/netty/FileServer.scala | 91 ++++++++++++++ .../netty/FileServerChannelInitializer.scala} | 31 ++--- .../network/netty/FileServerHandler.scala | 68 +++++++++++ .../spark/network/netty/PathResolver.scala} | 9 +- .../spark/network/netty/ShuffleSender.scala | 2 +- 12 files changed, 292 insertions(+), 364 deletions(-) delete mode 100644 core/src/main/java/org/apache/spark/network/netty/FileClient.java delete mode 100644 core/src/main/java/org/apache/spark/network/netty/FileServer.java delete mode 100644 core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClient.scala rename core/src/main/{java/org/apache/spark/network/netty/FileClientChannelInitializer.java => scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala} (57%) rename core/src/main/{java/org/apache/spark/network/netty/FileClientHandler.java => scala/org/apache/spark/network/netty/FileClientHandler.scala} (51%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServer.scala rename core/src/main/{java/org/apache/spark/network/netty/FileServerChannelInitializer.java => scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala} (54%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala rename core/src/main/{java/org/apache/spark/network/netty/PathResolver.java => scala/org/apache/spark/network/netty/PathResolver.scala} (80%) mode change 100755 => 100644 diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClient.java b/core/src/main/java/org/apache/spark/network/netty/FileClient.java deleted file mode 100644 index 0d31894d6ec7a..0000000000000 --- a/core/src/main/java/org/apache/spark/network/netty/FileClient.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty; - -import java.util.concurrent.TimeUnit; - -import io.netty.bootstrap.Bootstrap; -import io.netty.channel.Channel; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.oio.OioEventLoopGroup; -import io.netty.channel.socket.oio.OioSocketChannel; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class FileClient { - - private static final Logger LOG = LoggerFactory.getLogger(FileClient.class.getName()); - - private final FileClientHandler handler; - private Channel channel = null; - private Bootstrap bootstrap = null; - private EventLoopGroup group = null; - private final int connectTimeout; - private final int sendTimeout = 60; // 1 min - - FileClient(FileClientHandler handler, int connectTimeout) { - this.handler = handler; - this.connectTimeout = connectTimeout; - } - - public void init() { - group = new OioEventLoopGroup(); - bootstrap = new Bootstrap(); - bootstrap.group(group) - .channel(OioSocketChannel.class) - .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.TCP_NODELAY, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, connectTimeout) - .handler(new FileClientChannelInitializer(handler)); - } - - public void connect(String host, int port) { - try { - // Start the connection attempt. - channel = bootstrap.connect(host, port).sync().channel(); - // ChannelFuture cf = channel.closeFuture(); - //cf.addListener(new ChannelCloseListener(this)); - } catch (InterruptedException e) { - LOG.warn("FileClient interrupted while trying to connect", e); - close(); - } - } - - public void waitForClose() { - try { - channel.closeFuture().sync(); - } catch (InterruptedException e) { - LOG.warn("FileClient interrupted", e); - } - } - - public void sendRequest(String file) { - //assert(file == null); - //assert(channel == null); - try { - // Should be able to send the message to network link channel. - boolean bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS); - if (!bSent) { - throw new RuntimeException("Failed to send"); - } - } catch (InterruptedException e) { - LOG.error("Error", e); - } - } - - public void close() { - if (group != null) { - group.shutdownGracefully(); - group = null; - bootstrap = null; - } - } -} diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServer.java b/core/src/main/java/org/apache/spark/network/netty/FileServer.java deleted file mode 100644 index c93425e2787dc..0000000000000 --- a/core/src/main/java/org/apache/spark/network/netty/FileServer.java +++ /dev/null @@ -1,111 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty; - -import java.net.InetSocketAddress; - -import io.netty.bootstrap.ServerBootstrap; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelOption; -import io.netty.channel.EventLoopGroup; -import io.netty.channel.oio.OioEventLoopGroup; -import io.netty.channel.socket.oio.OioServerSocketChannel; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Server that accept the path of a file an echo back its content. - */ -class FileServer { - - private static final Logger LOG = LoggerFactory.getLogger(FileServer.class.getName()); - - private EventLoopGroup bossGroup = null; - private EventLoopGroup workerGroup = null; - private ChannelFuture channelFuture = null; - private int port = 0; - - FileServer(PathResolver pResolver, int port) { - InetSocketAddress addr = new InetSocketAddress(port); - - // Configure the server. - bossGroup = new OioEventLoopGroup(); - workerGroup = new OioEventLoopGroup(); - - ServerBootstrap bootstrap = new ServerBootstrap(); - bootstrap.group(bossGroup, workerGroup) - .channel(OioServerSocketChannel.class) - .option(ChannelOption.SO_BACKLOG, 100) - .option(ChannelOption.SO_RCVBUF, 1500) - .childHandler(new FileServerChannelInitializer(pResolver)); - // Start the server. - channelFuture = bootstrap.bind(addr); - try { - // Get the address we bound to. - InetSocketAddress boundAddress = - ((InetSocketAddress) channelFuture.sync().channel().localAddress()); - this.port = boundAddress.getPort(); - } catch (InterruptedException ie) { - this.port = 0; - } - } - - /** - * Start the file server asynchronously in a new thread. - */ - public void start() { - Thread blockingThread = new Thread() { - @Override - public void run() { - try { - channelFuture.channel().closeFuture().sync(); - LOG.info("FileServer exiting"); - } catch (InterruptedException e) { - LOG.error("File server start got interrupted", e); - } - // NOTE: bootstrap is shutdown in stop() - } - }; - blockingThread.setDaemon(true); - blockingThread.start(); - } - - public int getPort() { - return port; - } - - public void stop() { - // Close the bound channel. - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly(); - channelFuture = null; - } - - // Shutdown event groups - if (bossGroup != null) { - bossGroup.shutdownGracefully(); - bossGroup = null; - } - - if (workerGroup != null) { - workerGroup.shutdownGracefully(); - workerGroup = null; - } - // TODO: Shutdown all accepted channels as well ? - } -} diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java b/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java deleted file mode 100644 index c0133e19c7f79..0000000000000 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerHandler.java +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty; - -import java.io.File; -import java.io.FileInputStream; - -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; -import io.netty.channel.DefaultFileRegion; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.FileSegment; - -class FileServerHandler extends SimpleChannelInboundHandler { - - private static final Logger LOG = LoggerFactory.getLogger(FileServerHandler.class.getName()); - - private final PathResolver pResolver; - - FileServerHandler(PathResolver pResolver){ - this.pResolver = pResolver; - } - - @Override - public void channelRead0(ChannelHandlerContext ctx, String blockIdString) { - BlockId blockId = BlockId.apply(blockIdString); - FileSegment fileSegment = pResolver.getBlockLocation(blockId); - // if getBlockLocation returns null, close the channel - if (fileSegment == null) { - //ctx.close(); - return; - } - File file = fileSegment.file(); - if (file.exists()) { - if (!file.isFile()) { - ctx.write(new FileHeader(0, blockId).buffer()); - ctx.flush(); - return; - } - long length = fileSegment.length(); - if (length > Integer.MAX_VALUE || length <= 0) { - ctx.write(new FileHeader(0, blockId).buffer()); - ctx.flush(); - return; - } - int len = (int) length; - ctx.write((new FileHeader(len, blockId)).buffer()); - try { - ctx.write(new DefaultFileRegion(new FileInputStream(file) - .getChannel(), fileSegment.offset(), fileSegment.length())); - } catch (Exception e) { - LOG.error("Exception: ", e); - } - } else { - ctx.write(new FileHeader(0, blockId).buffer()); - } - ctx.flush(); - } - - @Override - public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { - LOG.error("Exception: ", cause); - ctx.close(); - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala new file mode 100644 index 0000000000000..c6d35f73db545 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.util.concurrent.TimeUnit + +import io.netty.bootstrap.Bootstrap +import io.netty.channel.{Channel, ChannelOption, EventLoopGroup} +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.oio.OioSocketChannel + +import org.apache.spark.Logging + +class FileClient(handler: FileClientHandler, connectTimeout: Int) extends Logging { + + private var channel: Channel = _ + private var bootstrap: Bootstrap = _ + private var group: EventLoopGroup = _ + private val sendTimeout = 60 + + def init(): Unit = { + group = new OioEventLoopGroup + bootstrap = new Bootstrap + bootstrap.group(group) + .channel(classOf[OioSocketChannel]) + .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) + .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, Integer.valueOf(connectTimeout)) + .handler(new FileClientChannelInitializer(handler)) + } + + def connect(host: String, port: Int) { + try { + channel = bootstrap.connect(host, port).sync().channel() + } catch { + case e: InterruptedException => + logWarning("FileClient interrupted while trying to connect", e) + close() + } + } + + def waitForClose(): Unit = { + try { + channel.closeFuture.sync() + } catch { + case e: InterruptedException => + logWarning("FileClient interrupted", e) + } + } + + def sendRequest(file: String): Unit = { + try { + val bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS) + if (!bSent) { + throw new RuntimeException("Failed to send") + } + } catch { + case e: InterruptedException => + logError("Error", e) + } + } + + def close(): Unit = { + if (group != null) { + group.shutdownGracefully() + group = null + bootstrap = null + } + } +} diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java b/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala similarity index 57% rename from core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java rename to core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala index 264cf97d0209f..f4261c13f70a8 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientChannelInitializer.java +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala @@ -15,25 +15,17 @@ * limitations under the License. */ -package org.apache.spark.network.netty; +package org.apache.spark.network.netty -import io.netty.channel.ChannelInitializer; -import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.string.StringEncoder; +import io.netty.channel.ChannelInitializer +import io.netty.channel.socket.SocketChannel +import io.netty.handler.codec.string.StringEncoder -class FileClientChannelInitializer extends ChannelInitializer { - private final FileClientHandler fhandler; +class FileClientChannelInitializer(handler: FileClientHandler) + extends ChannelInitializer[SocketChannel] { - FileClientChannelInitializer(FileClientHandler handler) { - fhandler = handler; - } - - @Override - public void initChannel(SocketChannel channel) { - // file no more than 2G - channel.pipeline() - .addLast("encoder", new StringEncoder()) - .addLast("handler", fhandler); + def initChannel(channel: SocketChannel) { + channel.pipeline.addLast("encoder", new StringEncoder).addLast("handler", handler) } } diff --git a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala similarity index 51% rename from core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java rename to core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala index 63d3d927255f9..017302ec7d33d 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileClientHandler.java +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala @@ -15,41 +15,36 @@ * limitations under the License. */ -package org.apache.spark.network.netty; +package org.apache.spark.network.netty -import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelHandlerContext; -import io.netty.channel.SimpleChannelInboundHandler; +import io.netty.buffer.ByteBuf +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} -import org.apache.spark.storage.BlockId; +import org.apache.spark.storage.BlockId -abstract class FileClientHandler extends SimpleChannelInboundHandler { - private FileHeader currentHeader = null; +abstract class FileClientHandler extends SimpleChannelInboundHandler[ByteBuf] { - private volatile boolean handlerCalled = false; + private var currentHeader: FileHeader = null - public boolean isComplete() { - return handlerCalled; - } + @volatile + private var handlerCalled: Boolean = false + + def isComplete: Boolean = handlerCalled + + def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) - public abstract void handle(ChannelHandlerContext ctx, ByteBuf in, FileHeader header); - public abstract void handleError(BlockId blockId); + def handleError(blockId: BlockId) - @Override - public void channelRead0(ChannelHandlerContext ctx, ByteBuf in) { - // get header - if (currentHeader == null && in.readableBytes() >= FileHeader.HEADER_SIZE()) { - currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE())); + override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { + if (currentHeader == null && in.readableBytes >= FileHeader.HEADER_SIZE) { + currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE)) } - // get file - if(in.readableBytes() >= currentHeader.fileLen()) { - handle(ctx, in, currentHeader); - handlerCalled = true; - currentHeader = null; - ctx.close(); + if (in.readableBytes >= currentHeader.fileLen) { + handle(ctx, in, currentHeader) + handlerCalled = true + currentHeader = null + ctx.close() } } - } - diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index 136c1912045aa..607e560ff277f 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -26,7 +26,7 @@ private[spark] class FileHeader ( val fileLen: Int, val blockId: BlockId) extends Logging { - lazy val buffer = { + lazy val buffer: ByteBuf = { val buf = Unpooled.buffer() buf.capacity(FileHeader.HEADER_SIZE) buf.writeInt(fileLen) @@ -62,11 +62,10 @@ private[spark] object FileHeader { new FileHeader(length, blockId) } - def main (args:Array[String]) { + def main(args:Array[String]) { val header = new FileHeader(25, TestBlockId("my_block")) val buf = header.buffer val newHeader = FileHeader.create(buf) System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen) } } - diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala new file mode 100644 index 0000000000000..dff77950659af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.net.InetSocketAddress + +import io.netty.bootstrap.ServerBootstrap +import io.netty.channel.{ChannelFuture, ChannelOption, EventLoopGroup} +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.oio.OioServerSocketChannel + +import org.apache.spark.Logging + +/** + * Server that accept the path of a file an echo back its content. + */ +class FileServer(pResolver: PathResolver, private var port: Int) extends Logging { + + private val addr: InetSocketAddress = new InetSocketAddress(port) + private var bossGroup: EventLoopGroup = new OioEventLoopGroup + private var workerGroup: EventLoopGroup = new OioEventLoopGroup + + private var channelFuture: ChannelFuture = { + val bootstrap = new ServerBootstrap + bootstrap.group(bossGroup, workerGroup) + .channel(classOf[OioServerSocketChannel]) + .option(ChannelOption.SO_BACKLOG, java.lang.Integer.valueOf(100)) + .option(ChannelOption.SO_RCVBUF, java.lang.Integer.valueOf(1500)) + .childHandler(new FileServerChannelInitializer(pResolver)) + bootstrap.bind(addr) + } + + try { + val boundAddress = channelFuture.sync.channel.localAddress.asInstanceOf[InetSocketAddress] + port = boundAddress.getPort + } catch { + case ie: InterruptedException => + port = 0 + } + + /** Start the file server asynchronously in a new thread. */ + def start(): Unit = { + val blockingThread: Thread = new Thread { + override def run(): Unit = { + try { + channelFuture.channel.closeFuture.sync + logInfo("FileServer exiting") + } catch { + case e: InterruptedException => + logError("File server start got interrupted", e) + } + // NOTE: bootstrap is shutdown in stop() + } + } + blockingThread.setDaemon(true) + blockingThread.start() + } + + def getPort: Int = port + + def stop(): Unit = { + if (channelFuture != null) { + channelFuture.channel().close().awaitUninterruptibly() + channelFuture = null + } + if (bossGroup != null) { + bossGroup.shutdownGracefully() + bossGroup = null + } + if (workerGroup != null) { + workerGroup.shutdownGracefully() + workerGroup = null + } + } +} + diff --git a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java b/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala similarity index 54% rename from core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java rename to core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala index 46efec8f8d963..aaa2f913d0269 100644 --- a/core/src/main/java/org/apache/spark/network/netty/FileServerChannelInitializer.java +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala @@ -15,27 +15,20 @@ * limitations under the License. */ -package org.apache.spark.network.netty; +package org.apache.spark.network.netty -import io.netty.channel.ChannelInitializer; -import io.netty.channel.socket.SocketChannel; -import io.netty.handler.codec.DelimiterBasedFrameDecoder; -import io.netty.handler.codec.Delimiters; -import io.netty.handler.codec.string.StringDecoder; +import io.netty.channel.ChannelInitializer +import io.netty.channel.socket.SocketChannel +import io.netty.handler.codec.{DelimiterBasedFrameDecoder, Delimiters} +import io.netty.handler.codec.string.StringDecoder -class FileServerChannelInitializer extends ChannelInitializer { +class FileServerChannelInitializer(pResolver: PathResolver) + extends ChannelInitializer[SocketChannel] { - private final PathResolver pResolver; - - FileServerChannelInitializer(PathResolver pResolver) { - this.pResolver = pResolver; - } - - @Override - public void initChannel(SocketChannel channel) { - channel.pipeline() - .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter())) - .addLast("stringDecoder", new StringDecoder()) - .addLast("handler", new FileServerHandler(pResolver)); + override def initChannel(channel: SocketChannel): Unit = { + channel.pipeline + .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter : _*)) + .addLast("stringDecoder", new StringDecoder) + .addLast("handler", new FileServerHandler(pResolver)) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala new file mode 100644 index 0000000000000..96f60b2883ad9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.io.FileInputStream + +import io.netty.channel.{DefaultFileRegion, ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging +import org.apache.spark.storage.{BlockId, FileSegment} + + +class FileServerHandler(pResolver: PathResolver) + extends SimpleChannelInboundHandler[String] with Logging { + + override def channelRead0(ctx: ChannelHandlerContext, blockIdString: String): Unit = { + val blockId: BlockId = BlockId(blockIdString) + val fileSegment: FileSegment = pResolver.getBlockLocation(blockId) + if (fileSegment == null) { + return + } + val file = fileSegment.file + if (file.exists) { + if (!file.isFile) { + ctx.write(new FileHeader(0, blockId).buffer) + ctx.flush() + return + } + val length: Long = fileSegment.length + if (length > Integer.MAX_VALUE || length <= 0) { + ctx.write(new FileHeader(0, blockId).buffer) + ctx.flush() + return + } + ctx.write(new FileHeader(length.toInt, blockId).buffer) + try { + val channel = new FileInputStream(file).getChannel + ctx.write(new DefaultFileRegion(channel, fileSegment.offset, fileSegment.length)) + } catch { + case e: Exception => + logError("Exception: ", e) + } + } else { + ctx.write(new FileHeader(0, blockId).buffer) + } + ctx.flush() + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + logError("Exception: ", cause) + ctx.close() + } +} diff --git a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java b/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala old mode 100755 new mode 100644 similarity index 80% rename from core/src/main/java/org/apache/spark/network/netty/PathResolver.java rename to core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala index 7ad8d03efbadc..0d7695072a7b1 --- a/core/src/main/java/org/apache/spark/network/netty/PathResolver.java +++ b/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.network.netty; +package org.apache.spark.network.netty -import org.apache.spark.storage.BlockId; -import org.apache.spark.storage.FileSegment; +import org.apache.spark.storage.{BlockId, FileSegment} -public interface PathResolver { +trait PathResolver { /** Get the file segment in which the given block resides. */ - FileSegment getBlockLocation(BlockId blockId); + def getBlockLocation(blockId: BlockId): FileSegment } diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala index 7ef7aecc6a9fb..95958e30f7eeb 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala @@ -32,7 +32,7 @@ private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) ext server.stop() } - def port: Int = server.getPort() + def port: Int = server.getPort } From db06a81fb7a413faa3fe0f8c35918f70454cb05d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 11 Aug 2014 11:54:09 -0700 Subject: [PATCH 0311/1492] [PySpark] [SPARK-2954] [SPARK-2948] [SPARK-2910] [SPARK-2101] Python 2.6 Fixes - Modify python/run-tests to test with Python 2.6 - Use unittest2 when running on Python 2.6. - Fix issue with namedtuple. - Skip TestOutputFormat.test_newhadoop on Python 2.6 until SPARK-2951 is fixed. - Fix MLlib _deserialize_double on Python 2.6. Closes #1868. Closes #1042. Author: Josh Rosen Closes #1874 from JoshRosen/python2.6 and squashes the following commits: 983d259 [Josh Rosen] [SPARK-2954] Fix MLlib _deserialize_double on Python 2.6. 5d18fd7 [Josh Rosen] [SPARK-2948] [SPARK-2910] [SPARK-2101] Python 2.6 fixes --- python/pyspark/mllib/_common.py | 11 ++++++++++- python/pyspark/mllib/tests.py | 7 ++++++- python/pyspark/serializers.py | 4 ++-- python/pyspark/tests.py | 13 ++++++++++--- python/run-tests | 8 ++++++++ 5 files changed, 36 insertions(+), 7 deletions(-) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index db341da85f865..bb60d3d0c8463 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -16,6 +16,7 @@ # import struct +import sys import numpy from numpy import ndarray, float64, int64, int32, array_equal, array from pyspark import SparkContext, RDD @@ -78,6 +79,14 @@ LABELED_POINT_MAGIC = 4 +# Workaround for SPARK-2954: before Python 2.7, struct.unpack couldn't unpack bytearray()s. +if sys.version_info[:2] <= (2, 6): + def _unpack(fmt, string): + return struct.unpack(fmt, buffer(string)) +else: + _unpack = struct.unpack + + def _deserialize_numpy_array(shape, ba, offset, dtype=float64): """ Deserialize a numpy array of the given type from an offset in @@ -191,7 +200,7 @@ def _deserialize_double(ba, offset=0): raise TypeError("_deserialize_double called on a %s; wanted bytearray" % type(ba)) if len(ba) - offset != 8: raise TypeError("_deserialize_double called on a %d-byte array; wanted 8 bytes." % nb) - return struct.unpack("d", ba[offset:])[0] + return _unpack("d", ba[offset:])[0] def _deserialize_double_vector(ba, offset=0): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 6f3ec8ac94bac..8a851bd35c0e8 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -19,8 +19,13 @@ Fuller unit tests for Python MLlib. """ +import sys from numpy import array, array_equal -import unittest + +if sys.version_info[:2] <= (2, 6): + import unittest2 as unittest +else: + import unittest from pyspark.mllib._common import _convert_vector, _serialize_double_vector, \ _deserialize_double_vector, _dot, _squared_distance diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index b35558db3e007..df90cafb245bf 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -314,8 +314,8 @@ def _copy_func(f): _old_namedtuple = _copy_func(collections.namedtuple) - def namedtuple(name, fields, verbose=False, rename=False): - cls = _old_namedtuple(name, fields, verbose, rename) + def namedtuple(*args, **kwargs): + cls = _old_namedtuple(*args, **kwargs) return _hack_namedtuple(cls) # replace namedtuple with new one diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 88a61176e51ab..22b51110ed671 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -29,9 +29,14 @@ import sys import tempfile import time -import unittest import zipfile +if sys.version_info[:2] <= (2, 6): + import unittest2 as unittest +else: + import unittest + + from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int @@ -605,6 +610,7 @@ def test_oldhadoop(self): conf=input_conf).collect()) self.assertEqual(old_dataset, dict_data) + @unittest.skipIf(sys.version_info[:2] <= (2, 6), "Skipped on 2.6 until SPARK-2951 is fixed") def test_newhadoop(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays @@ -905,8 +911,9 @@ def createFileInZip(self, name, content): pattern = re.compile(r'^ *\|', re.MULTILINE) content = re.sub(pattern, '', content.strip()) path = os.path.join(self.programDir, name + ".zip") - with zipfile.ZipFile(path, 'w') as zip: - zip.writestr(name, content) + zip = zipfile.ZipFile(path, 'w') + zip.writestr(name, content) + zip.close() return path def test_single_script(self): diff --git a/python/run-tests b/python/run-tests index 48feba2f5bd63..1218edcbd7e08 100755 --- a/python/run-tests +++ b/python/run-tests @@ -48,6 +48,14 @@ function run_test() { echo "Running PySpark tests. Output is in python/unit-tests.log." +# Try to test with Python 2.6, since that's the minimum version that we support: +if [ $(which python2.6) ]; then + export PYSPARK_PYTHON="python2.6" +fi + +echo "Testing with Python version:" +$PYSPARK_PYTHON --version + run_test "pyspark/rdd.py" run_test "pyspark/context.py" run_test "pyspark/conf.py" From 37338666655909502e424b4639d680271d6d4c12 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 11 Aug 2014 15:25:21 -0700 Subject: [PATCH 0312/1492] [SPARK-2952] Enable logging actor messages at DEBUG level Example messages: ``` 14/08/09 21:37:01 DEBUG BlockManagerMasterActor: [actor] received message RegisterBlockManager(BlockManagerId(0, rxin-mbp, 58092, 0),278302556,Actor[akka.tcp://spark@rxin-mbp:58088/user/BlockManagerActor1#-63596539]) from Actor[akka.tcp://spark@rxin-mbp:58088/temp/$c] 14/08/09 21:37:01 DEBUG BlockManagerMasterActor: [actor] handled message (0.279 ms) RegisterBlockManager(BlockManagerId(0, rxin-mbp, 58092, 0),278302556,Actor[akka.tcp://spark@rxin-mbp:58088/user/BlockManagerActor1#-63596539]) from Actor[akka.tcp://spark@rxin-mbp:58088/temp/$c] ``` cc @mengxr @tdas @pwendell Author: Reynold Xin Closes #1870 from rxin/actorLogging and squashes the following commits: c531ee5 [Reynold Xin] Added license header for ActorLogReceive. f6b1ebe [Reynold Xin] [SPARK-2952] Enable logging actor messages at DEBUG level --- .../org/apache/spark/HeartbeatReceiver.scala | 7 +- .../org/apache/spark/MapOutputTracker.scala | 4 +- .../org/apache/spark/deploy/Client.scala | 8 ++- .../spark/deploy/client/AppClient.scala | 6 +- .../apache/spark/deploy/master/Master.scala | 6 +- .../apache/spark/deploy/worker/Worker.scala | 6 +- .../spark/deploy/worker/WorkerWatcher.scala | 8 ++- .../CoarseGrainedExecutorBackend.scala | 7 +- .../CoarseGrainedSchedulerBackend.scala | 9 ++- .../spark/scheduler/local/LocalBackend.scala | 8 +-- .../storage/BlockManagerMasterActor.scala | 11 ++-- .../storage/BlockManagerSlaveActor.scala | 5 +- .../apache/spark/util/ActorLogReceive.scala | 64 +++++++++++++++++++ 13 files changed, 111 insertions(+), 38 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 24ccce21b62ca..83ae57b7f1516 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -21,6 +21,7 @@ import akka.actor.Actor import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.scheduler.TaskScheduler +import org.apache.spark.util.ActorLogReceive /** * A heartbeat from executors to the driver. This is a shared message used by several internal @@ -36,8 +37,10 @@ private[spark] case class HeartbeatResponse(reregisterBlockManager: Boolean) /** * Lives in the driver to receive heartbeats from executors.. */ -private[spark] class HeartbeatReceiver(scheduler: TaskScheduler) extends Actor { - override def receive = { +private[spark] class HeartbeatReceiver(scheduler: TaskScheduler) + extends Actor with ActorLogReceive with Logging { + + override def receiveWithLogging = { case Heartbeat(executorId, taskMetrics, blockManagerId) => val response = HeartbeatResponse( !scheduler.executorHeartbeatReceived(executorId, taskMetrics, blockManagerId)) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 894091761485d..51705c895a55c 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -38,10 +38,10 @@ private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage /** Actor class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) - def receive = { + override def receiveWithLogging = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = sender.path.address.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index c07003784e8ac..065ddda50e65e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -27,12 +27,14 @@ import org.apache.log4j.{Level, Logger} import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} /** * Proxy that relays messages to the driver. */ -private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends Actor with Logging { +private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) + extends Actor with ActorLogReceive with Logging { + var masterActor: ActorSelection = _ val timeout = AkkaUtils.askTimeout(conf) @@ -114,7 +116,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends } } - override def receive = { + override def receiveWithLogging = { case SubmitDriverResponse(success, driverId, message) => println(message) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index d38e9e79204c2..32790053a6be8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -30,7 +30,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.Master -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.util.{ActorLogReceive, Utils, AkkaUtils} /** * Interface allowing applications to speak with a Spark deploy cluster. Takes a master URL, @@ -56,7 +56,7 @@ private[spark] class AppClient( var registered = false var activeMasterUrl: String = null - class ClientActor extends Actor with Logging { + class ClientActor extends Actor with ActorLogReceive with Logging { var master: ActorSelection = null var alreadyDisconnected = false // To avoid calling listener.disconnected() multiple times var alreadyDead = false // To avoid calling listener.dead() multiple times @@ -119,7 +119,7 @@ private[spark] class AppClient( .contains(remoteUrl.hostPort) } - override def receive = { + override def receiveWithLogging = { case RegisteredApplication(appId_, masterUrl) => appId = appId_ registered = true 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 a70ecdb375373..cfa2c028a807b 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 @@ -42,14 +42,14 @@ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} private[spark] class Master( host: String, port: Int, webUiPort: Int, val securityMgr: SecurityManager) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -167,7 +167,7 @@ private[spark] class Master( context.stop(leaderElectionAgent) } - override def receive = { + override def receiveWithLogging = { case ElectedLeader => { val (storedApps, storedDrivers, storedWorkers) = persistenceEngine.readPersistedData() state = if (storedApps.isEmpty && storedDrivers.isEmpty && storedWorkers.isEmpty) { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index bacb514ed6335..80fde7e4b2624 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -34,7 +34,7 @@ import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} /** * @param masterUrls Each url should look like spark://host:port. @@ -51,7 +51,7 @@ private[spark] class Worker( workDirPath: String = null, val conf: SparkConf, val securityMgr: SecurityManager) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { import context.dispatcher Utils.checkHost(host, "Expected hostname") @@ -187,7 +187,7 @@ private[spark] class Worker( } } - override def receive = { + override def receiveWithLogging = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) registered = true diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 530c147000904..6d0d0bbe5ecec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -22,13 +22,15 @@ import akka.remote.{AssociatedEvent, AssociationErrorEvent, AssociationEvent, Di import org.apache.spark.Logging import org.apache.spark.deploy.DeployMessages.SendHeartbeat +import org.apache.spark.util.ActorLogReceive /** * Actor which connects to a worker process and terminates the JVM if the connection is severed. * Provides fate sharing between a worker and its associated child processes. */ -private[spark] class WorkerWatcher(workerUrl: String) extends Actor - with Logging { +private[spark] class WorkerWatcher(workerUrl: String) + extends Actor with ActorLogReceive with Logging { + override def preStart() { context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) @@ -48,7 +50,7 @@ private[spark] class WorkerWatcher(workerUrl: String) extends Actor def exitNonZero() = if (isTesting) isShutDown = true else System.exit(-1) - override def receive = { + override def receiveWithLogging = { case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => logInfo(s"Successfully connected to $workerUrl") diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 1f46a0f176490..13af5b6f5812d 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,14 +31,15 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, SignalLogger, Utils} private[spark] class CoarseGrainedExecutorBackend( driverUrl: String, executorId: String, hostPort: String, cores: Int, - sparkProperties: Seq[(String, String)]) extends Actor with ExecutorBackend with Logging { + sparkProperties: Seq[(String, String)]) + extends Actor with ActorLogReceive with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") @@ -52,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend( context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } - override def receive = { + override def receiveWithLogging = { case RegisteredExecutor => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? 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 33500d967ebb1..2a3711ae2a78c 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 @@ -30,7 +30,7 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} 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.util.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} import org.apache.spark.ui.JettyUtils /** @@ -61,7 +61,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) val createTime = System.currentTimeMillis() - class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor { + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { + + override protected def log = CoarseGrainedSchedulerBackend.this.log + private val executorActor = new HashMap[String, ActorRef] private val executorAddress = new HashMap[String, Address] private val executorHost = new HashMap[String, String] @@ -79,7 +82,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A context.system.scheduler.schedule(0.millis, reviveInterval.millis, self, ReviveOffers) } - def receive = { + def receiveWithLogging = { case RegisterExecutor(executorId, hostPort, cores) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) if (executorActor.contains(executorId)) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 3d1cf312ccc97..bec9502f20466 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -23,9 +23,9 @@ import akka.actor.{Actor, ActorRef, Props} import org.apache.spark.{Logging, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState -import org.apache.spark.executor.{TaskMetrics, Executor, ExecutorBackend} +import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} -import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.ActorLogReceive private case class ReviveOffers() @@ -43,7 +43,7 @@ private case class StopExecutor() private[spark] class LocalActor( scheduler: TaskSchedulerImpl, executorBackend: LocalBackend, - private val totalCores: Int) extends Actor with Logging { + private val totalCores: Int) extends Actor with ActorLogReceive with Logging { private var freeCores = totalCores @@ -53,7 +53,7 @@ private[spark] class LocalActor( val executor = new Executor( localExecutorId, localExecutorHostname, scheduler.conf.getAll, isLocal = true) - def receive = { + override def receiveWithLogging = { case ReviveOffers => reviveOffers() 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 bd31e3c5a187f..3ab07703b6f85 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -31,7 +31,7 @@ import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{ActorLogReceive, AkkaUtils, Utils} /** * BlockManagerMasterActor is an actor on the master node to track statuses of @@ -39,7 +39,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] @@ -55,8 +55,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val slaveTimeout = conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", math.max(conf.getInt("spark.executor.heartbeatInterval", 10000) * 3, 45000)) - val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", - 60000) + val checkTimeoutInterval = conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000) var timeoutCheckingTask: Cancellable = null @@ -67,9 +66,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus super.preStart() } - def receive = { + override def receiveWithLogging = { case RegisterBlockManager(blockManagerId, maxMemSize, slaveActor) => - logInfo("received a register") register(blockManagerId, maxMemSize, slaveActor) sender ! true @@ -118,7 +116,6 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus sender ! true case StopBlockManagerMaster => - logInfo("Stopping BlockManagerMaster") sender ! true if (timeoutCheckingTask != null) { timeoutCheckingTask.cancel() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index 6d4db064dff58..c194e0fed3367 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -23,6 +23,7 @@ import akka.actor.{ActorRef, Actor} import org.apache.spark.{Logging, MapOutputTracker} import org.apache.spark.storage.BlockManagerMessages._ +import org.apache.spark.util.ActorLogReceive /** * An actor to take commands from the master to execute options. For example, @@ -32,12 +33,12 @@ private[storage] class BlockManagerSlaveActor( blockManager: BlockManager, mapOutputTracker: MapOutputTracker) - extends Actor with Logging { + extends Actor with ActorLogReceive with Logging { import context.dispatcher // Operations that involve removing blocks may be slow and should be done asynchronously - override def receive = { + override def receiveWithLogging = { case RemoveBlock(blockId) => doAsync[Boolean]("removing block " + blockId, sender) { blockManager.removeBlock(blockId) diff --git a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala new file mode 100644 index 0000000000000..332d0cbb2dc0c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import akka.actor.Actor +import org.slf4j.Logger + +/** + * A trait to enable logging all Akka actor messages. Here's an example of using this: + * + * {{{ + * class BlockManagerMasterActor extends Actor with ActorLogReceive with Logging { + * ... + * override def receiveWithLogging = { + * case GetLocations(blockId) => + * sender ! getLocations(blockId) + * ... + * } + * ... + * } + * }}} + * + */ +private[spark] trait ActorLogReceive { + self: Actor => + + override def receive: Actor.Receive = new Actor.Receive { + + private val _receiveWithLogging = receiveWithLogging + + override def isDefinedAt(o: Any): Boolean = _receiveWithLogging.isDefinedAt(o) + + override def apply(o: Any): Unit = { + if (log.isDebugEnabled) { + log.debug(s"[actor] received message $o from ${self.sender}") + } + val start = System.nanoTime + _receiveWithLogging.apply(o) + val timeTaken = (System.nanoTime - start).toDouble / 1000000 + if (log.isDebugEnabled) { + log.debug(s"[actor] handled message ($timeTaken ms) $o from ${self.sender}") + } + } + } + + def receiveWithLogging: Actor.Receive + + protected def log: Logger +} From 7712e724ad69dd0b83754e938e9799d13a4d43b9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 11 Aug 2014 19:15:01 -0700 Subject: [PATCH 0313/1492] [SPARK-2931] In TaskSetManager, reset currentLocalityIndex after recomputing locality levels This addresses SPARK-2931, a bug where getAllowedLocalityLevel() could throw ArrayIndexOutOfBoundsException. The fix here is to reset currentLocalityIndex after recomputing the locality levels. Thanks to kayousterhout, mridulm, and lirui-intel for helping me to debug this. Author: Josh Rosen Closes #1896 from JoshRosen/SPARK-2931 and squashes the following commits: 48b60b5 [Josh Rosen] Move FakeRackUtil.cleanUp() info beforeEach(). 6fec474 [Josh Rosen] Set currentLocalityIndex after recomputing locality levels. 9384897 [Josh Rosen] Update SPARK-2931 test to reflect changes in 63bdb1f41b4895e3a9444f7938094438a94d3007. 9ecd455 [Josh Rosen] Apply @mridulm's patch for reproducing SPARK-2931. --- .../spark/scheduler/TaskSetManager.scala | 11 +++-- .../spark/scheduler/TaskSetManagerSuite.scala | 40 ++++++++++++++++++- 2 files changed, 46 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 20a4bd12f93f6..d9d53faf843ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -690,8 +690,7 @@ private[spark] class TaskSetManager( handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure) } // recalculate valid locality levels and waits when executor is lost - myLocalityLevels = computeValidLocalityLevels() - localityWaits = myLocalityLevels.map(getLocalityWait) + recomputeLocality() } /** @@ -775,9 +774,15 @@ private[spark] class TaskSetManager( levels.toArray } - def executorAdded() { + def recomputeLocality() { + val previousLocalityLevel = myLocalityLevels(currentLocalityIndex) myLocalityLevels = computeValidLocalityLevels() localityWaits = myLocalityLevels.map(getLocalityWait) + currentLocalityIndex = getLocalityIndex(previousLocalityLevel) + } + + def executorAdded() { + recomputeLocality() } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index ffd23380a886f..93e8ddacf8865 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -154,6 +154,11 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) val MAX_TASK_FAILURES = 4 + override def beforeEach() { + super.beforeEach() + FakeRackUtil.cleanUp() + } + test("TaskSet with no preferences") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) @@ -471,7 +476,6 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("new executors get added and lost") { // Assign host2 to rack2 - FakeRackUtil.cleanUp() FakeRackUtil.assignHostToRack("host2", "rack2") sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc) @@ -504,7 +508,6 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { } test("test RACK_LOCAL tasks") { - FakeRackUtil.cleanUp() // Assign host1 to rack1 FakeRackUtil.assignHostToRack("host1", "rack1") // Assign host2 to rack1 @@ -607,6 +610,39 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) } + test("Ensure TaskSetManager is usable after addition of levels") { + // Regression test for SPARK-2931 + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc) + val taskSet = FakeTask.createTaskSet(2, + Seq(TaskLocation("host1", "execA")), + Seq(TaskLocation("host2", "execB.1"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + // Only ANY is valid + assert(manager.myLocalityLevels.sameElements(Array(ANY))) + // Add a new executor + sched.addExecutor("execA", "host1") + sched.addExecutor("execB.2", "host2") + manager.executorAdded() + assert(manager.pendingTasksWithNoPrefs.size === 0) + // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + assert(manager.resourceOffer("execA", "host1", ANY) !== None) + clock.advance(LOCALITY_WAIT * 4) + assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) + sched.removeExecutor("execA") + sched.removeExecutor("execB.2") + manager.executorLost("execA", "host1") + manager.executorLost("execB.2", "host2") + clock.advance(LOCALITY_WAIT * 4) + sched.addExecutor("execC", "host3") + manager.executorAdded() + // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: + assert(manager.resourceOffer("execC", "host3", ANY) !== None) + } + + def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) From 32638b5e74e02410831b391f555223f90c830498 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Mon, 11 Aug 2014 19:22:14 -0700 Subject: [PATCH 0314/1492] [SPARK-2515][mllib] Chi Squared test Author: Doris Xin Closes #1733 from dorx/chisquare and squashes the following commits: cafb3a7 [Doris Xin] fixed p-value for extreme case. d286783 [Doris Xin] Merge branch 'master' into chisquare e95e485 [Doris Xin] reviewer comments. 7dde711 [Doris Xin] ChiSqTestResult renaming and changed to Class 80d03e2 [Doris Xin] Reviewer comments. c39eeb5 [Doris Xin] units passed with updated API e90d90a [Doris Xin] Merge branch 'master' into chisquare 7eea80b [Doris Xin] WIP d64c2fb [Doris Xin] Merge branch 'master' into chisquare 5686082 [Doris Xin] facelift bc7eb2e [Doris Xin] unit passed; still need docs and some refactoring 50703a5 [Doris Xin] merge master 4e4e361 [Doris Xin] WIP e6b83f3 [Doris Xin] reviewer comments 3d61582 [Doris Xin] input names 706d436 [Doris Xin] Added API for RDD[Vector] 6598379 [Doris Xin] API and code structure. ff17423 [Doris Xin] WIP --- .../apache/spark/mllib/stat/Statistics.scala | 64 +++++ .../spark/mllib/stat/test/ChiSqTest.scala | 221 ++++++++++++++++++ .../spark/mllib/stat/test/TestResult.scala | 88 +++++++ .../mllib/stat/HypothesisTestSuite.scala | 139 +++++++++++ 4 files changed, 512 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index f416a9fbb323d..cf8679610e191 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -19,7 +19,9 @@ package org.apache.spark.mllib.stat import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{Matrix, Vector} +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.correlation.Correlations +import org.apache.spark.mllib.stat.test.{ChiSqTest, ChiSqTestResult} import org.apache.spark.rdd.RDD /** @@ -89,4 +91,66 @@ object Statistics { */ @Experimental def corr(x: RDD[Double], y: RDD[Double], method: String): Double = Correlations.corr(x, y, method) + + /** + * :: Experimental :: + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the + * expected distribution. + * + * Note: the two input Vectors need to have the same size. + * `observed` cannot contain negative values. + * `expected` cannot contain nonpositive values. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @param expected Vector containing the expected categorical counts/relative frequencies. + * `expected` is rescaled if the `expected` sum differs from the `observed` sum. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Experimental + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + ChiSqTest.chiSquared(observed, expected) + } + + /** + * :: Experimental :: + * Conduct Pearson's chi-squared goodness of fit test of the observed data against the uniform + * distribution, with each category having an expected frequency of `1 / observed.size`. + * + * Note: `observed` cannot contain negative values. + * + * @param observed Vector containing the observed categorical counts/relative frequencies. + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Experimental + def chiSqTest(observed: Vector): ChiSqTestResult = ChiSqTest.chiSquared(observed) + + /** + * :: Experimental :: + * Conduct Pearson's independence test on the input contingency matrix, which cannot contain + * negative entries or columns or rows that sum up to 0. + * + * @param observed The contingency matrix (containing either counts or relative frequencies). + * @return ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, + * the method used, and the null hypothesis. + */ + @Experimental + def chiSqTest(observed: Matrix): ChiSqTestResult = ChiSqTest.chiSquaredMatrix(observed) + + /** + * :: Experimental :: + * Conduct Pearson's independence test for every feature against the label across the input RDD. + * For each feature, the (feature, label) pairs are converted into a contingency matrix for which + * the chi-squared statistic is computed. + * + * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. + * Real-valued features will be treated as categorical for each distinct value. + * @return an array containing the ChiSquaredTestResult for every feature against the label. + * The order of the elements in the returned array reflects the order of input features. + */ + @Experimental + def chiSqTest(data: RDD[LabeledPoint]): Array[ChiSqTestResult] = { + ChiSqTest.chiSquaredFeatures(data) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala new file mode 100644 index 0000000000000..8f6752737402e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -0,0 +1,221 @@ +/* + * 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.stat.test + +import breeze.linalg.{DenseMatrix => BDM} +import cern.jet.stat.Probability.chiSquareComplemented + +import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.rdd.RDD + +/** + * Conduct the chi-squared test for the input RDDs using the specified method. + * Goodness-of-fit test is conducted on two `Vectors`, whereas test of independence is conducted + * on an input of type `Matrix` in which independence between columns is assessed. + * We also provide a method for computing the chi-squared statistic between each feature and the + * label for an input `RDD[LabeledPoint]`, return an `Array[ChiSquaredTestResult]` of size = + * number of features in the inpuy RDD. + * + * Supported methods for goodness of fit: `pearson` (default) + * Supported methods for independence: `pearson` (default) + * + * More information on Chi-squared test: http://en.wikipedia.org/wiki/Chi-squared_test + */ +private[stat] object ChiSqTest extends Logging { + + /** + * @param name String name for the method. + * @param chiSqFunc Function for computing the statistic given the observed and expected counts. + */ + case class Method(name: String, chiSqFunc: (Double, Double) => Double) + + // Pearson's chi-squared test: http://en.wikipedia.org/wiki/Pearson%27s_chi-squared_test + val PEARSON = new Method("pearson", (observed: Double, expected: Double) => { + val dev = observed - expected + dev * dev / expected + }) + + // Null hypothesis for the two different types of chi-squared tests to be included in the result. + object NullHypothesis extends Enumeration { + type NullHypothesis = Value + val goodnessOfFit = Value("observed follows the same distribution as expected.") + val independence = Value("observations in each column are statistically independent.") + } + + // Method identification based on input methodName string + private def methodFromString(methodName: String): Method = { + methodName match { + case PEARSON.name => PEARSON + case _ => throw new IllegalArgumentException("Unrecognized method for Chi squared test.") + } + } + + /** + * Conduct Pearson's independence test for each feature against the label across the input RDD. + * The contingency table is constructed from the raw (feature, label) pairs and used to conduct + * the independence test. + * Returns an array containing the ChiSquaredTestResult for every feature against the label. + */ + def chiSquaredFeatures(data: RDD[LabeledPoint], + methodName: String = PEARSON.name): Array[ChiSqTestResult] = { + val numCols = data.first().features.size + val results = new Array[ChiSqTestResult](numCols) + var labels: Map[Double, Int] = null + // At most 100 columns at a time + val batchSize = 100 + var batch = 0 + while (batch * batchSize < numCols) { + // The following block of code can be cleaned up and made public as + // chiSquared(data: RDD[(V1, V2)]) + val startCol = batch * batchSize + val endCol = startCol + math.min(batchSize, numCols - startCol) + val pairCounts = data.flatMap { p => + // assume dense vectors + p.features.toArray.slice(startCol, endCol).zipWithIndex.map { case (feature, col) => + (col, feature, p.label) + } + }.countByValue() + + if (labels == null) { + // Do this only once for the first column since labels are invariant across features. + labels = + pairCounts.keys.filter(_._1 == startCol).map(_._3).toArray.distinct.zipWithIndex.toMap + } + val numLabels = labels.size + pairCounts.keys.groupBy(_._1).map { case (col, keys) => + val features = keys.map(_._2).toArray.distinct.zipWithIndex.toMap + val numRows = features.size + val contingency = new BDM(numRows, numLabels, new Array[Double](numRows * numLabels)) + keys.foreach { case (_, feature, label) => + val i = features(feature) + val j = labels(label) + contingency(i, j) += pairCounts((col, feature, label)) + } + results(col) = chiSquaredMatrix(Matrices.fromBreeze(contingency), methodName) + } + batch += 1 + } + results + } + + /* + * Pearon's goodness of fit test on the input observed and expected counts/relative frequencies. + * Uniform distribution is assumed when `expected` is not passed in. + */ + def chiSquared(observed: Vector, + expected: Vector = Vectors.dense(Array[Double]()), + methodName: String = PEARSON.name): ChiSqTestResult = { + + // Validate input arguments + val method = methodFromString(methodName) + if (expected.size != 0 && observed.size != expected.size) { + throw new IllegalArgumentException("observed and expected must be of the same size.") + } + val size = observed.size + if (size > 1000) { + logWarning("Chi-squared approximation may not be accurate due to low expected frequencies " + + s" as a result of a large number of categories: $size.") + } + val obsArr = observed.toArray + val expArr = if (expected.size == 0) Array.tabulate(size)(_ => 1.0 / size) else expected.toArray + if (!obsArr.forall(_ >= 0.0)) { + throw new IllegalArgumentException("Negative entries disallowed in the observed vector.") + } + if (expected.size != 0 && ! expArr.forall(_ >= 0.0)) { + throw new IllegalArgumentException("Negative entries disallowed in the expected vector.") + } + + // Determine the scaling factor for expected + val obsSum = obsArr.sum + val expSum = if (expected.size == 0.0) 1.0 else expArr.sum + val scale = if (math.abs(obsSum - expSum) < 1e-7) 1.0 else obsSum / expSum + + // compute chi-squared statistic + val statistic = obsArr.zip(expArr).foldLeft(0.0) { case (stat, (obs, exp)) => + if (exp == 0.0) { + if (obs == 0.0) { + throw new IllegalArgumentException("Chi-squared statistic undefined for input vectors due" + + " to 0.0 values in both observed and expected.") + } else { + return new ChiSqTestResult(0.0, size - 1, Double.PositiveInfinity, PEARSON.name, + NullHypothesis.goodnessOfFit.toString) + } + } + if (scale == 1.0) { + stat + method.chiSqFunc(obs, exp) + } else { + stat + method.chiSqFunc(obs, exp * scale) + } + } + val df = size - 1 + val pValue = chiSquareComplemented(df, statistic) + new ChiSqTestResult(pValue, df, statistic, PEARSON.name, NullHypothesis.goodnessOfFit.toString) + } + + /* + * Pearon's independence test on the input contingency matrix. + * TODO: optimize for SparseMatrix when it becomes supported. + */ + def chiSquaredMatrix(counts: Matrix, methodName:String = PEARSON.name): ChiSqTestResult = { + val method = methodFromString(methodName) + val numRows = counts.numRows + val numCols = counts.numCols + + // get row and column sums + val colSums = new Array[Double](numCols) + val rowSums = new Array[Double](numRows) + val colMajorArr = counts.toArray + var i = 0 + while (i < colMajorArr.size) { + val elem = colMajorArr(i) + if (elem < 0.0) { + throw new IllegalArgumentException("Contingency table cannot contain negative entries.") + } + colSums(i / numRows) += elem + rowSums(i % numRows) += elem + i += 1 + } + val total = colSums.sum + + // second pass to collect statistic + var statistic = 0.0 + var j = 0 + while (j < colMajorArr.size) { + val col = j / numRows + val colSum = colSums(col) + if (colSum == 0.0) { + throw new IllegalArgumentException("Chi-squared statistic undefined for input matrix due to" + + s"0 sum in column [$col].") + } + val row = j % numRows + val rowSum = rowSums(row) + if (rowSum == 0.0) { + throw new IllegalArgumentException("Chi-squared statistic undefined for input matrix due to" + + s"0 sum in row [$row].") + } + val expected = colSum * rowSum / total + statistic += method.chiSqFunc(colMajorArr(j), expected) + j += 1 + } + val df = (numCols - 1) * (numRows - 1) + val pValue = chiSquareComplemented(df, statistic) + new ChiSqTestResult(pValue, df, statistic, methodName, NullHypothesis.independence.toString) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala new file mode 100644 index 0000000000000..2f278621335e1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala @@ -0,0 +1,88 @@ +/* + * 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.stat.test + +import org.apache.spark.annotation.Experimental + +/** + * :: Experimental :: + * Trait for hypothesis test results. + * @tparam DF Return type of `degreesOfFreedom`. + */ +@Experimental +trait TestResult[DF] { + + /** + * The probability of obtaining a test statistic result at least as extreme as the one that was + * actually observed, assuming that the null hypothesis is true. + */ + def pValue: Double + + /** + * Returns the degree(s) of freedom of the hypothesis test. + * Return type should be Number(e.g. Int, Double) or tuples of Numbers for toString compatibility. + */ + def degreesOfFreedom: DF + + /** + * Test statistic. + */ + def statistic: Double + + /** + * String explaining the hypothesis test result. + * Specific classes implementing this trait should override this method to output test-specific + * information. + */ + override def toString: String = { + + // String explaining what the p-value indicates. + val pValueExplain = if (pValue <= 0.01) { + "Very strong presumption against null hypothesis." + } else if (0.01 < pValue && pValue <= 0.05) { + "Strong presumption against null hypothesis." + } else if (0.05 < pValue && pValue <= 0.01) { + "Low presumption against null hypothesis." + } else { + "No presumption against null hypothesis." + } + + s"degrees of freedom = ${degreesOfFreedom.toString} \n" + + s"statistic = $statistic \n" + + s"pValue = $pValue \n" + pValueExplain + } +} + +/** + * :: Experimental :: + * Object containing the test results for the chi squared hypothesis test. + */ +@Experimental +class ChiSqTestResult(override val pValue: Double, + override val degreesOfFreedom: Int, + override val statistic: Double, + val method: String, + val nullHypothesis: String) extends TestResult[Int] { + + override def toString: String = { + "Chi squared test summary: \n" + + s"method: $method \n" + + s"null hypothesis: $nullHypothesis \n" + + super.toString + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala new file mode 100644 index 0000000000000..5bd0521298c14 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala @@ -0,0 +1,139 @@ +/* + * 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.stat + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{DenseVector, Matrices, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.test.ChiSqTest +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class HypothesisTestSuite extends FunSuite with LocalSparkContext { + + test("chi squared pearson goodness of fit") { + + val observed = new DenseVector(Array[Double](4, 6, 5)) + val pearson = Statistics.chiSqTest(observed) + + // Results validated against the R command `chisq.test(c(4, 6, 5), p=c(1/3, 1/3, 1/3))` + assert(pearson.statistic === 0.4) + assert(pearson.degreesOfFreedom === 2) + assert(pearson.pValue ~== 0.8187 relTol 1e-4) + assert(pearson.method === ChiSqTest.PEARSON.name) + assert(pearson.nullHypothesis === ChiSqTest.NullHypothesis.goodnessOfFit.toString) + + // different expected and observed sum + val observed1 = new DenseVector(Array[Double](21, 38, 43, 80)) + val expected1 = new DenseVector(Array[Double](3, 5, 7, 20)) + val pearson1 = Statistics.chiSqTest(observed1, expected1) + + // Results validated against the R command + // `chisq.test(c(21, 38, 43, 80), p=c(3/35, 1/7, 1/5, 4/7))` + assert(pearson1.statistic ~== 14.1429 relTol 1e-4) + assert(pearson1.degreesOfFreedom === 3) + assert(pearson1.pValue ~== 0.002717 relTol 1e-4) + assert(pearson1.method === ChiSqTest.PEARSON.name) + assert(pearson1.nullHypothesis === ChiSqTest.NullHypothesis.goodnessOfFit.toString) + + // Vectors with different sizes + val observed3 = new DenseVector(Array(1.0, 2.0, 3.0)) + val expected3 = new DenseVector(Array(1.0, 2.0, 3.0, 4.0)) + intercept[IllegalArgumentException](Statistics.chiSqTest(observed3, expected3)) + + // negative counts in observed + val negObs = new DenseVector(Array(1.0, 2.0, 3.0, -4.0)) + intercept[IllegalArgumentException](Statistics.chiSqTest(negObs, expected1)) + + // count = 0.0 in expected but not observed + val zeroExpected = new DenseVector(Array(1.0, 0.0, 3.0)) + val inf = Statistics.chiSqTest(observed, zeroExpected) + assert(inf.statistic === Double.PositiveInfinity) + assert(inf.degreesOfFreedom === 2) + assert(inf.pValue === 0.0) + assert(inf.method === ChiSqTest.PEARSON.name) + assert(inf.nullHypothesis === ChiSqTest.NullHypothesis.goodnessOfFit.toString) + + // 0.0 in expected and observed simultaneously + val zeroObserved = new DenseVector(Array(2.0, 0.0, 1.0)) + intercept[IllegalArgumentException](Statistics.chiSqTest(zeroObserved, zeroExpected)) + } + + test("chi squared pearson matrix independence") { + val data = Array(40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0) + // [[40.0, 56.0, 31.0, 30.0], + // [24.0, 32.0, 10.0, 15.0], + // [29.0, 42.0, 0.0, 12.0]] + val chi = Statistics.chiSqTest(Matrices.dense(3, 4, data)) + // Results validated against R command + // `chisq.test(rbind(c(40, 56, 31, 30),c(24, 32, 10, 15), c(29, 42, 0, 12)))` + assert(chi.statistic ~== 21.9958 relTol 1e-4) + assert(chi.degreesOfFreedom === 6) + assert(chi.pValue ~== 0.001213 relTol 1e-4) + assert(chi.method === ChiSqTest.PEARSON.name) + assert(chi.nullHypothesis === ChiSqTest.NullHypothesis.independence.toString) + + // Negative counts + val negCounts = Array(4.0, 5.0, 3.0, -3.0) + intercept[IllegalArgumentException](Statistics.chiSqTest(Matrices.dense(2, 2, negCounts))) + + // Row sum = 0.0 + val rowZero = Array(0.0, 1.0, 0.0, 2.0) + intercept[IllegalArgumentException](Statistics.chiSqTest(Matrices.dense(2, 2, rowZero))) + + // Column sum = 0.0 + val colZero = Array(0.0, 0.0, 2.0, 2.0) + // IllegalArgumentException thrown here since it's thrown on driver, not inside a task + intercept[IllegalArgumentException](Statistics.chiSqTest(Matrices.dense(2, 2, colZero))) + } + + test("chi squared pearson RDD[LabeledPoint]") { + // labels: 1.0 (2 / 6), 0.0 (4 / 6) + // feature1: 0.5 (1 / 6), 1.5 (2 / 6), 3.5 (3 / 6) + // feature2: 10.0 (1 / 6), 20.0 (1 / 6), 30.0 (2 / 6), 40.0 (2 / 6) + val data = Array(new LabeledPoint(0.0, Vectors.dense(0.5, 10.0)), + new LabeledPoint(0.0, Vectors.dense(1.5, 20.0)), + new LabeledPoint(1.0, Vectors.dense(1.5, 30.0)), + new LabeledPoint(0.0, Vectors.dense(3.5, 30.0)), + new LabeledPoint(0.0, Vectors.dense(3.5, 40.0)), + new LabeledPoint(1.0, Vectors.dense(3.5, 40.0))) + for (numParts <- List(2, 4, 6, 8)) { + val chi = Statistics.chiSqTest(sc.parallelize(data, numParts)) + val feature1 = chi(0) + assert(feature1.statistic === 0.75) + assert(feature1.degreesOfFreedom === 2) + assert(feature1.pValue ~== 0.6873 relTol 1e-4) + assert(feature1.method === ChiSqTest.PEARSON.name) + assert(feature1.nullHypothesis === ChiSqTest.NullHypothesis.independence.toString) + val feature2 = chi(1) + assert(feature2.statistic === 1.5) + assert(feature2.degreesOfFreedom === 3) + assert(feature2.pValue ~== 0.6823 relTol 1e-4) + assert(feature2.method === ChiSqTest.PEARSON.name) + assert(feature2.nullHypothesis === ChiSqTest.NullHypothesis.independence.toString) + } + + // Test that the right number of results is returned + val numCols = 321 + val sparseData = Array(new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((100, 2.0)))), + new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((200, 1.0))))) + val chi = Statistics.chiSqTest(sc.parallelize(sparseData)) + assert(chi.size === numCols) + } +} From 6fab941b65f0cb6c9b32e0f8290d76889cda6a87 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Mon, 11 Aug 2014 19:49:29 -0700 Subject: [PATCH 0315/1492] [SPARK-2934][MLlib] Adding LogisticRegressionWithLBFGS Interface for training with LBFGS Optimizer which will converge faster than SGD. Author: DB Tsai Closes #1862 from dbtsai/dbtsai-lbfgs-lor and squashes the following commits: aa84b81 [DB Tsai] small change f852bcd [DB Tsai] Remove duplicate method f119fdc [DB Tsai] Formatting 97776aa [DB Tsai] address more feedback 85b4a91 [DB Tsai] address feedback 3cf50c2 [DB Tsai] LogisticRegressionWithLBFGS interface --- .../classification/LogisticRegression.scala | 51 ++++++++++- .../LogisticRegressionSuite.scala | 89 ++++++++++++++++++- 2 files changed, 136 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 2242329b7918e..31d474a20fa85 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -101,7 +101,7 @@ class LogisticRegressionWithSGD private ( } /** - * Top-level methods for calling Logistic Regression. + * Top-level methods for calling Logistic Regression using Stochastic Gradient Descent. * NOTE: Labels used in Logistic Regression should be {0, 1} */ object LogisticRegressionWithSGD { @@ -188,3 +188,52 @@ object LogisticRegressionWithSGD { train(input, numIterations, 1.0, 1.0) } } + +/** + * Train a classification model for Logistic Regression using Limited-memory BFGS. + * NOTE: Labels used in Logistic Regression should be {0, 1} + */ +class LogisticRegressionWithLBFGS private ( + private var convergenceTol: Double, + private var maxNumIterations: Int, + private var regParam: Double) + extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { + + /** + * Construct a LogisticRegression object with default parameters + */ + def this() = this(1E-4, 100, 0.0) + + private val gradient = new LogisticGradient() + private val updater = new SimpleUpdater() + // Have to return new LBFGS object every time since users can reset the parameters anytime. + override def optimizer = new LBFGS(gradient, updater) + .setNumCorrections(10) + .setConvergenceTol(convergenceTol) + .setMaxNumIterations(maxNumIterations) + .setRegParam(regParam) + + override protected val validators = List(DataValidators.binaryLabelValidator) + + /** + * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. + * Smaller value will lead to higher accuracy with the cost of more iterations. + */ + def setConvergenceTol(convergenceTol: Double): this.type = { + this.convergenceTol = convergenceTol + this + } + + /** + * Set the maximal number of iterations for L-BFGS. Default 100. + */ + def setNumIterations(numIterations: Int): this.type = { + this.maxNumIterations = numIterations + this + } + + override protected def createModel(weights: Vector, intercept: Double) = { + new LogisticRegressionModel(weights, intercept) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index da7c633bbd2af..2289c6cdc19de 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -67,7 +67,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match } // Test if we can correctly learn A, B where Y = logistic(A + B*X) - test("logistic regression") { + test("logistic regression with SGD") { val nPoints = 10000 val A = 2.0 val B = -1.5 @@ -94,7 +94,36 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } - test("logistic regression with initial weights") { + // Test if we can correctly learn A, B where Y = logistic(A + B*X) + test("logistic regression with LBFGS") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + + val model = lr.run(testRDD) + + // Test the weights + assert(model.weights(0) ~== -1.52 relTol 0.01) + assert(model.intercept ~== 2.00 relTol 0.01) + assert(model.weights(0) ~== model.weights(0) relTol 0.01) + assert(model.intercept ~== model.intercept relTol 0.01) + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } + + test("logistic regression with initial weights with SGD") { val nPoints = 10000 val A = 2.0 val B = -1.5 @@ -125,11 +154,42 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("logistic regression with initial weights with LBFGS") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Vectors.dense(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + + val model = lr.run(testRDD, initialWeights) + + // Test the weights + assert(model.weights(0) ~== -1.50 relTol 0.02) + assert(model.intercept ~== 1.97 relTol 0.02) + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } } class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { - test("task size should be small in both training and prediction") { + test("task size should be small in both training and prediction using SGD optimizer") { val m = 4 val n = 200000 val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => @@ -139,6 +199,29 @@ class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkCont // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. val model = LogisticRegressionWithSGD.train(points, 2) + val predictions = model.predict(points.map(_.features)) + + // Materialize the RDDs + predictions.count() } + + test("task size should be small in both training and prediction using LBFGS optimizer") { + val m = 4 + val n = 200000 + val points = sc.parallelize(0 until m, 2).mapPartitionsWithIndex { (idx, iter) => + val random = new Random(idx) + iter.map(i => LabeledPoint(1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) + }.cache() + // If we serialize data directly in the task closure, the size of the serialized task would be + // greater than 1MB and hence Spark would throw an error. + val model = + (new LogisticRegressionWithLBFGS().setIntercept(true).setNumIterations(2)).run(points) + + val predictions = model.predict(points.map(_.features)) + + // Materialize the RDDs + predictions.count() + } + } From 490ecfa20327a636289321ea447722aa32b81657 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Mon, 11 Aug 2014 20:06:06 -0700 Subject: [PATCH 0316/1492] [SPARK-2844][SQL] Correctly set JVM HiveContext if it is passed into Python HiveContext constructor https://issues.apache.org/jira/browse/SPARK-2844 Author: Ahir Reddy Closes #1768 from ahirreddy/python-hive-context-fix and squashes the following commits: 7972d3b [Ahir Reddy] Correctly set JVM HiveContext if it is passed into Python HiveContext constructor --- python/pyspark/sql.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 950e275adbf01..36040463e62a9 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -912,6 +912,8 @@ def __init__(self, sparkContext, sqlContext=None): """Create a new SQLContext. @param sparkContext: The SparkContext to wrap. + @param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new + SQLContext in the JVM, instead we make all calls to this object. >>> srdd = sqlCtx.inferSchema(rdd) >>> sqlCtx.inferSchema(srdd) # doctest: +IGNORE_EXCEPTION_DETAIL @@ -1315,6 +1317,18 @@ class HiveContext(SQLContext): It supports running both SQL and HiveQL commands. """ + def __init__(self, sparkContext, hiveContext=None): + """Create a new HiveContext. + + @param sparkContext: The SparkContext to wrap. + @param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new + HiveContext in the JVM, instead we make all calls to this object. + """ + SQLContext.__init__(self, sparkContext) + + if hiveContext: + self._scala_HiveContext = hiveContext + @property def _ssql_ctx(self): try: From 21a95ef051f7b23a80d147aadb00dfa4ebb169b0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 11 Aug 2014 20:08:06 -0700 Subject: [PATCH 0317/1492] [SPARK-2590][SQL] Added option to handle incremental collection, disabled by default JIRA issue: [SPARK-2590](https://issues.apache.org/jira/browse/SPARK-2590) Author: Cheng Lian Closes #1853 from liancheng/inc-collect-option and squashes the following commits: cb3ea45 [Cheng Lian] Moved incremental collection option to Thrift server 43ce3aa [Cheng Lian] Changed incremental collect option name 623abde [Cheng Lian] Added option to handle incremental collection, disabled by default --- .../server/SparkSQLOperationManager.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index dee092159dd4c..f192f490ac3d0 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -132,7 +132,16 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage logDebug(result.queryExecution.toString()) val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) - iter = result.queryExecution.toRdd.toLocalIterator + iter = { + val resultRdd = result.queryExecution.toRdd + val useIncrementalCollect = + hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean + if (useIncrementalCollect) { + resultRdd.toLocalIterator + } else { + resultRdd.collect().iterator + } + } dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray setHasResultSet(true) } catch { From e83fdcd421d132812411eb805565b76f087f1bc0 Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 11 Aug 2014 20:10:13 -0700 Subject: [PATCH 0318/1492] [sql]use SparkSQLEnv.stop() in ShutdownHook Author: wangfei Closes #1852 from scwf/patch-3 and squashes the following commits: ae28c29 [wangfei] use SparkSQLEnv.stop() in ShutdownHook --- .../apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 6f7942aba314a..cadf7aaf42157 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -60,7 +60,7 @@ private[hive] object HiveThriftServer2 extends Logging { Runtime.getRuntime.addShutdownHook( new Thread() { override def run() { - SparkSQLEnv.sparkContext.stop() + SparkSQLEnv.stop() } } ) From 647aeba3a9e101d35083f7c4afbcfe7a33f7fc62 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 11 Aug 2014 20:11:29 -0700 Subject: [PATCH 0319/1492] [SQL] A tiny refactoring in HiveContext#analyze I should use `EliminateAnalysisOperators` in `analyze` instead of manually pattern matching. Author: Yin Huai Closes #1881 from yhuai/useEliminateAnalysisOperators and squashes the following commits: f3e1e7f [Yin Huai] Use EliminateAnalysisOperators. --- .../scala/org/apache/spark/sql/hive/HiveContext.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 53f3dc11dbb9f..a8da676ffa0e0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -39,7 +39,8 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.{OverrideFunctionRegistry, Analyzer, OverrideCatalog} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EliminateAnalysisOperators} +import org.apache.spark.sql.catalyst.analysis.{OverrideCatalog, OverrideFunctionRegistry} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException @@ -119,10 +120,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * in the Hive metastore. */ def analyze(tableName: String) { - val relation = catalog.lookupRelation(None, tableName) match { - case LowerCaseSchema(r) => r - case o => o - } + val relation = EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) relation match { case relation: MetastoreRelation => { From c9c89c31b6114832fe282c21fecd663d8105b9bc Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 11 Aug 2014 20:15:01 -0700 Subject: [PATCH 0320/1492] [SPARK-2965][SQL] Fix HashOuterJoin output nullabilities. Output attributes of opposite side of `OuterJoin` should be nullable. Author: Takuya UESHIN Closes #1887 from ueshin/issues/SPARK-2965 and squashes the following commits: bcb2d37 [Takuya UESHIN] Fix HashOuterJoin output nullabilities. --- .../org/apache/spark/sql/execution/joins.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 51bb61530744c..ea075f8c65bff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -168,7 +168,18 @@ case class HashOuterJoin( override def requiredChildDistribution = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - def output = left.output ++ right.output + override def output = { + joinType match { + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case FullOuter => + left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case x => + throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } + } // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala // iterator for performance purpose. From c686b7dd4668b5e9fc3177f15edeae3446d2e634 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 11 Aug 2014 20:18:03 -0700 Subject: [PATCH 0321/1492] [SPARK-2968][SQL] Fix nullabilities of Explode. Output nullabilities of `Explode` could be detemined by `ArrayType.containsNull` or `MapType.valueContainsNull`. Author: Takuya UESHIN Closes #1888 from ueshin/issues/SPARK-2968 and squashes the following commits: d128c95 [Takuya UESHIN] Fix nullability of Explode. --- .../spark/sql/catalyst/expressions/generators.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 3d41acb79e5fd..e99c5b452d183 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -86,19 +86,19 @@ case class Explode(attributeNames: Seq[String], child: Expression) (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) private lazy val elementTypes = child.dataType match { - case ArrayType(et, _) => et :: Nil - case MapType(kt,vt, _) => kt :: vt :: Nil + case ArrayType(et, containsNull) => (et, containsNull) :: Nil + case MapType(kt, vt, valueContainsNull) => (kt, false) :: (vt, valueContainsNull) :: Nil } // TODO: Move this pattern into Generator. protected def makeOutput() = if (attributeNames.size == elementTypes.size) { attributeNames.zip(elementTypes).map { - case (n, t) => AttributeReference(n, t, nullable = true)() + case (n, (t, nullable)) => AttributeReference(n, t, nullable)() } } else { elementTypes.zipWithIndex.map { - case (t, i) => AttributeReference(s"c_$i", t, nullable = true)() + case ((t, nullable), i) => AttributeReference(s"c_$i", t, nullable)() } } From bad21ed085a505559dccc06223b486170371ddd2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 11 Aug 2014 20:21:56 -0700 Subject: [PATCH 0322/1492] [SPARK-2650][SQL] Build column buffers in smaller batches Author: Michael Armbrust Closes #1880 from marmbrus/columnBatches and squashes the following commits: 0649987 [Michael Armbrust] add test 4756fad [Michael Armbrust] fix compilation 2314532 [Michael Armbrust] Build column buffers in smaller batches --- .../scala/org/apache/spark/sql/SQLConf.scala | 4 + .../org/apache/spark/sql/SQLContext.scala | 4 +- .../columnar/InMemoryColumnarTableScan.scala | 76 ++++++++++++------- .../apache/spark/sql/CachedTableSuite.scala | 12 ++- .../columnar/InMemoryColumnarQuerySuite.scala | 6 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 2 +- 7 files changed, 70 insertions(+), 36 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 0fd7aaaa36eb8..35c51dec0bcf5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -25,6 +25,7 @@ import java.util.Properties private[spark] object SQLConf { val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed" + val COLUMN_BATCH_SIZE = "spark.sql.inMemoryColumnarStorage.batchSize" val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold" val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" @@ -71,6 +72,9 @@ trait SQLConf { /** When true tables cached using the in-memory columnar caching will be compressed. */ private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "false").toBoolean + /** The number of rows that will be */ + private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt + /** Number of partitions to use for shuffle operators. */ private[spark] def numShufflePartitions: Int = getConf(SHUFFLE_PARTITIONS, "200").toInt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 71d338d21d0f2..af9f7c62a1d25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -273,7 +273,7 @@ class SQLContext(@transient val sparkContext: SparkContext) currentTable.logicalPlan case _ => - InMemoryRelation(useCompression, executePlan(currentTable).executedPlan) + InMemoryRelation(useCompression, columnBatchSize, executePlan(currentTable).executedPlan) } catalog.registerTable(None, tableName, asInMemoryRelation) @@ -284,7 +284,7 @@ class SQLContext(@transient val sparkContext: SparkContext) table(tableName).queryExecution.analyzed match { // This is kind of a hack to make sure that if this was just an RDD registered as a table, // we reregister the RDD as a table. - case inMem @ InMemoryRelation(_, _, e: ExistingRdd) => + case inMem @ InMemoryRelation(_, _, _, e: ExistingRdd) => inMem.cachedColumnBuffers.unpersist() catalog.unregisterTable(None, tableName) catalog.registerTable(None, tableName, SparkLogicalPlan(e)(self)) 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 88901debbb4e9..3364d0e18bcc9 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 @@ -28,13 +28,14 @@ import org.apache.spark.sql.Row import org.apache.spark.SparkConf object InMemoryRelation { - def apply(useCompression: Boolean, child: SparkPlan): InMemoryRelation = - new InMemoryRelation(child.output, useCompression, child)() + def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = + new InMemoryRelation(child.output, useCompression, batchSize, child)() } private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, + batchSize: Int, child: SparkPlan) (private var _cachedColumnBuffers: RDD[Array[ByteBuffer]] = null) extends LogicalPlan with MultiInstanceRelation { @@ -43,22 +44,31 @@ private[sql] case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { val output = child.output - val cached = child.execute().mapPartitions { iterator => - val columnBuilders = output.map { attribute => - ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression) - }.toArray - - var row: Row = null - while (iterator.hasNext) { - row = iterator.next() - var i = 0 - while (i < row.length) { - columnBuilders(i).appendFrom(row, i) - i += 1 + val cached = child.execute().mapPartitions { baseIterator => + new Iterator[Array[ByteBuffer]] { + def next() = { + val columnBuilders = output.map { attribute => + ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression) + }.toArray + + var row: Row = null + var rowCount = 0 + + while (baseIterator.hasNext && rowCount < batchSize) { + row = baseIterator.next() + var i = 0 + while (i < row.length) { + columnBuilders(i).appendFrom(row, i) + i += 1 + } + rowCount += 1 + } + + columnBuilders.map(_.build()) } - } - Iterator.single(columnBuilders.map(_.build())) + def hasNext = baseIterator.hasNext + } }.cache() cached.setName(child.toString) @@ -74,6 +84,7 @@ private[sql] case class InMemoryRelation( new InMemoryRelation( output.map(_.newInstance), useCompression, + batchSize, child)( _cachedColumnBuffers).asInstanceOf[this.type] } @@ -90,22 +101,31 @@ private[sql] case class InMemoryColumnarTableScan( override def execute() = { relation.cachedColumnBuffers.mapPartitions { iterator => - val columnBuffers = iterator.next() - assert(!iterator.hasNext) + // Find the ordinals of the requested columns. If none are requested, use the first. + val requestedColumns = + if (attributes.isEmpty) { + Seq(0) + } else { + attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) + } new Iterator[Row] { - // Find the ordinals of the requested columns. If none are requested, use the first. - val requestedColumns = - if (attributes.isEmpty) { - Seq(0) - } else { - attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) - } + private[this] var columnBuffers: Array[ByteBuffer] = null + private[this] var columnAccessors: Seq[ColumnAccessor] = null + nextBatch() + + private[this] val nextRow = new GenericMutableRow(columnAccessors.length) - val columnAccessors = requestedColumns.map(columnBuffers(_)).map(ColumnAccessor(_)) - val nextRow = new GenericMutableRow(columnAccessors.length) + def nextBatch() = { + columnBuffers = iterator.next() + columnAccessors = requestedColumns.map(columnBuffers(_)).map(ColumnAccessor(_)) + } override def next() = { + if (!columnAccessors.head.hasNext) { + nextBatch() + } + var i = 0 while (i < nextRow.length) { columnAccessors(i).extractTo(nextRow, i) @@ -114,7 +134,7 @@ private[sql] case class InMemoryColumnarTableScan( nextRow } - override def hasNext = columnAccessors.head.hasNext + override def hasNext = columnAccessors.head.hasNext || iterator.hasNext } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index fbf9bd9dbcdea..befef46d93973 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,9 +22,19 @@ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableSca import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ +case class BigData(s: String) + class CachedTableSuite extends QueryTest { TestData // Load test tables. + test("too big for memory") { + val data = "*" * 10000 + sparkContext.parallelize(1 to 1000000, 1).map(_ => BigData(data)).registerTempTable("bigData") + cacheTable("bigData") + assert(table("bigData").count() === 1000000L) + uncacheTable("bigData") + } + test("SPARK-1669: cacheTable should be idempotent") { assume(!table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) @@ -37,7 +47,7 @@ class CachedTableSuite extends QueryTest { cacheTable("testData") table("testData").queryExecution.analyzed match { - case InMemoryRelation(_, _, _: InMemoryColumnarTableScan) => + case InMemoryRelation(_, _, _, _: InMemoryColumnarTableScan) => fail("cacheTable is not idempotent") case _ => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index b561b44ad7ee2..736c0f8571e9e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -28,14 +28,14 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("simple columnar query") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, plan) + val scan = InMemoryRelation(useCompression = true, 5, plan) checkAnswer(scan, testData.collect().toSeq) } test("projection") { val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, plan) + val scan = InMemoryRelation(useCompression = true, 5, plan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -44,7 +44,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, plan) + val scan = InMemoryRelation(useCompression = true, 5, plan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) 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 82e9c1a248626..3b371211e14cd 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 @@ -137,7 +137,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with castChildOutput(p, table, child) case p @ logical.InsertIntoTable( - InMemoryRelation(_, _, + InMemoryRelation(_, _, _, HiveTableScan(_, table, _)), _, child, _) => castChildOutput(p, table, child) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 85d2496a34cfb..5fcc1bd4b9adf 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -45,7 +45,7 @@ private[hive] trait HiveStrategies { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case logical.InsertIntoTable( - InMemoryRelation(_, _, + InMemoryRelation(_, _, _, HiveTableScan(_, table, _)), partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil From 5d54d71ddbac1fbb26925a8c9138bbb8c0e81db8 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 11 Aug 2014 20:45:14 -0700 Subject: [PATCH 0323/1492] [SQL] [SPARK-2826] Reduce the memory copy while building the hashmap for HashOuterJoin This is a follow up for #1147 , this PR will improve the performance about 10% - 15% in my local tests. ``` Before: LeftOuterJoin: took 16750 ms ([3000000] records) LeftOuterJoin: took 15179 ms ([3000000] records) RightOuterJoin: took 15515 ms ([3000000] records) RightOuterJoin: took 15276 ms ([3000000] records) FullOuterJoin: took 19150 ms ([6000000] records) FullOuterJoin: took 18935 ms ([6000000] records) After: LeftOuterJoin: took 15218 ms ([3000000] records) LeftOuterJoin: took 13503 ms ([3000000] records) RightOuterJoin: took 13663 ms ([3000000] records) RightOuterJoin: took 14025 ms ([3000000] records) FullOuterJoin: took 16624 ms ([6000000] records) FullOuterJoin: took 16578 ms ([6000000] records) ``` Besides the performance improvement, I also do some clean up as suggested in #1147 Author: Cheng Hao Closes #1765 from chenghao-intel/hash_outer_join_fixing and squashes the following commits: ab1f9e0 [Cheng Hao] Reduce the memory copy while building the hashmap --- .../apache/spark/sql/execution/joins.scala | 54 ++++++++++--------- 1 file changed, 28 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index ea075f8c65bff..c86811e838bd8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import java.util.{HashMap => JavaHashMap} + import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent._ @@ -136,14 +138,6 @@ trait HashJoin { } } -/** - * Constant Value for Binary Join Node - */ -object HashOuterJoin { - val DUMMY_LIST = Seq[Row](null) - val EMPTY_LIST = Seq[Row]() -} - /** * :: DeveloperApi :: * Performs a hash based outer join for two child relations by shuffling the data using @@ -181,6 +175,9 @@ case class HashOuterJoin( } } + @transient private[this] lazy val DUMMY_LIST = Seq[Row](null) + @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row] + // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala // iterator for performance purpose. @@ -199,8 +196,8 @@ case class HashOuterJoin( joinedRow.copy } else { Nil - }) ++ HashOuterJoin.DUMMY_LIST.filter(_ => !matched).map( _ => { - // HashOuterJoin.DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, // as we don't know whether we need to append it until finish iterating all of the // records in right side. // If we didn't get any proper row, then append a single row with empty right @@ -224,8 +221,8 @@ case class HashOuterJoin( joinedRow.copy } else { Nil - }) ++ HashOuterJoin.DUMMY_LIST.filter(_ => !matched).map( _ => { - // HashOuterJoin.DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, // as we don't know whether we need to append it until finish iterating all of the // records in left side. // If we didn't get any proper row, then append a single row with empty left. @@ -259,10 +256,10 @@ case class HashOuterJoin( rightMatchedSet.add(idx) joinedRow.copy } - } ++ HashOuterJoin.DUMMY_LIST.filter(_ => !matched).map( _ => { + } ++ DUMMY_LIST.filter(_ => !matched).map( _ => { // 2. For those unmatched records in left, append additional records with empty right. - // HashOuterJoin.DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, // as we don't know whether we need to append it until finish iterating all // of the records in right side. // If we didn't get any proper row, then append a single row with empty right. @@ -287,18 +284,22 @@ case class HashOuterJoin( } private[this] def buildHashTable( - iter: Iterator[Row], keyGenerator: Projection): Map[Row, ArrayBuffer[Row]] = { - // TODO: Use Spark's HashMap implementation. - val hashTable = scala.collection.mutable.Map[Row, ArrayBuffer[Row]]() + iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, ArrayBuffer[Row]] = { + val hashTable = new JavaHashMap[Row, ArrayBuffer[Row]]() while (iter.hasNext) { val currentRow = iter.next() val rowKey = keyGenerator(currentRow) - val existingMatchList = hashTable.getOrElseUpdate(rowKey, {new ArrayBuffer[Row]()}) + var existingMatchList = hashTable.get(rowKey) + if (existingMatchList == null) { + existingMatchList = new ArrayBuffer[Row]() + hashTable.put(rowKey, existingMatchList) + } + existingMatchList += currentRow.copy() } - - hashTable.toMap[Row, ArrayBuffer[Row]] + + hashTable } def execute() = { @@ -309,21 +310,22 @@ case class HashOuterJoin( // Build HashMap for current partition in right relation val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + import scala.collection.JavaConversions._ val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) joinType match { case LeftOuter => leftHashTable.keysIterator.flatMap { key => - leftOuterIterator(key, leftHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST), - rightHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST)) + leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) } case RightOuter => rightHashTable.keysIterator.flatMap { key => - rightOuterIterator(key, leftHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST), - rightHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST)) + rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) } case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => fullOuterIterator(key, - leftHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST), - rightHashTable.getOrElse(key, HashOuterJoin.EMPTY_LIST)) + leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) } case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") } From 9038d94e1e50e05de00fd51af4fd7b9280481cdc Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 11 Aug 2014 22:33:45 -0700 Subject: [PATCH 0324/1492] [SPARK-2923][MLLIB] Implement some basic BLAS routines Having some basic BLAS operations implemented in MLlib can help simplify the current implementation and improve some performance. Tested on my local machine: ~~~ bin/spark-submit --class org.apache.spark.examples.mllib.BinaryClassification \ examples/target/scala-*/spark-examples-*.jar --algorithm LR --regType L2 \ --regParam 1.0 --numIterations 1000 ~/share/data/rcv1.binary/rcv1_train.binary ~~~ 1. before: ~1m 2. after: ~30s CC: jkbradley Author: Xiangrui Meng Closes #1849 from mengxr/ml-blas and squashes the following commits: ba583a2 [Xiangrui Meng] exclude Vector.copy a4d7d2f [Xiangrui Meng] Merge branch 'master' into ml-blas 6edeab9 [Xiangrui Meng] address comments 940bdeb [Xiangrui Meng] rename MLlibBLAS to BLAS c2a38bc [Xiangrui Meng] enhance dot tests 4cfaac4 [Xiangrui Meng] add apache header 48d01d2 [Xiangrui Meng] add tests for zeros and copy 3b882b1 [Xiangrui Meng] use blas.scal in gradient 735eb23 [Xiangrui Meng] remove d from BLAS routines d2d7d3c [Xiangrui Meng] update gradient and lbfgs 7f78186 [Xiangrui Meng] add zeros to Vectors; add dscal and dcopy to BLAS 14e6645 [Xiangrui Meng] add ddot cbb8273 [Xiangrui Meng] add daxpy test 07db0bb [Xiangrui Meng] Merge branch 'master' into ml-blas e8c326d [Xiangrui Meng] axpy --- .../org/apache/spark/mllib/linalg/BLAS.scala | 200 ++++++++++++++++++ .../apache/spark/mllib/linalg/Vectors.scala | 35 ++- .../spark/mllib/optimization/Gradient.scala | 60 ++---- .../spark/mllib/optimization/LBFGS.scala | 39 ++-- .../apache/spark/mllib/linalg/BLASSuite.scala | 129 +++++++++++ .../spark/mllib/linalg/VectorsSuite.scala | 30 +++ project/MimaExcludes.scala | 5 +- 7 files changed, 432 insertions(+), 66 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala new file mode 100644 index 0000000000000..70e23033c8754 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -0,0 +1,200 @@ +/* + * 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.linalg + +import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} + +/** + * BLAS routines for MLlib's vectors and matrices. + */ +private[mllib] object BLAS extends Serializable { + + @transient private var _f2jBLAS: NetlibBLAS = _ + + // For level-1 routines, we use Java implementation. + private def f2jBLAS: NetlibBLAS = { + if (_f2jBLAS == null) { + _f2jBLAS = new F2jBLAS + } + _f2jBLAS + } + + /** + * y += a * x + */ + def axpy(a: Double, x: Vector, y: Vector): Unit = { + require(x.size == y.size) + y match { + case dy: DenseVector => + x match { + case sx: SparseVector => + axpy(a, sx, dy) + case dx: DenseVector => + axpy(a, dx, dy) + case _ => + throw new UnsupportedOperationException( + s"axpy doesn't support x type ${x.getClass}.") + } + case _ => + throw new IllegalArgumentException( + s"axpy only supports adding to a dense vector but got type ${y.getClass}.") + } + } + + /** + * y += a * x + */ + private def axpy(a: Double, x: DenseVector, y: DenseVector): Unit = { + val n = x.size + f2jBLAS.daxpy(n, a, x.values, 1, y.values, 1) + } + + /** + * y += a * x + */ + private def axpy(a: Double, x: SparseVector, y: DenseVector): Unit = { + val nnz = x.indices.size + if (a == 1.0) { + var k = 0 + while (k < nnz) { + y.values(x.indices(k)) += x.values(k) + k += 1 + } + } else { + var k = 0 + while (k < nnz) { + y.values(x.indices(k)) += a * x.values(k) + k += 1 + } + } + } + + /** + * dot(x, y) + */ + def dot(x: Vector, y: Vector): Double = { + require(x.size == y.size) + (x, y) match { + case (dx: DenseVector, dy: DenseVector) => + dot(dx, dy) + case (sx: SparseVector, dy: DenseVector) => + dot(sx, dy) + case (dx: DenseVector, sy: SparseVector) => + dot(sy, dx) + case (sx: SparseVector, sy: SparseVector) => + dot(sx, sy) + case _ => + throw new IllegalArgumentException(s"dot doesn't support (${x.getClass}, ${y.getClass}).") + } + } + + /** + * dot(x, y) + */ + private def dot(x: DenseVector, y: DenseVector): Double = { + val n = x.size + f2jBLAS.ddot(n, x.values, 1, y.values, 1) + } + + /** + * dot(x, y) + */ + private def dot(x: SparseVector, y: DenseVector): Double = { + val nnz = x.indices.size + var sum = 0.0 + var k = 0 + while (k < nnz) { + sum += x.values(k) * y.values(x.indices(k)) + k += 1 + } + sum + } + + /** + * dot(x, y) + */ + private def dot(x: SparseVector, y: SparseVector): Double = { + var kx = 0 + val nnzx = x.indices.size + var ky = 0 + val nnzy = y.indices.size + var sum = 0.0 + // y catching x + while (kx < nnzx && ky < nnzy) { + val ix = x.indices(kx) + while (ky < nnzy && y.indices(ky) < ix) { + ky += 1 + } + if (ky < nnzy && y.indices(ky) == ix) { + sum += x.values(kx) * y.values(ky) + ky += 1 + } + kx += 1 + } + sum + } + + /** + * y = x + */ + def copy(x: Vector, y: Vector): Unit = { + val n = y.size + require(x.size == n) + y match { + case dy: DenseVector => + x match { + case sx: SparseVector => + var i = 0 + var k = 0 + val nnz = sx.indices.size + while (k < nnz) { + val j = sx.indices(k) + while (i < j) { + dy.values(i) = 0.0 + i += 1 + } + dy.values(i) = sx.values(k) + i += 1 + k += 1 + } + while (i < n) { + dy.values(i) = 0.0 + i += 1 + } + case dx: DenseVector => + Array.copy(dx.values, 0, dy.values, 0, n) + } + case _ => + throw new IllegalArgumentException(s"y must be dense in copy but got ${y.getClass}") + } + } + + /** + * x = a * x + */ + def scal(a: Double, x: Vector): Unit = { + x match { + case sx: SparseVector => + f2jBLAS.dscal(sx.values.size, a, sx.values, 1) + case dx: DenseVector => + f2jBLAS.dscal(dx.values.size, a, dx.values, 1) + case _ => + throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 77b3e8c714997..a45781d12e41e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.linalg import java.lang.{Double => JavaDouble, Integer => JavaInteger, Iterable => JavaIterable} -import java.util.Arrays +import java.util import scala.annotation.varargs import scala.collection.JavaConverters._ @@ -30,6 +30,8 @@ import org.apache.spark.SparkException /** * Represents a numeric vector, whose index type is Int and value type is Double. + * + * Note: Users should not implement this interface. */ trait Vector extends Serializable { @@ -46,12 +48,12 @@ trait Vector extends Serializable { override def equals(other: Any): Boolean = { other match { case v: Vector => - Arrays.equals(this.toArray, v.toArray) + util.Arrays.equals(this.toArray, v.toArray) case _ => false } } - override def hashCode(): Int = Arrays.hashCode(this.toArray) + override def hashCode(): Int = util.Arrays.hashCode(this.toArray) /** * Converts the instance to a breeze vector. @@ -63,6 +65,13 @@ trait Vector extends Serializable { * @param i index */ def apply(i: Int): Double = toBreeze(i) + + /** + * Makes a deep copy of this vector. + */ + def copy: Vector = { + throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") + } } /** @@ -127,6 +136,16 @@ object Vectors { }.toSeq) } + /** + * Creates a dense vector of all zeros. + * + * @param size vector size + * @return a zero vector + */ + def zeros(size: Int): Vector = { + new DenseVector(new Array[Double](size)) + } + /** * Parses a string resulted from `Vector#toString` into * an [[org.apache.spark.mllib.linalg.Vector]]. @@ -142,7 +161,7 @@ object Vectors { case Seq(size: Double, indices: Array[Double], values: Array[Double]) => Vectors.sparse(size.toInt, indices.map(_.toInt), values) case other => - throw new SparkException(s"Cannot parse $other.") + throw new SparkException(s"Cannot parse $other.") } } @@ -183,6 +202,10 @@ class DenseVector(val values: Array[Double]) extends Vector { private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) override def apply(i: Int) = values(i) + + override def copy: DenseVector = { + new DenseVector(values.clone()) + } } /** @@ -213,5 +236,9 @@ class SparseVector( data } + override def copy: SparseVector = { + new SparseVector(size, indices.clone(), values.clone()) + } + private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 9d82f011e674a..fdd67160114ca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -17,10 +17,9 @@ package org.apache.spark.mllib.optimization -import breeze.linalg.{axpy => brzAxpy} - import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS.{axpy, dot, scal} /** * :: DeveloperApi :: @@ -61,11 +60,10 @@ abstract class Gradient extends Serializable { @DeveloperApi class LogisticGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val margin: Double = -1.0 * brzWeights.dot(brzData) + val margin = -1.0 * dot(data, weights) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - val gradient = brzData * gradientMultiplier + val gradient = data.copy + scal(gradientMultiplier, gradient) val loss = if (label > 0) { math.log1p(math.exp(margin)) // log1p is log(1+p) but more accurate for small p @@ -73,7 +71,7 @@ class LogisticGradient extends Gradient { math.log1p(math.exp(margin)) - margin } - (Vectors.fromBreeze(gradient), loss) + (gradient, loss) } override def compute( @@ -81,13 +79,9 @@ class LogisticGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val margin: Double = -1.0 * brzWeights.dot(brzData) + val margin = -1.0 * dot(data, weights) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - - brzAxpy(gradientMultiplier, brzData, cumGradient.toBreeze) - + axpy(gradientMultiplier, data, cumGradient) if (label > 0) { math.log1p(math.exp(margin)) } else { @@ -106,13 +100,11 @@ class LogisticGradient extends Gradient { @DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val diff = brzWeights.dot(brzData) - label + val diff = dot(data, weights) - label val loss = diff * diff - val gradient = brzData * (2.0 * diff) - - (Vectors.fromBreeze(gradient), loss) + val gradient = data.copy + scal(2.0 * diff, gradient) + (gradient, loss) } override def compute( @@ -120,12 +112,8 @@ class LeastSquaresGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val diff = brzWeights.dot(brzData) - label - - brzAxpy(2.0 * diff, brzData, cumGradient.toBreeze) - + val diff = dot(data, weights) - label + axpy(2.0 * diff, data, cumGradient) diff * diff } } @@ -139,18 +127,16 @@ class LeastSquaresGradient extends Gradient { @DeveloperApi class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val dotProduct = brzWeights.dot(brzData) - + val dotProduct = dot(data, weights) // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 - if (1.0 > labelScaled * dotProduct) { - (Vectors.fromBreeze(brzData * (-labelScaled)), 1.0 - labelScaled * dotProduct) + val gradient = data.copy + scal(-labelScaled, gradient) + (gradient, 1.0 - labelScaled * dotProduct) } else { - (Vectors.dense(new Array[Double](weights.size)), 0.0) + (Vectors.sparse(weights.size, Array.empty, Array.empty), 0.0) } } @@ -159,16 +145,12 @@ class HingeGradient extends Gradient { label: Double, weights: Vector, cumGradient: Vector): Double = { - val brzData = data.toBreeze - val brzWeights = weights.toBreeze - val dotProduct = brzWeights.dot(brzData) - + val dotProduct = dot(data, weights) // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 - if (1.0 > labelScaled * dotProduct) { - brzAxpy(-labelScaled, brzData, cumGradient.toBreeze) + axpy(-labelScaled, data, cumGradient) 1.0 - labelScaled * dotProduct } else { 0.0 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 26a2b62e76ed0..033fe44f34f3c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -19,14 +19,15 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseVector => BDV, axpy} +import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.mllib.rdd.RDDFunctions._ +import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: @@ -192,31 +193,29 @@ object LBFGS extends Logging { regParam: Double, numExamples: Long) extends DiffFunction[BDV[Double]] { - private var i = 0 - - override def calculate(weights: BDV[Double]) = { + override def calculate(weights: BDV[Double]): (Double, BDV[Double]) = { // Have a local copy to avoid the serialization of CostFun object which is not serializable. + val w = Vectors.fromBreeze(weights) + val n = w.size + val bcW = data.context.broadcast(w) val localGradient = gradient - val n = weights.length - val bcWeights = data.context.broadcast(weights) - val (gradientSum, lossSum) = data.treeAggregate((BDV.zeros[Double](n), 0.0))( + val (gradientSum, lossSum) = data.treeAggregate((Vectors.zeros(n), 0.0))( seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => val l = localGradient.compute( - features, label, Vectors.fromBreeze(bcWeights.value), Vectors.fromBreeze(grad)) + features, label, bcW.value, grad) (grad, loss + l) }, combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => - (grad1 += grad2, loss1 + loss2) + axpy(1.0, grad2, grad1) + (grad1, loss1 + loss2) }) /** * regVal is sum of weight squares if it's L2 updater; * for other updater, the same logic is followed. */ - val regVal = updater.compute( - Vectors.fromBreeze(weights), - Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 + val regVal = updater.compute(w, Vectors.zeros(n), 0, 1, regParam)._2 val loss = lossSum / numExamples + regVal /** @@ -236,17 +235,13 @@ object LBFGS extends Logging { */ // The following gradientTotal is actually the regularization part of gradient. // Will add the gradientSum computed from the data with weights in the next step. - val gradientTotal = weights - updater.compute( - Vectors.fromBreeze(weights), - Vectors.dense(new Array[Double](weights.size)), 1, 1, regParam)._1.toBreeze + val gradientTotal = w.copy + axpy(-1.0, updater.compute(w, Vectors.zeros(n), 1, 1, regParam)._1, gradientTotal) // gradientTotal = gradientSum / numExamples + gradientTotal axpy(1.0 / numExamples, gradientSum, gradientTotal) - i += 1 - - (loss, gradientTotal) + (loss, gradientTotal.toBreeze.asInstanceOf[BDV[Double]]) } } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala new file mode 100644 index 0000000000000..1952e6734ecf7 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.mllib.linalg.BLAS._ + +class BLASSuite extends FunSuite { + + test("copy") { + val sx = Vectors.sparse(4, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0, 0.0) + val sy = Vectors.sparse(4, Array(0, 1, 3), Array(2.0, 1.0, 1.0)) + val dy = Array(2.0, 1.0, 0.0, 1.0) + + val dy1 = Vectors.dense(dy.clone()) + copy(sx, dy1) + assert(dy1 ~== dx absTol 1e-15) + + val dy2 = Vectors.dense(dy.clone()) + copy(dx, dy2) + assert(dy2 ~== dx absTol 1e-15) + + intercept[IllegalArgumentException] { + copy(sx, sy) + } + + intercept[IllegalArgumentException] { + copy(dx, sy) + } + + withClue("vector sizes must match") { + intercept[Exception] { + copy(sx, Vectors.dense(0.0, 1.0, 2.0)) + } + } + } + + test("scal") { + val a = 0.1 + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + + scal(a, sx) + assert(sx ~== Vectors.sparse(3, Array(0, 2), Array(0.1, -0.2)) absTol 1e-15) + + scal(a, dx) + assert(dx ~== Vectors.dense(0.1, 0.0, -0.2) absTol 1e-15) + } + + test("axpy") { + val alpha = 0.1 + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + val dy = Array(2.0, 1.0, 0.0) + val expected = Vectors.dense(2.1, 1.0, -0.2) + + val dy1 = Vectors.dense(dy.clone()) + axpy(alpha, sx, dy1) + assert(dy1 ~== expected absTol 1e-15) + + val dy2 = Vectors.dense(dy.clone()) + axpy(alpha, dx, dy2) + assert(dy2 ~== expected absTol 1e-15) + + val sy = Vectors.sparse(4, Array(0, 1), Array(2.0, 1.0)) + + intercept[IllegalArgumentException] { + axpy(alpha, sx, sy) + } + + intercept[IllegalArgumentException] { + axpy(alpha, dx, sy) + } + + withClue("vector sizes must match") { + intercept[Exception] { + axpy(alpha, sx, Vectors.dense(1.0, 2.0)) + } + } + } + + test("dot") { + val sx = Vectors.sparse(3, Array(0, 2), Array(1.0, -2.0)) + val dx = Vectors.dense(1.0, 0.0, -2.0) + val sy = Vectors.sparse(3, Array(0, 1), Array(2.0, 1.0)) + val dy = Vectors.dense(2.0, 1.0, 0.0) + + assert(dot(sx, sy) ~== 2.0 absTol 1e-15) + assert(dot(sy, sx) ~== 2.0 absTol 1e-15) + assert(dot(sx, dy) ~== 2.0 absTol 1e-15) + assert(dot(dy, sx) ~== 2.0 absTol 1e-15) + assert(dot(dx, dy) ~== 2.0 absTol 1e-15) + assert(dot(dy, dx) ~== 2.0 absTol 1e-15) + + assert(dot(sx, sx) ~== 5.0 absTol 1e-15) + assert(dot(dx, dx) ~== 5.0 absTol 1e-15) + assert(dot(sx, dx) ~== 5.0 absTol 1e-15) + assert(dot(dx, sx) ~== 5.0 absTol 1e-15) + + val sx1 = Vectors.sparse(10, Array(0, 3, 5, 7, 8), Array(1.0, 2.0, 3.0, 4.0, 5.0)) + val sx2 = Vectors.sparse(10, Array(1, 3, 6, 7, 9), Array(1.0, 2.0, 3.0, 4.0, 5.0)) + assert(dot(sx1, sx2) ~== 20.0 absTol 1e-15) + assert(dot(sx2, sx1) ~== 20.0 absTol 1e-15) + + withClue("vector sizes must match") { + intercept[Exception] { + dot(sx, Vectors.dense(2.0, 1.0)) + } + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 7972ceea1fe8a..cd651fe2d2ddf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -125,4 +125,34 @@ class VectorsSuite extends FunSuite { } } } + + test("zeros") { + assert(Vectors.zeros(3) === Vectors.dense(0.0, 0.0, 0.0)) + } + + test("Vector.copy") { + val sv = Vectors.sparse(4, Array(0, 2), Array(1.0, 2.0)) + val svCopy = sv.copy + (sv, svCopy) match { + case (sv: SparseVector, svCopy: SparseVector) => + assert(sv.size === svCopy.size) + assert(sv.indices === svCopy.indices) + assert(sv.values === svCopy.values) + assert(!sv.indices.eq(svCopy.indices)) + assert(!sv.values.eq(svCopy.values)) + case _ => + throw new RuntimeException(s"copy returned ${svCopy.getClass} on ${sv.getClass}.") + } + + val dv = Vectors.dense(1.0, 0.0, 2.0) + val dvCopy = dv.copy + (dv, dvCopy) match { + case (dv: DenseVector, dvCopy: DenseVector) => + assert(dv.size === dvCopy.size) + assert(dv.values === dvCopy.values) + assert(!dv.values.eq(dvCopy.values)) + case _ => + throw new RuntimeException(s"copy returned ${dvCopy.getClass} on ${dv.getClass}.") + } + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b4653c72c10b5..6e72035f2c15b 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -111,9 +111,12 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") ) ++ - Seq ( // package-private classes removed in MLlib + Seq( // package-private classes removed in MLlib ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.regression.GeneralizedLinearAlgorithm.org$apache$spark$mllib$regression$GeneralizedLinearAlgorithm$$prependOne") + ) ++ + Seq( // new Vector methods in MLlib (binary compatible assuming users do not implement Vector) + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.copy") ) case v if v.startsWith("1.0") => Seq( From f0060b75ff67ab60babf54149a6860edc53cb6e9 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Tue, 12 Aug 2014 00:28:00 -0700 Subject: [PATCH 0325/1492] [MLlib] Correctly set vectorSize and alpha mengxr Correctly set vectorSize and alpha in Word2Vec training. Author: Liquan Pei Closes #1900 from Ishiihara/Word2Vec-bugfix and squashes the following commits: 85f64f2 [Liquan Pei] correctly set vectorSize and alpha --- .../apache/spark/mllib/feature/Word2Vec.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 395037e1ec47c..ecd49ea2ff533 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -119,7 +119,6 @@ class Word2Vec extends Serializable with Logging { private val MAX_EXP = 6 private val MAX_CODE_LENGTH = 40 private val MAX_SENTENCE_LENGTH = 1000 - private val layer1Size = vectorSize /** context words from [-window, window] */ private val window = 5 @@ -131,7 +130,6 @@ class Word2Vec extends Serializable with Logging { private var vocabSize = 0 private var vocab: Array[VocabWord] = null private var vocabHash = mutable.HashMap.empty[String, Int] - private var alpha = startingAlpha private def learnVocab(words: RDD[String]): Unit = { vocab = words.map(w => (w, 1)) @@ -287,9 +285,10 @@ class Word2Vec extends Serializable with Logging { val newSentences = sentences.repartition(numPartitions).cache() val initRandom = new XORShiftRandom(seed) var syn0Global = - Array.fill[Float](vocabSize * layer1Size)((initRandom.nextFloat() - 0.5f) / layer1Size) - var syn1Global = new Array[Float](vocabSize * layer1Size) + Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) + var syn1Global = new Array[Float](vocabSize * vectorSize) + var alpha = startingAlpha for (k <- 1 to numIterations) { val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) @@ -317,24 +316,24 @@ class Word2Vec extends Serializable with Logging { val c = pos - window + a if (c >= 0 && c < sentence.size) { val lastWord = sentence(c) - val l1 = lastWord * layer1Size - val neu1e = new Array[Float](layer1Size) + val l1 = lastWord * vectorSize + val neu1e = new Array[Float](vectorSize) // Hierarchical softmax var d = 0 while (d < bcVocab.value(word).codeLen) { - val l2 = bcVocab.value(word).point(d) * layer1Size + val l2 = bcVocab.value(word).point(d) * vectorSize // Propagate hidden -> output - var f = blas.sdot(layer1Size, syn0, l1, 1, syn1, l2, 1) + var f = blas.sdot(vectorSize, syn0, l1, 1, syn1, l2, 1) if (f > -MAX_EXP && f < MAX_EXP) { val ind = ((f + MAX_EXP) * (EXP_TABLE_SIZE / MAX_EXP / 2.0)).toInt f = expTable.value(ind) val g = ((1 - bcVocab.value(word).code(d) - f) * alpha).toFloat - blas.saxpy(layer1Size, g, syn1, l2, 1, neu1e, 0, 1) - blas.saxpy(layer1Size, g, syn0, l1, 1, syn1, l2, 1) + blas.saxpy(vectorSize, g, syn1, l2, 1, neu1e, 0, 1) + blas.saxpy(vectorSize, g, syn0, l1, 1, syn1, l2, 1) } d += 1 } - blas.saxpy(layer1Size, 1.0f, neu1e, 0, 1, syn0, l1, 1) + blas.saxpy(vectorSize, 1.0f, neu1e, 0, 1, syn0, l1, 1) } } a += 1 @@ -365,8 +364,8 @@ class Word2Vec extends Serializable with Logging { var i = 0 while (i < vocabSize) { val word = bcVocab.value(i).word - val vector = new Array[Float](layer1Size) - Array.copy(syn0Global, i * layer1Size, vector, 0, layer1Size) + val vector = new Array[Float](vectorSize) + Array.copy(syn0Global, i * vectorSize, vector, 0, vectorSize) word2VecMap += word -> vector i += 1 } From 882da57a1c8c075a87909d516b169b624941a6ec Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 12 Aug 2014 16:26:01 -0700 Subject: [PATCH 0326/1492] fix flaky tests Python 2.6 does not handle float error well as 2.7+ Author: Davies Liu Closes #1910 from davies/fix_test and squashes the following commits: 7e51200 [Davies Liu] fix flaky tests --- python/pyspark/sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 36040463e62a9..27f1d2ddf942a 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1094,7 +1094,7 @@ def applySchema(self, rdd, schema): ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + ... "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " + ... "float + 1.1 as float FROM table2").collect() - [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.1)] + [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.1...)] >>> rdd = sc.parallelize([(127, -32768, 1.0, ... datetime(2010, 1, 1, 1, 1, 1), From c235b83e2782cce0626ecc403c0a67e442be52c1 Mon Sep 17 00:00:00 2001 From: Ameet Talwalkar Date: Tue, 12 Aug 2014 17:15:21 -0700 Subject: [PATCH 0327/1492] SPARK-2830 [MLlib]: re-organize mllib documentation As per discussions with Xiangrui, I've reorganized and edited the mllib documentation. Author: Ameet Talwalkar Closes #1908 from atalwalkar/master and squashes the following commits: fe6938a [Ameet Talwalkar] made xiangruis suggested changes 840028b [Ameet Talwalkar] made xiangruis suggested changes 7ec366a [Ameet Talwalkar] reorganize and edit mllib documentation --- docs/mllib-basics.md | 117 +++++---------------- docs/mllib-classification-regression.md | 37 +++++++ docs/mllib-clustering.md | 15 +-- docs/mllib-collaborative-filtering.md | 21 ++-- docs/mllib-dimensionality-reduction.md | 44 ++++---- docs/mllib-feature-extraction.md | 12 +++ docs/mllib-guide.md | 30 +++--- docs/mllib-linear-methods.md | 134 ++++++++++++------------ docs/mllib-naive-bayes.md | 32 +++--- docs/mllib-stats.md | 95 +++++++++++++++++ 10 files changed, 317 insertions(+), 220 deletions(-) create mode 100644 docs/mllib-classification-regression.md create mode 100644 docs/mllib-feature-extraction.md create mode 100644 docs/mllib-stats.md diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md index f9585251fafac..8752df412950a 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-basics.md @@ -9,17 +9,17 @@ displayTitle: MLlib - Basics MLlib supports local vectors and matrices stored on a single machine, as well as distributed matrices backed by one or more RDDs. -In the current implementation, local vectors and matrices are simple data models -to serve public interfaces. The underlying linear algebra operations are provided by +Local vectors and local matrices are simple data models +that serve as public interfaces. The underlying linear algebra operations are provided by [Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). -A training example used in supervised learning is called "labeled point" in MLlib. +A training example used in supervised learning is called a "labeled point" in MLlib. ## Local vector A local vector has integer-typed and 0-based indices and double-typed values, stored on a single machine. MLlib supports two types of local vectors: dense and sparse. A dense vector is backed by a double array representing its entry values, while a sparse vector is backed by two parallel -arrays: indices and values. For example, a vector $(1.0, 0.0, 3.0)$ can be represented in dense +arrays: indices and values. For example, a vector `(1.0, 0.0, 3.0)` can be represented in dense format as `[1.0, 0.0, 3.0]` or in sparse format as `(3, [0, 2], [1.0, 3.0])`, where `3` is the size of the vector. @@ -44,8 +44,7 @@ val sv1: Vector = Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0)) val sv2: Vector = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0))) {% endhighlight %} -***Note*** - +***Note:*** Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. @@ -110,8 +109,8 @@ sv2 = sps.csc_matrix((np.array([1.0, 3.0]), np.array([0, 2]), np.array([0, 2])), A labeled point is a local vector, either dense or sparse, associated with a label/response. In MLlib, labeled points are used in supervised learning algorithms. We use a double to store a label, so we can use labeled points in both regression and classification. -For binary classification, label should be either $0$ (negative) or $1$ (positive). -For multiclass classification, labels should be class indices staring from zero: $0, 1, 2, \ldots$. +For binary classification, a label should be either `0` (negative) or `1` (positive). +For multiclass classification, labels should be class indices starting from zero: `0, 1, 2, ...`.
    @@ -172,7 +171,7 @@ neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) It is very common in practice to have sparse training data. MLlib supports reading training examples stored in `LIBSVM` format, which is the default format used by [`LIBSVM`](http://www.csie.ntu.edu.tw/~cjlin/libsvm/) and -[`LIBLINEAR`](http://www.csie.ntu.edu.tw/~cjlin/liblinear/). It is a text format. Each line +[`LIBLINEAR`](http://www.csie.ntu.edu.tw/~cjlin/liblinear/). It is a text format in which each line represents a labeled sparse feature vector using the following format: ~~~ @@ -226,7 +225,7 @@ examples = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") ## Local matrix A local matrix has integer-typed row and column indices and double-typed values, stored on a single -machine. MLlib supports dense matrix, whose entry values are stored in a single double array in +machine. MLlib supports dense matrices, whose entry values are stored in a single double array in column major. For example, the following matrix `\[ \begin{pmatrix} 1.0 & 2.0 \\ 3.0 & 4.0 \\ @@ -234,7 +233,6 @@ column major. For example, the following matrix `\[ \begin{pmatrix} \end{pmatrix} \]` is stored in a one-dimensional array `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]` with the matrix size `(3, 2)`. -We are going to add sparse matrix in the next release.
    @@ -242,7 +240,7 @@ We are going to add sparse matrix in the next release. The base class of local matrices is [`Matrix`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one implementation: [`DenseMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.DenseMatrix). -Sparse matrix will be added in the next release. We recommend using the factory methods implemented +We recommend using the factory methods implemented in [`Matrices`](api/scala/index.html#org.apache.spark.mllib.linalg.Matrices) to create local matrices. @@ -259,7 +257,7 @@ val dm: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0)) The base class of local matrices is [`Matrix`](api/java/org/apache/spark/mllib/linalg/Matrix.html), and we provide one implementation: [`DenseMatrix`](api/java/org/apache/spark/mllib/linalg/DenseMatrix.html). -Sparse matrix will be added in the next release. We recommend using the factory methods implemented +We recommend using the factory methods implemented in [`Matrices`](api/java/org/apache/spark/mllib/linalg/Matrices.html) to create local matrices. @@ -279,28 +277,30 @@ Matrix dm = Matrices.dense(3, 2, new double[] {1.0, 3.0, 5.0, 2.0, 4.0, 6.0}); A distributed matrix has long-typed row and column indices and double-typed values, stored distributively in one or more RDDs. It is very important to choose the right format to store large and distributed matrices. Converting a distributed matrix to a different format may require a -global shuffle, which is quite expensive. We implemented three types of distributed matrices in -this release and will add more types in the future. +global shuffle, which is quite expensive. Three types of distributed matrices have been implemented +so far. The basic type is called `RowMatrix`. A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, e.g., a collection of feature vectors. It is backed by an RDD of its rows, where each row is a local vector. -We assume that the number of columns is not huge for a `RowMatrix`. +We assume that the number of columns is not huge for a `RowMatrix` so that a single +local vector can be reasonably communicated to the driver and can also be stored / +operated on using a single node. An `IndexedRowMatrix` is similar to a `RowMatrix` but with row indices, -which can be used for identifying rows and joins. -A `CoordinateMatrix` is a distributed matrix stored in [coordinate list (COO)](https://en.wikipedia.org/wiki/Sparse_matrix) format, +which can be used for identifying rows and executing joins. +A `CoordinateMatrix` is a distributed matrix stored in [coordinate list (COO)](https://en.wikipedia.org/wiki/Sparse_matrix#Coordinate_list_.28COO.29) format, backed by an RDD of its entries. ***Note*** The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size. -It is always error-prone to have non-deterministic RDDs. +In general the use of non-deterministic RDDs can lead to errors. ### RowMatrix A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, backed by an RDD -of its rows, where each row is a local vector. This is similar to `data matrix` in the context of -multivariate statistics. Since each row is represented by a local vector, the number of columns is +of its rows, where each row is a local vector. +Since each row is represented by a local vector, the number of columns is limited by the integer range but it should be much smaller in practice.
    @@ -344,70 +344,10 @@ long n = mat.numCols();
    -#### Multivariate summary statistics - -We provide column summary statistics for `RowMatrix`. -If the number of columns is not large, say, smaller than 3000, you can also compute -the covariance matrix as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the -number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, -which could be faster if the rows are sparse. - -
    -
    - -[`RowMatrix#computeColumnSummaryStatistics`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of -[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. - -{% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary - -val mat: RowMatrix = ... // a RowMatrix - -// Compute column summary statistics. -val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() -println(summary.mean) // a dense vector containing the mean value for each column -println(summary.variance) // column-wise variance -println(summary.numNonzeros) // number of nonzeros in each column - -// Compute the covariance matrix. -val cov: Matrix = mat.computeCovariance() -{% endhighlight %} -
    - -
    - -[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of -[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. - -{% highlight java %} -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; - -RowMatrix mat = ... // a RowMatrix - -// Compute column summary statistics. -MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); -System.out.println(summary.mean()); // a dense vector containing the mean value for each column -System.out.println(summary.variance()); // column-wise variance -System.out.println(summary.numNonzeros()); // number of nonzeros in each column - -// Compute the covariance matrix. -Matrix cov = mat.computeCovariance(); -{% endhighlight %} -
    -
    - ### IndexedRowMatrix An `IndexedRowMatrix` is similar to a `RowMatrix` but with meaningful row indices. It is backed by -an RDD of indexed rows, which each row is represented by its index (long-typed) and a local vector. +an RDD of indexed rows, so that each row is represented by its index (long-typed) and a local vector.
    @@ -467,7 +407,7 @@ RowMatrix rowMat = mat.toRowMatrix(); A `CoordinateMatrix` is a distributed matrix backed by an RDD of its entries. Each entry is a tuple of `(i: Long, j: Long, value: Double)`, where `i` is the row index, `j` is the column index, and -`value` is the entry value. A `CoordinateMatrix` should be used only in the case when both +`value` is the entry value. A `CoordinateMatrix` should be used only when both dimensions of the matrix are huge and the matrix is very sparse.
    @@ -477,9 +417,9 @@ A [`CoordinateMatrix`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) can be created from an `RDD[MatrixEntry]` instance, where [`MatrixEntry`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a -wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` -with sparse rows by calling `toIndexedRowMatrix`. In this release, we do not provide other -computation for `CoordinateMatrix`. +wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to an `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. Other computations for +`CoordinateMatrix` are not currently supported. {% highlight scala %} import org.apache.spark.mllib.linalg.distributed.{CoordinateMatrix, MatrixEntry} @@ -503,8 +443,9 @@ A [`CoordinateMatrix`](api/java/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.html) can be created from a `JavaRDD` instance, where [`MatrixEntry`](api/java/org/apache/spark/mllib/linalg/distributed/MatrixEntry.html) is a -wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` -with sparse rows by calling `toIndexedRowMatrix`. +wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to an `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. Other computations for +`CoordinateMatrix` are not currently supported. {% highlight java %} import org.apache.spark.api.java.JavaRDD; diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md new file mode 100644 index 0000000000000..719cc95767b00 --- /dev/null +++ b/docs/mllib-classification-regression.md @@ -0,0 +1,37 @@ +--- +layout: global +title: Classification and Regression - MLlib +displayTitle: MLlib - Classification and Regression +--- + +MLlib supports various methods for +[binary classification](http://en.wikipedia.org/wiki/Binary_classification), +[multiclass +classification](http://en.wikipedia.org/wiki/Multiclass_classification), and +[regression analysis](http://en.wikipedia.org/wiki/Regression_analysis). The table below outlines +the supported algorithms for each type of problem. + + + + + + + + + + + + + + + + +
    Problem TypeSupported Methods
    Binary Classificationlinear SVMs, logistic regression, decision trees, naive Bayes
    Multiclass Classificationdecision trees, naive Bayes
    Regressionlinear least squares, Lasso, ridge regression, decision trees
    + +More details for these methods can be found here: + +* [Linear models](mllib-linear-methods.html) + * [binary classification (SVMs, logistic regression)](mllib-linear-methods.html#binary-classification) + * [linear regression (least squares, Lasso, ridge)](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) +* [Decision trees](mllib-decision-tree.html) +* [Naive Bayes](mllib-naive-bayes.html) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 561de48910132..dfd9cd572888c 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -38,7 +38,7 @@ a given dataset, the algorithm returns the best clustering result).
    -Following code snippets can be executed in `spark-shell`. +The following code snippets can be executed in `spark-shell`. In the following example after loading and parsing data, we use the [`KMeans`](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) object to cluster the data @@ -70,7 +70,7 @@ All of MLlib's methods use Java-friendly types, so you can import and call them way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. A standalone application example -that is equivalent to the provided example in Scala is given bellow: +that is equivalent to the provided example in Scala is given below: {% highlight java %} import org.apache.spark.api.java.*; @@ -113,14 +113,15 @@ public class KMeansExample { } {% endhighlight %} -In order to run the above standalone application using Spark framework make -sure that you follow the instructions provided at section [Standalone -Applications](quick-start.html) of the quick-start guide. What is more, you -should include to your build file *spark-mllib* as a dependency. +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    -Following examples can be tested in the PySpark shell. +The following examples can be tested in the PySpark shell. In the following example after loading and parsing data, we use the KMeans object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then compute diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 0d28b5f7c89b3..ab10b2f01f87b 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -14,13 +14,13 @@ is commonly used for recommender systems. These techniques aim to fill in the missing entries of a user-item association matrix. MLlib currently supports model-based collaborative filtering, in which users and products are described by a small set of latent factors that can be used to predict missing entries. -In particular, we implement the [alternating least squares +MLlib uses the [alternating least squares (ALS)](http://dl.acm.org/citation.cfm?id=1608614) algorithm to learn these latent factors. The implementation in MLlib has the following parameters: * *numBlocks* is the number of blocks used to parallelize computation (set to -1 to auto-configure). -* *rank* is the number of latent factors in our model. +* *rank* is the number of latent factors in the model. * *iterations* is the number of iterations to run. * *lambda* specifies the regularization parameter in ALS. * *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for @@ -86,8 +86,8 @@ val MSE = ratesAndPreds.map { case ((user, product), (r1, r2)) => println("Mean Squared Error = " + MSE) {% endhighlight %} -If the rating matrix is derived from other source of information (i.e., it is inferred from -other signals), you can use the trainImplicit method to get better results. +If the rating matrix is derived from another source of information (e.g., it is inferred from +other signals), you can use the `trainImplicit` method to get better results. {% highlight scala %} val alpha = 0.01 @@ -174,10 +174,11 @@ public class CollaborativeFiltering { } {% endhighlight %} -In order to run the above standalone application using Spark framework make -sure that you follow the instructions provided at section [Standalone -Applications](quick-start.html) of the quick-start guide. What is more, you -should include to your build file *spark-mllib* as a dependency. +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    @@ -219,5 +220,5 @@ model = ALS.trainImplicit(ratings, rank, numIterations, alpha = 0.01) ## Tutorial -[AMP Camp](http://ampcamp.berkeley.edu/) provides a hands-on tutorial for -[personalized movie recommendation with MLlib](http://ampcamp.berkeley.edu/big-data-mini-course/movie-recommendation-with-mllib.html). +The [training exercises](https://databricks-training.s3.amazonaws.com/index.html) from the Spark Summit 2014 include a hands-on tutorial for +[personalized movie recommendation with MLlib](https://databricks-training.s3.amazonaws.com/movie-recommendation-with-mllib.html). diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 8e434998c15ea..065d646496131 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -9,9 +9,9 @@ displayTitle: MLlib - Dimensionality Reduction [Dimensionality reduction](http://en.wikipedia.org/wiki/Dimensionality_reduction) is the process of reducing the number of variables under consideration. -It is used to extract latent features from raw and noisy features, +It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -In this release, we provide preliminary support for dimensionality reduction on tall-and-skinny matrices. +MLlib provides support for dimensionality reduction on tall-and-skinny matrices. ## Singular value decomposition (SVD) @@ -30,17 +30,17 @@ where * $V$ is an orthonormal matrix, whose columns are called right singular vectors. For large matrices, usually we don't need the complete factorization but only the top singular -values and its associated singular vectors. This can save storage, and more importantly, de-noise +values and its associated singular vectors. This can save storage, de-noise and recover the low-rank structure of the matrix. -If we keep the top $k$ singular values, then the dimensions of the return will be: +If we keep the top $k$ singular values, then the dimensions of the resulting low-rank matrix will be: * `$U$`: `$m \times k$`, * `$\Sigma$`: `$k \times k$`, * `$V$`: `$n \times k$`. -In this release, we provide SVD computation to row-oriented matrices that have only a few columns, -say, less than $1000$, but many rows, which we call *tall-and-skinny*. +MLlib provides SVD functionality to row-oriented matrices that have only a few columns, +say, less than $1000$, but many rows, i.e., *tall-and-skinny* matrices.
    @@ -58,15 +58,10 @@ val s: Vector = svd.s // The singular values are stored in a local dense vector. val V: Matrix = svd.V // The V factor is a local dense matrix. {% endhighlight %} -Same code applies to `IndexedRowMatrix`. -The only difference that the `U` matrix becomes an `IndexedRowMatrix`. +The same code applies to `IndexedRowMatrix` if `U` is defined as an +`IndexedRowMatrix`.
    -In order to run the following standalone application using Spark framework make -sure that you follow the instructions provided at section [Standalone -Applications](quick-start.html) of the quick-start guide. What is more, you -should include to your build file *spark-mllib* as a dependency. - {% highlight java %} import java.util.LinkedList; @@ -104,8 +99,16 @@ public class SVD { } } {% endhighlight %} -Same code applies to `IndexedRowMatrix`. -The only difference that the `U` matrix becomes an `IndexedRowMatrix`. + +The same code applies to `IndexedRowMatrix` if `U` is defined as an +`IndexedRowMatrix`. + +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency. +
    @@ -116,7 +119,7 @@ statistical method to find a rotation such that the first coordinate has the lar possible, and each succeeding coordinate in turn has the largest variance possible. The columns of the rotation matrix are called principal components. PCA is used widely in dimensionality reduction. -In this release, we implement PCA for tall-and-skinny matrices stored in row-oriented format. +MLlib supports PCA for tall-and-skinny matrices stored in row-oriented format.
    @@ -180,9 +183,10 @@ public class PCA { } {% endhighlight %} -In order to run the above standalone application using Spark framework make -sure that you follow the instructions provided at section [Standalone -Applications](quick-start.html) of the quick-start guide. What is more, you -should include to your build file *spark-mllib* as a dependency. +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md new file mode 100644 index 0000000000000..21453cb9cd8c9 --- /dev/null +++ b/docs/mllib-feature-extraction.md @@ -0,0 +1,12 @@ +--- +layout: global +title: Feature Extraction - MLlib +displayTitle: MLlib - Feature Extraction +--- + +* Table of contents +{:toc} + +## Word2Vec + +## TFIDF diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 95ee6bc96801f..23d5a0c4607af 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -3,18 +3,19 @@ layout: global title: Machine Learning Library (MLlib) --- -MLlib is a Spark implementation of some common machine learning algorithms and utilities, +MLlib is Spark's scalable machine learning library consisting of common learning algorithms and utilities, including classification, regression, clustering, collaborative -filtering, dimensionality reduction, as well as underlying optimization primitives: +filtering, dimensionality reduction, as well as underlying optimization primitives, as outlined below: -* [Basics](mllib-basics.html) - * data types +* [Data types](mllib-basics.html) +* [Basic statistics](mllib-stats.html) + * data generators + * stratified sampling * summary statistics -* Classification and regression - * [linear support vector machine (SVM)](mllib-linear-methods.html#linear-support-vector-machine-svm) - * [logistic regression](mllib-linear-methods.html#logistic-regression) - * [linear least squares, Lasso, and ridge regression](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) - * [decision tree](mllib-decision-tree.html) + * hypothesis testing +* [Classification and regression](mllib-classification-regression.html) + * [linear models (SVMs, logistic regression, linear regression)](mllib-linear-methods.html) + * [decision trees](mllib-decision-tree.html) * [naive Bayes](mllib-naive-bayes.html) * [Collaborative filtering](mllib-collaborative-filtering.html) * alternating least squares (ALS) @@ -23,17 +24,18 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * [Dimensionality reduction](mllib-dimensionality-reduction.html) * singular value decomposition (SVD) * principal component analysis (PCA) -* [Optimization](mllib-optimization.html) +* [Feature extraction and transformation](mllib-feature-extraction.html) +* [Optimization (developer)](mllib-optimization.html) * stochastic gradient descent * limited-memory BFGS (L-BFGS) -MLlib is a new component under active development. +MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, -and we will provide migration guide between releases. +and the migration guide below will explain all changes between releases. # Dependencies -MLlib uses linear algebra packages [Breeze](http://www.scalanlp.org/), which depends on +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on [netlib-java](https://github.com/fommil/netlib-java), and [jblas](https://github.com/mikiobraun/jblas). `netlib-java` and `jblas` depend on native Fortran routines. @@ -56,7 +58,7 @@ To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few breaking changes. If your data is sparse, please store it in a sparse format instead of dense to -take advantage of sparsity in both storage and computation. +take advantage of sparsity in both storage and computation. Details are described below.
    diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 254201147edc1..e504cd7f0f578 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -33,24 +33,24 @@ the task of finding a minimizer of a convex function `$f$` that depends on a var Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where the objective function is of the form `\begin{equation} - f(\wv) := - \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) + - \lambda\, R(\wv_i) + f(\wv) := \lambda\, R(\wv) + + \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) \label{eq:regPrimal} \ . \end{equation}` Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and `$y_i\in\R$` are their corresponding labels, which we want to predict. We call the method *linear* if $L(\wv; \x, y)$ can be expressed as a function of $\wv^T x$ and $y$. -Several MLlib's classification and regression algorithms fall into this category, +Several of MLlib's classification and regression algorithms fall into this category, and are discussed here. The objective function `$f$` has two parts: -the loss that measures the error of the model on the training data, -and the regularizer that measures the complexity of the model. -The loss function `$L(\wv;.)$` must be a convex function in `$\wv$`. -The fixed regularization parameter `$\lambda \ge 0$` (`regParam` in the code) defines the trade-off -between the two goals of small loss and small model complexity. +the regularizer that controls the complexity of the model, +and the loss that measures the error of the model on the training data. +The loss function `$L(\wv;.)$` is typically a convex function in `$\wv$`. The +fixed regularization parameter `$\lambda \ge 0$` (`regParam` in the code) +defines the trade-off between the two goals of minimizing the loss (i.e., +training error) and minimizing model complexity (i.e., to avoid overfitting). ### Loss functions @@ -80,10 +80,10 @@ methods MLlib supports: ### Regularizers -The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -We support the following regularizers in MLlib: +The purpose of the +[regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to +encourage simple models and avoid overfitting. We support the following +regularizers in MLlib: @@ -106,27 +106,28 @@ Here `$\mathrm{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of of `$\wv$`. L2-regularized problems are generally easier to solve than L1-regularized due to smoothness. -However, L1 regularization can help promote sparsity in weights, leading to simpler models, which is -also used for feature selection. It is not recommended to train models without any regularization, +However, L1 regularization can help promote sparsity in weights leading to smaller and more interpretable models, the latter of which can be useful for feature selection. +It is not recommended to train models without any regularization, especially when the number of training examples is small. ## Binary classification -[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) is to divide items into -two categories: positive and negative. MLlib supports two linear methods for binary classification: -linear support vector machine (SVM) and logistic regression. The training data set is represented -by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the mathematical -formulation, a training label $y$ is either $+1$ (positive) or $-1$ (negative), which is convenient -for the formulation. *However*, the negative label is represented by $0$ in MLlib instead of $-1$, -to be consistent with multiclass labeling. +[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) +aims to divide items into two categories: positive and negative. MLlib +supports two linear methods for binary classification: linear support vector +machines (SVMs) and logistic regression. For both methods, MLlib supports +L1 and L2 regularized variants. The training data set is represented by an RDD +of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the +mathematical formulation in this guide, a training label $y$ is denoted as +either $+1$ (positive) or $-1$ (negative), which is convenient for the +formulation. *However*, the negative label is represented by $0$ in MLlib +instead of $-1$, to be consistent with multiclass labeling. -### Linear support vector machine (SVM) +### Linear support vector machines (SVMs) The [linear SVM](http://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) -has become a standard choice for large-scale classification tasks. -The name "linear SVM" is actually ambiguous. -By "linear SVM", we mean specifically the linear method with the loss function in formulation -`$\eqref{eq:regPrimal}$` given by the hinge loss +is a standard method for large-scale classification tasks. It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss function in the formulation given by the hinge loss: + `\[ L(\wv;\x,y) := \max \{0, 1-y \wv^T \x \}. \]` @@ -134,39 +135,44 @@ By default, linear SVMs are trained with an L2 regularization. We also support alternative L1 regularization. In this case, the problem becomes a [linear program](http://en.wikipedia.org/wiki/Linear_programming). -Linear SVM algorithm outputs a SVM model, which makes predictions based on the value of $\wv^T \x$. -By the default, if $\wv^T \x \geq 0$, the outcome is positive, or negative otherwise. -However, quite often in practice, the default threshold $0$ is not a good choice. -The threshold should be determined via model evaluation. +The linear SVMs algorithm outputs an SVM model. Given a new data point, +denoted by $\x$, the model makes predictions based on the value of $\wv^T \x$. +By the default, if $\wv^T \x \geq 0$ then the outcome is positive, and negative +otherwise. ### Logistic regression [Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a -binary response. It is a linear method with the loss function in formulation -`$\eqref{eq:regPrimal}$` given by the logistic loss +binary response. +It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss +function in the formulation given by the logistic loss: `\[ L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). \]` -Logistic regression algorithm outputs a logistic regression model, which makes predictions by +The logistic regression algorithm outputs a logistic regression model. Given a +new data point, denoted by $\x$, the model makes predictions by applying the logistic function `\[ \mathrm{f}(z) = \frac{1}{1 + e^{-z}} \]` where $z = \wv^T \x$. -By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or negative otherwise. -For the same reason mentioned above, quite often in practice, this default threshold is not a good choice. -The threshold should be determined via model evaluation. +By default, if $\mathrm{f}(\wv^T x) > 0.5$, the outcome is positive, or +negative otherwise, though unlike linear SVMs, the raw output of the logistic regression +model, $\mathrm{f}(z)$, has a probabilistic interpretation (i.e., the probability +that $\x$ is positive). ### Evaluation metrics -MLlib supports common evaluation metrics for binary classification (not available in Python). This +MLlib supports common evaluation metrics for binary classification (not available in PySpark). +This includes precision, recall, [F-measure](http://en.wikipedia.org/wiki/F1_score), [receiver operating characteristic (ROC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic), precision-recall curve, and [area under the curves (AUC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). -Among the metrics, area under ROC is commonly used to compare models and precision/recall/F-measure -can help determine the threshold to use. +AUC is commonly used to compare the performance of various models while +precision/recall/F-measure can help determine the appropriate threshold to use +for prediction purposes. ### Examples @@ -233,8 +239,7 @@ svmAlg.optimizer. val modelL1 = svmAlg.run(training) {% endhighlight %} -Similarly, you can use replace `SVMWithSGD` by -[`LogisticRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). +[`LogisticRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) can be used in a similar fashion as `SVMWithSGD`. @@ -318,10 +323,11 @@ svmAlg.optimizer() final SVMModel modelL1 = svmAlg.run(training.rdd()); {% endhighlight %} -In order to run the above standalone application using Spark framework make -sure that you follow the instructions provided at section [Standalone -Applications](quick-start.html) of the quick-start guide. What is more, you -should include to your build file *spark-mllib* as a dependency. +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    @@ -354,24 +360,22 @@ print("Training Error = " + str(trainErr)) ## Linear least squares, Lasso, and ridge regression -Linear least squares is a family of linear methods with the loss function in formulation -`$\eqref{eq:regPrimal}$` given by the squared loss +Linear least squares is the most common formulation for regression problems. +It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss +function in the formulation given by the squared loss: `\[ L(\wv;\x,y) := \frac{1}{2} (\wv^T \x - y)^2. \]` -Depending on the regularization type, we call the method -[*ordinary least squares*](http://en.wikipedia.org/wiki/Ordinary_least_squares) or simply -[*linear least squares*](http://en.wikipedia.org/wiki/Linear_least_squares_(mathematics)) if there -is no regularization, [*ridge regression*](http://en.wikipedia.org/wiki/Ridge_regression) if L2 -regularization is used, and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) if L1 -regularization is used. This average loss $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$ is also +Various related regression methods are derived by using different types of regularization: +[*ordinary least squares*](http://en.wikipedia.org/wiki/Ordinary_least_squares) or +[*linear least squares*](http://en.wikipedia.org/wiki/Linear_least_squares_(mathematics)) uses + no regularization; [*ridge regression*](http://en.wikipedia.org/wiki/Ridge_regression) uses L2 +regularization; and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) uses L1 +regularization. For all of these models, the average loss or training error, $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$, is known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). -Note that the squared loss is sensitive to outliers. -Regularization or a robust alternative (e.g., $\ell_1$ regression) is usually necessary in practice. - ### Examples
    @@ -379,7 +383,7 @@ Regularization or a robust alternative (e.g., $\ell_1$ regression) is usually ne
    The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate +values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). {% highlight scala %} @@ -407,9 +411,8 @@ val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.mean() println("training Mean Squared Error = " + MSE) {% endhighlight %} -Similarly you can use [`RidgeRegressionWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD). +and [`LassoWithSGD`](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) can be used in a similar fashion as `LinearRegressionWithSGD`.
    @@ -479,16 +482,17 @@ public class LinearRegression { } {% endhighlight %} -In order to run the above standalone application using Spark framework make -sure that you follow the instructions provided at section [Standalone -Applications](quick-start.html) of the quick-start guide. What is more, you -should include to your build file *spark-mllib* as a dependency. +In order to run the above standalone application, follow the instructions +provided in the [Standalone +Applications](quick-start.html#standalone-applications) section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency.
    The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate +values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). {% highlight python %} diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index b1650c83c98b9..86d94aebd9442 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -4,23 +4,23 @@ title: Naive Bayes - MLlib displayTitle: MLlib - Naive Bayes --- -Naive Bayes is a simple multiclass classification algorithm with the assumption of independence -between every pair of features. Naive Bayes can be trained very efficiently. Within a single pass to -the training data, it computes the conditional probability distribution of each feature given label, -and then it applies Bayes' theorem to compute the conditional probability distribution of label -given an observation and use it for prediction. For more details, please visit the Wikipedia page -[Naive Bayes classifier](http://en.wikipedia.org/wiki/Naive_Bayes_classifier). - -In MLlib, we implemented multinomial naive Bayes, which is typically used for document -classification. Within that context, each observation is a document, each feature represents a term, -whose value is the frequency of the term. For its formulation, please visit the Wikipedia page -[Multinomial Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) -or the section -[Naive Bayes text classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) -from the book Introduction to Information -Retrieval. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by +[Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) is a simple +multiclass classification algorithm with the assumption of independence between +every pair of features. Naive Bayes can be trained very efficiently. Within a +single pass to the training data, it computes the conditional probability +distribution of each feature given label, and then it applies Bayes' theorem to +compute the conditional probability distribution of label given an observation +and use it for prediction. + +MLlib supports [multinomial naive +Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes), +which is typically used for [document +classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). +Within that context, each observation is a document and each +feature represents a term whose value is the frequency of the term. +[Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature -vectors are usually sparse. Please supply sparse vectors as input to take advantage of +vectors are usually sparse, and sparse vectors should be supplied as input to take advantage of sparsity. Since the training data is only used once, it is not necessary to cache it. ## Examples diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md new file mode 100644 index 0000000000000..ca9ef46c15186 --- /dev/null +++ b/docs/mllib-stats.md @@ -0,0 +1,95 @@ +--- +layout: global +title: Statistics Functionality - MLlib +displayTitle: MLlib - Statistics Functionality +--- + +* Table of contents +{:toc} + + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +## Data Generators + +## Stratified Sampling + +## Summary Statistics + +### Multivariate summary statistics + +We provide column summary statistics for `RowMatrix` (note: this functionality is not currently supported in `IndexedRowMatrix` or `CoordinateMatrix`). +If the number of columns is not large, e.g., on the order of thousands, then the +covariance matrix can also be computed as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the +number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, +and is faster if the rows are sparse. + +
    +
    + +[`computeColumnSummaryStatistics()`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of +[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +val mat: RowMatrix = ... // a RowMatrix + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzeros) // number of nonzeros in each column + +// Compute the covariance matrix. +val cov: Matrix = mat.computeCovariance() +{% endhighlight %} +
    + +
    + +[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of +[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; + +RowMatrix mat = ... // a RowMatrix + +// Compute column summary statistics. +MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); +System.out.println(summary.mean()); // a dense vector containing the mean value for each column +System.out.println(summary.variance()); // column-wise variance +System.out.println(summary.numNonzeros()); // number of nonzeros in each column + +// Compute the covariance matrix. +Matrix cov = mat.computeCovariance(); +{% endhighlight %} +
    +
    + + +## Hypothesis Testing From 676f98289dad61c091bb45bd35a2b9613b22d64a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 12 Aug 2014 22:50:29 -0700 Subject: [PATCH 0328/1492] [SPARK-2953] Allow using short names for io compression codecs Instead of requiring "org.apache.spark.io.LZ4CompressionCodec", it is easier for users if Spark just accepts "lz4", "lzf", "snappy". Author: Reynold Xin Closes #1873 from rxin/compressionCodecShortForm and squashes the following commits: 9f50962 [Reynold Xin] Specify short-form compression codec names first. 63f78ee [Reynold Xin] Updated configuration documentation. 47b3848 [Reynold Xin] [SPARK-2953] Allow using short names for io compression codecs --- .../org/apache/spark/io/CompressionCodec.scala | 11 +++++++++-- .../spark/io/CompressionCodecSuite.scala | 18 ++++++++++++++++++ docs/configuration.md | 8 +++++--- 3 files changed, 32 insertions(+), 5 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 1b66218d86dd9..ef9c43ecf14f6 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -46,17 +46,24 @@ trait CompressionCodec { private[spark] object CompressionCodec { + + private val shortCompressionCodecNames = Map( + "lz4" -> classOf[LZ4CompressionCodec].getName, + "lzf" -> classOf[LZFCompressionCodec].getName, + "snappy" -> classOf[SnappyCompressionCodec].getName) + def createCodec(conf: SparkConf): CompressionCodec = { createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { - val ctor = Class.forName(codecName, true, Utils.getContextOrSparkClassLoader) + val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase, codecName) + val ctor = Class.forName(codecClass, true, Utils.getContextOrSparkClassLoader) .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } - val DEFAULT_COMPRESSION_CODEC = classOf[SnappyCompressionCodec].getName + val DEFAULT_COMPRESSION_CODEC = "snappy" } 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 3f882a724b047..25be7f25c21bb 100644 --- a/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/CompressionCodecSuite.scala @@ -56,15 +56,33 @@ class CompressionCodecSuite extends FunSuite { testCodec(codec) } + test("lz4 compression codec short form") { + val codec = CompressionCodec.createCodec(conf, "lz4") + assert(codec.getClass === classOf[LZ4CompressionCodec]) + testCodec(codec) + } + test("lzf compression codec") { val codec = CompressionCodec.createCodec(conf, classOf[LZFCompressionCodec].getName) assert(codec.getClass === classOf[LZFCompressionCodec]) testCodec(codec) } + test("lzf compression codec short form") { + val codec = CompressionCodec.createCodec(conf, "lzf") + assert(codec.getClass === classOf[LZFCompressionCodec]) + testCodec(codec) + } + test("snappy compression codec") { val codec = CompressionCodec.createCodec(conf, classOf[SnappyCompressionCodec].getName) assert(codec.getClass === classOf[SnappyCompressionCodec]) testCodec(codec) } + + test("snappy compression codec short form") { + val codec = CompressionCodec.createCodec(conf, "snappy") + assert(codec.getClass === classOf[SnappyCompressionCodec]) + testCodec(codec) + } } diff --git a/docs/configuration.md b/docs/configuration.md index 617a72a021f6e..8136bd62ab6af 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -373,10 +373,12 @@ Apart from these, the following properties are also available, and may be useful
    - + From 246cb3f158686348a698d1c0da3001c314727129 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Tue, 12 Aug 2014 23:19:35 -0700 Subject: [PATCH 0329/1492] Use transferTo when copy merge files in ExternalSorter Since this is a file to file copy, using transferTo should be faster. Author: Raymond Liu Closes #1884 from colorant/externalSorter and squashes the following commits: 6e42f3c [Raymond Liu] More code into copyStream bfb496b [Raymond Liu] Use transferTo when copy merge files in ExternalSorter --- .../scala/org/apache/spark/util/Utils.scala | 29 ++++++++++++++----- .../util/collection/ExternalSorter.scala | 7 ++--- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c60be4f8a11d2..8cac5da644fa9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -284,17 +284,32 @@ private[spark] object Utils extends Logging { /** Copy all data from an InputStream to an OutputStream */ def copyStream(in: InputStream, out: OutputStream, - closeStreams: Boolean = false) + closeStreams: Boolean = false): Long = { + var count = 0L try { - val buf = new Array[Byte](8192) - var n = 0 - while (n != -1) { - n = in.read(buf) - if (n != -1) { - out.write(buf, 0, n) + if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream]) { + // When both streams are File stream, use transferTo to improve copy performance. + val inChannel = in.asInstanceOf[FileInputStream].getChannel() + val outChannel = out.asInstanceOf[FileOutputStream].getChannel() + val size = inChannel.size() + + // In case transferTo method transferred less data than we have required. + while (count < size) { + count += inChannel.transferTo(count, size - count, outChannel) + } + } else { + val buf = new Array[Byte](8192) + var n = 0 + while (n != -1) { + n = in.read(buf) + if (n != -1) { + out.write(buf, 0, n) + count += n + } } } + count } finally { if (closeStreams) { try { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index b73d5e0cf1714..5d8a648d9551e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -745,12 +745,11 @@ private[spark] class ExternalSorter[K, V, C]( try { out = new FileOutputStream(outputFile) for (i <- 0 until numPartitions) { - val file = partitionWriters(i).fileSegment().file - in = new FileInputStream(file) - org.apache.spark.util.Utils.copyStream(in, out) + in = new FileInputStream(partitionWriters(i).fileSegment().file) + val size = org.apache.spark.util.Utils.copyStream(in, out, false) in.close() in = null - lengths(i) = file.length() + lengths(i) = size offsets(i + 1) = offsets(i) + lengths(i) } } finally { From 2bd812639c3d8c62a725fb7577365ef0816f2898 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Tue, 12 Aug 2014 23:43:36 -0700 Subject: [PATCH 0330/1492] [SPARK-1777 (partial)] bugfix: make size of requested memory correctly Author: Zhang, Liye Closes #1892 from liyezhang556520/lazy_memory_request and squashes the following commits: 335ab61 [Zhang, Liye] [SPARK-1777 (partial)] bugfix: make size of requested memory correctly --- .../src/main/scala/org/apache/spark/storage/MemoryStore.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 28f675c2bbb1e..0a09c24d61879 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -238,7 +238,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // If our vector's size has exceeded the threshold, request more memory val currentSize = vector.estimateSize() if (currentSize >= memoryThreshold) { - val amountToRequest = (currentSize * (memoryGrowthFactor - 1)).toLong + val amountToRequest = (currentSize * memoryGrowthFactor - memoryThreshold).toLong // Hold the accounting lock, in case another thread concurrently puts a block that // takes up the unrolling space we just ensured here accountingLock.synchronized { @@ -254,7 +254,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } // New threshold is currentSize * memoryGrowthFactor - memoryThreshold = currentSize + amountToRequest + memoryThreshold += amountToRequest } } elementsUnrolled += 1 From fe4735958e62b1b32a01960503876000f3d2e520 Mon Sep 17 00:00:00 2001 From: Doris Xin Date: Tue, 12 Aug 2014 23:47:42 -0700 Subject: [PATCH 0331/1492] [SPARK-2993] [MLLib] colStats (wrapper around MultivariateStatisticalSummary) in Statistics For both Scala and Python. The ser/de util functions were moved out of `PythonMLLibAPI` and into their own object to avoid creating the `PythonMLLibAPI` object inside of `MultivariateStatisticalSummarySerialized`, which is then referenced inside of a method in `PythonMLLibAPI`. `MultivariateStatisticalSummarySerialized` was created to serialize the `Vector` fields in `MultivariateStatisticalSummary`. Author: Doris Xin Closes #1911 from dorx/colStats and squashes the following commits: 77b9924 [Doris Xin] developerAPI tag de9cbbe [Doris Xin] reviewer comments and moved more ser/de 459faba [Doris Xin] colStats in Statistics for both Scala and Python --- .../mllib/api/python/PythonMLLibAPI.scala | 532 ++++++++++-------- .../MatrixFactorizationModel.scala | 7 +- .../apache/spark/mllib/stat/Statistics.scala | 13 + .../api/python/PythonMLLibAPISuite.scala | 17 +- python/pyspark/mllib/stat.py | 66 ++- 5 files changed, 374 insertions(+), 261 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index ba7ccd8ce4b8b..18dc087856785 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -34,7 +34,7 @@ import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.DecisionTree import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model.DecisionTreeModel -import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD @@ -48,182 +48,7 @@ import org.apache.spark.util.Utils */ @DeveloperApi class PythonMLLibAPI extends Serializable { - private val DENSE_VECTOR_MAGIC: Byte = 1 - private val SPARSE_VECTOR_MAGIC: Byte = 2 - private val DENSE_MATRIX_MAGIC: Byte = 3 - private val LABELED_POINT_MAGIC: Byte = 4 - - private[python] def deserializeDoubleVector(bytes: Array[Byte], offset: Int = 0): Vector = { - require(bytes.length - offset >= 5, "Byte array too short") - val magic = bytes(offset) - if (magic == DENSE_VECTOR_MAGIC) { - deserializeDenseVector(bytes, offset) - } else if (magic == SPARSE_VECTOR_MAGIC) { - deserializeSparseVector(bytes, offset) - } else { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") - } - } - - private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { - require(bytes.length - offset == 8, "Wrong size byte array for Double") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - bb.getDouble - } - private def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 5, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == DENSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val length = bb.getInt() - require (packetLength == 5 + 8 * length, "Invalid packet length: " + packetLength) - val db = bb.asDoubleBuffer() - val ans = new Array[Double](length.toInt) - db.get(ans) - Vectors.dense(ans) - } - - private def deserializeSparseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 9, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == SPARSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val size = bb.getInt() - val nonZeros = bb.getInt() - require (packetLength == 9 + 12 * nonZeros, "Invalid packet length: " + packetLength) - val ib = bb.asIntBuffer() - val indices = new Array[Int](nonZeros) - ib.get(indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - val values = new Array[Double](nonZeros) - db.get(values) - Vectors.sparse(size, indices, values) - } - - /** - * Returns an 8-byte array for the input Double. - * - * Note: we currently do not use a magic byte for double for storage efficiency. - * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. - * The corresponding deserializer, deserializeDouble, needs to be modified as well if the - * serialization scheme changes. - */ - private[python] def serializeDouble(double: Double): Array[Byte] = { - val bytes = new Array[Byte](8) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putDouble(double) - bytes - } - - private def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { - val len = doubles.length - val bytes = new Array[Byte](5 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_VECTOR_MAGIC) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(doubles) - bytes - } - - private def serializeSparseVector(vector: SparseVector): Array[Byte] = { - val nonZeros = vector.indices.length - val bytes = new Array[Byte](9 + 12 * nonZeros) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(SPARSE_VECTOR_MAGIC) - bb.putInt(vector.size) - bb.putInt(nonZeros) - val ib = bb.asIntBuffer() - ib.put(vector.indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - db.put(vector.values) - bytes - } - - private[python] def serializeDoubleVector(vector: Vector): Array[Byte] = vector match { - case s: SparseVector => - serializeSparseVector(s) - case _ => - serializeDenseVector(vector.toArray) - } - - private def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = { - val packetLength = bytes.length - if (packetLength < 9) { - throw new IllegalArgumentException("Byte array too short.") - } - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - if (magic != DENSE_MATRIX_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") - } - val rows = bb.getInt() - val cols = bb.getInt() - if (packetLength != 9 + 8 * rows * cols) { - throw new IllegalArgumentException("Size " + rows + "x" + cols + " is wrong.") - } - val db = bb.asDoubleBuffer() - val ans = new Array[Array[Double]](rows.toInt) - for (i <- 0 until rows.toInt) { - ans(i) = new Array[Double](cols.toInt) - db.get(ans(i)) - } - ans - } - - private def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = { - val rows = doubles.length - var cols = 0 - if (rows > 0) { - cols = doubles(0).length - } - val bytes = new Array[Byte](9 + 8 * rows * cols) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_MATRIX_MAGIC) - bb.putInt(rows) - bb.putInt(cols) - val db = bb.asDoubleBuffer() - for (i <- 0 until rows) { - db.put(doubles(i)) - } - bytes - } - - private[python] def serializeLabeledPoint(p: LabeledPoint): Array[Byte] = { - val fb = serializeDoubleVector(p.features) - val bytes = new Array[Byte](1 + 8 + fb.length) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(LABELED_POINT_MAGIC) - bb.putDouble(p.label) - bb.put(fb) - bytes - } - - private[python] def deserializeLabeledPoint(bytes: Array[Byte]): LabeledPoint = { - require(bytes.length >= 9, "Byte array too short") - val magic = bytes(0) - if (magic != LABELED_POINT_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") - } - val labelBytes = ByteBuffer.wrap(bytes, 1, 8) - labelBytes.order(ByteOrder.nativeOrder()) - val label = labelBytes.asDoubleBuffer().get(0) - LabeledPoint(label, deserializeDoubleVector(bytes, 9)) - } /** * Loads and serializes labeled points saved with `RDD#saveAsTextFile`. @@ -236,17 +61,17 @@ class PythonMLLibAPI extends Serializable { jsc: JavaSparkContext, path: String, minPartitions: Int): JavaRDD[Array[Byte]] = - MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(serializeLabeledPoint) + MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(SerDe.serializeLabeledPoint) private def trainRegressionModel( trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, dataBytesJRDD: JavaRDD[Array[Byte]], initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint) - val initialWeights = deserializeDoubleVector(initialWeightsBA) + val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) + val initialWeights = SerDe.deserializeDoubleVector(initialWeightsBA) val model = trainFunc(data, initialWeights) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleVector(model.weights)) + ret.add(SerDe.serializeDoubleVector(model.weights)) ret.add(model.intercept: java.lang.Double) ret } @@ -405,12 +230,12 @@ class PythonMLLibAPI extends Serializable { def trainNaiveBayes( dataBytesJRDD: JavaRDD[Array[Byte]], lambda: Double): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint) + val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) val model = NaiveBayes.train(data, lambda) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleVector(Vectors.dense(model.labels))) - ret.add(serializeDoubleVector(Vectors.dense(model.pi))) - ret.add(serializeDoubleMatrix(model.theta)) + ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.labels))) + ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.pi))) + ret.add(SerDe.serializeDoubleMatrix(model.theta)) ret } @@ -423,52 +248,13 @@ class PythonMLLibAPI extends Serializable { maxIterations: Int, runs: Int, initializationMode: String): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(bytes => deserializeDoubleVector(bytes)) + val data = dataBytesJRDD.rdd.map(bytes => SerDe.deserializeDoubleVector(bytes)) val model = KMeans.train(data, k, maxIterations, runs, initializationMode) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) + ret.add(SerDe.serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) ret } - /** Unpack a Rating object from an array of bytes */ - private def unpackRating(ratingBytes: Array[Byte]): Rating = { - val bb = ByteBuffer.wrap(ratingBytes) - bb.order(ByteOrder.nativeOrder()) - val user = bb.getInt() - val product = bb.getInt() - val rating = bb.getDouble() - new Rating(user, product, rating) - } - - /** Unpack a tuple of Ints from an array of bytes */ - private[spark] def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { - val bb = ByteBuffer.wrap(tupleBytes) - bb.order(ByteOrder.nativeOrder()) - val v1 = bb.getInt() - val v2 = bb.getInt() - (v1, v2) - } - - /** - * Serialize a Rating object into an array of bytes. - * It can be deserialized using RatingDeserializer(). - * - * @param rate the Rating object to serialize - * @return - */ - private[spark] def serializeRating(rate: Rating): Array[Byte] = { - val len = 3 - val bytes = new Array[Byte](4 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(rate.user.toDouble) - db.put(rate.product.toDouble) - db.put(rate.rating) - bytes - } - /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -481,7 +267,7 @@ class PythonMLLibAPI extends Serializable { iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(unpackRating) + val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) ALS.train(ratings, rank, iterations, lambda, blocks) } @@ -498,7 +284,7 @@ class PythonMLLibAPI extends Serializable { lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(unpackRating) + val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) } @@ -519,7 +305,7 @@ class PythonMLLibAPI extends Serializable { maxDepth: Int, maxBins: Int): DecisionTreeModel = { - val data = dataBytesJRDD.rdd.map(deserializeLabeledPoint) + val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) val algo = Algo.fromString(algoStr) val impurity = Impurities.fromString(impurityStr) @@ -545,7 +331,7 @@ class PythonMLLibAPI extends Serializable { def predictDecisionTreeModel( model: DecisionTreeModel, featuresBytes: Array[Byte]): Double = { - val features: Vector = deserializeDoubleVector(featuresBytes) + val features: Vector = SerDe.deserializeDoubleVector(featuresBytes) model.predict(features) } @@ -559,8 +345,17 @@ class PythonMLLibAPI extends Serializable { def predictDecisionTreeModel( model: DecisionTreeModel, dataJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val data = dataJRDD.rdd.map(xBytes => deserializeDoubleVector(xBytes)) - model.predict(data).map(serializeDouble) + val data = dataJRDD.rdd.map(xBytes => SerDe.deserializeDoubleVector(xBytes)) + model.predict(data).map(SerDe.serializeDouble) + } + + /** + * Java stub for mllib Statistics.colStats(X: RDD[Vector]). + * TODO figure out return type. + */ + def colStats(X: JavaRDD[Array[Byte]]): MultivariateStatisticalSummarySerialized = { + val cStats = Statistics.colStats(X.rdd.map(SerDe.deserializeDoubleVector(_))) + new MultivariateStatisticalSummarySerialized(cStats) } /** @@ -569,17 +364,17 @@ class PythonMLLibAPI extends Serializable { * pyspark. */ def corr(X: JavaRDD[Array[Byte]], method: String): Array[Byte] = { - val inputMatrix = X.rdd.map(deserializeDoubleVector(_)) + val inputMatrix = X.rdd.map(SerDe.deserializeDoubleVector(_)) val result = Statistics.corr(inputMatrix, getCorrNameOrDefault(method)) - serializeDoubleMatrix(to2dArray(result)) + SerDe.serializeDoubleMatrix(SerDe.to2dArray(result)) } /** * Java stub for mllib Statistics.corr(x: RDD[Double], y: RDD[Double], method: String). */ def corr(x: JavaRDD[Array[Byte]], y: JavaRDD[Array[Byte]], method: String): Double = { - val xDeser = x.rdd.map(deserializeDouble(_)) - val yDeser = y.rdd.map(deserializeDouble(_)) + val xDeser = x.rdd.map(SerDe.deserializeDouble(_)) + val yDeser = y.rdd.map(SerDe.deserializeDouble(_)) Statistics.corr(xDeser, yDeser, getCorrNameOrDefault(method)) } @@ -588,12 +383,6 @@ class PythonMLLibAPI extends Serializable { if (method == null) CorrelationNames.defaultCorrName else method } - // Reformat a Matrix into Array[Array[Double]] for serialization - private[python] def to2dArray(matrix: Matrix): Array[Array[Double]] = { - val values = matrix.toArray - Array.tabulate(matrix.numRows, matrix.numCols)((i, j) => values(i + j * matrix.numRows)) - } - // Used by the *RDD methods to get default seed if not passed in from pyspark private def getSeedOrDefault(seed: java.lang.Long): Long = { if (seed == null) Utils.random.nextLong else seed @@ -621,7 +410,7 @@ class PythonMLLibAPI extends Serializable { seed: java.lang.Long): JavaRDD[Array[Byte]] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformRDD(jsc.sc, size, parts, s).map(serializeDouble) + RG.uniformRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) } /** @@ -633,7 +422,7 @@ class PythonMLLibAPI extends Serializable { seed: java.lang.Long): JavaRDD[Array[Byte]] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalRDD(jsc.sc, size, parts, s).map(serializeDouble) + RG.normalRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) } /** @@ -646,7 +435,7 @@ class PythonMLLibAPI extends Serializable { seed: java.lang.Long): JavaRDD[Array[Byte]] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonRDD(jsc.sc, mean, size, parts, s).map(serializeDouble) + RG.poissonRDD(jsc.sc, mean, size, parts, s).map(SerDe.serializeDouble) } /** @@ -659,7 +448,7 @@ class PythonMLLibAPI extends Serializable { seed: java.lang.Long): JavaRDD[Array[Byte]] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s).map(serializeDoubleVector) + RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) } /** @@ -672,7 +461,7 @@ class PythonMLLibAPI extends Serializable { seed: java.lang.Long): JavaRDD[Array[Byte]] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s).map(serializeDoubleVector) + RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) } /** @@ -686,7 +475,256 @@ class PythonMLLibAPI extends Serializable { seed: java.lang.Long): JavaRDD[Array[Byte]] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(serializeDoubleVector) + RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + } + +} + +/** + * :: DeveloperApi :: + * MultivariateStatisticalSummary with Vector fields serialized. + */ +@DeveloperApi +class MultivariateStatisticalSummarySerialized(val summary: MultivariateStatisticalSummary) + extends Serializable { + + def mean: Array[Byte] = SerDe.serializeDoubleVector(summary.mean) + + def variance: Array[Byte] = SerDe.serializeDoubleVector(summary.variance) + + def count: Long = summary.count + + def numNonzeros: Array[Byte] = SerDe.serializeDoubleVector(summary.numNonzeros) + + def max: Array[Byte] = SerDe.serializeDoubleVector(summary.max) + + def min: Array[Byte] = SerDe.serializeDoubleVector(summary.min) +} + +/** + * SerDe utility functions for PythonMLLibAPI. + */ +private[spark] object SerDe extends Serializable { + private val DENSE_VECTOR_MAGIC: Byte = 1 + private val SPARSE_VECTOR_MAGIC: Byte = 2 + private val DENSE_MATRIX_MAGIC: Byte = 3 + private val LABELED_POINT_MAGIC: Byte = 4 + + private[python] def deserializeDoubleVector(bytes: Array[Byte], offset: Int = 0): Vector = { + require(bytes.length - offset >= 5, "Byte array too short") + val magic = bytes(offset) + if (magic == DENSE_VECTOR_MAGIC) { + deserializeDenseVector(bytes, offset) + } else if (magic == SPARSE_VECTOR_MAGIC) { + deserializeSparseVector(bytes, offset) + } else { + throw new IllegalArgumentException("Magic " + magic + " is wrong.") + } } + private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { + require(bytes.length - offset == 8, "Wrong size byte array for Double") + val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) + bb.order(ByteOrder.nativeOrder()) + bb.getDouble + } + + private[python] def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { + val packetLength = bytes.length - offset + require(packetLength >= 5, "Byte array too short") + val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) + bb.order(ByteOrder.nativeOrder()) + val magic = bb.get() + require(magic == DENSE_VECTOR_MAGIC, "Invalid magic: " + magic) + val length = bb.getInt() + require (packetLength == 5 + 8 * length, "Invalid packet length: " + packetLength) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](length.toInt) + db.get(ans) + Vectors.dense(ans) + } + + private[python] def deserializeSparseVector(bytes: Array[Byte], offset: Int = 0): Vector = { + val packetLength = bytes.length - offset + require(packetLength >= 9, "Byte array too short") + val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) + bb.order(ByteOrder.nativeOrder()) + val magic = bb.get() + require(magic == SPARSE_VECTOR_MAGIC, "Invalid magic: " + magic) + val size = bb.getInt() + val nonZeros = bb.getInt() + require (packetLength == 9 + 12 * nonZeros, "Invalid packet length: " + packetLength) + val ib = bb.asIntBuffer() + val indices = new Array[Int](nonZeros) + ib.get(indices) + bb.position(bb.position() + 4 * nonZeros) + val db = bb.asDoubleBuffer() + val values = new Array[Double](nonZeros) + db.get(values) + Vectors.sparse(size, indices, values) + } + + /** + * Returns an 8-byte array for the input Double. + * + * Note: we currently do not use a magic byte for double for storage efficiency. + * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. + * The corresponding deserializer, deserializeDouble, needs to be modified as well if the + * serialization scheme changes. + */ + private[python] def serializeDouble(double: Double): Array[Byte] = { + val bytes = new Array[Byte](8) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.putDouble(double) + bytes + } + + private[python] def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { + val len = doubles.length + val bytes = new Array[Byte](5 + 8 * len) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.put(DENSE_VECTOR_MAGIC) + bb.putInt(len) + val db = bb.asDoubleBuffer() + db.put(doubles) + bytes + } + + private[python] def serializeSparseVector(vector: SparseVector): Array[Byte] = { + val nonZeros = vector.indices.length + val bytes = new Array[Byte](9 + 12 * nonZeros) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.put(SPARSE_VECTOR_MAGIC) + bb.putInt(vector.size) + bb.putInt(nonZeros) + val ib = bb.asIntBuffer() + ib.put(vector.indices) + bb.position(bb.position() + 4 * nonZeros) + val db = bb.asDoubleBuffer() + db.put(vector.values) + bytes + } + + private[python] def serializeDoubleVector(vector: Vector): Array[Byte] = vector match { + case s: SparseVector => + serializeSparseVector(s) + case _ => + serializeDenseVector(vector.toArray) + } + + private[python] def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = { + val packetLength = bytes.length + if (packetLength < 9) { + throw new IllegalArgumentException("Byte array too short.") + } + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val magic = bb.get() + if (magic != DENSE_MATRIX_MAGIC) { + throw new IllegalArgumentException("Magic " + magic + " is wrong.") + } + val rows = bb.getInt() + val cols = bb.getInt() + if (packetLength != 9 + 8 * rows * cols) { + throw new IllegalArgumentException("Size " + rows + "x" + cols + " is wrong.") + } + val db = bb.asDoubleBuffer() + val ans = new Array[Array[Double]](rows.toInt) + for (i <- 0 until rows.toInt) { + ans(i) = new Array[Double](cols.toInt) + db.get(ans(i)) + } + ans + } + + private[python] def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = { + val rows = doubles.length + var cols = 0 + if (rows > 0) { + cols = doubles(0).length + } + val bytes = new Array[Byte](9 + 8 * rows * cols) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.put(DENSE_MATRIX_MAGIC) + bb.putInt(rows) + bb.putInt(cols) + val db = bb.asDoubleBuffer() + for (i <- 0 until rows) { + db.put(doubles(i)) + } + bytes + } + + private[python] def serializeLabeledPoint(p: LabeledPoint): Array[Byte] = { + val fb = serializeDoubleVector(p.features) + val bytes = new Array[Byte](1 + 8 + fb.length) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.put(LABELED_POINT_MAGIC) + bb.putDouble(p.label) + bb.put(fb) + bytes + } + + private[python] def deserializeLabeledPoint(bytes: Array[Byte]): LabeledPoint = { + require(bytes.length >= 9, "Byte array too short") + val magic = bytes(0) + if (magic != LABELED_POINT_MAGIC) { + throw new IllegalArgumentException("Magic " + magic + " is wrong.") + } + val labelBytes = ByteBuffer.wrap(bytes, 1, 8) + labelBytes.order(ByteOrder.nativeOrder()) + val label = labelBytes.asDoubleBuffer().get(0) + LabeledPoint(label, deserializeDoubleVector(bytes, 9)) + } + + // Reformat a Matrix into Array[Array[Double]] for serialization + private[python] def to2dArray(matrix: Matrix): Array[Array[Double]] = { + val values = matrix.toArray + Array.tabulate(matrix.numRows, matrix.numCols)((i, j) => values(i + j * matrix.numRows)) + } + + + /** Unpack a Rating object from an array of bytes */ + private[python] def unpackRating(ratingBytes: Array[Byte]): Rating = { + val bb = ByteBuffer.wrap(ratingBytes) + bb.order(ByteOrder.nativeOrder()) + val user = bb.getInt() + val product = bb.getInt() + val rating = bb.getDouble() + new Rating(user, product, rating) + } + + /** Unpack a tuple of Ints from an array of bytes */ + def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { + val bb = ByteBuffer.wrap(tupleBytes) + bb.order(ByteOrder.nativeOrder()) + val v1 = bb.getInt() + val v2 = bb.getInt() + (v1, v2) + } + + /** + * Serialize a Rating object into an array of bytes. + * It can be deserialized using RatingDeserializer(). + * + * @param rate the Rating object to serialize + * @return + */ + def serializeRating(rate: Rating): Array[Byte] = { + val len = 3 + val bytes = new Array[Byte](4 + 8 * len) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + bb.putInt(len) + val db = bb.asDoubleBuffer() + db.put(rate.user.toDouble) + db.put(rate.product.toDouble) + db.put(rate.rating) + bytes + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index a1a76fcbe9f9c..478c6485052b6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -23,7 +23,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.api.python.PythonMLLibAPI +import org.apache.spark.mllib.api.python.SerDe /** * Model representing the result of matrix factorization. @@ -117,9 +117,8 @@ class MatrixFactorizationModel private[mllib] ( */ @DeveloperApi def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val pythonAPI = new PythonMLLibAPI() - val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes)) - predict(usersProducts).map(rate => pythonAPI.serializeRating(rate)) + val usersProducts = usersProductsJRDD.rdd.map(xBytes => SerDe.unpackTuple(xBytes)) + predict(usersProducts).map(rate => SerDe.serializeRating(rate)) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index cf8679610e191..3cf1028fbc725 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.stat import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.{Matrix, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.correlation.Correlations @@ -30,6 +31,18 @@ import org.apache.spark.rdd.RDD @Experimental object Statistics { + /** + * :: Experimental :: + * Computes column-wise summary statistics for the input RDD[Vector]. + * + * @param X an RDD[Vector] for which column-wise summary statistics are to be computed. + * @return [[MultivariateStatisticalSummary]] object containing column-wise summary statistics. + */ + @Experimental + def colStats(X: RDD[Vector]): MultivariateStatisticalSummary = { + new RowMatrix(X).computeColumnSummaryStatistics() + } + /** * :: Experimental :: * Compute the Pearson correlation matrix for the input RDD of Vectors. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index bd413a80f5107..092d67bbc5238 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -23,7 +23,6 @@ import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint class PythonMLLibAPISuite extends FunSuite { - val py = new PythonMLLibAPI test("vector serialization") { val vectors = Seq( @@ -34,8 +33,8 @@ class PythonMLLibAPISuite extends FunSuite { Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), Vectors.sparse(2, Array(1), Array(-2.0))) vectors.foreach { v => - val bytes = py.serializeDoubleVector(v) - val u = py.deserializeDoubleVector(bytes) + val bytes = SerDe.serializeDoubleVector(v) + val u = SerDe.deserializeDoubleVector(bytes) assert(u.getClass === v.getClass) assert(u === v) } @@ -50,8 +49,8 @@ class PythonMLLibAPISuite extends FunSuite { LabeledPoint(1.0, Vectors.sparse(1, Array.empty[Int], Array.empty[Double])), LabeledPoint(-0.5, Vectors.sparse(2, Array(1), Array(-2.0)))) points.foreach { p => - val bytes = py.serializeLabeledPoint(p) - val q = py.deserializeLabeledPoint(bytes) + val bytes = SerDe.serializeLabeledPoint(p) + val q = SerDe.deserializeLabeledPoint(bytes) assert(q.label === p.label) assert(q.features.getClass === p.features.getClass) assert(q.features === p.features) @@ -60,8 +59,8 @@ class PythonMLLibAPISuite extends FunSuite { test("double serialization") { for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { - val bytes = py.serializeDouble(x) - val deser = py.deserializeDouble(bytes) + val bytes = SerDe.serializeDouble(x) + val deser = SerDe.deserializeDouble(bytes) // We use `equals` here for comparison because we cannot use `==` for NaN assert(x.equals(deser)) } @@ -70,14 +69,14 @@ class PythonMLLibAPISuite extends FunSuite { test("matrix to 2D array") { val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) val matrix = Matrices.dense(2, 3, values) - val arr = py.to2dArray(matrix) + val arr = SerDe.to2dArray(matrix) val expected = Array(Array[Double](0, 3, 7), Array[Double](1.2, 4.56, 8)) assert(arr === expected) // Test conversion for empty matrix val empty = Array[Double]() val emptyMatrix = Matrices.dense(0, 0, empty) - val empty2D = py.to2dArray(emptyMatrix) + val empty2D = SerDe.to2dArray(emptyMatrix) assert(empty2D === Array[Array[Double]]()) } } diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 982906b9d09f0..a73abc5ff90df 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -22,11 +22,75 @@ from pyspark.mllib._common import \ _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ _serialize_double, _serialize_double_vector, \ - _deserialize_double, _deserialize_double_matrix + _deserialize_double, _deserialize_double_matrix, _deserialize_double_vector + + +class MultivariateStatisticalSummary(object): + + """ + Trait for multivariate statistical summary of a data matrix. + """ + + def __init__(self, sc, java_summary): + """ + :param sc: Spark context + :param java_summary: Handle to Java summary object + """ + self._sc = sc + self._java_summary = java_summary + + def __del__(self): + self._sc._gateway.detach(self._java_summary) + + def mean(self): + return _deserialize_double_vector(self._java_summary.mean()) + + def variance(self): + return _deserialize_double_vector(self._java_summary.variance()) + + def count(self): + return self._java_summary.count() + + def numNonzeros(self): + return _deserialize_double_vector(self._java_summary.numNonzeros()) + + def max(self): + return _deserialize_double_vector(self._java_summary.max()) + + def min(self): + return _deserialize_double_vector(self._java_summary.min()) class Statistics(object): + @staticmethod + def colStats(X): + """ + Computes column-wise summary statistics for the input RDD[Vector]. + + >>> from linalg import Vectors + >>> rdd = sc.parallelize([Vectors.dense([2, 0, 0, -2]), + ... Vectors.dense([4, 5, 0, 3]), + ... Vectors.dense([6, 7, 0, 8])]) + >>> cStats = Statistics.colStats(rdd) + >>> cStats.mean() + array([ 4., 4., 0., 3.]) + >>> cStats.variance() + array([ 4., 13., 0., 25.]) + >>> cStats.count() + 3L + >>> cStats.numNonzeros() + array([ 3., 2., 0., 3.]) + >>> cStats.max() + array([ 6., 7., 0., 8.]) + >>> cStats.min() + array([ 2., 0., 0., -2.]) + """ + sc = X.ctx + Xser = _get_unmangled_double_vector_rdd(X) + cStats = sc._jvm.PythonMLLibAPI().colStats(Xser._jrdd) + return MultivariateStatisticalSummary(sc, cStats) + @staticmethod def corr(x, y=None, method=None): """ From 869f06c759c29b09c8dc72e0e4034c03f908ba30 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 13 Aug 2014 14:42:57 -0700 Subject: [PATCH 0332/1492] [SPARK-2963] [SQL] There no documentation about building to use HiveServer and CLI for SparkSQL Author: Kousuke Saruta Closes #1885 from sarutak/SPARK-2963 and squashes the following commits: ed53329 [Kousuke Saruta] Modified description and notaton of proper noun 07c59fc [Kousuke Saruta] Added a description about how to build to use HiveServer and CLI for SparkSQL to building-with-maven.md 6e6645a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2963 c88fa93 [Kousuke Saruta] Added a description about building to use HiveServer and CLI for SparkSQL --- README.md | 9 +++++++++ docs/building-with-maven.md | 9 +++++++++ 2 files changed, 18 insertions(+) diff --git a/README.md b/README.md index f87e07aa5cc90..a1a48f5bd0819 100644 --- a/README.md +++ b/README.md @@ -115,6 +115,15 @@ If your project is built with Maven, add this to your POM file's ` +## A Note About Thrift JDBC server and CLI for Spark SQL + +Spark SQL supports Thrift JDBC server and CLI. +See sql-programming-guide.md for more information about those features. +You can use those features by setting `-Phive-thriftserver` when building Spark as follows. + + $ sbt/sbt -Phive-thriftserver assembly + + ## Configuration Please refer to the [Configuration guide](http://spark.apache.org/docs/latest/configuration.html) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 672d0ef114f6d..4d87ab92cec5b 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -96,6 +96,15 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package {% endhighlight %} +# Building Thrift JDBC server and CLI for Spark SQL + +Spark SQL supports Thrift JDBC server and CLI. +See sql-programming-guide.md for more information about those features. +You can use those features by setting `-Phive-thriftserver` when building Spark as follows. +{% highlight bash %} +mvn -Phive-thriftserver assembly +{% endhighlight %} + # Spark Tests in Maven Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). From c974a716e17c9fe2628b1ba1d4309ead1bd855ad Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 13 Aug 2014 14:56:11 -0700 Subject: [PATCH 0333/1492] [SPARK-3013] [SQL] [PySpark] convert array into list because Pyrolite does not support array from Python 2.6 Author: Davies Liu Closes #1928 from davies/fix_array and squashes the following commits: 858e6c5 [Davies Liu] convert array into list --- python/pyspark/sql.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 27f1d2ddf942a..46540ca3f1e8a 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -498,10 +498,7 @@ def _infer_schema(row): def _create_converter(obj, dataType): """Create an converter to drop the names of fields in obj """ - if not _has_struct(dataType): - return lambda x: x - - elif isinstance(dataType, ArrayType): + if isinstance(dataType, ArrayType): conv = _create_converter(obj[0], dataType.elementType) return lambda row: map(conv, row) @@ -510,6 +507,9 @@ def _create_converter(obj, dataType): conv = _create_converter(value, dataType.valueType) return lambda row: dict((k, conv(v)) for k, v in row.iteritems()) + elif not isinstance(dataType, StructType): + return lambda x: x + # dataType must be StructType names = [f.name for f in dataType.fields] @@ -529,8 +529,7 @@ def _create_converter(obj, dataType): elif hasattr(obj, "__dict__"): # object conv = lambda o: [o.__dict__.get(n, None) for n in names] - nested = any(_has_struct(f.dataType) for f in dataType.fields) - if not nested: + if all(isinstance(f.dataType, PrimitiveType) for f in dataType.fields): return conv row = conv(obj) @@ -1037,7 +1036,8 @@ def inferSchema(self, rdd): raise ValueError("The first row in RDD is empty, " "can not infer schema") if type(first) is dict: - warnings.warn("Using RDD of dict to inferSchema is deprecated") + warnings.warn("Using RDD of dict to inferSchema is deprecated," + "please use pyspark.Row instead") schema = _infer_schema(first) rdd = rdd.mapPartitions(lambda rows: _drop_schema(rows, schema)) From 434bea1c002b597cff9db899da101490e1f1e9ed Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 13 Aug 2014 14:57:12 -0700 Subject: [PATCH 0334/1492] [SPARK-2983] [PySpark] improve performance of sortByKey() 1. skip partitionBy() when numOfPartition is 1 2. use bisect_left (O(lg(N))) instread of loop (O(N)) in rangePartitioner Author: Davies Liu Closes #1898 from davies/sort and squashes the following commits: 0a9608b [Davies Liu] Merge branch 'master' into sort 1cf9565 [Davies Liu] improve performance of sortByKey() --- python/pyspark/rdd.py | 47 ++++++++++++++++++++++--------------------- 1 file changed, 24 insertions(+), 23 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 756e8f35fb03d..3934bdda0a466 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -30,6 +30,7 @@ from threading import Thread import warnings import heapq +import bisect from random import Random from math import sqrt, log @@ -574,6 +575,8 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): # noqa >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)] + >>> sc.parallelize(tmp).sortByKey(True, 1).collect() + [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)] >>> 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)] @@ -584,42 +587,40 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() - bounds = list() + if numPartitions == 1: + if self.getNumPartitions() > 1: + self = self.coalesce(1) + + def sort(iterator): + return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) + + return self.mapPartitions(sort) # first compute the boundary of each part via sampling: we want to partition # the key-space into bins such that the bins have roughly the same # number of (key, value) pairs falling into them - if numPartitions > 1: - rddSize = self.count() - # 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 = sorted(samples, reverse=(not ascending), key=keyfunc) - - # we have numPartitions many parts but one of the them has - # an implicit boundary - for i in range(0, numPartitions - 1): - index = (len(samples) - 1) * (i + 1) / numPartitions - bounds.append(samples[index]) + rddSize = self.count() + maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner + fraction = min(maxSampleSize / max(rddSize, 1), 1.0) + 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 + # an implicit boundary + bounds = [samples[len(samples) * (i + 1) / numPartitions] + for i in range(0, numPartitions - 1)] def rangePartitionFunc(k): - p = 0 - while p < len(bounds) and keyfunc(k) > bounds[p]: - p += 1 + p = bisect.bisect_left(bounds, keyfunc(k)) if ascending: return p else: return numPartitions - 1 - p def mapFunc(iterator): - yield sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) + return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) - return (self.partitionBy(numPartitions, partitionFunc=rangePartitionFunc) - .mapPartitions(mapFunc, preservesPartitioning=True) - .flatMap(lambda x: x, preservesPartitioning=True)) + return self.partitionBy(numPartitions, rangePartitionFunc).mapPartitions(mapFunc, True) def sortBy(self, keyfunc, ascending=True, numPartitions=None): """ From 7ecb867c4cd6916b6cb12f2ece1a4c88591ad5b5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 13 Aug 2014 16:20:49 -0700 Subject: [PATCH 0335/1492] [MLLIB] use Iterator.fill instead of Array.fill Iterator.fill uses less memory Author: Xiangrui Meng Closes #1930 from mengxr/rand-gen-iter and squashes the following commits: 24178ca [Xiangrui Meng] use Iterator.fill instead of Array.fill --- .../scala/org/apache/spark/mllib/rdd/RandomRDD.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala index c8db3910c6eab..910eff9540a47 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RandomRDD.scala @@ -105,16 +105,16 @@ private[mllib] object RandomRDD { def getPointIterator[T: ClassTag](partition: RandomRDDPartition[T]): Iterator[T] = { val generator = partition.generator.copy() generator.setSeed(partition.seed) - Array.fill(partition.size)(generator.nextValue()).toIterator + Iterator.fill(partition.size)(generator.nextValue()) } // The RNG has to be reset every time the iterator is requested to guarantee same data // every time the content of the RDD is examined. - def getVectorIterator(partition: RandomRDDPartition[Double], - vectorSize: Int): Iterator[Vector] = { + def getVectorIterator( + partition: RandomRDDPartition[Double], + vectorSize: Int): Iterator[Vector] = { val generator = partition.generator.copy() generator.setSeed(partition.seed) - Array.fill(partition.size)(new DenseVector( - (0 until vectorSize).map { _ => generator.nextValue() }.toArray)).toIterator + Iterator.fill(partition.size)(new DenseVector(Array.fill(vectorSize)(generator.nextValue()))) } } From bdc7a1a4749301f8d18617c130c7766684aa8789 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 13 Aug 2014 16:27:50 -0700 Subject: [PATCH 0336/1492] [SPARK-3004][SQL] Added null checking when retrieving row set JIRA issue: [SPARK-3004](https://issues.apache.org/jira/browse/SPARK-3004) HiveThriftServer2 throws exception when the result set contains `NULL`. Should check `isNullAt` in `SparkSQLOperationManager.getNextRowSet`. Note that simply using `row.addColumnValue(null)` doesn't work, since Hive set the column type of a null `ColumnValue` to String by default. Author: Cheng Lian Closes #1920 from liancheng/spark-3004 and squashes the following commits: 1b1db1c [Cheng Lian] Adding NULL column values in the Hive way 2217722 [Cheng Lian] Fixed SPARK-3004: added null checking when retrieving row set --- .../server/SparkSQLOperationManager.scala | 93 +++++++++++++------ .../data/files/small_kv_with_null.txt | 10 ++ .../thriftserver/HiveThriftServer2Suite.scala | 26 +++++- 3 files changed, 96 insertions(+), 33 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/resources/data/files/small_kv_with_null.txt diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index f192f490ac3d0..9338e8121b0fe 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -73,35 +73,10 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage var curCol = 0 while (curCol < sparkRow.length) { - dataTypes(curCol) match { - case StringType => - row.addString(sparkRow(curCol).asInstanceOf[String]) - case IntegerType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getInt(curCol))) - case BooleanType => - row.addColumnValue(ColumnValue.booleanValue(sparkRow.getBoolean(curCol))) - case DoubleType => - row.addColumnValue(ColumnValue.doubleValue(sparkRow.getDouble(curCol))) - case FloatType => - row.addColumnValue(ColumnValue.floatValue(sparkRow.getFloat(curCol))) - case DecimalType => - val hiveDecimal = sparkRow.get(curCol).asInstanceOf[BigDecimal].bigDecimal - row.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) - case LongType => - row.addColumnValue(ColumnValue.longValue(sparkRow.getLong(curCol))) - case ByteType => - row.addColumnValue(ColumnValue.byteValue(sparkRow.getByte(curCol))) - case ShortType => - row.addColumnValue(ColumnValue.intValue(sparkRow.getShort(curCol))) - case TimestampType => - row.addColumnValue( - ColumnValue.timestampValue(sparkRow.get(curCol).asInstanceOf[Timestamp])) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = result - .queryExecution - .asInstanceOf[HiveContext#QueryExecution] - .toHiveString((sparkRow.get(curCol), dataTypes(curCol))) - row.addColumnValue(ColumnValue.stringValue(hiveString)) + if (sparkRow.isNullAt(curCol)) { + addNullColumnValue(sparkRow, row, curCol) + } else { + addNonNullColumnValue(sparkRow, row, curCol) } curCol += 1 } @@ -112,6 +87,66 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } } + def addNonNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(from(ordinal).asInstanceOf[String]) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(from.getInt(ordinal))) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(from.getBoolean(ordinal))) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal))) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal))) + case DecimalType => + val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal + to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) + case LongType => + to.addColumnValue(ColumnValue.longValue(from.getLong(ordinal))) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) + case ShortType => + to.addColumnValue(ColumnValue.intValue(from.getShort(ordinal))) + case TimestampType => + to.addColumnValue( + ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((from.get(ordinal), dataTypes(ordinal))) + to.addColumnValue(ColumnValue.stringValue(hiveString)) + } + } + + def addNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(null) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(null)) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(null)) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(null)) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(null)) + case DecimalType => + to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal)) + case LongType => + to.addColumnValue(ColumnValue.longValue(null)) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(null)) + case ShortType => + to.addColumnValue(ColumnValue.intValue(null)) + case TimestampType => + to.addColumnValue(ColumnValue.timestampValue(null)) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + to.addColumnValue(ColumnValue.stringValue(null: String)) + } + } + def getResultSetSchema: TableSchema = { logWarning(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { diff --git a/sql/hive-thriftserver/src/test/resources/data/files/small_kv_with_null.txt b/sql/hive-thriftserver/src/test/resources/data/files/small_kv_with_null.txt new file mode 100644 index 0000000000000..ae08c640e6c13 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/data/files/small_kv_with_null.txt @@ -0,0 +1,10 @@ +238val_238 + +311val_311 +val_27 +val_165 +val_409 +255val_255 +278val_278 +98val_98 +val_484 diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 78bffa2607349..aedef6ce1f5f2 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -113,22 +113,40 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt val stmt = createStatement() stmt.execute("DROP TABLE IF EXISTS test") stmt.execute("DROP TABLE IF EXISTS test_cached") - stmt.execute("CREATE TABLE test(key int, val string)") + stmt.execute("CREATE TABLE test(key INT, val STRING)") stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") - stmt.execute("CREATE TABLE test_cached as select * from test limit 4") + stmt.execute("CREATE TABLE test_cached AS SELECT * FROM test LIMIT 4") stmt.execute("CACHE TABLE test_cached") - var rs = stmt.executeQuery("select count(*) from test") + var rs = stmt.executeQuery("SELECT COUNT(*) FROM test") rs.next() assert(rs.getInt(1) === 5) - rs = stmt.executeQuery("select count(*) from test_cached") + rs = stmt.executeQuery("SELECT COUNT(*) FROM test_cached") rs.next() assert(rs.getInt(1) === 4) stmt.close() } + test("SPARK-3004 regression: result set containing NULL") { + Thread.sleep(5 * 1000) + val dataFilePath = getDataFile("data/files/small_kv_with_null.txt") + val stmt = createStatement() + stmt.execute("DROP TABLE IF EXISTS test_null") + stmt.execute("CREATE TABLE test_null(key INT, val STRING)") + stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") + + val rs = stmt.executeQuery("SELECT * FROM test_null WHERE key IS NULL") + var count = 0 + while (rs.next()) { + count += 1 + } + assert(count === 5) + + stmt.close() + } + def getConnection: Connection = { val connectURI = s"jdbc:hive2://localhost:$PORT/" DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") From 13f54e2b97744beab45e1bdbcdf8d215ca481b78 Mon Sep 17 00:00:00 2001 From: tianyi Date: Wed, 13 Aug 2014 16:50:02 -0700 Subject: [PATCH 0337/1492] [SPARK-2817] [SQL] add "show create table" support In spark sql component, the "show create table" syntax had been disabled. We thought it is a useful funciton to describe a hive table. Author: tianyi Author: tianyi Author: tianyi Closes #1760 from tianyi/spark-2817 and squashes the following commits: 7d28b15 [tianyi] [SPARK-2817] fix too short prefix problem cbffe8b [tianyi] [SPARK-2817] fix the case problem 565ec14 [tianyi] [SPARK-2817] fix the case problem 60d48a9 [tianyi] [SPARK-2817] use system temporary folder instead of temporary files in the source tree, and also clean some empty line dbe1031 [tianyi] [SPARK-2817] move some code out of function rewritePaths, as it may be called multiple times 9b2ba11 [tianyi] [SPARK-2817] fix the line length problem 9f97586 [tianyi] [SPARK-2817] remove test.tmp.dir from pom.xml bfc2999 [tianyi] [SPARK-2817] add "File.separator" support, create a "testTmpDir" outside the rewritePaths bde800a [tianyi] [SPARK-2817] add "${system:test.tmp.dir}" support add "last_modified_by" to nonDeterministicLineIndicators in HiveComparisonTest bb82726 [tianyi] [SPARK-2817] remove test which requires a system from the whitelist. bbf6b42 [tianyi] [SPARK-2817] add a systemProperties named "test.tmp.dir" to pass the test which contains "${system:test.tmp.dir}" a337bd6 [tianyi] [SPARK-2817] add "show create table" support a03db77 [tianyi] [SPARK-2817] add "show create table" support --- .../execution/HiveCompatibilitySuite.scala | 8 +++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 1 + .../org/apache/spark/sql/hive/TestHive.scala | 8 +++++++ ...e_alter-0-813886d6cf0875c62e89cd1d06b8b0b4 | 0 ...e_alter-1-2a91d52719cf4552ebeb867204552a26 | 18 +++++++++++++++ ..._alter-10-259d978ed9543204c8b9c25b6e25b0de | 0 ...e_alter-2-928cc85c025440b731e5ee33e437e404 | 0 ...e_alter-3-2a91d52719cf4552ebeb867204552a26 | 22 +++++++++++++++++++ ...e_alter-4-c2cb6a7d942d4dddd1aababccb1239f9 | 0 ...e_alter-5-2a91d52719cf4552ebeb867204552a26 | 21 ++++++++++++++++++ ...le_alter-6-fdd1bd7f9acf0b2c8c9b7503d4046cb | 0 ...e_alter-7-2a91d52719cf4552ebeb867204552a26 | 21 ++++++++++++++++++ ...e_alter-8-22ab6ed5b15a018756f454dd2294847e | 0 ...e_alter-9-2a91d52719cf4552ebeb867204552a26 | 21 ++++++++++++++++++ ...b_table-0-67509558a4b2d39b25787cca33f52635 | 0 ...b_table-1-549981e00a3d95f03dd5a9ef6044aa20 | 2 ++ ...db_table-2-34ae7e611d0aedbc62b6e420347abee | 0 ...b_table-3-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...b_table-4-b585371b624cbab2616a49f553a870a0 | 13 +++++++++++ ...b_table-5-964757b7e7f2a69fe36132c1a5712199 | 0 ...b_table-6-ac09cf81e7e734cf10406f30b9fa566e | 0 ...limited-0-97228478b9925f06726ceebb6571bf34 | 0 ...limited-1-2a91d52719cf4552ebeb867204552a26 | 17 ++++++++++++++ ...limited-2-259d978ed9543204c8b9c25b6e25b0de | 0 ...itioned-0-4be9a3b1ff0840786a1f001cba170a0c | 0 ...itioned-1-2a91d52719cf4552ebeb867204552a26 | 16 ++++++++++++++ ...itioned-2-259d978ed9543204c8b9c25b6e25b0de | 0 ...e_serde-0-33f15d91810b75ee05c7b9dea0abb01c | 0 ...e_serde-1-2a91d52719cf4552ebeb867204552a26 | 15 +++++++++++++ ...e_serde-2-259d978ed9543204c8b9c25b6e25b0de | 0 ...e_serde-3-fd12b3e0fe30f5d71c67676791b4a33b | 0 ...e_serde-4-2a91d52719cf4552ebeb867204552a26 | 14 ++++++++++++ ...e_serde-5-259d978ed9543204c8b9c25b6e25b0de | 0 ...le_view-0-ecef6821e4e9212e553ca38142fd0250 | 0 ...le_view-1-1e931ea3fa6065107859ffbb29bb0ed7 | 1 + ...le_view-2-ed97e9e56d95c5b3db57485cba5ad17f | 0 .../hive/execution/HiveComparisonTest.scala | 1 + 37 files changed, 199 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-0-813886d6cf0875c62e89cd1d06b8b0b4 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-10-259d978ed9543204c8b9c25b6e25b0de create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-2-928cc85c025440b731e5ee33e437e404 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-4-c2cb6a7d942d4dddd1aababccb1239f9 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-6-fdd1bd7f9acf0b2c8c9b7503d4046cb create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-8-22ab6ed5b15a018756f454dd2294847e create mode 100644 sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_db_table-0-67509558a4b2d39b25787cca33f52635 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_db_table-1-549981e00a3d95f03dd5a9ef6044aa20 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_db_table-2-34ae7e611d0aedbc62b6e420347abee create mode 100644 sql/hive/src/test/resources/golden/show_create_table_db_table-3-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_db_table-5-964757b7e7f2a69fe36132c1a5712199 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_db_table-6-ac09cf81e7e734cf10406f30b9fa566e create mode 100644 sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_delimited-2-259d978ed9543204c8b9c25b6e25b0de create mode 100644 sql/hive/src/test/resources/golden/show_create_table_partitioned-0-4be9a3b1ff0840786a1f001cba170a0c create mode 100644 sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_partitioned-2-259d978ed9543204c8b9c25b6e25b0de create mode 100644 sql/hive/src/test/resources/golden/show_create_table_serde-0-33f15d91810b75ee05c7b9dea0abb01c create mode 100644 sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_serde-2-259d978ed9543204c8b9c25b6e25b0de create mode 100644 sql/hive/src/test/resources/golden/show_create_table_serde-3-fd12b3e0fe30f5d71c67676791b4a33b create mode 100644 sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_serde-5-259d978ed9543204c8b9c25b6e25b0de create mode 100644 sql/hive/src/test/resources/golden/show_create_table_view-0-ecef6821e4e9212e553ca38142fd0250 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 create mode 100644 sql/hive/src/test/resources/golden/show_create_table_view-2-ed97e9e56d95c5b3db57485cba5ad17f diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 4fef071161719..210753efe7678 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -635,6 +635,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "serde_regex", "serde_reported_schema", "set_variable_sub", + "show_create_table_partitioned", + "show_create_table_delimited", + "show_create_table_alter", + "show_create_table_view", + "show_create_table_serde", + "show_create_table_db_table", + "show_create_table_does_not_exist", + "show_create_table_index", "show_describe_func_quotes", "show_functions", "show_partitions", 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 05b2f5f6cd3f7..1d9ba1b24a7a4 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 @@ -53,6 +53,7 @@ private[hive] object HiveQl { protected val nativeCommands = Seq( "TOK_DESCFUNCTION", "TOK_DESCDATABASE", + "TOK_SHOW_CREATETABLE", "TOK_SHOW_TABLESTATUS", "TOK_SHOWDATABASES", "TOK_SHOWFUNCTIONS", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index d890df866fbe5..a013f3f7a805f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -70,6 +70,13 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { setConf("hive.metastore.warehouse.dir", warehousePath) } + val testTempDir = File.createTempFile("testTempFiles", "spark.hive.tmp") + testTempDir.delete() + testTempDir.mkdir() + + // For some hive test case which contain ${system:test.tmp.dir} + System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) + configure() // Must be called before initializing the catalog below. /** The location of the compiled hive distribution */ @@ -109,6 +116,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { hiveFilesTemp.mkdir() hiveFilesTemp.deleteOnExit() + val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) } else { diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-0-813886d6cf0875c62e89cd1d06b8b0b4 b/sql/hive/src/test/resources/golden/show_create_table_alter-0-813886d6cf0875c62e89cd1d06b8b0b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..3c1fc128bedce --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,18 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132100') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-10-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_alter-10-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-2-928cc85c025440b731e5ee33e437e404 b/sql/hive/src/test/resources/golden/show_create_table_alter-2-928cc85c025440b731e5ee33e437e404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..2ece813dd7d56 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,22 @@ +CREATE TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'temporary table' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'EXTERNAL'='FALSE', + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132100', + 'transient_lastDdlTime'='1407132100') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-4-c2cb6a7d942d4dddd1aababccb1239f9 b/sql/hive/src/test/resources/golden/show_create_table_alter-4-c2cb6a7d942d4dddd1aababccb1239f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..2af657bd29506 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,21 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'changed comment' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132100', + 'transient_lastDdlTime'='1407132100') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-6-fdd1bd7f9acf0b2c8c9b7503d4046cb b/sql/hive/src/test/resources/golden/show_create_table_alter-6-fdd1bd7f9acf0b2c8c9b7503d4046cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..f793ffb7a0bfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,21 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'changed comment' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132101', + 'transient_lastDdlTime'='1407132101') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-8-22ab6ed5b15a018756f454dd2294847e b/sql/hive/src/test/resources/golden/show_create_table_alter-8-22ab6ed5b15a018756f454dd2294847e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..c65aff26a7fc1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,21 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key smallint, + value float) +COMMENT 'changed comment' +CLUSTERED BY ( + key) +SORTED BY ( + value DESC) +INTO 5 BUCKETS +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED BY + 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler' +WITH SERDEPROPERTIES ( + 'serialization.format'='1') +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'last_modified_by'='tianyi', + 'last_modified_time'='1407132101', + 'transient_lastDdlTime'='1407132101') diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-0-67509558a4b2d39b25787cca33f52635 b/sql/hive/src/test/resources/golden/show_create_table_db_table-0-67509558a4b2d39b25787cca33f52635 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-1-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/show_create_table_db_table-1-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..707b2ae3ed1df --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_db_table-1-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1,2 @@ +default +tmp_feng diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-2-34ae7e611d0aedbc62b6e420347abee b/sql/hive/src/test/resources/golden/show_create_table_db_table-2-34ae7e611d0aedbc62b6e420347abee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-3-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/show_create_table_db_table-3-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 new file mode 100644 index 0000000000000..b5a18368ed85e --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 @@ -0,0 +1,13 @@ +CREATE TABLE tmp_feng.tmp_showcrt( + key string, + value int) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_feng.db/tmp_showcrt' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132107') diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-5-964757b7e7f2a69fe36132c1a5712199 b/sql/hive/src/test/resources/golden/show_create_table_db_table-5-964757b7e7f2a69fe36132c1a5712199 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-6-ac09cf81e7e734cf10406f30b9fa566e b/sql/hive/src/test/resources/golden/show_create_table_db_table-6-ac09cf81e7e734cf10406f30b9fa566e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..d36ad25dc8273 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,17 @@ +CREATE TABLE tmp_showcrt1( + key int, + value string, + newvalue bigint) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + COLLECTION ITEMS TERMINATED BY '|' + MAP KEYS TERMINATED BY '%' + LINES TERMINATED BY '\n' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132730') diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-2-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_delimited-2-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-0-4be9a3b1ff0840786a1f001cba170a0c b/sql/hive/src/test/resources/golden/show_create_table_partitioned-0-4be9a3b1ff0840786a1f001cba170a0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..9e572c0d7df6a --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,16 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key string, + newvalue boolean COMMENT 'a new value') +COMMENT 'temporary table' +PARTITIONED BY ( + value bigint COMMENT 'some value') +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.mapred.TextInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132112') diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-2-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_partitioned-2-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-0-33f15d91810b75ee05c7b9dea0abb01c b/sql/hive/src/test/resources/golden/show_create_table_serde-0-33f15d91810b75ee05c7b9dea0abb01c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..69a38e1a7b20a --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,15 @@ +CREATE TABLE tmp_showcrt1( + key int, + value string, + newvalue bigint) +COMMENT 'temporary table' +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED AS INPUTFORMAT + 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' +OUTPUTFORMAT + 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132115') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-2-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_serde-2-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-3-fd12b3e0fe30f5d71c67676791b4a33b b/sql/hive/src/test/resources/golden/show_create_table_serde-3-fd12b3e0fe30f5d71c67676791b4a33b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 new file mode 100644 index 0000000000000..b4e693dc622fb --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 @@ -0,0 +1,14 @@ +CREATE EXTERNAL TABLE tmp_showcrt1( + key string, + value boolean) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED BY + 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler' +WITH SERDEPROPERTIES ( + 'serialization.format'='$', + 'field.delim'=',') +LOCATION + 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' +TBLPROPERTIES ( + 'transient_lastDdlTime'='1407132115') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-5-259d978ed9543204c8b9c25b6e25b0de b/sql/hive/src/test/resources/golden/show_create_table_serde-5-259d978ed9543204c8b9c25b6e25b0de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-0-ecef6821e4e9212e553ca38142fd0250 b/sql/hive/src/test/resources/golden/show_create_table_view-0-ecef6821e4e9212e553ca38142fd0250 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 new file mode 100644 index 0000000000000..be3fb3ce30960 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 @@ -0,0 +1 @@ +CREATE VIEW tmp_copy_src AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src` diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-2-ed97e9e56d95c5b3db57485cba5ad17f b/sql/hive/src/test/resources/golden/show_create_table_view-2-ed97e9e56d95c5b3db57485cba5ad17f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 0ebaf6ffd5458..502ce8fb297e9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -161,6 +161,7 @@ abstract class HiveComparisonTest "transient_lastDdlTime", "grantTime", "lastUpdateTime", + "last_modified_by", "last_modified_time", "Owner:", // The following are hive specific schema parameters which we do not need to match exactly. From 9256d4a9c8c9ddb9ae6bbe3c3b99b03fb66b946b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 13 Aug 2014 17:35:38 -0700 Subject: [PATCH 0338/1492] [SPARK-2994][SQL] Support for udfs that take complex types Author: Michael Armbrust Closes #1915 from marmbrus/arrayUDF and squashes the following commits: a1c503d [Michael Armbrust] Support for udfs that take complex types --- .../spark/sql/hive/HiveInspectors.scala | 14 ++++++- .../org/apache/spark/sql/hive/hiveUdfs.scala | 41 +++++++++++-------- 2 files changed, 37 insertions(+), 18 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 354fcd53f303b..943bbaa8ce25e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -71,6 +71,9 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType case c: Class[_] if c.isArray => ArrayType(javaClassToDataType(c.getComponentType)) + + // Hive seems to return this for struct types? + case c: Class[_] if c == classOf[java.lang.Object] => NullType } /** Converts hive types to native catalyst types. */ @@ -147,7 +150,10 @@ private[hive] trait HiveInspectors { case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) case m: Map[_,_] => - mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) + // Some UDFs seem to assume we pass in a HashMap. + val hashMap = new java.util.HashMap[AnyRef, AnyRef]() + hashMap.putAll(m.map { case (k, v) => wrap(k) -> wrap(v) }) + hashMap case null => null } @@ -214,6 +220,12 @@ private[hive] trait HiveInspectors { import TypeInfoFactory._ def toTypeInfo: TypeInfo = dt match { + case ArrayType(elemType, _) => + getListTypeInfo(elemType.toTypeInfo) + case StructType(fields) => + getStructTypeInfo(fields.map(_.name), fields.map(_.dataType.toTypeInfo)) + case MapType(keyType, valueType, _) => + getMapTypeInfo(keyType.toTypeInfo, valueType.toTypeInfo) case BinaryType => binaryTypeInfo case BooleanType => booleanTypeInfo case ByteType => byteTypeInfo diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 179aac5cbd5cd..c6497a15efa0c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -55,7 +55,10 @@ private[hive] abstract class HiveFunctionRegistry HiveSimpleUdf( functionClassName, - children.zip(expectedDataTypes).map { case (e, t) => Cast(e, t) } + children.zip(expectedDataTypes).map { + case (e, NullType) => e + case (e, t) => Cast(e, t) + } ) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdf(functionClassName, children) @@ -115,22 +118,26 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) } - val constructor = matchingConstructor.getOrElse( - sys.error(s"No matching wrapper found, options: ${argClass.getConstructors.toSeq}.")) - - (a: Any) => { - logDebug( - s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} using $constructor.") - // We must make sure that primitives get boxed java style. - if (a == null) { - null - } else { - constructor.newInstance(a match { - case i: Int => i: java.lang.Integer - case bd: BigDecimal => new HiveDecimal(bd.underlying()) - case other: AnyRef => other - }).asInstanceOf[AnyRef] - } + matchingConstructor match { + case Some(constructor) => + (a: Any) => { + logDebug( + s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} $constructor.") + // We must make sure that primitives get boxed java style. + if (a == null) { + null + } else { + constructor.newInstance(a match { + case i: Int => i: java.lang.Integer + case bd: BigDecimal => new HiveDecimal(bd.underlying()) + case other: AnyRef => other + }).asInstanceOf[AnyRef] + } + } + case None => + (a: Any) => a match { + case wrapper => wrap(wrapper) + } } } From 376a82e196e102ef49b9722e8be0b01ac5890a8b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 13 Aug 2014 17:37:55 -0700 Subject: [PATCH 0339/1492] [SPARK-2650][SQL] More precise initial buffer size estimation for in-memory column buffer This is a follow up of #1880. Since the row number within a single batch is known, we can estimate a much more precise initial buffer size when building an in-memory column buffer. Author: Cheng Lian Closes #1901 from liancheng/precise-init-buffer-size and squashes the following commits: d5501fa [Cheng Lian] More precise initial buffer size estimation for in-memory column buffer --- .../sql/columnar/InMemoryColumnarTableScan.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 3364d0e18bcc9..e63b4903041f6 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 @@ -20,12 +20,11 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.{SparkPlan, LeafNode} -import org.apache.spark.sql.Row -import org.apache.spark.SparkConf +import org.apache.spark.sql.execution.{LeafNode, SparkPlan} object InMemoryRelation { def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = @@ -48,7 +47,9 @@ private[sql] case class InMemoryRelation( new Iterator[Array[ByteBuffer]] { def next() = { val columnBuilders = output.map { attribute => - ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression) + val columnType = ColumnType(attribute.dataType) + val initialBufferSize = columnType.defaultSize * batchSize + ColumnBuilder(columnType.typeId, initialBufferSize, attribute.name, useCompression) }.toArray var row: Row = null From 9fde1ff5fc114b5edb755ed40944607419b62184 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 13 Aug 2014 17:40:59 -0700 Subject: [PATCH 0340/1492] [SPARK-2935][SQL]Fix parquet predicate push down bug Author: Michael Armbrust Closes #1863 from marmbrus/parquetPredicates and squashes the following commits: 10ad202 [Michael Armbrust] left <=> right f249158 [Michael Armbrust] quiet parquet tests. 802da5b [Michael Armbrust] Add test case. eab2eda [Michael Armbrust] Fix parquet predicate push down bug --- .../scala/org/apache/spark/sql/parquet/ParquetFilters.scala | 5 +++-- sql/core/src/test/resources/log4j.properties | 3 +++ .../org/apache/spark/sql/parquet/ParquetQuerySuite.scala | 5 ++++- 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index cc575bedd8fcb..2298a9b933df5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -201,8 +201,9 @@ object ParquetFilters { (leftFilter, rightFilter) match { case (None, Some(filter)) => Some(filter) case (Some(filter), None) => Some(filter) - case (_, _) => - Some(new AndFilter(leftFilter.get, rightFilter.get)) + case (Some(leftF), Some(rightF)) => + Some(new AndFilter(leftF, rightF)) + case _ => None } } case p @ EqualTo(left: Literal, right: NamedExpression) if !right.nullable => diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index dffd15a61838b..c7e0ff1cf6494 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -36,6 +36,9 @@ log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n log4j.appender.FA.Threshold = INFO # Some packages are noisy for no good reason. +log4j.additivity.parquet.hadoop.ParquetRecordReader=false +log4j.logger.parquet.hadoop.ParquetRecordReader=OFF + log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF 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 9933575038bd3..502f6702e394e 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 @@ -381,11 +381,14 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val predicate5 = new GreaterThan(attribute1, attribute2) val badfilter = ParquetFilters.createFilter(predicate5) assert(badfilter.isDefined === false) + + val predicate6 = And(GreaterThan(attribute1, attribute2), GreaterThan(attribute1, attribute2)) + val badfilter2 = ParquetFilters.createFilter(predicate6) + assert(badfilter2.isDefined === false) } test("test filter by predicate pushdown") { for(myval <- Seq("myint", "mylong", "mydouble", "myfloat")) { - println(s"testing field $myval") val query1 = sql(s"SELECT * FROM testfiltersource WHERE $myval < 150 AND $myval >= 100") assert( query1.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], From 905dc4b405e679feb145f5e6b35e952db2442e0d Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 13 Aug 2014 17:42:38 -0700 Subject: [PATCH 0341/1492] [SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled Author: Kousuke Saruta Closes #1891 from sarutak/SPARK-2970 and squashes the following commits: 4a2d2fe [Kousuke Saruta] Modified comment style 8bd833c [Kousuke Saruta] Modified style 6c0997c [Kousuke Saruta] Modified the timing of shutdown hook execution. It should be executed before shutdown hook of o.a.h.f.FileSystem --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 4d0c506c5a397..4ed0f58ebc531 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -26,6 +26,8 @@ import jline.{ConsoleReader, History} import org.apache.commons.lang.StringUtils import org.apache.commons.logging.LogFactory import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} @@ -116,13 +118,17 @@ private[hive] object SparkSQLCLIDriver { SessionState.start(sessionState) // Clean up after we exit - Runtime.getRuntime.addShutdownHook( + /** + * This should be executed before shutdown hook of + * FileSystem to avoid race condition of FileSystem operation + */ + ShutdownHookManager.get.addShutdownHook( new Thread() { override def run() { SparkSQLEnv.stop() } } - ) + , FileSystem.SHUTDOWN_HOOK_PRIORITY - 1) // "-h" option has been passed, so connect to Hive thrift server. if (sessionState.getHost != null) { From 63d6777737ca8559d4344d1661500b8ad868bb47 Mon Sep 17 00:00:00 2001 From: guowei Date: Wed, 13 Aug 2014 17:45:24 -0700 Subject: [PATCH 0342/1492] [SPARK-2986] [SQL] fixed: setting properties does not effect it seems that set command does not run by SparkSQLDriver. it runs on hive api. user can not change reduce number by setting spark.sql.shuffle.partitions but i think setting hive properties seems just a role to spark sql. Author: guowei Closes #1904 from guowei2/temp-branch and squashes the following commits: 7d47dde [guowei] fixed: setting properties like spark.sql.shuffle.partitions does not effective --- .../spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 4ed0f58ebc531..c16a7d3661c66 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -34,7 +34,7 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.processors.{SetProcessor, CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket @@ -284,7 +284,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) if (proc != null) { - if (proc.isInstanceOf[Driver]) { + if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { val driver = new SparkSQLDriver driver.init() From 0c7b452904fe6b5a966a66b956369123d8a9dd4b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 13 Aug 2014 18:08:38 -0700 Subject: [PATCH 0343/1492] SPARK-3020: Print completed indices rather than tasks in web UI Author: Patrick Wendell Closes #1933 from pwendell/speculation and squashes the following commits: 33a3473 [Patrick Wendell] Use OpenHashSet 8ce2ff0 [Patrick Wendell] SPARK-3020: Print completed indices rather than tasks in web UI --- .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 1 + core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala | 2 +- core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala | 2 ++ 3 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index a57a354620163..a3e9566832d06 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -153,6 +153,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val (errorMessage, metrics): (Option[String], Option[TaskMetrics]) = taskEnd.reason match { case org.apache.spark.Success => + stageData.completedIndices.add(info.index) stageData.numCompleteTasks += 1 (None, Option(taskEnd.taskMetrics)) case e: ExceptionFailure => // Handle ExceptionFailure because we might have metrics diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 3dcfaf76e4aba..15998404ed612 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -168,7 +168,7 @@ private[ui] class StageTableBase( diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 85db15472a00c..a336bf7e1ed02 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -19,6 +19,7 @@ package org.apache.spark.ui.jobs import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} +import org.apache.spark.util.collection.OpenHashSet import scala.collection.mutable.HashMap @@ -38,6 +39,7 @@ private[jobs] object UIData { class StageUIData { var numActiveTasks: Int = _ var numCompleteTasks: Int = _ + var completedIndices = new OpenHashSet[Int]() var numFailedTasks: Int = _ var executorRunTime: Long = _ From 9497b12d429cf9d075807896637e40e205175203 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Wed, 13 Aug 2014 22:17:07 -0700 Subject: [PATCH 0344/1492] [SPARK-3006] Failed to execute spark-shell in Windows OS Modified the order of the options and arguments in spark-shell.cmd Author: Masayoshi TSUZUKI Closes #1918 from tsudukim/feature/SPARK-3006 and squashes the following commits: 8bba494 [Masayoshi TSUZUKI] [SPARK-3006] Failed to execute spark-shell in Windows OS 1a32410 [Masayoshi TSUZUKI] [SPARK-3006] Failed to execute spark-shell in Windows OS --- bin/spark-shell.cmd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index b56d69801171c..2ee60b4e2a2b3 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -19,4 +19,4 @@ rem set SPARK_HOME=%~dp0.. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd spark-shell --class org.apache.spark.repl.Main %* +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %* spark-shell From e4245656438d00714ebd59e89c4de3fdaae83494 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 13 Aug 2014 23:24:23 -0700 Subject: [PATCH 0345/1492] [Docs] Add missing tags (minor) These configs looked inconsistent from the rest. Author: Andrew Or Closes #1936 from andrewor14/docs-code and squashes the following commits: 15f578a [Andrew Or] Add tag --- docs/configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8136bd62ab6af..c8336b39133de 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -562,7 +562,7 @@ Apart from these, the following properties are also available, and may be useful - + - + + + + + +
    spark.io.compression.codecorg.apache.spark.io.
    SnappyCompressionCodec
    snappy - The codec used to compress internal data such as RDD partitions and shuffle outputs. - By default, Spark provides three codecs: org.apache.spark.io.LZ4CompressionCodec, + The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, + Spark provides three codecs: lz4, lzf, and snappy. You + can also use fully qualified class names to specify the codec, e.g. + org.apache.spark.io.LZ4CompressionCodec, org.apache.spark.io.LZFCompressionCodec, and org.apache.spark.io.SnappyCompressionCodec. {submissionTime} {formattedDuration} - {makeProgressBar(stageData.numActiveTasks, stageData.numCompleteTasks, + {makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, stageData.numFailedTasks, s.numTasks)} {inputReadWithUnit}
    spark.hadoop.validateOutputSpecsspark.hadoop.validateOutputSpecs true If set to true, validates the output specification (e.g. checking if the output directory already exists) used in saveAsHadoopFile and other variants. This can be disabled to silence exceptions due to pre-existing @@ -570,7 +570,7 @@ Apart from these, the following properties are also available, and may be useful previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand.
    spark.executor.heartbeatIntervalspark.executor.heartbeatInterval 10000 Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress From 69a57a18ee35af1cc5a00b67a80837ea317cd330 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 13 Aug 2014 23:53:44 -0700 Subject: [PATCH 0346/1492] [SPARK-2995][MLLIB] add ALS.setIntermediateRDDStorageLevel As mentioned in SPARK-2465, using `MEMORY_AND_DISK_SER` for user/product in/out links together with `spark.rdd.compress=true` can help reduce the space requirement by a lot, at the cost of speed. It might be useful to add this option so people can run ALS on much bigger datasets. Another option for the method name is `setIntermediateRDDStorageLevel`. Author: Xiangrui Meng Closes #1913 from mengxr/als-storagelevel and squashes the following commits: d942017 [Xiangrui Meng] rename to setIntermediateRDDStorageLevel 7550029 [Xiangrui Meng] add ALS.setIntermediateDataStorageLevel --- .../spark/mllib/recommendation/ALS.scala | 45 ++++++++++++------- 1 file changed, 30 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 8ebc7e27ed4dd..84d192db53e26 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -111,11 +111,17 @@ class ALS private ( */ def this() = this(-1, -1, 10, 10, 0.01, false, 1.0) + /** If true, do alternating nonnegative least squares. */ + private var nonnegative = false + + /** storage level for user/product in/out links */ + private var intermediateRDDStorageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK + /** * Set the number of blocks for both user blocks and product blocks to parallelize the computation * into; pass -1 for an auto-configured number of blocks. Default: -1. */ - def setBlocks(numBlocks: Int): ALS = { + def setBlocks(numBlocks: Int): this.type = { this.numUserBlocks = numBlocks this.numProductBlocks = numBlocks this @@ -124,7 +130,7 @@ class ALS private ( /** * Set the number of user blocks to parallelize the computation. */ - def setUserBlocks(numUserBlocks: Int): ALS = { + def setUserBlocks(numUserBlocks: Int): this.type = { this.numUserBlocks = numUserBlocks this } @@ -132,31 +138,31 @@ class ALS private ( /** * Set the number of product blocks to parallelize the computation. */ - def setProductBlocks(numProductBlocks: Int): ALS = { + def setProductBlocks(numProductBlocks: Int): this.type = { this.numProductBlocks = numProductBlocks this } /** Set the rank of the feature matrices computed (number of features). Default: 10. */ - def setRank(rank: Int): ALS = { + def setRank(rank: Int): this.type = { this.rank = rank this } /** Set the number of iterations to run. Default: 10. */ - def setIterations(iterations: Int): ALS = { + def setIterations(iterations: Int): this.type = { this.iterations = iterations this } /** Set the regularization parameter, lambda. Default: 0.01. */ - def setLambda(lambda: Double): ALS = { + def setLambda(lambda: Double): this.type = { this.lambda = lambda this } /** Sets whether to use implicit preference. Default: false. */ - def setImplicitPrefs(implicitPrefs: Boolean): ALS = { + def setImplicitPrefs(implicitPrefs: Boolean): this.type = { this.implicitPrefs = implicitPrefs this } @@ -166,29 +172,38 @@ class ALS private ( * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. */ @Experimental - def setAlpha(alpha: Double): ALS = { + def setAlpha(alpha: Double): this.type = { this.alpha = alpha this } /** Sets a random seed to have deterministic results. */ - def setSeed(seed: Long): ALS = { + def setSeed(seed: Long): this.type = { this.seed = seed this } - /** If true, do alternating nonnegative least squares. */ - private var nonnegative = false - /** * Set whether the least-squares problems solved at each iteration should have * nonnegativity constraints. */ - def setNonnegative(b: Boolean): ALS = { + def setNonnegative(b: Boolean): this.type = { this.nonnegative = b this } + /** + * :: DeveloperApi :: + * Sets storage level for intermediate RDDs (user/product in/out links). The default value is + * `MEMORY_AND_DISK`. Users can change it to a serialized storage, e.g., `MEMORY_AND_DISK_SER` and + * set `spark.rdd.compress` to `true` to reduce the space requirement, at the cost of speed. + */ + @DeveloperApi + def setIntermediateRDDStorageLevel(storageLevel: StorageLevel): this.type = { + this.intermediateRDDStorageLevel = storageLevel + this + } + /** * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. @@ -441,8 +456,8 @@ class ALS private ( }, preservesPartitioning = true) val inLinks = links.mapValues(_._1) val outLinks = links.mapValues(_._2) - inLinks.persist(StorageLevel.MEMORY_AND_DISK) - outLinks.persist(StorageLevel.MEMORY_AND_DISK) + inLinks.persist(intermediateRDDStorageLevel) + outLinks.persist(intermediateRDDStorageLevel) (inLinks, outLinks) } From d069c5d9d2f6ce06389ca2ddf0b3ae4db72c5797 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 14 Aug 2014 01:37:38 -0700 Subject: [PATCH 0347/1492] [SPARK-3029] Disable local execution of Spark jobs by default Currently, local execution of Spark jobs is only used by take(), and it can be problematic as it can load a significant amount of data onto the driver. The worst case scenarios occur if the RDD is cached (guaranteed to load whole partition), has very large elements, or the partition is just large and we apply a filter with high selectivity or computational overhead. Additionally, jobs that run locally in this manner do not show up in the web UI, and are thus harder to track or understand what is occurring. This PR adds a flag to disable local execution, which is turned OFF by default, with the intention of perhaps eventually removing this functionality altogether. Removing it now is a tougher proposition since it is part of the public runJob API. An alternative solution would be to limit the flag to take()/first() to avoid impacting any external users of this API, but such usage (or, at least, reliance upon the feature) is hopefully minimal. Author: Aaron Davidson Closes #1321 from aarondav/allowlocal and squashes the following commits: 136b253 [Aaron Davidson] Fix DAGSchedulerSuite 5599d55 [Aaron Davidson] [RFC] Disable local execution of Spark jobs by default --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 ++++++- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 4 +++- docs/configuration.md | 9 +++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 430e45ada5808..36bbaaa3f1c85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -121,6 +121,9 @@ class DAGScheduler( private[scheduler] var eventProcessActor: ActorRef = _ + /** If enabled, we may run certain actions like take() and first() locally. */ + private val localExecutionEnabled = sc.getConf.getBoolean("spark.localExecution.enabled", false) + private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is // not null before any job is submitted @@ -732,7 +735,9 @@ class DAGScheduler( logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") logInfo("Parents of final stage: " + finalStage.parents) logInfo("Missing parents: " + getMissingParentStages(finalStage)) - if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { + val shouldRunLocally = + localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 + if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 8c1b0fed11f72..bd829752eb401 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -141,7 +141,9 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } before { - sc = new SparkContext("local", "DAGSchedulerSuite") + // Enable local execution for this test + val conf = new SparkConf().set("spark.localExecution.enabled", "true") + sc = new SparkContext("local", "DAGSchedulerSuite", conf) sparkListener.successfulStages.clear() sparkListener.failedStages.clear() failure = null diff --git a/docs/configuration.md b/docs/configuration.md index c8336b39133de..c408c468dcd94 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -846,6 +846,15 @@ Apart from these, the following properties are also available, and may be useful (in milliseconds).
    spark.localExecution.enabledfalse + Enables Spark to run certain jobs, such as first() or take() on the driver, without sending + tasks to the cluster. This can make certain jobs execute very quickly, but may require + shipping a whole partition of data to the driver. +
    #### Security From 6b8de0e36c7548046c3b8a57f2c8e7e788dde8cc Mon Sep 17 00:00:00 2001 From: Graham Dennis Date: Thu, 14 Aug 2014 02:24:18 -0700 Subject: [PATCH 0348/1492] SPARK-2893: Do not swallow Exceptions when running a custom kryo registrator The previous behaviour of swallowing ClassNotFound exceptions when running a custom Kryo registrator could lead to difficult to debug problems later on at serialisation / deserialisation time, see SPARK-2878. Instead it is better to fail fast. Added test case. Author: Graham Dennis Closes #1827 from GrahamDennis/feature/spark-2893 and squashes the following commits: fbe4cb6 [Graham Dennis] [SPARK-2878]: Update the test case to match the updated exception message 65e53c5 [Graham Dennis] [SPARK-2893]: Improve message when a spark.kryo.registrator fails. f480d85 [Graham Dennis] [SPARK-2893] Fix typo. b59d2c2 [Graham Dennis] SPARK-2893: Do not swallow Exceptions when running a custom spark.kryo.registrator --- .../org/apache/spark/serializer/KryoSerializer.scala | 11 ++++++----- .../apache/spark/serializer/KryoSerializerSuite.scala | 10 ++++++++++ 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 407cb9db6ee9a..85944eabcfefc 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -79,15 +79,16 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) // Allow the user to register their own classes by setting spark.kryo.registrator - try { - for (regCls <- registrator) { - logDebug("Running user registrator: " + regCls) + for (regCls <- registrator) { + logDebug("Running user registrator: " + regCls) + try { val reg = Class.forName(regCls, true, classLoader).newInstance() .asInstanceOf[KryoRegistrator] reg.registerClasses(kryo) + } catch { + case e: Exception => + throw new SparkException(s"Failed to invoke $regCls", e) } - } catch { - case e: Exception => logError("Failed to run spark.kryo.registrator", e) } // Register Chill's classes; we do this after our ranges and the user's own classes to let diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 789b773bae316..3bf9efebb39d2 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -207,6 +207,16 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x assert(10 + control.sum === result) } + + test("kryo with nonexistent custom registrator should fail") { + import org.apache.spark.{SparkConf, SparkException} + + val conf = new SparkConf(false) + conf.set("spark.kryo.registrator", "this.class.does.not.exist") + + val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance()) + assert(thrown.getMessage.contains("Failed to invoke this.class.does.not.exist")) + } } class KryoSerializerResizableOutputSuite extends FunSuite { From 078f3fbda860e2f5de34153c55dfc3fecb4256e9 Mon Sep 17 00:00:00 2001 From: Chia-Yung Su Date: Thu, 14 Aug 2014 10:43:08 -0700 Subject: [PATCH 0349/1492] [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile Author: Chia-Yung Su Closes #1924 from joesu/bugfix-spark3011 and squashes the following commits: c7e44f2 [Chia-Yung Su] match syntax f8fc32a [Chia-Yung Su] filter out tmp dir --- .../main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index 2867dc0a8b1f9..37091bcf73dd6 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 @@ -375,7 +375,8 @@ private[parquet] object ParquetTypesConverter extends Logging { val children = fs.listStatus(path).filterNot { status => val name = status.getPath.getName - name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME + name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME || + name == FileOutputCommitter.TEMP_DIR_NAME } // NOTE (lian): Parquet "_metadata" file can be very slow if the file consists of lots of row From add75d4831fdc35712bf8b737574ea0bc677c37c Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 14 Aug 2014 10:46:33 -0700 Subject: [PATCH 0350/1492] [SPARK-2927][SQL] Add a conf to configure if we always read Binary columns stored in Parquet as String columns This PR adds a new conf flag `spark.sql.parquet.binaryAsString`. When it is `true`, if there is no parquet metadata file available to provide the schema of the data, we will always treat binary fields stored in parquet as string fields. This conf is used to provide a way to read string fields generated without UTF8 decoration. JIRA: https://issues.apache.org/jira/browse/SPARK-2927 Author: Yin Huai Closes #1855 from yhuai/parquetBinaryAsString and squashes the following commits: 689ffa9 [Yin Huai] Add missing "=". 80827de [Yin Huai] Unit test. 1765ca4 [Yin Huai] Use .toBoolean. 9d3f199 [Yin Huai] Merge remote-tracking branch 'upstream/master' into parquetBinaryAsString 5d436a1 [Yin Huai] The initial support of adding a conf to treat binary columns stored in Parquet as string columns. --- .../scala/org/apache/spark/sql/SQLConf.scala | 10 +++- .../spark/sql/parquet/ParquetRelation.scala | 6 ++- .../sql/parquet/ParquetTableSupport.scala | 3 +- .../spark/sql/parquet/ParquetTypes.scala | 36 +++++++------ .../spark/sql/parquet/ParquetQuerySuite.scala | 54 +++++++++++++++++-- 5 files changed, 87 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 35c51dec0bcf5..90de11182e605 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -31,6 +31,7 @@ private[spark] object SQLConf { val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" val CODEGEN_ENABLED = "spark.sql.codegen" val DIALECT = "spark.sql.dialect" + val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" @@ -87,8 +88,7 @@ trait SQLConf { * * Defaults to false as this feature is currently experimental. */ - private[spark] def codegenEnabled: Boolean = - if (getConf(CODEGEN_ENABLED, "false") == "true") true else false + private[spark] def codegenEnabled: Boolean = getConf(CODEGEN_ENABLED, "false").toBoolean /** * Upper bound on the sizes (in bytes) of the tables qualified for the auto conversion to @@ -108,6 +108,12 @@ trait SQLConf { private[spark] def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, (autoBroadcastJoinThreshold + 1).toString).toLong + /** + * When set to true, we always treat byte arrays in Parquet files as strings. + */ + private[spark] def isParquetBinaryAsString: Boolean = + getConf(PARQUET_BINARY_AS_STRING, "false").toBoolean + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index b3bae5db0edbc..053b2a154389c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -60,7 +60,11 @@ private[sql] case class ParquetRelation( .getSchema /** Attributes */ - override val output = ParquetTypesConverter.readSchemaFromFile(new Path(path), conf) + override val output = + ParquetTypesConverter.readSchemaFromFile( + new Path(path), + conf, + sqlContext.isParquetBinaryAsString) override def newInstance = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] 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 6d4ce32ac5bfa..6a657c20fe46c 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 @@ -80,9 +80,10 @@ private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { } } // if both unavailable, fall back to deducing the schema from the given Parquet schema + // TODO: Why it can be null? if (schema == null) { log.debug("falling back to Parquet read schema") - schema = ParquetTypesConverter.convertToAttributes(parquetSchema) + schema = ParquetTypesConverter.convertToAttributes(parquetSchema, false) } log.debug(s"list of attributes that will be read: $schema") new RowRecordMaterializer(parquetSchema, schema) 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 37091bcf73dd6..b0579f76da073 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 @@ -43,10 +43,13 @@ private[parquet] object ParquetTypesConverter extends Logging { def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass - def toPrimitiveDataType(parquetType: ParquetPrimitiveType): DataType = + def toPrimitiveDataType( + parquetType: ParquetPrimitiveType, + binayAsString: Boolean): DataType = parquetType.getPrimitiveTypeName match { case ParquetPrimitiveTypeName.BINARY - if parquetType.getOriginalType == ParquetOriginalType.UTF8 => StringType + if (parquetType.getOriginalType == ParquetOriginalType.UTF8 || + binayAsString) => StringType case ParquetPrimitiveTypeName.BINARY => BinaryType case ParquetPrimitiveTypeName.BOOLEAN => BooleanType case ParquetPrimitiveTypeName.DOUBLE => DoubleType @@ -85,7 +88,7 @@ private[parquet] object ParquetTypesConverter extends Logging { * @param parquetType The type to convert. * @return The corresponding Catalyst type. */ - def toDataType(parquetType: ParquetType): DataType = { + def toDataType(parquetType: ParquetType, isBinaryAsString: Boolean): DataType = { def correspondsToMap(groupType: ParquetGroupType): Boolean = { if (groupType.getFieldCount != 1 || groupType.getFields.apply(0).isPrimitive) { false @@ -107,7 +110,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } if (parquetType.isPrimitive) { - toPrimitiveDataType(parquetType.asPrimitiveType) + toPrimitiveDataType(parquetType.asPrimitiveType, isBinaryAsString) } else { val groupType = parquetType.asGroupType() parquetType.getOriginalType match { @@ -116,7 +119,7 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetOriginalType.LIST => { // TODO: check enums! assert(groupType.getFieldCount == 1) val field = groupType.getFields.apply(0) - ArrayType(toDataType(field), containsNull = false) + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) } case ParquetOriginalType.MAP => { assert( @@ -126,9 +129,9 @@ private[parquet] object ParquetTypesConverter extends Logging { assert( keyValueGroup.getFieldCount == 2, "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") - val keyType = toDataType(keyValueGroup.getFields.apply(0)) + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) - val valueType = toDataType(keyValueGroup.getFields.apply(1)) + val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true // at here. @@ -138,22 +141,22 @@ private[parquet] object ParquetTypesConverter extends Logging { // Note: the order of these checks is important! if (correspondsToMap(groupType)) { // MapType val keyValueGroup = groupType.getFields.apply(0).asGroupType() - val keyType = toDataType(keyValueGroup.getFields.apply(0)) + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) - val valueType = toDataType(keyValueGroup.getFields.apply(1)) + val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true // at here. MapType(keyType, valueType) } else if (correspondsToArray(groupType)) { // ArrayType - val elementType = toDataType(groupType.getFields.apply(0)) + val elementType = toDataType(groupType.getFields.apply(0), isBinaryAsString) ArrayType(elementType, containsNull = false) } else { // everything else: StructType val fields = groupType .getFields .map(ptype => new StructField( ptype.getName, - toDataType(ptype), + toDataType(ptype, isBinaryAsString), ptype.getRepetition != Repetition.REQUIRED)) StructType(fields) } @@ -276,7 +279,7 @@ private[parquet] object ParquetTypesConverter extends Logging { } } - def convertToAttributes(parquetSchema: ParquetType): Seq[Attribute] = { + def convertToAttributes(parquetSchema: ParquetType, isBinaryAsString: Boolean): Seq[Attribute] = { parquetSchema .asGroupType() .getFields @@ -284,7 +287,7 @@ private[parquet] object ParquetTypesConverter extends Logging { field => new AttributeReference( field.getName, - toDataType(field), + toDataType(field, isBinaryAsString), field.getRepetition != Repetition.REQUIRED)()) } @@ -404,7 +407,10 @@ private[parquet] object ParquetTypesConverter extends Logging { * @param conf The Hadoop configuration to use. * @return A list of attributes that make up the schema. */ - def readSchemaFromFile(origPath: Path, conf: Option[Configuration]): Seq[Attribute] = { + def readSchemaFromFile( + origPath: Path, + conf: Option[Configuration], + isBinaryAsString: Boolean): Seq[Attribute] = { val keyValueMetadata: java.util.Map[String, String] = readMetaData(origPath, conf) .getFileMetaData @@ -413,7 +419,7 @@ private[parquet] object ParquetTypesConverter extends Logging { convertFromString(keyValueMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) } else { val attributes = convertToAttributes( - readMetaData(origPath, conf).getFileMetaData.getSchema) + readMetaData(origPath, conf).getFileMetaData.getSchema, isBinaryAsString) log.info(s"Falling back to schema conversion from Parquet types; result: $attributes") attributes } 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 502f6702e394e..172dcd6aa0ee3 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 @@ -21,8 +21,6 @@ import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil -import parquet.schema.MessageTypeParser - import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job @@ -33,7 +31,6 @@ import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, IntegerType} import org.apache.spark.sql.catalyst.util.getTempFilePath -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.util.Utils @@ -138,6 +135,57 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } } + test("Treat binary as string") { + val oldIsParquetBinaryAsString = TestSQLContext.isParquetBinaryAsString + + // Create the test file. + val file = getTempFilePath("parquet") + val path = file.toString + val range = (0 to 255) + val rowRDD = TestSQLContext.sparkContext.parallelize(range) + .map(i => org.apache.spark.sql.Row(i, s"val_$i".getBytes)) + // We need to ask Parquet to store the String column as a Binary column. + val schema = StructType( + StructField("c1", IntegerType, false) :: + StructField("c2", BinaryType, false) :: Nil) + val schemaRDD1 = applySchema(rowRDD, schema) + schemaRDD1.saveAsParquetFile(path) + val resultWithBinary = parquetFile(path).collect + range.foreach { + i => + assert(resultWithBinary(i).getInt(0) === i) + assert(resultWithBinary(i)(1) === s"val_$i".getBytes) + } + + TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") + // This ParquetRelation always use Parquet types to derive output. + val parquetRelation = new ParquetRelation( + path.toString, + Some(TestSQLContext.sparkContext.hadoopConfiguration), + TestSQLContext) { + override val output = + ParquetTypesConverter.convertToAttributes( + ParquetTypesConverter.readMetaData(new Path(path), conf).getFileMetaData.getSchema, + TestSQLContext.isParquetBinaryAsString) + } + val schemaRDD = new SchemaRDD(TestSQLContext, parquetRelation) + val resultWithString = schemaRDD.collect + range.foreach { + i => + assert(resultWithString(i).getInt(0) === i) + assert(resultWithString(i)(1) === s"val_$i") + } + + schemaRDD.registerTempTable("tmp") + checkAnswer( + sql("SELECT c1, c2 FROM tmp WHERE c2 = 'val_5' OR c2 = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + // Set it back. + TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) + } + test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) From fde692b361773110c262abe219e7c8128bd76419 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Thu, 14 Aug 2014 10:48:52 -0700 Subject: [PATCH 0351/1492] [SQL] Python JsonRDD UTF8 Encoding Fix Only encode unicode objects to UTF-8, and not strings Author: Ahir Reddy Closes #1914 from ahirreddy/json-rdd-unicode-fix1 and squashes the following commits: ca4e9ba [Ahir Reddy] Encoding Fix --- python/pyspark/sql.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 46540ca3f1e8a..95086a2258222 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1267,7 +1267,9 @@ def func(iterator): for x in iterator: if not isinstance(x, basestring): x = unicode(x) - yield x.encode("utf-8") + if isinstance(x, unicode): + x = x.encode("utf-8") + yield x keyed = rdd.mapPartitions(func) keyed._bypass_serializer = True jrdd = keyed._jrdd.map(self._jvm.BytesToString()) From 267fdffe2743bc2dc706c8ac8af0ae33a358a5d3 Mon Sep 17 00:00:00 2001 From: wangfei Date: Thu, 14 Aug 2014 10:55:51 -0700 Subject: [PATCH 0352/1492] [SPARK-2925] [sql]fix spark-sql and start-thriftserver shell bugs when set --driver-java-options https://issues.apache.org/jira/browse/SPARK-2925 Run cmd like this will get the error bin/spark-sql --driver-java-options '-Xdebug -Xnoagent -Xrunjdwp:transport=dt_socket,address=8788,server=y,suspend=y' Error: Unrecognized option '-Xnoagent'. Run with --help for usage help or --verbose for debug output Author: wangfei Author: wangfei Closes #1851 from scwf/patch-2 and squashes the following commits: 516554d [wangfei] quote variables to fix this issue 8bd40f2 [wangfei] quote variables to fix this problem e6d79e3 [wangfei] fix start-thriftserver bug when set driver-java-options 948395d [wangfei] fix spark-sql error when set --driver-java-options --- bin/spark-sql | 18 +++++++++--------- sbin/start-thriftserver.sh | 8 ++++---- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/bin/spark-sql b/bin/spark-sql index 7813ccc361415..564f1f419060f 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -65,30 +65,30 @@ while (($#)); do case $1 in -d | --define | --database | -f | -h | --hiveconf | --hivevar | -i | -p) ensure_arg_number $# 2 - CLI_ARGS+=($1); shift - CLI_ARGS+=($1); shift + CLI_ARGS+=("$1"); shift + CLI_ARGS+=("$1"); shift ;; -e) ensure_arg_number $# 2 - CLI_ARGS+=($1); shift - CLI_ARGS+=(\"$1\"); shift + CLI_ARGS+=("$1"); shift + CLI_ARGS+=("$1"); shift ;; -s | --silent) - CLI_ARGS+=($1); shift + CLI_ARGS+=("$1"); shift ;; -v | --verbose) # Both SparkSubmit and SparkSQLCLIDriver recognizes -v | --verbose - CLI_ARGS+=($1) - SUBMISSION_ARGS+=($1); shift + CLI_ARGS+=("$1") + SUBMISSION_ARGS+=("$1"); shift ;; *) - SUBMISSION_ARGS+=($1); shift + SUBMISSION_ARGS+=("$1"); shift ;; esac done -eval exec "$FWDIR"/bin/spark-submit --class $CLASS ${SUBMISSION_ARGS[*]} spark-internal ${CLI_ARGS[*]} +exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${CLI_ARGS[@]}" diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 603f50ae13240..2c4452473ccbc 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -65,14 +65,14 @@ while (($#)); do case $1 in --hiveconf) ensure_arg_number $# 2 - THRIFT_SERVER_ARGS+=($1); shift - THRIFT_SERVER_ARGS+=($1); shift + THRIFT_SERVER_ARGS+=("$1"); shift + THRIFT_SERVER_ARGS+=("$1"); shift ;; *) - SUBMISSION_ARGS+=($1); shift + SUBMISSION_ARGS+=("$1"); shift ;; esac done -eval exec "$FWDIR"/bin/spark-submit --class $CLASS ${SUBMISSION_ARGS[*]} spark-internal ${THRIFT_SERVER_ARGS[*]} +exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${THRIFT_SERVER_ARGS[@]}" From eaeb0f76fa0f103c7db0f3975cb8562715410973 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 14 Aug 2014 11:22:41 -0700 Subject: [PATCH 0353/1492] Minor cleanup of metrics.Source - Added override. - Marked some variables as private. Author: Reynold Xin Closes #1943 from rxin/metricsSource and squashes the following commits: fbfa943 [Reynold Xin] Minor cleanup of metrics.Source. - Added override. - Marked some variables as private. --- .../spark/deploy/master/ApplicationSource.scala | 4 ++-- .../org/apache/spark/deploy/master/MasterSource.scala | 4 ++-- .../org/apache/spark/deploy/worker/WorkerSource.scala | 4 ++-- .../org/apache/spark/executor/ExecutorSource.scala | 5 +++-- .../org/apache/spark/metrics/source/JvmSource.scala | 11 ++++------- .../apache/spark/scheduler/DAGSchedulerSource.scala | 4 ++-- .../org/apache/spark/storage/BlockManagerSource.scala | 4 ++-- .../org/apache/spark/streaming/StreamingSource.scala | 6 +++--- 8 files changed, 20 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala index c87b66f047dc8..38db02cd2421b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationSource.scala @@ -22,8 +22,8 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source class ApplicationSource(val application: ApplicationInfo) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "%s.%s.%s".format("application", application.desc.name, + override val metricRegistry = new MetricRegistry() + override val sourceName = "%s.%s.%s".format("application", application.desc.name, System.currentTimeMillis()) metricRegistry.register(MetricRegistry.name("status"), new Gauge[String] { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala index 36c1b87b7f684..9c3f79f1244b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterSource.scala @@ -22,8 +22,8 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source private[spark] class MasterSource(val master: Master) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "master" + override val metricRegistry = new MetricRegistry() + override val sourceName = "master" // Gauge for worker numbers in cluster metricRegistry.register(MetricRegistry.name("workers"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala index b7ddd8c816cbc..df1e01b23b932 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerSource.scala @@ -22,8 +22,8 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.metrics.source.Source private[spark] class WorkerSource(val worker: Worker) extends Source { - val sourceName = "worker" - val metricRegistry = new MetricRegistry() + override val sourceName = "worker" + override val metricRegistry = new MetricRegistry() metricRegistry.register(MetricRegistry.name("executors"), new Gauge[Int] { override def getValue: Int = worker.executors.size diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 0ed52cfe9df61..d6721586566c2 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -35,9 +35,10 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) }) } - val metricRegistry = new MetricRegistry() + override val metricRegistry = new MetricRegistry() + // TODO: It would be nice to pass the application name here - val sourceName = "executor.%s".format(executorId) + override val sourceName = "executor.%s".format(executorId) // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index f865f9648a91e..635bff2cd7ec8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -21,12 +21,9 @@ import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} private[spark] class JvmSource extends Source { - val sourceName = "jvm" - val metricRegistry = new MetricRegistry() + override val sourceName = "jvm" + override val metricRegistry = new MetricRegistry() - val gcMetricSet = new GarbageCollectorMetricSet - val memGaugeSet = new MemoryUsageGaugeSet - - metricRegistry.registerAll(gcMetricSet) - metricRegistry.registerAll(memGaugeSet) + metricRegistry.registerAll(new GarbageCollectorMetricSet) + metricRegistry.registerAll(new MemoryUsageGaugeSet) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 5878e733908f5..94944399b134a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -24,8 +24,8 @@ import org.apache.spark.metrics.source.Source private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "%s.DAGScheduler".format(sc.appName) + override val metricRegistry = new MetricRegistry() + override val sourceName = "%s.DAGScheduler".format(sc.appName) metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { override def getValue: Int = dagScheduler.failedStages.size diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 3f14c40ec61cb..49fea6d9e2a76 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -24,8 +24,8 @@ import org.apache.spark.metrics.source.Source private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) extends Source { - val metricRegistry = new MetricRegistry() - val sourceName = "%s.BlockManager".format(sc.appName) + override val metricRegistry = new MetricRegistry() + override val sourceName = "%s.BlockManager".format(sc.appName) metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index 774adc3c23c21..75f0e8716dc7e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -23,10 +23,10 @@ import org.apache.spark.metrics.source.Source import org.apache.spark.streaming.ui.StreamingJobProgressListener private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { - val metricRegistry = new MetricRegistry - val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) + override val metricRegistry = new MetricRegistry + override val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) - val streamingListener = ssc.uiTab.listener + private val streamingListener = ssc.uiTab.listener private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, defaultValue: T) { From 96221067572e5955af1a7710b0cca33a73db4bd5 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 14 Aug 2014 11:56:13 -0700 Subject: [PATCH 0354/1492] [SPARK-2979][MLlib] Improve the convergence rate by minimizing the condition number MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In theory, the scale of your inputs are irrelevant to logistic regression. You can "theoretically" multiply X1 by 1E6 and the estimate for β1 will adjust accordingly. It will be 1E-6 times smaller than the original β1, due to the invariance property of MLEs. However, during the optimization process, the convergence (rate) depends on the condition number of the training dataset. Scaling the variables often reduces this condition number, thus improving the convergence rate. Without reducing the condition number, some training datasets mixing the columns with different scales may not be able to converge. GLMNET and LIBSVM packages perform the scaling to reduce the condition number, and return the weights in the original scale. See page 9 in http://cran.r-project.org/web/packages/glmnet/glmnet.pdf Here, if useFeatureScaling is enabled, we will standardize the training features by dividing the variance of each column (without subtracting the mean to densify the sparse vector), and train the model in the scaled space. Then we transform the coefficients from the scaled space to the original scale as GLMNET and LIBSVM do. Currently, it's only enabled in LogisticRegressionWithLBFGS. Author: DB Tsai Closes #1897 from dbtsai/dbtsai-feature-scaling and squashes the following commits: f19fc02 [DB Tsai] Added more comments 1d85289 [DB Tsai] Improve the convergence rate by minimize the condition number in LOR with LBFGS --- .../classification/LogisticRegression.scala | 4 +- .../GeneralizedLinearAlgorithm.scala | 69 ++++++++++++++++++- .../LogisticRegressionSuite.scala | 57 +++++++++++++++ 3 files changed, 126 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 31d474a20fa85..6790c86f651b4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -62,7 +62,7 @@ class LogisticRegressionModel ( override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept - val score = 1.0/ (1.0 + math.exp(-margin)) + val score = 1.0 / (1.0 + math.exp(-margin)) threshold match { case Some(t) => if (score < t) 0.0 else 1.0 case None => score @@ -204,6 +204,8 @@ class LogisticRegressionWithLBFGS private ( */ def this() = this(1E-4, 100, 0.0) + this.setFeatureScaling(true) + private val gradient = new LogisticGradient() private val updater = new SimpleUpdater() // Have to return new LBFGS object every time since users can reset the parameters anytime. 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 54854252d7477..20c1fdd2269ce 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 @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.feature.StandardScaler import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ @@ -94,6 +95,22 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected var validateData: Boolean = true + /** + * Whether to perform feature scaling before model training to reduce the condition numbers + * which can significantly help the optimizer converging faster. The scaling correction will be + * translated back to resulting model weights, so it's transparent to users. + * Note: This technique is used in both libsvm and glmnet packages. Default false. + */ + private var useFeatureScaling = false + + /** + * Set if the algorithm should use feature scaling to improve the convergence during optimization. + */ + private[mllib] def setFeatureScaling(useFeatureScaling: Boolean): this.type = { + this.useFeatureScaling = useFeatureScaling + this + } + /** * Create a model given the weights and intercept */ @@ -137,11 +154,45 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] throw new SparkException("Input validation failed.") } + /** + * Scaling columns to unit variance as a heuristic to reduce the condition number: + * + * During the optimization process, the convergence (rate) depends on the condition number of + * the training dataset. Scaling the variables often reduces this condition number + * heuristically, thus improving the convergence rate. Without reducing the condition number, + * some training datasets mixing the columns with different scales may not be able to converge. + * + * GLMNET and LIBSVM packages perform the scaling to reduce the condition number, and return + * the weights in the original scale. + * See page 9 in http://cran.r-project.org/web/packages/glmnet/glmnet.pdf + * + * Here, if useFeatureScaling is enabled, we will standardize the training features by dividing + * the variance of each column (without subtracting the mean), and train the model in the + * scaled space. Then we transform the coefficients from the scaled space to the original scale + * as GLMNET and LIBSVM do. + * + * Currently, it's only enabled in LogisticRegressionWithLBFGS + */ + val scaler = if (useFeatureScaling) { + (new StandardScaler).fit(input.map(x => x.features)) + } else { + null + } + // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, appendBias(labeledPoint.features))) + if(useFeatureScaling) { + input.map(labeledPoint => + (labeledPoint.label, appendBias(scaler.transform(labeledPoint.features)))) + } else { + input.map(labeledPoint => (labeledPoint.label, appendBias(labeledPoint.features))) + } } else { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) + if (useFeatureScaling) { + input.map(labeledPoint => (labeledPoint.label, scaler.transform(labeledPoint.features))) + } else { + input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) + } } val initialWeightsWithIntercept = if (addIntercept) { @@ -153,13 +204,25 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) val intercept = if (addIntercept) weightsWithIntercept(weightsWithIntercept.size - 1) else 0.0 - val weights = + var weights = if (addIntercept) { Vectors.dense(weightsWithIntercept.toArray.slice(0, weightsWithIntercept.size - 1)) } else { weightsWithIntercept } + /** + * The weights and intercept are trained in the scaled space; we're converting them back to + * the original scale. + * + * Math shows that if we only perform standardization without subtracting means, the intercept + * will not be changed. w_i = w_i' / v_i where w_i' is the coefficient in the scaled space, w_i + * is the coefficient in the original space, and v_i is the variance of the column i. + */ + if (useFeatureScaling) { + weights = scaler.transform(weights) + } + createModel(weights, intercept) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 2289c6cdc19de..bc05b2046878f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -185,6 +185,63 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("numerical stability of scaling features using logistic regression with LBFGS") { + /** + * If we rescale the features, the condition number will be changed so the convergence rate + * and the solution will not equal to the original solution multiple by the scaling factor + * which it should be. + * + * However, since in the LogisticRegressionWithLBFGS, we standardize the training dataset first, + * no matter how we multiple a scaling factor into the dataset, the convergence rate should be + * the same, and the solution should equal to the original solution multiple by the scaling + * factor. + */ + + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val initialWeights = Vectors.dense(0.0) + + val testRDD1 = sc.parallelize(testData, 2) + + val testRDD2 = sc.parallelize( + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E3))), 2) + + val testRDD3 = sc.parallelize( + testData.map(x => LabeledPoint(x.label, Vectors.fromBreeze(x.features.toBreeze * 1.0E6))), 2) + + testRDD1.cache() + testRDD2.cache() + testRDD3.cache() + + val lrA = new LogisticRegressionWithLBFGS().setIntercept(true) + val lrB = new LogisticRegressionWithLBFGS().setIntercept(true).setFeatureScaling(false) + + val modelA1 = lrA.run(testRDD1, initialWeights) + val modelA2 = lrA.run(testRDD2, initialWeights) + val modelA3 = lrA.run(testRDD3, initialWeights) + + val modelB1 = lrB.run(testRDD1, initialWeights) + val modelB2 = lrB.run(testRDD2, initialWeights) + val modelB3 = lrB.run(testRDD3, initialWeights) + + // For model trained with feature standardization, the weights should + // be the same in the scaled space. Note that the weights here are already + // in the original space, we transform back to scaled space to compare. + assert(modelA1.weights(0) ~== modelA2.weights(0) * 1.0E3 absTol 0.01) + assert(modelA1.weights(0) ~== modelA3.weights(0) * 1.0E6 absTol 0.01) + + // Training data with different scales without feature standardization + // will not yield the same result in the scaled space due to poor + // convergence rate. + assert(modelB1.weights(0) !~== modelB2.weights(0) * 1.0E3 absTol 0.1) + assert(modelB1.weights(0) !~== modelB3.weights(0) * 1.0E6 absTol 0.1) + } + } class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkContext { From a7f8a4f5ee757450ce8d4028021441435081cf53 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 14 Aug 2014 13:00:21 -0700 Subject: [PATCH 0355/1492] Revert [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile Reverts #1924 due to build failures with hadoop 0.23. Author: Michael Armbrust Closes #1949 from marmbrus/revert1924 and squashes the following commits: 6bff940 [Michael Armbrust] Revert "[SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile" --- .../main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 b0579f76da073..c79a9ac2dad81 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -378,8 +378,7 @@ private[parquet] object ParquetTypesConverter extends Logging { val children = fs.listStatus(path).filterNot { status => val name = status.getPath.getName - name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME || - name == FileOutputCommitter.TEMP_DIR_NAME + name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME } // NOTE (lian): Parquet "_metadata" file can be very slow if the file consists of lots of row From a75bc7a21db07258913d038bf604c0a3c1e55b46 Mon Sep 17 00:00:00 2001 From: Jacek Lewandowski Date: Thu, 14 Aug 2014 15:01:39 -0700 Subject: [PATCH 0356/1492] SPARK-3009: Reverted readObject method in ApplicationInfo so that Applic... ...ationInfo is initialized properly after deserialization Author: Jacek Lewandowski Closes #1947 from jacek-lewandowski/master and squashes the following commits: 713b2f1 [Jacek Lewandowski] SPARK-3009: Reverted readObject method in ApplicationInfo so that ApplicationInfo is initialized properly after deserialization --- .../org/apache/spark/deploy/master/ApplicationInfo.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 72d0589689e71..d3674427b1271 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -46,6 +46,11 @@ private[spark] class ApplicationInfo( init() + private def readObject(in: java.io.ObjectInputStream): Unit = { + in.defaultReadObject() + init() + } + private def init() { state = ApplicationState.WAITING executors = new mutable.HashMap[Int, ExecutorInfo] From fa5a08e67d1086045ac249c2090c5e4d0a17b828 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 14 Aug 2014 16:27:11 -0700 Subject: [PATCH 0357/1492] Make dev/mima runnable on Mac OS X. Mac OS X's find is from the BSD variant that doesn't have -printf option. Author: Reynold Xin Closes #1953 from rxin/mima and squashes the following commits: e284afe [Reynold Xin] Make dev/mima runnable on Mac OS X. --- dev/mima | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/mima b/dev/mima index 4c3e65039b160..09e4482af5f3d 100755 --- a/dev/mima +++ b/dev/mima @@ -26,7 +26,9 @@ cd "$FWDIR" echo -e "q\n" | sbt/sbt oldDeps/update -export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) -printf "%p:" ` +export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"` +echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" + ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" ret_val=$? From 655699f8b7156e8216431393436368e80626cdb2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 14 Aug 2014 18:37:02 -0700 Subject: [PATCH 0358/1492] [SPARK-3027] TaskContext: tighten visibility and provide Java friendly callback API Note this also passes the TaskContext itself to the TaskCompletionListener. In the future we can mark TaskContext with the exception object if exception occurs during task execution. Author: Reynold Xin Closes #1938 from rxin/TaskContext and squashes the following commits: 145de43 [Reynold Xin] Added JavaTaskCompletionListenerImpl for Java API friendly guarantee. f435ea5 [Reynold Xin] Added license header for TaskCompletionListener. dc4ed27 [Reynold Xin] [SPARK-3027] TaskContext: tighten the visibility and provide Java friendly callback API --- .../apache/spark/InterruptibleIterator.scala | 2 +- .../scala/org/apache/spark/TaskContext.scala | 63 ++++++++++++++++--- .../apache/spark/api/python/PythonRDD.scala | 12 ++-- .../org/apache/spark/rdd/CheckpointRDD.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/ResultTask.scala | 2 +- .../spark/scheduler/ShuffleMapTask.scala | 2 +- .../org/apache/spark/scheduler/Task.scala | 2 +- .../spark/util/TaskCompletionListener.scala | 33 ++++++++++ .../util/JavaTaskCompletionListenerImpl.java | 39 ++++++++++++ .../spark/scheduler/TaskContextSuite.scala | 2 +- 14 files changed, 144 insertions(+), 23 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/TaskCompletionListener.scala create mode 100644 core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index f40baa8e43592..5c262bcbddf76 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -33,7 +33,7 @@ class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator // is allowed. The assumption is that Thread.interrupted does not have a memory fence in read // (just a volatile field in C), while context.interrupted is a volatile in the JVM, which // introduces an expensive read fence. - if (context.interrupted) { + if (context.isInterrupted) { throw new TaskKilledException } else { delegate.hasNext diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 51f40c339d13c..2b99b8a5af250 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -21,10 +21,18 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.TaskCompletionListener + /** * :: DeveloperApi :: * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + * @param runningLocally whether the task is running locally in the driver JVM + * @param taskMetrics performance metrics of the task */ @DeveloperApi class TaskContext( @@ -39,13 +47,45 @@ class TaskContext( def splitId = partitionId // List of callback functions to execute when the task completes. - @transient private val onCompleteCallbacks = new ArrayBuffer[() => Unit] + @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] // Whether the corresponding task has been killed. - @volatile var interrupted: Boolean = false + @volatile private var interrupted: Boolean = false + + // Whether the task has completed. + @volatile private var completed: Boolean = false + + /** Checks whether the task has completed. */ + def isCompleted: Boolean = completed - // Whether the task has completed, before the onCompleteCallbacks are executed. - @volatile var completed: Boolean = false + /** Checks whether the task has been killed. */ + def isInterrupted: Boolean = interrupted + + // TODO: Also track whether the task has completed successfully or with exception. + + /** + * Add a (Java friendly) listener to be executed on task completion. + * This will be called in all situation - success, failure, or cancellation. + * + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { + onCompleteCallbacks += listener + this + } + + /** + * Add a listener in the form of a Scala closure to be executed on task completion. + * This will be called in all situation - success, failure, or cancellation. + * + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + def addTaskCompletionListener(f: TaskContext => Unit): this.type = { + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f(context) + } + this + } /** * Add a callback function to be executed on task completion. An example use @@ -53,13 +93,22 @@ class TaskContext( * Will be called in any situation - success, failure, or cancellation. * @param f Callback function. */ + @deprecated("use addTaskCompletionListener", "1.1.0") def addOnCompleteCallback(f: () => Unit) { - onCompleteCallbacks += f + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f() + } } - def executeOnCompleteCallbacks() { + /** Marks the task as completed and triggers the listeners. */ + private[spark] def markTaskCompleted(): Unit = { completed = true // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach { _() } + onCompleteCallbacks.reverse.foreach { _.onTaskCompletion(this) } + } + + /** Marks the task for interruption, i.e. cancellation. */ + private[spark] def markInterrupted(): Unit = { + interrupted = true } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 0b5322c6fb965..fefe1cb6f134c 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -68,7 +68,7 @@ private[spark] class PythonRDD( // Start a thread to feed the process input from our parent's iterator val writerThread = new WriterThread(env, worker, split, context) - context.addOnCompleteCallback { () => + context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() // Cleanup the worker socket. This will also cause the Python worker to exit. @@ -137,7 +137,7 @@ private[spark] class PythonRDD( } } catch { - case e: Exception if context.interrupted => + case e: Exception if context.isInterrupted => logDebug("Exception thrown after task interruption", e) throw new TaskKilledException @@ -176,7 +176,7 @@ private[spark] class PythonRDD( /** Terminates the writer thread, ignoring any exceptions that may occur due to cleanup. */ def shutdownOnTaskCompletion() { - assert(context.completed) + assert(context.isCompleted) this.interrupt() } @@ -209,7 +209,7 @@ private[spark] class PythonRDD( PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) dataOut.flush() } catch { - case e: Exception if context.completed || context.interrupted => + case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) case e: Exception => @@ -235,10 +235,10 @@ private[spark] class PythonRDD( override def run() { // Kill the worker if it is interrupted, checking until task completion. // TODO: This has a race condition if interruption occurs, as completed may still become true. - while (!context.interrupted && !context.completed) { + while (!context.isInterrupted && !context.isCompleted) { Thread.sleep(2000) } - if (!context.completed) { + if (!context.isCompleted) { try { logWarning("Incomplete task interrupted: Attempting to kill Python Worker") env.destroyPythonWorker(pythonExec, envVars.toMap, worker) diff --git a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala index 34c51b833025e..20938781ac694 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala @@ -141,7 +141,7 @@ private[spark] object CheckpointRDD extends Logging { val deserializeStream = serializer.deserializeStream(fileInputStream) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => deserializeStream.close()) + context.addTaskCompletionListener(context => deserializeStream.close()) deserializeStream.asIterator.asInstanceOf[Iterator[T]] } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 8d92ea01d9a3f..c8623314c98eb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -197,7 +197,7 @@ class HadoopRDD[K, V]( reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback{ () => closeIfNeeded() } + context.addTaskCompletionListener{ context => closeIfNeeded() } val key: K = reader.createKey() val value: V = reader.createValue() diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 8947e66f4577c..0e38f224ac81d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -68,7 +68,7 @@ class JdbcRDD[T: ClassTag]( } override def compute(thePart: Partition, context: TaskContext) = new NextIterator[T] { - context.addOnCompleteCallback{ () => closeIfNeeded() } + context.addTaskCompletionListener{ context => closeIfNeeded() } val part = thePart.asInstanceOf[JdbcPartition] val conn = getConnection() val stmt = conn.prepareStatement(sql, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY) diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 7dfec9a18ec67..58f707b9b4634 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -129,7 +129,7 @@ class NewHadoopRDD[K, V]( context.taskMetrics.inputMetrics = Some(inputMetrics) // Register an on-task-completion callback to close the input stream. - context.addOnCompleteCallback(() => close()) + context.addTaskCompletionListener(context => close()) var havePair = false var finished = false diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 36bbaaa3f1c85..b86cfbfa48fbe 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -634,7 +634,7 @@ class DAGScheduler( val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { - taskContext.executeOnCompleteCallbacks() + taskContext.markTaskCompleted() } } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index d09fd7aa57642..2ccbd8edeb028 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -61,7 +61,7 @@ private[spark] class ResultTask[T, U]( try { func(context, rdd.iterator(partition, context)) } finally { - context.executeOnCompleteCallbacks() + context.markTaskCompleted() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 11255c07469d4..381eff2147e95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -74,7 +74,7 @@ private[spark] class ShuffleMapTask( } throw e } finally { - context.executeOnCompleteCallbacks() + context.markTaskCompleted() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index cbe0bc0bcb0a5..6aa0cca06878d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -87,7 +87,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex def kill(interruptThread: Boolean) { _killed = true if (context != null) { - context.interrupted = true + context.markInterrupted() } if (interruptThread && taskThread != null) { taskThread.interrupt() diff --git a/core/src/main/scala/org/apache/spark/util/TaskCompletionListener.scala b/core/src/main/scala/org/apache/spark/util/TaskCompletionListener.scala new file mode 100644 index 0000000000000..c1b8bf052c0ca --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/TaskCompletionListener.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import java.util.EventListener + +import org.apache.spark.TaskContext +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * + * Listener providing a callback function to invoke when a task's execution completes. + */ +@DeveloperApi +trait TaskCompletionListener extends EventListener { + def onTaskCompletion(context: TaskContext) +} diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java new file mode 100644 index 0000000000000..af34cdb03e4d1 --- /dev/null +++ b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java @@ -0,0 +1,39 @@ +/* + * 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.util; + +import org.apache.spark.TaskContext; + + +/** + * A simple implementation of TaskCompletionListener that makes sure TaskCompletionListener and + * TaskContext is Java friendly. + */ +public class JavaTaskCompletionListenerImpl implements TaskCompletionListener { + + @Override + public void onTaskCompletion(TaskContext context) { + context.isCompleted(); + context.isInterrupted(); + context.stageId(); + context.partitionId(); + context.runningLocally(); + context.taskMetrics(); + context.addTaskCompletionListener(this); + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index 270f7e661045a..db2ad829a48f9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -32,7 +32,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte val rdd = new RDD[String](sc, List()) { override def getPartitions = Array[Partition](StubPartition(0)) override def compute(split: Partition, context: TaskContext) = { - context.addOnCompleteCallback(() => TaskContextSuite.completed = true) + context.addTaskCompletionListener(context => TaskContextSuite.completed = true) sys.error("failed") } } From 3a8b68b7353fea50245686903b308fa9eb52cb51 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 14 Aug 2014 19:01:33 -0700 Subject: [PATCH 0359/1492] [SPARK-2468] Netty based block server / client module This is a rewrite of the original Netty module that was added about 1.5 years ago. The old code was turned off by default and didn't really work because it lacked a frame decoder (only worked with very very small blocks). For this pull request, I tried to make the changes non-instrusive to the rest of Spark. I only added an init and shutdown to BlockManager/DiskBlockManager, and a bunch of comments to help me understand the existing code base. Compared with the old Netty module, this one features: - It appears to work :) - SPARK-2941: option to specicy nio vs oio vs epoll for channel/transport. By default nio is used. (Not using Epoll yet because I have found some bugs with its implementation) - SPARK-2943: options to specify send buf and receive buf for users who want to do hyper tuning - SPARK-2942: io errors are reported from server to client (the protocol uses negative length to indicate error) - SPARK-2940: fetching multiple blocks in a single request to reduce syscalls - SPARK-2959: clients share a single thread pool - SPARK-2990: use PooledByteBufAllocator to reduce GC (basically a Netty managed pool of buffers with jmalloc) - SPARK-2625: added fetchWaitTime metric and fixed thread-safety issue in metrics update. - SPARK-2367: bump Netty version to 4.0.21.Final to address an Epoll bug (https://groups.google.com/forum/#!topic/netty/O7m-HxCJpCA) Compared with the existing communication manager, this one features: - IMO it is substantially easier to understand - zero-copy send for the server for on-disk blocks - one-copy receive (due to a frame decoder) - don't quote me on this, but I think a lot less sys calls - SPARK-2990: use PooledByteBufAllocator to reduce GC (basically a Netty managed pool of buffers with jmalloc) - SPARK-2941: option to specicy nio vs oio vs epoll for channel/transport. By default nio is used. (Not using Epoll yet because I have found some bugs with its implementation) - SPARK-2943: options to specify send buf and receive buf for users who want to do hyper tuning TODOs before it can fully replace the existing ConnectionManager, if that ever happens (most of them should probably be done in separate PRs since this needs to be turned on explicitly) - [x] Basic test cases - [ ] More unit/integration tests for failures - [ ] Performance analysis - [ ] Support client connection reuse so we don't need to keep opening new connections (not sure how useful this would be) - [ ] Support putting blocks in addition to fetching blocks (i.e. two way transfer) - [x] Support serving non-disk blocks - [ ] Support SASL authentication For a more comprehensive list, see https://issues.apache.org/jira/browse/SPARK-2468 Thanks to @coderplay for peer coding with me on a Sunday. Author: Reynold Xin Closes #1907 from rxin/netty and squashes the following commits: f921421 [Reynold Xin] Upgrade Netty to 4.0.22.Final to fix another Epoll bug. 4b174ca [Reynold Xin] Shivaram's code review comment. 4a3dfe7 [Reynold Xin] Switched to nio for default (instead of epoll on Linux). 56bfb9d [Reynold Xin] Bump Netty version to 4.0.21.Final for some bug fixes. b443a4b [Reynold Xin] Added debug message to help debug Jenkins failures. 57fc4d7 [Reynold Xin] Added test cases for BlockHeaderEncoder and BlockFetchingClientHandlerSuite. 22623e9 [Reynold Xin] Added exception handling and test case for BlockServerHandler and BlockFetchingClientHandler. 6550dd7 [Reynold Xin] Fixed block mgr init bug. 60c2edf [Reynold Xin] Beefed up server/client integration tests. 38d88d5 [Reynold Xin] Added missing test files. 6ce3f3c [Reynold Xin] Added some basic test cases. 47f7ce0 [Reynold Xin] Created server and client packages and moved files there. b16f412 [Reynold Xin] Added commit count. f13022d [Reynold Xin] Remove unused clone() in BlockFetcherIterator. c57d68c [Reynold Xin] Added back missing files. 842dfa7 [Reynold Xin] Made everything work with proper reference counting. 3fae001 [Reynold Xin] Connected the new netty network module with rest of Spark. 1a8f6d4 [Reynold Xin] Completed protocol documentation. 2951478 [Reynold Xin] New Netty implementation. cc7843d [Reynold Xin] Basic skeleton. --- .../spark/network/netty/FileClient.scala | 85 - .../network/netty/FileClientHandler.scala | 50 - .../spark/network/netty/FileHeader.scala | 71 - .../spark/network/netty/FileServer.scala | 91 -- .../network/netty/FileServerHandler.scala | 68 - .../spark/network/netty/NettyConfig.scala | 59 + .../spark/network/netty/ShuffleCopier.scala | 118 -- .../spark/network/netty/ShuffleSender.scala | 71 - .../netty/client/BlockFetchingClient.scala | 135 ++ .../client/BlockFetchingClientFactory.scala | 99 ++ .../client/BlockFetchingClientHandler.scala | 63 + .../netty/client/LazyInitIterator.scala | 44 + .../netty/client/ReferenceCountedBuffer.scala | 47 + .../network/netty/server/BlockHeader.scala | 32 + .../netty/server/BlockHeaderEncoder.scala | 47 + .../network/netty/server/BlockServer.scala | 162 ++ .../BlockServerChannelInitializer.scala} | 22 +- .../netty/server/BlockServerHandler.scala | 140 ++ .../BlockDataProvider.scala} | 21 +- .../spark/storage/BlockFetcherIterator.scala | 138 +- .../apache/spark/storage/BlockManager.scala | 49 +- .../storage/BlockNotFoundException.scala | 21 + .../spark/storage/DiskBlockManager.scala | 13 +- core/src/test/resources/netty-test-file.txt | 1379 +++++++++++++++++ .../netty/ServerClientIntegrationSuite.scala | 158 ++ .../BlockFetchingClientHandlerSuite.scala | 87 ++ .../server/BlockHeaderEncoderSuite.scala | 64 + .../server/BlockServerHandlerSuite.scala | 101 ++ pom.xml | 2 +- 29 files changed, 2770 insertions(+), 667 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClient.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala rename core/src/main/scala/org/apache/spark/network/netty/{FileServerChannelInitializer.scala => server/BlockServerChannelInitializer.scala} (58%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala rename core/src/main/scala/org/apache/spark/{network/netty/FileClientChannelInitializer.scala => storage/BlockDataProvider.scala} (65%) create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala create mode 100644 core/src/test/resources/netty-test-file.txt create mode 100644 core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala deleted file mode 100644 index c6d35f73db545..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.util.concurrent.TimeUnit - -import io.netty.bootstrap.Bootstrap -import io.netty.channel.{Channel, ChannelOption, EventLoopGroup} -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.oio.OioSocketChannel - -import org.apache.spark.Logging - -class FileClient(handler: FileClientHandler, connectTimeout: Int) extends Logging { - - private var channel: Channel = _ - private var bootstrap: Bootstrap = _ - private var group: EventLoopGroup = _ - private val sendTimeout = 60 - - def init(): Unit = { - group = new OioEventLoopGroup - bootstrap = new Bootstrap - bootstrap.group(group) - .channel(classOf[OioSocketChannel]) - .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) - .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, Integer.valueOf(connectTimeout)) - .handler(new FileClientChannelInitializer(handler)) - } - - def connect(host: String, port: Int) { - try { - channel = bootstrap.connect(host, port).sync().channel() - } catch { - case e: InterruptedException => - logWarning("FileClient interrupted while trying to connect", e) - close() - } - } - - def waitForClose(): Unit = { - try { - channel.closeFuture.sync() - } catch { - case e: InterruptedException => - logWarning("FileClient interrupted", e) - } - } - - def sendRequest(file: String): Unit = { - try { - val bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS) - if (!bSent) { - throw new RuntimeException("Failed to send") - } - } catch { - case e: InterruptedException => - logError("Error", e) - } - } - - def close(): Unit = { - if (group != null) { - group.shutdownGracefully() - group = null - bootstrap = null - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala deleted file mode 100644 index 017302ec7d33d..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import io.netty.buffer.ByteBuf -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.storage.BlockId - - -abstract class FileClientHandler extends SimpleChannelInboundHandler[ByteBuf] { - - private var currentHeader: FileHeader = null - - @volatile - private var handlerCalled: Boolean = false - - def isComplete: Boolean = handlerCalled - - def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) - - def handleError(blockId: BlockId) - - override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { - if (currentHeader == null && in.readableBytes >= FileHeader.HEADER_SIZE) { - currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE)) - } - if (in.readableBytes >= currentHeader.fileLen) { - handle(ctx, in, currentHeader) - handlerCalled = true - currentHeader = null - ctx.close() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala deleted file mode 100644 index 607e560ff277f..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import io.netty.buffer._ - -import org.apache.spark.Logging -import org.apache.spark.storage.{BlockId, TestBlockId} - -private[spark] class FileHeader ( - val fileLen: Int, - val blockId: BlockId) extends Logging { - - lazy val buffer: ByteBuf = { - val buf = Unpooled.buffer() - buf.capacity(FileHeader.HEADER_SIZE) - buf.writeInt(fileLen) - buf.writeInt(blockId.name.length) - blockId.name.foreach((x: Char) => buf.writeByte(x)) - // padding the rest of header - if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { - buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) - } else { - throw new Exception("too long header " + buf.readableBytes) - logInfo("too long header") - } - buf - } - -} - -private[spark] object FileHeader { - - val HEADER_SIZE = 40 - - def getFileLenOffset = 0 - def getFileLenSize = Integer.SIZE/8 - - def create(buf: ByteBuf): FileHeader = { - val length = buf.readInt - val idLength = buf.readInt - val idBuilder = new StringBuilder(idLength) - for (i <- 1 to idLength) { - idBuilder += buf.readByte().asInstanceOf[Char] - } - val blockId = BlockId(idBuilder.toString()) - new FileHeader(length, blockId) - } - - def main(args:Array[String]) { - val header = new FileHeader(25, TestBlockId("my_block")) - val buf = header.buffer - val newHeader = FileHeader.create(buf) - System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala deleted file mode 100644 index dff77950659af..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.net.InetSocketAddress - -import io.netty.bootstrap.ServerBootstrap -import io.netty.channel.{ChannelFuture, ChannelOption, EventLoopGroup} -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.oio.OioServerSocketChannel - -import org.apache.spark.Logging - -/** - * Server that accept the path of a file an echo back its content. - */ -class FileServer(pResolver: PathResolver, private var port: Int) extends Logging { - - private val addr: InetSocketAddress = new InetSocketAddress(port) - private var bossGroup: EventLoopGroup = new OioEventLoopGroup - private var workerGroup: EventLoopGroup = new OioEventLoopGroup - - private var channelFuture: ChannelFuture = { - val bootstrap = new ServerBootstrap - bootstrap.group(bossGroup, workerGroup) - .channel(classOf[OioServerSocketChannel]) - .option(ChannelOption.SO_BACKLOG, java.lang.Integer.valueOf(100)) - .option(ChannelOption.SO_RCVBUF, java.lang.Integer.valueOf(1500)) - .childHandler(new FileServerChannelInitializer(pResolver)) - bootstrap.bind(addr) - } - - try { - val boundAddress = channelFuture.sync.channel.localAddress.asInstanceOf[InetSocketAddress] - port = boundAddress.getPort - } catch { - case ie: InterruptedException => - port = 0 - } - - /** Start the file server asynchronously in a new thread. */ - def start(): Unit = { - val blockingThread: Thread = new Thread { - override def run(): Unit = { - try { - channelFuture.channel.closeFuture.sync - logInfo("FileServer exiting") - } catch { - case e: InterruptedException => - logError("File server start got interrupted", e) - } - // NOTE: bootstrap is shutdown in stop() - } - } - blockingThread.setDaemon(true) - blockingThread.start() - } - - def getPort: Int = port - - def stop(): Unit = { - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly() - channelFuture = null - } - if (bossGroup != null) { - bossGroup.shutdownGracefully() - bossGroup = null - } - if (workerGroup != null) { - workerGroup.shutdownGracefully() - workerGroup = null - } - } -} - diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala deleted file mode 100644 index 96f60b2883ad9..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.FileInputStream - -import io.netty.channel.{DefaultFileRegion, ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.Logging -import org.apache.spark.storage.{BlockId, FileSegment} - - -class FileServerHandler(pResolver: PathResolver) - extends SimpleChannelInboundHandler[String] with Logging { - - override def channelRead0(ctx: ChannelHandlerContext, blockIdString: String): Unit = { - val blockId: BlockId = BlockId(blockIdString) - val fileSegment: FileSegment = pResolver.getBlockLocation(blockId) - if (fileSegment == null) { - return - } - val file = fileSegment.file - if (file.exists) { - if (!file.isFile) { - ctx.write(new FileHeader(0, blockId).buffer) - ctx.flush() - return - } - val length: Long = fileSegment.length - if (length > Integer.MAX_VALUE || length <= 0) { - ctx.write(new FileHeader(0, blockId).buffer) - ctx.flush() - return - } - ctx.write(new FileHeader(length.toInt, blockId).buffer) - try { - val channel = new FileInputStream(file).getChannel - ctx.write(new DefaultFileRegion(channel, fileSegment.offset, fileSegment.length)) - } catch { - case e: Exception => - logError("Exception: ", e) - } - } else { - ctx.write(new FileHeader(0, blockId).buffer) - } - ctx.flush() - } - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - logError("Exception: ", cause) - ctx.close() - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala new file mode 100644 index 0000000000000..b5870152c5a64 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import org.apache.spark.SparkConf + +/** + * A central location that tracks all the settings we exposed to users. + */ +private[spark] +class NettyConfig(conf: SparkConf) { + + /** Port the server listens on. Default to a random port. */ + private[netty] val serverPort = conf.getInt("spark.shuffle.io.port", 0) + + /** IO mode: nio, oio, epoll, or auto (try epoll first and then nio). */ + private[netty] val ioMode = conf.get("spark.shuffle.io.mode", "nio").toLowerCase + + /** Connect timeout in secs. Default 60 secs. */ + private[netty] val connectTimeoutMs = conf.getInt("spark.shuffle.io.connectionTimeout", 60) * 1000 + + /** + * Percentage of the desired amount of time spent for I/O in the child event loops. + * Only applicable in nio and epoll. + */ + private[netty] val ioRatio = conf.getInt("spark.shuffle.io.netty.ioRatio", 80) + + /** Requested maximum length of the queue of incoming connections. */ + private[netty] val backLog: Option[Int] = conf.getOption("spark.shuffle.io.backLog").map(_.toInt) + + /** + * Receive buffer size (SO_RCVBUF). + * Note: the optimal size for receive buffer and send buffer should be + * latency * network_bandwidth. + * Assuming latency = 1ms, network_bandwidth = 10Gbps + * buffer size should be ~ 1.25MB + */ + private[netty] val receiveBuf: Option[Int] = + conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) + + /** Send buffer size (SO_SNDBUF). */ + private[netty] val sendBuf: Option[Int] = + conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala deleted file mode 100644 index e7b2855e1ec91..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.util.concurrent.Executors - -import scala.collection.JavaConverters._ - -import io.netty.buffer.ByteBuf -import io.netty.channel.ChannelHandlerContext -import io.netty.util.CharsetUtil - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.storage.BlockId - -private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { - - def getBlock(host: String, port: Int, blockId: BlockId, - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - - val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.getInt("spark.shuffle.netty.connect.timeout", 60000) - val fc = new FileClient(handler, connectTimeout) - - try { - fc.init() - fc.connect(host, port) - fc.sendRequest(blockId.name) - fc.waitForClose() - fc.close() - } catch { - // Handle any socket-related exceptions in FileClient - case e: Exception => { - logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + " failed", e) - handler.handleError(blockId) - } - } - } - - def getBlock(cmId: ConnectionManagerId, blockId: BlockId, - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - getBlock(cmId.host, cmId.port, blockId, resultCollectCallback) - } - - def getBlocks(cmId: ConnectionManagerId, - blocks: Seq[(BlockId, Long)], - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - - for ((blockId, size) <- blocks) { - getBlock(cmId, blockId, resultCollectCallback) - } - } -} - - -private[spark] object ShuffleCopier extends Logging { - - private class ShuffleClientHandler(resultCollectCallBack: (BlockId, Long, ByteBuf) => Unit) - extends FileClientHandler with Logging { - - override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { - logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)") - resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) - } - - override def handleError(blockId: BlockId) { - if (!isComplete) { - resultCollectCallBack(blockId, -1, null) - } - } - } - - def echoResultCollectCallBack(blockId: BlockId, size: Long, content: ByteBuf) { - if (size != -1) { - logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") - } - } - - def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: ShuffleCopier ") - System.exit(1) - } - val host = args(0) - val port = args(1).toInt - val blockId = BlockId(args(2)) - val threads = if (args.length > 3) args(3).toInt else 10 - - val copiers = Executors.newFixedThreadPool(80) - val tasks = (for (i <- Range(0, threads)) yield { - Executors.callable(new Runnable() { - def run() { - val copier = new ShuffleCopier(new SparkConf) - copier.getBlock(host, port, blockId, echoResultCollectCallBack) - } - }) - }).asJava - copiers.invokeAll(tasks) - copiers.shutdown() - System.exit(0) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala deleted file mode 100644 index 95958e30f7eeb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.File - -import org.apache.spark.Logging -import org.apache.spark.util.Utils -import org.apache.spark.storage.{BlockId, FileSegment} - -private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { - - val server = new FileServer(pResolver, portIn) - server.start() - - def stop() { - server.stop() - } - - def port: Int = server.getPort -} - - -/** - * An application for testing the shuffle sender as a standalone program. - */ -private[spark] object ShuffleSender { - - def main(args: Array[String]) { - if (args.length < 3) { - System.err.println( - "Usage: ShuffleSender ") - System.exit(1) - } - - val port = args(0).toInt - val subDirsPerLocalDir = args(1).toInt - val localDirs = args.drop(2).map(new File(_)) - - val pResovler = new PathResolver { - override def getBlockLocation(blockId: BlockId): FileSegment = { - if (!blockId.isShuffle) { - throw new Exception("Block " + blockId + " is not a shuffle block") - } - // Figure out which local directory it hashes to, and which subdirectory in that - val hash = Utils.nonNegativeHash(blockId) - val dirId = hash % localDirs.length - val subDirId = (hash / localDirs.length) % subDirsPerLocalDir - val subDir = new File(localDirs(dirId), "%02x".format(subDirId)) - val file = new File(subDir, blockId.name) - new FileSegment(file, 0, file.length()) - } - } - val sender = new ShuffleSender(port, pResovler) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala new file mode 100644 index 0000000000000..9fed11b75c342 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.util.concurrent.TimeoutException + +import io.netty.bootstrap.Bootstrap +import io.netty.buffer.PooledByteBufAllocator +import io.netty.channel.socket.SocketChannel +import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} +import io.netty.handler.codec.LengthFieldBasedFrameDecoder +import io.netty.handler.codec.string.StringEncoder +import io.netty.util.CharsetUtil + +import org.apache.spark.Logging + +/** + * Client for fetching data blocks from [[org.apache.spark.network.netty.server.BlockServer]]. + * Use [[BlockFetchingClientFactory]] to instantiate this client. + * + * The constructor blocks until a connection is successfully established. + * + * See [[org.apache.spark.network.netty.server.BlockServer]] for client/server protocol. + * + * Concurrency: [[BlockFetchingClient]] is not thread safe and should not be shared. + */ +@throws[TimeoutException] +private[spark] +class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, port: Int) + extends Logging { + + val handler = new BlockFetchingClientHandler + + /** Netty Bootstrap for creating the TCP connection. */ + private val bootstrap: Bootstrap = { + val b = new Bootstrap + b.group(factory.workerGroup) + .channel(factory.socketChannelClass) + // Use pooled buffers to reduce temporary buffer allocation + .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + // Disable Nagle's Algorithm since we don't want packets to wait + .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) + .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) + .option[Integer](ChannelOption.CONNECT_TIMEOUT_MILLIS, factory.conf.connectTimeoutMs) + + b.handler(new ChannelInitializer[SocketChannel] { + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("encoder", new StringEncoder(CharsetUtil.UTF_8)) + // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 + .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) + .addLast("handler", handler) + } + }) + b + } + + /** Netty ChannelFuture for the connection. */ + private val cf: ChannelFuture = bootstrap.connect(hostname, port) + if (!cf.awaitUninterruptibly(factory.conf.connectTimeoutMs)) { + throw new TimeoutException( + s"Connecting to $hostname:$port timed out (${factory.conf.connectTimeoutMs} ms)") + } + + /** + * Ask the remote server for a sequence of blocks, and execute the callback. + * + * Note that this is asynchronous and returns immediately. Upstream caller should throttle the + * rate of fetching; otherwise we could run out of memory. + * + * @param blockIds sequence of block ids to fetch. + * @param blockFetchSuccessCallback callback function when a block is successfully fetched. + * First argument is the block id, and second argument is the + * raw data in a ByteBuffer. + * @param blockFetchFailureCallback callback function when we failed to fetch any of the blocks. + * First argument is the block id, and second argument is the + * error message. + */ + def fetchBlocks( + blockIds: Seq[String], + blockFetchSuccessCallback: (String, ReferenceCountedBuffer) => Unit, + blockFetchFailureCallback: (String, String) => Unit): Unit = { + // It's best to limit the number of "write" calls since it needs to traverse the whole pipeline. + // It's also best to limit the number of "flush" calls since it requires system calls. + // Let's concatenate the string and then call writeAndFlush once. + // This is also why this implementation might be more efficient than multiple, separate + // fetch block calls. + var startTime: Long = 0 + logTrace { + startTime = System.nanoTime + s"Sending request $blockIds to $hostname:$port" + } + + // TODO: This is not the most elegant way to handle this ... + handler.blockFetchSuccessCallback = blockFetchSuccessCallback + handler.blockFetchFailureCallback = blockFetchFailureCallback + + val writeFuture = cf.channel().writeAndFlush(blockIds.mkString("\n") + "\n") + writeFuture.addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture): Unit = { + if (future.isSuccess) { + logTrace { + val timeTaken = (System.nanoTime - startTime).toDouble / 1000000 + s"Sending request $blockIds to $hostname:$port took $timeTaken ms" + } + } else { + // Fail all blocks. + logError(s"Failed to send request $blockIds to $hostname:$port", future.cause) + blockIds.foreach(blockFetchFailureCallback(_, future.cause.getMessage)) + } + } + }) + } + + def waitForClose(): Unit = { + cf.channel().closeFuture().sync() + } + + def close(): Unit = cf.channel().close() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala new file mode 100644 index 0000000000000..2b28402c52b49 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import io.netty.channel.epoll.{EpollEventLoopGroup, EpollSocketChannel} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.nio.NioSocketChannel +import io.netty.channel.socket.oio.OioSocketChannel +import io.netty.channel.{EventLoopGroup, Channel} + +import org.apache.spark.SparkConf +import org.apache.spark.network.netty.NettyConfig +import org.apache.spark.util.Utils + +/** + * Factory for creating [[BlockFetchingClient]] by using createClient. This factory reuses + * the worker thread pool for Netty. + * + * Concurrency: createClient is safe to be called from multiple threads concurrently. + */ +private[spark] +class BlockFetchingClientFactory(val conf: NettyConfig) { + + def this(sparkConf: SparkConf) = this(new NettyConfig(sparkConf)) + + /** A thread factory so the threads are named (for debugging). */ + val threadFactory = Utils.namedThreadFactory("spark-shuffle-client") + + /** The following two are instantiated by the [[init]] method, depending ioMode. */ + var socketChannelClass: Class[_ <: Channel] = _ + var workerGroup: EventLoopGroup = _ + + init() + + /** Initialize [[socketChannelClass]] and [[workerGroup]] based on ioMode. */ + private def init(): Unit = { + def initOio(): Unit = { + socketChannelClass = classOf[OioSocketChannel] + workerGroup = new OioEventLoopGroup(0, threadFactory) + } + def initNio(): Unit = { + socketChannelClass = classOf[NioSocketChannel] + workerGroup = new NioEventLoopGroup(0, threadFactory) + } + def initEpoll(): Unit = { + socketChannelClass = classOf[EpollSocketChannel] + workerGroup = new EpollEventLoopGroup(0, threadFactory) + } + + conf.ioMode match { + case "nio" => initNio() + case "oio" => initOio() + case "epoll" => initEpoll() + case "auto" => + // For auto mode, first try epoll (only available on Linux), then nio. + try { + initEpoll() + } catch { + // TODO: Should we log the throwable? But that always happen on non-Linux systems. + // Perhaps the right thing to do is to check whether the system is Linux, and then only + // call initEpoll on Linux. + case e: Throwable => initNio() + } + } + } + + /** + * Create a new BlockFetchingClient connecting to the given remote host / port. + * + * This blocks until a connection is successfully established. + * + * Concurrency: This method is safe to call from multiple threads. + */ + def createClient(remoteHost: String, remotePort: Int): BlockFetchingClient = { + new BlockFetchingClient(this, remoteHost, remotePort) + } + + def stop(): Unit = { + if (workerGroup != null) { + workerGroup.shutdownGracefully() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala new file mode 100644 index 0000000000000..a1dbf6102c080 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import io.netty.buffer.ByteBuf +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging + + +/** + * Handler that processes server responses. It uses the protocol documented in + * [[org.apache.spark.network.netty.server.BlockServer]]. + */ +private[client] +class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] with Logging { + + var blockFetchSuccessCallback: (String, ReferenceCountedBuffer) => Unit = _ + var blockFetchFailureCallback: (String, String) => Unit = _ + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) + ctx.close() + } + + override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { + val totalLen = in.readInt() + val blockIdLen = in.readInt() + val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) + in.readBytes(blockIdBytes) + val blockId = new String(blockIdBytes) + val blockSize = totalLen - math.abs(blockIdLen) - 4 + + def server = ctx.channel.remoteAddress.toString + + // blockIdLen is negative when it is an error message. + if (blockIdLen < 0) { + val errorMessageBytes = new Array[Byte](blockSize) + in.readBytes(errorMessageBytes) + val errorMsg = new String(errorMessageBytes) + logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") + blockFetchFailureCallback(blockId, errorMsg) + } else { + logTrace(s"Received block $blockId ($blockSize B) from $server") + blockFetchSuccessCallback(blockId, new ReferenceCountedBuffer(in)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala new file mode 100644 index 0000000000000..9740ee64d1f2d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +/** + * A simple iterator that lazily initializes the underlying iterator. + * + * The use case is that sometimes we might have many iterators open at the same time, and each of + * the iterator might initialize its own buffer (e.g. decompression buffer, deserialization buffer). + * This could lead to too many buffers open. If this iterator is used, we lazily initialize those + * buffers. + */ +private[spark] +class LazyInitIterator(createIterator: => Iterator[Any]) extends Iterator[Any] { + + lazy val proxy = createIterator + + override def hasNext: Boolean = { + val gotNext = proxy.hasNext + if (!gotNext) { + close() + } + gotNext + } + + override def next(): Any = proxy.next() + + def close(): Unit = Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala new file mode 100644 index 0000000000000..ea1abf5eccc26 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.io.InputStream +import java.nio.ByteBuffer + +import io.netty.buffer.{ByteBuf, ByteBufInputStream} + + +/** + * A buffer abstraction based on Netty's ByteBuf so we don't expose Netty. + * This is a Scala value class. + * + * The buffer's life cycle is NOT managed by the JVM, and thus requiring explicit declaration of + * reference by the retain method and release method. + */ +private[spark] +class ReferenceCountedBuffer(val underlying: ByteBuf) extends AnyVal { + + /** Return the nio ByteBuffer view of the underlying buffer. */ + def byteBuffer(): ByteBuffer = underlying.nioBuffer + + /** Creates a new input stream that starts from the current position of the buffer. */ + def inputStream(): InputStream = new ByteBufInputStream(underlying) + + /** Increment the reference counter by one. */ + def retain(): Unit = underlying.retain() + + /** Decrement the reference counter by one and release the buffer if the ref count is 0. */ + def release(): Unit = underlying.release() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala new file mode 100644 index 0000000000000..162e9cc6828d4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +/** + * Header describing a block. This is used only in the server pipeline. + * + * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it. + * + * @param blockSize length of the block content, excluding the length itself. + * If positive, this is the header for a block (not part of the header). + * If negative, this is the header and content for an error message. + * @param blockId block id + * @param error some error message from reading the block + */ +private[server] +class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None) diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala new file mode 100644 index 0000000000000..8e4dda4ef8595 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import io.netty.buffer.ByteBuf +import io.netty.channel.ChannelHandlerContext +import io.netty.handler.codec.MessageToByteEncoder + +/** + * A simple encoder for BlockHeader. See [[BlockServer]] for the server to client protocol. + */ +private[server] +class BlockHeaderEncoder extends MessageToByteEncoder[BlockHeader] { + override def encode(ctx: ChannelHandlerContext, msg: BlockHeader, out: ByteBuf): Unit = { + // message = message length (4 bytes) + block id length (4 bytes) + block id + block data + // message length = block id length (4 bytes) + size of block id + size of block data + val blockIdBytes = msg.blockId.getBytes + msg.error match { + case Some(errorMsg) => + val errorBytes = errorMsg.getBytes + out.writeInt(4 + blockIdBytes.length + errorBytes.size) + out.writeInt(-blockIdBytes.length) // use negative block id length to represent errors + out.writeBytes(blockIdBytes) // next is blockId itself + out.writeBytes(errorBytes) // error message + case None => + out.writeInt(4 + blockIdBytes.length + msg.blockSize) + out.writeInt(blockIdBytes.length) // First 4 bytes is blockId length + out.writeBytes(blockIdBytes) // next is blockId itself + // msg of size blockSize will be written by ServerHandler + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala new file mode 100644 index 0000000000000..7b2f9a8d4dfd0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.net.InetSocketAddress + +import io.netty.bootstrap.ServerBootstrap +import io.netty.buffer.PooledByteBufAllocator +import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} +import io.netty.channel.epoll.{EpollEventLoopGroup, EpollServerSocketChannel} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.SocketChannel +import io.netty.channel.socket.nio.NioServerSocketChannel +import io.netty.channel.socket.oio.OioServerSocketChannel +import io.netty.handler.codec.LineBasedFrameDecoder +import io.netty.handler.codec.string.StringDecoder +import io.netty.util.CharsetUtil + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.network.netty.NettyConfig +import org.apache.spark.storage.BlockDataProvider +import org.apache.spark.util.Utils + + +/** + * Server for serving Spark data blocks. + * This should be used together with [[org.apache.spark.network.netty.client.BlockFetchingClient]]. + * + * Protocol for requesting blocks (client to server): + * One block id per line, e.g. to request 3 blocks: "block1\nblock2\nblock3\n" + * + * Protocol for sending blocks (server to client): + * frame-length (4 bytes), block-id-length (4 bytes), block-id, block-data. + * + * frame-length should not include the length of itself. + * If block-id-length is negative, then this is an error message rather than block-data. The real + * length is the absolute value of the frame-length. + * + */ +private[spark] +class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Logging { + + def this(sparkConf: SparkConf, dataProvider: BlockDataProvider) = { + this(new NettyConfig(sparkConf), dataProvider) + } + + def port: Int = _port + + def hostName: String = _hostName + + private var _port: Int = conf.serverPort + private var _hostName: String = "" + private var bootstrap: ServerBootstrap = _ + private var channelFuture: ChannelFuture = _ + + init() + + /** Initialize the server. */ + private def init(): Unit = { + bootstrap = new ServerBootstrap + val bossThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-boss") + val workerThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-worker") + + // Use only one thread to accept connections, and 2 * num_cores for worker. + def initNio(): Unit = { + val bossGroup = new NioEventLoopGroup(1, bossThreadFactory) + val workerGroup = new NioEventLoopGroup(0, workerThreadFactory) + workerGroup.setIoRatio(conf.ioRatio) + bootstrap.group(bossGroup, workerGroup).channel(classOf[NioServerSocketChannel]) + } + def initOio(): Unit = { + val bossGroup = new OioEventLoopGroup(1, bossThreadFactory) + val workerGroup = new OioEventLoopGroup(0, workerThreadFactory) + bootstrap.group(bossGroup, workerGroup).channel(classOf[OioServerSocketChannel]) + } + def initEpoll(): Unit = { + val bossGroup = new EpollEventLoopGroup(1, bossThreadFactory) + val workerGroup = new EpollEventLoopGroup(0, workerThreadFactory) + workerGroup.setIoRatio(conf.ioRatio) + bootstrap.group(bossGroup, workerGroup).channel(classOf[EpollServerSocketChannel]) + } + + conf.ioMode match { + case "nio" => initNio() + case "oio" => initOio() + case "epoll" => initEpoll() + case "auto" => + // For auto mode, first try epoll (only available on Linux), then nio. + try { + initEpoll() + } catch { + // TODO: Should we log the throwable? But that always happen on non-Linux systems. + // Perhaps the right thing to do is to check whether the system is Linux, and then only + // call initEpoll on Linux. + case e: Throwable => initNio() + } + } + + // Use pooled buffers to reduce temporary buffer allocation + bootstrap.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + + // Various (advanced) user-configured settings. + conf.backLog.foreach { backLog => + bootstrap.option[java.lang.Integer](ChannelOption.SO_BACKLOG, backLog) + } + conf.receiveBuf.foreach { receiveBuf => + bootstrap.option[java.lang.Integer](ChannelOption.SO_RCVBUF, receiveBuf) + } + conf.sendBuf.foreach { sendBuf => + bootstrap.option[java.lang.Integer](ChannelOption.SO_SNDBUF, sendBuf) + } + + bootstrap.childHandler(new ChannelInitializer[SocketChannel] { + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 + .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("blockHeaderEncoder", new BlockHeaderEncoder) + .addLast("handler", new BlockServerHandler(dataProvider)) + } + }) + + channelFuture = bootstrap.bind(new InetSocketAddress(_port)) + channelFuture.sync() + + val addr = channelFuture.channel.localAddress.asInstanceOf[InetSocketAddress] + _port = addr.getPort + _hostName = addr.getHostName + } + + /** Shutdown the server. */ + def stop(): Unit = { + if (channelFuture != null) { + channelFuture.channel().close().awaitUninterruptibly() + channelFuture = null + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully() + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully() + } + bootstrap = null + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala similarity index 58% rename from core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala rename to core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala index aaa2f913d0269..cc70bd0c5c477 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala @@ -15,20 +15,26 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.network.netty.server import io.netty.channel.ChannelInitializer import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.{DelimiterBasedFrameDecoder, Delimiters} +import io.netty.handler.codec.LineBasedFrameDecoder import io.netty.handler.codec.string.StringDecoder +import io.netty.util.CharsetUtil +import org.apache.spark.storage.BlockDataProvider -class FileServerChannelInitializer(pResolver: PathResolver) + +/** Channel initializer that sets up the pipeline for the BlockServer. */ +private[netty] +class BlockServerChannelInitializer(dataProvider: BlockDataProvider) extends ChannelInitializer[SocketChannel] { - override def initChannel(channel: SocketChannel): Unit = { - channel.pipeline - .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter : _*)) - .addLast("stringDecoder", new StringDecoder) - .addLast("handler", new FileServerHandler(pResolver)) + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 + .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("blockHeaderEncoder", new BlockHeaderEncoder) + .addLast("handler", new BlockServerHandler(dataProvider)) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala new file mode 100644 index 0000000000000..40dd5e5d1a2ac --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.io.FileInputStream +import java.nio.ByteBuffer +import java.nio.channels.FileChannel + +import io.netty.buffer.Unpooled +import io.netty.channel._ + +import org.apache.spark.Logging +import org.apache.spark.storage.{FileSegment, BlockDataProvider} + + +/** + * A handler that processes requests from clients and writes block data back. + * + * The messages should have been processed by a LineBasedFrameDecoder and a StringDecoder first + * so channelRead0 is called once per line (i.e. per block id). + */ +private[server] +class BlockServerHandler(dataProvider: BlockDataProvider) + extends SimpleChannelInboundHandler[String] with Logging { + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) + ctx.close() + } + + override def channelRead0(ctx: ChannelHandlerContext, blockId: String): Unit = { + def client = ctx.channel.remoteAddress.toString + + // A helper function to send error message back to the client. + def respondWithError(error: String): Unit = { + ctx.writeAndFlush(new BlockHeader(-1, blockId, Some(error))).addListener( + new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (!future.isSuccess) { + // TODO: Maybe log the success case as well. + logError(s"Error sending error back to $client", future.cause) + ctx.close() + } + } + } + ) + } + + def writeFileSegment(segment: FileSegment): Unit = { + // Send error message back if the block is too large. Even though we are capable of sending + // large (2G+) blocks, the receiving end cannot handle it so let's fail fast. + // Once we fixed the receiving end to be able to process large blocks, this should be removed. + // Also make sure we update BlockHeaderEncoder to support length > 2G. + + // See [[BlockHeaderEncoder]] for the way length is encoded. + if (segment.length + blockId.length + 4 > Int.MaxValue) { + respondWithError(s"Block $blockId size ($segment.length) greater than 2G") + return + } + + var fileChannel: FileChannel = null + try { + fileChannel = new FileInputStream(segment.file).getChannel + } catch { + case e: Exception => + logError( + s"Error opening channel for $blockId in ${segment.file} for request from $client", e) + respondWithError(e.getMessage) + } + + // Found the block. Send it back. + if (fileChannel != null) { + // Write the header and block data. In the case of failures, the listener on the block data + // write should close the connection. + ctx.write(new BlockHeader(segment.length.toInt, blockId)) + + val region = new DefaultFileRegion(fileChannel, segment.offset, segment.length) + ctx.writeAndFlush(region).addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (future.isSuccess) { + logTrace(s"Sent block $blockId (${segment.length} B) back to $client") + } else { + logError(s"Error sending block $blockId to $client; closing connection", future.cause) + ctx.close() + } + } + }) + } + } + + def writeByteBuffer(buf: ByteBuffer): Unit = { + ctx.write(new BlockHeader(buf.remaining, blockId)) + ctx.writeAndFlush(Unpooled.wrappedBuffer(buf)).addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (future.isSuccess) { + logTrace(s"Sent block $blockId (${buf.remaining} B) back to $client") + } else { + logError(s"Error sending block $blockId to $client; closing connection", future.cause) + ctx.close() + } + } + }) + } + + logTrace(s"Received request from $client to fetch block $blockId") + + var blockData: Either[FileSegment, ByteBuffer] = null + + // First make sure we can find the block. If not, send error back to the user. + try { + blockData = dataProvider.getBlockData(blockId) + } catch { + case e: Exception => + logError(s"Error opening block $blockId for request from $client", e) + respondWithError(e.getMessage) + return + } + + blockData match { + case Left(segment) => writeFileSegment(segment) + case Right(buf) => writeByteBuffer(buf) + } + + } // end of channelRead0 +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala b/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala similarity index 65% rename from core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala rename to core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala index f4261c13f70a8..5b6d086630834 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala @@ -15,17 +15,18 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.storage -import io.netty.channel.ChannelInitializer -import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.string.StringEncoder +import java.nio.ByteBuffer -class FileClientChannelInitializer(handler: FileClientHandler) - extends ChannelInitializer[SocketChannel] { - - def initChannel(channel: SocketChannel) { - channel.pipeline.addLast("encoder", new StringEncoder).addLast("handler", handler) - } +/** + * An interface for providing data for blocks. + * + * getBlockData returns either a FileSegment (for zero-copy send), or a ByteBuffer. + * + * Aside from unit tests, [[BlockManager]] is the main class that implements this. + */ +private[spark] trait BlockDataProvider { + def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 5f44f5f3197fd..91c0f47d51d02 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -18,19 +18,17 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue +import org.apache.spark.network.netty.client.{LazyInitIterator, ReferenceCountedBuffer} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue import scala.util.{Failure, Success} -import io.netty.buffer.ByteBuf - import org.apache.spark.{Logging, SparkException} import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.network.netty.ShuffleCopier import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -54,18 +52,28 @@ trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] wi private[storage] object BlockFetcherIterator { - // A request to fetch one or more blocks, complete with their sizes + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of tuple, where the first element is the block id, + * and the second element is the estimated size, used to calculate bytesInFlight. + */ class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { val size = blocks.map(_._2).sum } - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. + /** + * Result of a fetch from a remote block. A failure is represented as size == -1. + * @param blockId block id + * @param size estimated size of the block, used to calculate bytesInFlight. + * Note that this is NOT the exact bytes. + * @param deserialize closure to return the result in the form of an Iterator. + */ class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { def failed: Boolean = size == -1 } + // TODO: Refactor this whole thing to make code more reusable. class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], @@ -95,10 +103,10 @@ object BlockFetcherIterator { // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight - private val fetchRequests = new Queue[FetchRequest] + protected val fetchRequests = new Queue[FetchRequest] // Current bytes in flight from our requests - private var bytesInFlight = 0L + protected var bytesInFlight = 0L protected def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( @@ -262,77 +270,55 @@ object BlockFetcherIterator { readMetrics: ShuffleReadMetrics) extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer, readMetrics) { - import blockManager._ - - val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] - - private def startCopiers(numCopiers: Int): List[_ <: Thread] = { - (for ( i <- Range(0,numCopiers) ) yield { - val copier = new Thread { - override def run(){ - try { - while(!isInterrupted && !fetchRequestsSync.isEmpty) { - sendRequest(fetchRequestsSync.take()) - } - } catch { - case x: InterruptedException => logInfo("Copier Interrupted") - // case _ => throw new SparkException("Exception Throw in Shuffle Copier") - } - } - } - copier.start - copier - }).toList - } - - // keep this to interrupt the threads when necessary - private def stopCopiers() { - for (copier <- copiers) { - copier.interrupt() - } - } - override protected def sendRequest(req: FetchRequest) { - - def putResult(blockId: BlockId, blockSize: Long, blockData: ByteBuf) { - val fetchResult = new FetchResult(blockId, blockSize, - () => dataDeserialize(blockId, blockData.nioBuffer, serializer)) - results.put(fetchResult) - } - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.bytesToString(req.size), req.address.host)) - val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) - val cpier = new ShuffleCopier(blockManager.conf) - cpier.getBlocks(cmId, req.blocks, putResult) - logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) - } - - private var copiers: List[_ <: Thread] = null - - override def initialize() { - // Split Local Remote Blocks and set numBlocksToFetch - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - for (request <- Utils.randomize(remoteRequests)) { - fetchRequestsSync.put(request) - } - - copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) - logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + - Utils.getUsedTimeMs(startTime)) + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + val cmId = new ConnectionManagerId(req.address.host, req.address.port) - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } + bytesInFlight += req.size + val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + + // This could throw a TimeoutException. In that case we will just retry the task. + val client = blockManager.nettyBlockClientFactory.createClient( + cmId.host, req.address.nettyPort) + val blocks = req.blocks.map(_._1.toString) + + client.fetchBlocks( + blocks, + (blockId: String, refBuf: ReferenceCountedBuffer) => { + // Increment the reference count so the buffer won't be recycled. + // TODO: This could result in memory leaks when the task is stopped due to exception + // before the iterator is exhausted. + refBuf.retain() + val buf = refBuf.byteBuffer() + val blockSize = buf.remaining() + val bid = BlockId(blockId) + + // TODO: remove code duplication between here and BlockManager.dataDeserialization. + results.put(new FetchResult(bid, sizeMap(bid), () => { + def createIterator: Iterator[Any] = { + val stream = blockManager.wrapForCompression(bid, refBuf.inputStream()) + serializer.newInstance().deserializeStream(stream).asIterator + } + new LazyInitIterator(createIterator) { + // Release the buffer when we are done traversing it. + override def close(): Unit = refBuf.release() + } + })) - override def next(): (BlockId, Option[Iterator[Any]]) = { - resultsGotten += 1 - val result = results.take() - // If all the results has been retrieved, copiers will exit automatically - (result.blockId, if (result.failed) None else Some(result.deserialize())) + readMetrics.synchronized { + readMetrics.remoteBytesRead += blockSize + readMetrics.remoteBlocksFetched += 1 + } + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + }, + (blockId: String, errorMsg: String) => { + logError(s"Could not get block(s) from $cmId with error: $errorMsg") + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + ) } } // End of NettyBlockFetcherIterator diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e8bbd298c631a..e67676950b0ed 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -25,16 +25,19 @@ import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random -import akka.actor.{ActorSystem, Cancellable, Props} +import akka.actor.{ActorSystem, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ +import org.apache.spark.network.netty.client.BlockFetchingClientFactory +import org.apache.spark.network.netty.server.BlockServer import org.apache.spark.serializer.Serializer import org.apache.spark.util._ + private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues @@ -58,7 +61,7 @@ private[spark] class BlockManager( val conf: SparkConf, securityManager: SecurityManager, mapOutputTracker: MapOutputTracker) - extends Logging { + extends BlockDataProvider with Logging { private val port = conf.getInt("spark.blockManager.port", 0) val shuffleBlockManager = new ShuffleBlockManager(this) @@ -86,13 +89,25 @@ private[spark] class BlockManager( new TachyonStore(this, tachyonBlockManager) } + private val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) + // If we use Netty for shuffle, start a new Netty-based shuffle sender service. - private val nettyPort: Int = { - val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) - val nettyPortConfig = conf.getInt("spark.shuffle.sender.port", 0) - if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 + private[storage] val nettyBlockClientFactory: BlockFetchingClientFactory = { + if (useNetty) new BlockFetchingClientFactory(conf) else null } + private val nettyBlockServer: BlockServer = { + if (useNetty) { + val server = new BlockServer(conf, this) + logInfo(s"Created NettyBlockServer binding to port: ${server.port}") + server + } else { + null + } + } + + private val nettyPort: Int = if (useNetty) nettyBlockServer.port else 0 + val blockManagerId = BlockManagerId( executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) @@ -216,6 +231,20 @@ private[spark] class BlockManager( } } + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + val bid = BlockId(blockId) + if (bid.isShuffle) { + Left(diskBlockManager.getBlockLocation(bid)) + } else { + val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + if (blockBytesOpt.isDefined) { + Right(blockBytesOpt.get) + } else { + throw new BlockNotFoundException(blockId) + } + } + } + /** * Get the BlockStatus for the block identified by the given ID, if it exists. * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. @@ -1061,6 +1090,14 @@ private[spark] class BlockManager( connectionManager.stop() shuffleBlockManager.stop() diskBlockManager.stop() + + if (nettyBlockClientFactory != null) { + nettyBlockClientFactory.stop() + } + if (nettyBlockServer != null) { + nettyBlockServer.stop() + } + actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala new file mode 100644 index 0000000000000..9ef453605f4f1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + + +class BlockNotFoundException(blockId: String) extends Exception(s"Block $blockId not found") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 4d66ccea211fa..f3da816389581 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -23,7 +23,7 @@ import java.util.{Date, Random, UUID} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.{PathResolver, ShuffleSender} +import org.apache.spark.network.netty.PathResolver import org.apache.spark.util.Utils import org.apache.spark.shuffle.sort.SortShuffleManager @@ -52,7 +52,6 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) - private var shuffleSender : ShuffleSender = null addShutdownHook() @@ -186,15 +185,5 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, } } } - - if (shuffleSender != null) { - shuffleSender.stop() - } - } - - private[storage] def startShuffleBlockSender(port: Int): Int = { - shuffleSender = new ShuffleSender(port, this) - logInfo(s"Created ShuffleSender binding to port: ${shuffleSender.port}") - shuffleSender.port } } diff --git a/core/src/test/resources/netty-test-file.txt b/core/src/test/resources/netty-test-file.txt new file mode 100644 index 0000000000000..f59f293ee02ea --- /dev/null +++ b/core/src/test/resources/netty-test-file.txt @@ -0,0 +1,1379 @@ +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa +bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb +eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee +aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala new file mode 100644 index 0000000000000..ef3478a41e912 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.io.{RandomAccessFile, File} +import java.nio.ByteBuffer +import java.util.{Collections, HashSet} +import java.util.concurrent.{TimeUnit, Semaphore} + +import scala.collection.JavaConversions._ + +import io.netty.buffer.{ByteBufUtil, Unpooled} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.network.netty.client.{ReferenceCountedBuffer, BlockFetchingClientFactory} +import org.apache.spark.network.netty.server.BlockServer +import org.apache.spark.storage.{FileSegment, BlockDataProvider} + + +/** + * Test suite that makes sure the server and the client implementations share the same protocol. + */ +class ServerClientIntegrationSuite extends FunSuite with BeforeAndAfterAll { + + val bufSize = 100000 + var buf: ByteBuffer = _ + var testFile: File = _ + var server: BlockServer = _ + var clientFactory: BlockFetchingClientFactory = _ + + val bufferBlockId = "buffer_block" + val fileBlockId = "file_block" + + val fileContent = new Array[Byte](1024) + scala.util.Random.nextBytes(fileContent) + + override def beforeAll() = { + buf = ByteBuffer.allocate(bufSize) + for (i <- 1 to bufSize) { + buf.put(i.toByte) + } + buf.flip() + + testFile = File.createTempFile("netty-test-file", "txt") + val fp = new RandomAccessFile(testFile, "rw") + fp.write(fileContent) + fp.close() + + server = new BlockServer(new SparkConf, new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + if (blockId == bufferBlockId) { + Right(buf) + } else if (blockId == fileBlockId) { + Left(new FileSegment(testFile, 10, testFile.length - 25)) + } else { + throw new Exception("Unknown block id " + blockId) + } + } + }) + + clientFactory = new BlockFetchingClientFactory(new SparkConf) + } + + override def afterAll() = { + server.stop() + clientFactory.stop() + } + + /** A ByteBuf for buffer_block */ + lazy val byteBufferBlockReference = Unpooled.wrappedBuffer(buf) + + /** A ByteBuf for file_block */ + lazy val fileBlockReference = Unpooled.wrappedBuffer(fileContent, 10, fileContent.length - 25) + + def fetchBlocks(blockIds: Seq[String]): (Set[String], Set[ReferenceCountedBuffer], Set[String]) = + { + val client = clientFactory.createClient(server.hostName, server.port) + val sem = new Semaphore(0) + val receivedBlockIds = Collections.synchronizedSet(new HashSet[String]) + val errorBlockIds = Collections.synchronizedSet(new HashSet[String]) + val receivedBuffers = Collections.synchronizedSet(new HashSet[ReferenceCountedBuffer]) + + client.fetchBlocks( + blockIds, + (blockId, buf) => { + receivedBlockIds.add(blockId) + buf.retain() + receivedBuffers.add(buf) + sem.release() + }, + (blockId, errorMsg) => { + errorBlockIds.add(blockId) + sem.release() + } + ) + if (!sem.tryAcquire(blockIds.size, 30, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server") + } + client.close() + (receivedBlockIds.toSet, receivedBuffers.toSet, errorBlockIds.toSet) + } + + test("fetch a ByteBuffer block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId)) + assert(blockIds === Set(bufferBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch a FileSegment block via zero-copy send") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(fileBlockId)) + assert(blockIds === Set(fileBlockId)) + assert(buffers.map(_.underlying) === Set(fileBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch a non-existent block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq("random-block")) + assert(blockIds.isEmpty) + assert(buffers.isEmpty) + assert(failBlockIds === Set("random-block")) + } + + test("fetch both ByteBuffer block and FileSegment block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, fileBlockId)) + assert(blockIds === Set(bufferBlockId, fileBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference, fileBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch both ByteBuffer block and a non-existent block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, "random-block")) + assert(blockIds === Set(bufferBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) + assert(failBlockIds === Set("random-block")) + buffers.foreach(_.release()) + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala new file mode 100644 index 0000000000000..9afdad63b6988 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.nio.ByteBuffer + +import io.netty.buffer.Unpooled +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.FunSuite + + +class BlockFetchingClientHandlerSuite extends FunSuite { + + test("handling block data (successful fetch)") { + val blockId = "test_block" + val blockData = "blahblahblahblahblah" + val totalLength = 4 + blockId.length + blockData.length + + var parsedBlockId: String = "" + var parsedBlockData: String = "" + val handler = new BlockFetchingClientHandler + handler.blockFetchSuccessCallback = (bid, refCntBuf) => { + parsedBlockId = bid + val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) + refCntBuf.byteBuffer().get(bytes) + parsedBlockData = new String(bytes) + } + + val channel = new EmbeddedChannel(handler) + val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself + buf.putInt(totalLength) + buf.putInt(blockId.length) + buf.put(blockId.getBytes) + buf.put(blockData.getBytes) + buf.flip() + + channel.writeInbound(Unpooled.wrappedBuffer(buf)) + assert(parsedBlockId === blockId) + assert(parsedBlockData === blockData) + + channel.close() + } + + test("handling error message (failed fetch)") { + val blockId = "test_block" + val errorMsg = "error erro5r error err4or error3 error6 error erro1r" + val totalLength = 4 + blockId.length + errorMsg.length + + var parsedBlockId: String = "" + var parsedErrorMsg: String = "" + val handler = new BlockFetchingClientHandler + handler.blockFetchFailureCallback = (bid, msg) => { + parsedBlockId = bid + parsedErrorMsg = msg + } + + val channel = new EmbeddedChannel(handler) + val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself + buf.putInt(totalLength) + buf.putInt(-blockId.length) + buf.put(blockId.getBytes) + buf.put(errorMsg.getBytes) + buf.flip() + + channel.writeInbound(Unpooled.wrappedBuffer(buf)) + assert(parsedBlockId === blockId) + assert(parsedErrorMsg === errorMsg) + + channel.close() + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala new file mode 100644 index 0000000000000..3ee281cb1350b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import io.netty.buffer.ByteBuf +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.FunSuite + + +class BlockHeaderEncoderSuite extends FunSuite { + + test("encode normal block data") { + val blockId = "test_block" + val channel = new EmbeddedChannel(new BlockHeaderEncoder) + channel.writeOutbound(new BlockHeader(17, blockId, None)) + val out = channel.readOutbound().asInstanceOf[ByteBuf] + assert(out.readInt() === 4 + blockId.length + 17) + assert(out.readInt() === blockId.length) + + val blockIdBytes = new Array[Byte](blockId.length) + out.readBytes(blockIdBytes) + assert(new String(blockIdBytes) === blockId) + assert(out.readableBytes() === 0) + + channel.close() + } + + test("encode error message") { + val blockId = "error_block" + val errorMsg = "error encountered" + val channel = new EmbeddedChannel(new BlockHeaderEncoder) + channel.writeOutbound(new BlockHeader(17, blockId, Some(errorMsg))) + val out = channel.readOutbound().asInstanceOf[ByteBuf] + assert(out.readInt() === 4 + blockId.length + errorMsg.length) + assert(out.readInt() === -blockId.length) + + val blockIdBytes = new Array[Byte](blockId.length) + out.readBytes(blockIdBytes) + assert(new String(blockIdBytes) === blockId) + + val errorMsgBytes = new Array[Byte](errorMsg.length) + out.readBytes(errorMsgBytes) + assert(new String(errorMsgBytes) === errorMsg) + assert(out.readableBytes() === 0) + + channel.close() + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala new file mode 100644 index 0000000000000..12f6d87616644 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.io.File +import java.nio.ByteBuffer + +import io.netty.buffer.{Unpooled, ByteBuf} +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler, DefaultFileRegion} +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.FunSuite + +import org.apache.spark.storage.{BlockDataProvider, FileSegment} + + +class BlockServerHandlerSuite extends FunSuite { + + test("ByteBuffer block") { + val expectedBlockId = "test_bytebuffer_block" + val buf = ByteBuffer.allocate(10000) + for (i <- 1 to 10000) { + buf.put(i.toByte) + } + buf.flip() + + val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = Right(buf) + })) + + channel.writeInbound(expectedBlockId) + assert(channel.outboundMessages().size === 2) + + val out1 = channel.readOutbound().asInstanceOf[BlockHeader] + val out2 = channel.readOutbound().asInstanceOf[ByteBuf] + + assert(out1.blockId === expectedBlockId) + assert(out1.blockSize === buf.remaining) + assert(out1.error === None) + + assert(out2.equals(Unpooled.wrappedBuffer(buf))) + + channel.close() + } + + test("FileSegment block via zero-copy") { + val expectedBlockId = "test_file_block" + val url = Thread.currentThread.getContextClassLoader.getResource("netty-test-file.txt") + val testFile = new File(url.toURI) + + val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + Left(new FileSegment(testFile, 15, testFile.length - 25)) + } + })) + + channel.writeInbound(expectedBlockId) + assert(channel.outboundMessages().size === 2) + + val out1 = channel.readOutbound().asInstanceOf[BlockHeader] + val out2 = channel.readOutbound().asInstanceOf[DefaultFileRegion] + + assert(out1.blockId === expectedBlockId) + assert(out1.blockSize === testFile.length - 25) + assert(out1.error === None) + + assert(out2.count === testFile.length - 25) + assert(out2.position === 15) + } + + test("pipeline exception propagation") { + val blockServerHandler = new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = ??? + }) + val exceptionHandler = new SimpleChannelInboundHandler[String]() { + override def channelRead0(ctx: ChannelHandlerContext, msg: String): Unit = { + throw new Exception("this is an error") + } + } + + val channel = new EmbeddedChannel(exceptionHandler, blockServerHandler) + assert(channel.isOpen) + channel.writeInbound("a message to trigger the error") + assert(!channel.isOpen) + } +} diff --git a/pom.xml b/pom.xml index 920912353fe9c..71f7610c0e450 100644 --- a/pom.xml +++ b/pom.xml @@ -420,7 +420,7 @@ io.netty netty-all - 4.0.17.Final + 4.0.22.Final org.apache.derby From 9422a9b084e3fd5b2b9be2752013588adfb430d0 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Thu, 14 Aug 2014 19:03:51 -0700 Subject: [PATCH 0360/1492] [SPARK-2736] PySpark converter and example script for reading Avro files JIRA: https://issues.apache.org/jira/browse/SPARK-2736 This patch includes: 1. An Avro converter that converts Avro data types to Python. It handles all 3 Avro data mappings (Generic, Specific and Reflect). 2. An example Python script for reading Avro files using AvroKeyInputFormat and the converter. 3. Fixing a classloading issue. cc @MLnick @JoshRosen @mateiz Author: Kan Zhang Closes #1916 from kanzhang/SPARK-2736 and squashes the following commits: 02443f8 [Kan Zhang] [SPARK-2736] Adding .avsc files to .rat-excludes f74e9a9 [Kan Zhang] [SPARK-2736] nit: clazz -> className 82cc505 [Kan Zhang] [SPARK-2736] Update data sample 0be7761 [Kan Zhang] [SPARK-2736] Example pyspark script and data files c8e5881 [Kan Zhang] [SPARK-2736] Trying to work with all 3 Avro data models 2271a5b [Kan Zhang] [SPARK-2736] Using the right class loader to find Avro classes 536876b [Kan Zhang] [SPARK-2736] Adding Avro to Java converter --- .rat-excludes | 1 + .../spark/api/python/PythonHadoopUtil.scala | 3 +- .../apache/spark/api/python/PythonRDD.scala | 24 ++-- .../scala/org/apache/spark/util/Utils.scala | 3 + examples/src/main/python/avro_inputformat.py | 75 ++++++++++ examples/src/main/resources/user.avsc | 8 ++ examples/src/main/resources/users.avro | Bin 0 -> 334 bytes .../pythonconverters/AvroConverters.scala | 130 ++++++++++++++++++ 8 files changed, 231 insertions(+), 13 deletions(-) create mode 100644 examples/src/main/python/avro_inputformat.py create mode 100644 examples/src/main/resources/user.avsc create mode 100644 examples/src/main/resources/users.avro create mode 100644 examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala diff --git a/.rat-excludes b/.rat-excludes index bccb043c2bb55..eaefef1b0aa2e 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -25,6 +25,7 @@ log4j-defaults.properties bootstrap-tooltip.js jquery-1.11.1.min.js sorttable.js +.*avsc .*txt .*json .*data diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index f3b05e1243045..49dc95f349eac 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -19,6 +19,7 @@ package org.apache.spark.api.python import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils import org.apache.spark.{Logging, SerializableWritable, SparkException} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io._ @@ -42,7 +43,7 @@ private[python] object Converter extends Logging { defaultConverter: Converter[Any, Any]): Converter[Any, Any] = { converterClass.map { cc => Try { - val c = Class.forName(cc).newInstance().asInstanceOf[Converter[Any, Any]] + val c = Utils.classForName(cc).newInstance().asInstanceOf[Converter[Any, Any]] logInfo(s"Loaded converter: $cc") c } match { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index fefe1cb6f134c..9f5c5bd30f0c9 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -372,8 +372,8 @@ private[spark] object PythonRDD extends Logging { batchSize: Int) = { val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") - val kc = Class.forName(keyClass).asInstanceOf[Class[K]] - val vc = Class.forName(valueClass).asInstanceOf[Class[V]] + val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] + val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] val rdd = sc.sc.sequenceFile[K, V](path, kc, vc, minSplits) val confBroadcasted = sc.sc.broadcast(new SerializableWritable(sc.hadoopConfiguration())) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, @@ -440,9 +440,9 @@ private[spark] object PythonRDD extends Logging { keyClass: String, valueClass: String, conf: Configuration) = { - val kc = Class.forName(keyClass).asInstanceOf[Class[K]] - val vc = Class.forName(valueClass).asInstanceOf[Class[V]] - val fc = Class.forName(inputFormatClass).asInstanceOf[Class[F]] + val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] + val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] + val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] if (path.isDefined) { sc.sc.newAPIHadoopFile[K, V, F](path.get, fc, kc, vc, conf) } else { @@ -509,9 +509,9 @@ private[spark] object PythonRDD extends Logging { keyClass: String, valueClass: String, conf: Configuration) = { - val kc = Class.forName(keyClass).asInstanceOf[Class[K]] - val vc = Class.forName(valueClass).asInstanceOf[Class[V]] - val fc = Class.forName(inputFormatClass).asInstanceOf[Class[F]] + val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] + val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] + val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] if (path.isDefined) { sc.sc.hadoopFile(path.get, fc, kc, vc) } else { @@ -558,7 +558,7 @@ private[spark] object PythonRDD extends Logging { for { k <- Option(keyClass) v <- Option(valueClass) - } yield (Class.forName(k), Class.forName(v)) + } yield (Utils.classForName(k), Utils.classForName(v)) } private def getKeyValueConverters(keyConverterClass: String, valueConverterClass: String, @@ -621,10 +621,10 @@ private[spark] object PythonRDD extends Logging { val (kc, vc) = getKeyValueTypes(keyClass, valueClass).getOrElse( inferKeyValueTypes(rdd, keyConverterClass, valueConverterClass)) val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration) - val codec = Option(compressionCodecClass).map(Class.forName(_).asInstanceOf[Class[C]]) + val codec = Option(compressionCodecClass).map(Utils.classForName(_).asInstanceOf[Class[C]]) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, new JavaToWritableConverter) - val fc = Class.forName(outputFormatClass).asInstanceOf[Class[F]] + val fc = Utils.classForName(outputFormatClass).asInstanceOf[Class[F]] converted.saveAsHadoopFile(path, kc, vc, fc, new JobConf(mergedConf), codec=codec) } @@ -653,7 +653,7 @@ private[spark] object PythonRDD extends Logging { val mergedConf = getMergedConf(confAsMap, pyRDD.context.hadoopConfiguration) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, new JavaToWritableConverter) - val fc = Class.forName(outputFormatClass).asInstanceOf[Class[F]] + val fc = Utils.classForName(outputFormatClass).asInstanceOf[Class[F]] converted.saveAsNewAPIHadoopFile(path, kc, vc, fc, mergedConf) } 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 8cac5da644fa9..019f68b160894 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -146,6 +146,9 @@ private[spark] object Utils extends Logging { Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess } + /** Preferred alternative to Class.forName(className) */ + def classForName(className: String) = Class.forName(className, true, getContextOrSparkClassLoader) + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py new file mode 100644 index 0000000000000..e902ae29753c0 --- /dev/null +++ b/examples/src/main/python/avro_inputformat.py @@ -0,0 +1,75 @@ +# +# 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. +# + +import sys + +from pyspark import SparkContext + +""" +Read data file users.avro in local Spark distro: + +$ cd $SPARK_HOME +$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +> examples/src/main/resources/users.avro +{u'favorite_color': None, u'name': u'Alyssa', u'favorite_numbers': [3, 9, 15, 20]} +{u'favorite_color': u'red', u'name': u'Ben', u'favorite_numbers': []} + +To read name and favorite_color fields only, specify the following reader schema: + +$ cat examples/src/main/resources/user.avsc +{"namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string"}, + {"name": "favorite_color", "type": ["string", "null"]} + ] +} + +$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +> examples/src/main/resources/users.avro examples/src/main/resources/user.avsc +{u'favorite_color': None, u'name': u'Alyssa'} +{u'favorite_color': u'red', u'name': u'Ben'} +""" +if __name__ == "__main__": + if len(sys.argv) != 2 and len(sys.argv) != 3: + print >> sys.stderr, """ + Usage: avro_inputformat [reader_schema_file] + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/avro_inputformat.py [reader_schema_file] + Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. + """ + exit(-1) + + path = sys.argv[1] + sc = SparkContext(appName="AvroKeyInputFormat") + + conf = None + if len(sys.argv) == 3: + schema_rdd = sc.textFile(sys.argv[2], 1).collect() + conf = {"avro.schema.input.key" : reduce(lambda x, y: x+y, schema_rdd)} + + avro_rdd = sc.newAPIHadoopFile(path, + "org.apache.avro.mapreduce.AvroKeyInputFormat", + "org.apache.avro.mapred.AvroKey", + "org.apache.hadoop.io.NullWritable", + keyConverter="org.apache.spark.examples.pythonconverters.AvroWrapperToJavaConverter", + conf=conf) + output = avro_rdd.map(lambda x: x[0]).collect() + for k in output: + print k diff --git a/examples/src/main/resources/user.avsc b/examples/src/main/resources/user.avsc new file mode 100644 index 0000000000000..4995357ab3736 --- /dev/null +++ b/examples/src/main/resources/user.avsc @@ -0,0 +1,8 @@ +{"namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + {"name": "name", "type": "string"}, + {"name": "favorite_color", "type": ["string", "null"]} + ] +} diff --git a/examples/src/main/resources/users.avro b/examples/src/main/resources/users.avro new file mode 100644 index 0000000000000000000000000000000000000000..27c526ab114b2f42f6d4e13325c373706ba0f880 GIT binary patch literal 334 zcmeZI%3@>@ODrqO*DFrWNX<=rz+A0VQdy9yWTl`~l$xAhl%k}gpp=)Gn_66um<$$9 ztw_u*$Vt@$>4Hgul!q3l7J>L_nW;G`#Xym0gi*yMMVWc&$f`j`D%I*Jz|}-6At@@& z$x(`hS`0EfEwL=WD6=FrJ~=-pzX(NNwGvP~7i6DOW?l)%3Yhy7i;5B}L2AM7M=>U^ zG&d==s932swpIk}`{ewT)MSo4puG%vlk4vPb+WF0^sw`-e)omlECxJ|IhDo5iA)@9 TLUI}mY)+|p3~WWIDHtjNiNSH? literal 0 HcmV?d00001 diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala new file mode 100644 index 0000000000000..1b25983a38453 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.pythonconverters + +import java.util.{Collection => JCollection, Map => JMap} + +import scala.collection.JavaConversions._ + +import org.apache.avro.generic.{GenericFixed, IndexedRecord} +import org.apache.avro.mapred.AvroWrapper +import org.apache.avro.Schema +import org.apache.avro.Schema.Type._ + +import org.apache.spark.api.python.Converter +import org.apache.spark.SparkException + + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries + * to work with all 3 Avro data mappings (Generic, Specific and Reflect). + */ +class AvroWrapperToJavaConverter extends Converter[Any, Any] { + override def convert(obj: Any): Any = { + if (obj == null) { + return null + } + obj.asInstanceOf[AvroWrapper[_]].datum() match { + case null => null + case record: IndexedRecord => unpackRecord(record) + case other => throw new SparkException( + s"Unsupported top-level Avro data type ${other.getClass.getName}") + } + } + + def unpackRecord(obj: Any): JMap[String, Any] = { + val map = new java.util.HashMap[String, Any] + obj match { + case record: IndexedRecord => + record.getSchema.getFields.zipWithIndex.foreach { case (f, i) => + map.put(f.name, fromAvro(record.get(i), f.schema)) + } + case other => throw new SparkException( + s"Unsupported RECORD type ${other.getClass.getName}") + } + map + } + + def unpackMap(obj: Any, schema: Schema): JMap[String, Any] = { + obj.asInstanceOf[JMap[_, _]].map { case (key, value) => + (key.toString, fromAvro(value, schema.getValueType)) + } + } + + def unpackFixed(obj: Any, schema: Schema): Array[Byte] = { + unpackBytes(obj.asInstanceOf[GenericFixed].bytes()) + } + + def unpackBytes(obj: Any): Array[Byte] = { + val bytes: Array[Byte] = obj match { + case buf: java.nio.ByteBuffer => buf.array() + case arr: Array[Byte] => arr + case other => throw new SparkException( + s"Unknown BYTES type ${other.getClass.getName}") + } + val bytearray = new Array[Byte](bytes.length) + System.arraycopy(bytes, 0, bytearray, 0, bytes.length) + bytearray + } + + def unpackArray(obj: Any, schema: Schema): JCollection[Any] = obj match { + case c: JCollection[_] => + c.map(fromAvro(_, schema.getElementType)) + case arr: Array[_] if arr.getClass.getComponentType.isPrimitive => + arr.toSeq + case arr: Array[_] => + arr.map(fromAvro(_, schema.getElementType)).toSeq + case other => throw new SparkException( + s"Unknown ARRAY type ${other.getClass.getName}") + } + + def unpackUnion(obj: Any, schema: Schema): Any = { + schema.getTypes.toList match { + case List(s) => fromAvro(obj, s) + case List(n, s) if n.getType == NULL => fromAvro(obj, s) + case List(s, n) if n.getType == NULL => fromAvro(obj, s) + case _ => throw new SparkException( + "Unions may only consist of a concrete type and null") + } + } + + def fromAvro(obj: Any, schema: Schema): Any = { + if (obj == null) { + return null + } + schema.getType match { + case UNION => unpackUnion(obj, schema) + case ARRAY => unpackArray(obj, schema) + case FIXED => unpackFixed(obj, schema) + case MAP => unpackMap(obj, schema) + case BYTES => unpackBytes(obj) + case RECORD => unpackRecord(obj) + case STRING => obj.toString + case ENUM => obj.toString + case NULL => obj + case BOOLEAN => obj + case DOUBLE => obj + case FLOAT => obj + case INT => obj + case LONG => obj + case other => throw new SparkException( + s"Unknown Avro schema type ${other.getName}") + } + } +} From 500f84e49d0c109a9b7a1ff04678b5fb8f301984 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 14 Aug 2014 22:05:14 -0700 Subject: [PATCH 0361/1492] [SPARK-2912] [Spark QA] Include commit hash in Spark QA messages You can find the [discussion that motivated this PR here](http://mail-archives.apache.org/mod_mbox/spark-dev/201408.mbox/%3CCABPQxssy0ri2QAz=cc9Tx+EXYWARm7pNcVm8apqCwc-esLbO4Qmail.gmail.com%3E). As described in [SPARK-2912](https://issues.apache.org/jira/browse/SPARK-2912), the goal of this PR (and related ones to come) is to include useful detail in Spark QA's messages that are intended to make a committer's job easier to do. Since this work depends on Jenkins, I cannot test this locally. Hence, I will be iterating via this PR. Notes: * This is a duplicate of a [previous PR](https://github.com/apache/spark/pull/1811), without the extraneous commits. * This PR also resolves an issue targeted by [another open PR](https://github.com/apache/spark/pull/1809). Closes #1809. Author: Nicholas Chammas Author: nchammas Closes #1816 from nchammas/master and squashes the following commits: c1be644 [Nicholas Chammas] [SPARK-2912] include commit hash in messages 8f641ac [nchammas] Merge pull request #7 from apache/master --- dev/run-tests-jenkins | 187 +++++++++++++++++++++++++++++++----------- 1 file changed, 138 insertions(+), 49 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 3076eb847b420..721f09be5be6d 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -19,67 +19,156 @@ # Wrapper script that runs the Spark tests then reports QA results # to github via its API. +# Environment variables are populated by the code here: +#+ https://github.com/jenkinsci/ghprb-plugin/blob/master/src/main/java/org/jenkinsci/plugins/ghprb/GhprbTrigger.java#L139 # Go to the Spark project root directory FWDIR="$(cd `dirname $0`/..; pwd)" cd "$FWDIR" +function get_jq () { + # Get jq so we can parse some JSON, man. + # Essential if we want to do anything with the GitHub API responses. + local JQ_EXECUTABLE_URL="http://stedolan.github.io/jq/download/linux64/jq" + + echo "Fetching jq from ${JQ_EXECUTABLE_URL}" + + curl --silent --output "$FWDIR/dev/jq" "$JQ_EXECUTABLE_URL" + local curl_status=$? + + if [ $curl_status -ne 0 ]; then + echo "Failed to get jq." >&2 + return $curl_status + fi + + chmod u+x "$FWDIR/dev/jq" +} + COMMENTS_URL="https://api.github.com/repos/apache/spark/issues/$ghprbPullId/comments" +PULL_REQUEST_URL="https://github.com/apache/spark/pull/$ghprbPullId" + +function post_message () { + local message=$1 + local data="{\"body\": \"$message\"}" + local HTTP_CODE_HEADER="HTTP Response Code: " + + echo "Attempting to post to Github..." + + local curl_output=$( + curl `#--dump-header -` \ + --silent \ + --user x-oauth-basic:$GITHUB_OAUTH_KEY \ + --request POST \ + --data "$data" \ + --write-out "${HTTP_CODE_HEADER}%{http_code}\n" \ + --header "Content-Type: application/json" \ + "$COMMENTS_URL" #> /dev/null #| "$FWDIR/dev/jq" .id #| head -n 8 + ) + local curl_status=${PIPESTATUS[0]} + + if [ "$curl_status" -ne 0 ]; then + echo "Failed to post message to GitHub." >&2 + echo " > curl_status: ${curl_status}" >&2 + echo " > curl_output: ${curl_output}" >&2 + echo " > data: ${data}" >&2 + # exit $curl_status + fi + + local api_response=$( + echo "${curl_output}" \ + | grep -v -e "^${HTTP_CODE_HEADER}" + ) + + local http_code=$( + echo "${curl_output}" \ + | grep -e "^${HTTP_CODE_HEADER}" \ + | sed -r -e "s/^${HTTP_CODE_HEADER}//g" + ) + + if [ -n "$http_code" ] && [ "$http_code" -ne "201" ]; then + echo " > http_code: ${http_code}." >&2 + echo " > api_response: ${api_response}" >&2 + echo " > data: ${data}" >&2 + fi + + if [ "$curl_status" -eq 0 ] && [ "$http_code" -eq "201" ]; then + echo " > Post successful." + fi +} + +COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" +# GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( +short_commit_hash=${ghprbActualCommit:0:7} + +# check PR merge-ability and check for new public classes +{ + if [ "$sha1" == "$ghprbActualCommit" ]; then + merge_note=" * This patch **does not** merge cleanly!" + else + merge_note=" * This patch merges cleanly." + + non_test_files=$(git diff master --name-only | grep -v "\/test" | tr "\n" " ") + new_public_classes=$( + git diff master ${non_test_files} `# diff this patch against master and...` \ + | grep "^\+" `# filter in only added lines` \ + | sed -r -e "s/^\+//g" `# remove the leading +` \ + | grep -e "trait " -e "class " `# filter in lines with these key words` \ + | grep -e "{" -e "(" `# filter in lines with these key words, too` \ + | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ + | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ + | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ + | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ + | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ + | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ + | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ + | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ + | tr -d "\n" `# remove actual LF characters` + ) -function post_message { - message=$1 - data="{\"body\": \"$message\"}" - echo "Attempting to post to Github:" - echo "$data" + if [ "$new_public_classes" == "" ]; then + public_classes_note=" * This patch adds no public classes." + else + public_classes_note=" * This patch adds the following public classes _(experimental)_:" + public_classes_note="${public_classes_note}\n${new_public_classes}" + fi + fi +} - curl -D- -u x-oauth-basic:$GITHUB_OAUTH_KEY -X POST --data "$data" -H \ - "Content-Type: application/json" \ - $COMMENTS_URL | head -n 8 +# post start message +{ + start_message="\ + [QA tests have started](${BUILD_URL}consoleFull) for \ + PR $ghprbPullId at commit [\`${short_commit_hash}\`](${COMMIT_URL})." + + start_message="${start_message}\n${merge_note}" + # start_message="${start_message}\n${public_classes_note}" + + post_message "$start_message" } -start_message="QA tests have started for PR $ghprbPullId." -if [ "$sha1" == "$ghprbActualCommit" ]; then - start_message="$start_message This patch DID NOT merge cleanly! " -else - start_message="$start_message This patch merges cleanly. " -fi -start_message="$start_message
    View progress: " -start_message="$start_message${BUILD_URL}consoleFull" - -post_message "$start_message" - -./dev/run-tests -test_result="$?" - -result_message="QA results for PR $ghprbPullId:
    " - -if [ "$test_result" -eq "0" ]; then - result_message="$result_message- This patch PASSES unit tests.
    " -else - result_message="$result_message- This patch FAILED unit tests.
    " -fi - -if [ "$sha1" != "$ghprbActualCommit" ]; then - result_message="$result_message- This patch merges cleanly
    " - non_test_files=$(git diff master --name-only | grep -v "\/test" | tr "\n" " ") - new_public_classes=$(git diff master $non_test_files \ - | grep -e "trait " -e "class " \ - | grep -e "{" -e "(" \ - | grep -v -e \@\@ -e private \ - | grep \+ \ - | sed "s/\+ *//" \ - | tr "\n" "~" \ - | sed "s/~/
    /g") - if [ "$new_public_classes" == "" ]; then - result_message="$result_message- This patch adds no public classes
    " +# run tests +{ + ./dev/run-tests + test_result="$?" + + if [ "$test_result" -eq "0" ]; then + test_result_note=" * This patch **passes** unit tests." else - result_message="$result_message- This patch adds the following public classes (experimental):
    " - result_message="$result_message$new_public_classes" + test_result_note=" * This patch **fails** unit tests." fi -fi -result_message="${result_message}
    For more information see test ouptut:" -result_message="${result_message}
    ${BUILD_URL}consoleFull" +} -post_message "$result_message" +# post end message +{ + result_message="\ + [QA tests have finished](${BUILD_URL}consoleFull) for \ + PR $ghprbPullId at commit [\`${short_commit_hash}\`](${COMMIT_URL})." + + result_message="${result_message}\n${test_result_note}" + result_message="${result_message}\n${merge_note}" + result_message="${result_message}\n${public_classes_note}" + + post_message "$result_message" +} exit $test_result From e1b85f3102e5e25d0168b80aa953e1e76054a945 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 14 Aug 2014 22:08:44 -0700 Subject: [PATCH 0362/1492] SPARK-2955 [BUILD] Test code fails to compile with "mvn compile" without "install" (This is the corrected follow-up to https://issues.apache.org/jira/browse/SPARK-2903) Right now, `mvn compile test-compile` fails to compile Spark. (Don't worry; `mvn package` works, so this is not major.) The issue stems from test code in some modules depending on test code in other modules. That is perfectly fine and supported by Maven. It takes extra work to get this to work with scalatest, and this has been attempted: https://github.com/apache/spark/blob/master/sql/catalyst/pom.xml#L86 This formulation is not quite enough, since the SQL Core module's tests fail to compile for lack of finding test classes in SQL Catalyst, and likewise for most Streaming integration modules depending on core Streaming test code. Example: ``` [error] /Users/srowen/Documents/spark/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala:23: not found: type PlanTest [error] class QueryTest extends PlanTest { [error] ^ [error] /Users/srowen/Documents/spark/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala:28: package org.apache.spark.sql.test is not a value [error] test("SPARK-1669: cacheTable should be idempotent") { [error] ^ ... ``` The issue I believe is that generation of a `test-jar` is bound here to the `compile` phase, but the test classes are not being compiled in this phase. It should bind to the `test-compile` phase. It works when executing `mvn package` or `mvn install` since test-jar artifacts are actually generated available through normal Maven mechanisms as each module is built. They are then found normally, regardless of scalatest configuration. It would be nice for a simple `mvn compile test-compile` to work since the test code is perfectly compilable given the Maven declarations. On the plus side, this change is low-risk as it only affects tests. yhuai made the original scalatest change and has glanced at this and thinks it makes sense. Author: Sean Owen Closes #1879 from srowen/SPARK-2955 and squashes the following commits: ad8242f [Sean Owen] Generate test-jar on test-compile for modules whose tests are needed by others' tests --- sql/catalyst/pom.xml | 28 ++++++++++++++-------------- streaming/pom.xml | 8 ++++---- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 58d44e7923bee..830711a46a35b 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -77,28 +77,28 @@ org.apache.maven.plugins maven-jar-plugin - - - test-jar - - - - test-jar-on-compile - compile - - test-jar - - + + + test-jar + + + + test-jar-on-test-compile + test-compile + + test-jar + + diff --git a/streaming/pom.xml b/streaming/pom.xml index 1072f74aea0d9..ce35520a28609 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -81,11 +81,11 @@ org.apache.maven.plugins @@ -97,8 +97,8 @@ - test-jar-on-compile - compile + test-jar-on-test-compile + test-compile test-jar From fba8ec39ccf455a4a03504445bad9af420915b4f Mon Sep 17 00:00:00 2001 From: Nathan Kronenfeld Date: Thu, 14 Aug 2014 22:15:33 -0700 Subject: [PATCH 0363/1492] Add caching information to rdd.toDebugString I find it useful to see where in an RDD's DAG data is cached, so I figured others might too. I've added both the caching level, and the actual memory state of the RDD. Some of this is redundant with the web UI (notably the actual memory state), but (a) that is temporary, and (b) putting it in the DAG tree shows some context that can help a lot. For example: ``` (4) ShuffledRDD[3] at reduceByKey at :14 +-(4) MappedRDD[2] at map at :14 | MapPartitionsRDD[1] at mapPartitions at :12 | ParallelCollectionRDD[0] at parallelize at :12 ``` should change to ``` (4) ShuffledRDD[3] at reduceByKey at :14 [Memory Deserialized 1x Replicated] | CachedPartitions: 4; MemorySize: 50.8 MB; TachyonSize: 0.0 B; DiskSize: 0.0 B +-(4) MappedRDD[2] at map at :14 [Memory Deserialized 1x Replicated] | MapPartitionsRDD[1] at mapPartitions at :12 [Memory Deserialized 1x Replicated] | CachedPartitions: 4; MemorySize: 109.1 MB; TachyonSize: 0.0 B; DiskSize: 0.0 B | ParallelCollectionRDD[0] at parallelize at :12 [Memory Deserialized 1x Replicated] ``` Author: Nathan Kronenfeld Closes #1535 from nkronenfeld/feature/debug-caching2 and squashes the following commits: 40490bc [Nathan Kronenfeld] Back out DeveloperAPI and arguments to RDD.toDebugString, reinstate memory output 794e6a3 [Nathan Kronenfeld] Attempt to merge mima changes from master 6fe9e80 [Nathan Kronenfeld] Add exclusions to allow for signature change in toDebugString (will back out if necessary) 31d6769 [Nathan Kronenfeld] Attempt to get rid of style errors. Add comments for the new memory usage parameter. a0f6f76 [Nathan Kronenfeld] Add parameter to RDD.toDebugString to allow detailed memory info to be shown or not. Default is for it not to be shown. f8f565a [Nathan Kronenfeld] Fix code style error 8f54287 [Nathan Kronenfeld] Changed string addition to string interpolation as per PR comments 2a0cd4d [Nathan Kronenfeld] Fixed a small formatting issue I forgot to copy over from the old branch 8fbecb6 [Nathan Kronenfeld] Add caching information to rdd.toDebugString --- .../main/scala/org/apache/spark/rdd/RDD.scala | 30 +++++++++++++++---- 1 file changed, 25 insertions(+), 5 deletions(-) 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 19e10bd04681b..daea2617e62ea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1299,6 +1299,19 @@ abstract class RDD[T: ClassTag]( /** A description of this RDD and its recursive dependencies for debugging. */ def toDebugString: String = { + // Get a debug description of an rdd without its children + def debugSelf (rdd: RDD[_]): Seq[String] = { + import Utils.bytesToString + + val persistence = storageLevel.description + val storageInfo = rdd.context.getRDDStorageInfo.filter(_.id == rdd.id).map(info => + " CachedPartitions: %d; MemorySize: %s; TachyonSize: %s; DiskSize: %s".format( + info.numCachedPartitions, bytesToString(info.memSize), + bytesToString(info.tachyonSize), bytesToString(info.diskSize))) + + s"$rdd [$persistence]" +: storageInfo + } + // Apply a different rule to the last child def debugChildren(rdd: RDD[_], prefix: String): Seq[String] = { val len = rdd.dependencies.length @@ -1324,7 +1337,11 @@ abstract class RDD[T: ClassTag]( val partitionStr = "(" + rdd.partitions.size + ")" val leftOffset = (partitionStr.length - 1) / 2 val nextPrefix = (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset)) - Seq(partitionStr + " " + rdd) ++ debugChildren(rdd, nextPrefix) + + debugSelf(rdd).zipWithIndex.map{ + case (desc: String, 0) => s"$partitionStr $desc" + case (desc: String, _) => s"$nextPrefix $desc" + } ++ debugChildren(rdd, nextPrefix) } def shuffleDebugString(rdd: RDD[_], prefix: String = "", isLastChild: Boolean): Seq[String] = { val partitionStr = "(" + rdd.partitions.size + ")" @@ -1334,7 +1351,11 @@ abstract class RDD[T: ClassTag]( thisPrefix + (if (isLastChild) " " else "| ") + (" " * leftOffset) + "|" + (" " * (partitionStr.length - leftOffset))) - Seq(thisPrefix + "+-" + partitionStr + " " + rdd) ++ debugChildren(rdd, nextPrefix) + + debugSelf(rdd).zipWithIndex.map{ + case (desc: String, 0) => s"$thisPrefix+-$partitionStr $desc" + case (desc: String, _) => s"$nextPrefix$desc" + } ++ debugChildren(rdd, nextPrefix) } def debugString(rdd: RDD[_], prefix: String = "", @@ -1342,9 +1363,8 @@ abstract class RDD[T: ClassTag]( isLastChild: Boolean = false): Seq[String] = { if (isShuffle) { shuffleDebugString(rdd, prefix, isLastChild) - } - else { - Seq(prefix + rdd) ++ debugChildren(rdd, prefix) + } else { + debugSelf(rdd).map(prefix + _) ++ debugChildren(rdd, prefix) } } firstDebugString(this).mkString("\n") From 7589c39d39a8d0744fb689e5752ee8e0108a81eb Mon Sep 17 00:00:00 2001 From: Anand Avati Date: Fri, 15 Aug 2014 08:53:52 -0700 Subject: [PATCH 0364/1492] [SPARK-2924] remove default args to overloaded methods Not supported in Scala 2.11. Split them into separate methods instead. Author: Anand Avati Closes #1704 from avati/SPARK-1812-default-args and squashes the following commits: 3e3924a [Anand Avati] SPARK-1812: Add Mima excludes for the broken ABI 901dfc7 [Anand Avati] SPARK-1812: core - Fix overloaded methods with default arguments 07f00af [Anand Avati] SPARK-1812: streaming - Fix overloaded methods with default arguments --- core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 2 +- project/MimaExcludes.scala | 3 +++ .../org/apache/spark/streaming/StreamingContext.scala | 8 +++++++- 3 files changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 29e9cf947856f..6b4689291097f 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -93,7 +93,7 @@ private[spark] object JettyUtils extends Logging { def createServletHandler( path: String, servlet: HttpServlet, - basePath: String = ""): ServletContextHandler = { + basePath: String): ServletContextHandler = { val prefixedPath = attachPrefix(basePath, path) val contextHandler = new ServletContextHandler val holder = new ServletHolder(servlet) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 6e72035f2c15b..1e3c760b845de 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -117,6 +117,9 @@ object MimaExcludes { ) ++ Seq( // new Vector methods in MLlib (binary compatible assuming users do not implement Vector) ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.copy") + ) ++ + Seq ( // Scala 2.11 compatibility fix + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.$default$2") ) case v if v.startsWith("1.0") => Seq( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e0677b795cb94..101cec1c7a7c2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -98,9 +98,15 @@ class StreamingContext private[streaming] ( * @param hadoopConf Optional, configuration object if necessary for reading from * HDFS compatible filesystems */ - def this(path: String, hadoopConf: Configuration = new Configuration) = + def this(path: String, hadoopConf: Configuration) = this(null, CheckpointReader.read(path, new SparkConf(), hadoopConf).get, null) + /** + * Recreate a StreamingContext from a checkpoint file. + * @param path Path to the directory that was specified as the checkpoint directory + */ + def this(path: String) = this(path, new Configuration) + if (sc_ == null && cp_ == null) { throw new Exception("Spark Streaming cannot be initialized with " + "both SparkContext and checkpoint as null") From fd9fcd25e93c727b327909cde0027426204ca6c3 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 15 Aug 2014 09:01:04 -0700 Subject: [PATCH 0365/1492] Revert "[SPARK-2468] Netty based block server / client module" This reverts commit 3a8b68b7353fea50245686903b308fa9eb52cb51. --- .../spark/network/netty/FileClient.scala | 85 + .../netty/FileClientChannelInitializer.scala} | 21 +- .../network/netty/FileClientHandler.scala | 50 + .../spark/network/netty/FileHeader.scala | 71 + .../spark/network/netty/FileServer.scala | 91 ++ ...ala => FileServerChannelInitializer.scala} | 22 +- .../network/netty/FileServerHandler.scala | 68 + .../spark/network/netty/NettyConfig.scala | 59 - .../spark/network/netty/ShuffleCopier.scala | 118 ++ .../spark/network/netty/ShuffleSender.scala | 71 + .../netty/client/BlockFetchingClient.scala | 135 -- .../client/BlockFetchingClientFactory.scala | 99 -- .../client/BlockFetchingClientHandler.scala | 63 - .../netty/client/LazyInitIterator.scala | 44 - .../netty/client/ReferenceCountedBuffer.scala | 47 - .../network/netty/server/BlockHeader.scala | 32 - .../netty/server/BlockHeaderEncoder.scala | 47 - .../network/netty/server/BlockServer.scala | 162 -- .../netty/server/BlockServerHandler.scala | 140 -- .../spark/storage/BlockFetcherIterator.scala | 138 +- .../apache/spark/storage/BlockManager.scala | 49 +- .../storage/BlockNotFoundException.scala | 21 - .../spark/storage/DiskBlockManager.scala | 13 +- core/src/test/resources/netty-test-file.txt | 1379 ----------------- .../netty/ServerClientIntegrationSuite.scala | 158 -- .../BlockFetchingClientHandlerSuite.scala | 87 -- .../server/BlockHeaderEncoderSuite.scala | 64 - .../server/BlockServerHandlerSuite.scala | 101 -- pom.xml | 2 +- 29 files changed, 667 insertions(+), 2770 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClient.scala rename core/src/main/scala/org/apache/spark/{storage/BlockDataProvider.scala => network/netty/FileClientChannelInitializer.scala} (65%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServer.scala rename core/src/main/scala/org/apache/spark/network/netty/{server/BlockServerChannelInitializer.scala => FileServerChannelInitializer.scala} (58%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala delete mode 100644 core/src/test/resources/netty-test-file.txt delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala new file mode 100644 index 0000000000000..c6d35f73db545 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.util.concurrent.TimeUnit + +import io.netty.bootstrap.Bootstrap +import io.netty.channel.{Channel, ChannelOption, EventLoopGroup} +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.oio.OioSocketChannel + +import org.apache.spark.Logging + +class FileClient(handler: FileClientHandler, connectTimeout: Int) extends Logging { + + private var channel: Channel = _ + private var bootstrap: Bootstrap = _ + private var group: EventLoopGroup = _ + private val sendTimeout = 60 + + def init(): Unit = { + group = new OioEventLoopGroup + bootstrap = new Bootstrap + bootstrap.group(group) + .channel(classOf[OioSocketChannel]) + .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) + .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, Integer.valueOf(connectTimeout)) + .handler(new FileClientChannelInitializer(handler)) + } + + def connect(host: String, port: Int) { + try { + channel = bootstrap.connect(host, port).sync().channel() + } catch { + case e: InterruptedException => + logWarning("FileClient interrupted while trying to connect", e) + close() + } + } + + def waitForClose(): Unit = { + try { + channel.closeFuture.sync() + } catch { + case e: InterruptedException => + logWarning("FileClient interrupted", e) + } + } + + def sendRequest(file: String): Unit = { + try { + val bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS) + if (!bSent) { + throw new RuntimeException("Failed to send") + } + } catch { + case e: InterruptedException => + logError("Error", e) + } + } + + def close(): Unit = { + if (group != null) { + group.shutdownGracefully() + group = null + bootstrap = null + } + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala similarity index 65% rename from core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala rename to core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala index 5b6d086630834..f4261c13f70a8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.network.netty -import java.nio.ByteBuffer +import io.netty.channel.ChannelInitializer +import io.netty.channel.socket.SocketChannel +import io.netty.handler.codec.string.StringEncoder -/** - * An interface for providing data for blocks. - * - * getBlockData returns either a FileSegment (for zero-copy send), or a ByteBuffer. - * - * Aside from unit tests, [[BlockManager]] is the main class that implements this. - */ -private[spark] trait BlockDataProvider { - def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] +class FileClientChannelInitializer(handler: FileClientHandler) + extends ChannelInitializer[SocketChannel] { + + def initChannel(channel: SocketChannel) { + channel.pipeline.addLast("encoder", new StringEncoder).addLast("handler", handler) + } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala new file mode 100644 index 0000000000000..017302ec7d33d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import io.netty.buffer.ByteBuf +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.storage.BlockId + + +abstract class FileClientHandler extends SimpleChannelInboundHandler[ByteBuf] { + + private var currentHeader: FileHeader = null + + @volatile + private var handlerCalled: Boolean = false + + def isComplete: Boolean = handlerCalled + + def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) + + def handleError(blockId: BlockId) + + override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { + if (currentHeader == null && in.readableBytes >= FileHeader.HEADER_SIZE) { + currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE)) + } + if (in.readableBytes >= currentHeader.fileLen) { + handle(ctx, in, currentHeader) + handlerCalled = true + currentHeader = null + ctx.close() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala new file mode 100644 index 0000000000000..607e560ff277f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import io.netty.buffer._ + +import org.apache.spark.Logging +import org.apache.spark.storage.{BlockId, TestBlockId} + +private[spark] class FileHeader ( + val fileLen: Int, + val blockId: BlockId) extends Logging { + + lazy val buffer: ByteBuf = { + val buf = Unpooled.buffer() + buf.capacity(FileHeader.HEADER_SIZE) + buf.writeInt(fileLen) + buf.writeInt(blockId.name.length) + blockId.name.foreach((x: Char) => buf.writeByte(x)) + // padding the rest of header + if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { + buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) + } else { + throw new Exception("too long header " + buf.readableBytes) + logInfo("too long header") + } + buf + } + +} + +private[spark] object FileHeader { + + val HEADER_SIZE = 40 + + def getFileLenOffset = 0 + def getFileLenSize = Integer.SIZE/8 + + def create(buf: ByteBuf): FileHeader = { + val length = buf.readInt + val idLength = buf.readInt + val idBuilder = new StringBuilder(idLength) + for (i <- 1 to idLength) { + idBuilder += buf.readByte().asInstanceOf[Char] + } + val blockId = BlockId(idBuilder.toString()) + new FileHeader(length, blockId) + } + + def main(args:Array[String]) { + val header = new FileHeader(25, TestBlockId("my_block")) + val buf = header.buffer + val newHeader = FileHeader.create(buf) + System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala new file mode 100644 index 0000000000000..dff77950659af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.net.InetSocketAddress + +import io.netty.bootstrap.ServerBootstrap +import io.netty.channel.{ChannelFuture, ChannelOption, EventLoopGroup} +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.oio.OioServerSocketChannel + +import org.apache.spark.Logging + +/** + * Server that accept the path of a file an echo back its content. + */ +class FileServer(pResolver: PathResolver, private var port: Int) extends Logging { + + private val addr: InetSocketAddress = new InetSocketAddress(port) + private var bossGroup: EventLoopGroup = new OioEventLoopGroup + private var workerGroup: EventLoopGroup = new OioEventLoopGroup + + private var channelFuture: ChannelFuture = { + val bootstrap = new ServerBootstrap + bootstrap.group(bossGroup, workerGroup) + .channel(classOf[OioServerSocketChannel]) + .option(ChannelOption.SO_BACKLOG, java.lang.Integer.valueOf(100)) + .option(ChannelOption.SO_RCVBUF, java.lang.Integer.valueOf(1500)) + .childHandler(new FileServerChannelInitializer(pResolver)) + bootstrap.bind(addr) + } + + try { + val boundAddress = channelFuture.sync.channel.localAddress.asInstanceOf[InetSocketAddress] + port = boundAddress.getPort + } catch { + case ie: InterruptedException => + port = 0 + } + + /** Start the file server asynchronously in a new thread. */ + def start(): Unit = { + val blockingThread: Thread = new Thread { + override def run(): Unit = { + try { + channelFuture.channel.closeFuture.sync + logInfo("FileServer exiting") + } catch { + case e: InterruptedException => + logError("File server start got interrupted", e) + } + // NOTE: bootstrap is shutdown in stop() + } + } + blockingThread.setDaemon(true) + blockingThread.start() + } + + def getPort: Int = port + + def stop(): Unit = { + if (channelFuture != null) { + channelFuture.channel().close().awaitUninterruptibly() + channelFuture = null + } + if (bossGroup != null) { + bossGroup.shutdownGracefully() + bossGroup = null + } + if (workerGroup != null) { + workerGroup.shutdownGracefully() + workerGroup = null + } + } +} + diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala similarity index 58% rename from core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala rename to core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala index cc70bd0c5c477..aaa2f913d0269 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala @@ -15,26 +15,20 @@ * limitations under the License. */ -package org.apache.spark.network.netty.server +package org.apache.spark.network.netty import io.netty.channel.ChannelInitializer import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.LineBasedFrameDecoder +import io.netty.handler.codec.{DelimiterBasedFrameDecoder, Delimiters} import io.netty.handler.codec.string.StringDecoder -import io.netty.util.CharsetUtil -import org.apache.spark.storage.BlockDataProvider - -/** Channel initializer that sets up the pipeline for the BlockServer. */ -private[netty] -class BlockServerChannelInitializer(dataProvider: BlockDataProvider) +class FileServerChannelInitializer(pResolver: PathResolver) extends ChannelInitializer[SocketChannel] { - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) - .addLast("blockHeaderEncoder", new BlockHeaderEncoder) - .addLast("handler", new BlockServerHandler(dataProvider)) + override def initChannel(channel: SocketChannel): Unit = { + channel.pipeline + .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter : _*)) + .addLast("stringDecoder", new StringDecoder) + .addLast("handler", new FileServerHandler(pResolver)) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala new file mode 100644 index 0000000000000..96f60b2883ad9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.io.FileInputStream + +import io.netty.channel.{DefaultFileRegion, ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging +import org.apache.spark.storage.{BlockId, FileSegment} + + +class FileServerHandler(pResolver: PathResolver) + extends SimpleChannelInboundHandler[String] with Logging { + + override def channelRead0(ctx: ChannelHandlerContext, blockIdString: String): Unit = { + val blockId: BlockId = BlockId(blockIdString) + val fileSegment: FileSegment = pResolver.getBlockLocation(blockId) + if (fileSegment == null) { + return + } + val file = fileSegment.file + if (file.exists) { + if (!file.isFile) { + ctx.write(new FileHeader(0, blockId).buffer) + ctx.flush() + return + } + val length: Long = fileSegment.length + if (length > Integer.MAX_VALUE || length <= 0) { + ctx.write(new FileHeader(0, blockId).buffer) + ctx.flush() + return + } + ctx.write(new FileHeader(length.toInt, blockId).buffer) + try { + val channel = new FileInputStream(file).getChannel + ctx.write(new DefaultFileRegion(channel, fileSegment.offset, fileSegment.length)) + } catch { + case e: Exception => + logError("Exception: ", e) + } + } else { + ctx.write(new FileHeader(0, blockId).buffer) + } + ctx.flush() + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + logError("Exception: ", cause) + ctx.close() + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala deleted file mode 100644 index b5870152c5a64..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import org.apache.spark.SparkConf - -/** - * A central location that tracks all the settings we exposed to users. - */ -private[spark] -class NettyConfig(conf: SparkConf) { - - /** Port the server listens on. Default to a random port. */ - private[netty] val serverPort = conf.getInt("spark.shuffle.io.port", 0) - - /** IO mode: nio, oio, epoll, or auto (try epoll first and then nio). */ - private[netty] val ioMode = conf.get("spark.shuffle.io.mode", "nio").toLowerCase - - /** Connect timeout in secs. Default 60 secs. */ - private[netty] val connectTimeoutMs = conf.getInt("spark.shuffle.io.connectionTimeout", 60) * 1000 - - /** - * Percentage of the desired amount of time spent for I/O in the child event loops. - * Only applicable in nio and epoll. - */ - private[netty] val ioRatio = conf.getInt("spark.shuffle.io.netty.ioRatio", 80) - - /** Requested maximum length of the queue of incoming connections. */ - private[netty] val backLog: Option[Int] = conf.getOption("spark.shuffle.io.backLog").map(_.toInt) - - /** - * Receive buffer size (SO_RCVBUF). - * Note: the optimal size for receive buffer and send buffer should be - * latency * network_bandwidth. - * Assuming latency = 1ms, network_bandwidth = 10Gbps - * buffer size should be ~ 1.25MB - */ - private[netty] val receiveBuf: Option[Int] = - conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) - - /** Send buffer size (SO_SNDBUF). */ - private[netty] val sendBuf: Option[Int] = - conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala new file mode 100644 index 0000000000000..e7b2855e1ec91 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.util.concurrent.Executors + +import scala.collection.JavaConverters._ + +import io.netty.buffer.ByteBuf +import io.netty.channel.ChannelHandlerContext +import io.netty.util.CharsetUtil + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.network.ConnectionManagerId +import org.apache.spark.storage.BlockId + +private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { + + def getBlock(host: String, port: Int, blockId: BlockId, + resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { + + val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) + val connectTimeout = conf.getInt("spark.shuffle.netty.connect.timeout", 60000) + val fc = new FileClient(handler, connectTimeout) + + try { + fc.init() + fc.connect(host, port) + fc.sendRequest(blockId.name) + fc.waitForClose() + fc.close() + } catch { + // Handle any socket-related exceptions in FileClient + case e: Exception => { + logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + " failed", e) + handler.handleError(blockId) + } + } + } + + def getBlock(cmId: ConnectionManagerId, blockId: BlockId, + resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { + getBlock(cmId.host, cmId.port, blockId, resultCollectCallback) + } + + def getBlocks(cmId: ConnectionManagerId, + blocks: Seq[(BlockId, Long)], + resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { + + for ((blockId, size) <- blocks) { + getBlock(cmId, blockId, resultCollectCallback) + } + } +} + + +private[spark] object ShuffleCopier extends Logging { + + private class ShuffleClientHandler(resultCollectCallBack: (BlockId, Long, ByteBuf) => Unit) + extends FileClientHandler with Logging { + + override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { + logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)") + resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) + } + + override def handleError(blockId: BlockId) { + if (!isComplete) { + resultCollectCallBack(blockId, -1, null) + } + } + } + + def echoResultCollectCallBack(blockId: BlockId, size: Long, content: ByteBuf) { + if (size != -1) { + logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") + } + } + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: ShuffleCopier ") + System.exit(1) + } + val host = args(0) + val port = args(1).toInt + val blockId = BlockId(args(2)) + val threads = if (args.length > 3) args(3).toInt else 10 + + val copiers = Executors.newFixedThreadPool(80) + val tasks = (for (i <- Range(0, threads)) yield { + Executors.callable(new Runnable() { + def run() { + val copier = new ShuffleCopier(new SparkConf) + copier.getBlock(host, port, blockId, echoResultCollectCallBack) + } + }) + }).asJava + copiers.invokeAll(tasks) + copiers.shutdown() + System.exit(0) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala new file mode 100644 index 0000000000000..95958e30f7eeb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.io.File + +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.storage.{BlockId, FileSegment} + +private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { + + val server = new FileServer(pResolver, portIn) + server.start() + + def stop() { + server.stop() + } + + def port: Int = server.getPort +} + + +/** + * An application for testing the shuffle sender as a standalone program. + */ +private[spark] object ShuffleSender { + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println( + "Usage: ShuffleSender ") + System.exit(1) + } + + val port = args(0).toInt + val subDirsPerLocalDir = args(1).toInt + val localDirs = args.drop(2).map(new File(_)) + + val pResovler = new PathResolver { + override def getBlockLocation(blockId: BlockId): FileSegment = { + if (!blockId.isShuffle) { + throw new Exception("Block " + blockId + " is not a shuffle block") + } + // Figure out which local directory it hashes to, and which subdirectory in that + val hash = Utils.nonNegativeHash(blockId) + val dirId = hash % localDirs.length + val subDirId = (hash / localDirs.length) % subDirsPerLocalDir + val subDir = new File(localDirs(dirId), "%02x".format(subDirId)) + val file = new File(subDir, blockId.name) + new FileSegment(file, 0, file.length()) + } + } + val sender = new ShuffleSender(port, pResovler) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala deleted file mode 100644 index 9fed11b75c342..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala +++ /dev/null @@ -1,135 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import java.util.concurrent.TimeoutException - -import io.netty.bootstrap.Bootstrap -import io.netty.buffer.PooledByteBufAllocator -import io.netty.channel.socket.SocketChannel -import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} -import io.netty.handler.codec.LengthFieldBasedFrameDecoder -import io.netty.handler.codec.string.StringEncoder -import io.netty.util.CharsetUtil - -import org.apache.spark.Logging - -/** - * Client for fetching data blocks from [[org.apache.spark.network.netty.server.BlockServer]]. - * Use [[BlockFetchingClientFactory]] to instantiate this client. - * - * The constructor blocks until a connection is successfully established. - * - * See [[org.apache.spark.network.netty.server.BlockServer]] for client/server protocol. - * - * Concurrency: [[BlockFetchingClient]] is not thread safe and should not be shared. - */ -@throws[TimeoutException] -private[spark] -class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, port: Int) - extends Logging { - - val handler = new BlockFetchingClientHandler - - /** Netty Bootstrap for creating the TCP connection. */ - private val bootstrap: Bootstrap = { - val b = new Bootstrap - b.group(factory.workerGroup) - .channel(factory.socketChannelClass) - // Use pooled buffers to reduce temporary buffer allocation - .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - // Disable Nagle's Algorithm since we don't want packets to wait - .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) - .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) - .option[Integer](ChannelOption.CONNECT_TIMEOUT_MILLIS, factory.conf.connectTimeoutMs) - - b.handler(new ChannelInitializer[SocketChannel] { - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("encoder", new StringEncoder(CharsetUtil.UTF_8)) - // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 - .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) - .addLast("handler", handler) - } - }) - b - } - - /** Netty ChannelFuture for the connection. */ - private val cf: ChannelFuture = bootstrap.connect(hostname, port) - if (!cf.awaitUninterruptibly(factory.conf.connectTimeoutMs)) { - throw new TimeoutException( - s"Connecting to $hostname:$port timed out (${factory.conf.connectTimeoutMs} ms)") - } - - /** - * Ask the remote server for a sequence of blocks, and execute the callback. - * - * Note that this is asynchronous and returns immediately. Upstream caller should throttle the - * rate of fetching; otherwise we could run out of memory. - * - * @param blockIds sequence of block ids to fetch. - * @param blockFetchSuccessCallback callback function when a block is successfully fetched. - * First argument is the block id, and second argument is the - * raw data in a ByteBuffer. - * @param blockFetchFailureCallback callback function when we failed to fetch any of the blocks. - * First argument is the block id, and second argument is the - * error message. - */ - def fetchBlocks( - blockIds: Seq[String], - blockFetchSuccessCallback: (String, ReferenceCountedBuffer) => Unit, - blockFetchFailureCallback: (String, String) => Unit): Unit = { - // It's best to limit the number of "write" calls since it needs to traverse the whole pipeline. - // It's also best to limit the number of "flush" calls since it requires system calls. - // Let's concatenate the string and then call writeAndFlush once. - // This is also why this implementation might be more efficient than multiple, separate - // fetch block calls. - var startTime: Long = 0 - logTrace { - startTime = System.nanoTime - s"Sending request $blockIds to $hostname:$port" - } - - // TODO: This is not the most elegant way to handle this ... - handler.blockFetchSuccessCallback = blockFetchSuccessCallback - handler.blockFetchFailureCallback = blockFetchFailureCallback - - val writeFuture = cf.channel().writeAndFlush(blockIds.mkString("\n") + "\n") - writeFuture.addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture): Unit = { - if (future.isSuccess) { - logTrace { - val timeTaken = (System.nanoTime - startTime).toDouble / 1000000 - s"Sending request $blockIds to $hostname:$port took $timeTaken ms" - } - } else { - // Fail all blocks. - logError(s"Failed to send request $blockIds to $hostname:$port", future.cause) - blockIds.foreach(blockFetchFailureCallback(_, future.cause.getMessage)) - } - } - }) - } - - def waitForClose(): Unit = { - cf.channel().closeFuture().sync() - } - - def close(): Unit = cf.channel().close() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala deleted file mode 100644 index 2b28402c52b49..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import io.netty.channel.epoll.{EpollEventLoopGroup, EpollSocketChannel} -import io.netty.channel.nio.NioEventLoopGroup -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.nio.NioSocketChannel -import io.netty.channel.socket.oio.OioSocketChannel -import io.netty.channel.{EventLoopGroup, Channel} - -import org.apache.spark.SparkConf -import org.apache.spark.network.netty.NettyConfig -import org.apache.spark.util.Utils - -/** - * Factory for creating [[BlockFetchingClient]] by using createClient. This factory reuses - * the worker thread pool for Netty. - * - * Concurrency: createClient is safe to be called from multiple threads concurrently. - */ -private[spark] -class BlockFetchingClientFactory(val conf: NettyConfig) { - - def this(sparkConf: SparkConf) = this(new NettyConfig(sparkConf)) - - /** A thread factory so the threads are named (for debugging). */ - val threadFactory = Utils.namedThreadFactory("spark-shuffle-client") - - /** The following two are instantiated by the [[init]] method, depending ioMode. */ - var socketChannelClass: Class[_ <: Channel] = _ - var workerGroup: EventLoopGroup = _ - - init() - - /** Initialize [[socketChannelClass]] and [[workerGroup]] based on ioMode. */ - private def init(): Unit = { - def initOio(): Unit = { - socketChannelClass = classOf[OioSocketChannel] - workerGroup = new OioEventLoopGroup(0, threadFactory) - } - def initNio(): Unit = { - socketChannelClass = classOf[NioSocketChannel] - workerGroup = new NioEventLoopGroup(0, threadFactory) - } - def initEpoll(): Unit = { - socketChannelClass = classOf[EpollSocketChannel] - workerGroup = new EpollEventLoopGroup(0, threadFactory) - } - - conf.ioMode match { - case "nio" => initNio() - case "oio" => initOio() - case "epoll" => initEpoll() - case "auto" => - // For auto mode, first try epoll (only available on Linux), then nio. - try { - initEpoll() - } catch { - // TODO: Should we log the throwable? But that always happen on non-Linux systems. - // Perhaps the right thing to do is to check whether the system is Linux, and then only - // call initEpoll on Linux. - case e: Throwable => initNio() - } - } - } - - /** - * Create a new BlockFetchingClient connecting to the given remote host / port. - * - * This blocks until a connection is successfully established. - * - * Concurrency: This method is safe to call from multiple threads. - */ - def createClient(remoteHost: String, remotePort: Int): BlockFetchingClient = { - new BlockFetchingClient(this, remoteHost, remotePort) - } - - def stop(): Unit = { - if (workerGroup != null) { - workerGroup.shutdownGracefully() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala deleted file mode 100644 index a1dbf6102c080..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import io.netty.buffer.ByteBuf -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.Logging - - -/** - * Handler that processes server responses. It uses the protocol documented in - * [[org.apache.spark.network.netty.server.BlockServer]]. - */ -private[client] -class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] with Logging { - - var blockFetchSuccessCallback: (String, ReferenceCountedBuffer) => Unit = _ - var blockFetchFailureCallback: (String, String) => Unit = _ - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) - ctx.close() - } - - override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { - val totalLen = in.readInt() - val blockIdLen = in.readInt() - val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) - in.readBytes(blockIdBytes) - val blockId = new String(blockIdBytes) - val blockSize = totalLen - math.abs(blockIdLen) - 4 - - def server = ctx.channel.remoteAddress.toString - - // blockIdLen is negative when it is an error message. - if (blockIdLen < 0) { - val errorMessageBytes = new Array[Byte](blockSize) - in.readBytes(errorMessageBytes) - val errorMsg = new String(errorMessageBytes) - logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") - blockFetchFailureCallback(blockId, errorMsg) - } else { - logTrace(s"Received block $blockId ($blockSize B) from $server") - blockFetchSuccessCallback(blockId, new ReferenceCountedBuffer(in)) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala deleted file mode 100644 index 9740ee64d1f2d..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -/** - * A simple iterator that lazily initializes the underlying iterator. - * - * The use case is that sometimes we might have many iterators open at the same time, and each of - * the iterator might initialize its own buffer (e.g. decompression buffer, deserialization buffer). - * This could lead to too many buffers open. If this iterator is used, we lazily initialize those - * buffers. - */ -private[spark] -class LazyInitIterator(createIterator: => Iterator[Any]) extends Iterator[Any] { - - lazy val proxy = createIterator - - override def hasNext: Boolean = { - val gotNext = proxy.hasNext - if (!gotNext) { - close() - } - gotNext - } - - override def next(): Any = proxy.next() - - def close(): Unit = Unit -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala deleted file mode 100644 index ea1abf5eccc26..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import java.io.InputStream -import java.nio.ByteBuffer - -import io.netty.buffer.{ByteBuf, ByteBufInputStream} - - -/** - * A buffer abstraction based on Netty's ByteBuf so we don't expose Netty. - * This is a Scala value class. - * - * The buffer's life cycle is NOT managed by the JVM, and thus requiring explicit declaration of - * reference by the retain method and release method. - */ -private[spark] -class ReferenceCountedBuffer(val underlying: ByteBuf) extends AnyVal { - - /** Return the nio ByteBuffer view of the underlying buffer. */ - def byteBuffer(): ByteBuffer = underlying.nioBuffer - - /** Creates a new input stream that starts from the current position of the buffer. */ - def inputStream(): InputStream = new ByteBufInputStream(underlying) - - /** Increment the reference counter by one. */ - def retain(): Unit = underlying.retain() - - /** Decrement the reference counter by one and release the buffer if the ref count is 0. */ - def release(): Unit = underlying.release() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala deleted file mode 100644 index 162e9cc6828d4..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -/** - * Header describing a block. This is used only in the server pipeline. - * - * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it. - * - * @param blockSize length of the block content, excluding the length itself. - * If positive, this is the header for a block (not part of the header). - * If negative, this is the header and content for an error message. - * @param blockId block id - * @param error some error message from reading the block - */ -private[server] -class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None) diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala deleted file mode 100644 index 8e4dda4ef8595..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import io.netty.buffer.ByteBuf -import io.netty.channel.ChannelHandlerContext -import io.netty.handler.codec.MessageToByteEncoder - -/** - * A simple encoder for BlockHeader. See [[BlockServer]] for the server to client protocol. - */ -private[server] -class BlockHeaderEncoder extends MessageToByteEncoder[BlockHeader] { - override def encode(ctx: ChannelHandlerContext, msg: BlockHeader, out: ByteBuf): Unit = { - // message = message length (4 bytes) + block id length (4 bytes) + block id + block data - // message length = block id length (4 bytes) + size of block id + size of block data - val blockIdBytes = msg.blockId.getBytes - msg.error match { - case Some(errorMsg) => - val errorBytes = errorMsg.getBytes - out.writeInt(4 + blockIdBytes.length + errorBytes.size) - out.writeInt(-blockIdBytes.length) // use negative block id length to represent errors - out.writeBytes(blockIdBytes) // next is blockId itself - out.writeBytes(errorBytes) // error message - case None => - out.writeInt(4 + blockIdBytes.length + msg.blockSize) - out.writeInt(blockIdBytes.length) // First 4 bytes is blockId length - out.writeBytes(blockIdBytes) // next is blockId itself - // msg of size blockSize will be written by ServerHandler - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala deleted file mode 100644 index 7b2f9a8d4dfd0..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import java.net.InetSocketAddress - -import io.netty.bootstrap.ServerBootstrap -import io.netty.buffer.PooledByteBufAllocator -import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} -import io.netty.channel.epoll.{EpollEventLoopGroup, EpollServerSocketChannel} -import io.netty.channel.nio.NioEventLoopGroup -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.SocketChannel -import io.netty.channel.socket.nio.NioServerSocketChannel -import io.netty.channel.socket.oio.OioServerSocketChannel -import io.netty.handler.codec.LineBasedFrameDecoder -import io.netty.handler.codec.string.StringDecoder -import io.netty.util.CharsetUtil - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.network.netty.NettyConfig -import org.apache.spark.storage.BlockDataProvider -import org.apache.spark.util.Utils - - -/** - * Server for serving Spark data blocks. - * This should be used together with [[org.apache.spark.network.netty.client.BlockFetchingClient]]. - * - * Protocol for requesting blocks (client to server): - * One block id per line, e.g. to request 3 blocks: "block1\nblock2\nblock3\n" - * - * Protocol for sending blocks (server to client): - * frame-length (4 bytes), block-id-length (4 bytes), block-id, block-data. - * - * frame-length should not include the length of itself. - * If block-id-length is negative, then this is an error message rather than block-data. The real - * length is the absolute value of the frame-length. - * - */ -private[spark] -class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Logging { - - def this(sparkConf: SparkConf, dataProvider: BlockDataProvider) = { - this(new NettyConfig(sparkConf), dataProvider) - } - - def port: Int = _port - - def hostName: String = _hostName - - private var _port: Int = conf.serverPort - private var _hostName: String = "" - private var bootstrap: ServerBootstrap = _ - private var channelFuture: ChannelFuture = _ - - init() - - /** Initialize the server. */ - private def init(): Unit = { - bootstrap = new ServerBootstrap - val bossThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-boss") - val workerThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-worker") - - // Use only one thread to accept connections, and 2 * num_cores for worker. - def initNio(): Unit = { - val bossGroup = new NioEventLoopGroup(1, bossThreadFactory) - val workerGroup = new NioEventLoopGroup(0, workerThreadFactory) - workerGroup.setIoRatio(conf.ioRatio) - bootstrap.group(bossGroup, workerGroup).channel(classOf[NioServerSocketChannel]) - } - def initOio(): Unit = { - val bossGroup = new OioEventLoopGroup(1, bossThreadFactory) - val workerGroup = new OioEventLoopGroup(0, workerThreadFactory) - bootstrap.group(bossGroup, workerGroup).channel(classOf[OioServerSocketChannel]) - } - def initEpoll(): Unit = { - val bossGroup = new EpollEventLoopGroup(1, bossThreadFactory) - val workerGroup = new EpollEventLoopGroup(0, workerThreadFactory) - workerGroup.setIoRatio(conf.ioRatio) - bootstrap.group(bossGroup, workerGroup).channel(classOf[EpollServerSocketChannel]) - } - - conf.ioMode match { - case "nio" => initNio() - case "oio" => initOio() - case "epoll" => initEpoll() - case "auto" => - // For auto mode, first try epoll (only available on Linux), then nio. - try { - initEpoll() - } catch { - // TODO: Should we log the throwable? But that always happen on non-Linux systems. - // Perhaps the right thing to do is to check whether the system is Linux, and then only - // call initEpoll on Linux. - case e: Throwable => initNio() - } - } - - // Use pooled buffers to reduce temporary buffer allocation - bootstrap.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - - // Various (advanced) user-configured settings. - conf.backLog.foreach { backLog => - bootstrap.option[java.lang.Integer](ChannelOption.SO_BACKLOG, backLog) - } - conf.receiveBuf.foreach { receiveBuf => - bootstrap.option[java.lang.Integer](ChannelOption.SO_RCVBUF, receiveBuf) - } - conf.sendBuf.foreach { sendBuf => - bootstrap.option[java.lang.Integer](ChannelOption.SO_SNDBUF, sendBuf) - } - - bootstrap.childHandler(new ChannelInitializer[SocketChannel] { - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) - .addLast("blockHeaderEncoder", new BlockHeaderEncoder) - .addLast("handler", new BlockServerHandler(dataProvider)) - } - }) - - channelFuture = bootstrap.bind(new InetSocketAddress(_port)) - channelFuture.sync() - - val addr = channelFuture.channel.localAddress.asInstanceOf[InetSocketAddress] - _port = addr.getPort - _hostName = addr.getHostName - } - - /** Shutdown the server. */ - def stop(): Unit = { - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly() - channelFuture = null - } - if (bootstrap != null && bootstrap.group() != null) { - bootstrap.group().shutdownGracefully() - } - if (bootstrap != null && bootstrap.childGroup() != null) { - bootstrap.childGroup().shutdownGracefully() - } - bootstrap = null - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala deleted file mode 100644 index 40dd5e5d1a2ac..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import java.io.FileInputStream -import java.nio.ByteBuffer -import java.nio.channels.FileChannel - -import io.netty.buffer.Unpooled -import io.netty.channel._ - -import org.apache.spark.Logging -import org.apache.spark.storage.{FileSegment, BlockDataProvider} - - -/** - * A handler that processes requests from clients and writes block data back. - * - * The messages should have been processed by a LineBasedFrameDecoder and a StringDecoder first - * so channelRead0 is called once per line (i.e. per block id). - */ -private[server] -class BlockServerHandler(dataProvider: BlockDataProvider) - extends SimpleChannelInboundHandler[String] with Logging { - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) - ctx.close() - } - - override def channelRead0(ctx: ChannelHandlerContext, blockId: String): Unit = { - def client = ctx.channel.remoteAddress.toString - - // A helper function to send error message back to the client. - def respondWithError(error: String): Unit = { - ctx.writeAndFlush(new BlockHeader(-1, blockId, Some(error))).addListener( - new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (!future.isSuccess) { - // TODO: Maybe log the success case as well. - logError(s"Error sending error back to $client", future.cause) - ctx.close() - } - } - } - ) - } - - def writeFileSegment(segment: FileSegment): Unit = { - // Send error message back if the block is too large. Even though we are capable of sending - // large (2G+) blocks, the receiving end cannot handle it so let's fail fast. - // Once we fixed the receiving end to be able to process large blocks, this should be removed. - // Also make sure we update BlockHeaderEncoder to support length > 2G. - - // See [[BlockHeaderEncoder]] for the way length is encoded. - if (segment.length + blockId.length + 4 > Int.MaxValue) { - respondWithError(s"Block $blockId size ($segment.length) greater than 2G") - return - } - - var fileChannel: FileChannel = null - try { - fileChannel = new FileInputStream(segment.file).getChannel - } catch { - case e: Exception => - logError( - s"Error opening channel for $blockId in ${segment.file} for request from $client", e) - respondWithError(e.getMessage) - } - - // Found the block. Send it back. - if (fileChannel != null) { - // Write the header and block data. In the case of failures, the listener on the block data - // write should close the connection. - ctx.write(new BlockHeader(segment.length.toInt, blockId)) - - val region = new DefaultFileRegion(fileChannel, segment.offset, segment.length) - ctx.writeAndFlush(region).addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (future.isSuccess) { - logTrace(s"Sent block $blockId (${segment.length} B) back to $client") - } else { - logError(s"Error sending block $blockId to $client; closing connection", future.cause) - ctx.close() - } - } - }) - } - } - - def writeByteBuffer(buf: ByteBuffer): Unit = { - ctx.write(new BlockHeader(buf.remaining, blockId)) - ctx.writeAndFlush(Unpooled.wrappedBuffer(buf)).addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (future.isSuccess) { - logTrace(s"Sent block $blockId (${buf.remaining} B) back to $client") - } else { - logError(s"Error sending block $blockId to $client; closing connection", future.cause) - ctx.close() - } - } - }) - } - - logTrace(s"Received request from $client to fetch block $blockId") - - var blockData: Either[FileSegment, ByteBuffer] = null - - // First make sure we can find the block. If not, send error back to the user. - try { - blockData = dataProvider.getBlockData(blockId) - } catch { - case e: Exception => - logError(s"Error opening block $blockId for request from $client", e) - respondWithError(e.getMessage) - return - } - - blockData match { - case Left(segment) => writeFileSegment(segment) - case Right(buf) => writeByteBuffer(buf) - } - - } // end of channelRead0 -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 91c0f47d51d02..5f44f5f3197fd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -18,17 +18,19 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue -import org.apache.spark.network.netty.client.{LazyInitIterator, ReferenceCountedBuffer} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue import scala.util.{Failure, Success} +import io.netty.buffer.ByteBuf + import org.apache.spark.{Logging, SparkException} import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId +import org.apache.spark.network.netty.ShuffleCopier import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -52,28 +54,18 @@ trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] wi private[storage] object BlockFetcherIterator { - /** - * A request to fetch blocks from a remote BlockManager. - * @param address remote BlockManager to fetch from. - * @param blocks Sequence of tuple, where the first element is the block id, - * and the second element is the estimated size, used to calculate bytesInFlight. - */ + // A request to fetch one or more blocks, complete with their sizes class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { val size = blocks.map(_._2).sum } - /** - * Result of a fetch from a remote block. A failure is represented as size == -1. - * @param blockId block id - * @param size estimated size of the block, used to calculate bytesInFlight. - * Note that this is NOT the exact bytes. - * @param deserialize closure to return the result in the form of an Iterator. - */ + // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize + // the block (since we want all deserializaton to happen in the calling thread); can also + // represent a fetch failure if size == -1. class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { def failed: Boolean = size == -1 } - // TODO: Refactor this whole thing to make code more reusable. class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], @@ -103,10 +95,10 @@ object BlockFetcherIterator { // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight - protected val fetchRequests = new Queue[FetchRequest] + private val fetchRequests = new Queue[FetchRequest] // Current bytes in flight from our requests - protected var bytesInFlight = 0L + private var bytesInFlight = 0L protected def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( @@ -270,55 +262,77 @@ object BlockFetcherIterator { readMetrics: ShuffleReadMetrics) extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer, readMetrics) { - override protected def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) - val cmId = new ConnectionManagerId(req.address.host, req.address.port) + import blockManager._ - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - - // This could throw a TimeoutException. In that case we will just retry the task. - val client = blockManager.nettyBlockClientFactory.createClient( - cmId.host, req.address.nettyPort) - val blocks = req.blocks.map(_._1.toString) - - client.fetchBlocks( - blocks, - (blockId: String, refBuf: ReferenceCountedBuffer) => { - // Increment the reference count so the buffer won't be recycled. - // TODO: This could result in memory leaks when the task is stopped due to exception - // before the iterator is exhausted. - refBuf.retain() - val buf = refBuf.byteBuffer() - val blockSize = buf.remaining() - val bid = BlockId(blockId) - - // TODO: remove code duplication between here and BlockManager.dataDeserialization. - results.put(new FetchResult(bid, sizeMap(bid), () => { - def createIterator: Iterator[Any] = { - val stream = blockManager.wrapForCompression(bid, refBuf.inputStream()) - serializer.newInstance().deserializeStream(stream).asIterator - } - new LazyInitIterator(createIterator) { - // Release the buffer when we are done traversing it. - override def close(): Unit = refBuf.release() + val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] + + private def startCopiers(numCopiers: Int): List[_ <: Thread] = { + (for ( i <- Range(0,numCopiers) ) yield { + val copier = new Thread { + override def run(){ + try { + while(!isInterrupted && !fetchRequestsSync.isEmpty) { + sendRequest(fetchRequestsSync.take()) + } + } catch { + case x: InterruptedException => logInfo("Copier Interrupted") + // case _ => throw new SparkException("Exception Throw in Shuffle Copier") } - })) - - readMetrics.synchronized { - readMetrics.remoteBytesRead += blockSize - readMetrics.remoteBlocksFetched += 1 - } - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - }, - (blockId: String, errorMsg: String) => { - logError(s"Could not get block(s) from $cmId with error: $errorMsg") - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) } } - ) + copier.start + copier + }).toList + } + + // keep this to interrupt the threads when necessary + private def stopCopiers() { + for (copier <- copiers) { + copier.interrupt() + } + } + + override protected def sendRequest(req: FetchRequest) { + + def putResult(blockId: BlockId, blockSize: Long, blockData: ByteBuf) { + val fetchResult = new FetchResult(blockId, blockSize, + () => dataDeserialize(blockId, blockData.nioBuffer, serializer)) + results.put(fetchResult) + } + + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.host)) + val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) + val cpier = new ShuffleCopier(blockManager.conf) + cpier.getBlocks(cmId, req.blocks, putResult) + logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) + } + + private var copiers: List[_ <: Thread] = null + + override def initialize() { + // Split Local Remote Blocks and set numBlocksToFetch + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + for (request <- Utils.randomize(remoteRequests)) { + fetchRequestsSync.put(request) + } + + copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) + logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + startTime = System.currentTimeMillis + getLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def next(): (BlockId, Option[Iterator[Any]]) = { + resultsGotten += 1 + val result = results.take() + // If all the results has been retrieved, copiers will exit automatically + (result.blockId, if (result.failed) None else Some(result.deserialize())) } } // End of NettyBlockFetcherIterator diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e67676950b0ed..e8bbd298c631a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -25,19 +25,16 @@ import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random -import akka.actor.{ActorSystem, Props} +import akka.actor.{ActorSystem, Cancellable, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ -import org.apache.spark.network.netty.client.BlockFetchingClientFactory -import org.apache.spark.network.netty.server.BlockServer import org.apache.spark.serializer.Serializer import org.apache.spark.util._ - private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues @@ -61,7 +58,7 @@ private[spark] class BlockManager( val conf: SparkConf, securityManager: SecurityManager, mapOutputTracker: MapOutputTracker) - extends BlockDataProvider with Logging { + extends Logging { private val port = conf.getInt("spark.blockManager.port", 0) val shuffleBlockManager = new ShuffleBlockManager(this) @@ -89,25 +86,13 @@ private[spark] class BlockManager( new TachyonStore(this, tachyonBlockManager) } - private val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) - // If we use Netty for shuffle, start a new Netty-based shuffle sender service. - private[storage] val nettyBlockClientFactory: BlockFetchingClientFactory = { - if (useNetty) new BlockFetchingClientFactory(conf) else null + private val nettyPort: Int = { + val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) + val nettyPortConfig = conf.getInt("spark.shuffle.sender.port", 0) + if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } - private val nettyBlockServer: BlockServer = { - if (useNetty) { - val server = new BlockServer(conf, this) - logInfo(s"Created NettyBlockServer binding to port: ${server.port}") - server - } else { - null - } - } - - private val nettyPort: Int = if (useNetty) nettyBlockServer.port else 0 - val blockManagerId = BlockManagerId( executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) @@ -231,20 +216,6 @@ private[spark] class BlockManager( } } - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { - val bid = BlockId(blockId) - if (bid.isShuffle) { - Left(diskBlockManager.getBlockLocation(bid)) - } else { - val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] - if (blockBytesOpt.isDefined) { - Right(blockBytesOpt.get) - } else { - throw new BlockNotFoundException(blockId) - } - } - } - /** * Get the BlockStatus for the block identified by the given ID, if it exists. * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. @@ -1090,14 +1061,6 @@ private[spark] class BlockManager( connectionManager.stop() shuffleBlockManager.stop() diskBlockManager.stop() - - if (nettyBlockClientFactory != null) { - nettyBlockClientFactory.stop() - } - if (nettyBlockServer != null) { - nettyBlockServer.stop() - } - actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala deleted file mode 100644 index 9ef453605f4f1..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala +++ /dev/null @@ -1,21 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - - -class BlockNotFoundException(blockId: String) extends Exception(s"Block $blockId not found") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index f3da816389581..4d66ccea211fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -23,7 +23,7 @@ import java.util.{Date, Random, UUID} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.PathResolver +import org.apache.spark.network.netty.{PathResolver, ShuffleSender} import org.apache.spark.util.Utils import org.apache.spark.shuffle.sort.SortShuffleManager @@ -52,6 +52,7 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) + private var shuffleSender : ShuffleSender = null addShutdownHook() @@ -185,5 +186,15 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, } } } + + if (shuffleSender != null) { + shuffleSender.stop() + } + } + + private[storage] def startShuffleBlockSender(port: Int): Int = { + shuffleSender = new ShuffleSender(port, this) + logInfo(s"Created ShuffleSender binding to port: ${shuffleSender.port}") + shuffleSender.port } } diff --git a/core/src/test/resources/netty-test-file.txt b/core/src/test/resources/netty-test-file.txt deleted file mode 100644 index f59f293ee02ea..0000000000000 --- a/core/src/test/resources/netty-test-file.txt +++ /dev/null @@ -1,1379 +0,0 @@ -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa -bbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbbb -eeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeeee -aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala deleted file mode 100644 index ef3478a41e912..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.{RandomAccessFile, File} -import java.nio.ByteBuffer -import java.util.{Collections, HashSet} -import java.util.concurrent.{TimeUnit, Semaphore} - -import scala.collection.JavaConversions._ - -import io.netty.buffer.{ByteBufUtil, Unpooled} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.SparkConf -import org.apache.spark.network.netty.client.{ReferenceCountedBuffer, BlockFetchingClientFactory} -import org.apache.spark.network.netty.server.BlockServer -import org.apache.spark.storage.{FileSegment, BlockDataProvider} - - -/** - * Test suite that makes sure the server and the client implementations share the same protocol. - */ -class ServerClientIntegrationSuite extends FunSuite with BeforeAndAfterAll { - - val bufSize = 100000 - var buf: ByteBuffer = _ - var testFile: File = _ - var server: BlockServer = _ - var clientFactory: BlockFetchingClientFactory = _ - - val bufferBlockId = "buffer_block" - val fileBlockId = "file_block" - - val fileContent = new Array[Byte](1024) - scala.util.Random.nextBytes(fileContent) - - override def beforeAll() = { - buf = ByteBuffer.allocate(bufSize) - for (i <- 1 to bufSize) { - buf.put(i.toByte) - } - buf.flip() - - testFile = File.createTempFile("netty-test-file", "txt") - val fp = new RandomAccessFile(testFile, "rw") - fp.write(fileContent) - fp.close() - - server = new BlockServer(new SparkConf, new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { - if (blockId == bufferBlockId) { - Right(buf) - } else if (blockId == fileBlockId) { - Left(new FileSegment(testFile, 10, testFile.length - 25)) - } else { - throw new Exception("Unknown block id " + blockId) - } - } - }) - - clientFactory = new BlockFetchingClientFactory(new SparkConf) - } - - override def afterAll() = { - server.stop() - clientFactory.stop() - } - - /** A ByteBuf for buffer_block */ - lazy val byteBufferBlockReference = Unpooled.wrappedBuffer(buf) - - /** A ByteBuf for file_block */ - lazy val fileBlockReference = Unpooled.wrappedBuffer(fileContent, 10, fileContent.length - 25) - - def fetchBlocks(blockIds: Seq[String]): (Set[String], Set[ReferenceCountedBuffer], Set[String]) = - { - val client = clientFactory.createClient(server.hostName, server.port) - val sem = new Semaphore(0) - val receivedBlockIds = Collections.synchronizedSet(new HashSet[String]) - val errorBlockIds = Collections.synchronizedSet(new HashSet[String]) - val receivedBuffers = Collections.synchronizedSet(new HashSet[ReferenceCountedBuffer]) - - client.fetchBlocks( - blockIds, - (blockId, buf) => { - receivedBlockIds.add(blockId) - buf.retain() - receivedBuffers.add(buf) - sem.release() - }, - (blockId, errorMsg) => { - errorBlockIds.add(blockId) - sem.release() - } - ) - if (!sem.tryAcquire(blockIds.size, 30, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server") - } - client.close() - (receivedBlockIds.toSet, receivedBuffers.toSet, errorBlockIds.toSet) - } - - test("fetch a ByteBuffer block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId)) - assert(blockIds === Set(bufferBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch a FileSegment block via zero-copy send") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(fileBlockId)) - assert(blockIds === Set(fileBlockId)) - assert(buffers.map(_.underlying) === Set(fileBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch a non-existent block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq("random-block")) - assert(blockIds.isEmpty) - assert(buffers.isEmpty) - assert(failBlockIds === Set("random-block")) - } - - test("fetch both ByteBuffer block and FileSegment block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, fileBlockId)) - assert(blockIds === Set(bufferBlockId, fileBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference, fileBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch both ByteBuffer block and a non-existent block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, "random-block")) - assert(blockIds === Set(bufferBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) - assert(failBlockIds === Set("random-block")) - buffers.foreach(_.release()) - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala deleted file mode 100644 index 9afdad63b6988..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import java.nio.ByteBuffer - -import io.netty.buffer.Unpooled -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.FunSuite - - -class BlockFetchingClientHandlerSuite extends FunSuite { - - test("handling block data (successful fetch)") { - val blockId = "test_block" - val blockData = "blahblahblahblahblah" - val totalLength = 4 + blockId.length + blockData.length - - var parsedBlockId: String = "" - var parsedBlockData: String = "" - val handler = new BlockFetchingClientHandler - handler.blockFetchSuccessCallback = (bid, refCntBuf) => { - parsedBlockId = bid - val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) - refCntBuf.byteBuffer().get(bytes) - parsedBlockData = new String(bytes) - } - - val channel = new EmbeddedChannel(handler) - val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself - buf.putInt(totalLength) - buf.putInt(blockId.length) - buf.put(blockId.getBytes) - buf.put(blockData.getBytes) - buf.flip() - - channel.writeInbound(Unpooled.wrappedBuffer(buf)) - assert(parsedBlockId === blockId) - assert(parsedBlockData === blockData) - - channel.close() - } - - test("handling error message (failed fetch)") { - val blockId = "test_block" - val errorMsg = "error erro5r error err4or error3 error6 error erro1r" - val totalLength = 4 + blockId.length + errorMsg.length - - var parsedBlockId: String = "" - var parsedErrorMsg: String = "" - val handler = new BlockFetchingClientHandler - handler.blockFetchFailureCallback = (bid, msg) => { - parsedBlockId = bid - parsedErrorMsg = msg - } - - val channel = new EmbeddedChannel(handler) - val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself - buf.putInt(totalLength) - buf.putInt(-blockId.length) - buf.put(blockId.getBytes) - buf.put(errorMsg.getBytes) - buf.flip() - - channel.writeInbound(Unpooled.wrappedBuffer(buf)) - assert(parsedBlockId === blockId) - assert(parsedErrorMsg === errorMsg) - - channel.close() - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala deleted file mode 100644 index 3ee281cb1350b..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import io.netty.buffer.ByteBuf -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.FunSuite - - -class BlockHeaderEncoderSuite extends FunSuite { - - test("encode normal block data") { - val blockId = "test_block" - val channel = new EmbeddedChannel(new BlockHeaderEncoder) - channel.writeOutbound(new BlockHeader(17, blockId, None)) - val out = channel.readOutbound().asInstanceOf[ByteBuf] - assert(out.readInt() === 4 + blockId.length + 17) - assert(out.readInt() === blockId.length) - - val blockIdBytes = new Array[Byte](blockId.length) - out.readBytes(blockIdBytes) - assert(new String(blockIdBytes) === blockId) - assert(out.readableBytes() === 0) - - channel.close() - } - - test("encode error message") { - val blockId = "error_block" - val errorMsg = "error encountered" - val channel = new EmbeddedChannel(new BlockHeaderEncoder) - channel.writeOutbound(new BlockHeader(17, blockId, Some(errorMsg))) - val out = channel.readOutbound().asInstanceOf[ByteBuf] - assert(out.readInt() === 4 + blockId.length + errorMsg.length) - assert(out.readInt() === -blockId.length) - - val blockIdBytes = new Array[Byte](blockId.length) - out.readBytes(blockIdBytes) - assert(new String(blockIdBytes) === blockId) - - val errorMsgBytes = new Array[Byte](errorMsg.length) - out.readBytes(errorMsgBytes) - assert(new String(errorMsgBytes) === errorMsg) - assert(out.readableBytes() === 0) - - channel.close() - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala deleted file mode 100644 index 12f6d87616644..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala +++ /dev/null @@ -1,101 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import java.io.File -import java.nio.ByteBuffer - -import io.netty.buffer.{Unpooled, ByteBuf} -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler, DefaultFileRegion} -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.FunSuite - -import org.apache.spark.storage.{BlockDataProvider, FileSegment} - - -class BlockServerHandlerSuite extends FunSuite { - - test("ByteBuffer block") { - val expectedBlockId = "test_bytebuffer_block" - val buf = ByteBuffer.allocate(10000) - for (i <- 1 to 10000) { - buf.put(i.toByte) - } - buf.flip() - - val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = Right(buf) - })) - - channel.writeInbound(expectedBlockId) - assert(channel.outboundMessages().size === 2) - - val out1 = channel.readOutbound().asInstanceOf[BlockHeader] - val out2 = channel.readOutbound().asInstanceOf[ByteBuf] - - assert(out1.blockId === expectedBlockId) - assert(out1.blockSize === buf.remaining) - assert(out1.error === None) - - assert(out2.equals(Unpooled.wrappedBuffer(buf))) - - channel.close() - } - - test("FileSegment block via zero-copy") { - val expectedBlockId = "test_file_block" - val url = Thread.currentThread.getContextClassLoader.getResource("netty-test-file.txt") - val testFile = new File(url.toURI) - - val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { - Left(new FileSegment(testFile, 15, testFile.length - 25)) - } - })) - - channel.writeInbound(expectedBlockId) - assert(channel.outboundMessages().size === 2) - - val out1 = channel.readOutbound().asInstanceOf[BlockHeader] - val out2 = channel.readOutbound().asInstanceOf[DefaultFileRegion] - - assert(out1.blockId === expectedBlockId) - assert(out1.blockSize === testFile.length - 25) - assert(out1.error === None) - - assert(out2.count === testFile.length - 25) - assert(out2.position === 15) - } - - test("pipeline exception propagation") { - val blockServerHandler = new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = ??? - }) - val exceptionHandler = new SimpleChannelInboundHandler[String]() { - override def channelRead0(ctx: ChannelHandlerContext, msg: String): Unit = { - throw new Exception("this is an error") - } - } - - val channel = new EmbeddedChannel(exceptionHandler, blockServerHandler) - assert(channel.isOpen) - channel.writeInbound("a message to trigger the error") - assert(!channel.isOpen) - } -} diff --git a/pom.xml b/pom.xml index 71f7610c0e450..920912353fe9c 100644 --- a/pom.xml +++ b/pom.xml @@ -420,7 +420,7 @@ io.netty netty-all - 4.0.22.Final + 4.0.17.Final org.apache.derby From 0afe5cb65a195d2f14e8dfcefdbec5dac023651f Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 15 Aug 2014 11:35:08 -0700 Subject: [PATCH 0366/1492] SPARK-3028. sparkEventToJson should support SparkListenerExecutorMetrics... ...Update Author: Sandy Ryza Closes #1961 from sryza/sandy-spark-3028 and squashes the following commits: dccdff5 [Sandy Ryza] Fix compile error f883ded [Sandy Ryza] SPARK-3028. sparkEventToJson should support SparkListenerExecutorMetricsUpdate --- .../org/apache/spark/scheduler/EventLoggingListener.scala | 2 ++ core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 406147f167bf3..7378ce923f0ae 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -127,6 +127,8 @@ private[spark] class EventLoggingListener( logEvent(event, flushLogger = true) override def onApplicationEnd(event: SparkListenerApplicationEnd) = logEvent(event, flushLogger = true) + // No-op because logging every update would be overkill + override def onExecutorMetricsUpdate(event: SparkListenerExecutorMetricsUpdate) { } /** * Stop logging events. diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 6f8eb1ee12634..1e18ec688c40d 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -72,8 +72,9 @@ private[spark] object JsonProtocol { case applicationEnd: SparkListenerApplicationEnd => applicationEndToJson(applicationEnd) - // Not used, but keeps compiler happy + // These aren't used, but keeps compiler happy case SparkListenerShutdown => JNothing + case SparkListenerExecutorMetricsUpdate(_, _) => JNothing } } From c7032290a3f0f5545aa4f0a9a144c62571344dc8 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 15 Aug 2014 14:50:10 -0700 Subject: [PATCH 0367/1492] [SPARK-3022] [SPARK-3041] [mllib] Call findBins once per level + unordered feature bug fix DecisionTree improvements: (1) TreePoint representation to avoid binning multiple times (2) Bug fix: isSampleValid indexed bins incorrectly for unordered categorical features (3) Timing for DecisionTree internals Details: (1) TreePoint representation to avoid binning multiple times [https://issues.apache.org/jira/browse/SPARK-3022] Added private[tree] TreePoint class for representing binned feature values. The input RDD of LabeledPoint is converted to the TreePoint representation initially and then cached. This avoids the previous problem of re-computing bins multiple times. (2) Bug fix: isSampleValid indexed bins incorrectly for unordered categorical features [https://issues.apache.org/jira/browse/SPARK-3041] isSampleValid used to treat unordered categorical features incorrectly: It treated the bins as if indexed by featured values, rather than by subsets of values/categories. * exhibited for unordered features (multi-class classification with categorical features of low arity) * Fix: Index bins correctly for unordered categorical features. (3) Timing for DecisionTree internals Added tree/impl/TimeTracker.scala class which is private[tree] for now, for timing key parts of DT code. Prints timing info via logDebug. CC: mengxr manishamde chouqin Very similar update, with one bug fix. Many apologies for the conflicting update, but I hope that a few more optimizations I have on the way (which depend on this update) will prove valuable to you: SPARK-3042 and SPARK-3043 Author: Joseph K. Bradley Closes #1950 from jkbradley/dt-opt1 and squashes the following commits: 5f2dec2 [Joseph K. Bradley] Fixed scalastyle issue in TreePoint 6b5651e [Joseph K. Bradley] Updates based on code review. 1 major change: persisting to memory + disk, not just memory. 2d2aaaf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals --- .../spark/mllib/tree/DecisionTree.scala | 289 ++++++++---------- .../mllib/tree/configuration/Strategy.scala | 43 ++- .../spark/mllib/tree/impl/TimeTracker.scala | 73 +++++ .../spark/mllib/tree/impl/TreePoint.scala | 201 ++++++++++++ .../spark/mllib/tree/DecisionTreeSuite.scala | 50 +-- 5 files changed, 449 insertions(+), 207 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index bb50f07be5d7b..2a3107a13e916 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -17,22 +17,24 @@ package org.apache.spark.mllib.tree -import org.apache.spark.api.java.JavaRDD - import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} +import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impurity.{Impurities, Gini, Entropy, Impurity} +import org.apache.spark.mllib.tree.impl.{TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom + /** * :: Experimental :: * A class which implements a decision tree learning algorithm for classification and regression. @@ -53,16 +55,27 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo */ def train(input: RDD[LabeledPoint]): DecisionTreeModel = { - // Cache input RDD for speedup during multiple passes. - val retaggedInput = input.retag(classOf[LabeledPoint]).cache() + val timer = new TimeTracker() + + timer.start("total") + + timer.start("init") + + val retaggedInput = input.retag(classOf[LabeledPoint]) logDebug("algo = " + strategy.algo) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. + timer.start("findSplitsBins") val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, strategy) val numBins = bins(0).length + timer.stop("findSplitsBins") logDebug("numBins = " + numBins) + // Cache input RDD for speedup during multiple passes. + val treeInput = TreePoint.convertToTreeRDD(retaggedInput, strategy, bins) + .persist(StorageLevel.MEMORY_AND_DISK) + // depth of the decision tree val maxDepth = strategy.maxDepth // the max number of nodes possible given the depth of the tree @@ -76,7 +89,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // dummy value for top node (updated during first split calculation) val nodes = new Array[Node](maxNumNodes) // num features - val numFeatures = retaggedInput.take(1)(0).features.size + val numFeatures = treeInput.take(1)(0).binnedFeatures.size // Calculate level for single group construction @@ -96,6 +109,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo (math.log(maxNumberOfNodesPerGroup) / math.log(2)).floor.toInt, 0) logDebug("max level for single group = " + maxLevelForSingleGroup) + timer.stop("init") + /* * The main idea here is to perform level-wise training of the decision tree nodes thus * reducing the passes over the data from l to log2(l) where l is the total number of nodes. @@ -113,15 +128,21 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("#####################################") // Find best split for all nodes at a level. - val splitsStatsForLevel = DecisionTree.findBestSplits(retaggedInput, parentImpurities, - strategy, level, filters, splits, bins, maxLevelForSingleGroup) + timer.start("findBestSplits") + val splitsStatsForLevel = DecisionTree.findBestSplits(treeInput, parentImpurities, + strategy, level, filters, splits, bins, maxLevelForSingleGroup, timer) + timer.stop("findBestSplits") for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { + timer.start("extractNodeInfo") // Extract info for nodes at the current level. extractNodeInfo(nodeSplitStats, level, index, nodes) + timer.stop("extractNodeInfo") + timer.start("extractInfoForLowerLevels") // Extract info for nodes at the next lower level. extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, filters) + timer.stop("extractInfoForLowerLevels") logDebug("final best split = " + nodeSplitStats._1) } require(math.pow(2, level) == splitsStatsForLevel.length) @@ -144,6 +165,11 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Build the full tree using the node info calculated in the level-wise best split calculations. topNode.build(nodes) + timer.stop("total") + + logInfo("Internal timing for DecisionTree:") + logInfo(s"$timer") + new DecisionTreeModel(topNode, strategy.algo) } @@ -406,7 +432,7 @@ object DecisionTree extends Serializable with Logging { * Returns an array of optimal splits for all nodes at a given level. Splits the task into * multiple groups if the level-wise training task could lead to memory overflow. * - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing * parameters for constructing the DecisionTree @@ -415,44 +441,45 @@ object DecisionTree extends Serializable with Logging { * @param splits possible splits for all features * @param bins possible bins for all features * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. - * @return array of splits with best splits for all nodes at a given level. + * @return array (over nodes) of splits with best split for each node at a given level. */ protected[tree] def findBestSplits( - input: RDD[LabeledPoint], + input: RDD[TreePoint], parentImpurities: Array[Double], strategy: Strategy, level: Int, filters: Array[List[Filter]], splits: Array[Array[Split]], bins: Array[Array[Bin]], - maxLevelForSingleGroup: Int): Array[(Split, InformationGainStats)] = { + maxLevelForSingleGroup: Int, + timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats)] = { // split into groups to avoid memory overflow during aggregation if (level > maxLevelForSingleGroup) { // When information for all nodes at a given level cannot be stored in memory, // the nodes are divided into multiple groups at each level with the number of groups // increasing exponentially per level. For example, if maxLevelForSingleGroup is 10, // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. - val numGroups = math.pow(2, (level - maxLevelForSingleGroup)).toInt + val numGroups = math.pow(2, level - maxLevelForSingleGroup).toInt logDebug("numGroups = " + numGroups) var bestSplits = new Array[(Split, InformationGainStats)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 while (groupIndex < numGroups) { val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, strategy, level, - filters, splits, bins, numGroups, groupIndex) + filters, splits, bins, timer, numGroups, groupIndex) bestSplits = Array.concat(bestSplits, bestSplitsForGroup) groupIndex += 1 } bestSplits } else { - findBestSplitsPerGroup(input, parentImpurities, strategy, level, filters, splits, bins) + findBestSplitsPerGroup(input, parentImpurities, strategy, level, filters, splits, bins, timer) } } /** * Returns an array of optimal splits for a group of nodes at a given level * - * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing * parameters for constructing the DecisionTree @@ -465,13 +492,14 @@ object DecisionTree extends Serializable with Logging { * @return array of splits with best splits for all nodes at a given level. */ private def findBestSplitsPerGroup( - input: RDD[LabeledPoint], + input: RDD[TreePoint], parentImpurities: Array[Double], strategy: Strategy, level: Int, filters: Array[List[Filter]], splits: Array[Array[Split]], bins: Array[Array[Bin]], + timer: TimeTracker, numGroups: Int = 1, groupIndex: Int = 0): Array[(Split, InformationGainStats)] = { @@ -507,7 +535,7 @@ object DecisionTree extends Serializable with Logging { logDebug("numNodes = " + numNodes) // Find the number of features by looking at the first sample. - val numFeatures = input.first().features.size + val numFeatures = input.first().binnedFeatures.size logDebug("numFeatures = " + numFeatures) // numBins: Number of bins = 1 + number of possible splits @@ -542,33 +570,43 @@ object DecisionTree extends Serializable with Logging { * Find whether the sample is valid input for the current node, i.e., whether it passes through * all the filters for the current node. */ - def isSampleValid(parentFilters: List[Filter], labeledPoint: LabeledPoint): Boolean = { + def isSampleValid(parentFilters: List[Filter], treePoint: TreePoint): Boolean = { // leaf if ((level > 0) && (parentFilters.length == 0)) { return false } // Apply each filter and check sample validity. Return false when invalid condition found. - for (filter <- parentFilters) { - val features = labeledPoint.features + parentFilters.foreach { filter => val featureIndex = filter.split.feature - val threshold = filter.split.threshold val comparison = filter.comparison - val categories = filter.split.categories val isFeatureContinuous = filter.split.featureType == Continuous - val feature = features(featureIndex) if (isFeatureContinuous) { + val binId = treePoint.binnedFeatures(featureIndex) + val bin = bins(featureIndex)(binId) + val featureValue = bin.highSplit.threshold + val threshold = filter.split.threshold comparison match { - case -1 => if (feature > threshold) return false - case 1 => if (feature <= threshold) return false + case -1 => if (featureValue > threshold) return false + case 1 => if (featureValue <= threshold) return false } } else { - val containsFeature = categories.contains(feature) + val numFeatureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val isSpaceSufficientForAllCategoricalSplits = + numBins > math.pow(2, numFeatureCategories.toInt - 1) - 1 + val isUnorderedFeature = + isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits + val featureValue = if (isUnorderedFeature) { + treePoint.binnedFeatures(featureIndex) + } else { + val binId = treePoint.binnedFeatures(featureIndex) + bins(featureIndex)(binId).category + } + val containsFeature = filter.split.categories.contains(featureValue) comparison match { case -1 => if (!containsFeature) return false case 1 => if (containsFeature) return false } - } } @@ -576,103 +614,6 @@ object DecisionTree extends Serializable with Logging { true } - /** - * Find bin for one (labeledPoint, feature). - */ - def findBin( - featureIndex: Int, - labeledPoint: LabeledPoint, - isFeatureContinuous: Boolean, - isSpaceSufficientForAllCategoricalSplits: Boolean): Int = { - val binForFeatures = bins(featureIndex) - val feature = labeledPoint.features(featureIndex) - - /** - * Binary search helper method for continuous feature. - */ - def binarySearchForBins(): Int = { - var left = 0 - var right = binForFeatures.length - 1 - while (left <= right) { - val mid = left + (right - left) / 2 - val bin = binForFeatures(mid) - val lowThreshold = bin.lowSplit.threshold - val highThreshold = bin.highSplit.threshold - if ((lowThreshold < feature) && (highThreshold >= feature)) { - return mid - } - else if (lowThreshold >= feature) { - right = mid - 1 - } - else { - left = mid + 1 - } - } - -1 - } - - /** - * Sequential search helper method to find bin for categorical feature in multiclass - * classification. The category is returned since each category can belong to multiple - * splits. The actual left/right child allocation per split is performed in the - * sequential phase of the bin aggregate operation. - */ - def sequentialBinSearchForUnorderedCategoricalFeatureInClassification(): Int = { - labeledPoint.features(featureIndex).toInt - } - - /** - * Sequential search helper method to find bin for categorical feature - * (for classification and regression). - */ - def sequentialBinSearchForOrderedCategoricalFeature(): Int = { - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val featureValue = labeledPoint.features(featureIndex) - var binIndex = 0 - while (binIndex < featureCategories) { - val bin = bins(featureIndex)(binIndex) - val categories = bin.highSplit.categories - if (categories.contains(featureValue)) { - return binIndex - } - binIndex += 1 - } - if (featureValue < 0 || featureValue >= featureCategories) { - throw new IllegalArgumentException( - s"DecisionTree given invalid data:" + - s" Feature $featureIndex is categorical with values in" + - s" {0,...,${featureCategories - 1}," + - s" but a data point gives it value $featureValue.\n" + - " Bad data point: " + labeledPoint.toString) - } - -1 - } - - if (isFeatureContinuous) { - // Perform binary search for finding bin for continuous features. - val binIndex = binarySearchForBins() - if (binIndex == -1) { - throw new UnknownError("no bin was found for continuous variable.") - } - binIndex - } else { - // Perform sequential search to find bin for categorical features. - val binIndex = { - val isUnorderedFeature = - isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits - if (isUnorderedFeature) { - sequentialBinSearchForUnorderedCategoricalFeatureInClassification() - } else { - sequentialBinSearchForOrderedCategoricalFeature() - } - } - if (binIndex == -1) { - throw new UnknownError("no bin was found for categorical variable.") - } - binIndex - } - } - /** * Finds bins for all nodes (and all features) at a given level. * For l nodes, k features the storage is as follows: @@ -689,17 +630,17 @@ object DecisionTree extends Serializable with Logging { * bin index for this labeledPoint * (or InvalidBinIndex if labeledPoint is not handled by this node) */ - def findBinsForLevel(labeledPoint: LabeledPoint): Array[Double] = { + def findBinsForLevel(treePoint: TreePoint): Array[Double] = { // Calculate bin index and label per feature per node. val arr = new Array[Double](1 + (numFeatures * numNodes)) // First element of the array is the label of the instance. - arr(0) = labeledPoint.label + arr(0) = treePoint.label // Iterate over nodes. var nodeIndex = 0 while (nodeIndex < numNodes) { val parentFilters = findParentFilters(nodeIndex) // Find out whether the sample qualifies for the particular node. - val sampleValid = isSampleValid(parentFilters, labeledPoint) + val sampleValid = isSampleValid(parentFilters, treePoint) val shift = 1 + numFeatures * nodeIndex if (!sampleValid) { // Mark one bin as -1 is sufficient. @@ -707,19 +648,7 @@ object DecisionTree extends Serializable with Logging { } else { var featureIndex = 0 while (featureIndex < numFeatures) { - val featureInfo = strategy.categoricalFeaturesInfo.get(featureIndex) - val isFeatureContinuous = featureInfo.isEmpty - if (isFeatureContinuous) { - arr(shift + featureIndex) - = findBin(featureIndex, labeledPoint, isFeatureContinuous, false) - } else { - val featureCategories = featureInfo.get - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - arr(shift + featureIndex) - = findBin(featureIndex, labeledPoint, isFeatureContinuous, - isSpaceSufficientForAllCategoricalSplits) - } + arr(shift + featureIndex) = treePoint.binnedFeatures(featureIndex) featureIndex += 1 } } @@ -728,7 +657,8 @@ object DecisionTree extends Serializable with Logging { arr } - // Find feature bins for all nodes at a level. + // Find feature bins for all nodes at a level. + timer.start("aggregation") val binMappedRDD = input.map(x => findBinsForLevel(x)) /** @@ -830,6 +760,8 @@ object DecisionTree extends Serializable with Logging { } } + val rightChildShift = numClasses * numBins * numFeatures * numNodes + /** * Helper for binSeqOp. * @@ -853,7 +785,6 @@ object DecisionTree extends Serializable with Logging { val validSignalIndex = 1 + numFeatures * nodeIndex val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex if (isSampleValidForNode) { - val rightChildShift = numClasses * numBins * numFeatures * numNodes // actual class label val label = arr(0) // Iterate over all features. @@ -912,7 +843,7 @@ object DecisionTree extends Serializable with Logging { val aggIndex = aggShift + 3 * featureIndex * numBins + arr(arrIndex).toInt * 3 agg(aggIndex) = agg(aggIndex) + 1 agg(aggIndex + 1) = agg(aggIndex + 1) + label - agg(aggIndex + 2) = agg(aggIndex + 2) + label*label + agg(aggIndex + 2) = agg(aggIndex + 2) + label * label featureIndex += 1 } } @@ -977,6 +908,7 @@ object DecisionTree extends Serializable with Logging { val binAggregates = { binMappedRDD.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp,binCombOp) } + timer.stop("aggregation") logDebug("binAggregates.length = " + binAggregates.length) /** @@ -1031,10 +963,17 @@ object DecisionTree extends Serializable with Logging { def indexOfLargestArrayElement(array: Array[Double]): Int = { val result = array.foldLeft(-1, Double.MinValue, 0) { case ((maxIndex, maxValue, currentIndex), currentValue) => - if(currentValue > maxValue) (currentIndex, currentValue, currentIndex + 1) - else (maxIndex, maxValue, currentIndex + 1) + if (currentValue > maxValue) { + (currentIndex, currentValue, currentIndex + 1) + } else { + (maxIndex, maxValue, currentIndex + 1) + } + } + if (result._1 < 0) { + throw new RuntimeException("DecisionTree internal error:" + + " calculateGainForSplit failed in indexOfLargestArrayElement") } - if (result._1 < 0) 0 else result._1 + result._1 } val predict = indexOfLargestArrayElement(leftRightCounts) @@ -1057,6 +996,7 @@ object DecisionTree extends Serializable with Logging { val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + case Regression => val leftCount = leftNodeAgg(featureIndex)(splitIndex)(0) val leftSum = leftNodeAgg(featureIndex)(splitIndex)(1) @@ -1280,15 +1220,41 @@ object DecisionTree extends Serializable with Logging { nodeImpurity: Double): Array[Array[InformationGainStats]] = { val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1) - for (featureIndex <- 0 until numFeatures) { - for (splitIndex <- 0 until numBins - 1) { + var featureIndex = 0 + while (featureIndex < numFeatures) { + val numSplitsForFeature = getNumSplitsForFeature(featureIndex) + var splitIndex = 0 + while (splitIndex < numSplitsForFeature) { gains(featureIndex)(splitIndex) = calculateGainForSplit(leftNodeAgg, featureIndex, splitIndex, rightNodeAgg, nodeImpurity) + splitIndex += 1 } + featureIndex += 1 } gains } + /** + * Get the number of splits for a feature. + */ + def getNumSplitsForFeature(featureIndex: Int): Int = { + val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + if (isFeatureContinuous) { + numBins - 1 + } else { + // Categorical feature + val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) + val isSpaceSufficientForAllCategoricalSplits = + numBins > math.pow(2, featureCategories.toInt - 1) - 1 + if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { + math.pow(2.0, featureCategories - 1).toInt - 1 + } else { + // Ordered features + featureCategories + } + } + } + /** * Find the best split for a node. * @param binData Bin data slice for this node, given by getBinDataForNode. @@ -1307,7 +1273,7 @@ object DecisionTree extends Serializable with Logging { // Calculate gains for all splits. val gains = calculateGainsForAllNodeSplits(leftNodeAgg, rightNodeAgg, nodeImpurity) - val (bestFeatureIndex,bestSplitIndex, gainStats) = { + val (bestFeatureIndex, bestSplitIndex, gainStats) = { // Initialize with infeasible values. var bestFeatureIndex = Int.MinValue var bestSplitIndex = Int.MinValue @@ -1317,22 +1283,8 @@ object DecisionTree extends Serializable with Logging { while (featureIndex < numFeatures) { // Iterate over all splits. var splitIndex = 0 - val maxSplitIndex: Double = { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - if (isFeatureContinuous) { - numBins - 1 - } else { // Categorical feature - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { - math.pow(2.0, featureCategories - 1).toInt - 1 - } else { // Binary classification - featureCategories - } - } - } - while (splitIndex < maxSplitIndex) { + val numSplitsForFeature = getNumSplitsForFeature(featureIndex) + while (splitIndex < numSplitsForFeature) { val gainStats = gains(featureIndex)(splitIndex) if (gainStats.gain > bestGainStats.gain) { bestGainStats = gainStats @@ -1383,6 +1335,7 @@ object DecisionTree extends Serializable with Logging { } // Calculate best splits for all nodes at a given level + timer.start("chooseSplits") val bestSplits = new Array[(Split, InformationGainStats)](numNodes) // Iterating over all nodes at this level var node = 0 @@ -1395,6 +1348,8 @@ object DecisionTree extends Serializable with Logging { bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity) node += 1 } + timer.stop("chooseSplits") + bestSplits } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index f31a503608b22..cfc8192a85abd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -27,22 +27,30 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** * :: Experimental :: * Stores all the configuration options for tree construction - * @param algo classification or regression - * @param impurity criterion used for information gain calculation + * @param algo Learning goal. Supported: + * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], + * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] + * @param impurity Criterion used for information gain calculation. + * Supported for Classification: [[org.apache.spark.mllib.tree.impurity.Gini]], + * [[org.apache.spark.mllib.tree.impurity.Entropy]]. + * Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]]. * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification number of classes for classification. Default value is 2 - * leads to binary classification - * @param maxBins maximum number of bins used for splitting features - * @param quantileCalculationStrategy algorithm for calculating quantiles + * @param numClassesForClassification Number of classes for classification. + * (Ignored for regression.) + * Default value is 2 (binary classification). + * @param maxBins Maximum number of bins used for discretizing continuous features and + * for choosing how to split on features at each node. + * More bins give higher granularity. + * @param quantileCalculationStrategy Algorithm for calculating quantiles. Supported: + * [[org.apache.spark.mllib.tree.configuration.QuantileStrategy.Sort]] * @param categoricalFeaturesInfo A map storing information about the categorical variables and the * number of discrete values they take. For example, an entry (n -> * k) implies the feature n is categorical with k categories 0, * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. - * @param maxMemoryInMB maximum memory in MB allocated to histogram aggregation. Default value is + * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is * 128 MB. - * */ @Experimental class Strategy ( @@ -64,20 +72,7 @@ class Strategy ( = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) /** - * Java-friendly constructor. - * - * @param algo classification or regression - * @param impurity criterion used for information gain calculation - * @param maxDepth Maximum depth of the tree. - * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification number of classes for classification. Default value is 2 - * leads to binary classification - * @param maxBins maximum number of bins used for splitting features - * @param categoricalFeaturesInfo A map storing information about the categorical variables and - * the number of discrete values they take. For example, an entry - * (n -> k) implies the feature n is categorical with k categories - * 0, 1, 2, ... , k-1. It's important to note that features are - * zero-indexed. + * Java-friendly constructor for [[org.apache.spark.mllib.tree.configuration.Strategy]] */ def this( algo: Algo, @@ -90,6 +85,10 @@ class Strategy ( categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) } + /** + * Check validity of parameters. + * Throws exception if invalid. + */ private[tree] def assertValid(): Unit = { algo match { case Classification => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala new file mode 100644 index 0000000000000..d215d68c4279e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TimeTracker.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import scala.collection.mutable.{HashMap => MutableHashMap} + +import org.apache.spark.annotation.Experimental + +/** + * Time tracker implementation which holds labeled timers. + */ +@Experimental +private[tree] class TimeTracker extends Serializable { + + private val starts: MutableHashMap[String, Long] = new MutableHashMap[String, Long]() + + private val totals: MutableHashMap[String, Long] = new MutableHashMap[String, Long]() + + /** + * Starts a new timer, or re-starts a stopped timer. + */ + def start(timerLabel: String): Unit = { + val currentTime = System.nanoTime() + if (starts.contains(timerLabel)) { + throw new RuntimeException(s"TimeTracker.start(timerLabel) called again on" + + s" timerLabel = $timerLabel before that timer was stopped.") + } + starts(timerLabel) = currentTime + } + + /** + * Stops a timer and returns the elapsed time in seconds. + */ + def stop(timerLabel: String): Double = { + val currentTime = System.nanoTime() + if (!starts.contains(timerLabel)) { + throw new RuntimeException(s"TimeTracker.stop(timerLabel) called on" + + s" timerLabel = $timerLabel, but that timer was not started.") + } + val elapsed = currentTime - starts(timerLabel) + starts.remove(timerLabel) + if (totals.contains(timerLabel)) { + totals(timerLabel) += elapsed + } else { + totals(timerLabel) = elapsed + } + elapsed / 1e9 + } + + /** + * Print all timing results in seconds. + */ + override def toString: String = { + totals.map { case (label, elapsed) => + s" $label: ${elapsed / 1e9}" + }.mkString("\n") + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala new file mode 100644 index 0000000000000..ccac1031fd9d9 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.model.Bin +import org.apache.spark.rdd.RDD + + +/** + * Internal representation of LabeledPoint for DecisionTree. + * This bins feature values based on a subsampled of data as follows: + * (a) Continuous features are binned into ranges. + * (b) Unordered categorical features are binned based on subsets of feature values. + * "Unordered categorical features" are categorical features with low arity used in + * multiclass classification. + * (c) Ordered categorical features are binned based on feature values. + * "Ordered categorical features" are categorical features with high arity, + * or any categorical feature used in regression or binary classification. + * + * @param label Label from LabeledPoint + * @param binnedFeatures Binned feature values. + * Same length as LabeledPoint.features, but values are bin indices. + */ +private[tree] class TreePoint(val label: Double, val binnedFeatures: Array[Int]) + extends Serializable { +} + +private[tree] object TreePoint { + + /** + * Convert an input dataset into its TreePoint representation, + * binning feature values in preparation for DecisionTree training. + * @param input Input dataset. + * @param strategy DecisionTree training info, used for dataset metadata. + * @param bins Bins for features, of size (numFeatures, numBins). + * @return TreePoint dataset representation + */ + def convertToTreeRDD( + input: RDD[LabeledPoint], + strategy: Strategy, + bins: Array[Array[Bin]]): RDD[TreePoint] = { + input.map { x => + TreePoint.labeledPointToTreePoint(x, strategy.isMulticlassClassification, bins, + strategy.categoricalFeaturesInfo) + } + } + + /** + * Convert one LabeledPoint into its TreePoint representation. + * @param bins Bins for features, of size (numFeatures, numBins). + * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity + */ + private def labeledPointToTreePoint( + labeledPoint: LabeledPoint, + isMulticlassClassification: Boolean, + bins: Array[Array[Bin]], + categoricalFeaturesInfo: Map[Int, Int]): TreePoint = { + + val numFeatures = labeledPoint.features.size + val numBins = bins(0).size + val arr = new Array[Int](numFeatures) + var featureIndex = 0 + while (featureIndex < numFeatures) { + val featureInfo = categoricalFeaturesInfo.get(featureIndex) + val isFeatureContinuous = featureInfo.isEmpty + if (isFeatureContinuous) { + arr(featureIndex) = findBin(featureIndex, labeledPoint, isFeatureContinuous, false, + bins, categoricalFeaturesInfo) + } else { + val featureCategories = featureInfo.get + val isSpaceSufficientForAllCategoricalSplits + = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + val isUnorderedFeature = + isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits + arr(featureIndex) = findBin(featureIndex, labeledPoint, isFeatureContinuous, + isUnorderedFeature, bins, categoricalFeaturesInfo) + } + featureIndex += 1 + } + + new TreePoint(labeledPoint.label, arr) + } + + /** + * Find bin for one (labeledPoint, feature). + * + * @param isUnorderedFeature (only applies if feature is categorical) + * @param bins Bins for features, of size (numFeatures, numBins). + * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity + */ + private def findBin( + featureIndex: Int, + labeledPoint: LabeledPoint, + isFeatureContinuous: Boolean, + isUnorderedFeature: Boolean, + bins: Array[Array[Bin]], + categoricalFeaturesInfo: Map[Int, Int]): Int = { + + /** + * Binary search helper method for continuous feature. + */ + def binarySearchForBins(): Int = { + val binForFeatures = bins(featureIndex) + val feature = labeledPoint.features(featureIndex) + var left = 0 + var right = binForFeatures.length - 1 + while (left <= right) { + val mid = left + (right - left) / 2 + val bin = binForFeatures(mid) + val lowThreshold = bin.lowSplit.threshold + val highThreshold = bin.highSplit.threshold + if ((lowThreshold < feature) && (highThreshold >= feature)) { + return mid + } else if (lowThreshold >= feature) { + right = mid - 1 + } else { + left = mid + 1 + } + } + -1 + } + + /** + * Sequential search helper method to find bin for categorical feature in multiclass + * classification. The category is returned since each category can belong to multiple + * splits. The actual left/right child allocation per split is performed in the + * sequential phase of the bin aggregate operation. + */ + def sequentialBinSearchForUnorderedCategoricalFeatureInClassification(): Int = { + labeledPoint.features(featureIndex).toInt + } + + /** + * Sequential search helper method to find bin for categorical feature + * (for classification and regression). + */ + def sequentialBinSearchForOrderedCategoricalFeature(): Int = { + val featureCategories = categoricalFeaturesInfo(featureIndex) + val featureValue = labeledPoint.features(featureIndex) + var binIndex = 0 + while (binIndex < featureCategories) { + val bin = bins(featureIndex)(binIndex) + val categories = bin.highSplit.categories + if (categories.contains(featureValue)) { + return binIndex + } + binIndex += 1 + } + if (featureValue < 0 || featureValue >= featureCategories) { + throw new IllegalArgumentException( + s"DecisionTree given invalid data:" + + s" Feature $featureIndex is categorical with values in" + + s" {0,...,${featureCategories - 1}," + + s" but a data point gives it value $featureValue.\n" + + " Bad data point: " + labeledPoint.toString) + } + -1 + } + + if (isFeatureContinuous) { + // Perform binary search for finding bin for continuous features. + val binIndex = binarySearchForBins() + if (binIndex == -1) { + throw new RuntimeException("No bin was found for continuous feature." + + " This error can occur when given invalid data values (such as NaN)." + + s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") + } + binIndex + } else { + // Perform sequential search to find bin for categorical features. + val binIndex = if (isUnorderedFeature) { + sequentialBinSearchForUnorderedCategoricalFeatureInClassification() + } else { + sequentialBinSearchForOrderedCategoricalFeature() + } + if (binIndex == -1) { + throw new RuntimeException("No bin was found for categorical feature." + + " This error can occur when given invalid data values (such as NaN)." + + s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") + } + binIndex + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 70ca7c8a266f2..a5c49a38dc08f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -21,11 +21,12 @@ import scala.collection.JavaConverters._ import org.scalatest.FunSuite -import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Filter, Split} -import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} +import org.apache.spark.mllib.tree.impl.TreePoint +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Filter, Split} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.regression.LabeledPoint @@ -41,7 +42,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { prediction != expected.label } val accuracy = (input.length - numOffPredictions).toDouble / input.length - assert(accuracy >= requiredAccuracy) + assert(accuracy >= requiredAccuracy, + s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") } def validateRegressor( @@ -54,7 +56,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { err * err }.sum val mse = squaredError / input.length - assert(mse <= requiredMSE) + assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") } test("split and bin calculation") { @@ -427,7 +429,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) - val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, Array[List[Filter]](), splits, bins, 10) val split = bestSplits(0)._1 @@ -454,7 +457,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy) - val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, Array[List[Filter]](), splits, bins, 10) val split = bestSplits(0)._1 @@ -499,7 +503,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -521,7 +526,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -544,7 +550,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -567,7 +574,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -596,7 +604,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val parentImpurities = Array(0.5, 0.5, 0.5) // Single group second level tree construction. - val bestSplits = DecisionTree.findBestSplits(rdd, parentImpurities, strategy, 1, filters, + val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, strategy, 1, filters, splits, bins, 10) assert(bestSplits.length === 2) assert(bestSplits(0)._2.gain > 0) @@ -604,7 +613,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second // level tree construction. - val bestSplitsWithGroups = DecisionTree.findBestSplits(rdd, parentImpurities, strategy, 1, + val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, strategy, 1, filters, splits, bins, 0) assert(bestSplitsWithGroups.length === 2) assert(bestSplitsWithGroups(0)._2.gain > 0) @@ -630,7 +639,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) @@ -689,7 +699,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(model.depth === 1) val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) @@ -714,7 +725,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { validateClassifier(model, arr, 0.9) val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) @@ -738,7 +750,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { validateClassifier(model, arr, 0.9) val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) @@ -757,7 +770,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val bestSplits = DecisionTree.findBestSplits(input, new Array(31), strategy, 0, + val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, Array[List[Filter]](), splits, bins, 10) assert(bestSplits.length === 1) From cc3648774e9a744850107bb187f2828d447e0a48 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 15 Aug 2014 17:04:15 -0700 Subject: [PATCH 0368/1492] [SPARK-3046] use executor's class loader as the default serializer classloader The serializer is not always used in an executor thread (e.g. connection manager, broadcast), in which case the classloader might not have the user jar set, leading to corruption in deserialization. https://issues.apache.org/jira/browse/SPARK-3046 https://issues.apache.org/jira/browse/SPARK-2878 Author: Reynold Xin Closes #1972 from rxin/kryoBug and squashes the following commits: c1c7bf0 [Reynold Xin] Made change to JavaSerializer. 7204c33 [Reynold Xin] Added imports back. d879e67 [Reynold Xin] [SPARK-3046] use executor's class loader as the default serializer class loader. --- .../org/apache/spark/executor/Executor.scala | 3 + .../spark/serializer/JavaSerializer.scala | 9 ++- .../spark/serializer/KryoSerializer.scala | 9 ++- .../apache/spark/serializer/Serializer.scala | 17 +++++ .../KryoSerializerDistributedSuite.scala | 71 +++++++++++++++++++ .../serializer/KryoSerializerSuite.scala | 23 +++++- 6 files changed, 128 insertions(+), 4 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index eac1f2326a29d..fb3f7bd54bbfa 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -99,6 +99,9 @@ private[spark] class Executor( private val urlClassLoader = createClassLoader() private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) + // Set the classloader for serializer + env.serializer.setDefaultClassLoader(urlClassLoader) + // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 34bc3124097bb..af33a2f2ca3e1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -63,7 +63,9 @@ extends DeserializationStream { def close() { objIn.close() } } -private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { +private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoader: ClassLoader) + extends SerializerInstance { + def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) @@ -109,7 +111,10 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) - def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset) + override def newInstance(): SerializerInstance = { + val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) + new JavaSerializerInstance(counterReset, classLoader) + } override def writeExternal(out: ObjectOutput) { out.writeInt(counterReset) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 85944eabcfefc..99682220b4ab5 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -61,7 +61,9 @@ class KryoSerializer(conf: SparkConf) val instantiator = new EmptyScalaKryoInstantiator val kryo = instantiator.newKryo() kryo.setRegistrationRequired(registrationRequired) - val classLoader = Thread.currentThread.getContextClassLoader + + val oldClassLoader = Thread.currentThread.getContextClassLoader + val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. @@ -84,10 +86,15 @@ class KryoSerializer(conf: SparkConf) try { val reg = Class.forName(regCls, true, classLoader).newInstance() .asInstanceOf[KryoRegistrator] + + // Use the default classloader when calling the user registrator. + Thread.currentThread.setContextClassLoader(classLoader) reg.registerClasses(kryo) } catch { case e: Exception => throw new SparkException(s"Failed to invoke $regCls", e) + } finally { + Thread.currentThread.setContextClassLoader(oldClassLoader) } } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index f2f5cea469c61..e674438c8176c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -44,6 +44,23 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} */ @DeveloperApi trait Serializer { + + /** + * Default ClassLoader to use in deserialization. Implementations of [[Serializer]] should + * make sure it is using this when set. + */ + @volatile protected var defaultClassLoader: Option[ClassLoader] = None + + /** + * Sets a class loader for the serializer to use in deserialization. + * + * @return this Serializer object + */ + def setDefaultClassLoader(classLoader: ClassLoader): Serializer = { + defaultClassLoader = Some(classLoader) + this + } + def newInstance(): SerializerInstance } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala new file mode 100644 index 0000000000000..11e8c9c4cb37f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import org.apache.spark.util.Utils + +import com.esotericsoftware.kryo.Kryo +import org.scalatest.FunSuite + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, TestUtils} +import org.apache.spark.SparkContext._ +import org.apache.spark.serializer.KryoDistributedTest._ + +class KryoSerializerDistributedSuite extends FunSuite { + + test("kryo objects are serialised consistently in different processes") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) + conf.set("spark.task.maxFailures", "1") + + val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) + conf.setJars(List(jar.getPath)) + + val sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + val original = Thread.currentThread.getContextClassLoader + val loader = new java.net.URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + SparkEnv.get.serializer.setDefaultClassLoader(loader) + + val cachedRDD = sc.parallelize((0 until 10).map((_, new MyCustomClass)), 3).cache() + + // Randomly mix the keys so that the join below will require a shuffle with each partition + // sending data to multiple other partitions. + val shuffledRDD = cachedRDD.map { case (i, o) => (i * i * i - 10 * i * i, o)} + + // Join the two RDDs, and force evaluation + assert(shuffledRDD.join(cachedRDD).collect().size == 1) + + LocalSparkContext.stop(sc) + } +} + +object KryoDistributedTest { + class MyCustomClass + + class AppJarRegistrator extends KryoRegistrator { + override def registerClasses(k: Kryo) { + val classLoader = Thread.currentThread.getContextClassLoader + k.register(Class.forName(AppJarRegistrator.customClassName, true, classLoader)) + } + } + + object AppJarRegistrator { + val customClassName = "KryoSerializerDistributedSuiteCustomClass" + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 3bf9efebb39d2..a579fd50bd9e4 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -23,7 +23,7 @@ import scala.reflect.ClassTag import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite -import org.apache.spark.SharedSparkContext +import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ class KryoSerializerSuite extends FunSuite with SharedSparkContext { @@ -217,8 +217,29 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance()) assert(thrown.getMessage.contains("Failed to invoke this.class.does.not.exist")) } + + test("default class loader can be set by a different thread") { + val ser = new KryoSerializer(new SparkConf) + + // First serialize the object + val serInstance = ser.newInstance() + val bytes = serInstance.serialize(new ClassLoaderTestingObject) + + // Deserialize the object to make sure normal deserialization works + serInstance.deserialize[ClassLoaderTestingObject](bytes) + + // Set a special, broken ClassLoader and make sure we get an exception on deserialization + ser.setDefaultClassLoader(new ClassLoader() { + override def loadClass(name: String) = throw new UnsupportedOperationException + }) + intercept[UnsupportedOperationException] { + ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) + } + } } +class ClassLoaderTestingObject + class KryoSerializerResizableOutputSuite extends FunSuite { import org.apache.spark.SparkConf import org.apache.spark.SparkContext From 5d25c0b74f6397d78164b96afb8b8cbb1b15cfbd Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 15 Aug 2014 21:04:29 -0700 Subject: [PATCH 0369/1492] [SPARK-3078][MLLIB] Make LRWithLBFGS API consistent with others Should ask users to set parameters through the optimizer. dbtsai Author: Xiangrui Meng Closes #1973 from mengxr/lr-lbfgs and squashes the following commits: e3efbb1 [Xiangrui Meng] fix tests 21b3579 [Xiangrui Meng] fix method name 641eea4 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into lr-lbfgs 456ab7c [Xiangrui Meng] update LRWithLBFGS --- .../examples/mllib/BinaryClassification.scala | 8 ++-- .../classification/LogisticRegression.scala | 40 +++---------------- .../spark/mllib/optimization/LBFGS.scala | 9 +++++ .../LogisticRegressionSuite.scala | 5 ++- .../spark/mllib/optimization/LBFGSSuite.scala | 24 +++++------ 5 files changed, 33 insertions(+), 53 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index 56b02b65d8724..a6f78d2441db1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -21,7 +21,7 @@ import org.apache.log4j.{Level, Logger} import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD} +import org.apache.spark.mllib.classification.{LogisticRegressionWithLBFGS, SVMWithSGD} import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater} @@ -66,7 +66,8 @@ object BinaryClassification { .text("number of iterations") .action((x, c) => c.copy(numIterations = x)) opt[Double]("stepSize") - .text(s"initial step size, default: ${defaultParams.stepSize}") + .text("initial step size (ignored by logistic regression), " + + s"default: ${defaultParams.stepSize}") .action((x, c) => c.copy(stepSize = x)) opt[String]("algorithm") .text(s"algorithm (${Algorithm.values.mkString(",")}), " + @@ -125,10 +126,9 @@ object BinaryClassification { val model = params.algorithm match { case LR => - val algorithm = new LogisticRegressionWithSGD() + val algorithm = new LogisticRegressionWithLBFGS() algorithm.optimizer .setNumIterations(params.numIterations) - .setStepSize(params.stepSize) .setUpdater(updater) .setRegParam(params.regParam) algorithm.run(training).clearThreshold() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 6790c86f651b4..486bdbfa9cb47 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -73,6 +73,8 @@ class LogisticRegressionModel ( /** * Train a classification model for Logistic Regression using Stochastic Gradient Descent. * NOTE: Labels used in Logistic Regression should be {0, 1} + * + * Using [[LogisticRegressionWithLBFGS]] is recommended over this. */ class LogisticRegressionWithSGD private ( private var stepSize: Double, @@ -191,51 +193,19 @@ object LogisticRegressionWithSGD { /** * Train a classification model for Logistic Regression using Limited-memory BFGS. + * Standard feature scaling and L2 regularization are used by default. * NOTE: Labels used in Logistic Regression should be {0, 1} */ -class LogisticRegressionWithLBFGS private ( - private var convergenceTol: Double, - private var maxNumIterations: Int, - private var regParam: Double) +class LogisticRegressionWithLBFGS extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { - /** - * Construct a LogisticRegression object with default parameters - */ - def this() = this(1E-4, 100, 0.0) - this.setFeatureScaling(true) - private val gradient = new LogisticGradient() - private val updater = new SimpleUpdater() - // Have to return new LBFGS object every time since users can reset the parameters anytime. - override def optimizer = new LBFGS(gradient, updater) - .setNumCorrections(10) - .setConvergenceTol(convergenceTol) - .setMaxNumIterations(maxNumIterations) - .setRegParam(regParam) + override val optimizer = new LBFGS(new LogisticGradient, new SquaredL2Updater) override protected val validators = List(DataValidators.binaryLabelValidator) - /** - * Set the convergence tolerance of iterations for L-BFGS. Default 1E-4. - * Smaller value will lead to higher accuracy with the cost of more iterations. - */ - def setConvergenceTol(convergenceTol: Double): this.type = { - this.convergenceTol = convergenceTol - this - } - - /** - * Set the maximal number of iterations for L-BFGS. Default 100. - */ - def setNumIterations(numIterations: Int): this.type = { - this.maxNumIterations = numIterations - this - } - override protected def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 033fe44f34f3c..d16d0daf08565 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -69,8 +69,17 @@ class LBFGS(private var gradient: Gradient, private var updater: Updater) /** * Set the maximal number of iterations for L-BFGS. Default 100. + * @deprecated use [[LBFGS#setNumIterations]] instead */ + @deprecated("use setNumIterations instead", "1.1.0") def setMaxNumIterations(iters: Int): this.type = { + this.setNumIterations(iters) + } + + /** + * Set the maximal number of iterations for L-BFGS. Default 100. + */ + def setNumIterations(iters: Int): this.type = { this.maxNumIterations = iters this } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index bc05b2046878f..862178694a50e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -272,8 +272,9 @@ class LogisticRegressionClusterSuite extends FunSuite with LocalClusterSparkCont }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be // greater than 1MB and hence Spark would throw an error. - val model = - (new LogisticRegressionWithLBFGS().setIntercept(true).setNumIterations(2)).run(points) + val lr = new LogisticRegressionWithLBFGS().setIntercept(true) + lr.optimizer.setNumIterations(2) + val model = lr.run(points) val predictions = model.predict(points.map(_.features)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index 5f4c24115ac80..ccba004baa007 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -55,7 +55,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { val initialWeightsWithIntercept = Vectors.dense(1.0 +: initialWeights.toArray) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val (_, loss) = LBFGS.runLBFGS( dataRDD, @@ -63,7 +63,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { simpleUpdater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -99,7 +99,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { // Prepare another non-zero weights to compare the loss in the first iteration. val initialWeightsWithIntercept = Vectors.dense(0.3, 0.12) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val (weightLBFGS, lossLBFGS) = LBFGS.runLBFGS( dataRDD, @@ -107,7 +107,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -140,10 +140,10 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { /** * For the first run, we set the convergenceTol to 0.0, so that the algorithm will - * run up to the maxNumIterations which is 8 here. + * run up to the numIterations which is 8 here. */ val initialWeightsWithIntercept = Vectors.dense(0.0, 0.0) - val maxNumIterations = 8 + val numIterations = 8 var convergenceTol = 0.0 val (_, lossLBFGS1) = LBFGS.runLBFGS( @@ -152,7 +152,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -167,7 +167,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -182,7 +182,7 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { squaredL2Updater, numCorrections, convergenceTol, - maxNumIterations, + numIterations, regParam, initialWeightsWithIntercept) @@ -200,12 +200,12 @@ class LBFGSSuite extends FunSuite with LocalSparkContext with Matchers { // Prepare another non-zero weights to compare the loss in the first iteration. val initialWeightsWithIntercept = Vectors.dense(0.3, 0.12) val convergenceTol = 1e-12 - val maxNumIterations = 10 + val numIterations = 10 val lbfgsOptimizer = new LBFGS(gradient, squaredL2Updater) .setNumCorrections(numCorrections) .setConvergenceTol(convergenceTol) - .setMaxNumIterations(maxNumIterations) + .setNumIterations(numIterations) .setRegParam(regParam) val weightLBFGS = lbfgsOptimizer.optimize(dataRDD, initialWeightsWithIntercept) @@ -241,7 +241,7 @@ class LBFGSClusterSuite extends FunSuite with LocalClusterSparkContext { val lbfgs = new LBFGS(new LogisticGradient, new SquaredL2Updater) .setNumCorrections(1) .setConvergenceTol(1e-12) - .setMaxNumIterations(1) + .setNumIterations(1) .setRegParam(1.0) val random = new Random(0) // If we serialize data directly in the task closure, the size of the serialized task would be From 2e069ca6560bf7ab07bd019f9530b42f4fe45014 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 15 Aug 2014 21:07:55 -0700 Subject: [PATCH 0370/1492] [SPARK-3001][MLLIB] Improve Spearman's correlation The current implementation requires sorting individual columns, which could be done with a global sort. result on a 32-node cluster: m | n | prev | this ---|---|-------|----- 1000000 | 50 | 55s | 9s 10000000 | 50 | 97s | 76s 1000000 | 100 | 119s | 15s Author: Xiangrui Meng Closes #1917 from mengxr/spearman and squashes the following commits: 4d5d262 [Xiangrui Meng] remove unused import 85c48de [Xiangrui Meng] minor updates a048d0c [Xiangrui Meng] remove cache and set a limit to cachedIds b98bb18 [Xiangrui Meng] add comments 0846e07 [Xiangrui Meng] first version --- .../correlation/SpearmanCorrelation.scala | 120 ++++++------------ 1 file changed, 42 insertions(+), 78 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala index 9bd0c2cd05de4..4a6c677f06d28 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/correlation/SpearmanCorrelation.scala @@ -19,10 +19,10 @@ package org.apache.spark.mllib.stat.correlation import scala.collection.mutable.ArrayBuffer -import org.apache.spark.{Logging, HashPartitioner} +import org.apache.spark.Logging import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.{DenseVector, Matrix, Vector} -import org.apache.spark.rdd.{CoGroupedRDD, RDD} +import org.apache.spark.mllib.linalg.{Matrix, Vector, Vectors} +import org.apache.spark.rdd.RDD /** * Compute Spearman's correlation for two RDDs of the type RDD[Double] or the correlation matrix @@ -43,87 +43,51 @@ private[stat] object SpearmanCorrelation extends Correlation with Logging { /** * Compute Spearman's correlation matrix S, for the input matrix, where S(i, j) is the * correlation between column i and j. - * - * Input RDD[Vector] should be cached or checkpointed if possible since it would be split into - * numCol RDD[Double]s, each of which sorted, and the joined back into a single RDD[Vector]. */ override def computeCorrelationMatrix(X: RDD[Vector]): Matrix = { - val indexed = X.zipWithUniqueId() - - val numCols = X.first.size - if (numCols > 50) { - logWarning("Computing the Spearman correlation matrix can be slow for large RDDs with more" - + " than 50 columns.") - } - val ranks = new Array[RDD[(Long, Double)]](numCols) - - // Note: we use a for loop here instead of a while loop with a single index variable - // to avoid race condition caused by closure serialization - for (k <- 0 until numCols) { - val column = indexed.map { case (vector, index) => (vector(k), index) } - ranks(k) = getRanks(column) + // ((columnIndex, value), rowUid) + val colBased = X.zipWithUniqueId().flatMap { case (vec, uid) => + vec.toArray.view.zipWithIndex.map { case (v, j) => + ((j, v), uid) + } } - - val ranksMat: RDD[Vector] = makeRankMatrix(ranks, X) - PearsonCorrelation.computeCorrelationMatrix(ranksMat) - } - - /** - * Compute the ranks for elements in the input RDD, using the average method for ties. - * - * With the average method, elements with the same value receive the same rank that's computed - * by taking the average of their positions in the sorted list. - * e.g. ranks([2, 1, 0, 2]) = [2.5, 1.0, 0.0, 2.5] - * Note that positions here are 0-indexed, instead of the 1-indexed as in the definition for - * ranks in the standard definition for Spearman's correlation. This does not affect the final - * results and is slightly more performant. - * - * @param indexed RDD[(Double, Long)] containing pairs of the format (originalValue, uniqueId) - * @return RDD[(Long, Double)] containing pairs of the format (uniqueId, rank), where uniqueId is - * copied from the input RDD. - */ - private def getRanks(indexed: RDD[(Double, Long)]): RDD[(Long, Double)] = { - // Get elements' positions in the sorted list for computing average rank for duplicate values - val sorted = indexed.sortByKey().zipWithIndex() - - val ranks: RDD[(Long, Double)] = sorted.mapPartitions { iter => - // add an extra element to signify the end of the list so that flatMap can flush the last - // batch of duplicates - val end = -1L - val padded = iter ++ Iterator[((Double, Long), Long)](((Double.NaN, end), end)) - val firstEntry = padded.next() - var lastVal = firstEntry._1._1 - var firstRank = firstEntry._2.toDouble - val idBuffer = ArrayBuffer(firstEntry._1._2) - padded.flatMap { case ((v, id), rank) => - if (v == lastVal && id != end) { - idBuffer += id - Iterator.empty - } else { - val entries = if (idBuffer.size == 1) { - Iterator((idBuffer(0), firstRank)) - } else { - val averageRank = firstRank + (idBuffer.size - 1.0) / 2.0 - idBuffer.map(id => (id, averageRank)) - } - lastVal = v - firstRank = rank - idBuffer.clear() - idBuffer += id - entries + // global sort by (columnIndex, value) + val sorted = colBased.sortByKey() + // assign global ranks (using average ranks for tied values) + val globalRanks = sorted.zipWithIndex().mapPartitions { iter => + var preCol = -1 + var preVal = Double.NaN + var startRank = -1.0 + var cachedUids = ArrayBuffer.empty[Long] + val flush: () => Iterable[(Long, (Int, Double))] = () => { + val averageRank = startRank + (cachedUids.size - 1) / 2.0 + val output = cachedUids.map { uid => + (uid, (preCol, averageRank)) } + cachedUids.clear() + output } + iter.flatMap { case (((j, v), uid), rank) => + // If we see a new value or cachedUids is too big, we flush ids with their average rank. + if (j != preCol || v != preVal || cachedUids.size >= 10000000) { + val output = flush() + preCol = j + preVal = v + startRank = rank + cachedUids += uid + output + } else { + cachedUids += uid + Iterator.empty + } + } ++ flush() } - ranks - } - - private def makeRankMatrix(ranks: Array[RDD[(Long, Double)]], input: RDD[Vector]): RDD[Vector] = { - val partitioner = new HashPartitioner(input.partitions.size) - val cogrouped = new CoGroupedRDD[Long](ranks, partitioner) - cogrouped.map { - case (_, values: Array[Iterable[_]]) => - val doubles = values.asInstanceOf[Array[Iterable[Double]]] - new DenseVector(doubles.flatten.toArray) + // Replace values in the input matrix by their ranks compared with values in the same column. + // Note that shifting all ranks in a column by a constant value doesn't affect result. + val groupedRanks = globalRanks.groupByKey().map { case (uid, iter) => + // sort by column index and then convert values to a vector + Vectors.dense(iter.toSeq.sortBy(_._1).map(_._2).toArray) } + PearsonCorrelation.computeCorrelationMatrix(groupedRanks) } } From c9da466edb83e45a159ccc17c68856a511b9e8b7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 15 Aug 2014 22:55:32 -0700 Subject: [PATCH 0371/1492] [SPARK-3015] Block on cleaning tasks to prevent Akka timeouts More detail on the issue is described in [SPARK-3015](https://issues.apache.org/jira/browse/SPARK-3015), but the TLDR is if we send too many blocking Akka messages that are dependent on each other in quick successions, then we end up causing a few of these messages to time out and ultimately kill the executors. As of #1498, we broadcast each RDD whether or not it is persisted. This means if we create many RDDs (each of which becomes a broadcast) and the driver performs a GC that cleans up all of these broadcast blocks, then we end up sending many `RemoveBroadcast` messages in parallel and trigger the chain of blocking messages at high frequencies. We do not know of the Akka-level root cause yet, so this is intended to be a temporary solution until we identify the real issue. I have done some preliminary testing of enabling blocking and observed that the queue length remains quite low (< 1000) even under very intensive workloads. In the long run, we should do something more sophisticated to allow a limited degree of parallelism through batching clean up tasks or processing them in a sliding window. In the longer run, we should clean up the whole `BlockManager*` message passing interface to avoid unnecessarily awaiting on futures created from Akka asks. tdas pwendell mengxr Author: Andrew Or Closes #1931 from andrewor14/reference-blocking and squashes the following commits: d0f7195 [Andrew Or] Merge branch 'master' of github.com:apache/spark into reference-blocking ce9daf5 [Andrew Or] Remove logic for logging queue length 111192a [Andrew Or] Add missing space in log message (minor) a183b83 [Andrew Or] Switch order of code blocks (minor) 9fd1fe6 [Andrew Or] Remove outdated log 104b366 [Andrew Or] Use the actual reference queue length 0b7e768 [Andrew Or] Block on cleaning tasks by default + log error on queue full --- .../main/scala/org/apache/spark/ContextCleaner.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index bf3c3a6ceb5ef..3848734d6f639 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -66,10 +66,15 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** * Whether the cleaning thread will block on cleanup tasks. - * This is set to true only for tests. + * + * Due to SPARK-3015, this is set to true by default. This is intended to be only a temporary + * workaround for the issue, which is ultimately caused by the way the BlockManager actors + * issue inter-dependent blocking Akka messages to each other at high frequencies. This happens, + * for instance, when the driver performs a GC and cleans up all broadcast blocks that are no + * longer in scope. */ private val blockOnCleanupTasks = sc.conf.getBoolean( - "spark.cleaner.referenceTracking.blocking", false) + "spark.cleaner.referenceTracking.blocking", true) @volatile private var stopped = false @@ -174,9 +179,6 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] - - // Used for testing. These methods explicitly blocks until cleanup is completed - // to ensure that more reliable testing. } private object ContextCleaner { From a83c7723bf7a90dc6cd5dde98a179303b7542020 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 15 Aug 2014 23:12:34 -0700 Subject: [PATCH 0372/1492] [SPARK-3045] Make Serializer interface Java friendly Author: Reynold Xin Closes #1948 from rxin/kryo and squashes the following commits: a3a80d8 [Reynold Xin] [SPARK-3046] use executor's class loader as the default serializer classloader 3d13277 [Reynold Xin] Reverted that in TestJavaSerializerImpl too. 196f3dc [Reynold Xin] Ok one more commit to revert the classloader change. c49b50c [Reynold Xin] Removed JavaSerializer change. afbf37d [Reynold Xin] Moved the test case also. a2e693e [Reynold Xin] Removed the Kryo bug fix from this pull request. c81bd6c [Reynold Xin] Use defaultClassLoader when executing user specified custom registrator. 68f261e [Reynold Xin] Added license check excludes. 0c28179 [Reynold Xin] [SPARK-3045] Make Serializer interface Java friendly [SPARK-3046] Set executor's class loader as the default serializer class loader --- .../spark/serializer/JavaSerializer.scala | 15 +-- .../spark/serializer/KryoSerializer.scala | 32 +++---- .../apache/spark/serializer/Serializer.scala | 25 ++--- .../apache/spark/serializer/package-info.java | 2 +- .../serializer/TestJavaSerializerImpl.java | 95 +++++++++++++++++++ .../KryoSerializerResizableOutputSuite.scala | 52 ++++++++++ .../serializer/KryoSerializerSuite.scala | 34 +------ project/MimaExcludes.scala | 11 +++ 8 files changed, 193 insertions(+), 73 deletions(-) create mode 100644 core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java create mode 100644 core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index af33a2f2ca3e1..554a33ce7f1a6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -63,10 +63,11 @@ extends DeserializationStream { def close() { objIn.close() } } + private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoader: ClassLoader) extends SerializerInstance { - def serialize[T: ClassTag](t: T): ByteBuffer = { + override def serialize[T: ClassTag](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) out.writeObject(t) @@ -74,23 +75,23 @@ private[spark] class JavaSerializerInstance(counterReset: Int, defaultClassLoade ByteBuffer.wrap(bos.toByteArray) } - def deserialize[T: ClassTag](bytes: ByteBuffer): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis) - in.readObject().asInstanceOf[T] + in.readObject() } - def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val bis = new ByteBufferInputStream(bytes) val in = deserializeStream(bis, loader) - in.readObject().asInstanceOf[T] + in.readObject() } - def serializeStream(s: OutputStream): SerializationStream = { + override def serializeStream(s: OutputStream): SerializationStream = { new JavaSerializationStream(s, counterReset) } - def deserializeStream(s: InputStream): DeserializationStream = { + override def deserializeStream(s: InputStream): DeserializationStream = { new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) } diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 99682220b4ab5..87ef9bb0b43c6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -91,7 +91,7 @@ class KryoSerializer(conf: SparkConf) Thread.currentThread.setContextClassLoader(classLoader) reg.registerClasses(kryo) } catch { - case e: Exception => + case e: Exception => throw new SparkException(s"Failed to invoke $regCls", e) } finally { Thread.currentThread.setContextClassLoader(oldClassLoader) @@ -106,7 +106,7 @@ class KryoSerializer(conf: SparkConf) kryo } - def newInstance(): SerializerInstance = { + override def newInstance(): SerializerInstance = { new KryoSerializerInstance(this) } } @@ -115,20 +115,20 @@ private[spark] class KryoSerializationStream(kryo: Kryo, outStream: OutputStream) extends SerializationStream { val output = new KryoOutput(outStream) - def writeObject[T: ClassTag](t: T): SerializationStream = { + override def writeObject[T: ClassTag](t: T): SerializationStream = { kryo.writeClassAndObject(output, t) this } - def flush() { output.flush() } - def close() { output.close() } + override def flush() { output.flush() } + override def close() { output.close() } } private[spark] class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends DeserializationStream { - val input = new KryoInput(inStream) + private val input = new KryoInput(inStream) - def readObject[T: ClassTag](): T = { + override def readObject[T: ClassTag](): T = { try { kryo.readClassAndObject(input).asInstanceOf[T] } catch { @@ -138,31 +138,31 @@ class KryoDeserializationStream(kryo: Kryo, inStream: InputStream) extends Deser } } - def close() { + override def close() { // Kryo's Input automatically closes the input stream it is using. input.close() } } private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends SerializerInstance { - val kryo = ks.newKryo() + private val kryo = ks.newKryo() // Make these lazy vals to avoid creating a buffer unless we use them - lazy val output = ks.newKryoOutput() - lazy val input = new KryoInput() + private lazy val output = ks.newKryoOutput() + private lazy val input = new KryoInput() - def serialize[T: ClassTag](t: T): ByteBuffer = { + override def serialize[T: ClassTag](t: T): ByteBuffer = { output.clear() kryo.writeClassAndObject(output, t) ByteBuffer.wrap(output.toBytes) } - def deserialize[T: ClassTag](bytes: ByteBuffer): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { input.setBuffer(bytes.array) kryo.readClassAndObject(input).asInstanceOf[T] } - def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = { val oldClassLoader = kryo.getClassLoader kryo.setClassLoader(loader) input.setBuffer(bytes.array) @@ -171,11 +171,11 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer) extends Serializ obj } - def serializeStream(s: OutputStream): SerializationStream = { + override def serializeStream(s: OutputStream): SerializationStream = { new KryoSerializationStream(kryo, s) } - def deserializeStream(s: InputStream): DeserializationStream = { + override def deserializeStream(s: InputStream): DeserializationStream = { new KryoDeserializationStream(kryo, s) } } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index e674438c8176c..a9144cdd97b8c 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -43,7 +43,7 @@ import org.apache.spark.util.{ByteBufferInputStream, NextIterator} * They are intended to be used to serialize/de-serialize data within a single Spark application. */ @DeveloperApi -trait Serializer { +abstract class Serializer { /** * Default ClassLoader to use in deserialization. Implementations of [[Serializer]] should @@ -61,10 +61,12 @@ trait Serializer { this } + /** Creates a new [[SerializerInstance]]. */ def newInstance(): SerializerInstance } +@DeveloperApi object Serializer { def getSerializer(serializer: Serializer): Serializer = { if (serializer == null) SparkEnv.get.serializer else serializer @@ -81,7 +83,7 @@ object Serializer { * An instance of a serializer, for use by one thread at a time. */ @DeveloperApi -trait SerializerInstance { +abstract class SerializerInstance { def serialize[T: ClassTag](t: T): ByteBuffer def deserialize[T: ClassTag](bytes: ByteBuffer): T @@ -91,21 +93,6 @@ trait SerializerInstance { def serializeStream(s: OutputStream): SerializationStream def deserializeStream(s: InputStream): DeserializationStream - - def serializeMany[T: ClassTag](iterator: Iterator[T]): ByteBuffer = { - // Default implementation uses serializeStream - val stream = new ByteArrayOutputStream() - serializeStream(stream).writeAll(iterator) - val buffer = ByteBuffer.wrap(stream.toByteArray) - buffer.flip() - buffer - } - - def deserializeMany(buffer: ByteBuffer): Iterator[Any] = { - // Default implementation uses deserializeStream - buffer.rewind() - deserializeStream(new ByteBufferInputStream(buffer)).asIterator - } } /** @@ -113,7 +100,7 @@ trait SerializerInstance { * A stream for writing serialized objects. */ @DeveloperApi -trait SerializationStream { +abstract class SerializationStream { def writeObject[T: ClassTag](t: T): SerializationStream def flush(): Unit def close(): Unit @@ -132,7 +119,7 @@ trait SerializationStream { * A stream for reading serialized objects. */ @DeveloperApi -trait DeserializationStream { +abstract class DeserializationStream { def readObject[T: ClassTag](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/serializer/package-info.java b/core/src/main/scala/org/apache/spark/serializer/package-info.java index 4c0b73ab36a00..207c6e02e4293 100644 --- a/core/src/main/scala/org/apache/spark/serializer/package-info.java +++ b/core/src/main/scala/org/apache/spark/serializer/package-info.java @@ -18,4 +18,4 @@ /** * Pluggable serializers for RDD and shuffle data. */ -package org.apache.spark.serializer; \ No newline at end of file +package org.apache.spark.serializer; diff --git a/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java b/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java new file mode 100644 index 0000000000000..3d50ab4fabe42 --- /dev/null +++ b/core/src/test/java/org/apache/spark/serializer/TestJavaSerializerImpl.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer; + +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import scala.Option; +import scala.reflect.ClassTag; + + +/** + * A simple Serializer implementation to make sure the API is Java-friendly. + */ +class TestJavaSerializerImpl extends Serializer { + + @Override + public SerializerInstance newInstance() { + return null; + } + + static class SerializerInstanceImpl extends SerializerInstance { + @Override + public ByteBuffer serialize(T t, ClassTag evidence$1) { + return null; + } + + @Override + public T deserialize(ByteBuffer bytes, ClassLoader loader, ClassTag evidence$1) { + return null; + } + + @Override + public T deserialize(ByteBuffer bytes, ClassTag evidence$1) { + return null; + } + + @Override + public SerializationStream serializeStream(OutputStream s) { + return null; + } + + @Override + public DeserializationStream deserializeStream(InputStream s) { + return null; + } + } + + static class SerializationStreamImpl extends SerializationStream { + + @Override + public SerializationStream writeObject(T t, ClassTag evidence$1) { + return null; + } + + @Override + public void flush() { + + } + + @Override + public void close() { + + } + } + + static class DeserializationStreamImpl extends DeserializationStream { + + @Override + public T readObject(ClassTag evidence$1) { + return null; + } + + @Override + public void close() { + + } + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala new file mode 100644 index 0000000000000..967c9e9899c9d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.SparkContext +import org.apache.spark.LocalSparkContext +import org.apache.spark.SparkException + + +class KryoSerializerResizableOutputSuite extends FunSuite { + + // trial and error showed this will not serialize with 1mb buffer + val x = (1 to 400000).toArray + + test("kryo without resizable output buffer should fail on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "1") + val sc = new SparkContext("local", "test", conf) + intercept[SparkException](sc.parallelize(x).collect()) + LocalSparkContext.stop(sc) + } + + test("kryo with resizable output buffer should succeed on large array") { + val conf = new SparkConf(false) + conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set("spark.kryoserializer.buffer.mb", "1") + conf.set("spark.kryoserializer.buffer.max.mb", "2") + val sc = new SparkContext("local", "test", conf) + assert(sc.parallelize(x).collect() === x) + LocalSparkContext.stop(sc) + } +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index a579fd50bd9e4..e1e35b688d581 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -26,6 +26,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkConf, SharedSparkContext} import org.apache.spark.serializer.KryoTest._ + class KryoSerializerSuite extends FunSuite with SharedSparkContext { conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) @@ -207,7 +208,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x assert(10 + control.sum === result) } - + test("kryo with nonexistent custom registrator should fail") { import org.apache.spark.{SparkConf, SparkException} @@ -238,39 +239,12 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } } -class ClassLoaderTestingObject - -class KryoSerializerResizableOutputSuite extends FunSuite { - import org.apache.spark.SparkConf - import org.apache.spark.SparkContext - import org.apache.spark.LocalSparkContext - import org.apache.spark.SparkException - - // trial and error showed this will not serialize with 1mb buffer - val x = (1 to 400000).toArray - test("kryo without resizable output buffer should fail on large array") { - val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer.mb", "1") - conf.set("spark.kryoserializer.buffer.max.mb", "1") - val sc = new SparkContext("local", "test", conf) - intercept[SparkException](sc.parallelize(x).collect) - LocalSparkContext.stop(sc) - } +class ClassLoaderTestingObject - test("kryo with resizable output buffer should succeed on large array") { - val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer.mb", "1") - conf.set("spark.kryoserializer.buffer.max.mb", "2") - val sc = new SparkContext("local", "test", conf) - assert(sc.parallelize(x).collect === x) - LocalSparkContext.stop(sc) - } -} object KryoTest { + case class CaseClass(i: Int, s: String) {} class ClassWithNoArgConstructor { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1e3c760b845de..bbe68b29d2d8e 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -61,6 +61,17 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.storage.MemoryStore.Entry") ) ++ + Seq( + // Serializer interface change. See SPARK-3045. + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.DeserializationStream"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.Serializer"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.SerializationStream"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.serializer.SerializerInstance") + )++ Seq( // Renamed putValues -> putArray + putIterator ProblemFilters.exclude[MissingMethodProblem]( From 20fcf3d0b72f3707dc1ed95d453f570fabdefd16 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 16 Aug 2014 00:04:55 -0700 Subject: [PATCH 0373/1492] [SPARK-2977] Ensure ShuffleManager is created before ShuffleBlockManager This is intended to fix SPARK-2977. Before, there was an implicit ordering dependency where we needed to know the ShuffleManager implementation before creating the ShuffleBlockManager. This patch makes that dependency explicit by adding ShuffleManager to a bunch of constructors. I think it's a little odd for BlockManager to take a ShuffleManager only to pass it to ShuffleBlockManager without using it itself; there's an opportunity to clean this up later if we sever the circular dependencies between BlockManager and other components and pass those components to BlockManager's constructor. Author: Josh Rosen Closes #1976 from JoshRosen/SPARK-2977 and squashes the following commits: a9cd1e1 [Josh Rosen] [SPARK-2977] Ensure ShuffleManager is created before ShuffleBlockManager. --- .../scala/org/apache/spark/SparkEnv.scala | 22 +++++++++---------- .../apache/spark/storage/BlockManager.scala | 11 ++++++---- .../spark/storage/ShuffleBlockManager.scala | 7 +++--- .../apache/spark/storage/ThreadingTest.scala | 3 ++- .../spark/storage/BlockManagerSuite.scala | 12 +++++----- .../spark/storage/DiskBlockManagerSuite.scala | 8 +++++-- 6 files changed, 37 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 22d8d1cb1ddcf..fc36e37c53f5e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -210,12 +210,22 @@ object SparkEnv extends Logging { "MapOutputTracker", new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + // Let the user specify short names for shuffle managers + val shortShuffleMgrNames = Map( + "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", + "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") + val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") + val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) + val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) + + val shuffleMemoryManager = new ShuffleMemoryManager(conf) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager, mapOutputTracker) + serializer, conf, securityManager, mapOutputTracker, shuffleManager) val connectionManager = blockManager.connectionManager @@ -250,16 +260,6 @@ object SparkEnv extends Logging { "." } - // Let the user specify short names for shuffle managers - val shortShuffleMgrNames = Map( - "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", - "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") - val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") - val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) - val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) - - val shuffleMemoryManager = new ShuffleMemoryManager(conf) - // Warn about deprecated spark.cache.class property if (conf.contains("spark.cache.class")) { logWarning("The spark.cache.class property is no longer being used! Specify storage " + diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e8bbd298c631a..e4c3d58905e7f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -33,6 +33,7 @@ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ private[spark] sealed trait BlockValues @@ -57,11 +58,12 @@ private[spark] class BlockManager( maxMemory: Long, val conf: SparkConf, securityManager: SecurityManager, - mapOutputTracker: MapOutputTracker) + mapOutputTracker: MapOutputTracker, + shuffleManager: ShuffleManager) extends Logging { private val port = conf.getInt("spark.blockManager.port", 0) - val shuffleBlockManager = new ShuffleBlockManager(this) + val shuffleBlockManager = new ShuffleBlockManager(this, shuffleManager) val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) val connectionManager = @@ -142,9 +144,10 @@ private[spark] class BlockManager( serializer: Serializer, conf: SparkConf, securityManager: SecurityManager, - mapOutputTracker: MapOutputTracker) = { + mapOutputTracker: MapOutputTracker, + shuffleManager: ShuffleManager) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager, mapOutputTracker) + conf, securityManager, mapOutputTracker, shuffleManager) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 3565719b54545..b8f5d3a5b02aa 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.Logging import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.storage.ShuffleBlockManager.ShuffleFileGroup import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} import org.apache.spark.util.collection.{PrimitiveKeyOpenHashMap, PrimitiveVector} @@ -62,7 +63,8 @@ private[spark] trait ShuffleWriterGroup { */ // TODO: Factor this into a separate class for each ShuffleManager implementation private[spark] -class ShuffleBlockManager(blockManager: BlockManager) extends Logging { +class ShuffleBlockManager(blockManager: BlockManager, + shuffleManager: ShuffleManager) extends Logging { def conf = blockManager.conf // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. @@ -71,8 +73,7 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { conf.getBoolean("spark.shuffle.consolidateFiles", false) // Are we using sort-based shuffle? - val sortBasedShuffle = - conf.get("spark.shuffle.manager", "") == classOf[SortShuffleManager].getName + val sortBasedShuffle = shuffleManager.isInstanceOf[SortShuffleManager] private val bufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 75c2e09a6bbb8..aa83ea90ee9ee 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.util.concurrent.ArrayBlockingQueue import akka.actor._ +import org.apache.spark.shuffle.hash.HashShuffleManager import util.Random import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} @@ -101,7 +102,7 @@ private[spark] object ThreadingTest { conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf), new MapOutputTrackerMaster(conf)) + new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 94bb2c445d2e9..20bac66105a69 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -24,6 +24,7 @@ import java.util.concurrent.TimeUnit import akka.actor._ import akka.pattern.ask import akka.util.Timeout +import org.apache.spark.shuffle.hash.HashShuffleManager import org.mockito.invocation.InvocationOnMock import org.mockito.Matchers.any @@ -61,6 +62,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -71,8 +73,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { - new BlockManager( - name, actorSystem, master, serializer, maxMem, conf, securityMgr, mapOutputTracker) + new BlockManager(name, actorSystem, master, serializer, maxMem, conf, securityMgr, + mapOutputTracker, shuffleManager) } before { @@ -791,7 +793,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr, mapOutputTracker) + securityMgr, mapOutputTracker, shuffleManager) // The put should fail since a1 is not serializable. class UnserializableClass @@ -1007,7 +1009,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("return error message when error occurred in BlockManagerWorker#onBlockMessageReceive") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + securityMgr, mapOutputTracker, shuffleManager) val worker = spy(new BlockManagerWorker(store)) val connManagerId = mock(classOf[ConnectionManagerId]) @@ -1054,7 +1056,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("return ack message when no error occurred in BlocManagerWorker#onBlockMessageReceive") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker) + securityMgr, mapOutputTracker, shuffleManager) val worker = spy(new BlockManagerWorker(store)) val connManagerId = mock(classOf[ConnectionManagerId]) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index b8299e2ea187f..777579bc570db 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.storage import java.io.{File, FileWriter} +import org.apache.spark.shuffle.hash.HashShuffleManager + import scala.collection.mutable import scala.language.reflectiveCalls @@ -42,7 +44,9 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before // so we coerce consolidation if not already enabled. testConf.set("spark.shuffle.consolidateFiles", "true") - val shuffleBlockManager = new ShuffleBlockManager(null) { + private val shuffleManager = new HashShuffleManager(testConf.clone) + + val shuffleBlockManager = new ShuffleBlockManager(null, shuffleManager) { override def conf = testConf.clone var idToSegmentMap = mutable.Map[ShuffleBlockId, FileSegment]() override def getBlockLocation(id: ShuffleBlockId) = idToSegmentMap(id) @@ -148,7 +152,7 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before actorSystem.actorOf(Props(new BlockManagerMasterActor(true, confCopy, new LiveListenerBus))), confCopy) val store = new BlockManager("", actorSystem, master , serializer, confCopy, - securityManager, null) + securityManager, null, shuffleManager) try { From b4a05928e95c0f6973fd21e60ff9c108f226e38c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 16 Aug 2014 11:26:51 -0700 Subject: [PATCH 0374/1492] [SQL] Using safe floating-point numbers in doctest Test code in `sql.py` tries to compare two floating-point numbers directly, and cased [build failure(s)](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18365/consoleFull). [Doctest documentation](https://docs.python.org/3/library/doctest.html#warnings) recommends using numbers in the form of `I/2**J` to avoid the precision issue. Author: Cheng Lian Closes #1925 from liancheng/fix-pysql-fp-test and squashes the following commits: 0fbf584 [Cheng Lian] Removed unnecessary `...' from inferSchema doctest e8059d4 [Cheng Lian] Using safe floating-point numbers in doctest --- python/pyspark/sql.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 95086a2258222..d4ca0cc8f336e 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1093,8 +1093,8 @@ def applySchema(self, rdd, schema): >>> sqlCtx.sql( ... "SELECT byte1 - 1 AS byte1, byte2 + 1 AS byte2, " + ... "short1 + 1 AS short1, short2 - 1 AS short2, int - 1 AS int, " + - ... "float + 1.1 as float FROM table2").collect() - [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.1...)] + ... "float + 1.5 as float FROM table2").collect() + [Row(byte1=126, byte2=-127, short1=-32767, short2=32766, int=2147483646, float=2.5)] >>> rdd = sc.parallelize([(127, -32768, 1.0, ... datetime(2010, 1, 1, 1, 1, 1), From 4bdfaa16fce399bd97c98858151246b3b02f350f Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 16 Aug 2014 12:35:59 -0700 Subject: [PATCH 0375/1492] [SPARK-3076] [Jenkins] catch & report test timeouts * Remove unused code to get jq * Set timeout on tests and report gracefully on them Author: Nicholas Chammas Closes #1974 from nchammas/master and squashes the following commits: d1f1b6b [Nicholas Chammas] set timeout to realistic number 8b1ea41 [Nicholas Chammas] fix formatting 279526e [Nicholas Chammas] [SPARK-3076] catch & report test timeouts --- dev/run-tests-jenkins | 48 ++++++++++++++++++------------------------- 1 file changed, 20 insertions(+), 28 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 721f09be5be6d..31506e28e05af 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -26,27 +26,17 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd "$FWDIR" -function get_jq () { - # Get jq so we can parse some JSON, man. - # Essential if we want to do anything with the GitHub API responses. - local JQ_EXECUTABLE_URL="http://stedolan.github.io/jq/download/linux64/jq" - - echo "Fetching jq from ${JQ_EXECUTABLE_URL}" - - curl --silent --output "$FWDIR/dev/jq" "$JQ_EXECUTABLE_URL" - local curl_status=$? - - if [ $curl_status -ne 0 ]; then - echo "Failed to get jq." >&2 - return $curl_status - fi - - chmod u+x "$FWDIR/dev/jq" -} - COMMENTS_URL="https://api.github.com/repos/apache/spark/issues/$ghprbPullId/comments" PULL_REQUEST_URL="https://github.com/apache/spark/pull/$ghprbPullId" +COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" +# GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( +SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" + +# NOTE: Jenkins will kill the whole build after 120 minutes. +# Tests are a large part of that, but not all of it. +TESTS_TIMEOUT="120m" + function post_message () { local message=$1 local data="{\"body\": \"$message\"}" @@ -96,10 +86,6 @@ function post_message () { fi } -COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" -# GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( -short_commit_hash=${ghprbActualCommit:0:7} - # check PR merge-ability and check for new public classes { if [ "$sha1" == "$ghprbActualCommit" ]; then @@ -138,7 +124,7 @@ short_commit_hash=${ghprbActualCommit:0:7} { start_message="\ [QA tests have started](${BUILD_URL}consoleFull) for \ - PR $ghprbPullId at commit [\`${short_commit_hash}\`](${COMMIT_URL})." + PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." start_message="${start_message}\n${merge_note}" # start_message="${start_message}\n${public_classes_note}" @@ -148,13 +134,19 @@ short_commit_hash=${ghprbActualCommit:0:7} # run tests { - ./dev/run-tests + timeout "${TESTS_TIMEOUT}" ./dev/run-tests test_result="$?" - if [ "$test_result" -eq "0" ]; then - test_result_note=" * This patch **passes** unit tests." + if [ "$test_result" -eq "124" ]; then + fail_message="**Tests timed out** after a configured wait of \`${TESTS_TIMEOUT}\`." + post_message "$fail_message" + exit $test_result else - test_result_note=" * This patch **fails** unit tests." + if [ "$test_result" -eq "0" ]; then + test_result_note=" * This patch **passes** unit tests." + else + test_result_note=" * This patch **fails** unit tests." + fi fi } @@ -162,7 +154,7 @@ short_commit_hash=${ghprbActualCommit:0:7} { result_message="\ [QA tests have finished](${BUILD_URL}consoleFull) for \ - PR $ghprbPullId at commit [\`${short_commit_hash}\`](${COMMIT_URL})." + PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." result_message="${result_message}\n${test_result_note}" result_message="${result_message}\n${merge_note}" From 76fa0eaf515fd6771cdd69422b1259485debcae5 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sat, 16 Aug 2014 14:15:58 -0700 Subject: [PATCH 0376/1492] [SPARK-2677] BasicBlockFetchIterator#next can wait forever Author: Kousuke Saruta Closes #1632 from sarutak/SPARK-2677 and squashes the following commits: cddbc7b [Kousuke Saruta] Removed Exception throwing when ConnectionManager#handleMessage receives ack for non-referenced message d3bd2a8 [Kousuke Saruta] Modified configuration.md for spark.core.connection.ack.timeout e85f88b [Kousuke Saruta] Removed useless synchronized blocks 7ed48be [Kousuke Saruta] Modified ConnectionManager to use ackTimeoutMonitor ConnectionManager-wide 9b620a6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 0dd9ad3 [Kousuke Saruta] Modified typo in ConnectionManagerSuite.scala 7cbb8ca [Kousuke Saruta] Modified to match with scalastyle 8a73974 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 ade279a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 0174d6a [Kousuke Saruta] Modified ConnectionManager.scala to handle the case remote Executor cannot ack a454239 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2677 9b7b7c1 [Kousuke Saruta] (WIP) Modifying ConnectionManager.scala --- .../spark/network/ConnectionManager.scala | 45 ++++++++++++++----- .../network/ConnectionManagerSuite.scala | 44 +++++++++++++++++- docs/configuration.md | 9 ++++ 3 files changed, 87 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 95f96b8463a01..37d69a9ec4ce4 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -22,6 +22,7 @@ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ import java.net._ +import java.util.{Timer, TimerTask} import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} @@ -61,17 +62,17 @@ private[spark] class ConnectionManager( var ackMessage: Option[Message] = None def markDone(ackMessage: Option[Message]) { - this.synchronized { - this.ackMessage = ackMessage - completionHandler(this) - } + this.ackMessage = ackMessage + completionHandler(this) } } private val selector = SelectorProvider.provider.openSelector() + private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true) // default to 30 second timeout waiting for authentication private val authTimeout = conf.getInt("spark.core.connection.auth.wait.timeout", 30) + private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) private val handleMessageExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.handler.threads.min", 20), @@ -652,19 +653,27 @@ private[spark] class ConnectionManager( } } if (bufferMessage.hasAckId()) { - val sentMessageStatus = messageStatuses.synchronized { + messageStatuses.synchronized { messageStatuses.get(bufferMessage.ackId) match { case Some(status) => { messageStatuses -= bufferMessage.ackId - status + status.markDone(Some(message)) } case None => { - throw new Exception("Could not find reference for received ack message " + - message.id) + /** + * We can fall down on this code because of following 2 cases + * + * (1) Invalid ack sent due to buggy code. + * + * (2) Late-arriving ack for a SendMessageStatus + * To avoid unwilling late-arriving ack + * caused by long pause like GC, you can set + * larger value than default to spark.core.connection.ack.wait.timeout + */ + logWarning(s"Could not find reference for received ack Message ${message.id}") } } } - sentMessageStatus.markDone(Some(message)) } else { var ackMessage : Option[Message] = None try { @@ -836,9 +845,23 @@ private[spark] class ConnectionManager( def sendMessageReliably(connectionManagerId: ConnectionManagerId, message: Message) : Future[Message] = { val promise = Promise[Message]() + + val timeoutTask = new TimerTask { + override def run(): Unit = { + messageStatuses.synchronized { + messageStatuses.remove(message.id).foreach ( s => { + promise.failure( + new IOException(s"sendMessageReliably failed because ack " + + "was not received within ${ackTimeout} sec")) + }) + } + } + } + val status = new MessageStatus(message, connectionManagerId, s => { + timeoutTask.cancel() s.ackMessage match { - case None => // Indicates a failure where we either never sent or never got ACK'd + case None => // Indicates a failure where we either never sent or never got ACK'd promise.failure(new IOException("sendMessageReliably failed without being ACK'd")) case Some(ackMessage) => if (ackMessage.hasError) { @@ -852,6 +875,8 @@ private[spark] class ConnectionManager( messageStatuses.synchronized { messageStatuses += ((message.id, status)) } + + ackTimeoutMonitor.schedule(timeoutTask, ackTimeout * 1000) sendMessage(connectionManagerId, message) promise.future } diff --git a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala index 846537df003df..e2f4d4c57cdb5 100644 --- a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala @@ -19,14 +19,19 @@ package org.apache.spark.network import java.io.IOException import java.nio._ +import java.util.concurrent.TimeoutException import org.apache.spark.{SecurityManager, SparkConf} import org.scalatest.FunSuite +import org.mockito.Mockito._ +import org.mockito.Matchers._ + +import scala.concurrent.TimeoutException import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ import scala.language.postfixOps -import scala.util.Try +import scala.util.{Failure, Success, Try} /** * Test the ConnectionManager with various security settings. @@ -255,5 +260,42 @@ class ConnectionManagerSuite extends FunSuite { } + test("sendMessageReliably timeout") { + val clientConf = new SparkConf + clientConf.set("spark.authenticate", "false") + val ackTimeout = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + + val clientSecurityManager = new SecurityManager(clientConf) + val manager = new ConnectionManager(0, clientConf, clientSecurityManager) + + val serverConf = new SparkConf + serverConf.set("spark.authenticate", "false") + val serverSecurityManager = new SecurityManager(serverConf) + val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + // sleep 60 sec > ack timeout for simulating server slow down or hang up + Thread.sleep(ackTimeout * 3 * 1000) + None + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + + val future = manager.sendMessageReliably(managerServer.id, bufferMessage) + + // Future should throw IOException in 30 sec. + // Otherwise TimeoutExcepton is thrown from Await.result. + // We expect TimeoutException is not thrown. + intercept[IOException] { + Await.result(future, (ackTimeout * 2) second) + } + + manager.stop() + managerServer.stop() + } + } diff --git a/docs/configuration.md b/docs/configuration.md index c408c468dcd94..981170d8b49b7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -884,6 +884,15 @@ Apart from these, the following properties are also available, and may be useful out and giving up. + + spark.core.connection.ack.wait.timeout + 60 + + Number of seconds for the connection to wait for ack to occur before timing + out and giving up. To avoid unwilling timeout caused by long pause like GC, + you can set larger value. + + spark.ui.filters None From 7e70708a99949549adde00cb6246a9582bbc4929 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 16 Aug 2014 15:13:34 -0700 Subject: [PATCH 0377/1492] [SPARK-3048][MLLIB] add LabeledPoint.parse and remove loadStreamingLabeledPoints Move `parse()` from `LabeledPointParser` to `LabeledPoint` and make it public. This breaks binary compatibility only when a user uses synthesized methods like `tupled` and `curried`, which is rare. `LabeledPoint.parse` is more consistent with `Vectors.parse`, which is why `LabeledPointParser` is not preferred. freeman-lab tdas Author: Xiangrui Meng Closes #1952 from mengxr/labelparser and squashes the following commits: c818fb2 [Xiangrui Meng] merge master ce20e6f [Xiangrui Meng] update mima excludes b386b8d [Xiangrui Meng] fix tests 2436b3d [Xiangrui Meng] add parse() to LabeledPoint --- .../mllib/StreamingLinearRegression.scala | 7 +++---- .../spark/mllib/regression/LabeledPoint.scala | 2 +- .../StreamingLinearRegressionWithSGD.scala | 2 +- .../org/apache/spark/mllib/util/MLUtils.scala | 17 ++--------------- .../mllib/regression/LabeledPointSuite.scala | 4 ++-- .../StreamingLinearRegressionSuite.scala | 6 +++--- project/MimaExcludes.scala | 5 +++++ 7 files changed, 17 insertions(+), 26 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala index 1fd37edfa7427..0e992fa9967bb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -18,8 +18,7 @@ package org.apache.spark.examples.mllib import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD +import org.apache.spark.mllib.regression.{LabeledPoint, StreamingLinearRegressionWithSGD} import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} @@ -56,8 +55,8 @@ object StreamingLinearRegression { val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) - val trainingData = MLUtils.loadStreamingLabeledPoints(ssc, args(0)) - val testData = MLUtils.loadStreamingLabeledPoints(ssc, args(1)) + val trainingData = ssc.textFileStream(args(0)).map(LabeledPoint.parse) + val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(Array.fill[Double](args(3).toInt)(0))) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 62a03af4a9964..17c753c56681f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -36,7 +36,7 @@ case class LabeledPoint(label: Double, features: Vector) { /** * Parser for [[org.apache.spark.mllib.regression.LabeledPoint]]. */ -private[mllib] object LabeledPointParser { +object LabeledPoint { /** * Parses a string resulted from `LabeledPoint#toString` into * an [[org.apache.spark.mllib.regression.LabeledPoint]]. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala index 8851097050318..1d11fde24712c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionWithSGD.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vector /** * Train or predict a linear regression model on streaming data. Training uses diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index f4cce86a65ba7..ca35100aa99c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -27,7 +27,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD import org.apache.spark.util.random.BernoulliSampler -import org.apache.spark.mllib.regression.{LabeledPointParser, LabeledPoint} +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext @@ -185,7 +185,7 @@ object MLUtils { * @return labeled points stored as an RDD[LabeledPoint] */ def loadLabeledPoints(sc: SparkContext, path: String, minPartitions: Int): RDD[LabeledPoint] = - sc.textFile(path, minPartitions).map(LabeledPointParser.parse) + sc.textFile(path, minPartitions).map(LabeledPoint.parse) /** * Loads labeled points saved using `RDD[LabeledPoint].saveAsTextFile` with the default number of @@ -194,19 +194,6 @@ object MLUtils { def loadLabeledPoints(sc: SparkContext, dir: String): RDD[LabeledPoint] = loadLabeledPoints(sc, dir, sc.defaultMinPartitions) - /** - * Loads streaming labeled points from a stream of text files - * where points are in the same format as used in `RDD[LabeledPoint].saveAsTextFile`. - * See `StreamingContext.textFileStream` for more details on how to - * generate a stream from files - * - * @param ssc Streaming context - * @param dir Directory path in any Hadoop-supported file system URI - * @return Labeled points stored as a DStream[LabeledPoint] - */ - def loadStreamingLabeledPoints(ssc: StreamingContext, dir: String): DStream[LabeledPoint] = - ssc.textFileStream(dir).map(LabeledPointParser.parse) - /** * Load labeled data from a file. The data format used here is * , ... diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index d9308aaba6ee1..110c44a7193fd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -28,12 +28,12 @@ class LabeledPointSuite extends FunSuite { LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0)))) points.foreach { p => - assert(p === LabeledPointParser.parse(p.toString)) + assert(p === LabeledPoint.parse(p.toString)) } } test("parse labeled points with v0.9 format") { - val point = LabeledPointParser.parse("1.0,1.0 0.0 -2.0") + val point = LabeledPoint.parse("1.0,1.0 0.0 -2.0") assert(point === LabeledPoint(1.0, Vectors.dense(1.0, 0.0, -2.0))) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index ed21f84472c9a..45e25eecf508e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -26,7 +26,7 @@ import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext, MLUtils} +import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils @@ -55,7 +55,7 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { val numBatches = 10 val batchDuration = Milliseconds(1000) val ssc = new StreamingContext(sc, batchDuration) - val data = MLUtils.loadStreamingLabeledPoints(ssc, testDir.toString) + val data = ssc.textFileStream(testDir.toString).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) .setStepSize(0.1) @@ -97,7 +97,7 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { val batchDuration = Milliseconds(2000) val ssc = new StreamingContext(sc, batchDuration) val numBatches = 5 - val data = MLUtils.loadStreamingLabeledPoints(ssc, testDir.toString) + val data = ssc.textFileStream(testDir.toString()).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) .setStepSize(0.1) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index bbe68b29d2d8e..300589394b96f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -129,6 +129,11 @@ object MimaExcludes { Seq( // new Vector methods in MLlib (binary compatible assuming users do not implement Vector) ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.linalg.Vector.copy") ) ++ + Seq( // synthetic methods generated in LabeledPoint + ProblemFilters.exclude[MissingTypesProblem]("org.apache.spark.mllib.regression.LabeledPoint$"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.mllib.regression.LabeledPoint.apply"), + ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.regression.LabeledPoint.toString") + ) ++ Seq ( // Scala 2.11 compatibility fix ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.streaming.StreamingContext.$default$2") ) From ac6411c6e75906997c78de23dfdbc8d225b87cfd Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 16 Aug 2014 15:14:43 -0700 Subject: [PATCH 0378/1492] [SPARK-3081][MLLIB] rename RandomRDDGenerators to RandomRDDs `RandomRDDGenerators` means factory for `RandomRDDGenerator`. However, its methods return RDDs but not RDDGenerators. So a more proper (and shorter) name would be `RandomRDDs`. dorx brkyvz Author: Xiangrui Meng Closes #1979 from mengxr/randomrdds and squashes the following commits: b161a2d [Xiangrui Meng] rename RandomRDDGenerators to RandomRDDs --- .../mllib/api/python/PythonMLLibAPI.scala | 2 +- ...omRDDGenerators.scala => RandomRDDs.scala} | 6 ++--- ...atorsSuite.scala => RandomRDDsSuite.scala} | 16 ++++++------ python/pyspark/mllib/random.py | 25 +++++++++---------- 4 files changed, 24 insertions(+), 25 deletions(-) rename mllib/src/main/scala/org/apache/spark/mllib/random/{RandomRDDGenerators.scala => RandomRDDs.scala} (99%) rename mllib/src/test/scala/org/apache/spark/mllib/random/{RandomRDDGeneratorsSuite.scala => RandomRDDsSuite.scala} (88%) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 18dc087856785..4343124f102a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} -import org.apache.spark.mllib.random.{RandomRDDGenerators => RG} +import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala similarity index 99% rename from mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala rename to mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index b0a0593223910..36270369526cd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDGenerators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.random +import scala.reflect.ClassTag + import org.apache.spark.SparkContext import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.Vector @@ -24,14 +26,12 @@ import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils -import scala.reflect.ClassTag - /** * :: Experimental :: * Generator methods for creating RDDs comprised of i.i.d. samples from some distribution. */ @Experimental -object RandomRDDGenerators { +object RandomRDDs { /** * :: Experimental :: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala similarity index 88% rename from mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala rename to mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index 96e0bc63b0fa4..c50b78bcbcc61 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDGeneratorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.StatCounter * * TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged */ -class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Serializable { +class RandomRDDsSuite extends FunSuite with LocalSparkContext with Serializable { def testGeneratedRDD(rdd: RDD[Double], expectedSize: Long, @@ -113,18 +113,18 @@ class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Seri val poissonMean = 100.0 for (seed <- 0 until 5) { - val uniform = RandomRDDGenerators.uniformRDD(sc, size, numPartitions, seed) + val uniform = RandomRDDs.uniformRDD(sc, size, numPartitions, seed) testGeneratedRDD(uniform, size, numPartitions, 0.5, 1 / math.sqrt(12)) - val normal = RandomRDDGenerators.normalRDD(sc, size, numPartitions, seed) + val normal = RandomRDDs.normalRDD(sc, size, numPartitions, seed) testGeneratedRDD(normal, size, numPartitions, 0.0, 1.0) - val poisson = RandomRDDGenerators.poissonRDD(sc, poissonMean, size, numPartitions, seed) + val poisson = RandomRDDs.poissonRDD(sc, poissonMean, size, numPartitions, seed) testGeneratedRDD(poisson, size, numPartitions, poissonMean, math.sqrt(poissonMean), 0.1) } // mock distribution to check that partitions have unique seeds - val random = RandomRDDGenerators.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L) + val random = RandomRDDs.randomRDD(sc, new MockDistro(), 1000L, 1000, 0L) assert(random.collect.size === random.collect.distinct.size) } @@ -135,13 +135,13 @@ class RandomRDDGeneratorsSuite extends FunSuite with LocalSparkContext with Seri val poissonMean = 100.0 for (seed <- 0 until 5) { - val uniform = RandomRDDGenerators.uniformVectorRDD(sc, rows, cols, parts, seed) + val uniform = RandomRDDs.uniformVectorRDD(sc, rows, cols, parts, seed) testGeneratedVectorRDD(uniform, rows, cols, parts, 0.5, 1 / math.sqrt(12)) - val normal = RandomRDDGenerators.normalVectorRDD(sc, rows, cols, parts, seed) + val normal = RandomRDDs.normalVectorRDD(sc, rows, cols, parts, seed) testGeneratedVectorRDD(normal, rows, cols, parts, 0.0, 1.0) - val poisson = RandomRDDGenerators.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) + val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) } } diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index eb496688b6eef..3f3b19053d32e 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -25,8 +25,7 @@ from pyspark.serializers import NoOpSerializer -class RandomRDDGenerators: - +class RandomRDDs: """ Generator methods for creating RDDs comprised of i.i.d samples from some distribution. @@ -40,17 +39,17 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - C{RandomRDDGenerators.uniformRDD(sc, n, p, seed)\ + C{RandomRDDs.uniformRDD(sc, n, p, seed)\ .map(lambda v: a + (b - a) * v)} - >>> x = RandomRDDGenerators.uniformRDD(sc, 100).collect() + >>> x = RandomRDDs.uniformRDD(sc, 100).collect() >>> len(x) 100 >>> max(x) <= 1.0 and min(x) >= 0.0 True - >>> RandomRDDGenerators.uniformRDD(sc, 100, 4).getNumPartitions() + >>> RandomRDDs.uniformRDD(sc, 100, 4).getNumPartitions() 4 - >>> parts = RandomRDDGenerators.uniformRDD(sc, 100, seed=4).getNumPartitions() + >>> parts = RandomRDDs.uniformRDD(sc, 100, seed=4).getNumPartitions() >>> parts == sc.defaultParallelism True """ @@ -66,10 +65,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): To transform the distribution in the generated RDD from standard normal to some other normal N(mean, sigma), use - C{RandomRDDGenerators.normal(sc, n, p, seed)\ + C{RandomRDDs.normal(sc, n, p, seed)\ .map(lambda v: mean + sigma * v)} - >>> x = RandomRDDGenerators.normalRDD(sc, 1000, seed=1L) + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) >>> stats = x.stats() >>> stats.count() 1000L @@ -89,7 +88,7 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): distribution with the input mean. >>> mean = 100.0 - >>> x = RandomRDDGenerators.poissonRDD(sc, mean, 1000, seed=1L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=1L) >>> stats = x.stats() >>> stats.count() 1000L @@ -110,12 +109,12 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): from the uniform distribution on [0.0 1.0]. >>> import numpy as np - >>> mat = np.matrix(RandomRDDGenerators.uniformVectorRDD(sc, 10, 10).collect()) + >>> mat = np.matrix(RandomRDDs.uniformVectorRDD(sc, 10, 10).collect()) >>> mat.shape (10, 10) >>> mat.max() <= 1.0 and mat.min() >= 0.0 True - >>> RandomRDDGenerators.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() + >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 """ jrdd = sc._jvm.PythonMLLibAPI() \ @@ -130,7 +129,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): from the standard normal distribution. >>> import numpy as np - >>> mat = np.matrix(RandomRDDGenerators.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - 0.0) < 0.1 @@ -151,7 +150,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> import numpy as np >>> mean = 100.0 - >>> rdd = RandomRDDGenerators.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) From 379e7585c356f20bf8b4878ecba9401e2195da12 Mon Sep 17 00:00:00 2001 From: iAmGhost Date: Sat, 16 Aug 2014 16:48:38 -0700 Subject: [PATCH 0379/1492] [SPARK-3035] Wrong example with SparkContext.addFile https://issues.apache.org/jira/browse/SPARK-3035 fix for wrong document. Author: iAmGhost Closes #1942 from iAmGhost/master and squashes the following commits: 487528a [iAmGhost] [SPARK-3035] Wrong example with SparkContext.addFile fix for wrong document. --- python/pyspark/context.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 4001ecab5ea00..6c049238819a7 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -613,7 +613,7 @@ def addFile(self, path): >>> def func(iterator): ... with open(SparkFiles.get("test.txt")) as testFile: ... fileVal = int(testFile.readline()) - ... return [x * 100 for x in iterator] + ... return [x * fileVal for x in iterator] >>> sc.parallelize([1, 2, 3, 4]).mapPartitions(func).collect() [100, 200, 300, 400] """ From 2fc8aca086a2679b854038b7e2c488f19039ecbd Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 16 Aug 2014 16:59:34 -0700 Subject: [PATCH 0380/1492] [SPARK-1065] [PySpark] improve supporting for large broadcast Passing large object by py4j is very slow (cost much memory), so pass broadcast objects via files (similar to parallelize()). Add an option to keep object in driver (it's False by default) to save memory in driver. Author: Davies Liu Closes #1912 from davies/broadcast and squashes the following commits: e06df4a [Davies Liu] load broadcast from disk in driver automatically db3f232 [Davies Liu] fix serialization of accumulator 631a827 [Davies Liu] Merge branch 'master' into broadcast c7baa8c [Davies Liu] compress serrialized broadcast and command 9a7161f [Davies Liu] fix doc tests e93cf4b [Davies Liu] address comments: add test 6226189 [Davies Liu] improve large broadcast --- .../apache/spark/api/python/PythonRDD.scala | 8 ++++ python/pyspark/broadcast.py | 37 ++++++++++++++----- python/pyspark/context.py | 20 ++++++---- python/pyspark/rdd.py | 5 ++- python/pyspark/serializers.py | 17 +++++++++ python/pyspark/tests.py | 7 ++++ python/pyspark/worker.py | 8 ++-- 7 files changed, 81 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 9f5c5bd30f0c9..10210a2927dcc 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -315,6 +315,14 @@ private[spark] object PythonRDD extends Logging { JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } + def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = { + val file = new DataInputStream(new FileInputStream(filename)) + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + sc.broadcast(obj) + } + def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { // The right way to implement this would be to use TypeTags to get the full // type of T. Since I don't want to introduce breaking changes throughout the diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index f3e64989ed564..675a2fcd2ff4e 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -21,18 +21,16 @@ >>> b = sc.broadcast([1, 2, 3, 4, 5]) >>> b.value [1, 2, 3, 4, 5] - ->>> from pyspark.broadcast import _broadcastRegistry ->>> _broadcastRegistry[b.bid] = b ->>> from cPickle import dumps, loads ->>> loads(dumps(b)).value -[1, 2, 3, 4, 5] - >>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() [1, 2, 3, 4, 5, 1, 2, 3, 4, 5] +>>> b.unpersist() >>> large_broadcast = sc.broadcast(list(range(10000))) """ +import os + +from pyspark.serializers import CompressedSerializer, PickleSerializer + # Holds broadcasted data received from Java, keyed by its id. _broadcastRegistry = {} @@ -52,17 +50,38 @@ class Broadcast(object): Access its value through C{.value}. """ - def __init__(self, bid, value, java_broadcast=None, pickle_registry=None): + def __init__(self, bid, value, java_broadcast=None, + pickle_registry=None, path=None): """ Should not be called directly by users -- use L{SparkContext.broadcast()} instead. """ - self.value = value self.bid = bid + if path is None: + self.value = value self._jbroadcast = java_broadcast self._pickle_registry = pickle_registry + self.path = path + + def unpersist(self, blocking=False): + self._jbroadcast.unpersist(blocking) + os.unlink(self.path) def __reduce__(self): self._pickle_registry.add(self) return (_from_id, (self.bid, )) + + def __getattr__(self, item): + if item == 'value' and self.path is not None: + ser = CompressedSerializer(PickleSerializer()) + value = ser.load_stream(open(self.path)).next() + self.value = value + return value + + raise AttributeError(item) + + +if __name__ == "__main__": + import doctest + doctest.testmod() diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6c049238819a7..a90870ed3a353 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer + PairDeserializer, CompressedSerializer from pyspark.storagelevel import StorageLevel from pyspark import rdd from pyspark.rdd import RDD @@ -566,13 +566,19 @@ def broadcast(self, value): """ Broadcast a read-only variable to the cluster, returning a L{Broadcast} - object for reading it in distributed functions. The variable will be - sent to each cluster only once. + object for reading it in distributed functions. The variable will + be sent to each cluster only once. + + :keep: Keep the `value` in driver or not. """ - pickleSer = PickleSerializer() - pickled = pickleSer.dumps(value) - jbroadcast = self._jsc.broadcast(bytearray(pickled)) - return Broadcast(jbroadcast.id(), value, jbroadcast, self._pickled_broadcast_vars) + ser = CompressedSerializer(PickleSerializer()) + # pass large object by py4j is very slow and need much memory + tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) + ser.dump_stream([value], tempFile) + tempFile.close() + jbroadcast = self._jvm.PythonRDD.readBroadcastFromFile(self._jsc, tempFile.name) + return Broadcast(jbroadcast.id(), None, jbroadcast, + self._pickled_broadcast_vars, tempFile.name) def accumulator(self, value, accum_param=None): """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3934bdda0a466..240381e5bae12 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -36,7 +36,7 @@ from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ - PickleSerializer, pack_long + PickleSerializer, pack_long, CompressedSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -1810,7 +1810,8 @@ def _jrdd(self): self._jrdd_deserializer = NoOpSerializer() command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) - pickled_command = CloudPickleSerializer().dumps(command) + ser = CompressedSerializer(CloudPickleSerializer()) + pickled_command = ser.dumps(command) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index df90cafb245bf..74870c0edcf99 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -67,6 +67,7 @@ import sys import types import collections +import zlib from pyspark import cloudpickle @@ -403,6 +404,22 @@ def loads(self, obj): raise ValueError("invalid sevialization type: %s" % _type) +class CompressedSerializer(FramedSerializer): + """ + compress the serialized data + """ + + def __init__(self, serializer): + FramedSerializer.__init__(self) + self.serializer = serializer + + def dumps(self, obj): + return zlib.compress(self.serializer.dumps(obj), 1) + + def loads(self, obj): + return self.serializer.loads(zlib.decompress(obj)) + + class UTF8Deserializer(Serializer): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 22b51110ed671..f1fece998cd54 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -323,6 +323,13 @@ def test_namedtuple_in_rdd(self): theDoes = self.sc.parallelize([jon, jane]) self.assertEquals([jon, jane], theDoes.collect()) + def test_large_broadcast(self): + N = 100000 + data = [[float(i) for i in range(300)] for i in range(N)] + bdata = self.sc.broadcast(data) # 270MB + m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() + self.assertEquals(N, m) + class TestIO(PySparkTestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 2770f63059853..77a9c4a0e0677 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,7 +30,8 @@ from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, write_int, read_long, \ - write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer + write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ + CompressedSerializer pickleSer = PickleSerializer() @@ -65,12 +66,13 @@ def main(infile, outfile): # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) + ser = CompressedSerializer(pickleSer) for _ in range(num_broadcast_variables): bid = read_long(infile) - value = pickleSer._read_with_length(infile) + value = ser._read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, value) - command = pickleSer._read_with_length(infile) + command = ser._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() iterator = deserializer.load_stream(infile) From bc95fe08dff62a0abea314ab4ab9275c8f119598 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 16 Aug 2014 20:05:55 -0700 Subject: [PATCH 0381/1492] In the stop method of ConnectionManager to cancel the ackTimeoutMonitor cc JoshRosen sarutak Author: GuoQiang Li Closes #1989 from witgo/cancel_ackTimeoutMonitor and squashes the following commits: 4a700fa [GuoQiang Li] In the stop method of ConnectionManager to cancel the ackTimeoutMonitor --- .../main/scala/org/apache/spark/network/ConnectionManager.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 37d69a9ec4ce4..e77d762bdf221 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -886,6 +886,7 @@ private[spark] class ConnectionManager( } def stop() { + ackTimeoutMonitor.cancel() selectorThread.interrupt() selectorThread.join() selector.close() From fbad72288d8b6e641b00417a544cae6e8bfef2d7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 16 Aug 2014 21:16:27 -0700 Subject: [PATCH 0382/1492] [SPARK-3077][MLLIB] fix some chisq-test - promote nullHypothesis field in ChiSqTestResult to TestResult. Every test should have a null hypothesis - correct null hypothesis statement for independence test - p-value: 0.01 -> 0.1 Author: Xiangrui Meng Closes #1982 from mengxr/fix-chisq and squashes the following commits: 5f0de02 [Xiangrui Meng] make ChiSqTestResult constructor package private bc74ea1 [Xiangrui Meng] update chisq-test --- .../spark/mllib/stat/test/ChiSqTest.scala | 2 +- .../spark/mllib/stat/test/TestResult.scala | 28 +++++++++++-------- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 8f6752737402e..215de95db5113 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -56,7 +56,7 @@ private[stat] object ChiSqTest extends Logging { object NullHypothesis extends Enumeration { type NullHypothesis = Value val goodnessOfFit = Value("observed follows the same distribution as expected.") - val independence = Value("observations in each column are statistically independent.") + val independence = Value("the occurrence of the outcomes is statistically independent.") } // Method identification based on input methodName string diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala index 2f278621335e1..4784f9e947908 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/TestResult.scala @@ -44,6 +44,11 @@ trait TestResult[DF] { */ def statistic: Double + /** + * Null hypothesis of the test. + */ + def nullHypothesis: String + /** * String explaining the hypothesis test result. * Specific classes implementing this trait should override this method to output test-specific @@ -53,13 +58,13 @@ trait TestResult[DF] { // String explaining what the p-value indicates. val pValueExplain = if (pValue <= 0.01) { - "Very strong presumption against null hypothesis." + s"Very strong presumption against null hypothesis: $nullHypothesis." } else if (0.01 < pValue && pValue <= 0.05) { - "Strong presumption against null hypothesis." - } else if (0.05 < pValue && pValue <= 0.01) { - "Low presumption against null hypothesis." + s"Strong presumption against null hypothesis: $nullHypothesis." + } else if (0.05 < pValue && pValue <= 0.1) { + s"Low presumption against null hypothesis: $nullHypothesis." } else { - "No presumption against null hypothesis." + s"No presumption against null hypothesis: $nullHypothesis." } s"degrees of freedom = ${degreesOfFreedom.toString} \n" + @@ -70,19 +75,18 @@ trait TestResult[DF] { /** * :: Experimental :: - * Object containing the test results for the chi squared hypothesis test. + * Object containing the test results for the chi-squared hypothesis test. */ @Experimental -class ChiSqTestResult(override val pValue: Double, +class ChiSqTestResult private[stat] (override val pValue: Double, override val degreesOfFreedom: Int, override val statistic: Double, val method: String, - val nullHypothesis: String) extends TestResult[Int] { + override val nullHypothesis: String) extends TestResult[Int] { override def toString: String = { - "Chi squared test summary: \n" + - s"method: $method \n" + - s"null hypothesis: $nullHypothesis \n" + - super.toString + "Chi squared test summary:\n" + + s"method: $method\n" + + super.toString } } From 73ab7f141c205df277c6ac19252e590d6806c41f Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sat, 16 Aug 2014 23:53:14 -0700 Subject: [PATCH 0383/1492] [SPARK-3042] [mllib] DecisionTree Filter top-down instead of bottom-up DecisionTree needs to match each example to a node at each iteration. It currently does this with a set of filters very inefficiently: For each example, it examines each node at the current level and traces up to the root to see if that example should be handled by that node. Fix: Filter top-down using the partly built tree itself. Major changes: * Eliminated Filter class, findBinsForLevel() method. * Set up node parent links in main loop over levels in train(). * Added predictNodeIndex() for filtering top-down. * Added DTMetadata class Other changes: * Pre-compute set of unorderedFeatures. Notes for following expected PR based on [https://issues.apache.org/jira/browse/SPARK-3043]: * The unorderedFeatures set will next be stored in a metadata structure to simplify function calls (to store other items such as the data in strategy). I've done initial tests indicating that this speeds things up, but am only now running large-scale ones. CC: mengxr manishamde chouqin Any comments are welcome---thanks! Author: Joseph K. Bradley Closes #1975 from jkbradley/dt-opt2 and squashes the following commits: a0ed0da [Joseph K. Bradley] Renamed DTMetadata to DecisionTreeMetadata. Small doc updates. 3726d20 [Joseph K. Bradley] Small code improvements based on code review. ac0b9f8 [Joseph K. Bradley] Small updates based on code review. Main change: Now using << instead of math.pow. db0d773 [Joseph K. Bradley] scala style fix 6a38f48 [Joseph K. Bradley] Added DTMetadata class for cleaner code 931a3a7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 797f68a [Joseph K. Bradley] Fixed DecisionTreeSuite bug for training second level. Needed to update treePointToNodeIndex with groupShift. f40381c [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 5f2dec2 [Joseph K. Bradley] Fixed scalastyle issue in TreePoint 6b5651e [Joseph K. Bradley] Updates based on code review. 1 major change: persisting to memory + disk, not just memory. 2d2aaaf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 26d10dd [Joseph K. Bradley] Removed tree/model/Filter.scala since no longer used. Removed debugging println calls in DecisionTree.scala. 356daba [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 c1565a5 [Joseph K. Bradley] Small DecisionTree updates: * Simplification: Updated calculateGainForSplit to take aggregates for a single (feature, split) pair. * Internal doc: findAggForOrderedFeatureClassification b914f3b [Joseph K. Bradley] DecisionTree optimization: eliminated filters + small changes b2ed1f3 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals --- .../spark/mllib/tree/DecisionTree.scala | 878 ++++++++---------- .../tree/impl/DecisionTreeMetadata.scala | 101 ++ .../spark/mllib/tree/impl/TreePoint.scala | 30 +- .../apache/spark/mllib/tree/model/Bin.scala | 18 +- .../mllib/tree/model/DecisionTreeModel.scala | 2 +- .../spark/mllib/tree/model/Filter.scala | 28 - .../apache/spark/mllib/tree/model/Node.scala | 16 +- .../apache/spark/mllib/tree/model/Split.scala | 5 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 167 ++-- 9 files changed, 615 insertions(+), 630 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 2a3107a13e916..6b9a8f72c244e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -27,7 +27,7 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impl.{TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TimeTracker, TreePoint} import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD @@ -62,43 +62,38 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo timer.start("init") val retaggedInput = input.retag(classOf[LabeledPoint]) + val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) logDebug("algo = " + strategy.algo) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. timer.start("findSplitsBins") - val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) val numBins = bins(0).length timer.stop("findSplitsBins") logDebug("numBins = " + numBins) + // Bin feature values (TreePoint representation). // Cache input RDD for speedup during multiple passes. - val treeInput = TreePoint.convertToTreeRDD(retaggedInput, strategy, bins) + val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) .persist(StorageLevel.MEMORY_AND_DISK) + val numFeatures = metadata.numFeatures // depth of the decision tree val maxDepth = strategy.maxDepth // the max number of nodes possible given the depth of the tree - val maxNumNodes = math.pow(2, maxDepth + 1).toInt - 1 - // Initialize an array to hold filters applied to points for each node. - val filters = new Array[List[Filter]](maxNumNodes) - // The filter at the top node is an empty list. - filters(0) = List() + val maxNumNodes = (2 << maxDepth) - 1 // Initialize an array to hold parent impurity calculations for each node. val parentImpurities = new Array[Double](maxNumNodes) // dummy value for top node (updated during first split calculation) val nodes = new Array[Node](maxNumNodes) - // num features - val numFeatures = treeInput.take(1)(0).binnedFeatures.size // Calculate level for single group construction // Max memory usage for aggregates val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val numElementsPerNode = DecisionTree.getElementsPerNode(numFeatures, numBins, - strategy.numClassesForClassification, strategy.isMulticlassWithCategoricalFeatures, - strategy.algo) + val numElementsPerNode = DecisionTree.getElementsPerNode(metadata, numBins) logDebug("numElementsPerNode = " + numElementsPerNode) val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array @@ -114,9 +109,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo /* * The main idea here is to perform level-wise training of the decision tree nodes thus * reducing the passes over the data from l to log2(l) where l is the total number of nodes. - * Each data sample is checked for validity w.r.t to each node at a given level -- i.e., - * the sample is only used for the split calculation at the node if the sampled would have - * still survived the filters of the parent nodes. + * Each data sample is handled by a particular node at that level (or it reaches a leaf + * beforehand and is not used in later levels. */ var level = 0 @@ -130,22 +124,37 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. timer.start("findBestSplits") val splitsStatsForLevel = DecisionTree.findBestSplits(treeInput, parentImpurities, - strategy, level, filters, splits, bins, maxLevelForSingleGroup, timer) + metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") + val levelNodeIndexOffset = (1 << level) - 1 for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { + val nodeIndex = levelNodeIndexOffset + index + val isLeftChild = level != 0 && nodeIndex % 2 == 1 + val parentNodeIndex = if (isLeftChild) { // -1 for root node + (nodeIndex - 1) / 2 + } else { + (nodeIndex - 2) / 2 + } + // Extract info for this node (index) at the current level. timer.start("extractNodeInfo") - // Extract info for nodes at the current level. extractNodeInfo(nodeSplitStats, level, index, nodes) timer.stop("extractNodeInfo") - timer.start("extractInfoForLowerLevels") + if (level != 0) { + // Set parent. + if (isLeftChild) { + nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) + } else { + nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) + } + } // Extract info for nodes at the next lower level. - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, - filters) + timer.start("extractInfoForLowerLevels") + extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities) timer.stop("extractInfoForLowerLevels") logDebug("final best split = " + nodeSplitStats._1) } - require(math.pow(2, level) == splitsStatsForLevel.length) + require((1 << level) == splitsStatsForLevel.length) // Check whether all the nodes at the current level at leaves. val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) logDebug("all leaf = " + allLeaf) @@ -183,7 +192,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo nodes: Array[Node]): Unit = { val split = nodeSplitStats._1 val stats = nodeSplitStats._2 - val nodeIndex = math.pow(2, level).toInt - 1 + index + val nodeIndex = (1 << level) - 1 + index val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) logDebug("Node = " + node) @@ -198,31 +207,21 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo index: Int, maxDepth: Int, nodeSplitStats: (Split, InformationGainStats), - parentImpurities: Array[Double], - filters: Array[List[Filter]]): Unit = { - // 0 corresponds to the left child node and 1 corresponds to the right child node. - var i = 0 - while (i <= 1) { - // Calculate the index of the node from the node level and the index at the current level. - val nodeIndex = math.pow(2, level + 1).toInt - 1 + 2 * index + i - if (level < maxDepth) { - val impurity = if (i == 0) { - nodeSplitStats._2.leftImpurity - } else { - nodeSplitStats._2.rightImpurity - } - logDebug("nodeIndex = " + nodeIndex + ", impurity = " + impurity) - // noting the parent impurities - parentImpurities(nodeIndex) = impurity - // noting the parents filters for the child nodes - val childFilter = new Filter(nodeSplitStats._1, if (i == 0) -1 else 1) - filters(nodeIndex) = childFilter :: filters((nodeIndex - 1) / 2) - for (filter <- filters(nodeIndex)) { - logDebug("Filter = " + filter) - } - } - i += 1 + parentImpurities: Array[Double]): Unit = { + + if (level >= maxDepth) { + return } + + val leftNodeIndex = (2 << level) - 1 + 2 * index + val leftImpurity = nodeSplitStats._2.leftImpurity + logDebug("leftNodeIndex = " + leftNodeIndex + ", impurity = " + leftImpurity) + parentImpurities(leftNodeIndex) = leftImpurity + + val rightNodeIndex = leftNodeIndex + 1 + val rightImpurity = nodeSplitStats._2.rightImpurity + logDebug("rightNodeIndex = " + rightNodeIndex + ", impurity = " + rightImpurity) + parentImpurities(rightNodeIndex) = rightImpurity } } @@ -434,10 +433,8 @@ object DecisionTree extends Serializable with Logging { * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for constructing the DecisionTree + * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param filters Filters for all nodes at a given level * @param splits possible splits for all features * @param bins possible bins for all features * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. @@ -446,9 +443,9 @@ object DecisionTree extends Serializable with Logging { protected[tree] def findBestSplits( input: RDD[TreePoint], parentImpurities: Array[Double], - strategy: Strategy, + metadata: DecisionTreeMetadata, level: Int, - filters: Array[List[Filter]], + nodes: Array[Node], splits: Array[Array[Split]], bins: Array[Array[Bin]], maxLevelForSingleGroup: Int, @@ -459,34 +456,32 @@ object DecisionTree extends Serializable with Logging { // the nodes are divided into multiple groups at each level with the number of groups // increasing exponentially per level. For example, if maxLevelForSingleGroup is 10, // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. - val numGroups = math.pow(2, level - maxLevelForSingleGroup).toInt + val numGroups = 1 << level - maxLevelForSingleGroup logDebug("numGroups = " + numGroups) var bestSplits = new Array[(Split, InformationGainStats)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 while (groupIndex < numGroups) { - val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, strategy, level, - filters, splits, bins, timer, numGroups, groupIndex) + val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, metadata, level, + nodes, splits, bins, timer, numGroups, groupIndex) bestSplits = Array.concat(bestSplits, bestSplitsForGroup) groupIndex += 1 } bestSplits } else { - findBestSplitsPerGroup(input, parentImpurities, strategy, level, filters, splits, bins, timer) + findBestSplitsPerGroup(input, parentImpurities, metadata, level, nodes, splits, bins, timer) } } - /** + /** * Returns an array of optimal splits for a group of nodes at a given level * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param parentImpurities Impurities for all parent nodes for the current level - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for constructing the DecisionTree + * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param filters Filters for all nodes at a given level * @param splits possible splits for all features - * @param bins possible bins for all features + * @param bins possible bins for all features, indexed as (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. * @return array of splits with best splits for all nodes at a given level. @@ -494,9 +489,9 @@ object DecisionTree extends Serializable with Logging { private def findBestSplitsPerGroup( input: RDD[TreePoint], parentImpurities: Array[Double], - strategy: Strategy, + metadata: DecisionTreeMetadata, level: Int, - filters: Array[List[Filter]], + nodes: Array[Node], splits: Array[Array[Split]], bins: Array[Array[Bin]], timer: TimeTracker, @@ -515,7 +510,7 @@ object DecisionTree extends Serializable with Logging { * We use a bin-wise best split computation strategy instead of a straightforward best split * computation strategy. Instead of analyzing each sample for contribution to the left/right * child node impurity of every split, we first categorize each feature of a sample into a - * bin. Each bin is an interval between a low and high split. Since each splits, and thus bin, + * bin. Each bin is an interval between a low and high split. Since each split, and thus bin, * is ordered (read ordering for categorical variables in the findSplitsBins method), * we exploit this structure to calculate aggregates for bins and then use these aggregates * to calculate information gain for each split. @@ -531,160 +526,124 @@ object DecisionTree extends Serializable with Logging { // numNodes: Number of nodes in this (level of tree, group), // where nodes at deeper (larger) levels may be divided into groups. - val numNodes = math.pow(2, level).toInt / numGroups + val numNodes = (1 << level) / numGroups logDebug("numNodes = " + numNodes) // Find the number of features by looking at the first sample. - val numFeatures = input.first().binnedFeatures.size + val numFeatures = metadata.numFeatures logDebug("numFeatures = " + numFeatures) // numBins: Number of bins = 1 + number of possible splits val numBins = bins(0).length logDebug("numBins = " + numBins) - val numClasses = strategy.numClassesForClassification + val numClasses = metadata.numClasses logDebug("numClasses = " + numClasses) - val isMulticlassClassification = strategy.isMulticlassClassification - logDebug("isMulticlassClassification = " + isMulticlassClassification) + val isMulticlass = metadata.isMulticlass + logDebug("isMulticlass = " + isMulticlass) - val isMulticlassClassificationWithCategoricalFeatures - = strategy.isMulticlassWithCategoricalFeatures - logDebug("isMultiClassWithCategoricalFeatures = " + - isMulticlassClassificationWithCategoricalFeatures) + val isMulticlassWithCategoricalFeatures = metadata.isMulticlassWithCategoricalFeatures + logDebug("isMultiClassWithCategoricalFeatures = " + isMulticlassWithCategoricalFeatures) // shift when more than one group is used at deep tree level val groupShift = numNodes * groupIndex - /** Find the filters used before reaching the current code. */ - def findParentFilters(nodeIndex: Int): List[Filter] = { - if (level == 0) { - List[Filter]() - } else { - val nodeFilterIndex = math.pow(2, level).toInt - 1 + nodeIndex + groupShift - filters(nodeFilterIndex) - } - } - /** - * Find whether the sample is valid input for the current node, i.e., whether it passes through - * all the filters for the current node. + * Get the node index corresponding to this data point. + * This function mimics prediction, passing an example from the root node down to a node + * at the current level being trained; that node's index is returned. + * + * @return Leaf index if the data point reaches a leaf. + * Otherwise, last node reachable in tree matching this example. */ - def isSampleValid(parentFilters: List[Filter], treePoint: TreePoint): Boolean = { - // leaf - if ((level > 0) && (parentFilters.length == 0)) { - return false - } - - // Apply each filter and check sample validity. Return false when invalid condition found. - parentFilters.foreach { filter => - val featureIndex = filter.split.feature - val comparison = filter.comparison - val isFeatureContinuous = filter.split.featureType == Continuous - if (isFeatureContinuous) { - val binId = treePoint.binnedFeatures(featureIndex) - val bin = bins(featureIndex)(binId) - val featureValue = bin.highSplit.threshold - val threshold = filter.split.threshold - comparison match { - case -1 => if (featureValue > threshold) return false - case 1 => if (featureValue <= threshold) return false + def predictNodeIndex(node: Node, binnedFeatures: Array[Int]): Int = { + if (node.isLeaf) { + node.id + } else { + val featureIndex = node.split.get.feature + val splitLeft = node.split.get.featureType match { + case Continuous => { + val binIndex = binnedFeatures(featureIndex) + val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold + // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] + // We do not need to check lowSplit since bins are separated by splits. + featureValueUpperBound <= node.split.get.threshold } - } else { - val numFeatureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits = - numBins > math.pow(2, numFeatureCategories.toInt - 1) - 1 - val isUnorderedFeature = - isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits - val featureValue = if (isUnorderedFeature) { - treePoint.binnedFeatures(featureIndex) + case Categorical => { + val featureValue = if (metadata.isUnordered(featureIndex)) { + binnedFeatures(featureIndex) + } else { + val binIndex = binnedFeatures(featureIndex) + bins(featureIndex)(binIndex).category + } + node.split.get.categories.contains(featureValue) + } + case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") + } + if (node.leftNode.isEmpty || node.rightNode.isEmpty) { + // Return index from next layer of nodes to train + if (splitLeft) { + node.id * 2 + 1 // left } else { - val binId = treePoint.binnedFeatures(featureIndex) - bins(featureIndex)(binId).category + node.id * 2 + 2 // right } - val containsFeature = filter.split.categories.contains(featureValue) - comparison match { - case -1 => if (!containsFeature) return false - case 1 => if (containsFeature) return false + } else { + if (splitLeft) { + predictNodeIndex(node.leftNode.get, binnedFeatures) + } else { + predictNodeIndex(node.rightNode.get, binnedFeatures) } } } + } - // Return true when the sample is valid for all filters. - true + def nodeIndexToLevel(idx: Int): Int = { + if (idx == 0) { + 0 + } else { + math.floor(math.log(idx) / math.log(2)).toInt + } } + // Used for treePointToNodeIndex + val levelOffset = (1 << level) - 1 + /** - * Finds bins for all nodes (and all features) at a given level. - * For l nodes, k features the storage is as follows: - * label, b_11, b_12, .. , b_1k, b_21, b_22, .. , b_2k, b_l1, b_l2, .. , b_lk, - * where b_ij is an integer between 0 and numBins - 1 for regressions and binary - * classification and the categorical feature value in multiclass classification. - * Invalid sample is denoted by noting bin for feature 1 as -1. - * - * For unordered features, the "bin index" returned is actually the feature value (category). - * - * @return Array of size 1 + numFeatures * numNodes, where - * arr(0) = label for labeledPoint, and - * arr(1 + numFeatures * nodeIndex + featureIndex) = - * bin index for this labeledPoint - * (or InvalidBinIndex if labeledPoint is not handled by this node) + * Find the node index for the given example. + * Nodes are indexed from 0 at the start of this (level, group). + * If the example does not reach this level, returns a value < 0. */ - def findBinsForLevel(treePoint: TreePoint): Array[Double] = { - // Calculate bin index and label per feature per node. - val arr = new Array[Double](1 + (numFeatures * numNodes)) - // First element of the array is the label of the instance. - arr(0) = treePoint.label - // Iterate over nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - val parentFilters = findParentFilters(nodeIndex) - // Find out whether the sample qualifies for the particular node. - val sampleValid = isSampleValid(parentFilters, treePoint) - val shift = 1 + numFeatures * nodeIndex - if (!sampleValid) { - // Mark one bin as -1 is sufficient. - arr(shift) = InvalidBinIndex - } else { - var featureIndex = 0 - while (featureIndex < numFeatures) { - arr(shift + featureIndex) = treePoint.binnedFeatures(featureIndex) - featureIndex += 1 - } - } - nodeIndex += 1 + def treePointToNodeIndex(treePoint: TreePoint): Int = { + if (level == 0) { + 0 + } else { + val globalNodeIndex = predictNodeIndex(nodes(0), treePoint.binnedFeatures) + // Get index for this (level, group). + globalNodeIndex - levelOffset - groupShift } - arr } - // Find feature bins for all nodes at a level. - timer.start("aggregation") - val binMappedRDD = input.map(x => findBinsForLevel(x)) - /** * Increment aggregate in location for (node, feature, bin, label). * - * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. - * Array of size 1 + (numFeatures * numNodes). + * @param treePoint Data point being aggregated. * @param agg Array storing aggregate calculation, of size: * numClasses * numBins * numFeatures * numNodes. * Indexed by (node, feature, bin, label) where label is the least significant bit. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ def updateBinForOrderedFeature( - arr: Array[Double], + treePoint: TreePoint, agg: Array[Double], nodeIndex: Int, - label: Double, featureIndex: Int): Unit = { - // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex // Update the left or right count for one bin. val aggIndex = numClasses * numBins * numFeatures * nodeIndex + numClasses * numBins * featureIndex + - numClasses * arr(arrIndex).toInt + - label.toInt + numClasses * treePoint.binnedFeatures(featureIndex) + + treePoint.label.toInt agg(aggIndex) += 1 } @@ -693,8 +652,8 @@ object DecisionTree extends Serializable with Logging { * where [bins] ranges over all bins. * Updates left or right side of aggregate depending on split. * - * @param arr arr(0) = label. - * arr(1 + featureIndex + nodeIndex * numFeatures) = feature value (category) + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @param treePoint Data point being aggregated. * @param agg Indexed by (left/right, node, feature, bin, label) * where label is the least significant bit. * The left/right specifier is a 0/1 index indicating left/right child info. @@ -703,21 +662,18 @@ object DecisionTree extends Serializable with Logging { def updateBinForUnorderedFeature( nodeIndex: Int, featureIndex: Int, - arr: Array[Double], - label: Double, + treePoint: TreePoint, agg: Array[Double], rightChildShift: Int): Unit = { - // Find the bin index for this feature. - val arrIndex = 1 + numFeatures * nodeIndex + featureIndex - val featureValue = arr(arrIndex).toInt + val featureValue = treePoint.binnedFeatures(featureIndex) // Update the left or right count for one bin. val aggShift = numClasses * numBins * numFeatures * nodeIndex + numClasses * numBins * featureIndex + - label.toInt + treePoint.label.toInt // Find all matching bins and increment their values - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val numCategoricalBins = math.pow(2.0, featureCategories - 1).toInt - 1 + val featureCategories = metadata.featureArity(featureIndex) + val numCategoricalBins = (1 << featureCategories - 1) - 1 var binIndex = 0 while (binIndex < numCategoricalBins) { val aggIndex = aggShift + binIndex * numClasses @@ -733,30 +689,21 @@ object DecisionTree extends Serializable with Logging { /** * Helper for binSeqOp. * - * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. - * Array of size 1 + (numFeatures * numNodes). * @param agg Array storing aggregate calculation, of size: * numClasses * numBins * numFeatures * numNodes. * Indexed by (node, feature, bin, label) where label is the least significant bit. + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ - def binaryOrNotCategoricalBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) - featureIndex += 1 - } - } - nodeIndex += 1 + def binaryOrNotCategoricalBinSeqOp( + agg: Array[Double], + treePoint: TreePoint, + nodeIndex: Int): Unit = { + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) + featureIndex += 1 } } @@ -765,49 +712,28 @@ object DecisionTree extends Serializable with Logging { /** * Helper for binSeqOp. * - * @param arr Bin mapping from findBinsForLevel. arr(0) stores the class label. - * Array of size 1 + (numFeatures * numNodes). - * For ordered features, - * arr(1 + featureIndex + nodeIndex * numFeatures) = bin index. - * For unordered features, - * arr(1 + featureIndex + nodeIndex * numFeatures) = feature value (category). * @param agg Array storing aggregate calculation. * For ordered features, this is of size: * numClasses * numBins * numFeatures * numNodes. * For unordered features, this is of size: * 2 * numClasses * numBins * numFeatures * numNodes. + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). */ - def multiclassWithCategoricalBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - if (isFeatureContinuous) { - updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) - } else { - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - if (isSpaceSufficientForAllCategoricalSplits) { - updateBinForUnorderedFeature(nodeIndex, featureIndex, arr, label, agg, - rightChildShift) - } else { - updateBinForOrderedFeature(arr, agg, nodeIndex, label, featureIndex) - } - } - featureIndex += 1 - } + def multiclassWithCategoricalBinSeqOp( + agg: Array[Double], + treePoint: TreePoint, + nodeIndex: Int): Unit = { + val label = treePoint.label + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (metadata.isUnordered(featureIndex)) { + updateBinForUnorderedFeature(nodeIndex, featureIndex, treePoint, agg, rightChildShift) + } else { + updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) } - nodeIndex += 1 + featureIndex += 1 } } @@ -818,36 +744,25 @@ object DecisionTree extends Serializable with Logging { * * @param agg Array storing aggregate calculation, updated by this function. * Size: 3 * numBins * numFeatures * numNodes - * @param arr Bin mapping from findBinsForLevel. - * Array of size 1 + (numFeatures * numNodes). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). * @return agg */ - def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]): Unit = { - // Iterate over all nodes. - var nodeIndex = 0 - while (nodeIndex < numNodes) { - // Check whether the instance was valid for this nodeIndex. - val validSignalIndex = 1 + numFeatures * nodeIndex - val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex - if (isSampleValidForNode) { - // actual class label - val label = arr(0) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Find the bin index for this feature. - val arrShift = 1 + numFeatures * nodeIndex - val arrIndex = arrShift + featureIndex - // Update count, sum, and sum^2 for one bin. - val aggShift = 3 * numBins * numFeatures * nodeIndex - val aggIndex = aggShift + 3 * featureIndex * numBins + arr(arrIndex).toInt * 3 - agg(aggIndex) = agg(aggIndex) + 1 - agg(aggIndex + 1) = agg(aggIndex + 1) + label - agg(aggIndex + 2) = agg(aggIndex + 2) + label * label - featureIndex += 1 - } - } - nodeIndex += 1 + def regressionBinSeqOp(agg: Array[Double], treePoint: TreePoint, nodeIndex: Int): Unit = { + val label = treePoint.label + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // Update count, sum, and sum^2 for one bin. + val binIndex = treePoint.binnedFeatures(featureIndex) + val aggIndex = + 3 * numBins * numFeatures * nodeIndex + + 3 * numBins * featureIndex + + 3 * binIndex + agg(aggIndex) += 1 + agg(aggIndex + 1) += label + agg(aggIndex + 2) += label * label + featureIndex += 1 } } @@ -866,26 +781,30 @@ object DecisionTree extends Serializable with Logging { * 2 * numClasses * numBins * numFeatures * numNodes for unordered features. * Size for regression: * 3 * numBins * numFeatures * numNodes. - * @param arr Bin mapping from findBinsForLevel. - * Array of size 1 + (numFeatures * numNodes). + * @param treePoint Data point being aggregated. * @return agg */ - def binSeqOp(agg: Array[Double], arr: Array[Double]): Array[Double] = { - strategy.algo match { - case Classification => - if(isMulticlassClassificationWithCategoricalFeatures) { - multiclassWithCategoricalBinSeqOp(arr, agg) + def binSeqOp(agg: Array[Double], treePoint: TreePoint): Array[Double] = { + val nodeIndex = treePointToNodeIndex(treePoint) + // If the example does not reach this level, then nodeIndex < 0. + // If the example reaches this level but is handled in a different group, + // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). + if (nodeIndex >= 0 && nodeIndex < numNodes) { + if (metadata.isClassification) { + if (isMulticlassWithCategoricalFeatures) { + multiclassWithCategoricalBinSeqOp(agg, treePoint, nodeIndex) } else { - binaryOrNotCategoricalBinSeqOp(arr, agg) + binaryOrNotCategoricalBinSeqOp(agg, treePoint, nodeIndex) } - case Regression => regressionBinSeqOp(arr, agg) + } else { + regressionBinSeqOp(agg, treePoint, nodeIndex) + } } agg } // Calculate bin aggregate length for classification or regression. - val binAggregateLength = numNodes * getElementsPerNode(numFeatures, numBins, numClasses, - isMulticlassClassificationWithCategoricalFeatures, strategy.algo) + val binAggregateLength = numNodes * getElementsPerNode(metadata, numBins) logDebug("binAggregateLength = " + binAggregateLength) /** @@ -905,144 +824,134 @@ object DecisionTree extends Serializable with Logging { } // Calculate bin aggregates. + timer.start("aggregation") val binAggregates = { - binMappedRDD.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp,binCombOp) + input.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) } timer.stop("aggregation") logDebug("binAggregates.length = " + binAggregates.length) /** - * Calculates the information gain for all splits based upon left/right split aggregates. - * @param leftNodeAgg left node aggregates - * @param featureIndex feature index - * @param splitIndex split index - * @param rightNodeAgg right node aggregate + * Calculate the information gain for a given (feature, split) based upon left/right aggregates. + * @param leftNodeAgg left node aggregates for this (feature, split) + * @param rightNodeAgg right node aggregate for this (feature, split) * @param topImpurity impurity of the parent node * @return information gain and statistics for all splits */ def calculateGainForSplit( - leftNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int, - splitIndex: Int, - rightNodeAgg: Array[Array[Array[Double]]], + leftNodeAgg: Array[Double], + rightNodeAgg: Array[Double], topImpurity: Double): InformationGainStats = { - strategy.algo match { - case Classification => - val leftCounts: Array[Double] = leftNodeAgg(featureIndex)(splitIndex) - val rightCounts: Array[Double] = rightNodeAgg(featureIndex)(splitIndex) - val leftTotalCount = leftCounts.sum - val rightTotalCount = rightCounts.sum - - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val rootNodeCounts = new Array[Double](numClasses) - var classIndex = 0 - while (classIndex < numClasses) { - rootNodeCounts(classIndex) = leftCounts(classIndex) + rightCounts(classIndex) - classIndex += 1 - } - strategy.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) - } - } + if (metadata.isClassification) { + val leftTotalCount = leftNodeAgg.sum + val rightTotalCount = rightNodeAgg.sum - val totalCount = leftTotalCount + rightTotalCount - if (totalCount == 0) { - // Return arbitrary prediction. - return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + val rootNodeCounts = new Array[Double](numClasses) + var classIndex = 0 + while (classIndex < numClasses) { + rootNodeCounts(classIndex) = leftNodeAgg(classIndex) + rightNodeAgg(classIndex) + classIndex += 1 + } + metadata.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) } + } - // Sum of count for each label - val leftRightCounts: Array[Double] = - leftCounts.zip(rightCounts).map { case (leftCount, rightCount) => - leftCount + rightCount - } + val totalCount = leftTotalCount + rightTotalCount + if (totalCount == 0) { + // Return arbitrary prediction. + return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + } - def indexOfLargestArrayElement(array: Array[Double]): Int = { - val result = array.foldLeft(-1, Double.MinValue, 0) { - case ((maxIndex, maxValue, currentIndex), currentValue) => - if (currentValue > maxValue) { - (currentIndex, currentValue, currentIndex + 1) - } else { - (maxIndex, maxValue, currentIndex + 1) - } - } - if (result._1 < 0) { - throw new RuntimeException("DecisionTree internal error:" + - " calculateGainForSplit failed in indexOfLargestArrayElement") - } - result._1 + // Sum of count for each label + val leftrightNodeAgg: Array[Double] = + leftNodeAgg.zip(rightNodeAgg).map { case (leftCount, rightCount) => + leftCount + rightCount } - val predict = indexOfLargestArrayElement(leftRightCounts) - val prob = leftRightCounts(predict) / totalCount - - val leftImpurity = if (leftTotalCount == 0) { - topImpurity - } else { - strategy.impurity.calculate(leftCounts, leftTotalCount) + def indexOfLargestArrayElement(array: Array[Double]): Int = { + val result = array.foldLeft(-1, Double.MinValue, 0) { + case ((maxIndex, maxValue, currentIndex), currentValue) => + if (currentValue > maxValue) { + (currentIndex, currentValue, currentIndex + 1) + } else { + (maxIndex, maxValue, currentIndex + 1) + } } - val rightImpurity = if (rightTotalCount == 0) { - topImpurity - } else { - strategy.impurity.calculate(rightCounts, rightTotalCount) + if (result._1 < 0) { + throw new RuntimeException("DecisionTree internal error:" + + " calculateGainForSplit failed in indexOfLargestArrayElement") } + result._1 + } - val leftWeight = leftTotalCount / totalCount - val rightWeight = rightTotalCount / totalCount + val predict = indexOfLargestArrayElement(leftrightNodeAgg) + val prob = leftrightNodeAgg(predict) / totalCount - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + val leftImpurity = if (leftTotalCount == 0) { + topImpurity + } else { + metadata.impurity.calculate(leftNodeAgg, leftTotalCount) + } + val rightImpurity = if (rightTotalCount == 0) { + topImpurity + } else { + metadata.impurity.calculate(rightNodeAgg, rightTotalCount) + } - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + val leftWeight = leftTotalCount / totalCount + val rightWeight = rightTotalCount / totalCount - case Regression => - val leftCount = leftNodeAgg(featureIndex)(splitIndex)(0) - val leftSum = leftNodeAgg(featureIndex)(splitIndex)(1) - val leftSumSquares = leftNodeAgg(featureIndex)(splitIndex)(2) + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - val rightCount = rightNodeAgg(featureIndex)(splitIndex)(0) - val rightSum = rightNodeAgg(featureIndex)(splitIndex)(1) - val rightSumSquares = rightNodeAgg(featureIndex)(splitIndex)(2) + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val count = leftCount + rightCount - val sum = leftSum + rightSum - val sumSquares = leftSumSquares + rightSumSquares - strategy.impurity.calculate(count, sum, sumSquares) - } - } + } else { + // Regression - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, - rightSum / rightCount) - } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity ,topImpurity, - Double.MinValue, leftSum / leftCount) + val leftCount = leftNodeAgg(0) + val leftSum = leftNodeAgg(1) + val leftSumSquares = leftNodeAgg(2) + + val rightCount = rightNodeAgg(0) + val rightSum = rightNodeAgg(1) + val rightSumSquares = rightNodeAgg(2) + + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + val count = leftCount + rightCount + val sum = leftSum + rightSum + val sumSquares = leftSumSquares + rightSumSquares + metadata.impurity.calculate(count, sum, sumSquares) } + } + + if (leftCount == 0) { + return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, + rightSum / rightCount) + } + if (rightCount == 0) { + return new InformationGainStats(0, topImpurity, topImpurity, + Double.MinValue, leftSum / leftCount) + } - val leftImpurity = strategy.impurity.calculate(leftCount, leftSum, leftSumSquares) - val rightImpurity = strategy.impurity.calculate(rightCount, rightSum, rightSumSquares) + val leftImpurity = metadata.impurity.calculate(leftCount, leftSum, leftSumSquares) + val rightImpurity = metadata.impurity.calculate(rightCount, rightSum, rightSumSquares) - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val leftWeight = leftCount.toDouble / (leftCount + rightCount) + val rightWeight = rightCount.toDouble / (leftCount + rightCount) - val gain = { - if (level > 0) { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } else { - impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - } - } + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - val predict = (leftSum + rightSum) / (leftCount + rightCount) - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) + val predict = (leftSum + rightSum) / (leftCount + rightCount) + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) } } @@ -1065,6 +974,19 @@ object DecisionTree extends Serializable with Logging { binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { + /** + * The input binData is indexed as (feature, bin, class). + * This computes cumulative sums over splits. + * Each (feature, class) pair is handled separately. + * Note: numSplits = numBins - 1. + * @param leftNodeAgg Each (feature, class) slice is an array over splits. + * Element i (i = 0, ..., numSplits - 2) is set to be + * the cumulative sum (from left) over binData for bins 0, ..., i. + * @param rightNodeAgg Each (feature, class) slice is an array over splits. + * Element i (i = 1, ..., numSplits - 1) is set to be + * the cumulative sum (from right) over binData for bins + * numBins - 1, ..., numBins - 1 - i. + */ def findAggForOrderedFeatureClassification( leftNodeAgg: Array[Array[Array[Double]]], rightNodeAgg: Array[Array[Array[Double]]], @@ -1169,45 +1091,32 @@ object DecisionTree extends Serializable with Logging { } } - strategy.algo match { - case Classification => - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (isMulticlassClassificationWithCategoricalFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - if (isFeatureContinuous) { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } else { - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - if (isSpaceSufficientForAllCategoricalSplits) { - findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } else { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } - } - } else { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } - featureIndex += 1 - } - - (leftNodeAgg, rightNodeAgg) - case Regression => - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) - featureIndex += 1 + if (metadata.isClassification) { + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (metadata.isUnordered(featureIndex)) { + findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) + } else { + findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) } - (leftNodeAgg, rightNodeAgg) + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) + } else { + // Regression + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) + val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) } } @@ -1225,8 +1134,9 @@ object DecisionTree extends Serializable with Logging { val numSplitsForFeature = getNumSplitsForFeature(featureIndex) var splitIndex = 0 while (splitIndex < numSplitsForFeature) { - gains(featureIndex)(splitIndex) = calculateGainForSplit(leftNodeAgg, featureIndex, - splitIndex, rightNodeAgg, nodeImpurity) + gains(featureIndex)(splitIndex) = + calculateGainForSplit(leftNodeAgg(featureIndex)(splitIndex), + rightNodeAgg(featureIndex)(splitIndex), nodeImpurity) splitIndex += 1 } featureIndex += 1 @@ -1238,18 +1148,14 @@ object DecisionTree extends Serializable with Logging { * Get the number of splits for a feature. */ def getNumSplitsForFeature(featureIndex: Int): Int = { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty - if (isFeatureContinuous) { + if (metadata.isContinuous(featureIndex)) { numBins - 1 } else { // Categorical feature - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits = - numBins > math.pow(2, featureCategories.toInt - 1) - 1 - if (isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits) { - math.pow(2.0, featureCategories - 1).toInt - 1 + val featureCategories = metadata.featureArity(featureIndex) + if (metadata.isUnordered(featureIndex)) { + (1 << featureCategories - 1) - 1 } else { - // Ordered features featureCategories } } @@ -1308,29 +1214,29 @@ object DecisionTree extends Serializable with Logging { * Get bin data for one node. */ def getBinDataForNode(node: Int): Array[Double] = { - strategy.algo match { - case Classification => - if (isMulticlassClassificationWithCategoricalFeatures) { - val shift = numClasses * node * numBins * numFeatures - val rightChildShift = numClasses * numBins * numFeatures * numNodes - val binsForNode = { - val leftChildData - = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - val rightChildData - = binAggregates.slice(rightChildShift + shift, - rightChildShift + shift + numClasses * numBins * numFeatures) - leftChildData ++ rightChildData - } - binsForNode - } else { - val shift = numClasses * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - binsForNode + if (metadata.isClassification) { + if (isMulticlassWithCategoricalFeatures) { + val shift = numClasses * node * numBins * numFeatures + val rightChildShift = numClasses * numBins * numFeatures * numNodes + val binsForNode = { + val leftChildData + = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) + val rightChildData + = binAggregates.slice(rightChildShift + shift, + rightChildShift + shift + numClasses * numBins * numFeatures) + leftChildData ++ rightChildData } - case Regression => - val shift = 3 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) binsForNode + } else { + val shift = numClasses * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) + binsForNode + } + } else { + // Regression + val shift = 3 * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) + binsForNode } } @@ -1340,7 +1246,7 @@ object DecisionTree extends Serializable with Logging { // Iterating over all nodes at this level var node = 0 while (node < numNodes) { - val nodeImpurityIndex = math.pow(2, level).toInt - 1 + node + groupShift + val nodeImpurityIndex = (1 << level) - 1 + node + groupShift val binsForNode: Array[Double] = getBinDataForNode(node) logDebug("nodeImpurityIndex = " + nodeImpurityIndex) val parentNodeImpurity = parentImpurities(nodeImpurityIndex) @@ -1358,20 +1264,15 @@ object DecisionTree extends Serializable with Logging { * * @param numBins Number of bins = 1 + number of possible splits. */ - private def getElementsPerNode( - numFeatures: Int, - numBins: Int, - numClasses: Int, - isMulticlassClassificationWithCategoricalFeatures: Boolean, - algo: Algo): Int = { - algo match { - case Classification => - if (isMulticlassClassificationWithCategoricalFeatures) { - 2 * numClasses * numBins * numFeatures - } else { - numClasses * numBins * numFeatures - } - case Regression => 3 * numBins * numFeatures + private def getElementsPerNode(metadata: DecisionTreeMetadata, numBins: Int): Int = { + if (metadata.isClassification) { + if (metadata.isMulticlassWithCategoricalFeatures) { + 2 * metadata.numClasses * numBins * metadata.numFeatures + } else { + metadata.numClasses * numBins * metadata.numFeatures + } + } else { + 3 * numBins * metadata.numFeatures } } @@ -1390,16 +1291,15 @@ object DecisionTree extends Serializable with Logging { * For multiclass classification with a low-arity feature * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), * the feature is split based on subsets of categories. - * There are math.pow(2, maxFeatureValue - 1) - 1 splits. + * There are (1 << maxFeatureValue - 1) - 1 splits. * (b) "ordered features" * For regression and binary classification, * and for multiclass classification with a high-arity feature, * there is one bin per category. * * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing - * parameters for construction the DecisionTree - * @return A tuple of (splits,bins). + * @param metadata Learning and dataset metadata + * @return A tuple of (splits, bins). * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] * of size (numFeatures, numBins - 1). * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] @@ -1407,19 +1307,18 @@ object DecisionTree extends Serializable with Logging { */ protected[tree] def findSplitsBins( input: RDD[LabeledPoint], - strategy: Strategy): (Array[Array[Split]], Array[Array[Bin]]) = { + metadata: DecisionTreeMetadata): (Array[Array[Split]], Array[Array[Bin]]) = { val count = input.count() // Find the number of features by looking at the first sample val numFeatures = input.take(1)(0).features.size - val maxBins = strategy.maxBins + val maxBins = metadata.maxBins val numBins = if (maxBins <= count) maxBins else count.toInt logDebug("numBins = " + numBins) - val isMulticlassClassification = strategy.isMulticlassClassification - logDebug("isMulticlassClassification = " + isMulticlassClassification) - + val isMulticlass = metadata.isMulticlass + logDebug("isMulticlass = " + isMulticlass) /* * Ensure numBins is always greater than the categories. For multiclass classification, @@ -1431,13 +1330,12 @@ object DecisionTree extends Serializable with Logging { * by the number of training examples. * TODO: Allow this case, where we simply will know nothing about some categories. */ - if (strategy.categoricalFeaturesInfo.size > 0) { - val maxCategoriesForFeatures = strategy.categoricalFeaturesInfo.maxBy(_._2)._2 + if (metadata.featureArity.size > 0) { + val maxCategoriesForFeatures = metadata.featureArity.maxBy(_._2)._2 require(numBins > maxCategoriesForFeatures, "numBins should be greater than max categories " + "in categorical features") } - // Calculate the number of sample for approximate quantile calculation. val requiredSamples = numBins*numBins val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0 @@ -1451,7 +1349,7 @@ object DecisionTree extends Serializable with Logging { val stride: Double = numSamples.toDouble / numBins logDebug("stride = " + stride) - strategy.quantileCalculationStrategy match { + metadata.quantileStrategy match { case Sort => val splits = Array.ofDim[Split](numFeatures, numBins - 1) val bins = Array.ofDim[Bin](numFeatures, numBins) @@ -1462,7 +1360,7 @@ object DecisionTree extends Serializable with Logging { var featureIndex = 0 while (featureIndex < numFeatures) { // Check whether the feature is continuous. - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + val isFeatureContinuous = metadata.isContinuous(featureIndex) if (isFeatureContinuous) { val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted val stride: Double = numSamples.toDouble / numBins @@ -1475,18 +1373,14 @@ object DecisionTree extends Serializable with Logging { splits(featureIndex)(index) = split } } else { // Categorical feature - val featureCategories = strategy.categoricalFeaturesInfo(featureIndex) - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 + val featureCategories = metadata.featureArity(featureIndex) // Use different bin/split calculation strategy for categorical features in multiclass // classification that satisfy the space constraint. - val isUnorderedFeature = - isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits - if (isUnorderedFeature) { + if (metadata.isUnordered(featureIndex)) { // 2^(maxFeatureValue- 1) - 1 combinations var index = 0 - while (index < math.pow(2.0, featureCategories - 1).toInt - 1) { + while (index < (1 << featureCategories - 1) - 1) { val categories: List[Double] = extractMultiClassCategories(index + 1, featureCategories) splits(featureIndex)(index) @@ -1516,7 +1410,7 @@ object DecisionTree extends Serializable with Logging { * centroidForCategories is a mapping: category (for the given feature) --> centroid */ val centroidForCategories = { - if (isMulticlassClassification) { + if (isMulticlass) { // For categorical variables in multiclass classification, // each bin is a category. The bins are sorted and they // are ordered by calculating the impurity of their corresponding labels. @@ -1524,7 +1418,7 @@ object DecisionTree extends Serializable with Logging { .groupBy(_._1) .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) .map(x => (x._1, x._2.values.toArray)) - .map(x => (x._1, strategy.impurity.calculate(x._2, x._2.sum))) + .map(x => (x._1, metadata.impurity.calculate(x._2, x._2.sum))) } else { // regression or binary classification // For categorical variables in regression and binary classification, // each bin is a category. The bins are sorted and they @@ -1576,7 +1470,7 @@ object DecisionTree extends Serializable with Logging { // Find all bins. featureIndex = 0 while (featureIndex < numFeatures) { - val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + val isFeatureContinuous = metadata.isContinuous(featureIndex) if (isFeatureContinuous) { // Bins for categorical variables are already assigned. bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), splits(featureIndex)(0), Continuous, Double.MinValue) @@ -1590,7 +1484,7 @@ object DecisionTree extends Serializable with Logging { } featureIndex += 1 } - (splits,bins) + (splits, bins) case MinMax => throw new UnsupportedOperationException("minmax not supported yet.") case ApproxHist => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala new file mode 100644 index 0000000000000..d9eda354dc986 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import scala.collection.mutable + +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.rdd.RDD + + +/** + * Learning and dataset metadata for DecisionTree. + * + * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. + * For regression: fixed at 0 (no meaning). + * @param featureArity Map: categorical feature index --> arity. + * I.e., the feature takes values in {0, ..., arity - 1}. + */ +private[tree] class DecisionTreeMetadata( + val numFeatures: Int, + val numExamples: Long, + val numClasses: Int, + val maxBins: Int, + val featureArity: Map[Int, Int], + val unorderedFeatures: Set[Int], + val impurity: Impurity, + val quantileStrategy: QuantileStrategy) extends Serializable { + + def isUnordered(featureIndex: Int): Boolean = unorderedFeatures.contains(featureIndex) + + def isClassification: Boolean = numClasses >= 2 + + def isMulticlass: Boolean = numClasses > 2 + + def isMulticlassWithCategoricalFeatures: Boolean = isMulticlass && (featureArity.size > 0) + + def isCategorical(featureIndex: Int): Boolean = featureArity.contains(featureIndex) + + def isContinuous(featureIndex: Int): Boolean = !featureArity.contains(featureIndex) + +} + +private[tree] object DecisionTreeMetadata { + + def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { + + val numFeatures = input.take(1)(0).features.size + val numExamples = input.count() + val numClasses = strategy.algo match { + case Classification => strategy.numClassesForClassification + case Regression => 0 + } + + val maxBins = math.min(strategy.maxBins, numExamples).toInt + val log2MaxBinsp1 = math.log(maxBins + 1) / math.log(2.0) + + val unorderedFeatures = new mutable.HashSet[Int]() + if (numClasses > 2) { + strategy.categoricalFeaturesInfo.foreach { case (f, k) => + if (k - 1 < log2MaxBinsp1) { + // Note: The above check is equivalent to checking: + // numUnorderedBins = (1 << k - 1) - 1 < maxBins + unorderedFeatures.add(f) + } else { + // TODO: Allow this case, where we simply will know nothing about some categories? + require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + + s"in categorical features (>= $k)") + } + } + } else { + strategy.categoricalFeaturesInfo.foreach { case (f, k) => + require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + + s"in categorical features (>= $k)") + } + } + + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, maxBins, + strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, + strategy.impurity, strategy.quantileCalculationStrategy) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala index ccac1031fd9d9..170e43e222083 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -18,7 +18,6 @@ package org.apache.spark.mllib.tree.impl import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.model.Bin import org.apache.spark.rdd.RDD @@ -48,50 +47,35 @@ private[tree] object TreePoint { * Convert an input dataset into its TreePoint representation, * binning feature values in preparation for DecisionTree training. * @param input Input dataset. - * @param strategy DecisionTree training info, used for dataset metadata. * @param bins Bins for features, of size (numFeatures, numBins). + * @param metadata Learning and dataset metadata * @return TreePoint dataset representation */ def convertToTreeRDD( input: RDD[LabeledPoint], - strategy: Strategy, - bins: Array[Array[Bin]]): RDD[TreePoint] = { + bins: Array[Array[Bin]], + metadata: DecisionTreeMetadata): RDD[TreePoint] = { input.map { x => - TreePoint.labeledPointToTreePoint(x, strategy.isMulticlassClassification, bins, - strategy.categoricalFeaturesInfo) + TreePoint.labeledPointToTreePoint(x, bins, metadata) } } /** * Convert one LabeledPoint into its TreePoint representation. * @param bins Bins for features, of size (numFeatures, numBins). - * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity */ private def labeledPointToTreePoint( labeledPoint: LabeledPoint, - isMulticlassClassification: Boolean, bins: Array[Array[Bin]], - categoricalFeaturesInfo: Map[Int, Int]): TreePoint = { + metadata: DecisionTreeMetadata): TreePoint = { val numFeatures = labeledPoint.features.size val numBins = bins(0).size val arr = new Array[Int](numFeatures) var featureIndex = 0 while (featureIndex < numFeatures) { - val featureInfo = categoricalFeaturesInfo.get(featureIndex) - val isFeatureContinuous = featureInfo.isEmpty - if (isFeatureContinuous) { - arr(featureIndex) = findBin(featureIndex, labeledPoint, isFeatureContinuous, false, - bins, categoricalFeaturesInfo) - } else { - val featureCategories = featureInfo.get - val isSpaceSufficientForAllCategoricalSplits - = numBins > math.pow(2, featureCategories.toInt - 1) - 1 - val isUnorderedFeature = - isMulticlassClassification && isSpaceSufficientForAllCategoricalSplits - arr(featureIndex) = findBin(featureIndex, labeledPoint, isFeatureContinuous, - isUnorderedFeature, bins, categoricalFeaturesInfo) - } + arr(featureIndex) = findBin(featureIndex, labeledPoint, metadata.isContinuous(featureIndex), + metadata.isUnordered(featureIndex), bins, metadata.featureArity) featureIndex += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index c89c1e371a40e..af35d88f713e5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -20,15 +20,25 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.FeatureType._ /** - * Used for "binning" the features bins for faster best split calculation. For a continuous - * feature, a bin is determined by a low and a high "split". For a categorical feature, - * the a bin is determined using a single label value (category). + * Used for "binning" the features bins for faster best split calculation. + * + * For a continuous feature, the bin is determined by a low and a high split, + * where an example with featureValue falls into the bin s.t. + * lowSplit.threshold < featureValue <= highSplit.threshold. + * + * For ordered categorical features, there is a 1-1-1 correspondence between + * bins, splits, and feature values. The bin is determined by category/feature value. + * However, the bins are not necessarily ordered by feature value; + * they are ordered using impurity. + * For unordered categorical features, there is a 1-1 correspondence between bins, splits, + * where bins and splits correspond to subsets of feature values (in highSplit.categories). + * * @param lowSplit signifying the lower threshold for the continuous feature to be * accepted in the bin * @param highSplit signifying the upper threshold for the continuous feature to be * accepted in the bin * @param featureType type of feature -- categorical or continuous - * @param category categorical label value accepted in the bin for binary classification + * @param category categorical label value accepted in the bin for ordered features */ private[tree] case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 3d3406b5d5f22..0594fd0749d21 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -39,7 +39,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @return Double prediction from the trained model */ def predict(features: Vector): Double = { - topNode.predictIfLeaf(features) + topNode.predict(features) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala deleted file mode 100644 index 2deaf4ae8dcab..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.tree.model - -/** - * Filter specifying a split and type of comparison to be applied on features - * @param split split specifying the feature index, type and threshold - * @param comparison integer specifying <,=,> - */ -private[tree] case class Filter(split: Split, comparison: Int) { - // Comparison -1,0,1 signifies <.=,> - override def toString = " split = " + split + "comparison = " + comparison -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 944f11c2c2e4f..0eee6262781c1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -69,24 +69,24 @@ class Node ( /** * predict value if node is not leaf - * @param feature feature value + * @param features feature value * @return predicted value */ - def predictIfLeaf(feature: Vector) : Double = { + def predict(features: Vector) : Double = { if (isLeaf) { predict } else{ if (split.get.featureType == Continuous) { - if (feature(split.get.feature) <= split.get.threshold) { - leftNode.get.predictIfLeaf(feature) + if (features(split.get.feature) <= split.get.threshold) { + leftNode.get.predict(features) } else { - rightNode.get.predictIfLeaf(feature) + rightNode.get.predict(features) } } else { - if (split.get.categories.contains(feature(split.get.feature))) { - leftNode.get.predictIfLeaf(feature) + if (split.get.categories.contains(features(split.get.feature))) { + leftNode.get.predict(features) } else { - rightNode.get.predictIfLeaf(feature) + rightNode.get.predict(features) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index d7ffd386c05ee..50fb48b40de3d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -24,9 +24,10 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType * :: DeveloperApi :: * Split applied to a feature * @param feature feature index - * @param threshold threshold for continuous feature + * @param threshold Threshold for continuous feature. + * Split left if feature <= threshold, else right. * @param featureType type of feature -- categorical or continuous - * @param categories accepted values for categorical variables + * @param categories Split left if categorical feature value is in this set, else right. */ @DeveloperApi case class Split( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index a5c49a38dc08f..2f36fd907772c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -23,10 +23,10 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ -import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy} -import org.apache.spark.mllib.tree.impl.TreePoint +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Filter, Split} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.regression.LabeledPoint @@ -64,7 +64,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) assert(splits(0).length === 99) @@ -82,7 +83,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) assert(splits(0).length === 99) @@ -162,7 +164,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) // Check splits. @@ -279,7 +282,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) // Expecting 2^2 - 1 = 3 bins/splits assert(splits(0)(0).feature === 0) @@ -373,7 +377,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) // 2^10 - 1 > 100, so categorical variables will be ordered @@ -428,10 +433,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -456,10 +462,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 assert(split.categories.length === 1) @@ -495,7 +502,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -503,9 +511,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -518,7 +526,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -526,9 +535,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -542,7 +551,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -550,9 +560,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -566,7 +576,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -574,9 +585,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) assert(bestSplits(0)._2.gain === 0) @@ -590,7 +601,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, 3, 2, 100) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) assert(bins.length === 2) @@ -598,14 +610,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins(0).length === 100) - val leftFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()), -1) - val rightFilter = Filter(new Split(0, 400, FeatureType.Continuous, List()) ,1) - val filters = Array[List[Filter]](List(), List(leftFilter), List(rightFilter)) + // Train a 1-node model + val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) + val modelOneNode = DecisionTree.train(rdd, strategyOneNode) + val nodes: Array[Node] = new Array[Node](7) + nodes(0) = modelOneNode.topNode + nodes(0).leftNode = None + nodes(0).rightNode = None + val parentImpurities = Array(0.5, 0.5, 0.5) // Single group second level tree construction. - val treeInput = TreePoint.convertToTreeRDD(rdd, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, strategy, 1, filters, + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, nodes, splits, bins, 10) assert(bestSplits.length === 2) assert(bestSplits(0)._2.gain > 0) @@ -613,8 +630,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second // level tree construction. - val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, strategy, 1, - filters, splits, bins, 0) + val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, + nodes, splits, bins, 0) assert(bestSplitsWithGroups.length === 2) assert(bestSplitsWithGroups(0)._2.gain > 0) assert(bestSplitsWithGroups(1)._2.gain > 0) @@ -629,19 +646,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) assert(bestSplits(i)._2.predict === bestSplitsWithGroups(i)._2.predict) } - } test("stump with categorical variables for multiclass classification") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(strategy.isMulticlassClassification) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -657,11 +674,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) arr(2) = new LabeledPoint(1.0, Vectors.dense(2.0)) arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 2) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) @@ -688,20 +705,22 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with categorical variables for multiclass classification, with just enough bins") { val maxBins = math.pow(2, 3 - 1).toInt // just enough bins to allow unordered features val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + numClassesForClassification = 3, maxBins = maxBins, + categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -716,18 +735,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with continuous variables for multiclass classification") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3) assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -741,18 +761,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with continuous + categorical variables for multiclass classification") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 @@ -765,14 +786,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { test("stump with categorical variables for ordered multiclass classification") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) - val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) - val treeInput = TreePoint.convertToTreeRDD(input, strategy, bins) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), strategy, 0, - Array[List[Filter]](), splits, bins, 10) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) val bestSplit = bestSplits(0)._1 From 318e28b503f22a89c23b7b3624e5fcf689fb92a2 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 17 Aug 2014 17:06:55 -0700 Subject: [PATCH 0384/1492] SPARK-2881. Upgrade snappy-java to 1.1.1.3. This upgrades snappy-java which fixes the issue reported in SPARK-2881. This is the master branch equivalent to #1994 which provides a different work-around for the 1.1 branch. Author: Patrick Wendell Closes #1995 from pwendell/snappy-1.1 and squashes the following commits: 0c7c4c2 [Patrick Wendell] SPARK-2881. Upgrade snappy-java to 1.1.1.3. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 920912353fe9c..ef12c8f1a5c49 100644 --- a/pom.xml +++ b/pom.xml @@ -316,7 +316,7 @@ org.xerial.snappy snappy-java - 1.0.5 + 1.1.1.3 net.jpountz.lz4 From 5ecb08ea063166564178885b7515abef0d76eecb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 17 Aug 2014 18:10:45 -0700 Subject: [PATCH 0385/1492] Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled" Revert #1891 due to issues with hadoop 1 compatibility. Author: Michael Armbrust Closes #2007 from marmbrus/revert1891 and squashes the following commits: 68706c0 [Michael Armbrust] Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled" --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index c16a7d3661c66..b092f42372171 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -26,8 +26,6 @@ import jline.{ConsoleReader, History} import org.apache.commons.lang.StringUtils import org.apache.commons.logging.LogFactory import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} import org.apache.hadoop.hive.common.LogUtils.LogInitializationException import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils} @@ -118,17 +116,13 @@ private[hive] object SparkSQLCLIDriver { SessionState.start(sessionState) // Clean up after we exit - /** - * This should be executed before shutdown hook of - * FileSystem to avoid race condition of FileSystem operation - */ - ShutdownHookManager.get.addShutdownHook( + Runtime.getRuntime.addShutdownHook( new Thread() { override def run() { SparkSQLEnv.stop() } } - , FileSystem.SHUTDOWN_HOOK_PRIORITY - 1) + ) // "-h" option has been passed, so connect to Hive thrift server. if (sessionState.getHost != null) { From bfa09b01d7eddc572cd22ca2e418a735b4ccc826 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 17 Aug 2014 19:00:38 -0700 Subject: [PATCH 0386/1492] [SQL] Improve debug logging and toStrings. Author: Michael Armbrust Closes #2004 from marmbrus/codgenDebugging and squashes the following commits: b7a7e41 [Michael Armbrust] Improve debug logging and toStrings. --- .../expressions/codegen/CodeGenerator.scala | 21 +++++++++++++++++-- .../catalyst/expressions/nullFunctions.scala | 2 ++ 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 5b398695bf560..de2d67ce82ff1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -78,7 +78,12 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin .build( new CacheLoader[InType, OutType]() { override def load(in: InType): OutType = globalLock.synchronized { - create(in) + val startTime = System.nanoTime() + val result = create(in) + val endTime = System.nanoTime() + def timeMs = (endTime - startTime).toDouble / 1000000 + logInfo(s"Code generated expression $in in $timeMs ms") + result } }) @@ -413,7 +418,19 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin """.children } - EvaluatedExpression(code, nullTerm, primitiveTerm, objectTerm) + // Only inject debugging code if debugging is turned on. + val debugCode = + if (log.isDebugEnabled) { + val localLogger = log + val localLoggerTree = reify { localLogger } + q""" + $localLoggerTree.debug(${e.toString} + ": " + (if($nullTerm) "null" else $primitiveTerm)) + """ :: Nil + } else { + Nil + } + + EvaluatedExpression(code ++ debugCode, nullTerm, primitiveTerm, objectTerm) } protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index ce6d99c911ab3..e88c5d4fa178a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -60,6 +60,8 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr override def eval(input: Row): Any = { child.eval(input) == null } + + override def toString = s"IS NULL $child" } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { From 99243288b049f4a4fb4ba0505ea2310be5eb4bd2 Mon Sep 17 00:00:00 2001 From: Chris Fregly Date: Sun, 17 Aug 2014 19:33:15 -0700 Subject: [PATCH 0387/1492] [SPARK-1981] updated streaming-kinesis.md fixed markup, separated out sections more-clearly, more thorough explanations Author: Chris Fregly Closes #1757 from cfregly/master and squashes the following commits: 9b1c71a [Chris Fregly] better explained why spark checkpoints are disabled in the example (due to no stateful operations being used) 0f37061 [Chris Fregly] SPARK-1981: (Kinesis streaming support) updated streaming-kinesis.md 862df67 [Chris Fregly] Merge remote-tracking branch 'upstream/master' 8e1ae2e [Chris Fregly] Merge remote-tracking branch 'upstream/master' 4774581 [Chris Fregly] updated docs, renamed retry to retryRandom to be more clear, removed retries around store() method 0393795 [Chris Fregly] moved Kinesis examples out of examples/ and back into extras/kinesis-asl 691a6be [Chris Fregly] fixed tests and formatting, fixed a bug with JavaKinesisWordCount during union of streams 0e1c67b [Chris Fregly] Merge remote-tracking branch 'upstream/master' 74e5c7c [Chris Fregly] updated per TD's feedback. simplified examples, updated docs e33cbeb [Chris Fregly] Merge remote-tracking branch 'upstream/master' bf614e9 [Chris Fregly] per matei's feedback: moved the kinesis examples into the examples/ dir d17ca6d [Chris Fregly] per TD's feedback: updated docs, simplified the KinesisUtils api 912640c [Chris Fregly] changed the foundKinesis class to be a publically-avail class db3eefd [Chris Fregly] Merge remote-tracking branch 'upstream/master' 21de67f [Chris Fregly] Merge remote-tracking branch 'upstream/master' 6c39561 [Chris Fregly] parameterized the versions of the aws java sdk and kinesis client 338997e [Chris Fregly] improve build docs for kinesis 828f8ae [Chris Fregly] more cleanup e7c8978 [Chris Fregly] Merge remote-tracking branch 'upstream/master' cd68c0d [Chris Fregly] fixed typos and backward compatibility d18e680 [Chris Fregly] Merge remote-tracking branch 'upstream/master' b3b0ff1 [Chris Fregly] [SPARK-1981] Add AWS Kinesis streaming support --- docs/streaming-kinesis.md | 97 ++++++++++++++++++++------------------- 1 file changed, 49 insertions(+), 48 deletions(-) diff --git a/docs/streaming-kinesis.md b/docs/streaming-kinesis.md index 801c905c88df8..16ad3222105a2 100644 --- a/docs/streaming-kinesis.md +++ b/docs/streaming-kinesis.md @@ -3,56 +3,57 @@ layout: global title: Spark Streaming Kinesis Receiver --- -### Kinesis -Build notes: -
  • Spark supports a Kinesis Streaming Receiver which is not included in the default build due to licensing restrictions.
  • -
  • _**Note that by embedding this library you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • -
  • The Spark Kinesis Streaming Receiver source code, examples, tests, and artifacts live in $SPARK_HOME/extras/kinesis-asl.
  • -
  • To build with Kinesis, you must run the maven or sbt builds with -Pkinesis-asl`.
  • -
  • Applications will need to link to the 'spark-streaming-kinesis-asl` artifact.
  • +## Kinesis +###Design +
  • The KinesisReceiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License.
  • +
  • The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases.
  • +
  • The KCL uses DynamoDB to maintain all state. A DynamoDB table is created in the us-east-1 region (regardless of Kinesis stream region) during KCL initialization for each Kinesis application name.
  • +
  • A single KinesisReceiver can process many shards of a stream by spinning up multiple KinesisRecordProcessor threads.
  • +
  • You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread.
  • +
  • Horizontal scaling is achieved by autoscaling additional KinesisReceiver (separate processes) or spinning up new KinesisRecordProcessor threads within each KinesisReceiver - up to the number of current shards for a given stream, of course. Don't forget to autoscale back down!
  • -Kinesis examples notes: -
  • To build the Kinesis examples, you must run the maven or sbt builds with -Pkinesis-asl`.
  • -
  • These examples automatically determine the number of local threads and KinesisReceivers to spin up based on the number of shards for the stream.
  • -
  • KinesisWordCountProducerASL will generate random data to put onto the Kinesis stream for testing.
  • -
  • Checkpointing is disabled (no checkpoint dir is set). The examples as written will not recover from a driver failure.
  • +### Build +
  • Spark supports a Streaming KinesisReceiver, but it is not included in the default build due to Amazon Software Licensing (ASL) restrictions.
  • +
  • To build with the Kinesis Streaming Receiver and supporting ASL-licensed code, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • +
  • All KinesisReceiver-related code, examples, tests, and artifacts live in **$SPARK_HOME/extras/kinesis-asl/**.
  • +
  • Kinesis-based Spark Applications will need to link to the **spark-streaming-kinesis-asl** artifact that is built when **-Pkinesis-asl** is specified.
  • +
  • _**Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • -Deployment and runtime notes: -
  • A single KinesisReceiver can process many shards of a stream.
  • -
  • Each shard of a stream is processed by one or more KinesisReceiver's managed by the Kinesis Client Library (KCL) Worker.
  • -
  • You never need more KinesisReceivers than the number of shards in your stream.
  • -
  • You can horizontally scale the receiving by creating more KinesisReceiver/DStreams (up to the number of shards for a given stream)
  • -
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the Kinesis Client Library.
  • -
  • This code uses the DefaultAWSCredentialsProviderChain and searches for credentials in the following order of precedence:
    - 1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    - 2) Java System Properties - aws.accessKeyId and aws.secretKey
    - 3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    - 4) Instance profile credentials - delivered through the Amazon EC2 metadata service
    -
  • -
  • You need to setup a Kinesis stream with 1 or more shards per the following:
    - http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • -
  • Valid Kinesis endpoint urls can be found here: Valid endpoint urls: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • -
  • When you first start up the KinesisReceiver, the Kinesis Client Library (KCL) needs ~30s to establish connectivity with the AWS Kinesis service, -retrieve any checkpoint data, and negotiate with other KCL's reading from the same stream.
  • -
  • Be careful when changing the app name. Kinesis maintains a mapping table in DynamoDB based on this app name (http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization). -Changing the app name could lead to Kinesis errors as only 1 logical application can process a stream. In order to start fresh, -it's always best to delete the DynamoDB table that matches your app name. This DynamoDB table lives in us-east-1 regardless of the Kinesis endpoint URL.
  • +###Example +
  • To build the Kinesis example, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • +
  • You need to setup a Kinesis stream at one of the valid Kinesis endpoints with 1 or more shards per the following: http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • +
  • Valid Kinesis endpoints can be found here: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • +
  • When running **locally**, the example automatically determines the number of threads and KinesisReceivers to spin up based on the number of shards configured for the stream. Therefore, **local[n]** is not needed when starting the example as with other streaming examples.
  • +
  • While this example could use a single KinesisReceiver which spins up multiple KinesisRecordProcessor threads to process multiple shards, I wanted to demonstrate unioning multiple KinesisReceivers as a single DStream. (It's a bit confusing in local mode.)
  • +
  • **KinesisWordCountProducerASL** is provided to generate random records into the Kinesis stream for testing.
  • +
  • The example has been configured to immediately replicate incoming stream data to another node by using (StorageLevel.MEMORY_AND_DISK_2) +
  • Spark checkpointing is disabled because the example does not use any stateful or window-based DStream operations such as updateStateByKey and reduceByWindow. If those operations are introduced, you would need to enable checkpointing or risk losing data in the case of a failure.
  • +
  • Kinesis checkpointing is enabled. This means that the example will recover from a Kinesis failure.
  • +
  • The example uses InitialPositionInStream.LATEST strategy to pull from the latest tip of the stream if no Kinesis checkpoint info exists.
  • +
  • In our example, **KinesisWordCount** is the Kinesis application name for both the Scala and Java versions. The use of this application name is described next.
  • -Failure recovery notes: -
  • The combination of Spark Streaming and Kinesis creates 3 different checkpoints as follows:
    - 1) RDD data checkpoint (Spark Streaming) - frequency is configurable with DStream.checkpoint(Duration)
    - 2) RDD metadata checkpoint (Spark Streaming) - frequency is every DStream batch
    - 3) Kinesis checkpointing (Kinesis) - frequency is controlled by the developer calling ICheckpointer.checkpoint() directly
    +###Deployment and Runtime +
  • A Kinesis application name must be unique for a given account and region.
  • +
  • A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization
  • +
  • This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL.
  • +
  • Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream.
  • +
  • If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch.
  • +
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the KCL.
  • +
  • The KinesisReceiver uses the DefaultAWSCredentialsProviderChain for AWS credentials which searches for credentials in the following order of precedence:
    +1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    +2) Java System Properties - aws.accessKeyId and aws.secretKey
    +3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    +4) Instance profile credentials - delivered through the Amazon EC2 metadata service
  • -
  • Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling
  • -
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last checkpoint sequence number recorded per shard.
  • -
  • If no checkpoint info exists, the worker will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) -or from the tip/latest (InitialPostitionInStream.LATEST). This is configurable.
  • -
  • When pulling from the stream tip (InitialPositionInStream.LATEST), only new stream data will be picked up after the KinesisReceiver starts.
  • -
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running.
  • -
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data -depending on the checkpoint frequency.
  • -
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records depending on the checkpoint frequency.
  • + +###Fault-Tolerance +
  • The combination of Spark Streaming and Kinesis creates 2 different checkpoints that may occur at different intervals.
  • +
  • Checkpointing too frequently against Kinesis will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random backoff retry strategy.
  • +
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last Kinesis checkpoint sequence number recorded per shard (stored in the DynamoDB table).
  • +
  • If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable.
  • +
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored.)
  • +
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data.
  • +
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency.
  • Record processing should be idempotent when possible.
  • -
  • Failed or latent KinesisReceivers will be detected and automatically shutdown/load-balanced by the KCL.
  • -
  • If possible, explicitly shutdown the worker if a failure occurs in order to trigger the final checkpoint.
  • +
  • A failed or latent KinesisRecordProcessor within the KinesisReceiver will be detected and automatically restarted by the KCL.
  • +
  • If possible, the KinesisReceiver should be shutdown cleanly in order to trigger a final checkpoint of all KinesisRecordProcessors to avoid duplicate record processing.
  • \ No newline at end of file From 95470a03ae85d7d37d75f73435425a0e22918bc9 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Sun, 17 Aug 2014 19:50:31 -0700 Subject: [PATCH 0388/1492] [HOTFIX][STREAMING] Allow the JVM/Netty to decide which port to bind to in Flume Polling Tests. Author: Hari Shreedharan Closes #1820 from harishreedharan/use-free-ports and squashes the following commits: b939067 [Hari Shreedharan] Remove unused import. 67856a8 [Hari Shreedharan] Remove findFreePort. 0ea51d1 [Hari Shreedharan] Make some changes to getPort to use map on the serverOpt. 1fb0283 [Hari Shreedharan] Merge branch 'master' of https://github.com/apache/spark into use-free-ports b351651 [Hari Shreedharan] Allow Netty to choose port, and query it to decide the port to bind to. Leaving findFreePort as is, if other tests want to use it at some point. e6c9620 [Hari Shreedharan] Making sure the second sink uses the correct port. 11c340d [Hari Shreedharan] Add info about race condition to scaladoc. e89d135 [Hari Shreedharan] Adding Scaladoc. 6013bb0 [Hari Shreedharan] [STREAMING] Find free ports to use before attempting to create Flume Sink in Flume Polling Suite --- .../streaming/flume/sink/SparkSink.scala | 8 +++ .../flume/FlumePollingStreamSuite.scala | 55 +++++++++---------- 2 files changed, 34 insertions(+), 29 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 7b735133e3d14..948af5947f5e1 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -131,6 +131,14 @@ class SparkSink extends AbstractSink with Logging with Configurable { blockingLatch.await() Status.BACKOFF } + + private[flume] def getPort(): Int = { + serverOpt + .map(_.getPort) + .getOrElse( + throw new RuntimeException("Server was not started!") + ) + } } /** diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index a69baa16981a1..8a85b0f987e42 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -22,6 +22,8 @@ import java.net.InetSocketAddress import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} import java.util.Random +import org.apache.spark.TestUtils + import scala.collection.JavaConversions._ import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} @@ -39,9 +41,6 @@ import org.apache.spark.util.Utils class FlumePollingStreamSuite extends TestSuiteBase { - val random = new Random() - /** Return a port in the ephemeral range. */ - def getTestPort = random.nextInt(16382) + 49152 val batchCount = 5 val eventsPerBatch = 100 val totalEventsPerChannel = batchCount * eventsPerBatch @@ -77,17 +76,6 @@ class FlumePollingStreamSuite extends TestSuiteBase { } private def testFlumePolling(): Unit = { - val testPort = getTestPort - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", testPort)), - StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] - with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) - outputStream.register() - // Start the channel and sink. val context = new Context() context.put("capacity", channelCapacity.toString) @@ -98,10 +86,19 @@ class FlumePollingStreamSuite extends TestSuiteBase { val sink = new SparkSink() context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) Configurables.configure(sink, context) sink.setChannel(channel) sink.start() + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", sink.getPort())), + StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() ssc.start() writeAndVerify(Seq(channel), ssc, outputBuffer) @@ -111,18 +108,6 @@ class FlumePollingStreamSuite extends TestSuiteBase { } private def testFlumePollingMultipleHost(): Unit = { - val testPort = getTestPort - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _)) - val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, - eventsPerBatch, 5) - val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] - with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) - outputStream.register() - // Start the channel and sink. val context = new Context() context.put("capacity", channelCapacity.toString) @@ -136,17 +121,29 @@ class FlumePollingStreamSuite extends TestSuiteBase { val sink = new SparkSink() context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort)) + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) Configurables.configure(sink, context) sink.setChannel(channel) sink.start() val sink2 = new SparkSink() context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(testPort + 1)) + context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) Configurables.configure(sink2, context) sink2.setChannel(channel2) sink2.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val addresses = Seq(sink.getPort(), sink2.getPort()).map(new InetSocketAddress("localhost", _)) + val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, + eventsPerBatch, 5) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] + with SynchronizedBuffer[Seq[SparkFlumeEvent]] + val outputStream = new TestOutputStream(flumeStream, outputBuffer) + outputStream.register() + ssc.start() writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) assertChannelIsEmpty(channel) From c77f40668fbb5b8bca9a9b25c039895cb7a4a80c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 17 Aug 2014 20:53:18 -0700 Subject: [PATCH 0389/1492] [SPARK-3087][MLLIB] fix col indexing bug in chi-square and add a check for number of distinct values There is a bug determining the column index. dorx Author: Xiangrui Meng Closes #1997 from mengxr/chisq-index and squashes the following commits: 8fc2ab2 [Xiangrui Meng] fix col indexing bug and add a check for number of distinct values --- .../apache/spark/mllib/stat/Statistics.scala | 2 +- .../spark/mllib/stat/test/ChiSqTest.scala | 37 +++++++++++++++---- .../mllib/stat/HypothesisTestSuite.scala | 37 ++++++++++++++----- 3 files changed, 59 insertions(+), 17 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala index 3cf1028fbc725..3cf4e807b4cf7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/Statistics.scala @@ -155,7 +155,7 @@ object Statistics { * :: Experimental :: * Conduct Pearson's independence test for every feature against the label across the input RDD. * For each feature, the (feature, label) pairs are converted into a contingency matrix for which - * the chi-squared statistic is computed. + * the chi-squared statistic is computed. All label and feature values must be categorical. * * @param data an `RDD[LabeledPoint]` containing the labeled dataset with categorical features. * Real-valued features will be treated as categorical for each distinct value. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 215de95db5113..0089419c2c5d4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -20,11 +20,13 @@ package org.apache.spark.mllib.stat.test import breeze.linalg.{DenseMatrix => BDM} import cern.jet.stat.Probability.chiSquareComplemented -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD +import scala.collection.mutable + /** * Conduct the chi-squared test for the input RDDs using the specified method. * Goodness-of-fit test is conducted on two `Vectors`, whereas test of independence is conducted @@ -75,21 +77,42 @@ private[stat] object ChiSqTest extends Logging { */ def chiSquaredFeatures(data: RDD[LabeledPoint], methodName: String = PEARSON.name): Array[ChiSqTestResult] = { + val maxCategories = 10000 val numCols = data.first().features.size val results = new Array[ChiSqTestResult](numCols) var labels: Map[Double, Int] = null - // At most 100 columns at a time - val batchSize = 100 + // at most 1000 columns at a time + val batchSize = 1000 var batch = 0 while (batch * batchSize < numCols) { // The following block of code can be cleaned up and made public as // chiSquared(data: RDD[(V1, V2)]) val startCol = batch * batchSize val endCol = startCol + math.min(batchSize, numCols - startCol) - val pairCounts = data.flatMap { p => - // assume dense vectors - p.features.toArray.slice(startCol, endCol).zipWithIndex.map { case (feature, col) => - (col, feature, p.label) + val pairCounts = data.mapPartitions { iter => + val distinctLabels = mutable.HashSet.empty[Double] + val allDistinctFeatures: Map[Int, mutable.HashSet[Double]] = + Map((startCol until endCol).map(col => (col, mutable.HashSet.empty[Double])): _*) + var i = 1 + iter.flatMap { case LabeledPoint(label, features) => + if (i % 1000 == 0) { + if (distinctLabels.size > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct label values.") + } + allDistinctFeatures.foreach { case (col, distinctFeatures) => + if (distinctFeatures.size > maxCategories) { + throw new SparkException(s"Chi-square test expect factors (categorical values) but " + + s"found more than $maxCategories distinct values in column $col.") + } + } + } + i += 1 + distinctLabels += label + features.toArray.view.zipWithIndex.slice(startCol, endCol).map { case (feature, col) => + allDistinctFeatures(col) += feature + (col, feature, label) + } } }.countByValue() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala index 5bd0521298c14..6de3840b3f198 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/HypothesisTestSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark.mllib.stat +import java.util.Random + import org.scalatest.FunSuite +import org.apache.spark.SparkException import org.apache.spark.mllib.linalg.{DenseVector, Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.stat.test.ChiSqTest @@ -107,12 +110,13 @@ class HypothesisTestSuite extends FunSuite with LocalSparkContext { // labels: 1.0 (2 / 6), 0.0 (4 / 6) // feature1: 0.5 (1 / 6), 1.5 (2 / 6), 3.5 (3 / 6) // feature2: 10.0 (1 / 6), 20.0 (1 / 6), 30.0 (2 / 6), 40.0 (2 / 6) - val data = Array(new LabeledPoint(0.0, Vectors.dense(0.5, 10.0)), - new LabeledPoint(0.0, Vectors.dense(1.5, 20.0)), - new LabeledPoint(1.0, Vectors.dense(1.5, 30.0)), - new LabeledPoint(0.0, Vectors.dense(3.5, 30.0)), - new LabeledPoint(0.0, Vectors.dense(3.5, 40.0)), - new LabeledPoint(1.0, Vectors.dense(3.5, 40.0))) + val data = Seq( + LabeledPoint(0.0, Vectors.dense(0.5, 10.0)), + LabeledPoint(0.0, Vectors.dense(1.5, 20.0)), + LabeledPoint(1.0, Vectors.dense(1.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 30.0)), + LabeledPoint(0.0, Vectors.dense(3.5, 40.0)), + LabeledPoint(1.0, Vectors.dense(3.5, 40.0))) for (numParts <- List(2, 4, 6, 8)) { val chi = Statistics.chiSqTest(sc.parallelize(data, numParts)) val feature1 = chi(0) @@ -130,10 +134,25 @@ class HypothesisTestSuite extends FunSuite with LocalSparkContext { } // Test that the right number of results is returned - val numCols = 321 - val sparseData = Array(new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((100, 2.0)))), - new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((200, 1.0))))) + val numCols = 1001 + val sparseData = Array( + new LabeledPoint(0.0, Vectors.sparse(numCols, Seq((100, 2.0)))), + new LabeledPoint(0.1, Vectors.sparse(numCols, Seq((200, 1.0))))) val chi = Statistics.chiSqTest(sc.parallelize(sparseData)) assert(chi.size === numCols) + assert(chi(1000) != null) // SPARK-3087 + + // Detect continous features or labels + val random = new Random(11L) + val continuousLabel = + Seq.fill(100000)(LabeledPoint(random.nextDouble(), Vectors.dense(random.nextInt(2)))) + intercept[SparkException] { + Statistics.chiSqTest(sc.parallelize(continuousLabel, 2)) + } + val continuousFeature = + Seq.fill(100000)(LabeledPoint(random.nextInt(2), Vectors.dense(random.nextDouble()))) + intercept[SparkException] { + Statistics.chiSqTest(sc.parallelize(continuousFeature, 2)) + } } } From 5173f3c40f6b64f224f11364e038953826013895 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 17 Aug 2014 22:29:58 -0700 Subject: [PATCH 0390/1492] SPARK-2884: Create binary builds in parallel with release script. --- dev/create-release/create-release.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 1867cf4ec46ca..28f26d2368254 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -117,12 +117,13 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & +make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & make_binary_release "hadoop2" \ - "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & make_binary_release "hadoop2-without-hive" \ - "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" + "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & +wait # Copy data echo "Copying release tarballs" From df652ea02a3e42d987419308ef14874300347373 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sun, 17 Aug 2014 22:39:06 -0700 Subject: [PATCH 0391/1492] SPARK-2900. aggregate inputBytes per stage Author: Sandy Ryza Closes #1826 from sryza/sandy-spark-2900 and squashes the following commits: 43f9091 [Sandy Ryza] SPARK-2900 --- .../org/apache/spark/ui/jobs/JobProgressListener.scala | 6 ++++++ .../apache/spark/ui/jobs/JobProgressListenerSuite.scala | 9 ++++++++- 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index a3e9566832d06..74cd637d88155 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -200,6 +200,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.shuffleReadBytes += shuffleReadDelta execSummary.shuffleRead += shuffleReadDelta + val inputBytesDelta = + (taskMetrics.inputMetrics.map(_.bytesRead).getOrElse(0L) + - oldMetrics.flatMap(_.inputMetrics).map(_.bytesRead).getOrElse(0L)) + stageData.inputBytes += inputBytesDelta + execSummary.inputBytes += inputBytesDelta + val diskSpillDelta = taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) stageData.diskBytesSpilled += diskSpillDelta diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index f5ba31c309277..147ec0bc52e39 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.Matchers import org.apache.spark._ import org.apache.spark.{LocalSparkContext, SparkConf, Success} -import org.apache.spark.executor.{ShuffleWriteMetrics, ShuffleReadMetrics, TaskMetrics} +import org.apache.spark.executor._ import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -150,6 +150,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics.executorRunTime = base + 4 taskMetrics.diskBytesSpilled = base + 5 taskMetrics.memoryBytesSpilled = base + 6 + val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) + taskMetrics.inputMetrics = Some(inputMetrics) + inputMetrics.bytesRead = base + 7 taskMetrics } @@ -182,6 +185,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(stage1Data.diskBytesSpilled == 205) assert(stage0Data.memoryBytesSpilled == 112) assert(stage1Data.memoryBytesSpilled == 206) + assert(stage0Data.inputBytes == 114) + assert(stage1Data.inputBytes == 207) assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get .totalBlocksFetched == 2) assert(stage0Data.taskData.get(1235L).get.taskMetrics.get.shuffleReadMetrics.get @@ -208,6 +213,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc assert(stage1Data.diskBytesSpilled == 610) assert(stage0Data.memoryBytesSpilled == 412) assert(stage1Data.memoryBytesSpilled == 612) + assert(stage0Data.inputBytes == 414) + assert(stage1Data.inputBytes == 614) assert(stage0Data.taskData.get(1234L).get.taskMetrics.get.shuffleReadMetrics.get .totalBlocksFetched == 302) assert(stage1Data.taskData.get(1237L).get.taskMetrics.get.shuffleReadMetrics.get From 3c8fa505900ac158d57de36f6b0fd6da05f8893b Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Sun, 17 Aug 2014 23:29:44 -0700 Subject: [PATCH 0392/1492] [SPARK-3097][MLlib] Word2Vec performance improvement mengxr Please review the code. Adding weights in reduceByKey soon. Only output model entry for words appeared in the partition before merging and use reduceByKey to combine model. In general, this implementation is 30s or so faster than implementation using big array. Author: Liquan Pei Closes #1932 from Ishiihara/Word2Vec-improve2 and squashes the following commits: d5377a9 [Liquan Pei] use syn0Global and syn1Global to represent model cad2011 [Liquan Pei] bug fix for synModify array out of bound 083aa66 [Liquan Pei] update synGlobal in place and reduce synOut size 9075e1c [Liquan Pei] combine syn0Global and syn1Global to synGlobal aa2ab36 [Liquan Pei] use reduceByKey to combine models --- .../apache/spark/mllib/feature/Word2Vec.scala | 50 +++++++++++++------ 1 file changed, 35 insertions(+), 15 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index ecd49ea2ff533..d2ae62b482aff 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -34,6 +34,7 @@ import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap /** * Entry in vocabulary @@ -287,11 +288,12 @@ class Word2Vec extends Serializable with Logging { var syn0Global = Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) var syn1Global = new Array[Float](vocabSize * vectorSize) - var alpha = startingAlpha for (k <- 1 to numIterations) { val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) + val syn0Modify = new Array[Int](vocabSize) + val syn1Modify = new Array[Int](vocabSize) val model = iter.foldLeft((syn0Global, syn1Global, 0, 0)) { case ((syn0, syn1, lastWordCount, wordCount), sentence) => var lwc = lastWordCount @@ -321,7 +323,8 @@ class Word2Vec extends Serializable with Logging { // Hierarchical softmax var d = 0 while (d < bcVocab.value(word).codeLen) { - val l2 = bcVocab.value(word).point(d) * vectorSize + val inner = bcVocab.value(word).point(d) + val l2 = inner * vectorSize // Propagate hidden -> output var f = blas.sdot(vectorSize, syn0, l1, 1, syn1, l2, 1) if (f > -MAX_EXP && f < MAX_EXP) { @@ -330,10 +333,12 @@ class Word2Vec extends Serializable with Logging { val g = ((1 - bcVocab.value(word).code(d) - f) * alpha).toFloat blas.saxpy(vectorSize, g, syn1, l2, 1, neu1e, 0, 1) blas.saxpy(vectorSize, g, syn0, l1, 1, syn1, l2, 1) + syn1Modify(inner) += 1 } d += 1 } blas.saxpy(vectorSize, 1.0f, neu1e, 0, 1, syn0, l1, 1) + syn0Modify(lastWord) += 1 } } a += 1 @@ -342,21 +347,36 @@ class Word2Vec extends Serializable with Logging { } (syn0, syn1, lwc, wc) } - Iterator(model) + val syn0Local = model._1 + val syn1Local = model._2 + val synOut = new PrimitiveKeyOpenHashMap[Int, Array[Float]](vocabSize * 2) + var index = 0 + while(index < vocabSize) { + if (syn0Modify(index) != 0) { + synOut.update(index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize)) + } + if (syn1Modify(index) != 0) { + synOut.update(index + vocabSize, + syn1Local.slice(index * vectorSize, (index + 1) * vectorSize)) + } + index += 1 + } + Iterator(synOut) } - val (aggSyn0, aggSyn1, _, _) = - partial.treeReduce { case ((syn0_1, syn1_1, lwc_1, wc_1), (syn0_2, syn1_2, lwc_2, wc_2)) => - val n = syn0_1.length - val weight1 = 1.0f * wc_1 / (wc_1 + wc_2) - val weight2 = 1.0f * wc_2 / (wc_1 + wc_2) - blas.sscal(n, weight1, syn0_1, 1) - blas.sscal(n, weight1, syn1_1, 1) - blas.saxpy(n, weight2, syn0_2, 1, syn0_1, 1) - blas.saxpy(n, weight2, syn1_2, 1, syn1_1, 1) - (syn0_1, syn1_1, lwc_1 + lwc_2, wc_1 + wc_2) + val synAgg = partial.flatMap(x => x).reduceByKey { case (v1, v2) => + blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) + v1 + }.collect() + var i = 0 + while (i < synAgg.length) { + val index = synAgg(i)._1 + if (index < vocabSize) { + Array.copy(synAgg(i)._2, 0, syn0Global, index * vectorSize, vectorSize) + } else { + Array.copy(synAgg(i)._2, 0, syn1Global, (index - vocabSize) * vectorSize, vectorSize) } - syn0Global = aggSyn0 - syn1Global = aggSyn1 + i += 1 + } } newSentences.unpersist() From eef779b8d631de971d440051cae21040f4de558f Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Sun, 17 Aug 2014 23:30:47 -0700 Subject: [PATCH 0393/1492] [SPARK-2842][MLlib]Word2Vec documentation mengxr Documentation for Word2Vec Author: Liquan Pei Closes #2003 from Ishiihara/Word2Vec-doc and squashes the following commits: 4ff11d4 [Liquan Pei] minor fix 8d7458f [Liquan Pei] code reformat 6df0dcb [Liquan Pei] add Word2Vec documentation --- docs/mllib-feature-extraction.md | 63 +++++++++++++++++++++++++++++++- 1 file changed, 62 insertions(+), 1 deletion(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 21453cb9cd8c9..4b3cb715c58c7 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -9,4 +9,65 @@ displayTitle: MLlib - Feature Extraction ## Word2Vec -## TFIDF +Word2Vec computes distributed vector representation of words. The main advantage of the distributed +representations is that similar words are close in the vector space, which makes generalization to +novel patterns easier and model estimation more robust. Distributed vector representation is +showed to be useful in many natural language processing applications such as named entity +recognition, disambiguation, parsing, tagging and machine translation. + +### Model + +In our implementation of Word2Vec, we used skip-gram model. The training objective of skip-gram is +to learn word vector representations that are good at predicting its context in the same sentence. +Mathematically, given a sequence of training words `$w_1, w_2, \dots, w_T$`, the objective of the +skip-gram model is to maximize the average log-likelihood +`\[ +\frac{1}{T} \sum_{t = 1}^{T}\sum_{j=-k}^{j=k} \log p(w_{t+j} | w_t) +\]` +where $k$ is the size of the training window. + +In the skip-gram model, every word $w$ is associated with two vectors $u_w$ and $v_w$ which are +vector representations of $w$ as word and context respectively. The probability of correctly +predicting word $w_i$ given word $w_j$ is determined by the softmax model, which is +`\[ +p(w_i | w_j ) = \frac{\exp(u_{w_i}^{\top}v_{w_j})}{\sum_{l=1}^{V} \exp(u_l^{\top}v_{w_j})} +\]` +where $V$ is the vocabulary size. + +The skip-gram model with softmax is expensive because the cost of computing $\log p(w_i | w_j)$ +is proportional to $V$, which can be easily in order of millions. To speed up training of Word2Vec, +we used hierarchical softmax, which reduced the complexity of computing of $\log p(w_i | w_j)$ to +$O(\log(V))$ + +### Example + +The example below demonstrates how to load a text file, parse it as an RDD of `Seq[String]`, +construct a `Word2Vec` instance and then fit a `Word2VecModel` with the input data. Finally, +we display the top 40 synonyms of the specified word. To run the example, first download +the [text8](http://mattmahoney.net/dc/text8.zip) data and extract it to your preferred directory. +Here we assume the extracted file is `text8` and in same directory as you run the spark shell. + +
    +
    +{% highlight scala %} +import org.apache.spark._ +import org.apache.spark.rdd._ +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.Word2Vec + +val input = sc.textFile("text8").map(line => line.split(" ").toSeq) + +val word2vec = new Word2Vec() + +val model = word2vec.fit(input) + +val synonyms = model.findSynonyms("china", 40) + +for((synonym, cosineSimilarity) <- synonyms) { + println(s"$synonym $cosineSimilarity") +} +{% endhighlight %} +
    +
    + +## TFIDF \ No newline at end of file From 9306b8c6c8c412b9d0d5cffb6bd7a87784f0f6bf Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Mon, 18 Aug 2014 01:15:45 -0700 Subject: [PATCH 0394/1492] [MLlib] Remove transform(dataset: RDD[String]) from Word2Vec public API mengxr Remove transform(dataset: RDD[String]) from public API. Author: Liquan Pei Closes #2010 from Ishiihara/Word2Vec-api and squashes the following commits: 17b1031 [Liquan Pei] remove transform(dataset: RDD[String]) from public API --- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 9 --------- 1 file changed, 9 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index d2ae62b482aff..1dcaa2cd2e630 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -434,15 +434,6 @@ class Word2VecModel private[mllib] ( } } - /** - * Transforms an RDD to its vector representation - * @param dataset a an RDD of words - * @return RDD of vector representation - */ - def transform(dataset: RDD[String]): RDD[Vector] = { - dataset.map(word => transform(word)) - } - /** * Find synonyms of a word * @param word a word From c0cbbdeaf4f2033be03d32e3ea0288812b4edbf6 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Mon, 18 Aug 2014 09:34:36 -0700 Subject: [PATCH 0395/1492] SPARK-3093 : masterLock in Worker is no longer need there's no need to use masterLock in Worker now since all communications are within Akka actor Author: CrazyJvm Closes #2008 from CrazyJvm/no-need-master-lock and squashes the following commits: dd39e20 [CrazyJvm] fix format 58e7fa5 [CrazyJvm] there's no need to use masterLock now since all communications are within Akka actor --- .../apache/spark/deploy/worker/Worker.scala | 41 +++++++------------ 1 file changed, 14 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 80fde7e4b2624..81400af22c0bf 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -72,7 +72,6 @@ private[spark] class Worker( val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) val testing: Boolean = sys.props.contains("spark.testing") - val masterLock: Object = new Object() var master: ActorSelection = null var masterAddress: Address = null var activeMasterUrl: String = "" @@ -145,18 +144,16 @@ private[spark] class Worker( } def changeMaster(url: String, uiUrl: String) { - masterLock.synchronized { - activeMasterUrl = url - activeMasterWebUiUrl = uiUrl - master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) - masterAddress = activeMasterUrl match { - case Master.sparkUrlRegex(_host, _port) => - Address("akka.tcp", Master.systemName, _host, _port.toInt) - case x => - throw new SparkException("Invalid spark URL: " + x) - } - connected = true + activeMasterUrl = url + activeMasterWebUiUrl = uiUrl + master = context.actorSelection(Master.toAkkaUrl(activeMasterUrl)) + masterAddress = activeMasterUrl match { + case Master.sparkUrlRegex(_host, _port) => + Address("akka.tcp", Master.systemName, _host, _port.toInt) + case x => + throw new SparkException("Invalid spark URL: " + x) } + connected = true } def tryRegisterAllMasters() { @@ -199,9 +196,7 @@ private[spark] class Worker( } case SendHeartbeat => - masterLock.synchronized { - if (connected) { master ! Heartbeat(workerId) } - } + if (connected) { master ! Heartbeat(workerId) } case WorkDirCleanup => // Spin up a separate thread (in a future) to do the dir cleanup; don't tie up worker actor @@ -244,9 +239,7 @@ private[spark] class Worker( manager.start() coresUsed += cores_ memoryUsed += memory_ - masterLock.synchronized { - master ! ExecutorStateChanged(appId, execId, manager.state, None, None) - } + master ! ExecutorStateChanged(appId, execId, manager.state, None, None) } catch { case e: Exception => { logError("Failed to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) @@ -254,17 +247,13 @@ private[spark] class Worker( executors(appId + "/" + execId).kill() executors -= appId + "/" + execId } - masterLock.synchronized { - master ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, None, None) - } + master ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, None, None) } } } case ExecutorStateChanged(appId, execId, state, message, exitStatus) => - masterLock.synchronized { - master ! ExecutorStateChanged(appId, execId, state, message, exitStatus) - } + master ! ExecutorStateChanged(appId, execId, state, message, exitStatus) val fullId = appId + "/" + execId if (ExecutorState.isFinished(state)) { executors.get(fullId) match { @@ -330,9 +319,7 @@ private[spark] class Worker( case _ => logDebug(s"Driver $driverId changed state to $state") } - masterLock.synchronized { - master ! DriverStateChanged(driverId, state, exception) - } + master ! DriverStateChanged(driverId, state, exception) val driver = drivers.remove(driverId).get finishedDrivers(driverId) = driver memoryUsed -= driver.driverDesc.mem From f45efbb8aaa65bc46d65e77e93076fbc29f4455d Mon Sep 17 00:00:00 2001 From: Chandan Kumar Date: Mon, 18 Aug 2014 09:52:25 -0700 Subject: [PATCH 0396/1492] [SPARK-2862] histogram method fails on some choices of bucketCount Author: Chandan Kumar Closes #1787 from nrchandan/spark-2862 and squashes the following commits: a76bbf6 [Chandan Kumar] [SPARK-2862] Fix for a broken test case and add new test cases 4211eea [Chandan Kumar] [SPARK-2862] Add Scala bug id 13854f1 [Chandan Kumar] [SPARK-2862] Use shorthand range notation to avoid Scala bug --- .../apache/spark/rdd/DoubleRDDFunctions.scala | 15 ++++++++---- .../org/apache/spark/rdd/DoubleRDDSuite.scala | 23 +++++++++++++++++++ 2 files changed, 34 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index f233544d128f5..e0494ee39657c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -95,7 +95,12 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * If the elements in RDD do not vary (max == min) always returns a single bucket. */ def histogram(bucketCount: Int): Pair[Array[Double], Array[Long]] = { - // Compute the minimum and the maxium + // Scala's built-in range has issues. See #SI-8782 + def customRange(min: Double, max: Double, steps: Int): IndexedSeq[Double] = { + val span = max - min + Range.Int(0, steps, 1).map(s => min + (s * span) / steps) :+ max + } + // Compute the minimum and the maximum val (max: Double, min: Double) = self.mapPartitions { items => Iterator(items.foldRight(Double.NegativeInfinity, Double.PositiveInfinity)((e: Double, x: Pair[Double, Double]) => @@ -107,9 +112,11 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { throw new UnsupportedOperationException( "Histogram on either an empty RDD or RDD containing +/-infinity or NaN") } - val increment = (max-min)/bucketCount.toDouble - val range = if (increment != 0) { - Range.Double.inclusive(min, max, increment) + val range = if (min != max) { + // Range.Double.inclusive(min, max, increment) + // The above code doesn't always work. See Scala bug #SI-8782. + // https://issues.scala-lang.org/browse/SI-8782 + customRange(min, max, bucketCount) } else { List(min, min) } diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index a822bd18bfdbd..f89bdb6e07dea 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -245,6 +245,29 @@ class DoubleRDDSuite extends FunSuite with SharedSparkContext { assert(histogramBuckets === expectedHistogramBuckets) } + test("WorksWithoutBucketsForLargerDatasets") { + // Verify the case of slighly larger datasets + val rdd = sc.parallelize(6 to 99) + val (histogramBuckets, histogramResults) = rdd.histogram(8) + val expectedHistogramResults = + Array(12, 12, 11, 12, 12, 11, 12, 12) + val expectedHistogramBuckets = + Array(6.0, 17.625, 29.25, 40.875, 52.5, 64.125, 75.75, 87.375, 99.0) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets === expectedHistogramBuckets) + } + + test("WorksWithoutBucketsWithIrrationalBucketEdges") { + // Verify the case of buckets with irrational edges. See #SPARK-2862. + val rdd = sc.parallelize(6 to 99) + val (histogramBuckets, histogramResults) = rdd.histogram(9) + val expectedHistogramResults = + Array(11, 10, 11, 10, 10, 11, 10, 10, 11) + assert(histogramResults === expectedHistogramResults) + assert(histogramBuckets(0) === 6.0) + assert(histogramBuckets(9) === 99.0) + } + // Test the failure mode with an invalid RDD test("ThrowsExceptionOnInvalidRDDs") { // infinity From 7ae28d1247e4756219016206c51fec1656e3917b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 18 Aug 2014 10:00:46 -0700 Subject: [PATCH 0397/1492] SPARK-3096: Include parquet hive serde by default in build A small change - we should just add this dependency. It doesn't have any recursive deps and it's needed for reading have parquet tables. Author: Patrick Wendell Closes #2009 from pwendell/parquet and squashes the following commits: e411f9f [Patrick Wendell] SPARk-309: Include parquet hive serde by default in build --- sql/hive/pom.xml | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 93d00f7c37c9b..30ff277e67c88 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -36,6 +36,11 @@ + + com.twitter + parquet-hive-bundle + 1.5.0 + org.apache.spark spark-core_${scala.binary.version} From 6a13dca12fac06f3af892ffcc8922cc84f91b786 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 18 Aug 2014 10:05:52 -0700 Subject: [PATCH 0398/1492] [SPARK-3084] [SQL] Collect broadcasted tables in parallel in joins BroadcastHashJoin has a broadcastFuture variable that tries to collect the broadcasted table in a separate thread, but this doesn't help because it's a lazy val that only gets initialized when you attempt to build the RDD. Thus queries that broadcast multiple tables would collect and broadcast them sequentially. I changed this to a val to let it start collecting right when the operator is created. Author: Matei Zaharia Closes #1990 from mateiz/spark-3084 and squashes the following commits: f468766 [Matei Zaharia] [SPARK-3084] Collect broadcasted tables in parallel in joins --- .../src/main/scala/org/apache/spark/sql/execution/joins.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index c86811e838bd8..481bb8c05e71b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -424,7 +424,7 @@ case class BroadcastHashJoin( UnspecifiedDistribution :: UnspecifiedDistribution :: Nil @transient - lazy val broadcastFuture = future { + val broadcastFuture = future { sparkContext.broadcast(buildPlan.executeCollect()) } From 4bf3de71074053af94f077c99e9c65a1962739e1 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 18 Aug 2014 10:45:24 -0700 Subject: [PATCH 0399/1492] [SPARK-3085] [SQL] Use compact data structures in SQL joins This reuses the CompactBuffer from Spark Core to save memory and pointer dereferences. I also tried AppendOnlyMap instead of java.util.HashMap but unfortunately that slows things down because it seems to do more equals() calls and the equals on GenericRow, and especially JoinedRow, is pretty expensive. Author: Matei Zaharia Closes #1993 from mateiz/spark-3085 and squashes the following commits: 188221e [Matei Zaharia] Remove unneeded import 5f903ee [Matei Zaharia] [SPARK-3085] [SQL] Use compact data structures in SQL joins --- .../apache/spark/sql/execution/joins.scala | 67 +++++++++---------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index 481bb8c05e71b..b08f9aacc1fcb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -19,16 +19,15 @@ package org.apache.spark.sql.execution import java.util.{HashMap => JavaHashMap} -import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent._ import scala.concurrent.duration._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.util.collection.CompactBuffer @DeveloperApi sealed abstract class BuildSide @@ -67,7 +66,7 @@ trait HashJoin { def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = { // TODO: Use Spark's HashMap implementation. - val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]() + val hashTable = new java.util.HashMap[Row, CompactBuffer[Row]]() var currentRow: Row = null // Create a mapping of buildKeys -> rows @@ -77,7 +76,7 @@ trait HashJoin { if (!rowKey.anyNull) { val existingMatchList = hashTable.get(rowKey) val matchList = if (existingMatchList == null) { - val newMatchList = new ArrayBuffer[Row]() + val newMatchList = new CompactBuffer[Row]() hashTable.put(rowKey, newMatchList) newMatchList } else { @@ -89,7 +88,7 @@ trait HashJoin { new Iterator[Row] { private[this] var currentStreamedRow: Row = _ - private[this] var currentHashMatches: ArrayBuffer[Row] = _ + private[this] var currentHashMatches: CompactBuffer[Row] = _ private[this] var currentMatchPosition: Int = -1 // Mutable per row objects. @@ -140,7 +139,7 @@ trait HashJoin { /** * :: DeveloperApi :: - * Performs a hash based outer join for two child relations by shuffling the data using + * Performs a hash based outer join for two child relations by shuffling the data using * the join keys. This operator requires loading the associated partition in both side into memory. */ @DeveloperApi @@ -179,26 +178,26 @@ case class HashOuterJoin( @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row] // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala - // iterator for performance purpose. + // iterator for performance purpose. private[this] def leftOuterIterator( key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { val joinedRow = new JoinedRow() val rightNullRow = new GenericRow(right.output.length) - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - leftIter.iterator.flatMap { l => + leftIter.iterator.flatMap { l => joinedRow.withLeft(l) var matched = false - (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => + (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => matched = true joinedRow.copy } else { Nil }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the + // as we don't know whether we need to append it until finish iterating all of the // records in right side. // If we didn't get any proper row, then append a single row with empty right joinedRow.withRight(rightNullRow).copy @@ -210,20 +209,20 @@ case class HashOuterJoin( key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { val joinedRow = new JoinedRow() val leftNullRow = new GenericRow(left.output.length) - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - rightIter.iterator.flatMap { r => + rightIter.iterator.flatMap { r => joinedRow.withRight(r) var matched = false - (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => + (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => matched = true joinedRow.copy } else { Nil }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the + // as we don't know whether we need to append it until finish iterating all of the // records in left side. // If we didn't get any proper row, then append a single row with empty left. joinedRow.withLeft(leftNullRow).copy @@ -236,7 +235,7 @@ case class HashOuterJoin( val joinedRow = new JoinedRow() val leftNullRow = new GenericRow(left.output.length) val rightNullRow = new GenericRow(right.output.length) - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) if (!key.anyNull) { @@ -246,8 +245,8 @@ case class HashOuterJoin( leftIter.iterator.flatMap[Row] { l => joinedRow.withLeft(l) var matched = false - rightIter.zipWithIndex.collect { - // 1. For those matched (satisfy the join condition) records with both sides filled, + rightIter.zipWithIndex.collect { + // 1. For those matched (satisfy the join condition) records with both sides filled, // append them directly case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> { @@ -260,7 +259,7 @@ case class HashOuterJoin( // 2. For those unmatched records in left, append additional records with empty right. // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all + // as we don't know whether we need to append it until finish iterating all // of the records in right side. // If we didn't get any proper row, then append a single row with empty right. joinedRow.withRight(rightNullRow).copy @@ -268,8 +267,8 @@ case class HashOuterJoin( } ++ rightIter.zipWithIndex.collect { // 3. For those unmatched records in right, append additional records with empty left. - // Re-visiting the records in right, and append additional row with empty left, if its not - // in the matched set. + // Re-visiting the records in right, and append additional row with empty left, if its not + // in the matched set. case (r, idx) if (!rightMatchedSet.contains(idx)) => { joinedRow(leftNullRow, r).copy } @@ -284,15 +283,15 @@ case class HashOuterJoin( } private[this] def buildHashTable( - iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, ArrayBuffer[Row]] = { - val hashTable = new JavaHashMap[Row, ArrayBuffer[Row]]() + iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, CompactBuffer[Row]] = { + val hashTable = new JavaHashMap[Row, CompactBuffer[Row]]() while (iter.hasNext) { val currentRow = iter.next() val rowKey = keyGenerator(currentRow) var existingMatchList = hashTable.get(rowKey) if (existingMatchList == null) { - existingMatchList = new ArrayBuffer[Row]() + existingMatchList = new CompactBuffer[Row]() hashTable.put(rowKey, existingMatchList) } @@ -311,20 +310,20 @@ case class HashOuterJoin( val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) import scala.collection.JavaConversions._ - val boundCondition = + val boundCondition = condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) joinType match { case LeftOuter => leftHashTable.keysIterator.flatMap { key => - leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), rightHashTable.getOrElse(key, EMPTY_LIST)) } case RightOuter => rightHashTable.keysIterator.flatMap { key => - rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), rightHashTable.getOrElse(key, EMPTY_LIST)) } case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => - fullOuterIterator(key, - leftHashTable.getOrElse(key, EMPTY_LIST), + fullOuterIterator(key, + leftHashTable.getOrElse(key, EMPTY_LIST), rightHashTable.getOrElse(key, EMPTY_LIST)) } case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") @@ -550,7 +549,7 @@ case class BroadcastNestedLoopJoin( /** All rows that either match both-way, or rows from streamed joined with nulls. */ val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => - val matchedRows = new ArrayBuffer[Row] + val matchedRows = new CompactBuffer[Row] // TODO: Use Spark's BitSet. val includedBroadcastTuples = new scala.collection.mutable.BitSet(broadcastedRelation.value.size) @@ -602,20 +601,20 @@ case class BroadcastNestedLoopJoin( val rightNulls = new GenericMutableRow(right.output.size) /** Rows from broadcasted joined with nulls. */ val broadcastRowsWithNulls: Seq[Row] = { - val arrBuf: collection.mutable.ArrayBuffer[Row] = collection.mutable.ArrayBuffer() + val buf: CompactBuffer[Row] = new CompactBuffer() var i = 0 val rel = broadcastedRelation.value while (i < rel.length) { if (!allIncludedBroadcastTuples.contains(i)) { (joinType, buildSide) match { - case (RightOuter | FullOuter, BuildRight) => arrBuf += new JoinedRow(leftNulls, rel(i)) - case (LeftOuter | FullOuter, BuildLeft) => arrBuf += new JoinedRow(rel(i), rightNulls) + case (RightOuter | FullOuter, BuildRight) => buf += new JoinedRow(leftNulls, rel(i)) + case (LeftOuter | FullOuter, BuildLeft) => buf += new JoinedRow(rel(i), rightNulls) case _ => } } i += 1 } - arrBuf.toSeq + buf.toSeq } // TODO: Breaks lineage. From 6bca8898a1aa4ca7161492229bac1748b3da2ad7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 18 Aug 2014 10:52:20 -0700 Subject: [PATCH 0400/1492] SPARK-3025 [SQL]: Allow JDBC clients to set a fair scheduler pool This definitely needs review as I am not familiar with this part of Spark. I tested this locally and it did seem to work. Author: Patrick Wendell Closes #1937 from pwendell/scheduler and squashes the following commits: b858e33 [Patrick Wendell] SPARK-3025: Allow JDBC clients to set a fair scheduler pool --- docs/sql-programming-guide.md | 5 ++++ .../scala/org/apache/spark/sql/SQLConf.scala | 3 +++ .../server/SparkSQLOperationManager.scala | 27 ++++++++++++++----- 3 files changed, 28 insertions(+), 7 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index cd6543945c385..34accade36ea9 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -605,6 +605,11 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script comes with Hive. +To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, +users can set the `spark.sql.thriftserver.scheduler.pool` variable: + + SET spark.sql.thriftserver.scheduler.pool=accounting; + ### Migration Guide for Shark Users #### Reducer number diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 90de11182e605..56face2992bcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -33,6 +33,9 @@ private[spark] object SQLConf { val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" + // This is only used for the thriftserver + val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" + object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 9338e8121b0fe..699a1103f3248 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -17,24 +17,24 @@ package org.apache.spark.sql.hive.thriftserver.server -import scala.collection.JavaConversions._ -import scala.collection.mutable.ArrayBuffer -import scala.math.{random, round} - import java.sql.Timestamp import java.util.{Map => JMap} +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, Map} +import scala.math.{random, round} + import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession - import org.apache.spark.Logging +import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} +import org.apache.spark.sql.catalyst.plans.logical.SetCommand import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -43,6 +43,9 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage val handleToOperation = ReflectionUtils .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") + // TODO: Currenlty this will grow infinitely, even as sessions expire + val sessionToActivePool = Map[HiveSession, String]() + override def newExecuteStatementOperation( parentSession: HiveSession, statement: String, @@ -165,8 +168,18 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage try { result = hiveContext.sql(statement) logDebug(result.queryExecution.toString()) + result.queryExecution.logical match { + case SetCommand(Some(key), Some(value)) if (key == SQLConf.THRIFTSERVER_POOL) => + sessionToActivePool(parentSession) = value + logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") + case _ => + } + val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) + sessionToActivePool.get(parentSession).foreach { pool => + hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + } iter = { val resultRdd = result.queryExecution.toRdd val useIncrementalCollect = From 9eb74c7d2cbe127dd4c32bf1a8318497b2fb55b6 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 18 Aug 2014 11:00:10 -0700 Subject: [PATCH 0401/1492] [SPARK-3091] [SQL] Add support for caching metadata on Parquet files For larger Parquet files, reading the file footers (which is done in parallel on up to 5 threads) and HDFS block locations (which is serial) can take multiple seconds. We can add an option to cache this data within FilteringParquetInputFormat. Unfortunately ParquetInputFormat only caches footers within each instance of ParquetInputFormat, not across them. Note: this PR leaves this turned off by default for 1.1, but I believe it's safe to turn it on after. The keys in the hash maps are FileStatus objects that include a modification time, so this will work fine if files are modified. The location cache could become invalid if files have moved within HDFS, but that's rare so I just made it invalidate entries every 15 minutes. Author: Matei Zaharia Closes #2005 from mateiz/parquet-cache and squashes the following commits: dae8efe [Matei Zaharia] Bug fix c71e9ed [Matei Zaharia] Handle empty statuses directly 22072b0 [Matei Zaharia] Use Guava caches and add a config option for caching metadata 8fb56ce [Matei Zaharia] Cache file block locations too 453bd21 [Matei Zaharia] Bug fix 4094df6 [Matei Zaharia] First attempt at caching Parquet footers --- .../scala/org/apache/spark/sql/SQLConf.scala | 1 + .../sql/parquet/ParquetTableOperations.scala | 84 ++++++++++++++++--- 2 files changed, 72 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 56face2992bcf..4f2adb006fbc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -32,6 +32,7 @@ private[spark] object SQLConf { val CODEGEN_ENABLED = "spark.sql.codegen" val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" + val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 759a2a586b926..c6dca10f6ad7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -17,22 +17,23 @@ package org.apache.spark.sql.parquet -import scala.collection.JavaConversions._ -import scala.collection.mutable -import scala.util.Try - import java.io.IOException import java.lang.{Long => JLong} import java.text.SimpleDateFormat -import java.util.{Date, List => JList} +import java.util.concurrent.{Callable, TimeUnit} +import java.util.{ArrayList, Collections, Date, List => JList} +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.util.Try + +import com.google.common.cache.CacheBuilder import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path} import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter - import parquet.hadoop._ import parquet.hadoop.api.{InitContext, ReadSupport} import parquet.hadoop.metadata.GlobalMetaData @@ -41,7 +42,7 @@ import parquet.io.ParquetDecodingException import parquet.schema.MessageType import org.apache.spark.rdd.RDD -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} import org.apache.spark.{Logging, SerializableWritable, TaskContext} @@ -96,6 +97,11 @@ case class ParquetTableScan( ParquetFilters.serializeFilterExpressions(columnPruningPred, conf) } + // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata + conf.set( + SQLConf.PARQUET_CACHE_METADATA, + sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "false")) + sc.newAPIHadoopRDD( conf, classOf[FilteringParquetRowInputFormat], @@ -323,10 +329,40 @@ private[parquet] class FilteringParquetRowInputFormat } override def getFooters(jobContext: JobContext): JList[Footer] = { + import FilteringParquetRowInputFormat.footerCache + if (footers eq null) { + val conf = ContextUtil.getConfiguration(jobContext) + val cacheMetadata = conf.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) val statuses = listStatus(jobContext) fileStatuses = statuses.map(file => file.getPath -> file).toMap - footers = getFooters(ContextUtil.getConfiguration(jobContext), statuses) + if (statuses.isEmpty) { + footers = Collections.emptyList[Footer] + } else if (!cacheMetadata) { + // Read the footers from HDFS + footers = getFooters(conf, statuses) + } else { + // Read only the footers that are not in the footerCache + val foundFooters = footerCache.getAllPresent(statuses) + val toFetch = new ArrayList[FileStatus] + for (s <- statuses) { + if (!foundFooters.containsKey(s)) { + toFetch.add(s) + } + } + val newFooters = new mutable.HashMap[FileStatus, Footer] + if (toFetch.size > 0) { + val fetched = getFooters(conf, toFetch) + for ((status, i) <- toFetch.zipWithIndex) { + newFooters(status) = fetched.get(i) + } + footerCache.putAll(newFooters) + } + footers = new ArrayList[Footer](statuses.size) + for (status <- statuses) { + footers.add(newFooters.getOrElse(status, foundFooters.get(status))) + } + } } footers @@ -339,6 +375,10 @@ private[parquet] class FilteringParquetRowInputFormat configuration: Configuration, footers: JList[Footer]): JList[ParquetInputSplit] = { + import FilteringParquetRowInputFormat.blockLocationCache + + val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) + val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue) val minSplitSize: JLong = Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L)) @@ -366,16 +406,23 @@ private[parquet] class FilteringParquetRowInputFormat for (footer <- footers) { val fs = footer.getFile.getFileSystem(configuration) val file = footer.getFile - val fileStatus = fileStatuses.getOrElse(file, fs.getFileStatus(file)) + val status = fileStatuses.getOrElse(file, fs.getFileStatus(file)) val parquetMetaData = footer.getParquetMetadata val blocks = parquetMetaData.getBlocks - val fileBlockLocations = fs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen) + var blockLocations: Array[BlockLocation] = null + if (!cacheMetadata) { + blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) + } else { + blockLocations = blockLocationCache.get(status, new Callable[Array[BlockLocation]] { + def call(): Array[BlockLocation] = fs.getFileBlockLocations(status, 0, status.getLen) + }) + } splits.addAll( generateSplits.invoke( null, blocks, - fileBlockLocations, - fileStatus, + blockLocations, + status, parquetMetaData.getFileMetaData, readContext.getRequestedSchema.toString, readContext.getReadSupportMetadata, @@ -387,6 +434,17 @@ private[parquet] class FilteringParquetRowInputFormat } } +private[parquet] object FilteringParquetRowInputFormat { + private val footerCache = CacheBuilder.newBuilder() + .maximumSize(20000) + .build[FileStatus, Footer]() + + private val blockLocationCache = CacheBuilder.newBuilder() + .maximumSize(20000) + .expireAfterWrite(15, TimeUnit.MINUTES) // Expire locations since HDFS files might move + .build[FileStatus, Array[BlockLocation]]() +} + private[parquet] object FileSystemHelper { def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { val origPath = new Path(pathStr) From 3abd0c1cda09bb575adc99847a619bc84af37fd0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 18 Aug 2014 13:17:10 -0700 Subject: [PATCH 0402/1492] [SPARK-2406][SQL] Initial support for using ParquetTableScan to read HiveMetaStore tables. This PR adds an experimental flag `spark.sql.hive.convertMetastoreParquet` that when true causes the planner to detects tables that use Hive's Parquet SerDe and instead plans them using Spark SQL's native `ParquetTableScan`. Author: Michael Armbrust Author: Yin Huai Closes #1819 from marmbrus/parquetMetastore and squashes the following commits: 1620079 [Michael Armbrust] Revert "remove hive parquet bundle" cc30430 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore 4f3d54f [Michael Armbrust] fix style 41ebc5f [Michael Armbrust] remove hive parquet bundle a43e0da [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore 4c4dc19 [Michael Armbrust] Fix bug with tree splicing. ebb267e [Michael Armbrust] include parquet hive to tests pass (Remove this later). c0d9b72 [Michael Armbrust] Avoid creating a HadoopRDD per partition. Add dirty hacks to retrieve partition values from the InputSplit. 8cdc93c [Michael Armbrust] Merge pull request #8 from yhuai/parquetMetastore a0baec7 [Yin Huai] Partitioning columns can be resolved. 1161338 [Michael Armbrust] Add a test to make sure conversion is actually happening 212d5cd [Michael Armbrust] Initial support for using ParquetTableScan to read HiveMetaStore tables. --- project/SparkBuild.scala | 1 - .../spark/sql/execution/basicOperators.scala | 12 ++ .../spark/sql/parquet/ParquetRelation.scala | 8 +- .../sql/parquet/ParquetTableOperations.scala | 74 ++++++-- .../apache/spark/sql/hive/HiveContext.scala | 9 + .../spark/sql/hive/HiveStrategies.scala | 119 +++++++++++- .../sql/hive/parquet/FakeParquetSerDe.scala | 56 ++++++ .../sql/parquet/ParquetMetastoreSuite.scala | 171 ++++++++++++++++++ 8 files changed, 427 insertions(+), 23 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 63a285b81a60c..49d52aefca17a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -228,7 +228,6 @@ object SQL { object Hive { lazy val settings = Seq( - javaOptions += "-XX:MaxPermSize=1g", // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 0027f3cf1fc79..f9dfa3c92f1eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -303,3 +303,15 @@ case class Intersect(left: SparkPlan, right: SparkPlan) extends BinaryNode { left.execute().map(_.copy()).intersection(right.execute().map(_.copy())) } } + +/** + * :: DeveloperApi :: + * A plan node that does nothing but lie about the output of its child. Used to spice a + * (hopefully structurally equivalent) tree from a different optimization sequence into an already + * resolved tree. + */ +@DeveloperApi +case class OutputFaker(output: Seq[Attribute], child: SparkPlan) extends SparkPlan { + def children = child :: Nil + def execute() = child.execute() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 053b2a154389c..1713ae6fb5d93 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -47,7 +47,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} private[sql] case class ParquetRelation( path: String, @transient conf: Option[Configuration], - @transient sqlContext: SQLContext) + @transient sqlContext: SQLContext, + partitioningAttributes: Seq[Attribute] = Nil) extends LeafNode with MultiInstanceRelation { self: Product => @@ -61,12 +62,13 @@ private[sql] case class ParquetRelation( /** Attributes */ override val output = + partitioningAttributes ++ ParquetTypesConverter.readSchemaFromFile( - new Path(path), + new Path(path.split(",").head), conf, sqlContext.isParquetBinaryAsString) - override def newInstance = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] + override def newInstance() = ParquetRelation(path, conf, sqlContext).asInstanceOf[this.type] // Equals must also take into account the output attributes so that we can distinguish between // different instances of the same relation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index c6dca10f6ad7c..f6cfab736d98a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce._ import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + import parquet.hadoop._ import parquet.hadoop.api.{InitContext, ReadSupport} import parquet.hadoop.metadata.GlobalMetaData @@ -42,6 +43,7 @@ import parquet.io.ParquetDecodingException import parquet.schema.MessageType import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} @@ -60,11 +62,18 @@ case class ParquetTableScan( // The resolution of Parquet attributes is case sensitive, so we resolve the original attributes // by exprId. note: output cannot be transient, see // https://issues.apache.org/jira/browse/SPARK-1367 - val output = attributes.map { a => - relation.output - .find(o => o.exprId == a.exprId) - .getOrElse(sys.error(s"Invalid parquet attribute $a in ${relation.output.mkString(",")}")) - } + val normalOutput = + attributes + .filterNot(a => relation.partitioningAttributes.map(_.exprId).contains(a.exprId)) + .flatMap(a => relation.output.find(o => o.exprId == a.exprId)) + + val partOutput = + attributes.flatMap(a => relation.partitioningAttributes.find(o => o.exprId == a.exprId)) + + def output = partOutput ++ normalOutput + + assert(normalOutput.size + partOutput.size == attributes.size, + s"$normalOutput + $partOutput != $attributes, ${relation.output}") override def execute(): RDD[Row] = { val sc = sqlContext.sparkContext @@ -72,16 +81,19 @@ case class ParquetTableScan( ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) val conf: Configuration = ContextUtil.getConfiguration(job) - val qualifiedPath = { - val path = new Path(relation.path) - path.getFileSystem(conf).makeQualified(path) + + relation.path.split(",").foreach { curPath => + val qualifiedPath = { + val path = new Path(curPath) + path.getFileSystem(conf).makeQualified(path) + } + NewFileInputFormat.addInputPath(job, qualifiedPath) } - NewFileInputFormat.addInputPath(job, qualifiedPath) // Store both requested and original schema in `Configuration` conf.set( RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertToString(output)) + ParquetTypesConverter.convertToString(normalOutput)) conf.set( RowWriteSupport.SPARK_ROW_SCHEMA, ParquetTypesConverter.convertToString(relation.output)) @@ -102,13 +114,41 @@ case class ParquetTableScan( SQLConf.PARQUET_CACHE_METADATA, sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "false")) - sc.newAPIHadoopRDD( - conf, - classOf[FilteringParquetRowInputFormat], - classOf[Void], - classOf[Row]) - .map(_._2) - .filter(_ != null) // Parquet's record filters may produce null values + val baseRDD = + new org.apache.spark.rdd.NewHadoopRDD( + sc, + classOf[FilteringParquetRowInputFormat], + classOf[Void], + classOf[Row], + conf) + + if (partOutput.nonEmpty) { + baseRDD.mapPartitionsWithInputSplit { case (split, iter) => + val partValue = "([^=]+)=([^=]+)".r + val partValues = + split.asInstanceOf[parquet.hadoop.ParquetInputSplit] + .getPath + .toString + .split("/") + .flatMap { + case partValue(key, value) => Some(key -> value) + case _ => None + }.toMap + + val partitionRowValues = + partOutput.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) + + new Iterator[Row] { + private[this] val joinedRow = new JoinedRow(Row(partitionRowValues:_*), null) + + def hasNext = iter.hasNext + + def next() = joinedRow.withRight(iter.next()._2) + } + } + } else { + baseRDD.map(_._2) + }.filter(_ != null) // Parquet's record filters may produce null values } /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index a8da676ffa0e0..ff32c7c90a0d2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -79,6 +79,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // Change the default SQL dialect to HiveQL override private[spark] def dialect: String = getConf(SQLConf.DIALECT, "hiveql") + /** + * When true, enables an experimental feature where metastore tables that use the parquet SerDe + * are automatically converted to use the Spark SQL parquet table scan, instead of the Hive + * SerDe. + */ + private[spark] def convertMetastoreParquet: Boolean = + getConf("spark.sql.hive.convertMetastoreParquet", "false") == "true" + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } @@ -326,6 +334,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { TakeOrdered, ParquetOperations, InMemoryScans, + ParquetConversion, // Must be before HiveTableScans HiveTableScans, DataSinks, Scripts, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 5fcc1bd4b9adf..389ace726d205 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,14 +17,20 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.SQLContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} import org.apache.spark.sql.execution._ import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.columnar.InMemoryRelation +import org.apache.spark.sql.parquet.{ParquetRelation, ParquetTableScan} + +import scala.collection.JavaConversions._ private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. @@ -32,6 +38,115 @@ private[hive] trait HiveStrategies { val hiveContext: HiveContext + /** + * :: Experimental :: + * Finds table scans that would use the Hive SerDe and replaces them with our own native parquet + * table scan operator. + * + * TODO: Much of this logic is duplicated in HiveTableScan. Ideally we would do some refactoring + * but since this is after the code freeze for 1.1 all logic is here to minimize disruption. + * + * Other issues: + * - Much of this logic assumes case insensitive resolution. + */ + @Experimental + object ParquetConversion extends Strategy { + implicit class LogicalPlanHacks(s: SchemaRDD) { + def lowerCase = + new SchemaRDD(s.sqlContext, LowerCaseSchema(s.logicalPlan)) + + def addPartitioningAttributes(attrs: Seq[Attribute]) = + new SchemaRDD( + s.sqlContext, + s.logicalPlan transform { + case p: ParquetRelation => p.copy(partitioningAttributes = attrs) + }) + } + + implicit class PhysicalPlanHacks(originalPlan: SparkPlan) { + def fakeOutput(newOutput: Seq[Attribute]) = + OutputFaker( + originalPlan.output.map(a => + newOutput.find(a.name.toLowerCase == _.name.toLowerCase) + .getOrElse( + sys.error(s"Can't find attribute $a to fake in set ${newOutput.mkString(",")}"))), + originalPlan) + } + + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) + if relation.tableDesc.getSerdeClassName.contains("Parquet") && + hiveContext.convertMetastoreParquet => + + // Filter out all predicates that only deal with partition keys + val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val (pruningPredicates, otherPredicates) = predicates.partition { + _.references.map(_.exprId).subsetOf(partitionKeyIds) + } + + // We are going to throw the predicates and projection back at the whole optimization + // sequence so lets unresolve all the attributes, allowing them to be rebound to the + // matching parquet attributes. + val unresolvedOtherPredicates = otherPredicates.map(_ transform { + case a: AttributeReference => UnresolvedAttribute(a.name) + }).reduceOption(And).getOrElse(Literal(true)) + + val unresolvedProjection = projectList.map(_ transform { + case a: AttributeReference => UnresolvedAttribute(a.name) + }) + + if (relation.hiveQlTable.isPartitioned) { + val rawPredicate = pruningPredicates.reduceOption(And).getOrElse(Literal(true)) + // Translate the predicate so that it automatically casts the input values to the correct + // data types during evaluation + val castedPredicate = rawPredicate transform { + case a: AttributeReference => + val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId) + val key = relation.partitionKeys(idx) + Cast(BoundReference(idx, StringType, nullable = true), key.dataType) + } + + val inputData = new GenericMutableRow(relation.partitionKeys.size) + val pruningCondition = + if(codegenEnabled) { + GeneratePredicate(castedPredicate) + } else { + InterpretedPredicate(castedPredicate) + } + + val partitions = relation.hiveQlPartitions.filter { part => + val partitionValues = part.getValues + var i = 0 + while (i < partitionValues.size()) { + inputData(i) = partitionValues(i) + i += 1 + } + pruningCondition(inputData) + } + + hiveContext + .parquetFile(partitions.map(_.getLocation).mkString(",")) + .addPartitioningAttributes(relation.partitionKeys) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection:_*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)):: Nil + } else { + hiveContext + .parquetFile(relation.hiveQlTable.getDataLocation.getPath) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection:_*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)) :: Nil + } + case _ => Nil + } + } + object Scripts extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ScriptTransformation(input, script, output, child) => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala new file mode 100644 index 0000000000000..544abfc32423c --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.parquet + +import java.util.Properties + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category +import org.apache.hadoop.hive.serde2.{SerDeStats, SerDe} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.io.Writable + +/** + * A placeholder that allows SparkSQL users to create metastore tables that are stored as + * parquet files. It is only intended to pass the checks that the serde is valid and exists + * when a CREATE TABLE is run. The actual work of decoding will be done by ParquetTableScan + * when "spark.sql.hive.convertMetastoreParquet" is set to true. + */ +@deprecated("No code should depend on FakeParquetHiveSerDe as it is only intended as a " + + "placeholder in the Hive MetaStore") +class FakeParquetSerDe extends SerDe { + override def getObjectInspector: ObjectInspector = new ObjectInspector { + override def getCategory: Category = Category.PRIMITIVE + + override def getTypeName: String = "string" + } + + override def deserialize(p1: Writable): AnyRef = throwError + + override def initialize(p1: Configuration, p2: Properties): Unit = {} + + override def getSerializedClass: Class[_ <: Writable] = throwError + + override def getSerDeStats: SerDeStats = throwError + + override def serialize(p1: scala.Any, p2: ObjectInspector): Writable = throwError + + private def throwError = + sys.error( + "spark.sql.hive.convertMetastoreParquet must be set to true to use FakeParquetSerDe") +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala new file mode 100644 index 0000000000000..0723be7298e15 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala @@ -0,0 +1,171 @@ + +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parquet + +import java.io.File + +import org.apache.spark.sql.hive.execution.HiveTableScan +import org.scalatest.BeforeAndAfterAll + +import scala.reflect.ClassTag + +import org.apache.spark.sql.{SQLConf, QueryTest} +import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ + +case class ParquetData(intField: Int, stringField: String) + +/** + * Tests for our SerDe -> Native parquet scan conversion. + */ +class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { + + override def beforeAll(): Unit = { + setConf("spark.sql.hive.convertMetastoreParquet", "true") + } + + override def afterAll(): Unit = { + setConf("spark.sql.hive.convertMetastoreParquet", "false") + } + + val partitionedTableDir = File.createTempFile("parquettests", "sparksql") + partitionedTableDir.delete() + partitionedTableDir.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDir, s"p=$p") + sparkContext.makeRDD(1 to 10) + .map(i => ParquetData(i, s"part-$p")) + .saveAsParquetFile(partDir.getCanonicalPath) + } + + sql(s""" + create external table partitioned_parquet + ( + intField INT, + stringField STRING + ) + PARTITIONED BY (p int) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${partitionedTableDir.getCanonicalPath}' + """) + + sql(s""" + create external table normal_parquet + ( + intField INT, + stringField STRING + ) + ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + location '${new File(partitionedTableDir, "p=1").getCanonicalPath}' + """) + + (1 to 10).foreach { p => + sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") + } + + test("project the partitioning column") { + checkAnswer( + sql("SELECT p, count(*) FROM partitioned_parquet group by p"), + (1, 10) :: + (2, 10) :: + (3, 10) :: + (4, 10) :: + (5, 10) :: + (6, 10) :: + (7, 10) :: + (8, 10) :: + (9, 10) :: + (10, 10) :: Nil + ) + } + + test("project partitioning and non-partitioning columns") { + checkAnswer( + sql("SELECT stringField, p, count(intField) " + + "FROM partitioned_parquet GROUP BY p, stringField"), + ("part-1", 1, 10) :: + ("part-2", 2, 10) :: + ("part-3", 3, 10) :: + ("part-4", 4, 10) :: + ("part-5", 5, 10) :: + ("part-6", 6, 10) :: + ("part-7", 7, 10) :: + ("part-8", 8, 10) :: + ("part-9", 9, 10) :: + ("part-10", 10, 10) :: Nil + ) + } + + test("simple count") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet"), + 100) + } + + test("pruned count") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p = 1"), + 10) + } + + test("multi-partition pruned count") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet WHERE p IN (1,2,3)"), + 30) + } + + test("non-partition predicates") { + checkAnswer( + sql("SELECT COUNT(*) FROM partitioned_parquet WHERE intField IN (1,2,3)"), + 30) + } + + test("sum") { + checkAnswer( + sql("SELECT SUM(intField) FROM partitioned_parquet WHERE intField IN (1,2,3) AND p = 1"), + 1 + 2 + 3 + ) + } + + test("non-part select(*)") { + checkAnswer( + sql("SELECT COUNT(*) FROM normal_parquet"), + 10 + ) + } + + test("conversion is working") { + assert( + sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { + case _: HiveTableScan => true + }.isEmpty) + assert( + sql("SELECT * FROM normal_parquet").queryExecution.executedPlan.collect { + case _: ParquetTableScan => true + }.nonEmpty) + } +} From 66ade00f91a9343ac9277c5a7c09314087a4831e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Aug 2014 13:25:30 -0700 Subject: [PATCH 0403/1492] [SPARK-2169] Don't copy appName / basePath everywhere. Instead of keeping copies in all pages, just reference the values kept in the base SparkUI instance (by making them available via getters). Author: Marcelo Vanzin Closes #1252 from vanzin/SPARK-2169 and squashes the following commits: 4412fc6 [Marcelo Vanzin] Simplify UIUtils.headerSparkPage signature. 4e5d35a [Marcelo Vanzin] [SPARK-2169] Don't copy appName / basePath everywhere. --- .../apache/spark/deploy/master/Master.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 9 +++++++++ .../scala/org/apache/spark/ui/UIUtils.scala | 12 +++++------- .../scala/org/apache/spark/ui/WebUI.scala | 3 +++ .../apache/spark/ui/env/EnvironmentPage.scala | 4 +--- .../apache/spark/ui/env/EnvironmentTab.scala | 4 +--- .../apache/spark/ui/exec/ExecutorsPage.scala | 5 +---- .../apache/spark/ui/exec/ExecutorsTab.scala | 6 ++---- .../spark/ui/jobs/JobProgressPage.scala | 4 +--- .../apache/spark/ui/jobs/JobProgressTab.scala | 7 +++---- .../org/apache/spark/ui/jobs/PoolPage.scala | 5 +---- .../org/apache/spark/ui/jobs/PoolTable.scala | 7 +++---- .../org/apache/spark/ui/jobs/StagePage.scala | 8 ++------ .../org/apache/spark/ui/jobs/StageTable.scala | 19 ++++++++++--------- .../org/apache/spark/ui/storage/RDDPage.scala | 8 ++------ .../apache/spark/ui/storage/StoragePage.scala | 6 ++---- .../apache/spark/ui/storage/StorageTab.scala | 4 +--- .../spark/streaming/ui/StreamingPage.scala | 3 +-- .../spark/streaming/ui/StreamingTab.scala | 6 ++---- 19 files changed, 51 insertions(+), 71 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index cfa2c028a807b..5017273e87c07 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -697,7 +697,7 @@ private[spark] class Master( appIdToUI(app.id) = ui webUi.attachSparkUI(ui) // Application UI is successfully rebuilt, so link the Master UI to it - app.desc.appUiUrl = ui.basePath + app.desc.appUiUrl = ui.getBasePath true } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 6c788a37dc70b..cccd59d122a92 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -76,6 +76,8 @@ private[spark] class SparkUI( } } + def getAppName = appName + /** Set the app name for this UI. */ def setAppName(name: String) { appName = name @@ -100,6 +102,13 @@ private[spark] class SparkUI( private[spark] def appUIAddress = s"http://$appUIHostPort" } +private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) + extends WebUITab(parent, prefix) { + + def appName: String = parent.getAppName + +} + private[spark] object SparkUI { val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 715cc2f4df8dd..bee6dad3387e5 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -163,17 +163,15 @@ private[spark] object UIUtils extends Logging { /** Returns a spark page with correctly formatted headers */ def headerSparkPage( - content: => Seq[Node], - basePath: String, - appName: String, title: String, - tabs: Seq[WebUITab], - activeTab: WebUITab, + content: => Seq[Node], + activeTab: SparkUITab, refreshInterval: Option[Int] = None): Seq[Node] = { - val header = tabs.map { tab => + val appName = activeTab.appName + val header = activeTab.headerTabs.map { tab =>
  • - {tab.name} + {tab.name}
  • } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 5f52f95088007..5d88ca403a674 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -50,6 +50,7 @@ private[spark] abstract class WebUI( protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) private val className = Utils.getFormattedClassName(this) + def getBasePath: String = basePath def getTabs: Seq[WebUITab] = tabs.toSeq def getHandlers: Seq[ServletContextHandler] = handlers.toSeq def getSecurityManager: SecurityManager = securityManager @@ -135,6 +136,8 @@ private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { /** Get a list of header tabs from the parent UI. */ def headerTabs: Seq[WebUITab] = parent.getTabs + + def basePath: String = parent.getBasePath } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index b347eb1b83c1f..f0a1174a71d34 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -24,8 +24,6 @@ import scala.xml.Node import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { - private val appName = parent.appName - private val basePath = parent.basePath private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -45,7 +43,7 @@ private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("")

    Classpath Entries

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

    Failed Stages ({failedStages.size})

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

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

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

    Summary Metrics

    No tasks have started yet

    Tasks

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

    Tasks

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

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

    ++ generateReceiverStats() ++ generateBatchStatsTable() - UIUtils.headerSparkPage( - content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) + UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } /** Generate basic stats of the streaming program */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 51448d15c6516..34ac254f337eb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -19,15 +19,13 @@ package org.apache.spark.streaming.ui import org.apache.spark.Logging import org.apache.spark.streaming.StreamingContext -import org.apache.spark.ui.WebUITab +import org.apache.spark.ui.SparkUITab /** Spark Web UI tab that shows statistics of a streaming job */ private[spark] class StreamingTab(ssc: StreamingContext) - extends WebUITab(ssc.sc.ui, "streaming") with Logging { + extends SparkUITab(ssc.sc.ui, "streaming") with Logging { val parent = ssc.sc.ui - val appName = parent.appName - val basePath = parent.basePath val listener = new StreamingJobProgressListener(ssc) ssc.addStreamingListener(listener) From 3a5962f0f5acea5cbfd3cf1e3ed16e03b3bec37a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Aug 2014 13:38:03 -0700 Subject: [PATCH 0404/1492] Removed .travis.yml file since we are not using Travis. --- .travis.yml | 32 -------------------------------- 1 file changed, 32 deletions(-) delete mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 8ebd0d68429fc..0000000000000 --- a/.travis.yml +++ /dev/null @@ -1,32 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - - language: scala - scala: - - "2.10.3" - jdk: - - oraclejdk7 - env: - matrix: - - TEST="scalastyle assembly/assembly" - - TEST="catalyst/test sql/test streaming/test mllib/test graphx/test bagel/test" - - TEST=hive/test - cache: - directories: - - $HOME/.m2 - - $HOME/.ivy2 - - $HOME/.sbt - script: - - "sbt ++$TRAVIS_SCALA_VERSION $TEST" From d1d0ee41c27f1d07fed0c5d56ba26c723cc3dc26 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 18 Aug 2014 13:58:35 -0700 Subject: [PATCH 0405/1492] [SPARK-3103] [PySpark] fix saveAsTextFile() with utf-8 bugfix: It will raise an exception when it try to encode non-ASCII strings into unicode. It should only encode unicode as "utf-8". Author: Davies Liu Closes #2018 from davies/fix_utf8 and squashes the following commits: 4db7967 [Davies Liu] fix saveAsTextFile() with utf-8 --- python/pyspark/rdd.py | 4 +++- python/pyspark/tests.py | 9 +++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 240381e5bae12..c708b69cc1e31 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1191,7 +1191,9 @@ def func(split, iterator): for x in iterator: if not isinstance(x, basestring): x = unicode(x) - yield x.encode("utf-8") + if isinstance(x, unicode): + x = x.encode("utf-8") + yield x keyed = self.mapPartitionsWithIndex(func) keyed._bypass_serializer = True keyed._jrdd.map(self.ctx._jvm.BytesToString()).saveAsTextFile(path) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f1fece998cd54..69d543d9d045d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -256,6 +256,15 @@ def test_save_as_textfile_with_unicode(self): raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + def test_save_as_textfile_with_utf8(self): + x = u"\u00A1Hola, mundo!" + data = self.sc.parallelize([x.encode("utf-8")]) + tempFile = tempfile.NamedTemporaryFile(delete=True) + tempFile.close() + data.saveAsTextFile(tempFile.name) + raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) + self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + def test_transforming_cartesian_result(self): # Regression test for SPARK-1034 rdd1 = self.sc.parallelize([1, 2]) From 6201b27643023569e19b68aa9d5c4e4e59ce0d79 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Aug 2014 14:10:10 -0700 Subject: [PATCH 0406/1492] [SPARK-2718] [yarn] Handle quotes and other characters in user args. Due to the way Yarn runs things through bash, normal quoting doesn't work as expected. This change applies the necessary voodoo to the user args to avoid issues with bash and special characters. The change also uncovered an issue with the event logger app name sanitizing code; it wasn't cleaning up all "bad" characters, so sometimes it would fail to create the log dirs. I just added some more bad character replacements. Author: Marcelo Vanzin Closes #1724 from vanzin/SPARK-2718 and squashes the following commits: cc84b89 [Marcelo Vanzin] Review feedback. c1a257a [Marcelo Vanzin] Add test for backslashes. 55571d4 [Marcelo Vanzin] Unbreak yarn-client. 515613d [Marcelo Vanzin] [SPARK-2718] [yarn] Handle quotes and other characters in user args. --- .../scheduler/EventLoggingListener.scala | 3 +- .../yarn/ApplicationMasterArguments.scala | 6 +- .../apache/spark/deploy/yarn/ClientBase.scala | 9 +-- .../deploy/yarn/ExecutorRunnableUtil.scala | 4 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 25 ++++++++ .../yarn/YarnSparkHadoopUtilSuite.scala | 64 +++++++++++++++++++ 6 files changed, 101 insertions(+), 10 deletions(-) create mode 100644 yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 7378ce923f0ae..370fcd85aa680 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -54,7 +54,8 @@ private[spark] class EventLoggingListener( private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") - private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis + private val name = appName.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_") + .toLowerCase + "-" + System.currentTimeMillis val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 4c383ab574abe..424b0fb0936f2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -29,7 +29,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS parseArgs(args.toList) - + private def parseArgs(inputArgs: List[String]): Unit = { val userArgsBuffer = new ArrayBuffer[String]() @@ -47,7 +47,7 @@ class ApplicationMasterArguments(val args: Array[String]) { userClass = value args = tail - case ("--args") :: value :: tail => + case ("--args" | "--arg") :: value :: tail => userArgsBuffer += value args = tail @@ -75,7 +75,7 @@ class ApplicationMasterArguments(val args: Array[String]) { userArgs = userArgsBuffer.readOnly } - + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { if (unknownParam != null) { System.err.println("Unknown/unsupported param " + unknownParam) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 1da0a1b675554..3897b3a373a8c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -300,11 +300,11 @@ trait ClientBase extends Logging { } def userArgsToString(clientArgs: ClientArguments): String = { - val prefix = " --args " + val prefix = " --arg " val args = clientArgs.userArgs val retval = new StringBuilder() for (arg <- args) { - retval.append(prefix).append(" '").append(arg).append("' ") + retval.append(prefix).append(" ").append(YarnSparkHadoopUtil.escapeForShell(arg)) } retval.toString } @@ -386,7 +386,7 @@ trait ClientBase extends Logging { // TODO: it might be nicer to pass these as an internal environment variable rather than // as Java options, due to complications with string parsing of nested quotes. for ((k, v) <- sparkConf.getAll) { - javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" + javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } if (args.amClass == classOf[ApplicationMaster].getName) { @@ -400,7 +400,8 @@ trait ClientBase extends Logging { // Command for the ApplicationMaster val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ javaOpts ++ - Seq(args.amClass, "--class", args.userClass, "--jar ", args.userJar, + Seq(args.amClass, "--class", YarnSparkHadoopUtil.escapeForShell(args.userClass), + "--jar ", YarnSparkHadoopUtil.escapeForShell(args.userJar), userArgsToString(args), "--executor-memory", args.executorMemory.toString, "--executor-cores", args.executorCores.toString, diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 71a9e42846b2b..312d82a649792 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -68,10 +68,10 @@ trait ExecutorRunnableUtil extends Logging { // authentication settings. sparkConf.getAll. filter { case (k, v) => k.startsWith("spark.auth") || k.startsWith("spark.akka") }. - foreach { case (k, v) => javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" } + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } sparkConf.getAkkaConf. - foreach { case (k, v) => javaOpts += "-D" + k + "=" + "\\\"" + v + "\\\"" } + foreach { case (k, v) => javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } // Commenting it out for now - so that people can refer to the properties if required. Remove // it once cpuset version is pushed out. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index e98308cdbd74e..10aef5eb2486f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -148,4 +148,29 @@ object YarnSparkHadoopUtil { } } + /** + * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands + * using `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work. The + * argument is enclosed in single quotes and some key characters are escaped. + * + * @param arg A single argument. + * @return Argument quoted for execution via Yarn's generated shell script. + */ + def escapeForShell(arg: String): String = { + if (arg != null) { + val escaped = new StringBuilder("'") + for (i <- 0 to arg.length() - 1) { + arg.charAt(i) match { + case '$' => escaped.append("\\$") + case '"' => escaped.append("\\\"") + case '\'' => escaped.append("'\\''") + case c => escaped.append(c) + } + } + escaped.append("'").toString() + } else { + arg + } + } + } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala new file mode 100644 index 0000000000000..7650bd4396c12 --- /dev/null +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import java.io.{File, IOException} + +import com.google.common.io.{ByteStreams, Files} +import org.scalatest.{FunSuite, Matchers} + +import org.apache.spark.Logging + +class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { + + val hasBash = + try { + val exitCode = Runtime.getRuntime().exec(Array("bash", "--version")).waitFor() + exitCode == 0 + } catch { + case e: IOException => + false + } + + if (!hasBash) { + logWarning("Cannot execute bash, skipping bash tests.") + } + + def bashTest(name: String)(fn: => Unit) = + if (hasBash) test(name)(fn) else ignore(name)(fn) + + bashTest("shell script escaping") { + val scriptFile = File.createTempFile("script.", ".sh") + val args = Array("arg1", "${arg.2}", "\"arg3\"", "'arg4'", "$arg5", "\\arg6") + try { + val argLine = args.map(a => YarnSparkHadoopUtil.escapeForShell(a)).mkString(" ") + Files.write(("bash -c \"echo " + argLine + "\"").getBytes(), scriptFile) + scriptFile.setExecutable(true) + + val proc = Runtime.getRuntime().exec(Array(scriptFile.getAbsolutePath())) + val out = new String(ByteStreams.toByteArray(proc.getInputStream())).trim() + val err = new String(ByteStreams.toByteArray(proc.getErrorStream())) + val exitCode = proc.waitFor() + exitCode should be (0) + out should be (args.mkString(" ")) + } finally { + scriptFile.delete() + } + } + +} From 115eeb30dd9c9dd10685a71f2c23ca23794d3142 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 18 Aug 2014 14:40:05 -0700 Subject: [PATCH 0407/1492] [mllib] DecisionTree: treeAggregate + Python example bug fix Small DecisionTree updates: * Changed main DecisionTree aggregate to treeAggregate. * Fixed bug in python example decision_tree_runner.py with missing argument (since categoricalFeaturesInfo is no longer an optional argument for trainClassifier). * Fixed same bug in python doc tests, and added tree.py to doc tests. CC: mengxr Author: Joseph K. Bradley Closes #2015 from jkbradley/dt-opt2 and squashes the following commits: b5114fa [Joseph K. Bradley] Fixed python tree.py doc test (extra newline) 8e4665d [Joseph K. Bradley] Added tree.py to python doc tests. Fixed bug from missing categoricalFeaturesInfo argument. b7b2922 [Joseph K. Bradley] Fixed bug in python example decision_tree_runner.py with missing argument. Changed main DecisionTree aggregate to treeAggregate. 85bbc1f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 66d076f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 a0ed0da [Joseph K. Bradley] Renamed DTMetadata to DecisionTreeMetadata. Small doc updates. 3726d20 [Joseph K. Bradley] Small code improvements based on code review. ac0b9f8 [Joseph K. Bradley] Small updates based on code review. Main change: Now using << instead of math.pow. db0d773 [Joseph K. Bradley] scala style fix 6a38f48 [Joseph K. Bradley] Added DTMetadata class for cleaner code 931a3a7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt2 797f68a [Joseph K. Bradley] Fixed DecisionTreeSuite bug for training second level. Needed to update treePointToNodeIndex with groupShift. f40381c [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 5f2dec2 [Joseph K. Bradley] Fixed scalastyle issue in TreePoint 6b5651e [Joseph K. Bradley] Updates based on code review. 1 major change: persisting to memory + disk, not just memory. 2d2aaaf [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 26d10dd [Joseph K. Bradley] Removed tree/model/Filter.scala since no longer used. Removed debugging println calls in DecisionTree.scala. 356daba [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 c1565a5 [Joseph K. Bradley] Small DecisionTree updates: * Simplification: Updated calculateGainForSplit to take aggregates for a single (feature, split) pair. * Internal doc: findAggForOrderedFeatureClassification b914f3b [Joseph K. Bradley] DecisionTree optimization: eliminated filters + small changes b2ed1f3 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals --- .../src/main/python/mllib/decision_tree_runner.py | 4 +++- .../org/apache/spark/mllib/tree/DecisionTree.scala | 3 ++- python/pyspark/mllib/tree.py | 14 ++++++++------ python/run-tests | 1 + 4 files changed, 14 insertions(+), 8 deletions(-) diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index 8efadb5223f56..db96a7cb3730f 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -124,7 +124,9 @@ def usage(): (reindexedData, origToNewLabels) = reindexClassLabels(points) # Train a classifier. - model = DecisionTree.trainClassifier(reindexedData, numClasses=2) + categoricalFeaturesInfo={} # no categorical features + model = DecisionTree.trainClassifier(reindexedData, numClasses=2, + categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. print "Trained DecisionTree for classification:" print " Model numNodes: %d\n" % model.numNodes() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 6b9a8f72c244e..5cdd258f6c20b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -22,6 +22,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging +import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ @@ -826,7 +827,7 @@ object DecisionTree extends Serializable with Logging { // Calculate bin aggregates. timer.start("aggregation") val binAggregates = { - input.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) + input.treeAggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) } timer.stop("aggregation") logDebug("binAggregates.length = " + binAggregates.length) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index e1a4671709b7d..e9d778df5a24b 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -88,7 +88,8 @@ class DecisionTree(object): It will probably be modified for Spark v1.2. Example usage: - >>> from numpy import array, ndarray + >>> from numpy import array + >>> import sys >>> from pyspark.mllib.regression import LabeledPoint >>> from pyspark.mllib.tree import DecisionTree >>> from pyspark.mllib.linalg import SparseVector @@ -99,15 +100,15 @@ class DecisionTree(object): ... LabeledPoint(1.0, [2.0]), ... LabeledPoint(1.0, [3.0]) ... ] - >>> - >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2) - >>> print(model) + >>> categoricalFeaturesInfo = {} # no categorical features + >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2, + ... categoricalFeaturesInfo=categoricalFeaturesInfo) + >>> sys.stdout.write(model) DecisionTreeModel classifier If (feature 0 <= 0.5) Predict: 0.0 Else (feature 0 > 0.5) Predict: 1.0 - >>> model.predict(array([1.0])) > 0 True >>> model.predict(array([0.0])) == 0 @@ -119,7 +120,8 @@ class DecisionTree(object): ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) ... ] >>> - >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data)) + >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), + ... categoricalFeaturesInfo=categoricalFeaturesInfo) >>> model.predict(array([0.0, 1.0])) == 1 True >>> model.predict(array([0.0, 0.0])) == 0 diff --git a/python/run-tests b/python/run-tests index 1218edcbd7e08..a6271e0cf5fa9 100755 --- a/python/run-tests +++ b/python/run-tests @@ -79,6 +79,7 @@ run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" run_test "pyspark/mllib/tests.py" +run_test "pyspark/mllib/tree.py" run_test "pyspark/mllib/util.py" if [[ $FAILED == 0 ]]; then From c8b16ca0d86cc60fb960eebf0cb383f159a88b03 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 18 Aug 2014 18:01:39 -0700 Subject: [PATCH 0408/1492] [SPARK-2850] [SPARK-2626] [mllib] MLlib stats examples + small fixes Added examples for statistical summarization: * Scala: StatisticalSummary.scala ** Tests: correlation, MultivariateOnlineSummarizer * python: statistical_summary.py ** Tests: correlation (since MultivariateOnlineSummarizer has no Python API) Added examples for random and sampled RDDs: * Scala: RandomAndSampledRDDs.scala * python: random_and_sampled_rdds.py * Both test: ** RandomRDDGenerators.normalRDD, normalVectorRDD ** RDD.sample, takeSample, sampleByKey Added sc.stop() to all examples. CorrelationSuite.scala * Added 1 test for RDDs with only 1 value RowMatrix.scala * numCols(): Added check for numRows = 0, with error message. * computeCovariance(): Added check for numRows <= 1, with error message. Python SparseVector (pyspark/mllib/linalg.py) * Added toDense() function python/run-tests script * Added stat.py (doc test) CC: mengxr dorx Main changes were examples to show usage across APIs. Author: Joseph K. Bradley Closes #1878 from jkbradley/mllib-stats-api-check and squashes the following commits: ea5c047 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check dafebe2 [Joseph K. Bradley] Bug fixes for examples SampledRDDs.scala and sampled_rdds.py: Check for division by 0 and for missing key in maps. 8d1e555 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 60c72d9 [Joseph K. Bradley] Fixed stat.py doc test to work for Python versions printing nan or NaN. b20d90a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 4e5d15e [Joseph K. Bradley] Changed pyspark/mllib/stat.py doc tests to use NaN instead of nan. 32173b7 [Joseph K. Bradley] Stats examples update. c8c20dc [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check cf70b07 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 0b7cec3 [Joseph K. Bradley] Small updates based on code review. Renamed statistical_summary.py to correlations.py ab48f6e [Joseph K. Bradley] RowMatrix.scala * numCols(): Added check for numRows = 0, with error message. * computeCovariance(): Added check for numRows <= 1, with error message. 65e4ebc [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check 8195c78 [Joseph K. Bradley] Added examples for random and sampled RDDs: * Scala: RandomAndSampledRDDs.scala * python: random_and_sampled_rdds.py * Both test: ** RandomRDDGenerators.normalRDD, normalVectorRDD ** RDD.sample, takeSample, sampleByKey 064985b [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into mllib-stats-api-check ee918e9 [Joseph K. Bradley] Added examples for statistical summarization: * Scala: StatisticalSummary.scala ** Tests: correlation, MultivariateOnlineSummarizer * python: statistical_summary.py ** Tests: correlation (since MultivariateOnlineSummarizer has no Python API) --- examples/src/main/python/als.py | 2 + .../src/main/python/cassandra_inputformat.py | 2 + .../src/main/python/cassandra_outputformat.py | 2 + examples/src/main/python/hbase_inputformat.py | 2 + .../src/main/python/hbase_outputformat.py | 2 + examples/src/main/python/kmeans.py | 2 + .../src/main/python/logistic_regression.py | 2 + .../src/main/python/mllib/correlations.py | 60 +++++++++ .../main/python/mllib/decision_tree_runner.py | 5 + examples/src/main/python/mllib/kmeans.py | 1 + .../main/python/mllib/logistic_regression.py | 1 + .../python/mllib/random_rdd_generation.py | 55 ++++++++ .../src/main/python/mllib/sampled_rdds.py | 86 ++++++++++++ examples/src/main/python/pagerank.py | 2 + examples/src/main/python/pi.py | 2 + examples/src/main/python/sort.py | 2 + .../src/main/python/transitive_closure.py | 2 + examples/src/main/python/wordcount.py | 2 + .../spark/examples/mllib/Correlations.scala | 92 +++++++++++++ .../mllib/MultivariateSummarizer.scala | 98 ++++++++++++++ .../examples/mllib/RandomRDDGeneration.scala | 60 +++++++++ .../spark/examples/mllib/SampledRDDs.scala | 126 ++++++++++++++++++ .../mllib/linalg/distributed/RowMatrix.scala | 14 +- .../stat/MultivariateOnlineSummarizer.scala | 8 +- .../spark/mllib/stat/CorrelationSuite.scala | 15 ++- .../MultivariateOnlineSummarizerSuite.scala | 6 +- python/pyspark/mllib/linalg.py | 10 ++ python/pyspark/mllib/stat.py | 22 +-- python/run-tests | 1 + 29 files changed, 664 insertions(+), 20 deletions(-) create mode 100755 examples/src/main/python/mllib/correlations.py create mode 100755 examples/src/main/python/mllib/random_rdd_generation.py create mode 100755 examples/src/main/python/mllib/sampled_rdds.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index c862650b0aa1d..5b1fa4d997eeb 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -97,3 +97,5 @@ def update(i, vec, mat, ratings): error = rmse(R, ms, us) print "Iteration %d:" % i print "\nRMSE: %5.4f\n" % error + + sc.stop() diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index 39fa6b0d22ef5..e4a897f61e39d 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -77,3 +77,5 @@ output = cass_rdd.collect() for (k, v) in output: print (k, v) + + sc.stop() diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index 1dfbf98604425..836c35b5c6794 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -81,3 +81,5 @@ conf=conf, keyConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLKeyConverter", valueConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLValueConverter") + + sc.stop() diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index c9fa8e171c2a1..befacee0dea56 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -71,3 +71,5 @@ output = hbase_rdd.collect() for (k, v) in output: print (k, v) + + sc.stop() diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index 5e11548fd13f7..49bbc5aebdb0b 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -63,3 +63,5 @@ conf=conf, keyConverter="org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter", valueConverter="org.apache.spark.examples.pythonconverters.StringListToPutConverter") + + sc.stop() diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 036bdf4c4f999..86ef6f32c84e8 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -77,3 +77,5 @@ def closestPoint(p, centers): kPoints[x] = y print "Final centers: " + str(kPoints) + + sc.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 8456b272f9c05..3aa56b0528168 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -80,3 +80,5 @@ def add(x, y): w -= points.map(lambda m: gradient(m, w)).reduce(add) print "Final w: " + str(w) + + sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py new file mode 100755 index 0000000000000..6b16a56e44af7 --- /dev/null +++ b/examples/src/main/python/mllib/correlations.py @@ -0,0 +1,60 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Correlations using MLlib. +""" + +import sys + +from pyspark import SparkContext +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.stat import Statistics +from pyspark.mllib.util import MLUtils + + +if __name__ == "__main__": + if len(sys.argv) not in [1,2]: + print >> sys.stderr, "Usage: correlations ()" + exit(-1) + sc = SparkContext(appName="PythonCorrelations") + if len(sys.argv) == 2: + filepath = sys.argv[1] + else: + filepath = 'data/mllib/sample_linear_regression_data.txt' + corrType = 'pearson' + + points = MLUtils.loadLibSVMFile(sc, filepath)\ + .map(lambda lp: LabeledPoint(lp.label, lp.features.toArray())) + + print + print 'Summary of data file: ' + filepath + print '%d data points' % points.count() + + # Statistics (correlations) + print + print 'Correlation (%s) between label and each feature' % corrType + print 'Feature\tCorrelation' + numFeatures = points.take(1)[0].features.size + labelRDD = points.map(lambda lp: lp.label) + for i in range(numFeatures): + featureRDD = points.map(lambda lp: lp.features[i]) + corr = Statistics.corr(labelRDD, featureRDD, corrType) + print '%d\t%g' % (i, corr) + print + + sc.stop() diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index db96a7cb3730f..6e4a4a0cb6be0 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -17,6 +17,8 @@ """ Decision tree classification and regression using MLlib. + +This example requires NumPy (http://www.numpy.org/). """ import numpy, os, sys @@ -117,6 +119,7 @@ def usage(): if len(sys.argv) == 2: dataPath = sys.argv[1] if not os.path.isfile(dataPath): + sc.stop() usage() points = MLUtils.loadLibSVMFile(sc, dataPath) @@ -133,3 +136,5 @@ def usage(): print " Model depth: %d\n" % model.depth() print " Training accuracy: %g\n" % getAccuracy(model, reindexedData) print model + + sc.stop() diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index b308132c9aeeb..2eeb1abeeb12b 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -42,3 +42,4 @@ def parseVector(line): k = int(sys.argv[2]) model = KMeans.train(data, k) print "Final centers: " + str(model.clusterCenters) + sc.stop() diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 9d547ff77c984..8cae27fc4a52d 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -50,3 +50,4 @@ def parsePoint(line): model = LogisticRegressionWithSGD.train(points, iterations) print "Final weights: " + str(model.weights) print "Final intercept: " + str(model.intercept) + sc.stop() diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py new file mode 100755 index 0000000000000..b388d8d83fb86 --- /dev/null +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -0,0 +1,55 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Randomly generated RDDs. +""" + +import sys + +from pyspark import SparkContext +from pyspark.mllib.random import RandomRDDs + + +if __name__ == "__main__": + if len(sys.argv) not in [1, 2]: + print >> sys.stderr, "Usage: random_rdd_generation" + exit(-1) + + sc = SparkContext(appName="PythonRandomRDDGeneration") + + numExamples = 10000 # number of examples to generate + fraction = 0.1 # fraction of data to sample + + # Example: RandomRDDs.normalRDD + normalRDD = RandomRDDs.normalRDD(sc, numExamples) + print 'Generated RDD of %d examples sampled from the standard normal distribution'\ + % normalRDD.count() + print ' First 5 samples:' + for sample in normalRDD.take(5): + print ' ' + str(sample) + print + + # Example: RandomRDDs.normalVectorRDD + normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows = numExamples, numCols = 2) + print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() + print ' First 5 samples:' + for sample in normalVectorRDD.take(5): + print ' ' + str(sample) + print + + sc.stop() diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py new file mode 100755 index 0000000000000..ec64a5978c672 --- /dev/null +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -0,0 +1,86 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Randomly sampled RDDs. +""" + +import sys + +from pyspark import SparkContext +from pyspark.mllib.util import MLUtils + + +if __name__ == "__main__": + if len(sys.argv) not in [1, 2]: + print >> sys.stderr, "Usage: sampled_rdds " + exit(-1) + if len(sys.argv) == 2: + datapath = sys.argv[1] + else: + datapath = 'data/mllib/sample_binary_classification_data.txt' + + sc = SparkContext(appName="PythonSampledRDDs") + + fraction = 0.1 # fraction of data to sample + + examples = MLUtils.loadLibSVMFile(sc, datapath) + numExamples = examples.count() + if numExamples == 0: + print >> sys.stderr, "Error: Data file had no samples to load." + exit(1) + print 'Loaded data with %d examples from file: %s' % (numExamples, datapath) + + # Example: RDD.sample() and RDD.takeSample() + expectedSampleSize = int(numExamples * fraction) + print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ + % (fraction, expectedSampleSize) + sampledRDD = examples.sample(withReplacement = True, fraction = fraction) + print ' RDD.sample(): sample has %d examples' % sampledRDD.count() + sampledArray = examples.takeSample(withReplacement = True, num = expectedSampleSize) + print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) + + print + + # Example: RDD.sampleByKey() + keyedRDD = examples.map(lambda lp: (int(lp.label), lp.features)) + print ' Keyed data using label (Int) as key ==> Orig' + # Count examples per label in original data. + keyCountsA = keyedRDD.countByKey() + + # Subsample, and count examples per label in sampled data. + fractions = {} + for k in keyCountsA.keys(): + fractions[k] = fraction + sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement = True, fractions = fractions) + keyCountsB = sampledByKeyRDD.countByKey() + sizeB = sum(keyCountsB.values()) + print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ + % sizeB + + # Compare samples + print ' \tFractions of examples with key' + print 'Key\tOrig\tSample' + for k in sorted(keyCountsA.keys()): + fracA = keyCountsA[k] / float(numExamples) + if sizeB != 0: + fracB = keyCountsB.get(k, 0) / float(sizeB) + else: + fracB = 0 + print '%d\t%g\t%g' % (k, fracA, fracB) + + sc.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index 0b96343158d44..b539c4128cdcc 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -68,3 +68,5 @@ def parseNeighbors(urls): # Collects all URL ranks and dump them to console. for (link, rank) in ranks.collect(): print "%s has rank: %s." % (link, rank) + + sc.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index 21d94a2cd4b64..fc37459dc74aa 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -37,3 +37,5 @@ def f(_): count = sc.parallelize(xrange(1, n+1), slices).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) + + sc.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index 41d00c1b79133..bb686f17518a0 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -34,3 +34,5 @@ output = sortedCount.collect() for (num, unitcount) in output: print num + + sc.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 8698369b13d84..bf331b542c438 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -64,3 +64,5 @@ def generateGraph(): break print "TC has %i edges" % tc.count() + + sc.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index dcc095fdd0ed9..ae6cd13b83d92 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -33,3 +33,5 @@ output = counts.collect() for (word, count) in output: print "%s: %i" % (word, count) + + sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala new file mode 100644 index 0000000000000..d6b2fe430e5a4 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import scopt.OptionParser + +import org.apache.spark.mllib.stat.Statistics +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.{SparkConf, SparkContext} + + +/** + * An example app for summarizing multivariate data from a file. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.Correlations + * }}} + * By default, this loads a synthetic dataset from `data/mllib/sample_linear_regression_data.txt`. + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object Correlations { + + case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") + + def main(args: Array[String]) { + + val defaultParams = Params() + + val parser = new OptionParser[Params]("Correlations") { + head("Correlations: an example app for computing correlations") + opt[String]("input") + .text(s"Input path to labeled examples in LIBSVM format, default: ${defaultParams.input}") + .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.Correlations \ + | examples/target/scala-*/spark-examples-*.jar \ + | --input data/mllib/sample_linear_regression_data.txt + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"Correlations with $params") + val sc = new SparkContext(conf) + + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() + + println(s"Summary of data file: ${params.input}") + println(s"${examples.count()} data points") + + // Calculate label -- feature correlations + val labelRDD = examples.map(_.label) + val numFeatures = examples.take(1)(0).features.size + val corrType = "pearson" + println() + println(s"Correlation ($corrType) between label and each feature") + println(s"Feature\tCorrelation") + var feature = 0 + while (feature < numFeatures) { + val featureRDD = examples.map(_.features(feature)) + val corr = Statistics.corr(labelRDD, featureRDD) + println(s"$feature\t$corr") + feature += 1 + } + println() + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala new file mode 100644 index 0000000000000..4532512c01f84 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import scopt.OptionParser + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.{SparkConf, SparkContext} + + +/** + * An example app for summarizing multivariate data from a file. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.MultivariateSummarizer + * }}} + * By default, this loads a synthetic dataset from `data/mllib/sample_linear_regression_data.txt`. + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object MultivariateSummarizer { + + case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") + + def main(args: Array[String]) { + + val defaultParams = Params() + + val parser = new OptionParser[Params]("MultivariateSummarizer") { + head("MultivariateSummarizer: an example app for MultivariateOnlineSummarizer") + opt[String]("input") + .text(s"Input path to labeled examples in LIBSVM format, default: ${defaultParams.input}") + .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.MultivariateSummarizer \ + | examples/target/scala-*/spark-examples-*.jar \ + | --input data/mllib/sample_linear_regression_data.txt + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"MultivariateSummarizer with $params") + val sc = new SparkContext(conf) + + val examples = MLUtils.loadLibSVMFile(sc, params.input).cache() + + println(s"Summary of data file: ${params.input}") + println(s"${examples.count()} data points") + + // Summarize labels + val labelSummary = examples.aggregate(new MultivariateOnlineSummarizer())( + (summary, lp) => summary.add(Vectors.dense(lp.label)), + (sum1, sum2) => sum1.merge(sum2)) + + // Summarize features + val featureSummary = examples.aggregate(new MultivariateOnlineSummarizer())( + (summary, lp) => summary.add(lp.features), + (sum1, sum2) => sum1.merge(sum2)) + + println() + println(s"Summary statistics") + println(s"\tLabel\tFeatures") + println(s"mean\t${labelSummary.mean(0)}\t${featureSummary.mean.toArray.mkString("\t")}") + println(s"var\t${labelSummary.variance(0)}\t${featureSummary.variance.toArray.mkString("\t")}") + println( + s"nnz\t${labelSummary.numNonzeros(0)}\t${featureSummary.numNonzeros.toArray.mkString("\t")}") + println(s"max\t${labelSummary.max(0)}\t${featureSummary.max.toArray.mkString("\t")}") + println(s"min\t${labelSummary.min(0)}\t${featureSummary.min.toArray.mkString("\t")}") + println() + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala new file mode 100644 index 0000000000000..924b586e3af99 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RandomRDDGeneration.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.mllib.random.RandomRDDs +import org.apache.spark.rdd.RDD + +import org.apache.spark.{SparkConf, SparkContext} + +/** + * An example app for randomly generated RDDs. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.RandomRDDGeneration + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object RandomRDDGeneration { + + def main(args: Array[String]) { + + val conf = new SparkConf().setAppName(s"RandomRDDGeneration") + val sc = new SparkContext(conf) + + val numExamples = 10000 // number of examples to generate + val fraction = 0.1 // fraction of data to sample + + // Example: RandomRDDs.normalRDD + val normalRDD: RDD[Double] = RandomRDDs.normalRDD(sc, numExamples) + println(s"Generated RDD of ${normalRDD.count()}" + + " examples sampled from the standard normal distribution") + println(" First 5 samples:") + normalRDD.take(5).foreach( x => println(s" $x") ) + + // Example: RandomRDDs.normalVectorRDD + val normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows = numExamples, numCols = 2) + println(s"Generated RDD of ${normalVectorRDD.count()} examples of length-2 vectors.") + println(" First 5 samples:") + normalVectorRDD.take(5).foreach( x => println(s" $x") ) + + println() + + sc.stop() + } + +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala new file mode 100644 index 0000000000000..f01b8266e3fe3 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.mllib.util.MLUtils +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ + +/** + * An example app for randomly generated and sampled RDDs. Run with + * {{{ + * bin/run-example org.apache.spark.examples.mllib.SampledRDDs + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object SampledRDDs { + + case class Params(input: String = "data/mllib/sample_binary_classification_data.txt") + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("SampledRDDs") { + head("SampledRDDs: an example app for randomly generated and sampled RDDs.") + opt[String]("input") + .text(s"Input path to labeled examples in LIBSVM format, default: ${defaultParams.input}") + .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.SampledRDDs \ + | examples/target/scala-*/spark-examples-*.jar + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName(s"SampledRDDs with $params") + val sc = new SparkContext(conf) + + val fraction = 0.1 // fraction of data to sample + + val examples = MLUtils.loadLibSVMFile(sc, params.input) + val numExamples = examples.count() + if (numExamples == 0) { + throw new RuntimeException("Error: Data file had no samples to load.") + } + println(s"Loaded data with $numExamples examples from file: ${params.input}") + + // Example: RDD.sample() and RDD.takeSample() + val expectedSampleSize = (numExamples * fraction).toInt + println(s"Sampling RDD using fraction $fraction. Expected sample size = $expectedSampleSize.") + val sampledRDD = examples.sample(withReplacement = true, fraction = fraction) + println(s" RDD.sample(): sample has ${sampledRDD.count()} examples") + val sampledArray = examples.takeSample(withReplacement = true, num = expectedSampleSize) + println(s" RDD.takeSample(): sample has ${sampledArray.size} examples") + + println() + + // Example: RDD.sampleByKey() and RDD.sampleByKeyExact() + val keyedRDD = examples.map { lp => (lp.label.toInt, lp.features) } + println(s" Keyed data using label (Int) as key ==> Orig") + // Count examples per label in original data. + val keyCounts = keyedRDD.countByKey() + + // Subsample, and count examples per label in sampled data. (approximate) + val fractions = keyCounts.keys.map((_, fraction)).toMap + val sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement = true, fractions = fractions) + val keyCountsB = sampledByKeyRDD.countByKey() + val sizeB = keyCountsB.values.sum + println(s" Sampled $sizeB examples using approximate stratified sampling (by label)." + + " ==> Approx Sample") + + // Subsample, and count examples per label in sampled data. (approximate) + val sampledByKeyRDDExact = + keyedRDD.sampleByKeyExact(withReplacement = true, fractions = fractions) + val keyCountsBExact = sampledByKeyRDDExact.countByKey() + val sizeBExact = keyCountsBExact.values.sum + println(s" Sampled $sizeBExact examples using exact stratified sampling (by label)." + + " ==> Exact Sample") + + // Compare samples + println(s" \tFractions of examples with key") + println(s"Key\tOrig\tApprox Sample\tExact Sample") + keyCounts.keys.toSeq.sorted.foreach { key => + val origFrac = keyCounts(key) / numExamples.toDouble + val approxFrac = if (sizeB != 0) { + keyCountsB.getOrElse(key, 0L) / sizeB.toDouble + } else { + 0 + } + val exactFrac = if (sizeBExact != 0) { + keyCountsBExact.getOrElse(key, 0L) / sizeBExact.toDouble + } else { + 0 + } + println(s"$key\t$origFrac\t$approxFrac\t$exactFrac") + } + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index e76bc9fefff01..2e414a73be8e0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -53,8 +53,14 @@ class RowMatrix( /** Gets or computes the number of columns. */ override def numCols(): Long = { if (nCols <= 0) { - // Calling `first` will throw an exception if `rows` is empty. - nCols = rows.first().size + try { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().size + } catch { + case err: UnsupportedOperationException => + sys.error("Cannot determine the number of cols because it is not specified in the " + + "constructor and the rows RDD is empty.") + } } nCols } @@ -293,6 +299,10 @@ class RowMatrix( (s1._1 + s2._1, s1._2 += s2._2) ) + if (m <= 1) { + sys.error(s"RowMatrix.computeCovariance called on matrix with only $m rows." + + " Cannot compute the covariance of a RowMatrix with <= 1 row.") + } updateNumRows(m) mean :/= m.toDouble diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 5105b5c37aaaa..7d845c44365dd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -55,8 +55,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S */ def add(sample: Vector): this.type = { if (n == 0) { - require(sample.toBreeze.length > 0, s"Vector should have dimension larger than zero.") - n = sample.toBreeze.length + require(sample.size > 0, s"Vector should have dimension larger than zero.") + n = sample.size currMean = BDV.zeros[Double](n) currM2n = BDV.zeros[Double](n) @@ -65,8 +65,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currMin = BDV.fill(n)(Double.MaxValue) } - require(n == sample.toBreeze.length, s"Dimensions mismatch when adding new sample." + - s" Expecting $n but got ${sample.toBreeze.length}.") + require(n == sample.size, s"Dimensions mismatch when adding new sample." + + s" Expecting $n but got ${sample.size}.") sample.toBreeze.activeIterator.foreach { case (_, 0.0) => // Skip explicit zero elements. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala index a3f76f77a5dcc..34548c86ebc14 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -39,6 +39,17 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { Vectors.dense(9.0, 0.0, 0.0, 1.0) ) + test("corr(x, y) pearson, 1 value in data") { + val x = sc.parallelize(Array(1.0)) + val y = sc.parallelize(Array(4.0)) + intercept[RuntimeException] { + Statistics.corr(x, y, "pearson") + } + intercept[RuntimeException] { + Statistics.corr(x, y, "spearman") + } + } + test("corr(x, y) default, pearson") { val x = sc.parallelize(xData) val y = sc.parallelize(yData) @@ -58,7 +69,7 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { // RDD of zero variance val z = sc.parallelize(zeros) - assert(Statistics.corr(x, z).isNaN()) + assert(Statistics.corr(x, z).isNaN) } test("corr(x, y) spearman") { @@ -78,7 +89,7 @@ class CorrelationSuite extends FunSuite with LocalSparkContext { // RDD of zero variance => zero variance in ranks val z = sc.parallelize(zeros) - assert(Statistics.corr(x, z, "spearman").isNaN()) + assert(Statistics.corr(x, z, "spearman").isNaN) } test("corr(X) default, pearson") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala index db13f142df517..1e9415249104b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizerSuite.scala @@ -139,7 +139,8 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") assert(summarizer.variance ~== - Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") assert(summarizer.count === 6) } @@ -167,7 +168,8 @@ class MultivariateOnlineSummarizerSuite extends FunSuite { assert(summarizer.numNonzeros ~== Vectors.dense(3, 5, 2) absTol 1E-5, "numNonzeros mismatch") assert(summarizer.variance ~== - Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, "variance mismatch") + Vectors.dense(3.857666666666, 7.0456666666666, 2.48166666666666) absTol 1E-5, + "variance mismatch") assert(summarizer.count === 6) } diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 9a239abfbbeb1..f485a69db1fa2 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -23,6 +23,7 @@ SciPy is available in their environment. """ +import numpy from numpy import array, array_equal, ndarray, float64, int32 @@ -160,6 +161,15 @@ def squared_distance(self, other): j += 1 return result + def toArray(self): + """ + Returns a copy of this SparseVector as a 1-dimensional NumPy array. + """ + arr = numpy.zeros(self.size) + for i in xrange(self.indices.size): + arr[self.indices[i]] = self.values[i] + return arr + def __str__(self): inds = "[" + ",".join([str(i) for i in self.indices]) + "]" vals = "[" + ",".join([str(v) for v in self.values]) + "]" diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index a73abc5ff90df..feef0d16cd644 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -118,16 +118,18 @@ def corr(x, y=None, method=None): >>> from linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) - >>> Statistics.corr(rdd) - array([[ 1. , 0.05564149, nan, 0.40047142], - [ 0.05564149, 1. , nan, 0.91359586], - [ nan, nan, 1. , nan], - [ 0.40047142, 0.91359586, nan, 1. ]]) - >>> Statistics.corr(rdd, method="spearman") - array([[ 1. , 0.10540926, nan, 0.4 ], - [ 0.10540926, 1. , nan, 0.9486833 ], - [ nan, nan, 1. , nan], - [ 0.4 , 0.9486833 , nan, 1. ]]) + >>> pearsonCorr = Statistics.corr(rdd) + >>> print str(pearsonCorr).replace('nan', 'NaN') + [[ 1. 0.05564149 NaN 0.40047142] + [ 0.05564149 1. NaN 0.91359586] + [ NaN NaN 1. NaN] + [ 0.40047142 0.91359586 NaN 1. ]] + >>> spearmanCorr = Statistics.corr(rdd, method="spearman") + >>> print str(spearmanCorr).replace('nan', 'NaN') + [[ 1. 0.10540926 NaN 0.4 ] + [ 0.10540926 1. NaN 0.9486833 ] + [ NaN NaN 1. NaN] + [ 0.4 0.9486833 NaN 1. ]] >>> try: ... Statistics.corr(rdd, "spearman") ... print "Method name as second argument without 'method=' shouldn't be allowed." diff --git a/python/run-tests b/python/run-tests index a6271e0cf5fa9..b506559a5e810 100755 --- a/python/run-tests +++ b/python/run-tests @@ -78,6 +78,7 @@ run_test "pyspark/mllib/linalg.py" run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" +run_test "pyspark/mllib/stat.py" run_test "pyspark/mllib/tests.py" run_test "pyspark/mllib/tree.py" run_test "pyspark/mllib/util.py" From 217b5e915e2f21f047dfc4be680cd20d58baf9f8 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 18 Aug 2014 18:20:54 -0700 Subject: [PATCH 0409/1492] [SPARK-3108][MLLIB] add predictOnValues to StreamingLR and fix predictOn It is useful in streaming to allow users to carry extra data with the prediction, for monitoring the prediction error for example. freeman-lab Author: Xiangrui Meng Closes #2023 from mengxr/predict-on-values and squashes the following commits: cac47b8 [Xiangrui Meng] add classtag 2821b3b [Xiangrui Meng] use mapValues 0925efa [Xiangrui Meng] add predictOnValues to StreamingLR and fix predictOn --- .../mllib/StreamingLinearRegression.scala | 4 +-- .../regression/StreamingLinearAlgorithm.scala | 31 +++++++++++++++---- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala index 0e992fa9967bb..c5bd5b0b178d9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingLinearRegression.scala @@ -59,10 +59,10 @@ object StreamingLinearRegression { val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) val model = new StreamingLinearRegressionWithSGD() - .setInitialWeights(Vectors.dense(Array.fill[Double](args(3).toInt)(0))) + .setInitialWeights(Vectors.zeros(args(3).toInt)) model.trainOn(trainingData) - model.predictOn(testData).print() + model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() ssc.start() ssc.awaitTermination() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala index b8b0b42611775..8db0442a7a569 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/StreamingLinearAlgorithm.scala @@ -17,8 +17,12 @@ package org.apache.spark.mllib.regression -import org.apache.spark.annotation.DeveloperApi +import scala.reflect.ClassTag + import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream /** @@ -92,15 +96,30 @@ abstract class StreamingLinearAlgorithm[ /** * Use the model to make predictions on batches of data from a DStream * - * @param data DStream containing labeled data + * @param data DStream containing feature vectors * @return DStream containing predictions */ - def predictOn(data: DStream[LabeledPoint]): DStream[Double] = { + def predictOn(data: DStream[Vector]): DStream[Double] = { if (Option(model.weights) == None) { - logError("Initial weights must be set before starting prediction") - throw new IllegalArgumentException + val msg = "Initial weights must be set before starting prediction" + logError(msg) + throw new IllegalArgumentException(msg) } - data.map(x => model.predict(x.features)) + data.map(model.predict) } + /** + * Use the model to make predictions on the values of a DStream and carry over its keys. + * @param data DStream containing feature vectors + * @tparam K key type + * @return DStream containing the input keys and the predictions as values + */ + def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Double)] = { + if (Option(model.weights) == None) { + val msg = "Initial weights must be set before starting prediction" + logError(msg) + throw new IllegalArgumentException(msg) + } + data.mapValues(model.predict) + } } From 1f1819b20f887b487557c31e54b8bcd95b582dc6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 18 Aug 2014 20:42:19 -0700 Subject: [PATCH 0410/1492] [SPARK-3114] [PySpark] Fix Python UDFs in Spark SQL. This fixes SPARK-3114, an issue where we inadvertently broke Python UDFs in Spark SQL. This PR modifiers the test runner script to always run the PySpark SQL tests, irrespective of whether SparkSQL itself has been modified. It also includes Davies' fix for the bug. Closes #2026. Author: Josh Rosen Author: Davies Liu Closes #2027 from JoshRosen/pyspark-sql-fix and squashes the following commits: 9af2708 [Davies Liu] bugfix: disable compression of command 0d8d3a4 [Josh Rosen] Always run Python Spark SQL tests. --- dev/run-tests | 17 +++++++++++++---- python/pyspark/rdd.py | 2 +- python/pyspark/worker.py | 2 +- python/run-tests | 4 +--- 4 files changed, 16 insertions(+), 9 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 0e24515d1376c..132f696d6447a 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -58,7 +58,7 @@ if [ -n "$AMPLAB_JENKINS" ]; then diffs=`git diff --name-only master | grep "^sql/"` if [ -n "$diffs" ]; then echo "Detected changes in SQL. Will run Hive test suite." - export _RUN_SQL_TESTS=true # exported for PySpark tests + _RUN_SQL_TESTS=true fi fi @@ -89,13 +89,22 @@ echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" +# Build Spark; we always build with Hive because the PySpark SparkSQL tests need it. +# echo "q" is needed because sbt on encountering a build file with failure +# (either resolution or compilation) prompts the user for input either q, r, +# etc to quit or retry. This echo is there to make it not block. +BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver " +echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly | \ + grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + +# If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled: if [ -n "$_RUN_SQL_TESTS" ]; then SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" fi -# echo "q" is needed because sbt on encountering a build file with failure -# (either resolution or compilation) prompts the user for input either q, r, +# echo "q" is needed because sbt on encountering a build file with failure +# (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. -echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS clean package assembly/assembly test | \ +echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS test | \ grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" echo "" diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c708b69cc1e31..86cd89b245aea 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1812,7 +1812,7 @@ def _jrdd(self): self._jrdd_deserializer = NoOpSerializer() command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) - ser = CompressedSerializer(CloudPickleSerializer()) + ser = CloudPickleSerializer() pickled_command = ser.dumps(command) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 77a9c4a0e0677..6805063e06798 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -72,7 +72,7 @@ def main(infile, outfile): value = ser._read_with_length(infile) _broadcastRegistry[bid] = Broadcast(bid, value) - command = ser._read_with_length(infile) + command = pickleSer._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() iterator = deserializer.load_stream(infile) diff --git a/python/run-tests b/python/run-tests index b506559a5e810..7b1ee3e1cddba 100755 --- a/python/run-tests +++ b/python/run-tests @@ -59,9 +59,7 @@ $PYSPARK_PYTHON --version run_test "pyspark/rdd.py" run_test "pyspark/context.py" run_test "pyspark/conf.py" -if [ -n "$_RUN_SQL_TESTS" ]; then - run_test "pyspark/sql.py" -fi +run_test "pyspark/sql.py" # These tests are included in the module-level docs, and so must # be handled on a higher level rather than within the python file. export PYSPARK_DOC_TEST=1 From 82577339dd58b5811eab5d10667775e61e37ff51 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 18 Aug 2014 20:51:41 -0700 Subject: [PATCH 0411/1492] [SPARK-3116] Remove the excessive lockings in TorrentBroadcast Author: Reynold Xin Closes #2028 from rxin/torrentBroadcast and squashes the following commits: 92c62a5 [Reynold Xin] Revert the MEMORY_AND_DISK_SER changes. 03a5221 [Reynold Xin] [SPARK-3116] Remove the excessive lockings in TorrentBroadcast --- .../spark/broadcast/TorrentBroadcast.scala | 66 ++++++++----------- 1 file changed, 27 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index fe73456ef8fad..d8be649f96e5f 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -17,8 +17,7 @@ package org.apache.spark.broadcast -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, InputStream, - ObjectInputStream, ObjectOutputStream, OutputStream} +import java.io._ import scala.reflect.ClassTag import scala.util.Random @@ -53,10 +52,8 @@ private[spark] class TorrentBroadcast[T: ClassTag]( private val broadcastId = BroadcastBlockId(id) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - } + SparkEnv.get.blockManager.putSingle( + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) @transient private var arrayOfBlocks: Array[TorrentBlock] = null @transient private var totalBlocks = -1 @@ -91,18 +88,14 @@ private[spark] class TorrentBroadcast[T: ClassTag]( // Store meta-info val metaId = BroadcastBlockId(id, "meta") val metaInfo = TorrentInfo(null, totalBlocks, totalBytes) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle( - metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - } + SparkEnv.get.blockManager.putSingle( + metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // Store individual pieces for (i <- 0 until totalBlocks) { val pieceId = BroadcastBlockId(id, "piece" + i) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle( - pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - } + SparkEnv.get.blockManager.putSingle( + pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) } } @@ -165,21 +158,20 @@ private[spark] class TorrentBroadcast[T: ClassTag]( val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 while (attemptId > 0 && totalBlocks == -1) { - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.getSingle(metaId) match { - case Some(x) => - val tInfo = x.asInstanceOf[TorrentInfo] - totalBlocks = tInfo.totalBlocks - totalBytes = tInfo.totalBytes - arrayOfBlocks = new Array[TorrentBlock](totalBlocks) - hasBlocks = 0 - - case None => - Thread.sleep(500) - } + SparkEnv.get.blockManager.getSingle(metaId) match { + case Some(x) => + val tInfo = x.asInstanceOf[TorrentInfo] + totalBlocks = tInfo.totalBlocks + totalBytes = tInfo.totalBytes + arrayOfBlocks = new Array[TorrentBlock](totalBlocks) + hasBlocks = 0 + + case None => + Thread.sleep(500) } attemptId -= 1 } + if (totalBlocks == -1) { return false } @@ -192,17 +184,15 @@ private[spark] class TorrentBroadcast[T: ClassTag]( val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList) for (pid <- recvOrder) { val pieceId = BroadcastBlockId(id, "piece" + pid) - TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.getSingle(pieceId) match { - case Some(x) => - arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] - hasBlocks += 1 - SparkEnv.get.blockManager.putSingle( - pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) + SparkEnv.get.blockManager.getSingle(pieceId) match { + case Some(x) => + arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] + hasBlocks += 1 + SparkEnv.get.blockManager.putSingle( + pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - case None => - throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) - } + case None => + throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) } } @@ -291,9 +281,7 @@ private[broadcast] object TorrentBroadcast extends Logging { * If removeFromDriver is true, also remove these persisted blocks on the driver. */ def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = { - synchronized { - SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) - } + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) } } From cd0720ca77894d481fb73a8b5bb517013843cb1e Mon Sep 17 00:00:00 2001 From: Matt Forbes Date: Mon, 18 Aug 2014 21:43:32 -0700 Subject: [PATCH 0412/1492] Fix typo in decision tree docs Candidate splits were inconsistent with the example. Author: Matt Forbes Closes #1837 from emef/tree-doc and squashes the following commits: 3be14a1 [Matt Forbes] Fix typo in decision tree docs --- docs/mllib-decision-tree.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 9cbd880897578..c01a92a9a1b26 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -84,8 +84,8 @@ Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical -features are ordered as A followed by C followed B or A, B, C. The two split candidates are A \| C, B -and A , B \| C where \| denotes the split. A similar heuristic is used for multiclass classification +features are ordered as A followed by C followed B or A, C, B. The two split candidates are A \| C, B +and A , C \| B where \| denotes the split. A similar heuristic is used for multiclass classification when `$2^(M-1)-1$` is greater than the number of bins -- the impurity for each categorical feature value is used for ordering. From 7eb9cbc273d758522e787fcb2ef68ef65911475f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 19 Aug 2014 09:40:31 -0500 Subject: [PATCH 0413/1492] [SPARK-3072] YARN - Exit when reach max number failed executors In some cases on hadoop 2.x the spark application master doesn't properly exit and hangs around for 10 minutes after its really done. We should make sure it exits properly and stops the driver. Author: Thomas Graves Closes #2022 from tgravescs/SPARK-3072 and squashes the following commits: 665701d [Thomas Graves] Exit when reach max number failed executors --- .../spark/deploy/yarn/ApplicationMaster.scala | 33 ++++++++++++------- .../spark/deploy/yarn/ExecutorLauncher.scala | 5 +-- .../spark/deploy/yarn/ApplicationMaster.scala | 16 ++++++--- .../spark/deploy/yarn/ExecutorLauncher.scala | 5 +-- 4 files changed, 40 insertions(+), 19 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 62b5c3bc5f0f3..46a01f5a9a2cc 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -267,12 +267,10 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - // Exists the loop if the user thread exits. - while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } + // Exits the loop if the user thread exits. + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive + && !isFinished) { + checkNumExecutorsFailed() yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) @@ -303,11 +301,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { - while (userThread.isAlive) { - if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of executor failures reached") - } + while (userThread.isAlive && !isFinished) { + checkNumExecutorsFailed() val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". @@ -327,6 +322,22 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, t } + private def checkNumExecutorsFailed() { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + logInfo("max number of executor failures reached") + finishApplicationMaster(FinalApplicationStatus.FAILED, + "max number of executor failures reached") + // make sure to stop the user thread + val sparkContext = ApplicationMaster.sparkContextRef.get() + if (sparkContext != null) { + logInfo("Invoking sc stop from checkNumExecutorsFailed") + sparkContext.stop() + } else { + logError("sparkContext is null when should shutdown") + } + } + } + private def sendProgress() { logDebug("Sending progress") // Simulated with an allocate request with no nodes requested ... diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 184e2ad6c82cd..72c7143edcd71 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -249,7 +249,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed) && + !isFinished) { yarnAllocator.allocateContainers( math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) checkNumExecutorsFailed() @@ -271,7 +272,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp val t = new Thread { override def run() { - while (!driverClosed) { + while (!driverClosed && !isFinished) { checkNumExecutorsFailed() val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning if (missingExecutorCount > 0) { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 035356d390c80..9c2bcf17a8508 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -247,13 +247,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, yarnAllocator.allocateResources() // Exits the loop if the user thread exits. - var iters = 0 - while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive + && !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() Thread.sleep(ApplicationMaster.ALLOCATE_HEARTBEAT_INTERVAL) - iters += 1 } } logInfo("All executors have launched.") @@ -271,8 +270,17 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private def checkNumExecutorsFailed() { if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + logInfo("max number of executor failures reached") finishApplicationMaster(FinalApplicationStatus.FAILED, "max number of executor failures reached") + // make sure to stop the user thread + val sparkContext = ApplicationMaster.sparkContextRef.get() + if (sparkContext != null) { + logInfo("Invoking sc stop from checkNumExecutorsFailed") + sparkContext.stop() + } else { + logError("sparkContext is null when should shutdown") + } } } @@ -289,7 +297,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { - while (userThread.isAlive) { + while (userThread.isAlive && !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() logDebug("Sending progress") diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index fc7b8320d734d..a7585748b7f88 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -217,7 +217,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp // Wait until all containers have launched yarnAllocator.addResourceRequests(args.numExecutors) yarnAllocator.allocateResources() - while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed) && + !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() yarnAllocator.allocateResources() @@ -249,7 +250,7 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp val t = new Thread { override def run() { - while (!driverClosed) { + while (!driverClosed && !isFinished) { checkNumExecutorsFailed() allocateMissingExecutor() logDebug("Sending progress") From cbfc26ba45f49559e64276c72e3054c6fe30ddd5 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 19 Aug 2014 10:15:11 -0700 Subject: [PATCH 0414/1492] [SPARK-3089] Fix meaningless error message in ConnectionManager Author: Kousuke Saruta Closes #2000 from sarutak/SPARK-3089 and squashes the following commits: 02dfdea [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3089 e759ce7 [Kousuke Saruta] Improved error message when closing SendingConnection --- .../main/scala/org/apache/spark/network/ConnectionManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index e77d762bdf221..b3e951ded6e77 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -467,7 +467,7 @@ private[spark] class ConnectionManager( val sendingConnectionOpt = connectionsById.get(remoteConnectionManagerId) if (!sendingConnectionOpt.isDefined) { - logError("Corresponding SendingConnectionManagerId not found") + logError(s"Corresponding SendingConnection to ${remoteConnectionManagerId} not found") return } From 31f0b071efd0b63eb9d6a6a131e5c4fa28237583 Mon Sep 17 00:00:00 2001 From: freeman Date: Tue, 19 Aug 2014 13:28:57 -0700 Subject: [PATCH 0415/1492] [SPARK-3128][MLLIB] Use streaming test suite for StreamingLR Refactored tests for streaming linear regression to use existing streaming test utilities. Summary of changes: - Made ``mllib`` depend on tests from ``streaming`` - Rewrote accuracy and convergence tests to use ``setupStreams`` and ``runStreams`` - Added new test for the accuracy of predictions generated by ``predictOnValue`` These tests should run faster, be easier to extend/maintain, and provide a reference for new tests. mengxr tdas Author: freeman Closes #2037 from freeman-lab/streamingLR-predict-tests and squashes the following commits: e851ca7 [freeman] Fixed long lines 50eb0bf [freeman] Refactored tests to use streaming test tools 32c43c2 [freeman] Added test for prediction --- mllib/pom.xml | 7 + .../StreamingLinearRegressionSuite.scala | 121 ++++++++++-------- .../spark/streaming/TestSuiteBase.scala | 4 +- 3 files changed, 77 insertions(+), 55 deletions(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index fc1ecfbea708f..c7a1e2ae75c84 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -91,6 +91,13 @@ junit-interface test + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 45e25eecf508e..28489410f8225 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -17,20 +17,19 @@ package org.apache.spark.mllib.regression -import java.io.File -import java.nio.charset.Charset - import scala.collection.mutable.ArrayBuffer -import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} -import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.util.Utils +import org.apache.spark.mllib.util.LinearDataGenerator +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.TestSuiteBase + +class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { -class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { + // use longer wait time to ensure job completion + override def maxWaitTimeMillis = 20000 // Assert that two values are equal within tolerance epsilon def assertEqual(v1: Double, v2: Double, epsilon: Double) { @@ -49,35 +48,26 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { } // Test if we can accurately learn Y = 10*X1 + 10*X2 on streaming data - test("streaming linear regression parameter accuracy") { + test("parameter accuracy") { - val testDir = Files.createTempDir() - val numBatches = 10 - val batchDuration = Milliseconds(1000) - val ssc = new StreamingContext(sc, batchDuration) - val data = ssc.textFileStream(testDir.toString).map(LabeledPoint.parse) + // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) .setStepSize(0.1) - .setNumIterations(50) + .setNumIterations(25) - model.trainOn(data) - - ssc.start() - - // write data to a file stream - for (i <- 0 until numBatches) { - val samples = LinearDataGenerator.generateLinearInput( - 0.0, Array(10.0, 10.0), 100, 42 * (i + 1)) - val file = new File(testDir, i.toString) - Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) - Thread.sleep(batchDuration.milliseconds) + // generate sequence of simulated data + val numBatches = 10 + val input = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 42 * (i + 1)) } - ssc.stop(stopSparkContext=false) - - System.clearProperty("spark.driver.port") - Utils.deleteRecursively(testDir) + // apply model training to input stream + val ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) // check accuracy of final parameter estimates assertEqual(model.latestModel().intercept, 0.0, 0.1) @@ -91,39 +81,33 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { } // Test that parameter estimates improve when learning Y = 10*X1 on streaming data - test("streaming linear regression parameter convergence") { + test("parameter convergence") { - val testDir = Files.createTempDir() - val batchDuration = Milliseconds(2000) - val ssc = new StreamingContext(sc, batchDuration) - val numBatches = 5 - val data = ssc.textFileStream(testDir.toString()).map(LabeledPoint.parse) + // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) .setStepSize(0.1) - .setNumIterations(50) - - model.trainOn(data) - - ssc.start() + .setNumIterations(25) - // write data to a file stream - val history = new ArrayBuffer[Double](numBatches) - for (i <- 0 until numBatches) { - val samples = LinearDataGenerator.generateLinearInput(0.0, Array(10.0), 100, 42 * (i + 1)) - val file = new File(testDir, i.toString) - Files.write(samples.map(x => x.toString).mkString("\n"), file, Charset.forName("UTF-8")) - Thread.sleep(batchDuration.milliseconds) - // wait an extra few seconds to make sure the update finishes before new data arrive - Thread.sleep(4000) - history.append(math.abs(model.latestModel().weights(0) - 10.0)) + // generate sequence of simulated data + val numBatches = 10 + val input = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0), 100, 42 * (i + 1)) } - ssc.stop(stopSparkContext=false) + // create buffer to store intermediate fits + val history = new ArrayBuffer[Double](numBatches) - System.clearProperty("spark.driver.port") - Utils.deleteRecursively(testDir) + // apply model training to input stream, storing the intermediate results + // (we add a count to ensure the result is a DStream) + val ssc = setupStreams(input, (inputDStream: DStream[LabeledPoint]) => { + model.trainOn(inputDStream) + inputDStream.foreachRDD(x => history.append(math.abs(model.latestModel().weights(0) - 10.0))) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) + // compute change in error val deltas = history.drop(1).zip(history.dropRight(1)) // check error stability (it always either shrinks, or increases with small tol) assert(deltas.forall(x => (x._1 - x._2) <= 0.1)) @@ -132,4 +116,33 @@ class StreamingLinearRegressionSuite extends FunSuite with LocalSparkContext { } + // Test predictions on a stream + test("predictions") { + + // create model initialized with true weights + val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.dense(10.0, 10.0)) + .setStepSize(0.1) + .setNumIterations(25) + + // generate sequence of simulated data for testing + val numBatches = 10 + val nPoints = 100 + val testInput = (0 until numBatches).map { i => + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), nPoints, 42 * (i + 1)) + } + + // apply model predictions to test stream + val ssc = setupStreams(testInput, (inputDStream: DStream[LabeledPoint]) => { + model.predictOnValues(inputDStream.map(x => (x.label, x.features))) + }) + // collect the output as (true, estimated) tuples + val output: Seq[Seq[(Double, Double)]] = runStreams(ssc, numBatches, numBatches) + + // compute the mean absolute error and check that it's always less than 0.1 + val errors = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints) + assert(errors.forall(x => x <= 0.1)) + + } + } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index cc178fba12c9d..f095da9cb55d3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -242,7 +242,9 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput) // Get the output buffer - val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStreamWithPartitions[V]] + val outputStream = ssc.graph.getOutputStreams. + filter(_.isInstanceOf[TestOutputStreamWithPartitions[_]]). + head.asInstanceOf[TestOutputStreamWithPartitions[V]] val output = outputStream.output try { From 94053a7b766788bb62e2dbbf352ccbcc75f71fc0 Mon Sep 17 00:00:00 2001 From: Vida Ha Date: Tue, 19 Aug 2014 13:35:05 -0700 Subject: [PATCH 0416/1492] SPARK-2333 - spark_ec2 script should allow option for existing security group - Uses the name tag to identify machines in a cluster. - Allows overriding the security group name so it doesn't need to coincide with the cluster name. - Outputs the request id's of up to 10 pending spot instance requests. Author: Vida Ha Closes #1899 from vidaha/vida/ec2-reuse-security-group and squashes the following commits: c80d5c3 [Vida Ha] wrap retries in a try catch block b2989d5 [Vida Ha] SPARK-2333: spark_ec2 script should allow option for existing security group --- docs/ec2-scripts.md | 14 +++++---- ec2/spark_ec2.py | 71 +++++++++++++++++++++++++++++++-------------- 2 files changed, 57 insertions(+), 28 deletions(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 156a727026790..f5ac6d894e1eb 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -12,14 +12,16 @@ on the [Amazon Web Services site](http://aws.amazon.com/). `spark-ec2` is designed to manage multiple named clusters. You can launch a new cluster (telling the script its size and giving it a name), -shutdown an existing cluster, or log into a cluster. Each cluster is -identified by placing its machines into EC2 security groups whose names -are derived from the name of the cluster. For example, a cluster named +shutdown an existing cluster, or log into a cluster. Each cluster +launches a set of instances, which are tagged with the cluster name, +and placed into EC2 security groups. If you don't specify a security +group, the `spark-ec2` script will create security groups based on the +cluster name you request. For example, a cluster named `test` will contain a master node in a security group called `test-master`, and a number of slave nodes in a security group called -`test-slaves`. The `spark-ec2` script will create these security groups -for you based on the cluster name you request. You can also use them to -identify machines belonging to each cluster in the Amazon EC2 Console. +`test-slaves`. You can also specify a security group prefix to be used +in place of the cluster name. Machines in a cluster can be identified +by looking for the "Name" tag of the instance in the Amazon EC2 Console. # Before You Start diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0c2f85a3868f4..3a8c816cfffa1 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -124,7 +124,7 @@ def parse_args(): help="The SSH user you want to connect as (default: root)") parser.add_option( "--delete-groups", action="store_true", default=False, - help="When destroying a cluster, delete the security groups that were created") + help="When destroying a cluster, delete the security groups that were created.") parser.add_option( "--use-existing-master", action="store_true", default=False, help="Launch fresh slaves, but use an existing stopped master if possible") @@ -138,7 +138,9 @@ def parse_args(): parser.add_option( "--user-data", type="string", default="", help="Path to a user-data file (most AMI's interpret this as an initialization script)") - + parser.add_option( + "--security-group-prefix", type="string", default=None, + help="Use this prefix for the security group rather than the cluster name.") (opts, args) = parser.parse_args() if len(args) != 2: @@ -285,8 +287,12 @@ def launch_cluster(conn, opts, cluster_name): user_data_content = user_data_file.read() print "Setting up security groups..." - master_group = get_or_make_group(conn, cluster_name + "-master") - slave_group = get_or_make_group(conn, cluster_name + "-slaves") + if opts.security_group_prefix is None: + master_group = get_or_make_group(conn, cluster_name + "-master") + slave_group = get_or_make_group(conn, cluster_name + "-slaves") + else: + master_group = get_or_make_group(conn, opts.security_group_prefix + "-master") + slave_group = get_or_make_group(conn, opts.security_group_prefix + "-slaves") if master_group.rules == []: # Group was just now created master_group.authorize(src_group=master_group) master_group.authorize(src_group=slave_group) @@ -310,12 +316,11 @@ def launch_cluster(conn, opts, cluster_name): slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0') slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0') - # Check if instances are already running in our groups + # Check if instances are already running with the cluster name existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if existing_slaves or (existing_masters and not opts.use_existing_master): - print >> stderr, ("ERROR: There are already instances running in " + - "group %s or %s" % (master_group.name, slave_group.name)) + print >> stderr, ("ERROR: There are already instances for name: %s " % cluster_name) sys.exit(1) # Figure out Spark AMI @@ -371,9 +376,13 @@ def launch_cluster(conn, opts, cluster_name): for r in reqs: id_to_req[r.id] = r active_instance_ids = [] + outstanding_request_ids = [] for i in my_req_ids: - if i in id_to_req and id_to_req[i].state == "active": - active_instance_ids.append(id_to_req[i].instance_id) + if i in id_to_req: + if id_to_req[i].state == "active": + active_instance_ids.append(id_to_req[i].instance_id) + else: + outstanding_request_ids.append(i) if len(active_instance_ids) == opts.slaves: print "All %d slaves granted" % opts.slaves reservations = conn.get_all_instances(active_instance_ids) @@ -382,8 +391,8 @@ def launch_cluster(conn, opts, cluster_name): slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves) + print "%d of %d slaves granted, waiting longer for request ids including %s" % ( + len(active_instance_ids), opts.slaves, outstanding_request_ids[0:10]) except: print "Canceling spot instance requests" conn.cancel_spot_instance_requests(my_req_ids) @@ -440,14 +449,29 @@ def launch_cluster(conn, opts, cluster_name): print "Launched master in %s, regid = %s" % (zone, master_res.id) # Give the instances descriptive names + # TODO: Add retry logic for tagging with name since it's used to identify a cluster. for master in master_nodes: - master.add_tag( - key='Name', - value='{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) + name = '{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id) + for i in range(0, 5): + try: + master.add_tag(key='Name', value=name) + except: + print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) + if (i == 5): + raise "Error - failed max attempts to add name tag" + time.sleep(5) + + for slave in slave_nodes: - slave.add_tag( - key='Name', - value='{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) + name = '{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id) + for i in range(0, 5): + try: + slave.add_tag(key='Name', value=name) + except: + print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) + if (i == 5): + raise "Error - failed max attempts to add name tag" + time.sleep(5) # Return all the instances return (master_nodes, slave_nodes) @@ -463,10 +487,10 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): for res in reservations: active = [i for i in res.instances if is_active(i)] for inst in active: - group_names = [g.name for g in inst.groups] - if group_names == [cluster_name + "-master"]: + name = inst.tags.get(u'Name', "") + if name.startswith(cluster_name + "-master"): master_nodes.append(inst) - elif group_names == [cluster_name + "-slaves"]: + elif name.startswith(cluster_name + "-slave"): slave_nodes.append(inst) if any((master_nodes, slave_nodes)): print ("Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes))) @@ -474,7 +498,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name + "-master" + print >> sys.stderr, "ERROR: Could not find master in with name " + cluster_name + "-master" else: print >> sys.stderr, "ERROR: Could not find any existing cluster" sys.exit(1) @@ -816,7 +840,10 @@ def real_main(): # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." - group_names = [cluster_name + "-master", cluster_name + "-slaves"] + if opts.security_group_prefix is None: + group_names = [cluster_name + "-master", cluster_name + "-slaves"] + else: + group_names = [opts.security_group_prefix + "-master", opts.security_group_prefix + "-slaves"] attempt = 1 while attempt <= 3: From 76eaeb4523ee01cabbea2d867daac48a277885a1 Mon Sep 17 00:00:00 2001 From: hzw19900416 Date: Tue, 19 Aug 2014 14:04:49 -0700 Subject: [PATCH 0417/1492] Move a bracket in validateSettings of SparkConf Move a bracket in validateSettings of SparkConf Author: hzw19900416 Closes #2012 from hzw19900416/codereading and squashes the following commits: e717fb6 [hzw19900416] Move a bracket in validateSettings of SparkConf --- core/src/main/scala/org/apache/spark/SparkConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 13f0bff7ee507..b4f321ec99e78 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -227,7 +227,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { // Validate spark.executor.extraJavaOptions settings.get(executorOptsKey).map { javaOpts => if (javaOpts.contains("-Dspark")) { - val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts)'. " + + val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts'). " + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." throw new Exception(msg) } From d7e80c2597d4a9cae2e0cb35a86f7889323f4cbb Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 19 Aug 2014 14:46:32 -0700 Subject: [PATCH 0418/1492] [SPARK-2790] [PySpark] fix zip with serializers which have different batch sizes. If two RDDs have different batch size in serializers, then it will try to re-serialize the one with smaller batch size, then call RDD.zip() in Spark. Author: Davies Liu Closes #1894 from davies/zip and squashes the following commits: c4652ea [Davies Liu] add more test cases 6d05fc8 [Davies Liu] Merge branch 'master' into zip 813b1e4 [Davies Liu] add more tests for failed cases a4aafda [Davies Liu] fix zip with serializers which have different batch sizes. --- python/pyspark/rdd.py | 25 +++++++++++++++++++++++++ python/pyspark/serializers.py | 3 +++ python/pyspark/tests.py | 27 ++++++++++++++++++++++++++- 3 files changed, 54 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 86cd89b245aea..140cbe05a43b0 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1687,6 +1687,31 @@ def zip(self, other): >>> x.zip(y).collect() [(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)] """ + if self.getNumPartitions() != other.getNumPartitions(): + raise ValueError("Can only zip with RDD which has the same number of partitions") + + def get_batch_size(ser): + if isinstance(ser, BatchedSerializer): + return ser.batchSize + return 0 + + def batch_as(rdd, batchSize): + ser = rdd._jrdd_deserializer + if isinstance(ser, BatchedSerializer): + ser = ser.serializer + return rdd._reserialize(BatchedSerializer(ser, batchSize)) + + my_batch = get_batch_size(self._jrdd_deserializer) + other_batch = get_batch_size(other._jrdd_deserializer) + if my_batch != other_batch: + # use the greatest batchSize to batch the other one. + if my_batch > other_batch: + other = batch_as(other, my_batch) + else: + self = batch_as(self, other_batch) + + # There will be an Exception in JVM if there are different number + # of items in each partitions. pairRDD = self._jrdd.zip(other._jrdd) deserializer = PairDeserializer(self._jrdd_deserializer, other._jrdd_deserializer) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 74870c0edcf99..fc49aa42dbaf9 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -255,6 +255,9 @@ def __init__(self, key_ser, val_ser): def load_stream(self, stream): for (keys, vals) in self.prepare_keys_values(stream): + if len(keys) != len(vals): + raise ValueError("Can not deserialize RDD with different number of items" + " in pair: (%d, %d)" % (len(keys), len(vals))) for pair in izip(keys, vals): yield pair diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 69d543d9d045d..51bfbb47e53c2 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -39,7 +39,7 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles -from pyspark.serializers import read_int +from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger _have_scipy = False @@ -339,6 +339,31 @@ def test_large_broadcast(self): m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() self.assertEquals(N, m) + def test_zip_with_different_serializers(self): + a = self.sc.parallelize(range(5)) + b = self.sc.parallelize(range(100, 105)) + self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)]) + a = a._reserialize(BatchedSerializer(PickleSerializer(), 2)) + b = b._reserialize(MarshalSerializer()) + self.assertEqual(a.zip(b).collect(), [(0, 100), (1, 101), (2, 102), (3, 103), (4, 104)]) + + def test_zip_with_different_number_of_items(self): + a = self.sc.parallelize(range(5), 2) + # different number of partitions + b = self.sc.parallelize(range(100, 106), 3) + self.assertRaises(ValueError, lambda: a.zip(b)) + # different number of batched items in JVM + b = self.sc.parallelize(range(100, 104), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # different number of items in one pair + b = self.sc.parallelize(range(100, 106), 2) + self.assertRaises(Exception, lambda: a.zip(b).count()) + # same total number of items, but different distributions + a = self.sc.parallelize([2, 3], 2).flatMap(range) + b = self.sc.parallelize([3, 2], 2).flatMap(range) + self.assertEquals(a.count(), b.count()) + self.assertRaises(Exception, lambda: a.zip(b).count()) + class TestIO(PySparkTestCase): From 825d4fe47b9c4d48de88622dd48dcf83beb8b80a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 16:06:48 -0700 Subject: [PATCH 0419/1492] [SPARK-3136][MLLIB] Create Java-friendly methods in RandomRDDs Though we don't use default argument for methods in RandomRDDs, it is still not easy for Java users to use because the output type is either `RDD[Double]` or `RDD[Vector]`. Java users should expect `JavaDoubleRDD` and `JavaRDD[Vector]`, respectively. We should create dedicated methods for Java users, and allow default arguments in Scala methods in RandomRDDs, to make life easier for both Java and Scala users. This PR also contains documentation for random data generation. brkyvz Author: Xiangrui Meng Closes #2041 from mengxr/stat-doc and squashes the following commits: fc5eedf [Xiangrui Meng] add missing comma ffde810 [Xiangrui Meng] address comments aef6d07 [Xiangrui Meng] add doc for random data generation b99d94b [Xiangrui Meng] add java-friendly methods to RandomRDDs --- docs/mllib-guide.md | 2 +- docs/mllib-stats.md | 74 ++- .../mllib/random/RandomDataGenerator.scala | 18 +- .../spark/mllib/random/RandomRDDs.scala | 476 +++++++----------- .../mllib/random/JavaRandomRDDsSuite.java | 134 +++++ python/pyspark/mllib/random.py | 20 +- 6 files changed, 418 insertions(+), 306 deletions(-) create mode 100644 mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 23d5a0c4607af..ca0a84a8c53fd 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -9,7 +9,7 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * [Data types](mllib-basics.html) * [Basic statistics](mllib-stats.html) - * data generators + * random data generation * stratified sampling * summary statistics * hypothesis testing diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md index ca9ef46c15186..f25dca746ba3a 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-stats.md @@ -25,7 +25,79 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` -## Data Generators +## Random data generation + +Random data generation is useful for randomized algorithms, prototyping, and performance testing. +MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: +uniform, standard normal, or Poisson. + +
    +
    +[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.random.RandomRDDs._ + +val sc: SparkContext = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +val u = normalRDD(sc, 1000000L, 10) +// Apply a transform to get a random double RDD following `N(1, 4)`. +val v = u.map(x => 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight java %} +import org.apache.spark.SparkContext; +import org.apache.spark.api.JavaDoubleRDD; +import static org.apache.spark.mllib.random.RandomRDDs.*; + +JavaSparkContext jsc = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); +// Apply a transform to get a random double RDD following `N(1, 4)`. +JavaDoubleRDD v = u.map( + new Function() { + public Double call(Double x) { + return 1.0 + 2.0 * x; + } + }); +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight python %} +from pyspark.mllib.random import RandomRDDs + +sc = ... # SparkContext + +# Generate a random double RDD that contains 1 million i.i.d. values drawn from the +# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +u = RandomRDDs.uniformRDD(sc, 1000000L, 10) +# Apply a transform to get a random double RDD following `N(1, 4)`. +v = u.map(lambda x: 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    ## Stratified Sampling diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 9cab49f6ed1f0..28179fbc450c0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -20,14 +20,14 @@ package org.apache.spark.mllib.random import cern.jet.random.Poisson import cern.jet.random.engine.DRand -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} /** - * :: Experimental :: + * :: DeveloperApi :: * Trait for random data generators that generate i.i.d. data. */ -@Experimental +@DeveloperApi trait RandomDataGenerator[T] extends Pseudorandom with Serializable { /** @@ -43,10 +43,10 @@ trait RandomDataGenerator[T] extends Pseudorandom with Serializable { } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates i.i.d. samples from U[0.0, 1.0] */ -@Experimental +@DeveloperApi class UniformGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. @@ -62,10 +62,10 @@ class UniformGenerator extends RandomDataGenerator[Double] { } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates i.i.d. samples from the standard normal distribution. */ -@Experimental +@DeveloperApi class StandardNormalGenerator extends RandomDataGenerator[Double] { // XORShiftRandom for better performance. Thread safety isn't necessary here. @@ -81,12 +81,12 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates i.i.d. samples from the Poisson distribution with the given mean. * * @param mean mean for the Poisson distribution. */ -@Experimental +@DeveloperApi class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { private var rng = new Poisson(mean, new DRand) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala index 36270369526cd..c5f4b084321f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomRDDs.scala @@ -20,9 +20,10 @@ package org.apache.spark.mllib.random import scala.reflect.ClassTag import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental +import org.apache.spark.annotation.{DeveloperApi, Experimental} +import org.apache.spark.api.java.{JavaDoubleRDD, JavaRDD, JavaSparkContext} import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.rdd.{RandomVectorRDD, RandomRDD} +import org.apache.spark.mllib.rdd.{RandomRDD, RandomVectorRDD} import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -34,335 +35,279 @@ import org.apache.spark.util.Utils object RandomRDDs { /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. + * Generates an RDD comprised of i.i.d. samples from the uniform distribution `U(0.0, 1.0)`. * - * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - * `RandomRDDGenerators.uniformRDD(sc, n, p, seed).map(v => a + (b - a) * v)`. + * To transform the distribution in the generated RDD from `U(0.0, 1.0)` to `U(a, b)`, use + * `RandomRDDs.uniformRDD(sc, n, p, seed).map(v => a + (b - a) * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Double] comprised of i.i.d. samples ~ `U(0.0, 1.0)`. */ - @Experimental - def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { + def uniformRDD( + sc: SparkContext, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { val uniform = new UniformGenerator() - randomRDD(sc, uniform, size, numPartitions, seed) + randomRDD(sc, uniform, size, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. - * - * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - * `RandomRDDGenerators.uniformRDD(sc, n, p).map(v => a + (b - a) * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + * Java-friendly version of [[RandomRDDs#uniformRDD]]. */ - @Experimental - def uniformRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { - uniformRDD(sc, size, numPartitions, Utils.random.nextLong) + def uniformJavaRDD( + jsc: JavaSparkContext, + size: Long, + numPartitions: Int, + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size, numPartitions, seed)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the uniform distribution on [0.0, 1.0]. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * To transform the distribution in the generated RDD from U[0.0, 1.0] to U[a, b], use - * `RandomRDDGenerators.uniformRDD(sc, n).map(v => a + (b - a) * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ U[0.0, 1.0]. + * [[RandomRDDs#uniformJavaRDD]] with the default seed. */ - @Experimental - def uniformRDD(sc: SparkContext, size: Long): RDD[Double] = { - uniformRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + def uniformJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size, numPartitions)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. - * - * To transform the distribution in the generated RDD from standard normal to some other normal - * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p, seed).map(v => mean + sigma * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + * [[RandomRDDs#uniformJavaRDD]] with the default number of partitions and the default seed. */ - @Experimental - def normalRDD(sc: SparkContext, size: Long, numPartitions: Int, seed: Long): RDD[Double] = { - val normal = new StandardNormalGenerator() - randomRDD(sc, normal, size, numPartitions, seed) + def uniformJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(uniformRDD(jsc.sc, size)) } /** - * :: Experimental :: * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. * * To transform the distribution in the generated RDD from standard normal to some other normal - * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n, p).map(v => mean + sigma * v)`. + * `N(mean, sigma^2^)`, use `RandomRDDs.normalRDD(sc, n, p, seed).map(v => mean + sigma * v)`. * * @param sc SparkContext used to create the RDD. * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). */ - @Experimental - def normalRDD(sc: SparkContext, size: Long, numPartitions: Int): RDD[Double] = { - normalRDD(sc, size, numPartitions, Utils.random.nextLong) + def normalRDD( + sc: SparkContext, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { + val normal = new StandardNormalGenerator() + randomRDD(sc, normal, size, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the standard normal distribution. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * To transform the distribution in the generated RDD from standard normal to some other normal - * N(mean, sigma), use `RandomRDDGenerators.normalRDD(sc, n).map(v => mean + sigma * v)`. - * - * @param sc SparkContext used to create the RDD. - * @param size Size of the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ N(0.0, 1.0). + * Java-friendly version of [[RandomRDDs#normalRDD]]. */ - @Experimental - def normalRDD(sc: SparkContext, size: Long): RDD[Double] = { - normalRDD(sc, size, sc.defaultParallelism, Utils.random.nextLong) + def normalJavaRDD( + jsc: JavaSparkContext, + size: Long, + numPartitions: Int, + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size, numPartitions, seed)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * [[RandomRDDs#normalJavaRDD]] with the default seed. */ - @Experimental - def poissonRDD(sc: SparkContext, - mean: Double, - size: Long, - numPartitions: Int, - seed: Long): RDD[Double] = { - val poisson = new PoissonGenerator(mean) - randomRDD(sc, poisson, size, numPartitions, seed) + def normalJavaRDD(jsc: JavaSparkContext, size: Long, numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size, numPartitions)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). + * [[RandomRDDs#normalJavaRDD]] with the default number of partitions and the default seed. */ - @Experimental - def poissonRDD(sc: SparkContext, mean: Double, size: Long, numPartitions: Int): RDD[Double] = { - poissonRDD(sc, mean, size, numPartitions, Utils.random.nextLong) + def normalJavaRDD(jsc: JavaSparkContext, size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(normalRDD(jsc.sc, size)) } /** - * :: Experimental :: * Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. - * sc.defaultParallelism used for the number of partitions in the RDD. * * @param sc SparkContext used to create the RDD. * @param mean Mean, or lambda, for the Poisson distribution. * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of i.i.d. samples ~ Pois(mean). */ - @Experimental - def poissonRDD(sc: SparkContext, mean: Double, size: Long): RDD[Double] = { - poissonRDD(sc, mean, size, sc.defaultParallelism, Utils.random.nextLong) + def poissonRDD( + sc: SparkContext, + mean: Double, + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Double] = { + val poisson = new PoissonGenerator(mean) + randomRDD(sc, poisson, size, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Double] comprised of i.i.d. samples produced by generator. + * Java-friendly version of [[RandomRDDs#poissonRDD]]. */ - @Experimental - def randomRDD[T: ClassTag](sc: SparkContext, - generator: RandomDataGenerator[T], + def poissonJavaRDD( + jsc: JavaSparkContext, + mean: Double, size: Long, numPartitions: Int, - seed: Long): RDD[T] = { - new RandomRDD[T](sc, size, numPartitions, generator, seed) + seed: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size, numPartitions, seed)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param size Size of the RDD. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Double] comprised of i.i.d. samples produced by generator. + * [[RandomRDDs#poissonJavaRDD]] with the default seed. */ - @Experimental - def randomRDD[T: ClassTag](sc: SparkContext, - generator: RandomDataGenerator[T], + def poissonJavaRDD( + jsc: JavaSparkContext, + mean: Double, size: Long, - numPartitions: Int): RDD[T] = { - randomRDD[T](sc, generator, size, numPartitions, Utils.random.nextLong) + numPartitions: Int): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size, numPartitions)) } /** - * :: Experimental :: - * Generates an RDD comprised of i.i.d. samples produced by the input DistributionGenerator. - * sc.defaultParallelism used for the number of partitions in the RDD. + * [[RandomRDDs#poissonJavaRDD]] with the default number of partitions and the default seed. + */ + def poissonJavaRDD(jsc: JavaSparkContext, mean: Double, size: Long): JavaDoubleRDD = { + JavaDoubleRDD.fromRDD(poissonRDD(jsc.sc, mean, size)) + } + + /** + * :: DeveloperApi :: + * Generates an RDD comprised of i.i.d. samples produced by the input RandomDataGenerator. * * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. + * @param generator RandomDataGenerator used to populate the RDD. * @param size Size of the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Double] comprised of i.i.d. samples produced by generator. */ - @Experimental - def randomRDD[T: ClassTag](sc: SparkContext, + @DeveloperApi + def randomRDD[T: ClassTag]( + sc: SparkContext, generator: RandomDataGenerator[T], - size: Long): RDD[T] = { - randomRDD[T](sc, generator, size, sc.defaultParallelism, Utils.random.nextLong) + size: Long, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[T] = { + new RandomRDD[T](sc, size, numPartitionsOrDefault(sc, numPartitions), generator, seed) } // TODO Generate RDD[Vector] from multivariate distributions. /** - * :: Experimental :: * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * uniform distribution on [0.0 1.0]. + * uniform distribution on `U(0.0, 1.0)`. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. * @param numPartitions Number of partitions in the RDD. * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d samples ~ U[0.0, 1.0]. + * @return RDD[Vector] with vectors containing i.i.d samples ~ `U(0.0, 1.0)`. */ - @Experimental - def uniformVectorRDD(sc: SparkContext, + def uniformVectorRDD( + sc: SparkContext, numRows: Long, numCols: Int, - numPartitions: Int, - seed: Long): RDD[Vector] = { + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { val uniform = new UniformGenerator() - randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + randomVectorRDD(sc, uniform, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * uniform distribution on [0.0 1.0]. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. + * Java-friendly version of [[RandomRDDs#uniformVectorRDD]]. */ - @Experimental - def uniformVectorRDD(sc: SparkContext, + def uniformJavaVectorRDD( + jsc: JavaSparkContext, numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - uniformVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int, + seed: Long): JavaRDD[Vector] = { + uniformVectorRDD(jsc.sc, numRows, numCols, numPartitions, seed).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * uniform distribution on [0.0 1.0]. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ U[0.0, 1.0]. + * [[RandomRDDs#uniformJavaVectorRDD]] with the default seed. */ - @Experimental - def uniformVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { - uniformVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + def uniformJavaVectorRDD( + jsc: JavaSparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int): JavaRDD[Vector] = { + uniformVectorRDD(jsc.sc, numRows, numCols, numPartitions).toJavaRDD() + } + + /** + * [[RandomRDDs#uniformJavaVectorRDD]] with the default number of partitions and the default seed. + */ + def uniformJavaVectorRDD( + jsc: JavaSparkContext, + numRows: Long, + numCols: Int): JavaRDD[Vector] = { + uniformVectorRDD(jsc.sc, numRows, numCols).toJavaRDD() } /** - * :: Experimental :: * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * standard normal distribution. * * @param sc SparkContext used to create the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). + * @return RDD[Vector] with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. + */ + def normalVectorRDD( + sc: SparkContext, + numRows: Long, + numCols: Int, + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { + val normal = new StandardNormalGenerator() + randomVectorRDD(sc, normal, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed) + } + + /** + * Java-friendly version of [[RandomRDDs#normalVectorRDD]]. */ - @Experimental - def normalVectorRDD(sc: SparkContext, + def normalJavaVectorRDD( + jsc: JavaSparkContext, numRows: Long, numCols: Int, numPartitions: Int, - seed: Long): RDD[Vector] = { - val uniform = new StandardNormalGenerator() - randomVectorRDD(sc, uniform, numRows, numCols, numPartitions, seed) + seed: Long): JavaRDD[Vector] = { + normalVectorRDD(jsc.sc, numRows, numCols, numPartitions, seed).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * standard normal distribution. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + * [[RandomRDDs#normalJavaVectorRDD]] with the default seed. */ - @Experimental - def normalVectorRDD(sc: SparkContext, + def normalJavaVectorRDD( + jsc: JavaSparkContext, numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - normalVectorRDD(sc, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int): JavaRDD[Vector] = { + normalVectorRDD(jsc.sc, numRows, numCols, numPartitions).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * standard normal distribution. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ N(0.0, 1.0). + * [[RandomRDDs#normalJavaVectorRDD]] with the default number of partitions and the default seed. */ - @Experimental - def normalVectorRDD(sc: SparkContext, numRows: Long, numCols: Int): RDD[Vector] = { - normalVectorRDD(sc, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + def normalJavaVectorRDD( + jsc: JavaSparkContext, + numRows: Long, + numCols: Int): JavaRDD[Vector] = { + normalVectorRDD(jsc.sc, numRows, numCols).toJavaRDD() } /** - * :: Experimental :: * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the * Poisson distribution with the input mean. * @@ -370,124 +315,85 @@ object RandomRDDs { * @param mean Mean, or lambda, for the Poisson distribution. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`) + * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). */ - @Experimental - def poissonVectorRDD(sc: SparkContext, + def poissonVectorRDD( + sc: SparkContext, mean: Double, numRows: Long, numCols: Int, - numPartitions: Int, - seed: Long): RDD[Vector] = { + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { val poisson = new PoissonGenerator(mean) - randomVectorRDD(sc, poisson, numRows, numCols, numPartitions, seed) + randomVectorRDD(sc, poisson, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), seed) } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * Poisson distribution with the input mean. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + * Java-friendly version of [[RandomRDDs#poissonVectorRDD]]. */ - @Experimental - def poissonVectorRDD(sc: SparkContext, + def poissonJavaVectorRDD( + jsc: JavaSparkContext, mean: Double, numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - poissonVectorRDD(sc, mean, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int, + seed: Long): JavaRDD[Vector] = { + poissonVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions, seed).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples drawn from the - * Poisson distribution with the input mean. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param mean Mean, or lambda, for the Poisson distribution. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples ~ Pois(mean). + * [[RandomRDDs#poissonJavaVectorRDD]] with the default seed. */ - @Experimental - def poissonVectorRDD(sc: SparkContext, + def poissonJavaVectorRDD( + jsc: JavaSparkContext, mean: Double, numRows: Long, - numCols: Int): RDD[Vector] = { - poissonVectorRDD(sc, mean, numRows, numCols, sc.defaultParallelism, Utils.random.nextLong) + numCols: Int, + numPartitions: Int): JavaRDD[Vector] = { + poissonVectorRDD(jsc.sc, mean, numRows, numCols, numPartitions).toJavaRDD() } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the - * input DistributionGenerator. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. - * @param seed Seed for the RNG that generates the seed for the generator in each partition. - * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + * [[RandomRDDs#poissonJavaVectorRDD]] with the default number of partitions and the default seed. */ - @Experimental - def randomVectorRDD(sc: SparkContext, - generator: RandomDataGenerator[Double], + def poissonJavaVectorRDD( + jsc: JavaSparkContext, + mean: Double, numRows: Long, - numCols: Int, - numPartitions: Int, - seed: Long): RDD[Vector] = { - new RandomVectorRDD(sc, numRows, numCols, numPartitions, generator, seed) + numCols: Int): JavaRDD[Vector] = { + poissonVectorRDD(jsc.sc, mean, numRows, numCols).toJavaRDD() } /** - * :: Experimental :: + * :: DeveloperApi :: * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the - * input DistributionGenerator. + * input RandomDataGenerator. * * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. + * @param generator RandomDataGenerator used to populate the RDD. * @param numRows Number of Vectors in the RDD. * @param numCols Number of elements in each Vector. - * @param numPartitions Number of partitions in the RDD. + * @param numPartitions Number of partitions in the RDD (default: `sc.defaultParallelism`). + * @param seed Random seed (default: a random long integer). * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. */ - @Experimental + @DeveloperApi def randomVectorRDD(sc: SparkContext, generator: RandomDataGenerator[Double], numRows: Long, numCols: Int, - numPartitions: Int): RDD[Vector] = { - randomVectorRDD(sc, generator, numRows, numCols, numPartitions, Utils.random.nextLong) + numPartitions: Int = 0, + seed: Long = Utils.random.nextLong()): RDD[Vector] = { + new RandomVectorRDD( + sc, numRows, numCols, numPartitionsOrDefault(sc, numPartitions), generator, seed) } /** - * :: Experimental :: - * Generates an RDD[Vector] with vectors containing i.i.d. samples produced by the - * input DistributionGenerator. - * sc.defaultParallelism used for the number of partitions in the RDD. - * - * @param sc SparkContext used to create the RDD. - * @param generator DistributionGenerator used to populate the RDD. - * @param numRows Number of Vectors in the RDD. - * @param numCols Number of elements in each Vector. - * @return RDD[Vector] with vectors containing i.i.d. samples produced by generator. + * Returns `numPartitions` if it is positive, or `sc.defaultParallelism` otherwise. */ - @Experimental - def randomVectorRDD(sc: SparkContext, - generator: RandomDataGenerator[Double], - numRows: Long, - numCols: Int): RDD[Vector] = { - randomVectorRDD(sc, generator, numRows, numCols, - sc.defaultParallelism, Utils.random.nextLong) + private def numPartitionsOrDefault(sc: SparkContext, numPartitions: Int): Int = { + if (numPartitions > 0) numPartitions else sc.defaultMinPartitions } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java new file mode 100644 index 0000000000000..a725736ca1a58 --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/random/JavaRandomRDDsSuite.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.random; + +import com.google.common.collect.Lists; +import org.apache.spark.api.java.JavaRDD; +import org.junit.Assert; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import static org.apache.spark.mllib.random.RandomRDDs.*; + +public class JavaRandomRDDsSuite { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaRandomRDDsSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void testUniformRDD() { + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = uniformJavaRDD(sc, m); + JavaDoubleRDD rdd2 = uniformJavaRDD(sc, m, p); + JavaDoubleRDD rdd3 = uniformJavaRDD(sc, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test + public void testNormalRDD() { + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = normalJavaRDD(sc, m); + JavaDoubleRDD rdd2 = normalJavaRDD(sc, m, p); + JavaDoubleRDD rdd3 = normalJavaRDD(sc, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test + public void testPoissonRDD() { + double mean = 2.0; + long m = 1000L; + int p = 2; + long seed = 1L; + JavaDoubleRDD rdd1 = poissonJavaRDD(sc, mean, m); + JavaDoubleRDD rdd2 = poissonJavaRDD(sc, mean, m, p); + JavaDoubleRDD rdd3 = poissonJavaRDD(sc, mean, m, p, seed); + for (JavaDoubleRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testUniformVectorRDD() { + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = uniformJavaVectorRDD(sc, m, n); + JavaRDD rdd2 = uniformJavaVectorRDD(sc, m, n, p); + JavaRDD rdd3 = uniformJavaVectorRDD(sc, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testNormalVectorRDD() { + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = normalJavaVectorRDD(sc, m, n); + JavaRDD rdd2 = normalJavaVectorRDD(sc, m, n, p); + JavaRDD rdd3 = normalJavaVectorRDD(sc, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } + + @Test + @SuppressWarnings("unchecked") + public void testPoissonVectorRDD() { + double mean = 2.0; + long m = 100L; + int n = 10; + int p = 2; + long seed = 1L; + JavaRDD rdd1 = poissonJavaVectorRDD(sc, mean, m, n); + JavaRDD rdd2 = poissonJavaVectorRDD(sc, mean, m, n, p); + JavaRDD rdd3 = poissonJavaVectorRDD(sc, mean, m, n, p, seed); + for (JavaRDD rdd: Lists.newArrayList(rdd1, rdd2, rdd3)) { + Assert.assertEquals(m, rdd.count()); + Assert.assertEquals(n, rdd.first().size()); + } + } +} diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 3f3b19053d32e..4dc1a4a912421 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -35,10 +35,10 @@ class RandomRDDs: def uniformRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the - uniform distribution on [0.0, 1.0]. + uniform distribution U(0.0, 1.0). - To transform the distribution in the generated RDD from U[0.0, 1.0] - to U[a, b], use + To transform the distribution in the generated RDD from U(0.0, 1.0) + to U(a, b), use C{RandomRDDs.uniformRDD(sc, n, p, seed)\ .map(lambda v: a + (b - a) * v)} @@ -60,11 +60,11 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): @staticmethod def normalRDD(sc, size, numPartitions=None, seed=None): """ - Generates an RDD comprised of i.i.d samples from the standard normal + Generates an RDD comprised of i.i.d. samples from the standard normal distribution. To transform the distribution in the generated RDD from standard normal - to some other normal N(mean, sigma), use + to some other normal N(mean, sigma^2), use C{RandomRDDs.normal(sc, n, p, seed)\ .map(lambda v: mean + sigma * v)} @@ -84,7 +84,7 @@ def normalRDD(sc, size, numPartitions=None, seed=None): @staticmethod def poissonRDD(sc, mean, size, numPartitions=None, seed=None): """ - Generates an RDD comprised of i.i.d samples from the Poisson + Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. >>> mean = 100.0 @@ -105,8 +105,8 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): @staticmethod def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ - Generates an RDD comprised of vectors containing i.i.d samples drawn - from the uniform distribution on [0.0 1.0]. + Generates an RDD comprised of vectors containing i.i.d. samples drawn + from the uniform distribution U(0.0, 1.0). >>> import numpy as np >>> mat = np.matrix(RandomRDDs.uniformVectorRDD(sc, 10, 10).collect()) @@ -125,7 +125,7 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): @staticmethod def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ - Generates an RDD comprised of vectors containing i.i.d samples drawn + Generates an RDD comprised of vectors containing i.i.d. samples drawn from the standard normal distribution. >>> import numpy as np @@ -145,7 +145,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): @staticmethod def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ - Generates an RDD comprised of vectors containing i.i.d samples drawn + Generates an RDD comprised of vectors containing i.i.d. samples drawn from the Poisson distribution with the input mean. >>> import numpy as np From 8b9dc991018842e01f4b93870a2bc2c2cb9ea4ba Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Aug 2014 17:40:35 -0700 Subject: [PATCH 0420/1492] [SPARK-2468] Netty based block server / client module Previous pull request (#1907) was reverted. This brings it back. Still looking into the hang. Author: Reynold Xin Closes #1971 from rxin/netty1 and squashes the following commits: b0be96f [Reynold Xin] Added test to make sure outstandingRequests are cleaned after firing the events. 4c6d0ee [Reynold Xin] Pass callbacks cleanly. 603dce7 [Reynold Xin] Upgrade Netty to 4.0.23 to fix the DefaultFileRegion bug. 88be1d4 [Reynold Xin] Downgrade to 4.0.21 to work around a bug in writing DefaultFileRegion. 002626a [Reynold Xin] Remove netty-test-file.txt. db6e6e0 [Reynold Xin] Revert "Revert "[SPARK-2468] Netty based block server / client module"" --- .../spark/network/netty/FileClient.scala | 85 --------- .../network/netty/FileClientHandler.scala | 50 ------ .../spark/network/netty/FileHeader.scala | 71 -------- .../spark/network/netty/FileServer.scala | 91 ---------- .../network/netty/FileServerHandler.scala | 68 -------- .../spark/network/netty/NettyConfig.scala | 59 +++++++ .../spark/network/netty/ShuffleCopier.scala | 118 ------------- .../spark/network/netty/ShuffleSender.scala | 71 -------- .../BlockClientListener.scala} | 16 +- .../netty/client/BlockFetchingClient.scala | 132 ++++++++++++++ .../client/BlockFetchingClientFactory.scala | 99 +++++++++++ .../client/BlockFetchingClientHandler.scala | 103 +++++++++++ .../netty/client/LazyInitIterator.scala | 44 +++++ .../netty/client/ReferenceCountedBuffer.scala | 47 +++++ .../network/netty/server/BlockHeader.scala | 32 ++++ .../netty/server/BlockHeaderEncoder.scala | 47 +++++ .../network/netty/server/BlockServer.scala | 162 ++++++++++++++++++ .../BlockServerChannelInitializer.scala} | 22 ++- .../netty/server/BlockServerHandler.scala | 140 +++++++++++++++ .../spark/storage/BlockDataProvider.scala | 32 ++++ .../spark/storage/BlockFetcherIterator.scala | 137 +++++++-------- .../apache/spark/storage/BlockManager.scala | 49 +++++- .../storage/BlockNotFoundException.scala | 21 +++ .../spark/storage/DiskBlockManager.scala | 13 +- .../netty/ServerClientIntegrationSuite.scala | 161 +++++++++++++++++ .../BlockFetchingClientHandlerSuite.scala | 105 ++++++++++++ .../server/BlockHeaderEncoderSuite.scala | 64 +++++++ .../server/BlockServerHandlerSuite.scala | 107 ++++++++++++ pom.xml | 2 +- 29 files changed, 1484 insertions(+), 664 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClient.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala rename core/src/main/scala/org/apache/spark/network/netty/{FileClientChannelInitializer.scala => client/BlockClientListener.scala} (65%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala rename core/src/main/scala/org/apache/spark/network/netty/{FileServerChannelInitializer.scala => server/BlockServerChannelInitializer.scala} (58%) create mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala deleted file mode 100644 index c6d35f73db545..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClient.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.util.concurrent.TimeUnit - -import io.netty.bootstrap.Bootstrap -import io.netty.channel.{Channel, ChannelOption, EventLoopGroup} -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.oio.OioSocketChannel - -import org.apache.spark.Logging - -class FileClient(handler: FileClientHandler, connectTimeout: Int) extends Logging { - - private var channel: Channel = _ - private var bootstrap: Bootstrap = _ - private var group: EventLoopGroup = _ - private val sendTimeout = 60 - - def init(): Unit = { - group = new OioEventLoopGroup - bootstrap = new Bootstrap - bootstrap.group(group) - .channel(classOf[OioSocketChannel]) - .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) - .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, Integer.valueOf(connectTimeout)) - .handler(new FileClientChannelInitializer(handler)) - } - - def connect(host: String, port: Int) { - try { - channel = bootstrap.connect(host, port).sync().channel() - } catch { - case e: InterruptedException => - logWarning("FileClient interrupted while trying to connect", e) - close() - } - } - - def waitForClose(): Unit = { - try { - channel.closeFuture.sync() - } catch { - case e: InterruptedException => - logWarning("FileClient interrupted", e) - } - } - - def sendRequest(file: String): Unit = { - try { - val bSent = channel.writeAndFlush(file + "\r\n").await(sendTimeout, TimeUnit.SECONDS) - if (!bSent) { - throw new RuntimeException("Failed to send") - } - } catch { - case e: InterruptedException => - logError("Error", e) - } - } - - def close(): Unit = { - if (group != null) { - group.shutdownGracefully() - group = null - bootstrap = null - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala deleted file mode 100644 index 017302ec7d33d..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClientHandler.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import io.netty.buffer.ByteBuf -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.storage.BlockId - - -abstract class FileClientHandler extends SimpleChannelInboundHandler[ByteBuf] { - - private var currentHeader: FileHeader = null - - @volatile - private var handlerCalled: Boolean = false - - def isComplete: Boolean = handlerCalled - - def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) - - def handleError(blockId: BlockId) - - override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { - if (currentHeader == null && in.readableBytes >= FileHeader.HEADER_SIZE) { - currentHeader = FileHeader.create(in.readBytes(FileHeader.HEADER_SIZE)) - } - if (in.readableBytes >= currentHeader.fileLen) { - handle(ctx, in, currentHeader) - handlerCalled = true - currentHeader = null - ctx.close() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala deleted file mode 100644 index 607e560ff277f..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import io.netty.buffer._ - -import org.apache.spark.Logging -import org.apache.spark.storage.{BlockId, TestBlockId} - -private[spark] class FileHeader ( - val fileLen: Int, - val blockId: BlockId) extends Logging { - - lazy val buffer: ByteBuf = { - val buf = Unpooled.buffer() - buf.capacity(FileHeader.HEADER_SIZE) - buf.writeInt(fileLen) - buf.writeInt(blockId.name.length) - blockId.name.foreach((x: Char) => buf.writeByte(x)) - // padding the rest of header - if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { - buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) - } else { - throw new Exception("too long header " + buf.readableBytes) - logInfo("too long header") - } - buf - } - -} - -private[spark] object FileHeader { - - val HEADER_SIZE = 40 - - def getFileLenOffset = 0 - def getFileLenSize = Integer.SIZE/8 - - def create(buf: ByteBuf): FileHeader = { - val length = buf.readInt - val idLength = buf.readInt - val idBuilder = new StringBuilder(idLength) - for (i <- 1 to idLength) { - idBuilder += buf.readByte().asInstanceOf[Char] - } - val blockId = BlockId(idBuilder.toString()) - new FileHeader(length, blockId) - } - - def main(args:Array[String]) { - val header = new FileHeader(25, TestBlockId("my_block")) - val buf = header.buffer - val newHeader = FileHeader.create(buf) - System.out.println("id=" + newHeader.blockId + ",size=" + newHeader.fileLen) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala deleted file mode 100644 index dff77950659af..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServer.scala +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.net.InetSocketAddress - -import io.netty.bootstrap.ServerBootstrap -import io.netty.channel.{ChannelFuture, ChannelOption, EventLoopGroup} -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.oio.OioServerSocketChannel - -import org.apache.spark.Logging - -/** - * Server that accept the path of a file an echo back its content. - */ -class FileServer(pResolver: PathResolver, private var port: Int) extends Logging { - - private val addr: InetSocketAddress = new InetSocketAddress(port) - private var bossGroup: EventLoopGroup = new OioEventLoopGroup - private var workerGroup: EventLoopGroup = new OioEventLoopGroup - - private var channelFuture: ChannelFuture = { - val bootstrap = new ServerBootstrap - bootstrap.group(bossGroup, workerGroup) - .channel(classOf[OioServerSocketChannel]) - .option(ChannelOption.SO_BACKLOG, java.lang.Integer.valueOf(100)) - .option(ChannelOption.SO_RCVBUF, java.lang.Integer.valueOf(1500)) - .childHandler(new FileServerChannelInitializer(pResolver)) - bootstrap.bind(addr) - } - - try { - val boundAddress = channelFuture.sync.channel.localAddress.asInstanceOf[InetSocketAddress] - port = boundAddress.getPort - } catch { - case ie: InterruptedException => - port = 0 - } - - /** Start the file server asynchronously in a new thread. */ - def start(): Unit = { - val blockingThread: Thread = new Thread { - override def run(): Unit = { - try { - channelFuture.channel.closeFuture.sync - logInfo("FileServer exiting") - } catch { - case e: InterruptedException => - logError("File server start got interrupted", e) - } - // NOTE: bootstrap is shutdown in stop() - } - } - blockingThread.setDaemon(true) - blockingThread.start() - } - - def getPort: Int = port - - def stop(): Unit = { - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly() - channelFuture = null - } - if (bossGroup != null) { - bossGroup.shutdownGracefully() - bossGroup = null - } - if (workerGroup != null) { - workerGroup.shutdownGracefully() - workerGroup = null - } - } -} - diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala deleted file mode 100644 index 96f60b2883ad9..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServerHandler.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.FileInputStream - -import io.netty.channel.{DefaultFileRegion, ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.Logging -import org.apache.spark.storage.{BlockId, FileSegment} - - -class FileServerHandler(pResolver: PathResolver) - extends SimpleChannelInboundHandler[String] with Logging { - - override def channelRead0(ctx: ChannelHandlerContext, blockIdString: String): Unit = { - val blockId: BlockId = BlockId(blockIdString) - val fileSegment: FileSegment = pResolver.getBlockLocation(blockId) - if (fileSegment == null) { - return - } - val file = fileSegment.file - if (file.exists) { - if (!file.isFile) { - ctx.write(new FileHeader(0, blockId).buffer) - ctx.flush() - return - } - val length: Long = fileSegment.length - if (length > Integer.MAX_VALUE || length <= 0) { - ctx.write(new FileHeader(0, blockId).buffer) - ctx.flush() - return - } - ctx.write(new FileHeader(length.toInt, blockId).buffer) - try { - val channel = new FileInputStream(file).getChannel - ctx.write(new DefaultFileRegion(channel, fileSegment.offset, fileSegment.length)) - } catch { - case e: Exception => - logError("Exception: ", e) - } - } else { - ctx.write(new FileHeader(0, blockId).buffer) - } - ctx.flush() - } - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - logError("Exception: ", cause) - ctx.close() - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala new file mode 100644 index 0000000000000..b5870152c5a64 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import org.apache.spark.SparkConf + +/** + * A central location that tracks all the settings we exposed to users. + */ +private[spark] +class NettyConfig(conf: SparkConf) { + + /** Port the server listens on. Default to a random port. */ + private[netty] val serverPort = conf.getInt("spark.shuffle.io.port", 0) + + /** IO mode: nio, oio, epoll, or auto (try epoll first and then nio). */ + private[netty] val ioMode = conf.get("spark.shuffle.io.mode", "nio").toLowerCase + + /** Connect timeout in secs. Default 60 secs. */ + private[netty] val connectTimeoutMs = conf.getInt("spark.shuffle.io.connectionTimeout", 60) * 1000 + + /** + * Percentage of the desired amount of time spent for I/O in the child event loops. + * Only applicable in nio and epoll. + */ + private[netty] val ioRatio = conf.getInt("spark.shuffle.io.netty.ioRatio", 80) + + /** Requested maximum length of the queue of incoming connections. */ + private[netty] val backLog: Option[Int] = conf.getOption("spark.shuffle.io.backLog").map(_.toInt) + + /** + * Receive buffer size (SO_RCVBUF). + * Note: the optimal size for receive buffer and send buffer should be + * latency * network_bandwidth. + * Assuming latency = 1ms, network_bandwidth = 10Gbps + * buffer size should be ~ 1.25MB + */ + private[netty] val receiveBuf: Option[Int] = + conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) + + /** Send buffer size (SO_SNDBUF). */ + private[netty] val sendBuf: Option[Int] = + conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala deleted file mode 100644 index e7b2855e1ec91..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleCopier.scala +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.util.concurrent.Executors - -import scala.collection.JavaConverters._ - -import io.netty.buffer.ByteBuf -import io.netty.channel.ChannelHandlerContext -import io.netty.util.CharsetUtil - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.storage.BlockId - -private[spark] class ShuffleCopier(conf: SparkConf) extends Logging { - - def getBlock(host: String, port: Int, blockId: BlockId, - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - - val handler = new ShuffleCopier.ShuffleClientHandler(resultCollectCallback) - val connectTimeout = conf.getInt("spark.shuffle.netty.connect.timeout", 60000) - val fc = new FileClient(handler, connectTimeout) - - try { - fc.init() - fc.connect(host, port) - fc.sendRequest(blockId.name) - fc.waitForClose() - fc.close() - } catch { - // Handle any socket-related exceptions in FileClient - case e: Exception => { - logError("Shuffle copy of block " + blockId + " from " + host + ":" + port + " failed", e) - handler.handleError(blockId) - } - } - } - - def getBlock(cmId: ConnectionManagerId, blockId: BlockId, - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - getBlock(cmId.host, cmId.port, blockId, resultCollectCallback) - } - - def getBlocks(cmId: ConnectionManagerId, - blocks: Seq[(BlockId, Long)], - resultCollectCallback: (BlockId, Long, ByteBuf) => Unit) { - - for ((blockId, size) <- blocks) { - getBlock(cmId, blockId, resultCollectCallback) - } - } -} - - -private[spark] object ShuffleCopier extends Logging { - - private class ShuffleClientHandler(resultCollectCallBack: (BlockId, Long, ByteBuf) => Unit) - extends FileClientHandler with Logging { - - override def handle(ctx: ChannelHandlerContext, in: ByteBuf, header: FileHeader) { - logDebug("Received Block: " + header.blockId + " (" + header.fileLen + "B)") - resultCollectCallBack(header.blockId, header.fileLen.toLong, in.readBytes(header.fileLen)) - } - - override def handleError(blockId: BlockId) { - if (!isComplete) { - resultCollectCallBack(blockId, -1, null) - } - } - } - - def echoResultCollectCallBack(blockId: BlockId, size: Long, content: ByteBuf) { - if (size != -1) { - logInfo("File: " + blockId + " content is : \" " + content.toString(CharsetUtil.UTF_8) + "\"") - } - } - - def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: ShuffleCopier ") - System.exit(1) - } - val host = args(0) - val port = args(1).toInt - val blockId = BlockId(args(2)) - val threads = if (args.length > 3) args(3).toInt else 10 - - val copiers = Executors.newFixedThreadPool(80) - val tasks = (for (i <- Range(0, threads)) yield { - Executors.callable(new Runnable() { - def run() { - val copier = new ShuffleCopier(new SparkConf) - copier.getBlock(host, port, blockId, echoResultCollectCallBack) - } - }) - }).asJava - copiers.invokeAll(tasks) - copiers.shutdown() - System.exit(0) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala b/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala deleted file mode 100644 index 95958e30f7eeb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/ShuffleSender.scala +++ /dev/null @@ -1,71 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.File - -import org.apache.spark.Logging -import org.apache.spark.util.Utils -import org.apache.spark.storage.{BlockId, FileSegment} - -private[spark] class ShuffleSender(portIn: Int, val pResolver: PathResolver) extends Logging { - - val server = new FileServer(pResolver, portIn) - server.start() - - def stop() { - server.stop() - } - - def port: Int = server.getPort -} - - -/** - * An application for testing the shuffle sender as a standalone program. - */ -private[spark] object ShuffleSender { - - def main(args: Array[String]) { - if (args.length < 3) { - System.err.println( - "Usage: ShuffleSender ") - System.exit(1) - } - - val port = args(0).toInt - val subDirsPerLocalDir = args(1).toInt - val localDirs = args.drop(2).map(new File(_)) - - val pResovler = new PathResolver { - override def getBlockLocation(blockId: BlockId): FileSegment = { - if (!blockId.isShuffle) { - throw new Exception("Block " + blockId + " is not a shuffle block") - } - // Figure out which local directory it hashes to, and which subdirectory in that - val hash = Utils.nonNegativeHash(blockId) - val dirId = hash % localDirs.length - val subDirId = (hash / localDirs.length) % subDirsPerLocalDir - val subDir = new File(localDirs(dirId), "%02x".format(subDirId)) - val file = new File(subDir, blockId.name) - new FileSegment(file, 0, file.length()) - } - } - val sender = new ShuffleSender(port, pResovler) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala similarity index 65% rename from core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala rename to core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala index f4261c13f70a8..e28219dd7745b 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileClientChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala @@ -15,17 +15,15 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.network.netty.client -import io.netty.channel.ChannelInitializer -import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.string.StringEncoder +import java.util.EventListener -class FileClientChannelInitializer(handler: FileClientHandler) - extends ChannelInitializer[SocketChannel] { +trait BlockClientListener extends EventListener { + + def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit + + def onFetchFailure(blockId: String, errorMsg: String): Unit - def initChannel(channel: SocketChannel) { - channel.pipeline.addLast("encoder", new StringEncoder).addLast("handler", handler) - } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala new file mode 100644 index 0000000000000..5aea7ba2f3673 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.util.concurrent.TimeoutException + +import io.netty.bootstrap.Bootstrap +import io.netty.buffer.PooledByteBufAllocator +import io.netty.channel.socket.SocketChannel +import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} +import io.netty.handler.codec.LengthFieldBasedFrameDecoder +import io.netty.handler.codec.string.StringEncoder +import io.netty.util.CharsetUtil + +import org.apache.spark.Logging + +/** + * Client for fetching data blocks from [[org.apache.spark.network.netty.server.BlockServer]]. + * Use [[BlockFetchingClientFactory]] to instantiate this client. + * + * The constructor blocks until a connection is successfully established. + * + * See [[org.apache.spark.network.netty.server.BlockServer]] for client/server protocol. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +@throws[TimeoutException] +private[spark] +class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, port: Int) + extends Logging { + + private val handler = new BlockFetchingClientHandler + + /** Netty Bootstrap for creating the TCP connection. */ + private val bootstrap: Bootstrap = { + val b = new Bootstrap + b.group(factory.workerGroup) + .channel(factory.socketChannelClass) + // Use pooled buffers to reduce temporary buffer allocation + .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + // Disable Nagle's Algorithm since we don't want packets to wait + .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) + .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) + .option[Integer](ChannelOption.CONNECT_TIMEOUT_MILLIS, factory.conf.connectTimeoutMs) + + b.handler(new ChannelInitializer[SocketChannel] { + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("encoder", new StringEncoder(CharsetUtil.UTF_8)) + // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 + .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) + .addLast("handler", handler) + } + }) + b + } + + /** Netty ChannelFuture for the connection. */ + private val cf: ChannelFuture = bootstrap.connect(hostname, port) + if (!cf.awaitUninterruptibly(factory.conf.connectTimeoutMs)) { + throw new TimeoutException( + s"Connecting to $hostname:$port timed out (${factory.conf.connectTimeoutMs} ms)") + } + + /** + * Ask the remote server for a sequence of blocks, and execute the callback. + * + * Note that this is asynchronous and returns immediately. Upstream caller should throttle the + * rate of fetching; otherwise we could run out of memory. + * + * @param blockIds sequence of block ids to fetch. + * @param listener callback to fire on fetch success / failure. + */ + def fetchBlocks(blockIds: Seq[String], listener: BlockClientListener): Unit = { + // It's best to limit the number of "write" calls since it needs to traverse the whole pipeline. + // It's also best to limit the number of "flush" calls since it requires system calls. + // Let's concatenate the string and then call writeAndFlush once. + // This is also why this implementation might be more efficient than multiple, separate + // fetch block calls. + var startTime: Long = 0 + logTrace { + startTime = System.nanoTime + s"Sending request $blockIds to $hostname:$port" + } + + blockIds.foreach { blockId => + handler.addRequest(blockId, listener) + } + + val writeFuture = cf.channel().writeAndFlush(blockIds.mkString("\n") + "\n") + writeFuture.addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture): Unit = { + if (future.isSuccess) { + logTrace { + val timeTaken = (System.nanoTime - startTime).toDouble / 1000000 + s"Sending request $blockIds to $hostname:$port took $timeTaken ms" + } + } else { + // Fail all blocks. + val errorMsg = + s"Failed to send request $blockIds to $hostname:$port: ${future.cause.getMessage}" + logError(errorMsg, future.cause) + blockIds.foreach { blockId => + listener.onFetchFailure(blockId, errorMsg) + handler.removeRequest(blockId) + } + } + } + }) + } + + def waitForClose(): Unit = { + cf.channel().closeFuture().sync() + } + + def close(): Unit = cf.channel().close() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala new file mode 100644 index 0000000000000..2b28402c52b49 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import io.netty.channel.epoll.{EpollEventLoopGroup, EpollSocketChannel} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.nio.NioSocketChannel +import io.netty.channel.socket.oio.OioSocketChannel +import io.netty.channel.{EventLoopGroup, Channel} + +import org.apache.spark.SparkConf +import org.apache.spark.network.netty.NettyConfig +import org.apache.spark.util.Utils + +/** + * Factory for creating [[BlockFetchingClient]] by using createClient. This factory reuses + * the worker thread pool for Netty. + * + * Concurrency: createClient is safe to be called from multiple threads concurrently. + */ +private[spark] +class BlockFetchingClientFactory(val conf: NettyConfig) { + + def this(sparkConf: SparkConf) = this(new NettyConfig(sparkConf)) + + /** A thread factory so the threads are named (for debugging). */ + val threadFactory = Utils.namedThreadFactory("spark-shuffle-client") + + /** The following two are instantiated by the [[init]] method, depending ioMode. */ + var socketChannelClass: Class[_ <: Channel] = _ + var workerGroup: EventLoopGroup = _ + + init() + + /** Initialize [[socketChannelClass]] and [[workerGroup]] based on ioMode. */ + private def init(): Unit = { + def initOio(): Unit = { + socketChannelClass = classOf[OioSocketChannel] + workerGroup = new OioEventLoopGroup(0, threadFactory) + } + def initNio(): Unit = { + socketChannelClass = classOf[NioSocketChannel] + workerGroup = new NioEventLoopGroup(0, threadFactory) + } + def initEpoll(): Unit = { + socketChannelClass = classOf[EpollSocketChannel] + workerGroup = new EpollEventLoopGroup(0, threadFactory) + } + + conf.ioMode match { + case "nio" => initNio() + case "oio" => initOio() + case "epoll" => initEpoll() + case "auto" => + // For auto mode, first try epoll (only available on Linux), then nio. + try { + initEpoll() + } catch { + // TODO: Should we log the throwable? But that always happen on non-Linux systems. + // Perhaps the right thing to do is to check whether the system is Linux, and then only + // call initEpoll on Linux. + case e: Throwable => initNio() + } + } + } + + /** + * Create a new BlockFetchingClient connecting to the given remote host / port. + * + * This blocks until a connection is successfully established. + * + * Concurrency: This method is safe to call from multiple threads. + */ + def createClient(remoteHost: String, remotePort: Int): BlockFetchingClient = { + new BlockFetchingClient(this, remoteHost, remotePort) + } + + def stop(): Unit = { + if (workerGroup != null) { + workerGroup.shutdownGracefully() + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala new file mode 100644 index 0000000000000..83265b164299d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import io.netty.buffer.ByteBuf +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging + + +/** + * Handler that processes server responses. It uses the protocol documented in + * [[org.apache.spark.network.netty.server.BlockServer]]. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +private[client] +class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] with Logging { + + /** Tracks the list of outstanding requests and their listeners on success/failure. */ + private val outstandingRequests = java.util.Collections.synchronizedMap { + new java.util.HashMap[String, BlockClientListener] + } + + def addRequest(blockId: String, listener: BlockClientListener): Unit = { + outstandingRequests.put(blockId, listener) + } + + def removeRequest(blockId: String): Unit = { + outstandingRequests.remove(blockId) + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + val errorMsg = s"Exception in connection from ${ctx.channel.remoteAddress}: ${cause.getMessage}" + logError(errorMsg, cause) + + // Fire the failure callback for all outstanding blocks + outstandingRequests.synchronized { + val iter = outstandingRequests.entrySet().iterator() + while (iter.hasNext) { + val entry = iter.next() + entry.getValue.onFetchFailure(entry.getKey, errorMsg) + } + outstandingRequests.clear() + } + + ctx.close() + } + + override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { + val totalLen = in.readInt() + val blockIdLen = in.readInt() + val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) + in.readBytes(blockIdBytes) + val blockId = new String(blockIdBytes) + val blockSize = totalLen - math.abs(blockIdLen) - 4 + + def server = ctx.channel.remoteAddress.toString + + // blockIdLen is negative when it is an error message. + if (blockIdLen < 0) { + val errorMessageBytes = new Array[Byte](blockSize) + in.readBytes(errorMessageBytes) + val errorMsg = new String(errorMessageBytes) + logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") + + val listener = outstandingRequests.get(blockId) + if (listener == null) { + // Ignore callback + logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") + } else { + outstandingRequests.remove(blockId) + listener.onFetchFailure(blockId, errorMsg) + } + } else { + logTrace(s"Received block $blockId ($blockSize B) from $server") + + val listener = outstandingRequests.get(blockId) + if (listener == null) { + // Ignore callback + logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") + } else { + outstandingRequests.remove(blockId) + listener.onFetchSuccess(blockId, new ReferenceCountedBuffer(in)) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala new file mode 100644 index 0000000000000..9740ee64d1f2d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +/** + * A simple iterator that lazily initializes the underlying iterator. + * + * The use case is that sometimes we might have many iterators open at the same time, and each of + * the iterator might initialize its own buffer (e.g. decompression buffer, deserialization buffer). + * This could lead to too many buffers open. If this iterator is used, we lazily initialize those + * buffers. + */ +private[spark] +class LazyInitIterator(createIterator: => Iterator[Any]) extends Iterator[Any] { + + lazy val proxy = createIterator + + override def hasNext: Boolean = { + val gotNext = proxy.hasNext + if (!gotNext) { + close() + } + gotNext + } + + override def next(): Any = proxy.next() + + def close(): Unit = Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala new file mode 100644 index 0000000000000..ea1abf5eccc26 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.io.InputStream +import java.nio.ByteBuffer + +import io.netty.buffer.{ByteBuf, ByteBufInputStream} + + +/** + * A buffer abstraction based on Netty's ByteBuf so we don't expose Netty. + * This is a Scala value class. + * + * The buffer's life cycle is NOT managed by the JVM, and thus requiring explicit declaration of + * reference by the retain method and release method. + */ +private[spark] +class ReferenceCountedBuffer(val underlying: ByteBuf) extends AnyVal { + + /** Return the nio ByteBuffer view of the underlying buffer. */ + def byteBuffer(): ByteBuffer = underlying.nioBuffer + + /** Creates a new input stream that starts from the current position of the buffer. */ + def inputStream(): InputStream = new ByteBufInputStream(underlying) + + /** Increment the reference counter by one. */ + def retain(): Unit = underlying.retain() + + /** Decrement the reference counter by one and release the buffer if the ref count is 0. */ + def release(): Unit = underlying.release() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala new file mode 100644 index 0000000000000..162e9cc6828d4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +/** + * Header describing a block. This is used only in the server pipeline. + * + * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it. + * + * @param blockSize length of the block content, excluding the length itself. + * If positive, this is the header for a block (not part of the header). + * If negative, this is the header and content for an error message. + * @param blockId block id + * @param error some error message from reading the block + */ +private[server] +class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None) diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala new file mode 100644 index 0000000000000..8e4dda4ef8595 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import io.netty.buffer.ByteBuf +import io.netty.channel.ChannelHandlerContext +import io.netty.handler.codec.MessageToByteEncoder + +/** + * A simple encoder for BlockHeader. See [[BlockServer]] for the server to client protocol. + */ +private[server] +class BlockHeaderEncoder extends MessageToByteEncoder[BlockHeader] { + override def encode(ctx: ChannelHandlerContext, msg: BlockHeader, out: ByteBuf): Unit = { + // message = message length (4 bytes) + block id length (4 bytes) + block id + block data + // message length = block id length (4 bytes) + size of block id + size of block data + val blockIdBytes = msg.blockId.getBytes + msg.error match { + case Some(errorMsg) => + val errorBytes = errorMsg.getBytes + out.writeInt(4 + blockIdBytes.length + errorBytes.size) + out.writeInt(-blockIdBytes.length) // use negative block id length to represent errors + out.writeBytes(blockIdBytes) // next is blockId itself + out.writeBytes(errorBytes) // error message + case None => + out.writeInt(4 + blockIdBytes.length + msg.blockSize) + out.writeInt(blockIdBytes.length) // First 4 bytes is blockId length + out.writeBytes(blockIdBytes) // next is blockId itself + // msg of size blockSize will be written by ServerHandler + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala new file mode 100644 index 0000000000000..7b2f9a8d4dfd0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.net.InetSocketAddress + +import io.netty.bootstrap.ServerBootstrap +import io.netty.buffer.PooledByteBufAllocator +import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} +import io.netty.channel.epoll.{EpollEventLoopGroup, EpollServerSocketChannel} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.oio.OioEventLoopGroup +import io.netty.channel.socket.SocketChannel +import io.netty.channel.socket.nio.NioServerSocketChannel +import io.netty.channel.socket.oio.OioServerSocketChannel +import io.netty.handler.codec.LineBasedFrameDecoder +import io.netty.handler.codec.string.StringDecoder +import io.netty.util.CharsetUtil + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.network.netty.NettyConfig +import org.apache.spark.storage.BlockDataProvider +import org.apache.spark.util.Utils + + +/** + * Server for serving Spark data blocks. + * This should be used together with [[org.apache.spark.network.netty.client.BlockFetchingClient]]. + * + * Protocol for requesting blocks (client to server): + * One block id per line, e.g. to request 3 blocks: "block1\nblock2\nblock3\n" + * + * Protocol for sending blocks (server to client): + * frame-length (4 bytes), block-id-length (4 bytes), block-id, block-data. + * + * frame-length should not include the length of itself. + * If block-id-length is negative, then this is an error message rather than block-data. The real + * length is the absolute value of the frame-length. + * + */ +private[spark] +class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Logging { + + def this(sparkConf: SparkConf, dataProvider: BlockDataProvider) = { + this(new NettyConfig(sparkConf), dataProvider) + } + + def port: Int = _port + + def hostName: String = _hostName + + private var _port: Int = conf.serverPort + private var _hostName: String = "" + private var bootstrap: ServerBootstrap = _ + private var channelFuture: ChannelFuture = _ + + init() + + /** Initialize the server. */ + private def init(): Unit = { + bootstrap = new ServerBootstrap + val bossThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-boss") + val workerThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-worker") + + // Use only one thread to accept connections, and 2 * num_cores for worker. + def initNio(): Unit = { + val bossGroup = new NioEventLoopGroup(1, bossThreadFactory) + val workerGroup = new NioEventLoopGroup(0, workerThreadFactory) + workerGroup.setIoRatio(conf.ioRatio) + bootstrap.group(bossGroup, workerGroup).channel(classOf[NioServerSocketChannel]) + } + def initOio(): Unit = { + val bossGroup = new OioEventLoopGroup(1, bossThreadFactory) + val workerGroup = new OioEventLoopGroup(0, workerThreadFactory) + bootstrap.group(bossGroup, workerGroup).channel(classOf[OioServerSocketChannel]) + } + def initEpoll(): Unit = { + val bossGroup = new EpollEventLoopGroup(1, bossThreadFactory) + val workerGroup = new EpollEventLoopGroup(0, workerThreadFactory) + workerGroup.setIoRatio(conf.ioRatio) + bootstrap.group(bossGroup, workerGroup).channel(classOf[EpollServerSocketChannel]) + } + + conf.ioMode match { + case "nio" => initNio() + case "oio" => initOio() + case "epoll" => initEpoll() + case "auto" => + // For auto mode, first try epoll (only available on Linux), then nio. + try { + initEpoll() + } catch { + // TODO: Should we log the throwable? But that always happen on non-Linux systems. + // Perhaps the right thing to do is to check whether the system is Linux, and then only + // call initEpoll on Linux. + case e: Throwable => initNio() + } + } + + // Use pooled buffers to reduce temporary buffer allocation + bootstrap.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + + // Various (advanced) user-configured settings. + conf.backLog.foreach { backLog => + bootstrap.option[java.lang.Integer](ChannelOption.SO_BACKLOG, backLog) + } + conf.receiveBuf.foreach { receiveBuf => + bootstrap.option[java.lang.Integer](ChannelOption.SO_RCVBUF, receiveBuf) + } + conf.sendBuf.foreach { sendBuf => + bootstrap.option[java.lang.Integer](ChannelOption.SO_SNDBUF, sendBuf) + } + + bootstrap.childHandler(new ChannelInitializer[SocketChannel] { + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 + .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("blockHeaderEncoder", new BlockHeaderEncoder) + .addLast("handler", new BlockServerHandler(dataProvider)) + } + }) + + channelFuture = bootstrap.bind(new InetSocketAddress(_port)) + channelFuture.sync() + + val addr = channelFuture.channel.localAddress.asInstanceOf[InetSocketAddress] + _port = addr.getPort + _hostName = addr.getHostName + } + + /** Shutdown the server. */ + def stop(): Unit = { + if (channelFuture != null) { + channelFuture.channel().close().awaitUninterruptibly() + channelFuture = null + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully() + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully() + } + bootstrap = null + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala similarity index 58% rename from core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala rename to core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala index aaa2f913d0269..cc70bd0c5c477 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileServerChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala @@ -15,20 +15,26 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.network.netty.server import io.netty.channel.ChannelInitializer import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.{DelimiterBasedFrameDecoder, Delimiters} +import io.netty.handler.codec.LineBasedFrameDecoder import io.netty.handler.codec.string.StringDecoder +import io.netty.util.CharsetUtil +import org.apache.spark.storage.BlockDataProvider -class FileServerChannelInitializer(pResolver: PathResolver) + +/** Channel initializer that sets up the pipeline for the BlockServer. */ +private[netty] +class BlockServerChannelInitializer(dataProvider: BlockDataProvider) extends ChannelInitializer[SocketChannel] { - override def initChannel(channel: SocketChannel): Unit = { - channel.pipeline - .addLast("framer", new DelimiterBasedFrameDecoder(8192, Delimiters.lineDelimiter : _*)) - .addLast("stringDecoder", new StringDecoder) - .addLast("handler", new FileServerHandler(pResolver)) + override def initChannel(ch: SocketChannel): Unit = { + ch.pipeline + .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 + .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("blockHeaderEncoder", new BlockHeaderEncoder) + .addLast("handler", new BlockServerHandler(dataProvider)) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala new file mode 100644 index 0000000000000..40dd5e5d1a2ac --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.io.FileInputStream +import java.nio.ByteBuffer +import java.nio.channels.FileChannel + +import io.netty.buffer.Unpooled +import io.netty.channel._ + +import org.apache.spark.Logging +import org.apache.spark.storage.{FileSegment, BlockDataProvider} + + +/** + * A handler that processes requests from clients and writes block data back. + * + * The messages should have been processed by a LineBasedFrameDecoder and a StringDecoder first + * so channelRead0 is called once per line (i.e. per block id). + */ +private[server] +class BlockServerHandler(dataProvider: BlockDataProvider) + extends SimpleChannelInboundHandler[String] with Logging { + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) + ctx.close() + } + + override def channelRead0(ctx: ChannelHandlerContext, blockId: String): Unit = { + def client = ctx.channel.remoteAddress.toString + + // A helper function to send error message back to the client. + def respondWithError(error: String): Unit = { + ctx.writeAndFlush(new BlockHeader(-1, blockId, Some(error))).addListener( + new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (!future.isSuccess) { + // TODO: Maybe log the success case as well. + logError(s"Error sending error back to $client", future.cause) + ctx.close() + } + } + } + ) + } + + def writeFileSegment(segment: FileSegment): Unit = { + // Send error message back if the block is too large. Even though we are capable of sending + // large (2G+) blocks, the receiving end cannot handle it so let's fail fast. + // Once we fixed the receiving end to be able to process large blocks, this should be removed. + // Also make sure we update BlockHeaderEncoder to support length > 2G. + + // See [[BlockHeaderEncoder]] for the way length is encoded. + if (segment.length + blockId.length + 4 > Int.MaxValue) { + respondWithError(s"Block $blockId size ($segment.length) greater than 2G") + return + } + + var fileChannel: FileChannel = null + try { + fileChannel = new FileInputStream(segment.file).getChannel + } catch { + case e: Exception => + logError( + s"Error opening channel for $blockId in ${segment.file} for request from $client", e) + respondWithError(e.getMessage) + } + + // Found the block. Send it back. + if (fileChannel != null) { + // Write the header and block data. In the case of failures, the listener on the block data + // write should close the connection. + ctx.write(new BlockHeader(segment.length.toInt, blockId)) + + val region = new DefaultFileRegion(fileChannel, segment.offset, segment.length) + ctx.writeAndFlush(region).addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (future.isSuccess) { + logTrace(s"Sent block $blockId (${segment.length} B) back to $client") + } else { + logError(s"Error sending block $blockId to $client; closing connection", future.cause) + ctx.close() + } + } + }) + } + } + + def writeByteBuffer(buf: ByteBuffer): Unit = { + ctx.write(new BlockHeader(buf.remaining, blockId)) + ctx.writeAndFlush(Unpooled.wrappedBuffer(buf)).addListener(new ChannelFutureListener { + override def operationComplete(future: ChannelFuture) { + if (future.isSuccess) { + logTrace(s"Sent block $blockId (${buf.remaining} B) back to $client") + } else { + logError(s"Error sending block $blockId to $client; closing connection", future.cause) + ctx.close() + } + } + }) + } + + logTrace(s"Received request from $client to fetch block $blockId") + + var blockData: Either[FileSegment, ByteBuffer] = null + + // First make sure we can find the block. If not, send error back to the user. + try { + blockData = dataProvider.getBlockData(blockId) + } catch { + case e: Exception => + logError(s"Error opening block $blockId for request from $client", e) + respondWithError(e.getMessage) + return + } + + blockData match { + case Left(segment) => writeFileSegment(segment) + case Right(buf) => writeByteBuffer(buf) + } + + } // end of channelRead0 +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala b/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala new file mode 100644 index 0000000000000..5b6d086630834 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.nio.ByteBuffer + + +/** + * An interface for providing data for blocks. + * + * getBlockData returns either a FileSegment (for zero-copy send), or a ByteBuffer. + * + * Aside from unit tests, [[BlockManager]] is the main class that implements this. + */ +private[spark] trait BlockDataProvider { + def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 5f44f5f3197fd..ca60ec78b62ee 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -18,19 +18,17 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue +import org.apache.spark.network.netty.client.{BlockClientListener, LazyInitIterator, ReferenceCountedBuffer} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue import scala.util.{Failure, Success} -import io.netty.buffer.ByteBuf - import org.apache.spark.{Logging, SparkException} import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.network.BufferMessage import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.network.netty.ShuffleCopier import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils @@ -54,18 +52,28 @@ trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] wi private[storage] object BlockFetcherIterator { - // A request to fetch one or more blocks, complete with their sizes + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of tuple, where the first element is the block id, + * and the second element is the estimated size, used to calculate bytesInFlight. + */ class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { val size = blocks.map(_._2).sum } - // A result of a fetch. Includes the block ID, size in bytes, and a function to deserialize - // the block (since we want all deserializaton to happen in the calling thread); can also - // represent a fetch failure if size == -1. + /** + * Result of a fetch from a remote block. A failure is represented as size == -1. + * @param blockId block id + * @param size estimated size of the block, used to calculate bytesInFlight. + * Note that this is NOT the exact bytes. + * @param deserialize closure to return the result in the form of an Iterator. + */ class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { def failed: Boolean = size == -1 } + // TODO: Refactor this whole thing to make code more reusable. class BasicBlockFetcherIterator( private val blockManager: BlockManager, val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], @@ -95,10 +103,10 @@ object BlockFetcherIterator { // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that // the number of bytes in flight is limited to maxBytesInFlight - private val fetchRequests = new Queue[FetchRequest] + protected val fetchRequests = new Queue[FetchRequest] // Current bytes in flight from our requests - private var bytesInFlight = 0L + protected var bytesInFlight = 0L protected def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( @@ -262,77 +270,58 @@ object BlockFetcherIterator { readMetrics: ShuffleReadMetrics) extends BasicBlockFetcherIterator(blockManager, blocksByAddress, serializer, readMetrics) { - import blockManager._ + override protected def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + val cmId = new ConnectionManagerId(req.address.host, req.address.port) - val fetchRequestsSync = new LinkedBlockingQueue[FetchRequest] + bytesInFlight += req.size + val sizeMap = req.blocks.toMap // so we can look up the size of each blockID + + // This could throw a TimeoutException. In that case we will just retry the task. + val client = blockManager.nettyBlockClientFactory.createClient( + cmId.host, req.address.nettyPort) + val blocks = req.blocks.map(_._1.toString) + + client.fetchBlocks( + blocks, + new BlockClientListener { + override def onFetchFailure(blockId: String, errorMsg: String): Unit = { + logError(s"Could not get block(s) from $cmId with error: $errorMsg") + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } - private def startCopiers(numCopiers: Int): List[_ <: Thread] = { - (for ( i <- Range(0,numCopiers) ) yield { - val copier = new Thread { - override def run(){ - try { - while(!isInterrupted && !fetchRequestsSync.isEmpty) { - sendRequest(fetchRequestsSync.take()) + override def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit = { + // Increment the reference count so the buffer won't be recycled. + // TODO: This could result in memory leaks when the task is stopped due to exception + // before the iterator is exhausted. + data.retain() + val buf = data.byteBuffer() + val blockSize = buf.remaining() + val bid = BlockId(blockId) + + // TODO: remove code duplication between here and BlockManager.dataDeserialization. + results.put(new FetchResult(bid, sizeMap(bid), () => { + def createIterator: Iterator[Any] = { + val stream = blockManager.wrapForCompression(bid, data.inputStream()) + serializer.newInstance().deserializeStream(stream).asIterator } - } catch { - case x: InterruptedException => logInfo("Copier Interrupted") - // case _ => throw new SparkException("Exception Throw in Shuffle Copier") + new LazyInitIterator(createIterator) { + // Release the buffer when we are done traversing it. + override def close(): Unit = data.release() + } + })) + + readMetrics.synchronized { + readMetrics.remoteBytesRead += blockSize + readMetrics.remoteBlocksFetched += 1 } + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } } - copier.start - copier - }).toList - } - - // keep this to interrupt the threads when necessary - private def stopCopiers() { - for (copier <- copiers) { - copier.interrupt() - } - } - - override protected def sendRequest(req: FetchRequest) { - - def putResult(blockId: BlockId, blockSize: Long, blockData: ByteBuf) { - val fetchResult = new FetchResult(blockId, blockSize, - () => dataDeserialize(blockId, blockData.nioBuffer, serializer)) - results.put(fetchResult) - } - - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.bytesToString(req.size), req.address.host)) - val cmId = new ConnectionManagerId(req.address.host, req.address.nettyPort) - val cpier = new ShuffleCopier(blockManager.conf) - cpier.getBlocks(cmId, req.blocks, putResult) - logDebug("Sent request for remote blocks " + req.blocks + " from " + req.address.host ) - } - - private var copiers: List[_ <: Thread] = null - - override def initialize() { - // Split Local Remote Blocks and set numBlocksToFetch - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - for (request <- Utils.randomize(remoteRequests)) { - fetchRequestsSync.put(request) - } - - copiers = startCopiers(conf.getInt("spark.shuffle.copier.threads", 6)) - logInfo("Started " + fetchRequestsSync.size + " remote fetches in " + - Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } - - override def next(): (BlockId, Option[Iterator[Any]]) = { - resultsGotten += 1 - val result = results.take() - // If all the results has been retrieved, copiers will exit automatically - (result.blockId, if (result.failed) None else Some(result.deserialize())) + ) } } // End of NettyBlockFetcherIterator diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e4c3d58905e7f..c0491fb55e3a4 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -25,17 +25,20 @@ import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random -import akka.actor.{ActorSystem, Cancellable, Props} +import akka.actor.{ActorSystem, Props} import sun.nio.ch.DirectBuffer import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ +import org.apache.spark.network.netty.client.BlockFetchingClientFactory +import org.apache.spark.network.netty.server.BlockServer import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ + private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues @@ -60,7 +63,7 @@ private[spark] class BlockManager( securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, shuffleManager: ShuffleManager) - extends Logging { + extends BlockDataProvider with Logging { private val port = conf.getInt("spark.blockManager.port", 0) val shuffleBlockManager = new ShuffleBlockManager(this, shuffleManager) @@ -88,13 +91,25 @@ private[spark] class BlockManager( new TachyonStore(this, tachyonBlockManager) } + private val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) + // If we use Netty for shuffle, start a new Netty-based shuffle sender service. - private val nettyPort: Int = { - val useNetty = conf.getBoolean("spark.shuffle.use.netty", false) - val nettyPortConfig = conf.getInt("spark.shuffle.sender.port", 0) - if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 + private[storage] val nettyBlockClientFactory: BlockFetchingClientFactory = { + if (useNetty) new BlockFetchingClientFactory(conf) else null } + private val nettyBlockServer: BlockServer = { + if (useNetty) { + val server = new BlockServer(conf, this) + logInfo(s"Created NettyBlockServer binding to port: ${server.port}") + server + } else { + null + } + } + + private val nettyPort: Int = if (useNetty) nettyBlockServer.port else 0 + val blockManagerId = BlockManagerId( executorId, connectionManager.id.host, connectionManager.id.port, nettyPort) @@ -219,6 +234,20 @@ private[spark] class BlockManager( } } + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + val bid = BlockId(blockId) + if (bid.isShuffle) { + Left(diskBlockManager.getBlockLocation(bid)) + } else { + val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + if (blockBytesOpt.isDefined) { + Right(blockBytesOpt.get) + } else { + throw new BlockNotFoundException(blockId) + } + } + } + /** * Get the BlockStatus for the block identified by the given ID, if it exists. * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. @@ -1064,6 +1093,14 @@ private[spark] class BlockManager( connectionManager.stop() shuffleBlockManager.stop() diskBlockManager.stop() + + if (nettyBlockClientFactory != null) { + nettyBlockClientFactory.stop() + } + if (nettyBlockServer != null) { + nettyBlockServer.stop() + } + actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala new file mode 100644 index 0000000000000..9ef453605f4f1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + + +class BlockNotFoundException(blockId: String) extends Exception(s"Block $blockId not found") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 4d66ccea211fa..f3da816389581 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -23,7 +23,7 @@ import java.util.{Date, Random, UUID} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.network.netty.{PathResolver, ShuffleSender} +import org.apache.spark.network.netty.PathResolver import org.apache.spark.util.Utils import org.apache.spark.shuffle.sort.SortShuffleManager @@ -52,7 +52,6 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) - private var shuffleSender : ShuffleSender = null addShutdownHook() @@ -186,15 +185,5 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, } } } - - if (shuffleSender != null) { - shuffleSender.stop() - } - } - - private[storage] def startShuffleBlockSender(port: Int): Int = { - shuffleSender = new ShuffleSender(port, this) - logInfo(s"Created ShuffleSender binding to port: ${shuffleSender.port}") - shuffleSender.port } } diff --git a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala new file mode 100644 index 0000000000000..02d0ffc86f58f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.io.{RandomAccessFile, File} +import java.nio.ByteBuffer +import java.util.{Collections, HashSet} +import java.util.concurrent.{TimeUnit, Semaphore} + +import scala.collection.JavaConversions._ + +import io.netty.buffer.{ByteBufUtil, Unpooled} + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.SparkConf +import org.apache.spark.network.netty.client.{BlockClientListener, ReferenceCountedBuffer, BlockFetchingClientFactory} +import org.apache.spark.network.netty.server.BlockServer +import org.apache.spark.storage.{FileSegment, BlockDataProvider} + + +/** + * Test suite that makes sure the server and the client implementations share the same protocol. + */ +class ServerClientIntegrationSuite extends FunSuite with BeforeAndAfterAll { + + val bufSize = 100000 + var buf: ByteBuffer = _ + var testFile: File = _ + var server: BlockServer = _ + var clientFactory: BlockFetchingClientFactory = _ + + val bufferBlockId = "buffer_block" + val fileBlockId = "file_block" + + val fileContent = new Array[Byte](1024) + scala.util.Random.nextBytes(fileContent) + + override def beforeAll() = { + buf = ByteBuffer.allocate(bufSize) + for (i <- 1 to bufSize) { + buf.put(i.toByte) + } + buf.flip() + + testFile = File.createTempFile("netty-test-file", "txt") + val fp = new RandomAccessFile(testFile, "rw") + fp.write(fileContent) + fp.close() + + server = new BlockServer(new SparkConf, new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + if (blockId == bufferBlockId) { + Right(buf) + } else if (blockId == fileBlockId) { + Left(new FileSegment(testFile, 10, testFile.length - 25)) + } else { + throw new Exception("Unknown block id " + blockId) + } + } + }) + + clientFactory = new BlockFetchingClientFactory(new SparkConf) + } + + override def afterAll() = { + server.stop() + clientFactory.stop() + } + + /** A ByteBuf for buffer_block */ + lazy val byteBufferBlockReference = Unpooled.wrappedBuffer(buf) + + /** A ByteBuf for file_block */ + lazy val fileBlockReference = Unpooled.wrappedBuffer(fileContent, 10, fileContent.length - 25) + + def fetchBlocks(blockIds: Seq[String]): (Set[String], Set[ReferenceCountedBuffer], Set[String]) = + { + val client = clientFactory.createClient(server.hostName, server.port) + val sem = new Semaphore(0) + val receivedBlockIds = Collections.synchronizedSet(new HashSet[String]) + val errorBlockIds = Collections.synchronizedSet(new HashSet[String]) + val receivedBuffers = Collections.synchronizedSet(new HashSet[ReferenceCountedBuffer]) + + client.fetchBlocks( + blockIds, + new BlockClientListener { + override def onFetchFailure(blockId: String, errorMsg: String): Unit = { + errorBlockIds.add(blockId) + sem.release() + } + + override def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit = { + receivedBlockIds.add(blockId) + data.retain() + receivedBuffers.add(data) + sem.release() + } + } + ) + if (!sem.tryAcquire(blockIds.size, 30, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server") + } + client.close() + (receivedBlockIds.toSet, receivedBuffers.toSet, errorBlockIds.toSet) + } + + test("fetch a ByteBuffer block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId)) + assert(blockIds === Set(bufferBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch a FileSegment block via zero-copy send") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(fileBlockId)) + assert(blockIds === Set(fileBlockId)) + assert(buffers.map(_.underlying) === Set(fileBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch a non-existent block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq("random-block")) + assert(blockIds.isEmpty) + assert(buffers.isEmpty) + assert(failBlockIds === Set("random-block")) + } + + test("fetch both ByteBuffer block and FileSegment block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, fileBlockId)) + assert(blockIds === Set(bufferBlockId, fileBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference, fileBlockReference)) + assert(failBlockIds.isEmpty) + buffers.foreach(_.release()) + } + + test("fetch both ByteBuffer block and a non-existent block") { + val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, "random-block")) + assert(blockIds === Set(bufferBlockId)) + assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) + assert(failBlockIds === Set("random-block")) + buffers.foreach(_.release()) + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala new file mode 100644 index 0000000000000..903ab09ae4322 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.client + +import java.nio.ByteBuffer + +import io.netty.buffer.Unpooled +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.{PrivateMethodTester, FunSuite} + + +class BlockFetchingClientHandlerSuite extends FunSuite with PrivateMethodTester { + + test("handling block data (successful fetch)") { + val blockId = "test_block" + val blockData = "blahblahblahblahblah" + val totalLength = 4 + blockId.length + blockData.length + + var parsedBlockId: String = "" + var parsedBlockData: String = "" + val handler = new BlockFetchingClientHandler + handler.addRequest(blockId, + new BlockClientListener { + override def onFetchFailure(blockId: String, errorMsg: String): Unit = ??? + override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer): Unit = { + parsedBlockId = bid + val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) + refCntBuf.byteBuffer().get(bytes) + parsedBlockData = new String(bytes) + } + } + ) + + val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) + assert(handler.invokePrivate(outstandingRequests()).size === 1) + + val channel = new EmbeddedChannel(handler) + val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself + buf.putInt(totalLength) + buf.putInt(blockId.length) + buf.put(blockId.getBytes) + buf.put(blockData.getBytes) + buf.flip() + + channel.writeInbound(Unpooled.wrappedBuffer(buf)) + assert(parsedBlockId === blockId) + assert(parsedBlockData === blockData) + + assert(handler.invokePrivate(outstandingRequests()).size === 0) + + channel.close() + } + + test("handling error message (failed fetch)") { + val blockId = "test_block" + val errorMsg = "error erro5r error err4or error3 error6 error erro1r" + val totalLength = 4 + blockId.length + errorMsg.length + + var parsedBlockId: String = "" + var parsedErrorMsg: String = "" + val handler = new BlockFetchingClientHandler + handler.addRequest(blockId, new BlockClientListener { + override def onFetchFailure(bid: String, msg: String) ={ + parsedBlockId = bid + parsedErrorMsg = msg + } + override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer) = ??? + }) + + val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) + assert(handler.invokePrivate(outstandingRequests()).size === 1) + + val channel = new EmbeddedChannel(handler) + val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself + buf.putInt(totalLength) + buf.putInt(-blockId.length) + buf.put(blockId.getBytes) + buf.put(errorMsg.getBytes) + buf.flip() + + channel.writeInbound(Unpooled.wrappedBuffer(buf)) + assert(parsedBlockId === blockId) + assert(parsedErrorMsg === errorMsg) + + assert(handler.invokePrivate(outstandingRequests()).size === 0) + + channel.close() + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala new file mode 100644 index 0000000000000..3ee281cb1350b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import io.netty.buffer.ByteBuf +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.FunSuite + + +class BlockHeaderEncoderSuite extends FunSuite { + + test("encode normal block data") { + val blockId = "test_block" + val channel = new EmbeddedChannel(new BlockHeaderEncoder) + channel.writeOutbound(new BlockHeader(17, blockId, None)) + val out = channel.readOutbound().asInstanceOf[ByteBuf] + assert(out.readInt() === 4 + blockId.length + 17) + assert(out.readInt() === blockId.length) + + val blockIdBytes = new Array[Byte](blockId.length) + out.readBytes(blockIdBytes) + assert(new String(blockIdBytes) === blockId) + assert(out.readableBytes() === 0) + + channel.close() + } + + test("encode error message") { + val blockId = "error_block" + val errorMsg = "error encountered" + val channel = new EmbeddedChannel(new BlockHeaderEncoder) + channel.writeOutbound(new BlockHeader(17, blockId, Some(errorMsg))) + val out = channel.readOutbound().asInstanceOf[ByteBuf] + assert(out.readInt() === 4 + blockId.length + errorMsg.length) + assert(out.readInt() === -blockId.length) + + val blockIdBytes = new Array[Byte](blockId.length) + out.readBytes(blockIdBytes) + assert(new String(blockIdBytes) === blockId) + + val errorMsgBytes = new Array[Byte](errorMsg.length) + out.readBytes(errorMsgBytes) + assert(new String(errorMsgBytes) === errorMsg) + assert(out.readableBytes() === 0) + + channel.close() + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala new file mode 100644 index 0000000000000..3239c710f1639 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty.server + +import java.io.{RandomAccessFile, File} +import java.nio.ByteBuffer + +import io.netty.buffer.{Unpooled, ByteBuf} +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler, DefaultFileRegion} +import io.netty.channel.embedded.EmbeddedChannel + +import org.scalatest.FunSuite + +import org.apache.spark.storage.{BlockDataProvider, FileSegment} + + +class BlockServerHandlerSuite extends FunSuite { + + test("ByteBuffer block") { + val expectedBlockId = "test_bytebuffer_block" + val buf = ByteBuffer.allocate(10000) + for (i <- 1 to 10000) { + buf.put(i.toByte) + } + buf.flip() + + val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = Right(buf) + })) + + channel.writeInbound(expectedBlockId) + assert(channel.outboundMessages().size === 2) + + val out1 = channel.readOutbound().asInstanceOf[BlockHeader] + val out2 = channel.readOutbound().asInstanceOf[ByteBuf] + + assert(out1.blockId === expectedBlockId) + assert(out1.blockSize === buf.remaining) + assert(out1.error === None) + + assert(out2.equals(Unpooled.wrappedBuffer(buf))) + + channel.close() + } + + test("FileSegment block via zero-copy") { + val expectedBlockId = "test_file_block" + + // Create random file data + val fileContent = new Array[Byte](1024) + scala.util.Random.nextBytes(fileContent) + val testFile = File.createTempFile("netty-test-file", "txt") + val fp = new RandomAccessFile(testFile, "rw") + fp.write(fileContent) + fp.close() + + val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + Left(new FileSegment(testFile, 15, testFile.length - 25)) + } + })) + + channel.writeInbound(expectedBlockId) + assert(channel.outboundMessages().size === 2) + + val out1 = channel.readOutbound().asInstanceOf[BlockHeader] + val out2 = channel.readOutbound().asInstanceOf[DefaultFileRegion] + + assert(out1.blockId === expectedBlockId) + assert(out1.blockSize === testFile.length - 25) + assert(out1.error === None) + + assert(out2.count === testFile.length - 25) + assert(out2.position === 15) + } + + test("pipeline exception propagation") { + val blockServerHandler = new BlockServerHandler(new BlockDataProvider { + override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = ??? + }) + val exceptionHandler = new SimpleChannelInboundHandler[String]() { + override def channelRead0(ctx: ChannelHandlerContext, msg: String): Unit = { + throw new Exception("this is an error") + } + } + + val channel = new EmbeddedChannel(exceptionHandler, blockServerHandler) + assert(channel.isOpen) + channel.writeInbound("a message to trigger the error") + assert(!channel.isOpen) + } +} diff --git a/pom.xml b/pom.xml index ef12c8f1a5c49..0d44cf4ea5f92 100644 --- a/pom.xml +++ b/pom.xml @@ -420,7 +420,7 @@ io.netty netty-all - 4.0.17.Final + 4.0.23.Final org.apache.derby From 1870dbaa5591883e61b2173d064c1a67e871b0f5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 17:41:37 -0700 Subject: [PATCH 0421/1492] [MLLIB] minor update to word2vec very minor update Ishiihara Author: Xiangrui Meng Closes #2043 from mengxr/minor-w2v and squashes the following commits: be649fd [Xiangrui Meng] remove map because we only need append eccefcc [Xiangrui Meng] minor updates to word2vec --- .../apache/spark/mllib/feature/Word2Vec.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 1dcaa2cd2e630..c3375ed44fd99 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -30,11 +30,9 @@ import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.{Vector, Vectors} -import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom -import org.apache.spark.util.collection.PrimitiveKeyOpenHashMap /** * Entry in vocabulary @@ -285,9 +283,9 @@ class Word2Vec extends Serializable with Logging { val newSentences = sentences.repartition(numPartitions).cache() val initRandom = new XORShiftRandom(seed) - var syn0Global = + val syn0Global = Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) - var syn1Global = new Array[Float](vocabSize * vectorSize) + val syn1Global = new Array[Float](vocabSize * vectorSize) var alpha = startingAlpha for (k <- 1 to numIterations) { val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => @@ -349,21 +347,21 @@ class Word2Vec extends Serializable with Logging { } val syn0Local = model._1 val syn1Local = model._2 - val synOut = new PrimitiveKeyOpenHashMap[Int, Array[Float]](vocabSize * 2) + val synOut = mutable.ListBuffer.empty[(Int, Array[Float])] var index = 0 while(index < vocabSize) { if (syn0Modify(index) != 0) { - synOut.update(index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize)) + synOut += ((index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize))) } if (syn1Modify(index) != 0) { - synOut.update(index + vocabSize, - syn1Local.slice(index * vectorSize, (index + 1) * vectorSize)) + synOut += ((index + vocabSize, + syn1Local.slice(index * vectorSize, (index + 1) * vectorSize))) } index += 1 } - Iterator(synOut) + synOut.toIterator } - val synAgg = partial.flatMap(x => x).reduceByKey { case (v1, v2) => + val synAgg = partial.reduceByKey { case (v1, v2) => blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) v1 }.collect() From c7252b0097cfacd36f17357d195b12a59e503b35 Mon Sep 17 00:00:00 2001 From: freeman Date: Tue, 19 Aug 2014 18:07:42 -0700 Subject: [PATCH 0422/1492] [SPARK-3112][MLLIB] Add documentation and example for StreamingLR Added a documentation section on StreamingLR to the ``MLlib - Linear Methods``, including a worked example. mengxr tdas Author: freeman Closes #2047 from freeman-lab/streaming-lr-docs and squashes the following commits: 568d250 [freeman] Tweaks to wording / formatting 05a1139 [freeman] Added documentation and example for StreamingLR --- docs/mllib-linear-methods.md | 75 ++++++++++++++++++++++++++++++++++++ 1 file changed, 75 insertions(+) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index e504cd7f0f578..9137f9dc1b692 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -518,6 +518,81 @@ print("Mean Squared Error = " + str(MSE))
    +## Streaming linear regression + +When data arrive in a streaming fashion, it is useful to fit regression models online, +updating the parameters of the model as new data arrives. MLlib currently supports +streaming linear regression using ordinary least squares. The fitting is similar +to that performed offline, except fitting occurs on each batch of data, so that +the model continually updates to reflect the data from the stream. + +### Examples + +The following example demonstrates how to load training and testing data from two different +input streams of text files, parse the streams as labeled points, fit a linear regression model +online to the first stream, and make predictions on the second stream. + +
    + +
    + +First, we import the necessary classes for parsing our input data and creating the model. + +{% highlight scala %} + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.regression.StreamingLinearRegressionWithSGD + +{% endhighlight %} + +Then we make input streams for training and testing data. We assume a StreamingContext `ssc` +has already been created, see [Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) +for more info. For this example, we use labeled points in training and testing streams, +but in practice you will likely want to use unlabeled vectors for test data. + +{% highlight scala %} + +val trainingData = ssc.textFileStream('/training/data/dir').map(LabeledPoint.parse) +val testData = ssc.textFileStream('/testing/data/dir').map(LabeledPoint.parse) + +{% endhighlight %} + +We create our model by initializing the weights to 0 + +{% highlight scala %} + +val numFeatures = 3 +val model = new StreamingLinearRegressionWithSGD() + .setInitialWeights(Vectors.zeros(numFeatures)) + +{% endhighlight %} + +Now we register the streams for training and testing and start the job. +Printing predictions alongside true labels lets us easily see the result. + +{% highlight scala %} + +model.trainOn(trainingData) +model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() + +ssc.start() +ssc.awaitTermination() + +{% endhighlight %} + +We can now save text files with data to the training or testing folders. +Each line should be a data point formatted as `(y,[x1,x2,x3])` where `y` is the label +and `x1,x2,x3` are the features. Anytime a text file is placed in `/training/data/dir` +the model will update. Anytime a text file is placed in `/testing/data/dir` you will see predictions. +As you feed more data to the training directory, the predictions +will get better! + +
    + +
    + + ## Implementation (developer) Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent From 0e3ab94d413fd70fff748fded42ab5e2ebd66fcc Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 19 Aug 2014 19:37:02 -0700 Subject: [PATCH 0423/1492] [SQL] add note of use synchronizedMap in SQLConf Refer to: http://stackoverflow.com/questions/510632/whats-the-difference-between-concurrenthashmap-and-collections-synchronizedmap Collections.synchronizedMap(map) creates a blocking Map which will degrade performance, albeit ensure consistency. So use ConcurrentHashMap(a more effective thread-safe hashmap) instead. also update HiveQuerySuite to fix test error when changed to ConcurrentHashMap. Author: wangfei Author: scwf Closes #1996 from scwf/sqlconf and squashes the following commits: 93bc0c5 [wangfei] revert change of HiveQuerySuite 0cc05dd [wangfei] add note for use synchronizedMap 3c224d31 [scwf] fix formate a7bcb98 [scwf] use ConcurrentHashMap in sql conf, intead synchronizedMap --- sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 4f2adb006fbc7..5cc41a83cc792 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -54,6 +54,7 @@ private[spark] object SQLConf { trait SQLConf { import SQLConf._ + /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ @transient protected[spark] val settings = java.util.Collections.synchronizedMap( new java.util.HashMap[String, String]()) From 068b6fe6a10eb1c6b2102d88832203267f030e85 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 21:01:23 -0700 Subject: [PATCH 0424/1492] [SPARK-3130][MLLIB] detect negative values in naive Bayes because NB treats feature values as term frequencies. jkbradley Author: Xiangrui Meng Closes #2038 from mengxr/nb-neg and squashes the following commits: 52c37c3 [Xiangrui Meng] address comments 65f892d [Xiangrui Meng] detect negative values in nb --- docs/mllib-naive-bayes.md | 3 +- .../mllib/classification/NaiveBayes.scala | 28 +++++++++++++++---- .../classification/NaiveBayesSuite.scala | 28 +++++++++++++++++++ 3 files changed, 53 insertions(+), 6 deletions(-) diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 86d94aebd9442..7f9d4c6563944 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -17,7 +17,8 @@ Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bay which is typically used for [document classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html). Within that context, each observation is a document and each -feature represents a term whose value is the frequency of the term. +feature represents a term whose value is the frequency of the term. +Feature values must be nonnegative to represent term frequencies. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature vectors are usually sparse, and sparse vectors should be supplied as input to take advantage of diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 6c7be0a4f1dcb..8c8e4a161aa5b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.SparkContext._ -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD @@ -73,7 +73,7 @@ class NaiveBayesModel private[mllib] ( * This is the Multinomial NB ([[http://tinyurl.com/lsdw6p]]) which can handle all kinds of * discrete data. For example, by converting documents into TF-IDF vectors, it can be used for * document classification. By making every vector a 0-1 vector, it can also be used as - * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). + * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). The input feature values must be nonnegative. */ class NaiveBayes private (private var lambda: Double) extends Serializable with Logging { @@ -91,12 +91,30 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ def run(data: RDD[LabeledPoint]) = { + val requireNonnegativeValues: Vector => Unit = (v: Vector) => { + val values = v match { + case sv: SparseVector => + sv.values + case dv: DenseVector => + dv.values + } + if (!values.forall(_ >= 0.0)) { + throw new SparkException(s"Naive Bayes requires nonnegative feature values but found $v.") + } + } + // Aggregates term frequencies per label. // TODO: Calling combineByKey and collect creates two stages, we can implement something // TODO: similar to reduceByKeyLocally to save one stage. val aggregated = data.map(p => (p.label, p.features)).combineByKey[(Long, BDV[Double])]( - createCombiner = (v: Vector) => (1L, v.toBreeze.toDenseVector), - mergeValue = (c: (Long, BDV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze), + createCombiner = (v: Vector) => { + requireNonnegativeValues(v) + (1L, v.toBreeze.toDenseVector) + }, + mergeValue = (c: (Long, BDV[Double]), v: Vector) => { + requireNonnegativeValues(v) + (c._1 + 1L, c._2 += v.toBreeze) + }, mergeCombiners = (c1: (Long, BDV[Double]), c2: (Long, BDV[Double])) => (c1._1 + c2._1, c1._2 += c2._2) ).collect() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 06cdd04f5fdae..80989bc074e84 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -21,6 +21,7 @@ import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkException import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.{LocalClusterSparkContext, LocalSparkContext} @@ -95,6 +96,33 @@ class NaiveBayesSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("detect negative values") { + val dense = Seq( + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(0.0, Vectors.dense(-1.0)), + LabeledPoint(1.0, Vectors.dense(1.0)), + LabeledPoint(1.0, Vectors.dense(0.0))) + intercept[SparkException] { + NaiveBayes.train(sc.makeRDD(dense, 2)) + } + val sparse = Seq( + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(0.0, Vectors.sparse(1, Array(0), Array(-1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array.empty, Array.empty))) + intercept[SparkException] { + NaiveBayes.train(sc.makeRDD(sparse, 2)) + } + val nan = Seq( + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(0.0, Vectors.sparse(1, Array(0), Array(Double.NaN))), + LabeledPoint(1.0, Vectors.sparse(1, Array(0), Array(1.0))), + LabeledPoint(1.0, Vectors.sparse(1, Array.empty, Array.empty))) + intercept[SparkException] { + NaiveBayes.train(sc.makeRDD(nan, 2)) + } + } } class NaiveBayesClusterSuite extends FunSuite with LocalClusterSparkContext { From fce5c0fb6384f3a142a4155525a5d62640725150 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 22:05:29 -0700 Subject: [PATCH 0425/1492] [HOTFIX][Streaming][MLlib] use temp folder for checkpoint or Jenkins will complain about no Apache header in checkpoint files. tdas rxin Author: Xiangrui Meng Closes #2046 from mengxr/tmp-checkpoint and squashes the following commits: 0d3ec73 [Xiangrui Meng] remove ssc.stop 9797843 [Xiangrui Meng] change checkpointDir to lazy val 89964ab [Xiangrui Meng] use temp folder for checkpoint --- .../StreamingLinearRegressionSuite.scala | 6 ------ .../apache/spark/streaming/TestSuiteBase.scala | 17 +++++++++++------ 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 28489410f8225..03b71301e9ab1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -49,7 +49,6 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // Test if we can accurately learn Y = 10*X1 + 10*X2 on streaming data test("parameter accuracy") { - // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0, 0.0)) @@ -82,7 +81,6 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // Test that parameter estimates improve when learning Y = 10*X1 on streaming data test("parameter convergence") { - // create model val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(0.0)) @@ -113,12 +111,10 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { assert(deltas.forall(x => (x._1 - x._2) <= 0.1)) // check that error shrunk on at least 2 batches assert(deltas.map(x => if ((x._1 - x._2) < 0) 1 else 0).sum > 1) - } // Test predictions on a stream test("predictions") { - // create model initialized with true weights val model = new StreamingLinearRegressionWithSGD() .setInitialWeights(Vectors.dense(10.0, 10.0)) @@ -142,7 +138,5 @@ class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // compute the mean absolute error and check that it's always less than 0.1 val errors = output.map(batch => batch.map(p => math.abs(p._1 - p._2)).sum / nPoints) assert(errors.forall(x => x <= 0.1)) - } - } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index f095da9cb55d3..759baacaa4308 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -17,18 +17,18 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} -import org.apache.spark.streaming.util.ManualClock +import java.io.{ObjectInputStream, IOException} import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.SynchronizedBuffer import scala.reflect.ClassTag -import java.io.{ObjectInputStream, IOException} - import org.scalatest.{BeforeAndAfter, FunSuite} +import com.google.common.io.Files -import org.apache.spark.{SparkContext, SparkConf, Logging} +import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.rdd.RDD /** @@ -119,7 +119,12 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { def batchDuration = Seconds(1) // Directory where the checkpoint data will be saved - def checkpointDir = "checkpoint" + lazy val checkpointDir = { + val dir = Files.createTempDir() + logDebug(s"checkpointDir: $dir") + dir.deleteOnExit() + dir.toString + } // Number of partitions of the input parallel collections created for testing def numInputPartitions = 2 From 8adfbc2b6b5b647e450d30f89c141f935b6aa94b Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 19 Aug 2014 22:11:13 -0700 Subject: [PATCH 0426/1492] [SPARK-3119] Re-implementation of TorrentBroadcast. This is a re-implementation of TorrentBroadcast, with the following changes: 1. Removes most of the mutable, transient state from TorrentBroadcast (e.g. totalBytes, num of blocks fetched). 2. Removes TorrentInfo and TorrentBlock 3. Replaces the BlockManager.getSingle call in readObject with a getLocal, resuling in one less RPC call to the BlockManagerMasterActor to find the location of the block. 4. Removes the metadata block, resulting in one less block to fetch. 5. Removes an extra memory copy for deserialization (by using Java's SequenceInputStream). Basically for a regular broadcasted object with only one block, the number of RPC calls goes from 5+1 to 2+1). Old TorrentBroadcast for object of a single block: 1 RPC to ask for location of the broadcast variable 1 RPC to ask for location of the metadata block 1 RPC to fetch the metadata block 1 RPC to ask for location of the first data block 1 RPC to fetch the first data block 1 RPC to tell the driver we put the first data block in i.e. 5 + 1 New TorrentBroadcast for object of a single block: 1 RPC to ask for location of the first data block 1 RPC to get the first data block 1 RPC to tell the driver we put the first data block in i.e. 2 + 1 Author: Reynold Xin Closes #2030 from rxin/torrentBroadcast and squashes the following commits: 5bacb9d [Reynold Xin] Always add the object to driver's block manager. 0d8ed5b [Reynold Xin] Added getBytes to BlockManager and uses that in TorrentBroadcast. 2d6a5fb [Reynold Xin] Use putBytes/getRemoteBytes throughout. 3670f00 [Reynold Xin] Code review feedback. c1185cd [Reynold Xin] [SPARK-3119] Re-implementation of TorrentBroadcast. --- .../spark/broadcast/BroadcastFactory.scala | 11 + .../spark/broadcast/TorrentBroadcast.scala | 282 +++++++----------- .../spark/broadcast/BroadcastSuite.scala | 128 ++++---- 3 files changed, 181 insertions(+), 240 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index a8c827030a1ef..6a187b40628a2 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -32,8 +32,19 @@ import org.apache.spark.annotation.DeveloperApi */ @DeveloperApi trait BroadcastFactory { + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit + + /** + * Creates a new broadcast variable. + * + * @param value value to broadcast + * @param isLocal whether we are in local mode (single JVM process) + * @param id unique id representing this broadcast variable + */ def newBroadcast[T: ClassTag](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit + def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index d8be649f96e5f..6173fd3a69fc7 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -18,50 +18,116 @@ package org.apache.spark.broadcast import java.io._ +import java.nio.ByteBuffer +import scala.collection.JavaConversions.asJavaEnumeration import scala.reflect.ClassTag import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} +import org.apache.spark.util.ByteBufferInputStream /** - * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like - * protocol to do a distributed transfer of the broadcasted data to the executors. - * The mechanism is as follows. The driver divides the serializes the broadcasted data, - * divides it into smaller chunks, and stores them in the BlockManager of the driver. - * These chunks are reported to the BlockManagerMaster so that all the executors can - * learn the location of those chunks. The first time the broadcast variable (sent as - * part of task) is deserialized at a executor, all the chunks are fetched using - * the BlockManager. When all the chunks are fetched (initially from the driver's - * BlockManager), they are combined and deserialized to recreate the broadcasted data. - * However, the chunks are also stored in the BlockManager and reported to the - * BlockManagerMaster. As more executors fetch the chunks, BlockManagerMaster learns - * multiple locations for each chunk. Hence, subsequent fetches of each chunk will be - * made to other executors who already have those chunks, resulting in a distributed - * fetching. This prevents the driver from being the bottleneck in sending out multiple - * copies of the broadcast data (one per executor) as done by the - * [[org.apache.spark.broadcast.HttpBroadcast]]. + * A BitTorrent-like implementation of [[org.apache.spark.broadcast.Broadcast]]. + * + * The mechanism is as follows: + * + * The driver divides the serialized object into small chunks and + * stores those chunks in the BlockManager of the driver. + * + * On each executor, the executor first attempts to fetch the object from its BlockManager. If + * it does not exist, it then uses remote fetches to fetch the small chunks from the driver and/or + * other executors if available. Once it gets the chunks, it puts the chunks in its own + * BlockManager, ready for other executors to fetch from. + * + * This prevents the driver from being the bottleneck in sending out multiple copies of the + * broadcast data (one per executor) as done by the [[org.apache.spark.broadcast.HttpBroadcast]]. + * + * @param obj object to broadcast + * @param isLocal whether Spark is running in local mode (single JVM process). + * @param id A unique identifier for the broadcast variable. */ private[spark] class TorrentBroadcast[T: ClassTag]( - @transient var value_ : T, isLocal: Boolean, id: Long) + obj : T, + @transient private val isLocal: Boolean, + id: Long) extends Broadcast[T](id) with Logging with Serializable { - override protected def getValue() = value_ + /** + * Value of the broadcast object. On driver, this is set directly by the constructor. + * On executors, this is reconstructed by [[readObject]], which builds this value by reading + * blocks from the driver and/or other executors. + */ + @transient private var _value: T = obj private val broadcastId = BroadcastBlockId(id) - SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + /** Total number of blocks this broadcast variable contains. */ + private val numBlocks: Int = writeBlocks() + + override protected def getValue() = _value + + /** + * Divide the object into multiple blocks and put those blocks in the block manager. + * + * @return number of blocks this broadcast variable is divided into + */ + private def writeBlocks(): Int = { + // For local mode, just put the object in the BlockManager so we can find it later. + SparkEnv.get.blockManager.putSingle( + broadcastId, _value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + + if (!isLocal) { + val blocks = TorrentBroadcast.blockifyObject(_value) + blocks.zipWithIndex.foreach { case (block, i) => + SparkEnv.get.blockManager.putBytes( + BroadcastBlockId(id, "piece" + i), + block, + StorageLevel.MEMORY_AND_DISK_SER, + tellMaster = true) + } + blocks.length + } else { + 0 + } + } + + /** Fetch torrent blocks from the driver and/or other executors. */ + private def readBlocks(): Array[ByteBuffer] = { + // Fetch chunks of data. Note that all these chunks are stored in the BlockManager and reported + // to the driver, so other executors can pull these chunks from this executor as well. + val blocks = new Array[ByteBuffer](numBlocks) + val bm = SparkEnv.get.blockManager - @transient private var arrayOfBlocks: Array[TorrentBlock] = null - @transient private var totalBlocks = -1 - @transient private var totalBytes = -1 - @transient private var hasBlocks = 0 + for (pid <- Random.shuffle(Seq.range(0, numBlocks))) { + val pieceId = BroadcastBlockId(id, "piece" + pid) - if (!isLocal) { - sendBroadcast() + // First try getLocalBytes because there is a chance that previous attempts to fetch the + // broadcast blocks have already fetched some of the blocks. In that case, some blocks + // would be available locally (on this executor). + var blockOpt = bm.getLocalBytes(pieceId) + if (!blockOpt.isDefined) { + blockOpt = bm.getRemoteBytes(pieceId) + blockOpt match { + case Some(block) => + // If we found the block from remote executors/driver's BlockManager, put the block + // in this executor's BlockManager. + SparkEnv.get.blockManager.putBytes( + pieceId, + block, + StorageLevel.MEMORY_AND_DISK_SER, + tellMaster = true) + + case None => + throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) + } + } + // If we get here, the option is defined. + blocks(pid) = blockOpt.get + } + blocks } /** @@ -79,26 +145,6 @@ private[spark] class TorrentBroadcast[T: ClassTag]( TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking) } - private def sendBroadcast() { - val tInfo = TorrentBroadcast.blockifyObject(value_) - totalBlocks = tInfo.totalBlocks - totalBytes = tInfo.totalBytes - hasBlocks = tInfo.totalBlocks - - // Store meta-info - val metaId = BroadcastBlockId(id, "meta") - val metaInfo = TorrentInfo(null, totalBlocks, totalBytes) - SparkEnv.get.blockManager.putSingle( - metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) - - // Store individual pieces - for (i <- 0 until totalBlocks) { - val pieceId = BroadcastBlockId(id, "piece" + i) - SparkEnv.get.blockManager.putSingle( - pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - } - } - /** Used by the JVM when serializing this object. */ private def writeObject(out: ObjectOutputStream) { assertValid() @@ -109,99 +155,30 @@ private[spark] class TorrentBroadcast[T: ClassTag]( private def readObject(in: ObjectInputStream) { in.defaultReadObject() TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.getSingle(broadcastId) match { + SparkEnv.get.blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => - value_ = x.asInstanceOf[T] + _value = x.asInstanceOf[T] case None => - val start = System.nanoTime logInfo("Started reading broadcast variable " + id) - - // Initialize @transient variables that will receive garbage values from the master. - resetWorkerVariables() - - if (receiveBroadcast()) { - value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) - - /* Store the merged copy in cache so that the next worker doesn't need to rebuild it. - * This creates a trade-off between memory usage and latency. Storing copy doubles - * the memory footprint; not storing doubles deserialization cost. Also, - * this does not need to be reported to BlockManagerMaster since other executors - * does not need to access this block (they only need to fetch the chunks, - * which are reported). - */ - SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - - // Remove arrayOfBlocks from memory once value_ is on local cache - resetWorkerVariables() - } else { - logError("Reading broadcast variable " + id + " failed") - } - - val time = (System.nanoTime - start) / 1e9 + val start = System.nanoTime() + val blocks = readBlocks() + val time = (System.nanoTime() - start) / 1e9 logInfo("Reading broadcast variable " + id + " took " + time + " s") - } - } - } - - private def resetWorkerVariables() { - arrayOfBlocks = null - totalBytes = -1 - totalBlocks = -1 - hasBlocks = 0 - } - - private def receiveBroadcast(): Boolean = { - // Receive meta-info about the size of broadcast data, - // the number of chunks it is divided into, etc. - val metaId = BroadcastBlockId(id, "meta") - var attemptId = 10 - while (attemptId > 0 && totalBlocks == -1) { - SparkEnv.get.blockManager.getSingle(metaId) match { - case Some(x) => - val tInfo = x.asInstanceOf[TorrentInfo] - totalBlocks = tInfo.totalBlocks - totalBytes = tInfo.totalBytes - arrayOfBlocks = new Array[TorrentBlock](totalBlocks) - hasBlocks = 0 - case None => - Thread.sleep(500) - } - attemptId -= 1 - } - - if (totalBlocks == -1) { - return false - } - - /* - * Fetch actual chunks of data. Note that all these chunks are stored in - * the BlockManager and reported to the master, so that other executors - * can find out and pull the chunks from this executor. - */ - val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList) - for (pid <- recvOrder) { - val pieceId = BroadcastBlockId(id, "piece" + pid) - SparkEnv.get.blockManager.getSingle(pieceId) match { - case Some(x) => - arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] - hasBlocks += 1 + _value = TorrentBroadcast.unBlockifyObject[T](blocks) + // Store the merged copy in BlockManager so other tasks on this executor don't + // need to re-fetch it. SparkEnv.get.blockManager.putSingle( - pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) - - case None => - throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) + broadcastId, _value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } } - - hasBlocks == totalBlocks } - } -private[broadcast] object TorrentBroadcast extends Logging { + +private object TorrentBroadcast extends Logging { + /** Size of each block. Default value is 4MB. */ private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 private var initialized = false private var conf: SparkConf = null @@ -223,7 +200,9 @@ private[broadcast] object TorrentBroadcast extends Logging { initialized = false } - def blockifyObject[T: ClassTag](obj: T): TorrentInfo = { + def blockifyObject[T: ClassTag](obj: T): Array[ByteBuffer] = { + // TODO: Create a special ByteArrayOutputStream that splits the output directly into chunks + // so we don't need to do the extra memory copy. val bos = new ByteArrayOutputStream() val out: OutputStream = if (compress) compressionCodec.compressedOutputStream(bos) else bos val ser = SparkEnv.get.serializer.newInstance() @@ -231,44 +210,27 @@ private[broadcast] object TorrentBroadcast extends Logging { serOut.writeObject[T](obj).close() val byteArray = bos.toByteArray val bais = new ByteArrayInputStream(byteArray) + val numBlocks = math.ceil(byteArray.length.toDouble / BLOCK_SIZE).toInt + val blocks = new Array[ByteBuffer](numBlocks) - var blockNum = byteArray.length / BLOCK_SIZE - if (byteArray.length % BLOCK_SIZE != 0) { - blockNum += 1 - } - - val blocks = new Array[TorrentBlock](blockNum) var blockId = 0 - for (i <- 0 until (byteArray.length, BLOCK_SIZE)) { val thisBlockSize = math.min(BLOCK_SIZE, byteArray.length - i) val tempByteArray = new Array[Byte](thisBlockSize) bais.read(tempByteArray, 0, thisBlockSize) - blocks(blockId) = new TorrentBlock(blockId, tempByteArray) + blocks(blockId) = ByteBuffer.wrap(tempByteArray) blockId += 1 } bais.close() - - val info = TorrentInfo(blocks, blockNum, byteArray.length) - info.hasBlocks = blockNum - info + blocks } - def unBlockifyObject[T: ClassTag]( - arrayOfBlocks: Array[TorrentBlock], - totalBytes: Int, - totalBlocks: Int): T = { - val retByteArray = new Array[Byte](totalBytes) - for (i <- 0 until totalBlocks) { - System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray, - i * BLOCK_SIZE, arrayOfBlocks(i).byteArray.length) - } + def unBlockifyObject[T: ClassTag](blocks: Array[ByteBuffer]): T = { + val is = new SequenceInputStream( + asJavaEnumeration(blocks.iterator.map(block => new ByteBufferInputStream(block)))) + val in: InputStream = if (compress) compressionCodec.compressedInputStream(is) else is - val in: InputStream = { - val arrIn = new ByteArrayInputStream(retByteArray) - if (compress) compressionCodec.compressedInputStream(arrIn) else arrIn - } val ser = SparkEnv.get.serializer.newInstance() val serIn = ser.deserializeStream(in) val obj = serIn.readObject[T]() @@ -284,17 +246,3 @@ private[broadcast] object TorrentBroadcast extends Logging { SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) } } - -private[broadcast] case class TorrentBlock( - blockID: Int, - byteArray: Array[Byte]) - extends Serializable - -private[broadcast] case class TorrentInfo( - @transient arrayOfBlocks: Array[TorrentBlock], - totalBlocks: Int, - totalBytes: Int) - extends Serializable { - - @transient var hasBlocks = 0 -} diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 17c64455b2429..978a6ded80829 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.broadcast -import org.apache.spark.storage.{BroadcastBlockId, _} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} import org.scalatest.FunSuite +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} +import org.apache.spark.storage._ + + class BroadcastSuite extends FunSuite with LocalSparkContext { private val httpConf = broadcastConf("HttpBroadcastFactory") @@ -124,12 +126,10 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { private def testUnpersistHttpBroadcast(distributed: Boolean, removeFromDriver: Boolean) { val numSlaves = if (distributed) 2 else 0 - def getBlockIds(id: Long) = Seq[BroadcastBlockId](BroadcastBlockId(id)) - // Verify that the broadcast file is created, and blocks are persisted only on the driver - def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - assert(blockIds.size === 1) - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + def afterCreation(broadcastId: Long, bmm: BlockManagerMaster) { + val blockId = BroadcastBlockId(broadcastId) + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === 1) statuses.head match { case (bm, status) => assert(bm.executorId === "", "Block should only be on the driver") @@ -139,14 +139,14 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { } if (distributed) { // this file is only generated in distributed mode - assert(HttpBroadcast.getFile(blockIds.head.broadcastId).exists, "Broadcast file not found!") + assert(HttpBroadcast.getFile(blockId.broadcastId).exists, "Broadcast file not found!") } } // Verify that blocks are persisted in both the executors and the driver - def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - assert(blockIds.size === 1) - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + def afterUsingBroadcast(broadcastId: Long, bmm: BlockManagerMaster) { + val blockId = BroadcastBlockId(broadcastId) + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === numSlaves + 1) statuses.foreach { case (_, status) => assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) @@ -157,21 +157,21 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver // is true. In the latter case, also verify that the broadcast file is deleted on the driver. - def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - assert(blockIds.size === 1) - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + def afterUnpersist(broadcastId: Long, bmm: BlockManagerMaster) { + val blockId = BroadcastBlockId(broadcastId) + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) val expectedNumBlocks = if (removeFromDriver) 0 else 1 val possiblyNot = if (removeFromDriver) "" else " not" assert(statuses.size === expectedNumBlocks, "Block should%s be unpersisted on the driver".format(possiblyNot)) if (distributed && removeFromDriver) { // this file is only generated in distributed mode - assert(!HttpBroadcast.getFile(blockIds.head.broadcastId).exists, + assert(!HttpBroadcast.getFile(blockId.broadcastId).exists, "Broadcast file should%s be deleted".format(possiblyNot)) } } - testUnpersistBroadcast(distributed, numSlaves, httpConf, getBlockIds, afterCreation, + testUnpersistBroadcast(distributed, numSlaves, httpConf, afterCreation, afterUsingBroadcast, afterUnpersist, removeFromDriver) } @@ -185,67 +185,51 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { private def testUnpersistTorrentBroadcast(distributed: Boolean, removeFromDriver: Boolean) { val numSlaves = if (distributed) 2 else 0 - def getBlockIds(id: Long) = { - val broadcastBlockId = BroadcastBlockId(id) - val metaBlockId = BroadcastBlockId(id, "meta") - // Assume broadcast value is small enough to fit into 1 piece - val pieceBlockId = BroadcastBlockId(id, "piece0") - if (distributed) { - // the metadata and piece blocks are generated only in distributed mode - Seq[BroadcastBlockId](broadcastBlockId, metaBlockId, pieceBlockId) - } else { - Seq[BroadcastBlockId](broadcastBlockId) - } + // Verify that blocks are persisted only on the driver + def afterCreation(broadcastId: Long, bmm: BlockManagerMaster) { + var blockId = BroadcastBlockId(broadcastId) + var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === 1) + + blockId = BroadcastBlockId(broadcastId, "piece0") + statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === (if (distributed) 1 else 0)) } - // Verify that blocks are persisted only on the driver - def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - blockIds.foreach { blockId => - val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(broadcastId: Long, bmm: BlockManagerMaster) { + var blockId = BroadcastBlockId(broadcastId) + var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + if (distributed) { + assert(statuses.size === numSlaves + 1) + } else { assert(statuses.size === 1) - statuses.head match { case (bm, status) => - assert(bm.executorId === "", "Block should only be on the driver") - assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) - assert(status.memSize > 0, "Block should be in memory store on the driver") - assert(status.diskSize === 0, "Block should not be in disk store on the driver") - } } - } - // Verify that blocks are persisted in both the executors and the driver - def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - blockIds.foreach { blockId => - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - if (blockId.field == "meta") { - // Meta data is only on the driver - assert(statuses.size === 1) - statuses.head match { case (bm, _) => assert(bm.executorId === "") } - } else { - // Other blocks are on both the executors and the driver - assert(statuses.size === numSlaves + 1, - blockId + " has " + statuses.size + " statuses: " + statuses.mkString(",")) - statuses.foreach { case (_, status) => - assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) - assert(status.memSize > 0, "Block should be in memory store") - assert(status.diskSize === 0, "Block should not be in disk store") - } - } + blockId = BroadcastBlockId(broadcastId, "piece0") + statuses = bmm.getBlockStatus(blockId, askSlaves = true) + if (distributed) { + assert(statuses.size === numSlaves + 1) + } else { + assert(statuses.size === 0) } } // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver // is true. - def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { - val expectedNumBlocks = if (removeFromDriver) 0 else 1 - val possiblyNot = if (removeFromDriver) "" else " not" - blockIds.foreach { blockId => - val statuses = bmm.getBlockStatus(blockId, askSlaves = true) - assert(statuses.size === expectedNumBlocks, - "Block should%s be unpersisted on the driver".format(possiblyNot)) - } + def afterUnpersist(broadcastId: Long, bmm: BlockManagerMaster) { + var blockId = BroadcastBlockId(broadcastId) + var expectedNumBlocks = if (removeFromDriver) 0 else 1 + var statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === expectedNumBlocks) + + blockId = BroadcastBlockId(broadcastId, "piece0") + expectedNumBlocks = if (removeFromDriver || !distributed) 0 else 1 + statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === expectedNumBlocks) } - testUnpersistBroadcast(distributed, numSlaves, torrentConf, getBlockIds, afterCreation, + testUnpersistBroadcast(distributed, numSlaves, torrentConf, afterCreation, afterUsingBroadcast, afterUnpersist, removeFromDriver) } @@ -262,10 +246,9 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { distributed: Boolean, numSlaves: Int, // used only when distributed = true broadcastConf: SparkConf, - getBlockIds: Long => Seq[BroadcastBlockId], - afterCreation: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, - afterUsingBroadcast: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, - afterUnpersist: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterCreation: (Long, BlockManagerMaster) => Unit, + afterUsingBroadcast: (Long, BlockManagerMaster) => Unit, + afterUnpersist: (Long, BlockManagerMaster) => Unit, removeFromDriver: Boolean) { sc = if (distributed) { @@ -278,15 +261,14 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { // Create broadcast variable val broadcast = sc.broadcast(list) - val blocks = getBlockIds(broadcast.id) - afterCreation(blocks, blockManagerMaster) + afterCreation(broadcast.id, blockManagerMaster) // Use broadcast variable on all executors val partitions = 10 assert(partitions > numSlaves) val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) - afterUsingBroadcast(blocks, blockManagerMaster) + afterUsingBroadcast(broadcast.id, blockManagerMaster) // Unpersist broadcast if (removeFromDriver) { @@ -294,7 +276,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { } else { broadcast.unpersist(blocking = true) } - afterUnpersist(blocks, blockManagerMaster) + afterUnpersist(broadcast.id, blockManagerMaster) // If the broadcast is removed from driver, all subsequent uses of the broadcast variable // should throw SparkExceptions. Otherwise, the result should be the same as before. From 0a984aa155fb7f532fe87620dcf1a2814c5b8b49 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 19 Aug 2014 22:16:22 -0700 Subject: [PATCH 0427/1492] [SPARK-3142][MLLIB] output shuffle data directly in Word2Vec Sorry I didn't realize this in #2043. Ishiihara Author: Xiangrui Meng Closes #2049 from mengxr/more-w2v and squashes the following commits: 050b1c5 [Xiangrui Meng] output shuffle data directly --- .../apache/spark/mllib/feature/Word2Vec.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index c3375ed44fd99..fc1444705364a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -347,19 +347,20 @@ class Word2Vec extends Serializable with Logging { } val syn0Local = model._1 val syn1Local = model._2 - val synOut = mutable.ListBuffer.empty[(Int, Array[Float])] - var index = 0 - while(index < vocabSize) { - if (syn0Modify(index) != 0) { - synOut += ((index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize))) + // Only output modified vectors. + Iterator.tabulate(vocabSize) { index => + if (syn0Modify(index) > 0) { + Some((index, syn0Local.slice(index * vectorSize, (index + 1) * vectorSize))) + } else { + None } - if (syn1Modify(index) != 0) { - synOut += ((index + vocabSize, - syn1Local.slice(index * vectorSize, (index + 1) * vectorSize))) + }.flatten ++ Iterator.tabulate(vocabSize) { index => + if (syn1Modify(index) > 0) { + Some((index + vocabSize, syn1Local.slice(index * vectorSize, (index + 1) * vectorSize))) + } else { + None } - index += 1 - } - synOut.toIterator + }.flatten } val synAgg = partial.reduceByKey { case (v1, v2) => blas.saxpy(vectorSize, 1.0f, v2, 1, v1, 1) From ebcb94f701273b56851dade677e047388a8bca09 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 19 Aug 2014 22:42:50 -0700 Subject: [PATCH 0428/1492] [SPARK-2974] [SPARK-2975] Fix two bugs related to spark.local.dirs This PR fixes two bugs related to `spark.local.dirs` and `SPARK_LOCAL_DIRS`, one where `Utils.getLocalDir()` might return an invalid directory (SPARK-2974) and another where the `SPARK_LOCAL_DIRS` override didn't affect the driver, which could cause problems when running tasks in local mode (SPARK-2975). This patch fixes both issues: the new `Utils.getOrCreateLocalRootDirs(conf: SparkConf)` utility method manages the creation of local directories and handles the precedence among the different configuration options, so we should see the same behavior whether we're running in local mode or on a worker. It's kind of a pain to mock out environment variables in tests (no easy way to mock System.getenv), so I added a `private[spark]` method to SparkConf for accessing environment variables (by default, it just delegates to System.getenv). By subclassing SparkConf and overriding this method, we can mock out SPARK_LOCAL_DIRS in tests. I also fixed a typo in PySpark where we used `SPARK_LOCAL_DIR` instead of `SPARK_LOCAL_DIRS` (I think this was technically innocuous, but it seemed worth fixing). Author: Josh Rosen Closes #2002 from JoshRosen/local-dirs and squashes the following commits: efad8c6 [Josh Rosen] Address review comments: 1dec709 [Josh Rosen] Minor updates to Javadocs. 7f36999 [Josh Rosen] Use env vars to detect if running in YARN container. 399ac25 [Josh Rosen] Update getLocalDir() documentation. bb3ad89 [Josh Rosen] Remove duplicated YARN getLocalDirs() code. 3e92d44 [Josh Rosen] Move local dirs override logic into Utils; fix bugs: b2c4736 [Josh Rosen] Add failing tests for SPARK-2974 and SPARK-2975. 007298b [Josh Rosen] Allow environment variables to be mocked in tests. 6d9259b [Josh Rosen] Fix typo in PySpark: SPARK_LOCAL_DIR should be SPARK_LOCAL_DIRS --- .../scala/org/apache/spark/SparkConf.scala | 8 ++- .../apache/spark/api/python/PythonRDD.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 25 ------- .../apache/spark/storage/BlockManager.scala | 3 +- .../spark/storage/DiskBlockManager.scala | 14 ++-- .../scala/org/apache/spark/util/Utils.scala | 67 +++++++++++++++++-- .../spark/storage/BlockManagerSuite.scala | 3 +- .../spark/storage/DiskBlockManagerSuite.scala | 4 +- .../apache/spark/storage/LocalDirsSuite.scala | 61 +++++++++++++++++ python/pyspark/shuffle.py | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 18 ----- .../spark/deploy/yarn/ExecutorLauncher.scala | 19 ------ .../spark/deploy/yarn/ApplicationMaster.scala | 18 ----- .../spark/deploy/yarn/ExecutorLauncher.scala | 19 ------ 14 files changed, 145 insertions(+), 118 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b4f321ec99e78..605df0e929faa 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -45,7 +45,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { /** Create a SparkConf that loads defaults from system properties and the classpath */ def this() = this(true) - private val settings = new HashMap[String, String]() + private[spark] val settings = new HashMap[String, String]() if (loadDefaults) { // Load any spark.* system properties @@ -210,6 +210,12 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } + /** + * By using this instead of System.getenv(), environment variables can be mocked + * in unit tests. + */ + private[spark] def getenv(name: String): String = System.getenv(name) + /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ private[spark] def validateSettings() { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 10210a2927dcc..747023812f754 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -62,7 +62,7 @@ private[spark] class PythonRDD( val env = SparkEnv.get val localdir = env.blockManager.diskBlockManager.localDirs.map( f => f.getPath()).mkString(",") - envVars += ("SPARK_LOCAL_DIR" -> localdir) // it's also used in monitor thread + envVars += ("SPARK_LOCAL_DIRS" -> localdir) // it's also used in monitor thread val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index fb3f7bd54bbfa..2f76e532aeb76 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -62,16 +62,6 @@ private[spark] class Executor( val conf = new SparkConf(true) conf.setAll(properties) - // If we are in yarn mode, systems can have different disk layouts so we must set it - // to what Yarn on this system said was available. This will be used later when SparkEnv - // created. - if (java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { - conf.set("spark.local.dir", getYarnLocalDirs()) - } else if (sys.env.contains("SPARK_LOCAL_DIRS")) { - conf.set("spark.local.dir", sys.env("SPARK_LOCAL_DIRS")) - } - if (!isLocal) { // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire @@ -134,21 +124,6 @@ private[spark] class Executor( threadPool.shutdown() } - /** Get the Yarn approved local directories. */ - private def getYarnLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .getOrElse(Option(System.getenv("LOCAL_DIRS")) - .getOrElse("")) - - if (localDirs.isEmpty) { - throw new Exception("Yarn Local dirs can't be empty") - } - localDirs - } - class TaskRunner( execBackend: ExecutorBackend, val taskId: Long, taskName: String, serializedTask: ByteBuffer) extends Runnable { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c0491fb55e3a4..12a92d44f4c36 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -67,8 +67,7 @@ private[spark] class BlockManager( private val port = conf.getInt("spark.blockManager.port", 0) val shuffleBlockManager = new ShuffleBlockManager(this, shuffleManager) - val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf) val connectionManager = new ConnectionManager(port, conf, securityManager, "Connection manager for block manager") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index f3da816389581..ec022ce9c048a 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -21,7 +21,7 @@ import java.io.File import java.text.SimpleDateFormat import java.util.{Date, Random, UUID} -import org.apache.spark.{SparkEnv, Logging} +import org.apache.spark.{SparkConf, SparkEnv, Logging} import org.apache.spark.executor.ExecutorExitCode import org.apache.spark.network.netty.PathResolver import org.apache.spark.util.Utils @@ -33,9 +33,10 @@ import org.apache.spark.shuffle.sort.SortShuffleManager * However, it is also possible to have a block map to only a segment of a file, by calling * mapBlockToFileSegment(). * - * @param rootDirs The directories to use for storing block files. Data will be hashed among these. + * Block files are hashed among the directories listed in spark.local.dir (or in + * SPARK_LOCAL_DIRS, if it's set). */ -private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, rootDirs: String) +private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, conf: SparkConf) extends PathResolver with Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @@ -46,7 +47,7 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid * having really large inodes at the top level. */ - val localDirs: Array[File] = createLocalDirs() + val localDirs: Array[File] = createLocalDirs(conf) if (localDirs.isEmpty) { logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) @@ -130,10 +131,9 @@ private[spark] class DiskBlockManager(shuffleBlockManager: ShuffleBlockManager, (blockId, getFile(blockId)) } - private def createLocalDirs(): Array[File] = { - logDebug(s"Creating local directories at root dirs '$rootDirs'") + private def createLocalDirs(conf: SparkConf): Array[File] = { val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") - rootDirs.split(",").flatMap { rootDir => + Utils.getOrCreateLocalRootDirs(conf).flatMap { rootDir => var foundLocalDir = false var localDir: File = null var localDirId: String = null diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 019f68b160894..d6d74ce269219 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -449,12 +449,71 @@ private[spark] object Utils extends Logging { } /** - * Get a temporary directory using Spark's spark.local.dir property, if set. This will always - * return a single directory, even though the spark.local.dir property might be a list of - * multiple paths. + * Get the path of a temporary directory. Spark's local directories can be configured through + * multiple settings, which are used with the following precedence: + * + * - If called from inside of a YARN container, this will return a directory chosen by YARN. + * - If the SPARK_LOCAL_DIRS environment variable is set, this will return a directory from it. + * - Otherwise, if the spark.local.dir is set, this will return a directory from it. + * - Otherwise, this will return java.io.tmpdir. + * + * Some of these configuration options might be lists of multiple paths, but this method will + * always return a single directory. */ def getLocalDir(conf: SparkConf): String = { - conf.get("spark.local.dir", System.getProperty("java.io.tmpdir")).split(',')(0) + getOrCreateLocalRootDirs(conf)(0) + } + + private[spark] def isRunningInYarnContainer(conf: SparkConf): Boolean = { + // These environment variables are set by YARN. + // For Hadoop 0.23.X, we check for YARN_LOCAL_DIRS (we use this below in getYarnLocalDirs()) + // For Hadoop 2.X, we check for CONTAINER_ID. + conf.getenv("CONTAINER_ID") != null || conf.getenv("YARN_LOCAL_DIRS") != null + } + + /** + * Gets or creates the directories listed in spark.local.dir or SPARK_LOCAL_DIRS, + * and returns only the directories that exist / could be created. + * + * If no directories could be created, this will return an empty list. + */ + private[spark] def getOrCreateLocalRootDirs(conf: SparkConf): Array[String] = { + val confValue = if (isRunningInYarnContainer(conf)) { + // If we are in yarn mode, systems can have different disk layouts so we must set it + // to what Yarn on this system said was available. + getYarnLocalDirs(conf) + } else { + Option(conf.getenv("SPARK_LOCAL_DIRS")).getOrElse( + conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))) + } + val rootDirs = confValue.split(',') + logDebug(s"Getting/creating local root dirs at '$confValue'") + + rootDirs.flatMap { rootDir => + val localDir: File = new File(rootDir) + val foundLocalDir = localDir.exists || localDir.mkdirs() + if (!foundLocalDir) { + logError(s"Failed to create local root dir in $rootDir. Ignoring this directory.") + None + } else { + Some(rootDir) + } + } + } + + /** Get the Yarn approved local directories. */ + private def getYarnLocalDirs(conf: SparkConf): String = { + // Hadoop 0.23 and 2.x have different Environment variable names for the + // local dirs, so lets check both. We assume one of the 2 is set. + // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X + val localDirs = Option(conf.getenv("YARN_LOCAL_DIRS")) + .getOrElse(Option(conf.getenv("LOCAL_DIRS")) + .getOrElse("")) + + if (localDirs.isEmpty) { + throw new Exception("Yarn Local dirs can't be empty") + } + localDirs } /** diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 20bac66105a69..f32ce6f9fcc7f 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -825,8 +825,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val blockManager = mock(classOf[BlockManager]) val shuffleBlockManager = mock(classOf[ShuffleBlockManager]) when(shuffleBlockManager.conf).thenReturn(conf) - val diskBlockManager = new DiskBlockManager(shuffleBlockManager, - System.getProperty("java.io.tmpdir")) + val diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf) when(blockManager.conf).thenReturn(conf.clone.set(confKey, 0.toString)) val diskStoreMapped = new DiskStore(blockManager, diskBlockManager) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 777579bc570db..aabaeadd7a071 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -71,7 +71,9 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before } override def beforeEach() { - diskBlockManager = new DiskBlockManager(shuffleBlockManager, rootDirs) + val conf = testConf.clone + conf.set("spark.local.dir", rootDirs) + diskBlockManager = new DiskBlockManager(shuffleBlockManager, conf) shuffleBlockManager.idToSegmentMap.clear() } diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala new file mode 100644 index 0000000000000..dae7bf0e336de --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.io.File + +import org.apache.spark.util.Utils +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf + + +/** + * Tests for the spark.local.dir and SPARK_LOCAL_DIRS configuration options. + */ +class LocalDirsSuite extends FunSuite { + + test("Utils.getLocalDir() returns a valid directory, even if some local dirs are missing") { + // Regression test for SPARK-2974 + assert(!new File("/NONEXISTENT_DIR").exists()) + val conf = new SparkConf(false) + .set("spark.local.dir", s"/NONEXISTENT_PATH,${System.getProperty("java.io.tmpdir")}") + assert(new File(Utils.getLocalDir(conf)).exists()) + } + + test("SPARK_LOCAL_DIRS override also affects driver") { + // Regression test for SPARK-2975 + assert(!new File("/NONEXISTENT_DIR").exists()) + // SPARK_LOCAL_DIRS is a valid directory: + class MySparkConf extends SparkConf(false) { + override def getenv(name: String) = { + if (name == "SPARK_LOCAL_DIRS") System.getProperty("java.io.tmpdir") + else super.getenv(name) + } + + override def clone: SparkConf = { + new MySparkConf().setAll(settings) + } + } + // spark.local.dir only contains invalid directories, but that's not a problem since + // SPARK_LOCAL_DIRS will override it on both the driver and workers: + val conf = new MySparkConf().set("spark.local.dir", "/NONEXISTENT_PATH") + assert(new File(Utils.getLocalDir(conf)).exists()) + } + +} diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 2c68cd4921deb..1ebe7df418327 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -214,7 +214,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, def _get_dirs(self): """ Get all the directories """ - path = os.environ.get("SPARK_LOCAL_DIR", "/tmp") + path = os.environ.get("SPARK_LOCAL_DIRS", "/tmp") dirs = path.split(",") return [os.path.join(d, "python", str(os.getpid()), str(id(self))) for d in dirs] diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 46a01f5a9a2cc..4d4848b1bd8f8 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -72,10 +72,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var registered = false def run() { - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - // set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box System.setProperty("spark.ui.port", "0") @@ -138,20 +134,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, params) } - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 72c7143edcd71..c3310fbc24a98 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -95,11 +95,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } def run() { - - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - appAttemptId = getApplicationAttemptId() resourceManager = registerWithResourceManager() @@ -152,20 +147,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp System.exit(0) } - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9c2bcf17a8508..1c4005fd8e78e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -72,10 +72,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var registered = false def run() { - // Setup the directories so things go to YARN approved directories rather - // than user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box System.setProperty("spark.ui.port", "0") @@ -144,20 +140,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", params) } - // Get the Yarn approved local directories. - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn local dirs can't be empty") - case Some(l) => l - } - } - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { logInfo("Registering the ApplicationMaster") amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index a7585748b7f88..45925f1fea005 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -94,11 +94,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp } def run() { - - // Setup the directories so things go to yarn approved directories rather - // then user specified and /tmp. - System.setProperty("spark.local.dir", getLocalDirs()) - amClient = AMRMClient.createAMRMClient() amClient.init(yarnConf) amClient.start() @@ -141,20 +136,6 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp System.exit(0) } - /** Get the Yarn approved local directories. */ - private def getLocalDirs(): String = { - // Hadoop 0.23 and 2.x have different Environment variable names for the - // local dirs, so lets check both. We assume one of the 2 is set. - // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X - val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) - .orElse(Option(System.getenv("LOCAL_DIRS"))) - - localDirs match { - case None => throw new Exception("Yarn Local dirs can't be empty") - case Some(l) => l - } - } - private def registerApplicationMaster(): RegisterApplicationMasterResponse = { val appUIAddress = sparkConf.get("spark.driver.appUIAddress", "") logInfo(s"Registering the ApplicationMaster with appUIAddress: $appUIAddress") From 8a74e4b2a8c7dab154b406539487cf29d578d208 Mon Sep 17 00:00:00 2001 From: Ken Takagiwa Date: Tue, 19 Aug 2014 22:43:22 -0700 Subject: [PATCH 0429/1492] [DOCS] Fixed wrong links Author: Ken Takagiwa Closes #2042 from giwa/patch-1 and squashes the following commits: 216fe0e [Ken Takagiwa] Fixed wrong links --- docs/streaming-custom-receivers.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 1e045a3dd0ca9..27cd085782f66 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -186,7 +186,7 @@ JavaDStream words = lines.flatMap(new FlatMapFunction() ... {% endhighlight %} -The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java). +The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java).
    @@ -215,7 +215,7 @@ And a new input stream can be created with this custom actor as val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver") {% endhighlight %} -See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala) +See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala) for an end-to-end example. From 0a7ef6339f18e68d703599aff7db2dd9c2003866 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 19 Aug 2014 22:43:49 -0700 Subject: [PATCH 0430/1492] [SPARK-3141] [PySpark] fix sortByKey() with take() Fix sortByKey() with take() The function `f` used in mapPartitions should always return an iterator. Author: Davies Liu Closes #2045 from davies/fix_sortbykey and squashes the following commits: 1160f59 [Davies Liu] fix sortByKey() with take() --- python/pyspark/rdd.py | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 140cbe05a43b0..3eefc878d274e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -575,6 +575,8 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): # noqa >>> tmp = [('a', 1), ('b', 2), ('1', 3), ('d', 4), ('2', 5)] + >>> sc.parallelize(tmp).sortByKey().first() + ('1', 3) >>> sc.parallelize(tmp).sortByKey(True, 1).collect() [('1', 3), ('2', 5), ('a', 1), ('b', 2), ('d', 4)] >>> sc.parallelize(tmp).sortByKey(True, 2).collect() @@ -587,14 +589,13 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() + def sortPartition(iterator): + return iter(sorted(iterator, key=lambda (k, v): keyfunc(k), reverse=not ascending)) + if numPartitions == 1: if self.getNumPartitions() > 1: self = self.coalesce(1) - - def sort(iterator): - return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) - - return self.mapPartitions(sort) + return self.mapPartitions(sortPartition) # first compute the boundary of each part via sampling: we want to partition # the key-space into bins such that the bins have roughly the same @@ -610,17 +611,14 @@ def sort(iterator): bounds = [samples[len(samples) * (i + 1) / numPartitions] for i in range(0, numPartitions - 1)] - def rangePartitionFunc(k): + def rangePartitioner(k): p = bisect.bisect_left(bounds, keyfunc(k)) if ascending: return p else: return numPartitions - 1 - p - def mapFunc(iterator): - return sorted(iterator, reverse=(not ascending), key=lambda (k, v): keyfunc(k)) - - return self.partitionBy(numPartitions, rangePartitionFunc).mapPartitions(mapFunc, True) + return self.partitionBy(numPartitions, rangePartitioner).mapPartitions(sortPartition, True) def sortBy(self, keyfunc, ascending=True, numPartitions=None): """ From 8c5a2226932c572898c76eb6fab9283f02ad4103 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 20 Aug 2014 04:09:54 -0700 Subject: [PATCH 0431/1492] [SPARK-3054][STREAMING] Add unit tests for Spark Sink. This patch adds unit tests for Spark Sink. It also removes the private[flume] for Spark Sink, since the sink is instantiated from Flume configuration (looks like this is ignored by reflection which is used by Flume, but we should still remove it anyway). Author: Hari Shreedharan Author: Hari Shreedharan Closes #1958 from harishreedharan/spark-sink-test and squashes the following commits: e3110b9 [Hari Shreedharan] Add a sleep to allow sink to commit the transactions 120b81e [Hari Shreedharan] Fix complexity in threading model in test 4df5be6 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into spark-sink-test c9190d1 [Hari Shreedharan] Indentation and spaces changes 7fedc5a [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into spark-sink-test abc20cb [Hari Shreedharan] Minor test changes 7b9b649 [Hari Shreedharan] Merge branch 'master' into spark-sink-test f2c56c9 [Hari Shreedharan] Update SparkSinkSuite.scala a24aac8 [Hari Shreedharan] Remove unused var c86d615 [Hari Shreedharan] [SPARK-3054][STREAMING] Add unit tests for Spark Sink. --- external/flume-sink/pom.xml | 7 + .../streaming/flume/sink/SparkSink.scala | 1 - .../streaming/flume/sink/SparkSinkSuite.scala | 204 ++++++++++++++++++ .../flume/FlumePollingStreamSuite.scala | 2 +- 4 files changed, 212 insertions(+), 2 deletions(-) create mode 100644 external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d0bf1cf1ea796..0c68defa5e101 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -72,6 +72,13 @@ org.scalatest scalatest_${scala.binary.version}
    + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + test +
    target/scala-${scala.binary.version}/classes diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 948af5947f5e1..98ae7d783aec8 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -53,7 +53,6 @@ import org.apache.flume.sink.AbstractSink * */ -private[flume] class SparkSink extends AbstractSink with Logging with Configurable { // Size of the pool to use for holding transaction processors. diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala new file mode 100644 index 0000000000000..44b27edf85ce8 --- /dev/null +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.flume.sink + +import java.net.InetSocketAddress +import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{TimeUnit, CountDownLatch, Executors} + +import scala.collection.JavaConversions._ +import scala.concurrent.{ExecutionContext, Future} +import scala.util.{Failure, Success} + +import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.avro.ipc.NettyTransceiver +import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.flume.Context +import org.apache.flume.channel.MemoryChannel +import org.apache.flume.event.EventBuilder +import org.apache.spark.streaming.TestSuiteBase +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory + +class SparkSinkSuite extends TestSuiteBase { + val eventsPerBatch = 1000 + val channelCapacity = 5000 + + test("Success") { + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + client.ack(events.getSequenceNumber) + assert(events.getEvents.size() === 1000) + assertChannelIsEmpty(channel) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Nack") { + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + client.nack(events.getSequenceNumber) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Timeout") { + val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig + .CONF_TRANSACTION_TIMEOUT -> 1.toString)) + channel.start() + sink.start() + putEvents(channel, eventsPerBatch) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + + val (transceiver, client) = getTransceiverAndClient(address, 1)(0) + val events = client.getEventBatch(1000) + assert(events.getEvents.size() === 1000) + Thread.sleep(1000) + assert(availableChannelSlots(channel) === 4000) + sink.stop() + channel.stop() + transceiver.close() + } + + test("Multiple consumers") { + testMultipleConsumers(failSome = false) + } + + test("Multiple consumers with some failures") { + testMultipleConsumers(failSome = true) + } + + def testMultipleConsumers(failSome: Boolean): Unit = { + implicit val executorContext = ExecutionContext + .fromExecutorService(Executors.newFixedThreadPool(5)) + val (channel, sink) = initializeChannelAndSink() + channel.start() + sink.start() + (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) + val port = sink.getPort + val address = new InetSocketAddress("0.0.0.0", port) + val transceiversAndClients = getTransceiverAndClient(address, 5) + val batchCounter = new CountDownLatch(5) + val counter = new AtomicInteger(0) + transceiversAndClients.foreach(x => { + Future { + val client = x._2 + val events = client.getEventBatch(1000) + if (!failSome || counter.getAndIncrement() % 2 == 0) { + client.ack(events.getSequenceNumber) + } else { + client.nack(events.getSequenceNumber) + throw new RuntimeException("Sending NACK for failure!") + } + events + }.onComplete { + case Success(events) => + assert(events.getEvents.size() === 1000) + batchCounter.countDown() + case Failure(t) => + // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout + batchCounter.countDown() + } + }) + batchCounter.await() + TimeUnit.SECONDS.sleep(1) // Allow the sink to commit the transactions. + executorContext.shutdown() + if(failSome) { + assert(availableChannelSlots(channel) === 3000) + } else { + assertChannelIsEmpty(channel) + } + sink.stop() + channel.stop() + transceiversAndClients.foreach(x => x._1.close()) + } + + private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty): (MemoryChannel, + SparkSink) = { + val channel = new MemoryChannel() + val channelContext = new Context() + + channelContext.put("capacity", channelCapacity.toString) + channelContext.put("transactionCapacity", 1000.toString) + channelContext.put("keep-alive", 0.toString) + channelContext.putAll(overrides) + channel.configure(channelContext) + + val sink = new SparkSink() + val sinkContext = new Context() + sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") + sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) + sink.configure(sinkContext) + sink.setChannel(channel) + (channel, sink) + } + + private def putEvents(ch: MemoryChannel, count: Int): Unit = { + val tx = ch.getTransaction + tx.begin() + (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) + tx.commit() + tx.close() + } + + private def getTransceiverAndClient(address: InetSocketAddress, + count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { + + (1 to count).map(_ => { + lazy val channelFactoryExecutor = + Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). + setNameFormat("Flume Receiver Channel Thread - %d").build()) + lazy val channelFactory = + new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) + val transceiver = new NettyTransceiver(address, channelFactory) + val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) + (transceiver, client) + }) + } + + private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { + assert(availableChannelSlots(channel) === channelCapacity) + } + + private def availableChannelSlots(channel: MemoryChannel): Int = { + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") + queueRemaining.setAccessible(true) + val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") + m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] + } +} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 8a85b0f987e42..32a19787a28e1 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -198,7 +198,7 @@ class FlumePollingStreamSuite extends TestSuiteBase { } def assertChannelIsEmpty(channel: MemoryChannel) = { - val queueRemaining = channel.getClass.getDeclaredField("queueRemaining"); + val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") queueRemaining.setAccessible(true) val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") assert(m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] === 5000) From f2f26c2a1dc6d60078c3be9c3d11a21866d9a24f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 Aug 2014 12:13:31 -0700 Subject: [PATCH 0432/1492] SPARK-3092 [SQL]: Always include the thriftserver when -Phive is enabled. Currently we have a separate profile called hive-thriftserver. I originally suggested this in case users did not want to bundle the thriftserver, but it's ultimately lead to a lot of confusion. Since the thriftserver is only a few classes, I don't see a really good reason to isolate it from the rest of Hive. So let's go ahead and just include it in the same profile to simplify things. This has been suggested in the past by liancheng. Author: Patrick Wendell Closes #2006 from pwendell/hiveserver and squashes the following commits: 742ea40 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into hiveserver 034ad47 [Patrick Wendell] SPARK-3092: Always include the thriftserver when -Phive is enabled. --- README.md | 6 +----- assembly/pom.xml | 5 ----- dev/create-release/create-release.sh | 10 +++++----- dev/run-tests | 2 +- dev/scalastyle | 2 +- docs/building-with-maven.md | 8 ++------ docs/sql-programming-guide.md | 4 +--- pom.xml | 2 +- 8 files changed, 12 insertions(+), 27 deletions(-) diff --git a/README.md b/README.md index a1a48f5bd0819..8906e4c1416b1 100644 --- a/README.md +++ b/README.md @@ -118,11 +118,7 @@ If your project is built with Maven, add this to your POM file's ` ## A Note About Thrift JDBC server and CLI for Spark SQL Spark SQL supports Thrift JDBC server and CLI. -See sql-programming-guide.md for more information about those features. -You can use those features by setting `-Phive-thriftserver` when building Spark as follows. - - $ sbt/sbt -Phive-thriftserver assembly - +See sql-programming-guide.md for more information about using the JDBC server. ## Configuration diff --git a/assembly/pom.xml b/assembly/pom.xml index 703f15925bc44..9fbb037115db3 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -163,11 +163,6 @@ spark-hive_${scala.binary.version} ${project.version}
    -
    - - - hive-thriftserver - org.apache.spark spark-hive-thriftserver_${scala.binary.version} diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 28f26d2368254..905dec0ced383 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -60,14 +60,14 @@ if [[ ! "$@" =~ --package-only ]]; then -Dmaven.javadoc.skip=true \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dmaven.javadoc.skip=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ -Dmaven.javadoc.skip=true \ - -Pyarn -Phive -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ + -Pyarn -Phive -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ release:perform cd .. @@ -117,10 +117,10 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & -make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & +make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" & +make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & make_binary_release "hadoop2" \ - "-Phive -Phive-thriftserver -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & + "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & make_binary_release "hadoop2-without-hive" \ "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & wait diff --git a/dev/run-tests b/dev/run-tests index 132f696d6447a..20a67cfb361b9 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -99,7 +99,7 @@ echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled: if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" fi # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, diff --git a/dev/scalastyle b/dev/scalastyle index b53053a04ff42..eb9b467965636 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 4d87ab92cec5b..a7d7bd3ccb1f2 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -98,12 +98,8 @@ mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -Dski # Building Thrift JDBC server and CLI for Spark SQL -Spark SQL supports Thrift JDBC server and CLI. -See sql-programming-guide.md for more information about those features. -You can use those features by setting `-Phive-thriftserver` when building Spark as follows. -{% highlight bash %} -mvn -Phive-thriftserver assembly -{% endhighlight %} +Spark SQL supports Thrift JDBC server and CLI. See sql-programming-guide.md for +more information about the JDBC server. # Spark Tests in Maven diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 34accade36ea9..c41f2804a6021 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -578,9 +578,7 @@ evaluated by the SQL execution engine. A full list of the functions supported c The Thrift JDBC server implemented here corresponds to the [`HiveServer2`] (https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test -the JDBC server with the beeline script comes with either Spark or Hive 0.12. In order to use Hive -you must first run '`sbt/sbt -Phive-thriftserver assembly/assembly`' (or use `-Phive-thriftserver` -for maven). +the JDBC server with the beeline script comes with either Spark or Hive 0.12. To start the JDBC server, run the following in the Spark directory: diff --git a/pom.xml b/pom.xml index 0d44cf4ea5f92..dd4c4ee80a0df 100644 --- a/pom.xml +++ b/pom.xml @@ -1179,7 +1179,7 @@ - hive-thriftserver + hive false From ceb19830b88486faa87ff41e18d03ede713a73cc Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 20 Aug 2014 12:18:41 -0700 Subject: [PATCH 0433/1492] BUILD: Bump Hadoop versions in the release build. Also, minor modifications to the MapR profile. --- dev/create-release/create-release.sh | 10 +++---- pom.xml | 39 +++++++++++++++++++++++++--- 2 files changed, 40 insertions(+), 9 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 905dec0ced383..eab6313733dfd 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -118,11 +118,11 @@ make_binary_release() { } make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" & -make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & -make_binary_release "hadoop2" \ - "-Phive -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & -make_binary_release "hadoop2-without-hive" \ - "-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -Pyarn.version=2.2.0" & +make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Pyarn" & +make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Pyarn" & +make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & +make_binary_release "mapr3" "-Pmapr3 -Pyarn -Phive" & +make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive" & wait # Copy data diff --git a/pom.xml b/pom.xml index dd4c4ee80a0df..7ed07ad7df88d 100644 --- a/pom.xml +++ b/pom.xml @@ -1115,18 +1115,49 @@ - mapr + mapr3 false 1.0.3-mapr-3.0.3 - 2.3.0-mapr-4.0.0-beta - 0.94.17-mapr-1403 - 3.4.5-mapr-1401 + 2.3.0-mapr-4.0.0-FCS + 0.94.17-mapr-1405 + 3.4.5-mapr-1406 + + mapr4 + + false + + + 2.3.0-mapr-4.0.0-FCS + 2.3.0-mapr-4.0.0-FCS + 0.94.17-mapr-1405-4.0.0-FCS + 3.4.5-mapr-1406 + + + + org.apache.curator + curator-recipes + 2.4.0 + + + org.apache.zookeeper + zookeeper + + + + + org.apache.zookeeper + zookeeper + 3.4.5-mapr-1406 + + + + hadoop-provided From cf46e725814f575ebb417e80d2571bccc6dac4a7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 20 Aug 2014 12:57:39 -0700 Subject: [PATCH 0434/1492] [SPARK-3126][SPARK-3127][SQL] Fixed HiveThriftServer2Suite This PR fixes two issues: 1. Fixes wrongly quoted command line option in `HiveThriftServer2Suite` that makes test cases hang until timeout. 1. Asks `dev/run-test` to run Spark SQL tests when `bin/spark-sql` and/or `sbin/start-thriftserver.sh` are modified. Author: Cheng Lian Closes #2036 from liancheng/fix-thriftserver-test and squashes the following commits: f38c4eb [Cheng Lian] Fixed the same quotation issue in CliSuite 26b82a0 [Cheng Lian] Run SQL tests when dff contains bin/spark-sql and/or sbin/start-thriftserver.sh a87f83d [Cheng Lian] Extended timeout e5aa31a [Cheng Lian] Fixed metastore JDBC URI quotation --- dev/run-tests | 2 +- .../spark/sql/hive/thriftserver/CliSuite.scala | 2 +- .../thriftserver/HiveThriftServer2Suite.scala | 18 ++++-------------- 3 files changed, 6 insertions(+), 16 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 20a67cfb361b9..d751961605dfd 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -55,7 +55,7 @@ JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..* # Partial solution for SPARK-1455. Only run Hive tests if there are sql changes. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - diffs=`git diff --name-only master | grep "^sql/"` + diffs=`git diff --name-only master | grep "^\(sql/\)\|\(bin/spark-sql\)\|\(sbin/start-thriftserver.sh\)"` if [ -n "$diffs" ]; then echo "Detected changes in SQL. Will run Hive test suite." _RUN_SQL_TESTS=true diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 2bf8cfdcacd22..70bea1ed80fda 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -32,7 +32,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { val commands = s"""../../bin/spark-sql | --master local - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$WAREHOUSE_PATH """.stripMargin.split("\\s+") diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index aedef6ce1f5f2..326b0a7275b34 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -51,9 +51,6 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt port } - // If verbose is true, the test program will print all outputs coming from the Hive Thrift server. - val VERBOSE = Option(System.getenv("SPARK_SQL_TEST_VERBOSE")).getOrElse("false").toBoolean - Class.forName(DRIVER_NAME) override def beforeAll() { launchServer() } @@ -68,8 +65,7 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt val command = s"""../../sbin/start-thriftserver.sh | --master local - | --hiveconf hive.root.logger=INFO,console - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}="$jdbcUrl" + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$METASTORE_PATH | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$HOST | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$PORT @@ -77,12 +73,10 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt val pb = new ProcessBuilder(command ++ args: _*) val environment = pb.environment() - environment.put("HIVE_SERVER2_THRIFT_PORT", PORT.toString) - environment.put("HIVE_SERVER2_THRIFT_BIND_HOST", HOST) process = pb.start() inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on") + waitForOutput(inputReader, "ThriftBinaryCLIService listening on", 300000) // Spawn a thread to read the output from the forked process. // Note that this is necessary since in some configurations, log4j could be blocked @@ -91,12 +85,8 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt while (true) { val stdout = readFrom(inputReader) val stderr = readFrom(errorReader) - if (VERBOSE && stdout.length > 0) { - println(stdout) - } - if (VERBOSE && stderr.length > 0) { - println(stderr) - } + print(stdout) + print(stderr) Thread.sleep(50) } } From 0ea46ac80089e9091d247704b17afbc423c0060d Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 20 Aug 2014 13:26:11 -0700 Subject: [PATCH 0435/1492] [SPARK-3062] [SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled #1891 was to avoid IOException when EventLogging is enabled. The solution used ShutdownHookManager but it was defined only Hadoop 2.x. Hadoop 1.x don't have ShutdownHookManager so #1891 doesn't compile on Hadoop 1.x Now, I had a compromised solution for both Hadoop 1.x and 2.x. Only for FileLogger, an unique FileSystem object is created. Author: Kousuke Saruta Closes #1970 from sarutak/SPARK-2970 and squashes the following commits: 240c91e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2970 0e7b45d [Kousuke Saruta] Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled" e1262ec [Kousuke Saruta] Modified Filelogger to use unique FileSystem instance --- .../scala/org/apache/spark/util/FileLogger.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 2e8fbf5a91ee7..ad8b79af877d8 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -52,7 +52,20 @@ private[spark] class FileLogger( override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } - private val fileSystem = Utils.getHadoopFileSystem(logDir) + /** + * To avoid effects of FileSystem#close or FileSystem.closeAll called from other modules, + * create unique FileSystem instance only for FileLogger + */ + private val fileSystem = { + val conf = SparkHadoopUtil.get.newConfiguration() + val logUri = new URI(logDir) + val scheme = logUri.getScheme + if (scheme == "hdfs") { + conf.setBoolean("fs.hdfs.impl.disable.cache", true) + } + FileSystem.get(logUri, conf) + } + var fileIndex = 0 // Only used if compression is enabled From c1ba4cd6b4db22a9325eee50dc40a78593a10de1 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 20 Aug 2014 14:04:39 -0700 Subject: [PATCH 0436/1492] [SPARK-3149] Connection establishment information is not enough. Author: Kousuke Saruta Closes #2060 from sarutak/SPARK-3149 and squashes the following commits: 1cc89af [Kousuke Saruta] Modified log message of accepting connection --- .../main/scala/org/apache/spark/network/ConnectionManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index b3e951ded6e77..e5e1e72cd912b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -418,7 +418,7 @@ private[spark] class ConnectionManager( newConnection.onReceive(receiveMessage) addListeners(newConnection) addConnection(newConnection) - logInfo("Accepted connection from [" + newConnection.remoteAddress.getAddress + "]") + logInfo("Accepted connection from [" + newConnection.remoteAddress + "]") } catch { // might happen in case of issues with registering with selector case e: Exception => logError("Error in accept loop", e) From b3ec51bfd795772ff96d18228e979a52ebc82ec4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 20 Aug 2014 15:01:47 -0700 Subject: [PATCH 0437/1492] [SPARK-2849] Handle driver configs separately in client mode In client deploy mode, the driver is launched from within `SparkSubmit`'s JVM. This means by the time we parse Spark configs from `spark-defaults.conf`, it is already too late to control certain properties of the driver's JVM. We currently ignore these configs in client mode altogether. ``` spark.driver.memory spark.driver.extraJavaOptions spark.driver.extraClassPath spark.driver.extraLibraryPath ``` This PR handles these properties before launching the driver JVM. It achieves this by spawning a separate JVM that runs a new class called `SparkSubmitDriverBootstrapper`, which spawns `SparkSubmit` as a sub-process with the appropriate classpath, library paths, java opts and memory. Author: Andrew Or Closes #1845 from andrewor14/handle-configs-bash and squashes the following commits: bed4bdf [Andrew Or] Change a few comments / messages (minor) 24dba60 [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 08fd788 [Andrew Or] Warn against external usages of SparkSubmitDriverBootstrapper ff34728 [Andrew Or] Minor comments 51aeb01 [Andrew Or] Filter out JVM memory in Scala rather than Bash (minor) 9a778f6 [Andrew Or] Fix PySpark: actually kill driver on termination d0f20db [Andrew Or] Don't pass empty library paths, classpath, java opts etc. a78cb26 [Andrew Or] Revert a few changes in utils.sh (minor) 9ba37e2 [Andrew Or] Don't barf when the properties file does not exist 8867a09 [Andrew Or] A few more naming things (minor) 19464ad [Andrew Or] SPARK_SUBMIT_JAVA_OPTS -> SPARK_SUBMIT_OPTS d6488f9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 1ea6bbe [Andrew Or] SparkClassLauncher -> SparkSubmitDriverBootstrapper a91ea19 [Andrew Or] Fix precedence of library paths, classpath, java opts and memory 158f813 [Andrew Or] Remove "client mode" boolean argument c84f5c8 [Andrew Or] Remove debug print statement (minor) b71f52b [Andrew Or] Revert a few more changes (minor) 7d94a8d [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 3a8235d [Andrew Or] Only parse the properties file if special configs exist c37e08d [Andrew Or] Revert a few more changes a396eda [Andrew Or] Nullify my own hard work to simplify bash 0effa1e [Andrew Or] Add code in Scala that handles special configs c886568 [Andrew Or] Fix lines too long + a few comments / style (minor) 7a4190a [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 7396be2 [Andrew Or] Explicitly comment that multi-line properties are not supported fa11ef8 [Andrew Or] Parse the properties file only if the special configs exist 371cac4 [Andrew Or] Add function prefix (minor) be99eb3 [Andrew Or] Fix tests to not include multi-line configs bd0d468 [Andrew Or] Simplify parsing config file by ignoring multi-line arguments 56ac247 [Andrew Or] Use eval and set to simplify splitting 8d4614c [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash aeb79c7 [Andrew Or] Merge branch 'master' of github.com:apache/spark into handle-configs-bash 2732ac0 [Andrew Or] Integrate BASH tests into dev/run-tests + log error properly 8d26a5c [Andrew Or] Add tests for bash/utils.sh 4ae24c3 [Andrew Or] Fix bug: escape properly in quote_java_property b3c4cd5 [Andrew Or] Fix bug: count the number of quotes instead of detecting presence c2273fc [Andrew Or] Fix typo (minor) e793e5f [Andrew Or] Handle multi-line arguments 5d8f8c4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra c7b9926 [Andrew Or] Minor changes to spark-defaults.conf.template a992ae2 [Andrew Or] Escape spark.*.extraJavaOptions correctly aabfc7e [Andrew Or] escape -> split (minor) 45a1eb9 [Andrew Or] Fix bug: escape escaped backslashes and quotes properly... 1cdc6b1 [Andrew Or] Fix bug: escape escaped double quotes properly c854859 [Andrew Or] Add small comment c13a2cb [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra 8e552b7 [Andrew Or] Include an example of spark.*.extraJavaOptions de765c9 [Andrew Or] Print spark-class command properly a4df3c4 [Andrew Or] Move parsing and escaping logic to utils.sh dec2343 [Andrew Or] Only export variables if they exist fa2136e [Andrew Or] Escape Java options + parse java properties files properly ef12f74 [Andrew Or] Minor formatting 4ec22a1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra e5cfb46 [Andrew Or] Collapse duplicate code + fix potential whitespace issues 4edcaa8 [Andrew Or] Redirect stdout to stderr for python 130f295 [Andrew Or] Handle spark.driver.memory too 98dd8e3 [Andrew Or] Add warning if properties file does not exist 8843562 [Andrew Or] Fix compilation issues... 75ee6b4 [Andrew Or] Remove accidentally added file 63ed2e9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into submit-driver-extra 0025474 [Andrew Or] Revert SparkSubmit handling of --driver-* options for only cluster mode a2ab1b0 [Andrew Or] Parse spark.driver.extra* in bash 250cb95 [Andrew Or] Do not ignore spark.driver.extra* for client mode --- bin/spark-class | 49 ++++-- bin/spark-submit | 28 +++- bin/utils.sh | 0 conf/spark-defaults.conf.template | 10 +- .../apache/spark/api/python/PythonUtils.scala | 25 --- .../api/python/PythonWorkerFactory.scala | 3 +- .../apache/spark/deploy/PythonRunner.scala | 4 +- .../org/apache/spark/deploy/SparkSubmit.scala | 17 +- .../SparkSubmitDriverBootstrapper.scala | 149 ++++++++++++++++++ .../scala/org/apache/spark/util/Utils.scala | 21 +++ 10 files changed, 250 insertions(+), 56 deletions(-) mode change 100644 => 100755 bin/utils.sh create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala diff --git a/bin/spark-class b/bin/spark-class index 3f6beca5becf0..22acf92288b3b 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -17,6 +17,8 @@ # limitations under the License. # +# NOTE: Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! + cygwin=false case "`uname`" in CYGWIN*) cygwin=true;; @@ -39,7 +41,7 @@ fi if [ -n "$SPARK_MEM" ]; then echo -e "Warning: SPARK_MEM is deprecated, please use a more specific config option" 1>&2 - echo -e "(e.g., spark.executor.memory or SPARK_DRIVER_MEMORY)." 1>&2 + echo -e "(e.g., spark.executor.memory or spark.driver.memory)." 1>&2 fi # Use SPARK_MEM or 512m as the default memory, to be overridden by specific options @@ -73,11 +75,17 @@ case "$1" in OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; - # Spark submit uses SPARK_SUBMIT_OPTS and SPARK_JAVA_OPTS - 'org.apache.spark.deploy.SparkSubmit') - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS \ - -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" + # Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + + # SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. + 'org.apache.spark.deploy.SparkSubmit') + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} + if [ -n "$SPARK_SUBMIT_LIBRARY_PATH" ]; then + OUR_JAVA_OPTS="$OUR_JAVA_OPTS -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" + fi + if [ -n "$SPARK_SUBMIT_DRIVER_MEMORY" ]; then + OUR_JAVA_MEM="$SPARK_SUBMIT_DRIVER_MEMORY" + fi ;; *) @@ -101,11 +109,12 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" + # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" fi -export JAVA_OPTS + # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! TOOLS_DIR="$FWDIR"/tools @@ -146,10 +155,28 @@ if $cygwin; then fi export CLASSPATH -if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then - echo -n "Spark Command: " 1>&2 - echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" 1>&2 - echo -e "========================================\n" 1>&2 +# In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. +# Here we must parse the properties file for relevant "spark.driver.*" configs before launching +# the driver JVM itself. Instead of handling this complexity in Bash, we launch a separate JVM +# to prepare the launch environment of this driver JVM. + +if [ -n "$SPARK_SUBMIT_BOOTSTRAP_DRIVER" ]; then + # This is used only if the properties file actually contains these special configs + # Export the environment variables needed by SparkSubmitDriverBootstrapper + export RUNNER + export CLASSPATH + export JAVA_OPTS + export OUR_JAVA_MEM + export SPARK_CLASS=1 + shift # Ignore main class (org.apache.spark.deploy.SparkSubmit) and use our own + exec "$RUNNER" org.apache.spark.deploy.SparkSubmitDriverBootstrapper "$@" +else + # Note: The format of this command is closely echoed in SparkSubmitDriverBootstrapper.scala + if [ -n "$SPARK_PRINT_LAUNCH_COMMAND" ]; then + echo -n "Spark Command: " 1>&2 + echo "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" 1>&2 + echo -e "========================================\n" 1>&2 + fi + exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" fi -exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 9e7cecedd0325..32c911cd0438b 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -17,14 +17,18 @@ # limitations under the License. # +# NOTE: Any changes in this file must be reflected in SparkClassLauncher.scala! + export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=("$@") while (($#)); do if [ "$1" = "--deploy-mode" ]; then - DEPLOY_MODE=$2 + SPARK_SUBMIT_DEPLOY_MODE=$2 + elif [ "$1" = "--properties-file" ]; then + SPARK_SUBMIT_PROPERTIES_FILE=$2 elif [ "$1" = "--driver-memory" ]; then - DRIVER_MEMORY=$2 + export SPARK_SUBMIT_DRIVER_MEMORY=$2 elif [ "$1" = "--driver-library-path" ]; then export SPARK_SUBMIT_LIBRARY_PATH=$2 elif [ "$1" = "--driver-class-path" ]; then @@ -35,10 +39,24 @@ while (($#)); do shift done -DEPLOY_MODE=${DEPLOY_MODE:-"client"} +DEFAULT_PROPERTIES_FILE="$SPARK_HOME/conf/spark-defaults.conf" +export SPARK_SUBMIT_DEPLOY_MODE=${SPARK_SUBMIT_DEPLOY_MODE:-"client"} +export SPARK_SUBMIT_PROPERTIES_FILE=${SPARK_SUBMIT_PROPERTIES_FILE:-"$DEFAULT_PROPERTIES_FILE"} + +# For client mode, the driver will be launched in the same JVM that launches +# SparkSubmit, so we may need to read the properties file for any extra class +# paths, library paths, java options and memory early on. Otherwise, it will +# be too late by the time the driver JVM has started. -if [ -n "$DRIVER_MEMORY" ] && [ $DEPLOY_MODE == "client" ]; then - export SPARK_DRIVER_MEMORY=$DRIVER_MEMORY +if [[ "$SPARK_SUBMIT_DEPLOY_MODE" == "client" && -f "$SPARK_SUBMIT_PROPERTIES_FILE" ]]; then + # Parse the properties file only if the special configs exist + contains_special_configs=$( + grep -e "spark.driver.extra*\|spark.driver.memory" "$SPARK_SUBMIT_PROPERTIES_FILE" | \ + grep -v "^[[:space:]]*#" + ) + if [ -n "$contains_special_configs" ]; then + export SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 + fi fi exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/bin/utils.sh b/bin/utils.sh old mode 100644 new mode 100755 diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index 2779342769c14..94427029b94d7 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -2,7 +2,9 @@ # This is useful for setting default environmental settings. # Example: -# spark.master spark://master:7077 -# spark.eventLog.enabled true -# spark.eventLog.dir hdfs://namenode:8021/directory -# spark.serializer org.apache.spark.serializer.KryoSerializer +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory +# spark.serializer org.apache.spark.serializer.KryoSerializer +# spark.driver.memory 5g +# spark.executor.extraJavaOptions -XX:+PrintGCDetail -Dkey=value -Dnumbers="one two three" diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 52c70712eea3d..be5ebfa9219d3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -40,28 +40,3 @@ private[spark] object PythonUtils { paths.filter(_ != "").mkString(File.pathSeparator) } } - - -/** - * A utility class to redirect the child process's stdout or stderr. - */ -private[spark] class RedirectThread( - in: InputStream, - out: OutputStream, - name: String) - extends Thread(name) { - - setDaemon(true) - override def run() { - scala.util.control.Exception.ignoring(classOf[IOException]) { - // FIXME: We copy the stream on the level of bytes to avoid encoding problems. - val buf = new Array[Byte](1024) - var len = in.read(buf) - while (len != -1) { - out.write(buf, 0, len) - out.flush() - len = in.read(buf) - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index bf716a8ab025b..4c4796f6c59ba 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,7 +17,6 @@ package org.apache.spark.api.python -import java.lang.Runtime import java.io.{DataOutputStream, DataInputStream, InputStream, OutputStreamWriter} import java.net.{InetAddress, ServerSocket, Socket, SocketException} @@ -25,7 +24,7 @@ import scala.collection.mutable import scala.collection.JavaConversions._ import org.apache.spark._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{RedirectThread, Utils} private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 0d6751f3fa6d2..b66c3ba4d5fb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -22,8 +22,8 @@ import java.net.URI import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ -import org.apache.spark.api.python.{PythonUtils, RedirectThread} -import org.apache.spark.util.Utils +import org.apache.spark.api.python.PythonUtils +import org.apache.spark.util.{RedirectThread, Utils} /** * A main class used by spark-submit to launch Python applications. It executes python as a diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 318509a67a36f..f8cdbc3c392b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -195,18 +195,21 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), // Other options - OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, CLUSTER, - sysProp = "spark.driver.extraClassPath"), - OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, CLUSTER, - sysProp = "spark.driver.extraJavaOptions"), - OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, CLUSTER, - sysProp = "spark.driver.extraLibraryPath"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, sysProp = "spark.cores.max"), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS, ALL_DEPLOY_MODES, - sysProp = "spark.files") + sysProp = "spark.files"), + + // Only process driver specific options for cluster mode here, + // because they have already been processed in bash for client mode + OptionAssigner(args.driverExtraClassPath, STANDALONE | YARN, CLUSTER, + sysProp = "spark.driver.extraClassPath"), + OptionAssigner(args.driverExtraJavaOptions, STANDALONE | YARN, CLUSTER, + sysProp = "spark.driver.extraJavaOptions"), + OptionAssigner(args.driverExtraLibraryPath, STANDALONE | YARN, CLUSTER, + sysProp = "spark.driver.extraLibraryPath") ) // In client mode, launch the application main class directly diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala new file mode 100644 index 0000000000000..af607e6a4a065 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.io.File + +import scala.collection.JavaConversions._ + +import org.apache.spark.util.{RedirectThread, Utils} + +/** + * Launch an application through Spark submit in client mode with the appropriate classpath, + * library paths, java options and memory. These properties of the JVM must be set before the + * driver JVM is launched. The sole purpose of this class is to avoid handling the complexity + * of parsing the properties file for such relevant configs in Bash. + * + * Usage: org.apache.spark.deploy.SparkSubmitDriverBootstrapper + */ +private[spark] object SparkSubmitDriverBootstrapper { + + // Note: This class depends on the behavior of `bin/spark-class` and `bin/spark-submit`. + // Any changes made there must be reflected in this file. + + def main(args: Array[String]): Unit = { + + // This should be called only from `bin/spark-class` + if (!sys.env.contains("SPARK_CLASS")) { + System.err.println("SparkSubmitDriverBootstrapper must be called from `bin/spark-class`!") + System.exit(1) + } + + val submitArgs = args + val runner = sys.env("RUNNER") + val classpath = sys.env("CLASSPATH") + val javaOpts = sys.env("JAVA_OPTS") + val defaultDriverMemory = sys.env("OUR_JAVA_MEM") + + // Spark submit specific environment variables + val deployMode = sys.env("SPARK_SUBMIT_DEPLOY_MODE") + val propertiesFile = sys.env("SPARK_SUBMIT_PROPERTIES_FILE") + val bootstrapDriver = sys.env("SPARK_SUBMIT_BOOTSTRAP_DRIVER") + val submitDriverMemory = sys.env.get("SPARK_SUBMIT_DRIVER_MEMORY") + val submitLibraryPath = sys.env.get("SPARK_SUBMIT_LIBRARY_PATH") + val submitClasspath = sys.env.get("SPARK_SUBMIT_CLASSPATH") + val submitJavaOpts = sys.env.get("SPARK_SUBMIT_OPTS") + + assume(runner != null, "RUNNER must be set") + assume(classpath != null, "CLASSPATH must be set") + assume(javaOpts != null, "JAVA_OPTS must be set") + assume(defaultDriverMemory != null, "OUR_JAVA_MEM must be set") + assume(deployMode == "client", "SPARK_SUBMIT_DEPLOY_MODE must be \"client\"!") + assume(propertiesFile != null, "SPARK_SUBMIT_PROPERTIES_FILE must be set") + assume(bootstrapDriver != null, "SPARK_SUBMIT_BOOTSTRAP_DRIVER must be set") + + // Parse the properties file for the equivalent spark.driver.* configs + val properties = SparkSubmitArguments.getPropertiesFromFile(new File(propertiesFile)).toMap + val confDriverMemory = properties.get("spark.driver.memory") + val confLibraryPath = properties.get("spark.driver.extraLibraryPath") + val confClasspath = properties.get("spark.driver.extraClassPath") + val confJavaOpts = properties.get("spark.driver.extraJavaOptions") + + // Favor Spark submit arguments over the equivalent configs in the properties file. + // Note that we do not actually use the Spark submit values for library path, classpath, + // and Java opts here, because we have already captured them in Bash. + + val newDriverMemory = submitDriverMemory + .orElse(confDriverMemory) + .getOrElse(defaultDriverMemory) + + val newLibraryPath = + if (submitLibraryPath.isDefined) { + // SPARK_SUBMIT_LIBRARY_PATH is already captured in JAVA_OPTS + "" + } else { + confLibraryPath.map("-Djava.library.path=" + _).getOrElse("") + } + + val newClasspath = + if (submitClasspath.isDefined) { + // SPARK_SUBMIT_CLASSPATH is already captured in CLASSPATH + classpath + } else { + classpath + confClasspath.map(sys.props("path.separator") + _).getOrElse("") + } + + val newJavaOpts = + if (submitJavaOpts.isDefined) { + // SPARK_SUBMIT_OPTS is already captured in JAVA_OPTS + javaOpts + } else { + javaOpts + confJavaOpts.map(" " + _).getOrElse("") + } + + val filteredJavaOpts = Utils.splitCommandString(newJavaOpts) + .filterNot(_.startsWith("-Xms")) + .filterNot(_.startsWith("-Xmx")) + + // Build up command + val command: Seq[String] = + Seq(runner) ++ + Seq("-cp", newClasspath) ++ + Seq(newLibraryPath) ++ + filteredJavaOpts ++ + Seq(s"-Xms$newDriverMemory", s"-Xmx$newDriverMemory") ++ + Seq("org.apache.spark.deploy.SparkSubmit") ++ + submitArgs + + // Print the launch command. This follows closely the format used in `bin/spark-class`. + if (sys.env.contains("SPARK_PRINT_LAUNCH_COMMAND")) { + System.err.print("Spark Command: ") + System.err.println(command.mkString(" ")) + System.err.println("========================================\n") + } + + // Start the driver JVM + val filteredCommand = command.filter(_.nonEmpty) + val builder = new ProcessBuilder(filteredCommand) + val process = builder.start() + + // Redirect stdin, stdout, and stderr to/from the child JVM + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") + val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") + stdinThread.start() + stdoutThread.start() + stderrThread.start() + + // Terminate on broken pipe, which signals that the parent process has exited. This is + // important for the PySpark shell, where Spark submit itself is a python subprocess. + stdinThread.join() + process.destroy() + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d6d74ce269219..69a84a3604a52 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1480,3 +1480,24 @@ private[spark] object Utils extends Logging { } } + +/** + * A utility class to redirect the child process's stdout or stderr. + */ +private[spark] class RedirectThread(in: InputStream, out: OutputStream, name: String) + extends Thread(name) { + + setDaemon(true) + override def run() { + scala.util.control.Exception.ignoring(classOf[IOException]) { + // FIXME: We copy the stream on the level of bytes to avoid encoding problems. + val buf = new Array[Byte](1024) + var len = in.read(buf) + while (len != -1) { + out.write(buf, 0, len) + out.flush() + len = in.read(buf) + } + } + } +} From fb60bec34e0b20ae95b4b865a79744916e0a5737 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 20 Aug 2014 15:37:27 -0700 Subject: [PATCH 0438/1492] [SPARK-2298] Encode stage attempt in SparkListener & UI. Simple way to reproduce this in the UI: ```scala val f = new java.io.File("/tmp/test") f.delete() sc.parallelize(1 to 2, 2).map(x => (x,x )).repartition(3).mapPartitionsWithContext { case (context, iter) => if (context.partitionId == 0) { val f = new java.io.File("/tmp/test") if (!f.exists) { f.mkdir() System.exit(0); } } iter }.count() ``` Author: Reynold Xin Closes #1545 from rxin/stage-attempt and squashes the following commits: 3ee1d2a [Reynold Xin] - Rename attempt to retry in UI. - Properly report stage failure in FetchFailed. 40a6bd5 [Reynold Xin] Updated test suites. c414c36 [Reynold Xin] Fixed the hanging in JobCancellationSuite. b3e2eed [Reynold Xin] Oops previous code didn't compile. 0f36075 [Reynold Xin] Mark unknown stage attempt with id -1 and drop that in JobProgressListener. 6c08b07 [Reynold Xin] Addressed code review feedback. 4e5faa2 [Reynold Xin] [SPARK-2298] Encode stage attempt in SparkListener & UI. --- .../apache/spark/scheduler/DAGScheduler.scala | 77 +-- .../spark/scheduler/SparkListener.scala | 11 +- .../org/apache/spark/scheduler/Stage.scala | 8 +- .../apache/spark/scheduler/StageInfo.scala | 11 +- .../spark/scheduler/TaskSchedulerImpl.scala | 8 +- .../org/apache/spark/scheduler/TaskSet.scala | 4 - .../apache/spark/ui/jobs/ExecutorTable.scala | 6 +- .../spark/ui/jobs/JobProgressListener.scala | 40 +- .../org/apache/spark/ui/jobs/StagePage.scala | 11 +- .../org/apache/spark/ui/jobs/StageTable.scala | 14 +- .../org/apache/spark/util/JsonProtocol.scala | 12 +- .../storage/StorageStatusListenerSuite.scala | 17 +- .../ui/jobs/JobProgressListenerSuite.scala | 68 +-- .../spark/ui/storage/StorageTabSuite.scala | 16 +- .../apache/spark/util/JsonProtocolSuite.scala | 476 ++++++++++++++---- 15 files changed, 555 insertions(+), 224 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b86cfbfa48fbe..34131984570e4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -164,7 +164,7 @@ class DAGScheduler( */ def executorHeartbeatReceived( execId: String, - taskMetrics: Array[(Long, Int, TaskMetrics)], // (taskId, stageId, metrics) + taskMetrics: Array[(Long, Int, Int, TaskMetrics)], // (taskId, stageId, stateAttempt, metrics) blockManagerId: BlockManagerId): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, taskMetrics)) implicit val timeout = Timeout(600 seconds) @@ -677,7 +677,10 @@ class DAGScheduler( } private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { - listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) + // Note that there is a chance that this task is launched after the stage is cancelled. + // In that case, we wouldn't have the stage anymore in stageIdToStage. + val stageAttemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1) + listenerBus.post(SparkListenerTaskStart(task.stageId, stageAttemptId, taskInfo)) submitWaitingStages() } @@ -695,8 +698,8 @@ class DAGScheduler( // is in the process of getting stopped. val stageFailedMessage = "Stage cancelled because SparkContext was shut down" runningStages.foreach { stage => - stage.info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(stage.info)) + stage.latestInfo.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) } listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } @@ -781,7 +784,16 @@ class DAGScheduler( logDebug("submitMissingTasks(" + stage + ")") // Get our pending tasks and remember them in our pendingTasks entry stage.pendingTasks.clear() - var tasks = ArrayBuffer[Task[_]]() + + // First figure out the indexes of partition ids to compute. + val partitionsToCompute: Seq[Int] = { + if (stage.isShuffleMap) { + (0 until stage.numPartitions).filter(id => stage.outputLocs(id) == Nil) + } else { + val job = stage.resultOfJob.get + (0 until job.numPartitions).filter(id => !job.finished(id)) + } + } val properties = if (jobIdToActiveJob.contains(jobId)) { jobIdToActiveJob(stage.jobId).properties @@ -795,7 +807,8 @@ class DAGScheduler( // serializable. If tasks are not serializable, a SparkListenerStageCompleted event // will be posted, which should always come after a corresponding SparkListenerStageSubmitted // event. - listenerBus.post(SparkListenerStageSubmitted(stage.info, properties)) + stage.latestInfo = StageInfo.fromStage(stage, Some(partitionsToCompute.size)) + listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties)) // TODO: Maybe we can keep the taskBinary in Stage to avoid serializing it multiple times. // Broadcasted binary for the task, used to dispatch tasks to executors. Note that we broadcast @@ -826,20 +839,19 @@ class DAGScheduler( return } - if (stage.isShuffleMap) { - for (p <- 0 until stage.numPartitions if stage.outputLocs(p) == Nil) { - val locs = getPreferredLocs(stage.rdd, p) - val part = stage.rdd.partitions(p) - tasks += new ShuffleMapTask(stage.id, taskBinary, part, locs) + val tasks: Seq[Task[_]] = if (stage.isShuffleMap) { + partitionsToCompute.map { id => + val locs = getPreferredLocs(stage.rdd, id) + val part = stage.rdd.partitions(id) + new ShuffleMapTask(stage.id, taskBinary, part, locs) } } else { - // This is a final stage; figure out its job's missing partitions val job = stage.resultOfJob.get - for (id <- 0 until job.numPartitions if !job.finished(id)) { + partitionsToCompute.map { id => val p: Int = job.partitions(id) val part = stage.rdd.partitions(p) val locs = getPreferredLocs(stage.rdd, p) - tasks += new ResultTask(stage.id, taskBinary, part, locs, id) + new ResultTask(stage.id, taskBinary, part, locs, id) } } @@ -869,11 +881,11 @@ class DAGScheduler( logDebug("New pending tasks: " + stage.pendingTasks) taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) - stage.info.submissionTime = Some(clock.getTime()) + stage.latestInfo.submissionTime = Some(clock.getTime()) } else { // Because we posted SparkListenerStageSubmitted earlier, we should post // SparkListenerStageCompleted here in case there are no tasks to run. - listenerBus.post(SparkListenerStageCompleted(stage.info)) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) runningStages -= stage @@ -892,8 +904,9 @@ class DAGScheduler( // The success case is dealt with separately below, since we need to compute accumulator // updates before posting. if (event.reason != Success) { - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, - event.taskMetrics)) + val attemptId = stageIdToStage.get(task.stageId).map(_.latestInfo.attemptId).getOrElse(-1) + listenerBus.post(SparkListenerTaskEnd(stageId, attemptId, taskType, event.reason, + event.taskInfo, event.taskMetrics)) } if (!stageIdToStage.contains(task.stageId)) { @@ -902,14 +915,19 @@ class DAGScheduler( } val stage = stageIdToStage(task.stageId) - def markStageAsFinished(stage: Stage) = { - val serviceTime = stage.info.submissionTime match { + def markStageAsFinished(stage: Stage, errorMessage: Option[String] = None) = { + val serviceTime = stage.latestInfo.submissionTime match { case Some(t) => "%.03f".format((clock.getTime() - t) / 1000.0) case _ => "Unknown" } - logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) - stage.info.completionTime = Some(clock.getTime()) - listenerBus.post(SparkListenerStageCompleted(stage.info)) + if (errorMessage.isEmpty) { + logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) + stage.latestInfo.completionTime = Some(clock.getTime()) + } else { + stage.latestInfo.stageFailed(errorMessage.get) + logInfo("%s (%s) failed in %s s".format(stage, stage.name, serviceTime)) + } + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) runningStages -= stage } event.reason match { @@ -924,7 +942,7 @@ class DAGScheduler( val name = acc.name.get val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) val stringValue = Accumulators.stringifyValue(acc.value) - stage.info.accumulables(id) = AccumulableInfo(id, name, stringValue) + stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) event.taskInfo.accumulables += AccumulableInfo(id, name, Some(stringPartialValue), stringValue) } @@ -935,8 +953,8 @@ class DAGScheduler( logError(s"Failed to update accumulators for $task", e) } } - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, - event.taskMetrics)) + listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, + event.reason, event.taskInfo, event.taskMetrics)) stage.pendingTasks -= task task match { case rt: ResultTask[_, _] => @@ -1029,6 +1047,7 @@ class DAGScheduler( case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) + markStageAsFinished(failedStage, Some("Fetch failure")) runningStages -= failedStage // TODO: Cancel running tasks in the stage logInfo("Marking " + failedStage + " (" + failedStage.name + @@ -1142,7 +1161,7 @@ class DAGScheduler( } val dependentJobs: Seq[ActiveJob] = activeJobs.filter(job => stageDependsOn(job.finalStage, failedStage)).toSeq - failedStage.info.completionTime = Some(clock.getTime()) + failedStage.latestInfo.completionTime = Some(clock.getTime()) for (job <- dependentJobs) { failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } @@ -1182,8 +1201,8 @@ class DAGScheduler( if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask taskScheduler.cancelTasks(stageId, shouldInterruptThread) - stage.info.stageFailed(failureReason) - listenerBus.post(SparkListenerStageCompleted(stage.info)) + stage.latestInfo.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stage.latestInfo)) } catch { case e: UnsupportedOperationException => logInfo(s"Could not cancel tasks for stage $stageId", e) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d01d318633877..86ca8445a1124 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -39,7 +39,8 @@ case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Propert case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent @DeveloperApi -case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent +case class SparkListenerTaskStart(stageId: Int, stageAttemptId: Int, taskInfo: TaskInfo) + extends SparkListenerEvent @DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent @@ -47,6 +48,7 @@ case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListe @DeveloperApi case class SparkListenerTaskEnd( stageId: Int, + stageAttemptId: Int, taskType: String, reason: TaskEndReason, taskInfo: TaskInfo, @@ -75,10 +77,15 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +/** + * Periodic updates from executors. + * @param execId executor id + * @param taskMetrics sequence of (task id, stage id, stage attempt, metrics) + */ @DeveloperApi case class SparkListenerExecutorMetricsUpdate( execId: String, - taskMetrics: Seq[(Long, Int, TaskMetrics)]) + taskMetrics: Seq[(Long, Int, Int, TaskMetrics)]) extends SparkListenerEvent @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 800905413d145..071568cdfb429 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -43,6 +43,9 @@ import org.apache.spark.util.CallSite * stage, the callSite gives the user code that created the RDD being shuffled. For a result * stage, the callSite gives the user code that executes the associated action (e.g. count()). * + * A single stage can consist of multiple attempts. In that case, the latestInfo field will + * be updated for each attempt. + * */ private[spark] class Stage( val id: Int, @@ -71,8 +74,8 @@ private[spark] class Stage( val name = callSite.shortForm val details = callSite.longForm - /** Pointer to the [StageInfo] object, set by DAGScheduler. */ - var info: StageInfo = StageInfo.fromStage(this) + /** Pointer to the latest [StageInfo] object, set by DAGScheduler. */ + var latestInfo: StageInfo = StageInfo.fromStage(this) def isAvailable: Boolean = { if (!isShuffleMap) { @@ -116,6 +119,7 @@ private[spark] class Stage( } } + /** Return a new attempt id, starting with 0. */ def newAttemptId(): Int = { val id = nextAttemptId nextAttemptId += 1 diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 2a407e47a05bd..c6dc3369ba5cc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -29,6 +29,7 @@ import org.apache.spark.storage.RDDInfo @DeveloperApi class StageInfo( val stageId: Int, + val attemptId: Int, val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo], @@ -56,9 +57,15 @@ private[spark] object StageInfo { * shuffle dependencies. Therefore, all ancestor RDDs related to this Stage's RDD through a * sequence of narrow dependencies should also be associated with this Stage. */ - def fromStage(stage: Stage): StageInfo = { + def fromStage(stage: Stage, numTasks: Option[Int] = None): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos - new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos, stage.details) + new StageInfo( + stage.id, + stage.attemptId, + stage.name, + numTasks.getOrElse(stage.numTasks), + rddInfos, + stage.details) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 6c0d1b2752a81..ad051e59af86d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -333,12 +333,12 @@ private[spark] class TaskSchedulerImpl( execId: String, taskMetrics: Array[(Long, TaskMetrics)], // taskId -> TaskMetrics blockManagerId: BlockManagerId): Boolean = { - val metricsWithStageIds = taskMetrics.flatMap { - case (id, metrics) => { + + val metricsWithStageIds: Array[(Long, Int, Int, TaskMetrics)] = synchronized { + taskMetrics.flatMap { case (id, metrics) => taskIdToTaskSetId.get(id) .flatMap(activeTaskSets.get) - .map(_.stageId) - .map(x => (id, x, metrics)) + .map(taskSetMgr => (id, taskSetMgr.stageId, taskSetMgr.taskSet.attempt, metrics)) } } dagScheduler.executorHeartbeatReceived(execId, metricsWithStageIds, blockManagerId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 613fa7850bb25..c3ad325156f53 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -31,9 +31,5 @@ private[spark] class TaskSet( val properties: Properties) { val id: String = stageId + "." + attempt - def kill(interruptThread: Boolean) { - tasks.foreach(_.kill(interruptThread)) - } - override def toString: String = "TaskSet " + id } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 0cc51c873727d..2987dc04494a5 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -24,8 +24,8 @@ import org.apache.spark.ui.{ToolTips, UIUtils} import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils -/** Page showing executor summary */ -private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { +/** Stage summary grouped by executors. */ +private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobProgressTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { @@ -65,7 +65,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { executorIdToAddress.put(executorId, address) } - listener.stageIdToData.get(stageId) match { + listener.stageIdToData.get((stageId, stageAttemptId)) match { case Some(stageData: StageUIData) => stageData.executorSummary.toSeq.sortBy(_._1).map { case (k, v) => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 74cd637d88155..f7f918fd521a9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -43,12 +43,16 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // How many stages to remember val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) - val activeStages = HashMap[Int, StageInfo]() + // Map from stageId to StageInfo + val activeStages = new HashMap[Int, StageInfo] + + // Map from (stageId, attemptId) to StageUIData + val stageIdToData = new HashMap[(Int, Int), StageUIData] + val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() - val stageIdToData = new HashMap[Int, StageUIData] - + // Map from pool name to a hash map (map from stage id to StageInfo). val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() val executorIdToBlockManagerId = HashMap[String, BlockManagerId]() @@ -59,9 +63,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo - val stageId = stage.stageId - val stageData = stageIdToData.getOrElseUpdate(stageId, { - logWarning("Stage completed for unknown stage " + stageId) + val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), { + logWarning("Stage completed for unknown stage " + stage.stageId) new StageUIData }) @@ -69,8 +72,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.accumulables(id) = info } - poolToActiveStages.get(stageData.schedulingPool).foreach(_.remove(stageId)) - activeStages.remove(stageId) + poolToActiveStages.get(stageData.schedulingPool).foreach { hashMap => + hashMap.remove(stage.stageId) + } + activeStages.remove(stage.stageId) if (stage.failureReason.isEmpty) { completedStages += stage trimIfNecessary(completedStages) @@ -84,7 +89,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) - stages.take(toRemove).foreach { s => stageIdToData.remove(s.stageId) } + stages.take(toRemove).foreach { s => stageIdToData.remove((s.stageId, s.attemptId)) } stages.trimStart(toRemove) } } @@ -98,21 +103,21 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) }.getOrElse(DEFAULT_POOL_NAME) - val stageData = stageIdToData.getOrElseUpdate(stage.stageId, new StageUIData) + val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), new StageUIData) stageData.schedulingPool = poolName stageData.description = Option(stageSubmitted.properties).flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_DESCRIPTION)) } - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]()) + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]) stages(stage.stageId) = stage } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { val taskInfo = taskStart.taskInfo if (taskInfo != null) { - val stageData = stageIdToData.getOrElseUpdate(taskStart.stageId, { + val stageData = stageIdToData.getOrElseUpdate((taskStart.stageId, taskStart.stageAttemptId), { logWarning("Task start for unknown stage " + taskStart.stageId) new StageUIData }) @@ -128,8 +133,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo - if (info != null) { - val stageData = stageIdToData.getOrElseUpdate(taskEnd.stageId, { + // If stage attempt id is -1, it means the DAGScheduler had no idea which attempt this task + // compeletion event is for. Let's just drop it here. This means we might have some speculation + // tasks on the web ui that's never marked as complete. + if (info != null && taskEnd.stageAttemptId != -1) { + val stageData = stageIdToData.getOrElseUpdate((taskEnd.stageId, taskEnd.stageAttemptId), { logWarning("Task end for unknown stage " + taskEnd.stageId) new StageUIData }) @@ -222,8 +230,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } override def onExecutorMetricsUpdate(executorMetricsUpdate: SparkListenerExecutorMetricsUpdate) { - for ((taskId, sid, taskMetrics) <- executorMetricsUpdate.taskMetrics) { - val stageData = stageIdToData.getOrElseUpdate(sid, { + for ((taskId, sid, sAttempt, taskMetrics) <- executorMetricsUpdate.taskMetrics) { + val stageData = stageIdToData.getOrElseUpdate((sid, sAttempt), { logWarning("Metrics update for task in unknown stage " + sid) new StageUIData }) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index d4eb02722ad12..db01be596e073 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -34,7 +34,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt - val stageDataOption = listener.stageIdToData.get(stageId) + val stageAttemptId = request.getParameter("attempt").toInt + val stageDataOption = listener.stageIdToData.get((stageId, stageAttemptId)) if (stageDataOption.isEmpty || stageDataOption.get.taskData.isEmpty) { val content = @@ -42,14 +43,15 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {

    Summary Metrics

    No tasks have started yet

    Tasks

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

    Accumulators

    ++ accumulableTable } else Seq() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 16ad0df45aa0d..2e67310594784 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -97,8 +97,8 @@ private[ui] class StageTableBase( } // scalastyle:on - val nameLinkUri ="%s/stages/stage?id=%s" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" + .format(UIUtils.prependBaseUri(parent.basePath), s.stageId, s.attemptId) val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -121,7 +121,7 @@ private[ui] class StageTableBase( } val stageDesc = for { - stageData <- listener.stageIdToData.get(s.stageId) + stageData <- listener.stageIdToData.get((s.stageId, s.attemptId)) desc <- stageData.description } yield {
    {desc}
    @@ -131,7 +131,7 @@ private[ui] class StageTableBase( } protected def stageRow(s: StageInfo): Seq[Node] = { - val stageDataOption = listener.stageIdToData.get(s.stageId) + val stageDataOption = listener.stageIdToData.get((s.stageId, s.attemptId)) if (stageDataOption.isEmpty) { return {s.stageId}No data available for this stage } @@ -154,7 +154,11 @@ private[ui] class StageTableBase( val shuffleWrite = stageData.shuffleWriteBytes val shuffleWriteWithUnit = if (shuffleWrite > 0) Utils.bytesToString(shuffleWrite) else "" - {s.stageId} ++ + {if (s.attemptId > 0) { + {s.stageId} (retry {s.attemptId}) + } else { + {s.stageId} + }} ++ {if (isFairScheduler) { Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ + ("Stage Attempt ID" -> taskStart.stageAttemptId) ~ ("Task Info" -> taskInfoToJson(taskInfo)) } @@ -112,6 +113,7 @@ private[spark] object JsonProtocol { val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ + ("Stage Attempt ID" -> taskEnd.stageAttemptId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ ("Task Info" -> taskInfoToJson(taskInfo)) ~ @@ -187,6 +189,7 @@ private[spark] object JsonProtocol { val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ + ("Stage Attempt ID" -> stageInfo.attemptId) ~ ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ @@ -419,8 +422,9 @@ private[spark] object JsonProtocol { def taskStartFromJson(json: JValue): SparkListenerTaskStart = { val stageId = (json \ "Stage ID").extract[Int] + val stageAttemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) val taskInfo = taskInfoFromJson(json \ "Task Info") - SparkListenerTaskStart(stageId, taskInfo) + SparkListenerTaskStart(stageId, stageAttemptId, taskInfo) } def taskGettingResultFromJson(json: JValue): SparkListenerTaskGettingResult = { @@ -430,11 +434,12 @@ private[spark] object JsonProtocol { def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { val stageId = (json \ "Stage ID").extract[Int] + val stageAttemptId = (json \ "Stage Attempt ID").extractOpt[Int].getOrElse(0) val taskType = (json \ "Task Type").extract[String] val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") val taskInfo = taskInfoFromJson(json \ "Task Info") val taskMetrics = taskMetricsFromJson(json \ "Task Metrics") - SparkListenerTaskEnd(stageId, taskType, taskEndReason, taskInfo, taskMetrics) + SparkListenerTaskEnd(stageId, stageAttemptId, taskType, taskEndReason, taskInfo, taskMetrics) } def jobStartFromJson(json: JValue): SparkListenerJobStart = { @@ -492,6 +497,7 @@ private[spark] object JsonProtocol { def stageInfoFromJson(json: JValue): StageInfo = { val stageId = (json \ "Stage ID").extract[Int] + val attemptId = (json \ "Attempt ID").extractOpt[Int].getOrElse(0) val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson(_)) @@ -504,7 +510,7 @@ private[spark] object JsonProtocol { case None => Seq[AccumulableInfo]() } - val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos, details) + val stageInfo = new StageInfo(stageId, attemptId, stageName, numTasks, rddInfos, details) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 51fb646a3cb61..7671cb969a26b 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -69,10 +69,10 @@ class StorageStatusListenerSuite extends FunSuite { // Task end with no updated blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics)) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics)) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) } @@ -92,13 +92,13 @@ class StorageStatusListenerSuite extends FunSuite { // Task end with new blocks assert(listener.executorIdToStorageStatus("big").numBlocks === 0) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 2) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) @@ -111,13 +111,14 @@ class StorageStatusListenerSuite extends FunSuite { val droppedBlock3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.NONE, 0L, 0L, 0L)) taskMetrics1.updatedBlocks = Some(Seq(droppedBlock1, droppedBlock3)) taskMetrics2.updatedBlocks = Some(Seq(droppedBlock2, droppedBlock3)) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) + + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 1) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) assert(listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 2))) assert(listener.executorIdToStorageStatus("fat").containsBlock(RDDBlockId(4, 0))) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo2, taskMetrics2)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo2, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 1) assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) assert(!listener.executorIdToStorageStatus("big").containsBlock(RDDBlockId(1, 1))) @@ -135,8 +136,8 @@ class StorageStatusListenerSuite extends FunSuite { val block3 = (RDDBlockId(4, 0), BlockStatus(StorageLevel.DISK_ONLY, 0L, 300L, 0L)) taskMetrics1.updatedBlocks = Some(Seq(block1, block2)) taskMetrics2.updatedBlocks = Some(Seq(block3)) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics1)) - listener.onTaskEnd(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo1, taskMetrics2)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics1)) + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo1, taskMetrics2)) assert(listener.executorIdToStorageStatus("big").numBlocks === 3) // Unpersist RDD diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 147ec0bc52e39..3370dd4156c3f 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -34,12 +34,12 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val listener = new JobProgressListener(conf) def createStageStartEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, stageId.toString, 0, null, "") + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") SparkListenerStageSubmitted(stageInfo) } def createStageEndEvent(stageId: Int) = { - val stageInfo = new StageInfo(stageId, stageId.toString, 0, null, "") + val stageInfo = new StageInfo(stageId, 0, stageId.toString, 0, null, "") SparkListenerStageCompleted(stageInfo) } @@ -70,33 +70,37 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskInfo.finishTime = 1 var task = new ShuffleMapTask(0) val taskType = Utils.getFormattedClassName(task) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) - .shuffleRead === 1000) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + assert(listener.stageIdToData.getOrElse((0, 0), fail()) + .executorSummary.getOrElse("exe-1", fail()).shuffleRead === 1000) // finish a task with unknown executor-id, nothing should happen taskInfo = new TaskInfo(1234L, 0, 1, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL, true) taskInfo.finishTime = 1 task = new ShuffleMapTask(0) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToData.size === 1) // finish this task, should get updated duration taskInfo = new TaskInfo(1235L, 0, 1, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-1", fail()) - .shuffleRead === 2000) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + assert(listener.stageIdToData.getOrElse((0, 0), fail()) + .executorSummary.getOrElse("exe-1", fail()).shuffleRead === 2000) // finish this task, should get updated duration taskInfo = new TaskInfo(1236L, 0, 2, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = 1 task = new ShuffleMapTask(0) - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) - assert(listener.stageIdToData.getOrElse(0, fail()).executorSummary.getOrElse("exe-2", fail()) - .shuffleRead === 1000) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, Success, taskInfo, taskMetrics)) + assert(listener.stageIdToData.getOrElse((0, 0), fail()) + .executorSummary.getOrElse("exe-2", fail()).shuffleRead === 1000) } test("test task success vs failure counting for different task end reasons") { @@ -119,16 +123,18 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc UnknownReason) var failCount = 0 for (reason <- taskFailedReasons) { - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, reason, taskInfo, metrics)) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 0, taskType, reason, taskInfo, metrics)) failCount += 1 - assert(listener.stageIdToData(task.stageId).numCompleteTasks === 0) - assert(listener.stageIdToData(task.stageId).numFailedTasks === failCount) + assert(listener.stageIdToData((task.stageId, 0)).numCompleteTasks === 0) + assert(listener.stageIdToData((task.stageId, 0)).numFailedTasks === failCount) } // Make sure we count success as success. - listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, metrics)) - assert(listener.stageIdToData(task.stageId).numCompleteTasks === 1) - assert(listener.stageIdToData(task.stageId).numFailedTasks === failCount) + listener.onTaskEnd( + SparkListenerTaskEnd(task.stageId, 1, taskType, Success, taskInfo, metrics)) + assert(listener.stageIdToData((task.stageId, 1)).numCompleteTasks === 1) + assert(listener.stageIdToData((task.stageId, 0)).numFailedTasks === failCount) } test("test update metrics") { @@ -163,18 +169,18 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskInfo } - listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1234L))) - listener.onTaskStart(SparkListenerTaskStart(0, makeTaskInfo(1235L))) - listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1236L))) - listener.onTaskStart(SparkListenerTaskStart(1, makeTaskInfo(1237L))) + listener.onTaskStart(SparkListenerTaskStart(0, 0, makeTaskInfo(1234L))) + listener.onTaskStart(SparkListenerTaskStart(0, 0, makeTaskInfo(1235L))) + listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1236L))) + listener.onTaskStart(SparkListenerTaskStart(1, 0, makeTaskInfo(1237L))) listener.onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate(execId, Array( - (1234L, 0, makeTaskMetrics(0)), - (1235L, 0, makeTaskMetrics(100)), - (1236L, 1, makeTaskMetrics(200))))) + (1234L, 0, 0, makeTaskMetrics(0)), + (1235L, 0, 0, makeTaskMetrics(100)), + (1236L, 1, 0, makeTaskMetrics(200))))) - var stage0Data = listener.stageIdToData.get(0).get - var stage1Data = listener.stageIdToData.get(1).get + var stage0Data = listener.stageIdToData.get((0, 0)).get + var stage1Data = listener.stageIdToData.get((1, 0)).get assert(stage0Data.shuffleReadBytes == 102) assert(stage1Data.shuffleReadBytes == 201) assert(stage0Data.shuffleWriteBytes == 106) @@ -195,14 +201,14 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc .totalBlocksFetched == 202) // task that was included in a heartbeat - listener.onTaskEnd(SparkListenerTaskEnd(0, taskType, Success, makeTaskInfo(1234L, 1), + listener.onTaskEnd(SparkListenerTaskEnd(0, 0, taskType, Success, makeTaskInfo(1234L, 1), makeTaskMetrics(300))) // task that wasn't included in a heartbeat - listener.onTaskEnd(SparkListenerTaskEnd(1, taskType, Success, makeTaskInfo(1237L, 1), + listener.onTaskEnd(SparkListenerTaskEnd(1, 0, taskType, Success, makeTaskInfo(1237L, 1), makeTaskMetrics(400))) - stage0Data = listener.stageIdToData.get(0).get - stage1Data = listener.stageIdToData.get(1).get + stage0Data = listener.stageIdToData.get((0, 0)).get + stage1Data = listener.stageIdToData.get((1, 0)).get assert(stage0Data.shuffleReadBytes == 402) assert(stage1Data.shuffleReadBytes == 602) assert(stage0Data.shuffleWriteBytes == 406) diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index 6e68dcb3425aa..b860177705d84 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -53,7 +53,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { assert(storageListener.rddInfoList.isEmpty) // 2 RDDs are known, but none are cached - val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0, rddInfo1), "details") + val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0, rddInfo1), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 2) assert(storageListener.rddInfoList.isEmpty) @@ -63,7 +63,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val rddInfo3Cached = rddInfo3 rddInfo2Cached.numCachedPartitions = 1 rddInfo3Cached.numCachedPartitions = 1 - val stageInfo1 = new StageInfo(1, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") + val stageInfo1 = new StageInfo(1, 0, "0", 100, Seq(rddInfo2Cached, rddInfo3Cached), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo1)) assert(storageListener._rddInfoMap.size === 4) assert(storageListener.rddInfoList.size === 2) @@ -71,7 +71,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { // Submitting RDDInfos with duplicate IDs does nothing val rddInfo0Cached = new RDDInfo(0, "freedom", 100, StorageLevel.MEMORY_ONLY) rddInfo0Cached.numCachedPartitions = 1 - val stageInfo0Cached = new StageInfo(0, "0", 100, Seq(rddInfo0), "details") + val stageInfo0Cached = new StageInfo(0, 0, "0", 100, Seq(rddInfo0), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0Cached)) assert(storageListener._rddInfoMap.size === 4) assert(storageListener.rddInfoList.size === 2) @@ -87,7 +87,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val rddInfo1Cached = rddInfo1 rddInfo0Cached.numCachedPartitions = 1 rddInfo1Cached.numCachedPartitions = 1 - val stageInfo0 = new StageInfo(0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") + val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(rddInfo0Cached, rddInfo1Cached), "details") bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 2) assert(storageListener.rddInfoList.size === 2) @@ -106,7 +106,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val myRddInfo0 = rddInfo0 val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 - val stageInfo0 = new StageInfo(0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 3) @@ -116,7 +116,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { assert(!storageListener._rddInfoMap(2).isCached) // Task end with no updated blocks. This should not change anything. - bus.postToAll(SparkListenerTaskEnd(0, "obliteration", Success, taskInfo, new TaskMetrics)) + bus.postToAll(SparkListenerTaskEnd(0, 0, "obliteration", Success, taskInfo, new TaskMetrics)) assert(storageListener._rddInfoMap.size === 3) assert(storageListener.rddInfoList.size === 0) @@ -128,7 +128,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { (RDDBlockId(0, 102), BlockStatus(memAndDisk, 400L, 0L, 200L)), (RDDBlockId(1, 20), BlockStatus(memAndDisk, 0L, 240L, 0L)) )) - bus.postToAll(SparkListenerTaskEnd(1, "obliteration", Success, taskInfo, metrics1)) + bus.postToAll(SparkListenerTaskEnd(1, 0, "obliteration", Success, taskInfo, metrics1)) assert(storageListener._rddInfoMap(0).memSize === 800L) assert(storageListener._rddInfoMap(0).diskSize === 400L) assert(storageListener._rddInfoMap(0).tachyonSize === 200L) @@ -150,7 +150,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { (RDDBlockId(2, 40), BlockStatus(none, 0L, 0L, 0L)), // doesn't actually exist (RDDBlockId(4, 80), BlockStatus(none, 0L, 0L, 0L)) // doesn't actually exist )) - bus.postToAll(SparkListenerTaskEnd(2, "obliteration", Success, taskInfo, metrics2)) + bus.postToAll(SparkListenerTaskEnd(2, 0, "obliteration", Success, taskInfo, metrics2)) assert(storageListener._rddInfoMap(0).memSize === 400L) assert(storageListener._rddInfoMap(0).diskSize === 400L) assert(storageListener._rddInfoMap(0).tachyonSize === 200L) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 97ffb07662482..2fd3b9cfd221a 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -35,13 +35,13 @@ class JsonProtocolSuite extends FunSuite { val stageSubmitted = SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) val stageCompleted = SparkListenerStageCompleted(makeStageInfo(101, 201, 301, 401L, 501L)) - val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 1, 444L, false)) + val taskStart = SparkListenerTaskStart(111, 0, makeTaskInfo(222L, 333, 1, 444L, false)) val taskGettingResult = SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 5, 3000L, true)) - val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + val taskEnd = SparkListenerTaskEnd(1, 0, "ShuffleMapTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = false)) - val taskEndWithHadoopInput = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + val taskEndWithHadoopInput = SparkListenerTaskEnd(1, 0, "ShuffleMapTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true)) val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) @@ -397,7 +397,8 @@ class JsonProtocolSuite extends FunSuite { private def assertJsonStringEquals(json1: String, json2: String) { val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") - assert(formatJsonString(json1) === formatJsonString(json2)) + assert(formatJsonString(json1) === formatJsonString(json2), + s"input ${formatJsonString(json1)} got ${formatJsonString(json2)}") } private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { @@ -485,7 +486,7 @@ class JsonProtocolSuite extends FunSuite { private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { val rddInfos = (0 until a % 5).map { i => makeRddInfo(a + i, b + i, c + i, d + i, e + i) } - val stageInfo = new StageInfo(a, "greetings", b, rddInfos, "details") + val stageInfo = new StageInfo(a, 0, "greetings", b, rddInfos, "details") val (acc1, acc2) = (makeAccumulableInfo(1), makeAccumulableInfo(2)) stageInfo.accumulables(acc1.id) = acc1 stageInfo.accumulables(acc2.id) = acc2 @@ -558,84 +559,246 @@ class JsonProtocolSuite extends FunSuite { private val stageSubmittedJsonString = """ - {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": - "greetings","Number of Tasks":200,"RDD Info":[],"Details":"details", - "Accumulables":[{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - {"ID":1,"Name":"Accumulable1","Update":"delta1","Value":"val1"}]},"Properties": - {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + |{ + | "Event": "SparkListenerStageSubmitted", + | "Stage Info": { + | "Stage ID": 100, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | "Properties": { + | "France": "Paris", + | "Germany": "Berlin", + | "Russia": "Moscow", + | "Ukraine": "Kiev" + | } + |} """ private val stageCompletedJsonString = """ - {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": - "greetings","Number of Tasks":201,"RDD Info":[{"RDD ID":101,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, - "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, - "Memory Size":401,"Tachyon Size":0,"Disk Size":501}],"Details":"details", - "Accumulables":[{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - {"ID":1,"Name":"Accumulable1","Update":"delta1","Value":"val1"}]}} + |{ + | "Event": "SparkListenerStageCompleted", + | "Stage Info": { + | "Stage ID": 101, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 201, + | "RDD Info": [ + | { + | "RDD ID": 101, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 201, + | "Number of Cached Partitions": 301, + | "Memory Size": 401, + | "Tachyon Size": 0, + | "Disk Size": 501 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | } + |} """ private val taskStartJsonString = """ - |{"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, - |"Index":333,"Attempt":1,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", - |"Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0, - |"Failed":false,"Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - |"Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - |{"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}]}} + |{ + | "Event": "SparkListenerTaskStart", + | "Stage ID": 111, + | "Stage Attempt ID": 0, + | "Task Info": { + | "Task ID": 222, + | "Index": 333, + | "Attempt": 1, + | "Launch Time": 444, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": false, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" + | } + | ] + | } + |} """.stripMargin private val taskGettingResultJsonString = """ - |{"Event":"SparkListenerTaskGettingResult","Task Info": - | {"Task ID":1000,"Index":2000,"Attempt":5,"Launch Time":3000,"Executor ID":"executor", - | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":true,"Getting Result Time":0, - | "Finish Time":0,"Failed":false, - | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] + |{ + | "Event": "SparkListenerTaskGettingResult", + | "Task Info": { + | "Task ID": 1000, + | "Index": 2000, + | "Attempt": 5, + | "Launch Time": 3000, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": true, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" + | } + | ] | } |} """.stripMargin private val taskEndJsonString = """ - |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", - |"Task End Reason":{"Reason":"Success"}, - |"Task Info":{ - | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", - | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false, - | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] - |}, - |"Task Metrics":{ - | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, - | "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700, - | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, - | "Shuffle Read Metrics":{ - | "Shuffle Finish Time":900, - | "Remote Blocks Fetched":800, - | "Local Blocks Fetched":700, - | "Fetch Wait Time":900, - | "Remote Bytes Read":1000 + |{ + | "Event": "SparkListenerTaskEnd", + | "Stage ID": 1, + | "Stage Attempt ID": 0, + | "Task Type": "ShuffleMapTask", + | "Task End Reason": { + | "Reason": "Success" | }, - | "Shuffle Write Metrics":{ - | "Shuffle Bytes Written":1200, - | "Shuffle Write Time":1500 + | "Task Info": { + | "Task ID": 123, + | "Index": 234, + | "Attempt": 67, + | "Launch Time": 345, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": false, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" + | } + | ] | }, - | "Updated Blocks":[ - | {"Block ID":"rdd_0_0", - | "Status":{ - | "Storage Level":{ - | "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - | "Replication":2 - | }, - | "Memory Size":0,"Tachyon Size":0,"Disk Size":0 + | "Task Metrics": { + | "Host Name": "localhost", + | "Executor Deserialize Time": 300, + | "Executor Run Time": 400, + | "Result Size": 500, + | "JVM GC Time": 600, + | "Result Serialization Time": 700, + | "Memory Bytes Spilled": 800, + | "Disk Bytes Spilled": 0, + | "Shuffle Read Metrics": { + | "Shuffle Finish Time": 900, + | "Remote Blocks Fetched": 800, + | "Local Blocks Fetched": 700, + | "Fetch Wait Time": 900, + | "Remote Bytes Read": 1000 + | }, + | "Shuffle Write Metrics": { + | "Shuffle Bytes Written": 1200, + | "Shuffle Write Time": 1500 + | }, + | "Updated Blocks": [ + | { + | "Block ID": "rdd_0_0", + | "Status": { + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": false, + | "Replication": 2 + | }, + | "Memory Size": 0, + | "Tachyon Size": 0, + | "Disk Size": 0 + | } | } - | } | ] | } |} @@ -643,80 +806,187 @@ class JsonProtocolSuite extends FunSuite { private val taskEndWithHadoopInputJsonString = """ - |{"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", - |"Task End Reason":{"Reason":"Success"}, - |"Task Info":{ - | "Task ID":123,"Index":234,"Attempt":67,"Launch Time":345,"Executor ID":"executor", - | "Host":"your kind sir","Locality":"NODE_LOCAL","Speculative":false, - | "Getting Result Time":0,"Finish Time":0,"Failed":false, - | "Accumulables":[{"ID":1,"Name":"Accumulable1","Update":"delta1", - | "Value":"val1"},{"ID":2,"Name":"Accumulable2","Update":"delta2","Value":"val2"}, - | {"ID":3,"Name":"Accumulable3","Update":"delta3","Value":"val3"}] - |}, - |"Task Metrics":{ - | "Host Name":"localhost","Executor Deserialize Time":300,"Executor Run Time":400, - | "Result Size":500,"JVM GC Time":600,"Result Serialization Time":700, - | "Memory Bytes Spilled":800,"Disk Bytes Spilled":0, - | "Shuffle Write Metrics":{"Shuffle Bytes Written":1200,"Shuffle Write Time":1500}, - | "Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":2100}, - | "Updated Blocks":[ - | {"Block ID":"rdd_0_0", - | "Status":{ - | "Storage Level":{ - | "Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - | "Replication":2 - | }, - | "Memory Size":0,"Tachyon Size":0,"Disk Size":0 + |{ + | "Event": "SparkListenerTaskEnd", + | "Stage ID": 1, + | "Stage Attempt ID": 0, + | "Task Type": "ShuffleMapTask", + | "Task End Reason": { + | "Reason": "Success" + | }, + | "Task Info": { + | "Task ID": 123, + | "Index": 234, + | "Attempt": 67, + | "Launch Time": 345, + | "Executor ID": "executor", + | "Host": "your kind sir", + | "Locality": "NODE_LOCAL", + | "Speculative": false, + | "Getting Result Time": 0, + | "Finish Time": 0, + | "Failed": false, + | "Accumulables": [ + | { + | "ID": 1, + | "Name": "Accumulable1", + | "Update": "delta1", + | "Value": "val1" + | }, + | { + | "ID": 2, + | "Name": "Accumulable2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 3, + | "Name": "Accumulable3", + | "Update": "delta3", + | "Value": "val3" | } - | } - | ]} + | ] + | }, + | "Task Metrics": { + | "Host Name": "localhost", + | "Executor Deserialize Time": 300, + | "Executor Run Time": 400, + | "Result Size": 500, + | "JVM GC Time": 600, + | "Result Serialization Time": 700, + | "Memory Bytes Spilled": 800, + | "Disk Bytes Spilled": 0, + | "Shuffle Write Metrics": { + | "Shuffle Bytes Written": 1200, + | "Shuffle Write Time": 1500 + | }, + | "Input Metrics": { + | "Data Read Method": "Hadoop", + | "Bytes Read": 2100 + | }, + | "Updated Blocks": [ + | { + | "Block ID": "rdd_0_0", + | "Status": { + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": false, + | "Replication": 2 + | }, + | "Memory Size": 0, + | "Tachyon Size": 0, + | "Disk Size": 0 + | } + | } + | ] + | } |} """ private val jobStartJsonString = """ - {"Event":"SparkListenerJobStart","Job ID":10,"Stage IDs":[1,2,3,4],"Properties": - {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + |{ + | "Event": "SparkListenerJobStart", + | "Job ID": 10, + | "Stage IDs": [ + | 1, + | 2, + | 3, + | 4 + | ], + | "Properties": { + | "France": "Paris", + | "Germany": "Berlin", + | "Russia": "Moscow", + | "Ukraine": "Kiev" + | } + |} """ private val jobEndJsonString = """ - {"Event":"SparkListenerJobEnd","Job ID":20,"Job Result":{"Result":"JobSucceeded"}} + |{ + | "Event": "SparkListenerJobEnd", + | "Job ID": 20, + | "Job Result": { + | "Result": "JobSucceeded" + | } + |} """ private val environmentUpdateJsonString = """ - {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"GC speed":"9999 objects/s", - "Java home":"Land of coffee"},"Spark Properties":{"Job throughput":"80000 jobs/s, - regardless of job type"},"System Properties":{"Username":"guest","Password":"guest"}, - "Classpath Entries":{"Super library":"/tmp/super_library"}} + |{ + | "Event": "SparkListenerEnvironmentUpdate", + | "JVM Information": { + | "GC speed": "9999 objects/s", + | "Java home": "Land of coffee" + | }, + | "Spark Properties": { + | "Job throughput": "80000 jobs/s, regardless of job type" + | }, + | "System Properties": { + | "Username": "guest", + | "Password": "guest" + | }, + | "Classpath Entries": { + | "Super library": "/tmp/super_library" + | } + |} """ private val blockManagerAddedJsonString = """ - {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"Stars", - "Host":"In your multitude...","Port":300,"Netty Port":400},"Maximum Memory":500} + |{ + | "Event": "SparkListenerBlockManagerAdded", + | "Block Manager ID": { + | "Executor ID": "Stars", + | "Host": "In your multitude...", + | "Port": 300, + | "Netty Port": 400 + | }, + | "Maximum Memory": 500 + |} """ private val blockManagerRemovedJsonString = """ - {"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"Scarce", - "Host":"to be counted...","Port":100,"Netty Port":200}} + |{ + | "Event": "SparkListenerBlockManagerRemoved", + | "Block Manager ID": { + | "Executor ID": "Scarce", + | "Host": "to be counted...", + | "Port": 100, + | "Netty Port": 200 + | } + |} """ private val unpersistRDDJsonString = """ - {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} + |{ + | "Event": "SparkListenerUnpersistRDD", + | "RDD ID": 12345 + |} """ private val applicationStartJsonString = """ - {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, - "User":"Garfield"} + |{ + | "Event": "SparkListenerApplicationStart", + | "App Name": "The winner of all", + | "Timestamp": 42, + | "User": "Garfield" + |} """ private val applicationEndJsonString = """ - {"Event":"SparkListenerApplicationEnd","Timestamp":42} + |{ + | "Event": "SparkListenerApplicationEnd", + | "Timestamp": 42 + |} """ } From a2e658dcdab614058eefcf50ae2d419ece9b1fe7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 20 Aug 2014 15:51:14 -0700 Subject: [PATCH 0439/1492] [SPARK-2967][SQL] Fix sort based shuffle for spark sql. Add explicit row copies when sort based shuffle is on. Author: Michael Armbrust Closes #2066 from marmbrus/sortShuffle and squashes the following commits: fcd7bb2 [Michael Armbrust] Fix sort based shuffle for spark sql. --- .../apache/spark/sql/execution/Exchange.scala | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 77dc2ad733215..09c34b7059fc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} +import org.apache.spark.shuffle.sort.SortShuffleManager +import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree @@ -37,6 +38,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una def output = child.output + /** We must copy rows when sort based shuffle is on */ + protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] + def execute() = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => @@ -45,8 +49,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una @transient val hashExpressions = newMutableProjection(expressions, child.output)() - val mutablePair = new MutablePair[Row, Row]() - iter.map(r => mutablePair.update(hashExpressions(r), r)) + if (sortBasedShuffleOn) { + iter.map(r => (hashExpressions(r), r.copy())) + } else { + val mutablePair = new MutablePair[Row, Row]() + iter.map(r => mutablePair.update(hashExpressions(r), r)) + } } val part = new HashPartitioner(numPartitions) val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) @@ -58,8 +66,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una implicit val ordering = new RowOrdering(sortingExpressions, child.output) val rdd = child.execute().mapPartitions { iter => - val mutablePair = new MutablePair[Row, Null](null, null) - iter.map(row => mutablePair.update(row, null)) + if (sortBasedShuffleOn) { + iter.map(row => (row.copy(), null)) + } else { + val mutablePair = new MutablePair[Row, Null](null, null) + iter.map(row => mutablePair.update(row, null)) + } } val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) @@ -69,8 +81,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una case SinglePartition => val rdd = child.execute().mapPartitions { iter => - val mutablePair = new MutablePair[Null, Row]() - iter.map(r => mutablePair.update(null, r)) + if (sortBasedShuffleOn) { + iter.map(r => (null, r.copy())) + } else { + val mutablePair = new MutablePair[Null, Row]() + iter.map(r => mutablePair.update(null, r)) + } } val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) From a1e8b1bc973bc0517681c09e5a5a475c0f395d31 Mon Sep 17 00:00:00 2001 From: wangfei Date: Wed, 20 Aug 2014 16:00:46 -0700 Subject: [PATCH 0440/1492] SPARK_LOGFILE and SPARK_ROOT_LOGGER no longer need in spark-daemon.sh Author: wangfei Closes #2057 from scwf/patch-7 and squashes the following commits: 1b7b9a5 [wangfei] SPARK_LOGFILE and SPARK_ROOT_LOGGER no longer need in spark-daemon.sh --- sbin/spark-daemon.sh | 2 -- 1 file changed, 2 deletions(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 323f675b17848..9032f23ea8eff 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -113,8 +113,6 @@ if [ "$SPARK_PID_DIR" = "" ]; then fi # some variables -export SPARK_LOGFILE=spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.log -export SPARK_ROOT_LOGGER="INFO,DRFA" log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid From d9e94146a6e65be110a62e3bd0351148912a41d1 Mon Sep 17 00:00:00 2001 From: Alex Liu Date: Wed, 20 Aug 2014 16:14:06 -0700 Subject: [PATCH 0441/1492] [SPARK-2846][SQL] Add configureInputJobPropertiesForStorageHandler to initialization of job conf ...al job conf Author: Alex Liu Closes #1927 from alexliu68/SPARK-SQL-2846 and squashes the following commits: e4bdc4c [Alex Liu] SPARK-SQL-2846 add configureInputJobPropertiesForStorageHandler to initial job conf --- .../src/main/scala/org/apache/spark/sql/hive/TableReader.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 82c88280d7754..329f80cad471e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} -import org.apache.hadoop.hive.ql.plan.TableDesc +import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector @@ -249,6 +249,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { def initializeLocalJobConfFunc(path: String, tableDesc: TableDesc)(jobConf: JobConf) { FileInputFormat.setInputPaths(jobConf, path) if (tableDesc != null) { + PlanUtils.configureInputJobPropertiesForStorageHandler(tableDesc) Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) } val bufferSize = System.getProperty("spark.buffer.size", "65536") From c9f743957fa963bc1dbed7a44a346ffce1a45cf2 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 20 Aug 2014 16:23:10 -0700 Subject: [PATCH 0442/1492] [SPARK-2848] Shade Guava in uber-jars. For further discussion, please check the JIRA entry. This change moves Guava classes to a different package so that they don't conflict with the user-provided Guava (or the Hadoop-provided one). Since one class (Optional) was exposed through Spark's public API, that class was forked from Guava at the current dependency version (14.0.1) so that it can be kept going forward (until the API is cleaned). Note this change has a few implications: - *all* classes in the final jars will reference the relocated classes. If Hadoop classes are included (i.e. "-Phadoop-provided" is not activated), those will also reference the Guava 14 classes (instead of the Guava 11 classes from the Hadoop classpath). - if the Guava version in Spark is ever changed, the new Guava will still reference the forked Optional class; this may or may not be a problem, but in the long term it's better to think about removing Optional from the public API. For the end user, there are two visible implications: - Guava is not provided as a transitive dependency anymore (since it's "provided" in Spark) - At runtime, unless they provide their own, they'll either have no Guava or Hadoop's version of Guava (11), depending on how they set up their classpath. Note that this patch does not change the sbt deliverables; those will still contain guava in its original package, and provide guava as a compile-time dependency. This assumes that maven is the canonical build, and sbt-built artifacts are not (officially) published. Author: Marcelo Vanzin Closes #1813 from vanzin/SPARK-2848 and squashes the following commits: 9bdffb0 [Marcelo Vanzin] Undo sbt build changes. 819b445 [Marcelo Vanzin] Review feedback. 05e0a3d [Marcelo Vanzin] Merge branch 'master' into SPARK-2848 fef4370 [Marcelo Vanzin] Unfork Optional.java. d3ea8e1 [Marcelo Vanzin] Exclude asm classes from final jar. 637189b [Marcelo Vanzin] Add hacky filter to prefer Spark's copy of Optional. 2fec990 [Marcelo Vanzin] Shade Guava in the sbt build. 616998e [Marcelo Vanzin] Shade Guava in the maven build, fork Guava's Optional.java. --- assembly/pom.xml | 18 ++++++++++++++++++ core/pom.xml | 35 +++++++++++++++++++++++++++++++++++ examples/pom.xml | 26 +++++++++++++++++++++++++- pom.xml | 16 ++++++++++++++++ project/SparkBuild.scala | 4 ++-- project/plugins.sbt | 4 ++++ 6 files changed, 100 insertions(+), 3 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9fbb037115db3..de7b75258e3c5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -43,6 +43,12 @@ + + + com.google.guava + guava + compile + org.apache.spark spark-core_${scala.binary.version} @@ -113,6 +119,18 @@ shade + + + com.google + org.spark-project.guava + + com.google.common.** + + + com.google.common.base.Optional** + + + diff --git a/core/pom.xml b/core/pom.xml index 6d8be37037729..83c708dfc9619 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -68,9 +68,15 @@ org.eclipse.jetty jetty-server + com.google.guava guava + compile org.apache.commons @@ -322,6 +328,35 @@ + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + false + + + com.google.guava:guava + + + + + + com.google.guava:guava + + com/google/common/base/Optional* + + + + + + + diff --git a/examples/pom.xml b/examples/pom.xml index 8c4c128bb484d..9b12cb0c29c9f 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -46,8 +46,14 @@
    - + + + + com.google.guava + guava + compile + org.apache.spark spark-core_${scala.binary.version} @@ -209,6 +215,12 @@ + + com.google.guava:guava + + com/google/common/base/Optional* + + *:* @@ -226,6 +238,18 @@ shade + + + com.google + org.spark-project.guava + + com.google.common.** + + + com.google.common.base.Optional** + + + diff --git a/pom.xml b/pom.xml index 7ed07ad7df88d..9cbf3ea5995c3 100644 --- a/pom.xml +++ b/pom.xml @@ -260,6 +260,7 @@ com.google.guava guava 14.0.1 + provided org.apache.commons @@ -1017,6 +1018,21 @@ + + + sbt + + + com.google.guava + guava + compile + + + + spark-ganglia-lgpl diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 49d52aefca17a..4c696d3d385fb 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -61,7 +61,7 @@ object SparkBuild extends PomBuild { def backwardCompatibility = { import scala.collection.mutable var isAlphaYarn = false - var profiles: mutable.Seq[String] = mutable.Seq.empty + var profiles: mutable.Seq[String] = mutable.Seq("sbt") if (Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined) { println("NOTE: SPARK_GANGLIA_LGPL is deprecated, please use -Pspark-ganglia-lgpl flag.") profiles ++= Seq("spark-ganglia-lgpl") @@ -116,7 +116,7 @@ object SparkBuild extends PomBuild { retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", publishMavenStyle := true, - + resolvers += Resolver.mavenLocal, otherResolvers <<= SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))), publishLocalConfiguration in MavenCompile <<= (packagedArtifacts, deliverLocal, ivyLoggingLevel) map { diff --git a/project/plugins.sbt b/project/plugins.sbt index 2a61f56c2ea60..8096c61414660 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -26,3 +26,7 @@ addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.1") addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.1") addSbtPlugin("com.cavorite" % "sbt-avro" % "0.3.2") + +libraryDependencies += "org.ow2.asm" % "asm" % "5.0.3" + +libraryDependencies += "org.ow2.asm" % "asm-commons" % "5.0.3" From ba3c730e35bcdb662396955c3cc6f7de628034c8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 20 Aug 2014 17:07:39 -0700 Subject: [PATCH 0443/1492] [SPARK-3140] Clarify confusing PySpark exception message We read the py4j port from the stdout of the `bin/spark-submit` subprocess. If there is interference in stdout (e.g. a random echo in `spark-submit`), we throw an exception with a warning message. We do not, however, distinguish between this case from the case where no stdout is produced at all. I wasted a non-trivial amount of time being baffled by this exception in search of places where I print random whitespace (in vain, of course). A clearer exception message that distinguishes between these cases will prevent similar headaches that I have gone through. Author: Andrew Or Closes #2067 from andrewor14/python-exception and squashes the following commits: 742f823 [Andrew Or] Further clarify warning messages e96a7a0 [Andrew Or] Distinguish between unexpected output and no output at all --- python/pyspark/java_gateway.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index c7f7c1fe591b0..6f4f62f23bc4d 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -54,12 +54,19 @@ def preexec_func(): gateway_port = proc.stdout.readline() gateway_port = int(gateway_port) except ValueError: + # Grab the remaining lines of stdout (stdout, _) = proc.communicate() exit_code = proc.poll() error_msg = "Launching GatewayServer failed" - error_msg += " with exit code %d! " % exit_code if exit_code else "! " - error_msg += "(Warning: unexpected output detected.)\n\n" - error_msg += gateway_port + stdout + error_msg += " with exit code %d!\n" % exit_code if exit_code else "!\n" + error_msg += "Warning: Expected GatewayServer to output a port, but found " + if gateway_port == "" and stdout == "": + error_msg += "no output.\n" + else: + error_msg += "the following:\n\n" + error_msg += "--------------------------------------------------------------\n" + error_msg += gateway_port + stdout + error_msg += "--------------------------------------------------------------\n" raise Exception(error_msg) # Create a thread to echo output from the GatewayServer, which is required From e1571874f26c1df2dfd5ac2959612372716cd2d8 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 20 Aug 2014 17:41:36 -0700 Subject: [PATCH 0444/1492] [SPARK-3143][MLLIB] add tf-idf user guide Moved TF-IDF before Word2Vec because the former is more basic. I also added a link for Word2Vec. atalwalkar Author: Xiangrui Meng Closes #2061 from mengxr/tfidf-doc and squashes the following commits: ca04c70 [Xiangrui Meng] address comments a5ea4b4 [Xiangrui Meng] add tf-idf user guide --- docs/mllib-feature-extraction.md | 83 ++++++++++++++++++++++++++++++-- 1 file changed, 80 insertions(+), 3 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 4b3cb715c58c7..2031b96235ee9 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -7,9 +7,88 @@ displayTitle: MLlib - Feature Extraction * Table of contents {:toc} + +## TF-IDF + +[Term frequency-inverse document frequency (TF-IDF)](http://en.wikipedia.org/wiki/Tf%E2%80%93idf) is a feature +vectorization method widely used in text mining to reflect the importance of a term to a document in the corpus. +Denote a term by `$t$`, a document by `$d$`, and the corpus by `$D$`. +Term frequency `$TF(t, d)$` is the number of times that term `$t$` appears in document `$d$`, +while document frequency `$DF(t, D)$` is the number of documents that contains term `$t$`. +If we only use term frequency to measure the importance, it is very easy to over-emphasize terms that +appear very often but carry little information about the document, e.g., "a", "the", and "of". +If a term appears very often across the corpus, it means it doesn't carry special information about +a particular document. +Inverse document frequency is a numerical measure of how much information a term provides: +`\[ +IDF(t, D) = \log \frac{|D| + 1}{DF(t, D) + 1}, +\]` +where `$|D|$` is the total number of documents in the corpus. +Since logarithm is used, if a term appears in all documents, its IDF value becomes 0. +Note that a smoothing term is applied to avoid dividing by zero for terms outside the corpus. +The TF-IDF measure is simply the product of TF and IDF: +`\[ +TFIDF(t, d, D) = TF(t, d) \cdot IDF(t, D). +\]` +There are several variants on the definition of term frequency and document frequency. +In MLlib, we separate TF and IDF to make them flexible. + +Our implementation of term frequency utilizes the +[hashing trick](http://en.wikipedia.org/wiki/Feature_hashing). +A raw feature is mapped into an index (term) by applying a hash function. +Then term frequencies are calculated based on the mapped indices. +This approach avoids the need to compute a global term-to-index map, +which can be expensive for a large corpus, but it suffers from potential hash collisions, +where different raw features may become the same term after hashing. +To reduce the chance of collision, we can increase the target feature dimension, i.e., +the number of buckets of the hash table. +The default feature dimension is `$2^{20} = 1,048,576$`. + +**Note:** MLlib doesn't provide tools for text segmentation. +We refer users to the [Stanford NLP Group](http://nlp.stanford.edu/) and +[scalanlp/chalk](https://github.com/scalanlp/chalk). + +
    +
    + +TF and IDF are implemented in [HashingTF](api/scala/index.html#org.apache.spark.mllib.feature.HashingTF) +and [IDF](api/scala/index.html#org.apache.spark.mllib.feature.IDF). +`HashingTF` takes an `RDD[Iterable[_]]` as the input. +Each record could be an iterable of strings or other types. + +{% highlight scala %} +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.mllib.feature.HashingTF +import org.apache.spark.mllib.linalg.Vector + +val sc: SparkContext = ... + +// Load documents (one per line). +val documents: RDD[Seq[String]] = sc.textFile("...").map(_.split(" ").toSeq) + +val hashingTF = new HashingTF() +val tf: RDD[Vector] = hasingTF.transform(documents) +{% endhighlight %} + +While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: +first to compute the IDF vector and second to scale the term frequencies by IDF. + +{% highlight scala %} +import org.apache.spark.mllib.feature.IDF + +// ... continue from the previous example +tf.cache() +val idf = new IDF().fit(tf) +val tfidf: RDD[Vector] = idf.transform(tf) +{% endhighlight %} +
    +
    + ## Word2Vec -Word2Vec computes distributed vector representation of words. The main advantage of the distributed +[Word2Vec](https://code.google.com/p/word2vec/) computes distributed vector representation of words. +The main advantage of the distributed representations is that similar words are close in the vector space, which makes generalization to novel patterns easier and model estimation more robust. Distributed vector representation is showed to be useful in many natural language processing applications such as named entity @@ -69,5 +148,3 @@ for((synonym, cosineSimilarity) <- synonyms) { {% endhighlight %} - -## TFIDF \ No newline at end of file From e0f946265b9ea5bc48849cf7794c2c03d5e29fba Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 20 Aug 2014 17:47:39 -0700 Subject: [PATCH 0445/1492] [SPARK-2843][MLLIB] add a section about regularization parameter in ALS atalwalkar srowen Author: Xiangrui Meng Closes #2064 from mengxr/als-doc and squashes the following commits: b2e20ab [Xiangrui Meng] introduced -> discussed 98abdd7 [Xiangrui Meng] add reference 339bd08 [Xiangrui Meng] add a section about regularization parameter in ALS --- docs/mllib-collaborative-filtering.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index ab10b2f01f87b..d5c539db791be 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -43,6 +43,17 @@ level of confidence in observed user preferences, rather than explicit ratings g model then tries to find latent factors that can be used to predict the expected preference of a user for an item. +### Scaling of the regularization parameter + +Since v1.1, we scale the regularization parameter `lambda` in solving each least squares problem by +the number of ratings the user generated in updating user factors, +or the number of ratings the product received in updating product factors. +This approach is named "ALS-WR" and discussed in the paper +"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](http://dx.doi.org/10.1007/978-3-540-68880-8_32)". +It makes `lambda` less dependent on the scale of the dataset. +So we can apply the best parameter learned from a sampled subset to the full dataset +and expect similar performance. + ## Examples
    From 050f8d01e47b9b67b02ce50d83fb7b4e528b7204 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 21 Aug 2014 00:17:29 -0700 Subject: [PATCH 0446/1492] [SPARK-2840] [mllib] DecisionTree doc update (Java, Python examples) Updated DecisionTree documentation, with examples for Java, Python. Added same Java example to code as well. CC: @mengxr @manishamde @atalwalkar Author: Joseph K. Bradley Closes #2063 from jkbradley/dt-docs and squashes the following commits: 2dd2c19 [Joseph K. Bradley] Last updates based on github review. 9dd1b6b [Joseph K. Bradley] Updated decision tree doc. d802369 [Joseph K. Bradley] Updates based on comments: cache data, corrected doc text. b9bee04 [Joseph K. Bradley] Updated DT examples 57eee9f [Joseph K. Bradley] Created JavaDecisionTree example from example in docs, and corrected doc example as needed. d939a92 [Joseph K. Bradley] Updated DecisionTree documentation. Added Java, Python examples. --- docs/mllib-decision-tree.md | 352 ++++++++++++++---- .../examples/mllib/JavaDecisionTree.java | 116 ++++++ 2 files changed, 399 insertions(+), 69 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index c01a92a9a1b26..1166d9cd150c4 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -7,20 +7,26 @@ displayTitle: MLlib - Decision Tree * Table of contents {:toc} -Decision trees and their ensembles are popular methods for the machine learning tasks of +[Decision trees](http://en.wikipedia.org/wiki/Decision_tree_learning) +and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, -handle categorical variables, extend to the multiclass classification setting, do not require +handle categorical features, extend to the multiclass classification setting, do not require feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble -algorithms such as decision forest and boosting are among the top performers for classification and +algorithms such as random forests and boosting are among the top performers for classification and regression tasks. +MLlib supports decision trees for binary and multiclass classification and for regression, +using both continuous and categorical features. The implementation partitions data by rows, +allowing distributed training with millions of instances. + ## Basic algorithm The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature -space by choosing a single element from the *best split set* where each element of the set maximizes -the information gain at a tree node. In other words, the split chosen at each tree node is chosen -from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information -gain when a split `$s$` is applied to a dataset `$D$`. +space. The tree predicts the same label for each bottommost (leaf) partition. +Each partition is chosen greedily by selecting the *best split* from a set of possible splits, +in order to maximize the information gain at a tree node. In other words, the split chosen at each +tree node is chosen from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` +is the information gain when a split `$s$` is applied to a dataset `$D$`. ### Node impurity and information gain @@ -52,9 +58,10 @@ impurity measure for regression (variance). -The *information gain* is the difference in the parent node impurity and the weighted sum of the two -child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two -datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: +The *information gain* is the difference between the parent node impurity and the weighted sum of +the two child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` +into two datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, +respectively, the information gain is: `$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` @@ -62,14 +69,15 @@ datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, **Continuous features** -For small datasets in single machine implementations, the split candidates for each continuous +For small datasets in single-machine implementations, the split candidates for each continuous feature are typically the unique values for the feature. Some implementations sort the feature values and then use the ordered unique values as split candidates for faster tree calculations. -Finding ordered unique feature values is computationally intensive for large distributed -datasets. One can get an approximate set of split candidates by performing a quantile calculation -over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such -bins can be specified using the `maxBins` parameters. +Sorting feature values is expensive for large distributed datasets. +This implementation computes an approximate set of split candidates by performing a quantile +calculation over a sampled fraction of the data. +The ordered splits create "bins" and the maximum number of such +bins can be specified using the `maxBins` parameter. Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of @@ -77,109 +85,315 @@ bins if the condition is not satisfied. **Categorical features** -For `$M$` categorical feature values, one could come up with `$2^(M-1)-1$` split candidates. For -binary classification, we can reduce the number of split candidates to `$M-1$` by ordering the -categorical feature values by the proportion of labels falling in one of the two classes (see -Section 9.2.4 in +For a categorical feature with `$M$` possible values (categories), one could come up with +`$2^{M-1}-1$` split candidates. For binary (0/1) classification and regression, +we can reduce the number of split candidates to `$M-1$` by ordering the +categorical feature values by the average label. (See Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for -details). For example, for a binary classification problem with one categorical feature with three -categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical -features are ordered as A followed by C followed B or A, C, B. The two split candidates are A \| C, B -and A , C \| B where \| denotes the split. A similar heuristic is used for multiclass classification -when `$2^(M-1)-1$` is greater than the number of bins -- the impurity for each categorical feature value -is used for ordering. +details.) For example, for a binary classification problem with one categorical feature with three +categories A, B and C whose corresponding proportions of label 1 are 0.2, 0.6 and 0.4, the categorical +features are ordered as A, C, B. The two split candidates are A \| C, B +and A , C \| B where \| denotes the split. + +In multiclass classification, all `$2^{M-1}-1$` possible splits are used whenever possible. +When `$2^{M-1}-1$` is greater than the `maxBins` parameter, we use a (heuristic) method +similar to the method used for binary classification and regression. +The `$M$` categorical feature values are ordered by impurity, +and the resulting `$M-1$` split candidates are considered. ### Stopping rule The recursive tree construction is stopped at a node when one of the two conditions is met: -1. The node depth is equal to the `maxDepth` training parameter +1. The node depth is equal to the `maxDepth` training parameter. 2. No split candidate leads to an information gain at the node. +## Implementation details + ### Max memory requirements -For faster processing, the decision tree algorithm performs simultaneous histogram computations for all nodes at each level of the tree. This could lead to high memory requirements at deeper levels of the tree leading to memory overflow errors. To alleviate this problem, a 'maxMemoryInMB' training parameter is provided which specifies the maximum amount of memory at the workers (twice as much at the master) to be allocated to the histogram computation. The default value is conservatively chosen to be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements for a level-wise computation crosses the `maxMemoryInMB` threshold, the node training tasks at each subsequent level is split into smaller tasks. +For faster processing, the decision tree algorithm performs simultaneous histogram computations for +all nodes at each level of the tree. This could lead to high memory requirements at deeper levels +of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` +training parameter specifies the maximum amount of memory at the workers (twice as much at the +master) to be allocated to the histogram computation. The default value is conservatively chosen to +be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements +for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each +subsequent level are split into smaller tasks. + +Note that, if you have a large amount of memory, increasing `maxMemoryInMB` can lead to faster +training by requiring fewer passes over the data. + +### Binning feature values + +Increasing `maxBins` allows the algorithm to consider more split candidates and make fine-grained +split decisions. However, it also increases computation and communication. + +Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for +any categorical feature. + +### Scaling -### Practical limitations +Computation scales approximately linearly in the number of training instances, +in the number of features, and in the `maxBins` parameter. +Communication scales approximately linearly in the number of features and in `maxBins`. -1. The implemented algorithm reads both sparse and dense data. However, it is not optimized for sparse input. -2. Python is not supported in this release. +The implemented algorithm reads both sparse and dense data. However, it is not optimized for sparse input. ## Examples ### Classification -The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then -perform classification using a decision tree using Gini impurity as an impurity measure and a +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform classification using a decision tree with Gini impurity as an impurity measure and a maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy.
    +
    {% highlight scala %} -import org.apache.spark.SparkContext import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Gini - -// Load and parse the data file -val data = sc.textFile("data/mllib/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} +import org.apache.spark.mllib.util.MLUtils -// Run training algorithm to build the model +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() + +// Train a DecisionTree model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val numClasses = 2 +val categoricalFeaturesInfo = Map[Int, Int]() +val impurity = "gini" val maxDepth = 5 -val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) +val maxBins = 100 + +val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, + maxDepth, maxBins) -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => +// Evaluate model on training instances and compute training error +val labelAndPreds = data.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count +val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / data.count println("Training Error = " + trainErr) +println("Learned classification tree model:\n" + model) +{% endhighlight %} +
    + +
    +{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +Integer numClasses = 2; +HashMap categoricalFeaturesInfo = new HashMap(); +String impurity = "gini"; +Integer maxDepth = 5; +Integer maxBins = 100; + +// Train a DecisionTree model for classification. +final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on training instances and compute training error +JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double trainErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / data.count(); +System.out.println("Training error: " + trainErr); +System.out.println("Learned classification tree model:\n" + model); +{% endhighlight %} +
    + +
    +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +# Cache the data since we will use it again to compute training error. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() + +# Train a DecisionTree model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, + impurity='gini', maxDepth=5, maxBins=100) + +# Evaluate model on training instances and compute training error +predictions = model.predict(data.map(lambda x: x.features)) +labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) +trainErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(data.count()) +print('Training Error = ' + str(trainErr)) +print('Learned classification tree model:') +print(model) {% endhighlight %} + +Note: When making predictions for a dataset, it is more efficient to do batch prediction rather +than separately calling `predict` on each data point. This is because the Python code makes calls +to an underlying `DecisionTree` model in Scala.
    +
    ### Regression -The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then -perform regression using a decision tree using variance as an impurity measure and a maximum tree +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform regression using a decision tree with variance as an impurity measure and a maximum tree depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit).
    +
    {% highlight scala %} -import org.apache.spark.SparkContext import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Variance - -// Load and parse the data file -val data = sc.textFile("data/mllib/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} +import org.apache.spark.mllib.util.MLUtils -// Run training algorithm to build the model +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() + +// Train a DecisionTree model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val categoricalFeaturesInfo = Map[Int, Int]() +val impurity = "variance" val maxDepth = 5 -val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) +val maxBins = 100 + +val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, + maxDepth, maxBins) -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => +// Evaluate model on training instances and compute training error +val labelsAndPredictions = data.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("training Mean Squared Error = " + MSE) +val trainMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Training Mean Squared Error = " + trainMSE) +println("Learned regression tree model:\n" + model) {% endhighlight %}
    + +
    +{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +// Load and parse the data file. +// Cache the data since we will use it again to compute training error. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + +SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +HashMap categoricalFeaturesInfo = new HashMap(); +String impurity = "variance"; +Integer maxDepth = 5; +Integer maxBins = 100; + +// Train a DecisionTree model. +final DecisionTreeModel model = DecisionTree.trainRegressor(data, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on training instances and compute training error +JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double trainMSE = + predictionAndLabel.map(new Function, Double>() { + @Override public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); +System.out.println("Training Mean Squared Error: " + trainMSE); +System.out.println("Learned regression tree model:\n" + model); +{% endhighlight %} +
    + +
    +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.tree import DecisionTree +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +# Cache the data since we will use it again to compute training error. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() + +# Train a DecisionTree model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, + impurity='variance', maxDepth=5, maxBins=100) + +# Evaluate model on training instances and compute training error +predictions = model.predict(data.map(lambda x: x.features)) +labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) +trainMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(data.count()) +print('Training Mean Squared Error = ' + str(trainMSE)) +print('Learned regression tree model:') +print(model) +{% endhighlight %} + +Note: When making predictions for a dataset, it is more efficient to do batch prediction rather +than separately calling `predict` on each data point. This is because the Python code makes calls +to an underlying `DecisionTree` model in Scala. +
    +
    diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java new file mode 100644 index 0000000000000..e4468e8bf1744 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import java.util.HashMap; + +import scala.Tuple2; + +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.DecisionTree; +import org.apache.spark.mllib.tree.model.DecisionTreeModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +/** + * Classification and regression using decision trees. + */ +public final class JavaDecisionTree { + + public static void main(String[] args) { + String datapath = "data/mllib/sample_libsvm_data.txt"; + if (args.length == 1) { + datapath = args[0]; + } else if (args.length > 1) { + System.err.println("Usage: JavaDecisionTree "); + System.exit(1); + } + SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + + // Compute the number of classes from the data. + Integer numClasses = data.map(new Function() { + @Override public Double call(LabeledPoint p) { + return p.label(); + } + }).countByValue().size(); + + // Set parameters. + // Empty categoricalFeaturesInfo indicates all features are continuous. + HashMap categoricalFeaturesInfo = new HashMap(); + String impurity = "gini"; + Integer maxDepth = 5; + Integer maxBins = 100; + + // Train a DecisionTree model for classification. + final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on training instances and compute training error + JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double trainErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / data.count(); + System.out.println("Training error: " + trainErr); + System.out.println("Learned classification tree model:\n" + model); + + // Train a DecisionTree model for regression. + impurity = "variance"; + final DecisionTreeModel regressionModel = DecisionTree.trainRegressor(data, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + + // Evaluate model on training instances and compute training error + JavaPairRDD regressorPredictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(regressionModel.predict(p.features()), p.label()); + } + }); + Double trainMSE = + regressorPredictionAndLabel.map(new Function, Double>() { + @Override public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); + System.out.println("Training Mean Squared Error: " + trainMSE); + System.out.println("Learned regression tree model:\n" + regressionModel); + + sc.stop(); + } +} From 220c2d7699a522c50d26cd6fdd94f4635823e408 Mon Sep 17 00:00:00 2001 From: XuTingjun <1039320815@qq.com> Date: Fri, 22 Aug 2014 10:45:00 -0500 Subject: [PATCH 0447/1492] [SPARK-2742][yarn] delete useless variables Author: XuTingjun <1039320815@qq.com> Closes #1614 from XuTingjun/yarn-bug and squashes the following commits: f07096e [XuTingjun] Update ClientArguments.scala --- .../scala/org/apache/spark/deploy/yarn/ClientArguments.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 62f9b3cf5ab88..afa4fd4c6959e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -39,7 +39,6 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" var appName: String = "Spark" - var inputFormatInfo: List[InputFormatInfo] = null var priority = 0 parseArgs(args.toList) @@ -58,8 +57,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { private def parseArgs(inputArgs: List[String]): Unit = { val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() - val inputFormatMap: HashMap[String, InputFormatInfo] = new HashMap[String, InputFormatInfo]() - + var args = inputArgs while (!args.isEmpty) { @@ -145,7 +143,6 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { } userArgs = userArgsBuffer.readOnly - inputFormatInfo = inputFormatMap.values.toList } From a5219db1ece20a1c45f927d4d0e085474f0bf34e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 22 Aug 2014 17:05:35 -0700 Subject: [PATCH 0448/1492] Link to Contributing to Spark wiki page on README.md. --- README.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/README.md b/README.md index 8906e4c1416b1..c25a4fc6a6c67 100644 --- a/README.md +++ b/README.md @@ -136,3 +136,5 @@ submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. +Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) +for more information. From 3004074152b7261c2a968bb8e94ec7c41a7b43c1 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 22 Aug 2014 21:34:48 -0700 Subject: [PATCH 0449/1492] [SPARK-3169] Removed dependency on spark streaming test from spark flume sink Due to maven bug https://jira.codehaus.org/browse/MNG-1378, maven could not resolve spark streaming classes required by the spark-streaming test-jar dependency of external/flume-sink. There is no particular reason that the external/flume-sink has to depend on Spark Streaming at all, so I am eliminating this dependency. Also I have removed the exclusions present in the Flume dependencies, as there is no reason to exclude them (they were excluded in the external/flume module to prevent dependency collisions with Spark). Since Jenkins will test the sbt build and the unit test, I only tested maven compilation locally. Author: Tathagata Das Closes #2101 from tdas/spark-sink-pom-fix and squashes the following commits: 8f42621 [Tathagata Das] Added Flume sink exclusions back, and added netty to test dependencies 93b559f [Tathagata Das] Removed dependency on spark streaming test from spark flume sink --- external/flume-sink/pom.xml | 18 ++++++++++++------ .../streaming/flume/sink/SparkSinkSuite.scala | 10 +++++----- 2 files changed, 17 insertions(+), 11 deletions(-) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 0c68defa5e101..19192e40a7dc3 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -62,7 +62,7 @@ org.apache.thrift libthrift - + org.scala-lang @@ -71,13 +71,19 @@ org.scalatest scalatest_${scala.binary.version} + test - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - test-jar - test + + io.netty + netty + 3.4.0.Final + test diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index 44b27edf85ce8..75a6668c6210b 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -30,14 +30,14 @@ import org.apache.avro.ipc.specific.SpecificRequestor import org.apache.flume.Context import org.apache.flume.channel.MemoryChannel import org.apache.flume.event.EventBuilder -import org.apache.spark.streaming.TestSuiteBase import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.scalatest.FunSuite -class SparkSinkSuite extends TestSuiteBase { +class SparkSinkSuite extends FunSuite { val eventsPerBatch = 1000 val channelCapacity = 5000 - test("Success") { + test("Success with ack") { val (channel, sink) = initializeChannelAndSink() channel.start() sink.start() @@ -57,7 +57,7 @@ class SparkSinkSuite extends TestSuiteBase { transceiver.close() } - test("Nack") { + test("Failure with nack") { val (channel, sink) = initializeChannelAndSink() channel.start() sink.start() @@ -76,7 +76,7 @@ class SparkSinkSuite extends TestSuiteBase { transceiver.close() } - test("Timeout") { + test("Failure with timeout") { val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig .CONF_TRANSACTION_TIMEOUT -> 1.toString)) channel.start() From 323cd92b9b762dc3ae456cc97564a71dd6e58244 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 22 Aug 2014 22:28:05 -0700 Subject: [PATCH 0450/1492] [SPARK-2963] REGRESSION - The description about how to build for using CLI and Thrift JDBC server is absent in proper document - The most important things I mentioned in #1885 is as follows. * People who build Spark is not always programmer. * If a person who build Spark is not a programmer, he/she won't read programmer's guide before building. So, how to build for using CLI and JDBC server is not only in programmer's guide. Author: Kousuke Saruta Closes #2080 from sarutak/SPARK-2963 and squashes the following commits: ee07c76 [Kousuke Saruta] Modified regression of the description about building for using Thrift JDBC server and CLI ed53329 [Kousuke Saruta] Modified description and notaton of proper noun 07c59fc [Kousuke Saruta] Added a description about how to build to use HiveServer and CLI for SparkSQL to building-with-maven.md 6e6645a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2963 c88fa93 [Kousuke Saruta] Added a description about building to use HiveServer and CLI for SparkSQL --- README.md | 5 ++++- docs/building-with-maven.md | 11 +++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index c25a4fc6a6c67..f75226ce11335 100644 --- a/README.md +++ b/README.md @@ -118,7 +118,10 @@ If your project is built with Maven, add this to your POM file's ` ## A Note About Thrift JDBC server and CLI for Spark SQL Spark SQL supports Thrift JDBC server and CLI. -See sql-programming-guide.md for more information about using the JDBC server. +See sql-programming-guide.md for more information about using the JDBC server and CLI. +You can use those features by setting `-Phive` when building Spark as follows. + + $ sbt/sbt -Phive assembly ## Configuration diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a7d7bd3ccb1f2..bce7412c7d4c9 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -96,10 +96,13 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package {% endhighlight %} -# Building Thrift JDBC server and CLI for Spark SQL - -Spark SQL supports Thrift JDBC server and CLI. See sql-programming-guide.md for -more information about the JDBC server. +# Building With Hive and JDBC Support +To enable Hive integration for Spark SQL along with its JDBC server and CLI, +add the `-Phive` profile to your existing build options. +{% highlight bash %} +# Apache Hadoop 2.4.X with Hive support +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package +{% endhighlight %} # Spark Tests in Maven From f3d65cd0bf34d8c362da492472c61796ef059428 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Sat, 23 Aug 2014 08:09:30 -0700 Subject: [PATCH 0451/1492] [SPARK-3068]remove MaxPermSize option for jvm 1.8 In JVM 1.8.0, MaxPermSize is no longer supported. In spark `stderr` output, there would be a line of Java HotSpot(TM) 64-Bit Server VM warning: ignoring option MaxPermSize=128m; support was removed in 8.0 Author: Daoyuan Wang Closes #2011 from adrian-wang/maxpermsize and squashes the following commits: ef1d660 [Daoyuan Wang] direct get java version in runtime 37db9c1 [Daoyuan Wang] code refine 3c1d554 [Daoyuan Wang] remove MaxPermSize option for jvm 1.8 --- bin/spark-class | 7 ++++++- bin/spark-class2.cmd | 8 +++++++- .../org/apache/spark/deploy/worker/CommandUtils.scala | 4 ++-- 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 22acf92288b3b..c6543545a5e64 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -105,9 +105,14 @@ else exit 1 fi fi +JAVA_VERSION=$($RUNNER -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size -JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" +if [ "$JAVA_VERSION" -ge 18 ]; then + JAVA_OPTS="$OUR_JAVA_OPTS" +else + JAVA_OPTS="-XX:MaxPermSize=128m $OUR_JAVA_OPTS" +fi JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index e420eb409e529..e2c5f9c385189 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -77,7 +77,13 @@ rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SP ) rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +for /f "tokens=3" %%i in ('java -version 2^>^&1 ^| find "version"') do set jversion=%%i +for /f "tokens=1 delims=_" %%i in ("%jversion:~1,-1%") do set jversion=%%i +if "%jversion%" geq "1.8.0" ( + set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +) else ( + set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% +) rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! rem Test whether the user has built Spark diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 687e492a0d6fc..12e98fd40d6c9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -64,8 +64,6 @@ object CommandUtils extends Logging { Seq() } - val permGenOpt = Seq("-XX:MaxPermSize=128m") - // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( @@ -73,6 +71,8 @@ object CommandUtils extends Logging { extraEnvironment = command.environment) val userClassPath = command.classPathEntries ++ Seq(classPath) + val javaVersion = System.getProperty("java.version") + val permGenOpt = if (!javaVersion.startsWith("1.8")) Some("-XX:MaxPermSize=128m") else None Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ permGenOpt ++ libraryOpts ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts } From 76bb044b9e327639002034ad4c06e6bd8e20d018 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 23 Aug 2014 10:08:25 -0700 Subject: [PATCH 0452/1492] [Minor] fix typo Fix a typo in comment. Author: Liang-Chi Hsieh Closes #2105 from viirya/fix_typo and squashes the following commits: 6596a80 [Liang-Chi Hsieh] fix typo. --- core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index a74f80094434c..d5336284571d2 100644 --- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -22,7 +22,7 @@ import cern.jet.stat.Probability import org.apache.spark.util.StatCounter /** - * An ApproximateEvaluator for sums. It estimates the mean and the cont and multiplies them + * An ApproximateEvaluator for sums. It estimates the mean and the count and multiplies them * together, then uses the formula for the variance of two independent random variables to get * a variance for the result and compute a confidence interval. */ From 2fb1c72ea21e137c8b60a72e5aecd554c71b16e1 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 23 Aug 2014 12:46:41 -0700 Subject: [PATCH 0453/1492] [SQL] Make functionRegistry in HiveContext transient. Seems we missed `transient` for the `functionRegistry` in `HiveContext`. cc: marmbrus Author: Yin Huai Closes #2074 from yhuai/makeFunctionRegistryTransient and squashes the following commits: 6534e7d [Yin Huai] Make functionRegistry transient. --- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ff32c7c90a0d2..29baefe714c64 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -255,6 +255,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } // Note that HiveUDFs will be overridden by functions registered in this context. + @transient override protected[sql] lazy val functionRegistry = new HiveFunctionRegistry with OverrideFunctionRegistry From 7e191fe29bb09a8560cd75d453c4f7f662dff406 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 23 Aug 2014 16:19:10 -0700 Subject: [PATCH 0454/1492] [SPARK-2554][SQL] CountDistinct partial aggregation and object allocation improvements Author: Michael Armbrust Author: Gregory Owen Closes #1935 from marmbrus/countDistinctPartial and squashes the following commits: 5c7848d [Michael Armbrust] turn off caching in the constructor 8074a80 [Michael Armbrust] fix tests 32d216f [Michael Armbrust] reynolds comments c122cca [Michael Armbrust] Address comments, add tests b2e8ef3 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial fae38f4 [Michael Armbrust] Fix style fdca896 [Michael Armbrust] cleanup 93d0f64 [Michael Armbrust] metastore concurrency fix. db44a30 [Michael Armbrust] JIT hax. 3868f6c [Michael Armbrust] Merge pull request #9 from GregOwen/countDistinctPartial c9e67de [Gregory Owen] Made SpecificRow and types serializable by Kryo 2b46c4b [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial 8ff6402 [Michael Armbrust] Add specific row. 58d15f1 [Michael Armbrust] disable codegen logging 87d101d [Michael Armbrust] Fix isNullAt bug abee26d [Michael Armbrust] WIP 27984d0 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial 57ae3b1 [Michael Armbrust] Fix order dependent test b3d0f64 [Michael Armbrust] Add golden files. c1f7114 [Michael Armbrust] Improve tests / fix serialization. f31b8ad [Michael Armbrust] more fixes 38c7449 [Michael Armbrust] comments and style 9153652 [Michael Armbrust] better toString d494598 [Michael Armbrust] Fix tests now that the planner is better 41fbd1d [Michael Armbrust] Never try and create an empty hash set. 050bb97 [Michael Armbrust] Skip no-arg constructors for kryo, bd08239 [Michael Armbrust] WIP 213ada8 [Michael Armbrust] First draft of partially aggregated and code generated count distinct / max --- .../sql/catalyst/expressions/Projection.scala | 344 +++++++++++++++++- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../catalyst/expressions/SpecificRow.scala | 307 ++++++++++++++++ .../sql/catalyst/expressions/aggregates.scala | 93 ++++- .../sql/catalyst/expressions/arithmetic.scala | 31 ++ .../expressions/codegen/CodeGenerator.scala | 93 ++++- .../codegen/GenerateProjection.scala | 9 +- .../spark/sql/catalyst/expressions/sets.scala | 129 +++++++ .../spark/sql/catalyst/trees/TreeNode.scala | 3 +- .../ExpressionEvaluationSuite.scala | 10 + .../spark/sql/execution/Aggregate.scala | 2 +- .../sql/execution/GeneratedAggregate.scala | 36 +- .../sql/execution/SparkSqlSerializer.scala | 86 +++++ .../spark/sql/execution/SparkStrategies.scala | 6 +- .../apache/spark/sql/execution/joins.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 8 +- .../sql/parquet/ParquetTableOperations.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 8 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +- ... values-0-1843b7947729b771fee3a4abd050bfdc | 1 + ...ll long-0-89b850197b326239d60a5e1d5db7c9c9 | 1 + ... + null-0-a014038c00fb81e88041ed4a8368e6f7 | 1 + ...ue long-0-77b9ed1d7ae65fa53830a3bc586856ff | 1 + ...strings-0-c68e75ec4c884b93765a466e992e391d | 1 + ...1 value-0-a4047b06a324fb5ea400c94350c9e038 | 1 + ...ng null-0-75672236a30e10dab13b9b246c5a3a1e | 1 + ...es long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 | 1 + ... values-0-c61df65af167acaf7edb174e77898f3e | 1 + ...limited-0-52b0e534c7df544258a1c59df9f816ce | 0 .../spark/sql/hive/StatisticsSuite.scala | 8 +- .../sql/hive/execution/HiveQuerySuite.scala | 65 ++++ .../sql/hive/execution/HiveSerDeSuite.scala | 11 +- .../sql/hive/execution/PruningSuite.scala | 5 +- 33 files changed, 1239 insertions(+), 34 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala create mode 100644 sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d create mode 100644 sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 create mode 100644 sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e create mode 100644 sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 create mode 100644 sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e create mode 100644 sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 8fc5896974438..ef1d12531f109 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -27,7 +27,8 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = this(expressions.map(BindReferences.bindReference(_, inputSchema))) - protected val exprArray = expressions.toArray + // null check is required for when Kryo invokes the no-arg constructor. + protected val exprArray = if (expressions != null) expressions.toArray else null def apply(input: Row): Row = { val outputArray = new Array[Any](exprArray.length) @@ -109,7 +110,346 @@ class JoinedRow extends Row { def apply(i: Int) = if (i < row1.size) row1(i) else row2(i - row1.size) - def isNullAt(i: Int) = apply(i) == null + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + * The `JoinedRow` class is used in many performance critical situation. Unfortunately, since there + * are multiple different types of `Rows` that could be stored as `row1` and `row2` most of the + * calls in the critical path are polymorphic. By creating special versions of this class that are + * used in only a single location of the code, we increase the chance that only a single type of + * Row will be referenced, increasing the opportunity for the JIT to play tricks. This sounds + * crazy but in benchmarks it had noticeable effects. + */ +class JoinedRow2 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow3 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow4 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) + + def getInt(i: Int): Int = + if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) + + def getLong(i: Int): Long = + if (i < row1.size) row1.getLong(i) else row2.getLong(i - row1.size) + + def getDouble(i: Int): Double = + if (i < row1.size) row1.getDouble(i) else row2.getDouble(i - row1.size) + + def getBoolean(i: Int): Boolean = + if (i < row1.size) row1.getBoolean(i) else row2.getBoolean(i - row1.size) + + def getShort(i: Int): Short = + if (i < row1.size) row1.getShort(i) else row2.getShort(i - row1.size) + + def getByte(i: Int): Byte = + if (i < row1.size) row1.getByte(i) else row2.getByte(i - row1.size) + + def getFloat(i: Int): Float = + if (i < row1.size) row1.getFloat(i) else row2.getFloat(i - row1.size) + + def getString(i: Int): String = + if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + + def copy() = { + val totalSize = row1.size + row2.size + val copiedValues = new Array[Any](totalSize) + var i = 0 + while(i < totalSize) { + copiedValues(i) = apply(i) + i += 1 + } + new GenericRow(copiedValues) + } + + override def toString() = { + val row = (if (row1 != null) row1 else Seq[Any]()) ++ (if (row2 != null) row2 else Seq[Any]()) + s"[${row.mkString(",")}]" + } +} + +/** + * JIT HACK: Replace with macros + */ +class JoinedRow5 extends Row { + private[this] var row1: Row = _ + private[this] var row2: Row = _ + + def this(left: Row, right: Row) = { + this() + row1 = left + row2 = right + } + + /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ + def apply(r1: Row, r2: Row): Row = { + row1 = r1 + row2 = r2 + this + } + + /** Updates this JoinedRow by updating its left base row. Returns itself. */ + def withLeft(newLeft: Row): Row = { + row1 = newLeft + this + } + + /** Updates this JoinedRow by updating its right base row. Returns itself. */ + def withRight(newRight: Row): Row = { + row2 = newRight + this + } + + def iterator = row1.iterator ++ row2.iterator + + def length = row1.length + row2.length + + def apply(i: Int) = + if (i < row1.size) row1(i) else row2(i - row1.size) + + def isNullAt(i: Int) = + if (i < row1.size) row1.isNullAt(i) else row2.isNullAt(i - row1.size) def getInt(i: Int): Int = if (i < row1.size) row1.getInt(i) else row2.getInt(i - row1.size) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index c9a63e201ef60..d68a4fabeac77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -127,7 +127,7 @@ object EmptyRow extends Row { * the array is not copied, and thus could technically be mutated after creation, this is not * allowed. */ -class GenericRow(protected[catalyst] val values: Array[Any]) extends Row { +class GenericRow(protected[sql] val values: Array[Any]) extends Row { /** No-arg constructor for serialization. */ def this() = this(null) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala new file mode 100644 index 0000000000000..75ea0e8459df8 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.types._ + +/** + * A parent class for mutable container objects that are reused when the values are changed, + * resulting in less garbage. These values are held by a [[SpecificMutableRow]]. + * + * The following code was roughly used to generate these objects: + * {{{ + * val types = "Int,Float,Boolean,Double,Short,Long,Byte,Any".split(",") + * types.map {tpe => + * s""" + * final class Mutable$tpe extends MutableValue { + * var value: $tpe = 0 + * def boxed = if (isNull) null else value + * def update(v: Any) = value = { + * isNull = false + * v.asInstanceOf[$tpe] + * } + * def copy() = { + * val newCopy = new Mutable$tpe + * newCopy.isNull = isNull + * newCopy.value = value + * newCopy.asInstanceOf[this.type] + * } + * }""" + * }.foreach(println) + * + * types.map { tpe => + * s""" + * override def set$tpe(ordinal: Int, value: $tpe): Unit = { + * val currentValue = values(ordinal).asInstanceOf[Mutable$tpe] + * currentValue.isNull = false + * currentValue.value = value + * } + * + * override def get$tpe(i: Int): $tpe = { + * values(i).asInstanceOf[Mutable$tpe].value + * }""" + * }.foreach(println) + * }}} + */ +abstract class MutableValue extends Serializable { + var isNull: Boolean = true + def boxed: Any + def update(v: Any) + def copy(): this.type +} + +final class MutableInt extends MutableValue { + var value: Int = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Int] + } + def copy() = { + val newCopy = new MutableInt + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableFloat extends MutableValue { + var value: Float = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Float] + } + def copy() = { + val newCopy = new MutableFloat + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableBoolean extends MutableValue { + var value: Boolean = false + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Boolean] + } + def copy() = { + val newCopy = new MutableBoolean + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableDouble extends MutableValue { + var value: Double = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Double] + } + def copy() = { + val newCopy = new MutableDouble + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableShort extends MutableValue { + var value: Short = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Short] + } + def copy() = { + val newCopy = new MutableShort + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableLong extends MutableValue { + var value: Long = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Long] + } + def copy() = { + val newCopy = new MutableLong + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableByte extends MutableValue { + var value: Byte = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Byte] + } + def copy() = { + val newCopy = new MutableByte + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableAny extends MutableValue { + var value: Any = 0 + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Any] + } + def copy() = { + val newCopy = new MutableAny + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +/** + * A row type that holds an array specialized container objects, of type [[MutableValue]], chosen + * based on the dataTypes of each column. The intent is to decrease garbage when modifying the + * values of primitive columns. + */ +final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableRow { + + def this(dataTypes: Seq[DataType]) = + this( + dataTypes.map { + case IntegerType => new MutableInt + case ByteType => new MutableByte + case FloatType => new MutableFloat + case ShortType => new MutableShort + case DoubleType => new MutableDouble + case BooleanType => new MutableBoolean + case LongType => new MutableLong + case _ => new MutableAny + }.toArray) + + def this() = this(Seq.empty) + + override def length: Int = values.length + + override def setNullAt(i: Int): Unit = { + values(i).isNull = true + } + + override def apply(i: Int): Any = values(i).boxed + + override def isNullAt(i: Int): Boolean = values(i).isNull + + override def copy(): Row = { + val newValues = new Array[MutableValue](values.length) + var i = 0 + while (i < values.length) { + newValues(i) = values(i).copy() + i += 1 + } + new SpecificMutableRow(newValues) + } + + override def update(ordinal: Int, value: Any): Unit = values(ordinal).update(value) + + override def iterator: Iterator[Any] = values.map(_.boxed).iterator + + def setString(ordinal: Int, value: String) = update(ordinal, value) + + def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String] + + override def setInt(ordinal: Int, value: Int): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableInt] + currentValue.isNull = false + currentValue.value = value + } + + override def getInt(i: Int): Int = { + values(i).asInstanceOf[MutableInt].value + } + + override def setFloat(ordinal: Int, value: Float): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableFloat] + currentValue.isNull = false + currentValue.value = value + } + + override def getFloat(i: Int): Float = { + values(i).asInstanceOf[MutableFloat].value + } + + override def setBoolean(ordinal: Int, value: Boolean): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableBoolean] + currentValue.isNull = false + currentValue.value = value + } + + override def getBoolean(i: Int): Boolean = { + values(i).asInstanceOf[MutableBoolean].value + } + + override def setDouble(ordinal: Int, value: Double): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableDouble] + currentValue.isNull = false + currentValue.value = value + } + + override def getDouble(i: Int): Double = { + values(i).asInstanceOf[MutableDouble].value + } + + override def setShort(ordinal: Int, value: Short): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableShort] + currentValue.isNull = false + currentValue.value = value + } + + override def getShort(i: Int): Short = { + values(i).asInstanceOf[MutableShort].value + } + + override def setLong(ordinal: Int, value: Long): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableLong] + currentValue.isNull = false + currentValue.value = value + } + + override def getLong(i: Int): Long = { + values(i).asInstanceOf[MutableLong].value + } + + override def setByte(ordinal: Int, value: Byte): Unit = { + val currentValue = values(ordinal).asInstanceOf[MutableByte] + currentValue.isNull = false + currentValue.value = value + } + + override def getByte(i: Int): Byte = { + values(i).asInstanceOf[MutableByte].value + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 01947273b6ccc..613b87ca98d97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -22,6 +22,7 @@ import com.clearspring.analytics.stream.cardinality.HyperLogLog import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.util.collection.OpenHashSet abstract class AggregateExpression extends Expression { self: Product => @@ -161,13 +162,88 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod override def newInstance() = new CountFunction(child, this) } -case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { +case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate { + def this() = this(null) + override def children = expressions override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = LongType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" override def newInstance() = new CountDistinctFunction(expressions, this) + + override def asPartial = { + val partialSet = Alias(CollectHashSet(expressions), "partialSets")() + SplitEvaluation( + CombineSetsAndCount(partialSet.toAttribute), + partialSet :: Nil) + } +} + +case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpression { + def this() = this(null) + + override def children = expressions + override def references = expressions.flatMap(_.references).toSet + override def nullable = false + override def dataType = ArrayType(expressions.head.dataType) + override def toString = s"AddToHashSet(${expressions.mkString(",")})" + override def newInstance() = new CollectHashSetFunction(expressions, this) +} + +case class CollectHashSetFunction( + @transient expr: Seq[Expression], + @transient base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new OpenHashSet[Any]() + + @transient + val distinctValue = new InterpretedProjection(expr) + + override def update(input: Row): Unit = { + val evaluatedExpr = distinctValue(input) + if (!evaluatedExpr.anyNull) { + seen.add(evaluatedExpr) + } + } + + override def eval(input: Row): Any = { + seen + } +} + +case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression { + def this() = this(null) + + override def children = inputSet :: Nil + override def references = inputSet.references + override def nullable = false + override def dataType = LongType + override def toString = s"CombineAndCount($inputSet)" + override def newInstance() = new CombineSetsAndCountFunction(inputSet, this) +} + +case class CombineSetsAndCountFunction( + @transient inputSet: Expression, + @transient base: AggregateExpression) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new OpenHashSet[Any]() + + override def update(input: Row): Unit = { + val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]] + val inputIterator = inputSetEval.iterator + while (inputIterator.hasNext) { + seen.add(inputIterator.next) + } + } + + override def eval(input: Row): Any = seen.size.toLong } case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) @@ -379,17 +455,22 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) } -case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpression) +case class CountDistinctFunction( + @transient expr: Seq[Expression], + @transient base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - val seen = new scala.collection.mutable.HashSet[Any]() + val seen = new OpenHashSet[Any]() + + @transient + val distinctValue = new InterpretedProjection(expr) override def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.eval(input)) - if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { - seen += evaluatedExpr + val evaluatedExpr = distinctValue(input) + if (!evaluatedExpr.anyNull) { + seen.add(evaluatedExpr) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index c79c1847cedf5..8d90614e4501a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -85,3 +85,34 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } + +case class MaxOf(left: Expression, right: Expression) extends Expression { + type EvaluatedType = Any + + override def nullable = left.nullable && right.nullable + + override def children = left :: right :: Nil + + override def references = left.references ++ right.references + + override def dataType = left.dataType + + override def eval(input: Row): Any = { + val leftEval = left.eval(input) + val rightEval = right.eval(input) + if (leftEval == null) { + rightEval + } else if (rightEval == null) { + leftEval + } else { + val numeric = left.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]] + if (numeric.compare(leftEval, rightEval) < 0) { + rightEval + } else { + leftEval + } + } + } + + override def toString = s"MaxOf($left, $right)" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index de2d67ce82ff1..5a3f013c34579 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -26,6 +26,10 @@ import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ +// These classes are here to avoid issues with serialization and integration with quasiquotes. +class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int] +class LongHashSet extends org.apache.spark.util.collection.OpenHashSet[Long] + /** * A base class for generators of byte code to perform expression evaluation. Includes a set of * helpers for referring to Catalyst types and building trees that perform evaluation of individual @@ -50,6 +54,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin private val curId = new java.util.concurrent.atomic.AtomicInteger() private val javaSeparator = "$" + /** + * Can be flipped on manually in the console to add (expensive) expression evaluation trace code. + */ + var debugLogging = false + /** * Generates a class for a given input expression. Called when there is not cached code * already available. @@ -71,7 +80,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin * From the Guava Docs: A Cache is similar to ConcurrentMap, but not quite the same. The most * fundamental difference is that a ConcurrentMap persists all elements that are added to it until * they are explicitly removed. A Cache on the other hand is generally configured to evict entries - * automatically, in order to constrain its memory footprint + * automatically, in order to constrain its memory footprint. Note that this cache does not use + * weak keys/values and thus does not respond to memory pressure. */ protected val cache = CacheBuilder.newBuilder() .maximumSize(1000) @@ -403,6 +413,78 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin $primitiveTerm = ${falseEval.primitiveTerm} } """.children + + case NewSet(elementType) => + q""" + val $nullTerm = false + val $primitiveTerm = new ${hashSetForType(elementType)}() + """.children + + case AddItemToSet(item, set) => + val itemEval = expressionEvaluator(item) + val setEval = expressionEvaluator(set) + + val ArrayType(elementType, _) = set.dataType + + itemEval.code ++ setEval.code ++ + q""" + if (!${itemEval.nullTerm}) { + ${setEval.primitiveTerm} + .asInstanceOf[${hashSetForType(elementType)}] + .add(${itemEval.primitiveTerm}) + } + + val $nullTerm = false + val $primitiveTerm = ${setEval.primitiveTerm} + """.children + + case CombineSets(left, right) => + val leftEval = expressionEvaluator(left) + val rightEval = expressionEvaluator(right) + + val ArrayType(elementType, _) = left.dataType + + leftEval.code ++ rightEval.code ++ + q""" + val $nullTerm = false + var $primitiveTerm: ${hashSetForType(elementType)} = null + + { + val leftSet = ${leftEval.primitiveTerm}.asInstanceOf[${hashSetForType(elementType)}] + val rightSet = ${rightEval.primitiveTerm}.asInstanceOf[${hashSetForType(elementType)}] + val iterator = rightSet.iterator + while (iterator.hasNext) { + leftSet.add(iterator.next()) + } + $primitiveTerm = leftSet + } + """.children + + case MaxOf(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = ${defaultPrimitive(e1.dataType)} + + if (${eval1.nullTerm}) { + $nullTerm = ${eval2.nullTerm} + $primitiveTerm = ${eval2.primitiveTerm} + } else if (${eval2.nullTerm}) { + $nullTerm = ${eval1.nullTerm} + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $nullTerm = false + if (${eval1.primitiveTerm} > ${eval2.primitiveTerm}) { + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $primitiveTerm = ${eval2.primitiveTerm} + } + } + """.children + } // If there was no match in the partial function above, we fall back on calling the interpreted @@ -420,7 +502,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin // Only inject debugging code if debugging is turned on. val debugCode = - if (log.isDebugEnabled) { + if (debugLogging) { val localLogger = log val localLoggerTree = reify { localLogger } q""" @@ -454,6 +536,13 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def accessorForType(dt: DataType) = newTermName(s"get${primitiveForType(dt)}") protected def mutatorForType(dt: DataType) = newTermName(s"set${primitiveForType(dt)}") + protected def hashSetForType(dt: DataType) = dt match { + case IntegerType => typeOf[IntegerHashSet] + case LongType => typeOf[LongHashSet] + case unsupportedType => + sys.error(s"Code generation not support for hashset of type $unsupportedType") + } + protected def primitiveForType(dt: DataType) = dt match { case IntegerType => "Int" case LongType => "Long" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 77fa02c13de30..7871a62620478 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -69,8 +69,10 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { ..${evaluatedExpression.code} if(${evaluatedExpression.nullTerm}) setNullAt($iLit) - else + else { + nullBits($iLit) = false $elementName = ${evaluatedExpression.primitiveTerm} + } } """.children : Seq[Tree] } @@ -106,9 +108,10 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { if(value == null) { setNullAt(i) } else { + nullBits(i) = false $elementName = value.asInstanceOf[${termForType(e.dataType)}] - return } + return }""" } q"final def update(i: Int, value: Any): Unit = { ..$cases; $accessorFailure }" @@ -137,7 +140,7 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? - q"if(i == $i) { $elementName = value; return }" :: Nil + q"if(i == $i) { nullBits($i) = false; $elementName = value; return }" :: Nil case _ => Nil } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala new file mode 100644 index 0000000000000..e6c570b47bee2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.util.collection.OpenHashSet + +/** + * Creates a new set of the specified type + */ +case class NewSet(elementType: DataType) extends LeafExpression { + type EvaluatedType = Any + + def references = Set.empty + + def nullable = false + + // We are currently only using these Expressions internally for aggregation. However, if we ever + // expose these to users we'll want to create a proper type instead of hijacking ArrayType. + def dataType = ArrayType(elementType) + + def eval(input: Row): Any = { + new OpenHashSet[Any]() + } + + override def toString = s"new Set($dataType)" +} + +/** + * Adds an item to a set. + * For performance, this expression mutates its input during evaluation. + */ +case class AddItemToSet(item: Expression, set: Expression) extends Expression { + type EvaluatedType = Any + + def children = item :: set :: Nil + + def nullable = set.nullable + + def dataType = set.dataType + + def references = (item.flatMap(_.references) ++ set.flatMap(_.references)).toSet + + def eval(input: Row): Any = { + val itemEval = item.eval(input) + val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] + + if (itemEval != null) { + if (setEval != null) { + setEval.add(itemEval) + setEval + } else { + null + } + } else { + setEval + } + } + + override def toString = s"$set += $item" +} + +/** + * Combines the elements of two sets. + * For performance, this expression mutates its left input set during evaluation. + */ +case class CombineSets(left: Expression, right: Expression) extends BinaryExpression { + type EvaluatedType = Any + + def nullable = left.nullable || right.nullable + + def dataType = left.dataType + + def symbol = "++=" + + def eval(input: Row): Any = { + val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]] + if(leftEval != null) { + val rightEval = right.eval(input).asInstanceOf[OpenHashSet[Any]] + if (rightEval != null) { + val iterator = rightEval.iterator + while(iterator.hasNext) { + val rightValue = iterator.next() + leftEval.add(rightValue) + } + leftEval + } else { + null + } + } else { + null + } + } +} + +/** + * Returns the number of elements in the input set. + */ +case class CountSet(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def nullable = child.nullable + + def dataType = LongType + + def eval(input: Row): Any = { + val childEval = child.eval(input).asInstanceOf[OpenHashSet[Any]] + if (childEval != null) { + childEval.size.toLong + } + } + + override def toString = s"$child.count()" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index cd04bdf02cf84..96ce35939e2cc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -280,7 +280,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { try { - val defaultCtor = getClass.getConstructors.head + // Skip no-arg constructors that are just there for kryo. + val defaultCtor = getClass.getConstructors.find(_.getParameterTypes.size != 0).head if (otherCopyArgs.isEmpty) { defaultCtor.newInstance(newArgs: _*).asInstanceOf[this.type] } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 999c9fff38d60..f1df817c41362 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -136,6 +136,16 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) } + test("MaxOf") { + checkEvaluation(MaxOf(1, 2), 2) + checkEvaluation(MaxOf(2, 1), 2) + checkEvaluation(MaxOf(1L, 2L), 2L) + checkEvaluation(MaxOf(2L, 1L), 2L) + + checkEvaluation(MaxOf(Literal(null, IntegerType), 2), 2) + checkEvaluation(MaxOf(2, Literal(null, IntegerType)), 2) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal(null, StringType).like("a"), null) checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 463a1d32d7fd7..be9f155253d77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -175,7 +175,7 @@ case class Aggregate( private[this] val resultProjection = new InterpretedMutableProjection( resultExpressions, computedSchema ++ namedGroups.map(_._2)) - private[this] val joinedRow = new JoinedRow + private[this] val joinedRow = new JoinedRow4 override final def hasNext: Boolean = hashTableIter.hasNext diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 4a26934c49c93..31ad5e8aabb0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -103,6 +103,40 @@ case class GeneratedAggregate( updateCount :: updateSum :: Nil, result ) + + case m @ Max(expr) => + val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() + val initialValue = Literal(null, expr.dataType) + val updateMax = MaxOf(currentMax, expr) + + AggregateEvaluation( + currentMax :: Nil, + initialValue :: Nil, + updateMax :: Nil, + currentMax) + + case CollectHashSet(Seq(expr)) => + val set = AttributeReference("hashSet", ArrayType(expr.dataType), nullable = false)() + val initialValue = NewSet(expr.dataType) + val addToSet = AddItemToSet(expr, set) + + AggregateEvaluation( + set :: Nil, + initialValue :: Nil, + addToSet :: Nil, + set) + + case CombineSetsAndCount(inputSet) => + val ArrayType(inputType, _) = inputSet.dataType + val set = AttributeReference("hashSet", inputSet.dataType, nullable = false)() + val initialValue = NewSet(inputType) + val collectSets = CombineSets(set, inputSet) + + AggregateEvaluation( + set :: Nil, + initialValue :: Nil, + collectSets :: Nil, + CountSet(set)) } val computationSchema = computeFunctions.flatMap(_.schema) @@ -151,7 +185,7 @@ case class GeneratedAggregate( (namedGroups.map(_._2.toAttribute) ++ computationSchema).toSeq) log.info(s"Result Projection: ${resultExpressions.mkString(",")}") - val joinedRow = new JoinedRow + val joinedRow = new JoinedRow3 if (groupingExpressions.isEmpty) { // TODO: Codegening anything other than the updateProjection is probably over kill. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 34654447a5f4b..077e6ebc5f11e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -28,9 +28,13 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.expressions.codegen.{IntegerHashSet, LongHashSet} + private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { val kryo = new Kryo() @@ -41,6 +45,13 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[com.clearspring.analytics.stream.cardinality.HyperLogLog], new HyperLogLogSerializer) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) + + // Specific hashsets must come first TODO: Move to core. + kryo.register(classOf[IntegerHashSet], new IntegerHashSetSerializer) + kryo.register(classOf[LongHashSet], new LongHashSetSerializer) + kryo.register(classOf[org.apache.spark.util.collection.OpenHashSet[_]], + new OpenHashSetSerializer) + kryo.setReferences(false) kryo.setClassLoader(Utils.getSparkClassLoader) new AllScalaRegistrar().apply(kryo) @@ -109,3 +120,78 @@ private[sql] class HyperLogLogSerializer extends Serializer[HyperLogLog] { HyperLogLog.Builder.build(bytes) } } + +private[sql] class OpenHashSetSerializer extends Serializer[OpenHashSet[_]] { + def write(kryo: Kryo, output: Output, hs: OpenHashSet[_]) { + val rowSerializer = kryo.getDefaultSerializer(classOf[Array[Any]]).asInstanceOf[Serializer[Any]] + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val row = iterator.next() + rowSerializer.write(kryo, output, row.asInstanceOf[GenericRow].values) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[OpenHashSet[_]]): OpenHashSet[_] = { + val rowSerializer = kryo.getDefaultSerializer(classOf[Array[Any]]).asInstanceOf[Serializer[Any]] + val numItems = input.readInt() + val set = new OpenHashSet[Any](numItems + 1) + var i = 0 + while (i < numItems) { + val row = + new GenericRow(rowSerializer.read( + kryo, + input, + classOf[Array[Any]].asInstanceOf[Class[Any]]).asInstanceOf[Array[Any]]) + set.add(row) + i += 1 + } + set + } +} + +private[sql] class IntegerHashSetSerializer extends Serializer[IntegerHashSet] { + def write(kryo: Kryo, output: Output, hs: IntegerHashSet) { + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val value: Int = iterator.next() + output.writeInt(value) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[IntegerHashSet]): IntegerHashSet = { + val numItems = input.readInt() + val set = new IntegerHashSet + var i = 0 + while (i < numItems) { + val value = input.readInt() + set.add(value) + i += 1 + } + set + } +} + +private[sql] class LongHashSetSerializer extends Serializer[LongHashSet] { + def write(kryo: Kryo, output: Output, hs: LongHashSet) { + output.writeInt(hs.size) + val iterator = hs.iterator + while(iterator.hasNext) { + val value = iterator.next() + output.writeLong(value) + } + } + + def read(kryo: Kryo, input: Input, tpe: Class[LongHashSet]): LongHashSet = { + val numItems = input.readInt() + val set = new LongHashSet + var i = 0 + while (i < numItems) { + val value = input.readLong() + set.add(value) + i += 1 + } + set + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f0c958fdb537f..517b77804ae2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ @@ -148,7 +149,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]) = !aggs.exists { - case _: Sum | _: Count => false + case _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + // The generated set implementation is pretty limited ATM. + case CollectHashSet(exprs) if exprs.size == 1 && + Seq(IntegerType, LongType).contains(exprs.head.dataType) => false case _ => true } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala index b08f9aacc1fcb..2890a563bed48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -92,7 +92,7 @@ trait HashJoin { private[this] var currentMatchPosition: Int = -1 // Mutable per row objects. - private[this] val joinRow = new JoinedRow + private[this] val joinRow = new JoinedRow2 private[this] val joinKeys = streamSideKeyGenerator() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 0a3b59cbc233a..ef4526ec03439 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -23,7 +23,7 @@ import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} import parquet.schema.MessageType import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.parquet.CatalystConverter.FieldType /** @@ -278,14 +278,14 @@ private[parquet] class CatalystGroupConverter( */ private[parquet] class CatalystPrimitiveRowConverter( protected[parquet] val schema: Array[FieldType], - protected[parquet] var current: ParquetRelation.RowType) + protected[parquet] var current: MutableRow) extends CatalystConverter { // This constructor is used for the root converter only def this(attributes: Array[Attribute]) = this( attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), - new ParquetRelation.RowType(attributes.length)) + new SpecificMutableRow(attributes.map(_.dataType))) protected [parquet] val converters: Array[Converter] = schema.zipWithIndex.map { @@ -299,7 +299,7 @@ private[parquet] class CatalystPrimitiveRowConverter( override val parent = null // Should be only called in root group converter! - override def getCurrentRecord: ParquetRelation.RowType = current + override def getCurrentRecord: Row = current override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index f6cfab736d98a..a5a5d139a65cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -139,7 +139,7 @@ case class ParquetTableScan( partOutput.map(a => Cast(Literal(partValues(a.name)), a.dataType).eval(EmptyRow)) new Iterator[Row] { - private[this] val joinedRow = new JoinedRow(Row(partitionRowValues:_*), null) + private[this] val joinedRow = new JoinedRow5(Row(partitionRowValues:_*), null) def hasNext = iter.hasNext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 76b1724471442..37d64f0de7bab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -45,16 +45,16 @@ class PlannerSuite extends FunSuite { assert(aggregations.size === 2) } - test("count distinct is not partially aggregated") { + test("count distinct is partially aggregated") { val query = testData.groupBy('value)(CountDistinct('key :: Nil)).queryExecution.analyzed val planned = HashAggregation(query) - assert(planned.isEmpty) + assert(planned.nonEmpty) } - test("mixed aggregates are not partially aggregated") { + test("mixed aggregates are partially aggregated") { val query = testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).queryExecution.analyzed val planned = HashAggregation(query) - assert(planned.isEmpty) + assert(planned.nonEmpty) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 3b371211e14cd..6571c35499ef4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -265,9 +265,9 @@ private[hive] case class MetastoreRelation // org.apache.hadoop.hive.ql.metadata.Partition will cause a NotSerializableException // which indicates the SerDe we used is not Serializable. - @transient lazy val hiveQlTable = new Table(table) + @transient val hiveQlTable = new Table(table) - def hiveQlPartitions = partitions.map { p => + @transient val hiveQlPartitions = partitions.map { p => new Partition(hiveQlTable, p) } diff --git a/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc b/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 0 values-0-1843b7947729b771fee3a4abd050bfdc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 b/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value + null long-0-89b850197b326239d60a5e1d5db7c9c9 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 b/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value + null-0-a014038c00fb81e88041ed4a8368e6f7 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff b/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value long-0-77b9ed1d7ae65fa53830a3bc586856ff @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d b/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value strings-0-c68e75ec4c884b93765a466e992e391d @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 b/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 1 value-0-a4047b06a324fb5ea400c94350c9e038 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e b/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values including null-0-75672236a30e10dab13b9b246c5a3a1e @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 b/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values long-0-f4ec7d767ba8c49d41edf5d6f58cf6d1 @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e b/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/count distinct 2 values-0-c61df65af167acaf7edb174e77898f3e @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-52b0e534c7df544258a1c59df9f816ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 7c82964b5ecdc..8d6ca9939a730 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.scalatest.BeforeAndAfterAll + import scala.reflect.ClassTag @@ -26,7 +28,9 @@ import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -class StatisticsSuite extends QueryTest { +class StatisticsSuite extends QueryTest with BeforeAndAfterAll { + TestHive.reset() + TestHive.cacheTables = false test("parse analyze commands") { def assertAnalyzeCommand(analyzeCommand: String, c: Class[_]) { @@ -126,7 +130,7 @@ class StatisticsSuite extends QueryTest { val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => mr.statistics.sizeInBytes } - assert(sizes.size === 1) + assert(sizes.size === 1, s"Size wrong for:\n ${rdd.queryExecution}") assert(sizes(0).equals(BigInt(5812)), s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index fdb2f41f5a5b6..26e4ec6e6dcce 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -32,6 +32,71 @@ case class TestData(a: Int, b: String) */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("count distinct 0 values", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 'a' AS a FROM src LIMIT 0) table + """.stripMargin) + + createQueryTest("count distinct 1 value strings", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 'a' AS a FROM src LIMIT 1 UNION ALL + | SELECT 'b' AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 2 AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values including null", + """ + |SELECT COUNT(DISTINCT a, 1) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value + null", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT 1 AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 1L AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 2 values long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 2L AS a FROM src LIMIT 1) table + """.stripMargin) + + createQueryTest("count distinct 1 value + null long", + """ + |SELECT COUNT(DISTINCT a) FROM ( + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT 1L AS a FROM src LIMIT 1 UNION ALL + | SELECT null AS a FROM src LIMIT 1) table + """.stripMargin) + createQueryTest("null case", "SELECT case when(true) then 1 else null end FROM src LIMIT 1") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index df9bae96494d5..8bc72384a64ee 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -17,10 +17,19 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.hive.test.TestHive + /** * A set of tests that validates support for Hive SerDe. */ -class HiveSerDeSuite extends HiveComparisonTest { +class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { + + override def beforeAll() = { + TestHive.cacheTables = false + } + createQueryTest( "Read and write with LazySimpleSerDe (tab separated)", "SELECT * from serdeins") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 1a6dbc0ce0c0d..8275e2d3bcce3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.test.TestHive /* Implicit conversions */ @@ -25,9 +27,10 @@ import scala.collection.JavaConversions._ /** * A set of test cases that validate partition and column pruning. */ -class PruningSuite extends HiveComparisonTest { +class PruningSuite extends HiveComparisonTest with BeforeAndAfter { // MINOR HACK: You must run a query before calling reset the first time. TestHive.sql("SHOW TABLES") + TestHive.cacheTables = false // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset // the environment to ensure all referenced tables in this suites are not cached in-memory. From 3519b5e8e55b4530d7f7c0bcab254f863dbfa814 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 23 Aug 2014 16:21:08 -0700 Subject: [PATCH 0455/1492] [SPARK-2967][SQL] Follow-up: Also copy hash expressions in sort based shuffle fix. Follow-up to #2066 Author: Michael Armbrust Closes #2072 from marmbrus/sortShuffle and squashes the following commits: 2ff8114 [Michael Armbrust] Fix bug --- .../scala/org/apache/spark/sql/execution/Exchange.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 09c34b7059fc3..4802e40595807 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -46,12 +46,15 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. val rdd = child.execute().mapPartitions { iter => - @transient val hashExpressions = - newMutableProjection(expressions, child.output)() - if (sortBasedShuffleOn) { + @transient val hashExpressions = + newProjection(expressions, child.output) + iter.map(r => (hashExpressions(r), r.copy())) } else { + @transient val hashExpressions = + newMutableProjection(expressions, child.output)() + val mutablePair = new MutablePair[Row, Row]() iter.map(r => mutablePair.update(hashExpressions(r), r)) } From db436e36c4e20893de708a0bc07a5a8877c49563 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 23 Aug 2014 18:55:13 -0700 Subject: [PATCH 0456/1492] [SPARK-2871] [PySpark] add `key` argument for max(), min() and top(n) RDD.max(key=None) param key: A function used to generate key for comparing >>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0]) >>> rdd.max() 43.0 >>> rdd.max(key=str) 5.0 RDD.min(key=None) Find the minimum item in this RDD. param key: A function used to generate key for comparing >>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0]) >>> rdd.min() 2.0 >>> rdd.min(key=str) 10.0 RDD.top(num, key=None) Get the top N elements from a RDD. Note: It returns the list sorted in descending order. >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) [12] >>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2) [6, 5] >>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str) [4, 3, 2] Author: Davies Liu Closes #2094 from davies/cmp and squashes the following commits: ccbaf25 [Davies Liu] add `key` to top() ad7e374 [Davies Liu] fix tests 2f63512 [Davies Liu] change `comp` to `key` in min/max dd91e08 [Davies Liu] add `comp` argument for RDD.max() and RDD.min() --- python/pyspark/rdd.py | 44 ++++++++++++++++++++++++++----------------- 1 file changed, 27 insertions(+), 17 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3eefc878d274e..bdd8bc82869fb 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -810,23 +810,37 @@ def func(iterator): return self.mapPartitions(func).fold(zeroValue, combOp) - def max(self): + def max(self, key=None): """ Find the maximum item in this RDD. - >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).max() + @param key: A function used to generate key for comparing + + >>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0]) + >>> rdd.max() 43.0 + >>> rdd.max(key=str) + 5.0 """ - return self.reduce(max) + if key is None: + return self.reduce(max) + return self.reduce(lambda a, b: max(a, b, key=key)) - def min(self): + def min(self, key=None): """ Find the minimum item in this RDD. - >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).min() - 1.0 + @param key: A function used to generate key for comparing + + >>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0]) + >>> rdd.min() + 2.0 + >>> rdd.min(key=str) + 10.0 """ - return self.reduce(min) + if key is None: + return self.reduce(min) + return self.reduce(lambda a, b: min(a, b, key=key)) def sum(self): """ @@ -924,7 +938,7 @@ def mergeMaps(m1, m2): return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) - def top(self, num): + def top(self, num, key=None): """ Get the top N elements from a RDD. @@ -933,20 +947,16 @@ def top(self, num): [12] >>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2) [6, 5] + >>> sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str) + [4, 3, 2] """ def topIterator(iterator): - q = [] - for k in iterator: - if len(q) < num: - heapq.heappush(q, k) - else: - heapq.heappushpop(q, k) - yield q + yield heapq.nlargest(num, iterator, key=key) def merge(a, b): - return next(topIterator(a + b)) + return heapq.nlargest(num, a + b, key=key) - return sorted(self.mapPartitions(topIterator).reduce(merge), reverse=True) + return self.mapPartitions(topIterator).reduce(merge) def takeOrdered(self, num, key=None): """ From 8df4dad4951ca6e687df1288331909878922a55f Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 23 Aug 2014 19:33:34 -0700 Subject: [PATCH 0457/1492] [SPARK-2871] [PySpark] add approx API for RDD RDD.countApprox(self, timeout, confidence=0.95) :: Experimental :: Approximate version of count() that returns a potentially incomplete result within a timeout, even if not all tasks have finished. >>> rdd = sc.parallelize(range(1000), 10) >>> rdd.countApprox(1000, 1.0) 1000 RDD.sumApprox(self, timeout, confidence=0.95) Approximate operation to return the sum within a timeout or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) >>> r = sum(xrange(1000)) >>> (rdd.sumApprox(1000) - r) / r < 0.05 RDD.meanApprox(self, timeout, confidence=0.95) :: Experimental :: Approximate operation to return the mean within a timeout or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) >>> r = sum(xrange(1000)) / 1000.0 >>> (rdd.meanApprox(1000) - r) / r < 0.05 True Author: Davies Liu Closes #2095 from davies/approx and squashes the following commits: e8c252b [Davies Liu] add approx API for RDD --- .../apache/spark/api/python/PythonRDD.scala | 17 ++++ python/pyspark/rdd.py | 81 +++++++++++++++++++ 2 files changed, 98 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 747023812f754..ae8010300a500 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -749,6 +749,23 @@ private[spark] object PythonRDD extends Logging { } } } + + /** + * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. + */ + def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj.asInstanceOf[JArrayList[_]] + } else { + Seq(obj) + } + } + }.toJavaRDD() + } } private diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index bdd8bc82869fb..9f88340d03778 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -131,6 +131,22 @@ def __exit__(self, type, value, tb): self._context._jsc.setCallSite(None) +class BoundedFloat(float): + """ + Bounded value is generated by approximate job, with confidence and low + bound and high bound. + + >>> BoundedFloat(100.0, 0.95, 95.0, 105.0) + 100.0 + """ + def __new__(cls, mean, confidence, low, high): + obj = float.__new__(cls, mean) + obj.confidence = confidence + obj.low = low + obj.high = high + return obj + + class MaxHeapQ(object): """ @@ -1792,6 +1808,71 @@ def _defaultReducePartitions(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. + def _is_pickled(self): + """ Return this RDD is serialized by Pickle or not. """ + der = self._jrdd_deserializer + if isinstance(der, PickleSerializer): + return True + if isinstance(der, BatchedSerializer) and isinstance(der.serializer, PickleSerializer): + return True + return False + + def _to_jrdd(self): + """ Return an JavaRDD of Object by unpickling + + It will convert each Python object into Java object by Pyrolite, whenever the + RDD is serialized in batch or not. + """ + if not self._is_pickled(): + self = self._reserialize(BatchedSerializer(PickleSerializer(), 1024)) + batched = isinstance(self._jrdd_deserializer, BatchedSerializer) + return self.ctx._jvm.PythonRDD.pythonToJava(self._jrdd, batched) + + def countApprox(self, timeout, confidence=0.95): + """ + :: Experimental :: + Approximate version of count() that returns a potentially incomplete + result within a timeout, even if not all tasks have finished. + + >>> rdd = sc.parallelize(range(1000), 10) + >>> rdd.countApprox(1000, 1.0) + 1000 + """ + drdd = self.mapPartitions(lambda it: [float(sum(1 for i in it))]) + return int(drdd.sumApprox(timeout, confidence)) + + def sumApprox(self, timeout, confidence=0.95): + """ + :: Experimental :: + Approximate operation to return the sum within a timeout + or meet the confidence. + + >>> rdd = sc.parallelize(range(1000), 10) + >>> r = sum(xrange(1000)) + >>> (rdd.sumApprox(1000) - r) / r < 0.05 + True + """ + jrdd = self.mapPartitions(lambda it: [float(sum(it))])._to_jrdd() + jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd()) + r = jdrdd.sumApprox(timeout, confidence).getFinalValue() + return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high()) + + def meanApprox(self, timeout, confidence=0.95): + """ + :: Experimental :: + Approximate operation to return the mean within a timeout + or meet the confidence. + + >>> rdd = sc.parallelize(range(1000), 10) + >>> r = sum(xrange(1000)) / 1000.0 + >>> (rdd.meanApprox(1000) - r) / r < 0.05 + True + """ + jrdd = self.map(float)._to_jrdd() + jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd()) + r = jdrdd.meanApprox(timeout, confidence).getFinalValue() + return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high()) + class PipelinedRDD(RDD): From 8861cdf11288f7597809e9e0e1cad66fb85dd946 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Sat, 23 Aug 2014 19:47:14 -0700 Subject: [PATCH 0458/1492] Clean unused code in SortShuffleWriter Just clean unused code which have been moved into ExternalSorter. Author: Raymond Liu Closes #1882 from colorant/sortShuffleWriter and squashes the following commits: e6337be [Raymond Liu] Clean unused code in SortShuffleWriter --- .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 22f656fa371ea..b8c9ad46ab035 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -17,12 +17,11 @@ package org.apache.spark.shuffle.sort -import java.io.{BufferedOutputStream, File, FileOutputStream, DataOutputStream} +import java.io.File import org.apache.spark.{MapOutputTracker, SparkEnv, Logging, TaskContext} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.scheduler.MapStatus -import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleWriter, BaseShuffleHandle} import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.collection.ExternalSorter @@ -37,10 +36,6 @@ private[spark] class SortShuffleWriter[K, V, C]( private val numPartitions = dep.partitioner.numPartitions private val blockManager = SparkEnv.get.blockManager - private val ser = Serializer.getSerializer(dep.serializer.orNull) - - private val conf = SparkEnv.get.conf - private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 private var sorter: ExternalSorter[K, V, _] = null private var outputFile: File = null From ded6796bf54f5c005b27135d7dec19634038a1c6 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 24 Aug 2014 09:43:44 -0700 Subject: [PATCH 0459/1492] [SPARK-3192] Some scripts have 2 space indentation but other scripts have 4 space indentation. Author: Kousuke Saruta Closes #2104 from sarutak/SPARK-3192 and squashes the following commits: db78419 [Kousuke Saruta] Modified indentation of spark-shell --- bin/spark-shell | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 8b7ccd7439551..0ab4e14f5b744 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -22,7 +22,7 @@ cygwin=false case "`uname`" in - CYGWIN*) cygwin=true;; + CYGWIN*) cygwin=true;; esac # Enter posix mode for bash @@ -32,9 +32,9 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" function usage() { - echo "Usage: ./bin/spark-shell [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 - exit 0 + echo "Usage: ./bin/spark-shell [options]" + $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -46,20 +46,20 @@ SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" function main() { - if $cygwin; then - # Workaround for issue involving JLine and Cygwin - # (see http://sourceforge.net/p/jline/bugs/40/). - # If you're using the Mintty terminal emulator in Cygwin, may need to set the - # "Backspace sends ^H" setting in "Keys" section of the Mintty options - # (see https://github.com/sbt/sbt/issues/562). - stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" - stty icanon echo > /dev/null 2>&1 - else - export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" - fi + if $cygwin; then + # Workaround for issue involving JLine and Cygwin + # (see http://sourceforge.net/p/jline/bugs/40/). + # If you're using the Mintty terminal emulator in Cygwin, may need to set the + # "Backspace sends ^H" setting in "Keys" section of the Mintty options + # (see https://github.com/sbt/sbt/issues/562). + stty -icanon min 1 -echo > /dev/null 2>&1 + export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + stty icanon echo > /dev/null 2>&1 + else + export SPARK_SUBMIT_OPTS + $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + fi } # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in From 572952ae615895efaaabcd509d582262000c0852 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Sun, 24 Aug 2014 17:33:33 -0700 Subject: [PATCH 0460/1492] [SPARK-2841][MLlib] Documentation for feature transformations Documentation for newly added feature transformations: 1. TF-IDF 2. StandardScaler 3. Normalizer Author: DB Tsai Closes #2068 from dbtsai/transformer-documentation and squashes the following commits: 109f324 [DB Tsai] address feedback --- docs/mllib-feature-extraction.md | 109 ++++++++++++++++++++++++++++++- 1 file changed, 107 insertions(+), 2 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 2031b96235ee9..44f0f76220b6e 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -1,7 +1,7 @@ --- layout: global -title: Feature Extraction - MLlib -displayTitle: MLlib - Feature Extraction +title: Feature Extraction and Transformation - MLlib +displayTitle: MLlib - Feature Extraction and Transformation --- * Table of contents @@ -148,3 +148,108 @@ for((synonym, cosineSimilarity) <- synonyms) { {% endhighlight %}
    + +## StandardScaler + +Standardizes features by scaling to unit variance and/or removing the mean using column summary +statistics on the samples in the training set. This is a very common pre-processing step. + +For example, RBF kernel of Support Vector Machines or the L1 and L2 regularized linear models +typically work better when all features have unit variance and/or zero mean. + +Standardization can improve the convergence rate during the optimization process, and also prevents +against features with very large variances exerting an overly large influence during model training. + +### Model Fitting + +[`StandardScaler`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) has the +following parameters in the constructor: + +* `withMean` False by default. Centers the data with mean before scaling. It will build a dense +output, so this does not work on sparse input and will raise an exception. +* `withStd` True by default. Scales the data to unit variance. + +We provide a [`fit`](api/scala/index.html#org.apache.spark.mllib.feature.StandardScaler) method in +`StandardScaler` which can take an input of `RDD[Vector]`, learn the summary statistics, and then +return a model which can transform the input dataset into unit variance and/or zero mean features +depending how we configure the `StandardScaler`. + +This model implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) +which can apply the standardization on a `Vector` to produce a transformed `Vector` or on +an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +Note that if the variance of a feature is zero, it will return default `0.0` value in the `Vector` +for that feature. + +### Example + +The example below demonstrates how to load a dataset in libsvm format, and standardize the features +so that the new features have unit variance and/or zero mean. + +
    +
    +{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.StandardScaler +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + +val scaler1 = new StandardScaler().fit(data.map(x => x.features)) +val scaler2 = new StandardScaler(withMean = true, withStd = true).fit(data.map(x => x.features)) + +// data1 will be unit variance. +val data1 = data.map(x => (x.label, scaler1.transform(x.features))) + +// Without converting the features into dense vectors, transformation with zero mean will raise +// exception on sparse vector. +// data2 will be unit variance and zero mean. +val data2 = data.map(x => (x.label, scaler2.transform(Vectors.dense(x.features.toArray)))) +{% endhighlight %} +
    +
    + +## Normalizer + +Normalizer scales individual samples to have unit $L^p$ norm. This is a common operation for text +classification or clustering. For example, the dot product of two $L^2$ normalized TF-IDF vectors +is the cosine similarity of the vectors. + +[`Normalizer`](api/scala/index.html#org.apache.spark.mllib.feature.Normalizer) has the following +parameter in the constructor: + +* `p` Normalization in $L^p$ space, $p = 2$ by default. + +`Normalizer` implements [`VectorTransformer`](api/scala/index.html#org.apache.spark.mllib.feature.VectorTransformer) +which can apply the normalization on a `Vector` to produce a transformed `Vector` or on +an `RDD[Vector]` to produce a transformed `RDD[Vector]`. + +Note that if the norm of the input is zero, it will return the input vector. + +### Example + +The example below demonstrates how to load a dataset in libsvm format, and normalizes the features +with $L^2$ norm, and $L^\infty$ norm. + +
    +
    +{% highlight scala %} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.feature.Normalizer +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") + +val normalizer1 = new Normalizer() +val normalizer2 = new Normalizer(p = Double.PositiveInfinity) + +// Each sample in data1 will be normalized using $L^2$ norm. +val data1 = data.map(x => (x.label, normalizer1.transform(x.features))) + +// Each sample in data2 will be normalized using $L^\infty$ norm. +val data2 = data.map(x => (x.label, normalizer2.transform(x.features))) +{% endhighlight %} +
    +
    From b1b20301b3a1b35564d61e58eb5964d5ad5e4d7d Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Sun, 24 Aug 2014 17:35:54 -0700 Subject: [PATCH 0461/1492] [MLlib][SPARK-2997] Update SVD documentation to reflect roughly square Update the documentation to reflect the fact we can handle roughly square matrices. Author: Reza Zadeh Closes #2070 from rezazadeh/svddocs and squashes the following commits: 826b8fe [Reza Zadeh] left singular vectors 3f34fc6 [Reza Zadeh] PCA is still TS 7ffa2aa [Reza Zadeh] better title aeaf39d [Reza Zadeh] More docs 788ed13 [Reza Zadeh] add computational cost explanation 6429c59 [Reza Zadeh] Add link to rowmatrix docs 1eeab8b [Reza Zadeh] Update SVD documentation to reflect roughly square --- docs/mllib-dimensionality-reduction.md | 29 ++++++++++++++++++++------ 1 file changed, 23 insertions(+), 6 deletions(-) diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 065d646496131..9f2cf6d48ec75 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Dimensionality Reduction of reducing the number of variables under consideration. It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -MLlib provides support for dimensionality reduction on tall-and-skinny matrices. +MLlib provides support for dimensionality reduction on the RowMatrix class. ## Singular value decomposition (SVD) @@ -39,8 +39,26 @@ If we keep the top $k$ singular values, then the dimensions of the resulting low * `$\Sigma$`: `$k \times k$`, * `$V$`: `$n \times k$`. -MLlib provides SVD functionality to row-oriented matrices that have only a few columns, -say, less than $1000$, but many rows, i.e., *tall-and-skinny* matrices. +### Performance +We assume $n$ is smaller than $m$. The singular values and the right singular vectors are derived +from the eigenvalues and the eigenvectors of the Gramian matrix $A^T A$. The matrix +storing the left singular vectors $U$, is computed via matrix multiplication as +$U = A (V S^{-1})$, if requested by the user via the computeU parameter. +The actual method to use is determined automatically based on the computational cost: + +* If $n$ is small ($n < 100$) or $k$ is large compared with $n$ ($k > n / 2$), we compute the Gramian matrix +first and then compute its top eigenvalues and eigenvectors locally on the driver. +This requires a single pass with $O(n^2)$ storage on each executor and on the driver, and +$O(n^2 k)$ time on the driver. +* Otherwise, we compute $(A^T A) v$ in a distributive way and send it to +ARPACK to +compute $(A^T A)$'s top eigenvalues and eigenvectors on the driver node. This requires $O(k)$ +passes, $O(n)$ storage on each executor, and $O(n k)$ storage on the driver. + +### SVD Example + +MLlib provides SVD functionality to row-oriented matrices, provided in the +RowMatrix class.
    @@ -124,9 +142,8 @@ MLlib supports PCA for tall-and-skinny matrices stored in row-oriented format.
    -The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +The following code demonstrates how to compute principal components on a `RowMatrix` and use them to project the vectors into a low-dimensional space. -The number of columns should be small, e.g, less than 1000. {% highlight scala %} import org.apache.spark.mllib.linalg.Matrix @@ -144,7 +161,7 @@ val projected: RowMatrix = mat.multiply(pc)
    -The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +The following code demonstrates how to compute principal components on a `RowMatrix` and use them to project the vectors into a low-dimensional space. The number of columns should be small, e.g, less than 1000. From fb0db772421b6902b80137bf769db3b418ab2ccf Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 24 Aug 2014 21:16:05 -0700 Subject: [PATCH 0462/1492] [SPARK-2871] [PySpark] add zipWithIndex() and zipWithUniqueId() RDD.zipWithIndex() Zips this RDD with its element indices. The ordering is first based on the partition index and then the ordering of items within each partition. So the first item in the first partition gets index 0, and the last item in the last partition receives the largest index. This method needs to trigger a spark job when this RDD contains more than one partitions. >>> sc.parallelize(range(4), 2).zipWithIndex().collect() [(0, 0), (1, 1), (2, 2), (3, 3)] RDD.zipWithUniqueId() Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method won't trigger a spark job, which is different from L{zipWithIndex} >>> sc.parallelize(range(4), 2).zipWithUniqueId().collect() [(0, 0), (2, 1), (1, 2), (3, 3)] Author: Davies Liu Closes #2092 from davies/zipWith and squashes the following commits: cebe5bf [Davies Liu] improve test cases, reverse the order of index 0d2a128 [Davies Liu] add zipWithIndex() and zipWithUniqueId() --- python/pyspark/rdd.py | 47 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 47 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 9f88340d03778..1374f74968c9e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1741,6 +1741,53 @@ def batch_as(rdd, batchSize): other._jrdd_deserializer) return RDD(pairRDD, self.ctx, deserializer) + def zipWithIndex(self): + """ + Zips this RDD with its element indices. + + The ordering is first based on the partition index and then the + ordering of items within each partition. So the first item in + the first partition gets index 0, and the last item in the last + partition receives the largest index. + + This method needs to trigger a spark job when this RDD contains + more than one partitions. + + >>> sc.parallelize(["a", "b", "c", "d"], 3).zipWithIndex().collect() + [('a', 0), ('b', 1), ('c', 2), ('d', 3)] + """ + starts = [0] + if self.getNumPartitions() > 1: + nums = self.mapPartitions(lambda it: [sum(1 for i in it)]).collect() + for i in range(len(nums) - 1): + starts.append(starts[-1] + nums[i]) + + def func(k, it): + for i, v in enumerate(it, starts[k]): + yield v, i + + return self.mapPartitionsWithIndex(func) + + def zipWithUniqueId(self): + """ + Zips this RDD with generated unique Long ids. + + Items in the kth partition will get ids k, n+k, 2*n+k, ..., where + n is the number of partitions. So there may exist gaps, but this + method won't trigger a spark job, which is different from + L{zipWithIndex} + + >>> sc.parallelize(["a", "b", "c", "d", "e"], 3).zipWithUniqueId().collect() + [('a', 0), ('b', 1), ('c', 4), ('d', 2), ('e', 5)] + """ + n = self.getNumPartitions() + + def func(k, it): + for i, v in enumerate(it): + yield v, i * n + k + + return self.mapPartitionsWithIndex(func) + def name(self): """ Return the name of this RDD. From 220f413686ae922bd11776576bf37610cce92c23 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 25 Aug 2014 12:30:02 -0700 Subject: [PATCH 0463/1492] [SPARK-2495][MLLIB] make KMeans constructor public to re-construct k-means models freeman-lab Author: Xiangrui Meng Closes #2112 from mengxr/public-constructors and squashes the following commits: 18d53a9 [Xiangrui Meng] make KMeans constructor public --- .../scala/org/apache/spark/mllib/clustering/KMeansModel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 5823cb6e52e7f..12a3d91cd31a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -25,7 +25,7 @@ import org.apache.spark.mllib.linalg.Vector /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Serializable { +class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { /** Total number of clusters. */ def k: Int = clusterCenters.length From cd30db566a327ddf63cd242c758e46ce2d9479df Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 25 Aug 2014 13:29:07 -0700 Subject: [PATCH 0464/1492] SPARK-2798 [BUILD] Correct several small errors in Flume module pom.xml files (EDIT) Since the scalatest issue was since resolved, this is now about a few small problems in the Flume Sink `pom.xml` - `scalatest` is not declared as a test-scope dependency - Its Avro version doesn't match the rest of the build - Its Flume version is not synced with the other Flume module - The other Flume module declares its dependency on Flume Sink slightly incorrectly, hard-coding the Scala 2.10 version - It depends on Scala Lang directly, which it shouldn't Author: Sean Owen Closes #1726 from srowen/SPARK-2798 and squashes the following commits: a46e2c6 [Sean Owen] scalatest to test scope, harmonize Avro and Flume versions, remove direct Scala dependency, fix '2.10' in Flume dependency --- external/flume-sink/pom.xml | 15 ++++++--------- external/flume/pom.xml | 12 ++++++------ pom.xml | 1 + 3 files changed, 13 insertions(+), 15 deletions(-) diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 19192e40a7dc3..c1e8e65464fc1 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -25,19 +25,20 @@ ../../pom.xml + org.apache.spark spark-streaming-flume-sink_2.10 streaming-flume-sink - jar Spark Project External Flume Sink http://spark.apache.org/ + org.apache.flume flume-ng-sdk - 1.4.0 + ${flume.version} io.netty @@ -52,7 +53,7 @@ org.apache.flume flume-ng-core - 1.4.0 + ${flume.version} io.netty @@ -62,11 +63,7 @@ org.apache.thrift libthrift - - - - org.scala-lang - scala-library + org.scalatest @@ -97,7 +94,7 @@ org.apache.avro avro-maven-plugin - 1.7.3 + ${avro.version} ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro diff --git a/external/flume/pom.xml b/external/flume/pom.xml index c532705f3950c..f71f6b6c4f931 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -40,6 +40,11 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.apache.spark + spark-streaming-flume-sink_${scala.binary.version} + ${project.version} + org.apache.spark spark-streaming_${scala.binary.version} @@ -50,7 +55,7 @@ org.apache.flume flume-ng-sdk - 1.4.0 + ${flume.version} io.netty @@ -82,11 +87,6 @@ junit-interface test - - org.apache.spark - spark-streaming-flume-sink_2.10 - ${project.version} - target/scala-${scala.binary.version}/classes diff --git a/pom.xml b/pom.xml index 9cbf3ea5995c3..556b9da3d6d90 100644 --- a/pom.xml +++ b/pom.xml @@ -125,6 +125,7 @@ 2.4.1 ${hadoop.version} 0.94.6 + 1.4.0 3.4.5 0.12.0 1.4.3 From cc40a709c0494b68404a90769efc7f2b415eb125 Mon Sep 17 00:00:00 2001 From: "Allan Douglas R. de Oliveira" Date: Mon, 25 Aug 2014 13:55:04 -0700 Subject: [PATCH 0465/1492] SPARK-3180 - Better control of security groups Adds the --authorized-address and --additional-security-group options as explained in the issue. Author: Allan Douglas R. de Oliveira Closes #2088 from douglaz/configurable_sg and squashes the following commits: e3e48ca [Allan Douglas R. de Oliveira] Adds the option to specify the address authorized to access the SG and another option to provide an additional existing SG --- ec2/spark_ec2.py | 50 +++++++++++++++++++++++++++++++----------------- 1 file changed, 32 insertions(+), 18 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 3a8c816cfffa1..77a246fffe6a7 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -141,6 +141,12 @@ def parse_args(): parser.add_option( "--security-group-prefix", type="string", default=None, help="Use this prefix for the security group rather than the cluster name.") + parser.add_option( + "--authorized-address", type="string", default="0.0.0.0/0", + help="Address to authorize on created security groups (default: 0.0.0.0/0)") + parser.add_option( + "--additional-security-group", type="string", default="", + help="Additional security group to place the machines in") (opts, args) = parser.parse_args() if len(args) != 2: @@ -293,28 +299,29 @@ def launch_cluster(conn, opts, cluster_name): else: master_group = get_or_make_group(conn, opts.security_group_prefix + "-master") slave_group = get_or_make_group(conn, opts.security_group_prefix + "-slaves") + authorized_address = opts.authorized_address if master_group.rules == []: # Group was just now created master_group.authorize(src_group=master_group) master_group.authorize(src_group=slave_group) - master_group.authorize('tcp', 22, 22, '0.0.0.0/0') - master_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') - master_group.authorize('tcp', 18080, 18080, '0.0.0.0/0') - master_group.authorize('tcp', 19999, 19999, '0.0.0.0/0') - master_group.authorize('tcp', 50030, 50030, '0.0.0.0/0') - master_group.authorize('tcp', 50070, 50070, '0.0.0.0/0') - master_group.authorize('tcp', 60070, 60070, '0.0.0.0/0') - master_group.authorize('tcp', 4040, 4045, '0.0.0.0/0') + master_group.authorize('tcp', 22, 22, authorized_address) + master_group.authorize('tcp', 8080, 8081, authorized_address) + master_group.authorize('tcp', 18080, 18080, authorized_address) + master_group.authorize('tcp', 19999, 19999, authorized_address) + master_group.authorize('tcp', 50030, 50030, authorized_address) + master_group.authorize('tcp', 50070, 50070, authorized_address) + master_group.authorize('tcp', 60070, 60070, authorized_address) + master_group.authorize('tcp', 4040, 4045, authorized_address) if opts.ganglia: - master_group.authorize('tcp', 5080, 5080, '0.0.0.0/0') + master_group.authorize('tcp', 5080, 5080, authorized_address) if slave_group.rules == []: # Group was just now created slave_group.authorize(src_group=master_group) slave_group.authorize(src_group=slave_group) - slave_group.authorize('tcp', 22, 22, '0.0.0.0/0') - slave_group.authorize('tcp', 8080, 8081, '0.0.0.0/0') - slave_group.authorize('tcp', 50060, 50060, '0.0.0.0/0') - slave_group.authorize('tcp', 50075, 50075, '0.0.0.0/0') - slave_group.authorize('tcp', 60060, 60060, '0.0.0.0/0') - slave_group.authorize('tcp', 60075, 60075, '0.0.0.0/0') + slave_group.authorize('tcp', 22, 22, authorized_address) + slave_group.authorize('tcp', 8080, 8081, authorized_address) + slave_group.authorize('tcp', 50060, 50060, authorized_address) + slave_group.authorize('tcp', 50075, 50075, authorized_address) + slave_group.authorize('tcp', 60060, 60060, authorized_address) + slave_group.authorize('tcp', 60075, 60075, authorized_address) # Check if instances are already running with the cluster name existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, @@ -326,6 +333,13 @@ def launch_cluster(conn, opts, cluster_name): # Figure out Spark AMI if opts.ami is None: opts.ami = get_spark_ami(opts) + + + additional_groups = [] + if opts.additional_security_group: + additional_groups = [sg + for sg in conn.get_all_security_groups() + if opts.additional_security_group in (sg.name, sg.id)] print "Launching instances..." try: @@ -360,7 +374,7 @@ def launch_cluster(conn, opts, cluster_name): placement=zone, count=num_slaves_this_zone, key_name=opts.key_pair, - security_groups=[slave_group], + security_groups=[slave_group] + additional_groups, instance_type=opts.instance_type, block_device_map=block_map, user_data=user_data_content) @@ -413,7 +427,7 @@ def launch_cluster(conn, opts, cluster_name): num_slaves_this_zone = get_partition(opts.slaves, num_zones, i) if num_slaves_this_zone > 0: slave_res = image.run(key_name=opts.key_pair, - security_groups=[slave_group], + security_groups=[slave_group] + additional_groups, instance_type=opts.instance_type, placement=zone, min_count=num_slaves_this_zone, @@ -439,7 +453,7 @@ def launch_cluster(conn, opts, cluster_name): if opts.zone == 'all': opts.zone = random.choice(conn.get_all_zones()).name master_res = image.run(key_name=opts.key_pair, - security_groups=[master_group], + security_groups=[master_group] + additional_groups, instance_type=master_type, placement=opts.zone, min_count=1, From fd8ace2d9a796f69ce34ad202907008cd6e4d274 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 25 Aug 2014 14:55:20 -0700 Subject: [PATCH 0466/1492] [FIX] fix error message in sendMessageReliably rxin Author: Xiangrui Meng Closes #2120 from mengxr/sendMessageReliably and squashes the following commits: b14400c [Xiangrui Meng] fix error message in sendMessageReliably --- .../scala/org/apache/spark/network/ConnectionManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index e5e1e72cd912b..578d806263006 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -851,8 +851,8 @@ private[spark] class ConnectionManager( messageStatuses.synchronized { messageStatuses.remove(message.id).foreach ( s => { promise.failure( - new IOException(s"sendMessageReliably failed because ack " + - "was not received within ${ackTimeout} sec")) + new IOException("sendMessageReliably failed because ack " + + s"was not received within $ackTimeout sec")) }) } } From 805fec845b7aa8b4763e3e0e34bec6c3872469f4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 25 Aug 2014 14:56:51 -0700 Subject: [PATCH 0467/1492] Fixed a typo in docs/running-on-mesos.md It should be `spark-env.sh` rather than `spark.env.sh`. Author: Cheng Lian Closes #2119 from liancheng/fix-mesos-doc and squashes the following commits: f360548 [Cheng Lian] Fixed a typo in docs/running-on-mesos.md --- docs/running-on-mesos.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index bd046cfc1837d..9998dddc652a6 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -107,7 +107,7 @@ cluster, or `mesos://zk://host:2181` for a multi-master Mesos cluster using ZooK The driver also needs some configuration in `spark-env.sh` to interact properly with Mesos: -1. In `spark.env.sh` set some environment variables: +1. In `spark-env.sh` set some environment variables: * `export MESOS_NATIVE_LIBRARY=`. This path is typically `/lib/libmesos.so` where the prefix is `/usr/local` by default. See Mesos installation instructions above. On Mac OS X, the library is called `libmesos.dylib` instead of From d299e2bf2f6733a6267b7ce85e2b288608b17db3 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 25 Aug 2014 16:27:00 -0700 Subject: [PATCH 0468/1492] [SPARK-3204][SQL] MaxOf would be foldable if both left and right are foldable. Author: Takuya UESHIN Closes #2116 from ueshin/issues/SPARK-3204 and squashes the following commits: 7d9b107 [Takuya UESHIN] Make MaxOf foldable if both left and right are foldable. --- .../org/apache/spark/sql/catalyst/expressions/arithmetic.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 8d90614e4501a..5f8b6ae10f0c4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -89,6 +89,8 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet case class MaxOf(left: Expression, right: Expression) extends Expression { type EvaluatedType = Any + override def foldable = left.foldable && right.foldable + override def nullable = left.nullable && right.nullable override def children = left :: right :: Nil From cae9414d3805c6cf00eab6a6144d8f90cd0212f8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 25 Aug 2014 16:29:59 -0700 Subject: [PATCH 0469/1492] [SPARK-2929][SQL] Refactored Thrift server and CLI suites Removed most hard coded timeout, timing assumptions and all `Thread.sleep`. Simplified IPC and synchronization with `scala.sys.process` and future/promise so that the test suites can run more robustly and faster. Author: Cheng Lian Closes #1856 from liancheng/thriftserver-tests and squashes the following commits: 2d914ca [Cheng Lian] Minor refactoring 0e12e71 [Cheng Lian] Cleaned up test output 0ee921d [Cheng Lian] Refactored Thrift server and CLI suites --- .../sql/hive/thriftserver/CliSuite.scala | 121 +++++++--- .../thriftserver/HiveThriftServer2Suite.scala | 212 ++++++++++-------- .../sql/hive/thriftserver/TestUtils.scala | 108 --------- 3 files changed, 217 insertions(+), 224 deletions(-) delete mode 100644 sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 70bea1ed80fda..3475c2c9db080 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -18,41 +18,112 @@ package org.apache.spark.sql.hive.thriftserver -import java.io.{BufferedReader, InputStreamReader, PrintWriter} +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ +import scala.concurrent.{Await, Future, Promise} +import scala.sys.process.{Process, ProcessLogger} + +import java.io._ +import java.util.concurrent.atomic.AtomicInteger import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} -class CliSuite extends FunSuite with BeforeAndAfterAll with TestUtils { - val WAREHOUSE_PATH = TestUtils.getWarehousePath("cli") - val METASTORE_PATH = TestUtils.getMetastorePath("cli") +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.util.getTempFilePath + +class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { + def runCliWithin( + timeout: FiniteDuration, + extraArgs: Seq[String] = Seq.empty)( + queriesAndExpectedAnswers: (String, String)*) { + + val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip + val warehousePath = getTempFilePath("warehouse") + val metastorePath = getTempFilePath("metastore") + val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) - override def beforeAll() { - val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" - val commands = - s"""../../bin/spark-sql + val command = { + val jdbcUrl = s"jdbc:derby:;databaseName=$metastorePath;create=true" + s"""$cliScript | --master local | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$WAREHOUSE_PATH - """.stripMargin.split("\\s+") - - val pb = new ProcessBuilder(commands: _*) - process = pb.start() - outputWriter = new PrintWriter(process.getOutputStream, true) - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "spark-sql>") + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + """.stripMargin.split("\\s+").toSeq ++ extraArgs + } + + // AtomicInteger is needed because stderr and stdout of the forked process are handled in + // different threads. + val next = new AtomicInteger(0) + val foundAllExpectedAnswers = Promise.apply[Unit]() + val queryStream = new ByteArrayInputStream(queries.mkString("\n").getBytes) + val buffer = new ArrayBuffer[String]() + + def captureOutput(source: String)(line: String) { + buffer += s"$source> $line" + if (line.contains(expectedAnswers(next.get()))) { + if (next.incrementAndGet() == expectedAnswers.size) { + foundAllExpectedAnswers.trySuccess(()) + } + } + } + + // Searching expected output line from both stdout and stderr of the CLI process + val process = (Process(command) #< queryStream).run( + ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) + + Future { + val exitValue = process.exitValue() + logInfo(s"Spark SQL CLI process exit value: $exitValue") + } + + try { + Await.result(foundAllExpectedAnswers.future, timeout) + } catch { case cause: Throwable => + logError( + s""" + |======================= + |CliSuite failure output + |======================= + |Spark SQL CLI command line: ${command.mkString(" ")} + | + |Executed query ${next.get()} "${queries(next.get())}", + |But failed to capture expected output "${expectedAnswers(next.get())}" within $timeout. + | + |${buffer.mkString("\n")} + |=========================== + |End CliSuite failure output + |=========================== + """.stripMargin, cause) + } finally { + warehousePath.delete() + metastorePath.delete() + process.destroy() + } } - override def afterAll() { - process.destroy() - process.waitFor() + test("Simple commands") { + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + runCliWithin(1.minute)( + "CREATE TABLE hive_test(key INT, val STRING);" + -> "OK", + "SHOW TABLES;" + -> "hive_test", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE hive_test;" + -> "OK", + "CACHE TABLE hive_test;" + -> "Time taken: ", + "SELECT COUNT(*) FROM hive_test;" + -> "5", + "DROP TABLE hive_test" + -> "Time taken: " + ) } - test("simple commands") { - val dataFilePath = getDataFile("data/files/small_kv.txt") - executeQuery("create table hive_test1(key int, val string);") - executeQuery("load data local inpath '" + dataFilePath+ "' overwrite into table hive_test1;") - executeQuery("cache table hive_test1", "Time taken") + test("Single command with -e") { + runCliWithin(1.minute, Seq("-e", "SHOW TABLES;"))("" -> "OK") } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 326b0a7275b34..38977ff162097 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -17,32 +17,32 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent._ +import scala.concurrent.duration._ +import scala.concurrent.{Await, Future, Promise} +import scala.sys.process.{Process, ProcessLogger} -import java.io.{BufferedReader, InputStreamReader} +import java.io.File import java.net.ServerSocket -import java.sql.{Connection, DriverManager, Statement} +import java.sql.{DriverManager, Statement} +import java.util.concurrent.TimeoutException import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.apache.hive.jdbc.HiveDriver +import org.scalatest.FunSuite import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath /** - * Test for the HiveThriftServer2 using JDBC. + * Tests for the HiveThriftServer2 using JDBC. */ -class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUtils with Logging { +class HiveThriftServer2Suite extends FunSuite with Logging { + Class.forName(classOf[HiveDriver].getCanonicalName) - val WAREHOUSE_PATH = getTempFilePath("warehouse") - val METASTORE_PATH = getTempFilePath("metastore") - - val DRIVER_NAME = "org.apache.hive.jdbc.HiveDriver" - val TABLE = "test" - val HOST = "localhost" - val PORT = { + private val listeningHost = "localhost" + private val listeningPort = { // Let the system to choose a random available port to avoid collision with other parallel // builds. val socket = new ServerSocket(0) @@ -51,96 +51,126 @@ class HiveThriftServer2Suite extends FunSuite with BeforeAndAfterAll with TestUt port } - Class.forName(DRIVER_NAME) - - override def beforeAll() { launchServer() } + private val warehousePath = getTempFilePath("warehouse") + private val metastorePath = getTempFilePath("metastore") + private val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" - override def afterAll() { stopServer() } + def startThriftServerWithin(timeout: FiniteDuration = 30.seconds)(f: Statement => Unit) { + val serverScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) - private def launchServer(args: Seq[String] = Seq.empty) { - // Forking a new process to start the Hive Thrift server. The reason to do this is it is - // hard to clean up Hive resources entirely, so we just start a new process and kill - // that process for cleanup. - val jdbcUrl = s"jdbc:derby:;databaseName=$METASTORE_PATH;create=true" val command = - s"""../../sbin/start-thriftserver.sh + s"""$serverScript | --master local - | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$jdbcUrl - | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$METASTORE_PATH - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$HOST - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$PORT - """.stripMargin.split("\\s+") - - val pb = new ProcessBuilder(command ++ args: _*) - val environment = pb.environment() - process = pb.start() - inputReader = new BufferedReader(new InputStreamReader(process.getInputStream)) - errorReader = new BufferedReader(new InputStreamReader(process.getErrorStream)) - waitForOutput(inputReader, "ThriftBinaryCLIService listening on", 300000) - - // Spawn a thread to read the output from the forked process. - // Note that this is necessary since in some configurations, log4j could be blocked - // if its output to stderr are not read, and eventually blocking the entire test suite. - future { - while (true) { - val stdout = readFrom(inputReader) - val stderr = readFrom(errorReader) - print(stdout) - print(stderr) - Thread.sleep(50) + | --hiveconf hive.root.logger=INFO,console + | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri + | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$listeningHost + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$listeningPort + """.stripMargin.split("\\s+").toSeq + + val serverStarted = Promise[Unit]() + val buffer = new ArrayBuffer[String]() + + def captureOutput(source: String)(line: String) { + buffer += s"$source> $line" + if (line.contains("ThriftBinaryCLIService listening on")) { + serverStarted.success(()) } } - } - private def stopServer() { - process.destroy() - process.waitFor() + val process = Process(command).run( + ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) + + Future { + val exitValue = process.exitValue() + logInfo(s"Spark SQL Thrift server process exit value: $exitValue") + } + + val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/" + val user = System.getProperty("user.name") + + try { + Await.result(serverStarted.future, timeout) + + val connection = DriverManager.getConnection(jdbcUri, user, "") + val statement = connection.createStatement() + + try { + f(statement) + } finally { + statement.close() + connection.close() + } + } catch { + case cause: Exception => + cause match { + case _: TimeoutException => + logError(s"Failed to start Hive Thrift server within $timeout", cause) + case _ => + } + logError( + s""" + |===================================== + |HiveThriftServer2Suite failure output + |===================================== + |HiveThriftServer2 command line: ${command.mkString(" ")} + |JDBC URI: $jdbcUri + |User: $user + | + |${buffer.mkString("\n")} + |========================================= + |End HiveThriftServer2Suite failure output + |========================================= + """.stripMargin, cause) + } finally { + warehousePath.delete() + metastorePath.delete() + process.destroy() + } } - test("test query execution against a Hive Thrift server") { - Thread.sleep(5 * 1000) - val dataFilePath = getDataFile("data/files/small_kv.txt") - val stmt = createStatement() - stmt.execute("DROP TABLE IF EXISTS test") - stmt.execute("DROP TABLE IF EXISTS test_cached") - stmt.execute("CREATE TABLE test(key INT, val STRING)") - stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test") - stmt.execute("CREATE TABLE test_cached AS SELECT * FROM test LIMIT 4") - stmt.execute("CACHE TABLE test_cached") - - var rs = stmt.executeQuery("SELECT COUNT(*) FROM test") - rs.next() - assert(rs.getInt(1) === 5) - - rs = stmt.executeQuery("SELECT COUNT(*) FROM test_cached") - rs.next() - assert(rs.getInt(1) === 4) - - stmt.close() + test("Test JDBC query execution") { + startThriftServerWithin() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + val queries = Seq( + "CREATE TABLE test(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test", + "CACHE TABLE test") + + queries.foreach(statement.execute) + + assertResult(5, "Row count mismatch") { + val resultSet = statement.executeQuery("SELECT COUNT(*) FROM test") + resultSet.next() + resultSet.getInt(1) + } + } } test("SPARK-3004 regression: result set containing NULL") { - Thread.sleep(5 * 1000) - val dataFilePath = getDataFile("data/files/small_kv_with_null.txt") - val stmt = createStatement() - stmt.execute("DROP TABLE IF EXISTS test_null") - stmt.execute("CREATE TABLE test_null(key INT, val STRING)") - stmt.execute(s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") - - val rs = stmt.executeQuery("SELECT * FROM test_null WHERE key IS NULL") - var count = 0 - while (rs.next()) { - count += 1 - } - assert(count === 5) + startThriftServerWithin() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource( + "data/files/small_kv_with_null.txt") - stmt.close() - } + val queries = Seq( + "DROP TABLE IF EXISTS test_null", + "CREATE TABLE test_null(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_null") - def getConnection: Connection = { - val connectURI = s"jdbc:hive2://localhost:$PORT/" - DriverManager.getConnection(connectURI, System.getProperty("user.name"), "") - } + queries.foreach(statement.execute) + + val resultSet = statement.executeQuery("SELECT * FROM test_null WHERE key IS NULL") + + (0 until 5).foreach { _ => + resultSet.next() + assert(resultSet.getInt(1) === 0) + assert(resultSet.wasNull()) + } - def createStatement(): Statement = getConnection.createStatement() + assert(!resultSet.next()) + } + } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala deleted file mode 100644 index bb2242618fbef..0000000000000 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/TestUtils.scala +++ /dev/null @@ -1,108 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.thriftserver - -import java.io.{BufferedReader, PrintWriter} -import java.text.SimpleDateFormat -import java.util.Date - -import org.apache.hadoop.hive.common.LogUtils -import org.apache.hadoop.hive.common.LogUtils.LogInitializationException - -object TestUtils { - val timestamp = new SimpleDateFormat("yyyyMMdd-HHmmss") - - def getWarehousePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-warehouse-" + - timestamp.format(new Date) - } - - def getMetastorePath(prefix: String): String = { - System.getProperty("user.dir") + "/test_warehouses/" + prefix + "-metastore-" + - timestamp.format(new Date) - } - - // Dummy function for initialize the log4j properties. - def init() { } - - // initialize log4j - try { - LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => // Ignore the error. - } -} - -trait TestUtils { - var process : Process = null - var outputWriter : PrintWriter = null - var inputReader : BufferedReader = null - var errorReader : BufferedReader = null - - def executeQuery( - cmd: String, outputMessage: String = "OK", timeout: Long = 15000): String = { - println("Executing: " + cmd + ", expecting output: " + outputMessage) - outputWriter.write(cmd + "\n") - outputWriter.flush() - waitForQuery(timeout, outputMessage) - } - - protected def waitForQuery(timeout: Long, message: String): String = { - if (waitForOutput(errorReader, message, timeout)) { - Thread.sleep(500) - readOutput() - } else { - assert(false, "Didn't find \"" + message + "\" in the output:\n" + readOutput()) - null - } - } - - // Wait for the specified str to appear in the output. - protected def waitForOutput( - reader: BufferedReader, str: String, timeout: Long = 10000): Boolean = { - val startTime = System.currentTimeMillis - var out = "" - while (!out.contains(str) && System.currentTimeMillis < (startTime + timeout)) { - out += readFrom(reader) - } - out.contains(str) - } - - // Read stdout output and filter out garbage collection messages. - protected def readOutput(): String = { - val output = readFrom(inputReader) - // Remove GC Messages - val filteredOutput = output.lines.filterNot(x => x.contains("[GC") || x.contains("[Full GC")) - .mkString("\n") - filteredOutput - } - - protected def readFrom(reader: BufferedReader): String = { - var out = "" - var c = 0 - while (reader.ready) { - c = reader.read() - out += c.asInstanceOf[Char] - } - out - } - - protected def getDataFile(name: String) = { - Thread.currentThread().getContextClassLoader.getResource(name) - } -} From 156eb3966176de02ec3ec90ae10e50a7ebfbbf4f Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 25 Aug 2014 17:43:56 -0700 Subject: [PATCH 0470/1492] [SPARK-3058] [SQL] Support EXTENDED for EXPLAIN Provide `extended` keyword support for `explain` command in SQL. e.g. ``` explain extended select key as a1, value as a2 from src where key=1; == Parsed Logical Plan == Project ['key AS a1#3,'value AS a2#4] Filter ('key = 1) UnresolvedRelation None, src, None == Analyzed Logical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = CAST(1, DoubleType)) MetastoreRelation default, src, None == Optimized Logical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = 1.0) MetastoreRelation default, src, None == Physical Plan == Project [key#8 AS a1#3,value#9 AS a2#4] Filter (CAST(key#8, DoubleType) = 1.0) HiveTableScan [key#8,value#9], (MetastoreRelation default, src, None), None Code Generation: false == RDD == (2) MappedRDD[14] at map at HiveContext.scala:350 MapPartitionsRDD[13] at mapPartitions at basicOperators.scala:42 MapPartitionsRDD[12] at mapPartitions at basicOperators.scala:57 MapPartitionsRDD[11] at mapPartitions at TableReader.scala:112 MappedRDD[10] at map at TableReader.scala:240 HadoopRDD[9] at HadoopRDD at TableReader.scala:230 ``` It's the sub task of #1847. But can go without any dependency. Author: Cheng Hao Closes #1962 from chenghao-intel/explain_extended and squashes the following commits: 295db74 [Cheng Hao] Fix bug in printing the simple execution plan 48bc989 [Cheng Hao] Support EXTENDED for EXPLAIN --- .../sql/catalyst/plans/logical/commands.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 13 +++-- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../apache/spark/sql/execution/commands.scala | 10 ++-- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +- .../sql/hive/execution/HiveExplainSuite.scala | 54 +++++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 8 files changed, 78 insertions(+), 14 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 481a5a4f212b2..a01809c1fc5e2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -50,7 +50,7 @@ case class SetCommand(key: Option[String], value: Option[String]) extends Comman * Returned by a parser when the users only wants to see what query plan would be executed, without * actually performing the execution. */ -case class ExplainCommand(plan: LogicalPlan) extends Command { +case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends Command { override def output = Seq(AttributeReference("plan", StringType, nullable = false)()) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index af9f7c62a1d25..8a9f4deb6a19e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -408,10 +408,18 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } - def simpleString: String = stringOrError(executedPlan) + def simpleString: String = + s"""== Physical Plan == + |${stringOrError(executedPlan)} + """ override def toString: String = - s"""== Logical Plan == + // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)}) + // however, the `toRdd` will cause the real execution, which is not what we want. + // We need to think about how to avoid the side effect. + s"""== Parsed Logical Plan == + |${stringOrError(logical)} + |== Analyzed Logical Plan == |${stringOrError(analyzed)} |== Optimized Logical Plan == |${stringOrError(optimizedPlan)} @@ -419,7 +427,6 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(executedPlan)} |Code Generation: ${executedPlan.codegenEnabled} |== RDD == - |${stringOrError(toRdd.toDebugString)} """.stripMargin.trim } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 517b77804ae2c..8dacb84c8a17e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -301,8 +301,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.SetCommand(key, value) => Seq(execution.SetCommand(key, value, plan.output)(context)) - case logical.ExplainCommand(logicalPlan) => - Seq(execution.ExplainCommand(logicalPlan, plan.output)(context)) + case logical.ExplainCommand(logicalPlan, extended) => + Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) case logical.CacheCommand(tableName, cache) => Seq(execution.CacheCommand(tableName, cache)(context)) case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 38f37564f1788..031b695169cea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -108,15 +108,19 @@ case class SetCommand( */ @DeveloperApi case class ExplainCommand( - logicalPlan: LogicalPlan, output: Seq[Attribute])( + logicalPlan: LogicalPlan, output: Seq[Attribute], extended: Boolean)( @transient context: SQLContext) extends LeafNode with Command { // Run through the optimizer to generate the physical plan. override protected[sql] lazy val sideEffectResult: Seq[String] = try { - "Physical execution plan:" +: context.executePlan(logicalPlan).executedPlan.toString.split("\n") + // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. + val queryExecution = context.executePlan(logicalPlan) + val outputString = if (extended) queryExecution.toString else queryExecution.simpleString + + outputString.split("\n") } catch { case cause: TreeNodeException[_] => - "Error occurred during query planning: " +: cause.getMessage.split("\n") + ("Error occurred during query planning: \n" + cause.getMessage).split("\n") } def execute(): RDD[Row] = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 29baefe714c64..d9b2bc7348ad2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -424,7 +424,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { logical match { case _: NativeCommand => "" case _: SetCommand => "" - case _ => executedPlan.toString + case _ => super.simpleString } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 1d9ba1b24a7a4..5da6e8df03aee 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -409,10 +409,9 @@ private[hive] object HiveQl { ExplainCommand(NoRelation) case Token("TOK_EXPLAIN", explainArgs) => // Ignore FORMATTED if present. - val Some(query) :: _ :: _ :: Nil = + val Some(query) :: _ :: extended :: Nil = getClauses(Seq("TOK_QUERY", "FORMATTED", "EXTENDED"), explainArgs) - // TODO: support EXTENDED? - ExplainCommand(nodeToPlan(query)) + ExplainCommand(nodeToPlan(query), extended != None) case Token("TOK_DESCTABLE", describeArgs) => // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala new file mode 100644 index 0000000000000..4ed58f4be1167 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.Row + +/** + * A set of tests that validates support for Hive Explain command. + */ +class HiveExplainSuite extends QueryTest { + private def check(sqlCmd: String, exists: Boolean, keywords: String*) { + val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString + for (key <- keywords) { + if (exists) { + assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist in result)") + } else { + assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in the result)") + } + } + } + + test("explain extended command") { + check(" explain select * from src where key=123 ", true, + "== Physical Plan ==") + check(" explain select * from src where key=123 ", false, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==") + check(" explain extended select * from src where key=123 ", true, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==", + "== Physical Plan ==", + "Code Generation", "== RDD ==") + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 26e4ec6e6dcce..6d925e56e6838 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -327,7 +327,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.size > 1 && explanation.head.startsWith("Physical execution plan") + explanation.exists(_ == "== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { From 507a1b520063ad3e10b909767d9e3fd72d24415b Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 25 Aug 2014 17:46:43 -0700 Subject: [PATCH 0471/1492] [SQL] logWarning should be logInfo in getResultSetSchema Author: wangfei Closes #1939 from scwf/patch-5 and squashes the following commits: f952d10 [wangfei] [SQL] logWarning should be logInfo in getResultSetSchema --- .../sql/hive/thriftserver/server/SparkSQLOperationManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 699a1103f3248..6eccb1ba6d4dc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -151,7 +151,7 @@ class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManage } def getResultSetSchema: TableSchema = { - logWarning(s"Result Schema: ${result.queryExecution.analyzed.output}") + logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") if (result.queryExecution.analyzed.output.size == 0) { new TableSchema(new FieldSchema("Result", "string", "") :: Nil) } else { From 4243bb6634aca5b9ddf6d42778aa7b4866ce6256 Mon Sep 17 00:00:00 2001 From: Chia-Yung Su Date: Mon, 25 Aug 2014 18:20:19 -0700 Subject: [PATCH 0472/1492] [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile fix compile error on hadoop 0.23 for the pull request #1924. Author: Chia-Yung Su Closes #1959 from joesu/bugfix-spark3011 and squashes the following commits: be30793 [Chia-Yung Su] remove .* and _* except _metadata 8fe2398 [Chia-Yung Su] add note to explain 40ea9bd [Chia-Yung Su] fix hadoop-0.23 compile error c7e44f2 [Chia-Yung Su] match syntax f8fc32a [Chia-Yung Su] filter out tmp dir --- .../main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index c79a9ac2dad81..af8cd0a73b674 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -378,7 +378,7 @@ private[parquet] object ParquetTypesConverter extends Logging { val children = fs.listStatus(path).filterNot { status => val name = status.getPath.getName - name(0) == '.' || name == FileOutputCommitter.SUCCEEDED_FILE_NAME + (name(0) == '.' || name(0) == '_') && name != ParquetFileWriter.PARQUET_METADATA_FILE } // NOTE (lian): Parquet "_metadata" file can be very slow if the file consists of lots of row From 9f04db17e50568d5580091add9100693177d7c4f Mon Sep 17 00:00:00 2001 From: witgo Date: Mon, 25 Aug 2014 19:22:27 -0700 Subject: [PATCH 0473/1492] SPARK-2481: The environment variables SPARK_HISTORY_OPTS is covered in spark-env.sh Author: witgo Author: GuoQiang Li Closes #1341 from witgo/history_env and squashes the following commits: b4fd9f8 [GuoQiang Li] review commit 0ebe401 [witgo] *-history-server.sh load spark-config.sh --- sbin/start-history-server.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index e30493da32a7a..580ab471b8a79 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -27,6 +27,9 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` +. "$sbin/spark-config.sh" +. "$SPARK_PREFIX/bin/load-spark-env.sh" + if [ $# != 0 ]; then echo "Using command line arguments for setting the log directory is deprecated. Please " echo "set the spark.history.fs.logDirectory configuration option instead." From 62f5009f6795b17638d2a1e8e51db0890030d8d6 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 25 Aug 2014 19:40:08 -0700 Subject: [PATCH 0474/1492] [SPARK-2976] Replace tabs with spaces Author: Kousuke Saruta Closes #1895 from sarutak/SPARK-2976 and squashes the following commits: 1cf7e69 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2976 d1e0666 [Kousuke Saruta] Modified styles c5e80a4 [Kousuke Saruta] Remove tab from JavaPageRank.java and JavaKinesisWordCountASL.java c003b36 [Kousuke Saruta] Removed tab from sorttable.js --- .../org/apache/spark/ui/static/sorttable.js | 275 +++++++++--------- .../apache/spark/examples/JavaPageRank.java | 2 +- .../streaming/JavaKinesisWordCountASL.java | 8 +- 3 files changed, 142 insertions(+), 143 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js index 7abb9011ccf36..dbacbf19beee5 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js +++ b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js @@ -81,15 +81,15 @@ sorttable = { if (!headrow[i].className.match(/\bsorttable_nosort\b/)) { // skip this col mtch = headrow[i].className.match(/\bsorttable_([a-z0-9]+)\b/); if (mtch) { override = mtch[1]; } - if (mtch && typeof sorttable["sort_"+override] == 'function') { - headrow[i].sorttable_sortfunction = sorttable["sort_"+override]; - } else { - headrow[i].sorttable_sortfunction = sorttable.guessType(table,i); - } - // make it clickable to sort - headrow[i].sorttable_columnindex = i; - headrow[i].sorttable_tbody = table.tBodies[0]; - dean_addEvent(headrow[i],"click", function(e) { + if (mtch && typeof sorttable["sort_"+override] == 'function') { + headrow[i].sorttable_sortfunction = sorttable["sort_"+override]; + } else { + headrow[i].sorttable_sortfunction = sorttable.guessType(table,i); + } + // make it clickable to sort + headrow[i].sorttable_columnindex = i; + headrow[i].sorttable_tbody = table.tBodies[0]; + dean_addEvent(headrow[i],"click", function(e) { if (this.className.search(/\bsorttable_sorted\b/) != -1) { // if we're already sorted by this column, just @@ -109,7 +109,7 @@ sorttable = { // re-reverse the table, which is quicker sorttable.reverse(this.sorttable_tbody); this.className = this.className.replace('sorttable_sorted_reverse', - 'sorttable_sorted'); + 'sorttable_sorted'); this.removeChild(document.getElementById('sorttable_sortrevind')); sortfwdind = document.createElement('span'); sortfwdind.id = "sorttable_sortfwdind"; @@ -117,7 +117,7 @@ sorttable = { this.appendChild(sortfwdind); return; } - + // remove sorttable_sorted classes theadrow = this.parentNode; forEach(theadrow.childNodes, function(cell) { @@ -130,36 +130,36 @@ sorttable = { if (sortfwdind) { sortfwdind.parentNode.removeChild(sortfwdind); } sortrevind = document.getElementById('sorttable_sortrevind'); if (sortrevind) { sortrevind.parentNode.removeChild(sortrevind); } - + this.className += ' sorttable_sorted'; sortfwdind = document.createElement('span'); sortfwdind.id = "sorttable_sortfwdind"; sortfwdind.innerHTML = stIsIE ? ' 6' : ' ▾'; this.appendChild(sortfwdind); - // build an array to sort. This is a Schwartzian transform thing, - // i.e., we "decorate" each row with the actual sort key, - // sort based on the sort keys, and then put the rows back in order - // which is a lot faster because you only do getInnerText once per row - row_array = []; - col = this.sorttable_columnindex; - rows = this.sorttable_tbody.rows; - for (var j=0; j 0 ) { - var q = list[i]; list[i] = list[i+1]; list[i+1] = q; - swap = true; - } - } // for - t--; + swap = false; + for(var i = b; i < t; ++i) { + if ( comp_func(list[i], list[i+1]) > 0 ) { + var q = list[i]; list[i] = list[i+1]; list[i+1] = q; + swap = true; + } + } // for + t--; - if (!swap) break; - - for(var i = t; i > b; --i) { - if ( comp_func(list[i], list[i-1]) < 0 ) { - var q = list[i]; list[i] = list[i-1]; list[i-1] = q; - swap = true; - } - } // for - b++; + if (!swap) break; + for(var i = t; i > b; --i) { + if ( comp_func(list[i], list[i-1]) < 0 ) { + var q = list[i]; list[i] = list[i-1]; list[i-1] = q; + swap = true; + } + } // for + b++; } // while(swap) } } @@ -358,11 +357,11 @@ if (document.addEventListener) { /* for Safari */ if (/WebKit/i.test(navigator.userAgent)) { // sniff - var _timer = setInterval(function() { - if (/loaded|complete/.test(document.readyState)) { - sorttable.init(); // call the onload handler - } - }, 10); + var _timer = setInterval(function() { + if (/loaded|complete/.test(document.readyState)) { + sorttable.init(); // call the onload handler + } + }, 10); } /* for other browsers */ @@ -374,66 +373,66 @@ window.onload = sorttable.init; // http://dean.edwards.name/weblog/2005/10/add-event/ function dean_addEvent(element, type, handler) { - if (element.addEventListener) { - element.addEventListener(type, handler, false); - } else { - // assign each event handler a unique ID - if (!handler.$$guid) handler.$$guid = dean_addEvent.guid++; - // create a hash table of event types for the element - if (!element.events) element.events = {}; - // create a hash table of event handlers for each element/event pair - var handlers = element.events[type]; - if (!handlers) { - handlers = element.events[type] = {}; - // store the existing event handler (if there is one) - if (element["on" + type]) { - handlers[0] = element["on" + type]; - } - } - // store the event handler in the hash table - handlers[handler.$$guid] = handler; - // assign a global event handler to do all the work - element["on" + type] = handleEvent; - } + if (element.addEventListener) { + element.addEventListener(type, handler, false); + } else { + // assign each event handler a unique ID + if (!handler.$$guid) handler.$$guid = dean_addEvent.guid++; + // create a hash table of event types for the element + if (!element.events) element.events = {}; + // create a hash table of event handlers for each element/event pair + var handlers = element.events[type]; + if (!handlers) { + handlers = element.events[type] = {}; + // store the existing event handler (if there is one) + if (element["on" + type]) { + handlers[0] = element["on" + type]; + } + } + // store the event handler in the hash table + handlers[handler.$$guid] = handler; + // assign a global event handler to do all the work + element["on" + type] = handleEvent; + } }; // a counter used to create unique IDs dean_addEvent.guid = 1; function removeEvent(element, type, handler) { - if (element.removeEventListener) { - element.removeEventListener(type, handler, false); - } else { - // delete the event handler from the hash table - if (element.events && element.events[type]) { - delete element.events[type][handler.$$guid]; - } - } + if (element.removeEventListener) { + element.removeEventListener(type, handler, false); + } else { + // delete the event handler from the hash table + if (element.events && element.events[type]) { + delete element.events[type][handler.$$guid]; + } + } }; function handleEvent(event) { - var returnValue = true; - // grab the event object (IE uses a global event object) - event = event || fixEvent(((this.ownerDocument || this.document || this).parentWindow || window).event); - // get a reference to the hash table of event handlers - var handlers = this.events[event.type]; - // execute each event handler - for (var i in handlers) { - this.$$handleEvent = handlers[i]; - if (this.$$handleEvent(event) === false) { - returnValue = false; - } - } - return returnValue; + var returnValue = true; + // grab the event object (IE uses a global event object) + event = event || fixEvent(((this.ownerDocument || this.document || this).parentWindow || window).event); + // get a reference to the hash table of event handlers + var handlers = this.events[event.type]; + // execute each event handler + for (var i in handlers) { + this.$$handleEvent = handlers[i]; + if (this.$$handleEvent(event) === false) { + returnValue = false; + } + } + return returnValue; }; function fixEvent(event) { - // add W3C standard event methods - event.preventDefault = fixEvent.preventDefault; - event.stopPropagation = fixEvent.stopPropagation; - return event; + // add W3C standard event methods + event.preventDefault = fixEvent.preventDefault; + event.stopPropagation = fixEvent.stopPropagation; + return event; }; fixEvent.preventDefault = function() { - this.returnValue = false; + this.returnValue = false; }; fixEvent.stopPropagation = function() { this.cancelBubble = true; @@ -441,55 +440,55 @@ fixEvent.stopPropagation = function() { // Dean's forEach: http://dean.edwards.name/base/forEach.js /* - forEach, version 1.0 - Copyright 2006, Dean Edwards - License: http://www.opensource.org/licenses/mit-license.php +forEach, version 1.0 +Copyright 2006, Dean Edwards +License: http://www.opensource.org/licenses/mit-license.php */ // array-like enumeration if (!Array.forEach) { // mozilla already supports this - Array.forEach = function(array, block, context) { - for (var i = 0; i < array.length; i++) { - block.call(context, array[i], i, array); - } - }; + Array.forEach = function(array, block, context) { + for (var i = 0; i < array.length; i++) { + block.call(context, array[i], i, array); + } + }; } // generic enumeration Function.prototype.forEach = function(object, block, context) { - for (var key in object) { - if (typeof this.prototype[key] == "undefined") { - block.call(context, object[key], key, object); - } - } + for (var key in object) { + if (typeof this.prototype[key] == "undefined") { + block.call(context, object[key], key, object); + } + } }; // character enumeration String.forEach = function(string, block, context) { - Array.forEach(string.split(""), function(chr, index) { - block.call(context, chr, index, string); - }); + Array.forEach(string.split(""), function(chr, index) { + block.call(context, chr, index, string); + }); }; // globally resolve forEach enumeration var forEach = function(object, block, context) { - if (object) { - var resolve = Object; // default - if (object instanceof Function) { - // functions have a "length" property - resolve = Function; - } else if (object.forEach instanceof Function) { - // the object implements a custom forEach method so use that - object.forEach(block, context); - return; - } else if (typeof object == "string") { - // the object is a string - resolve = String; - } else if (typeof object.length == "number") { - // the object is array-like - resolve = Array; - } - resolve.forEach(object, block, context); - } + if (object) { + var resolve = Object; // default + if (object instanceof Function) { + // functions have a "length" property + resolve = Function; + } else if (object.forEach instanceof Function) { + // the object implements a custom forEach method so use that + object.forEach(block, context); + return; + } else if (typeof object == "string") { + // the object is a string + resolve = String; + } else if (typeof object.length == "number") { + // the object is array-like + resolve = Array; + } + resolve.forEach(object, block, context); + } }; diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 7ea6df9c17245..c22506491fbff 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -96,7 +96,7 @@ public Double call(Iterable rs) { .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override public Iterable> call(Tuple2, Double> s) { - int urlCount = Iterables.size(s._1); + int urlCount = Iterables.size(s._1); List> results = new ArrayList>(); for (String n : s._1) { results.add(new Tuple2(n, s._2() / urlCount)); diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index a8b907b241893..1a710d7b18c6f 100644 --- a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -130,10 +130,10 @@ public static void main(String[] args) { /* Create the same number of Kinesis DStreams/Receivers as Kinesis stream's shards */ List> streamsList = new ArrayList>(numStreams); for (int i = 0; i < numStreams; i++) { - streamsList.add( - KinesisUtils.createStream(jssc, streamName, endpointUrl, checkpointInterval, - InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2()) - ); + streamsList.add( + KinesisUtils.createStream(jssc, streamName, endpointUrl, checkpointInterval, + InitialPositionInStream.LATEST, StorageLevel.MEMORY_AND_DISK_2()) + ); } /* Union all the streams if there is more than 1 stream */ From 52fbdc2deddcdba02bf5945a36e15870021ec890 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 25 Aug 2014 22:56:35 -0700 Subject: [PATCH 0475/1492] [Spark-3222] [SQL] Cross join support in HiveQL We can simple treat cross join as inner join without join conditions. Author: Daoyuan Wang Author: adrian-wang Closes #2124 from adrian-wang/crossjoin and squashes the following commits: 8c9b7c5 [Daoyuan Wang] add a test 7d47bbb [adrian-wang] add cross join support for hql --- .../apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala | 1 + sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 1 + .../golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 | 0 .../golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 | 0 .../golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 | 0 5 files changed, 2 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 create mode 100644 sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 create mode 100644 sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 210753efe7678..66243879b9019 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -310,6 +310,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_nested_type", "create_skewed_table1", "create_struct_table", + "cross_join", "ct_case_insensitive", "database_location", "database_properties", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index 5da6e8df03aee..581332e600183 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -772,6 +772,7 @@ private[hive] object HiveQl { val joinType = joinToken match { case "TOK_JOIN" => Inner + case "TOK_CROSSJOIN" => Inner case "TOK_RIGHTOUTERJOIN" => RightOuter case "TOK_LEFTOUTERJOIN" => LeftOuter case "TOK_FULLOUTERJOIN" => FullOuter diff --git a/sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 b/sql/hive/src/test/resources/golden/cross_join-0-7e4af1870bc73decae43b3383c7d2046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 b/sql/hive/src/test/resources/golden/cross_join-1-1a96761bf3e47ace9a422ed58273ff35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 b/sql/hive/src/test/resources/golden/cross_join-2-85c93a81eae05bf56a04a904bb80a229 new file mode 100644 index 0000000000000..e69de29bb2d1d From b21ae5bbb9baa966f69303a30659aa8bbb2098da Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 25 Aug 2014 23:36:09 -0700 Subject: [PATCH 0476/1492] [SPARK-2886] Use more specific actor system name than "spark" As of #1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001. ``` 14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses: [akka.tcp://sparkandrews-mbp:5001] 14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses: [akka.tcp://sparkandrews-mbp:5001] 14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001. ``` This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality. Author: Andrew Or Closes #1810 from andrewor14/service-name and squashes the following commits: 8c459ed [Andrew Or] Use a common variable for driver/executor actor system names 3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor 921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name c8c6a62 [Andrew Or] Do not include hyphens in actor name 1c1b42e [Andrew Or] Avoid spaces in akka system name f644b55 [Andrew Or] Use more specific service name --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 9 ++++++--- .../spark/scheduler/cluster/SimrSchedulerBackend.scala | 8 +++++--- .../cluster/SparkDeploySchedulerBackend.scala | 8 +++++--- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 5 +++-- .../main/scala/org/apache/spark/util/AkkaUtils.scala | 5 +++-- .../streaming/receiver/ReceiverSupervisorImpl.scala | 10 +++++----- .../apache/spark/deploy/yarn/ExecutorLauncher.scala | 9 ++++++--- .../spark/deploy/yarn/YarnAllocationHandler.scala | 8 +++++--- .../apache/spark/deploy/yarn/ExecutorLauncher.scala | 9 ++++++--- .../spark/deploy/yarn/YarnAllocationHandler.scala | 5 +++-- 10 files changed, 47 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index fc36e37c53f5e..72716567ca99b 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -111,6 +111,9 @@ object SparkEnv extends Logging { private val env = new ThreadLocal[SparkEnv] @volatile private var lastSetSparkEnv : SparkEnv = _ + private[spark] val driverActorSystemName = "sparkDriver" + private[spark] val executorActorSystemName = "sparkExecutor" + def set(e: SparkEnv) { lastSetSparkEnv = e env.set(e) @@ -146,9 +149,9 @@ object SparkEnv extends Logging { } val securityManager = new SecurityManager(conf) - - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, - securityManager = securityManager) + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + actorSystemName, hostname, port, conf, securityManager) // Figure out which port Akka actually bound to in case the original port is 0 or occupied. // This is so that we tell the executors the correct port to connect to. diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index d99c76117c168..4f7133c4bc17c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.scheduler.TaskSchedulerImpl private[spark] class SimrSchedulerBackend( @@ -38,8 +38,10 @@ private[spark] class SimrSchedulerBackend( override def start() { super.start() - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - sc.conf.get("spark.driver.host"), sc.conf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sc.conf.get("spark.driver.host"), + sc.conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val conf = new Configuration() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 589dba2e40d20..32138e5246700 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler.cluster -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} @@ -42,8 +42,10 @@ private[spark] class SparkDeploySchedulerBackend( super.start() // The endpoint for executors to talk to us - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + conf.get("spark.driver.host"), + conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 9f45400bcf852..f0172504c55aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -28,7 +28,7 @@ import org.apache.mesos.{Scheduler => MScheduler} import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} -import org.apache.spark.{Logging, SparkContext, SparkException} +import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -130,7 +130,8 @@ private[spark] class CoarseMesosSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index d6afb73b74242..e2d32c859bbda 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -27,7 +27,7 @@ import akka.pattern.ask import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} -import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} /** * Various utility classes for working with Akka. @@ -192,10 +192,11 @@ private[spark] object AkkaUtils extends Logging { } def makeDriverRef(name: String, conf: SparkConf, actorSystem: ActorSystem): ActorRef = { + val driverActorSystemName = SparkEnv.driverActorSystemName val driverHost: String = conf.get("spark.driver.host", "localhost") val driverPort: Int = conf.getInt("spark.driver.port", 7077) Utils.checkHost(driverHost, "Expected hostname") - val url = s"akka.tcp://spark@$driverHost:$driverPort/user/$name" + val url = s"akka.tcp://$driverActorSystemName@$driverHost:$driverPort/user/$name" val timeout = AkkaUtils.lookupTimeout(conf) logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index d934b9cbfc3e8..53a3e6200e340 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -20,22 +20,21 @@ package org.apache.spark.streaming.receiver import java.nio.ByteBuffer import java.util.concurrent.atomic.AtomicLong -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await import akka.actor.{Actor, Props} import akka.pattern.ask +import com.google.common.base.Throwables + import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.scheduler._ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.scheduler.DeregisterReceiver import org.apache.spark.streaming.scheduler.AddBlock -import scala.Some import org.apache.spark.streaming.scheduler.RegisterReceiver -import com.google.common.base.Throwables /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -56,7 +55,8 @@ private[streaming] class ReceiverSupervisorImpl( private val trackerActor = { val ip = env.conf.get("spark.driver.host", "localhost") val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/ReceiverTracker".format(ip, port) + val url = "akka.tcp://%s@%s:%s/user/ReceiverTracker".format( + SparkEnv.driverActorSystemName, ip, port) env.actorSystem.actorSelection(url) } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index c3310fbc24a98..155dd88aa2b81 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter @@ -210,8 +210,11 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + driverHost, + driverPort.toString, + CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 80e0162e9f277..568a6ef932bbd 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -245,8 +245,10 @@ private[yarn] class YarnAllocationHandler( // Deallocate + allocate can result in reusing id's wrongly - so use a different counter // (executorIdCounter) val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sparkConf.get("spark.driver.host"), + sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("launching container on " + containerId + " host " + executorHostname) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 45925f1fea005..e093fe4ae6ff8 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.conf.YarnConfiguration import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter @@ -174,8 +174,11 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - driverHost, driverPort.toString, CoarseGrainedSchedulerBackend.ACTOR_NAME) + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + driverHost, + driverPort.toString, + CoarseGrainedSchedulerBackend.ACTOR_NAME) actor = actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 29ccec2adcac3..0a461749c819d 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -26,7 +26,7 @@ import scala.collection import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -262,7 +262,8 @@ private[yarn] class YarnAllocationHandler( numExecutorsRunning.decrementAndGet() } else { val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) From 8856c3d86009295be871989a5dc7270f31b420cd Mon Sep 17 00:00:00 2001 From: chutium Date: Tue, 26 Aug 2014 11:51:26 -0700 Subject: [PATCH 0477/1492] [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext There are 4 different compression codec available for ```ParquetOutputFormat``` in Spark SQL, it was set as a hard-coded value in ```ParquetRelation.defaultCompression``` original discuss: https://github.com/apache/spark/pull/195#discussion-diff-11002083 i added a new config property in SQLConf to allow user to change this compression codec, and i used similar short names syntax as described in SPARK-2953 #1873 (https://github.com/apache/spark/pull/1873/files#diff-0) btw, which codec should we use as default? it was set to GZIP (https://github.com/apache/spark/pull/195/files#diff-4), but i think maybe we should change this to SNAPPY, since SNAPPY is already the default codec for shuffling in spark-core (SPARK-2469, #1415), and parquet-mr supports Snappy codec natively (https://github.com/Parquet/parquet-mr/commit/e440108de57199c12d66801ca93804086e7f7632). Author: chutium Closes #2039 from chutium/parquet-compression and squashes the following commits: 2f44964 [chutium] [SPARK-3131][SQL] parquet compression default codec set to snappy, also in test suite e578e21 [chutium] [SPARK-3131][SQL] compression codec config property name and default codec set to snappy 21235dc [chutium] [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext --- .../scala/org/apache/spark/sql/SQLConf.scala | 4 + .../spark/sql/parquet/ParquetRelation.scala | 14 ++- .../spark/sql/parquet/ParquetQuerySuite.scala | 94 +++++++++++++++++++ 3 files changed, 107 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 5cc41a83cc792..f0df19112ae37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -33,6 +33,7 @@ private[spark] object SQLConf { val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" + val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -78,6 +79,9 @@ trait SQLConf { /** When true tables cached using the in-memory columnar caching will be compressed. */ private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "false").toBoolean + /** The compression codec for writing to a Parquetfile */ + private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "snappy") + /** The number of rows that will be */ private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 1713ae6fb5d93..5ae768293a22e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -100,8 +100,13 @@ private[sql] object ParquetRelation { // The compression type type CompressionType = parquet.hadoop.metadata.CompressionCodecName - // The default compression - val defaultCompression = CompressionCodecName.GZIP + // The parquet compression short names + val shortParquetCompressionCodecNames = Map( + "NONE" -> CompressionCodecName.UNCOMPRESSED, + "UNCOMPRESSED" -> CompressionCodecName.UNCOMPRESSED, + "SNAPPY" -> CompressionCodecName.SNAPPY, + "GZIP" -> CompressionCodecName.GZIP, + "LZO" -> CompressionCodecName.LZO) /** * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that @@ -141,9 +146,8 @@ private[sql] object ParquetRelation { conf: Configuration, sqlContext: SQLContext): ParquetRelation = { val path = checkPath(pathString, allowExisting, conf) - if (conf.get(ParquetOutputFormat.COMPRESSION) == null) { - conf.set(ParquetOutputFormat.COMPRESSION, ParquetRelation.defaultCompression.name()) - } + conf.set(ParquetOutputFormat.COMPRESSION, shortParquetCompressionCodecNames.getOrElse( + sqlContext.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED).name()) ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) new ParquetRelation(path.toString, Some(conf), sqlContext) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 172dcd6aa0ee3..28f43b36832ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -186,6 +186,100 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) } + test("Compression options for writing to a Parquetfile") { + val defaultParquetCompressionCodec = TestSQLContext.parquetCompressionCodec + import scala.collection.JavaConversions._ + + val file = getTempFilePath("parquet") + val path = file.toString + val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + + // test default compression codec + rdd.saveAsParquetFile(path) + var actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "UNCOMPRESSED" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "UNCOMPRESSED") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "none" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "none") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === "UNCOMPRESSED" :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test gzip compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "gzip") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test snappy compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "snappy") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // TODO: Lzo requires additional external setup steps so leave it out for now + // ref.: https://github.com/Parquet/parquet-mr/blob/parquet-1.5.0/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java#L169 + + // Set it back. + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, defaultParquetCompressionCodec) + } + test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) From 3cedc4f4d78e093fd362085e0a077bb9e4f28ca5 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 26 Aug 2014 13:04:30 -0700 Subject: [PATCH 0478/1492] [SPARK-2871] [PySpark] add histgram() API RDD.histogram(buckets) Compute a histogram using the provided buckets. The buckets are all open to the right except for the last which is closed. e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50], which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1 and 50 we would have a histogram of 1,0,1. If your histogram is evenly spaced (e.g. [0, 10, 20, 30]), this can be switched from an O(log n) inseration to O(1) per element(where n = # buckets). Buckets must be sorted and not contain any duplicates, must be at least two elements. If `buckets` is a number, it will generates buckets which is evenly spaced between the minimum and maximum of the RDD. For example, if the min value is 0 and the max is 100, given buckets as 2, the resulting buckets will be [0,50) [50,100]. buckets must be at least 1 If the RDD contains infinity, NaN throws an exception If the elements in RDD do not vary (max == min) always returns a single bucket. It will return an tuple of buckets and histogram. >>> rdd = sc.parallelize(range(51)) >>> rdd.histogram(2) ([0, 25, 50], [25, 26]) >>> rdd.histogram([0, 5, 25, 50]) ([0, 5, 25, 50], [5, 20, 26]) >>> rdd.histogram([0, 15, 30, 45, 60], True) ([0, 15, 30, 45, 60], [15, 15, 15, 6]) >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"]) >>> rdd.histogram(("a", "b", "c")) (('a', 'b', 'c'), [2, 2]) closes #122, it's duplicated. Author: Davies Liu Closes #2091 from davies/histgram and squashes the following commits: a322f8a [Davies Liu] fix deprecation of e.message 84e85fa [Davies Liu] remove evenBuckets, add more tests (including str) d9a0722 [Davies Liu] address comments 0e18a2d [Davies Liu] add histgram() API --- python/pyspark/rdd.py | 129 +++++++++++++++++++++++++++++++++++++++- python/pyspark/tests.py | 104 ++++++++++++++++++++++++++++++++ 2 files changed, 232 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1374f74968c9e..3a2e7649e6827 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -32,7 +32,7 @@ import heapq import bisect from random import Random -from math import sqrt, log +from math import sqrt, log, isinf, isnan from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ @@ -886,6 +886,133 @@ def redFunc(left_counter, right_counter): return self.mapPartitions(lambda i: [StatCounter(i)]).reduce(redFunc) + def histogram(self, buckets): + """ + Compute a histogram using the provided buckets. The buckets + are all open to the right except for the last which is closed. + e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50], + which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1 + and 50 we would have a histogram of 1,0,1. + + If your histogram is evenly spaced (e.g. [0, 10, 20, 30]), + this can be switched from an O(log n) inseration to O(1) per + element(where n = # buckets). + + Buckets must be sorted and not contain any duplicates, must be + at least two elements. + + If `buckets` is a number, it will generates buckets which are + evenly spaced between the minimum and maximum of the RDD. For + example, if the min value is 0 and the max is 100, given buckets + as 2, the resulting buckets will be [0,50) [50,100]. buckets must + be at least 1 If the RDD contains infinity, NaN throws an exception + If the elements in RDD do not vary (max == min) always returns + a single bucket. + + It will return an tuple of buckets and histogram. + + >>> rdd = sc.parallelize(range(51)) + >>> rdd.histogram(2) + ([0, 25, 50], [25, 26]) + >>> rdd.histogram([0, 5, 25, 50]) + ([0, 5, 25, 50], [5, 20, 26]) + >>> rdd.histogram([0, 15, 30, 45, 60]) # evenly spaced buckets + ([0, 15, 30, 45, 60], [15, 15, 15, 6]) + >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"]) + >>> rdd.histogram(("a", "b", "c")) + (('a', 'b', 'c'), [2, 2]) + """ + + if isinstance(buckets, (int, long)): + if buckets < 1: + raise ValueError("number of buckets must be >= 1") + + # filter out non-comparable elements + def comparable(x): + if x is None: + return False + if type(x) is float and isnan(x): + return False + return True + + filtered = self.filter(comparable) + + # faster than stats() + def minmax(a, b): + return min(a[0], b[0]), max(a[1], b[1]) + try: + minv, maxv = filtered.map(lambda x: (x, x)).reduce(minmax) + except TypeError as e: + if " empty " in str(e): + raise ValueError("can not generate buckets from empty RDD") + raise + + if minv == maxv or buckets == 1: + return [minv, maxv], [filtered.count()] + + try: + inc = (maxv - minv) / buckets + except TypeError: + raise TypeError("Can not generate buckets with non-number in RDD") + + if isinf(inc): + raise ValueError("Can not generate buckets with infinite value") + + # keep them as integer if possible + if inc * buckets != maxv - minv: + inc = (maxv - minv) * 1.0 / buckets + + buckets = [i * inc + minv for i in range(buckets)] + buckets.append(maxv) # fix accumulated error + even = True + + elif isinstance(buckets, (list, tuple)): + if len(buckets) < 2: + raise ValueError("buckets should have more than one value") + + if any(i is None or isinstance(i, float) and isnan(i) for i in buckets): + raise ValueError("can not have None or NaN in buckets") + + if sorted(buckets) != list(buckets): + raise ValueError("buckets should be sorted") + + if len(set(buckets)) != len(buckets): + raise ValueError("buckets should not contain duplicated values") + + minv = buckets[0] + maxv = buckets[-1] + even = False + inc = None + try: + steps = [buckets[i + 1] - buckets[i] for i in range(len(buckets) - 1)] + except TypeError: + pass # objects in buckets do not support '-' + else: + if max(steps) - min(steps) < 1e-10: # handle precision errors + even = True + inc = (maxv - minv) / (len(buckets) - 1) + + else: + raise TypeError("buckets should be a list or tuple or number(int or long)") + + def histogram(iterator): + counters = [0] * len(buckets) + for i in iterator: + if i is None or (type(i) is float and isnan(i)) or i > maxv or i < minv: + continue + t = (int((i - minv) / inc) if even + else bisect.bisect_right(buckets, i) - 1) + counters[t] += 1 + # add last two together + last = counters.pop() + counters[-1] += last + return [counters] + + def mergeCounters(a, b): + return [i + j for i, j in zip(a, b)] + + return buckets, self.mapPartitions(histogram).reduce(mergeCounters) + def mean(self): """ Compute the mean of this RDD's elements. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 51bfbb47e53c2..1db922f513743 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -364,6 +364,110 @@ def test_zip_with_different_number_of_items(self): self.assertEquals(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) + def test_histogram(self): + # empty + rdd = self.sc.parallelize([]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertRaises(ValueError, lambda: rdd.histogram(1)) + + # out of range + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + + # in range with one bucket + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals([4], rdd.histogram([0, 10])[1]) + self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + + # in range with one bucket exact match + self.assertEquals([4], rdd.histogram([1, 4])[1]) + + # out of range with two buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + + # out of range with two uneven buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + + # in range with two buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two bucket and None + rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two uneven buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + + # mixed range with two uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) + self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + + # mixed range with four uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # mixed range with uneven buckets and NaN + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, + 199.0, 200.0, 200.1, None, float('nan')]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # out of range with infinite buckets + rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) + self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + + # invalid buckets + self.assertRaises(ValueError, lambda: rdd.histogram([])) + self.assertRaises(ValueError, lambda: rdd.histogram([1])) + self.assertRaises(ValueError, lambda: rdd.histogram(0)) + self.assertRaises(TypeError, lambda: rdd.histogram({})) + + # without buckets + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + + # without buckets single element + rdd = self.sc.parallelize([1]) + self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + + # without bucket no range + rdd = self.sc.parallelize([1] * 4) + self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + + # without buckets basic two + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + + # without buckets with more requested than elements + rdd = self.sc.parallelize([1, 2]) + buckets = [1 + 0.2 * i for i in range(6)] + hist = [1, 0, 0, 0, 1] + self.assertEquals((buckets, hist), rdd.histogram(5)) + + # invalid RDDs + rdd = self.sc.parallelize([1, float('inf')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + rdd = self.sc.parallelize([float('nan')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + + # string + rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) + self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + + # mixed RDD + rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) + self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + class TestIO(PySparkTestCase): From 98c2bb0bbde6fb2b6f64af3efffefcb0dae94c12 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 13:22:55 -0700 Subject: [PATCH 0479/1492] [SPARK-2969][SQL] Make ScalaReflection be able to handle ArrayType.containsNull and MapType.valueContainsNull. Make `ScalaReflection` be able to handle like: - `Seq[Int]` as `ArrayType(IntegerType, containsNull = false)` - `Seq[java.lang.Integer]` as `ArrayType(IntegerType, containsNull = true)` - `Map[Int, Long]` as `MapType(IntegerType, LongType, valueContainsNull = false)` - `Map[Int, java.lang.Long]` as `MapType(IntegerType, LongType, valueContainsNull = true)` Author: Takuya UESHIN Closes #1889 from ueshin/issues/SPARK-2969 and squashes the following commits: 24f1c5c [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Python API. 79f5b65 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Java API. 7cd1a7a [Takuya UESHIN] Fix json test failures. 2cfb862 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true. 2f38e61 [Takuya UESHIN] Revert the default value of MapTypes.valueContainsNull. 9fa02f5 [Takuya UESHIN] Fix a test failure. 1a9a96b [Takuya UESHIN] Modify ScalaReflection to handle ArrayType.containsNull and MapType.valueContainsNull. --- python/pyspark/sql.py | 6 ++-- .../spark/sql/catalyst/ScalaReflection.scala | 9 ++++-- .../spark/sql/catalyst/types/dataTypes.scala | 4 +-- .../sql/catalyst/ScalaReflectionSuite.scala | 22 +++++++++++-- .../apache/spark/sql/api/java/DataType.java | 4 +-- .../org/apache/spark/sql/DataTypeSuite.scala | 2 +- .../org/apache/spark/sql/json/JsonSuite.scala | 32 +++++++++---------- 7 files changed, 49 insertions(+), 30 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d4ca0cc8f336e..0ff6a548a85f1 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -186,15 +186,15 @@ class ArrayType(DataType): """ - def __init__(self, elementType, containsNull=False): + def __init__(self, elementType, containsNull=True): """Creates an ArrayType :param elementType: the data type of elements. :param containsNull: indicates whether the list contains None values. - >>> ArrayType(StringType) == ArrayType(StringType, False) + >>> ArrayType(StringType) == ArrayType(StringType, True) True - >>> ArrayType(StringType, True) == ArrayType(StringType) + >>> ArrayType(StringType, False) == ArrayType(StringType) False """ self.elementType = elementType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 0d26b52a84695..6b6b636cd96dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -62,11 +62,14 @@ object ScalaReflection { sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") case t if t <:< typeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t - Schema(ArrayType(schemaFor(elementType).dataType), nullable = true) + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) case t if t <:< typeOf[Map[_,_]] => val TypeRef(_, _, Seq(keyType, valueType)) = t - Schema(MapType(schemaFor(keyType).dataType, schemaFor(valueType).dataType), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + Schema(MapType(schemaFor(keyType).dataType, + valueDataType, valueContainsNull = valueNullable), nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index b52ee6d3378a3..70c6d06cf2534 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -270,8 +270,8 @@ case object FloatType extends FractionalType { } object ArrayType { - /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is false. */ - def apply(elementType: DataType): ArrayType = ArrayType(elementType, false) + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index e75373d5a74a7..428607d8c8253 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -57,7 +57,9 @@ case class OptionalData( case class ComplexData( arrayField: Seq[Int], - mapField: Map[Int, String], + arrayFieldContainsNull: Seq[java.lang.Integer], + mapField: Map[Int, Long], + mapFieldValueContainsNull: Map[Int, java.lang.Long], structField: PrimitiveData) case class GenericData[A]( @@ -116,8 +118,22 @@ class ScalaReflectionSuite extends FunSuite { val schema = schemaFor[ComplexData] assert(schema === Schema( StructType(Seq( - StructField("arrayField", ArrayType(IntegerType), nullable = true), - StructField("mapField", MapType(IntegerType, StringType), nullable = true), + StructField( + "arrayField", + ArrayType(IntegerType, containsNull = false), + nullable = true), + StructField( + "arrayFieldContainsNull", + ArrayType(IntegerType, containsNull = true), + nullable = true), + StructField( + "mapField", + MapType(IntegerType, LongType, valueContainsNull = false), + nullable = true), + StructField( + "mapFieldValueContainsNull", + MapType(IntegerType, LongType, valueContainsNull = true), + nullable = true), StructField( "structField", StructType(Seq( diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index 3eccddef88134..37b4c8ffcba0b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -86,14 +86,14 @@ public abstract class DataType { /** * Creates an ArrayType by specifying the data type of elements ({@code elementType}). - * The field of {@code containsNull} is set to {@code false}. + * The field of {@code containsNull} is set to {@code true}. */ public static ArrayType createArrayType(DataType elementType) { if (elementType == null) { throw new IllegalArgumentException("elementType should not be null."); } - return new ArrayType(elementType, false); + return new ArrayType(elementType, true); } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala index cf7d79f42db1d..8fb59c5830f6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -24,7 +24,7 @@ class DataTypeSuite extends FunSuite { test("construct an ArrayType") { val array = ArrayType(StringType) - assert(ArrayType(StringType, false) === array) + assert(ArrayType(StringType, true) === array) } test("construct an MapType") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 58b1e23891a3b..05513a127150c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -130,11 +130,11 @@ class JsonSuite extends QueryTest { checkDataType( ArrayType(IntegerType, true), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, false)) + ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, true)) checkDataType( ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType, false)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType)) + ArrayType(IntegerType, false), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) // StructType checkDataType(StructType(Nil), StructType(Nil), StructType(Nil)) @@ -201,26 +201,26 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldAndType) val expectedSchema = StructType( - StructField("arrayOfArray1", ArrayType(ArrayType(StringType)), true) :: - StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true) :: - StructField("arrayOfBigInteger", ArrayType(DecimalType), true) :: - StructField("arrayOfBoolean", ArrayType(BooleanType), true) :: - StructField("arrayOfDouble", ArrayType(DoubleType), true) :: - StructField("arrayOfInteger", ArrayType(IntegerType), true) :: - StructField("arrayOfLong", ArrayType(LongType), true) :: + StructField("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: + StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, false), false), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType, false), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType, false), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType, false), true) :: + StructField("arrayOfInteger", ArrayType(IntegerType, false), true) :: + StructField("arrayOfLong", ArrayType(LongType, false), true) :: StructField("arrayOfNull", ArrayType(StringType, true), true) :: - StructField("arrayOfString", ArrayType(StringType), true) :: + StructField("arrayOfString", ArrayType(StringType, false), true) :: StructField("arrayOfStruct", ArrayType( StructType( StructField("field1", BooleanType, true) :: StructField("field2", StringType, true) :: - StructField("field3", StringType, true) :: Nil)), true) :: + StructField("field3", StringType, true) :: Nil), false), true) :: StructField("struct", StructType( StructField("field1", BooleanType, true) :: StructField("field2", DecimalType, true) :: Nil), true) :: StructField("structWithArrayFields", StructType( - StructField("field1", ArrayType(IntegerType), true) :: - StructField("field2", ArrayType(StringType), true) :: Nil), true) :: Nil) + StructField("field1", ArrayType(IntegerType, false), true) :: + StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -441,7 +441,7 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldValueTypeConflict) val expectedSchema = StructType( - StructField("array", ArrayType(IntegerType), true) :: + StructField("array", ArrayType(IntegerType, false), true) :: StructField("num_struct", StringType, true) :: StructField("str_array", StringType, true) :: StructField("struct", StructType( @@ -467,7 +467,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("array1", ArrayType(StringType, true), true) :: StructField("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil)), true) :: Nil) + StructField("field", LongType, true) :: Nil), false), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -492,7 +492,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("a", BooleanType, true) :: StructField("b", LongType, true) :: - StructField("c", ArrayType(IntegerType), true) :: + StructField("c", ArrayType(IntegerType, false), true) :: StructField("d", StructType( StructField("field", BooleanType, true) :: Nil), true) :: StructField("e", StringType, true) :: Nil) From 6b5584ef1c605cd30f25dbe7099ab32aea1746fb Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 15:04:08 -0700 Subject: [PATCH 0480/1492] [SPARK-3063][SQL] ExistingRdd should convert Map to catalyst Map. Currently `ExistingRdd.convertToCatalyst` doesn't convert `Map` value. Author: Takuya UESHIN Closes #1963 from ueshin/issues/SPARK-3063 and squashes the following commits: 3ba41f2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 4d7bae2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 9321379 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 d8a900a [Takuya UESHIN] Make ExistingRdd.convertToCatalyst be able to convert Map value. --- .../spark/sql/execution/basicOperators.scala | 3 +- .../sql/ScalaReflectionRelationSuite.scala | 46 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index f9dfa3c92f1eb..374af48b820c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -206,7 +206,8 @@ case class Sort( object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { case o: Option[_] => o.orNull - case s: Seq[Any] => s.map(convertToCatalyst) + case s: Seq[_] => s.map(convertToCatalyst) + case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) case other => other } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 5b84c658db942..e24c521d24c7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -21,6 +21,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.TestSQLContext._ case class ReflectData( @@ -56,6 +57,22 @@ case class OptionalReflectData( case class ReflectBinary(data: Array[Byte]) +case class Nested(i: Option[Int], s: String) + +case class Data( + array: Seq[Int], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapContainsNul: Map[Int, Option[Long]], + nested: Nested) + +case class ComplexReflectData( + arrayField: Seq[Int], + arrayFieldContainsNull: Seq[Option[Int]], + mapField: Map[Int, Long], + mapFieldContainsNull: Map[Int, Option[Long]], + dataField: Data) + class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -90,4 +107,33 @@ class ScalaReflectionRelationSuite extends FunSuite { val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) } + + test("query complex data") { + val data = ComplexReflectData( + Seq(1, 2, 3), + Seq(Some(1), Some(2), None), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), + Data( + Seq(10, 20, 30), + Seq(Some(10), Some(20), None), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), + Nested(None, "abc"))) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerTempTable("reflectComplexData") + + assert(sql("SELECT * FROM reflectComplexData").collect().head === + new GenericRow(Array[Any]( + Seq(1, 2, 3), + Seq(1, 2, null), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> 10L, 2 -> 20L, 3 -> null), + new GenericRow(Array[Any]( + Seq(10, 20, 30), + Seq(10, 20, null), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> 100L, 20 -> 200L, 30 -> null), + new GenericRow(Array[Any](null, "abc"))))))) + } } From adbd5c1636669fc474ab02b54cd1ced353f68712 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 26 Aug 2014 15:12:27 -0700 Subject: [PATCH 0481/1492] [SPARK-3226][MLLIB] doc update for native libraries to mention `-Pnetlib-lgpl` option. atalwalkar Author: Xiangrui Meng Closes #2128 from mengxr/mllib-native and squashes the following commits: 4cbba57 [Xiangrui Meng] update mllib dependencies --- docs/mllib-guide.md | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index ca0a84a8c53fd..4d4198b9e0452 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -35,18 +35,23 @@ and the migration guide below will explain all changes between releases. # Dependencies -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java), and -[jblas](https://github.com/mikiobraun/jblas). +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), +which depends on [netlib-java](https://github.com/fommil/netlib-java), +and [jblas](https://github.com/mikiobraun/jblas). `netlib-java` and `jblas` depend on native Fortran routines. You need to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not -already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries -automatically. Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's -dependency set. If no native library is available at runtime, you will see a warning message. To -use native libraries from `netlib-java`, please include artifact -`com.github.fommil.netlib:all:1.1.2` as a dependency of your project or build your own (see -[instructions](https://github.com/fommil/netlib-java/blob/master/README.md#machine-optimised-system-libraries)). +[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) +if it is not already present on your nodes. +MLlib will throw a linking error if it cannot detect these libraries automatically. +Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's +dependency set under default settings. +If no native library is available at runtime, you will see a warning message. +To use native libraries from `netlib-java`, please build Spark with `-Pnetlib-lgpl` or +include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project. +If you want to use optimized BLAS/LAPACK libraries such as +[OpenBLAS](http://www.openblas.net/), please link its shared libraries to +`/usr/lib/libblas.so.3` and `/usr/lib/liblapack.so.3`, respectively. +BLAS/LAPACK libraries on worker nodes should be built without multithreading. To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. From 1208f72ac78960fe5060187761479b2a9a417c1b Mon Sep 17 00:00:00 2001 From: Burak Date: Tue, 26 Aug 2014 15:18:42 -0700 Subject: [PATCH 0482/1492] [SPARK-2839][MLlib] Stats Toolkit documentation updated Documentation updated for the Statistics Toolkit of MLlib. mengxr atalwalkar https://issues.apache.org/jira/browse/SPARK-2839 P.S. Accidentally closed #2123. New commits didn't show up after I reopened the PR. I've opened this instead and closed the old one. Author: Burak Closes #2130 from brkyvz/StatsLib-Docs and squashes the following commits: a54a855 [Burak] [SPARK-2839][MLlib] Addressed comments bfc6896 [Burak] [SPARK-2839][MLlib] Added a more specific link to colStats() for pyspark 213fe3f [Burak] [SPARK-2839][MLlib] Modifications made according to review fec4d9d [Burak] [SPARK-2830][MLlib] Stats Toolkit documentation updated --- docs/mllib-stats.md | 372 +++++++++++++++++++++++++++++++++++++++----- 1 file changed, 331 insertions(+), 41 deletions(-) diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md index f25dca746ba3a..511a9fbf710cc 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-stats.md @@ -25,6 +25,85 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` +## Summary Statistics + +We provide column summary statistics for `RDD[Vector]` through the function `colStats` +available in `Statistics`. + +
    +
    + +[`colStats()`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) returns an instance of +[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} + +val observations: RDD[Vector] = ... // an RDD of Vectors + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = Statistics.colStats(observations) +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzeros) // number of nonzeros in each column + +{% endhighlight %} +
    + +
    + +[`colStats()`](api/java/org/apache/spark/mllib/stat/Statistics.html) returns an instance of +[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaRDD mat = ... // an RDD of Vectors + +// Compute column summary statistics. +MultivariateStatisticalSummary summary = Statistics.colStats(mat.rdd()); +System.out.println(summary.mean()); // a dense vector containing the mean value for each column +System.out.println(summary.variance()); // column-wise variance +System.out.println(summary.numNonzeros()); // number of nonzeros in each column + +{% endhighlight %} +
    + +
    +[`colStats()`](api/python/pyspark.mllib.stat.Statistics-class.html#colStats) returns an instance of +[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.stat.MultivariateStatisticalSummary-class.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +mat = ... # an RDD of Vectors + +# Compute column summary statistics. +summary = Statistics.colStats(mat) +print summary.mean() +print summary.variance() +print summary.numNonzeros() + +{% endhighlight %} +
    + +
    + ## Random data generation Random data generation is useful for randomized algorithms, prototyping, and performance testing. @@ -99,69 +178,280 @@ v = u.map(lambda x: 1.0 + 2.0 * x)
    -## Stratified Sampling +## Correlations calculation -## Summary Statistics +Calculating the correlation between two series of data is a common operation in Statistics. In MLlib +we provide the flexibility to calculate pairwise correlations among many series. The supported +correlation methods are currently Pearson's and Spearman's correlation. + +
    +
    +[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.stat.Statistics + +val sc: SparkContext = ... + +val seriesX: RDD[Double] = ... // a series +val seriesY: RDD[Double] = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +val correlation: Double = Statistics.corr(seriesX, seriesY, "pearson") + +val data: RDD[Vector] = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +val correlMatrix: Matrix = Statistics.corr(data, "pearson") + +{% endhighlight %} +
    + +
    +[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +calculate correlations between series. Depending on the type of input, two `JavaDoubleRDD`s or +a `JavaRDD`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight java %} +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaDoubleRDD seriesX = ... // a series +JavaDoubleRDD seriesY = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); + +JavaRDD data = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson"); + +{% endhighlight %} +
    -### Multivariate summary statistics +
    +[`Statistics`](api/python/pyspark.mllib.stat.Statistics-class.html) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +seriesX = ... # a series +seriesY = ... # must have the same number of partitions and cardinality as seriesX -We provide column summary statistics for `RowMatrix` (note: this functionality is not currently supported in `IndexedRowMatrix` or `CoordinateMatrix`). -If the number of columns is not large, e.g., on the order of thousands, then the -covariance matrix can also be computed as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the -number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, -and is faster if the rows are sparse. +# Compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +# method is not specified, Pearson's method will be used by default. +print Statistics.corr(seriesX, seriesY, method="pearson") + +data = ... # an RDD of Vectors +# calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +# If a method is not specified, Pearson's method will be used by default. +print Statistics.corr(data, method="pearson") + +{% endhighlight %} +
    + +
    + +## Stratified sampling + +Unlike the other statistics functions, which reside in MLLib, stratified sampling methods, +`sampleByKey` and `sampleByKeyExact`, can be performed on RDD's of key-value pairs. For stratified +sampling, the keys can be thought of as a label and the value as a specific attribute. For example +the key can be man or woman, or document ids, and the respective values can be the list of ages +of the people in the population or the list of words in the documents. The `sampleByKey` method +will flip a coin to decide whether an observation will be sampled or not, therefore requires one +pass over the data, and provides an *expected* sample size. `sampleByKeyExact` requires significant +more resources than the per-stratum simple random sampling used in `sampleByKey`, but will provide +the exact sampling size with 99.99% confidence. `sampleByKeyExact` is currently not supported in +python.
    - -[`computeColumnSummaryStatistics()`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of -[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. +[`sampleByKeyExact()`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.PairRDDFunctions -val mat: RowMatrix = ... // a RowMatrix +val sc: SparkContext = ... -// Compute column summary statistics. -val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() -println(summary.mean) // a dense vector containing the mean value for each column -println(summary.variance) // column-wise variance -println(summary.numNonzeros) // number of nonzeros in each column +val data = ... // an RDD[(K, V)] of any key value pairs +val fractions: Map[K, Double] = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +val approxSample = data.sampleByKey(withReplacement = false, fractions) +val exactSample = data.sampleByKeyExact(withReplacement = false, fractions) -// Compute the covariance matrix. -val cov: Matrix = mat.computeCovariance() {% endhighlight %}
    - -[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of -[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. +[`sampleByKeyExact()`](api/java/org/apache/spark/api/java/JavaPairRDD.html) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight java %} -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import java.util.Map; -RowMatrix mat = ... // a RowMatrix +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; -// Compute column summary statistics. -MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); -System.out.println(summary.mean()); // a dense vector containing the mean value for each column -System.out.println(summary.variance()); // column-wise variance -System.out.println(summary.numNonzeros()); // number of nonzeros in each column +JavaSparkContext jsc = ... + +JavaPairRDD data = ... // an RDD of any key value pairs +Map fractions = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +JavaPairRDD approxSample = data.sampleByKey(false, fractions); +JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); + +{% endhighlight %} +
    +
    +[`sampleByKey()`](api/python/pyspark.rdd.RDD-class.html#sampleByKey) allows users to +sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the +desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the +set of keys. + +*Note:* `sampleByKeyExact()` is currently not supported in Python. + +{% highlight python %} + +sc = ... # SparkContext + +data = ... # an RDD of any key value pairs +fractions = ... # specify the exact fraction desired from each key as a dictionary + +approxSample = data.sampleByKey(False, fractions); -// Compute the covariance matrix. -Matrix cov = mat.computeCovariance(); {% endhighlight %}
    + +
    + +## Hypothesis testing + +Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically +significant, whether this result occurred by chance or not. MLlib currently supports Pearson's +chi-squared ( $\chi^2$) tests for goodness of fit and independence. The input data types determine +whether the goodness of fit or the independence test is conducted. The goodness of fit test requires +an input type of `Vector`, whereas the independence test requires a `Matrix` as input. + +MLlib also supports the input type `RDD[LabeledPoint]` to enable feature selection via chi-squared +independence tests. + +
    +
    +[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.Statistics._ + +val sc: SparkContext = ... + +val vec: Vector = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +val goodnessOfFitTestResult = Statistics.chiSqTest(vec) +println(goodnessOfFitTestResult) // summary of the test including the p-value, degrees of freedom, + // test statistic, the method used, and the null hypothesis. + +val mat: Matrix = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +val independenceTestResult = Statistics.chiSqTest(mat) +println(independenceTestResult) // summary of the test including the p-value, degrees of freedom... + +val obs: RDD[LabeledPoint] = ... // (feature, label) pairs. + +// The contingency table is constructed from the raw (feature, label) pairs and used to conduct +// the independence test. Returns an array containing the ChiSquaredTestResult for every feature +// against the label. +val featureTestResults: Array[ChiSqTestResult] = Statistics.chiSqTest(obs) +var i = 1 +featureTestResults.foreach { result => + println(s"Column $i:\n$result") + i += 1 +} // summary of the test + +{% endhighlight %}
    +
    +[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.stat.Statistics; +import org.apache.spark.mllib.stat.test.ChiSqTestResult; + +JavaSparkContext jsc = ... -## Hypothesis Testing +Vector vec = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +ChiSqTestResult goodnessOfFitTestResult = Statistics.chiSqTest(vec); +// summary of the test including the p-value, degrees of freedom, test statistic, the method used, +// and the null hypothesis. +System.out.println(goodnessOfFitTestResult); + +Matrix mat = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +ChiSqTestResult independenceTestResult = Statistics.chiSqTest(mat); +// summary of the test including the p-value, degrees of freedom... +System.out.println(independenceTestResult); + +JavaRDD obs = ... // an RDD of labeled points + +// The contingency table is constructed from the raw (feature, label) pairs and used to conduct +// the independence test. Returns an array containing the ChiSquaredTestResult for every feature +// against the label. +ChiSqTestResult[] featureTestResults = Statistics.chiSqTest(obs.rdd()); +int i = 1; +for (ChiSqTestResult result : featureTestResults) { + System.out.println("Column " + i + ":"); + System.out.println(result); // summary of the test + i++; +} + +{% endhighlight %} +
    + +
    From c4787a3690a9ed3b8b2c6c294fc4a6915436b6f7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 26 Aug 2014 16:29:14 -0700 Subject: [PATCH 0483/1492] [SPARK-3194][SQL] Add AttributeSet to fix bugs with invalid comparisons of AttributeReferences It is common to want to describe sets of attributes that are in various parts of a query plan. However, the semantics of putting `AttributeReference` objects into a standard Scala `Set` result in subtle bugs when references differ cosmetically. For example, with case insensitive resolution it is possible to have two references to the same attribute whose names are not equal. In this PR I introduce a new abstraction, an `AttributeSet`, which performs all comparisons using the globally unique `ExpressionId` instead of case class equality. (There is already a related class, [`AttributeMap`](https://github.com/marmbrus/spark/blob/inMemStats/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala#L32)) This new type of set is used to fix a bug in the optimizer where needed attributes were getting projected away underneath join operators. I also took this opportunity to refactor the expression and query plan base classes. In all but one instance the logic for computing the `references` of an `Expression` were the same. Thus, I moved this logic into the base class. For query plans the semantics of the `references` method were ill defined (is it the references output? or is it those used by expression evaluation? or what?). As a result, this method wasn't really used very much. So, I removed it. TODO: - [x] Finish scala doc for `AttributeSet` - [x] Scan the code for other instances of `Set[Attribute]` and refactor them. - [x] Finish removing `references` from `QueryPlan` Author: Michael Armbrust Closes #2109 from marmbrus/attributeSets and squashes the following commits: 1c0dae5 [Michael Armbrust] work on serialization bug. 9ba868d [Michael Armbrust] Merge remote-tracking branch 'origin/master' into attributeSets 3ae5288 [Michael Armbrust] review comments 40ce7f6 [Michael Armbrust] style d577cc7 [Michael Armbrust] Scaladoc cae5d22 [Michael Armbrust] remove more references implementations d6e16be [Michael Armbrust] Remove more instances of "def references" and normal sets of attributes. fc26b49 [Michael Armbrust] Add AttributeSet class, remove references from Expression. --- .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../sql/catalyst/analysis/unresolved.scala | 1 - .../catalyst/expressions/AttributeSet.scala | 106 ++++++++++++++++++ .../catalyst/expressions/BoundAttribute.scala | 2 - .../sql/catalyst/expressions/Expression.scala | 6 +- .../spark/sql/catalyst/expressions/Rand.scala | 1 - .../sql/catalyst/expressions/ScalaUdf.scala | 1 - .../sql/catalyst/expressions/SortOrder.scala | 1 - .../catalyst/expressions/WrapDynamic.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 25 ++--- .../sql/catalyst/expressions/arithmetic.scala | 2 - .../catalyst/expressions/complexTypes.scala | 2 +- .../sql/catalyst/expressions/generators.scala | 2 - .../sql/catalyst/expressions/literals.scala | 4 +- .../expressions/namedExpressions.scala | 6 +- .../catalyst/expressions/nullFunctions.scala | 3 - .../sql/catalyst/expressions/predicates.scala | 6 +- .../spark/sql/catalyst/expressions/sets.scala | 5 - .../expressions/stringOperations.scala | 2 - .../sql/catalyst/optimizer/Optimizer.scala | 12 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../catalyst/plans/logical/LogicalPlan.scala | 11 +- .../plans/logical/ScriptTransformation.scala | 4 +- .../plans/logical/basicOperators.scala | 29 +---- .../catalyst/plans/logical/partitioning.scala | 4 - .../plans/physical/partitioning.scala | 3 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 1 - .../org/apache/spark/sql/SQLContext.scala | 7 +- .../columnar/InMemoryColumnarTableScan.scala | 2 - .../spark/sql/execution/SparkPlan.scala | 3 +- .../spark/sql/execution/debug/package.scala | 2 - .../spark/sql/execution/pythonUdfs.scala | 2 - .../spark/sql/hive/HiveStrategies.scala | 8 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 5 - .../hive/execution/HiveResolutionSuite.scala | 9 +- 35 files changed, 166 insertions(+), 123 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index c18d7858f0a43..4a9524074132e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -132,7 +132,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) val resolved = unresolved.flatMap(child.resolveChildren) - val requiredAttributes = resolved.collect { case a: Attribute => a }.toSet + val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) val missingInProject = requiredAttributes -- p.output if (missingInProject.nonEmpty) { @@ -152,8 +152,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve).toSet - val missingInAggs = resolved -- a.outputSet + val resolved = unresolved.flatMap(groupingRelation.resolve) + val missingInAggs = resolved.filterNot(a.outputSet.contains) logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") if (missingInAggs.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a0e25775da6dd..a2c61c65487cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -66,7 +66,6 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") override def nullable = throw new UnresolvedException(this, "nullable") - override def references = children.flatMap(_.references).toSet override lazy val resolved = false // Unresolved functions are transient at compile time and don't get evaluated during execution. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala new file mode 100644 index 0000000000000..c3a08bbdb6bc7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +protected class AttributeEquals(val a: Attribute) { + override def hashCode() = a.exprId.hashCode() + override def equals(other: Any) = other match { + case otherReference: AttributeEquals => a.exprId == otherReference.a.exprId + case otherAttribute => false + } +} + +object AttributeSet { + /** Constructs a new [[AttributeSet]] given a sequence of [[Attribute Attributes]]. */ + def apply(baseSet: Seq[Attribute]) = { + new AttributeSet(baseSet.map(new AttributeEquals(_)).toSet) + } +} + +/** + * A Set designed to hold [[AttributeReference]] objects, that performs equality checking using + * expression id instead of standard java equality. Using expression id means that these + * sets will correctly test for membership, even when the AttributeReferences in question differ + * cosmetically (e.g., the names have different capitalizations). + * + * Note that we do not override equality for Attribute references as it is really weird when + * `AttributeReference("a"...) == AttrributeReference("b", ...)`. This tactic leads to broken tests, + * and also makes doing transformations hard (we always try keep older trees instead of new ones + * when the transformation was a no-op). + */ +class AttributeSet private (val baseSet: Set[AttributeEquals]) + extends Traversable[Attribute] with Serializable { + + /** Returns true if the members of this AttributeSet and other are the same. */ + override def equals(other: Any) = other match { + case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains) + case _ => false + } + + /** Returns true if this set contains an Attribute with the same expression id as `elem` */ + def contains(elem: NamedExpression): Boolean = + baseSet.contains(new AttributeEquals(elem.toAttribute)) + + /** Returns a new [[AttributeSet]] that contains `elem` in addition to the current elements. */ + def +(elem: Attribute): AttributeSet = // scalastyle:ignore + new AttributeSet(baseSet + new AttributeEquals(elem)) + + /** Returns a new [[AttributeSet]] that does not contain `elem`. */ + def -(elem: Attribute): AttributeSet = + new AttributeSet(baseSet - new AttributeEquals(elem)) + + /** Returns an iterator containing all of the attributes in the set. */ + def iterator: Iterator[Attribute] = baseSet.map(_.a).iterator + + /** + * Returns true if the [[Attribute Attributes]] in this set are a subset of the Attributes in + * `other`. + */ + def subsetOf(other: AttributeSet) = baseSet.subsetOf(other.baseSet) + + /** + * Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found + * in `other`. + */ + def --(other: Traversable[NamedExpression]) = + new AttributeSet(baseSet -- other.map(a => new AttributeEquals(a.toAttribute))) + + /** + * Returns a new [[AttributeSet]] that contains all of the [[Attribute Attributes]] found + * in `other`. + */ + def ++(other: AttributeSet) = new AttributeSet(baseSet ++ other.baseSet) + + /** + * Returns a new [[AttributeSet]] contain only the [[Attribute Attributes]] where `f` evaluates to + * true. + */ + override def filter(f: Attribute => Boolean) = new AttributeSet(baseSet.filter(ae => f(ae.a))) + + /** + * Returns a new [[AttributeSet]] that only contains [[Attribute Attributes]] that are found in + * `this` and `other`. + */ + def intersect(other: AttributeSet) = new AttributeSet(baseSet.intersect(other.baseSet)) + + override def foreach[U](f: (Attribute) => U): Unit = baseSet.map(_.a).foreach(f) + + // We must force toSeq to not be strict otherwise we end up with a [[Stream]] that captures all + // sorts of things in its closure. + override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 0913f15888780..54c6baf1af3bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -32,8 +32,6 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) type EvaluatedType = Any - override def references = Set.empty - override def toString = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ba62dabe3dd6a..70507e7ee2be8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -41,7 +41,7 @@ abstract class Expression extends TreeNode[Expression] { */ def foldable: Boolean = false def nullable: Boolean - def references: Set[Attribute] + def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator)) /** Returns the result of evaluating this expression on a given input Row */ def eval(input: Row = null): EvaluatedType @@ -230,8 +230,6 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def foldable = left.foldable && right.foldable - override def references = left.references ++ right.references - override def toString = s"($left $symbol $right)" } @@ -242,5 +240,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => - override def references = child.references + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index 38f836f0a1a0e..851db95b9177e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.types.DoubleType case object Rand extends LeafExpression { override def dataType = DoubleType override def nullable = false - override def references = Set.empty private[this] lazy val rand = new Random diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 95633dd0c9870..63ac2a608b6ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -24,7 +24,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi type EvaluatedType = Any - def references = children.flatMap(_.references).toSet def nullable = true /** This method has been generated by this script diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index d2b7685e73065..d00b2ac09745c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -31,7 +31,6 @@ case object Descending extends SortDirection case class SortOrder(child: Expression, direction: SortDirection) extends Expression with trees.UnaryNode[Expression] { - override def references = child.references override def dataType = child.dataType override def nullable = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index eb8898900d6a5..1eb55715794a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -35,7 +35,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { type EvaluatedType = DynamicRow def nullable = false - def references = children.toSet + def dataType = DynamicType override def eval(input: Row): DynamicRow = input match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 613b87ca98d97..dbc0c2965a805 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -78,7 +78,7 @@ abstract class AggregateFunction /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - override def references = base.references + override def nullable = base.nullable override def dataType = base.dataType @@ -89,7 +89,7 @@ abstract class AggregateFunction } case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MIN($child)" @@ -119,7 +119,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MAX($child)" @@ -149,7 +149,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"COUNT($child)" @@ -166,7 +166,7 @@ case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate def this() = this(null) override def children = expressions - override def references = expressions.flatMap(_.references).toSet + override def nullable = false override def dataType = LongType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" @@ -184,7 +184,6 @@ case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpress def this() = this(null) override def children = expressions - override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = ArrayType(expressions.head.dataType) override def toString = s"AddToHashSet(${expressions.mkString(",")})" @@ -219,7 +218,6 @@ case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression def this() = this(null) override def children = inputSet :: Nil - override def references = inputSet.references override def nullable = false override def dataType = LongType override def toString = s"CombineAndCount($inputSet)" @@ -248,7 +246,7 @@ case class CombineSetsAndCountFunction( case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -257,7 +255,7 @@ case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -266,7 +264,7 @@ case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -284,7 +282,7 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = DoubleType override def toString = s"AVG($child)" @@ -304,7 +302,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM($child)" @@ -322,7 +320,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case class SumDistinct(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" @@ -331,7 +329,6 @@ case class SumDistinct(child: Expression) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references override def nullable = true override def dataType = child.dataType override def toString = s"FIRST($child)" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 5f8b6ae10f0c4..aae86a3628be1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -95,8 +95,6 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def children = left :: right :: Nil - override def references = left.references ++ right.references - override def dataType = left.dataType override def eval(input: Row): Any = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index c1154eb81c319..dafd745ec96c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -31,7 +31,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { /** `Null` is returned for invalid ordinals. */ override def nullable = true override def foldable = child.foldable && ordinal.foldable - override def references = children.flatMap(_.references).toSet + def dataType = child.dataType match { case ArrayType(dt, _) => dt case MapType(_, vt, _) => vt diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index e99c5b452d183..9c865254e0be9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -47,8 +47,6 @@ abstract class Generator extends Expression { override def nullable = false - override def references = children.flatMap(_.references).toSet - /** * Should be overridden by specific generators. Called only once for each instance to ensure * that rule application does not change the output schema of a generator. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index e15e16d633365..a8c2396d62632 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -52,7 +52,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def foldable = true def nullable = value == null - def references = Set.empty + override def toString = if (value != null) value.toString else "null" @@ -66,8 +66,6 @@ case class MutableLiteral(var value: Any, nullable: Boolean = true) extends Leaf val dataType = Literal(value).dataType - def references = Set.empty - def update(expression: Expression, input: Row) = { value = expression.eval(input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 02d04762629f5..7c4b9d4847e26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -62,7 +62,7 @@ abstract class Attribute extends NamedExpression { def toAttribute = this def newInstance: Attribute - override def references = Set(this) + } /** @@ -85,7 +85,7 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable - override def references = child.references + override def toAttribute = { if (resolved) { @@ -116,6 +116,8 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { + override def references = AttributeSet(this :: Nil) + override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index e88c5d4fa178a..086d0a3e073e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -26,7 +26,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ def nullable = !children.exists(!_.nullable) - def references = children.flatMap(_.references).toSet // Coalesce is foldable if all children are foldable. override def foldable = !children.exists(!_.foldable) @@ -53,7 +52,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false @@ -65,7 +63,6 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false override def toString = s"IS NOT NULL $child" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 5976b0ddf3e03..1313ccd120c1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -85,7 +85,7 @@ case class Not(child: Expression) extends UnaryExpression with Predicate { */ case class In(value: Expression, list: Seq[Expression]) extends Predicate { def children = value +: list - def references = children.flatMap(_.references).toSet + def nullable = true // TODO: Figure out correct nullability semantics of IN. override def toString = s"$value IN ${list.mkString("(", ",", ")")}" @@ -197,7 +197,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi def children = predicate :: trueValue :: falseValue :: Nil override def nullable = trueValue.nullable || falseValue.nullable - def references = children.flatMap(_.references).toSet + override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { if (!resolved) { @@ -239,7 +239,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi case class CaseWhen(branches: Seq[Expression]) extends Expression { type EvaluatedType = Any def children = branches - def references = children.flatMap(_.references).toSet + def dataType = { if (!resolved) { throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index e6c570b47bee2..3d4c4a8853c12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -26,8 +26,6 @@ import org.apache.spark.util.collection.OpenHashSet case class NewSet(elementType: DataType) extends LeafExpression { type EvaluatedType = Any - def references = Set.empty - def nullable = false // We are currently only using these Expressions internally for aggregation. However, if we ever @@ -53,9 +51,6 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { def nullable = set.nullable def dataType = set.dataType - - def references = (item.flatMap(_.references) ++ set.flatMap(_.references)).toSet - def eval(input: Row): Any = { val itemEval = item.eval(input) val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 97fc3a3b14b88..c2a3a5ca3ca8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -226,8 +226,6 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends if (str.dataType == BinaryType) str.dataType else StringType } - def references = children.flatMap(_.references).toSet - override def children = str :: pos :: len :: Nil @inline diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 5f86d6047cb9c..ddd4b3755d629 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -65,8 +65,10 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => // Collect the list of all references required either above or to evaluate the condition. - val allReferences: Set[Attribute] = - projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + AttributeSet( + projectList.flatMap(_.references.iterator)) ++ + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) /** Applies a projection only when the child is producing unnecessary attributes */ def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences) @@ -76,8 +78,8 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from right side of a LeftSemiJoin. case Join(left, right, LeftSemi, condition) => // Collect the list of all references required to evaluate the condition. - val allReferences: Set[Attribute] = - condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) Join(left, prunedChild(right, allReferences), LeftSemi, condition) @@ -104,7 +106,7 @@ object ColumnPruning extends Rule[LogicalPlan] { } /** Applies a projection only when the child is producing unnecessary attributes */ - private def prunedChild(c: LogicalPlan, allReferences: Set[Attribute]) = + private def prunedChild(c: LogicalPlan, allReferences: AttributeSet) = if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { Project(allReferences.filter(c.outputSet.contains).toSeq, c) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 0988b0c6d990c..1e177e28f80b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} @@ -29,7 +29,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy /** * Returns the set of attributes that are output by this node. */ - def outputSet: Set[Attribute] = output.toSet + def outputSet: AttributeSet = AttributeSet(output) /** * Runs [[transform]] with `rule` on all expressions present in this query operator. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 278569f0cb14a..8616ac45b0e95 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -45,17 +45,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product ) - /** - * Returns the set of attributes that are referenced by this node - * during evaluation. - */ - def references: Set[Attribute] - /** * Returns the set of attributes that this node takes as * input from its children. */ - lazy val inputSet: Set[Attribute] = children.flatMap(_.output).toSet + lazy val inputSet: AttributeSet = AttributeSet(children.flatMap(_.output)) /** * Returns true if this expression and all its children have been resolved to a specific schema @@ -126,9 +120,6 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { override lazy val statistics: Statistics = throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") - - // Leaf nodes by definition cannot reference any input attributes. - override def references = Set.empty } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index d3f9d0fb93237..4460c86ed9026 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -30,6 +30,4 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { - def references = input.flatMap(_.references).toSet -} + child: LogicalPlan) extends UnaryNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3cb407217c4c3..4adfb189372d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) - def references = projectList.flatMap(_.references).toSet } /** @@ -59,14 +58,10 @@ case class Generate( override def output = if (join) child.output ++ generatorOutput else generatorOutput - - override def references = - if (join) child.outputSet else generator.references } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = condition.references } case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { @@ -76,8 +71,6 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved = childrenResolved && !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } - - override def references = Set.empty } case class Join( @@ -86,8 +79,6 @@ case class Join( joinType: JoinType, condition: Option[Expression]) extends BinaryNode { - override def references = condition.map(_.references).getOrElse(Set.empty) - override def output = { joinType match { case LeftSemi => @@ -106,8 +97,6 @@ case class Join( case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { def output = left.output - - def references = Set.empty } case class InsertIntoTable( @@ -118,7 +107,6 @@ case class InsertIntoTable( extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. override def children = table :: child :: Nil - override def references = Set.empty override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { @@ -130,20 +118,17 @@ case class InsertIntoCreatedTable( databaseName: Option[String], tableName: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = order.flatMap(_.references).toSet } case class Aggregate( @@ -152,19 +137,20 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { + /** The set of all AttributeReferences required for this aggregation. */ + def references = + AttributeSet( + groupingExpressions.flatMap(_.references) ++ aggregateExpressions.flatMap(_.references)) + override def output = aggregateExpressions.map(_.toAttribute) - override def references = - (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet } case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = limitExpr.references } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { override def output = child.output.map(_.withQualifiers(alias :: Nil)) - override def references = Set.empty } /** @@ -191,20 +177,16 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { a.qualifiers) case other => other } - - override def references = Set.empty } case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = Set.empty } case class Distinct(child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = child.outputSet } case object NoRelation extends LeafNode { @@ -213,5 +195,4 @@ case object NoRelation extends LeafNode { case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override def output = left.output - override def references = Set.empty } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 7146fbd540f29..72b0c5c8e7a26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -31,13 +31,9 @@ abstract class RedistributeData extends UnaryNode { case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) extends RedistributeData { - - def references = sortExpressions.flatMap(_.references).toSet } case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) extends RedistributeData { - - def references = partitionExpressions.flatMap(_.references).toSet } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 4bb022cf238af..ccb0df113c063 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -71,6 +71,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { "An AllTuples should be used to represent a distribution that only has " + "a single partition.") + // TODO: This is not really valid... def clustering = ordering.map(_.child).toSet } @@ -139,7 +140,6 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) with Partitioning { override def children = expressions - override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType @@ -179,7 +179,6 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) with Partitioning { override def children = ordering - override def references = ordering.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6344874538d67..296202543e2ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.types.{StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children = optKey.toSeq - def references = Set.empty[Attribute] def nullable = true def dataType = NullType override lazy val resolved = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 8a9f4deb6a19e..6f0eed3f63c41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -344,8 +344,8 @@ class SQLContext(@transient val sparkContext: SparkContext) prunePushedDownFilters: Seq[Expression] => Seq[Expression], scanBuilder: Seq[Attribute] => SparkPlan): SparkPlan = { - val projectSet = projectList.flatMap(_.references).toSet - val filterSet = filterPredicates.flatMap(_.references).toSet + val projectSet = AttributeSet(projectList.flatMap(_.references)) + val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) val filterCondition = prunePushedDownFilters(filterPredicates).reduceLeftOption(And) // Right now we still use a projection even if the only evaluation is applying an alias @@ -354,7 +354,8 @@ class SQLContext(@transient val sparkContext: SparkContext) // TODO: Decouple final output schema from expression evaluation so this copy can be // avoided safely. - if (projectList.toSet == projectSet && filterSet.subsetOf(projectSet)) { + if (AttributeSet(projectList.map(_.toAttribute)) == projectSet && + filterSet.subsetOf(projectSet)) { // When it is possible to just use column pruning to get the right projection and // when the columns of this projection are enough to evaluate all filter conditions, // just do a scan followed by a filter, with no extra project. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index e63b4903041f6..24e88eea3189e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -79,8 +79,6 @@ private[sql] case class InMemoryRelation( override def children = Seq.empty - override def references = Set.empty - override def newInstance() = { new InMemoryRelation( output.map(_.newInstance), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 21cbbc9772a00..7d33ea5b021e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -141,10 +141,9 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ extends LogicalPlan with MultiInstanceRelation { def output = alreadyPlanned.output - override def references = Set.empty override def children = Nil - override final def newInstance: this.type = { + override final def newInstance(): this.type = { SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index f31df051824d7..5b896c55b7393 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -58,8 +58,6 @@ package object debug { } private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { - def references = Set.empty - def output = child.output implicit object SetAccumulatorParam extends AccumulatorParam[HashSet[String]] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index b92091b560b1c..aef6ebf86b1eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -49,7 +49,6 @@ private[spark] case class PythonUDF( override def toString = s"PythonUDF#$name(${children.mkString(",")})" def nullable: Boolean = true - def references: Set[Attribute] = children.flatMap(_.references).toSet override def eval(input: Row) = sys.error("PythonUDFs can not be directly evaluated.") } @@ -113,7 +112,6 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() - def references = Set.empty def output = child.output :+ resultAttribute } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 389ace726d205..10fa8314c9156 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -79,9 +79,9 @@ private[hive] trait HiveStrategies { hiveContext.convertMetastoreParquet => // Filter out all predicates that only deal with partition keys - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionsKeys = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionsKeys) } // We are going to throw the predicates and projection back at the whole optimization @@ -176,9 +176,9 @@ private[hive] trait HiveStrategies { case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => // Filter out all predicates that only deal with partition keys, these are given to the // hive table scan operator to be used for partition pruning. - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionKeyIds = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionKeyIds) } pruneFilterProject( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index c6497a15efa0c..7d1ad53d8bdb3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -88,7 +88,6 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu type EvaluatedType = Any def nullable = true - def references = children.flatMap(_.references).toSet lazy val function = createFunction[UDFType]() @@ -229,8 +228,6 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - def references: Set[Attribute] = children.map(_.references).flatten.toSet - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" def newInstance() = new HiveUdafFunction(functionClassName, children, this) @@ -253,8 +250,6 @@ private[hive] case class HiveGenericUdtf( children: Seq[Expression]) extends Generator with HiveInspectors with HiveFunctionFactory { - override def references = children.flatMap(_.references).toSet - @transient protected lazy val function: GenericUDTF = createFunction() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 6b3ffd1c0ffe2..b6be6bc1bfefe 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) case class Nested(a: Int, B: Int) +case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. @@ -57,6 +57,13 @@ class HiveResolutionSuite extends HiveComparisonTest { .registerTempTable("caseSensitivityTest") sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + + println(sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").queryExecution) + + sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").collect() + + // TODO: sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a") + } test("nested repeated resolution") { From f1e71d4c3ba678fc108effb05cf2d6101dadc0ce Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 26 Aug 2014 16:57:40 -0700 Subject: [PATCH 0484/1492] [SPARK-3073] [PySpark] use external sort in sortBy() and sortByKey() Using external sort to support sort large datasets in reduce stage. Author: Davies Liu Closes #1978 from davies/sort and squashes the following commits: bbcd9ba [Davies Liu] check spilled bytes in tests b125d2f [Davies Liu] add test for external sort in rdd eae0176 [Davies Liu] choose different disks from different processes and instances 1f075ed [Davies Liu] Merge branch 'master' into sort eb53ca6 [Davies Liu] Merge branch 'master' into sort 644abaf [Davies Liu] add license in LICENSE 19f7873 [Davies Liu] improve tests 55602ee [Davies Liu] use external sort in sortBy() and sortByKey() --- .rat-excludes | 1 + LICENSE | 283 ++++++++++++ python/pyspark/heapq3.py | 890 ++++++++++++++++++++++++++++++++++++++ python/pyspark/rdd.py | 9 +- python/pyspark/shuffle.py | 91 +++- python/pyspark/tests.py | 42 +- tox.ini | 2 +- 7 files changed, 1306 insertions(+), 12 deletions(-) create mode 100644 python/pyspark/heapq3.py diff --git a/.rat-excludes b/.rat-excludes index eaefef1b0aa2e..fb6323daf9211 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -31,6 +31,7 @@ sorttable.js .*data .*log cloudpickle.py +heapq3.py join.py SparkExprTyper.scala SparkILoop.scala diff --git a/LICENSE b/LICENSE index e9a1153fdc5db..a7eee041129cb 100644 --- a/LICENSE +++ b/LICENSE @@ -338,6 +338,289 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +======================================================================== +For heapq (pyspark/heapq3.py): +======================================================================== + +# A. HISTORY OF THE SOFTWARE +# ========================== +# +# Python was created in the early 1990s by Guido van Rossum at Stichting +# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands +# as a successor of a language called ABC. Guido remains Python's +# principal author, although it includes many contributions from others. +# +# In 1995, Guido continued his work on Python at the Corporation for +# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) +# in Reston, Virginia where he released several versions of the +# software. +# +# In May 2000, Guido and the Python core development team moved to +# BeOpen.com to form the BeOpen PythonLabs team. In October of the same +# year, the PythonLabs team moved to Digital Creations (now Zope +# Corporation, see http://www.zope.com). In 2001, the Python Software +# Foundation (PSF, see http://www.python.org/psf/) was formed, a +# non-profit organization created specifically to own Python-related +# Intellectual Property. Zope Corporation is a sponsoring member of +# the PSF. +# +# All Python releases are Open Source (see http://www.opensource.org for +# the Open Source Definition). Historically, most, but not all, Python +# releases have also been GPL-compatible; the table below summarizes +# the various releases. +# +# Release Derived Year Owner GPL- +# from compatible? (1) +# +# 0.9.0 thru 1.2 1991-1995 CWI yes +# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes +# 1.6 1.5.2 2000 CNRI no +# 2.0 1.6 2000 BeOpen.com no +# 1.6.1 1.6 2001 CNRI yes (2) +# 2.1 2.0+1.6.1 2001 PSF no +# 2.0.1 2.0+1.6.1 2001 PSF yes +# 2.1.1 2.1+2.0.1 2001 PSF yes +# 2.2 2.1.1 2001 PSF yes +# 2.1.2 2.1.1 2002 PSF yes +# 2.1.3 2.1.2 2002 PSF yes +# 2.2.1 2.2 2002 PSF yes +# 2.2.2 2.2.1 2002 PSF yes +# 2.2.3 2.2.2 2003 PSF yes +# 2.3 2.2.2 2002-2003 PSF yes +# 2.3.1 2.3 2002-2003 PSF yes +# 2.3.2 2.3.1 2002-2003 PSF yes +# 2.3.3 2.3.2 2002-2003 PSF yes +# 2.3.4 2.3.3 2004 PSF yes +# 2.3.5 2.3.4 2005 PSF yes +# 2.4 2.3 2004 PSF yes +# 2.4.1 2.4 2005 PSF yes +# 2.4.2 2.4.1 2005 PSF yes +# 2.4.3 2.4.2 2006 PSF yes +# 2.4.4 2.4.3 2006 PSF yes +# 2.5 2.4 2006 PSF yes +# 2.5.1 2.5 2007 PSF yes +# 2.5.2 2.5.1 2008 PSF yes +# 2.5.3 2.5.2 2008 PSF yes +# 2.6 2.5 2008 PSF yes +# 2.6.1 2.6 2008 PSF yes +# 2.6.2 2.6.1 2009 PSF yes +# 2.6.3 2.6.2 2009 PSF yes +# 2.6.4 2.6.3 2009 PSF yes +# 2.6.5 2.6.4 2010 PSF yes +# 2.7 2.6 2010 PSF yes +# +# Footnotes: +# +# (1) GPL-compatible doesn't mean that we're distributing Python under +# the GPL. All Python licenses, unlike the GPL, let you distribute +# a modified version without making your changes open source. The +# GPL-compatible licenses make it possible to combine Python with +# other software that is released under the GPL; the others don't. +# +# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, +# because its license has a choice of law clause. According to +# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 +# is "not incompatible" with the GPL. +# +# Thanks to the many outside volunteers who have worked under Guido's +# direction to make these releases possible. +# +# +# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON +# =============================================================== +# +# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 +# -------------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Python Software Foundation +# ("PSF"), and the Individual or Organization ("Licensee") accessing and +# otherwise using this software ("Python") in source or binary form and +# its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, PSF hereby +# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, +# analyze, test, perform and/or display publicly, prepare derivative works, +# distribute, and otherwise use Python alone or in any derivative version, +# provided, however, that PSF's License Agreement and PSF's notice of copyright, +# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, +# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained +# in Python alone or in any derivative version prepared by Licensee. +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python. +# +# 4. PSF is making Python available to Licensee on an "AS IS" +# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. Nothing in this License Agreement shall be deemed to create any +# relationship of agency, partnership, or joint venture between PSF and +# Licensee. This License Agreement does not grant permission to use PSF +# trademarks or trade name in a trademark sense to endorse or promote +# products or services of Licensee, or any third party. +# +# 8. By copying, installing or otherwise using Python, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 +# ------------------------------------------- +# +# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 +# +# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an +# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the +# Individual or Organization ("Licensee") accessing and otherwise using +# this software in source or binary form and its associated +# documentation ("the Software"). +# +# 2. Subject to the terms and conditions of this BeOpen Python License +# Agreement, BeOpen hereby grants Licensee a non-exclusive, +# royalty-free, world-wide license to reproduce, analyze, test, perform +# and/or display publicly, prepare derivative works, distribute, and +# otherwise use the Software alone or in any derivative version, +# provided, however, that the BeOpen Python License is retained in the +# Software, alone or in any derivative version prepared by Licensee. +# +# 3. BeOpen is making the Software available to Licensee on an "AS IS" +# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE +# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS +# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY +# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 5. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 6. This License Agreement shall be governed by and interpreted in all +# respects by the law of the State of California, excluding conflict of +# law provisions. Nothing in this License Agreement shall be deemed to +# create any relationship of agency, partnership, or joint venture +# between BeOpen and Licensee. This License Agreement does not grant +# permission to use BeOpen trademarks or trade names in a trademark +# sense to endorse or promote products or services of Licensee, or any +# third party. As an exception, the "BeOpen Python" logos available at +# http://www.pythonlabs.com/logos.html may be used according to the +# permissions granted on that web page. +# +# 7. By copying, installing or otherwise using the software, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 +# --------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Corporation for National +# Research Initiatives, having an office at 1895 Preston White Drive, +# Reston, VA 20191 ("CNRI"), and the Individual or Organization +# ("Licensee") accessing and otherwise using Python 1.6.1 software in +# source or binary form and its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, CNRI +# hereby grants Licensee a nonexclusive, royalty-free, world-wide +# license to reproduce, analyze, test, perform and/or display publicly, +# prepare derivative works, distribute, and otherwise use Python 1.6.1 +# alone or in any derivative version, provided, however, that CNRI's +# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) +# 1995-2001 Corporation for National Research Initiatives; All Rights +# Reserved" are retained in Python 1.6.1 alone or in any derivative +# version prepared by Licensee. Alternately, in lieu of CNRI's License +# Agreement, Licensee may substitute the following text (omitting the +# quotes): "Python 1.6.1 is made available subject to the terms and +# conditions in CNRI's License Agreement. This Agreement together with +# Python 1.6.1 may be located on the Internet using the following +# unique, persistent identifier (known as a handle): 1895.22/1013. This +# Agreement may also be obtained from a proxy server on the Internet +# using the following URL: http://hdl.handle.net/1895.22/1013". +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python 1.6.1 or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python 1.6.1. +# +# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" +# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. This License Agreement shall be governed by the federal +# intellectual property law of the United States, including without +# limitation the federal copyright law, and, to the extent such +# U.S. federal law does not apply, by the law of the Commonwealth of +# Virginia, excluding Virginia's conflict of law provisions. +# Notwithstanding the foregoing, with regard to derivative works based +# on Python 1.6.1 that incorporate non-separable material that was +# previously distributed under the GNU General Public License (GPL), the +# law of the Commonwealth of Virginia shall govern this License +# Agreement only as to issues arising under or with respect to +# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this +# License Agreement shall be deemed to create any relationship of +# agency, partnership, or joint venture between CNRI and Licensee. This +# License Agreement does not grant permission to use CNRI trademarks or +# trade name in a trademark sense to endorse or promote products or +# services of Licensee, or any third party. +# +# 8. By clicking on the "ACCEPT" button where indicated, or by copying, +# installing or otherwise using Python 1.6.1, Licensee agrees to be +# bound by the terms and conditions of this License Agreement. +# +# ACCEPT +# +# +# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 +# -------------------------------------------------- +# +# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, +# The Netherlands. All rights reserved. +# +# Permission to use, copy, modify, and distribute this software and its +# documentation for any purpose and without fee is hereby granted, +# provided that the above copyright notice appear in all copies and that +# both that copyright notice and this permission notice appear in +# supporting documentation, and that the name of Stichting Mathematisch +# Centrum or CWI not be used in advertising or publicity pertaining to +# distribution of the software without specific, written prior +# permission. +# +# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO +# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND +# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE +# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES +# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN +# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT +# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. ======================================================================== For sorttable (core/src/main/resources/org/apache/spark/ui/static/sorttable.js): diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py new file mode 100644 index 0000000000000..bc441f138f7fc --- /dev/null +++ b/python/pyspark/heapq3.py @@ -0,0 +1,890 @@ +# -*- encoding: utf-8 -*- +# back ported from CPython 3 +# A. HISTORY OF THE SOFTWARE +# ========================== +# +# Python was created in the early 1990s by Guido van Rossum at Stichting +# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands +# as a successor of a language called ABC. Guido remains Python's +# principal author, although it includes many contributions from others. +# +# In 1995, Guido continued his work on Python at the Corporation for +# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) +# in Reston, Virginia where he released several versions of the +# software. +# +# In May 2000, Guido and the Python core development team moved to +# BeOpen.com to form the BeOpen PythonLabs team. In October of the same +# year, the PythonLabs team moved to Digital Creations (now Zope +# Corporation, see http://www.zope.com). In 2001, the Python Software +# Foundation (PSF, see http://www.python.org/psf/) was formed, a +# non-profit organization created specifically to own Python-related +# Intellectual Property. Zope Corporation is a sponsoring member of +# the PSF. +# +# All Python releases are Open Source (see http://www.opensource.org for +# the Open Source Definition). Historically, most, but not all, Python +# releases have also been GPL-compatible; the table below summarizes +# the various releases. +# +# Release Derived Year Owner GPL- +# from compatible? (1) +# +# 0.9.0 thru 1.2 1991-1995 CWI yes +# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes +# 1.6 1.5.2 2000 CNRI no +# 2.0 1.6 2000 BeOpen.com no +# 1.6.1 1.6 2001 CNRI yes (2) +# 2.1 2.0+1.6.1 2001 PSF no +# 2.0.1 2.0+1.6.1 2001 PSF yes +# 2.1.1 2.1+2.0.1 2001 PSF yes +# 2.2 2.1.1 2001 PSF yes +# 2.1.2 2.1.1 2002 PSF yes +# 2.1.3 2.1.2 2002 PSF yes +# 2.2.1 2.2 2002 PSF yes +# 2.2.2 2.2.1 2002 PSF yes +# 2.2.3 2.2.2 2003 PSF yes +# 2.3 2.2.2 2002-2003 PSF yes +# 2.3.1 2.3 2002-2003 PSF yes +# 2.3.2 2.3.1 2002-2003 PSF yes +# 2.3.3 2.3.2 2002-2003 PSF yes +# 2.3.4 2.3.3 2004 PSF yes +# 2.3.5 2.3.4 2005 PSF yes +# 2.4 2.3 2004 PSF yes +# 2.4.1 2.4 2005 PSF yes +# 2.4.2 2.4.1 2005 PSF yes +# 2.4.3 2.4.2 2006 PSF yes +# 2.4.4 2.4.3 2006 PSF yes +# 2.5 2.4 2006 PSF yes +# 2.5.1 2.5 2007 PSF yes +# 2.5.2 2.5.1 2008 PSF yes +# 2.5.3 2.5.2 2008 PSF yes +# 2.6 2.5 2008 PSF yes +# 2.6.1 2.6 2008 PSF yes +# 2.6.2 2.6.1 2009 PSF yes +# 2.6.3 2.6.2 2009 PSF yes +# 2.6.4 2.6.3 2009 PSF yes +# 2.6.5 2.6.4 2010 PSF yes +# 2.7 2.6 2010 PSF yes +# +# Footnotes: +# +# (1) GPL-compatible doesn't mean that we're distributing Python under +# the GPL. All Python licenses, unlike the GPL, let you distribute +# a modified version without making your changes open source. The +# GPL-compatible licenses make it possible to combine Python with +# other software that is released under the GPL; the others don't. +# +# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, +# because its license has a choice of law clause. According to +# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 +# is "not incompatible" with the GPL. +# +# Thanks to the many outside volunteers who have worked under Guido's +# direction to make these releases possible. +# +# +# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON +# =============================================================== +# +# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 +# -------------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Python Software Foundation +# ("PSF"), and the Individual or Organization ("Licensee") accessing and +# otherwise using this software ("Python") in source or binary form and +# its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, PSF hereby +# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, +# analyze, test, perform and/or display publicly, prepare derivative works, +# distribute, and otherwise use Python alone or in any derivative version, +# provided, however, that PSF's License Agreement and PSF's notice of copyright, +# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, +# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained +# in Python alone or in any derivative version prepared by Licensee. +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python. +# +# 4. PSF is making Python available to Licensee on an "AS IS" +# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. Nothing in this License Agreement shall be deemed to create any +# relationship of agency, partnership, or joint venture between PSF and +# Licensee. This License Agreement does not grant permission to use PSF +# trademarks or trade name in a trademark sense to endorse or promote +# products or services of Licensee, or any third party. +# +# 8. By copying, installing or otherwise using Python, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 +# ------------------------------------------- +# +# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 +# +# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an +# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the +# Individual or Organization ("Licensee") accessing and otherwise using +# this software in source or binary form and its associated +# documentation ("the Software"). +# +# 2. Subject to the terms and conditions of this BeOpen Python License +# Agreement, BeOpen hereby grants Licensee a non-exclusive, +# royalty-free, world-wide license to reproduce, analyze, test, perform +# and/or display publicly, prepare derivative works, distribute, and +# otherwise use the Software alone or in any derivative version, +# provided, however, that the BeOpen Python License is retained in the +# Software, alone or in any derivative version prepared by Licensee. +# +# 3. BeOpen is making the Software available to Licensee on an "AS IS" +# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE +# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS +# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY +# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 5. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 6. This License Agreement shall be governed by and interpreted in all +# respects by the law of the State of California, excluding conflict of +# law provisions. Nothing in this License Agreement shall be deemed to +# create any relationship of agency, partnership, or joint venture +# between BeOpen and Licensee. This License Agreement does not grant +# permission to use BeOpen trademarks or trade names in a trademark +# sense to endorse or promote products or services of Licensee, or any +# third party. As an exception, the "BeOpen Python" logos available at +# http://www.pythonlabs.com/logos.html may be used according to the +# permissions granted on that web page. +# +# 7. By copying, installing or otherwise using the software, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 +# --------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Corporation for National +# Research Initiatives, having an office at 1895 Preston White Drive, +# Reston, VA 20191 ("CNRI"), and the Individual or Organization +# ("Licensee") accessing and otherwise using Python 1.6.1 software in +# source or binary form and its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, CNRI +# hereby grants Licensee a nonexclusive, royalty-free, world-wide +# license to reproduce, analyze, test, perform and/or display publicly, +# prepare derivative works, distribute, and otherwise use Python 1.6.1 +# alone or in any derivative version, provided, however, that CNRI's +# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) +# 1995-2001 Corporation for National Research Initiatives; All Rights +# Reserved" are retained in Python 1.6.1 alone or in any derivative +# version prepared by Licensee. Alternately, in lieu of CNRI's License +# Agreement, Licensee may substitute the following text (omitting the +# quotes): "Python 1.6.1 is made available subject to the terms and +# conditions in CNRI's License Agreement. This Agreement together with +# Python 1.6.1 may be located on the Internet using the following +# unique, persistent identifier (known as a handle): 1895.22/1013. This +# Agreement may also be obtained from a proxy server on the Internet +# using the following URL: http://hdl.handle.net/1895.22/1013". +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python 1.6.1 or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python 1.6.1. +# +# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" +# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. This License Agreement shall be governed by the federal +# intellectual property law of the United States, including without +# limitation the federal copyright law, and, to the extent such +# U.S. federal law does not apply, by the law of the Commonwealth of +# Virginia, excluding Virginia's conflict of law provisions. +# Notwithstanding the foregoing, with regard to derivative works based +# on Python 1.6.1 that incorporate non-separable material that was +# previously distributed under the GNU General Public License (GPL), the +# law of the Commonwealth of Virginia shall govern this License +# Agreement only as to issues arising under or with respect to +# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this +# License Agreement shall be deemed to create any relationship of +# agency, partnership, or joint venture between CNRI and Licensee. This +# License Agreement does not grant permission to use CNRI trademarks or +# trade name in a trademark sense to endorse or promote products or +# services of Licensee, or any third party. +# +# 8. By clicking on the "ACCEPT" button where indicated, or by copying, +# installing or otherwise using Python 1.6.1, Licensee agrees to be +# bound by the terms and conditions of this License Agreement. +# +# ACCEPT +# +# +# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 +# -------------------------------------------------- +# +# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, +# The Netherlands. All rights reserved. +# +# Permission to use, copy, modify, and distribute this software and its +# documentation for any purpose and without fee is hereby granted, +# provided that the above copyright notice appear in all copies and that +# both that copyright notice and this permission notice appear in +# supporting documentation, and that the name of Stichting Mathematisch +# Centrum or CWI not be used in advertising or publicity pertaining to +# distribution of the software without specific, written prior +# permission. +# +# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO +# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND +# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE +# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES +# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN +# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT +# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. +"""Heap queue algorithm (a.k.a. priority queue). + +Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for +all k, counting elements from 0. For the sake of comparison, +non-existing elements are considered to be infinite. The interesting +property of a heap is that a[0] is always its smallest element. + +Usage: + +heap = [] # creates an empty heap +heappush(heap, item) # pushes a new item on the heap +item = heappop(heap) # pops the smallest item from the heap +item = heap[0] # smallest item on the heap without popping it +heapify(x) # transforms list into a heap, in-place, in linear time +item = heapreplace(heap, item) # pops and returns smallest item, and adds + # new item; the heap size is unchanged + +Our API differs from textbook heap algorithms as follows: + +- We use 0-based indexing. This makes the relationship between the + index for a node and the indexes for its children slightly less + obvious, but is more suitable since Python uses 0-based indexing. + +- Our heappop() method returns the smallest item, not the largest. + +These two make it possible to view the heap as a regular Python list +without surprises: heap[0] is the smallest item, and heap.sort() +maintains the heap invariant! +""" + +# Original code by Kevin O'Connor, augmented by Tim Peters and Raymond Hettinger + +__about__ = """Heap queues + +[explanation by François Pinard] + +Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for +all k, counting elements from 0. For the sake of comparison, +non-existing elements are considered to be infinite. The interesting +property of a heap is that a[0] is always its smallest element. + +The strange invariant above is meant to be an efficient memory +representation for a tournament. The numbers below are `k', not a[k]: + + 0 + + 1 2 + + 3 4 5 6 + + 7 8 9 10 11 12 13 14 + + 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 + + +In the tree above, each cell `k' is topping `2*k+1' and `2*k+2'. In +an usual binary tournament we see in sports, each cell is the winner +over the two cells it tops, and we can trace the winner down the tree +to see all opponents s/he had. However, in many computer applications +of such tournaments, we do not need to trace the history of a winner. +To be more memory efficient, when a winner is promoted, we try to +replace it by something else at a lower level, and the rule becomes +that a cell and the two cells it tops contain three different items, +but the top cell "wins" over the two topped cells. + +If this heap invariant is protected at all time, index 0 is clearly +the overall winner. The simplest algorithmic way to remove it and +find the "next" winner is to move some loser (let's say cell 30 in the +diagram above) into the 0 position, and then percolate this new 0 down +the tree, exchanging values, until the invariant is re-established. +This is clearly logarithmic on the total number of items in the tree. +By iterating over all items, you get an O(n ln n) sort. + +A nice feature of this sort is that you can efficiently insert new +items while the sort is going on, provided that the inserted items are +not "better" than the last 0'th element you extracted. This is +especially useful in simulation contexts, where the tree holds all +incoming events, and the "win" condition means the smallest scheduled +time. When an event schedule other events for execution, they are +scheduled into the future, so they can easily go into the heap. So, a +heap is a good structure for implementing schedulers (this is what I +used for my MIDI sequencer :-). + +Various structures for implementing schedulers have been extensively +studied, and heaps are good for this, as they are reasonably speedy, +the speed is almost constant, and the worst case is not much different +than the average case. However, there are other representations which +are more efficient overall, yet the worst cases might be terrible. + +Heaps are also very useful in big disk sorts. You most probably all +know that a big sort implies producing "runs" (which are pre-sorted +sequences, which size is usually related to the amount of CPU memory), +followed by a merging passes for these runs, which merging is often +very cleverly organised[1]. It is very important that the initial +sort produces the longest runs possible. Tournaments are a good way +to that. If, using all the memory available to hold a tournament, you +replace and percolate items that happen to fit the current run, you'll +produce runs which are twice the size of the memory for random input, +and much better for input fuzzily ordered. + +Moreover, if you output the 0'th item on disk and get an input which +may not fit in the current tournament (because the value "wins" over +the last output value), it cannot fit in the heap, so the size of the +heap decreases. The freed memory could be cleverly reused immediately +for progressively building a second heap, which grows at exactly the +same rate the first heap is melting. When the first heap completely +vanishes, you switch heaps and start a new run. Clever and quite +effective! + +In a word, heaps are useful memory structures to know. I use them in +a few applications, and I think it is good to keep a `heap' module +around. :-) + +-------------------- +[1] The disk balancing algorithms which are current, nowadays, are +more annoying than clever, and this is a consequence of the seeking +capabilities of the disks. On devices which cannot seek, like big +tape drives, the story was quite different, and one had to be very +clever to ensure (far in advance) that each tape movement will be the +most effective possible (that is, will best participate at +"progressing" the merge). Some tapes were even able to read +backwards, and this was also used to avoid the rewinding time. +Believe me, real good tape sorts were quite spectacular to watch! +From all times, sorting has always been a Great Art! :-) +""" + +__all__ = ['heappush', 'heappop', 'heapify', 'heapreplace', 'merge', + 'nlargest', 'nsmallest', 'heappushpop'] + +def heappush(heap, item): + """Push item onto heap, maintaining the heap invariant.""" + heap.append(item) + _siftdown(heap, 0, len(heap)-1) + +def heappop(heap): + """Pop the smallest item off the heap, maintaining the heap invariant.""" + lastelt = heap.pop() # raises appropriate IndexError if heap is empty + if heap: + returnitem = heap[0] + heap[0] = lastelt + _siftup(heap, 0) + return returnitem + return lastelt + +def heapreplace(heap, item): + """Pop and return the current smallest value, and add the new item. + + This is more efficient than heappop() followed by heappush(), and can be + more appropriate when using a fixed-size heap. Note that the value + returned may be larger than item! That constrains reasonable uses of + this routine unless written as part of a conditional replacement: + + if item > heap[0]: + item = heapreplace(heap, item) + """ + returnitem = heap[0] # raises appropriate IndexError if heap is empty + heap[0] = item + _siftup(heap, 0) + return returnitem + +def heappushpop(heap, item): + """Fast version of a heappush followed by a heappop.""" + if heap and heap[0] < item: + item, heap[0] = heap[0], item + _siftup(heap, 0) + return item + +def heapify(x): + """Transform list into a heap, in-place, in O(len(x)) time.""" + n = len(x) + # Transform bottom-up. The largest index there's any point to looking at + # is the largest with a child index in-range, so must have 2*i + 1 < n, + # or i < (n-1)/2. If n is even = 2*j, this is (2*j-1)/2 = j-1/2 so + # j-1 is the largest, which is n//2 - 1. If n is odd = 2*j+1, this is + # (2*j+1-1)/2 = j so j-1 is the largest, and that's again n//2-1. + for i in reversed(range(n//2)): + _siftup(x, i) + +def _heappop_max(heap): + """Maxheap version of a heappop.""" + lastelt = heap.pop() # raises appropriate IndexError if heap is empty + if heap: + returnitem = heap[0] + heap[0] = lastelt + _siftup_max(heap, 0) + return returnitem + return lastelt + +def _heapreplace_max(heap, item): + """Maxheap version of a heappop followed by a heappush.""" + returnitem = heap[0] # raises appropriate IndexError if heap is empty + heap[0] = item + _siftup_max(heap, 0) + return returnitem + +def _heapify_max(x): + """Transform list into a maxheap, in-place, in O(len(x)) time.""" + n = len(x) + for i in reversed(range(n//2)): + _siftup_max(x, i) + +# 'heap' is a heap at all indices >= startpos, except possibly for pos. pos +# is the index of a leaf with a possibly out-of-order value. Restore the +# heap invariant. +def _siftdown(heap, startpos, pos): + newitem = heap[pos] + # Follow the path to the root, moving parents down until finding a place + # newitem fits. + while pos > startpos: + parentpos = (pos - 1) >> 1 + parent = heap[parentpos] + if newitem < parent: + heap[pos] = parent + pos = parentpos + continue + break + heap[pos] = newitem + +# The child indices of heap index pos are already heaps, and we want to make +# a heap at index pos too. We do this by bubbling the smaller child of +# pos up (and so on with that child's children, etc) until hitting a leaf, +# then using _siftdown to move the oddball originally at index pos into place. +# +# We *could* break out of the loop as soon as we find a pos where newitem <= +# both its children, but turns out that's not a good idea, and despite that +# many books write the algorithm that way. During a heap pop, the last array +# element is sifted in, and that tends to be large, so that comparing it +# against values starting from the root usually doesn't pay (= usually doesn't +# get us out of the loop early). See Knuth, Volume 3, where this is +# explained and quantified in an exercise. +# +# Cutting the # of comparisons is important, since these routines have no +# way to extract "the priority" from an array element, so that intelligence +# is likely to be hiding in custom comparison methods, or in array elements +# storing (priority, record) tuples. Comparisons are thus potentially +# expensive. +# +# On random arrays of length 1000, making this change cut the number of +# comparisons made by heapify() a little, and those made by exhaustive +# heappop() a lot, in accord with theory. Here are typical results from 3 +# runs (3 just to demonstrate how small the variance is): +# +# Compares needed by heapify Compares needed by 1000 heappops +# -------------------------- -------------------------------- +# 1837 cut to 1663 14996 cut to 8680 +# 1855 cut to 1659 14966 cut to 8678 +# 1847 cut to 1660 15024 cut to 8703 +# +# Building the heap by using heappush() 1000 times instead required +# 2198, 2148, and 2219 compares: heapify() is more efficient, when +# you can use it. +# +# The total compares needed by list.sort() on the same lists were 8627, +# 8627, and 8632 (this should be compared to the sum of heapify() and +# heappop() compares): list.sort() is (unsurprisingly!) more efficient +# for sorting. + +def _siftup(heap, pos): + endpos = len(heap) + startpos = pos + newitem = heap[pos] + # Bubble up the smaller child until hitting a leaf. + childpos = 2*pos + 1 # leftmost child position + while childpos < endpos: + # Set childpos to index of smaller child. + rightpos = childpos + 1 + if rightpos < endpos and not heap[childpos] < heap[rightpos]: + childpos = rightpos + # Move the smaller child up. + heap[pos] = heap[childpos] + pos = childpos + childpos = 2*pos + 1 + # The leaf at pos is empty now. Put newitem there, and bubble it up + # to its final resting place (by sifting its parents down). + heap[pos] = newitem + _siftdown(heap, startpos, pos) + +def _siftdown_max(heap, startpos, pos): + 'Maxheap variant of _siftdown' + newitem = heap[pos] + # Follow the path to the root, moving parents down until finding a place + # newitem fits. + while pos > startpos: + parentpos = (pos - 1) >> 1 + parent = heap[parentpos] + if parent < newitem: + heap[pos] = parent + pos = parentpos + continue + break + heap[pos] = newitem + +def _siftup_max(heap, pos): + 'Maxheap variant of _siftup' + endpos = len(heap) + startpos = pos + newitem = heap[pos] + # Bubble up the larger child until hitting a leaf. + childpos = 2*pos + 1 # leftmost child position + while childpos < endpos: + # Set childpos to index of larger child. + rightpos = childpos + 1 + if rightpos < endpos and not heap[rightpos] < heap[childpos]: + childpos = rightpos + # Move the larger child up. + heap[pos] = heap[childpos] + pos = childpos + childpos = 2*pos + 1 + # The leaf at pos is empty now. Put newitem there, and bubble it up + # to its final resting place (by sifting its parents down). + heap[pos] = newitem + _siftdown_max(heap, startpos, pos) + +def merge(iterables, key=None, reverse=False): + '''Merge multiple sorted inputs into a single sorted output. + + Similar to sorted(itertools.chain(*iterables)) but returns a generator, + does not pull the data into memory all at once, and assumes that each of + the input streams is already sorted (smallest to largest). + + >>> list(merge([1,3,5,7], [0,2,4,8], [5,10,15,20], [], [25])) + [0, 1, 2, 3, 4, 5, 5, 7, 8, 10, 15, 20, 25] + + If *key* is not None, applies a key function to each element to determine + its sort order. + + >>> list(merge(['dog', 'horse'], ['cat', 'fish', 'kangaroo'], key=len)) + ['dog', 'cat', 'fish', 'horse', 'kangaroo'] + + ''' + + h = [] + h_append = h.append + + if reverse: + _heapify = _heapify_max + _heappop = _heappop_max + _heapreplace = _heapreplace_max + direction = -1 + else: + _heapify = heapify + _heappop = heappop + _heapreplace = heapreplace + direction = 1 + + if key is None: + for order, it in enumerate(map(iter, iterables)): + try: + next = it.next + h_append([next(), order * direction, next]) + except StopIteration: + pass + _heapify(h) + while len(h) > 1: + try: + while True: + value, order, next = s = h[0] + yield value + s[0] = next() # raises StopIteration when exhausted + _heapreplace(h, s) # restore heap condition + except StopIteration: + _heappop(h) # remove empty iterator + if h: + # fast case when only a single iterator remains + value, order, next = h[0] + yield value + for value in next.__self__: + yield value + return + + for order, it in enumerate(map(iter, iterables)): + try: + next = it.next + value = next() + h_append([key(value), order * direction, value, next]) + except StopIteration: + pass + _heapify(h) + while len(h) > 1: + try: + while True: + key_value, order, value, next = s = h[0] + yield value + value = next() + s[0] = key(value) + s[2] = value + _heapreplace(h, s) + except StopIteration: + _heappop(h) + if h: + key_value, order, value, next = h[0] + yield value + for value in next.__self__: + yield value + + +# Algorithm notes for nlargest() and nsmallest() +# ============================================== +# +# Make a single pass over the data while keeping the k most extreme values +# in a heap. Memory consumption is limited to keeping k values in a list. +# +# Measured performance for random inputs: +# +# number of comparisons +# n inputs k-extreme values (average of 5 trials) % more than min() +# ------------- ---------------- --------------------- ----------------- +# 1,000 100 3,317 231.7% +# 10,000 100 14,046 40.5% +# 100,000 100 105,749 5.7% +# 1,000,000 100 1,007,751 0.8% +# 10,000,000 100 10,009,401 0.1% +# +# Theoretical number of comparisons for k smallest of n random inputs: +# +# Step Comparisons Action +# ---- -------------------------- --------------------------- +# 1 1.66 * k heapify the first k-inputs +# 2 n - k compare remaining elements to top of heap +# 3 k * (1 + lg2(k)) * ln(n/k) replace the topmost value on the heap +# 4 k * lg2(k) - (k/2) final sort of the k most extreme values +# +# Combining and simplifying for a rough estimate gives: +# +# comparisons = n + k * (log(k, 2) * log(n/k) + log(k, 2) + log(n/k)) +# +# Computing the number of comparisons for step 3: +# ----------------------------------------------- +# * For the i-th new value from the iterable, the probability of being in the +# k most extreme values is k/i. For example, the probability of the 101st +# value seen being in the 100 most extreme values is 100/101. +# * If the value is a new extreme value, the cost of inserting it into the +# heap is 1 + log(k, 2). +# * The probabilty times the cost gives: +# (k/i) * (1 + log(k, 2)) +# * Summing across the remaining n-k elements gives: +# sum((k/i) * (1 + log(k, 2)) for i in range(k+1, n+1)) +# * This reduces to: +# (H(n) - H(k)) * k * (1 + log(k, 2)) +# * Where H(n) is the n-th harmonic number estimated by: +# gamma = 0.5772156649 +# H(n) = log(n, e) + gamma + 1 / (2 * n) +# http://en.wikipedia.org/wiki/Harmonic_series_(mathematics)#Rate_of_divergence +# * Substituting the H(n) formula: +# comparisons = k * (1 + log(k, 2)) * (log(n/k, e) + (1/n - 1/k) / 2) +# +# Worst-case for step 3: +# ---------------------- +# In the worst case, the input data is reversed sorted so that every new element +# must be inserted in the heap: +# +# comparisons = 1.66 * k + log(k, 2) * (n - k) +# +# Alternative Algorithms +# ---------------------- +# Other algorithms were not used because they: +# 1) Took much more auxiliary memory, +# 2) Made multiple passes over the data. +# 3) Made more comparisons in common cases (small k, large n, semi-random input). +# See the more detailed comparison of approach at: +# http://code.activestate.com/recipes/577573-compare-algorithms-for-heapqsmallest + +def nsmallest(n, iterable, key=None): + """Find the n smallest elements in a dataset. + + Equivalent to: sorted(iterable, key=key)[:n] + """ + + # Short-cut for n==1 is to use min() + if n == 1: + it = iter(iterable) + sentinel = object() + if key is None: + result = min(it, default=sentinel) + else: + result = min(it, default=sentinel, key=key) + return [] if result is sentinel else [result] + + # When n>=size, it's faster to use sorted() + try: + size = len(iterable) + except (TypeError, AttributeError): + pass + else: + if n >= size: + return sorted(iterable, key=key)[:n] + + # When key is none, use simpler decoration + if key is None: + it = iter(iterable) + # put the range(n) first so that zip() doesn't + # consume one too many elements from the iterator + result = [(elem, i) for i, elem in zip(range(n), it)] + if not result: + return result + _heapify_max(result) + top = result[0][0] + order = n + _heapreplace = _heapreplace_max + for elem in it: + if elem < top: + _heapreplace(result, (elem, order)) + top = result[0][0] + order += 1 + result.sort() + return [r[0] for r in result] + + # General case, slowest method + it = iter(iterable) + result = [(key(elem), i, elem) for i, elem in zip(range(n), it)] + if not result: + return result + _heapify_max(result) + top = result[0][0] + order = n + _heapreplace = _heapreplace_max + for elem in it: + k = key(elem) + if k < top: + _heapreplace(result, (k, order, elem)) + top = result[0][0] + order += 1 + result.sort() + return [r[2] for r in result] + +def nlargest(n, iterable, key=None): + """Find the n largest elements in a dataset. + + Equivalent to: sorted(iterable, key=key, reverse=True)[:n] + """ + + # Short-cut for n==1 is to use max() + if n == 1: + it = iter(iterable) + sentinel = object() + if key is None: + result = max(it, default=sentinel) + else: + result = max(it, default=sentinel, key=key) + return [] if result is sentinel else [result] + + # When n>=size, it's faster to use sorted() + try: + size = len(iterable) + except (TypeError, AttributeError): + pass + else: + if n >= size: + return sorted(iterable, key=key, reverse=True)[:n] + + # When key is none, use simpler decoration + if key is None: + it = iter(iterable) + result = [(elem, i) for i, elem in zip(range(0, -n, -1), it)] + if not result: + return result + heapify(result) + top = result[0][0] + order = -n + _heapreplace = heapreplace + for elem in it: + if top < elem: + _heapreplace(result, (elem, order)) + top = result[0][0] + order -= 1 + result.sort(reverse=True) + return [r[0] for r in result] + + # General case, slowest method + it = iter(iterable) + result = [(key(elem), i, elem) for i, elem in zip(range(0, -n, -1), it)] + if not result: + return result + heapify(result) + top = result[0][0] + order = -n + _heapreplace = heapreplace + for elem in it: + k = key(elem) + if top < k: + _heapreplace(result, (k, order, elem)) + top = result[0][0] + order -= 1 + result.sort(reverse=True) + return [r[2] for r in result] + +# If available, use C implementation +try: + from _heapq import * +except ImportError: + pass +try: + from _heapq import _heapreplace_max +except ImportError: + pass +try: + from _heapq import _heapify_max +except ImportError: + pass +try: + from _heapq import _heappop_max +except ImportError: + pass + + +if __name__ == "__main__": + + import doctest + print(doctest.testmod()) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3a2e7649e6827..31919741e9d73 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -44,7 +44,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ - get_used_memory + get_used_memory, ExternalSorter from py4j.java_collections import ListConverter, MapConverter @@ -605,8 +605,13 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() + spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == 'true') + memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + serializer = self._jrdd_deserializer + def sortPartition(iterator): - return iter(sorted(iterator, key=lambda (k, v): keyfunc(k), reverse=not ascending)) + sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted + return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) if numPartitions == 1: if self.getNumPartitions() > 1: diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 1ebe7df418327..49829f5280a5f 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -21,7 +21,10 @@ import shutil import warnings import gc +import itertools +import random +import pyspark.heapq3 as heapq from pyspark.serializers import BatchedSerializer, PickleSerializer try: @@ -54,6 +57,17 @@ def get_used_memory(): return 0 +def _get_local_dirs(sub): + """ Get all the directories """ + path = os.environ.get("SPARK_LOCAL_DIRS", "/tmp") + dirs = path.split(",") + if len(dirs) > 1: + # different order in different processes and instances + rnd = random.Random(os.getpid() + id(dirs)) + random.shuffle(dirs, rnd.random) + return [os.path.join(d, "python", str(os.getpid()), sub) for d in dirs] + + class Aggregator(object): """ @@ -196,7 +210,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, # default serializer is only used for tests self.serializer = serializer or \ BatchedSerializer(PickleSerializer(), 1024) - self.localdirs = localdirs or self._get_dirs() + self.localdirs = localdirs or _get_local_dirs(str(id(self))) # number of partitions when spill data into disks self.partitions = partitions # check the memory after # of items merged @@ -212,13 +226,6 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, # randomize the hash of key, id(o) is the address of o (aligned by 8) self._seed = id(self) + 7 - def _get_dirs(self): - """ Get all the directories """ - path = os.environ.get("SPARK_LOCAL_DIRS", "/tmp") - dirs = path.split(",") - return [os.path.join(d, "python", str(os.getpid()), str(id(self))) - for d in dirs] - def _get_spill_dir(self, n): """ Choose one directory for spill by number n """ return os.path.join(self.localdirs[n % len(self.localdirs)], str(n)) @@ -434,6 +441,74 @@ def _recursive_merged_items(self, start): os.remove(os.path.join(path, str(i))) +class ExternalSorter(object): + """ + ExtenalSorter will divide the elements into chunks, sort them in + memory and dump them into disks, finally merge them back. + + The spilling will only happen when the used memory goes above + the limit. + + >>> sorter = ExternalSorter(1) # 1M + >>> import random + >>> l = range(1024) + >>> random.shuffle(l) + >>> sorted(l) == list(sorter.sorted(l)) + True + >>> sorted(l) == list(sorter.sorted(l, key=lambda x: -x, reverse=True)) + True + """ + def __init__(self, memory_limit, serializer=None): + self.memory_limit = memory_limit + self.local_dirs = _get_local_dirs("sort") + self.serializer = serializer or BatchedSerializer(PickleSerializer(), 1024) + self._spilled_bytes = 0 + + def _get_path(self, n): + """ Choose one directory for spill by number n """ + d = self.local_dirs[n % len(self.local_dirs)] + if not os.path.exists(d): + os.makedirs(d) + return os.path.join(d, str(n)) + + def sorted(self, iterator, key=None, reverse=False): + """ + Sort the elements in iterator, do external sort when the memory + goes above the limit. + """ + batch = 10 + chunks, current_chunk = [], [] + iterator = iter(iterator) + while True: + # pick elements in batch + chunk = list(itertools.islice(iterator, batch)) + current_chunk.extend(chunk) + if len(chunk) < batch: + break + + if get_used_memory() > self.memory_limit: + # sort them inplace will save memory + current_chunk.sort(key=key, reverse=reverse) + path = self._get_path(len(chunks)) + with open(path, 'w') as f: + self.serializer.dump_stream(current_chunk, f) + self._spilled_bytes += os.path.getsize(path) + chunks.append(self.serializer.load_stream(open(path))) + current_chunk = [] + + elif not chunks: + batch = min(batch * 2, 10000) + + current_chunk.sort(key=key, reverse=reverse) + if not chunks: + return current_chunk + + if current_chunk: + chunks.append(iter(current_chunk)) + + return heapq.merge(chunks, key=key, reverse=reverse) + + if __name__ == "__main__": import doctest doctest.testmod() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1db922f513743..3e7040eade1ab 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -30,6 +30,7 @@ import tempfile import time import zipfile +import random if sys.version_info[:2] <= (2, 6): import unittest2 as unittest @@ -37,10 +38,11 @@ import unittest +from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer -from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger +from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter _have_scipy = False _have_numpy = False @@ -117,6 +119,44 @@ def test_huge_dataset(self): m._cleanup() +class TestSorter(unittest.TestCase): + def test_in_memory_sort(self): + l = range(1024) + random.shuffle(l) + sorter = ExternalSorter(1024) + self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + + def test_external_sort(self): + l = range(1024) + random.shuffle(l) + sorter = ExternalSorter(1) + self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertGreater(sorter._spilled_bytes, 0) + last = sorter._spilled_bytes + self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertGreater(sorter._spilled_bytes, last) + last = sorter._spilled_bytes + self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertGreater(sorter._spilled_bytes, last) + last = sorter._spilled_bytes + self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertGreater(sorter._spilled_bytes, last) + + def test_external_sort_in_rdd(self): + conf = SparkConf().set("spark.python.worker.memory", "1m") + sc = SparkContext(conf=conf) + l = range(10240) + random.shuffle(l) + rdd = sc.parallelize(l, 10) + self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect()) + sc.stop() + + class SerializationTestCase(unittest.TestCase): def test_namedtuple(self): diff --git a/tox.ini b/tox.ini index a1fefdd0e176f..b568029a204cc 100644 --- a/tox.ini +++ b/tox.ini @@ -15,4 +15,4 @@ [pep8] max-line-length=100 -exclude=cloudpickle.py +exclude=cloudpickle.py,heapq3.py From 2ffd3290fe30c23df8da1efe153b84c23eb2e1cd Mon Sep 17 00:00:00 2001 From: WangTao Date: Tue, 26 Aug 2014 17:30:59 -0700 Subject: [PATCH 0485/1492] [SPARK-3225]Typo in script use_conf_dir => user_conf_dir in load-spark-env.sh. Author: WangTao Closes #1926 from WangTaoTheTonic/TypoInScript and squashes the following commits: 0c104ad [WangTao] Typo in script --- bin/load-spark-env.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index d425f9feaac54..493d3785a081b 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -27,12 +27,12 @@ if [ -z "$SPARK_ENV_LOADED" ]; then # Returns the parent of the directory this script lives in. parent_dir="$(cd `dirname $0`/..; pwd)" - use_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} + user_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} - if [ -f "${use_conf_dir}/spark-env.sh" ]; then + if [ -f "${user_conf_dir}/spark-env.sh" ]; then # Promote all variable declarations to environment (exported) variables set -a - . "${use_conf_dir}/spark-env.sh" + . "${user_conf_dir}/spark-env.sh" set +a fi fi From faeb9c0e1440f4af888be0dfc5de7b57efc92b00 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 26 Aug 2014 17:33:40 -0700 Subject: [PATCH 0486/1492] [SPARK-2964] [SQL] Remove duplicated code from spark-sql and start-thriftserver.sh Author: Cheng Lian Author: Kousuke Saruta Closes #1886 from sarutak/SPARK-2964 and squashes the following commits: 8ef8751 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2964 26e7c95 [Kousuke Saruta] Revert "Shorten timeout to more reasonable value" ffb68fa [Kousuke Saruta] Modified spark-sql and start-thriftserver.sh to use bin/utils.sh 8c6f658 [Kousuke Saruta] Merge branch 'spark-3026' of https://github.com/liancheng/spark into SPARK-2964 81b43a8 [Cheng Lian] Shorten timeout to more reasonable value a89e66d [Cheng Lian] Fixed command line options quotation in scripts 9c894d3 [Cheng Lian] Fixed bin/spark-sql -S option typo be4736b [Cheng Lian] Report better error message when running JDBC/CLI without hive-thriftserver profile enabled --- bin/spark-sql | 55 +++++-------------- .../org/apache/spark/deploy/SparkSubmit.scala | 14 ++++- sbin/start-thriftserver.sh | 39 +++++-------- 3 files changed, 39 insertions(+), 69 deletions(-) diff --git a/bin/spark-sql b/bin/spark-sql index 564f1f419060f..2a3cb31f58e8d 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,6 +24,7 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" @@ -43,52 +44,22 @@ function usage { $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } -function ensure_arg_number { - arg_number=$1 - at_least=$2 - - if [[ $arg_number -lt $at_least ]]; then - usage - exit 1 - fi -} - -if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage exit 0 fi -CLI_ARGS=() -SUBMISSION_ARGS=() - -while (($#)); do - case $1 in - -d | --define | --database | -f | -h | --hiveconf | --hivevar | -i | -p) - ensure_arg_number $# 2 - CLI_ARGS+=("$1"); shift - CLI_ARGS+=("$1"); shift - ;; +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" - -e) - ensure_arg_number $# 2 - CLI_ARGS+=("$1"); shift - CLI_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - -s | --silent) - CLI_ARGS+=("$1"); shift - ;; - - -v | --verbose) - # Both SparkSubmit and SparkSQLCLIDriver recognizes -v | --verbose - CLI_ARGS+=("$1") - SUBMISSION_ARGS+=("$1"); shift - ;; - - *) - SUBMISSION_ARGS+=("$1"); shift - ;; - esac -done +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Spark SQL CLI main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${CLI_ARGS[@]}" +exit $exit_status diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index f8cdbc3c392b5..550ee72538900 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -54,6 +54,8 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + private val CLASS_NOT_FOUND_EXIT_STATUS = 1 + // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(-1) private[spark] var printStream: PrintStream = System.err @@ -311,8 +313,18 @@ object SparkSubmit { System.setProperty(key, value) } - val mainClass = Class.forName(childMainClass, true, loader) + var mainClass: Class[_] = null + + try { + mainClass = Class.forName(childMainClass, true, loader) + } catch { + case e: ClassNotFoundException => + e.printStackTrace(printStream) + System.exit(CLASS_NOT_FOUND_EXIT_STATUS) + } + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) + try { mainMethod.invoke(null, childArgs.toArray) } catch { diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 2c4452473ccbc..c519a77df4a14 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,6 +27,7 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" +CLASS_NOT_FOUND_EXIT_STATUS=1 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" @@ -43,36 +44,22 @@ function usage { $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } -function ensure_arg_number { - arg_number=$1 - at_least=$2 - - if [[ $arg_number -lt $at_least ]]; then - usage - exit 1 - fi -} - -if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage exit 0 fi -THRIFT_SERVER_ARGS=() -SUBMISSION_ARGS=() +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" -while (($#)); do - case $1 in - --hiveconf) - ensure_arg_number $# 2 - THRIFT_SERVER_ARGS+=("$1"); shift - THRIFT_SERVER_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - *) - SUBMISSION_ARGS+=("$1"); shift - ;; - esac -done +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Hive Thrift server main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${THRIFT_SERVER_ARGS[@]}" +exit $exit_status From 73b3089b8d2901dab11bb1ef6f46c29625b677fe Mon Sep 17 00:00:00 2001 From: nchammas Date: Tue, 26 Aug 2014 17:50:04 -0700 Subject: [PATCH 0487/1492] [Docs] Run tests like in contributing guide The Contributing to Spark guide [recommends](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting) running tests by calling `./dev/run-tests`. The README should, too. `./sbt/sbt test` does not cover Python tests or style tests. Author: nchammas Closes #2149 from nchammas/patch-2 and squashes the following commits: 2b3b132 [nchammas] [Docs] Run tests like in contributing guide --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index f75226ce11335..0a683a460ffac 100644 --- a/README.md +++ b/README.md @@ -69,7 +69,7 @@ Many of the example programs print usage help if no params are given. Testing first requires [building Spark](#building-spark). Once Spark is built, tests can be run using: - ./sbt/sbt test + ./dev/run-tests ## A Note About Hadoop Versions From 727cb25bcc29481d6b744abef1ca091e64b5f91f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 18:28:41 -0700 Subject: [PATCH 0488/1492] [SPARK-3036][SPARK-3037][SQL] Add MapType/ArrayType containing null value support to Parquet. JIRA: - https://issues.apache.org/jira/browse/SPARK-3036 - https://issues.apache.org/jira/browse/SPARK-3037 Currently this uses the following Parquet schema for `MapType` when `valueContainsNull` is `true`: ``` message root { optional group a (MAP) { repeated group map (MAP_KEY_VALUE) { required int32 key; optional int32 value; } } } ``` for `ArrayType` when `containsNull` is `true`: ``` message root { optional group a (LIST) { repeated group bag { optional int32 array; } } } ``` We have to think about compatibilities with older version of Spark or Hive or others I mentioned in the JIRA issues. Notice: This PR is based on #1963 and #1889. Please check them first. /cc marmbrus, yhuai Author: Takuya UESHIN Closes #2032 from ueshin/issues/SPARK-3036_3037 and squashes the following commits: 4e8e9e7 [Takuya UESHIN] Add ArrayType containing null value support to Parquet. 013c2ca [Takuya UESHIN] Add MapType containing null value support to Parquet. 62989de [Takuya UESHIN] Merge branch 'issues/SPARK-2969' into issues/SPARK-3036_3037 8e38b53 [Takuya UESHIN] Merge branch 'issues/SPARK-3063' into issues/SPARK-3036_3037 --- .../spark/sql/parquet/ParquetConverter.scala | 83 +++++++++++++++++++ .../sql/parquet/ParquetTableSupport.scala | 54 +++++++----- .../spark/sql/parquet/ParquetTypes.scala | 54 ++++++++---- .../spark/sql/parquet/ParquetQuerySuite.scala | 16 +++- 4 files changed, 167 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index ef4526ec03439..9fd6aed402838 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -58,6 +58,7 @@ private[sql] object CatalystConverter { // This is mostly Parquet convention (see, e.g., `ConversionPatterns`). // Note that "array" for the array elements is chosen by ParquetAvro. // Using a different value will result in Parquet silently dropping columns. + val ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME = "bag" val ARRAY_ELEMENTS_SCHEMA_NAME = "array" val MAP_KEY_SCHEMA_NAME = "key" val MAP_VALUE_SCHEMA_NAME = "value" @@ -82,6 +83,9 @@ private[sql] object CatalystConverter { case ArrayType(elementType: DataType, false) => { new CatalystArrayConverter(elementType, fieldIndex, parent) } + case ArrayType(elementType: DataType, true) => { + new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent) + } case StructType(fields: Seq[StructField]) => { new CatalystStructConverter(fields.toArray, fieldIndex, parent) } @@ -567,6 +571,85 @@ private[parquet] class CatalystNativeArrayConverter( } } +/** + * A `parquet.io.api.GroupConverter` that converts a single-element groups that + * match the characteristics of an array contains null (see + * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an + * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * + * @param elementType The type of the array elements (complex or primitive) + * @param index The position of this (array) field inside its parent converter + * @param parent The parent converter + * @param buffer A data buffer + */ +private[parquet] class CatalystArrayContainsNullConverter( + val elementType: DataType, + val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] var buffer: Buffer[Any]) + extends CatalystConverter { + + def this(elementType: DataType, index: Int, parent: CatalystConverter) = + this( + elementType, + index, + parent, + new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) + + protected[parquet] val converter: Converter = new CatalystConverter { + + private var current: Any = null + + val converter = CatalystConverter.createConverter( + new CatalystConverter.FieldType( + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + elementType, + false), + fieldIndex = 0, + parent = this) + + override def getConverter(fieldIndex: Int): Converter = converter + + override def end(): Unit = parent.updateField(index, current) + + override def start(): Unit = { + current = null + } + + override protected[parquet] val size: Int = 1 + override protected[parquet] val index: Int = 0 + override protected[parquet] val parent = CatalystArrayContainsNullConverter.this + + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + current = value + } + + override protected[parquet] def clearBuffer(): Unit = {} + } + + override def getConverter(fieldIndex: Int): Converter = converter + + // arrays have only one (repeated) field, which is its elements + override val size = 1 + + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + buffer += value + } + + override protected[parquet] def clearBuffer(): Unit = { + buffer.clear() + } + + override def start(): Unit = {} + + override def end(): Unit = { + assert(parent != null) + // here we need to make sure to use ArrayScalaType + parent.updateField(index, buffer.toArray.toSeq) + clearBuffer() + } +} + /** * This converter is for multi-element groups of primitive or complex types * that have repetition level optional or required (so struct fields). diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 6a657c20fe46c..bdf02401b21be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -173,7 +173,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeValue(schema: DataType, value: Any): Unit = { if (value != null) { schema match { - case t @ ArrayType(_, false) => writeArray( + case t @ ArrayType(_, _) => writeArray( t, value.asInstanceOf[CatalystConverter.ArrayScalaType[_]]) case t @ MapType(_, _, _) => writeMap( @@ -228,45 +228,57 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } } - // TODO: support null values, see - // https://issues.apache.org/jira/browse/SPARK-1649 private[parquet] def writeArray( schema: ArrayType, array: CatalystConverter.ArrayScalaType[_]): Unit = { val elementType = schema.elementType writer.startGroup() if (array.size > 0) { - writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) - var i = 0 - while(i < array.size) { - writeValue(elementType, array(i)) - i = i + 1 + if (schema.containsNull) { + writer.startField(CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, 0) + var i = 0 + while (i < array.size) { + writer.startGroup() + if (array(i) != null) { + writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + writeValue(elementType, array(i)) + writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + } + writer.endGroup() + i = i + 1 + } + writer.endField(CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, 0) + } else { + writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + var i = 0 + while (i < array.size) { + writeValue(elementType, array(i)) + i = i + 1 + } + writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) } - writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) } writer.endGroup() } - // TODO: support null values, see - // https://issues.apache.org/jira/browse/SPARK-1649 private[parquet] def writeMap( schema: MapType, map: CatalystConverter.MapScalaType[_, _]): Unit = { writer.startGroup() if (map.size > 0) { writer.startField(CatalystConverter.MAP_SCHEMA_NAME, 0) - writer.startGroup() - writer.startField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) - for(key <- map.keys) { + for ((key, value) <- map) { + writer.startGroup() + writer.startField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) writeValue(schema.keyType, key) + writer.endField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) + if (value != null) { + writer.startField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) + writeValue(schema.valueType, value) + writer.endField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) + } + writer.endGroup() } - writer.endField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) - writer.startField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) - for(value <- map.values) { - writeValue(schema.valueType, value) - } - writer.endField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) - writer.endGroup() writer.endField(CatalystConverter.MAP_SCHEMA_NAME, 0) } writer.endGroup() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index af8cd0a73b674..1a52377651737 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -119,7 +119,13 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetOriginalType.LIST => { // TODO: check enums! assert(groupType.getFieldCount == 1) val field = groupType.getFields.apply(0) - ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { + val bag = field.asGroupType() + assert(bag.getFieldCount == 1) + ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + } else { + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + } } case ParquetOriginalType.MAP => { assert( @@ -129,28 +135,32 @@ private[parquet] object ParquetTypesConverter extends Logging { assert( keyValueGroup.getFieldCount == 2, "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) + + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) - assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true - // at here. - MapType(keyType, valueType) + MapType(keyType, valueType, + keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } case _ => { // Note: the order of these checks is important! if (correspondsToMap(groupType)) { // MapType val keyValueGroup = groupType.getFields.apply(0).asGroupType() - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) + + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) - assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true - // at here. - MapType(keyType, valueType) + MapType(keyType, valueType, + keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } else if (correspondsToArray(groupType)) { // ArrayType - val elementType = toDataType(groupType.getFields.apply(0), isBinaryAsString) - ArrayType(elementType, containsNull = false) + val field = groupType.getFields.apply(0) + if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { + val bag = field.asGroupType() + assert(bag.getFieldCount == 1) + ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + } else { + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + } } else { // everything else: StructType val fields = groupType .getFields @@ -249,13 +259,27 @@ private[parquet] object ParquetTypesConverter extends Logging { inArray = true) ConversionPatterns.listType(repetition, name, parquetElementType) } + case ArrayType(elementType, true) => { + val parquetElementType = fromDataType( + elementType, + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + nullable = true, + inArray = false) + ConversionPatterns.listType( + repetition, + name, + new ParquetGroupType( + Repetition.REPEATED, + CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, + parquetElementType)) + } case StructType(structFields) => { val fields = structFields.map { field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) } new ParquetGroupType(repetition, name, fields) } - case MapType(keyType, valueType, _) => { + case MapType(keyType, valueType, valueContainsNull) => { val parquetKeyType = fromDataType( keyType, @@ -266,7 +290,7 @@ private[parquet] object ParquetTypesConverter extends Logging { fromDataType( valueType, CatalystConverter.MAP_VALUE_SCHEMA_NAME, - nullable = false, + nullable = valueContainsNull, inArray = false) ConversionPatterns.mapType( repetition, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 28f43b36832ac..4219cc080000b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -78,7 +78,9 @@ case class AllDataTypesWithNonPrimitiveType( booleanField: Boolean, binaryField: Array[Byte], array: Seq[Int], - map: Map[Int, String], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapValueContainsNull: Map[Int, Option[Long]], data: Data) class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { @@ -287,7 +289,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .map(x => AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, (0 to x).map(_.toByte).toArray, - (0 until x), (0 until x).map(i => i -> s"$i").toMap, Data((0 until x), Nested(x, s"$x")))) + (0 until x), + (0 until x).map(Option(_).filter(_ % 3 == 0)), + (0 until x).map(i => i -> i.toLong).toMap, + (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), + Data((0 until x), Nested(x, s"$x")))) .saveAsParquetFile(tempDir) val result = parquetFile(tempDir).collect() range.foreach { @@ -302,8 +308,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(i).getBoolean(7) === (i % 2 == 0)) assert(result(i)(8) === (0 to i).map(_.toByte).toArray) assert(result(i)(9) === (0 until i)) - assert(result(i)(10) === (0 until i).map(i => i -> s"$i").toMap) - assert(result(i)(11) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) + assert(result(i)(10) === (0 until i).map(i => if (i % 3 == 0) i else null)) + assert(result(i)(11) === (0 until i).map(i => i -> i.toLong).toMap) + assert(result(i)(12) === (0 until i).map(i => i -> i.toLong).toMap + (i -> null)) + assert(result(i)(13) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) } } From be043e3f20c6562482f9e4e739d8bb3fc9c1f201 Mon Sep 17 00:00:00 2001 From: Martin Weindel Date: Tue, 26 Aug 2014 18:30:39 -0700 Subject: [PATCH 0489/1492] [SPARK-3240] Adding known issue for MESOS-1688 When using Mesos with the fine-grained mode, a Spark job can run into a dead lock on low allocatable memory on Mesos slaves. As a work-around 32 MB (= Mesos MIN_MEM) are allocated for each task, to ensure Mesos making new offers after task completion. From my perspective, it would be better to fix this problem in Mesos by dropping the constraint on memory for offers, but as temporary solution this patch helps to avoid the dead lock on current Mesos versions. See [[MESOS-1688] No offers if no memory is allocatable](https://issues.apache.org/jira/browse/MESOS-1688) for details for this problem. Author: Martin Weindel Closes #1860 from MartinWeindel/master and squashes the following commits: 5762030 [Martin Weindel] reverting work-around a6bf837 [Martin Weindel] added known issue for issue MESOS-1688 d9d2ca6 [Martin Weindel] work around for problem with Mesos offering semantic (see [https://issues.apache.org/jira/browse/MESOS-1688]) --- docs/running-on-mesos.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 9998dddc652a6..1073abb202c56 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -165,6 +165,8 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). +# Known issues +- When using the "fine-grained" mode, make sure that your executors always leave 32 MB free on the slaves. Otherwise it can happen that your Spark job does not proceed anymore. Currently, Apache Mesos only offers resources if there are at least 32 MB memory allocatable. But as Spark allocates memory only for the executor and cpu only for tasks, it can happen on high slave memory usage that no new tasks will be started anymore. More details can be found in [MESOS-1688](https://issues.apache.org/jira/browse/MESOS-1688). Alternatively use the "coarse-gained" mode, which is not affected by this issue. # Running Alongside Hadoop From d8345471ce8e59d693f7f59c480df339953a4d32 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 26 Aug 2014 18:55:00 -0700 Subject: [PATCH 0490/1492] Fix unclosed HTML tag in Yarn docs. --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9bc20dbf926b2..943f06b114cb9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -75,7 +75,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes (none) Comma-separated list of files to be placed in the working directory of each executor. - + spark.yarn.executor.memoryOverhead From ee91eb8c53da84f8875fb5cec5820a6e189057b9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Aug 2014 19:29:54 -0700 Subject: [PATCH 0491/1492] Manually close some old pull requests Closes #530, Closes #223, Closes #738, Closes #546 From e70aff6c2d7f216060def0bd02be6a3d9017cd13 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Aug 2014 19:35:19 -0700 Subject: [PATCH 0492/1492] Manually close old pull requests Closes #671, Closes #515 From bf719056b71d55e1194554661dfa194ed03d364d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 26 Aug 2014 21:59:48 -0700 Subject: [PATCH 0493/1492] [SPARK-3224] FetchFailed reduce stages should only show up once in failed stages (in UI) This is a HOTFIX for 1.1. Author: Reynold Xin Author: Kay Ousterhout Closes #2127 from rxin/SPARK-3224 and squashes the following commits: effb1ce [Reynold Xin] Move log message. 49282b3 [Reynold Xin] Kay's feedback. 3f01847 [Reynold Xin] Merge pull request #2 from kayousterhout/SPARK-3224 796d282 [Kay Ousterhout] Added unit test for SPARK-3224 3d3d356 [Reynold Xin] Remove map output loc even for repeated FetchFaileds. 1dd3eb5 [Reynold Xin] [SPARK-3224] FetchFailed reduce stages should only show up once in the failed stages UI. --- .../apache/spark/scheduler/DAGScheduler.scala | 32 +++++++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 41 ++++++++++++++++++- 2 files changed, 59 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 34131984570e4..2ccc27324ac8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1045,31 +1045,39 @@ class DAGScheduler( stage.pendingTasks += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => - // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) - markStageAsFinished(failedStage, Some("Fetch failure")) - runningStages -= failedStage - // TODO: Cancel running tasks in the stage - logInfo("Marking " + failedStage + " (" + failedStage.name + - ") for resubmision due to a fetch failure") - // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) - if (mapId != -1) { - mapStage.removeOutputLoc(mapId, bmAddress) - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + + // It is likely that we receive multiple FetchFailed for a single stage (because we have + // multiple tasks running concurrently on different executors). In that case, it is possible + // the fetch failure has already been handled by the scheduler. + if (runningStages.contains(failedStage)) { + logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + + s"due to a fetch failure from $mapStage (${mapStage.name})") + markStageAsFinished(failedStage, Some("Fetch failure")) + runningStages -= failedStage } - logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + - "); marking it for resubmission") + if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. eventProcessActor may be // null during unit tests. + // TODO: Cancel running tasks in the stage import env.actorSystem.dispatcher + logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " + + s"$failedStage (${failedStage.name}) due to fetch failure") env.actorSystem.scheduler.scheduleOnce( RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) } failedStages += failedStage failedStages += mapStage + + // Mark the map whose fetch failed as broken in the map stage + if (mapId != -1) { + mapStage.removeOutputLoc(mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + } + // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, Some(task.epoch)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index bd829752eb401..f5fed988ade24 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{HashSet, HashMap, Map} +import scala.collection.mutable.{ArrayBuffer, HashSet, HashMap, Map} import scala.language.reflectiveCalls import akka.actor._ @@ -98,7 +98,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val WAIT_TIMEOUT_MILLIS = 10000 val sparkListener = new SparkListener() { val successfulStages = new HashSet[Int]() - val failedStages = new HashSet[Int]() + val failedStages = new ArrayBuffer[Int]() override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageInfo = stageCompleted.stageInfo if (stageInfo.failureReason.isEmpty) { @@ -435,6 +435,43 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("trivial shuffle with multiple fetch failures") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + // The MapOutputTracker should know about both map output locations. + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + Array("hostA", "hostB")) + + // The first result task fails, with a fetch failure for the output from the first mapper. + runEvent(CompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), + null, + Map[Long, Any](), + null, + null)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + + // The second ResultTask fails, with a fetch failure for the output from the second mapper. + runEvent(CompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1), + null, + Map[Long, Any](), + null, + null)) + // The SparkListener should not receive redundant failure events. + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.size == 1) + } + test("ignore late map task completions") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 7557c4cfef2398d124b00472e2696f0559a36ef7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 26 Aug 2014 22:52:16 -0700 Subject: [PATCH 0494/1492] [SPARK-3167] Handle special driver configs in Windows This is an effort to bring the Windows scripts up to speed after recent splashing changes in #1845. Author: Andrew Or Closes #2129 from andrewor14/windows-config and squashes the following commits: 881a8f0 [Andrew Or] Add reference to Windows taskkill 92e6047 [Andrew Or] Update a few comments (minor) 22b1acd [Andrew Or] Fix style again (minor) afcffea [Andrew Or] Fix style (minor) 72004c2 [Andrew Or] Actually respect --driver-java-options 803218b [Andrew Or] Actually respect SPARK_*_CLASSPATH eeb34a0 [Andrew Or] Update outdated comment (minor) 35caecc [Andrew Or] In Windows, actually kill Java processes on exit f97daa2 [Andrew Or] Fix Windows spark shell stdin issue 83ebe60 [Andrew Or] Parse special driver configs in Windows (broken) --- bin/compute-classpath.cmd | 3 +- bin/spark-class2.cmd | 46 ++++++++++++++++--- bin/spark-submit | 2 +- bin/spark-submit.cmd | 34 +++++++++----- .../SparkSubmitDriverBootstrapper.scala | 19 +++++--- python/pyspark/java_gateway.py | 17 +++++++ 6 files changed, 95 insertions(+), 26 deletions(-) mode change 100755 => 100644 bin/spark-class2.cmd diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 58710cd1bd548..5ad52452a5c98 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -36,7 +36,8 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath -set CLASSPATH=%FWDIR%conf +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH%;%FWDIR%conf + if exist "%FWDIR%RELEASE" ( for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( set ASSEMBLY_JAR=%%d diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd old mode 100755 new mode 100644 index e2c5f9c385189..6c5672819172b --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,6 +17,8 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +rem Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! + setlocal enabledelayedexpansion set SCALA_VERSION=2.10 @@ -38,7 +40,7 @@ if not "x%1"=="x" goto arg_given if not "x%SPARK_MEM%"=="x" ( echo Warning: SPARK_MEM is deprecated, please use a more specific config option - echo e.g., spark.executor.memory or SPARK_DRIVER_MEMORY. + echo e.g., spark.executor.memory or spark.driver.memory. ) rem Use SPARK_MEM or 512m as the default memory, to be overridden by specific options @@ -67,10 +69,18 @@ rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% -rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. -) else if "%1"=="org.apache.spark.repl.Main" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_REPL_OPTS% +rem Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + +rem SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. +rem The repl also uses SPARK_REPL_OPTS. +) else if "%1"=="org.apache.spark.deploy.SparkSubmit" ( + set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_SUBMIT_OPTS% %SPARK_REPL_OPTS% + if not "x%SPARK_SUBMIT_LIBRARY_PATH%"=="x" ( + set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_SUBMIT_LIBRARY_PATH% + ) else if not "x%SPARK_LIBRARY_PATH%"=="x" ( + set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_LIBRARY_PATH% + ) if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% + if not "x%SPARK_SUBMIT_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_SUBMIT_DRIVER_MEMORY% ) else ( set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% @@ -80,9 +90,9 @@ rem Set JAVA_OPTS to be able to load native libraries and to set heap size for /f "tokens=3" %%i in ('java -version 2^>^&1 ^| find "version"') do set jversion=%%i for /f "tokens=1 delims=_" %%i in ("%jversion:~1,-1%") do set jversion=%%i if "%jversion%" geq "1.8.0" ( - set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% + set JAVA_OPTS=%OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% ) else ( - set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% + set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% ) rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! @@ -115,5 +125,27 @@ rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java -"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +rem In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. +rem Here we must parse the properties file for relevant "spark.driver.*" configs before launching +rem the driver JVM itself. Instead of handling this complexity here, we launch a separate JVM +rem to prepare the launch environment of this driver JVM. + +rem In this case, leave out the main class (org.apache.spark.deploy.SparkSubmit) and use our own. +rem Leaving out the first argument is surprisingly difficult to do in Windows. Note that this must +rem be done here because the Windows "shift" command does not work in a conditional block. +set BOOTSTRAP_ARGS= +shift +:start_parse +if "%~1" == "" goto end_parse +set BOOTSTRAP_ARGS=%BOOTSTRAP_ARGS% %~1 +shift +goto start_parse +:end_parse + +if not [%SPARK_SUBMIT_BOOTSTRAP_DRIVER%] == [] ( + set SPARK_CLASS=1 + "%RUNNER%" org.apache.spark.deploy.SparkSubmitDriverBootstrapper %BOOTSTRAP_ARGS% +) else ( + "%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +) :exit diff --git a/bin/spark-submit b/bin/spark-submit index 32c911cd0438b..277c4ce571ca2 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -17,7 +17,7 @@ # limitations under the License. # -# NOTE: Any changes in this file must be reflected in SparkClassLauncher.scala! +# NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=("$@") diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index 6eb702ed8c561..cf6046d1547ad 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -17,23 +17,28 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +rem NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! + set SPARK_HOME=%~dp0.. set ORIG_ARGS=%* -rem Clear the values of all variables used -set DEPLOY_MODE= -set DRIVER_MEMORY= +rem Reset the values of all variables used +set SPARK_SUBMIT_DEPLOY_MODE=client +set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_HOME%\conf\spark-defaults.conf +set SPARK_SUBMIT_DRIVER_MEMORY= set SPARK_SUBMIT_LIBRARY_PATH= set SPARK_SUBMIT_CLASSPATH= set SPARK_SUBMIT_OPTS= -set SPARK_DRIVER_MEMORY= +set SPARK_SUBMIT_BOOTSTRAP_DRIVER= :loop if [%1] == [] goto continue if [%1] == [--deploy-mode] ( - set DEPLOY_MODE=%2 + set SPARK_SUBMIT_DEPLOY_MODE=%2 + ) else if [%1] == [--properties-file] ( + set SPARK_SUBMIT_PROPERTIES_FILE=%2 ) else if [%1] == [--driver-memory] ( - set DRIVER_MEMORY=%2 + set SPARK_SUBMIT_DRIVER_MEMORY=%2 ) else if [%1] == [--driver-library-path] ( set SPARK_SUBMIT_LIBRARY_PATH=%2 ) else if [%1] == [--driver-class-path] ( @@ -45,12 +50,19 @@ if [%1] == [] goto continue goto loop :continue -if [%DEPLOY_MODE%] == [] ( - set DEPLOY_MODE=client -) +rem For client mode, the driver will be launched in the same JVM that launches +rem SparkSubmit, so we may need to read the properties file for any extra class +rem paths, library paths, java options and memory early on. Otherwise, it will +rem be too late by the time the driver JVM has started. -if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] ( - set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY% +if [%SPARK_SUBMIT_DEPLOY_MODE%] == [client] ( + if exist %SPARK_SUBMIT_PROPERTIES_FILE% ( + rem Parse the properties file only if the special configs exist + for /f %%i in ('findstr /r /c:"^[\t ]*spark.driver.memory" /c:"^[\t ]*spark.driver.extra" ^ + %SPARK_SUBMIT_PROPERTIES_FILE%') do ( + set SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 + ) + ) ) cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index af607e6a4a065..7ca96ed57c2db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -133,17 +133,24 @@ private[spark] object SparkSubmitDriverBootstrapper { val process = builder.start() // Redirect stdin, stdout, and stderr to/from the child JVM - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") - stdinThread.start() stdoutThread.start() stderrThread.start() - // Terminate on broken pipe, which signals that the parent process has exited. This is - // important for the PySpark shell, where Spark submit itself is a python subprocess. - stdinThread.join() - process.destroy() + // In Windows, the subprocess reads directly from our stdin, so we should avoid spawning + // a thread that contends with the subprocess in reading from System.in. + if (Utils.isWindows) { + // For the PySpark shell, the termination of this process is handled in java_gateway.py + process.waitFor() + } else { + // Terminate on broken pipe, which signals that the parent process has exited. This is + // important for the PySpark shell, where Spark submit itself is a python subprocess. + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + stdinThread.start() + stdinThread.join() + process.destroy() + } } } diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 6f4f62f23bc4d..9c70fa5c16d0c 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -15,6 +15,7 @@ # limitations under the License. # +import atexit import os import sys import signal @@ -69,6 +70,22 @@ def preexec_func(): error_msg += "--------------------------------------------------------------\n" raise Exception(error_msg) + # In Windows, ensure the Java child processes do not linger after Python has exited. + # In UNIX-based systems, the child process can kill itself on broken pipe (i.e. when + # the parent process' stdin sends an EOF). In Windows, however, this is not possible + # because java.lang.Process reads directly from the parent process' stdin, contending + # with any opportunity to read an EOF from the parent. Note that this is only best + # effort and will not take effect if the python process is violently terminated. + if on_windows: + # In Windows, the child process here is "spark-submit.cmd", not the JVM itself + # (because the UNIX "exec" command is not available). This means we cannot simply + # call proc.kill(), which kills only the "spark-submit.cmd" process but not the + # JVMs. Instead, we use "taskkill" with the tree-kill option "/t" to terminate all + # child processes in the tree (http://technet.microsoft.com/en-us/library/bb491009.aspx) + def killChild(): + Popen(["cmd", "/c", "taskkill", "/f", "/t", "/pid", str(proc.pid)]) + atexit.register(killChild) + # Create a thread to echo output from the GatewayServer, which is required # for Java log output to show up: class EchoOutputThread(Thread): From 9d65f2712c250a561c9c1f6259aa12e861ed239d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 26 Aug 2014 23:40:50 -0700 Subject: [PATCH 0495/1492] HOTFIX: Minor typo in conf template --- conf/spark-defaults.conf.template | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index 94427029b94d7..a48dcc70e1363 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -7,4 +7,4 @@ # spark.eventLog.dir hdfs://namenode:8021/directory # spark.serializer org.apache.spark.serializer.KryoSerializer # spark.driver.memory 5g -# spark.executor.extraJavaOptions -XX:+PrintGCDetail -Dkey=value -Dnumbers="one two three" +# spark.executor.extraJavaOptions -XX:+PrintGCDetails -Dkey=value -Dnumbers="one two three" From 3e2864e40472b32e6a7eec5ba3bc83562d2a1a62 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 27 Aug 2014 00:13:38 -0700 Subject: [PATCH 0496/1492] [SPARK-3139] Made ContextCleaner to not block on shuffles As a workaround for SPARK-3015, the ContextCleaner was made "blocking", that is, it cleaned items one-by-one. But shuffles can take a long time to be deleted. Given that the RC for 1.1 is imminent, this PR makes a narrow change in the context cleaner - not wait for shuffle cleanups to complete. Also it changes the error messages on failure to delete to be milder warnings, as exceptions in the delete code path for one item does not really stop the actual functioning of the system. Author: Tathagata Das Closes #2143 from tdas/cleaner-shuffle-fix and squashes the following commits: 9c84202 [Tathagata Das] Restoring default blocking behavior in ContextCleanerSuite, and added docs to identify that spark.cleaner.referenceTracking.blocking does not control shuffle. 2181329 [Tathagata Das] Mark shuffle cleanup as non-blocking. e337cc2 [Tathagata Das] Changed semantics based on PR comments. 387b578 [Tathagata Das] Made ContextCleaner to not block on shuffles --- .../org/apache/spark/ContextCleaner.scala | 18 ++++++++++++++++-- .../spark/storage/BlockManagerMaster.scala | 12 +++++++----- .../org/apache/spark/ContextCleanerSuite.scala | 3 +++ 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 3848734d6f639..ede1e23f4fcc5 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -65,7 +65,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private val cleaningThread = new Thread() { override def run() { keepCleaning() }} /** - * Whether the cleaning thread will block on cleanup tasks. + * Whether the cleaning thread will block on cleanup tasks (other than shuffle, which + * is controlled by the `spark.cleaner.referenceTracking.blocking.shuffle` parameter). * * Due to SPARK-3015, this is set to true by default. This is intended to be only a temporary * workaround for the issue, which is ultimately caused by the way the BlockManager actors @@ -76,6 +77,19 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private val blockOnCleanupTasks = sc.conf.getBoolean( "spark.cleaner.referenceTracking.blocking", true) + /** + * Whether the cleaning thread will block on shuffle cleanup tasks. + * + * When context cleaner is configured to block on every delete request, it can throw timeout + * exceptions on cleanup of shuffle blocks, as reported in SPARK-3139. To avoid that, this + * parameter by default disables blocking on shuffle cleanups. Note that this does not affect + * the cleanup of RDDs and broadcasts. This is intended to be a temporary workaround, + * until the real Akka issue (referred to in the comment above `blockOnCleanupTasks`) is + * resolved. + */ + private val blockOnShuffleCleanupTasks = sc.conf.getBoolean( + "spark.cleaner.referenceTracking.blocking.shuffle", false) + @volatile private var stopped = false /** Attach a listener object to get information of when objects are cleaned. */ @@ -128,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { case CleanRDD(rddId) => doCleanupRDD(rddId, blocking = blockOnCleanupTasks) case CleanShuffle(shuffleId) => - doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks) + doCleanupShuffle(shuffleId, blocking = blockOnShuffleCleanupTasks) case CleanBroadcast(broadcastId) => doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 669307765d1fa..e67b3dc5ce02e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -101,7 +101,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log def removeRdd(rddId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) future.onFailure { - case e: Throwable => logError("Failed to remove RDD " + rddId, e) + case e: Exception => + logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) @@ -112,7 +113,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log def removeShuffle(shuffleId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) future.onFailure { - case e: Throwable => logError("Failed to remove shuffle " + shuffleId, e) + case e: Exception => + logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) @@ -124,9 +126,9 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log val future = askDriverWithReply[Future[Seq[Int]]]( RemoveBroadcast(broadcastId, removeFromMaster)) future.onFailure { - case e: Throwable => - logError("Failed to remove broadcast " + broadcastId + - " with removeFromMaster = " + removeFromMaster, e) + case e: Exception => + logWarning(s"Failed to remove broadcast $broadcastId" + + s" with removeFromMaster = $removeFromMaster - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 4bc4346c0a288..2744894277ae8 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -52,6 +52,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[Ha .setMaster("local[2]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) before { @@ -243,6 +244,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) @@ -319,6 +321,7 @@ class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[Sor .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) From e1139dd60e0692e8adb1337c1f605165ce4b8895 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 27 Aug 2014 00:59:23 -0700 Subject: [PATCH 0497/1492] [SPARK-3237][SQL] Fix parquet filters with UDFs Author: Michael Armbrust Closes #2153 from marmbrus/parquetFilters and squashes the following commits: 712731a [Michael Armbrust] Use closure serializer for sending filters. 1e83f80 [Michael Armbrust] Clean udf functions. --- .../apache/spark/sql/catalyst/expressions/ScalaUdf.scala | 4 ++++ .../org/apache/spark/sql/parquet/ParquetFilters.scala | 8 ++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 63ac2a608b6ff..0b3c1df453fb8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -18,10 +18,14 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.util.ClosureCleaner case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) extends Expression { + // Clean function when not called with default no-arg constructor. + if (function != null) { ClosureCleaner.clean(function) } + type EvaluatedType = Any def nullable = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 2298a9b933df5..fe28e0d7269e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import java.nio.ByteBuffer + import org.apache.hadoop.conf.Configuration import parquet.filter._ @@ -25,6 +27,7 @@ import parquet.column.ColumnReader import com.google.common.io.BaseEncoding +import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate} import org.apache.spark.sql.catalyst.expressions._ @@ -237,7 +240,8 @@ object ParquetFilters { */ def serializeFilterExpressions(filters: Seq[Expression], conf: Configuration): Unit = { if (filters.length > 0) { - val serialized: Array[Byte] = SparkSqlSerializer.serialize(filters) + val serialized: Array[Byte] = + SparkEnv.get.closureSerializer.newInstance().serialize(filters).array() val encoded: String = BaseEncoding.base64().encode(serialized) conf.set(PARQUET_FILTER_DATA, encoded) } @@ -252,7 +256,7 @@ object ParquetFilters { val data = conf.get(PARQUET_FILTER_DATA) if (data != null) { val decoded: Array[Byte] = BaseEncoding.base64().decode(data) - SparkSqlSerializer.deserialize(decoded) + SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(decoded)) } else { Seq() } From 43dfc84f883822ea27b6e312d4353bf301c2e7ef Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 27 Aug 2014 01:19:48 -0700 Subject: [PATCH 0498/1492] [SPARK-2830][MLLIB] doc update for 1.1 1. renamed mllib-basics to mllib-data-types 1. renamed mllib-stats to mllib-statistics 1. moved random data generation to the bottom of mllib-stats 1. updated toc accordingly atalwalkar Author: Xiangrui Meng Closes #2151 from mengxr/mllib-doc-1.1 and squashes the following commits: 0bd79f3 [Xiangrui Meng] add mllib-data-types b64a5d7 [Xiangrui Meng] update the content list of basis statistics in mllib-guide f625cc2 [Xiangrui Meng] move mllib-basics to mllib-data-types 4d69250 [Xiangrui Meng] move random data generation to the bottom of statistics e64f3ce [Xiangrui Meng] move mllib-stats.md to mllib-statistics.md --- docs/{mllib-basics.md => mllib-data-types.md} | 4 +- docs/mllib-dimensionality-reduction.md | 4 +- docs/mllib-guide.md | 9 +- docs/{mllib-stats.md => mllib-statistics.md} | 156 +++++++++--------- 4 files changed, 87 insertions(+), 86 deletions(-) rename docs/{mllib-basics.md => mllib-data-types.md} (99%) rename docs/{mllib-stats.md => mllib-statistics.md} (99%) diff --git a/docs/mllib-basics.md b/docs/mllib-data-types.md similarity index 99% rename from docs/mllib-basics.md rename to docs/mllib-data-types.md index 8752df412950a..101dc2f8695f3 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-data-types.md @@ -1,7 +1,7 @@ --- layout: global -title: Basics - MLlib -displayTitle: MLlib - Basics +title: Data Types - MLlib +displayTitle: MLlib - Data Types --- * Table of contents diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 9f2cf6d48ec75..21cb35b4270ca 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Dimensionality Reduction of reducing the number of variables under consideration. It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -MLlib provides support for dimensionality reduction on the RowMatrix class. +MLlib provides support for dimensionality reduction on the RowMatrix class. ## Singular value decomposition (SVD) @@ -58,7 +58,7 @@ passes, $O(n)$ storage on each executor, and $O(n k)$ storage on the driver. ### SVD Example MLlib provides SVD functionality to row-oriented matrices, provided in the -RowMatrix class. +RowMatrix class.
    diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4d4198b9e0452..d3a510b3c17c6 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -7,12 +7,13 @@ MLlib is Spark's scalable machine learning library consisting of common learning including classification, regression, clustering, collaborative filtering, dimensionality reduction, as well as underlying optimization primitives, as outlined below: -* [Data types](mllib-basics.html) -* [Basic statistics](mllib-stats.html) - * random data generation - * stratified sampling +* [Data types](mllib-data-types.html) +* [Basic statistics](mllib-statistics.html) * summary statistics + * correlations + * stratified sampling * hypothesis testing + * random data generation * [Classification and regression](mllib-classification-regression.html) * [linear models (SVMs, logistic regression, linear regression)](mllib-linear-methods.html) * [decision trees](mllib-decision-tree.html) diff --git a/docs/mllib-stats.md b/docs/mllib-statistics.md similarity index 99% rename from docs/mllib-stats.md rename to docs/mllib-statistics.md index 511a9fbf710cc..c4632413991f1 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-statistics.md @@ -1,7 +1,7 @@ --- layout: global -title: Statistics Functionality - MLlib -displayTitle: MLlib - Statistics Functionality +title: Basic Statistics - MLlib +displayTitle: MLlib - Basic Statistics --- * Table of contents @@ -25,7 +25,7 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` -## Summary Statistics +## Summary statistics We provide column summary statistics for `RDD[Vector]` through the function `colStats` available in `Statistics`. @@ -104,81 +104,7 @@ print summary.numNonzeros()
    -## Random data generation - -Random data generation is useful for randomized algorithms, prototyping, and performance testing. -MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: -uniform, standard normal, or Poisson. - -
    -
    -[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.random.RandomRDDs._ - -val sc: SparkContext = ... - -// Generate a random double RDD that contains 1 million i.i.d. values drawn from the -// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -val u = normalRDD(sc, 1000000L, 10) -// Apply a transform to get a random double RDD following `N(1, 4)`. -val v = u.map(x => 1.0 + 2.0 * x) -{% endhighlight %} -
    - -
    -[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight java %} -import org.apache.spark.SparkContext; -import org.apache.spark.api.JavaDoubleRDD; -import static org.apache.spark.mllib.random.RandomRDDs.*; - -JavaSparkContext jsc = ... - -// Generate a random double RDD that contains 1 million i.i.d. values drawn from the -// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); -// Apply a transform to get a random double RDD following `N(1, 4)`. -JavaDoubleRDD v = u.map( - new Function() { - public Double call(Double x) { - return 1.0 + 2.0 * x; - } - }); -{% endhighlight %} -
    - -
    -[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight python %} -from pyspark.mllib.random import RandomRDDs - -sc = ... # SparkContext - -# Generate a random double RDD that contains 1 million i.i.d. values drawn from the -# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -u = RandomRDDs.uniformRDD(sc, 1000000L, 10) -# Apply a transform to get a random double RDD following `N(1, 4)`. -v = u.map(lambda x: 1.0 + 2.0 * x) -{% endhighlight %} -
    - -
    - -## Correlations calculation +## Correlations Calculating the correlation between two series of data is a common operation in Statistics. In MLlib we provide the flexibility to calculate pairwise correlations among many series. The supported @@ -455,3 +381,77 @@ for (ChiSqTestResult result : featureTestResults) {
    + +## Random data generation + +Random data generation is useful for randomized algorithms, prototyping, and performance testing. +MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: +uniform, standard normal, or Poisson. + +
    +
    +[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.random.RandomRDDs._ + +val sc: SparkContext = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +val u = normalRDD(sc, 1000000L, 10) +// Apply a transform to get a random double RDD following `N(1, 4)`. +val v = u.map(x => 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight java %} +import org.apache.spark.SparkContext; +import org.apache.spark.api.JavaDoubleRDD; +import static org.apache.spark.mllib.random.RandomRDDs.*; + +JavaSparkContext jsc = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); +// Apply a transform to get a random double RDD following `N(1, 4)`. +JavaDoubleRDD v = u.map( + new Function() { + public Double call(Double x) { + return 1.0 + 2.0 * x; + } + }); +{% endhighlight %} +
    + +
    +[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight python %} +from pyspark.mllib.random import RandomRDDs + +sc = ... # SparkContext + +# Generate a random double RDD that contains 1 million i.i.d. values drawn from the +# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +u = RandomRDDs.uniformRDD(sc, 1000000L, 10) +# Apply a transform to get a random double RDD following `N(1, 4)`. +v = u.map(lambda x: 1.0 + 2.0 * x) +{% endhighlight %} +
    + +
    From 171a41cb034f4ea80f6a3c91a6872970de16a14a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 27 Aug 2014 01:45:59 -0700 Subject: [PATCH 0499/1492] [SPARK-3227] [mllib] Added migration guide for v1.0 to v1.1 The only updates are in DecisionTree. CC: mengxr Author: Joseph K. Bradley Closes #2146 from jkbradley/mllib-migration and squashes the following commits: 5a1f487 [Joseph K. Bradley] small edit to doc 411d6d9 [Joseph K. Bradley] Added migration guide for v1.0 to v1.1. The only updates are in DecisionTree. --- docs/mllib-guide.md | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index d3a510b3c17c6..94fc98ce4fabe 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -60,6 +60,32 @@ To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 # Migration Guide +## From 1.0 to 1.1 + +The only API changes in MLlib v1.1 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.1: + +1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match +the implementations of trees in +[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) +and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). +In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. +In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. +This depth is specified by the `maxDepth` parameter in +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` +methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +rather than using the old parameter class `Strategy`. These new training methods explicitly +separate classification and regression, and they replace specialized parameter types with +simple `String` types. + +Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the +[Decision Trees Guide](mllib-decision-tree.html#examples). + ## From 0.9 to 1.0 In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few @@ -85,7 +111,7 @@ val vector: Vector = Vectors.dense(array) // a dense vector [`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. -*Note*. Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. +*Note*: Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`.
    From 6f671d04fa98f97fd48c5e749b9f47dd4a8b4f44 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 Aug 2014 02:39:02 -0700 Subject: [PATCH 0500/1492] [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner. Currently lot of errors get thrown from Avro IPC layer when the dstream or sink is shutdown. This PR cleans it up. Some refactoring is done in the receiver code to put all of the RPC code into a single Try and just recover from that. The sink code has also been cleaned up. Author: Hari Shreedharan Closes #2065 from harishreedharan/clean-flume-shutdown and squashes the following commits: f93a07c [Hari Shreedharan] Formatting fixes. d7427cc [Hari Shreedharan] More fixes! a0a8852 [Hari Shreedharan] Fix race condition, hopefully! Minor other changes. 4c9ed02 [Hari Shreedharan] Remove unneeded list in Callback handler. Other misc changes. 8fee36f [Hari Shreedharan] Scala-library is required, else maven build fails. Also catch InterruptedException in TxnProcessor. 445e700 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown 87232e0 [Hari Shreedharan] Refactor Flume Input Stream. Clean up code, better error handling. 9001d26 [Hari Shreedharan] Change log level to debug in TransactionProcessor#shutdown method e7b8d82 [Hari Shreedharan] Incorporate review feedback 598efa7 [Hari Shreedharan] Clean up some exception handling code e1027c6 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown ed608c8 [Hari Shreedharan] [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner. --- external/flume-sink/pom.xml | 4 + .../flume/sink/SparkAvroCallbackHandler.scala | 56 ++++-- .../flume/sink/TransactionProcessor.scala | 18 +- .../streaming/flume/FlumeBatchFetcher.scala | 167 ++++++++++++++++++ .../flume/FlumePollingInputDStream.scala | 77 ++------ 5 files changed, 236 insertions(+), 86 deletions(-) create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c1e8e65464fc1..b345276b08ba3 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -70,6 +70,10 @@ scalatest_${scala.binary.version} test + + org.scala-lang + scala-library + - org.codehaus.mojo - exec-maven-plugin - 1.2.1 + org.apache.maven.plugins + maven-antrun-plugin generate-resources - exec + run - unzip - ../python - - -o - lib/py4j*.zip - -d - build - + + + diff --git a/pom.xml b/pom.xml index 556b9da3d6d90..a5eaea80afd71 100644 --- a/pom.xml +++ b/pom.xml @@ -880,7 +880,7 @@ ${project.build.directory}/surefire-reports . - ${project.build.directory}/SparkTestSuite.txt + SparkTestSuite.txt -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m From 8f1f9aaf408d6f058000be6ddd66179ba8a69bfa Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 2 Sep 2014 10:47:05 -0700 Subject: [PATCH 0561/1492] [SPARK-1919] Fix Windows spark-shell --jars We were trying to add `file:/C:/path/to/my.jar` to the class path. We should add `C:/path/to/my.jar` instead. Tested on Windows 8.1. Author: Andrew Or Closes #2211 from andrewor14/windows-shell-jars and squashes the following commits: 262c6a2 [Andrew Or] Oops... Add the new code to the correct place 0d5a0c1 [Andrew Or] Format jar path only for adding to shell classpath 42bd626 [Andrew Or] Remove unnecessary code 0049f1b [Andrew Or] Remove embarrassing log messages b1755a0 [Andrew Or] Format jar paths properly before adding them to the classpath --- .../main/scala/org/apache/spark/repl/SparkILoop.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 53df599cf8121..d9eeffa86016a 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -18,6 +18,7 @@ import scala.tools.nsc.interpreter._ import scala.tools.nsc.interpreter.{ Results => IR } import Predef.{ println => _, _ } import java.io.{ BufferedReader, FileReader } +import java.net.URI import java.util.concurrent.locks.ReentrantLock import scala.sys.process.Process import scala.tools.nsc.interpreter.session._ @@ -189,8 +190,16 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, require(settings != null) if (addedClasspath != "") settings.classpath.append(addedClasspath) + val addedJars = + if (Utils.isWindows) { + // Strip any URI scheme prefix so we can add the correct path to the classpath + // e.g. file:/C:/my/path.jar -> C:/my/path.jar + SparkILoop.getAddedJars.map { jar => new URI(jar).getPath.stripPrefix("/") } + } else { + SparkILoop.getAddedJars + } // work around for Scala bug - val totalClassPath = SparkILoop.getAddedJars.foldLeft( + val totalClassPath = addedJars.foldLeft( settings.classpath.value)((l, r) => ClassPath.join(l, r)) this.settings.classpath.value = totalClassPath From 066f31a6b213121441fc9618abd5bae4a706a215 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 2 Sep 2014 13:33:23 -0500 Subject: [PATCH 0562/1492] [SPARK-3347] [yarn] Fix yarn-alpha compilation. Missing import. Oops. Author: Marcelo Vanzin Closes #2236 from vanzin/SPARK-3347 and squashes the following commits: 594fc39 [Marcelo Vanzin] [SPARK-3347] [yarn] Fix yarn-alpha compilation. --- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 12f1cd3813a05..10fc39bba87d1 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -32,6 +32,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, Records} import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. From 81b9d5b628229ed69aa9dae45ec4c94068dcd71e Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 2 Sep 2014 11:34:55 -0700 Subject: [PATCH 0563/1492] SPARK-3052. Misleading and spurious FileSystem closed errors whenever a ... ...job fails while reading from Hadoop Author: Sandy Ryza Closes #1956 from sryza/sandy-spark-3052 and squashes the following commits: 815813a [Sandy Ryza] SPARK-3052. Misleading and spurious FileSystem closed errors whenever a job fails while reading from Hadoop --- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 9 +++++++-- .../main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 7 ++++++- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index c8623314c98eb..036dcc49664ef 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -42,7 +42,8 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD -import org.apache.spark.util.NextIterator +import org.apache.spark.util.{NextIterator, Utils} + /** * A Spark split class that wraps around a Hadoop InputSplit. @@ -228,7 +229,11 @@ class HadoopRDD[K, V]( try { reader.close() } catch { - case e: Exception => logWarning("Exception in RecordReader.close()", e) + case e: Exception => { + if (!Utils.inShutdown()) { + logWarning("Exception in RecordReader.close()", e) + } + } } } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 58f707b9b4634..4c84b3f62354d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD +import org.apache.spark.util.Utils private[spark] class NewHadoopPartition( rddId: Int, @@ -153,7 +154,11 @@ class NewHadoopRDD[K, V]( try { reader.close() } catch { - case e: Exception => logWarning("Exception in RecordReader.close()", e) + case e: Exception => { + if (!Utils.inShutdown()) { + logWarning("Exception in RecordReader.close()", e) + } + } } } } From e2c901b4c72b247bb422dd5acf057bc583e639ab Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 2 Sep 2014 15:47:47 -0700 Subject: [PATCH 0564/1492] [SPARK-2871] [PySpark] add countApproxDistinct() API RDD.countApproxDistinct(relativeSD=0.05): :: Experimental :: Return approximate number of distinct elements in the RDD. The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available here. This support all the types of objects, which is supported by Pyrolite, nearly all builtin types. param relativeSD Relative accuracy. Smaller values create counters that require more space. It must be greater than 0.000017. >>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct() >>> 950 < n < 1050 True >>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct() >>> 18 < n < 22 True Author: Davies Liu Closes #2142 from davies/countApproxDistinct and squashes the following commits: e20da47 [Davies Liu] remove the correction in Python c38c4e4 [Davies Liu] fix doc tests 2ab157c [Davies Liu] fix doc tests 9d2565f [Davies Liu] add commments and link for hash collision correction d306492 [Davies Liu] change range of hash of tuple to [0, maxint] ded624f [Davies Liu] calculate hash in Python 4cba98f [Davies Liu] add more tests a85a8c6 [Davies Liu] Merge branch 'master' into countApproxDistinct e97e342 [Davies Liu] add countApproxDistinct() --- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- python/pyspark/rdd.py | 39 ++++++++++++++++--- python/pyspark/tests.py | 16 ++++++++ 3 files changed, 51 insertions(+), 6 deletions(-) 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 daea2617e62ea..af9e31ba7b720 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -993,7 +993,7 @@ abstract class RDD[T: ClassTag]( */ @Experimental def countApproxDistinct(p: Int, sp: Int): Long = { - require(p >= 4, s"p ($p) must be greater than 0") + require(p >= 4, s"p ($p) must be at least 4") require(sp <= 32, s"sp ($sp) cannot be greater than 32") require(sp == 0 || p <= sp, s"p ($p) cannot be greater than sp ($sp)") val zeroCounter = new HyperLogLogPlus(p, sp) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 2d80fad796957..6fc9f66bc5a94 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -62,7 +62,7 @@ def portable_hash(x): >>> portable_hash(None) 0 - >>> portable_hash((None, 1)) + >>> portable_hash((None, 1)) & 0xffffffff 219750521 """ if x is None: @@ -72,7 +72,7 @@ def portable_hash(x): for i in x: h ^= portable_hash(i) h *= 1000003 - h &= 0xffffffff + h &= sys.maxint h ^= len(x) if h == -1: h = -2 @@ -1942,7 +1942,7 @@ def _is_pickled(self): return True return False - def _to_jrdd(self): + def _to_java_object_rdd(self): """ Return an JavaRDD of Object by unpickling It will convert each Python object into Java object by Pyrolite, whenever the @@ -1977,7 +1977,7 @@ def sumApprox(self, timeout, confidence=0.95): >>> (rdd.sumApprox(1000) - r) / r < 0.05 True """ - jrdd = self.mapPartitions(lambda it: [float(sum(it))])._to_jrdd() + jrdd = self.mapPartitions(lambda it: [float(sum(it))])._to_java_object_rdd() jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd()) r = jdrdd.sumApprox(timeout, confidence).getFinalValue() return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high()) @@ -1993,11 +1993,40 @@ def meanApprox(self, timeout, confidence=0.95): >>> (rdd.meanApprox(1000) - r) / r < 0.05 True """ - jrdd = self.map(float)._to_jrdd() + jrdd = self.map(float)._to_java_object_rdd() jdrdd = self.ctx._jvm.JavaDoubleRDD.fromRDD(jrdd.rdd()) r = jdrdd.meanApprox(timeout, confidence).getFinalValue() return BoundedFloat(r.mean(), r.confidence(), r.low(), r.high()) + def countApproxDistinct(self, relativeSD=0.05): + """ + :: Experimental :: + Return approximate number of distinct elements in the RDD. + + The algorithm used is based on streamlib's implementation of + "HyperLogLog in Practice: Algorithmic Engineering of a State + of The Art Cardinality Estimation Algorithm", available + here. + + @param relativeSD Relative accuracy. Smaller values create + counters that require more space. + It must be greater than 0.000017. + + >>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct() + >>> 950 < n < 1050 + True + >>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct() + >>> 18 < n < 22 + True + """ + if relativeSD < 0.000017: + raise ValueError("relativeSD should be greater than 0.000017") + if relativeSD > 0.37: + raise ValueError("relativeSD should be smaller than 0.37") + # the hash space in Java is 2^32 + hashRDD = self.map(lambda x: portable_hash(x) & 0xFFFFFFFF) + return hashRDD._to_java_object_rdd().countApproxDistinct(relativeSD) + class PipelinedRDD(RDD): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 3e7040eade1ab..f1a75cbff5c19 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -404,6 +404,22 @@ def test_zip_with_different_number_of_items(self): self.assertEquals(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) + def test_count_approx_distinct(self): + rdd = self.sc.parallelize(range(1000)) + self.assertTrue(950 < rdd.countApproxDistinct(0.04) < 1050) + self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.04) < 1050) + self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.04) < 1050) + self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.04) < 1050) + + rdd = self.sc.parallelize([i % 20 for i in range(1000)], 7) + self.assertTrue(18 < rdd.countApproxDistinct() < 22) + self.assertTrue(18 < rdd.map(float).countApproxDistinct() < 22) + self.assertTrue(18 < rdd.map(str).countApproxDistinct() < 22) + self.assertTrue(18 < rdd.map(lambda x: (x, -x)).countApproxDistinct() < 22) + + self.assertRaises(ValueError, lambda: rdd.countApproxDistinct(0.00000001)) + self.assertRaises(ValueError, lambda: rdd.countApproxDistinct(0.5)) + def test_histogram(self): # empty rdd = self.sc.parallelize([]) From 644e31524a6a9a22c671a368aeb3b4eaeb61cf29 Mon Sep 17 00:00:00 2001 From: Prudhvi Krishna Date: Tue, 2 Sep 2014 17:36:53 -0700 Subject: [PATCH 0565/1492] SPARK-3328 fixed make-distribution script --with-tachyon option. Directory path for dependencies jar and resources in Tachyon 0.5.0 has been changed. Author: Prudhvi Krishna Closes #2228 from prudhvije/SPARK-3328/make-dist-fix and squashes the following commits: d1d2c22 [Prudhvi Krishna] SPARK-3328 fixed make-distribution script --with-tachyon option. --- make-distribution.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index ee1399071112d..f030d3f430581 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -219,10 +219,10 @@ if [ "$SPARK_TACHYON" == "true" ]; then wget "$TACHYON_URL" tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" - cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" + cp "tachyon-${TACHYON_VERSION}/core/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" - cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" + cp -r "tachyon-${TACHYON_VERSION}"/core/src/main/java/tachyon/web "$DISTDIR/tachyon/src/main/java/tachyon/web" if [[ `uname -a` == Darwin* ]]; then # need to run sed differently on osx From 7c92b49d6b62f88fcde883aacb60c5e32ae54b30 Mon Sep 17 00:00:00 2001 From: Larry Xiao Date: Tue, 2 Sep 2014 18:29:08 -0700 Subject: [PATCH 0566/1492] [SPARK-1986][GraphX]move lib.Analytics to org.apache.spark.examples to support ~/spark/bin/run-example GraphXAnalytics triangles /soc-LiveJournal1.txt --numEPart=256 Author: Larry Xiao Closes #1766 from larryxiao/1986 and squashes the following commits: bb77cd9 [Larry Xiao] [SPARK-1986][GraphX]move lib.Analytics to org.apache.spark.examples --- .../scala/org/apache/spark/examples/graphx}/Analytics.scala | 3 ++- .../org/apache/spark/examples/graphx/LiveJournalPageRank.scala | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) rename {graphx/src/main/scala/org/apache/spark/graphx/lib => examples/src/main/scala/org/apache/spark/examples/graphx}/Analytics.scala (98%) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala similarity index 98% rename from graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala rename to examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index c1513a00453cf..c4317a6aec798 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.graphx.lib +package org.apache.spark.examples.graphx import scala.collection.mutable import org.apache.spark._ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ +import org.apache.spark.graphx.lib._ import org.apache.spark.graphx.PartitionStrategy._ /** diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index 6ef3b62dcbedc..bdc8fa7f99f2e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.graphx import org.apache.spark.SparkContext._ import org.apache.spark._ import org.apache.spark.graphx._ -import org.apache.spark.graphx.lib.Analytics +import org.apache.spark.examples.graphx.Analytics /** * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from From 7c9bbf172512701c75992671bcb2f4b6d9e5034b Mon Sep 17 00:00:00 2001 From: uncleGen Date: Tue, 2 Sep 2014 18:41:54 -0700 Subject: [PATCH 0567/1492] [SPARK-3123][GraphX]: override the "setName" function to set EdgeRDD's name manually just as VertexRDD does. Author: uncleGen Closes #2033 from uncleGen/master_origin and squashes the following commits: 801994b [uncleGen] Update EdgeRDD.scala --- .../main/scala/org/apache/spark/graphx/EdgeRDD.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 899a3cbd62b60..5bcb96b136ed7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -37,7 +37,15 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) extends RDD[Edge[ED]](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { - partitionsRDD.setName("EdgeRDD") + override def setName(_name: String): this.type = { + if (partitionsRDD.name != null) { + partitionsRDD.setName(partitionsRDD.name + ", " + _name) + } else { + partitionsRDD.setName(_name) + } + this + } + setName("EdgeRDD") override protected def getPartitions: Array[Partition] = partitionsRDD.partitions From aa7de128c5987fd2e134736f07ae913ad1f5eb26 Mon Sep 17 00:00:00 2001 From: Larry Xiao Date: Tue, 2 Sep 2014 18:50:52 -0700 Subject: [PATCH 0568/1492] [SPARK-2981][GraphX] EdgePartition1D Int overflow minor fix detail is here: https://issues.apache.org/jira/browse/SPARK-2981 Author: Larry Xiao Closes #1902 from larryxiao/2981 and squashes the following commits: 88059a2 [Larry Xiao] [SPARK-2981][GraphX] EdgePartition1D Int overflow --- .../main/scala/org/apache/spark/graphx/PartitionStrategy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala index 5e7e72a764cc8..13033fee0e6b5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/PartitionStrategy.scala @@ -91,7 +91,7 @@ object PartitionStrategy { case object EdgePartition1D extends PartitionStrategy { override def getPartition(src: VertexId, dst: VertexId, numParts: PartitionID): PartitionID = { val mixingPrime: VertexId = 1125899906842597L - (math.abs(src) * mixingPrime).toInt % numParts + (math.abs(src * mixingPrime) % numParts).toInt } } From e9bb12bea9fbef94332fbec88e3cd9197a27b7ad Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 2 Sep 2014 19:02:48 -0700 Subject: [PATCH 0569/1492] [SPARK-1981][Streaming][Hotfix] Fixed docs related to kinesis - Include kinesis in the unidocs - Hide non-public classes from docs Author: Tathagata Das Closes #2239 from tdas/kinesis-doc-fix and squashes the following commits: 156e20c [Tathagata Das] More fixes, based on PR comments. e9a6c01 [Tathagata Das] Fixed docs related to kinesis --- docs/_plugins/copy_api_dirs.rb | 4 ++-- .../examples/streaming/JavaKinesisWordCountASL.java | 10 +++++----- .../spark/examples/streaming/KinesisWordCountASL.scala | 6 +++--- .../apache/spark/streaming/kinesis/KinesisUtils.scala | 7 ++++--- project/SparkBuild.scala | 6 +++--- 5 files changed, 17 insertions(+), 16 deletions(-) diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 2dbbbf6feb4b8..3b02e090aec28 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -25,8 +25,8 @@ curr_dir = pwd cd("..") - puts "Running 'sbt/sbt compile unidoc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt compile unidoc` + puts "Running 'sbt/sbt -Pkinesis-asl compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt -Pkinesis-asl compile unidoc` puts "Moving back into docs dir." cd("docs") diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 1a710d7b18c6f..aa917d0575c4c 100644 --- a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -75,7 +75,7 @@ * onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in the class definition. */ -public final class JavaKinesisWordCountASL { +public final class JavaKinesisWordCountASL { // needs to be public for access from run-example private static final Pattern WORD_SEPARATOR = Pattern.compile(" "); private static final Logger logger = Logger.getLogger(JavaKinesisWordCountASL.class); @@ -87,10 +87,10 @@ public static void main(String[] args) { /* Check that all required args were passed in. */ if (args.length < 2) { System.err.println( - "|Usage: KinesisWordCount \n" + - "| is the name of the Kinesis stream\n" + - "| is the endpoint of the Kinesis service\n" + - "| (e.g. https://kinesis.us-east-1.amazonaws.com)\n"); + "Usage: JavaKinesisWordCountASL \n" + + " is the name of the Kinesis stream\n" + + " is the endpoint of the Kinesis service\n" + + " (e.g. https://kinesis.us-east-1.amazonaws.com)\n"); System.exit(1); } diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index d03edf8b30a9f..fffd90de08240 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -69,7 +69,7 @@ import org.apache.log4j.Level * dummy data onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in that class definition. */ -object KinesisWordCountASL extends Logging { +private object KinesisWordCountASL extends Logging { def main(args: Array[String]) { /* Check that all required args were passed in. */ if (args.length < 2) { @@ -154,7 +154,7 @@ object KinesisWordCountASL extends Logging { * org.apache.spark.examples.streaming.KinesisWordCountProducerASL mySparkStream \ * https://kinesis.us-east-1.amazonaws.com 10 5 */ -object KinesisWordCountProducerASL { +private object KinesisWordCountProducerASL { def main(args: Array[String]) { if (args.length < 4) { System.err.println("Usage: KinesisWordCountProducerASL " + @@ -235,7 +235,7 @@ object KinesisWordCountProducerASL { * Utility functions for Spark Streaming examples. * This has been lifted from the examples/ project to remove the circular dependency. */ -object StreamingExamples extends Logging { +private[streaming] object StreamingExamples extends Logging { /** Set reasonable logging levels for streaming if the user has not configured log4j. */ def setStreamingLogLevels() { diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala index 713cac0e293c0..96f4399accd3a 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisUtils.scala @@ -35,7 +35,7 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionIn object KinesisUtils { /** * Create an InputDStream that pulls messages from a Kinesis stream. - * + * :: Experimental :: * @param ssc StreamingContext object * @param streamName Kinesis stream name * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) @@ -52,6 +52,7 @@ object KinesisUtils { * * @return ReceiverInputDStream[Array[Byte]] */ + @Experimental def createStream( ssc: StreamingContext, streamName: String, @@ -65,9 +66,8 @@ object KinesisUtils { /** * Create a Java-friendly InputDStream that pulls messages from a Kinesis stream. - * + * :: Experimental :: * @param jssc Java StreamingContext object - * @param ssc StreamingContext object * @param streamName Kinesis stream name * @param endpointUrl Url of Kinesis service (e.g., https://kinesis.us-east-1.amazonaws.com) * @param checkpointInterval Checkpoint interval for Kinesis checkpointing. @@ -83,6 +83,7 @@ object KinesisUtils { * * @return JavaReceiverInputDStream[Array[Byte]] */ + @Experimental def createStream( jssc: JavaStreamingContext, streamName: String, diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 4c696d3d385fb..a26c2c90cb321 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -290,9 +290,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, bagel, graphx, examples, tools, catalyst, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, bagel, graphx, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { @@ -314,7 +314,7 @@ object Unidoc { "-group", "Core Java API", packageList("api.java", "api.java.function"), "-group", "Spark Streaming", packageList( "streaming.api.java", "streaming.flume", "streaming.kafka", - "streaming.mqtt", "streaming.twitter", "streaming.zeromq" + "streaming.mqtt", "streaming.twitter", "streaming.zeromq", "streaming.kinesis" ), "-group", "MLlib", packageList( "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", From 9b225ac3072de522b40b46aba6df1f1c231f13ef Mon Sep 17 00:00:00 2001 From: luluorta Date: Tue, 2 Sep 2014 19:25:52 -0700 Subject: [PATCH 0570/1492] [SPARK-2823][GraphX]fix GraphX EdgeRDD zipPartitions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If the users set “spark.default.parallelism” and the value is different with the EdgeRDD partition number, GraphX jobs will throw: java.lang.IllegalArgumentException: Can't zip RDDs with unequal numbers of partitions Author: luluorta Closes #1763 from luluorta/fix-graph-zip and squashes the following commits: 8338961 [luluorta] fix GraphX EdgeRDD zipPartitions --- .../scala/org/apache/spark/graphx/EdgeRDD.scala | 4 ++-- .../org/apache/spark/graphx/GraphSuite.scala | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 5bcb96b136ed7..35fbd47e6c2a5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} -import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} +import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -55,7 +55,7 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( * partitioner that allows co-partitioning with `partitionsRDD`. */ override val partitioner = - partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) + partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitionsRDD.partitions.size))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 6506bac73d71c..eaaa4499b6b93 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.graphx import org.scalatest.FunSuite +import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ @@ -350,4 +351,19 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("non-default number of edge partitions") { + val n = 10 + val defaultParallelism = 3 + val numEdgePartitions = 4 + assert(defaultParallelism != numEdgePartitions) + val conf = new SparkConf() + .set("spark.default.parallelism", defaultParallelism.toString) + val sc = new SparkContext("local", "test", conf) + val edges = sc.parallelize((1 to n).map(x => (x: VertexId, 0: VertexId)), + numEdgePartitions) + val graph = Graph.fromEdgeTuples(edges, 1) + val neighborAttrSums = graph.mapReduceTriplets[Int]( + et => Iterator((et.dstId, et.srcAttr)), _ + _) + assert(neighborAttrSums.collect.toSet === Set((0: VertexId, n))) + } } From 0cd91f666dfe318934d5d41d3cb7085b2f9ae278 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 2 Sep 2014 20:31:15 -0700 Subject: [PATCH 0571/1492] [SPARK-3341][SQL] The dataType of Sqrt expression should be DoubleType. Author: Takuya UESHIN Closes #2233 from ueshin/issues/SPARK-3341 and squashes the following commits: e497320 [Takuya UESHIN] Fix data type of Sqrt expression. --- .../org/apache/spark/sql/catalyst/expressions/arithmetic.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 56f042891a2e6..f988fb010b107 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -36,7 +36,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { case class Sqrt(child: Expression) extends UnaryExpression { type EvaluatedType = Any - def dataType = child.dataType + def dataType = DoubleType override def foldable = child.foldable def nullable = child.nullable override def toString = s"SQRT($child)" From 19d3e1e8e9c25c0936c0ad6efbc2092b473723aa Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 2 Sep 2014 20:49:36 -0700 Subject: [PATCH 0572/1492] [SQL] Renamed ColumnStat to ColumnMetrics to avoid confusion between ColumnStats Class names of these two are just too similar. Author: Cheng Lian Closes #2189 from liancheng/column-metrics and squashes the following commits: 8bb3b21 [Cheng Lian] Renamed ColumnStat to ColumnMetrics to avoid confusion between ColumnStats --- .../org/apache/spark/sql/execution/debug/package.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index 8ff757bbe3508..a9535a750bcd7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -74,22 +74,22 @@ package object debug { } /** - * A collection of stats for each column of output. + * A collection of metrics for each column of output. * @param elementTypes the actual runtime types for the output. Useful when there are bugs * causing the wrong data to be projected. */ - case class ColumnStat( + case class ColumnMetrics( elementTypes: Accumulator[HashSet[String]] = sparkContext.accumulator(HashSet.empty)) val tupleCount = sparkContext.accumulator[Int](0) val numColumns = child.output.size - val columnStats = Array.fill(child.output.size)(new ColumnStat()) + val columnStats = Array.fill(child.output.size)(new ColumnMetrics()) def dumpStats(): Unit = { println(s"== ${child.simpleString} ==") println(s"Tuples output: ${tupleCount.value}") - child.output.zip(columnStats).foreach { case(attr, stat) => - val actualDataTypes =stat.elementTypes.value.mkString("{", ",", "}") + child.output.zip(columnStats).foreach { case(attr, metric) => + val actualDataTypes = metric.elementTypes.value.mkString("{", ",", "}") println(s" ${attr.name} ${attr.dataType}: $actualDataTypes") } } From 24ab384018270e4f7af7eb8ca7192f337498eaf5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 2 Sep 2014 20:51:25 -0700 Subject: [PATCH 0573/1492] [SPARK-3300][SQL] No need to call clear() and shorten build() The function `ensureFreeSpace` in object `ColumnBuilder` clears old buffer before copying its content to new buffer. This PR fixes it. Author: Liang-Chi Hsieh Closes #2195 from viirya/fix_buffer_clear and squashes the following commits: 792f009 [Liang-Chi Hsieh] no need to call clear(). use flip() instead of calling limit(), position() and rewind(). df2169f [Liang-Chi Hsieh] should clean old buffer after copying its content. --- .../scala/org/apache/spark/sql/columnar/ColumnBuilder.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 7e7bb2859bbcd..247337a875c75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -75,8 +75,7 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( } override def build() = { - buffer.limit(buffer.position()).rewind() - buffer + buffer.flip().asInstanceOf[ByteBuffer] } } @@ -129,7 +128,6 @@ private[sql] object ColumnBuilder { val newSize = capacity + size.max(capacity / 8 + 1) val pos = orig.position() - orig.clear() ByteBuffer .allocate(newSize) .order(ByteOrder.nativeOrder()) From c64cc435e2a29c6f0ff66022fd4d5b4cb5011718 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 2 Sep 2014 21:30:09 -0700 Subject: [PATCH 0574/1492] SPARK-3358: [EC2] Switch back to HVM instances for m3.X. During regression tests of Spark 1.1 we discovered perf issues with PVM instances when running PySpark. This reverts a change added in #1156 which changed the default type for m3 instances to PVM. Author: Patrick Wendell Closes #2244 from pwendell/ec2-hvm and squashes the following commits: 1342d7e [Patrick Wendell] SPARK-3358: [EC2] Switch back to HVM instances for m3.X. --- ec2/spark_ec2.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7e25df57ee45b..eed6eb8485183 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -234,10 +234,10 @@ def get_spark_ami(opts): "cg1.4xlarge": "hvm", "hs1.8xlarge": "pvm", "hi1.4xlarge": "pvm", - "m3.medium": "pvm", - "m3.large": "pvm", - "m3.xlarge": "pvm", - "m3.2xlarge": "pvm", + "m3.medium": "hvm", + "m3.large": "hvm", + "m3.xlarge": "hvm", + "m3.2xlarge": "hvm", "cr1.8xlarge": "hvm", "i2.xlarge": "hvm", "i2.2xlarge": "hvm", From 6a72a36940311fcb3429bd34c8818bc7d513115c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 3 Sep 2014 08:22:50 -0500 Subject: [PATCH 0575/1492] [SPARK-3187] [yarn] Cleanup allocator code. Move all shared logic to the base YarnAllocator class, and leave the version-specific logic in the version-specific module. Author: Marcelo Vanzin Closes #2169 from vanzin/SPARK-3187 and squashes the following commits: 46c2826 [Marcelo Vanzin] Hide the privates. 4dc9c83 [Marcelo Vanzin] Actually release containers. 8b1a077 [Marcelo Vanzin] Changes to the Yarn alpha allocator. f3f5f1d [Marcelo Vanzin] [SPARK-3187] [yarn] Cleanup allocator code. --- .../deploy/yarn/YarnAllocationHandler.scala | 462 +++--------------- .../spark/deploy/yarn/YarnAllocator.scala | 425 +++++++++++++++- .../deploy/yarn/YarnAllocationHandler.scala | 402 +-------------- 3 files changed, 495 insertions(+), 794 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 629cd13f67145..9f9e16c06452b 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,35 +17,21 @@ package org.apache.spark.deploy.yarn -import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} +import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.{Logging, SparkConf, SparkEnv} -import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.AMRMProtocol -import org.apache.hadoop.yarn.api.records.{AMResponse, ApplicationAttemptId} -import org.apache.hadoop.yarn.api.records.{Container, ContainerId} -import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} -import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest import org.apache.hadoop.yarn.util.Records -// TODO: -// Too many params. -// Needs to be mt-safe -// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should -// make it more proactive and decoupled. - -// Note that right now, we assume all node asks as uniform in terms of capabilities and priority -// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for -// more info on how we are requesting for containers. - /** * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ @@ -56,357 +42,20 @@ private[yarn] class YarnAllocationHandler( appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator with Logging { - - // These three are locked on allocatedHostToContainersMap. Complementary data structures - // allocatedHostToContainersMap : containers which are running : host, Set - // allocatedContainerToHostMap: container to host mapping. - private val allocatedHostToContainersMap = - new HashMap[String, collection.mutable.Set[ContainerId]]() - - private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() - - // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an - // allocated node) - // As with the two data structures above, tightly coupled with them, and to be locked on - // allocatedHostToContainersMap - private val allocatedRackCount = new HashMap[String, Int]() - - // Containers which have been released. - private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() - // Containers to be released in next request to RM - private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] - - // Additional memory overhead - in mb. - private def memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - - private val numExecutorsRunning = new AtomicInteger() - // Used to generate a unique id per executor - private val executorIdCounter = new AtomicInteger() - private val lastResponseId = new AtomicInteger() - private val numExecutorsFailed = new AtomicInteger() - - private val maxExecutors = args.numExecutors - private val executorMemory = args.executorMemory - private val executorCores = args.executorCores - private val (preferredHostToCount, preferredRackToCount) = - generateNodeToWeight(conf, preferredNodes) - - def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - - def getNumExecutorsFailed: Int = numExecutorsFailed.intValue - - def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (executorMemory + memoryOverhead) - } - - override def allocateResources() = { - // We need to send the request only once from what I understand ... but for now, not modifying - // this much. - val executorsToRequest = Math.max(maxExecutors - numExecutorsRunning.get(), 0) - - // Keep polling the Resource Manager for containers - val amResp = allocateExecutorResources(executorsToRequest).getAMResponse - - val _allocatedContainers = amResp.getAllocatedContainers() - - if (_allocatedContainers.size > 0) { - logDebug(""" - Allocated containers: %d - Current executor count: %d - Containers released: %s - Containers to be released: %s - Cluster resources: %s - """.format( - _allocatedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers, - amResp.getAvailableResources)) - - val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() - - // Ignore if not satisfying constraints { - for (container <- _allocatedContainers) { - if (isResourceConstraintSatisfied(container)) { - // allocatedContainers += container - - val host = container.getNodeId.getHost - val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]()) - - containers += container - } else { - // Add all ignored containers to released list - releasedContainerList.add(container.getId()) - } - } - - // Find the appropriate containers to use. Slightly non trivial groupBy ... - val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() - - for (candidateHost <- hostToContainers.keySet) - { - val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) - val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) - - var remainingContainers = hostToContainers.get(candidateHost).getOrElse(null) - assert(remainingContainers != null) - - if (requiredHostCount >= remainingContainers.size){ - // Since we got <= required containers, add all to dataLocalContainers - dataLocalContainers.put(candidateHost, remainingContainers) - // all consumed - remainingContainers = null - } else if (requiredHostCount > 0) { - // Container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - - // requiredHostCount) and rest as remainingContainer - val (dataLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredHostCount) - dataLocalContainers.put(candidateHost, dataLocal) - // remainingContainers = remaining - - // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation - // cycle will reallocate (but wont treat it as data local) - for (container <- remaining) releasedContainerList.add(container.getId()) - remainingContainers = null - } - - // Now rack local - if (remainingContainers != null){ - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - - if (rack != null){ - val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - - rackLocalContainers.get(rack).getOrElse(List()).size - - - if (requiredRackCount >= remainingContainers.size){ - // Add all to dataLocalContainers - dataLocalContainers.put(rack, remainingContainers) - // All consumed - remainingContainers = null - } else if (requiredRackCount > 0) { - // container list has more containers than we need for data locality. - // Split into two : data local container count of (remainingContainers.size - - // requiredRackCount) and rest as remainingContainer - val (rackLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredRackCount) - val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, - new ArrayBuffer[Container]()) - - existingRackLocal ++= rackLocal - remainingContainers = remaining - } - } - } - - // If still not consumed, then it is off rack host - add to that list. - if (remainingContainers != null){ - offRackContainers.put(candidateHost, remainingContainers) - } - } - - // Now that we have split the containers into various groups, go through them in order : - // first host local, then rack local and then off rack (everything else). - // Note that the list we create below tries to ensure that not all containers end up within a - // host if there are sufficiently large number of hosts/containers. - - val allocatedContainers = new ArrayBuffer[Container](_allocatedContainers.size) - allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers) - allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers) - allocatedContainers ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers) - - // Run each of the allocated containers - for (container <- allocatedContainers) { - val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() - val executorHostname = container.getNodeId.getHost - val containerId = container.getId - - assert( container.getResource.getMemory >= - (executorMemory + memoryOverhead)) - - if (numExecutorsRunningNow > maxExecutors) { - logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, executorHostname)) - releasedContainerList.add(containerId) - // reset counter back to old value. - numExecutorsRunning.decrementAndGet() - } else { - // Deallocate + allocate can result in reusing id's wrongly - so use a different counter - // (executorIdCounter) - val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( - SparkEnv.driverActorSystemName, - sparkConf.get("spark.driver.host"), - sparkConf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) - - logInfo("launching container on " + containerId + " host " + executorHostname) - // Just to be safe, simply remove it from pendingReleaseContainers. - // Should not be there, but .. - pendingReleaseContainers.remove(containerId) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) - allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]()) - - containerSet += containerId - allocatedContainerToHostMap.put(containerId, executorHostname) - if (rack != null) { - allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) - } - } - - new Thread( - new ExecutorRunnable(container, conf, sparkConf, driverUrl, executorId, - executorHostname, executorMemory, executorCores) - ).start() - } - } - logDebug(""" - Finished processing %d containers. - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - allocatedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } - - - val completedContainers = amResp.getCompletedContainersStatuses() - if (completedContainers.size > 0){ - logDebug("Completed %d containers, to-be-released: %s".format( - completedContainers.size, releasedContainerList)) - for (completedContainer <- completedContainers){ - val containerId = completedContainer.getContainerId - - // Was this released by us ? If yes, then simply remove from containerSet and move on. - if (pendingReleaseContainers.containsKey(containerId)) { - pendingReleaseContainers.remove(containerId) - } else { - // Simply decrement count - next iteration of ReporterThread will take care of allocating. - numExecutorsRunning.decrementAndGet() - logInfo("Completed container %s (state: %s, exit status: %s)".format( - containerId, - completedContainer.getState, - completedContainer.getExitStatus())) - // Hadoop 2.2.X added a ContainerExitStatus we should switch to use - // there are some exit status' we shouldn't necessarily count against us, but for - // now I think its ok as none of the containers are expected to exit - if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) - numExecutorsFailed.incrementAndGet() - } - } - - allocatedHostToContainersMap.synchronized { - if (allocatedContainerToHostMap.containsKey(containerId)) { - val host = allocatedContainerToHostMap.get(containerId).getOrElse(null) - assert (host != null) - - val containerSet = allocatedHostToContainersMap.get(host).getOrElse(null) - assert (containerSet != null) - - containerSet -= containerId - if (containerSet.isEmpty) { - allocatedHostToContainersMap.remove(host) - } else { - allocatedHostToContainersMap.update(host, containerSet) - } - - allocatedContainerToHostMap -= containerId - - // Doing this within locked context, sigh ... move to outside ? - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null) { - val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) { - allocatedRackCount.put(rack, rackCount) - } else { - allocatedRackCount.remove(rack) - } - } - } - } - } - logDebug(""" - Finished processing %d completed containers. - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - completedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } - } - - def createRackResourceRequests(hostContainers: List[ResourceRequest]): List[ResourceRequest] = { - // First generate modified racks and new set of hosts under it : then issue requests - val rackToCounts = new HashMap[String, Int]() - - // Within this lock - used to read/write to the rack related maps too. - for (container <- hostContainers) { - val candidateHost = container.getHostName - val candidateNumContainers = container.getNumContainers - assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - var count = rackToCounts.getOrElse(rack, 0) - count += candidateNumContainers - rackToCounts.put(rack, count) - } - } - - val requestedContainers: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](rackToCounts.size) - for ((rack, count) <- rackToCounts){ - requestedContainers += - createResourceRequest(AllocationType.RACK, rack, count, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - - requestedContainers.toList - } - - def allocatedContainersOnHost(host: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedHostToContainersMap.getOrElse(host, Set()).size - } - retval - } + extends YarnAllocator(conf, sparkConf, args, preferredNodes) { - def allocatedContainersOnRack(rack: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedRackCount.getOrElse(rack, 0) - } - retval - } - - private def allocateExecutorResources(numExecutors: Int): AllocateResponse = { + private val lastResponseId = new AtomicInteger() + private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() + override protected def allocateContainers(count: Int): YarnAllocateResponse = { var resourceRequests: List[ResourceRequest] = null - // default. - if (numExecutors <= 0 || preferredHostToCount.isEmpty) { - logDebug("numExecutors: " + numExecutors + ", host preferences: " + + // default. + if (count <= 0 || preferredHostToCount.isEmpty) { + logDebug("numExecutors: " + count + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, numExecutors, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) + AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. @@ -429,7 +78,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests: ResourceRequest = createResourceRequest( AllocationType.ANY, resource = null, - numExecutors, + count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( @@ -451,8 +100,8 @@ private[yarn] class YarnAllocationHandler( val releasedContainerList = createReleasedContainerList() req.addAllReleases(releasedContainerList) - if (numExecutors > 0) { - logInfo("Allocating %d executor containers with %d of memory each.".format(numExecutors, + if (count > 0) { + logInfo("Allocating %d executor containers with %d of memory each.".format(count, executorMemory + memoryOverhead)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) @@ -466,9 +115,42 @@ private[yarn] class YarnAllocationHandler( request.getPriority, request.getCapability)) } - resourceManager.allocate(req) + new AlphaAllocateResponse(resourceManager.allocate(req).getAMResponse()) } + override protected def releaseContainer(container: Container) = { + releaseList.add(container.getId()) + } + + private def createRackResourceRequests(hostContainers: List[ResourceRequest]): + List[ResourceRequest] = { + // First generate modified racks and new set of hosts under it : then issue requests + val rackToCounts = new HashMap[String, Int]() + + // Within this lock - used to read/write to the rack related maps too. + for (container <- hostContainers) { + val candidateHost = container.getHostName + val candidateNumContainers = container.getNumContainers + assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) + if (rack != null) { + var count = rackToCounts.getOrElse(rack, 0) + count += candidateNumContainers + rackToCounts.put(rack, count) + } + } + + val requestedContainers: ArrayBuffer[ResourceRequest] = + new ArrayBuffer[ResourceRequest](rackToCounts.size) + for ((rack, count) <- rackToCounts){ + requestedContainers += + createResourceRequest(AllocationType.RACK, rack, count, + YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) + } + + requestedContainers.toList + } private def createResourceRequest( requestType: AllocationType.AllocationType, @@ -521,48 +203,24 @@ private[yarn] class YarnAllocationHandler( rsrcRequest } - def createReleasedContainerList(): ArrayBuffer[ContainerId] = { - + private def createReleasedContainerList(): ArrayBuffer[ContainerId] = { val retval = new ArrayBuffer[ContainerId](1) // Iterator on COW list ... - for (container <- releasedContainerList.iterator()){ + for (container <- releaseList.iterator()){ retval += container } // Remove from the original list. - if (! retval.isEmpty) { - releasedContainerList.removeAll(retval) - for (v <- retval) pendingReleaseContainers.put(v, true) - logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + - pendingReleaseContainers) + if (!retval.isEmpty) { + releaseList.removeAll(retval) + logInfo("Releasing " + retval.size + " containers.") } - retval } - // A simple method to copy the split info map. - private def generateNodeToWeight( - conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]]) : - // host to count, rack to count - (Map[String, Int], Map[String, Int]) = { - - if (input == null) return (Map[String, Int](), Map[String, Int]()) - - val hostToCount = new HashMap[String, Int] - val rackToCount = new HashMap[String, Int] - - for ((host, splits) <- input) { - val hostCount = hostToCount.getOrElse(host, 0) - hostToCount.put(host, hostCount + splits.size) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null){ - val rackCount = rackToCount.getOrElse(host, 0) - rackToCount.put(host, rackCount + splits.size) - } - } - - (hostToCount.toMap, rackToCount.toMap) + private class AlphaAllocateResponse(response: AMResponse) extends YarnAllocateResponse { + override def getAllocatedContainers() = response.getAllocatedContainers() + override def getAvailableResources() = response.getAvailableResources() + override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index cad94e5e19e1f..c74dd1c2b21dc 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -17,18 +17,431 @@ package org.apache.spark.deploy.yarn +import java.util.{List => JList} +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse + +import org.apache.spark.{Logging, SparkConf, SparkEnv} +import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend + object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value } +// TODO: +// Too many params. +// Needs to be mt-safe +// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should +// make it more proactive and decoupled. + +// Note that right now, we assume all node asks as uniform in terms of capabilities and priority +// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for +// more info on how we are requesting for containers. + /** - * Interface that defines a Yarn allocator. + * Common code for the Yarn container allocator. Contains all the version-agnostic code to + * manage container allocation for a running Spark application. */ -trait YarnAllocator { +private[yarn] abstract class YarnAllocator( + conf: Configuration, + sparkConf: SparkConf, + args: ApplicationMasterArguments, + preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) + extends Logging { - def allocateResources(): Unit - def getNumExecutorsFailed: Int - def getNumExecutorsRunning: Int + // These three are locked on allocatedHostToContainersMap. Complementary data structures + // allocatedHostToContainersMap : containers which are running : host, Set + // allocatedContainerToHostMap: container to host mapping. + private val allocatedHostToContainersMap = + new HashMap[String, collection.mutable.Set[ContainerId]]() -} + private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() + + // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an + // allocated node) + // As with the two data structures above, tightly coupled with them, and to be locked on + // allocatedHostToContainersMap + private val allocatedRackCount = new HashMap[String, Int]() + + // Containers to be released in next request to RM + private val releasedContainers = new ConcurrentHashMap[ContainerId, Boolean] + + // Additional memory overhead - in mb. + protected val memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) + + // Number of container requests that have been sent to, but not yet allocated by the + // ApplicationMaster. + private val numPendingAllocate = new AtomicInteger() + private val numExecutorsRunning = new AtomicInteger() + // Used to generate a unique id per executor + private val executorIdCounter = new AtomicInteger() + private val numExecutorsFailed = new AtomicInteger() + + private val maxExecutors = args.numExecutors + + protected val executorMemory = args.executorMemory + protected val executorCores = args.executorCores + protected val (preferredHostToCount, preferredRackToCount) = + generateNodeToWeight(conf, preferredNodes) + + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue + + def getNumExecutorsFailed: Int = numExecutorsFailed.intValue + + def allocateResources() = { + val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() + + if (missing > 0) { + numPendingAllocate.addAndGet(missing) + logInfo("Will Allocate %d executor containers, each with %d memory".format( + missing, + (executorMemory + memoryOverhead))) + } else { + logDebug("Empty allocation request ...") + } + + val allocateResponse = allocateContainers(missing) + val allocatedContainers = allocateResponse.getAllocatedContainers() + + if (allocatedContainers.size > 0) { + var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size) + + if (numPendingAllocateNow < 0) { + numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow) + } + + logDebug(""" + Allocated containers: %d + Current executor count: %d + Containers released: %s + Cluster resources: %s + """.format( + allocatedContainers.size, + numExecutorsRunning.get(), + releasedContainers, + allocateResponse.getAvailableResources)) + + val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (container <- allocatedContainers) { + if (isResourceConstraintSatisfied(container)) { + // Add the accepted `container` to the host's list of already accepted, + // allocated containers + val host = container.getNodeId.getHost + val containersForHost = hostToContainers.getOrElseUpdate(host, + new ArrayBuffer[Container]()) + containersForHost += container + } else { + // Release container, since it doesn't satisfy resource constraints. + internalReleaseContainer(container) + } + } + + // Find the appropriate containers to use. + // TODO: Cleanup this group-by... + val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() + val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() + + for (candidateHost <- hostToContainers.keySet) { + val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) + val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) + + val remainingContainersOpt = hostToContainers.get(candidateHost) + assert(remainingContainersOpt.isDefined) + var remainingContainers = remainingContainersOpt.get + + if (requiredHostCount >= remainingContainers.size) { + // Since we have <= required containers, add all remaining containers to + // `dataLocalContainers`. + dataLocalContainers.put(candidateHost, remainingContainers) + // There are no more free containers remaining. + remainingContainers = null + } else if (requiredHostCount > 0) { + // Container list has more containers than we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (dataLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredHostCount) + dataLocalContainers.put(candidateHost, dataLocal) + + // Invariant: remainingContainers == remaining + + // YARN has a nasty habit of allocating a ton of containers on a host - discourage this. + // Add each container in `remaining` to list of containers to release. If we have an + // insufficient number of containers, then the next allocation cycle will reallocate + // (but won't treat it as data local). + // TODO(harvey): Rephrase this comment some more. + for (container <- remaining) internalReleaseContainer(container) + remainingContainers = null + } + + // For rack local containers + if (remainingContainers != null) { + val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) + if (rack != null) { + val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + rackLocalContainers.getOrElse(rack, List()).size + + if (requiredRackCount >= remainingContainers.size) { + // Add all remaining containers to to `dataLocalContainers`. + dataLocalContainers.put(rack, remainingContainers) + remainingContainers = null + } else if (requiredRackCount > 0) { + // Container list has more containers that we need for data locality. + // Split the list into two: one based on the data local container count, + // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining + // containers. + val (rackLocal, remaining) = remainingContainers.splitAt( + remainingContainers.size - requiredRackCount) + val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, + new ArrayBuffer[Container]()) + + existingRackLocal ++= rackLocal + + remainingContainers = remaining + } + } + } + + if (remainingContainers != null) { + // Not all containers have been consumed - add them to the list of off-rack containers. + offRackContainers.put(candidateHost, remainingContainers) + } + } + + // Now that we have split the containers into various groups, go through them in order: + // first host-local, then rack-local, and finally off-rack. + // Note that the list we create below tries to ensure that not all containers end up within + // a host if there is a sufficiently large number of hosts/containers. + val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size) + allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers) + allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers) + allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers) + + // Run each of the allocated containers. + for (container <- allocatedContainersToProcess) { + val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() + val executorHostname = container.getNodeId.getHost + val containerId = container.getId + + val executorMemoryOverhead = (executorMemory + memoryOverhead) + assert(container.getResource.getMemory >= executorMemoryOverhead) + + if (numExecutorsRunningNow > maxExecutors) { + logInfo("""Ignoring container %s at host %s, since we already have the required number of + containers for it.""".format(containerId, executorHostname)) + internalReleaseContainer(container) + numExecutorsRunning.decrementAndGet() + } else { + val executorId = executorIdCounter.incrementAndGet().toString + val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( + SparkEnv.driverActorSystemName, + sparkConf.get("spark.driver.host"), + sparkConf.get("spark.driver.port"), + CoarseGrainedSchedulerBackend.ACTOR_NAME) + + logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) + + // To be safe, remove the container from `releasedContainers`. + releasedContainers.remove(containerId) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) + allocatedHostToContainersMap.synchronized { + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, + new HashSet[ContainerId]()) + + containerSet += containerId + allocatedContainerToHostMap.put(containerId, executorHostname) + + if (rack != null) { + allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) + } + } + logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( + driverUrl, executorHostname)) + val executorRunnable = new ExecutorRunnable( + container, + conf, + sparkConf, + driverUrl, + executorId, + executorHostname, + executorMemory, + executorCores) + new Thread(executorRunnable).start() + } + } + logDebug(""" + Finished allocating %s containers (from %s originally). + Current number of executors running: %d, + Released containers: %s + """.format( + allocatedContainersToProcess, + allocatedContainers, + numExecutorsRunning.get(), + releasedContainers)) + } + + val completedContainers = allocateResponse.getCompletedContainersStatuses() + if (completedContainers.size > 0) { + logDebug("Completed %d containers".format(completedContainers.size)) + + for (completedContainer <- completedContainers) { + val containerId = completedContainer.getContainerId + + if (releasedContainers.containsKey(containerId)) { + // YarnAllocationHandler already marked the container for release, so remove it from + // `releasedContainers`. + releasedContainers.remove(containerId) + } else { + // Decrement the number of executors running. The next iteration of + // the ApplicationMaster's reporting thread will take care of allocating. + numExecutorsRunning.decrementAndGet() + logInfo("Completed container %s (state: %s, exit status: %s)".format( + containerId, + completedContainer.getState, + completedContainer.getExitStatus())) + // Hadoop 2.2.X added a ContainerExitStatus we should switch to use + // there are some exit status' we shouldn't necessarily count against us, but for + // now I think its ok as none of the containers are expected to exit + if (completedContainer.getExitStatus() != 0) { + logInfo("Container marked as failed: " + containerId) + numExecutorsFailed.incrementAndGet() + } + } + + allocatedHostToContainersMap.synchronized { + if (allocatedContainerToHostMap.containsKey(containerId)) { + val hostOpt = allocatedContainerToHostMap.get(containerId) + assert(hostOpt.isDefined) + val host = hostOpt.get + + val containerSetOpt = allocatedHostToContainersMap.get(host) + assert(containerSetOpt.isDefined) + val containerSet = containerSetOpt.get + + containerSet.remove(containerId) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } + + allocatedContainerToHostMap.remove(containerId) + + // TODO: Move this part outside the synchronized block? + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) + if (rack != null) { + val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } + } + } + } + } + logDebug(""" + Finished processing %d completed containers. + Current number of executors running: %d, + Released containers: %s + """.format( + completedContainers.size, + numExecutorsRunning.get(), + releasedContainers)) + } + } + + protected def allocatedContainersOnHost(host: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedHostToContainersMap.getOrElse(host, Set()).size + } + retval + } + + protected def allocatedContainersOnRack(rack: String): Int = { + var retval = 0 + allocatedHostToContainersMap.synchronized { + retval = allocatedRackCount.getOrElse(rack, 0) + } + retval + } + + private def isResourceConstraintSatisfied(container: Container): Boolean = { + container.getResource.getMemory >= (executorMemory + memoryOverhead) + } + + // A simple method to copy the split info map. + private def generateNodeToWeight( + conf: Configuration, + input: collection.Map[String, collection.Set[SplitInfo]] + ): (Map[String, Int], Map[String, Int]) = { + + if (input == null) { + return (Map[String, Int](), Map[String, Int]()) + } + + val hostToCount = new HashMap[String, Int] + val rackToCount = new HashMap[String, Int] + + for ((host, splits) <- input) { + val hostCount = hostToCount.getOrElse(host, 0) + hostToCount.put(host, hostCount + splits.size) + + val rack = YarnSparkHadoopUtil.lookupRack(conf, host) + if (rack != null) { + val rackCount = rackToCount.getOrElse(host, 0) + rackToCount.put(host, rackCount + splits.size) + } + } + + (hostToCount.toMap, rackToCount.toMap) + } + + private def internalReleaseContainer(container: Container) = { + releasedContainers.put(container.getId(), true) + releaseContainer(container) + } + + /** + * Called to allocate containers in the cluster. + * + * @param count Number of containers to allocate. + * If zero, should still contact RM (as a heartbeat). + * @return Response to the allocation request. + */ + protected def allocateContainers(count: Int): YarnAllocateResponse + + /** Called to release a previously allocated container. */ + protected def releaseContainer(container: Container): Unit + + /** + * Defines the interface for an allocate response from the RM. This is needed since the alpha + * and stable interfaces differ here in ways that cannot be fixed using other routes. + */ + protected trait YarnAllocateResponse { + + def getAllocatedContainers(): JList[Container] + + def getAvailableResources(): Resource + + def getCompletedContainersStatuses(): JList[ContainerStatus] + + } + +} \ No newline at end of file diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 4d5144989991f..ed31457b61571 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -17,36 +17,19 @@ package org.apache.spark.deploy.yarn -import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap} -import java.util.concurrent.atomic.AtomicInteger - import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.{Logging, SparkConf, SparkEnv} -import org.apache.spark.scheduler.{SplitInfo,TaskSchedulerImpl} -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.util.Utils +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.records.ApplicationAttemptId -import org.apache.hadoop.yarn.api.records.{Container, ContainerId} -import org.apache.hadoop.yarn.api.records.{Priority, Resource, ResourceRequest} -import org.apache.hadoop.yarn.api.protocolrecords.{AllocateRequest, AllocateResponse} +import org.apache.hadoop.yarn.api.records._ +import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.util.Records -// TODO: -// Too many params. -// Needs to be mt-safe -// Need to refactor this to make it 'cleaner' ... right now, all computation is reactive - should -// make it more proactive and decoupled. - -// Note that right now, we assume all node asks as uniform in terms of capabilities and priority -// Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for -// more info on how we are requesting for containers. - /** * Acquires resources for executors from a ResourceManager and launches executors in new containers. */ @@ -57,329 +40,22 @@ private[yarn] class YarnAllocationHandler( appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator with Logging { - - // These three are locked on allocatedHostToContainersMap. Complementary data structures - // allocatedHostToContainersMap : containers which are running : host, Set - // allocatedContainerToHostMap: container to host mapping. - private val allocatedHostToContainersMap = - new HashMap[String, collection.mutable.Set[ContainerId]]() - - private val allocatedContainerToHostMap = new HashMap[ContainerId, String]() - - // allocatedRackCount is populated ONLY if allocation happens (or decremented if this is an - // allocated node) - // As with the two data structures above, tightly coupled with them, and to be locked on - // allocatedHostToContainersMap - private val allocatedRackCount = new HashMap[String, Int]() - - // Containers which have been released. - private val releasedContainerList = new CopyOnWriteArrayList[ContainerId]() - // Containers to be released in next request to RM - private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] - - // Additional memory overhead - in mb. - private def memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - - // Number of container requests that have been sent to, but not yet allocated by the - // ApplicationMaster. - private val numPendingAllocate = new AtomicInteger() - private val numExecutorsRunning = new AtomicInteger() - // Used to generate a unique id per executor - private val executorIdCounter = new AtomicInteger() - private val lastResponseId = new AtomicInteger() - private val numExecutorsFailed = new AtomicInteger() - - private val maxExecutors = args.numExecutors - private val executorMemory = args.executorMemory - private val executorCores = args.executorCores - private val (preferredHostToCount, preferredRackToCount) = - generateNodeToWeight(conf, preferredNodes) - - override def getNumExecutorsRunning: Int = numExecutorsRunning.intValue + extends YarnAllocator(conf, sparkConf, args, preferredNodes) { - override def getNumExecutorsFailed: Int = numExecutorsFailed.intValue - - def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (executorMemory + memoryOverhead) - } - - def releaseContainer(container: Container) { - val containerId = container.getId - pendingReleaseContainers.put(containerId, true) - amClient.releaseAssignedContainer(containerId) + override protected def releaseContainer(container: Container) = { + amClient.releaseAssignedContainer(container.getId()) } - override def allocateResources() = { - addResourceRequests(maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get()) + override protected def allocateContainers(count: Int): YarnAllocateResponse = { + addResourceRequests(count) // We have already set the container request. Poll the ResourceManager for a response. // This doubles as a heartbeat if there are no pending container requests. val progressIndicator = 0.1f - val allocateResponse = amClient.allocate(progressIndicator) - - val allocatedContainers = allocateResponse.getAllocatedContainers() - if (allocatedContainers.size > 0) { - var numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * allocatedContainers.size) - - if (numPendingAllocateNow < 0) { - numPendingAllocateNow = numPendingAllocate.addAndGet(-1 * numPendingAllocateNow) - } - - logDebug(""" - Allocated containers: %d - Current executor count: %d - Containers released: %s - Containers to-be-released: %s - Cluster resources: %s - """.format( - allocatedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers, - allocateResponse.getAvailableResources)) - - val hostToContainers = new HashMap[String, ArrayBuffer[Container]]() - - for (container <- allocatedContainers) { - if (isResourceConstraintSatisfied(container)) { - // Add the accepted `container` to the host's list of already accepted, - // allocated containers - val host = container.getNodeId.getHost - val containersForHost = hostToContainers.getOrElseUpdate(host, - new ArrayBuffer[Container]()) - containersForHost += container - } else { - // Release container, since it doesn't satisfy resource constraints. - releaseContainer(container) - } - } - - // Find the appropriate containers to use. - // TODO: Cleanup this group-by... - val dataLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val rackLocalContainers = new HashMap[String, ArrayBuffer[Container]]() - val offRackContainers = new HashMap[String, ArrayBuffer[Container]]() - - for (candidateHost <- hostToContainers.keySet) { - val maxExpectedHostCount = preferredHostToCount.getOrElse(candidateHost, 0) - val requiredHostCount = maxExpectedHostCount - allocatedContainersOnHost(candidateHost) - - val remainingContainersOpt = hostToContainers.get(candidateHost) - assert(remainingContainersOpt.isDefined) - var remainingContainers = remainingContainersOpt.get - - if (requiredHostCount >= remainingContainers.size) { - // Since we have <= required containers, add all remaining containers to - // `dataLocalContainers`. - dataLocalContainers.put(candidateHost, remainingContainers) - // There are no more free containers remaining. - remainingContainers = null - } else if (requiredHostCount > 0) { - // Container list has more containers than we need for data locality. - // Split the list into two: one based on the data local container count, - // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining - // containers. - val (dataLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredHostCount) - dataLocalContainers.put(candidateHost, dataLocal) - - // Invariant: remainingContainers == remaining - - // YARN has a nasty habit of allocating a ton of containers on a host - discourage this. - // Add each container in `remaining` to list of containers to release. If we have an - // insufficient number of containers, then the next allocation cycle will reallocate - // (but won't treat it as data local). - // TODO(harvey): Rephrase this comment some more. - for (container <- remaining) releaseContainer(container) - remainingContainers = null - } - - // For rack local containers - if (remainingContainers != null) { - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - - rackLocalContainers.getOrElse(rack, List()).size - - if (requiredRackCount >= remainingContainers.size) { - // Add all remaining containers to to `dataLocalContainers`. - dataLocalContainers.put(rack, remainingContainers) - remainingContainers = null - } else if (requiredRackCount > 0) { - // Container list has more containers that we need for data locality. - // Split the list into two: one based on the data local container count, - // (`remainingContainers.size` - `requiredHostCount`), and the other to hold remaining - // containers. - val (rackLocal, remaining) = remainingContainers.splitAt( - remainingContainers.size - requiredRackCount) - val existingRackLocal = rackLocalContainers.getOrElseUpdate(rack, - new ArrayBuffer[Container]()) - - existingRackLocal ++= rackLocal - - remainingContainers = remaining - } - } - } - - if (remainingContainers != null) { - // Not all containers have been consumed - add them to the list of off-rack containers. - offRackContainers.put(candidateHost, remainingContainers) - } - } - - // Now that we have split the containers into various groups, go through them in order: - // first host-local, then rack-local, and finally off-rack. - // Note that the list we create below tries to ensure that not all containers end up within - // a host if there is a sufficiently large number of hosts/containers. - val allocatedContainersToProcess = new ArrayBuffer[Container](allocatedContainers.size) - allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(dataLocalContainers) - allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(rackLocalContainers) - allocatedContainersToProcess ++= TaskSchedulerImpl.prioritizeContainers(offRackContainers) - - // Run each of the allocated containers. - for (container <- allocatedContainersToProcess) { - val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() - val executorHostname = container.getNodeId.getHost - val containerId = container.getId - - val executorMemoryOverhead = (executorMemory + memoryOverhead) - assert(container.getResource.getMemory >= executorMemoryOverhead) - - if (numExecutorsRunningNow > maxExecutors) { - logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, executorHostname)) - releaseContainer(container) - numExecutorsRunning.decrementAndGet() - } else { - val executorId = executorIdCounter.incrementAndGet().toString - val driverUrl = "akka.tcp://%s@%s:%s/user/%s".format( - SparkEnv.driverActorSystemName, - sparkConf.get("spark.driver.host"), - sparkConf.get("spark.driver.port"), - CoarseGrainedSchedulerBackend.ACTOR_NAME) - - logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) - - // To be safe, remove the container from `pendingReleaseContainers`. - pendingReleaseContainers.remove(containerId) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, executorHostname) - allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, - new HashSet[ContainerId]()) - - containerSet += containerId - allocatedContainerToHostMap.put(containerId, executorHostname) - - if (rack != null) { - allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) - } - } - logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format( - driverUrl, executorHostname)) - val executorRunnable = new ExecutorRunnable( - container, - conf, - sparkConf, - driverUrl, - executorId, - executorHostname, - executorMemory, - executorCores) - new Thread(executorRunnable).start() - } - } - logDebug(""" - Finished allocating %s containers (from %s originally). - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - allocatedContainersToProcess, - allocatedContainers, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } - - val completedContainers = allocateResponse.getCompletedContainersStatuses() - if (completedContainers.size > 0) { - logDebug("Completed %d containers".format(completedContainers.size)) - - for (completedContainer <- completedContainers) { - val containerId = completedContainer.getContainerId - - if (pendingReleaseContainers.containsKey(containerId)) { - // YarnAllocationHandler already marked the container for release, so remove it from - // `pendingReleaseContainers`. - pendingReleaseContainers.remove(containerId) - } else { - // Decrement the number of executors running. The next iteration of - // the ApplicationMaster's reporting thread will take care of allocating. - numExecutorsRunning.decrementAndGet() - logInfo("Completed container %s (state: %s, exit status: %s)".format( - containerId, - completedContainer.getState, - completedContainer.getExitStatus())) - // Hadoop 2.2.X added a ContainerExitStatus we should switch to use - // there are some exit status' we shouldn't necessarily count against us, but for - // now I think its ok as none of the containers are expected to exit - if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) - numExecutorsFailed.incrementAndGet() - } - } - - allocatedHostToContainersMap.synchronized { - if (allocatedContainerToHostMap.containsKey(containerId)) { - val hostOpt = allocatedContainerToHostMap.get(containerId) - assert(hostOpt.isDefined) - val host = hostOpt.get - - val containerSetOpt = allocatedHostToContainersMap.get(host) - assert(containerSetOpt.isDefined) - val containerSet = containerSetOpt.get - - containerSet.remove(containerId) - if (containerSet.isEmpty) { - allocatedHostToContainersMap.remove(host) - } else { - allocatedHostToContainersMap.update(host, containerSet) - } - - allocatedContainerToHostMap.remove(containerId) - - // TODO: Move this part outside the synchronized block? - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null) { - val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) { - allocatedRackCount.put(rack, rackCount) - } else { - allocatedRackCount.remove(rack) - } - } - } - } - } - logDebug(""" - Finished processing %d completed containers. - Current number of executors running: %d, - releasedContainerList: %s, - pendingReleaseContainers: %s - """.format( - completedContainers.size, - numExecutorsRunning.get(), - releasedContainerList, - pendingReleaseContainers)) - } + new StableAllocateResponse(amClient.allocate(progressIndicator)) } - def createRackResourceRequests( + private def createRackResourceRequests( hostContainers: ArrayBuffer[ContainerRequest] ): ArrayBuffer[ContainerRequest] = { // Generate modified racks and new set of hosts under it before issuing requests. @@ -409,22 +85,6 @@ private[yarn] class YarnAllocationHandler( requestedContainers } - def allocatedContainersOnHost(host: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedHostToContainersMap.getOrElse(host, Set()).size - } - retval - } - - def allocatedContainersOnRack(rack: String): Int = { - var retval = 0 - allocatedHostToContainersMap.synchronized { - retval = allocatedRackCount.getOrElse(rack, 0) - } - retval - } - private def addResourceRequests(numExecutors: Int) { val containerRequests: List[ContainerRequest] = if (numExecutors <= 0 || preferredHostToCount.isEmpty) { @@ -472,15 +132,6 @@ private[yarn] class YarnAllocationHandler( amClient.addContainerRequest(request) } - if (numExecutors > 0) { - numPendingAllocate.addAndGet(numExecutors) - logInfo("Will Allocate %d executor containers, each with %d memory".format( - numExecutors, - (executorMemory + memoryOverhead))) - } else { - logDebug("Empty allocation request ...") - } - for (request <- containerRequests) { val nodes = request.getNodes var hostStr = if (nodes == null || nodes.isEmpty) { @@ -549,31 +200,10 @@ private[yarn] class YarnAllocationHandler( requests } - // A simple method to copy the split info map. - private def generateNodeToWeight( - conf: Configuration, - input: collection.Map[String, collection.Set[SplitInfo]] - ): (Map[String, Int], Map[String, Int]) = { - - if (input == null) { - return (Map[String, Int](), Map[String, Int]()) - } - - val hostToCount = new HashMap[String, Int] - val rackToCount = new HashMap[String, Int] - - for ((host, splits) <- input) { - val hostCount = hostToCount.getOrElse(host, 0) - hostToCount.put(host, hostCount + splits.size) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, host) - if (rack != null){ - val rackCount = rackToCount.getOrElse(host, 0) - rackToCount.put(host, rackCount + splits.size) - } - } - - (hostToCount.toMap, rackToCount.toMap) + private class StableAllocateResponse(response: AllocateResponse) extends YarnAllocateResponse { + override def getAllocatedContainers() = response.getAllocatedContainers() + override def getAvailableResources() = response.getAvailableResources() + override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() } } From 6481d27425f6d42ead36663c9a4ef7ee13b3a8c9 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 3 Sep 2014 11:49:45 -0700 Subject: [PATCH 0576/1492] [SPARK-3309] [PySpark] Put all public API in __all__ Put all public API in __all__, also put them all in pyspark.__init__.py, then we can got all the documents for public API by `pydoc pyspark`. It also can be used by other programs (such as Sphinx or Epydoc) to generate only documents for public APIs. Author: Davies Liu Closes #2205 from davies/public and squashes the following commits: c6c5567 [Davies Liu] fix message f7b35be [Davies Liu] put SchemeRDD, Row in pyspark.sql module 7e3016a [Davies Liu] add __all__ in mllib 6281b48 [Davies Liu] fix doc for SchemaRDD 6caab21 [Davies Liu] add public interfaces into pyspark.__init__.py --- python/pyspark/__init__.py | 14 +++++++++----- python/pyspark/accumulators.py | 3 +++ python/pyspark/broadcast.py | 24 ++++++++++++++---------- python/pyspark/conf.py | 2 ++ python/pyspark/context.py | 3 +++ python/pyspark/files.py | 3 +++ python/pyspark/mllib/classification.py | 4 ++++ python/pyspark/mllib/clustering.py | 2 ++ python/pyspark/mllib/linalg.py | 3 +++ python/pyspark/mllib/random.py | 3 +++ python/pyspark/mllib/recommendation.py | 2 ++ python/pyspark/mllib/regression.py | 10 +++++----- python/pyspark/mllib/stat.py | 6 ++++-- python/pyspark/mllib/tree.py | 4 ++++ python/pyspark/rdd.py | 1 + python/pyspark/serializers.py | 2 +- python/pyspark/sql.py | 21 ++++++++++++++++++--- 17 files changed, 81 insertions(+), 26 deletions(-) diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index c58555fc9d2c5..1a2e774738fe7 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -61,13 +61,17 @@ from pyspark.conf import SparkConf from pyspark.context import SparkContext -from pyspark.sql import SQLContext from pyspark.rdd import RDD -from pyspark.sql import SchemaRDD -from pyspark.sql import Row from pyspark.files import SparkFiles from pyspark.storagelevel import StorageLevel +from pyspark.accumulators import Accumulator, AccumulatorParam +from pyspark.broadcast import Broadcast +from pyspark.serializers import MarshalSerializer, PickleSerializer +# for back compatibility +from pyspark.sql import SQLContext, HiveContext, SchemaRDD, Row -__all__ = ["SparkConf", "SparkContext", "SQLContext", "RDD", "SchemaRDD", - "SparkFiles", "StorageLevel", "Row"] +__all__ = [ + "SparkConf", "SparkContext", "SparkFiles", "RDD", "StorageLevel", "Broadcast", + "Accumulator", "AccumulatorParam", "MarshalSerializer", "PickleSerializer", +] diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index f133cf6f7befc..ccbca67656c8d 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -94,6 +94,9 @@ from pyspark.serializers import read_int, PickleSerializer +__all__ = ['Accumulator', 'AccumulatorParam'] + + pickleSer = PickleSerializer() # Holds accumulators registered on the current machine, keyed by ID. This is then used to send diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 675a2fcd2ff4e..5c7c9cc161dff 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -31,6 +31,10 @@ from pyspark.serializers import CompressedSerializer, PickleSerializer + +__all__ = ['Broadcast'] + + # Holds broadcasted data received from Java, keyed by its id. _broadcastRegistry = {} @@ -59,11 +63,20 @@ def __init__(self, bid, value, java_broadcast=None, """ self.bid = bid if path is None: - self.value = value + self._value = value self._jbroadcast = java_broadcast self._pickle_registry = pickle_registry self.path = path + @property + def value(self): + """ Return the broadcasted value + """ + if not hasattr(self, "_value") and self.path is not None: + ser = CompressedSerializer(PickleSerializer()) + self._value = ser.load_stream(open(self.path)).next() + return self._value + def unpersist(self, blocking=False): self._jbroadcast.unpersist(blocking) os.unlink(self.path) @@ -72,15 +85,6 @@ def __reduce__(self): self._pickle_registry.add(self) return (_from_id, (self.bid, )) - def __getattr__(self, item): - if item == 'value' and self.path is not None: - ser = CompressedSerializer(PickleSerializer()) - value = ser.load_stream(open(self.path)).next() - self.value = value - return value - - raise AttributeError(item) - if __name__ == "__main__": import doctest diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index fb716f6753a45..b64875a3f495a 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -54,6 +54,8 @@ (u'spark.executorEnv.VAR4', u'value4'), (u'spark.home', u'/path')] """ +__all__ = ['SparkConf'] + class SparkConf(object): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6e4fdaa6eec9d..5a30431568b16 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -37,6 +37,9 @@ from py4j.java_collections import ListConverter +__all__ = ['SparkContext'] + + # These are special default configs for PySpark, they will overwrite # the default ones for Spark if they are not configured by user. DEFAULT_CONFIGS = { diff --git a/python/pyspark/files.py b/python/pyspark/files.py index 331de9a9b2212..797573f49dac8 100644 --- a/python/pyspark/files.py +++ b/python/pyspark/files.py @@ -18,6 +18,9 @@ import os +__all__ = ['SparkFiles'] + + class SparkFiles(object): """ diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index ffdda7ee19302..71ab46b61d7fa 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -30,6 +30,10 @@ from math import exp, log +__all__ = ['LogisticRegressionModel', 'LogisticRegressionWithSGD', 'SVMModel', + 'SVMWithSGD', 'NaiveBayesModel', 'NaiveBayes'] + + class LogisticRegressionModel(LinearModel): """A linear binary classification model derived from logistic regression. diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index a0630d1d5c58b..f3e952a1d842a 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -25,6 +25,8 @@ _get_initial_weights, _serialize_rating, _regression_train_wrapper from pyspark.mllib.linalg import SparseVector +__all__ = ['KMeansModel', 'KMeans'] + class KMeansModel(object): diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index f485a69db1fa2..e69051c104e37 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -27,6 +27,9 @@ from numpy import array, array_equal, ndarray, float64, int32 +__all__ = ['SparseVector', 'Vectors'] + + class SparseVector(object): """ diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 4dc1a4a912421..3e59c73db85e3 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -25,6 +25,9 @@ from pyspark.serializers import NoOpSerializer +__all__ = ['RandomRDDs', ] + + class RandomRDDs: """ Generator methods for creating RDDs comprised of i.i.d samples from diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index e863fc249ec36..2df23394da6f8 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -24,6 +24,8 @@ _serialize_tuple, RatingDeserializer from pyspark.rdd import RDD +__all__ = ['MatrixFactorizationModel', 'ALS'] + class MatrixFactorizationModel(object): diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index d8792cf44872f..f572dcfb840b6 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -17,15 +17,15 @@ from numpy import array, ndarray from pyspark import SparkContext -from pyspark.mllib._common import \ - _dot, _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ +from pyspark.mllib._common import _dot, _regression_train_wrapper, \ _linear_predictor_typecheck, _have_scipy, _scipy_issparse from pyspark.mllib.linalg import SparseVector, Vectors +__all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel' + 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] + + class LabeledPoint(object): """ diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index feef0d16cd644..8c726f171c978 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -21,8 +21,10 @@ from pyspark.mllib._common import \ _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ - _serialize_double, _serialize_double_vector, \ - _deserialize_double, _deserialize_double_matrix, _deserialize_double_vector + _serialize_double, _deserialize_double_matrix, _deserialize_double_vector + + +__all__ = ['MultivariateStatisticalSummary', 'Statistics'] class MultivariateStatisticalSummary(object): diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index e9d778df5a24b..a2fade61e9a71 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -26,6 +26,9 @@ from pyspark.serializers import NoOpSerializer +__all__ = ['DecisionTreeModel', 'DecisionTree'] + + class DecisionTreeModel(object): """ @@ -88,6 +91,7 @@ class DecisionTree(object): It will probably be modified for Spark v1.2. Example usage: + >>> from numpy import array >>> import sys >>> from pyspark.mllib.regression import LabeledPoint diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6fc9f66bc5a94..dff6fc26fcb18 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -48,6 +48,7 @@ from py4j.java_collections import ListConverter, MapConverter + __all__ = ["RDD"] diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index fc49aa42dbaf9..55e6cf3308611 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -409,7 +409,7 @@ def loads(self, obj): class CompressedSerializer(FramedSerializer): """ - compress the serialized data + Compress the serialized data """ def __init__(self, serializer): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 0ff6a548a85f1..44316926ba334 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -40,8 +40,7 @@ "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", - "SQLContext", "HiveContext", "LocalHiveContext", "TestHiveContext", - "SchemaRDD", "Row"] + "SQLContext", "HiveContext", "SchemaRDD", "Row"] class DataType(object): @@ -1037,7 +1036,7 @@ def inferSchema(self, rdd): "can not infer schema") if type(first) is dict: warnings.warn("Using RDD of dict to inferSchema is deprecated," - "please use pyspark.Row instead") + "please use pyspark.sql.Row instead") schema = _infer_schema(first) rdd = rdd.mapPartitions(lambda rows: _drop_schema(rows, schema)) @@ -1487,6 +1486,21 @@ def __repr__(self): return "" % ", ".join(self) +def inherit_doc(cls): + for name, func in vars(cls).items(): + # only inherit docstring for public functions + if name.startswith("_"): + continue + if not func.__doc__: + for parent in cls.__bases__: + parent_func = getattr(parent, name, None) + if parent_func and getattr(parent_func, "__doc__", None): + func.__doc__ = parent_func.__doc__ + break + return cls + + +@inherit_doc class SchemaRDD(RDD): """An RDD of L{Row} objects that has an associated schema. @@ -1563,6 +1577,7 @@ def registerTempTable(self, name): self._jschema_rdd.registerTempTable(name) def registerAsTable(self, name): + """DEPRECATED: use registerTempTable() instead""" warnings.warn("Use registerTempTable instead of registerAsTable.", DeprecationWarning) self.registerTempTable(name) From e5d376801d57dffb0791980a1786a0a9b45bc491 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Wed, 3 Sep 2014 14:15:22 -0700 Subject: [PATCH 0577/1492] [SPARK-3263][GraphX] Fix changes made to GraphGenerator.logNormalGraph in PR #720 PR #720 made multiple changes to GraphGenerator.logNormalGraph including: * Replacing the call to functions for generating random vertices and edges with in-line implementations with different equations. Based on reading the Pregel paper, I believe the in-line functions are incorrect. * Hard-coding of RNG seeds so that method now generates the same graph for a given number of vertices, edges, mu, and sigma -- user is not able to override seed or specify that seed should be randomly generated. * Backwards-incompatible change to logNormalGraph signature with introduction of new required parameter. * Failed to update scala docs and programming guide for API changes * Added a Synthetic Benchmark in the examples. This PR: * Removes the in-line calls and calls original vertex / edge generation functions again * Adds an optional seed parameter for deterministic behavior (when desired) * Keeps the number of partitions parameter that was added. * Keeps compatibility with the synthetic benchmark example * Maintains backwards-compatible API Author: RJ Nowling Author: Ankur Dave Closes #2168 from rnowling/graphgenrand and squashes the following commits: f1cd79f [Ankur Dave] Style fixes e11918e [RJ Nowling] Fix bad comparisons in unit tests 785ac70 [RJ Nowling] Fix style error c70868d [RJ Nowling] Fix logNormalGraph scala doc for seed 41fd1f8 [RJ Nowling] Fix logNormalGraph scala doc for seed 799f002 [RJ Nowling] Added test for different seeds for sampleLogNormal 43949ad [RJ Nowling] Added test for different seeds for generateRandomEdges 2faf75f [RJ Nowling] Added unit test for logNormalGraph 82f22397 [RJ Nowling] Add unit test for sampleLogNormal b99cba9 [RJ Nowling] Make sampleLogNormal private to Spark (vs private) for unit testing 6803da1 [RJ Nowling] Add GraphGeneratorsSuite with test for generateRandomEdges 1c8fc44 [RJ Nowling] Connected components part of SynthBenchmark was failing to call count on RDD before printing dfbb6dd [RJ Nowling] Fix parameter name in SynthBenchmark docs b5eeb80 [RJ Nowling] Add optional seed parameter to SynthBenchmark and set default to randomly generate a seed 1ff8d30 [RJ Nowling] Fix bug in generateRandomEdges where numVertices instead of numEdges was used to control number of edges to generate 98bb73c [RJ Nowling] Add documentation for logNormalGraph parameters d40141a [RJ Nowling] Fix style error 684804d [RJ Nowling] revert PR #720 which introduce errors in logNormalGraph and messed up seeding of RNGs. Add user-defined optional seed for deterministic behavior c183136 [RJ Nowling] Fix to deterministic GraphGenerators.logNormalGraph that allows generating graphs randomly using optional seed. 015010c [RJ Nowling] Fixed GraphGenerator logNormalGraph API to make backward-incompatible change in commit 894ecde04 --- .../examples/graphx/SynthBenchmark.scala | 9 +- .../spark/graphx/util/GraphGenerators.scala | 65 ++++++----- .../graphx/util/GraphGeneratorsSuite.scala | 110 ++++++++++++++++++ 3 files changed, 152 insertions(+), 32 deletions(-) create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 551c339b19523..5f35a5836462e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -38,12 +38,13 @@ object SynthBenchmark { * Options: * -app "pagerank" or "cc" for pagerank or connected components. (Default: pagerank) * -niters the number of iterations of pagerank to use (Default: 10) - * -numVertices the number of vertices in the graph (Default: 1000000) + * -nverts the number of vertices in the graph (Default: 1000000) * -numEPart the number of edge partitions in the graph (Default: number of cores) * -partStrategy the graph partitioning strategy to use * -mu the mean parameter for the log-normal graph (Default: 4.0) * -sigma the stdev parameter for the log-normal graph (Default: 1.3) * -degFile the local file to save the degree information (Default: Empty) + * -seed seed to use for RNGs (Default: -1, picks seed randomly) */ def main(args: Array[String]) { val options = args.map { @@ -62,6 +63,7 @@ object SynthBenchmark { var mu: Double = 4.0 var sigma: Double = 1.3 var degFile: String = "" + var seed: Int = -1 options.foreach { case ("app", v) => app = v @@ -72,6 +74,7 @@ object SynthBenchmark { case ("mu", v) => mu = v.toDouble case ("sigma", v) => sigma = v.toDouble case ("degFile", v) => degFile = v + case ("seed", v) => seed = v.toInt case (opt, _) => throw new IllegalArgumentException("Invalid option: " + opt) } @@ -85,7 +88,7 @@ object SynthBenchmark { // Create the graph println(s"Creating graph...") val unpartitionedGraph = GraphGenerators.logNormalGraph(sc, numVertices, - numEPart.getOrElse(sc.defaultParallelism), mu, sigma) + numEPart.getOrElse(sc.defaultParallelism), mu, sigma, seed) // Repartition the graph val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)).cache() @@ -113,7 +116,7 @@ object SynthBenchmark { println(s"Total PageRank = $totalPR") } else if (app == "cc") { println("Running Connected Components") - val numComponents = graph.connectedComponents.vertices.map(_._2).distinct() + val numComponents = graph.connectedComponents.vertices.map(_._2).distinct().count() println(s"Number of components = $numComponents") } val runTime = System.currentTimeMillis() - startTime diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index 60149548ab852..b8309289fe475 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -40,7 +40,7 @@ object GraphGenerators { val RMATd = 0.25 /** - * Generate a graph whose vertex out degree is log normal. + * Generate a graph whose vertex out degree distribution is log normal. * * The default values for mu and sigma are taken from the Pregel paper: * @@ -48,33 +48,36 @@ object GraphGenerators { * Ilan Horn, Naty Leiser, and Grzegorz Czajkowski. 2010. * Pregel: a system for large-scale graph processing. SIGMOD '10. * - * @param sc - * @param numVertices - * @param mu - * @param sigma - * @return + * If the seed is -1 (default), a random seed is chosen. Otherwise, use + * the user-specified seed. + * + * @param sc Spark Context + * @param numVertices number of vertices in generated graph + * @param numEParts (optional) number of partitions + * @param mu (optional, default: 4.0) mean of out-degree distribution + * @param sigma (optional, default: 1.3) standard deviation of out-degree distribution + * @param seed (optional, default: -1) seed for RNGs, -1 causes a random seed to be chosen + * @return Graph object */ - def logNormalGraph(sc: SparkContext, numVertices: Int, numEParts: Int, - mu: Double = 4.0, sigma: Double = 1.3): Graph[Long, Int] = { - val vertices = sc.parallelize(0 until numVertices, numEParts).map { src => - // Initialize the random number generator with the source vertex id - val rand = new Random(src) - val degree = math.min(numVertices.toLong, math.exp(rand.nextGaussian() * sigma + mu).toLong) - (src.toLong, degree) + def logNormalGraph( + sc: SparkContext, numVertices: Int, numEParts: Int = 0, mu: Double = 4.0, + sigma: Double = 1.3, seed: Long = -1): Graph[Long, Int] = { + + val evalNumEParts = if (numEParts == 0) sc.defaultParallelism else numEParts + + // Enable deterministic seeding + val seedRand = if (seed == -1) new Random() else new Random(seed) + val seed1 = seedRand.nextInt() + val seed2 = seedRand.nextInt() + + val vertices: RDD[(VertexId, Long)] = sc.parallelize(0 until numVertices, evalNumEParts).map { + src => (src, sampleLogNormal(mu, sigma, numVertices, seed = (seed1 ^ src))) } + val edges = vertices.flatMap { case (src, degree) => - new Iterator[Edge[Int]] { - // Initialize the random number generator with the source vertex id - val rand = new Random(src) - var i = 0 - override def hasNext(): Boolean = { i < degree } - override def next(): Edge[Int] = { - val nextEdge = Edge[Int](src, rand.nextInt(numVertices), i) - i += 1 - nextEdge - } - } + generateRandomEdges(src.toInt, degree.toInt, numVertices, seed = (seed2 ^ src)) } + Graph(vertices, edges, 0) } @@ -82,9 +85,10 @@ object GraphGenerators { // the edge data is the weight (default 1) val RMATc = 0.15 - def generateRandomEdges(src: Int, numEdges: Int, maxVertexId: Int): Array[Edge[Int]] = { - val rand = new Random() - Array.fill(maxVertexId) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } + def generateRandomEdges( + src: Int, numEdges: Int, maxVertexId: Int, seed: Long = -1): Array[Edge[Int]] = { + val rand = if (seed == -1) new Random() else new Random(seed) + Array.fill(numEdges) { Edge[Int](src, rand.nextInt(maxVertexId), 1) } } /** @@ -97,9 +101,12 @@ object GraphGenerators { * @param mu the mean of the normal distribution * @param sigma the standard deviation of the normal distribution * @param maxVal exclusive upper bound on the value of the sample + * @param seed optional seed */ - private def sampleLogNormal(mu: Double, sigma: Double, maxVal: Int): Int = { - val rand = new Random() + private[spark] def sampleLogNormal( + mu: Double, sigma: Double, maxVal: Int, seed: Long = -1): Int = { + val rand = if (seed == -1) new Random() else new Random(seed) + val sigmaSq = sigma * sigma val m = math.exp(mu + sigmaSq / 2.0) // expm1 is exp(m)-1 with better accuracy for tiny m diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala new file mode 100644 index 0000000000000..b346d4db2ef96 --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -0,0 +1,110 @@ +/* + * 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.graphx.util + +import org.scalatest.FunSuite + +import org.apache.spark.graphx.LocalSparkContext + +class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { + + test("GraphGenerators.generateRandomEdges") { + val src = 5 + val numEdges10 = 10 + val numEdges20 = 20 + val maxVertexId = 100 + + val edges10 = GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId) + assert(edges10.length == numEdges10) + + val correctSrc = edges10.forall(e => e.srcId == src) + assert(correctSrc) + + val correctWeight = edges10.forall(e => e.attr == 1) + assert(correctWeight) + + val correctRange = edges10.forall(e => e.dstId >= 0 && e.dstId <= maxVertexId) + assert(correctRange) + + val edges20 = GraphGenerators.generateRandomEdges(src, numEdges20, maxVertexId) + assert(edges20.length == numEdges20) + + val edges10_round1 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 12345) + val edges10_round2 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 12345) + assert(edges10_round1.zip(edges10_round2).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + + val edges10_round3 = + GraphGenerators.generateRandomEdges(src, numEdges10, maxVertexId, seed = 3467) + assert(!edges10_round1.zip(edges10_round3).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + } + + test("GraphGenerators.sampleLogNormal") { + val mu = 4.0 + val sigma = 1.3 + val maxVal = 100 + + val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) + assert(dstId < maxVal) + + val dstId_round1 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + val dstId_round2 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) + assert(dstId_round1 == dstId_round2) + + val dstId_round3 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 789) + assert(dstId_round1 != dstId_round3) + } + + test("GraphGenerators.logNormalGraph") { + withSpark { sc => + val mu = 4.0 + val sigma = 1.3 + val numVertices100 = 100 + + val graph = GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma) + assert(graph.vertices.count() == numVertices100) + + val graph_round1 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 12345) + val graph_round2 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 12345) + + val graph_round1_edges = graph_round1.edges.collect() + val graph_round2_edges = graph_round2.edges.collect() + + assert(graph_round1_edges.zip(graph_round2_edges).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + + val graph_round3 = + GraphGenerators.logNormalGraph(sc, numVertices100, mu = mu, sigma = sigma, seed = 567) + + val graph_round3_edges = graph_round3.edges.collect() + + assert(!graph_round1_edges.zip(graph_round3_edges).forall { case (e1, e2) => + e1.srcId == e2.srcId && e1.dstId == e2.dstId && e1.attr == e2.attr + }) + } + } + +} From ccc69e26ec2fadd90886990b90a5a600efd08aba Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 3 Sep 2014 14:47:11 -0700 Subject: [PATCH 0578/1492] [SPARK-2845] Add timestamps to block manager events. These are not used by the UI but are useful when analysing the logs from a spark job. Author: Marcelo Vanzin Closes #654 from vanzin/bm-event-tstamp and squashes the following commits: d5d6e66 [Marcelo Vanzin] Fix tests. ec06218 [Marcelo Vanzin] Review feedback. f134dbc [Marcelo Vanzin] Merge branch 'master' into bm-event-tstamp b495b7c [Marcelo Vanzin] Merge branch 'master' into bm-event-tstamp 7d2fe9e [Marcelo Vanzin] Review feedback. d6f381c [Marcelo Vanzin] Update tests added after patch was created. 45e3bf8 [Marcelo Vanzin] Fix unit test after merge. b37a10f [Marcelo Vanzin] Use === in test assertions. ef72824 [Marcelo Vanzin] Handle backwards compatibility with 1.0.0. aca1151 [Marcelo Vanzin] Fix unit test to check new fields. efdda8e [Marcelo Vanzin] Add timestamps to block manager events. --- .../spark/scheduler/SparkListener.scala | 4 +- .../storage/BlockManagerMasterActor.scala | 7 ++-- .../org/apache/spark/util/JsonProtocol.scala | 12 ++++-- .../storage/StorageStatusListenerSuite.scala | 18 ++++----- .../spark/ui/storage/StorageTabSuite.scala | 4 +- .../apache/spark/util/JsonProtocolSuite.scala | 37 +++++++++++++++++-- 6 files changed, 58 insertions(+), 24 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 86ca8445a1124..f33c2e065a200 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -67,11 +67,11 @@ case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(S extends SparkListenerEvent @DeveloperApi -case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) +case class SparkListenerBlockManagerAdded(time: Long, blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent @DeveloperApi -case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) +case class SparkListenerBlockManagerRemoved(time: Long, blockManagerId: BlockManagerId) extends SparkListenerEvent @DeveloperApi 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 3ab07703b6f85..1a6c7cb24f9ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -203,7 +203,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockLocations.remove(blockId) } } - listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) + listenerBus.post(SparkListenerBlockManagerRemoved(System.currentTimeMillis(), blockManagerId)) } private def expireDeadHosts() { @@ -325,6 +325,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + val time = System.currentTimeMillis() if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { case Some(manager) => @@ -340,9 +341,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus id.hostPort, Utils.bytesToString(maxMemSize))) blockManagerInfo(id) = - new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) + new BlockManagerInfo(id, time, maxMemSize, slaveActor) } - listenerBus.post(SparkListenerBlockManagerAdded(id, maxMemSize)) + listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) } private def updateBlockInfo( diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index a7543454eca1f..1fc536b096996 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -152,13 +152,15 @@ private[spark] object JsonProtocol { val blockManagerId = blockManagerIdToJson(blockManagerAdded.blockManagerId) ("Event" -> Utils.getFormattedClassName(blockManagerAdded)) ~ ("Block Manager ID" -> blockManagerId) ~ - ("Maximum Memory" -> blockManagerAdded.maxMem) + ("Maximum Memory" -> blockManagerAdded.maxMem) ~ + ("Timestamp" -> blockManagerAdded.time) } def blockManagerRemovedToJson(blockManagerRemoved: SparkListenerBlockManagerRemoved): JValue = { val blockManagerId = blockManagerIdToJson(blockManagerRemoved.blockManagerId) ("Event" -> Utils.getFormattedClassName(blockManagerRemoved)) ~ - ("Block Manager ID" -> blockManagerId) + ("Block Manager ID" -> blockManagerId) ~ + ("Timestamp" -> blockManagerRemoved.time) } def unpersistRDDToJson(unpersistRDD: SparkListenerUnpersistRDD): JValue = { @@ -466,12 +468,14 @@ private[spark] object JsonProtocol { def blockManagerAddedFromJson(json: JValue): SparkListenerBlockManagerAdded = { val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") val maxMem = (json \ "Maximum Memory").extract[Long] - SparkListenerBlockManagerAdded(blockManagerId, maxMem) + val time = Utils.jsonOption(json \ "Timestamp").map(_.extract[Long]).getOrElse(-1L) + SparkListenerBlockManagerAdded(time, blockManagerId, maxMem) } def blockManagerRemovedFromJson(json: JValue): SparkListenerBlockManagerRemoved = { val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") - SparkListenerBlockManagerRemoved(blockManagerId) + val time = Utils.jsonOption(json \ "Timestamp").map(_.extract[Long]).getOrElse(-1L) + SparkListenerBlockManagerRemoved(time, blockManagerId) } def unpersistRDDFromJson(json: JValue): SparkListenerUnpersistRDD = { diff --git a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala index 4e022a69c8212..3a45875391e29 100644 --- a/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/StorageStatusListenerSuite.scala @@ -36,13 +36,13 @@ class StorageStatusListenerSuite extends FunSuite { // Block manager add assert(listener.executorIdToStorageStatus.size === 0) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) assert(listener.executorIdToStorageStatus.size === 1) assert(listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus("big").blockManagerId === bm1) assert(listener.executorIdToStorageStatus("big").maxMem === 1000L) assert(listener.executorIdToStorageStatus("big").numBlocks === 0) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) assert(listener.executorIdToStorageStatus.size === 2) assert(listener.executorIdToStorageStatus.get("fat").isDefined) assert(listener.executorIdToStorageStatus("fat").blockManagerId === bm2) @@ -50,11 +50,11 @@ class StorageStatusListenerSuite extends FunSuite { assert(listener.executorIdToStorageStatus("fat").numBlocks === 0) // Block manager remove - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm1)) + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(1L, bm1)) assert(listener.executorIdToStorageStatus.size === 1) assert(!listener.executorIdToStorageStatus.get("big").isDefined) assert(listener.executorIdToStorageStatus.get("fat").isDefined) - listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(bm2)) + listener.onBlockManagerRemoved(SparkListenerBlockManagerRemoved(1L, bm2)) assert(listener.executorIdToStorageStatus.size === 0) assert(!listener.executorIdToStorageStatus.get("big").isDefined) assert(!listener.executorIdToStorageStatus.get("fat").isDefined) @@ -62,8 +62,8 @@ class StorageStatusListenerSuite extends FunSuite { test("task end without updated blocks") { val listener = new StorageStatusListener - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) val taskMetrics = new TaskMetrics // Task end with no updated blocks @@ -79,8 +79,8 @@ class StorageStatusListenerSuite extends FunSuite { test("task end with updated blocks") { val listener = new StorageStatusListener - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm2, 2000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm2, 2000L)) val taskMetrics1 = new TaskMetrics val taskMetrics2 = new TaskMetrics val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) @@ -128,7 +128,7 @@ class StorageStatusListenerSuite extends FunSuite { test("unpersist RDD") { val listener = new StorageStatusListener - listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(bm1, 1000L)) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) val taskMetrics1 = new TaskMetrics val taskMetrics2 = new TaskMetrics val block1 = (RDDBlockId(1, 1), BlockStatus(StorageLevel.DISK_ONLY, 0L, 100L, 0L)) diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index d9e9c70a8a9e7..e1bc1379b5d80 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -108,7 +108,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") - bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) + bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 3) assert(storageListener.rddInfoList.size === 0) // not cached @@ -175,7 +175,7 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val block1 = (RDDBlockId(1, 1), BlockStatus(memOnly, 200L, 0L, 0L)) taskMetrics0.updatedBlocks = Some(Seq(block0)) taskMetrics1.updatedBlocks = Some(Seq(block1)) - bus.postToAll(SparkListenerBlockManagerAdded(bm1, 1000L)) + bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener.rddInfoList.size === 0) bus.postToAll(SparkListenerTaskEnd(0, 0, "big", Success, taskInfo, taskMetrics0)) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 66a17de9ec9ce..c84bafce37f70 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -21,6 +21,9 @@ import java.util.Properties import scala.collection.Map +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.JsonAST._ import org.json4s.jackson.JsonMethods._ import org.scalatest.FunSuite @@ -52,9 +55,9 @@ class JsonProtocolSuite extends FunSuite { "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")), "Classpath Entries" -> Seq(("Super library", "/tmp/super_library")) )) - val blockManagerAdded = SparkListenerBlockManagerAdded( + val blockManagerAdded = SparkListenerBlockManagerAdded(1L, BlockManagerId("Stars", "In your multitude...", 300), 500) - val blockManagerRemoved = SparkListenerBlockManagerRemoved( + val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") @@ -151,6 +154,28 @@ class JsonProtocolSuite extends FunSuite { assert(newMetrics.inputMetrics.isEmpty) } + test("BlockManager events backward compatibility") { + // SparkListenerBlockManagerAdded/Removed in Spark 1.0.0 do not have a "time" property. + val blockManagerAdded = SparkListenerBlockManagerAdded(1L, + BlockManagerId("Stars", "In your multitude...", 300), 500) + val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, + BlockManagerId("Scarce", "to be counted...", 100)) + + val oldBmAdded = JsonProtocol.blockManagerAddedToJson(blockManagerAdded) + .removeField({ _._1 == "Timestamp" }) + + val deserializedBmAdded = JsonProtocol.blockManagerAddedFromJson(oldBmAdded) + assert(SparkListenerBlockManagerAdded(-1L, blockManagerAdded.blockManagerId, + blockManagerAdded.maxMem) === deserializedBmAdded) + + val oldBmRemoved = JsonProtocol.blockManagerRemovedToJson(blockManagerRemoved) + .removeField({ _._1 == "Timestamp" }) + + val deserializedBmRemoved = JsonProtocol.blockManagerRemovedFromJson(oldBmRemoved) + assert(SparkListenerBlockManagerRemoved(-1L, blockManagerRemoved.blockManagerId) === + deserializedBmRemoved) + } + /** -------------------------- * | Helper test running methods | @@ -242,8 +267,10 @@ class JsonProtocolSuite extends FunSuite { assertEquals(e1.environmentDetails, e2.environmentDetails) case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => assert(e1.maxMem === e2.maxMem) + assert(e1.time === e2.time) assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => + assert(e1.time === e2.time) assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => assert(e1.rddId == e2.rddId) @@ -945,7 +972,8 @@ class JsonProtocolSuite extends FunSuite { | "Host": "In your multitude...", | "Port": 300 | }, - | "Maximum Memory": 500 + | "Maximum Memory": 500, + | "Timestamp": 1 |} """ @@ -957,7 +985,8 @@ class JsonProtocolSuite extends FunSuite { | "Executor ID": "Scarce", | "Host": "to be counted...", | "Port": 100 - | } + | }, + | "Timestamp": 2 |} """ From f2b5b619a9efee91573c0e546792e68e72afce21 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 3 Sep 2014 14:57:38 -0700 Subject: [PATCH 0579/1492] [SPARK-3388] Expose aplication ID in ApplicationStart event, use it in history server. This change exposes the application ID generated by the Spark Master, Mesos or Yarn via the SparkListenerApplicationStart event. It then uses that information to expose the application via its ID in the history server, instead of using the internal directory name generated by the event logger as an application id. This allows someone who knows the application ID to easily figure out the URL for the application's entry in the HS, aside from looking better. In Yarn mode, this is used to generate a direct link from the RM application list to the Spark history server entry (thus providing a fix for SPARK-2150). Note this sort of assumes that the different managers will generate app ids that are sufficiently different from each other that clashes will not occur. Author: Marcelo Vanzin This patch had conflicts when merged, resolved by Committer: Andrew Or Closes #1218 from vanzin/yarn-hs-link-2 and squashes the following commits: 2d19f3c [Marcelo Vanzin] Review feedback. 6706d3a [Marcelo Vanzin] Implement applicationId() in base classes. 56fe42e [Marcelo Vanzin] Fix cluster mode history address, plus a cleanup. 44112a8 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 8278316 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 a86bbcf [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 a0056e6 [Marcelo Vanzin] Unbreak test. 4b10cfd [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 cb0cab2 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 25f2826 [Marcelo Vanzin] Add MIMA excludes. f0ba90f [Marcelo Vanzin] Use BufferedIterator. c90a08d [Marcelo Vanzin] Remove unused code. 3f8ec66 [Marcelo Vanzin] Review feedback. 21aa71b [Marcelo Vanzin] Fix JSON test. b022bae [Marcelo Vanzin] Undo SparkContext cleanup. c6d7478 [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 4e3483f [Marcelo Vanzin] Fix test. 57517b8 [Marcelo Vanzin] Review feedback. Mostly, more consistent use of Scala's Option. 311e49d [Marcelo Vanzin] Merge branch 'master' into yarn-hs-link-2 d35d86f [Marcelo Vanzin] Fix yarn backend after rebase. 36dc362 [Marcelo Vanzin] Don't use Iterator::takeWhile(). 0afd696 [Marcelo Vanzin] Wait until master responds before returning from start(). abc4697 [Marcelo Vanzin] Make FsHistoryProvider keep a map of applications by id. 26b266e [Marcelo Vanzin] Use Mesos framework ID as Spark application ID. b3f3664 [Marcelo Vanzin] [yarn] Make the RM link point to the app direcly in the HS. 2fb7de4 [Marcelo Vanzin] Expose the application ID in the ApplicationStart event. ed10348 [Marcelo Vanzin] Expose application id to spark context. --- .../scala/org/apache/spark/SparkContext.scala | 5 +- .../history/ApplicationHistoryProvider.scala | 6 +- .../deploy/history/FsHistoryProvider.scala | 176 +++++++++++------- .../spark/deploy/history/HistoryServer.scala | 5 +- .../scheduler/ApplicationEventListener.scala | 35 ++-- .../spark/scheduler/SchedulerBackend.scala | 8 + .../spark/scheduler/SparkListener.scala | 4 +- .../spark/scheduler/TaskScheduler.scala | 8 + .../spark/scheduler/TaskSchedulerImpl.scala | 4 + .../CoarseGrainedSchedulerBackend.scala | 4 +- .../cluster/SimrSchedulerBackend.scala | 1 + .../cluster/SparkDeploySchedulerBackend.scala | 28 +++ .../mesos/CoarseMesosSchedulerBackend.scala | 1 + .../cluster/mesos/MesosSchedulerBackend.scala | 1 + .../spark/scheduler/local/LocalBackend.scala | 1 + .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 4 +- .../scheduler/EventLoggingListenerSuite.scala | 3 +- .../spark/scheduler/ReplayListenerSuite.scala | 3 +- .../apache/spark/util/JsonProtocolSuite.scala | 9 +- project/MimaExcludes.scala | 6 +- .../spark/deploy/yarn/ApplicationMaster.scala | 24 ++- .../deploy/yarn/YarnSparkHadoopUtil.scala | 14 -- .../cluster/YarnClientSchedulerBackend.scala | 4 +- .../cluster/YarnClusterSchedulerBackend.scala | 5 +- 25 files changed, 228 insertions(+), 133 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cb4fb7cfbd32f..529febff94196 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1261,7 +1261,10 @@ class SparkContext(config: SparkConf) extends Logging { /** Post the application start event */ private def postApplicationStart() { - listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) + // Note: this code assumes that the task scheduler has been initialized and has contacted + // the cluster manager to get an application ID (in case the cluster manager provides one). + listenerBus.post(SparkListenerApplicationStart(appName, taskScheduler.applicationId(), + startTime, sparkUser)) } /** Post the application end event */ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala index a0e8bd403a41d..fbe39b27649f6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/ApplicationHistoryProvider.scala @@ -34,15 +34,15 @@ private[spark] abstract class ApplicationHistoryProvider { * * @return List of all know applications. */ - def getListing(): Seq[ApplicationHistoryInfo] + def getListing(): Iterable[ApplicationHistoryInfo] /** * Returns the Spark UI for a specific application. * * @param appId The application ID. - * @return The application's UI, or null if application is not found. + * @return The application's UI, or None if application is not found. */ - def getAppUI(appId: String): SparkUI + def getAppUI(appId: String): Option[SparkUI] /** * Called when the server is shutting down. diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 05c8a90782c74..481f6c93c6a8d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -32,6 +32,8 @@ import org.apache.spark.util.Utils private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider with Logging { + private val NOT_STARTED = "" + // Interval between each check for event log updates private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", conf.getInt("spark.history.updateInterval", 10)) * 1000 @@ -47,8 +49,15 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTimeMs = -1L - // List of applications, in order from newest to oldest. - @volatile private var appList: Seq[ApplicationHistoryInfo] = Nil + // The modification time of the newest log detected during the last scan. This is used + // to ignore logs that are older during subsequent scans, to avoid processing data that + // is already known. + private var lastModifiedTime = -1L + + // Mapping of application IDs to their metadata, in descending end time order. Apps are inserted + // into the map in order, so the LinkedHashMap maintains the correct ordering. + @volatile private var applications: mutable.LinkedHashMap[String, FsApplicationHistoryInfo] + = new mutable.LinkedHashMap() /** * A background thread that periodically checks for event log updates on disk. @@ -93,15 +102,35 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis logCheckingThread.start() } - override def getListing() = appList + override def getListing() = applications.values - override def getAppUI(appId: String): SparkUI = { + override def getAppUI(appId: String): Option[SparkUI] = { try { - val appLogDir = fs.getFileStatus(new Path(resolvedLogDir.toString, appId)) - val (_, ui) = loadAppInfo(appLogDir, renderUI = true) - ui + applications.get(appId).map { info => + val (replayBus, appListener) = createReplayBus(fs.getFileStatus( + new Path(logDir, info.logDir))) + val ui = { + val conf = this.conf.clone() + val appSecManager = new SecurityManager(conf) + new SparkUI(conf, appSecManager, replayBus, appId, + s"${HistoryServer.UI_PATH_PREFIX}/$appId") + // Do not call ui.bind() to avoid creating a new server for each application + } + + replayBus.replay() + + ui.setAppName(s"${appListener.appName.getOrElse(NOT_STARTED)} ($appId)") + + val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) + ui.getSecurityManager.setAcls(uiAclsEnabled) + // make sure to set admin acls before view acls so they are properly picked up + ui.getSecurityManager.setAdminAcls(appListener.adminAcls.getOrElse("")) + ui.getSecurityManager.setViewAcls(appListener.sparkUser.getOrElse(NOT_STARTED), + appListener.viewAcls.getOrElse("")) + ui + } } catch { - case e: FileNotFoundException => null + case e: FileNotFoundException => None } } @@ -119,84 +148,79 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis try { val logStatus = fs.listStatus(new Path(resolvedLogDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() - val logInfos = logDirs.filter { dir => - fs.isFile(new Path(dir.getPath, EventLoggingListener.APPLICATION_COMPLETE)) - } - val currentApps = Map[String, ApplicationHistoryInfo]( - appList.map(app => app.id -> app):_*) - - // For any application that either (i) is not listed or (ii) has changed since the last time - // the listing was created (defined by the log dir's modification time), load the app's info. - // Otherwise just reuse what's already in memory. - val newApps = new mutable.ArrayBuffer[ApplicationHistoryInfo](logInfos.size) - for (dir <- logInfos) { - val curr = currentApps.getOrElse(dir.getPath().getName(), null) - if (curr == null || curr.lastUpdated < getModificationTime(dir)) { + // Load all new logs from the log directory. Only directories that have a modification time + // later than the last known log directory will be loaded. + var newLastModifiedTime = lastModifiedTime + val logInfos = logDirs + .filter { dir => + if (fs.isFile(new Path(dir.getPath(), EventLoggingListener.APPLICATION_COMPLETE))) { + val modTime = getModificationTime(dir) + newLastModifiedTime = math.max(newLastModifiedTime, modTime) + modTime > lastModifiedTime + } else { + false + } + } + .flatMap { dir => try { - val (app, _) = loadAppInfo(dir, renderUI = false) - newApps += app + val (replayBus, appListener) = createReplayBus(dir) + replayBus.replay() + Some(new FsApplicationHistoryInfo( + dir.getPath().getName(), + appListener.appId.getOrElse(dir.getPath().getName()), + appListener.appName.getOrElse(NOT_STARTED), + appListener.startTime.getOrElse(-1L), + appListener.endTime.getOrElse(-1L), + getModificationTime(dir), + appListener.sparkUser.getOrElse(NOT_STARTED))) } catch { - case e: Exception => logError(s"Failed to load app info from directory $dir.") + case e: Exception => + logInfo(s"Failed to load application log data from $dir.", e) + None + } + } + .sortBy { info => -info.endTime } + + lastModifiedTime = newLastModifiedTime + + // When there are new logs, merge the new list with the existing one, maintaining + // the expected ordering (descending end time). Maintaining the order is important + // to avoid having to sort the list every time there is a request for the log list. + if (!logInfos.isEmpty) { + val newApps = new mutable.LinkedHashMap[String, FsApplicationHistoryInfo]() + def addIfAbsent(info: FsApplicationHistoryInfo) = { + if (!newApps.contains(info.id)) { + newApps += (info.id -> info) } - } else { - newApps += curr } - } - appList = newApps.sortBy { info => -info.endTime } + val newIterator = logInfos.iterator.buffered + val oldIterator = applications.values.iterator.buffered + while (newIterator.hasNext && oldIterator.hasNext) { + if (newIterator.head.endTime > oldIterator.head.endTime) { + addIfAbsent(newIterator.next) + } else { + addIfAbsent(oldIterator.next) + } + } + newIterator.foreach(addIfAbsent) + oldIterator.foreach(addIfAbsent) + + applications = newApps + } } catch { case t: Throwable => logError("Exception in checking for event log updates", t) } } - /** - * Parse the application's logs to find out the information we need to build the - * listing page. - * - * When creating the listing of available apps, there is no need to load the whole UI for the - * application. The UI is requested by the HistoryServer (by calling getAppInfo()) when the user - * clicks on a specific application. - * - * @param logDir Directory with application's log files. - * @param renderUI Whether to create the SparkUI for the application. - * @return A 2-tuple `(app info, ui)`. `ui` will be null if `renderUI` is false. - */ - private def loadAppInfo(logDir: FileStatus, renderUI: Boolean) = { - val path = logDir.getPath - val appId = path.getName + private def createReplayBus(logDir: FileStatus): (ReplayListenerBus, ApplicationEventListener) = { + val path = logDir.getPath() val elogInfo = EventLoggingListener.parseLoggingInfo(path, fs) val replayBus = new ReplayListenerBus(elogInfo.logPaths, fs, elogInfo.compressionCodec) val appListener = new ApplicationEventListener replayBus.addListener(appListener) - - val ui: SparkUI = if (renderUI) { - val conf = this.conf.clone() - val appSecManager = new SecurityManager(conf) - new SparkUI(conf, appSecManager, replayBus, appId, - HistoryServer.UI_PATH_PREFIX + s"/$appId") - // Do not call ui.bind() to avoid creating a new server for each application - } else { - null - } - - replayBus.replay() - val appInfo = ApplicationHistoryInfo( - appId, - appListener.appName, - appListener.startTime, - appListener.endTime, - getModificationTime(logDir), - appListener.sparkUser) - - if (ui != null) { - val uiAclsEnabled = conf.getBoolean("spark.history.ui.acls.enable", false) - ui.getSecurityManager.setAcls(uiAclsEnabled) - // make sure to set admin acls before view acls so properly picked up - ui.getSecurityManager.setAdminAcls(appListener.adminAcls) - ui.getSecurityManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) - } - (appInfo, ui) + (replayBus, appListener) } /** Return when this directory was last modified. */ @@ -219,3 +243,13 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def getMonotonicTimeMs() = System.nanoTime() / (1000 * 1000) } + +private class FsApplicationHistoryInfo( + val logDir: String, + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String) + extends ApplicationHistoryInfo(id, name, startTime, endTime, lastUpdated, sparkUser) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index d1a64c1912cb8..ce00c0ffd21e0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -52,10 +52,7 @@ class HistoryServer( private val appLoader = new CacheLoader[String, SparkUI] { override def load(key: String): SparkUI = { - val ui = provider.getAppUI(key) - if (ui == null) { - throw new NoSuchElementException() - } + val ui = provider.getAppUI(key).getOrElse(throw new NoSuchElementException()) attachSparkUI(ui) ui } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index 162158babc35b..6d39a5e3fa64c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -24,38 +24,31 @@ package org.apache.spark.scheduler * from multiple applications are seen, the behavior is unspecified. */ private[spark] class ApplicationEventListener extends SparkListener { - var appName = "" - var sparkUser = "" - var startTime = -1L - var endTime = -1L - var viewAcls = "" - var adminAcls = "" - - def applicationStarted = startTime != -1 - - def applicationCompleted = endTime != -1 - - def applicationDuration: Long = { - val difference = endTime - startTime - if (applicationStarted && applicationCompleted && difference > 0) difference else -1L - } + var appName: Option[String] = None + var appId: Option[String] = None + var sparkUser: Option[String] = None + var startTime: Option[Long] = None + var endTime: Option[Long] = None + var viewAcls: Option[String] = None + var adminAcls: Option[String] = None override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { - appName = applicationStart.appName - startTime = applicationStart.time - sparkUser = applicationStart.sparkUser + appName = Some(applicationStart.appName) + appId = applicationStart.appId + startTime = Some(applicationStart.time) + sparkUser = Some(applicationStart.sparkUser) } override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { - endTime = applicationEnd.time + endTime = Some(applicationEnd.time) } override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { val environmentDetails = environmentUpdate.environmentDetails val allProperties = environmentDetails("Spark Properties").toMap - viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") - adminAcls = allProperties.getOrElse("spark.admin.acls", "") + viewAcls = allProperties.get("spark.ui.view.acls") + adminAcls = allProperties.get("spark.admin.acls") } } } 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 e41e0a9841691..a0be8307eff27 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -31,4 +31,12 @@ private[spark] trait SchedulerBackend { def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = throw new UnsupportedOperationException def isReady(): Boolean = true + + /** + * The application ID associated with the job, if any. + * + * @return The application ID, or None if the backend does not provide an ID. + */ + def applicationId(): Option[String] = None + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index f33c2e065a200..86afe3bd5265f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -89,8 +89,8 @@ case class SparkListenerExecutorMetricsUpdate( extends SparkListenerEvent @DeveloperApi -case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) - extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, appId: Option[String], time: Long, + sparkUser: String) extends SparkListenerEvent @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 1a0b877c8a5e1..1c1ce666eab0f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -64,4 +64,12 @@ private[spark] trait TaskScheduler { */ def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], blockManagerId: BlockManagerId): Boolean + + /** + * The application ID associated with the job, if any. + * + * @return The application ID, or None if the backend does not provide an ID. + */ + def applicationId(): Option[String] = None + } 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 ad051e59af86d..633e892554c50 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -491,6 +491,9 @@ private[spark] class TaskSchedulerImpl( } } } + + override def applicationId(): Option[String] = backend.applicationId() + } @@ -535,4 +538,5 @@ private[spark] object TaskSchedulerImpl { retval.toList } + } 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 2a3711ae2a78c..5b5257269d92f 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 @@ -51,12 +51,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A val conf = scheduler.sc.conf private val timeout = AkkaUtils.askTimeout(conf) private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) - // Submit tasks only after (registered resources / total expected resources) + // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. var minRegisteredRatio = math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds - // if minRegisteredRatio has not yet been reached + // if minRegisteredRatio has not yet been reached val maxRegisteredWaitingTime = conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) val createTime = System.currentTimeMillis() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index bc7670f4a804d..513d74a08a47f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -69,4 +69,5 @@ private[spark] class SimrSchedulerBackend( fs.delete(new Path(driverFilePath), false) super.stop() } + } 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 32138e5246700..06872ace2ecf4 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 @@ -34,6 +34,10 @@ private[spark] class SparkDeploySchedulerBackend( var client: AppClient = null var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ + var appId: String = _ + + val registrationLock = new Object() + var registrationDone = false val maxCores = conf.getOption("spark.cores.max").map(_.toInt) val totalExpectedCores = maxCores.getOrElse(0) @@ -68,6 +72,8 @@ private[spark] class SparkDeploySchedulerBackend( client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() + + waitForRegistration() } override def stop() { @@ -81,15 +87,19 @@ private[spark] class SparkDeploySchedulerBackend( override def connected(appId: String) { logInfo("Connected to Spark cluster with app ID " + appId) + this.appId = appId + notifyContext() } override def disconnected() { + notifyContext() if (!stopping) { logWarning("Disconnected from Spark cluster! Waiting for reconnection...") } } override def dead(reason: String) { + notifyContext() if (!stopping) { logError("Application has been killed. Reason: " + reason) scheduler.error(reason) @@ -116,4 +126,22 @@ private[spark] class SparkDeploySchedulerBackend( override def sufficientResourcesRegistered(): Boolean = { totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio } + + override def applicationId(): Option[String] = Option(appId) + + private def waitForRegistration() = { + registrationLock.synchronized { + while (!registrationDone) { + registrationLock.wait() + } + } + } + + private def notifyContext() = { + registrationLock.synchronized { + registrationDone = true + registrationLock.notifyAll() + } + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 87e181e773fdf..da43ef567608c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -309,4 +309,5 @@ private[spark] class CoarseMesosSchedulerBackend( logInfo("Executor lost: %s, marking slave %s as lost".format(e.getValue, s.getValue)) slaveLost(d, s) } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 67ee4d66f151b..a9ef126f5de0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -349,4 +349,5 @@ private[spark] class MesosSchedulerBackend( // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index bec9502f20466..9ea25c2bc7090 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -114,4 +114,5 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { localActor ! StatusUpdate(taskId, state, serializedData) } + } 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 bee6dad3387e5..f0006b42aee4f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -232,7 +232,7 @@ private[spark] object UIUtils extends Logging { def listingTable[T]( headers: Seq[String], generateDataRow: T => Seq[Node], - data: Seq[T], + data: Iterable[T], fixedWidth: Boolean = false): Seq[Node] = { var listingTableClass = TABLE_CLASS diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 1fc536b096996..b0754e3ce10db 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -171,6 +171,7 @@ private[spark] object JsonProtocol { def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ ("App Name" -> applicationStart.appName) ~ + ("App ID" -> applicationStart.appId.map(JString(_)).getOrElse(JNothing)) ~ ("Timestamp" -> applicationStart.time) ~ ("User" -> applicationStart.sparkUser) } @@ -484,9 +485,10 @@ private[spark] object JsonProtocol { def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { val appName = (json \ "App Name").extract[String] + val appId = Utils.jsonOption(json \ "App ID").map(_.extract[String]) val time = (json \ "Timestamp").extract[Long] val sparkUser = (json \ "User").extract[String] - SparkListenerApplicationStart(appName, time, sparkUser) + SparkListenerApplicationStart(appName, appId, time, sparkUser) } def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 41e58a008c533..fead883793430 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -229,7 +229,8 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { val conf = getLoggingConf(logDirPath, compressionCodec) val eventLogger = new EventLoggingListener("test", conf) val listenerBus = new LiveListenerBus - val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, + 125L, "Mickey") val applicationEnd = SparkListenerApplicationEnd(1000L) // A comprehensive test on JSON de/serialization of all events is in JsonProtocolSuite diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 8f0ee9f4dbafd..7ab351d1b4d24 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -83,7 +83,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { val fstream = fileSystem.create(logFilePath) val cstream = codec.map(_.compressedOutputStream(fstream)).getOrElse(fstream) val writer = new PrintWriter(cstream) - val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", 125L, "Mickey") + val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, + 125L, "Mickey") val applicationEnd = SparkListenerApplicationEnd(1000L) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index c84bafce37f70..2b45d8b695853 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -60,7 +60,7 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved(2L, BlockManagerId("Scarce", "to be counted...", 100)) val unpersistRdd = SparkListenerUnpersistRDD(12345) - val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") + val applicationStart = SparkListenerApplicationStart("The winner of all", None, 42L, "Garfield") val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) @@ -176,6 +176,13 @@ class JsonProtocolSuite extends FunSuite { deserializedBmRemoved) } + test("SparkListenerApplicationStart backwards compatibility") { + // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. + val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user") + val oldEvent = JsonProtocol.applicationStartToJson(applicationStart) + .removeField({ _._1 == "App ID" }) + assert(applicationStart === JsonProtocol.applicationStartFromJson(oldEvent)) + } /** -------------------------- * | Helper test running methods | diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a2f1b3582ab71..855d5cc8cf3fd 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -111,6 +111,8 @@ object MimaExcludes { MimaBuild.excludeSparkClass("storage.Values") ++ MimaBuild.excludeSparkClass("storage.Entry") ++ MimaBuild.excludeSparkClass("storage.MemoryStore$Entry") ++ + // Class was missing "@DeveloperApi" annotation in 1.0. + MimaBuild.excludeSparkClass("scheduler.SparkListenerApplicationStart") ++ Seq( ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.mllib.tree.impurity.Gini.calculate"), @@ -119,14 +121,14 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleMethTypeProblem]( "org.apache.spark.mllib.tree.impurity.Variance.calculate") ) ++ - Seq ( // Package-private classes removed in SPARK-2341 + Seq( // Package-private classes removed in SPARK-2341 ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.BinaryLabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.LabelParser$"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.mllib.util.MulticlassLabelParser$") - ) ++ + ) ++ Seq( // package-private classes removed in MLlib ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.regression.GeneralizedLinearAlgorithm.org$apache$spark$mllib$regression$GeneralizedLinearAlgorithm$$prependOne") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 8c548409719da..98039a20de245 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} @@ -70,6 +71,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val sparkContextRef = new AtomicReference[SparkContext](null) final def run(): Int = { + val appAttemptId = client.getAttemptId() + if (isDriver) { // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box @@ -77,9 +80,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Set the master property to match the requested mode. System.setProperty("spark.master", "yarn-cluster") + + // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. + System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) } - logInfo("ApplicationAttemptId: " + client.getAttemptId()) + logInfo("ApplicationAttemptId: " + appAttemptId) val cleanupHook = new Runnable { override def run() { @@ -151,13 +157,20 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, sparkContextRef.compareAndSet(sc, null) } - private def registerAM(uiAddress: String, uiHistoryAddress: String) = { + private def registerAM(uiAddress: String) = { val sc = sparkContextRef.get() + + val appId = client.getAttemptId().getApplicationId().toString() + val historyAddress = + sparkConf.getOption("spark.yarn.historyServer.address") + .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}" } + .getOrElse("") + allocator = client.register(yarnConf, if (sc != null) sc.getConf else sparkConf, if (sc != null) sc.preferredNodeLocationData else Map(), uiAddress, - uiHistoryAddress) + historyAddress) allocator.allocateResources() reporterThread = launchReporterThread() @@ -175,7 +188,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIHostPort, YarnSparkHadoopUtil.getUIHistoryAddress(sc, sparkConf)) + registerAM(sc.ui.appUIHostPort) try { userThread.join() } finally { @@ -190,8 +203,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, conf = sparkConf, securityManager = securityMgr)._1 actor = waitForSparkDriver() addAmIpFilter() - registerAM(sparkConf.get("spark.driver.appUIAddress", ""), - sparkConf.get("spark.driver.appUIHistoryAddress", "")) + registerAM(sparkConf.get("spark.driver.appUIAddress", "")) // In client mode the actor will stop the reporter thread. reporterThread.join() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index ffe2731ca1d17..dc77f1236492d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -156,19 +155,6 @@ object YarnSparkHadoopUtil { } } - def getUIHistoryAddress(sc: SparkContext, conf: SparkConf) : String = { - val eventLogDir = sc.eventLogger match { - case Some(logger) => logger.getApplicationLogDir() - case None => "" - } - val historyServerAddress = conf.get("spark.yarn.historyServer.address", "") - if (historyServerAddress != "" && eventLogDir != "") { - historyServerAddress + HistoryServer.UI_PATH_PREFIX + s"/$eventLogDir" - } else { - "" - } - } - /** * Escapes a string for inclusion in a command line executed by Yarn. Yarn executes commands * using `bash -c "command arg1 arg2"` and that means plain quoting doesn't really work. The 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 a5f537dd9de30..41c662cd7a6de 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 @@ -56,7 +56,6 @@ private[spark] class YarnClientSchedulerBackend( val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort conf.set("spark.driver.appUIAddress", sc.ui.appUIHostPort) - conf.set("spark.driver.appUIHistoryAddress", YarnSparkHadoopUtil.getUIHistoryAddress(sc, conf)) val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( @@ -150,4 +149,7 @@ private[spark] class YarnClientSchedulerBackend( override def sufficientResourcesRegistered(): Boolean = { totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } + + override def applicationId(): Option[String] = Option(appId).map(_.toString()) + } 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 index 55665220a6f96..39436d0999663 100644 --- 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 @@ -28,7 +28,7 @@ private[spark] class YarnClusterSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { var totalExpectedExecutors = 0 - + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { minRegisteredRatio = 0.8 } @@ -47,4 +47,7 @@ private[spark] class YarnClusterSchedulerBackend( override def sufficientResourcesRegistered(): Boolean = { totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } + + override def applicationId(): Option[String] = sc.getConf.getOption("spark.yarn.app.id") + } From 2784822e4c63083a647cc2d6c7089065ef3b947d Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 3 Sep 2014 16:58:19 -0700 Subject: [PATCH 0580/1492] [Minor] Fix outdated Spark version This is causing the event logs to include a file called SPARK_VERSION_1.0.0, which is not accurate. Author: Andrew Or Author: andrewor14 Closes #2255 from andrewor14/spark-version and squashes the following commits: 1fbdfe9 [andrewor14] Snapshot 805a1c8 [Andrew Or] JK. Update Spark version to 1.2.0 instead. bffbaab [Andrew Or] Update Spark version to 1.1.0 --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 529febff94196..6eaf6794764c7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1297,7 +1297,7 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { - private[spark] val SPARK_VERSION = "1.0.0" + private[spark] val SPARK_VERSION = "1.2.0-SNAPSHOT" private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" From 996b7434ee0d0c7c26987eb9cf050c139fdd2db2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 3 Sep 2014 17:04:53 -0700 Subject: [PATCH 0581/1492] [SPARK-3345] Do correct parameters for ShuffleFileGroup In the method `newFileGroup` of class `FileShuffleBlockManager`, the parameters for creating new `ShuffleFileGroup` object is in wrong order. Because in current codes, the parameters `shuffleId` and `fileId` are not used. So it doesn't cause problem now. However it should be corrected for readability and avoid future problem. Author: Liang-Chi Hsieh Closes #2235 from viirya/correct_shufflefilegroup_params and squashes the following commits: fe72567 [Liang-Chi Hsieh] Do correct parameters for ShuffleFileGroup. --- .../org/apache/spark/shuffle/FileShuffleBlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 76e3932a9bb91..96facccd52373 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -155,7 +155,7 @@ class FileShuffleBlockManager(conf: SparkConf) val filename = physicalFileName(shuffleId, bucketId, fileId) blockManager.diskBlockManager.getFile(filename) } - val fileGroup = new ShuffleFileGroup(fileId, shuffleId, files) + val fileGroup = new ShuffleFileGroup(shuffleId, fileId, files) shuffleState.allFileGroups.add(fileGroup) fileGroup } From a5224079286d1777864cf9fa77330aadae10cd7b Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 3 Sep 2014 17:38:01 -0700 Subject: [PATCH 0582/1492] [SPARK-2419][Streaming][Docs] Updates to the streaming programming guide Updated the main streaming programming guide, and also added source-specific guides for Kafka, Flume, Kinesis. Author: Tathagata Das Author: Jacek Laskowski Closes #2254 from tdas/streaming-doc-fix and squashes the following commits: e45c6d7 [Jacek Laskowski] More fixes from an old PR 5125316 [Tathagata Das] Fixed links dc02f26 [Tathagata Das] Refactored streaming kinesis guide and made many other changes. acbc3e3 [Tathagata Das] Fixed links between streaming guides. cb7007f [Tathagata Das] Added Streaming + Flume integration guide. 9bd9407 [Tathagata Das] Updated streaming programming guide with additional information from SPARK-2419. --- docs/streaming-flume-integration.md | 132 +++++++ docs/streaming-kafka-integration.md | 42 +++ docs/streaming-kinesis-integration.md | 110 ++++++ docs/streaming-kinesis.md | 59 --- docs/streaming-programming-guide.md | 518 +++++++++++++++++--------- 5 files changed, 622 insertions(+), 239 deletions(-) create mode 100644 docs/streaming-flume-integration.md create mode 100644 docs/streaming-kafka-integration.md create mode 100644 docs/streaming-kinesis-integration.md delete mode 100644 docs/streaming-kinesis.md diff --git a/docs/streaming-flume-integration.md b/docs/streaming-flume-integration.md new file mode 100644 index 0000000000000..d57c3e0ef9ba0 --- /dev/null +++ b/docs/streaming-flume-integration.md @@ -0,0 +1,132 @@ +--- +layout: global +title: Spark Streaming + Flume Integration Guide +--- + +[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and available service for efficiently collecting, aggregating, and moving large amounts of log data. Here we explain how to configure Flume and Spark Streaming to receive data from Flume. There are two approaches to this. + +## Approach 1: Flume-style Push-based Approach +Flume is designed to push data between Flume agents. In this approach, Spark Streaming essentially sets up a receiver that acts an Avro agent for Flume, to which Flume can push the data. Here are the configuration steps. + +#### General Requirements +Choose a machine in your cluster such that + +- When your Flume + Spark Streaming application is launched, one of the Spark workers must run on that machine. + +- Flume can be configured to push data to a port on that machine. + +Due to the push model, the streaming application needs to be up, with the receiver scheduled and listening on the chosen port, for Flume to be able push data. + +#### Configuring Flume +Configure Flume agent to send data to an Avro sink by having the following in the configuration file. + + agent.sinks = avroSink + agent.sinks.avroSink.type = avro + agent.sinks.avroSink.channel = memoryChannel + agent.sinks.avroSink.hostname = + agent.sinks.avroSink.port = + +See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about +configuring Flume agents. + +#### Configuring Spark Streaming Application +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-flume_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. + +
    +
    + import org.apache.spark.streaming.flume._ + + val flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala). +
    +
    + import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStream flumeStream = + FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java). +
    +
    + + Note that the hostname should be the same as the one used by the resource manager in the + cluster (Mesos, YARN or Spark Standalone), so that resource allocation can match the names and launch + the receiver in the right machine. + +3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + +## Approach 2 (Experimental): Pull-based Approach using a Custom Sink +Instead of Flume pushing data directly to Spark Streaming, this approach runs a custom Flume sink that allows the following. +- Flume pushes data into the sink, and the data stays buffered. +- Spark Streaming uses transactions to pull data from the sink. Transactions succeed only after data is received and replicated by Spark Streaming. +This ensures that better reliability and fault-tolerance than the previous approach. However, this requires configuring Flume to run a custom sink. Here are the configuration steps. + +#### General Requirements +Choose a machine that will run the custom sink in a Flume agent. The rest of the Flume pipeline is configured to send data to that agent. Machines in the Spark cluster should have access to the chosen machine running the custom sink. + +#### Configuring Flume +Configuring Flume on the chosen machine requires the following two steps. + +1. **Sink JARs**: Add the following JARs to Flume's classpath (see [Flume's documentation](https://flume.apache.org/documentation.html) to see how) in the machine designated to run the custom sink . + + (i) *Custom sink JAR*: Download the JAR corresponding to the following artifact (or [direct link](http://search.maven.org/remotecontent?filepath=org/apache/spark/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}/{{site.SPARK_VERSION_SHORT}}/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION_SHORT}}.jar)). + + groupId = org.apache.spark + artifactId = spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + (ii) *Scala library JAR*: Download the Scala library JAR for Scala {{site.SCALA_VERSION}}. It can be found with the following artifact detail (or, [direct link](http://search.maven.org/remotecontent?filepath=org/scala-lang/scala-library/{{site.SCALA_VERSION}}/scala-library-{{site.SCALA_VERSION}}.jar)). + + groupId = org.scala-lang + artifactId = scala-library + version = {{site.SCALA_VERSION}} + +2. **Configuration file**: On that machine, configure Flume agent to send data to an Avro sink by having the following in the configuration file. + + agent.sinks = spark + agent.sinks.spark.type = org.apache.spark.streaming.flume.sink.SparkSink + agent.sinks.spark.hostname = + agent.sinks.spark.port = + agent.sinks.spark.channel = memoryChannel + + Also make sure that the upstream Flume pipeline is configured to send the data to the Flume agent running this sink. + +See the [Flume's documentation](https://flume.apache.org/documentation.html) for more information about +configuring Flume agents. + +#### Configuring Spark Streaming Application +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide). + +2. **Programming:** In the streaming application code, import `FlumeUtils` and create input DStream as follows. + +
    +
    + import org.apache.spark.streaming.flume._ + + val flumeStream = FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]) +
    +
    + import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStreamflumeStream = + FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]); +
    +
    + + See the Scala example [FlumePollingEventCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala). + + Note that each input DStream can be configured to receive data from multiple sinks. + +3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + + diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md new file mode 100644 index 0000000000000..a3b705d4c31d0 --- /dev/null +++ b/docs/streaming-kafka-integration.md @@ -0,0 +1,42 @@ +--- +layout: global +title: Spark Streaming + Kafka Integration Guide +--- +[Apache Kafka](http://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Here we explain how to configure Spark Streaming to receive data from Kafka. + +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + +2. **Programming:** In the streaming application code, import `KafkaUtils` and create input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kafka._ + + val kafkaStream = KafkaUtils.createStream( + streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala). +
    +
    + import org.apache.spark.streaming.kafka.*; + + JavaPairReceiverInputDStream kafkaStream = KafkaUtils.createStream( + streamingContext, [zookeeperQuorum], [group id of the consumer], [per-topic number of Kafka partitions to consume]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java). +
    +
    + + *Points to remember:* + + - Topic partitions in Kafka does not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. + + - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. + +3. **Deploying:** Package `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md new file mode 100644 index 0000000000000..079d4c5550537 --- /dev/null +++ b/docs/streaming-kinesis-integration.md @@ -0,0 +1,110 @@ +--- +layout: global +title: Spark Streaming + Kinesis Integration +--- +[Amazon Kinesis](http://aws.amazon.com/kinesis/) is a fully managed service for real-time processing of streaming data at massive scale. +The Kinesis input DStream and receiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License (ASL). +The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases. +Here we explain how to configure Spark Streaming to receive data from Kinesis. + +#### Configuring Kinesis + +A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or more shards per the following +[guide](http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html). + + +#### Configuring Spark Streaming Application + +1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). + + groupId = org.apache.spark + artifactId = spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} + version = {{site.SPARK_VERSION_SHORT}} + + **Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your application.** + +2. **Programming:** In the streaming application code, import `KinesisUtils` and create input DStream as follows. + +
    +
    + import org.apache.spark.streaming.kinesis._ + import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream + + val kinesisStream = KinesisUtils.createStream( + streamingContext, [Kinesis stream name], [endpoint URL], [checkpoint interval], [initial position]) + + See the [API docs](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala). Refer to the next subsection for instructions to run the example. + +
    +
    + import org.apache.spark.streaming.flume.*; + + JavaReceiverInputDStream kinesisStream = KinesisUtils.createStream( + streamingContext, [Kinesis stream name], [endpoint URL], [checkpoint interval], [initial position]); + + See the [API docs](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) + and the [example]({{site.SPARK_GITHUB_URL}}/tree/master/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java). Refer to the next subsection for instructions to run the example. + +
    +
    + + `[endpoint URL]`: Valid Kinesis endpoints URL can be found [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region). + + `[checkpoint interval]`: The interval at which the Kinesis client library is going to save its position in the stream. For starters, set it to the same as the batch interval of the streaming application. + + `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see later section and Amazon Kinesis API documentation for more details). + + *Points to remember:* + + - The name used in the context of the streaming application must be unique for a given account and region. Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream. + - A single Kinesis input DStream can receive many Kinesis shards by spinning up multiple KinesisRecordProcessor threads. Note that there is no correlation between number of shards in Kinesis and the number of partitions in the generated RDDs that is used for processing the data. + - You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread. + - Horizontal scaling is achieved by autoscaling additional Kinesis input DStreams (separate processes) up to the number of current shards for a given stream, of course. + +3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + - A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL. It is used to store KCL's checkpoint information. + + - If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch. + +#### Running the Example +To run the example, +- Download Spark source and follow the [instructions](building-with-maven.html) to build Spark with profile *-Pkinesis-asl*. + + mvn -Pkinesis-asl -DskipTests clean package + +- Set up Kinesis stream (see earlier section). Note the name of the Kinesis stream, and the endpoint URL corresponding to the region the stream is based on. + +- Set up the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_KEY with your AWS credentials. + +- In the Spark root directory, run the example as +
    +
    + + bin/run-example streaming.KinesisWordCountASL [Kinesis stream name] [endpoint URL] + +
    +
    + + bin/run-example streaming.JavaKinesisWordCountASL [Kinesis stream name] [endpoint URL] + +
    +
    + + This will wait for data to be received from Kinesis. + +- To generate random string data, in another terminal, run the associated Kinesis data producer. + + bin/run-example streaming.KinesisWordCountProducerASL [Kinesis stream name] [endpoint URL] 1000 10 + + This will push random words to the Kinesis stream, which should then be received and processed by the running example. + +#### Kinesis Checkpointing +The Kinesis receiver checkpoints the position of the stream that has been read periodically, so that the system can recover from failures and continue processing where it had left off. Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. + +- If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable. + +- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored). In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data. + +- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency. diff --git a/docs/streaming-kinesis.md b/docs/streaming-kinesis.md deleted file mode 100644 index 16ad3222105a2..0000000000000 --- a/docs/streaming-kinesis.md +++ /dev/null @@ -1,59 +0,0 @@ ---- -layout: global -title: Spark Streaming Kinesis Receiver ---- - -## Kinesis -###Design -
  • The KinesisReceiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License.
  • -
  • The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases.
  • -
  • The KCL uses DynamoDB to maintain all state. A DynamoDB table is created in the us-east-1 region (regardless of Kinesis stream region) during KCL initialization for each Kinesis application name.
  • -
  • A single KinesisReceiver can process many shards of a stream by spinning up multiple KinesisRecordProcessor threads.
  • -
  • You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread.
  • -
  • Horizontal scaling is achieved by autoscaling additional KinesisReceiver (separate processes) or spinning up new KinesisRecordProcessor threads within each KinesisReceiver - up to the number of current shards for a given stream, of course. Don't forget to autoscale back down!
  • - -### Build -
  • Spark supports a Streaming KinesisReceiver, but it is not included in the default build due to Amazon Software Licensing (ASL) restrictions.
  • -
  • To build with the Kinesis Streaming Receiver and supporting ASL-licensed code, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • -
  • All KinesisReceiver-related code, examples, tests, and artifacts live in **$SPARK_HOME/extras/kinesis-asl/**.
  • -
  • Kinesis-based Spark Applications will need to link to the **spark-streaming-kinesis-asl** artifact that is built when **-Pkinesis-asl** is specified.
  • -
  • _**Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your Spark package**_.
  • - -###Example -
  • To build the Kinesis example, you must run the maven or sbt builds with the **-Pkinesis-asl** profile.
  • -
  • You need to setup a Kinesis stream at one of the valid Kinesis endpoints with 1 or more shards per the following: http://docs.aws.amazon.com/kinesis/latest/dev/step-one-create-stream.html
  • -
  • Valid Kinesis endpoints can be found here: http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region
  • -
  • When running **locally**, the example automatically determines the number of threads and KinesisReceivers to spin up based on the number of shards configured for the stream. Therefore, **local[n]** is not needed when starting the example as with other streaming examples.
  • -
  • While this example could use a single KinesisReceiver which spins up multiple KinesisRecordProcessor threads to process multiple shards, I wanted to demonstrate unioning multiple KinesisReceivers as a single DStream. (It's a bit confusing in local mode.)
  • -
  • **KinesisWordCountProducerASL** is provided to generate random records into the Kinesis stream for testing.
  • -
  • The example has been configured to immediately replicate incoming stream data to another node by using (StorageLevel.MEMORY_AND_DISK_2) -
  • Spark checkpointing is disabled because the example does not use any stateful or window-based DStream operations such as updateStateByKey and reduceByWindow. If those operations are introduced, you would need to enable checkpointing or risk losing data in the case of a failure.
  • -
  • Kinesis checkpointing is enabled. This means that the example will recover from a Kinesis failure.
  • -
  • The example uses InitialPositionInStream.LATEST strategy to pull from the latest tip of the stream if no Kinesis checkpoint info exists.
  • -
  • In our example, **KinesisWordCount** is the Kinesis application name for both the Scala and Java versions. The use of this application name is described next.
  • - -###Deployment and Runtime -
  • A Kinesis application name must be unique for a given account and region.
  • -
  • A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. http://docs.aws.amazon.com/kinesis/latest/dev/kinesis-record-processor-implementation-app.html#kinesis-record-processor-initialization
  • -
  • This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL.
  • -
  • Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream.
  • -
  • If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch.
  • -
  • The Kinesis libraries must be present on all worker nodes, as they will need access to the KCL.
  • -
  • The KinesisReceiver uses the DefaultAWSCredentialsProviderChain for AWS credentials which searches for credentials in the following order of precedence:
    -1) Environment Variables - AWS_ACCESS_KEY_ID and AWS_SECRET_KEY
    -2) Java System Properties - aws.accessKeyId and aws.secretKey
    -3) Credential profiles file - default location (~/.aws/credentials) shared by all AWS SDKs
    -4) Instance profile credentials - delivered through the Amazon EC2 metadata service -
  • - -###Fault-Tolerance -
  • The combination of Spark Streaming and Kinesis creates 2 different checkpoints that may occur at different intervals.
  • -
  • Checkpointing too frequently against Kinesis will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random backoff retry strategy.
  • -
  • Upon startup, a KinesisReceiver will begin processing records with sequence numbers greater than the last Kinesis checkpoint sequence number recorded per shard (stored in the DynamoDB table).
  • -
  • If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable.
  • -
  • InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored.)
  • -
  • In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data.
  • -
  • InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency.
  • -
  • Record processing should be idempotent when possible.
  • -
  • A failed or latent KinesisRecordProcessor within the KinesisReceiver will be detected and automatically restarted by the KCL.
  • -
  • If possible, the KinesisReceiver should be shutdown cleanly in order to trigger a final checkpoint of all KinesisRecordProcessors to avoid duplicate record processing.
  • \ No newline at end of file diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 9f331ed50d2a4..3d4bce49666ed 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -7,12 +7,12 @@ title: Spark Streaming Programming Guide {:toc} # Overview -Spark Streaming is an extension of the core Spark API that allows enables high-throughput, +Spark Streaming is an extension of the core Spark API that allows enables scalable, high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources like Kafka, Flume, Twitter, ZeroMQ, Kinesis or plain old TCP sockets and be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, -and live dashboards. In fact, you can apply Spark's in-built +and live dashboards. In fact, you can apply Spark's [machine learning](mllib-guide.html) algorithms, and [graph processing](graphx-programming-guide.html) algorithms on data streams. @@ -60,35 +60,24 @@ do is as follows.
    First, we import the names of the Spark Streaming classes, and some implicit conversions from StreamingContext into our environment, to add useful methods to -other classes we need (like DStream). - -[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the -main entry point for all streaming functionality. +other classes we need (like DStream). [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the +main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and batch interval of 1 second. {% highlight scala %} +import org.apache.spark._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ -{% endhighlight %} - -Then we create a -[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object. -Besides Spark's configuration, we specify that any DStream will be processed -in 1 second batches. -{% highlight scala %} -import org.apache.spark.api.java.function._ -import org.apache.spark.streaming._ -import org.apache.spark.streaming.api._ -// Create a StreamingContext with a local master -// Spark Streaming needs at least two working thread -val ssc = new StreamingContext("local[2]", "NetworkWordCount", Seconds(1)) +// Create a local StreamingContext with two working thread and batch interval of 1 second +val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") +val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %} -Using this context, we then create a new DStream -by specifying the IP address and port of the data server. +Using this context, we can create a DStream that represents streaming data from a TCP +source hostname, e.g. `localhost`, and port, e.g. `9999` {% highlight scala %} -// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +// Create a DStream that will connect to hostname:port, like localhost:9999 val lines = ssc.socketTextStream("localhost", 9999) {% endhighlight %} @@ -112,7 +101,7 @@ import org.apache.spark.streaming.StreamingContext._ val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) -// Print a few of the counts to the console +// Print the first ten elements of each RDD generated in this DStream to the console wordCounts.print() {% endhighlight %} @@ -139,23 +128,25 @@ The complete code can be found in the Spark Streaming example First, we create a [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) object, which is the main entry point for all streaming -functionality. Besides Spark's configuration, we specify that any DStream would be processed -in 1 second batches. +functionality. We create a local StreamingContext with two execution threads, and a batch interval of 1 second. {% highlight java %} +import org.apache.spark.*; import org.apache.spark.api.java.function.*; import org.apache.spark.streaming.*; import org.apache.spark.streaming.api.java.*; import scala.Tuple2; -// Create a StreamingContext with a local master -JavaStreamingContext jssc = new JavaStreamingContext("local[2]", "JavaNetworkWordCount", new Duration(1000)) + +// Create a local StreamingContext with two working thread and batch interval of 1 second +val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") +JavaStreamingContext jssc = new JavaStreamingContext(conf, new Duration(1000)) {% endhighlight %} -Using this context, we then create a new DStream -by specifying the IP address and port of the data server. +Using this context, we can create a DStream that represents streaming data from a TCP +source hostname, e.g. `localhost`, and port, e.g. `9999` {% highlight java %} -// Create a DStream that will connect to serverIP:serverPort, like localhost:9999 +// Create a DStream that will connect to hostname:port, like localhost:9999 JavaReceiverInputDStream lines = jssc.socketTextStream("localhost", 9999); {% endhighlight %} @@ -197,7 +188,9 @@ JavaPairDStream wordCounts = pairs.reduceByKey( return i1 + i2; } }); -wordCounts.print(); // Print a few of the counts to the console + +// Print the first ten elements of each RDD generated in this DStream to the console +wordCounts.print(); {% endhighlight %} The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, @@ -207,8 +200,8 @@ using a [Function2](api/scala/index.html#org.apache.spark.api.java.function.Func Finally, `wordCounts.print()` will print a few of the counts generated every second. Note that when these lines are executed, Spark Streaming only sets up the computation it -will perform when it is started, and no real processing has started yet. To start the processing -after all the transformations have been setup, we finally call +will perform after it is started, and no real processing has started yet. To start the processing +after all the transformations have been setup, we finally call `start` method. {% highlight java %} jssc.start(); // Start the computation @@ -235,12 +228,12 @@ Then, in a different terminal, you can start the example by using
    {% highlight bash %} -$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999 +$ ./bin/run-example streaming.NetworkWordCount localhost 9999 {% endhighlight %}
    {% highlight bash %} -$ ./bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount localhost 9999 +$ ./bin/run-example JavaNetworkWordCount localhost 9999 {% endhighlight %}
    @@ -281,25 +274,11 @@ Time: 1357008430000 ms -You can also use Spark Streaming directly from the Spark shell: - -{% highlight bash %} -$ bin/spark-shell -{% endhighlight %} - -... and create your StreamingContext by wrapping the existing interactive shell -SparkContext object, `sc`: - -{% highlight scala %} -val ssc = new StreamingContext(sc, Seconds(1)) -{% endhighlight %} -When working with the shell, you may also need to send a `^D` to your netcat session -to force the pipeline to print the word counts to the console at the sink. - -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** -# Basics +# Basic Concepts Next, we move beyond the simple example and elaborate on the basics of Spark Streaming that you need to know to write your streaming applications. @@ -319,68 +298,120 @@ Streaming core artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the dependencies. For example, some of the common ones are as follows. - + - - +
    SourceArtifact
    Kafka spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
    Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}
    Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}
    ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}}
    MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    (built separately)
    kinesis-asl_{{site.SCALA_BINARY_VERSION}}
    For an up-to-date list, please refer to the -[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION}}%22) +[Apache repository](http://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.spark%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) for the full list of supported sources and artifacts. -## Initializing +*** + +## Initializing StreamingContext + +To initialize a Spark Streaming program, a **StreamingContext** object has to be created which is the main entry point of all Spark Streaming functionality.
    -To initialize a Spark Streaming program in Scala, a -[`StreamingContext`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) -object has to be created, which is the main entry point of all Spark Streaming functionality. -A `StreamingContext` object can be created by using +A [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object can be created from a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object. {% highlight scala %} -new StreamingContext(master, appName, batchDuration, [sparkHome], [jars]) +import org.apache.spark._ +import org.apache.spark.streaming._ + +val conf = new SparkConf().setAppName(appName).setMaster(master) +val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %} -
    -
    -To initialize a Spark Streaming program in Java, a -[`JavaStreamingContext`](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) -object has to be created, which is the main entry point of all Spark Streaming functionality. -A `JavaStreamingContext` object can be created by using +The `appName` parameter is a name for your application to show on the cluster UI. +`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +or a special __"local[\*]"__ string to run in local mode. In practice, when running on a cluster, +you will not want to hardcode `master` in the program, +but rather [launch the application with `spark-submit`](submitting-applications.html) and +receive it there. However, for local testing and unit tests, you can pass "local[\*]" to run Spark Streaming +in-process (detects the number of cores in the local system). Note that this internally creates a [SparkContext](api/scala/index.html#org.apache.spark.SparkContext) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. + +The batch interval must be set based on the latency requirements of your application +and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) +section for more details. + +A `StreamingContext` object can also be created from an existing `SparkContext` object. {% highlight scala %} -new JavaStreamingContext(master, appName, batchInterval, [sparkHome], [jars]) +import org.apache.spark.streaming._ + +val sc = ... // existing SparkContext +val ssc = new StreamingContext(sc, Seconds(1)) {% endhighlight %} + +
    -
    +
    -The `master` parameter is a standard [Spark cluster URL](programming-guide.html#master-urls) -and can be "local" for local testing. The `appName` is a name of your program, -which will be shown on your cluster's web UI. The `batchInterval` is the size of the batches, -as explained earlier. Finally, the last two parameters are needed to deploy your code to a cluster - if running in distributed mode, as described in the - [Spark programming guide](programming-guide.html#deploying-code-on-a-cluster). - Additionally, the underlying SparkContext can be accessed as -`ssc.sparkContext`. +A [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) object can be created from a [SparkConf](api/java/index.html?org/apache/spark/SparkConf.html) object. + +{% highlight java %} +import org.apache.spark.*; +import org.apache.spark.streaming.api.java.*; + +SparkConf conf = new SparkConf().setAppName(appName).setMaster(master); +JavaStreamingContext ssc = new JavaStreamingContext(conf, Duration(1000)); +{% endhighlight %} + +The `appName` parameter is a name for your application to show on the cluster UI. +`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +or a special __"local[\*]"__ string to run in local mode. In practice, when running on a cluster, +you will not want to hardcode `master` in the program, +but rather [launch the application with `spark-submit`](submitting-applications.html) and +receive it there. However, for local testing and unit tests, you can pass "local[*]" to run Spark Streaming +in-process. Note that this internally creates a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) section for more details. -## DStreams -*Discretized Stream* or *DStream* is the basic abstraction provided by Spark Streaming. +A `JavaStreamingContext` object can also be created from an existing `JavaSparkContext`. + +{% highlight java %} +import org.apache.spark.streaming.api.java.*; + +JavaSparkContext sc = ... //existing JavaSparkContext +JavaStreamingContext ssc = new JavaStreamingContext(sc, new Duration(1000)); +{% endhighlight %} +
    +
    + +After a context is defined, you have to do the follow steps. +1. Define the input sources. +1. Setup the streaming computations. +1. Start the receiving and procesing of data using `streamingContext.start()`. +1. The processing will continue until `streamingContext.stop()` is called. + +##### Points to remember: +{:.no_toc} +- Once a context has been started, no new streaming computations can be setup or added to it. +- Once a context has been stopped, it cannot be started (that is, re-used) again. +- Only one StreamingContext can be active in a JVM at the same time. +- stop() on StreamingContext also stops the SparkContext. To stop only the StreamingContext, set optional parameter of `stop()` called `stopSparkContext` to false. +- A SparkContext can be re-used to create multiple StreamingContexts, as long as the previous StreamingContext is stopped (without stopping the SparkContext) before the next StreamingContext is created. + +*** + +## Discretized Streams (DStreams) +**Discretized Stream** or **DStream** is the basic abstraction provided by Spark Streaming. It represents a continuous stream of data, either the input data stream received from source, or the processed data stream generated by transforming the input stream. Internally, -it is represented by a continuous sequence of RDDs, which is Spark's abstraction of an immutable, -distributed dataset. Each RDD in a DStream contains data from a certain interval, +a DStream is represented by a continuous series of RDDs, which is Spark's abstraction of an immutable, +distributed dataset (see [Spark Programming Guide](programming-guide.html#resilient-distributed-datasets-rdds) for more details). Each RDD in a DStream contains data from a certain interval, as shown in the following figure.

    @@ -392,8 +423,8 @@ as shown in the following figure. Any operation applied on a DStream translates to operations on the underlying RDDs. For example, in the [earlier example](#a-quick-example) of converting a stream of lines to words, -the `flatmap` operation is applied on each RDD in the `lines` DStream to generate the RDDs of the - `words` DStream. This is shown the following figure. +the `flatMap` operation is applied on each RDD in the `lines` DStream to generate the RDDs of the + `words` DStream. This is shown in the following figure.

    -

    -{% highlight scala %} -ssc.fileStream(dataDirectory) -{% endhighlight %} -
    -
    -{% highlight java %} -jssc.fileStream(dataDirectory); -{% endhighlight %} -
    -
    +
    +
    + streamingContext.fileStream[keyClass, valueClass, inputFormatClass](dataDirectory) +
    +
    + streamingContext.fileStream(dataDirectory); +
    +
    -Spark Streaming will monitor the directory `dataDirectory` for any Hadoop-compatible filesystem -and process any files created in that directory. Note that + Spark Streaming will monitor the directory `dataDirectory` and process any files created in that directory (files written in nested directories not supported). Note that - * The files must have the same data format. - * The files must be created in the `dataDirectory` by atomically *moving* or *renaming* them into - the data directory. - * Once moved the files must not be changed. + + The files must have the same data format. + + The files must be created in the `dataDirectory` by atomically *moving* or *renaming* them into + the data directory. + + Once moved, the files must not be changed. So if the files are being continuously appended, the new data will not be read. -For more details on streams from files, Akka actors and sockets, + For simple text files, there is an easier method `streamingContext.textFileStream(dataDirectory)`. And file streams do not require running a receiver, hence does not require allocating cores. + +- **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver Guide](#implementing-and-using-a-custom-actor-based-receiver) for more details. + +- **Queue of RDDs as a Stream:** For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using `streamingContext.queueStream(queueOfRDDs)`. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream. + +For more details on streams from sockets, files, and actors, see the API documentations of the relevant functions in [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) for -Scala and [JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) - for Java. +Scala and [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) for Java. + +### Advanced Sources +{:.no_toc} +This category of sources require interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources have been moved to separate libraries, that can be [linked to](#linking) explicitly as necessary. For example, if you want to create a DStream using data from Twitter's stream of tweets, you have to do the following. -Additional functionality for creating DStreams from sources such as Kafka, Flume, Kinesis, and Twitter -can be imported by adding the right dependencies as explained in an -[earlier](#linking) section. To take the -case of Kafka, after adding the artifact `spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}` to the -project dependencies, you can create a DStream from Kafka as +1. *Linking*: Add the artifact `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` to the SBT/Maven project dependencies. +1. *Programming*: Import the `TwitterUtils` class and create a DStream with `TwitterUtils.createStream` as shown below. +1. *Deploying*: Generate an uber JAR with all the dependencies (including the dependency `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` and its transitive dependencies) and then deploy the application. This is further explained in the [Deploying section](#deploying-applications).
    {% highlight scala %} -import org.apache.spark.streaming.kafka._ -KafkaUtils.createStream(ssc, kafkaParams, ...) +import org.apache.spark.streaming.twitter._ + +TwitterUtils.createStream(ssc) {% endhighlight %}
    {% highlight java %} -import org.apache.spark.streaming.kafka.*; -KafkaUtils.createStream(jssc, kafkaParams, ...); +import org.apache.spark.streaming.twitter.*; + +TwitterUtils.createStream(jssc); {% endhighlight %}
    -For more details on these additional sources, see the corresponding [API documentation](#where-to-go-from-here). -Furthermore, you can also implement your own custom receiver for your sources. See the -[Custom Receiver Guide](streaming-custom-receivers.html). +Note that these advanced sources are not available in the `spark-shell`, hence applications based on these +advanced sources cannot be tested in the shell. + +Some of these advanced sources are as follows. + +- **Twitter:** Spark Streaming's TwitterUtils uses Twitter4j 3.0.3 to get the public stream of tweets using + [Twitter's Streaming API](https://dev.twitter.com/docs/streaming-apis). Authentication information + can be provided by any of the [methods](http://twitter4j.org/en/configuration.html) supported by + Twitter4J library. You can either get the public stream, or get the filtered stream based on a + keywords. See the API documentation ([Scala](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), [Java](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html)) and examples ([TwitterPopularTags]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala) and + [TwitterAlgebirdCMS]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala)). + +- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can received data from Flume 1.4.0. See the [Flume Integration Guide](streaming-flume-integration.html) for more details. -### Kinesis -[Kinesis](streaming-kinesis.html) +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} can receive data from Kafka 0.8.0. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. -## Operations -There are two kinds of DStream operations - _transformations_ and _output operations_. Similar to -RDD transformations, DStream transformations operate on one or more DStreams to create new DStreams -with transformed data. After applying a sequence of transformations to the input streams, output -operations need to called, which write data out to an external data sink, such as a filesystem or a -database. +- **Kinesis:** See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. -### Transformations -DStreams support many of the transformations available on normal Spark RDD's. Some of the -common ones are as follows. +### Custom Sources +{:.no_toc} +Input DStreams can also be created out of custom data sources. All you have to do is implement an user-defined **receiver** (see next section to understand what that is) that can receive data from the custom sources and push it into Spark. See the +[Custom Receiver Guide](streaming-custom-receivers.html) for details. + +*** + +## Transformations on DStreams +Similar to that of RDDs, transformations allow the data from the input DStream to be modified. +DStreams support many of the transformations available on normal Spark RDD's. +Some of the common ones are as follows. @@ -557,8 +622,8 @@ common ones are as follows. The last two transformations are worth highlighting again. -

    UpdateStateByKey Operation

    - +#### UpdateStateByKey Operation +{:.no_toc} The `updateStateByKey` operation allows you to maintain arbitrary state while continuously updating it with new information. To use this, you will have to do two steps. @@ -616,8 +681,8 @@ the `(word, 1)` pairs) and the `runningCount` having the previous count. For the Scala code, take a look at the example [StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala). -

    Transform Operation

    - +#### Transform Operation +{:.no_toc} The `transform` operation (along with its variations like `transformWith`) allows arbitrary RDD-to-RDD functions to be applied on a DStream. It can be used to apply any RDD operation that is not exposed in the DStream API. @@ -662,8 +727,8 @@ JavaPairDStream cleanedDStream = wordCounts.transform( In fact, you can also use [machine learning](mllib-guide.html) and [graph computation](graphx-programming-guide.html) algorithms in the `transform` method. -

    Window Operations

    - +#### Window Operations +{:.no_toc} Finally, Spark Streaming also provides *windowed computations*, which allow you to apply transformations over a sliding window of data. This following figure illustrates this sliding window. @@ -678,11 +743,11 @@ window. As shown in the figure, every time the window *slides* over a source DStream, the source RDDs that fall within the window are combined and operated upon to produce the RDDs of the windowed DStream. In this specific case, the operation is applied over last 3 time -units of data, and slides by 2 time units. This shows that any window-based operation needs to +units of data, and slides by 2 time units. This shows that any window operation needs to specify two parameters. * window length - The duration of the window (3 in the figure) - * slide interval - The interval at which the window-based operation is performed (2 in + * sliding interval - The interval at which the window operation is performed (2 in the figure). These two parameters must be multiples of the batch interval of the source DStream (1 in the @@ -720,7 +785,7 @@ JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow -Some of the common window-based operations are as follows. All of these operations take the +Some of the common window operations are as follows. All of these operations take the said two parameters - windowLength and slideInterval.
    TransformationMeaning
    @@ -778,21 +843,27 @@ said two parameters - windowLength and slideInterval.
    -### Output Operations -When an output operator is called, it triggers the computation of a stream. Currently the following -output operators are defined: + +The complete list of DStream transformations is available in the API documentation. For the Scala API, +see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) +and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). +For the Java API, see [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) +and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html). + +*** + +## Output Operations on DStreams +Output operations allow DStream's data to be pushed out external systems like a database or a file systems. +Since the output operations actually allow the transformed data to be consumed by external systems, +they trigger the actual execution of all the DStream transformations (similar to actions for RDDs). +Currently, the following output operations are defined: - - - - - + @@ -811,17 +882,84 @@ output operators are defined: + + + +
    Output OperationMeaning
    print() Prints first ten elements of every batch of data in a DStream on the driver.
    foreachRDD(func) The fundamental output operator. Applies a function, func, to each RDD generated from - the stream. This function should have side effects, such as printing output, saving the RDD to - external files, or writing it over the network to an external system. Prints first ten elements of every batch of data in a DStream on the driver. + This is useful for development and debugging.
    saveAsObjectFiles(prefix, [suffix]) Save this DStream's contents as a Hadoop file. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]".
    foreachRDD(func) The most generic output operator that applies a function, func, to each RDD generated from + the stream. This function should push the data in each RDD to a external system, like saving the RDD to + files, or writing it over the network to a database. Note that the function func is executed + at the driver, and will usually have RDD actions in it that will force the computation of the streaming RDDs.
    +### Design Patterns for using foreachRDD +{:.no_toc} +`dstream.foreachRDD` is a powerful primitive that allows data to sent out to external systems. +However, it is important to understand how to use this primitive correctly and efficiently. +Some of the common mistakes to avoid are as follows. -The complete list of DStream operations is available in the API documentation. For the Scala API, -see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) -and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). -For the Java API, see [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) -and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html). +- Often writing data to external system requires creating a connection object +(e.g. TCP connection to a remote server) and using it to send data to a remote system. +For this purpose, a developer may inadvertantly try creating a connection object at +the Spark driver, but try to use it in a Spark worker to save records in the RDDs. +For example (in Scala), + + dstream.foreachRDD(rdd => { + val connection = createNewConnection() // executed at the driver + rdd.foreach(record => { + connection.send(record) // executed at the worker + }) + }) + + This is incorrect as this requires the connection object to be serialized and sent from the driver to the worker. Such connection objects are rarely transferrable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. + +- However, this can lead to another common mistake - creating a new connection for every record. For example, + + dstream.foreachRDD(rdd => { + rdd.foreach(record => { + val connection = createNewConnection() + connection.send(record) + connection.close() + }) + }) + + Typically, creating a connection object has time and resource overheads. Therefore, creating and destroying a connection object for each record can incur unnecessarily high overheads and can significantly reduce the overall throughput of the system. A better solution is to use `rdd.foreachPartition` - create a single connection object and send all the records in a RDD partition using that connection. + + dstream.foreachRDD(rdd => { + rdd.foreachPartition(partitionOfRecords => { + val connection = createNewConnection() + partitionOfRecords.foreach(record => connection.send(record)) + connection.close() + }) + }) + + This amortizes the connection creation overheads over many records. + +- Finally, this can be further optimized by reusing connection objects across multiple RDDs/batches. + One can maintain a static pool of connection objects than can be reused as + RDDs of multiple batches are pushed to the external system, thus further reducing the overheads. -## Persistence + dstream.foreachRDD(rdd => { + rdd.foreachPartition(partitionOfRecords => { + // ConnectionPool is a static, lazily initialized pool of connections + val connection = ConnectionPool.getConnection() + partitionOfRecords.foreach(record => connection.send(record)) + ConnectionPool.returnConnection(connection) // return to the pool for future reuse + }) + }) + + Note that the connections in the pool should be lazily created on demand and timed out if not used for a while. This achieves the most efficient sending of data to external systems. + + +##### Other points to remember: +{:.no_toc} +- DStreams are executed lazily by the output operations, just like RDDs are lazily executed by RDD actions. Specifically, RDD actions inside the DStream output operations force the processing of the received data. Hence, if your application does not have any output operation, or has output operations like `dstream.foreachRDD()` without any RDD action inside them, then nothing will get executed. The system will simply receive the data and discard it. + +- By default, output operations are executed one-at-a-time. And they are executed in the order they are defined in the application. + +*** + +## Caching / Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, using `persist()` method on a DStream would automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple @@ -838,7 +976,9 @@ memory. This is further discussed in the [Performance Tuning](#memory-tuning) se information on different persistence levels can be found in [Spark Programming Guide](programming-guide.html#rdd-persistence). -## RDD Checkpointing +*** + +## Checkpointing A _stateful operation_ is one which operates over multiple batches of data. This includes all window-based operations and the `updateStateByKey` operation. Since stateful operations have a dependency on previous batches of data, they continuously accumulate metadata over time. @@ -867,10 +1007,19 @@ For DStreams that must be checkpointed (that is, DStreams created by `updateStat `reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by default set to a multiple of the DStream's sliding interval such that its at least 10 seconds. -## Deployment +*** + +## Deploying Applications A Spark Streaming application is deployed on a cluster in the same way as any other Spark application. Please refer to the [deployment guide](cluster-overview.html) for more details. +Note that the applications +that use [advanced sources](#advanced-sources) (e.g. Kafka, Flume, Twitter) are also required to package the +extra artifact they link to, along with their dependencies, in the JAR that is used to deploy the application. +For example, an application using `TwitterUtils` will have to include +`spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` and all its transitive +dependencies in the application JAR. + If a running Spark Streaming application needs to be upgraded (with new application code), then there are two possible mechanism. @@ -889,7 +1038,9 @@ application left off. Note that this can be done only with input sources that su (like Kafka, and Flume) as data needs to be buffered while the previous application down and the upgraded application is not yet up. -## Monitoring +*** + +## Monitoring Applications Beyond Spark's [monitoring capabilities](monitoring.html), there are additional capabilities specific to Spark Streaming. When a StreamingContext is used, the [Spark web UI](monitoring.html#web-interfaces) shows @@ -912,22 +1063,18 @@ The progress of a Spark Streaming program can also be monitored using the which allows you to get receiver status and processing times. Note that this is a developer API and it is likely to be improved upon (i.e., more information reported) in the future. -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** # Performance Tuning Getting the best performance of a Spark Streaming application on a cluster requires a bit of tuning. This section explains a number of the parameters and configurations that can tuned to improve the performance of you application. At a high level, you need to consider two things: -
      -
    1. - Reducing the processing time of each batch of data by efficiently using cluster resources. -
    2. -
    3. - Setting the right batch size such that the batches of data can be processed as fast as they - are received (that is, data processing keeps up with the data ingestion). -
    4. -
    +1. Reducing the processing time of each batch of data by efficiently using cluster resources. + +2. Setting the right batch size such that the batches of data can be processed as fast as they + are received (that is, data processing keeps up with the data ingestion). ## Reducing the Processing Time of each Batch There are a number of optimizations that can be done in Spark to minimize the processing time of @@ -935,6 +1082,7 @@ each batch. These have been discussed in detail in [Tuning Guide](tuning.html). highlights some of the most important ones. ### Level of Parallelism in Data Receiving +{:.no_toc} Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to deserialized and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider parallelizing the data receiving. Note that each input DStream @@ -958,6 +1106,7 @@ This distributes the received batches of data across specified number of machine before further processing. ### Level of Parallelism in Data Processing +{:.no_toc} Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is decided by the [config property] @@ -968,6 +1117,7 @@ documentation), or set the [config property](configuration.html#spark-properties `spark.default.parallelism` to change the default. ### Data Serialization +{:.no_toc} The overhead of data serialization can be significant, especially when sub-second batch sizes are to be achieved. There are two aspects to it. @@ -980,6 +1130,7 @@ The overhead of data serialization can be significant, especially when sub-secon serialization format. Hence, the deserialization overhead of input data may be a bottleneck. ### Task Launching Overheads +{:.no_toc} If the number of tasks launched per second is high (say, 50 or more per second), then the overhead of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: @@ -994,6 +1145,8 @@ latencies. The overhead can be reduced by the following changes: These changes may reduce batch processing time by 100s of milliseconds, thus allowing sub-second batch size to be viable. +*** + ## Setting the Right Batch Size For a Spark Streaming application running on a cluster to be stable, the system should be able to process data as fast as it is being received. In other words, batches of data should be processed @@ -1022,6 +1175,8 @@ data rate and/or reducing the batch size. Note that momentary increase in the de temporary data rate increases maybe fine as long as the delay reduces back to a low value (i.e., less than batch size). +*** + ## Memory Tuning Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section, @@ -1037,7 +1192,7 @@ Even though keeping the data serialized incurs higher serialization/deserializat it significantly reduces GC pauses. * **Clearing persistent RDDs**: By default, all persistent RDDs generated by Spark Streaming will - be cleared from memory based on Spark's in-built policy (LRU). If `spark.cleaner.ttl` is set, + be cleared from memory based on Spark's built-in policy (LRU). If `spark.cleaner.ttl` is set, then persistent RDDs that are older than that value are periodically cleared. As mentioned [earlier](#operation), this needs to be careful set based on operations used in the Spark Streaming program. However, a smarter unpersisting of RDDs can be enabled by setting the @@ -1051,7 +1206,8 @@ minimizes the variability of GC pauses. Even though concurrent GC is known to re overall processing throughput of the system, its use is still recommended to achieve more consistent batch processing times. -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** # Fault-tolerance Properties In this section, we are going to discuss the behavior of Spark Streaming application in the event @@ -1124,7 +1280,7 @@ def functionToCreateContext(): StreamingContext = { ssc } -// Get StreaminContext from checkpoint data or create a new one +// Get StreamingContext from checkpoint data or create a new one val context = StreamingContext.getOrCreate(checkpointDirectory, functionToCreateContext _) // Do additional setup on context that needs to be done, @@ -1178,10 +1334,7 @@ context.awaitTermination(); If the `checkpointDirectory` exists, then the context will be recreated from the checkpoint data. If the directory does not exist (i.e., running for the first time), then the function `contextFactory` will be called to create a new -context and set up the DStreams. See the Scala example -[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/JavaRecoverableWordCount.scala) -(note that this example is missing in the 0.9 release, so you can test it using the master branch). -This example appends the word counts of network data into a file. +context and set up the DStreams. You can also explicitly create a `JavaStreamingContext` from the checkpoint data and start the computation by using `new JavaStreamingContext(checkpointDirectory)`. @@ -1208,7 +1361,8 @@ automatically restarted, and the word counts will cont For other deployment environments like Mesos and Yarn, you have to restart the driver through other mechanisms. -

    Recovery Semantics

    +#### Recovery Semantics +{:.no_toc} There are two different failure behaviors based on which input sources are used. @@ -1306,7 +1460,8 @@ in the file. This is what the sequence of outputs would be with and without a dr If the driver had crashed in the middle of the processing of time 3, then it will process time 3 and output 30 after recovery. -*************************************************************************************************** +*************************************************************************************************** +*************************************************************************************************** # Migration Guide from 0.9.1 or below to 1.x Between Spark 0.9.1 and Spark 1.0, there were a few API changes made to ensure future API stability. @@ -1332,7 +1487,7 @@ replaced by [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver. the following advantages. * Methods like `stop` and `restart` have been added to for better control of the lifecycle of a receiver. See -the [custom receiver guide](streaming-custom-receiver.html) for more details. +the [custom receiver guide](streaming-custom-receivers.html) for more details. * Custom receivers can be implemented using both Scala and Java. To migrate your existing custom receivers from the earlier NetworkReceiver to the new Receiver, you have @@ -1356,6 +1511,7 @@ the `org.apache.spark.streaming.receivers` package were also moved to [`org.apache.spark.streaming.receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.package) package and renamed for better clarity. +*************************************************************************************************** *************************************************************************************************** # Where to Go from Here @@ -1366,6 +1522,7 @@ package and renamed for better clarity. [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), + [KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$), [TwitterUtils](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), [ZeroMQUtils](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and [MQTTUtils](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) @@ -1375,6 +1532,7 @@ package and renamed for better clarity. [PairJavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/PairJavaDStream.html) * [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html), [FlumeUtils](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html), + [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) [TwitterUtils](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html), [ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and [MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html) From e08ea7393df46567f552aa67c60a690c231775e4 Mon Sep 17 00:00:00 2001 From: scwf Date: Wed, 3 Sep 2014 18:39:13 -0700 Subject: [PATCH 0583/1492] [SPARK-3303][core] fix SparkContextSchedulerCreationSuite test error run test with the master branch with this command when mesos native lib is set sbt/sbt -Phive "test-only org.apache.spark.SparkContextSchedulerCreationSuite" get this error: [info] SparkContextSchedulerCreationSuite: [info] - bad-master [info] - local [info] - local-* [info] - local-n [info] - local--n-failures [info] - local-n-failures [info] - bad-local-n [info] - bad-local-n-failures [info] - local-default-parallelism [info] - simr [info] - local-cluster [info] - yarn-cluster [info] - yarn-standalone [info] - yarn-client [info] - mesos fine-grained [info] - mesos coarse-grained ** FAILED *** [info] Executor Spark home `spark.mesos.executor.home` is not set! Since `executorSparkHome` only used in `createCommand`, move `val executorSparkHome...` to `createCommand` to fix this issue. Author: scwf Author: wangfei Closes #2199 from scwf/SparkContextSchedulerCreationSuite and squashes the following commits: ef1de22 [scwf] fix code fomate 19d26f3 [scwf] fix conflict d9a8a60 [wangfei] fix SparkContextSchedulerCreationSuite test error --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index da43ef567608c..64568409dbafd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -71,11 +71,6 @@ private[spark] class CoarseMesosSchedulerBackend( val taskIdToSlaveId = new HashMap[Int, String] val failuresBySlaveId = new HashMap[String, Int] // How many times tasks on each slave failed - val executorSparkHome = conf.getOption("spark.mesos.executor.home") - .orElse(sc.getSparkHome()) - .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) @@ -112,6 +107,11 @@ private[spark] class CoarseMesosSchedulerBackend( } def createCommand(offer: Offer, numCores: Int): CommandInfo = { + val executorSparkHome = conf.getOption("spark.mesos.executor.home") + .orElse(sc.getSparkHome()) + .getOrElse { + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } val environment = Environment.newBuilder() val extraClassPath = conf.getOption("spark.executor.extraClassPath") extraClassPath.foreach { cp => From 4bba10c41acaf84a1c4a8e2db467c22f5ab7cbb9 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 18:42:01 -0700 Subject: [PATCH 0584/1492] [SPARK-3233] Executor never stop its SparnEnv, BlockManager, ConnectionManager etc. Author: Kousuke Saruta Closes #2138 from sarutak/SPARK-3233 and squashes the following commits: c0205b7 [Kousuke Saruta] Merge branch 'SPARK-3233' of github.com:sarutak/spark into SPARK-3233 064679d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 d3005fd [Kousuke Saruta] Modified Class definition format of BlockManagerMaster 039b747 [Kousuke Saruta] Modified style 889e2d1 [Kousuke Saruta] Modified BlockManagerMaster to be able to be past isDriver flag 4da8535 [Kousuke Saruta] Modified BlockManagerMaster#stop to send StopBlockManagerMaster message when sender is Driver 6518c3a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 d5ab19a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 6bce25c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3233 6058a58 [Kousuke Saruta] Modified Executor not to invoke SparkEnv#stop in local mode e5ad9d3 [Kousuke Saruta] Modified Executor to stop SparnEnv at the end of itself --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../main/scala/org/apache/spark/executor/Executor.scala | 3 +++ .../org/apache/spark/storage/BlockManagerMaster.scala | 8 ++++++-- .../scala/org/apache/spark/storage/ThreadingTest.scala | 2 +- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../org/apache/spark/storage/BlockManagerSuite.scala | 2 +- 6 files changed, 13 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 72716567ca99b..2973d002cc428 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -225,7 +225,7 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) + new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf, securityManager, mapOutputTracker, shuffleManager) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index d7d19f6fa3b96..dd903dc65d204 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -123,6 +123,9 @@ private[spark] class Executor( env.metricsSystem.report() isStopped = true threadPool.shutdown() + if (!isLocal) { + env.stop() + } } class TaskRunner( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e67b3dc5ce02e..2e262594b3538 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -27,7 +27,11 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { +class BlockManagerMaster( + var driverActor: ActorRef, + conf: SparkConf, + isDriver: Boolean) + extends Logging { private val AKKA_RETRY_ATTEMPTS: Int = AkkaUtils.numRetries(conf) private val AKKA_RETRY_INTERVAL_MS: Int = AkkaUtils.retryWaitMs(conf) @@ -196,7 +200,7 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log /** Stop the driver actor, called only on the Spark driver node */ def stop() { - if (driverActor != null) { + if (driverActor != null && isDriver) { tell(StopBlockManagerMaster) driverActor = null logInfo("BlockManagerMaster stopped") diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index aa83ea90ee9ee..7540f0d5e2a5a 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -99,7 +99,7 @@ private[spark] object ThreadingTest { val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf) + conf, true) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 1a42fc1b233ba..0bb91febde9d7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -120,7 +120,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F */ val cacheLocations = new HashMap[(Int, Int), Seq[BlockManagerId]] // stub out BlockManagerMaster.getLocations to use our cacheLocations - val blockManagerMaster = new BlockManagerMaster(null, conf) { + val blockManagerMaster = new BlockManagerMaster(null, conf, true) { override def getLocations(blockIds: Array[BlockId]): Seq[Seq[BlockManagerId]] = { blockIds.map { _.asRDDId.map(id => (id.rddId -> id.splitIndex)).flatMap(key => cacheLocations.get(key)). diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 14ffadab99cae..c200654162268 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -93,7 +93,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter master = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf) + conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() From f48420fde58d554480cc8830d2f8c4d17618f283 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 3 Sep 2014 18:57:20 -0700 Subject: [PATCH 0585/1492] [SPARK-2973][SQL] Lightweight SQL commands without distributed jobs when calling .collect() By overriding `executeCollect()` in physical plan classes of all commands, we can avoid to kick off a distributed job when collecting result of a SQL command, e.g. `sql("SET").collect()`. Previously, `Command.sideEffectResult` returns a `Seq[Any]`, and the `execute()` method in sub-classes of `Command` typically convert that to a `Seq[Row]` then parallelize it to an RDD. Now with this PR, `sideEffectResult` is required to return a `Seq[Row]` directly, so that `executeCollect()` can directly leverage that and be factored to the `Command` parent class. Author: Cheng Lian Closes #2215 from liancheng/lightweight-commands and squashes the following commits: 3fbef60 [Cheng Lian] Factored execute() method of physical commands to parent class Command 5a0e16c [Cheng Lian] Passes test suites e0e12e9 [Cheng Lian] Refactored Command.sideEffectResult and Command.executeCollect 995bdd8 [Cheng Lian] Cleaned up DescribeHiveTableCommand 542977c [Cheng Lian] Avoids confusion between logical and physical plan by adding package prefixes 55b2aa5 [Cheng Lian] Avoids distributed jobs when execution SQL commands --- .../apache/spark/sql/execution/commands.scala | 63 +++++++------------ .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../spark/sql/hive/HiveStrategies.scala | 14 +++-- .../execution/DescribeHiveTableCommand.scala | 30 +++------ .../sql/hive/execution/NativeCommand.scala | 11 +--- .../spark/sql/hive/execution/commands.scala | 20 ++---- 6 files changed, 48 insertions(+), 94 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 031b695169cea..286c6d264f86a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -21,11 +21,13 @@ import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.{Row, SQLConf, SQLContext} trait Command { + this: SparkPlan => + /** * A concrete command should override this lazy field to wrap up any side effects caused by the * command or any other computation that should be evaluated exactly once. The value of this field @@ -35,7 +37,11 @@ trait Command { * The `execute()` method of all the physical command classes should reference `sideEffectResult` * so that the command can be executed eagerly right after the command query is created. */ - protected[sql] lazy val sideEffectResult: Seq[Any] = Seq.empty[Any] + protected[sql] lazy val sideEffectResult: Seq[Row] = Seq.empty[Row] + + override def executeCollect(): Array[Row] = sideEffectResult.toArray + + override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) } /** @@ -47,17 +53,17 @@ case class SetCommand( @transient context: SQLContext) extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[String] = (key, value) match { + override protected[sql] lazy val sideEffectResult: Seq[Row] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) - Array(s"${SQLConf.SHUFFLE_PARTITIONS}=$v") + Array(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")) } else { context.setConf(k, v) - Array(s"$k=$v") + Array(Row(s"$k=$v")) } // Query the value bound to key k. @@ -73,28 +79,22 @@ case class SetCommand( "hive-0.12.0.jar").mkString(":") Array( - "system:java.class.path=" + hiveJars, - "system:sun.java.command=shark.SharkServer2") - } - else { - Array(s"$k=${context.getConf(k, "")}") + Row("system:java.class.path=" + hiveJars), + Row("system:sun.java.command=shark.SharkServer2")) + } else { + Array(Row(s"$k=${context.getConf(k, "")}")) } // Query all key-value pairs that are set in the SQLConf of the context. case (None, None) => context.getAllConfs.map { case (k, v) => - s"$k=$v" + Row(s"$k=$v") }.toSeq case _ => throw new IllegalArgumentException() } - def execute(): RDD[Row] = { - val rows = sideEffectResult.map { line => new GenericRow(Array[Any](line)) } - context.sparkContext.parallelize(rows, 1) - } - override def otherCopyArgs = context :: Nil } @@ -113,19 +113,14 @@ case class ExplainCommand( extends LeafNode with Command { // Run through the optimizer to generate the physical plan. - override protected[sql] lazy val sideEffectResult: Seq[String] = try { + override protected[sql] lazy val sideEffectResult: Seq[Row] = try { // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. val queryExecution = context.executePlan(logicalPlan) val outputString = if (extended) queryExecution.toString else queryExecution.simpleString - outputString.split("\n") + outputString.split("\n").map(Row(_)) } catch { case cause: TreeNodeException[_] => - ("Error occurred during query planning: \n" + cause.getMessage).split("\n") - } - - def execute(): RDD[Row] = { - val explanation = sideEffectResult.map(row => new GenericRow(Array[Any](row))) - context.sparkContext.parallelize(explanation, 1) + ("Error occurred during query planning: \n" + cause.getMessage).split("\n").map(Row(_)) } override def otherCopyArgs = context :: Nil @@ -144,12 +139,7 @@ case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: } else { context.uncacheTable(tableName) } - Seq.empty[Any] - } - - override def execute(): RDD[Row] = { - sideEffectResult - context.emptyResult + Seq.empty[Row] } override def output: Seq[Attribute] = Seq.empty @@ -163,15 +153,8 @@ case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( @transient context: SQLContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = { - Seq(("# Registered as a temporary table", null, null)) ++ - child.output.map(field => (field.name, field.dataType.toString, null)) - } - - override def execute(): RDD[Row] = { - val rows = sideEffectResult.map { - case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment)) - } - context.sparkContext.parallelize(rows, 1) + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + Row("# Registered as a temporary table", null, null) +: + child.output.map(field => Row(field.name, field.dataType.toString, null)) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index d9b2bc7348ad2..ced8397972fbd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -389,7 +389,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { }.mkString("{", ",", "}") case (seq: Seq[_], ArrayType(typ, _)) => seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_,_], MapType(kType, vType, _)) => + case (map: Map[_, _], MapType(kType, vType, _)) => map.map { case (key, value) => toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) @@ -409,7 +409,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // be similar with Hive. describeHiveTableCommand.hiveString case command: PhysicalCommand => - command.sideEffectResult.map(_.toString) + command.sideEffectResult.map(_.head.toString) case other => val result: Seq[Seq[Any]] = toRdd.collect().toSeq diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 47e24f0dec146..24abb1b5bd1a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -18,17 +18,19 @@ package org.apache.spark.sql.hive import org.apache.spark.annotation.Experimental -import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.columnar.InMemoryRelation -import org.apache.spark.sql.parquet.{ParquetRelation, ParquetTableScan} +import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} +import org.apache.spark.sql.hive +import org.apache.spark.sql.hive.execution._ +import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.{SQLContext, SchemaRDD} import scala.collection.JavaConversions._ @@ -196,9 +198,9 @@ private[hive] trait HiveStrategies { case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil - case DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil + case hive.DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil - case AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil + case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil case describe: logical.DescribeCommand => val resolvedTable = context.executePlan(describe.table).analyzed diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index a40e89e0d382b..317801001c7a4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericRow, Row} +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.execution.{Command, LeafNode} import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} @@ -41,26 +41,21 @@ case class DescribeHiveTableCommand( extends LeafNode with Command { // Strings with the format like Hive. It is used for result comparison in our unit tests. - lazy val hiveString: Seq[String] = { - val alignment = 20 - val delim = "\t" - - sideEffectResult.map { - case (name, dataType, comment) => - String.format("%-" + alignment + "s", name) + delim + - String.format("%-" + alignment + "s", dataType) + delim + - String.format("%-" + alignment + "s", Option(comment).getOrElse("None")) - } + lazy val hiveString: Seq[String] = sideEffectResult.map { + case Row(name: String, dataType: String, comment) => + Seq(name, dataType, Option(comment.asInstanceOf[String]).getOrElse("None")) + .map(s => String.format(s"%-20s", s)) + .mkString("\t") } - override protected[sql] lazy val sideEffectResult: Seq[(String, String, String)] = { + override protected[sql] lazy val sideEffectResult: Seq[Row] = { // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil val columns: Seq[FieldSchema] = table.hiveQlTable.getCols val partitionColumns: Seq[FieldSchema] = table.hiveQlTable.getPartCols results ++= columns.map(field => (field.getName, field.getType, field.getComment)) - if (!partitionColumns.isEmpty) { + if (partitionColumns.nonEmpty) { val partColumnInfo = partitionColumns.map(field => (field.getName, field.getType, field.getComment)) results ++= @@ -74,14 +69,9 @@ case class DescribeHiveTableCommand( results ++= Seq(("Detailed Table Information", table.hiveQlTable.getTTable.toString, "")) } - results - } - - override def execute(): RDD[Row] = { - val rows = sideEffectResult.map { - case (name, dataType, comment) => new GenericRow(Array[Any](name, dataType, comment)) + results.map { case (name, dataType, comment) => + Row(name, dataType, comment) } - context.sparkContext.parallelize(rows, 1) } override def otherCopyArgs = context :: Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala index fe6031678f70f..8f10e1ba7f426 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala @@ -32,16 +32,7 @@ case class NativeCommand( @transient context: HiveContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[String] = context.runSqlHive(sql) - - override def execute(): RDD[Row] = { - if (sideEffectResult.size == 0) { - context.emptyResult - } else { - val rows = sideEffectResult.map(r => new GenericRow(Array[Any](r))) - context.sparkContext.parallelize(rows, 1) - } - } + override protected[sql] lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_)) override def otherCopyArgs = context :: Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 2985169da033c..a1a4aa7de7bf7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -33,19 +33,13 @@ import org.apache.spark.sql.hive.HiveContext */ @DeveloperApi case class AnalyzeTable(tableName: String) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] def output = Seq.empty - override protected[sql] lazy val sideEffectResult = { + override protected[sql] lazy val sideEffectResult: Seq[Row] = { hiveContext.analyze(tableName) - Seq.empty[Any] - } - - override def execute(): RDD[Row] = { - sideEffectResult - sparkContext.emptyRDD[Row] + Seq.empty[Row] } } @@ -55,20 +49,14 @@ case class AnalyzeTable(tableName: String) extends LeafNode with Command { */ @DeveloperApi case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Command { - def hiveContext = sqlContext.asInstanceOf[HiveContext] def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Any] = { + override protected[sql] lazy val sideEffectResult: Seq[Row] = { val ifExistsClause = if (ifExists) "IF EXISTS " else "" hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") hiveContext.catalog.unregisterTable(None, tableName) - Seq.empty - } - - override def execute(): RDD[Row] = { - sideEffectResult - sparkContext.emptyRDD[Row] + Seq.empty[Row] } } From 248067adbe90f93c7d5e23aa61b3072dfdf48a8a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 3 Sep 2014 18:59:26 -0700 Subject: [PATCH 0586/1492] [SPARK-2961][SQL] Use statistics to prune batches within cached partitions This PR is based on #1883 authored by marmbrus. Key differences: 1. Batch pruning instead of partition pruning When #1883 was authored, batched column buffer building (#1880) hadn't been introduced. This PR combines these two and provide partition batch level pruning, which leads to smaller memory footprints and can generally skip more elements. The cost is that the pruning predicates are evaluated more frequently (partition number multiplies batch number per partition). 1. More filters are supported Filter predicates consist of `=`, `<`, `<=`, `>`, `>=` and their conjunctions and disjunctions are supported. Author: Cheng Lian Closes #2188 from liancheng/in-mem-batch-pruning and squashes the following commits: 68cf019 [Cheng Lian] Marked sqlContext as @transient 4254f6c [Cheng Lian] Enables in-memory partition pruning in PartitionBatchPruningSuite 3784105 [Cheng Lian] Overrides InMemoryColumnarTableScan.sqlContext d2a1d66 [Cheng Lian] Disables in-memory partition pruning by default 062c315 [Cheng Lian] HiveCompatibilitySuite code cleanup 16b77bf [Cheng Lian] Fixed pruning predication conjunctions and disjunctions 16195c5 [Cheng Lian] Enabled both disjunction and conjunction 89950d0 [Cheng Lian] Worked around Scala style check 9c167f6 [Cheng Lian] Minor code cleanup 3c4d5c7 [Cheng Lian] Minor code cleanup ea59ee5 [Cheng Lian] Renamed PartitionSkippingSuite to PartitionBatchPruningSuite fc517d0 [Cheng Lian] More test cases 1868c18 [Cheng Lian] Code cleanup, bugfix, and adding tests cb76da4 [Cheng Lian] Added more predicate filters, fixed table scan stats for testing purposes 385474a [Cheng Lian] Merge branch 'inMemStats' into in-mem-batch-pruning --- .../catalyst/expressions/AttributeMap.scala | 41 ++ .../catalyst/expressions/BoundAttribute.scala | 12 +- .../scala/org/apache/spark/sql/SQLConf.scala | 7 + .../spark/sql/columnar/ColumnBuilder.scala | 10 +- .../spark/sql/columnar/ColumnStats.scala | 434 +++++------------- .../columnar/InMemoryColumnarTableScan.scala | 131 +++++- .../sql/columnar/NullableColumnBuilder.scala | 1 + .../spark/sql/execution/SparkStrategies.scala | 4 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 39 +- .../columnar/NullableColumnBuilderSuite.scala | 2 +- .../columnar/PartitionBatchPruningSuite.scala | 95 ++++ .../compression/BooleanBitSetSuite.scala | 4 +- .../compression/DictionaryEncodingSuite.scala | 2 +- .../compression/IntegralDeltaSuite.scala | 2 +- .../compression/RunLengthEncodingSuite.scala | 4 +- .../TestCompressibleColumnBuilder.scala | 4 +- .../execution/HiveCompatibilitySuite.scala | 13 +- 17 files changed, 446 insertions(+), 359 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala new file mode 100644 index 0000000000000..8364379644c90 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +/** + * Builds a map that is keyed by an Attribute's expression id. Using the expression id allows values + * to be looked up even when the attributes used differ cosmetically (i.e., the capitalization + * of the name, or the expected nullability). + */ +object AttributeMap { + def apply[A](kvs: Seq[(Attribute, A)]) = + new AttributeMap(kvs.map(kv => (kv._1.exprId, (kv._1, kv._2))).toMap) +} + +class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) + extends Map[Attribute, A] with Serializable { + + override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2) + + override def + [B1 >: A](kv: (Attribute, B1)): Map[Attribute, B1] = + (baseMap.map(_._2) + kv).toMap + + override def iterator: Iterator[(Attribute, A)] = baseMap.map(_._2).iterator + + override def -(key: Attribute): Map[Attribute, A] = (baseMap.map(_._2) - key).toMap +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 54c6baf1af3bf..fa80b07f8e6be 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -38,12 +38,20 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) } object BindReferences extends Logging { - def bindReference[A <: Expression](expression: A, input: Seq[Attribute]): A = { + + def bindReference[A <: Expression]( + expression: A, + input: Seq[Attribute], + allowFailures: Boolean = false): A = { expression.transform { case a: AttributeReference => attachTree(a, "Binding attribute") { val ordinal = input.indexWhere(_.exprId == a.exprId) if (ordinal == -1) { - sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") + if (allowFailures) { + a + } else { + sys.error(s"Couldn't find $a in ${input.mkString("[", ",", "]")}") + } } else { BoundReference(ordinal, a.dataType, a.nullable) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 64d49354dadcd..4137ac7663739 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -26,6 +26,7 @@ import java.util.Properties private[spark] object SQLConf { val COMPRESS_CACHED = "spark.sql.inMemoryColumnarStorage.compressed" val COLUMN_BATCH_SIZE = "spark.sql.inMemoryColumnarStorage.batchSize" + val IN_MEMORY_PARTITION_PRUNING = "spark.sql.inMemoryColumnarStorage.partitionPruning" val AUTO_BROADCASTJOIN_THRESHOLD = "spark.sql.autoBroadcastJoinThreshold" val DEFAULT_SIZE_IN_BYTES = "spark.sql.defaultSizeInBytes" val SHUFFLE_PARTITIONS = "spark.sql.shuffle.partitions" @@ -124,6 +125,12 @@ trait SQLConf { private[spark] def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING, "false").toBoolean + /** + * When set to true, partition pruning for in-memory columnar tables is enabled. + */ + private[spark] def inMemoryPartitionPruning: Boolean = + getConf(IN_MEMORY_PARTITION_PRUNING, "false").toBoolean + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 247337a875c75..b3ec5ded22422 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -38,7 +38,7 @@ private[sql] trait ColumnBuilder { /** * Column statistics information */ - def columnStats: ColumnStats[_, _] + def columnStats: ColumnStats /** * Returns the final columnar byte buffer. @@ -47,7 +47,7 @@ private[sql] trait ColumnBuilder { } private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( - val columnStats: ColumnStats[T, JvmType], + val columnStats: ColumnStats, val columnType: ColumnType[T, JvmType]) extends ColumnBuilder { @@ -81,18 +81,18 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType]) - extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType) with NullableColumnBuilder private[sql] abstract class NativeColumnBuilder[T <: NativeType]( - override val columnStats: NativeColumnStats[T], + override val columnStats: ColumnStats, override val columnType: NativeColumnType[T]) extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType) with NullableColumnBuilder with AllCompressionSchemes with CompressibleColumnBuilder[T] -private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN) +private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new NoopColumnStats, BOOLEAN) private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 6502110e903fe..fc343ccb995c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -17,381 +17,193 @@ package org.apache.spark.sql.columnar +import java.sql.Timestamp + import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} import org.apache.spark.sql.catalyst.types._ +private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { + val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = false)() + val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = false)() + val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() + + val schema = Seq(lowerBound, upperBound, nullCount) +} + +private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { + val (forAttribute, schema) = { + val allStats = tableSchema.map(a => a -> new ColumnStatisticsSchema(a)) + (AttributeMap(allStats), allStats.map(_._2.schema).foldLeft(Seq.empty[Attribute])(_ ++ _)) + } +} + /** * Used to collect statistical information when building in-memory columns. * * NOTE: we intentionally avoid using `Ordering[T]` to compare values here because `Ordering[T]` * brings significant performance penalty. */ -private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable { - /** - * Closed lower bound of this column. - */ - def lowerBound: JvmType - - /** - * Closed upper bound of this column. - */ - def upperBound: JvmType - +private[sql] sealed trait ColumnStats extends Serializable { /** * Gathers statistics information from `row(ordinal)`. */ - def gatherStats(row: Row, ordinal: Int) - - /** - * Returns `true` if `lower <= row(ordinal) <= upper`. - */ - def contains(row: Row, ordinal: Int): Boolean + def gatherStats(row: Row, ordinal: Int): Unit /** - * Returns `true` if `row(ordinal) < upper` holds. + * Column statistics represented as a single row, currently including closed lower bound, closed + * upper bound and null count. */ - def isAbove(row: Row, ordinal: Int): Boolean - - /** - * Returns `true` if `lower < row(ordinal)` holds. - */ - def isBelow(row: Row, ordinal: Int): Boolean - - /** - * Returns `true` if `row(ordinal) <= upper` holds. - */ - def isAtOrAbove(row: Row, ordinal: Int): Boolean - - /** - * Returns `true` if `lower <= row(ordinal)` holds. - */ - def isAtOrBelow(row: Row, ordinal: Int): Boolean -} - -private[sql] sealed abstract class NativeColumnStats[T <: NativeType] - extends ColumnStats[T, T#JvmType] { - - type JvmType = T#JvmType - - protected var (_lower, _upper) = initialBounds - - def initialBounds: (JvmType, JvmType) - - protected def columnType: NativeColumnType[T] - - override def lowerBound: T#JvmType = _lower - - override def upperBound: T#JvmType = _upper - - override def isAtOrAbove(row: Row, ordinal: Int) = { - contains(row, ordinal) || isAbove(row, ordinal) - } - - override def isAtOrBelow(row: Row, ordinal: Int) = { - contains(row, ordinal) || isBelow(row, ordinal) - } + def collectedStatistics: Row } -private[sql] class NoopColumnStats[T <: DataType, JvmType] extends ColumnStats[T, JvmType] { - override def isAtOrBelow(row: Row, ordinal: Int) = true - - override def isAtOrAbove(row: Row, ordinal: Int) = true - - override def isBelow(row: Row, ordinal: Int) = true - - override def isAbove(row: Row, ordinal: Int) = true +private[sql] class NoopColumnStats extends ColumnStats { - override def contains(row: Row, ordinal: Int) = true + override def gatherStats(row: Row, ordinal: Int): Unit = {} - override def gatherStats(row: Row, ordinal: Int) {} - - override def upperBound = null.asInstanceOf[JvmType] - - override def lowerBound = null.asInstanceOf[JvmType] + override def collectedStatistics = Row() } -private[sql] abstract class BasicColumnStats[T <: NativeType]( - protected val columnType: NativeColumnType[T]) - extends NativeColumnStats[T] - -private[sql] class BooleanColumnStats extends BasicColumnStats(BOOLEAN) { - override def initialBounds = (true, false) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class ByteColumnStats extends ColumnStats { + var upper = Byte.MinValue + var lower = Byte.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } -} - -private[sql] class ByteColumnStats extends BasicColumnStats(BYTE) { - override def initialBounds = (Byte.MaxValue, Byte.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound + if (!row.isNullAt(ordinal)) { + val value = row.getByte(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class ShortColumnStats extends BasicColumnStats(SHORT) { - override def initialBounds = (Short.MaxValue, Short.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class ShortColumnStats extends ColumnStats { + var upper = Short.MinValue + var lower = Short.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } -} - -private[sql] class LongColumnStats extends BasicColumnStats(LONG) { - override def initialBounds = (Long.MaxValue, Long.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound + if (!row.isNullAt(ordinal)) { + val value = row.getShort(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class DoubleColumnStats extends BasicColumnStats(DOUBLE) { - override def initialBounds = (Double.MaxValue, Double.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) - } - - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } - - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class LongColumnStats extends ColumnStats { + var upper = Long.MinValue + var lower = Long.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - } -} - -private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) { - override def initialBounds = (Float.MaxValue, Float.MinValue) - - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) + if (!row.isNullAt(ordinal)) { + val value = row.getLong(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } + def collectedStatistics = Row(lower, upper, nullCount) +} - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class DoubleColumnStats extends ColumnStats { + var upper = Double.MinValue + var lower = Double.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field + if (!row.isNullAt(ordinal)) { + val value = row.getDouble(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } -} -private[sql] object IntColumnStats { - val UNINITIALIZED = 0 - val INITIALIZED = 1 - val ASCENDING = 2 - val DESCENDING = 3 - val UNORDERED = 4 + def collectedStatistics = Row(lower, upper, nullCount) } -/** - * Statistical information for `Int` columns. More information is collected since `Int` is - * frequently used. Extra information include: - * - * - Ordering state (ascending/descending/unordered), may be used to decide whether binary search - * is applicable when searching elements. - * - Maximum delta between adjacent elements, may be used to guide the `IntDelta` compression - * scheme. - * - * (This two kinds of information are not used anywhere yet and might be removed later.) - */ -private[sql] class IntColumnStats extends BasicColumnStats(INT) { - import IntColumnStats._ - - private var orderedState = UNINITIALIZED - private var lastValue: Int = _ - private var _maxDelta: Int = _ - - def isAscending = orderedState != DESCENDING && orderedState != UNORDERED - def isDescending = orderedState != ASCENDING && orderedState != UNORDERED - def isOrdered = isAscending || isDescending - def maxDelta = _maxDelta - - override def initialBounds = (Int.MaxValue, Int.MinValue) +private[sql] class FloatColumnStats extends ColumnStats { + var upper = Float.MinValue + var lower = Float.MaxValue + var nullCount = 0 - override def isBelow(row: Row, ordinal: Int) = { - lowerBound < columnType.getField(row, ordinal) + override def gatherStats(row: Row, ordinal: Int) { + if (!row.isNullAt(ordinal)) { + val value = row.getFloat(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 + } } - override def isAbove(row: Row, ordinal: Int) = { - columnType.getField(row, ordinal) < upperBound - } + def collectedStatistics = Row(lower, upper, nullCount) +} - override def contains(row: Row, ordinal: Int) = { - val field = columnType.getField(row, ordinal) - lowerBound <= field && field <= upperBound - } +private[sql] class IntColumnStats extends ColumnStats { + var upper = Int.MinValue + var lower = Int.MaxValue + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - - if (field > upperBound) _upper = field - if (field < lowerBound) _lower = field - - orderedState = orderedState match { - case UNINITIALIZED => - lastValue = field - INITIALIZED - - case INITIALIZED => - // If all the integers in the column are the same, ordered state is set to Ascending. - // TODO (lian) Confirm whether this is the standard behaviour. - val nextState = if (field >= lastValue) ASCENDING else DESCENDING - _maxDelta = math.abs(field - lastValue) - lastValue = field - nextState - - case ASCENDING if field < lastValue => - UNORDERED - - case DESCENDING if field > lastValue => - UNORDERED - - case state @ (ASCENDING | DESCENDING) => - _maxDelta = _maxDelta.max(field - lastValue) - lastValue = field - state - - case _ => - orderedState + if (!row.isNullAt(ordinal)) { + val value = row.getInt(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + } else { + nullCount += 1 } } + + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class StringColumnStats extends BasicColumnStats(STRING) { - override def initialBounds = (null, null) +private[sql] class StringColumnStats extends ColumnStats { + var upper: String = null + var lower: String = null + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field - if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field - } - - override def contains(row: Row, ordinal: Int) = { - (upperBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 - } - } - - override def isAbove(row: Row, ordinal: Int) = { - (upperBound ne null) && { - val field = columnType.getField(row, ordinal) - field.compareTo(upperBound) < 0 + if (!row.isNullAt(ordinal)) { + val value = row.getString(ordinal) + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + } else { + nullCount += 1 } } - override def isBelow(row: Row, ordinal: Int) = { - (lowerBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) < 0 - } - } + def collectedStatistics = Row(lower, upper, nullCount) } -private[sql] class TimestampColumnStats extends BasicColumnStats(TIMESTAMP) { - override def initialBounds = (null, null) +private[sql] class TimestampColumnStats extends ColumnStats { + var upper: Timestamp = null + var lower: Timestamp = null + var nullCount = 0 override def gatherStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field - if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field - } - - override def contains(row: Row, ordinal: Int) = { - (upperBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 + if (!row.isNullAt(ordinal)) { + val value = row(ordinal).asInstanceOf[Timestamp] + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + } else { + nullCount += 1 } } - override def isAbove(row: Row, ordinal: Int) = { - (lowerBound ne null) && { - val field = columnType.getField(row, ordinal) - field.compareTo(upperBound) < 0 - } - } - - override def isBelow(row: Row, ordinal: Int) = { - (lowerBound ne null) && { - val field = columnType.getField(row, ordinal) - lowerBound.compareTo(field) < 0 - } - } + def collectedStatistics = Row(lower, upper, nullCount) } 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 cb055cd74a5e5..dc668e7dc934c 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 @@ -19,10 +19,12 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{LeafNode, SparkPlan} @@ -31,23 +33,27 @@ object InMemoryRelation { new InMemoryRelation(child.output, useCompression, batchSize, child)() } +private[sql] case class CachedBatch(buffers: Array[ByteBuffer], stats: Row) + private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, batchSize: Int, child: SparkPlan) - (private var _cachedColumnBuffers: RDD[Array[ByteBuffer]] = null) + (private var _cachedColumnBuffers: RDD[CachedBatch] = null) extends LogicalPlan with MultiInstanceRelation { override lazy val statistics = Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes) + val partitionStatistics = new PartitionStatistics(output) + // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { val output = child.output val cached = child.execute().mapPartitions { baseIterator => - new Iterator[Array[ByteBuffer]] { + new Iterator[CachedBatch] { def next() = { val columnBuilders = output.map { attribute => val columnType = ColumnType(attribute.dataType) @@ -68,7 +74,10 @@ private[sql] case class InMemoryRelation( rowCount += 1 } - columnBuilders.map(_.build()) + val stats = Row.fromSeq( + columnBuilders.map(_.columnStats.collectedStatistics).foldLeft(Seq.empty[Any])(_ ++ _)) + + CachedBatch(columnBuilders.map(_.build()), stats) } def hasNext = baseIterator.hasNext @@ -79,7 +88,6 @@ private[sql] case class InMemoryRelation( _cachedColumnBuffers = cached } - override def children = Seq.empty override def newInstance() = { @@ -96,13 +104,98 @@ private[sql] case class InMemoryRelation( private[sql] case class InMemoryColumnarTableScan( attributes: Seq[Attribute], + predicates: Seq[Expression], relation: InMemoryRelation) extends LeafNode { + @transient override val sqlContext = relation.child.sqlContext + override def output: Seq[Attribute] = attributes + // Returned filter predicate should return false iff it is impossible for the input expression + // to evaluate to `true' based on statistics collected about this partition batch. + val buildFilter: PartialFunction[Expression, Expression] = { + case And(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) && buildFilter(rhs) + + case Or(lhs: Expression, rhs: Expression) + if buildFilter.isDefinedAt(lhs) && buildFilter.isDefinedAt(rhs) => + buildFilter(lhs) || buildFilter(rhs) + + case EqualTo(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l && l <= aStats.upperBound + + case EqualTo(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l && l <= aStats.upperBound + + case LessThan(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound < l + + case LessThan(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + l < aStats.upperBound + + case LessThanOrEqual(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l + + case LessThanOrEqual(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + l <= aStats.upperBound + + case GreaterThan(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + l < aStats.upperBound + + case GreaterThan(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound < l + + case GreaterThanOrEqual(a: AttributeReference, l: Literal) => + val aStats = relation.partitionStatistics.forAttribute(a) + l <= aStats.upperBound + + case GreaterThanOrEqual(l: Literal, a: AttributeReference) => + val aStats = relation.partitionStatistics.forAttribute(a) + aStats.lowerBound <= l + } + + val partitionFilters = { + predicates.flatMap { p => + val filter = buildFilter.lift(p) + val boundFilter = + filter.map( + BindReferences.bindReference( + _, + relation.partitionStatistics.schema, + allowFailures = true)) + + boundFilter.foreach(_ => + filter.foreach(f => logInfo(s"Predicate $p generates partition filter: $f"))) + + // If the filter can't be resolved then we are missing required statistics. + boundFilter.filter(_.resolved) + } + } + + val readPartitions = sparkContext.accumulator(0) + val readBatches = sparkContext.accumulator(0) + + private val inMemoryPartitionPruningEnabled = sqlContext.inMemoryPartitionPruning + override def execute() = { + readPartitions.setValue(0) + readBatches.setValue(0) + relation.cachedColumnBuffers.mapPartitions { iterator => + val partitionFilter = newPredicate( + partitionFilters.reduceOption(And).getOrElse(Literal(true)), + relation.partitionStatistics.schema) + // Find the ordinals of the requested columns. If none are requested, use the first. val requestedColumns = if (attributes.isEmpty) { Seq(0) @@ -110,8 +203,26 @@ private[sql] case class InMemoryColumnarTableScan( attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) } - iterator - .map(batch => requestedColumns.map(batch(_)).map(ColumnAccessor(_))) + val rows = iterator + // Skip pruned batches + .filter { cachedBatch => + if (inMemoryPartitionPruningEnabled && !partitionFilter(cachedBatch.stats)) { + def statsString = relation.partitionStatistics.schema + .zip(cachedBatch.stats) + .map { case (a, s) => s"${a.name}: $s" } + .mkString(", ") + logInfo(s"Skipping partition based on stats $statsString") + false + } else { + readBatches += 1 + true + } + } + // Build column accessors + .map { cachedBatch => + requestedColumns.map(cachedBatch.buffers(_)).map(ColumnAccessor(_)) + } + // Extract rows via column accessors .flatMap { columnAccessors => val nextRow = new GenericMutableRow(columnAccessors.length) new Iterator[Row] { @@ -127,6 +238,12 @@ private[sql] case class InMemoryColumnarTableScan( override def hasNext = columnAccessors.head.hasNext } } + + if (rows.hasNext) { + readPartitions += 1 + } + + rows } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index f631ee76fcd78..a72970eef7aa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -49,6 +49,7 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { } abstract override def appendFrom(row: Row, ordinal: Int) { + columnStats.gatherStats(row, ordinal) if (row.isNullAt(ordinal)) { nulls = ColumnBuilder.ensureFreeSpace(nulls, 4) nulls.putInt(pos) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 8dacb84c8a17e..7943d6e1b6fb5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -243,8 +243,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { pruneFilterProject( projectList, filters, - identity[Seq[Expression]], // No filters are pushed down. - InMemoryColumnarTableScan(_, mem)) :: Nil + identity[Seq[Expression]], // All filters still need to be evaluated. + InMemoryColumnarTableScan(_, filters, mem)) :: Nil case _ => Nil } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 5f61fb5e16ea3..cde91ceb68c98 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -19,29 +19,30 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.types._ class ColumnStatsSuite extends FunSuite { - testColumnStats(classOf[BooleanColumnStats], BOOLEAN) - testColumnStats(classOf[ByteColumnStats], BYTE) - testColumnStats(classOf[ShortColumnStats], SHORT) - testColumnStats(classOf[IntColumnStats], INT) - testColumnStats(classOf[LongColumnStats], LONG) - testColumnStats(classOf[FloatColumnStats], FLOAT) - testColumnStats(classOf[DoubleColumnStats], DOUBLE) - testColumnStats(classOf[StringColumnStats], STRING) - testColumnStats(classOf[TimestampColumnStats], TIMESTAMP) - - def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]]( + testColumnStats(classOf[ByteColumnStats], BYTE, Row(Byte.MaxValue, Byte.MinValue, 0)) + testColumnStats(classOf[ShortColumnStats], SHORT, Row(Short.MaxValue, Short.MinValue, 0)) + testColumnStats(classOf[IntColumnStats], INT, Row(Int.MaxValue, Int.MinValue, 0)) + testColumnStats(classOf[LongColumnStats], LONG, Row(Long.MaxValue, Long.MinValue, 0)) + testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) + testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) + testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) + testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) + + def testColumnStats[T <: NativeType, U <: ColumnStats]( columnStatsClass: Class[U], - columnType: NativeColumnType[T]) { + columnType: NativeColumnType[T], + initialStatistics: Row) { val columnStatsName = columnStatsClass.getSimpleName test(s"$columnStatsName: empty") { val columnStats = columnStatsClass.newInstance() - assertResult(columnStats.initialBounds, "Wrong initial bounds") { - (columnStats.lowerBound, columnStats.upperBound) + columnStats.collectedStatistics.zip(initialStatistics).foreach { case (actual, expected) => + assert(actual === expected) } } @@ -49,14 +50,16 @@ class ColumnStatsSuite extends FunSuite { import ColumnarTestUtils._ val columnStats = columnStatsClass.newInstance() - val rows = Seq.fill(10)(makeRandomRow(columnType)) + val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) rows.foreach(columnStats.gatherStats(_, 0)) - val values = rows.map(_.head.asInstanceOf[T#JvmType]) + val values = rows.take(10).map(_.head.asInstanceOf[T#JvmType]) val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] + val stats = columnStats.collectedStatistics - assertResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) - assertResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) + assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) + assertResult(values.max(ordering), "Wrong upper bound")(stats(1)) + assertResult(10, "Wrong null count")(stats(2)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index dc813fe146c47..a77262534a352 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) - extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType) with NullableColumnBuilder object TestNullableColumnBuilder { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala new file mode 100644 index 0000000000000..5d2fd4959197c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.columnar + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} + +import org.apache.spark.sql._ +import org.apache.spark.sql.test.TestSQLContext._ + +case class IntegerData(i: Int) + +class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { + val originalColumnBatchSize = columnBatchSize + val originalInMemoryPartitionPruning = inMemoryPartitionPruning + + override protected def beforeAll() { + // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch + setConf(SQLConf.COLUMN_BATCH_SIZE, "10") + val rawData = sparkContext.makeRDD(1 to 100, 5).map(IntegerData) + rawData.registerTempTable("intData") + + // Enable in-memory partition pruning + setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") + } + + override protected def afterAll() { + setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize.toString) + setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning.toString) + } + + before { + cacheTable("intData") + } + + after { + uncacheTable("intData") + } + + // Comparisons + checkBatchPruning("i = 1", Seq(1), 1, 1) + checkBatchPruning("1 = i", Seq(1), 1, 1) + checkBatchPruning("i < 12", 1 to 11, 1, 2) + checkBatchPruning("i <= 11", 1 to 11, 1, 2) + checkBatchPruning("i > 88", 89 to 100, 1, 2) + checkBatchPruning("i >= 89", 89 to 100, 1, 2) + checkBatchPruning("12 > i", 1 to 11, 1, 2) + checkBatchPruning("11 >= i", 1 to 11, 1, 2) + checkBatchPruning("88 < i", 89 to 100, 1, 2) + checkBatchPruning("89 <= i", 89 to 100, 1, 2) + + // Conjunction and disjunction + checkBatchPruning("i > 8 AND i <= 21", 9 to 21, 2, 3) + checkBatchPruning("i < 2 OR i > 99", Seq(1, 100), 2, 2) + checkBatchPruning("i < 2 OR (i > 78 AND i < 92)", Seq(1) ++ (79 to 91), 3, 4) + + // With unsupported predicate + checkBatchPruning("i < 12 AND i IS NOT NULL", 1 to 11, 1, 2) + checkBatchPruning("NOT (i < 88)", 88 to 100, 5, 10) + + def checkBatchPruning( + filter: String, + expectedQueryResult: Seq[Int], + expectedReadPartitions: Int, + expectedReadBatches: Int) { + + test(filter) { + val query = sql(s"SELECT * FROM intData WHERE $filter") + assertResult(expectedQueryResult.toArray, "Wrong query result") { + query.collect().map(_.head).toArray + } + + val (readPartitions, readBatches) = query.queryExecution.executedPlan.collect { + case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) + }.head + + assert(readBatches === expectedReadBatches, "Wrong number of read batches") + assert(readPartitions === expectedReadPartitions, "Wrong number of read partitions") + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index 5fba00480967c..e01cc8b4d20f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.Row -import org.apache.spark.sql.columnar.{BOOLEAN, BooleanColumnStats} +import org.apache.spark.sql.columnar.{NoopColumnStats, BOOLEAN} import org.apache.spark.sql.columnar.ColumnarTestUtils._ class BooleanBitSetSuite extends FunSuite { @@ -31,7 +31,7 @@ class BooleanBitSetSuite extends FunSuite { // Tests encoder // ------------- - val builder = TestCompressibleColumnBuilder(new BooleanColumnStats, BOOLEAN, BooleanBitSet) + val builder = TestCompressibleColumnBuilder(new NoopColumnStats, BOOLEAN, BooleanBitSet) val rows = Seq.fill[Row](count)(makeRandomRow(BOOLEAN)) val values = rows.map(_.head) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index d8ae2a26778c9..d2969d906c943 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -31,7 +31,7 @@ class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new StringColumnStats, STRING) def testDictionaryEncoding[T <: NativeType]( - columnStats: NativeColumnStats[T], + columnStats: ColumnStats, columnType: NativeColumnType[T]) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 17619dcf974e3..322f447c24840 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -29,7 +29,7 @@ class IntegralDeltaSuite extends FunSuite { testIntegralDelta(new LongColumnStats, LONG, LongDelta) def testIntegralDelta[I <: IntegralType]( - columnStats: NativeColumnStats[I], + columnStats: ColumnStats, columnType: NativeColumnType[I], scheme: IntegralDelta[I]) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 40115beb98899..218c09ac26362 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ class RunLengthEncodingSuite extends FunSuite { - testRunLengthEncoding(new BooleanColumnStats, BOOLEAN) + testRunLengthEncoding(new NoopColumnStats, BOOLEAN) testRunLengthEncoding(new ByteColumnStats, BYTE) testRunLengthEncoding(new ShortColumnStats, SHORT) testRunLengthEncoding(new IntColumnStats, INT) @@ -32,7 +32,7 @@ class RunLengthEncodingSuite extends FunSuite { testRunLengthEncoding(new StringColumnStats, STRING) def testRunLengthEncoding[T <: NativeType]( - columnStats: NativeColumnStats[T], + columnStats: ColumnStats, columnType: NativeColumnType[T]) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 72c19fa31d980..7db723d648d80 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ class TestCompressibleColumnBuilder[T <: NativeType]( - override val columnStats: NativeColumnStats[T], + override val columnStats: ColumnStats, override val columnType: NativeColumnType[T], override val schemes: Seq[CompressionScheme]) extends NativeColumnBuilder(columnStats, columnType) @@ -33,7 +33,7 @@ class TestCompressibleColumnBuilder[T <: NativeType]( object TestCompressibleColumnBuilder { def apply[T <: NativeType]( - columnStats: NativeColumnStats[T], + columnStats: ColumnStats, columnType: NativeColumnType[T], scheme: CompressionScheme) = { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index b589994bd25fa..ab487d673e813 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -35,26 +35,29 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault - private val originalUseCompression = TestHive.useCompression + private val originalColumnBatchSize = TestHive.columnBatchSize + private val originalInMemoryPartitionPruning = TestHive.inMemoryPartitionPruning def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) override def beforeAll() { - // Enable in-memory columnar caching TestHive.cacheTables = true // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) // Add Locale setting Locale.setDefault(Locale.US) - // Enable in-memory columnar compression - TestHive.setConf(SQLConf.COMPRESS_CACHED, "true") + // Set a relatively small column batch size for testing purposes + TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, "5") + // Enable in-memory partition pruning for testing purposes + TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") } override def afterAll() { TestHive.cacheTables = false TimeZone.setDefault(originalTimeZone) Locale.setDefault(originalLocale) - TestHive.setConf(SQLConf.COMPRESS_CACHED, originalUseCompression.toString) + TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize.toString) + TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning.toString) } /** A list of tests deemed out of scope currently and thus completely disregarded. */ From c5cbc49233193836b321cb6b77ce69dae798570b Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 3 Sep 2014 19:08:39 -0700 Subject: [PATCH 0587/1492] [SPARK-3335] [SQL] [PySpark] support broadcast in Python UDF After this patch, broadcast can be used in Python UDF. Author: Davies Liu Closes #2243 from davies/udf_broadcast and squashes the following commits: 7b88861 [Davies Liu] support broadcast in UDF --- python/pyspark/sql.py | 17 +++++++------- python/pyspark/tests.py | 22 +++++++++++++++++++ .../apache/spark/sql/UdfRegistration.scala | 3 +++ .../spark/sql/execution/pythonUdfs.scala | 3 ++- 4 files changed, 36 insertions(+), 9 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 44316926ba334..aaa35dadc203e 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -942,9 +942,7 @@ def __init__(self, sparkContext, sqlContext=None): self._jsc = self._sc._jsc self._jvm = self._sc._jvm self._pythonToJava = self._jvm.PythonRDD.pythonToJavaArray - - if sqlContext: - self._scala_SQLContext = sqlContext + self._scala_SQLContext = sqlContext @property def _ssql_ctx(self): @@ -953,7 +951,7 @@ def _ssql_ctx(self): Subclasses can override this property to provide their own JVM Contexts. """ - if not hasattr(self, '_scala_SQLContext'): + if self._scala_SQLContext is None: self._scala_SQLContext = self._jvm.SQLContext(self._jsc.sc()) return self._scala_SQLContext @@ -970,23 +968,26 @@ def registerFunction(self, name, f, returnType=StringType()): >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] - >>> sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) - >>> sqlCtx.sql("SELECT twoArgs('test', 1)").collect() - [Row(c0=5)] """ func = lambda _, it: imap(lambda x: f(*x), it) command = (func, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) + pickled_command = CloudPickleSerializer().dumps(command) + broadcast_vars = ListConverter().convert( + [x._jbroadcast for x in self._sc._pickled_broadcast_vars], + self._sc._gateway._gateway_client) + self._sc._pickled_broadcast_vars.clear() env = MapConverter().convert(self._sc.environment, self._sc._gateway._gateway_client) includes = ListConverter().convert(self._sc._python_includes, self._sc._gateway._gateway_client) self._ssql_ctx.registerPython(name, - bytearray(CloudPickleSerializer().dumps(command)), + bytearray(pickled_command), env, includes, self._sc.pythonExec, + broadcast_vars, self._sc._javaAccumulator, str(returnType)) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f1a75cbff5c19..3e74799e82845 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -43,6 +43,7 @@ from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter +from pyspark.sql import SQLContext, IntegerType _have_scipy = False _have_numpy = False @@ -525,6 +526,27 @@ def test_histogram(self): self.assertRaises(TypeError, lambda: rdd.histogram(2)) +class TestSQL(PySparkTestCase): + + def setUp(self): + PySparkTestCase.setUp(self) + self.sqlCtx = SQLContext(self.sc) + + def test_udf(self): + self.sqlCtx.registerFunction("twoArgs", lambda x, y: len(x) + y, IntegerType()) + [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() + self.assertEqual(row[0], 5) + + def test_broadcast_in_udf(self): + bar = {"a": "aa", "b": "bb", "c": "abc"} + foo = self.sc.broadcast(bar) + self.sqlCtx.registerFunction("MYUDF", lambda x: foo.value[x] if x else '') + [res] = self.sqlCtx.sql("SELECT MYUDF('c')").collect() + self.assertEqual("abc", res[0]) + [res] = self.sqlCtx.sql("SELECT MYUDF('')").collect() + self.assertEqual("", res[0]) + + class TestIO(PySparkTestCase): def test_stdout_redirection(self): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 0b48e9e659faa..0ea1105f082a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.util.{List => JList, Map => JMap} import org.apache.spark.Accumulator +import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} import org.apache.spark.sql.execution.PythonUDF @@ -38,6 +39,7 @@ protected[sql] trait UDFRegistration { envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, + broadcastVars: JList[Broadcast[Array[Byte]]], accumulator: Accumulator[JList[Array[Byte]]], stringDataType: String): Unit = { log.debug( @@ -61,6 +63,7 @@ protected[sql] trait UDFRegistration { envVars, pythonIncludes, pythonExec, + broadcastVars, accumulator, dataType, e) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 3dc8be2456781..0977da3e8577c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -42,6 +42,7 @@ private[spark] case class PythonUDF( envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, + broadcastVars: JList[Broadcast[Array[Byte]]], accumulator: Accumulator[JList[Array[Byte]]], dataType: DataType, children: Seq[Expression]) extends Expression with SparkLogging { @@ -145,7 +146,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: udf.pythonIncludes, false, udf.pythonExec, - Seq[Broadcast[Array[Byte]]](), + udf.broadcastVars, udf.accumulator ).mapPartitions { iter => val pickle = new Unpickler From 7c6e71f05f4f5e0cd2d038ee81d1cda4a3e5cb39 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Wed, 3 Sep 2014 19:37:37 -0700 Subject: [PATCH 0588/1492] [SPARK-2435] Add shutdown hook to pyspark Author: Matthew Farrellee Closes #2183 from mattf/SPARK-2435 and squashes the following commits: ee0ee99 [Matthew Farrellee] [SPARK-2435] Add shutdown hook to pyspark --- python/pyspark/shell.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index e1e7cd954189f..fde3c29e5e790 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -28,6 +28,7 @@ sys.exit(1) +import atexit import os import platform import pyspark @@ -42,6 +43,7 @@ SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) sc = SparkContext(appName="PySparkShell", pyFiles=add_files) +atexit.register(lambda: sc.stop()) print("""Welcome to ____ __ From 1bed0a3869a526241381d2a74ba064e5b3721336 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 3 Sep 2014 20:47:00 -0700 Subject: [PATCH 0589/1492] [SPARK-3372] [MLlib] MLlib doesn't pass maven build / checkstyle due to multi-byte character contained in Gradient.scala Author: Kousuke Saruta Closes #2248 from sarutak/SPARK-3372 and squashes the following commits: 73a28b8 [Kousuke Saruta] Replaced UTF-8 hyphen with ascii hyphen --- .../scala/org/apache/spark/mllib/optimization/Gradient.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index fdd67160114ca..45dbf6044fcc5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -128,7 +128,7 @@ class LeastSquaresGradient extends Gradient { class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val dotProduct = dot(data, weights) - // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) + // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 if (1.0 > labelScaled * dotProduct) { @@ -146,7 +146,7 @@ class HingeGradient extends Gradient { weights: Vector, cumGradient: Vector): Double = { val dotProduct = dot(data, weights) - // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) + // Our loss function with {0, 1} labels is max(0, 1 - (2y - 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 if (1.0 > labelScaled * dotProduct) { From 00362dac976cd05b06638deb11d990d612429e0b Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 3 Sep 2014 23:49:47 -0700 Subject: [PATCH 0590/1492] [HOTFIX] [SPARK-3400] Revert 9b225ac "fix GraphX EdgeRDD zipPartitions" 9b225ac3072de522b40b46aba6df1f1c231f13ef has been causing GraphX tests to fail nondeterministically, which is blocking development for others. Author: Ankur Dave Closes #2271 from ankurdave/SPARK-3400 and squashes the following commits: 10c2a97 [Ankur Dave] [HOTFIX] [SPARK-3400] Revert 9b225ac "fix GraphX EdgeRDD zipPartitions" --- .../scala/org/apache/spark/graphx/EdgeRDD.scala | 4 ++-- .../org/apache/spark/graphx/GraphSuite.scala | 16 ---------------- 2 files changed, 2 insertions(+), 18 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 35fbd47e6c2a5..5bcb96b136ed7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.graphx import scala.reflect.{classTag, ClassTag} -import org.apache.spark._ +import org.apache.spark.{OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -55,7 +55,7 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( * partitioner that allows co-partitioning with `partitionsRDD`. */ override val partitioner = - partitionsRDD.partitioner.orElse(Some(new HashPartitioner(partitionsRDD.partitions.size))) + partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { val p = firstParent[(PartitionID, EdgePartition[ED, VD])].iterator(part, context) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index eaaa4499b6b93..6506bac73d71c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.graphx import org.scalatest.FunSuite -import org.apache.spark.SparkConf import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ @@ -351,19 +350,4 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } - test("non-default number of edge partitions") { - val n = 10 - val defaultParallelism = 3 - val numEdgePartitions = 4 - assert(defaultParallelism != numEdgePartitions) - val conf = new SparkConf() - .set("spark.default.parallelism", defaultParallelism.toString) - val sc = new SparkContext("local", "test", conf) - val edges = sc.parallelize((1 to n).map(x => (x: VertexId, 0: VertexId)), - numEdgePartitions) - val graph = Graph.fromEdgeTuples(edges, 1) - val neighborAttrSums = graph.mapReduceTriplets[Int]( - et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(neighborAttrSums.collect.toSet === Set((0: VertexId, n))) - } } From 905861906e250a0d6e6c91d9b8fd1aaa366781d3 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Thu, 4 Sep 2014 10:28:23 -0700 Subject: [PATCH 0591/1492] [Minor]Remove extra semicolon in FlumeStreamSuite.scala Author: GuoQiang Li Closes #2265 from witgo/FlumeStreamSuite and squashes the following commits: 6c99e6e [GuoQiang Li] Remove extra semicolon in FlumeStreamSuite.scala --- .../spark/streaming/flume/FlumeStreamSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 73dffef953309..6ee7ac974b4a0 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -109,11 +109,11 @@ class FlumeStreamSuite extends TestSuiteBase { } class CompressionChannelFactory(compressionLevel: Int) extends NioClientSocketChannelFactory { - override def newChannel(pipeline:ChannelPipeline) : SocketChannel = { - var encoder : ZlibEncoder = new ZlibEncoder(compressionLevel); - pipeline.addFirst("deflater", encoder); - pipeline.addFirst("inflater", new ZlibDecoder()); - super.newChannel(pipeline); + override def newChannel(pipeline: ChannelPipeline): SocketChannel = { + val encoder = new ZlibEncoder(compressionLevel) + pipeline.addFirst("deflater", encoder) + pipeline.addFirst("inflater", new ZlibDecoder()) + super.newChannel(pipeline) } } } From 4feb46c5feca8d48ec340dc9c8d0eccbcd41f505 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 10:29:11 -0700 Subject: [PATCH 0592/1492] [SPARK-3401][PySpark] Wrong usage of tee command in python/run-tests Author: Kousuke Saruta Closes #2272 from sarutak/SPARK-3401 and squashes the following commits: 2b35a59 [Kousuke Saruta] Modified wrong usage of tee command in python/run-tests --- python/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/run-tests b/python/run-tests index 7b1ee3e1cddba..d671da40031c8 100755 --- a/python/run-tests +++ b/python/run-tests @@ -33,7 +33,7 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" - SPARK_TESTING=1 $FWDIR/bin/pyspark $1 2>&1 | tee -a > unit-tests.log + SPARK_TESTING=1 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log FAILED=$((PIPESTATUS[0]||$FAILED)) # Fail and exit on the first test failure. From dc1ba9e9fc169962a9282ea6644dce09281ff598 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 4 Sep 2014 15:06:08 -0700 Subject: [PATCH 0593/1492] [SPARK-3378] [DOCS] Replace the word "SparkSQL" with right word "Spark SQL" Author: Kousuke Saruta Closes #2251 from sarutak/SPARK-3378 and squashes the following commits: 0bfe234 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3378 bb5938f [Kousuke Saruta] Replaced rest of "SparkSQL" with "Spark SQL" 6df66de [Kousuke Saruta] Replaced "SparkSQL" with "Spark SQL" --- dev/run-tests | 2 +- docs/programming-guide.md | 2 +- python/pyspark/sql.py | 6 +++--- python/run-tests | 2 +- .../src/main/scala/org/apache/spark/sql/api/java/Row.scala | 2 +- .../apache/spark/sql/hive/parquet/FakeParquetSerDe.scala | 2 +- 6 files changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index d751961605dfd..90a8ce16f0f06 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -89,7 +89,7 @@ echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -# Build Spark; we always build with Hive because the PySpark SparkSQL tests need it. +# Build Spark; we always build with Hive because the PySpark Spark SQL tests need it. # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 6ae780d94046a..624cc744dfd51 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -385,7 +385,7 @@ Apart from text files, Spark's Python API also supports several other data forma * SequenceFile and Hadoop Input/Output Formats -**Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on SparkSQL, in which case SparkSQL is the preferred approach. +**Note** this feature is currently marked ```Experimental``` and is intended for advanced users. It may be replaced in future with read/write support based on Spark SQL, in which case Spark SQL is the preferred approach. **Writable Support** diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index aaa35dadc203e..e7f573cf6da44 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -900,7 +900,7 @@ def __reduce__(self): class SQLContext: - """Main entry point for SparkSQL functionality. + """Main entry point for Spark SQL functionality. A SQLContext can be used create L{SchemaRDD}s, register L{SchemaRDD}s as tables, execute SQL over tables, cache tables, and read parquet files. @@ -946,7 +946,7 @@ def __init__(self, sparkContext, sqlContext=None): @property def _ssql_ctx(self): - """Accessor for the JVM SparkSQL context. + """Accessor for the JVM Spark SQL context. Subclasses can override this property to provide their own JVM Contexts. @@ -1507,7 +1507,7 @@ class SchemaRDD(RDD): """An RDD of L{Row} objects that has an associated schema. The underlying JVM object is a SchemaRDD, not a PythonRDD, so we can - utilize the relational query api exposed by SparkSQL. + utilize the relational query api exposed by Spark SQL. For normal L{pyspark.rdd.RDD} operations (map, count, etc.) the L{SchemaRDD} is not operated on directly, as it's underlying diff --git a/python/run-tests b/python/run-tests index d671da40031c8..f2a80b4f1838b 100755 --- a/python/run-tests +++ b/python/run-tests @@ -28,7 +28,7 @@ FAILED=0 rm -f unit-tests.log -# Remove the metastore and warehouse directory created by the HiveContext tests in SparkSQL +# Remove the metastore and warehouse directory created by the HiveContext tests in Spark SQL rm -rf metastore warehouse function run_test() { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index 6c67934bda5b8..e9d04ce7aae4c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -25,7 +25,7 @@ import scala.math.BigDecimal import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** - * A result row from a SparkSQL query. + * A result row from a Spark SQL query. */ class Row(private[spark] val row: ScalaRow) extends Serializable { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala index 544abfc32423c..abed299cd957f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/parquet/FakeParquetSerDe.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.io.Writable /** - * A placeholder that allows SparkSQL users to create metastore tables that are stored as + * A placeholder that allows Spark SQL users to create metastore tables that are stored as * parquet files. It is only intended to pass the checks that the serde is valid and exists * when a CREATE TABLE is run. The actual work of decoding will be done by ParquetTableScan * when "spark.sql.hive.convertMetastoreParquet" is set to true. From 0fdf2f5a18d88a600ca1ab7b4bf02fb0537e9411 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 4 Sep 2014 17:47:16 -0700 Subject: [PATCH 0594/1492] Manually close old PR Closes #1588 From 90b17a70c703c403d397e24cfbc20da22a32102d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Thu, 4 Sep 2014 17:51:14 -0700 Subject: [PATCH 0595/1492] Manually close old PR Closes #544 From 3eb6ef316c2a5ee43d5ecfcf9f10c2d7adc6b819 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 4 Sep 2014 18:46:09 -0700 Subject: [PATCH 0596/1492] [SPARK-3310][SQL] Directly use currentTable without unnecessary implicit conversion We can directly use currentTable there without unnecessary implicit conversion. Author: Liang-Chi Hsieh Closes #2203 from viirya/direct_use_inmemoryrelation and squashes the following commits: 4741d02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into direct_use_inmemoryrelation b671f67 [Liang-Chi Hsieh] Can directly use currentTable there without unnecessary implicit conversion. --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a75af94d29303..5acb45c155ba5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -272,7 +272,7 @@ class SQLContext(@transient val sparkContext: SparkContext) val currentTable = table(tableName).queryExecution.analyzed val asInMemoryRelation = currentTable match { case _: InMemoryRelation => - currentTable.logicalPlan + currentTable case _ => InMemoryRelation(useCompression, columnBatchSize, executePlan(currentTable).executedPlan) From ee575f12f2ab059d9c1b4fa8d6c1e62248c3d11b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 4 Sep 2014 18:47:45 -0700 Subject: [PATCH 0597/1492] [SPARK-2219][SQL] Added support for the "add jar" command Adds logical and physical command classes for the "add jar" command. Note that this PR conflicts with and should be merged after #2215. Author: Cheng Lian Closes #2242 from liancheng/add-jar and squashes the following commits: e43a2f1 [Cheng Lian] Updates AddJar according to conventions introduced in #2215 b99107f [Cheng Lian] Added test case for ADD JAR command 095b2c7 [Cheng Lian] Also forward ADD JAR command to Hive 9be031b [Cheng Lian] Trims Jar path string 8195056 [Cheng Lian] Added support for the "add jar" command --- .../org/apache/spark/sql/hive/HiveQl.scala | 8 +++--- .../spark/sql/hive/HiveStrategies.scala | 5 ++-- .../spark/sql/hive/execution/commands.scala | 16 ++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 25 ++++++++++++++++--- 4 files changed, 46 insertions(+), 8 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 a4dd6be5f9e35..c98287c6aa662 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 @@ -44,6 +44,8 @@ private[hive] case class SourceCommand(filePath: String) extends Command private[hive] case class AddFile(filePath: String) extends Command +private[hive] case class AddJar(path: String) extends Command + private[hive] case class DropTable(tableName: String, ifExists: Boolean) extends Command private[hive] case class AnalyzeTable(tableName: String) extends Command @@ -231,7 +233,7 @@ private[hive] object HiveQl { } else if (sql.trim.toLowerCase.startsWith("uncache table")) { CacheCommand(sql.trim.drop(14).trim, false) } else if (sql.trim.toLowerCase.startsWith("add jar")) { - NativeCommand(sql) + AddJar(sql.trim.drop(8).trim) } else if (sql.trim.toLowerCase.startsWith("add file")) { AddFile(sql.trim.drop(9)) } else if (sql.trim.toLowerCase.startsWith("dfs")) { @@ -1018,9 +1020,9 @@ 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) => + 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) => + 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 */ diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 24abb1b5bd1a8..72cc01cdf4c84 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -195,11 +195,12 @@ private[hive] trait HiveStrategies { case class HiveCommandStrategy(context: HiveContext) extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.NativeCommand(sql) => - NativeCommand(sql, plan.output)(context) :: Nil + case logical.NativeCommand(sql) => NativeCommand(sql, plan.output)(context) :: Nil case hive.DropTable(tableName, ifExists) => execution.DropTable(tableName, ifExists) :: Nil + case hive.AddJar(path) => execution.AddJar(path) :: Nil + case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil case describe: logical.DescribeCommand => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index a1a4aa7de7bf7..d61c5e274a596 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -60,3 +60,19 @@ case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with Seq.empty[Row] } } + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class AddJar(path: String) extends LeafNode with Command { + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + override def output = Seq.empty + + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + hiveContext.runSqlHive(s"ADD JAR $path") + hiveContext.sparkContext.addJar(path) + Seq.empty[Row] + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index c4abb3eb4861f..f4217a52c3822 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.hive.execution +import java.io.File + import scala.util.Try -import org.apache.spark.sql.{SchemaRDD, Row} +import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -313,7 +315,7 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT srcalias.KEY, SRCALIAS.value FROM sRc SrCAlias WHERE SrCAlias.kEy < 15") test("case sensitivity: registered table") { - val testData: SchemaRDD = + val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(2, "str2") :: Nil) @@ -467,7 +469,7 @@ class HiveQuerySuite extends HiveComparisonTest { } // Describe a registered temporary table. - val testData: SchemaRDD = + val testData = TestHive.sparkContext.parallelize( TestData(1, "str1") :: TestData(1, "str2") :: Nil) @@ -495,6 +497,23 @@ class HiveQuerySuite extends HiveComparisonTest { } } + test("ADD JAR command") { + val testJar = TestHive.getHiveFile("data/files/TestSerDe.jar").getCanonicalPath + sql("CREATE TABLE alter1(a INT, b INT)") + intercept[Exception] { + sql( + """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' + |WITH serdeproperties('s1'='9') + """.stripMargin) + } + sql(s"ADD JAR $testJar") + sql( + """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' + |WITH serdeproperties('s1'='9') + """.stripMargin) + sql("DROP TABLE alter1") + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" From 1904bac38d97df5ae9fb193e92a83c7f8ff6d255 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 4 Sep 2014 19:16:12 -0700 Subject: [PATCH 0598/1492] [SPARK-3392] [SQL] Show value spark.sql.shuffle.partitions for mapred.reduce.tasks This is a tiny fix for getting the value of "mapred.reduce.tasks", which make more sense for the hive user. As well as the command "set -v", which should output verbose information for all of the key/values. Author: Cheng Hao Closes #2261 from chenghao-intel/set_mapreduce_tasks and squashes the following commits: 653858a [Cheng Hao] show value spark.sql.shuffle.partitions for mapred.reduce.tasks --- .../apache/spark/sql/execution/commands.scala | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 286c6d264f86a..94543fc95b470 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -60,10 +60,10 @@ case class SetCommand( logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) - Array(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")) } else { context.setConf(k, v) - Array(Row(s"$k=$v")) + Seq(Row(s"$k=$v")) } // Query the value bound to key k. @@ -78,11 +78,19 @@ case class SetCommand( "hive-hwi-0.12.0.jar", "hive-0.12.0.jar").mkString(":") - Array( + context.getAllConfs.map { case (k, v) => + Row(s"$k=$v") + }.toSeq ++ Seq( Row("system:java.class.path=" + hiveJars), Row("system:sun.java.command=shark.SharkServer2")) } else { - Array(Row(s"$k=${context.getConf(k, "")}")) + if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${context.numShufflePartitions}")) + } else { + Seq(Row(s"$k=${context.getConf(k, "")}")) + } } // Query all key-value pairs that are set in the SQLConf of the context. From 1725a1a5d10a53762bd80f391eddbf306f2841ee Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 4 Sep 2014 23:34:58 -0700 Subject: [PATCH 0599/1492] [SPARK-3391][EC2] Support attaching up to 8 EBS volumes. Please merge this at the same time as https://github.com/mesos/spark-ec2/pull/66 Author: Reynold Xin Closes #2260 from rxin/ec2-ebs-vol and squashes the following commits: b9527d9 [Reynold Xin] Removed io1 ebs type. bf9c403 [Reynold Xin] Made EBS volume type configurable. c8e25ea [Reynold Xin] Support up to 8 EBS volumes. adf4f2e [Reynold Xin] Revert git repo change. 020c542 [Reynold Xin] [SPARK-3391] Support attaching more than 1 EBS volumes. --- ec2/spark_ec2.py | 33 +++++++++++++++++++++++++-------- 1 file changed, 25 insertions(+), 8 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index eed6eb8485183..1670faca4a480 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -102,9 +102,17 @@ def parse_args(): "(for debugging)") parser.add_option( "--ebs-vol-size", metavar="SIZE", type="int", default=0, - help="Attach a new EBS volume of size SIZE (in GB) to each node as " + - "/vol. The volumes will be deleted when the instances terminate. " + - "Only possible on EBS-backed AMIs.") + help="Size (in GB) of each EBS volume.") + parser.add_option( + "--ebs-vol-type", default="standard", + help="EBS volume type (e.g. 'gp2', 'standard').") + parser.add_option( + "--ebs-vol-num", type="int", default=1, + help="Number of EBS volumes to attach to each node as /vol[x]. " + + "The volumes will be deleted when the instances terminate. " + + "Only possible on EBS-backed AMIs. " + + "EBS volumes are only attached if --ebs-vol-size > 0." + + "Only support up to 8 EBS volumes.") parser.add_option( "--swap", metavar="SWAP", type="int", default=1024, help="Swap space to set up per node, in MB (default: 1024)") @@ -348,13 +356,16 @@ def launch_cluster(conn, opts, cluster_name): print >> stderr, "Could not find AMI " + opts.ami sys.exit(1) - # Create block device mapping so that we can add an EBS volume if asked to + # Create block device mapping so that we can add EBS volumes if asked to. + # The first drive is attached as /dev/sds, 2nd as /dev/sdt, ... /dev/sdz block_map = BlockDeviceMapping() if opts.ebs_vol_size > 0: - device = EBSBlockDeviceType() - device.size = opts.ebs_vol_size - device.delete_on_termination = True - block_map["/dev/sdv"] = device + for i in range(opts.ebs_vol_num): + device = EBSBlockDeviceType() + device.size = opts.ebs_vol_size + device.volume_type=opts.ebs_vol_type + device.delete_on_termination = True + block_map["/dev/sd" + chr(ord('s') + i)] = device # AWS ignores the AMI-specified block device mapping for M3 (see SPARK-3342). if opts.instance_type.startswith('m3.'): @@ -828,6 +839,12 @@ def get_partition(total, num_partitions, current_partitions): def real_main(): (opts, action, cluster_name) = parse_args() + + # Input parameter validation + if opts.ebs_vol_num > 8: + print >> stderr, "ebs-vol-num cannot be greater than 8" + sys.exit(1) + try: conn = ec2.connect_to_region(opts.region) except Exception as e: From 6a37ed838b3cbf96f7a904f3d3dabf99141729f5 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Thu, 4 Sep 2014 23:37:06 -0700 Subject: [PATCH 0600/1492] [Docs] fix minor MLlib case typo Also make the list of features consistent in style. Author: Nicholas Chammas Closes #2278 from nchammas/patch-1 and squashes the following commits: 56df319 [Nicholas Chammas] [Docs] fix minor MLlib case typo --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 0a683a460ffac..5b09ad86849e7 100644 --- a/README.md +++ b/README.md @@ -4,8 +4,8 @@ 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 -data processing, MLLib for machine learning, GraphX for graph processing, -and Spark Streaming. +data processing, MLlib for machine learning, GraphX for graph processing, +and Spark Streaming for stream processing. From 51b53a758c85f2e20ad9bd73ed815fcfa9c7180b Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 5 Sep 2014 09:54:40 -0500 Subject: [PATCH 0601/1492] [SPARK-3260] yarn - pass acls along with executor launch Pass along the acl settings when we launch a container so that they can be applied to viewing the logs on a running NodeManager. Author: Thomas Graves Closes #2185 from tgravescs/SPARK-3260 and squashes the following commits: 6f94b5a [Thomas Graves] make unit test more robust 28b9dd3 [Thomas Graves] yarn - pass acls along with executor launch --- .../spark/deploy/yarn/ExecutorRunnable.scala | 7 +- .../deploy/yarn/YarnAllocationHandler.scala | 7 +- .../spark/deploy/yarn/YarnRMClientImpl.scala | 7 +- .../spark/deploy/yarn/ApplicationMaster.scala | 13 ++-- .../apache/spark/deploy/yarn/ClientBase.scala | 6 +- .../spark/deploy/yarn/YarnAllocator.scala | 10 ++- .../spark/deploy/yarn/YarnRMClient.scala | 5 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 11 ++- .../yarn/YarnSparkHadoopUtilSuite.scala | 76 ++++++++++++++++++- .../spark/deploy/yarn/ExecutorRunnable.scala | 7 +- .../deploy/yarn/YarnAllocationHandler.scala | 7 +- .../spark/deploy/yarn/YarnRMClientImpl.scala | 7 +- 12 files changed, 129 insertions(+), 34 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 7dae248e3e7db..10cbeb8b94325 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{SecurityManager, SparkConf, Logging} class ExecutorRunnable( @@ -46,7 +46,8 @@ class ExecutorRunnable( slaveId: String, hostname: String, executorMemory: Int, - executorCores: Int) + executorCores: Int, + securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) @@ -86,6 +87,8 @@ class ExecutorRunnable( logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) + ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) + // Send the start request to the ContainerManager val startReq = Records.newRecord(classOf[StartContainerRequest]) .asInstanceOf[StartContainerRequest] diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 9f9e16c06452b..85d6274df2fcb 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration @@ -41,8 +41,9 @@ private[yarn] class YarnAllocationHandler( resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator(conf, sparkConf, args, preferredNodes) { + preferredNodes: collection.Map[String, collection.Set[SplitInfo]], + securityMgr: SecurityManager) + extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { private val lastResponseId = new AtomicInteger() private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index cc5392192ec51..ad27a9ab781d2 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.Utils @@ -45,7 +45,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC sparkConf: SparkConf, preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, - uiHistoryAddress: String) = { + uiHistoryAddress: String, + securityMgr: SecurityManager) = { this.rpc = YarnRPC.create(conf) this.uiHistoryAddress = uiHistoryAddress @@ -53,7 +54,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC registerApplicationMaster(uiAddress) new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, - preferredNodeLocations) + preferredNodeLocations, securityMgr) } override def getAttemptId() = { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 98039a20de245..a879c833a014f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -116,7 +116,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val securityMgr = new SecurityManager(sparkConf) if (isDriver) { - runDriver() + runDriver(securityMgr) } else { runExecutorLauncher(securityMgr) } @@ -157,7 +157,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, sparkContextRef.compareAndSet(sc, null) } - private def registerAM(uiAddress: String) = { + private def registerAM(uiAddress: String, securityMgr: SecurityManager) = { val sc = sparkContextRef.get() val appId = client.getAttemptId().getApplicationId().toString() @@ -170,13 +170,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc != null) sc.getConf else sparkConf, if (sc != null) sc.preferredNodeLocationData else Map(), uiAddress, - historyAddress) + historyAddress, + securityMgr) allocator.allocateResources() reporterThread = launchReporterThread() } - private def runDriver(): Unit = { + private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() val userThread = startUserClass() @@ -188,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIHostPort) + registerAM(sc.ui.appUIHostPort, securityMgr) try { userThread.join() } finally { @@ -203,7 +204,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, conf = sparkConf, securityManager = securityMgr)._1 actor = waitForSparkDriver() addAmIpFilter() - registerAM(sparkConf.get("spark.driver.appUIAddress", "")) + registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr) // In client mode the actor will stop the reporter thread. reporterThread.join() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 5d8e5e6dffe7f..8075b7a7fb837 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -430,10 +430,8 @@ trait ClientBase extends Logging { // send the acl settings into YARN to control who has access via YARN interfaces val securityManager = new SecurityManager(sparkConf) - val acls = Map[ApplicationAccessType, String] ( - ApplicationAccessType.VIEW_APP -> securityManager.getViewAcls, - ApplicationAccessType.MODIFY_APP -> securityManager.getModifyAcls) - amContainer.setApplicationACLs(acls) + amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) + amContainer } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index c74dd1c2b21dc..02b9a81bf6b50 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse -import org.apache.spark.{Logging, SparkConf, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -55,7 +55,8 @@ private[yarn] abstract class YarnAllocator( conf: Configuration, sparkConf: SparkConf, args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) + preferredNodes: collection.Map[String, collection.Set[SplitInfo]], + securityMgr: SecurityManager) extends Logging { // These three are locked on allocatedHostToContainersMap. Complementary data structures @@ -280,7 +281,8 @@ private[yarn] abstract class YarnAllocator( executorId, executorHostname, executorMemory, - executorCores) + executorCores, + securityMgr) new Thread(executorRunnable).start() } } @@ -444,4 +446,4 @@ private[yarn] abstract class YarnAllocator( } -} \ No newline at end of file +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 922d7d1a854a5..ed65e56b3e413 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -22,7 +22,7 @@ import scala.collection.{Map, Set} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.records._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler.SplitInfo /** @@ -45,7 +45,8 @@ trait YarnRMClient { sparkConf: SparkConf, preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, - uiHistoryAddress: String): YarnAllocator + uiHistoryAddress: String, + securityMgr: SecurityManager): YarnAllocator /** * Shuts down the AM. Guaranteed to only be called once. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index dc77f1236492d..4a33e34c3bfc7 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -32,10 +32,11 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants +import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -211,4 +212,12 @@ object YarnSparkHadoopUtil { } } + private[spark] def getApplicationAclsForYarn(securityMgr: SecurityManager): + Map[ApplicationAccessType, String] = { + Map[ApplicationAccessType, String] ( + ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, + ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls + ) + } + } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 75db8ee6d468f..2cc5abb3a890c 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -23,7 +23,10 @@ import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.{FunSuite, Matchers} -import org.apache.spark.{Logging, SparkConf} +import org.apache.hadoop.yarn.api.records.ApplicationAccessType + +import org.apache.spark.{Logging, SecurityManager, SparkConf} + class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { @@ -74,4 +77,75 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { yarnConf.get(key) should not be default.get(key) } + + test("test getApplicationAclsForYarn acls on") { + + // spark acls on, just pick up default user + val sparkConf = new SparkConf() + sparkConf.set("spark.acls.enable", "true") + + val securityMgr = new SecurityManager(sparkConf) + val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) + + val viewAcls = acls.get(ApplicationAccessType.VIEW_APP) + val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP) + + viewAcls match { + case Some(vacls) => { + val aclSet = vacls.split(',').map(_.trim).toSet + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + modifyAcls match { + case Some(macls) => { + val aclSet = macls.split(',').map(_.trim).toSet + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + } + + test("test getApplicationAclsForYarn acls on and specify users") { + + // default spark acls are on and specify acls + val sparkConf = new SparkConf() + sparkConf.set("spark.acls.enable", "true") + sparkConf.set("spark.ui.view.acls", "user1,user2") + sparkConf.set("spark.modify.acls", "user3,user4") + + val securityMgr = new SecurityManager(sparkConf) + val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) + + val viewAcls = acls.get(ApplicationAccessType.VIEW_APP) + val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP) + + viewAcls match { + case Some(vacls) => { + val aclSet = vacls.split(',').map(_.trim).toSet + assert(aclSet.contains("user1")) + assert(aclSet.contains("user2")) + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + modifyAcls match { + case Some(macls) => { + val aclSet = macls.split(',').map(_.trim).toSet + assert(aclSet.contains("user3")) + assert(aclSet.contains("user4")) + assert(aclSet.contains(System.getProperty("user.name", "invalid"))) + } + case None => { + fail() + } + } + + } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 07ba0a4b30bd7..833be12982e71 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} -import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.{SecurityManager, SparkConf, Logging} class ExecutorRunnable( @@ -46,7 +46,8 @@ class ExecutorRunnable( slaveId: String, hostname: String, executorMemory: Int, - executorCores: Int) + executorCores: Int, + securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) @@ -85,6 +86,8 @@ class ExecutorRunnable( logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) + ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) + // Send the start request to the ContainerManager nmClient.startContainer(container, ctx) } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index ed31457b61571..c887cb52dd9cf 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.conf.Configuration @@ -39,8 +39,9 @@ private[yarn] class YarnAllocationHandler( amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) - extends YarnAllocator(conf, sparkConf, args, preferredNodes) { + preferredNodes: collection.Map[String, collection.Set[SplitInfo]], + securityMgr: SecurityManager) + extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { override protected def releaseContainer(container: Container) = { amClient.releaseAssignedContainer(container.getId()) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index e8b8d9bc722bd..54bc6b14c44ce 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.hadoop.yarn.webapp.util.WebAppUtils -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.Utils @@ -46,7 +46,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC sparkConf: SparkConf, preferredNodeLocations: Map[String, Set[SplitInfo]], uiAddress: String, - uiHistoryAddress: String) = { + uiHistoryAddress: String, + securityMgr: SecurityManager) = { amClient = AMRMClient.createAMRMClient() amClient.init(conf) amClient.start() @@ -55,7 +56,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC logInfo("Registering the ApplicationMaster") amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, - preferredNodeLocations) + preferredNodeLocations, securityMgr) } override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = From 62c557609929982eeec170fe12f810bedfcf97f2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 5 Sep 2014 09:56:22 -0500 Subject: [PATCH 0602/1492] [SPARK-3375] spark on yarn container allocation issues If yarn doesn't get the containers immediately it stops asking for them and the yarn application hangs with never getting any executors. The issue here is that we are sending the number of containers as 0 after we send the original one of X. on the yarn side this clears out the original request. For a ping we should just send empty asks. Author: Thomas Graves Closes #2275 from tgravescs/SPARK-3375 and squashes the following commits: 74b6820 [Thomas Graves] send empty resource requests when we aren't asking for containers --- .../spark/deploy/yarn/YarnAllocationHandler.scala | 13 +++++++------ .../spark/deploy/yarn/YarnAllocationHandler.scala | 8 +++++--- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 85d6274df2fcb..5a1b42c1e17d5 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -51,12 +51,13 @@ private[yarn] class YarnAllocationHandler( override protected def allocateContainers(count: Int): YarnAllocateResponse = { var resourceRequests: List[ResourceRequest] = null - // default. - if (count <= 0 || preferredHostToCount.isEmpty) { - logDebug("numExecutors: " + count + ", host preferences: " + - preferredHostToCount.isEmpty) - resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) + logDebug("numExecutors: " + count) + if (count <= 0) { + resourceRequests = List() + } else if (preferredHostToCount.isEmpty) { + logDebug("host preferences is empty") + resourceRequests = List(createResourceRequest( + AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index c887cb52dd9cf..5438f151ac0ad 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -88,9 +88,11 @@ private[yarn] class YarnAllocationHandler( private def addResourceRequests(numExecutors: Int) { val containerRequests: List[ContainerRequest] = - if (numExecutors <= 0 || preferredHostToCount.isEmpty) { - logDebug("numExecutors: " + numExecutors + ", host preferences: " + - preferredHostToCount.isEmpty) + if (numExecutors <= 0) { + logDebug("numExecutors: " + numExecutors) + List() + } else if (preferredHostToCount.isEmpty) { + logDebug("host preferences is empty") createResourceRequests( AllocationType.ANY, resource = null, From 7ff8c45d714e0f2315910838b739c0c034672015 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 5 Sep 2014 11:07:00 -0700 Subject: [PATCH 0603/1492] [SPARK-3399][PySpark] Test for PySpark should ignore HADOOP_CONF_DIR and YARN_CONF_DIR Author: Kousuke Saruta Closes #2270 from sarutak/SPARK-3399 and squashes the following commits: 7613be6 [Kousuke Saruta] Modified pyspark script to ignore environment variables YARN_CONF_DIR and HADOOP_CONF_DIR while testing --- bin/pyspark | 2 ++ 1 file changed, 2 insertions(+) diff --git a/bin/pyspark b/bin/pyspark index f553b314c5991..26a16dd600b7a 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -85,6 +85,8 @@ export PYSPARK_SUBMIT_ARGS # For pyspark tests if [[ -n "$SPARK_TESTING" ]]; then + unset YARN_CONF_DIR + unset HADOOP_CONF_DIR if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_PYTHON" -m doctest $1 else From ba5bcaddecd54811d45c5fc79a013b3857d4c633 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Fri, 5 Sep 2014 18:52:05 -0700 Subject: [PATCH 0604/1492] SPARK-3211 .take() is OOM-prone with empty partitions Instead of jumping straight from 1 partition to all partitions, do exponential growth and double the number of partitions to attempt each time instead. Fix proposed by Paul Nepywoda Author: Andrew Ash Closes #2117 from ash211/SPARK-3211 and squashes the following commits: 8b2299a [Andrew Ash] Quadruple instead of double for a minor speedup e5f7e4d [Andrew Ash] Update comment to better reflect what we're doing 09a27f7 [Andrew Ash] Update PySpark to be less OOM-prone as well 3a156b8 [Andrew Ash] SPARK-3211 .take() is OOM-prone with empty partitions --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 7 +++---- python/pyspark/rdd.py | 8 ++++---- 2 files changed, 7 insertions(+), 8 deletions(-) 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 af9e31ba7b720..1cf55e86f6c81 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1064,11 +1064,10 @@ abstract class RDD[T: ClassTag]( // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 1 if (partsScanned > 0) { - // If we didn't find any rows after the first iteration, just try all partitions next. - // Otherwise, interpolate the number of partitions we need to try, but overestimate it - // by 50%. + // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise, + // interpolate the number of partitions we need to try, but overestimate it by 50%. if (buf.size == 0) { - numPartsToTry = totalParts - 1 + numPartsToTry = partsScanned * 4 } else { numPartsToTry = (1.5 * num * partsScanned / buf.size).toInt } diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index dff6fc26fcb18..04f13523b431d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1089,11 +1089,11 @@ def take(self, num): # we actually cap it at totalParts in runJob. numPartsToTry = 1 if partsScanned > 0: - # If we didn't find any rows after the first iteration, just - # try all partitions next. Otherwise, interpolate the number - # of partitions we need to try, but overestimate it by 50%. + # If we didn't find any rows after the previous iteration, + # quadruple and retry. Otherwise, interpolate the number of + # partitions we need to try, but overestimate it by 50%. if len(items) == 0: - numPartsToTry = totalParts - 1 + numPartsToTry = partsScanned * 4 else: numPartsToTry = int(1.5 * num * partsScanned / len(items)) From 19f61c165932059e7ce156da2c71429fa8dc27f0 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 5 Sep 2014 21:46:45 -0700 Subject: [PATCH 0605/1492] [Build] suppress curl/wget progress bars In the Jenkins console output, `curl` gives us mountains of `#` symbols as it tries to show its download progress. ![noise from curl in Jenkins output](http://i.imgur.com/P2E7yUw.png) I don't think this is useful so I've changed things to suppress these progress bars. If there is actually some use to this, feel free to reject this proposal. Author: Nicholas Chammas Closes #2279 from nchammas/trim-test-output and squashes the following commits: 14a720c [Nicholas Chammas] suppress curl/wget progress bars --- dev/check-license | 4 ++-- sbt/sbt-launch-lib.bash | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/dev/check-license b/dev/check-license index 625ec161bc571..558e038afc01a 100755 --- a/dev/check-license +++ b/dev/check-license @@ -32,9 +32,9 @@ acquire_rat_jar () { printf "Attempting to fetch rat\n" JAR_DL=${JAR}.part if hash curl 2>/dev/null; then - (curl --progress-bar ${URL1} > "$JAR_DL" || curl --progress-bar ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" + (curl --silent ${URL1} > "$JAR_DL" || curl --silent ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then - (wget --progress=bar ${URL1} -O "$JAR_DL" || wget --progress=bar ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" + (wget --quiet ${URL1} -O "$JAR_DL" || wget --quiet ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" else printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index c91fecf024ad4..fecc3d38a5fbd 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -51,9 +51,9 @@ acquire_sbt_jar () { printf "Attempting to fetch sbt\n" JAR_DL=${JAR}.part if hash curl 2>/dev/null; then - (curl --progress-bar ${URL1} > ${JAR_DL} || curl --progress-bar ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (curl --silent ${URL1} > ${JAR_DL} || curl --silent ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} elif hash wget 2>/dev/null; then - (wget --progress=bar ${URL1} -O ${JAR_DL} || wget --progress=bar ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (wget --quiet ${URL1} -O ${JAR_DL} || wget --quiet ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} else printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" exit -1 From 9422c4ee0eaf4a32d2ed7c96799feac2f5f79d40 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 5 Sep 2014 23:08:54 -0700 Subject: [PATCH 0606/1492] [SPARK-3361] Expand PEP 8 checks to include EC2 script and Python examples This PR resolves [SPARK-3361](https://issues.apache.org/jira/browse/SPARK-3361) by expanding the PEP 8 checks to cover the remaining Python code base: * The EC2 script * All Python / PySpark examples Author: Nicholas Chammas Closes #2297 from nchammas/pep8-rulez and squashes the following commits: 1e5ac9a [Nicholas Chammas] PEP 8 fixes to Python examples c3dbeff [Nicholas Chammas] PEP 8 fixes to EC2 script 65ef6e8 [Nicholas Chammas] expand PEP 8 checks --- dev/lint-python | 5 ++-- ec2/spark_ec2.py | 20 +++++++++++----- examples/src/main/python/avro_inputformat.py | 17 +++++++++----- .../src/main/python/cassandra_inputformat.py | 15 ++++++------ .../src/main/python/cassandra_outputformat.py | 23 ++++++++++--------- examples/src/main/python/hbase_inputformat.py | 10 +++++--- .../src/main/python/hbase_outputformat.py | 18 +++++++++------ .../src/main/python/mllib/correlations.py | 2 +- .../main/python/mllib/decision_tree_runner.py | 6 +++-- .../python/mllib/random_rdd_generation.py | 6 ++--- .../src/main/python/mllib/sampled_rdds.py | 8 +++---- examples/src/main/python/pi.py | 2 +- 12 files changed, 79 insertions(+), 53 deletions(-) diff --git a/dev/lint-python b/dev/lint-python index a1e890faa8fa6..79bf70f0b8b13 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -30,6 +30,7 @@ cd $SPARK_ROOT_DIR #+ - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?)) PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py" PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py" +PEP8_PATHS_TO_CHECK="./python/pyspark/ ./ec2/spark_ec2.py ./examples/src/main/python/" curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH" curl_status=$? @@ -44,7 +45,7 @@ fi #+ first, but we do so so that the check status can #+ be output before the report, like with the #+ scalastyle and RAT checks. -python $PEP8_SCRIPT_PATH ./python/pyspark > "$PEP8_REPORT_PATH" +python $PEP8_SCRIPT_PATH $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" pep8_status=${PIPESTATUS[0]} #$? if [ $pep8_status -ne 0 ]; then @@ -54,7 +55,7 @@ else echo "PEP 8 checks passed." fi -rm -f "$PEP8_REPORT_PATH" +rm "$PEP8_REPORT_PATH" rm "$PEP8_SCRIPT_PATH" exit $pep8_status diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 1670faca4a480..8ec88d95e34af 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -41,6 +41,7 @@ # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" + class UsageError(Exception): pass @@ -342,7 +343,6 @@ def launch_cluster(conn, opts, cluster_name): if opts.ami is None: opts.ami = get_spark_ami(opts) - additional_groups = [] if opts.additional_security_group: additional_groups = [sg @@ -363,7 +363,7 @@ def launch_cluster(conn, opts, cluster_name): for i in range(opts.ebs_vol_num): device = EBSBlockDeviceType() device.size = opts.ebs_vol_size - device.volume_type=opts.ebs_vol_type + device.volume_type = opts.ebs_vol_type device.delete_on_termination = True block_map["/dev/sd" + chr(ord('s') + i)] = device @@ -495,6 +495,7 @@ def launch_cluster(conn, opts, cluster_name): # Return all the instances return (master_nodes, slave_nodes) + def tag_instance(instance, name): for i in range(0, 5): try: @@ -507,9 +508,12 @@ def tag_instance(instance, name): # Get the EC2 instances in an existing cluster if available. # Returns a tuple of lists of EC2 instance objects for the masters and slaves + + def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." - # Search all the spot instance requests, and copy any tags from the spot instance request to the cluster. + # Search all the spot instance requests, and copy any tags from the spot + # instance request to the cluster. spot_instance_requests = conn.get_all_spot_instance_requests() for req in spot_instance_requests: if req.state != u'active': @@ -520,7 +524,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): for res in reservations: active = [i for i in res.instances if is_active(i)] for instance in active: - if (instance.tags.get(u'Name') == None): + if (instance.tags.get(u'Name') is None): tag_instance(instance, name) # Now proceed to detect master and slaves instances. reservations = conn.get_all_instances() @@ -540,13 +544,16 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in with name " + cluster_name + "-master" + print >> sys.stderr, "ERROR: Could not find master in with name " + \ + cluster_name + "-master" else: print >> sys.stderr, "ERROR: Could not find any existing cluster" sys.exit(1) # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. + + def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = master_nodes[0].public_dns_name if deploy_ssh_key: @@ -890,7 +897,8 @@ def real_main(): if opts.security_group_prefix is None: group_names = [cluster_name + "-master", cluster_name + "-slaves"] else: - group_names = [opts.security_group_prefix + "-master", opts.security_group_prefix + "-slaves"] + group_names = [opts.security_group_prefix + "-master", + opts.security_group_prefix + "-slaves"] attempt = 1 while attempt <= 3: diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index e902ae29753c0..cfda8d8327aa3 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -23,7 +23,8 @@ Read data file users.avro in local Spark distro: $ cd $SPARK_HOME -$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +$ ./bin/spark-submit --driver-class-path /path/to/example/jar \ +> ./examples/src/main/python/avro_inputformat.py \ > examples/src/main/resources/users.avro {u'favorite_color': None, u'name': u'Alyssa', u'favorite_numbers': [3, 9, 15, 20]} {u'favorite_color': u'red', u'name': u'Ben', u'favorite_numbers': []} @@ -40,7 +41,8 @@ ] } -$ ./bin/spark-submit --driver-class-path /path/to/example/jar ./examples/src/main/python/avro_inputformat.py \ +$ ./bin/spark-submit --driver-class-path /path/to/example/jar \ +> ./examples/src/main/python/avro_inputformat.py \ > examples/src/main/resources/users.avro examples/src/main/resources/user.avsc {u'favorite_color': None, u'name': u'Alyssa'} {u'favorite_color': u'red', u'name': u'Ben'} @@ -51,8 +53,10 @@ Usage: avro_inputformat [reader_schema_file] Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/avro_inputformat.py [reader_schema_file] - Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/avro_inputformat.py [reader_schema_file] + Assumes you have Avro data stored in . Reader schema can be optionally specified + in [reader_schema_file]. """ exit(-1) @@ -62,9 +66,10 @@ conf = None if len(sys.argv) == 3: schema_rdd = sc.textFile(sys.argv[2], 1).collect() - conf = {"avro.schema.input.key" : reduce(lambda x, y: x+y, schema_rdd)} + conf = {"avro.schema.input.key": reduce(lambda x, y: x + y, schema_rdd)} - avro_rdd = sc.newAPIHadoopFile(path, + avro_rdd = sc.newAPIHadoopFile( + path, "org.apache.avro.mapreduce.AvroKeyInputFormat", "org.apache.avro.mapred.AvroKey", "org.apache.hadoop.io.NullWritable", diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index e4a897f61e39d..05f34b74df45a 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -51,7 +51,8 @@ Usage: cassandra_inputformat Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/cassandra_inputformat.py + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/cassandra_inputformat.py Assumes you have some data in Cassandra already, running on , in and """ exit(-1) @@ -61,12 +62,12 @@ cf = sys.argv[3] sc = SparkContext(appName="CassandraInputFormat") - conf = {"cassandra.input.thrift.address":host, - "cassandra.input.thrift.port":"9160", - "cassandra.input.keyspace":keyspace, - "cassandra.input.columnfamily":cf, - "cassandra.input.partitioner.class":"Murmur3Partitioner", - "cassandra.input.page.row.size":"3"} + conf = {"cassandra.input.thrift.address": host, + "cassandra.input.thrift.port": "9160", + "cassandra.input.keyspace": keyspace, + "cassandra.input.columnfamily": cf, + "cassandra.input.partitioner.class": "Murmur3Partitioner", + "cassandra.input.page.row.size": "3"} cass_rdd = sc.newAPIHadoopRDD( "org.apache.cassandra.hadoop.cql3.CqlPagingInputFormat", "java.util.Map", diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index 836c35b5c6794..d144539e58b8f 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -50,7 +50,8 @@ Usage: cassandra_outputformat Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/cassandra_outputformat.py + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/cassandra_outputformat.py Assumes you have created the following table in Cassandra already, running on , in . @@ -67,16 +68,16 @@ cf = sys.argv[3] sc = SparkContext(appName="CassandraOutputFormat") - conf = {"cassandra.output.thrift.address":host, - "cassandra.output.thrift.port":"9160", - "cassandra.output.keyspace":keyspace, - "cassandra.output.partitioner.class":"Murmur3Partitioner", - "cassandra.output.cql":"UPDATE " + keyspace + "." + cf + " SET fname = ?, lname = ?", - "mapreduce.output.basename":cf, - "mapreduce.outputformat.class":"org.apache.cassandra.hadoop.cql3.CqlOutputFormat", - "mapreduce.job.output.key.class":"java.util.Map", - "mapreduce.job.output.value.class":"java.util.List"} - key = {"user_id" : int(sys.argv[4])} + conf = {"cassandra.output.thrift.address": host, + "cassandra.output.thrift.port": "9160", + "cassandra.output.keyspace": keyspace, + "cassandra.output.partitioner.class": "Murmur3Partitioner", + "cassandra.output.cql": "UPDATE " + keyspace + "." + cf + " SET fname = ?, lname = ?", + "mapreduce.output.basename": cf, + "mapreduce.outputformat.class": "org.apache.cassandra.hadoop.cql3.CqlOutputFormat", + "mapreduce.job.output.key.class": "java.util.Map", + "mapreduce.job.output.value.class": "java.util.List"} + key = {"user_id": int(sys.argv[4])} sc.parallelize([(key, sys.argv[5:])]).saveAsNewAPIHadoopDataset( conf=conf, keyConverter="org.apache.spark.examples.pythonconverters.ToCassandraCQLKeyConverter", diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index befacee0dea56..3b16010f1cb97 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -51,7 +51,8 @@ Usage: hbase_inputformat Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/hbase_inputformat.py
    + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/hbase_inputformat.py
    Assumes you have some data in HBase already, running on , in
    """ exit(-1) @@ -61,12 +62,15 @@ sc = SparkContext(appName="HBaseInputFormat") conf = {"hbase.zookeeper.quorum": host, "hbase.mapreduce.inputtable": table} + keyConv = "org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter" + valueConv = "org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter" + hbase_rdd = sc.newAPIHadoopRDD( "org.apache.hadoop.hbase.mapreduce.TableInputFormat", "org.apache.hadoop.hbase.io.ImmutableBytesWritable", "org.apache.hadoop.hbase.client.Result", - keyConverter="org.apache.spark.examples.pythonconverters.ImmutableBytesWritableToStringConverter", - valueConverter="org.apache.spark.examples.pythonconverters.HBaseResultToStringConverter", + keyConverter=keyConv, + valueConverter=valueConv, conf=conf) output = hbase_rdd.collect() for (k, v) in output: diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index 49bbc5aebdb0b..abb425b1f886a 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -44,8 +44,10 @@ Usage: hbase_outputformat
    Run with example jar: - ./bin/spark-submit --driver-class-path /path/to/example/jar /path/to/examples/hbase_outputformat.py - Assumes you have created
    with column family in HBase running on already + ./bin/spark-submit --driver-class-path /path/to/example/jar \ + /path/to/examples/hbase_outputformat.py + Assumes you have created
    with column family in HBase + running on already """ exit(-1) @@ -55,13 +57,15 @@ conf = {"hbase.zookeeper.quorum": host, "hbase.mapred.outputtable": table, - "mapreduce.outputformat.class" : "org.apache.hadoop.hbase.mapreduce.TableOutputFormat", - "mapreduce.job.output.key.class" : "org.apache.hadoop.hbase.io.ImmutableBytesWritable", - "mapreduce.job.output.value.class" : "org.apache.hadoop.io.Writable"} + "mapreduce.outputformat.class": "org.apache.hadoop.hbase.mapreduce.TableOutputFormat", + "mapreduce.job.output.key.class": "org.apache.hadoop.hbase.io.ImmutableBytesWritable", + "mapreduce.job.output.value.class": "org.apache.hadoop.io.Writable"} + keyConv = "org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter" + valueConv = "org.apache.spark.examples.pythonconverters.StringListToPutConverter" sc.parallelize([sys.argv[3:]]).map(lambda x: (x[0], x)).saveAsNewAPIHadoopDataset( conf=conf, - keyConverter="org.apache.spark.examples.pythonconverters.StringToImmutableBytesWritableConverter", - valueConverter="org.apache.spark.examples.pythonconverters.StringListToPutConverter") + keyConverter=keyConv, + valueConverter=valueConv) sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py index 6b16a56e44af7..4218eca822a99 100755 --- a/examples/src/main/python/mllib/correlations.py +++ b/examples/src/main/python/mllib/correlations.py @@ -28,7 +28,7 @@ if __name__ == "__main__": - if len(sys.argv) not in [1,2]: + if len(sys.argv) not in [1, 2]: print >> sys.stderr, "Usage: correlations ()" exit(-1) sc = SparkContext(appName="PythonCorrelations") diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index 6e4a4a0cb6be0..61ea4e06ecf3a 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -21,7 +21,9 @@ This example requires NumPy (http://www.numpy.org/). """ -import numpy, os, sys +import numpy +import os +import sys from operator import add @@ -127,7 +129,7 @@ def usage(): (reindexedData, origToNewLabels) = reindexClassLabels(points) # Train a classifier. - categoricalFeaturesInfo={} # no categorical features + categoricalFeaturesInfo = {} # no categorical features model = DecisionTree.trainClassifier(reindexedData, numClasses=2, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py index b388d8d83fb86..1e8892741e714 100755 --- a/examples/src/main/python/mllib/random_rdd_generation.py +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -32,8 +32,8 @@ sc = SparkContext(appName="PythonRandomRDDGeneration") - numExamples = 10000 # number of examples to generate - fraction = 0.1 # fraction of data to sample + numExamples = 10000 # number of examples to generate + fraction = 0.1 # fraction of data to sample # Example: RandomRDDs.normalRDD normalRDD = RandomRDDs.normalRDD(sc, numExamples) @@ -45,7 +45,7 @@ print # Example: RandomRDDs.normalVectorRDD - normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows = numExamples, numCols = 2) + normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows=numExamples, numCols=2) print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() print ' First 5 samples:' for sample in normalVectorRDD.take(5): diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py index ec64a5978c672..92af3af5ebd1e 100755 --- a/examples/src/main/python/mllib/sampled_rdds.py +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -36,7 +36,7 @@ sc = SparkContext(appName="PythonSampledRDDs") - fraction = 0.1 # fraction of data to sample + fraction = 0.1 # fraction of data to sample examples = MLUtils.loadLibSVMFile(sc, datapath) numExamples = examples.count() @@ -49,9 +49,9 @@ expectedSampleSize = int(numExamples * fraction) print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ % (fraction, expectedSampleSize) - sampledRDD = examples.sample(withReplacement = True, fraction = fraction) + sampledRDD = examples.sample(withReplacement=True, fraction=fraction) print ' RDD.sample(): sample has %d examples' % sampledRDD.count() - sampledArray = examples.takeSample(withReplacement = True, num = expectedSampleSize) + sampledArray = examples.takeSample(withReplacement=True, num=expectedSampleSize) print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) print @@ -66,7 +66,7 @@ fractions = {} for k in keyCountsA.keys(): fractions[k] = fraction - sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement = True, fractions = fractions) + sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement=True, fractions=fractions) keyCountsB = sampledByKeyRDD.countByKey() sizeB = sum(keyCountsB.values()) print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index fc37459dc74aa..ee9036adfa281 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -35,7 +35,7 @@ def f(_): y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n+1), slices).map(f).reduce(add) + count = sc.parallelize(xrange(1, n + 1), slices).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) sc.stop() From 1b9001f78d96faefff02b846b169c249d9e4d612 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 6 Sep 2014 00:33:00 -0700 Subject: [PATCH 0607/1492] [SPARK-3409][SQL] Avoid pulling in Exchange operator itself in Exchange's closures. This is a tiny teeny optimization to move the if check of sortBasedShuffledOn to outside the closures so the closures don't need to pull in the entire Exchange operator object. Author: Reynold Xin Closes #2282 from rxin/SPARK-3409 and squashes the following commits: 1de3f88 [Reynold Xin] [SPARK-3409][SQL] Avoid pulling in Exchange operator itself in Exchange's closures. --- .../apache/spark/sql/execution/Exchange.scala | 43 +++++++++---------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 4802e40595807..927f40063e47e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -36,25 +36,23 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una override def outputPartitioning = newPartitioning - def output = child.output + override def output = child.output /** We must copy rows when sort based shuffle is on */ protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] - def execute() = attachTree(this , "execute") { + override def execute() = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. - val rdd = child.execute().mapPartitions { iter => - if (sortBasedShuffleOn) { - @transient val hashExpressions = - newProjection(expressions, child.output) - + val rdd = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => + val hashExpressions = newProjection(expressions, child.output) iter.map(r => (hashExpressions(r), r.copy())) - } else { - @transient val hashExpressions = - newMutableProjection(expressions, child.output)() - + } + } else { + child.execute().mapPartitions { iter => + val hashExpressions = newMutableProjection(expressions, child.output)() val mutablePair = new MutablePair[Row, Row]() iter.map(r => mutablePair.update(hashExpressions(r), r)) } @@ -65,17 +63,18 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => - // TODO: RangePartitioner should take an Ordering. - implicit val ordering = new RowOrdering(sortingExpressions, child.output) - - val rdd = child.execute().mapPartitions { iter => - if (sortBasedShuffleOn) { - iter.map(row => (row.copy(), null)) - } else { + val rdd = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => iter.map(row => (row.copy(), null))} + } else { + child.execute().mapPartitions { iter => val mutablePair = new MutablePair[Row, Null](null, null) iter.map(row => mutablePair.update(row, null)) } } + + // TODO: RangePartitioner should take an Ordering. + implicit val ordering = new RowOrdering(sortingExpressions, child.output) + val part = new RangePartitioner(numPartitions, rdd, ascending = true) val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) @@ -83,10 +82,10 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._1) case SinglePartition => - val rdd = child.execute().mapPartitions { iter => - if (sortBasedShuffleOn) { - iter.map(r => (null, r.copy())) - } else { + val rdd = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => iter.map(r => (null, r.copy())) } + } else { + child.execute().mapPartitions { iter => val mutablePair = new MutablePair[Null, Row]() iter.map(r => mutablePair.update(null, r)) } From 0c681dd6b24431eb35770884e50f22ebaeaade33 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 6 Sep 2014 14:39:29 -0700 Subject: [PATCH 0608/1492] [EC2] don't duplicate default values This PR makes two minor changes to the `spark-ec2` script: 1. The script's input parameter default values are duplicated into the help text. This is unnecessary. This PR replaces the duplicated info with the appropriate `optparse` placeholder. 2. The default Spark version currently needs to be updated by hand during each release, which is known to be a faulty process. This PR places that default value in an easy-to-spot place. Author: Nicholas Chammas Closes #2290 from nchammas/spark-ec2-default-version and squashes the following commits: 0c6d3bb [Nicholas Chammas] don't duplicate default values --- ec2/spark_ec2.py | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 8ec88d95e34af..bfd07593b92ed 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -38,6 +38,8 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType from boto import ec2 +DEFAULT_SPARK_VERSION = "1.0.0" + # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" @@ -57,10 +59,10 @@ def parse_args(): help="Show this help message and exit") parser.add_option( "-s", "--slaves", type="int", default=1, - help="Number of slaves to launch (default: 1)") + help="Number of slaves to launch (default: %default)") parser.add_option( "-w", "--wait", type="int", default=120, - help="Seconds to wait for nodes to start (default: 120)") + help="Seconds to wait for nodes to start (default: %default)") parser.add_option( "-k", "--key-pair", help="Key pair to use on instances") @@ -69,7 +71,7 @@ def parse_args(): help="SSH private key file to use for logging into instances") parser.add_option( "-t", "--instance-type", default="m1.large", - help="Type of instance to launch (default: m1.large). " + + help="Type of instance to launch (default: %default). " + "WARNING: must be 64-bit; small instances won't work") parser.add_option( "-m", "--master-instance-type", default="", @@ -84,15 +86,15 @@ def parse_args(): "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") parser.add_option( - "-v", "--spark-version", default="1.0.0", - help="Version of Spark to use: 'X.Y.Z' or a specific git hash") + "-v", "--spark-version", default=DEFAULT_SPARK_VERSION, + help="Version of Spark to use: 'X.Y.Z' or a specific git hash (default: %default)") parser.add_option( "--spark-git-repo", default="https://github.com/apache/spark", help="Github repo from which to checkout supplied commit hash") parser.add_option( "--hadoop-major-version", default="1", - help="Major version of Hadoop (default: 1)") + help="Major version of Hadoop (default: %default)") parser.add_option( "-D", metavar="[ADDRESS:]PORT", dest="proxy_port", help="Use SSH dynamic port forwarding to create a SOCKS proxy at " + @@ -116,21 +118,21 @@ def parse_args(): "Only support up to 8 EBS volumes.") parser.add_option( "--swap", metavar="SWAP", type="int", default=1024, - help="Swap space to set up per node, in MB (default: 1024)") + help="Swap space to set up per node, in MB (default: %default)") parser.add_option( "--spot-price", metavar="PRICE", type="float", help="If specified, launch slaves as spot instances with the given " + "maximum price (in dollars)") parser.add_option( "--ganglia", action="store_true", default=True, - help="Setup Ganglia monitoring on cluster (default: on). NOTE: " + + help="Setup Ganglia monitoring on cluster (default: %default). NOTE: " + "the Ganglia page will be publicly accessible") parser.add_option( "--no-ganglia", action="store_false", dest="ganglia", help="Disable Ganglia monitoring for the cluster") parser.add_option( "-u", "--user", default="root", - help="The SSH user you want to connect as (default: root)") + help="The SSH user you want to connect as (default: %default)") parser.add_option( "--delete-groups", action="store_true", default=False, help="When destroying a cluster, delete the security groups that were created.") @@ -139,7 +141,7 @@ def parse_args(): help="Launch fresh slaves, but use an existing stopped master if possible") parser.add_option( "--worker-instances", type="int", default=1, - help="Number of instances per worker: variable SPARK_WORKER_INSTANCES (default: 1)") + help="Number of instances per worker: variable SPARK_WORKER_INSTANCES (default: %default)") parser.add_option( "--master-opts", type="string", default="", help="Extra options to give to master through SPARK_MASTER_OPTS variable " + @@ -152,7 +154,7 @@ def parse_args(): help="Use this prefix for the security group rather than the cluster name.") parser.add_option( "--authorized-address", type="string", default="0.0.0.0/0", - help="Address to authorize on created security groups (default: 0.0.0.0/0)") + help="Address to authorize on created security groups (default: %default)") parser.add_option( "--additional-security-group", type="string", default="", help="Additional security group to place the machines in") From baff7e936101635d9bd4245e45335878bafb75e0 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Sat, 6 Sep 2014 14:46:43 -0700 Subject: [PATCH 0609/1492] [SPARK-2419][Streaming][Docs] More updates to the streaming programming guide - Improvements to the kinesis integration guide from @cfregly - More information about unified input dstreams in main guide Author: Tathagata Das Author: Chris Fregly Closes #2307 from tdas/streaming-doc-fix1 and squashes the following commits: ec40b5d [Tathagata Das] Updated figure with kinesis fdb9c5e [Tathagata Das] Fixed style issues with kinesis guide 036d219 [Chris Fregly] updated kinesis docs and added an arch diagram 24f622a [Tathagata Das] More modifications. --- docs/img/streaming-arch.png | Bin 78856 -> 78954 bytes docs/img/streaming-figures.pptx | Bin 887545 -> 887551 bytes docs/img/streaming-kinesis-arch.png | Bin 0 -> 115277 bytes docs/streaming-kinesis-integration.md | 94 ++++++++++++++++++-------- docs/streaming-programming-guide.md | 64 ++++++++++++++---- 5 files changed, 117 insertions(+), 41 deletions(-) create mode 100644 docs/img/streaming-kinesis-arch.png diff --git a/docs/img/streaming-arch.png b/docs/img/streaming-arch.png index bc57b460fdf8bb07eb290082a0655c1052834687..ac35f1d34cf3d0c94d039b16f2ab8aa0cedefabd 100644 GIT binary patch delta 52633 zcmX6^Ra9Kt62#rzHMqOGI|O$N!8N!Y+}(n^I|K_J++7C<8r)q1JaXTB%*ULWwPx+^ zuCA)?9dQVCdXeiqDsw`TJBE-35i4iMkBV)b{+RYBA|9Mxm=mSnh>J55 z;fn1al>yT)xoF@dv?ZcaZ9z_wIP5uGYS@*M+UQ88nGs9f?hi-B|E%gnDp=_sZ}^is z$d?!4(`Uy7CR4x>n*zU260&)4BR7nb3zfRxqVMFuDFJzN-pxy0Z#Yf`Wq8W8t^GF# z?Pv|5md34NRjEyw#VTshH$GA#*hJY4Q}vk!2Gf^{$i2zpQNe1OTM0#(PQrmv%Kn=h zSgQWhU%`k|iin3l*n}?iR-PaixY0n^=@$7$y{J7zsDnU?>^Rb@z6RlKzz8%tR%;KK zBa=dFlhScB^0uh5^w1qRB`Z3&E1daWVq}u2%j`R1O|$L8=k=yPG+b#deF=^H$bz_Z zr={wXIP}mR%cdx0T{Ip1?UR$t*@z#<4$cC_ELF%q$pgkxQZ{QWI~5PAl2$hVcp$xn zMvm6H3%Ev{+PF5WtK_^d{um$Hh`!^}Kn-%W9v@x?4f0I*`>j`rk9@ve^g{VoH2Vip zj}Ssf8fISUw6Mz}`9`&WugtW|+9G0)v4}^ngKXh51NuaKVGcbo9(vMXzE}_f_7oCRE&hzo` z8lg-7z}|0e4##n|k?J>dl93b_eXO8kkk~6xRqegm#!OsL&ZT(JR^DZUS_xf7&PgQM zY?Hww`|Cv0ov$L5kCW6W2sbggG@VP|Dz;2j4}|4Op#+wd7!M&3BGjE}At=A<2=Jeh z0{JIq5GyHtzk*>^hLwFuIcJ^BW&Phc4Wn$GUcnMLoVvu0;8zeTrGJDrw=T6Ji`MC0qo$Snl_RM`9@)%;?yPAS ztUt?Q_Rod-BMDZ=g_27Ir(eicn-YUUfejEDJZC~<1w=51o3&QvbGSgwCB)XTE(!-k zjA#&G|f$r9K@Fg9Z7%2+zh?;D+^ z;NsLAQQD2ei!j|QjA2zKr%k2v?u+dVQ(>+E5+iR@Nbt`=GO2~-FiL#!wG$VQ+1L$(Ej+;DHLfa_B&K#Zs1PL zPv!^|Unq@_`;;KrF4F2I5eU+Y^vKlI8iQAwa2_&KnD_k_l&gV?MVN;d9}j{=#3@+5 zz&FxJxzlvM@?Dr`{0zURY&yaX)WKSbNE7?1kz^)mob(T=Cz9Fo{|Y|ElHed^AmBU< z+iAA_&eKAjz=Ind;LhRiDUM~jmc+{ z;G-)_b%J5OJM_w(g5k>vlZhJ1TsCECTdqFT>T7qejTG{B;}$bV2kS0=Bv} zvKMjjCXjZM{L)dq9+%*zpQy3#M|dx2`*ANMAK4w*KX=&o4bJkCIF z9-jgs5GVuF`011O(0>(m}Rc`LBb&AZw&9qtIckwUt4Vket+FoYUWbxkrfMx1L;xhJxiB( z>PW03cN?beYIs6`>cxVUr65XMwuH+|e%4aC$jHC^AyD`omGeAiP$pCa{XS!Zs^9Lo z7I46z1(=+B`CK?WWn(uvG<>ltw2vNF{jHux{SM$JxJ#iWlXGAz4U={ijM`xRP2-HU zeb5ZY!O;Dqi%!G-0W#5KU@FBtS8Ft=a}oJkq;j3}W@0KY07ow~B| zB{;$T>(6m&oUo9mZ>|{ezj_O4U?{BVR!tlc3_7hn)qa^Ja6_somU#vPW%SiG zu*~wL6}s0B3?lE;HBBkMQ8~Yr>$0u*cI3w_iX~BCISq;kiFs^8oEn0$CoCxOf8GF0ya4hFtkxO#pjc_M zq+@BDe8aaOVDp?|E5$+h(&ZH?xb~Rqx<~So_+@`P(Ms1PrZmqIgqtN#wCf5v<~HB!DGjL)j*Bdv3l1x z8hE*K6y6EH7N1tSn@JZ8d+_ddcRU9Sx5M^dUwfN#|Niv&{E4$<=GU4bNppWA*R^Vt zESbq2*!~6vc_orUEi*uWEYh2{gYFqY9NN>Lt`2p=_G7b&-qwMUj)f&oW2G~onkJ(1 z%0r{e`jrk;xn)ZHyE0W8Wl0VJifJaxyPSW#9JJ;m>?k=WSpH-qo$m!V<1Ke7+vm{f zlBSN)2QA=n{t4n0g(fyW2J9muO`1@jb5(7sc*4%Uevsm&c%XOv=eK1un(sa)bk;gE zYJv`vnOsc{5NCgflDJtBs+s|<%m>_4^hx6zpVrmA7ZR$N|Ga>ha3zza>uU^}HQY|k8+2Lzi z;l#nDgqqH-jtW3|r|TW+NE})oxBShOq`Hzt*~zJA(21*wRzUdn?HzuHqH*O1E*hmw z+(d0{O*ek67>&8tdmCbQ@+-FvtEyxkSU*tM{vK966P9m(n%qKM=%smdbVOmIm*LiT z?4a{3OG!Jz@radRzT!6tOJtyMzuSGgz~lv8D0%CpYz;Kc$Eep!tc!YPKgoA>HKN?t zN-n~B+L=a?YL%kYs3OYE2Z#Ssr^h@~Pew&jFo=DsZD%4`a((@jAeXM|Zo(~3&JO>l zOWV?VIW_G&5L%4K*Hx#&>DH@%i>o1qI3>e`m3w%X9sjRdZ~~zbaVB6YT*-4`Y52KT zTl{thfJjW-sVv;L%d<~6iT?=|?r<2Mo5@6@rKk5UFE0K9!52bX`$|C&!_vSpbBZtw~jvvBhvwNn4xN!he9Cba+tew*)oq2ol(=DcR{P{1l8J-$? zrdJ3H&L|B6{-#R})6rQ*JgEWa36mA0{Vf`I+_6f!C1+@vTZA2F+b6-tsrM`>cXKn(( z-q;*+o;NlUnM!|<4vf=-pi<;rE2J&a>b|2y`+aT-xQc+S?HPMB8soY@uU39x)*l9) z4!G8VTxujgJWxpJVAl^!hn0*oO6%woc?AU#nc9CLIMDcL$g;#xmx2joDa^lqL<^ho zn%B8{^3KOKl(mMX#Tii`H&`rs09VD&_#|5WPNzXM6kdiu{&f^gw16ykZ2myP(Apk* z#NEXbZEQPRaW?xMjDvzD>KF^k|5GY^I1?NFjSzTD=0O7ZaqKnz#l=OTsK>e#P84eR z!0YSl;pPKcq%>21kcn!8aviICCn7W*uk0!~H|fYv>%qYtgH1C<5Sfjg07OUwwp zN+ zHNDq;8ElVN``qb6QpB$O4{JHhH;>}{P`hx-&r z)%=F#x%n@6|3Hot7QF38Pw2@(rNx%Q9*Y=fPC=D{11~vw*L(P2ajq z&|o-Ve9c|zjSsR8YdWW=CrF4_!1N}CakJ;_R98k)!M>@8ANho)44^J9yc%0rA61bh zw~+Z1MT90LF|D;;tll7$dTqoJcXij$c$#Sw+^?fw7ymYF-=o02qI6&r4K%blEJX9m z_B&{4>d*u+#;~9$Bi+W#cJ!A8rS{NG&|!)#uacc8kN9$5xT%rDtUwO{IVT z9xnCc%s>RP?{UQ?wd4q}A!}YA9Eej$#C<9DhN*=SMobUC=%Cg%!>6Q3ehX&6q@Z6U z;GOvBxfPM3WN2-O(-FwEOF~GW=^9O1PYj}^hbPx(ECypV-}~A(8lEVImoNs94-T){wjTm-zSEq=PVTUY^pD+(bmD-Jv+U7))g|RQL`o zgAKwOR#6j=ziCTn#Sdlb>D*oZ-u!#L6qnomD}#c9HbqZaYqEOXW@1FnvEB)v{uofy zzpYR^(QyW%!Z<`cSENBe(q*WN)z`d`OA9)Fw-`|x zHd6IX2eH50A3a%43J{%hZT=PfqQcX38_t?9GBxcpVE7HPA_clNJ;8$gK=$~}h;03q=3Q_{u&}McMC6M}u~Rwu(`xi;y=(ToqQ%vU0>_q^KCX1zO`W_>moysVJT z3jW6C@l)fP8xY|2`-xQj*Y(e5EDcvtQ}G$j4ZzIClMl~-Gae7tU>yFVq|wB{1TtLv zncHLvHeJ37C29Z&e!Ps|$@1_y%FEx%MxDu%?OQNU6 zm^&qHIe*>+<{w48c22=y&d$BoE!-LwE6S=UKsN=SuT#aq$Fzzm5%j#WkWz{y%R7A)rpXE2BQi(XFpCio(KwX|9n z59H;{iupM8!6On1EcW*x2(QsX&5)-gQ!BMY3AFk~9mH`smHiSq+rwD&veo#N%|`-3 ze%t$Aq`P0EQ~V;#nCT0(^lpt+0Y#k;`pJ`he0HtH zTsNM;3Zs+m`$A~mN~L9K({^1&3VKB#Y;}nlP5FlH zq>d`WT3_#;hAeakLvs#ij}41Sd{2so2ru}I>ydE9cMJa=OCs}Ud+AG(!5bB4Hu5Gm zJ}c7e-&+$W7x85$>VGcRcerf7`_=f}90^iZil9egOd91=Tsf;i(YxGCHQJU|J-T`;f*vs$4sX4x1?o>-g%Yayplv~@MVg+m)$yUtfax{`)$2gw zxjpS+Dt3q(Sa7-ybH_1lOBO;zRG2i0NKtq9`;GD*#b$703exsN3Qy(6COim9`an(Z zaja{eOnV?Sw7q61K|cLC?JME6cA)6N8!P*@|A1?&E1L2qcq{R9NVW3qvIEi)Qds%CU-MU807Dt0#+r^Cv;8!qQ(-%}_Oi=doR zmDRhAJ(@&jPGS5`>@m9Z!16B#vdOa!=mR^7CW9YjbpB;!Pkb{;NWE>~0ljlbM)7&Z&+H|{A%DhcRuxd`N{ zrbi#FK?Kkg!SSt6BN-%}=rcqOd~-g*wOu~HnmlluV}Zo=InHZNs;s-UqrX@ABmE#jU5V zN4u1&+*=4;T(5C^yH1y)Rg9Z{`Ni$Vi0aFjOs(xARP&#;;sxZi_Aq@O4z_tqS}p^S z3BC&jB@2N5kkW%~?;#Za+gQPIJ6Nt*2l+*h-N{44dz7NK%K;&tKO{=q zh=ZSD8w!TVT=Ow6#R~4in*wSGX*`ZFbdVru6lp^t0{z7dPl8@#sP3R_xQ z2Hgko`5LwBYxTx2x$}in%L&pOzi9)kcu&XnEAEM`O#yiTGU#hi;*;56)T8>`-{S`X zOFS)XoaL5>7yrSDE<+(mmugH0BUHhIg}p;obotEuim>Kcnw?=M^%qR&JqC^L8o!z~ zkSnXZgsFOWo{$cnkxAlXvOR;OZQ(jf>^kbE_v5F3CI|!6*6iP`s3?Tven6_UVB72O z%ZJTsw(3P1 zs)b(;cQpZlX}TT)-pQAs6_&$GLk_ov9mG=hj3!bnloBim+}uv9$q!>ar|!h89mMJ? z-F}9R%~NSjB>ec#0<#4!i7HiXaMGXeZ=0;=Gkc7pKhwgZN(RW5IC`A>5Ze-Lz((B> zW}>;7Fvnw!Swt5!^3TljMQ6eEfg30L^1?>9ka7%`D(CPwuee{Ort%rO6}W{)3a5oc z#=D$SU|H8ORr|Mn+J6x~^)nxr_hqLPFCC0Pk~dSK z;L>l|@7kgIVaD!qlx6;y%s?mVYo(?6o9gZdKW1I`OyAW^^!lZ1X`}$uhLis~SWEyR zxF_d@m`Gs6K}0_4_l}QYLSz$ie6Vh*t-^WSjYyQyXtl{?kOo~@*?^;9 zrJ+47yW{W9N!ACg@1y-!T-Wag)JphdCgm`Sa+bXzSb-hDexXGSR*B#mXWsN+&mf9at)%|2L7qZj}`OG(ze7JhKhuMijdpTz@-{=qm!cD$}x z?f@j&lBy3$?@^pNWcq)#MF~O2wvwV{g=H}PgAxoekR@EVRdR>O1PTNW0gfJXE!ziH zG5l>jO2r0N$#GFh@F4qJuB6Nr=A!I?W0s(oOK#@ScA6NNE&W-uW!wF#eYV2?VPkQvk#kOqeCI0BO&FF{t= zvj4O@819P6CHpP%!riXbS&wC@$^mNd--llxc~(OhxImV|c?6P^>h`%RU*!}@3`Mu% zfm0OrJJ*{SkDy>?m3MvR1|bvGlom84Bf=b?=NL5BFDXZ#@6FkOSwV+k;X3wdFgL)t z&i?k?U*M2+Qj%Bk5CnW#i!_K}TKc}LLa2!oPCt}a9!mrV6eHqzoPSlgGEX37lOv_z z*(y=j((UMgg5kZNKez0@4~QJs{)de?ijX4@4*~8@Gtn$ zRKPb_r%uu{_(pkH@;uK%gZ;P;g)%iismhT8wI;x!Q!~y z?WTnN38Sq!4TzBd%m^vUoIuNKuj(T+`D-Q_3L%X+ z>TR%GgS`zcN=HYA)<`K`bzHh8*An7f<2zCv1kL1)tor7q_98lB(I%^_)vv4E4E@*w z9566ZFgZyv4c>n)Do6+-G&B@-$Ey(pJ>Yj~6-RK;HudfQiy#tFqU*{FU zKw!8Jpl}+MP|o#X((-&0)7ds9EgC zfDHUMZ$Kf=6e*pn_7T^4hP=E#4z7Nq^o(zGU#{I?=JKOC4x?6D8;sI_x<5>~0<9M?wwAB*hU4@}mA`yL3tB^Uvj+OLNPYervZFPir3; zgfk&nZRLH$upWq9$>ge3$rEN1%3om5!a^DGZ)0}vf#f!c!b{|?d<|pX+x9K0P z!hSD}t`@lh7hl`S>xZqYF6u_||7c5-9~2I=fh^SE zVr|-OO-04ej{&h7nbS_>&kV{itT)$qX43HGu??~ye;d5#2Z*`9;7f%Cv3LYj8<1W{ zPRe6YY+EYQ+l-xZ@-YBWO?yEQiPOt3kVpzqhrDM!nkM_F1d>O8_0wog_H9+$44wc* zf;c)yw$0UESe%h}&GVHT;}mQzWJD&T=n9wa5DdXXrwra^1?4FxD6=KZ@hPY|Q}U(4 z#@Eu$Ul68_lDg={J^C6cC9BV-iBNS>hjH>xlkILD3&hx6E+jz0*AQ8E3@<%)S|wrHPq3bbI)iuF5~0|Fnh#9HUPC7E`)_ zHW&W%fw59L>o(HID$p4U_%oTrm)*%3+x%fsUA#c8Krjz0y+L~u%6}y*L>7=^kXD#e zS79$DbLHX+V8v_AgS>d{2e$qudt6NXCC1V9pP7L?Fy}M|Yy5GElnS>~wC2nGD`U|`l4AH|(kvue8m^^|M#7^XXnuzCpQAYY zNDIU(cydk;EVN1kiq@@UrAGau3ii~u!nC39Fczjzde9Jxw9Qd#z0x23LDWYKEO=^s zR?E%AMkj4h;Ag-zo)+(=z{l8a({gP`t@?!nM|~tLXln!|84WKc88_A;0+ZeCwbQ7} z+k)nqC)h2aF&NyBy)DL@f?lv*`qj{wHqMxOqVc z(Fl^Q_fk|sPTI|J=6@Sif%mb!*?iwxw)vccqac0*Pal#qN*eQBLK1cVBhqkgp++q|S;Ws7Ydrau#*)a?A z!SyOw6AOx!-xQBEP*G8lp@7O_7(cvny>y_X6#6ScrayUmluSVO(H@$dTED0M*R?Qr zspZ<$vQLeoJm?&5T$=trz8%Pfx0Uh0eR|dP!B_GHfSGT*oV;9$vD}4?{tK>!BIIoW|s)yR3jO z4Sq4WgeLi#DY-5w$pZxajAYEL(Vxv#N`&x*Sg~%v6r3Beoutw3r(4Ub*XwdWJ4mA@ z9je?(+_*EtmY=7T5+yY~0@b9G99?+fRG}$bejI&@G}!!J9%RLBH%PWK+5u0%{nUhu z3A#^2oX{ckO)xA@`gw5Xq+nXKC&(cnF0k=mGBbzWLsmu}=gA1+w2G(a3dNhCFy=T%RxW2O9skq;LN+L+eFAynB zLtaqatwoX$5MW_p*~bFAWS5xCtn8?-yeZL6nzb+2?x*FkgNi>u;+w#iF9wHs8Por^LVP5_rqS_{U>fj);pooF zEDzP?1+jx(vu5Bjwca3H`BH=#Nf~uxqO61zLA+)Hx6=R4^&DNcEqVPs(G2Wkm5@_b z0GPATXkeI9lg`(qV6|PUo@sO5We#KOA*9^QfVtibWb=a^{2(E;48()*bbUR26J%5A?p7`U`Q z1;_2!M>#g#uRGB-+efZ6`}?JfUo1T;QeDvO1WBp{a)L%}CDSdGd6-4}Xe&zF&u>NLmt^^h} zH9=D!!^%r$yUVw+dIpA$D9)mhrW(vp`(A(DkZV*4A^y^2Km6(uut?1kUGh~Hw1rUY zBDIhnsxw1;catoej@nUhWD#^OK6*P-4}tNNKWdTil6I!IgMM<+rk{(UOc<2}{z?fZ z?PKex8Fl}#+UoVW3*g0hpkAK7i3E-Z&Q2mamyz+4-bX7nOWv>OlmG2m<-by}dYrFg z?+(WUP}2)pSXjzp@1_);IR|vsXI!2L1aEY=+B3KXOiX3@K`6+nCVnEZLllfLq8&0H zW##@(<1_=7&$8a+KPqgciD5&0K_A@R*>fs?6Gh#RqDL57x+c|$*Gazva7y>0g1f?O zij`e@xngOehO%9L&I~p*GP>#DUYxS8DSvC!Kj1$XMBt^b`qxugl(Lu@eP%wm($5jp__K970M zA{0X2(H{?+tHjP^@H4(k12$34YLmZue8EYw(0_pJSglQyZ?Q+u6!s0egRq|e)0v4)YGXJaQ0_eUKHaq4!88TY@ zz}S;GY`PROj%AtD&t3VecDFoSY-C@Z#S6r@XP6sZc>cwzxJ1&*mg!xB|MOKKaXuGl%-%nv9^*RprmS3cMyC#9uhEI-rNM3Kmu$Fug^UgWSWG#3Su{ zU~D8b$uV}_4`sR3;OCn=2%E5m9||~YhI(3KkdQIzd})9@@4v7VDpm0E=`hU8;?`pW zU{?GoA>E?ygzZr6SOjmNSo1=0_c&*ZH-?>t(r=v0apnZaFXVC1Gj_D_Kf7v!yOCd+ zZV5lIC4m(}$9Ug+o)d_hBKJE&K%mj>OgG@OPD;fF3UE zk29Slj(}0>t92|rOX3OS&ICK+bD=xbPN&Ydqwl?#GfRJhKm{6ZJm71o}xYhob9+!)x5w3+$LkxwhP%@(6;(FzKi z>SH&J6iF@S)7>8J9-~8?fxUx|SqjnF5$JHqWG{E7pt zDM|tA4plpuK8SFBuAYrFa^ZPfc<_^sDoiZVSmOMzFe4&s9gQO=148{ne!A{KKU>aU zb%+_^uNfo}9Nm#!cb#rW}qPxCKmjm4-04N%`P?>Xj<7T!3i%wxz5Ef8;v8v!jJgkq>f z7<8CkB&#lnzLBJK{Z z!k1qt-l$8rTr<4q2n5@L6996GA7%=8!VS>)6p|*V6SJ0#hI@l&P-;(~?Lr@Bau2J_ zhqXRDH*+ha-$I!rf(Yq%RE1en>H^F71P6mG72pKglK+N~Qus$YK3L=3Pf{1dI2WcH@BR^Bb zX=n*r*Aici9o&!ra3KPlBuCi`{OE_9P(KKMI2o#s4R2A%)4s9a#*{CtB$D%T3Eo>z zb6^@&tf9S^F)-n~GfHPPjC|5|BM_Rg6jgD{+v26%3J2qlnhyTtIh=a%jL6&;;7@oa z{UIl#?2{&St?IL69odljuu|$p;6T^8iyv?;Ew%7725ui1y938ZL-L@cte8ZSHK?

    6ng9;{A0xgwd zU$%{okBHs_|fgnRP=P z9pb+rRd8R9tmjLiQ=9HF45GsP4`)}5J&XU4pSjT*$$&OgpJ{VoC&MtrBnP1H?k)mq z!NJ01;e5hLg8YO1{5wa zBSUYhETa-;>~;z=d)LFJ^k2dWgSX`<%v=(3~|qhi|_w5hY;@&gNe zGx`OjVgMvpzEKjAKCMdMai*+hhFiUF=xNHaokx=Pwhfkv}$0JqOyc7e>9e)5> zGQj;w=+J$6lv8`1B{Mkr*6DL7$(Pd104MfSCy&$C0t0Ve0KO);J=9-ix($(bAp4>;4>Z-Tw)-tJbdW1?_O(-lk58iSG$v$xb-&>Mii_7(@EZu-WhsLx#+!9P88>vdg&~(!2#0nI9S6nuwHnT zci8NB^Rycme4Dle%3)1onFrhJ_XPfH&NrCWVa>g9DDWp?>+1&3WPSg+_o4vQhkfrT zz1Jcu?q~AY?f2q;CU46!dqjdve42PKP#k+L`yhWDGC-40nUX9lW5%$7v9sIAM$*?uZ=2;d(9ns4`y@K&}aLA_37aU$~eS`%6knd?ta+%uVyj%w^FZ zi!D$U8Y-XQF(DsCGW0ek2M298VQaaJR?Pks8?1;l^?1rc4)p26#bUa&jscd97M@;6J~U7{W7crgx2rrX3R`=gS8=@()10 zvU)MCZHh-P^B3~En+TKc@<2lsAMF@M&oug8-gd*?>d@mo3Mjh?y7tO6ghfXxwl5Xs ze_3&d8tWR-ytj>bM|l258>{w?(oBFAPD}ndh6qe!ILJO?6P#-LQ+uxPP8B;rB5651 z9M3X}wlk4EBSbgxpCO!~CscTd7uA!vu*}GGy5PC~Yr4alnY99#1#}82k}zdwj2$8c z=eHbQuk85&W}pg{M?MeIFCHRlCODLYw^*{bI*^*4K2U18I@o`!omba=J;+5Q-2V@K zULj$=dV%Y2rdx|$dk>0Xd~gvc@4I6+)TOp3Z2lO?9A7d_l1yIb9b@YnDjKH6_>wH{ zk1KOJo~@;8={CjhNP)=aZNlIdD$FDp?_AV;`(}@74j+clcDAb9?1s8(t3avEQ@=(sdqaXTqz1zN z8Y=~*Aqi~d^MFY`apS2AAJ&kDd?e6pcK?Aj~Z%GpR7<75}4@)4x zOwJ&^wpzh)(UgX~Pq5u7EuU3|BQyN<9gM=L`(-|H@&xrkuZx)>6w(;k-u@nlh*Y$o zO`p>Dvv!U^aOqcu+0L?F2Wdv4BVfvjU69B`Bmnu&;IYxUVYT6vNwFh9et)XubHwS$ zH@F$zu-wfH@eT=MFU-L49Jo0B6jtag{njepY1&wc5f}o*#PB88L%eeOWVC`KGcNfL ztQHWXix0}mw-j6SU)+sA4Ve>iLX7>IasDF$^IS%Pn4JCc6HSo$w=>nK z9l%5#?ec7$vqZGJbdgNd@S!8$T!^t9&AQCHn1@I1wL!q50+oeENr*d4@i-JwrUIAp z^v`oTqX15!FAMu9+cqLpCrE~c4@Wv*4Q{5^p}n?>7*mh1l+ZoUr(6xX`E|lYN^Vr! zP3w@GF-!J(XOmEC2^phnh=`I~{UQ}HbVT^mF)C5gkq2ZRnp34`MP2(Q!0&Xx1EIIYx&KhcJd8Jxo%OCAq)Q0@>dYmP> zbx6&o>T)7i79phtxr&NBQft-A6;S*%_mF=fapcyF;0>N4ORd>XAubtDKU!atr$^Ses>_AGLi>S+X-YX^BW6#f*1F1Io-jB}a406sP1>>$Lh)HV#wyJ@^wJehSj_ zctLdk!)j}2Xqh`FzY0VqA21(lRP?EK5`a|_c zXbH$|7cF!T>8Utk^8Xfp&_AzjICJcp9#$TFR@Q2F*yX{#M~uoqFE)!cRF?oal+u!z z^jmO|30zTi)K~#|J@3vSD`9FlZpz{#PES|hD$av1@1LoOB>|Q@=6B_Mq^$izYpd&~ zoS~*m$}JvdK3Xi5p9_PsbCfbiHj1`RH(u~6Nw3j?LeEN1=dnN53~upB&^0VcT8w8l z4QVYxQav=H3J{B|ewG*7MlU^zJb&0eA$g*W6yEBv0MIamZDsA^h|gX4enWjIdK(BU z`^%)G`X^H64%m*J(FkuL!RMl_hkVTZDb$_{w^$FuN4lLV18XdMYk_Qf;0!8d*A~uE zeTV1^6hSE1`f?pO`ZXrfTuvR!0-ra6z)vE+4_i;Q2*GwOrttDUxMs*i$-Zv)x}X zpl~k7rMF{-S*D<|v9U{8uTZ9_^4MO0;|H(Sss;)@b}31t{Mb05lFrYwUXb=X6XQe& zzsWBKdTF0*+hZBo&s9w}ox*XvtN+~xK7$HWrM>0NJidi5U%XZP3!GJ}h@QDCfe-5i5K7+QC3DU71^l@zHr5HCSQ$ zdEH}rmp2#Bpel*a#>c|3^^d0A(K=Bryqb`NHL2vYh>?c)L)lkj?F8)_;CcaGG-_#S zO~I_i;pcZV3i*{Y5_ei$f&DR}=IFg(e6V{Gncl=Z*koE;oO5qq#?zQIoo00q3rjnf z7K9N%?gt6AZnEFMS~8$P{WATUv`evHyBNe<*?7105xNa+rb&c`s;??XQ#q0v0S{K$ zP>MIK)_6x`K4vKI4a%y>BdI_QSu({#X`l{8T@uquO%*SV%vW75Yd&_v;@~j3A9jQ! zD&hqEv{-Z}Hd0El)-b<3YN#%<>t$4S|HsrjM&}i6ZNrV#*tVLav2ELC+n}+z<1}`f zq_J(=Y;4b**_}(aYPMF;(YTutE8@C3XBB9v(*0an^Q8 zP09|r$-)&2KiLd1!&xM$EEq4ED-rEB`f?fO?1X63>sMnr7hD-DvrMDi-ckmXDPyyd zu`(n_p2Eae71PnuIs)R>f&FQTe_JinP_IBC1a+M{^ zW+@_&dGftk(Jl?uoC2IRR#3lr;3IUvOo3vi0&x9D1Mdgk6W)PWEP!C7Y|%S?#V!Zc zbzL9_@hUVyp{SBu-L({a7*1&mUQH#|4H5QY>VmK06<))Cj>zB@>6m9gxH^#PRCPV- zQJG&ZhEe@bywU{=YC)SRX+gAeiY+B;NhvArwRGkD5-^@@sRb3y{0Z=}5<5`_M31^W zM@n<@uSkte&$PfWNWm>oOe~hCqUf6c13k9>0}nyT+ZxBc&kM|6@w6K6EDP7F*x$xT zW@mj-!?Gx6_T!WIkG^ZHRA0Wl>Ok1yP8#Y^iWW#2!PBg`3TVK0sCBY`vUu{7IiEec zjO`XUnZ1d0&1n|QnR4|f3n{}NJBdoE``taTY`9=vp+MUQLo&%`_yx5e0~wOP_qnf`Y{@<&V_t zSYYqt4yLj@FXgykR~7aM9mkxyhuOPHf+X2EPZ&-`&v|IX;Y`gL8PK=nrcu3*;6L*a}1Bf2Y3IzHI=4+Wl4>D7Es5CV1) z+#fbMVvpjDKS`SUaE0Uva^HsqUYq0=bGHcvQU;&0^dtT z9(VJ_WTGIDMLQv!1FN1WZ`OhXoyXz!9yof&^lclfU1;VUtBPUWVn50=gjZfm_zGlp zVoM@JgT)>g8?bcP^^y6n3wz%PwW73)<~bk4jlv+W24L-(RN{Mc4-cGuRku#T_YcSxIx_9ocj#UIC(z zhRzT93ss*ee>4q3Qy1f+yg?sR#p`K%<(}QYl4Pz9SFGr(yHe;KD0~NgC|*cu4t{2^ z#*b|8fLYtFF-=A@EDcQGyiXU3iwa$g%_~`UWoG6b{ez#B*N_1nx=Xu5ZM(T^;e(s% z{uzBi5>X9*WF=Op84%AvHr*vZPX`9LB!_P{y@`0RJ+N+UimKKE620}a7@vY4p;&gFmr3^K@KT{$gYH^I?KhTq#9Xj;Y?6}m7 zz@G_~cO{XHI{CCE-C>U0JEoBKo;53q_f{-$%{0DUN*CIhLc0M~SnR?345N&|;zi6r z$~&R1GLb3r(cD89hVl3Ad>4Nfh{<=2YSVp>@pE}0KeRN zU|l-V(HR9m7I7AyeM3^}V7Q4|NGyODF^qp?5Pf}$(o1HeWQ@;2}eitSey3^Iv`6OAG zYoEzM6N@7DVJfwfQ?z}EqUS`sNLRp-(u~xHb2deq32jR=ORR^23|~K`C|U_V9WEV4 zeGqcE0>9sLgj|ZUbtt$odqQ|k#9NLmkw7}hx>LeW4WuTh&F0Ud2)t&WJXaAzofGyu z18CO})Xm!((LA^l27>sr1G3b{>n8yrSD|zNl4%0oS%<&f_lLTfv^S*9jrq1Nkp)A# zSm1d1Bzd-=qdG?qrfbelx=^n@XB6O?&(5)o!P|D6#YhUG7TnVKz%6bl1AKn4W&YwSgm0Fy~)3|?;!4Ys3!9oJlw?wHG2a|OHL!5#?p|t8&FbW=8KscjdrEn!N zG53G1HV5{%&_IRdS2>{b^zk6GQh7#&Zx#6uc_j5Nt`qgmFb?idQhrFpPY0D^$*|%2 zlQuFQWWrBF3dHg1M}b56h;WG`p=Jx^7=%bFE5p2D6xhd$C<;n%Kz8RZ$wV%w_%FIT zqRngtjswnfu-`cHN?~g5DcQlc2D7k55fYp%XV7I<>7738bo?1lcvo}Z<`Ovr*f|rRv&Sbyc z=R?9^;I z;dV-$sh8Z*w7$O_$9&P5(pYNdNfp@d)9N(X-jM>i@n4 zP-4rvV(-m;p&stNQOSyPMV(qWTI_YcV(#4*fp=Rz9~*R5v6}eguPo23%Ux8ntRH1;007*gJ=`V<<7>cMBYZ zvy%q5B*?l{*YV)A@Q2n>Ik1F>YY(7{z?>1PZ_k2GR^3z400L~(&|4S~&%+2{y{qLZ z2DONL$M7B;LS@18o16RC@ks|0Qs<&Q9bbALn$WaQu8JUc9i55wEr^=(>+#n(s(6YS ztv1;o_{ot?t0fjzmEwETnsNui9=70oaA2e#lXf!7v46wzLAh~1{^(2*_xv?JV;SF>yngZ;o5UU54FK5y{@xwNY4Q!yh5(? zi&m5Z0e~kVX`v{{18yyPYLaMD#Qbv|xQtnd%o7RTXI^b*!{DnQa-`2g*F)9O10JrT zp?b$q`UtO#v+^vYv@&si2)BOba_$IDGzkNlYguL+9yCieFnEgJ0kh<}EN(6=HkwEa zSK-cJA2`aib_#75x5P0EI1&^o669|dYp`-5?wah$i};c%g3ZL#hBu>=5svBh46pL~ z%UL&722lASGE_G-het|{ZN88m*+tWYvBbr2uwssQ5)F@la0FqkB5u-i=Ys?# zj<9ml>!@<_oiZq;Dxl3+3PjvUhmLTVQOJwMsdECT>;@uYqZ@4s6fvM-!6CA%cmS1B zzAe_1@4H37)7KlW6gghLFhz~HE?b4i@|6)}L2Y${U58{Po*~~xA?H^mDhs})sE@89 znGrv$aSzt$ZC6XiLfC5gyzj@jrZxwGo_)-}9F%=Gw#L8>j1d%3aP|oPQPX z^!{T`{u#}2!D6IYg{hPcXHZkcg=RK{-zYRUoph@K8U%sixZSybBnu22Sn+U%rMjI5 zwMdnGxM$aT0sah9gzX5i9-)wge zq(jj0TNtWhFyVR5cL!?uXyp*zl=X?-NA`Yi|D6cmun$;kd_6T~bhB(;p?bmMoS_kB z7Eqw%8XcW=_}@n8{|8J|2>dQVTAJQvX`ASKNo)q41oOBTF5J-mA>1 zy--I7SMb!p%?D3n*ENEN05h`3K%13Y>ghteHah zTT&5?JA3!I-1!r*h{-FHwak7arQyR_(`NttxHPHttnT@PJ{6L(#sRX9xDLjNF#(oN zCyODYa87GiIcTV107b^hn&p&aAlc_3Ps{<8djSzIFiChplv8+^JtYYBNB9S+j$)~Fm;0nm9{&D48lIS@=JQl!a6j)E(g6JF7vj~;=Wg4P0$Rj4D z+n7b2Lu=LQSGM@xYrd9L*ygNSsPG4bfeQ;Qo5jSBd0(wsc z>8~Dy4^l#~kF2yUQ{R7EzPSS zQT_k#12=|zyZYIw7a-6bYl$vQvj%paCpNFI&S<6LwvH+4@^Uq`?Z{@)9>A<-=E@z| znv3lI>uXA5klU4ReAQk!jvo`ay^m0`4Ewj?deKZd%sR5)Zn`!oK*NeB@3N2kZra{X z4$2{Bqg)UeE&|LuPn7o(?B(CKYDXK(w4RW@qeObPH{c&{fJw^0Ush zOxmv|#>NVt6#RDx@^>!4EBv9gWzU54?8wq_iu$1@D{C`L!P|W+HRs(@gcyo9wMH() zQXmAB4FKYwVELGDin5hw^zmKYPe#+x0;dyZuwP+RJ#$>PYvlf1Mhj+0gj3hnF+d)| za1cIyNPu3Yy$ulc)DcVava>Gu5-OPskF7J@Pd#CXnK%#|5Y-h$$ZOO8a|pcN#EGc# z7wfli4v;Yazv$@$dCW3wmY`58%mGcg7!cP6IkRDbI1}(mwu11bBR0c_es#&v_bT-l z45oKQSv#q32ar+_ZpiR@TSZrx`3dRiyq)q=0Zs&tQ)*lI;Rig%Do5x2QAHS(W0GQ2 zlS9b?_o^kT8$!|=TszzolM`J+KaAhEjbGEowtMq}#>P)Hm3%Q}ZHu+YHZVK0fdCwO z*rc+PCz-de{@Ol4#8WfU|F`Dmgnv&*cl0-K;~M4NIQuF6;n?rG^1R7gf(99#7Dx4> z04B;)>>=lRbdY&xUu@DZqrdk4eew$iA4OSqQDVktBw8Jhu}SPyL$eb)Qp$NHGA%^@ zKX9`1`e87)LQwn60LZ*Jd&)VmmLALS#XqHxyr(Yf9EEl0HR8FwVo-f`pH++E3fj0$puzoQdnlUXdnEroyACe+p=@YuFZBCr z&7QrM>X+onx<4Y=?=t~M*BuYr;UgMfKmY25t@8l2Y~WW?z3oQF;N?o#VpzuqdCq5f ziY*jqEJM}@1U1E{Uq@Q>UOSHcOLyx|J1D7r>a5__cz#I1+OH4lQ1#X=sBn0hEP>NO zg#w1xAz#oZ=EM%YoqWhD@^H+t%^Nr>TSRI>UhQyNsx5sCw7`&BX7F-!up3E-600xB z1XB|m8vr+{Rcw_iYsB^759YSKZPGSxEox&l*nCxWbSHhl?d|2!@PB?Qc2Xvm6Vn3u^nzT9sQMBun_Fe zRc3TV-C9sMPK(SD8>$fh_Kzo2DF9a_G^@jHwv91EpX2PHVbWkW?-UKObC&txY>&uG z@?Z;xe9{F)Nx(roBKX_1hCXr{|K99leykW0ffHm^sV;Vmw6ZGFl>KTAM7Oq5;QP~ z6{WNQtOB|>^Q$YjS^w2%9U-@nxk)ljIot1(_ABeF{)6ha92(J zhKZ%_S8!>oBOfE#vE$)(`BDw2Br_l2J)A4c-P3^K_w^(~?zv^MZ70aXyw_tT!u7Z`9@W3_#ZI*Je@(n2ae999D25^WyOXl%{Flmiq1{G%NJBGD856m7*NoFDkv2p>M*t zHPiW_I-w9@CnQ;=WrmSB6Vxbp!9i}8tZ3w*Pe^VsYfPSZr{$pZr+;N41qHbr6f#|V zE`LN~c_OhmeN5{_&VXq{(3tH_PGVg;&U9lNQStN>Jv!NNsbh+(uDtWKX4JvueEqeE zXAY5Yu+kB5aIoR6OUgI>Ne*580nP~)x#1H=dcHdbl`H_}u zqoF?kf&A>K2KHI#2b4u>)6tB2X+zWazg%z(0Fvpo{llx@uLO$IY_hjnqi%l?m6=EN zFYV@EYWNp8O1|Iyj-!k?3=Gd?tp)pP@*@{#Hq+Zn)U=mBM{R>kQ9fF_VbN%z=0OAWfPC)#Z^F}{^RIO-{Td94LosC46GwH=%3Hv#Lx|)-k z)Qm)cKR@)FXT#*KW-yT##JX!+we(4g{&)EkaU&`@SJ zr`@d9T#9DubJ%?DsM!i<2?RQE3ulQLIKK*l(B{mx$(4{tnYGra1`!|!Ik}PSf3zmZ zmF(Q0rr%P*=-)VrF#H(+g}kDFUm?m*3UB%P`xj^Q{2xaJoI+9C=7f53?3w5lFWC z%9X%D2Pi|vJ4QvzO9EL%!24S&k=w^`o9B4x@h~a43;El;IA!y^r3Ea^Tf=G(XKn2= z#-Sn4u+H1t6Y4FG_wQaw{|c5P>;r6mHdA`t7{cj4OON%W=%cHa12C(jL4_(Q*10AwsjRaQqEjFNYPheb4 zp%Cnl!WpD2OynLN%&^-k4%g&8@{$t(CyDnQl;Q+MNN+bXc$6%MOR5kox^p1gwtFEN@_fT&C*AK&#`Cu8aTsITibyO2L0UXy zAnM;B9>iV#5YpFQgOK`@p8=NQwD8=0xciWJ&)hZrm?Vd=Gfv=xHtpTB?1IroJT_wyH}9-TGN>F|0Q0_D4( zu|`TZ`v*jdfCxv=9ob#}if|1~3;=^~TV}_c6?I6C)yIEr$;ZOe+Csq$ZAwxE8P51T zso@2B^x#jmb8GGA+jWrz9uj%|Q8E+6w9vy5@6_`4l%)K2=RrbODYta-ab7S3!C_(l z|0JCWrmYU<@i$a+fYB_`CU1sH%lB#@vclSpfGo8oz zIL#(dVe+&~3oXhCecvX?NeZ zHY8AGl6#z~8rp8xE=v%FAE2=Yqhh`f*LZ>^@PZQ5o)J1k8Jvzvnk7zca|MLc7cnm{ zxorBpQ;sG0^bjc+JsYRbDH2a#GRt-92bf(}8mz?{-YmL&!QRJS-cR*md@;o5&yJS> zDV7f=(9*4=;KxTPy*U||a)ntxxJe6USYN-+UVl6ocBjEn5{82>U?u&gbA3b@()AT3 z!WaCL{fykgd%Lk`LqS7f>o?c0u3zu?-B#2quNL^%)GH0!$A?p4^QV;`tap`odJv~s zu%OE~A)D_OINLfd_6LFKoRN{5MR?j)K=YC~1etMBp5uqUsESQUxfg#vZ9p*Xr_s?; z!O+fzBuW=zmOL(+fZjmL#`uC}!n=<)lDbeqp#0La`huA77N?E<_zA)HHqkRU&p-7G z8n@o??~DHpBlyKh@KstFJfKEDOLZ0Lp8Y#5>I&HlU5_AH8~p3r09!vk!2f)z1%HfE zDtE))T4B@MAt8mt7T8Rfkla3XFs2k-fl=_nQ7C&q4GM6uwQm z@rwL1!mbN5Rw+4@<4pR!IKRWfg8yyWp3Cyey~Ga zg)ZFKC}sJ3ivXzLieb$HMUIMHOOU(ZmKFG=wZGnBmdUB0ZrK4;PIF zH3Cr)s(26u9j*$O%^~u1(L-$z+R0AN_v1-V+)UXgafb#brMg6A@kAOY;=ERVOSdHv zy`jQFpzIXwf|Qe{HAH!%xdtTJMoINAam!a%d!wDScx>i5v6NL%<6IwBwvG{TpBg_7 zpCr-iH{iHm2HgOQyQ4rHX)6{sq5)f;(wcKiz5;zzVWK)isKYc~D|-YCA%TIOT^%ON z8;P+tW?AFY@?-TOM-_iGv^V-&gG@GLzU0s4NPNWsk9#R2S25Efd&v@xwx{|YhCp&V zadDFXHcWh<*QHON32vGb_Q@aDTE>D^`Y>zlx;b9p=>UnFuPR1wb!uv39 zou=9?!92C&jakptr(N|@&GLulCcBGLKLY!M9Pzo)l}SWHap|S1{ldh;-swMeE%Nv> zc5f{~2qLD^V(lOA38q@)PdJfjab6^ZlJ6NRN>Yi(H!ngkRF!hlqgo`qTVG01xm*yx zAs{ywe;5Za>y@exN9RNl5DegVM_REympyi8^C(Ukh>P>ij%;?WvTSqSeRaQ$!(Y8} z%b#k(*lfe3(9|=r+1=X>q~SahsN=euZF6!9a!i2Rw)LAX!AR0~;km*!Z^^(`ZTe2b+J4 zud@CoKVgH_b6bp1%y9d5qnVtETm?f|hleE2MHCdLpAd}%+jX>Gjv{%HPoFGhzzI6D z0_;dj+OryvdDVqhm_4xb+VcxG?w0uYf+Gl!`$DAP2(XW~TIh0RWp5?F%Olc3tphCy z_3>d0v3|5jk`{b)&aLcXD3%-|aW7l~nc-6t(E>U{U86ZU0X*c#+F9Y#Pd(u7y7V!O zm1}iix=dyhhxO4`$}()R0AOBw7vg^atQ(8Me>T)pepQ@MvHvPW+{o#Vs3|&S2N!nt zqt=a!w`v8;&ruZ70VBby@gr`_vtXQYH+qA_a@*E+rbc-a7N@U|?h{^p)+(9G8@MP< z)1H)U@SBpJ+J(`%0PiZaRS6F-0`2oLr%f9yj$uHm%cXju@lDb+p4D79E7STrKv!9o zVzzK_uSFd|SzWOqMJO`qlA$q`;d!>ynr(AU*>pDmR!Vr-<05G(%gXhWO^_c)R}~p6 zdw@49&z-HZ)PC^gkzKlVVfjD^8oxrE*wtptm$@aKscY09(fJ!P1CG;Zaqb{r1cP$k zyXlA!BpyCKwTYH)LI><7RD4i5;BP7!DRaJt`b|)8saC_x)|3sJ+^~AAYi3Z?1Trp6 zh=kY5aVrLlZZH~!(1bgEw%a1{Tc8LWdQ`!%rkRtWga{}Yi3b&Y5&VXdstz?MB&OhP z9E1j09v|24*YnAYt z=y*-lAU{F`f6<4rDj_{wk**WBeTeA}S+HKc;5u8YYO;jtF2OMM$!jv2ysM&UVd%gU zgQIm{r6B>&w>eUbF+Hgk6a*amp`bNZgz5Clh~0na_-3i!cwEtmcN&OybaG5 zg&&x+S+#?yI-Fbf$Qjv66AHhXJU#fD0WLnrd}-qvQ4&s08~R}%k~-Uzx+uMu|4G?v z?vJB-(&sH@^iNSjhLSs+c6!zULFN6oN7K${)Z{U7uz|;-5|932n?o>k0)F*>=rn$= z79KT8I00T{s4iz;0Plxj}ob5I-6-!_>VUO%tT3S-wLO#^j z&xo?8hd2Kxcono5)NRz*x?31RQ0EvE+_)$&YH`l_I1+l9tLtUISC36(l*N_qRc2;VXh%~gge#3JJgwIIGbO)mELs2TN{`L(T z8WPhAoqX)!=zgA3UU)9a^xHIWAb7fyv+v#<>yBybQpBy~YkIH^qMT6NQ)gFRaabG4 zmJ8M8IdifCus~sII|2t8^3sF`YITM*vlkQ>#5q4!2*eB<2j&MFgE0{W?Crmd@mg1) zV4i9#RqagXbSvgoN>b%?U^z>~^m&o(_)*KKa^hEkWKb#K(7+?&+`*mW=Z7s4#IAN$SmK2FA zj+?F26M7{$Gh8$?7+?&^2FXOOM5zVY^v03lzK?s=I`wG--;%c&rC{RDvR6^7@q+4# zZ`Qj=W6N;Msa7Q3p3NQ>vKJ}R9!k=1ZVa(&K{1JTj6Vr8VdKqXU$LT8i|Aq--P)NfS==Z1nV2&JX<34mw}$b*j&W`Vv@Hf zy+7w}L){1QX$wRZ1&Q>Px_ppF;R=TaxsYU|dEK2`l83|>+uk)hQyFXXAzhE%`FPfF zCi~rmSdw4Mz3n5%c$Xs$x!1Sloo({JRUoxA!?G)AgyH z=%JW}>kQ@}cV;dAL4niLBu7dE4pqdxNrEaQt0Hc8xeHsq(a4=-#Xm(LOrF~J%Y{P1%}sjIjlytRz zXs(6Z%@_}m1!0#t6d3VNz8;#Y0EyBrm6PM@b?ReLgN|a_V5OOZXytNYg~*q?ltHpp z>kr|XEXMEB8My6<$!g;03PA34n0d`XlKUjbb3=~<9vMbpW6#pNw4c(_w$RV zM1giLZquJK$C-}qPe*&j+{9sgQ-5Fj-e)-<%bH#YZy z&uWr|#-CA@Nw%8Zr;9o@*8^E@^rSq)R#9erDEL17<7ChxJ~Xrt2L5_$)~T%@b7Uq~``Y}wquUHe5&ik%1A%n&>9_U~7gzA9bGCEXRi%Bpr8&|5m zY??WSCn)CuF2N!$d^OKk=#Ynv0}I0nl5d4N98&;2Da_W1kM{GYC!EF}1bkP{UX1dO z*5&Mht3)ybZ?Jo}S#1(vtoD-+gejkwWtFV_HsjhUM|Dpa3@S zVk+4q^P;I1bi%w!6A9Rb2aIObsV|Naa2jOdN+4m(Bo(AaF5BAqL1dcJEhLerBZ`KP z>s0|OwHhD$>oF9cFtj(>o|V3WF83QTBOT83TCYHd!-z*Gx#xT$iZ17R-=}&1K!2i{mbnurn)SVD6eZyx-$2pzN@c;MVRVFnwg4L zo{~epvtXq+f$hPz5>=?CoN8Lem*gOFMdF$Cr?3vfaGy7-O5NynKVOm!`SVr|yma%C z7~EU&g2UazMS-9o{x4DrA+wGdfq>6%PKbSNve8AyQfp7=VfNqYc4txD>{l*Mfj zlBHX>?>c&;FwhQH4P)o?rYB{-T#!i+y9n?eB!Xsyn@Qm>)LUM4OThGGj_{zi%9Bb9 zSfP9Yit8pPtJlICMC-O5(&tJY2eyhEG6RS_<1zXF4;3H}+ioi43!*mP2 zOgdMBvwhbQ^euk*1@Og7mYC6xnzhcqkEVn6e`x?CT1LSpJ@B8Yb%KyQ0d}xl(t|&b~VU- zJ^hCa)FB?!iKYRm{KXstMj$dmOSLLUWw*@XKO>&6kQ_E2fis=^R)=dUoXRFP(3b@t%hYs{_!>N7oOmCk^e+wq5tG|wpx0%1V z@!|+@Xj_hnr`sYDs<`ntFvg;rE5;?QPOz$~2FnLKvtoSaU%P@({a&7+nV_zrDB2`F zRT~&K>#x~mIJ*L1hGPl9Qw{{#+!k4INycY0wW6BNVqy}LZb&Zf-_m-$xA|6;<{+dW zWzUy}pl=Dnjd!}j&j%&i95&EtKbT$BeW$h2R?;Wz0Ja;^)w7OLUq0&I%6u%{B26YR zKuhfAp3bdyd9{t%eN4Fn9kmob29@Ebm4+s5-<`|Yc|WEsOVS|;I|8lg_@ zH`nV0CMoTj3IDNk6fl0=0R00FQuC#WvcdeSKOgSa$*-LwT!-|Syn~pgg@4;x9U_<) znj64LuVJff)tzT0ur8*c@`b<#8xl{Xz#TmSjD68&yRqA+%BH#4{(7Kx*fx4YWbCml zZokSAb{->toR%S9Gz9D8;H{I!D26;)pP><$eTwZDm8nhgtczyV()=%KIIH4~+E zGKy$u?Tz%EiRoS6P!mze{W$|VFl0PIGx7dMhuctpI@F)}%W66B;u0m{t2A7grf(b zH40N5DM?DR1yu=)IVz16-ltDLBLa*M1?wpW&83o8xTt7)vPDCXszm15eyrM(hRiyHz!O2&Q+ zPUmSAu(hyPQC4@XHHAVF`F`7dFoUQ=(K;^+Pl`IURN2cy+%soQ(;o@+$<-z4&LG3sg6b)@GPJwu5XNj zO+d|1@%}gcxk&KV1Q7S~AzZR+mnQ%<}|-;h1CwK4+7thy2Y#3gf0Q2*h*rPgQ~*oMR)z@VpfEgIgYaiKzU z&_*TGrECBx*tbFM0+`^Qe;1}f9^C@n2*NCUa>qcIeuz|l#<*HNakOf!w7%DdDN zH=W@vo&542J{V}evL5>oM3^qXttr_HwRYsy=QJ)&`l%4!E#f2lc5lJU{=`JvUK6X7 z=(dVQB!*9Tk&Z2mfFyq%<@ragfkcnp@RT-lWl9a6Vdgt(3#%mEj3PEI3Gx2hA1HJlw@Xuv zB8Vlw*F7*9wpG*`V%Xu9j|DA8lgg8#4sw@!lRa#K&K-T_8ZNS~WPz%ec1_e7STA%s z#YGVX9Uv;vX4E*v3Ab#=pk}9>KXuxzS8*xEVu~+_77>N(`Zp_r9dUTe!uIkDBx7=h zT)-Kx3o0X6PDX}Ga+0Y8)?Xo|3QE@w!!alIr35W4V!dg8M*NA(DFWvnvLAJfe5TxI z-`_ySYbF6-F|%)Xt2ot^ceYX4>4^??vFU&KEj#^2%-`eb(3M zw3#XJ6iK=j(<P@ zI;yUx(#Uo8`@Yjhb+BBr$}EReZ=C2PISIgj_C*`NwX!#37cp$|-y3c=R8uc^E<@>Q z8MCuI-o1`|a(8k0$s)K^ojzx}{q=Rdl<2!_ZDyl^c4U=F`5r%I2?5WI ziiL#*lguGntjI?P=>iJ6Mk~!!>^>4@G;&fhm+MRld_08#GG5-eB~o8zeyoXPf)bEj za*dWJarkVSvS3>-7;nHscPMn}%iNs(>s#70ytT7JN2PT>7#t>xst$>f_Pm*sTD?EQ zU%{KyRtgrxrUK|Sg8)1d?1gAKp^$Gqx0R&^&z-G{`7aq*ni1uHsMU48xFn^tBR<=W+zZToI zwGX$l{C*BfiH5p@;|8-QW0N@(&_w$qIGig$-=_I9QT=_vlR(!YuyNA;p`v@V0xleT zLG7`)w#n^6c;oZ)u`CU&jiV}BRwn7wcm9``k955*6pbwwD+CGB*iC!cbwHW#C1)j{ z&kQyw&3IhWASFGWaIaN}(G=Gq@N;-nID7afI&a#_BE8oFd6=C|<^>j6+DgN#tIMG0 zMR3l+^#}2UrpCQ9L1}oa1YP&K4RgpE-TFECa_kY=`^r@|LSp-Hqe*wS5I#>s%c0}n_Ci#Kg`XD4wgm5Cpz1v*FdB^h|jtZMO3qddd>vZUj`YW)S z#OUiERb_Q=Lad8lrr%cMu6i6N2*H0hx{ifZnlO|UJr1!e5`;d){VeVOs2=24y;}b6 zGmfn=2XmRmh=Twmx_uM{jh6*3hR~(gttz|Q?U=D8A(6n0aq^M2tBT6s;$g{2+_lP{ zq@yxd@E6gVMi`k7GcCO`u z??PTtR68`FzxIWG=pb|9ri%8o=s)ZGHMEx+kxT2}1yZwcywn(@msyq-zS1O0-e)vc_YJn4amq{U#4PmFx-7vS>L1T^N86q}h*NDX2m!?=!I_$tsg-D0zH685S zskUu;2mB;KpKlr$3pX@Q<^10Gs_@+PiF$uqpV2yW(xOE!x;Jd(2r4vV%#|RV|6H*e z;mA@uCgM71w4%=$rH3X{u=8uA6%l2%O}slz4)erPQdu=a2B^mt!;x$55w;1pl{z)6 zce9sj{5iN?@e8j*Hc=<42|Qq@0r%fmf+PP91+0$MvT(^BR0etEB*5dINmA#m%5D-l zKJG*do)s%828}xs|Fck-D?&;scPBi->P3Dg%5jeGxNs?_s?BBFN(1)+d!c_Is6r=1 zB+XY9NY#oJM_1Oo2--Z_Jn!D{Hw4>&Eocn(`EeBq6>kgf0U@<=p5$u?y#cijaT)>~ z{&`blnybdsLwJLJQM`cK)T>hP8C2ZUQSDqO8U#h5a&1#(N;}C*z@iU^ovpP}f9F-m zGeM%j0Q(N;*4`~dx{k!8f#svE;hV(eVF(Lo_Yv`)w>csoL}biU1pl&4Ndj^B0`yf| z#fwj(qb_QpZVHjb#x)9lm3i2rNnQZ%JigOfYmA$w*#Q!W3+fOgGUS?-G{!A#Qj=3C z$9FyAG-}YzXqpkp-pcqA*u2QEXfmN0Z+exCWj~9pHJs1_N@Kh|C4W?Wl46`Az@{8C zK8Sf{KU?xi>qI>ec&IwbDu+ z#pyd3a1K%WmWhIhP8$Xk1SqbdMLxCCne@%NnjHU>@fp}idYWtCO52A89Tr4!M3BGW zpmi{P?*28T)yYUn2g8f!r(UX2eiWPN{7qw_&4$aCoPn;P)`%WntSjNQ#fiC5q7#^Z zKHK{RJvzybK=)}xInrR3`AnVIQkzf_G>0}=L=LMiYTWne)o%orkAap5tVPjq_|GM^ z{`a{r8XhfpgNr^AcUE*IrI{LgKD|CquvXRijtArA_qv6dUP(6^G~FWB*}D1$2XyP# znd5KqCi**}12prG7nU%c>CS~|K2QGrJA;iyCehp5q)p?lHnCZ-N`QP$rSSLL6-f&+ zjSInQG_CbZ*e<|>zO<#~iTQ@YA%4)-SX4t}o|w18bu|D()OVhA(9c3Zn6XYOR*-mA zN5;v!{EsA60y({GSKZkvtPio=0IHm{2nn6^twVjNFbVz~AA&}?9g`x5|Sv3P^W%HzF-D zAkxyIQlp>Fvl7f`0 z&Yhw!!%J+AHE!4m3YjR z0tE|BHUiqXFalXDD<{_Uw5o|QK)bpG<5IBuqlk{!gXdu1OPPdz?>fxqnVnC7cnNE} z&?V*jSJU&S0$PJpH0XIAz{J7gM@B#zUHz(xtKPN{b8E}A3)+&YD1YvL)TWH>WE?Rv zEAU6GAUOl`?TCb@iQ|OerzGpez3rfno(~+I_q}H#v9B1Jp?ilY|PonRQE17UCgYca83`dc=tneD;0hE?(btZr2fog8w zy{1T?XjUh9(C3E`^fx=1FFPclC{MDrRD>h-3z91M=okHaHAx!Cf#}Q>lnmw-Jk{WZ z!Awj?R*!bh8#%THtCNal5v83Ps;1f2QC2A4*8LEE^+GMPn#slCRMwC#iod_Y_+`MU zICectnm+9VSN1v@kT!fp)9iT9hvQ-VD+sCG!56M{G65Sv6~pf#Ud2=$Zz=<;FR5X@ z*KXPKb;-xoc@|dJ)F<*dLJ81)w2Ew|GLH~Uu@N6hH$xg7x1H#t(AGRMa7GCnO&`AC z&Op_+u`?B*i4AN zJJ$+U#&N@CKS_hv_plr&IT#n!;)O=1wsC`MpcC};U#C${&2;*Kmi~2jj8Rss&p%}3 zR=oYH8u?f*XT7yx08K=9?eT*imAOY84CnNYcNboo*lw|{MYnKr#0|Ay#7>Mv!;pDg z%Ezv>YNM(HB0J^LP|8eAtnhBQl)xO)h#_atn8^7ugV~!+MhJ@gTK3$&&^*`12rCM?LB=Tf;p#Y246P<`qgk%y!SF=`JwwF!9hT|44wc1U zKFW}LW5em#Nd)xH{!y(+SI?z6%0)Q$HIa>fJ!&!b&SIv( zS<8D&;_&TIuLfL~%-tWQo3Y~NTaq14haz|cRZU8q0u!tzUh7fu@!dQsy}*WG!~UFJ30-dAY#pp`wxURVuTAw&#CDi*?_;gYoCIU zpp9zON`*O|!PL_nydU_1k1~GhH4#v6T`VS4-BbG$rKxGKjcJc2blzQRM;-j~YNkZ? zwsZTWh$OWH=Ww-Ufjw#Yy5*j+Yq6xV;@u;iH<_c*EK!Yjon@2H5iQcKhW13YW%;G< zW}+qY<-k%mT%Yc_e)u;M7VqDvsG?-BhK%>Hp5Yr#Z8i z3^RnX-c8s!B?lfF%_JKm39--V)I}F`2%eI?4k5L%;ZA3xv7JUbVrhWok#AVAxgiW6 z4ObS_0!&z~6$!t<{t%KvPSbu~;et(VYg|gfGuCLnFB+|b2HOG&-6&V|&{eY{1%-h- z@%9|6cBc|W4?b7;fkVSqT!F=V%&xHfXf$3f>1=2REUH^Hn3PS#_~GBqY*_Lg+5T^} zKW-Q3N6CC4Jr^^K>|QswSbj^QLQmEcd&gZu8TkIEb5tA1IZ2GYO-BESuXnA1eN*BJ z`|+)^8K>%L`(2~>#fv9ITnN?ezJ>UIjEWeKCLo}&n7AVgOR=NW0+O1;(d}+(rp+mg zzGrZ1*klZkZ*B*5T@$=&T-F)D>UgAk(c3)qMy>NT^qe~Nk zS;yecq=w&14J+)mK>UaOa4BSCBN&MaQ{zpACxSCIQnL&*H5WrgDxuaMtMMnBH@9>| zr^Dq~>~LPTscSPTqBCB_(N*5FVsJ!`y%@1Ez}%}f7A4z@DM^YrPYRIa9|a+wti1iWyD63UxvZz< zrV+|$6dVR>t(Cq(R^EF2Vzv;5GuODaxBmt~v@fA#?PpOG&0FxBY)q9kr5rh$D*G(YjO?f$XWWI^dDogFx%wk?zI{9z!>h=Rv0 z)_r>r;rmL-9n0Uis&(90I{Q0m^stu}&wj)dv~L^FFspi-u(8h8%t^dra#@( zEuwo42++D;e6gE9JVR-t>9NR-=Iou)hn|M;Lfg?q-|3{{+&dis17|JQR7d*|EW?U@ zEPp9{_otS`+JvzBOce)RC587e50cRw-`PV9KO816Se8n-IA3oV`v=U#gC5Jr_LIt?V3@Xse^hlB|W+vdht84Donzqq9wHQuZyY^{PfHHNEEaJ2| zp+QMoize3Sv%B#x4pjvKGC8eP#As$=P45ffpVI`qCswm` zNyQEair_yVD*Ze&;nP>^l5y`NF)g-5g9i-+Z02S%v(vAym>WAZV2uKm4gH01lNi-2 ztq-=|C8|Y+h{IZ&?W{5(+^xlsyKXfF{=ur1SLaCnua$K1W-w5TLqTkD zc;9Xc+8!v0&aWc4`DMNuofUj7Pvvq)Tx(f~5^>~T`! zd-F8q_myR$VW=a)w=^NZc;iD_yDp>TOl<=3M}9vQ*|neWA5c&)lX8@wgWYToE#)IL z90>wh2a2aST&AW=bfsV}<-`QSlS(+x#E@}A6uqGot*kE|5)9<90T=7Y6#FxC{>cgM z$t$JQNUNYqyQx=P%niRN3=s-ZDKOU&en(#@RMtSI!F2;?b33HKk>mhjsI}y2aX8dXrkT=_^+2&K#cqHz^bxil9YtNvPa^y0Cg1SHz}sI!bYgIoZM2p$+xu>i z2Z$eY%C^LMJ1+wOOvL&Qqp@2foG>ZKr0qY;!1k8EQa<`-VUFut;oK6wA*>sOZ_{m? zR+w<3A()IJ4tp#N5d?2XjD$yrdza?@&VwYK!BHMV4Q&}}enl@PWEUwoL_Fm{T9syQ}6mhG7d2A=JATal-1j}CW#$`}R8ut)t%dU}G& z3bWW6!_k57lO<8`+b|t8Ssc-ER&uaX{zDxbA%TQ&=K;ZZilTL|@i1<1y!Us~8z(}U z3+-OrB!>^ss!>Y{$Suk6iUv~W$3DvWN28OrCpg;^QK9ceLRpNZMT6nrBN=wpwi-i= zRm~pKaDYBBd^C?+jgw>Ln>hw4V={Xu%LaV1RT@AVyVEpntfw-AgQ&IPq$)%5ch$IY zq&O5XK9D&))U6*aU<(?s+e$t+q{Lt*IYT?Zf5e?*o8EWK_+o-{x#qh;(YCjI=x8)x z?S5ZiNxVcMgOZyTiQHe1T4^!)>{jCqvRX7fn^B7m*@0xgo@nS*QI%SrLYzzGg?#AG z%_1P;axqsN|M-&MNOGtMhO%UA-<}&ZRNh(aJ$eoq8BkA0{pQ8hRyN;=g;g{&RDN_s z6CjWl&am4p6GX_BA=wo>W&y^8vTyWNwfyq}sJL2Ul|AHzf}SPH>s~l;H4uB3Znxgo z_aT211q1Mi10v@URM?$Kvf`07YUPGsQ2|vHYSsC{)@Kii2$DVYB-c(fUXq4Dm*O<~ zcT_N8*u7@ybZ?OWuYiXBa3Y&0f{x`vg+M827sfO zM#~K-_o`|Zsps_ma$|Y7*mtjeza^pn0WC@GlUwb%m=BptzFxoorqK~^E2yP{hRT(N zv*}?-{c_U@VSlFgtzINN`?{)1E^y_KL!h`qJ3mkJnG5+%1t-=6tmYytr)sM0#;}Mm zjACx_HBE8P3!5Kgs2`UUIzZL#m5XA^-V?RuW4Om%m$>M}eDycQu<-LFmP*=FKNNXj zg`8va9Y?q&v|bpxGYwnB8B_2M1F0j7JErYn-VBe*Ua*k97LdxP$8RlHYQO~Qpi#IJ zGwSv9$RQ~i&*%h6$flFW?dZzU-n41nbC<5AmfWLCmr}EFi5G29BAj2?>e$(oY%r{io=}aN^tTv^>4|b4`81NKfzjYD2I7k{% z$-bBWDrqwojOH_u+sh&xH~`+9vUk-S@)`VUq2Imor7CZI5XQPIkWJi>6(Cpj_Qz&6 zd;m!h#BV#IHU!alD4t`BP*C4M&vZtAD<%E}dm^RsuoKJ*aVzqWSlMcB#fn|3v=#9F z3Z(fW{7pGWEA4zr9HX}jv!xH?BoRxvcUD+vfczTkoE)w)DqrP{fd;@%|9yiytJqkE zk0h1jMg9igLEbIAC^7?@>fMi%Qi6N+(wh4+uBeURWHz04f^TyFtQsrC(JO~FZ#y$#Hd`wsFqP=h?&(C74D?9Zwu99-XXG-AM zlx$DHyR`aUcbA1!X9AqSu9ZM|x#Yzt?;=EYm1IPq&d@}J)Q7mWwDgfrSd9(*TRQK_}PsCH9_AoMNy77 z?|udp6BB$#;9sxhYFFGWEMrQ`bsQiB6X@$j9nwSDGQ^bM0YTDDhQ-=KXe$it8Nxck zQk38naSYC)J2tsZw}R(a(vV=AIe_t|Ct;U*4?p7}ZuA`_a6kvV5ur!(aY(kdn7KzZ zk=|h$_in&kWlbAYTN;_@Rd!wIXE|+WqS`p#+Ffx>+UWbl>6dAQ)emKYsM%)z7 zy^w2vrqdsS2!$**&F+L-wLGY^%~1w4?p59SF;kDvAiM*Z;YH8tyb4GhQ2OIGHMy7{ zlHmF=FJ#9=E%j!F zU9Fyhy=cqwc-KBJg&)Gz$*m;Y*RAeBa0lA8-?+NAQs$1=opq5MW2^hzOLT*czG8_-W^lZ zVqG!z{caalARPQov2qrK^zlqE&lQj#y{XatB-QtF=4!oG71KIp?8MI2miqRz6YO?vB9@}uDu?0zY{c*FDbHh7&L~- zg>m0SYO19>`8RqNJC7^NouTP;ea^2%r}B_yks=^OC;&R}gEq!??`GYTK9@DX1}vH7Y<8ne|gam?+F1+4*R|B0j?x45ulv%NF6 zj3LWDW0OUzz86aho@nEH;|kiFE_6+v&6TS#T|^ozPj@#aJn1QMyL6j2U5AK=57 zlb}SO0`0j`zM`TP!B7Lu-^0P9Sxbf{!iO=dW(xdX0#t1jMjN~>d zQPwcaA>GIigYR@nqh$48X&f+MB~5DG*D^)$(Br=TmK*~{_{ljH)PqgmIpQ;?y(*Y< zCJvde@cBIL@og}daOwm7KbR%LMXq6p4*#pg=;JY~`|77sae2XLS!w9a$}8>b65bP2nH;hPzNXt#3KVuI~Elc0VZY5zY5>k)qADx>ASf4 zca$5^Q~0OuN;1z9eWZxn7Ao;fY!Hgm;|vN3j&8am2W?5d$zbH@`Yy@5U}!z{Gv;^ECvml}+yQ2N1@ zWoqSMs)_`aXqpzHJJ{@>pbJ!cIPuusxn*9Tk_?~X9VO0{-k4nza@zSojFSSqFBbI9 zh?4J~VJNnQ&c(WWmo0s7T)swyEC2nf%#yCxZS+>E)D!k*b+EJ&4x{u1xlSYE%!p+E! ztISnJeN_7Xg4cKB;~Dsi{L$HiLh#E}pE``yz-8$uNo2x27KTfC7|z~QeELs{IU*FWK!C^UNC~MrK?`3;8R73U)oYYH?ra~{ndiy<;nU9ou^Y4+ zlTvoDMytXc?ztEg3jKy|dgeEUzZvoGU(G1%F`>`lV*8q;&62%Y(q}n0pjG=O2Wd3e z8)4~bGDGs{I2N)=!fD&?N^%s^p*6}8Y33tc7CXzTj~`u4jcGj|0Nh8fr%kZZeDcPV zl{beOhN)EzHV+#h2N03w#-MN})sR!IpORs2&jJ^f{n?K;)mG6JyWe8#05zQDHC2Kz zH|PYVTq*JcYLL7ESui}L&|)wO&*kih>Z38+35kKpUeY4RgeVqlE?&6R1|E?0F=ANImzFGf%_U>;58C+fb_sT`O@>Qn9V|(<$+C~XL#y+XwM() ziCa{Ner_uifBP^#Uf~5e;AUhk2cgN2u5*e>;ugKl8nLeX#NFB6Zos<8t~R=iCgJgD z!krn z7ptip{2~!vQh-AtAD_p7%e}3WyKKcAP#=S)9E z$gDKY7+4=V&OD!AZeO5u1HLl$V;Op}_QT80Ju98ZWwg}0-mcjtTX3`#yH_RDf zJ^}<-Wvi}F3N~T{oV3i8Q!T#bVO|MwJx%9{VwZIX0q5iX=cpAf(1tjz_ys7c0@Dq(rv*_O= zHPe6$!@3QdG+PXjQcjehOLk%+VgAI^=cvyDgfqPyyAeu?(QZ|^p)F9`6YT9jbEMM; z=4H-Kr1F>(g|)%BxMX5EO>&s(^JT>DlHzF_C?Mz4%=vw(sn^lZY~uuV_Ol(%S52Gl zudsCd3{R5&n{o7`E@dd2%Ur-_v-tu_Uu60|alqX|B}mT@u7ZvXxtNo@O8a_| z)hCyXdM~avZ?0-yyQMoSVOQa1c(gc|2Ki|bzC>ARUBG~3qpfD_ibtsLS!?rrBuTQ1 zQX`%_UcDtZdH-_)DGG<7w2li~FN{^2mh2^*2quVU_bcI`^Sd?$28-yiO>!uF)1(C2 zbGYiKxDaCDiJ70pE_-c-<`FFC%|theworzAb=BsW-4d~_+3A^sTZ?cMn-k8#wigG* z5pq7jSH#xcFKNFJ^P?nAkM`Rdp>RH~R-{lrpn}rrI-Q{+9p!!&D(G+*a)JuNUO&9m ze1yJ>I`L?O#2d##k~+?~VvZ{3=0PafUC`LIE`EtiSG0GjDl-8r`;c& ztLCSLPZbnOU>?=ks5;j&M1GG&j{$0$QAu6Y3DoA~^KpqsnD$Dp<5U~tk5RAM1JQ>x zLf6^wf#eKU-}E5~MKUMnS(gAsPc)c6wEL zR7ehk4Xr|pWfS(7pOn~(bkgzUW;reMJo%}rO|HEnKT3Yu8;fqJbrvN(P6M+vC~lF< zb5UBpA}^kcjUlT-$sH_b>y}DOd$p@3UunHcu$ti*j3_BJ_0#(y)5;qeuMa5qkbj9HuI7F)nJiiV8hA zR7{!P_1*nZo=k*-HjwvhL-gqlx} z+1lEw6M)R*g~i8^q)NO{`_5`j3hBD^8Yz(C0YpxFEA7eTmQB>71PF{_RUp+&LmCt2 zwTgBPDvfVghZ_#Y$T2?`vuRHZ9>4C!o`tsR83L|WLAjS$G7XCN0_xZP63mTo#xUi# zIBo0l`7!OwsoGLz1A^OBlc3&~bTE z%WhD#*cwd<8$Xe+;xk;jueK#@%54S~8(uj%tbWf(nqQ%WokaX;eS1s0zroXi1n1*o z#mTiq^?@8>xE)*XPBQStBCk;j!gIuF@rq(UOKCtxPG7YBhdTS@DZ7meZHK2eVWq@YyR%y|P)Z?RNY4?n9xp$Sxu#OXMm9FkEq$9?=Fi+(aq zqNa9tVcJa6u+5jE9$qFJcjm&ku6N}ia=-284P(WqtVIjoY=>hFRwk{C$+w4E6zocJ z!CIc!`dB4uYt}bd&eE+63uDSTawzrC%-v2-8X4n{u}>B!-1u=oh{E6)Bm;wZmzvgg zV&3BG(ogx;d@9!Jy@8uKnv~AKb6WiyE4n*aRefbweI=VO3d=Eh)XKF*fnJimOxAcR zaJNy4lL0@*I!5lW^;KpT#;f;(u59rJbi0^()iEzKv14#ynbK6LIYb_h?8eUC>0mR& zW%IiFBhcJQq$^JYJ}fYP&Hq-27E--gS7|p??M0?u(0p{y@!sj-j>Ooeo$5-lRB-8i zuD-S81lDa6R7B;6U#{~qjkR-_ZSME@s0=;+x9{9l5NWuOJ;SfbKZn>k-k_sp5>23o!w^MObTVG@AS>9YJ6K0c#@^Yxt4=Vd43bLwT zj^`#M<30_(49dw7fzkx)*d}sAXZfL2P4_GHSAn7_9#jkJ61zI9rx!_RlYE?${aX!; zv>q6L)aqwqu|D@UySta6)l9>prmg?T25uT7TkPrArTIv-&*16qk7!;Yl9ids4Z~*8 zt9sb_+kT?;NO!obPe`KlwC5sXad9=QpX&E2oR{eqMvdmd2-CtH&-%W$5;_W=1jl+d zvIOztXcWN^%j1P3hQANrj@Y?OEb@(1Dv?s5)#g}GET4rnM#Y3B3=Irdo{hSV1dj2u z@pN4m&|}vIY%G7+VjnUdwpT~rJP+2qeeRO{DpmqX>YfHl84LeTzxym9&3y%qK^}+M zi>=vQ1@0iE`Dhn&@hxXI*DX;GPCd?f+8IoFkVzQ>)3^5gz>3_faN;x4c(i=*lu-AP z@BBvg!y)&xT>;`{;!H70-%N5|S0Gu8N}(7T)^Ib3HY&*KwxL^1DCBU&I?4J{Q^t58b$J{3aMLp$8uS(Ie z`o5)ORmI;(y%GGrq)p>~84+dhSe@u=%A+M|&{eLZZ8FuXxac|t>hJn#m&7dl)orf9 z*|E~L94enH;n`2T=h2C8V&NKCd-l5*ZQ|=nYKx#mh5E}+s_|0Bw4un-&)VA?c+{?o zP8BBacp4DXz*Dka?OrP`K*#-aP~BGTKdSK@Zdhu$k4a1$4V5|@XA=WsJM;?zsHiL& zW_vREBk^w9-<}a6UZ!Fz$-;4jILl|53p}c1zWhYjVr8VszpZ?%In0voygJr|GWh6X zWit6Bh&-waH>{Q~q{WXrHK}z{{qL7vV1u`shWG(lQk-M{4viLhNDt=(YgTIs3CB26 zt0?I6<~x}Fga{h+=l-C%G!8NFah{w?SCdh&B5&Q)(r-kbLn zbdP-c+mQh??Dy~A&^8380 zV&2Gkjif|9`qm>7BI(?pf~iMNj_v*gWm3bzSC_V#^dH%ce;kSjUR-QVvoyrwt2hMT zbe=-3mIW;rRQaN5pNsMz_7UL(VLYULg+oYJl0M?4VXbS% z%lz@W8aboP;ai@{{f|B_&`u(FG*<6@!cE{0z^NIbM~NKOO!dy~f->%9U5gjGT@Pl3 zmpJ#T+g~sBAE?u(K4QKjLmz?*aZ69M3yWWDt(M8ru!!XASiTVLGd4CRan{4^gd&2{ zKQ)WI>bHSEm^CS&NX5g1xgkfBWsTQS=Htp|l1p+4hK=HK&t&KT13@&QNujfzb|wyn z{oEw7(Nts=p3>hdL%F6Sw3%t^P0J9ram!ji37AY_IQu;=hFyJMNu1;IocEzx^?2I2 z^UdLzlg2A$PSdqoh^p1Le>dvWVwp;WsgI>VK)^iI(`}V4z!q6(-s*de70m5~Z%n_;7U=SIaV*SN$f~bw%@t+Y?$8@_SSv1hbTH?)>WEJ;@1yGfusX76?^N! zKm1my0$qd|v!8QugB%jUnRT7BhW*zg-OO%DAqD^(XgXi=`xoJy(qr}0`44N*&ZAEQ z{!0q(r-6Q>-03vG2jWF!-b*-b)G*;Jmky7u--j6Z8*Y9mnL$-m`iGr@@lr2oU?%9D z^Dy#H?iQnpZ03-7zedl+Cry2R2}Q1xsU;A?OzLSTeFob(<}Ij9z27v8EyX%Lxm}dZ zG@#+{3oU|!Mks5;?}dM`>&cfO z-!5dpgAK~#D>?VD1^z8+hd3#&q3Ot0aew85t5XBQX?a{v#fRQnDWa20cXVww^tWx) zHdJG3p6BsW2((m==ng@sZYq?{l%pYb-!me1dA*y7YczAOAAXtT$tIjyyq)D7jx|A4 z6U&apR&s+of=rJj)Dm9oDllUfEA68Y*Jn%#eINW_2ZUs=mu;=z%1Q~;UPEZ8O%nbp z-$G-ZQyQws4D$VD+MAqN|CDbjD@c=Fwap}!R)5i35N&b_N`;yy(EOw~Jo#W1@^ejGxwApn&rsD6#X7Jq!W-88>rY2C66y6~5p%C;*UsKSDkUm3AwYv~5HW9Z~&{6n@s-zE9DUUUY(>(Fdi;VmT`d&|_ zk0C+KmhWnu&~#C5<1%x^94ScuTI5hlxMEeu8({o6bJb5lPMr5{VZi^NZyv|FiPD10lf09 z|6BhBgs}TLRA>M1kO#o=sFc_!)y1K}x8EOSHRwtStvXbrA5SAv1EV5gP(lb9d)u1N(F&+=4YO% zA>hv-{iID^sk_ytju-JR_x8?7!ZH8DzmFDRhP@8JK4%Mse{8b)?X=+| zLFYI0bnWT)V-=t*(pZJ`y+3`~XsI~6Fsve93B12I zLhl=sg5EbKm9h6n@xS;PR_I~9M(az=%y)_iP=5D+bA=0o>fAN?4(B?3T@TQ1Jzp`c^we<639YzY0O<8rqhYk_i5N@tVuWK+##H zO!t2_0xGM{4yxC5WPkM{5jwI@J)@;w9POe|56QHdP&<$}4GT{`4KQ>s`(sttq2E&! zRflQK5e$xyF4^MmKjx4F|McviK0*LzSh7HH*2(r~=`z0#O+Weci(@Qh*5f~sVlM)? zVkwe}MEvdwf2*OZ;{QD&^lsy)yN$Ox$*A3tuFN4QM^%vLVAZGZ4MT#Hvq=0Z*4}&W z=J)$^upTt6@z<&IPy(>mL&(8s%w>0sIodtR*RoVVVih1k za!D}+Z7c%l6=rUFV@rgkoQgH$bhXHy_|Rhc1{C_*P2{6S3!2(Pnj=9xODGu`))y`k zPv%dq8$vC=o=YcPrOvMEQR z?$WqAene0gV*wXLm5j?(_}WMCJMFW*SXcBCbb~9Vaif10jRz*|wc?{Vio`iWwJf)q zH0NIS-zxxg(L_>%Q7R&(ztq0zbJDBHjQZ+`z0jfn;20O{M<{EZ{i&U9#?6rWOrDQ^ zfV@sQ`6Sn&EF~3R(1C*x!C9wg^4dEi=x2Q+R16k6-%oZgCmIG%S%zduZFlqte&*5E z{A9`Q|E|VMnP5ZM-CX3GU4S1=a>t6~wV$zDjB@ky%{HpXn2a$Gm^PTEOI-eVIwfv6 zRZB6f)Ymsr9fjw421)XQl&WtVU-|xfHJli!57vrBFp>dUnQJp#_v~S6?t1?oBVLLK zwq0}gGiq3SNboPPxsX>}zN_eueJ0yfSwf?y*8pn5rKURl%^?Y1HD4)v+WImZ(skv8 z{_Gm5m()^yY31~8dWuo(GZUU~E{Yn)|6Y1}OF9tryH()ckeiQi`~u|&Tq301exw}m z4F^Dbaw}5@FgnCEC;oE&KuL^3tN81a3wUb@t9sWrO@k@FG>?oR4FhcuUjRH07C@J_ zAj=-VXpT}F`6eV$Kla;RQnZ+iQ~K&xrBH$gPiHA}BgF}pFk%m*pRYT2_7RiQcS4Ul z9J09T1a3&s5ek>u_-M*;EMx_&OUZos07_g0=(xE-Ef0Aw)Bfjzjj)I|daz^dY8tVX9kKF8(8X@N47$2A< z1W`+c9cb{ULX|9j(U7#qKq*XULYWxrr9@(|C>gTQx*E5gp=k3ImOZ_XQXt%x^~h!I zJCXx#_xTQF+^q4{!ZnA3;ipOdvl{5gV0pjszlwsgm#z)wGHB0i`4~yQ0aWoBm+tJG z!HyR|?j(Eee&*+4zFt@DqZxf^)0}ca*&hc8dlI#0K|XhWaJsgs?8_@1KcE8NP&Dl9 zK`()C>gJ2G7w^oJon(TOd??rI}-(&G2Tm9+8HyXEmI*k=5@U^`9hWG zB{hkFdKPGBvcG8LhDK9xr<%5$dOD6I42$=41egkx{GZQ;*1m*2)xLyT__*u2+3#=D zG_tDNuPjd)OT3xOY!2r4Hy^rn;^{?a?~*@bRziWwyS$QW#9^*dIvCnqh*z^r{o-FU zRh1zoNV}>No5r)#0TCT|giC92kZg7}_2#q*iDeU`fSDKQLmZe6%OMJDl>(?Zf#|?u zd{croqrJm(4Ki^t<6-W93)D;YkKfZCC2p2V=z)o_-GwIZ!TdV{1P7e|UMS$ncAbfs zyY_l(U)-s)a>>NiX^9_NI$G&`u0fA>)+!|#ss(rRj-so*QklON9re1Q0EtpAI?M|o zCBvtx0OeQL3iwp2E9VExHn+6u<9wX8vahYd%VBn*4&egGP+M0H5BlCy_Q0z;CoHucTt6+2M0<{eR`zb0}fmV?}F;4X#?JqSQl@ryK>BO2+7sq7#3Y zesU&~(Df(fi7@x74@f<+{nq$wuW5Ro`Prt`m+t^dh_5Vq5kFsuwqg#7gZBpHx}&t6 z>eJd6(pUQ<$ZGq^cv@KI#>T4fnA7_fcv(D7G(IjVp-_>jn!~+%7W~-6adsX?Bz;O? zz&u@D5VT-$z4>!3vyXPO@NcAWjb)$hbtatI>>Bm4M&^6oiyZGl8VJ?j*@Kq40eo2a3U>dd#+RvD@I;2v&H#yvhmmXy-gobsNwgW0kj!4Lm7e=-8ng1FCuN2aA( z>`5M7{BWpcAqPh=?F~f{fUx5j>_!L(n-Y2LHBe9N^{ z%uVz$TN80WpL=Zhp_XA^B;Z4*H?lZv>EWROCWiI+mbXD9Q<%VxJv*-bT)n=?>r81| zo6Yv0al(fqp%f1QKJdK!z@@z6K=qLBx$~lMxqLhp_#W~XpCMz-P{o8OOlq!sZb~N-_pKHQrrHaS_(b)Q*>5UP=7-t{9+%B#J;KKncM- z%2xYKAjEP#Pw=22TBoXN756{T8w$gk&U47tA4?4_Z~NLUfc65^zJm)#0!AH>40mBI z8bCTk;DD{$YRynrb5k=MGGfvX+1L5?%~%p_F4(W+A@J!7aoD6R)#IGM&MqT)TY>H9 z`~6wVuvGN5<9H}iyBr1vwv?MMK!v+CitR{j5T zyRC?zo*xb3nywoH`RS52Z{HQnp?^J4kkc1=M1cyBy@-YgeA1$gY3}1!SH2*RUpKQR zay2)YKh>taw{AVlh`6&rB#wjn3@KTv%Q=4lzPg`Xyg!Q2q-$Vh%osP{k2pD0`G zI+sHDA=N9fQFU;oefd?ud1v)zz%cv2T$tBODCSM~w<}`vscp3^X!z(7g%j&-I~!%Z zu7Rx)={Z8#8ZdV3%c=HK(cm-e55Hs_{fJguwT*QjUYp)URSe6UQoz&~Q5f1CP(4cO zptB3W8_b4q{UY6CAU|&jx+!lUpELZohM^KSqk>6e4S{FX(Z>i!+|0I8E;xNhW|RAb zM9d1GJOZ$*UjF!BaFLAn^kjI3qCMJcD#~kLmBzCkS;T`#En9S3kWI`=nY7G(Zv|@g zKLEu}Zp?l-SHxWBA1d8`hrSZSxBNP+AW@_;(Eiqt9I1~) zjIf6wh!F}qe=Q(cNYi}N+0m%P68t3vak)}H_ijTaVW5=nr~U7c$w#i(bzJM&UjJK( zw9N8rRlILb1xY{6S{|7^ey;*I)o_XbJT$Z=WDE@ZZVnG7tZ7Mk;*qtSJ!zhrBsJux z+h_yFdkjg%HJmr&&hQk75~x8ipBc1qf?XBUHxH1^F(vTCbWfD7{5qz)rZ@%xWlx#d1Wfr%2GI) zpM^c@1;Cw*^z!4kM~sjRQptNpjv95e#`iBZ8C`B1ZP(H={}12LfzFf2jSK~^M}65R z0toa=Tg4@1G_TxHkURtq7NG@#e@|Hc?4lUxwiUq&%BlpUzpBs|w3qLMq@-^VNNZ0e zdnfi)6|#Aoxu&deRmu{y0ikmF^>{sfCvp$Dx^tV3BOCX3JXhzD%OZ}uu_8seApW^*U@fpRS zHVH+tP+Vs225!o-YIG!#kyYxyrMH!2DvBxx~qD#Xjts8 z6+#N`J*B6Z4U7u=bzL9oi}z(n1UK3J})N+)R%WTz7!-N(+=B#~Lt2V^J)}~+AObHLCY12dM3avIotF|a@ z<0s@_XjrA5`eY01&dKk*oHOS#e{;UyoH>{8f0<`{x@%9S6|e0dzm91F1PBl~ zu|TVq=Wx;NZw!q+66ez>{5y94dTfidKRVy;?|FD|a;@xRtC%7{fB*pk-=jdK5=!AX zxs<$B3&ud;Oca=S?5E>?8)u@4n5Q7ny=vRLQCsP{#(|+nV)H*t?$aZ3_))8|fBaRW zIkr0b{dglDe%9A}=Wb2Ee7;p{(LjIz0RsOMDBl|BYJBjIh2>cBK%;0bi9*o&3DZbX zKpOL(?9JSj&tU%abCP5l1S&BD*m`$ zzZR{Gf782q@Smqz&oTl82oNAJ69g)iP-=m$d-I+>QET`|ZQOXUd1v{eEGriHYt^=h zan%LZZ(sUEQIrGiIOFEDOuc;4xuPLxx6b>-sCw^u1QYLR_k0*BY#en}c_JQ|;We>8nN_q6sSrGs$++8^iZe@CL} z%b5N;mv1Wp9ewCBaV^G|0#BH=z9+rF50<$vi}Ik@|DwzVz+0t5&Um;7BTM%erchuUPQHp7HZd6R3i~=C|%S@9WX*_i=Oj>v*s~ z`n!&av94PEnqMsZ>)tBFDTbLT5PdB5jT|_1ca+HgJb%m5{O_u@G&7^HR|E(Um?nXg z-itEMjXg#8#hA0_5zDj+{f4g$=rayONXm+|I1ijt`ls;|eH4KS0_ECO-6Pqj*A``QV_6p0$3xZCkz~3e&dbLq zKPPPI1pxvCI#b|Czj0amFp}!KQ<-)}bGDsbKRElnboWb#J2N7Gdrg1<0RjY0E>PKo zQY+*Q+wcA9SUI*OYHnO`aso>TR7)WG3LB1+(BG|Cw0TXnA`?v&1fqFdy>amHMOoIo zxGeJvqR;L*QJZv5%2Qt?o9ahu+QmU!g;>QvfIwvhMkBIjoUexCY;~~sZ$EIKr&{f! z`nj_{mOQHL_bxDh0t5&Um?Tj7oAy_Bt(x`4e-1sCifnb1ivMs@{9Y2MjzG%3iXZfr zyt{Ycf%(sUQk}p=LVy4P0t5&UAV7cs0Rl5kpz;Z&Hh7ePr}aCQT_2@MKZ$C{kyuJS z?PbOUI#nQQcZ?Qgb|BC5_vg-=z2nx)pKf$&EcTlK0RjZKr7i(V1(%>M0e>05R0X~T X?Xus-Yic~l00000NkvXXu0mjf`~yGL delta 52575 zcmXtfgIlHD_jk6N$+l~<-K5F3ZM!?$I5kbSYqC9YvTaXxP4za<_x+te;6B&2*IFN~ zy-$vyB95SHy^Vp|0E-Uih$tQo{lpM4Ve!8{HvZXHeSBANc%M^qhY3mXFgd=U|Kh`Q z*jq6RgvLsF(Y((*uYP|p_Xx``x3NRkrd(RbL6O#bsy9E`m9lW#iUZ?d3cQls5utF5eQO!CfEC=@F zh$L0GL81c$EzpUxLPf%yMAr-5d0FE!gvo}_PbQM!`{$lsgHqxMbMV5l`W@Y}%oKxs zG&B`2{VLj=+su|x16^9d2w)SX*7K5u3N_ZARj@}gq?36rsc%gfBs;JNMhg0!U0^9i z&w7JzCSudQhy^o&A8#vKsKBpJu>Ghzt6JP_qNm64-OFC+Vt~tXn=$~9Vz-$<0 zn;XVY8t_|u#vTnxWiT|hvPGEnC z@NXb+)@Y`pcU+#wZX@at6xT`o#-WNBq-r%ry|667HSX>5$3}D*BWB(U*(G>-A5j}0 z!jU|9pUS=W%^`YXu%WP7Md&x7edl8XwoKNOQz5D$h6$J5jcD-!L|;W ziv=&qoRky^oo4&k#^^#FHRMt9G5l@=p~dp5R#mPBlU(Y`1hh+q%cb0vl)k*vNRsIW z+a=c5zJ@zp21*}Gv#Yi=#UxNV2TwwLiLlJ5sU}%0?2p7~1U@Lnu8cx{Yo1bo?iL`< zJvHWEO8MIx46D#D?@P>~>|`Qkf3B(*b?W#EmcYi+DRSm?gi!vqKD4oWu*nK`K3L@B zHM0MTEXXlKyR6~Lhu_nPg(~b|Uk_ZTo?b~zlNs;w}W$XO^7N3M<_X+sbslRaMuxm-8#{VV}pnh z4no&#aopB^EcFGO-ykf+LkiY5^JOn3tm1D$QNNSDy!}P?iT{LrfO_E(5_S|7UTA;3%4NUS!k4e7cK~&qAMwos0$bKDO*->e9sFk+-TYqZ%dW|0AiJuHlZ_3u5HWZD z2*=s8P`O)p>4!>;_SI0yD2H8Ffop}rd88umIVz7arr9wzg^$5VFxZ{nqqw3w1~mqZ zd`cXu&Ew)NJd4jGPK@Jr0s)4@+%IKQUj7OM?grwLLL!!Fd?} zEWZAQiAzMcIHo=Jxs}D}T~bzf^RLX-^9OpTi#3~j3&J@XjidfR*&46oboTy1cBjLm z&2Q>&1qa^1dL7dY!zm0y>`Qb*Lt|Kj&Fo3qAiCHgR;hqCZ;8!4iVh~K`Pu42o(|81 z$p>f+kY5g9D$G6l3Ti)Np^&8T_8UFiI>JcefPb)z6a-XKlw$r1=>WUWXsHAje%qDb zw`w7&AfEv}Lnvc%urV_3aRL9)?}!)Cdg-$4p96vKY!>Ii!S>qLm?$_j<`>KLhZ`SX z!-xwY^>tVQsmbuo4e9WJ)O#c2IH6`weEzn*_4nlhot`yudMIy6Fj{H{ub0TT^rV!3 zDU9CqhfIHUwa`+YM1Io4gsk;FiTeu<8IfXj$=Asu^bXrZNj&&+&8{1tb(@i|3Rx|| zM*^hQvMLh9ni-=a;jjd_6G!7BAjAzMNxm?3{E4@%tQL8eKp=CS+rS`QeCq9 zB(`3xzxwRofHvF4lLFzJ7^{W-6m&BFt^w3%&O%sX#q>#0#GQE~bgF;j*keyoRiUxa zwDwofELcB4sGbB&wy^oCSplnXJK$&ElGE;B7W%{%-2WPz031)4(k$I~@NfLoPvQwJ zUQ!{F-v-7Bo&;1_0$=b8_Wn5GgM@ug9*%(=$CD1^Gvt8&bpznf^F0}; zRc>(Wawv*PgWLfH)Fr`vu0G+?01wB&Ot->cN#cMm^&_yfSfz3VD5ZT|fW$6}pJMQ; zN&Vpyt*$J30jte3SI9!f#kGp0y2iKC8`a3!w6*oM_=SVa6420qe2c|!<=Z!2pNhAF`)aQico*$ye*e+ukk9jobia_MT2l~^ ziTRJXRieTT={Q5q#Tr(z8onjncmKw_S#T>VqWpsINXz04uRdz^vIU5JBVVq)7PSlg zp7BQ9D%9vBlR;4UBY!H%J+A7M(1=kDacz1lBGchu1ls z#%CArWQ+vE9=@53XC%FUZHEoz;pzN|=Rjubx1+6$)3>TceAi#cQKyIin=^R={=5B` zm?|zvS4D9O00J0)71W~f!F2bR%0iv8)NeM?l{+vo(bFSvEj2_`l1G$Z@|CZ27!@|H z0OH{u)`f}CLehfW(B3Mjs%AK;FFupIu~Hc@90ucX?8xBPX;9G=e+KC-!mLI49Ujit z#d9}5kM90|68IcgNS(5B5^L0n`%#@cauy35oF+NnEAs8GO7a#GWx|74-^sO9AsT+TWyEZQ~rz>olB^+gG=EeP@ z+HbtXrpjr#Fhpw#2}{Can%NC7HZyOK5c^`QDW`w5D?}CgnG761+ZPue391yI3L2mS zDu@IQaIXyOAfW_H3ak;44or9Gu{t*=Er0j>ZV}y(EP;0LFUA8Azsobqb&T*_>DFF` zTpi_~R}FiG(fz=Y-eV60ku?Kj{a3k3?D&LF%av7BLMIz$Y(nSPYj4rR3=+&Pr4me* z{Kg>(bY-7+yKd(gy*?+2-?lPYHK@b@iA$vRBn%V$Vp&~vSkKj>v*DVyMv=t)DzNJ1 zsf9;xr;JR>#JlX9kBdZr1J~x+Le2NHK6^} zYP7PQ@Y2cjM>p!x3as+8?pn1KQn!jnh}O(t!7S%UbBs7sBiCiiUIPm6}IZ2cBZ zJmb@oi(TnEM1a2%!Rb#BHJO&4%Rf)@l?06KOa=*p3tkUc#2U1btsYsOj<8-Cpv()` z{-Fp;{U5M3k_HsnHSP|@cM}L=fN0y?%B!mUkd8NLya?pbAk56n)SjN6Zd_-HEFWW{ zf0}5Rcg(tF%l)u~(_kO?YB!O~Zq@yFV0X}O~;FOFm(N>3J)zSvv zC(GCU;dg5OEdA_HRiyhpP8gN7UCkPDbly7#)h@S(&{TKP#UqB86CA8vVA3fDGoXVRc+QW(X#xiYZA!{`}6|Gj_zT%-GrYaQh_S-?KH z{15dR;Ce20xN0rm$e&%3XG&(bJ+8=rN4>(L(QwLd$;t*Ce&Z6_%jG^-ID)LAkw@`= zB8)Tt)kF*(?(NCYk(|*cKP+(W=(yrwn0@ZVV~|%wl}&2@3+y z(2&kfR=O8r-PL+ z-}3iWku&rY-oRJ=;q3$g6b4IakOfKT_U=guwS`3qAL7B@UXN8%)>)A-8nI4w6Z>n; zT_LDGsR+b`8I(WR<7TipMeo*F_h=YoizoRio4 z0Zyu@w%>lJ)m$zv>*vq0(c_AD$Zk*Otz|rEDh``5T^{nO2{a&~b`j)Ur*2k7^g1+e zL9KIPf=rWK)xAdi-+^X82R1Y`;459(H###^WCRvDSXmPdVTSi)a1`M)j<=vvNnRL% z3{GY8#I52Y;EimoG)3Kt^KhH7Q3!BiL*N&M+uH(faiWp55ztcg;W6-M{M(!A?(dUk zW)x&OuAOFqmRh^jCIpwm>_y@C-(f9Gn+WsY)CEa$yaRsY#C3>)=#U6AJos+~u@Ct@ z_$Xh0$}aVtXutHTQBqbYZp^RH&Q&iaMr4)aoqFkvz8m*!OIJ_T3H0!KYJB|)SY(A+ zxrYmtRqLzELm)4;!KBLG^VP&BhP_ZM3)#2D`13OXdFNeY^Ox~=tNHnE7X?zmAmujI zk$>_gK^(3}!~>ctRii>rw2gWX)mmS6fSSLxO((Y);s?nvn|8i{eeefN!+|e@Uyp{) zYADE1@q3Qwu&&FUBYqTvfD_7mGP@H}5W^lS@>W+09tp$c=j|D1;RW3O)t5XtDeY=b zPr&irDh%|@LUl{+sTI$yQ%#-9YKV0sYpegxSZPQU;84&x;he{=nOeUc=fxt0Y`}h< zm#wNv?YGd76pKG1luR9oQNDVr+2CT~Wq)|9Rpx!d{L|GIZiSk6jAQGxInK;M(#UE< zNVu{zB#h}B4|oiwoPI|&DsR$4g$2Deo1?-u{}^4)(4KsWX#HO^&D|o$H=ll$kT`HoNUZNG)2{9vHJHsb11Do@ zvWiN*q^a;CW3KP9McZbEj{-u*1kQ{WEFan*2mFue408JxQLxjPmYcZpI&IAM7L$k| zpZ9Y{V{{GiU!%t$P6BCfQLCT2wts+^)C-fmUMC!u`vGbz0Kdzil9wD!9L$BTWA7K| z68c|El)jl&=tAJKMRAvr?6l-+zR1qoX4lhu46s4P03dqQCI^ZZLJn#}Y9PaoXSQ!-_yW5+20cs(kGnd;> zJnB`iD*SBuU67GT{WIa~#dE|A8x4_7EecC6MRj%NR^~D?v~UHQA?c-`z4c2s^9s!H zU;UN}P*=x8>(=5$Q`&MOvZKvtvK*VJ&-HFaoiclS5^;Vi9h1G|%F*i+4^irdeDvxH zDa0d-6SAOSR%Ersya#a&z;wmn%Swc(Bm~8d$V-x3yyfJUq0O^l_rp!B{gHVQ#ZPwp z@E^2%Xv;d7N`zramVxxbxfF;r-u5);4$;WnFdxIwqVc{=V?Kf`N@8Vd+niHN0ndvf(!;D><0 zggEaEM5Y-Xf|#$?;N{k=t`+!N#gR7fRKH-yg$jd_U&eEcbr8z85eLlf6>Pnih!kR^ z2M}$kRR5a!RWr3@Ed}*`XZE)Vl7LyM-ENoh-WgUz0(kf>_`=33Wa3w57ncfrugmeO zPWZk<)+4+u#Xf)!#lOI(?~7XqXny{Bs7!ErwW+`=9?KHFgBE)crU?kq6ouu+|44c` z8%Q1yHG}u_i1WIU*p*2kyMN=gb^fBqw%ojOzw zb$IH&g4quV^5d&qufALwVUAmA`R7|H=ph`Z^upp50FMCG45AbAY=eFyCm(?s)bQ(! zU2d!Uu!bmsR1b*-M$XZ>+vGRreXx!1oESLrUm)bZ8lJ0(x#uNCGyB@^d*INl<*RaQ z+G=s2A0MW6aN-tio6LkBrp3LaR+j3H#jtM95m@4r zJ6(jSWq>UniQ&;DG>u6Cb5KO`vggG|%y&cL_X0o01OjNtU9A~OJ7nTAS<(oO_REKwvP zb2vs!WvUaeJCHFv`)cA#;rA+=HtLamJ+as2Exkm+{wZduyBF$TZphfiA2{P4*~4>~ zc>}eG{D7UbKLUd64&F4wVY%BIQ zczC)-EEBdFuXzqa=sWhwb?V`2g(Wiuge@zZ>FGCzl3K-esG;#0nVFY^;fL%mrIp}| z+AtFZlRom8KcVz-VbF2|tSu4Qxy?%g&)u76D-!Z8_rufu*V9V3n7T0l?r!6eGunrD$&?@8wC*fECh%?N< z=>kd2R)5^=@VMotpt}dBeuT408FruQg3D>J45@1HQa-64T4aY%}hU&&mhaq zaq3?$`z#h4y(Y-YKpn4M$1MiCB3{Gt@Rw}?U$S$V3nj-q)YH8nLZq7-BD zm)+Q6)#{Mjp#J;~_Dkd?QoX_I=5d=kVzc1%1)SY#$>KP`!MTt82`5_K zqv-rUm%~8*8o{BN7!XDKcDQaf&2h|$ax2w&66fhVJWU9w}J#!Deb&+AI!J2NLq+u8{forY_=kAbAp-B-1PHUm_GQK( zKJ-WScV0K^+1EyzROeeos0h`Su9u!4msYo6Gz%A&B6E2~y3VC3jA5g6218R?YwTz! zj~RQtJem!!>9m|4Cv#=XP0=-_!zbm4RrIR2QxsCZDMr|Nj)0xfT|cGL=7Y#7zK;4< zGA=sn_*dU_kqX+n%`23)Ij#yyW#tC_CSZ!!OHVo=K$=;tCYsGYSATNfRTY&=56=0t zJSk}(WF{E0neyOQeQJhx-ai)r#Y9S4+i7_v92RWcGK*mS!xwp#go<|a%sz!ht! zm9_v&kSK8o{ws+zddpr+sE3L0wX*m_A*aNsmGl)C(R2-GN;;p%IaaApyV$&B(KP5bHH0jrp?I01Z?Z6FZ?db4pHfQ zhBihriw zmfpnz+X4PaG`&S6P1YwHz}mQR>d_y;D0c4lh_6v;yvc0Qy7+!v#*B-DRy@q3Vn7kT z+X!den#ZD~*nRNaN90?xmdQEee5TB7yOqXyOAdlgd|ZB@|6ugFcWe_FHk#ws90o(Y$ac710I-mAen^3kDn>B3{ZLvLLgo@B#e)wK}8Hp zP*aI64oqhokfsTw$%{SoQB}XslNRoS%;fW^FM6aMjxQ}lO~kLAp{*)i)xCR z$j|2}+$(u$+|Wcy7Xc|{gC9D@_lKxCU4^CAn!Z2+_U*W*yl4u7>nsx&>DhwQ^$a;5 z;1lQ`-IK|K({Hu%N(WY}%$rwQdh}lAMPTt?tlmh9=4QCK^672{7Im$boR2QNW=ukV z|A~M+p#e?`_iA;Bpj2b#z7g5YcB8{9XPf6PMQq>T!_6)kHSZEo6Tnge5yN&k*;pGfmCY-kcYR!q!K1eHN>G2|95&U&shem6=<1?eeuk#IT@}`);a4Fq5y9fa%1Kj!|DxJh)Sr_l0b8~I*AE4T> zAT2GsP8!A+hgv$g{Z=BYV|;%#!fl+_;*Ts--j#9S`^QX8oudzLMWEn=feC|2i;JjE zP6BX~$s8yoL;(^m1N=iy*~$OG6@faGF-ks4h5HpJQ$qwe|L>O{CK@50K;yP^OplQm z*eu?Vg8{qswciqN!T@qV&&AotXjQnQqqGtK;P+yEI@v~p4#aeE;X@(J;A`uv^i$S+ zCN?~qhn48GxMXrWtI}##v3|B+gXFuPe~!x-m8mnYqo=2D&-K-K#JQ%@7RKx0KYA%s zcDIW0XQB(Qk0avehF$mq)H)|?>4Vs3K#jRy-+sECF05SsiZH}ob|>JEWZx0D9mkt5 zQOGI9Qnb#Bje0Q^(#7^$11GE%kCDPx1H`berE2^a-wE7dVqVe3Rrz`Q^69Bzh(lHt zqDGumR3$Y9XJ@#^v3q8?>`ZE7B!fi&8Rb1oa4#;cA6}pKRbs#Z@+o=ro@-A2{<|^4 z^$S%U9lrJdyD_7U3^5JsUiDQ%{hxKj32ch*34%t9$Zc7ePVSnYpMT)F=7xuo7v$md zdrbJ7b1}P*S*zWOEj~4gEcPC#kRcSrjJFbzCz99BO#OLK`j`Z!V}vF%ltcPwVv*WB z*A%3=&?WFHwmkhK1R7C^Qp~rki7odeJJIlYfI&)n5>nUApJgpGzXx1x>xEnyQSUCm8;=*|15e!LyPWN z(~6tXoS+WL%PIdJxJkB5fYZ(~=Jl`J!qC!%fHyuxkGOKy?8$CiA^*SMKS+I*5aL!O zYrOEoFGs`R^CwI1?7(?gghXZoE6BWn?eTDc9{r3L5 zn_5xM?$PZP4vo~GRXOr=4|ZW7YkEd%k~78Vpp!%;9>@Eep4H+>m_N8ou@4AzNoqRA zQ^e;^AnN@Xq6^&+eQoT}QnF*J zV>~4mKkNj?vfFRCEMcXbFydP}~{Te_`VE6I`^cCi`U1Ofw7se6wmMp)wd6 zk+Fl>1Bd%%5dOcz`$p3kwb~=|j~9JJz>+8Er#0M+tu;Zv_*w%dX?_zwcJ>Cl5ZBsW z;rxAz>BXqaBjIKZrhuZ_HI*p9-zjv}6YAEe(f@w+X1)bWJ9{xp9XM7>o*#tyB346H z4N0-!FP5HDx%QJBkm`{dAB}6B_w!RDsbEhLkbi8j`M6NAL7^Px%EU0-PUJJB7nuD3 zhlfU3?#vP%fuv)d^QWS+82o?~#A>$co+hPZ3$czK>XDqDZ z9(_7K^>vX@H#@cy$9dzA#NVe-BG?xnXXuCsoJ1V;5}5tiypOugM&JD7M?Z2FpZh9D986v2)CgWQn3{vYKvx%OC)Y0w{bK9f_>76#tB^c&EW#vh0$2V z7zRm}6Xtp%Q1=$!%*Lv18-KP!C+!E`DLt&XBv7rU<|O3W3F9aEGTn%Rfuqc4hmP zhc}60j>q-0-o_(Fu$I`VPQBQkrB)bDm?&-AlW{{&i- z<+~DFhd%4^Ha7#E3|}byU;XW;5)~;RNrW81wVB4%pZqhc*9_W}nIC2uLOK=Y6&N^GigvG@ z;phMAMfwpOJi)<4RSLq$jQFMB z2L-Ic+~q%hbCh6PW?>ogsSiCBH~W{QbNH%ZCBzgggvGTDlv^D4Apzlt7q)vtax~Co z@H*}ivT5++-k-)E|O^sVUf#uB! zabIzyyAzmA()h?c_#qQzdW2Sc28#N({=RIZWp%*}aj4_4nhL*@Lbon{SI)^h-60@6 zFiTCvw}Q{EdXXSTE2G_rjixh_JfAaPCM%x(Lqbepn_~@ zI<8nq4n$iWpU=SRhC~@4l8~bL?5-c)&=B2;c%${vh?eniTRdd1{z9B4Xy8Tfw@7~4 z($r!N06DM7*r-7WLvX@oHwYnm3hk-CFqYh1)IRJ3GFAj2oKTZi1(cEgc@q{0cu~WH z-`@j&nhYhm{PG~_U8e5hcORH(9!hxQDAWu6)ZR4A{(37o1DsZw^1q-)W9V@i{xXe* zWu5UAB1jTCVqDKrN4n4NJA5dCwQU!oNl3O+$&4#<2~F?tfKOIcvPN;1n|UkzI@cAd zHzO*Gq2lP_9azh9a;}^bO}IROy;XcaRm4rq7Fb~9cZUTBG7BtUNiEANZ)P^=VrdEoCV)d3B`A)%uYFdEK_5TL%|$XlzDrXR%?bSal)Rqml3J5m;&Iinw9sx- zua-*Z zpII@XzA2=4-=c;6LE$6PO)BOOOJ4|i=|BlFpGsI8Q;qFQe#a65 z9==hBSbh`A-pipwH#-xgu5w8)fg|4RDr&SAcF{%0SY*+RVJbxFB*iWTWv%WNYv&-e zXIhD572rvsK7q;G!%)7RnU(1b3?WIJE&_UrESKndv&5nH+1)=#1~Y~eK^=TTLI44% za|YYte}+HgpW*M^-`^Kc-wR>=V(+<^B+KM=)_RzI^EUyFZWAVcQTQ+v&6_PjNYvXR z3l107DT$jsEel)=Mp~VJisPf4rxG9KT<~!)w)$#QdJ%qG#tgK49?~IY|9OdU%aRRu zj8qngR&8|muf<}$!>#HV$pk0 zjoKYCJ$3e%YwUFW=;T{pM40xOO*y=LMC&N|kPw7+!u~UuNUQ`#*2yV1Z}l9Z+97fbjA znXOn3%YGVIUgr7{48MqFE8u-mzT--`nF+ACM~9RJ(QN0KSR!> z(Rq!0DndofBb1G_WDfO(@xl|Gl+Gw9y-3cM5`y@S^dIdp#(-hUFd}(E{NRIh%ep9y zuf#J(m$|JK&}%)1jo3tn7ic-2Qy>9ud+eo_V*0p*BRV2KqVa7EjUL26ALL)JGDvtYQ8F0+IZiY znD5pBCz~Mui(>VHXG}v8UVbt79qKH`mDi0$ffk2SMq|0}Z+zu+HVmyup-4Ia&f=%d zoWsIOv9Lh+J61VS@?Jz9+%zq~#E~OFD!;8iw){|cM}$>+w5m3sb+>(;PhTQYCe657 zOJs-1WotJ#@GDrUN{?Ty#FaXPw;lqSNV^1F=NGTGNm8 z%ev=S`L2+|(-D>wQruhuRpb$X@>7KWNZ=7AvwylKV!EJ--^u<&UJ!dBtBK3jt~VR2 z&g0>Euxfk9F0QNZ72}PY^Pk5E24l#0hgUwPj6r3B zyTiMZttB{z=DsM7Z#$!$df6iywvKS=a{6+8AOaFu>r*tH={OEg0DR%b(Je8dGrALe zA-o87{&I^f>Y?O4Xh=rj9reE?%M5Bn%ytam;;lB-6E(ZSu-1g>!i@y03LlAKju?=v zlIoKxwGqaDtq%Gm>`bbKX8nWZ3D#6_;8Z})q#!cEghWzgSjrIJRHCyZgna%pJ-b@G z=G~1KHdIisLz?e|HLwYH>mdGsze7^<<<0~H6P@N2fdcRQCncOn(gdT4Z}Myq;rQT4 zpA5jHFSEts@tNC>$ezcWe$?8pbJ`}`PKbxAIk#)Q5Etf#ye4jf9;&pu{gLqsdd;{E zt8|GIz~8o5STk1#A_nft*>l_4%5pEtam6NHymT6O?s2wF)+RdY{0v0<-83mdyP6 zXla8^RTVozZ@(9#LPoYbK%w-wb}K6K3%YaeiuV$z!_CwG zttYtAG3LWn12}=aP=?dKjM>Zk2t=N3OOxNSc;{>O8q;X1cpnFf>bsps#zZ!LpmoGG z_>c;P?FqAzild=BU8xM~B_Jo1p>f1W21O*%lZtPY8i8S?5GTRTCr2C#XUsGew238& zXHPBdKP2pu%;k`ih%=${d^I%zB6NmD%*kA3B?M$ifE-aOQTZU(a5RxQ)N5I|MB8Z6 zX$W+w9FbwiEmpYsX>iejxmL92;b)=Y3lf+CMha2XW^W#56sQhZ4=2?g2^?@(2=c@? z42-v^9q!E~3OSs)ODK#1W4`{wVG^tt$YJ@Rkx!Bt$sZw$Bp9T4rDq?L>8LzYI?2m+ z?h7sf@FmNR3p_hD7BInGG_K4om!gT>g&<9o4tUd38RY|EJLCpVk%oX{xtr}$9=o)5 zNYn?lVUW!Dud-L2Hl3~Aoal#qr8Cl5Xp0wiSqswJ#FM1^HO(@dm=VC(C8o4`Sl_n2 zggl^lyIEhTl&@c|URCl`%a$2&tb%w7ZG|xf^@81k7N&Kt~G(|Yoq1k!*6r?m^SdQTO<&*mK#BHgs?6_KaF5f2p+4* zYdR@PT3uahWjBZuxkf2B87nYZb3%|b zQ+~#oKEg1hoaVZ^_eJ6qIk01oV}a>ozN^-!Od_I>0lk zig~1pc*&&G(|g|M=d3?2tZ6@xI##*&V`V4TgadUg9k=-Dm$0 z@^}WL1CinKa0Uyq4L&IV(@A7OAWwMgbO(8&PR?$%LT`pmjMi~j^5?UQIF(*a3ZevN z-l|gRpCrZ5yCE*ezsP%OXc>HOn8x#nDCmJwUs%x;g&r@2pPAsasHX2Wh36beF@%)a z&>YH%;!e1i6`}ZHN0F;y#;7?UV5i#k?gS`|i$95??w637MtM>L;v8asB5%nPMiM+l z;SoBq6+}`2gJA};-Beqe#*A<(R zKV`BNsK%*j*VQxtSGv)nR*G55T${AEjx7Z&nPTY_yOhod@+>7x2H?Mr!Ja_2&3VR78JpZ{IxYj zfsiI9uCOFzfoiS#t9nwe;9{GwHy_uKHcSpE;6U@a`1HOpH1x@!Mpx?(r&o-(z<=bo zCt-NC^J{SP?VhWy4-S^Uo*ud1ksPtlEQwn=xGV8CmXHKoU?0;9xIWz}8vFyk*?Yhk}6 zTHRCz3@)-#zc__ut7c=;cer(V<&%OV^WkMQH{Cm^8AwuB)XZ&8Q&$yM4-)yMii@?w z1gf?8&=82kz>*LQZca7SBDy$E81TtnITN})8(@VHf?c8omUXb`5S#phJ?(F?c^PV| zqTY!ZYf1^x_ixJ6Tb>nS0q$Ls=}bQBs`*fph%5FS0a_ek77 zH(>e#LC~D#YGm|5A))pcKRI3`NUP}YGJ0ZCDJg9f^LURUOW*gB>qSf>^;?#(okXP* zWPm=bO{l_jtBm{9%SJZrzmPC3;wtRlHgiFx3D590gm_5Z;D}|X4do~J_iAdL2&|DX z*5VEn3<*UuyyCYw#Lig#*{ozrOpkcDC@w?n8Ne3mVX%F1oi;c$kgs57B)$U7&` z9>ddMUo4_KDRr9u#2salUZc>SmO?iEFa$QbVq2Kvpcn$3k@~UqN}k=}92xv5{@~+7 zs7uV-u429SZhBn4dEdcTfzuQSXIxjCARs)mHz{?+eXwT|VxWe;S7;?ioRUwd#eIX< z&4Hcb&5BmXTv0c4Tt`Jhar_#W*jjD5@LLY4^{{AuT9OEH&e|Z-{@s&noT-pFG7xF0 zQ#FZ8jd2W%R@YFxIBz{HfkyUyP8h3I{I--7z3ongUYs9uXOOW{@~tASPA{Q&vjLAt z#JC_M(35hHRB~fP%0uxwBR^ajSrR{5crMV{gnsXIg%Lt_>PCTtnis7{lk$| ztTf^;5~Sw54+RiDYFfi+y=NjEcF8R7A6=WK!hVy`07Zr+GbSd!KRT)~8 zpJGs{+iipywKz4eZ}i^WCEr)$7XBK2m`|@rp))!*)_ppaC#Ds)t)r)=0^B4?O)CuX zHYQr~5g)RhrbcfN@PgMxOBFV@F$%#vT~^atYmDmc>nyPMUAf|yzLTuIG z^_Y14J*C`5UFgUAn@<6}<#oHCX8ewh60=)r9CR=L2&RqPcP_TdDp>lHuA7!|9dBTH zVc$R9t}|XgDe^53+~(PK9ncQNR70mOx@Yti=#)Kbfo#5Kpiwl2`O^Q=KBToQjoV=RkA1BZf;Aw)mTlVzsK&rgnXC)7&+z`$9&SUQrQ#1>rb{4aOKF zeUsg6lrTdo_{9UHGSRle)qdVhYttNTfafu^ZM~~iNdV3lqgb`j#J}>20y+#jwwFc2 z#_if5wPoiaAPm{&L3sJnPz~ommX%~9ZUJ1=6rSP3nmI)fBVnb6J+-hme)|h9z88K1 zej(aU04*acn`ICxCXr7RJ38udc6*y|={6f`shaNHL&rpK!(y~=i_=vjI6gVqx?s^u zM<)Y_F3yc76sjAkwfphIfBx?+OamqqL|_l*ku_lXO}MkzV&~n)!UW=^nQf1oEg`PK zRiL3q*pv|bSc^I2R55}rWDslOZ*d{!GAFtWUp~$5eb*U&ku6R8mo~=!UxkuiOECx( zLVY2QcPDQO*SEZ66f{ zhD5ay^*;-0DXElfl_x0--|3V^`U^FikfW(dkqJLVr8IKnx2>?hZnZBYy2(%Fml$PG zHMu5e&ai7k2&>C(ntr^3c&=JO;**e_g>2irEf{L76=OxRP0-M@nX^;_@9xgG-1i3B z2HoJdPv-HvfAJ*2gmT!($l|+-Kq4gf+kSG8?kjF;X_cn`3?=3ATUOMroe))WbyfG16M0HTIeyoy z6_-f1tJQ_%Fu50)Gn*;6II?{RFk{*%9K-!IiV#|SgkByTFDLvFQBegfA~0tmWJ#fD zUOoq@-P+MVcYO!Ssn};}41|I~fFro#^hAhGZj$pmjTaxgQ zp?f&R4O%*`By`ZA5ozk;mu;ke61uf-;5+-qx=`3du7#D)&Fq_H&ChU>Mg8wONe^5D z3q&zuil^TX6xAzEU0cH#qP9D%Re*0W8$xam9b zB8Os~@^~<)#k;0;0wN2ZLjcRqr&jBu*9V;2;_PaVB4@j{n7@sN#O6ay5p4YS7;0oK zl*C{S7G4w^#Wc8v?yJrhNs4I`6c3+=1WyjBv4`hq(P-LeDOeb>U-M)>%rV$o(z`Ul zen20StfPxiPMw8M%W&;pU z>MtNF+RuLTS^ShW{rUY4G+T6xrLv5XX!swhDJ8H69k;g)g`J z=+m~MFpCY9zUS zlcc5EA34O3lPm$Q(rj2aVxF1F9M8U-i#lTA~^?nakdg)RTG|QVa2%LA?sl??|1N+Zp_|k*o9cmP&q(kXK9S zF5))kw3`v=WCTlD`6%nyczPft;vgbY!lYQT%MomIGo4u$FLB`iMjm}GDCAr9#IgJO z6n>kHK60tkmv#ixnC(F`?pRk1dl1ncU1^m7J5~6EYzV9I#BMfZU(AE5r<8*#c%IGc zVW$fqXq?Wt&>WkKkWn3Hql0{#nh@Y5aYTMZ?aDeAlvkGD;k2{=*tdzITv!O*4J?=_ z^0!P4!78ch=n&pz&5@Aa0D$7&cmRagrn6sbqj2?yK<@om*q-|v?8bK zyLIm0;ol{ud6tZCfEaprjdzTfMSj30aU?MlJ_~HBzK)A|7ayYgXTy>JEFcem+-5lU z_AyRboNgvR!1tH#ucRop+ zxeyT&4muC4_}b}wH6RYA2=)cp0ln{Z{e^pX)>8nnWXE)*(1S+)wMou#3!d~cfRRRo z&sWF*mQ3X21i^#`7!+wNS0;>{33kwQ;3q_5E6ntHXhz|fk}Hx<2nQWkNYt(SdL>sH z%`jLE(pxx_&4+Td!e$&s+v8um>p@(PW9!+wUviz0iz|<2nhFlK%Ccr71O#mRI6UFV zQ4oFI2zQ%xhRzYEt$=1&c2|mh63jDOIz!i>K?f^U zj8t|s&ANBk$Kr}bl6?UW0U8B83xsDcDH=fzf?q$R?u8JIq28{+WbhEGkzS zY?)4LqHchAv6PQOi`U)BavI1<)<^)}q1+x^aI8^xLh>>`In9&qlGcN<;j-K^@3i8? z9qJJKcSZi;)3JRIX_izN!}reB+AezF=Ph+(2cNS6ifV8T=ygPQIw`@L5ja3#F5h2` zTsEMe=w}+BYAlX;bNQ?mS(_2v(Z(n5tQdXJKsXLvo-_0{ROmBBo-m8>W3V@86s<)l zl~iN=lZQnGW~!|UX{K&gCDDOydzmT88`rLqL}skKmxq#Mt0Y%BRWi`rM3bYTEd@w75rQ-1SLSQ@Qea0a8Hng%v{?nNP9#s(lm!hh%Ao`eM?-;SaAtjidREOXJ@ zzR3@I#!`q*lJMU7H(r6io=M6XC+9@QEnDRQI(X75dtZ(_eMY`uw%;cMUlSE_wFB0IqD%C)aUa=N&NJd*sJcu~LYB4}n1&ymQUyP5$c#Ioe1fOBS%JhRg-b&Yio35EnenW8tO zJ5COohJ+=1&Fr%#ZVTwJ@jqh7SEuC*sp#R9B!ABN0PV+d-ex*7w24N69Oc1@D?U$pS5dv zybdl4hY|2qV~x8rBr!3l*y)aNq5&ei+5h%}HJoow4jGw5dWVGVfD&gRhnUupDo+cV zy>Lf(D*Y69>d(x6l%Axr)B`~H_dZH0f57jpdy+S52|8ou015m?f%_dH31kUA3VbCb z(i&k(uV{pYjG~500WXn)Xm4Y8jG3rE`u9RVwU3}QU>-5HDN>>_QPF(iBYe#(T=9)* z0{#T9ljLnc!-5?DX!J`oEVa5dc4MX3sa7*HeYT_`U(qIPNEJPr=Enhddy=jIB?1!r z7!vT=i*t@!^_zj1K6;*?am?kP^hn3=YyrL8&_3If#V&* z3!=~@A*20W1py_w4HsgW_7ksBKxFjThI)HQI>6?pS4V@{p8ZfGfMEtycPIUrsL_jq zrEfz<&>qshy)9VpA@eKB*4gobN;k` zRGY9%H|kyMYTMz8YY(SriFAcu4+9E4J5A90b5ZC{5pm|EwURXlAy@~cJmu&GZ15zD zg68WdjL~jG6doxmAb_?#m*v)bP2)yj%oMP&AO?TK#)s^{lSg_hUJ68A zLkyZJNG6ZLq$ndEQ)qS!`~FX%0jXQyz`qsBod&RW=UTe5QqdgGFz-|K}$0#TWd5w_>8u%uIZe&u(R4NX^lBqfW-7xw0AU7^eYNk z*$}F>5kt8hE(q$oqxg{9kVVQVS-}iDZM-~QA99ZV{}23^65(xG(K=;<&9?%EE74;1dbU)1?akpsEVCs50Z8H0t5! zR0%Zr{7qe+EiDp#C=@vg*8teO(_1v;6bvDMg5en^C>vv?A{j?w{L%^3c)-6E$Q%Z)I(DwG&4~UDR{%=?6IK5U>vqM! zh?C_6kPA!N`;s@$LB*YugL(Rra&h-`D!^?;@m#(?aUN*DSa!}JcufB*)P^Ln+cGNk zX0&dGe2YVJUGk$bf`d|u$(Rqj8yLY7wrs^H??_o4&0WXjr zH0^Rx<4XC0B{D-XWHBKXKxL!HvnQbm7|i_}_@T+yx*~u!6I``BbGF z-)$cU^Cd9A2BcgztdV|iytm1FyE<4N9T4LwBA;{&MU~)#%mjXn6{~M1=H89n9Y%H{ z-ZBhof%}$Lc6`M=9aC8C>9UvC)Wmya+xg$+1IbPyKb#yTB8sog&Yc;`T*!iuO&F7x z`(=xtJV!PVOHZ1UpdG^RW0@ReaYzryq3*B&I!mKuan79NxWySZg-< z8`M6f|KCxL#s3fvlPQ8+8?Q?yts4aFog-cbmT3QeIM_ah0nGCXh>)|Sn1z=1O?R?) z>2ZCNVf9b+N)ERZtG@6!rt2lm8;;OV*M0)5M$j?_)50HYR8E^n+rYPPE6u@ z3eu07RPz7iP6THo6Y4QyUKf-t2r6{IKG68=E-h zl7vl+Vk9A`OOc(0l&RTKabCd~49zwNVZ%#{c^BCtoC8zeGV6>__uLeam2KwCN6UCk zL3xYkvZELPF)#zZ0w`rdkVj*{GPnWXlS(w*eak932q7(j`fqM-jOT7o?j(#@#mfY? z29Wv-a@@sJv5g8WUVIxSf@nPPz2g5oxh5_+j&ulWpg&--d~#4;$Zy)xdtwfN*R7qn0$bs|I2umtn|tP&yRNgF44$*s zqA7Q8$~!D2eX^gxt?_)3f;4XXINUj`ThQ(4ve*M=`U~3?Uib6SCuT)9g%P}a3i6W8 z_bnPeH~X2SaqL!fh5jY`AJ+zFHaR7xpGqk}BI_G7MX0VRtn0Fie@%b%(2 zpY81p85^WPRs?dYbN;${_q2hBra1BE+Mb{^7%DM zyfbw2pUO4mf}%hS*jS4nmjse3?7)5Y8Hi@ zqjkXtLCfE21NogYqd*2)QGt);SlswF$5ZsKw+B5I_15*o!v&lTueX@k*t1CzGEF^O zV4l=E#B%okS~XOIx~dGI$WMHD*>vj>R+sQ?HJAor>(n5bS`KnDAG&Tx%Hn%ax-{JZ zYY>_8hVp-DYQA|?@j~}O1TN%nCN$&!p0HF15S-7!S2(w;w?QDX^>hU-`ry?6Fe|;_Z)MK1mJ$wyw6u}NBX?}y+F@r2;?P`Muhx|fXkML4SnA0$@inEsvnlY zk*Mu;bsS&S^Y*b5Ydq~f;?E~e3=E7D$U|}OGQ2j3?P+mhV12owc=+x}++#2@jqh=FW27AMCk>sXe1eSPVB$S6+qS11Q6 zoU?z2rUkad^nYdI<*>fgf>eQC^;`6Lh|S+`4yF;Qm^Br%G|TBoVrLM04>A(YxWA3u z)Ux9OT)$Cr3ab#wje{ViLp@K}i`olziTSt)9B@t!cRkshxc7YKY3R0Q=)%vMd$sA4Ea(4eV!^# za&rH&WY7Qrw6@X8uoUY|wvx`SO~9wWM~Dgq=acCe6sX%%N)Uab1tsSwkMFn%p-lY% z**bH-mAvRLAE7A9n`Tm0DIy z39^hJSGd7jT~2eNkRR!g++fz(trp6xLFSr&R#*ZWawRZu=I@CG`g3y293sS^ZEa1>8CP2>rO7cZ zNYMrHY4(NF=n9noKos-hoTtbz8tP#vY58@iaLXg~T)zE>lQ+w!PuF`OsY7zy4l22r zv1a}Jpb@T)vcgv8q0ZX<0aT9eFg2(NiRuF0pEpXjgs+&YtgYd^`*;cQ42WS_`9;wi zJ)1plRR_Pwfnx7`{J;`B_YHXL!eRq?mHP&-_b0OgqW~Gp&;PyPN66Sjg`Qq*p3ajT z_}}Ckk5c<~6u+cI)vK(o(bbFMk*fa6U1A@p|LONlX+6GoE>M z`_1Kbtj!G~5xp2IWtAF}zw%J;Q_wT+Ay>t7UH-aN?0x#4%VnPB+j5q+K-#O9wpxEo zPYJZT{&?c8v9jun`axh6tZ*sU%f2-O`cOifY4cVa+y%cjlYI(wU_ah*EuAk5#nB_I zzhjl1yBWnTIJc^&ErfL;6eX4vnLT-Sj6V4L`@xF;ExN@h5>i5kkNG=6e2q>g1fC|n z*d~x~rs>N3nrCBzkSxQp!bjL7QUL2QFkVf3Ae6@CKG$;#BW+jq1r?Nv*2C&6uqjoK zh=BKwNWG>W+E3~WdmwxToo@bdgF$%KAw_FpR32lyC!nagFMX? zk1!2XZ=634AFi2*INo1fUJnZc+Yt69;#C5<@luz%U$zcnozVLk}^7VUr zeYR>jx93{FqlyzEirL;wI|3vq$u}gbE178yIvvJSXJ*Xxq7_~88t8adpo7BhJh92^``MYr1-Rd4|_>;HRBKL3p%eY##Xit@Kj zNwSZd$>99h`=Xg_YunG`|2odqz2%f94EDxQbD?#Ht_h6b9)UqiuFH&jIN#N(gAZFJ+ zLtH8AVHsHK%ZR8EO?^K-jYl}mXgzC@?m1%cn3 zT5QjsCPugO=g`nl^G9y4ZP-AFD!CS_`2WI%hrvg1_o@xq_&66E7)i=k}d83BJvlp=I#+dZr$mO(HCW%+6G11OUrHFNt1c;1ZsMPrQ6K5At`!(F{a+U#CMd*$HD z79-KSLx;JfV1;sNGhcxNA^8r$TGxN zH9v$`)dPobQewfbeD5#UBf{rGH|bfgCuYTIoZq={U-5r-+=)rvXq`hRmZTt1) z@@du>OQsR*5P028XF&Ld60BpbS6>^tz7V?hY)tXJOt`MJtSv9kEqbXdD-#iBB!2ni zuqy%bpAl|k7Ny)-xVaI8Jg-Q@zGRr^;4eWuwJrs9rdX47aKZ^cFNR`@X5|nwow$x3 zqz$R+>(p|^+!9W-uvB~@>Z~>;jTfNOiRy?@cn;p^%CLLVDfqAB^)(hHp@q%gPr}OZ z!=L-sF3GnJaKuT{krBw7LV9Q3i;R~6je|Oc8h0>T-cr_6@V4>{=}u4S30rMTsKn4Z zgofBwuaqpmMkVSJ)sGUPkMpSKJ*Fp^^j+dW__L3M#QD@ItD>-l$yFFVx&E|w?i4pc zMoH^;l|knr!mDvb~v zNcA!A3$bs|@oc2eNv99>q)^|zTuJzyG}(uqZe-zSG0jHnDicBaeMtP%w?hq$Fu_na zL#(zcxOd2qpwyCLu$}ZHh-Y~qz{bv_a;Xq10r5>ZZ>G!R^ztAMG(fO9KI`dzd}mSW z;~h=Hwl?>}v)7R_?wvT@c!N%~>8z-rAr5&?!fZncr-YheEBX>UkNHd=jJ4Zo1N(}+ zQKoV2Uga~&FCybB@{D+d*KX08Z)q@pRq2B8{C)!Hz~~e7&;MTj z=A64!fW1RCj0j8N6b+FbgjMCw+K!t>v?9-m0{+2I7z|(L(ziYEx{*0&Ho&G)>TTN*UgklbI*gFlwQUT5t(ksm? zb8=3tj2~o~MPCMVnM*YRkMg8QJ-pje@9Q8VGNo_~B_bugS_`hL%{Pd+Bbhp?stse6k;eJRgkRYQf3k-Qx(`Ie}jpe zVT}U>;cO$2BKfw`CgM@9(bGlD&T#)^I^#l4JfDU~5qtd@!^X)%uHq(aI6@ZWLiLZ$ zNQgp$?K=D;i!yjYiHUHDIZF9IVFVPFnH*LNcreab^_U+lPC&DHmY({=+k_-JS>fJb z_iT_B2uGv}T<8tI6ppB<=qYE8J}LvN4?rtGeS8>4ti3K0rxnsGv2VO6Qp^>?Gzl9- zpr>nsUPPCzODjV~!W9bA*e1EdFz{>Rpc=wn{J!GDh*RVGvNrgMo-%c619_qBRe!VD z>LJ=MEGb^?b(T}j@1yySi&)d!M;tsRkp*2iz|>Y5#zl+VPBl+AGAQPJfV9`;UHnJc zriFGyoqI17e}$$!rVZF@>Ixob_IET|z?T$Z%^{wW9Kg|3w$t0NZtTP@Ew;4AWV@TU zUUTDP=|H1AB`A{VJ)qC$m?%Ln%;|hrQ&r?$W@0|5Rt*YUuh>w>6`FF%RGkLJmn=5| z9WK{_O?LwjrKg8IcVZ>Qci!#2Xzbq$Q4Xq1($_CJ){t-as;$rP4=8xUF7I_$PuJ) z>_lMns(&+v2Xb}6I*~$FL79_6*RlEsM;E-U9tfFJG?p3-gwys>?9-LcAzxxS;2}t* zCg(~DfVIwUt7RUUknwVIN}nKoS-&Rt_}bt3bX2+ma@V(oM`*wQR#tmw+%8QAZqiLu zt800pmH#+5dNvA1(t!T7Y7S^+8tyOYG2;?szpS^uFUn8KZP<3cUM;@Y-C?m&ZLLJe zdYs&dM6nM?ij*YCzZ@23El_9%RmhhL8s}|c$!uGMRpdfA^G4~lO}mlr zdF@GK5e2J=h!+RpceZ&S*2w}@rn66G`s^2?6 zcGf)tFNvvQ-8>z-EK*Bb_ci-lMtMTEC+l~K?uXO}d>be%FHJ2uZN78>T|MRubhC}c z=@)&an?8-SH?gOnMD0r6gk(SAs<&?v=MS%J@hTXZq2%#UdNeRU#(9B`EG}-Ws&8V_ zNu4+~hEziHmC)&<(NZ8Ha@KnsRJVSvnPHK)YyYKxmg#Z$Z{B9FUX3(`Ke0CSeZ>VWa_c!b9d{s!sN4MbmXS41H#JP z&6HnvS}&@*O*D`9*QN*C9;r;?QM%HdhCPox;Y`Cyr?tZc20CWuUatu)SxG`gw%Oyov@A=f(AhK}pO;K37rH^6z6bBe4Exa>^G#jGR~o-x&h=FH#>=U%GCJ zuIt9jn_2m|jR@7p-wMZ)fh=APYRBF)sezm^X1OQAb>Qc|v*Ql31Jk>tC^k;!;q+W@ zB{mgc9o{uE7H6ap7g@ol4Xc#-dfsr<Zq)4XxgXRSoi+U zqbLczs8v=#ii_$BUofUTXHT>f`H@AV8PCW&qw0K{&(}zs2t~|Q--@8_=SdoB)4clU z=rQt@``|B=6K&WgZ^evzhG{G77YQPz*0Eyd_^J>TX}fx$p65bZ(36hiNxr7Xzg-jT z$K!KIy584HJ2wD(Bmlm87yS%seKYS(rR-$Yh_wwMZOn(piG5RN-ouD*R?!-{oTQKP zyS!jM$k?kK#XIx?)9qW2bQvfI8$92=vY0J+jYT?bT;+P|2sJWkGJwt$#@*tcJ;Nr( z^^F%182Oz@`tCdL12aPSBa_4IX2M!`FN5y&dlHOyiBsfi_dmU)7OKze_-`DVnUVk; zQs8Z`Vb$h&KiLQ3)DNt~c<=NWHdp^aWaLwzdtCx31oZvwwNXM<*zZC9s$!JEUW!_L zB6&!T+9>v0OVZdRfaZl+_Wd$v-@rF*DRY3LHG&12FhoV{ViY5NXfsgdOOyfk)VjG* zD_-1*YIf$RMQJLb{O->-@KSFg$p%*17~nHSjx=LXS|Cx7X%(pQg>SCioF{IRfy+ZtBd{7DQ>rdgD(SEITI?_L=KUcINnZ=I{Br+~>< z-!5yDzr*M*#19NsoiLoQT%C9$OQg>jx<;_%qt2^X;ZjR@ zi|g^*WhdV+89f_iy0NoAH?*Ig?(oCWW9mz=%E{JwHT~gvrJfEgsqB2&HjzO>lCC>x zI4}}oL)F?KLae!scsc5-oR&$i0peDa%;>K|gCvqY;a^#&+gNzk5UhrwLjD{Zzkc(% z6@BZ3Pz*oruPqjQp`QNbW^T|uOLNRV*%O;WvuD6EmZ`3(_^OoC-y`tAi37V@XUI7H zRdOSCfsl(}DYY$~nggY4K?5Crhe*)M@S1up%}RDQJsOV@VIlMzQPCRd41khSewWKy z$dZWPlOE5w+ZJ^C!yBg2{kB`mnDOsm)3>X{L<^q&g?{a2((mvf3P-vRLBh=@ljZ3g zbCDdG7kehtkh8Du9M1J$29<)`r`!f+y8uX3J^B9KIjwY&yW-s1G9mcg^~v*3T;IWp#%_ytHhe5y6 zMMuY|qdC1scRn?h4JAHOB^p#zN48e$FeZ_Q6Y-X$*6T}p=%&E(F4-IQ(MEDEIY7}F zQuug}izzs(z}x(qgqR z7b$kR_00{oPytYc#rr^d6`bbp^?w?vEV7_ z9&UvKKbqk{LkbXtfd$9bwiEnJ=Yd1;BiHA{;?v6u@DhLsPB^oPP_`RP?7d|CvkVl& zvht^P^S2^Sf(yIo!BtCYMB!{Gp{wk9d_kL`Ei)s|mp=x#1HU8F86y|3bH*|*L=THw z)@UQ)kp>8NsA5crXlfihdWRj(uSzTj=Q)xI!d7JQP4yiJhmr#vfIk2C)HNI43(=}0 zuM3qbfPRFo`Jj4cajgQWZU=D z?RbV>1_I}irOY|ix>NJIYc+RXk6Xk*$VVHG$=aC*k#AySP0Vcl)PsZG-_)11#CDw| zyapxB2R%O-XEN1<^ENKE$Tw>DuA%Oq(P)#~g4i28h9Ji}Jl*%oo_bFK8fx3)7ozrt zv5z7C)eb(Mg55|M0cEf4bwdE!gm~I3H6lHpQ#hpwNiM$Kit$H#dhbu z`U+i`5sc>IO9(G{c7s?dx zS&Oj-SGbgHt79up4VLXO^Kq$s{kOQj&O~L^3FMV`39r*EgPX%Yg~-AayKO`RjNW=b zYe#W)xgI=j9&xzOT>AQ`!-vlao7aE*K5gW+2Bn_vs^58XsaG_ldTuv?LSQJ827Grr zEzZ`Oo#jH844qi1-s@8PlBV9JNXJH{K+(8$3E1-K<}mnC(+Qf6ihuBXgD`acPo;Mu zSCaB5?{8H?>Atg-;p^KSp5`OVRM#u%DrqX-V)Q5qPb!TlLFj$}6m1ijb#GY0arWI4 zatpQyWPQ6&zTPNuDtp4V1IX3m$LJ4l?nDOi!q|AT(u``z)V%;&I&h9tVa?u>y_{*h z6*nwzB5l9pS5h+YOzq8WkDIf&TewuB_Fet<-e`C$(Ej5@j4mR_F%U?xF^X zx{~;lhasj9%6y8OYGbUYiX0qiywRNghW`cWz9D)Yd98IJxFwLOVFx7B28rbb^(RcA zh`UKSbs0v&Qmvw?$vkBpd3c30A!2V%&>BJoCPR)y_S+ayiwSt5hHoEY9u_3Nsbu}J2S*()d+M4w@L~X-wjdYG5 zd);E*$FKGx=?$x_Pr!DYCuy^&N&~02j@pQL#ft_gWC3a>DKUbwiYI#2vlnuvZc0B+ ze_sTtA6S^Dq1Std=um28OwvZ|s!K9fpQ-xQctBCr5}CL;^W0prZ;-+RJL;^TyStMy zMrHW?{mMMif^JxKL$}qlnLcz{@n~Dg^ffP1FWKj#%8`CcSJrI4Ay`euDEuX=UslXq zss5D+UKn^Uru-sC-m^BsDu{}N|M|NQW_70D&@(hyAGASeSWx(qS0yE;w9x=Xei4=+ z4rs0?vd}^E2r!~$K+fQN^?b0~_{?o$qOnhEcT`+R6tpr}#MpKIX2G@XEuWAtnfx>( zqtwA-rQMxs`XTr)>lxyt zoZFOnCk~bc>5WpqyUJm(DS_T&?}mBFXm~>bi7ZR}LpaO!Z5fY7IrhTOAL@J%JK_5c6gRvKBIfMj zNcTO-Gry8{us56=-xj-+E^@PoR;meC@*bG&CZ3v1fA7y9-TdJhijqgLPvc#yGP8O9YcgUaCa8eh zSq*8v+ixRh$4>@u=QiWjHwY&AX0@|9^VGQsZ?<__E_>2H&!Eko8t^=3g-2gb1i5AB z)ciNVJHh0PwXM2se5i?tr z`JH1}i|WqKH~4R`C(mh=?~+TI4UI{qcnsn0#Bt(toAOI{7<77VcJWD6Hu@Z0o$C1*Og>y~8<}%@zTyB$-7phn z5Ao(>ALV`QCv1N~66`(NDZi8$;eH6Ho3~Nl+N6D0|Giz985f7XdbE`%4GkO5u|hkZ zn7zS2QpVurXF<(P=%qHirI}y;QxE~oNBZe#>xseD5_Wh9P>R4LI!Tms=(Jk1jBecN zb2kk6ycT3-RHDXfVda!Oha0 zf=O5Vb}u?daB`E=F)~=wNNMP!`}UuttXXa>Uy#=7w~v(sC%EqOZk+r~GRLa~9`80D zqlw2Ic?CLw0ptwq2U<|<*m!5}Pc*q)n<7~g|JYD5hu;0%rd}6kmUI<` z-Jw8Q7_RY0*J4v1cgQevPlg;^^z?plBTS{UWMhA)=%(ZaQ-i>lSjA$!_~9iHC*0(f zC*n{R6KL@LbVs>KkRPbis$Ii3M3L&9sQ#?Lv~G;$2uA7)$0}QcukfEyMqd~f(>h(q zPO2qBNb#wHrMXVvJz^9{VlK3%O;pSDd5e>38&Q}Y8O0{{GquSbxnbcE9tREehbLRs z80EA~%bqk_&YR;<=D)bF%LVaMOeWA-ez<`7XbIaL;ayCLFhf`8g^y?P4!I4vV8E{|UuqT?20$gceip{_DeM?A4OA!pHm;03f{ zXw{xHouks{@;C>mPn)e0>T=MP@XnuzB(ps~X+PnVhJMu2<8PP;ly+_x6wyEHhhn}BJymI|G`}| z=kG2C_VSt$J&B(Tz{Ym{*=0N0i)5%4qoU`^I6$V=LbqA!mi>L_p~%r1xxRqHp$Kts zxiP5x0vzBsLi_`mn?+*E^#UImfIP>H8D$OCp@%d)TWPm$Kd58y-K6E^<;)<7 z(P;AaQB*4|HDyRPUBxe;4L}&+KdC;-K)U{GcYg=|z8p$7vi&lQ=B{ECM4w_Tm{-0O zb#}v=$luPu#X|4o->cmrT(+z6U-ov+RAxrA90n>W4YA8kT8&mvkkzQI81NMUOyD@^ z>ApV<*IvOE?=~{s|D<$vzR#T!HvUZq0_mPeH$M$Y>Wb=AQKTj#U2GZVaX$*VL0nac z!Pg_f-16tkm+|}_vE;xRASV4%6*$KtMUu;|SzSliMD51Rf*bYO-?;_VucIY;=g?nZ z2NK72j@3>1JcI*;CCv1tvWNf^7wIGr#n?g&JOy^V+&Q7-qD|0SPb|}7utgoR>UU+@ z97MlU$OzrLNK-dw+2V3AH>}-s8E4Py#y6CBk*%4d~@_U_c^#Sd7cxy1&*9x)(*qE}D zRa_~Ar0Q>V3W`QJQZ0kF{1W4%&dUsJ+2Ua9$*9WDW4YnA#|x9hbMMg zA%FFP>PlLrYimwWe>{}~-}`qJT&ZOlwLeb6P!0tnPSChm@N?uFMBFLAqzKvbPK!fX zDXygA48a|8$j`0zuYD+vd=(`tn<_IGIo;IX)$bjl+9Y*gdB^NCLZ~`>*elmfTL}Sw zl`j=>uDp$uWPow>Jw;iL)XE4-32a<@IS>7J<=9Mm_>~>cts45f?@25pe!&N*m= zwux;Xwa5omyVgCgyEVQ7jC0ue$_h>R5kG$@{`@i4{|zXVyX7b<6xwM<{C7q{0d6X; zI^f(-dI8fH85XC15%~ZqTOf8AvCehTyD8tjyJ$lDK*@s3J_NfXDo%EZ%YrLwP9hqF zlC#r&-Ce(lxf#}NBLcCE-tFr14)+Ew6NjDpeGGB~feDgYs*>EF5AI4U--+s~2*Qf3 zuN(nTzciUTEJWu1!P6Ygn73fUJFgJvpV7Y2)LautZ%LGZ)jO1wW={$#Og zKZ6ikXTj9aS+4OsInTsem9&eVH{v<(4)Ow9=CQqf50y}!rOk-N0)+Dd6Q_-VNLNaN!&1ICM{A; z=)movx6^S;{eW+8CFY0-*j;tw>-9evqG^1;DoVn`- zLRH6HMuQO|WENNl77yx+8cX@Yv;`03AgvG^!zoljNUe2pHgitI2Y%QE3p3bENa7M| z5;=ds0V5iiCT>KmCO|lgl%DTEihBsxA_MO0$!pM=x z$g0=w_{aHsp*88jpdpD7)vYmH1W(WS`DK=b-rV)YHGu)9{f||;7JD$-3fF>xkOOolXJGc9(CKAtxFcB z*XR%Tg)oLL868ZKHchx%#sI%y6}+8mDh0paK1V18YF+T#z%iVALRNV&l$Mknq1I8j z#0}aS2&<}MwFej*LMlVL2=3yrL{OU}U6Us3PBbRQ>gZan#=t$ojvBIz$8h>>KmZV? zmei1AE_6v@w_=-e6vwu~hvSFsOv^)E*I(2*sSb~pQPw_W=2JKIMFD}@98}Ae%8_!N z!9FVo#xgU`j-(Ip$=+Qd+D1=lw}deO#ZrjGZ+XdIX^&x2YxEL9bKa>vU;FmW*&1nZ zhfTRC&h|-+6hD%)xSTBIPPN6&QyE@7^xYiJTIhGh6TuE0op)%F;+KWYgfjGmB9Bq> zO8jJGq^Fi&G)I1yOa|g!G%Kl@GJm~Oly>8xCFzB2uISt@n`KRp%;Q9WxKI$ytN@-n zLk>2unr!@YU+9awXgLR!Vt$PvRzIC9+%H1C7K(b!a7x;a`wh?8m!WdMB3qKl>%Bd# zp=RyUB@Lpug538!3L}hOGAt6%RWWgbkIn@ZaGAZjQ2i7262Pb562h!&;V#}agUlUi zQv)dLzi6w|qjcTf)5pl_S-8xCsIFHp`p_MQl!z`0CtJ4UKinAJh-+98iedqskoFjlM%9R3~ol0o6;(cjK~-DwGk zjR>cy7JjPcWtlnt#q}uP(|wKS`<{n3<4hs#fu~D&U8jgMrQxv?+V@$te|Z(|1K0L% zOwo-p0JC#50A22vB<(jl$b66C2nx zn~?4XNol0JOIlKpZcvc8>FyQ*=`N9OkVd+Z?go+0Z{s=VdB4A~@0yvl*2Fc})Wv0n z=Z#Y9MKWXCx>gS;mc%*}(&v<2 znGi|6%fUoC@cs2h%=zc@rK#<%ELXf`$qG$#l4BAOA_mDED)-4VIaTE0O?V~EC>m%B8;w4tO?g2T&E2yXO;cnq4UNhwDWv6%2CKct`|>L zk&#*ODOuKFm;ii6dZs&;(UOll3%u1OoT4~URs7!1)lD&!8ADE+R~kxM1zSXp?oOiK~{vn4Pl zX|}*p`M{+_IvSggWNBP9oW`$y1{Yd)&N(`wJR=jC9GMbmsaje}xwNbTslBY9LH5^- zEHA^}5Oy(U4Yor5cxX4-EjR&o6Kq zjTFECh6g!WH~5ia(T`Cz2=m0XQ!5=3UHp3V|7@{dif63mTwcXUiP@XP*HX26Bgj`!VASlW3CZl*^IHZ`aEVbXFUok9!X zdOw2y(>Ut!`*ZL=F%QHqU~;G1Mov{vR~ZRiL%43C#EeBIc~@E+(I$*QK)IW4v{gCw zEpewKdQwe^<;c`A*w}4?oSp8vRcqWV68SNAfNwvhqvDZ?h3i9bprLR((#(z=OQP;G zsSx;u1f)w!t-fVBVqvkD!T$xj#V3l~j9{g%#OugQjVU{Lqq)wF8kLQ6q;$@Dm5yy8SH!v@cta5F(GQ_ z7N0K^ryX;T_hNqM9$^Xi+0gP#qO33xowr(~{gXhKkUo1`C7r zG7YF_$i|1@Rl;@FMD|hi^?HrgWER^hXOJgb_}e`zg;Jp2eaXSXZ%PCN3rrT0H#}uk z?7Q-_y=A)z4#e3EWsVPk)dvGRkwMdT&NE4^urJE7f2OA&=xZ^hSw>~`AL!M>uH$BE zUp5Bi&F)V#W7<&K<#&vNrF%YAK3bdiq?Zqa6;s*$G6}Cr5!Qt3s-feTc}qAkK7!$T zip*iJ^Dv5|m`$KjTpmRSVt!r+K@rx%OabwGL%&0*n8g5+3D#iXkP@k_S8R4P>-^PO ztW$qPRZk8=qq2C`jT5TfkKGdr7o50kX`{1S!T!34_Y{ifNuXCbR$`dMB)IiXu^wTE zmqh|1#Sn}_IyjvSR}8%mu?qrdWG6n`pDmz72O)xpu(ozXEH7Uk)=S0Qzl*YUqPIoI z?A-q&NFeG=Lvm&YAUk=j`NDc4Sw?*(j(;F(uh}%c?979PQprYZLq1N3w0vgE7>K>) zt^2-=RvFq&Y00*L;)i)11I>3KOVPWx?q!c*%~z8sH2n^pFG^#?`AuW0!%Kcan;TB7 z1IeMOpcl3zpWt*bI%g+^NwoN`4byg;r;7*GM(MbZiIUHN?bh({x{pejsiKJMb+;6OQ%jfLe^*;K55CYT!OC4j{f2bo;wxs z?1qn10!igVX5CHngr$j4sDgMYw%%GnebUBJNXMwilb7HzwO}##YD9L*hC-#^$&Ixk zh9H$AW1e>h;N98m2cQ?+L{56au>}+D&}ELA?Xp8zEkwhgl@`qpvB*l3s4cM+m*CZ4 zquK~l<|52o?XcJj4?R$>alJVKflnF~#)cT?K zU;AZpnCc%Ro*6k{tb%q-Y*UCJl}dWa{k=O%;_hmtCs&RxQoXrfb*6ZXd?r;zqZfsS z=JSf4I;)>b0pAZr*_o4)VnNBRs-=(QEx8i@0sTxLuzJJvzUzslt{*grr&BjVbLm`O z*Fe!P|ETf8NbeeGUT*a{bCU6$5K$>IE3VxA(|1bSR0%_q5r;H&d}*&QfI#5!&TCGE1YK);SbREX>`A!iHW6-f5o z+`i_DNOK>VXN^_!C#-iy>&%OKR6pW30cyuyLW{AJ#1AOlV7AolHHFICh+!Q8B)uS* zVGqklMd|a(jwb{WkT8}Ps?OJ#F@wQ3>Eu}1t z%Du#P?;sKcv!~4L#B;Yfnet`wEh{94_r;1Op|VVRg*N9%F&9o;!F(ap1qJ||#I}47 z_^GhTg{*g&$3e4Kffz>NhCC-s>-ydvv%K^Cz zhDrJvKc~0tM?c5#f1sd@6fc>*q4Wx8(T<75Hx^uykw8U!xG2hv9@Ly%@5(;(2o;>c zDU^}dfu;SiE;47xtEJ)+9z(><#8u`vM*ij<;hl=LILgikSgnjkB6UYVJUoNlF4@B< z;pl@BFK-sdrZ%R}VCG{&T6n^VO;3}w6ME(wqh^oJ)2^-O12}06Ff^DHCmlx<_ zzT4{OKX0%VrJ0+MlKpo8e|rCVIHc^>Y%X-gKbW`IkXF}QqI;FoM&m0BGHmWa>Ga6N z8hCTsYvR+P)q@T{ty^&w+E4thEUA%>Q|)-hobD4Pm08WqAXgR8S z;aqz}eE$GdeJ2s@q}T91WcQd=t=DVrfja?Kua&)92o9zDDAs%c_F)%f903Qe*<(!Q z#CVTmoN~c;=lU2euL^7!Cm~H8cb4VZU%CR)elp&c2b?ARzjZ1iQ^eAKpPl(}=?iepInhq^$7j2& zNfa4%C$eF%FzD#v7k{rt8z15l{}F^nZuhyj?Yc`R)YYxa8te9a66M|-(2)3{E<{=o z6`Wk1j7^|fL7>OpprTxVtj(%Z<%^iX66!T9xo^WEYBi4m=r`7u;z2nT+kK64%~L^` zHABvV;yb6o3&VJ0?IYl=*G5pdwc2bxp7z=&r-iC6|5twcaWVkEh z$^H1m(5ZiooX3_l)t*I({C>+x%SSDW}VkmP8W z(m*>g(?!JmdX6Py({R?z({R%Ii>3Zn40HR6c2X@Ebwh$Px-OR4pTw$7*TTJoVF4^Z zKZ-qrBoU4%6y@3T6v#E?+^nB}ncVGtIbWzfe0$Lh=xALtyafvNf@?!!P16m)K7u_T z=8opl@WSlPzrE%MzcgraeSIv_uB9MxKdK^hfvZ&aS>S~qf6?))Q(?VYdg{kax`UAG z+|?gkjimXWv>b%2mLsm{?o4xG{izsG#mXsn2L+K9ZyFImfyUv_ntO3hwEp zZ#+o?0IXi@s9?rH2MsomwA2kY335m!G=Jss6-2(DB|2(#r|bP+#zf#EfP%0gMutDI zSZS2H3O{vw$UTvJ;__M$@!nX2>vC3d*Zwq#Lr%jtlel}wf7Ai{=27Xg@v59X0k4{~ z(S>o0_x5*Yb}hC4HG;9g?YJ{i&=n=1zf0l37N-zg^p9-0Rxyt)_(Yi z1J1L-dSQ39d9ZpEaDztSW$8e!$=t$j5-h? z!899heH4*9OrU3kVDw{Hb`U~H7G_=1mfAnjuRa}@oPRpol-_#sid^&zuHeACYONcU zvm2V$KAx(IO_4D6UT<@R$h4;)=D$k`wuNEeD(H@hAhkOJW0)TPxR~4(L1tr5i&#^&pso@U9?4dbXGTFmvE&?{;s+~A*1FV|pw{w9 zoxqcCQ+&j>gQX;-6-HolQgsJcCy;J7Sq8E?h9+c(Y}4Khi%Rul~~YF8N+G^kSJ&;9iK*_$odkiJV~Kb ziOQcz^jw1D))Oxsd(%YbgRNBphw+gQ4gwkc95t{wDz>dRmP;v5PG zayzg<&M;!;ojV-0tk8sr1DN+`uiL6S+wI}^(bHdeuA|7x>c_rG=tx^z10~mrZKWE< zwJ1s5JIG|fFZuB1u`A6^7p^q@GuS1{RSgcw*GNagXtC(xgq?jb;$`gB)tDh1dr0j~ zAD8V`;eD6)=GRg*l$>c-^&~H&p52cz}6nIF-Hj>opbJKCpAz1chK-$F< z)N1Hcfzz?dKSce)uC;Tz#nn5ITM>M+n1I}d=v!RPQC|+GPx9lY}` z+JJ~iC#bmsRX#2S!jtHY+>*(byj`e{?~?UR0hk-=v24ya3)pcVJL&x|R;0nzajTe} zu79+`>_1%$UIWnjh{fU$6*83v96~a1pBdo968|KRjgGsc~vJ84kh~4{x`5Qd#(nn5Tn2v_|6)COK_^z>aHYW)@vnMcU9sy zX%YQA#@4eZ9r}JJpaWMyPzguGL;_TyDWrve0Jdu`_Tl1l{?+N&y>krEj}9~7T7=}D zj1OEp+lZ=d{270TQG8-to5Ml2Mb!e{1D6cH6Lo+Tl%)cy7|&7 zL;Z_3^7(V#FNV677aCzDqHXV0Zcn^7Z7H6`dl`r1B@mhMD_0k?gFl1F_$9o6&2lzwV=3Q6@n-RQ3 z7Zi+)NV-3R6u;5L;-oaQ#UIsv^nou`_VNMIHlda zMSWCP$lax@dOX~NB^>_xWoSOkDc@uOa??|6ea9Pl99CqlThTsab{{PA|p%r!)? z$fZ>kbLr5PirMoC#tYSni67;@*>w)zml^Uizn_ z$2eRNLbxcOj;%9u1CAsWyo`aiF`UTdBW3#B?je-;{>X^vh#wrhNp)KaTEku%+Wl@{ zPQ6L3fsXM-tzWz$$%|dEc_(6Szpk@Vl+)Jar3VB!o9LjS_)J@pGd)(-{*%Gcd7<`ITdTrOsWDfX|3bXxVsB>CZ7_hR)nUl* z)!TPr5{C^NkjI4nuhH*_0>wt~*;$>2PH6hJv=3=w21vAOdnQQW#OmIxL9L~Kfx-Iw zQyvU3>zYHrU+XqpqG_f$=qCT&{8}kxZjGW(Dx##F_q#Jj@b9E!W;yC#05)3uuT>p(F((vcq{V|R#)ZZ zB9d!UkDrPvCBIy(H6_k96a6&GNXf+9rPhv->2KSb9HPaa0lsYBp}$+S8Q%S*R!D9? zD?xc9RDim0@_t8~S$8sQ7wI19bO%??_ncS{PFKH39l$_J$bStsQC!-!2t8A6VA+J$ zpQ&t(EETfHTU_4>k$EwtF4NQm*GU%K7&Ac|wy-(^T~JSHYUK#{e4ylrz;~t%M{5dI z!!}zjNwWupkC8zIxZJ6rc=-1K73qZCRAO!ad4A2N%|zs4db{f$t`|$>vKC@abxDbK zmF58&gFY;0jgFP+fb2J;HnVcAjuc>oNK)(4(wJ+qazt|CH?b9akUK?X2!W2C=7l4b zhns;O0yuYKo*`j*hR)9{%s`g&o*qHKXz+z#vwi)Bq34m*Rl3m9gj_|X^`8kl`S0Q0 zt_x6nR5ge5NrZ!liOUDx^u{FAuSP5D%m5Pn4d#+7c0BWA0u{QO1+~Luc^{CEbuauB zbkJVDL6I_{n+^gA2eS3hC#YiJ$B}?;`Emaw1SsNN$ZW!}6Jq!JRl<>@&M+$Aw1n^x z+Px6~K|aMj2&eYK@vmk{nl}!&YFP)3b-{?_oM|;Pv>s5bsKudcb$vM15L|V%sW1JNu%mEPSmz8wU5`lNB9&J_7Xo^lnmp^i;*fV_;ByecamvWU* zcM#4GJwYlU-uh_F8!xK^YJ+$c2240B74+3^Hdu0-BKff)0q_M)t~p=G3EQG!jL-@i zaJ&&rTH{4~(~S}vh`fj(B`UOOo4Z<8f*G!w@5q*l?(JWG&ytvNU;^r~%j(d_3Dg>& zKnL)XnX%*H#W5W+{z~i`NYtXtFKU>s{A@jyKFs5&yk2Y8lN$Ol9xc`ZZMpMe!sTR* zkwpzAZ83>A6&f)KEU=C0r4{%wUS(%1UIQzBn*qSS8Y51|{b;uHMLRJC;J{Bd%%hc_ zE8*5KGuetby>a*dArIixXo(}Tq+qjp!|umg%IiK z>3SDGKzR<|3Zt04VJa%h&9? zaxC&jA4E8ELsycS(yu;Td`+~CT=YFZqSyMF;X25Ox_IU}um>;$Tbk2IU%L2^XIDAd z4BEL?)x-sIrK3;?lilLJwpj_>3jQAI-GinEeFV|3Cz#7~T!}_Mh@v*2p!<1s*H4?_ z8XQpMep`C-+xvWmklMS#=5Iv;U|!o+k&k=r6XE9vPy#55fAYSu3WSATI2De%bk$of zWaK1Rs}kkWQvu2kZq!5)dmH(W7=r^H25dYzs3hyn0{tzw8(C!Z`~5ED+D--1)tt5O z7{@9P#t_%d;)47Oq(5u1jSu@b;HTPym^;lhXC3sgdUjC)?cow&10$&u*&sPQi{$3k z60i+eip((myHdP{6Tfq3vqW#KJgf*59mp5ScF%JQi2>xjZU16`xYPbz7`q6_Q?bjg z4yXw`?+pkOH!evyT|!nwEI#Q>GA{YD@SEh?y-FU5f)oFq<64y4@5fOL3_6Nf1s>&t z5j1t;HLmH01I^z-;o2g*EA3A?Z>bOJyQ(^Ru_&d;e?EGyv&Kd69g{kfzP6wJv05`y zwTYQb2`~ql_`DC?7bz;zk-qBUQPb7EkYNuze7JXc^_$Ss<7M3fNsx_=V6CcmKJv{* zC;c%{zQ1dt`2H8v4`fSN&&R`aKqnP&4P^Y0n{Q4Ub4lj+VbXgnq?_p8Rye5k_@>2P z=xbjIeSHdy@YXebik4QgF~TTW?U$L!)qBbGAz)=~RvriJfNdJUzWIgysimo^`mFT5 zx$ZOmeoIz&J$+(RWZ=n*cAprU!YyTEa`;vy*|+e^Nu zlV+u&0E5Znn|DdqpnhQDIWID+!6Nn~r)-|2#p(^2T`gt5j|v>+X}OP+S&if9mYW5V zVONM2J%xW<+-6&aV@iHlE=<&FGaEo|@cCGGksjS~QJ8+8=^&W@t?UP+qd1a&aq^@m zHA_kQ_!^skMjB)(oQ-2=ko&t)-c{ue=FrSb{lY(x+#kDj$_J|@N&=B{LJ*OC>++@>P~xP?7h! z1^e}eOqL=IJbIoW+Qwy3Xe1G8PP*kyVoF1~r1{SnXPZGPg@%NnrSg{#e(MD?XY8qD zSmm_U$!syz8icNsLWGK?V|2b3Uswr~KLyPC3nmA43nZ`gVC4vsb*d@RR0m5msTO<0Fa4wx!)%@W;?93eNm8?-xbhi zyXU#$+ONr25MA8um@lh5Ax^$d_fE#+8D6ZMme0{YaDJmJ6KqwBy`SrM;!2PL?I1I2 z4^0$A*06_9xUFkYaO-PbvmgMLXXH&DQq2Rm>Gv(0%OA4sJ~6x_;ux{1za1tH=YTs7 zc~#GX=K9MtNuDhbkk&WZ#V*ZbeEL;?f`RJjvzJLGqw`tnYE?_aGo7vMun7g7N*P5; zJFNCriKqOhm_#omM+OFlO;1iz?9KU(#%f(HlC*Nt#{EP$#V8}oeDyBC4Td+*t;Br? z#oz0P-QLV=kEC))c+b@tnZI@Bgv6%C#&H0#?n<1f$~9%Xo1bvb zypt@U6KG6N1o3+clE)ol?o*>xh zWSDfgGd!*AlIfC+H3$>48hT1B=Vw;Yc_Qv-r1OvhL@O|zq*X7wP8*bia5 zTxkC4&_C0kQPC}pv?`WP8|7Hr^cc0kKZqm2-h4m zWLE8Gfh}kOT+BTuAX1Xv@9KlxADqI$#w#0(!F-EAF{?X+Iv@P{>?jV`{5V=0>yz*v zerV$(8{7qh4^e6)zo7r#+T5D{@T%Yz*%UD~fgZrOGv;4muNt4NYDrSZHXhDN_ZQ0c z+<2o2HfG-IsER#V??JaCb)qGBNhJAU1VFDqNguNUuP&C24h`kT46(bh!nLYGMRkjM zjdMy-r93Mk%F6I&K^z)!ZS7XS?3cFsq^!n?)zcnmH<=*01ywA~2`i|P+=pSQxNf+ikh7UpqP`Q&hF`-_F2j`>9-Mkl>4(GtG zJB$?Wei?oZAVJ&)tAdzdLTld^!3VLcq|J?2YNv1vAq^vQar8n46q_vk`uv|@>At=Z z>SP-CDAoiGbQ>`8I6A^}Sy(P`ZGT12L@GGgGC2Ld`nrldf2i^5KxQy60gsnjc60Ml zcWOFgdUxu3qs&LsoYs2wVd9a8qw|vY93cdLKSfu^>qiMFBiRWPqQg((uGKcTCdf`J z6iYeWTk4qMJG?ER^H=}MMF{TM<98K-yOtGmKt+( zg6ZZp{FWMPj&{(;xXiDs9EwSlEOjL6m^c2yDz-{>I%@1pMw-~)Aud_^;V?;%8@!XC zN^LzGkyh54`^Ms{C9U8Z-*Ub<=l%T>WmKe8;7|o1PqfCvIvCE!dd*>Kma4}1xuJB6 z4!gj{UrW8X4pkr|M20YgAPzRVf}dCZ!sTSGl#!N962_Cd)e#9zIcvKD9 zz}|GgWfjM4{G>Xtb0g`Rh8>+f^{+sTK}{5JP~^0(vJvum0Nl;lOxarxt()~A?KMKd z4wbJH9H(znXn*1eSCk!2j4o5?xYb~6!$BgEE$^DA$K5`5-s-tLB&coru$Oa<*IO53 zB~yy)uN|uefhVJhr&A?M!}=-S5r<>h0%BbBQv(C3l89vq9V`2TM4f{rz)iG^us9HxMOyOS~y=V{hv?#l?q(wd6JsG5|AKLQ|lX0jUh4 zDfT%Yzdz53JRP!QlU+} zkNhFR5(*;CxF%Jh9PcJyVXa*mD}E|afbGcm%M9QP36(Iu7{t?*Kt*Ef^7?bray;_| z3Qw4%`Hs*jg5COUxu=HcRyTOg;bQQFt#{Jw%Zu=1xFN2d;s_?D(j`s4d>lw>BuD4H za#=gG<^z#{Lc-G;YX+SWpBcbK^{(Ia6zHhIi_l5}Ww(o2AzXiO1X$pc6!e?y;f#># z(B;kE>Y{4x8A>{Lnl(7xmM2h#^~M1Rf)5ob5*6`eSMn|s4~7GxN3^+poyPk+QpOA6 z78(SwT}O(i>iWBmc^}9@Z6L*8EtW~j6wWITU_99n-WLP^*Tx0213#839uLOnrV^d7Al{d`!~1thk#ttb!&B+iu9vX@2#$n~)MdPIV8nYZ;&CfR@j2v2N=$_ay6VpNBxmjbw$%o zv|IFE0cF70f+XXrd9PCG>1qzP_Df_bxt{Y8yQj2Gp2 z!5*J=mw3DDJDeA`@haof01i&vSh2vlsQ%bZ8CRSv7K84v328G5 z3~@2nf!Iw+VhXQeMoKZlT0b<0?(%%IRWcB2`=uJ;i93UupteVzhC1|fRK^Vl;MyCv zEn*m{7aS&-9jpt?^TfikgzZ)c51bm1^^NU)ayBDd(S{a&#aFWl=PM7TeJ1d$k%oZ> z{6E>+>6@pWiCDy>H?o%tCUp1yXsEM2D6fB1z>e6=`FA^-!0}uEU{{A_BPB}$tw>8L z+d^+}E-GqVEB)u~43_%tuilAV@8@&+rcen?a1U7zO_j`9BXG)4N-!x7x$c{0^dLX2GQ`1Auu z40W`=)u8t{YDs57gCMf62-kKb1f@EI8(3|xUhF(mE7TCf9a#GaBO)41Uuk%c%;+c9 z>j#m(akROL_^A4)f#|P}{3&BIG9Eb7?b}EdztsNoVE~8|5BBUU=&SQD`ewjg?&g*I zTPyQvF;wQc)I?ja#+FMNQ53Q0sCS?zle_4q_OpH{Qw#Ln$5_FNUMlZ()fbEK9*r&yZbJP~9LKli$1B$_=I{$xNb)v1&gj&gf_Q(FrVHR5G=;EK^QH*T_)j zDcs5Rbt@v10QIy?K6{>j#P%!(=x=?g;d-JcB!k4ueA~P9T~4|A2;xVe_?EakZR+v-FEGk@M+(jQt!@) zu&3d+GRrz9g;~emJ5~8voF7O3_Tv1XSM#-FbX*i|^-B6!lCD2hDMEhph9Dw#`*Pks zL|a~7zN_3C*1{B|j_bF>O3b+t161-M_vYdx(%Azn`P?|t`GPVkOaQ9>Mq5wc=0TF{ zGaHb$erSqXuf%PIMRSx!w%J*QZnEWS)0Zx}4aLz#`}7{BUocqCn6h zzBJ5}J@!f3oT)&>)a`SGx>H zPn!G;+QY`e>XrV%bsWERhVtGbl_P9yU=B}SrPaH4HErRv-PnyhF>uH?!WcAx1j~38 z4*?D~`c?=%ZyQ5K+!?)zMp%@X;I4VB)Q2{@{@Zz8_&ElCMYi%hTWDMxHQ;b%ElCyJ zKS@I9ifs?EiOrtC(kBuRUTCyt&4lzDgCwIUqa0S1_8yMJ=N&{G_p+K4k8Guh65qG+ z7L0Do{&-zBx4YxKKGTVf`Q8*X0T9E!zW(EsyC2ZnrmZ474zK9WbpU+bk7$>zOTGs_?q$r_ zhzM4%pXTUyUS!rkZEU(HG6q1I`)T)$Wo?hMvG|7}7*HAirIN7%5q(7-9tF1q4bIhX z8Qi{h+4&no?izzQN3zc6eif23ns0*fWcU~!-x4iF3qDo#t%4)MApI{v(lO~R5TQ@U ztygqyz^RnkUcmL?0bg?nfxsmehXhOI{fX+M=XVf^L!hw|4LfObD) zT}^aiIgLhEM4(;$J60o$1#FhCVQ(c_p#|GXoa)dVw8>$ssIIo=NJqu)yxD!)7@|xb zqeA_parD)k04LMXRiG{SXW{^O)o>QLFSc=ggK`&<{CUylnD}2C#{wBEhWl(a$Ptf* zHdsi*E&ZuTJ#1V_xp9{j(fYoX2Ng{%jOJ%SJ(vj`|nzl zqr3Pw$;ZDG{m_V13A{TBfG7v>Ksf(A7rL$UfH()U z-e|C42BOAo+j(6}u3R=>`n1_$IoV57v|{n?#L~C-Fy((-*JCkvdYexDIkcl*ukhB& zDf%yg3?M?ap5c{yPLGtdOpb=_qmZP9-kylNuoQp31#M6E=)Uj%av=Vwt zZy2hMLfUQf;kwpX17O;&@pg#C2sn@H$IX3rJL&7GE#dh5=LaXkA5_`04?21g2y?NzO5nbM=4Rp-YgAsU&9SE zSv&u}`#Oqfhy7XT08}=!5?O7^-9o5U{!jQE$I}i?WY~-E{(XiSH53GFMm@Oo*YL-5 z!Kv#j*3<6P-oF3d;GgSzzKe#AHj$zH@48D-UuYr4k16Hvm>f}h3YZmLa+_%~8jBbd zE6)EZixvRoai|}$YBB7MqHury@jrbTgXXeu37EA0PL-gjh4c@Mco^AHruW+s8a6u_ zY4-k42m!ucXS+veCWjT5bBWSn|9dT{nDwAHnbOJ>5gw+D1#5jpbug!$R=@gN$fa#_CDwdVC%UoDo`$S2aQ;2Yk*H!{`5gN% zFov-8ya*BdP@9a$k7@E)5`zwapQnUC?+6E;;+@I}NW4UFs-`!)?u5jQCJn(5 zvj9&s@kFnR7+s?4qys^%;M9@*`RO203VO5G$Ora6We~SgJf_i)U}5gaT?e%RBDW z203*|4&{~Wn(|Qc>c8G6#dj=<6}gFc-s0A-7rSgO;g?^^|G#gbfrmBBPDH~MY>&(s z2k9*=;wv8G128LykB~APJHci=MUT+_9mpQ=s0fL&^hDb`M(K6$)*+3m?26mPI^A1k z>EvFm?^N-#Bl7QXXWu$9N6qbn@Cg?U99oJ9W|6Q-UiVw3-|`{6C5tKzc`Z2tIze70 zeT}Q=o+y2TK!WeT6uM7aq zt+Ph3uLbm4+9DFUva=O`iWjP@Cuh?IA6cD_2y%mp|E;u$a@i5=**U?{UX8X z-@}072of7Z@~d#Bqtn?`cH(PJs& z>6Uo`V@g-5N-4tJcOpvLc*B?Sw(rsmc?jJtOS9}1CrY!uKKiYhMoaFln%WR8S_2eAI4vt&Dx^}llvB;v4Fn@pc~HoU&(H=iS6n!conigp zkJB|&D)|!Dg(UodGTwiWrtpRoWq5U)OvGCBU5JK$zS-Dr#ec3Q{Eh3N;j$$3I7yUb z7i=t=t+{o9%%aoP9NEw-#pKk`+L9}^F|R*xe>MzBUZlDO(IBi+vGY@Chec}8d|dxs zV#=k+sh=|Fr9R#WyS6N!9x2z32we+M{9&Zho&UwO=0~v?iIzcqt;P$zAIYzEq1#C8 z3-IQa13+JvGqgc$^U!)cPA$_t#Mk?;!tBBQ^fAlIDE#ous_)o3syN-e4Gnh(T@O&} zQ~nVp)Lk+TQ2&&{RyiX;=;>2vPEg!4T~u+W&&@&fq2z}ci}IT&9e`HnBVr>ZQz2JI zdyrz`2}RXKk>Zy+abGQpV#(mIm71_!H3ld4N_FJxW{Q7u^%zJH+K67mE5g?d7}H5p zuwY?XsFZRmI>jyMD84>3(-}$rE_0!}^`ECMOAgvCwRhu=`+Q9^#G_YQiEq(vf7db< z&o}tbcR=Sq(I7$B5iL^R8wPV^qHJPQj9H@OP%*icRcJ{hd)3r6MI~ncrkZrLYlnwN_zlFcsl{7o@QP`f(j!i1IrX%65mbX`m`(x z1&N{&$XTNK(a3JI6n-pVo25cMQ}5MQDBD@zXfwaJXD0PykS##*uc$MMhi_v+7;s z?#C1D6;kmNCXR?n(DR+@VvtgVEE{$kN&glPnVdj@<*>JwFjSr)|E3;>mOo6hrBsVa zF4s=@Ph|*zdWND|mvPk5-_Q=e@wLl!S%L|-8BM!veqUX4?PBu&n?9isf;<~jCU3n~ zdjQDWJo<=ufak@$hZXAVYe8(Gg{Is6JZ{;i`hqV^<95I1V&5ET6F2wF6T(`6x;-iZ zJFo|0on$w$0?Uo)(k8VG|Bfl+-*p8%A2MU7r*U_L$sEL}2{jpr>ni*>+h+}Qn##EK z0e^j0R$t#sihVetkwaE{=_~mT8K$MzFNs*S%<^RPmwEi3O8V^r;*Ul4(06sh4Ve(y z-7J4@#l}rKu;n1E|LNQ^luz#kn^D1eY^>aC4{uVlO0JOA-XW2b(4i>-ulC7=Ihn7z zc;C-E3;$;o7;xyAO_CBWUvZ4zfp$LoEhu^W}&7SnF%6Ng=v zHcjaF6Of<)x?1}7>BzrDnfD7GE}QtrqyPFHxxgrM+0(sAUYGuyse3l%j2O|SL~4~l z4lBCjX%B0gQ=Zj<#orEZ3;(}U*A)cUvQ9305k^35Q&fWDV^>Q%_=lG$ zUV;b)WBHv89r+`^=Ko@k5s9o1XKVPzw0k5=se$!&mD}fgZazD;kv9M4rgjIRJ}={+ z5#jQ=;;AOL;1~7i@-#D1Qae^nUnw%tcA{5ow&FKV0O7Qs(G^wjgTF;-%Mh)G(%M>& z5c_XS+t2@9Q{yH>~GfxZI$enHT5;Y1tcZaVvR6~>hU+)WgZlgXQF!(xxt*YKhKzvhe zTqN6*rr!07t{U|LI$MvhTr=;bqc08W=m8b2=Bop%g`HCvU)t)R?`RfiMNE9-#RU5+ zxY7;0f}s^wcN&+yM|JAZ@ypjrY6^Chxx=N?rnBA;nw@2hf}x|q7EXGER#?h0K+C~j zRkiiMXZZ|5&IL{Xdvib(WKAw;YP>W%Q#cXNl^I6uRsLsf5%r3gB|1|dKcxfmPap=} zF5n?qazYy0lv%`yQ)E7YV7qtU_hdjV_vju;<|&ZPqRPOW*ZRJF0V+r*m~~;1l3(W` zTMdF;v%Z+Hm>Qc`)GcqqG-9SGgz4SQ^r>2J) zU^_PPs1k87J{;~#68QwHXjGvPs70Ievnf)vfI3#>G0I9u(7lEdjT1WRBHc}4sG-n) zxao<sj~Dk%1YN>Q(<*P^|_A&{th%Q6-;Px6=nkd{UyM9O-ENWUAQz?=PWJ z#ChJw=>1RBE}#w0#d*MFGS?DYeEcGnY%if+M(#;*VUhq_?HEtwUTlvN1*A|D#mC5l z3Q%I&bK;I#P)GAI#A8~UmrwPlHP(~cta&)X?>B-ieG}gAv+A^ZG6iB$)M15N0k>%= z2_44YrB4|rETzy(&bcr)NG`Gu zivT#=8=%rj=wMNW%m;J|`VAD~5W2e-E@5$F*Lw4wQn0rc-@SNEuK4hq${K&`+n;&q zCfe9N<(3O)y-E7kNaZMz$C0_A-ydA!m%wO}D*`7~x*@-upc|$N`FEjEPYOJ0M8o4mK0SMDbaoi?uX6*6zR;Tjf&C3@>&#% zn{1x#aco;k`(_+*rt_~G26Y%vVb7=&IB0^3LVm>P0H+r=oM?HkZU}wvZ>13_y|?1D zGqVsWYa^pVvFOE1vlU4K`x|Os4vL-Iky=msGyJN~>YF)dw45i+K|%sDf6um|nUNgE z);ZDc`H7yFit+L)3f)J^N-74E$1F`JZz%uzQT5o~>WLaxDhvQ_0X5 zrOA4ds#UV(D5lt0coDG7sYS)x69j1&6StcI2NL8cOt=Wh9K{nfJ=bGMQ5BS7R5Ur8 z`IKVkV(9Xd69kSLUoz%_C6`+z*Z8Gjsp_sKmoq`Oou7?A(@%IUMz-#Z=se5RFe-_B z9=Gv7HQA`-=R;9e?a@|nATSo+H__7M9mnxQ2ehd2@IO|#kEG3jP@U8Qeul%<)oVV! zMy<BSise%zvskum3Uq_SNxyMzSP8fB*pkGfE&@ zEmUWCaoggbj1~^TpSu*dkoBXwETg15P0&BM|e4;4Ifp(m6b6Td}z8@{QVckF5 z^Nb067lFo_+b%CgMxTgd>&{4Aw)#0c>Cig<;s}g?i!66Uv*ca1ytcdhtn!0&&l^K0 z-N*t01PFAXK&xZzkz#-JXOT3zvPi|%arV1D!ni6DN#EKcAa2OF)bf0wd&S_p9T>V@ zCP07yfiqN~f(fOC{c$1x^Z4&GnqWM)Wd4T!NG`=S-9Vta0_(Oex^t{7Uyg_OW1T3c z^xCrd8*i^})FS)71P-sc{gO1=csM?PaWs89=d|`CrGs$++82N4>;FWe>5G{DDofGX zetF~{CQs|FtW1CafoT;;ksvCwXs(uKrK~tNZZ2D$X~h=luaZ2NNFl|R$7 zY;gDKY-?Qt1PBlyFcSnSmr&ZYcc4DB|DzA0)cn%%z>!qil6BR7zx1ZR?ioMdG=VAz zY<%nP^S&C*9*=*U(}&{0{^;*ICdRsI^{aj}_ocm6h*JzRQy}_S>Ki$5=&mS{zc0(u zjo(&lX=X-YuLuwzFiip}y&q+q>wAjs^=akisr(_zH0`L5AwYlt0RpEMIQrmu>eV`H z>ALOr{-P+x-j0Ww=qz^P5CQMWQra7(;+k7br17dF1yU>P7{8=U<+V{VdVH1ol>cGb z{0&ccWN3D}BL&Lm7o6Y74ljsWa<{~Jc3zk9ECCvSMw}D>-uK|(r&E77TM{5ZfB=D+ zBT&hNQj7P6Z3};$73KQ4aMx!Jw>=>+jRJZ0?6P@(>z~F?^ic#R2$ZXrcaLPBUtN^N z^<`OH6Ax8aMw02WI4>Wc{G70*7X%0p=uCkl{l;bK<4CISPG#B^&DnNz{qP6xr@LM` z+?f%7*=qs>2oNA}a)HVwlv*LL+jh@&W98VUsJU_B$q6hWP%VMzD{MGQLLXZ?Z{u^- zicBb!@DtT1d?_FSj1PBlyFiD{DH|?+PT0ZMvUmkiS71_!t z6~ACo{9Y2MjzG%(8$akxd3W!?12_KVv+4vU5&{GW5FkK+009C72oRWQ0+mlFwZWqV zJgwcn=$a@^`ejr@j>J;xX)iM-(5V7ZyJNH{vjcga|6|TY=Wf6Cr_VMzH5U6#fB*rv n1TO(i1()zH0eu-@ssjH9J5p3Q-nMyY00000NkvXXu0mjf-hUcE diff --git a/docs/img/streaming-figures.pptx b/docs/img/streaming-figures.pptx index 1b18c2ee0ea3e6c07bdb24c0a5d5784c972631f7..d1cc25e379f461095f932728eefc60c0ff2d2dde 100644 GIT binary patch delta 18403 zcmYhhWmFwa7c6{mcXtB8J-D9W?(UZ0?moCn@C0`fAP_vbLvVL@ch~drymzhd-XArx z#(H|yu3g=|A5(cCsXTvBNst|XF0nKrfk2;ml0RVMrvYei$s8m^K=!Kp;@S|Wf+)7F z7mCb!(on}fXhX3^#`a6!kI{p^0?`#>zS5RStALk7hNyak{up}s(&opm3GTG;MJhRM z_1A>u;lwIiN!e+*xY`uOTT872aQ5ph8^yIO78_TcQVde|(Q(ZP-L^AUrLI^EhVGbJ ztGq(#+R>JrhWP-a$39l33;)Uxm8(5uS|(A#yaW;!7)AC z(b|>2&oDpJM4Vk4M=_@i{#q}}rT$K17RP0Zh~M6$VnE;S@j7|sUwoE7M$Ktcu)_rj z3H;?-Z~z4u8s z&gReFX{86(0?d=WJsSJpf3OCMQMBV0ck1d(;+8Ic#T~$wb8c1*j4puvsgm61_(%$~ zU+|K_0%gB3mXa+mR_i@dk3g*KP8@C$XMntjViRK_A3MWUXXL0eOQ5*uf+U*JzjNEK zh{ErZ))1FWKl)IKzGX2auW2k>RWwGoJI$;w6mxcV!Gq#({VB&1lOVLn`tMj?UUw2x zXTPL3HStg?T;QK0E75Jd8%y=5;uxl(L+-WSz;O-B!-E8pJ5I!%Qka&*480(Mb+Hh&*!Z z(@QnSn`%U3_jU8h6zI8gq%|H&7F2Nt7X2v@`?}=85S*Q2;76tfEm$ud5iVznNJJvQ zRBwK=~k}eN}a1*bj!jz6DDtg3qgJ_B4JF zC=!f9H7un?uX~xbM%zc$%S}U+=M%Vwecw!;wK^fM-3!m6IJ%NUtX~_4Mb(#W{&hU< zRq6fxo-0#rqrw)3hj}L=hJS#txu>0Y>LNZJoLe~!+X6Ii<{k{v!w0wCMV=?KXtMxu z9tz3@mr+5UCiBWv8n)aNs5SPdKaYa3?S=#$X83m(Z=S^dyRXCC1BmDOaXiv5oP+?Y z8^-$8bG;xDla53yEL2ww6~jJ9A>SvBWu2VY)9KwfKC+Z4jbR79>o&kSdg09eLW9F( zW%Kk7TYl?>zPBb)(ZKb%4xqx{`tmz!J@he3rzL&{0p}Qp<&ca;=d%4IIAnR4)!Hwi zhn4q&Gy;|II(bo``uH|*mSjP*;{4MP!9c0@=?7$tbIpOKs0FQfSCY!timlTK_yHei z{kRwzR5L#IV(~7Hbq`>qYvcRkwEV!d-B#0W&ClV;fM*Ac1y9(e^)0^4BI-m6osU0P zzFTN2tluANKC)R+R&=SH_;mWFB9wXF8XaRns@mq|NtG9J;2Y-A`WtR~`w>(`Of0(s z>2Na!$Y53;K$AyVBi==NV*8_Vdo=@U08nka;Q8h6G4YrSSsX2!ct5-0{z-)0Yi} zVswEJ9T0>@P8W-8Jr!q9FHE>EP86?3{F}r_)dj_o>nF*;afj{P2m@|6D5b2i`kg^# zZ=!&Q4_psyZjY($JFH@w(pYkPUHIof2*b}$^=3&=O~MtYfaudn8Z=gR{>nTfyx zoyVM|6DzM7D>cBAA)frx7(xqU8qvc)%5rRltWNdUdKEgY4FnJWpk4kF(;+NDx&!eu zo95Cp80DC_xJxsr5no;j2E?hoHbFwdHfkHYG1dl*hM+)f%cJ24pT2p=5Tqc z@h@+^v+Vn|%2U0$JDReu2$lO~I|#$ocA-isLk?`JzJP#rpn>aU{R75@v1Fd(=GjOM z-KbrNS$F1?1aij6O|sO_Og=4NYve08LDgBYfD%oj95e=-4&+T{iUJ2V32GShCj<*l z4=S)bykT;~{Xd2zAO=`bAG=Y)&Q(D8H-0W(B%$>qiE&c7)$7N2qaeQCi))$wRkTo2 zy0;|x7NBkL<2>H-pD6E?|HDzGY*F1GA!JgW0fqLoB|sd3cbG}AP~t&wq;e(CGCw<;gBI6;w@d5PBx;sgL;fHkIg6p`5@*o^qe9y? zqa&b8ne1x}O%Tr||IXm*EU03^w&!*flLD&K3qTt@KKP( zzFCHKE@fLVjpSQsF|lHx_2(aa&^-g;%ovh$-vASM1ARVBLc~qvOV|w92KM6M&pel9 zJ15A)a{DK6dvl&2a&8n;@UV!b{4C9uoQyU;$Z+}QSbkuoF1Cw>@cuY~;@Z{>v(^3e z)?X4Xx2EY++ezaJ@MQeB+x_>3>s5FyKTl-X?OwCb3Gs6i|1J`T4^IJYM%R%KWr4nMEv zyGZ=x*9f;~iQ<9cXj+y&=$@83YN_IwWQ3`xt%M+^eT}0$o7FeAN&PE1NDjaD?qm7~ zWU$5+`w$EWG=h?>6hNKaEQJev)Yr26;ISEF!2FF?^z=+nUKLAKPD6I7-cvQNqyCxy zH3^-8L${Q5gDW7B#m0-z)nd=7a_xjMzCnYL_Nbb`FqFK28;igEd|)l&c8bDB#@b;2 zdrU@ro3Q3u=3qIvGDG9?h4{Qrin{ZsX7qs3FZX2lAS#;Qf0{ATesAgnAHIZfw+=m1 zraz#VXbT;LnPO{vpEYVU&Z#lRg`;-+@UNtx-a+)^u1*M~>`R=B)hL5iToJO#$(eZi zO?!}45MkSz_rIkZ#fA%q%z7 z=yH2bowYBn$@aR0%;kHD0{8ixwN>2TyK*-M?|dUdrwu=p58^2W;nk7&ers*>?xxKO z#)+(p*>O%yX_U5;{XtfZz5=z;)sz42z9jBo-^qtHD=p>VL)jc|&z!BO2!jSn&s>(oZ9Q`8Q5BpM7$9%B6Dt`d1li*k zborMzq?#y{7uCAoWH z@`f&GqYz#Lyzc~+H@{(Z5;@dvsOlLuGS~$mJ;2A|m%juFFPnB)PnT2% zv3r9DpY%{We6n|TG0QCQ3~Qd_iW*5=e`L|>6;dJ+3ylBqt0p=PHi_Hxv0F_#`#sQdeM4{wm#purG)rYP>W8UJD1 zz^j;K%0u#yz6e}-w_L0NXWrKiv-?!&ex<7FYEhTJeQpGd0m$7tkxH?mV9rQf!DS8!#;LDbs`nUgC8vck&E&Yfw!)!7?*R&p_jkzVbHhKF>NMt55K z9@DAGd2)=vrb{6Y=G*#@_K@&5+Q7IdQ?qqK)a|hMZt7;&~)TUxsmI z=lf+iYlU#?T+8k_*swnF%Om&N^>4Aj$$yA!6>7f#9v4zsIX!dlF8!s>#^4xYCNI1M zCDseN&dML0&a0ijoM&}HI{5l?2_z*CGLr2TNoGS3hyDaVeATUYk+4k=wS`LsnYS8; z7u~UZ8K!vq^Nj*>^1t&pWtiS2vTgOOP)3BGVhcK zJ2KCdfIP-|zL1VxR0?>`>!m=V7iXg;aJ&rP5?+Ck(I(=*&n;na+{?#we4CMHxKyuH z-bOULr%?K8B?uMS&M=W1*6Q@0%a%9h0?8cNIq2sM;j+p7Gpc?H1WqBd-3tQViJktH~yvOf&_>mP39eb z{C=+hUk``mwWeo0Lb&Q*qbQcHy-t_jfF;8SSxh6J<3|Ep%xY3KO-;qxBWI?AQ!Qf* zl}BweXPIy@Iwv7WleQ^v>a&(K(x3Dt%l5~!k-Y**wqA-@R68T!P#s9t-<_ozJ~k}F z{;9@PQ)zOvA;7C|_=F4ZjE(nu7U+qRTgrEAr8Zd;b7b1f!nnlLO4$%Fix6c8s;9X! z`v-+7gnJS*$6n=RI zcbPU7VJY&(@Yh+{!ry{Y`gGr6xqWVJWo(v%>dVx!bR}YWj$aALNwxmD{EORpcuW3<8PyG zB`Eh%MV5D5-b((ch~iQ3*yi20x4yZCTOOrX(3c2kgawaZz zILBH?S(!*M0-kAPvB45hH}M3tmCN!c`m5zz&3qR3B*OCm%R*j1+bs({_ZrgG~Y z*`3iQ3t#`xNbrN=Jp=v0VI#uBm{KV)8(%P-u;DE;Tq9jTS!W>XB~C?w_j|_f zKm)1;<8Rk7rV_srY+To=pKuKBos8d%c>}BfhZiKxSRp69KVOICfAR;bue=loQ|)8W zS2GxQwiP)--pu-kG<`7;Tu)^xl-;1}%XQJ$CR@sEpl-SGhk>yY-n9qX?@JCs*-QBp z79Ii_e?9n|=@$9MupN^Es>7SBVbwJ?Yjtm1u}=bunRiaVZ!x=?4WM|^5S}E+E9-{> zlHp0Z%Vw92%5q-FYWON`R$2kZs!)Q2W^cu;FpU(7Bi6#$Bnby^Q7gZXN_hh&;e@yh zO7tI;Utzh*@}BZ+-H0mmW2XBm0GxqCH{8WRu3bUGRNcImhZ1MIt2ygTmA-C_%C!3 zUXy_fIaqt%T(-&%nE&b2_;Ew>5dsJ_RqLp}0|U(4M%WJ+HzBv?-nr<+Alk%y$AzPr zJvLR+y>zuV3x^fWf2g9t@d~L%A&aB;Uz%qX8n``{uxT-sXxm`3Cv>m8eNx!|lZ9%l zWYoRVbnD#oUBsm1%4_^9Eu(DYnv{~{3myL{J7Oza@^#f&y*tsx)6YdO`+F|b!b^55 z_W_{IX!!WSp`0Mx!B!~8t0$57%p)sxzB1CAe?gSolfv@#aT{gjCFQVNM86Rs`D+sTkajy;tuO0GK~K>&jI)jcmiIuP zT7V2H_65mpx=W%NiZtP|snQKU(%!UFgjXNI{)~v8~qNnkNHt{lzsQiDEd=@*JL$cg| zR0;I%Ox25QYJ#KLJBY|r_O2c==dw{~sfvx5nJY}w^ux;B(IP_{Wa=L|aeKsSQ%lva z@;0sWq`VJDIjTL&lom&fG8%7~mC}?G6xX&lftX*8YtbUXeknTXg0=Lasw4*#dhrxb zhMIWmqXTFKJ|<|inr!)*__i+>h7|L!aB0kzAsk_E72jfL9_0+fpwhw(VGO<0XDCSy zefk}H=)z^$9ev612MjzVkQ;``(P6SdX||)3wbsLwwcbkp#N9SzT;w7(>=b4F9n=#C zonm~^TvFhB>-6%BMe1;)mgWk4=0k+p4s69Ebtmyg!{Ajbo}ApcTo76zxc-9B8}gam z>%rFRJN_xwc6Fb~Kd5$@FZ`6`u)$r_I-8pw!IfsyL&jw3{x5IK=AQ1TPMGlU@lI&* zotLSe0$O|xlCE-?Ms41hi0}z##N-DiWgGY510qouP^)-BCKr)fUI#0H2X&hOtEYS2 zohT)tkdLjGgzC=>f5*E^x@Cl*Ky8PV@UI=;Tv;>3u<&NXq(*17j=`U-%9Lx1SXbPM zuBqUL^}hhPsI1~pf)=y>2Wt^D!2+aNo+Me zQD8b;uu;8c&Ll%%lAR@`(gnlNkdsIBN02Pu4i=8zO^@!o>EQt+35J_Lj8S=G>LomQ)4T~diLu>k99T|Rm+!Un_|=h`pnR{kzzG>vt`BR9XMM+ zsVpP?yS_$^2{L%MEnD^usTsx8Hy&pJZdkrnpDjVxm<8W^h^@>`^Fpm&?jwN$6i_>I zTr#gkQTRKpHHf(F>BGnR-<$S5)5d(m3gRe3+~Ee-sO_b$i9|F^?CkptM7%QJpstxfNK=%1yUFY_q)%YiHO3E8|ZETpWix? z*}keuH~WnrDlsmxC#IJY-;9)xBKuTXB~Cq&_F|ftPXy#2Dbw^$)DpZ8C z)NH>oTxZPlX@fZlk6P6&aS}@b3N%e3q2bia9PVg&@d(UWl-MFeYZ5&gwptQeqAtN1 z1zl4lW@_ZNdjZOnAAmA=Yc;K##ra(7PYN4TkQuv>3-;{P{-dr!-a#fkiBmJ(SS{ja zpAu#;sereCs?f*Y3Fj61lTXw5iyWbUZQVsABZCL!4fDKzn3m=%4;`)oN65>+q+CAZ z*hQVXsYeNA?uvb@SW|qIkTD;HU1-c_K|CChpN$U<{QV-L8=;PsLG8=ANRf7M0qa8fXm5Ju8UWzfCQb$jo$Dtdq&z1=OJV-4L3 zO@jC(!qY2|_lipHYPm-qpcHKFBCpz)L_F8;m483kgj$=RAT@TV8bp9z+>E>PLOQv; zz02@I`Vzq_(n*5a)2QnRTeBL4(TwQ7hvQ@C($Ztta{EFah%T@S&4u9;xSw?$0E`*4 z_fceBKdHQ~xr(S;*-tRD{Ls1C_xG((rLO3aq;Mr`YEYbBm0SmY-S}sSXg=5KKqj0G z^UO=vp2rt&Rx3XdZ9n ze)QIuPhl+wojyJ1^@y7PBBH9GpB-S^d&&}W2nh?l^6Sw6deR%sqn%(nb9AwlIkR)eZ>A2Tnv5Eo(-_bNn;LTWG zqW3eoj`8HqyR))f*tn5 z^R{hp+oCWp(j@Wu#L9$ud{JkT|L)&pcTOn&H(7)H#t+Nq{VsV{*va{FRJCWOCNMxX zj)2jH7gEoo&GGUYcfRvrD93EedBeo%eEmed`d-q!(W7k?f0haH5jEK@f4sN8lt6eQ z!zgEPWqRK#IoqcG1#~S4)Bjrrb@~vfoIt4gvf_bY?YsM_}@%+psSlbq>dGXvRN0!iVtPL`-&) zdv(F8v7Rlw$}Tj}ooZpCBB)ZtKEg+98+|LvIuPxn(B)(g>a!G&r&8v}v14UZ)8f0W z>;1I=vxzr++u7@y`D++i9OoP%U^zh)NnP)={Hg$veHSeEsftZ$3eY<{icI9gl-22n zum$`~XZ(H;NP%~QRba=9Q?Lg7pDyDzJylqQ1A%x+|JP-u7HzP==Y>mwv=;xyz~}Aa zs%5bruRj@82}C_zHm<#g9a3t0&`Wzg2nn~@-utfD*UeZ+ni90u*y^VqKSuAis0J|D z3@iTB!^}vFUuP2yxr){5*50M=!xf6oUr3u;@s-$;^TDP$@_XtGzK|$=>>B#i;k!isutc<$cG-PFId*nnYHr9qXDT^RY&zadxnlgmtY(vp_7s~}Z{CF3sU?QZ_@xW}$aQlp$#Ob<;rPr5~*Il#3emk}P@+`ap zs7>ZV=$oo^YfYxQBiuS5x|Y~v=d4oBOh2ezlNfBrMb-B}jL&?E%$%^VD~6oW)9p_j zgJ=2vZuw6oct;<;9vua7hEMf9c^ZwD`?5B!Ri5+9;)L3K;3|C0$4fRw!3@FZk2zIF zj*`lR8?R!uh9&=ynGYYdyk)!~LEV7$(E)$^)uy#okcxz@RJsU2a|c=d5yUk!;>K^| z8PM7jDxxy)UDBP6R9o;-@6WYWOEIC#SpQD2UWrGz6nin7#C1l1I6K4*mILS?^`0~s zny*85BO1s)uLbWqfOIX|FNQkNgb47Fo-8)J+j0Jh<6@Ieeb!D~h#mS8!|ERj!Q|KL z9mf82i{&{%U)~28*l|X3`Wac`TS-{m!4{{8;!n_<#(+(Ky?vUYT;KdB{q<3aJb0xK zieMU%6bdl0m8!8*HIvFM; zE>=U;RGL+w7V(b|g}7_bq@_l;8EX5|V}cCNO%wi}C~gs`8NpWRmLd?X{t3!Wg;oh7 zPC?nJk^6p)-BoM1PQfLrK07%GGhf55tE3ppTbQdJKwZwkf}Mx|5}EI3unG+f86xm+ zepaiaYgi1axtQ#iu*q8qN>JOu74mfyMVZ>-_ideU`FfQb8j+;v03r{r^Ex9Rckx4! zKjDI_M)UvzBd4Iz(Lpdj*RDZW#c(%wft~Pq8@j|)awrEBpr^{%6(U+{ITXSZD6G?) zJU72cdKMEe&;i;DXGk#p0RJ!JGbcPxAroP8?1&JetJ>ZmKeSM!-j<@pf?Z5m*_sF$ zlIo_h;-=_JO%%Q*1$gwj=o0M{eY*$=XdFFx*GO@(r(6rF#SI+i$dmMg4`~AHWJW{Fm5Khd^3e96A_bToIV0t z)3Av^Ja*y^m^KQm01Zo?V(q*X^zY4OhjLdN&qvj%kx`Y0u-|`t`kyU*ATCuo0Ge?kG$D z=GtSRbY=d0sV^W^QW4+7C>gnjEincX9)_r|nH33}40jXQUbb!%E1Z5I928bQieCFA z*n05W8v7ctlcBi*#No~6%?;!}Ilq-lIW4(7O4_YYT^po^Q6|o2G+y6MbJyE}=f((w zR6_&9gCHLVo4@LsS2=B447lr7xITA$goo6_*`kAC?E%8q@K1Idf6u&d4MHqAj72Yp zTnfK)K5kt|Pqq7XZk5kR3&nZsrlUAxr8i9q`4VbAj(BO7TB6U)B` zbUkDU=jdJRb7_2qn^d))0BwnRH!5=1|II~ppp^`vo&U~ooaClQUu6#k(a5Zd#I>yQ zbAWtCIiZf9vwu=4hb3NSO!@a+Eq@I2)RP}qNySHu@XMqAW6_;6r%zC*^f?q#N0JhQ zB~qj;EI%pP)HII1ypU2eP-xkx1!hu5LW1-dH!Bfe=<@|5XAIYcS6`ARo$-*({{0(5 zeP0~1K=?0t0=%qV_I4GT5Qj{5%r{_a4XV0>0W5};_fb>kAJ;G2!eW2W{C8M9yK#@2 zFY3kZb9a3HN!e(1`pdGZX#h6g+1S1R`lEwegQJbDbZ7-3+45VIrGBWHmWG>fLsb>N zlAXTV2fe%u)lUr=5=)lJ_vcbVsqW74=yv7I|3tq=1O0bW3^~n}y(CkV-raop7r(z0 zh$0*HEwR8p!8c_tp$W0MkZ7p3#1J?}S$@?u{bcF2j9IqyF?dEs^wmY(dC_v)aj+jQ z@ms$H#4t&~>9emowU4cSzv4;f#xl<5cx`u6)tSH#oa9UNkExUr;K^0#E^R-C*z-OI zhKjO@YQQk|)Pq8R_gbzCjeNy&^H>B&?!MTHmO2F*ZZ5zsL=E8;MUti(A+D?^M%PW*h*LFU!X3POiR$c_BbhOSev1yH|yJ+Yx7}~B7^9#S_gO*+j)R$3- z7+m9%r&i*2ktoNCv?npqZGQwTRl}uDiok0$9iZo%2M>mSD{M1bf=zDqRvv#-5MVB0 z%`z)TW@GjjT7sTMRFq{XK~m(_a$5AvOEpK(6q8mdYgCvn%lG;VK8-j)zP{+M3ewa> z)73csO_{C_cN(Y$1uag}8TMhO$w+iHE@Kw9E9_`THMuv=089!sjLgWpPqJ!NEdlNC z4Wx-qhp5rxvfl-6RO4;0C!zYt;>R*zZYiqA9^#T!dnWf@|P+vn|ZgnIqa8QzGHPZk$WDTfVy z(e4|DFyxnIv6NDNb7R9%&+9k_#(hWp1W;b`_4yYf^^t+twJT>uLqYIx`d3-pk*$%z z7bhrfRw99ENw*_rB?MEx;Vp*1rjYI?iV_`6&QdVmYCC8_UVYqwu zgaj)~)57?{sL{>iC;77YIz})%`nCg=XyV?~Z67qW@QM5EU;JFGj+L$*Q3g9+D@@1_ z4(A2SRUu!O$%*GzzQ^QhA3WrOwxI@*_ZVqH{4Yk5uY5#db05eVl6if3fX)qp0dq(D z*pZOW$=vpTM6i=wyc)5~y5PxY?Nu0F;Qo-{_nLl(N2_d_?<=40jhQW@bz}WklD|?% z8U4wQ5@@R|KAYUJohT|GHZi!#W%$ts+fc$2Yvj^!|IJlXpIvGVb|IOj9CHATu<|#7 zi^E-9+I+N{??B^us=*M`E&$Eipp8763U>hAmghU!)Fh4W;DjTD(s7#T)UN-_!X?;s z0{XO{JjKqj3NzHTnKQCZ>dK1dc$mO%u-UIm@PHr?l^nmmLQe{qj`9`wjXnueK8GiM z@PpEm*cnQQZtcqiy0lA`gyA9-+Xyo>1P~6v`}KWY0)A zYpa~VQ{i?OC%8S8njDV1G}13)mIn;qW)NMDpjDZPBEW=t zkw9Y6+KY^&m9iRer0gRoEBd(#8{h2^>&QXkd&wSZETHgd8U7pFmqkd6UDp@OQpHSm z_xRmi{CM)L9t4;h$jFb6epGdn$_C8$Z^`CY$*SsjNNu=D5d`l;&x&Nl02W}G57Xb+ z1QIkM!qlvj#PVMSP^%85V(`+bAk!p=s@#rRo$dcBfL^l#WFu0(bH&F9ketWZ$@cX8 zaJ2|89M*1$&SyzP+r6DYV)a8vY~G;xeI7oxyt_ii_fz#RE5|lBk$~?MUzJn`#Dk$G zJXPCkoEI5NYW0iPK}9My@ME95FGJ&R(?;;jg+t%hFuau^!K#!8Y7^}d6o2p~qk?Uz zKAW=cxHgmCHzGD3ykD9qR7gF@FOG8>T=Eq|y4n{`o(JU?o1a$%|6P+6L2W{MPT35I8 zo9S0vQ_tnBf`tCA5>1lv^}g8fZtfa!zLE!ss$_GCN0^ZLp$rq(tN2RAl6Ml1QGWaz$kEe0A#>Vp z_&HzGW(YA{?4(Y;kfE>-p@@%)M)QE%0XOlV3g|1+w{*(*U|{8dQr2+IENY#U(pJkE zUNB^R&KM`y;%;ByF6ZcH?P1zxa0Ne8SE`CMVwefsvvQKn7CLTNO<-32qf>K@>Pkev z*%jpbpD2nVnAbN5HvSb_HA{>3MVOdvbKKY{b-q4FT!B-yK5;`%qCSe6la+d`9g-l_ zn#?Qe!*0gVG%)FF|0`=`KFhI4FZaan#=xstAR7Jkrh*245j)dVl3X+G!%lIkCgTBP z?go9Y-L4z=SD%=}q~_WTFt|ZpeY`<-p~OO9ewwEtcw0Hw7&3D0%uaUiGOv8K^1dYQ zj*$H0T#6G&Q%bI*7=h2tt_(@XEGxkKG2IgPk(b0@2FRI1Q)jyjKKSrnh|qS^n$)Y6 z#J}IX{I+nWK-{L0lA83J5petyN>;(%ao{nIwbvT^F)b0UkVbSq^XL_dZO_e*fsI`S z`bO18;hf)fEPVST(N6Nh>Av;R-3z^&Iw9(cQ0_hLw9(pqNJN&s+YYyvr%hSF^VK=U zdb^*nI8d1pvV{K4iPC`EX#<6G-u`3QR;I$#$1HT(vyYK`DlXRRWaG&T*#m9#8A|Zx zh*uyGWu8lC-N4z*V(+?A*aZWQsOs6jp=Q>*~7S7>hN4#D+LLqD0RtxStKT{yfHyu4vK7D+dh!V6?TZI(^Dr6|#=aMp`Pyf2Z^si+#|!uZ0)l|#zU}lwK;wN7P>3}Ibd9vxj`JY-%zS7>2+aN# zLNa~E)b#>OTu4=IkGf2EJ$0NU5=7jLM;M|1d5GF*^XwN$t&Di4eGOjjS|gke_~@r| zkyc){(%;OJXt{kJ>4P@A+G@mjNmKmzn!9EEin{(r@c=+T#WF7D!2T8l)Kv!oJ%-*v zKw;KJ51R9v=Urfm7qp9=HOfnXBGys6a~)NL)(?7$cFs>f-#_LJ=Q(jR0VQAr|7!gq zMgpUhEtcZs(~+JIk8DMz^A0k&_C+27%Fl#=dLSSS2ePhLrU54z%SiFu9+2F>v6cRlQjSFjB?;`f(%Hm*a5x=csquwQ#zoFjW_{>+ko*Uy}Ezixl+5t0Vr z>nc?9w>qWJze-bX{#C7#Cn4=G`Jg#`G?T?-y3{`ttBER-qA@EMUG7poh%h=%B-xWu z%4kscbMcp}(S`l0?>O-F%<|6gB6amH#C?k*5+H3gw%k0&eNgZB(8GvsXL%^qoM06a z-$=kZ%wfH#-zzThzL5SYUF4N6IaMF|o0aT0T~l=!>*n8vPhE8DWVDI{FFKBK<$mDF zH>`lkoDYrlGYIl*FaZmUxY_=w9Y#p(#T7W3WZzDqz}YKjPdU+*E;n;vv$ z90tCk%Y7yZpazcmE;ap>BRZ}Wfz`hLj;AGOs5?@32#5ybb;3&=0@5XoeRG9?9CBmU zYvyVpAUKKFY(Gu%{Acul#V3lmH(Ebx2q@SzU^2HRw=Wm|Mey>x9~@#NG}G$3bx!Q1 zz1`9Ms@_KqkY73!lgDjj%Wu7vZgj-_v1&*QcxQQNsZ_!yUDR1UIBNS@c>`lXw$lA# z=v1_CvF>bje!Xhtc6ryV_}?4-!Cq0O>?2s;(EYgK5vUzM-M@6b3%lqz>A7s_S-(&# zw^!fJJ;iu-_2N}-d~cEK3;%=X1RXs-{?SFJ@8>EU3xLi_tUt)N@`&*%Exyrf0?(cP zEgos&QhivqxoSx8wOMWQebPyICtaiRspRFr##-F7`3wJRMcpggUGWsy`rSzBMZ91^ z0FcBCVChe31wi{LJ{!L^hRf0*B7i*Dz<6dni_QY?R)4G@ZvoO*u< zghHTkQ5r0Q?Wn5+EWtw1Y zJkm-Fpt>~f!9cMEFA7wa78tHQ`1XP8$4}BXAt0Fp?y&t&DVR#52cz+$P`ohIe^8mXDg@870JGU3jP>zAZ2OX{wh@ z&o(B8WK~Z7FLi796kB5k>BEJ=uxks*j2nD>+NJUhmd%nI+b9S1!k}6UC zHfV&&(D<3nY=KPZ!ya$06w46srk2fE|CO&Sv6O=Q0oWD<#|Rj8{C;76n19Cglbvj3 z__<H6Lmd%}0NK@bIAz@f&wi1O3 z0oh&bUpZI3eO#b;bH;m3ea}(jWA8EC>r3)}R9M*Th@;yPguOiguIbK=da5}vMqd>n zuuT@pgvg`J~@Tcj5>EiG%`BekNa1O=rNC9Q3Gp9bLJ&GKoVfpS-K9 z`eYUZSK=oh6Kg*Zx~~ue8Dm}!#~HaKe5fC1;_o^S$JQSiZqR6htLa>V|K0XT!j1$rs+Gk%cn}k>V0^B%f;>ru>V$4E7nv? z*+yzsa!M{8$D@HysaMPiG2%Fe1L1;Hpg#X^hcK8!bNOO~amm|vj<9k-oEy1yX;kP~ z&Xd2XUrw^f1z6p7KzVugR2AFuxVC;eT$*h@$X&YBAKdG4Ejn6K+>+0XGL5(@ThaJy z8OfIm!`z6I@#^}i+oG&NTQAHq?5STqbz=>MLdqpff%Ox3@tA;6#A~L@oISNra3Qu*0d%Yd@jpjw@{7JsQk*_b!>9e!|O)w13*{R-2v(dJgGCrdYedpBfn@U zELycq%VCY4A9$mkx93Vg0lI{4yzIHaBoFt!?X>fLSZsIaFE~}3V^yVwhqT9USZFu{ zBFq=1JEvn*O+5aA(=e zx_IY}{==yKhCRle-vh@p+l%VOc_fA5ymbG1m@&p1r58c(8*juGY{q)R?I`Qjc3(UC z#d_Quv6obj^u!I^gZ(C3tO)-E*KcE$3;W#A7m7TJOB@FG&Hug(xB0&5%D)|da@IaG z-t%#MPZ5hP@(+%e#+%??=c*UOnL0o@q=T%T-Oua7v8UV~+ZT*%B~aCu8%j?3!0WnE zg?jij-pht_sD}{bcFNYb#&((F0V6fT9f#51&T=W4-kt5Z3GN=pvNc9IAAg8Sfq)J& z#(@h|Fa*ScQ3-y$)PDEVoJSo|0QL=br3H8isMFjVA8PXeM~Xb@mL{J9RUI5-T)VN;797CQ>yJw2na{y!^4og zt`OTc1cVF$bvHU{p#|at0zXAR)kD3PoxjfuVSzv(Cnq;H59_asa{343-zqF`MBU{%SeEVF`;#tM1Xvp8N>nt~1Wr=+c2aDuz_oaP4mNVOt zWRzz^RagCLQgeui(3%(HaOL|Ehg4=Sp$l?JbOhBNJwiDS8Y zO#Tjo@|Zrs+-)|{(eLfSIq}0!L-aQzU(^ttVIPS&^-A}ex2C(m^KlP^`(NOfaL?|m zPzeUEk&=Q=mcllR!|$A8@l$Kmsh%)jKh+I1b^B;C^)%3QWPMwU|5G^l7e&UGcOzQi&YOpSLhh%RwWa|F4WQ4{GWN19^sM2;mf1jns2@w?`Z4la=$+4 z&&4-B(O@+SIv3r<-I(gcXi^Vp&Z03PzXqxA{ym(0J1 zNjyPAd^U6N`Ph<{%=%v?i2-axRhUS!tcqj+8!YEiFHx`d-~{JE?8&uy;}sEE*#`}N zG~n*dZHjqW>TKzc>#GPnBPp9Ah+W6;(kA&kYOg0;zcf}ZnPl>X=I8xHzg0@0iHO)))v+fR@3RV)ltlQ(LL+R*aP zkxo@aSG1fk*xDir(@WJLh8*s(nU2|7bJka9M%&}@aN!h%b>L_hyvYQS93yiicHQFT z{Md&bdDNX?_%m8jCZsvEI#`Qb3^!g4fbj%rvUx|Hd35BSoX)q82NI@f>BWTaqpg-6 zj{DC3!gwiI^sY_g&4PF15Slx$)gc-f(tHi@lBCuzem{(LRRJt zT+_ERPHp{@jkCSw(hw4;FROiatL&$Oftm(3RyJvB0XaTi^s4)(0RhWN%?tLeC+fc# z^hQJc2ERIZF}=lc{U*vDs@1O1!yQ3Ij7T`wJk-|*^5<=&<>ayDv+pGpk~-UB9I1>i z)HEd!5Ip}?7>_shno^kCR7|8@5xM&44haR-b+5THHwT((4i(;2_W1qa*C!)O2HT*1~-MM5?B~K&>Ipmta$p=UlN&57qF?qiPx$ z&0YaUOVd`>%^!BOZ8mJBjo#5E-x>+3xJt>aJvSA1Dykr?tt2oz_jY^Ni_v{mC+~ar zT~*y1LW}JTpq2|j^}W~X#rIS!)@$p8h6`ANAc<>H+zZ-Jlz$Y}xZP{MmFeHx;5p1# zXCBUX8?ay(a7=rRHOK>XF_*<5v(sj!lO{XFz*dVqeHnMGjEn%at ziEkzLzPu{O{;GvDW24*&eQEGW;7wJlQG7$so8)+P{lJl_4~L|hD_@K^3#4giuKr^L z_OZc!>;xoo02ONtnP`Ii`2}ySgCT6?09&NN2{Mua0h%WQ2c(s}BJgx5T5uu>m_d39 z&_zTPIr2(|h@IuamL%YSeFu*vfkVjqFXd%JXc;o+Di@rR!8Ys%NKZxq4R@ZU`=>IM zwg}y0rJh&nq5Ype5_Za4m81X@GevMO24jUT=87k27i6RXb2y&@Hkhr(3uPelf#rvV2u zg*TjXAC<@CeTdW0p(s4Sl(EP#Ic9$m#S|`2%2-O&3WjrWsFMNoaI2TE=mo6d$FpexV5tQ#vh(xABKGJBT^+Lq z1uQYjo4%seB0Qe~T(Cp1CIh%)YvE!BAehQ_5iJCR!J)nvdI}cJ1wnEqz?-hDTj(J| qfI_lEgyJcL37Nn}WhV>3BP_6agP*6Gx~z69`p)QLFmvYsD*X#N?MS8o delta 18371 zcmY(qbyOAK8$NpIZV72=B&CrSkP;B-?vj-5p*sYWZlp`PySqWUyUrmGc`l#t@2(qv zJTrUNnl(G-efFCh_M7kiH{V}OGIYJ%cD+V45J-wI@f|)rn9CffGrt ze2a~GPvP7B9XY5?j>tOZRG_xjtJ(^;5qaXso+s@B@RBt-i-pjP%#_%7aJ$457&nB^ z%&Wo9_MLW5bW~43HF(P$4C&U&rd+ezK3Y&C#E3-(m2hDJaaNS$<&qaAiWV zC%vZYC+i?+iPREb(OBq$Jlu2Q@y;*mDU_QbYo;|$xO1Y@@%|yXl|wIzNQ*;X!8*`< zdT>6fH0_CBnpZg5J+10E>u}e2II;H51%9b6tR%J{cmqrW;Y$A`OF~LHZ54WNs=SXw zPohNX2X8c~5)Aec%WE#34A}B_IOnoj`+>VX?*O-i`wwjB#D-c+FFD|VDuU0&~-RKh? z;H=`;pXATyXgM-LpSvWyzf2KR^bt$)?`QUqQ>1$7TKdo5Np&Xq%a5Z9ZJUFs6-_UF zb_zqt1r=5XnAo0>TiH%a)U7!())adzkk~(*3NpBo#1sl~qKblt z<|CdHN5YDQ0=EfIw<^K2@8rFS=M+?czF*|5g<;V<8RDOt2rT7B&;|PlMh)1H& z1W{ysr76kpJgI68^u&f?zq<_1rgL+@k$f5m zosd5xd4#USF3b@rvT&7!w(mNS!GK{7{%bP##i~ZU(&pM$3;n0D^M>zMvJgz*E0&Ig zpn$Gg;!twr0ZFNZpoeIQ9-J7MEq{hErbAFAhQ4Cl4#HAVi|Dtwo>hF?H{sW)ri-=# zbPeY%s=S5s1Di2 zqJrs;ko{5jK4opC%hv`uVp|}$UbJ}!l{t1nsvUnm%}RlC6pSGAQIz6`TTDkvUdqPL zaTHWHy{lNo^AsEN;ZYQG|Hf!h+ZW1whsFNMln{hASo>!~PngsAhaO$4{4+cNXCND6 zt=Ij@?@VI=od}OZLi}&J?1<+WQ!M5@Uy>zo@94>S1DI5OTM98uZ?vksIDMC;M03l>0gP%ioHQ> zASI3`>2oxa4+}KyI0I6Gxy;qy+Ez&?EAqdCqpClSN3c3 zhw;*sj*G>zOR*_&t)Se0H~x3;anu}++ZSkGrq)mBV;N7y5mru+O?ACYb`Y4T{?;0G z+2pR$WS#xg7xMn(>CAW?CVrouJ$;(95mmRxJ3eh-rWv-w)$0PZt^J%rTNC4Kc{^CZ zU_Wy!Mf1|_aeME8^8PAwXdu2CwKc-qKm_e_$qw^6CSU<0w087elFxGPg=eZ>f4JjaOT3FO zns`_~Jp0so6W$r7YB?z~n8N|F6)%qPQ-jOp+lJ)%|1@t!#wVdiWmEWbb9Sh^nz`J> zqz3nH(T&aKj%L0FVbNI~K6p=yuZL0Iypgy>B$G$HvKb2KVWL8-@}~G5-7S`aEE)nn zY$;mPU@2NW%F)g#QW8`VF7|V*DQeXJ%(qq#wN!{)uKFZ98dltjF+Jqp`x5^)Ci74^ zQ+n~w`I>Vi;^uZX*+}ZT+-@Q>-2Hu_l#bbNwvvSBzNjKW-^ZQY{?5V{`l?~tJ>i3Q ze*N|LFDan_xeLo)Se1RcnLNg5epdkRQVTqh_IQbg=)SF-WmQ(-c~DwfIpB*4YgjTk?5 z#y5=Yp=~cPyUN#}Q;M_td8>%^Nyo~X&ycn~F_TyjIDbN-3!X?ot~_c}C@BQ<#hPLJ zYY9QhA!rKrZ!Nk2gW^MatKc>R6$#w$y#YxLCeIk%)wO;;cRC%zTGs<;(3vrd{oLwo!NEq-ga(-^`*oMDN+J+7F6A z_95*DNhe#&Qm#c!EVp+Ap6YA5q<;GEy!CcJ(Y?jT=Vis#qQQ=$k%5bQ->Q)zp&p=i zyOPLDOAU~P5gw%z|0{GOK6boRWm}w|&%%o7BsgMlY?d(j-9QPIl3s{YcZ#v-Mfl3l zJ*V|idkR614nZ@Ll0>!Z_veDB{zb&ff!fOpmF5nES#oswy&MsV8n(Yo;Kyg$C2Q+h zX{YK!@{Jn3AUf80EBG%6BYk)zes)8bU(K-5URR|)1R)>*w-Q%5N6rXC^*yQ?L6!Ve z<==9b2?-;6U*c`K4JmEBT;sk*hC-|ZB+5Xk+5=CbZLV)415hAwm{OeEK)osH_wj}8 z_u^gs0)7lVt~<;8RBOZ7Q=9<)kDV@=)AV|b-lA8N2MZxzy#(hZQBN2|Q6HLLc+f(^ zO4epLPpv*!(U0RCV5~tu^5{>3QXjE zMi(l-diP6Xj@7!0P~tfUdMJOFoMN;USw&Mie6vpYOBRI%!hLsV_qITVd6hKOp+*24 z&*jsKv~P*whTvdanmnwY^n1MMD@THXqNWlK$|ub$0_FLP{)sKg5*U0UoaCEiG5`y= z7L`8i9Tsm$*VG6C2sDlX0?{OHeSDMn_cJxXoX{1|^{ zTn|Q}{6@Nnw4YZ<>!AGm&f@7fu8cvot!co!dK#7U`|qvz#)X#QK=F9qioY$mSQT5k z0JB*LZ`;UY&QB?0VM6qo{|Nb>eQ$<9~?d(dy8=HCWC9zV!|ZuGMEQgAh*m6V{P= z{%mXa>|w|VAPBFE+Wqmn+#&8?f~#NOM@JNY6r8Y6>*;)X*}W0^0&L9Fl4=#qp%sgP zKHrLoaEs77vEo*3DN-Vqy7+D83dPC%gj#R&_j;FDHs=xw+*MUh6*sB27TLF>S%CIA zZzZ;On|u=>a!PTr!NMNvGeGN8h^E&7b13%V(Cp4z%UyyQ_~}6FYit%p`kD!GG0-mI z-~4xi6m%6)LadNS#vumLjPBvhx+~n5hBq&1Ls_(!IfYXcgTa+Z99_Sm& z3|3`pK`p^~KTZ=_x7CWy$8a z<&ESb{5$qt6z zkapq2e><()X`CyVntB3y)dM__i ze^L1u!fbA0J7Kw53DL91({-~Ite2&SC0nZV)wAKE!WVu-{HgcHCqJX(9ffqObN1twj^f+5n&B3I)ko=u;?y|q4NS3#`Br^N73HR4?N-hVj1}=e zWdn=)JPN1z|6EsLv z>3Lu90!7fm4;s*C?@mm)bD^OmktbtnyFPCZ*J%3POb4p~oz<=$Up8imE^K)%7C1Sw z@bfH!ye}&+n(KBzi%{jW{ew{K@u#immr!0mlJRRB3=8m}*WQc3yDcltOAF5sT55zd zBIG&Ss;tdm+L}ZrDx#}{`L-4jqjb$9I2)T~5RJ$3n~^oFx6diDLIvDYTf*kNDJG8X z7WXm%I>?@-*J0O}>3;C4f-KO=PY74vJ_s#iQSnkGwu6)aUTr7hT9Yb#z~8Vvoc8_h z=X2)iVDWzf@Qxm!=Qm{o3>Ejr2r+9z!T6XlVhN1NXcbp+vRqP;I(c_mvOV$G$gyzZ z25>_7k=&mW8k%~^VK3;~dttQy-UepT1a~5Yz!d-R;|aPEwB!pg66PZzZe@M8(HB?! zGL{{n!UW0#M11qkr9#zLz6bVWgrX4f{syNs%vrbP7Lz4#3*`U04yJ^wO7%l{@>Rdu znHEtWcTV|_hjXRB z5^&7#oudR#Io0>0Ke`xv4vdJ$s=x@eZy4`oF|a5a20ONPA0kenluU0s?-#miN^mOg zqpq#Ue{vbTJ#MT5V3%hnVwQ*MzS2sy1gofv z@RiF-aXh*Py+=_yoQ{c}wga)*-KiKHinps z^=&6N@?+7H$hp1uPfGG`9?Y}s!vgI6jqXPwg5M83${esnV=O|nwRf3gnTQ||a$=4i zZDOMm0Z?&Hj3Gw^sbKo482IrKE?R1^=RfWmwD!Y4>Yg2(CcA{7eKvmMSIzug zK$zPyG-JB|(XMwmR8BqWb_Q+z9^2TCxwOT%#YI$fY;EUft=;(ucja&Id?X zROQ#UmRzCg1w@y+(wyW8`IAbpuzIDIBebe%y9Z`;~gFltJ^1d2biBS*b%YU$!_Nq$OA+-y# zf~`08v8xPJ5S`R=&j=4cuhSai-v3Bx1FjX4%#EYr4>PGJd^(c_>RL3{2)%@e=CRu! zP+_5jI*tXzUW2I+K4>tJKe?a3uF?lntrI^z@c|yMp0^>&HFhIEh_Y(9rW!~?Ps;YSMudLvNf`nK!pK$1$>o*HBZCL|qDpaBtC@GEbQ^@kX1=h0nATJVD_~ddgVI3VO{3CNluvKz&IaCDHvrq$Jeq)Sn~{uk;>HTeM1$AIGDw8ZFsOQjq+O zuJBv`?w0xIFGpQ|9K^Y9Gf{&9Np?C#?rDwizhsus?y}tLr8nhZ{hZL!CJ55>- zYXoY3b(Sh>x@WP zY6g-4iKj$cRW}G!DPji@CTS_{e=@ppACmV$*GT%>L89ZQtBNM1W${wLiO@+kK4vYF zpc9vZ9JyMVKB(#Ufl$~*uR$NGjD_S4f-L#I^?qi!b~c=6PwLY z2gKu7Xrj+)Y>)@GApgkfiDz2V60PYkr+g4`lyY9ca_Ik_#s&Akb{BgX%57{wiCl3GG%ebAQf%1W^i0{s3=%j9Tn?X(uw^Yo^k<>8+ z96aV5yF~UrMaCq@Zp0yJ`i9~}9Jw2TRpLzI2l9#|Q+PCTWM)nc-tvAou zMW$&CrF)yX-=l>mvDDb2o=qIyH& zTG^whOd!*%7Wiy~$TB7^66h&1Qw}-Co3r*pN8u*Ek^LG+J#tj?mogwO)zaDz!F(#& zZanf-SLCaqicSHk(!nXvp*4aNAp!4~PpN_ya01moq`mfL%6gpB!lS~U{qe7!NPF#j zMG2#OWk@AyGlR-nGtq^VTs1(&QjRzA0r+5@4fg|My=9igpFH6}XJDz^o|Jx&r1mLpe0R`{_T z$k=8wtFT+;iXtn!CbV@bOoh&g)ASAJ1Ak*qA2}8t&wGv}&uFEXqr2abpATMhwUHbq ziEE{_@4Wj{QZw1b^0Z46PIx(@U$A%A9VKEGdk-%Xz!H7pm8SX)|^VVwf3FH0kI z0shgyL{c`tDEG|+K@m-P5&Cj>g&NaSQc=6{k;GpzaBL^tyfDIyOSaX$ zC~7lk<~8z{Xsu=|BKcKQRLeLW!QAS&DSHuZTg%McZuJPZsNZi zJ5Ya{v?=hWhvs7l3>EM?HmUeOU>Gd^a;c(>1;he-V+=-|0BlqgV)?Nkc%nJ2 zG`N_Q)95&vz>57J&B49O*acBk#2^_5tT-lg_1cSZ(HFBvYFzM0%*@|YVy6jdRi?bW zmZ{>@q>re#mWfgc& z{fB}P1sBBF81WSwd`2hQ-!ARi|6<=u>THjP?FK-qJ~b3#@w-O6xbaXMCD8yy!%FE_X4 zkYrZJ%9|c7=cQeBL9R@K6&{aN4Q+$ehn7FVsIu-+A&^^BXayZ~<4I^xJMFU&<5 z;Y}mI!t5w=j(RvL`B+Z#gv@&P?b{;;AU&uu zpR5o_A_9alys&x$mEgps&ew zjv)xuEsAEt^<=v7ys2HNu^E5W(aoS<@@=7k#?jya6AM)Cyz#t1^JJam{>%PJ!E~rW ze2Kl@O3BW2oP*03@0?cI;K~z0APYO}VnxDM3|*civ@-5<(5D@b0Glw$c3B6ssJg-v zDZ2snhJv5#TVh;;BQDwj1(6nirFgzqHo$uh-0292n{ob#MI1^k87N;0IQ`b0$QP*?LN6;1ia^+2VS0Q{#y}!GrUAJRi#lN#h zRJgCfXF~3!XH;SH&C2dajO>W0fh+Rf7LLNTw?%n3l$*GyIV|}J({3fAhn?FloD4=3AMwQLW2UL63ZO zNbfQ|OO}&x|MC+2Y+EzR96=B|w9%ZQ5L&Q4Zjo9eDiOmBfZq~@uOw)v(d&`5M`&&g zBNJk}GTlP1LQUp$b9@%j8^oaa^h3{Mm|M|b(+DBp6pW1n{%kKL=l9Lq1I;Xq|KE=5 z8tRsB)c+oYyT~5aA$zxJKGZ)c)~(`_yC@AEsv?rb-8^;>+PXP?(r-RfuVg~Sr4PBY zr%V%9D9)EF4s1Xl59<)ED6JByjotu1`EV5!7>J%dHz;lBf?b;qO)F^^M|djOy$WVd zczjE5EvsL0cUt^S7|)&$Y`xCEOthDnqs*4S^!yZGvR&ym|fB;Np7-~0B{ zX&iE2QGGjwneCg?;$P#XN=p9N`L4by5*elOv@xo&>Sz2gJY_YHu?9`}<2o5cofaFW zArNCaTI;(Q8Pru@IGf6!*!lDfuQ%`1-1%&B(QNB@ zR(w<7Lhr1$;X#gaPDiS1I+zXQ?r%?h%tn&5;k)($xMC-^#wApi<(!4wdWC_ zm<_#?DB<7gs%&#|I)p*DzD8d)vzy9UWX*)}1l;Z>a&Br(TitfO1J<}nm~LWl<({A-iNm*UWdyp&cim1PxXD| zN}Loo@ktNllWlCY6um>g_TNe@j&9q>C4<>K0*pc7qO#-hy7%PY8zfHddO1mPmfbAL z2i-=oe1xziHz?y6E4u&ko;R(VPNAE7qAh2VmWg%1u2$kM09c1a`7xIXGW^uB0UFe= z{M!8?;ozrlkOcWduayx_n-RPqvHR%@o$X4Z8w+Y^P%+Mc4?;f+hwL|{>p$CZ<*6xM z+qKLx@kJSEe96SjEE!&U+iIr);K)5MlkMj5f94^%QFjQPXM;JxC2Tnb=b6zBMuDX6 zO!cZGV%x_SWJu$LN=97>5V1K0!f2PSIP?cpm4oZ z7VU_D|9o|3?4G?Ldl#tf*45i81xEwn_JgT_w%vrl^WxBmp7@3jbho;L8|| z5;HJibQtZELg+wb7m;;fq`y-Ca5E~_>GKR+Ua*v_3;x#d76MB(F&j4-4zVTEE9X2Q z2Tj$6b$5|yos)cLzm6%ZUgRCnA9M{1+CS_3kfG_f?Ft^?418T%xC$w}8|AA1mN7&se?0I&GnofI&MG|@J78to@ShZI1D?(*>%il@ho`#~ zpMg>wAe(*+nlf@{>TCPX>SY&OG3F^NJ-|~_TZ&ik_ogsfY*9xgFw@G<)WD}z!^n!e zM^mBVN2hqPM&-TAgXHkyOp*L_CSy3&vKz<~BdHb6EcK>`1%V%|r!_alk1%*NHLpcfAyDG*Zd(KLWi;FE$@RK zqk#ciZ3{FeaU>&y^K9UvRo9;c*mTm~Cg~zf7*uv&{veM~K$!|M11&GutR!@VE#rvs zH5p)X9!yKS$eZtEqYlm?;X4GZ)w6HHeG(x4(L)3* zgHna^aJbn%PWe-oG*Qx#4)|joI3y+$A$ysf);o8)l#6jMtrUsT6t(HQeX(?%7G0HD zV(d9vvbAh)g_s{<-b7%7AKN~~n7CQC9=m{+@_r^`e1@^9W3?hUR^3ltAlzDr;m4f; z3X-Ek=UTVwht%KnN_An+60twTR;Y*q`Hk;wqy;l_3dg9TQ!gUEbOMx+-!l8!X`E+n zzr{MhQHwGSJr$HWaV`5OYgZzbOY{}KXP(_IJe*I~B&4x>%byQD8$M)CWK91{eIQz5 zw2Mh1&RHvIet2FH>-2@)I^Z?B8Xfo*#tXi*h z(++t%JkJ}iPQU#|?>W)foe+J@jj&(ibxZ60Ke`j!Ne~hE=cI?>-P(W#KNNY@vG#6C z^es1o<>~Xw=4kA#M%5Tn)oW@Z!BHUh2p2kr=@VhOzDPr17cZB@@MewUYAIen@a`{$ z9yjPlZZ_j6D>=o2tZu>%H#on;!R@n(Sxdt~iVCj4z7>z#BZB)}HH6)YrNk3%X4u1q zplqY#C}e%Qy?IpG#O&!<8eAuFVn0r4-09*U z>^SiZ^7FtL@qz1BICV8unx`0YkaH$Rh~hX`zU0|2H*x+!Si4%vVN#wTqH0j1w$^Xd z+B~I=N7AY<=8x8Y)hpW`!L&3NonL@NxT_c`pBYUF(Czig$Dlj9B0|KzD$wf@1yIqu zpv1SFj25N;-?$vriYteKss{7r*A;;wJfdHuUeS zx;#Od682whA~Gbm##Wm}dqU>L?{SHVE?U&qiE5E*uxjI1=lpK{bPQ^{F*gv4s{P^P zhc$vh9@PHw;zKVJ1H(NHGtzJGbaK*ExIW-WrCHA2T?`1vyZ(s9wk^XPk}wZs*#FF& z^$%$1Bb%Z2>=DQXSD58Vpd0irvm-trH>WS(k~7wksj1Y)5j#X!nrj=^7 z>tkIfZZLk0Ch~bSbU2bfxeB_icpKR$LJlGnrd-ZD#%&MY!Ez*$#WU5W{lZty9?rQc zT7W%6U=q{@lG?m4-66+&DyoaO zJ}X!Q;HhFJwx_Ywe_8ag$K6!>2W#6j^%A~E9wlKfLiq&ud%JpD)Q z4*Ph20%UV!X`#0;g||*>p92Hp)%6Mcn*Q-X~f}|~^hZz4V zC|Y2OG97h?pN#CCs;yzYn{CdHA-^lJHjm2qE6;(sc+@z7Jx}1g+mb~rB{3IA$MnsxC09#)O6jrT zyI#~}xZ%0cbq?YJSshF-U4Rv{&M!x#xo#>9N$?r#e+g9d?lT0jSCX`#{*NRMuOvyl z^b$s7?igfDFD;r_Vj8&Si+?%Ip3fdyY^bRVaB99 zP5TDKRR#nc8f9ry&D80kPO}Uuzl;%XPF&esvlTlS($&-?Nr|5=X0*r8iu;S}h+34b;~S(k|8Lkgf%64e*$NCCYO-U4hm(+H&tgp!A; z)b6A|-~`y7h$x%?-43%_q#IOD6Ko50-WKfS-z|2P(2ky>tDLccmBAO$0+!3H<5?;) zsJ~*IWwD8KvW?4ZMxN8@4_iKPfBtaTK|A@yqMX5f2vw?VImfviOMR#WV{~pWJ?2q~{^ym1EZUj4jEy%$>pF zA$m?)OK#MITmIqRfJc>``kTHWYI#Z_cmV0&zusHABkvmAp6^@Mn#f^?BO}EPY}x!; z-RQoS5Bg4^8>*!Eove8JGL}eozVRZ>rLAdMq5viPX2T>>H;xp>9$QzxZhxo4^!OX$hKj&18ZXni%efIT~AqJ<+t-*PW$->iKOQPC&l; zP2)^?$NHMQ6iduPzJLKNkDt2{E2W$2!sY}d@Jr1hF@1aJDOP8qu3iC{6-ZIQ?0uZAsFbo>A4A5~`iE`V6>dAQA5 zXaKus{O*aEY-O#+>NLi4@1MO2_&PVId*xetY*h%%@OR%F!MErR(BxN1p zOha9B;|F{@gFltcN8@3J{Ie+<;QlU)~_T+j$UHHQQ_QZq0unNZl$ zkKZZuX`y_Z+zq+2qLTq}{8a3OL}XsYa+a-^e#ucIfF(B;7rh((Hk)p~kAfN998S>Y zYRkub#|#wHzYAkFRuvlxeRzh^v+k0a2)R{lVuUQXkY%pOd~!D-mEJOGj_5XOKUob& z@hV?^Mf|eiK<71k;MAQV&Y*NeXILp|qfx5|TO$>)cAr$)cYQ0zMyv3dC9tybW{}ok z3EaUrZnTC^@YIwt63`9{f|Qn&bLj{)!~J;#U#*f$(e?am&k2G z{F<5lv-#*0@7pvTKtpGmzX)%1OSAuay^Q%ziVDNJ9U8>+Rqgisw}`MWe>Uf$*8(;s z&xJ6<^nW2)W7;k)z$>Lqm}G{aUKiUkATey1{HP6D2y%wX68c|fr^>*KVo>+i9|c8V zulZR-!;pgl0>c6V;a&m%3WQf6z5?kL$geQIC^sB#xyIBzw$mt#>lydY1c0 zWp5=Ow{&E0DYQPT4rDPx-!F$275Kb`m}TP%<2ZYISNe@uY~LwERd+I-49Pq+tP7G> zy4G12=pYO~mkp|lc~1v#fe7H58-M~^Fwo2Icf&C-P<9jygggra?GqeuVEYg}r?v^@ zY8}4dPj{O5;sCum9j(rrG5iYP^P)$3>lTKRZ95ck$V7nW78#2O|LvF1fk%Vswhs`) zZH*Y$oNn@#*z8;%dB{XIZ@C8LhsEtVF&FdGtY&ye8WqMyLxz{~vCJwd$#>f{;Z4drFozU{3~0vS z=8qRcg?n+jR(|vRmQ93x$2hr@oeFDHIEvZh6PZsCrxrvR_o_j4UWRT`47<$TeI;b+Sw)g(s{nXZkcJ#cH{{vUReI z0|vLue~l8P52AgFn{&b@3n9I(t{(etLa5hKA=lGDQEn&O|F!o>T2ZaIwDEo0PoGuD zFFnChr-D)Wn6zZx3*@!Mwp)XJz5?s1VB3?F$Tp?>d&*h+e^7G7&ENELxE>Tgx-v=_KFI$)ZeI@qHmPW~)GdD0I28 z_nB&o`0VXAnYX^UOAVO+3`wNUEzJ&$fcNhZAiZ88aVcvM)o(L=Q)l-s-rlteIpQwC zAIChoLKBM%5W78swm-7jh=J9Vmlpd~j;a10oMq=iH?}Q%#GnkX-+g3$(gr*9c2Yi> zLV0(_6$VAwAw?m6LQfp?!YW%AM8_(J>u7Tw9LIhb&{pIqYs<|}sJx5S3%*Czi7GTp ztnLN$9GrGwh*>#DuAi=Sw&wfrJgMKI@m?=(I~IujE?1YGcvHV<(CU!m^ifO1~vAS!hSV`_I2!9fLba|OGB=^HQ+-P|1czINL1_t`B zPkWmR_6sQnx~Z-eWp3SPki)=?eH-;KP|xJjCuRRmf^%WdE$@pMKYVD}ju^qJhh*P! z=b7ftP@O(UHx;MC<#habawV$-RPaQHzyN1@qR9k2CZr3vI@mOv`lLTf`OSCMJDdeP^1(peq8+cYPCD5~6${i24qmJz-p)T~ zi;_YosL)Q93$zzf$`R#N zm|N_T*sbY@1DVK$d3N97GnRJKmB}$gmGvW{wi{T$k=^YtS+bZXfQEER&R&|I08yu0Y-1 zNi?}!Wndvl-ujS|!Dg#Fpwa&l5eShH8ubyJeEN`^2rhTZq5=9gZ7j6sxU{yK@+yQI zd3z58FzIbKkivTgKlB=Zm@6B_5I#Ul-3B+=_7Vs6niOwoV8;?%mv(*;<3_MYZqZ{s*idv!+ul{%Ac-ml;?vgp_4jOH*At} zbKjQF37m`8SApfB0p>p%&gT{aM2run{>0GGfPeB|ZU6AR?KZ2sfj25P;X z{M8FSOGRt;R#>=?y0_cnHxHTK+iHJ114AEiXxI)AuHzoc=7(?nz}UOyAPN4Z`N9Lv zs^76QHE5k{!`U*V(mTc-F7in1Sx6K8jVQ`fJIlr@F<;o!RNX%Jr!`P$$d2aGi*vZf zr-;4}FtzzX;iKdp+;5?jq@J?n)W%Nqn|zUTgn1^RB5-i7Uo)eor77}v=h7CsYl%vX z;7G=?!p`yBc&*+fe{N9G&*|Phl5#%7Gj3sI#t(hl@8?Gy1qibl%QR zbi=vOBk)cfI$ErIxExGGbu_De%%!U*VgU-f|cGLcCqb}8edFvGCK3z(uC9Us z2<5i|U=5v7rIK;sh3R*flplni zHHUUjuSmG6YOl4-Ms4hE-R^P>y>boCdR?%yni0}IvZxcKUSLiojA|D z5PCE`4jN)VF>bf3D%Z^1i0CS4%$@Qq?KjWb)dOU5`+qz2KYFLIRhrE%pD$Rmc1$C> zV|<6Oj_#KwZvDX4S74r~CGQCN-V{mVBdNxs8EkA&fc`h;JA0ckbGL@tv9$VbM$R#_ zK!--0CGFt+I9D|G$M*LwESmXVbHx*?`RH?d@72|rl{nM?9H4#lfqeUFt)RAa$h$-N z_bVJAjs$uG`U>}Y^J({nNcfR?+-)mGF^2Ktyn%sipiy#=k82(l=Of@Fzz%E~%Hwcq z=QN`k>dUVc_kss0{e!jN|0DnSL2=jVJWI>?skD)h{p~h3W7jBK-+rgw*x%IBir)*l z+FO3r340#=frHZ-xvCh^akpOm12|c{oPgFCz174GJ*s2N`ap%%rF;?3yh3-;sF!0L zYhvxUJHfUcx^o}HN8=|hFyq*I_K9r^2j5x|!f-w?+voQ#ZlD@EsxfzM_$r{J3^AovEf(=xV%SUnvLOVoayiYL z1p{r8053XJdcf6o)z*vUAuxD)w!;8L1DkfeO7nRg3>4LgPY*WFDzfTehuDq-6$ahK zC(D)fHbrFP4u%k%aC?_*tmWgcP|Mx~Xnx;1E5y{pU1p|p>pjFhN?7{2Xq)ZJD??J1 zUn0uyV6UNe80d29|CyoSMg)L=jb*c)-0cuf`n;dq82$w-1QpnE5A6KT0cC!Pqf2wG z2(f3vx%akPG+lWoFcFBJ*kbV>QOXTYj6;SVsIt(;!k*)PQtjPy4 zC?D0+&x=SZpsw+WP*)u?ezVvcHMs;xfja}96d~MSrg{DedxBAW7FeMGtl`y6uh$DLPW+h1M zuyQNzNo>#XX@Tnh({=pZtC>Xw*u?=@Pqt9nR|9)FY`Yd3))lRqex#+UP9IwrcQQ*q9&ff-r)q zBT%{e76y`b$NbMNdH?E){CyapdI6?it(Lzq&~Ds<9nH?`KPM1<2OVhyG_azG0O7eH zi&41OD9>xO2k*7!z|ql#!_C^<{i~CMqw{|ULVl@sEA4RNw$q-8X>~}>{0g<7ds9*O z8hW9@yS2CF=XJ6j-T9SNwNXuwBmVPWx~1^NxHtFhvD1ciO(NwlUf#^iydNfLiE}8A`vrWA zT3u6(so+zz2%$_uPXh}I)zDZE7)9Hf?VIFIwhJNa9`Wr03%LGX4M7iM=M7_L1Na%b zYj3$U324;$cXS<|l@4#tFGnf4Hazxu1#i|LNNAP+8bp#cqakMfh>qgo0GNz@|91A7 z;^YjsHoqXGyysFXqqGplZ>aX$XGnS;PUr|3U9_JYE?J^7AWAs*bn9MLjPRgIu z&UXRaY>O-!-C3IxQBN88QWKYd@Mgmzc+g3IRMr^OkNosiR6fAC_7ry%M)!m*gVhwHN}NT+l!t{ z9G~xDq4#mok^3H;?%{_EZ#>Sc$b!u3Y=4)>TgSvwWu0TZy`Yd+ov}V&%1~f$_^bL2 z->%3dEL^DJcr4IWq)>yk)3>VZZRE98*SF62Sc4bJlkY>NK&jrVTF>jUnrjR{=B z&mu(^&wg4dAbQ1y$1ukrC@TJQeaEuY#dh4o2-~4rMCiXYKto~}W;p3CGXPlS(&t>T>RgNrN=(A;^lJswuy5x(CD(834ayW4CR{g#0#hbTEvOHwuxf%AYbI+W6V z`6H@yx8QvFyxm$G`ll}}=FP6Zu|rkihqB?*tmUe$T~deQr#OgioZBtbYuwwp^*pPG z>NVB6Nk6*eF7K1e&GziQ@^shT$|A)hfs-zqP5c-2Mg7Q6bjn8f!LEqSu%=axhl@(a!H)2l1GpqjVny6+^{qi6Y2j_>3#+5UG%oVCr*hrAs1 zZ+Xl2Ju}%|-uoB450f?d&Tf_b?HfyYRT%4gCY|*=Y#`9`-eZsRlj>K0Rw@8@_N}N| z6r|eZa7U;5c9cq4ioWmfZx)%-K@SdGoO$#7=EqMyJIAtCZ}5|}RsN`=#F&w|F%bSqO$&YTdGnJK>Ad}qcC1F>t@OcDYcxoteH)n-ZFR5a@eQZEZAHvh=? z_Gs_MdQ@1D(laH=!EFAvSByacD;BWz_8pr% zqpn&o=j}h47gHy04y(xdH&8jW2gu?v!H*xd(q9k*U3DDJwHKJmNt{PGC{?osVnvl_LtSSL=CQuQ)2Z z?$6^KkrVBY{XgeBnE)bpI^QYhwD zDB*uvCH1AO_u}t*cdpiK|NPRqw(kdj(xj<@hb~QVz1dO0C{p|A_5U?Lz+t~VxSY3@ zofCeD$n=MGydjJ)rkmCCIx&h*uLM#|pN%IAnut$tt>YD#*2K#*{X;!3AJY-5>4ELM z!jm^d3r<&Q;5A`7V~r4~0|{KQK?vLc3EZ)T3rv=?M$g9Ql#11Yvy{D1agz?Sv zgN?kAOkW%j;tox`rc8ew;R2KGow%k))$$5V-`d10#mGASLK9F&o(n?31{dz>QO!W( z%U$7u(^oa~O39$?^aWN22vF|^QYtt7Uo)?m3}WXX1A`($4UpC81ySbL!Ye8R-;)Ha zSx}YL_(7D-1u8@A2$;URg*SnzDH`mk=?1O5+A{Dpz8FSy#XyXh+6uG@u|f{rrY(sO zWluoL5R1almF1>Elo_`Hl_8cvp(|_4fhcPNDMKtRKv#C75TfiNP? zUK44=G$Oi!9n%AV3S>~GV^D(ItPWz)zII*%#@^}g+j*TCzfL#q;B{n7m|hB^mIB*b zK3R|PoJIFgMA%#q@i+@0v{S_-5$mttewDUju09& zza50bNd<=+m}7@w?O_B62;)(1k%1d_5&(#iL6JKO`9cPq>Y=hA40m-{S0~oG#k2ap zA3hTy>PIna2FwNfPC+Yv<8+4Aht}#rc1A=3N(m!;@0DYQ z*a)`y^z8~2vrwcW0jv0PhUfw+zV0UCWrV!VwXr-3|TIbnSfG^-x@2O=&}a|8(v_N zj)V0ZcQ;%oX4_QLWWiLHc|P4NT~i+WfYTC{E%@nMiRpa(4{fR{Ay0VDFIlk+1DeM1 z#(H(2dZ;CnGd=>8p&r~F0_VUM9DP?BDZ;Wd|X#nBNqRH`A*BQ~P=oFt^2 zLQRRGBugM6x~#|}+bZlM>?2m9id^P6r(KcWmZPiGDIOpjz$BH-2D%09j#MXCCfcQ* zg8V3yQWim8AS1>+DkG90+4mo>Es!d`!o6z2MRl45lG{yv0%0-|nT4sQ5vG}@k)|O# z9D^zp9ZA7SztEJ*g^%lIC`w51L}yEUW?GsQns`?%R+tH;xQbt9laB}Qh>kk%;4LUD z^!BOu3HI^!8~>=n_AwhjvZ>;h#3aYG$Hd>VAGA*HPOfs++8BOx(4;=mI@0>5lBQaw zLZrr5$ZI&wW6bANI8-RjG22qwn%FMezRWXL#!i+_dgV^$7_oosy@sFEwMF{2Jlp>5 zKM^=d#>>R(!o$VWzyq=W=D^K#%pA$w&pc!wYO>Zv{0XhQuY35@kU2HuVocqVqA{AM zpt|O~lwn%1c)e1f+N`Wm+p0*haRS{SiBXSBb&A^ zscyZXKRr{#E5^|xS-oWiQ3c&5xh6&1W@HtKmD99Ex(+F1sw5ZkdHVT>{fm6Ep6Cz-qP3%Qu-37R z=+EiZ7*6SlbZB&7v>lpG>Yn>o1~Tmo%mRkK+wPh*PW)Iu_*q@fu<6--9>AArP`8fN zuGl_#k8)4?Lixf1ClKx(-ih+C}1MSlB!!yHD(Xm)^SYDDNQiBrnQVJ4&5+DgV zkT$(e1Gye2|}LK{M-#X$oGk!CVA zGMF;EkV+yyw>%n*Zi4s9QxZ~&XxPf9%g1d(n;x5HMJY_8>gWvE1gJWh`q_(^#oGSZ z%wK4r7Ya`&%2T`PShlCfz03cVjv>#Y2+=`ov~Ae1lCsGAqcZm8;BGH-&vBG+tpw!=ba_@B`;oEo}awd){on6wKGh0440>BwhXHG$8HL5ezj|zb!nRDxfp-8i8qgL zVl8Xvl&!UunlC*mJ_*K)&phZn@%we2vof^0*uCixGz$?6H3%p86@#E7X!t?du3E%l z&}419s(rWesDO%Ns<*uR<|*AM-GYU+g~;Y&P8&aYzs+Od zY!2ZBA%jrXhyPv7eMV;EA1k@Z#0kvoES@~Cz#GxuVb{@-*x|fd?t*PidowvTbu}BK z!Jt~h>sHtD)y|h!4N#?34UYj|Ku$pC`S-Wjg|F?oX+~#(Hpf3N9XgMl8{g7)tK+(0 z0uG-IUYIuS+XD9EK63D(qG6ve7`yyF4A%sX12*2K;mSW73XS=4zgfSTo$VGKyf;6jPFLmydcAa7nT`xx*-f8j2+3}gb@4ne+&dhaAMMS0*Xql<>~dI&J^}BKp67 z2tZ~w4%pqCw^7%2*H)0{HFI)cG%{w6?b z?e6Z(%f#g6<;Cd5#^~f~#l*tH!^6bP%EZdb0H$DY^Ko=H@n&#zBm1vO{?|O>7H(#) zHqP!gPL9O?<~1>O@^BX*CH?oH|NZx0>vXrV{69}}bo+0%zzj0|`-O>xk(ud#&kY{R z|F4%<%*nyo)xymUJij0Z|9>d|Z+-vgIsZkkZ0+Rk1eSuUjhT$2yM-%wu)E2>)d{lx z_vrt3ivQfR7%Iq+**{SQNvlb2SLT@)muNgQ%%pL7~5s7M+@?REa(QOHFXp6S6$ zd}*kah@i)`V6sz5J9}~cI23rBT;zXf-Fyqo%l+zRSbOPpk;8Y|>SiVAb}*LxAufwX z?M(bXT0V-4aWJnsNIA79_^3Q|zigFPEsgvB{^Dd(2amz6tVZDd_II(w<8;AwgaR~- zKv$)#w*1{65)KlM9|an?%d)%a&f|6<-+Iw@Y7R2+OiW|Y`_tmGd!X~Tw)FrH^kC?B zo~nxrP%s;cOI1@6&{#0^bKp75cH8^487L?J(f?UU{;qaBm&pH0q?&ePh$twVM#+I@ zwS#uZ@3NC>z0D&znz2p#WU=HABK$GudY5|=~#pldZtrx6oj3%XN zgQf4>>wT@-DJAOjxE+qzLmP>(J=pPxG9mb4kIV1vteO~Nv?a_-5CJ)FgK{*Z=Duj) zB>|R!m-}yAD{-cdE74-%{DnCk2(6|`!Iu<`ujkoCY?J(KwyVt!#UT!N;GM(F-bj@C zihJJu@eUJSiwvQmMWbq?Rjm38k1kHOo7BLyJZ&>sNf4KzX-YIvc6hDPN2S>|Of}v| zC5|ZYk)-Xa53$H7=xrXeTp}7PQC2oN3RFR~`M^3#kDeHWhS?XL`Hr#?^zkN5MUbc{ zm=s0u3JIsW4J=NQl`htFHSPT>1RX2$!)k zCoD`}dMqSwy2k;Tfnl_!QNZ5_2wI|YoXxOVixBI(qv>gy(rmPHq`7f_@R53_oxV+N zm)(MNbhlxC-ic;H_@v_(vgUJ`Gi{mWaDedmbeJav_to3D39KBv#w<*-<9s^;q(1qb z%A6W!3@y}c*}GlHJ-tbCj5;^Ql?cz%dQP2~If2i|(_Whahc|G})nFB!1}m{{3IAXd zUC&3v7cx=<->5AwPT!|}P>}?-LDSZ~#+X+E!=(MY+7Edbv?{vVjs{s&aykE^E3v=t zg+COs+MJZQu*8Ml?$lE2Jd?>Se%H=yR_qh4we^BUa`M|xzG@J>QA7+~k2^KDR>DM_ z_S(g(_2~W2xlEmRNvBsDK@_?^`>E?=Z_TTYw;?U3V0~|~sEy^?;5N;_vNCt-da`U< z_gvM8Hv3xfu?tv=!DY+F?g^RD|Ld$zWxeg%TA_VRE83XrTJ4a z8O8Is1mrpMJ;!Tfnc))PsQc3v6B&=4n)kG-k#%{G&WkeNa4%Vx|MjYWx~BaC&!J(2 zGEPYPN5dB&N+9Kz5Es?(%kUCe;E`y4So#Oidn#18v z_^N}xaL;kvI}t}))%bnv_qCpo* zewbmE>yj_xg(2av(iF+LWE>Kq=)5x!sR7vyUhx-SD5Q6HbtG8#C?$cvDfr5-ir*Cd zOLIJ<0Ai9zN0Qk3&LYKsQl~9I23X}gPQ?iI_rtAi{R7A-NE_|=7cFH^l3#J8$)8f6!8uo>nl=F$5YMnwNhEyYI~W*yh?2inK;Q5}JOm?RvHa zG4%jOs)O&jnGoJkhMA1`;0PS+2eBM|kd86=VfUtTW3^YZ*=>^AOV*V+*XNAmD_fg< zR<4)o%_{l*z($aeNgIpf&yZT*eq~4r=33jxKfOEf8`q)BI%a4;Ye;mQo(Q!NHcu!) zH`N)8nDCv~(E7O=Q9>eqWfb^i@%uA;=;kz8Y=N8Q_mQz!3xK6B>I{AgX+|uIn8Zl> z1gk>f>-p-h_)HEvO$k;}LK=JkX{2jv)|IqOnx128nL)d!qvbddGJ!3Jgje0NGGZs7 zus?3HB2oPHQ0T*@L*f~_JD~=^zP(CYS%r4YLp!nAL)M11M@Y4PaY^nl=Mzbh=8Q%L zi$KnSiiSNy7C>e>0Lbx&mX>Uge6ogH_6Akkcj)bS9?uMRx(Sn<$COaakOurwc(T?5 zntG1`4$#ri*If?Neq-s`vn)C$37a*$<{Jm~K){I)Sol=?JzUHKf8`m~*!93|L3T|A zdK{|X9Tmm5^%4ROlwhT)wAvwGp)PzD1)Wq*Ho%4vJvx{ik$9L3+F0tu#27EKAh^Z8 zWB7q^&+bq<^$5zpoZMI0fATsK9W(!&44JboS)C@W#hdF0>3P7<$Sjs)Ep+%{a_~R9 zk*C>Gll9LlMCsDv)#@%9)07ZjMWU6AI61C*(|Ie+3q0v##t}Ob$R|EJ?PRu-pfv`3 zcOIM4L1}O$h8InOVga&jv7ll!%I}bVeB{aS`gcEa0Ae79EdzkiJ>kgE*H>$Yv|Nqm z!cZYhgnX}}58o(Qg~^DA$fW{TkopAkd|;WjAfOn~hyb_`15%};zuNe6;`lCF`q!Z# z%h*Q6V0Gy~11VgdScrAsv`>U>!x`a>`{1;Z@E(x~xTlgvjv+Y#W_!UYNFzX{tWqXN}9q)=)U^V#?ns9p9s95qRMEZTsvUA4Ds&dWAkuc zhkpRa3b86hvB|P`;w9hk;snGgaITnQh&+H~16frgS@id5pFr|gawF;G$29-Kyh4G; z0F=qb>lCY}6XA4Wk31-dt+`$=`NP;7GsZ$Qi-q{Yyf2NSLBFM~2Db(pFiEX>a(f7L>vQT~M(BPKLOjYfwe3qi+mu_(wj?$3L=oOXQm;8~OKUtAA~r>|h`;L(~EZ()bcll|AorVv_#+ zW~;;wVZ7`=>V+e6AH@am{Q11p@C=2f$bUUBw)GWGEe)1&(mEMdb!UGV6ZF`@*5ktx z>K@u~f4W$!%U;X~yK`nz<2h*l?XnWbv^?<@cVJiCHx`e>^_+MDvG3po1;OSJnB`5l zyMJ}L&C)YS$@k9IF8OW93(Y7^oK+$#o+4Ot`9J)v?{8<9JGWOMw(t7S;h)W?=am zg@HzKEgr=_Uf@2$Pph^2VZ(2|$EF8T0GIbbKc#XF$rAACQ#n=$$Eb=ROHs_}_d28j zV{>0N0ID}k4~uZb1kxxnwe1UO5RjJmk7bQC%BNlQHo@X}Mrc)BvXjXdGh*RC!GTFi zzt&u%qe1Mu_x&ykZS+$4d{87B`ESfuORTD@%S-bEH-yGhAcQ`s4I$YsdJR`B1Yu}w z!vU)+;6+OwUBr7q5agx22%jN}Fi-}CA?}iNkRl7jmt9e9w{6{@`2rEShZVp}>;viZMUE`e1><^1VRz7kC zUM^T7a-{bp-2C|cJo}W|HECgPr*PalCHh`s6%%Y^;54~aarg$J3jip2MnIA+^Go`3 z06sbGH7?@}F@6<*79eWm8Wsn!^_}0pZV(N#-8ubD6zp7wB$pL1g?^1WK1ThOaGMB# z50yAE71z;u?;|Q#?640NtA%z(CyKGj11kKH*DF@MZc}6Ii4LZjrKx|lfofveXdJ&t z_6{|KOX4d|7K9*wf(2Z`#=Rac*Fc1JOtySu-!6ypYZ zD5Ko|PD6xV`lif19B9JcpaDeE{TGy*pKfV9K8t~b=!iwR(%2a>QbeCRM0PFwP< z)`xVw$=HPK5_}#iBf>v^)5bdgz-w~DfZ=kTk)eu&j2LrkSGI{~iz=5RDuhZYBbP+s zJt9`wBIgy3KJkq};VguzaIWe`am1Q$MSvEkpe=;TWp`ZDR(~e;{T^{dtWKd%!qw(C znvpZ5Xygub_54BIeIe>%p$adOYF<_}VJ(r0DS@dV!KYybU2oF=y@rthH?Mk)9GXbu zY5H2d1YXpzB7XO=Ochh*7_^Je*~v^$K`g0`tL-kyPm;5@We$-dEgljEA#388E4vR79qkJl-j_dD@OI^dH z(Jf8P(B&avNBP;2S?(em9cCxx#s#vduiGWEJ{2jk9hl5~X30x@w=y6~35cdL$SP`U zG!1P`!fXi&#Txo1d;vNMb-eAF;xpR6uKw!6WC&;CAQ=gWtj31{zOW?k&~8YQp`g?> z?nd%~(pA3mrde?+NNAe$yKG7o!0}ixxkO<$!j1$L)P+lzX9-jrwY7^!h*BFlJ;JNO zSA;Z);7~iet8_c>AJ0;Say|16i;*EmTMQM+sj1jcT_K_?Je#$$x7oM|E6s}5YcE>L z6ttbL_a#S0TQCmkm)V@Md`ZR5oGXa%NP-pwc#lq^Q z)D;+vycw=C<)9Eo(B4K)lr_S5 zydzz>^SfMqW070mq2cAj*2r`Zah(XgE0ERK-vC19oF5|QKI&UVqL}aD7e`hhq>I9R zo+|u`MER;-uBEQKUju=@j|vp|t588+FoOq2dwirpRr06No{M(bg*X{PdGK!e8^tq} zvxm+oGJhn*9Qx>aiWKNmBTRXEx2y<3i+KWFN?GzrK!of$f%LouBG(coo)HCSp&YTJ zu|^~a&_FBF_DV-9fzbmS;z4qbVEp?p_eq_L8+Zpzyag5BrqZY!-4i^+t^9~l(R3$Pr8@?(zCv3ev`rs1S!r4!C?YbG z>!Z=-li^g-gX}TWD7%I>(@|wK%x8eIr*|*U^(#SlOZfayIALq<-Ag?YjTFVo1|b0T zP73nPkGrSAkZisla_t0{{UbLjJQEfL8s)VT9Q#akpLl$z>7?+hAj~zgR#Z8Cpp)v4 zPZC6&GCX=r&qhmpo<$evrt0v^W7*~fLxlN?F2sOr-oLzvwMeIbsbNldD8i%=NDlE) zj@VKX^^RNau6!UEWpXL(gPz`?b zDQUHtA3LA>Qrp;#dH#Q4h;3v4ocWtoWRI6cH{f|kc3yuXgfF1*1+!7$#b~mp2l$Wf zDGluHjQy%*#NhuQ&?PZGT*X(8EC%(f&;J9a{6vr*|BqS!y9QAeC@{|UzZ$~7(;aqA z6N>OZsQJ2xc%a=o<4SAcC%ssGiv z;IXl_OXGtEr#O`(neZwVCM9$%W91p<`dkuYt1# zZ0Uv<^qd&L5MUW*H8mF?yKo5;5)OPIpi$Iwq1q`mb$gF`=XbYFrei#zo|Ul-wwxC*7{|j#ZuxCsXA8@h@G}Tw@%0b9oB)+VCb=9_ zXA`^q(v%F<&hZz@tj%m5Hv*u6$Gir-CPnt`@Qu~6g^#`hnt4)5%c=J?^jvASFa zv&lF5pL42?sOH3bDjB~CI|HEG^tD7^cI`3ERsW~2gWH}}k#qs`OP)o@?0%JUT#mS1U`%bIP)lGJb zikgN#7CcXT$rk;{Tq$Pp0?l*9l>=tXU0^6+Kiw>@d?#;V931$Q|HG6z-k(ozw!?}3 zfPr8#e1|5Lg+1J(%l1pbUIQ?Cum=wQu%Nib`zF?+=xV6(?dwDq*bj!_F1uc8s@gBw zYN{tPIYxgTnx1?~D~sCJ{PbU1t89Y8d?aGze;o!zZeukR^HNb+^AG&qq!{1w27?&9 z<&gP@A)f`yKJ}PoK%x zZ0E`nub}elW0rn78R2r;r|WwzTi^mdbzk{?yggXVDsW+*$mHyXLhpg`zqD1SZh?2} zu5PE9;z%~6n1N`EiiK)DY>m>o=n87ju$~fET@$^-|cQ~EJY>nQhahx;Rz-#^ITrt}< z08Z>Xl1mai0CNBko6<{_3sG0+A5Tt(4FR1X7>mQwH@^1|gXh@}MJK3hJt~ZLl^ExL zIjO8Qzdu{fc(KQ({+SEbmiezxykEVyf^FaTy^#pGl|5G-xNMdiO71S!`I6@#;0TDN zcYW{AXrwEXdXV{p;iT)eFESSgRJY@W-qUSav>U>#sRmU6+=H(4p-hob5x!3e;0V&l z1er&9tLJqL0*Ksb3ETT}*)23*fkQ03S%NlT9wO2e@N$o$o_`s_P7STtd3U+OgMKRaNTCz-%~N<^_C#5|-#mRvx&hE*Fc z5O?HuPEE<exP9koWRIFcaeDqI!1V^(uYx0nDSutfd(Kf6;T`2l;0ah z8TC|oQ)5DB3eUAoxDXo20?&l6YvIGSaafvZ#;0ADCSj@R>^juyoe$6^cG8I+uI*mO zHfsVvSDsE$ex7-wWb&6sC+KRbEPl?)*^)-KVeW-*AAA?dP1W`)PV> z`A$B}QzYx)pd|KoN6+{k3AOchf-i@CJJ;_b@n7}>nQLvUXZmIooI_o62}k^FZ3yP3(s=u(m@e?G%n>5Zzukz-*Ym(35NMrDbW{QFOF0Ub{ie7#H}J9)QU3u zJL~%398%5KxrZsS&mK^%gC%kFpX=$v-@nY@y6c1SUEb0~H!vWK;357CW~)c_%P6^w z56Mr_3aO9ZaYWwRrz6_AU2=kRL)PG>y9FQYXB>_#ad_+gs4_54u#PKjAd) z0f_vH^!#i=C#LH2bO8}{qzAF!!$j>c%XPb+a1S;DM;mN%;y|s9-Ns45)5n4K2=uM! zOC|-@L6;k*%qG=fNI2RF2#M_&&x;2z?^^sqtjfx0Hl>*5)@(CB&!mNxj;b0bqW&Sh z(tOv_R)L9uYl%_gWN*}L`;D%!UKcMFbpA`=Oo^JE7j4FH6E@+as+<-a7O3W1``kii%yzQb9aK!rMp|`Y zAL+Sd5-tt*M;=fA2tb5PB)B%HMT}Dp-7`p`kcZ(dNvjhX$A2>{4X65{l4`d@82GrI zNV_k2^|oSHTYZy_pNgVI2l^Irlpux@iGg9>7X(gnW!Mqkk--or%GQ!@6re192?Q%& z>;(Uh1;}^W@fifUE6jGuUYrdfcL}PHb9_0l-m;05jDF=38!sJea zM}6h6TcB5kIg+NZO8i!B4Nf%1TbIf}Z5aRBx%ofD8+}#vpUC?;~0HL z&@!pzHJxxphf#X^F~c2sG4K&RzA5oNe~8(7inuiW57N@6fcTRvNfrw_*Z94>0Lh95 z#iv~2Sdm328)!K(X|p!Z^OG(RL-SIf#i$#hvjjwfo}kD|!`uBTpM++y7QI2Ql%A&Q z)^76+8ZOxL^-e&$uCD6@SnGg==UGn=-%5eXG`Ptc0_jue8lV}esmJ=n+%~ffpZ`{Ht3^L z^z9a-(RK6zB*=xicSP^1Sd+U0Nr*nA9>GBeN#WnPseBr^oAkWH?_%3>!LG$jwcan= znIv!A$L3^jF`-E4??g7YijN|^@EGJWWKwbN&qDZ49rZnx8k#zTGC!^HmshApwQS^^ z%f!eGL@nUs0@ulTe`o1CT67ia^L)4ZkVTowz%Ah+KI^!hs3H5I7J*JgLTcLFIde?v zdoUD-xQ3~z-Rsfh?)&;GLYJ%AlGn|{?6}5Fbr7_ls>HwKjvzwt8N4A-$Il& z_CheuHQ=RRudC{Fg@=yt-;L#3tErgRGFpFPDXZ6a>4m}mgwQ##|FXj;8vordjb}l} zim;RpLJ229N0yYSPZ)@PC!m~f0kqPQEBVTUnYH}xApQh_e{>6>OVXt`6AMBVhSlC) zwZi>Qs6U5pq5lk>s``nSjNb|}gmF|;&tdS9AJ4h>7Q7k(g4hpEs{E&MaHI|DD%rr=Xdqh;Xlyrprp0@SlK~?{+!g z1E?WJqn<7exXEb^eG?U7Ui)O+)j!`8=M2n$JChF*e>(+{(ye6ENzba;P zv#0`7&RsBrZd7d~ij(4{d#m+a%hB?IS)$Kh<_$*loEl6=^#A&jre&(PazBF&W@DPL zi0RLLAtB~b4yLY0(_%EnU*_u8xb(5!oft=w2=?bL9N30vl-SpbyK-r*eb~=qU7N-G zBlcO((-VUQwQBGh}+~5K%fsKG#m>~`t@d!-XO3{5FFWKPSxXhzwHR)(vYqYfh$F23}d!%%SS`Y_Ca>; zdU==9u^sWfIrkD8o{`fK@~*P~F^KkfU7BYxErB($R1qn__;R1+)D3cB>Y$yB&7*ek z+B9rlwY-2YOskt;yf~!#HHT~5%=>))w8_}J-NQ|;)Z(Aq;yEsmr0LemIbydq{@d>M z*VP)8L6%Wc*7)Q@ypfwxj8Su)%)2JsN5)fz-Q*cKhKoxk+Q=O&SzWVwS!O}w|7CaQ z{^;Vg*h2s1xkjLIgwmow;Yy5&Do-#ml`G2*P^t~D2;g4k#(P!U2MX;f*axG2dxab z_OZZaX)QD6TaLAWJYM$c&+D)f6gX{ZL5AKHxkjoUk$Z<`NOA}b4Qp16q!13q@+?(tm?w(d|t_|-ZKE(1l)vO!V^A;QgKq-!6SRI914-x7N zBsZs7y^h#O*eLq-+cD8;fkCtdU}CgZA<;0P281}lyD`XrGu3HPf_WIFd`qseuv`6W zNm|pc#Z|{fdrzmaJ8AX8UpdXl(Pr6dlF(w=BCSCx@`B;lI^C*$tISU0Q*SaZC0rNF z((UeAnrj&vFy|(v(j4h%h!dR>8$ih+@-DxTckDJkNeg?T5fgpk|JPIHZYMNUFXLNncAD5tv ziIF(|d=1B{808gUC++XdgEJ90*71~x#b|L8#0hMLT&x%JT&<_yfm+V2Af=LSjQ!e^ z6ra#{-ubXGv1Lc1_jFsL6?sUYJUlY>>!ctmdrco<+7Lov4jYIC=ej(D1fasV`W962 z&parr0pM)l+L(TA{>`^<5dBIg!>bDL{MK@lepT zR?&J^nUtzOY6va~$yzHNHCGb4O{!|2z?(5`L@p6~X|AbiHT+5l-O>L1yWKn)Ije;q zV28-a{fp20_S|Ns@)Eh>MwV{)LFkn_Zz6lYnBVNT!Z4spP-~6UD#z2k#vU!Np4@Zt zO=)U9h zv$pKFC63?q_o?JPh1V(m+(n_i8ER3uV{zbOs<}q3cGn9G(~qVNRaT$V>R;JU5Gk5G zEI4$#f+l#7x6srjZUe%tPb!W8DoH-KFUdlH-uBC`QzTDaZ6sS)SOtPR=f0+uRatgH zTbIM1t2G9EaL#p^hO|8xfzVM zq=26hQ>~(}T3gPFO7o`YEs}8k;qn>>K1a%S$O{56%C+O~1l&!EY*3e)#W7*?s%Z0u zs~PUx0|9752OFGlKFT>85pGz{fKl}(F@0aAK zuT=;fg(%&qC^B!Xe!+|hxEqm&ON$e@0rGsJhGh2l2qD-0KoEcd!0yJNuvmI>4sHAg2$WGjq#wX9V=^gO zL})r=+H2r4Eg6l99WL8B>LIDB#Fv3$Xc#@=q%RgylTl4}Y`9J=VIp}36$6CIP(1|> zFhI4e&ByLK`}Y{GYKQ_#07{Npr*HI#mQm0ytl`IeKBQH# z-s^;}-tC?0JZ=KGK9^>SH*PZB_!iIMd4FOWpnl^;wPkZj@-u0i{!%%CcXvtiY9Q7ic!0nqwF{uKHreX&TqfP2^m*D|LT(N~$XH zowHjlsWIeJQ4Tf`l;XRYUW>pHrr^LT{cL8K6nMxk3vBTg-c%QtSH z5g&I0%EFwjE)npxpFW|cn4gxbl`75`nljn;03qgIXYYjLqbdHle$q2~We!BrZc-bJ zR+#NFBIgcMwJ}mDD96T$libpSy#M)N1AI+N$=La$dVS{YXujBm-+izVIM8@1Y9JJ6 z0Djw`9*YE04g7o)2EU)k_N`9W4{oyEaez;s&WYJT&J3@&Rvov=&~Tlgd5!vCr!-P0 zu2ZwFXCnf8cgegh!A~*uGFIX2tN%G-}hr?wiTC^?Cnm;<8a+8U# zZM)TF;?9p&tlud6KA-po24y@iurKE?uUrToe}KFAGlhSQQ&8eg&{>dKnZ~IN{zX-grhnK ztz)DZ`=4q*n~*4s5lPz;ju}X)l@{Fd{#vB3M^ayohxE)Y>r-Of6?ksr{*p0KMB2Kz z2dtHpTV<>JrKJruGIijcw<(p%yP6Gk5}Y@9l}dwh3*S)A(aKrt{O(^agT%w#&+vZo zHQ2EqW?g1)I@!8kB^0$uunB#T+BR-pCp!L|C^o}xgAr&eoBZpU73nn|>m?cDlxLT6 zcY^<23S=0lMk)X60*ciqooauYqNW2@GCbCw+jn>Tfj-mCYAhSm!0R@`y8FIVa_;lW zE^s388U^!SmF6NWNjgN`18hM#w$SC93tYamc)wpIJ`Z=b~IM zdSC=KqSqP6!EjPz4~|BNiFYDvIm}VTXVoq>1AI_C%AQ6H-HAE7D&3}rt6KD2A|sqc zg&3Mc+wqSt0TACw!6sl$`Y&7I!FZNbMfdJb*&L?V&8+p~)%KU)QnBi)7y{2n zMV_Zf))hZ*%fPkMjN@Y=^w?B6?yBsTW9jrthrX5Mw3(a^neCB>TsAYsM~}Pl@i;?s zvObiAFeqO<)ZsOY`Ibmmu-{lDMLzSLF~4aBj)abT6eN{OLkRyQt_VUuZjk?NM4_C> zSRi4GyncNGu4s0dWDb=vDkEl81>*F|!6+ok24#OyhnY;&MN~qP)88mhgxoI!mn^I3 zo@cU7G0yw8S*SEs5>JBnIVgKgEWahGpaqLg?+oo9+Fg>>Gmi%(TW*o|GDn8D zUN^mE1RK0KzGlAI2j0_zei~}X3G?pFD zmW~p7t&x=G`r1-<LLcCq0E8 zJoUw<>Y3!IpT2%zXX0~b4ugAnsmL29du%|l*GHDeLU%fS^U+%WGhS84x05^j<1`?~ z)UYlsypO|--pwG$&3v;{*XqubbG=x{k|sQqkDvi`N5noSW4fUcw&%6+p}b%>qg`_r zpvgfYu7!H(RJ+~`5`=P0luE;)YSg%dgI5znvY%!k345d?LO{^6WEu@F z0lvUCIwx0?4yYk??y4$F!oSOz;JekPdTJnbyV}ZiXr|Ni74vPD&rE}^M{~fl;ZUzh zVhOOOh{SD00e+h^*#f7K!52Fgo#?2)5Mr?DkQs=$_Au7`EBJN?qW?TQ*MD|N@tN;y z<}1L0M-9N6!G~e^1|3{NLpUq}k>`H8pwQ-ZnJa@&GZIb|NKKzgQXo8{)2ngd^C}7< zHF2oKe;rC^wfj5PKwU^^=6vO6 z`XLelGpb}rmS9&dqC$okE`!WtA61L3TWhEyK5Xl)CssvQ1yz`DO9>g}`2D zIi4wjpsmjfu27LIMqiXp^JW<(myiqShFync)_CZe1v zKKD0b8~N8qH9-i?h>4~|yF=pW7+yR3D_xC|ZwCg(7})l%$*3gklzokiR@9Z|QIF&# zGdTvA)!vPJ>R9E(V2nbl{6J)EA#S*jVwVJ+v7$Bpy97X$>up+` zycY590WM5xDw_zqwDlV|AZt}g*d^zcF-dGYHt}bqM=D*JSurPmjnKm@5n}tZE%I3dHCLBKU}~yX%B+$g zsS}=k&>;v`2VcJOO~*Hdp_*Ocn_5Lu{qa^xB~E7iW_p+KAezB;sL5Qw{qoT9JTZ$1E2K*T zi7H~wRtvLqU7{s%*sw&~D>(v=ewTFg7RLsQCs9xnYhXi;nJDN!mXF zqeR9Mxr0E@xdUz(;~0_Fs6e`G&ABaSyr8zp@^U||v}S;7B%3m}h-*fgwIQPhS`Hkq=0Jl_3jW~q3N6s1eSd5K7qRZ$ZKcgcM+WiO5RW&a;THAj+7z9bpNYlcvHRds1?AtW@T z4}RPa9iGLM|BI-zaEKz>+>-?QBWyE=bWZmVcwOeXpqUy66{wZm1Qu@|HF zok!qr9toeL*(kY@=DTe-{TNo=h#@?-J44HfX=rs2Vp|ek!~QtXtxI+4FLe*m-8@e0 z2j6MDikvo)@(=;?(Om5Z^Xgo9b_FO9%#Vr_HRCnZMM`&)DH3Y#zlttfmlU$?d34FwRFzKzrefpK=P_0hkJuI*+Kc)SJUx z#jg#_$RRODcCMl)T%FyYM)fD_#fd~hBSVW4%5Lu|tCV`RLLofwQWy(HJ_w|KL1!5* z2^qYHU=+$&W1RWhboQFa=M7+@eJ9Q5iWPnnH{XAMi{iWC@J?rY!?(6w3Y1zi&H)5| zrJ&O{3;I!!v}Nl7w|XIMhb-{0x~?`HL{~TEoIR9gmf5;r~;(#+tLrbJRWa3Ulq`x*D$;5;<0Jjk^=e-FcI-O+t{ItexPd zA=$^muUh)=4=!bROV5~VGca+ndw2;&0`_d$ojwq{n9f8 z^_dXlc{c>8(9mH9!NW6-oBr*T^8I<(4MU=Enfzff2}#ObExfU7d3aX)*J1A>F*m`a z?&qvCdJ>Xt9pr3&1rQjCjQpMfGRWT>1|MV|RHQ+sR>_|dwBeUBd~hVPJyVS6)y(QX zHOqv$B(h4CPlBsB!sF}WN!P3u*TT1IcPC8t;d_sc&7lj)VWI6y%Ic5F9?#oF>GWo= z9Uhn4&2`L)D;=4$JsyD_KDZyyy5q6cBgyth@R|4yDmP>}}wWjt~&1^~xLc8SeDB_rDDarx| zixv=cLbZ#mB{{NU^fCMer|GD^$`p^H`=>lnTpg0lg38UwNIXy0yukhW1e}$l2b)X* z8=}Q&diL+rFWwu)!_O^h(ob8%!Jhf@rK6kujPz3UsfJOR`TBFj(PZW-fkp7E_1pQB zLwhJPgCfCR!$k5Iwd}Cm9h~Rp!`Vr!qGuKobY4Qx?w9sQt619^|WDiOT!+0 z?6K(@LVU}V8Me=7JtMy`sP8u0+PDQPf%TVpaSQQp3O!UGb18EeyGz$=%(UUf+T)W7 zQ!I~n3qSn}rscIvBlK8};KxG=_5E-8*rqI9dA2rwczaV`kkyvbm=E*!n z8e`p^95%?g2epVJ><+|}5S9$@yaSZlDo(Z^T76`_=MFJWuDY*hyqo3_rDZ4EH;4l_ zu9-Hm^o80ZXnA8j&->__(fwj^jKL1KXobYy@eEN_(S{9QWPw@Y(@ebZ0Mb6^79u zvWgU|mbIp&Oxrd=StiNIXqc%Now&=ad~$JsAt&R9A?E5ht0p_A;%a}}cKOR?FEql( zC;18PAK1Xbj-Kyn(9y|P9~Udn#M?e0n^WTHtB^Mx$-6Wmcg_J4589Ul-otNJs9~jOV6(0v?K}%l1r)f4!qGW6)?1QD2aKh(~sx&Eqe1 zUu>`&WzHs%908mt5Sj>zW_t$byQ5EO&v-Bxk0~1wG3;MTsY2&RU)6fNb6Gn3LvnTZ zm87gHre?v1`yiDdM$h|Rc2wZVUeQBnlxQ8juV#X0WoU|fkxreyLlFb&E~qoDSsa}@ zx${!fy^#8|^#;%7(&Op`Hl}s+MZSemW!8Sh0l}ybba+lK+U@>gI>yqLg6cwDOV0gD zU}WdP)l>droNvB{Z%zecn|jNoY5J8%WN2GWpbb|0zi0NJ^)ua02WZ)=GjcOAn=o-f zj6NpPZipnND})uW3u_xv;?TT-kGJvOx?LM)VfxQ5G+zLA;Sua|e_khr*d7*nmzxm~ zv~ z0s`4U$($Z=nI;2ygy~9l#wLRT?I1Ez{SVbnn>vY7@y~eN9h*o!c|S*MmVt?O_1l$( z$k$)Pz$HA$Je9eg{Zv8t8NSepPb(rKlD}M|83kL$r6$@i9<@)ZhCR%IJB%RG2-GL+ z#7#l7h=z?n2;L7iaKKFYPMZo5oi>A)aeVyyqhtm4wEsub{Le<}SB4r+%H32E+qfcU z%Qcb40r2g^OfARg0H>Lbqj5%Y~y;L@1t9j7K?U z%O!SN&xo!Kqfp)r1GH&xRCT*U-;9~wbj>aTtbGobv zT|)xTDInDqqpzj=ERdj|Gzn*ZyQ09rV{9K&{aqkb>`x|JE; zt6n3biF>u;#rhq}IA9#*Nez}GdRArEZgM~MJaY&Fh?>-(2z5v z;G`e3#D#!)q!}weXAR_2M1&K*w0Oq(N%-O1!Q!ZYaHzp=uuCjba8HH!H$F?#zqfww zUHx|yiIm=m0l!g)D_}`uR#dNwaDQg={p7sRQDN?=^j@Z^Ydozo<*fT`%pR*Gtiu2i za)IA6^nai4cfV-uO!Htj9d(ggF=9kL%{J^)ZtS5T@ep0TPCwG^P|la}X> z`)GFT554=w`}x9BEgr6tbiIB^JA@o5lp{5iQ5d()FlUo8QiLA>JqzDG+Aon-`$rY# zZB$Y-*JvF7F}|rXf~!=nZXEvpo?{SDKuElk;n>xS9F3aZT2JkuUz;h^TZ03)6v(d{ zy9iE)+xWZk|E5d!4VoDDmJ`uGTm3U-mS`mFdnNBw=pFrqK&kMiD#@bde2|n)i`)AD zx~IFJ>!MAHXcQ#h!md=^eE{RMAO?%ym_Nh2G8XZ!lva1XTdzod`YNQ;e$3l?ktR-s zO`0k?t6RL;rK9Nce2*s2%b@ElGz$@$^vIjUA_j~5mW#WS^Vma6%+FOHZA3c`eMWY5I23T1JGrhM}z`*HH!igRA zk~c+q&a(a>AWcQWds-(a>#TS)!>PvHsVbhnm{Vb#B|qAPAu?~RAce`Q^5K(wmPUjs zm#04}mJX`MJCnk>xTMg5DG~pRFTFjoFDzHcVdJ-#C01=U>~u#~-T|Q0@ji0Op%nOYPsY5dqzMm4@K7r zUaI59Y}2lrY#CcBVd~Lv&9Dh+yw4w^S9*KC_wjjJioN$8S=o)) zv&~8~;vR*cX%_124C8QYj>RnS5*IZzR!~%a#0je#ZvYAWHJN#2QEG4m7Sg9zXBk)A zEae(<285^!DU}joaw?T^&S)vl9{ECWsU}O(4mWx5XyC{evjJxN-77alA>n4o^3vj{ z%C0uLGS`IA_2+bN6TgK|9+$*@4c^{B%y&vaop#6~4!=z+^LJky_evszM+P-;=QMa8 z|HaI*`NNgEP#ZhRf?MGwxUGG&IRP`q6ZFo-{V7M^m&=$_gctvI^iMYIGoJ>XuVF07 zg07~7t_9}mY>@WuaBkEfKJJ#>5_1~>57}e1@+xh#Ky#F)3+}X&oXn%GNk;d{60}^4GJR9!$kb! zw|O!~;mM)6dAn-zT#%A+ed=HG%)nWPa7lWU-s3hY(P+tOI*K60`UClVy^|YlFCAA) z1~gbPlGj|wor_GcYM1^39=9<+6r(w+dA4yo0~5Fk5xCtvds5R>4%RT-_S;AieQxEf zo1W9=;@enP=d+1M?RETcEOj~10+J10cW%w#{eu+oHgLU9+nTjRqdnvNW&7Rvar<7` z{x_@1?C$NDH?Pw)(x3OfnHP{$J+IW0vn+udnfZs{qz+2LA}vih0ZxX4@$xrhhzTtH zKtzs?IxB7L(2P5e+2Q6Ue<@wEdC|Ly3ru!SMEgI-a>8J-QWYb^i*0KTy~-&xTvgn3 zE$oqa`!e=ybpL&h}(nnKT$Q=Ci9*Eo)$B|y@CPL}_;iGOkP;yntdBwDLhixvD! z5B-V*0=(xX2{M=VQ!7kUmBbuCAxelB(8 zP*N1HblJ+isOvQ$H5%0t`*CNW>S=;v0EmY6PAm3jSogWg?OeWw_a;iHsPnS3(fBmh z?LJzc=IQ!?ZU}Kms9;(2LNB47N+8Z*iUX?RJCs5N8gt*k7q3UCEq#BI`dn&Tq$hkM zG4CDH=i>pd_VBVF9a=Y5kfm7qlHI=bo?c7PdS~g=BVUY_upm!>6lIm5Yff$WzT{$; zX)BkEawqc;&ppcPLL=foF5)}lPwU9OJS|J#iYl7Y!&S^HSHcfVDoN2UqIQ~| zajqsW6oFEQ+*9ex<-OFg$mFB%DBZPS4BGObPv^h>V<4(79Y-^G=8K>c5?KrF9^dN_ ze{;XN`0U#~T44oG0}P_ZgVDd^_a%5n=y0qW=x%Yt2weg>-if#StzbvVh}^=zvX$1n z&OwG+iE+xr>(V6QPs98<&g+LP-u&nvDr_~5(rui;>_yW^6W6Titxub6Q=S@^vzjP?NB zIWBDpIa!Is$_B4q>m=3Bph^r@Hhz5+LmOJP+=XwRt~t)Hx6OuvZDODN@%gp7G; z>b~}9PPflxrW9oAOJ5X|6E5`Uu&&A%D3q3DnpDe>wB^az|BuhcH(Rag4^KaGBIRm& zmnsjv`%d>99ae<3XBrpRt}+mIF!ukwGq6lWv2%zYu5+j8O_%MMl{F#rPGq+ZZy^Qb z8TktQZPM{BzApAcrqr$AVKB{oCm~?(vHqFA6|W|I1>B`iR!6YA^bZ?RRBZp%I8w>YFt8JEdx{Mr$(o7QrqT()emFY4&(3=q4ABy(I=~SymUx0;+`yc zW@Fqk7tr{P6;>SB{CW78e=qXh_F!x-8>rLlPUr}k{Ymw@{i}*^rl&IS!o+!6+eNnV zAtAb{r?xTHK1#FgxJ*RzLBpPZWAh!GJb$ryhENkVA|i!l=-kvdMNs;nv7ZqOkYl1Q@zje_n`te^Gce5F80kaX8uwn(ol(Ba=JbxB`(qHT|R#je*Vk$@6ZSmG@j{Be5L4@`8IZMo;FZ54y~+B z(zCwlgb5k`TvZaZ@!w>ckILHMi1?vFO~!&3U=DLZ$7?^QZ6` z2eM37U9z|wUP4-JH$~sAeM(TD6a#ZPKT`V?b&D;$e#OTn=8uUi=^nwE;ASd?dee4O z7t~52VG{l)&3Sfk=T%lk>(RGT_Hh=539cg8`46e2ioAwZQ4IQ*w_<#|Y~AljlG~q*>c^CCp-snBxAFWLpo9D`P<7cKk$RjJOuhh#%2!*IZOTLj|HV*{K28ca+ z4yrCYI4<1wHBpnj8~caLRE!3DYp$i4r|vwq|KslPEu!kLv&Cu>-p$5O@Qz``tni`k zyDOS_Rys?i{AqdZ_B4E�)RnUQbI-)c;5+s)&_`VUjS;%=tl_yXPM=usGmxwJ{gK z;~aH;kj^wX1skozpS?a#rF1C|3_)-(E-4Le_EmC7%fdi%%;t&UR4Eq4n+y8OWPfE0 zYE*%9@WsryJDRo`V>v$>2)%?kt@eosYyT)5-WSTthb^_Qq zUGDyW{}hbE;dah2Fd80sT1`k}Tl&VH>Rxn6PtwoB<3oXcn&%0}{ ztMhyNzl0eYOYWGB2tI|GyTn2@6K7SCnyZ>$)xu?sCvwCP&H9>HWeDjL3qFd1XwAvDx z$KY2{KOu#mDco1{PoIg7P>B54X+Zt<<;Xm@)kEVV9~iQn@$d^#h>X;e?qkwqc63y| z9Wlz1n8it$`BOL{8yev;H&TK#Z#y%*x=x99NSngcLYY5l>2Xr4J20F3c9I>-SJ|pn z`1apBK|@2yMj+!4f9lT|Cul;!X%Xc7QuntP#T6@q$zOSk-=}V){#OTeHrcP-bGBcv zJg|p~xObmi)PM7_8-?TvwMbjn&ne#I~5 zAB=1vY4H50@1)$CCj3O~x(BDC8&-z(8=h=!N>9Lygmwz?y%bN$4u3tamq0&IiEhoh zp_2yUscyTL(uip~onL^92^xYeYXM11XwUNIW?OGl{B?7ALsyk%w~@zjmvrpZIYQY+ zT5qV7E1q4Bm#yGp_ItPK5O<<57Up*aZ=m8)Y}Hu@#dkrcyb4{}ei6Hul!u3N2DqMU z@ZddSc9RB63`!R_qN{3;bXJK^m_`7DOSQH_p!&FBm<@8H@VobPZ>l&EAPp&N)tFHM z1&ONt%07xwi&}7MGTF1=bE%6LN-oeZL1Obth?euX^ye6$!k>Bi*wP*kd6M>n=t0Iz zeb00stQ8ITxf9&TRzGUziq*OpuMHa)?rx>nj`|Xrcuz|n0=Oe)8_(mbA0lOz=>YXj zJ0TufdXcu*fbPQ~WjpsVtMk4Fw?-;ZpBw$&bko<}cf>@NNiM{?YNVa7Wl@-?(`Z>` zZA$c-d6x{W?eW1wy})yvGwr#=sv}2=johv>6zEP=|(DCCw6i2P1xKrw(IXr`3qf-bC7FvF^)d z!yuW;WlxXf>MF!d3Cj{O)yTzXO~yBde7AGVg6 z?ZG6?;b-h`468pUI8nIof2Q`vh>eqS_Yrj!$vBZMa< z{XpC>y8ctJT-mYtX@^N&saCC2UaV}jmA4kas*46L3+4tya2%>e`GBrpXd4AU5FvA# zR2*;f@^}ISE?>(F-Q%R=o`qki3a;K-d3{NPfmccHaBGD1aP5Nm>f0*j9a#VSTLuc{06TeE~ReLQ(|x(S^3~|&hNoCFnTOC`{_jRo}-9-xYxkkp{ALAZ>(v0eG>`LZ58;X)i9x8RFp-g8FBsmtKP z>>I;my*t968!?CaY6HtaYT%vra_8g9&<}tm2UTqw2lcgS-UVW;xsII6XyK-4;wv|Z z(u@4Qm~>tEHUEoyrJ-8a{A?;RL;@pGR(7-QR!*UwSkX zvq9%)0|Ja*hGaLrqj2FKrt11fiqGM|4e%*9+#=K-9XXW_n|{k`Iws zcLqr2dvbnXaKg_67wy{Locu8HehA>uHI=pQYvzMI3|cBX-5k=I(wY@XdDczmEK#}| zm3U^;mf6>Ma@ntbMH@_hYbi|AS?Nj2VfF^Ve8mGj&Vim^c<`DDIR}6TR_uM9A}LW) zUBe+|S1QmWYbhS%HIygw42#TIJ2^WQ6?gkfGtHyOBLr0Y;(dw4?M*_ZA*XN*m@Glu zL9r1xRjDl;SSC#iGjb_9N}GiDY22nuI;G75qes&Nbhm6DQtb}mO_S%9QsxAO-^~sJ zsa%T5O!sZxuo7C6wi}ar05fsDfb~86&%z%K0HwJ*Qn{$etPY19E`{yyUVUF zJiTnD{BsBk3mb{_T)VHCXsC-|vR=+tkLpeV{!~ZgJ51UZm_-MC~4r zL=DFaJ5B8S>j{{OQjJJ{D?aYg3I(duFLl~hI?5u)gu2Gzap}UcfA(y4rwgN~gm|&P zlvf6=GzulX?eKfnslA5)2xl$BuKm!+fpk^;LK|<2kr4(!Q~3DPs~a#~A2_igA#|yF zV^%Ya;!b8OY6mI9wIe@%!t;wg`jLwa$ILEN+0r;DsmIA=1{Op*OI{CE|63`kp5A4X zVOneha>s@0wyd5PS@9Itgzp_+P9Z{ zVu>YNqb6qoFttRb`%=RpP6r0g1CEV3(3j*!)I!lS#!G+m$v=u;paO0p1eZyfKihR&Fi#N-Y~fA=KMoWW6KsiyjU0SmPx2JoqwRedYAj+-auw zK3>B_3j{sX$R`wlIDJXqy-#_PYTw>}?LP|#WJ^;H9ierPYtr5qMdf* z&DEn*^K!gjt`;1!j0Y3-=+YYU%cpr@SSw=?MNLqudsmj~bx%DyE9(z>yG4I#o*}(P z@lW_KAcN)$_^tTIPmcYIM6Qmkb-Rhv&_X$ECE|=R+lGfh3{RGzWZTvU-0eQA3qbZ) zcRO6svG1fCm6#7ps!ml^UwL?b`w9`p<3X$LijkOL$v&k+uC3)3A1gPH?1!bqWfreo z=c=<|KK_OnCk1*xXu+A}ya2Q}%EVIHxA_Zfk%Nh6ul6zDG#UcwoGLA)%TBDD9R6xR zB-9~0x4i{S2PTek`k}&3vH1(Ya@eR-sLpjiB<@qAW9!%s7Q0GV;I!tWx^DE>^8T8g z6lyZ^pw%{QKmCk~3A_mktKLb(SHcdpTt5|EH^SJEhKz&ur^OTKf7NoO6jOe9-GI|{ zh<&4RK}cM)BW}&sXHfK`$X_)>6{DmS6TYv?)9R`3_F?$bf0F)FldnKOEumGphQTB*kUFwxVev6yTd!Ym z6^i=0XM<}DAD3JEx4rOhs~sgdQY(8bg1Pm?-N05%&RMbE&P9AWjj%yVaNV;Sk)4@WMHJL!fBWhp)i&)gk$iqvc*tdh zsJrmOFvH`E+88n`FA=-k#H8QV1WX1Zg)l8Pw)GK{60j<~(m0RMzVN=fk2J#4RTNoa z|3&Y%D5m7Sl?YXQnM#YIO0ueP$BZgANz_r44Mbb8uGWm-Heb{~X(TF!!8Zr8ycHS= zDoRT=7U&9O+Mdydb6Fph{l;EoY43$i*_2X_zJ|(Alz}N2Ev-vQ8QyR~QeFOv@a|T4 zVF)>Qj7!;PC@P%(p3aDwlzyMBC+@Poyt*S@yoZ|vIgP@_YaD?BJ8@>RwR7R`2xr{IM^ zCB=E@_VEm%dv?!o22RAu_p#kR{}9}n1z(G|3xAdOH1*9W6Sskof!)I;cr*qduVZ1G z|Ixm6LyXV*Dxz@)w2wR$0xpFAVRx++1wqGg+`r$Se?ZCI7%O3EOBgNCE1{7|NT`+< zhVCe-{VL%WydZiR=h2)t|Cjt%#j4*_kRVscVsT$krrxi%zfe*xyJoW)EeB5U2Z+ri zOFM!ld`3gHZq^Ffl#e=HzV2>L3p&H`7#nGuLG*sc4rjk+Fu{ents?e1N8W@_!D&Al z%_t&xRX5)Y|FbNZ%yOl-W@*=!@8osdkvX|k;lEM@?&K1_l}OnLZfcEbbBnB{GM8E2 zMWA(p_Ha;~+A>v|5q2&L#7eR^K|0ofVR#0O$6Nw3XzjuU+C-Az`pdYL!kf2eESH4f zs&(VimJeUiZ*|(t|0H<+C!i7c<>M?k!I#HbKXFsLLEj6= zG*P=66L;dmAz5iNSiYZ(W`J#Ul;0-J4QwmpB@g7?N$TGnmr>oT7(5UZ>|5au-4=cH zxw5`jCH9Fy{*X}=@xH*fUg72Uo`}x>6>{EQ37^*}&kFAzdo7)kGzZ;ck*s6?G!|cv zE((PzVwB6eOOGC~Dtxt=XBWN`9jb32%S@348Gm?*>}XXudOEO$t#Y5Yp}He1Id9%4 zxZl#PCU^UpV!sM$m?~f-;Y#~R1~y38@n*F^db@FOWq;J@Oh!!bY!~rOy>Wz_4CO~S zZ^t*_nLi6Ii%(dcy^1dO>Rb5~4o+1PDseUF_F_YmQYDj-D`jgl$3cx_1EmhO6wm** z$m;9dW@yIRpLRYCfL#b2a9NYP7IrlKzJ>TDJQduF(=nbk!OLbb8-Ivkn{b=^e{~fX6{Kc!dBNg({^guL z-ZUod?W)}B+`}>Bn_%|w*?6;yRjV3NbHvYq)7WHGDZU4)6+4{G=r~~af#~O6_rAh? ziJN)mjva$>{&4!KUTyP3FkqiA!fQ_l9r7#=Y1YbrUnad2bZ|9g#k&Dy_35l<=yK!5 z3@|_dtkf1B8meuWb?zXl1a9V-h}uF?e#>=yQBcR%SOkWnGS5lVKIkLcFc zz4N2O@*(X-)}Vry=n+#@3g`s5vCA)%s)%37@j3enK#S+1tW~7+GdoVdB>SFk9XngY zi}r$4!Q2E;o!k#8Hj6Sm|JdX}9}-5IT=<#Ko_}O>yf$-*;Z9G)At@SC6IZXCfK~#d zr!J;-m0dvId)_Cy4GwKLa;Pib84G3ieMde;52F)ohy z2&Ms-MPz*@x+5J=V&PkK^K$LKRL(7SEdy^~fOvWEMUpNRUwR_XzBbyBX!Cyn+NK2U zKpqW%NKQ`=UH)DH$?*n--3(WK8boD!%59=TM75cMLU*8?Vx>StPdmG+PFar5Yb^b1 z3YV`EeMQ}MBP!@ltt5QA=rL0GzN8P2G5+=9OK#v1hM2?ln!4zQPujFDY4RT52{&z4 zRi!Hja{O@ktwFNmGo|3N*$GJBEDrdP%oaonghLq z2Qgs0pHTOx{L&(=6K@q?J^@jRA1sRDdUe%Dc`*n456n-#dZr@1dnlp6;$t^hnXPVg zFN3yroBm*qnX{V9*k5hFwp69bJn?KL>RT2;=-Ek^h$T?mw1t^_xtL>t8c|=ya@-7A z-nb?l@aj{cFB)NFZ0a3q8vG{gcY20&MCz5SRHyrlF z6on`7_B^^c=-;02M1aFdxkrbBl|UgGe`GfuYHfuDW`M>{<)?S0ozGEq_m8Gp9twByxZ8i%r%BTB1fi*rgp3zpc$IBS9WKHr*4qm+hgrIbDs7ESK8 zd&A^wh*x62Qo52JIyV(vJdLU#Wep<`PJD(&PSYlQeKX2`)Sz6w_Ukd{-XYSm{n|?) zOz=v#>|U>|=}c%}oQH-5aSe^t*&SrRyD?z2;Y##$P1w{Te&Iw{98kOWe&#Av&%xFR zk%KiGc{Sj~HPe+e)VwyooYwfzygE-?><_bb#r&|=N7SuIiO^uqzaLH))PCY3CCbFo zIJo?|F^859q44DOl(YJZ`TOZR7yiU0MkC59~*bo%@Dvq4TVu4)mAxz ze%$r=SN#l-q0g4IHC8SrHs=ej1&?~Yf3g?)v-~=y#-|yn*nVISO&Ou4Ouw-~%n!!E z@=6GxWwAOdEZ!rn(I>F?$_+H7gIl)-2lGC*&S`sI8suRa$3fm>-V*m3RI9;}K(;_9 z;hVd8pZ9;zuq+ninz>cp)2j{`+Ojpj~lAp7q$+f0ymq&@v2lvKMHEY4%&jY-!szM7}$z8F`+5cHm+U zWTqO~$=P0=BbiTByuul>j9syxw(+>yU+1%S45=b6KHH$xCGGgM`_X<=Vg~?_>p1@X zV&8>M+b>C*_8u5r@?EKhA?%_FQj$&p% zM9K}dqxIO^N~1SGadb8U;FeA%313r85^)B@h)%27A;6S`&GGXg<^9?W+e*9pa{qLVtT zX`uFT_Yjnk+t)~|#v4es*G#53+Nq)2?5N^Fw>`*FTNbJtYh_%|wrPJLfRT%>ZB-et z2uvelv!3VJS!r+2obx{JL>qe2{{TNxt;~r2VVLrz)IrBafYE9o(Z`8=rus#t!}dpS zz(A6jmkc@&*%WDYzsNNnO7J_BXf(RV=_n#&`}m2(yw>#Ys}zciT7J!-GPaNCV#g02 zGyu)gE8z>={CMS7tXJhNTwh#dN;esMvSm^ot(;>z>(^X_-?FrDNTL>RqI8>o9b%~u z+Xn_Cp0=f@4~$PtC1w<`cR!*J$q~IrmiQ$6L7{V2dqz3Zk-+Yx`+Dw%5X&AE=x9UX zUcR3{{sx)*r}-aaxG4t^?KD;D~u#}hCSR!1Gbw7%uw24q-__a)g3?XHN|PRV}IzgeIV3r?h58{rfpvh64B$`f+@h5&I| znL~Z`tfNJ5Ju~IL9t&d86K2Md3WxLkEUo5GM-Hdcek_rcf{`u>71gWo}cE@pA6s;GJ6e`V%k|4=JqSW@QINA%TM z!@8pjz$*OTjHQYi+m&w>#N)M=>T{Omt&5r84Hyp80OnHEPpc@#d~D!Xq2y`U%r8Zv zim?kKLH8$6>ql0Z!t@rF(Fh)3ypRzzj93oV7ndw8%idG$hOLqZo+=;d}?<92V@4`RVDcg$Nm(@ z%4U-Dy>sw~tv1-I^9%FPMGNGACTaRr5@b_Kcha6^`*`kHk?# zbYVwFbBv%-06!SLIj7K<023af$3G&u#+wMP=7BkaXmMtq_FPDcSi?g)z zeKE}sZA?=I@Uywdl5>OUp|lU|*x&r@v0@nY(p=YnJwZ2z{wU_88ljnEOREEwT2~9S zqZ@dL-rW(oxhRQ|c|y!`#}8z9VL|$!RebH~$A(OwGXN)+`@-g_|4+cMy)0jmR1Bq6 zxSqdWsC+Leh>+T5yAG_ZWw=X)qm|p=>v6vG8Gd!AesX4eNbZ~$vi#^cq`n1HnyJus zUntdfeu|c{T^yLRS>MvqtCiDYkxnL#a4(4vs7hCTC3F#oDz2wsE={Jbg~#i;hB@J!iMG)mi`O{cPP$jFO}w3tX1E7nK8JeSFH+8jug=OP z&mL#_@vckr4<3(J9v=?%TTFh%iHg2|s~pqss3--Uv=#|ui7RYC$ z7xE2M=b}{Ab;M9h4+9wHQg8=Qh&2u?2hqMj`^vt#u)4kweCmZyzS@B7uO?)-QSUo% z@rz-%!|JX-`rSy8YVk0TFBt}ixB5@xp@wt;g6F?XrlpjI4lX@BzstOjHGt?sB}%Fin&S3CVj0M4pTzT~wCI0jvHnP9A~kM!q`kHIpB9*B1M zSx0V<)NTK}$QrKT3^=ezQb_8kJgB9%8c0~1qhgTr!R(2$wysoT#+ho8au~*mOQu6|N z1e121j9p;xMgb2Q(a1Yy9(t0FwGH36@S@l9i;G$!U~GS7W|dkCRSeJ=M&skd-3WkD zH3JC;D*{=7S)Hmna|~80vvg0hPoMR=xn7dd3%y)M84Yjt1MSDf#r+0sYL+9icpBN{ z-&A6Ozb_9K3T_4Qo?DZ}ZxEb&OKO}Gj7fHpm*i|KCBI85d;Bt@-Us~%qW)luA~E+_ zMTWx@Cu{%!$Y2f9qf2YOQNhUUoB=abeglPVgv9!Wu0 zq0INiUOh=G^W{CSx^K=-wH*Y1b<38pvt{vp)OgVZb;VPm z%_uP|Rp8k?5~dn56hNhn!V@f)qwp6Lad@iO%%n)4)FZosM3nFUF+}ZyC@mU!0+93| z<+H7z^iKAhc@V7Aq#(NYo39SHDowA2-38<{fkrE`@JNnNJLY>HLFu|?{HU}~OP+7~V z)E-Qwtq$Gpv{&{MD@oV<$BrL3Rsi*^&H#SFZa)6L+1MVS;cPW*OlP?3eY=nxWLiR+ z$2prqlip5W&@BUrPp}6Xy@lTrAg`8O?>i79Kbq97*rce$n}l3koA$U`>hXr!b}Z!# zClyfI!nUNpeDDoo6V|LLnPe4c#6Jrlqnx}6N!jQ9c`zzCvNiNOoD;Et8u8II_@LUl zj(IVg-C{{R+#C9URuF8r;2BGS{PT}6`Csm&zTfSty41RhnGoYJ6%F4U0s{b1{H;4) z!CT8W@`d8Kc|lr22Bz4ipAAMqLSnG}A&^utjNse-P^=Sp(g8rQ1wLb+QrAx8ISOT}xf2(#ApvL^^;!jqxN&u}2D925GSHx)3nq!b`yy~sk!0U%c8PlF zMs7uLh+0FJ14N;uXZd-ey@ZG(M4nAI1ObWn_ZtG`kdf=4qC{oci90)TgYlPM zd@hGp*!=qy_WZ@Bz%)J1r^CG9hDELwCz{#cIWQlUf25Bt>nJ#?Am#p5Dz>PezbwyVqrnZ0DRW zOUrd2@oag@oWm6f7pC;dP5u{q&r#{}VK(W+QjQs&xlV*~ShY>l4vmt@MMt*ZI@(u9 zq!o`xJoZ0&E`PlihdrCl*(>vf2S}U`+?;5Yso(0~nT)thXFq`N%JD9Muj?IB{N!<@ z%HX8o_>Ikp&A_H>xC0mtVF^2Cc-HX?BZ}>eEiJKsO1o5x(_c60r@bwh#)F6Y!x+;0 z!4y!GM)m$=VXWW}(&$%puU_hc-CZdXg&q+Omeqkv3Vmb_VQ#yH1;YS~%&6MXld z<&$4+8WrpLr(^Ye)3`xg)pOe6S8S;sf~rbkwf!#C%(?Bb%f?~h+hZe+g) zz{vnxzUZY6Ch8Uelh$nLt$`0JHVha!@SRspaXD32A~;LIYO95DG*FadE2hkKL+-iF zH#>yP2XcOAj>!vPRsr{dRrNR68#!;uTW))aclIBBKqXAd<&O1<+#J$1MN$-!Om{8)eXJJ*Y#KSnGP#4 zW%hDw4kl|s#;+X}^}ToJ2uE66gz?KBDD#)SYeyPO<+++?7F9?2bS%@8cx0hddIxU5 z&?r@ACjSp#U;PzT*seW*w17$~DN2`szyMMzAPv&pB{g(+halaZ(m8aOba!{>kVBo# z`+na#znwpTS!?Eb*!#Y(T+R+)B1Aq8&bnz*YcuRbxm$3Y>CBu^;_O9j2%2iy8ewJ@ z9CKUGV_C^9o94IvPz!QU&y!5l!!M|aOD@$H9tO*xm;b>~MzTGnO@FV2@vTc>FDLq? z$oa3SLPG);EpvPG`_+J^e%s2gU9-U2Y=)3TYJ^^g4m+W?W~=eC(OE}ykRm5o#Y^)2 z<_^{%QlpPv(1%^ZN;t1vi5IYAD;Hci~5t6rwXqkf^As@ zZ1xow8d~ulxQ!Q#ud^rNSh|dzi}ogahfsSp_?w z)buz8hSwWGW*}|p5VNOhlmENcW-8t&bclNenYp17O zcEbj>9`l(0a`8nq#I_<W=TH>T=a>?-@x!8{9&S)1XOVHzh)fir=TxL1K?oGTUO_sP% z>L550BRXk3yQq8T2n>lO9k#O#zzZ{2JAq&np^oy=s{U%H9Q0QOgt`StT|IPaeX|8a zhs9=&Ym=hVQipbXsgo=0ipO(%%V-iUo}Hi+ZhYKpV5F9>A|X{zq<8?2R51Q zi+E`YU*YK2_^|k=q*l!na+KVmdV3rHga$4&4S4A(`CS9cprYCmahO0rzUL+k6xH6f5T#&%9`k|nuX$R0gYSTfY`wwMsDEtrX(uW6=tXjD~kdaxv zn_QW>zgk&g(jnoypZ?jmqIy~}XA?T)?^~_Jx#?uo0h<^a>A*tVC8b+0KhFWK19`?>gH*0z2%A@h4G@cz6t_tv-iu;T%aNo4KBVU-aIHwrFv7Fj0xmN6=Th-4brc$AVsc8XuL#j_~ zNG~=&o4N(v?ruI-Mx~7(m+Ae!kw%duu^RT+nX0d{EsF%-c!VHC&+P)frK&digIsA94o8zUPxWtCK z!B1}9{%XRP(Jm=(fv5X2ED=6oqV3xuH=))vvMQkxekE;`*Jf|&)!zu3AT{O96hKs1`` zG!9xS@_I3Bp_*pI-xdsT-_r?-aa|dkeHaq>1%I(>U^XWXZvtSAX--U$Yk;DaudZ$T z$=(YFnLu1G_YL_0o69%%jyL(<`uGX#WmmKMIE@=2B^&1N9qA6ej`M=ojc1gBg&T{6 z#4hV*dnR8EHuT=H}_p#sKy@_D-YDQ$Rwem!g%sMqw22;jD=Y%yR zfiR;0cn?Kj{t02uS_k_84I#6kz8`?kW5}<3{KM1Z9RMnM9U-zYzJYmxhNOWKn;}8m zWp59{u;IX2FA@k#rfcE1)dYrgDin!pJ+JZ5c4-zqB}8RZRxp*yjRi4;02j`b^7#9l znFfeS`kWe<%5n|3W~#4#1-yK#2NY|V%fNue`7)q+T@p>DcXQH)kx~G_gr|S9<>yuX zCV8Fmm~Odz>q7qB#N5~usGX1mLK)6uEuv(&f!+5`va~D@L}lqY?Kp7t`@ldxQdprL zY8cg0#{>ZLb|4yHR;dk2ZUN%N0JLKp;3cb##Z9|$xvmLxD2LtwY4tuo%2J-Ev{&- z(CegXD)>9phPx2ar_mS99KaDJ;y){ZH97%;XvPgN6m7>UYoXUfGbr?kZ^nIz*+`{QPX~5Q8A@8(U61V z=x^QQT8kLW~$H1Q0-G^1|(%@B@UEj`2J zlajSyVbKxEA~0*1L%K06P%8bvAi9LC5B9qV`IQWycqj9^ef52QCkqj8NMb@1`G>GP z+q8v&HU92}FmA0KrVct0{InBCLIAB!-_Qw2Zk;XJP~i(WcoVRzk%I&VYP?40i-bP= zHz$2s?a>av?s_~exNguBrU`_Du0Z^N^`sx;X0fsbzR!ROzgo1F!JK^q*?UtC4a(!%33iOV3dMCiPha{5yg3~A3=#b9Vfd;eUai)~;0$&JJh|7YC z?F$6e&hED*kQrHJxA*f-%Q(e4fG_<3D4ds|`+#aFeE$uHZ!)BDmIME?+2>rcgWeLe z&Ye15t>HlxHb^Bf{)^voS?>I0GXG2;wRmb&9kB;|Gdz(eQ9Gf^+_r|Ii2d~`vB0#ppyI+9i5Pp`6 z$d+1>{L!)pt_SWYm>Y%8#;7t}oT$8W`Xz61IjgUn;CDH>NE@ill8`#;xw&a2pQtHm zKlF_SP@Ts?a0392s>pjl&gvX}Q+AQB4`&h5;eJ~#(B1NkB$=$(-sz{*+E19JS)W@^ zm;m?S7*_r$xmWep02a^sw1hKM6|KbzAc%ic2y4>a7q7!n2Ys)XKVsy1o{6rQoNzba z*Y{0yTP}DG?T@s^A$@M;NVbChRc&y~g$z8-EIJ=>7-IK+k-^6BZ?HQthF>(~>{@XB zx?sQZ8j+VaPm!N15EV>x7^6p}UgA7A#~;0rLy+!h@0+gV{_?aIm_As76u9%}TU`$Q zU6+ZT%GbQ$AaQ)NlZFH4Mhnl&{V8WuF5Lq>jx=esIpE~fYb`a4BnL&uvxG5=WQ)nM zkA!9cg-pg6t|O8;!V!C-pkTs`1Mp>a$73BykioiOFoaA*fWJ@D^@wSrHm3+!2sK9Kz+EX zc9=g+Xt@t$9Skt@(3DstX*}7DM>uVo%1iJ#Wr|tgn9%eqknwN?nS;0>RwBRO8Db0E zLy})G_dGKV_Ri-e5cMGR_t8Q01z4|SbWPvNEdq6tK&I;#eI`RQ4MAz(u~Cvrf&CCJzyRA+0c{uLNg>BczsueAr zcm1ZTo{?C%a;A|f{SV^|Rfp(HGyhDNo?_q)FY+C?4;_&UfO<=-#+v%om6F9?=d&3A zVsLgDOf@`racu9+dvG2a@H1=Ky!}8K|QSufbXzl7Be%%Zd); z4?PG#2FC;rrP0af&0dSNqIK)zCpT;%*^GWeyeDxztT@i@YDK6PoaK)`Qp6lfuZm4P zs-LEtxXyLwf7zDV6tA6X$b90K#`JC}`7R)$iX^a**TO1K+I7Ob>QZ3H`*J$5@Sl>v z6U=ZxIgvHXq|t6p{CP_rpu%va2d6}x0j8KXqr5q@k)`6B;-R^G6~VkMH|RMg`&HkL za1*qP#M;Dt;(n9bD4nPAV568>FVC1-n*C^@`rkV)+Kwhi{!fhxOYMgWRI~5kKK`MD zfcE{@{#1#D$IY2B`&KU3VWVuXM!(0CuQ1d?$@4L`#P#9fAm;C~#Dq~2LH=hZ++!S~ zhuQD!_iJZHo_{xLS4ug0$(BY_+M=azd?Ai^jV#F+^F@oJnA$a*7%u z*pLnn6(N>)R4*-u5@@JZw@RXuIM@gszp#s+r)==ES^Dg`oH?K88)(yQsU=hY>Dfa|Rkz8d-Sp}znIuh7k ze!5z>UYB#Z!kU^cFDjC!o0}WhO^}v`9lfL7d&0M)=1nvbIdQ4hW*H%QrYo>yTq+1S zn>Ig`#%(xqqy;oy?<>cH_eVD2>4xGv{II+Fk?;3h@TUTD!iubhn+L*?f)7UlCHWT6 zIcj;<%=A6iXwQ}(ut8-HYuK1$zXCa&9?ys*?ei7~0?SAs zBwK3EC-#VPmb)-pu8*|d_t>?pQ>*^7G%T&p@-3;oBN15HALtkobu>E5XJF*Y*7bdm z41~!LGQd6AC$!V`d-yn11^;;t4l6oVZ~3h`nQ(u2L6n0e+KMK4V0-Xt0$6(!P(}Dm zr`_(F!@ji=f%&;E=K1vJqVroUFSFtR==WZ|?i+@}DAGH(CllwHciU0Y#)&15_5Kg! z1tZsHo}Y2I_>?`*YD{N!JYiciW?h>3^0}8-Hj#p1#B(dq4=I(RJ^aelNyp1uem-_F zm~DMj7Gns~3NPYH+u8hK-r6pgOi)-1>fH(EZQZUcD!Iyx67ExTXHODY2S3vbx;D8h z_Gm<7P6YGvcUaemmXa!0WWMAfJt;X$1#%x;2az-)G=88`p97r@qIAC3e&c(qZPR|Z z$R_RK5m4NI$YaUbF3aBg@^TIC2n=gdTpyn#T6>B0?J{Ig+AYtR=JSBDG5LAs!n%Jj zPUIlxWVZ+PAB#rasE+;u$s!eIwX?aVYXdlId zH1&3aYuN@x8NOzBR2t!cHD9vH^me56$Xsx1j;Jwa!MxqRDFHg#T=sM+r!)X{Zl@xX zqsQO#!7glB65$=fdX+Pr1(lu9?Xy6Xgqf8TWAEvZ*mQNd(f6n&#L)t9gTM81Ad%F0 z4*x3KFmKz7RzSbcS;tJ^)kbj4<2wDh1Slh_j8|u!x9%<0KFy&gUKKKc9&xIpmoG4^ z{;l9GY{lUe4Fs1$^SvNbayg?A6*KkP&6i4d34ZyY*9NpN+Psk=NL9IyQ ze~BuZi)jr8V?eU#iR4?e5K?FvP%Y}}m{VV1IVim{?=5N^AyZo3jUW2;ST3qc<@T{y zQt6{{&6oF6VnMiF=L217$o1(i>tmd*G5_ELs4{8(lzetk=%l2$-{p#(xc|JdmgfQe z`OE(j59nXae-<5X=}|17g;ulWUJFck2cBji%#sTXx>I{0sUTz=0(H30#Ih=?T`|OF z>QBk$Ks*aq&;`h)e>|Umk$rA`A6aK6RIIZ(hM~Q&DBZ1^>7IFHJhErY`Hz5n0zak= z$Ka51=cIl5?Zcum<)hg{>r}*mXFjvXQ#X6Rs6h~`xH<7qDleD3TxOc#K6@tH4SUoA z=OhRD=Js0}frsH0=N;08K!KZ8B+OrsxqfwkoAu4&%dwt_^&Q60OGUZ7=GaNSE&J&n z{uzQ(c+wEP)epg;LEMM*U2dNSpb~inOMuYqoJIKIVcctVALLg9z>SAZ+0CCH2PTYF zv5~iM-kv4=z50r;j=j3gu=d5gb&1Dv`SEU=Q)6^$2K!TxTj9~q?(m1hQy)xQ$&GJ^ z$c{Nu`DELDqNs1bD=S~r4K2>^U{swfH=0#U^KIp14~?RxmwTKe{XkG~z-z;cqc!^% zWWC+uJ%dc0c)v~@uw;s}?5h9wT7=7Pv}9=_ZmDQQli-yiKnp!HCRdte2OgQfalm?9 zF^KedUFmK{h1_!UaDez=^}V}}Vm&E?`DUC3nbY%^8^(O z73g4Lpcb4KeYhjZpDZ|+b(I!>GCijf>AWY7(K9dhJL&ueK0 zUxGMi*eK~(1-&i+p}s;acLubRC5YBx#r+@m#QwmBz}8vZKtt<57hM@BRplPH z6F`!TE{(x4RY1mjWKn)2;;-pq)1zJ5^57J?D2g2d9o;Udhah$aJK-NG!|}_+6o#b0 zm(}D9Xag}+?j+%1n0lZ7q4Qmge!IFu9o0ENcLeCp8r+u@)|>w#4MCym1fg7B9r*Fp z*1i(AYc_4SkiBG%mA9o6?hT|t+A0?aaig`|kFy^Z4GiyT+Z=w)jDH5AD$7lt8byNt z{dK3?n&nz?=?$L>s+-MnuMIMFacrcBWf31V59Yu1%Ql}4tKC+AwT;Sk!gLP^XXjHF z7DIvn2XS)2g_qZFl*vqMWkJ$&Lj>o8Pw)%GbkODmq8a? z*d+J7Ha1(8rBmLXph{2lsEmEu+=yU130=x!s-0aNHrthza`I@nRRs~PVo{w-`X%*B zAadTGxHz_E;L+Mo?9D|f7Ae{L%6ou6xj7aHn2^ixypN`G`9Z5hQe|x*6+slOCF4TW?O>v;`k9 z53G4hc#2wzXxD~-N(6P?NgmY|LU>Ee8dM6@t-Py8tJOdFh|7(ujhKqII7Mr0lPO>e z1j8&0#DAPDNh|3n^#YSC7JQBRk*Vf|{JQa>T+WYQwG>2CFydDj#PSmmzFg@)@E0&N z=rac)OhPJb-z8+g1svHGoMgLVvba#+=u()JruS_Ec*A;lAW;jhw#Nd&XT)^e3$ zUfxyesRqKbydeeR1c&9sKt7w5Q4*PZfnuKf!%+P;Cu{21dW^my<|VUItG_?6L_AQS zE?JuXQUXkjmex(q3Nh*{Nw74@kmUJPMfE;QayyavprZr?M4KcuV-z5@FP)Lqb*9u1 zI7!R;d`6wM&!2`*nDu$HDp*?8^tmPb-Mxyeum6nYNz{561s>H_EOtJ<<||KiSR8J9 zy~fA9Yva4;Y}lKz-AB}u<=l~DYE%a?_9=+i#jf~sg_Pgh)2d_d`GXyP7HrsW+lMY?pPe=U2$=>N#WWQrE6N5nO^8J;br>4mrOz{j4M1#GB z2oY2ge$S1I#$QUq5aJU@Q-o{c#_!BYTc(~uPTplSiyR&#rdg^t3NABsQm(kZ~ zF^YC_Zf1GuXS)$R9Cke)Rx5wo1n*f`7GvUd_4(Co$e`pLZ-0>Nl9xs`zz+*IcW80k zw6PYXQXP)2%@CgbvIFP`KC>mM6R| zza35ALW@p0e?!V*)$W753=lBpFub;PfJyL3+YNK(_%&Is|BTq}@d3{tvaJxIk+$>i zD}#B22b-y`Af) zN7zDT$s+D>BWbN~|Ls|Cdh=e{$klg8>Rs!ePuo6Tu9 zS?!%GUb(Nw4Sm^ubHWmZ$wgb8&iSfOl%5|pS2iy5@G)gdA^lQd!gZ}_9<29T+n2H) zao)m3RYz1@bTccsASBnf^GZlkm+Q@t2i42t3L70oFE+IW+vyTZG6JRdZ1c6k#OcO| zh;{4Zu%nRoqyso}makmnDn44d#&&H} zM`mXn$Q{wtc(2sdfMK{5>djxwidk1-emjbf85H!DT0zb+0$IV?7aLF0wXS3P;<3k!~Jqf$&)c;*vF>Vz8K4eMbj`F zOdg-LoBblT!XNSi8$X%LIv6?vO-)JF_P~~(J)gZzvNUr{_%doQCaUNsf%iRIbTJMi zTphX+AiH>-Pg&hQoJCSvy+{2dYenbS9fE&F_S%GJ{7m7fCO0&m`3oVoG~AO?)0DMS z5mi*6BVo=!N@QnsL=cvKYsBf{zIz1p(q0RDtkvBBSD+ov_mEWHk0oI5mZM11IpaN5ZB7RaVY-E zSYHTkr(8hK(3YUAJJJJ%9FKIty}F>`U4#0E_;>Xg;CMH-+^v6j$SkD;4r*&WY9OQ_ zgg+a4QYqGFKuA`Faqd)+@(3AX^53uwJiNS5{Uzfu|tZh+_kHhFE2 ziA8(Yh@Uz^Y;)Va&L3VW@yiWVZg>k*k@DZ3gIcM}df_yOJ+;qbGYK z+B6J|ctFg^k~G01*!mcMwM)~wi;i6^x&?W%MT8#|?z0XEJ8J_hT^u5Hlmy#fB3179 zu5wx~&MtT6j&%nm$HQ&6x(b=KsE@IjjB&BGFXb zH6Y1n&CcmF(6JhAjz}ODMrT-pS~K+`bNRKOV|HF4DO4EZWQ%u$fXbnLzp_h~oB_dG zK<+(jv755?+2L?D%)wz1U5F-8;~>8=RaVfDDutzg%eP;dG|Fwd)~ob>7M{+q$Cb4r z3BBJfO2Cpm&0*0HmP=kTZ)0 zLW@qmQmscxZ6LVx;QH?1jok>Z zNgbTM{+RvVgOX)F5GrqrI~afSP)eLm7ki4CAG^u-I1@U3<0X2n0K}c znb?PYB3FX&U}Q^_(6fOkEGhn7<>0h*f}0c25-?P*+nGfW*(IZvp@3O_v+4@oFUS`6 zZ{))8Sw~FK01FBuf#G8WYN)*yDy`&5k{jJOEAGB7cjQ|S{J5xeC;qYe0WbS$uk@G_ zkD6xJ2~n5$vzBWJrxlTri)F52XL-B3Oh0>?^;cfupA@N96kgRz zeFpo{Vw?Aq_VvX5SZj%Jpw4jeWaQh4^?UovB|H;&J?3I#3R)emOIZqr{SxGUj(JFZ z;A4w`WV)e^Gpc*(qa^bP#rTWYYt=YMMJaS}M32^k08&%`Qd8x z2RRn7r^W?qi)eV0kX@K{tKCWVNdYL(|0_u$9*_$o-jn$5)sRc|l<$QNTdqW!p(O$j zcshFTkoHzOm1l}x-bfn%fv#A(3#cVH1ImNuV%h6YS+}DV%(U^@V2O@V$`=H(OK(tq zT}WJ{rGVZYE!1)WqU4&4_)jRSKtJk=4=Fy~-$ti1y(|?9Y3=TD$1<&Eb5BL=px?6C z0vHgx;xIMjqcV)qiCon3c6%vWQ*XU$uQbk5xVGFtF&;=%nW{hFH$3V#B;707t;S%^ zZFU~xpSNb8stPK(#?@j}<=+l}iW7_J;>4lk4{4&m3oNg+neIW z4HapN75btRRmLL<4w6o?nP5N|Ery@q)L(*}aDVw}4zOV*zN=)4t@utTZP^_Zc#8UU?Q$c{nyLHwhl^^bL>~R zjwFTeGjKX~sgy~^#S{@D5Ggb|@EEPjyu`T87lF0?qER6py7N;ogXcpXhIE3Uua`uM zqu98g6~g0(2di70OWv_3$)}=I?$XfUrTk}g1KB-^;}=CzvI~!_pJQOKola21$IN>u zC>1BmR|)THE5gF(6ioaX5A=mm^_pLXj;=3KOF9M7&xC;Mz1dz!n;9~LNXylLLS0da zm66Ej3e6nRu~6?0dz>dM9Jv_?wk-pR-f2&M`Si-^YR-6Js!Ce2j)!{<5#-Cf<@8Fl zo=k3lMZt|)8;nzp6K_KmQ{`Qhg1$|7$v|r>l!Q5mmr>8`(S!THXZGNM)#-nK{mzK| z@wU^m*E2T*r!<&;Y(jMMq~V>YX8b}f#W-+Y>1L`i@xQAdZdr&vaIYp}4)al?ZT>)U z?X&)nr&jZ8u5tjbfe;>|Q$-D-#KT%O=^Sf+({F{TFeyY#iuw{!?=pc@MlOhy*&Mmo z0tlPE5P+rM~4%rL+hW*0Pljt-Une!8T<5yyvq~s*lSaT-V>=$ zSVWA(DWxBL@9o+_eBZ46$FIm^J|@)lG8%_4rF|2^x1`ERXj zqx`eouB`mepL$wuXWe0Y7#ym57(57I-KUC<-iarkL&V@d)}YViC`Tb87?8>%A9{o4 z#HNI!vyA5@EQ<$WlK0k6BW;x`(q#)cIsGgno{4+@dJk_9} zjXU60TGaal6}PuM!Vls$vUhMy;S^l`1~oeE zh^`ns3%;y2ahCov+!&CMq4rx9_%1Q~4o1f9u)>@!uWeW6rBpAJX`l(-sZTRoj&eDX zEXop1`?XCte#Hdk`ot8vXaht0i$NIsIK(v!0*Vs8h^Nyn5+m;vH)gEw7bEX9SO+w$ zbF`*Eb_v(PHRJi$TW3CLTxV(LHSQ$xE4Mb}cwa1E;^u9_IV9_u!gM{{iKas3;#nmh zj){t$74hOVRT4Z_Plo|+VYa;f1?eyV7|hXYx$Nau6gCI}BjLS&yipEl_*pQw*E5I*8i``L!9(wP)J_jsmmRN|C9bdKA z0l@+niQ13+>8ij=MG1k&PtnV`GGBBBGN2X<^9XwY(^k#HKPz!2Ux9!Patpaw`Sxy- zV+Z;eAA~yY8RxZYOw+6M=V}ko$%hlE%z>n%silg?&wyUgUp*1zR<-w){(nkJ+)Q-o`VDfC zD%843xNUOyDDarag$Xqc<}ps&U*VI#b5}%tDz^_NJ^_$&)nmPrV?y!71JqEv+vb~o zN!!WSsH(4`n5CK5yz+6wdD#>Qo~7oD+78GEn!fJU2m|155=+32wSwOdlwnc+6}sj(k7Lv{MY%dKDm<*yoY(Vsyd@kPxa~eH zceg7!aWnH@u<2ox?u=rPqkY~K&IPh>v^uGRLuzy^YKUX_^Y!sjB_>vLc$|lGMP*wHW zCsQ5%=Yz>sL$&M0@-jR;_)iNb(@8XGX?ns?)TxzTKnYquwdQQJ{&}KBDpa9~VuS^- z4lv$BD)uH0)-!tlewsh7w0wBhJxDe`T(O-rT}ReG4IfBNrDh$)NEjw`M~^&x?|n~Q z5%MFNMh1#9z8mU-(C-7Lbh~z;`?RC=;4$N#w>O!qSy!hlP#RaVWtIwgZ*QNa6mPYF zj1_j*kEO#VFYSFm6Xv=_OPep$si_RJ(M0}ShaPKz@F>$l^|WP4rEH1mj@o+A;Evq7 zz*S8yOYh$H@N_A#{`9RZ64T$M$Nge>7Wu6)rqdi3BXN{xwH)*%@mF6#e-N1cgl#0U-pWJ4Dxa2lkl#e#Fza~%C-uOVA1825&dJL4z;ytPMHFX%&WqWN%Mi8{H zU%gQr{z@Z16Ng`yz`dauSfEi1e<+V$QTIf63PwZ<>%6g}_UX#=j*M>0L8hz*dErVs zYxugp4k`R3#~xweTx|3nVVF9(Slt51#&q0%h6kSPVm_$8C11G+noMSVGOuy|DUG)x z*`<`9K0ld4U9`gg5HLG+)J3SpffMF39&@p4UV04dHhYuuD989l5o(R|7ro@Jewr06 zocg8p_Nx%gDCWjfk^ywY#U^SBAXg@FJ)b@XH3G`S=C-xE`F^3v=nR0Ea?g>*9FG_0 z=-(kRei{vUYvrI`V>&^N&yppq zIggJV`Ax`NO}v!em)vpPK7qtu@#4>o*((j;j`xgQdhdVVLbBV&T-}6Ai@|e{qX=l- zFoB`3jm#IL{oS%U%2_CHs-behVahXsa!4PyM0?WwR8jx;ng{D}#q+6C!xP50{q%Sv zV7tUYnB{@l4<&XJ@*}RkQIRXn2HlfNV-Hyo%%|8MbjBM)SqlIF)e4Qmrue`U%K!BeC!u| ztnSST8NTUPgyGmD@4IT6E$ap}u6OucOx-0K(0Yv-sy5Qs723W~&JUB0RGU7&Uo6J& z5b_DdjRUD59g`V0XxNp`>4gq;P=}JpAV~8=k3{#_BN>uD{!pVZ^~RV&j3rww_aXUW z7gi5drzzTXJwFpoWN^JAl$&5%8^8aC$42~b-y?Ob?2g=Rkpg2^eSFG?_z}HJ(YSvf zrU*C^{=?ENmJgD5VVCSr6()VaDN1*N!eM2DlcrYl7EQ&__#Z8STwmJwfzYK`v9vG! z?R0)OEHa9Wu|stEDN%@AaFAgo*NWRjNHMs)#PjKq&N{Z|tdX7?4Uzt_E~sm?zK>Ik zzKsqA6?Hf;B);Ips`bH=z@vIL%-Jn!s$|a-#-o{hVA;@XcK|>6`GDh|qP*g?5LQjJhk>*i zOrx{}BSCT#*bt|E)o`PBEM?V$qb`OtNIF4DeElO+xo)-K7)UykTDMEiVwk^sa<-=M zL3FkJC9#HHTI>{SBd^ZbH(JAvA;3ctPiuRu+ex(ec6Y2*H-0L+V>_^<^>#a14f>>1 zsR;$N55+AA(LL8@bIVs%q_cBx!hKi)u_29RE?al4|IQgIA08T2X^6P( z&ajkXiP4b#lNkM~jLcM_;t&vFVbpCNRJeL>ur4l~THGP|2Ln>gR3YXAHVb8fo+@+g zxH{ZwF6>w$pd}MsSOV$$tx7LK^gdBC1ZgVq2A`K=mZE!F&r90QzQ0pH4QC!E?GQ7y zyAeWMebg;I9v{AFF)i2KAtUKYQ+D_$M#uAX2nfTX^VMGyh77A)R@CoV8wM7qB&fr2 zs@gmv)TS&RqgOSc;1-4D+ZH*mXq+X4t&cl}zR@R+lIMWg+teGe+N;ENxce2eC4ki&F-H$8^Kbz1-nQO(s0%vZP^xMq#gg~p@QEUS!uWSGlMiIuTr0~gmg!f|PKQPHGv%Q(7iJwD zD2DlwNtT~U_A_>PX$nTBZ#3rd!q++u-M|}RE2O85)+5gcFi;2O*&MOxL$R92u1y^O*K)TL`^{coZRZT*=ay6; zPBFfn1a?Zd0C3F(piQkJEJ2Nl5XiV3kM)@U6MweC8wa`i7shAwaxo)Zf3YLBL*?4O zM$a?%qY_H>X8QTc?|L%z37C4HTX_RIhOfZ|{jKN&ttj|%-@BiNuyZ5$;s3~5AuI0J z!oO&}KCNaemuXG&W20JCVrkJ8I*Cu*YT8+V#3z;Oz`0~dPO}fWffR8r35$3o$QP%5 zZudY_2L1fkB9q^A$YV40Dnx&^H9-^}^XRQwi>R7zyh4O0*_f~FJ6o~^Jn~%K0Rv?{ zobewP#>AIJ<5GbITP{l;Pg$~eaeemb^&U)tp9BnDwtMUeLvAi9!{HCin8;Y2!c*8Oo|<=jQi z7^|Qx82rd+>K!tR*ckj;Wjpen-WHf1A8U2Dt&+7YCF_w}cjF*K@%49w6eb@gGA{)2 zj@#Ke|B@6^huL-#_lq$CJ4kr!RlWudAu?xmD)hrq3lL9}kSCQK@PHZRglvwmRqW4~ z&8VqZJ+g^mIlh%%27#xVanWnv{^I|oNHlXYgU`u4V#2_DJXKa&x@e6)VI`;8FW=gr z7ARRNVENx)KJowh<;yAhuTnlUoAj2&^G~?WY$dNR+n+VY+vuZ7?Cd1Iv7hol8}0on z5NIKtP=5SJ_cvh=)4Mh?Fj)67+{s$ODu+ltnSJWd22nWgG1%;{3SOwZuX^5R-Sqa? z>x%^w(!4_~e@oE{y}@M0(x8-PCFgZ0eo#RG1_jh~I)eP-k5ZOQe^|V- z^29vgyj4!1?!46wmm|P=^ zQ%5*RI}KzEbg7CQb|pmGBy5Y(lvL%Z?{dz$UoTHL5I&=c@kirM}cJ${&T?7_r$SZz6>Z{Gy%T**AEl0qsU^z{n zkfch_pIIF`xM)A4i9*Wokls1{$9fOomRWbXEzN-r1m3x)x0+2d0uP_Z4`@I%KW!BU z|It^ma>QDSiajG{-PU>&!9A` zsh;k39*nXxC45-4SXKA7$~QnW#+*&1+9ZTPmfQXLw=7uK#x+9qfI+9VM2E(;X?58?@miz&`N;x-Em`? zdv}3*wFyWR)z7UY4XkkbfJ2Z-zG+_Y^%YHoZHBHP1v21BQ?6~nnS~0axE?laju}XF|yn#pBFs8e+Hs?t8Mn+t{h16XxZw?5+&Ta!)9zl&v~$)NyAMcWDz6+L~U<@zLT2C@ZJCP zJj4o+NH{p?rH8VTF7;%73rqrnX5x90BOcu*q`0;Q=NJ{8kelUP4FK8w*iA-*OTS?a zY)jc}aAKs{T;@lqqAj7k#SVSmlVD#Pb~@{VRqb!?t*##Dq1so8 ze}f7~L4i-@z;vZVK)=J8cm!Q2eN~1_FH5@>_wmTs9i8XkT`Mb!dA#!gt{1*k2hK zi5yQ;muV5(iKw$(Tma;OOnf#OZ9^ng;w?U1|0A76PahO58b$5N6Aanlipua3D~`9wA<5r!tA#fBP6g)P{VWV)DU<;mz@SkE?xppfb}@uNV&(j0wqTN z&}yRnxI3BF!jLptF%TWTlj#M`u-88#m3GYc#o_DW7vDG1saJW9mo-Q}WU(08 z{D=YK{?@Pq&nGh(((Ck74QjnKL<7MJ3kEh*naw@tetUOLN}4M__VvENE%Z^Yy@wYl=ciU3nYwPto^$yWhYF z7RSCUzEuxd`+9P_lP=>wwq2QU=7+oAm(`A&igzbzC)xx!Q%21I=8+~D^Cu9oac?UV z{`0IWE{4Xd1D7Jvg&Mb(MuXIqB~(ME?n4^Urwfq?vLkdSv2KZ-OtR_+Bi$3%$()x$Fmvh#^&e#as>!S%kgCB8=w5}y6M2~)*#k(6k#3Dmrl?M7+28bCs zTOe6Y^|m|nJZJ8-xkcY(-Ra`5Hs79zs=-0upR$*3Rt#OCG=jaj?h<9fp#*6`!PRww z@tk@Z0YoQDSJ;xT-pdYCRzG-E@MGXheW!|z@@QDLt8EUxVZZI>8WbiS#$!l3kk(G) zxYJ}m?&|{trOiV#D{jvz7DLrm6O?+ThS{y3E%HpQ#V3B^(3*7v7h_-|ylTx8E%a=E z*)hRZJ>7i&M7uYDNxh{%%z53p>i_vKo{c@+1&<9?(c*^NjeWVt6c+Pw_~~v$4z;7t zMjF{R{cG9Mt`HpF7i>Z5x9-9^C!Ndze3x{OE z)T&|5TQ`~KQvYdMVhHSb)i68b;(@k8euhEnGkR6B;MaNTgiospL)Mf#lzQ?|t`9!) zHYs2VDw|kV>y|MR*34z6s{=Req{1&lw$=aq_rVv2_YW;1OdFh;IWmDK=u$81G*9(F zXjXfIXqAZ)IS_r@LYBcmC^N@2G93MFE0(Lh65%KUyj@G08q$UZCh+9&Y(|4;(-YRe zmVnb|54DPI5Z_mT_=n_PiIU`_%qKEOHX3dbMH8K^wbFryq@qS@xK~ltfx(-S$dx)0 zX~gH4+HCX3C1b;IN^wQ`An!aa|f zya!aIRpv_csuMNx;`hPCA8Eswn%v--%%1rcVH=Orh7z%GlqYnFKfA8k<^J8izY_R^26Zb}$*n(t?9 z83jgMn@{(Lel37hzO~Hnou0i;+x*WX_%mbn-;ww#oq^u>a)1Edn1<@b`1k0@<1~;1cbP|Gz8%dEzEac~VBEW*{HNXTSKqVXzzr znaqQy-^&sFoiSc>+<=%_fkF1cZ?eegt-#{z8Ghg(7EA5Nx7z4(xbvk)bsr^4s|=gf z2*POmrDJ6%$xNAnhf=dWaXTR#!r0>yFa;EnbGlb)RT+7!MPbQ5J=|!PyJ&Y*pNI0& z2{L4oZPlzp9g!;vH!zE@JTLdw!@)`{!*G0u1V+O-)Qi_q}oc`m=2?_W~671@y>BJJHzIPcXY2GlS&vPLk-s&69VXOA`@Me zH!&jDDK|i`;Fq@Nc5Qt8w7DFhae06J#ee9C(T{p^CixNhXkqJ-7Kf7W64`n21q+-) zCh?al!z4yzcC)QO*|H7#BIb9uG=(~S0dz_-_w-;zP`+0XFL~lH&6CK_m)=}sI&jpi zqDUHfdK|JJ7C3H7`G5Goa;uO4*qI!1bZ7uCo zZo@_z$vYDOC}cB@#;&Zn=FdWt61zagjhB;j3~Z!q#+9J^iI&%9ny^qVPx60gOLi@2 z=*h({EUU`N-sbn0l*WK$g;{;2p|f}@aIwNqnCaJp-L{LW?eaW#$wY3tm@&6Bz3fFD zs0Ro=h55TZxkL@oQ&PDyy&@JgWLNnBS!F?Y`EQ2R)UTpxr;vXs5(d3xwZ#6lfFPuR z`&SxlrFdIi{7mbw>UL;)1fm(BDs75Cc4VYPcY+C(DoP^h-gdkjz$A$hrZAIY!^e`X z#VkTX1GDvzpOJ#mB8f}9OI=<1G|sr*1i4$wV3ereBM}7 zv)99t{850$cWw+vB^sTB)M{p?dAb0*ElbdHL844_uCb8=-tN_DyB-KzYrNI|eCTBR zy0b=c`Yq?++_}3l4>byy=(8tYJ>Dr=~hUCk2&(Y#;XW zVitLqKP`qdAz<8AE)msJ0`?Q9`6uML|48Ir`EjlG^PskCU!Y@3G?!{z-Kc0mwzH3+ z3DQ*Pqo8{lC}J2|9m*(+?l)&=VC65M$uX4|`@sJnPTRni&4|PjP=;D7oT`$q>>ndBQlW768j zU2i>2T2?0HdV8q4qfbv7k`NU1$i*TeWc72q_1~BZo^0?;n9{si@HR8dQ(lD7aMJ|{ zk_~-kz7n_lPgr=x%X4EW(?33e21k&03P*K^mX}~1^!$Asf95s^=Aa3`pt9~TP9bD>H_#6 ze10%~bQi;GUNSl@d|VIztubrcC=usibv+f8=|ulyEG%+nOFg##sel*eCXZ5hR&j4J z!+`nS!$3QqnFpiHpIzJ_cdVUwFcBc3ieCMdnl?EMQ~Kx;@2@i9_Q?J9{n~7He7C$9 zVJt1Y$de*<163qw^L1oxwv1UP)5io$k<@hdK1nG}vkg;BJg&rjhlA9xq3Txt*zF40 zA&gKj#JS;_EIG65B1WCGF1diI*Vp#K+6wMQhuPb=t2lO~>0QomHp)xyqxOUc5IfB! zo-ALy+t_``v}B!(TcD;2MJD(-M${rp60kE_W#}~52!3^6YV>y@2WKio*e6h4a#dfo3MYU`yNnQ4!|@w1Je-|Dp2Eg3rjVZg6uA^RAb=8boEMFzt_r1J>; z`(KdkolQm7KmFJ|d~c>5Bkmh7%(m$-4&zi$C%a(nK+dnnAYmX&3v>S8!NI1p3nwAB zB1jPHha)sF*NF5R)*lbkwNgGU#ddrLv&H=FbcUKW>51`oBuzZq(xv+UY9+luJ3#j7 ztEO-%RpJ&>)e2XMh3i`L{XvOFW)wS)iq!A}ruJcCJ#ifh0De#|`_#Hx`-v(Yag`EQZgXRWxK51IIs&je!Xz1T2ls1==|a}HoF5=T^nZ# z`4iN8(?;1e%!9uq;D)CCNq2d>yV^NDGuv6EbB&3cv!WV&qC@sY%p^EGLz2o&d%v5A zucQto>`@2cjJ4z3MfrN?{>vWlXzP1+Ec^#yBdNFHh!{vv6m_wl;|a2B>V15M!Oia? zX3S5YOok2KKJqWnL>KP^87ZxHr14*Wb>2Hkim42iXw2v`+&wXYWL>qifx-koY7-^F z*4PHyHK>Zb=hX0>m7aC<|EGNvqKWLbQ^C-CLf!pn4iRZ=2cdxhl29s`xdaI+Z`j zPO*I8DY@-s#2ND9%-^K?Ttq|t<{r6i=(mv%ZwHR2vwNF*^K0-Q`|H)ZT5URwL{fHr zjy)Hr>W?!lg&hGdBMdLMUMwq$Eu4yXGx5JpC>YZyQPwQ=`{fn}Kw%(-8e7;bjaxeh ziBXG%^x;qCR=rV62|k>n|Fjx4BE|H`7h^i1(090(6yqNW3O>Jv;e5C0s%b>N7k1!3 zBKhRMY0#mOU$;XUeEug$Ep~xRbGn(z9vMr zqc@9tRfT0YDGdWgal<)iaC_;`Xjl~i@tc8Z&im&oHxQ7_VUX6xqc3E&iM)Grcp>i- zoU{qgf>O8_I#GND#Wa}!#LP5LG>mpw$yG+EwJRvsEI98|sT_DR3xUn42T z2FVCOjQEjk+Wj=ot>v&1tP!^BHD&$tN*La)6Al_+G2sm`)ZwtiE^{1}8qeW16|QBs zulq*^DSS@2FVzdruDL_c^InLsJhskWjyXnlMGHn?XF0$or}R+|qs3fHDyk<#fVQ+_k3lP=Jg#xAH=p24 zPz#Nt$)NmmSMBswd$u)en`-7)xloybJrG<=wY#13ZWnUQ?i-nL~-=F2#{R#!r zP?m`wnUfJC{ka|NDoC)%r9+1R0n$9J+#pgDKHI(xbEqmJ&SiMT6VLDNIoAvDhNkMjb0DdbOBH z&3HvOS&A_R#r{E=t+AlREJAKhIsdLPm_XV$>7xnJYxa0HbYe};I$hMs58y(mFCJQ>NH(%h=A`u5WVHlaa)Ygptt{YL8PF&z(nMzRNg3Uy4rdR%tujxG@l{ zYgmF@vAX>SpP3P64sDg76|al?PR&Ng{e+*KAeS5I&vi$nC{;j(TFY4dJ~ep`H*|nw zxxgW8jP}4+_8kLG$@h`plA84I0?QPpXNWP+$YkpiwhtP&iaZ*92Pb)!#dy>t)$D&T zx>{du=yNLAIC1}|55j0Mu<; zzo8@!4}F0z{`TY>X{E*8b%;GE5H>pPey&;2tU#G~X&ej%-P83*NGRt{V&cmHX@`*D_cOLJO+3n(BqtC~*LkYm6Oug*pH$wXmwt3H>TbhoW2W*#IwC zLrQU+bt6b8a<)2`-aCa_b1~ko0HcFKi6GSLZrHV> zC)aL-lbY|K29_TT~g9d*-KY+kio2=*G1Gv6xYd z*W>M3;?0=6HO+4nty7i@I`u%uus3J{i-d#Znz4fY0jZVNIUpHQ1S1}1igt#eQ;_T!rkniuPI*q5 z{0oB}S?)@I!{?np9J+5i8xpF6haOnCHuU}qAN0vbCX%x9XqZ!p%eXUaXdzx&K-f4b zlNr@SQ!`aaFFhLF;C3MXls6Wz8)~;DBLRLR7>gSDvqcS#dRQ}l5epMC9^$lfa%vmR zfX}uS0t#fIHTB&4HFdKKSIieuJ=tIK;?tLM-X>eaRr4~!Ta&(*$^s>$ z7|USa=@&MVcAa(%*>+6iGG9AwBbl3{@tjX)S=0c&(>|=gG`JK9TbN_etii%h(UVn8L=NYl+f1ar%H;oFiut zhKLEhWng95cgqd(nD+b~ZIaWM_D_)~+S|RfvXRfR?_dbDDf%ko!%5=%TKX@bx`S2X zlOk|ITbB+}*S}--{##4W1NzmwqJbs^9NzS)gpYnk2$5lhz`)v$JHS^7fA@{62z!RC z<0nT$ugT;lmqT;qC-87-AWAMNYJ11*7hV56EW-N_S}Z@Q(<_MOH@1sV=I^>mpc=>y zxanAG@dY1~7ujc_c554kYCt0Sx)SzhyIZHJ(;ZNMM}Vmh4MWqo1JPP>nd)9JK7A*3 z=Wkv#XUCMaTOx^L-qFmo^o4Y77?oM6T1+#+Og0IT{cvsH24Qz=O zfy-Qs;tlg^YT3V59a1zE{ux4kOF7(LXuHtCrHovY^<0@{kF`_VGX(asgA@Qoo?Yi= zT1}Z981I05s9e7=JB&_uhG1RsC6xsI_ex6r8M9L7sPV9@+Gf`KNa<`hYl*Z7#d*63 z$;?6iyM}x#1QLP_8A@XR7=XjD4LDrCng=AGt26bLs}(CLx`C&*j2- zyD$nR<)ba>bY^$gk_-oC)51aC6PHeN#n{z5$LG2F{8d{-=tiyKTjf4$j9e8Xw|#CGS;kBnJy|;s}+5lu(C=1sc0& z%BRNqpEP>o^~P^Ty5KBD7*ML|=;vFB8(etDCP0VadSWVsbWyJ%`5cZ1$mg*Sp>nV>)gm1cmi=#Ge} z>R`&f87@1#5*aHM!~IceCKFmB!aWfi(f2Um;h6~?5fPx9PXn?bjM<%phC359MzZ2@ zj98*EH;x0VnZksW=9xA;XHu&Zgw<4;P)Fg3&~wxC8{}>e$KK33?>on|?;wbdJwocW zzB=k)H*st%W7?-9r>Awjp8U|2oe^S=^ke?cKM{#VDaRH2)lbQGJKh^@m|=B5ur+Kt#hD*FB3xG1H09)i%-? z#UT7gv)#`m#&BokqC#6(?LG|*dJv7E9(ES}s50G~kHMf+jMU`+>gb2WkzrsOyWy~| z*YRHs<(QjJ8#G;8?dApd1z)@t?#$aho;s0%pxQieO=|6HYxD%%qA)Jks>#ef;DFOS z-`_cjZdoHgdgT;4a9Zqy4-ixlYBBn04-u{W0FYM<4s})paa!`f>|sd_6BTBQ?4ve)qy> zDsdZE#t6J^s}RHcRM%<{dYB{LU%Fg#3mn4t-8b0TAfo-IibFX@U1v*Ygl-|k+WF;T zx&iZz2MHOjDJK%`X*p6`ExkT2YL|wqSsiw*HIeZcqZ?v6Bs1;a4>KL%QogTEqj1D%Op>+afWjC+_PKw$m%n+m4<@5P*r(jW>xcW62o z{+)yHsz)hRj3bnpmB~Dm{F|X($wkNCXi|Af3IE^5DigKgq=zAVg^oL;8DDKjPh&f9 z1{g80Rvc}|lh-%R|9-YKUK1y2aom=F$l>OyOpM1(B+mP|jR7r2<1rHlBN4I^l)lq6 z9rcH~QUa!GP`a)Q)VYcUsL3Nh>7FjrgB?^G3ad+fc8&X%hg7L0frb)|F5e6_nExJZ zhv`YDP`}sNZe{*?jKBLs06fzkV-#ty#aqq)Es0R$F_m~w71w`|^)or@o?Co9B|WPP^Z_V~ZZvNn&vjtLb644{%X=U)2;a$yR1(I$%`XmTJ+_R(^SkV*ls~Va#U!s_Tb*DVDo;e>YXswEN_)9QFuUetneN zKgq{1>IovyW!iw{ZXm=;!oA)IE}2KD3L#i@nBKGq7r*ZDQqNRfol$zX8>s%8xX_=G zMo2jRh9bfd*b4P z5U~@;+TLR=6Z`(ZO9%|(9h@ShDz*19;63<(fSdH5uOd^sGJ)|GlEro)LI-1}duJJs z)>%ca2?|n!trm?7K&MP2zZP05#2{oySp20z<03zZZ5Qx8ASmb#wHVN{){;i32Kgz> zVU1Z>u^gJ`n_)j8`NnY&E8YT!RovG(P{+$SvWNqs;F_%i=ibwDs( z)t098R2E4~FDCoKb>&oUh(ij*%!7D+m(6m5-dnBHwXc6d`Wzhc3L?>8t>P#d9c}wF zLa9H#0c19By++MZzlJz8BjKvA-|ZKn!dY)v%7GxTt(|pl9U7r)2ly5CZblFpz@MeS zO6fPUlSD-*Qu$U0O5EaYG_<*N!UvQG-oGv^QZ9k0H8w{@r==`WC!`u3g$=;&d3O&5 zof=-<;r745A*B`JVD%(*ZjZ+DwBlEI^5s`}{W$^h*a4zM5o*TLNnAJ2ZYCvJGdynw z^5Kruduvp|@=Y2Hs$D8amVr8mdlD+R9GDSd+tN%Lxez&Oh)&@rwDhF8htE5XtVrzu zU}i~AUC=}Q4gP1d6JAQsW$t3Kqe?nqueiUyAT$v{oj^l-2NhGf!v2&x3Rap* zpDnx?{WGcYL2}N{m{!IuJkQBJzpEWZhVqnyefqo+(xs$`YkT-l+7kyHWGc zqY|k1wQ)eM>Bs>30bsoNls%uVv8?+ljBI>RhC!}B9J;gvKmAZD_6C+GDAbXf@v~GS zDE)ljMDEu<(2){{xiQs;?Mq-0p!%*ss*Kj|l$9|jOmYvxWcSKbRnf(kTxqZ5hg?*!&Lx2(WKlCD?QtmrmR7soT&Gb{ZR8Y+2$u zzc%QefotkL=QuwrJk2Q*5{8`V|IJjsi z7rz{YmJlT^FOHLrQ^{#0%}@O3@|WtW9vj8jIHNX)y@CVN0@qrgaw5= zx#An95-WXbMb9&Y+FW+MG4N+H3Pb0e5ixL)J=b1ERA~N`fZ*tJuyO0`Qc|*9T<{+P z!236pX28Ao&!dgat%q=?Ww%{+vVqpZ=576@ys_=~n!()L=p%#@GK zYzz4MH@?s>>0Pl}wBzL2Hn_>bXI!!hE-WGr{gU=Yk9hg@f2`ui{gxK;p%3^7js?IOzK;J9eg16#MOqtBVG-XwaGQ;4cyxQ*jT?db zFOOk!2S{fsB^-Dbt?klr@nNj*_+RM>Y0N^O%3ZQY=DaKohGl#Ol@6@=tuu`fz)!KS ziBhsRPOQVf)V+EBwt$#UUJv;HV7!Ar$e@ps3=t14-3XBIIEO+%`aTN)SRM~Vf=;QN z1~PSfRMSN!wdzrUdcFDV4jguZo7b<{S(QhN0-SZ&eNjEJRS+GWfwvil(X;;xa!?W( zqA9CTi5W+|ZSY(5W|A;o@xl^4BEt-hm~80wS~m&=Q=K`G{b~hbx{a~MUxu{#A?BIK0mWgGHM-AXLE_c;x?PZb zGI~rj*C0|A?&6V@tLT52T(i&5v}#^46gz=_bv!RwE!CiklKT7Gg=N!CM1&7?vIP0{ zO+=v|0_fRKR3rQ>13mNb;Z_l-#~J6CYBoS_iS3qV4W z$T`4NB<6!qhgRZsL(@TI~QX-+8w-&l*2g&%m^ z7SxZVd`Op7`Ega;H+x=fZfW(8qG`wQCBK8}CX*()-M-a=MSU@qG%t?5Bfgto$)N$} zrM^x*r_=8oeXhZAjrP&sT#;-CvXVtsy?3yn&G3HyjCns#OrWH^Z{^{X_u(Kdt0vp< z8?{gO%7NNX3$Kp`ft|XVdM-0~Trgjo6?>w^+B;FMJMT8d|L%+}7;;QK9^3ykyu@1A zT&A4WPF<(LxUDS}txh~n<%G^skly)|)NXro%`tqDyZS9@4pt z0u_J0V-AbKd*!gl)by1Xb`?`CD(*eCMPxYe1fcKLpp6B{kO`$mb=$5UwGc;px~Llj z|0^3*5w0jFzv?^oxuyZNLhYCnJ=@8xU$fOR=jpkQ1=7t^$Phsn1s3&W7V2p{Uvlon z))$AAr^|~|&?LW>y^D^^_V&>hFK0E<^}+&)ezDB2I98U-pA_u-%G26^85|1u*epA; zW2REVeTBZ?eQ)L5daz9dg@SkM)i|q`T~`*tOBTsY-+yk=eCp|IdKN4RxvYQL9l@O^ zXQb6nx@_US82!T*{;pcq=M(d#i5xpP2y^JTfeVyQ>wEiZm_#=6W;_+EEXH7%!F`wS zXlid;7vr3~g%M}0305zyXmT=f;|u3IyY&&ReNZG7J?Ir-I1t;PJe@mawssmrvFNqk zb(FTnMOvOuk9&#&>{Thy=8s+Kj$Y^FhN5gIOU4}E%Z(WWykEwn_!hNV>cf6BgGMV1 z{$Fj-+2}6`Ou<#9A2bf8#yL2QFhyxQF}upG|$4yuKChR#h-t7>?Q^?0I!z3 znV1@*^_fK3*ctsRdHo_8mR$R}#MNj2bKOFg>uSz|KP9I3u8(rC;JjHMyS(bN<(pFV zo(edHxJU*#)o<($YM5))4PfoxUw+q_F{t|L}_1w2=NkSdiON>f_FVK3n@4d{v-vVe;yFnTYHC~PSJDZr=#2yZfL|s#6+~H6 z*V$SGEuwOx7e-TylQ`Szw^V6rMNElLcXY;RP}OALT0dVr`ON}55H=#Oq7maLYWSP^ z`oRfoA$XebGfAvY9;{?$7NY07&Kz)s0mCENK#CFm3cOBgT0dUT+N`_%+*XGTYgZp9 zRCCnDh~7EUAg<~d70$cx1J{MMHvD6B6!>Vt%2roO?X@MEgYS2Dg&W}dqG5u7ZCiUL z$d~PB0CTLq3VPt|0=SuMY=eF)=h^HlP02WdH!p!FF@!DCwQiQ&XhqHmR|x*Z))O72 z&dlB1+mZ+xRv^27_EU2N==p$GnJ?CUd!7R+BJ^iq(8YJq=s`&-UX|x+*Vw$Df%jAy zh4Ys^O$WBV4^ePmGHcG3b0s{=r_bq5Qbrjg<$`v|r&*2}R1?xSY{@@-@FzQzzU9V`bndP%r zJJZ7a%2z|($FDjb-#OQ}JT5lPDmiMIR%HmeU;~7%Gce2rdoV-aDMCzeJb}jm41LUF zH(PShy;C#XFGvOl{NtoOB8_3KugWw$WPjk*Is*N`Z9lu#vjc-mB19i=KYBeE)Exy@ z$5j9EZIVDbZ+<)wsy##fDlfCESzwvqG$yz$`{3R`XQUJJ^js7Fn`g}3W z3Z%O){g)}Q7k9yp9V;(;PD01Qk~WbevtKJbt%4cyO?nk8-lJ43nO?wsCl{hnqKMh( z1T?Z4GJXbz4kfH+g+^DT!QH3%mfdZe+}FZg;gZS=qy}#bkZ4FK!x4Uy(E8jbYFN)& z3xDB`5*oTPSR^~lboZ>#KpO+=R$R^L{n1zTU)co`4A!qvIQVt#zkj;MDR|UX+Vo=a zfXOQkF_MKKy6>)SAwkcB`hN+GZZ^z!ZvP}IekAlEQwb`30C1FcXjBg0{09hJPksH+ z%TrZsT6$2g14JX###%otrJEh`gx7*ZyS~L#De`|2x_wyhGq4#?tP7se8WvugJi2~( zQygCC)F+}mA1(v87wJLuKK_1_S3c;unuMU#fPP2`O7a3RX+G=dG;>T@!0Cx%vA9>z z5)s#=0$e`stR+ykBe68O9#j23WW+g8;gt?LEtq>#X9RVFyoP#%@IKz?3HyhJ(s*$c zQ>Ne7u^tr)(NtbMyOw77EM^@`G{}tfASwT_?i{$znnCkB=$E2JZh@>+NV++nuk_Q+}6Cs4d^bMiAy< z-Ef?1U@w?|s^FRHbi;4)^W3T}>!116vW6{ZugBJs;~Ut^>?oDH;whhZH`(Fr-^e!@ zFq8~h=QNN6+>hx;(OHxOLNxh}j$NjD>!tQPb~WKGO0-ue-O&Ez<1c(pBnT1iqLHtK190Z!!P8*U+1&=K^83Bb#*M1rXdr6-=Hpx@d)q9xLcj zL_W6T?-Ub`RlBXw(mQB5_LO>!C9SfSH%*l_Jq~9RZaw?(9FH-gXWLe-f8jfg%3Efh zKP+pJ&&JxsY0N7{ms}+bYD8%ijuE-b-4J7iR%lJQ-2HV6cHnSl*^G&iOugSRMo@Z1 zmCk?$DM%4{uOUU)24NVzn4w~50_=*VHQ_~*Ud?pn7{3ttw+f9sU~T?Ef$ zYr_VdBPvoMj)yW?zT`lmG0?9!5ME?pj5H>^c;dBZC3M^^DJd~uXR{_f2P25&G@sM= z{O^e_IvNY>Kd{ga(9f_^GIcPe>wBDWt?3|{i&^tRXb~b!&-zl>whTR#Byjjc%Fy^` zHs$4x`Jn`ha5X+gsCZ?4uj^*kLAY)~q)u12jV!=nQ!*gN`K}=S?_U?3c10i43p)_A zu2=rvutEw%Ad2=A;T|@PQ+Nuih?{^~o9$%Gs zULXRY(SjAq28U9nGCsO(?b6t|4J*0Lfy*2Xm===gYF9mmw3M<8`# zGBWrbmj0DrBlBROxPU)TEnb-Hk;OL~AyJfjhUSBunBzxJNNam7^Yn{` z&T&MEj0-l$v`CEBiHWF|4ns<=z3nAkUKq&p(80!!RDVa3C{aY{r&-6JoKKQ>tqAvd zpXfLUSp#4nWh^2r!O2UIg$_mbto2|HZ;=K zz{?~sfQ@v)sb*P6E0M=nD~E2X^c_tTFyB*lzqgZ?yg%X7EYFD6#Yh%X`iBSEtaZQ$ zL)D&4xmFPg*I(@Bv`*6Vl-rD;@Q)6nc3ZVO{NBa#IrF6kc5SX}g-wZ^>33Z@S@77& zH~Gv9Jb)H-7fLX_P;q=Oa9m4`E9{xAo$LN=0yVb$# z6XG+Y=X#jydPFaIiv>Q;eg3a+$I4(a%NX@t1`pTltY1%p?GAbw-n`rEPFAZlZt>y8 z@(T^_I~RXdmOOPS_Fo$BkGz9+j>YZ+wy7cDaV8Y`_r<32OsJ4lrs}n&md(p(j@5K` z+AaEJ?=G1qnN6ld%lV7l?Wn@!Asx>L^tH8^IG9}y*zOlxBv0#pYqy4{^bvy{N(M-uC8&M+>>|k$%6Q0 zH&}tymE+`H%<@k$lw96@)t%V}utUnMB0|?OYm?NU1LW6Mo8*te+yf0iNL#x#C-(0`g zkB3pUTYbl26IrH|&84=`?U;gDw`VTvtcLU$Pb_^v!;NvNIqou6p? zR%eF;h!%J*tXaVE(TM-9Nh{)5O~kwKc53_A04UlGF~qldpNF>TE%VwvQx6!#E#g<; z?^nv*ewWCAbw3JgfZgpBX@T$+If354cSJr0{Vg}K6ZIG2eHXEj1cgZHJEed_cEO|# zD-*GmhjVI0zQ+*d(C}Ub+4~!t)%>Jr`@7ZrNw_I4VDyJ{lOfIJlS=kyJ;@R zlMB%=L%vsO;rN_s+P>!tbG^L=+gzoBQrv9~WcIrYph3WpU^%Djq*$NndB6+Gl>GAcuyN(@4f=cV4R78=*3S$}_%~xI@l$4|LFFm!u z>hEj>Q6UpwzQeH!K&vF6V0+$p^Z`c1i2;eSeGk8M<`adpZV)4(*_R()!yU{%X2kar z#A$>F=YQNpxksB)e`RQC>iTLwT;%vEc(I72`4|l%K0nQkR0~ zfnF~^{dgt$;oKgmOY3*D*8OE9oBVG75`hK$1t0}VBkisCZo`**M_~LnmEp{v*ib-s zWd42(o5$67VDKs;!|L_oZv-&o?<$?M8Xyozva0y1bN$EPOp!*j-8GTxdGIc(QNu1=m$LN)z^G&{m%apcsjos3E8$aP&;+6XnQj@@TdRw z?!zH9z0%wpLaTXIn6;*BNmT$&9t=nYbe*R|gO@76n;|$%lOcVwA5HOouoXzXdV;?l zqCWz;O`P2??@{R}yx7B-?*|LS4h)>Vvuz%qLStUZ_r5E?$< zyR(46ytOed>B1Hn_3O>Q#Im4e%N|`J*VkTIi*zAW^XN?b>VU7iq|!Ib1*u zN{{nWK_f;K&)tP@5pkg@r>|f1!f55*Tp#ZD{_23(TNJyRm{Iz;D%)blV?sTz4hrC< zUve7-tm0*)u7Ei8tP>ZNxG<2-4tV%rv2TcwmPZFP>Mggk--{9(mI94^f;pbvvR?BQ zMzwpnk%c2NWzqc0&hg7Y2p)k=TKa3lH{}B5La0JmHu*jFFFIzml=&86BExhFKJNsa z2Q&4QFnHlbk!lP&i}GN>_r!~JF6et+@+?Cl;8;Oy{}%&QbORjALP7`UC`*46FrX5o}QhIH7#~HB+M|6LI9Y&D5@%h$3Zyz53DuDi} z8f0-js_a_?B1`1{xDEnnp$KJ5|J^d|%?lUm)%wv0PC`afc70=nmkOMz~=b--)rh{+Y+Qc0wKMkJR~ZL4w!u#H4zQ@ZR#(FJD=Yf z#~BYJJB+ZV=~Eyy?&P`R?UeFU8^u;#Y1_@5Jr`+rYAxQ+!WFT(w9U|bImH)@#!vqVMSLzd-BXgqm~I?sZ=K^-=xcy!-O^ z1^D@-&T2BAqOrckxzNk5F{H{(kqa+`i~Ov zin?x+$V-J@3Qp_eq7cH;(#`l3pUWb)(AnOWKkrtg{qgcmXz)}*XbhDJ>z{Fh&|6#k z)%_oBz`vJ`O6j)l;w6VLGr8uw%Ldo}U;EWj$XPvikR41i=4p;b!x#KLoordF{=;e! zB2`Q%-)W!NGq)}i4}DFtXr#X@93(nVF**;w}yVJBDK`i1n6srF4Ge5X5LHFq1^qqUKrAOZ}-`72l6+#y9 z_H@a|v_0WIW0#ZO@06s*gWcp;1OJtvK%UQ%emjM0%<3FmuHLSO3d;LR2Bb~Bhccjv zJ+>22@!EZW7_W3KIlu`66ldEE47_&k=7MNCm=Rf0>{}n0D_;I-)YLj{>3~Ms&E#zMuHo>+G3icup`2d8wc=o3(ShpPOKy=sKN#1C(6iR1?z!`YNUKnA_E)R$yAZ@+S(QYc`kN>D)+D>f<#GKqctc5sP~f@Z#i|xq(&C@ zF5}C6BaU8#R73$`U6pmijwMEo9?fR8udG%U8+4sFkN1R+eH3o$D}yc0tv(Y$1e^On z_hW)LUxX}Hc$Y>Z(K6|xm6Z=ga1&mhIuB7MaetHYHs^!g3y2@)`ZA8cIm1bd!msNF z<&mm906kmxGS|=u4s21@DD{Tzwww}c z-8n*kdgrzEinjJ*{ygz1Qj(J%ci3dGx)y8+NG>UB`{;NhcV#E}9?3(!5nM47`jO}q zZJ!+^MP><1qL_uwZ=Gz1M&|0!PU?qSpPns7oI__qmyKYG$LmdHHLb>Q1IL*^y~A$9 z^Z7^k(2GWKjcL;h+AGbY7Pp!^SFN7HCb}-KB24uDnNx=xzo7J60ECXoYO?}Zj9`8* zPn7-{d3P`N59mHD3AC3UypJr;fT7oi6N_u3y|vC7iG9x zB0+;hN59(p8n(@`odJ8vDxaR`-gXKxcpPPEp7Z#4hlDUnF~U;4#FjXvvh1+bPiUxq ztQe5eZ&SEFQ4RB64^KOq7lVJr6~4D&uz%~snV@(~&UEFrLH}yQUV&}Pqu-nqtKAVr zHdCqf38g&KCP~h08QU zZ!s(3X{zTk2BV5OR=j1)-hHoKhvi7;k8gxeCf-*#O*9C}5WXTIG37Dojlw7=(Xpwl z(WhFg@dPX7RL9q-r|s5PnWy(rH7Jj>`|CxFBI1PO;#L z{1C*-&H^;OoF}CeE`dFmP3_sD>qvp$AGA0Aa8VzU_e*)G%cz>w!gYTxm@BF3fVA&KM$_-rocijJ)1xV=qW{+Ir>1cba?;VtvZz z<-{UH08!9aS1XC|%;?BvO}I%&Nz-7dJi2CZkm`}73G5N05Ebmr<(?78wdp<}+$7_+ z2l&3!pNR_H%?AaJ&R%e~5iEJLu{5JY~gdYYB8Gp_4L%#t_S~K+@1n*dVk>hek z>P(GQ9#T8Ob{grIm>T-zO5>3kO>C2P$ERd4p&_VmIJMeVysg0I$wd>(OWos{qFRPE zhE}%Wwm$Fw(DapIQT1Q6Go*A&r?gU1LxZ45N`ulJ5<^JGAl)F{9fH(LcegZ1!_eK` z_wc{>egYog%*o%`d#$zCx>DHZP1U1Jo0YdT&!v%OX#W3R0CJ4F!h7%a(}ngp7xv1= zbEShED$%*pS%}&#Yh`}a!a#Z^;L?DJv4en8Z# zHvZ+OinujzG-^CJieJ6ev51d(xr||Kq*p52)(Uk1(YK*@HBx>ZrW-|-FU;ZeTl=RX zSN<4m!~(A84plTCSrKkF!+(RX)oCdx-d7zp?r+60Wu^mAOf+D=y(n@Z`xk!r=Xpy0 zsBt~Put&C7h+^PqsS#yc*LktidEW3Gg>rml^d@Y{;r@+tuS^iKt&-KvVcl|f{TbJv z*HQ-obm!ruRR>!>Y2+x0o@@Oxc8=@I5KsX{zSWy$=?QFfE~?hbkVwQd&k2F2sj#ll zhNkL<4U1jjvf49r=c?MSo!Lssek;MpvcuYCNU*{vF%w?pwr$&rAe3Ms#KU1IJt{El z>9*`XGkW%tYfN&Lp?tOAla6$J9Qm^lAZW_J*AJE{>A(ENR{qdxkfrke9F^ri@`9D(&ZpOyTO zC3W{-soea`^1%Jj)YsRC=Wzr$r!x#yUj@3R)q3Lz&(k~Ho;;N})V=Qbr62O~jqi&8 zd}xrshLT}?x$oUThbBqU4GJH$i*zo_?e2-BdlNsd$(ZX{C8oLjDs4P-pjCLD`&bO- zM|0J8TT0)H9VP5LojD|d=QK%(ZN#qgh$)3fP@lKAf2?{B1QD}FPY0rNCC&U;bE5k=MJS%r)*$43XQk$ufu?w%V=K|$#1&OF`KAz*N#N2IWo_S?{P1W9?el3DmE8kwpWh?0ZQRlbuE05nx&e!V_teh^2 z5h71ha?UHIiBpBK`&dG8`d*Jq&hyWdg8I0OOJ$nGL?TUkOE%BKU;6C+%BMQ>H8x*% zpIiPog&TmGb+?ZuN87LjUCvG4_4tN}0Fpay3iY2~jY8;iX{YTfDJV4Gk6ho|5rLTE zlWv907yiEEr7~${zgF-{JQC1<-d*eO85TNLsT_2aag?|}Rh&1^&Ho!US;YqKRhcNK zuqS%kHTvUZl=pZT4-OF@jOSi|0%Ds>LGEmeZ{F0?S_5Ze@hkRY6!}pVfg(9!y%#W% z*-d)(cx|QWkz(3M%Kc%rp z=2|z9T@}-%l}Jf-N!0aT>`}Q3d(H1~n&u33VqD6Xgh}hZ&T#~?&|Q7o^MYAYeA>)$ zC|Xe?El)&>AJP}{*Wf(buysm#;{#Qu&(_G{KbW5v>htg8$GQ6u9S}Ah&B}+aE@>OW5xm2!cbQhN_V4-gThGmx zyGYIbhK_6aCp!^DUKo@$ZF#!t4QJ}sL*Jl~`FBXZL#G{mnL)$4<6$8wI`zA)7W8E7 z?ok#Y)8PS003+<%z5rx{;cBwG|45!ND=)ToI#m!88Qoc1Do{X*`bX+=ySG?c-}&Fw zhUAQ!+WB1T?w*PqZ=TmwE&#;-fTsDSmIpTuK!6^i*Y8vHyqqHd&`ykv8=pqDsojxL z*;x}Sf{$r1bj`jg!q#EgCld^k?$mQ6;sbGtWuQLuxmJ6lf*S{WX_t4D^#crl?pYEh zmp`W7-I1B0uUvaT^XOW~U3e2yQ@x5z^mu=*JT#Tab zO+Ng?dYwAL4h$6}HF`Hv@T_h5eZqv2(W-&Vz4*QsN7lmL@wnkqBQ>jXn4+XDko-TY zh5}9~i0A}UEf&1Ai&Tl1MeSQ~sH8~i?QF^^#3AdMmZxYeRfPvp|B}1AzuZrZA#9-N zP3Ef5gP<19g%(l3d9F@P-G+M6*pNg*Puvsdc)28c4h+@p`%GRXq{q?b(zR%+ih)0BIaxGO$KjP@DcfM%h-RZS* z>iyJ^W28`^r!n04O9vfvkBJkJJO0hvE60Zi z0V@E_2lVZr!nr;8*a;`DAYXny?0*L*m)c}vJsveDRBJSG>f9}@Wwq{h$s2|m2O_& zeNnnQ^{{Hey@j6Y@uWx-IbWcwYQ$s|*CfZbn(2Q_`cqeeVe@*Z_4#BRxs&+hk%jv5 zSq&D)P?fXQ0{Nt9;i^i&exZapwG^N%33Gb>@6HF|jAErLRrW5#)Rh>mhXr2Ow;)Eg zh8+`QA_EYnk`--lPkp{hcNFVR0y^BaVFJtji9PjjMZIjL8wj&YV{5!wLdXGKexXK! ze#Z*5i#iUlIZz7+#sYC(i@Lh@8aKJ3-#Cl{D|*=I_ybopJ+Q=u)y>_? z5j64eBG1ZWP67sJKYy|FqV|pU(sJMaHK{UfoxR$+`#7vy9TS0a+VoG_*QiaGS*0kr z7$9#(S<%ze^EB?|(~TYgaT{sj?}RRB$~$edI;2T_P~?o*#XIEwOGplYRg`&Vok&!w zo%Ws*D*74;$&|~LFLXNo!u*Pjtyt-m>sZ+Qrze36n_4xCkdTDmRwvjj=HqJ?X2Dk8 zJ2yuB<>PHK<=E49Pv>8DD7nanvX8ZkG$7+b{Mzc531LCEtC}T)PF-D5%JlT*RB2SL-R?`H(v6- z1zGL0mBy0q!`KC%ZD$(tjy()+pA;JW?Zp&Opq>S~%WiIwaWoB@Qiy4dS>g)XR?<8rsUSMWsy5?KPVN zIyW-^*Rzp7TQQdYK5xGM_9L11acz^^UA=J@GMzBH#m1WOwWbyM2o`zBxQxnnfb*}B zX=Wl+rpC}>)k6I*o$Lh<%Oh$3ZhJSy-=@@gt_MH%5i;{4_lA=aXF&Bx+d1?jiYF-Bd<<) z?3Y<5?_8Rk1`q``NJpPmx7D~V2|+;tEM}P|T7$d(KQ#b4i;KGJ;1RrjRdVFjYg7p#Qqb2-12+9j8{0tv{(3$<`~8$@4UtX}B3 zH>XJ!)bgUx>;R(uq-P(agj0nfM}3CQr|kqbOEsF7xL+EBjLwcIZ5%Z?3;v2G*^g#v z{V{CsmO^beS`QCgF-x^M048kfGQICu_7>NUC=K%GvFn+mSEOxRt`R)am7}ohLHc9V zCvZ^bW8$FMG%}52)i7UGtfJs=xCKChB0xrX86<%(%LTbBvZBaWL}sn@{FxP%D@h2O zSs<_O7b0zH4txD?LM`xKkF>OWj~z+eE+p$SGX3d!YjU&|t#DE?`8^wj;WO}XE^HEK z;DbgTIa&>IM6c4EzonVVv5zm*&RVy)WB=`zQAM~}b#+_&TJ}6qyY8GihrM7+_AB$x zO@{Z8I=O&*noa#O8K2oA(p<@m`C$u=Se0CP>#%LZQ?q;B!tqnxa6%3@93|!-puZ6( z_x@3li?cGsD4$|2oetH55Y>soY+Lc_c9&5z{|(gXzesb<{YFH%FxhtqO$R-AJR3cZ zE##&L3gmJH8o3ON2a%z+EteYP5U4?LCOBwF0LwO(PxE5#xbpGr#5UFrl|UKZ0oq;z zD(c|8^+5RR%~0qU9q(HMnQxmsaUqJBAm778zbrM|*h!?+3X)?2W&)5uhAk3z?T5tp zf`o=NP@!5ZhTY^?0q#2Y2LG~a7tkx0;UZUG@~zg^#ZO(WkN!T+9i@^M6q+xPS==XD z*q?6$A6RI^g>p(wnwo743-8WP09ksQ%H)~(@_oxf27jIX^?lK>moc$Z@ZBn@pY{|s z4xUR!%hAKV%Mml*%4b2Ly&w;qu;Guo1@jq^GrwoFd7{Q>WVQETl}v;7rY^?quRtC^ zh7h0U4HOuPIQ~gJSk+7R-C)xubgrj)0Iay2uL@H(?p^Q>O1FJsE_I!U5Ba=$js9hG8Tkj1maPe&FE_)>sx0Zu4YAI{yx)nu_4734TY(D!n z6QSMkA?w}zJBx+gug>S`QL`GUb_U=7)aIlEh}s(dp*r<3j@XmYHdbQ?yU25%d;7(c zbml_rXsc13o=+L8U1U1fiIva>Y{|MNJ`o3ayQSL#dZJI{bdRj5ZQC7%2MI#dHNNH) zHia3k!IZ8W042q#?cF|+z;b+<-0!dss7L^5q;WR$6WEkRBe(qKGnbRoCg%p%!fyTM z=PHI?v~-F%p6!Q3GPtW}T=H3BC+Cd;$!Byffcg6NiqG}LU_Dep-gisfH(ul00KltS zEC3PC7=wx>fWX;{Q|fJj`vS^-+3UEF)wsMAKv<_pmrd|zrc`h*{z9ZAuxXdL@SBOk z8W7)!`9A}Q-J#vKE;$JEbkSkk_)5s$k75!2yZmf$6G)F)seBeb54C9=pLg!E!I@Ke z))fw}1sph>+^#Dk=Gc(oJQ|UnD*!gLk(xQ{b6x-|4}METa(cu3(uMjCiujg zdLJ9|y^o!zGFJ-Mot5qCrT0hor}l@>eX+1|sso<>%=>u^^bD%jmqK;tN|Sm(a_kKz zklyUErM`43Z+SV|Qyu}-zL4!?yT6~ByC+Uk%P5N?^Sr(t2fO|+!QSm~_!pOB-FY2QoQ^P26Y-1w@T^$|pYOT(xS^J^W< z#(y?J;O}-@>H4ye%tYN`gXp-Jd0hi2VtGJ`Wh(M0+4!TfZ?P*%I37!;|CUx}cYT_F z(WxJ9ifT6U1gG%LK}$idAVI`CHq&kFyF|9CDSYSVwNyrs)80zitDN$eaSH%xb8NaO z*3OS>F1l}9#zOzK z*&zUFb&-2rj&Tt`iqbFDYD1&jn=D~q1dELntkh-&s>L{2ea87Ciln@PlKAY4;v49tWDA)FO@XN}7?fKID#NOY}!ns*^on`_aQUk->kPqO!467k;hK zdNwSnddmS6spd_m24iZxv;R%nE1Lh7YEU*fG-%R77r1fY)Ufunzez$U^+oU=v!!K< zVcPtpAdY;Dl=Eka@pJ7o)3^#K5>baW50E;62#i=kj!qSpmqb*=6*TBKvf(NUNI8#5 zVR9IQTJ+J|Y$y=CobEcC41}IE(`ge>pOCLBC##LerLLS@js9m#$LsIb7VsvC@g~CF zcp=@k976kE`)X12_Dv6f>XqzB&=+lpm#oYWuU)lRU}w|=T0{k60!<3O0`c$JuuN0+ ze#3O2#MH^dO~0*9K-Y@iOcMNI^?A9@Qa{>lAi){vpz=1|rOSfi*dA|m7xF-<5tVOw z7kUtFXYm>c3n)N+E>@T6+CV=EiA+LZO9u|Q%+4hFi(863YMOJG3S z1>A3Q?3~tgZ0jXu z0#E}DQ+XV4c>wMmlF2|LZaymZ>^WnW=*uOk0tU9x2`F zJg2ZNK)mC=)pap8ZSo5t& zt(dWS)AjYPwZ%`5;zQfjkKIvXs)bYhR@6G<-lO+zJ9V+{nR_eiCpQ*ii^lgoRdr$q zIuSDwGX$|*nQ6Bh!}n6XypYoBXS>#0Mv8M#DHqV3YnRFJoy?d3lVVtoV-lzVLymZ* z%uE#viaOR05?buf@La6OUJS0}-*?st9Tr%%7Ir`T~VeQm{x!51t^x=A9-LYygJ74!9y>(0JK&`D(p%_MU={ zk#ZPxVH3z!^Zg7Il42KK3@TLZ80ZuW=Crh1 zHUhH0E|3JN=NOI1F@MXbd`*j3#Jqpdk>vki&HS#z~6qk#S&PI{)c%{vUWQC+Ij0;#j zAQoZCI>o-?inD49XyFlJ&;7LQ*F63fZvtR2k~SJ_Vkz5^z%y6l^YL z`Pp6Z)RJs=&jhe010NUigmOEo+cK{?uAGOncuW~ZrMA592WIi6v&yFE69pp13_U)g z`~_D*pwz5BwF(W(e1n_=3K|gw{?!PbM-06iRCqfrQ}S0?KuWFjmy&(Eae`rh1 ztCtOaR8vhG;8XM^0X`CCOsywM0HW}57m{9(1#BrFrH=mPuM*U>?e!K|Uv^OAduzxLC1M&Q zqidRuyl)Z$7xFyhEWrY$M%9;$SN$jj!+eriH8h<>V&0J<4I>0 zTtM=teXa1#!CCbO0U|}Alyx7W#R4ISx%F)D?J8_sMRje~IEq&>k)sB48DFX60dvUqE`uu$;oleJzA(9*};LVz)#%WC)}l6{Ok@3bd-YnSAK) z=&T&bM8J4>SNu?Ay*jmO94XKXgiqMqAGi6?Do+KG@>wTc?oGd?F!_2+vJ#huFdj(< z3@xQK+xr8}kd{=uSO=3riph+=j~9f}^JT?59}z0IMzz-SzTI^0A=2E48~9%~n%f56 z{!DqMJ@bY4{b3^CqS(`E6clDVpJ?hN^%vzPRN;xx!X6Yf__QtdOiH#IjQ8e@!(R54 z87?6Cm4WPf0;6oTn~D0OJYY+c9Vmj|O)OGpQDDMJGaR zM}SJ`en|hxXuU70?oE{NM25#vLp40`b2P%(ao3?6aMiH5;>g$LY?}@b0CjQ8OQh1O zHa$+#i_uQ&%^Jq_e0!wB^#ih?2;kb6eE8oG`xOv~UGLa}<&_gC`aUIbr%B%Z=`USfS|p#(mFYMQ`5Y`B~d*Cne}#OHd+rL8si_`M2@?*fP`V?cu7 z1JH}H5xm_u1Tf%a;7MXC_>!5S1z3v_07(a5KJP-ffQ*)IAn++|Og#MtIDz;GtZ3k9 z_%&doh;6wDwEJkgNAx4W6X_3v7y7`L){%>!>@dsa|XK76=p4uegn&bNpGap<~~Jv>7ufwFa2@f3^nnajQ*}-HfzK zTF03@S>r%nRs7A;v3DY{o``fd0Z*_PFlQ=+A_)JKQi-=;%7E*OLMug0CU`4ySUria zLRT^>NR3Jq+T`Sb4=C339)(1)hmjCYqBXoMDwMPWq0(b)1dBy*(s@~eh#tMp?lF?qeWxRzbZH_7bEq*QQ>K|@#b6PLs74ygA#y)MijFK2KJ{F@QHMW1fgyH(`a z;@>7w!a%e3n}fimXI~i?r4w51~2f1X@dtx9k!J1N& z6p0Bg*}zOK%l!Bc9>!*Sj5QIRrT$%?P>1S>hJ%7a4yEXT>7%-zK~(O{c$?pFe>qEw zv8Dev)=P4+x(8z1A+GtxLJwUHAZDchgnkV*2SP_)Gd98sN^9p?>+f;A>Ze}gs|fXn z?2qu{@p>}hXEM~z8igUbF^Y<-6}z2upb`-4Iw>QX_o6}M#n36nXNHrxnVQ9b3*|TL zf)qRl?ZWEf2KjsAMkBkB+Y+siqZoAE_#1Yv$C^&W?MaG=|0;Uhkw?kt3cM==Z2Ov?OfxKKAPzNEJH~-v^gAp#j?axG=$+nI7>1w+3iwFk1gg); zBh`*&2CQRfW*BP12hy`JTY6)xs#j!4SVaGtQ+Ez{C#^pWj?m&%sOR>s_h3a!8JqU{ zCJkd{y@E^LfCNG4*+S+b>IiyG|uUbFEWia|q$?AMNR(3h66zA7?DonjSEbk=3&VfWKlNOHx za&Z(8SP{>NS-getmmfNkVe8}70uAFQ3M6r31gux{4nu4N6UmmiYy?JkM}Vy+6BJs# z#tP$o=o}XS8UO;0=93RbbxY0|>lp25(Sg2`en}%(@W2orCvocla30NF=Mom(`(D=8 z?+-D2PKKRPwB_IbRQ7tpC%Wp8o4z7aY{Y;PEc_DBwEScXlvQ^z5qO?(L~m%O$f=2T zlIK_>LIN?hvL_^lmM#5UR}&2KR`pl1eo790DVPJ`lOnBfgK<|kGYy#b)CnfP><}U1 z9-}uVDA<>M5GyqBd&u?Q*)%^yCR)C8RX7j|_}M~TKWUs!19_*tEOKOxgfJ;Pzm`aF z$BFy%TF6f)vtO7znApE<2qTrI#=yDM@HO9^rdn@6k?^1miXC!7Q3o?-CCfXVn2QI+ z#wc_PI&Ac375>7w#uH2KHs1)UMD>ESvb7?%BN43XV4ZSr)r0@G_l%lVfBUZ^DRIEA z;22+NGw_(7U}@>g_uACU_x3z{OQ^ z%w3vWx4nGWeGj2AG-#ZvQ(WD9^EKem!>(&L_TLAW$F2E~#xEQe>x}MQS@=+i4}9 z5HsogRihYtZuYwr0n#KUV5l959ws|OC$FWg1 z^6llOz7J*@dxbo~Hfg3$F`vr_BbAiC9?N+q79ZV7Bo5~gZR5=CGuQBY;JL>W1%2(24 zUkM&$W)bQ3jtuI1-!OoHw(aAn^=iRbgZY z%eU2*yBtXxR!GncHuZ$6_rB4Px{hu-?WEcpTOfS2L<~i|(UUP!QJY>;*|ea|gpT#} zOUd4TD4lGKZ+XL3N<|DEfr))XXE!rCZdiD053-#-2o)XiJvt={XShrn%?A<_#pN~B?}HRU?*vu*>vLt8!COplNjXQ(1R|GW#E@R3O?y(6hj9! zu72xN7j9!VTbkNLx0!I%$? zXR$bFg)!r9TF|}@2kQ7F1e;WZZ^~1(Xi6r#`GuA$2H1U&&-^Ub>=-*C9mD1RHZdU7 zZVsCZeL1dU>?^^bJeiOmtuB>aKl6d+YVeEC_H49@ZmGD5ZiKp;#@7#;zru_e6KrN} zDtSZ1hQ7=}0+JAhqYzp=u4{Kjwud=_Fhr%<@ZlCcpT7-OZVE40^%dMI)1X6@{zWRB z5<-;;xu*TdL+s2IGObmm9}Er)!Ygbs*i6)07({pf3;s%!=@Ia>A-1^7^uOk^o6sVk z&>{v@82|5M3A=_9gU-7gRt5DY+laA<>5+6(ZpQE$88IKjH==G`pZ51`-dNWVy{;J< zDl-o;<*N;4(5#cfVcDol1VV?aa@&$(RZ|*+jYEJQF$>r!6v%M7;y2UC9Evm&IH6w8 zNXy^YGNN0c7^yW zlV3_!gIXnbca|kd7gbIU#icop997uqk~-rZjrBUc*~aMA*60-a+ z9Fj)8r-hib+*E`9?KtLp;Q^7k0Y;%?x0p3rQoD)RjCeiHvo8;{)qVV<<%_XoDCf;1 zRl(GmGWr>HZo7fR=W7_Ir}e?1#@myWA{=70KL7bqkD`WHm%jDB=}c7H^iFa%SFVpLj$XtfpYlwhFq{7~(-LB8$KC-m z&;aW{ey0>7`n3DX;O6xHtsnzMngtOZ zOcjAsKk4Z%EwkqL7kwnx&{i!>HpZ`*njRj%H_?=3N<|_^D$ zST`V%SUvw8UaH*q+u+mgzY&$Rb3)?~<{M>q^=XZ{*0AMaY*Iyf!i?#+>SwBjB`PV{ z>Xe(hx5`NsIRPhRIv+^R{x}VMQP~h*%`Qb@jgNqaxb6P>!apF4f=-{_DTMN2TxApE ztH0iRJyBCwp_1K{I%nwn-AutEjHc!Pgm^qu(8Cb&dzelE%HBQ1SXD^}!Z z%xxborJOkzjH6#uiNL2S{cPgMkn`Ulwnky?XUa+0t!Ymh2@76_cI*{g7ox)t+Gxi= z%Ws$p#E%GFfuUUaz99ccO^ZTnr{Z<-4@kmQh{CCP2@_*>G?6U5H(TX`C4+G9ipjc> zhwl}}!a_q;%GN9sZ^{K@Sr4>Uv|6*No1zMB?K?Y@slAgjbDa)rW>DZ$TFo|(UW%Zg zBQDGet7fN+c(Tu6Cd@n#g&pd7XHmmK+BLP(b52qiW|Y~@ z`ebFF!6#E_H&FeU&j@G&BVnjxlX{8b*~0!0|6YA&5u=@|DB0O6t9J3zn{wMR>6%k z`l#-%|@$^`#cp_9CB%Gm&Abr`@)W!i%(7;Cy}3_GMDNGIfxiR1Jw?sVHs z-dYzKM;OzDap zipJZ>2)I_Z8X{;Nf(kj>d@tx%qJB`Ao6Wn)}L49eG(NN76CFZnO@f|-0k{LPLepAdb6R`^E*t#ebh3)$Fzd(2uDV~~!oj+*5cdKC&On2EUDhf2ItxSoSgz^uCpguVwQFyM}0&n>f84kgHQYOMv z#K?2OWmARUIS`ADH?lT4aAC-iuv2$Pn8ju;^owEc-;&EjpkJp3b((U|C+Fl^q;YW4 z>OsfLeC>r>u*=TOd(z0mRm``j{5JrC*0*>6Ed#H7*Tmj-GR5h38f_ixA;ahCR^qP| zDJ0P71soGE^eyDUd5Kq`bo|R@xy<8q2cSy|sGIY^gw!}3nfxiA`v9x2>`Aa=DZ_0m z8O{)^k26{+xqFp-e$6)UZzm%cExo+g1I69ml4J^Rz@U>vJAYO0;H=_-2g=Oz4Fww` z4TV?OjmY>@gc2yom*=8frh-sB>;nqdYeU>)0}wvu{2AuX?^o%N(AZ{tkV?`9%@;Dv zBLvP3RhppsUr%b?AnXlf{ZKMRMR;&4(l>fDllsfH`&Dm8EaSj;FS!iLf;ea$533!4#0N{})@?#tBv6=eoGzQYtg^3P=nOd1b=Sk`M6@4u5@ z-fHt5l$zVj*BffQx&Aiqg8NbV`xIquDxmP#G@H?MdoM(*Z{r3Ujj*zrB))S5=Kj-` zfGGIYqR<@yFR1j@OFX>WFYG^{_#yKFlEu}I!(7S|*^hGA80H_$y@pg5LVOXxbwdksZz!axp1rmeZEHDeyh~_tn$&W2egaPwKB<~G=o-KHp z+Vn0r1eE|3zwPftSXVD#IEJD`9~-TipDTO)Ph;jbV*0rj#zF{1Dv#Rb|_-DQNL>BSehg4iD zE2PdnO{YA?E&Ki~R6}(cs8p^r zQ+P%~^GrWLV=|ZbfS2o3=I>8=FW4*dm)}wmw7D23OiVDzCm7R3`OZRnvSEc)xoJUm zY+x*|pu)1ctfx99ik*|~Sc554@eKVHPV^Y4WQ4Fs@25R7B~^DjjXp+_AJ3CRnOx}Ep}?0VYfCvzHW+M2mR7N>0gm)Tv5xEij%;oA zU41GqTYsUS)^0v&bh#RBZFy?KOic{Ucb=0pD=@F^6FKYOq7lSTXUuUvyzsV;tVJtM5sHIo3t5)cRRsicj%F&65la~b1ANgmlAw_mE%J@F&*P=!GQei6fKovG z;T?dTu=?jHXmD0PclG{6zFfW5bnn30%}47`UNK+x81{(Eh_YTGYSCd{ShFz#x6@28 zbJ2%x1TD^?@(CL3>wpfg5giC7nZE>h&wt@}tThLS9xO+a#*4s&>iA=aC_L*gh2(bW zQHn3QFy5!{aO98yY(F~(rNHm_Ae?bT+Y32U_J7KaA_$Vk=EF4thJRlhC_&6M|FPOJ zi`XsJCI0wrYhE6aW|V6#6W$9K9gU{5L`dn^hUYMIbbqlTJBz*TV7uwzzwHR-$p5=O z0#bf^r=~Hf#QP>wRJM$@+)-EX0+%MvHnN^WeWC$P{O3hhN#{tqsPZaOGOkUCqE4O` zgKk!qbRGop@Q)y-H+U(y_po@J0AB%Q%jyv@tOPlu$ca@|Nh1l(<;N9bhOSFHeg6FJ zlK|&{y2-#jTxl&6m?kLQg+nTQ^Pqfqydx>^l;dE-!BP5&XBB)q_q#(h8B#v)f^*fx zrnLWSh4HCbaRoYKy>TcT{ulZuImJ!!Zc`{#Xv;%eZMaCYjp$EK-H84Ee?{s}2d^1P zXg{_Nn;^0>oiy6=g3g!G z7l5LYAqR*Zcq1^jEvy>(^mvad_FD`fL+wEXFn%c$Xcl0W^A_=4j-S64c2_YkVZs2$ zLQQ*szzy8buRp@mX$rD)amoYY*+9Fi_=d@O*MUm?cbnLKjGs2)LmcYZ`RU}k)j(|$ z(uLmKu-CPi0V8)e=Bt9;zY-n1jWN{y@_r{*r~ECs}thtTKTlg4Z@ zCRT5getuGfc8NdY|3UrM$~8=5_&4!gOyyh}Gevq>w-jqqe!n5?pxCeI2knnl26cu7 zjv7X`9a3ie4>=7kU4l6^o3bVQs3SgiZiE1s>>zR4Gv<8YZ&3XJN$HY9Q14x z#`bNVAwvnZYSCdRU?j|gkO&o%q}`^3@5ycacc6QkkPs*}WNjk+LF#@M(`|?m>sjNH zO)<3f_t=xA&%hunj1P8XLVnO@5nno2*YaaG%T7(u@sQ`XXumlr;^@Cdtkm(j0?Xvv zYOb!*rN3OY^S=(Q=bpSub7mY8v>Yf)bevVIdJ=G8I;0$olE`t{x2V2C9pZk5N=8|c z3hwA*N2C_3>?YiQ(tfUwk+9*R2}sikS`NOhYaeaqJPeQ2aKmmuY-ja-18Dp1@Q~}Fxq{a{Qvt=tWPJo zSPPg|1*+R$#?<{xO@lxNj($-Q!@6jq%7S%jB(0Plro&Z7KD<(Q&jqVr!&1NbhDZf; z^Zvx{B!BPz8bRsZ8OPViBc)f^Ml`-?Qku-m@5N)$pTxkZHc?xdg zgBtWF8xX=i8G?Cadd!F{^;SltW$|t$h!L{2p|KwCmvGUMn7*L-irG6@6=)w`j3bat zXyTirR!Sa*qiHPV*8Mm3(IY8b)lEpNFd%XK1Dn08;E(vF$zs7Dbpon=tZebRSnB8K zB2eYI={2Z_t3I!!(&uLEmlWL&^J#}gH~_N zsHl6>O5<%|!SE+QDW7M9)r({OFtJ>(`uNi*dK$%Iqb%Ro5;knBUYx4I5Xn-9n;;E- z$6|G7-6{N~UmkSvf^rP+mJmV)_v5>FQH^4g5ypzSpprXk>{10JV5E23?A>wSqsp0s z!QwP`13eq=e-Y^3m#M{$V2ScDX_xCr0ESH#IrVhcw4epy-E#3=7$iUM=Qa*$6}cp*!^dOdXQ0H@ep=%`A>3# za;RqMx3ri_my3)3)Ua^(YS*!FTkjSRXy{J&n4$le{>+~*E69*eP{sQ_-Bi^{EWfzd zeaI&YTh+$8uvOy?3^5BV7@UZIhUA%@YE)+m!laQQ{jx>Agh~a*_9bF)X_=fAhvl_7 z%fRB^<*0@7T1=zE)nvB|y!wu^XQA@51m;HvWRC^Q&lT!r^0y`{x4V@e%G)$=kf(Sy#v$R0jpxGbi#FCc& zZc^IR{Z4ll{^hCAsytZ`oEXqeo05kWf?xAS!K%ryJtMax=hF>0XwkXLZ>`un73%Qo6bH+`QHco%hxbaSPFVxY zg-r|T_mg8s;HW@{0DM%&_FWd1zkX)ou*@3&lP2k1t!wWUm4xEg08(2c5QZp=^i|%< z_I(6y#Ku+G5cgGg6^IegnIqsc7iCcWF;LE#F;=?6)5YZ(ui{l$r(WU!iDNWp#^exZ zLRPl zGkuCpA2&+PSzH@|3stn-=h0tY%1QgsMwMbngZ9cryGu6y0ESVxCr?Z&0NTM*$R7Ep z^+in*jNzID|H&wW56T;xm;x~cnmq+rr3Ju=_RY;c(|KetCx8>WE_HZ_mo(xldhOqW zs6_0FDv76XXJ!b9fhz(BCY*f}aEb&V>5Es9C4Nw-ex*JU1czpM=d+{IfNH%vL6G% zh+pwd0vKr-ccQ%%CV>4YCGn~nZJIb+pHuWZ30%6?SH!qPZGZwOGc%62R+AgRVMZsBieQDf7j({*0u~w~ zHL%ZRnYFkGa9Y}H$6^$C0Vs6lT3lr{AO!iV0?-6tkbIZqnCPDP=uH6z;#LGs4jc?^ zmRBjoSacI+chUf_0l7xN1W{O+F;>pC=>k%-#c7$LxaU?&fmd9LI_*B3klqQL4|&!e z-Q#WMl%K7=@-kbZni>JDDuJd-iGdJ%q2dAnOD&Lx5^wB3cF!r0&aQ+M@UzOnMv79TJ(KPA2-}45BpBywUN>t8l`iwM%nZUS%UgqICGSa@K9u+J7nF+OelS2wZ>2)|B6R`=wbD6BbkHgxg~6>-=k8Q$ zRaCoKQ>=2~a4`5dn-pN+Jea zYH_xR55T77^VQv#3?igjacequ6BkP1(TVA~=u+0HYY(le5r;*IbnVjBW}}2Gu-8dT z|Ek%SM1JQ2a0G5p9In>NwN72S*~C#l1^|{2NSZ5!&`v#ufdE)5FRg6;k=r3N!6{!UlpAEG${q5fe3YwEMDvCFzyRv}F!XpeYTdB^c$CRT#D6 z($v*VsmHu|HNZ0h)NHdCCNIWwjiTe`Enyzg{zRcCx*HhF|4w9GsPI4z-) z<2=R`X#Et(3E;pDfFr{2*bWhs?%7|4J#2V~*hxbrh$Z-`klqa_CM*fp5Zggq5%g>T zHtg_=N{jkWfeV2HaXzXwa92unR9QDmQ7HaSI>U)05Lm*jrOqM#IpXTfonlpCmtcV; z?bA`KblWSbO6@UI*=gaR%$t}wA#g^t8;^jIbR7f$lHxJ54uBSMKmazCHB(d!ZH^Z< zjYyd;GZ1w;Wmav+Rog{kdC+>}a zhiHY)OGn+$Kp#h(H3!*^n1!_+C0E1Cvi&@H}E#0S;Lg!KiGtsPiuMaESsgHBHE1^ zVhc_B-r9+ETQj%9v@3&JyTr4a;M9uyr@al#rj8Qp6X4wXO)R2n{dF9du;o)AH-Mu7 zvST=WlHooHW`ZaQet-Z8Vi-gq0GhCIp9XxuP*OJ+c)-3i4tUfD>K#~t)lQY_7%QY( zQi0-cJ5EOx^3f7%l!-Ou1}+h2J4u-(0G&FCZK|{?3Gvk&rAHb;??r&CT_@?-%${gf zb0%9tVlC2<9&+##FbQC)y|^`6aW!|E08Z_6J+sW5as}R0l7XSTp{Oi86aXA=!zO<7 zR!T7l@C=Gcf*bOn@`#&LO+Iw5bt>9asgreHdJWM88VNed6JnZf-KA@zuq+W2C9HPZ zT&t0IYO#eo*osRGu!TF%vK3Zb%qC2lZ6AO2vn|ti4V$Jte8x?ZVzNMAdu6OzAZA8~ zjy7)S=ig%A~+)*O2IUE5YLU-*c00Wz+0J0>H}BiG<3F+Kv*t*oJKL6Dq|OE`uC`NKZ9k2zu`fr@ zwV5cqVbu7T3NWb%6NIE|{2Nmsra+MtsG0bS&HZYy)eikYDi}J@digD^{c;;=b#C_} zV{RM4a|1XUoEf0oNx`=Uu7zJaCbDCC-reOW}=1zQwKer zb{gd8YMFCFgE4}vum=dDsA_>0#aUS=W!ND062Oosshxlg8wJ*&Gb2$719{jePqRpd zV?;t71s0M@n@U6e5*onU3ly;n@LZ+Grhfbd*d@O;02eioga-F^i7p!aIq4fvqaR&J zX{rTQxB;xteTU`hx>FYC?Tw=t1(!+I!>>#wT~19kMm z|GufRAwxUalxfl-QW~?T*I1Ws;#~DxJ3PMh5##m>GuJlq|ooaE9 z5Ra+eI!w}@JEC_*PXQ)@E9viWcq-eams4Rf=HSqlnzvyxLK~PpZL&=nH_N8am5@^+ z*Sb0md##e_G-2I3(azP#_)sE-cBkO6i760Mpac|{JK}SjIrv(u`$}hIX_CQ+H|z6f zZ0@@ISf`B+wyGYBmmrna9Ysz62W8Y5TAJLYeP4-IqzA)5E$)F3gh(zC6(T5so4I0b zbJPK6Q3Vl>26;p(h_S@$3E;)1Pm`EQ92tR{+UlNq2j@iIC9(ksu%xXIOwlnz+H7pqrFgw_$vyd zCZn{J;Sd&eT^bSb69n-NKn;Bw)_kFt0Oti!)&IBm=1-bkWuD*jX1=-am9_6xT1yhz zR|EnC5(1%y2D+J^wr8_{@DGk~gu@Y@e`6eu7`wycarZO~&2(d6glK_;FhUaA_r3PL zva%}oeR=))JU8D`A_+Q0t7xh6o~k$R``&x*Ip;q2EZ_62%~Yyhk!5;UBhOLgoFGcm za(FgeHxKv%(4)0gBlWg6!R2YGa%|$*8vWdV>z>J2hbj24UKozif#cCQy*)arGjZ{@ zo>+1nueBd<6}ME2O>- zKx!6?t*LRq3v`dGii6NEGp34Cq@T@>1;W5QIFrt(yoiRcMTBGRq&T17e}RO z%WdW+1Ay8NIW0x(E_HwaIOx_ib8Kp=Mt{d*T;FvzF7IuR-p;1D7!kl2&(+g`?;GN4 z=>x943d{nKG_*9vrga^0Pa6*VP#pf{>uqrydW{brjlrXD#x#n?9X*>+g*X+HC@q^h z@sG1WkS^m~&BD=Xf&-?W&Kw-08lY$v9UY^W&jVluH-$q&pZ5HuQbcur=z3mCrSE13 zrV|%E-`|b~f2Hs_ue%{AQ%Ql60v|2~<_6x1+VPig!!TJ$7AW44mOux~;MwQC8x_2a zx83l~R6Y1`Yrk+xTM@t!Sg2dGfCB3R3v30qdRje#cOsEwkM@5#3JQh$GT#l|3 z=QJ!}eT0=(wwpD+foD!@C+aYFa&u@(1zj013}^U5<%Ja%O-v9bGAQsLZ zWc4XF5G)_84XMEWgcROKP-7p~_QMN3%q(Di53xxtTxHb!CkHoxlHcO9dSfyhv z(hQ^wgeO{L<1fgmp_VZ7#D9zi9J@B}y7eD?{6$1vi=%i_KfNy&2M@u;$K48>-lm;5 zVc)ZZN|vvd|6p++iwr9RI0-56POR@3ITJXOn}xzv%$w7?h()}D*WN|EoC<_$yog1V zVu>{XNI^?Ra03@X0e%H>Rc&G*m0KG$2k!vb7+L`TE)9r@&_HHghZ7>G05Az689dTn!Hz(~YSQta&q1*;Z<3p{ zrhN_N7G|s~DQhx@xfyfD6qKU~EIeR7sP7-k(Gw!zb&S*hFd2{D< z^mMkxj!Uk_i!r<`%UjJzt`{-=8w(s$%Jn>iPovw}Q`isv!j~2Bxz|0)_mGdYo&mDa=76ysd zK71S(DHF*!$kmndXLsEeZFl`I(R}gk=kC8yzN`%3Km@IwJDM#>X)!2jBsW%-V#Lzg9(3otTr4 zVnGG4@%iN>jSdY(N2})O^N6k*V%_>}F@tW=z=^|g`tTbWgtK@3PQXz=b*Uq@L?_xQ zr>hmYt>vhP<1i7per}DY&RZ%uIOmPDWx*u{N(x*c6j(ryvSEymYYzVw zkd8#x=4iX}o|t8WS{pqPaqJbiIIJ?v?=!FbB=fWYP!`V|irJ$t!=c#^x8?-yRIn&^ z!C^(xV8D&L*1{~Uq#D;)-|%0xFgB29F0?-@12}RY6y>-uf(*M`n`2#dHl`5k%q^nX z!ooQZ*P*2Yoe;zvychI7P#o4P@f6ljSTZfP;*_tg9kXrx&Yp;^pwV#_DBH;4%{u+FJ0V_cZK``h7F1DP{-F)F&rT;>23p9a zvF%i!?l6jyoavn?Awx_k7uTEo(QLDfp@Yfw=3U8LE8`e?`i}%j)m2c91}dLpE8vc zC@JuJQ($oz-#7TbalH`aRPbrm*uNbW4%8SfzcUu#!Zc3&?-3kRA4e;z=e`%yFMO}q z^rz&Y7>`Oe44I@-MH>Aq8=tOQH6$Gs!mBiIzy9}b???H@6$2atK+eY~z-11dkmi^~ zk7gc=Dph?aM(~)6oqi+UgkAC*;pjAAevK!blpD?>HkrW-Z$ok^P&UR#jnX4oG|vCY zw*YJ;Xo1_4t1xf_p%^b1Py+}NQ#24(L5_$ork?!ThRCc1ZjshXqSV5=DuqPo)x6_; zY#wW?3g+WY002Rbet8soRrzRq-Uvr&1~9NVjjXc;@eL>?|Je~l26An(Er1+*4)@6o z(&*f$P`xO*H{=(zi1bjmDL$DUaJ8UMk+~S2A{0aS*4WtA0&oFj7;*t_P6eRRT0@T} z*-U^x6t7#l3CcNi8oYxd@ESQY5rY%0F*64d^<#sz*wEk!z!4_n=t!M9^m1}5ng!G)78FwfR8m6Fy)GK@XXaX9YQN#@sCf*=;$!~-%VXAFI61m5 z@#q?;>WFKRCRg)~=*sqPWPiF?N z;ppp9**=F5J?;Fb`dhRvhC(I}`IU)t`Ss!|e>use91#LDGkY+vW#lN+X{vt>Dx1>Nxf!{^}frq29#)exC`T~l>vj?9<;sCeZLFU@gw=Ej^)Kdj7{cX4W zd9G#i&wr1_2v8(5p^8OqA3$i`#nH5RPc&}4JQ{j7qkr>q%s%;-C@o_(OPQ)yG}%m}rMOL*gSrj^P>vG+az@Q;4xB8)Pj25|pXj{O0lI z*wE1td-~TzJ9;%u>fERp+=HBzJe670s55Lk^n=6G0MHpeI~|W7jCk|V={S1iNDQ7l z8Yhn*!km08+B~#(L@y;06U@p473(N)wgdEz#Khn_i0>d$&f*ZT}LE z{?=%)yo~^wbr%6>HnLVMZKe=i&Afbn%>DA;$NZ_+Q$(eua20M)^KJhoT5kR`m=OIR zYnuK;jb9PK5!_TEnlDI`Vwb=LMLp>c2Mz#>@EjB2JSbp+4_FeA!iR}aC?YVCuj0#9 z7sUxOCwoB%Z*oC-l`ouJxxD7&%XcTuyvC$TnZPUZIkEDcqZ7BDTYxEfet{S5wFEzI zMO8TF)66*^R7hBI)2 zw*^l2Zg&+~~VrH9S#?rw&1_g`W7tmf7ee zzEnf2Pej-Dx#c+F*C+LUc(xO#rsG#{)Z&3(yb{0u_2YmY#a8XnffwYp8+Sy{nvDe1 zY>D-JpPU$p!*9O?P;82gJFd!WzQ;urqElg+1+WJ_oEd1LIec>~0YOm?FWSuEJf!6S zXAbbS8j>817T(E%98Bn31wi8siC4U$q4#@`$lg!!^vx!mL+T;C0nd|FfeL+6CTTOj zTlvU|akR9ctghZx6_OWJ18sxm2rcH2Y{JbYo=OWGkr=R{VX5{}jIy=WwAz2Upjq5Lp=G*^=XuIx@ zVL%i>=soveKqpo<%Q8vpATHxZaa49!F)BD~_#A$0oDn;_@a59cg_1EG5Y5{lk`Ck) z(P*XA0EjS@f#DdG;D8kpv@8H9QbCuOQs~a08v}R|Or;4o$6W9cHM!@M8>AZe0_?mh zAixKho5Z0YcKwaA^+2tZ#JvkB?vBOm*cswd8OOtB0v4>F_Rmk@y%;f&;xB>JJSe1s zWs`bA?z(lLmq=O#Uqo770i)%Gq*v?r*OtKQ$Rye8YzBSl{1= z?$T_W9U|@%?ZIq|N0}vnW>Y*l*BqxOTVu~;CAPtR>g-107=XA42Woz%7DM>7*>`*{ zUO6})2TySiWeg7u;OGxI>|%ekclF2G^_y9M+R*pl83AJk)EvWVts85q&TgWu^V~J! zdlvt8HTHI~8R!-u;~^XI>aIQWv4Nr3)zT0hee5zj_+aUwhw{~smcPG(Bexo!x_rhD zL;)6p`CC)%MKtZW zDOxZ2bb6X!N!>~p76CMq_{W)h_B*kN%MkT*j1O7edR?@jZ_{%57gFbPCFOnG^Q;Kq zNVK`79q1y;z^>iCjCqw@L8pcQFe=u_hd6ax!wU)hf zni!2cK6!Iow&&`2_JubvIiE<7(}DF>Z{NBY8$J``BWI#_4bSY0V?=dg;@=5?1JMyY zQ4h`yF@tVSM`s_vRoUA4Ny|Ans|DcH7s5vRPv4HqqI2D5UV)wHAE2@Q>K~Z|CUoN* zcUEbV>$#Z;Z-WferR2CQ0>&06>;Y@Z(<|QRyZv3cy?H#*!>5D(PL?dYz{QLal=hKP zFO^yzN(!uQ3d|vTnj~0e?X{nf{d3GU>bXD13>#3afEzYl8SUGyjVj`% zs#a6DJNZC~HE99AH&eg*KV$xdAD}jZV?PF|jEUxpKNYQa{!h_-(e1^6{y+tPd^4{I z;N+|Z@g$ZL^9+_c3n0Q`qK2D-4$dsxi0SF6XhVlYkxg4`8z96$mkV%PX7an9jP-&n zpb>Z|9!iiR8UXkRe$td15L46oDf3aKtz?F_K41s=K)alh2S9TYkW}XIj0`4na<;VM znmLPe=J&hxh|rpCA6y(2Y*oC~QcJ5X8T?M>>Z-`$nHhtv2q&mfXLwBTf<1dDd)aJ{ z0XNU-CRn^}#x2tzIf-&F?f0?o_gkRTy)VkE`#c4X8|}f(ZOQS*V~nOCngD< zFkJ!2sIP<=5Bjzve$sq>YH}t<#*n8%M?sHVpO5@pLgwC8X59w+z2!RIy$etFB-Ssh+fPl))ht9zyIjT=;OcmcQOA8+#&!~u3^kM zO?&Q+Ho#5O)~i?3E68VEK8O9zhgUSC*hdA{CM=^2FsV31KP=6PtdJREDizb1ueD>Y zT&=XF*Iu31HNZltVI5W18df7m2qFhZdH;eO9C+V~!4^R&f`HWD$w?Y6Om+@AD}V_n z^@44Z-Nl93Q+A(va&u}o$&QrArAe74Xc7)4p$HZn0K_~NMm@~<6EoBg_aYafC4l22 zA$N;4?S!TvEW!xTTW`Xic6v|6Tx#;!Q@@BS>hd&ZEUt)+Yx|Gt!LJXW9j^3WmxbfO6;_}OOM`s5h3mvCd zUwt!v_4spmE1ry7ZoMw9zUnf{JsZQL3o$raf$>Egq$VWh(KAr^Le78K$QGp$mJ}>9hj&C1l*vqN2 z5sd5n`}5{I$uUsEiKgyx`DxovBqZDA{-gX5c$NKuH%)))fPs+z~Rdxk%;; z&znk_Arq$)?;ISQ!ZK-r&;e61hv;JtPK8R1i|O=lCYOIjPWsZx*UV(iw45)brh+Tr zA+^lqvT&`cGBV(%LGZ(bFIz7M2fxk7l$}j+$$)hft!vi1bf8m!G9;7q^vDpAwFgqs z*r`|!dwO-`#XCp)AXPw4d&7LpqE0ynf?fo1$(=!2$N%hN5u33PO^jm#kKdmr&A6#o z)f*@E!U-X5VX&ItV!dChsyE->A4iS=R4(2TU-;~;@#6=7fo{xf-1n6`<9Yzhks~MK z?Y#$L+t$r-%_lB{ZZiO(m*dl)zBw-0wJi=DJ`%4CoQ++4zxC#8(Wx1OlQR~d{nSl} zt|sEd@e|R%rY|`njh(qM!G%{G4v6#|PqZ}#`p>#TD z67D+^L_*W^nO36?PWD&n3&--z-k0LolRrt|-%PO8R(4k{D3!~RYidUIm=GKl`fi1z z{mDtPP-b@wFcw@G1T^xv1w77T0v*If#c#lm@0fR4;ac`%I(i1f^;lm}5~v7l?66A! z$HA3@!bQjPeC!U6IqMN8-H|X7V)bVPklQYMvkOs{oIJK zp14V=;XQbfutigQe~F1V+}*W1;j&yGRXh?`;p$YnH>C(mozRt}rd>D39DZ$L=txxY zr6bpe(b$2M!H+efdt#M&wS-ax6Xs zCvFhcqft;*SY`oOnA-(i)lJ)@edBHdYfOPk7%ZG?$GXVq+6|*;0a}G6mFBKBOmwv zgmkF33L;3%y*WQKSz1RC*NO6&ts68qCs)R&qU;*nmiqhgUO7bS&@^GeRb?SiucwwA z926|^E7R@*KSqox$}S$38=TC6F--l%TZXOfj$(F#AeA`!ljGq+kmZwuRuSHy6X2je z&UmzRc4JvZY3`i4=3ha!&mfR<^V@~*lP<(d2N6B};D-+bV7?Z6_FRn4O*aF(GmZju z&I}Sja~da202Y<8Cs86^zqS|JG{&KWN8{89!z)Z-nbw#(PBv%_BCqM`={R|EAZFo^ z_3?WQuFh+(y%}2>2%r4q4RL1RbezF)p9;}NGUp|UD~sex=)2@t069+k04#_;Y4L-@ zc}S}k;AGE$x{Q_SY4p(y8-m3iuOUfaaMFA)1<~zCplSL!{gRHQ14zxdbzjh8UkB)9 zyfT4LyWoe34KO5^PI0lM^oq!qF8c&!TBQ_F{Jd}P+wu1g{3L$<^M{z{ z2HxpE*_R;9EWXrm7<=l}iFoFjXX00nJ{o`e&%PBm-F!>-=L<%3#YU!i=PjW%RQvcK z=HbuA{NBeR4nGqC*r_0*a#+>cv1r|PLyT|vR6IXeiATQoi`;zw*`NP6?+b8TCr$6B zQzL$G+OEe0d{rA%&azpoqJhvMDtQg=($byBa0&NYJBW!6b zr`oZ`5s)k_u&Z>AmP3F!vOCuC(K9hQJ{psN8YlN=w8Cfn&aSYPiCPN};Ea+ffPtI@ zUzl&3>67mQ1$A`f-ss(!8JY8(;Nv-89a*AHw!YwpX_sj>0Z>lAWFS*hx7PRq8wBgH#o|Xfg)uMy5oQU}b1e}r?xNTq0uj1 zeJ1|;ul_P#eEzvKiYPeUa4Fct*+-IV&%EQ5?#GSVbI&~$!>ANYOisk-?)*G++xZO} z^M`&*si)bsK3d?gG++6}Xv5Fi+>z&F{=k#aV=;!h_rxW?{&nd1J{vJLLo6ud)it1*sw6@nNs+0RGE1Gnywo4%DnvH#FC_@}7v0{|Qs?_vQ@=XOl2@<5tlr7NDQ2i$ZKGd#mIT$?%AlupQ+ zkjvFEUeyF<&8^&9rL!BJApOOFjEafdU2Sm>GcgOZ(C~km zmj^mYvfEv#y-jeyN@2+=8~dPwoGgbw#aA=(A*+C?0ZStTQqE6e!Yy~oC(JgGivd}M zlhV-8788iP8loR=PcaA+_))*W%i?C>%8U{ib4_1&-20`^A#H-~kts{8=kY?lj4Y`9vuY^dCiaz=UWjl1r@xG6pLsevi~UBC zYB!s-{{FS;m{c*s_~`>bh9+=US&$T8_x1JS$Lu%t*#v`cy#8AJl~K>eHSJxgf&d zc1SrUlVigy*t`^XgY?u^08c9mzxpP6Mn%7jBPf$jJTi0|5!F<*clW_raixX4jDTd( z>j#IQN_}#nbrpug;f$)D!jd$InyrUx9L?{-EP&yl_KwZqe(vLi%DIq9;bJ1VDIS*G ze933ZRfgx`n)j!JKb}h;J90Ogn+l&E0xj>imI^^Vi6)B9&N!Zj5h3vLdM+Ee5NHNI zACLd?X~a;+Q-q|I)Tt9hG$-HZzt9$k4;+u5KlDh9jt|4p>B3jdL>xYHIL5Gu>O%bW z=+7UEy>IQKb9d#={q&ikcmnRu-u-V!?Z2v|jRUr6mAPl*yWjZ%A}W;k090qm_rLwS ze~+J>jd2>~WLG-P>QydPlql2_geBSiWqW<_YF5=F97ROHfnXm>0aK2tC4hL`C1Rs0HL|NxOINW&`mO3vmlNa+_DnWB<Eb)gZA)4KZ^%{@?)$f@uKO|VI ze;hWheJL(xO#Qy)TV6fY%kX&O_)+vV(VIapC&xS^sx{yP9?nzi#;xMSb> z_Fu<~&psRXed8N3xpims4)4b_{H2&C0CrkUA*QP0u~@V7Mgl7@5`WS0uEW0?qA@{E z!j|tJT(nd=jEzjh>#yuhQI<8z#Zg)K?Ag)S`^H=GgCBf9 zP7a*JLo8G3=PCfFFzOky=!9}Cx*=^k7;S}^z*+?x*2;*AZd)L zS6+KDdO7OY@GSQa5>>nh55;S7^4CG0CPybJX~g%Wg@c!uR=hFJNVm*&e*d@_)Kh(c z*MDEgN5!@ca0J(0c0*kAXa94I9(pT=kM4`nQwPz|KI>ez%eN7SS6_;OuNdDHOljDBJ&W`x!=$7o*u_NAm<8?&z*T=QjUk~_s|Ih)h zCzDuU8MJl~&dM}kexuE6LF1tXWhi_+prOZ+)bOKpZC$--OVN(k6A38wZ zN(`Jj9cLFW=h=6}?7>$Uk*rS(s3n{|81sCb^K*T4E+P3_UA*~ykHyT6Thf_e4(7ai zVg6YW!12+Y_tI1w6d?0u7Rw_^*1*g6w+Poow;jJRrYRJO=QR(f^T<}q1OQO>N6=$3-iPhVI3T?IswVJ&jh0p{To&6fG{FK(<<17Y9V1RtA4t|=?ivJb z)022QCe#6n%=!nhKK>NXMEP#m1#Bu}ay5qFusra$5264(7)OpBNl}&Jw<$ymeLZW^ zlIskE>&TG<0FZ^)ux3LHjhu}uFTOH57{oJh$~JG<3JZ5Kj+{6|=RF*)hy#b17^Ja4 z%|WjytCjcCP3Q)>Uz!XTU`DQi6fWrIk|pcXg{z}Nk4J9qAk6DLl@x4!kQ z^H|eIt7u+-_M5(~gvYq=zbDSwa7+VY$4?)N@e@RL2k;CXd^5(+9K)poYCJCH2<+yqv5eFo6% zCItMI;$$30Ma|ea{kx)J{iP9o7sVpciff(giQmwVh!@YRSXQovT&Yt0XYc%cetw%7vzjtqffkKDR{!6ke(@EtHZQrpguDkw*xap>w;*!f)`~f=F_uJAqmR=`L98aM2 z99)stUwaMUa4OB(&nf>~1>>ig*iaibZHlX|{6u{6Q=g8juDS{^wJyQRyUTZu7VJIy z^fTG+BZuJ71DKMF!+7n2JO3wt`e*UAKmK|QogD%MjaW+-5HD;-UM)m4+qrQYzHr842F_3; z&n3X?;kqwWy4N_*N|``VZVip5ZMnavSxar=+0|z%8fu+O-#K$Wk~`hKOi+Tc-^GSL zA9epzm6^c4UODfz8Ee93`b`%;iuy151mZ?~gwS`V0J4(<2V;2u%Q5)&b1{5mA6!68 zY8gujPTg40_vH2nvWs43!2*(c1-@q9HUEouS{%+@&GY`UM9c9zf0E0?oh(owYtAh# zRGOmWakW8z92>8S*m`X=^j{J+VxQI8Sg7D`7K;eun}pZzRe zdF2&0jzsa!!DRohXFm4@&@+eH!C~|rk6`-#6yWR&anG0U#X~&-EZKn8lm335{Ph#@ zgTMPP@$xG#rz%0t!AlvO6FPAJ&;c_*t7NTd<@D5(b@u`UTQPR=S$^>Szl+Cs9@iQN zdpBaM8!zr7HUI!MiAh93RBygH?)#&!#U)o=7fo}QM-?lrxsz{2gV6w06@ zn2G~<(j?ur05KjO`Ky=S5miD$WSEG0r5EUNMF7XQ-Hao$DxwlQuFAqHK#Gi2AhR>h zsTZ=;sTJASdF@0hAZsyYH&GuzE&0X{HR%BZ$~rkO%2!9kjw=AEFSCkboPZkUCq$TB*#YxWk5y;{nFO8Y)Ju!3UDx$@oiBrdq#Spu;G1%C0=jH+P z_7OS<@7KeUQ$RXp_9f~}eKk3W^nvW`zHHO77i|A#aB{^wOP2xa87#Avm}DJ%+Sf)y z2OJyxN##ya4$b03G%(!I#Aj^4(Ww%MmZM=EP4UJX zuf>7=2U4_RJDrb?BUbv`@BIy;mVx;4SMH08FUFcjRfYFSaz9>r;f44iV*IC{ev*EV z*u^=~xfAQaP8RrfxYp`;x<)8Ql50IBM?C>Lp5gcJi|>8+J9++(zxgKsuZ`zwa$P#G ze?N-4_s7FO`)Qm;9A)`Tg{)_dfRAo;BRjGBg_F-e!w`*)jwJ~3UZ3bRWAeZL@CPx< zCgfZH{LkakOD@d~yw_jAdhEO3{%b(f%bC}53kgTd61S&!Ry#lk6{38=MUnH5HLnqUuJ$xLs}si@qtE>B*uCA&+RkhCE`;cJ?A#!3{7shJ+ zV{*Tc1T5rOg@Lg zq;1g3XY6ZTp~A~hsG_ZzYFmZq02+jRT2EGdqI4B#&kz2iGQdw^9`nP8sr?y0Gh5io z&8X=RJffWf1~>V1nzb~PA&p+4mR$HfFZQ}ful07NagioFp7;>g#G@A#;{aLk?vxGZ z-70}iktOr2%%>H8UBB}0pk<09>j>%4_yckR4 zzN*g=Avgi@838qPLrkiJ;L-XveC|aD-|vTdA$YD|w#@2PFAQgi+uY6vjFz$&TTiH5 z?C1AMD(xrMo7W}hZlJoY*fSHi$z0{geQ#1~;Z8VOXKgUGqdAmvysi;5t2%p3ah+RT zmYRHJ47`@ihtn{5>{gmkotDu6GYnKrHxA$+BbCn_4dj3(@(*S3Aqv`iRK@f)v&*gVtsTMDplJ)Bz}vPf3Bj#(tEU;rxi zOa7jhz8}MosZ&izX2S_K-)L`L`|b{1?HfmA`@-oL`DP2x!1CQ0mE7Xrh+r5sv+Um2 zj?sWKCan8wO$;jcqN14*V}eLtkP0%<`8g!A5s@QgwhI=oXgaRP>ilaEchi2wBp8V^ zRoP5e#Jsi%LEne$1TxG zBkuGU`)nS}`)u0W51!1ZsHm#YgQ(f0)_+aclcCZ;tBFMd(2ZOB&Fed$+aYFMamC~; zS+UmQb^ovrXHItNtyrydXV_^svI1;tVmjRu171sJHIO+)9t4xD{rEVS_JbuO+$rQ8$qXv&-FbvWO87>D}BofV1DWk$Ic{=(VILkJ!@g5Uqp{~F&8IiM!Ly8eeP#p%OLI{ zf4zKH4E29<;atkmQ1*DalG%cw^pX3&5?>un`taU@<9lUAy`Nc8I6B{bUF^Rale8#7 z+K!$VphC{yu2a->mq1`2mVz-ucVA92Xj`B6i#I&q^o5D=dZlb3Am8C*%T;lPdYLY1 zGWo`P@53JZI*>0gG?KlQzteGH?W#=KpRUPvEuJLByuZXDd%?)}Q|EI|SHfA&mVggg zQjU?wxh}spUom0(<{1qrB!sQ*Y~tyJ&sJ<>UpeJ4KW;+ssLbdc)UlLNomxgH2l?F_ zigp#%Z%|cqz6YstcpGl6k!&N~!Qw9?r=WG$L_1+pV20wKF~>_VTcV;eR;|b5Jl;p3 z+#MUMhaQ(A;GiLz&s@d-6nj)kLPD?W&+H|;*we>aGo-r~(_Nx61B=swm7uoT>%E|w zZCr}<1&brxG3y=IdEBWF6yx=CMJ={iB~meXxK*_cnLu@Hv;~?mF;0N)rsgY_JFR%h@;o=ygZ47V57 z*_|gh5{Mo}>RW?jWe0`jzLPk%ZA%`0DZ=-{9M{or@e0YBVI(i-xNK&_5NtedZqID< zaV8+&6K3GRi&cPMV4w9dVNSmZ^0ut9==jfcb95rA_|bHRCIxaXSdZo@1E)=dwys&| zLBu5|1mTAHL*071dAoHY;>B^h?#~6-zkU~MdC@$e$pskdcu@)hpCdTLM=qPd_sMth>vrW(MHV6gNdg(zhA_Rs>&!==WyX zkuF)QXV$-W)%rh1HgQalI!2yCnIcs;Jj|D(lEW@$7OI!w1NiVqdqXC&JVWo3tZ&tF zOG@ZFT`oGDK6F$Dg3!0eAM!%gcreg4(%W$8Si!#H|;61(Vm4?<@Y8+P>Szt5)sJF8g1ueYewX)Dn46?r-`J zawZ=WC>3*ecFNcb@hxx@m9H3yMqU)X-E$v&mdw4^sngx|0_^hIE_eJl47@YF z?TS*LOfS)7cyM`yR!dJEYp6t1c&)-&VET6RHkWvGsU&9n=9GVA!-5b-xY4k`?9V-_ zlqD%n`LcM5>3^qOKe2zY2>0}XE#M%~@LxzHkK~*isIQMLD{YGG6d{9LS?EX)GuL}F z?zfb%83foTQir}zbHfB?DXs6f+6CU@%ZoMU^0uqB-1DZD(>=TA#G39h93zTLEpyzL zhE7VGb0{rV{n29Nf>MG}#Aw2Xe7PH@453Kd<^|ZqQ@n$$_lUWXehH3rqvN7d+(#yL zb;F9Oq02BJqyl;qRTXZy(lkQWmPVn->0VrO^FF7i4ME|{EUHn5U$Ic>;znMNd(*+O z!~*xC(1#=-4~_*u**6ERA2&n2=mq5U0{zONXk=l{7_djQqHThnyA?i9+&PhzVJ>fZ z6Xsd^KyhLQhxblj`9?_3(%}T<2(Cx_Fux%3L$l99Npw`&%Ebf9o4ZIHMM#F94Ywp5SEM47GIo=u z5-Z)mD3NBshT87J($KGa*PXd=cr(&tGUlJ$noOGp&uchPmYd19MUc5Zt%G54mU^!ff|iRab~r@ z#7^xs*YJ>v@x-uiwLL|~>we{o7OAt#KD+rt*5FI z{n5{vnStlI%5-GnPuraxr!W}B3CY~7FQMmXV&^!X zd#%epqU8TX5BwyP57$zr$c=ci@xwMS4Q|;Exq><;@~dSR^9O7k1PcR`(QF(2{IR6Q zkyv{#AO~=)e{qHXw@8nx+41&t;^EQ7x+aq;1cZpBd>?L|GKw*;q-o958q0t-JAzZ3 zLDga|kyIWImXmUF!d_L+;$>?Nd@{V%QDR>8;X4RsNK@q4-#KeR)rBDD4mYpn<@@n* zD^A0wXraxvB^YS4w%*uw@o+e2wV@cd5Yp}*sA`FLC0@~IImVQxwNxpbKQAv}S#`9v zU*CH()=)uq><|TXR1S`WBGO@aJIxOGN*tD~B7T;G1@LNesThhyv>JSlX%5X3CXQ9u z><9QFi>oK1*q;6nmPlP<7NdoG;3SwNMWc-EkJmA zppH}Tu`_o+_q;yk5ca?}jTDfgVsE&KMh zQ~2l`%q+d95X)K~p>W&xx!PVQK+SBIlf3O8*5`pr{ere?QF_I}fO9~dv&xS>K|zon z9s#|4Sh|sCaF-PNq>&rLqzAd(Cc1Ahd1@kC3}qn5J9X!(R1pX!xM&*jA?!;AMR=bi zExQ8-=IvCQ3KV|DL#2ZR-I=9j&tI@`=Wp&y&>~;1yV73lPgm}>?u6MhHcEFs_E^4s zLOPfhXv#LQ++D1uDKlvIe_z)aGtBnyeL^jRciz2>`?Kw~)c@x!kX2u|E=E@Jk3H1! z@U0@iGz<|KIWl-FV>QU9{J#9={I+#`?~&+CO_A>2q4ztttMP+Ko90}+uO$B8EB_rc zztShbvbCfidM=V`wDlqW0G)K`F;6xrQ8qO|dGdFeC`=#cK-gbK6Nw4$Gte4LNWU$%83H(-G0htr zW}bFeAB<`6aSp8oBZ}xJ@eUZMTMZ0`Yp@-Es)dHXL@;&7AO1yv+p3r@wq7oy)%|6{ zHKEV2bnUB%P3slo!+mP2RPfH(Z~rY5ULRVfiV!e2A>1UsRyv2R_g&ia;gVsqC2ptI z;-22*z_N>wKF6P?crvrE5foqu9HF?%N52%A%lN=3wbdB0ZzEgMyl#=KKf_g=WX~;N zo>o{|uEL&(26fIDWjZ=r${7u>*+XIG-H%Z$N)cu z6-2QqRo5W27T8kODep1VnL1HM((l6jA{ zw$X-uHA7rRe#e-(X$OpAS3aIG2rw054Ta&q-CeZ0urgKt{gXn5_Hj=ZEhOd~Mz$+5 zlKRr)s2@LE?rn6}ZBg;8;dRyh5!ZW0;RC@mZ@xY^?v%-a2!D4Wk6f~sJ>StJB+&GN zLVZpy(X#ZqekUcpOdP$cPwWGn^QK{B?9i%UWIit;g(qIJ~}51 ziQ>bL+WJQ-+TRFsi)#N+nS>^2MkoH=)uubS;~`E1TC9(`wp*&Rzz+;$BRRP^;-PZg zMBk@krx3TkatV};WYvIObsDg++J_o{ZcQv?mUqk#1aim^I=5OC+C`){tfw26tB4Vc=dV1(DN#^NWR@jeTL4M7DxxSqvm_eE7nX{ zSo&0zKvjF;39TH}QUatbU5+TzMiMSyyGKh36nXoXxo{;%d(3PU43t2_wO8u{(qnM2 zTr{qX`&TBVxT)jVg%2@NQap-N%I!i$vhRLwNbq4?Y`(h|c83G-um;7jhY2^^FA*S# zdA&DXM%Yv4{=QX=4nel)leZWD=sf){EO_$wZz%0I1nW`g(1ogSAuBraN)a_VZ{|jO zIoJi%!^5sWM0geWW1t=Ks%KZoPgI3g{{&8VwN5Qqo6P3f1Zzxbc#g&>5&Gd&;&p`n zo5Mf+%YsIJ;EX~_oJe(*oB23g@+m1((_#EPO=ff($GAc*BhR06;x~Z%q&g?4Go@RSEb<}0ONG?yPbwUg2kl(KS>&owpf1lG~rlBXqx580*R=vuT5_6bI@D8wamC@cdhMMh9b^)+P=TUfSNrm0@41NJQ|s~aEfH`bYB zjRqYmz=&^>zS$DX&0oEe_KCpdss74SahD3$PPR+bXOg);7)CBTjxM{IOu#tu{0Zm; zPAoT2KueXR4)-mbE`&Bz8G1yOBr{rwDPVNl4GjaW!)VK_mCt2f`cB6eG1W(XOxh%X z^g~{9mPTtGwMWYU(zMUysK}+kaRrxy%F=dt z9q9_k>j-0nJW}`hN`r(YSQXw4C7W+qSE^^mTgxtXeNyhycKF@JmdL9MfAYa)-Peu* z>SB<8i`9OF8_yZ!yX)G|Dc!?ZwxuI@!2o!S)=!4mAHIdy)o(i-SQv(T4szR#u? zD-OMT0tlADDzH`ZH@Y3Kes5LmeSDpdm5aeK(Xvu|E26&elsK21_3jQ6MS*!GIbKB5 z^L{K@Y_Y=HUuE0NSRW>AWSy4sf+xUz~L? zD%G(im~Q+edwspQ)ulcdeNYR`^a8gN%h<5K`5jYr3VT$YPG`06V?-=waMa}1dabRCZzMv zYlZkB-hv(%V!`aYGo`27yH(1Z8jm&ES$d>Rfq|G$V_bBM+rAX4c_3=FL z6pfOURbx*5=OO_1fVer|Emdq^UHF4inQ47y22(7XBC?ktt68^tT)J%r5mjE*-qJ`T(-(j;9f< z`<)KwpV`;wr=maQh9-soDS(}C3afQS96!xI1Y<3#fRpX4LfX0>_8zu{?+b%EKsrB( zzfVnm<7f4kAqXyzi2=uY6*)gt2@Dr+*|tB8Fe93*Mjd`Kkbyj1>4sv9_A-}J)8sP8 zvk`+#P|3o5$)GLJc3LJ?&FZ4aOypdt|9H8_GjoT*COILb)E($>Yb_9o+?uTv#eO54 z)AZUs5S?uNdj$$~-}}%SMnE3^{=!HhgIvw^N(ZYGqMd9ol$0>RuCSZW2F}vdjsF0f z+?gbh5W6{z+3WoPAkKU?!*aig76YhvjUM7;3I)_r}s829kY?wUk*3DlDPY_ha z7Qo4Bbu$G8kW}?AEzBsm|1V900KX7sM%+?UTvy1a^xP1KMC&2A+aCseJ_DaEhXQPv z@Z|rg(ib`d6^z6JiEh(9#N6N2xg~3*UgnT0_hg1+gn7B}1_$VJC9Z5=8M>kaBbfNH z)UI6dgiFx?s#mtUhK~ml#gK~&5`;@ZVU67S$KQha?=v>0mzDV2i`?k_h%PSDFd%@1 zRM?$(n+y}wZ1Kyu;6KA7y*%L0@6}T{cYRy0PS-;{haQdeceWRi^3O*b|0JT+`_yh; zv^FZDS zu@kItT!j#LEbFMUi{ICV)Ex(h^Prmz_M7mvf_`E%bb3YQ>pKwfcaO-xAzhl-lTA2l z1BmDmia5Tks!mM;K-**2f)Q@_lom6nLljQg#T-z%&9fpdwu{}>;wZE- z`#5amD1@qAbogIjckZ3XEDaa~lJ5l`NJP=TK-=Ek-0oLDkq0cDN3tQk!Xh4K`)r5P zifPNQWrmXbV04Uq|A4jlUz4qscoF`(Z83pR2jGaNWe%+hEPL*Xl@4;D!{^2YH>BP~ zEPM)GpMW6+6KU<*uK1uRXIgl=mJAjm*IiikCu*KU|7zTKU^N{dR{TaHxe`e9$qg4| zWe!&Zybl=mBqB$2egLPzLlATn_PNJr#BnoNSrq=MI_`Bd@C%_e3bb(lQ&?n0snhbZ z3}lp(ofL7DQ2U0JIJL|G&=&)yWHBQ{a>Y4umRW?_x`RMVoQEftFmB;gxt$ikIOG_R z#KT&EJde1yF-$*^q{Q()|DDaya53=H-w33x$mwv>-!B@YM<~96<%wDj*vj9$admVk8Fv>K>W4Rp z{mDS;>+PpAy+Kl0n~eQX4AIlo1fjbxd&R5QZ8DBs#?#VT1Y~@ywKk4uFhHXi#7y5 z5~mCrp7TB4S`W5A^nYT4;P0`gy%Ek4oP|)`HCj3w^T9PSKN&K4WY z>!~SLeq&#yU=EaQU0cbv2iwr)n4bN9yU{HW8E;DfmI-&cGH)9)7YHr%849`=2`0H8 z@LgmxSAD^tzL$$LGGwtb&G9=fL)mbg6gE=I%5~r9()0JHfT+`}`GV!4W{K-0E|d3x zuwQ@G>LEj1P7j>$lmha|V7R3gw#e}Q&L7R+py>Lm8rD3I(Qi&OE5R?{hWy~bNE>O- zszH#toG)o94d@qT4Sx^`4BfJBFK?P<+{;9;cNybDW{}53yH|kv`gJh+j@Km+G#y-< zz#CrzTQnj78AXxnIli1^=fy4}&J7^ww#3?M09xz=(3RdkmkX@YD^tW|Y$K4vD9M`I zT9sshOCM^=K2U1gG!&LrCU9bb(*EAqXi!I|5cIf7oI}IfS-pws6Ts>fuR-R%fOUy` zHbzaWBb#JKaTi zZrdw}j$X7qB!i4Qj|u#OyLx<&-jo(4Z%z!BXvQghdx)bhevB&Q67%t<1Nch*)_8{o z&xDfN3;oH&=Mlc>{(MRuFAQiwlQa10R{hN2Q9XrxC3}G-C^z90syDcQrtPt zE7_X!G)8K@URh0e>tWlRpJ(wRd)o_J#|p(J9OnHZw8=0?erAgg0Q$gCfVQ4jvy6t& zHt8hEwNDUYbn)Z%f{R2!d^}%D@=Nkr1d5~Qq=X?o_wqBRzfT;#NFAwGRufzoXMzYK z^_&rDMsLtyQ-Pu$U~oNmMDYi85DasY>{9E)G4fqIdvnX~k3*=hyH{_+lBe;FOe30@R;LKSrhd2(E9nEBxW)-*aNEkG)ALW|#Q!ecLmbwj2D!R`uLYdo!^Kl8?}30`?MW zH%Rr{H?3&RdM#XByqXP(pdCa=L&@KrP}-j)6=Ji2hY+8?Ov+VgxuKmFsn~;=j=^lw z*XUD%gRz#d)cr^KWXiboOboMpJ$SZLtKR=^Ra9c?#PU8+kW{+NKR@y72w7p~p@ z)hql;g<@;Z*fc3sX*-DcrN$?HLg$K|vG_x4O925e;aTmEUx}kUEX+6`zMXH7j0%Ac zh$~(fbU#yairHIsIP_v^|(G>~ea)7%}QH0+g8kdHSs*Y9#t5;!?WdO-bg@4xz?u z%*;39%=;t4eiwZJ3g{ldr=XXrSEJX%Uxu?-XXNKm`pI@$NB;|e+|M8&pq|T_xDSL@ zgv4mn{HWhXt=ogaT0cSTke*-#B2J?8QfXWasB($ZkN12IbsrlBqcm`J zZjdOmG+nbMm40N}Dxc1M&$Pe@*^g9utIWK2hhOd$B3;d6GDu@uMZaM%UT6Ts&~24B zaBqHRaA3>Fr_OMRvm{ANWm+ERMnUaW4TPJqaGLI6G!Y*RJ5Ze~SuiVO^3_!0UCcjA z2ze_oRBFA)lL_=@ax1a(#qDry z+1aC-1`O{&MZUjhNm`0>`>h)ruFeNzM`nT8#KD+5+d4T-)+Z|#lX5S$>R;%>IJc?_ zcLKopoPS9$oL!0)x96ag*nqtaf9+0pqhIe|{ob!HZcn<-bl?rT&b&N3bGzZ;_#d%S z{i9=dBA8uTIc>I=yq^qPc%_CKN+t5(~a6tXZp3n@}tTDP4>> z!E*R%%k;yplb<8Oh(x16z5X4dzb>DxJy1M8@yLOEaiqRq2d^!ai&dCtD?KZ`>7je( zTzLyIq5WG|urQg%Fu_3%aXr9xnjrn*3T9l9SvuE}eD)#2^lt3Vn{HS)N#CM#3MLJ7 z2O9>HW=mn1-Z*t|o{)O--dd^5!2NtQTQ4EVi}Pu{Yz>>vgpHA<#uLB?;TGd6+dNgC z_z6fle`pe8QSF>6P?NhjZ0@quU}*7^WkUT>B&bK0YLvwjbjePicdGKZzP=Epgq=`y zxk2}iWN(L+@2foAuh?)l&Gb4u;=ehJ-M+C=&b4$-LVbGajQKVVt>?QkL@j>|ac<^O>hZ-d8A=; zDRQ3r1K^V@Hi^2WW^dW3)X|lwa?Y&6$9+>z67DyT;ua(8-c==y^o*vlvb#9hE+i(u zBvf7w(Qa@MSK6<5B91nF64t1`r!^12V1t*2 zJhQ#k^3u>Jei}3hswtQcF@nz&{!x&#(k6j)*cIBMoP2i7q*6!PUKTnN zkpSb00WFnoS^yK!NX7Qser2AY|3@rTg-o#u%lVNmw)d3PhWE`E6cki?J}Yb_Q*A>R z>-o54M#f?$ewC)#o+LjzF^NDzW6aI43;w5BwGlu*6lkvjtc=3Kl*>J3_3d*kKLYCu zjK*%xP6Ay_GAu>hb7EtS%APh5>xPTgkp4A(Lnu!6i~3M%9^=A{8r(A8M+O@PMHM2Z zHtz4z)720zXXGaVO~okSXZ0WYIa@6NK1vR+VWC^vujrzMgn@pgqr7bTT{01i$2u}^po61QvRl=5(X+KtQg8I^QjX|?v0jYR zYoK%Sl(TL9!P}7DFhEh{$-5duG*ZD<2Nyr5Ct+T_51VLFyP`@m2ld4ox}t%CO4UY+ z6mvld*vnG)j@V1A5S1qu$xeDb+K(xMXu?C-{4K&~UTxq`#0PbACpW+m{S%#oL380! z1vXa!)D$rF_%QmGzmZrB*$;~Q^sI(Ykd#L0E^+UrlE4aeUQReGiwNW-m>@6?23M=r zCV&c!14060adOEU`n(^8ZH0md>IFoRz(c!Fm?n~D)ii`V3Nwjlk9=+^)! z)xfCCocu$%%9FOJ+1+NaQadfAY9isl=UI}19ZHvtlbr9IlNL}zWXjV|Sp6hr-iZw` z$3}0+mEcRvFJX}8zCpV9Ou;9=26vxX;)n3_tZ9|Ymxu8Z?NA2_bQ5hMslDRxYGT`R zy-(ory2SsqtGqqr{DHJwGHv-Fq7&Wh%yqV#Yud=xQms7;f#F4qIK3At-=nL+T)pg2 zpoM)?!KvvgYdIQ#J3R~Z5;_`DLp%rLjPM&O7h3y0*h zGAtzZYzPeu8DM^~7g&X+BX9O9B83>ByzI_haCUiYWnIj`8pL-i{JTZUnP3vL1Wrym zs!mW~$#Hw)W9k)lHMYa&=863H{cMl2eVTjX-mJz_xuyq2`oKI!U3OPb{yN{xQ*qJJ*>_-#V=+C zwU)`q&l}7Z1r?V>`|pTW=aPpzra;VR6G#2`NY8OD0v<+dH?07Nhfu%dnOznmQ3jmK znXsP_YqP|&?;r)t0CZqZOQX@s($~7l)3-IvK|e^497o8zB>FjHZ#Q9?F|_aWwNc&2OVmx zgl*~DPLaR$`-!pjCp-Htv|qE&XjzGgoK_b^03<-#pLBJE2*4d-aSZzbhbYa34heKV z?r#@7dN)Q8<#|@l`pg0cf&8KzEC|CdY6W?Tg{|&-@f2JX1Sf8GLmVo>()@uKxj-Jw z768kPGVpfh)HX;)KWCdR0RFL^W|h&!^(8w^Ac!HdZq88cvsU7kM5#JA@y?H~)LnrR zOVwkoF|%(pKlF9Je8Z$Wu_>$glhjGK-=r^sQMi>Iq9GmJ%kv4<5?OQ-M-;YM0hDsI zBM|_ay7GNqyFZYb9GcebYq*O3j!WB6c2K4->+6%MeQt&$lW(C;fvPEnWO=z1aDj}FY`b{?Q^ej~Wom_~$wH=LR1BfKNThw%x zUs#<)%Df&52dD8HF-%>gNAP{4Ef#R+R?0&%q@|E!;z(e}$4em>M>QZnxBi@_=afpt z@W}AwZiT_4OBNk>0%bj%;NAnI+U=k7fXZNG{i^|1Qu+6lBc380bTa%CTf{q0Q1f(C zQzl#HDAg!!-22!s;P^giFcu&7*tfy2;WkEMqr-PtluTjtACBcn*MEq>6LDHD87{qn zB*PR7687WQ!!sAb!I&!F2%GnOevS_>^Q)GNocPL|hnbW<&78a9U*hz(K>m3bm1VEM zQq9(7z=~fe&GvbLZ!aG=bfs-a>xfT`#+z>@!D9thlVbMZ@Z+1rra-hE|MA?gN`Ery z)OYE$3;Usq2UcMi!CM~XR$f_^+vj9m_FqDNJd7$A^5o<5NtH~DDj)r@`?09|0@cx0 zE8<)7vUZL_N{(QTqPvq;S3j*jFuDs_9MveB_>T?$R`NQE$k{@E?vEmay&+qm5Yn}f-`9EF5 z1V?P)lH3Bl)W{U)g3E9bN2OZMAIH^xHTbn*Z}$KEbO$SZq{U?-rvdir|L@v|?|3MD zGFYT$F8p=pB<24dfd8@jK^16%lnAiI?RvSMmj7s~|GsT^uv`Gvra`GglTX!uZ~IS4 zO$4I~RHo+3^{5Q{|5fV4FcYjC#5u2X>ALklZT-Inn2N;^9ks4T*~vTpQ}_QGF(}1h z08RuyS&Sb1_w)Zx$;Zl=wp+KKz02q3|NC;xQ`>xphj|;WP;0zP0(=~@Qc99lpN#|m EA1lRg!2kdN literal 0 HcmV?d00001 diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 079d4c5550537..c6090d9ec30c7 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -3,8 +3,8 @@ layout: global title: Spark Streaming + Kinesis Integration --- [Amazon Kinesis](http://aws.amazon.com/kinesis/) is a fully managed service for real-time processing of streaming data at massive scale. -The Kinesis input DStream and receiver uses the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License (ASL). -The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concept of Workers, Checkpoints, and Shard Leases. +The Kinesis receiver creates an input DStream using the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License (ASL). +The KCL builds on top of the Apache 2.0 licensed AWS Java SDK and provides load-balancing, fault-tolerance, checkpointing through the concepts of Workers, Checkpoints, and Shard Leases. Here we explain how to configure Spark Streaming to receive data from Kinesis. #### Configuring Kinesis @@ -15,7 +15,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m #### Configuring Spark Streaming Application -1. **Linking:** In your SBT/Maven projrect definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). +1. **Linking:** In your SBT/Maven project definition, link your streaming application against the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). groupId = org.apache.spark artifactId = spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} @@ -23,10 +23,11 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m **Note that by linking to this library, you will include [ASL](https://aws.amazon.com/asl/)-licensed code in your application.** -2. **Programming:** In the streaming application code, import `KinesisUtils` and create input DStream as follows. +2. **Programming:** In the streaming application code, import `KinesisUtils` and create the input DStream as follows:

    - `[endpoint URL]`: Valid Kinesis endpoints URL can be found [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region). + - `streamingContext`: StreamingContext containg an application name used by Kinesis to tie this Kinesis application to the Kinesis stream - `[checkpoint interval]`: The interval at which the Kinesis client library is going to save its position in the stream. For starters, set it to the same as the batch interval of the streaming application. + - `[Kinesis stream name]`: The Kinesis stream that this streaming application receives from + - The application name used in the streaming context becomes the Kinesis application name + - The application name must be unique for a given account and region. + - The Kinesis backend automatically associates the application name to the Kinesis stream using a DynamoDB table (always in the us-east-1 region) created during Kinesis Client Library initialization. + - Changing the application name or stream name can lead to Kinesis errors in some cases. If you see errors, you may need to manually delete the DynamoDB table. - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see later section and Amazon Kinesis API documentation for more details). - *Points to remember:* + - `[endpoint URL]`: Valid Kinesis endpoints URL can be found [here](http://docs.aws.amazon.com/general/latest/gr/rande.html#ak_region). - - The name used in the context of the streaming application must be unique for a given account and region. Changing the app name or stream name could lead to Kinesis errors as only a single logical application can process a single stream. - - A single Kinesis input DStream can receive many Kinesis shards by spinning up multiple KinesisRecordProcessor threads. Note that there is no correlation between number of shards in Kinesis and the number of partitions in the generated RDDs that is used for processing the data. - - You never need more KinesisReceivers than the number of shards in your stream as each will spin up at least one KinesisRecordProcessor thread. - - Horizontal scaling is achieved by autoscaling additional Kinesis input DStreams (separate processes) up to the number of current shards for a given stream, of course. + - `[checkpoint interval]`: The interval (e.g., Duration(2000) = 2 seconds) at which the Kinesis Client Library saves its position in the stream. For starters, set it to the same as the batch interval of the streaming application. -3. **Deploying:** Package `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + - `[initial position]`: Can be either `InitialPositionInStream.TRIM_HORIZON` or `InitialPositionInStream.LATEST` (see Kinesis Checkpointing section and Amazon Kinesis API documentation for more details). - - A DynamoDB table and CloudWatch namespace are created during KCL initialization using this Kinesis application name. This DynamoDB table lives in the us-east-1 region regardless of the Kinesis endpoint URL. It is used to store KCL's checkpoint information. - - If you are seeing errors after changing the app name or stream name, it may be necessary to manually delete the DynamoDB table and start from scratch. +3. **Deploying:** Package `spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}` and its dependencies (except `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` which are provided by `spark-submit`) into the application JAR. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). + + *Points to remember at runtime:* + + - Kinesis data processing is ordered per partition and occurs at-least once per message. + + - Multiple applications can read from the same Kinesis stream. Kinesis will maintain the application-specific shard and checkpoint info in DynamodDB. + + - A single Kinesis stream shard is processed by one input DStream at a time. + +

    + Spark Streaming Kinesis Architecture + +

    + + - A single Kinesis input DStream can read from multiple shards of a Kinesis stream by creating multiple KinesisRecordProcessor threads. + + - Multiple input DStreams running in separate processes/instances can read from a Kinesis stream. + + - You never need more Kinesis input DStreams than the number of Kinesis stream shards as each input DStream will create at least one KinesisRecordProcessor thread that handles a single shard. + + - Horizontal scaling is achieved by adding/removing Kinesis input DStreams (within a single process or across multiple processes/instances) - up to the total number of Kinesis stream shards per the previous point. + + - The Kinesis input DStream will balance the load between all DStreams - even across processes/instances. + + - The Kinesis input DStream will balance the load during re-shard events (merging and splitting) due to changes in load. + + - As a best practice, it's recommended that you avoid re-shard jitter by over-provisioning when possible. + + - Each Kinesis input DStream maintains its own checkpoint info. See the Kinesis Checkpointing section for more details. + + - There is no correlation between the number of Kinesis stream shards and the number of RDD partitions/shards created across the Spark cluster during input DStream processing. These are 2 independent partitioning schemes. #### Running the Example To run the example, + - Download Spark source and follow the [instructions](building-with-maven.html) to build Spark with profile *-Pkinesis-asl*. - mvn -Pkinesis-asl -DskipTests clean package + mvn -Pkinesis-asl -DskipTests clean package + -- Set up Kinesis stream (see earlier section). Note the name of the Kinesis stream, and the endpoint URL corresponding to the region the stream is based on. +- Set up Kinesis stream (see earlier section) within AWS. Note the name of the Kinesis stream and the endpoint URL corresponding to the region where the stream was created. - Set up the environment variables AWS_ACCESS_KEY_ID and AWS_SECRET_KEY with your AWS credentials. - In the Spark root directory, run the example as +
    @@ -92,19 +132,19 @@ To run the example,
    - This will wait for data to be received from Kinesis. + This will wait for data to be received from the Kinesis stream. -- To generate random string data, in another terminal, run the associated Kinesis data producer. +- To generate random string data to put onto the Kinesis stream, in another terminal, run the associated Kinesis data producer. bin/run-example streaming.KinesisWordCountProducerASL [Kinesis stream name] [endpoint URL] 1000 10 - This will push random words to the Kinesis stream, which should then be received and processed by the running example. + This will push 1000 lines per second of 10 random numbers per line to the Kinesis stream. This data should then be received and processed by the running example. #### Kinesis Checkpointing -The Kinesis receiver checkpoints the position of the stream that has been read periodically, so that the system can recover from failures and continue processing where it had left off. Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. - -- If no Kinesis checkpoint info exists, the KinesisReceiver will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable. +- Each Kinesis input DStream periodically stores the current position of the stream in the backing DynamoDB table. This allows the system to recover from failures and continue processing where the DStream left off. -- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no KinesisReceivers are running (and no checkpoint info is being stored). In production, you'll want to switch to InitialPositionInStream.TRIM_HORIZON which will read up to 24 hours (Kinesis limit) of previous stream data. +- Checkpointing too frequently will cause excess load on the AWS checkpoint storage layer and may lead to AWS throttling. The provided example handles this throttling with a random-backoff-retry strategy. -- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency. +- If no Kinesis checkpoint info exists when the input DStream starts, it will start either from the oldest record available (InitialPositionInStream.TRIM_HORIZON) or from the latest tip (InitialPostitionInStream.LATEST). This is configurable. +- InitialPositionInStream.LATEST could lead to missed records if data is added to the stream while no input DStreams are running (and no checkpoint info is being stored). +- InitialPositionInStream.TRIM_HORIZON may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 3d4bce49666ed..41f170580f452 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -233,7 +233,7 @@ $ ./bin/run-example streaming.NetworkWordCount localhost 9999
    {% highlight bash %} -$ ./bin/run-example JavaNetworkWordCount localhost 9999 +$ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 {% endhighlight %}
    @@ -262,7 +262,7 @@ hello world {% highlight bash %} # TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount -$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount localhost 9999 +$ ./bin/run-example streaming.NetworkWordCount localhost 9999 ... ------------------------------------------- Time: 1357008430000 ms @@ -285,12 +285,22 @@ need to know to write your streaming applications. ## Linking -To write your own Spark Streaming program, you will have to add the following dependency to your - SBT or Maven project: +Similar to Spark, Spark Streaming is available through Maven Central. To write your own Spark Streaming program, you will have to add the following dependency to your SBT or Maven project. + +
    +
    - groupId = org.apache.spark - artifactId = spark-streaming_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} + + org.apache.spark + spark-streaming_{{site.SCALA_BINARY_VERSION}} + {{site.SPARK_VERSION}} + +
    +
    + + libraryDependencies += "org.apache.spark" % "spark-streaming_{{site.SCALA_BINARY_VERSION}}" % "{{site.SPARK_VERSION}}" +
    +
    For ingesting data from sources like Kafka, Flume, and Kinesis that are not present in the Spark Streaming core @@ -302,7 +312,7 @@ some of the common ones are as follows.
    - + @@ -373,7 +383,7 @@ or a special __"local[\*]"__ string to run in local mode. In practice, when runn you will not want to hardcode `master` in the program, but rather [launch the application with `spark-submit`](submitting-applications.html) and receive it there. However, for local testing and unit tests, you can pass "local[*]" to run Spark Streaming -in-process. Note that this internally creates a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. +in-process. Note that this internally creates a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) @@ -447,11 +457,12 @@ Spark Streaming has two categories of streaming sources. - *Basic sources*: Sources directly available in the StreamingContext API. Example: file systems, socket connections, and Akka actors. - *Advanced sources*: Sources like Kafka, Flume, Kinesis, Twitter, etc. are available through extra utility classes. These require linking against extra dependencies as discussed in the [linking](#linking) section. -Every input DStream (except file stream) is associated with a single [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) object which receives the data from a source and stores it in Spark's memory for processing. A receiver is run within a Spark worker/executor as a long-running task, hence it occupies one of the cores allocated to the Spark Streaming application. Hence, it is important to remember that Spark Streaming application needs to be allocated enough cores to process the received data, as well as, to run the receiver(s). Therefore, few important points to remember are: +Every input DStream (except file stream) is associated with a single [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) object which receives the data from a source and stores it in Spark's memory for processing. So every input DStream receives a single stream of data. Note that in a streaming application, you can create multiple input DStreams to receive multiple streams of data in parallel. This is discussed later in the [Performance Tuning](#level-of-parallelism-in-data-receiving) section. + +A receiver is run within a Spark worker/executor as a long-running task, hence it occupies one of the cores allocated to the Spark Streaming application. Hence, it is important to remember that Spark Streaming application needs to be allocated enough cores to process the received data, as well as, to run the receiver(s). Therefore, few important points to remember are: ##### Points to remember: {:.no_toc} - - If the number of cores allocated to the application is less than or equal to the number of input DStreams / receivers, then the system will receive data, but not be able to process them. - When running locally, if you master URL is set to "local", then there is only one core to run tasks. That is insufficient for programs with even one input DStream (file streams are okay) as the receiver will occupy that core and there will be no core left to process the data. @@ -1089,9 +1100,34 @@ parallelizing the data receiving. Note that each input DStream creates a single receiver (running on a worker machine) that receives a single stream of data. Receiving multiple data streams can therefore be achieved by creating multiple input DStreams and configuring them to receive different partitions of the data stream from the source(s). -For example, a single Kafka input stream receiving two topics of data can be split into two +For example, a single Kafka input DStream receiving two topics of data can be split into two Kafka input streams, each receiving only one topic. This would run two receivers on two workers, -thus allowing data to be received in parallel, and increasing overall throughput. +thus allowing data to be received in parallel, and increasing overall throughput. These multiple +DStream can be unioned together to create a single DStream. Then the transformations that was +being applied on the single input DStream can applied on the unified stream. This is done as follows. + +
    +
    +{% highlight scala %} +val numStreams = 5 +val kafkaStreams = (1 to numStreams).map { i => KafkaUtils.createStream(...) } +val unifiedStream = streamingContext.union(kafkaStreams) +unifiedStream.print() +{% endhighlight %} +
    +
    +{% highlight java %} +int numStreams = 5; +List> kafkaStreams = new ArrayList>(numStreams); +for (int i = 0; i < numStreams; i++) { + kafkaStreams.add(KafkaUtils.createStream(...)); +} +JavaPairDStream unifiedStream = streamingContext.union(kafkaStreams.get(0), kafkaStreams.subList(1, kafkaStreams.size())); +unifiedStream.print(); +{% endhighlight %} +
    +
    + Another parameter that should be considered is the receiver's blocking interval. For most receivers, the received data is coalesced together into large blocks of data before storing inside Spark's memory. @@ -1107,7 +1143,7 @@ before further processing. ### Level of Parallelism in Data Processing {:.no_toc} -Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the +Cluster resources can be under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is decided by the [config property] (configuration.html#spark-properties) `spark.default.parallelism`. You can pass the level of From da35330e830a85008c0bf9f0725418e4dfe7ac66 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Sat, 6 Sep 2014 14:49:25 -0700 Subject: [PATCH 0610/1492] Spark-3406 add a default storage level to python RDD persist API Author: Holden Karau Closes #2280 from holdenk/SPARK-3406-Python-RDD-persist-api-does-not-have-default-storage-level and squashes the following commits: 33eaade [Holden Karau] As Josh pointed out, sql also override persist. Make persist behave the same as in the underlying RDD as well e658227 [Holden Karau] Fix the test I added e95a6c5 [Holden Karau] The Python persist function did not have a default storageLevel unlike the Scala API. Noticed this issue because we got a bug report back from the book where we had documented it as if it was the same as the Scala API --- python/pyspark/rdd.py | 7 ++++++- python/pyspark/sql.py | 3 ++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 04f13523b431d..aa90297855c93 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -212,11 +212,16 @@ def cache(self): self.persist(StorageLevel.MEMORY_ONLY_SER) return self - def persist(self, storageLevel): + def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): """ Set this RDD's storage level to persist its values across operations after the first time it is computed. This can only be used to assign a new storage level if the RDD does not have a storage level set yet. + If no storage level is specified defaults to (C{MEMORY_ONLY_SER}). + + >>> rdd = sc.parallelize(["b", "a", "c"]) + >>> rdd.persist().is_cached + True """ self.is_cached = True javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index e7f573cf6da44..97a51b9f8a24f 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -29,6 +29,7 @@ from pyspark.rdd import RDD, PipelinedRDD from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer +from pyspark.storagelevel import StorageLevel from itertools import chain, ifilter, imap @@ -1665,7 +1666,7 @@ def cache(self): self._jschema_rdd.cache() return self - def persist(self, storageLevel): + def persist(self, storageLevel=StorageLevel.MEMORY_ONLY_SER): self.is_cached = True javaStorageLevel = self.ctx._getJavaStorageLevel(storageLevel) self._jschema_rdd.persist(javaStorageLevel) From 607ae39c22947dad8e65cbcec310367925f62eba Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 6 Sep 2014 15:04:50 -0700 Subject: [PATCH 0611/1492] [SPARK-3397] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT Author: GuoQiang Li Closes #2268 from witgo/SPARK-3397 and squashes the following commits: eaf913f [GuoQiang Li] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT --- assembly/pom.xml | 2 +- bagel/pom.xml | 2 +- core/pom.xml | 2 +- examples/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka/pom.xml | 2 +- external/mqtt/pom.xml | 2 +- external/twitter/pom.xml | 2 +- external/zeromq/pom.xml | 2 +- extras/java8-tests/pom.xml | 2 +- extras/kinesis-asl/pom.xml | 2 +- extras/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 26 files changed, 26 insertions(+), 26 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index de7b75258e3c5..4146168fc804b 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/bagel/pom.xml b/bagel/pom.xml index bd51b112e26fa..93db0d5efda5f 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 55bfe0b841ea4..b2b788a4bc13b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/examples/pom.xml b/examples/pom.xml index 9b12cb0c29c9f..3f46c40464d3b 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index b345276b08ba3..ac291bd4fde20 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f71f6b6c4f931..7d31e32283d88 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 4e2275ab238f7..2067c473f0e3f 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index dc48a08c93de2..371f1f1e9d39a 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index b93ad016f84f0..1d7dd49d15c22 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 22c1fff23d9a2..7e48968feb3bc 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 5308bb4e440ea..8658ecf5abfab 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/extras/kinesis-asl/pom.xml b/extras/kinesis-asl/pom.xml index a54b34235dfb4..560244ad93369 100644 --- a/extras/kinesis-asl/pom.xml +++ b/extras/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml index a5b162a0482e4..71a078d58a8d8 100644 --- a/extras/spark-ganglia-lgpl/pom.xml +++ b/extras/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 6dd52fc618b1e..3f49b1d63b6e1 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index c7a1e2ae75c84..a5eeef88e9d62 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a5eaea80afd71..1efa9045208fd 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/repl/pom.xml b/repl/pom.xml index 68f4504450778..fcc5f90d870e8 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 830711a46a35b..0d756f873e486 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c8016e41256d5..bd110218d34f7 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index c6f60c18804a4..124fc107cb8aa 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 30ff277e67c88..45a4c6dc98da0 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index ce35520a28609..12f900c91eb98 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 97abb6b2b63e0..f36674476770c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index 51744ece0412d..7dadbba58fd82 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 3faaf053634d6..7fcd7ee0d4547 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index b6c8456d06684..fd934b7726181 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.1.0-SNAPSHOT + 1.2.0-SNAPSHOT ../pom.xml From 21a1e1bb893512b2f68598ab0c0ec8c33e8d9909 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 6 Sep 2014 15:08:43 -0700 Subject: [PATCH 0612/1492] [SPARK-3273][SPARK-3301]We should read the version information from the same place Author: GuoQiang Li Closes #2175 from witgo/SPARK-3273 and squashes the following commits: cf9c65a [GuoQiang Li] We should read the version information from the same place 2a44e2f [GuoQiang Li] The spark version in the welcome message of pyspark is not correct --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 ++--- core/src/main/scala/org/apache/spark/package.scala | 1 + .../org/apache/spark/scheduler/EventLoggingListener.scala | 3 ++- .../apache/spark/scheduler/EventLoggingListenerSuite.scala | 5 +++-- python/pyspark/shell.py | 4 ++-- .../main/scala/org/apache/spark/repl/SparkILoopInit.scala | 6 ++++-- 6 files changed, 14 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6eaf6794764c7..24d1a8f9eceae 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -49,6 +49,7 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkD import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ +import org.apache.spark.SPARK_VERSION import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} @@ -825,7 +826,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** The version of Spark on which this application is running. */ - def version = SparkContext.SPARK_VERSION + def version = SPARK_VERSION /** * Return a map from the slave to the max memory available for caching and the remaining @@ -1297,8 +1298,6 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { - private[spark] val SPARK_VERSION = "1.2.0-SNAPSHOT" - private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 5cdbc306e56a0..e2fc9c649925e 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -44,4 +44,5 @@ package org.apache package object spark { // For package docs only + val SPARK_VERSION = "1.2.0-SNAPSHOT" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 4b99f630440ad..64b32ae0edaac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -29,6 +29,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec +import org.apache.spark.SPARK_VERSION import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} /** @@ -86,7 +87,7 @@ private[spark] class EventLoggingListener( sparkConf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) logger.newFile(COMPRESSION_CODEC_PREFIX + codec) } - logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) + logger.newFile(SPARK_VERSION_PREFIX + SPARK_VERSION) logger.newFile(LOG_PREFIX + logger.fileIndex) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index fead883793430..e5315bc93e217 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -28,6 +28,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec +import org.apache.spark.SPARK_VERSION import org.apache.spark.util.{JsonProtocol, Utils} import java.io.File @@ -196,7 +197,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { def assertInfoCorrect(info: EventLoggingInfo, loggerStopped: Boolean) { assert(info.logPaths.size > 0) - assert(info.sparkVersion === SparkContext.SPARK_VERSION) + assert(info.sparkVersion === SPARK_VERSION) assert(info.compressionCodec.isDefined === compressionCodec.isDefined) info.compressionCodec.foreach { codec => assert(compressionCodec.isDefined) @@ -381,7 +382,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { private def assertSparkVersionIsValid(logFiles: Array[FileStatus]) { val file = logFiles.map(_.getPath.getName).find(EventLoggingListener.isSparkVersionFile) assert(file.isDefined) - assert(EventLoggingListener.parseSparkVersion(file.get) === SparkContext.SPARK_VERSION) + assert(EventLoggingListener.parseSparkVersion(file.get) === SPARK_VERSION) } private def assertCompressionCodecIsValid(logFiles: Array[FileStatus], compressionCodec: String) { diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index fde3c29e5e790..89cf76920e353 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -49,9 +49,9 @@ ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /__ / .__/\_,_/_/ /_/\_\ version 1.0.0-SNAPSHOT + /__ / .__/\_,_/_/ /_/\_\ version %s /_/ -""") +""" % sc.version) print("Using Python version %s (%s, %s)" % ( platform.python_version(), platform.python_build()[0], diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 910b31d209e13..7667a9c11979e 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -14,6 +14,8 @@ import scala.reflect.internal.util.Position import scala.util.control.Exception.ignoring import scala.tools.nsc.util.stackTraceString +import org.apache.spark.SPARK_VERSION + /** * Machinery for the asynchronous initialization of the repl. */ @@ -26,9 +28,9 @@ trait SparkILoopInit { ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ - /___/ .__/\_,_/_/ /_/\_\ version 1.0.0-SNAPSHOT + /___/ .__/\_,_/_/ /_/\_\ version %s /_/ -""") +""".format(SPARK_VERSION)) import Properties._ val welcomeMsg = "Using Scala %s (%s, Java %s)".format( versionString, javaVmName, javaVersion) From 110fb8b24d2454ad7c979c3934dbed87650f17b8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 6 Sep 2014 16:12:29 -0700 Subject: [PATCH 0613/1492] [SPARK-2334] fix AttributeError when call PipelineRDD.id() The underline JavaRDD for PipelineRDD is created lazily, it's delayed until call _jrdd. The id of JavaRDD is cached as `_id`, it saves a RPC call in py4j for later calls. closes #1276 Author: Davies Liu Closes #2296 from davies/id and squashes the following commits: e197958 [Davies Liu] fix style 9721716 [Davies Liu] fix id of PipelineRDD --- python/pyspark/rdd.py | 6 ++++++ python/pyspark/sql.py | 9 +++++---- python/pyspark/tests.py | 9 +++++++++ 3 files changed, 20 insertions(+), 4 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index aa90297855c93..266090e3ae8f3 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2075,6 +2075,7 @@ def pipeline_func(split, iterator): self.ctx = prev.ctx self.prev = prev self._jrdd_val = None + self._id = None self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self._partitionFunc = prev._partitionFunc if self.preservesPartitioning else None @@ -2105,6 +2106,11 @@ def _jrdd(self): self._jrdd_val = python_rdd.asJavaRDD() return self._jrdd_val + def id(self): + if self._id is None: + self._id = self._jrdd.id() + return self._id + def _is_pipelinable(self): return not (self.is_cached or self.is_checkpointed) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 97a51b9f8a24f..004d4937cbe1c 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1525,7 +1525,7 @@ def __init__(self, jschema_rdd, sql_ctx): self.sql_ctx = sql_ctx self._sc = sql_ctx._sc self._jschema_rdd = jschema_rdd - + self._id = None self.is_cached = False self.is_checkpointed = False self.ctx = self.sql_ctx._sc @@ -1543,9 +1543,10 @@ def _jrdd(self): self._lazy_jrdd = self._jschema_rdd.javaToPython() return self._lazy_jrdd - @property - def _id(self): - return self._jrdd.id() + def id(self): + if self._id is None: + self._id = self._jrdd.id() + return self._id def saveAsParquetFile(self, path): """Save the contents as a Parquet file, preserving the schema. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 3e74799e82845..2ade15b35ab4e 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -281,6 +281,15 @@ def func(): class TestRDDFunctions(PySparkTestCase): + def test_id(self): + rdd = self.sc.parallelize(range(10)) + id = rdd.id() + self.assertEqual(id, rdd.id()) + rdd2 = rdd.map(str).filter(bool) + id2 = rdd2.id() + self.assertEqual(id + 1, id2) + self.assertEqual(id2, rdd2.id()) + def test_failed_sparkcontext_creation(self): # Regression test for SPARK-1550 self.sc.stop() From 3fb57a0ab3d76fda2301dbe9f2f3fa6743b4ed78 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 6 Sep 2014 19:06:30 -0700 Subject: [PATCH 0614/1492] [SPARK-3353] parent stage should have lower stage id. Previously parent stages had higher stage id, but parent stages are executed first. This pull request changes the behavior so parent stages would have lower stage id. For example, command: ```scala sc.parallelize(1 to 10).map(x=>(x,x)).reduceByKey(_+_).count ``` breaks down into 2 stages. The old web UI: ![screen shot 2014-09-04 at 12 42 44 am](https://cloud.githubusercontent.com/assets/323388/4146177/60fb4f42-3407-11e4-819f-853eb0e22b25.png) Web UI with this patch: ![screen shot 2014-09-04 at 12 44 55 am](https://cloud.githubusercontent.com/assets/323388/4146178/62e08e62-3407-11e4-867b-a36b10534464.png) Author: Reynold Xin Closes #2273 from rxin/lower-stage-id and squashes the following commits: abbb4c6 [Reynold Xin] Fixed SparkListenerSuite. 0e02379 [Reynold Xin] Updated DAGSchedulerSuite. 54ccea3 [Reynold Xin] [SPARK-3353] parent stage should have lower stage id. --- .../apache/spark/scheduler/DAGScheduler.scala | 4 +-- .../spark/scheduler/DAGSchedulerSuite.scala | 25 ++++++++++++------- .../spark/scheduler/SparkListenerSuite.scala | 2 +- 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2ccc27324ac8c..6fcf9e31543ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -241,9 +241,9 @@ class DAGScheduler( callSite: CallSite) : Stage = { + val parentStages = getParentStages(rdd, jobId) val id = nextStageId.getAndIncrement() - val stage = - new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) + val stage = new Stage(id, rdd, numTasks, shuffleDep, parentStages, jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) stage diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 0bb91febde9d7..aa73469b6acd8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} @@ -97,10 +98,12 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 val sparkListener = new SparkListener() { - val successfulStages = new HashSet[Int]() - val failedStages = new ArrayBuffer[Int]() + val successfulStages = new HashSet[Int] + val failedStages = new ArrayBuffer[Int] + val stageByOrderOfExecution = new ArrayBuffer[Int] override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageInfo = stageCompleted.stageInfo + stageByOrderOfExecution += stageInfo.stageId if (stageInfo.failureReason.isEmpty) { successfulStages += stageInfo.stageId } else { @@ -231,6 +234,13 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F runEvent(JobCancelled(jobId)) } + test("[SPARK-3353] parent stage should have lower stage id") { + sparkListener.stageByOrderOfExecution.clear() + sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() + assert(sparkListener.stageByOrderOfExecution.length === 2) + assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) + } + test("zero split job") { var numResults = 0 val fakeListener = new JobListener() { @@ -457,7 +467,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F null, null)) assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(CompletionEvent( @@ -515,8 +525,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F // Listener bus should get told about the map stage failing, but not the reduce stage // (since the reduce stage hasn't been started yet). assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - assert(sparkListener.failedStages.contains(1)) - assert(sparkListener.failedStages.size === 1) + assert(sparkListener.failedStages.toSet === Set(0)) assertDataStructuresEmpty } @@ -563,14 +572,12 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val stageFailureMessage = "Exception failure in map stage" failed(taskSets(0), stageFailureMessage) - assert(cancelledStages.contains(1)) + assert(cancelledStages.toSet === Set(0, 2)) // Make sure the listeners got told about both failed stages. assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.successfulStages.isEmpty) - assert(sparkListener.failedStages.contains(1)) - assert(sparkListener.failedStages.contains(3)) - assert(sparkListener.failedStages.size === 2) + assert(sparkListener.failedStages.toSet === Set(0, 2)) assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 3b0b8e2f68c97..ab35e8edc4ebf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -180,7 +180,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers rdd3.count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {2} // Shuffle map stage + result stage - val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get + val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 3).get stageInfo3.rddInfos.size should be {1} // ShuffledRDD stageInfo3.rddInfos.forall(_.numPartitions == 4) should be {true} stageInfo3.rddInfos.exists(_.name == "Trois") should be {true} From 6754570d83044c4fbaf0d2ac2378a0e081a93629 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Sun, 7 Sep 2014 17:57:59 -0700 Subject: [PATCH 0615/1492] [SPARK-3394] [SQL] Fix crash in TakeOrdered when limit is 0 This resolves https://issues.apache.org/jira/browse/SPARK-3394 Author: Eric Liang Closes #2264 from ericl/spark-3394 and squashes the following commits: c87355b [Eric Liang] refactor bfb6140 [Eric Liang] change RDD takeOrdered instead 7a51528 [Eric Liang] fix takeordered when limit = 0 --- .../main/scala/org/apache/spark/rdd/RDD.scala | 22 +++++++++++-------- .../scala/org/apache/spark/rdd/RDDSuite.scala | 7 ++++++ 2 files changed, 20 insertions(+), 9 deletions(-) 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 1cf55e86f6c81..a9b905b0d1a63 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1127,15 +1127,19 @@ abstract class RDD[T: ClassTag]( * @return an array of top elements */ def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { - mapPartitions { items => - // Priority keeps the largest elements, so let's reverse the ordering. - val queue = new BoundedPriorityQueue[T](num)(ord.reverse) - queue ++= util.collection.Utils.takeOrdered(items, num)(ord) - Iterator.single(queue) - }.reduce { (queue1, queue2) => - queue1 ++= queue2 - queue1 - }.toArray.sorted(ord) + if (num == 0) { + Array.empty + } else { + mapPartitions { items => + // Priority keeps the largest elements, so let's reverse the ordering. + val queue = new BoundedPriorityQueue[T](num)(ord.reverse) + queue ++= util.collection.Utils.takeOrdered(items, num)(ord) + Iterator.single(queue) + }.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord) + } } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 926d4fecb5b91..499dcda3dae8f 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -521,6 +521,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sortedLowerK === Array(1, 2, 3, 4, 5)) } + test("takeOrdered with limit 0") { + val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) + val rdd = sc.makeRDD(nums, 2) + val sortedLowerK = rdd.takeOrdered(0) + assert(sortedLowerK.size === 0) + } + test("takeOrdered with custom ordering") { val nums = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) implicit val ord = implicitly[Ordering[Int]].reverse From 39db1bfdab434c867044ad4c70fe93a96fb287ad Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 7 Sep 2014 21:34:46 -0400 Subject: [PATCH 0616/1492] [SQL] Update SQL Programming Guide Author: Michael Armbrust Author: Yin Huai Closes #2258 from marmbrus/sqlDocUpdate and squashes the following commits: f3d450b [Michael Armbrust] fix brackets bea3bfa [Michael Armbrust] Davies suggestions 3a29fe2 [Michael Armbrust] tighten visibility a71aa36 [Michael Armbrust] Draft of doc updates 52932c0 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into sqlDocUpdate 1e8c849 [Yin Huai] Update the example used for applySchema. 9457c39 [Yin Huai] Update doc. 31ba240 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeDoc 29bc668 [Yin Huai] Draft doc for data type and schema APIs. --- docs/sql-programming-guide.md | 952 ++++++++++++++++-- .../scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../apache/spark/sql/UdfRegistration.scala | 2 +- .../columnar/InMemoryColumnarTableScan.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 2 +- .../spark/sql/parquet/ParquetFilters.scala | 2 +- .../server/SparkSQLOperationManager.scala | 4 +- 7 files changed, 865 insertions(+), 101 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 8f7fb5431cfb6..1814fef465cac 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -68,6 +68,16 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD {% endhighlight %} +In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict +super set of the functionality provided by the basic SQLContext. Additional features include +the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the +ability to read data from Hive tables. To use a HiveContext, you do not need to have an +existing hive setup, and all of the data sources available to a SQLContext are still available. +HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using HiveContext +is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to +feature parity with a HiveContext. +
    @@ -81,6 +91,16 @@ JavaSparkContext sc = ...; // An existing JavaSparkContext. JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); {% endhighlight %} +In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict +super set of the functionality provided by the basic SQLContext. Additional features include +the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the +ability to read data from Hive tables. To use a HiveContext, you do not need to have an +existing hive setup, and all of the data sources available to a SQLContext are still available. +HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using HiveContext +is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to +feature parity with a HiveContext. +
    @@ -94,36 +114,52 @@ from pyspark.sql import SQLContext sqlContext = SQLContext(sc) {% endhighlight %} -
    +In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict +super set of the functionality provided by the basic SQLContext. Additional features include +the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the +ability to read data from Hive tables. To use a HiveContext, you do not need to have an +existing hive setup, and all of the data sources available to a SQLContext are still available. +HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default +Spark build. If these dependencies are not a problem for your application then using HiveContext +is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to +feature parity with a HiveContext. -# Data Sources - -
    -
    -Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. -Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources.
    -
    -Spark SQL supports operating on a variety of data sources through the `JavaSchemaRDD` interface. -Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources. -
    +The specific variant of SQL that is used to parse queries can also be selected using the +`spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on +a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect +available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the +default is "hiveql", though "sql" is also available. Since the HiveQL parser is much more complete, + this is recommended for most use cases. + +# Data Sources -
    Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. -Once a dataset has been loaded, it can be registered as a table and even joined with data from other sources. -
    -
    +A SchemaRDD can be operated on as normal RDDs and can also be registered as a temporary table. +Registering a SchemaRDD as a table allows you to run SQL queries over its data. This section +describes the various methods for loading data into a SchemaRDD. ## RDDs +Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. The first +method uses reflection to infer the schema of an RDD that contains specific types of objects. This +reflection based approach leads to more concise code and works well went the schema is known ahead +of time, while you are writing your Spark application. + +The second method for creating SchemaRDDs is through a programmatic interface that allows you to +construct a schema and then apply it to and existing RDD. While this method is more verbose, it allows +you to construct SchemaRDDs when the columns and their types are not known until runtime. + +### Inferring the Schema Using Reflection
    -One type of table that is supported by Spark SQL is an RDD of Scala case classes. The case class +The Scala interaface for Spark SQL supports automatically converting an RDD containing case classes +to a SchemaRDD. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be @@ -156,8 +192,9 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
    -One type of table that is supported by Spark SQL is an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly). The BeanInfo -defines the schema of the table. Currently, Spark SQL does not support JavaBeans that contain +Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) +into a Schema RDD. The BeanInfo, obtained using reflection, defines the schema of the table. +Currently, Spark SQL does not support JavaBeans that contain nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a class that implements Serializable and has getters and setters for all of its fields. @@ -192,7 +229,7 @@ for the JavaBean. {% highlight java %} // sc is an existing JavaSparkContext. -JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc) +JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); // Load a text file and convert each line to a JavaBean. JavaRDD people = sc.textFile("examples/src/main/resources/people.txt").map( @@ -229,24 +266,24 @@ List teenagerNames = teenagers.map(new Function() {
    -One type of table that is supported by Spark SQL is an RDD of dictionaries. The keys of the -dictionary define the columns names of the table, and the types are inferred by looking at the first -row. Any RDD of dictionaries can converted to a SchemaRDD and then registered as a table. Tables -can be used in subsequent SQL statements. +Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes . Rows are constructed by passing a list of +key/value pairs as kwargs to the Row class. The keys of this list define the columns names of the table, +and the types are inferred by looking at the first row. Since we currently only look at the first +row, it is important that there is no missing data in the first row of the RDD. In future version we +plan to more completely infer the schema by looking at more data, similar to the inference that is +performed on JSON files. {% highlight python %} # sc is an existing SparkContext. -from pyspark.sql import SQLContext +from pyspark.sql import SQLContext, Row sqlContext = SQLContext(sc) # Load a text file and convert each line to a dictionary. lines = sc.textFile("examples/src/main/resources/people.txt") parts = lines.map(lambda l: l.split(",")) -people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) +people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) # Infer the schema, and register the SchemaRDD as a table. -# In future versions of PySpark we would like to add support for registering RDDs with other -# datatypes as tables schemaPeople = sqlContext.inferSchema(people) schemaPeople.registerTempTable("people") @@ -263,15 +300,191 @@ for teenName in teenNames.collect():
    -**Note that Spark SQL currently uses a very basic SQL parser.** -Users that want a more complete dialect of SQL should look at the HiveQL support provided by -`HiveContext`. +### Programmatically Specifying the Schema + +
    + +
    + +In cases that case classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string or a text dataset will be parsed +and fields will be projected differently for different users), +a `SchemaRDD` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in the step 1. +3. Apply the schema to the RDD of `Row`s via `applySchema` method provided +by `SQLContext`. + +For example: +{% highlight scala %} +// sc is an existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) + +// Create an RDD +val people = sc.textFile("examples/src/main/resources/people.txt") + +// The schema is encoded in a string +val schemaString = "name age" + +// Import Spark SQL data types and Row. +import org.apache.spark.sql._ + +// Generate the schema based on the string of schema +val schema = + StructType( + schemaString.split(" ").map(fieldName => StructField(fieldName, StringType, true))) + +// Convert records of the RDD (people) to Rows. +val rowRDD = people.map(_.split(",")).map(p => Row(p(0), p(1).trim)) + +// Apply the schema to the RDD. +val peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema) + +// Register the SchemaRDD as a table. +peopleSchemaRDD.registerTempTable("people") + +// SQL statements can be run by using the sql methods provided by sqlContext. +val results = sqlContext.sql("SELECT name FROM people") + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +results.map(t => "Name: " + t(0)).collect().foreach(println) +{% endhighlight %} + + +
    + +
    + +In cases that JavaBean classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string or a text dataset will be parsed and +fields will be projected differently for different users), +a `SchemaRDD` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in the step 1. +3. Apply the schema to the RDD of `Row`s via `applySchema` method provided +by `JavaSQLContext`. + +For example: +{% highlight java %} +// Import factory methods provided by DataType. +import org.apache.spark.sql.api.java.DataType +// Import StructType and StructField +import org.apache.spark.sql.api.java.StructType +import org.apache.spark.sql.api.java.StructField +// Import Row. +import org.apache.spark.sql.api.java.Row + +// sc is an existing JavaSparkContext. +JavaSQLContext sqlContext = new org.apache.spark.sql.api.java.JavaSQLContext(sc); + +// Load a text file and convert each line to a JavaBean. +JavaRDD people = sc.textFile("examples/src/main/resources/people.txt"); + +// The schema is encoded in a string +String schemaString = "name age"; + +// Generate the schema based on the string of schema +List fields = new ArrayList(); +for (String fieldName: schemaString.split(" ")) { + fields.add(DataType.createStructField(fieldName, DataType.StringType, true)); +} +StructType schema = DataType.createStructType(fields); + +// Convert records of the RDD (people) to Rows. +JavaRDD rowRDD = people.map( + new Function() { + public Row call(String record) throws Exception { + String[] fields = record.split(","); + return Row.create(fields[0], fields[1].trim()); + } + }); + +// Apply the schema to the RDD. +JavaSchemaRDD peopleSchemaRDD = sqlContext.applySchema(rowRDD, schema); + +// Register the SchemaRDD as a table. +peopleSchemaRDD.registerTempTable("people"); + +// SQL can be run over RDDs that have been registered as tables. +JavaSchemaRDD results = sqlContext.sql("SELECT name FROM people"); + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +List names = results.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } +}).collect(); + +{% endhighlight %} + +
    + +
    + +For some cases (for example, the structure of records is encoded in a string or +a text dataset will be parsed and fields will be projected differently for +different users), it is desired to create `SchemaRDD` with a programmatically way. +It can be done with three steps. + +1. Create an RDD of tuples or lists from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +tuples or lists in the RDD created in the step 1. +3. Apply the schema to the RDD via `applySchema` method provided by `SQLContext`. + +For example: +{% highlight python %} +# Import SQLContext and data types +from pyspark.sql import * + +# sc is an existing SparkContext. +sqlContext = SQLContext(sc) + +# Load a text file and convert each line to a tuple. +lines = sc.textFile("examples/src/main/resources/people.txt") +parts = lines.map(lambda l: l.split(",")) +people = parts.map(lambda p: (p[0], p[1].strip())) + +# The schema is encoded in a string. +schemaString = "name age" + +fields = [StructField(field_name, StringType(), True) for field_name in schemaString.split()] +schema = StructType(fields) + +# Apply the schema to the RDD. +schemaPeople = sqlContext.applySchema(people, schema) + +# Register the SchemaRDD as a table. +schemaPeople.registerTempTable("people") + +# SQL can be run over SchemaRDDs that have been registered as a table. +results = sqlContext.sql("SELECT name FROM people") + +# The results of SQL queries are RDDs and support all the normal RDD operations. +names = results.map(lambda p: "Name: " + p.name) +for name in names.collect(): + print name +{% endhighlight %} + + +
    + +
    ## Parquet Files [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. Using the data from the above example: +of the original data. + +### Loading Data Programmatically + +Using the data from the above example:
    @@ -349,7 +562,40 @@ for teenName in teenNames.collect():
    -
    +
    + +### Configuration + +Configuration of parquet can be done using the `setConf` method on SQLContext or by running +`SET key=value` commands using SQL. + +
    SourceArtifact
    Kafka spark-streaming-kafka_{{site.SCALA_BINARY_VERSION}}
    Flume spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}}
    Kinesis
    spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} [Apache Software License]
    Twitter spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}
    ZeroMQ spark-streaming-zeromq_{{site.SCALA_BINARY_VERSION}}
    MQTT spark-streaming-mqtt_{{site.SCALA_BINARY_VERSION}}
    + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.parquet.binaryAsStringfalse + Some other parquet producing systems, in particular Impala and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the parquet schema. This + flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. +
    spark.sql.parquet.cacheMetadatafalse + Turns on caching of parquet schema metadata. Can speed up querying +
    spark.sql.parquet.compression.codecsnappy + Sets the compression codec use when writing parquet files. Acceptable values include: + uncompressed, snappy, gzip, lzo. +
    ## JSON Datasets
    @@ -493,13 +739,13 @@ directory. {% highlight scala %} // sc is an existing SparkContext. -val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) +val sqlContext = new org.apache.spark.sql.hive.HiveContext(sc) -hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -hiveContext.sql("FROM src SELECT key, value").collect().foreach(println) +sqlContext.sql("FROM src SELECT key, value").collect().foreach(println) {% endhighlight %}
    @@ -513,13 +759,13 @@ expressed in HiveQL. {% highlight java %} // sc is an existing JavaSparkContext. -JavaHiveContext hiveContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); +JavaHiveContext sqlContext = new org.apache.spark.sql.hive.api.java.HiveContext(sc); -hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); -hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); +sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); // Queries are expressed in HiveQL. -Row[] results = hiveContext.sql("FROM src SELECT key, value").collect(); +Row[] results = sqlContext.sql("FROM src SELECT key, value").collect(); {% endhighlight %} @@ -535,44 +781,97 @@ expressed in HiveQL. {% highlight python %} # sc is an existing SparkContext. from pyspark.sql import HiveContext -hiveContext = HiveContext(sc) +sqlContext = HiveContext(sc) -hiveContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -hiveContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +sqlContext.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sqlContext.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") # Queries can be expressed in HiveQL. -results = hiveContext.sql("FROM src SELECT key, value").collect() +results = sqlContext.sql("FROM src SELECT key, value").collect() {% endhighlight %}
    -# Writing Language-Integrated Relational Queries +# Performance Tuning -**Language-Integrated queries are currently only supported in Scala.** - -Spark SQL also supports a domain specific language for writing queries. Once again, -using the data from the above examples: +For some workloads it is possible to improve performance by either caching data in memory, or by +turning on some experimental options. -{% highlight scala %} -// sc is an existing SparkContext. -val sqlContext = new org.apache.spark.sql.SQLContext(sc) -// Importing the SQL context gives access to all the public SQL functions and implicit conversions. -import sqlContext._ -val people: RDD[Person] = ... // An RDD of case class objects, from the first example. +## Caching Data In Memory -// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' -val teenagers = people.where('age >= 10).where('age <= 19).select('name) -teenagers.map(t => "Name: " + t(0)).collect().foreach(println) -{% endhighlight %} +Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. +Then Spark SQL will scan only required columns and will automatically tune compression to minimize +memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. -The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers -prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are -evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). +Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in +in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to +cache tables. - +Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.inMemoryColumnarStorage.compressedfalse + When set to true Spark SQL will automatically select a compression codec for each column based + on statistics of the data. +
    spark.sql.inMemoryColumnarStorage.batchSize1000 + Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization + and compression, but risk OOMs when caching data. +
    + +## Other Configuration + +The following options can also be used to tune the performance of query execution. It is possible +that these options will be deprecated in future release as more optimizations are performed automatically. + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.autoBroadcastJoinThresholdfalse + Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when + performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently + statistics are only supported for Hive Metastore tables where the command + `ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan` has been run. +
    spark.sql.codegenfalse + When true, code will be dynamically generated at runtime for expression evaluation in a specific + query. For some queries with complicated expression this option can lead to significant speed-ups. + However, for simple queries this can actually slow down query execution. +
    spark.sql.shuffle.partitions200 + Configures the number of partitions to use when shuffling data for joins or aggregations. +
    + +# Other SQL Interfaces + +Spark SQL also supports interfaces for running SQL queries directly without the need to write any +code. ## Running the Thrift JDBC server @@ -602,14 +901,28 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. You may also use the beeline script comes with Hive. +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available +options. + +# Compatibility with Other Systems + +## Migration Guide for Shark Users To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, users can set the `spark.sql.thriftserver.scheduler.pool` variable: SET spark.sql.thriftserver.scheduler.pool=accounting; -### Migration Guide for Shark Users - -#### Reducer number +### Reducer number In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value @@ -625,7 +938,7 @@ You may also put this property in `hive-site.xml` to override the default value. For now, the `mapred.reduce.tasks` property is still recognized, and is converted to `spark.sql.shuffle.partitions` automatically. -#### Caching +### Caching The `shark.cache` table property no longer exists, and tables whose name end with `_cached` are no longer automatically cached. Instead, we provide `CACHE TABLE` and `UNCACHE TABLE` statements to @@ -634,9 +947,9 @@ let user control table caching explicitly: CACHE TABLE logs_last_month; UNCACHE TABLE logs_last_month; -**NOTE:** `CACHE TABLE tbl` is lazy, it only marks table `tbl` as "need to by cached if necessary", -but doesn't actually cache it until a query that touches `tbl` is executed. To force the table to be -cached, you may simply count the table immediately after executing `CACHE TABLE`: +**NOTE:** `CACHE TABLE tbl` is lazy, similar to `.cache` on an RDD. This command only marks `tbl` to ensure that +partitions are cached when calculated but doesn't actually cache it until a query that touches `tbl` is executed. +To force the table to be cached, you may simply count the table immediately after executing `CACHE TABLE`: CACHE TABLE logs_last_month; SELECT COUNT(1) FROM logs_last_month; @@ -647,15 +960,18 @@ Several caching related features are not supported yet: * RDD reloading * In-memory cache write through policy -### Compatibility with Apache Hive +## Compatibility with Apache Hive + +Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. Currently Spark +SQL is based on Hive 0.12.0. #### Deploying in Existing Hive Warehouses -Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive installations. You do not need to modify your existing Hive Metastore or change the data placement or partitioning of your tables. -#### Supported Hive Features +### Supported Hive Features Spark SQL supports the vast majority of Hive features, such as: @@ -705,13 +1021,14 @@ Spark SQL supports the vast majority of Hive features, such as: * `MAP<>` * `STRUCT<>` -#### Unsupported Hive Functionality +### Unsupported Hive Functionality Below is a list of Hive features that we don't support yet. Most of these features are rarely used in Hive deployments. **Major Hive Features** +* Spark SQL does not currently support inserting to tables using dynamic partitioning. * Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL doesn't support buckets yet. @@ -721,11 +1038,11 @@ in Hive deployments. have the same input format. * Non-equi outer join: For the uncommon use case of using outer joins with non-equi join conditions (e.g. condition "`key < 10`"), Spark SQL will output wrong result for the `NULL` tuple. -* `UNIONTYPE` +* `UNION` type and `DATE` type * Unique join * Single query multi insert * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment. + the moment and only supports populating the sizeInBytes field of the hive metastore. **Hive Input/Output Formats** @@ -735,7 +1052,7 @@ in Hive deployments. **Hive Optimizations** A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -not necessary due to Spark SQL's in-memory computational model. Others are slotted for future +less important due to Spark SQL's in-memory computational model. Others are slotted for future releases of Spark SQL. * Block level bitmap indexes and virtual columns (used to build indexes) @@ -743,8 +1060,7 @@ releases of Spark SQL. Hive automatically converts the join into a map join. We are adding this auto conversion in the next release. * Automatically determine the number of reducers for joins and groupbys: Currently in Spark SQL, you - need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". We are going to add auto-setting of parallelism in the - next release. + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". * Meta-data only query: For queries that can be answered by using only meta data, Spark SQL still launches tasks to compute the result. * Skew data flag: Spark SQL does not follow the skew data flags in Hive. @@ -753,25 +1069,471 @@ releases of Spark SQL. Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS metadata. Spark SQL does not support that. -## Running the Spark SQL CLI +# Writing Language-Integrated Relational Queries -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. +**Language-Integrated queries are experimental and currently only supported in Scala.** -To start the Spark SQL CLI, run the following in the Spark directory: +Spark SQL also supports a domain specific language for writing queries. Once again, +using the data from the above examples: - ./bin/spark-sql +{% highlight scala %} +// sc is an existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) +// Importing the SQL context gives access to all the public SQL functions and implicit conversions. +import sqlContext._ +val people: RDD[Person] = ... // An RDD of case class objects, from the first example. -Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available -options. +// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' +val teenagers = people.where('age >= 10).where('age <= 19).select('name) +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) +{% endhighlight %} -# Cached tables +The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers +prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are +evaluated by the SQL execution engine. A full list of the functions supported can be found in the +[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). -Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. -Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. + + +# Spark SQL DataType Reference + +* Numeric types + - `ByteType`: Represents 1-byte signed integer numbers. + The range of numbers is from `-128` to `127`. + - `ShortType`: Represents 2-byte signed integer numbers. + The range of numbers is from `-32768` to `32767`. + - `IntegerType`: Represents 4-byte signed integer numbers. + The range of numbers is from `-2147483648` to `2147483647`. + - `LongType`: Represents 8-byte signed integer numbers. + The range of numbers is from `-9223372036854775808` to `9223372036854775807`. + - `FloatType`: Represents 4-byte single-precision floating point numbers. + - `DoubleType`: Represents 8-byte double-precision floating point numbers. + - `DecimalType`: +* String type + - `StringType`: Represents character string values. +* Binary type + - `BinaryType`: Represents byte sequence values. +* Boolean type + - `BooleanType`: Represents boolean values. +* Datetime type + - `TimestampType`: Represents values comprising values of fields year, month, day, + hour, minute, and second. +* Complex types + - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of + elements with the type of `elementType`. `containsNull` is used to indicate if + elements in a `ArrayType` value can have `null` values. + - `MapType(keyType, valueType, valueContainsNull)`: + Represents values comprising a set of key-value pairs. The data type of keys are + described by `keyType` and the data type of values are described by `valueType`. + For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` + is used to indicate if values of a `MapType` value can have `null` values. + - `StructType(fields)`: Represents values with the structure described by + a sequence of `StructField`s (`fields`). + * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. + The name of a field is indicated by `name`. The data type of a field is indicated + by `dataType`. `nullable` is used to indicate if values of this fields can have + `null` values. + +
    +
    + +All data types of Spark SQL are located in the package `org.apache.spark.sql`. +You can access them by doing +{% highlight scala %} +import org.apache.spark.sql._ +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in ScalaAPI to access or create a data type
    ByteType Byte + ByteType +
    ShortType Short + ShortType +
    IntegerType Int + IntegerType +
    LongType Long + LongType +
    FloatType Float + FloatType +
    DoubleType Double + DoubleType +
    DecimalType scala.math.sql.BigDecimal + DecimalType +
    StringType String + StringType +
    BinaryType Array[Byte] + BinaryType +
    BooleanType Boolean + BooleanType +
    TimestampType java.sql.Timestamp + TimestampType +
    ArrayType scala.collection.Seq + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is false. +
    MapType scala.collection.Map + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is true. +
    StructType org.apache.spark.sql.Row + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Scala of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, nullable) +
    + +
    + +
    + +All data types of Spark SQL are located in the package of +`org.apache.spark.sql.api.java`. To access or create a data type, +please use factory methods provided in +`org.apache.spark.sql.api.java.DataType`. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in JavaAPI to access or create a data type
    ByteType byte or Byte + DataType.ByteType +
    ShortType short or Short + DataType.ShortType +
    IntegerType int or Integer + DataType.IntegerType +
    LongType long or Long + DataType.LongType +
    FloatType float or Float + DataType.FloatType +
    DoubleType double or Double + DataType.DoubleType +
    DecimalType java.math.BigDecimal + DataType.DecimalType +
    StringType String + DataType.StringType +
    BinaryType byte[] + DataType.BinaryType +
    BooleanType boolean or Boolean + DataType.BooleanType +
    TimestampType java.sql.Timestamp + DataType.TimestampType +
    ArrayType java.util.List + DataType.createArrayType(elementType)
    + Note: The value of containsNull will be false
    + DataType.createArrayType(elementType, containsNull). +
    MapType java.util.Map + DataType.createMapType(keyType, valueType)
    + Note: The value of valueContainsNull will be true.
    + DataType.createMapType(keyType, valueType, valueContainsNull)
    +
    StructType org.apache.spark.sql.api.java + DataType.createStructType(fields)
    + Note: fields is a List or an array of StructFields. + Also, two fields with the same name are not allowed. +
    StructField The value type in Java of the data type of this field + (For example, int for a StructField with the data type IntegerType) + DataType.createStructField(name, dataType, nullable) +
    + +
    + +
    + +All data types of Spark SQL are located in the package of `pyspark.sql`. +You can access them by doing +{% highlight python %} +from pyspark.sql import * +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in PythonAPI to access or create a data type
    ByteType + int or long
    + Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
    + ByteType() +
    ShortType + int or long
    + Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
    + ShortType() +
    IntegerType int or long + IntegerType() +
    LongType + long
    + Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
    + LongType() +
    FloatType + float
    + Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
    + FloatType() +
    DoubleType float + DoubleType() +
    DecimalType decimal.Decimal + DecimalType() +
    StringType string + StringType() +
    BinaryType bytearray + BinaryType() +
    BooleanType bool + BooleanType() +
    TimestampType datetime.datetime + TimestampType() +
    ArrayType list, tuple, or array + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is False. +
    MapType dict + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is True. +
    StructType list or tuple + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Python of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, nullable) +
    + +
    + +
    -Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in -in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to -cache tables. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 4137ac7663739..f6f4cf3b80d41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -53,7 +53,7 @@ private[spark] object SQLConf { * * SQLConf is thread-safe (internally synchronized, so safe to be used in multiple threads). */ -trait SQLConf { +private[sql] trait SQLConf { import SQLConf._ /** Only low degree of contention is expected for conf, thus NOT using ConcurrentHashMap. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 0ea1105f082a4..595b4aa36eae3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -30,7 +30,7 @@ import scala.reflect.runtime.universe.{TypeTag, typeTag} /** * Functions for registering scala lambda functions as UDFs in a SQLContext. */ -protected[sql] trait UDFRegistration { +private[sql] trait UDFRegistration { self: SQLContext => private[spark] def registerPython( 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 dc668e7dc934c..6eab2f23c18e1 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{LeafNode, SparkPlan} -object InMemoryRelation { +private[sql] object InMemoryRelation { def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = new InMemoryRelation(child.output, useCompression, batchSize, child)() } 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 9fd6aed402838..2fc7e1cf23ab7 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 @@ -382,7 +382,7 @@ private[parquet] class CatalystPrimitiveConverter( parent.updateLong(fieldIndex, value) } -object CatalystArrayConverter { +private[parquet] object CatalystArrayConverter { val INITIAL_ARRAY_SIZE = 20 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index fe28e0d7269e0..7c83f1cad7d71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkSqlSerializer -object ParquetFilters { +private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" // set this to false if pushdown should be disabled val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.hints.parquetFilterPushdown" diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index f12b5a69a09f7..bd3f68d92d8c7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -39,7 +39,9 @@ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. */ -class SparkSQLOperationManager(hiveContext: HiveContext) extends OperationManager with Logging { +private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) + extends OperationManager with Logging { + val handleToOperation = ReflectionUtils .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") From e2614038e78f4693fafedeee15b6fdf0ea1be473 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Sep 2014 18:42:24 -0700 Subject: [PATCH 0617/1492] [SPARK-3408] Fixed Limit operator so it works with sort-based shuffle. Author: Reynold Xin Closes #2281 from rxin/sql-limit-sort and squashes the following commits: 1ef7780 [Reynold Xin] [SPARK-3408] Fixed Limit operator so it works with sort-based shuffle. --- .../spark/sql/execution/basicOperators.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 4abda21ffec96..47bff0c730b8a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -20,10 +20,10 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.{HashPartitioner, SparkConf} import org.apache.spark.rdd.{RDD, ShuffledRDD} -import org.apache.spark.sql.SQLContext +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ @@ -96,6 +96,9 @@ case class Limit(limit: Int, child: SparkPlan) // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again + /** We must copy rows when sort based shuffle is on */ + private def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] + override def output = child.output /** @@ -143,9 +146,15 @@ case class Limit(limit: Int, child: SparkPlan) } override def execute() = { - val rdd = child.execute().mapPartitions { iter => - val mutablePair = new MutablePair[Boolean, Row]() - iter.take(limit).map(row => mutablePair.update(false, row)) + val rdd: RDD[_ <: Product2[Boolean, Row]] = if (sortBasedShuffleOn) { + child.execute().mapPartitions { iter => + iter.take(limit).map(row => (false, row.copy())) + } + } else { + child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Boolean, Row]() + iter.take(limit).map(row => mutablePair.update(false, row)) + } } val part = new HashPartitioner(1) val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) From ecfa76cdfe846c75e1b7ebc556167e46963289c5 Mon Sep 17 00:00:00 2001 From: Ward Viaene Date: Sun, 7 Sep 2014 18:54:36 -0700 Subject: [PATCH 0618/1492] [SPARK-3415] [PySpark] removes SerializingAdapter code This code removes the SerializingAdapter code that was copied from PiCloud Author: Ward Viaene Closes #2287 from wardviaene/feature/pythonsys and squashes the following commits: 5f0d426 [Ward Viaene] SPARK-3415: modified test class to do dump and load 5f5d559 [Ward Viaene] SPARK-3415: modified test class name and call cloudpickle.dumps instead using StringIO afc4a9a [Ward Viaene] SPARK-3415: added newlines to pass lint aaf10b7 [Ward Viaene] SPARK-3415: removed references to SerializingAdapter and rewrote test 65ffeff [Ward Viaene] removed duplicate test a958866 [Ward Viaene] SPARK-3415: test script e263bf5 [Ward Viaene] SPARK-3415: removes legacy SerializingAdapter code --- python/pyspark/cloudpickle.py | 6 +----- python/pyspark/tests.py | 11 +++++++++++ 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 68062483dedaa..80e51d1a583a0 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -657,7 +657,6 @@ def save_partial(self, obj): def save_file(self, obj): """Save a file""" import StringIO as pystringIO #we can't use cStringIO as it lacks the name attribute - from ..transport.adapter import SerializingAdapter if not hasattr(obj, 'name') or not hasattr(obj, 'mode'): raise pickle.PicklingError("Cannot pickle files that do not map to an actual file") @@ -691,13 +690,10 @@ def save_file(self, obj): tmpfile.close() if tst != '': raise pickle.PicklingError("Cannot pickle file %s as it does not appear to map to a physical, real file" % name) - elif fsize > SerializingAdapter.max_transmit_data: - raise pickle.PicklingError("Cannot pickle file %s as it exceeds cloudconf.py's max_transmit_data of %d" % - (name,SerializingAdapter.max_transmit_data)) else: try: tmpfile = file(name) - contents = tmpfile.read(SerializingAdapter.max_transmit_data) + contents = tmpfile.read() tmpfile.close() except IOError: raise pickle.PicklingError("Cannot pickle file %s as it cannot be read" % name) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 2ade15b35ab4e..9fbeb36f4f1dd 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -169,6 +169,17 @@ def test_namedtuple(self): self.assertEquals(p1, p2) +# Regression test for SPARK-3415 +class CloudPickleTest(unittest.TestCase): + def test_pickling_file_handles(self): + from pyspark.cloudpickle import dumps + from StringIO import StringIO + from pickle import load + out1 = sys.stderr + out2 = load(StringIO(dumps(out1))) + self.assertEquals(out1, out2) + + class PySparkTestCase(unittest.TestCase): def setUp(self): From 9d69a782bd2fc45193f269d8d8434795ea1580a4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 7 Sep 2014 20:38:32 -0700 Subject: [PATCH 0619/1492] Fixed typos in make-distribution.sh `hadoop.version` and `yarn.version` are properties rather then profiles, should use `-D` instead of `-P`. /cc pwendell Author: Cheng Lian Closes #2121 from liancheng/fix-make-dist and squashes the following commits: 4c49158 [Cheng Lian] Also mentions Hadoop version related Maven profiles ed5b42a [Cheng Lian] Fixed typos in make-distribution.sh --- make-distribution.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index f030d3f430581..14aed4a4b655b 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -50,7 +50,8 @@ while (( "$#" )); do case $1 in --hadoop) echo "Error: '--hadoop' is no longer supported:" - echo "Error: use Maven options -Phadoop.version and -Pyarn.version" + echo "Error: use Maven profiles and options -Dhadoop.version and -Dyarn.version instead." + echo "Error: Related profiles include hadoop-0.23, hdaoop-2.2, hadoop-2.3 and hadoop-2.4." exit_with_usage ;; --with-yarn) From 4ba2673569f8c6da7f7348977f52f98f40dfbfec Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 7 Sep 2014 20:39:53 -0700 Subject: [PATCH 0620/1492] [HOTFIX] Fix broken Mima tests on the master branch By merging #2268, which bumped the Spark version to 1.2.0-SNAPSHOT, I inadvertently broke the Mima binary compatibility tests. The issue is that we were comparing 1.2.0-SNAPSHOT against Spark 1.0.0 without using any Mima excludes. The right long-term fix for this is probably to publish nightly snapshots on Maven central and change the master branch to test binary compatibility against the current release candidate branch's snapshots until that release is finalized. As a short-term fix until 1.1.0 is published on Maven central, I've configured the build to test the master branch for binary compatibility against the 1.1.0-RC4 jars. I'll loop back and remove the Apache staging repo as soon as 1.1.0 final is available. Author: Josh Rosen Closes #2315 from JoshRosen/mima-fix and squashes the following commits: 776bc2c [Josh Rosen] Add two excludes to workaround Mima annotation issues. ec90e21 [Josh Rosen] Add deploy and graphx to 1.2 MiMa excludes. 57569be [Josh Rosen] Fix MiMa tests in master branch; test against 1.1.0 RC. --- pom.xml | 12 ++++++++++++ project/MimaBuild.scala | 2 +- project/MimaExcludes.scala | 12 ++++++++++++ 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 1efa9045208fd..d05190512f742 100644 --- a/pom.xml +++ b/pom.xml @@ -221,6 +221,18 @@ false + + + spark-staging-1030 + Spark 1.1.0 Staging (1030) + https://repository.apache.org/content/repositories/orgapachespark-1030/ + + true + + + false + + diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 034ba6a7bf50f..0f5d71afcf616 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -85,7 +85,7 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" - val previousSparkVersion = "1.0.0" + val previousSparkVersion = "1.1.0" val fullId = "spark-" + projectRef.project + "_2.10" mimaDefaultSettings ++ Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 855d5cc8cf3fd..46b78bd5c7061 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -33,6 +33,18 @@ import com.typesafe.tools.mima.core._ object MimaExcludes { def excludes(version: String) = version match { + case v if v.startsWith("1.2") => + Seq( + MimaBuild.excludeSparkPackage("deploy"), + MimaBuild.excludeSparkPackage("graphx") + ) ++ + // This is @DeveloperAPI, but Mima still gives false-positives: + MimaBuild.excludeSparkClass("scheduler.SparkListenerApplicationStart") ++ + Seq( + // This is @Experimental, but Mima still gives false-positives: + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.foreachAsync") + ) case v if v.startsWith("1.1") => Seq( MimaBuild.excludeSparkPackage("deploy"), From f25bbbdb3ac5620850c7d09d6a63af888411ecf1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Sep 2014 20:42:07 -0700 Subject: [PATCH 0621/1492] [SPARK-3280] Made sort-based shuffle the default implementation Sort-based shuffle has lower memory usage and seems to outperform hash-based in almost all of our testing. Author: Reynold Xin Closes #2178 from rxin/sort-shuffle and squashes the following commits: 713d341 [Reynold Xin] Fixed test failures by setting spark.shuffle.compress to the same value as spark.shuffle.spill.compress. 85165e6 [Reynold Xin] Fixed a comment typo. aa0d372 [Reynold Xin] [SPARK-3280] Made sort-based shuffle the default implementation --- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/HashShuffleSuite.scala | 33 +++++++++++++++++++ .../scala/org/apache/spark/ShuffleSuite.scala | 2 +- .../org/apache/spark/SortShuffleSuite.scala | 3 +- .../ExternalAppendOnlyMapSuite.scala | 1 + docs/configuration.md | 9 +++-- 6 files changed, 41 insertions(+), 9 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/HashShuffleSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 2973d002cc428..20a7444cfc5ee 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -217,7 +217,7 @@ object SparkEnv extends Logging { val shortShuffleMgrNames = Map( "hash" -> "org.apache.spark.shuffle.hash.HashShuffleManager", "sort" -> "org.apache.spark.shuffle.sort.SortShuffleManager") - val shuffleMgrName = conf.get("spark.shuffle.manager", "hash") + val shuffleMgrName = conf.get("spark.shuffle.manager", "sort") val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase, shuffleMgrName) val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) diff --git a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala new file mode 100644 index 0000000000000..2acc02a54fa3d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.scalatest.BeforeAndAfterAll + +class HashShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { + + // This test suite should run all tests in ShuffleSuite with hash-based shuffle. + + override def beforeAll() { + System.setProperty("spark.shuffle.manager", "hash") + } + + override def afterAll() { + System.clearProperty("spark.shuffle.manager") + } +} diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index b13ddf96bc77c..15aa4d83800fa 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair -class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { +abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { val conf = new SparkConf(loadDefaults = false) diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index 5c02c00586ef4..639e56c488db4 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -24,8 +24,7 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with sort-based shuffle. override def beforeAll() { - System.setProperty("spark.shuffle.manager", - "org.apache.spark.shuffle.sort.SortShuffleManager") + System.setProperty("spark.shuffle.manager", "sort") } override def afterAll() { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index ac3931e3d0a73..511d76c9144cc 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -42,6 +42,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { conf.set("spark.serializer.objectStreamReset", "1") conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") conf.set("spark.shuffle.spill.compress", codec.isDefined.toString) + conf.set("spark.shuffle.compress", codec.isDefined.toString) codec.foreach { c => conf.set("spark.io.compression.codec", c) } // Ensure that we actually have multiple batches per spill file conf.set("spark.shuffle.spill.batchSize", "10") diff --git a/docs/configuration.md b/docs/configuration.md index 65a422caabb7e..36178efb97103 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -293,12 +293,11 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.manager - HASH + sort - Implementation to use for shuffling data. A hash-based shuffle manager is the default, but - starting in Spark 1.1 there is an experimental sort-based shuffle manager that is more - memory-efficient in environments with small executors, such as YARN. To use that, change - this value to SORT. + Implementation to use for shuffling data. There are two implementations available: + sort and hash. Sort-based shuffle is more memory-efficient and is + the default option starting in 1.2. From eddfeddac19870fc265ef406d87e1c3db9b54249 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 7 Sep 2014 20:56:04 -0700 Subject: [PATCH 0622/1492] [SPARK-938][doc] Add OpenStack Swift support See compiled doc at http://people.apache.org/~rxin/tmp/openstack-swift/_site/storage-openstack-swift.html This is based on #1010. Closes #1010. Author: Reynold Xin Author: Gil Vernik Closes #2298 from rxin/openstack-swift and squashes the following commits: ff4e394 [Reynold Xin] Two minor comments from Patrick. 279f6de [Reynold Xin] core-sites -> core-site dfb8fea [Reynold Xin] Updated based on Gil's suggestion. 846f5cb [Reynold Xin] Added a link from overview page. 0447c9f [Reynold Xin] Removed sample code. e9c3761 [Reynold Xin] Merge pull request #1010 from gilv/master 9233fef [Gil Vernik] Fixed typos 6994827 [Gil Vernik] Merge pull request #1 from rxin/openstack ac0679e [Reynold Xin] Fixed an unclosed tr. 47ce99d [Reynold Xin] Merge branch 'master' into openstack cca7192 [Gil Vernik] Removed white spases from pom.xml 99f095d [Reynold Xin] Pending openstack changes. eb22295 [Reynold Xin] Merge pull request #1010 from gilv/master 39a9737 [Gil Vernik] Spark integration with Openstack Swift c977658 [Gil Vernik] Merge branch 'master' of https://github.com/gilv/spark 2aba763 [Gil Vernik] Fix to docs/openstack-integration.md 9b625b5 [Gil Vernik] Merge branch 'master' of https://github.com/gilv/spark eff538d [Gil Vernik] SPARK-938 - Openstack Swift object storage support ce483d7 [Gil Vernik] SPARK-938 - Openstack Swift object storage support b6c37ef [Gil Vernik] Openstack Swift support --- docs/index.md | 2 + docs/storage-openstack-swift.md | 152 ++++++++++++++++++++++++++++++++ 2 files changed, 154 insertions(+) create mode 100644 docs/storage-openstack-swift.md diff --git a/docs/index.md b/docs/index.md index 4ac0982ae54f1..7fe6b43d32af7 100644 --- a/docs/index.md +++ b/docs/index.md @@ -103,6 +103,8 @@ options for deployment: * [Security](security.html): Spark security support * [Hardware Provisioning](hardware-provisioning.html): recommendations for cluster hardware * [3rd Party Hadoop Distributions](hadoop-third-party-distributions.html): using common Hadoop distributions +* Integration with other storage systems: + * [OpenStack Swift](storage-openstack-swift.html) * [Building Spark with Maven](building-with-maven.html): build Spark using the Maven system * [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md new file mode 100644 index 0000000000000..c39ef1ce59e1c --- /dev/null +++ b/docs/storage-openstack-swift.md @@ -0,0 +1,152 @@ +--- +layout: global +title: Accessing OpenStack Swift from Spark +--- + +Spark's support for Hadoop InputFormat allows it to process data in OpenStack Swift using the +same URI formats as in Hadoop. You can specify a path in Swift as input through a +URI of the form swift://container.PROVIDER/path. You will also need to set your +Swift security credentials, through core-site.xml or via +SparkContext.hadoopConfiguration. +Current Swift driver requires Swift to use Keystone authentication method. + +# Configuring Swift for Better Data Locality + +Although not mandatory, it is recommended to configure the proxy server of Swift with +list_endpoints to have better data locality. More information is +[available here](https://github.com/openstack/swift/blob/master/swift/common/middleware/list_endpoints.py). + + +# Dependencies + +The Spark application should include hadoop-openstack dependency. +For example, for Maven support, add the following to the pom.xml file: + +{% highlight xml %} + + ... + + org.apache.hadoop + hadoop-openstack + 2.3.0 + + ... + +{% endhighlight %} + + +# Configuration Parameters + +Create core-site.xml and place it inside Spark's conf directory. +There are two main categories of parameters that should to be configured: declaration of the +Swift driver and the parameters that are required by Keystone. + +Configuration of Hadoop to use Swift File system achieved via + + + + + + + +
    Property NameValue
    fs.swift.implorg.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem
    + +Additional parameters required by Keystone (v2.0) and should be provided to the Swift driver. Those +parameters will be used to perform authentication in Keystone to access Swift. The following table +contains a list of Keystone mandatory parameters. PROVIDER can be any name. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaningRequired
    fs.swift.service.PROVIDER.auth.urlKeystone Authentication URLMandatory
    fs.swift.service.PROVIDER.auth.endpoint.prefixKeystone endpoints prefixOptional
    fs.swift.service.PROVIDER.tenantTenantMandatory
    fs.swift.service.PROVIDER.usernameUsernameMandatory
    fs.swift.service.PROVIDER.passwordPasswordMandatory
    fs.swift.service.PROVIDER.http.portHTTP portMandatory
    fs.swift.service.PROVIDER.regionKeystone regionMandatory
    fs.swift.service.PROVIDER.publicIndicates if all URLs are publicMandatory
    + +For example, assume PROVIDER=SparkTest and Keystone contains user tester with password testing +defined for tenant test. Then core-site.xml should include: + +{% highlight xml %} + + + fs.swift.impl + org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem + + + fs.swift.service.SparkTest.auth.url + http://127.0.0.1:5000/v2.0/tokens + + + fs.swift.service.SparkTest.auth.endpoint.prefix + endpoints + + fs.swift.service.SparkTest.http.port + 8080 + + + fs.swift.service.SparkTest.region + RegionOne + + + fs.swift.service.SparkTest.public + true + + + fs.swift.service.SparkTest.tenant + test + + + fs.swift.service.SparkTest.username + tester + + + fs.swift.service.SparkTest.password + testing + + +{% endhighlight %} + +Notice that +fs.swift.service.PROVIDER.tenant, +fs.swift.service.PROVIDER.username, +fs.swift.service.PROVIDER.password contains sensitive information and keeping them in +core-site.xml is not always a good approach. +We suggest to keep those parameters in core-site.xml for testing purposes when running Spark +via spark-shell. +For job submissions they should be provided via sparkContext.hadoopConfiguration. From 0d1cc4ae42e1f73538dd8b9b1880ca9e5b124108 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 8 Sep 2014 14:32:53 +0530 Subject: [PATCH 0623/1492] [HOTFIX] A left over version change. It should make mima happy. Author: Prashant Sharma Closes #2317 from ScrapCodes/hotfix and squashes the following commits: b6472d4 [Prashant Sharma] [HOTFIX] for hotfixes, a left over version change. --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a26c2c90cb321..45f6d2973ea90 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -184,7 +184,7 @@ object OldDeps { def versionArtifact(id: String): Option[sbt.ModuleID] = { val fullId = id + "_2.10" - Some("org.apache.spark" % fullId % "1.0.0") + Some("org.apache.spark" % fullId % "1.1.0") } def oldDepsSettings() = Defaults.defaultSettings ++ Seq( From 711356b422c66e2a80377a9f43fce97282460520 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Mon, 8 Sep 2014 09:47:13 -0700 Subject: [PATCH 0624/1492] [SPARK-3086] [SPARK-3043] [SPARK-3156] [mllib] DecisionTree aggregation improvements MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Summary: 1. Variable numBins for each feature [SPARK-3043] 2. Reduced data reshaping in aggregation [SPARK-3043] 3. Choose ordering for ordered categorical features adaptively [SPARK-3156] 4. Changed nodes to use 1-indexing [SPARK-3086] 5. Small clean-ups Note: This PR looks bigger than it is since I moved several functions from inside findBestSplitsPerGroup to outside of it (to make it clear what was being serialized in the aggregation). Speedups: This update helps most when many features use few bins but a few features use many bins. Some example results on speedups with 2M examples, 3.5K features (15-worker EC2 cluster): * Example where old code was reasonably efficient (1/2 continuous, 1/4 binary, 1/4 20-category): 164.813 --> 116.491 sec * Example where old code wasted many bins (1/10 continuous, 81/100 binary, 9/100 20-category): 128.701 --> 39.334 sec Details: (1) Variable numBins for each feature [SPARK-3043] DecisionTreeMetadata now computes a variable numBins for each feature. It also tracks numSplits. (2) Reduced data reshaping in aggregation [SPARK-3043] Added DTStatsAggregator, a wrapper around the aggregate statistics array for easy but efficient indexing. * Added ImpurityAggregator and ImpurityCalculator classes, to make DecisionTree code more oblivious to the type of impurity. * Design note: I originally tried creating Impurity classes which stored data and storing the aggregates in an Array[Array[Array[Impurity]]]. However, this led to significant slowdowns, perhaps because of overhead in creating so many objects. The aggregate statistics are never reshaped, and cumulative sums are computed in-place. Updated the layout of aggregation functions. The update simplifies things by (1) dividing features into ordered/unordered (instead of ordered/unordered/continuous) and (2) making use of the DTStatsAggregator for indexing. For this update, the following functions were refactored: * updateBinForOrderedFeature * updateBinForUnorderedFeature * binaryOrNotCategoricalBinSeqOp * multiclassWithCategoricalBinSeqOp * regressionBinSeqOp The above 5 functions were replaced with: * orderedBinSeqOp * someUnorderedBinSeqOp Other changes: * calculateGainForSplit now treats all feature types the same way. * Eliminated extractLeftRightNodeAggregates. (3) Choose ordering for ordered categorical features adaptively [SPARK-3156] Updated binsToBestSplit(): * This now computes cumulative sums of stats for ordered features. * For ordered categorical features, it chooses an ordering for categories. (This uses to be done by findSplitsBins.) * Uses iterators to shorten code and avoid building an Array[Array[InformationGainStats]]. Side effects: * In findSplitsBins: A sample of the data is only taken for data with continuous features. It is not needed for data with only categorical features. * In findSplitsBins: splits and bins are no longer pre-computed for ordered categorical features since they are not needed. * TreePoint binning is simpler for categorical features. (4) Changed nodes to use 1-indexing [SPARK-3086] Nodes used to be indexed from 0. Now they are indexed from 1. Node indexing functions are now collected in object Node (Node.scala). (5) Small clean-ups Eliminated functions extractNodeInfo() and extractInfoForLowerLevels() to reduce duplicate code. Eliminated InvalidBinIndex since it is no longer used. CC: mengxr manishamde Please let me know if you have thoughts on this—thanks! Author: Joseph K. Bradley Closes #2125 from jkbradley/dt-opt3alt and squashes the following commits: 42c192a [Joseph K. Bradley] Merge branch 'rfs' into dt-opt3alt d3cc46b [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 00e4404 [Joseph K. Bradley] optimization for TreePoint construction (pre-computing featureArity and isUnordered as arrays) 425716c [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into rfs a2acea5 [Joseph K. Bradley] Small optimizations based on profiling aa4e4df [Joseph K. Bradley] Updated DTStatsAggregator with bug fix (nodeString should not be multiplied by statsSize) 4651154 [Joseph K. Bradley] Changed numBins semantics for unordered features. * Before: numBins = numSplits = (1 << k - 1) - 1 * Now: numBins = 2 * numSplits = 2 * [(1 << k - 1) - 1] * This also involved changing the semantics of: ** DecisionTreeMetadata.numUnorderedBins() 1e3b1c7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 1485fcc [Joseph K. Bradley] Made some DecisionTree methods private. 92f934f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt e676da1 [Joseph K. Bradley] Updated documentation for DecisionTree 37ca845 [Joseph K. Bradley] Fixed problem with how DecisionTree handles ordered categorical features. 105f8ab [Joseph K. Bradley] Removed commented-out getEmptyBinAggregates from DecisionTree 062c31d [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3alt 6d32ccd [Joseph K. Bradley] In DecisionTree.binsToBestSplit, changed loops to iterators to shorten code. 807cd00 [Joseph K. Bradley] Finished DTStatsAggregator, a wrapper around the aggregate statistics for easy but hopefully efficient indexing. Modified old ImpurityAggregator classes and renamed them ImpurityCalculator; added ImpurityAggregator classes which work with DTStatsAggregator but do not store data. Unit tests all succeed. f2166fd [Joseph K. Bradley] still working on DTStatsAggregator 92f7118 [Joseph K. Bradley] Added partly written DTStatsAggregator fd8df30 [Joseph K. Bradley] Moved some aggregation helpers outside of findBestSplitsPerGroup d7c53ee [Joseph K. Bradley] Added more doc for ImpurityAggregator a40f8f1 [Joseph K. Bradley] Changed nodes to be indexed from 1. Tests work. 95cad7c [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 5f94342 [Joseph K. Bradley] Added treeAggregate since not yet merged from master. Moved node indexing functions to Node. 61c4509 [Joseph K. Bradley] Fixed bugs from merge: missing DT timer call, and numBins setting. Cleaned up DT Suite some. 3ba7166 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 b314659 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt3 9c83363 [Joseph K. Bradley] partial merge but not done yet 45f7ea7 [Joseph K. Bradley] partial merge, not yet done 5fce635 [Joseph K. Bradley] Merge branch 'dt-opt2' into dt-opt3 26d10dd [Joseph K. Bradley] Removed tree/model/Filter.scala since no longer used. Removed debugging println calls in DecisionTree.scala. 356daba [Joseph K. Bradley] Merge branch 'dt-opt1' into dt-opt2 430d782 [Joseph K. Bradley] Added more debug info on binning error. Added some docs. d036089 [Joseph K. Bradley] Print timing info to logDebug. e66f1b1 [Joseph K. Bradley] TreePoint * Updated doc * Made some methods private 8464a6e [Joseph K. Bradley] Moved TimeTracker to tree/impl/ in its own file, and cleaned it up. Removed debugging println calls from DecisionTree. Made TreePoint extend Serialiable a87e08f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt1 dd4d3aa [Joseph K. Bradley] Mid-process in bug fix: bug for binary classification with categorical features * Bug: Categorical features were all treated as ordered for binary classification. This is possible but would require the bin ordering to be determined on-the-fly after the aggregation. Currently, the ordering is determined a priori and fixed for all splits. * (Temp) Fix: Treat low-arity categorical features as unordered for binary classification. * Related change: I removed most tests for isMulticlass in the code. I instead test metadata for whether there are unordered features. * Status: The bug may be fixed, but more testing needs to be done. 438a660 [Joseph K. Bradley] removed subsampling for mnist8m from DT 86e217f [Joseph K. Bradley] added cache to DT input e3c84cc [Joseph K. Bradley] Added stuff fro mnist8m to D T Runner 51ef781 [Joseph K. Bradley] Fixed bug introduced by last commit: Variance impurity calculation was incorrect since counts were swapped accidentally fd65372 [Joseph K. Bradley] Major changes: * Created ImpurityAggregator classes, rather than old aggregates. * Feature split/bin semantics are based on ordered vs. unordered ** E.g.: numSplits = numBins for all unordered features, and numSplits = numBins - 1 for all ordered features. * numBins can differ for each feature c1565a5 [Joseph K. Bradley] Small DecisionTree updates: * Simplification: Updated calculateGainForSplit to take aggregates for a single (feature, split) pair. * Internal doc: findAggForOrderedFeatureClassification b914f3b [Joseph K. Bradley] DecisionTree optimization: eliminated filters + small changes b2ed1f3 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-opt 0f676e2 [Joseph K. Bradley] Optimizations + Bug fix for DecisionTree 3211f02 [Joseph K. Bradley] Optimizing DecisionTree * Added TreePoint representation to avoid calling findBin multiple times. * (not working yet, but debugging) f61e9d2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing bcf874a [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing 511ec85 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-timing a95bc22 [Joseph K. Bradley] timing for DecisionTree internals --- .../spark/mllib/tree/DecisionTree.scala | 1341 ++++++----------- .../mllib/tree/impl/DTStatsAggregator.scala | 213 +++ .../tree/impl/DecisionTreeMetadata.scala | 73 +- .../spark/mllib/tree/impl/TreePoint.scala | 93 +- .../spark/mllib/tree/impurity/Entropy.scala | 84 ++ .../spark/mllib/tree/impurity/Gini.scala | 84 ++ .../spark/mllib/tree/impurity/Impurity.scala | 127 ++ .../spark/mllib/tree/impurity/Variance.scala | 72 + .../apache/spark/mllib/tree/model/Bin.scala | 7 +- .../apache/spark/mllib/tree/model/Node.scala | 85 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 391 ++--- 11 files changed, 1322 insertions(+), 1248 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 5cdd258f6c20b..dd766c12d28a4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -28,8 +28,9 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ -import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TimeTracker, TreePoint} +import org.apache.spark.mllib.tree.impl._ import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} +import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -65,36 +66,41 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val retaggedInput = input.retag(classOf[LabeledPoint]) val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) logDebug("algo = " + strategy.algo) + logDebug("maxBins = " + metadata.maxBins) // Find the splits and the corresponding bins (interval between the splits) using a sample // of the input data. timer.start("findSplitsBins") val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) - val numBins = bins(0).length timer.stop("findSplitsBins") - logDebug("numBins = " + numBins) + logDebug("numBins: feature: number of bins") + logDebug(Range(0, metadata.numFeatures).map { featureIndex => + s"\t$featureIndex\t${metadata.numBins(featureIndex)}" + }.mkString("\n")) // Bin feature values (TreePoint representation). // Cache input RDD for speedup during multiple passes. val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) .persist(StorageLevel.MEMORY_AND_DISK) - val numFeatures = metadata.numFeatures // depth of the decision tree val maxDepth = strategy.maxDepth - // the max number of nodes possible given the depth of the tree - val maxNumNodes = (2 << maxDepth) - 1 + require(maxDepth <= 30, + s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") + // Number of nodes to allocate: max number of nodes possible given the depth of the tree, plus 1 + val maxNumNodesPlus1 = Node.startIndexInLevel(maxDepth + 1) // Initialize an array to hold parent impurity calculations for each node. - val parentImpurities = new Array[Double](maxNumNodes) + val parentImpurities = new Array[Double](maxNumNodesPlus1) // dummy value for top node (updated during first split calculation) - val nodes = new Array[Node](maxNumNodes) + val nodes = new Array[Node](maxNumNodesPlus1) // Calculate level for single group construction // Max memory usage for aggregates val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - val numElementsPerNode = DecisionTree.getElementsPerNode(metadata, numBins) + // TODO: Calculate memory usage more precisely. + val numElementsPerNode = DecisionTree.getElementsPerNode(metadata) logDebug("numElementsPerNode = " + numElementsPerNode) val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array @@ -124,26 +130,29 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. timer.start("findBestSplits") - val splitsStatsForLevel = DecisionTree.findBestSplits(treeInput, parentImpurities, - metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) + val splitsStatsForLevel: Array[(Split, InformationGainStats)] = + DecisionTree.findBestSplits(treeInput, parentImpurities, + metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") - val levelNodeIndexOffset = (1 << level) - 1 + val levelNodeIndexOffset = Node.startIndexInLevel(level) for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { val nodeIndex = levelNodeIndexOffset + index - val isLeftChild = level != 0 && nodeIndex % 2 == 1 - val parentNodeIndex = if (isLeftChild) { // -1 for root node - (nodeIndex - 1) / 2 - } else { - (nodeIndex - 2) / 2 - } + // Extract info for this node (index) at the current level. timer.start("extractNodeInfo") - extractNodeInfo(nodeSplitStats, level, index, nodes) + val split = nodeSplitStats._1 + val stats = nodeSplitStats._2 + val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) + val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) + logDebug("Node = " + node) + nodes(nodeIndex) = node timer.stop("extractNodeInfo") + if (level != 0) { // Set parent. - if (isLeftChild) { + val parentNodeIndex = Node.parentIndex(nodeIndex) + if (Node.isLeftChild(nodeIndex)) { nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) } else { nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) @@ -151,11 +160,21 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo } // Extract info for nodes at the next lower level. timer.start("extractInfoForLowerLevels") - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities) + if (level < maxDepth) { + val leftChildIndex = Node.leftChildIndex(nodeIndex) + val leftImpurity = stats.leftImpurity + logDebug("leftChildIndex = " + leftChildIndex + ", impurity = " + leftImpurity) + parentImpurities(leftChildIndex) = leftImpurity + + val rightChildIndex = Node.rightChildIndex(nodeIndex) + val rightImpurity = stats.rightImpurity + logDebug("rightChildIndex = " + rightChildIndex + ", impurity = " + rightImpurity) + parentImpurities(rightChildIndex) = rightImpurity + } timer.stop("extractInfoForLowerLevels") - logDebug("final best split = " + nodeSplitStats._1) + logDebug("final best split = " + split) } - require((1 << level) == splitsStatsForLevel.length) + require(Node.maxNodesInLevel(level) == splitsStatsForLevel.length) // Check whether all the nodes at the current level at leaves. val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) logDebug("all leaf = " + allLeaf) @@ -171,7 +190,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("#####################################") // Initialize the top or root node of the tree. - val topNode = nodes(0) + val topNode = nodes(1) // Build the full tree using the node info calculated in the level-wise best split calculations. topNode.build(nodes) @@ -183,47 +202,6 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo new DecisionTreeModel(topNode, strategy.algo) } - /** - * Extract the decision tree node information for the given tree level and node index - */ - private def extractNodeInfo( - nodeSplitStats: (Split, InformationGainStats), - level: Int, - index: Int, - nodes: Array[Node]): Unit = { - val split = nodeSplitStats._1 - val stats = nodeSplitStats._2 - val nodeIndex = (1 << level) - 1 + index - val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) - val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) - logDebug("Node = " + node) - nodes(nodeIndex) = node - } - - /** - * Extract the decision tree node information for the children of the node - */ - private def extractInfoForLowerLevels( - level: Int, - index: Int, - maxDepth: Int, - nodeSplitStats: (Split, InformationGainStats), - parentImpurities: Array[Double]): Unit = { - - if (level >= maxDepth) { - return - } - - val leftNodeIndex = (2 << level) - 1 + 2 * index - val leftImpurity = nodeSplitStats._2.leftImpurity - logDebug("leftNodeIndex = " + leftNodeIndex + ", impurity = " + leftImpurity) - parentImpurities(leftNodeIndex) = leftImpurity - - val rightNodeIndex = leftNodeIndex + 1 - val rightImpurity = nodeSplitStats._2.rightImpurity - logDebug("rightNodeIndex = " + rightNodeIndex + ", impurity = " + rightImpurity) - parentImpurities(rightNodeIndex) = rightImpurity - } } object DecisionTree extends Serializable with Logging { @@ -425,9 +403,6 @@ object DecisionTree extends Serializable with Logging { impurity, maxDepth, maxBins) } - - private val InvalidBinIndex = -1 - /** * Returns an array of optimal splits for all nodes at a given level. Splits the task into * multiple groups if the level-wise training task could lead to memory overflow. @@ -436,12 +411,12 @@ object DecisionTree extends Serializable with Logging { * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param splits possible splits for all features - * @param bins possible bins for all features + * @param splits possible splits for all features, indexed (numFeatures)(numSplits) + * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. * @return array (over nodes) of splits with best split for each node at a given level. */ - protected[tree] def findBestSplits( + private[tree] def findBestSplits( input: RDD[TreePoint], parentImpurities: Array[Double], metadata: DecisionTreeMetadata, @@ -474,6 +449,138 @@ object DecisionTree extends Serializable with Logging { } } + /** + * Get the node index corresponding to this data point. + * This function mimics prediction, passing an example from the root node down to a node + * at the current level being trained; that node's index is returned. + * + * @param node Node in tree from which to classify the given data point. + * @param binnedFeatures Binned feature vector for data point. + * @param bins possible bins for all features, indexed (numFeatures)(numBins) + * @param unorderedFeatures Set of indices of unordered features. + * @return Leaf index if the data point reaches a leaf. + * Otherwise, last node reachable in tree matching this example. + * Note: This is the global node index, i.e., the index used in the tree. + * This index is different from the index used during training a particular + * set of nodes in a (level, group). + */ + private def predictNodeIndex( + node: Node, + binnedFeatures: Array[Int], + bins: Array[Array[Bin]], + unorderedFeatures: Set[Int]): Int = { + if (node.isLeaf) { + node.id + } else { + val featureIndex = node.split.get.feature + val splitLeft = node.split.get.featureType match { + case Continuous => { + val binIndex = binnedFeatures(featureIndex) + val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold + // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] + // We do not need to check lowSplit since bins are separated by splits. + featureValueUpperBound <= node.split.get.threshold + } + case Categorical => { + val featureValue = binnedFeatures(featureIndex) + node.split.get.categories.contains(featureValue) + } + case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") + } + if (node.leftNode.isEmpty || node.rightNode.isEmpty) { + // Return index from next layer of nodes to train + if (splitLeft) { + Node.leftChildIndex(node.id) + } else { + Node.rightChildIndex(node.id) + } + } else { + if (splitLeft) { + predictNodeIndex(node.leftNode.get, binnedFeatures, bins, unorderedFeatures) + } else { + predictNodeIndex(node.rightNode.get, binnedFeatures, bins, unorderedFeatures) + } + } + } + } + + /** + * Helper for binSeqOp, for data which can contain a mix of ordered and unordered features. + * + * For ordered features, a single bin is updated. + * For unordered features, bins correspond to subsets of categories; either the left or right bin + * for each subset is updated. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @param bins possible bins for all features, indexed (numFeatures)(numBins) + * @param unorderedFeatures Set of indices of unordered features. + */ + private def mixedBinSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint, + nodeIndex: Int, + bins: Array[Array[Bin]], + unorderedFeatures: Set[Int]): Unit = { + // Iterate over all features. + val numFeatures = treePoint.binnedFeatures.size + val nodeOffset = agg.getNodeOffset(nodeIndex) + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (unorderedFeatures.contains(featureIndex)) { + // Unordered feature + val featureValue = treePoint.binnedFeatures(featureIndex) + val (leftNodeFeatureOffset, rightNodeFeatureOffset) = + agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + // Update the left or right bin for each split. + val numSplits = agg.numSplits(featureIndex) + var splitIndex = 0 + while (splitIndex < numSplits) { + if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { + agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label) + } else { + agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label) + } + splitIndex += 1 + } + } else { + // Ordered feature + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.nodeUpdate(nodeOffset, featureIndex, binIndex, treePoint.label) + } + featureIndex += 1 + } + } + + /** + * Helper for binSeqOp, for regression and for classification with only ordered features. + * + * For each feature, the sufficient statistics of one bin are updated. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). + * @param treePoint Data point being aggregated. + * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @return agg + */ + private def orderedBinSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint, + nodeIndex: Int): Unit = { + val label = treePoint.label + val nodeOffset = agg.getNodeOffset(nodeIndex) + // Iterate over all features. + val numFeatures = agg.numFeatures + var featureIndex = 0 + while (featureIndex < numFeatures) { + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.nodeUpdate(nodeOffset, featureIndex, binIndex, label) + featureIndex += 1 + } + } + /** * Returns an array of optimal splits for a group of nodes at a given level * @@ -481,8 +588,9 @@ object DecisionTree extends Serializable with Logging { * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param splits possible splits for all features - * @param bins possible bins for all features, indexed as (numFeatures)(numBins) + * @param nodes Array of all nodes in the tree. Used for matching data points to nodes. + * @param splits possible splits for all features, indexed (numFeatures)(numSplits) + * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. * @return array of splits with best splits for all nodes at a given level. @@ -527,88 +635,22 @@ object DecisionTree extends Serializable with Logging { // numNodes: Number of nodes in this (level of tree, group), // where nodes at deeper (larger) levels may be divided into groups. - val numNodes = (1 << level) / numGroups + val numNodes = Node.maxNodesInLevel(level) / numGroups logDebug("numNodes = " + numNodes) - // Find the number of features by looking at the first sample. - val numFeatures = metadata.numFeatures - logDebug("numFeatures = " + numFeatures) - - // numBins: Number of bins = 1 + number of possible splits - val numBins = bins(0).length - logDebug("numBins = " + numBins) - - val numClasses = metadata.numClasses - logDebug("numClasses = " + numClasses) - - val isMulticlass = metadata.isMulticlass - logDebug("isMulticlass = " + isMulticlass) - - val isMulticlassWithCategoricalFeatures = metadata.isMulticlassWithCategoricalFeatures - logDebug("isMultiClassWithCategoricalFeatures = " + isMulticlassWithCategoricalFeatures) + logDebug("numFeatures = " + metadata.numFeatures) + logDebug("numClasses = " + metadata.numClasses) + logDebug("isMulticlass = " + metadata.isMulticlass) + logDebug("isMulticlassWithCategoricalFeatures = " + + metadata.isMulticlassWithCategoricalFeatures) // shift when more than one group is used at deep tree level val groupShift = numNodes * groupIndex - /** - * Get the node index corresponding to this data point. - * This function mimics prediction, passing an example from the root node down to a node - * at the current level being trained; that node's index is returned. - * - * @return Leaf index if the data point reaches a leaf. - * Otherwise, last node reachable in tree matching this example. - */ - def predictNodeIndex(node: Node, binnedFeatures: Array[Int]): Int = { - if (node.isLeaf) { - node.id - } else { - val featureIndex = node.split.get.feature - val splitLeft = node.split.get.featureType match { - case Continuous => { - val binIndex = binnedFeatures(featureIndex) - val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold - // bin binIndex has range (bin.lowSplit.threshold, bin.highSplit.threshold] - // We do not need to check lowSplit since bins are separated by splits. - featureValueUpperBound <= node.split.get.threshold - } - case Categorical => { - val featureValue = if (metadata.isUnordered(featureIndex)) { - binnedFeatures(featureIndex) - } else { - val binIndex = binnedFeatures(featureIndex) - bins(featureIndex)(binIndex).category - } - node.split.get.categories.contains(featureValue) - } - case _ => throw new RuntimeException(s"predictNodeIndex failed for unknown reason.") - } - if (node.leftNode.isEmpty || node.rightNode.isEmpty) { - // Return index from next layer of nodes to train - if (splitLeft) { - node.id * 2 + 1 // left - } else { - node.id * 2 + 2 // right - } - } else { - if (splitLeft) { - predictNodeIndex(node.leftNode.get, binnedFeatures) - } else { - predictNodeIndex(node.rightNode.get, binnedFeatures) - } - } - } - } - - def nodeIndexToLevel(idx: Int): Int = { - if (idx == 0) { - 0 - } else { - math.floor(math.log(idx) / math.log(2)).toInt - } - } - - // Used for treePointToNodeIndex - val levelOffset = (1 << level) - 1 + // Used for treePointToNodeIndex to get an index for this (level, group). + // - Node.startIndexInLevel(level) gives the global index offset for nodes at this level. + // - groupShift corrects for groups in this level before the current group. + val globalNodeIndexOffset = Node.startIndexInLevel(level) + groupShift /** * Find the node index for the given example. @@ -619,661 +661,254 @@ object DecisionTree extends Serializable with Logging { if (level == 0) { 0 } else { - val globalNodeIndex = predictNodeIndex(nodes(0), treePoint.binnedFeatures) - // Get index for this (level, group). - globalNodeIndex - levelOffset - groupShift - } - } - - /** - * Increment aggregate in location for (node, feature, bin, label). - * - * @param treePoint Data point being aggregated. - * @param agg Array storing aggregate calculation, of size: - * numClasses * numBins * numFeatures * numNodes. - * Indexed by (node, feature, bin, label) where label is the least significant bit. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def updateBinForOrderedFeature( - treePoint: TreePoint, - agg: Array[Double], - nodeIndex: Int, - featureIndex: Int): Unit = { - // Update the left or right count for one bin. - val aggIndex = - numClasses * numBins * numFeatures * nodeIndex + - numClasses * numBins * featureIndex + - numClasses * treePoint.binnedFeatures(featureIndex) + - treePoint.label.toInt - agg(aggIndex) += 1 - } - - /** - * Increment aggregate in location for (nodeIndex, featureIndex, [bins], label), - * where [bins] ranges over all bins. - * Updates left or right side of aggregate depending on split. - * - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @param treePoint Data point being aggregated. - * @param agg Indexed by (left/right, node, feature, bin, label) - * where label is the least significant bit. - * The left/right specifier is a 0/1 index indicating left/right child info. - * @param rightChildShift Offset for right side of agg. - */ - def updateBinForUnorderedFeature( - nodeIndex: Int, - featureIndex: Int, - treePoint: TreePoint, - agg: Array[Double], - rightChildShift: Int): Unit = { - val featureValue = treePoint.binnedFeatures(featureIndex) - // Update the left or right count for one bin. - val aggShift = - numClasses * numBins * numFeatures * nodeIndex + - numClasses * numBins * featureIndex + - treePoint.label.toInt - // Find all matching bins and increment their values - val featureCategories = metadata.featureArity(featureIndex) - val numCategoricalBins = (1 << featureCategories - 1) - 1 - var binIndex = 0 - while (binIndex < numCategoricalBins) { - val aggIndex = aggShift + binIndex * numClasses - if (bins(featureIndex)(binIndex).highSplit.categories.contains(featureValue)) { - agg(aggIndex) += 1 - } else { - agg(rightChildShift + aggIndex) += 1 - } - binIndex += 1 - } - } - - /** - * Helper for binSeqOp. - * - * @param agg Array storing aggregate calculation, of size: - * numClasses * numBins * numFeatures * numNodes. - * Indexed by (node, feature, bin, label) where label is the least significant bit. - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def binaryOrNotCategoricalBinSeqOp( - agg: Array[Double], - treePoint: TreePoint, - nodeIndex: Int): Unit = { - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) - featureIndex += 1 - } - } - - val rightChildShift = numClasses * numBins * numFeatures * numNodes - - /** - * Helper for binSeqOp. - * - * @param agg Array storing aggregate calculation. - * For ordered features, this is of size: - * numClasses * numBins * numFeatures * numNodes. - * For unordered features, this is of size: - * 2 * numClasses * numBins * numFeatures * numNodes. - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - */ - def multiclassWithCategoricalBinSeqOp( - agg: Array[Double], - treePoint: TreePoint, - nodeIndex: Int): Unit = { - val label = treePoint.label - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (metadata.isUnordered(featureIndex)) { - updateBinForUnorderedFeature(nodeIndex, featureIndex, treePoint, agg, rightChildShift) - } else { - updateBinForOrderedFeature(treePoint, agg, nodeIndex, featureIndex) - } - featureIndex += 1 - } - } - - /** - * Performs a sequential aggregation over a partition for regression. - * For l nodes, k features, - * the count, sum, sum of squares of one of the p bins is incremented. - * - * @param agg Array storing aggregate calculation, updated by this function. - * Size: 3 * numBins * numFeatures * numNodes - * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @return agg - */ - def regressionBinSeqOp(agg: Array[Double], treePoint: TreePoint, nodeIndex: Int): Unit = { - val label = treePoint.label - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Update count, sum, and sum^2 for one bin. - val binIndex = treePoint.binnedFeatures(featureIndex) - val aggIndex = - 3 * numBins * numFeatures * nodeIndex + - 3 * numBins * featureIndex + - 3 * binIndex - agg(aggIndex) += 1 - agg(aggIndex + 1) += label - agg(aggIndex + 2) += label * label - featureIndex += 1 + val globalNodeIndex = + predictNodeIndex(nodes(1), treePoint.binnedFeatures, bins, metadata.unorderedFeatures) + globalNodeIndex - globalNodeIndexOffset } } /** * Performs a sequential aggregation over a partition. - * For l nodes, k features, - * For classification: - * Either the left count or the right count of one of the bins is - * incremented based upon whether the feature is classified as 0 or 1. - * For regression: - * The count, sum, sum of squares of one of the bins is incremented. * - * @param agg Array storing aggregate calculation, updated by this function. - * Size for classification: - * numClasses * numBins * numFeatures * numNodes for ordered features, or - * 2 * numClasses * numBins * numFeatures * numNodes for unordered features. - * Size for regression: - * 3 * numBins * numFeatures * numNodes. + * Each data point contributes to one node. For each feature, + * the aggregate sufficient statistics are updated for the relevant bins. + * + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (node, feature, bin). * @param treePoint Data point being aggregated. * @return agg */ - def binSeqOp(agg: Array[Double], treePoint: TreePoint): Array[Double] = { + def binSeqOp( + agg: DTStatsAggregator, + treePoint: TreePoint): DTStatsAggregator = { val nodeIndex = treePointToNodeIndex(treePoint) // If the example does not reach this level, then nodeIndex < 0. // If the example reaches this level but is handled in a different group, // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). if (nodeIndex >= 0 && nodeIndex < numNodes) { - if (metadata.isClassification) { - if (isMulticlassWithCategoricalFeatures) { - multiclassWithCategoricalBinSeqOp(agg, treePoint, nodeIndex) - } else { - binaryOrNotCategoricalBinSeqOp(agg, treePoint, nodeIndex) - } + if (metadata.unorderedFeatures.isEmpty) { + orderedBinSeqOp(agg, treePoint, nodeIndex) } else { - regressionBinSeqOp(agg, treePoint, nodeIndex) + mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures) } } agg } - // Calculate bin aggregate length for classification or regression. - val binAggregateLength = numNodes * getElementsPerNode(metadata, numBins) - logDebug("binAggregateLength = " + binAggregateLength) - - /** - * Combines the aggregates from partitions. - * @param agg1 Array containing aggregates from one or more partitions - * @param agg2 Array containing aggregates from one or more partitions - * @return Combined aggregate from agg1 and agg2 - */ - def binCombOp(agg1: Array[Double], agg2: Array[Double]): Array[Double] = { - var index = 0 - val combinedAggregate = new Array[Double](binAggregateLength) - while (index < binAggregateLength) { - combinedAggregate(index) = agg1(index) + agg2(index) - index += 1 - } - combinedAggregate - } - // Calculate bin aggregates. timer.start("aggregation") - val binAggregates = { - input.treeAggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp, binCombOp) + val binAggregates: DTStatsAggregator = { + val initAgg = new DTStatsAggregator(metadata, numNodes) + input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp) } timer.stop("aggregation") - logDebug("binAggregates.length = " + binAggregates.length) - /** - * Calculate the information gain for a given (feature, split) based upon left/right aggregates. - * @param leftNodeAgg left node aggregates for this (feature, split) - * @param rightNodeAgg right node aggregate for this (feature, split) - * @param topImpurity impurity of the parent node - * @return information gain and statistics for all splits - */ - def calculateGainForSplit( - leftNodeAgg: Array[Double], - rightNodeAgg: Array[Double], - topImpurity: Double): InformationGainStats = { - if (metadata.isClassification) { - val leftTotalCount = leftNodeAgg.sum - val rightTotalCount = rightNodeAgg.sum - - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val rootNodeCounts = new Array[Double](numClasses) - var classIndex = 0 - while (classIndex < numClasses) { - rootNodeCounts(classIndex) = leftNodeAgg(classIndex) + rightNodeAgg(classIndex) - classIndex += 1 - } - metadata.impurity.calculate(rootNodeCounts, leftTotalCount + rightTotalCount) - } - } - - val totalCount = leftTotalCount + rightTotalCount - if (totalCount == 0) { - // Return arbitrary prediction. - return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) - } - - // Sum of count for each label - val leftrightNodeAgg: Array[Double] = - leftNodeAgg.zip(rightNodeAgg).map { case (leftCount, rightCount) => - leftCount + rightCount - } - - def indexOfLargestArrayElement(array: Array[Double]): Int = { - val result = array.foldLeft(-1, Double.MinValue, 0) { - case ((maxIndex, maxValue, currentIndex), currentValue) => - if (currentValue > maxValue) { - (currentIndex, currentValue, currentIndex + 1) - } else { - (maxIndex, maxValue, currentIndex + 1) - } - } - if (result._1 < 0) { - throw new RuntimeException("DecisionTree internal error:" + - " calculateGainForSplit failed in indexOfLargestArrayElement") - } - result._1 - } - - val predict = indexOfLargestArrayElement(leftrightNodeAgg) - val prob = leftrightNodeAgg(predict) / totalCount - - val leftImpurity = if (leftTotalCount == 0) { - topImpurity - } else { - metadata.impurity.calculate(leftNodeAgg, leftTotalCount) - } - val rightImpurity = if (rightTotalCount == 0) { - topImpurity - } else { - metadata.impurity.calculate(rightNodeAgg, rightTotalCount) - } - - val leftWeight = leftTotalCount / totalCount - val rightWeight = rightTotalCount / totalCount - - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) - - } else { - // Regression - - val leftCount = leftNodeAgg(0) - val leftSum = leftNodeAgg(1) - val leftSumSquares = leftNodeAgg(2) + // Calculate best splits for all nodes at a given level + timer.start("chooseSplits") + val bestSplits = new Array[(Split, InformationGainStats)](numNodes) + // Iterating over all nodes at this level + var nodeIndex = 0 + while (nodeIndex < numNodes) { + val nodeImpurity = parentImpurities(globalNodeIndexOffset + nodeIndex) + logDebug("node impurity = " + nodeImpurity) + bestSplits(nodeIndex) = + binsToBestSplit(binAggregates, nodeIndex, nodeImpurity, level, metadata, splits) + logDebug("best split = " + bestSplits(nodeIndex)._1) + nodeIndex += 1 + } + timer.stop("chooseSplits") - val rightCount = rightNodeAgg(0) - val rightSum = rightNodeAgg(1) - val rightSumSquares = rightNodeAgg(2) + bestSplits + } - val impurity = { - if (level > 0) { - topImpurity - } else { - // Calculate impurity for root node. - val count = leftCount + rightCount - val sum = leftSum + rightSum - val sumSquares = leftSumSquares + rightSumSquares - metadata.impurity.calculate(count, sum, sumSquares) - } - } + /** + * Calculate the information gain for a given (feature, split) based upon left/right aggregates. + * @param leftImpurityCalculator left node aggregates for this (feature, split) + * @param rightImpurityCalculator right node aggregate for this (feature, split) + * @param topImpurity impurity of the parent node + * @return information gain and statistics for all splits + */ + private def calculateGainForSplit( + leftImpurityCalculator: ImpurityCalculator, + rightImpurityCalculator: ImpurityCalculator, + topImpurity: Double, + level: Int, + metadata: DecisionTreeMetadata): InformationGainStats = { - if (leftCount == 0) { - return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, - rightSum / rightCount) - } - if (rightCount == 0) { - return new InformationGainStats(0, topImpurity, topImpurity, - Double.MinValue, leftSum / leftCount) - } + val leftCount = leftImpurityCalculator.count + val rightCount = rightImpurityCalculator.count - val leftImpurity = metadata.impurity.calculate(leftCount, leftSum, leftSumSquares) - val rightImpurity = metadata.impurity.calculate(rightCount, rightSum, rightSumSquares) + val totalCount = leftCount + rightCount + if (totalCount == 0) { + // Return arbitrary prediction. + return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + } - val leftWeight = leftCount.toDouble / (leftCount + rightCount) - val rightWeight = rightCount.toDouble / (leftCount + rightCount) + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) + // impurity of parent node + val impurity = if (level > 0) { + topImpurity + } else { + parentNodeAgg.calculate() + } - val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + val predict = parentNodeAgg.predict + val prob = parentNodeAgg.prob(predict) - val predict = (leftSum + rightSum) / (leftCount + rightCount) - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) - } - } + val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 + val rightImpurity = rightImpurityCalculator.calculate() - /** - * Extracts left and right split aggregates. - * @param binData Aggregate array slice from getBinDataForNode. - * For classification: - * For unordered features, this is leftChildData ++ rightChildData, - * each of which is indexed by (feature, split/bin, class), - * with class being the least significant bit. - * For ordered features, this is of size numClasses * numBins * numFeatures. - * For regression: - * This is of size 2 * numFeatures * numBins. - * @return (leftNodeAgg, rightNodeAgg) pair of arrays. - * For classification, each array is of size (numFeatures, (numBins - 1), numClasses). - * For regression, each array is of size (numFeatures, (numBins - 1), 3). - * - */ - def extractLeftRightNodeAggregates( - binData: Array[Double]): (Array[Array[Array[Double]]], Array[Array[Array[Double]]]) = { - - - /** - * The input binData is indexed as (feature, bin, class). - * This computes cumulative sums over splits. - * Each (feature, class) pair is handled separately. - * Note: numSplits = numBins - 1. - * @param leftNodeAgg Each (feature, class) slice is an array over splits. - * Element i (i = 0, ..., numSplits - 2) is set to be - * the cumulative sum (from left) over binData for bins 0, ..., i. - * @param rightNodeAgg Each (feature, class) slice is an array over splits. - * Element i (i = 1, ..., numSplits - 1) is set to be - * the cumulative sum (from right) over binData for bins - * numBins - 1, ..., numBins - 1 - i. - */ - def findAggForOrderedFeatureClassification( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - // shift for this featureIndex - val shift = numClasses * featureIndex * numBins - - var classIndex = 0 - while (classIndex < numClasses) { - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0)(classIndex) = binData(shift + classIndex) - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(numBins - 2)(classIndex) - = binData(shift + (numClasses * (numBins - 1)) + classIndex) - classIndex += 1 - } + val leftWeight = leftCount / totalCount.toDouble + val rightWeight = rightCount / totalCount.toDouble - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - var innerClassIndex = 0 - while (innerClassIndex < numClasses) { - leftNodeAgg(featureIndex)(splitIndex)(innerClassIndex) - = binData(shift + numClasses * splitIndex + innerClassIndex) + - leftNodeAgg(featureIndex)(splitIndex - 1)(innerClassIndex) - rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(innerClassIndex) = - binData(shift + (numClasses * (numBins - 1 - splitIndex) + innerClassIndex)) + - rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(innerClassIndex) - innerClassIndex += 1 - } - splitIndex += 1 - } - } + val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - /** - * Reshape binData for this feature. - * Indexes binData as (feature, split, class) with class as the least significant bit. - * @param leftNodeAgg leftNodeAgg(featureIndex)(splitIndex)(classIndex) = aggregate value - */ - def findAggForUnorderedFeatureClassification( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - val rightChildShift = numClasses * numBins * numFeatures - var splitIndex = 0 - while (splitIndex < numBins - 1) { - var classIndex = 0 - while (classIndex < numClasses) { - // shift for this featureIndex - val shift = numClasses * featureIndex * numBins + splitIndex * numClasses - val leftBinValue = binData(shift + classIndex) - val rightBinValue = binData(rightChildShift + shift + classIndex) - leftNodeAgg(featureIndex)(splitIndex)(classIndex) = leftBinValue - rightNodeAgg(featureIndex)(splitIndex)(classIndex) = rightBinValue - classIndex += 1 - } - splitIndex += 1 - } - } + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + } - def findAggForRegression( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - featureIndex: Int) { - - // shift for this featureIndex - val shift = 3 * featureIndex * numBins - // left node aggregate for the lowest split - leftNodeAgg(featureIndex)(0)(0) = binData(shift + 0) - leftNodeAgg(featureIndex)(0)(1) = binData(shift + 1) - leftNodeAgg(featureIndex)(0)(2) = binData(shift + 2) - - // right node aggregate for the highest split - rightNodeAgg(featureIndex)(numBins - 2)(0) = - binData(shift + (3 * (numBins - 1))) - rightNodeAgg(featureIndex)(numBins - 2)(1) = - binData(shift + (3 * (numBins - 1)) + 1) - rightNodeAgg(featureIndex)(numBins - 2)(2) = - binData(shift + (3 * (numBins - 1)) + 2) - - // Iterate over all splits. - var splitIndex = 1 - while (splitIndex < numBins - 1) { - var i = 0 // index for regression histograms - while (i < 3) { // count, sum, sum^2 - // calculating left node aggregate for a split as a sum of left node aggregate of a - // lower split and the left bin aggregate of a bin where the split is a high split - leftNodeAgg(featureIndex)(splitIndex)(i) = binData(shift + 3 * splitIndex + i) + - leftNodeAgg(featureIndex)(splitIndex - 1)(i) - // calculating right node aggregate for a split as a sum of right node aggregate of a - // higher split and the right bin aggregate of a bin where the split is a low split - rightNodeAgg(featureIndex)(numBins - 2 - splitIndex)(i) = - binData(shift + (3 * (numBins - 1 - splitIndex) + i)) + - rightNodeAgg(featureIndex)(numBins - 1 - splitIndex)(i) - i += 1 - } - splitIndex += 1 - } - } + /** + * Find the best split for a node. + * @param binAggregates Bin statistics. + * @param nodeIndex Index for node to split in this (level, group). + * @param nodeImpurity Impurity of the node (nodeIndex). + * @return tuple for best split: (Split, information gain) + */ + private def binsToBestSplit( + binAggregates: DTStatsAggregator, + nodeIndex: Int, + nodeImpurity: Double, + level: Int, + metadata: DecisionTreeMetadata, + splits: Array[Array[Split]]): (Split, InformationGainStats) = { - if (metadata.isClassification) { - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, numClasses) - var featureIndex = 0 - while (featureIndex < numFeatures) { - if (metadata.isUnordered(featureIndex)) { - findAggForUnorderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } else { - findAggForOrderedFeatureClassification(leftNodeAgg, rightNodeAgg, featureIndex) - } - featureIndex += 1 - } - (leftNodeAgg, rightNodeAgg) - } else { - // Regression - // Initialize left and right split aggregates. - val leftNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - val rightNodeAgg = Array.ofDim[Double](numFeatures, numBins - 1, 3) - // Iterate over all features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - findAggForRegression(leftNodeAgg, rightNodeAgg, featureIndex) - featureIndex += 1 - } - (leftNodeAgg, rightNodeAgg) - } - } + logDebug("node impurity = " + nodeImpurity) - /** - * Calculates information gain for all nodes splits. - */ - def calculateGainsForAllNodeSplits( - leftNodeAgg: Array[Array[Array[Double]]], - rightNodeAgg: Array[Array[Array[Double]]], - nodeImpurity: Double): Array[Array[InformationGainStats]] = { - val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1) - - var featureIndex = 0 - while (featureIndex < numFeatures) { - val numSplitsForFeature = getNumSplitsForFeature(featureIndex) + // For each (feature, split), calculate the gain, and select the best (feature, split). + Range(0, metadata.numFeatures).map { featureIndex => + val numSplits = metadata.numSplits(featureIndex) + if (metadata.isContinuous(featureIndex)) { + // Cumulative sum (scanLeft) of bin statistics. + // Afterwards, binAggregates for a bin is the sum of aggregates for + // that bin + all preceding bins. + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) var splitIndex = 0 - while (splitIndex < numSplitsForFeature) { - gains(featureIndex)(splitIndex) = - calculateGainForSplit(leftNodeAgg(featureIndex)(splitIndex), - rightNodeAgg(featureIndex)(splitIndex), nodeImpurity) + while (splitIndex < numSplits) { + binAggregates.mergeForNodeFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) splitIndex += 1 } - featureIndex += 1 - } - gains - } - - /** - * Get the number of splits for a feature. - */ - def getNumSplitsForFeature(featureIndex: Int): Int = { - if (metadata.isContinuous(featureIndex)) { - numBins - 1 + // Find best split. + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { case splitIdx => + val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) + val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) + rightChildStats.subtract(leftChildStats) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIdx, gainStats) + }.maxBy(_._2.gain) + (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) + } else if (metadata.isUnordered(featureIndex)) { + // Unordered categorical feature + val (leftChildOffset, rightChildOffset) = + binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { splitIndex => + val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) + val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIndex, gainStats) + }.maxBy(_._2.gain) + (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else { - // Categorical feature - val featureCategories = metadata.featureArity(featureIndex) - if (metadata.isUnordered(featureIndex)) { - (1 << featureCategories - 1) - 1 - } else { - featureCategories - } - } - } - - /** - * Find the best split for a node. - * @param binData Bin data slice for this node, given by getBinDataForNode. - * @param nodeImpurity impurity of the top node - * @return tuple of split and information gain - */ - def binsToBestSplit( - binData: Array[Double], - nodeImpurity: Double): (Split, InformationGainStats) = { - - logDebug("node impurity = " + nodeImpurity) - - // Extract left right node aggregates. - val (leftNodeAgg, rightNodeAgg) = extractLeftRightNodeAggregates(binData) - - // Calculate gains for all splits. - val gains = calculateGainsForAllNodeSplits(leftNodeAgg, rightNodeAgg, nodeImpurity) - - val (bestFeatureIndex, bestSplitIndex, gainStats) = { - // Initialize with infeasible values. - var bestFeatureIndex = Int.MinValue - var bestSplitIndex = Int.MinValue - var bestGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0, -1.0) - // Iterate over features. - var featureIndex = 0 - while (featureIndex < numFeatures) { - // Iterate over all splits. - var splitIndex = 0 - val numSplitsForFeature = getNumSplitsForFeature(featureIndex) - while (splitIndex < numSplitsForFeature) { - val gainStats = gains(featureIndex)(splitIndex) - if (gainStats.gain > bestGainStats.gain) { - bestGainStats = gainStats - bestFeatureIndex = featureIndex - bestSplitIndex = splitIndex + // Ordered categorical feature + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) + val numBins = metadata.numBins(featureIndex) + + /* Each bin is one category (feature value). + * The bins are ordered based on centroidForCategories, and this ordering determines which + * splits are considered. (With K categories, we consider K - 1 possible splits.) + * + * centroidForCategories is a list: (category, centroid) + */ + val centroidForCategories = if (metadata.isMulticlass) { + // For categorical variables in multiclass classification, + // the bins are ordered by the impurity of their corresponding labels. + Range(0, numBins).map { case featureValue => + val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val centroid = if (categoryStats.count != 0) { + categoryStats.calculate() + } else { + Double.MaxValue } - splitIndex += 1 + (featureValue, centroid) + } + } else { // regression or binary classification + // For categorical variables in regression and binary classification, + // the bins are ordered by the centroid of their corresponding labels. + Range(0, numBins).map { case featureValue => + val categoryStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val centroid = if (categoryStats.count != 0) { + categoryStats.predict + } else { + Double.MaxValue + } + (featureValue, centroid) } - featureIndex += 1 } - (bestFeatureIndex, bestSplitIndex, bestGainStats) - } - logDebug("best split = " + splits(bestFeatureIndex)(bestSplitIndex)) - logDebug("best split bin = " + bins(bestFeatureIndex)(bestSplitIndex)) + logDebug("Centroids for categorical variable: " + centroidForCategories.mkString(",")) - (splits(bestFeatureIndex)(bestSplitIndex), gainStats) - } + // bins sorted by centroids + val categoriesSortedByCentroid = centroidForCategories.toList.sortBy(_._2) - /** - * Get bin data for one node. - */ - def getBinDataForNode(node: Int): Array[Double] = { - if (metadata.isClassification) { - if (isMulticlassWithCategoricalFeatures) { - val shift = numClasses * node * numBins * numFeatures - val rightChildShift = numClasses * numBins * numFeatures * numNodes - val binsForNode = { - val leftChildData - = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - val rightChildData - = binAggregates.slice(rightChildShift + shift, - rightChildShift + shift + numClasses * numBins * numFeatures) - leftChildData ++ rightChildData - } - binsForNode - } else { - val shift = numClasses * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + numClasses * numBins * numFeatures) - binsForNode + logDebug("Sorted centroids for categorical variable = " + + categoriesSortedByCentroid.mkString(",")) + + // Cumulative sum (scanLeft) of bin statistics. + // Afterwards, binAggregates for a bin is the sum of aggregates for + // that bin + all preceding bins. + var splitIndex = 0 + while (splitIndex < numSplits) { + val currentCategory = categoriesSortedByCentroid(splitIndex)._1 + val nextCategory = categoriesSortedByCentroid(splitIndex + 1)._1 + binAggregates.mergeForNodeFeature(nodeFeatureOffset, nextCategory, currentCategory) + splitIndex += 1 } - } else { - // Regression - val shift = 3 * node * numBins * numFeatures - val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) - binsForNode + // lastCategory = index of bin with total aggregates for this (node, feature) + val lastCategory = categoriesSortedByCentroid.last._1 + // Find best split. + val (bestFeatureSplitIndex, bestFeatureGainStats) = + Range(0, numSplits).map { splitIndex => + val featureValue = categoriesSortedByCentroid(splitIndex)._1 + val leftChildStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, featureValue) + val rightChildStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) + rightChildStats.subtract(leftChildStats) + val gainStats = + calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + (splitIndex, gainStats) + }.maxBy(_._2.gain) + val categoriesForSplit = + categoriesSortedByCentroid.map(_._1.toDouble).slice(0, bestFeatureSplitIndex + 1) + val bestFeatureSplit = + new Split(featureIndex, Double.MinValue, Categorical, categoriesForSplit) + (bestFeatureSplit, bestFeatureGainStats) } - } - - // Calculate best splits for all nodes at a given level - timer.start("chooseSplits") - val bestSplits = new Array[(Split, InformationGainStats)](numNodes) - // Iterating over all nodes at this level - var node = 0 - while (node < numNodes) { - val nodeImpurityIndex = (1 << level) - 1 + node + groupShift - val binsForNode: Array[Double] = getBinDataForNode(node) - logDebug("nodeImpurityIndex = " + nodeImpurityIndex) - val parentNodeImpurity = parentImpurities(nodeImpurityIndex) - logDebug("parent node impurity = " + parentNodeImpurity) - bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity) - node += 1 - } - timer.stop("chooseSplits") - - bestSplits + }.maxBy(_._2.gain) } /** * Get the number of values to be stored per node in the bin aggregates. - * - * @param numBins Number of bins = 1 + number of possible splits. */ - private def getElementsPerNode(metadata: DecisionTreeMetadata, numBins: Int): Int = { + private def getElementsPerNode(metadata: DecisionTreeMetadata): Int = { + val totalBins = metadata.numBins.sum if (metadata.isClassification) { - if (metadata.isMulticlassWithCategoricalFeatures) { - 2 * metadata.numClasses * numBins * metadata.numFeatures - } else { - metadata.numClasses * numBins * metadata.numFeatures - } + metadata.numClasses * totalBins } else { - 3 * numBins * metadata.numFeatures + 3 * totalBins } } @@ -1284,6 +919,7 @@ object DecisionTree extends Serializable with Logging { * Continuous features: * For each feature, there are numBins - 1 possible splits representing the possible binary * decisions at each node in the tree. + * This finds locations (feature values) for splits using a subsample of the data. * * Categorical features: * For each feature, there is 1 bin per split. @@ -1292,7 +928,6 @@ object DecisionTree extends Serializable with Logging { * For multiclass classification with a low-arity feature * (i.e., if isMulticlass && isSpaceSufficientForAllCategoricalSplits), * the feature is split based on subsets of categories. - * There are (1 << maxFeatureValue - 1) - 1 splits. * (b) "ordered features" * For regression and binary classification, * and for multiclass classification with a high-arity feature, @@ -1302,7 +937,7 @@ object DecisionTree extends Serializable with Logging { * @param metadata Learning and dataset metadata * @return A tuple of (splits, bins). * Splits is an Array of [[org.apache.spark.mllib.tree.model.Split]] - * of size (numFeatures, numBins - 1). + * of size (numFeatures, numSplits). * Bins is an Array of [[org.apache.spark.mllib.tree.model.Bin]] * of size (numFeatures, numBins). */ @@ -1310,84 +945,80 @@ object DecisionTree extends Serializable with Logging { input: RDD[LabeledPoint], metadata: DecisionTreeMetadata): (Array[Array[Split]], Array[Array[Bin]]) = { - val count = input.count() + logDebug("isMulticlass = " + metadata.isMulticlass) - // Find the number of features by looking at the first sample - val numFeatures = input.take(1)(0).features.size - - val maxBins = metadata.maxBins - val numBins = if (maxBins <= count) maxBins else count.toInt - logDebug("numBins = " + numBins) - val isMulticlass = metadata.isMulticlass - logDebug("isMulticlass = " + isMulticlass) - - /* - * Ensure numBins is always greater than the categories. For multiclass classification, - * numBins should be greater than 2^(maxCategories - 1) - 1. - * It's a limitation of the current implementation but a reasonable trade-off since features - * with large number of categories get favored over continuous features. - * - * This needs to be checked here instead of in Strategy since numBins can be determined - * by the number of training examples. - * TODO: Allow this case, where we simply will know nothing about some categories. - */ - if (metadata.featureArity.size > 0) { - val maxCategoriesForFeatures = metadata.featureArity.maxBy(_._2)._2 - require(numBins > maxCategoriesForFeatures, "numBins should be greater than max categories " + - "in categorical features") - } - - // Calculate the number of sample for approximate quantile calculation. - val requiredSamples = numBins*numBins - val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0 - logDebug("fraction of data used for calculating quantiles = " + fraction) + val numFeatures = metadata.numFeatures - // sampled input for RDD calculation - val sampledInput = + // Sample the input only if there are continuous features. + val hasContinuousFeatures = Range(0, numFeatures).exists(metadata.isContinuous) + val sampledInput = if (hasContinuousFeatures) { + // Calculate the number of samples for approximate quantile calculation. + val requiredSamples = math.max(metadata.maxBins * metadata.maxBins, 10000) + val fraction = if (requiredSamples < metadata.numExamples) { + requiredSamples.toDouble / metadata.numExamples + } else { + 1.0 + } + logDebug("fraction of data used for calculating quantiles = " + fraction) input.sample(withReplacement = false, fraction, new XORShiftRandom().nextInt()).collect() - val numSamples = sampledInput.length - - val stride: Double = numSamples.toDouble / numBins - logDebug("stride = " + stride) + } else { + new Array[LabeledPoint](0) + } metadata.quantileStrategy match { case Sort => - val splits = Array.ofDim[Split](numFeatures, numBins - 1) - val bins = Array.ofDim[Bin](numFeatures, numBins) + val splits = new Array[Array[Split]](numFeatures) + val bins = new Array[Array[Bin]](numFeatures) // Find all splits. - // Iterate over all features. var featureIndex = 0 while (featureIndex < numFeatures) { - // Check whether the feature is continuous. - val isFeatureContinuous = metadata.isContinuous(featureIndex) - if (isFeatureContinuous) { + val numSplits = metadata.numSplits(featureIndex) + val numBins = metadata.numBins(featureIndex) + if (metadata.isContinuous(featureIndex)) { + val numSamples = sampledInput.length + splits(featureIndex) = new Array[Split](numSplits) + bins(featureIndex) = new Array[Bin](numBins) val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted - val stride: Double = numSamples.toDouble / numBins + val stride: Double = numSamples.toDouble / metadata.numBins(featureIndex) logDebug("stride = " + stride) - for (index <- 0 until numBins - 1) { - val sampleIndex = index * stride.toInt + for (splitIndex <- 0 until numSplits) { + val sampleIndex = splitIndex * stride.toInt // Set threshold halfway in between 2 samples. val threshold = (featureSamples(sampleIndex) + featureSamples(sampleIndex + 1)) / 2.0 - val split = new Split(featureIndex, threshold, Continuous, List()) - splits(featureIndex)(index) = split + splits(featureIndex)(splitIndex) = + new Split(featureIndex, threshold, Continuous, List()) } - } else { // Categorical feature - val featureCategories = metadata.featureArity(featureIndex) - - // Use different bin/split calculation strategy for categorical features in multiclass - // classification that satisfy the space constraint. + bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), + splits(featureIndex)(0), Continuous, Double.MinValue) + for (splitIndex <- 1 until numSplits) { + bins(featureIndex)(splitIndex) = + new Bin(splits(featureIndex)(splitIndex - 1), splits(featureIndex)(splitIndex), + Continuous, Double.MinValue) + } + bins(featureIndex)(numSplits) = new Bin(splits(featureIndex)(numSplits - 1), + new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) + } else { + // Categorical feature + val featureArity = metadata.featureArity(featureIndex) if (metadata.isUnordered(featureIndex)) { - // 2^(maxFeatureValue- 1) - 1 combinations - var index = 0 - while (index < (1 << featureCategories - 1) - 1) { - val categories: List[Double] - = extractMultiClassCategories(index + 1, featureCategories) - splits(featureIndex)(index) - = new Split(featureIndex, Double.MinValue, Categorical, categories) - bins(featureIndex)(index) = { - if (index == 0) { + // TODO: The second half of the bins are unused. Actually, we could just use + // splits and not build bins for unordered features. That should be part of + // a later PR since it will require changing other code (using splits instead + // of bins in a few places). + // Unordered features + // 2^(maxFeatureValue - 1) - 1 combinations + splits(featureIndex) = new Array[Split](numSplits) + bins(featureIndex) = new Array[Bin](numBins) + var splitIndex = 0 + while (splitIndex < numSplits) { + val categories: List[Double] = + extractMultiClassCategories(splitIndex + 1, featureArity) + splits(featureIndex)(splitIndex) = + new Split(featureIndex, Double.MinValue, Categorical, categories) + bins(featureIndex)(splitIndex) = { + if (splitIndex == 0) { new Bin( new DummyCategoricalSplit(featureIndex, Categorical), splits(featureIndex)(0), @@ -1395,96 +1026,24 @@ object DecisionTree extends Serializable with Logging { Double.MinValue) } else { new Bin( - splits(featureIndex)(index - 1), - splits(featureIndex)(index), + splits(featureIndex)(splitIndex - 1), + splits(featureIndex)(splitIndex), Categorical, Double.MinValue) } } - index += 1 - } - } else { // ordered feature - /* For a given categorical feature, use a subsample of the data - * to choose how to arrange possible splits. - * This examines each category and computes a centroid. - * These centroids are later used to sort the possible splits. - * centroidForCategories is a mapping: category (for the given feature) --> centroid - */ - val centroidForCategories = { - if (isMulticlass) { - // For categorical variables in multiclass classification, - // each bin is a category. The bins are sorted and they - // are ordered by calculating the impurity of their corresponding labels. - sampledInput.map(lp => (lp.features(featureIndex), lp.label)) - .groupBy(_._1) - .mapValues(x => x.groupBy(_._2).mapValues(x => x.size.toDouble)) - .map(x => (x._1, x._2.values.toArray)) - .map(x => (x._1, metadata.impurity.calculate(x._2, x._2.sum))) - } else { // regression or binary classification - // For categorical variables in regression and binary classification, - // each bin is a category. The bins are sorted and they - // are ordered by calculating the centroid of their corresponding labels. - sampledInput.map(lp => (lp.features(featureIndex), lp.label)) - .groupBy(_._1) - .mapValues(x => x.map(_._2).sum / x.map(_._1).length) - } - } - - logDebug("centroid for categories = " + centroidForCategories.mkString(",")) - - // Check for missing categorical variables and putting them last in the sorted list. - val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() - for (i <- 0 until featureCategories) { - if (centroidForCategories.contains(i)) { - fullCentroidForCategories(i) = centroidForCategories(i) - } else { - fullCentroidForCategories(i) = Double.MaxValue - } - } - - // bins sorted by centroids - val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) - - logDebug("centroid for categorical variable = " + categoriesSortedByCentroid) - - var categoriesForSplit = List[Double]() - categoriesSortedByCentroid.iterator.zipWithIndex.foreach { - case ((key, value), index) => - categoriesForSplit = key :: categoriesForSplit - splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, - Categorical, categoriesForSplit) - bins(featureIndex)(index) = { - if (index == 0) { - new Bin(new DummyCategoricalSplit(featureIndex, Categorical), - splits(featureIndex)(0), Categorical, key) - } else { - new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Categorical, key) - } - } + splitIndex += 1 } + } else { + // Ordered features + // Bins correspond to feature values, so we do not need to compute splits or bins + // beforehand. Splits are constructed as needed during training. + splits(featureIndex) = new Array[Split](0) + bins(featureIndex) = new Array[Bin](0) } } featureIndex += 1 } - - // Find all bins. - featureIndex = 0 - while (featureIndex < numFeatures) { - val isFeatureContinuous = metadata.isContinuous(featureIndex) - if (isFeatureContinuous) { // Bins for categorical variables are already assigned. - bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), - splits(featureIndex)(0), Continuous, Double.MinValue) - for (index <- 1 until numBins - 1) { - val bin = new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), - Continuous, Double.MinValue) - bins(featureIndex)(index) = bin - } - bins(featureIndex)(numBins-1) = new Bin(splits(featureIndex)(numBins-2), - new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) - } - featureIndex += 1 - } (splits, bins) case MinMax => throw new UnsupportedOperationException("minmax not supported yet.") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala new file mode 100644 index 0000000000000..866d85a79bea1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -0,0 +1,213 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import org.apache.spark.mllib.tree.impurity._ + +/** + * DecisionTree statistics aggregator. + * This holds a flat array of statistics for a set of (nodes, features, bins) + * and helps with indexing. + */ +private[tree] class DTStatsAggregator( + val metadata: DecisionTreeMetadata, + val numNodes: Int) extends Serializable { + + /** + * [[ImpurityAggregator]] instance specifying the impurity type. + */ + val impurityAggregator: ImpurityAggregator = metadata.impurity match { + case Gini => new GiniAggregator(metadata.numClasses) + case Entropy => new EntropyAggregator(metadata.numClasses) + case Variance => new VarianceAggregator() + case _ => throw new IllegalArgumentException(s"Bad impurity parameter: ${metadata.impurity}") + } + + /** + * Number of elements (Double values) used for the sufficient statistics of each bin. + */ + val statsSize: Int = impurityAggregator.statsSize + + val numFeatures: Int = metadata.numFeatures + + /** + * Number of bins for each feature. This is indexed by the feature index. + */ + val numBins: Array[Int] = metadata.numBins + + /** + * Number of splits for the given feature. + */ + def numSplits(featureIndex: Int): Int = metadata.numSplits(featureIndex) + + /** + * Indicator for each feature of whether that feature is an unordered feature. + * TODO: Is Array[Boolean] any faster? + */ + def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) + + /** + * Offset for each feature for calculating indices into the [[allStats]] array. + */ + private val featureOffsets: Array[Int] = { + def featureOffsetsCalc(total: Int, featureIndex: Int): Int = { + if (isUnordered(featureIndex)) { + total + 2 * numBins(featureIndex) + } else { + total + numBins(featureIndex) + } + } + Range(0, numFeatures).scanLeft(0)(featureOffsetsCalc).map(statsSize * _).toArray + } + + /** + * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. + */ + private val nodeStride: Int = featureOffsets.last + + /** + * Total number of elements stored in this aggregator. + */ + val allStatsSize: Int = numNodes * nodeStride + + /** + * Flat array of elements. + * Index for start of stats for a (node, feature, bin) is: + * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, the left child stats have binIndex in [0, numBins(featureIndex)) + * and the right child stats in [numBins(featureIndex), 2 * numBins(featureIndex)) + */ + val allStats: Array[Double] = new Array[Double](allStatsSize) + + /** + * Get an [[ImpurityCalculator]] for a given (node, feature, bin). + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + */ + def getImpurityCalculator(nodeFeatureOffset: Int, binIndex: Int): ImpurityCalculator = { + impurityAggregator.getCalculator(allStats, nodeFeatureOffset + binIndex * statsSize) + } + + /** + * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + */ + def update(nodeIndex: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { + val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label) + } + + /** + * Pre-compute node offset for use with [[nodeUpdate]]. + */ + def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride + + /** + * Faster version of [[update]]. + * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. + */ + def nodeUpdate(nodeOffset: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { + val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label) + } + + /** + * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * For ordered features only. + */ + def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { + require(!isUnordered(featureIndex), + s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" + + s" for unordered feature $featureIndex.") + nodeIndex * nodeStride + featureOffsets(featureIndex) + } + + /** + * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * For unordered features only. + */ + def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = { + require(isUnordered(featureIndex), + s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + + s" but was called for ordered feature $featureIndex.") + val baseOffset = nodeIndex * nodeStride + featureOffsets(featureIndex) + (baseOffset, baseOffset + numBins(featureIndex) * statsSize) + } + + /** + * Faster version of [[update]]. + * Update the stats for a given (node, feature, bin), using the given label. + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + */ + def nodeFeatureUpdate(nodeFeatureOffset: Int, binIndex: Int, label: Double): Unit = { + impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label) + } + + /** + * For a given (node, feature), merge the stats for two bins. + * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getNodeFeatureOffset]]. + * For unordered features, this is a pre-computed + * (node, feature, left/right child) offset from + * [[getLeftRightNodeFeatureOffsets]]. + * @param binIndex The other bin is merged into this bin. + * @param otherBinIndex This bin is not modified. + */ + def mergeForNodeFeature(nodeFeatureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { + impurityAggregator.merge(allStats, nodeFeatureOffset + binIndex * statsSize, + nodeFeatureOffset + otherBinIndex * statsSize) + } + + /** + * Merge this aggregator with another, and returns this aggregator. + * This method modifies this aggregator in-place. + */ + def merge(other: DTStatsAggregator): DTStatsAggregator = { + require(allStatsSize == other.allStatsSize, + s"DTStatsAggregator.merge requires that both aggregators have the same length stats vectors." + + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") + var i = 0 + // TODO: Test BLAS.axpy + while (i < allStatsSize) { + allStats(i) += other.allStats(i) + i += 1 + } + this + } + +} + +private[tree] object DTStatsAggregator extends Serializable { + + /** + * Combines two aggregates (modifying the first) and returns the combination. + */ + def binCombOp( + agg1: DTStatsAggregator, + agg2: DTStatsAggregator): DTStatsAggregator = { + agg1.merge(agg2) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index d9eda354dc986..e95add7558bcf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -26,14 +26,15 @@ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.rdd.RDD - /** * Learning and dataset metadata for DecisionTree. * * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. * For regression: fixed at 0 (no meaning). + * @param maxBins Maximum number of bins, for all features. * @param featureArity Map: categorical feature index --> arity. * I.e., the feature takes values in {0, ..., arity - 1}. + * @param numBins Number of bins for each feature. */ private[tree] class DecisionTreeMetadata( val numFeatures: Int, @@ -42,6 +43,7 @@ private[tree] class DecisionTreeMetadata( val maxBins: Int, val featureArity: Map[Int, Int], val unorderedFeatures: Set[Int], + val numBins: Array[Int], val impurity: Impurity, val quantileStrategy: QuantileStrategy) extends Serializable { @@ -57,10 +59,26 @@ private[tree] class DecisionTreeMetadata( def isContinuous(featureIndex: Int): Boolean = !featureArity.contains(featureIndex) + /** + * Number of splits for the given feature. + * For unordered features, there are 2 bins per split. + * For ordered features, there is 1 more bin than split. + */ + def numSplits(featureIndex: Int): Int = if (isUnordered(featureIndex)) { + numBins(featureIndex) >> 1 + } else { + numBins(featureIndex) - 1 + } + } private[tree] object DecisionTreeMetadata { + /** + * Construct a [[DecisionTreeMetadata]] instance for this dataset and parameters. + * This computes which categorical features will be ordered vs. unordered, + * as well as the number of splits and bins for each feature. + */ def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { val numFeatures = input.take(1)(0).features.size @@ -70,32 +88,55 @@ private[tree] object DecisionTreeMetadata { case Regression => 0 } - val maxBins = math.min(strategy.maxBins, numExamples).toInt - val log2MaxBinsp1 = math.log(maxBins + 1) / math.log(2.0) + val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt + + // We check the number of bins here against maxPossibleBins. + // This needs to be checked here instead of in Strategy since maxPossibleBins can be modified + // based on the number of training examples. + if (strategy.categoricalFeaturesInfo.nonEmpty) { + val maxCategoriesPerFeature = strategy.categoricalFeaturesInfo.values.max + require(maxCategoriesPerFeature <= maxPossibleBins, + s"DecisionTree requires maxBins (= $maxPossibleBins) >= max categories " + + s"in categorical features (= $maxCategoriesPerFeature)") + } val unorderedFeatures = new mutable.HashSet[Int]() + val numBins = Array.fill[Int](numFeatures)(maxPossibleBins) if (numClasses > 2) { - strategy.categoricalFeaturesInfo.foreach { case (f, k) => - if (k - 1 < log2MaxBinsp1) { - // Note: The above check is equivalent to checking: - // numUnorderedBins = (1 << k - 1) - 1 < maxBins - unorderedFeatures.add(f) + // Multiclass classification + val maxCategoriesForUnorderedFeature = + ((math.log(maxPossibleBins / 2 + 1) / math.log(2.0)) + 1).floor.toInt + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + // Decide if some categorical features should be treated as unordered features, + // which require 2 * ((1 << numCategories - 1) - 1) bins. + // We do this check with log values to prevent overflows in case numCategories is large. + // The next check is equivalent to: 2 * ((1 << numCategories - 1) - 1) <= maxBins + if (numCategories <= maxCategoriesForUnorderedFeature) { + unorderedFeatures.add(featureIndex) + numBins(featureIndex) = numUnorderedBins(numCategories) } else { - // TODO: Allow this case, where we simply will know nothing about some categories? - require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + - s"in categorical features (>= $k)") + numBins(featureIndex) = numCategories } } } else { - strategy.categoricalFeaturesInfo.foreach { case (f, k) => - require(k < maxBins, s"maxBins (= $maxBins) should be greater than max categories " + - s"in categorical features (>= $k)") + // Binary classification or regression + strategy.categoricalFeaturesInfo.foreach { case (featureIndex, numCategories) => + numBins(featureIndex) = numCategories } } - new DecisionTreeMetadata(numFeatures, numExamples, numClasses, maxBins, - strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, + strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, strategy.impurity, strategy.quantileCalculationStrategy) } + /** + * Given the arity of a categorical feature (arity = number of categories), + * return the number of bins for the feature if it is to be treated as an unordered feature. + * There is 1 split for every partitioning of categories into 2 disjoint, non-empty sets; + * there are math.pow(2, arity - 1) - 1 such splits. + * Each split has 2 corresponding bins. + */ + def numUnorderedBins(arity: Int): Int = 2 * ((1 << arity - 1) - 1) + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala index 170e43e222083..35e361ae309cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/TreePoint.scala @@ -48,54 +48,63 @@ private[tree] object TreePoint { * binning feature values in preparation for DecisionTree training. * @param input Input dataset. * @param bins Bins for features, of size (numFeatures, numBins). - * @param metadata Learning and dataset metadata + * @param metadata Learning and dataset metadata * @return TreePoint dataset representation */ def convertToTreeRDD( input: RDD[LabeledPoint], bins: Array[Array[Bin]], metadata: DecisionTreeMetadata): RDD[TreePoint] = { + // Construct arrays for featureArity and isUnordered for efficiency in the inner loop. + val featureArity: Array[Int] = new Array[Int](metadata.numFeatures) + val isUnordered: Array[Boolean] = new Array[Boolean](metadata.numFeatures) + var featureIndex = 0 + while (featureIndex < metadata.numFeatures) { + featureArity(featureIndex) = metadata.featureArity.getOrElse(featureIndex, 0) + isUnordered(featureIndex) = metadata.isUnordered(featureIndex) + featureIndex += 1 + } input.map { x => - TreePoint.labeledPointToTreePoint(x, bins, metadata) + TreePoint.labeledPointToTreePoint(x, bins, featureArity, isUnordered) } } /** * Convert one LabeledPoint into its TreePoint representation. * @param bins Bins for features, of size (numFeatures, numBins). + * @param featureArity Array indexed by feature, with value 0 for continuous and numCategories + * for categorical features. + * @param isUnordered Array index by feature, with value true for unordered categorical features. */ private def labeledPointToTreePoint( labeledPoint: LabeledPoint, bins: Array[Array[Bin]], - metadata: DecisionTreeMetadata): TreePoint = { - + featureArity: Array[Int], + isUnordered: Array[Boolean]): TreePoint = { val numFeatures = labeledPoint.features.size - val numBins = bins(0).size val arr = new Array[Int](numFeatures) var featureIndex = 0 while (featureIndex < numFeatures) { - arr(featureIndex) = findBin(featureIndex, labeledPoint, metadata.isContinuous(featureIndex), - metadata.isUnordered(featureIndex), bins, metadata.featureArity) + arr(featureIndex) = findBin(featureIndex, labeledPoint, featureArity(featureIndex), + isUnordered(featureIndex), bins) featureIndex += 1 } - new TreePoint(labeledPoint.label, arr) } /** * Find bin for one (labeledPoint, feature). * + * @param featureArity 0 for continuous features; number of categories for categorical features. * @param isUnorderedFeature (only applies if feature is categorical) * @param bins Bins for features, of size (numFeatures, numBins). - * @param categoricalFeaturesInfo Map over categorical features: feature index --> feature arity */ private def findBin( featureIndex: Int, labeledPoint: LabeledPoint, - isFeatureContinuous: Boolean, + featureArity: Int, isUnorderedFeature: Boolean, - bins: Array[Array[Bin]], - categoricalFeaturesInfo: Map[Int, Int]): Int = { + bins: Array[Array[Bin]]): Int = { /** * Binary search helper method for continuous feature. @@ -121,44 +130,7 @@ private[tree] object TreePoint { -1 } - /** - * Sequential search helper method to find bin for categorical feature in multiclass - * classification. The category is returned since each category can belong to multiple - * splits. The actual left/right child allocation per split is performed in the - * sequential phase of the bin aggregate operation. - */ - def sequentialBinSearchForUnorderedCategoricalFeatureInClassification(): Int = { - labeledPoint.features(featureIndex).toInt - } - - /** - * Sequential search helper method to find bin for categorical feature - * (for classification and regression). - */ - def sequentialBinSearchForOrderedCategoricalFeature(): Int = { - val featureCategories = categoricalFeaturesInfo(featureIndex) - val featureValue = labeledPoint.features(featureIndex) - var binIndex = 0 - while (binIndex < featureCategories) { - val bin = bins(featureIndex)(binIndex) - val categories = bin.highSplit.categories - if (categories.contains(featureValue)) { - return binIndex - } - binIndex += 1 - } - if (featureValue < 0 || featureValue >= featureCategories) { - throw new IllegalArgumentException( - s"DecisionTree given invalid data:" + - s" Feature $featureIndex is categorical with values in" + - s" {0,...,${featureCategories - 1}," + - s" but a data point gives it value $featureValue.\n" + - " Bad data point: " + labeledPoint.toString) - } - -1 - } - - if (isFeatureContinuous) { + if (featureArity == 0) { // Perform binary search for finding bin for continuous features. val binIndex = binarySearchForBins() if (binIndex == -1) { @@ -168,18 +140,17 @@ private[tree] object TreePoint { } binIndex } else { - // Perform sequential search to find bin for categorical features. - val binIndex = if (isUnorderedFeature) { - sequentialBinSearchForUnorderedCategoricalFeatureInClassification() - } else { - sequentialBinSearchForOrderedCategoricalFeature() - } - if (binIndex == -1) { - throw new RuntimeException("No bin was found for categorical feature." + - " This error can occur when given invalid data values (such as NaN)." + - s" Feature index: $featureIndex. Feature value: ${labeledPoint.features(featureIndex)}") + // Categorical feature bins are indexed by feature values. + val featureValue = labeledPoint.features(featureIndex) + if (featureValue < 0 || featureValue >= featureArity) { + throw new IllegalArgumentException( + s"DecisionTree given invalid data:" + + s" Feature $featureIndex is categorical with values in" + + s" {0,...,${featureArity - 1}," + + s" but a data point gives it value $featureValue.\n" + + " Bad data point: " + labeledPoint.toString) } - binIndex + featureValue.toInt } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 96d2471e1f88c..1c8afc2d0f4bc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -74,3 +74,87 @@ object Entropy extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param numClasses Number of classes for label. + */ +private[tree] class EntropyAggregator(numClasses: Int) + extends ImpurityAggregator(numClasses) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"EntropyAggregator given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + allStats(offset + label.toInt) += 1 + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): EntropyCalculator = { + new EntropyCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[EntropyAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class EntropyCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: EntropyCalculator = new EntropyCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Entropy.calculate(stats, stats.sum) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats.sum.toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + indexOfLargestArrayElement(stats) + } + + /** + * Probability of the label given by [[predict]]. + */ + override def prob(label: Double): Double = { + val lbl = label.toInt + require(lbl < stats.length, + s"EntropyCalculator.prob given invalid label: $lbl (should be < ${stats.length}") + val cnt = count + if (cnt == 0) { + 0 + } else { + stats(lbl) / cnt + } + } + + override def toString: String = s"EntropyCalculator(stats = [${stats.mkString(", ")}])" + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index d586f449048bb..5cfdf345d163c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -70,3 +70,87 @@ object Gini extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param numClasses Number of classes for label. + */ +private[tree] class GiniAggregator(numClasses: Int) + extends ImpurityAggregator(numClasses) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + if (label >= statsSize) { + throw new IllegalArgumentException(s"GiniAggregator given label $label" + + s" but requires label < numClasses (= $statsSize).") + } + allStats(offset + label.toInt) += 1 + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): GiniCalculator = { + new GiniCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[GiniAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class GiniCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: GiniCalculator = new GiniCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Gini.calculate(stats, stats.sum) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats.sum.toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + indexOfLargestArrayElement(stats) + } + + /** + * Probability of the label given by [[predict]]. + */ + override def prob(label: Double): Double = { + val lbl = label.toInt + require(lbl < stats.length, + s"GiniCalculator.prob given invalid label: $lbl (should be < ${stats.length}") + val cnt = count + if (cnt == 0) { + 0 + } else { + stats(lbl) / cnt + } + } + + override def toString: String = s"GiniCalculator(stats = [${stats.mkString(", ")}])" + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 92b0c7b4a6fbc..5a047d6cb5480 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -22,6 +22,9 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: * Trait for calculating information gain. + * This trait is used for + * (a) setting the impurity parameter in [[org.apache.spark.mllib.tree.configuration.Strategy]] + * (b) calculating impurity values from sufficient statistics. */ @Experimental trait Impurity extends Serializable { @@ -47,3 +50,127 @@ trait Impurity extends Serializable { @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double } + +/** + * Interface for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + * @param statsSize Length of the vector of sufficient statistics for one bin. + */ +private[tree] abstract class ImpurityAggregator(val statsSize: Int) extends Serializable { + + /** + * Merge the stats from one bin into another. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for (node, feature, bin) which is modified by the merge. + * @param otherOffset Start index of stats for (node, feature, other bin) which is not modified. + */ + def merge(allStats: Array[Double], offset: Int, otherOffset: Int): Unit = { + var i = 0 + while (i < statsSize) { + allStats(offset + i) += allStats(otherOffset + i) + i += 1 + } + } + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): ImpurityCalculator + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[ImpurityAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] abstract class ImpurityCalculator(val stats: Array[Double]) { + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: ImpurityCalculator + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double + + /** + * Add the stats from another calculator into this one, modifying and returning this calculator. + */ + def add(other: ImpurityCalculator): ImpurityCalculator = { + require(stats.size == other.stats.size, + s"Two ImpurityCalculator instances cannot be added with different counts sizes." + + s" Sizes are ${stats.size} and ${other.stats.size}.") + var i = 0 + while (i < other.stats.size) { + stats(i) += other.stats(i) + i += 1 + } + this + } + + /** + * Subtract the stats from another calculator from this one, modifying and returning this + * calculator. + */ + def subtract(other: ImpurityCalculator): ImpurityCalculator = { + require(stats.size == other.stats.size, + s"Two ImpurityCalculator instances cannot be subtracted with different counts sizes." + + s" Sizes are ${stats.size} and ${other.stats.size}.") + var i = 0 + while (i < other.stats.size) { + stats(i) -= other.stats(i) + i += 1 + } + this + } + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double + + /** + * Probability of the label given by [[predict]], or -1 if no probability is available. + */ + def prob(label: Double): Double = -1 + + /** + * Return the index of the largest array element. + * Fails if the array is empty. + */ + protected def indexOfLargestArrayElement(array: Array[Double]): Int = { + val result = array.foldLeft(-1, Double.MinValue, 0) { + case ((maxIndex, maxValue, currentIndex), currentValue) => + if (currentValue > maxValue) { + (currentIndex, currentValue, currentIndex + 1) + } else { + (maxIndex, maxValue, currentIndex + 1) + } + } + if (result._1 < 0) { + throw new RuntimeException("ImpurityCalculator internal error:" + + " indexOfLargestArrayElement failed") + } + result._1 + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index f7d99a40eb380..e9ccecb1b8067 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -61,3 +61,75 @@ object Variance extends Impurity { def instance = this } + +/** + * Class for updating views of a vector of sufficient statistics, + * in order to compute impurity from a sample. + * Note: Instances of this class do not hold the data; they operate on views of the data. + */ +private[tree] class VarianceAggregator() + extends ImpurityAggregator(statsSize = 3) with Serializable { + + /** + * Update stats for one (node, feature, bin) with the given label. + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + allStats(offset) += 1 + allStats(offset + 1) += label + allStats(offset + 2) += label * label + } + + /** + * Get an [[ImpurityCalculator]] for a (node, feature, bin). + * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. + * @param offset Start index of stats for this (node, feature, bin). + */ + def getCalculator(allStats: Array[Double], offset: Int): VarianceCalculator = { + new VarianceCalculator(allStats.view(offset, offset + statsSize).toArray) + } + +} + +/** + * Stores statistics for one (node, feature, bin) for calculating impurity. + * Unlike [[GiniAggregator]], this class stores its own data and is for a specific + * (node, feature, bin). + * @param stats Array of sufficient statistics for a (node, feature, bin). + */ +private[tree] class VarianceCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { + + require(stats.size == 3, + s"VarianceCalculator requires sufficient statistics array stats to be of length 3," + + s" but was given array of length ${stats.size}.") + + /** + * Make a deep copy of this [[ImpurityCalculator]]. + */ + def copy: VarianceCalculator = new VarianceCalculator(stats.clone()) + + /** + * Calculate the impurity from the stored sufficient statistics. + */ + def calculate(): Double = Variance.calculate(stats(0), stats(1), stats(2)) + + /** + * Number of data points accounted for in the sufficient statistics. + */ + def count: Long = stats(0).toLong + + /** + * Prediction which should be made based on the sufficient statistics. + */ + def predict: Double = if (count == 0) { + 0 + } else { + stats(1) / count + } + + override def toString: String = { + s"VarianceAggregator(cnt = ${stats(0)}, sum = ${stats(1)}, sum2 = ${stats(2)})" + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index af35d88f713e5..0cad473782af1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.FeatureType._ /** - * Used for "binning" the features bins for faster best split calculation. + * Used for "binning" the feature values for faster best split calculation. * * For a continuous feature, the bin is determined by a low and a high split, * where an example with featureValue falls into the bin s.t. @@ -30,13 +30,16 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * bins, splits, and feature values. The bin is determined by category/feature value. * However, the bins are not necessarily ordered by feature value; * they are ordered using impurity. + * * For unordered categorical features, there is a 1-1 correspondence between bins, splits, * where bins and splits correspond to subsets of feature values (in highSplit.categories). + * An unordered feature with k categories uses (1 << k - 1) - 1 bins, corresponding to all + * partitionings of categories into 2 disjoint, non-empty sets. * * @param lowSplit signifying the lower threshold for the continuous feature to be * accepted in the bin * @param highSplit signifying the upper threshold for the continuous feature to be - * accepted in the bin + * accepted in the bin * @param featureType type of feature -- categorical or continuous * @param category categorical label value accepted in the bin for ordered features */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 0eee6262781c1..5b8a4cbed2306 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -24,8 +24,13 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * Node in a decision tree - * @param id integer node id + * Node in a decision tree. + * + * About node indexing: + * Nodes are indexed from 1. Node 1 is the root; nodes 2, 3 are the left, right children. + * Node index 0 is not used. + * + * @param id integer node id, from 1 * @param predict predicted value at the node * @param isLeaf whether the leaf is a node * @param split split to calculate left and right nodes @@ -51,17 +56,13 @@ class Node ( * @param nodes array of nodes */ def build(nodes: Array[Node]): Unit = { - - logDebug("building node " + id + " at level " + - (scala.math.log(id + 1)/scala.math.log(2)).toInt ) + logDebug("building node " + id + " at level " + Node.indexToLevel(id)) logDebug("id = " + id + ", split = " + split) logDebug("stats = " + stats) logDebug("predict = " + predict) if (!isLeaf) { - val leftNodeIndex = id * 2 + 1 - val rightNodeIndex = id * 2 + 2 - leftNode = Some(nodes(leftNodeIndex)) - rightNode = Some(nodes(rightNodeIndex)) + leftNode = Some(nodes(Node.leftChildIndex(id))) + rightNode = Some(nodes(Node.rightChildIndex(id))) leftNode.get.build(nodes) rightNode.get.build(nodes) } @@ -96,24 +97,20 @@ class Node ( * Get the number of nodes in tree below this node, including leaf nodes. * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. */ - private[tree] def numDescendants: Int = { - if (isLeaf) { - 0 - } else { - 2 + leftNode.get.numDescendants + rightNode.get.numDescendants - } + private[tree] def numDescendants: Int = if (isLeaf) { + 0 + } else { + 2 + leftNode.get.numDescendants + rightNode.get.numDescendants } /** * Get depth of tree from this node. * E.g.: Depth 0 means this is a leaf node. */ - private[tree] def subtreeDepth: Int = { - if (isLeaf) { - 0 - } else { - 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) - } + private[tree] def subtreeDepth: Int = if (isLeaf) { + 0 + } else { + 1 + math.max(leftNode.get.subtreeDepth, rightNode.get.subtreeDepth) } /** @@ -148,3 +145,49 @@ class Node ( } } + +private[tree] object Node { + + /** + * Return the index of the left child of this node. + */ + def leftChildIndex(nodeIndex: Int): Int = nodeIndex << 1 + + /** + * Return the index of the right child of this node. + */ + def rightChildIndex(nodeIndex: Int): Int = (nodeIndex << 1) + 1 + + /** + * Get the parent index of the given node, or 0 if it is the root. + */ + def parentIndex(nodeIndex: Int): Int = nodeIndex >> 1 + + /** + * Return the level of a tree which the given node is in. + */ + def indexToLevel(nodeIndex: Int): Int = if (nodeIndex == 0) { + throw new IllegalArgumentException(s"0 is not a valid node index.") + } else { + java.lang.Integer.numberOfTrailingZeros(java.lang.Integer.highestOneBit(nodeIndex)) + } + + /** + * Returns true if this is a left child. + * Note: Returns false for the root. + */ + def isLeftChild(nodeIndex: Int): Boolean = nodeIndex > 1 && nodeIndex % 2 == 0 + + /** + * Return the maximum number of nodes which can be in the given level of the tree. + * @param level Level of tree (0 = root). + */ + def maxNodesInLevel(level: Int): Int = 1 << level + + /** + * Return the index of the first node in the given level. + * @param level Level of tree (0 = root). + */ + def startIndexInLevel(level: Int): Int = 1 << level + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 2f36fd907772c..8e556c917b2e7 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -21,15 +21,16 @@ import scala.collection.JavaConverters._ import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} -import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.mllib.regression.LabeledPoint + class DecisionTreeSuite extends FunSuite with LocalSparkContext { @@ -59,12 +60,13 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") } - test("split and bin calculation") { + test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, 3, 2, 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(bins.length === 2) @@ -72,7 +74,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) } - test("split and bin calculation for categorical variables") { + test("Binary classification with binary (ordered) categorical features:" + + " split and bin calculation") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -83,77 +86,20 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) assert(splits.length === 2) assert(bins.length === 2) - assert(splits(0).length === 99) - assert(bins(0).length === 100) - - // Check splits. - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(1.0)) - assert(splits(0)(1).categories.contains(0.0)) - - assert(splits(0)(2) === null) - - assert(splits(1)(0).feature === 1) - assert(splits(1)(0).threshold === Double.MinValue) - assert(splits(1)(0).featureType === Categorical) - assert(splits(1)(0).categories.length === 1) - assert(splits(1)(0).categories.contains(0.0)) - - assert(splits(1)(1).feature === 1) - assert(splits(1)(1).threshold === Double.MinValue) - assert(splits(1)(1).featureType === Categorical) - assert(splits(1)(1).categories.length === 2) - assert(splits(1)(1).categories.contains(1.0)) - assert(splits(1)(1).categories.contains(0.0)) - - assert(splits(1)(2) === null) - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - - assert(bins(0)(1).category === 0.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).lowSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(0.0)) - - assert(bins(0)(2) === null) - - assert(bins(1)(0).category === 0.0) - assert(bins(1)(0).lowSplit.categories.length === 0) - assert(bins(1)(0).highSplit.categories.length === 1) - assert(bins(1)(0).highSplit.categories.contains(0.0)) - - assert(bins(1)(1).category === 1.0) - assert(bins(1)(1).lowSplit.categories.length === 1) - assert(bins(1)(1).lowSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.length === 2) - assert(bins(1)(1).highSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.contains(1.0)) - - assert(bins(1)(2) === null) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } - test("split and bin calculations for categorical variables with no sample for one category") { + test("Binary classification with 3-ary (ordered) categorical features," + + " with no samples for one category") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -164,104 +110,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - - // Check splits. - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(1.0)) - assert(splits(0)(1).categories.contains(0.0)) - - assert(splits(0)(2).feature === 0) - assert(splits(0)(2).threshold === Double.MinValue) - assert(splits(0)(2).featureType === Categorical) - assert(splits(0)(2).categories.length === 3) - assert(splits(0)(2).categories.contains(1.0)) - assert(splits(0)(2).categories.contains(0.0)) - assert(splits(0)(2).categories.contains(2.0)) - - assert(splits(0)(3) === null) - - assert(splits(1)(0).feature === 1) - assert(splits(1)(0).threshold === Double.MinValue) - assert(splits(1)(0).featureType === Categorical) - assert(splits(1)(0).categories.length === 1) - assert(splits(1)(0).categories.contains(0.0)) - - assert(splits(1)(1).feature === 1) - assert(splits(1)(1).threshold === Double.MinValue) - assert(splits(1)(1).featureType === Categorical) - assert(splits(1)(1).categories.length === 2) - assert(splits(1)(1).categories.contains(1.0)) - assert(splits(1)(1).categories.contains(0.0)) - - assert(splits(1)(2).feature === 1) - assert(splits(1)(2).threshold === Double.MinValue) - assert(splits(1)(2).featureType === Categorical) - assert(splits(1)(2).categories.length === 3) - assert(splits(1)(2).categories.contains(1.0)) - assert(splits(1)(2).categories.contains(0.0)) - assert(splits(1)(2).categories.contains(2.0)) - - assert(splits(1)(3) === null) - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - - assert(bins(0)(1).category === 0.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).lowSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(0.0)) - - assert(bins(0)(2).category === 2.0) - assert(bins(0)(2).lowSplit.categories.length === 2) - assert(bins(0)(2).lowSplit.categories.contains(1.0)) - assert(bins(0)(2).lowSplit.categories.contains(0.0)) - assert(bins(0)(2).highSplit.categories.length === 3) - assert(bins(0)(2).highSplit.categories.contains(1.0)) - assert(bins(0)(2).highSplit.categories.contains(0.0)) - assert(bins(0)(2).highSplit.categories.contains(2.0)) - - assert(bins(0)(3) === null) - - assert(bins(1)(0).category === 0.0) - assert(bins(1)(0).lowSplit.categories.length === 0) - assert(bins(1)(0).highSplit.categories.length === 1) - assert(bins(1)(0).highSplit.categories.contains(0.0)) - - assert(bins(1)(1).category === 1.0) - assert(bins(1)(1).lowSplit.categories.length === 1) - assert(bins(1)(1).lowSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.length === 2) - assert(bins(1)(1).highSplit.categories.contains(0.0)) - assert(bins(1)(1).highSplit.categories.contains(1.0)) - - assert(bins(1)(2).category === 2.0) - assert(bins(1)(2).lowSplit.categories.length === 2) - assert(bins(1)(2).lowSplit.categories.contains(0.0)) - assert(bins(1)(2).lowSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.length === 3) - assert(bins(1)(2).highSplit.categories.contains(0.0)) - assert(bins(1)(2).highSplit.categories.contains(1.0)) - assert(bins(1)(2).highSplit.categories.contains(2.0)) - - assert(bins(1)(3) === null) + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } test("extract categories from a number for multiclass classification") { @@ -270,8 +128,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) } - test("split and bin calculations for unordered categorical variables with multiclass " + - "classification") { + test("Multiclass classification with unordered categorical features:" + + " split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -282,8 +140,15 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(bins.length === 2) + assert(splits(0).length === 3) + assert(bins(0).length === 6) // Expecting 2^2 - 1 = 3 bins/splits assert(splits(0)(0).feature === 0) @@ -321,10 +186,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(1)(2).categories.contains(0.0)) assert(splits(1)(2).categories.contains(1.0)) - assert(splits(0)(3) === null) - assert(splits(1)(3) === null) - - // Check bins. assert(bins(0)(0).category === Double.MinValue) @@ -360,13 +221,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(1)(2).highSplit.categories.contains(1.0)) assert(bins(1)(2).highSplit.categories.contains(0.0)) - assert(bins(0)(3) === null) - assert(bins(1)(3) === null) - } - test("split and bin calculations for ordered categorical variables with multiclass " + - "classification") { + test("Multiclass classification with ordered categorical features: split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() assert(arr.length === 3000) val rdd = sc.parallelize(arr) @@ -377,52 +234,21 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) + // 2^10 - 1 > 100, so categorical features will be ordered + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - - // 2^10 - 1 > 100, so categorical variables will be ordered - - assert(splits(0)(0).feature === 0) - assert(splits(0)(0).threshold === Double.MinValue) - assert(splits(0)(0).featureType === Categorical) - assert(splits(0)(0).categories.length === 1) - assert(splits(0)(0).categories.contains(1.0)) - - assert(splits(0)(1).feature === 0) - assert(splits(0)(1).threshold === Double.MinValue) - assert(splits(0)(1).featureType === Categorical) - assert(splits(0)(1).categories.length === 2) - assert(splits(0)(1).categories.contains(2.0)) - - assert(splits(0)(2).feature === 0) - assert(splits(0)(2).threshold === Double.MinValue) - assert(splits(0)(2).featureType === Categorical) - assert(splits(0)(2).categories.length === 3) - assert(splits(0)(2).categories.contains(2.0)) - assert(splits(0)(2).categories.contains(1.0)) - - assert(splits(0)(10) === null) - assert(splits(1)(10) === null) - - - // Check bins. - - assert(bins(0)(0).category === 1.0) - assert(bins(0)(0).lowSplit.categories.length === 0) - assert(bins(0)(0).highSplit.categories.length === 1) - assert(bins(0)(0).highSplit.categories.contains(1.0)) - assert(bins(0)(1).category === 2.0) - assert(bins(0)(1).lowSplit.categories.length === 1) - assert(bins(0)(1).highSplit.categories.length === 2) - assert(bins(0)(1).highSplit.categories.contains(1.0)) - assert(bins(0)(1).highSplit.categories.contains(2.0)) - - assert(bins(0)(10) === null) - + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) } - test("classification stump with all categorical variables") { + test("Binary classification stump with ordered categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -433,15 +259,23 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) + assert(splits.length === 2) + assert(bins.length === 2) + // no bins or splits pre-computed for ordered categorical features + assert(splits(0).length === 0) + assert(bins(0).length === 0) + val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 - assert(split.categories.length === 1) - assert(split.categories.contains(1.0)) + assert(split.categories === List(1.0)) assert(split.featureType === Categorical) assert(split.threshold === Double.MinValue) @@ -452,7 +286,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.impurity > 0.2) } - test("regression stump with all categorical variables") { + test("Regression stump with 3-ary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -462,10 +296,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) val split = bestSplits(0)._1 @@ -480,7 +318,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.impurity > 0.2) } - test("regression stump with categorical variables of arity 2") { + test("Regression stump with binary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -490,6 +328,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) validateRegressor(model, arr, 0.0) @@ -497,12 +338,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(model.depth === 1) } - test("stump with fixed label 0 for Gini") { + test("Binary classification stump with fixed label 0 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val strategy = new Strategy(Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -512,7 +357,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(7), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -521,12 +366,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.rightImpurity === 0) } - test("stump with fixed label 1 for Gini") { + test("Binary classification stump with fixed label 1 for Gini") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Gini, 3, 2, 100) + val strategy = new Strategy(Classification, Gini, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -536,7 +385,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -546,12 +395,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 1) } - test("stump with fixed label 0 for Entropy") { + test("Binary classification stump with fixed label 0 for Entropy") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val strategy = new Strategy(Classification, Entropy, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -561,7 +414,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -571,12 +424,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 0) } - test("stump with fixed label 1 for Entropy") { + test("Binary classification stump with fixed label 1 for Entropy") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) val rdd = sc.parallelize(arr) - val strategy = new Strategy(Classification, Entropy, 3, 2, 100) + val strategy = new Strategy(Classification, Entropy, maxDepth = 3, + numClassesForClassification = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) assert(splits.length === 2) assert(splits(0).length === 99) @@ -586,7 +443,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, Array(0.0), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) assert(bestSplits(0)._1.feature === 0) @@ -596,7 +453,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.predict === 1) } - test("second level node building with/without groups") { + test("Second level node building with vs. without groups") { val arr = DecisionTreeSuite.generateOrderedLabeledPoints() assert(arr.length === 1000) val rdd = sc.parallelize(arr) @@ -613,12 +470,12 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val nodes: Array[Node] = new Array[Node](7) - nodes(0) = modelOneNode.topNode - nodes(0).leftNode = None - nodes(0).rightNode = None + val nodes: Array[Node] = new Array[Node](8) + nodes(1) = modelOneNode.topNode + nodes(1).leftNode = None + nodes(1).rightNode = None - val parentImpurities = Array(0.5, 0.5, 0.5) + val parentImpurities = Array(0, 0.5, 0.5, 0.5) // Single group second level tree construction. val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) @@ -648,16 +505,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { } } - test("stump with categorical variables for multiclass classification") { + test("Multiclass classification stump with 3-ary (unordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(strategy.isMulticlassClassification) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -668,7 +528,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.featureType === Categorical) } - test("stump with 1 continuous variable for binary classification, to check off-by-1 error") { + test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0)) arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0)) @@ -684,26 +544,27 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(model.depth === 1) } - test("stump with 2 continuous variables for binary classification") { + test("Binary classification stump with 2 continuous features") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) arr(3) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 2.0)))) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 2) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) assert(model.topNode.split.get.feature === 1) } - test("stump with categorical variables for multiclass classification, with just enough bins") { - val maxBins = math.pow(2, 3 - 1).toInt // just enough bins to allow unordered features + test("Multiclass classification stump with unordered categorical features," + + " with just enough bins") { + val maxBins = 2 * (math.pow(2, 3 - 1).toInt - 1) // just enough bins to allow unordered features val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -711,6 +572,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) + assert(metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 1.0) @@ -719,7 +582,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -733,7 +596,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(gain.rightImpurity === 0) } - test("stump with continuous variables for multiclass classification") { + test("Multiclass classification stump with continuous features") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, @@ -746,7 +609,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -759,20 +622,21 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { } - test("stump with continuous + categorical variables for multiclass classification") { + test("Multiclass classification stump with continuous + unordered categorical features") { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) validateClassifier(model, arr, 0.9) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -784,17 +648,19 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.threshold < 2020) } - test("stump with categorical variables for ordered multiclass classification") { + test("Multiclass classification stump with 10-ary (ordered) categorical features") { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) + assert(!metadata.isUnordered(featureIndex = 0)) + assert(!metadata.isUnordered(featureIndex = 1)) val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(31), metadata, 0, + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, new Array[Node](0), splits, bins, 10) assert(bestSplits.length === 1) @@ -805,6 +671,18 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplit.featureType === Categorical) } + test("Multiclass classification tree with 10-ary (ordered) categorical features," + + " with just enough bins") { + val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + val rdd = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, + numClassesForClassification = 3, maxBins = 10, + categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + assert(strategy.isMulticlassClassification) + + val model = DecisionTree.train(rdd, strategy) + validateClassifier(model, arr, 0.6) + } } @@ -899,5 +777,4 @@ object DecisionTreeSuite { arr } - } From e16a8e7db5a3b1065b14baf89cb723a59b99226b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 8 Sep 2014 10:24:15 -0700 Subject: [PATCH 0625/1492] SPARK-3337 Paranoid quoting in shell to allow install dirs with spaces within. ... Tested ! TBH, it isn't a great idea to have directory with spaces within. Because emacs doesn't like it then hadoop doesn't like it. and so on... Author: Prashant Sharma Closes #2229 from ScrapCodes/SPARK-3337/quoting-shell-scripts and squashes the following commits: d4ad660 [Prashant Sharma] SPARK-3337 Paranoid quoting in shell to allow install dirs with spaces within. --- bin/beeline | 2 +- bin/compute-classpath.sh | 12 ++++++------ bin/load-spark-env.sh | 4 ++-- bin/pyspark | 20 ++++++++++---------- bin/run-example | 8 ++++---- bin/spark-class | 20 ++++++++++---------- bin/spark-shell | 10 +++++----- bin/spark-sql | 8 ++++---- bin/spark-submit | 4 ++-- dev/check-license | 16 ++++++++-------- dev/lint-python | 6 +++--- dev/mima | 4 ++-- dev/run-tests | 2 +- dev/scalastyle | 2 +- make-distribution.sh | 2 +- python/run-tests | 6 ++++-- sbin/slaves.sh | 12 ++++++------ sbin/spark-config.sh | 16 ++++++++-------- sbin/spark-daemon.sh | 20 ++++++++++---------- sbin/spark-executor | 8 ++++---- sbin/start-all.sh | 4 ++-- sbin/start-history-server.sh | 4 ++-- sbin/start-master.sh | 4 ++-- sbin/start-slave.sh | 4 ++-- sbin/start-slaves.sh | 12 ++++++------ sbin/start-thriftserver.sh | 8 ++++---- sbin/stop-all.sh | 4 ++-- sbin/stop-history-server.sh | 4 ++-- sbt/sbt | 20 ++++++++++---------- sbt/sbt-launch-lib.bash | 12 ++++++------ 30 files changed, 130 insertions(+), 128 deletions(-) diff --git a/bin/beeline b/bin/beeline index 1bda4dba50605..3fcb6df34339d 100755 --- a/bin/beeline +++ b/bin/beeline @@ -24,7 +24,7 @@ set -o posix # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.hive.beeline.BeeLine" exec "$FWDIR/bin/spark-class" $CLASS "$@" diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 16b794a1592e8..15c6779402994 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -23,9 +23,9 @@ SCALA_VERSION=2.10 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" @@ -63,7 +63,7 @@ else assembly_folder="$ASSEMBLY_DIR" fi -num_jars=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l) +num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l)" if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $assembly_folder" echo "You need to build Spark before running this program." @@ -77,7 +77,7 @@ if [ "$num_jars" -gt "1" ]; then exit 1 fi -ASSEMBLY_JAR=$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null) +ASSEMBLY_JAR="$(ls "$assembly_folder"/spark-assembly*hadoop*.jar 2>/dev/null)" # Verify that versions of java used to build the jars and run Spark are compatible jar_error_check=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" nonexistent/class/path 2>&1) @@ -103,8 +103,8 @@ else datanucleus_dir="$FWDIR"/lib_managed/jars fi -datanucleus_jars=$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar") -datanucleus_jars=$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g) +datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar")" +datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" if [ -n "$datanucleus_jars" ]; then hive_files=$("$JAR_CMD" -tf "$ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 493d3785a081b..6d4231b204595 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -25,9 +25,9 @@ if [ -z "$SPARK_ENV_LOADED" ]; then export SPARK_ENV_LOADED=1 # Returns the parent of the directory this script lives in. - parent_dir="$(cd `dirname $0`/..; pwd)" + parent_dir="$(cd "`dirname "$0"`"/..; pwd)" - user_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} + user_conf_dir="${SPARK_CONF_DIR:-"$parent_dir"/conf}" if [ -f "${user_conf_dir}/spark-env.sh" ]; then # Promote all variable declarations to environment (exported) variables diff --git a/bin/pyspark b/bin/pyspark index 26a16dd600b7a..5142411e36974 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -18,18 +18,18 @@ # # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -source $FWDIR/bin/utils.sh +source "$FWDIR/bin/utils.sh" SCALA_VERSION=2.10 function usage() { echo "Usage: ./bin/pyspark [options]" 1>&2 - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 } @@ -48,7 +48,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then fi fi -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh # Figure out which Python executable to use if [[ -z "$PYSPARK_PYTHON" ]]; then @@ -57,12 +57,12 @@ fi export PYSPARK_PYTHON # Add the PySpark classes to the Python path: -export PYTHONPATH=$SPARK_HOME/python/:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$SPARK_HOME/python/:$PYTHONPATH" +export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: -export OLD_PYTHONSTARTUP=$PYTHONSTARTUP -export PYTHONSTARTUP=$FWDIR/python/pyspark/shell.py +export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" +export PYTHONSTARTUP="$FWDIR/python/pyspark/shell.py" # If IPython options are specified, assume user wants to run IPython if [[ -n "$IPYTHON_OPTS" ]]; then @@ -99,10 +99,10 @@ fi if [[ "$1" =~ \.py$ ]]; then echo -e "\nWARNING: Running python applications through ./bin/pyspark is deprecated as of Spark 1.0." 1>&2 echo -e "Use ./bin/spark-submit \n" 1>&2 - primary=$1 + primary="$1" shift gatherSparkSubmitOpts "$@" - exec $FWDIR/bin/spark-submit "${SUBMISSION_OPTS[@]}" $primary "${APPLICATION_OPTS[@]}" + exec "$FWDIR"/bin/spark-submit "${SUBMISSION_OPTS[@]}" "$primary" "${APPLICATION_OPTS[@]}" else # PySpark shell requires special handling downstream export PYSPARK_SHELL=1 diff --git a/bin/run-example b/bin/run-example index 68a35702eddd3..34dd71c71880e 100755 --- a/bin/run-example +++ b/bin/run-example @@ -19,7 +19,7 @@ SCALA_VERSION=2.10 -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" export SPARK_HOME="$FWDIR" EXAMPLES_DIR="$FWDIR"/examples @@ -35,12 +35,12 @@ else fi if [ -f "$FWDIR/RELEASE" ]; then - export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` + export SPARK_EXAMPLES_JAR="`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar`" elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar` + export SPARK_EXAMPLES_JAR="`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar`" fi -if [[ -z $SPARK_EXAMPLES_JAR ]]; then +if [[ -z "$SPARK_EXAMPLES_JAR" ]]; then echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" 1>&2 echo "You need to build Spark before running this program" 1>&2 exit 1 diff --git a/bin/spark-class b/bin/spark-class index c6543545a5e64..5f5f9ea74888d 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -27,12 +27,12 @@ esac SCALA_VERSION=2.10 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -. $FWDIR/bin/load-spark-env.sh +. "$FWDIR"/bin/load-spark-env.sh if [ -z "$1" ]; then echo "Usage: spark-class []" 1>&2 @@ -105,7 +105,7 @@ else exit 1 fi fi -JAVA_VERSION=$($RUNNER -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') +JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size if [ "$JAVA_VERSION" -ge 18 ]; then @@ -117,7 +117,7 @@ JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then - JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" + JAVA_OPTS="$JAVA_OPTS `cat "$FWDIR"/conf/java-opts`" fi # Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! @@ -126,21 +126,21 @@ TOOLS_DIR="$FWDIR"/tools SPARK_TOOLS_JAR="" if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the SBT build - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar` + export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar`" fi if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the Maven build # TODO: this also needs to become an assembly! - export SPARK_TOOLS_JAR=`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar` + export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar`" fi # Compute classpath using external script -classpath_output=$($FWDIR/bin/compute-classpath.sh) +classpath_output=$("$FWDIR"/bin/compute-classpath.sh) if [[ "$?" != "0" ]]; then echo "$classpath_output" exit 1 else - CLASSPATH=$classpath_output + CLASSPATH="$classpath_output" fi if [[ "$1" =~ org.apache.spark.tools.* ]]; then @@ -153,9 +153,9 @@ if [[ "$1" =~ org.apache.spark.tools.* ]]; then fi if $cygwin; then - CLASSPATH=`cygpath -wp $CLASSPATH` + CLASSPATH="`cygpath -wp "$CLASSPATH"`" if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then - export SPARK_TOOLS_JAR=`cygpath -w $SPARK_TOOLS_JAR` + export SPARK_TOOLS_JAR="`cygpath -w "$SPARK_TOOLS_JAR"`" fi fi export CLASSPATH diff --git a/bin/spark-shell b/bin/spark-shell index 0ab4e14f5b744..4a0670fc6c8aa 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -29,11 +29,11 @@ esac set -o posix ## Global script variables -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage() { echo "Usage: ./bin/spark-shell [options]" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 exit 0 } @@ -41,7 +41,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" @@ -54,11 +54,11 @@ function main() { # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" stty icanon echo > /dev/null 2>&1 else export SPARK_SUBMIT_OPTS - $FWDIR/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" + "$FWDIR"/bin/spark-submit --class org.apache.spark.repl.Main "${SUBMISSION_OPTS[@]}" spark-shell "${APPLICATION_OPTS[@]}" fi } diff --git a/bin/spark-sql b/bin/spark-sql index 2a3cb31f58e8d..ae096530cad04 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -27,7 +27,7 @@ CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" function usage { echo "Usage: ./bin/spark-sql [options] [cli option]" @@ -38,10 +38,10 @@ function usage { pattern+="\|--help" pattern+="\|=======" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "CLI options:" - $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -49,7 +49,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" diff --git a/bin/spark-submit b/bin/spark-submit index 277c4ce571ca2..c557311b4b20e 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,7 +19,7 @@ # NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! -export SPARK_HOME="$(cd `dirname $0`/..; pwd)" +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" ORIG_ARGS=("$@") while (($#)); do @@ -59,5 +59,5 @@ if [[ "$SPARK_SUBMIT_DEPLOY_MODE" == "client" && -f "$SPARK_SUBMIT_PROPERTIES_FI fi fi -exec $SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" +exec "$SPARK_HOME"/bin/spark-class org.apache.spark.deploy.SparkSubmit "${ORIG_ARGS[@]}" diff --git a/dev/check-license b/dev/check-license index 558e038afc01a..9ff0929e9a5e8 100755 --- a/dev/check-license +++ b/dev/check-license @@ -23,16 +23,16 @@ acquire_rat_jar () { URL1="http://search.maven.org/remotecontent?filepath=org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" URL2="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" - JAR=$rat_jar + JAR="$rat_jar" if [[ ! -f "$rat_jar" ]]; then # Download rat launch jar if it hasn't been downloaded yet if [ ! -f "$JAR" ]; then # Download printf "Attempting to fetch rat\n" - JAR_DL=${JAR}.part + JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent ${URL1} > "$JAR_DL" || curl --silent ${URL2} > "$JAR_DL") && mv "$JAR_DL" "$JAR" + (curl --silent "${URL1}" > "$JAR_DL" || curl --silent "${URL2}" > "$JAR_DL") && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then (wget --quiet ${URL1} -O "$JAR_DL" || wget --quiet ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" else @@ -50,7 +50,7 @@ acquire_rat_jar () { } # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" if test -x "$JAVA_HOME/bin/java"; then @@ -60,17 +60,17 @@ else fi export RAT_VERSION=0.10 -export rat_jar=$FWDIR/lib/apache-rat-${RAT_VERSION}.jar -mkdir -p $FWDIR/lib +export rat_jar="$FWDIR"/lib/apache-rat-${RAT_VERSION}.jar +mkdir -p "$FWDIR"/lib [[ -f "$rat_jar" ]] || acquire_rat_jar || { echo "Download failed. Obtain the rat jar manually and place it at $rat_jar" exit 1 } -$java_cmd -jar $rat_jar -E $FWDIR/.rat-excludes -d $FWDIR > rat-results.txt +$java_cmd -jar "$rat_jar" -E "$FWDIR"/.rat-excludes -d "$FWDIR" > rat-results.txt -ERRORS=$(cat rat-results.txt | grep -e "??") +ERRORS="$(cat rat-results.txt | grep -e "??")" if test ! -z "$ERRORS"; then echo "Could not find Apache license headers in the following files:" diff --git a/dev/lint-python b/dev/lint-python index 79bf70f0b8b13..772f856154ae0 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -18,10 +18,10 @@ # SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )" -SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)" +SPARK_ROOT_DIR="$(dirname "$SCRIPT_DIR")" PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt" -cd $SPARK_ROOT_DIR +cd "$SPARK_ROOT_DIR" # Get pep8 at runtime so that we don't rely on it being installed on the build server. #+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 @@ -45,7 +45,7 @@ fi #+ first, but we do so so that the check status can #+ be output before the report, like with the #+ scalastyle and RAT checks. -python $PEP8_SCRIPT_PATH $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" +python "$PEP8_SCRIPT_PATH" $PEP8_PATHS_TO_CHECK > "$PEP8_REPORT_PATH" pep8_status=${PIPESTATUS[0]} #$? if [ $pep8_status -ne 0 ]; then diff --git a/dev/mima b/dev/mima index 09e4482af5f3d..f9b9b03538f15 100755 --- a/dev/mima +++ b/dev/mima @@ -21,12 +21,12 @@ set -o pipefail set -e # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" echo -e "q\n" | sbt/sbt oldDeps/update -export SPARK_CLASSPATH=`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"` +export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore diff --git a/dev/run-tests b/dev/run-tests index 90a8ce16f0f06..49a88085c80f7 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -18,7 +18,7 @@ # # Go to the Spark project root directory -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then diff --git a/dev/scalastyle b/dev/scalastyle index eb9b467965636..efb5f291ea3b7 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -19,7 +19,7 @@ echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt # Check style with YARN alpha built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt # Check style with YARN built too echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ diff --git a/make-distribution.sh b/make-distribution.sh index 14aed4a4b655b..9b012b9222db4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -28,7 +28,7 @@ set -o pipefail set -e # Figure out where the Spark framework is installed -FWDIR="$(cd `dirname $0`; pwd)" +FWDIR="$(cd "`dirname "$0"`"; pwd)" DISTDIR="$FWDIR/dist" SPARK_TACHYON=false diff --git a/python/run-tests b/python/run-tests index f2a80b4f1838b..226e9e2c3770a 100755 --- a/python/run-tests +++ b/python/run-tests @@ -19,7 +19,7 @@ # Figure out where the Spark framework is installed -FWDIR="$(cd `dirname $0`; cd ../; pwd)" +FWDIR="$(cd "`dirname "$0"`"; cd ../; pwd)" # CD into the python directory to find things on the right path cd "$FWDIR/python" @@ -33,7 +33,9 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" - SPARK_TESTING=1 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log + + SPARK_TESTING=1 "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log + FAILED=$((PIPESTATUS[0]||$FAILED)) # Fail and exit on the first test failure. diff --git a/sbin/slaves.sh b/sbin/slaves.sh index f89547fef9e46..1d4dc5edf9858 100755 --- a/sbin/slaves.sh +++ b/sbin/slaves.sh @@ -36,29 +36,29 @@ if [ $# -le 0 ]; then exit 1 fi -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" # If the slaves file is specified in the command line, # then it takes precedence over the definition in # spark-env.sh. Save it here. -HOSTLIST=$SPARK_SLAVES +HOSTLIST="$SPARK_SLAVES" # Check if --config is passed as an argument. It is an optional parameter. # Exit if the argument is not a directory. if [ "$1" == "--config" ] then shift - conf_dir=$1 + conf_dir="$1" if [ ! -d "$conf_dir" ] then echo "ERROR : $conf_dir is not a directory" echo $usage exit 1 else - export SPARK_CONF_DIR=$conf_dir + export SPARK_CONF_DIR="$conf_dir" fi shift fi @@ -79,7 +79,7 @@ if [ "$SPARK_SSH_OPTS" = "" ]; then fi for slave in `cat "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do - ssh $SPARK_SSH_OPTS $slave $"${@// /\\ }" \ + ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ 2>&1 | sed "s/^/$slave: /" & if [ "$SPARK_SLAVE_SLEEP" != "" ]; then sleep $SPARK_SLAVE_SLEEP diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 5c87da5815b64..2718d6cba1c9a 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -21,19 +21,19 @@ # resolve links - $0 may be a softlink this="${BASH_SOURCE-$0}" -common_bin=$(cd -P -- "$(dirname -- "$this")" && pwd -P) +common_bin="$(cd -P -- "$(dirname -- "$this")" && pwd -P)" script="$(basename -- "$this")" this="$common_bin/$script" # convert relative path to absolute path -config_bin=`dirname "$this"` -script=`basename "$this"` -config_bin=`cd "$config_bin"; pwd` +config_bin="`dirname "$this"`" +script="`basename "$this"`" +config_bin="`cd "$config_bin"; pwd`" this="$config_bin/$script" -export SPARK_PREFIX=`dirname "$this"`/.. -export SPARK_HOME=${SPARK_PREFIX} +export SPARK_PREFIX="`dirname "$this"`"/.. +export SPARK_HOME="${SPARK_PREFIX}" export SPARK_CONF_DIR="$SPARK_HOME/conf" # Add the PySpark classes to the PYTHONPATH: -export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH -export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$SPARK_HOME/python:$PYTHONPATH" +export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 9032f23ea8eff..bd476b400e1c3 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -37,8 +37,8 @@ if [ $# -le 1 ]; then exit 1 fi -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" @@ -50,14 +50,14 @@ sbin=`cd "$sbin"; pwd` if [ "$1" == "--config" ] then shift - conf_dir=$1 + conf_dir="$1" if [ ! -d "$conf_dir" ] then echo "ERROR : $conf_dir is not a directory" echo $usage exit 1 else - export SPARK_CONF_DIR=$conf_dir + export SPARK_CONF_DIR="$conf_dir" fi shift fi @@ -100,12 +100,12 @@ if [ "$SPARK_LOG_DIR" = "" ]; then export SPARK_LOG_DIR="$SPARK_HOME/logs" fi mkdir -p "$SPARK_LOG_DIR" -touch $SPARK_LOG_DIR/.spark_test > /dev/null 2>&1 +touch "$SPARK_LOG_DIR"/.spark_test > /dev/null 2>&1 TEST_LOG_DIR=$? if [ "${TEST_LOG_DIR}" = "0" ]; then - rm -f $SPARK_LOG_DIR/.spark_test + rm -f "$SPARK_LOG_DIR"/.spark_test else - chown $SPARK_IDENT_STRING $SPARK_LOG_DIR + chown "$SPARK_IDENT_STRING" "$SPARK_LOG_DIR" fi if [ "$SPARK_PID_DIR" = "" ]; then @@ -113,8 +113,8 @@ if [ "$SPARK_PID_DIR" = "" ]; then fi # some variables -log=$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out -pid=$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid +log="$SPARK_LOG_DIR/spark-$SPARK_IDENT_STRING-$command-$instance-$HOSTNAME.out" +pid="$SPARK_PID_DIR/spark-$SPARK_IDENT_STRING-$command-$instance.pid" # Set default scheduling priority if [ "$SPARK_NICENESS" = "" ]; then @@ -136,7 +136,7 @@ case $startStop in fi if [ "$SPARK_MASTER" != "" ]; then - echo rsync from $SPARK_MASTER + echo rsync from "$SPARK_MASTER" rsync -a -e ssh --delete --exclude=.svn --exclude='logs/*' --exclude='contrib/hod/logs/*' $SPARK_MASTER/ "$SPARK_HOME" fi diff --git a/sbin/spark-executor b/sbin/spark-executor index 3621321a9bc8d..674ce906d9421 100755 --- a/sbin/spark-executor +++ b/sbin/spark-executor @@ -17,10 +17,10 @@ # limitations under the License. # -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" -export PYTHONPATH=$FWDIR/python:$PYTHONPATH -export PYTHONPATH=$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH +export PYTHONPATH="$FWDIR/python:$PYTHONPATH" +export PYTHONPATH="$FWDIR/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" echo "Running spark-executor with framework dir = $FWDIR" -exec $FWDIR/bin/spark-class org.apache.spark.executor.MesosExecutorBackend +exec "$FWDIR"/bin/spark-class org.apache.spark.executor.MesosExecutorBackend diff --git a/sbin/start-all.sh b/sbin/start-all.sh index 5c89ab4d86b3a..1baf57cea09ee 100755 --- a/sbin/start-all.sh +++ b/sbin/start-all.sh @@ -21,8 +21,8 @@ # Starts the master on this node. # Starts a worker on each node specified in conf/slaves -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" TACHYON_STR="" diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 580ab471b8a79..7172ad15d88fc 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -24,8 +24,8 @@ # Use the SPARK_HISTORY_OPTS environment variable to set history server configuration. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" . "$SPARK_PREFIX/bin/load-spark-env.sh" diff --git a/sbin/start-master.sh b/sbin/start-master.sh index c5c02491f78e1..17fff58f4f768 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -19,8 +19,8 @@ # Starts the master on the machine this script is executed on. -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" START_TACHYON=false diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index b563400dc24f3..2fc35309f4ca5 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -20,7 +20,7 @@ # Usage: start-slave.sh # where is like "spark://localhost:7077" -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 4912d0c0c7dfd..ba1a84abc1fef 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" START_TACHYON=false @@ -46,11 +46,11 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_IP" = "" ]; then - SPARK_MASTER_IP=`hostname` + SPARK_MASTER_IP="`hostname`" fi if [ "$START_TACHYON" == "true" ]; then - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon bootstrap-conf $SPARK_MASTER_IP + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon bootstrap-conf "$SPARK_MASTER_IP" # set -t so we can call sudo SPARK_SSH_OPTS="-o StrictHostKeyChecking=no -t" "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/../tachyon/bin/tachyon-start.sh" worker SudoMount \; sleep 1 @@ -58,12 +58,12 @@ fi # Launch the slaves if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT + exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" else if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then SPARK_WORKER_WEBUI_PORT=8081 fi for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) done fi diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index c519a77df4a14..4ce40fe750384 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -24,7 +24,7 @@ set -o posix # Figure out where Spark is installed -FWDIR="$(cd `dirname $0`/..; pwd)" +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" CLASS_NOT_FOUND_EXIT_STATUS=1 @@ -38,10 +38,10 @@ function usage { pattern+="\|=======" pattern+="\|--help" - $FWDIR/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + "$FWDIR"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo echo "Thrift server options:" - $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + "$FWDIR"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then @@ -49,7 +49,7 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source $FWDIR/bin/utils.sh +source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh index 60b358d374565..298c6a9859795 100755 --- a/sbin/stop-all.sh +++ b/sbin/stop-all.sh @@ -21,8 +21,8 @@ # Run this on the master nde -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" # Load the Spark configuration . "$sbin/spark-config.sh" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh index c0034ad641cbe..6e6056359510f 100755 --- a/sbin/stop-history-server.sh +++ b/sbin/stop-history-server.sh @@ -19,7 +19,7 @@ # Stops the history server on the machine this script is executed on. -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 diff --git a/sbt/sbt b/sbt/sbt index 1b1aa1483a829..c172fa74bc771 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -3,32 +3,32 @@ # When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so # that we can run Hive to generate the golden answer. This is not required for normal development # or testing. -for i in $HIVE_HOME/lib/* -do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i +for i in "$HIVE_HOME"/lib/* +do HADOOP_CLASSPATH="$HADOOP_CLASSPATH:$i" done export HADOOP_CLASSPATH realpath () { ( - TARGET_FILE=$1 + TARGET_FILE="$1" - cd $(dirname $TARGET_FILE) - TARGET_FILE=$(basename $TARGET_FILE) + cd "$(dirname "$TARGET_FILE")" + TARGET_FILE="$(basename "$TARGET_FILE")" COUNT=0 while [ -L "$TARGET_FILE" -a $COUNT -lt 100 ] do - TARGET_FILE=$(readlink $TARGET_FILE) - cd $(dirname $TARGET_FILE) - TARGET_FILE=$(basename $TARGET_FILE) + TARGET_FILE="$(readlink "$TARGET_FILE")" + cd $(dirname "$TARGET_FILE") + TARGET_FILE="$(basename $TARGET_FILE)" COUNT=$(($COUNT + 1)) done - echo $(pwd -P)/$TARGET_FILE + echo "$(pwd -P)/"$TARGET_FILE"" ) } -. $(dirname $(realpath $0))/sbt-launch-lib.bash +. "$(dirname "$(realpath "$0")")"/sbt-launch-lib.bash declare -r noshare_opts="-Dsbt.global.base=project/.sbtboot -Dsbt.boot.directory=project/.boot -Dsbt.ivy.home=project/.ivy" diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index fecc3d38a5fbd..7f05d2ef491a3 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -7,7 +7,7 @@ # TODO - Should we merge the main SBT script with this library? if test -z "$HOME"; then - declare -r script_dir="$(dirname $script_path)" + declare -r script_dir="$(dirname "$script_path")" else declare -r script_dir="$HOME/.sbt" fi @@ -46,20 +46,20 @@ acquire_sbt_jar () { if [[ ! -f "$sbt_jar" ]]; then # Download sbt launch jar if it hasn't been downloaded yet - if [ ! -f ${JAR} ]; then + if [ ! -f "${JAR}" ]; then # Download printf "Attempting to fetch sbt\n" - JAR_DL=${JAR}.part + JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent ${URL1} > ${JAR_DL} || curl --silent ${URL2} > ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (curl --silent ${URL1} > "${JAR_DL}" || curl --silent ${URL2} > "${JAR_DL}") && mv "${JAR_DL}" "${JAR}" elif hash wget 2>/dev/null; then - (wget --quiet ${URL1} -O ${JAR_DL} || wget --quiet ${URL2} -O ${JAR_DL}) && mv ${JAR_DL} ${JAR} + (wget --quiet ${URL1} -O "${JAR_DL}" || wget --quiet ${URL2} -O "${JAR_DL}") && mv "${JAR_DL}" "${JAR}" else printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" exit -1 fi fi - if [ ! -f ${JAR} ]; then + if [ ! -f "${JAR}" ]; then # We failed to download printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" exit -1 From 16a73c2473181e03d88001aa3e08e6ffac92eb8b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 8 Sep 2014 11:20:00 -0700 Subject: [PATCH 0626/1492] SPARK-2978. Transformation with MR shuffle semantics I didn't add this to the transformations list in the docs because it's kind of obscure, but would be happy to do so if others think it would be helpful. Author: Sandy Ryza Closes #2274 from sryza/sandy-spark-2978 and squashes the following commits: 4a5332a [Sandy Ryza] Fix Java test c04b447 [Sandy Ryza] Fix Python doc and add back deleted code 433ad5b [Sandy Ryza] Add Java test 4c25a54 [Sandy Ryza] Add s at the end and a couple other fixes 9b0ba99 [Sandy Ryza] Fix compilation 36e0571 [Sandy Ryza] Fix import ordering 48c12c2 [Sandy Ryza] Add Java version and additional doc e5381cd [Sandy Ryza] Fix python style warnings f147634 [Sandy Ryza] SPARK-2978. Transformation with MR shuffle semantics --- .../apache/spark/api/java/JavaPairRDD.scala | 26 ++++++++++++++++ .../spark/rdd/OrderedRDDFunctions.scala | 14 ++++++++- .../java/org/apache/spark/JavaAPISuite.java | 30 +++++++++++++++++++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 14 +++++++++ python/pyspark/rdd.py | 24 +++++++++++++++ python/pyspark/tests.py | 8 +++++ 6 files changed, 115 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index feeb6c02caa78..880f61c49726e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -758,6 +758,32 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) rdd.saveAsHadoopDataset(conf) } + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner): JavaPairRDD[K, V] = { + val comp = com.google.common.collect.Ordering.natural().asInstanceOf[Comparator[K]] + repartitionAndSortWithinPartitions(partitioner, comp) + } + + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner, comp: Comparator[K]) + : JavaPairRDD[K, V] = { + implicit val ordering = comp // Allow implicit conversion of Comparator to Ordering. + fromRDD( + new OrderedRDDFunctions[K, V, (K, V)](rdd).repartitionAndSortWithinPartitions(partitioner)) + } + /** * Sort the RDD by key, so that each partition contains a sorted range of the elements in * ascending order. Calling `collect` or `save` on the resulting RDD will return or output an diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index e98bad2026e32..d0dbfef35d03c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -19,7 +19,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Logging, RangePartitioner} +import org.apache.spark.{Logging, Partitioner, RangePartitioner} import org.apache.spark.annotation.DeveloperApi /** @@ -64,4 +64,16 @@ class OrderedRDDFunctions[K : Ordering : ClassTag, new ShuffledRDD[K, V, V](self, part) .setKeyOrdering(if (ascending) ordering else ordering.reverse) } + + /** + * Repartition the RDD according to the given partitioner and, within each resulting partition, + * sort records by their keys. + * + * This is more efficient than calling `repartition` and then sorting within each partition + * because it can push the sorting down into the shuffle machinery. + */ + def repartitionAndSortWithinPartitions(partitioner: Partitioner): RDD[(K, V)] = { + new ShuffledRDD[K, V, V](self, partitioner).setKeyOrdering(ordering) + } + } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index e1c13de04a0be..be99dc501c4b2 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -189,6 +189,36 @@ public void sortByKey() { Assert.assertEquals(new Tuple2(3, 2), sortedPairs.get(2)); } + @Test + public void repartitionAndSortWithinPartitions() { + List> pairs = new ArrayList>(); + pairs.add(new Tuple2(0, 5)); + pairs.add(new Tuple2(3, 8)); + pairs.add(new Tuple2(2, 6)); + pairs.add(new Tuple2(0, 8)); + pairs.add(new Tuple2(3, 8)); + pairs.add(new Tuple2(1, 3)); + + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + Partitioner partitioner = new Partitioner() { + public int numPartitions() { + return 2; + } + public int getPartition(Object key) { + return ((Integer)key).intValue() % 2; + } + }; + + JavaPairRDD repartitioned = + rdd.repartitionAndSortWithinPartitions(partitioner); + List>> partitions = repartitioned.glom().collect(); + Assert.assertEquals(partitions.get(0), Arrays.asList(new Tuple2(0, 5), + new Tuple2(0, 8), new Tuple2(2, 6))); + Assert.assertEquals(partitions.get(1), Arrays.asList(new Tuple2(1, 3), + new Tuple2(3, 8), new Tuple2(3, 8))); + } + @Test public void emptyRDD() { JavaRDD rdd = sc.emptyRDD(); diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 499dcda3dae8f..c1b501a75c8b8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -682,6 +682,20 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(data.sortBy(parse, true, 2)(NameOrdering, classTag[Person]).collect() === nameOrdered) } + test("repartitionAndSortWithinPartitions") { + val data = sc.parallelize(Seq((0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)), 2) + + val partitioner = new Partitioner { + def numPartitions: Int = 2 + def getPartition(key: Any): Int = key.asInstanceOf[Int] % 2 + } + + val repartitioned = data.repartitionAndSortWithinPartitions(partitioner) + val partitions = repartitioned.glom().collect() + assert(partitions(0) === Seq((0, 5), (0, 8), (2, 6))) + assert(partitions(1) === Seq((1, 3), (3, 8), (3, 8))) + } + test("intersection") { val all = sc.parallelize(1 to 10) val evens = sc.parallelize(2 to 10 by 2) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 266090e3ae8f3..5667154cb84a8 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -520,6 +520,30 @@ def __add__(self, other): raise TypeError return self.union(other) + def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=portable_hash, + ascending=True, keyfunc=lambda x: x): + """ + Repartition the RDD according to the given partitioner and, within each resulting partition, + sort records by their keys. + + >>> rdd = sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)]) + >>> rdd2 = rdd.repartitionAndSortWithinPartitions(2, lambda x: x % 2, 2) + >>> rdd2.glom().collect() + [[(0, 5), (0, 8), (2, 6)], [(1, 3), (3, 8), (3, 8)]] + """ + if numPartitions is None: + numPartitions = self._defaultReducePartitions() + + spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == "true") + memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + serializer = self._jrdd_deserializer + + def sortPartition(iterator): + sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted + return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + + return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True) + def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): """ Sorts this RDD, which is assumed to consist of (key, value) pairs. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 9fbeb36f4f1dd..0bd2a9e6c507d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -545,6 +545,14 @@ def test_histogram(self): self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) self.assertRaises(TypeError, lambda: rdd.histogram(2)) + def test_repartitionAndSortWithinPartitions(self): + rdd = self.sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)], 2) + + repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2) + partitions = repartitioned.glom().collect() + self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) + self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + class TestSQL(PySparkTestCase): From 386bc24ebe3e75875b9647d9223c62d7b9dc9963 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Mon, 8 Sep 2014 12:37:52 -0700 Subject: [PATCH 0627/1492] Provide a default PYSPARK_PYTHON for python/run_tests Without this the version of python used in the test is not recorded. The error is, Testing with Python version: ./run-tests: line 57: --version: command not found Author: Matthew Farrellee Closes #2300 from mattf/master-fix-python-run-tests and squashes the following commits: 65a09f5 [Matthew Farrellee] Provide a default PYSPARK_PYTHON for python/run_tests --- python/run-tests | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/run-tests b/python/run-tests index 226e9e2c3770a..d98840de59d2c 100755 --- a/python/run-tests +++ b/python/run-tests @@ -50,6 +50,8 @@ function run_test() { echo "Running PySpark tests. Output is in python/unit-tests.log." +export PYSPARK_PYTHON="python" + # Try to test with Python 2.6, since that's the minimum version that we support: if [ $(which python2.6) ]; then export PYSPARK_PYTHON="python2.6" From 26bc7655de18ab0191ded3f75cb77bc756dc1c03 Mon Sep 17 00:00:00 2001 From: Henry Cook Date: Mon, 8 Sep 2014 14:56:37 -0700 Subject: [PATCH 0628/1492] [SQL] Minor edits to sql programming guide. Author: Henry Cook Closes #2316 from hcook/sql-docs and squashes the following commits: 373f94b [Henry Cook] Minor edits to sql programming guide. --- docs/sql-programming-guide.md | 92 ++++++++++++++++++----------------- 1 file changed, 47 insertions(+), 45 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 1814fef465cac..d83efa4bab324 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -13,10 +13,10 @@ title: Spark SQL Programming Guide Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed -[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with +[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed of +[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects, along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`. @@ -26,10 +26,10 @@ All of the examples on this page use sample data included in the Spark distribut
    Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed -[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects along with +[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed of +[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects, along with a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table -in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
    @@ -37,10 +37,10 @@ file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive]( Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed -[Row](api/python/pyspark.sql.Row-class.html) objects along with +[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed of +[Row](api/python/pyspark.sql.Row-class.html) objects, along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table -in a traditional relational database. A SchemaRDD can be created from an existing RDD, [Parquet](http://parquet.io) +in a traditional relational database. A SchemaRDD can be created from an existing RDD, a [Parquet](http://parquet.io) file, a JSON dataset, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell. @@ -68,11 +68,11 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext.createSchemaRDD {% endhighlight %} -In addition to the basic SQLContext, you can also create a HiveContext, which provides a strict -super set of the functionality provided by the basic SQLContext. Additional features include +In addition to the basic SQLContext, you can also create a HiveContext, which provides a +superset of the functionality provided by the basic SQLContext. Additional features include the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing hive setup, and all of the data sources available to a SQLContext are still available. +existing Hive setup, and all of the data sources available to a SQLContext are still available. HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default Spark build. If these dependencies are not a problem for your application then using HiveContext is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to @@ -95,7 +95,7 @@ In addition to the basic SQLContext, you can also create a HiveContext, which pr super set of the functionality provided by the basic SQLContext. Additional features include the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing hive setup, and all of the data sources available to a SQLContext are still available. +existing Hive setup, and all of the data sources available to a SQLContext are still available. HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default Spark build. If these dependencies are not a problem for your application then using HiveContext is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to @@ -118,7 +118,7 @@ In addition to the basic SQLContext, you can also create a HiveContext, which pr super set of the functionality provided by the basic SQLContext. Additional features include the ability to write queries using the more complete HiveQL parser, access to HiveUDFs, and the ability to read data from Hive tables. To use a HiveContext, you do not need to have an -existing hive setup, and all of the data sources available to a SQLContext are still available. +existing Hive setup, and all of the data sources available to a SQLContext are still available. HiveContext is only packaged separately to avoid including all of Hive's dependencies in the default Spark build. If these dependencies are not a problem for your application then using HiveContext is recommended for the 1.2 release of Spark. Future releases will focus on bringing SQLContext up to @@ -146,11 +146,11 @@ describes the various methods for loading data into a SchemaRDD. Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This -reflection based approach leads to more concise code and works well went the schema is known ahead -of time, while you are writing your Spark application. +reflection based approach leads to more concise code and works well when you already know the schema +while writing your Spark application. The second method for creating SchemaRDDs is through a programmatic interface that allows you to -construct a schema and then apply it to and existing RDD. While this method is more verbose, it allows +construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows you to construct SchemaRDDs when the columns and their types are not known until runtime. ### Inferring the Schema Using Reflection @@ -266,10 +266,10 @@ List teenagerNames = teenagers.map(new Function() {
    From f90ad5d426cb726079c490a9bb4b1100e2b4e602 Mon Sep 17 00:00:00 2001 From: Niklas Wilcke <1wilcke@informatik.uni-hamburg.de> Date: Tue, 4 Nov 2014 09:57:03 -0800 Subject: [PATCH 1219/1492] [Spark-4060] [MLlib] exposing special rdd functions to the public Author: Niklas Wilcke <1wilcke@informatik.uni-hamburg.de> Closes #2907 from numbnut/master and squashes the following commits: 7f7c767 [Niklas Wilcke] [Spark-4060] [MLlib] exposing special rdd functions to the public, #2907 --- .../spark/mllib/evaluation/AreaUnderCurve.scala | 2 +- .../org/apache/spark/mllib/rdd/RDDFunctions.scala | 11 ++++++----- .../scala/org/apache/spark/mllib/rdd/SlidingRDD.scala | 5 +++-- .../apache/spark/mllib/rdd/RDDFunctionsSuite.scala | 6 +++--- 4 files changed, 13 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala index 7858ec602483f..078fbfbe4f0e1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala @@ -43,7 +43,7 @@ private[evaluation] object AreaUnderCurve { */ def of(curve: RDD[(Double, Double)]): Double = { curve.sliding(2).aggregate(0.0)( - seqOp = (auc: Double, points: Seq[(Double, Double)]) => auc + trapezoid(points), + seqOp = (auc: Double, points: Array[(Double, Double)]) => auc + trapezoid(points), combOp = _ + _ ) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index b5e403bc8c14d..57c0768084e41 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.rdd import scala.language.implicitConversions import scala.reflect.ClassTag +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.HashPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD @@ -28,8 +29,8 @@ import org.apache.spark.util.Utils /** * Machine learning specific RDD functions. */ -private[mllib] -class RDDFunctions[T: ClassTag](self: RDD[T]) { +@DeveloperApi +class RDDFunctions[T: ClassTag](self: RDD[T]) extends Serializable { /** * Returns a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding @@ -39,10 +40,10 @@ class RDDFunctions[T: ClassTag](self: RDD[T]) { * trigger a Spark job if the parent RDD has more than one partitions and the window size is * greater than 1. */ - def sliding(windowSize: Int): RDD[Seq[T]] = { + def sliding(windowSize: Int): RDD[Array[T]] = { require(windowSize > 0, s"Sliding window size must be positive, but got $windowSize.") if (windowSize == 1) { - self.map(Seq(_)) + self.map(Array(_)) } else { new SlidingRDD[T](self, windowSize) } @@ -112,7 +113,7 @@ class RDDFunctions[T: ClassTag](self: RDD[T]) { } } -private[mllib] +@DeveloperApi object RDDFunctions { /** Implicit conversion from an RDD to RDDFunctions. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala index dd80782c0f001..35e81fcb3de0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala @@ -45,15 +45,16 @@ class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T] */ private[mllib] class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int) - extends RDD[Seq[T]](parent) { + extends RDD[Array[T]](parent) { require(windowSize > 1, s"Window size must be greater than 1, but got $windowSize.") - override def compute(split: Partition, context: TaskContext): Iterator[Seq[T]] = { + override def compute(split: Partition, context: TaskContext): Iterator[Array[T]] = { val part = split.asInstanceOf[SlidingRDDPartition[T]] (firstParent[T].iterator(part.prev, context) ++ part.tail) .sliding(windowSize) .withPartial(false) + .map(_.toArray) } override def getPreferredLocations(split: Partition): Seq[String] = diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala index 27a19f793242b..4ef67a40b9f49 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala @@ -42,9 +42,9 @@ class RDDFunctionsSuite extends FunSuite with LocalSparkContext { val data = Seq(Seq(1, 2, 3), Seq.empty[Int], Seq(4), Seq.empty[Int], Seq(5, 6, 7)) val rdd = sc.parallelize(data, data.length).flatMap(s => s) assert(rdd.partitions.size === data.length) - val sliding = rdd.sliding(3) - val expected = data.flatMap(x => x).sliding(3).toList - assert(sliding.collect().toList === expected) + val sliding = rdd.sliding(3).collect().toSeq.map(_.toSeq) + val expected = data.flatMap(x => x).sliding(3).toSeq.map(_.toSeq) + assert(sliding === expected) } test("treeAggregate") { From 5e73138a0152b78380b3f1def4b969b58e70dd11 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 4 Nov 2014 16:15:38 -0800 Subject: [PATCH 1220/1492] [SPARK-2938] Support SASL authentication in NettyBlockTransferService Also lays the groundwork for supporting it inside the external shuffle service. Author: Aaron Davidson Closes #3087 from aarondav/sasl and squashes the following commits: 3481718 [Aaron Davidson] Delete rogue println 44f8410 [Aaron Davidson] Delete documentation - muahaha! eb9f065 [Aaron Davidson] Improve documentation and add end-to-end test at Spark-level a6b95f1 [Aaron Davidson] Address comments 785bbde [Aaron Davidson] Cleanup 79973cb [Aaron Davidson] Remove unused file 151b3c5 [Aaron Davidson] Add docs, timeout config, better failure handling f6177d7 [Aaron Davidson] Cleanup SASL state upon connection termination 7b42adb [Aaron Davidson] Add unit tests 8191bcb [Aaron Davidson] [SPARK-2938] Support SASL authentication in NettyBlockTransferService --- .../org/apache/spark/SecurityManager.scala | 23 ++- .../scala/org/apache/spark/SparkConf.scala | 6 + .../scala/org/apache/spark/SparkContext.scala | 2 + .../scala/org/apache/spark/SparkEnv.scala | 3 +- .../org/apache/spark/SparkSaslClient.scala | 147 --------------- .../org/apache/spark/SparkSaslServer.scala | 176 ------------------ .../org/apache/spark/executor/Executor.scala | 1 + .../netty/NettyBlockTransferService.scala | 28 ++- .../apache/spark/network/nio/Connection.scala | 5 +- .../spark/network/nio/ConnectionManager.scala | 7 +- .../apache/spark/storage/BlockManager.scala | 45 +++-- .../NettyBlockTransferSecuritySuite.scala | 161 ++++++++++++++++ .../network/nio/ConnectionManagerSuite.scala | 6 +- .../BlockManagerReplicationSuite.scala | 2 + .../spark/storage/BlockManagerSuite.scala | 4 +- docs/security.md | 1 - .../spark/network/TransportContext.java | 15 +- .../spark/network/client/TransportClient.java | 11 +- .../client/TransportClientBootstrap.java | 32 ++++ .../client/TransportClientFactory.java | 64 +++++-- .../spark/network/server/NoOpRpcHandler.java | 2 +- .../spark/network/server/RpcHandler.java | 19 +- .../server/TransportRequestHandler.java | 1 + .../spark/network/util/TransportConf.java | 3 + .../network/sasl/SaslClientBootstrap.java | 74 ++++++++ .../spark/network/sasl/SaslMessage.java | 74 ++++++++ .../spark/network/sasl/SaslRpcHandler.java | 97 ++++++++++ .../spark/network/sasl/SecretKeyHolder.java | 35 ++++ .../spark/network/sasl/SparkSaslClient.java | 138 ++++++++++++++ .../spark/network/sasl/SparkSaslServer.java | 170 +++++++++++++++++ .../shuffle/ExternalShuffleBlockHandler.java | 2 +- .../shuffle/ExternalShuffleClient.java | 15 +- .../spark/network/shuffle/ShuffleClient.java | 11 +- .../network/sasl/SaslIntegrationSuite.java | 172 +++++++++++++++++ .../spark/network/sasl/SparkSaslSuite.java | 89 +++++++++ .../ExternalShuffleIntegrationSuite.java | 7 +- .../streaming/ReceivedBlockHandlerSuite.scala | 1 + 37 files changed, 1257 insertions(+), 392 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/SparkSaslClient.scala delete mode 100644 core/src/main/scala/org/apache/spark/SparkSaslServer.scala create mode 100644 core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala create mode 100644 network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 0e0f1a7b2377e..dee935ffad51f 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -22,6 +22,7 @@ import java.net.{Authenticator, PasswordAuthentication} import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.network.sasl.SecretKeyHolder /** * Spark class responsible for security. @@ -84,7 +85,7 @@ import org.apache.spark.deploy.SparkHadoopUtil * Authenticator installed in the SecurityManager to how it does the authentication * and in this case gets the user name and password from the request. * - * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously + * - BlockTransferService -> The Spark BlockTransferServices uses java nio to asynchronously * exchange messages. For this we use the Java SASL * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 * as the authentication mechanism. This means the shared secret is not passed @@ -98,7 +99,7 @@ import org.apache.spark.deploy.SparkHadoopUtil * of protection they want. If we support those, the messages will also have to * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. * - * Since the connectionManager does asynchronous messages passing, the SASL + * Since the NioBlockTransferService does asynchronous messages passing, the SASL * authentication is a bit more complex. A ConnectionManager can be both a client * and a Server, so for a particular connection is has to determine what to do. * A ConnectionId was added to be able to track connections and is used to @@ -107,6 +108,10 @@ import org.apache.spark.deploy.SparkHadoopUtil * and waits for the response from the server and does the handshake before sending * the real message. * + * The NettyBlockTransferService ensures that SASL authentication is performed + * synchronously prior to any other communication on a connection. This is done in + * SaslClientBootstrap on the client side and SaslRpcHandler on the server side. + * * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work * properly. For non-Yarn deployments, users can write a filter to go through a @@ -139,7 +144,7 @@ import org.apache.spark.deploy.SparkHadoopUtil * can take place. */ -private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { +private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with SecretKeyHolder { // key used to store the spark secret in the Hadoop UGI private val sparkSecretLookupKey = "sparkCookie" @@ -337,4 +342,16 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { * @return the secret key as a String if authentication is enabled, otherwise returns null */ def getSecretKey(): String = secretKey + + override def getSaslUser(appId: String): String = { + val myAppId = sparkConf.getAppId + require(appId == myAppId, s"SASL appId $appId did not match my appId ${myAppId}") + getSaslUser() + } + + override def getSecretKey(appId: String): String = { + val myAppId = sparkConf.getAppId + require(appId == myAppId, s"SASL appId $appId did not match my appId ${myAppId}") + getSecretKey() + } } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index ad0a9017afead..4c6c86c7bad78 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -217,6 +217,12 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { */ getAll.filter { case (k, _) => isAkkaConf(k) } + /** + * Returns the Spark application id, valid in the Driver after TaskScheduler registration and + * from the start in the Executor. + */ + def getAppId: String = get("spark.app.id") + /** Does the configuration contain a given parameter? */ def contains(key: String): Boolean = settings.contains(key) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 40444c237b738..3cdaa6a9cc8a8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -313,6 +313,8 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { val applicationId: String = taskScheduler.applicationId() conf.set("spark.app.id", applicationId) + env.blockManager.initialize(applicationId) + val metricsSystem = env.metricsSystem // The metrics system for Driver need to be set spark.app.id to app ID. diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index e2f13accdfab5..45e9d7f243e96 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -276,7 +276,7 @@ object SparkEnv extends Logging { val blockTransferService = conf.get("spark.shuffle.blockTransferService", "netty").toLowerCase match { case "netty" => - new NettyBlockTransferService(conf) + new NettyBlockTransferService(conf, securityManager) case "nio" => new NioBlockTransferService(conf, securityManager) } @@ -285,6 +285,7 @@ object SparkEnv extends Logging { "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) + // NB: blockManager is not valid until initialize() is called later. val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf, mapOutputTracker, shuffleManager, blockTransferService) diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala deleted file mode 100644 index a954fcc0c31fa..0000000000000 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import javax.security.auth.callback.Callback -import javax.security.auth.callback.CallbackHandler -import javax.security.auth.callback.NameCallback -import javax.security.auth.callback.PasswordCallback -import javax.security.auth.callback.UnsupportedCallbackException -import javax.security.sasl.RealmCallback -import javax.security.sasl.RealmChoiceCallback -import javax.security.sasl.Sasl -import javax.security.sasl.SaslClient -import javax.security.sasl.SaslException - -import scala.collection.JavaConversions.mapAsJavaMap - -import com.google.common.base.Charsets.UTF_8 - -/** - * Implements SASL Client logic for Spark - */ -private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logging { - - /** - * Used to respond to server's counterpart, SaslServer with SASL tokens - * represented as byte arrays. - * - * The authentication mechanism used here is DIGEST-MD5. This could be changed to be - * configurable in the future. - */ - private var saslClient: SaslClient = Sasl.createSaslClient(Array[String](SparkSaslServer.DIGEST), - null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, - new SparkSaslClientCallbackHandler(securityMgr)) - - /** - * Used to initiate SASL handshake with server. - * @return response to challenge if needed - */ - def firstToken(): Array[Byte] = { - synchronized { - val saslToken: Array[Byte] = - if (saslClient != null && saslClient.hasInitialResponse()) { - logDebug("has initial response") - saslClient.evaluateChallenge(new Array[Byte](0)) - } else { - new Array[Byte](0) - } - saslToken - } - } - - /** - * Determines whether the authentication exchange has completed. - * @return true is complete, otherwise false - */ - def isComplete(): Boolean = { - synchronized { - if (saslClient != null) saslClient.isComplete() else false - } - } - - /** - * Respond to server's SASL token. - * @param saslTokenMessage contains server's SASL token - * @return client's response SASL token - */ - def saslResponse(saslTokenMessage: Array[Byte]): Array[Byte] = { - synchronized { - if (saslClient != null) saslClient.evaluateChallenge(saslTokenMessage) else new Array[Byte](0) - } - } - - /** - * Disposes of any system resources or security-sensitive information the - * SaslClient might be using. - */ - def dispose() { - synchronized { - if (saslClient != null) { - try { - saslClient.dispose() - } catch { - case e: SaslException => // ignored - } finally { - saslClient = null - } - } - } - } - - /** - * Implementation of javax.security.auth.callback.CallbackHandler - * that works with share secrets. - */ - private class SparkSaslClientCallbackHandler(securityMgr: SecurityManager) extends - CallbackHandler { - - private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes(UTF_8)) - private val secretKey = securityMgr.getSecretKey() - private val userPassword: Array[Char] = SparkSaslServer.encodePassword( - if (secretKey != null) secretKey.getBytes(UTF_8) else "".getBytes(UTF_8)) - - /** - * Implementation used to respond to SASL request from the server. - * - * @param callbacks objects that indicate what credential information the - * server's SaslServer requires from the client. - */ - override def handle(callbacks: Array[Callback]) { - logDebug("in the sasl client callback handler") - callbacks foreach { - case nc: NameCallback => { - logDebug("handle: SASL client callback: setting username: " + userName) - nc.setName(userName) - } - case pc: PasswordCallback => { - logDebug("handle: SASL client callback: setting userPassword") - pc.setPassword(userPassword) - } - case rc: RealmCallback => { - logDebug("handle: SASL client callback: setting realm: " + rc.getDefaultText()) - rc.setText(rc.getDefaultText()) - } - case cb: RealmChoiceCallback => {} - case cb: Callback => throw - new UnsupportedCallbackException(cb, "handle: Unrecognized SASL client callback") - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala deleted file mode 100644 index 7c2afb364661f..0000000000000 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ /dev/null @@ -1,176 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import javax.security.auth.callback.Callback -import javax.security.auth.callback.CallbackHandler -import javax.security.auth.callback.NameCallback -import javax.security.auth.callback.PasswordCallback -import javax.security.auth.callback.UnsupportedCallbackException -import javax.security.sasl.AuthorizeCallback -import javax.security.sasl.RealmCallback -import javax.security.sasl.Sasl -import javax.security.sasl.SaslException -import javax.security.sasl.SaslServer -import scala.collection.JavaConversions.mapAsJavaMap - -import com.google.common.base.Charsets.UTF_8 -import org.apache.commons.net.util.Base64 - -/** - * Encapsulates SASL server logic - */ -private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Logging { - - /** - * Actual SASL work done by this object from javax.security.sasl. - */ - private var saslServer: SaslServer = Sasl.createSaslServer(SparkSaslServer.DIGEST, null, - SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, - new SparkSaslDigestCallbackHandler(securityMgr)) - - /** - * Determines whether the authentication exchange has completed. - * @return true is complete, otherwise false - */ - def isComplete(): Boolean = { - synchronized { - if (saslServer != null) saslServer.isComplete() else false - } - } - - /** - * Used to respond to server SASL tokens. - * @param token Server's SASL token - * @return response to send back to the server. - */ - def response(token: Array[Byte]): Array[Byte] = { - synchronized { - if (saslServer != null) saslServer.evaluateResponse(token) else new Array[Byte](0) - } - } - - /** - * Disposes of any system resources or security-sensitive information the - * SaslServer might be using. - */ - def dispose() { - synchronized { - if (saslServer != null) { - try { - saslServer.dispose() - } catch { - case e: SaslException => // ignore - } finally { - saslServer = null - } - } - } - } - - /** - * Implementation of javax.security.auth.callback.CallbackHandler - * for SASL DIGEST-MD5 mechanism - */ - private class SparkSaslDigestCallbackHandler(securityMgr: SecurityManager) - extends CallbackHandler { - - private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes(UTF_8)) - - override def handle(callbacks: Array[Callback]) { - logDebug("In the sasl server callback handler") - callbacks foreach { - case nc: NameCallback => { - logDebug("handle: SASL server callback: setting username") - nc.setName(userName) - } - case pc: PasswordCallback => { - logDebug("handle: SASL server callback: setting userPassword") - val password: Array[Char] = - SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes(UTF_8)) - pc.setPassword(password) - } - case rc: RealmCallback => { - logDebug("handle: SASL server callback: setting realm: " + rc.getDefaultText()) - rc.setText(rc.getDefaultText()) - } - case ac: AuthorizeCallback => { - val authid = ac.getAuthenticationID() - val authzid = ac.getAuthorizationID() - if (authid.equals(authzid)) { - logDebug("set auth to true") - ac.setAuthorized(true) - } else { - logDebug("set auth to false") - ac.setAuthorized(false) - } - if (ac.isAuthorized()) { - logDebug("sasl server is authorized") - ac.setAuthorizedID(authzid) - } - } - case cb: Callback => throw - new UnsupportedCallbackException(cb, "handle: Unrecognized SASL DIGEST-MD5 Callback") - } - } - } -} - -private[spark] object SparkSaslServer { - - /** - * This is passed as the server name when creating the sasl client/server. - * This could be changed to be configurable in the future. - */ - val SASL_DEFAULT_REALM = "default" - - /** - * The authentication mechanism used here is DIGEST-MD5. This could be changed to be - * configurable in the future. - */ - val DIGEST = "DIGEST-MD5" - - /** - * The quality of protection is just "auth". This means that we are doing - * authentication only, we are not supporting integrity or privacy protection of the - * communication channel after authentication. This could be changed to be configurable - * in the future. - */ - val SASL_PROPS = Map(Sasl.QOP -> "auth", Sasl.SERVER_AUTH ->"true") - - /** - * Encode a byte[] identifier as a Base64-encoded string. - * - * @param identifier identifier to encode - * @return Base64-encoded string - */ - def encodeIdentifier(identifier: Array[Byte]): String = { - new String(Base64.encodeBase64(identifier), UTF_8) - } - - /** - * Encode a password as a base64-encoded char[] array. - * @param password as a byte array. - * @return password as a char array. - */ - def encodePassword(password: Array[Byte]): Array[Char] = { - new String(Base64.encodeBase64(password), UTF_8).toCharArray() - } -} - diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 8b095e23f32ff..abc1dd0be6237 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -86,6 +86,7 @@ private[spark] class Executor( conf, executorId, slaveHostname, port, isLocal, actorSystem) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) + _env.blockManager.initialize(conf.getAppId) _env } else { SparkEnv.get diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 1c4327cf13b51..0d1fc81d2a16f 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -17,13 +17,15 @@ package org.apache.spark.network.netty +import scala.collection.JavaConversions._ import scala.concurrent.{Future, Promise} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.client.{RpcResponseCallback, TransportClientFactory} +import org.apache.spark.network.client.{TransportClientBootstrap, RpcResponseCallback, TransportClientFactory} import org.apache.spark.network.netty.NettyMessages.{OpenBlocks, UploadBlock} +import org.apache.spark.network.sasl.{SaslRpcHandler, SaslClientBootstrap} import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher} import org.apache.spark.serializer.JavaSerializer @@ -33,18 +35,30 @@ import org.apache.spark.util.Utils /** * A BlockTransferService that uses Netty to fetch a set of blocks at at time. */ -class NettyBlockTransferService(conf: SparkConf) extends BlockTransferService { +class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManager) + extends BlockTransferService { + // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. - val serializer = new JavaSerializer(conf) + private val serializer = new JavaSerializer(conf) + private val authEnabled = securityManager.isAuthenticationEnabled() + private val transportConf = SparkTransportConf.fromSparkConf(conf) private[this] var transportContext: TransportContext = _ private[this] var server: TransportServer = _ private[this] var clientFactory: TransportClientFactory = _ override def init(blockDataManager: BlockDataManager): Unit = { - val rpcHandler = new NettyBlockRpcServer(serializer, blockDataManager) - transportContext = new TransportContext(SparkTransportConf.fromSparkConf(conf), rpcHandler) - clientFactory = transportContext.createClientFactory() + val (rpcHandler: RpcHandler, bootstrap: Option[TransportClientBootstrap]) = { + val nettyRpcHandler = new NettyBlockRpcServer(serializer, blockDataManager) + if (!authEnabled) { + (nettyRpcHandler, None) + } else { + (new SaslRpcHandler(nettyRpcHandler, securityManager), + Some(new SaslClientBootstrap(transportConf, conf.getAppId, securityManager))) + } + } + transportContext = new TransportContext(transportConf, rpcHandler) + clientFactory = transportContext.createClientFactory(bootstrap.toList) server = transportContext.createServer() logInfo("Server created on " + server.getPort) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 4f6f5e235811d..c2d9578be7ebb 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -23,12 +23,13 @@ import java.nio.channels._ import java.util.concurrent.ConcurrentLinkedQueue import java.util.LinkedList -import org.apache.spark._ - import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal +import org.apache.spark._ +import org.apache.spark.network.sasl.{SparkSaslClient, SparkSaslServer} + private[nio] abstract class Connection(val channel: SocketChannel, val selector: Selector, val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId, diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 8408b75bb4d65..f198aa8564a54 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -34,6 +34,7 @@ import scala.language.postfixOps import com.google.common.base.Charsets.UTF_8 import org.apache.spark._ +import org.apache.spark.network.sasl.{SparkSaslClient, SparkSaslServer} import org.apache.spark.util.Utils import scala.util.Try @@ -600,7 +601,7 @@ private[nio] class ConnectionManager( } else { var replyToken : Array[Byte] = null try { - replyToken = waitingConn.sparkSaslClient.saslResponse(securityMsg.getToken) + replyToken = waitingConn.sparkSaslClient.response(securityMsg.getToken) if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed after evaluate for id: " + waitingConn.connectionId) connectionsAwaitingSasl -= waitingConn.connectionId @@ -634,7 +635,7 @@ private[nio] class ConnectionManager( connection.synchronized { if (connection.sparkSaslServer == null) { logDebug("Creating sasl Server") - connection.sparkSaslServer = new SparkSaslServer(securityManager) + connection.sparkSaslServer = new SparkSaslServer(conf.getAppId, securityManager) } } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) @@ -778,7 +779,7 @@ private[nio] class ConnectionManager( if (!conn.isSaslComplete()) { conn.synchronized { if (conn.sparkSaslClient == null) { - conn.sparkSaslClient = new SparkSaslClient(securityManager) + conn.sparkSaslClient = new SparkSaslClient(conf.getAppId, securityManager) var firstResponse: Array[Byte] = null try { firstResponse = conn.sparkSaslClient.firstToken() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 5f5dd0dc1c63f..655d16c65c8b5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -57,6 +57,12 @@ private[spark] class BlockResult( inputMetrics.bytesRead = bytes } +/** + * Manager running on every node (driver and executors) which provides interfaces for putting and + * retrieving blocks both locally and remotely into various stores (memory, disk, and off-heap). + * + * Note that #initialize() must be called before the BlockManager is usable. + */ private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, @@ -69,8 +75,6 @@ private[spark] class BlockManager( blockTransferService: BlockTransferService) extends BlockDataManager with Logging { - blockTransferService.init(this) - val diskBlockManager = new DiskBlockManager(this, conf) private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -102,22 +106,16 @@ private[spark] class BlockManager( + " switch to sort-based shuffle.") } - val blockManagerId = BlockManagerId( - executorId, blockTransferService.hostName, blockTransferService.port) + var blockManagerId: BlockManagerId = _ // Address of the server that serves this executor's shuffle files. This is either an external // service, or just our own Executor's BlockManager. - private[spark] val shuffleServerId = if (externalShuffleServiceEnabled) { - BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort) - } else { - blockManagerId - } + private[spark] var shuffleServerId: BlockManagerId = _ // Client to read other executors' shuffle files. This is either an external service, or just the // standard BlockTranserService to directly connect to other Executors. private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { - val appId = conf.get("spark.app.id", "unknown-app-id") - new ExternalShuffleClient(SparkTransportConf.fromSparkConf(conf), appId) + new ExternalShuffleClient(SparkTransportConf.fromSparkConf(conf)) } else { blockTransferService } @@ -150,8 +148,6 @@ private[spark] class BlockManager( private val peerFetchLock = new Object private var lastPeerFetchTime = 0L - initialize() - /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay * the initialization of the compression codec until it is first used. The reason is that a Spark * program could be using a user-defined codec in a third party jar, which is loaded in @@ -176,10 +172,27 @@ private[spark] class BlockManager( } /** - * Initialize the BlockManager. Register to the BlockManagerMaster, and start the - * BlockManagerWorker actor. Additionally registers with a local shuffle service if configured. + * Initializes the BlockManager with the given appId. This is not performed in the constructor as + * the appId may not be known at BlockManager instantiation time (in particular for the driver, + * where it is only learned after registration with the TaskScheduler). + * + * This method initializes the BlockTransferService and ShuffleClient, registers with the + * BlockManagerMaster, starts the BlockManagerWorker actor, and registers with a local shuffle + * service if configured. */ - private def initialize(): Unit = { + def initialize(appId: String): Unit = { + blockTransferService.init(this) + shuffleClient.init(appId) + + blockManagerId = BlockManagerId( + executorId, blockTransferService.hostName, blockTransferService.port) + + shuffleServerId = if (externalShuffleServiceEnabled) { + BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort) + } else { + blockManagerId + } + master.registerBlockManager(blockManagerId, maxMemory, slaveActor) // Register Executors' configuration with the local shuffle service, if one should exist. diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala new file mode 100644 index 0000000000000..bed0ed9d713dd --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.nio._ +import java.util.concurrent.TimeUnit + +import scala.concurrent.duration._ +import scala.concurrent.{Await, Promise} +import scala.util.{Failure, Success, Try} + +import org.apache.commons.io.IOUtils +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.shuffle.BlockFetchingListener +import org.apache.spark.network.{BlockDataManager, BlockTransferService} +import org.apache.spark.storage.{BlockId, ShuffleBlockId} +import org.apache.spark.{SecurityManager, SparkConf} +import org.mockito.Mockito._ +import org.scalatest.mock.MockitoSugar +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, ShouldMatchers} + +class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with ShouldMatchers { + test("security default off") { + testConnection(new SparkConf, new SparkConf) match { + case Success(_) => // expected + case Failure(t) => fail(t) + } + } + + test("security on same password") { + val conf = new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.app.id", "app-id") + testConnection(conf, conf) match { + case Success(_) => // expected + case Failure(t) => fail(t) + } + } + + test("security on mismatch password") { + val conf0 = new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.app.id", "app-id") + val conf1 = conf0.clone.set("spark.authenticate.secret", "bad") + testConnection(conf0, conf1) match { + case Success(_) => fail("Should have failed") + case Failure(t) => t.getMessage should include ("Mismatched response") + } + } + + test("security mismatch auth off on server") { + val conf0 = new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.app.id", "app-id") + val conf1 = conf0.clone.set("spark.authenticate", "false") + testConnection(conf0, conf1) match { + case Success(_) => fail("Should have failed") + case Failure(t) => // any funny error may occur, sever will interpret SASL token as RPC + } + } + + test("security mismatch auth off on client") { + val conf0 = new SparkConf() + .set("spark.authenticate", "false") + .set("spark.authenticate.secret", "good") + .set("spark.app.id", "app-id") + val conf1 = conf0.clone.set("spark.authenticate", "true") + testConnection(conf0, conf1) match { + case Success(_) => fail("Should have failed") + case Failure(t) => t.getMessage should include ("Expected SaslMessage") + } + } + + test("security mismatch app ids") { + val conf0 = new SparkConf() + .set("spark.authenticate", "true") + .set("spark.authenticate.secret", "good") + .set("spark.app.id", "app-id") + val conf1 = conf0.clone.set("spark.app.id", "other-id") + testConnection(conf0, conf1) match { + case Success(_) => fail("Should have failed") + case Failure(t) => t.getMessage should include ("SASL appId app-id did not match") + } + } + + /** + * Creates two servers with different configurations and sees if they can talk. + * Returns Success() if they can transfer a block, and Failure() if the block transfer was failed + * properly. We will throw an out-of-band exception if something other than that goes wrong. + */ + private def testConnection(conf0: SparkConf, conf1: SparkConf): Try[Unit] = { + val blockManager = mock[BlockDataManager] + val blockId = ShuffleBlockId(0, 1, 2) + val blockString = "Hello, world!" + val blockBuffer = new NioManagedBuffer(ByteBuffer.wrap(blockString.getBytes)) + when(blockManager.getBlockData(blockId)).thenReturn(blockBuffer) + + val securityManager0 = new SecurityManager(conf0) + val exec0 = new NettyBlockTransferService(conf0, securityManager0) + exec0.init(blockManager) + + val securityManager1 = new SecurityManager(conf1) + val exec1 = new NettyBlockTransferService(conf1, securityManager1) + exec1.init(blockManager) + + val result = fetchBlock(exec0, exec1, "1", blockId) match { + case Success(buf) => + IOUtils.toString(buf.createInputStream()) should equal(blockString) + buf.release() + Success() + case Failure(t) => + Failure(t) + } + exec0.close() + exec1.close() + result + } + + /** Synchronously fetches a single block, acting as the given executor fetching from another. */ + private def fetchBlock( + self: BlockTransferService, + from: BlockTransferService, + execId: String, + blockId: BlockId): Try[ManagedBuffer] = { + + val promise = Promise[ManagedBuffer]() + + self.fetchBlocks(from.hostName, from.port, execId, Array(blockId.toString), + new BlockFetchingListener { + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + promise.failure(exception) + } + + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + promise.success(data.retain()) + } + }) + + Await.ready(promise.future, FiniteDuration(1000, TimeUnit.MILLISECONDS)) + promise.future.value.get + } +} + diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index b70734dfe37cf..716f875d30b8a 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -60,6 +60,7 @@ class ConnectionManagerSuite extends FunSuite { val conf = new SparkConf conf.set("spark.authenticate", "true") conf.set("spark.authenticate.secret", "good") + conf.set("spark.app.id", "app-id") val securityManager = new SecurityManager(conf) val manager = new ConnectionManager(0, conf, securityManager) var numReceivedMessages = 0 @@ -95,6 +96,7 @@ class ConnectionManagerSuite extends FunSuite { test("security mismatch password") { val conf = new SparkConf conf.set("spark.authenticate", "true") + conf.set("spark.app.id", "app-id") conf.set("spark.authenticate.secret", "good") val securityManager = new SecurityManager(conf) val manager = new ConnectionManager(0, conf, securityManager) @@ -105,9 +107,7 @@ class ConnectionManagerSuite extends FunSuite { None }) - val badconf = new SparkConf - badconf.set("spark.authenticate", "true") - badconf.set("spark.authenticate.secret", "bad") + val badconf = conf.clone.set("spark.authenticate.secret", "bad") val badsecurityManager = new SecurityManager(badconf) val managerServer = new ConnectionManager(0, badconf, badsecurityManager) var numReceivedServerMessages = 0 diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index c6d7105592096..1461fa69db90d 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -63,6 +63,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd val transfer = new NioBlockTransferService(conf, securityMgr) val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer) + store.initialize("app-id") allStores += store store } @@ -263,6 +264,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd when(failableTransfer.port).thenReturn(1000) val failableStore = new BlockManager("failable-store", actorSystem, master, serializer, 10000, conf, mapOutputTracker, shuffleManager, failableTransfer) + failableStore.initialize("app-id") allStores += failableStore // so that this gets stopped after test assert(master.getPeers(store.blockManagerId).toSet === Set(failableStore.blockManagerId)) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 715b740b857b2..0782876c8e3c6 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -73,8 +73,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) - new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + val manager = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer) + manager.initialize("app-id") + manager } before { diff --git a/docs/security.md b/docs/security.md index ec0523184d665..1e206a139fb72 100644 --- a/docs/security.md +++ b/docs/security.md @@ -7,7 +7,6 @@ Spark currently supports authentication via a shared secret. Authentication can * For Spark on [YARN](running-on-yarn.html) deployments, configuring `spark.authenticate` to `true` will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. * For other types of Spark deployments, the Spark parameter `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. -* **IMPORTANT NOTE:** *The experimental Netty shuffle path (`spark.shuffle.use.netty`) is not secured, so do not use Netty for shuffles if running with authentication.* ## Web UI diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java index a271841e4e56c..5bc6e5a2418a9 100644 --- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -17,12 +17,16 @@ package org.apache.spark.network; +import java.util.List; + +import com.google.common.collect.Lists; import io.netty.channel.Channel; import io.netty.channel.socket.SocketChannel; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; import org.apache.spark.network.client.TransportClientFactory; import org.apache.spark.network.client.TransportResponseHandler; import org.apache.spark.network.protocol.MessageDecoder; @@ -64,8 +68,17 @@ public TransportContext(TransportConf conf, RpcHandler rpcHandler) { this.decoder = new MessageDecoder(); } + /** + * Initializes a ClientFactory which runs the given TransportClientBootstraps prior to returning + * a new Client. Bootstraps will be executed synchronously, and must run successfully in order + * to create a Client. + */ + public TransportClientFactory createClientFactory(List bootstraps) { + return new TransportClientFactory(this, bootstraps); + } + public TransportClientFactory createClientFactory() { - return new TransportClientFactory(this); + return createClientFactory(Lists.newArrayList()); } /** Create a server which will attempt to bind to a specific port. */ diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java index 01c143fff423c..a08cee02dd576 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -19,10 +19,9 @@ import java.io.Closeable; import java.util.UUID; -import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; +import com.google.common.base.Objects; import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.util.concurrent.SettableFuture; @@ -186,4 +185,12 @@ public void close() { // close is a local operation and should finish with milliseconds; timeout just to be safe channel.close().awaitUninterruptibly(10, TimeUnit.SECONDS); } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("remoteAdress", channel.remoteAddress()) + .add("isActive", isActive()) + .toString(); + } } diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java new file mode 100644 index 0000000000000..65e8020e34121 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientBootstrap.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +/** + * A bootstrap which is executed on a TransportClient before it is returned to the user. + * This enables an initial exchange of information (e.g., SASL authentication tokens) on a once-per- + * connection basis. + * + * Since connections (and TransportClients) are reused as much as possible, it is generally + * reasonable to perform an expensive bootstrapping operation, as they often share a lifespan with + * the JVM itself. + */ +public interface TransportClientBootstrap { + /** Performs the bootstrapping operation, throwing an exception on failure. */ + public void doBootstrap(TransportClient client) throws RuntimeException; +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 0b4a1d8286407..1723fed307257 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -21,10 +21,14 @@ import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.net.SocketAddress; +import java.util.List; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.Lists; import io.netty.bootstrap.Bootstrap; import io.netty.buffer.PooledByteBufAllocator; import io.netty.channel.Channel; @@ -40,6 +44,7 @@ import org.apache.spark.network.TransportContext; import org.apache.spark.network.server.TransportChannelHandler; import org.apache.spark.network.util.IOMode; +import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -47,22 +52,29 @@ * Factory for creating {@link TransportClient}s by using createClient. * * The factory maintains a connection pool to other hosts and should return the same - * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for - * all {@link TransportClient}s. + * TransportClient for the same remote host. It also shares a single worker thread pool for + * all TransportClients. + * + * TransportClients will be reused whenever possible. Prior to completing the creation of a new + * TransportClient, all given {@link TransportClientBootstrap}s will be run. */ public class TransportClientFactory implements Closeable { private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); private final TransportContext context; private final TransportConf conf; + private final List clientBootstraps; private final ConcurrentHashMap connectionPool; private final Class socketChannelClass; private EventLoopGroup workerGroup; - public TransportClientFactory(TransportContext context) { - this.context = context; + public TransportClientFactory( + TransportContext context, + List clientBootstraps) { + this.context = Preconditions.checkNotNull(context); this.conf = context.getConf(); + this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps)); this.connectionPool = new ConcurrentHashMap(); IOMode ioMode = IOMode.valueOf(conf.ioMode()); @@ -72,9 +84,12 @@ public TransportClientFactory(TransportContext context) { } /** - * Create a new BlockFetchingClient connecting to the given remote host / port. + * Create a new {@link TransportClient} connecting to the given remote host / port. This will + * reuse TransportClients if they are still active and are for the same remote address. Prior + * to the creation of a new TransportClient, we will execute all {@link TransportClientBootstrap}s + * that are registered with this factory. * - * This blocks until a connection is successfully established. + * This blocks until a connection is successfully established and fully bootstrapped. * * Concurrency: This method is safe to call from multiple threads. */ @@ -104,17 +119,18 @@ public TransportClient createClient(String remoteHost, int remotePort) { // Use pooled buffers to reduce temporary buffer allocation bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator()); - final AtomicReference client = new AtomicReference(); + final AtomicReference clientRef = new AtomicReference(); bootstrap.handler(new ChannelInitializer() { @Override public void initChannel(SocketChannel ch) { TransportChannelHandler clientHandler = context.initializePipeline(ch); - client.set(clientHandler.getClient()); + clientRef.set(clientHandler.getClient()); } }); // Connect to the remote server + long preConnect = System.currentTimeMillis(); ChannelFuture cf = bootstrap.connect(address); if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { throw new RuntimeException( @@ -123,15 +139,35 @@ public void initChannel(SocketChannel ch) { throw new RuntimeException(String.format("Failed to connect to %s", address), cf.cause()); } - // Successful connection -- in the event that two threads raced to create a client, we will + TransportClient client = clientRef.get(); + assert client != null : "Channel future completed successfully with null client"; + + // Execute any client bootstraps synchronously before marking the Client as successful. + long preBootstrap = System.currentTimeMillis(); + logger.debug("Connection to {} successful, running bootstraps...", address); + try { + for (TransportClientBootstrap clientBootstrap : clientBootstraps) { + clientBootstrap.doBootstrap(client); + } + } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala + long bootstrapTime = System.currentTimeMillis() - preBootstrap; + logger.error("Exception while bootstrapping client after " + bootstrapTime + " ms", e); + client.close(); + throw Throwables.propagate(e); + } + long postBootstrap = System.currentTimeMillis(); + + // Successful connection & bootstrap -- in the event that two threads raced to create a client, // use the first one that was put into the connectionPool and close the one we made here. - assert client.get() != null : "Channel future completed successfully with null client"; - TransportClient oldClient = connectionPool.putIfAbsent(address, client.get()); + TransportClient oldClient = connectionPool.putIfAbsent(address, client); if (oldClient == null) { - return client.get(); + logger.debug("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", + address, postBootstrap - preConnect, postBootstrap - preBootstrap); + return client; } else { - logger.debug("Two clients were created concurrently, second one will be disposed."); - client.get().close(); + logger.debug("Two clients were created concurrently after {} ms, second will be disposed.", + postBootstrap - preConnect); + client.close(); return oldClient; } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java index 5a3f003726fc1..1502b7489e864 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java @@ -21,7 +21,7 @@ import org.apache.spark.network.client.TransportClient; /** An RpcHandler suitable for a client-only TransportContext, which cannot receive RPCs. */ -public class NoOpRpcHandler implements RpcHandler { +public class NoOpRpcHandler extends RpcHandler { private final StreamManager streamManager; public NoOpRpcHandler() { diff --git a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java index 2369dc6203944..2ba92a40f8b0a 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -23,22 +23,33 @@ /** * Handler for sendRPC() messages sent by {@link org.apache.spark.network.client.TransportClient}s. */ -public interface RpcHandler { +public abstract class RpcHandler { /** * Receive a single RPC message. Any exception thrown while in this method will be sent back to * the client in string form as a standard RPC failure. * + * This method will not be called in parallel for a single TransportClient (i.e., channel). + * * @param client A channel client which enables the handler to make requests back to the sender - * of this RPC. + * of this RPC. This will always be the exact same object for a particular channel. * @param message The serialized bytes of the RPC. * @param callback Callback which should be invoked exactly once upon success or failure of the * RPC. */ - void receive(TransportClient client, byte[] message, RpcResponseCallback callback); + public abstract void receive( + TransportClient client, + byte[] message, + RpcResponseCallback callback); /** * Returns the StreamManager which contains the state about which streams are currently being * fetched by a TransportClient. */ - StreamManager getStreamManager(); + public abstract StreamManager getStreamManager(); + + /** + * Invoked when the connection associated with the given client has been invalidated. + * No further requests will come from this client. + */ + public void connectionTerminated(TransportClient client) { } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index 17fe9001b35cc..1580180cc17e9 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -86,6 +86,7 @@ public void channelUnregistered() { for (long streamId : streamIds) { streamManager.connectionTerminated(streamId); } + rpcHandler.connectionTerminated(reverseClient); } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index a68f38e0e94c9..823790dd3c66f 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -55,4 +55,7 @@ public int connectionTimeoutMs() { /** Send buffer size (SO_SNDBUF). */ public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } + + /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ + public int saslRTTimeout() { return conf.getInt("spark.shuffle.sasl.timeout", 30000); } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java new file mode 100644 index 0000000000000..7bc91e375371f --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.util.TransportConf; + +/** + * Bootstraps a {@link TransportClient} by performing SASL authentication on the connection. The + * server should be setup with a {@link SaslRpcHandler} with matching keys for the given appId. + */ +public class SaslClientBootstrap implements TransportClientBootstrap { + private final Logger logger = LoggerFactory.getLogger(SaslClientBootstrap.class); + + private final TransportConf conf; + private final String appId; + private final SecretKeyHolder secretKeyHolder; + + public SaslClientBootstrap(TransportConf conf, String appId, SecretKeyHolder secretKeyHolder) { + this.conf = conf; + this.appId = appId; + this.secretKeyHolder = secretKeyHolder; + } + + /** + * Performs SASL authentication by sending a token, and then proceeding with the SASL + * challenge-response tokens until we either successfully authenticate or throw an exception + * due to mismatch. + */ + @Override + public void doBootstrap(TransportClient client) { + SparkSaslClient saslClient = new SparkSaslClient(appId, secretKeyHolder); + try { + byte[] payload = saslClient.firstToken(); + + while (!saslClient.isComplete()) { + SaslMessage msg = new SaslMessage(appId, payload); + ByteBuf buf = Unpooled.buffer(msg.encodedLength()); + msg.encode(buf); + + byte[] response = client.sendRpcSync(buf.array(), conf.saslRTTimeout()); + payload = saslClient.response(response); + } + } finally { + try { + // Once authentication is complete, the server will trust all remaining communication. + saslClient.dispose(); + } catch (RuntimeException e) { + logger.error("Error while disposing SASL client", e); + } + } + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java new file mode 100644 index 0000000000000..5b77e18c26bf4 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import com.google.common.base.Charsets; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encodable; + +/** + * Encodes a Sasl-related message which is attempting to authenticate using some credentials tagged + * with the given appId. This appId allows a single SaslRpcHandler to multiplex different + * applications which may be using different sets of credentials. + */ +class SaslMessage implements Encodable { + + /** Serialization tag used to catch incorrect payloads. */ + private static final byte TAG_BYTE = (byte) 0xEA; + + public final String appId; + public final byte[] payload; + + public SaslMessage(String appId, byte[] payload) { + this.appId = appId; + this.payload = payload; + } + + @Override + public int encodedLength() { + // tag + appIdLength + appId + payloadLength + payload + return 1 + 4 + appId.getBytes(Charsets.UTF_8).length + 4 + payload.length; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeByte(TAG_BYTE); + byte[] idBytes = appId.getBytes(Charsets.UTF_8); + buf.writeInt(idBytes.length); + buf.writeBytes(idBytes); + buf.writeInt(payload.length); + buf.writeBytes(payload); + } + + public static SaslMessage decode(ByteBuf buf) { + if (buf.readByte() != TAG_BYTE) { + throw new IllegalStateException("Expected SaslMessage, received something else"); + } + + int idLength = buf.readInt(); + byte[] idBytes = new byte[idLength]; + buf.readBytes(idBytes); + + int payloadLength = buf.readInt(); + byte[] payload = new byte[payloadLength]; + buf.readBytes(payload); + + return new SaslMessage(new String(idBytes, Charsets.UTF_8), payload); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java new file mode 100644 index 0000000000000..3777a18e33f78 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslRpcHandler.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import java.util.concurrent.ConcurrentMap; + +import com.google.common.base.Charsets; +import com.google.common.collect.Maps; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; + +/** + * RPC Handler which performs SASL authentication before delegating to a child RPC handler. + * The delegate will only receive messages if the given connection has been successfully + * authenticated. A connection may be authenticated at most once. + * + * Note that the authentication process consists of multiple challenge-response pairs, each of + * which are individual RPCs. + */ +public class SaslRpcHandler extends RpcHandler { + private final Logger logger = LoggerFactory.getLogger(SaslRpcHandler.class); + + /** RpcHandler we will delegate to for authenticated connections. */ + private final RpcHandler delegate; + + /** Class which provides secret keys which are shared by server and client on a per-app basis. */ + private final SecretKeyHolder secretKeyHolder; + + /** Maps each channel to its SASL authentication state. */ + private final ConcurrentMap channelAuthenticationMap; + + public SaslRpcHandler(RpcHandler delegate, SecretKeyHolder secretKeyHolder) { + this.delegate = delegate; + this.secretKeyHolder = secretKeyHolder; + this.channelAuthenticationMap = Maps.newConcurrentMap(); + } + + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + SparkSaslServer saslServer = channelAuthenticationMap.get(client); + if (saslServer != null && saslServer.isComplete()) { + // Authentication complete, delegate to base handler. + delegate.receive(client, message, callback); + return; + } + + SaslMessage saslMessage = SaslMessage.decode(Unpooled.wrappedBuffer(message)); + + if (saslServer == null) { + // First message in the handshake, setup the necessary state. + saslServer = new SparkSaslServer(saslMessage.appId, secretKeyHolder); + channelAuthenticationMap.put(client, saslServer); + } + + byte[] response = saslServer.response(saslMessage.payload); + if (saslServer.isComplete()) { + logger.debug("SASL authentication successful for channel {}", client); + } + callback.onSuccess(response); + } + + @Override + public StreamManager getStreamManager() { + return delegate.getStreamManager(); + } + + @Override + public void connectionTerminated(TransportClient client) { + SparkSaslServer saslServer = channelAuthenticationMap.remove(client); + if (saslServer != null) { + saslServer.dispose(); + } + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java new file mode 100644 index 0000000000000..81d5766794688 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SecretKeyHolder.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +/** + * Interface for getting a secret key associated with some application. + */ +public interface SecretKeyHolder { + /** + * Gets an appropriate SASL User for the given appId. + * @throws IllegalArgumentException if the given appId is not associated with a SASL user. + */ + String getSaslUser(String appId); + + /** + * Gets an appropriate SASL secret key for the given appId. + * @throws IllegalArgumentException if the given appId is not associated with a SASL secret key. + */ + String getSecretKey(String appId); +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java new file mode 100644 index 0000000000000..72ba737b998bc --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java @@ -0,0 +1,138 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.RealmChoiceCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslClient; +import javax.security.sasl.SaslException; +import java.io.IOException; + +import com.google.common.base.Throwables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.spark.network.sasl.SparkSaslServer.*; + +/** + * A SASL Client for Spark which simply keeps track of the state of a single SASL session, from the + * initial state to the "authenticated" state. This client initializes the protocol via a + * firstToken, which is then followed by a set of challenges and responses. + */ +public class SparkSaslClient { + private final Logger logger = LoggerFactory.getLogger(SparkSaslClient.class); + + private final String secretKeyId; + private final SecretKeyHolder secretKeyHolder; + private SaslClient saslClient; + + public SparkSaslClient(String secretKeyId, SecretKeyHolder secretKeyHolder) { + this.secretKeyId = secretKeyId; + this.secretKeyHolder = secretKeyHolder; + try { + this.saslClient = Sasl.createSaslClient(new String[] { DIGEST }, null, null, DEFAULT_REALM, + SASL_PROPS, new ClientCallbackHandler()); + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** Used to initiate SASL handshake with server. */ + public synchronized byte[] firstToken() { + if (saslClient != null && saslClient.hasInitialResponse()) { + try { + return saslClient.evaluateChallenge(new byte[0]); + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } else { + return new byte[0]; + } + } + + /** Determines whether the authentication exchange has completed. */ + public synchronized boolean isComplete() { + return saslClient != null && saslClient.isComplete(); + } + + /** + * Respond to server's SASL token. + * @param token contains server's SASL token + * @return client's response SASL token + */ + public synchronized byte[] response(byte[] token) { + try { + return saslClient != null ? saslClient.evaluateChallenge(token) : new byte[0]; + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** + * Disposes of any system resources or security-sensitive information the + * SaslClient might be using. + */ + public synchronized void dispose() { + if (saslClient != null) { + try { + saslClient.dispose(); + } catch (SaslException e) { + // ignore + } finally { + saslClient = null; + } + } + } + + /** + * Implementation of javax.security.auth.callback.CallbackHandler + * that works with share secrets. + */ + private class ClientCallbackHandler implements CallbackHandler { + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + logger.trace("SASL client callback: setting username"); + NameCallback nc = (NameCallback) callback; + nc.setName(encodeIdentifier(secretKeyHolder.getSaslUser(secretKeyId))); + } else if (callback instanceof PasswordCallback) { + logger.trace("SASL client callback: setting password"); + PasswordCallback pc = (PasswordCallback) callback; + pc.setPassword(encodePassword(secretKeyHolder.getSecretKey(secretKeyId))); + } else if (callback instanceof RealmCallback) { + logger.trace("SASL client callback: setting realm"); + RealmCallback rc = (RealmCallback) callback; + rc.setText(rc.getDefaultText()); + logger.info("Realm callback"); + } else if (callback instanceof RealmChoiceCallback) { + // ignore (?) + } else { + throw new UnsupportedCallbackException(callback, "Unrecognized SASL DIGEST-MD5 Callback"); + } + } + } + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java new file mode 100644 index 0000000000000..2c0ce40c75e80 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import java.io.IOException; +import java.util.Map; + +import com.google.common.base.Charsets; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.BaseEncoding; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A SASL Server for Spark which simply keeps track of the state of a single SASL session, from the + * initial state to the "authenticated" state. (It is not a server in the sense of accepting + * connections on some socket.) + */ +public class SparkSaslServer { + private final Logger logger = LoggerFactory.getLogger(SparkSaslServer.class); + + /** + * This is passed as the server name when creating the sasl client/server. + * This could be changed to be configurable in the future. + */ + static final String DEFAULT_REALM = "default"; + + /** + * The authentication mechanism used here is DIGEST-MD5. This could be changed to be + * configurable in the future. + */ + static final String DIGEST = "DIGEST-MD5"; + + /** + * The quality of protection is just "auth". This means that we are doing + * authentication only, we are not supporting integrity or privacy protection of the + * communication channel after authentication. This could be changed to be configurable + * in the future. + */ + static final Map SASL_PROPS = ImmutableMap.builder() + .put(Sasl.QOP, "auth") + .put(Sasl.SERVER_AUTH, "true") + .build(); + + /** Identifier for a certain secret key within the secretKeyHolder. */ + private final String secretKeyId; + private final SecretKeyHolder secretKeyHolder; + private SaslServer saslServer; + + public SparkSaslServer(String secretKeyId, SecretKeyHolder secretKeyHolder) { + this.secretKeyId = secretKeyId; + this.secretKeyHolder = secretKeyHolder; + try { + this.saslServer = Sasl.createSaslServer(DIGEST, null, DEFAULT_REALM, SASL_PROPS, + new DigestCallbackHandler()); + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** + * Determines whether the authentication exchange has completed successfully. + */ + public synchronized boolean isComplete() { + return saslServer != null && saslServer.isComplete(); + } + + /** + * Used to respond to server SASL tokens. + * @param token Server's SASL token + * @return response to send back to the server. + */ + public synchronized byte[] response(byte[] token) { + try { + return saslServer != null ? saslServer.evaluateResponse(token) : new byte[0]; + } catch (SaslException e) { + throw Throwables.propagate(e); + } + } + + /** + * Disposes of any system resources or security-sensitive information the + * SaslServer might be using. + */ + public synchronized void dispose() { + if (saslServer != null) { + try { + saslServer.dispose(); + } catch (SaslException e) { + // ignore + } finally { + saslServer = null; + } + } + } + + /** + * Implementation of javax.security.auth.callback.CallbackHandler for SASL DIGEST-MD5 mechanism. + */ + private class DigestCallbackHandler implements CallbackHandler { + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + logger.trace("SASL server callback: setting username"); + NameCallback nc = (NameCallback) callback; + nc.setName(encodeIdentifier(secretKeyHolder.getSaslUser(secretKeyId))); + } else if (callback instanceof PasswordCallback) { + logger.trace("SASL server callback: setting password"); + PasswordCallback pc = (PasswordCallback) callback; + pc.setPassword(encodePassword(secretKeyHolder.getSecretKey(secretKeyId))); + } else if (callback instanceof RealmCallback) { + logger.trace("SASL server callback: setting realm"); + RealmCallback rc = (RealmCallback) callback; + rc.setText(rc.getDefaultText()); + } else if (callback instanceof AuthorizeCallback) { + AuthorizeCallback ac = (AuthorizeCallback) callback; + String authId = ac.getAuthenticationID(); + String authzId = ac.getAuthorizationID(); + ac.setAuthorized(authId.equals(authzId)); + if (ac.isAuthorized()) { + ac.setAuthorizedID(authzId); + } + logger.debug("SASL Authorization complete, authorized set to {}", ac.isAuthorized()); + } else { + throw new UnsupportedCallbackException(callback, "Unrecognized SASL DIGEST-MD5 Callback"); + } + } + } + } + + /* Encode a byte[] identifier as a Base64-encoded string. */ + public static String encodeIdentifier(String identifier) { + Preconditions.checkNotNull(identifier, "User cannot be null if SASL is enabled"); + return BaseEncoding.base64().encode(identifier.getBytes(Charsets.UTF_8)); + } + + /** Encode a password as a base64-encoded char[] array. */ + public static char[] encodePassword(String password) { + Preconditions.checkNotNull(password, "Password cannot be null if SASL is enabled"); + return BaseEncoding.base64().encode(password.getBytes(Charsets.UTF_8)).toCharArray(); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index a9dff31decc83..cd3fea85b19a4 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -41,7 +41,7 @@ * with the "one-for-one" strategy, meaning each Transport-layer Chunk is equivalent to one Spark- * level shuffle block. */ -public class ExternalShuffleBlockHandler implements RpcHandler { +public class ExternalShuffleBlockHandler extends RpcHandler { private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); private final ExternalShuffleBlockManager blockManager; diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index 6bbabc44b958b..b0b19ba67bddc 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -17,8 +17,6 @@ package org.apache.spark.network.shuffle; -import java.io.Closeable; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,15 +34,20 @@ * BlockTransferService), which has the downside of losing the shuffle data if we lose the * executors. */ -public class ExternalShuffleClient implements ShuffleClient { +public class ExternalShuffleClient extends ShuffleClient { private final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class); private final TransportClientFactory clientFactory; - private final String appId; - public ExternalShuffleClient(TransportConf conf, String appId) { + private String appId; + + public ExternalShuffleClient(TransportConf conf) { TransportContext context = new TransportContext(conf, new NoOpRpcHandler()); this.clientFactory = context.createClientFactory(); + } + + @Override + public void init(String appId) { this.appId = appId; } @@ -55,6 +58,7 @@ public void fetchBlocks( String execId, String[] blockIds, BlockFetchingListener listener) { + assert appId != null : "Called before init()"; logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId); try { TransportClient client = clientFactory.createClient(host, port); @@ -82,6 +86,7 @@ public void registerWithShuffleServer( int port, String execId, ExecutorShuffleInfo executorInfo) { + assert appId != null : "Called before init()"; TransportClient client = clientFactory.createClient(host, port); byte[] registerExecutorMessage = JavaUtils.serialize(new RegisterExecutor(appId, execId, executorInfo)); diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java index d46a562394557..f72ab40690d0d 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -20,7 +20,14 @@ import java.io.Closeable; /** Provides an interface for reading shuffle files, either from an Executor or external service. */ -public interface ShuffleClient extends Closeable { +public abstract class ShuffleClient implements Closeable { + + /** + * Initializes the ShuffleClient, specifying this Executor's appId. + * Must be called before any other method on the ShuffleClient. + */ + public void init(String appId) { } + /** * Fetch a sequence of blocks from a remote node asynchronously, * @@ -28,7 +35,7 @@ public interface ShuffleClient extends Closeable { * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as * the data of a block is fetched, rather than waiting for all blocks to be fetched. */ - public void fetchBlocks( + public abstract void fetchBlocks( String host, int port, String execId, diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java new file mode 100644 index 0000000000000..84781207861ed --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -0,0 +1,172 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import java.io.IOException; + +import com.google.common.collect.Lists; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; +import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class SaslIntegrationSuite { + static ExternalShuffleBlockHandler handler; + static TransportServer server; + static TransportConf conf; + static TransportContext context; + + TransportClientFactory clientFactory; + + /** Provides a secret key holder which always returns the given secret key. */ + static class TestSecretKeyHolder implements SecretKeyHolder { + + private final String secretKey; + + TestSecretKeyHolder(String secretKey) { + this.secretKey = secretKey; + } + + @Override + public String getSaslUser(String appId) { + return "user"; + } + @Override + public String getSecretKey(String appId) { + return secretKey; + } + } + + + @BeforeClass + public static void beforeAll() throws IOException { + SecretKeyHolder secretKeyHolder = new TestSecretKeyHolder("good-key"); + SaslRpcHandler handler = new SaslRpcHandler(new TestRpcHandler(), secretKeyHolder); + conf = new TransportConf(new SystemPropertyConfigProvider()); + context = new TransportContext(conf, handler); + server = context.createServer(); + } + + + @AfterClass + public static void afterAll() { + server.close(); + } + + @After + public void afterEach() { + if (clientFactory != null) { + clientFactory.close(); + clientFactory = null; + } + } + + @Test + public void testGoodClient() { + clientFactory = context.createClientFactory( + Lists.newArrayList( + new SaslClientBootstrap(conf, "app-id", new TestSecretKeyHolder("good-key")))); + + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + String msg = "Hello, World!"; + byte[] resp = client.sendRpcSync(msg.getBytes(), 1000); + assertEquals(msg, new String(resp)); // our rpc handler should just return the given msg + } + + @Test + public void testBadClient() { + clientFactory = context.createClientFactory( + Lists.newArrayList( + new SaslClientBootstrap(conf, "app-id", new TestSecretKeyHolder("bad-key")))); + + try { + // Bootstrap should fail on startup. + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("Mismatched response")); + } + } + + @Test + public void testNoSaslClient() { + clientFactory = context.createClientFactory( + Lists.newArrayList()); + + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + try { + client.sendRpcSync(new byte[13], 1000); + fail("Should have failed"); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("Expected SaslMessage")); + } + + try { + // Guessing the right tag byte doesn't magically get you in... + client.sendRpcSync(new byte[] { (byte) 0xEA }, 1000); + fail("Should have failed"); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("java.lang.IndexOutOfBoundsException")); + } + } + + @Test + public void testNoSaslServer() { + RpcHandler handler = new TestRpcHandler(); + TransportContext context = new TransportContext(conf, handler); + clientFactory = context.createClientFactory( + Lists.newArrayList( + new SaslClientBootstrap(conf, "app-id", new TestSecretKeyHolder("key")))); + TransportServer server = context.createServer(); + try { + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); + } finally { + server.close(); + } + } + + /** RPC handler which simply responds with the message it received. */ + public static class TestRpcHandler extends RpcHandler { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + callback.onSuccess(message); + } + + @Override + public StreamManager getStreamManager() { + return new OneForOneStreamManager(); + } + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java new file mode 100644 index 0000000000000..67a07f38eb5a0 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import java.util.Map; + +import com.google.common.collect.ImmutableMap; +import org.junit.Test; + +import static org.junit.Assert.*; + +/** + * Jointly tests SparkSaslClient and SparkSaslServer, as both are black boxes. + */ +public class SparkSaslSuite { + + /** Provides a secret key holder which returns secret key == appId */ + private SecretKeyHolder secretKeyHolder = new SecretKeyHolder() { + @Override + public String getSaslUser(String appId) { + return "user"; + } + + @Override + public String getSecretKey(String appId) { + return appId; + } + }; + + @Test + public void testMatching() { + SparkSaslClient client = new SparkSaslClient("shared-secret", secretKeyHolder); + SparkSaslServer server = new SparkSaslServer("shared-secret", secretKeyHolder); + + assertFalse(client.isComplete()); + assertFalse(server.isComplete()); + + byte[] clientMessage = client.firstToken(); + + while (!client.isComplete()) { + clientMessage = client.response(server.response(clientMessage)); + } + assertTrue(server.isComplete()); + + // Disposal should invalidate + server.dispose(); + assertFalse(server.isComplete()); + client.dispose(); + assertFalse(client.isComplete()); + } + + + @Test + public void testNonMatching() { + SparkSaslClient client = new SparkSaslClient("my-secret", secretKeyHolder); + SparkSaslServer server = new SparkSaslServer("your-secret", secretKeyHolder); + + assertFalse(client.isComplete()); + assertFalse(server.isComplete()); + + byte[] clientMessage = client.firstToken(); + + try { + while (!client.isComplete()) { + clientMessage = client.response(server.response(clientMessage)); + } + fail("Should not have completed"); + } catch (Exception e) { + assertTrue(e.getMessage().contains("Mismatched response")); + assertFalse(client.isComplete()); + assertFalse(server.isComplete()); + } + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index b3bcf5fd68e73..bc101f53844d5 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -135,7 +135,8 @@ private FetchResult fetchBlocks(String execId, String[] blockIds, int port) thro final Semaphore requestsRemaining = new Semaphore(0); - ExternalShuffleClient client = new ExternalShuffleClient(conf, APP_ID); + ExternalShuffleClient client = new ExternalShuffleClient(conf); + client.init(APP_ID); client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, new BlockFetchingListener() { @Override @@ -164,6 +165,7 @@ public void onBlockFetchFailure(String blockId, Throwable exception) { if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { fail("Timeout getting response from the server"); } + client.close(); return res; } @@ -265,7 +267,8 @@ public void testFetchNoServer() throws Exception { } private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) { - ExternalShuffleClient client = new ExternalShuffleClient(conf, APP_ID); + ExternalShuffleClient client = new ExternalShuffleClient(conf); + client.init(APP_ID); client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), executorId, executorInfo); } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index ad1a6f01b3a57..0f27f55fec4f3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -74,6 +74,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche blockManager = new BlockManager("bm", actorSystem, blockManagerMaster, serializer, blockManagerSize, conf, mapOutputTracker, shuffleManager, new NioBlockTransferService(conf, securityMgr)) + blockManager.initialize("app-id") tempDirectory = Files.createTempDir() manualClock.setTime(0) From 515abb9afa2d6b58947af6bb079a493b49d315ca Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 4 Nov 2014 18:14:28 -0800 Subject: [PATCH 1221/1492] [SQL] Add String option for DSL AS Author: Michael Armbrust Closes #3097 from marmbrus/asString and squashes the following commits: 6430520 [Michael Armbrust] Add String option for DSL AS --- .../main/scala/org/apache/spark/sql/catalyst/dsl/package.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 3314e15477016..31dc5a58e68e5 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -110,7 +110,8 @@ package object dsl { def asc = SortOrder(expr, Ascending) def desc = SortOrder(expr, Descending) - def as(s: Symbol) = Alias(expr, s.name)() + def as(alias: String) = Alias(expr, alias)() + def as(alias: Symbol) = Alias(expr, alias.name)() } trait ExpressionConversions { From c8abddc5164d8cf11cdede6ab3d5d1ea08028708 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 4 Nov 2014 21:35:52 -0800 Subject: [PATCH 1222/1492] [SPARK-3964] [MLlib] [PySpark] add Hypothesis test Python API ``` pyspark.mllib.stat.StatisticschiSqTest(observed, expected=None) :: Experimental :: If `observed` is Vector, conduct Pearson's chi-squared goodness of fit test of the observed data against the expected distribution, or againt the uniform distribution (by default), with each category having an expected frequency of `1 / len(observed)`. (Note: `observed` cannot contain negative values) If `observed` is matrix, conduct Pearson's independence test on the input contingency matrix, which cannot contain negative entries or columns or rows that sum up to 0. If `observed` is an RDD of LabeledPoint, conduct Pearson's independence test for every feature against the label across the input RDD. For each feature, the (feature, label) pairs are converted into a contingency matrix for which the chi-squared statistic is computed. All label and feature values must be categorical. :param observed: it could be a vector containing the observed categorical counts/relative frequencies, or the contingency matrix (containing either counts or relative frequencies), or an RDD of LabeledPoint containing the labeled dataset with categorical features. Real-valued features will be treated as categorical for each distinct value. :param expected: Vector containing the expected categorical counts/relative frequencies. `expected` is rescaled if the `expected` sum differs from the `observed` sum. :return: ChiSquaredTest object containing the test statistic, degrees of freedom, p-value, the method used, and the null hypothesis. ``` Author: Davies Liu Closes #3091 from davies/his and squashes the following commits: 145d16c [Davies Liu] address comments 0ab0764 [Davies Liu] fix float 5097d54 [Davies Liu] add Hypothesis test Python API --- docs/mllib-statistics.md | 40 +++++ .../mllib/api/python/PythonMLLibAPI.scala | 26 ++++ python/pyspark/mllib/common.py | 7 +- python/pyspark/mllib/linalg.py | 13 +- python/pyspark/mllib/stat.py | 137 +++++++++++++++++- 5 files changed, 219 insertions(+), 4 deletions(-) diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index 10a5131c07414..ca8c29218f52d 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -380,6 +380,46 @@ for (ChiSqTestResult result : featureTestResults) { {% endhighlight %} +
    +[`Statistics`](api/python/index.html#pyspark.mllib.stat.Statistics$) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight python %} +from pyspark import SparkContext +from pyspark.mllib.linalg import Vectors, Matrices +from pyspark.mllib.regresssion import LabeledPoint +from pyspark.mllib.stat import Statistics + +sc = SparkContext() + +vec = Vectors.dense(...) # a vector composed of the frequencies of events + +# compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +# the test runs against a uniform distribution. +goodnessOfFitTestResult = Statistics.chiSqTest(vec) +print goodnessOfFitTestResult # summary of the test including the p-value, degrees of freedom, + # test statistic, the method used, and the null hypothesis. + +mat = Matrices.dense(...) # a contingency matrix + +# conduct Pearson's independence test on the input contingency matrix +independenceTestResult = Statistics.chiSqTest(mat) +print independenceTestResult # summary of the test including the p-value, degrees of freedom... + +obs = sc.parallelize(...) # LabeledPoint(feature, label) . + +# The contingency table is constructed from an RDD of LabeledPoint and used to conduct +# the independence test. Returns an array containing the ChiSquaredTestResult for every feature +# against the label. +featureTestResults = Statistics.chiSqTest(obs) + +for i, result in enumerate(featureTestResults): + print "Column $d:" % (i + 1) + print result +{% endhighlight %} +
    + ## Random data generation diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 65b98a8ceea55..d832ae34b55e4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -43,6 +43,7 @@ import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model.DecisionTreeModel import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames +import org.apache.spark.mllib.stat.test.ChiSqTestResult import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -454,6 +455,31 @@ class PythonMLLibAPI extends Serializable { Statistics.corr(x.rdd, y.rdd, getCorrNameOrDefault(method)) } + /** + * Java stub for mllib Statistics.chiSqTest() + */ + def chiSqTest(observed: Vector, expected: Vector): ChiSqTestResult = { + if (expected == null) { + Statistics.chiSqTest(observed) + } else { + Statistics.chiSqTest(observed, expected) + } + } + + /** + * Java stub for mllib Statistics.chiSqTest(observed: Matrix) + */ + def chiSqTest(observed: Matrix): ChiSqTestResult = { + Statistics.chiSqTest(observed) + } + + /** + * Java stub for mllib Statistics.chiSqTest(RDD[LabelPoint]) + */ + def chiSqTest(data: JavaRDD[LabeledPoint]): Array[ChiSqTestResult] = { + Statistics.chiSqTest(data.rdd) + } + // used by the corr methods to retrieve the name of the correlation method passed in via pyspark private def getCorrNameOrDefault(method: String) = { if (method == null) CorrelationNames.defaultCorrName else method diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index dbe5f698b7345..c6149fe391ec8 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -98,8 +98,13 @@ def _java2py(sc, r): jrdd = sc._jvm.SerDe.javaToPython(r) return RDD(jrdd, sc) - elif isinstance(r, (JavaArray, JavaList)) or clsName in _picklable_classes: + if clsName in _picklable_classes: r = sc._jvm.SerDe.dumps(r) + elif isinstance(r, (JavaArray, JavaList)): + try: + r = sc._jvm.SerDe.dumps(r) + except Py4JJavaError: + pass # not pickable if isinstance(r, bytearray): r = PickleSerializer().loads(str(r)) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index c0c3dff31e7f8..e35202dca0acc 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -33,7 +33,7 @@ IntegerType, ByteType, Row -__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', 'DenseMatrix', 'Matrices'] if sys.version_info[:2] == (2, 7): @@ -578,6 +578,8 @@ class DenseMatrix(Matrix): def __init__(self, numRows, numCols, values): Matrix.__init__(self, numRows, numCols) assert len(values) == numRows * numCols + if not isinstance(values, array.array): + values = array.array('d', values) self.values = values def __reduce__(self): @@ -596,6 +598,15 @@ def toArray(self): return np.reshape(self.values, (self.numRows, self.numCols), order='F') +class Matrices(object): + @staticmethod + def dense(numRows, numCols, values): + """ + Create a DenseMatrix + """ + return DenseMatrix(numRows, numCols, values) + + def _test(): import doctest (failure_count, test_count) = doctest.testmod(optionflags=doctest.ELLIPSIS) diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 15f0652f833d7..0700f8a8e5a8e 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -19,11 +19,12 @@ Python package for statistical functions in MLlib. """ +from pyspark import RDD from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper -from pyspark.mllib.linalg import _convert_to_vector +from pyspark.mllib.linalg import Matrix, _convert_to_vector -__all__ = ['MultivariateStatisticalSummary', 'Statistics'] +__all__ = ['MultivariateStatisticalSummary', 'ChiSqTestResult', 'Statistics'] class MultivariateStatisticalSummary(JavaModelWrapper): @@ -51,6 +52,54 @@ def min(self): return self.call("min").toArray() +class ChiSqTestResult(JavaModelWrapper): + """ + :: Experimental :: + + Object containing the test results for the chi-squared hypothesis test. + """ + @property + def method(self): + """ + Name of the test method + """ + return self._java_model.method() + + @property + def pValue(self): + """ + The probability of obtaining a test statistic result at least as + extreme as the one that was actually observed, assuming that the + null hypothesis is true. + """ + return self._java_model.pValue() + + @property + def degreesOfFreedom(self): + """ + Returns the degree(s) of freedom of the hypothesis test. + Return type should be Number(e.g. Int, Double) or tuples of Numbers. + """ + return self._java_model.degreesOfFreedom() + + @property + def statistic(self): + """ + Test statistic. + """ + return self._java_model.statistic() + + @property + def nullHypothesis(self): + """ + Null hypothesis of the test. + """ + return self._java_model.nullHypothesis() + + def __str__(self): + return self._java_model.toString() + + class Statistics(object): @staticmethod @@ -135,6 +184,90 @@ def corr(x, y=None, method=None): else: return callMLlibFunc("corr", x.map(float), y.map(float), method) + @staticmethod + def chiSqTest(observed, expected=None): + """ + :: Experimental :: + + If `observed` is Vector, conduct Pearson's chi-squared goodness + of fit test of the observed data against the expected distribution, + or againt the uniform distribution (by default), with each category + having an expected frequency of `1 / len(observed)`. + (Note: `observed` cannot contain negative values) + + If `observed` is matrix, conduct Pearson's independence test on the + input contingency matrix, which cannot contain negative entries or + columns or rows that sum up to 0. + + If `observed` is an RDD of LabeledPoint, conduct Pearson's independence + test for every feature against the label across the input RDD. + For each feature, the (feature, label) pairs are converted into a + contingency matrix for which the chi-squared statistic is computed. + All label and feature values must be categorical. + + :param observed: it could be a vector containing the observed categorical + counts/relative frequencies, or the contingency matrix + (containing either counts or relative frequencies), + or an RDD of LabeledPoint containing the labeled dataset + with categorical features. Real-valued features will be + treated as categorical for each distinct value. + :param expected: Vector containing the expected categorical counts/relative + frequencies. `expected` is rescaled if the `expected` sum + differs from the `observed` sum. + :return: ChiSquaredTest object containing the test statistic, degrees + of freedom, p-value, the method used, and the null hypothesis. + + >>> from pyspark.mllib.linalg import Vectors, Matrices + >>> observed = Vectors.dense([4, 6, 5]) + >>> pearson = Statistics.chiSqTest(observed) + >>> print pearson.statistic + 0.4 + >>> pearson.degreesOfFreedom + 2 + >>> print round(pearson.pValue, 4) + 0.8187 + >>> pearson.method + u'pearson' + >>> pearson.nullHypothesis + u'observed follows the same distribution as expected.' + + >>> observed = Vectors.dense([21, 38, 43, 80]) + >>> expected = Vectors.dense([3, 5, 7, 20]) + >>> pearson = Statistics.chiSqTest(observed, expected) + >>> print round(pearson.pValue, 4) + 0.0027 + + >>> data = [40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0] + >>> chi = Statistics.chiSqTest(Matrices.dense(3, 4, data)) + >>> print round(chi.statistic, 4) + 21.9958 + + >>> from pyspark.mllib.regression import LabeledPoint + >>> data = [LabeledPoint(0.0, Vectors.dense([0.5, 10.0])), + ... LabeledPoint(0.0, Vectors.dense([1.5, 20.0])), + ... LabeledPoint(1.0, Vectors.dense([1.5, 30.0])), + ... LabeledPoint(0.0, Vectors.dense([3.5, 30.0])), + ... LabeledPoint(0.0, Vectors.dense([3.5, 40.0])), + ... LabeledPoint(1.0, Vectors.dense([3.5, 40.0])),] + >>> rdd = sc.parallelize(data, 4) + >>> chi = Statistics.chiSqTest(rdd) + >>> print chi[0].statistic + 0.75 + >>> print chi[1].statistic + 1.5 + """ + if isinstance(observed, RDD): + jmodels = callMLlibFunc("chiSqTest", observed) + return [ChiSqTestResult(m) for m in jmodels] + + if isinstance(observed, Matrix): + jmodel = callMLlibFunc("chiSqTest", observed) + else: + if expected and len(expected) != len(observed): + raise ValueError("`expected` should have same length with `observed`") + jmodel = callMLlibFunc("chiSqTest", _convert_to_vector(observed), expected) + return ChiSqTestResult(jmodel) + def _test(): import doctest From 5f13759d3642ea5b58c12a756e7125ac19aff10e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 5 Nov 2014 01:21:53 -0800 Subject: [PATCH 1223/1492] [SPARK-4029][Streaming] Update streaming driver to reliably save and recover received block metadata on driver failures As part of the initiative of preventing data loss on driver failure, this JIRA tracks the sub task of modifying the streaming driver to reliably save received block metadata, and recover them on driver restart. This was solved by introducing a `ReceivedBlockTracker` that takes all the responsibility of managing the metadata of received blocks (i.e. `ReceivedBlockInfo`, and any actions on them (e.g, allocating blocks to batches, etc.). All actions to block info get written out to a write ahead log (using `WriteAheadLogManager`). On recovery, all the actions are replaying to recreate the pre-failure state of the `ReceivedBlockTracker`, which include the batch-to-block allocations and the unallocated blocks. Furthermore, the `ReceiverInputDStream` was modified to create `WriteAheadLogBackedBlockRDD`s when file segment info is present in the `ReceivedBlockInfo`. After recovery of all the block info (through recovery `ReceivedBlockTracker`), the `WriteAheadLogBackedBlockRDD`s gets recreated with the recovered info, and jobs submitted. The data of the blocks gets pulled from the write ahead logs, thanks to the segment info present in the `ReceivedBlockInfo`. This is still a WIP. Things that are missing here are. - *End-to-end integration tests:* Unit tests that tests the driver recovery, by killing and restarting the streaming context, and verifying all the input data gets processed. This has been implemented but not included in this PR yet. A sneak peek of that DriverFailureSuite can be found in this PR (on my personal repo): https://github.com/tdas/spark/pull/25 I can either include it in this PR, or submit that as a separate PR after this gets in. - *WAL cleanup:* Cleaning up the received data write ahead log, by calling `ReceivedBlockHandler.cleanupOldBlocks`. This is being worked on. Author: Tathagata Das Closes #3026 from tdas/driver-ha-rbt and squashes the following commits: a8009ed [Tathagata Das] Added comment 1d704bb [Tathagata Das] Enabled storing recovered WAL-backed blocks to BM 2ee2484 [Tathagata Das] More minor changes based on PR 47fc1e3 [Tathagata Das] Addressed PR comments. 9a7e3e4 [Tathagata Das] Refactored ReceivedBlockTracker API a bit to make things a little cleaner for users of the tracker. af63655 [Tathagata Das] Minor changes. fce2b21 [Tathagata Das] Removed commented lines 59496d3 [Tathagata Das] Changed class names, made allocation more explicit and added cleanup 19aec7d [Tathagata Das] Fixed casting bug. f66d277 [Tathagata Das] Fix line lengths. cda62ee [Tathagata Das] Added license 25611d6 [Tathagata Das] Minor changes before submitting PR 7ae0a7fb [Tathagata Das] Transferred changes from driver-ha-working branch --- .../dstream/ReceiverInputDStream.scala | 69 +++-- .../rdd/WriteAheadLogBackedBlockRDD.scala | 3 +- .../streaming/scheduler/JobGenerator.scala | 21 +- .../scheduler/ReceivedBlockTracker.scala | 230 +++++++++++++++++ .../streaming/scheduler/ReceiverTracker.scala | 98 ++++--- .../streaming/BasicOperationsSuite.scala | 19 +- .../streaming/ReceivedBlockTrackerSuite.scala | 242 ++++++++++++++++++ .../WriteAheadLogBackedBlockRDDSuite.scala | 4 +- 8 files changed, 597 insertions(+), 89 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index bb47d373de63d..3e67161363e50 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -17,15 +17,14 @@ package org.apache.spark.streaming.dstream -import scala.collection.mutable.HashMap import scala.reflect.ClassTag import org.apache.spark.rdd.{BlockRDD, RDD} -import org.apache.spark.storage.BlockId +import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.streaming._ -import org.apache.spark.streaming.receiver.{WriteAheadLogBasedStoreResult, BlockManagerBasedStoreResult, Receiver} +import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD +import org.apache.spark.streaming.receiver.{Receiver, WriteAheadLogBasedStoreResult} import org.apache.spark.streaming.scheduler.ReceivedBlockInfo -import org.apache.spark.SparkException /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -40,9 +39,6 @@ import org.apache.spark.SparkException abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - /** Keeps all received blocks information */ - private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] - /** This is an unique identifier for the network input stream. */ val id = ssc.getNewReceiverStreamId() @@ -58,24 +54,45 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont def stop() {} - /** Ask ReceiverInputTracker for received data blocks and generates RDDs with them. */ + /** + * Generates RDDs with blocks received by the receiver of this stream. */ override def compute(validTime: Time): Option[RDD[T]] = { - // If this is called for any time before the start time of the context, - // then this returns an empty RDD. This may happen when recovering from a - // master failure - if (validTime >= graph.startTime) { - val blockInfo = ssc.scheduler.receiverTracker.getReceivedBlockInfo(id) - receivedBlockInfo(validTime) = blockInfo - val blockIds = blockInfo.map { _.blockStoreResult.blockId.asInstanceOf[BlockId] } - Some(new BlockRDD[T](ssc.sc, blockIds)) - } else { - Some(new BlockRDD[T](ssc.sc, Array.empty)) - } - } + val blockRDD = { - /** Get information on received blocks. */ - private[streaming] def getReceivedBlockInfo(time: Time) = { - receivedBlockInfo.get(time).getOrElse(Array.empty[ReceivedBlockInfo]) + if (validTime < graph.startTime) { + // If this is called for any time before the start time of the context, + // then this returns an empty RDD. This may happen when recovering from a + // driver failure without any write ahead log to recover pre-failure data. + new BlockRDD[T](ssc.sc, Array.empty) + } else { + // Otherwise, ask the tracker for all the blocks that have been allocated to this stream + // for this batch + val blockInfos = + ssc.scheduler.receiverTracker.getBlocksOfBatch(validTime).get(id).getOrElse(Seq.empty) + val blockStoreResults = blockInfos.map { _.blockStoreResult } + val blockIds = blockStoreResults.map { _.blockId.asInstanceOf[BlockId] }.toArray + + // Check whether all the results are of the same type + val resultTypes = blockStoreResults.map { _.getClass }.distinct + if (resultTypes.size > 1) { + logWarning("Multiple result types in block information, WAL information will be ignored.") + } + + // If all the results are of type WriteAheadLogBasedStoreResult, then create + // WriteAheadLogBackedBlockRDD else create simple BlockRDD. + if (resultTypes.size == 1 && resultTypes.head == classOf[WriteAheadLogBasedStoreResult]) { + val logSegments = blockStoreResults.map { + _.asInstanceOf[WriteAheadLogBasedStoreResult].segment + }.toArray + // Since storeInBlockManager = false, the storage level does not matter. + new WriteAheadLogBackedBlockRDD[T](ssc.sparkContext, + blockIds, logSegments, storeInBlockManager = true, StorageLevel.MEMORY_ONLY_SER) + } else { + new BlockRDD[T](ssc.sc, blockIds) + } + } + } + Some(blockRDD) } /** @@ -86,10 +103,6 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont */ private[streaming] override def clearMetadata(time: Time) { super.clearMetadata(time) - val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) - receivedBlockInfo --= oldReceivedBlocks.keys - logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + - (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + ssc.scheduler.receiverTracker.cleanupOldMetadata(time - rememberDuration) } } - diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 23295bf658712..dd1e96334952f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -48,7 +48,6 @@ class WriteAheadLogBackedBlockRDDPartition( * If it does not find them, it looks up the corresponding file segment. * * @param sc SparkContext - * @param hadoopConfig Hadoop configuration * @param blockIds Ids of the blocks that contains this RDD's data * @param segments Segments in write ahead logs that contain this RDD's data * @param storeInBlockManager Whether to store in the block manager after reading from the segment @@ -58,7 +57,6 @@ class WriteAheadLogBackedBlockRDDPartition( private[streaming] class WriteAheadLogBackedBlockRDD[T: ClassTag]( @transient sc: SparkContext, - @transient hadoopConfig: Configuration, @transient blockIds: Array[BlockId], @transient segments: Array[WriteAheadLogFileSegment], storeInBlockManager: Boolean, @@ -71,6 +69,7 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( s"the same as number of segments (${segments.length}})!") // Hadoop configuration is not serializable, so broadcast it as a serializable. + @transient private val hadoopConfig = sc.hadoopConfiguration private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig) override def getPartitions: Array[Partition] = { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 7d73ada12d107..39b66e1130768 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -112,7 +112,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // Wait until all the received blocks in the network input tracker has // been consumed by network input DStreams, and jobs have been generated with them logInfo("Waiting for all received blocks to be consumed for job generation") - while(!hasTimedOut && jobScheduler.receiverTracker.hasMoreReceivedBlockIds) { + while(!hasTimedOut && jobScheduler.receiverTracker.hasUnallocatedBlocks) { Thread.sleep(pollTime) } logInfo("Waited for all received blocks to be consumed for job generation") @@ -217,14 +217,18 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** Generate jobs and perform checkpoint for the given `time`. */ private def generateJobs(time: Time) { - Try(graph.generateJobs(time)) match { + // Set the SparkEnv in this thread, so that job generation code can access the environment + // Example: BlockRDDs are created in this thread, and it needs to access BlockManager + // Update: This is probably redundant after threadlocal stuff in SparkEnv has been removed. + SparkEnv.set(ssc.env) + Try { + jobScheduler.receiverTracker.allocateBlocksToBatch(time) // allocate received blocks to batch + graph.generateJobs(time) // generate jobs using allocated block + } match { case Success(jobs) => - val receivedBlockInfo = graph.getReceiverInputStreams.map { stream => - val streamId = stream.id - val receivedBlockInfo = stream.getReceivedBlockInfo(time) - (streamId, receivedBlockInfo) - }.toMap - jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfo)) + val receivedBlockInfos = + jobScheduler.receiverTracker.getBlocksOfBatch(time).mapValues { _.toArray } + jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfos)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } @@ -234,6 +238,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** Clear DStream metadata for the given `time`. */ private def clearMetadata(time: Time) { ssc.graph.clearMetadata(time) + jobScheduler.receiverTracker.cleanupOldMetadata(time - graph.batchDuration) // If checkpointing is enabled, then checkpoint, // else mark batch to be fully processed diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala new file mode 100644 index 0000000000000..5f5e1909908d5 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.scheduler + +import java.nio.ByteBuffer + +import scala.collection.mutable +import scala.language.implicitConversions + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.{SparkException, Logging, SparkConf} +import org.apache.spark.streaming.Time +import org.apache.spark.streaming.util.{Clock, WriteAheadLogManager} +import org.apache.spark.util.Utils + +/** Trait representing any event in the ReceivedBlockTracker that updates its state. */ +private[streaming] sealed trait ReceivedBlockTrackerLogEvent + +private[streaming] case class BlockAdditionEvent(receivedBlockInfo: ReceivedBlockInfo) + extends ReceivedBlockTrackerLogEvent +private[streaming] case class BatchAllocationEvent(time: Time, allocatedBlocks: AllocatedBlocks) + extends ReceivedBlockTrackerLogEvent +private[streaming] case class BatchCleanupEvent(times: Seq[Time]) + extends ReceivedBlockTrackerLogEvent + + +/** Class representing the blocks of all the streams allocated to a batch */ +private[streaming] +case class AllocatedBlocks(streamIdToAllocatedBlocks: Map[Int, Seq[ReceivedBlockInfo]]) { + def getBlocksOfStream(streamId: Int): Seq[ReceivedBlockInfo] = { + streamIdToAllocatedBlocks.get(streamId).getOrElse(Seq.empty) + } +} + +/** + * Class that keep track of all the received blocks, and allocate them to batches + * when required. All actions taken by this class can be saved to a write ahead log + * (if a checkpoint directory has been provided), so that the state of the tracker + * (received blocks and block-to-batch allocations) can be recovered after driver failure. + * + * Note that when any instance of this class is created with a checkpoint directory, + * it will try reading events from logs in the directory. + */ +private[streaming] class ReceivedBlockTracker( + conf: SparkConf, + hadoopConf: Configuration, + streamIds: Seq[Int], + clock: Clock, + checkpointDirOption: Option[String]) + extends Logging { + + private type ReceivedBlockQueue = mutable.Queue[ReceivedBlockInfo] + + private val streamIdToUnallocatedBlockQueues = new mutable.HashMap[Int, ReceivedBlockQueue] + private val timeToAllocatedBlocks = new mutable.HashMap[Time, AllocatedBlocks] + + private val logManagerRollingIntervalSecs = conf.getInt( + "spark.streaming.receivedBlockTracker.writeAheadLog.rotationIntervalSecs", 60) + private val logManagerOption = checkpointDirOption.map { checkpointDir => + new WriteAheadLogManager( + ReceivedBlockTracker.checkpointDirToLogDir(checkpointDir), + hadoopConf, + rollingIntervalSecs = logManagerRollingIntervalSecs, + callerName = "ReceivedBlockHandlerMaster", + clock = clock + ) + } + + private var lastAllocatedBatchTime: Time = null + + // Recover block information from write ahead logs + recoverFromWriteAheadLogs() + + /** Add received block. This event will get written to the write ahead log (if enabled). */ + def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = synchronized { + try { + writeToLog(BlockAdditionEvent(receivedBlockInfo)) + getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo + logDebug(s"Stream ${receivedBlockInfo.streamId} received " + + s"block ${receivedBlockInfo.blockStoreResult.blockId}") + true + } catch { + case e: Exception => + logError(s"Error adding block $receivedBlockInfo", e) + false + } + } + + /** + * Allocate all unallocated blocks to the given batch. + * This event will get written to the write ahead log (if enabled). + */ + def allocateBlocksToBatch(batchTime: Time): Unit = synchronized { + if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) { + val streamIdToBlocks = streamIds.map { streamId => + (streamId, getReceivedBlockQueue(streamId).dequeueAll(x => true)) + }.toMap + val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) + writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks)) + timeToAllocatedBlocks(batchTime) = allocatedBlocks + lastAllocatedBatchTime = batchTime + allocatedBlocks + } else { + throw new SparkException(s"Unexpected allocation of blocks, " + + s"last batch = $lastAllocatedBatchTime, batch time to allocate = $batchTime ") + } + } + + /** Get the blocks allocated to the given batch. */ + def getBlocksOfBatch(batchTime: Time): Map[Int, Seq[ReceivedBlockInfo]] = synchronized { + timeToAllocatedBlocks.get(batchTime).map { _.streamIdToAllocatedBlocks }.getOrElse(Map.empty) + } + + /** Get the blocks allocated to the given batch and stream. */ + def getBlocksOfBatchAndStream(batchTime: Time, streamId: Int): Seq[ReceivedBlockInfo] = { + synchronized { + timeToAllocatedBlocks.get(batchTime).map { + _.getBlocksOfStream(streamId) + }.getOrElse(Seq.empty) + } + } + + /** Check if any blocks are left to be allocated to batches. */ + def hasUnallocatedReceivedBlocks: Boolean = synchronized { + !streamIdToUnallocatedBlockQueues.values.forall(_.isEmpty) + } + + /** + * Get blocks that have been added but not yet allocated to any batch. This method + * is primarily used for testing. + */ + def getUnallocatedBlocks(streamId: Int): Seq[ReceivedBlockInfo] = synchronized { + getReceivedBlockQueue(streamId).toSeq + } + + /** Clean up block information of old batches. */ + def cleanupOldBatches(cleanupThreshTime: Time): Unit = synchronized { + assert(cleanupThreshTime.milliseconds < clock.currentTime()) + val timesToCleanup = timeToAllocatedBlocks.keys.filter { _ < cleanupThreshTime }.toSeq + logInfo("Deleting batches " + timesToCleanup) + writeToLog(BatchCleanupEvent(timesToCleanup)) + timeToAllocatedBlocks --= timesToCleanup + logManagerOption.foreach(_.cleanupOldLogs(cleanupThreshTime.milliseconds)) + log + } + + /** Stop the block tracker. */ + def stop() { + logManagerOption.foreach { _.stop() } + } + + /** + * Recover all the tracker actions from the write ahead logs to recover the state (unallocated + * and allocated block info) prior to failure. + */ + private def recoverFromWriteAheadLogs(): Unit = synchronized { + // Insert the recovered block information + def insertAddedBlock(receivedBlockInfo: ReceivedBlockInfo) { + logTrace(s"Recovery: Inserting added block $receivedBlockInfo") + getReceivedBlockQueue(receivedBlockInfo.streamId) += receivedBlockInfo + } + + // Insert the recovered block-to-batch allocations and clear the queue of received blocks + // (when the blocks were originally allocated to the batch, the queue must have been cleared). + def insertAllocatedBatch(batchTime: Time, allocatedBlocks: AllocatedBlocks) { + logTrace(s"Recovery: Inserting allocated batch for time $batchTime to " + + s"${allocatedBlocks.streamIdToAllocatedBlocks}") + streamIdToUnallocatedBlockQueues.values.foreach { _.clear() } + lastAllocatedBatchTime = batchTime + timeToAllocatedBlocks.put(batchTime, allocatedBlocks) + } + + // Cleanup the batch allocations + def cleanupBatches(batchTimes: Seq[Time]) { + logTrace(s"Recovery: Cleaning up batches $batchTimes") + timeToAllocatedBlocks --= batchTimes + } + + logManagerOption.foreach { logManager => + logInfo(s"Recovering from write ahead logs in ${checkpointDirOption.get}") + logManager.readFromLog().foreach { byteBuffer => + logTrace("Recovering record " + byteBuffer) + Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) match { + case BlockAdditionEvent(receivedBlockInfo) => + insertAddedBlock(receivedBlockInfo) + case BatchAllocationEvent(time, allocatedBlocks) => + insertAllocatedBatch(time, allocatedBlocks) + case BatchCleanupEvent(batchTimes) => + cleanupBatches(batchTimes) + } + } + } + } + + /** Write an update to the tracker to the write ahead log */ + private def writeToLog(record: ReceivedBlockTrackerLogEvent) { + logDebug(s"Writing to log $record") + logManagerOption.foreach { logManager => + logManager.writeToLog(ByteBuffer.wrap(Utils.serialize(record))) + } + } + + /** Get the queue of received blocks belonging to a particular stream */ + private def getReceivedBlockQueue(streamId: Int): ReceivedBlockQueue = { + streamIdToUnallocatedBlockQueues.getOrElseUpdate(streamId, new ReceivedBlockQueue) + } +} + +private[streaming] object ReceivedBlockTracker { + def checkpointDirToLogDir(checkpointDir: String): String = { + new Path(checkpointDir, "receivedBlockMetadata").toString + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index d696563bcee83..1c3984d968d20 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -17,15 +17,16 @@ package org.apache.spark.streaming.scheduler -import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} + +import scala.collection.mutable.{HashMap, SynchronizedMap} import scala.language.existentials import akka.actor._ -import org.apache.spark.{SerializableWritable, Logging, SparkEnv, SparkException} + +import org.apache.spark.{Logging, SerializableWritable, SparkEnv, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} -import org.apache.spark.util.AkkaUtils /** * Messages used by the NetworkReceiver and the ReceiverTracker to communicate @@ -48,23 +49,28 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, err * This class manages the execution of the receivers of NetworkInputDStreams. Instance of * this class must be created after all input streams have been added and StreamingContext.start() * has been called because it needs the final set of input streams at the time of instantiation. + * + * @param skipReceiverLaunch Do not launch the receiver. This is useful for testing. */ private[streaming] -class ReceiverTracker(ssc: StreamingContext) extends Logging { +class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false) extends Logging { - val receiverInputStreams = ssc.graph.getReceiverInputStreams() - val receiverInputStreamMap = Map(receiverInputStreams.map(x => (x.id, x)): _*) - val receiverExecutor = new ReceiverLauncher() - val receiverInfo = new HashMap[Int, ReceiverInfo] with SynchronizedMap[Int, ReceiverInfo] - val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] - with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] - val timeout = AkkaUtils.askTimeout(ssc.conf) - val listenerBus = ssc.scheduler.listenerBus + private val receiverInputStreams = ssc.graph.getReceiverInputStreams() + private val receiverInputStreamIds = receiverInputStreams.map { _.id } + private val receiverExecutor = new ReceiverLauncher() + private val receiverInfo = new HashMap[Int, ReceiverInfo] with SynchronizedMap[Int, ReceiverInfo] + private val receivedBlockTracker = new ReceivedBlockTracker( + ssc.sparkContext.conf, + ssc.sparkContext.hadoopConfiguration, + receiverInputStreamIds, + ssc.scheduler.clock, + Option(ssc.checkpointDir) + ) + private val listenerBus = ssc.scheduler.listenerBus // actor is created when generator starts. // This not being null means the tracker has been started and not stopped - var actor: ActorRef = null - var currentTime: Time = null + private var actor: ActorRef = null /** Start the actor and receiver execution thread. */ def start() = synchronized { @@ -75,7 +81,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { if (!receiverInputStreams.isEmpty) { actor = ssc.env.actorSystem.actorOf(Props(new ReceiverTrackerActor), "ReceiverTracker") - receiverExecutor.start() + if (!skipReceiverLaunch) receiverExecutor.start() logInfo("ReceiverTracker started") } } @@ -84,45 +90,59 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { def stop() = synchronized { if (!receiverInputStreams.isEmpty && actor != null) { // First, stop the receivers - receiverExecutor.stop() + if (!skipReceiverLaunch) receiverExecutor.stop() // Finally, stop the actor ssc.env.actorSystem.stop(actor) actor = null + receivedBlockTracker.stop() logInfo("ReceiverTracker stopped") } } - /** Return all the blocks received from a receiver. */ - def getReceivedBlockInfo(streamId: Int): Array[ReceivedBlockInfo] = { - val receivedBlockInfo = getReceivedBlockInfoQueue(streamId).dequeueAll(x => true) - logInfo("Stream " + streamId + " received " + receivedBlockInfo.size + " blocks") - receivedBlockInfo.toArray + /** Allocate all unallocated blocks to the given batch. */ + def allocateBlocksToBatch(batchTime: Time): Unit = { + if (receiverInputStreams.nonEmpty) { + receivedBlockTracker.allocateBlocksToBatch(batchTime) + } + } + + /** Get the blocks for the given batch and all input streams. */ + def getBlocksOfBatch(batchTime: Time): Map[Int, Seq[ReceivedBlockInfo]] = { + receivedBlockTracker.getBlocksOfBatch(batchTime) } - private def getReceivedBlockInfoQueue(streamId: Int) = { - receivedBlockInfo.getOrElseUpdate(streamId, new SynchronizedQueue[ReceivedBlockInfo]) + /** Get the blocks allocated to the given batch and stream. */ + def getBlocksOfBatchAndStream(batchTime: Time, streamId: Int): Seq[ReceivedBlockInfo] = { + synchronized { + receivedBlockTracker.getBlocksOfBatchAndStream(batchTime, streamId) + } + } + + /** Clean up metadata older than the given threshold time */ + def cleanupOldMetadata(cleanupThreshTime: Time) { + receivedBlockTracker.cleanupOldBatches(cleanupThreshTime) } /** Register a receiver */ - def registerReceiver( + private def registerReceiver( streamId: Int, typ: String, host: String, receiverActor: ActorRef, sender: ActorRef ) { - if (!receiverInputStreamMap.contains(streamId)) { - throw new Exception("Register received for unexpected id " + streamId) + if (!receiverInputStreamIds.contains(streamId)) { + throw new SparkException("Register received for unexpected id " + streamId) } receiverInfo(streamId) = ReceiverInfo( streamId, s"${typ}-${streamId}", receiverActor, true, host) - ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) + listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) } /** Deregister a receiver */ - def deregisterReceiver(streamId: Int, message: String, error: String) { + private def deregisterReceiver(streamId: Int, message: String, error: String) { val newReceiverInfo = receiverInfo.get(streamId) match { case Some(oldInfo) => oldInfo.copy(actor = null, active = false, lastErrorMessage = message, lastError = error) @@ -131,7 +151,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) } receiverInfo(streamId) = newReceiverInfo - ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId))) + listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId))) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { @@ -141,14 +161,12 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { } /** Add new blocks for the given stream */ - def addBlocks(receivedBlockInfo: ReceivedBlockInfo) { - getReceivedBlockInfoQueue(receivedBlockInfo.streamId) += receivedBlockInfo - logDebug("Stream " + receivedBlockInfo.streamId + " received new blocks: " + - receivedBlockInfo.blockStoreResult.blockId) + private def addBlock(receivedBlockInfo: ReceivedBlockInfo): Boolean = { + receivedBlockTracker.addBlock(receivedBlockInfo) } /** Report error sent by a receiver */ - def reportError(streamId: Int, message: String, error: String) { + private def reportError(streamId: Int, message: String, error: String) { val newReceiverInfo = receiverInfo.get(streamId) match { case Some(oldInfo) => oldInfo.copy(lastErrorMessage = message, lastError = error) @@ -157,7 +175,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) } receiverInfo(streamId) = newReceiverInfo - ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(receiverInfo(streamId))) + listenerBus.post(StreamingListenerReceiverError(receiverInfo(streamId))) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { @@ -167,8 +185,8 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { } /** Check if any blocks are left to be processed */ - def hasMoreReceivedBlockIds: Boolean = { - !receivedBlockInfo.values.forall(_.isEmpty) + def hasUnallocatedBlocks: Boolean = { + receivedBlockTracker.hasUnallocatedReceivedBlocks } /** Actor to receive messages from the receivers. */ @@ -178,8 +196,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true case AddBlock(receivedBlockInfo) => - addBlocks(receivedBlockInfo) - sender ! true + sender ! addBlock(receivedBlockInfo) case ReportError(streamId, message, error) => reportError(streamId, message, error) case DeregisterReceiver(streamId, message, error) => @@ -194,6 +211,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { @transient val thread = new Thread() { override def run() { try { + SparkEnv.set(env) startReceivers() } catch { case ie: InterruptedException => logInfo("ReceiverLauncher interrupted") @@ -267,7 +285,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { // Distribute the receivers and start them logInfo("Starting " + receivers.length + " receivers") - ssc.sparkContext.runJob(tempRDD, startReceiver) + ssc.sparkContext.runJob(tempRDD, ssc.sparkContext.clean(startReceiver)) logInfo("All of the receivers have been terminated") } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 6c8bb50145367..dbab685dc3511 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.StreamingContext._ - -import org.apache.spark.rdd.{BlockRDD, RDD} -import org.apache.spark.SparkContext._ +import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} +import scala.language.existentials +import scala.reflect.ClassTag import util.ManualClock -import org.apache.spark.{SparkException, SparkConf} -import org.apache.spark.streaming.dstream.{WindowedDStream, DStream} -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} -import scala.reflect.ClassTag + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel -import scala.collection.mutable +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.dstream.{DStream, WindowedDStream} class BasicOperationsSuite extends TestSuiteBase { test("map") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala new file mode 100644 index 0000000000000..fd9c97f551c62 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -0,0 +1,242 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import java.io.File + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.{implicitConversions, postfixOps} +import scala.util.Random + +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.receiver.BlockManagerBasedStoreResult +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.util.{Clock, ManualClock, SystemClock, WriteAheadLogReader} +import org.apache.spark.streaming.util.WriteAheadLogSuite._ +import org.apache.spark.util.Utils + +class ReceivedBlockTrackerSuite + extends FunSuite with BeforeAndAfter with Matchers with Logging { + + val conf = new SparkConf().setMaster("local[2]").setAppName("ReceivedBlockTrackerSuite") + conf.set("spark.streaming.receivedBlockTracker.writeAheadLog.rotationIntervalSecs", "1") + + val hadoopConf = new Configuration() + val akkaTimeout = 10 seconds + val streamId = 1 + + var allReceivedBlockTrackers = new ArrayBuffer[ReceivedBlockTracker]() + var checkpointDirectory: File = null + + before { + checkpointDirectory = Files.createTempDir() + } + + after { + allReceivedBlockTrackers.foreach { _.stop() } + if (checkpointDirectory != null && checkpointDirectory.exists()) { + FileUtils.deleteDirectory(checkpointDirectory) + checkpointDirectory = null + } + } + + test("block addition, and block to batch allocation") { + val receivedBlockTracker = createTracker(enableCheckpoint = false) + receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual Seq.empty + + val blockInfos = generateBlockInfos() + blockInfos.map(receivedBlockTracker.addBlock) + + // Verify added blocks are unallocated blocks + receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos + + // Allocate the blocks to a batch and verify that all of them have been allocated + receivedBlockTracker.allocateBlocksToBatch(1) + receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos + receivedBlockTracker.getUnallocatedBlocks(streamId) shouldBe empty + + // Allocate no blocks to another batch + receivedBlockTracker.allocateBlocksToBatch(2) + receivedBlockTracker.getBlocksOfBatchAndStream(2, streamId) shouldBe empty + + // Verify that batch 2 cannot be allocated again + intercept[SparkException] { + receivedBlockTracker.allocateBlocksToBatch(2) + } + + // Verify that older batches cannot be allocated again + intercept[SparkException] { + receivedBlockTracker.allocateBlocksToBatch(1) + } + } + + test("block addition, block to batch allocation and cleanup with write ahead log") { + val manualClock = new ManualClock + conf.getInt( + "spark.streaming.receivedBlockTracker.writeAheadLog.rotationIntervalSecs", -1) should be (1) + + // Set the time increment level to twice the rotation interval so that every increment creates + // a new log file + val timeIncrementMillis = 2000L + def incrementTime() { + manualClock.addToTime(timeIncrementMillis) + } + + // Generate and add blocks to the given tracker + def addBlockInfos(tracker: ReceivedBlockTracker): Seq[ReceivedBlockInfo] = { + val blockInfos = generateBlockInfos() + blockInfos.map(tracker.addBlock) + blockInfos + } + + // Print the data present in the log ahead files in the log directory + def printLogFiles(message: String) { + val fileContents = getWriteAheadLogFiles().map { file => + (s"\n>>>>> $file: <<<<<\n${getWrittenLogData(file).mkString("\n")}") + }.mkString("\n") + logInfo(s"\n\n=====================\n$message\n$fileContents\n=====================\n") + } + + // Start tracker and add blocks + val tracker1 = createTracker(enableCheckpoint = true, clock = manualClock) + val blockInfos1 = addBlockInfos(tracker1) + tracker1.getUnallocatedBlocks(streamId).toList shouldEqual blockInfos1 + + // Verify whether write ahead log has correct contents + val expectedWrittenData1 = blockInfos1.map(BlockAdditionEvent) + getWrittenLogData() shouldEqual expectedWrittenData1 + getWriteAheadLogFiles() should have size 1 + + // Restart tracker and verify recovered list of unallocated blocks + incrementTime() + val tracker2 = createTracker(enableCheckpoint = true, clock = manualClock) + tracker2.getUnallocatedBlocks(streamId).toList shouldEqual blockInfos1 + + // Allocate blocks to batch and verify whether the unallocated blocks got allocated + val batchTime1 = manualClock.currentTime + tracker2.allocateBlocksToBatch(batchTime1) + tracker2.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual blockInfos1 + + // Add more blocks and allocate to another batch + incrementTime() + val batchTime2 = manualClock.currentTime + val blockInfos2 = addBlockInfos(tracker2) + tracker2.allocateBlocksToBatch(batchTime2) + tracker2.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 + + // Verify whether log has correct contents + val expectedWrittenData2 = expectedWrittenData1 ++ + Seq(createBatchAllocation(batchTime1, blockInfos1)) ++ + blockInfos2.map(BlockAdditionEvent) ++ + Seq(createBatchAllocation(batchTime2, blockInfos2)) + getWrittenLogData() shouldEqual expectedWrittenData2 + + // Restart tracker and verify recovered state + incrementTime() + val tracker3 = createTracker(enableCheckpoint = true, clock = manualClock) + tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual blockInfos1 + tracker3.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 + tracker3.getUnallocatedBlocks(streamId) shouldBe empty + + // Cleanup first batch but not second batch + val oldestLogFile = getWriteAheadLogFiles().head + incrementTime() + tracker3.cleanupOldBatches(batchTime2) + + // Verify that the batch allocations have been cleaned, and the act has been written to log + tracker3.getBlocksOfBatchAndStream(batchTime1, streamId) shouldEqual Seq.empty + getWrittenLogData(getWriteAheadLogFiles().last) should contain(createBatchCleanup(batchTime1)) + + // Verify that at least one log file gets deleted + eventually(timeout(10 seconds), interval(10 millisecond)) { + getWriteAheadLogFiles() should not contain oldestLogFile + } + printLogFiles("After cleanup") + + // Restart tracker and verify recovered state, specifically whether info about the first + // batch has been removed, but not the second batch + incrementTime() + val tracker4 = createTracker(enableCheckpoint = true, clock = manualClock) + tracker4.getUnallocatedBlocks(streamId) shouldBe empty + tracker4.getBlocksOfBatchAndStream(batchTime1, streamId) shouldBe empty // should be cleaned + tracker4.getBlocksOfBatchAndStream(batchTime2, streamId) shouldEqual blockInfos2 + } + + /** + * Create tracker object with the optional provided clock. Use fake clock if you + * want to control time by manually incrementing it to test log cleanup. + */ + def createTracker(enableCheckpoint: Boolean, clock: Clock = new SystemClock): ReceivedBlockTracker = { + val cpDirOption = if (enableCheckpoint) Some(checkpointDirectory.toString) else None + val tracker = new ReceivedBlockTracker(conf, hadoopConf, Seq(streamId), clock, cpDirOption) + allReceivedBlockTrackers += tracker + tracker + } + + /** Generate blocks infos using random ids */ + def generateBlockInfos(): Seq[ReceivedBlockInfo] = { + List.fill(5)(ReceivedBlockInfo(streamId, 0, + BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt))))) + } + + /** Get all the data written in the given write ahead log file. */ + def getWrittenLogData(logFile: String): Seq[ReceivedBlockTrackerLogEvent] = { + getWrittenLogData(Seq(logFile)) + } + + /** + * Get all the data written in the given write ahead log files. By default, it will read all + * files in the test log directory. + */ + def getWrittenLogData(logFiles: Seq[String] = getWriteAheadLogFiles): Seq[ReceivedBlockTrackerLogEvent] = { + logFiles.flatMap { + file => new WriteAheadLogReader(file, hadoopConf).toSeq + }.map { byteBuffer => + Utils.deserialize[ReceivedBlockTrackerLogEvent](byteBuffer.array) + }.toList + } + + /** Get all the write ahead log files in the test directory */ + def getWriteAheadLogFiles(): Seq[String] = { + import ReceivedBlockTracker._ + val logDir = checkpointDirToLogDir(checkpointDirectory.toString) + getLogFilesInDirectory(logDir).map { _.toString } + } + + /** Create batch allocation object from the given info */ + def createBatchAllocation(time: Long, blockInfos: Seq[ReceivedBlockInfo]): BatchAllocationEvent = { + BatchAllocationEvent(time, AllocatedBlocks(Map((streamId -> blockInfos)))) + } + + /** Create batch cleanup object from the given info */ + def createBatchCleanup(time: Long, moreTimes: Long*): BatchCleanupEvent = { + BatchCleanupEvent((Seq(time) ++ moreTimes).map(Time.apply)) + } + + implicit def millisToTime(milliseconds: Long): Time = Time(milliseconds) + + implicit def timeToMillis(time: Time): Long = time.milliseconds +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala index 10160244bcc91..d2b983c4b4d1a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -117,12 +117,12 @@ class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { ) // Create the RDD and verify whether the returned data is correct - val rdd = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, + val rdd = new WriteAheadLogBackedBlockRDD[String](sparkContext, blockIds.toArray, segments.toArray, storeInBlockManager = false, StorageLevel.MEMORY_ONLY) assert(rdd.collect() === data.flatten) if (testStoreInBM) { - val rdd2 = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, + val rdd2 = new WriteAheadLogBackedBlockRDD[String](sparkContext, blockIds.toArray, segments.toArray, storeInBlockManager = true, StorageLevel.MEMORY_ONLY) assert(rdd2.collect() === data.flatten) assert( From 5b3b6f6f5f029164d7749366506e142b104c1d43 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 5 Nov 2014 10:33:13 -0800 Subject: [PATCH 1224/1492] [SPARK-4197] [mllib] GradientBoosting API cleanup and examples in Scala, Java MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### Summary * Made it easier to construct default Strategy and BoostingStrategy and to set parameters using simple types. * Added Scala and Java examples for GradientBoostedTrees * small cleanups and fixes ### Details GradientBoosting bug fixes (“bug” = bad default options) * Force boostingStrategy.weakLearnerParams.algo = Regression * Force boostingStrategy.weakLearnerParams.impurity = impurity.Variance * Only persist data if not yet persisted (since it causes an error if persisted twice) BoostingStrategy * numEstimators: renamed to numIterations * removed subsamplingRate (duplicated by Strategy) * removed categoricalFeaturesInfo since it belongs with the weak learner params (since boosting can be oblivious to feature type) * Changed algo to var (not val) and added BeanProperty, with overload taking String argument * Added assertValid() method * Updated defaultParams() method and eliminated defaultWeakLearnerParams() since that belongs in Strategy Strategy (for DecisionTree) * Changed algo to var (not val) and added BeanProperty, with overload taking String argument * Added setCategoricalFeaturesInfo method taking Java Map. * Cleaned up assertValid * Changed val’s to def’s since parameters can now be changed. CC: manishamde mengxr codedeft Author: Joseph K. Bradley Closes #3094 from jkbradley/gbt-api and squashes the following commits: 7a27e22 [Joseph K. Bradley] scalastyle fix 52013d5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into gbt-api e9b8410 [Joseph K. Bradley] Summary of changes --- .../mllib/JavaGradientBoostedTrees.java | 126 +++++++++++++ .../examples/mllib/DecisionTreeRunner.scala | 64 +++++-- .../examples/mllib/GradientBoostedTrees.scala | 146 +++++++++++++++ .../spark/mllib/tree/GradientBoosting.scala | 169 ++++++------------ .../tree/configuration/BoostingStrategy.scala | 78 ++++---- .../mllib/tree/configuration/Strategy.scala | 51 ++++-- .../mllib/tree/GradientBoostingSuite.scala | 34 ++-- 7 files changed, 462 insertions(+), 206 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTrees.scala diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java new file mode 100644 index 0000000000000..1af2067b2b929 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTrees.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import scala.Tuple2; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.GradientBoosting; +import org.apache.spark.mllib.tree.configuration.BoostingStrategy; +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel; +import org.apache.spark.mllib.util.MLUtils; + +/** + * Classification and regression using gradient-boosted decision trees. + */ +public final class JavaGradientBoostedTrees { + + private static void usage() { + System.err.println("Usage: JavaGradientBoostedTrees " + + " "); + System.exit(-1); + } + + public static void main(String[] args) { + String datapath = "data/mllib/sample_libsvm_data.txt"; + String algo = "Classification"; + if (args.length >= 1) { + datapath = args[0]; + } + if (args.length >= 2) { + algo = args[1]; + } + if (args.length > 2) { + usage(); + } + SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTrees"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); + + // Set parameters. + // Note: All features are treated as continuous. + BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams(algo); + boostingStrategy.setNumIterations(10); + boostingStrategy.weakLearnerParams().setMaxDepth(5); + + if (algo.equals("Classification")) { + // Compute the number of classes from the data. + Integer numClasses = data.map(new Function() { + @Override public Double call(LabeledPoint p) { + return p.label(); + } + }).countByValue().size(); + boostingStrategy.setNumClassesForClassification(numClasses); // ignored for Regression + + // Train a GradientBoosting model for classification. + final WeightedEnsembleModel model = GradientBoosting.trainClassifier(data, boostingStrategy); + + // Evaluate model on training instances and compute training error + JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double trainErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / data.count(); + System.out.println("Training error: " + trainErr); + System.out.println("Learned classification tree model:\n" + model); + } else if (algo.equals("Regression")) { + // Train a GradientBoosting model for classification. + final WeightedEnsembleModel model = GradientBoosting.trainRegressor(data, boostingStrategy); + + // Evaluate model on training instances and compute training error + JavaPairRDD predictionAndLabel = + data.mapToPair(new PairFunction() { + @Override public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double trainMSE = + predictionAndLabel.map(new Function, Double>() { + @Override public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); + System.out.println("Training Mean Squared Error: " + trainMSE); + System.out.println("Learned regression tree model:\n" + model); + } else { + usage(); + } + + sc.stop(); + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 49751a30491d0..63f02cf7b98b9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -154,20 +154,30 @@ object DecisionTreeRunner { } } - def run(params: Params) { - - val conf = new SparkConf().setAppName(s"DecisionTreeRunner with $params") - val sc = new SparkContext(conf) - - println(s"DecisionTreeRunner with parameters:\n$params") - + /** + * Load training and test data from files. + * @param input Path to input dataset. + * @param dataFormat "libsvm" or "dense" + * @param testInput Path to test dataset. + * @param algo Classification or Regression + * @param fracTest Fraction of input data to hold out for testing. Ignored if testInput given. + * @return (training dataset, test dataset, number of classes), + * where the number of classes is inferred from data (and set to 0 for Regression) + */ + private[mllib] def loadDatasets( + sc: SparkContext, + input: String, + dataFormat: String, + testInput: String, + algo: Algo, + fracTest: Double): (RDD[LabeledPoint], RDD[LabeledPoint], Int) = { // Load training data and cache it. - val origExamples = params.dataFormat match { - case "dense" => MLUtils.loadLabeledPoints(sc, params.input).cache() - case "libsvm" => MLUtils.loadLibSVMFile(sc, params.input).cache() + val origExamples = dataFormat match { + case "dense" => MLUtils.loadLabeledPoints(sc, input).cache() + case "libsvm" => MLUtils.loadLibSVMFile(sc, input).cache() } // For classification, re-index classes if needed. - val (examples, classIndexMap, numClasses) = params.algo match { + val (examples, classIndexMap, numClasses) = algo match { case Classification => { // classCounts: class --> # examples in class val classCounts = origExamples.map(_.label).countByValue() @@ -205,14 +215,14 @@ object DecisionTreeRunner { } // Create training, test sets. - val splits = if (params.testInput != "") { + val splits = if (testInput != "") { // Load testInput. val numFeatures = examples.take(1)(0).features.size - val origTestExamples = params.dataFormat match { - case "dense" => MLUtils.loadLabeledPoints(sc, params.testInput) - case "libsvm" => MLUtils.loadLibSVMFile(sc, params.testInput, numFeatures) + val origTestExamples = dataFormat match { + case "dense" => MLUtils.loadLabeledPoints(sc, testInput) + case "libsvm" => MLUtils.loadLibSVMFile(sc, testInput, numFeatures) } - params.algo match { + algo match { case Classification => { // classCounts: class --> # examples in class val testExamples = { @@ -229,17 +239,31 @@ object DecisionTreeRunner { } } else { // Split input into training, test. - examples.randomSplit(Array(1.0 - params.fracTest, params.fracTest)) + examples.randomSplit(Array(1.0 - fracTest, fracTest)) } val training = splits(0).cache() val test = splits(1).cache() + val numTraining = training.count() val numTest = test.count() - println(s"numTraining = $numTraining, numTest = $numTest.") examples.unpersist(blocking = false) + (training, test, numClasses) + } + + def run(params: Params) { + + val conf = new SparkConf().setAppName(s"DecisionTreeRunner with $params") + val sc = new SparkContext(conf) + + println(s"DecisionTreeRunner with parameters:\n$params") + + // Load training and test data and cache it. + val (training, test, numClasses) = loadDatasets(sc, params.input, params.dataFormat, + params.testInput, params.algo, params.fracTest) + val impurityCalculator = params.impurity match { case Gini => impurity.Gini case Entropy => impurity.Entropy @@ -338,7 +362,9 @@ object DecisionTreeRunner { /** * Calculates the mean squared error for regression. */ - private def meanSquaredError(tree: WeightedEnsembleModel, data: RDD[LabeledPoint]): Double = { + private[mllib] def meanSquaredError( + tree: WeightedEnsembleModel, + data: RDD[LabeledPoint]): Double = { data.map { y => val err = tree.predict(y.features) - y.label err * err diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTrees.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTrees.scala new file mode 100644 index 0000000000000..9b6db01448be0 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTrees.scala @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.evaluation.MulticlassMetrics +import org.apache.spark.mllib.tree.GradientBoosting +import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Algo} +import org.apache.spark.util.Utils + +/** + * An example runner for Gradient Boosting using decision trees as weak learners. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.GradientBoostedTrees [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + * + * Note: This script treats all features as real-valued (not categorical). + * To include categorical features, modify categoricalFeaturesInfo. + */ +object GradientBoostedTrees { + + case class Params( + input: String = null, + testInput: String = "", + dataFormat: String = "libsvm", + algo: String = "Classification", + maxDepth: Int = 5, + numIterations: Int = 10, + fracTest: Double = 0.2) extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("GradientBoostedTrees") { + head("GradientBoostedTrees: an example decision tree app.") + opt[String]("algo") + .text(s"algorithm (${Algo.values.mkString(",")}), default: ${defaultParams.algo}") + .action((x, c) => c.copy(algo = x)) + opt[Int]("maxDepth") + .text(s"max depth of the tree, default: ${defaultParams.maxDepth}") + .action((x, c) => c.copy(maxDepth = x)) + opt[Int]("numIterations") + .text(s"number of iterations of boosting," + s" default: ${defaultParams.numIterations}") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("fracTest") + .text(s"fraction of data to hold out for testing. If given option testInput, " + + s"this option is ignored. default: ${defaultParams.fracTest}") + .action((x, c) => c.copy(fracTest = x)) + opt[String]("testInput") + .text(s"input path to test dataset. If given, option fracTest is ignored." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) + opt[String]("") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(dataFormat = x)) + arg[String]("") + .text("input path to labeled examples") + .required() + .action((x, c) => c.copy(input = x)) + checkConfig { params => + if (params.fracTest < 0 || params.fracTest > 1) { + failure(s"fracTest ${params.fracTest} value incorrect; should be in [0,1].") + } else { + success + } + } + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + + val conf = new SparkConf().setAppName(s"GradientBoostedTrees with $params") + val sc = new SparkContext(conf) + + println(s"GradientBoostedTrees with parameters:\n$params") + + // Load training and test data and cache it. + val (training, test, numClasses) = DecisionTreeRunner.loadDatasets(sc, params.input, + params.dataFormat, params.testInput, Algo.withName(params.algo), params.fracTest) + + val boostingStrategy = BoostingStrategy.defaultParams(params.algo) + boostingStrategy.numClassesForClassification = numClasses + boostingStrategy.numIterations = params.numIterations + boostingStrategy.weakLearnerParams.maxDepth = params.maxDepth + + val randomSeed = Utils.random.nextInt() + if (params.algo == "Classification") { + val startTime = System.nanoTime() + val model = GradientBoosting.trainClassifier(training, boostingStrategy) + val elapsedTime = (System.nanoTime() - startTime) / 1e9 + println(s"Training time: $elapsedTime seconds") + if (model.totalNumNodes < 30) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } + val trainAccuracy = + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) + .precision + println(s"Train accuracy = $trainAccuracy") + val testAccuracy = + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + println(s"Test accuracy = $testAccuracy") + } else if (params.algo == "Regression") { + val startTime = System.nanoTime() + val model = GradientBoosting.trainRegressor(training, boostingStrategy) + val elapsedTime = (System.nanoTime() - startTime) / 1e9 + println(s"Training time: $elapsedTime seconds") + if (model.totalNumNodes < 30) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } + val trainMSE = DecisionTreeRunner.meanSquaredError(model, training) + println(s"Train mean squared error = $trainMSE") + val testMSE = DecisionTreeRunner.meanSquaredError(model, test) + println(s"Test mean squared error = $testMSE") + } + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala index 1a847201ce157..f729344a682e2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala @@ -17,30 +17,49 @@ package org.apache.spark.mllib.tree -import scala.collection.JavaConverters._ - +import org.apache.spark.Logging import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.tree.configuration.{Strategy, BoostingStrategy} -import org.apache.spark.Logging -import org.apache.spark.mllib.tree.impl.TimeTracker -import org.apache.spark.mllib.tree.loss.Losses -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel} import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.storage.StorageLevel +import org.apache.spark.mllib.tree.configuration.BoostingStrategy import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum +import org.apache.spark.mllib.tree.impl.TimeTracker +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel} +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * :: Experimental :: - * A class that implements gradient boosting for regression and binary classification problems. + * A class that implements Stochastic Gradient Boosting + * for regression and binary classification problems. + * + * The implementation is based upon: + * J.H. Friedman. "Stochastic Gradient Boosting." 1999. + * + * Notes: + * - This currently can be run with several loss functions. However, only SquaredError is + * fully supported. Specifically, the loss function should be used to compute the gradient + * (to re-label training instances on each iteration) and to weight weak hypotheses. + * Currently, gradients are computed correctly for the available loss functions, + * but weak hypothesis weights are not computed correctly for LogLoss or AbsoluteError. + * Running with those losses will likely behave reasonably, but lacks the same guarantees. + * * @param boostingStrategy Parameters for the gradient boosting algorithm */ @Experimental class GradientBoosting ( private val boostingStrategy: BoostingStrategy) extends Serializable with Logging { + boostingStrategy.weakLearnerParams.algo = Regression + boostingStrategy.weakLearnerParams.impurity = impurity.Variance + + // Ensure values for weak learner are the same as what is provided to the boosting algorithm. + boostingStrategy.weakLearnerParams.numClassesForClassification = + boostingStrategy.numClassesForClassification + + boostingStrategy.assertValid() + /** * Method to train a gradient boosting model * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. @@ -51,6 +70,7 @@ class GradientBoosting ( algo match { case Regression => GradientBoosting.boost(input, boostingStrategy) case Classification => + // Map labels to -1, +1 so binary classification can be treated as regression. val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) GradientBoosting.boost(remappedInput, boostingStrategy) case _ => @@ -118,120 +138,32 @@ object GradientBoosting extends Logging { } /** - * Method to train a gradient boosting binary classification model. - * - * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * For classification, labels should take values {0, 1, ..., numClasses-1}. - * For regression, labels are real numbers. - * @param numEstimators Number of estimators used in boosting stages. In other words, - * number of boosting iterations performed. - * @param loss Loss function used for minimization during gradient boosting. - * @param learningRate Learning rate for shrinking the contribution of each estimator. The - * learning rate should be between in the interval (0, 1] - * @param subsamplingRate Fraction of the training data used for learning the decision tree. - * @param numClassesForClassification Number of classes for classification. - * (Ignored for regression.) - * @param categoricalFeaturesInfo A map storing information about the categorical variables and - * the number of discrete values they take. For example, - * an entry (n -> k) implies the feature n is categorical with k - * categories 0, 1, 2, ... , k-1. It's important to note that - * features are zero-indexed. - * @param weakLearnerParams Parameters for the weak learner. (Currently only decision tree is - * supported.) - * @return WeightedEnsembleModel that can be used for prediction + * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#train]] */ - def trainClassifier( - input: RDD[LabeledPoint], - numEstimators: Int, - loss: String, - learningRate: Double, - subsamplingRate: Double, - numClassesForClassification: Int, - categoricalFeaturesInfo: Map[Int, Int], - weakLearnerParams: Strategy): WeightedEnsembleModel = { - val lossType = Losses.fromString(loss) - val boostingStrategy = new BoostingStrategy(Classification, numEstimators, lossType, - learningRate, subsamplingRate, numClassesForClassification, categoricalFeaturesInfo, - weakLearnerParams) - new GradientBoosting(boostingStrategy).train(input) - } - - /** - * Method to train a gradient boosting regression model. - * - * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * For classification, labels should take values {0, 1, ..., numClasses-1}. - * For regression, labels are real numbers. - * @param numEstimators Number of estimators used in boosting stages. In other words, - * number of boosting iterations performed. - * @param loss Loss function used for minimization during gradient boosting. - * @param learningRate Learning rate for shrinking the contribution of each estimator. The - * learning rate should be between in the interval (0, 1] - * @param subsamplingRate Fraction of the training data used for learning the decision tree. - * @param numClassesForClassification Number of classes for classification. - * (Ignored for regression.) - * @param categoricalFeaturesInfo A map storing information about the categorical variables and - * the number of discrete values they take. For example, - * an entry (n -> k) implies the feature n is categorical with k - * categories 0, 1, 2, ... , k-1. It's important to note that - * features are zero-indexed. - * @param weakLearnerParams Parameters for the weak learner. (Currently only decision tree is - * supported.) - * @return WeightedEnsembleModel that can be used for prediction - */ - def trainRegressor( - input: RDD[LabeledPoint], - numEstimators: Int, - loss: String, - learningRate: Double, - subsamplingRate: Double, - numClassesForClassification: Int, - categoricalFeaturesInfo: Map[Int, Int], - weakLearnerParams: Strategy): WeightedEnsembleModel = { - val lossType = Losses.fromString(loss) - val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType, - learningRate, subsamplingRate, numClassesForClassification, categoricalFeaturesInfo, - weakLearnerParams) - new GradientBoosting(boostingStrategy).train(input) + def train( + input: JavaRDD[LabeledPoint], + boostingStrategy: BoostingStrategy): WeightedEnsembleModel = { + train(input.rdd, boostingStrategy) } /** * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]] */ def trainClassifier( - input: RDD[LabeledPoint], - numEstimators: Int, - loss: String, - learningRate: Double, - subsamplingRate: Double, - numClassesForClassification: Int, - categoricalFeaturesInfo:java.util.Map[java.lang.Integer, java.lang.Integer], - weakLearnerParams: Strategy): WeightedEnsembleModel = { - trainClassifier(input, numEstimators, loss, learningRate, subsamplingRate, - numClassesForClassification, - categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, - weakLearnerParams) + input: JavaRDD[LabeledPoint], + boostingStrategy: BoostingStrategy): WeightedEnsembleModel = { + trainClassifier(input.rdd, boostingStrategy) } /** * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]] */ def trainRegressor( - input: RDD[LabeledPoint], - numEstimators: Int, - loss: String, - learningRate: Double, - subsamplingRate: Double, - numClassesForClassification: Int, - categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], - weakLearnerParams: Strategy): WeightedEnsembleModel = { - trainRegressor(input, numEstimators, loss, learningRate, subsamplingRate, - numClassesForClassification, - categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, - weakLearnerParams) + input: JavaRDD[LabeledPoint], + boostingStrategy: BoostingStrategy): WeightedEnsembleModel = { + trainRegressor(input.rdd, boostingStrategy) } - /** * Internal method for performing regression using trees as base learners. * @param input training dataset @@ -247,15 +179,17 @@ object GradientBoosting extends Logging { timer.start("init") // Initialize gradient boosting parameters - val numEstimators = boostingStrategy.numEstimators - val baseLearners = new Array[DecisionTreeModel](numEstimators) - val baseLearnerWeights = new Array[Double](numEstimators) + val numIterations = boostingStrategy.numIterations + val baseLearners = new Array[DecisionTreeModel](numIterations) + val baseLearnerWeights = new Array[Double](numIterations) val loss = boostingStrategy.loss val learningRate = boostingStrategy.learningRate val strategy = boostingStrategy.weakLearnerParams // Cache input - input.persist(StorageLevel.MEMORY_AND_DISK) + if (input.getStorageLevel == StorageLevel.NONE) { + input.persist(StorageLevel.MEMORY_AND_DISK) + } timer.stop("init") @@ -264,7 +198,7 @@ object GradientBoosting extends Logging { logDebug("##########") var data = input - // 1. Initialize tree + // Initialize tree timer.start("building tree 0") val firstTreeModel = new DecisionTree(strategy).train(data) baseLearners(0) = firstTreeModel @@ -280,7 +214,7 @@ object GradientBoosting extends Logging { point.features)) var m = 1 - while (m < numEstimators) { + while (m < numIterations) { timer.start(s"building tree $m") logDebug("###################################################") logDebug("Gradient boosting tree iteration " + m) @@ -289,6 +223,9 @@ object GradientBoosting extends Logging { timer.stop(s"building tree $m") // Create partial model baseLearners(m) = model + // Note: The setting of baseLearnerWeights is incorrect for losses other than SquaredError. + // Technically, the weight should be optimized for the particular loss. + // However, the behavior should be reasonable, though not optimal. baseLearnerWeights(m) = learningRate // Note: A model of type regression is used since we require raw prediction val partialModel = new WeightedEnsembleModel(baseLearners.slice(0, m + 1), @@ -305,8 +242,6 @@ object GradientBoosting extends Logging { logInfo("Internal timing for DecisionTree:") logInfo(s"$timer") - - // 3. Output classifier new WeightedEnsembleModel(baseLearners, baseLearnerWeights, boostingStrategy.algo, Sum) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index 501d9ff9ea9b7..abbda040bd528 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -21,7 +21,6 @@ import scala.beans.BeanProperty import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.{Gini, Variance} import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} /** @@ -30,46 +29,58 @@ import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} * @param algo Learning goal. Supported: * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] - * @param numEstimators Number of estimators used in boosting stages. In other words, - * number of boosting iterations performed. + * @param numIterations Number of iterations of boosting. In other words, the number of + * weak hypotheses used in the final model. * @param loss Loss function used for minimization during gradient boosting. * @param learningRate Learning rate for shrinking the contribution of each estimator. The * learning rate should be between in the interval (0, 1] - * @param subsamplingRate Fraction of the training data used for learning the decision tree. * @param numClassesForClassification Number of classes for classification. * (Ignored for regression.) + * This setting overrides any setting in [[weakLearnerParams]]. * Default value is 2 (binary classification). - * @param categoricalFeaturesInfo A map storing information about the categorical variables and the - * number of discrete values they take. For example, an entry (n -> - * k) implies the feature n is categorical with k categories 0, - * 1, 2, ... , k-1. It's important to note that features are - * zero-indexed. * @param weakLearnerParams Parameters for weak learners. Currently only decision trees are * supported. */ @Experimental case class BoostingStrategy( // Required boosting parameters - algo: Algo, - @BeanProperty var numEstimators: Int, + @BeanProperty var algo: Algo, + @BeanProperty var numIterations: Int, @BeanProperty var loss: Loss, // Optional boosting parameters @BeanProperty var learningRate: Double = 0.1, - @BeanProperty var subsamplingRate: Double = 1.0, @BeanProperty var numClassesForClassification: Int = 2, - @BeanProperty var categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), @BeanProperty var weakLearnerParams: Strategy) extends Serializable { - require(learningRate <= 1, "Learning rate should be <= 1. Provided learning rate is " + - s"$learningRate.") - require(learningRate > 0, "Learning rate should be > 0. Provided learning rate is " + - s"$learningRate.") - // Ensure values for weak learner are the same as what is provided to the boosting algorithm. - weakLearnerParams.categoricalFeaturesInfo = categoricalFeaturesInfo weakLearnerParams.numClassesForClassification = numClassesForClassification - weakLearnerParams.subsamplingRate = subsamplingRate + /** + * Sets Algorithm using a String. + */ + def setAlgo(algo: String): Unit = algo match { + case "Classification" => setAlgo(Classification) + case "Regression" => setAlgo(Regression) + } + + /** + * Check validity of parameters. + * Throws exception if invalid. + */ + private[tree] def assertValid(): Unit = { + algo match { + case Classification => + require(numClassesForClassification == 2) + case Regression => + // nothing + case _ => + throw new IllegalArgumentException( + s"BoostingStrategy given invalid algo parameter: $algo." + + s" Valid settings are: Classification, Regression.") + } + require(learningRate > 0 && learningRate <= 1, + "Learning rate should be in range (0, 1]. Provided learning rate is " + s"$learningRate.") + } } @Experimental @@ -82,28 +93,17 @@ object BoostingStrategy { * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] * @return Configuration for boosting algorithm */ - def defaultParams(algo: Algo): BoostingStrategy = { - val treeStrategy = defaultWeakLearnerParams(algo) + def defaultParams(algo: String): BoostingStrategy = { + val treeStrategy = Strategy.defaultStrategy("Regression") + treeStrategy.maxDepth = 3 algo match { - case Classification => - new BoostingStrategy(algo, 100, LogLoss, weakLearnerParams = treeStrategy) - case Regression => - new BoostingStrategy(algo, 100, SquaredError, weakLearnerParams = treeStrategy) + case "Classification" => + new BoostingStrategy(Algo.withName(algo), 100, LogLoss, weakLearnerParams = treeStrategy) + case "Regression" => + new BoostingStrategy(Algo.withName(algo), 100, SquaredError, + weakLearnerParams = treeStrategy) case _ => throw new IllegalArgumentException(s"$algo is not supported by the boosting.") } } - - /** - * Returns default configuration for the weak learner (decision tree) algorithm - * @param algo Learning goal. Supported: - * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], - * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] - * @return Configuration for weak learner - */ - def defaultWeakLearnerParams(algo: Algo): Strategy = { - // Note: Regression tree used even for classification for GBT. - new Strategy(Regression, Variance, 3) - } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index d09295c507d67..b5b1f82177edc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -70,7 +70,7 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ */ @Experimental class Strategy ( - val algo: Algo, + @BeanProperty var algo: Algo, @BeanProperty var impurity: Impurity, @BeanProperty var maxDepth: Int, @BeanProperty var numClassesForClassification: Int = 2, @@ -85,17 +85,9 @@ class Strategy ( @BeanProperty var checkpointDir: Option[String] = None, @BeanProperty var checkpointInterval: Int = 10) extends Serializable { - if (algo == Classification) { - require(numClassesForClassification >= 2) - } - require(minInstancesPerNode >= 1, - s"DecisionTree Strategy requires minInstancesPerNode >= 1 but was given $minInstancesPerNode") - require(maxMemoryInMB <= 10240, - s"DecisionTree Strategy requires maxMemoryInMB <= 10240, but was given $maxMemoryInMB") - - val isMulticlassClassification = + def isMulticlassClassification = algo == Classification && numClassesForClassification > 2 - val isMulticlassWithCategoricalFeatures + def isMulticlassWithCategoricalFeatures = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) /** @@ -112,6 +104,23 @@ class Strategy ( categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) } + /** + * Sets Algorithm using a String. + */ + def setAlgo(algo: String): Unit = algo match { + case "Classification" => setAlgo(Classification) + case "Regression" => setAlgo(Regression) + } + + /** + * Sets categoricalFeaturesInfo using a Java Map. + */ + def setCategoricalFeaturesInfo( + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]): Unit = { + setCategoricalFeaturesInfo( + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) + } + /** * Check validity of parameters. * Throws exception if invalid. @@ -143,6 +152,26 @@ class Strategy ( s"DecisionTree Strategy given invalid categoricalFeaturesInfo setting:" + s" feature $feature has $arity categories. The number of categories should be >= 2.") } + require(minInstancesPerNode >= 1, + s"DecisionTree Strategy requires minInstancesPerNode >= 1 but was given $minInstancesPerNode") + require(maxMemoryInMB <= 10240, + s"DecisionTree Strategy requires maxMemoryInMB <= 10240, but was given $maxMemoryInMB") } +} + +@Experimental +object Strategy { + /** + * Construct a default set of parameters for [[org.apache.spark.mllib.tree.DecisionTree]] + * @param algo "Classification" or "Regression" + */ + def defaultStrategy(algo: String): Strategy = algo match { + case "Classification" => + new Strategy(algo = Classification, impurity = Gini, maxDepth = 10, + numClassesForClassification = 2) + case "Regression" => + new Strategy(algo = Regression, impurity = Variance, maxDepth = 10, + numClassesForClassification = 0) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala index 970fff82215e2..99a02eda60baf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala @@ -22,9 +22,8 @@ import org.scalatest.FunSuite import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} -import org.apache.spark.mllib.tree.impurity.{Variance, Gini} +import org.apache.spark.mllib.tree.impurity.Variance import org.apache.spark.mllib.tree.loss.{SquaredError, LogLoss} -import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel} import org.apache.spark.mllib.util.LocalSparkContext @@ -34,9 +33,8 @@ import org.apache.spark.mllib.util.LocalSparkContext class GradientBoostingSuite extends FunSuite with LocalSparkContext { test("Regression with continuous features: SquaredError") { - GradientBoostingSuite.testCombinations.foreach { - case (numEstimators, learningRate, subsamplingRate) => + case (numIterations, learningRate, subsamplingRate) => val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) val categoricalFeaturesInfo = Map.empty[Int, Int] @@ -48,11 +46,11 @@ class GradientBoostingSuite extends FunSuite with LocalSparkContext { val dt = DecisionTree.train(remappedInput, treeStrategy) - val boostingStrategy = new BoostingStrategy(Regression, numEstimators, SquaredError, - subsamplingRate, learningRate, 1, categoricalFeaturesInfo, treeStrategy) + val boostingStrategy = new BoostingStrategy(Regression, numIterations, SquaredError, + learningRate, 1, treeStrategy) val gbt = GradientBoosting.trainRegressor(rdd, boostingStrategy) - assert(gbt.weakHypotheses.size === numEstimators) + assert(gbt.weakHypotheses.size === numIterations) val gbtTree = gbt.weakHypotheses(0) EnsembleTestHelper.validateRegressor(gbt, arr, 0.02) @@ -63,9 +61,8 @@ class GradientBoostingSuite extends FunSuite with LocalSparkContext { } test("Regression with continuous features: Absolute Error") { - GradientBoostingSuite.testCombinations.foreach { - case (numEstimators, learningRate, subsamplingRate) => + case (numIterations, learningRate, subsamplingRate) => val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) val categoricalFeaturesInfo = Map.empty[Int, Int] @@ -77,11 +74,11 @@ class GradientBoostingSuite extends FunSuite with LocalSparkContext { val dt = DecisionTree.train(remappedInput, treeStrategy) - val boostingStrategy = new BoostingStrategy(Regression, numEstimators, SquaredError, - subsamplingRate, learningRate, 1, categoricalFeaturesInfo, treeStrategy) + val boostingStrategy = new BoostingStrategy(Regression, numIterations, SquaredError, + learningRate, numClassesForClassification = 2, treeStrategy) val gbt = GradientBoosting.trainRegressor(rdd, boostingStrategy) - assert(gbt.weakHypotheses.size === numEstimators) + assert(gbt.weakHypotheses.size === numIterations) val gbtTree = gbt.weakHypotheses(0) EnsembleTestHelper.validateRegressor(gbt, arr, 0.02) @@ -91,11 +88,9 @@ class GradientBoostingSuite extends FunSuite with LocalSparkContext { } } - test("Binary classification with continuous features: Log Loss") { - GradientBoostingSuite.testCombinations.foreach { - case (numEstimators, learningRate, subsamplingRate) => + case (numIterations, learningRate, subsamplingRate) => val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) val categoricalFeaturesInfo = Map.empty[Int, Int] @@ -107,11 +102,11 @@ class GradientBoostingSuite extends FunSuite with LocalSparkContext { val dt = DecisionTree.train(remappedInput, treeStrategy) - val boostingStrategy = new BoostingStrategy(Classification, numEstimators, LogLoss, - subsamplingRate, learningRate, 1, categoricalFeaturesInfo, treeStrategy) + val boostingStrategy = new BoostingStrategy(Classification, numIterations, LogLoss, + learningRate, numClassesForClassification = 2, treeStrategy) val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy) - assert(gbt.weakHypotheses.size === numEstimators) + assert(gbt.weakHypotheses.size === numIterations) val gbtTree = gbt.weakHypotheses(0) EnsembleTestHelper.validateClassifier(gbt, arr, 0.9) @@ -126,7 +121,6 @@ class GradientBoostingSuite extends FunSuite with LocalSparkContext { object GradientBoostingSuite { // Combinations for estimators, learning rates and subsamplingRate - val testCombinations - = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 1.0, 0.75), (10, 0.1, 0.75)) + val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 1.0, 0.75), (10, 0.1, 0.75)) } From 4c42986cc070d9c5c55c7bf8a2a67585967b1082 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 5 Nov 2014 14:38:43 -0800 Subject: [PATCH 1225/1492] [SPARK-4242] [Core] Add SASL to external shuffle service Does three things: (1) Adds SASL to ExternalShuffleClient, (2) puts SecurityManager in BlockManager's constructor, and (3) adds unit test. Author: Aaron Davidson Closes #3108 from aarondav/sasl-client and squashes the following commits: 48b622d [Aaron Davidson] Screw it, let's just get LimitedInputStream 3543b70 [Aaron Davidson] Back out of pom change due to unknown test issue? b58518a [Aaron Davidson] ByteStreams.limit() not available :( cbe451a [Aaron Davidson] Address comments 2bf2908 [Aaron Davidson] [SPARK-4242] [Core] Add SASL to external shuffle service --- LICENSE | 21 +++- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 12 +- .../BlockManagerReplicationSuite.scala | 4 +- .../spark/storage/BlockManagerSuite.scala | 4 +- network/common/pom.xml | 1 + .../buffer/FileSegmentManagedBuffer.java | 3 +- .../network/util/LimitedInputStream.java | 87 ++++++++++++++ network/shuffle/pom.xml | 1 + .../spark/network/sasl/SparkSaslClient.java | 1 - .../spark/network/sasl/SparkSaslServer.java | 9 +- .../shuffle/ExternalShuffleClient.java | 31 ++++- .../ExternalShuffleIntegrationSuite.java | 4 +- .../shuffle/ExternalShuffleSecuritySuite.java | 113 ++++++++++++++++++ .../streaming/ReceivedBlockHandlerSuite.scala | 2 +- 15 files changed, 272 insertions(+), 23 deletions(-) create mode 100644 network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java diff --git a/LICENSE b/LICENSE index f1732fb47afc0..3c667bf45059a 100644 --- a/LICENSE +++ b/LICENSE @@ -754,7 +754,7 @@ SUCH DAMAGE. ======================================================================== -For Timsort (core/src/main/java/org/apache/spark/util/collection/Sorter.java): +For Timsort (core/src/main/java/org/apache/spark/util/collection/TimSort.java): ======================================================================== Copyright (C) 2008 The Android Open Source Project @@ -771,6 +771,25 @@ See the License for the specific language governing permissions and limitations under the License. +======================================================================== +For LimitedInputStream + (network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java): +======================================================================== +Copyright (C) 2007 The Guava Authors + +Licensed 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. + + ======================================================================== BSD-style licenses ======================================================================== diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 45e9d7f243e96..e7454beddbfd0 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -287,7 +287,7 @@ object SparkEnv extends Logging { // NB: blockManager is not valid until initialize() is called later. val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, mapOutputTracker, shuffleManager, blockTransferService) + serializer, conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager) val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 655d16c65c8b5..a5fb87b9b2c51 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -72,7 +72,8 @@ private[spark] class BlockManager( val conf: SparkConf, mapOutputTracker: MapOutputTracker, shuffleManager: ShuffleManager, - blockTransferService: BlockTransferService) + blockTransferService: BlockTransferService, + securityManager: SecurityManager) extends BlockDataManager with Logging { val diskBlockManager = new DiskBlockManager(this, conf) @@ -115,7 +116,8 @@ private[spark] class BlockManager( // Client to read other executors' shuffle files. This is either an external service, or just the // standard BlockTranserService to directly connect to other Executors. private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { - new ExternalShuffleClient(SparkTransportConf.fromSparkConf(conf)) + new ExternalShuffleClient(SparkTransportConf.fromSparkConf(conf), securityManager, + securityManager.isAuthenticationEnabled()) } else { blockTransferService } @@ -166,9 +168,10 @@ private[spark] class BlockManager( conf: SparkConf, mapOutputTracker: MapOutputTracker, shuffleManager: ShuffleManager, - blockTransferService: BlockTransferService) = { + blockTransferService: BlockTransferService, + securityManager: SecurityManager) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, mapOutputTracker, shuffleManager, blockTransferService) + conf, mapOutputTracker, shuffleManager, blockTransferService, securityManager) } /** @@ -219,7 +222,6 @@ private[spark] class BlockManager( return } catch { case e: Exception if i < MAX_ATTEMPTS => - val attemptsRemaining = logError(s"Failed to connect to external shuffle server, will retry ${MAX_ATTEMPTS - i}}" + s" more times after waiting $SLEEP_TIME_SECS seconds...", e) Thread.sleep(SLEEP_TIME_SECS * 1000) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 1461fa69db90d..f63e772bf1e59 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -62,7 +62,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, - mapOutputTracker, shuffleManager, transfer) + mapOutputTracker, shuffleManager, transfer, securityMgr) store.initialize("app-id") allStores += store store @@ -263,7 +263,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd when(failableTransfer.hostName).thenReturn("some-hostname") when(failableTransfer.port).thenReturn(1000) val failableStore = new BlockManager("failable-store", actorSystem, master, serializer, - 10000, conf, mapOutputTracker, shuffleManager, failableTransfer) + 10000, conf, mapOutputTracker, shuffleManager, failableTransfer, securityMgr) failableStore.initialize("app-id") allStores += failableStore // so that this gets stopped after test assert(master.getPeers(store.blockManagerId).toSet === Set(failableStore.blockManagerId)) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 0782876c8e3c6..9529502bc8e10 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -74,7 +74,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) val manager = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, - mapOutputTracker, shuffleManager, transfer) + mapOutputTracker, shuffleManager, transfer, securityMgr) manager.initialize("app-id") manager } @@ -795,7 +795,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter // Use Java serializer so we can create an unserializable error. val transfer = new NioBlockTransferService(conf, securityMgr) store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, actorSystem, master, - new JavaSerializer(conf), 1200, conf, mapOutputTracker, shuffleManager, transfer) + new JavaSerializer(conf), 1200, conf, mapOutputTracker, shuffleManager, transfer, securityMgr) // The put should fail since a1 is not serializable. class UnserializableClass diff --git a/network/common/pom.xml b/network/common/pom.xml index ea887148d98ba..6144548a8f998 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -50,6 +50,7 @@ com.google.guava guava + 11.0.2 provided diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 89ed79bc63903..5fa1527ddff92 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -30,6 +30,7 @@ import io.netty.channel.DefaultFileRegion; import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.LimitedInputStream; /** * A {@link ManagedBuffer} backed by a segment in a file. @@ -101,7 +102,7 @@ public InputStream createInputStream() throws IOException { try { is = new FileInputStream(file); ByteStreams.skipFully(is, offset); - return ByteStreams.limit(is, length); + return new LimitedInputStream(is, length); } catch (IOException e) { try { if (is != null) { diff --git a/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java b/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java new file mode 100644 index 0000000000000..63ca43c046525 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +import com.google.common.base.Preconditions; + +/** + * Wraps a {@link InputStream}, limiting the number of bytes which can be read. + * + * This code is from Guava's 14.0 source code, because there is no compatible way to + * use this functionality in both a Guava 11 environment and a Guava >14 environment. + */ +public final class LimitedInputStream extends FilterInputStream { + private long left; + private long mark = -1; + + public LimitedInputStream(InputStream in, long limit) { + super(in); + Preconditions.checkNotNull(in); + Preconditions.checkArgument(limit >= 0, "limit must be non-negative"); + left = limit; + } + @Override public int available() throws IOException { + return (int) Math.min(in.available(), left); + } + // it's okay to mark even if mark isn't supported, as reset won't work + @Override public synchronized void mark(int readLimit) { + in.mark(readLimit); + mark = left; + } + @Override public int read() throws IOException { + if (left == 0) { + return -1; + } + int result = in.read(); + if (result != -1) { + --left; + } + return result; + } + @Override public int read(byte[] b, int off, int len) throws IOException { + if (left == 0) { + return -1; + } + len = (int) Math.min(len, left); + int result = in.read(b, off, len); + if (result != -1) { + left -= result; + } + return result; + } + @Override public synchronized void reset() throws IOException { + if (!in.markSupported()) { + throw new IOException("Mark not supported"); + } + if (mark == -1) { + throw new IOException("Mark not set"); + } + in.reset(); + left = mark; + } + @Override public long skip(long n) throws IOException { + n = Math.min(n, left); + long skipped = in.skip(n); + left -= skipped; + return skipped; + } +} diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index d271704d98a7a..fe5681d463499 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -51,6 +51,7 @@ com.google.guava guava + 11.0.2 provided diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java index 72ba737b998bc..9abad1f30a259 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslClient.java @@ -126,7 +126,6 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback logger.trace("SASL client callback: setting realm"); RealmCallback rc = (RealmCallback) callback; rc.setText(rc.getDefaultText()); - logger.info("Realm callback"); } else if (callback instanceof RealmChoiceCallback) { // ignore (?) } else { diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java index 2c0ce40c75e80..e87b17ead1e1a 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SparkSaslServer.java @@ -34,7 +34,8 @@ import com.google.common.base.Preconditions; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; -import com.google.common.io.BaseEncoding; +import io.netty.buffer.Unpooled; +import io.netty.handler.codec.base64.Base64; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -159,12 +160,14 @@ public void handle(Callback[] callbacks) throws IOException, UnsupportedCallback /* Encode a byte[] identifier as a Base64-encoded string. */ public static String encodeIdentifier(String identifier) { Preconditions.checkNotNull(identifier, "User cannot be null if SASL is enabled"); - return BaseEncoding.base64().encode(identifier.getBytes(Charsets.UTF_8)); + return Base64.encode(Unpooled.wrappedBuffer(identifier.getBytes(Charsets.UTF_8))) + .toString(Charsets.UTF_8); } /** Encode a password as a base64-encoded char[] array. */ public static char[] encodePassword(String password) { Preconditions.checkNotNull(password, "Password cannot be null if SASL is enabled"); - return BaseEncoding.base64().encode(password.getBytes(Charsets.UTF_8)).toCharArray(); + return Base64.encode(Unpooled.wrappedBuffer(password.getBytes(Charsets.UTF_8))) + .toString(Charsets.UTF_8).toCharArray(); } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index b0b19ba67bddc..3aa95d00f6b20 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -17,12 +17,18 @@ package org.apache.spark.network.shuffle; +import java.util.List; + +import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.TransportContext; import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientBootstrap; import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.sasl.SaslClientBootstrap; +import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.shuffle.ExternalShuffleMessages.RegisterExecutor; import org.apache.spark.network.util.JavaUtils; @@ -37,18 +43,35 @@ public class ExternalShuffleClient extends ShuffleClient { private final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class); - private final TransportClientFactory clientFactory; + private final TransportConf conf; + private final boolean saslEnabled; + private final SecretKeyHolder secretKeyHolder; + private TransportClientFactory clientFactory; private String appId; - public ExternalShuffleClient(TransportConf conf) { - TransportContext context = new TransportContext(conf, new NoOpRpcHandler()); - this.clientFactory = context.createClientFactory(); + /** + * Creates an external shuffle client, with SASL optionally enabled. If SASL is not enabled, + * then secretKeyHolder may be null. + */ + public ExternalShuffleClient( + TransportConf conf, + SecretKeyHolder secretKeyHolder, + boolean saslEnabled) { + this.conf = conf; + this.secretKeyHolder = secretKeyHolder; + this.saslEnabled = saslEnabled; } @Override public void init(String appId) { this.appId = appId; + TransportContext context = new TransportContext(conf, new NoOpRpcHandler()); + List bootstraps = Lists.newArrayList(); + if (saslEnabled) { + bootstraps.add(new SaslClientBootstrap(conf, appId, secretKeyHolder)); + } + clientFactory = context.createClientFactory(bootstraps); } @Override diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index bc101f53844d5..71e017b9e4e74 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -135,7 +135,7 @@ private FetchResult fetchBlocks(String execId, String[] blockIds, int port) thro final Semaphore requestsRemaining = new Semaphore(0); - ExternalShuffleClient client = new ExternalShuffleClient(conf); + ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false); client.init(APP_ID); client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, new BlockFetchingListener() { @@ -267,7 +267,7 @@ public void testFetchNoServer() throws Exception { } private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) { - ExternalShuffleClient client = new ExternalShuffleClient(conf); + ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false); client.init(APP_ID); client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), executorId, executorInfo); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java new file mode 100644 index 0000000000000..4c18fcdfbcd88 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class ExternalShuffleSecuritySuite { + + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + TransportServer server; + + @Before + public void beforeEach() { + RpcHandler handler = new SaslRpcHandler(new ExternalShuffleBlockHandler(), + new TestSecretKeyHolder("my-app-id", "secret")); + TransportContext context = new TransportContext(conf, handler); + this.server = context.createServer(); + } + + @After + public void afterEach() { + if (server != null) { + server.close(); + server = null; + } + } + + @Test + public void testValid() { + validate("my-app-id", "secret"); + } + + @Test + public void testBadAppId() { + try { + validate("wrong-app-id", "secret"); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("Wrong appId!")); + } + } + + @Test + public void testBadSecret() { + try { + validate("my-app-id", "bad-secret"); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("Mismatched response")); + } + } + + /** Creates an ExternalShuffleClient and attempts to register with the server. */ + private void validate(String appId, String secretKey) { + ExternalShuffleClient client = + new ExternalShuffleClient(conf, new TestSecretKeyHolder(appId, secretKey), true); + client.init(appId); + // Registration either succeeds or throws an exception. + client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), "exec0", + new ExecutorShuffleInfo(new String[0], 0, "")); + client.close(); + } + + /** Provides a secret key holder which always returns the given secret key, for a single appId. */ + static class TestSecretKeyHolder implements SecretKeyHolder { + private final String appId; + private final String secretKey; + + TestSecretKeyHolder(String appId, String secretKey) { + this.appId = appId; + this.secretKey = secretKey; + } + + @Override + public String getSaslUser(String appId) { + return "user"; + } + + @Override + public String getSecretKey(String appId) { + if (!appId.equals(this.appId)) { + throw new IllegalArgumentException("Wrong appId!"); + } + return secretKey; + } + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index 0f27f55fec4f3..9efe15d01ed0c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -73,7 +73,7 @@ class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matche blockManager = new BlockManager("bm", actorSystem, blockManagerMaster, serializer, blockManagerSize, conf, mapOutputTracker, shuffleManager, - new NioBlockTransferService(conf, securityMgr)) + new NioBlockTransferService(conf, securityMgr), securityMgr) blockManager.initialize("app-id") tempDirectory = Files.createTempDir() From a46497eecc50f854c5c5701dc2b8a2468b76c085 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 5 Nov 2014 15:30:31 -0800 Subject: [PATCH 1226/1492] [SPARK-3984] [SPARK-3983] Fix incorrect scheduler delay and display task deserialization time in UI This commit fixes the scheduler delay in the UI (which previously included things that are not scheduler delay, like time to deserialize the task and serialize the result), and also adds information about time to deserialize tasks to the optional additional metrics. Time to deserialize the task can be large relative to task time for short jobs, and understanding when it is high can help developers realize that they should try to reduce closure size (e.g, by including less data in the task description). cc shivaram etrain Author: Kay Ousterhout Closes #2832 from kayousterhout/SPARK-3983 and squashes the following commits: 0c1398e [Kay Ousterhout] Fixed ordering 531575d [Kay Ousterhout] Removed executor launch time 1f13afe [Kay Ousterhout] Minor spacing fixes 335be4b [Kay Ousterhout] Made metrics hideable 5bc3cba [Kay Ousterhout] [SPARK-3984] [SPARK-3983] Improve UI task metrics. --- .../org/apache/spark/executor/Executor.scala | 4 +-- .../scala/org/apache/spark/ui/ToolTips.scala | 3 ++ .../org/apache/spark/ui/jobs/StagePage.scala | 31 ++++++++++++++++++- .../spark/ui/jobs/TaskDetailsClassNames.scala | 1 + 4 files changed, 36 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index abc1dd0be6237..96114571d6c77 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -161,7 +161,7 @@ private[spark] class Executor( } override def run() { - val startTime = System.currentTimeMillis() + val deserializeStartTime = System.currentTimeMillis() Thread.currentThread.setContextClassLoader(replClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") @@ -206,7 +206,7 @@ private[spark] class Executor( val afterSerialization = System.currentTimeMillis() for (m <- task.metrics) { - m.executorDeserializeTime = taskStart - startTime + m.executorDeserializeTime = taskStart - deserializeStartTime m.executorRunTime = taskFinish - taskStart m.jvmGCTime = gcTime - startGCTime m.resultSerializationTime = afterSerialization - beforeSerialization diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index f02904df31fcf..51dc08f668a43 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -24,6 +24,9 @@ private[spark] object ToolTips { scheduler delay is large, consider decreasing the size of tasks or decreasing the size of task results.""" + val TASK_DESERIALIZATION_TIME = + """Time spent deserializating the task closure on the executor.""" + val INPUT = "Bytes read from Hadoop or from Spark storage." val SHUFFLE_WRITE = "Bytes written to disk in order to be read by a shuffle in a future stage." diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 7cc03b7d333df..63ed5fc4949c2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -112,6 +112,13 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { Scheduler Delay +
  • + + + Task Deserialization Time + +
  • @@ -147,6 +154,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), + ("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME), ("GC Time", TaskDetailsClassNames.GC_TIME), ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ @@ -179,6 +187,17 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { } } + val deserializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.executorDeserializeTime.toDouble + } + val deserializationQuantiles = + + + Task Deserialization Time + + +: getFormattedTimeQuantiles(deserializationTimes) + val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } @@ -266,6 +285,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val listings: Seq[Seq[Node]] = Seq( {serviceQuantiles}, {schedulerDelayQuantiles}, + + {deserializationQuantiles} + {gcQuantiles}, {serializationQuantiles} @@ -314,6 +336,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") val schedulerDelay = metrics.map(getSchedulerDelay(info, _)).getOrElse(0L) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) + val taskDeserializationTime = metrics.map(_.executorDeserializeTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = info.gettingResultTime @@ -367,6 +390,10 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { class={TaskDetailsClassNames.SCHEDULER_DELAY}> {UIUtils.formatDuration(schedulerDelay.toLong)} + + {UIUtils.formatDuration(taskDeserializationTime.toLong)} + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} @@ -424,6 +451,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { (info.finishTime - info.launchTime) } } - totalExecutionTime - metrics.executorRunTime + val executorOverhead = (metrics.executorDeserializeTime + + metrics.resultSerializationTime) + totalExecutionTime - metrics.executorRunTime - executorOverhead } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala index 23d672cabda07..eb371bd0ea7ed 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala @@ -24,6 +24,7 @@ package org.apache.spark.ui.jobs private object TaskDetailsClassNames { val SCHEDULER_DELAY = "scheduler_delay" val GC_TIME = "gc_time" + val TASK_DESERIALIZATION_TIME = "deserialization_time" val RESULT_SERIALIZATION_TIME = "serialization_time" val GETTING_RESULT_TIME = "getting_result_time" } From f37817b18a479839b2e6118cc1cbd1059a94db52 Mon Sep 17 00:00:00 2001 From: industrial-sloth Date: Wed, 5 Nov 2014 15:38:48 -0800 Subject: [PATCH 1227/1492] SPARK-4222 [CORE] use readFully in FixedLengthBinaryRecordReader replaces the existing read() call with readFully(). Author: industrial-sloth Closes #3093 from industrial-sloth/branch-1.2-fixedLenRecRdr and squashes the following commits: a245c8a [industrial-sloth] use readFully in FixedLengthBinaryRecordReader (cherry picked from commit 6844e7a8219ac78790a422ffd5054924e7d2bea1) Signed-off-by: Matei Zaharia --- .../org/apache/spark/input/FixedLengthBinaryRecordReader.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala index 5164a74bec4e9..36a1e5d475f46 100644 --- a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -115,7 +115,7 @@ private[spark] class FixedLengthBinaryRecordReader if (currentPosition < splitEnd) { // setup a buffer to store the record val buffer = recordValue.getBytes - fileInputStream.read(buffer, 0, recordLength) + fileInputStream.readFully(buffer) // update our current position currentPosition = currentPosition + recordLength // return true From 61a5cced049a8056292ba94f23fa7bd040f50685 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 5 Nov 2014 15:42:05 -0800 Subject: [PATCH 1228/1492] [SPARK-3797] Run external shuffle service in Yarn NM This creates a new module `network/yarn` that depends on `network/shuffle` recently created in #3001. This PR introduces a custom Yarn auxiliary service that runs the external shuffle service. As of the changes here this shuffle service is required for using dynamic allocation with Spark. This is still WIP mainly because it doesn't handle security yet. I have tested this on a stable Yarn cluster. Author: Andrew Or Closes #3082 from andrewor14/yarn-shuffle-service and squashes the following commits: ef3ddae [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service 0ee67a2 [Andrew Or] Minor wording suggestions 1c66046 [Andrew Or] Remove unused provided dependencies 0eb6233 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service 6489db5 [Andrew Or] Try catch at the right places 7b71d8f [Andrew Or] Add detailed java docs + reword a few comments d1124e4 [Andrew Or] Add security to shuffle service (INCOMPLETE) 5f8a96f [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service 9b6e058 [Andrew Or] Address various feedback f48b20c [Andrew Or] Fix tests again f39daa6 [Andrew Or] Do not make network-yarn an assembly module 761f58a [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-shuffle-service 15a5b37 [Andrew Or] Fix build for Hadoop 1.x baff916 [Andrew Or] Fix tests 5bf9b7e [Andrew Or] Address a few minor comments 5b419b8 [Andrew Or] Add missing license header 804e7ff [Andrew Or] Include the Yarn shuffle service jar in the distribution cd076a4 [Andrew Or] Require external shuffle service for dynamic allocation ea764e0 [Andrew Or] Connect to Yarn shuffle service only if it's enabled 1bf5109 [Andrew Or] Use the shuffle service port specified through hadoop config b4b1f0c [Andrew Or] 4 tabs -> 2 tabs 43dcb96 [Andrew Or] First cut integration of shuffle service with Yarn aux service b54a0c4 [Andrew Or] Initial skeleton for Yarn shuffle service --- .../spark/ExecutorAllocationManager.scala | 37 +++- .../apache/spark/storage/BlockManager.scala | 8 +- .../scala/org/apache/spark/util/Utils.scala | 16 ++ make-distribution.sh | 3 + .../network/sasl/ShuffleSecretManager.java | 117 ++++++++++++ network/yarn/pom.xml | 58 ++++++ .../network/yarn/YarnShuffleService.java | 176 ++++++++++++++++++ .../yarn/util/HadoopConfigProvider.java | 42 +++++ pom.xml | 2 + project/SparkBuild.scala | 8 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 16 ++ .../spark/deploy/yarn/ExecutorRunnable.scala | 16 ++ 12 files changed, 483 insertions(+), 16 deletions(-) create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java create mode 100644 network/yarn/pom.xml create mode 100644 network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java create mode 100644 network/yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index c11f1db0064fd..ef93009a074e7 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -66,7 +66,6 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging // Lower and upper bounds on the number of executors. These are required. private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1) private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1) - verifyBounds() // How long there must be backlogged tasks for before an addition is triggered private val schedulerBacklogTimeout = conf.getLong( @@ -77,9 +76,14 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) // How long an executor must be idle for before it is removed - private val removeThresholdSeconds = conf.getLong( + private val executorIdleTimeout = conf.getLong( "spark.dynamicAllocation.executorIdleTimeout", 600) + // During testing, the methods to actually kill and add executors are mocked out + private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) + + validateSettings() + // Number of executors to add in the next round private var numExecutorsToAdd = 1 @@ -103,17 +107,14 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging // Polling loop interval (ms) private val intervalMillis: Long = 100 - // Whether we are testing this class. This should only be used internally. - private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) - // Clock used to schedule when executors should be added and removed private var clock: Clock = new RealClock /** - * Verify that the lower and upper bounds on the number of executors are valid. + * Verify that the settings specified through the config are valid. * If not, throw an appropriate exception. */ - private def verifyBounds(): Unit = { + private def validateSettings(): Unit = { if (minNumExecutors < 0 || maxNumExecutors < 0) { throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!") } @@ -124,6 +125,22 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") } + if (schedulerBacklogTimeout <= 0) { + throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") + } + if (sustainedSchedulerBacklogTimeout <= 0) { + throw new SparkException( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") + } + if (executorIdleTimeout <= 0) { + throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!") + } + // Require external shuffle service for dynamic allocation + // Otherwise, we may lose shuffle files when killing executors + if (!conf.getBoolean("spark.shuffle.service.enabled", false) && !testing) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } } /** @@ -254,7 +271,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging val removeRequestAcknowledged = testing || sc.killExecutor(executorId) if (removeRequestAcknowledged) { logInfo(s"Removing executor $executorId because it has been idle for " + - s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})") + s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})") executorsPendingToRemove.add(executorId) true } else { @@ -329,8 +346,8 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging private def onExecutorIdle(executorId: String): Unit = synchronized { if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)") - removeTimes(executorId) = clock.getTimeMillis + removeThresholdSeconds * 1000 + s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a5fb87b9b2c51..e48d7772d6ee9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -40,7 +40,6 @@ import org.apache.spark.network.util.{ConfigProvider, TransportConf} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.shuffle.hash.HashShuffleManager -import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.util._ private[spark] sealed trait BlockValues @@ -97,7 +96,12 @@ private[spark] class BlockManager( private[spark] val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) - private val externalShuffleServicePort = conf.getInt("spark.shuffle.service.port", 7337) + + // Port used by the external shuffle service. In Yarn mode, this may be already be + // set through the Hadoop configuration as the server is launched in the Yarn NM. + private val externalShuffleServicePort = + Utils.getSparkOrYarnConfig(conf, "spark.shuffle.service.port", "7337").toInt + // Check that we're not using external shuffle service with consolidated shuffle files. if (externalShuffleServiceEnabled && conf.getBoolean("spark.shuffle.consolidateFiles", false) 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 6ab94af9f3739..7caf6bcf94ef3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -45,6 +45,7 @@ import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -1780,6 +1781,21 @@ private[spark] object Utils extends Logging { val manifest = new JarManifest(manifestUrl.openStream()) manifest.getMainAttributes.getValue(Name.IMPLEMENTATION_VERSION) }.getOrElse("Unknown") + + /** + * Return the value of a config either through the SparkConf or the Hadoop configuration + * if this is Yarn mode. In the latter case, this defaults to the value set through SparkConf + * if the key is not set in the Hadoop configuration. + */ + def getSparkOrYarnConfig(conf: SparkConf, key: String, default: String): String = { + val sparkValue = conf.get(key, default) + if (SparkHadoopUtil.get.isYarnMode) { + SparkHadoopUtil.get.newConfiguration(conf).get(key, sparkValue) + } else { + sparkValue + } + } + } /** diff --git a/make-distribution.sh b/make-distribution.sh index 0bc839e1dbe4d..fac7f7e284be4 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -181,6 +181,9 @@ echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DI # Copy jars cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" +cp "$FWDIR"/network/yarn/target/scala*/spark-network-yarn*.jar "$DISTDIR/lib/" +cp "$FWDIR"/network/yarn/target/scala*/spark-network-shuffle*.jar "$DISTDIR/lib/" +cp "$FWDIR"/network/yarn/target/scala*/spark-network-common*.jar "$DISTDIR/lib/" # Copy example sources (needed for python and SQL) mkdir -p "$DISTDIR/examples/src/main" diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java new file mode 100644 index 0000000000000..e66c4af0f1ebd --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.sasl; + +import java.lang.Override; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.util.concurrent.ConcurrentHashMap; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.sasl.SecretKeyHolder; + +/** + * A class that manages shuffle secret used by the external shuffle service. + */ +public class ShuffleSecretManager implements SecretKeyHolder { + private final Logger logger = LoggerFactory.getLogger(ShuffleSecretManager.class); + private final ConcurrentHashMap shuffleSecretMap; + + private static final Charset UTF8_CHARSET = Charset.forName("UTF-8"); + + // Spark user used for authenticating SASL connections + // Note that this must match the value in org.apache.spark.SecurityManager + private static final String SPARK_SASL_USER = "sparkSaslUser"; + + /** + * Convert the given string to a byte buffer. The resulting buffer can be converted back to + * the same string through {@link #bytesToString(ByteBuffer)}. This is used if the external + * shuffle service represents shuffle secrets as bytes buffers instead of strings. + */ + public static ByteBuffer stringToBytes(String s) { + return ByteBuffer.wrap(s.getBytes(UTF8_CHARSET)); + } + + /** + * Convert the given byte buffer to a string. The resulting string can be converted back to + * the same byte buffer through {@link #stringToBytes(String)}. This is used if the external + * shuffle service represents shuffle secrets as bytes buffers instead of strings. + */ + public static String bytesToString(ByteBuffer b) { + return new String(b.array(), UTF8_CHARSET); + } + + public ShuffleSecretManager() { + shuffleSecretMap = new ConcurrentHashMap(); + } + + /** + * Register an application with its secret. + * Executors need to first authenticate themselves with the same secret before + * fetching shuffle files written by other executors in this application. + */ + public void registerApp(String appId, String shuffleSecret) { + if (!shuffleSecretMap.contains(appId)) { + shuffleSecretMap.put(appId, shuffleSecret); + logger.info("Registered shuffle secret for application {}", appId); + } else { + logger.debug("Application {} already registered", appId); + } + } + + /** + * Register an application with its secret specified as a byte buffer. + */ + public void registerApp(String appId, ByteBuffer shuffleSecret) { + registerApp(appId, bytesToString(shuffleSecret)); + } + + /** + * Unregister an application along with its secret. + * This is called when the application terminates. + */ + public void unregisterApp(String appId) { + if (shuffleSecretMap.contains(appId)) { + shuffleSecretMap.remove(appId); + logger.info("Unregistered shuffle secret for application {}", appId); + } else { + logger.warn("Attempted to unregister application {} when it is not registered", appId); + } + } + + /** + * Return the Spark user for authenticating SASL connections. + */ + @Override + public String getSaslUser(String appId) { + return SPARK_SASL_USER; + } + + /** + * Return the secret key registered with the given application. + * This key is used to authenticate the executors before they can fetch shuffle files + * written by this application from the external shuffle service. If the specified + * application is not registered, return null. + */ + @Override + public String getSecretKey(String appId) { + return shuffleSecretMap.get(appId); + } +} diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml new file mode 100644 index 0000000000000..e60d8c1f7876c --- /dev/null +++ b/network/yarn/pom.xml @@ -0,0 +1,58 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.2.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-network-yarn_2.10 + jar + Spark Project Yarn Shuffle Service Code + http://spark.apache.org/ + + network-yarn + + + + + + org.apache.spark + spark-network-shuffle_2.10 + ${project.version} + + + + + org.apache.hadoop + hadoop-client + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java new file mode 100644 index 0000000000000..bb0b8f7e6cba6 --- /dev/null +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -0,0 +1,176 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.yarn; + +import java.lang.Override; +import java.nio.ByteBuffer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.server.api.AuxiliaryService; +import org.apache.hadoop.yarn.server.api.ApplicationInitializationContext; +import org.apache.hadoop.yarn.server.api.ApplicationTerminationContext; +import org.apache.hadoop.yarn.server.api.ContainerInitializationContext; +import org.apache.hadoop.yarn.server.api.ContainerTerminationContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.sasl.SaslRpcHandler; +import org.apache.spark.network.sasl.ShuffleSecretManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler; +import org.apache.spark.network.util.TransportConf; +import org.apache.spark.network.yarn.util.HadoopConfigProvider; + +/** + * An external shuffle service used by Spark on Yarn. + * + * This is intended to be a long-running auxiliary service that runs in the NodeManager process. + * A Spark application may connect to this service by setting `spark.shuffle.service.enabled`. + * The application also automatically derives the service port through `spark.shuffle.service.port` + * specified in the Yarn configuration. This is so that both the clients and the server agree on + * the same port to communicate on. + * + * The service also optionally supports authentication. This ensures that executors from one + * application cannot read the shuffle files written by those from another. This feature can be + * enabled by setting `spark.authenticate` in the Yarn configuration before starting the NM. + * Note that the Spark application must also set `spark.authenticate` manually and, unlike in + * the case of the service port, will not inherit this setting from the Yarn configuration. This + * is because an application running on the same Yarn cluster may choose to not use the external + * shuffle service, in which case its setting of `spark.authenticate` should be independent of + * the service's. + */ +public class YarnShuffleService extends AuxiliaryService { + private final Logger logger = LoggerFactory.getLogger(YarnShuffleService.class); + + // Port on which the shuffle server listens for fetch requests + private static final String SPARK_SHUFFLE_SERVICE_PORT_KEY = "spark.shuffle.service.port"; + private static final int DEFAULT_SPARK_SHUFFLE_SERVICE_PORT = 7337; + + // Whether the shuffle server should authenticate fetch requests + private static final String SPARK_AUTHENTICATE_KEY = "spark.authenticate"; + private static final boolean DEFAULT_SPARK_AUTHENTICATE = false; + + // An entity that manages the shuffle secret per application + // This is used only if authentication is enabled + private ShuffleSecretManager secretManager; + + // The actual server that serves shuffle files + private TransportServer shuffleServer = null; + + public YarnShuffleService() { + super("spark_shuffle"); + logger.info("Initializing YARN shuffle service for Spark"); + } + + /** + * Return whether authentication is enabled as specified by the configuration. + * If so, fetch requests will fail unless the appropriate authentication secret + * for the application is provided. + */ + private boolean isAuthenticationEnabled() { + return secretManager != null; + } + + /** + * Start the shuffle server with the given configuration. + */ + @Override + protected void serviceInit(Configuration conf) { + // If authentication is enabled, set up the shuffle server to use a + // special RPC handler that filters out unauthenticated fetch requests + boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); + RpcHandler rpcHandler = new ExternalShuffleBlockHandler(); + if (authEnabled) { + secretManager = new ShuffleSecretManager(); + rpcHandler = new SaslRpcHandler(rpcHandler, secretManager); + } + + int port = conf.getInt( + SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); + TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); + TransportContext transportContext = new TransportContext(transportConf, rpcHandler); + shuffleServer = transportContext.createServer(port); + String authEnabledString = authEnabled ? "enabled" : "not enabled"; + logger.info("Started YARN shuffle service for Spark on port {}. " + + "Authentication is {}.", port, authEnabledString); + } + + @Override + public void initializeApplication(ApplicationInitializationContext context) { + String appId = context.getApplicationId().toString(); + try { + ByteBuffer shuffleSecret = context.getApplicationDataForService(); + logger.info("Initializing application {}", appId); + if (isAuthenticationEnabled()) { + secretManager.registerApp(appId, shuffleSecret); + } + } catch (Exception e) { + logger.error("Exception when initializing application {}", appId, e); + } + } + + @Override + public void stopApplication(ApplicationTerminationContext context) { + String appId = context.getApplicationId().toString(); + try { + logger.info("Stopping application {}", appId); + if (isAuthenticationEnabled()) { + secretManager.unregisterApp(appId); + } + } catch (Exception e) { + logger.error("Exception when stopping application {}", appId, e); + } + } + + @Override + public void initializeContainer(ContainerInitializationContext context) { + ContainerId containerId = context.getContainerId(); + logger.info("Initializing container {}", containerId); + } + + @Override + public void stopContainer(ContainerTerminationContext context) { + ContainerId containerId = context.getContainerId(); + logger.info("Stopping container {}", containerId); + } + + /** + * Close the shuffle server to clean up any associated state. + */ + @Override + protected void serviceStop() { + try { + if (shuffleServer != null) { + shuffleServer.close(); + } + } catch (Exception e) { + logger.error("Exception when stopping service", e); + } + } + + // Not currently used + @Override + public ByteBuffer getMetaData() { + return ByteBuffer.allocate(0); + } + +} diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java new file mode 100644 index 0000000000000..884861752e80d --- /dev/null +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/util/HadoopConfigProvider.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.yarn.util; + +import java.util.NoSuchElementException; + +import org.apache.hadoop.conf.Configuration; + +import org.apache.spark.network.util.ConfigProvider; + +/** Use the Hadoop configuration to obtain config values. */ +public class HadoopConfigProvider extends ConfigProvider { + private final Configuration conf; + + public HadoopConfigProvider(Configuration conf) { + this.conf = conf; + } + + @Override + public String get(String name) { + String value = conf.get(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } +} diff --git a/pom.xml b/pom.xml index eb613531b8a5f..88ef67c515b3a 100644 --- a/pom.xml +++ b/pom.xml @@ -1229,6 +1229,7 @@ yarn-alpha yarn + network/yarn @@ -1236,6 +1237,7 @@ yarn yarn + network/yarn diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 33618f5401768..657e4b4432775 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -38,9 +38,9 @@ object BuildCommons { "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) - val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = - Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl", "kinesis-asl") - .map(ProjectRef(buildLocation, _)) + val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, networkYarn, java8Tests, + sparkGangliaLgpl, sparkKinesisAsl) = Seq("yarn", "yarn-stable", "yarn-alpha", "network-yarn", + "java8-tests", "ganglia-lgpl", "kinesis-asl").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(assembly, examples) = Seq("assembly", "examples") .map(ProjectRef(buildLocation, _)) @@ -143,7 +143,7 @@ object SparkBuild extends PomBuild { // TODO: Add Sql to mima checks allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - streamingFlumeSink, networkCommon, networkShuffle).contains(x)).foreach { + streamingFlumeSink, networkCommon, networkShuffle, networkYarn).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 7ee4b5c842df1..5f47c79cabaee 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.network.sasl.ShuffleSecretManager @deprecated("use yarn/stable", "1.2.0") class ExecutorRunnable( @@ -90,6 +91,21 @@ class ExecutorRunnable( ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) + // If external shuffle service is enabled, register with the Yarn shuffle service already + // started on the NodeManager and, if authentication is enabled, provide it with our secret + // key for fetching shuffle files later + if (sparkConf.getBoolean("spark.shuffle.service.enabled", false)) { + val secretString = securityMgr.getSecretKey() + val secretBytes = + if (secretString != null) { + ShuffleSecretManager.stringToBytes(secretString) + } else { + // Authentication is not enabled, so just provide dummy metadata + ByteBuffer.allocate(0) + } + ctx.setServiceData(Map[String, ByteBuffer]("spark_shuffle" -> secretBytes)) + } + // Send the start request to the ContainerManager val startReq = Records.newRecord(classOf[StartContainerRequest]) .asInstanceOf[StartContainerRequest] diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 0b5a92d87d722..18f48b4b6caf6 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -36,6 +36,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} import org.apache.spark.{SecurityManager, SparkConf, Logging} +import org.apache.spark.network.sasl.ShuffleSecretManager class ExecutorRunnable( @@ -89,6 +90,21 @@ class ExecutorRunnable( ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) + // If external shuffle service is enabled, register with the Yarn shuffle service already + // started on the NodeManager and, if authentication is enabled, provide it with our secret + // key for fetching shuffle files later + if (sparkConf.getBoolean("spark.shuffle.service.enabled", false)) { + val secretString = securityMgr.getSecretKey() + val secretBytes = + if (secretString != null) { + ShuffleSecretManager.stringToBytes(secretString) + } else { + // Authentication is not enabled, so just provide dummy metadata + ByteBuffer.allocate(0) + } + ctx.setServiceData(Map[String, ByteBuffer]("spark_shuffle" -> secretBytes)) + } + // Send the start request to the ContainerManager nmClient.startContainer(container, ctx) } From 868cd4c3ca11e6ecc4425b972d9a20c360b52425 Mon Sep 17 00:00:00 2001 From: "jay@apache.org" Date: Wed, 5 Nov 2014 15:45:34 -0800 Subject: [PATCH 1229/1492] SPARK-4040. Update documentation to exemplify use of local (n) value, fo... This is a minor docs update which helps to clarify the way local[n] is used for streaming apps. Author: jay@apache.org Closes #2964 from jayunit100/SPARK-4040 and squashes the following commits: 35b5a5e [jay@apache.org] SPARK-4040: Update documentation to exemplify use of local (n) value. --- docs/configuration.md | 10 ++++++++-- docs/streaming-programming-guide.md | 14 +++++++++----- 2 files changed, 17 insertions(+), 7 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 685101ea5c9c9..0f9eb81f6e993 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -21,16 +21,22 @@ application. These properties can be set directly on a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) passed to your `SparkContext`. `SparkConf` allows you to configure some of the common properties (e.g. master URL and application name), as well as arbitrary key-value pairs through the -`set()` method. For example, we could initialize an application as follows: +`set()` method. For example, we could initialize an application with two threads as follows: + +Note that we run with local[2], meaning two threads - which represents "minimal" parallelism, +which can help detect bugs that only exist when we run in a distributed context. {% highlight scala %} val conf = new SparkConf() - .setMaster("local") + .setMaster("local[2]") .setAppName("CountingSheep") .set("spark.executor.memory", "1g") val sc = new SparkContext(conf) {% endhighlight %} +Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually +require one to prevent any sort of starvation issues. + ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For instance, if you'd like to run the same application with different masters or different diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 8bbba88b31978..44a1f3ad7560b 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -68,7 +68,9 @@ import org.apache.spark._ import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ -// Create a local StreamingContext with two working thread and batch interval of 1 second +// Create a local StreamingContext with two working thread and batch interval of 1 second. +// The master requires 2 cores to prevent from a starvation scenario. + val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") val ssc = new StreamingContext(conf, Seconds(1)) {% endhighlight %} @@ -586,11 +588,13 @@ Every input DStream (except file stream) is associated with a single [Receiver]( A receiver is run within a Spark worker/executor as a long-running task, hence it occupies one of the cores allocated to the Spark Streaming application. Hence, it is important to remember that Spark Streaming application needs to be allocated enough cores to process the received data, as well as, to run the receiver(s). Therefore, few important points to remember are: -##### Points to remember: +##### Points to remember {:.no_toc} -- If the number of cores allocated to the application is less than or equal to the number of input DStreams / receivers, then the system will receive data, but not be able to process them. -- When running locally, if you master URL is set to "local", then there is only one core to run tasks. That is insufficient for programs with even one input DStream (file streams are okay) as the receiver will occupy that core and there will be no core left to process the data. - +- If the number of threads allocated to the application is less than or equal to the number of input DStreams / receivers, then the system will receive data, but not be able to process them. +- When running locally, if you master URL is set to "local", then there is only one core to run tasks. That is insufficient for programs using a DStream as the receiver (file streams are okay). So, a "local" master URL in a streaming app is generally going to cause starvation for the processor. +Thus in any streaming app, you generally will want to allocate more than one thread (i.e. set your master to "local[2]") when testing locally. +See [Spark Properties] (configuration.html#spark-properties.html). + ### Basic Sources {:.no_toc} From f7ac8c2b1de96151231617846b7468d23379c74a Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Wed, 5 Nov 2014 15:49:42 -0800 Subject: [PATCH 1230/1492] SPARK-3223 runAsSparkUser cannot change HDFS write permission properly i... ...n mesos cluster mode - change master newer Author: Jongyoul Lee Closes #3034 from jongyoul/SPARK-3223 and squashes the following commits: 42b2ed3 [Jongyoul Lee] SPARK-3223 runAsSparkUser cannot change HDFS write permission properly in mesos cluster mode - change master newer --- .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 2 +- .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d8c0e2f66df01..e4b859846035c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -93,7 +93,7 @@ private[spark] class CoarseMesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = CoarseMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { { val ret = driver.run() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 8e2faff90f9b2..7d097a3a7aaa3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -72,7 +72,7 @@ private[spark] class MesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = MesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser(sc.sparkUser).setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { val ret = driver.run() From cb0eae3b78d7f6f56c0b9521ee48564a4967d3de Mon Sep 17 00:00:00 2001 From: Brenden Matthews Date: Wed, 5 Nov 2014 16:02:44 -0800 Subject: [PATCH 1231/1492] [SPARK-4158] Fix for missing resources. Mesos offers may not contain all resources, and Spark needs to check to ensure they are present and sufficient. Spark may throw an erroneous exception when resources aren't present. Author: Brenden Matthews Closes #3024 from brndnmtthws/fix-mesos-resource-misuse and squashes the following commits: e5f9580 [Brenden Matthews] [SPARK-4158] Fix for missing resources. --- .../scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala | 3 +-- .../spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index e4b859846035c..5289661eb896b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -242,8 +242,7 @@ private[spark] class CoarseMesosSchedulerBackend( for (r <- res if r.getName == name) { return r.getScalar.getValue } - // If we reached here, no resource with the required name was present - throw new IllegalArgumentException("No resource called " + name + " in " + res) + 0 } /** Build a Mesos resource protobuf object */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 7d097a3a7aaa3..c5f3493477bc5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -278,8 +278,7 @@ private[spark] class MesosSchedulerBackend( for (r <- res if r.getName == name) { return r.getScalar.getValue } - // If we reached here, no resource with the required name was present - throw new IllegalArgumentException("No resource called " + name + " in " + res) + 0 } /** Turn a Spark TaskDescription into a Mesos task */ From c315d1316cb2372e90ae3a12f72d5b3304435a6b Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 5 Nov 2014 19:51:18 -0800 Subject: [PATCH 1232/1492] [SPARK-4254] [mllib] MovieLensALS bug fix Changed code so it does not try to serialize Params. CC: mengxr debasish83 srowen Author: Joseph K. Bradley Closes #3116 from jkbradley/als-bugfix and squashes the following commits: e575bd8 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into als-bugfix 9401b16 [Joseph K. Bradley] changed implicitPrefs so it is not serialized to fix MovieLensALS example bug --- .../scala/org/apache/spark/examples/mllib/MovieLensALS.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 8796c28db8a66..91a0a860d6c71 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -106,9 +106,11 @@ object MovieLensALS { Logger.getRootLogger.setLevel(Level.WARN) + val implicitPrefs = params.implicitPrefs + val ratings = sc.textFile(params.input).map { line => val fields = line.split("::") - if (params.implicitPrefs) { + if (implicitPrefs) { /* * MovieLens ratings are on a scale of 1-5: * 5: Must see From 3d2b5bc5bb979d8b0b71e06bc0f4548376fdbb98 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 5 Nov 2014 19:56:16 -0800 Subject: [PATCH 1233/1492] [SPARK-4262][SQL] add .schemaRDD to JavaSchemaRDD marmbrus Author: Xiangrui Meng Closes #3125 from mengxr/SPARK-4262 and squashes the following commits: 307695e [Xiangrui Meng] add .schemaRDD to JavaSchemaRDD --- .../scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala | 3 +++ 1 file changed, 3 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 1e0ccb368a276..78e8d908fe0c8 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 @@ -47,6 +47,9 @@ class JavaSchemaRDD( private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan) + /** Returns the underlying Scala SchemaRDD. */ + val schemaRDD: SchemaRDD = baseSchemaRDD + override val classTag = scala.reflect.classTag[Row] override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd) From db45f5ad0368760dbeaa618a04f66ae9b2bed656 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 5 Nov 2014 20:45:35 -0800 Subject: [PATCH 1234/1492] [SPARK-4137] [EC2] Don't change working dir on user This issue was uncovered after [this discussion](https://issues.apache.org/jira/browse/SPARK-3398?focusedCommentId=14187471&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14187471). Don't change the working directory on the user. This breaks relative paths the user may pass in, e.g., for the SSH identity file. ``` ./ec2/spark-ec2 -i ../my.pem ``` This patch will preserve the user's current working directory and allow calls like the one above to work. Author: Nicholas Chammas Closes #2988 from nchammas/spark-ec2-cwd and squashes the following commits: f3850b5 [Nicholas Chammas] pep8 fix fbc20c7 [Nicholas Chammas] revert to old commenting style 752f958 [Nicholas Chammas] specify deploy.generic path absolutely bcdf6a5 [Nicholas Chammas] fix typo 77871a2 [Nicholas Chammas] add clarifying comment ce071fc [Nicholas Chammas] don't change working dir --- ec2/spark-ec2 | 8 ++++++-- ec2/spark_ec2.py | 12 +++++++++++- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/ec2/spark-ec2 b/ec2/spark-ec2 index 31f9771223e51..4aa908242eeaa 100755 --- a/ec2/spark-ec2 +++ b/ec2/spark-ec2 @@ -18,5 +18,9 @@ # limitations under the License. # -cd "`dirname $0`" -PYTHONPATH="./third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" python ./spark_ec2.py "$@" +# Preserve the user's CWD so that relative paths are passed correctly to +#+ the underlying Python script. +SPARK_EC2_DIR="$(dirname $0)" + +PYTHONPATH="${SPARK_EC2_DIR}/third_party/boto-2.4.1.zip/boto-2.4.1:$PYTHONPATH" \ + python "${SPARK_EC2_DIR}/spark_ec2.py" "$@" diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 50f88f735650e..a5396c2375915 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -40,6 +40,7 @@ from boto import ec2 DEFAULT_SPARK_VERSION = "1.1.0" +SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) MESOS_SPARK_EC2_BRANCH = "v4" # A URL prefix from which to fetch AMI information @@ -593,7 +594,14 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): ) print "Deploying files to master..." - deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, modules) + deploy_files( + conn=conn, + root_dir=SPARK_EC2_DIR + "/" + "deploy.generic", + opts=opts, + master_nodes=master_nodes, + slave_nodes=slave_nodes, + modules=modules + ) print "Running setup on master..." setup_spark_cluster(master, opts) @@ -730,6 +738,8 @@ def get_num_disks(instance_type): # cluster (e.g. lists of masters and slaves). Files are only deployed to # the first master instance in the cluster, and we expect the setup # script to be run on that instance to copy them to other nodes. +# +# root_dir should be an absolute path to the directory with the files we want to deploy. def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): active_master = master_nodes[0].public_dns_name From 5f27ae16d5b016fae4afeb0f2ad779fd3130b390 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 6 Nov 2014 00:03:03 -0800 Subject: [PATCH 1235/1492] [SPARK-4255] Fix incorrect table striping This commit stripes table rows after hiding some rows, to ensure that rows are correct striped to alternate white and grey even when rows are hidden by default. Author: Kay Ousterhout Closes #3117 from kayousterhout/striping and squashes the following commits: be6e10a [Kay Ousterhout] [SPARK-4255] Fix incorrect table striping --- .../org/apache/spark/ui/static/additional-metrics.js | 2 ++ core/src/main/resources/org/apache/spark/ui/static/table.js | 5 ----- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index c5936b5038ac9..badd85ed48c82 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -39,6 +39,8 @@ $(function() { var column = "table ." + $(this).attr("name"); $(column).hide(); }); + // Stripe table rows after rows have been hidden to ensure correct striping. + stripeTables(); $("input:checkbox").click(function() { var column = "table ." + $(this).attr("name"); diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js index 32187ba6e8df0..6bb03015abb51 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/table.js +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -28,8 +28,3 @@ function stripeTables() { }); }); } - -/* Stripe all tables after pages finish loading. */ -$(function() { - stripeTables(); -}); From b41a39e24038876359aeb7ce2bbbb4de2234e5f3 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 6 Nov 2014 00:22:19 -0800 Subject: [PATCH 1236/1492] [SPARK-4186] add binaryFiles and binaryRecords in Python add binaryFiles() and binaryRecords() in Python ``` binaryFiles(self, path, minPartitions=None): :: Developer API :: Read a directory of binary files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI as a byte array. Each file is read as a single record and returned in a key-value pair, where the key is the path of each file, the value is the content of each file. Note: Small files are preferred, large file is also allowable, but may cause bad performance. binaryRecords(self, path, recordLength): Load data from a flat binary file, assuming each record is a set of numbers with the specified numerical format (see ByteBuffer), and the number of bytes per record is constant. :param path: Directory to the input data files :param recordLength: The length at which to split the records ``` Author: Davies Liu Closes #3078 from davies/binary and squashes the following commits: cd0bdbd [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary 3aa349b [Davies Liu] add experimental notes 24e84b6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary 5ceaa8a [Davies Liu] Merge branch 'master' of github.com:apache/spark into binary 1900085 [Davies Liu] bugfix bb22442 [Davies Liu] add binaryFiles and binaryRecords in Python --- .../scala/org/apache/spark/SparkContext.scala | 4 ++ .../spark/api/java/JavaSparkContext.scala | 12 ++--- .../apache/spark/api/python/PythonRDD.scala | 45 ++++++++++++------- python/pyspark/context.py | 32 ++++++++++++- python/pyspark/tests.py | 19 ++++++++ 5 files changed, 90 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3cdaa6a9cc8a8..03ea672c813d1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -560,6 +560,8 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { /** + * :: Experimental :: + * * Get an RDD for a Hadoop-readable dataset as PortableDataStream for each file * (useful for binary data) * @@ -602,6 +604,8 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { } /** + * :: Experimental :: + * * Load data from a flat binary file, assuming the length of each record is constant. * * @param path Directory to the input data files diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index e3aeba7e6c39d..5c6e8d32c5c8a 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -21,11 +21,6 @@ import java.io.Closeable import java.util import java.util.{Map => JMap} -import java.io.DataInputStream - -import org.apache.hadoop.io.{BytesWritable, LongWritable} -import org.apache.spark.input.{PortableDataStream, FixedLengthBinaryInputFormat} - import scala.collection.JavaConversions import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -33,6 +28,7 @@ import scala.reflect.ClassTag import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration +import org.apache.spark.input.PortableDataStream import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} @@ -286,6 +282,8 @@ class JavaSparkContext(val sc: SparkContext) new JavaPairRDD(sc.binaryFiles(path, minPartitions)) /** + * :: Experimental :: + * * Read a directory of binary files from HDFS, a local file system (available on all nodes), * or any Hadoop-supported file system URI as a byte array. Each file is read as a single * record and returned in a key-value pair, where the key is the path of each file, @@ -312,15 +310,19 @@ class JavaSparkContext(val sc: SparkContext) * * @note Small files are preferred; very large files but may cause bad performance. */ + @Experimental def binaryFiles(path: String): JavaPairRDD[String, PortableDataStream] = new JavaPairRDD(sc.binaryFiles(path, defaultMinPartitions)) /** + * :: Experimental :: + * * Load data from a flat binary file, assuming the length of each record is constant. * * @param path Directory to the input data files * @return An RDD of data with values, represented as byte arrays */ + @Experimental def binaryRecords(path: String, recordLength: Int): JavaRDD[Array[Byte]] = { new JavaRDD(sc.binaryRecords(path, recordLength)) } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e94ccdcd47bb7..45beb8fc8c925 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -21,6 +21,8 @@ import java.io._ import java.net._ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} +import org.apache.spark.input.PortableDataStream + import scala.collection.JavaConversions._ import scala.collection.mutable import scala.language.existentials @@ -395,22 +397,33 @@ private[spark] object PythonRDD extends Logging { newIter.asInstanceOf[Iterator[String]].foreach { str => writeUTF(str, dataOut) } - case pair: Tuple2[_, _] => - pair._1 match { - case bytePair: Array[Byte] => - newIter.asInstanceOf[Iterator[Tuple2[Array[Byte], Array[Byte]]]].foreach { pair => - dataOut.writeInt(pair._1.length) - dataOut.write(pair._1) - dataOut.writeInt(pair._2.length) - dataOut.write(pair._2) - } - case stringPair: String => - newIter.asInstanceOf[Iterator[Tuple2[String, String]]].foreach { pair => - writeUTF(pair._1, dataOut) - writeUTF(pair._2, dataOut) - } - case other => - throw new SparkException("Unexpected Tuple2 element type " + pair._1.getClass) + case stream: PortableDataStream => + newIter.asInstanceOf[Iterator[PortableDataStream]].foreach { stream => + val bytes = stream.toArray() + dataOut.writeInt(bytes.length) + dataOut.write(bytes) + } + case (key: String, stream: PortableDataStream) => + newIter.asInstanceOf[Iterator[(String, PortableDataStream)]].foreach { + case (key, stream) => + writeUTF(key, dataOut) + val bytes = stream.toArray() + dataOut.writeInt(bytes.length) + dataOut.write(bytes) + } + case (key: String, value: String) => + newIter.asInstanceOf[Iterator[(String, String)]].foreach { + case (key, value) => + writeUTF(key, dataOut) + writeUTF(value, dataOut) + } + case (key: Array[Byte], value: Array[Byte]) => + newIter.asInstanceOf[Iterator[(Array[Byte], Array[Byte])]].foreach { + case (key, value) => + dataOut.writeInt(key.length) + dataOut.write(key) + dataOut.writeInt(value.length) + dataOut.write(value) } case other => throw new SparkException("Unexpected element type " + first.getClass) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a0e4821728c8b..faa5952258aef 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer, CompressedSerializer, AutoBatchedSerializer + PairDeserializer, CompressedSerializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -388,6 +388,36 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): return RDD(self._jsc.wholeTextFiles(path, minPartitions), self, PairDeserializer(UTF8Deserializer(use_unicode), UTF8Deserializer(use_unicode))) + def binaryFiles(self, path, minPartitions=None): + """ + :: Experimental :: + + Read a directory of binary files from HDFS, a local file system + (available on all nodes), or any Hadoop-supported file system URI + as a byte array. Each file is read as a single record and returned + in a key-value pair, where the key is the path of each file, the + value is the content of each file. + + Note: Small files are preferred, large file is also allowable, but + may cause bad performance. + """ + minPartitions = minPartitions or self.defaultMinPartitions + return RDD(self._jsc.binaryFiles(path, minPartitions), self, + PairDeserializer(UTF8Deserializer(), NoOpSerializer())) + + def binaryRecords(self, path, recordLength): + """ + :: Experimental :: + + Load data from a flat binary file, assuming each record is a set of numbers + with the specified numerical format (see ByteBuffer), and the number of + bytes per record is constant. + + :param path: Directory to the input data files + :param recordLength: The length at which to split the records + """ + return RDD(self._jsc.binaryRecords(path, recordLength), self, NoOpSerializer()) + def _dictToJavaMap(self, d): jm = self._jvm.java.util.HashMap() if not d: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 7e61b017efa75..9f625c5c6ca48 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1110,6 +1110,25 @@ def test_converters(self): (u'\x03', [2.0])] self.assertEqual(maps, em) + def test_binary_files(self): + path = os.path.join(self.tempdir.name, "binaryfiles") + os.mkdir(path) + data = "short binary data" + with open(os.path.join(path, "part-0000"), 'w') as f: + f.write(data) + [(p, d)] = self.sc.binaryFiles(path).collect() + self.assertTrue(p.endswith("part-0000")) + self.assertEqual(d, data) + + def test_binary_records(self): + path = os.path.join(self.tempdir.name, "binaryrecords") + os.mkdir(path) + with open(os.path.join(path, "part-0000"), 'w') as f: + for i in range(100): + f.write('%04d' % i) + result = self.sc.binaryRecords(path, 4).map(int).collect() + self.assertEqual(range(100), result) + class OutputFormatTests(ReusedPySparkTestCase): From 23eaf0e12ff221dcca40a79e61b6cc5e7c846cb5 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 6 Nov 2014 10:45:46 -0800 Subject: [PATCH 1237/1492] [SPARK-4264] Completion iterator should only invoke callback once Author: Aaron Davidson Closes #3128 from aarondav/compiter and squashes the following commits: 698e4be [Aaron Davidson] [SPARK-4264] Completion iterator should only invoke callback once --- .../spark/util/CompletionIterator.scala | 5 +- .../spark/util/CompletionIteratorSuite.scala | 47 +++++++++++++++++++ 2 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index b6a099825f01b..390310243ee0a 100644 --- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -25,10 +25,13 @@ private[spark] // scalastyle:off abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterator[A] { // scalastyle:on + + private[this] var completed = false def next() = sub.next() def hasNext = { val r = sub.hasNext - if (!r) { + if (!r && !completed) { + completed = true completion() } r diff --git a/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala new file mode 100644 index 0000000000000..3755d43e25ea8 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import org.scalatest.FunSuite + +class CompletionIteratorSuite extends FunSuite { + test("basic test") { + var numTimesCompleted = 0 + val iter = List(1, 2, 3).iterator + val completionIter = CompletionIterator[Int, Iterator[Int]](iter, { numTimesCompleted += 1 }) + + assert(completionIter.hasNext) + assert(completionIter.next() === 1) + assert(numTimesCompleted === 0) + + assert(completionIter.hasNext) + assert(completionIter.next() === 2) + assert(numTimesCompleted === 0) + + assert(completionIter.hasNext) + assert(completionIter.next() === 3) + assert(numTimesCompleted === 0) + + assert(!completionIter.hasNext) + assert(numTimesCompleted === 1) + + // SPARK-4264: Calling hasNext should not trigger the completion callback again. + assert(!completionIter.hasNext) + assert(numTimesCompleted === 1) + } +} From d15c6e9dc2860bbe56e31ddf71218ccc6d5c841d Mon Sep 17 00:00:00 2001 From: lianhuiwang Date: Thu, 6 Nov 2014 10:46:45 -0800 Subject: [PATCH 1238/1492] [SPARK-4249][GraphX]fix a problem of EdgePartitionBuilder in Graphx at first srcIds is not initialized and are all 0. so we use edgeArray(0).srcId to currSrcId Author: lianhuiwang Closes #3138 from lianhuiwang/SPARK-4249 and squashes the following commits: 3f4e503 [lianhuiwang] fix a problem of EdgePartitionBuilder in Graphx --- .../org/apache/spark/graphx/impl/EdgePartitionBuilder.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index 4520beb991515..2b6137be25547 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -45,8 +45,8 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla // Copy edges into columnar structures, tracking the beginnings of source vertex id clusters and // adding them to the index if (edgeArray.length > 0) { - index.update(srcIds(0), 0) - var currSrcId: VertexId = srcIds(0) + index.update(edgeArray(0).srcId, 0) + var currSrcId: VertexId = edgeArray(0).srcId var i = 0 while (i < edgeArray.size) { srcIds(i) = edgeArray(i).srcId From 470881b24a503c9edcaed159c29bafa446ab0e9a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 6 Nov 2014 15:31:07 -0800 Subject: [PATCH 1239/1492] [HOT FIX] Make distribution fails This was added by me in https://github.com/apache/spark/commit/61a5cced049a8056292ba94f23fa7bd040f50685. The real fix will be added in [SPARK-4281](https://issues.apache.org/jira/browse/SPARK-4281). Author: Andrew Or Closes #3145 from andrewor14/fix-make-distribution and squashes the following commits: c78be61 [Andrew Or] Hot fix make distribution --- make-distribution.sh | 3 --- 1 file changed, 3 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index fac7f7e284be4..0bc839e1dbe4d 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -181,9 +181,6 @@ echo "Spark $VERSION$GITREVSTRING built for Hadoop $SPARK_HADOOP_VERSION" > "$DI # Copy jars cp "$FWDIR"/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" cp "$FWDIR"/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" -cp "$FWDIR"/network/yarn/target/scala*/spark-network-yarn*.jar "$DISTDIR/lib/" -cp "$FWDIR"/network/yarn/target/scala*/spark-network-shuffle*.jar "$DISTDIR/lib/" -cp "$FWDIR"/network/yarn/target/scala*/spark-network-common*.jar "$DISTDIR/lib/" # Copy example sources (needed for python and SQL) mkdir -p "$DISTDIR/examples/src/main" From 96136f222abd4f3abd10cb78a4ebecdb21f3bde7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 6 Nov 2014 17:18:49 -0800 Subject: [PATCH 1240/1492] [SPARK-3797] Minor addendum to Yarn shuffle service I did not realize there was a `network.util.JavaUtils` when I wrote this code. This PR moves the `ByteBuffer` string conversion to the appropriate place. I tested the changes on a stable yarn cluster. Author: Andrew Or Closes #3144 from andrewor14/yarn-shuffle-util and squashes the following commits: b6c08bf [Andrew Or] Remove unused import 94e205c [Andrew Or] Use netty Unpooled 85202a5 [Andrew Or] Use guava Charsets 057135b [Andrew Or] Reword comment adf186d [Andrew Or] Move byte buffer String conversion logic to JavaUtils --- .../apache/spark/network/util/JavaUtils.java | 20 ++++++++++++++++ .../network/sasl/ShuffleSecretManager.java | 24 ++----------------- .../spark/deploy/yarn/ExecutorRunnable.scala | 5 ++-- .../spark/deploy/yarn/ExecutorRunnable.scala | 5 ++-- 4 files changed, 28 insertions(+), 26 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 40b71b0c87a47..2856d1c8c9337 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -17,6 +17,8 @@ package org.apache.spark.network.util; +import java.nio.ByteBuffer; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.Closeable; @@ -25,6 +27,8 @@ import java.io.ObjectOutputStream; import com.google.common.io.Closeables; +import com.google.common.base.Charsets; +import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,4 +77,20 @@ public static int nonNegativeHash(Object obj) { int hash = obj.hashCode(); return hash != Integer.MIN_VALUE ? Math.abs(hash) : 0; } + + /** + * Convert the given string to a byte buffer. The resulting buffer can be + * converted back to the same string through {@link #bytesToString(ByteBuffer)}. + */ + public static ByteBuffer stringToBytes(String s) { + return Unpooled.wrappedBuffer(s.getBytes(Charsets.UTF_8)).nioBuffer(); + } + + /** + * Convert the given byte buffer to a string. The resulting string can be + * converted back to the same byte buffer through {@link #stringToBytes(String)}. + */ + public static String bytesToString(ByteBuffer b) { + return Unpooled.wrappedBuffer(b).toString(Charsets.UTF_8); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java index e66c4af0f1ebd..351c7930a900f 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/ShuffleSecretManager.java @@ -19,13 +19,13 @@ import java.lang.Override; import java.nio.ByteBuffer; -import java.nio.charset.Charset; import java.util.concurrent.ConcurrentHashMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.spark.network.sasl.SecretKeyHolder; +import org.apache.spark.network.util.JavaUtils; /** * A class that manages shuffle secret used by the external shuffle service. @@ -34,30 +34,10 @@ public class ShuffleSecretManager implements SecretKeyHolder { private final Logger logger = LoggerFactory.getLogger(ShuffleSecretManager.class); private final ConcurrentHashMap shuffleSecretMap; - private static final Charset UTF8_CHARSET = Charset.forName("UTF-8"); - // Spark user used for authenticating SASL connections // Note that this must match the value in org.apache.spark.SecurityManager private static final String SPARK_SASL_USER = "sparkSaslUser"; - /** - * Convert the given string to a byte buffer. The resulting buffer can be converted back to - * the same string through {@link #bytesToString(ByteBuffer)}. This is used if the external - * shuffle service represents shuffle secrets as bytes buffers instead of strings. - */ - public static ByteBuffer stringToBytes(String s) { - return ByteBuffer.wrap(s.getBytes(UTF8_CHARSET)); - } - - /** - * Convert the given byte buffer to a string. The resulting string can be converted back to - * the same byte buffer through {@link #stringToBytes(String)}. This is used if the external - * shuffle service represents shuffle secrets as bytes buffers instead of strings. - */ - public static String bytesToString(ByteBuffer b) { - return new String(b.array(), UTF8_CHARSET); - } - public ShuffleSecretManager() { shuffleSecretMap = new ConcurrentHashMap(); } @@ -80,7 +60,7 @@ public void registerApp(String appId, String shuffleSecret) { * Register an application with its secret specified as a byte buffer. */ public void registerApp(String appId, ByteBuffer shuffleSecret) { - registerApp(appId, bytesToString(shuffleSecret)); + registerApp(appId, JavaUtils.bytesToString(shuffleSecret)); } /** diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 5f47c79cabaee..7023a1170654f 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.spark.{SecurityManager, SparkConf, Logging} -import org.apache.spark.network.sasl.ShuffleSecretManager +import org.apache.spark.network.util.JavaUtils @deprecated("use yarn/stable", "1.2.0") class ExecutorRunnable( @@ -98,7 +98,8 @@ class ExecutorRunnable( val secretString = securityMgr.getSecretKey() val secretBytes = if (secretString != null) { - ShuffleSecretManager.stringToBytes(secretString) + // This conversion must match how the YarnShuffleService decodes our secret + JavaUtils.stringToBytes(secretString) } else { // Authentication is not enabled, so just provide dummy metadata ByteBuffer.allocate(0) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 18f48b4b6caf6..fdd3c2300fa78 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} import org.apache.spark.{SecurityManager, SparkConf, Logging} -import org.apache.spark.network.sasl.ShuffleSecretManager +import org.apache.spark.network.util.JavaUtils class ExecutorRunnable( @@ -97,7 +97,8 @@ class ExecutorRunnable( val secretString = securityMgr.getSecretKey() val secretBytes = if (secretString != null) { - ShuffleSecretManager.stringToBytes(secretString) + // This conversion must match how the YarnShuffleService decodes our secret + JavaUtils.stringToBytes(secretString) } else { // Authentication is not enabled, so just provide dummy metadata ByteBuffer.allocate(0) From 6e9ef10fd7446a11f37446c961916ba2a8e02cb8 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 6 Nov 2014 17:20:46 -0800 Subject: [PATCH 1241/1492] [SPARK-4277] Support external shuffle service on Standalone Worker Author: Aaron Davidson Closes #3142 from aarondav/worker and squashes the following commits: 3780bd7 [Aaron Davidson] Address comments 2dcdfc1 [Aaron Davidson] Add private[worker] 47f49d3 [Aaron Davidson] NettyBlockTransferService shouldn't care about app ids (it's only b/t executors) 258417c [Aaron Davidson] [SPARK-4277] Support external shuffle service on executor --- .../org/apache/spark/SecurityManager.scala | 14 +--- .../StandaloneWorkerShuffleService.scala | 66 +++++++++++++++++++ .../apache/spark/deploy/worker/Worker.scala | 8 ++- .../storage/ShuffleBlockFetcherIterator.scala | 2 +- .../NettyBlockTransferSecuritySuite.scala | 12 ---- .../spark/network/sasl/SaslMessage.java | 3 +- 6 files changed, 79 insertions(+), 26 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index dee935ffad51f..dbff9d12b5ad7 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -343,15 +343,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging with */ def getSecretKey(): String = secretKey - override def getSaslUser(appId: String): String = { - val myAppId = sparkConf.getAppId - require(appId == myAppId, s"SASL appId $appId did not match my appId ${myAppId}") - getSaslUser() - } - - override def getSecretKey(appId: String): String = { - val myAppId = sparkConf.getAppId - require(appId == myAppId, s"SASL appId $appId did not match my appId ${myAppId}") - getSecretKey() - } + // Default SecurityManager only has a single secret key, so ignore appId. + override def getSaslUser(appId: String): String = getSaslUser() + override def getSecretKey(appId: String): String = getSecretKey() } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala new file mode 100644 index 0000000000000..88118e2837741 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.worker + +import org.apache.spark.{Logging, SparkConf, SecurityManager} +import org.apache.spark.network.TransportContext +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.sasl.SaslRpcHandler +import org.apache.spark.network.server.TransportServer +import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler + +/** + * Provides a server from which Executors can read shuffle files (rather than reading directly from + * each other), to provide uninterrupted access to the files in the face of executors being turned + * off or killed. + * + * Optionally requires SASL authentication in order to read. See [[SecurityManager]]. + */ +private[worker] +class StandaloneWorkerShuffleService(sparkConf: SparkConf, securityManager: SecurityManager) + extends Logging { + + private val enabled = sparkConf.getBoolean("spark.shuffle.service.enabled", false) + private val port = sparkConf.getInt("spark.shuffle.service.port", 7337) + private val useSasl: Boolean = securityManager.isAuthenticationEnabled() + + private val transportConf = SparkTransportConf.fromSparkConf(sparkConf) + private val blockHandler = new ExternalShuffleBlockHandler() + private val transportContext: TransportContext = { + val handler = if (useSasl) new SaslRpcHandler(blockHandler, securityManager) else blockHandler + new TransportContext(transportConf, handler) + } + + private var server: TransportServer = _ + + /** Starts the external shuffle service if the user has configured us to. */ + def startIfEnabled() { + if (enabled) { + require(server == null, "Shuffle server already started") + logInfo(s"Starting shuffle service on port $port with useSasl = $useSasl") + server = transportContext.createServer(port) + } + } + + def stop() { + if (enabled && server != null) { + server.close() + server = null + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index f1f66d0903f1c..ca262de832e25 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -111,6 +111,9 @@ private[spark] class Worker( val drivers = new HashMap[String, DriverRunner] val finishedDrivers = new HashMap[String, DriverRunner] + // The shuffle service is not actually started unless configured. + val shuffleService = new StandaloneWorkerShuffleService(conf, securityMgr) + val publicAddress = { val envVar = System.getenv("SPARK_PUBLIC_DNS") if (envVar != null) envVar else host @@ -154,6 +157,7 @@ private[spark] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + shuffleService.startIfEnabled() webUi = new WorkerWebUI(this, workDir, webUiPort) webUi.bind() registerWithMaster() @@ -419,6 +423,7 @@ private[spark] class Worker( registrationRetryTimer.foreach(_.cancel()) executors.values.foreach(_.kill()) drivers.values.foreach(_.kill()) + shuffleService.stop() webUi.stop() metricsSystem.stop() } @@ -441,7 +446,8 @@ private[spark] object Worker extends Logging { cores: Int, memory: Int, masterUrls: Array[String], - workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = { + workDir: String, + workerNumber: Option[Int] = None): (ActorSystem, Int) = { // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val conf = new SparkConf diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 1e579187e4193..6b1f57a069431 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -92,7 +92,7 @@ final class ShuffleBlockFetcherIterator( * Current [[FetchResult]] being processed. We track this so we can release the current buffer * in case of a runtime exception when processing the current buffer. */ - private[this] var currentResult: FetchResult = null + @volatile private[this] var currentResult: FetchResult = null /** * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index bed0ed9d713dd..9162ec9801663 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -89,18 +89,6 @@ class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with Sh } } - test("security mismatch app ids") { - val conf0 = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.app.id", "other-id") - testConnection(conf0, conf1) match { - case Success(_) => fail("Should have failed") - case Failure(t) => t.getMessage should include ("SASL appId app-id did not match") - } - } - /** * Creates two servers with different configurations and sees if they can talk. * Returns Success() if they can transfer a block, and Failure() if the block transfer was failed diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java index 5b77e18c26bf4..599cc6428c90e 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java @@ -58,7 +58,8 @@ public void encode(ByteBuf buf) { public static SaslMessage decode(ByteBuf buf) { if (buf.readByte() != TAG_BYTE) { - throw new IllegalStateException("Expected SaslMessage, received something else"); + throw new IllegalStateException("Expected SaslMessage, received something else" + + " (maybe your client does not have SASL enabled?)"); } int idLength = buf.readInt(); From f165b2bbf5d4acf34d826fa55b900f5bbc295654 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 6 Nov 2014 18:39:14 -0800 Subject: [PATCH 1242/1492] [SPARK-4188] [Core] Perform network-level retry of shuffle file fetches This adds a RetryingBlockFetcher to the NettyBlockTransferService which is wrapped around our typical OneForOneBlockFetcher, adding retry logic in the event of an IOException. This sort of retry allows us to avoid marking an entire executor as failed due to garbage collection or high network load. TODO: - [x] unit tests - [x] put in ExternalShuffleClient too Author: Aaron Davidson Closes #3101 from aarondav/retry and squashes the following commits: 72a2a32 [Aaron Davidson] Add that we should remove the condition around the retry thingy c7fd107 [Aaron Davidson] Fix unit tests e80e4c2 [Aaron Davidson] Address initial comments 6f594cd [Aaron Davidson] Fix unit test 05ff43c [Aaron Davidson] Add to external shuffle client and add unit test 66e5a24 [Aaron Davidson] [SPARK-4238] [Core] Perform network-level retry of shuffle file fetches --- .../netty/NettyBlockTransferService.scala | 21 +- .../spark/network/client/TransportClient.java | 16 +- .../client/TransportClientFactory.java | 13 +- .../client/TransportResponseHandler.java | 3 +- .../network/protocol/MessageEncoder.java | 2 +- .../spark/network/server/TransportServer.java | 8 +- .../apache/spark/network/util/NettyUtils.java | 14 +- .../spark/network/util/TransportConf.java | 17 + .../network/TransportClientFactorySuite.java | 7 +- .../shuffle/ExternalShuffleClient.java | 31 +- .../shuffle/OneForOneBlockFetcher.java | 9 +- .../network/shuffle/RetryingBlockFetcher.java | 234 +++++++++++++ .../network/sasl/SaslIntegrationSuite.java | 4 +- .../ExternalShuffleIntegrationSuite.java | 18 +- .../shuffle/ExternalShuffleSecuritySuite.java | 6 +- .../shuffle/RetryingBlockFetcherSuite.java | 310 ++++++++++++++++++ 16 files changed, 668 insertions(+), 45 deletions(-) create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 0d1fc81d2a16f..b937ea825f49e 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -27,7 +27,7 @@ import org.apache.spark.network.client.{TransportClientBootstrap, RpcResponseCal import org.apache.spark.network.netty.NettyMessages.{OpenBlocks, UploadBlock} import org.apache.spark.network.sasl.{SaslRpcHandler, SaslClientBootstrap} import org.apache.spark.network.server._ -import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher} +import org.apache.spark.network.shuffle.{RetryingBlockFetcher, BlockFetchingListener, OneForOneBlockFetcher} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils @@ -71,9 +71,22 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage listener: BlockFetchingListener): Unit = { logTrace(s"Fetch blocks from $host:$port (executor id $execId)") try { - val client = clientFactory.createClient(host, port) - new OneForOneBlockFetcher(client, blockIds.toArray, listener) - .start(OpenBlocks(blockIds.map(BlockId.apply))) + val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter { + override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) { + val client = clientFactory.createClient(host, port) + new OneForOneBlockFetcher(client, blockIds.toArray, listener) + .start(OpenBlocks(blockIds.map(BlockId.apply))) + } + } + + val maxRetries = transportConf.maxIORetries() + if (maxRetries > 0) { + // Note this Fetcher will correctly handle maxRetries == 0; we avoid it just in case there's + // a bug in this code. We should remove the if statement once we're sure of the stability. + new RetryingBlockFetcher(transportConf, blockFetchStarter, blockIds, listener).start() + } else { + blockFetchStarter.createAndStart(blockIds, listener) + } } catch { case e: Exception => logError("Exception while beginning fetchBlocks", e) diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java index a08cee02dd576..4e944114e8176 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -18,7 +18,9 @@ package org.apache.spark.network.client; import java.io.Closeable; +import java.io.IOException; import java.util.UUID; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import com.google.common.base.Objects; @@ -116,8 +118,12 @@ public void operationComplete(ChannelFuture future) throws Exception { serverAddr, future.cause()); logger.error(errorMsg, future.cause()); handler.removeFetchRequest(streamChunkId); - callback.onFailure(chunkIndex, new RuntimeException(errorMsg, future.cause())); channel.close(); + try { + callback.onFailure(chunkIndex, new IOException(errorMsg, future.cause())); + } catch (Exception e) { + logger.error("Uncaught exception in RPC response callback handler!", e); + } } } }); @@ -147,8 +153,12 @@ public void operationComplete(ChannelFuture future) throws Exception { serverAddr, future.cause()); logger.error(errorMsg, future.cause()); handler.removeRpcRequest(requestId); - callback.onFailure(new RuntimeException(errorMsg, future.cause())); channel.close(); + try { + callback.onFailure(new IOException(errorMsg, future.cause())); + } catch (Exception e) { + logger.error("Uncaught exception in RPC response callback handler!", e); + } } } }); @@ -175,6 +185,8 @@ public void onFailure(Throwable e) { try { return result.get(timeoutMs, TimeUnit.MILLISECONDS); + } catch (ExecutionException e) { + throw Throwables.propagate(e.getCause()); } catch (Exception e) { throw Throwables.propagate(e); } diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 1723fed307257..397d3a8455c86 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -18,12 +18,12 @@ package org.apache.spark.network.client; import java.io.Closeable; +import java.io.IOException; import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.List; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import com.google.common.base.Preconditions; @@ -44,7 +44,6 @@ import org.apache.spark.network.TransportContext; import org.apache.spark.network.server.TransportChannelHandler; import org.apache.spark.network.util.IOMode; -import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -93,15 +92,17 @@ public TransportClientFactory( * * Concurrency: This method is safe to call from multiple threads. */ - public TransportClient createClient(String remoteHost, int remotePort) { + public TransportClient createClient(String remoteHost, int remotePort) throws IOException { // Get connection from the connection pool first. // If it is not found or not active, create a new one. final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); TransportClient cachedClient = connectionPool.get(address); if (cachedClient != null) { if (cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", address, cachedClient); return cachedClient; } else { + logger.info("Found inactive connection to {}, closing it.", address); connectionPool.remove(address, cachedClient); // Remove inactive clients. } } @@ -133,10 +134,10 @@ public void initChannel(SocketChannel ch) { long preConnect = System.currentTimeMillis(); ChannelFuture cf = bootstrap.connect(address); if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { - throw new RuntimeException( + throw new IOException( String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); } else if (cf.cause() != null) { - throw new RuntimeException(String.format("Failed to connect to %s", address), cf.cause()); + throw new IOException(String.format("Failed to connect to %s", address), cf.cause()); } TransportClient client = clientRef.get(); @@ -198,7 +199,7 @@ public void close() { */ private PooledByteBufAllocator createPooledByteBufAllocator() { return new PooledByteBufAllocator( - PlatformDependent.directBufferPreferred(), + conf.preferDirectBufs() && PlatformDependent.directBufferPreferred(), getPrivateStaticField("DEFAULT_NUM_HEAP_ARENA"), getPrivateStaticField("DEFAULT_NUM_DIRECT_ARENA"), getPrivateStaticField("DEFAULT_PAGE_SIZE"), diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index d8965590b34da..2044afb0d85db 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -17,6 +17,7 @@ package org.apache.spark.network.client; +import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -94,7 +95,7 @@ public void channelUnregistered() { String remoteAddress = NettyUtils.getRemoteAddress(channel); logger.error("Still have {} requests outstanding when connection from {} is closed", numOutstandingRequests(), remoteAddress); - failOutstandingRequests(new RuntimeException("Connection from " + remoteAddress + " closed")); + failOutstandingRequests(new IOException("Connection from " + remoteAddress + " closed")); } } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index 4cb8becc3ed22..91d1e8a538a77 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -66,7 +66,7 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) { // All messages have the frame length, message type, and message itself. int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); long frameLength = headerLength + bodyLength; - ByteBuf header = ctx.alloc().buffer(headerLength); + ByteBuf header = ctx.alloc().heapBuffer(headerLength); header.writeLong(frameLength); msgType.encode(header); in.encode(header); diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java index 70da48ca8ee79..579676c2c3564 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -28,6 +28,7 @@ import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.socket.SocketChannel; +import io.netty.util.internal.PlatformDependent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,11 +72,14 @@ private void init(int portToBind) { NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); EventLoopGroup workerGroup = bossGroup; + PooledByteBufAllocator allocator = new PooledByteBufAllocator( + conf.preferDirectBufs() && PlatformDependent.directBufferPreferred()); + bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) .channel(NettyUtils.getServerChannelClass(ioMode)) - .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT); + .option(ChannelOption.ALLOCATOR, allocator) + .childOption(ChannelOption.ALLOCATOR, allocator); if (conf.backLog() > 0) { bootstrap.option(ChannelOption.SO_BACKLOG, conf.backLog()); diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java index b1872341198e0..2a7664fe89388 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -37,13 +37,17 @@ * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO. */ public class NettyUtils { - /** Creates a Netty EventLoopGroup based on the IOMode. */ - public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) { - - ThreadFactory threadFactory = new ThreadFactoryBuilder() + /** Creates a new ThreadFactory which prefixes each thread with the given name. */ + public static ThreadFactory createThreadFactory(String threadPoolPrefix) { + return new ThreadFactoryBuilder() .setDaemon(true) - .setNameFormat(threadPrefix + "-%d") + .setNameFormat(threadPoolPrefix + "-%d") .build(); + } + + /** Creates a Netty EventLoopGroup based on the IOMode. */ + public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) { + ThreadFactory threadFactory = createThreadFactory(threadPrefix); switch (mode) { case NIO: diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 823790dd3c66f..787a8f0031af1 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -30,6 +30,11 @@ public TransportConf(ConfigProvider conf) { /** IO mode: nio or epoll */ public String ioMode() { return conf.get("spark.shuffle.io.mode", "NIO").toUpperCase(); } + /** If true, we will prefer allocating off-heap byte buffers within Netty. */ + public boolean preferDirectBufs() { + return conf.getBoolean("spark.shuffle.io.preferDirectBufs", true); + } + /** Connect timeout in secs. Default 120 secs. */ public int connectionTimeoutMs() { return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; @@ -58,4 +63,16 @@ public int connectionTimeoutMs() { /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ public int saslRTTimeout() { return conf.getInt("spark.shuffle.sasl.timeout", 30000); } + + /** + * Max number of times we will try IO exceptions (such as connection timeouts) per request. + * If set to 0, we will not do any retries. + */ + public int maxIORetries() { return conf.getInt("spark.shuffle.io.maxRetries", 3); } + + /** + * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. + * Only relevant if maxIORetries > 0. + */ + public int ioRetryWaitTime() { return conf.getInt("spark.shuffle.io.retryWaitMs", 5000); } } diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 5a10fdb3842ef..822bef1d81b2a 100644 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -17,6 +17,7 @@ package org.apache.spark.network; +import java.io.IOException; import java.util.concurrent.TimeoutException; import org.junit.After; @@ -57,7 +58,7 @@ public void tearDown() { } @Test - public void createAndReuseBlockClients() throws TimeoutException { + public void createAndReuseBlockClients() throws IOException { TransportClientFactory factory = context.createClientFactory(); TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); @@ -70,7 +71,7 @@ public void createAndReuseBlockClients() throws TimeoutException { } @Test - public void neverReturnInactiveClients() throws Exception { + public void neverReturnInactiveClients() throws IOException, InterruptedException { TransportClientFactory factory = context.createClientFactory(); TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); c1.close(); @@ -88,7 +89,7 @@ public void neverReturnInactiveClients() throws Exception { } @Test - public void closeBlockClientsWithFactory() throws TimeoutException { + public void closeBlockClientsWithFactory() throws IOException { TransportClientFactory factory = context.createClientFactory(); TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index 3aa95d00f6b20..27884b82c8cb9 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -17,6 +17,7 @@ package org.apache.spark.network.shuffle; +import java.io.IOException; import java.util.List; import com.google.common.collect.Lists; @@ -76,17 +77,33 @@ public void init(String appId) { @Override public void fetchBlocks( - String host, - int port, - String execId, + final String host, + final int port, + final String execId, String[] blockIds, BlockFetchingListener listener) { assert appId != null : "Called before init()"; logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId); try { - TransportClient client = clientFactory.createClient(host, port); - new OneForOneBlockFetcher(client, blockIds, listener) - .start(new ExternalShuffleMessages.OpenShuffleBlocks(appId, execId, blockIds)); + RetryingBlockFetcher.BlockFetchStarter blockFetchStarter = + new RetryingBlockFetcher.BlockFetchStarter() { + @Override + public void createAndStart(String[] blockIds, BlockFetchingListener listener) + throws IOException { + TransportClient client = clientFactory.createClient(host, port); + new OneForOneBlockFetcher(client, blockIds, listener) + .start(new ExternalShuffleMessages.OpenShuffleBlocks(appId, execId, blockIds)); + } + }; + + int maxRetries = conf.maxIORetries(); + if (maxRetries > 0) { + // Note this Fetcher will correctly handle maxRetries == 0; we avoid it just in case there's + // a bug in this code. We should remove the if statement once we're sure of the stability. + new RetryingBlockFetcher(conf, blockFetchStarter, blockIds, listener).start(); + } else { + blockFetchStarter.createAndStart(blockIds, listener); + } } catch (Exception e) { logger.error("Exception while beginning fetchBlocks", e); for (String blockId : blockIds) { @@ -108,7 +125,7 @@ public void registerWithShuffleServer( String host, int port, String execId, - ExecutorShuffleInfo executorInfo) { + ExecutorShuffleInfo executorInfo) throws IOException { assert appId != null : "Called before init()"; TransportClient client = clientFactory.createClient(host, port); byte[] registerExecutorMessage = diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 39b6f30f92baf..9e77a1f68c4b0 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -51,9 +51,6 @@ public OneForOneBlockFetcher( TransportClient client, String[] blockIds, BlockFetchingListener listener) { - if (blockIds.length == 0) { - throw new IllegalArgumentException("Zero-sized blockIds array"); - } this.client = client; this.blockIds = blockIds; this.listener = listener; @@ -82,6 +79,10 @@ public void onFailure(int chunkIndex, Throwable e) { * {@link ShuffleStreamHandle}. We will send all fetch requests immediately, without throttling. */ public void start(Object openBlocksMessage) { + if (blockIds.length == 0) { + throw new IllegalArgumentException("Zero-sized blockIds array"); + } + client.sendRpc(JavaUtils.serialize(openBlocksMessage), new RpcResponseCallback() { @Override public void onSuccess(byte[] response) { @@ -95,7 +96,7 @@ public void onSuccess(byte[] response) { client.fetchChunk(streamHandle.streamId, i, chunkCallback); } } catch (Exception e) { - logger.error("Failed while starting block fetches", e); + logger.error("Failed while starting block fetches after success", e); failRemainingBlocks(blockIds, e); } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java new file mode 100644 index 0000000000000..f8a1a266863bb --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -0,0 +1,234 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; +import java.util.Collections; +import java.util.LinkedHashSet; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.Uninterruptibles; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Wraps another BlockFetcher with the ability to automatically retry fetches which fail due to + * IOExceptions, which we hope are due to transient network conditions. + * + * This fetcher provides stronger guarantees regarding the parent BlockFetchingListener. In + * particular, the listener will be invoked exactly once per blockId, with a success or failure. + */ +public class RetryingBlockFetcher { + + /** + * Used to initiate the first fetch for all blocks, and subsequently for retrying the fetch on any + * remaining blocks. + */ + public static interface BlockFetchStarter { + /** + * Creates a new BlockFetcher to fetch the given block ids which may do some synchronous + * bootstrapping followed by fully asynchronous block fetching. + * The BlockFetcher must eventually invoke the Listener on every input blockId, or else this + * method must throw an exception. + * + * This method should always attempt to get a new TransportClient from the + * {@link org.apache.spark.network.client.TransportClientFactory} in order to fix connection + * issues. + */ + void createAndStart(String[] blockIds, BlockFetchingListener listener) throws IOException; + } + + /** Shared executor service used for waiting and retrying. */ + private static final ExecutorService executorService = Executors.newCachedThreadPool( + NettyUtils.createThreadFactory("Block Fetch Retry")); + + private final Logger logger = LoggerFactory.getLogger(RetryingBlockFetcher.class); + + /** Used to initiate new Block Fetches on our remaining blocks. */ + private final BlockFetchStarter fetchStarter; + + /** Parent listener which we delegate all successful or permanently failed block fetches to. */ + private final BlockFetchingListener listener; + + /** Max number of times we are allowed to retry. */ + private final int maxRetries; + + /** Milliseconds to wait before each retry. */ + private final int retryWaitTime; + + // NOTE: + // All of our non-final fields are synchronized under 'this' and should only be accessed/mutated + // while inside a synchronized block. + /** Number of times we've attempted to retry so far. */ + private int retryCount = 0; + + /** + * Set of all block ids which have not been fetched successfully or with a non-IO Exception. + * A retry involves requesting every outstanding block. Note that since this is a LinkedHashSet, + * input ordering is preserved, so we always request blocks in the same order the user provided. + */ + private final LinkedHashSet outstandingBlocksIds; + + /** + * The BlockFetchingListener that is active with our current BlockFetcher. + * When we start a retry, we immediately replace this with a new Listener, which causes all any + * old Listeners to ignore all further responses. + */ + private RetryingBlockFetchListener currentListener; + + public RetryingBlockFetcher( + TransportConf conf, + BlockFetchStarter fetchStarter, + String[] blockIds, + BlockFetchingListener listener) { + this.fetchStarter = fetchStarter; + this.listener = listener; + this.maxRetries = conf.maxIORetries(); + this.retryWaitTime = conf.ioRetryWaitTime(); + this.outstandingBlocksIds = Sets.newLinkedHashSet(); + Collections.addAll(outstandingBlocksIds, blockIds); + this.currentListener = new RetryingBlockFetchListener(); + } + + /** + * Initiates the fetch of all blocks provided in the constructor, with possible retries in the + * event of transient IOExceptions. + */ + public void start() { + fetchAllOutstanding(); + } + + /** + * Fires off a request to fetch all blocks that have not been fetched successfully or permanently + * failed (i.e., by a non-IOException). + */ + private void fetchAllOutstanding() { + // Start by retrieving our shared state within a synchronized block. + String[] blockIdsToFetch; + int numRetries; + RetryingBlockFetchListener myListener; + synchronized (this) { + blockIdsToFetch = outstandingBlocksIds.toArray(new String[outstandingBlocksIds.size()]); + numRetries = retryCount; + myListener = currentListener; + } + + // Now initiate the fetch on all outstanding blocks, possibly initiating a retry if that fails. + try { + fetchStarter.createAndStart(blockIdsToFetch, myListener); + } catch (Exception e) { + logger.error(String.format("Exception while beginning fetch of %s outstanding blocks %s", + blockIdsToFetch.length, numRetries > 0 ? "(after " + numRetries + " retries)" : ""), e); + + if (shouldRetry(e)) { + initiateRetry(); + } else { + for (String bid : blockIdsToFetch) { + listener.onBlockFetchFailure(bid, e); + } + } + } + } + + /** + * Lightweight method which initiates a retry in a different thread. The retry will involve + * calling fetchAllOutstanding() after a configured wait time. + */ + private synchronized void initiateRetry() { + retryCount += 1; + currentListener = new RetryingBlockFetchListener(); + + logger.info("Retrying fetch ({}/{}) for {} outstanding blocks after {} ms", + retryCount, maxRetries, outstandingBlocksIds.size(), retryWaitTime); + + executorService.submit(new Runnable() { + @Override + public void run() { + Uninterruptibles.sleepUninterruptibly(retryWaitTime, TimeUnit.MILLISECONDS); + fetchAllOutstanding(); + } + }); + } + + /** + * Returns true if we should retry due a block fetch failure. We will retry if and only if + * the exception was an IOException and we haven't retried 'maxRetries' times already. + */ + private synchronized boolean shouldRetry(Throwable e) { + boolean isIOException = e instanceof IOException + || (e.getCause() != null && e.getCause() instanceof IOException); + boolean hasRemainingRetries = retryCount < maxRetries; + return isIOException && hasRemainingRetries; + } + + /** + * Our RetryListener intercepts block fetch responses and forwards them to our parent listener. + * Note that in the event of a retry, we will immediately replace the 'currentListener' field, + * indicating that any responses from non-current Listeners should be ignored. + */ + private class RetryingBlockFetchListener implements BlockFetchingListener { + @Override + public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { + // We will only forward this success message to our parent listener if this block request is + // outstanding and we are still the active listener. + boolean shouldForwardSuccess = false; + synchronized (RetryingBlockFetcher.this) { + if (this == currentListener && outstandingBlocksIds.contains(blockId)) { + outstandingBlocksIds.remove(blockId); + shouldForwardSuccess = true; + } + } + + // Now actually invoke the parent listener, outside of the synchronized block. + if (shouldForwardSuccess) { + listener.onBlockFetchSuccess(blockId, data); + } + } + + @Override + public void onBlockFetchFailure(String blockId, Throwable exception) { + // We will only forward this failure to our parent listener if this block request is + // outstanding, we are still the active listener, AND we cannot retry the fetch. + boolean shouldForwardFailure = false; + synchronized (RetryingBlockFetcher.this) { + if (this == currentListener && outstandingBlocksIds.contains(blockId)) { + if (shouldRetry(exception)) { + initiateRetry(); + } else { + logger.error(String.format("Failed to fetch block %s, and will not retry (%s retries)", + blockId, retryCount), exception); + outstandingBlocksIds.remove(blockId); + shouldForwardFailure = true; + } + } + } + + // Now actually invoke the parent listener, outside of the synchronized block. + if (shouldForwardFailure) { + listener.onBlockFetchFailure(blockId, exception); + } + } + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index 84781207861ed..d25283e46ef96 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -93,7 +93,7 @@ public void afterEach() { } @Test - public void testGoodClient() { + public void testGoodClient() throws IOException { clientFactory = context.createClientFactory( Lists.newArrayList( new SaslClientBootstrap(conf, "app-id", new TestSecretKeyHolder("good-key")))); @@ -119,7 +119,7 @@ public void testBadClient() { } @Test - public void testNoSaslClient() { + public void testNoSaslClient() throws IOException { clientFactory = context.createClientFactory( Lists.newArrayList()); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 71e017b9e4e74..06294fef19621 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -259,14 +259,20 @@ public void testFetchUnregisteredExecutor() throws Exception { @Test public void testFetchNoServer() throws Exception { - registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-0", - new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }, 1 /* port */); - assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + System.setProperty("spark.shuffle.io.maxRetries", "0"); + try { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-0", + new String[]{"shuffle_1_0_0", "shuffle_1_0_1"}, 1 /* port */); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + } finally { + System.clearProperty("spark.shuffle.io.maxRetries"); + } } - private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) { + private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) + throws IOException { ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false); client.init(APP_ID); client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 4c18fcdfbcd88..848c88f743d50 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -17,6 +17,8 @@ package org.apache.spark.network.shuffle; +import java.io.IOException; + import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -54,7 +56,7 @@ public void afterEach() { } @Test - public void testValid() { + public void testValid() throws IOException { validate("my-app-id", "secret"); } @@ -77,7 +79,7 @@ public void testBadSecret() { } /** Creates an ExternalShuffleClient and attempts to register with the server. */ - private void validate(String appId, String secretKey) { + private void validate(String appId, String secretKey) throws IOException { ExternalShuffleClient client = new ExternalShuffleClient(conf, new TestSecretKeyHolder(appId, secretKey), true); client.init(appId); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java new file mode 100644 index 0000000000000..0191fe529e1be --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/RetryingBlockFetcherSuite.java @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.LinkedHashSet; +import java.util.Map; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; +import org.mockito.stubbing.Stubber; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.TransportConf; +import static org.apache.spark.network.shuffle.RetryingBlockFetcher.BlockFetchStarter; + +/** + * Tests retry logic by throwing IOExceptions and ensuring that subsequent attempts are made to + * fetch the lost blocks. + */ +public class RetryingBlockFetcherSuite { + + ManagedBuffer block0 = new NioManagedBuffer(ByteBuffer.wrap(new byte[13])); + ManagedBuffer block1 = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); + ManagedBuffer block2 = new NioManagedBuffer(ByteBuffer.wrap(new byte[19])); + + @Before + public void beforeEach() { + System.setProperty("spark.shuffle.io.maxRetries", "2"); + System.setProperty("spark.shuffle.io.retryWaitMs", "0"); + } + + @After + public void afterEach() { + System.clearProperty("spark.shuffle.io.maxRetries"); + System.clearProperty("spark.shuffle.io.retryWaitMs"); + } + + @Test + public void testNoFailures() throws IOException { + BlockFetchingListener listener = mock(BlockFetchingListener.class); + + Map[] interactions = new Map[] { + // Immediately return both blocks successfully. + ImmutableMap.builder() + .put("b0", block0) + .put("b1", block1) + .build(), + }; + + performInteractions(interactions, listener); + + verify(listener).onBlockFetchSuccess("b0", block0); + verify(listener).onBlockFetchSuccess("b1", block1); + verifyNoMoreInteractions(listener); + } + + @Test + public void testUnrecoverableFailure() throws IOException { + BlockFetchingListener listener = mock(BlockFetchingListener.class); + + Map[] interactions = new Map[] { + // b0 throws a non-IOException error, so it will be failed without retry. + ImmutableMap.builder() + .put("b0", new RuntimeException("Ouch!")) + .put("b1", block1) + .build(), + }; + + performInteractions(interactions, listener); + + verify(listener).onBlockFetchFailure(eq("b0"), (Throwable) any()); + verify(listener).onBlockFetchSuccess("b1", block1); + verifyNoMoreInteractions(listener); + } + + @Test + public void testSingleIOExceptionOnFirst() throws IOException { + BlockFetchingListener listener = mock(BlockFetchingListener.class); + + Map[] interactions = new Map[] { + // IOException will cause a retry. Since b0 fails, we will retry both. + ImmutableMap.builder() + .put("b0", new IOException("Connection failed or something")) + .put("b1", block1) + .build(), + ImmutableMap.builder() + .put("b0", block0) + .put("b1", block1) + .build(), + }; + + performInteractions(interactions, listener); + + verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); + verify(listener, timeout(5000)).onBlockFetchSuccess("b1", block1); + verifyNoMoreInteractions(listener); + } + + @Test + public void testSingleIOExceptionOnSecond() throws IOException { + BlockFetchingListener listener = mock(BlockFetchingListener.class); + + Map[] interactions = new Map[] { + // IOException will cause a retry. Since b1 fails, we will not retry b0. + ImmutableMap.builder() + .put("b0", block0) + .put("b1", new IOException("Connection failed or something")) + .build(), + ImmutableMap.builder() + .put("b1", block1) + .build(), + }; + + performInteractions(interactions, listener); + + verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); + verify(listener, timeout(5000)).onBlockFetchSuccess("b1", block1); + verifyNoMoreInteractions(listener); + } + + @Test + public void testTwoIOExceptions() throws IOException { + BlockFetchingListener listener = mock(BlockFetchingListener.class); + + Map[] interactions = new Map[] { + // b0's IOException will trigger retry, b1's will be ignored. + ImmutableMap.builder() + .put("b0", new IOException()) + .put("b1", new IOException()) + .build(), + // Next, b0 is successful and b1 errors again, so we just request that one. + ImmutableMap.builder() + .put("b0", block0) + .put("b1", new IOException()) + .build(), + // b1 returns successfully within 2 retries. + ImmutableMap.builder() + .put("b1", block1) + .build(), + }; + + performInteractions(interactions, listener); + + verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); + verify(listener, timeout(5000)).onBlockFetchSuccess("b1", block1); + verifyNoMoreInteractions(listener); + } + + @Test + public void testThreeIOExceptions() throws IOException { + BlockFetchingListener listener = mock(BlockFetchingListener.class); + + Map[] interactions = new Map[] { + // b0's IOException will trigger retry, b1's will be ignored. + ImmutableMap.builder() + .put("b0", new IOException()) + .put("b1", new IOException()) + .build(), + // Next, b0 is successful and b1 errors again, so we just request that one. + ImmutableMap.builder() + .put("b0", block0) + .put("b1", new IOException()) + .build(), + // b1 errors again, but this was the last retry + ImmutableMap.builder() + .put("b1", new IOException()) + .build(), + // This is not reached -- b1 has failed. + ImmutableMap.builder() + .put("b1", block1) + .build(), + }; + + performInteractions(interactions, listener); + + verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); + verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verifyNoMoreInteractions(listener); + } + + @Test + public void testRetryAndUnrecoverable() throws IOException { + BlockFetchingListener listener = mock(BlockFetchingListener.class); + + Map[] interactions = new Map[] { + // b0's IOException will trigger retry, subsequent messages will be ignored. + ImmutableMap.builder() + .put("b0", new IOException()) + .put("b1", new RuntimeException()) + .put("b2", block2) + .build(), + // Next, b0 is successful, b1 errors unrecoverably, and b2 triggers a retry. + ImmutableMap.builder() + .put("b0", block0) + .put("b1", new RuntimeException()) + .put("b2", new IOException()) + .build(), + // b2 succeeds in its last retry. + ImmutableMap.builder() + .put("b2", block2) + .build(), + }; + + performInteractions(interactions, listener); + + verify(listener, timeout(5000)).onBlockFetchSuccess("b0", block0); + verify(listener, timeout(5000)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, timeout(5000)).onBlockFetchSuccess("b2", block2); + verifyNoMoreInteractions(listener); + } + + /** + * Performs a set of interactions in response to block requests from a RetryingBlockFetcher. + * Each interaction is a Map from BlockId to either ManagedBuffer or Exception. This interaction + * means "respond to the next block fetch request with these Successful buffers and these Failure + * exceptions". We verify that the expected block ids are exactly the ones requested. + * + * If multiple interactions are supplied, they will be used in order. This is useful for encoding + * retries -- the first interaction may include an IOException, which causes a retry of some + * subset of the original blocks in a second interaction. + */ + @SuppressWarnings("unchecked") + private void performInteractions(final Map[] interactions, BlockFetchingListener listener) + throws IOException { + + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + BlockFetchStarter fetchStarter = mock(BlockFetchStarter.class); + + Stubber stub = null; + + // Contains all blockIds that are referenced across all interactions. + final LinkedHashSet blockIds = Sets.newLinkedHashSet(); + + for (final Map interaction : interactions) { + blockIds.addAll(interaction.keySet()); + + Answer answer = new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + try { + // Verify that the RetryingBlockFetcher requested the expected blocks. + String[] requestedBlockIds = (String[]) invocationOnMock.getArguments()[0]; + String[] desiredBlockIds = interaction.keySet().toArray(new String[interaction.size()]); + assertArrayEquals(desiredBlockIds, requestedBlockIds); + + // Now actually invoke the success/failure callbacks on each block. + BlockFetchingListener retryListener = + (BlockFetchingListener) invocationOnMock.getArguments()[1]; + for (Map.Entry block : interaction.entrySet()) { + String blockId = block.getKey(); + Object blockValue = block.getValue(); + + if (blockValue instanceof ManagedBuffer) { + retryListener.onBlockFetchSuccess(blockId, (ManagedBuffer) blockValue); + } else if (blockValue instanceof Exception) { + retryListener.onBlockFetchFailure(blockId, (Exception) blockValue); + } else { + fail("Can only handle ManagedBuffers and Exceptions, got " + blockValue); + } + } + return null; + } catch (Throwable e) { + e.printStackTrace(); + throw e; + } + } + }; + + // This is either the first stub, or should be chained behind the prior ones. + if (stub == null) { + stub = doAnswer(answer); + } else { + stub.doAnswer(answer); + } + } + + assert stub != null; + stub.when(fetchStarter).createAndStart((String[]) any(), (BlockFetchingListener) anyObject()); + String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]); + new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start(); + } +} From 48a19a6dba896f7d0b637f84e114b7efbb814e51 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 6 Nov 2014 19:54:32 -0800 Subject: [PATCH 1243/1492] [SPARK-4236] Cleanup removed applications' files in shuffle service This relies on a hook from whoever is hosting the shuffle service to invoke removeApplication() when the application is completed. Once invoked, we will clean up all the executors' shuffle directories we know about. Author: Aaron Davidson Closes #3126 from aarondav/cleanup and squashes the following commits: 33a64a9 [Aaron Davidson] Missing brace e6e428f [Aaron Davidson] Address comments 16a0d27 [Aaron Davidson] Cleanup e4df3e7 [Aaron Davidson] [SPARK-4236] Cleanup removed applications' files in shuffle service --- .../scala/org/apache/spark/util/Utils.scala | 1 + .../spark/ExternalShuffleServiceSuite.scala | 5 +- .../apache/spark/network/util/JavaUtils.java | 59 ++++++++ .../shuffle/ExternalShuffleBlockHandler.java | 10 +- .../shuffle/ExternalShuffleBlockManager.java | 118 +++++++++++++-- .../shuffle/ExternalShuffleCleanupSuite.java | 142 ++++++++++++++++++ .../ExternalShuffleIntegrationSuite.java | 2 +- .../shuffle/TestShuffleDataContext.java | 4 +- 8 files changed, 319 insertions(+), 22 deletions(-) create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java 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 7caf6bcf94ef3..2cbd38d72caa1 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -755,6 +755,7 @@ private[spark] object Utils extends Logging { /** * Delete a file or directory and its contents recursively. * Don't follow directories if they are symlinks. + * Throws an exception if deletion is unsuccessful. */ def deleteRecursively(file: File) { if (file != null) { diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 792b9cd8b6ff2..6608ed1e57b38 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -63,8 +63,9 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { rdd.count() rdd.count() - // Invalidate the registered executors, disallowing access to their shuffle blocks. - rpcHandler.clearRegisteredExecutors() + // Invalidate the registered executors, disallowing access to their shuffle blocks (without + // deleting the actual shuffle files, so we could access them without the shuffle service). + rpcHandler.applicationRemoved(sc.conf.getAppId, false /* cleanupLocalDirs */) // Now Spark will receive FetchFailed, and not retry the stage due to "spark.test.noStageRetry" // being set. diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 2856d1c8c9337..75c4a3981a240 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -22,16 +22,22 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.Closeable; +import java.io.File; import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import com.google.common.base.Preconditions; import com.google.common.io.Closeables; import com.google.common.base.Charsets; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * General utilities available in the network package. Many of these are sourced from Spark's + * own Utils, just accessible within this package. + */ public class JavaUtils { private static final Logger logger = LoggerFactory.getLogger(JavaUtils.class); @@ -93,4 +99,57 @@ public static ByteBuffer stringToBytes(String s) { public static String bytesToString(ByteBuffer b) { return Unpooled.wrappedBuffer(b).toString(Charsets.UTF_8); } + + /* + * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. + * Throws an exception if deletion is unsuccessful. + */ + public static void deleteRecursively(File file) throws IOException { + if (file == null) { return; } + + if (file.isDirectory() && !isSymlink(file)) { + IOException savedIOException = null; + for (File child : listFilesSafely(file)) { + try { + deleteRecursively(child); + } catch (IOException e) { + // In case of multiple exceptions, only last one will be thrown + savedIOException = e; + } + } + if (savedIOException != null) { + throw savedIOException; + } + } + + boolean deleted = file.delete(); + // Delete can also fail if the file simply did not exist. + if (!deleted && file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath()); + } + } + + private static File[] listFilesSafely(File file) throws IOException { + if (file.exists()) { + File[] files = file.listFiles(); + if (files == null) { + throw new IOException("Failed to list files for dir: " + file); + } + return files; + } else { + return new File[0]; + } + } + + private static boolean isSymlink(File file) throws IOException { + Preconditions.checkNotNull(file); + File fileInCanonicalDir = null; + if (file.getParent() == null) { + fileInCanonicalDir = file; + } else { + fileInCanonicalDir = new File(file.getParentFile().getCanonicalFile(), file.getName()); + } + return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index cd3fea85b19a4..75ebf8c7b0604 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -94,9 +94,11 @@ public StreamManager getStreamManager() { return streamManager; } - /** For testing, clears all executors registered with "RegisterExecutor". */ - @VisibleForTesting - public void clearRegisteredExecutors() { - blockManager.clearRegisteredExecutors(); + /** + * Removes an application (once it has been terminated), and optionally will clean up any + * local directories associated with the executors of that application in a separate thread. + */ + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + blockManager.applicationRemoved(appId, cleanupLocalDirs); } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java index 6589889fe1be7..98fcfb82aa5d1 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java @@ -21,9 +21,15 @@ import java.io.File; import java.io.FileInputStream; import java.io.IOException; -import java.util.concurrent.ConcurrentHashMap; +import java.util.Iterator; +import java.util.Map; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.Executor; +import java.util.concurrent.Executors; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Objects; +import com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,13 +49,22 @@ public class ExternalShuffleBlockManager { private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class); - // Map from "appId-execId" to the executor's configuration. - private final ConcurrentHashMap executors = - new ConcurrentHashMap(); + // Map containing all registered executors' metadata. + private final ConcurrentMap executors; - // Returns an id suitable for a single executor within a single application. - private String getAppExecId(String appId, String execId) { - return appId + "-" + execId; + // Single-threaded Java executor used to perform expensive recursive directory deletion. + private final Executor directoryCleaner; + + public ExternalShuffleBlockManager() { + // TODO: Give this thread a name. + this(Executors.newSingleThreadExecutor()); + } + + // Allows tests to have more control over when directories are cleaned up. + @VisibleForTesting + ExternalShuffleBlockManager(Executor directoryCleaner) { + this.executors = Maps.newConcurrentMap(); + this.directoryCleaner = directoryCleaner; } /** Registers a new Executor with all the configuration we need to find its shuffle files. */ @@ -57,7 +72,7 @@ public void registerExecutor( String appId, String execId, ExecutorShuffleInfo executorInfo) { - String fullId = getAppExecId(appId, execId); + AppExecId fullId = new AppExecId(appId, execId); logger.info("Registered executor {} with {}", fullId, executorInfo); executors.put(fullId, executorInfo); } @@ -78,7 +93,7 @@ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { int mapId = Integer.parseInt(blockIdParts[2]); int reduceId = Integer.parseInt(blockIdParts[3]); - ExecutorShuffleInfo executor = executors.get(getAppExecId(appId, execId)); + ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); if (executor == null) { throw new RuntimeException( String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); @@ -94,6 +109,56 @@ public ManagedBuffer getBlockData(String appId, String execId, String blockId) { } } + /** + * Removes our metadata of all executors registered for the given application, and optionally + * also deletes the local directories associated with the executors of that application in a + * separate thread. + * + * It is not valid to call registerExecutor() for an executor with this appId after invoking + * this method. + */ + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); + Iterator> it = executors.entrySet().iterator(); + while (it.hasNext()) { + Map.Entry entry = it.next(); + AppExecId fullId = entry.getKey(); + final ExecutorShuffleInfo executor = entry.getValue(); + + // Only touch executors associated with the appId that was removed. + if (appId.equals(fullId.appId)) { + it.remove(); + + if (cleanupLocalDirs) { + logger.info("Cleaning up executor {}'s {} local dirs", fullId, executor.localDirs.length); + + // Execute the actual deletion in a different thread, as it may take some time. + directoryCleaner.execute(new Runnable() { + @Override + public void run() { + deleteExecutorDirs(executor.localDirs); + } + }); + } + } + } + } + + /** + * Synchronously deletes each directory one at a time. + * Should be executed in its own thread, as this may take a long time. + */ + private void deleteExecutorDirs(String[] dirs) { + for (String localDir : dirs) { + try { + JavaUtils.deleteRecursively(new File(localDir)); + logger.debug("Successfully cleaned up directory: " + localDir); + } catch (Exception e) { + logger.error("Failed to delete directory: " + localDir, e); + } + } + } + /** * Hash-based shuffle data is simply stored as one file per block. * This logic is from FileShuffleBlockManager. @@ -146,9 +211,36 @@ static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) return new File(new File(localDir, String.format("%02x", subDirId)), filename); } - /** For testing, clears all registered executors. */ - @VisibleForTesting - void clearRegisteredExecutors() { - executors.clear(); + /** Simply encodes an executor's full ID, which is appId + execId. */ + private static class AppExecId { + final String appId; + final String execId; + + private AppExecId(String appId, String execId) { + this.appId = appId; + this.execId = execId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + AppExecId appExecId = (AppExecId) o; + return Objects.equal(appId, appExecId.appId) && Objects.equal(execId, appExecId.execId); + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .toString(); + } } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java new file mode 100644 index 0000000000000..c8ece3bc53ac3 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.File; +import java.io.IOException; +import java.util.Random; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; + +import com.google.common.util.concurrent.MoreExecutors; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ExternalShuffleCleanupSuite { + + // Same-thread Executor used to ensure cleanup happens synchronously in test thread. + Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); + + @Test + public void noCleanupAndCleanup() throws IOException { + TestShuffleDataContext dataContext = createSomeData(); + + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(sameThreadExecutor); + manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); + manager.applicationRemoved("app", false /* cleanup */); + + assertStillThere(dataContext); + + manager.registerExecutor("app", "exec1", dataContext.createExecutorInfo("shuffleMgr")); + manager.applicationRemoved("app", true /* cleanup */); + + assertCleanedUp(dataContext); + } + + @Test + public void cleanupUsesExecutor() throws IOException { + TestShuffleDataContext dataContext = createSomeData(); + + final AtomicBoolean cleanupCalled = new AtomicBoolean(false); + + // Executor which does nothing to ensure we're actually using it. + Executor noThreadExecutor = new Executor() { + @Override public void execute(Runnable runnable) { cleanupCalled.set(true); } + }; + + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(noThreadExecutor); + + manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); + manager.applicationRemoved("app", true); + + assertTrue(cleanupCalled.get()); + assertStillThere(dataContext); + + dataContext.cleanup(); + assertCleanedUp(dataContext); + } + + @Test + public void cleanupMultipleExecutors() throws IOException { + TestShuffleDataContext dataContext0 = createSomeData(); + TestShuffleDataContext dataContext1 = createSomeData(); + + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(sameThreadExecutor); + + manager.registerExecutor("app", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); + manager.registerExecutor("app", "exec1", dataContext1.createExecutorInfo("shuffleMgr")); + manager.applicationRemoved("app", true); + + assertCleanedUp(dataContext0); + assertCleanedUp(dataContext1); + } + + @Test + public void cleanupOnlyRemovedApp() throws IOException { + TestShuffleDataContext dataContext0 = createSomeData(); + TestShuffleDataContext dataContext1 = createSomeData(); + + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(sameThreadExecutor); + + manager.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); + manager.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo("shuffleMgr")); + + manager.applicationRemoved("app-nonexistent", true); + assertStillThere(dataContext0); + assertStillThere(dataContext1); + + manager.applicationRemoved("app-0", true); + assertCleanedUp(dataContext0); + assertStillThere(dataContext1); + + manager.applicationRemoved("app-1", true); + assertCleanedUp(dataContext0); + assertCleanedUp(dataContext1); + + // Make sure it's not an error to cleanup multiple times + manager.applicationRemoved("app-1", true); + assertCleanedUp(dataContext0); + assertCleanedUp(dataContext1); + } + + private void assertStillThere(TestShuffleDataContext dataContext) { + for (String localDir : dataContext.localDirs) { + assertTrue(localDir + " was cleaned up prematurely", new File(localDir).exists()); + } + } + + private void assertCleanedUp(TestShuffleDataContext dataContext) { + for (String localDir : dataContext.localDirs) { + assertFalse(localDir + " wasn't cleaned up", new File(localDir).exists()); + } + } + + private TestShuffleDataContext createSomeData() throws IOException { + Random rand = new Random(123); + TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); + + dataContext.create(); + dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), + new byte[][] { "ABC".getBytes(), "DEF".getBytes() } ); + dataContext.insertHashShuffleData(rand.nextInt(1000), rand.nextInt(1000) + 1000, + new byte[][] { "GHI".getBytes(), "JKLMNOPQRSTUVWXYZ".getBytes() } ); + return dataContext; + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 06294fef19621..3bea5b0f253c6 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -105,7 +105,7 @@ public static void afterAll() { @After public void afterEach() { - handler.clearRegisteredExecutors(); + handler.applicationRemoved(APP_ID, false /* cleanupLocalDirs */); } class FetchResult { diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 442b756467442..337b5c7bdb5da 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -30,8 +30,8 @@ * and cleanup of directories that can be read by the {@link ExternalShuffleBlockManager}. */ public class TestShuffleDataContext { - private final String[] localDirs; - private final int subDirsPerLocalDir; + public final String[] localDirs; + public final int subDirsPerLocalDir; public TestShuffleDataContext(int numLocalDirs, int subDirsPerLocalDir) { this.localDirs = new String[numLocalDirs]; From 3abdb1b24aa48f21e7eed1232c01d3933873688c Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 6 Nov 2014 21:52:12 -0800 Subject: [PATCH 1244/1492] [SPARK-4204][Core][WebUI] Change Utils.exceptionString to contain the inner exceptions and make the error information in Web UI more friendly This PR fixed `Utils.exceptionString` to output the full exception information. However, the stack trace may become very huge, so I also updated the Web UI to collapse the error information by default (display the first line and clicking `+detail` will display the full info). Here are the screenshots: Stages: ![stages](https://cloud.githubusercontent.com/assets/1000778/4882441/66d8cc68-6356-11e4-8346-6318677d9470.png) Details for one stage: ![stage](https://cloud.githubusercontent.com/assets/1000778/4882513/1311043c-6357-11e4-8804-ca14240a9145.png) The full information in the gray text field is: ```Java org.apache.spark.shuffle.FetchFailedException: Connection reset by peer at org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher$.org$apache$spark$shuffle$hash$BlockStoreShuffleFetcher$$unpackBlock$1(BlockStoreShuffleFetcher.scala:67) at org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher$$anonfun$3.apply(BlockStoreShuffleFetcher.scala:83) at org.apache.spark.shuffle.hash.BlockStoreShuffleFetcher$$anonfun$3.apply(BlockStoreShuffleFetcher.scala:83) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:30) at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39) at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327) at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:327) at org.apache.spark.util.collection.ExternalAppendOnlyMap.insertAll(ExternalAppendOnlyMap.scala:129) at org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$5.apply(CoGroupedRDD.scala:160) at org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$5.apply(CoGroupedRDD.scala:159) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771) at org.apache.spark.rdd.CoGroupedRDD.compute(CoGroupedRDD.scala:159) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263) at org.apache.spark.rdd.RDD.iterator(RDD.scala:230) at org.apache.spark.rdd.MappedValuesRDD.compute(MappedValuesRDD.scala:31) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263) at org.apache.spark.rdd.RDD.iterator(RDD.scala:230) at org.apache.spark.rdd.FlatMappedValuesRDD.compute(FlatMappedValuesRDD.scala:31) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:263) at org.apache.spark.rdd.RDD.iterator(RDD.scala:230) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61) at org.apache.spark.scheduler.Task.run(Task.scala:56) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:189) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) Caused by: java.io.IOException: Connection reset by peer at sun.nio.ch.FileDispatcher.read0(Native Method) at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:21) at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:198) at sun.nio.ch.IOUtil.read(IOUtil.java:166) at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:245) at io.netty.buffer.PooledUnsafeDirectByteBuf.setBytes(PooledUnsafeDirectByteBuf.java:311) at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:881) at io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:225) at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119) at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511) at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468) at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382) at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354) at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) ... 1 more ``` /cc aarondav Author: zsxwing Closes #3073 from zsxwing/SPARK-4204 and squashes the following commits: 176d1e3 [zsxwing] Add comments to explain the stack trace difference ca509d3 [zsxwing] Add fullStackTrace to the constructor of ExceptionFailure a07057b [zsxwing] Core style fix dfb0032 [zsxwing] Backward compatibility for old history server 1e50f71 [zsxwing] Update as per review and increase the max height of the stack trace details 94f2566 [zsxwing] Change Utils.exceptionString to contain the inner exceptions and make the error information in Web UI more friendly --- .../org/apache/spark/ui/static/webui.css | 14 ++++++++ .../org/apache/spark/TaskEndReason.scala | 35 ++++++++++++++++++- .../org/apache/spark/executor/Executor.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 4 +-- .../spark/shuffle/FetchFailedException.scala | 17 +++++++-- .../hash/BlockStoreShuffleFetcher.scala | 5 ++- .../org/apache/spark/ui/jobs/StagePage.scala | 32 +++++++++++++++-- .../org/apache/spark/ui/jobs/StageTable.scala | 28 +++++++++++++-- .../org/apache/spark/util/JsonProtocol.scala | 5 ++- .../scala/org/apache/spark/util/Utils.scala | 24 ++++++------- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 10 +++++- 12 files changed, 148 insertions(+), 30 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index a2220e761ac98..db57712c83503 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -120,6 +120,20 @@ pre { border: none; } +.stacktrace-details { + max-height: 300px; + overflow-y: auto; + margin: 0; + transition: max-height 0.5s ease-out, padding 0.5s ease-out; +} + +.stacktrace-details.collapsed { + max-height: 0; + padding-top: 0; + padding-bottom: 0; + border: none; +} + span.expand-additional-metrics { cursor: pointer; } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index f45b463fb6f62..af5fd8e0ac00c 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -83,15 +83,48 @@ case class FetchFailed( * :: DeveloperApi :: * Task failed due to a runtime exception. This is the most common failure case and also captures * user program exceptions. + * + * `stackTrace` contains the stack trace of the exception itself. It still exists for backward + * compatibility. It's better to use `this(e: Throwable, metrics: Option[TaskMetrics])` to + * create `ExceptionFailure` as it will handle the backward compatibility properly. + * + * `fullStackTrace` is a better representation of the stack trace because it contains the whole + * stack trace including the exception and its causes */ @DeveloperApi case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], + fullStackTrace: String, metrics: Option[TaskMetrics]) extends TaskFailedReason { - override def toErrorString: String = Utils.exceptionString(className, description, stackTrace) + + private[spark] def this(e: Throwable, metrics: Option[TaskMetrics]) { + this(e.getClass.getName, e.getMessage, e.getStackTrace, Utils.exceptionString(e), metrics) + } + + override def toErrorString: String = + if (fullStackTrace == null) { + // fullStackTrace is added in 1.2.0 + // If fullStackTrace is null, use the old error string for backward compatibility + exceptionString(className, description, stackTrace) + } else { + fullStackTrace + } + + /** + * Return a nice string representation of the exception, including the stack trace. + * Note: It does not include the exception's causes, and is only used for backward compatibility. + */ + private def exceptionString( + className: String, + description: String, + stackTrace: Array[StackTraceElement]): String = { + val desc = if (description == null) "" else description + val st = if (stackTrace == null) "" else stackTrace.map(" " + _).mkString("\n") + s"$className: $desc\n$st" + } } /** diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 96114571d6c77..caf4d76713d49 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -263,7 +263,7 @@ private[spark] class Executor( m.executorRunTime = serviceTime m.jvmGCTime = gcTime - startGCTime } - val reason = ExceptionFailure(t.getClass.getName, t.getMessage, t.getStackTrace, metrics) + val reason = new ExceptionFailure(t, metrics) execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) // Don't forcibly exit unless the exception was inherently fatal, to avoid diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 96114c0423a9e..22449517d100f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1063,7 +1063,7 @@ class DAGScheduler( if (runningStages.contains(failedStage)) { logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + s"due to a fetch failure from $mapStage (${mapStage.name})") - markStageAsFinished(failedStage, Some("Fetch failure: " + failureMessage)) + markStageAsFinished(failedStage, Some(failureMessage)) runningStages -= failedStage } @@ -1094,7 +1094,7 @@ class DAGScheduler( handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) } - case ExceptionFailure(className, description, stackTrace, metrics) => + case ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics) => // Do nothing here, left up to the TaskScheduler to decide how to handle user failures case TaskResultLost => diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index 0c1b6f4defdb3..be184464e0ae9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -32,10 +32,21 @@ private[spark] class FetchFailedException( shuffleId: Int, mapId: Int, reduceId: Int, - message: String) - extends Exception(message) { + message: String, + cause: Throwable = null) + extends Exception(message, cause) { + + def this( + bmAddress: BlockManagerId, + shuffleId: Int, + mapId: Int, + reduceId: Int, + cause: Throwable) { + this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) + } - def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, message) + def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, + Utils.exceptionString(this)) } /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 0d5247f4176d4..e3e7434df45b0 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -25,7 +25,7 @@ import org.apache.spark._ import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} -import org.apache.spark.util.{CompletionIterator, Utils} +import org.apache.spark.util.CompletionIterator private[hash] object BlockStoreShuffleFetcher extends Logging { def fetch[T]( @@ -64,8 +64,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { blockId match { case ShuffleBlockId(shufId, mapId, _) => val address = statuses(mapId.toInt)._1 - throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, - Utils.exceptionString(e)) + throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) case _ => throw new SparkException( "Failed to get block " + blockId + ", which is not a shuffle block", e) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 63ed5fc4949c2..250bddbe2f262 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,6 +22,8 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Unparsed} +import org.apache.commons.lang3.StringEscapeUtils + import org.apache.spark.executor.TaskMetrics import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData._ @@ -436,13 +438,37 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { {diskBytesSpilledReadable} }} - - {errorMessage.map { e =>
    {e}
    }.getOrElse("")} - + {errorMessageCell(errorMessage)} } } + private def errorMessageCell(errorMessage: Option[String]): Seq[Node] = { + val error = errorMessage.getOrElse("") + val isMultiline = error.indexOf('\n') >= 0 + // Display the first line by default + val errorSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + error.substring(0, error.indexOf('\n')) + } else { + error + }) + val details = if (isMultiline) { + // scalastyle:off + + +details + ++ + + // scalastyle:on + } else { + "" + } + {errorSummary}{details} + } + private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { val totalExecutionTime = { if (info.gettingResultTime > 0) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 4ee7f08ab47a2..3b4866e05956d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -22,6 +22,8 @@ import scala.xml.Text import java.util.Date +import org.apache.commons.lang3.StringEscapeUtils + import org.apache.spark.scheduler.StageInfo import org.apache.spark.ui.{ToolTips, UIUtils} import org.apache.spark.util.Utils @@ -195,7 +197,29 @@ private[ui] class FailedStageTable( override protected def stageRow(s: StageInfo): Seq[Node] = { val basicColumns = super.stageRow(s) - val failureReason =
    {s.failureReason.getOrElse("")}
    - basicColumns ++ failureReason + val failureReason = s.failureReason.getOrElse("") + val isMultiline = failureReason.indexOf('\n') >= 0 + // Display the first line by default + val failureReasonSummary = StringEscapeUtils.escapeHtml4( + if (isMultiline) { + failureReason.substring(0, failureReason.indexOf('\n')) + } else { + failureReason + }) + val details = if (isMultiline) { + // scalastyle:off + + +details + ++ + + // scalastyle:on + } else { + "" + } + val failureReasonHtml = {failureReasonSummary}{details} + basicColumns ++ failureReasonHtml } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f7ae1f7f334de..f15d0c856663f 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -287,6 +287,7 @@ private[spark] object JsonProtocol { ("Class Name" -> exceptionFailure.className) ~ ("Description" -> exceptionFailure.description) ~ ("Stack Trace" -> stackTrace) ~ + ("Full Stack Trace" -> exceptionFailure.fullStackTrace) ~ ("Metrics" -> metrics) case ExecutorLostFailure(executorId) => ("Executor ID" -> executorId) @@ -637,8 +638,10 @@ private[spark] object JsonProtocol { val className = (json \ "Class Name").extract[String] val description = (json \ "Description").extract[String] val stackTrace = stackTraceFromJson(json \ "Stack Trace") + val fullStackTrace = Utils.jsonOption(json \ "Full Stack Trace"). + map(_.extract[String]).orNull val metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) - new ExceptionFailure(className, description, stackTrace, metrics) + ExceptionFailure(className, description, stackTrace, fullStackTrace, metrics) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled case `executorLostFailure` => 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 2cbd38d72caa1..a14d6125484fe 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1599,19 +1599,19 @@ private[spark] object Utils extends Logging { .orNull } - /** Return a nice string representation of the exception, including the stack trace. */ + /** + * Return a nice string representation of the exception. It will call "printStackTrace" to + * recursively generate the stack trace including the exception and its causes. + */ def exceptionString(e: Throwable): String = { - if (e == null) "" else exceptionString(getFormattedClassName(e), e.getMessage, e.getStackTrace) - } - - /** Return a nice string representation of the exception, including the stack trace. */ - def exceptionString( - className: String, - description: String, - stackTrace: Array[StackTraceElement]): String = { - val desc = if (description == null) "" else description - val st = if (stackTrace == null) "" else stackTrace.map(" " + _).mkString("\n") - s"$className: $desc\n$st" + if (e == null) { + "" + } else { + // Use e.printStackTrace here because e.getStackTrace doesn't include the cause + val stringWriter = new StringWriter() + e.printStackTrace(new PrintWriter(stringWriter)) + stringWriter.toString + } } /** Return a thread dump of all threads' stacktraces. Used to capture dumps for the web UI */ diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 2efbae689771a..2608ad4b32e1e 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -116,7 +116,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val taskFailedReasons = Seq( Resubmitted, new FetchFailed(null, 0, 0, 0, "ignored"), - new ExceptionFailure("Exception", "description", null, None), + ExceptionFailure("Exception", "description", null, null, None), TaskResultLost, TaskKilled, ExecutorLostFailure("0"), diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index aec1e409db95c..39e69851e7e3c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -109,7 +109,7 @@ class JsonProtocolSuite extends FunSuite { // TaskEndReason val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, "Some exception") - val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, None) + val exceptionFailure = new ExceptionFailure(exception, None) testTaskEndReason(Success) testTaskEndReason(Resubmitted) testTaskEndReason(fetchFailed) @@ -127,6 +127,13 @@ class JsonProtocolSuite extends FunSuite { testBlockId(StreamBlockId(1, 2L)) } + test("ExceptionFailure backward compatibility") { + val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, null, None) + val oldEvent = JsonProtocol.taskEndReasonToJson(exceptionFailure) + .removeField({ _._1 == "Full Stack Trace" }) + assertEquals(exceptionFailure, JsonProtocol.taskEndReasonFromJson(oldEvent)) + } + test("StageInfo backward compatibility") { val info = makeStageInfo(1, 2, 3, 4L, 5L) val newJson = JsonProtocol.stageInfoToJson(info) @@ -422,6 +429,7 @@ class JsonProtocolSuite extends FunSuite { assert(r1.className === r2.className) assert(r1.description === r2.description) assertSeqEquals(r1.stackTrace, r2.stackTrace, assertStackTraceElementEquals) + assert(r1.fullStackTrace === r2.fullStackTrace) assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals) case (TaskResultLost, TaskResultLost) => case (TaskKilled, TaskKilled) => From d4fa04e50d299e9cad349b3781772956453a696b Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 7 Nov 2014 09:42:21 -0800 Subject: [PATCH 1245/1492] [SPARK-4187] [Core] Switch to binary protocol for external shuffle service messages This PR elimiantes the network package's usage of the Java serializer and replaces it with Encodable, which is a lightweight binary protocol. Each message is preceded by a type id, which will allow us to change messages (by only adding new ones), or to change the format entirely by switching to a special id (such as -1). This protocol has the advantage over Java that we can guarantee that messages will remain compatible across compiled versions and JVMs, though it does not provide a clean way to do schema migration. In the future, it may be good to use a more heavy-weight serialization format like protobuf, thrift, or avro, but these all add several dependencies which are unnecessary at the present time. Additionally this unifies the RPC messages of NettyBlockTransferService and ExternalShuffleClient. Author: Aaron Davidson Closes #3146 from aarondav/free and squashes the following commits: ed1102a [Aaron Davidson] Remove some unused imports b8e2a49 [Aaron Davidson] Add appId to test 538f2a3 [Aaron Davidson] [SPARK-4187] [Core] Switch to binary protocol for external shuffle service messages --- .../spark/network/BlockTransferService.scala | 4 +- .../network/netty/NettyBlockRpcServer.scala | 31 ++--- .../netty/NettyBlockTransferService.scala | 15 ++- .../network/nio/NioBlockTransferService.scala | 1 + .../apache/spark/storage/BlockManager.scala | 5 +- .../NettyBlockTransferSecuritySuite.scala | 4 +- .../network/protocol/ChunkFetchFailure.java | 12 +- .../spark/network/protocol/Encoders.java | 93 ++++++++++++++ .../spark/network/protocol/RpcFailure.java | 12 +- .../spark/network/protocol/RpcRequest.java | 9 +- .../spark/network/protocol/RpcResponse.java | 9 +- .../apache/spark/network/util/JavaUtils.java | 27 ----- .../spark/network/sasl/SaslMessage.java | 24 ++-- .../shuffle/ExternalShuffleBlockHandler.java | 21 ++-- .../shuffle/ExternalShuffleBlockManager.java | 1 + .../shuffle/ExternalShuffleClient.java | 12 +- .../shuffle/ExternalShuffleMessages.java | 106 ---------------- .../shuffle/OneForOneBlockFetcher.java | 17 ++- .../protocol/BlockTransferMessage.java | 76 ++++++++++++ .../{ => protocol}/ExecutorShuffleInfo.java | 36 +++++- .../network/shuffle/protocol/OpenBlocks.java | 87 ++++++++++++++ .../shuffle/protocol/RegisterExecutor.java | 91 ++++++++++++++ .../StreamHandle.java} | 34 ++++-- .../network/shuffle/protocol/UploadBlock.java | 113 ++++++++++++++++++ ...e.java => BlockTransferMessagesSuite.java} | 33 ++--- .../ExternalShuffleBlockHandlerSuite.java | 29 ++--- .../ExternalShuffleIntegrationSuite.java | 1 + .../shuffle/ExternalShuffleSecuritySuite.java | 1 + .../shuffle/OneForOneBlockFetcherSuite.java | 18 +-- .../shuffle/TestShuffleDataContext.java | 2 + 30 files changed, 640 insertions(+), 284 deletions(-) create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java delete mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java rename network/shuffle/src/main/java/org/apache/spark/network/shuffle/{ => protocol}/ExecutorShuffleInfo.java (68%) create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java rename network/shuffle/src/main/java/org/apache/spark/network/shuffle/{ShuffleStreamHandle.java => protocol/StreamHandle.java} (65%) create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java rename network/shuffle/src/test/java/org/apache/spark/network/shuffle/{ShuffleMessagesSuite.java => BlockTransferMessagesSuite.java} (55%) diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 210a581db466e..dcbda5a8515dd 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -73,6 +73,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo def uploadBlock( hostname: String, port: Int, + execId: String, blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Future[Unit] @@ -110,9 +111,10 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo def uploadBlockSync( hostname: String, port: Int, + execId: String, blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Unit = { - Await.result(uploadBlock(hostname, port, blockId, blockData, level), Duration.Inf) + Await.result(uploadBlock(hostname, port, execId, blockId, blockData, level), Duration.Inf) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 1950e7bd634ee..b089da8596e2b 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -26,18 +26,10 @@ import org.apache.spark.network.BlockDataManager import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager} -import org.apache.spark.network.shuffle.ShuffleStreamHandle +import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, OpenBlocks, StreamHandle, UploadBlock} import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, StorageLevel} -object NettyMessages { - /** Request to read a set of blocks. Returns [[ShuffleStreamHandle]] to identify the stream. */ - case class OpenBlocks(blockIds: Seq[BlockId]) - - /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */ - case class UploadBlock(blockId: BlockId, blockData: Array[Byte], level: StorageLevel) -} - /** * Serves requests to open blocks by simply registering one chunk per block requested. * Handles opening and uploading arbitrary BlockManager blocks. @@ -50,28 +42,29 @@ class NettyBlockRpcServer( blockManager: BlockDataManager) extends RpcHandler with Logging { - import NettyMessages._ - private val streamManager = new OneForOneStreamManager() override def receive( client: TransportClient, messageBytes: Array[Byte], responseContext: RpcResponseCallback): Unit = { - val ser = serializer.newInstance() - val message = ser.deserialize[AnyRef](ByteBuffer.wrap(messageBytes)) + val message = BlockTransferMessage.Decoder.fromByteArray(messageBytes) logTrace(s"Received request: $message") message match { - case OpenBlocks(blockIds) => - val blocks: Seq[ManagedBuffer] = blockIds.map(blockManager.getBlockData) + case openBlocks: OpenBlocks => + val blocks: Seq[ManagedBuffer] = + openBlocks.blockIds.map(BlockId.apply).map(blockManager.getBlockData) val streamId = streamManager.registerStream(blocks.iterator) logTrace(s"Registered streamId $streamId with ${blocks.size} buffers") - responseContext.onSuccess( - ser.serialize(new ShuffleStreamHandle(streamId, blocks.size)).array()) + responseContext.onSuccess(new StreamHandle(streamId, blocks.size).toByteArray) - case UploadBlock(blockId, blockData, level) => - blockManager.putBlockData(blockId, new NioManagedBuffer(ByteBuffer.wrap(blockData)), level) + case uploadBlock: UploadBlock => + // StorageLevel is serialized as bytes using our JavaSerializer. + val level: StorageLevel = + serializer.newInstance().deserialize(ByteBuffer.wrap(uploadBlock.metadata)) + val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData)) + blockManager.putBlockData(BlockId(uploadBlock.blockId), data, level) responseContext.onSuccess(new Array[Byte](0)) } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index b937ea825f49e..f8a7f640689a2 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -24,10 +24,10 @@ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.{TransportClientBootstrap, RpcResponseCallback, TransportClientFactory} -import org.apache.spark.network.netty.NettyMessages.{OpenBlocks, UploadBlock} import org.apache.spark.network.sasl.{SaslRpcHandler, SaslClientBootstrap} import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{RetryingBlockFetcher, BlockFetchingListener, OneForOneBlockFetcher} +import org.apache.spark.network.shuffle.protocol.UploadBlock import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils @@ -46,6 +46,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage private[this] var transportContext: TransportContext = _ private[this] var server: TransportServer = _ private[this] var clientFactory: TransportClientFactory = _ + private[this] var appId: String = _ override def init(blockDataManager: BlockDataManager): Unit = { val (rpcHandler: RpcHandler, bootstrap: Option[TransportClientBootstrap]) = { @@ -60,6 +61,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage transportContext = new TransportContext(transportConf, rpcHandler) clientFactory = transportContext.createClientFactory(bootstrap.toList) server = transportContext.createServer() + appId = conf.getAppId logInfo("Server created on " + server.getPort) } @@ -74,8 +76,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter { override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) { val client = clientFactory.createClient(host, port) - new OneForOneBlockFetcher(client, blockIds.toArray, listener) - .start(OpenBlocks(blockIds.map(BlockId.apply))) + new OneForOneBlockFetcher(client, appId, execId, blockIds.toArray, listener).start() } } @@ -101,12 +102,17 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage override def uploadBlock( hostname: String, port: Int, + execId: String, blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Future[Unit] = { val result = Promise[Unit]() val client = clientFactory.createClient(hostname, port) + // StorageLevel is serialized as bytes using our JavaSerializer. Everything else is encoded + // using our binary protocol. + val levelBytes = serializer.newInstance().serialize(level).array() + // Convert or copy nio buffer into array in order to serialize it. val nioBuffer = blockData.nioByteBuffer() val array = if (nioBuffer.hasArray) { @@ -117,8 +123,7 @@ class NettyBlockTransferService(conf: SparkConf, securityManager: SecurityManage data } - val ser = serializer.newInstance() - client.sendRpc(ser.serialize(new UploadBlock(blockId, array, level)).array(), + client.sendRpc(new UploadBlock(appId, execId, blockId.toString, levelBytes, array).toByteArray, new RpcResponseCallback { override def onSuccess(response: Array[Byte]): Unit = { logTrace(s"Successfully uploaded block $blockId") diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index f56d165daba55..b2aec160635c7 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -137,6 +137,7 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa override def uploadBlock( hostname: String, port: Int, + execId: String, blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index e48d7772d6ee9..39434f473a9d8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -35,7 +35,8 @@ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.netty.{SparkTransportConf, NettyBlockTransferService} -import org.apache.spark.network.shuffle.{ExecutorShuffleInfo, ExternalShuffleClient} +import org.apache.spark.network.shuffle.ExternalShuffleClient +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo import org.apache.spark.network.util.{ConfigProvider, TransportConf} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager @@ -939,7 +940,7 @@ private[spark] class BlockManager( data.rewind() logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") blockTransferService.uploadBlockSync( - peer.host, peer.port, blockId, new NioManagedBuffer(data), tLevel) + peer.host, peer.port, peer.executorId, blockId, new NioManagedBuffer(data), tLevel) logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %s ms" .format(System.currentTimeMillis - onePeerStartTime)) peersReplicatedTo += peer diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 9162ec9801663..530f5d6db5a29 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -36,7 +36,9 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, ShouldMat class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with ShouldMatchers { test("security default off") { - testConnection(new SparkConf, new SparkConf) match { + val conf = new SparkConf() + .set("spark.app.id", "app-id") + testConnection(conf, conf) match { case Success(_) => // expected case Failure(t) => fail(t) } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java index 152af98ced7ce..986957c1509fd 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -38,23 +38,19 @@ public ChunkFetchFailure(StreamChunkId streamChunkId, String errorString) { @Override public int encodedLength() { - return streamChunkId.encodedLength() + 4 + errorString.getBytes(Charsets.UTF_8).length; + return streamChunkId.encodedLength() + Encoders.Strings.encodedLength(errorString); } @Override public void encode(ByteBuf buf) { streamChunkId.encode(buf); - byte[] errorBytes = errorString.getBytes(Charsets.UTF_8); - buf.writeInt(errorBytes.length); - buf.writeBytes(errorBytes); + Encoders.Strings.encode(buf, errorString); } public static ChunkFetchFailure decode(ByteBuf buf) { StreamChunkId streamChunkId = StreamChunkId.decode(buf); - int numErrorStringBytes = buf.readInt(); - byte[] errorBytes = new byte[numErrorStringBytes]; - buf.readBytes(errorBytes); - return new ChunkFetchFailure(streamChunkId, new String(errorBytes, Charsets.UTF_8)); + String errorString = Encoders.Strings.decode(buf); + return new ChunkFetchFailure(streamChunkId, errorString); } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java new file mode 100644 index 0000000000000..873c694250942 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + + +import com.google.common.base.Charsets; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +/** Provides a canonical set of Encoders for simple types. */ +public class Encoders { + + /** Strings are encoded with their length followed by UTF-8 bytes. */ + public static class Strings { + public static int encodedLength(String s) { + return 4 + s.getBytes(Charsets.UTF_8).length; + } + + public static void encode(ByteBuf buf, String s) { + byte[] bytes = s.getBytes(Charsets.UTF_8); + buf.writeInt(bytes.length); + buf.writeBytes(bytes); + } + + public static String decode(ByteBuf buf) { + int length = buf.readInt(); + byte[] bytes = new byte[length]; + buf.readBytes(bytes); + return new String(bytes, Charsets.UTF_8); + } + } + + /** Byte arrays are encoded with their length followed by bytes. */ + public static class ByteArrays { + public static int encodedLength(byte[] arr) { + return 4 + arr.length; + } + + public static void encode(ByteBuf buf, byte[] arr) { + buf.writeInt(arr.length); + buf.writeBytes(arr); + } + + public static byte[] decode(ByteBuf buf) { + int length = buf.readInt(); + byte[] bytes = new byte[length]; + buf.readBytes(bytes); + return bytes; + } + } + + /** String arrays are encoded with the number of strings followed by per-String encoding. */ + public static class StringArrays { + public static int encodedLength(String[] strings) { + int totalLength = 4; + for (String s : strings) { + totalLength += Strings.encodedLength(s); + } + return totalLength; + } + + public static void encode(ByteBuf buf, String[] strings) { + buf.writeInt(strings.length); + for (String s : strings) { + Strings.encode(buf, s); + } + } + + public static String[] decode(ByteBuf buf) { + int numStrings = buf.readInt(); + String[] strings = new String[numStrings]; + for (int i = 0; i < strings.length; i ++) { + strings[i] = Strings.decode(buf); + } + return strings; + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java index e239d4ffbd29c..ebd764eb5eb5f 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -36,23 +36,19 @@ public RpcFailure(long requestId, String errorString) { @Override public int encodedLength() { - return 8 + 4 + errorString.getBytes(Charsets.UTF_8).length; + return 8 + Encoders.Strings.encodedLength(errorString); } @Override public void encode(ByteBuf buf) { buf.writeLong(requestId); - byte[] errorBytes = errorString.getBytes(Charsets.UTF_8); - buf.writeInt(errorBytes.length); - buf.writeBytes(errorBytes); + Encoders.Strings.encode(buf, errorString); } public static RpcFailure decode(ByteBuf buf) { long requestId = buf.readLong(); - int numErrorStringBytes = buf.readInt(); - byte[] errorBytes = new byte[numErrorStringBytes]; - buf.readBytes(errorBytes); - return new RpcFailure(requestId, new String(errorBytes, Charsets.UTF_8)); + String errorString = Encoders.Strings.decode(buf); + return new RpcFailure(requestId, errorString); } @Override diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java index 099e934ae018c..cdee0b0e0316b 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -44,21 +44,18 @@ public RpcRequest(long requestId, byte[] message) { @Override public int encodedLength() { - return 8 + 4 + message.length; + return 8 + Encoders.ByteArrays.encodedLength(message); } @Override public void encode(ByteBuf buf) { buf.writeLong(requestId); - buf.writeInt(message.length); - buf.writeBytes(message); + Encoders.ByteArrays.encode(buf, message); } public static RpcRequest decode(ByteBuf buf) { long requestId = buf.readLong(); - int messageLen = buf.readInt(); - byte[] message = new byte[messageLen]; - buf.readBytes(message); + byte[] message = Encoders.ByteArrays.decode(buf); return new RpcRequest(requestId, message); } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java index ed479478325b6..0a62e09a8115c 100644 --- a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -36,20 +36,17 @@ public RpcResponse(long requestId, byte[] response) { public Type type() { return Type.RpcResponse; } @Override - public int encodedLength() { return 8 + 4 + response.length; } + public int encodedLength() { return 8 + Encoders.ByteArrays.encodedLength(response); } @Override public void encode(ByteBuf buf) { buf.writeLong(requestId); - buf.writeInt(response.length); - buf.writeBytes(response); + Encoders.ByteArrays.encode(buf, response); } public static RpcResponse decode(ByteBuf buf) { long requestId = buf.readLong(); - int responseLen = buf.readInt(); - byte[] response = new byte[responseLen]; - buf.readBytes(response); + byte[] response = Encoders.ByteArrays.decode(buf); return new RpcResponse(requestId, response); } diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 75c4a3981a240..009dbcf01323f 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -50,33 +50,6 @@ public static void closeQuietly(Closeable closeable) { } } - // TODO: Make this configurable, do not use Java serialization! - public static T deserialize(byte[] bytes) { - try { - ObjectInputStream is = new ObjectInputStream(new ByteArrayInputStream(bytes)); - Object out = is.readObject(); - is.close(); - return (T) out; - } catch (ClassNotFoundException e) { - throw new RuntimeException("Could not deserialize object", e); - } catch (IOException e) { - throw new RuntimeException("Could not deserialize object", e); - } - } - - // TODO: Make this configurable, do not use Java serialization! - public static byte[] serialize(Object object) { - try { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - ObjectOutputStream os = new ObjectOutputStream(baos); - os.writeObject(object); - os.close(); - return baos.toByteArray(); - } catch (IOException e) { - throw new RuntimeException("Could not serialize object", e); - } - } - /** Returns a hash consistent with Spark's Utils.nonNegativeHash(). */ public static int nonNegativeHash(Object obj) { if (obj == null) { return 0; } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java index 599cc6428c90e..cad76ab7aa54e 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslMessage.java @@ -17,10 +17,10 @@ package org.apache.spark.network.sasl; -import com.google.common.base.Charsets; import io.netty.buffer.ByteBuf; import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.Encoders; /** * Encodes a Sasl-related message which is attempting to authenticate using some credentials tagged @@ -42,18 +42,14 @@ public SaslMessage(String appId, byte[] payload) { @Override public int encodedLength() { - // tag + appIdLength + appId + payloadLength + payload - return 1 + 4 + appId.getBytes(Charsets.UTF_8).length + 4 + payload.length; + return 1 + Encoders.Strings.encodedLength(appId) + Encoders.ByteArrays.encodedLength(payload); } @Override public void encode(ByteBuf buf) { buf.writeByte(TAG_BYTE); - byte[] idBytes = appId.getBytes(Charsets.UTF_8); - buf.writeInt(idBytes.length); - buf.writeBytes(idBytes); - buf.writeInt(payload.length); - buf.writeBytes(payload); + Encoders.Strings.encode(buf, appId); + Encoders.ByteArrays.encode(buf, payload); } public static SaslMessage decode(ByteBuf buf) { @@ -62,14 +58,8 @@ public static SaslMessage decode(ByteBuf buf) { + " (maybe your client does not have SASL enabled?)"); } - int idLength = buf.readInt(); - byte[] idBytes = new byte[idLength]; - buf.readBytes(idBytes); - - int payloadLength = buf.readInt(); - byte[] payload = new byte[payloadLength]; - buf.readBytes(payload); - - return new SaslMessage(new String(idBytes, Charsets.UTF_8), payload); + String appId = Encoders.Strings.decode(buf); + byte[] payload = Encoders.ByteArrays.decode(buf); + return new SaslMessage(appId, payload); } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 75ebf8c7b0604..a6db4b2abd6c9 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -24,15 +24,16 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.spark.network.shuffle.ExternalShuffleMessages.*; - import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.StreamManager; -import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.RegisterExecutor; +import org.apache.spark.network.shuffle.protocol.StreamHandle; /** * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. @@ -62,12 +63,10 @@ public ExternalShuffleBlockHandler() { @Override public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - Object msgObj = JavaUtils.deserialize(message); - - logger.trace("Received message: " + msgObj); + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteArray(message); - if (msgObj instanceof OpenShuffleBlocks) { - OpenShuffleBlocks msg = (OpenShuffleBlocks) msgObj; + if (msgObj instanceof OpenBlocks) { + OpenBlocks msg = (OpenBlocks) msgObj; List blocks = Lists.newArrayList(); for (String blockId : msg.blockIds) { @@ -75,8 +74,7 @@ public void receive(TransportClient client, byte[] message, RpcResponseCallback } long streamId = streamManager.registerStream(blocks.iterator()); logger.trace("Registered streamId {} with {} buffers", streamId, msg.blockIds.length); - callback.onSuccess(JavaUtils.serialize( - new ShuffleStreamHandle(streamId, msg.blockIds.length))); + callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteArray()); } else if (msgObj instanceof RegisterExecutor) { RegisterExecutor msg = (RegisterExecutor) msgObj; @@ -84,8 +82,7 @@ public void receive(TransportClient client, byte[] message, RpcResponseCallback callback.onSuccess(new byte[0]); } else { - throw new UnsupportedOperationException(String.format( - "Unexpected message: %s (class = %s)", msgObj, msgObj.getClass())); + throw new UnsupportedOperationException("Unexpected message: " + msgObj); } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java index 98fcfb82aa5d1..ffb7faa3dbdca 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java @@ -35,6 +35,7 @@ import org.apache.spark.network.buffer.FileSegmentManagedBuffer; import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.JavaUtils; /** diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index 27884b82c8cb9..6e8018b723dc6 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -31,8 +31,8 @@ import org.apache.spark.network.sasl.SaslClientBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; -import org.apache.spark.network.shuffle.ExternalShuffleMessages.RegisterExecutor; -import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.shuffle.protocol.RegisterExecutor; import org.apache.spark.network.util.TransportConf; /** @@ -91,8 +91,7 @@ public void fetchBlocks( public void createAndStart(String[] blockIds, BlockFetchingListener listener) throws IOException { TransportClient client = clientFactory.createClient(host, port); - new OneForOneBlockFetcher(client, blockIds, listener) - .start(new ExternalShuffleMessages.OpenShuffleBlocks(appId, execId, blockIds)); + new OneForOneBlockFetcher(client, appId, execId, blockIds, listener).start(); } }; @@ -128,9 +127,8 @@ public void registerWithShuffleServer( ExecutorShuffleInfo executorInfo) throws IOException { assert appId != null : "Called before init()"; TransportClient client = clientFactory.createClient(host, port); - byte[] registerExecutorMessage = - JavaUtils.serialize(new RegisterExecutor(appId, execId, executorInfo)); - client.sendRpcSync(registerExecutorMessage, 5000 /* timeoutMs */); + byte[] registerMessage = new RegisterExecutor(appId, execId, executorInfo).toByteArray(); + client.sendRpcSync(registerMessage, 5000 /* timeoutMs */); } @Override diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java deleted file mode 100644 index e79420ed8254f..0000000000000 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java +++ /dev/null @@ -1,106 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.shuffle; - -import java.io.Serializable; -import java.util.Arrays; - -import com.google.common.base.Objects; - -/** Messages handled by the {@link ExternalShuffleBlockHandler}. */ -public class ExternalShuffleMessages { - - /** Request to read a set of shuffle blocks. Returns [[ShuffleStreamHandle]]. */ - public static class OpenShuffleBlocks implements Serializable { - public final String appId; - public final String execId; - public final String[] blockIds; - - public OpenShuffleBlocks(String appId, String execId, String[] blockIds) { - this.appId = appId; - this.execId = execId; - this.blockIds = blockIds; - } - - @Override - public int hashCode() { - return Objects.hashCode(appId, execId) * 41 + Arrays.hashCode(blockIds); - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .add("blockIds", Arrays.toString(blockIds)) - .toString(); - } - - @Override - public boolean equals(Object other) { - if (other != null && other instanceof OpenShuffleBlocks) { - OpenShuffleBlocks o = (OpenShuffleBlocks) other; - return Objects.equal(appId, o.appId) - && Objects.equal(execId, o.execId) - && Arrays.equals(blockIds, o.blockIds); - } - return false; - } - } - - /** Initial registration message between an executor and its local shuffle server. */ - public static class RegisterExecutor implements Serializable { - public final String appId; - public final String execId; - public final ExecutorShuffleInfo executorInfo; - - public RegisterExecutor( - String appId, - String execId, - ExecutorShuffleInfo executorInfo) { - this.appId = appId; - this.execId = execId; - this.executorInfo = executorInfo; - } - - @Override - public int hashCode() { - return Objects.hashCode(appId, execId, executorInfo); - } - - @Override - public String toString() { - return Objects.toStringHelper(this) - .add("appId", appId) - .add("execId", execId) - .add("executorInfo", executorInfo) - .toString(); - } - - @Override - public boolean equals(Object other) { - if (other != null && other instanceof RegisterExecutor) { - RegisterExecutor o = (RegisterExecutor) other; - return Objects.equal(appId, o.appId) - && Objects.equal(execId, o.execId) - && Objects.equal(executorInfo, o.executorInfo); - } - return false; - } - } -} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 9e77a1f68c4b0..8ed2e0b39ad23 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -26,6 +26,9 @@ import org.apache.spark.network.client.ChunkReceivedCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.StreamHandle; import org.apache.spark.network.util.JavaUtils; /** @@ -41,17 +44,21 @@ public class OneForOneBlockFetcher { private final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class); private final TransportClient client; + private final OpenBlocks openMessage; private final String[] blockIds; private final BlockFetchingListener listener; private final ChunkReceivedCallback chunkCallback; - private ShuffleStreamHandle streamHandle = null; + private StreamHandle streamHandle = null; public OneForOneBlockFetcher( TransportClient client, + String appId, + String execId, String[] blockIds, BlockFetchingListener listener) { this.client = client; + this.openMessage = new OpenBlocks(appId, execId, blockIds); this.blockIds = blockIds; this.listener = listener; this.chunkCallback = new ChunkCallback(); @@ -76,18 +83,18 @@ public void onFailure(int chunkIndex, Throwable e) { /** * Begins the fetching process, calling the listener with every block fetched. * The given message will be serialized with the Java serializer, and the RPC must return a - * {@link ShuffleStreamHandle}. We will send all fetch requests immediately, without throttling. + * {@link StreamHandle}. We will send all fetch requests immediately, without throttling. */ - public void start(Object openBlocksMessage) { + public void start() { if (blockIds.length == 0) { throw new IllegalArgumentException("Zero-sized blockIds array"); } - client.sendRpc(JavaUtils.serialize(openBlocksMessage), new RpcResponseCallback() { + client.sendRpc(openMessage.toByteArray(), new RpcResponseCallback() { @Override public void onSuccess(byte[] response) { try { - streamHandle = JavaUtils.deserialize(response); + streamHandle = (StreamHandle) BlockTransferMessage.Decoder.fromByteArray(response); logger.trace("Successfully opened blocks {}, preparing to fetch chunks.", streamHandle); // Immediately request all chunks -- we expect that the total size of the request is diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java new file mode 100644 index 0000000000000..b4b13b8a6ef5d --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.protocol.Encodable; + +/** + * Messages handled by the {@link org.apache.spark.network.shuffle.ExternalShuffleBlockHandler}, or + * by Spark's NettyBlockTransferService. + * + * At a high level: + * - OpenBlock is handled by both services, but only services shuffle files for the external + * shuffle service. It returns a StreamHandle. + * - UploadBlock is only handled by the NettyBlockTransferService. + * - RegisterExecutor is only handled by the external shuffle service. + */ +public abstract class BlockTransferMessage implements Encodable { + protected abstract Type type(); + + /** Preceding every serialized message is its type, which allows us to deserialize it. */ + public static enum Type { + OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3); + + private final byte id; + + private Type(int id) { + assert id < 128 : "Cannot have more than 128 message types"; + this.id = (byte) id; + } + + public byte id() { return id; } + } + + // NB: Java does not support static methods in interfaces, so we must put this in a static class. + public static class Decoder { + /** Deserializes the 'type' byte followed by the message itself. */ + public static BlockTransferMessage fromByteArray(byte[] msg) { + ByteBuf buf = Unpooled.wrappedBuffer(msg); + byte type = buf.readByte(); + switch (type) { + case 0: return OpenBlocks.decode(buf); + case 1: return UploadBlock.decode(buf); + case 2: return RegisterExecutor.decode(buf); + case 3: return StreamHandle.decode(buf); + default: throw new IllegalArgumentException("Unknown message type: " + type); + } + } + } + + /** Serializes the 'type' byte followed by the message itself. */ + public byte[] toByteArray() { + ByteBuf buf = Unpooled.buffer(encodedLength()); + buf.writeByte(type().id); + encode(buf); + assert buf.writableBytes() == 0 : "Writable bytes remain: " + buf.writableBytes(); + return buf.array(); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorShuffleInfo.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java similarity index 68% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorShuffleInfo.java rename to network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java index d45e64656a0e3..cadc8e8369c6a 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorShuffleInfo.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java @@ -15,21 +15,24 @@ * limitations under the License. */ -package org.apache.spark.network.shuffle; +package org.apache.spark.network.shuffle.protocol; -import java.io.Serializable; import java.util.Arrays; import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.Encoders; /** Contains all configuration necessary for locating the shuffle files of an executor. */ -public class ExecutorShuffleInfo implements Serializable { +public class ExecutorShuffleInfo implements Encodable { /** The base set of local directories that the executor stores its shuffle files in. */ - final String[] localDirs; + public final String[] localDirs; /** Number of subdirectories created within each localDir. */ - final int subDirsPerLocalDir; + public final int subDirsPerLocalDir; /** Shuffle manager (SortShuffleManager or HashShuffleManager) that the executor is using. */ - final String shuffleManager; + public final String shuffleManager; public ExecutorShuffleInfo(String[] localDirs, int subDirsPerLocalDir, String shuffleManager) { this.localDirs = localDirs; @@ -61,4 +64,25 @@ public boolean equals(Object other) { } return false; } + + @Override + public int encodedLength() { + return Encoders.StringArrays.encodedLength(localDirs) + + 4 // int + + Encoders.Strings.encodedLength(shuffleManager); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.StringArrays.encode(buf, localDirs); + buf.writeInt(subDirsPerLocalDir); + Encoders.Strings.encode(buf, shuffleManager); + } + + public static ExecutorShuffleInfo decode(ByteBuf buf) { + String[] localDirs = Encoders.StringArrays.decode(buf); + int subDirsPerLocalDir = buf.readInt(); + String shuffleManager = Encoders.Strings.decode(buf); + return new ExecutorShuffleInfo(localDirs, subDirsPerLocalDir, shuffleManager); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java new file mode 100644 index 0000000000000..60485bace643c --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/OpenBlocks.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +/** Request to read a set of blocks. Returns {@link StreamHandle}. */ +public class OpenBlocks extends BlockTransferMessage { + public final String appId; + public final String execId; + public final String[] blockIds; + + public OpenBlocks(String appId, String execId, String[] blockIds) { + this.appId = appId; + this.execId = execId; + this.blockIds = blockIds; + } + + @Override + protected Type type() { return Type.OPEN_BLOCKS; } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId) * 41 + Arrays.hashCode(blockIds); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("blockIds", Arrays.toString(blockIds)) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof OpenBlocks) { + OpenBlocks o = (OpenBlocks) other; + return Objects.equal(appId, o.appId) + && Objects.equal(execId, o.execId) + && Arrays.equals(blockIds, o.blockIds); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + Encoders.StringArrays.encodedLength(blockIds); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + Encoders.StringArrays.encode(buf, blockIds); + } + + public static OpenBlocks decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String execId = Encoders.Strings.decode(buf); + String[] blockIds = Encoders.StringArrays.decode(buf); + return new OpenBlocks(appId, execId, blockIds); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java new file mode 100644 index 0000000000000..38acae3b31d64 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RegisterExecutor.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +/** + * Initial registration message between an executor and its local shuffle server. + * Returns nothing (empty bye array). + */ +public class RegisterExecutor extends BlockTransferMessage { + public final String appId; + public final String execId; + public final ExecutorShuffleInfo executorInfo; + + public RegisterExecutor( + String appId, + String execId, + ExecutorShuffleInfo executorInfo) { + this.appId = appId; + this.execId = execId; + this.executorInfo = executorInfo; + } + + @Override + protected Type type() { return Type.REGISTER_EXECUTOR; } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId, executorInfo); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("executorInfo", executorInfo) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof RegisterExecutor) { + RegisterExecutor o = (RegisterExecutor) other; + return Objects.equal(appId, o.appId) + && Objects.equal(execId, o.execId) + && Objects.equal(executorInfo, o.executorInfo); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + executorInfo.encodedLength(); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + executorInfo.encode(buf); + } + + public static RegisterExecutor decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String execId = Encoders.Strings.decode(buf); + ExecutorShuffleInfo executorShuffleInfo = ExecutorShuffleInfo.decode(buf); + return new RegisterExecutor(appId, execId, executorShuffleInfo); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleStreamHandle.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java similarity index 65% rename from network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleStreamHandle.java rename to network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java index 9c94691224328..21369c8cfb0d6 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleStreamHandle.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/StreamHandle.java @@ -15,26 +15,29 @@ * limitations under the License. */ -package org.apache.spark.network.shuffle; +package org.apache.spark.network.shuffle.protocol; import java.io.Serializable; -import java.util.Arrays; import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; /** * Identifier for a fixed number of chunks to read from a stream created by an "open blocks" - * message. This is used by {@link OneForOneBlockFetcher}. + * message. This is used by {@link org.apache.spark.network.shuffle.OneForOneBlockFetcher}. */ -public class ShuffleStreamHandle implements Serializable { +public class StreamHandle extends BlockTransferMessage { public final long streamId; public final int numChunks; - public ShuffleStreamHandle(long streamId, int numChunks) { + public StreamHandle(long streamId, int numChunks) { this.streamId = streamId; this.numChunks = numChunks; } + @Override + protected Type type() { return Type.STREAM_HANDLE; } + @Override public int hashCode() { return Objects.hashCode(streamId, numChunks); @@ -50,11 +53,28 @@ public String toString() { @Override public boolean equals(Object other) { - if (other != null && other instanceof ShuffleStreamHandle) { - ShuffleStreamHandle o = (ShuffleStreamHandle) other; + if (other != null && other instanceof StreamHandle) { + StreamHandle o = (StreamHandle) other; return Objects.equal(streamId, o.streamId) && Objects.equal(numChunks, o.numChunks); } return false; } + + @Override + public int encodedLength() { + return 8 + 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(streamId); + buf.writeInt(numChunks); + } + + public static StreamHandle decode(ByteBuf buf) { + long streamId = buf.readLong(); + int numChunks = buf.readInt(); + return new StreamHandle(streamId, numChunks); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java new file mode 100644 index 0000000000000..38abe29cc585f --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/UploadBlock.java @@ -0,0 +1,113 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +/** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */ +public class UploadBlock extends BlockTransferMessage { + public final String appId; + public final String execId; + public final String blockId; + // TODO: StorageLevel is serialized separately in here because StorageLevel is not available in + // this package. We should avoid this hack. + public final byte[] metadata; + public final byte[] blockData; + + /** + * @param metadata Meta-information about block, typically StorageLevel. + * @param blockData The actual block's bytes. + */ + public UploadBlock( + String appId, + String execId, + String blockId, + byte[] metadata, + byte[] blockData) { + this.appId = appId; + this.execId = execId; + this.blockId = blockId; + this.metadata = metadata; + this.blockData = blockData; + } + + @Override + protected Type type() { return Type.UPLOAD_BLOCK; } + + @Override + public int hashCode() { + int objectsHashCode = Objects.hashCode(appId, execId, blockId); + return (objectsHashCode * 41 + Arrays.hashCode(metadata)) * 41 + Arrays.hashCode(blockData); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("blockId", blockId) + .add("metadata size", metadata.length) + .add("block size", blockData.length) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof UploadBlock) { + UploadBlock o = (UploadBlock) other; + return Objects.equal(appId, o.appId) + && Objects.equal(execId, o.execId) + && Objects.equal(blockId, o.blockId) + && Arrays.equals(metadata, o.metadata) + && Arrays.equals(blockData, o.blockData); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + Encoders.Strings.encodedLength(blockId) + + Encoders.ByteArrays.encodedLength(metadata) + + Encoders.ByteArrays.encodedLength(blockData); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + Encoders.Strings.encode(buf, blockId); + Encoders.ByteArrays.encode(buf, metadata); + Encoders.ByteArrays.encode(buf, blockData); + } + + public static UploadBlock decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String execId = Encoders.Strings.decode(buf); + String blockId = Encoders.Strings.decode(buf); + byte[] metadata = Encoders.ByteArrays.decode(buf); + byte[] blockData = Encoders.ByteArrays.decode(buf); + return new UploadBlock(appId, execId, blockId, metadata, blockData); + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleMessagesSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java similarity index 55% rename from network/shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleMessagesSuite.java rename to network/shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java index ee9482b49cfc3..d65de9ca550a3 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleMessagesSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java @@ -21,31 +21,24 @@ import static org.junit.Assert.*; -import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.shuffle.protocol.*; -import static org.apache.spark.network.shuffle.ExternalShuffleMessages.*; - -public class ShuffleMessagesSuite { +/** Verifies that all BlockTransferMessages can be serialized correctly. */ +public class BlockTransferMessagesSuite { @Test public void serializeOpenShuffleBlocks() { - OpenShuffleBlocks msg = new OpenShuffleBlocks("app-1", "exec-2", - new String[] { "block0", "block1" }); - OpenShuffleBlocks msg2 = JavaUtils.deserialize(JavaUtils.serialize(msg)); - assertEquals(msg, msg2); + checkSerializeDeserialize(new OpenBlocks("app-1", "exec-2", new String[] { "b1", "b2" })); + checkSerializeDeserialize(new RegisterExecutor("app-1", "exec-2", new ExecutorShuffleInfo( + new String[] { "/local1", "/local2" }, 32, "MyShuffleManager"))); + checkSerializeDeserialize(new UploadBlock("app-1", "exec-2", "block-3", new byte[] { 1, 2 }, + new byte[] { 4, 5, 6, 7} )); + checkSerializeDeserialize(new StreamHandle(12345, 16)); } - @Test - public void serializeRegisterExecutor() { - RegisterExecutor msg = new RegisterExecutor("app-1", "exec-2", new ExecutorShuffleInfo( - new String[] { "/local1", "/local2" }, 32, "MyShuffleManager")); - RegisterExecutor msg2 = JavaUtils.deserialize(JavaUtils.serialize(msg)); - assertEquals(msg, msg2); - } - - @Test - public void serializeShuffleStreamHandle() { - ShuffleStreamHandle msg = new ShuffleStreamHandle(12345, 16); - ShuffleStreamHandle msg2 = JavaUtils.deserialize(JavaUtils.serialize(msg)); + private void checkSerializeDeserialize(BlockTransferMessage msg) { + BlockTransferMessage msg2 = BlockTransferMessage.Decoder.fromByteArray(msg.toByteArray()); assertEquals(msg, msg2); + assertEquals(msg.hashCode(), msg2.hashCode()); + assertEquals(msg.toString(), msg2.toString()); } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 7939cb4d32690..3f9fe1681cf27 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -24,8 +24,6 @@ import org.junit.Test; import org.mockito.ArgumentCaptor; -import static org.apache.spark.network.shuffle.ExternalShuffleMessages.OpenShuffleBlocks; -import static org.apache.spark.network.shuffle.ExternalShuffleMessages.RegisterExecutor; import static org.junit.Assert.*; import static org.mockito.Matchers.any; import static org.mockito.Mockito.*; @@ -36,7 +34,12 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; -import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.RegisterExecutor; +import org.apache.spark.network.shuffle.protocol.StreamHandle; +import org.apache.spark.network.shuffle.protocol.UploadBlock; public class ExternalShuffleBlockHandlerSuite { TransportClient client = mock(TransportClient.class); @@ -57,8 +60,7 @@ public void testRegisterExecutor() { RpcResponseCallback callback = mock(RpcResponseCallback.class); ExecutorShuffleInfo config = new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort"); - byte[] registerMessage = JavaUtils.serialize( - new RegisterExecutor("app0", "exec1", config)); + byte[] registerMessage = new RegisterExecutor("app0", "exec1", config).toByteArray(); handler.receive(client, registerMessage, callback); verify(blockManager, times(1)).registerExecutor("app0", "exec1", config); @@ -75,9 +77,8 @@ public void testOpenShuffleBlocks() { ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); when(blockManager.getBlockData("app0", "exec1", "b0")).thenReturn(block0Marker); when(blockManager.getBlockData("app0", "exec1", "b1")).thenReturn(block1Marker); - byte[] openBlocksMessage = JavaUtils.serialize( - new OpenShuffleBlocks("app0", "exec1", new String[] { "b0", "b1" })); - handler.receive(client, openBlocksMessage, callback); + byte[] openBlocks = new OpenBlocks("app0", "exec1", new String[] { "b0", "b1" }).toByteArray(); + handler.receive(client, openBlocks, callback); verify(blockManager, times(1)).getBlockData("app0", "exec1", "b0"); verify(blockManager, times(1)).getBlockData("app0", "exec1", "b1"); @@ -85,7 +86,8 @@ public void testOpenShuffleBlocks() { verify(callback, times(1)).onSuccess(response.capture()); verify(callback, never()).onFailure((Throwable) any()); - ShuffleStreamHandle handle = JavaUtils.deserialize(response.getValue()); + StreamHandle handle = + (StreamHandle) BlockTransferMessage.Decoder.fromByteArray(response.getValue()); assertEquals(2, handle.numChunks); ArgumentCaptor stream = ArgumentCaptor.forClass(Iterator.class); @@ -100,18 +102,17 @@ public void testOpenShuffleBlocks() { public void testBadMessages() { RpcResponseCallback callback = mock(RpcResponseCallback.class); - byte[] unserializableMessage = new byte[] { 0x12, 0x34, 0x56 }; + byte[] unserializableMsg = new byte[] { 0x12, 0x34, 0x56 }; try { - handler.receive(client, unserializableMessage, callback); + handler.receive(client, unserializableMsg, callback); fail("Should have thrown"); } catch (Exception e) { // pass } - byte[] unexpectedMessage = JavaUtils.serialize( - new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort")); + byte[] unexpectedMsg = new UploadBlock("a", "e", "b", new byte[1], new byte[2]).toByteArray(); try { - handler.receive(client, unexpectedMessage, callback); + handler.receive(client, unexpectedMsg, callback); fail("Should have thrown"); } catch (UnsupportedOperationException e) { // pass diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 3bea5b0f253c6..687bde59fdae4 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -42,6 +42,7 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NioManagedBuffer; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 848c88f743d50..8afceab1d585a 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -31,6 +31,7 @@ import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index c18346f6966d6..842741e3d354f 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -40,7 +40,9 @@ import org.apache.spark.network.client.ChunkReceivedCallback; import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.OpenBlocks; +import org.apache.spark.network.shuffle.protocol.StreamHandle; public class OneForOneBlockFetcherSuite { @Test @@ -119,17 +121,19 @@ public void testEmptyBlockFetch() { private BlockFetchingListener fetchBlocks(final LinkedHashMap blocks) { TransportClient client = mock(TransportClient.class); BlockFetchingListener listener = mock(BlockFetchingListener.class); - String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - OneForOneBlockFetcher fetcher = new OneForOneBlockFetcher(client, blockIds, listener); + final String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + OneForOneBlockFetcher fetcher = + new OneForOneBlockFetcher(client, "app-id", "exec-id", blockIds, listener); // Respond to the "OpenBlocks" message with an appropirate ShuffleStreamHandle with streamId 123 doAnswer(new Answer() { @Override public Void answer(InvocationOnMock invocationOnMock) throws Throwable { - String message = JavaUtils.deserialize((byte[]) invocationOnMock.getArguments()[0]); + BlockTransferMessage message = BlockTransferMessage.Decoder.fromByteArray( + (byte[]) invocationOnMock.getArguments()[0]); RpcResponseCallback callback = (RpcResponseCallback) invocationOnMock.getArguments()[1]; - callback.onSuccess(JavaUtils.serialize(new ShuffleStreamHandle(123, blocks.size()))); - assertEquals("OpenZeBlocks", message); + callback.onSuccess(new StreamHandle(123, blocks.size()).toByteArray()); + assertEquals(new OpenBlocks("app-id", "exec-id", blockIds), message); return null; } }).when(client).sendRpc((byte[]) any(), (RpcResponseCallback) any()); @@ -161,7 +165,7 @@ public Void answer(InvocationOnMock invocation) throws Throwable { } }).when(client).fetchChunk(anyLong(), anyInt(), (ChunkReceivedCallback) any()); - fetcher.start("OpenZeBlocks"); + fetcher.start(); return listener; } } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 337b5c7bdb5da..76639114df5d9 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -25,6 +25,8 @@ import com.google.common.io.Files; +import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; + /** * Manages some sort- and hash-based shuffle data, including the creation * and cleanup of directories that can be read by the {@link ExternalShuffleBlockManager}. From 636d7bcc96b912f5b5caa91110cd55b55fa38ad8 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 7 Nov 2014 11:43:35 -0800 Subject: [PATCH 1246/1492] [SQL][DOC][Minor] Spark SQL Hive now support dynamic partitioning Author: wangfei Closes #3127 from scwf/patch-9 and squashes the following commits: e39a560 [wangfei] now support dynamic partitioning --- docs/sql-programming-guide.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e399fecbbc78c..ffcce2c588879 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1059,7 +1059,6 @@ in Hive deployments. **Major Hive Features** -* Spark SQL does not currently support inserting to tables using dynamic partitioning. * Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL doesn't support buckets yet. From 86e9eaa3f0ec23cb38bce67585adb2d5f484f4ee Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 7 Nov 2014 11:45:25 -0800 Subject: [PATCH 1247/1492] [SPARK-4225][SQL] Resorts to SparkContext.version to inspect Spark version This PR resorts to `SparkContext.version` rather than META-INF/MANIFEST.MF in the assembly jar to inspect Spark version. Currently, when built with Maven, the MANIFEST.MF file in the assembly jar is incorrectly replaced by Guava 15.0 MANIFEST.MF, probably because of the assembly/shading tricks. Another related PR is #3103, which tries to fix the MANIFEST issue. Author: Cheng Lian Closes #3105 from liancheng/spark-4225 and squashes the following commits: d9585e1 [Cheng Lian] Resorts to SparkContext.version to inspect Spark version --- .../scala/org/apache/spark/util/Utils.scala | 24 ++++++------------- .../thriftserver/SparkSQLCLIService.scala | 12 ++++------ 2 files changed, 12 insertions(+), 24 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 a14d6125484fe..6b85c03da533c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,10 +21,8 @@ import java.io._ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer -import java.util.jar.Attributes.Name -import java.util.{Properties, Locale, Random, UUID} -import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} -import java.util.jar.{Manifest => JarManifest} +import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadFactory, ThreadPoolExecutor} +import java.util.{Locale, Properties, Random, UUID} import scala.collection.JavaConversions._ import scala.collection.Map @@ -38,11 +36,11 @@ import com.google.common.io.{ByteStreams, Files} import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration -import org.apache.log4j.PropertyConfigurator import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.apache.log4j.PropertyConfigurator import org.eclipse.jetty.util.MultiException import org.json4s._ -import tachyon.client.{TachyonFile,TachyonFS} +import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil @@ -352,8 +350,8 @@ private[spark] object Utils extends Logging { * Download a file to target directory. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. * - * If `useCache` is true, first attempts to fetch the file to a local cache that's shared - * across executors running the same application. `useCache` is used mainly for + * If `useCache` is true, first attempts to fetch the file to a local cache that's shared + * across executors running the same application. `useCache` is used mainly for * the executors, and not in local mode. * * Throws SparkException if the target file already exists and has different contents than @@ -400,7 +398,7 @@ private[spark] object Utils extends Logging { } else { doFetchFile(url, targetDir, fileName, conf, securityMgr, hadoopConf) } - + // Decompress the file if it's a .tar or .tar.gz if (fileName.endsWith(".tar.gz") || fileName.endsWith(".tgz")) { logInfo("Untarring " + fileName) @@ -1776,13 +1774,6 @@ private[spark] object Utils extends Logging { s"$libraryPathEnvName=$libraryPath$ampersand" } - lazy val sparkVersion = - SparkContext.jarOfObject(this).map { path => - val manifestUrl = new URL(s"jar:file:$path!/META-INF/MANIFEST.MF") - val manifest = new JarManifest(manifestUrl.openStream()) - manifest.getMainAttributes.getValue(Name.IMPLEMENTATION_VERSION) - }.getOrElse("Unknown") - /** * Return the value of a config either through the SparkConf or the Hadoop configuration * if this is Yarn mode. In the latter case, this defaults to the value set through SparkConf @@ -1796,7 +1787,6 @@ private[spark] object Utils extends Logging { sparkValue } } - } /** diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index ecfb74473e921..499e077d7294a 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -17,18 +17,16 @@ package org.apache.spark.sql.hive.thriftserver -import java.util.jar.Attributes.Name - -import scala.collection.JavaConversions._ - import java.io.IOException import java.util.{List => JList} import javax.security.auth.login.LoginException +import scala.collection.JavaConversions._ + import org.apache.commons.logging.Log -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.Service.STATE import org.apache.hive.service.auth.HiveAuthFactory import org.apache.hive.service.cli._ @@ -50,7 +48,7 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext) addService(sparkSqlSessionManager) var sparkServiceUGI: UserGroupInformation = null - if (ShimLoader.getHadoopShims().isSecurityEnabled()) { + if (ShimLoader.getHadoopShims.isSecurityEnabled) { try { HiveAuthFactory.loginFromKeytab(hiveConf) sparkServiceUGI = ShimLoader.getHadoopShims.getUGIForConf(hiveConf) @@ -68,7 +66,7 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext) getInfoType match { case GetInfoType.CLI_SERVER_NAME => new GetInfoValue("Spark SQL") case GetInfoType.CLI_DBMS_NAME => new GetInfoValue("Spark SQL") - case GetInfoType.CLI_DBMS_VER => new GetInfoValue(Utils.sparkVersion) + case GetInfoType.CLI_DBMS_VER => new GetInfoValue(hiveContext.sparkContext.version) case _ => super.getInfo(sessionHandle, getInfoType) } } From 8154ed7df6c5407e638f465d3bd86b43f36216ef Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 7 Nov 2014 11:51:20 -0800 Subject: [PATCH 1248/1492] [SQL] Support ScalaReflection of schema in different universes Author: Michael Armbrust Closes #3096 from marmbrus/reflectionContext and squashes the following commits: adc221f [Michael Armbrust] Support ScalaReflection of schema in different universes --- .../spark/sql/catalyst/ScalaReflection.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 9cda373623cb5..71034c2c43c77 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -26,14 +26,26 @@ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.decimal.Decimal + /** - * Provides experimental support for generating catalyst schemas for scala objects. + * A default version of ScalaReflection that uses the runtime universe. */ -object ScalaReflection { +object ScalaReflection extends ScalaReflection { + val universe: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe +} + +/** + * Support for generating catalyst schemas for scala objects. + */ +trait ScalaReflection { + /** The universe we work in (runtime or macro) */ + val universe: scala.reflect.api.Universe + + import universe._ + // The Predef.Map is scala.collection.immutable.Map. // Since the map values can be mutable, we explicitly import scala.collection.Map at here. import scala.collection.Map - import scala.reflect.runtime.universe._ case class Schema(dataType: DataType, nullable: Boolean) From 68609c51ad1ab2def302df3c4a1c0bc1ec6e1075 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Fri, 7 Nov 2014 11:52:08 -0800 Subject: [PATCH 1249/1492] [SQL] Modify keyword val location according to ordering 'DOUBLE' should be moved before 'ELSE' according to the ordering convension Author: Jacky Li Closes #3080 from jackylk/patch-5 and squashes the following commits: 3c11df7 [Jacky Li] [SQL] Modify keyword val location according to ordering --- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 5e613e0f18ba6..affef276c2a88 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -55,10 +55,10 @@ class SqlParser extends AbstractSparkSQLParser { protected val DECIMAL = Keyword("DECIMAL") protected val DESC = Keyword("DESC") protected val DISTINCT = Keyword("DISTINCT") + protected val DOUBLE = Keyword("DOUBLE") protected val ELSE = Keyword("ELSE") protected val END = Keyword("END") protected val EXCEPT = Keyword("EXCEPT") - protected val DOUBLE = Keyword("DOUBLE") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") protected val FROM = Keyword("FROM") From 14c54f1876fcf91b5c10e80be2df5421c7328557 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 7 Nov 2014 11:56:40 -0800 Subject: [PATCH 1250/1492] [SPARK-4213][SQL] ParquetFilters - No support for LT, LTE, GT, GTE operators Following description is quoted from JIRA: When I issue a hql query against a HiveContext where my predicate uses a column of string type with one of LT, LTE, GT, or GTE operator, I get the following error: scala.MatchError: StringType (of class org.apache.spark.sql.catalyst.types.StringType$) Looking at the code in org.apache.spark.sql.parquet.ParquetFilters, StringType is absent from the corresponding functions for creating these filters. To reproduce, in a Hive 0.13.1 shell, I created the following table (at a specified DB): create table sparkbug ( id int, event string ) stored as parquet; Insert some sample data: insert into table sparkbug select 1, '2011-06-18' from limit 1; insert into table sparkbug select 2, '2012-01-01' from limit 1; Launch a spark shell and create a HiveContext to the metastore where the table above is located. import org.apache.spark.sql._ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveContext val hc = new HiveContext(sc) hc.setConf("spark.sql.shuffle.partitions", "10") hc.setConf("spark.sql.hive.convertMetastoreParquet", "true") hc.setConf("spark.sql.parquet.compression.codec", "snappy") import hc._ hc.hql("select * from .sparkbug where event >= '2011-12-01'") A scala.MatchError will appear in the output. Author: Kousuke Saruta Closes #3083 from sarutak/SPARK-4213 and squashes the following commits: 4ab6e56 [Kousuke Saruta] WIP b6890c6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-4213 9a1fae7 [Kousuke Saruta] Fixed ParquetFilters so that compare Strings --- .../spark/sql/parquet/ParquetFilters.scala | 335 +++++++++++++++++- .../spark/sql/parquet/ParquetQuerySuite.scala | 40 +++ 2 files changed, 364 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 517a5cf0029ed..1e67799e8399a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -18,13 +18,15 @@ package org.apache.spark.sql.parquet import java.nio.ByteBuffer +import java.sql.{Date, Timestamp} import org.apache.hadoop.conf.Configuration +import parquet.common.schema.ColumnPath import parquet.filter2.compat.FilterCompat import parquet.filter2.compat.FilterCompat._ -import parquet.filter2.predicate.FilterPredicate -import parquet.filter2.predicate.FilterApi +import parquet.filter2.predicate.Operators.{Column, SupportsLtGt} +import parquet.filter2.predicate.{FilterApi, FilterPredicate} import parquet.filter2.predicate.FilterApi._ import parquet.io.api.Binary import parquet.column.ColumnReader @@ -33,9 +35,11 @@ import com.google.common.io.BaseEncoding import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.parquet.ParquetColumns._ private[sql] object ParquetFilters { val PARQUET_FILTER_DATA = "org.apache.spark.sql.parquet.row.filter" @@ -50,15 +54,25 @@ private[sql] object ParquetFilters { if (filters.length > 0) FilterCompat.get(filters.reduce(FilterApi.and)) else null } - def createFilter(expression: Expression): Option[CatalystFilter] ={ + def createFilter(expression: Expression): Option[CatalystFilter] = { def createEqualityFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case BooleanType => - ComparisonFilter.createBooleanFilter( + ComparisonFilter.createBooleanEqualityFilter( name, - literal.value.asInstanceOf[Boolean], + literal.value.asInstanceOf[Boolean], + predicate) + case ByteType => + new ComparisonFilter( + name, + FilterApi.eq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), + predicate) + case ShortType => + new ComparisonFilter( + name, + FilterApi.eq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), predicate) case IntegerType => new ComparisonFilter( @@ -81,18 +95,49 @@ private[sql] object ParquetFilters { FilterApi.eq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) case StringType => - ComparisonFilter.createStringFilter( + ComparisonFilter.createStringEqualityFilter( name, literal.value.asInstanceOf[String], predicate) + case BinaryType => + ComparisonFilter.createBinaryEqualityFilter( + name, + literal.value.asInstanceOf[Array[Byte]], + predicate) + case DateType => + new ComparisonFilter( + name, + FilterApi.eq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), + predicate) + case TimestampType => + new ComparisonFilter( + name, + FilterApi.eq(timestampColumn(name), + new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), + predicate) + case DecimalType.Unlimited => + new ComparisonFilter( + name, + FilterApi.eq(decimalColumn(name), literal.value.asInstanceOf[Decimal]), + predicate) } def createLessThanFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { + case ByteType => + new ComparisonFilter( + name, + FilterApi.lt(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), + predicate) + case ShortType => + new ComparisonFilter( + name, + FilterApi.lt(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), + predicate) case IntegerType => - new ComparisonFilter( + new ComparisonFilter( name, FilterApi.lt(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) @@ -111,11 +156,47 @@ private[sql] object ParquetFilters { name, FilterApi.lt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) + case StringType => + ComparisonFilter.createStringLessThanFilter( + name, + literal.value.asInstanceOf[String], + predicate) + case BinaryType => + ComparisonFilter.createBinaryLessThanFilter( + name, + literal.value.asInstanceOf[Array[Byte]], + predicate) + case DateType => + new ComparisonFilter( + name, + FilterApi.lt(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), + predicate) + case TimestampType => + new ComparisonFilter( + name, + FilterApi.lt(timestampColumn(name), + new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), + predicate) + case DecimalType.Unlimited => + new ComparisonFilter( + name, + FilterApi.lt(decimalColumn(name), literal.value.asInstanceOf[Decimal]), + predicate) } def createLessThanOrEqualFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { + case ByteType => + new ComparisonFilter( + name, + FilterApi.ltEq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), + predicate) + case ShortType => + new ComparisonFilter( + name, + FilterApi.ltEq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), + predicate) case IntegerType => new ComparisonFilter( name, @@ -136,12 +217,48 @@ private[sql] object ParquetFilters { name, FilterApi.ltEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) + case StringType => + ComparisonFilter.createStringLessThanOrEqualFilter( + name, + literal.value.asInstanceOf[String], + predicate) + case BinaryType => + ComparisonFilter.createBinaryLessThanOrEqualFilter( + name, + literal.value.asInstanceOf[Array[Byte]], + predicate) + case DateType => + new ComparisonFilter( + name, + FilterApi.ltEq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), + predicate) + case TimestampType => + new ComparisonFilter( + name, + FilterApi.ltEq(timestampColumn(name), + new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), + predicate) + case DecimalType.Unlimited => + new ComparisonFilter( + name, + FilterApi.ltEq(decimalColumn(name), literal.value.asInstanceOf[Decimal]), + predicate) } // TODO: combine these two types somehow? def createGreaterThanFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { + case ByteType => + new ComparisonFilter( + name, + FilterApi.gt(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), + predicate) + case ShortType => + new ComparisonFilter( + name, + FilterApi.gt(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), + predicate) case IntegerType => new ComparisonFilter( name, @@ -162,11 +279,47 @@ private[sql] object ParquetFilters { name, FilterApi.gt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) + case StringType => + ComparisonFilter.createStringGreaterThanFilter( + name, + literal.value.asInstanceOf[String], + predicate) + case BinaryType => + ComparisonFilter.createBinaryGreaterThanFilter( + name, + literal.value.asInstanceOf[Array[Byte]], + predicate) + case DateType => + new ComparisonFilter( + name, + FilterApi.gt(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), + predicate) + case TimestampType => + new ComparisonFilter( + name, + FilterApi.gt(timestampColumn(name), + new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), + predicate) + case DecimalType.Unlimited => + new ComparisonFilter( + name, + FilterApi.gt(decimalColumn(name), literal.value.asInstanceOf[Decimal]), + predicate) } def createGreaterThanOrEqualFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { + case ByteType => + new ComparisonFilter( + name, + FilterApi.gtEq(byteColumn(name), literal.value.asInstanceOf[java.lang.Byte]), + predicate) + case ShortType => + new ComparisonFilter( + name, + FilterApi.gtEq(shortColumn(name), literal.value.asInstanceOf[java.lang.Short]), + predicate) case IntegerType => new ComparisonFilter( name, @@ -187,6 +340,32 @@ private[sql] object ParquetFilters { name, FilterApi.gtEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) + case StringType => + ComparisonFilter.createStringGreaterThanOrEqualFilter( + name, + literal.value.asInstanceOf[String], + predicate) + case BinaryType => + ComparisonFilter.createBinaryGreaterThanOrEqualFilter( + name, + literal.value.asInstanceOf[Array[Byte]], + predicate) + case DateType => + new ComparisonFilter( + name, + FilterApi.gtEq(dateColumn(name), new WrappedDate(literal.value.asInstanceOf[Date])), + predicate) + case TimestampType => + new ComparisonFilter( + name, + FilterApi.gtEq(timestampColumn(name), + new WrappedTimestamp(literal.value.asInstanceOf[Timestamp])), + predicate) + case DecimalType.Unlimited => + new ComparisonFilter( + name, + FilterApi.gtEq(decimalColumn(name), literal.value.asInstanceOf[Decimal]), + predicate) } /** @@ -221,9 +400,9 @@ private[sql] object ParquetFilters { case _ => None } } - case p @ EqualTo(left: Literal, right: NamedExpression) => + case p @ EqualTo(left: Literal, right: NamedExpression) if left.dataType != NullType => Some(createEqualityFilter(right.name, left, p)) - case p @ EqualTo(left: NamedExpression, right: Literal) => + case p @ EqualTo(left: NamedExpression, right: Literal) if right.dataType != NullType => Some(createEqualityFilter(left.name, right, p)) case p @ LessThan(left: Literal, right: NamedExpression) => Some(createLessThanFilter(right.name, left, p)) @@ -363,7 +542,7 @@ private[parquet] case class AndFilter( } private[parquet] object ComparisonFilter { - def createBooleanFilter( + def createBooleanEqualityFilter( columnName: String, value: Boolean, predicate: CatalystPredicate): CatalystFilter = @@ -372,7 +551,7 @@ private[parquet] object ComparisonFilter { FilterApi.eq(booleanColumn(columnName), value.asInstanceOf[java.lang.Boolean]), predicate) - def createStringFilter( + def createStringEqualityFilter( columnName: String, value: String, predicate: CatalystPredicate): CatalystFilter = @@ -380,4 +559,138 @@ private[parquet] object ComparisonFilter { columnName, FilterApi.eq(binaryColumn(columnName), Binary.fromString(value)), predicate) + + def createStringLessThanFilter( + columnName: String, + value: String, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.lt(binaryColumn(columnName), Binary.fromString(value)), + predicate) + + def createStringLessThanOrEqualFilter( + columnName: String, + value: String, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.ltEq(binaryColumn(columnName), Binary.fromString(value)), + predicate) + + def createStringGreaterThanFilter( + columnName: String, + value: String, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.gt(binaryColumn(columnName), Binary.fromString(value)), + predicate) + + def createStringGreaterThanOrEqualFilter( + columnName: String, + value: String, + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.gtEq(binaryColumn(columnName), Binary.fromString(value)), + predicate) + + def createBinaryEqualityFilter( + columnName: String, + value: Array[Byte], + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.eq(binaryColumn(columnName), Binary.fromByteArray(value)), + predicate) + + def createBinaryLessThanFilter( + columnName: String, + value: Array[Byte], + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.lt(binaryColumn(columnName), Binary.fromByteArray(value)), + predicate) + + def createBinaryLessThanOrEqualFilter( + columnName: String, + value: Array[Byte], + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.ltEq(binaryColumn(columnName), Binary.fromByteArray(value)), + predicate) + + def createBinaryGreaterThanFilter( + columnName: String, + value: Array[Byte], + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.gt(binaryColumn(columnName), Binary.fromByteArray(value)), + predicate) + + def createBinaryGreaterThanOrEqualFilter( + columnName: String, + value: Array[Byte], + predicate: CatalystPredicate): CatalystFilter = + new ComparisonFilter( + columnName, + FilterApi.gtEq(binaryColumn(columnName), Binary.fromByteArray(value)), + predicate) +} + +private[spark] object ParquetColumns { + + def byteColumn(columnPath: String): ByteColumn = { + new ByteColumn(ColumnPath.fromDotString(columnPath)) + } + + final class ByteColumn(columnPath: ColumnPath) + extends Column[java.lang.Byte](columnPath, classOf[java.lang.Byte]) with SupportsLtGt + + def shortColumn(columnPath: String): ShortColumn = { + new ShortColumn(ColumnPath.fromDotString(columnPath)) + } + + final class ShortColumn(columnPath: ColumnPath) + extends Column[java.lang.Short](columnPath, classOf[java.lang.Short]) with SupportsLtGt + + + def dateColumn(columnPath: String): DateColumn = { + new DateColumn(ColumnPath.fromDotString(columnPath)) + } + + final class DateColumn(columnPath: ColumnPath) + extends Column[WrappedDate](columnPath, classOf[WrappedDate]) with SupportsLtGt + + def timestampColumn(columnPath: String): TimestampColumn = { + new TimestampColumn(ColumnPath.fromDotString(columnPath)) + } + + final class TimestampColumn(columnPath: ColumnPath) + extends Column[WrappedTimestamp](columnPath, classOf[WrappedTimestamp]) with SupportsLtGt + + def decimalColumn(columnPath: String): DecimalColumn = { + new DecimalColumn(ColumnPath.fromDotString(columnPath)) + } + + final class DecimalColumn(columnPath: ColumnPath) + extends Column[Decimal](columnPath, classOf[Decimal]) with SupportsLtGt + + final class WrappedDate(val date: Date) extends Comparable[WrappedDate] { + + override def compareTo(other: WrappedDate): Int = { + date.compareTo(other.date) + } + } + + final class WrappedTimestamp(val timestamp: Timestamp) extends Comparable[WrappedTimestamp] { + + override def compareTo(other: WrappedTimestamp): Int = { + timestamp.compareTo(other.timestamp) + } + } } 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 08d9da27f1b11..3cccafe92d4f3 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 @@ -619,6 +619,46 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA fail(s"optional Int value in result row $i should be ${6*i}") } } + + val query12 = sql("SELECT * FROM testfiltersource WHERE mystring >= \"50\"") + assert( + query12.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result12 = query12.collect() + assert(result12.size === 54) + assert(result12(0).getString(2) == "6") + assert(result12(4).getString(2) == "50") + assert(result12(53).getString(2) == "99") + + val query13 = sql("SELECT * FROM testfiltersource WHERE mystring > \"50\"") + assert( + query13.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result13 = query13.collect() + assert(result13.size === 53) + assert(result13(0).getString(2) == "6") + assert(result13(4).getString(2) == "51") + assert(result13(52).getString(2) == "99") + + val query14 = sql("SELECT * FROM testfiltersource WHERE mystring <= \"50\"") + assert( + query14.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result14 = query14.collect() + assert(result14.size === 148) + assert(result14(0).getString(2) == "0") + assert(result14(46).getString(2) == "50") + assert(result14(147).getString(2) == "200") + + val query15 = sql("SELECT * FROM testfiltersource WHERE mystring < \"50\"") + assert( + query15.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result15 = query15.collect() + assert(result15.size === 147) + assert(result15(0).getString(2) == "0") + assert(result15(46).getString(2) == "100") + assert(result15(146).getString(2) == "200") } test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { From 60ab80f501b8384ddf48a9ac0ba0c2b9eb548b28 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 7 Nov 2014 12:15:53 -0800 Subject: [PATCH 1251/1492] [SPARK-4272] [SQL] Add more unwrapper functions for primitive type in TableReader Currently, the data "unwrap" only support couple of primitive types, not all, it will not cause exception, but may get some performance in table scanning for the type like binary, date, timestamp, decimal etc. Author: Cheng Hao Closes #3136 from chenghao-intel/table_reader and squashes the following commits: fffb729 [Cheng Hao] fix bug for retrieving the timestamp object e9c97a4 [Cheng Hao] Add more unwrapper functions for primitive type in TableReader --- .../apache/spark/sql/hive/HiveInspectors.scala | 4 ---- .../org/apache/spark/sql/hive/TableReader.scala | 15 +++++++++++++++ 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 58815daa82276..bdc7e1dac1922 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -115,10 +115,6 @@ private[hive] trait HiveInspectors { } - /** - * Wraps with Hive types based on object inspector. - * TODO: Consolidate all hive OI/data interface code. - */ /** * Wraps with Hive types based on object inspector. * TODO: Consolidate all hive OI/data interface code. 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 e49f0957d188a..f60bc3788e3e4 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 @@ -290,6 +290,21 @@ private[hive] object HadoopTableReader extends HiveInspectors { (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) case oi: DoubleObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) + case oi: HiveVarcharObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => + row.setString(ordinal, oi.getPrimitiveJavaObject(value).getValue) + case oi: HiveDecimalObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => + row.update(ordinal, HiveShim.toCatalystDecimal(oi, value)) + case oi: TimestampObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => + row.update(ordinal, oi.getPrimitiveJavaObject(value).clone()) + case oi: DateObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => + row.update(ordinal, oi.getPrimitiveJavaObject(value)) + case oi: BinaryObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => + row.update(ordinal, oi.getPrimitiveJavaObject(value)) case oi => (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrap(value, oi) } From a6405c5ddcda112f8efd7d50d8e5f44f78a0fa41 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 7 Nov 2014 12:30:47 -0800 Subject: [PATCH 1252/1492] [SPARK-4270][SQL] Fix Cast from DateType to DecimalType. `Cast` from `DateType` to `DecimalType` throws `NullPointerException`. Author: Takuya UESHIN Closes #3134 from ueshin/issues/SPARK-4270 and squashes the following commits: 7394e4b [Takuya UESHIN] Fix Cast from DateType to DecimalType. --- .../scala/org/apache/spark/sql/catalyst/expressions/Cast.scala | 2 +- .../sql/catalyst/expressions/ExpressionEvaluationSuite.scala | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 22009666196a1..55319e7a79103 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -281,7 +281,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case BooleanType => buildCast[Boolean](_, b => changePrecision(if (b) Decimal(1) else Decimal(0), target)) case DateType => - buildCast[Date](_, d => changePrecision(null, target)) // date can't cast to decimal in Hive + buildCast[Date](_, d => null) // date can't cast to decimal in Hive case TimestampType => // Note that we lose precision here. buildCast[Timestamp](_, t => changePrecision(Decimal(timestampToDouble(t)), target)) 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 6bfa0dbd65ba7..918996f11da2c 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 @@ -412,6 +412,8 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(d, LongType), null) checkEvaluation(Cast(d, FloatType), null) checkEvaluation(Cast(d, DoubleType), null) + checkEvaluation(Cast(d, DecimalType.Unlimited), null) + checkEvaluation(Cast(d, DecimalType(10, 2)), null) checkEvaluation(Cast(d, StringType), "1970-01-01") checkEvaluation(Cast(Cast(d, TimestampType), StringType), "1970-01-01 00:00:00") } From ac70c972a51952f801fd02dd5962c0a0c1aba8f8 Mon Sep 17 00:00:00 2001 From: Matthew Taylor Date: Fri, 7 Nov 2014 12:53:08 -0800 Subject: [PATCH 1253/1492] [SPARK-4203][SQL] Partition directories in random order when inserting into hive table When doing an insert into hive table with partitions the folders written to the file system are in a random order instead of the order defined in table creation. Seems that the loadPartition method in Hive.java has a Map parameter but expects to be called with a map that has a defined ordering such as LinkedHashMap. Working on a test but having intillij problems Author: Matthew Taylor Closes #3076 from tbfenet/partition_dir_order_problem and squashes the following commits: f1b9a52 [Matthew Taylor] Comment format fix bca709f [Matthew Taylor] review changes 0e50f6b [Matthew Taylor] test fix 99f1a31 [Matthew Taylor] partition ordering fix 369e618 [Matthew Taylor] partition ordering fix --- .../hive/execution/InsertIntoHiveTable.scala | 13 +++++-- .../sql/hive/InsertIntoHiveTableSuite.scala | 34 +++++++++++++++++-- 2 files changed, 43 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 74b4e7aaa47a5..81390f626726c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import java.util + import scala.collection.JavaConversions._ import org.apache.hadoop.hive.common.`type`.HiveVarchar @@ -203,6 +205,13 @@ case class InsertIntoHiveTable( // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. val holdDDLTime = false if (partition.nonEmpty) { + + // loadPartition call orders directories created on the iteration order of the this map + val orderedPartitionSpec = new util.LinkedHashMap[String,String]() + table.hiveQlTable.getPartCols().foreach{ + entry=> + orderedPartitionSpec.put(entry.getName,partitionSpec.get(entry.getName).getOrElse("")) + } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) db.validatePartitionNameCharacters(partVals) // inheritTableSpecs is set to true. It should be set to false for a IMPORT query @@ -214,7 +223,7 @@ case class InsertIntoHiveTable( db.loadDynamicPartitions( outputPath, qualifiedTableName, - partitionSpec, + orderedPartitionSpec, overwrite, numDynamicPartitions, holdDDLTime, @@ -224,7 +233,7 @@ case class InsertIntoHiveTable( db.loadPartition( outputPath, qualifiedTableName, - partitionSpec, + orderedPartitionSpec, overwrite, holdDDLTime, inheritTableSpecs, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 18dc937dd2b27..5dbfb923139fa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql._ +import java.io.File + +import com.google.common.io.Files +import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.hive.test.TestHive /* Implicits */ @@ -91,4 +93,32 @@ class InsertIntoHiveTableSuite extends QueryTest { sql("DROP TABLE hiveTableWithMapValue") } + + test("SPARK-4203:random partition directory order") { + createTable[TestData]("tmp_table") + val tmpDir = Files.createTempDir() + sql(s"CREATE TABLE table_with_partition(c1 string) PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='1') SELECT 'blarr' FROM tmp_table") + sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='2') SELECT 'blarr' FROM tmp_table") + sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='3') SELECT 'blarr' FROM tmp_table") + sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='4') SELECT 'blarr' FROM tmp_table") + def listFolders(path: File, acc: List[String]): List[List[String]] = { + val dir = path.listFiles() + val folders = dir.filter(_.isDirectory).toList + if (folders.isEmpty) { + List(acc.reverse) + } else { + folders.flatMap(x => listFolders(x, x.getName :: acc)) + } + } + val expected = List( + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=2"::Nil, + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=3"::Nil , + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=1"::Nil , + "p1=a"::"p2=b"::"p3=c"::"p4=c"::"p5=4"::Nil + ) + assert(listFolders(tmpDir,List()).sortBy(_.toString()) == expected.sortBy(_.toString)) + sql("DROP TABLE table_with_partition") + sql("DROP TABLE tmp_table") + } } From d6e55524437026c0c76addeba8f99249a8316716 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 7 Nov 2014 12:55:11 -0800 Subject: [PATCH 1254/1492] [SPARK-4292][SQL] Result set iterator bug in JDBC/ODBC select * from src, get the wrong result set as follows: ``` ... | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 309 | val_309 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | | 97 | val_97 | ... ``` Author: wangfei Closes #3149 from scwf/SPARK-4292 and squashes the following commits: 1574a43 [wangfei] using result.collect 8b2d845 [wangfei] adding test f64eddf [wangfei] result set iter bug --- .../thriftserver/HiveThriftServer2Suite.scala | 23 +++++++++++++++++++ .../spark/sql/hive/thriftserver/Shim12.scala | 5 ++-- .../spark/sql/hive/thriftserver/Shim13.scala | 5 ++-- 3 files changed, 27 insertions(+), 6 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 65d910a0c3ffc..bba29b2bdca4d 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -267,4 +267,27 @@ class HiveThriftServer2Suite extends FunSuite with Logging { assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}") } } + + test("SPARK-4292 regression: result set iterator issue") { + withJdbcStatement() { statement => + val dataFilePath = + Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") + + val queries = Seq( + "DROP TABLE IF EXISTS test_4292", + "CREATE TABLE test_4292(key INT, val STRING)", + s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test_4292") + + queries.foreach(statement.execute) + + val resultSet = statement.executeQuery("SELECT key FROM test_4292") + + Seq(238, 86, 311, 27, 165).foreach { key => + resultSet.next() + assert(resultSet.getInt(1) == key) + } + + statement.executeQuery("DROP TABLE IF EXISTS test_4292") + } + } } diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index 8077d0ec46fd7..e3ba9914c6cc0 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -202,13 +202,12 @@ private[hive] class SparkExecuteStatementOperation( hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) } iter = { - val resultRdd = result.queryExecution.toRdd val useIncrementalCollect = hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean if (useIncrementalCollect) { - resultRdd.toLocalIterator + result.toLocalIterator } else { - resultRdd.collect().iterator + result.collect().iterator } } dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 2c1983de1d0d5..f2ceba828296b 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -87,13 +87,12 @@ private[hive] class SparkExecuteStatementOperation( val groupId = round(random * 1000000).toString hiveContext.sparkContext.setJobGroup(groupId, statement) iter = { - val resultRdd = result.queryExecution.toRdd val useIncrementalCollect = hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean if (useIncrementalCollect) { - resultRdd.toLocalIterator + result.toLocalIterator } else { - resultRdd.collect().iterator + result.collect().iterator } } dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray From 7c9ec529a3483fab48f728481dd1d3663369e50a Mon Sep 17 00:00:00 2001 From: xiao321 <1042460381@qq.com> Date: Fri, 7 Nov 2014 12:56:49 -0800 Subject: [PATCH 1255/1492] Update JavaCustomReceiver.java MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit 数组下标越界 Author: xiao321 <1042460381@qq.com> Closes #3153 from xiao321/patch-1 and squashes the following commits: 0ed17b5 [xiao321] Update JavaCustomReceiver.java --- .../org/apache/spark/examples/streaming/JavaCustomReceiver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index 981bc4f0613a9..99df259b4e8e6 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -70,7 +70,7 @@ public static void main(String[] args) { // Create a input stream with the custom receiver on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') JavaReceiverInputDStream lines = ssc.receiverStream( - new JavaCustomReceiver(args[1], Integer.parseInt(args[2]))); + new JavaCustomReceiver(args[0], Integer.parseInt(args[1]))); JavaDStream words = lines.flatMap(new FlatMapFunction() { @Override public Iterable call(String x) { From 5923dd986ba26d0fcc8707dd8d16863f1c1005cb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 7 Nov 2014 13:08:25 -0800 Subject: [PATCH 1256/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #3016 (close requested by 'andrewor14') Closes #2798 (close requested by 'andrewor14') Closes #2864 (close requested by 'andrewor14') Closes #3154 (close requested by 'JoshRosen') Closes #3156 (close requested by 'JoshRosen') Closes #214 (close requested by 'kayousterhout') Closes #2584 (close requested by 'andrewor14') From 7779109796c90d789464ab0be35917f963bbe867 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 7 Nov 2014 20:53:03 -0800 Subject: [PATCH 1257/1492] [SPARK-4304] [PySpark] Fix sort on empty RDD This PR fix sortBy()/sortByKey() on empty RDD. This should be back ported into 1.1/1.2 Author: Davies Liu Closes #3162 from davies/fix_sort and squashes the following commits: 84f64b7 [Davies Liu] add tests 52995b5 [Davies Liu] fix sortByKey() on empty RDD --- python/pyspark/rdd.py | 2 ++ python/pyspark/tests.py | 3 +++ 2 files changed, 5 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 879655dc53f4a..08d047402625f 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -521,6 +521,8 @@ def sortPartition(iterator): # the key-space into bins such that the bins have roughly the same # number of (key, value) pairs falling into them rddSize = self.count() + if not rddSize: + return self # empty RDD maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner fraction = min(maxSampleSize / max(rddSize, 1), 1.0) samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 9f625c5c6ca48..491e445a216bf 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -649,6 +649,9 @@ def test_distinct(self): self.assertEquals(result.getNumPartitions(), 5) self.assertEquals(result.count(), 3) + def test_sort_on_empty_rdd(self): + self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect()) + def test_sample(self): rdd = self.sc.parallelize(range(0, 100), 4) wo = rdd.sample(False, 0.1, 2).collect() From 7e9d975676d56ace0e84c2200137e4cd4eba074a Mon Sep 17 00:00:00 2001 From: Michelangelo D'Agostino Date: Fri, 7 Nov 2014 22:53:01 -0800 Subject: [PATCH 1258/1492] [MLLIB] [PYTHON] SPARK-4221: Expose nonnegative ALS in the python API SPARK-1553 added alternating nonnegative least squares to MLLib, however it's not possible to access it via the python API. This pull request resolves that. Author: Michelangelo D'Agostino Closes #3095 from mdagost/python_nmf and squashes the following commits: a6743ad [Michelangelo D'Agostino] Use setters instead of static methods in PythonMLLibAPI. Remove the new static methods I added. Set seed in tests. Change ratings to ratingsRDD in both train and trainImplicit for consistency. 7cffd39 [Michelangelo D'Agostino] Swapped nonnegative and seed in a few more places. 3fdc851 [Michelangelo D'Agostino] Moved seed to the end of the python parameter list. bdcc154 [Michelangelo D'Agostino] Change seed type to java.lang.Long so that it can handle null. cedf043 [Michelangelo D'Agostino] Added in ability to set the seed from python and made that play nice with the nonnegative changes. Also made the python ALS tests more exact. a72fdc9 [Michelangelo D'Agostino] Expose nonnegative ALS in the python API. --- .../mllib/api/python/PythonMLLibAPI.scala | 39 +++++++++++++++--- python/pyspark/mllib/recommendation.py | 40 ++++++++++++------- 2 files changed, 58 insertions(+), 21 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index d832ae34b55e4..70d7138e3060f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -275,12 +275,25 @@ class PythonMLLibAPI extends Serializable { * the Py4J documentation. */ def trainALSModel( - ratings: JavaRDD[Rating], + ratingsJRDD: JavaRDD[Rating], rank: Int, iterations: Int, lambda: Double, - blocks: Int): MatrixFactorizationModel = { - new MatrixFactorizationModelWrapper(ALS.train(ratings.rdd, rank, iterations, lambda, blocks)) + blocks: Int, + nonnegative: Boolean, + seed: java.lang.Long): MatrixFactorizationModel = { + + val als = new ALS() + .setRank(rank) + .setIterations(iterations) + .setLambda(lambda) + .setBlocks(blocks) + .setNonnegative(nonnegative) + + if (seed != null) als.setSeed(seed) + + val model = als.run(ratingsJRDD.rdd) + new MatrixFactorizationModelWrapper(model) } /** @@ -295,9 +308,23 @@ class PythonMLLibAPI extends Serializable { iterations: Int, lambda: Double, blocks: Int, - alpha: Double): MatrixFactorizationModel = { - new MatrixFactorizationModelWrapper( - ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha)) + alpha: Double, + nonnegative: Boolean, + seed: java.lang.Long): MatrixFactorizationModel = { + + val als = new ALS() + .setImplicitPrefs(true) + .setRank(rank) + .setIterations(iterations) + .setLambda(lambda) + .setBlocks(blocks) + .setAlpha(alpha) + .setNonnegative(nonnegative) + + if (seed != null) als.setSeed(seed) + + val model = als.run(ratingsJRDD.rdd) + new MatrixFactorizationModelWrapper(model) } /** diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index e8b998414d319..e26b152e0cdfd 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -44,31 +44,39 @@ class MatrixFactorizationModel(JavaModelWrapper): >>> r2 = (1, 2, 2.0) >>> r3 = (2, 1, 2.0) >>> ratings = sc.parallelize([r1, r2, r3]) - >>> model = ALS.trainImplicit(ratings, 1) - >>> model.predict(2,2) is not None - True + >>> model = ALS.trainImplicit(ratings, 1, seed=10) + >>> model.predict(2,2) + 0.4473... >>> testset = sc.parallelize([(1, 2), (1, 1)]) - >>> model = ALS.train(ratings, 1) - >>> model.predictAll(testset).count() == 2 - True + >>> model = ALS.train(ratings, 1, seed=10) + >>> model.predictAll(testset).collect() + [Rating(1, 1, 1), Rating(1, 2, 1)] - >>> model = ALS.train(ratings, 4) - >>> model.userFeatures().count() == 2 - True + >>> model = ALS.train(ratings, 4, seed=10) + >>> model.userFeatures().collect() + [(2, array('d', [...])), (1, array('d', [...]))] >>> first_user = model.userFeatures().take(1)[0] >>> latents = first_user[1] >>> len(latents) == 4 True - >>> model.productFeatures().count() == 2 - True + >>> model.productFeatures().collect() + [(2, array('d', [...])), (1, array('d', [...]))] >>> first_product = model.productFeatures().take(1)[0] >>> latents = first_product[1] >>> len(latents) == 4 True + + >>> model = ALS.train(ratings, 1, nonnegative=True, seed=10) + >>> model.predict(2,2) + 3.735... + + >>> model = ALS.trainImplicit(ratings, 1, nonnegative=True, seed=10) + >>> model.predict(2,2) + 0.4473... """ def predict(self, user, product): return self._java_model.predict(user, product) @@ -101,15 +109,17 @@ def _prepare(cls, ratings): return _to_java_object_rdd(ratings, True) @classmethod - def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): + def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, nonnegative=False, + seed=None): model = callMLlibFunc("trainALSModel", cls._prepare(ratings), rank, iterations, - lambda_, blocks) + lambda_, blocks, nonnegative, seed) return MatrixFactorizationModel(model) @classmethod - def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01): + def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01, + nonnegative=False, seed=None): model = callMLlibFunc("trainImplicitALSModel", cls._prepare(ratings), rank, - iterations, lambda_, blocks, alpha) + iterations, lambda_, blocks, alpha, nonnegative, seed) return MatrixFactorizationModel(model) From 7afc8564f33eb2868f458f85046f59a51b516ed6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 7 Nov 2014 23:16:13 -0800 Subject: [PATCH 1259/1492] [SPARK-4291][Build] Rename network module projects The names of the recently introduced network modules are inconsistent with those of the other modules in the project. We should just drop the "Code" suffix since it doesn't sacrifice any meaning, especially before they get into an official release. ``` [INFO] Reactor Build Order: [INFO] [INFO] Spark Project Parent POM [INFO] Spark Project Common Network Code [INFO] Spark Project Shuffle Streaming Service Code [INFO] Spark Project Core [INFO] Spark Project Bagel [INFO] Spark Project GraphX [INFO] Spark Project Streaming [INFO] Spark Project Catalyst [INFO] Spark Project SQL [INFO] Spark Project ML Library [INFO] Spark Project Tools [INFO] Spark Project Hive [INFO] Spark Project REPL [INFO] Spark Project YARN Parent POM [INFO] Spark Project YARN Stable API [INFO] Spark Project Assembly [INFO] Spark Project External Twitter [INFO] Spark Project External Kafka [INFO] Spark Project External Flume Sink [INFO] Spark Project External Flume [INFO] Spark Project External ZeroMQ [INFO] Spark Project External MQTT [INFO] Spark Project Examples [INFO] Spark Project Yarn Shuffle Service Code ``` Author: Andrew Or Closes #3148 from andrewor14/build-drop-code and squashes the following commits: eac839b [Andrew Or] Network -> Networking d01ad47 [Andrew Or] Rename network module project names --- network/common/pom.xml | 2 +- network/shuffle/pom.xml | 2 +- network/yarn/pom.xml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/network/common/pom.xml b/network/common/pom.xml index 6144548a8f998..8b24ebf1ba1f2 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -29,7 +29,7 @@ org.apache.spark spark-network-common_2.10 jar - Spark Project Common Network Code + Spark Project Networking http://spark.apache.org/ network-common diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index fe5681d463499..27c8467687f10 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -29,7 +29,7 @@ org.apache.spark spark-network-shuffle_2.10 jar - Spark Project Shuffle Streaming Service Code + Spark Project Shuffle Streaming Service http://spark.apache.org/ network-shuffle diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml index e60d8c1f7876c..6e6f6f3e79296 100644 --- a/network/yarn/pom.xml +++ b/network/yarn/pom.xml @@ -29,7 +29,7 @@ org.apache.spark spark-network-yarn_2.10 jar - Spark Project Yarn Shuffle Service Code + Spark Project YARN Shuffle Service http://spark.apache.org/ network-yarn From 4af5c7e24455246c61c1f3c22225507e720d721d Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sat, 8 Nov 2014 13:03:51 -0800 Subject: [PATCH 1260/1492] [Minor] [Core] Don't NPE on closeQuietly(null) Author: Aaron Davidson Closes #3166 from aarondav/closeQuietlyer and squashes the following commits: 78096b5 [Aaron Davidson] Don't NPE on closeQuietly(null) --- .../main/java/org/apache/spark/network/util/JavaUtils.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 009dbcf01323f..bf8a1fc42fc6d 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -44,7 +44,9 @@ public class JavaUtils { /** Closes the given object, ignoring IOExceptions. */ public static void closeQuietly(Closeable closeable) { try { - closeable.close(); + if (closeable != null) { + closeable.close(); + } } catch (IOException e) { logger.error("IOException should not have been thrown.", e); } From 7b41b17f3296eea3282efbdceb6b28baf128287d Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 8 Nov 2014 18:10:23 -0800 Subject: [PATCH 1261/1492] [SPARK-4301] StreamingContext should not allow start() to be called after calling stop() In Spark 1.0.0+, calling `stop()` on a StreamingContext that has not been started is a no-op which has no side-effects. This allows users to call `stop()` on a fresh StreamingContext followed by `start()`. I believe that this almost always indicates an error and is not behavior that we should support. Since we don't allow `start() stop() start()` then I don't think it makes sense to allow `stop() start()`. The current behavior can lead to resource leaks when StreamingContext constructs its own SparkContext: if I call `stop(stopSparkContext=True)`, then I expect StreamingContext's underlying SparkContext to be stopped irrespective of whether the StreamingContext has been started. This is useful when writing unit test fixtures. Prior discussions: - https://github.com/apache/spark/pull/3053#discussion-diff-19710333R490 - https://github.com/apache/spark/pull/3121#issuecomment-61927353 Author: Josh Rosen Closes #3160 from JoshRosen/SPARK-4301 and squashes the following commits: dbcc929 [Josh Rosen] Address more review comments bdbe5da [Josh Rosen] Stop SparkContext after stopping scheduler, not before. 03e9c40 [Josh Rosen] Always stop SparkContext, even if stop(false) has already been called. 832a7f4 [Josh Rosen] Address review comment 5142517 [Josh Rosen] Add tests; improve Scaladoc. 813e471 [Josh Rosen] Revert workaround added in https://github.com/apache/spark/pull/3053/files#diff-e144dbee130ed84f9465853ddce65f8eR49 5558e70 [Josh Rosen] StreamingContext.stop() should stop SparkContext even if StreamingContext has not been started yet. --- .../spark/streaming/StreamingContext.scala | 38 ++++++++++--------- .../streaming/StreamingContextSuite.scala | 25 +++++++++--- 2 files changed, 40 insertions(+), 23 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 23d6d1c5e50fa..54b219711efb9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -436,10 +436,10 @@ class StreamingContext private[streaming] ( /** * Start the execution of the streams. + * + * @throws SparkException if the context has already been started or stopped. */ def start(): Unit = synchronized { - // Throw exception if the context has already been started once - // or if a stopped context is being started again if (state == Started) { throw new SparkException("StreamingContext has already been started") } @@ -472,8 +472,10 @@ class StreamingContext private[streaming] ( /** * Stop the execution of the streams immediately (does not wait for all received data * to be processed). - * @param stopSparkContext Stop the associated SparkContext or not * + * @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext + * will be stopped regardless of whether this StreamingContext has been + * started. */ def stop(stopSparkContext: Boolean = true): Unit = synchronized { stop(stopSparkContext, false) @@ -482,25 +484,27 @@ class StreamingContext private[streaming] ( /** * Stop the execution of the streams, with option of ensuring all received data * has been processed. - * @param stopSparkContext Stop the associated SparkContext or not - * @param stopGracefully Stop gracefully by waiting for the processing of all + * + * @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext + * will be stopped regardless of whether this StreamingContext has been + * started. + * @param stopGracefully if true, stops gracefully by waiting for the processing of all * received data to be completed */ def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = synchronized { - // Warn (but not fail) if context is stopped twice, - // or context is stopped before starting - if (state == Initialized) { - logWarning("StreamingContext has not been started yet") - return + state match { + case Initialized => logWarning("StreamingContext has not been started yet") + case Stopped => logWarning("StreamingContext has already been stopped") + case Started => + scheduler.stop(stopGracefully) + logInfo("StreamingContext stopped successfully") + waiter.notifyStop() } - if (state == Stopped) { - logWarning("StreamingContext has already been stopped") - return - } // no need to throw an exception as its okay to stop twice - scheduler.stop(stopGracefully) - logInfo("StreamingContext stopped successfully") - waiter.notifyStop() + // Even if the streaming context has not been started, we still need to stop the SparkContext. + // Even if we have already stopped, we still need to attempt to stop the SparkContext because + // a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true). if (stopSparkContext) sc.stop() + // The state should always be Stopped after calling `stop()`, even if we haven't started yet: state = Stopped } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index f47772947d67c..4b49c4d251645 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -46,10 +46,6 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w after { if (ssc != null) { ssc.stop() - if (ssc.sc != null) { - // Calling ssc.stop() does not always stop the associated SparkContext. - ssc.sc.stop() - } ssc = null } if (sc != null) { @@ -137,11 +133,16 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc.stop() } - test("stop before start and start after stop") { + test("stop before start") { ssc = new StreamingContext(master, appName, batchDuration) addInputStream(ssc).register() ssc.stop() // stop before start should not throw exception - ssc.start() + } + + test("start after stop") { + // Regression test for SPARK-4301 + ssc = new StreamingContext(master, appName, batchDuration) + addInputStream(ssc).register() ssc.stop() intercept[SparkException] { ssc.start() // start after stop should throw exception @@ -161,6 +162,18 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc.stop() } + test("stop(stopSparkContext=true) after stop(stopSparkContext=false)") { + ssc = new StreamingContext(master, appName, batchDuration) + addInputStream(ssc).register() + ssc.stop(stopSparkContext = false) + assert(ssc.sc.makeRDD(1 to 100).collect().size === 100) + ssc.stop(stopSparkContext = true) + // Check that the SparkContext is actually stopped: + intercept[Exception] { + ssc.sc.makeRDD(1 to 100).collect() + } + } + test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) conf.set("spark.cleaner.ttl", "3600") From 8c99a47a4f0369ff3c1ecaeb860fa61ee789e987 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 9 Nov 2014 17:40:48 -0800 Subject: [PATCH 1262/1492] SPARK-971 [DOCS] Link to Confluence wiki from project website / documentation This is a trivial change to add links to the wiki from `README.md` and the main docs page. It is already linked to from spark.apache.org. Author: Sean Owen Closes #3169 from srowen/SPARK-971 and squashes the following commits: dcb84d0 [Sean Owen] Add link to wiki from README, docs home page --- README.md | 3 ++- docs/index.md | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 9916ac7b1ae8e..8d57d50da96c9 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,8 @@ and Spark Streaming for stream processing. ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the [project web page](http://spark.apache.org/documentation.html). +guide, on the [project web page](http://spark.apache.org/documentation.html) +and [project wiki](https://cwiki.apache.org/confluence/display/SPARK). This README file only contains basic setup instructions. ## Building Spark diff --git a/docs/index.md b/docs/index.md index edd622ec90f64..171d6ddad62f3 100644 --- a/docs/index.md +++ b/docs/index.md @@ -112,6 +112,7 @@ options for deployment: **External Resources:** * [Spark Homepage](http://spark.apache.org) +* [Spark Wiki](https://cwiki.apache.org/confluence/display/SPARK) * [Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here * [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and exercises about Spark, Spark Streaming, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/3/), From d1362659ef5d62db2c9ff0d2a24639abcef4e118 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 9 Nov 2014 17:42:08 -0800 Subject: [PATCH 1263/1492] SPARK-1344 [DOCS] Scala API docs for top methods Use "k" in javadoc of top and takeOrdered to avoid confusion with type K in pair RDDs. I think this resolves the discussion in SPARK-1344. Author: Sean Owen Closes #3168 from srowen/SPARK-1344 and squashes the following commits: 6963fcc [Sean Owen] Use "k" in javadoc of top and takeOrdered to avoid confusion with type K in pair RDDs --- .../org/apache/spark/api/java/JavaRDDLike.scala | 16 ++++++++-------- .../main/scala/org/apache/spark/rdd/RDD.scala | 8 ++++---- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index efb8978f7ce12..5a8e5bb1f721a 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -493,9 +493,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD as defined by + * Returns the top k (largest) elements from this RDD as defined by * the specified Comparator[T]. - * @param num the number of top elements to return + * @param num k, the number of top elements to return * @param comp the comparator that defines the order * @return an array of top elements */ @@ -507,9 +507,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the top K elements from this RDD using the + * Returns the top k (largest) elements from this RDD using the * natural ordering for T. - * @param num the number of top elements to return + * @param num k, the number of top elements to return * @return an array of top elements */ def top(num: Int): JList[T] = { @@ -518,9 +518,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the first K elements from this RDD as defined by + * Returns the first k (smallest) elements from this RDD as defined by * the specified Comparator[T] and maintains the order. - * @param num the number of top elements to return + * @param num k, the number of elements to return * @param comp the comparator that defines the order * @return an array of top elements */ @@ -552,9 +552,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { } /** - * Returns the first K elements from this RDD using the + * Returns the first k (smallest) elements from this RDD using the * natural ordering for T while maintain the order. - * @param num the number of top elements to return + * @param num k, the number of top elements to return * @return an array of top elements */ def takeOrdered(num: Int): JList[T] = { 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 c169b2d3fe97f..716f2dd17733b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1096,7 +1096,7 @@ abstract class RDD[T: ClassTag]( } /** - * Returns the top K (largest) elements from this RDD as defined by the specified + * Returns the top k (largest) elements from this RDD as defined by the specified * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For example: * {{{ * sc.parallelize(Seq(10, 4, 2, 12, 3)).top(1) @@ -1106,14 +1106,14 @@ abstract class RDD[T: ClassTag]( * // returns Array(6, 5) * }}} * - * @param num the number of top elements to return + * @param num k, the number of top elements to return * @param ord the implicit ordering for T * @return an array of top elements */ def top(num: Int)(implicit ord: Ordering[T]): Array[T] = takeOrdered(num)(ord.reverse) /** - * Returns the first K (smallest) elements from this RDD as defined by the specified + * Returns the first k (smallest) elements from this RDD as defined by the specified * implicit Ordering[T] and maintains the ordering. This does the opposite of [[top]]. * For example: * {{{ @@ -1124,7 +1124,7 @@ abstract class RDD[T: ClassTag]( * // returns Array(2, 3) * }}} * - * @param num the number of top elements to return + * @param num k, the number of elements to return * @param ord the implicit ordering for T * @return an array of top elements */ From f73b56f5e5d94f83d980475d3f39548986a92dd6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Nov 2014 18:16:20 -0800 Subject: [PATCH 1264/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #464 (close requested by 'JoshRosen') Closes #283 (close requested by 'pwendell') Closes #449 (close requested by 'pwendell') Closes #907 (close requested by 'pwendell') Closes #2478 (close requested by 'JoshRosen') Closes #2192 (close requested by 'tdas') Closes #918 (close requested by 'pwendell') Closes #1465 (close requested by 'pwendell') Closes #3135 (close requested by 'JoshRosen') Closes #1693 (close requested by 'tdas') Closes #1279 (close requested by 'pwendell') From f8e5732307dcb1482d9bcf1162a1090ef9a7b913 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 9 Nov 2014 22:11:20 -0800 Subject: [PATCH 1265/1492] SPARK-1209 [CORE] (Take 2) SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop andrewor14 Another try at SPARK-1209, to address https://github.com/apache/spark/pull/2814#issuecomment-61197619 I successfully tested with `mvn -Dhadoop.version=1.0.4 -DskipTests clean package; mvn -Dhadoop.version=1.0.4 test` I assume that is what failed Jenkins last time. I also tried `-Dhadoop.version1.2.1` and `-Phadoop-2.4 -Pyarn -Phive` for more coverage. So this is why the class was put in `org.apache.hadoop` to begin with, I assume. One option is to leave this as-is for now and move it only when Hadoop 1.0.x support goes away. This is the other option, which adds a call to force the constructor to be public at run-time. It's probably less surprising than putting Spark code in `org.apache.hadoop`, but, does involve reflection. A `SecurityManager` might forbid this, but it would forbid a lot of stuff Spark does. This would also only affect Hadoop 1.0.x it seems. Author: Sean Owen Closes #3048 from srowen/SPARK-1209 and squashes the following commits: 0d48f4b [Sean Owen] For Hadoop 1.0.x, make certain constructors public, which were public in later versions 466e179 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though? eb61820 [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark --- .../org/apache/spark/SparkHadoopWriter.scala | 1 + .../mapred/SparkHadoopMapRedUtil.scala | 17 +++++++++++++++-- .../mapreduce/SparkHadoopMapReduceUtil.scala | 5 +++-- .../org/apache/spark/rdd/NewHadoopRDD.scala | 1 + .../org/apache/spark/rdd/PairRDDFunctions.scala | 3 ++- project/MimaExcludes.scala | 8 ++++++++ .../sql/parquet/ParquetTableOperations.scala | 1 + .../spark/sql/hive/hiveWriterContainers.scala | 1 + 8 files changed, 32 insertions(+), 5 deletions(-) rename core/src/main/scala/org/apache/{hadoop => spark}/mapred/SparkHadoopMapRedUtil.scala (79%) rename core/src/main/scala/org/apache/{hadoop => spark}/mapreduce/SparkHadoopMapReduceUtil.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 376e69cd997d5..40237596570de 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD /** diff --git a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala similarity index 79% rename from core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala rename to core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 0c47afae54c8b..21b782edd2a9e 100644 --- a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -15,15 +15,24 @@ * limitations under the License. */ -package org.apache.hadoop.mapred +package org.apache.spark.mapred -private[apache] +import java.lang.reflect.Modifier + +import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} + +private[spark] trait SparkHadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = { val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", "org.apache.hadoop.mapred.JobContext") val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[org.apache.hadoop.mapreduce.JobID]) + // In Hadoop 1.0.x, JobContext is an interface, and JobContextImpl is package private. + // Make it accessible if it's not in order to access it. + if (!Modifier.isPublic(ctor.getModifiers)) { + ctor.setAccessible(true) + } ctor.newInstance(conf, jobId).asInstanceOf[JobContext] } @@ -31,6 +40,10 @@ trait SparkHadoopMapRedUtil { val klass = firstAvailableClass("org.apache.hadoop.mapred.TaskAttemptContextImpl", "org.apache.hadoop.mapred.TaskAttemptContext") val ctor = klass.getDeclaredConstructor(classOf[JobConf], classOf[TaskAttemptID]) + // See above + if (!Modifier.isPublic(ctor.getModifiers)) { + ctor.setAccessible(true) + } ctor.newInstance(conf, attemptId).asInstanceOf[TaskAttemptContext] } diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala similarity index 96% rename from core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala rename to core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala index 1fca5729c6092..3340673f91156 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.hadoop.mapreduce +package org.apache.spark.mapreduce import java.lang.{Boolean => JBoolean, Integer => JInteger} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} -private[apache] +private[spark] trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 351e145f96f9a..e55d03d391e03 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil 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 da89f634abaea..462f0d6268a86 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -33,13 +33,14 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, -RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} +RecordWriter => NewRecordWriter} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 6a0495f8fd540..a94d09be3bec6 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -77,6 +77,14 @@ object MimaExcludes { // SPARK-3822 ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") + ) ++ Seq( + // SPARK-1209 + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.hadoop.mapred.SparkHadoopMapRedUtil"), + ProblemFilters.exclude[MissingTypesProblem]( + "org.apache.spark.rdd.PairRDDFunctions") ) case v if v.startsWith("1.1") => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index d00860a8bb8a6..74c43e053b03c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -43,6 +43,7 @@ import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index bf2ce9df67c58..cc8bb3e172c6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc} From 3c2cff4b9464f8d7535564fcd194631a8e5bb0a5 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sun, 9 Nov 2014 22:29:03 -0800 Subject: [PATCH 1266/1492] SPARK-3179. Add task OutputMetrics. Author: Sandy Ryza This patch had conflicts when merged, resolved by Committer: Kay Ousterhout Closes #2968 from sryza/sandy-spark-3179 and squashes the following commits: dce4784 [Sandy Ryza] More review feedback 8d350d1 [Sandy Ryza] Fix test against Hadoop 2.5+ e7c74d0 [Sandy Ryza] More review feedback 6cff9c4 [Sandy Ryza] Review feedback fb2dde0 [Sandy Ryza] SPARK-3179 --- .../apache/spark/deploy/SparkHadoopUtil.scala | 46 ++++++- .../apache/spark/executor/TaskMetrics.scala | 28 ++++ .../apache/spark/rdd/PairRDDFunctions.scala | 51 ++++++- .../apache/spark/scheduler/JobLogger.scala | 7 +- .../scala/org/apache/spark/ui/ToolTips.scala | 2 + .../apache/spark/ui/exec/ExecutorsTab.scala | 5 + .../apache/spark/ui/jobs/ExecutorTable.scala | 3 + .../spark/ui/jobs/JobProgressListener.scala | 6 + .../org/apache/spark/ui/jobs/StagePage.scala | 29 +++- .../org/apache/spark/ui/jobs/StageTable.scala | 4 + .../org/apache/spark/ui/jobs/UIData.scala | 2 + .../org/apache/spark/util/JsonProtocol.scala | 21 ++- ...te.scala => InputOutputMetricsSuite.scala} | 41 +++++- .../spark/scheduler/SparkListenerSuite.scala | 1 + .../ui/jobs/JobProgressListenerSuite.scala | 7 + .../apache/spark/util/JsonProtocolSuite.scala | 124 ++++++++++++++++-- 16 files changed, 346 insertions(+), 31 deletions(-) rename core/src/test/scala/org/apache/spark/metrics/{InputMetricsSuite.scala => InputOutputMetricsSuite.scala} (67%) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index e28eaad8a5180..60ee115e393ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy +import java.lang.reflect.Method import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration @@ -133,14 +134,9 @@ class SparkHadoopUtil extends Logging { */ private[spark] def getFSBytesReadOnThreadCallback(path: Path, conf: Configuration) : Option[() => Long] = { - val qualifiedPath = path.getFileSystem(conf).makeQualified(path) - val scheme = qualifiedPath.toUri().getScheme() - val stats = FileSystem.getAllStatistics().filter(_.getScheme().equals(scheme)) try { - val threadStats = stats.map(Utils.invoke(classOf[Statistics], _, "getThreadStatistics")) - val statisticsDataClass = - Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") - val getBytesReadMethod = statisticsDataClass.getDeclaredMethod("getBytesRead") + val threadStats = getFileSystemThreadStatistics(path, conf) + val getBytesReadMethod = getFileSystemThreadStatisticsMethod("getBytesRead") val f = () => threadStats.map(getBytesReadMethod.invoke(_).asInstanceOf[Long]).sum val baselineBytesRead = f() Some(() => f() - baselineBytesRead) @@ -151,6 +147,42 @@ class SparkHadoopUtil extends Logging { } } } + + /** + * Returns a function that can be called to find Hadoop FileSystem bytes written. If + * getFSBytesWrittenOnThreadCallback is called from thread r at time t, the returned callback will + * return the bytes written on r since t. Reflection is required because thread-level FileSystem + * statistics are only available as of Hadoop 2.5 (see HADOOP-10688). + * Returns None if the required method can't be found. + */ + private[spark] def getFSBytesWrittenOnThreadCallback(path: Path, conf: Configuration) + : Option[() => Long] = { + try { + val threadStats = getFileSystemThreadStatistics(path, conf) + val getBytesWrittenMethod = getFileSystemThreadStatisticsMethod("getBytesWritten") + val f = () => threadStats.map(getBytesWrittenMethod.invoke(_).asInstanceOf[Long]).sum + val baselineBytesWritten = f() + Some(() => f() - baselineBytesWritten) + } catch { + case e: NoSuchMethodException => { + logDebug("Couldn't find method for retrieving thread-level FileSystem output data", e) + None + } + } + } + + private def getFileSystemThreadStatistics(path: Path, conf: Configuration): Seq[AnyRef] = { + val qualifiedPath = path.getFileSystem(conf).makeQualified(path) + val scheme = qualifiedPath.toUri().getScheme() + val stats = FileSystem.getAllStatistics().filter(_.getScheme().equals(scheme)) + stats.map(Utils.invoke(classOf[Statistics], _, "getThreadStatistics")) + } + + private def getFileSystemThreadStatisticsMethod(methodName: String): Method = { + val statisticsDataClass = + Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") + statisticsDataClass.getDeclaredMethod(methodName) + } } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 57bc2b40cec44..51b5328cb4c8f 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -82,6 +82,12 @@ class TaskMetrics extends Serializable { */ var inputMetrics: Option[InputMetrics] = None + /** + * If this task writes data externally (e.g. to a distributed filesystem), metrics on how much + * data was written are stored here. + */ + var outputMetrics: Option[OutputMetrics] = None + /** * If this task reads from shuffle output, metrics on getting shuffle data will be collected here. * This includes read metrics aggregated over all the task's shuffle dependencies. @@ -157,6 +163,16 @@ object DataReadMethod extends Enumeration with Serializable { val Memory, Disk, Hadoop, Network = Value } +/** + * :: DeveloperApi :: + * Method by which output data was written. + */ +@DeveloperApi +object DataWriteMethod extends Enumeration with Serializable { + type DataWriteMethod = Value + val Hadoop = Value +} + /** * :: DeveloperApi :: * Metrics about reading input data. @@ -169,6 +185,18 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { var bytesRead: Long = 0L } +/** + * :: DeveloperApi :: + * Metrics about writing output data. + */ +@DeveloperApi +case class OutputMetrics(writeMethod: DataWriteMethod.Value) { + /** + * Total bytes written + */ + var bytesWritten: Long = 0L +} + /** * :: DeveloperApi :: * Metrics pertaining to shuffle data read in a given task. 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 462f0d6268a86..8c2c959e73bb6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -28,7 +28,7 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} @@ -40,6 +40,7 @@ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer @@ -962,30 +963,40 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { + val config = wrappedConf.value // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) - val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) + val hadoopContext = newTaskAttemptContext(config, attemptId) val format = outfmt.newInstance format match { - case c: Configurable => c.setConf(wrappedConf.value) + case c: Configurable => c.setConf(config) case _ => () } val committer = format.getOutputCommitter(hadoopContext) committer.setupTask(hadoopContext) + + val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context, config) + val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] try { + var recordsWritten = 0L while (iter.hasNext) { val pair = iter.next() writer.write(pair._1, pair._2) + + // Update bytes written metric every few records + maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) + recordsWritten += 1 } } finally { writer.close(hadoopContext) } committer.commitTask(hadoopContext) + bytesWrittenCallback.foreach { fn => outputMetrics.bytesWritten = fn() } 1 } : Int @@ -1006,6 +1017,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def saveAsHadoopDataset(conf: JobConf) { // Rename this as hadoopConf internally to avoid shadowing (see SPARK-2038). val hadoopConf = conf + val wrappedConf = new SerializableWritable(hadoopConf) val outputFormatInstance = hadoopConf.getOutputFormat val keyClass = hadoopConf.getOutputKeyClass val valueClass = hadoopConf.getOutputValueClass @@ -1033,27 +1045,56 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.preSetup() val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { + val config = wrappedConf.value // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val (outputMetrics, bytesWrittenCallback) = initHadoopOutputMetrics(context, config) + writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() try { + var recordsWritten = 0L while (iter.hasNext) { val record = iter.next() writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) + + // Update bytes written metric every few records + maybeUpdateOutputMetrics(bytesWrittenCallback, outputMetrics, recordsWritten) + recordsWritten += 1 } } finally { writer.close() } writer.commit() + bytesWrittenCallback.foreach { fn => outputMetrics.bytesWritten = fn() } } self.context.runJob(self, writeToFile) writer.commitJob() } + private def initHadoopOutputMetrics(context: TaskContext, config: Configuration) + : (OutputMetrics, Option[() => Long]) = { + val bytesWrittenCallback = Option(config.get("mapreduce.output.fileoutputformat.outputdir")) + .map(new Path(_)) + .flatMap(SparkHadoopUtil.get.getFSBytesWrittenOnThreadCallback(_, config)) + val outputMetrics = new OutputMetrics(DataWriteMethod.Hadoop) + if (bytesWrittenCallback.isDefined) { + context.taskMetrics.outputMetrics = Some(outputMetrics) + } + (outputMetrics, bytesWrittenCallback) + } + + private def maybeUpdateOutputMetrics(bytesWrittenCallback: Option[() => Long], + outputMetrics: OutputMetrics, recordsWritten: Long): Unit = { + if (recordsWritten % PairRDDFunctions.RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES == 0 + && bytesWrittenCallback.isDefined) { + bytesWrittenCallback.foreach { fn => outputMetrics.bytesWritten = fn() } + } + } + /** * Return an RDD with the keys of each tuple. */ @@ -1070,3 +1111,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) } + +private[spark] object PairRDDFunctions { + val RECORDS_BETWEEN_BYTES_WRITTEN_METRIC_UPDATES = 256 +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 4e3d9de540783..3bb54855bae44 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -158,6 +158,11 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener " INPUT_BYTES=" + metrics.bytesRead case None => "" } + val outputMetrics = taskMetrics.outputMetrics match { + case Some(metrics) => + " OUTPUT_BYTES=" + metrics.bytesWritten + case None => "" + } val shuffleReadMetrics = taskMetrics.shuffleReadMetrics match { case Some(metrics) => " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + @@ -173,7 +178,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener " SHUFFLE_WRITE_TIME=" + metrics.shuffleWriteTime case None => "" } - stageLogInfo(stageId, status + info + executorRunTime + gcTime + inputMetrics + + stageLogInfo(stageId, status + info + executorRunTime + gcTime + inputMetrics + outputMetrics + shuffleReadMetrics + writeMetrics) } diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 51dc08f668a43..6f446c5a95a0a 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -29,6 +29,8 @@ private[spark] object ToolTips { val INPUT = "Bytes read from Hadoop or from Spark storage." + val OUTPUT = "Bytes written to Hadoop." + val SHUFFLE_WRITE = "Bytes written to disk in order to be read by a shuffle in a future stage." val SHUFFLE_READ = diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index ba97630f025c1..dd1c2b78c4094 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -48,6 +48,7 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp val executorToTasksFailed = HashMap[String, Int]() val executorToDuration = HashMap[String, Long]() val executorToInputBytes = HashMap[String, Long]() + val executorToOutputBytes = HashMap[String, Long]() val executorToShuffleRead = HashMap[String, Long]() val executorToShuffleWrite = HashMap[String, Long]() @@ -78,6 +79,10 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp executorToInputBytes(eid) = executorToInputBytes.getOrElse(eid, 0L) + inputMetrics.bytesRead } + metrics.outputMetrics.foreach { outputMetrics => + executorToOutputBytes(eid) = + executorToOutputBytes.getOrElse(eid, 0L) + outputMetrics.bytesWritten + } metrics.shuffleReadMetrics.foreach { shuffleRead => executorToShuffleRead(eid) = executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index f0e43fbf70976..fa0f96bff34ff 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -45,6 +45,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobPr Failed Tasks Succeeded Tasks Input + Output Shuffle Read Shuffle Write Shuffle Spill (Memory) @@ -77,6 +78,8 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobPr {v.succeededTasks} {Utils.bytesToString(v.inputBytes)} + + {Utils.bytesToString(v.outputBytes)} {Utils.bytesToString(v.shuffleRead)} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index e3223403c17f4..8bbde51e1801c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -259,6 +259,12 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.inputBytes += inputBytesDelta execSummary.inputBytes += inputBytesDelta + val outputBytesDelta = + (taskMetrics.outputMetrics.map(_.bytesWritten).getOrElse(0L) + - oldMetrics.flatMap(_.outputMetrics).map(_.bytesWritten).getOrElse(0L)) + stageData.outputBytes += outputBytesDelta + execSummary.outputBytes += outputBytesDelta + val diskSpillDelta = taskMetrics.diskBytesSpilled - oldMetrics.map(_.diskBytesSpilled).getOrElse(0L) stageData.diskBytesSpilled += diskSpillDelta diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 250bddbe2f262..16bc3f6c18d09 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -57,6 +57,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables val hasAccumulators = accumulables.size > 0 val hasInput = stageData.inputBytes > 0 + val hasOutput = stageData.outputBytes > 0 val hasShuffleRead = stageData.shuffleReadBytes > 0 val hasShuffleWrite = stageData.shuffleWriteBytes > 0 val hasBytesSpilled = stageData.memoryBytesSpilled > 0 && stageData.diskBytesSpilled > 0 @@ -74,6 +75,12 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { {Utils.bytesToString(stageData.inputBytes)} }} + {if (hasOutput) { +
  • + Output: + {Utils.bytesToString(stageData.outputBytes)} +
  • + }} {if (hasShuffleRead) {
  • Shuffle read: @@ -162,6 +169,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ {if (hasInput) Seq(("Input", "")) else Nil} ++ + {if (hasOutput) Seq(("Output", "")) else Nil} ++ {if (hasShuffleRead) Seq(("Shuffle Read", "")) else Nil} ++ {if (hasShuffleWrite) Seq(("Write Time", ""), ("Shuffle Write", "")) else Nil} ++ {if (hasBytesSpilled) Seq(("Shuffle Spill (Memory)", ""), ("Shuffle Spill (Disk)", "")) @@ -172,7 +180,8 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val taskTable = UIUtils.listingTable( unzipped._1, - taskRow(hasAccumulators, hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), + taskRow(hasAccumulators, hasInput, hasOutput, hasShuffleRead, hasShuffleWrite, + hasBytesSpilled), tasks, headerClasses = unzipped._2) // Excludes tasks which failed and have incomplete metrics @@ -260,6 +269,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { } val inputQuantiles = Input +: getFormattedSizeQuantiles(inputSizes) + val outputSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.outputMetrics.map(_.bytesWritten).getOrElse(0L).toDouble + } + val outputQuantiles = Output +: getFormattedSizeQuantiles(outputSizes) + val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } @@ -296,6 +310,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { , {gettingResultQuantiles}, if (hasInput) {inputQuantiles} else Nil, + if (hasOutput) {outputQuantiles} else Nil, if (hasShuffleRead) {shuffleReadQuantiles} else Nil, if (hasShuffleWrite) {shuffleWriteQuantiles} else Nil, if (hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, @@ -328,6 +343,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { def taskRow( hasAccumulators: Boolean, hasInput: Boolean, + hasOutput: Boolean, hasShuffleRead: Boolean, hasShuffleWrite: Boolean, hasBytesSpilled: Boolean)(taskData: TaskUIData): Seq[Node] = { @@ -351,6 +367,12 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { .map(m => s"${Utils.bytesToString(m.bytesRead)} (${m.readMethod.toString.toLowerCase()})") .getOrElse("") + val maybeOutput = metrics.flatMap(_.outputMetrics) + val outputSortable = maybeOutput.map(_.bytesWritten.toString).getOrElse("") + val outputReadable = maybeOutput + .map(m => s"${Utils.bytesToString(m.bytesWritten)}") + .getOrElse("") + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") @@ -417,6 +439,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { {inputReadable} }} + {if (hasOutput) { + + {outputReadable} + + }} {if (hasShuffleRead) { {shuffleReadReadable} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 3b4866e05956d..eae542df85d08 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -45,6 +45,7 @@ private[ui] class StageTableBase( Duration Tasks: Succeeded/Total Input + Output Shuffle Read + commons-io + commons-io + + + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + + + org.apache.hbase + hbase-hadoop-compat + ${hbase.version} + test-jar + test + com.twitter algebird-core_${scala.binary.version} From 974d334cf06a84317234a6c8e2e9ecca8271fa41 Mon Sep 17 00:00:00 2001 From: Varadharajan Mukundan Date: Mon, 10 Nov 2014 14:32:29 -0800 Subject: [PATCH 1276/1492] [SPARK-4047] - Generate runtime warnings for example implementation of PageRank Based on SPARK-2434, this PR generates runtime warnings for example implementations (Python, Scala) of PageRank. Author: Varadharajan Mukundan Closes #2894 from varadharajan/SPARK-4047 and squashes the following commits: 5f9406b [Varadharajan Mukundan] [SPARK-4047] - Point users to LogisticRegressionWithSGD and LogisticRegressionWithLBFGS instead of LogisticRegressionModel 252f595 [Varadharajan Mukundan] a. Generate runtime warnings for 05a018b [Varadharajan Mukundan] Fix PageRank implementation's package reference 5c2bf54 [Varadharajan Mukundan] [SPARK-4047] - Generate runtime warnings for example implementation of PageRank --- .../org/apache/spark/examples/JavaHdfsLR.java | 15 +++++++++++++++ .../org/apache/spark/examples/JavaPageRank.java | 13 +++++++++++++ examples/src/main/python/pagerank.py | 8 ++++++++ .../org/apache/spark/examples/LocalFileLR.scala | 6 ++++-- .../org/apache/spark/examples/LocalLR.scala | 6 ++++-- .../org/apache/spark/examples/SparkHdfsLR.scala | 6 ++++-- .../org/apache/spark/examples/SparkLR.scala | 6 ++++-- .../apache/spark/examples/SparkPageRank.scala | 15 +++++++++++++++ .../spark/examples/SparkTachyonHdfsLR.scala | 16 ++++++++++++++++ 9 files changed, 83 insertions(+), 8 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 6c177de359b60..31a79ddd3fff1 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -30,12 +30,25 @@ /** * Logistic regression based classification. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. */ public final class JavaHdfsLR { private static final int D = 10; // Number of dimensions private static final Random rand = new Random(42); + static void showWarning() { + String warning = "WARN: This is a naive implementation of Logistic Regression " + + "and is given as an example!\n" + + "Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD " + + "or org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS " + + "for more conventional use."; + System.err.println(warning); + } + static class DataPoint implements Serializable { DataPoint(double[] x, double y) { this.x = x; @@ -109,6 +122,8 @@ public static void main(String[] args) { System.exit(1); } + showWarning(); + SparkConf sparkConf = new SparkConf().setAppName("JavaHdfsLR"); JavaSparkContext sc = new JavaSparkContext(sparkConf); JavaRDD lines = sc.textFile(args[0]); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index c22506491fbff..a5db8accdf138 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -45,10 +45,21 @@ * URL neighbor URL * ... * where URL and their neighbors are separated by space(s). + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.graphx.lib.PageRank */ public final class JavaPageRank { private static final Pattern SPACES = Pattern.compile("\\s+"); + static void showWarning() { + String warning = "WARN: This is a naive implementation of PageRank " + + "and is given as an example! \n" + + "Please use the PageRank implementation found in " + + "org.apache.spark.graphx.lib.PageRank for more conventional use."; + System.err.println(warning); + } + private static class Sum implements Function2 { @Override public Double call(Double a, Double b) { @@ -62,6 +73,8 @@ public static void main(String[] args) throws Exception { System.exit(1); } + showWarning(); + SparkConf sparkConf = new SparkConf().setAppName("JavaPageRank"); JavaSparkContext ctx = new JavaSparkContext(sparkConf); diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index b539c4128cdcc..a5f25d78c1146 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -15,6 +15,11 @@ # limitations under the License. # +""" +This is an example implementation of PageRank. For more conventional use, +Please refer to PageRank implementation provided by graphx +""" + import re import sys from operator import add @@ -40,6 +45,9 @@ def parseNeighbors(urls): print >> sys.stderr, "Usage: pagerank " exit(-1) + print >> sys.stderr, """WARN: This is a naive implementation of PageRank and is + given as an example! Please refer to PageRank implementation provided by graphx""" + # Initialize the spark context. sc = SparkContext(appName="PythonPageRank") diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala index 931faac5463c4..ac2ea35bbd0e0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalFileLR.scala @@ -25,7 +25,8 @@ import breeze.linalg.{Vector, DenseVector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.classification.LogisticRegression + * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. */ object LocalFileLR { val D = 10 // Numer of dimensions @@ -41,7 +42,8 @@ object LocalFileLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala index 2d75b9d2590f8..92a683ad57ea1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalLR.scala @@ -25,7 +25,8 @@ import breeze.linalg.{Vector, DenseVector} * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.classification.LogisticRegression + * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. */ object LocalLR { val N = 10000 // Number of data points @@ -48,7 +49,8 @@ object LocalLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 3258510894372..9099c2fcc90b3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -32,7 +32,8 @@ import org.apache.spark.scheduler.InputFormatInfo * Logistic regression based classification. * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.classification.LogisticRegression + * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. */ object SparkHdfsLR { val D = 10 // Numer of dimensions @@ -54,7 +55,8 @@ object SparkHdfsLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index fc23308fc4adf..257a7d29f922a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -30,7 +30,8 @@ import org.apache.spark._ * Usage: SparkLR [slices] * * This is an example implementation for learning how to use Spark. For more conventional use, - * please refer to org.apache.spark.mllib.classification.LogisticRegression + * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. */ object SparkLR { val N = 10000 // Number of data points @@ -53,7 +54,8 @@ object SparkLR { def showWarning() { System.err.println( """WARN: This is a naive implementation of Logistic Regression and is given as an example! - |Please use the LogisticRegression method found in org.apache.spark.mllib.classification + |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS |for more conventional use. """.stripMargin) } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 4c7e006da0618..8d092b6506d33 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -28,13 +28,28 @@ import org.apache.spark.{SparkConf, SparkContext} * URL neighbor URL * ... * where URL and their neighbors are separated by space(s). + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to org.apache.spark.graphx.lib.PageRank */ object SparkPageRank { + + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of PageRank and is given as an example! + |Please use the PageRank implementation found in org.apache.spark.graphx.lib.PageRank + |for more conventional use. + """.stripMargin) + } + def main(args: Array[String]) { if (args.length < 1) { System.err.println("Usage: SparkPageRank ") System.exit(1) } + + showWarning() + val sparkConf = new SparkConf().setAppName("PageRank") val iters = if (args.length > 0) args(1).toInt else 10 val ctx = new SparkContext(sparkConf) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 96d13612e46dd..4393b99e636b6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -32,11 +32,24 @@ import org.apache.spark.storage.StorageLevel /** * Logistic regression based classification. * This example uses Tachyon to persist rdds during computation. + * + * This is an example implementation for learning how to use Spark. For more conventional use, + * please refer to either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + * org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS based on your needs. */ object SparkTachyonHdfsLR { val D = 10 // Numer of dimensions val rand = new Random(42) + def showWarning() { + System.err.println( + """WARN: This is a naive implementation of Logistic Regression and is given as an example! + |Please use either org.apache.spark.mllib.classification.LogisticRegressionWithSGD or + |org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS + |for more conventional use. + """.stripMargin) + } + case class DataPoint(x: Vector[Double], y: Double) def parsePoint(line: String): DataPoint = { @@ -51,6 +64,9 @@ object SparkTachyonHdfsLR { } def main(args: Array[String]) { + + showWarning() + val inputPath = args(0) val sparkConf = new SparkConf().setAppName("SparkTachyonHdfsLR") val conf = new Configuration() From 6e7a309b814291d5936c2b5a7b22151b30ea2614 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 10 Nov 2014 14:56:06 -0800 Subject: [PATCH 1277/1492] Revert "[SPARK-2703][Core]Make Tachyon related unit tests execute without deploying a Tachyon system locally." This reverts commit bd86cb1738800a0aa4c88b9afdba2f97ac6cbf25. --- core/pom.xml | 7 ------- .../org/apache/spark/storage/BlockManagerSuite.scala | 11 ++--------- project/SparkBuild.scala | 2 -- 3 files changed, 2 insertions(+), 18 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 92e9f1fc46275..41296e0eca330 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -204,13 +204,6 @@ derby test - - org.tachyonproject - tachyon - 0.5.0 - test-jar - test - org.tachyonproject tachyon-client diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 86503c9a02058..9529502bc8e10 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -36,7 +36,6 @@ import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ -import tachyon.master.LocalTachyonCluster import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.executor.DataReadMethod @@ -537,14 +536,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter } test("tachyon storage") { - val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", true) + // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. + val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { - val tachyonCluster = new LocalTachyonCluster(30000000) - tachyonCluster.start() - val tachyonURL = tachyon.Constants.HEADER + - tachyonCluster.getMasterHostname() + ":" + tachyonCluster.getMasterPort() - conf.set("spark.tachyonStore.url", tachyonURL) - conf.set("spark.tachyonStore.folderName", "app-test") store = makeBlockManager(1200) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -555,7 +549,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(store.getSingle("a3").isDefined, "a3 was in store") assert(store.getSingle("a2").isDefined, "a2 was in store") assert(store.getSingle("a1").isDefined, "a1 was in store") - tachyonCluster.stop() } else { info("tachyon storage test disabled.") } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 351e57a4b578b..657e4b4432775 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -360,8 +360,6 @@ object TestSettings { testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), // Enable Junit testing. libraryDependencies += "com.novocode" % "junit-interface" % "0.9" % "test", - // Enable Tachyon local testing. - libraryDependencies += "org.tachyonproject" % "tachyon" % "0.5.0" % "test" classifier "tests", // Only allow one test at a time, even across projects, since they run in the same JVM parallelExecution in Test := false, concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), From dbf10588de03e8ea993fff687a78727eff55db1f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 10 Nov 2014 15:55:15 -0800 Subject: [PATCH 1278/1492] [SPARK-4319][SQL] Enable an ignored test "null count". Author: Takuya UESHIN Closes #3185 from ueshin/issues/SPARK-4319 and squashes the following commits: a44a38e [Takuya UESHIN] Enable an ignored test "null count". --- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 ++++----- .../src/test/scala/org/apache/spark/sql/TestData.scala | 9 +++++---- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 702714af5308d..8a80724c08c7c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -281,14 +281,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { 3) } - // No support for primitive nulls yet. - ignore("null count") { + test("null count") { checkAnswer( - sql("SELECT a, COUNT(b) FROM testData3"), - Seq((1,0), (2, 1))) + sql("SELECT a, COUNT(b) FROM testData3 GROUP BY a"), + Seq((1, 0), (2, 1))) checkAnswer( - testData3.groupBy()(Count('a), Count('b), Count(1), CountDistinct('a :: Nil), CountDistinct('b :: Nil)), + sql("SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3"), (2, 1, 2, 2, 1) :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index ef87a230639bc..92b49e8155900 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -64,11 +64,12 @@ object TestData { BinaryData("123".getBytes(), 4) :: Nil).toSchemaRDD binaryData.registerTempTable("binaryData") - // TODO: There is no way to express null primitives as case classes currently... + case class TestData3(a: Int, b: Option[Int]) val testData3 = - logical.LocalRelation('a.int, 'b.int).loadData( - (1, null) :: - (2, 2) :: Nil) + TestSQLContext.sparkContext.parallelize( + TestData3(1, None) :: + TestData3(2, Some(2)) :: Nil).toSchemaRDD + testData3.registerTempTable("testData3") val emptyTableData = logical.LocalRelation('a.int, 'b.int) From 534b23141715b69a89531d93d4b9b78cf2789ff4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 10 Nov 2014 16:17:52 -0800 Subject: [PATCH 1279/1492] [SPARK-4000][Build] Uploads HiveCompatibilitySuite logs This is a follow up of #2845. In addition to unit-tests.log files, also upload failure output files generated by `HiveCompatibilitySuite` to Jenkins master. These files can be very helpful to debug Hive compatibility test failures. /cc pwendell marmbrus Author: Cheng Lian Closes #2993 from liancheng/upload-hive-compat-logs and squashes the following commits: 8e6247f [Cheng Lian] Uploads HiveCompatibilitySuite logs --- dev/run-tests-jenkins | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 87c6715153da7..6a849e4f77207 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -101,7 +101,13 @@ function post_message () { function send_archived_logs () { echo "Archiving unit tests logs..." - local log_files=$(find . -name "unit-tests.log") + local log_files=$( + find .\ + -name "unit-tests.log" -o\ + -path "./sql/hive/target/HiveCompatibilitySuite.failed" -o\ + -path "./sql/hive/target/HiveCompatibilitySuite.hiveFailed" -o\ + -path "./sql/hive/target/HiveCompatibilitySuite.wrong" + ) if [ -z "$log_files" ]; then echo "> No log files found." >&2 From acb55aeddbe58758d75b9aed130634afe21797cf Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 10 Nov 2014 16:56:36 -0800 Subject: [PATCH 1280/1492] [SPARK-4308][SQL] Sets SQL operation state to ERROR when exception is thrown In `HiveThriftServer2`, when an exception is thrown during a SQL execution, the SQL operation state should be set to `ERROR`, but now it remains `RUNNING`. This affects the result of the `GetOperationStatus` Thrift API. Author: Cheng Lian Closes #3175 from liancheng/fix-op-state and squashes the following commits: 6d4c1fe [Cheng Lian] Sets SQL operation state to ERROR when exception is thrown --- .../thriftserver/AbstractSparkSQLDriver.scala | 2 -- .../spark/sql/hive/thriftserver/Shim12.scala | 12 +++---- .../spark/sql/hive/thriftserver/Shim13.scala | 36 ++++++++----------- 3 files changed, 21 insertions(+), 29 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala index fcb302edbffa8..6ed8fd2768f95 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive.thriftserver import scala.collection.JavaConversions._ -import java.util.{ArrayList => JArrayList} - import org.apache.commons.lang.exception.ExceptionUtils import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema} import org.apache.hadoop.hive.ql.Driver diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index e3ba9914c6cc0..aa2e3cab72bb9 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -25,9 +25,7 @@ import scala.collection.mutable.{ArrayBuffer, Map => SMap} import scala.math._ import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli._ @@ -37,9 +35,9 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans.logical.SetCommand import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} -import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveContext} import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.{SQLConf, SchemaRDD, Row => SparkRow} /** * A compatibility layer for interacting with Hive version 0.12.0. @@ -71,8 +69,9 @@ private[hive] class SparkExecuteStatementOperation( statement: String, confOverlay: JMap[String, String])( hiveContext: HiveContext, - sessionToActivePool: SMap[HiveSession, String]) extends ExecuteStatementOperation( - parentSession, statement, confOverlay) with Logging { + sessionToActivePool: SMap[HiveSession, String]) + extends ExecuteStatementOperation(parentSession, statement, confOverlay) with Logging { + private var result: SchemaRDD = _ private var iter: Iterator[SparkRow] = _ private var dataTypes: Array[DataType] = _ @@ -216,6 +215,7 @@ private[hive] class SparkExecuteStatementOperation( // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => + setState(OperationState.ERROR) logError("Error executing query:",e) throw new HiveSQLException(e.toString) } diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index f2ceba828296b..a642478d08857 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -27,10 +27,9 @@ import scala.collection.mutable.{ArrayBuffer, Map => SMap} import scala.math._ import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli._ @@ -39,9 +38,9 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.{Row => SparkRow, SchemaRDD} -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} /** * A compatibility layer for interacting with Hive version 0.12.0. @@ -100,6 +99,7 @@ private[hive] class SparkExecuteStatementOperation( // Actually do need to catch Throwable as some failures don't inherit from Exception and // HiveServer will silently swallow them. case e: Throwable => + setState(OperationState.ERROR) logError("Error executing query:",e) throw new HiveSQLException(e.toString) } @@ -194,14 +194,12 @@ private[hive] class SparkExecuteStatementOperation( try { sqlOperationConf.verifyAndSet(confEntry.getKey, confEntry.getValue) } - catch { - case e: IllegalArgumentException => { - throw new HiveSQLException("Error applying statement specific settings", e) - } + catch { case e: IllegalArgumentException => + throw new HiveSQLException("Error applying statement specific settings", e) } } } - return sqlOperationConf + sqlOperationConf } def run(): Unit = { @@ -219,7 +217,7 @@ private[hive] class SparkExecuteStatementOperation( val currentUGI: UserGroupInformation = ShimLoader.getHadoopShims.getUGIForConf(opConfig) val backgroundOperation: Runnable = new Runnable { - def run { + def run() { val doAsAction: PrivilegedExceptionAction[AnyRef] = new PrivilegedExceptionAction[AnyRef] { def run: AnyRef = { @@ -228,23 +226,19 @@ private[hive] class SparkExecuteStatementOperation( try { runInternal(statement) } - catch { - case e: HiveSQLException => { - setOperationException(e) - logError("Error running hive query: ", e) - } + catch { case e: HiveSQLException => + setOperationException(e) + logError("Error running hive query: ", e) } - return null + null } } try { ShimLoader.getHadoopShims.doAs(currentUGI, doAsAction) } - catch { - case e: Exception => { - setOperationException(new HiveSQLException(e)) - logError("Error running hive query as user : " + currentUGI.getShortUserName, e) - } + catch { case e: Exception => + setOperationException(new HiveSQLException(e)) + logError("Error running hive query as user : " + currentUGI.getShortUserName, e) } setState(OperationState.FINISHED) } From d793d80c8084923ea04dcf7d268eec8ede490127 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 10 Nov 2014 17:20:52 -0800 Subject: [PATCH 1281/1492] [SQL] remove a decimal case branch that has no effect at runtime it generates warnings at compile time marmbrus Author: Xiangrui Meng Closes #3192 from mengxr/dtc-decimal and squashes the following commits: 955e9fb [Xiangrui Meng] remove a decimal case branch that has no effect --- .../org/apache/spark/sql/types/util/DataTypeConversions.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 3fa4a7c6481d3..9aad7b3df4eed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -133,7 +133,6 @@ protected[sql] object DataTypeConversions { def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type case (d: java.math.BigDecimal, _) => Decimal(BigDecimal(d)) - case (d: java.math.BigDecimal, _) => BigDecimal(d) case (other, _) => other } From fa777833b52b6f339cdc335e8e3935cfe9a2a7eb Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 10 Nov 2014 17:22:57 -0800 Subject: [PATCH 1282/1492] [SPARK-4250] [SQL] Fix bug of constant null value mapping to ConstantObjectInspector Author: Cheng Hao Closes #3114 from chenghao-intel/constant_null_oi and squashes the following commits: e603bda [Cheng Hao] fix the bug of null value for primitive types 50a13ba [Cheng Hao] fix the timezone issue f54f369 [Cheng Hao] fix bug of constant null value for ObjectInspector --- .../spark/sql/hive/HiveInspectors.scala | 78 ++++++++++-------- ...testing-0-9a02bc7de09bcabcbd4c91f54a814c20 | 1 + .../udf_if-0-b7ffa85b5785cccef2af1b285348cc2c | 1 + .../udf_if-1-30cf7f51f92b5684e556deff3032d49a | 1 + .../udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 | 0 .../udf_if-3-20206f17367ff284d67044abd745ce9f | 1 + .../udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca | 0 .../udf_if-5-a7db13aec05c97792f9331d63709d8cc | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 52 +++++++++++- .../org/apache/spark/sql/hive/Shim12.scala | 70 ++++++++++------ .../org/apache/spark/sql/hive/Shim13.scala | 80 +++++++++++++------ 11 files changed, 199 insertions(+), 86 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/constant null testing-0-9a02bc7de09bcabcbd4c91f54a814c20 create mode 100644 sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c create mode 100644 sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a create mode 100644 sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 create mode 100644 sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f create mode 100644 sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca create mode 100644 sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index bdc7e1dac1922..7e76aff642bb5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -88,6 +88,7 @@ private[hive] trait HiveInspectors { * @return convert the data into catalyst type */ def unwrap(data: Any, oi: ObjectInspector): Any = oi match { + case _ if data == null => null case hvoi: HiveVarcharObjectInspector => if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue case hdoi: HiveDecimalObjectInspector => @@ -250,46 +251,53 @@ private[hive] trait HiveInspectors { } def toInspector(expr: Expression): ObjectInspector = expr match { - case Literal(value: String, StringType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Int, IntegerType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Double, DoubleType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Boolean, BooleanType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Long, LongType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Float, FloatType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Short, ShortType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Byte, ByteType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Array[Byte], BinaryType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: java.sql.Date, DateType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: java.sql.Timestamp, TimestampType) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: BigDecimal, DecimalType()) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: Decimal, DecimalType()) => - HiveShim.getPrimitiveWritableConstantObjectInspector(value.toBigDecimal) + case Literal(value, StringType) => + HiveShim.getStringWritableConstantObjectInspector(value) + case Literal(value, IntegerType) => + HiveShim.getIntWritableConstantObjectInspector(value) + case Literal(value, DoubleType) => + HiveShim.getDoubleWritableConstantObjectInspector(value) + case Literal(value, BooleanType) => + HiveShim.getBooleanWritableConstantObjectInspector(value) + case Literal(value, LongType) => + HiveShim.getLongWritableConstantObjectInspector(value) + case Literal(value, FloatType) => + HiveShim.getFloatWritableConstantObjectInspector(value) + case Literal(value, ShortType) => + HiveShim.getShortWritableConstantObjectInspector(value) + case Literal(value, ByteType) => + HiveShim.getByteWritableConstantObjectInspector(value) + case Literal(value, BinaryType) => + HiveShim.getBinaryWritableConstantObjectInspector(value) + case Literal(value, DateType) => + HiveShim.getDateWritableConstantObjectInspector(value) + case Literal(value, TimestampType) => + HiveShim.getTimestampWritableConstantObjectInspector(value) + case Literal(value, DecimalType()) => + HiveShim.getDecimalWritableConstantObjectInspector(value) case Literal(_, NullType) => HiveShim.getPrimitiveNullWritableConstantObjectInspector - case Literal(value: Seq[_], ArrayType(dt, _)) => + case Literal(value, ArrayType(dt, _)) => val listObjectInspector = toInspector(dt) - val list = new java.util.ArrayList[Object]() - value.foreach(v => list.add(wrap(v, listObjectInspector))) - ObjectInspectorFactory.getStandardConstantListObjectInspector(listObjectInspector, list) - case Literal(map: Map[_, _], MapType(keyType, valueType, _)) => - val value = new java.util.HashMap[Object, Object]() + if (value == null) { + ObjectInspectorFactory.getStandardConstantListObjectInspector(listObjectInspector, null) + } else { + val list = new java.util.ArrayList[Object]() + value.asInstanceOf[Seq[_]].foreach(v => list.add(wrap(v, listObjectInspector))) + ObjectInspectorFactory.getStandardConstantListObjectInspector(listObjectInspector, list) + } + case Literal(value, MapType(keyType, valueType, _)) => val keyOI = toInspector(keyType) val valueOI = toInspector(valueType) - map.foreach (entry => value.put(wrap(entry._1, keyOI), wrap(entry._2, valueOI))) - ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, value) - case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") + if (value == null) { + ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, null) + } else { + val map = new java.util.HashMap[Object, Object]() + value.asInstanceOf[Map[_, _]].foreach (entry => { + map.put(wrap(entry._1, keyOI), wrap(entry._2, valueOI)) + }) + ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, map) + } case _ => toInspector(expr.dataType) } diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-9a02bc7de09bcabcbd4c91f54a814c20 b/sql/hive/src/test/resources/golden/constant null testing-0-9a02bc7de09bcabcbd4c91f54a814c20 new file mode 100644 index 0000000000000..7c41615f8c184 --- /dev/null +++ b/sql/hive/src/test/resources/golden/constant null testing-0-9a02bc7de09bcabcbd4c91f54a814c20 @@ -0,0 +1 @@ +1 NULL 1 NULL 1.0 NULL true NULL 1 NULL 1.0 NULL 1 NULL 1 NULL 1 NULL 1970-01-01 NULL 1969-12-31 16:00:00.001 NULL 1 NULL diff --git a/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c new file mode 100644 index 0000000000000..2cf0d9d61882e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c @@ -0,0 +1 @@ +There is no documentation for function 'if' diff --git a/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a new file mode 100644 index 0000000000000..2cf0d9d61882e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a @@ -0,0 +1 @@ +There is no documentation for function 'if' diff --git a/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 b/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f b/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f new file mode 100644 index 0000000000000..a29e96cbd1db7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f @@ -0,0 +1 @@ +1 1 1 1 NULL 2 diff --git a/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca b/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc b/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc new file mode 100644 index 0000000000000..f0669b86989d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc @@ -0,0 +1 @@ +128 1.1 ABC 12.3 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b897dff0159ff..684d22807c0c6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -18,6 +18,9 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.util.{Locale, TimeZone} + +import org.scalatest.BeforeAndAfter import scala.util.Try @@ -28,14 +31,59 @@ import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.{Row, SchemaRDD} +import org.apache.spark.sql.{SQLConf, Row, SchemaRDD} case class TestData(a: Int, b: String) /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ -class HiveQuerySuite extends HiveComparisonTest { +class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + + override def beforeAll() { + TestHive.cacheTables = true + // Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) + TimeZone.setDefault(TimeZone.getTimeZone("America/Los_Angeles")) + // Add Locale setting + Locale.setDefault(Locale.US) + } + + override def afterAll() { + TestHive.cacheTables = false + TimeZone.setDefault(originalTimeZone) + Locale.setDefault(originalLocale) + } + + createQueryTest("constant null testing", + """SELECT + |IF(FALSE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL1, + |IF(TRUE, CAST(NULL AS STRING), CAST(1 AS STRING)) AS COL2, + |IF(FALSE, CAST(NULL AS INT), CAST(1 AS INT)) AS COL3, + |IF(TRUE, CAST(NULL AS INT), CAST(1 AS INT)) AS COL4, + |IF(FALSE, CAST(NULL AS DOUBLE), CAST(1 AS DOUBLE)) AS COL5, + |IF(TRUE, CAST(NULL AS DOUBLE), CAST(1 AS DOUBLE)) AS COL6, + |IF(FALSE, CAST(NULL AS BOOLEAN), CAST(1 AS BOOLEAN)) AS COL7, + |IF(TRUE, CAST(NULL AS BOOLEAN), CAST(1 AS BOOLEAN)) AS COL8, + |IF(FALSE, CAST(NULL AS BIGINT), CAST(1 AS BIGINT)) AS COL9, + |IF(TRUE, CAST(NULL AS BIGINT), CAST(1 AS BIGINT)) AS COL10, + |IF(FALSE, CAST(NULL AS FLOAT), CAST(1 AS FLOAT)) AS COL11, + |IF(TRUE, CAST(NULL AS FLOAT), CAST(1 AS FLOAT)) AS COL12, + |IF(FALSE, CAST(NULL AS SMALLINT), CAST(1 AS SMALLINT)) AS COL13, + |IF(TRUE, CAST(NULL AS SMALLINT), CAST(1 AS SMALLINT)) AS COL14, + |IF(FALSE, CAST(NULL AS TINYINT), CAST(1 AS TINYINT)) AS COL15, + |IF(TRUE, CAST(NULL AS TINYINT), CAST(1 AS TINYINT)) AS COL16, + |IF(FALSE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL17, + |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, + |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, + |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, + |IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL22, + |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23, + |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL24 + |FROM src LIMIT 1""".stripMargin) + createQueryTest("constant array", """ |SELECT sort_array( diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 8e946b7e82f5d..8ba25f889d176 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -57,54 +57,74 @@ private[hive] object HiveShim { new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) } - def getPrimitiveWritableConstantObjectInspector(value: String): ObjectInspector = + def getStringWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.STRING, new hadoopIo.Text(value)) + PrimitiveCategory.STRING, + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String])) - def getPrimitiveWritableConstantObjectInspector(value: Int): ObjectInspector = + def getIntWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.INT, new hadoopIo.IntWritable(value)) + PrimitiveCategory.INT, + if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])) - def getPrimitiveWritableConstantObjectInspector(value: Double): ObjectInspector = + def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.DOUBLE, new hiveIo.DoubleWritable(value)) + PrimitiveCategory.DOUBLE, + if (value == null) null else new hiveIo.DoubleWritable(value.asInstanceOf[Double])) - def getPrimitiveWritableConstantObjectInspector(value: Boolean): ObjectInspector = + def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.BOOLEAN, new hadoopIo.BooleanWritable(value)) + PrimitiveCategory.BOOLEAN, + if (value == null) null else new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean])) - def getPrimitiveWritableConstantObjectInspector(value: Long): ObjectInspector = + def getLongWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.LONG, new hadoopIo.LongWritable(value)) + PrimitiveCategory.LONG, + if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])) - def getPrimitiveWritableConstantObjectInspector(value: Float): ObjectInspector = + def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.FLOAT, new hadoopIo.FloatWritable(value)) + PrimitiveCategory.FLOAT, + if (value == null) null else new hadoopIo.FloatWritable(value.asInstanceOf[Float])) - def getPrimitiveWritableConstantObjectInspector(value: Short): ObjectInspector = + def getShortWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.SHORT, new hiveIo.ShortWritable(value)) + PrimitiveCategory.SHORT, + if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])) - def getPrimitiveWritableConstantObjectInspector(value: Byte): ObjectInspector = + def getByteWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.BYTE, new hiveIo.ByteWritable(value)) + PrimitiveCategory.BYTE, + if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])) - def getPrimitiveWritableConstantObjectInspector(value: Array[Byte]): ObjectInspector = + def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.BINARY, new hadoopIo.BytesWritable(value)) + PrimitiveCategory.BINARY, + if (value == null) null else new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]])) - def getPrimitiveWritableConstantObjectInspector(value: java.sql.Date): ObjectInspector = + def getDateWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.DATE, new hiveIo.DateWritable(value)) + PrimitiveCategory.DATE, + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date])) - def getPrimitiveWritableConstantObjectInspector(value: java.sql.Timestamp): ObjectInspector = + def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - PrimitiveCategory.TIMESTAMP, new hiveIo.TimestampWritable(value)) - - def getPrimitiveWritableConstantObjectInspector(value: BigDecimal): ObjectInspector = + PrimitiveCategory.TIMESTAMP, + if (value == null) { + null + } else { + new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp]) + }) + + def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( PrimitiveCategory.DECIMAL, - new hiveIo.HiveDecimalWritable(HiveShim.createDecimal(value.underlying()))) + if (value == null) { + null + } else { + new hiveIo.HiveDecimalWritable( + HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying())) + }) def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 0bc330cdbecb1..e4aee57f0ad9f 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -56,54 +56,86 @@ private[hive] object HiveShim { new TableDesc(inputFormatClass, outputFormatClass, properties) } - def getPrimitiveWritableConstantObjectInspector(value: String): ObjectInspector = + def getStringWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.stringTypeInfo, new hadoopIo.Text(value)) + TypeInfoFactory.stringTypeInfo, + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String])) - def getPrimitiveWritableConstantObjectInspector(value: Int): ObjectInspector = + def getIntWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.intTypeInfo, new hadoopIo.IntWritable(value)) + TypeInfoFactory.intTypeInfo, + if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int])) - def getPrimitiveWritableConstantObjectInspector(value: Double): ObjectInspector = + def getDoubleWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.doubleTypeInfo, new hiveIo.DoubleWritable(value)) + TypeInfoFactory.doubleTypeInfo, if (value == null) { + null + } else { + new hiveIo.DoubleWritable(value.asInstanceOf[Double]) + }) - def getPrimitiveWritableConstantObjectInspector(value: Boolean): ObjectInspector = + def getBooleanWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.booleanTypeInfo, new hadoopIo.BooleanWritable(value)) + TypeInfoFactory.booleanTypeInfo, if (value == null) { + null + } else { + new hadoopIo.BooleanWritable(value.asInstanceOf[Boolean]) + }) - def getPrimitiveWritableConstantObjectInspector(value: Long): ObjectInspector = + def getLongWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.longTypeInfo, new hadoopIo.LongWritable(value)) + TypeInfoFactory.longTypeInfo, + if (value == null) null else new hadoopIo.LongWritable(value.asInstanceOf[Long])) - def getPrimitiveWritableConstantObjectInspector(value: Float): ObjectInspector = + def getFloatWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.floatTypeInfo, new hadoopIo.FloatWritable(value)) + TypeInfoFactory.floatTypeInfo, if (value == null) { + null + } else { + new hadoopIo.FloatWritable(value.asInstanceOf[Float]) + }) - def getPrimitiveWritableConstantObjectInspector(value: Short): ObjectInspector = + def getShortWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.shortTypeInfo, new hiveIo.ShortWritable(value)) + TypeInfoFactory.shortTypeInfo, + if (value == null) null else new hiveIo.ShortWritable(value.asInstanceOf[Short])) - def getPrimitiveWritableConstantObjectInspector(value: Byte): ObjectInspector = + def getByteWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.byteTypeInfo, new hiveIo.ByteWritable(value)) + TypeInfoFactory.byteTypeInfo, + if (value == null) null else new hiveIo.ByteWritable(value.asInstanceOf[Byte])) - def getPrimitiveWritableConstantObjectInspector(value: Array[Byte]): ObjectInspector = + def getBinaryWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.binaryTypeInfo, new hadoopIo.BytesWritable(value)) + TypeInfoFactory.binaryTypeInfo, if (value == null) { + null + } else { + new hadoopIo.BytesWritable(value.asInstanceOf[Array[Byte]]) + }) - def getPrimitiveWritableConstantObjectInspector(value: java.sql.Date): ObjectInspector = + def getDateWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.dateTypeInfo, new hiveIo.DateWritable(value)) + TypeInfoFactory.dateTypeInfo, + if (value == null) null else new hiveIo.DateWritable(value.asInstanceOf[java.sql.Date])) - def getPrimitiveWritableConstantObjectInspector(value: java.sql.Timestamp): ObjectInspector = + def getTimestampWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( - TypeInfoFactory.timestampTypeInfo, new hiveIo.TimestampWritable(value)) + TypeInfoFactory.timestampTypeInfo, if (value == null) { + null + } else { + new hiveIo.TimestampWritable(value.asInstanceOf[java.sql.Timestamp]) + }) - def getPrimitiveWritableConstantObjectInspector(value: BigDecimal): ObjectInspector = + def getDecimalWritableConstantObjectInspector(value: Any): ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( TypeInfoFactory.decimalTypeInfo, - new hiveIo.HiveDecimalWritable(HiveShim.createDecimal(value.underlying()))) + if (value == null) { + null + } else { + // TODO precise, scale? + new hiveIo.HiveDecimalWritable( + HiveShim.createDecimal(value.asInstanceOf[Decimal].toBigDecimal.underlying())) + }) def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( From a1fc059b69c9ed150bf8a284404cc149ddaa27d6 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 10 Nov 2014 17:26:03 -0800 Subject: [PATCH 1283/1492] [SPARK-4149][SQL] ISO 8601 support for json date time strings This implement the feature davies mentioned in https://github.com/apache/spark/pull/2901#discussion-diff-19313312 Author: Daoyuan Wang Closes #3012 from adrian-wang/iso8601 and squashes the following commits: 50df6e7 [Daoyuan Wang] json data timestamp ISO8601 support --- .../org/apache/spark/sql/json/JsonRDD.scala | 5 ++-- .../sql/types/util/DataTypeConversions.scala | 30 +++++++++++++++++++ .../org/apache/spark/sql/json/JsonSuite.scala | 7 +++++ 3 files changed, 40 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 0f2dcdcacf0ca..d9d7a3fea3963 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.json import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.types.util.DataTypeConversions import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} @@ -378,7 +379,7 @@ private[sql] object JsonRDD extends Logging { private def toDate(value: Any): Date = { value match { // only support string as date - case value: java.lang.String => Date.valueOf(value) + case value: java.lang.String => new Date(DataTypeConversions.stringToTime(value).getTime) } } @@ -386,7 +387,7 @@ private[sql] object JsonRDD extends Logging { value match { case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) case value: java.lang.Long => new Timestamp(value) - case value: java.lang.String => Timestamp.valueOf(value) + case value: java.lang.String => toTimestamp(DataTypeConversions.stringToTime(value).getTime) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 9aad7b3df4eed..d4258156f18f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.types.util +import java.text.SimpleDateFormat + import scala.collection.JavaConverters._ import org.apache.spark.sql._ @@ -129,6 +131,34 @@ protected[sql] object DataTypeConversions { StructType(structType.getFields.map(asScalaStructField)) } + def stringToTime(s: String): java.util.Date = { + if (!s.contains('T')) { + // JDBC escape string + if (s.contains(' ')) { + java.sql.Timestamp.valueOf(s) + } else { + java.sql.Date.valueOf(s) + } + } else if (s.endsWith("Z")) { + // this is zero timezone of ISO8601 + stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") + } else if (s.indexOf("GMT") == -1) { + // timezone with ISO8601 + val inset = "+00.00".length + val s0 = s.substring(0, s.length - inset) + val s1 = s.substring(s.length - inset, s.length) + if (s0.substring(s0.lastIndexOf(':')).contains('.')) { + stringToTime(s0 + "GMT" + s1) + } else { + stringToTime(s0 + ".0GMT" + s1) + } + } else { + // ISO8601 with GMT insert + val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) + ISO8601GMT.parse(s) + } + } + /** Converts Java objects to catalyst rows / types */ def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index cade244f7ac39..f8ca2c773d9ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -66,6 +66,13 @@ class JsonSuite extends QueryTest { val strDate = "2014-10-15" checkTypePromotion(Date.valueOf(strDate), enforceCorrectType(strDate, DateType)) + + val ISO8601Time1 = "1970-01-01T01:00:01.0Z" + checkTypePromotion(new Timestamp(3601000), enforceCorrectType(ISO8601Time1, TimestampType)) + checkTypePromotion(new Date(3601000), enforceCorrectType(ISO8601Time1, DateType)) + val ISO8601Time2 = "1970-01-01T02:00:01-01:00" + checkTypePromotion(new Timestamp(10801000), enforceCorrectType(ISO8601Time2, TimestampType)) + checkTypePromotion(new Date(10801000), enforceCorrectType(ISO8601Time2, DateType)) } test("Get compatible type") { From ce6ed2abd14de26b9ceaa415e9a42fbb1338f5fa Mon Sep 17 00:00:00 2001 From: surq Date: Mon, 10 Nov 2014 17:37:16 -0800 Subject: [PATCH 1284/1492] [SPARK-3954][Streaming] Optimization to FileInputDStream about convert files to RDDS there are 3 loops with files sequence in spark source. loops files sequence: 1.files.map(...) 2.files.zip(fileRDDs) 3.files-size.foreach It's will very time consuming when lots of files.So I do the following correction: 3 loops with files sequence => only one loop Author: surq Closes #2811 from surq/SPARK-3954 and squashes the following commits: 321bbe8 [surq] updated the code style.The style from [for...yield]to [files.map(file=>{})] 88a2c20 [surq] Merge branch 'master' of https://github.com/apache/spark into SPARK-3954 178066f [surq] modify code's style. [Exceeds 100 columns] 626ef97 [surq] remove redundant import(ArrayBuffer) 739341f [surq] promote the speed of convert files to RDDS --- .../apache/spark/streaming/dstream/FileInputDStream.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 8152b7542ac57..55d6cf6a783ea 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -120,14 +120,15 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas /** Generate one RDD from an array of files */ private def filesToRDD(files: Seq[String]): RDD[(K, V)] = { - val fileRDDs = files.map(file => context.sparkContext.newAPIHadoopFile[K, V, F](file)) - files.zip(fileRDDs).foreach { case (file, rdd) => { + val fileRDDs = files.map(file =>{ + val rdd = context.sparkContext.newAPIHadoopFile[K, V, F](file) if (rdd.partitions.size == 0) { logError("File " + file + " has no data in it. Spark Streaming can only ingest " + "files that have been \"moved\" to the directory assigned to the file stream. " + "Refer to the streaming programming guide for more details.") } - }} + rdd + }) new UnionRDD(context.sparkContext, fileRDDs) } From c764d0ac1c6410ca2dd2558cb6bcbe8ad5f02481 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 10 Nov 2014 17:46:05 -0800 Subject: [PATCH 1285/1492] [SPARK-4274] [SQL] Fix NPE in printing the details of the query plan Author: Cheng Hao Closes #3139 from chenghao-intel/comparison_test and squashes the following commits: f5d7146 [Cheng Hao] avoid exception in printing the codegen enabled --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 84eaf401f240c..31cc4170aa867 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -444,7 +444,7 @@ class SQLContext(@transient val sparkContext: SparkContext) |${stringOrError(optimizedPlan)} |== Physical Plan == |${stringOrError(executedPlan)} - |Code Generation: ${executedPlan.codegenEnabled} + |Code Generation: ${stringOrError(executedPlan.codegenEnabled)} |== RDD == """.stripMargin.trim } From 300887bd76c5018bfe396c5d47443be251368359 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 10 Nov 2014 19:31:52 -0800 Subject: [PATCH 1286/1492] [SPARK-3649] Remove GraphX custom serializers As [reported][1] on the mailing list, GraphX throws ``` java.lang.ClassCastException: java.lang.Long cannot be cast to scala.Tuple2 at org.apache.spark.graphx.impl.RoutingTableMessageSerializer$$anon$1$$anon$2.writeObject(Serializers.scala:39) at org.apache.spark.storage.DiskBlockObjectWriter.write(BlockObjectWriter.scala:195) at org.apache.spark.util.collection.ExternalSorter.spillToMergeableFile(ExternalSorter.scala:329) ``` when sort-based shuffle attempts to spill to disk. This is because GraphX defines custom serializers for shuffling pair RDDs that assume Spark will always serialize the entire pair object rather than breaking it up into its components. However, the spill code path in sort-based shuffle [violates this assumption][2]. GraphX uses the custom serializers to compress vertex ID keys using variable-length integer encoding. However, since the serializer can no longer rely on the key and value being serialized and deserialized together, performing such encoding would either require writing a tag byte (costly) or maintaining state in the serializer and assuming that serialization calls will alternate between key and value (fragile). Instead, this PR simply removes the custom serializers. This causes a **10% slowdown** (494 s to 543 s) and **16% increase in per-iteration communication** (2176 MB to 2518 MB) for PageRank (averages across 3 trials, 10 iterations per trial, uk-2007-05 graph, 16 r3.2xlarge nodes). [1]: http://apache-spark-user-list.1001560.n3.nabble.com/java-lang-ClassCastException-java-lang-Long-cannot-be-cast-to-scala-Tuple2-td13926.html#a14501 [2]: https://github.com/apache/spark/blob/f9d6220c792b779be385f3022d146911a22c2130/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala#L329 Author: Ankur Dave Closes #2503 from ankurdave/SPARK-3649 and squashes the following commits: a49c2ad [Ankur Dave] [SPARK-3649] Remove GraphX custom serializers --- .../org/apache/spark/graphx/VertexRDD.scala | 14 +- .../graphx/impl/MessageToPartition.scala | 50 --- .../graphx/impl/RoutingTablePartition.scala | 18 - .../spark/graphx/impl/Serializers.scala | 369 ------------------ .../apache/spark/graphx/SerializerSuite.scala | 122 ------ 5 files changed, 6 insertions(+), 567 deletions(-) delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala delete mode 100644 graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala delete mode 100644 graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 2c8b245955d12..12216d9d33d66 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -27,8 +27,6 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx.impl.RoutingTablePartition import org.apache.spark.graphx.impl.ShippableVertexPartition import org.apache.spark.graphx.impl.VertexAttributeBlock -import org.apache.spark.graphx.impl.RoutingTableMessageRDDFunctions._ -import org.apache.spark.graphx.impl.VertexRDDFunctions._ /** * Extends `RDD[(VertexId, VD)]` by ensuring that there is only one entry for each vertex and by @@ -233,7 +231,7 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => this.withPartitionsRDD[VD3]( partitionsRDD.zipPartitions( - other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + other.partitionBy(this.partitioner.get), preservesPartitioning = true) { (partIter, msgs) => partIter.map(_.leftJoin(msgs)(f)) } ) @@ -277,7 +275,7 @@ class VertexRDD[@specialized VD: ClassTag]( case _ => this.withPartitionsRDD( partitionsRDD.zipPartitions( - other.copartitionWithVertices(this.partitioner.get), preservesPartitioning = true) { + other.partitionBy(this.partitioner.get), preservesPartitioning = true) { (partIter, msgs) => partIter.map(_.innerJoin(msgs)(f)) } ) @@ -297,7 +295,7 @@ class VertexRDD[@specialized VD: ClassTag]( */ def aggregateUsingIndex[VD2: ClassTag]( messages: RDD[(VertexId, VD2)], reduceFunc: (VD2, VD2) => VD2): VertexRDD[VD2] = { - val shuffled = messages.copartitionWithVertices(this.partitioner.get) + val shuffled = messages.partitionBy(this.partitioner.get) val parts = partitionsRDD.zipPartitions(shuffled, true) { (thisIter, msgIter) => thisIter.map(_.aggregateUsingIndex(msgIter, reduceFunc)) } @@ -371,7 +369,7 @@ object VertexRDD { def apply[VD: ClassTag](vertices: RDD[(VertexId, VD)]): VertexRDD[VD] = { val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { case Some(p) => vertices - case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) + case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size)) } val vertexPartitions = vPartitioned.mapPartitions( iter => Iterator(ShippableVertexPartition(iter)), @@ -412,7 +410,7 @@ object VertexRDD { ): VertexRDD[VD] = { val vPartitioned: RDD[(VertexId, VD)] = vertices.partitioner match { case Some(p) => vertices - case None => vertices.copartitionWithVertices(new HashPartitioner(vertices.partitions.size)) + case None => vertices.partitionBy(new HashPartitioner(vertices.partitions.size)) } val routingTables = createRoutingTables(edges, vPartitioned.partitioner.get) val vertexPartitions = vPartitioned.zipPartitions(routingTables, preservesPartitioning = true) { @@ -454,7 +452,7 @@ object VertexRDD { .setName("VertexRDD.createRoutingTables - vid2pid (aggregation)") val numEdgePartitions = edges.partitions.size - vid2pid.copartitionWithVertices(vertexPartitioner).mapPartitions( + vid2pid.partitionBy(vertexPartitioner).mapPartitions( iter => Iterator(RoutingTablePartition.fromMsgs(numEdgePartitions, iter)), preservesPartitioning = true) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala deleted file mode 100644 index 714f3b81c9dad..0000000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.graphx.impl - -import scala.language.implicitConversions -import scala.reflect.{classTag, ClassTag} - -import org.apache.spark.Partitioner -import org.apache.spark.graphx.{PartitionID, VertexId} -import org.apache.spark.rdd.{ShuffledRDD, RDD} - - -private[graphx] -class VertexRDDFunctions[VD: ClassTag](self: RDD[(VertexId, VD)]) { - def copartitionWithVertices(partitioner: Partitioner): RDD[(VertexId, VD)] = { - val rdd = new ShuffledRDD[VertexId, VD, VD](self, partitioner) - - // Set a custom serializer if the data is of int or double type. - if (classTag[VD] == ClassTag.Int) { - rdd.setSerializer(new IntAggMsgSerializer) - } else if (classTag[VD] == ClassTag.Long) { - rdd.setSerializer(new LongAggMsgSerializer) - } else if (classTag[VD] == ClassTag.Double) { - rdd.setSerializer(new DoubleAggMsgSerializer) - } - rdd - } -} - -private[graphx] -object VertexRDDFunctions { - implicit def rdd2VertexRDDFunctions[VD: ClassTag](rdd: RDD[(VertexId, VD)]) = { - new VertexRDDFunctions(rdd) - } -} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala index b27485953f719..7a7fa91aadfe1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/RoutingTablePartition.scala @@ -29,24 +29,6 @@ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage -private[graphx] -class RoutingTableMessageRDDFunctions(self: RDD[RoutingTableMessage]) { - /** Copartition an `RDD[RoutingTableMessage]` with the vertex RDD with the given `partitioner`. */ - def copartitionWithVertices(partitioner: Partitioner): RDD[RoutingTableMessage] = { - new ShuffledRDD[VertexId, Int, Int]( - self, partitioner).setSerializer(new RoutingTableMessageSerializer) - } -} - -private[graphx] -object RoutingTableMessageRDDFunctions { - import scala.language.implicitConversions - - implicit def rdd2RoutingTableMessageRDDFunctions(rdd: RDD[RoutingTableMessage]) = { - new RoutingTableMessageRDDFunctions(rdd) - } -} - private[graphx] object RoutingTablePartition { /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala deleted file mode 100644 index 3909efcdfc993..0000000000000 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ /dev/null @@ -1,369 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.graphx.impl - -import scala.language.existentials - -import java.io.{EOFException, InputStream, OutputStream} -import java.nio.ByteBuffer - -import scala.reflect.ClassTag - -import org.apache.spark.serializer._ - -import org.apache.spark.graphx._ -import org.apache.spark.graphx.impl.RoutingTablePartition.RoutingTableMessage - -private[graphx] -class RoutingTableMessageSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream): SerializationStream = - new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T): SerializationStream = { - val msg = t.asInstanceOf[RoutingTableMessage] - writeVarLong(msg._1, optimizePositive = false) - writeInt(msg._2) - this - } - } - - override def deserializeStream(s: InputStream): DeserializationStream = - new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readInt() - (a, b).asInstanceOf[T] - } - } - } -} - -private[graphx] -class VertexIdMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[(VertexId, _)] - writeVarLong(msg._1, optimizePositive = false) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - (readVarLong(optimizePositive = false), null).asInstanceOf[T] - } - } - } -} - -/** A special shuffle serializer for AggregationMessage[Int]. */ -private[graphx] -class IntAggMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[(VertexId, Int)] - writeVarLong(msg._1, optimizePositive = false) - writeUnsignedVarInt(msg._2) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readUnsignedVarInt() - (a, b).asInstanceOf[T] - } - } - } -} - -/** A special shuffle serializer for AggregationMessage[Long]. */ -private[graphx] -class LongAggMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[(VertexId, Long)] - writeVarLong(msg._1, optimizePositive = false) - writeVarLong(msg._2, optimizePositive = true) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - override def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readVarLong(optimizePositive = true) - (a, b).asInstanceOf[T] - } - } - } -} - -/** A special shuffle serializer for AggregationMessage[Double]. */ -private[graphx] -class DoubleAggMsgSerializer extends Serializer with Serializable { - override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { - - override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { - def writeObject[T: ClassTag](t: T) = { - val msg = t.asInstanceOf[(VertexId, Double)] - writeVarLong(msg._1, optimizePositive = false) - writeDouble(msg._2) - this - } - } - - override def deserializeStream(s: InputStream) = new ShuffleDeserializationStream(s) { - def readObject[T: ClassTag](): T = { - val a = readVarLong(optimizePositive = false) - val b = readDouble() - (a, b).asInstanceOf[T] - } - } - } -} - -//////////////////////////////////////////////////////////////////////////////// -// Helper classes to shorten the implementation of those special serializers. -//////////////////////////////////////////////////////////////////////////////// - -private[graphx] -abstract class ShuffleSerializationStream(s: OutputStream) extends SerializationStream { - // The implementation should override this one. - def writeObject[T: ClassTag](t: T): SerializationStream - - def writeInt(v: Int) { - s.write(v >> 24) - s.write(v >> 16) - s.write(v >> 8) - s.write(v) - } - - def writeUnsignedVarInt(value: Int) { - if ((value >>> 7) == 0) { - s.write(value.toInt) - } else if ((value >>> 14) == 0) { - s.write((value & 0x7F) | 0x80) - s.write(value >>> 7) - } else if ((value >>> 21) == 0) { - s.write((value & 0x7F) | 0x80) - s.write(value >>> 7 | 0x80) - s.write(value >>> 14) - } else if ((value >>> 28) == 0) { - s.write((value & 0x7F) | 0x80) - s.write(value >>> 7 | 0x80) - s.write(value >>> 14 | 0x80) - s.write(value >>> 21) - } else { - s.write((value & 0x7F) | 0x80) - s.write(value >>> 7 | 0x80) - s.write(value >>> 14 | 0x80) - s.write(value >>> 21 | 0x80) - s.write(value >>> 28) - } - } - - def writeVarLong(value: Long, optimizePositive: Boolean) { - val v = if (!optimizePositive) (value << 1) ^ (value >> 63) else value - if ((v >>> 7) == 0) { - s.write(v.toInt) - } else if ((v >>> 14) == 0) { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7).toInt) - } else if ((v >>> 21) == 0) { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7 | 0x80).toInt) - s.write((v >>> 14).toInt) - } else if ((v >>> 28) == 0) { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7 | 0x80).toInt) - s.write((v >>> 14 | 0x80).toInt) - s.write((v >>> 21).toInt) - } else if ((v >>> 35) == 0) { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7 | 0x80).toInt) - s.write((v >>> 14 | 0x80).toInt) - s.write((v >>> 21 | 0x80).toInt) - s.write((v >>> 28).toInt) - } else if ((v >>> 42) == 0) { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7 | 0x80).toInt) - s.write((v >>> 14 | 0x80).toInt) - s.write((v >>> 21 | 0x80).toInt) - s.write((v >>> 28 | 0x80).toInt) - s.write((v >>> 35).toInt) - } else if ((v >>> 49) == 0) { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7 | 0x80).toInt) - s.write((v >>> 14 | 0x80).toInt) - s.write((v >>> 21 | 0x80).toInt) - s.write((v >>> 28 | 0x80).toInt) - s.write((v >>> 35 | 0x80).toInt) - s.write((v >>> 42).toInt) - } else if ((v >>> 56) == 0) { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7 | 0x80).toInt) - s.write((v >>> 14 | 0x80).toInt) - s.write((v >>> 21 | 0x80).toInt) - s.write((v >>> 28 | 0x80).toInt) - s.write((v >>> 35 | 0x80).toInt) - s.write((v >>> 42 | 0x80).toInt) - s.write((v >>> 49).toInt) - } else { - s.write(((v & 0x7F) | 0x80).toInt) - s.write((v >>> 7 | 0x80).toInt) - s.write((v >>> 14 | 0x80).toInt) - s.write((v >>> 21 | 0x80).toInt) - s.write((v >>> 28 | 0x80).toInt) - s.write((v >>> 35 | 0x80).toInt) - s.write((v >>> 42 | 0x80).toInt) - s.write((v >>> 49 | 0x80).toInt) - s.write((v >>> 56).toInt) - } - } - - def writeLong(v: Long) { - s.write((v >>> 56).toInt) - s.write((v >>> 48).toInt) - s.write((v >>> 40).toInt) - s.write((v >>> 32).toInt) - s.write((v >>> 24).toInt) - s.write((v >>> 16).toInt) - s.write((v >>> 8).toInt) - s.write(v.toInt) - } - - def writeDouble(v: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(v)) - - override def flush(): Unit = s.flush() - - override def close(): Unit = s.close() -} - -private[graphx] -abstract class ShuffleDeserializationStream(s: InputStream) extends DeserializationStream { - // The implementation should override this one. - def readObject[T: ClassTag](): T - - def readInt(): Int = { - val first = s.read() - if (first < 0) throw new EOFException - (first & 0xFF) << 24 | (s.read() & 0xFF) << 16 | (s.read() & 0xFF) << 8 | (s.read() & 0xFF) - } - - def readUnsignedVarInt(): Int = { - var value: Int = 0 - var i: Int = 0 - def readOrThrow(): Int = { - val in = s.read() - if (in < 0) throw new EOFException - in & 0xFF - } - var b: Int = readOrThrow() - while ((b & 0x80) != 0) { - value |= (b & 0x7F) << i - i += 7 - if (i > 35) throw new IllegalArgumentException("Variable length quantity is too long") - b = readOrThrow() - } - value | (b << i) - } - - def readVarLong(optimizePositive: Boolean): Long = { - def readOrThrow(): Int = { - val in = s.read() - if (in < 0) throw new EOFException - in & 0xFF - } - var b = readOrThrow() - var ret: Long = b & 0x7F - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= (b & 0x7F) << 7 - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= (b & 0x7F) << 14 - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= (b & 0x7F) << 21 - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= (b & 0x7F).toLong << 28 - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= (b & 0x7F).toLong << 35 - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= (b & 0x7F).toLong << 42 - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= (b & 0x7F).toLong << 49 - if ((b & 0x80) != 0) { - b = readOrThrow() - ret |= b.toLong << 56 - } - } - } - } - } - } - } - } - if (!optimizePositive) (ret >>> 1) ^ -(ret & 1) else ret - } - - def readLong(): Long = { - val first = s.read() - if (first < 0) throw new EOFException() - (first.toLong << 56) | - (s.read() & 0xFF).toLong << 48 | - (s.read() & 0xFF).toLong << 40 | - (s.read() & 0xFF).toLong << 32 | - (s.read() & 0xFF).toLong << 24 | - (s.read() & 0xFF) << 16 | - (s.read() & 0xFF) << 8 | - (s.read() & 0xFF) - } - - def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) - - override def close(): Unit = s.close() -} - -private[graphx] sealed trait ShuffleSerializerInstance extends SerializerInstance { - - override def serialize[T: ClassTag](t: T): ByteBuffer = throw new UnsupportedOperationException - - override def deserialize[T: ClassTag](bytes: ByteBuffer): T = - throw new UnsupportedOperationException - - override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = - throw new UnsupportedOperationException - - // The implementation should override the following two. - override def serializeStream(s: OutputStream): SerializationStream - override def deserializeStream(s: InputStream): DeserializationStream -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala deleted file mode 100644 index 864cb1fdf0022..0000000000000 --- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.graphx - -import java.io.{EOFException, ByteArrayInputStream, ByteArrayOutputStream} - -import scala.util.Random -import scala.reflect.ClassTag - -import org.scalatest.FunSuite - -import org.apache.spark._ -import org.apache.spark.graphx.impl._ -import org.apache.spark.serializer.SerializationStream - - -class SerializerSuite extends FunSuite with LocalSparkContext { - - test("IntAggMsgSerializer") { - val outMsg = (4: VertexId, 5) - val bout = new ByteArrayOutputStream - val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: (VertexId, Int) = inStrm.readObject() - val inMsg2: (VertexId, Int) = inStrm.readObject() - assert(outMsg === inMsg1) - assert(outMsg === inMsg2) - - intercept[EOFException] { - inStrm.readObject() - } - } - - test("LongAggMsgSerializer") { - val outMsg = (4: VertexId, 1L << 32) - val bout = new ByteArrayOutputStream - val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: (VertexId, Long) = inStrm.readObject() - val inMsg2: (VertexId, Long) = inStrm.readObject() - assert(outMsg === inMsg1) - assert(outMsg === inMsg2) - - intercept[EOFException] { - inStrm.readObject() - } - } - - test("DoubleAggMsgSerializer") { - val outMsg = (4: VertexId, 5.0) - val bout = new ByteArrayOutputStream - val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) - outStrm.writeObject(outMsg) - outStrm.writeObject(outMsg) - bout.flush() - val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) - val inMsg1: (VertexId, Double) = inStrm.readObject() - val inMsg2: (VertexId, Double) = inStrm.readObject() - assert(outMsg === inMsg1) - assert(outMsg === inMsg2) - - intercept[EOFException] { - inStrm.readObject() - } - } - - test("variable long encoding") { - def testVarLongEncoding(v: Long, optimizePositive: Boolean) { - val bout = new ByteArrayOutputStream - val stream = new ShuffleSerializationStream(bout) { - def writeObject[T: ClassTag](t: T): SerializationStream = { - writeVarLong(t.asInstanceOf[Long], optimizePositive = optimizePositive) - this - } - } - stream.writeObject(v) - - val bin = new ByteArrayInputStream(bout.toByteArray) - val dstream = new ShuffleDeserializationStream(bin) { - def readObject[T: ClassTag](): T = { - readVarLong(optimizePositive).asInstanceOf[T] - } - } - val read = dstream.readObject[Long]() - assert(read === v) - } - - // Test all variable encoding code path (each branch uses 7 bits, i.e. 1L << 7 difference) - val d = Random.nextLong() % 128 - Seq[Long](0, 1L << 0 + d, 1L << 7 + d, 1L << 14 + d, 1L << 21 + d, 1L << 28 + d, 1L << 35 + d, - 1L << 42 + d, 1L << 49 + d, 1L << 56 + d, 1L << 63 + d).foreach { number => - testVarLongEncoding(number, optimizePositive = false) - testVarLongEncoding(number, optimizePositive = true) - testVarLongEncoding(-number, optimizePositive = false) - testVarLongEncoding(-number, optimizePositive = true) - } - } -} From 3c07b8f08240bafcdff5d174989fb433f4bc80b6 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 10 Nov 2014 22:18:00 -0800 Subject: [PATCH 1287/1492] [SPARK-4330][Doc] Link to proper URL for YARN overview In running-on-yarn.md, a link to YARN overview is here. But the URL is to YARN alpha's. It should be stable's. Author: Kousuke Saruta Closes #3196 from sarutak/SPARK-4330 and squashes the following commits: 30baa21 [Kousuke Saruta] Fixed running-on-yarn.md to point proper URL for YARN --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 695813a2ba881..2f7e4981e5bb9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -4,7 +4,7 @@ title: Running Spark on YARN --- Support for running on [YARN (Hadoop -NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html) +NextGen)](http://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html) was added to Spark in version 0.6.0, and improved in subsequent releases. # Preparations From 65083e93ddd552b7d3e4eb09f87c091ef2ae83a2 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 10 Nov 2014 22:26:16 -0800 Subject: [PATCH 1288/1492] [SPARK-4324] [PySpark] [MLlib] support numpy.array for all MLlib API This PR check all of the existing Python MLlib API to make sure that numpy.array is supported as Vector (also RDD of numpy.array). It also improve some docstring and doctest. cc mateiz mengxr Author: Davies Liu Closes #3189 from davies/numpy and squashes the following commits: d5057c4 [Davies Liu] fix tests 6987611 [Davies Liu] support numpy.array for all MLlib API --- python/pyspark/mllib/classification.py | 13 +++++--- python/pyspark/mllib/feature.py | 31 ++++++++++++++---- python/pyspark/mllib/random.py | 45 ++++++++++++++++++++++++-- python/pyspark/mllib/recommendation.py | 6 ++-- python/pyspark/mllib/regression.py | 15 ++++++--- python/pyspark/mllib/stat.py | 16 ++++++++- python/pyspark/mllib/util.py | 11 ++----- 7 files changed, 105 insertions(+), 32 deletions(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 297a2bf37d2cf..5d90dddb5df1c 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -62,6 +62,7 @@ class LogisticRegressionModel(LinearModel): """ def predict(self, x): + x = _convert_to_vector(x) margin = self.weights.dot(x) + self._intercept if margin > 0: prob = 1 / (1 + exp(-margin)) @@ -79,7 +80,7 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, """ Train a logistic regression model on the given data. - :param data: The training data. + :param data: The training data, an RDD of LabeledPoint. :param iterations: The number of iterations (default: 100). :param step: The step parameter used in SGD (default: 1.0). @@ -136,6 +137,7 @@ class SVMModel(LinearModel): """ def predict(self, x): + x = _convert_to_vector(x) margin = self.weights.dot(x) + self.intercept return 1 if margin >= 0 else 0 @@ -148,7 +150,7 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, """ Train a support vector machine on the given data. - :param data: The training data. + :param data: The training data, an RDD of LabeledPoint. :param iterations: The number of iterations (default: 100). :param step: The step parameter used in SGD (default: 1.0). @@ -233,11 +235,12 @@ def train(cls, data, lambda_=1.0): classification. By making every vector a 0-1 vector, it can also be used as Bernoulli NB (U{http://tinyurl.com/p7c96j6}). - :param data: RDD of NumPy vectors, one per element, where the first - coordinate is the label and the rest is the feature vector - (e.g. a count vector). + :param data: RDD of LabeledPoint. :param lambda_: The smoothing parameter """ + first = data.first() + if not isinstance(first, LabeledPoint): + raise ValueError("`data` should be an RDD of LabeledPoint") labels, pi, theta = callMLlibFunc("trainNaiveBayes", data, lambda_) return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 44bf6f269d7a3..9ec28079aef43 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -25,7 +25,7 @@ from pyspark import RDD, SparkContext from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper -from pyspark.mllib.linalg import Vectors +from pyspark.mllib.linalg import Vectors, _convert_to_vector __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] @@ -81,12 +81,16 @@ def transform(self, vector): """ Applies unit length normalization on a vector. - :param vector: vector to be normalized. + :param vector: vector or RDD of vector to be normalized. :return: normalized vector. If the norm of the input is zero, it will return the input vector. """ sc = SparkContext._active_spark_context assert sc is not None, "SparkContext should be initialized first" + if isinstance(vector, RDD): + vector = vector.map(_convert_to_vector) + else: + vector = _convert_to_vector(vector) return callMLlibFunc("normalizeVector", self.p, vector) @@ -95,8 +99,12 @@ class JavaVectorTransformer(JavaModelWrapper, VectorTransformer): Wrapper for the model in JVM """ - def transform(self, dataset): - return self.call("transform", dataset) + def transform(self, vector): + if isinstance(vector, RDD): + vector = vector.map(_convert_to_vector) + else: + vector = _convert_to_vector(vector) + return self.call("transform", vector) class StandardScalerModel(JavaVectorTransformer): @@ -109,7 +117,7 @@ def transform(self, vector): """ Applies standardization transformation on a vector. - :param vector: Vector to be standardized. + :param vector: Vector or RDD of Vector to be standardized. :return: Standardized vector. If the variance of a column is zero, it will return default `0.0` for the column with zero variance. """ @@ -154,6 +162,7 @@ def fit(self, dataset): the transformation model. :return: a StandardScalarModel """ + dataset = dataset.map(_convert_to_vector) jmodel = callMLlibFunc("fitStandardScaler", self.withMean, self.withStd, dataset) return StandardScalerModel(jmodel) @@ -211,6 +220,8 @@ def transform(self, dataset): :param dataset: an RDD of term frequency vectors :return: an RDD of TF-IDF vectors """ + if not isinstance(dataset, RDD): + raise TypeError("dataset should be an RDD of term frequency vectors") return JavaVectorTransformer.transform(self, dataset) @@ -255,7 +266,9 @@ def fit(self, dataset): :param dataset: an RDD of term frequency vectors """ - jmodel = callMLlibFunc("fitIDF", self.minDocFreq, dataset) + if not isinstance(dataset, RDD): + raise TypeError("dataset should be an RDD of term frequency vectors") + jmodel = callMLlibFunc("fitIDF", self.minDocFreq, dataset.map(_convert_to_vector)) return IDFModel(jmodel) @@ -287,6 +300,8 @@ def findSynonyms(self, word, num): Note: local use only """ + if not isinstance(word, basestring): + word = _convert_to_vector(word) words, similarity = self.call("findSynonyms", word, num) return zip(words, similarity) @@ -374,9 +389,11 @@ def fit(self, data): """ Computes the vector representation of each word in vocabulary. - :param data: training data. RDD of subtype of Iterable[String] + :param data: training data. RDD of list of string :return: Word2VecModel instance """ + if not isinstance(data, RDD): + raise TypeError("data should be an RDD of list of string") jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), float(self.learningRate), int(self.numPartitions), int(self.numIterations), long(self.seed)) diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 7eebfc6bcd894..cb4304f92152b 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -52,6 +52,12 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): C{RandomRDDs.uniformRDD(sc, n, p, seed)\ .map(lambda v: a + (b - a) * v)} + :param sc: SparkContext used to create the RDD. + :param size: Size of the RDD. + :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`). + :param seed: Random seed (default: a random long integer). + :return: RDD of float comprised of i.i.d. samples ~ `U(0.0, 1.0)`. + >>> x = RandomRDDs.uniformRDD(sc, 100).collect() >>> len(x) 100 @@ -76,6 +82,12 @@ def normalRDD(sc, size, numPartitions=None, seed=None): C{RandomRDDs.normal(sc, n, p, seed)\ .map(lambda v: mean + sigma * v)} + :param sc: SparkContext used to create the RDD. + :param size: Size of the RDD. + :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`). + :param seed: Random seed (default: a random long integer). + :return: RDD of float comprised of i.i.d. samples ~ N(0.0, 1.0). + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) >>> stats = x.stats() >>> stats.count() @@ -93,6 +105,13 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): Generates an RDD comprised of i.i.d. samples from the Poisson distribution with the input mean. + :param sc: SparkContext used to create the RDD. + :param mean: Mean, or lambda, for the Poisson distribution. + :param size: Size of the RDD. + :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`). + :param seed: Random seed (default: a random long integer). + :return: RDD of float comprised of i.i.d. samples ~ Pois(mean). + >>> mean = 100.0 >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2L) >>> stats = x.stats() @@ -104,7 +123,7 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - sqrt(mean)) < 0.5 True """ - return callMLlibFunc("poissonRDD", sc._jsc, mean, size, numPartitions, seed) + return callMLlibFunc("poissonRDD", sc._jsc, float(mean), size, numPartitions, seed) @staticmethod @toArray @@ -113,6 +132,13 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): Generates an RDD comprised of vectors containing i.i.d. samples drawn from the uniform distribution U(0.0, 1.0). + :param sc: SparkContext used to create the RDD. + :param numRows: Number of Vectors in the RDD. + :param numCols: Number of elements in each Vector. + :param numPartitions: Number of partitions in the RDD. + :param seed: Seed for the RNG that generates the seed for the generator in each partition. + :return: RDD of Vector with vectors containing i.i.d samples ~ `U(0.0, 1.0)`. + >>> import numpy as np >>> mat = np.matrix(RandomRDDs.uniformVectorRDD(sc, 10, 10).collect()) >>> mat.shape @@ -131,6 +157,13 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): Generates an RDD comprised of vectors containing i.i.d. samples drawn from the standard normal distribution. + :param sc: SparkContext used to create the RDD. + :param numRows: Number of Vectors in the RDD. + :param numCols: Number of elements in each Vector. + :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`). + :param seed: Random seed (default: a random long integer). + :return: RDD of Vector with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. + >>> import numpy as np >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) >>> mat.shape @@ -149,6 +182,14 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): Generates an RDD comprised of vectors containing i.i.d. samples drawn from the Poisson distribution with the input mean. + :param sc: SparkContext used to create the RDD. + :param mean: Mean, or lambda, for the Poisson distribution. + :param numRows: Number of Vectors in the RDD. + :param numCols: Number of elements in each Vector. + :param numPartitions: Number of partitions in the RDD (default: `sc.defaultParallelism`) + :param seed: Random seed (default: a random long integer). + :return: RDD of Vector with vectors containing i.i.d. samples ~ Pois(mean). + >>> import numpy as np >>> mean = 100.0 >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) @@ -161,7 +202,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - sqrt(mean)) < 0.5 True """ - return callMLlibFunc("poissonVectorRDD", sc._jsc, mean, numRows, numCols, + return callMLlibFunc("poissonVectorRDD", sc._jsc, float(mean), numRows, numCols, numPartitions, seed) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index e26b152e0cdfd..41bbd9a779c70 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -32,7 +32,7 @@ def __reduce__(self): return Rating, (self.user, self.product, self.rating) def __repr__(self): - return "Rating(%d, %d, %d)" % (self.user, self.product, self.rating) + return "Rating(%d, %d, %s)" % (self.user, self.product, self.rating) class MatrixFactorizationModel(JavaModelWrapper): @@ -51,7 +51,7 @@ class MatrixFactorizationModel(JavaModelWrapper): >>> testset = sc.parallelize([(1, 2), (1, 1)]) >>> model = ALS.train(ratings, 1, seed=10) >>> model.predictAll(testset).collect() - [Rating(1, 1, 1), Rating(1, 2, 1)] + [Rating(1, 1, 1.0471...), Rating(1, 2, 1.9679...)] >>> model = ALS.train(ratings, 4, seed=10) >>> model.userFeatures().collect() @@ -79,7 +79,7 @@ class MatrixFactorizationModel(JavaModelWrapper): 0.4473... """ def predict(self, user, product): - return self._java_model.predict(user, product) + return self._java_model.predict(int(user), int(product)) def predictAll(self, user_product): assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 43c1a2fc101dd..66e25a48dfa71 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -36,7 +36,7 @@ class LabeledPoint(object): """ def __init__(self, label, features): - self.label = label + self.label = float(label) self.features = _convert_to_vector(features) def __reduce__(self): @@ -46,7 +46,7 @@ def __str__(self): return "(" + ",".join((str(self.label), str(self.features))) + ")" def __repr__(self): - return "LabeledPoint(" + ",".join((repr(self.label), repr(self.features))) + ")" + return "LabeledPoint(%s, %s)" % (self.label, self.features) class LinearModel(object): @@ -55,7 +55,7 @@ class LinearModel(object): def __init__(self, weights, intercept): self._coeff = _convert_to_vector(weights) - self._intercept = intercept + self._intercept = float(intercept) @property def weights(self): @@ -66,7 +66,7 @@ def intercept(self): return self._intercept def __repr__(self): - return "(weights=%s, intercept=%s)" % (self._coeff, self._intercept) + return "(weights=%s, intercept=%r)" % (self._coeff, self._intercept) class LinearRegressionModelBase(LinearModel): @@ -85,6 +85,7 @@ def predict(self, x): Predict the value of the dependent variable given a vector x containing values for the independent variables. """ + x = _convert_to_vector(x) return self.weights.dot(x) + self.intercept @@ -124,6 +125,9 @@ class LinearRegressionModel(LinearRegressionModelBase): # return the result of a call to the appropriate JVM stub. # _regression_train_wrapper is responsible for setup and error checking. def _regression_train_wrapper(train_func, modelClass, data, initial_weights): + first = data.first() + if not isinstance(first, LabeledPoint): + raise ValueError("data should be an RDD of LabeledPoint, but got %s" % first) initial_weights = initial_weights or [0.0] * len(data.first().features) weights, intercept = train_func(_to_java_object_rdd(data, cache=True), _convert_to_vector(initial_weights)) @@ -264,7 +268,8 @@ def train(rdd, i): def _test(): import doctest from pyspark import SparkContext - globs = globals().copy() + import pyspark.mllib.regression + globs = pyspark.mllib.regression.__dict__.copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 0700f8a8e5a8e..1980f5b03f430 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -22,6 +22,7 @@ from pyspark import RDD from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import Matrix, _convert_to_vector +from pyspark.mllib.regression import LabeledPoint __all__ = ['MultivariateStatisticalSummary', 'ChiSqTestResult', 'Statistics'] @@ -107,6 +108,11 @@ def colStats(rdd): """ Computes column-wise summary statistics for the input RDD[Vector]. + :param rdd: an RDD[Vector] for which column-wise summary statistics + are to be computed. + :return: :class:`MultivariateStatisticalSummary` object containing + column-wise summary statistics. + >>> from pyspark.mllib.linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([2, 0, 0, -2]), ... Vectors.dense([4, 5, 0, 3]), @@ -140,6 +146,13 @@ def corr(x, y=None, method=None): to specify the method to be used for single RDD inout. If two RDDs of floats are passed in, a single float is returned. + :param x: an RDD of vector for which the correlation matrix is to be computed, + or an RDD of float of the same cardinality as y when y is specified. + :param y: an RDD of float of the same cardinality as x. + :param method: String specifying the method to use for computing correlation. + Supported: `pearson` (default), `spearman` + :return: Correlation matrix comparing columns in x. + >>> x = sc.parallelize([1.0, 0.0, -2.0], 2) >>> y = sc.parallelize([4.0, 5.0, 3.0], 2) >>> zeros = sc.parallelize([0.0, 0.0, 0.0], 2) @@ -242,7 +255,6 @@ def chiSqTest(observed, expected=None): >>> print round(chi.statistic, 4) 21.9958 - >>> from pyspark.mllib.regression import LabeledPoint >>> data = [LabeledPoint(0.0, Vectors.dense([0.5, 10.0])), ... LabeledPoint(0.0, Vectors.dense([1.5, 20.0])), ... LabeledPoint(1.0, Vectors.dense([1.5, 30.0])), @@ -257,6 +269,8 @@ def chiSqTest(observed, expected=None): 1.5 """ if isinstance(observed, RDD): + if not isinstance(observed.first(), LabeledPoint): + raise ValueError("observed should be an RDD of LabeledPoint") jmodels = callMLlibFunc("chiSqTest", observed) return [ChiSqTestResult(m) for m in jmodels] diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 96aef8f510fa6..4ed978b45409c 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -161,15 +161,8 @@ def loadLabeledPoints(sc, path, minPartitions=None): >>> tempFile = NamedTemporaryFile(delete=True) >>> tempFile.close() >>> sc.parallelize(examples, 1).saveAsTextFile(tempFile.name) - >>> loaded = MLUtils.loadLabeledPoints(sc, tempFile.name).collect() - >>> type(loaded[0]) == LabeledPoint - True - >>> print examples[0] - (1.1,(3,[0,2],[-1.23,4.56e-07])) - >>> type(examples[1]) == LabeledPoint - True - >>> print examples[1] - (0.0,[1.01,2.02,3.03]) + >>> MLUtils.loadLabeledPoints(sc, tempFile.name).collect() + [LabeledPoint(1.1, (3,[0,2],[-1.23,4.56e-07])), LabeledPoint(0.0, [1.01,2.02,3.03])] """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) return callMLlibFunc("loadLabeledPoints", sc, path, minPartitions) From ef29a9a9aa85468869eb67ca67b66c65f508d0ee Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 11 Nov 2014 00:25:31 -0800 Subject: [PATCH 1289/1492] [SPARK-4307] Initialize FileDescriptor lazily in FileRegion. Netty's DefaultFileRegion requires a FileDescriptor in its constructor, which means we need to have a opened file handle. In super large workloads, this could lead to too many open files due to the way these file descriptors are cleaned. This pull request creates a new LazyFileRegion that initializes the FileDescriptor when we are sending data for the first time. Author: Reynold Xin Author: Reynold Xin Closes #3172 from rxin/lazyFD and squashes the following commits: 0bdcdc6 [Reynold Xin] Added reference to Netty's DefaultFileRegion d4564ae [Reynold Xin] Added SparkConf to the ctor argument of IndexShuffleBlockManager. 6ed369e [Reynold Xin] Code review feedback. 04cddc8 [Reynold Xin] [SPARK-4307] Initialize FileDescriptor lazily in FileRegion. --- .../StandaloneWorkerShuffleService.scala | 2 +- .../shuffle/FileShuffleBlockManager.scala | 8 +- .../shuffle/IndexShuffleBlockManager.scala | 8 +- .../shuffle/sort/SortShuffleManager.scala | 2 +- .../spark/ExternalShuffleServiceSuite.scala | 2 +- .../buffer/FileSegmentManagedBuffer.java | 23 ++-- .../spark/network/buffer/LazyFileRegion.java | 111 ++++++++++++++++++ .../spark/network/util/TransportConf.java | 17 +++ .../network/ChunkFetchIntegrationSuite.java | 9 +- .../shuffle/ExternalShuffleBlockHandler.java | 5 +- .../shuffle/ExternalShuffleBlockManager.java | 13 +- .../ExternalShuffleBlockManagerSuite.java | 10 +- .../shuffle/ExternalShuffleCleanupSuite.java | 13 +- .../ExternalShuffleIntegrationSuite.java | 2 +- .../shuffle/ExternalShuffleSecuritySuite.java | 2 +- .../network/yarn/YarnShuffleService.java | 4 +- 16 files changed, 191 insertions(+), 40 deletions(-) create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala index 88118e2837741..d044e1d01d429 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala @@ -40,7 +40,7 @@ class StandaloneWorkerShuffleService(sparkConf: SparkConf, securityManager: Secu private val useSasl: Boolean = securityManager.isAuthenticationEnabled() private val transportConf = SparkTransportConf.fromSparkConf(sparkConf) - private val blockHandler = new ExternalShuffleBlockHandler() + private val blockHandler = new ExternalShuffleBlockHandler(transportConf) private val transportContext: TransportContext = { val handler = if (useSasl) new SaslRpcHandler(blockHandler, securityManager) else blockHandler new TransportContext(transportConf, handler) diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index f03e8e4bf1b7e..7de2f9cbb2866 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -27,6 +27,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup import org.apache.spark.storage._ @@ -68,6 +69,8 @@ private[spark] class FileShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager with Logging { + private val transportConf = SparkTransportConf.fromSparkConf(conf) + private lazy val blockManager = SparkEnv.get.blockManager // Turning off shuffle file consolidation causes all shuffle Blocks to get their own file. @@ -182,13 +185,14 @@ class FileShuffleBlockManager(conf: SparkConf) val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId) if (segmentOpt.isDefined) { val segment = segmentOpt.get - return new FileSegmentManagedBuffer(segment.file, segment.offset, segment.length) + return new FileSegmentManagedBuffer( + transportConf, segment.file, segment.offset, segment.length) } } throw new IllegalStateException("Failed to find shuffle block: " + blockId) } else { val file = blockManager.diskBlockManager.getFile(blockId) - new FileSegmentManagedBuffer(file, 0, file.length) + new FileSegmentManagedBuffer(transportConf, file, 0, file.length) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index a48f0c9eceb5e..b292587d37028 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -22,8 +22,9 @@ import java.nio.ByteBuffer import com.google.common.io.ByteStreams -import org.apache.spark.SparkEnv +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.storage._ /** @@ -38,10 +39,12 @@ import org.apache.spark.storage._ // Note: Changes to the format in this file should be kept in sync with // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getSortBasedShuffleBlockData(). private[spark] -class IndexShuffleBlockManager extends ShuffleBlockManager { +class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager { private lazy val blockManager = SparkEnv.get.blockManager + private val transportConf = SparkTransportConf.fromSparkConf(conf) + /** * Mapping to a single shuffleBlockId with reduce ID 0. * */ @@ -109,6 +112,7 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { val offset = in.readLong() val nextOffset = in.readLong() new FileSegmentManagedBuffer( + transportConf, getDataFile(blockId.shuffleId, blockId.mapId), offset, nextOffset - offset) diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index b727438ae7e47..bda30a56d808e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -25,7 +25,7 @@ import org.apache.spark.shuffle.hash.HashShuffleReader private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager { - private val indexShuffleBlockManager = new IndexShuffleBlockManager() + private val indexShuffleBlockManager = new IndexShuffleBlockManager(conf) private val shuffleMapNumber = new ConcurrentHashMap[Int, Int]() /** diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 6608ed1e57b38..9623d665177ef 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -39,7 +39,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { override def beforeAll() { val transportConf = SparkTransportConf.fromSparkConf(conf) - rpcHandler = new ExternalShuffleBlockHandler() + rpcHandler = new ExternalShuffleBlockHandler(transportConf) val transportContext = new TransportContext(transportConf, rpcHandler) server = transportContext.createServer() diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java index 5fa1527ddff92..844eff4f4c701 100644 --- a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -31,24 +31,19 @@ import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.LimitedInputStream; +import org.apache.spark.network.util.TransportConf; /** * A {@link ManagedBuffer} backed by a segment in a file. */ public final class FileSegmentManagedBuffer extends ManagedBuffer { - - /** - * Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889). - * Avoid unless there's a good reason not to. - */ - // TODO: Make this configurable - private static final long MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024; - + private final TransportConf conf; private final File file; private final long offset; private final long length; - public FileSegmentManagedBuffer(File file, long offset, long length) { + public FileSegmentManagedBuffer(TransportConf conf, File file, long offset, long length) { + this.conf = conf; this.file = file; this.offset = offset; this.length = length; @@ -65,7 +60,7 @@ public ByteBuffer nioByteBuffer() throws IOException { try { channel = new RandomAccessFile(file, "r").getChannel(); // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. - if (length < MIN_MEMORY_MAP_BYTES) { + if (length < conf.memoryMapBytes()) { ByteBuffer buf = ByteBuffer.allocate((int) length); channel.position(offset); while (buf.remaining() != 0) { @@ -134,8 +129,12 @@ public ManagedBuffer release() { @Override public Object convertToNetty() throws IOException { - FileChannel fileChannel = new FileInputStream(file).getChannel(); - return new DefaultFileRegion(fileChannel, offset, length); + if (conf.lazyFileDescriptor()) { + return new LazyFileRegion(file, offset, length); + } else { + FileChannel fileChannel = new FileInputStream(file).getChannel(); + return new DefaultFileRegion(fileChannel, offset, length); + } } public File getFile() { return file; } diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java b/network/common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java new file mode 100644 index 0000000000000..81bc8ec40fc82 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/LazyFileRegion.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.buffer; + +import java.io.FileInputStream; +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.WritableByteChannel; + +import com.google.common.base.Objects; +import io.netty.channel.FileRegion; +import io.netty.util.AbstractReferenceCounted; + +import org.apache.spark.network.util.JavaUtils; + +/** + * A FileRegion implementation that only creates the file descriptor when the region is being + * transferred. This cannot be used with Epoll because there is no native support for it. + * + * This is mostly copied from DefaultFileRegion implementation in Netty. In the future, we + * should push this into Netty so the native Epoll transport can support this feature. + */ +public final class LazyFileRegion extends AbstractReferenceCounted implements FileRegion { + + private final File file; + private final long position; + private final long count; + + private FileChannel channel; + + private long numBytesTransferred = 0L; + + /** + * @param file file to transfer. + * @param position start position for the transfer. + * @param count number of bytes to transfer starting from position. + */ + public LazyFileRegion(File file, long position, long count) { + this.file = file; + this.position = position; + this.count = count; + } + + @Override + protected void deallocate() { + JavaUtils.closeQuietly(channel); + } + + @Override + public long position() { + return position; + } + + @Override + public long transfered() { + return numBytesTransferred; + } + + @Override + public long count() { + return count; + } + + @Override + public long transferTo(WritableByteChannel target, long position) throws IOException { + if (channel == null) { + channel = new FileInputStream(file).getChannel(); + } + + long count = this.count - position; + if (count < 0 || position < 0) { + throw new IllegalArgumentException( + "position out of range: " + position + " (expected: 0 - " + (count - 1) + ')'); + } + + if (count == 0) { + return 0L; + } + + long written = channel.transferTo(this.position + position, count, target); + if (written > 0) { + numBytesTransferred += written; + } + return written; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("file", file) + .add("position", position) + .add("count", count) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 787a8f0031af1..621427d8cba5e 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -75,4 +75,21 @@ public int connectionTimeoutMs() { * Only relevant if maxIORetries > 0. */ public int ioRetryWaitTime() { return conf.getInt("spark.shuffle.io.retryWaitMs", 5000); } + + /** + * Minimum size of a block that we should start using memory map rather than reading in through + * normal IO operations. This prevents Spark from memory mapping very small blocks. In general, + * memory mapping has high overhead for blocks close to or below the page size of the OS. + */ + public int memoryMapBytes() { + return conf.getInt("spark.storage.memoryMapThreshold", 2 * 1024 * 1024); + } + + /** + * Whether to initialize shuffle FileDescriptor lazily or not. If true, file descriptors are + * created only when data is going to be transferred. This can reduce the number of open files. + */ + public boolean lazyFileDescriptor() { + return conf.getBoolean("spark.shuffle.io.lazyFD", true); + } } diff --git a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index c4158833976aa..dfb7740344ed0 100644 --- a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -63,6 +63,8 @@ public class ChunkFetchIntegrationSuite { static ManagedBuffer bufferChunk; static ManagedBuffer fileChunk; + private TransportConf transportConf; + @BeforeClass public static void setUp() throws Exception { int bufSize = 100000; @@ -80,9 +82,10 @@ public static void setUp() throws Exception { new Random().nextBytes(fileContent); fp.write(fileContent); fp.close(); - fileChunk = new FileSegmentManagedBuffer(testFile, 10, testFile.length() - 25); - TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + final TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + fileChunk = new FileSegmentManagedBuffer(conf, testFile, 10, testFile.length() - 25); + streamManager = new StreamManager() { @Override public ManagedBuffer getChunk(long streamId, int chunkIndex) { @@ -90,7 +93,7 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) { if (chunkIndex == BUFFER_CHUNK_INDEX) { return new NioManagedBuffer(buf); } else if (chunkIndex == FILE_CHUNK_INDEX) { - return new FileSegmentManagedBuffer(testFile, 10, testFile.length() - 25); + return new FileSegmentManagedBuffer(conf, testFile, 10, testFile.length() - 25); } else { throw new IllegalArgumentException("Invalid chunk index: " + chunkIndex); } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index a6db4b2abd6c9..46ca9708621b9 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; +import org.apache.spark.network.util.TransportConf; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,8 +49,8 @@ public class ExternalShuffleBlockHandler extends RpcHandler { private final ExternalShuffleBlockManager blockManager; private final OneForOneStreamManager streamManager; - public ExternalShuffleBlockHandler() { - this(new OneForOneStreamManager(), new ExternalShuffleBlockManager()); + public ExternalShuffleBlockHandler(TransportConf conf) { + this(new OneForOneStreamManager(), new ExternalShuffleBlockManager(conf)); } /** Enables mocking out the StreamManager and BlockManager. */ diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java index ffb7faa3dbdca..dfe0ba0595090 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java @@ -37,6 +37,7 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; /** * Manages converting shuffle BlockIds into physical segments of local files, from a process outside @@ -56,14 +57,17 @@ public class ExternalShuffleBlockManager { // Single-threaded Java executor used to perform expensive recursive directory deletion. private final Executor directoryCleaner; - public ExternalShuffleBlockManager() { + private final TransportConf conf; + + public ExternalShuffleBlockManager(TransportConf conf) { // TODO: Give this thread a name. - this(Executors.newSingleThreadExecutor()); + this(conf, Executors.newSingleThreadExecutor()); } // Allows tests to have more control over when directories are cleaned up. @VisibleForTesting - ExternalShuffleBlockManager(Executor directoryCleaner) { + ExternalShuffleBlockManager(TransportConf conf, Executor directoryCleaner) { + this.conf = conf; this.executors = Maps.newConcurrentMap(); this.directoryCleaner = directoryCleaner; } @@ -167,7 +171,7 @@ private void deleteExecutorDirs(String[] dirs) { // TODO: Support consolidated hash shuffle files private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, String blockId) { File shuffleFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId); - return new FileSegmentManagedBuffer(shuffleFile, 0, shuffleFile.length()); + return new FileSegmentManagedBuffer(conf, shuffleFile, 0, shuffleFile.length()); } /** @@ -187,6 +191,7 @@ private ManagedBuffer getSortBasedShuffleBlockData( long offset = in.readLong(); long nextOffset = in.readLong(); return new FileSegmentManagedBuffer( + conf, getFile(executor.localDirs, executor.subDirsPerLocalDir, "shuffle_" + shuffleId + "_" + mapId + "_0.data"), offset, diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java index da54797e8923c..dad6428a836fc 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java @@ -22,6 +22,8 @@ import java.io.InputStreamReader; import com.google.common.io.CharStreams; +import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.TransportConf; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -37,6 +39,8 @@ public class ExternalShuffleBlockManagerSuite { static TestShuffleDataContext dataContext; + static TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + @BeforeClass public static void beforeAll() throws IOException { dataContext = new TestShuffleDataContext(2, 5); @@ -56,7 +60,7 @@ public static void afterAll() { @Test public void testBadRequests() { - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(); + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf); // Unregistered executor try { manager.getBlockData("app0", "exec1", "shuffle_1_1_0"); @@ -87,7 +91,7 @@ public void testBadRequests() { @Test public void testSortShuffleBlocks() throws IOException { - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(); + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf); manager.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); @@ -106,7 +110,7 @@ public void testSortShuffleBlocks() throws IOException { @Test public void testHashShuffleBlocks() throws IOException { - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(); + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf); manager.registerExecutor("app0", "exec0", dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java index c8ece3bc53ac3..254e3a7a32b98 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleCleanupSuite.java @@ -25,20 +25,23 @@ import com.google.common.util.concurrent.MoreExecutors; import org.junit.Test; - import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.TransportConf; + public class ExternalShuffleCleanupSuite { // Same-thread Executor used to ensure cleanup happens synchronously in test thread. Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); @Test public void noCleanupAndCleanup() throws IOException { TestShuffleDataContext dataContext = createSomeData(); - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(sameThreadExecutor); + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, sameThreadExecutor); manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); manager.applicationRemoved("app", false /* cleanup */); @@ -61,7 +64,7 @@ public void cleanupUsesExecutor() throws IOException { @Override public void execute(Runnable runnable) { cleanupCalled.set(true); } }; - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(noThreadExecutor); + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, noThreadExecutor); manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo("shuffleMgr")); manager.applicationRemoved("app", true); @@ -78,7 +81,7 @@ public void cleanupMultipleExecutors() throws IOException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(sameThreadExecutor); + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, sameThreadExecutor); manager.registerExecutor("app", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); manager.registerExecutor("app", "exec1", dataContext1.createExecutorInfo("shuffleMgr")); @@ -93,7 +96,7 @@ public void cleanupOnlyRemovedApp() throws IOException { TestShuffleDataContext dataContext0 = createSomeData(); TestShuffleDataContext dataContext1 = createSomeData(); - ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(sameThreadExecutor); + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(conf, sameThreadExecutor); manager.registerExecutor("app-0", "exec0", dataContext0.createExecutorInfo("shuffleMgr")); manager.registerExecutor("app-1", "exec0", dataContext1.createExecutorInfo("shuffleMgr")); diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 687bde59fdae4..02c10bcb7b261 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -92,7 +92,7 @@ public static void beforeAll() throws IOException { dataContext1.insertHashShuffleData(1, 0, exec1Blocks); conf = new TransportConf(new SystemPropertyConfigProvider()); - handler = new ExternalShuffleBlockHandler(); + handler = new ExternalShuffleBlockHandler(conf); TransportContext transportContext = new TransportContext(conf, handler); server = transportContext.createServer(); } diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 8afceab1d585a..759a12910c94d 100644 --- a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -42,7 +42,7 @@ public class ExternalShuffleSecuritySuite { @Before public void beforeEach() { - RpcHandler handler = new SaslRpcHandler(new ExternalShuffleBlockHandler(), + RpcHandler handler = new SaslRpcHandler(new ExternalShuffleBlockHandler(conf), new TestSecretKeyHolder("my-app-id", "secret")); TransportContext context = new TransportContext(conf, handler); this.server = context.createServer(); diff --git a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index bb0b8f7e6cba6..a34aabe9e78a6 100644 --- a/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/network/yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -95,10 +95,11 @@ private boolean isAuthenticationEnabled() { */ @Override protected void serviceInit(Configuration conf) { + TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests boolean authEnabled = conf.getBoolean(SPARK_AUTHENTICATE_KEY, DEFAULT_SPARK_AUTHENTICATE); - RpcHandler rpcHandler = new ExternalShuffleBlockHandler(); + RpcHandler rpcHandler = new ExternalShuffleBlockHandler(transportConf); if (authEnabled) { secretManager = new ShuffleSecretManager(); rpcHandler = new SaslRpcHandler(rpcHandler, secretManager); @@ -106,7 +107,6 @@ protected void serviceInit(Configuration conf) { int port = conf.getInt( SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - TransportConf transportConf = new TransportConf(new HadoopConfigProvider(conf)); TransportContext transportContext = new TransportContext(transportConf, rpcHandler); shuffleServer = transportContext.createServer(port); String authEnabledString = authEnabled ? "enabled" : "not enabled"; From f8811a5695af2dfe156f07431288db7b8cd97159 Mon Sep 17 00:00:00 2001 From: maji2014 Date: Tue, 11 Nov 2014 02:18:27 -0800 Subject: [PATCH 1290/1492] [SPARK-4295][External]Fix exception in SparkSinkSuite Handle exception in SparkSinkSuite, please refer to [SPARK-4295] Author: maji2014 Closes #3177 from maji2014/spark-4295 and squashes the following commits: 312620a [maji2014] change a new statement for spark-4295 24c3d21 [maji2014] add log4j.properties for SparkSinkSuite and spark-4295 c807bf6 [maji2014] Fix exception in SparkSinkSuite --- .../src/test/resources/log4j.properties | 29 +++++++++++++++++++ .../streaming/flume/sink/SparkSinkSuite.scala | 1 + 2 files changed, 30 insertions(+) create mode 100644 external/flume-sink/src/test/resources/log4j.properties diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..4411d6e20c52a --- /dev/null +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -0,0 +1,29 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file streaming/target/unit-tests.log +log4j.rootCategory=INFO, file +# log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN + diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index a2b2cc6149d95..650b2fbe1c142 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -159,6 +159,7 @@ class SparkSinkSuite extends FunSuite { channelContext.put("transactionCapacity", 1000.toString) channelContext.put("keep-alive", 0.toString) channelContext.putAll(overrides) + channel.setName(scala.util.Random.nextString(10)) channel.configure(channelContext) val sink = new SparkSink() From c8850a3d6d948f9dd9ee026ee350428968d3c21b Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 11 Nov 2014 02:22:23 -0800 Subject: [PATCH 1291/1492] [SPARK-2492][Streaming] kafkaReceiver minor changes to align with Kafka 0.8 Update the KafkaReceiver's behavior when auto.offset.reset is set. In Kafka 0.8, `auto.offset.reset` is a hint for out-range offset to seek to the beginning or end of the partition. While in the previous code `auto.offset.reset` is a enforcement to seek to the beginning or end immediately, this is different from Kafka 0.8 defined behavior. Also deleting extesting ZK metadata in Receiver when multiple consumers are launched will introduce issue as mentioned in [SPARK-2383](https://issues.apache.org/jira/browse/SPARK-2383). So Here we change to offer user to API to explicitly reset offset before create Kafka stream, while in the meantime keep the same behavior as Kafka 0.8 for parameter `auto.offset.reset`. @tdas, would you please review this PR? Thanks a lot. Author: jerryshao Closes #1420 from jerryshao/kafka-fix and squashes the following commits: d6ae94d [jerryshao] Address the comment to remove the resetOffset() function de3a4c8 [jerryshao] Fix compile error 4a1c3f9 [jerryshao] Doc changes b2c1430 [jerryshao] Move offset reset to a helper function to let user explicitly delete ZK metadata by calling this API fac8fd6 [jerryshao] Changes to align with Kafka 0.8 --- .../streaming/kafka/KafkaInputDStream.scala | 30 ------------------- .../spark/streaming/kafka/KafkaUtils.scala | 11 ++++--- 2 files changed, 5 insertions(+), 36 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index e20e2c8f26991..28ac5929df44a 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -26,8 +26,6 @@ import java.util.concurrent.Executors import kafka.consumer._ import kafka.serializer.Decoder import kafka.utils.VerifiableProperties -import kafka.utils.ZKStringSerializer -import org.I0Itec.zkclient._ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel @@ -97,12 +95,6 @@ class KafkaReceiver[ consumerConnector = Consumer.create(consumerConfig) logInfo("Connected to " + zkConnect) - // When auto.offset.reset is defined, it is our responsibility to try and whack the - // consumer group zk node. - if (kafkaParams.contains("auto.offset.reset")) { - tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id")) - } - val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) .newInstance(consumerConfig.props) .asInstanceOf[Decoder[K]] @@ -139,26 +131,4 @@ class KafkaReceiver[ } } } - - // It is our responsibility to delete the consumer group when specifying auto.offset.reset. This - // is because Kafka 0.7.2 only honors this param when the group is not in zookeeper. - // - // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied - // from Kafka's ConsoleConsumer. See code related to 'auto.offset.reset' when it is set to - // 'smallest'/'largest': - // scalastyle:off - // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala - // scalastyle:on - private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { - val dir = "/consumers/" + groupId - logInfo("Cleaning up temporary Zookeeper data under " + dir + ".") - val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) - try { - zk.deleteRecursive(dir) - } catch { - case e: Throwable => logWarning("Error cleaning up temporary Zookeeper data", e) - } finally { - zk.close() - } - } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 48668f763e41e..ec812e1ef3b04 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -17,19 +17,18 @@ package org.apache.spark.streaming.kafka -import scala.reflect.ClassTag -import scala.collection.JavaConversions._ - import java.lang.{Integer => JInt} import java.util.{Map => JMap} +import scala.reflect.ClassTag +import scala.collection.JavaConversions._ + import kafka.serializer.{Decoder, StringDecoder} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext, JavaPairDStream} -import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} - +import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.ReceiverInputDStream object KafkaUtils { /** From 6e03de304e0294017d832763fd71e642736f8c33 Mon Sep 17 00:00:00 2001 From: huangzhaowei Date: Tue, 11 Nov 2014 03:02:12 -0800 Subject: [PATCH 1292/1492] [Streaming][Minor]Replace some 'if-else' in Clock Replace some 'if-else' statement by math.min and math.max in Clock.scala Author: huangzhaowei Closes #3088 from SaintBacchus/StreamingClock and squashes the following commits: 7b7f8e7 [huangzhaowei] [Streaming][Minor]Replace some 'if-else' in Clock --- .../org/apache/spark/streaming/util/Clock.scala | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index 39145a3ab081a..7cd867ce34b87 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -41,13 +41,7 @@ class SystemClock() extends Clock { return currentTime } - val pollTime = { - if (waitTime / 10.0 > minPollTime) { - (waitTime / 10.0).toLong - } else { - minPollTime - } - } + val pollTime = math.max(waitTime / 10.0, minPollTime).toLong while (true) { currentTime = System.currentTimeMillis() @@ -55,12 +49,7 @@ class SystemClock() extends Clock { if (waitTime <= 0) { return currentTime } - val sleepTime = - if (waitTime < pollTime) { - waitTime - } else { - pollTime - } + val sleepTime = math.min(waitTime, pollTime) Thread.sleep(sleepTime) } -1 From deefd9d7377a8091a1d184b99066febd0e9f6afd Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 11 Nov 2014 09:29:48 -0800 Subject: [PATCH 1293/1492] SPARK-1830 Deploy failover, Make Persistence engine and LeaderAgent Pluggable Author: Prashant Sharma Closes #771 from ScrapCodes/deploy-failover-pluggable and squashes the following commits: 29ba440 [Prashant Sharma] fixed a compilation error fef35ec [Prashant Sharma] Code review 57ee6f0 [Prashant Sharma] SPARK-1830 Deploy failover, Make Persistence engine and LeaderAgent Pluggable. --- .../spark/deploy/master/ApplicationInfo.scala | 1 + .../spark/deploy/master/DriverInfo.scala | 1 + .../master/FileSystemPersistenceEngine.scala | 62 +++++----------- .../deploy/master/LeaderElectionAgent.scala | 37 +++++----- .../apache/spark/deploy/master/Master.scala | 40 ++++++----- .../deploy/master/PersistenceEngine.scala | 70 ++++++++++++++----- .../deploy/master/RecoveryModeFactory.scala | 69 ++++++++++++++++++ .../spark/deploy/master/WorkerInfo.scala | 1 + .../master/ZooKeeperLeaderElectionAgent.scala | 24 ++----- .../master/ZooKeeperPersistenceEngine.scala | 56 +++++---------- 10 files changed, 211 insertions(+), 150 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index 6ba395be1cc2c..ad7d81747c377 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.ArrayBuffer import akka.actor.ActorRef +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.ApplicationDescription import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 2ac21186881fa..9d3d7938c6ccb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.master import java.util.Date +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.DriverDescription import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 08a99bbe68578..6ff2aa5244847 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -18,10 +18,12 @@ package org.apache.spark.deploy.master import java.io._ - -import akka.serialization.Serialization +import java.nio.ByteBuffer import org.apache.spark.Logging +import org.apache.spark.serializer.Serializer + +import scala.reflect.ClassTag /** * Stores data in a single on-disk directory with one file per application and worker. @@ -32,65 +34,39 @@ import org.apache.spark.Logging */ private[spark] class FileSystemPersistenceEngine( val dir: String, - val serialization: Serialization) + val serialization: Serializer) extends PersistenceEngine with Logging { + val serializer = serialization.newInstance() new File(dir).mkdir() - override def addApplication(app: ApplicationInfo) { - val appFile = new File(dir + File.separator + "app_" + app.id) - serializeIntoFile(appFile, app) - } - - override def removeApplication(app: ApplicationInfo) { - new File(dir + File.separator + "app_" + app.id).delete() - } - - override def addDriver(driver: DriverInfo) { - val driverFile = new File(dir + File.separator + "driver_" + driver.id) - serializeIntoFile(driverFile, driver) + override def persist(name: String, obj: Object): Unit = { + serializeIntoFile(new File(dir + File.separator + name), obj) } - override def removeDriver(driver: DriverInfo) { - new File(dir + File.separator + "driver_" + driver.id).delete() + override def unpersist(name: String): Unit = { + new File(dir + File.separator + name).delete() } - override def addWorker(worker: WorkerInfo) { - val workerFile = new File(dir + File.separator + "worker_" + worker.id) - serializeIntoFile(workerFile, worker) - } - - override def removeWorker(worker: WorkerInfo) { - new File(dir + File.separator + "worker_" + worker.id).delete() - } - - override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = { - val sortedFiles = new File(dir).listFiles().sortBy(_.getName) - val appFiles = sortedFiles.filter(_.getName.startsWith("app_")) - val apps = appFiles.map(deserializeFromFile[ApplicationInfo]) - val driverFiles = sortedFiles.filter(_.getName.startsWith("driver_")) - val drivers = driverFiles.map(deserializeFromFile[DriverInfo]) - val workerFiles = sortedFiles.filter(_.getName.startsWith("worker_")) - val workers = workerFiles.map(deserializeFromFile[WorkerInfo]) - (apps, drivers, workers) + override def read[T: ClassTag](prefix: String) = { + val files = new File(dir).listFiles().filter(_.getName.startsWith(prefix)) + files.map(deserializeFromFile[T]) } private def serializeIntoFile(file: File, value: AnyRef) { val created = file.createNewFile() if (!created) { throw new IllegalStateException("Could not create file: " + file) } - val serializer = serialization.findSerializerFor(value) - val serialized = serializer.toBinary(value) - - val out = new FileOutputStream(file) + val out = serializer.serializeStream(new FileOutputStream(file)) try { - out.write(serialized) + out.writeObject(value) } finally { out.close() } + } - def deserializeFromFile[T](file: File)(implicit m: Manifest[T]): T = { + def deserializeFromFile[T](file: File): T = { val fileData = new Array[Byte](file.length().asInstanceOf[Int]) val dis = new DataInputStream(new FileInputStream(file)) try { @@ -99,8 +75,6 @@ private[spark] class FileSystemPersistenceEngine( dis.close() } - val clazz = m.runtimeClass.asInstanceOf[Class[T]] - val serializer = serialization.serializerFor(clazz) - serializer.fromBinary(fileData).asInstanceOf[T] + serializer.deserializeStream(dis).readObject() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala index 4433a2ec29be6..cf77c86d760cf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -17,30 +17,27 @@ package org.apache.spark.deploy.master -import akka.actor.{Actor, ActorRef} - -import org.apache.spark.deploy.master.MasterMessages.ElectedLeader +import org.apache.spark.annotation.DeveloperApi /** - * A LeaderElectionAgent keeps track of whether the current Master is the leader, meaning it - * is the only Master serving requests. - * In addition to the API provided, the LeaderElectionAgent will use of the following messages - * to inform the Master of leader changes: - * [[org.apache.spark.deploy.master.MasterMessages.ElectedLeader ElectedLeader]] - * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]] + * :: DeveloperApi :: + * + * A LeaderElectionAgent tracks current master and is a common interface for all election Agents. */ -private[spark] trait LeaderElectionAgent extends Actor { - // TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring. - val masterActor: ActorRef +@DeveloperApi +trait LeaderElectionAgent { + val masterActor: LeaderElectable + def stop() {} // to avoid noops in implementations. } -/** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */ -private[spark] class MonarchyLeaderAgent(val masterActor: ActorRef) extends LeaderElectionAgent { - override def preStart() { - masterActor ! ElectedLeader - } +@DeveloperApi +trait LeaderElectable { + def electedLeader() + def revokedLeadership() +} - override def receive = { - case _ => - } +/** Single-node implementation of LeaderElectionAgent -- we're initially and always the leader. */ +private[spark] class MonarchyLeaderAgent(val masterActor: LeaderElectable) + extends LeaderElectionAgent { + masterActor.electedLeader() } 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 2f81d472d7b78..021454e25804c 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 @@ -50,7 +50,7 @@ private[spark] class Master( port: Int, webUiPort: Int, val securityMgr: SecurityManager) - extends Actor with ActorLogReceive with Logging { + extends Actor with ActorLogReceive with Logging with LeaderElectable { import context.dispatcher // to use Akka's scheduler.schedule() @@ -61,7 +61,6 @@ private[spark] class Master( 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") val workers = new HashSet[WorkerInfo] @@ -103,7 +102,7 @@ private[spark] class Master( var persistenceEngine: PersistenceEngine = _ - var leaderElectionAgent: ActorRef = _ + var leaderElectionAgent: LeaderElectionAgent = _ private var recoveryCompletionTask: Cancellable = _ @@ -130,23 +129,24 @@ private[spark] class Master( masterMetricsSystem.start() applicationMetricsSystem.start() - persistenceEngine = RECOVERY_MODE match { + val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") - new ZooKeeperPersistenceEngine(SerializationExtension(context.system), conf) + val zkFactory = new ZooKeeperRecoveryModeFactory(conf) + (zkFactory.createPersistenceEngine(), zkFactory.createLeaderElectionAgent(this)) case "FILESYSTEM" => - logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) - new FileSystemPersistenceEngine(RECOVERY_DIR, SerializationExtension(context.system)) + val fsFactory = new FileSystemRecoveryModeFactory(conf) + (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) + case "CUSTOM" => + val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory")) + val factory = clazz.getConstructor(conf.getClass) + .newInstance(conf).asInstanceOf[StandaloneRecoveryModeFactory] + (factory.createPersistenceEngine(), factory.createLeaderElectionAgent(this)) case _ => - new BlackHolePersistenceEngine() + (new BlackHolePersistenceEngine(), new MonarchyLeaderAgent(this)) } - - leaderElectionAgent = RECOVERY_MODE match { - case "ZOOKEEPER" => - context.actorOf(Props(classOf[ZooKeeperLeaderElectionAgent], self, masterUrl, conf)) - case _ => - context.actorOf(Props(classOf[MonarchyLeaderAgent], self)) - } + persistenceEngine = persistenceEngine_ + leaderElectionAgent = leaderElectionAgent_ } override def preRestart(reason: Throwable, message: Option[Any]) { @@ -165,7 +165,15 @@ private[spark] class Master( masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() - context.stop(leaderElectionAgent) + leaderElectionAgent.stop() + } + + override def electedLeader() { + self ! ElectedLeader + } + + override def revokedLeadership() { + self ! RevokedLeadership } override def receiveWithLogging = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala index e3640ea4f7e64..2e0e1e7036ac8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/PersistenceEngine.scala @@ -17,6 +17,10 @@ package org.apache.spark.deploy.master +import org.apache.spark.annotation.DeveloperApi + +import scala.reflect.ClassTag + /** * Allows Master to persist any state that is necessary in order to recover from a failure. * The following semantics are required: @@ -25,36 +29,70 @@ package org.apache.spark.deploy.master * Given these two requirements, we will have all apps and workers persisted, but * we might not have yet deleted apps or workers that finished (so their liveness must be verified * during recovery). + * + * The implementation of this trait defines how name-object pairs are stored or retrieved. */ -private[spark] trait PersistenceEngine { - def addApplication(app: ApplicationInfo) +@DeveloperApi +trait PersistenceEngine { - def removeApplication(app: ApplicationInfo) + /** + * Defines how the object is serialized and persisted. Implementation will + * depend on the store used. + */ + def persist(name: String, obj: Object) - def addWorker(worker: WorkerInfo) + /** + * Defines how the object referred by its name is removed from the store. + */ + def unpersist(name: String) - def removeWorker(worker: WorkerInfo) + /** + * Gives all objects, matching a prefix. This defines how objects are + * read/deserialized back. + */ + def read[T: ClassTag](prefix: String): Seq[T] - def addDriver(driver: DriverInfo) + final def addApplication(app: ApplicationInfo): Unit = { + persist("app_" + app.id, app) + } - def removeDriver(driver: DriverInfo) + final def removeApplication(app: ApplicationInfo): Unit = { + unpersist("app_" + app.id) + } + + final def addWorker(worker: WorkerInfo): Unit = { + persist("worker_" + worker.id, worker) + } + + final def removeWorker(worker: WorkerInfo): Unit = { + unpersist("worker_" + worker.id) + } + + final def addDriver(driver: DriverInfo): Unit = { + persist("driver_" + driver.id, driver) + } + + final def removeDriver(driver: DriverInfo): Unit = { + unpersist("driver_" + driver.id) + } /** * Returns the persisted data sorted by their respective ids (which implies that they're * sorted by time of creation). */ - def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) + final def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = { + (read[ApplicationInfo]("app_"), read[DriverInfo]("driver_"), read[WorkerInfo]("worker_")) + } def close() {} } private[spark] class BlackHolePersistenceEngine extends PersistenceEngine { - override def addApplication(app: ApplicationInfo) {} - override def removeApplication(app: ApplicationInfo) {} - override def addWorker(worker: WorkerInfo) {} - override def removeWorker(worker: WorkerInfo) {} - override def addDriver(driver: DriverInfo) {} - override def removeDriver(driver: DriverInfo) {} - - override def readPersistedData() = (Nil, Nil, Nil) + + override def persist(name: String, obj: Object): Unit = {} + + override def unpersist(name: String): Unit = {} + + override def read[T: ClassTag](name: String): Seq[T] = Nil + } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala new file mode 100644 index 0000000000000..d9d36c1ed5f9f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.master + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.serializer.JavaSerializer + +/** + * ::DeveloperApi:: + * + * Implementation of this class can be plugged in as recovery mode alternative for Spark's + * Standalone mode. + * + */ +@DeveloperApi +abstract class StandaloneRecoveryModeFactory(conf: SparkConf) { + + /** + * PersistenceEngine defines how the persistent data(Information about worker, driver etc..) + * is handled for recovery. + * + */ + def createPersistenceEngine(): PersistenceEngine + + /** + * Create an instance of LeaderAgent that decides who gets elected as master. + */ + def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent +} + +/** + * LeaderAgent in this case is a no-op. Since leader is forever leader as the actual + * recovery is made by restoring from filesystem. + */ +private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf) + extends StandaloneRecoveryModeFactory(conf) with Logging { + val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") + + def createPersistenceEngine() = { + logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) + new FileSystemPersistenceEngine(RECOVERY_DIR, new JavaSerializer(conf)) + } + + def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) +} + +private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf) + extends StandaloneRecoveryModeFactory(conf) { + def createPersistenceEngine() = new ZooKeeperPersistenceEngine(new JavaSerializer(conf), conf) + + def createLeaderElectionAgent(master: LeaderElectable) = + new ZooKeeperLeaderElectionAgent(master, conf) +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index d221b0f6cc86b..473ddc23ff0f3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -21,6 +21,7 @@ import scala.collection.mutable import akka.actor.ActorRef +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils private[spark] class WorkerInfo( diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 285f9b014e291..8eaa0ad948519 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -24,9 +24,8 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.curator.framework.CuratorFramework import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} -private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, - masterUrl: String, conf: SparkConf) - extends LeaderElectionAgent with LeaderLatchListener with Logging { +private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: LeaderElectable, + conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" @@ -34,30 +33,21 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, private var leaderLatch: LeaderLatch = _ private var status = LeadershipStatus.NOT_LEADER - override def preStart() { + start() + def start() { logInfo("Starting ZooKeeper LeaderElection agent") zk = SparkCuratorUtil.newClient(conf) leaderLatch = new LeaderLatch(zk, WORKING_DIR) leaderLatch.addListener(this) - leaderLatch.start() } - override def preRestart(reason: scala.Throwable, message: scala.Option[scala.Any]) { - logError("LeaderElectionAgent failed...", reason) - super.preRestart(reason, message) - } - - override def postStop() { + override def stop() { leaderLatch.close() zk.close() } - override def receive = { - case _ => - } - override def isLeader() { synchronized { // could have lost leadership by now. @@ -85,10 +75,10 @@ private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, def updateLeadershipStatus(isLeader: Boolean) { if (isLeader && status == LeadershipStatus.NOT_LEADER) { status = LeadershipStatus.LEADER - masterActor ! ElectedLeader + masterActor.electedLeader() } else if (!isLeader && status == LeadershipStatus.LEADER) { status = LeadershipStatus.NOT_LEADER - masterActor ! RevokedLeadership + masterActor.revokedLeadership() } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 834dfedee52ce..96c2139eb02f0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -19,72 +19,54 @@ package org.apache.spark.deploy.master import scala.collection.JavaConversions._ -import akka.serialization.Serialization import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.serializer.Serializer +import java.nio.ByteBuffer -class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) +import scala.reflect.ClassTag + + +private[spark] class ZooKeeperPersistenceEngine(val serialization: Serializer, conf: SparkConf) extends PersistenceEngine with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) - SparkCuratorUtil.mkdir(zk, WORKING_DIR) - - override def addApplication(app: ApplicationInfo) { - serializeIntoFile(WORKING_DIR + "/app_" + app.id, app) - } + val serializer = serialization.newInstance() - override def removeApplication(app: ApplicationInfo) { - zk.delete().forPath(WORKING_DIR + "/app_" + app.id) - } + SparkCuratorUtil.mkdir(zk, WORKING_DIR) - override def addDriver(driver: DriverInfo) { - serializeIntoFile(WORKING_DIR + "/driver_" + driver.id, driver) - } - override def removeDriver(driver: DriverInfo) { - zk.delete().forPath(WORKING_DIR + "/driver_" + driver.id) + override def persist(name: String, obj: Object): Unit = { + serializeIntoFile(WORKING_DIR + "/" + name, obj) } - override def addWorker(worker: WorkerInfo) { - serializeIntoFile(WORKING_DIR + "/worker_" + worker.id, worker) + override def unpersist(name: String): Unit = { + zk.delete().forPath(WORKING_DIR + "/" + name) } - override def removeWorker(worker: WorkerInfo) { - zk.delete().forPath(WORKING_DIR + "/worker_" + worker.id) + override def read[T: ClassTag](prefix: String) = { + val file = zk.getChildren.forPath(WORKING_DIR).filter(_.startsWith(prefix)) + file.map(deserializeFromFile[T]).flatten } override def close() { zk.close() } - override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = { - val sortedFiles = zk.getChildren().forPath(WORKING_DIR).toList.sorted - val appFiles = sortedFiles.filter(_.startsWith("app_")) - val apps = appFiles.map(deserializeFromFile[ApplicationInfo]).flatten - val driverFiles = sortedFiles.filter(_.startsWith("driver_")) - val drivers = driverFiles.map(deserializeFromFile[DriverInfo]).flatten - val workerFiles = sortedFiles.filter(_.startsWith("worker_")) - val workers = workerFiles.map(deserializeFromFile[WorkerInfo]).flatten - (apps, drivers, workers) - } - private def serializeIntoFile(path: String, value: AnyRef) { - val serializer = serialization.findSerializerFor(value) - val serialized = serializer.toBinary(value) - zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) + val serialized = serializer.serialize(value) + zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized.array()) } - def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): Option[T] = { + def deserializeFromFile[T](filename: String): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) - val clazz = m.runtimeClass.asInstanceOf[Class[T]] - val serializer = serialization.serializerFor(clazz) try { - Some(serializer.fromBinary(fileData).asInstanceOf[T]) + Some(serializer.deserialize(ByteBuffer.wrap(fileData))) } catch { case e: Exception => { logWarning("Exception while reading persisted file, deleting", e) From f820b563d88f6a972c219d9340fe95110493fb87 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 11 Nov 2014 12:30:35 -0600 Subject: [PATCH 1294/1492] SPARK-4305 [BUILD] yarn-alpha profile won't build due to network/yarn module SPARK-3797 introduced the `network/yarn` module, but its YARN code depends on YARN APIs not present in older versions covered by the `yarn-alpha` profile. As a result builds like `mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package` fail. The solution is just to not build `network/yarn` with profile `yarn-alpha`. Author: Sean Owen Closes #3167 from srowen/SPARK-4305 and squashes the following commits: 88938cb [Sean Owen] Don't build network/yarn in yarn-alpha profile as it won't compile --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index 88ef67c515b3a..4e0cd6c151d0b 100644 --- a/pom.xml +++ b/pom.xml @@ -1229,7 +1229,6 @@ yarn-alpha yarn - network/yarn From 7f3718842cc4025bb2ee2f5a3ec12efd100f6589 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 11 Nov 2014 12:33:53 -0600 Subject: [PATCH 1295/1492] [SPARK-4282][YARN] Stopping flag in YarnClientSchedulerBackend should be volatile In YarnClientSchedulerBackend, a variable "stopping" is used as a flag and it's accessed by some threads so it should be volatile. Author: Kousuke Saruta Closes #3143 from sarutak/stopping-flag-volatile and squashes the following commits: 58fdcc9 [Kousuke Saruta] Marked stoppig flag as volatile --- .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f6f6dc52433e5..2923e6729cd6b 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 @@ -33,7 +33,7 @@ private[spark] class YarnClientSchedulerBackend( private var client: Client = null private var appId: ApplicationId = null - private var stopping: Boolean = false + @volatile private var stopping: Boolean = false /** * Create a Yarn client to submit an application to the ResourceManager. From a878660d2d7bb7ad9b5818a674e1e7c651077e78 Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 11 Nov 2014 14:29:18 -0800 Subject: [PATCH 1296/1492] SPARK-2269 Refactor mesos scheduler resourceOffers and add unit test Author: Timothy Chen Closes #1487 from tnachen/resource_offer_refactor and squashes the following commits: 4ea5dec [Timothy Chen] Rebase from master and address comments 9ccab09 [Timothy Chen] Address review comments e6494dc [Timothy Chen] Refactor class loading 8207428 [Timothy Chen] Refactor mesos scheduler resourceOffers and add unit test --- .../cluster/mesos/MesosSchedulerBackend.scala | 137 ++++++++---------- .../mesos/MesosSchedulerBackendSuite.scala | 94 ++++++++++++ 2 files changed, 152 insertions(+), 79 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c5f3493477bc5..d13795186c48e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -166,29 +166,16 @@ private[spark] class MesosSchedulerBackend( execArgs } - private def setClassLoader(): ClassLoader = { - val oldClassLoader = Thread.currentThread.getContextClassLoader - Thread.currentThread.setContextClassLoader(classLoader) - oldClassLoader - } - - private def restoreClassLoader(oldClassLoader: ClassLoader) { - Thread.currentThread.setContextClassLoader(oldClassLoader) - } - override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - val oldClassLoader = setClassLoader() - try { + inClassLoader() { appId = frameworkId.getValue logInfo("Registered as framework ID " + appId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() } - } finally { - restoreClassLoader(oldClassLoader) } } @@ -200,6 +187,16 @@ private[spark] class MesosSchedulerBackend( } } + private def inClassLoader()(fun: => Unit) = { + val oldClassLoader = Thread.currentThread.getContextClassLoader + Thread.currentThread.setContextClassLoader(classLoader) + try { + fun + } finally { + Thread.currentThread.setContextClassLoader(oldClassLoader) + } + } + override def disconnected(d: SchedulerDriver) {} override def reregistered(d: SchedulerDriver, masterInfo: MasterInfo) {} @@ -210,66 +207,57 @@ private[spark] class MesosSchedulerBackend( * tasks are balanced across the cluster. */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { - val oldClassLoader = setClassLoader() - try { - synchronized { - // Build a big list of the offerable workers, and remember their indices so that we can - // figure out which Offer to reply to for each worker - val offerableWorkers = new ArrayBuffer[WorkerOffer] - val offerableIndices = new HashMap[String, Int] - - def sufficientOffer(o: Offer) = { - val mem = getResource(o.getResourcesList, "mem") - val cpus = getResource(o.getResourcesList, "cpus") - val slaveId = o.getSlaveId.getValue - (mem >= MemoryUtils.calculateTotalMemory(sc) && - // need at least 1 for executor, 1 for task - cpus >= 2 * scheduler.CPUS_PER_TASK) || - (slaveIdsWithExecutors.contains(slaveId) && - cpus >= scheduler.CPUS_PER_TASK) - } + inClassLoader() { + val (acceptedOffers, declinedOffers) = offers.partition { o => + val mem = getResource(o.getResourcesList, "mem") + val cpus = getResource(o.getResourcesList, "cpus") + val slaveId = o.getSlaveId.getValue + (mem >= MemoryUtils.calculateTotalMemory(sc) && + // need at least 1 for executor, 1 for task + cpus >= 2 * scheduler.CPUS_PER_TASK) || + (slaveIdsWithExecutors.contains(slaveId) && + cpus >= scheduler.CPUS_PER_TASK) + } - for ((offer, index) <- offers.zipWithIndex if sufficientOffer(offer)) { - val slaveId = offer.getSlaveId.getValue - offerableIndices.put(slaveId, index) - val cpus = if (slaveIdsWithExecutors.contains(slaveId)) { - getResource(offer.getResourcesList, "cpus").toInt - } else { - // If the executor doesn't exist yet, subtract CPU for executor - getResource(offer.getResourcesList, "cpus").toInt - - scheduler.CPUS_PER_TASK - } - offerableWorkers += new WorkerOffer( - offer.getSlaveId.getValue, - offer.getHostname, - cpus) + val offerableWorkers = acceptedOffers.map { o => + val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { + getResource(o.getResourcesList, "cpus").toInt + } else { + // If the executor doesn't exist yet, subtract CPU for executor + getResource(o.getResourcesList, "cpus").toInt - + scheduler.CPUS_PER_TASK } + new WorkerOffer( + o.getSlaveId.getValue, + o.getHostname, + cpus) + } - // Call into the TaskSchedulerImpl - val taskLists = scheduler.resourceOffers(offerableWorkers) - - // Build a list of Mesos tasks for each slave - val mesosTasks = offers.map(o => new JArrayList[MesosTaskInfo]()) - for ((taskList, index) <- taskLists.zipWithIndex) { - if (!taskList.isEmpty) { - for (taskDesc <- taskList) { - val slaveId = taskDesc.executorId - val offerNum = offerableIndices(slaveId) - slaveIdsWithExecutors += slaveId - taskIdToSlaveId(taskDesc.taskId) = slaveId - mesosTasks(offerNum).add(createMesosTask(taskDesc, slaveId)) - } + val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap + + val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] + + // Call into the TaskSchedulerImpl + scheduler.resourceOffers(offerableWorkers) + .filter(!_.isEmpty) + .foreach { offer => + offer.foreach { taskDesc => + val slaveId = taskDesc.executorId + slaveIdsWithExecutors += slaveId + taskIdToSlaveId(taskDesc.taskId) = slaveId + mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) + .add(createMesosTask(taskDesc, slaveId)) } } - // Reply to the offers - val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? - for (i <- 0 until offers.size) { - d.launchTasks(Collections.singleton(offers(i).getId), mesosTasks(i), filters) - } + // Reply to the offers + val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? + + mesosTasks.foreach { case (slaveId, tasks) => + d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - } finally { - restoreClassLoader(oldClassLoader) + + declinedOffers.foreach(o => d.declineOffer(o.getId)) } } @@ -308,8 +296,7 @@ private[spark] class MesosSchedulerBackend( } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { - val oldClassLoader = setClassLoader() - try { + inClassLoader() { val tid = status.getTaskId.getValue.toLong val state = TaskState.fromMesos(status.getState) synchronized { @@ -322,18 +309,13 @@ private[spark] class MesosSchedulerBackend( } } scheduler.statusUpdate(tid, state, status.getData.asReadOnlyByteBuffer) - } finally { - restoreClassLoader(oldClassLoader) } } override def error(d: SchedulerDriver, message: String) { - val oldClassLoader = setClassLoader() - try { + inClassLoader() { logError("Mesos error: " + message) scheduler.error(message) - } finally { - restoreClassLoader(oldClassLoader) } } @@ -350,15 +332,12 @@ private[spark] class MesosSchedulerBackend( override def frameworkMessage(d: SchedulerDriver, e: ExecutorID, s: SlaveID, b: Array[Byte]) {} private def recordSlaveLost(d: SchedulerDriver, slaveId: SlaveID, reason: ExecutorLossReason) { - val oldClassLoader = setClassLoader() - try { + inClassLoader() { logInfo("Mesos slave lost: " + slaveId.getValue) synchronized { slaveIdsWithExecutors -= slaveId.getValue } scheduler.executorLost(slaveId.getValue, reason) - } finally { - restoreClassLoader(oldClassLoader) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala new file mode 100644 index 0000000000000..bef8d3a58ba63 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -0,0 +1,94 @@ +/* + * 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.mesos + +import org.scalatest.FunSuite +import org.apache.spark.{scheduler, SparkConf, SparkContext, LocalSparkContext} +import org.apache.spark.scheduler.{TaskDescription, WorkerOffer, TaskSchedulerImpl} +import org.apache.spark.scheduler.cluster.mesos.{MemoryUtils, MesosSchedulerBackend} +import org.apache.mesos.SchedulerDriver +import org.apache.mesos.Protos._ +import org.scalatest.mock.EasyMockSugar +import org.apache.mesos.Protos.Value.Scalar +import org.easymock.{Capture, EasyMock} +import java.nio.ByteBuffer +import java.util.Collections +import java.util +import scala.collection.mutable + +class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { + test("mesos resource offer is launching tasks") { + def createOffer(id: Int, mem: Int, cpu: Int) = { + val builder = Offer.newBuilder() + builder.addResourcesBuilder() + .setName("mem") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(mem)) + builder.addResourcesBuilder() + .setName("cpus") + .setType(Value.Type.SCALAR) + .setScalar(Scalar.newBuilder().setValue(cpu)) + builder.setId(OfferID.newBuilder().setValue(id.toString).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue("s1")).setHostname("localhost").build() + } + + val driver = EasyMock.createMock(classOf[SchedulerDriver]) + val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) + + val sc = EasyMock.createMock(classOf[SparkContext]) + + EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() + EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() + EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() + EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() + EasyMock.replay(sc) + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + val minCpu = 4 + val offers = new java.util.ArrayList[Offer] + offers.add(createOffer(1, minMem, minCpu)) + offers.add(createOffer(1, minMem - 1, minCpu)) + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + val workerOffers = Seq(offers.get(0)).map(o => new WorkerOffer( + o.getSlaveId.getValue, + o.getHostname, + 2 + )) + val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(workerOffers))).andReturn(Seq(Seq(taskDesc))) + EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() + EasyMock.replay(taskScheduler) + val capture = new Capture[util.Collection[TaskInfo]] + EasyMock.expect( + driver.launchTasks( + EasyMock.eq(Collections.singleton(offers.get(0).getId)), + EasyMock.capture(capture), + EasyMock.anyObject(classOf[Filters]) + ) + ).andReturn(Status.valueOf(1)) + EasyMock.expect(driver.declineOffer(offers.get(1).getId)).andReturn(Status.valueOf(1)) + EasyMock.replay(driver) + backend.resourceOffers(driver, offers) + assert(capture.getValue.size() == 1) + val taskInfo = capture.getValue.iterator().next() + assert(taskInfo.getName.equals("n1")) + val cpus = taskInfo.getResourcesList.get(0) + assert(cpus.getName.equals("cpus")) + assert(cpus.getScalar.getValue.equals(2.0)) + assert(taskInfo.getSlaveId.getValue.equals("s1")) + } +} From 2ddb1415e2bea94004947506ded090c2e8ff8dad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 11 Nov 2014 18:02:59 -0800 Subject: [PATCH 1297/1492] [Release] Log build output for each distribution --- dev/create-release/create-release.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 281e8d4de6d71..50a9a2fa1cb9a 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -27,6 +27,7 @@ # Would be nice to add: # - Send output to stderr and have useful logging in stdout +# Note: The following variables must be set before use! GIT_USERNAME=${GIT_USERNAME:-pwendell} GIT_PASSWORD=${GIT_PASSWORD:-XXX} GPG_PASSPHRASE=${GPG_PASSPHRASE:-XXX} @@ -101,7 +102,7 @@ make_binary_release() { cp -r spark spark-$RELEASE_VERSION-bin-$NAME cd spark-$RELEASE_VERSION-bin-$NAME - ./make-distribution.sh --name $NAME --tgz $FLAGS + ./make-distribution.sh --name $NAME --tgz $FLAGS 2>&1 | tee binary-release-$NAME.log cd .. cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . rm -rf spark-$RELEASE_VERSION-bin-$NAME From daaca14c16dc2c1abc98f15ab8c6f7c14761b627 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 11 Nov 2014 21:36:48 -0800 Subject: [PATCH 1298/1492] Support cross building for Scala 2.11 Let's give this another go using a version of Hive that shades its JLine dependency. Author: Prashant Sharma Author: Patrick Wendell Closes #3159 from pwendell/scala-2.11-prashant and squashes the following commits: e93aa3e [Patrick Wendell] Restoring -Phive-thriftserver profile and cleaning up build script. f65d17d [Patrick Wendell] Fixing build issue due to merge conflict a8c41eb [Patrick Wendell] Reverting dev/run-tests back to master state. 7a6eb18 [Patrick Wendell] Merge remote-tracking branch 'apache/master' into scala-2.11-prashant 583aa07 [Prashant Sharma] REVERT ME: removed hive thirftserver 3680e58 [Prashant Sharma] Revert "REVERT ME: Temporarily removing some Cli tests." 935fb47 [Prashant Sharma] Revert "Fixed by disabling a few tests temporarily." 925e90f [Prashant Sharma] Fixed by disabling a few tests temporarily. 2fffed3 [Prashant Sharma] Exclude groovy from sbt build, and also provide a way for such instances in future. 8bd4e40 [Prashant Sharma] Switched to gmaven plus, it fixes random failures observer with its predecessor gmaven. 5272ce5 [Prashant Sharma] SPARK_SCALA_VERSION related bugs. 2121071 [Patrick Wendell] Migrating version detection to PySpark b1ed44d [Patrick Wendell] REVERT ME: Temporarily removing some Cli tests. 1743a73 [Patrick Wendell] Removing decimal test that doesn't work with Scala 2.11 f5cad4e [Patrick Wendell] Add Scala 2.11 docs 210d7e1 [Patrick Wendell] Revert "Testing new Hive version with shaded jline" 48518ce [Patrick Wendell] Remove association of Hive and Thriftserver profiles. e9d0a06 [Patrick Wendell] Revert "Enable thritfserver for Scala 2.10 only" 67ec364 [Patrick Wendell] Guard building of thriftserver around Scala 2.10 check 8502c23 [Patrick Wendell] Enable thritfserver for Scala 2.10 only e22b104 [Patrick Wendell] Small fix in pom file ec402ab [Patrick Wendell] Various fixes 0be5a9d [Patrick Wendell] Testing new Hive version with shaded jline 4eaec65 [Prashant Sharma] Changed scripts to ignore target. 5167bea [Prashant Sharma] small correction a4fcac6 [Prashant Sharma] Run against scala 2.11 on jenkins. 80285f4 [Prashant Sharma] MAven equivalent of setting spark.executor.extraClasspath during tests. 034b369 [Prashant Sharma] Setting test jars on executor classpath during tests from sbt. d4874cb [Prashant Sharma] Fixed Python Runner suite. null check should be first case in scala 2.11. 6f50f13 [Prashant Sharma] Fixed build after rebasing with master. We should use ${scala.binary.version} instead of just 2.10 e56ca9d [Prashant Sharma] Print an error if build for 2.10 and 2.11 is spotted. 937c0b8 [Prashant Sharma] SCALA_VERSION -> SPARK_SCALA_VERSION cb059b0 [Prashant Sharma] Code review 0476e5e [Prashant Sharma] Scala 2.11 support with repl and all build changes. --- .rat-excludes | 1 + assembly/pom.xml | 13 +- bin/compute-classpath.sh | 46 +- bin/load-spark-env.sh | 20 + bin/pyspark | 6 +- bin/run-example | 8 +- bin/spark-class | 8 +- core/pom.xml | 57 +- .../apache/spark/deploy/PythonRunner.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- dev/change-version-to-2.10.sh | 20 + dev/change-version-to-2.11.sh | 21 + dev/create-release/create-release.sh | 12 +- dev/run-tests | 13 +- dev/scalastyle | 2 +- docs/building-spark.md | 31 +- docs/sql-programming-guide.md | 2 +- examples/pom.xml | 199 ++- .../streaming/JavaKafkaWordCount.java | 0 .../examples/streaming/KafkaWordCount.scala | 0 .../streaming/TwitterAlgebirdCMS.scala | 0 .../streaming/TwitterAlgebirdHLL.scala | 0 external/mqtt/pom.xml | 5 - make-distribution.sh | 2 +- network/shuffle/pom.xml | 4 +- network/yarn/pom.xml | 2 +- pom.xml | 178 ++- project/SparkBuild.scala | 36 +- project/project/SparkPluginBuild.scala | 2 +- repl/pom.xml | 90 +- .../scala/org/apache/spark/repl/Main.scala | 0 .../apache/spark/repl/SparkCommandLine.scala | 0 .../apache/spark/repl/SparkExprTyper.scala | 0 .../org/apache/spark/repl/SparkHelper.scala | 0 .../org/apache/spark/repl/SparkILoop.scala | 0 .../apache/spark/repl/SparkILoopInit.scala | 0 .../org/apache/spark/repl/SparkIMain.scala | 0 .../org/apache/spark/repl/SparkImports.scala | 0 .../spark/repl/SparkJLineCompletion.scala | 0 .../apache/spark/repl/SparkJLineReader.scala | 0 .../spark/repl/SparkMemberHandlers.scala | 0 .../spark/repl/SparkRunnerSettings.scala | 0 .../org/apache/spark/repl/ReplSuite.scala | 0 .../scala/org/apache/spark/repl/Main.scala | 85 ++ .../apache/spark/repl/SparkExprTyper.scala | 86 ++ .../org/apache/spark/repl/SparkILoop.scala | 966 ++++++++++++ .../org/apache/spark/repl/SparkIMain.scala | 1319 +++++++++++++++++ .../org/apache/spark/repl/SparkImports.scala | 201 +++ .../spark/repl/SparkJLineCompletion.scala | 350 +++++ .../spark/repl/SparkMemberHandlers.scala | 221 +++ .../apache/spark/repl/SparkReplReporter.scala | 53 + .../org/apache/spark/repl/ReplSuite.scala | 326 ++++ sql/catalyst/pom.xml | 29 +- .../catalyst/types/decimal/DecimalSuite.scala | 1 - 54 files changed, 4204 insertions(+), 215 deletions(-) create mode 100755 dev/change-version-to-2.10.sh create mode 100755 dev/change-version-to-2.11.sh rename examples/{ => scala-2.10}/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java (100%) rename examples/{ => scala-2.10}/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala (100%) rename examples/{ => scala-2.10}/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala (100%) rename examples/{ => scala-2.10}/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/Main.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkHelper.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkILoop.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkIMain.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkImports.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala (100%) rename repl/{ => scala-2.10}/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala (100%) rename repl/{ => scala-2.10}/src/test/scala/org/apache/spark/repl/ReplSuite.scala (100%) create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkImports.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala create mode 100644 repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkReplReporter.scala create mode 100644 repl/scala-2.11/src/test/scala/org/apache/spark/repl/ReplSuite.scala diff --git a/.rat-excludes b/.rat-excludes index 20e3372464386..d8bee1f8e49c9 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -44,6 +44,7 @@ SparkImports.scala SparkJLineCompletion.scala SparkJLineReader.scala SparkMemberHandlers.scala +SparkReplReporter.scala sbt sbt-launch-lib.bash plugins.sbt diff --git a/assembly/pom.xml b/assembly/pom.xml index 31a01e4d8e1de..c65192bde64c6 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -66,22 +66,22 @@ org.apache.spark - spark-repl_${scala.binary.version} + spark-streaming_${scala.binary.version} ${project.version} org.apache.spark - spark-streaming_${scala.binary.version} + spark-graphx_${scala.binary.version} ${project.version} org.apache.spark - spark-graphx_${scala.binary.version} + spark-sql_${scala.binary.version} ${project.version} org.apache.spark - spark-sql_${scala.binary.version} + spark-repl_${scala.binary.version} ${project.version} @@ -197,6 +197,11 @@ spark-hive_${scala.binary.version} ${project.version} + + + + hive-thriftserver + org.apache.spark spark-hive-thriftserver_${scala.binary.version} diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 905bbaf99b374..298641f2684de 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -20,8 +20,6 @@ # This script computes Spark's classpath and prints it to stdout; it's used by both the "run" # script and the ExecutorRunner in standalone cluster mode. -SCALA_VERSION=2.10 - # Figure out where Spark is installed FWDIR="$(cd "`dirname "$0"`"/..; pwd)" @@ -36,7 +34,7 @@ else CLASSPATH="$CLASSPATH:$FWDIR/conf" fi -ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" +ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SPARK_SCALA_VERSION" if [ -n "$JAVA_HOME" ]; then JAR_CMD="$JAVA_HOME/bin/jar" @@ -48,19 +46,19 @@ fi if [ -n "$SPARK_PREPEND_CLASSES" ]; then echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ "classes ahead of assembly." >&2 - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*" - CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/tools/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SCALA_VERSION/classes" - CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/tools/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive-thriftserver/target/scala-$SPARK_SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/yarn/stable/target/scala-$SPARK_SCALA_VERSION/classes" fi # Use spark-assembly jar from either RELEASE or assembly directory @@ -123,15 +121,15 @@ fi # Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 if [[ $SPARK_TESTING == 1 ]]; then - CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/test-classes" - CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/graphx/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/streaming/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/catalyst/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SPARK_SCALA_VERSION/test-classes" + CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SPARK_SCALA_VERSION/test-classes" fi # Add hadoop conf dir if given -- otherwise FileSystem.*, etc fail ! diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 6d4231b204595..356b3d49b2ffe 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -36,3 +36,23 @@ if [ -z "$SPARK_ENV_LOADED" ]; then set +a fi fi + +# Setting SPARK_SCALA_VERSION if not already set. + +if [ -z "$SPARK_SCALA_VERSION" ]; then + + ASSEMBLY_DIR2="$FWDIR/assembly/target/scala-2.11" + ASSEMBLY_DIR1="$FWDIR/assembly/target/scala-2.10" + + if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then + echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 + echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION=2.11 in spark-env.sh.' 1>&2 + exit 1 + fi + + if [ -d "$ASSEMBLY_DIR2" ]; then + export SPARK_SCALA_VERSION="2.11" + else + export SPARK_SCALA_VERSION="2.10" + fi +fi diff --git a/bin/pyspark b/bin/pyspark index 96f30a260a09e..1d8c94d43d285 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -25,7 +25,7 @@ export SPARK_HOME="$FWDIR" source "$FWDIR/bin/utils.sh" -SCALA_VERSION=2.10 +source "$FWDIR"/bin/load-spark-env.sh function usage() { echo "Usage: ./bin/pyspark [options]" 1>&2 @@ -40,7 +40,7 @@ fi # Exit if the user hasn't compiled Spark if [ ! -f "$FWDIR/RELEASE" ]; then # Exit if the user hasn't compiled Spark - ls "$FWDIR"/assembly/target/scala-$SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null + ls "$FWDIR"/assembly/target/scala-$SPARK_SCALA_VERSION/spark-assembly*hadoop*.jar >& /dev/null if [[ $? != 0 ]]; then echo "Failed to find Spark assembly in $FWDIR/assembly/target" 1>&2 echo "You need to build Spark before running this program" 1>&2 @@ -48,8 +48,6 @@ if [ ! -f "$FWDIR/RELEASE" ]; then fi fi -. "$FWDIR"/bin/load-spark-env.sh - # In Spark <= 1.1, setting IPYTHON=1 would cause the driver to be launched using the `ipython` # executable, while the worker would still be launched using PYSPARK_PYTHON. # diff --git a/bin/run-example b/bin/run-example index 34dd71c71880e..3d932509426fc 100755 --- a/bin/run-example +++ b/bin/run-example @@ -17,12 +17,12 @@ # limitations under the License. # -SCALA_VERSION=2.10 - FWDIR="$(cd "`dirname "$0"`"/..; pwd)" export SPARK_HOME="$FWDIR" EXAMPLES_DIR="$FWDIR"/examples +. "$FWDIR"/bin/load-spark-env.sh + if [ -n "$1" ]; then EXAMPLE_CLASS="$1" shift @@ -36,8 +36,8 @@ fi if [ -f "$FWDIR/RELEASE" ]; then export SPARK_EXAMPLES_JAR="`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar`" -elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then - export SPARK_EXAMPLES_JAR="`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar`" +elif [ -e "$EXAMPLES_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-examples-*hadoop*.jar ]; then + export SPARK_EXAMPLES_JAR="`ls "$EXAMPLES_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-examples-*hadoop*.jar`" fi if [[ -z "$SPARK_EXAMPLES_JAR" ]]; then diff --git a/bin/spark-class b/bin/spark-class index 925367b0dd187..0d58d95c1aee3 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -24,8 +24,6 @@ case "`uname`" in CYGWIN*) cygwin=true;; esac -SCALA_VERSION=2.10 - # Figure out where Spark is installed FWDIR="$(cd "`dirname "$0"`"/..; pwd)" @@ -128,9 +126,9 @@ fi TOOLS_DIR="$FWDIR"/tools SPARK_TOOLS_JAR="" -if [ -e "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then +if [ -e "$TOOLS_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the SBT build - export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/scala-$SCALA_VERSION/spark-tools*[0-9Tg].jar`" + export SPARK_TOOLS_JAR="`ls "$TOOLS_DIR"/target/scala-$SPARK_SCALA_VERSION/spark-tools*[0-9Tg].jar`" fi if [ -e "$TOOLS_DIR"/target/spark-tools*[0-9Tg].jar ]; then # Use the JAR from the Maven build @@ -149,7 +147,7 @@ fi if [[ "$1" =~ org.apache.spark.tools.* ]]; then if test -z "$SPARK_TOOLS_JAR"; then - echo "Failed to find Spark Tools Jar in $FWDIR/tools/target/scala-$SCALA_VERSION/" 1>&2 + echo "Failed to find Spark Tools Jar in $FWDIR/tools/target/scala-$SPARK_SCALA_VERSION/" 1>&2 echo "You need to build Spark before running $1." 1>&2 exit 1 fi diff --git a/core/pom.xml b/core/pom.xml index 41296e0eca330..492eddda744c2 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -34,6 +34,34 @@ Spark Project Core http://spark.apache.org/ + + com.twitter + chill_${scala.binary.version} + + + org.ow2.asm + asm + + + org.ow2.asm + asm-commons + + + + + com.twitter + chill-java + + + org.ow2.asm + asm + + + org.ow2.asm + asm-commons + + + org.apache.hadoop hadoop-client @@ -46,12 +74,12 @@ org.apache.spark - spark-network-common_2.10 + spark-network-common_${scala.binary.version} ${project.version} org.apache.spark - spark-network-shuffle_2.10 + spark-network-shuffle_${scala.binary.version} ${project.version} @@ -132,14 +160,6 @@ net.jpountz.lz4 lz4 - - com.twitter - chill_${scala.binary.version} - - - com.twitter - chill-java - org.roaringbitmap RoaringBitmap @@ -309,14 +329,16 @@ org.scalatest scalatest-maven-plugin - - - ${basedir}/.. - 1 - ${spark.classpath} - - + + + test + + test + + + + org.apache.maven.plugins @@ -424,4 +446,5 @@ + diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index af94b05ce3847..039c8719e2867 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -87,8 +87,8 @@ object PythonRunner { // Strip the URI scheme from the path formattedPath = new URI(formattedPath).getScheme match { - case Utils.windowsDrive(d) if windows => formattedPath case null => formattedPath + case Utils.windowsDrive(d) if windows => formattedPath case _ => new URI(formattedPath).getPath } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index b43e68e40f791..8a62519bd2315 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -340,7 +340,7 @@ object SparkSubmit { e.printStackTrace(printStream) if (childMainClass.contains("thriftserver")) { println(s"Failed to load main class $childMainClass.") - println("You need to build Spark with -Phive.") + println("You need to build Spark with -Phive and -Phive-thriftserver.") } System.exit(CLASS_NOT_FOUND_EXIT_STATUS) } diff --git a/dev/change-version-to-2.10.sh b/dev/change-version-to-2.10.sh new file mode 100755 index 0000000000000..7473c20d28e09 --- /dev/null +++ b/dev/change-version-to-2.10.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +# +# 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. +# +find . -name 'pom.xml' | grep -v target \ + | xargs -I {} sed -i -e 's|\(artifactId.*\)_2.11|\1_2.10|g' {} diff --git a/dev/change-version-to-2.11.sh b/dev/change-version-to-2.11.sh new file mode 100755 index 0000000000000..3957a9f3ba258 --- /dev/null +++ b/dev/change-version-to-2.11.sh @@ -0,0 +1,21 @@ +#!/usr/bin/env bash + +# +# 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. +# + +find . -name 'pom.xml' | grep -v target \ + | xargs -I {} sed -i -e 's|\(artifactId.*\)_2.10|\1_2.11|g' {} diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 50a9a2fa1cb9a..db441b3e49790 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -118,13 +118,13 @@ make_binary_release() { spark-$RELEASE_VERSION-bin-$NAME.tgz.sha } -make_binary_release "hadoop1" "-Phive -Dhadoop.version=1.0.4" & -make_binary_release "cdh4" "-Phive -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & -make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Pyarn" & -make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Pyarn" & +make_binary_release "hadoop1" "-Phive -Phive-thriftserver -Dhadoop.version=1.0.4" & +make_binary_release "cdh4" "-Phive -Phive-thriftserver -Dhadoop.version=2.0.0-mr1-cdh4.2.0" & +make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" & +make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & -make_binary_release "mapr3" "-Pmapr3 -Phive" & -make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive" & +make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & +make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & wait # Copy data diff --git a/dev/run-tests b/dev/run-tests index de607e4344453..328a73bd8b26d 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -139,9 +139,6 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_BUILD { - # We always build with Hive because the PySpark Spark SQL tests need it. - BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" - # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc @@ -151,15 +148,17 @@ CURRENT_BLOCK=$BLOCK_BUILD # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? # First build with 0.12 to ensure patches do not break the hive 12 build + HIVE_12_BUILD_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver -Phive-0.12.0" echo "[info] Compile with hive 0.12" echo -e "q\n" \ - | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean hive/compile hive-thriftserver/compile \ + | sbt/sbt $HIVE_12_BUILD_ARGS clean hive/compile hive-thriftserver/compile \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" # Then build with default version(0.13.1) because tests are based on this version - echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS -Phive" + echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS"\ + " -Phive -Phive-thriftserver" echo -e "q\n" \ - | sbt/sbt $SBT_MAVEN_PROFILES_ARGS -Phive package assembly/assembly \ + | sbt/sbt $SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver package assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } @@ -174,7 +173,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. # This must be a single argument, as it is. if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver" fi if [ -n "$_SQL_TESTS_ONLY" ]; then diff --git a/dev/scalastyle b/dev/scalastyle index ed1b6b730af6e..c3c6012e74ffa 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,7 +17,7 @@ # limitations under the License. # -echo -e "q\n" | sbt/sbt -Phive scalastyle > scalastyle.txt +echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt # Check style with YARN alpha built too echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ >> scalastyle.txt diff --git a/docs/building-spark.md b/docs/building-spark.md index 238ddae15545e..20ba7da5d71ff 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -101,25 +101,34 @@ mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -Dski # Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, -add the `-Phive` profile to your existing build options. By default Spark -will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using -the `-Phive-0.12.0` profile. +add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. +By default Spark will build with Hive 0.13.1 bindings. You can also build for +Hive 0.12.0 using the `-Phive-0.12.0` profile. {% highlight bash %} # Apache Hadoop 2.4.X with Hive 13 support -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package # Apache Hadoop 2.4.X with Hive 12 support -mvn -Pyarn -Phive-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package +mvn -Pyarn -Phive -Phive-thriftserver-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package {% endhighlight %} +# Building for Scala 2.11 +To produce a Spark package compiled with Scala 2.11, use the `-Pscala-2.11` profile: + + mvn -Pyarn -Phadoop-2.4 -Pscala-2.11 -DskipTests clean package + +Scala 2.11 support in Spark is experimental and does not support a few features. +Specifically, Spark's external Kafka library and JDBC component are not yet +supported in Scala 2.11 builds. + # Spark Tests in Maven Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.org/user_guide/using_the_scalatest_maven_plugin). Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive clean package - mvn -Pyarn -Phadoop-2.3 -Phive test + mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-thriftserver clean package + mvn -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test The ScalaTest plugin also supports running only a specific test suite as follows: @@ -182,16 +191,16 @@ can be set to control the SBT build. For example: Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly - sbt/sbt -Pyarn -Phadoop-2.3 -Phive test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver assembly + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver test To run only a specific test suite as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite" + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver "test-only org.apache.spark.repl.ReplSuite" To run test suites of a specific sub project as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-thriftserver core/test # Speeding up Compilation with Zinc diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index ffcce2c588879..48e8267ac072c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -728,7 +728,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 "`sbt/sbt -Phive assembly/assembly`" (or use `-Phive` for maven). +Hive support is enabled by adding the `-Phive` and `-Phive-thriftserver` flags to Spark's build. 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 access data stored in Hive. diff --git a/examples/pom.xml b/examples/pom.xml index 910eb55308b9d..2ec5728154abf 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -34,48 +34,6 @@ Spark Project Examples http://spark.apache.org/ - - - kinesis-asl - - - org.apache.spark - spark-streaming-kinesis-asl_${scala.binary.version} - ${project.version} - - - org.apache.httpcomponents - httpclient - ${commons.httpclient.version} - - - - - hbase-hadoop2 - - - hbase.profile - hadoop2 - - - - 0.98.7-hadoop2 - - - - hbase-hadoop1 - - - !hbase.profile - - - - 0.98.7-hadoop1 - - - - - @@ -124,11 +82,6 @@ spark-streaming-twitter_${scala.binary.version} ${project.version} - - org.apache.spark - spark-streaming-kafka_${scala.binary.version} - ${project.version} - org.apache.spark spark-streaming-flume_${scala.binary.version} @@ -136,12 +89,12 @@ org.apache.spark - spark-streaming-zeromq_${scala.binary.version} + spark-streaming-mqtt_${scala.binary.version} ${project.version} org.apache.spark - spark-streaming-mqtt_${scala.binary.version} + spark-streaming-zeromq_${scala.binary.version} ${project.version} @@ -260,11 +213,6 @@ test-jar test - - com.twitter - algebird-core_${scala.binary.version} - 0.1.11 - org.apache.commons commons-math3 @@ -401,4 +349,147 @@ + + + kinesis-asl + + + org.apache.spark + spark-streaming-kinesis-asl_${scala.binary.version} + ${project.version} + + + org.apache.httpcomponents + httpclient + ${commons.httpclient.version} + + + + + hbase-hadoop2 + + + hbase.profile + hadoop2 + + + + 0.98.7-hadoop2 + + + + hbase-hadoop1 + + + !hbase.profile + + + + 0.98.7-hadoop1 + + + + + scala-2.10 + + true + + + + org.apache.spark + spark-streaming-kafka_${scala.binary.version} + ${project.version} + + + com.twitter + algebird-core_${scala.binary.version} + 0.1.11 + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + src/main/scala + scala-2.10/src/main/scala + scala-2.10/src/main/java + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + scala-2.10/src/test/scala + scala-2.10/src/test/java + + + + + + + + + + scala-2.11 + + false + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + src/main/scala + scala-2.11/src/main/scala + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + scala-2.11/src/test/scala + + + + + + + + + diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java b/examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java similarity index 100% rename from examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java rename to examples/scala-2.10/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala similarity index 100% rename from examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala rename to examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala similarity index 100% rename from examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala rename to examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala similarity index 100% rename from examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala rename to examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 371f1f1e9d39a..362a76e515938 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -52,11 +52,6 @@ mqtt-client 0.4.0 - - ${akka.group} - akka-zeromq_${scala.binary.version} - ${akka.version} - org.scalatest scalatest_${scala.binary.version} diff --git a/make-distribution.sh b/make-distribution.sh index 0bc839e1dbe4d..d46edbc50d152 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -59,7 +59,7 @@ while (( "$#" )); do exit_with_usage ;; --with-hive) - echo "Error: '--with-hive' is no longer supported, use Maven option -Phive" + echo "Error: '--with-hive' is no longer supported, use Maven options -Phive and -Phive-thriftserver" exit_with_usage ;; --skip-java-test) diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml index 27c8467687f10..a180a5e5f926e 100644 --- a/network/shuffle/pom.xml +++ b/network/shuffle/pom.xml @@ -39,7 +39,7 @@ org.apache.spark - spark-network-common_2.10 + spark-network-common_${scala.binary.version} ${project.version} @@ -58,7 +58,7 @@ org.apache.spark - spark-network-common_2.10 + spark-network-common_${scala.binary.version} ${project.version} test-jar test diff --git a/network/yarn/pom.xml b/network/yarn/pom.xml index 6e6f6f3e79296..85960eb85b482 100644 --- a/network/yarn/pom.xml +++ b/network/yarn/pom.xml @@ -39,7 +39,7 @@ org.apache.spark - spark-network-shuffle_2.10 + spark-network-shuffle_${scala.binary.version} ${project.version} diff --git a/pom.xml b/pom.xml index 4e0cd6c151d0b..7bbde31e572d9 100644 --- a/pom.xml +++ b/pom.xml @@ -97,30 +97,26 @@ sql/catalyst sql/core sql/hive - repl assembly external/twitter - external/kafka external/flume external/flume-sink - external/zeromq external/mqtt + external/zeromq examples + repl UTF-8 UTF-8 - + org.spark-project.akka + 2.3.4-spark 1.6 spark - 2.10.4 - 2.10 2.0.1 0.18.1 shaded-protobuf - org.spark-project.akka - 2.3.4-spark 1.7.5 1.2.17 1.0.4 @@ -137,7 +133,7 @@ 1.6.0rc3 1.2.3 8.1.14.v20131031 - 0.3.6 + 0.5.0 3.0.0 1.7.6 @@ -146,9 +142,13 @@ 1.1.0 4.2.6 3.1.1 - + ${project.build.directory}/spark-test-classpath.txt 64m 512m + 2.10.4 + 2.10 + ${scala.version} + org.scala-lang @@ -267,19 +267,66 @@ + + - org.spark-project.spark unused 1.0.0 + + + org.codehaus.groovy + groovy-all + 2.3.7 + provided + + + ${jline.groupid} + jline + ${jline.version} + + + com.twitter + chill_${scala.binary.version} + ${chill.version} + + + org.ow2.asm + asm + + + org.ow2.asm + asm-commons + + + + + com.twitter + chill-java + ${chill.version} + + + org.ow2.asm + asm + + + org.ow2.asm + asm-commons + + + org.eclipse.jetty jetty-util @@ -395,36 +442,6 @@ protobuf-java ${protobuf.version} - - com.twitter - chill_${scala.binary.version} - ${chill.version} - - - org.ow2.asm - asm - - - org.ow2.asm - asm-commons - - - - - com.twitter - chill-java - ${chill.version} - - - org.ow2.asm - asm - - - org.ow2.asm - asm-commons - - - ${akka.group} akka-actor_${scala.binary.version} @@ -512,11 +529,6 @@ scala-reflect ${scala.version} - - org.scala-lang - jline - ${scala.version} - org.scala-lang scala-library @@ -965,6 +977,7 @@ ${session.executionRootDirectory} 1 false + ${test_classpath} @@ -1026,6 +1039,47 @@ + + + org.apache.maven.plugins + maven-dependency-plugin + 2.9 + + + test-compile + + build-classpath + + + test + ${test_classpath_file} + + + + + + + + org.codehaus.gmavenplus + gmavenplus-plugin + 1.2 + + + process-test-classes + + execute + + + + + + + + + org.apache.maven.plugins @@ -1335,7 +1389,7 @@ - hive + hive-thriftserver false @@ -1365,5 +1419,35 @@ 10.10.1.1 + + + scala-2.10 + + true + + + 2.10.4 + 2.10 + ${scala.version} + org.scala-lang + + + external/kafka + + + + + scala-2.11 + + false + + + 2.11.2 + 2.11 + 2.12 + jline + + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 657e4b4432775..5eb3ed439cde9 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -31,8 +31,8 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, - sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka, - streamingMqtt, streamingTwitter, streamingZeromq) = + sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka, + streamingMqtt, streamingTwitter, streamingZeromq) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", "sql", "network-common", "network-shuffle", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", @@ -68,8 +68,8 @@ object SparkBuild extends PomBuild { profiles ++= Seq("spark-ganglia-lgpl") } if (Properties.envOrNone("SPARK_HIVE").isDefined) { - println("NOTE: SPARK_HIVE is deprecated, please use -Phive flag.") - profiles ++= Seq("hive") + println("NOTE: SPARK_HIVE is deprecated, please use -Phive and -Phive-thriftserver flags.") + profiles ++= Seq("hive", "hive-thriftserver") } Properties.envOrNone("SPARK_HADOOP_VERSION") match { case Some(v) => @@ -91,13 +91,21 @@ object SparkBuild extends PomBuild { profiles } - override val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match { + override val profiles = { + val profiles = Properties.envOrNone("SBT_MAVEN_PROFILES") match { case None => backwardCompatibility case Some(v) => if (backwardCompatibility.nonEmpty) println("Note: We ignore environment variables, when use of profile is detected in " + "conjunction with environment variable.") v.split("(\\s+|,)").filterNot(_.isEmpty).map(_.trim.replaceAll("-P", "")).toSeq + } + if (profiles.exists(_.contains("scala-"))) { + profiles + } else { + println("Enabled default scala profile") + profiles ++ Seq("scala-2.10") + } } Properties.envOrNone("SBT_MAVEN_PROPERTIES") match { @@ -136,7 +144,8 @@ object SparkBuild extends PomBuild { // Note ordering of these settings matter. /* Enable shared settings on all projects */ - (allProjects ++ optionallyEnabledProjects ++ assemblyProjects).foreach(enable(sharedSettings)) + (allProjects ++ optionallyEnabledProjects ++ assemblyProjects ++ Seq(spark, tools)) + .foreach(enable(sharedSettings ++ ExludedDependencies.settings)) /* Enable tests settings for all projects except examples, assembly and tools */ (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) @@ -178,6 +187,16 @@ object Flume { lazy val settings = sbtavro.SbtAvro.avroSettings } +/** + This excludes library dependencies in sbt, which are specified in maven but are + not needed by sbt build. + */ +object ExludedDependencies { + lazy val settings = Seq( + libraryDependencies ~= { libs => libs.filterNot(_.name == "groovy-all") } + ) +} + /** * Following project only exists to pull previous artifacts of Spark for generating * Mima ignores. For more information see: SPARK 2071 @@ -353,8 +372,11 @@ object TestSettings { .map { case (k,v) => s"-D$k=$v" }.toSeq, javaOptions in Test ++= "-Xmx3g -XX:PermSize=128M -XX:MaxNewSize=256m -XX:MaxPermSize=1g" .split(" ").toSeq, + // This places test scope jars on the classpath of executors during tests. + javaOptions in Test += + "-Dspark.executor.extraClassPath=" + (fullClasspath in Test).value.files. + map(_.getAbsolutePath).mkString(":").stripSuffix(":"), javaOptions += "-Xmx3g", - // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index 3ef2d5451da0d..8863f272da415 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -26,7 +26,7 @@ import sbt.Keys._ object SparkPluginDef extends Build { lazy val root = Project("plugins", file(".")) dependsOn(sparkStyle, sbtPomReader) lazy val sparkStyle = Project("spark-style", file("spark-style"), settings = styleSettings) - lazy val sbtPomReader = uri("https://github.com/ScrapCodes/sbt-pom-reader.git") + lazy val sbtPomReader = uri("https://github.com/ScrapCodes/sbt-pom-reader.git#ignore_artifact_id") // There is actually no need to publish this artifact. def styleSettings = Defaults.defaultSettings ++ Seq ( diff --git a/repl/pom.xml b/repl/pom.xml index af528c8914335..bd688c8c1e752 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -38,6 +38,11 @@ + + ${jline.groupid} + jline + ${jline.version} + org.apache.spark spark-core_${scala.binary.version} @@ -75,11 +80,6 @@ scala-reflect ${scala.version} - - org.scala-lang - jline - ${scala.version} - org.slf4j jul-to-slf4j @@ -124,4 +124,84 @@ + + + scala-2.10 + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + src/main/scala + scala-2.10/src/main/scala + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + scala-2.10/src/test/scala + + + + + + + + + + scala-2.11 + + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-sources + generate-sources + + add-source + + + + src/main/scala + scala-2.11/src/main/scala + + + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + scala-2.11/src/test/scala + + + + + + + + + diff --git a/repl/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/Main.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/Main.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkCommandLine.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkHelper.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkHelper.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkIMain.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkImports.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkImports.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineCompletion.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkMemberHandlers.scala diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala similarity index 100% rename from repl/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala rename to repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkRunnerSettings.scala diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala similarity index 100% rename from repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala rename to repl/scala-2.10/src/test/scala/org/apache/spark/repl/ReplSuite.scala diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala new file mode 100644 index 0000000000000..5e93a71995072 --- /dev/null +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala @@ -0,0 +1,85 @@ +/* + * 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.repl + +import org.apache.spark.util.Utils +import org.apache.spark._ + +import scala.tools.nsc.Settings +import scala.tools.nsc.interpreter.SparkILoop + +object Main extends Logging { + + val conf = new SparkConf() + val tmp = System.getProperty("java.io.tmpdir") + val rootDir = conf.get("spark.repl.classdir", tmp) + val outputDir = Utils.createTempDir(rootDir) + val s = new Settings() + s.processArguments(List("-Yrepl-class-based", + "-Yrepl-outdir", s"${outputDir.getAbsolutePath}", "-Yrepl-sync"), true) + val classServer = new HttpServer(outputDir, new SecurityManager(conf)) + var sparkContext: SparkContext = _ + var interp = new SparkILoop // this is a public var because tests reset it. + + def main(args: Array[String]) { + if (getMaster == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + // Start the classServer and store its URI in a spark system property + // (which will be passed to executors so that they can connect to it) + classServer.start() + interp.process(s) // Repl starts and goes in loop of R.E.P.L + classServer.stop() + Option(sparkContext).map(_.stop) + } + + + def getAddedJars: Array[String] = { + val envJars = sys.env.get("ADD_JARS") + val propJars = sys.props.get("spark.jars").flatMap { p => if (p == "") None else Some(p) } + val jars = propJars.orElse(envJars).getOrElse("") + Utils.resolveURIs(jars).split(",").filter(_.nonEmpty) + } + + def createSparkContext(): SparkContext = { + val execUri = System.getenv("SPARK_EXECUTOR_URI") + val jars = getAddedJars + val conf = new SparkConf() + .setMaster(getMaster) + .setAppName("Spark shell") + .setJars(jars) + .set("spark.repl.class.uri", classServer.uri) + logInfo("Spark class server started at " + classServer.uri) + if (execUri != null) { + conf.set("spark.executor.uri", execUri) + } + if (System.getenv("SPARK_HOME") != null) { + conf.setSparkHome(System.getenv("SPARK_HOME")) + } + sparkContext = new SparkContext(conf) + logInfo("Created spark context..") + sparkContext + } + + private def getMaster: String = { + val master = { + val envMaster = sys.env.get("MASTER") + val propMaster = sys.props.get("spark.master") + propMaster.orElse(envMaster).getOrElse("local[*]") + } + master + } +} diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala new file mode 100644 index 0000000000000..8e519fa67f649 --- /dev/null +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -0,0 +1,86 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Paul Phillips + */ + +package scala.tools.nsc +package interpreter + +import scala.tools.nsc.ast.parser.Tokens.EOF + +trait SparkExprTyper { + val repl: SparkIMain + + import repl._ + import global.{ reporter => _, Import => _, _ } + import naming.freshInternalVarName + + def symbolOfLine(code: String): Symbol = { + def asExpr(): Symbol = { + val name = freshInternalVarName() + // Typing it with a lazy val would give us the right type, but runs + // into compiler bugs with things like existentials, so we compile it + // behind a def and strip the NullaryMethodType which wraps the expr. + val line = "def " + name + " = " + code + + interpretSynthetic(line) match { + case IR.Success => + val sym0 = symbolOfTerm(name) + // drop NullaryMethodType + sym0.cloneSymbol setInfo exitingTyper(sym0.tpe_*.finalResultType) + case _ => NoSymbol + } + } + def asDefn(): Symbol = { + val old = repl.definedSymbolList.toSet + + interpretSynthetic(code) match { + case IR.Success => + repl.definedSymbolList filterNot old match { + case Nil => NoSymbol + case sym :: Nil => sym + case syms => NoSymbol.newOverloaded(NoPrefix, syms) + } + case _ => NoSymbol + } + } + def asError(): Symbol = { + interpretSynthetic(code) + NoSymbol + } + beSilentDuring(asExpr()) orElse beSilentDuring(asDefn()) orElse asError() + } + + private var typeOfExpressionDepth = 0 + def typeOfExpression(expr: String, silent: Boolean = true): Type = { + if (typeOfExpressionDepth > 2) { + repldbg("Terminating typeOfExpression recursion for expression: " + expr) + return NoType + } + typeOfExpressionDepth += 1 + // Don't presently have a good way to suppress undesirable success output + // while letting errors through, so it is first trying it silently: if there + // is an error, and errors are desired, then it re-evaluates non-silently + // to induce the error message. + try beSilentDuring(symbolOfLine(expr).tpe) match { + case NoType if !silent => symbolOfLine(expr).tpe // generate error + case tpe => tpe + } + finally typeOfExpressionDepth -= 1 + } + + // This only works for proper types. + def typeOfTypeString(typeString: String): Type = { + def asProperType(): Option[Type] = { + val name = freshInternalVarName() + val line = "def %s: %s = ???" format (name, typeString) + interpretSynthetic(line) match { + case IR.Success => + val sym0 = symbolOfTerm(name) + Some(sym0.asMethod.returnType) + case _ => None + } + } + beSilentDuring(asProperType()) getOrElse NoType + } +} diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala new file mode 100644 index 0000000000000..a591e9fc4622b --- /dev/null +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -0,0 +1,966 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Alexander Spoon + */ + +package scala +package tools.nsc +package interpreter + +import scala.language.{ implicitConversions, existentials } +import scala.annotation.tailrec +import Predef.{ println => _, _ } +import interpreter.session._ +import StdReplTags._ +import scala.reflect.api.{Mirror, Universe, TypeCreator} +import scala.util.Properties.{ jdkHome, javaVersion, versionString, javaVmName } +import scala.tools.nsc.util.{ ClassPath, Exceptional, stringFromWriter, stringFromStream } +import scala.reflect.{ClassTag, classTag} +import scala.reflect.internal.util.{ BatchSourceFile, ScalaClassLoader } +import ScalaClassLoader._ +import scala.reflect.io.{ File, Directory } +import scala.tools.util._ +import scala.collection.generic.Clearable +import scala.concurrent.{ ExecutionContext, Await, Future, future } +import ExecutionContext.Implicits._ +import java.io.{ BufferedReader, FileReader } + +/** The Scala interactive shell. It provides a read-eval-print loop + * around the Interpreter class. + * After instantiation, clients should call the main() method. + * + * If no in0 is specified, then input will come from the console, and + * the class will attempt to provide input editing feature such as + * input history. + * + * @author Moez A. Abdel-Gawad + * @author Lex Spoon + * @version 1.2 + */ +class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter) + extends AnyRef + with LoopCommands +{ + def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out) + def this() = this(None, new JPrintWriter(Console.out, true)) +// +// @deprecated("Use `intp` instead.", "2.9.0") def interpreter = intp +// @deprecated("Use `intp` instead.", "2.9.0") def interpreter_= (i: Interpreter): Unit = intp = i + + var in: InteractiveReader = _ // the input stream from which commands come + var settings: Settings = _ + var intp: SparkIMain = _ + + var globalFuture: Future[Boolean] = _ + + protected def asyncMessage(msg: String) { + if (isReplInfo || isReplPower) + echoAndRefresh(msg) + } + + def initializeSpark() { + intp.beQuietDuring { + command( """ + @transient val sc = org.apache.spark.repl.Main.createSparkContext(); + """) + command("import org.apache.spark.SparkContext._") + } + echo("Spark context available as sc.") + } + + /** Print a welcome message */ + def printWelcome() { + import org.apache.spark.SPARK_VERSION + echo("""Welcome to + ____ __ + / __/__ ___ _____/ /__ + _\ \/ _ \/ _ `/ __/ '_/ + /___/ .__/\_,_/_/ /_/\_\ version %s + /_/ + """.format(SPARK_VERSION)) + val welcomeMsg = "Using Scala %s (%s, Java %s)".format( + versionString, javaVmName, javaVersion) + echo(welcomeMsg) + echo("Type in expressions to have them evaluated.") + echo("Type :help for more information.") + } + + override def echoCommandMessage(msg: String) { + intp.reporter printUntruncatedMessage msg + } + + // lazy val power = new Power(intp, new StdReplVals(this))(tagOfStdReplVals, classTag[StdReplVals]) + def history = in.history + + // classpath entries added via :cp + var addedClasspath: String = "" + + /** A reverse list of commands to replay if the user requests a :replay */ + var replayCommandStack: List[String] = Nil + + /** A list of commands to replay if the user requests a :replay */ + def replayCommands = replayCommandStack.reverse + + /** Record a command for replay should the user request a :replay */ + def addReplay(cmd: String) = replayCommandStack ::= cmd + + def savingReplayStack[T](body: => T): T = { + val saved = replayCommandStack + try body + finally replayCommandStack = saved + } + def savingReader[T](body: => T): T = { + val saved = in + try body + finally in = saved + } + + /** Close the interpreter and set the var to null. */ + def closeInterpreter() { + if (intp ne null) { + intp.close() + intp = null + } + } + + class SparkILoopInterpreter extends SparkIMain(settings, out) { + outer => + + override lazy val formatting = new Formatting { + def prompt = SparkILoop.this.prompt + } + override protected def parentClassLoader = + settings.explicitParentLoader.getOrElse( classOf[SparkILoop].getClassLoader ) + } + + /** Create a new interpreter. */ + def createInterpreter() { + if (addedClasspath != "") + settings.classpath append addedClasspath + + intp = new SparkILoopInterpreter + } + + /** print a friendly help message */ + def helpCommand(line: String): Result = { + if (line == "") helpSummary() + else uniqueCommand(line) match { + case Some(lc) => echo("\n" + lc.help) + case _ => ambiguousError(line) + } + } + private def helpSummary() = { + val usageWidth = commands map (_.usageMsg.length) max + val formatStr = "%-" + usageWidth + "s %s" + + echo("All commands can be abbreviated, e.g. :he instead of :help.") + + commands foreach { cmd => + echo(formatStr.format(cmd.usageMsg, cmd.help)) + } + } + private def ambiguousError(cmd: String): Result = { + matchingCommands(cmd) match { + case Nil => echo(cmd + ": no such command. Type :help for help.") + case xs => echo(cmd + " is ambiguous: did you mean " + xs.map(":" + _.name).mkString(" or ") + "?") + } + Result(keepRunning = true, None) + } + private def matchingCommands(cmd: String) = commands filter (_.name startsWith cmd) + private def uniqueCommand(cmd: String): Option[LoopCommand] = { + // this lets us add commands willy-nilly and only requires enough command to disambiguate + matchingCommands(cmd) match { + case List(x) => Some(x) + // exact match OK even if otherwise appears ambiguous + case xs => xs find (_.name == cmd) + } + } + + /** Show the history */ + lazy val historyCommand = new LoopCommand("history", "show the history (optional num is commands to show)") { + override def usage = "[num]" + def defaultLines = 20 + + def apply(line: String): Result = { + if (history eq NoHistory) + return "No history available." + + val xs = words(line) + val current = history.index + val count = try xs.head.toInt catch { case _: Exception => defaultLines } + val lines = history.asStrings takeRight count + val offset = current - lines.size + 1 + + for ((line, index) <- lines.zipWithIndex) + echo("%3d %s".format(index + offset, line)) + } + } + + // When you know you are most likely breaking into the middle + // of a line being typed. This softens the blow. + protected def echoAndRefresh(msg: String) = { + echo("\n" + msg) + in.redrawLine() + } + protected def echo(msg: String) = { + out println msg + out.flush() + } + + /** Search the history */ + def searchHistory(_cmdline: String) { + val cmdline = _cmdline.toLowerCase + val offset = history.index - history.size + 1 + + for ((line, index) <- history.asStrings.zipWithIndex ; if line.toLowerCase contains cmdline) + echo("%d %s".format(index + offset, line)) + } + + private val currentPrompt = Properties.shellPromptString + + /** Prompt to print when awaiting input */ + def prompt = currentPrompt + + import LoopCommand.{ cmd, nullary } + + /** Standard commands **/ + lazy val standardCommands = List( + cmd("cp", "", "add a jar or directory to the classpath", addClasspath), + cmd("edit", "|", "edit history", editCommand), + cmd("help", "[command]", "print this summary or command-specific help", helpCommand), + historyCommand, + cmd("h?", "", "search the history", searchHistory), + cmd("imports", "[name name ...]", "show import history, identifying sources of names", importsCommand), + //cmd("implicits", "[-v]", "show the implicits in scope", intp.implicitsCommand), + cmd("javap", "", "disassemble a file or class name", javapCommand), + cmd("line", "|", "place line(s) at the end of history", lineCommand), + cmd("load", "", "interpret lines in a file", loadCommand), + cmd("paste", "[-raw] [path]", "enter paste mode or paste a file", pasteCommand), + // nullary("power", "enable power user mode", powerCmd), + nullary("quit", "exit the interpreter", () => Result(keepRunning = false, None)), + nullary("replay", "reset execution and replay all previous commands", replay), + nullary("reset", "reset the repl to its initial state, forgetting all session entries", resetCommand), + cmd("save", "", "save replayable session to a file", saveCommand), + shCommand, + cmd("settings", "[+|-]", "+enable/-disable flags, set compiler options", changeSettings), + nullary("silent", "disable/enable automatic printing of results", verbosity), +// cmd("type", "[-v] ", "display the type of an expression without evaluating it", typeCommand), +// cmd("kind", "[-v] ", "display the kind of expression's type", kindCommand), + nullary("warnings", "show the suppressed warnings from the most recent line which had any", warningsCommand) + ) + + /** Power user commands */ +// lazy val powerCommands: List[LoopCommand] = List( +// cmd("phase", "", "set the implicit phase for power commands", phaseCommand) +// ) + + private def importsCommand(line: String): Result = { + val tokens = words(line) + val handlers = intp.languageWildcardHandlers ++ intp.importHandlers + + handlers.filterNot(_.importedSymbols.isEmpty).zipWithIndex foreach { + case (handler, idx) => + val (types, terms) = handler.importedSymbols partition (_.name.isTypeName) + val imps = handler.implicitSymbols + val found = tokens filter (handler importsSymbolNamed _) + val typeMsg = if (types.isEmpty) "" else types.size + " types" + val termMsg = if (terms.isEmpty) "" else terms.size + " terms" + val implicitMsg = if (imps.isEmpty) "" else imps.size + " are implicit" + val foundMsg = if (found.isEmpty) "" else found.mkString(" // imports: ", ", ", "") + val statsMsg = List(typeMsg, termMsg, implicitMsg) filterNot (_ == "") mkString ("(", ", ", ")") + + intp.reporter.printMessage("%2d) %-30s %s%s".format( + idx + 1, + handler.importString, + statsMsg, + foundMsg + )) + } + } + + private def findToolsJar() = PathResolver.SupplementalLocations.platformTools + + private def addToolsJarToLoader() = { + val cl = findToolsJar() match { + case Some(tools) => ScalaClassLoader.fromURLs(Seq(tools.toURL), intp.classLoader) + case _ => intp.classLoader + } + if (Javap.isAvailable(cl)) { + repldbg(":javap available.") + cl + } + else { + repldbg(":javap unavailable: no tools.jar at " + jdkHome) + intp.classLoader + } + } +// +// protected def newJavap() = +// JavapClass(addToolsJarToLoader(), new IMain.ReplStrippingWriter(intp), Some(intp)) +// +// private lazy val javap = substituteAndLog[Javap]("javap", NoJavap)(newJavap()) + + // Still todo: modules. +// private def typeCommand(line0: String): Result = { +// line0.trim match { +// case "" => ":type [-v] " +// case s => intp.typeCommandInternal(s stripPrefix "-v " trim, verbose = s startsWith "-v ") +// } +// } + +// private def kindCommand(expr: String): Result = { +// expr.trim match { +// case "" => ":kind [-v] " +// case s => intp.kindCommandInternal(s stripPrefix "-v " trim, verbose = s startsWith "-v ") +// } +// } + + private def warningsCommand(): Result = { + if (intp.lastWarnings.isEmpty) + "Can't find any cached warnings." + else + intp.lastWarnings foreach { case (pos, msg) => intp.reporter.warning(pos, msg) } + } + + private def changeSettings(args: String): Result = { + def showSettings() = { + for (s <- settings.userSetSettings.toSeq.sorted) echo(s.toString) + } + def updateSettings() = { + // put aside +flag options + val (pluses, rest) = (args split "\\s+").toList partition (_.startsWith("+")) + val tmps = new Settings + val (ok, leftover) = tmps.processArguments(rest, processAll = true) + if (!ok) echo("Bad settings request.") + else if (leftover.nonEmpty) echo("Unprocessed settings.") + else { + // boolean flags set-by-user on tmp copy should be off, not on + val offs = tmps.userSetSettings filter (_.isInstanceOf[Settings#BooleanSetting]) + val (minuses, nonbools) = rest partition (arg => offs exists (_ respondsTo arg)) + // update non-flags + settings.processArguments(nonbools, processAll = true) + // also snag multi-value options for clearing, e.g. -Ylog: and -language: + for { + s <- settings.userSetSettings + if s.isInstanceOf[Settings#MultiStringSetting] || s.isInstanceOf[Settings#PhasesSetting] + if nonbools exists (arg => arg.head == '-' && arg.last == ':' && (s respondsTo arg.init)) + } s match { + case c: Clearable => c.clear() + case _ => + } + def update(bs: Seq[String], name: String=>String, setter: Settings#Setting=>Unit) = { + for (b <- bs) + settings.lookupSetting(name(b)) match { + case Some(s) => + if (s.isInstanceOf[Settings#BooleanSetting]) setter(s) + else echo(s"Not a boolean flag: $b") + case _ => + echo(s"Not an option: $b") + } + } + update(minuses, identity, _.tryToSetFromPropertyValue("false")) // turn off + update(pluses, "-" + _.drop(1), _.tryToSet(Nil)) // turn on + } + } + if (args.isEmpty) showSettings() else updateSettings() + } + + private def javapCommand(line: String): Result = { +// if (javap == null) +// ":javap unavailable, no tools.jar at %s. Set JDK_HOME.".format(jdkHome) +// else if (line == "") +// ":javap [-lcsvp] [path1 path2 ...]" +// else +// javap(words(line)) foreach { res => +// if (res.isError) return "Failed: " + res.value +// else res.show() +// } + } + + private def pathToPhaseWrapper = intp.originalPath("$r") + ".phased.atCurrent" + + private def phaseCommand(name: String): Result = { +// val phased: Phased = power.phased +// import phased.NoPhaseName +// +// if (name == "clear") { +// phased.set(NoPhaseName) +// intp.clearExecutionWrapper() +// "Cleared active phase." +// } +// else if (name == "") phased.get match { +// case NoPhaseName => "Usage: :phase (e.g. typer, erasure.next, erasure+3)" +// case ph => "Active phase is '%s'. (To clear, :phase clear)".format(phased.get) +// } +// else { +// val what = phased.parse(name) +// if (what.isEmpty || !phased.set(what)) +// "'" + name + "' does not appear to represent a valid phase." +// else { +// intp.setExecutionWrapper(pathToPhaseWrapper) +// val activeMessage = +// if (what.toString.length == name.length) "" + what +// else "%s (%s)".format(what, name) +// +// "Active phase is now: " + activeMessage +// } +// } + } + + /** Available commands */ + def commands: List[LoopCommand] = standardCommands ++ ( + // if (isReplPower) + // powerCommands + // else + Nil + ) + + val replayQuestionMessage = + """|That entry seems to have slain the compiler. Shall I replay + |your session? I can re-run each line except the last one. + |[y/n] + """.trim.stripMargin + + private val crashRecovery: PartialFunction[Throwable, Boolean] = { + case ex: Throwable => + val (err, explain) = ( + if (intp.isInitializeComplete) + (intp.global.throwableAsString(ex), "") + else + (ex.getMessage, "The compiler did not initialize.\n") + ) + echo(err) + + ex match { + case _: NoSuchMethodError | _: NoClassDefFoundError => + echo("\nUnrecoverable error.") + throw ex + case _ => + def fn(): Boolean = + try in.readYesOrNo(explain + replayQuestionMessage, { echo("\nYou must enter y or n.") ; fn() }) + catch { case _: RuntimeException => false } + + if (fn()) replay() + else echo("\nAbandoning crashed session.") + } + true + } + + // return false if repl should exit + def processLine(line: String): Boolean = { + import scala.concurrent.duration._ + Await.ready(globalFuture, 60.seconds) + + (line ne null) && (command(line) match { + case Result(false, _) => false + case Result(_, Some(line)) => addReplay(line) ; true + case _ => true + }) + } + + private def readOneLine() = { + out.flush() + in readLine prompt + } + + /** The main read-eval-print loop for the repl. It calls + * command() for each line of input, and stops when + * command() returns false. + */ + @tailrec final def loop() { + if ( try processLine(readOneLine()) catch crashRecovery ) + loop() + } + + /** interpret all lines from a specified file */ + def interpretAllFrom(file: File) { + savingReader { + savingReplayStack { + file applyReader { reader => + in = SimpleReader(reader, out, interactive = false) + echo("Loading " + file + "...") + loop() + } + } + } + } + + /** create a new interpreter and replay the given commands */ + def replay() { + reset() + if (replayCommandStack.isEmpty) + echo("Nothing to replay.") + else for (cmd <- replayCommands) { + echo("Replaying: " + cmd) // flush because maybe cmd will have its own output + command(cmd) + echo("") + } + } + def resetCommand() { + echo("Resetting interpreter state.") + if (replayCommandStack.nonEmpty) { + echo("Forgetting this session history:\n") + replayCommands foreach echo + echo("") + replayCommandStack = Nil + } + if (intp.namedDefinedTerms.nonEmpty) + echo("Forgetting all expression results and named terms: " + intp.namedDefinedTerms.mkString(", ")) + if (intp.definedTypes.nonEmpty) + echo("Forgetting defined types: " + intp.definedTypes.mkString(", ")) + + reset() + } + def reset() { + intp.reset() + unleashAndSetPhase() + } + + def lineCommand(what: String): Result = editCommand(what, None) + + // :edit id or :edit line + def editCommand(what: String): Result = editCommand(what, Properties.envOrNone("EDITOR")) + + def editCommand(what: String, editor: Option[String]): Result = { + def diagnose(code: String) = { + echo("The edited code is incomplete!\n") + val errless = intp compileSources new BatchSourceFile("", s"object pastel {\n$code\n}") + if (errless) echo("The compiler reports no errors.") + } + def historicize(text: String) = history match { + case jlh: JLineHistory => text.lines foreach jlh.add ; jlh.moveToEnd() ; true + case _ => false + } + def edit(text: String): Result = editor match { + case Some(ed) => + val tmp = File.makeTemp() + tmp.writeAll(text) + try { + val pr = new ProcessResult(s"$ed ${tmp.path}") + pr.exitCode match { + case 0 => + tmp.safeSlurp() match { + case Some(edited) if edited.trim.isEmpty => echo("Edited text is empty.") + case Some(edited) => + echo(edited.lines map ("+" + _) mkString "\n") + val res = intp interpret edited + if (res == IR.Incomplete) diagnose(edited) + else { + historicize(edited) + Result(lineToRecord = Some(edited), keepRunning = true) + } + case None => echo("Can't read edited text. Did you delete it?") + } + case x => echo(s"Error exit from $ed ($x), ignoring") + } + } finally { + tmp.delete() + } + case None => + if (historicize(text)) echo("Placing text in recent history.") + else echo(f"No EDITOR defined and you can't change history, echoing your text:%n$text") + } + + // if what is a number, use it as a line number or range in history + def isNum = what forall (c => c.isDigit || c == '-' || c == '+') + // except that "-" means last value + def isLast = (what == "-") + if (isLast || !isNum) { + val name = if (isLast) intp.mostRecentVar else what + val sym = intp.symbolOfIdent(name) + intp.prevRequestList collectFirst { case r if r.defines contains sym => r } match { + case Some(req) => edit(req.line) + case None => echo(s"No symbol in scope: $what") + } + } else try { + val s = what + // line 123, 120+3, -3, 120-123, 120-, note -3 is not 0-3 but (cur-3,cur) + val (start, len) = + if ((s indexOf '+') > 0) { + val (a,b) = s splitAt (s indexOf '+') + (a.toInt, b.drop(1).toInt) + } else { + (s indexOf '-') match { + case -1 => (s.toInt, 1) + case 0 => val n = s.drop(1).toInt ; (history.index - n, n) + case _ if s.last == '-' => val n = s.init.toInt ; (n, history.index - n) + case i => val n = s.take(i).toInt ; (n, s.drop(i+1).toInt - n) + } + } + import scala.collection.JavaConverters._ + val index = (start - 1) max 0 + val text = history match { + case jlh: JLineHistory => jlh.entries(index).asScala.take(len) map (_.value) mkString "\n" + case _ => history.asStrings.slice(index, index + len) mkString "\n" + } + edit(text) + } catch { + case _: NumberFormatException => echo(s"Bad range '$what'") + echo("Use line 123, 120+3, -3, 120-123, 120-, note -3 is not 0-3 but (cur-3,cur)") + } + } + + /** fork a shell and run a command */ + lazy val shCommand = new LoopCommand("sh", "run a shell command (result is implicitly => List[String])") { + override def usage = "" + def apply(line: String): Result = line match { + case "" => showUsage() + case _ => + val toRun = s"new ${classOf[ProcessResult].getName}(${string2codeQuoted(line)})" + intp interpret toRun + () + } + } + + def withFile[A](filename: String)(action: File => A): Option[A] = { + val res = Some(File(filename)) filter (_.exists) map action + if (res.isEmpty) echo("That file does not exist") // courtesy side-effect + res + } + + def loadCommand(arg: String) = { + var shouldReplay: Option[String] = None + withFile(arg)(f => { + interpretAllFrom(f) + shouldReplay = Some(":load " + arg) + }) + Result(keepRunning = true, shouldReplay) + } + + def saveCommand(filename: String): Result = ( + if (filename.isEmpty) echo("File name is required.") + else if (replayCommandStack.isEmpty) echo("No replay commands in session") + else File(filename).printlnAll(replayCommands: _*) + ) + + def addClasspath(arg: String): Unit = { + val f = File(arg).normalize + if (f.exists) { + addedClasspath = ClassPath.join(addedClasspath, f.path) + val totalClasspath = ClassPath.join(settings.classpath.value, addedClasspath) + echo("Added '%s'. Your new classpath is:\n\"%s\"".format(f.path, totalClasspath)) + replay() + } + else echo("The path '" + f + "' doesn't seem to exist.") + } + + def powerCmd(): Result = { + if (isReplPower) "Already in power mode." + else enablePowerMode(isDuringInit = false) + } + def enablePowerMode(isDuringInit: Boolean) = { + replProps.power setValue true + unleashAndSetPhase() + // asyncEcho(isDuringInit, power.banner) + } + private def unleashAndSetPhase() { + if (isReplPower) { + // power.unleash() + // Set the phase to "typer" + // intp beSilentDuring phaseCommand("typer") + } + } + + def asyncEcho(async: Boolean, msg: => String) { + if (async) asyncMessage(msg) + else echo(msg) + } + + def verbosity() = { + val old = intp.printResults + intp.printResults = !old + echo("Switched " + (if (old) "off" else "on") + " result printing.") + } + + /** Run one command submitted by the user. Two values are returned: + * (1) whether to keep running, (2) the line to record for replay, + * if any. */ + def command(line: String): Result = { + if (line startsWith ":") { + val cmd = line.tail takeWhile (x => !x.isWhitespace) + uniqueCommand(cmd) match { + case Some(lc) => lc(line.tail stripPrefix cmd dropWhile (_.isWhitespace)) + case _ => ambiguousError(cmd) + } + } + else if (intp.global == null) Result(keepRunning = false, None) // Notice failure to create compiler + else Result(keepRunning = true, interpretStartingWith(line)) + } + + private def readWhile(cond: String => Boolean) = { + Iterator continually in.readLine("") takeWhile (x => x != null && cond(x)) + } + + def pasteCommand(arg: String): Result = { + var shouldReplay: Option[String] = None + def result = Result(keepRunning = true, shouldReplay) + val (raw, file) = + if (arg.isEmpty) (false, None) + else { + val r = """(-raw)?(\s+)?([^\-]\S*)?""".r + arg match { + case r(flag, sep, name) => + if (flag != null && name != null && sep == null) + echo(s"""I assume you mean "$flag $name"?""") + (flag != null, Option(name)) + case _ => + echo("usage: :paste -raw file") + return result + } + } + val code = file match { + case Some(name) => + withFile(name)(f => { + shouldReplay = Some(s":paste $arg") + val s = f.slurp.trim + if (s.isEmpty) echo(s"File contains no code: $f") + else echo(s"Pasting file $f...") + s + }) getOrElse "" + case None => + echo("// Entering paste mode (ctrl-D to finish)\n") + val text = (readWhile(_ => true) mkString "\n").trim + if (text.isEmpty) echo("\n// Nothing pasted, nothing gained.\n") + else echo("\n// Exiting paste mode, now interpreting.\n") + text + } + def interpretCode() = { + val res = intp interpret code + // if input is incomplete, let the compiler try to say why + if (res == IR.Incomplete) { + echo("The pasted code is incomplete!\n") + // Remembrance of Things Pasted in an object + val errless = intp compileSources new BatchSourceFile("", s"object pastel {\n$code\n}") + if (errless) echo("...but compilation found no error? Good luck with that.") + } + } + def compileCode() = { + val errless = intp compileSources new BatchSourceFile("", code) + if (!errless) echo("There were compilation errors!") + } + if (code.nonEmpty) { + if (raw) compileCode() else interpretCode() + } + result + } + + private object paste extends Pasted { + val ContinueString = " | " + val PromptString = "scala> " + + def interpret(line: String): Unit = { + echo(line.trim) + intp interpret line + echo("") + } + + def transcript(start: String) = { + echo("\n// Detected repl transcript paste: ctrl-D to finish.\n") + apply(Iterator(start) ++ readWhile(_.trim != PromptString.trim)) + } + } + import paste.{ ContinueString, PromptString } + + /** Interpret expressions starting with the first line. + * Read lines until a complete compilation unit is available + * or until a syntax error has been seen. If a full unit is + * read, go ahead and interpret it. Return the full string + * to be recorded for replay, if any. + */ + def interpretStartingWith(code: String): Option[String] = { + // signal completion non-completion input has been received + in.completion.resetVerbosity() + + def reallyInterpret = { + val reallyResult = intp.interpret(code) + (reallyResult, reallyResult match { + case IR.Error => None + case IR.Success => Some(code) + case IR.Incomplete => + if (in.interactive && code.endsWith("\n\n")) { + echo("You typed two blank lines. Starting a new command.") + None + } + else in.readLine(ContinueString) match { + case null => + // we know compilation is going to fail since we're at EOF and the + // parser thinks the input is still incomplete, but since this is + // a file being read non-interactively we want to fail. So we send + // it straight to the compiler for the nice error message. + intp.compileString(code) + None + + case line => interpretStartingWith(code + "\n" + line) + } + }) + } + + /** Here we place ourselves between the user and the interpreter and examine + * the input they are ostensibly submitting. We intervene in several cases: + * + * 1) If the line starts with "scala> " it is assumed to be an interpreter paste. + * 2) If the line starts with "." (but not ".." or "./") it is treated as an invocation + * on the previous result. + * 3) If the Completion object's execute returns Some(_), we inject that value + * and avoid the interpreter, as it's likely not valid scala code. + */ + if (code == "") None + else if (!paste.running && code.trim.startsWith(PromptString)) { + paste.transcript(code) + None + } + else if (Completion.looksLikeInvocation(code) && intp.mostRecentVar != "") { + interpretStartingWith(intp.mostRecentVar + code) + } + else if (code.trim startsWith "//") { + // line comment, do nothing + None + } + else + reallyInterpret._2 + } + + // runs :load `file` on any files passed via -i + def loadFiles(settings: Settings) = settings match { + case settings: GenericRunnerSettings => + for (filename <- settings.loadfiles.value) { + val cmd = ":load " + filename + command(cmd) + addReplay(cmd) + echo("") + } + case _ => + } + + /** Tries to create a JLineReader, falling back to SimpleReader: + * unless settings or properties are such that it should start + * with SimpleReader. + */ + def chooseReader(settings: Settings): InteractiveReader = { + if (settings.Xnojline || Properties.isEmacsShell) + SimpleReader() + else try new JLineReader( + if (settings.noCompletion) NoCompletion + else new SparkJLineCompletion(intp) + ) + catch { + case ex @ (_: Exception | _: NoClassDefFoundError) => + echo("Failed to created JLineReader: " + ex + "\nFalling back to SimpleReader.") + SimpleReader() + } + } + protected def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = + u.TypeTag[T]( + m, + new TypeCreator { + def apply[U <: Universe with Singleton](m: Mirror[U]): U # Type = + m.staticClass(classTag[T].runtimeClass.getName).toTypeConstructor.asInstanceOf[U # Type] + }) + + private def loopPostInit() { + // Bind intp somewhere out of the regular namespace where + // we can get at it in generated code. + intp.quietBind(NamedParam[SparkIMain]("$intp", intp)(tagOfStaticClass[SparkIMain], classTag[SparkIMain])) + // Auto-run code via some setting. + ( replProps.replAutorunCode.option + flatMap (f => io.File(f).safeSlurp()) + foreach (intp quietRun _) + ) + // classloader and power mode setup + intp.setContextClassLoader() + if (isReplPower) { + // replProps.power setValue true + // unleashAndSetPhase() + // asyncMessage(power.banner) + } + // SI-7418 Now, and only now, can we enable TAB completion. + in match { + case x: JLineReader => x.consoleReader.postInit + case _ => + } + } + def process(settings: Settings): Boolean = savingContextLoader { + this.settings = settings + createInterpreter() + + // sets in to some kind of reader depending on environmental cues + in = in0.fold(chooseReader(settings))(r => SimpleReader(r, out, interactive = true)) + globalFuture = future { + intp.initializeSynchronous() + loopPostInit() + !intp.reporter.hasErrors + } + import scala.concurrent.duration._ + Await.ready(globalFuture, 10 seconds) + printWelcome() + initializeSpark() + loadFiles(settings) + + try loop() + catch AbstractOrMissingHandler() + finally closeInterpreter() + + true + } + + @deprecated("Use `process` instead", "2.9.0") + def main(settings: Settings): Unit = process(settings) //used by sbt +} + +object SparkILoop { + implicit def loopToInterpreter(repl: SparkILoop): SparkIMain = repl.intp + + // Designed primarily for use by test code: take a String with a + // bunch of code, and prints out a transcript of what it would look + // like if you'd just typed it into the repl. + def runForTranscript(code: String, settings: Settings): String = { + import java.io.{ BufferedReader, StringReader, OutputStreamWriter } + + stringFromStream { ostream => + Console.withOut(ostream) { + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) { + override def write(str: String) = { + // completely skip continuation lines + if (str forall (ch => ch.isWhitespace || ch == '|')) () + else super.write(str) + } + } + val input = new BufferedReader(new StringReader(code.trim + "\n")) { + override def readLine(): String = { + val s = super.readLine() + // helping out by printing the line being interpreted. + if (s != null) + output.println(s) + s + } + } + val repl = new SparkILoop(input, output) + if (settings.classpath.isDefault) + settings.classpath.value = sys.props("java.class.path") + + repl process settings + } + } + } + + /** Creates an interpreter loop with default settings and feeds + * the given code to it as input. + */ + def run(code: String, sets: Settings = new Settings): String = { + import java.io.{ BufferedReader, StringReader, OutputStreamWriter } + + stringFromStream { ostream => + Console.withOut(ostream) { + val input = new BufferedReader(new StringReader(code)) + val output = new JPrintWriter(new OutputStreamWriter(ostream), true) + val repl = new SparkILoop(input, output) + + if (sets.classpath.isDefault) + sets.classpath.value = sys.props("java.class.path") + + repl process sets + } + } + } + def run(lines: List[String]): String = run(lines map (_ + "\n") mkString) +} diff --git a/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala new file mode 100644 index 0000000000000..1bb62c84abddc --- /dev/null +++ b/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -0,0 +1,1319 @@ +/* NSC -- new Scala compiler + * Copyright 2005-2013 LAMP/EPFL + * @author Martin Odersky + */ + +package scala +package tools.nsc +package interpreter + +import PartialFunction.cond +import scala.language.implicitConversions +import scala.beans.BeanProperty +import scala.collection.mutable +import scala.concurrent.{ Future, ExecutionContext } +import scala.reflect.runtime.{ universe => ru } +import scala.reflect.{ ClassTag, classTag } +import scala.reflect.internal.util.{ BatchSourceFile, SourceFile } +import scala.tools.util.PathResolver +import scala.tools.nsc.io.AbstractFile +import scala.tools.nsc.typechecker.{ TypeStrings, StructuredTypeStrings } +import scala.tools.nsc.util.{ ScalaClassLoader, stringFromReader, stringFromWriter, StackTraceOps } +import scala.tools.nsc.util.Exceptional.unwrap +import javax.script.{AbstractScriptEngine, Bindings, ScriptContext, ScriptEngine, ScriptEngineFactory, ScriptException, CompiledScript, Compilable} + +/** An interpreter for Scala code. + * + * The main public entry points are compile(), interpret(), and bind(). + * The compile() method loads a complete Scala file. The interpret() method + * executes one line of Scala code at the request of the user. The bind() + * method binds an object to a variable that can then be used by later + * interpreted code. + * + * The overall approach is based on compiling the requested code and then + * using a Java classloader and Java reflection to run the code + * and access its results. + * + * In more detail, a single compiler instance is used + * to accumulate all successfully compiled or interpreted Scala code. To + * "interpret" a line of code, the compiler generates a fresh object that + * includes the line of code and which has public member(s) to export + * all variables defined by that code. To extract the result of an + * interpreted line to show the user, a second "result object" is created + * which imports the variables exported by the above object and then + * exports members called "$eval" and "$print". To accomodate user expressions + * that read from variables or methods defined in previous statements, "import" + * statements are used. + * + * This interpreter shares the strengths and weaknesses of using the + * full compiler-to-Java. The main strength is that interpreted code + * behaves exactly as does compiled code, including running at full speed. + * The main weakness is that redefining classes and methods is not handled + * properly, because rebinding at the Java level is technically difficult. + * + * @author Moez A. Abdel-Gawad + * @author Lex Spoon + */ +class SparkIMain(@BeanProperty val factory: ScriptEngineFactory, initialSettings: Settings, + protected val out: JPrintWriter) extends AbstractScriptEngine with Compilable with SparkImports { + imain => + + setBindings(createBindings, ScriptContext.ENGINE_SCOPE) + object replOutput extends ReplOutput(settings.Yreploutdir) { } + + @deprecated("Use replOutput.dir instead", "2.11.0") + def virtualDirectory = replOutput.dir + // Used in a test case. + def showDirectory() = replOutput.show(out) + + private[nsc] var printResults = true // whether to print result lines + private[nsc] var totalSilence = false // whether to print anything + private var _initializeComplete = false // compiler is initialized + private var _isInitialized: Future[Boolean] = null // set up initialization future + private var bindExceptions = true // whether to bind the lastException variable + private var _executionWrapper = "" // code to be wrapped around all lines + + /** We're going to go to some trouble to initialize the compiler asynchronously. + * It's critical that nothing call into it until it's been initialized or we will + * run into unrecoverable issues, but the perceived repl startup time goes + * through the roof if we wait for it. So we initialize it with a future and + * use a lazy val to ensure that any attempt to use the compiler object waits + * on the future. + */ + private var _classLoader: util.AbstractFileClassLoader = null // active classloader + private val _compiler: ReplGlobal = newCompiler(settings, reporter) // our private compiler + + def compilerClasspath: Seq[java.net.URL] = ( + if (isInitializeComplete) global.classPath.asURLs + else new PathResolver(settings).result.asURLs // the compiler's classpath + ) + def settings = initialSettings + // Run the code body with the given boolean settings flipped to true. + def withoutWarnings[T](body: => T): T = beQuietDuring { + val saved = settings.nowarn.value + if (!saved) + settings.nowarn.value = true + + try body + finally if (!saved) settings.nowarn.value = false + } + + /** construct an interpreter that reports to Console */ + def this(settings: Settings, out: JPrintWriter) = this(null, settings, out) + def this(factory: ScriptEngineFactory, settings: Settings) = this(factory, settings, new NewLinePrintWriter(new ConsoleWriter, true)) + def this(settings: Settings) = this(settings, new NewLinePrintWriter(new ConsoleWriter, true)) + def this(factory: ScriptEngineFactory) = this(factory, new Settings()) + def this() = this(new Settings()) + + lazy val formatting: Formatting = new Formatting { + val prompt = Properties.shellPromptString + } + lazy val reporter: SparkReplReporter = new SparkReplReporter(this) + + import formatting._ + import reporter.{ printMessage, printUntruncatedMessage } + + // This exists mostly because using the reporter too early leads to deadlock. + private def echo(msg: String) { Console println msg } + private def _initSources = List(new BatchSourceFile("", "class $repl_$init { }")) + private def _initialize() = { + try { + // if this crashes, REPL will hang its head in shame + val run = new _compiler.Run() + assert(run.typerPhase != NoPhase, "REPL requires a typer phase.") + run compileSources _initSources + _initializeComplete = true + true + } + catch AbstractOrMissingHandler() + } + private def tquoted(s: String) = "\"\"\"" + s + "\"\"\"" + private val logScope = scala.sys.props contains "scala.repl.scope" + private def scopelog(msg: String) = if (logScope) Console.err.println(msg) + + // argument is a thunk to execute after init is done + def initialize(postInitSignal: => Unit) { + synchronized { + if (_isInitialized == null) { + _isInitialized = + Future(try _initialize() finally postInitSignal)(ExecutionContext.global) + } + } + } + def initializeSynchronous(): Unit = { + if (!isInitializeComplete) { + _initialize() + assert(global != null, global) + } + } + def isInitializeComplete = _initializeComplete + + lazy val global: Global = { + if (!isInitializeComplete) _initialize() + _compiler + } + + import global._ + import definitions.{ ObjectClass, termMember, dropNullaryMethod} + + lazy val runtimeMirror = ru.runtimeMirror(classLoader) + + private def noFatal(body: => Symbol): Symbol = try body catch { case _: FatalError => NoSymbol } + + def getClassIfDefined(path: String) = ( + noFatal(runtimeMirror staticClass path) + orElse noFatal(rootMirror staticClass path) + ) + def getModuleIfDefined(path: String) = ( + noFatal(runtimeMirror staticModule path) + orElse noFatal(rootMirror staticModule path) + ) + + implicit class ReplTypeOps(tp: Type) { + def andAlso(fn: Type => Type): Type = if (tp eq NoType) tp else fn(tp) + } + + // TODO: If we try to make naming a lazy val, we run into big time + // scalac unhappiness with what look like cycles. It has not been easy to + // reduce, but name resolution clearly takes different paths. + object naming extends { + val global: imain.global.type = imain.global + } with Naming { + // make sure we don't overwrite their unwisely named res3 etc. + def freshUserTermName(): TermName = { + val name = newTermName(freshUserVarName()) + if (replScope containsName name) freshUserTermName() + else name + } + def isInternalTermName(name: Name) = isInternalVarName("" + name) + } + import naming._ + + object deconstruct extends { + val global: imain.global.type = imain.global + } with StructuredTypeStrings + + lazy val memberHandlers = new { + val intp: imain.type = imain + } with SparkMemberHandlers + import memberHandlers._ + + /** Temporarily be quiet */ + def beQuietDuring[T](body: => T): T = { + val saved = printResults + printResults = false + try body + finally printResults = saved + } + def beSilentDuring[T](operation: => T): T = { + val saved = totalSilence + totalSilence = true + try operation + finally totalSilence = saved + } + + def quietRun[T](code: String) = beQuietDuring(interpret(code)) + + /** takes AnyRef because it may be binding a Throwable or an Exceptional */ + private def withLastExceptionLock[T](body: => T, alt: => T): T = { + assert(bindExceptions, "withLastExceptionLock called incorrectly.") + bindExceptions = false + + try beQuietDuring(body) + catch logAndDiscard("withLastExceptionLock", alt) + finally bindExceptions = true + } + + def executionWrapper = _executionWrapper + def setExecutionWrapper(code: String) = _executionWrapper = code + def clearExecutionWrapper() = _executionWrapper = "" + + /** interpreter settings */ + lazy val isettings = new SparkISettings(this) + + /** Instantiate a compiler. Overridable. */ + protected def newCompiler(settings: Settings, reporter: reporters.Reporter): ReplGlobal = { + settings.outputDirs setSingleOutput replOutput.dir + settings.exposeEmptyPackage.value = true + new Global(settings, reporter) with ReplGlobal { override def toString: String = "" } + } + + /** Parent classloader. Overridable. */ + protected def parentClassLoader: ClassLoader = + settings.explicitParentLoader.getOrElse( this.getClass.getClassLoader() ) + + /* A single class loader is used for all commands interpreted by this Interpreter. + It would also be possible to create a new class loader for each command + to interpret. The advantages of the current approach are: + + - Expressions are only evaluated one time. This is especially + significant for I/O, e.g. "val x = Console.readLine" + + The main disadvantage is: + + - Objects, classes, and methods cannot be rebound. Instead, definitions + shadow the old ones, and old code objects refer to the old + definitions. + */ + def resetClassLoader() = { + repldbg("Setting new classloader: was " + _classLoader) + _classLoader = null + ensureClassLoader() + } + final def ensureClassLoader() { + if (_classLoader == null) + _classLoader = makeClassLoader() + } + def classLoader: util.AbstractFileClassLoader = { + ensureClassLoader() + _classLoader + } + + def backticked(s: String): String = ( + (s split '.').toList map { + case "_" => "_" + case s if nme.keywords(newTermName(s)) => s"`$s`" + case s => s + } mkString "." + ) + def readRootPath(readPath: String) = getModuleIfDefined(readPath) + + abstract class PhaseDependentOps { + def shift[T](op: => T): T + + def path(name: => Name): String = shift(path(symbolOfName(name))) + def path(sym: Symbol): String = backticked(shift(sym.fullName)) + def sig(sym: Symbol): String = shift(sym.defString) + } + object typerOp extends PhaseDependentOps { + def shift[T](op: => T): T = exitingTyper(op) + } + object flatOp extends PhaseDependentOps { + def shift[T](op: => T): T = exitingFlatten(op) + } + + def originalPath(name: String): String = originalPath(name: TermName) + def originalPath(name: Name): String = typerOp path name + def originalPath(sym: Symbol): String = typerOp path sym + def flatPath(sym: Symbol): String = flatOp shift sym.javaClassName + def translatePath(path: String) = { + val sym = if (path endsWith "$") symbolOfTerm(path.init) else symbolOfIdent(path) + sym.toOption map flatPath + } + def translateEnclosingClass(n: String) = symbolOfTerm(n).enclClass.toOption map flatPath + + private class TranslatingClassLoader(parent: ClassLoader) extends util.AbstractFileClassLoader(replOutput.dir, parent) { + /** Overridden here to try translating a simple name to the generated + * class name if the original attempt fails. This method is used by + * getResourceAsStream as well as findClass. + */ + override protected def findAbstractFile(name: String): AbstractFile = + super.findAbstractFile(name) match { + case null if _initializeComplete => translatePath(name) map (super.findAbstractFile(_)) orNull + case file => file + } + } + private def makeClassLoader(): util.AbstractFileClassLoader = + new TranslatingClassLoader(parentClassLoader match { + case null => ScalaClassLoader fromURLs compilerClasspath + case p => new ScalaClassLoader.URLClassLoader(compilerClasspath, p) + }) + + // Set the current Java "context" class loader to this interpreter's class loader + def setContextClassLoader() = classLoader.setAsContext() + + def allDefinedNames: List[Name] = exitingTyper(replScope.toList.map(_.name).sorted) + def unqualifiedIds: List[String] = allDefinedNames map (_.decode) sorted + + /** Most recent tree handled which wasn't wholly synthetic. */ + private def mostRecentlyHandledTree: Option[Tree] = { + prevRequests.reverse foreach { req => + req.handlers.reverse foreach { + case x: MemberDefHandler if x.definesValue && !isInternalTermName(x.name) => return Some(x.member) + case _ => () + } + } + None + } + + private def updateReplScope(sym: Symbol, isDefined: Boolean) { + def log(what: String) { + val mark = if (sym.isType) "t " else "v " + val name = exitingTyper(sym.nameString) + val info = cleanTypeAfterTyper(sym) + val defn = sym defStringSeenAs info + + scopelog(f"[$mark$what%6s] $name%-25s $defn%s") + } + if (ObjectClass isSubClass sym.owner) return + // unlink previous + replScope lookupAll sym.name foreach { sym => + log("unlink") + replScope unlink sym + } + val what = if (isDefined) "define" else "import" + log(what) + replScope enter sym + } + + def recordRequest(req: Request) { + if (req == null) + return + + prevRequests += req + + // warning about serially defining companions. It'd be easy + // enough to just redefine them together but that may not always + // be what people want so I'm waiting until I can do it better. + exitingTyper { + req.defines filterNot (s => req.defines contains s.companionSymbol) foreach { newSym => + val oldSym = replScope lookup newSym.name.companionName + if (Seq(oldSym, newSym).permutations exists { case Seq(s1, s2) => s1.isClass && s2.isModule }) { + replwarn(s"warning: previously defined $oldSym is not a companion to $newSym.") + replwarn("Companions must be defined together; you may wish to use :paste mode for this.") + } + } + } + exitingTyper { + req.imports foreach (sym => updateReplScope(sym, isDefined = false)) + req.defines foreach (sym => updateReplScope(sym, isDefined = true)) + } + } + + private[nsc] def replwarn(msg: => String) { + if (!settings.nowarnings) + printMessage(msg) + } + + def compileSourcesKeepingRun(sources: SourceFile*) = { + val run = new Run() + assert(run.typerPhase != NoPhase, "REPL requires a typer phase.") + reporter.reset() + run compileSources sources.toList + (!reporter.hasErrors, run) + } + + /** Compile an nsc SourceFile. Returns true if there are + * no compilation errors, or false otherwise. + */ + def compileSources(sources: SourceFile*): Boolean = + compileSourcesKeepingRun(sources: _*)._1 + + /** Compile a string. Returns true if there are no + * compilation errors, or false otherwise. + */ + def compileString(code: String): Boolean = + compileSources(new BatchSourceFile("
  • -Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes . Rows are constructed by passing a list of -key/value pairs as kwargs to the Row class. The keys of this list define the columns names of the table, +Spark SQL can convert an RDD of Row objects to a SchemaRDD, inferring the datatypes. Rows are constructed by passing a list of +key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, and the types are inferred by looking at the first row. Since we currently only look at the first -row, it is important that there is no missing data in the first row of the RDD. In future version we +row, it is important that there is no missing data in the first row of the RDD. In future versions we plan to more completely infer the schema by looking at more data, similar to the inference that is performed on JSON files. @@ -306,14 +306,14 @@ for teenName in teenNames.collect():
    -In cases that case classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string or a text dataset will be parsed +When case classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in the step 1. +`Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `applySchema` method provided by `SQLContext`. @@ -358,14 +358,14 @@ results.map(t => "Name: " + t(0)).collect().foreach(println)
    -In cases that JavaBean classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string or a text dataset will be parsed and +When JavaBean classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of `Row`s from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in the step 1. +`Row`s in the RDD created in Step 1. 3. Apply the schema to the RDD of `Row`s via `applySchema` method provided by `JavaSQLContext`. @@ -427,10 +427,10 @@ List names = results.map(new Function() {
    -For some cases (for example, the structure of records is encoded in a string or -a text dataset will be parsed and fields will be projected differently for -different users), it is desired to create `SchemaRDD` with a programmatically way. -It can be done with three steps. +When a dictionary of kwargs cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `SchemaRDD` can be created programmatically with three steps. 1. Create an RDD of tuples or lists from the original RDD; 2. Create the schema represented by a `StructType` matching the structure of @@ -566,7 +566,7 @@ for teenName in teenNames.collect(): ### Configuration -Configuration of parquet can be done using the `setConf` method on SQLContext or by running +Configuration of Parquet can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -575,8 +575,8 @@ Configuration of parquet can be done using the `setConf` method on SQLContext or @@ -584,14 +584,14 @@ Configuration of parquet can be done using the `setConf` method on SQLContext or @@ -805,9 +805,8 @@ Spark SQL can cache tables using an in-memory columnar format by calling `cacheT Then Spark SQL will scan only required columns and will automatically tune compression to minimize memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. -Note that if you just call `cache` rather than `cacheTable`, tables will _not_ be cached in -in-memory columnar format. So we strongly recommend using `cacheTable` whenever you want to -cache tables. +Note that if you call `cache` rather than `cacheTable`, tables will _not_ be cached using +the in-memory columnar format, and therefore `cacheTable` is strongly recommended for this use case. Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -833,7 +832,7 @@ Configuration of in-memory caching can be done using the `setConf` method on SQL
    spark.sql.parquet.binaryAsString false - Some other parquet producing systems, in particular Impala and older versions of Spark SQL, do - not differentiate between binary data and strings when writing out the parquet schema. This + Some other Parquet-producing systems, in particular Impala and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems.
    spark.sql.parquet.cacheMetadata false - Turns on caching of parquet schema metadata. Can speed up querying + Turns on caching of Parquet schema metadata. Can speed up querying of static data.
    spark.sql.parquet.compression.codec snappy - Sets the compression codec use when writing parquet files. Acceptable values include: + Sets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo.
    -## Other Configuration +## Other Configuration Options The following options can also be used to tune the performance of query execution. It is possible that these options will be deprecated in future release as more optimizations are performed automatically. @@ -842,7 +841,7 @@ that these options will be deprecated in future release as more optimizations ar Property NameDefaultMeaning spark.sql.autoBroadcastJoinThreshold - false + 10000 Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently @@ -876,7 +875,7 @@ code. ## Running the Thrift JDBC server The Thrift JDBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 0.12. You can test the JDBC server with the beeline script comes with either Spark or Hive 0.12. +in Hive 0.12. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.12. To start the JDBC server, run the following in the Spark directory: @@ -899,12 +898,12 @@ your machine and a blank password. For secure mode, please follow the instructio Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. -You may also use the beeline script comes with Hive. +You may also use the beeline script that comes with Hive. ## Running the Spark SQL CLI The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from command line. Note: the Spark SQL CLI cannot talk to the Thrift JDBC server. +queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. To start the Spark SQL CLI, run the following in the Spark directory: @@ -916,7 +915,10 @@ options. # Compatibility with Other Systems -## Migration Guide for Shark Users +## Migration Guide for Shark User + +### Scheduling +s To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, users can set the `spark.sql.thriftserver.scheduler.pool` variable: @@ -925,7 +927,7 @@ users can set the `spark.sql.thriftserver.scheduler.pool` variable: ### Reducer number In Shark, default reducer number is 1 and is controlled by the property `mapred.reduce.tasks`. Spark -SQL deprecates this property by a new property `spark.sql.shuffle.partitions`, whose default value +SQL deprecates this property in favor of `spark.sql.shuffle.partitions`, whose default value is 200. Users may customize this property via `SET`: SET spark.sql.shuffle.partitions=10; From 939a322c85956eda150b10afb2ed1d8d959a7bdf Mon Sep 17 00:00:00 2001 From: Matthew Rocklin Date: Mon, 8 Sep 2014 15:45:28 -0700 Subject: [PATCH 0629/1492] [SPARK-3417] Use new-style classes in PySpark Tiny PR making SQLContext a new-style class. This allows various type logic to work more effectively ```Python In [1]: import pyspark In [2]: pyspark.sql.SQLContext.mro() Out[2]: [pyspark.sql.SQLContext, object] ``` Author: Matthew Rocklin Closes #2288 from mrocklin/sqlcontext-new-style-class and squashes the following commits: 4aadab6 [Matthew Rocklin] update other old-style classes a2dc02f [Matthew Rocklin] pyspark.sql.SQLContext is new-style class --- python/pyspark/mllib/random.py | 2 +- python/pyspark/mllib/util.py | 2 +- python/pyspark/sql.py | 2 +- python/pyspark/storagelevel.py | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 3e59c73db85e3..d53c95fd59c25 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -28,7 +28,7 @@ __all__ = ['RandomRDDs', ] -class RandomRDDs: +class RandomRDDs(object): """ Generator methods for creating RDDs comprised of i.i.d samples from some distribution. diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 4962d05491c03..1c7b8c809ab5b 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -25,7 +25,7 @@ from pyspark.serializers import NoOpSerializer -class MLUtils: +class MLUtils(object): """ Helper methods to load, save and pre-process data used in MLlib. diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 004d4937cbe1c..53eea6d6cf3ba 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -899,7 +899,7 @@ def __reduce__(self): return Row -class SQLContext: +class SQLContext(object): """Main entry point for Spark SQL functionality. diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index 2aa0fb9d2c1ed..676aa0f7144aa 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -18,7 +18,7 @@ __all__ = ["StorageLevel"] -class StorageLevel: +class StorageLevel(object): """ Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, From 08ce18881e09c6e91db9c410d1d9ce1e5ae63a62 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 8 Sep 2014 15:59:20 -0700 Subject: [PATCH 0630/1492] [SPARK-3019] Pluggable block transfer interface (BlockTransferService) This pull request creates a new BlockTransferService interface for block fetch/upload and refactors the existing ConnectionManager to implement BlockTransferService (NioBlockTransferService). Most of the changes are simply moving code around. The main class to inspect is ShuffleBlockFetcherIterator. Review guide: - Most of the ConnectionManager code is now in network.cm package - ManagedBuffer is a new buffer abstraction backed by several different implementations (file segment, nio ByteBuffer, Netty ByteBuf) - BlockTransferService is the main internal interface introduced in this PR - NioBlockTransferService implements BlockTransferService and replaces the old BlockManagerWorker - ShuffleBlockFetcherIterator replaces the told BlockFetcherIterator to use the new interface TODOs that should be separate PRs: - Implement NettyBlockTransferService - Finalize the API/semantics for ManagedBuffer.release() Author: Reynold Xin Closes #2240 from rxin/blockTransferService and squashes the following commits: 64cd9d7 [Reynold Xin] Merge branch 'master' into blockTransferService 1dfd3d7 [Reynold Xin] Limit the length of the FileInputStream. 1332156 [Reynold Xin] Fixed style violation from refactoring. 2960c93 [Reynold Xin] Added ShuffleBlockFetcherIteratorSuite. e29c721 [Reynold Xin] Updated comment for ShuffleBlockFetcherIterator. 8a1046e [Reynold Xin] Code review feedback: 2c6b1e1 [Reynold Xin] Removed println in test cases. 2a907e4 [Reynold Xin] Merge branch 'master' into blockTransferService-merge 07ccf0d [Reynold Xin] Added init check to CMBlockTransferService. 98c668a [Reynold Xin] Added failure handling and fixed unit tests. ae05fcd [Reynold Xin] Updated tests, although DistributedSuite is hanging. d8d595c [Reynold Xin] Merge branch 'master' of github.com:apache/spark into blockTransferService 9ef279c [Reynold Xin] Initial refactoring to move ConnectionManager to use the BlockTransferService. --- .../scala/org/apache/spark/SparkEnv.scala | 15 +- ...eiverTest.scala => BlockDataManager.scala} | 29 +- .../spark/network/BlockFetchingListener.scala | 37 +++ .../spark/network/BlockTransferService.scala | 131 +++++++++ .../spark/network/ConnectionManagerTest.scala | 103 ------- .../apache/spark/network/ManagedBuffer.scala | 107 +++++++ .../org/apache/spark/network/SenderTest.scala | 76 ----- .../nio}/BlockMessage.scala | 24 +- .../nio}/BlockMessageArray.scala | 12 +- .../network/{ => nio}/BufferMessage.scala | 5 +- .../spark/network/{ => nio}/Connection.scala | 10 +- .../network/{ => nio}/ConnectionId.scala | 6 +- .../network/{ => nio}/ConnectionManager.scala | 23 +- .../{ => nio}/ConnectionManagerId.scala | 6 +- .../spark/network/{ => nio}/Message.scala | 7 +- .../network/{ => nio}/MessageChunk.scala | 4 +- .../{ => nio}/MessageChunkHeader.scala | 9 +- .../network/nio/NioBlockTransferService.scala | 205 +++++++++++++ .../network/{ => nio}/SecurityMessage.scala | 10 +- .../spark/serializer/KryoSerializer.scala | 2 +- .../shuffle/FileShuffleBlockManager.scala | 35 ++- .../shuffle/IndexShuffleBlockManager.scala | 24 +- .../spark/shuffle/ShuffleBlockManager.scala | 6 +- .../hash/BlockStoreShuffleFetcher.scala | 14 +- .../shuffle/hash/HashShuffleReader.scala | 4 +- .../spark/storage/BlockFetcherIterator.scala | 254 ---------------- .../apache/spark/storage/BlockManager.scala | 98 +++---- .../apache/spark/storage/BlockManagerId.scala | 4 +- .../spark/storage/BlockManagerWorker.scala | 147 ---------- .../storage/ShuffleBlockFetcherIterator.scala | 271 ++++++++++++++++++ .../apache/spark/storage/ThreadingTest.scala | 120 -------- .../org/apache/spark/DistributedSuite.scala | 15 +- .../{ => nio}/ConnectionManagerSuite.scala | 17 +- .../hash/HashShuffleManagerSuite.scala | 17 +- .../storage/BlockFetcherIteratorSuite.scala | 237 --------------- .../spark/storage/BlockManagerSuite.scala | 133 +-------- .../spark/storage/DiskBlockManagerSuite.scala | 2 +- .../ShuffleBlockFetcherIteratorSuite.scala | 183 ++++++++++++ 38 files changed, 1129 insertions(+), 1273 deletions(-) rename core/src/main/scala/org/apache/spark/network/{ReceiverTest.scala => BlockDataManager.scala} (56%) create mode 100644 core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala create mode 100644 core/src/main/scala/org/apache/spark/network/BlockTransferService.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala create mode 100644 core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/SenderTest.scala rename core/src/main/scala/org/apache/spark/{storage => network/nio}/BlockMessage.scala (89%) rename core/src/main/scala/org/apache/spark/{storage => network/nio}/BlockMessageArray.scala (97%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/BufferMessage.scala (98%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/Connection.scala (99%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionId.scala (88%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionManager.scala (98%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/ConnectionManagerId.scala (88%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/Message.scala (95%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/MessageChunk.scala (96%) rename core/src/main/scala/org/apache/spark/network/{ => nio}/MessageChunkHeader.scala (93%) create mode 100644 core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala rename core/src/main/scala/org/apache/spark/network/{ => nio}/SecurityMessage.scala (95%) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala rename core/src/test/scala/org/apache/spark/network/{ => nio}/ConnectionManagerSuite.scala (97%) delete mode 100644 core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 20a7444cfc5ee..dd95e406f2a8e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -31,7 +31,8 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.ConnectionManager +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.{ShuffleMemoryManager, ShuffleManager} @@ -59,8 +60,8 @@ class SparkEnv ( val mapOutputTracker: MapOutputTracker, val shuffleManager: ShuffleManager, val broadcastManager: BroadcastManager, + val blockTransferService: BlockTransferService, val blockManager: BlockManager, - val connectionManager: ConnectionManager, val securityManager: SecurityManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, @@ -88,6 +89,8 @@ class SparkEnv ( // down, but let's call it anyway in case it gets fixed in a later release // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. // actorSystem.awaitTermination() + + // Note that blockTransferService is stopped by BlockManager since it is started by it. } private[spark] @@ -223,14 +226,14 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) + val blockTransferService = new NioBlockTransferService(conf, securityManager) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf, isDriver) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager, mapOutputTracker, shuffleManager) - - val connectionManager = blockManager.connectionManager + serializer, conf, mapOutputTracker, shuffleManager, blockTransferService) val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) @@ -278,8 +281,8 @@ object SparkEnv extends Logging { mapOutputTracker, shuffleManager, broadcastManager, + blockTransferService, blockManager, - connectionManager, securityManager, httpFileServer, sparkFilesDir, diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala similarity index 56% rename from core/src/main/scala/org/apache/spark/network/ReceiverTest.scala rename to core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 53a6038a9b59e..e0e91724271c8 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -17,21 +17,20 @@ package org.apache.spark.network -import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.storage.StorageLevel -private[spark] object ReceiverTest { - def main(args: Array[String]) { - val conf = new SparkConf - val manager = new ConnectionManager(9999, conf, new SecurityManager(conf)) - println("Started connection manager with id = " + manager.id) - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ - val buffer = ByteBuffer.wrap("response".getBytes("utf-8")) - Some(Message.createBufferMessage(buffer, msg.id)) - }) - Thread.currentThread.join() - } -} +trait BlockDataManager { + + /** + * Interface to get local block data. + * + * @return Some(buffer) if the block exists locally, and None if it doesn't. + */ + def getBlockData(blockId: String): Option[ManagedBuffer] + /** + * Put the block locally, using the given storage level. + */ + def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala new file mode 100644 index 0000000000000..34acaa563ca58 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network + +import java.util.EventListener + + +/** + * Listener callback interface for [[BlockTransferService.fetchBlocks]]. + */ +trait BlockFetchingListener extends EventListener { + + /** + * Called once per successfully fetched block. + */ + def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit + + /** + * Called upon failures. For each failure, this is called only once (i.e. not once per block). + */ + def onBlockFetchFailure(exception: Throwable): Unit +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala new file mode 100644 index 0000000000000..84d991fa6808c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network + +import scala.concurrent.{Await, Future} +import scala.concurrent.duration.Duration + +import org.apache.spark.storage.StorageLevel + + +abstract class BlockTransferService { + + /** + * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch + * local blocks or put local blocks. + */ + def init(blockDataManager: BlockDataManager) + + /** + * Tear down the transfer service. + */ + def stop(): Unit + + /** + * Port number the service is listening on, available only after [[init]] is invoked. + */ + def port: Int + + /** + * Host name the service is listening on, available only after [[init]] is invoked. + */ + def hostName: String + + /** + * Fetch a sequence of blocks from a remote node asynchronously, + * available only after [[init]] is invoked. + * + * Note that [[BlockFetchingListener.onBlockFetchSuccess]] is called once per block, + * while [[BlockFetchingListener.onBlockFetchFailure]] is called once per failure (not per block). + * + * Note that this API takes a sequence so the implementation can batch requests, and does not + * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as + * the data of a block is fetched, rather than waiting for all blocks to be fetched. + */ + def fetchBlocks( + hostName: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + */ + def uploadBlock( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel): Future[Unit] + + /** + * A special case of [[fetchBlocks]], as it fetches only one block and is blocking. + * + * It is also only available after [[init]] is invoked. + */ + def fetchBlockSync(hostName: String, port: Int, blockId: String): ManagedBuffer = { + // A monitor for the thread to wait on. + val lock = new Object + @volatile var result: Either[ManagedBuffer, Throwable] = null + fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener { + override def onBlockFetchFailure(exception: Throwable): Unit = { + lock.synchronized { + result = Right(exception) + lock.notify() + } + } + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + lock.synchronized { + result = Left(data) + lock.notify() + } + } + }) + + // Sleep until result is no longer null + lock.synchronized { + while (result == null) { + try { + lock.wait() + } catch { + case e: InterruptedException => + } + } + } + + result match { + case Left(data) => data + case Right(e) => throw e + } + } + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + * + * This method is similar to [[uploadBlock]], except this one blocks the thread + * until the upload finishes. + */ + def uploadBlockSync( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel): Unit = { + Await.result(uploadBlock(hostname, port, blockId, blockData, level), Duration.Inf) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala deleted file mode 100644 index 4894ecd41f6eb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ /dev/null @@ -1,103 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network - -import java.nio.ByteBuffer - -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.io.Source - -import org.apache.spark._ - -private[spark] object ConnectionManagerTest extends Logging{ - def main(args: Array[String]) { - // - the master URL - a list slaves to run connectionTest on - // [num of tasks] - the number of parallel tasks to be initiated default is number of slave - // hosts [size of msg in MB (integer)] - the size of messages to be sent in each task, - // default is 10 [count] - how many times to run, default is 3 [await time in seconds] : - // await time (in seconds), default is 600 - if (args.length < 2) { - println("Usage: ConnectionManagerTest [num of tasks] " + - "[size of msg in MB (integer)] [count] [await time in seconds)] ") - System.exit(1) - } - - if (args(0).startsWith("local")) { - println("This runs only on a mesos cluster") - } - - val sc = new SparkContext(args(0), "ConnectionManagerTest") - val slavesFile = Source.fromFile(args(1)) - val slaves = slavesFile.mkString.split("\n") - slavesFile.close() - - /* println("Slaves") */ - /* slaves.foreach(println) */ - val tasknum = if (args.length > 2) args(2).toInt else slaves.length - val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 - val count = if (args.length > 4) args(4).toInt else 3 - val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second - println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + - "msg size = " + size/1024/1024 + " MB, awaitTime = " + awaitTime) - val slaveConnManagerIds = sc.parallelize(0 until tasknum, tasknum).map( - i => SparkEnv.get.connectionManager.id).collect() - println("\nSlave ConnectionManagerIds") - slaveConnManagerIds.foreach(println) - println - - (0 until count).foreach(i => { - val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { - val connManager = SparkEnv.get.connectionManager - val thisConnManagerId = connManager.id - connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - logInfo("Received [" + msg + "] from [" + id + "]") - None - }) - - val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) - buffer.flip - - val startTime = System.currentTimeMillis - val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => - { - val bufferMessage = Message.createBufferMessage(buffer.duplicate) - logInfo("Sending [" + bufferMessage + "] to [" + slaveConnManagerId + "]") - connManager.sendMessageReliably(slaveConnManagerId, bufferMessage) - } - } - val results = futures.map(f => Await.result(f, awaitTime)) - val finishTime = System.currentTimeMillis - Thread.sleep(5000) - - val mb = size * results.size / 1024.0 / 1024.0 - val ms = finishTime - startTime - val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * - 1000.0) + " MB/s" - logInfo(resultStr) - resultStr - }).collect() - - println("---------------------") - println("Run " + i) - resultStrs.foreach(println) - println("---------------------") - }) - } -} - diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala new file mode 100644 index 0000000000000..dcecb6beeea9b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network + +import java.io.{FileInputStream, RandomAccessFile, File, InputStream} +import java.nio.ByteBuffer +import java.nio.channels.FileChannel.MapMode + +import com.google.common.io.ByteStreams +import io.netty.buffer.{ByteBufInputStream, ByteBuf} + +import org.apache.spark.util.ByteBufferInputStream + + +/** + * This interface provides an immutable view for data in the form of bytes. The implementation + * should specify how the data is provided: + * + * - FileSegmentManagedBuffer: data backed by part of a file + * - NioByteBufferManagedBuffer: data backed by a NIO ByteBuffer + * - NettyByteBufManagedBuffer: data backed by a Netty ByteBuf + */ +sealed abstract class ManagedBuffer { + // Note that all the methods are defined with parenthesis because their implementations can + // have side effects (io operations). + + /** Number of bytes of the data. */ + def size: Long + + /** + * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the + * returned ByteBuffer should not affect the content of this buffer. + */ + def nioByteBuffer(): ByteBuffer + + /** + * Exposes this buffer's data as an InputStream. The underlying implementation does not + * necessarily check for the length of bytes read, so the caller is responsible for making sure + * it does not go over the limit. + */ + def inputStream(): InputStream +} + + +/** + * A [[ManagedBuffer]] backed by a segment in a file + */ +final class FileSegmentManagedBuffer(val file: File, val offset: Long, val length: Long) + extends ManagedBuffer { + + override def size: Long = length + + override def nioByteBuffer(): ByteBuffer = { + val channel = new RandomAccessFile(file, "r").getChannel + channel.map(MapMode.READ_ONLY, offset, length) + } + + override def inputStream(): InputStream = { + val is = new FileInputStream(file) + is.skip(offset) + ByteStreams.limit(is, length) + } +} + + +/** + * A [[ManagedBuffer]] backed by [[java.nio.ByteBuffer]]. + */ +final class NioByteBufferManagedBuffer(buf: ByteBuffer) extends ManagedBuffer { + + override def size: Long = buf.remaining() + + override def nioByteBuffer() = buf.duplicate() + + override def inputStream() = new ByteBufferInputStream(buf) +} + + +/** + * A [[ManagedBuffer]] backed by a Netty [[ByteBuf]]. + */ +final class NettyByteBufManagedBuffer(buf: ByteBuf) extends ManagedBuffer { + + override def size: Long = buf.readableBytes() + + override def nioByteBuffer() = buf.nioBuffer() + + override def inputStream() = new ByteBufInputStream(buf) + + // TODO(rxin): Promote this to top level ManagedBuffer interface and add documentation for it. + def release(): Unit = buf.release() +} diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala deleted file mode 100644 index ea2ad104ecae1..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network - -import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} - -import scala.concurrent.Await -import scala.concurrent.duration.Duration -import scala.util.Try - -private[spark] object SenderTest { - def main(args: Array[String]) { - - if (args.length < 2) { - println("Usage: SenderTest ") - System.exit(1) - } - - val targetHost = args(0) - val targetPort = args(1).toInt - val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - val conf = new SparkConf - val manager = new ConnectionManager(0, conf, new SecurityManager(conf)) - println("Started connection manager with id = " + manager.id) - - manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - println("Received [" + msg + "] from [" + id + "]") - None - }) - - val size = 100 * 1024 * 1024 - val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) - buffer.flip - - val targetServer = args(0) - - val count = 100 - (0 until count).foreach(i => { - val dataMessage = Message.createBufferMessage(buffer.duplicate) - val startTime = System.currentTimeMillis - /* println("Started timer at " + startTime) */ - val promise = manager.sendMessageReliably(targetConnectionManagerId, dataMessage) - val responseStr: String = Try(Await.result(promise, Duration.Inf)) - .map { response => - val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array, "utf-8") - }.getOrElse("none") - - val finishTime = System.currentTimeMillis - val mb = size / 1024.0 / 1024.0 - val ms = finishTime - startTime - // val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms at " + (mb / ms - // * 1000.0) + " MB/s" - val resultStr = "Sent " + mb + " MB " + targetServer + " in " + ms + " ms (" + - (mb / ms * 1000.0).toInt + "MB/s) | Response = " + responseStr - println(resultStr) - }) - } -} - diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala similarity index 89% rename from core/src/main/scala/org/apache/spark/storage/BlockMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala index a2bfce7b4a0fa..b573f1a8a5fcb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessage.scala @@ -15,20 +15,20 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.StringBuilder +import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} -import org.apache.spark.network._ +import scala.collection.mutable.{ArrayBuffer, StringBuilder} +// private[spark] because we need to register them in Kryo private[spark] case class GetBlock(id: BlockId) private[spark] case class GotBlock(id: BlockId, data: ByteBuffer) private[spark] case class PutBlock(id: BlockId, data: ByteBuffer, level: StorageLevel) -private[spark] class BlockMessage() { +private[nio] class BlockMessage() { // Un-initialized: typ = 0 // GetBlock: typ = 1 // GotBlock: typ = 2 @@ -159,7 +159,7 @@ private[spark] class BlockMessage() { } } -private[spark] object BlockMessage { +private[nio] object BlockMessage { val TYPE_NON_INITIALIZED: Int = 0 val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 @@ -194,16 +194,4 @@ private[spark] object BlockMessage { newBlockMessage.set(putBlock) newBlockMessage } - - def main(args: Array[String]) { - val B = new BlockMessage() - val blockId = TestBlockId("ABC") - B.set(new PutBlock(blockId, ByteBuffer.allocate(10), StorageLevel.MEMORY_AND_DISK_SER_2)) - val bMsg = B.toBufferMessage - val C = new BlockMessage() - C.set(bMsg) - - println(B.getId + " " + B.getLevel) - println(C.getId + " " + C.getLevel) - } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala rename to core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala index 973d85c0a9b3a..a1a2c00ed1542 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BlockMessageArray.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer - import org.apache.spark._ -import org.apache.spark.network._ +import org.apache.spark.storage.{StorageLevel, TestBlockId} + +import scala.collection.mutable.ArrayBuffer -private[spark] +private[nio] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { @@ -102,7 +102,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) } } -private[spark] object BlockMessageArray { +private[nio] object BlockMessageArray { def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() diff --git a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/network/BufferMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala index af35f1fc3e459..3b245c5c7a4f3 100644 --- a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/BufferMessage.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer @@ -23,7 +23,8 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.storage.BlockManager -private[spark] + +private[nio] class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: Int) extends Message(Message.BUFFER_MESSAGE, id_) { diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala similarity index 99% rename from core/src/main/scala/org/apache/spark/network/Connection.scala rename to core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 5285ec82c1b64..74074a8dcbfff 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -15,17 +15,17 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net._ import java.nio._ import java.nio.channels._ -import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} - import org.apache.spark._ -private[spark] +import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} + +private[nio] abstract class Connection(val channel: SocketChannel, val selector: Selector, val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId) extends Logging { @@ -190,7 +190,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, } -private[spark] +private[nio] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, remoteId_ : ConnectionManagerId, id_ : ConnectionId) extends Connection(SocketChannel.open, selector_, remoteId_, id_) { diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/network/ConnectionId.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala index d579c165a1917..764dc5e5503ed 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionId.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio -private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { +private[nio] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId } -private[spark] object ConnectionId { +private[nio] object ConnectionId { def createConnectionIdFromString(connectionIdString: String): ConnectionId = { val res = connectionIdString.split("_").map(_.trim()) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala similarity index 98% rename from core/src/main/scala/org/apache/spark/network/ConnectionManager.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 578d806263006..09d3ea306515b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -15,32 +15,27 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.io.IOException +import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ -import java.net._ -import java.util.{Timer, TimerTask} import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.{LinkedBlockingDeque, ThreadPoolExecutor, TimeUnit} +import java.util.{Timer, TimerTask} -import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet -import scala.collection.mutable.SynchronizedMap -import scala.collection.mutable.SynchronizedQueue - -import scala.concurrent.{Await, ExecutionContext, Future, Promise} +import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, SynchronizedMap, SynchronizedQueue} import scala.concurrent.duration._ +import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} -private[spark] class ConnectionManager( + +private[nio] class ConnectionManager( port: Int, conf: SparkConf, securityManager: SecurityManager, @@ -904,7 +899,7 @@ private[spark] class ConnectionManager( private[spark] object ConnectionManager { - import ExecutionContext.Implicits.global + import scala.concurrent.ExecutionContext.Implicits.global def main(args: Array[String]) { val conf = new SparkConf diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala similarity index 88% rename from core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala rename to core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala index 57f7586883af1..cbb37ec5ced1f 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerId.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManagerId.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net.InetSocketAddress import org.apache.spark.util.Utils -private[spark] case class ConnectionManagerId(host: String, port: Int) { +private[nio] case class ConnectionManagerId(host: String, port: Int) { // DEBUG code Utils.checkHost(host) assert (port > 0) @@ -30,7 +30,7 @@ private[spark] case class ConnectionManagerId(host: String, port: Int) { } -private[spark] object ConnectionManagerId { +private[nio] object ConnectionManagerId { def fromSocketAddress(socketAddress: InetSocketAddress): ConnectionManagerId = { new ConnectionManagerId(socketAddress.getHostName, socketAddress.getPort) } diff --git a/core/src/main/scala/org/apache/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/network/Message.scala rename to core/src/main/scala/org/apache/spark/network/nio/Message.scala index 04ea50f62918c..0b874c2891255 100644 --- a/core/src/main/scala/org/apache/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -15,14 +15,15 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.net.InetSocketAddress import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -private[spark] abstract class Message(val typ: Long, val id: Int) { + +private[nio] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null var started = false var startTime = -1L @@ -42,7 +43,7 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { } -private[spark] object Message { +private[nio] object Message { val BUFFER_MESSAGE = 1111111111L var lastId = 1 diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/network/MessageChunk.scala rename to core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala index d0f986a12bfe0..278c5ac356ef2 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunk.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunk.scala @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -private[network] +private[nio] class MessageChunk(val header: MessageChunkHeader, val buffer: ByteBuffer) { val size = if (buffer == null) 0 else buffer.remaining diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala rename to core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala index f3ecca5f992e0..6e20f291c5cec 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/MessageChunkHeader.scala @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio -import java.net.InetAddress -import java.net.InetSocketAddress +import java.net.{InetAddress, InetSocketAddress} import java.nio.ByteBuffer -private[spark] class MessageChunkHeader( +private[nio] class MessageChunkHeader( val typ: Long, val id: Int, val totalSize: Int, @@ -57,7 +56,7 @@ private[spark] class MessageChunkHeader( } -private[spark] object MessageChunkHeader { +private[nio] object MessageChunkHeader { val HEADER_SIZE = 45 def create(buffer: ByteBuffer): MessageChunkHeader = { diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala new file mode 100644 index 0000000000000..59958ee894230 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.nio + +import java.nio.ByteBuffer + +import scala.concurrent.Future + +import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} +import org.apache.spark.network._ +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.Utils + + +/** + * A [[BlockTransferService]] implementation based on [[ConnectionManager]], a custom + * implementation using Java NIO. + */ +final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityManager) + extends BlockTransferService with Logging { + + private var cm: ConnectionManager = _ + + private var blockDataManager: BlockDataManager = _ + + /** + * Port number the service is listening on, available only after [[init]] is invoked. + */ + override def port: Int = { + checkInit() + cm.id.port + } + + /** + * Host name the service is listening on, available only after [[init]] is invoked. + */ + override def hostName: String = { + checkInit() + cm.id.host + } + + /** + * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch + * local blocks or put local blocks. + */ + override def init(blockDataManager: BlockDataManager): Unit = { + this.blockDataManager = blockDataManager + cm = new ConnectionManager( + conf.getInt("spark.blockManager.port", 0), + conf, + securityManager, + "Connection manager for block manager") + cm.onReceiveMessage(onBlockMessageReceive) + } + + /** + * Tear down the transfer service. + */ + override def stop(): Unit = { + if (cm != null) { + cm.stop() + } + } + + override def fetchBlocks( + hostName: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit = { + checkInit() + + val cmId = new ConnectionManagerId(hostName, port) + val blockMessageArray = new BlockMessageArray(blockIds.map { blockId => + BlockMessage.fromGetBlock(GetBlock(BlockId(blockId))) + }) + + val future = cm.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) + + // Register the listener on success/failure future callback. + future.onSuccess { case message => + val bufferMessage = message.asInstanceOf[BufferMessage] + val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + + for (blockMessage <- blockMessageArray) { + if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { + listener.onBlockFetchFailure( + new SparkException(s"Unexpected message ${blockMessage.getType} received from $cmId")) + } else { + val blockId = blockMessage.getId + val networkSize = blockMessage.getData.limit() + listener.onBlockFetchSuccess( + blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + } + } + }(cm.futureExecContext) + + future.onFailure { case exception => + listener.onBlockFetchFailure(exception) + }(cm.futureExecContext) + } + + /** + * Upload a single block to a remote node, available only after [[init]] is invoked. + * + * This call blocks until the upload completes, or throws an exception upon failures. + */ + override def uploadBlock( + hostname: String, + port: Int, + blockId: String, + blockData: ManagedBuffer, + level: StorageLevel) + : Future[Unit] = { + checkInit() + val msg = PutBlock(BlockId(blockId), blockData.nioByteBuffer(), level) + val blockMessageArray = new BlockMessageArray(BlockMessage.fromPutBlock(msg)) + val remoteCmId = new ConnectionManagerId(hostName, port) + val reply = cm.sendMessageReliably(remoteCmId, blockMessageArray.toBufferMessage) + reply.map(x => ())(cm.futureExecContext) + } + + private def checkInit(): Unit = if (cm == null) { + throw new IllegalStateException(getClass.getName + " has not been initialized") + } + + private def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { + logDebug("Handling message " + msg) + msg match { + case bufferMessage: BufferMessage => + try { + logDebug("Handling as a buffer message " + bufferMessage) + val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage) + logDebug("Parsed as a block message array") + val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) + Some(new BlockMessageArray(responseMessages).toBufferMessage) + } catch { + case e: Exception => { + logError("Exception handling buffer message", e) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } + } + + case otherMessage: Any => + logError("Unknown type message received: " + otherMessage) + val errorMessage = Message.createBufferMessage(msg.id) + errorMessage.hasError = true + Some(errorMessage) + } + } + + private def processBlockMessage(blockMessage: BlockMessage): Option[BlockMessage] = { + blockMessage.getType match { + case BlockMessage.TYPE_PUT_BLOCK => + val msg = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) + logDebug("Received [" + msg + "]") + putBlock(msg.id.toString, msg.data, msg.level) + None + + case BlockMessage.TYPE_GET_BLOCK => + val msg = new GetBlock(blockMessage.getId) + logDebug("Received [" + msg + "]") + val buffer = getBlock(msg.id.toString) + if (buffer == null) { + return None + } + Some(BlockMessage.fromGotBlock(GotBlock(msg.id, buffer))) + + case _ => None + } + } + + private def putBlock(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + val startTimeMs = System.currentTimeMillis() + logDebug("PutBlock " + blockId + " started from " + startTimeMs + " with data: " + bytes) + blockDataManager.putBlockData(blockId, new NioByteBufferManagedBuffer(bytes), level) + logDebug("PutBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + + " with data size: " + bytes.limit) + } + + private def getBlock(blockId: String): ByteBuffer = { + val startTimeMs = System.currentTimeMillis() + logDebug("GetBlock " + blockId + " started from " + startTimeMs) + val buffer = blockDataManager.getBlockData(blockId).orNull + logDebug("GetBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + + " and got buffer " + buffer) + buffer.nioByteBuffer() + } +} diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala similarity index 95% rename from core/src/main/scala/org/apache/spark/network/SecurityMessage.scala rename to core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala index 9af9e2e8e9e59..747a2088a7258 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/SecurityMessage.scala @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.StringBuilder +import scala.collection.mutable.{ArrayBuffer, StringBuilder} import org.apache.spark._ -import org.apache.spark.network._ /** * SecurityMessage is class that contains the connectionId and sasl token @@ -54,7 +52,7 @@ import org.apache.spark.network._ * - Length of the token * - Token */ -private[spark] class SecurityMessage() extends Logging { +private[nio] class SecurityMessage extends Logging { private var connectionId: String = null private var token: Array[Byte] = null @@ -134,7 +132,7 @@ private[spark] class SecurityMessage() extends Logging { } } -private[spark] object SecurityMessage { +private[nio] object SecurityMessage { /** * Convert the given BufferMessage to a SecurityMessage by parsing the contents diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 87ef9bb0b43c6..d6386f8c06fff 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -27,9 +27,9 @@ import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.spark._ import org.apache.spark.broadcast.HttpBroadcast +import org.apache.spark.network.nio.{PutBlock, GotBlock, GetBlock} import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage._ -import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} import org.apache.spark.util.BoundedPriorityQueue import org.apache.spark.util.collection.CompactBuffer diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 96facccd52373..439981d232349 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -26,6 +26,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.{SparkEnv, SparkConf, Logging} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup import org.apache.spark.storage._ @@ -166,34 +167,30 @@ class FileShuffleBlockManager(conf: SparkConf) } } - /** - * Returns the physical file segment in which the given BlockId is located. - */ - private def getBlockLocation(id: ShuffleBlockId): FileSegment = { + override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { + val segment = getBlockData(blockId) + Some(segment.nioByteBuffer()) + } + + override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { if (consolidateShuffleFiles) { // Search all file groups associated with this shuffle. - val shuffleState = shuffleStates(id.shuffleId) + val shuffleState = shuffleStates(blockId.shuffleId) val iter = shuffleState.allFileGroups.iterator while (iter.hasNext) { - val segment = iter.next.getFileSegmentFor(id.mapId, id.reduceId) - if (segment.isDefined) { return segment.get } + val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId) + if (segmentOpt.isDefined) { + val segment = segmentOpt.get + return new FileSegmentManagedBuffer(segment.file, segment.offset, segment.length) + } } - throw new IllegalStateException("Failed to find shuffle block: " + id) + throw new IllegalStateException("Failed to find shuffle block: " + blockId) } else { - val file = blockManager.diskBlockManager.getFile(id) - new FileSegment(file, 0, file.length()) + val file = blockManager.diskBlockManager.getFile(blockId) + new FileSegmentManagedBuffer(file, 0, file.length) } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockLocation(blockId) - blockManager.diskStore.getBytes(segment) - } - - override def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] = { - Left(getBlockLocation(blockId.asInstanceOf[ShuffleBlockId])) - } - /** Remove all the blocks / files and metadata related to a particular shuffle. */ def removeShuffle(shuffleId: ShuffleId): Boolean = { // Do not change the ordering of this, if shuffleStates should be removed only diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 8bb9efc46cc58..4ab34336d3f01 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkEnv +import org.apache.spark.network.{ManagedBuffer, FileSegmentManagedBuffer} import org.apache.spark.storage._ /** @@ -89,10 +90,11 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { } } - /** - * Get the location of a block in a map output file. Uses the index file we create for it. - * */ - private def getBlockLocation(blockId: ShuffleBlockId): FileSegment = { + override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { + Some(getBlockData(blockId).nioByteBuffer()) + } + + override def getBlockData(blockId: ShuffleBlockId): ManagedBuffer = { // The block is actually going to be a range of a single map output file for this map, so // find out the consolidated file, then the offset within that from our index val indexFile = getIndexFile(blockId.shuffleId, blockId.mapId) @@ -102,20 +104,14 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { in.skip(blockId.reduceId * 8) val offset = in.readLong() val nextOffset = in.readLong() - new FileSegment(getDataFile(blockId.shuffleId, blockId.mapId), offset, nextOffset - offset) + new FileSegmentManagedBuffer( + getDataFile(blockId.shuffleId, blockId.mapId), + offset, + nextOffset - offset) } finally { in.close() } } - override def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] = { - val segment = getBlockLocation(blockId) - blockManager.diskStore.getBytes(segment) - } - - override def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] = { - Left(getBlockLocation(blockId.asInstanceOf[ShuffleBlockId])) - } - override def stop() = {} } diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala index 4240580250046..63863cc0250a3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala @@ -19,7 +19,8 @@ package org.apache.spark.shuffle import java.nio.ByteBuffer -import org.apache.spark.storage.{FileSegment, ShuffleBlockId} +import org.apache.spark.network.ManagedBuffer +import org.apache.spark.storage.ShuffleBlockId private[spark] trait ShuffleBlockManager { @@ -31,8 +32,7 @@ trait ShuffleBlockManager { */ def getBytes(blockId: ShuffleBlockId): Option[ByteBuffer] - def getBlockData(blockId: ShuffleBlockId): Either[FileSegment, ByteBuffer] + def getBlockData(blockId: ShuffleBlockId): ManagedBuffer def stop(): Unit } - diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 12b475658e29d..6cf9305977a3c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -21,10 +21,9 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.ShuffleReadMetrics import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId} +import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} import org.apache.spark.util.CompletionIterator private[hash] object BlockStoreShuffleFetcher extends Logging { @@ -32,8 +31,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { shuffleId: Int, reduceId: Int, context: TaskContext, - serializer: Serializer, - shuffleMetrics: ShuffleReadMetrics) + serializer: Serializer) : Iterator[T] = { logDebug("Fetching outputs for shuffle %d, reduce %d".format(shuffleId, reduceId)) @@ -74,7 +72,13 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { } } - val blockFetcherItr = blockManager.getMultiple(blocksByAddress, serializer, shuffleMetrics) + val blockFetcherItr = new ShuffleBlockFetcherIterator( + context, + SparkEnv.get.blockTransferService, + blockManager, + blocksByAddress, + serializer, + SparkEnv.get.conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024) val itr = blockFetcherItr.flatMap(unpackBlock) val completionIter = CompletionIterator[T, Iterator[T]](itr, { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 7bed97a63f0f6..88a5f1e5ddf58 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -36,10 +36,8 @@ private[spark] class HashShuffleReader[K, C]( /** Read the combined key-values for this reduce task */ override def read(): Iterator[Product2[K, C]] = { - val readMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() val ser = Serializer.getSerializer(dep.serializer) - val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser, - readMetrics) + val iter = BlockStoreShuffleFetcher.fetch(handle.shuffleId, startPartition, context, ser) val aggregatedIter: Iterator[Product2[K, C]] = if (dep.aggregator.isDefined) { if (dep.mapSideCombine) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala deleted file mode 100644 index e35b7fe62c753..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ /dev/null @@ -1,254 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import java.util.concurrent.LinkedBlockingQueue -import org.apache.spark.network.netty.client.{BlockClientListener, LazyInitIterator, ReferenceCountedBuffer} - -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet -import scala.collection.mutable.Queue -import scala.util.{Failure, Success} - -import org.apache.spark.{Logging, SparkException} -import org.apache.spark.executor.ShuffleReadMetrics -import org.apache.spark.network.BufferMessage -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils - -/** - * A block fetcher iterator interface for fetching shuffle blocks. - */ -private[storage] -trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { - def initialize() -} - - -private[storage] -object BlockFetcherIterator { - - /** - * A request to fetch blocks from a remote BlockManager. - * @param address remote BlockManager to fetch from. - * @param blocks Sequence of tuple, where the first element is the block id, - * and the second element is the estimated size, used to calculate bytesInFlight. - */ - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { - val size = blocks.map(_._2).sum - } - - /** - * Result of a fetch from a remote block. A failure is represented as size == -1. - * @param blockId block id - * @param size estimated size of the block, used to calculate bytesInFlight. - * Note that this is NOT the exact bytes. - * @param deserialize closure to return the result in the form of an Iterator. - */ - class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { - def failed: Boolean = size == -1 - } - - // TODO: Refactor this whole thing to make code more reusable. - class BasicBlockFetcherIterator( - private val blockManager: BlockManager, - val blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer, - readMetrics: ShuffleReadMetrics) - extends BlockFetcherIterator { - - import blockManager._ - - if (blocksByAddress == null) { - throw new IllegalArgumentException("BlocksByAddress is null") - } - - // Total number blocks fetched (local + remote). Also number of FetchResults expected - protected var _numBlocksToFetch = 0 - - protected var startTime = System.currentTimeMillis - - // BlockIds for local blocks that need to be fetched. Excludes zero-sized blocks - protected val localBlocksToFetch = new ArrayBuffer[BlockId]() - - // BlockIds for remote blocks that need to be fetched. Excludes zero-sized blocks - protected val remoteBlocksToFetch = new HashSet[BlockId]() - - // A queue to hold our results. - protected val results = new LinkedBlockingQueue[FetchResult] - - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight - protected val fetchRequests = new Queue[FetchRequest] - - // Current bytes in flight from our requests - protected var bytesInFlight = 0L - - protected def sendRequest(req: FetchRequest) { - logDebug("Sending request for %d blocks (%s) from %s".format( - req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) - val cmId = new ConnectionManagerId(req.address.host, req.address.port) - val blockMessageArray = new BlockMessageArray(req.blocks.map { - case (blockId, size) => BlockMessage.fromGetBlock(GetBlock(blockId)) - }) - bytesInFlight += req.size - val sizeMap = req.blocks.toMap // so we can look up the size of each blockID - val future = connectionManager.sendMessageReliably(cmId, blockMessageArray.toBufferMessage) - future.onComplete { - case Success(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - throw new SparkException( - "Unexpected message " + blockMessage.getType + " received from " + cmId) - } - val blockId = blockMessage.getId - val networkSize = blockMessage.getData.limit() - results.put(new FetchResult(blockId, sizeMap(blockId), - () => dataDeserialize(blockId, blockMessage.getData, serializer))) - // TODO: NettyBlockFetcherIterator has some race conditions where multiple threads can - // be incrementing bytes read at the same time (SPARK-2625). - readMetrics.remoteBytesRead += networkSize - readMetrics.remoteBlocksFetched += 1 - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) - } - } - case Failure(exception) => { - logError("Could not get block(s) from " + cmId, exception) - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } - } - } - } - - protected def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { - // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them - // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 - // nodes, rather than blocking on reading output from one node. - val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) - - // Split local and remote blocks. Remote blocks are further split into FetchRequests of size - // at most maxBytesInFlight in order to limit the amount of data in flight. - val remoteRequests = new ArrayBuffer[FetchRequest] - var totalBlocks = 0 - for ((address, blockInfos) <- blocksByAddress) { - totalBlocks += blockInfos.size - if (address == blockManagerId) { - // Filter out zero-sized blocks - localBlocksToFetch ++= blockInfos.filter(_._2 != 0).map(_._1) - _numBlocksToFetch += localBlocksToFetch.size - } else { - val iterator = blockInfos.iterator - var curRequestSize = 0L - var curBlocks = new ArrayBuffer[(BlockId, Long)] - while (iterator.hasNext) { - val (blockId, size) = iterator.next() - // Skip empty blocks - if (size > 0) { - curBlocks += ((blockId, size)) - remoteBlocksToFetch += blockId - _numBlocksToFetch += 1 - curRequestSize += size - } else if (size < 0) { - throw new BlockException(blockId, "Negative block size " + size) - } - if (curRequestSize >= targetRequestSize) { - // Add this FetchRequest - remoteRequests += new FetchRequest(address, curBlocks) - curBlocks = new ArrayBuffer[(BlockId, Long)] - logDebug(s"Creating fetch request of $curRequestSize at $address") - curRequestSize = 0 - } - } - // Add in the final request - if (!curBlocks.isEmpty) { - remoteRequests += new FetchRequest(address, curBlocks) - } - } - } - logInfo("Getting " + _numBlocksToFetch + " non-empty blocks out of " + - totalBlocks + " blocks") - remoteRequests - } - - protected def getLocalBlocks() { - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - for (id <- localBlocksToFetch) { - try { - readMetrics.localBlocksFetched += 1 - results.put(new FetchResult(id, 0, () => getLocalShuffleFromDisk(id, serializer).get)) - logDebug("Got local block " + id) - } catch { - case e: Exception => { - logError(s"Error occurred while fetching local blocks", e) - results.put(new FetchResult(id, -1, null)) - return - } - } - } - } - - override def initialize() { - // Split local and remote blocks. - val remoteRequests = splitLocalRemoteBlocks() - // Add the remote requests into our queue in a random order - fetchRequests ++= Utils.randomize(remoteRequests) - - // Send out initial requests for blocks, up to our maxBytesInFlight - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - - val numFetches = remoteRequests.size - fetchRequests.size - logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) - - // Get Local Blocks - startTime = System.currentTimeMillis - getLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") - } - - // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue - // as they arrive. - @volatile protected var resultsGotten = 0 - - override def hasNext: Boolean = resultsGotten < _numBlocksToFetch - - override def next(): (BlockId, Option[Iterator[Any]]) = { - resultsGotten += 1 - val startFetchWait = System.currentTimeMillis() - val result = results.take() - val stopFetchWait = System.currentTimeMillis() - readMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) - if (! result.failed) bytesInFlight -= result.size - while (!fetchRequests.isEmpty && - (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { - sendRequest(fetchRequests.dequeue()) - } - (result.blockId, if (result.failed) None else Some(result.deserialize())) - } - } - // End of BasicBlockFetcherIterator -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a714142763243..d1bee3d2c033c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -20,6 +20,8 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} +import scala.concurrent.ExecutionContext.Implicits.global + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ @@ -58,18 +60,14 @@ private[spark] class BlockManager( defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, - shuffleManager: ShuffleManager) - extends BlockDataProvider with Logging { + shuffleManager: ShuffleManager, + blockTransferService: BlockTransferService) + extends BlockDataManager with Logging { - private val port = conf.getInt("spark.blockManager.port", 0) + blockTransferService.init(this) val diskBlockManager = new DiskBlockManager(this, conf) - val connectionManager = - new ConnectionManager(port, conf, securityManager, "Connection manager for block manager") - - implicit val futureExecContext = connectionManager.futureExecContext private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] @@ -89,11 +87,7 @@ private[spark] class BlockManager( } val blockManagerId = BlockManagerId( - executorId, connectionManager.id.host, connectionManager.id.port) - - // Max megabytes of data to keep in flight per reducer (to avoid over-allocating memory - // for receiving shuffle outputs) - val maxBytesInFlight = conf.getLong("spark.reducer.maxMbInFlight", 48) * 1024 * 1024 + executorId, blockTransferService.hostName, blockTransferService.port) // Whether to compress broadcast variables that are stored private val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) @@ -136,11 +130,11 @@ private[spark] class BlockManager( master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, - securityManager: SecurityManager, mapOutputTracker: MapOutputTracker, - shuffleManager: ShuffleManager) = { + shuffleManager: ShuffleManager, + blockTransferService: BlockTransferService) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager, mapOutputTracker, shuffleManager) + conf, mapOutputTracker, shuffleManager, blockTransferService) } /** @@ -149,7 +143,6 @@ private[spark] class BlockManager( */ private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) - BlockManagerWorker.startBlockManagerWorker(this) } /** @@ -212,20 +205,33 @@ private[spark] class BlockManager( } } - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { + /** + * Interface to get local block data. + * + * @return Some(buffer) if the block exists locally, and None if it doesn't. + */ + override def getBlockData(blockId: String): Option[ManagedBuffer] = { val bid = BlockId(blockId) if (bid.isShuffle) { - shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId]) + Some(shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId])) } else { val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] if (blockBytesOpt.isDefined) { - Right(blockBytesOpt.get) + val buffer = blockBytesOpt.get + Some(new NioByteBufferManagedBuffer(buffer)) } else { - throw new BlockNotFoundException(blockId) + None } } } + /** + * Put the block locally, using the given storage level. + */ + override def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit = { + putBytes(BlockId(blockId), data.nioByteBuffer(), level) + } + /** * Get the BlockStatus for the block identified by the given ID, if it exists. * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. @@ -333,16 +339,10 @@ private[spark] class BlockManager( * shuffle blocks. It is safe to do so without a lock on block info since disk store * never deletes (recent) items. */ - def getLocalShuffleFromDisk( - blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - - val shuffleBlockManager = shuffleManager.shuffleBlockManager - val values = shuffleBlockManager.getBytes(blockId.asInstanceOf[ShuffleBlockId]).map( - bytes => this.dataDeserialize(blockId, bytes, serializer)) - - values.orElse { - throw new BlockException(blockId, s"Block $blockId not found on disk, though it should be") - } + def getLocalShuffleFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { + val buf = shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) + val is = wrapForCompression(blockId, buf.inputStream()) + Some(serializer.newInstance().deserializeStream(is).asIterator) } /** @@ -513,8 +513,9 @@ private[spark] class BlockManager( val locations = Random.shuffle(master.getLocations(blockId)) for (loc <- locations) { logDebug(s"Getting remote block $blockId from $loc") - val data = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(loc.host, loc.port)) + val data = blockTransferService.fetchBlockSync( + loc.host, loc.port, blockId.toString).nioByteBuffer() + if (data != null) { if (asBlockResult) { return Some(new BlockResult( @@ -548,22 +549,6 @@ private[spark] class BlockManager( None } - /** - * Get multiple blocks from local and remote block manager using their BlockManagerIds. Returns - * an Iterator of (block ID, value) pairs so that clients may handle blocks in a pipelined - * fashion as they're received. Expects a size in bytes to be provided for each block fetched, - * so that we can control the maxMegabytesInFlight for the fetch. - */ - def getMultiple( - blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], - serializer: Serializer, - readMetrics: ShuffleReadMetrics): BlockFetcherIterator = { - val iter = new BlockFetcherIterator.BasicBlockFetcherIterator(this, blocksByAddress, serializer, - readMetrics) - iter.initialize() - iter - } - def putIterator( blockId: BlockId, values: Iterator[Any], @@ -816,12 +801,15 @@ private[spark] class BlockManager( data.rewind() logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + s"To node: $peer") - val putBlock = PutBlock(blockId, data, tLevel) - val cmId = new ConnectionManagerId(peer.host, peer.port) - val syncPutBlockSuccess = BlockManagerWorker.syncPutBlock(putBlock, cmId) - if (!syncPutBlockSuccess) { - logError(s"Failed to call syncPutBlock to $peer") + + try { + blockTransferService.uploadBlockSync( + peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) + } catch { + case e: Exception => + logError(s"Failed to replicate block to $peer", e) } + logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) } @@ -1051,7 +1039,7 @@ private[spark] class BlockManager( } def stop(): Unit = { - connectionManager.stop() + blockTransferService.stop() diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index b7bcb2d85d0ee..d4487fce49ab6 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -36,8 +36,8 @@ import org.apache.spark.util.Utils class BlockManagerId private ( private var executorId_ : String, private var host_ : String, - private var port_ : Int - ) extends Externalizable { + private var port_ : Int) + extends Externalizable { private def this() = this(null, null, 0) // For deserialization only diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala deleted file mode 100644 index bf002a42d5dc5..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerWorker.scala +++ /dev/null @@ -1,147 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import java.nio.ByteBuffer - -import org.apache.spark.Logging -import org.apache.spark.network._ -import org.apache.spark.util.Utils - -import scala.concurrent.Await -import scala.concurrent.duration.Duration -import scala.util.{Try, Failure, Success} - -/** - * A network interface for BlockManager. Each slave should have one - * BlockManagerWorker. - * - * TODO: Use event model. - */ -private[spark] class BlockManagerWorker(val blockManager: BlockManager) extends Logging { - - blockManager.connectionManager.onReceiveMessage(onBlockMessageReceive) - - def onBlockMessageReceive(msg: Message, id: ConnectionManagerId): Option[Message] = { - logDebug("Handling message " + msg) - msg match { - case bufferMessage: BufferMessage => { - try { - logDebug("Handling as a buffer message " + bufferMessage) - val blockMessages = BlockMessageArray.fromBufferMessage(bufferMessage) - logDebug("Parsed as a block message array") - val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) - Some(new BlockMessageArray(responseMessages).toBufferMessage) - } catch { - case e: Exception => { - logError("Exception handling buffer message", e) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) - } - } - } - case otherMessage: Any => { - logError("Unknown type message received: " + otherMessage) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) - } - } - } - - def processBlockMessage(blockMessage: BlockMessage): Option[BlockMessage] = { - blockMessage.getType match { - case BlockMessage.TYPE_PUT_BLOCK => { - val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) - logDebug("Received [" + pB + "]") - putBlock(pB.id, pB.data, pB.level) - None - } - case BlockMessage.TYPE_GET_BLOCK => { - val gB = new GetBlock(blockMessage.getId) - logDebug("Received [" + gB + "]") - val buffer = getBlock(gB.id) - if (buffer == null) { - return None - } - Some(BlockMessage.fromGotBlock(GotBlock(gB.id, buffer))) - } - case _ => None - } - } - - private def putBlock(id: BlockId, bytes: ByteBuffer, level: StorageLevel) { - val startTimeMs = System.currentTimeMillis() - logDebug("PutBlock " + id + " started from " + startTimeMs + " with data: " + bytes) - blockManager.putBytes(id, bytes, level) - logDebug("PutBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) - + " with data size: " + bytes.limit) - } - - private def getBlock(id: BlockId): ByteBuffer = { - val startTimeMs = System.currentTimeMillis() - logDebug("GetBlock " + id + " started from " + startTimeMs) - val buffer = blockManager.getLocalBytes(id) match { - case Some(bytes) => bytes - case None => null - } - logDebug("GetBlock " + id + " used " + Utils.getUsedTimeMs(startTimeMs) - + " and got buffer " + buffer) - buffer - } -} - -private[spark] object BlockManagerWorker extends Logging { - private var blockManagerWorker: BlockManagerWorker = null - - def startBlockManagerWorker(manager: BlockManager) { - blockManagerWorker = new BlockManagerWorker(manager) - } - - def syncPutBlock(msg: PutBlock, toConnManagerId: ConnectionManagerId): Boolean = { - val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val blockMessage = BlockMessage.fromPutBlock(msg) - val blockMessageArray = new BlockMessageArray(blockMessage) - val resultMessage = Try(Await.result(connectionManager.sendMessageReliably( - toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) - resultMessage.isSuccess - } - - def syncGetBlock(msg: GetBlock, toConnManagerId: ConnectionManagerId): ByteBuffer = { - val blockManager = blockManagerWorker.blockManager - val connectionManager = blockManager.connectionManager - val blockMessage = BlockMessage.fromGetBlock(msg) - val blockMessageArray = new BlockMessageArray(blockMessage) - val responseMessage = Try(Await.result(connectionManager.sendMessageReliably( - toConnManagerId, blockMessageArray.toBufferMessage), Duration.Inf)) - responseMessage match { - case Success(message) => { - val bufferMessage = message.asInstanceOf[BufferMessage] - logDebug("Response message received " + bufferMessage) - BlockMessageArray.fromBufferMessage(bufferMessage).foreach(blockMessage => { - logDebug("Found " + blockMessage) - return blockMessage.getData - }) - } - case Failure(exception) => logDebug("No response message received") - } - null - } -} diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala new file mode 100644 index 0000000000000..c8e708aa6b1bc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -0,0 +1,271 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashSet +import scala.collection.mutable.Queue + +import org.apache.spark.{TaskContext, Logging, SparkException} +import org.apache.spark.network.{ManagedBuffer, BlockFetchingListener, BlockTransferService} +import org.apache.spark.serializer.Serializer +import org.apache.spark.util.Utils + + +/** + * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block + * manager. For remote blocks, it fetches them using the provided BlockTransferService. + * + * This creates an iterator of (BlockID, values) tuples so the caller can handle blocks in a + * pipelined fashion as they are received. + * + * The implementation throttles the remote fetches to they don't exceed maxBytesInFlight to avoid + * using too much memory. + * + * @param context [[TaskContext]], used for metrics update + * @param blockTransferService [[BlockTransferService]] for fetching remote blocks + * @param blockManager [[BlockManager]] for reading local blocks + * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. + * For each block we also require the size (in bytes as a long field) in + * order to throttle the memory usage. + * @param serializer serializer used to deserialize the data. + * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. + */ +private[spark] +final class ShuffleBlockFetcherIterator( + context: TaskContext, + blockTransferService: BlockTransferService, + blockManager: BlockManager, + blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], + serializer: Serializer, + maxBytesInFlight: Long) + extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { + + import ShuffleBlockFetcherIterator._ + + /** + * Total number of blocks to fetch. This can be smaller than the total number of blocks + * in [[blocksByAddress]] because we filter out zero-sized blocks in [[initialize]]. + * + * This should equal localBlocks.size + remoteBlocks.size. + */ + private[this] var numBlocksToFetch = 0 + + /** + * The number of blocks proccessed by the caller. The iterator is exhausted when + * [[numBlocksProcessed]] == [[numBlocksToFetch]]. + */ + private[this] var numBlocksProcessed = 0 + + private[this] val startTime = System.currentTimeMillis + + /** Local blocks to fetch, excluding zero-sized blocks. */ + private[this] val localBlocks = new ArrayBuffer[BlockId]() + + /** Remote blocks to fetch, excluding zero-sized blocks. */ + private[this] val remoteBlocks = new HashSet[BlockId]() + + /** + * A queue to hold our results. This turns the asynchronous model provided by + * [[BlockTransferService]] into a synchronous model (iterator). + */ + private[this] val results = new LinkedBlockingQueue[FetchResult] + + // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + // the number of bytes in flight is limited to maxBytesInFlight + private[this] val fetchRequests = new Queue[FetchRequest] + + // Current bytes in flight from our requests + private[this] var bytesInFlight = 0L + + private[this] val shuffleMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + + initialize() + + private[this] def sendRequest(req: FetchRequest) { + logDebug("Sending request for %d blocks (%s) from %s".format( + req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) + bytesInFlight += req.size + + // so we can look up the size of each blockID + val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap + val blockIds = req.blocks.map(_._1.toString) + + blockTransferService.fetchBlocks(req.address.host, req.address.port, blockIds, + new BlockFetchingListener { + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { + results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), + () => serializer.newInstance().deserializeStream( + blockManager.wrapForCompression(BlockId(blockId), data.inputStream())).asIterator + )) + shuffleMetrics.remoteBytesRead += data.size + shuffleMetrics.remoteBlocksFetched += 1 + logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + } + + override def onBlockFetchFailure(e: Throwable): Unit = { + logError("Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + // Note that there is a chance that some blocks have been fetched successfully, but we + // still add them to the failed queue. This is fine because when the caller see a + // FetchFailedException, it is going to fail the entire task anyway. + for ((blockId, size) <- req.blocks) { + results.put(new FetchResult(blockId, -1, null)) + } + } + } + ) + } + + private[this] def splitLocalRemoteBlocks(): ArrayBuffer[FetchRequest] = { + // Make remote requests at most maxBytesInFlight / 5 in length; the reason to keep them + // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 + // nodes, rather than blocking on reading output from one node. + val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) + logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + + // Split local and remote blocks. Remote blocks are further split into FetchRequests of size + // at most maxBytesInFlight in order to limit the amount of data in flight. + val remoteRequests = new ArrayBuffer[FetchRequest] + + // Tracks total number of blocks (including zero sized blocks) + var totalBlocks = 0 + for ((address, blockInfos) <- blocksByAddress) { + totalBlocks += blockInfos.size + if (address == blockManager.blockManagerId) { + // Filter out zero-sized blocks + localBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) + numBlocksToFetch += localBlocks.size + } else { + val iterator = blockInfos.iterator + var curRequestSize = 0L + var curBlocks = new ArrayBuffer[(BlockId, Long)] + while (iterator.hasNext) { + val (blockId, size) = iterator.next() + // Skip empty blocks + if (size > 0) { + curBlocks += ((blockId, size)) + remoteBlocks += blockId + numBlocksToFetch += 1 + curRequestSize += size + } else if (size < 0) { + throw new BlockException(blockId, "Negative block size " + size) + } + if (curRequestSize >= targetRequestSize) { + // Add this FetchRequest + remoteRequests += new FetchRequest(address, curBlocks) + curBlocks = new ArrayBuffer[(BlockId, Long)] + logDebug(s"Creating fetch request of $curRequestSize at $address") + curRequestSize = 0 + } + } + // Add in the final request + if (curBlocks.nonEmpty) { + remoteRequests += new FetchRequest(address, curBlocks) + } + } + } + logInfo(s"Getting $numBlocksToFetch non-empty blocks out of $totalBlocks blocks") + remoteRequests + } + + private[this] def fetchLocalBlocks() { + // Get the local blocks while remote blocks are being fetched. Note that it's okay to do + // these all at once because they will just memory-map some files, so they won't consume + // any memory that might exceed our maxBytesInFlight + for (id <- localBlocks) { + try { + shuffleMetrics.localBlocksFetched += 1 + results.put(new FetchResult( + id, 0, () => blockManager.getLocalShuffleFromDisk(id, serializer).get)) + logDebug("Got local block " + id) + } catch { + case e: Exception => + logError(s"Error occurred while fetching local blocks", e) + results.put(new FetchResult(id, -1, null)) + return + } + } + } + + private[this] def initialize(): Unit = { + // Split local and remote blocks. + val remoteRequests = splitLocalRemoteBlocks() + // Add the remote requests into our queue in a random order + fetchRequests ++= Utils.randomize(remoteRequests) + + // Send out initial requests for blocks, up to our maxBytesInFlight + while (fetchRequests.nonEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + + val numFetches = remoteRequests.size - fetchRequests.size + logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) + + // Get Local Blocks + fetchLocalBlocks() + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + } + + override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch + + override def next(): (BlockId, Option[Iterator[Any]]) = { + numBlocksProcessed += 1 + val startFetchWait = System.currentTimeMillis() + val result = results.take() + val stopFetchWait = System.currentTimeMillis() + shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) + if (!result.failed) { + bytesInFlight -= result.size + } + // Send fetch requests up to maxBytesInFlight + while (fetchRequests.nonEmpty && + (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { + sendRequest(fetchRequests.dequeue()) + } + (result.blockId, if (result.failed) None else Some(result.deserialize())) + } +} + + +private[storage] +object ShuffleBlockFetcherIterator { + + /** + * A request to fetch blocks from a remote BlockManager. + * @param address remote BlockManager to fetch from. + * @param blocks Sequence of tuple, where the first element is the block id, + * and the second element is the estimated size, used to calculate bytesInFlight. + */ + class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { + val size = blocks.map(_._2).sum + } + + /** + * Result of a fetch from a remote block. A failure is represented as size == -1. + * @param blockId block id + * @param size estimated size of the block, used to calculate bytesInFlight. + * Note that this is NOT the exact bytes. + * @param deserialize closure to return the result in the form of an Iterator. + */ + class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { + def failed: Boolean = size == -1 + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala deleted file mode 100644 index 7540f0d5e2a5a..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ /dev/null @@ -1,120 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import java.util.concurrent.ArrayBlockingQueue - -import akka.actor._ -import org.apache.spark.shuffle.hash.HashShuffleManager -import util.Random - -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} -import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.serializer.KryoSerializer - -/** - * This class tests the BlockManager and MemoryStore for thread safety and - * deadlocks. It spawns a number of producer and consumer threads. Producer - * threads continuously pushes blocks into the BlockManager and consumer - * threads continuously retrieves the blocks form the BlockManager and tests - * whether the block is correct or not. - */ -private[spark] object ThreadingTest { - - val numProducers = 5 - val numBlocksPerProducer = 20000 - - private[spark] class ProducerThread(manager: BlockManager, id: Int) extends Thread { - val queue = new ArrayBlockingQueue[(BlockId, Seq[Int])](100) - - override def run() { - for (i <- 1 to numBlocksPerProducer) { - val blockId = TestBlockId("b-" + id + "-" + i) - val blockSize = Random.nextInt(1000) - val block = (1 to blockSize).map(_ => Random.nextInt()) - val level = randomLevel() - val startTime = System.currentTimeMillis() - manager.putIterator(blockId, block.iterator, level, tellMaster = true) - println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") - queue.add((blockId, block)) - } - println("Producer thread " + id + " terminated") - } - - def randomLevel(): StorageLevel = { - math.abs(Random.nextInt()) % 4 match { - case 0 => StorageLevel.MEMORY_ONLY - case 1 => StorageLevel.MEMORY_ONLY_SER - case 2 => StorageLevel.MEMORY_AND_DISK - case 3 => StorageLevel.MEMORY_AND_DISK_SER - } - } - } - - private[spark] class ConsumerThread( - manager: BlockManager, - queue: ArrayBlockingQueue[(BlockId, Seq[Int])] - ) extends Thread { - var numBlockConsumed = 0 - - override def run() { - println("Consumer thread started") - while(numBlockConsumed < numBlocksPerProducer) { - val (blockId, block) = queue.take() - val startTime = System.currentTimeMillis() - manager.get(blockId) match { - case Some(retrievedBlock) => - assert(retrievedBlock.data.toList.asInstanceOf[List[Int]] == block.toList, - "Block " + blockId + " did not match") - println("Got block " + blockId + " in " + - (System.currentTimeMillis - startTime) + " ms") - case None => - assert(false, "Block " + blockId + " could not be retrieved") - } - numBlockConsumed += 1 - } - println("Consumer thread terminated") - } - } - - def main(args: Array[String]) { - System.setProperty("spark.kryoserializer.buffer.mb", "1") - val actorSystem = ActorSystem("test") - val conf = new SparkConf() - val serializer = new KryoSerializer(conf) - val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), - conf, true) - val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf), new MapOutputTrackerMaster(conf), new HashShuffleManager(conf)) - val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) - val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) - producers.foreach(_.start) - consumers.foreach(_.start) - producers.foreach(_.join) - consumers.foreach(_.join) - blockManager.stop() - blockManagerMaster.stop() - actorSystem.shutdown() - actorSystem.awaitTermination() - println("Everything stopped.") - println( - "It will take sometime for the JVM to clean all temporary files and shutdown. Sit tight.") - } -} diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 41c294f727b3c..81b64c36ddca1 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -24,8 +24,7 @@ import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} import org.apache.spark.SparkContext._ -import org.apache.spark.network.ConnectionManagerId -import org.apache.spark.storage.{BlockManagerWorker, GetBlock, RDDBlockId, StorageLevel} +import org.apache.spark.storage.{RDDBlockId, StorageLevel} class NotSerializableClass class NotSerializableExn(val notSer: NotSerializableClass) extends Throwable() {} @@ -136,7 +135,6 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter sc.parallelize(1 to 10, 2).foreach { x => if (x == 1) System.exit(42) } } assert(thrown.getClass === classOf[SparkException]) - System.out.println(thrown.getMessage) assert(thrown.getMessage.contains("failed 4 times")) } } @@ -202,12 +200,13 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter val blockIds = data.partitions.indices.map(index => RDDBlockId(data.id, index)).toArray val blockId = blockIds(0) val blockManager = SparkEnv.get.blockManager - blockManager.master.getLocations(blockId).foreach(id => { - val bytes = BlockManagerWorker.syncGetBlock( - GetBlock(blockId), ConnectionManagerId(id.host, id.port)) - val deserialized = blockManager.dataDeserialize(blockId, bytes).asInstanceOf[Iterator[Int]].toList + val blockTransfer = SparkEnv.get.blockTransferService + blockManager.master.getLocations(blockId).foreach { cmId => + val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, blockId.toString) + val deserialized = blockManager.dataDeserialize(blockId, bytes.nioByteBuffer()) + .asInstanceOf[Iterator[Int]].toList assert(deserialized === (1 to 100).toList) - }) + } } test("compute without caching when no partitions fit in memory") { diff --git a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala similarity index 97% rename from core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala rename to core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index e2f4d4c57cdb5..9f49587cdc670 100644 --- a/core/src/test/scala/org/apache/spark/network/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -15,23 +15,18 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.nio import java.io.IOException import java.nio._ -import java.util.concurrent.TimeoutException -import org.apache.spark.{SecurityManager, SparkConf} -import org.scalatest.FunSuite - -import org.mockito.Mockito._ -import org.mockito.Matchers._ - -import scala.concurrent.TimeoutException -import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration._ +import scala.concurrent.{Await, TimeoutException} import scala.language.postfixOps -import scala.util.{Failure, Success, Try} + +import org.scalatest.FunSuite + +import org.apache.spark.{SecurityManager, SparkConf} /** * Test the ConnectionManager with various security settings. diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index 6061e544e79b4..ba47fe5e25b9b 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -25,6 +25,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkEnv, SparkContext, LocalSparkContext, SparkConf} import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.shuffle.FileShuffleBlockManager import org.apache.spark.storage.{ShuffleBlockId, FileSegment} @@ -32,10 +33,12 @@ import org.apache.spark.storage.{ShuffleBlockId, FileSegment} class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { private val testConf = new SparkConf(false) - private def checkSegments(segment1: FileSegment, segment2: FileSegment) { - assert (segment1.file.getCanonicalPath === segment2.file.getCanonicalPath) - assert (segment1.offset === segment2.offset) - assert (segment1.length === segment2.length) + private def checkSegments(expected: FileSegment, buffer: ManagedBuffer) { + assert(buffer.isInstanceOf[FileSegmentManagedBuffer]) + val segment = buffer.asInstanceOf[FileSegmentManagedBuffer] + assert(expected.file.getCanonicalPath === segment.file.getCanonicalPath) + assert(expected.offset === segment.offset) + assert(expected.length === segment.length) } test("consolidated shuffle can write to shuffle group without messing existing offsets/lengths") { @@ -95,14 +98,12 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { writer.commitAndClose() } // check before we register. - checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0)).left.get) + checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0))) shuffle3.releaseWriters(success = true) - checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0)).left.get) + checkSegments(shuffle2Segment, shuffleBlockManager.getBlockData(ShuffleBlockId(1, 2, 0))) shuffleBlockManager.removeShuffle(1) - } - def writeToFile(file: File, numBytes: Int) { val writer = new FileWriter(file, true) for (i <- 0 until numBytes) writer.write(i) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala deleted file mode 100644 index 3c86f6bafcaa3..0000000000000 --- a/core/src/test/scala/org/apache/spark/storage/BlockFetcherIteratorSuite.scala +++ /dev/null @@ -1,237 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import java.io.IOException -import java.nio.ByteBuffer - -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.future -import scala.concurrent.ExecutionContext.Implicits.global - -import org.scalatest.{FunSuite, Matchers} - -import org.mockito.Mockito._ -import org.mockito.Matchers.{any, eq => meq} -import org.mockito.stubbing.Answer -import org.mockito.invocation.InvocationOnMock - -import org.apache.spark.storage.BlockFetcherIterator._ -import org.apache.spark.network.{ConnectionManager, Message} -import org.apache.spark.executor.ShuffleReadMetrics - -class BlockFetcherIteratorSuite extends FunSuite with Matchers { - - test("block fetch from local fails using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - doReturn(connManager).when(blockManager).connectionManager - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - val answer = new Answer[Option[Iterator[Any]]] { - override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { - throw new Exception - } - } - - // 3rd block is going to fail - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) - - val bmId = BlockManagerId("test-client", "test-client", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, - new ShuffleReadMetrics()) - - iterator.initialize() - - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - // the 2nd element of the tuple returned by iterator.next should be defined when fetching successfully - assert(iterator.next()._2.isDefined, "1st element should be defined but is not actually defined") - verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next()._2.isDefined, "2nd element should be defined but is not actually defined") - verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - // 3rd fetch should be failed - intercept[Exception] { - iterator.next() - } - verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) - } - - - test("block fetch from local succeed using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - doReturn(connManager).when(blockManager).connectionManager - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - doReturn((48 * 1024 * 1024).asInstanceOf[Long]).when(blockManager).maxBytesInFlight - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - - // All blocks should be fetched successfully - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) - - val bmId = BlockManagerId("test-client", "test-client", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, blocksByAddress, null, - new ShuffleReadMetrics()) - - iterator.initialize() - - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 1st element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next._2.isDefined, "All elements should be defined but 2nd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 3rd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 4th element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") - assert(iterator.next._2.isDefined, "All elements should be defined but 5th element is not actually defined") - - verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) - } - - test("block fetch from remote fails using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - when(blockManager.connectionManager).thenReturn(connManager) - - val f = future { - throw new IOException("Send failed or we received an error ACK") - } - when(connManager.sendMessageReliably(any(), - any())).thenReturn(f) - when(blockManager.futureExecContext).thenReturn(global) - - when(blockManager.blockManagerId).thenReturn( - BlockManagerId("test-client", "test-client", 1)) - when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) - - val blId1 = ShuffleBlockId(0,0,0) - val blId2 = ShuffleBlockId(0,1,0) - val bmId = BlockManagerId("test-server", "test-server", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L))) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null, new ShuffleReadMetrics()) - - iterator.initialize() - iterator.foreach{ - case (_, r) => { - (!r.isDefined) should be(true) - } - } - } - - test("block fetch from remote succeed using BasicBlockFetcherIterator") { - val blockManager = mock(classOf[BlockManager]) - val connManager = mock(classOf[ConnectionManager]) - when(blockManager.connectionManager).thenReturn(connManager) - - val blId1 = ShuffleBlockId(0,0,0) - val blId2 = ShuffleBlockId(0,1,0) - val buf1 = ByteBuffer.allocate(4) - val buf2 = ByteBuffer.allocate(4) - buf1.putInt(1) - buf1.flip() - buf2.putInt(1) - buf2.flip() - val blockMessage1 = BlockMessage.fromGotBlock(GotBlock(blId1, buf1)) - val blockMessage2 = BlockMessage.fromGotBlock(GotBlock(blId2, buf2)) - val blockMessageArray = new BlockMessageArray( - Seq(blockMessage1, blockMessage2)) - - val bufferMessage = blockMessageArray.toBufferMessage - val buffer = ByteBuffer.allocate(bufferMessage.size) - val arrayBuffer = new ArrayBuffer[ByteBuffer] - bufferMessage.buffers.foreach{ b => - buffer.put(b) - } - buffer.flip() - arrayBuffer += buffer - - val f = future { - Message.createBufferMessage(arrayBuffer) - } - when(connManager.sendMessageReliably(any(), - any())).thenReturn(f) - when(blockManager.futureExecContext).thenReturn(global) - - when(blockManager.blockManagerId).thenReturn( - BlockManagerId("test-client", "test-client", 1)) - when(blockManager.maxBytesInFlight).thenReturn(48 * 1024 * 1024) - - val bmId = BlockManagerId("test-server", "test-server", 1) - val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L))) - ) - - val iterator = new BasicBlockFetcherIterator(blockManager, - blocksByAddress, null, new ShuffleReadMetrics()) - iterator.initialize() - iterator.foreach{ - case (_, r) => { - (r.isDefined) should be(true) - } - } - } -} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index c200654162268..e251660dae5de 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,15 +21,19 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import java.util.concurrent.TimeUnit +import org.apache.spark.network.nio.NioBlockTransferService + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.Await +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + import akka.actor._ import akka.pattern.ask import akka.util.Timeout -import org.apache.spark.shuffle.hash.HashShuffleManager -import org.mockito.invocation.InvocationOnMock -import org.mockito.Matchers.any -import org.mockito.Mockito.{doAnswer, mock, spy, when} -import org.mockito.stubbing.Answer +import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ @@ -38,18 +42,12 @@ import org.scalatest.Matchers import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod -import org.apache.spark.network.{Message, ConnectionManagerId} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.storage.BlockManagerMessages.BlockManagerHeartbeat import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.Await -import scala.concurrent.duration._ -import scala.language.implicitConversions -import scala.language.postfixOps -import org.apache.spark.shuffle.ShuffleBlockManager class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter with PrivateMethodTester { @@ -74,8 +72,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { - new BlockManager(name, actorSystem, master, serializer, maxMem, conf, securityMgr, - mapOutputTracker, shuffleManager) + val transfer = new NioBlockTransferService(conf, securityMgr) + new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + mapOutputTracker, shuffleManager, transfer) } before { @@ -793,8 +792,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. + val transfer = new NioBlockTransferService(conf, securityMgr) store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) + mapOutputTracker, shuffleManager, transfer) // The put should fail since a1 is not serializable. class UnserializableClass @@ -1005,109 +1005,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") } - test("return error message when error occurred in BlockManagerWorker#onBlockMessageReceive") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) - - val worker = spy(new BlockManagerWorker(store)) - val connManagerId = mock(classOf[ConnectionManagerId]) - - // setup request block messages - val reqBlId1 = ShuffleBlockId(0,0,0) - val reqBlId2 = ShuffleBlockId(0,1,0) - val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) - val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) - val reqBlockMessages = new BlockMessageArray( - Seq(reqBlockMessage1, reqBlockMessage2)) - val reqBufferMessage = reqBlockMessages.toBufferMessage - - val answer = new Answer[Option[BlockMessage]] { - override def answer(invocation: InvocationOnMock) - :Option[BlockMessage]= { - throw new Exception - } - } - - doAnswer(answer).when(worker).processBlockMessage(any()) - - // Test when exception was thrown during processing block messages - var ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) - - assert(ackMessage.isDefined, "When Exception was thrown in " + - "BlockManagerWorker#processBlockMessage, " + - "ackMessage should be defined") - assert(ackMessage.get.hasError, "When Exception was thown in " + - "BlockManagerWorker#processBlockMessage, " + - "ackMessage should have error") - - val notBufferMessage = mock(classOf[Message]) - - // Test when not BufferMessage was received - ackMessage = worker.onBlockMessageReceive(notBufferMessage, connManagerId) - assert(ackMessage.isDefined, "When not BufferMessage was passed to " + - "BlockManagerWorker#onBlockMessageReceive, " + - "ackMessage should be defined") - assert(ackMessage.get.hasError, "When not BufferMessage was passed to " + - "BlockManagerWorker#onBlockMessageReceive, " + - "ackMessage should have error") - } - - test("return ack message when no error occurred in BlocManagerWorker#onBlockMessageReceive") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, - securityMgr, mapOutputTracker, shuffleManager) - - val worker = spy(new BlockManagerWorker(store)) - val connManagerId = mock(classOf[ConnectionManagerId]) - - // setup request block messages - val reqBlId1 = ShuffleBlockId(0,0,0) - val reqBlId2 = ShuffleBlockId(0,1,0) - val reqBlockMessage1 = BlockMessage.fromGetBlock(GetBlock(reqBlId1)) - val reqBlockMessage2 = BlockMessage.fromGetBlock(GetBlock(reqBlId2)) - val reqBlockMessages = new BlockMessageArray( - Seq(reqBlockMessage1, reqBlockMessage2)) - - val tmpBufferMessage = reqBlockMessages.toBufferMessage - val buffer = ByteBuffer.allocate(tmpBufferMessage.size) - val arrayBuffer = new ArrayBuffer[ByteBuffer] - tmpBufferMessage.buffers.foreach{ b => - buffer.put(b) - } - buffer.flip() - arrayBuffer += buffer - val reqBufferMessage = Message.createBufferMessage(arrayBuffer) - - // setup ack block messages - val buf1 = ByteBuffer.allocate(4) - val buf2 = ByteBuffer.allocate(4) - buf1.putInt(1) - buf1.flip() - buf2.putInt(1) - buf2.flip() - val ackBlockMessage1 = BlockMessage.fromGotBlock(GotBlock(reqBlId1, buf1)) - val ackBlockMessage2 = BlockMessage.fromGotBlock(GotBlock(reqBlId2, buf2)) - - val answer = new Answer[Option[BlockMessage]] { - override def answer(invocation: InvocationOnMock) - :Option[BlockMessage]= { - if (invocation.getArguments()(0).asInstanceOf[BlockMessage].eq( - reqBlockMessage1)) { - return Some(ackBlockMessage1) - } else { - return Some(ackBlockMessage2) - } - } - } - - doAnswer(answer).when(worker).processBlockMessage(any()) - - val ackMessage = worker.onBlockMessageReceive(reqBufferMessage, connManagerId) - assert(ackMessage.isDefined, "When BlockManagerWorker#onBlockMessageReceive " + - "was executed successfully, ackMessage should be defined") - assert(!ackMessage.get.hasError, "When BlockManagerWorker#onBlockMessageReceive " + - "was executed successfully, ackMessage should not have error") - } - test("reserve/release unroll memory") { store = makeBlockManager(12000) val memoryStore = store.memoryStore diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 26082ded8ca7a..e4522e00a622d 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.{File, FileWriter} +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.shuffle.hash.HashShuffleManager import scala.collection.mutable @@ -52,7 +53,6 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before rootDir1 = Files.createTempDir() rootDir1.deleteOnExit() rootDirs = rootDir0.getAbsolutePath + "," + rootDir1.getAbsolutePath - println("Created root dirs: " + rootDirs) } override def afterAll() { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala new file mode 100644 index 0000000000000..809bd70929656 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import org.apache.spark.TaskContext +import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} + +import org.mockito.Mockito._ +import org.mockito.Matchers.{any, eq => meq} +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer + +import org.scalatest.FunSuite + + +class ShuffleBlockFetcherIteratorSuite extends FunSuite { + + test("handle local read failures in BlockManager") { + val transfer = mock(classOf[BlockTransferService]) + val blockManager = mock(classOf[BlockManager]) + doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + val answer = new Answer[Option[Iterator[Any]]] { + override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { + throw new Exception + } + } + + // 3rd block is going to fail + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) + doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + // Without exhausting the iterator, the iterator should be lazy and not call + // getLocalShuffleFromDisk. + verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + // the 2nd element of the tuple returned by iterator.next should be defined when + // fetching successfully + assert(iterator.next()._2.isDefined, + "1st element should be defined but is not actually defined") + verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next()._2.isDefined, + "2nd element should be defined but is not actually defined") + verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + // 3rd fetch should be failed + intercept[Exception] { + iterator.next() + } + verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) + } + + test("handle local read successes") { + val transfer = mock(classOf[BlockTransferService]) + val blockManager = mock(classOf[BlockManager]) + doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId + + val blIds = Array[BlockId]( + ShuffleBlockId(0,0,0), + ShuffleBlockId(0,1,0), + ShuffleBlockId(0,2,0), + ShuffleBlockId(0,3,0), + ShuffleBlockId(0,4,0)) + + val optItr = mock(classOf[Option[Iterator[Any]]]) + + // All blocks should be fetched successfully + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) + doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + + val bmId = BlockManagerId("test-client", "test-client", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + ) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. + verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) + + assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 1st element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 2nd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 3rd element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 4th element is not actually defined") + assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") + assert(iterator.next()._2.isDefined, + "All elements should be defined but 5th element is not actually defined") + + verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) + } + + test("handle remote fetch failures in BlockTransferService") { + val transfer = mock(classOf[BlockTransferService]) + when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + listener.onBlockFetchFailure(new Exception("blah")) + } + }) + + val blockManager = mock(classOf[BlockManager]) + + when(blockManager.blockManagerId).thenReturn(BlockManagerId("test-client", "test-client", 1)) + + val blId1 = ShuffleBlockId(0, 0, 0) + val blId2 = ShuffleBlockId(0, 1, 0) + val bmId = BlockManagerId("test-server", "test-server", 1) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (bmId, Seq((blId1, 1L), (blId2, 1L)))) + + val iterator = new ShuffleBlockFetcherIterator( + new TaskContext(0, 0, 0), + transfer, + blockManager, + blocksByAddress, + null, + 48 * 1024 * 1024) + + iterator.foreach { case (_, iterOption) => + assert(!iterOption.isDefined) + } + } +} From 7db53391f1b349d1f49844197b34f94806f5e336 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Mon, 8 Sep 2014 16:14:32 -0700 Subject: [PATCH 0631/1492] [SPARK-3349][SQL] Output partitioning of limit should not be inherited from child This resolves https://issues.apache.org/jira/browse/SPARK-3349 Author: Eric Liang Closes #2262 from ericl/spark-3349 and squashes the following commits: 3e1b05c [Eric Liang] add regression test ac32723 [Eric Liang] make limit/takeOrdered output SinglePartition --- .../spark/sql/execution/basicOperators.scala | 4 +++- .../org/apache/spark/sql/SQLQuerySuite.scala | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 47bff0c730b8a..cac376608be29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -27,7 +27,7 @@ import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, OrderedDistribution, SinglePartition, UnspecifiedDistribution} import org.apache.spark.util.MutablePair /** @@ -100,6 +100,7 @@ case class Limit(limit: Int, child: SparkPlan) private def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] override def output = child.output + override def outputPartitioning = SinglePartition /** * A custom implementation modeled after the take function on RDDs but which never runs any job @@ -173,6 +174,7 @@ case class Limit(limit: Int, child: SparkPlan) case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) extends UnaryNode { override def output = child.output + override def outputPartitioning = SinglePartition val ordering = new RowOrdering(sortOrder, child.output) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1ac205937714c..e8fbc28d0ad60 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -359,6 +359,23 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (null, null, 6, "F") :: Nil) } + test("SPARK-3349 partitioning after limit") { + sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") + .limit(2) + .registerTempTable("subset1") + sql("SELECT DISTINCT n FROM lowerCaseData") + .limit(2) + .registerTempTable("subset2") + checkAnswer( + sql("SELECT * FROM lowerCaseData INNER JOIN subset1 ON subset1.n = lowerCaseData.n"), + (3, "c", 3) :: + (4, "d", 4) :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), + (1, "a", 1) :: + (2, "b", 2) :: Nil) + } + test("mixed-case keywords") { checkAnswer( sql( From 50a4fa774a0e8a17d7743b33ce8941bf4041144d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 8 Sep 2014 18:59:57 -0700 Subject: [PATCH 0632/1492] [SPARK-3443][MLLIB] update default values of tree: Adjust the default values of decision tree, based on the memory requirement discussed in https://github.com/apache/spark/pull/2125 : 1. maxMemoryInMB: 128 -> 256 2. maxBins: 100 -> 32 3. maxDepth: 4 -> 5 (in some example code) jkbradley Author: Xiangrui Meng Closes #2322 from mengxr/tree-defaults and squashes the following commits: cda453a [Xiangrui Meng] fix tests 5900445 [Xiangrui Meng] update comments 8c81831 [Xiangrui Meng] update default values of tree: --- docs/mllib-decision-tree.md | 16 ++++++++-------- .../spark/examples/mllib/JavaDecisionTree.java | 2 +- .../examples/mllib/DecisionTreeRunner.scala | 4 ++-- .../apache/spark/mllib/tree/DecisionTree.scala | 8 ++++---- .../mllib/tree/configuration/Strategy.scala | 6 +++--- .../spark/mllib/tree/DecisionTreeSuite.scala | 18 ++++-------------- python/pyspark/mllib/tree.py | 4 ++-- 7 files changed, 24 insertions(+), 34 deletions(-) diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 1166d9cd150c4..12a6afbeea829 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -80,7 +80,7 @@ The ordered splits create "bins" and the maximum number of such bins can be specified using the `maxBins` parameter. Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario -since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of +since the default `maxBins` value is 32). The tree algorithm automatically reduces the number of bins if the condition is not satisfied. **Categorical features** @@ -117,7 +117,7 @@ all nodes at each level of the tree. This could lead to high memory requirements of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` training parameter specifies the maximum amount of memory at the workers (twice as much at the master) to be allocated to the histogram computation. The default value is conservatively chosen to -be 128 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements +be 256 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each subsequent level are split into smaller tasks. @@ -167,7 +167,7 @@ val numClasses = 2 val categoricalFeaturesInfo = Map[Int, Int]() val impurity = "gini" val maxDepth = 5 -val maxBins = 100 +val maxBins = 32 val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, maxDepth, maxBins) @@ -213,7 +213,7 @@ Integer numClasses = 2; HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; -Integer maxBins = 100; +Integer maxBins = 32; // Train a DecisionTree model for classification. final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, @@ -250,7 +250,7 @@ data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, - impurity='gini', maxDepth=5, maxBins=100) + impurity='gini', maxDepth=5, maxBins=32) # Evaluate model on training instances and compute training error predictions = model.predict(data.map(lambda x: x.features)) @@ -293,7 +293,7 @@ val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache val categoricalFeaturesInfo = Map[Int, Int]() val impurity = "variance" val maxDepth = 5 -val maxBins = 100 +val maxBins = 32 val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, maxDepth, maxBins) @@ -338,7 +338,7 @@ JavaSparkContext sc = new JavaSparkContext(sparkConf); HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "variance"; Integer maxDepth = 5; -Integer maxBins = 100; +Integer maxBins = 32; // Train a DecisionTree model. final DecisionTreeModel model = DecisionTree.trainRegressor(data, @@ -380,7 +380,7 @@ data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, - impurity='variance', maxDepth=5, maxBins=100) + impurity='variance', maxDepth=5, maxBins=32) # Evaluate model on training instances and compute training error predictions = model.predict(data.map(lambda x: x.features)) diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java index e4468e8bf1744..1f82e3f4cb18e 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaDecisionTree.java @@ -63,7 +63,7 @@ public static void main(String[] args) { HashMap categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; - Integer maxBins = 100; + Integer maxBins = 32; // Train a DecisionTree model for classification. final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index cf3d2cca81ff6..72c3ab475b61f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -52,9 +52,9 @@ object DecisionTreeRunner { input: String = null, dataFormat: String = "libsvm", algo: Algo = Classification, - maxDepth: Int = 4, + maxDepth: Int = 5, impurity: ImpurityType = Gini, - maxBins: Int = 100, + maxBins: Int = 32, fracTest: Double = 0.2) def main(args: Array[String]) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index dd766c12d28a4..d1309b2b20f54 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -330,9 +330,9 @@ object DecisionTree extends Serializable with Logging { * Supported values: "gini" (recommended) or "entropy". * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) + * (suggested value: 5) * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) + * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction */ def trainClassifier( @@ -374,9 +374,9 @@ object DecisionTree extends Serializable with Logging { * Supported values: "variance". * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * (suggested value: 4) + * (suggested value: 5) * @param maxBins maximum number of bins used for splitting features - * (suggested value: 100) + * (suggested value: 32) * @return DecisionTreeModel that can be used for prediction */ def trainRegressor( diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index cfc8192a85abd..23f74d5360fe5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -50,7 +50,7 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is - * 128 MB. + * 256 MB. */ @Experimental class Strategy ( @@ -58,10 +58,10 @@ class Strategy ( val impurity: Impurity, val maxDepth: Int, val numClassesForClassification: Int = 2, - val maxBins: Int = 100, + val maxBins: Int = 32, val quantileCalculationStrategy: QuantileStrategy = Sort, val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), - val maxMemoryInMB: Int = 128) extends Serializable { + val maxMemoryInMB: Int = 256) extends Serializable { if (algo == Classification) { require(numClassesForClassification >= 2) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 8e556c917b2e7..69482f2acbb40 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} import org.apache.spark.mllib.util.LocalSparkContext - class DecisionTreeSuite extends FunSuite with LocalSparkContext { def validateClassifier( @@ -353,8 +352,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, @@ -381,8 +378,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -410,8 +405,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -439,8 +432,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, @@ -464,8 +455,6 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 99) assert(bins.length === 2) assert(bins(0).length === 100) - assert(splits(0).length === 99) - assert(bins(0).length === 100) // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) @@ -600,7 +589,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3) + numClassesForClassification = 3, maxBins = 100) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -626,7 +615,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3)) + numClassesForClassification = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(metadata.isUnordered(featureIndex = 0)) @@ -652,7 +641,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) + numClassesForClassification = 3, maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index a2fade61e9a71..ccc000ac70ba6 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -138,7 +138,7 @@ class DecisionTree(object): @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, - impurity="gini", maxDepth=4, maxBins=100): + impurity="gini", maxDepth=5, maxBins=32): """ Train a DecisionTreeModel for classification. @@ -170,7 +170,7 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, @staticmethod def trainRegressor(data, categoricalFeaturesInfo, - impurity="variance", maxDepth=4, maxBins=100): + impurity="variance", maxDepth=5, maxBins=32): """ Train a DecisionTreeModel for regression. From ca0348e68213c2c7589f2018ebf9d889c0ce59c3 Mon Sep 17 00:00:00 2001 From: William Benton Date: Mon, 8 Sep 2014 19:05:02 -0700 Subject: [PATCH 0633/1492] SPARK-3423: [SQL] Implement BETWEEN for SQLParser This patch improves the SQLParser by adding support for BETWEEN conditions Author: William Benton Closes #2295 from willb/sql-between and squashes the following commits: 0016d30 [William Benton] Implement BETWEEN for SQLParser --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 ++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 18 ++++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a88bd859fc85e..bfc197cf7a938 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -73,6 +73,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val ASC = Keyword("ASC") protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AVG = Keyword("AVG") + protected val BETWEEN = Keyword("BETWEEN") protected val BY = Keyword("BY") protected val CACHE = Keyword("CACHE") protected val CAST = Keyword("CAST") @@ -272,6 +273,9 @@ class SqlParser extends StandardTokenParsers with PackratParsers { termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | + termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { + case e ~ _ ~ el ~ _ ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + } | termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | termExpression ~ REGEXP ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | termExpression ~ LIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => Like(e1, e2) } | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index e8fbc28d0ad60..45c0ca8ea101d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -597,4 +597,22 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null) :: (4, 2147483644) :: Nil) } + + test("SPARK-3423 BETWEEN") { + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 5 and 7"), + Seq((5, "5"), (6, "6"), (7, "7")) + ) + + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 7 and 7"), + Seq((7, "7")) + ) + + checkAnswer( + sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), + Seq() + ) + + } } From dc1dbf206e0076a43ad2120d8bb5b1fc6912fe25 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 8 Sep 2014 19:08:05 -0700 Subject: [PATCH 0634/1492] [SPARK-3414][SQL] Stores analyzed logical plan when registering a temp table Case insensitivity breaks when unresolved relation contains attributes with uppercase letters in their names, because we store unanalyzed logical plan when registering temp tables while the `CaseInsensitivityAttributeReferences` batch runs before the `Resolution` batch. To fix this issue, we need to store analyzed logical plan. Author: Cheng Lian Closes #2293 from liancheng/spark-3414 and squashes the following commits: d9fa1d6 [Cheng Lian] Stores analyzed logical plan when registering a temp table --- .../org/apache/spark/sql/SQLContext.scala | 4 +-- .../sql/hive/execution/HiveQuerySuite.scala | 25 ++++++++++++++++--- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 5acb45c155ba5..a2f334aab9fdf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -246,7 +246,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - catalog.registerTable(None, tableName, rdd.logicalPlan) + catalog.registerTable(None, tableName, rdd.queryExecution.analyzed) } /** @@ -411,7 +411,7 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } - def simpleString: String = + def simpleString: String = s"""== Physical Plan == |${stringOrError(executedPlan)} """ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index f4217a52c3822..305998c150327 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,11 +17,8 @@ package org.apache.spark.sql.hive.execution -import java.io.File - import scala.util.Try -import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -514,6 +511,28 @@ class HiveQuerySuite extends HiveComparisonTest { sql("DROP TABLE alter1") } + case class LogEntry(filename: String, message: String) + case class LogFile(name: String) + + test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { + sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") + sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") + + sql( + """ + SELECT name, message + FROM rawLogs + JOIN ( + SELECT name + FROM logFiles + ) files + ON rawLogs.filename = files.name + """).registerTempTable("boom") + + // This should be successfully analyzed + sql("SELECT * FROM boom").queryExecution.analyzed + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" From 2b7ab814f9bde65ebc57ebd04386e56c97f06f4a Mon Sep 17 00:00:00 2001 From: William Benton Date: Mon, 8 Sep 2014 19:29:18 -0700 Subject: [PATCH 0635/1492] [SPARK-3329][SQL] Don't depend on Hive SET pair ordering in tests. This fixes some possible spurious test failures in `HiveQuerySuite` by comparing sets of key-value pairs as sets, rather than as lists. Author: William Benton Author: Aaron Davidson Closes #2220 from willb/spark-3329 and squashes the following commits: 3b3e205 [William Benton] Collapse collectResults case match in HiveQuerySuite 6525d8e [William Benton] Handle cases where SET returns Rows of (single) strings cf11b0e [Aaron Davidson] Fix flakey HiveQuerySuite test --- .../sql/hive/execution/HiveQuerySuite.scala | 47 ++++++++++--------- 1 file changed, 26 insertions(+), 21 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 305998c150327..6bf8d18a5c32c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -558,62 +558,67 @@ class HiveQuerySuite extends HiveComparisonTest { val testKey = "spark.sql.key.usedfortestonly" val testVal = "test.val.0" val nonexistentKey = "nonexistent" - + val KV = "([^=]+)=([^=]*)".r + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { + case Row(key: String, value: String) => key -> value + case Row(KV(key, value)) => key -> value + }.toSet clear() // "set" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql(s"SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(hql("SET")) } // "set key" - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(hql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as sql() by repeating the above using sql(). clear() assert(sql("SET").collect().size == 0) - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey=$testVal").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Array(s"$testKey=$testVal")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) - assertResult(Array(s"$testKey=$testVal", s"${testKey + testKey}=${testVal + testVal}")) { - sql("SET").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { + collectResults(sql("SET")) } - assertResult(Array(s"$testKey=$testVal")) { - sql(s"SET $testKey").collect().map(_.getString(0)) + assertResult(Set(testKey -> testVal)) { + collectResults(sql(s"SET $testKey")) } - assertResult(Array(s"$nonexistentKey=")) { - sql(s"SET $nonexistentKey").collect().map(_.getString(0)) + assertResult(Set(nonexistentKey -> "")) { + collectResults(sql(s"SET $nonexistentKey")) } clear() From 092e2f152fb674e7200cc8a2cb99a8fe0a9b2b33 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Mon, 8 Sep 2014 20:51:56 -0700 Subject: [PATCH 0636/1492] SPARK-2425 Don't kill a still-running Application because of some misbehaving Executors Introduces a LOADING -> RUNNING ApplicationState transition and prevents Master from removing an Application with RUNNING Executors. Two basic changes: 1) Instead of allowing MAX_NUM_RETRY abnormal Executor exits over the entire lifetime of the Application, allow that many since any Executor successfully began running the Application; 2) Don't remove the Application while Master still thinks that there are RUNNING Executors. This should be fine as long as the ApplicationInfo doesn't believe any Executors are forever RUNNING when they are not. I think that any non-RUNNING Executors will eventually no longer be RUNNING in Master's accounting, but another set of eyes should confirm that. This PR also doesn't try to detect which nodes have gone rogue or to kill off bad Workers, so repeatedly failing Executors will continue to fail and fill up log files with failure reports as long as the Application keeps running. Author: Mark Hamstra Closes #1360 from markhamstra/SPARK-2425 and squashes the following commits: f099c0b [Mark Hamstra] Reuse appInfo b2b7b25 [Mark Hamstra] Moved 'Application failed' logging bdd0928 [Mark Hamstra] switched to string interpolation 1dd591b [Mark Hamstra] SPARK-2425 introduce LOADING -> RUNNING ApplicationState transition and prevent Master from removing Application with RUNNING Executors --- .../spark/deploy/master/ApplicationInfo.scala | 4 ++- .../apache/spark/deploy/master/Master.scala | 26 ++++++++++++------- .../spark/deploy/worker/ExecutorRunner.scala | 2 ++ .../apache/spark/deploy/worker/Worker.scala | 2 +- 4 files changed, 22 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index d3674427b1271..c3ca43f8d0734 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -96,11 +96,13 @@ private[spark] class ApplicationInfo( def retryCount = _retryCount - def incrementRetryCount = { + def incrementRetryCount() = { _retryCount += 1 _retryCount } + def resetRetryCount() = _retryCount = 0 + def markFinished(endState: ApplicationState.Value) { state = endState endTime = System.currentTimeMillis() 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 2a66fcfe4801c..a3909d6ea95c0 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 @@ -296,28 +296,34 @@ private[spark] class Master( val execOption = idToApp.get(appId).flatMap(app => app.executors.get(execId)) execOption match { case Some(exec) => { + val appInfo = idToApp(appId) exec.state = state + if (state == ExecutorState.RUNNING) { appInfo.resetRetryCount() } exec.application.driver ! ExecutorUpdated(execId, state, message, exitStatus) if (ExecutorState.isFinished(state)) { - val appInfo = idToApp(appId) // Remove this executor from the worker and app - logInfo("Removing executor " + exec.fullId + " because it is " + state) + logInfo(s"Removing executor ${exec.fullId} because it is $state") appInfo.removeExecutor(exec) exec.worker.removeExecutor(exec) - val normalExit = exitStatus.exists(_ == 0) + val normalExit = exitStatus == Some(0) // Only retry certain number of times so we don't go into an infinite loop. - if (!normalExit && appInfo.incrementRetryCount < ApplicationState.MAX_NUM_RETRY) { - schedule() - } else if (!normalExit) { - logError("Application %s with ID %s failed %d times, removing it".format( - appInfo.desc.name, appInfo.id, appInfo.retryCount)) - removeApplication(appInfo, ApplicationState.FAILED) + if (!normalExit) { + if (appInfo.incrementRetryCount() < ApplicationState.MAX_NUM_RETRY) { + schedule() + } else { + val execs = appInfo.executors.values + if (!execs.exists(_.state == ExecutorState.RUNNING)) { + logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + + s"${appInfo.retryCount} times; removing it") + removeApplication(appInfo, ApplicationState.FAILED) + } + } } } } case None => - logWarning("Got status update for unknown executor " + appId + "/" + execId) + logWarning(s"Got status update for unknown executor $appId/$execId") } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 7be89f9aff0f3..00a43673e5cd3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -159,6 +159,8 @@ private[spark] class ExecutorRunner( Files.write(header, stderr, Charsets.UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) + state = ExecutorState.RUNNING + worker ! ExecutorStateChanged(appId, execId, state, None, None) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) // or with nonzero exit code val exitCode = process.waitFor() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index e475567db6a20..0c454e4138c96 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -234,7 +234,7 @@ private[spark] class Worker( try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.RUNNING) + self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ From ce5cb325877e3fa8281ffe2076f93b4124ed0eb5 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 9 Sep 2014 00:50:59 -0700 Subject: [PATCH 0637/1492] [Build] Removed -Phive-thriftserver since this profile has been removed Author: Cheng Lian Closes #2269 from liancheng/clean-run-tests-profile and squashes the following commits: 08617bd [Cheng Lian] Removed -Phive-thriftserver since this profile has been removed --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 49a88085c80f7..79401213a7fa2 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -93,7 +93,7 @@ echo "=========================================================================" # echo "q" is needed because sbt on encountering a build file with failure # (either resolution or compilation) prompts the user for input either q, r, # etc to quit or retry. This echo is there to make it not block. -BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-thriftserver " +BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive " echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly | \ grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" From c419e4f1bd2a50c558179b7118c3fe75a94b7a5b Mon Sep 17 00:00:00 2001 From: Mario Pastorelli Date: Tue, 9 Sep 2014 00:51:28 -0700 Subject: [PATCH 0638/1492] [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2 Comment of the storageLevel param of actorStream says that it defaults to memory-only while the default is MEMORY_AND_DISK_SER_2. Author: Mario Pastorelli Closes #2319 from melrief/master and squashes the following commits: 7b6ce68 [Mario Pastorelli] [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2 --- .../scala/org/apache/spark/streaming/StreamingContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 101cec1c7a7c2..457e8ab28ed82 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -240,7 +240,7 @@ class StreamingContext private[streaming] ( * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param props Props object defining creation of the actor * @param name Name of the actor - * @param storageLevel RDD storage level. Defaults to memory-only. + * @param storageLevel RDD storage level (default: StorageLevel.MEMORY_AND_DISK_SER_2) * * @note An important point to note: * Since Actor may exist outside the spark framework, It is thus user's responsibility From 1e03cf79f82b166b2e18dcbd181e074f0276a0a9 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 9 Sep 2014 10:18:25 -0700 Subject: [PATCH 0639/1492] [SPARK-3455] [SQL] **HOT FIX** Fix the unit test failure Unit test failed due to can not resolve the attribute references. Temporally disable this test case for a quick fixing, otherwise it will block the others. Author: Cheng Hao Closes #2334 from chenghao-intel/unit_test_failure and squashes the following commits: 661f784 [Cheng Hao] temporally disable the failed test case --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 45c0ca8ea101d..739c12f338f34 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -360,6 +360,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3349 partitioning after limit") { + /* sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") @@ -374,6 +375,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), (1, "a", 1) :: (2, "b", 2) :: Nil) + */ } test("mixed-case keywords") { From 88547a09fcc25df132b401ecec4ebe1ef6778576 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 9 Sep 2014 10:23:28 -0700 Subject: [PATCH 0640/1492] SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere. Author: Sandy Ryza Closes #2324 from sryza/sandy-spark-3422 and squashes the following commits: 6446175 [Sandy Ryza] SPARK-3422. JavaAPISuite.getHadoopInputSplits isn't used anywhere. --- .../java/org/apache/spark/JavaAPISuite.java | 25 ------------------- 1 file changed, 25 deletions(-) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index be99dc501c4b2..b8574dfb42e6b 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -29,19 +29,14 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; -import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.compress.DefaultCodec; -import org.apache.hadoop.mapred.FileSplit; -import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; -import org.apache.hadoop.mapred.TextInputFormat; import org.apache.hadoop.mapreduce.Job; import org.junit.After; import org.junit.Assert; @@ -49,7 +44,6 @@ import org.junit.Test; import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaHadoopRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -1313,23 +1307,4 @@ public void collectUnderlyingScalaRDD() { SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); Assert.assertEquals(data.size(), collected.length); } - - public void getHadoopInputSplits() { - String outDir = new File(tempDir, "output").getAbsolutePath(); - sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2).saveAsTextFile(outDir); - - JavaHadoopRDD hadoopRDD = (JavaHadoopRDD) - sc.hadoopFile(outDir, TextInputFormat.class, LongWritable.class, Text.class); - List inputPaths = hadoopRDD.mapPartitionsWithInputSplit( - new Function2>, Iterator>() { - @Override - public Iterator call(InputSplit split, Iterator> it) - throws Exception { - FileSplit fileSplit = (FileSplit) split; - return Lists.newArrayList(fileSplit.getPath().toUri().getPath()).iterator(); - } - }, true).collect(); - Assert.assertEquals(Sets.newHashSet(inputPaths), - Sets.newHashSet(outDir + "/part-00000", outDir + "/part-00001")); - } } From f0f1ba09b195f23f0c89af6fa040c9e01dfa8951 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 9 Sep 2014 10:24:00 -0700 Subject: [PATCH 0641/1492] SPARK-3404 [BUILD] SparkSubmitSuite fails with "spark-submit exits with code 1" This fixes the `SparkSubmitSuite` failure by setting `0` in the Maven build, to match the SBT build. This avoids a port conflict which causes failures. (This also updates the `scalatest` plugin off of a release candidate, to the identical final release.) Author: Sean Owen Closes #2328 from srowen/SPARK-3404 and squashes the following commits: 512d782 [Sean Owen] Set spark.ui.port=0 in Maven scalatest config to match SBT build and avoid SparkSubmitSuite failure due to port conflict --- pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d05190512f742..64fb1e57e30e0 100644 --- a/pom.xml +++ b/pom.xml @@ -888,7 +888,7 @@ org.scalatest scalatest-maven-plugin - 1.0-RC2 + 1.0 ${project.build.directory}/surefire-reports . @@ -899,6 +899,7 @@ true ${session.executionRootDirectory} 1 + 0 From 26862337c97ce14794178d6378fb4155dd24acb9 Mon Sep 17 00:00:00 2001 From: scwf Date: Tue, 9 Sep 2014 11:57:01 -0700 Subject: [PATCH 0642/1492] [SPARK-3193]output errer info when Process exit code is not zero in test suite https://issues.apache.org/jira/browse/SPARK-3193 I noticed that sometimes pr tests failed due to the Process exitcode != 0,refer to https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18688/consoleFull https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19118/consoleFull [info] SparkSubmitSuite: [info] - prints usage on empty input [info] - prints usage with only --help [info] - prints error with unrecognized options [info] - handle binary specified but not class [info] - handles arguments with --key=val [info] - handles arguments to user program [info] - handles arguments to user program with name collision [info] - handles YARN cluster mode [info] - handles YARN client mode [info] - handles standalone cluster mode [info] - handles standalone client mode [info] - handles mesos client mode [info] - handles confs with flag equivalents [info] - launch simple application with spark-submit *** FAILED *** [info] org.apache.spark.SparkException: Process List(./bin/spark-submit, --class, org.apache.spark.deploy.SimpleApplicationTest, --name, testApp, --master, local, file:/tmp/1408854098404-0/testJar-1408854098404.jar) exited with code 1 [info] at org.apache.spark.util.Utils$.executeAndGetOutput(Utils.scala:872) [info] at org.apache.spark.deploy.SparkSubmitSuite.runSparkSubmit(SparkSubmitSuite.scala:311) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply$mcV$sp(SparkSubmitSuite.scala:291) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply(SparkSubmitSuite.scala:284) [info] at org.apacSpark assembly has been built with Hive, including Datanucleus jars on classpath this PR output the process error info when failed, it can be helpful for diagnosis. Author: scwf Closes #2108 from scwf/output-test-error-info and squashes the following commits: 0c48082 [scwf] minor fix according to comments 563fde1 [scwf] output errer info when Process exitcode not zero --- .../scala/org/apache/spark/util/Utils.scala | 19 ++++++++++++++++++- .../scala/org/apache/spark/DriverSuite.scala | 5 +---- .../spark/deploy/SparkSubmitSuite.scala | 2 ++ 3 files changed, 21 insertions(+), 5 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 0ae28f911e302..79943766d0f0f 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -20,9 +20,11 @@ package org.apache.spark.util import java.io._ import java.net._ import java.nio.ByteBuffer -import java.util.{Locale, Random, UUID} +import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import org.apache.log4j.PropertyConfigurator + import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -834,6 +836,7 @@ private[spark] object Utils extends Logging { val exitCode = process.waitFor() stdoutThread.join() // Wait for it to finish reading output if (exitCode != 0) { + logError(s"Process $command exited with code $exitCode: ${output}") throw new SparkException("Process " + command + " exited with code " + exitCode) } output.toString @@ -1444,6 +1447,20 @@ private[spark] object Utils extends Logging { } } + /** + * config a log4j properties used for testsuite + */ + def configTestLog4j(level: String): Unit = { + val pro = new Properties() + pro.put("log4j.rootLogger", s"$level, console") + pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender") + pro.put("log4j.appender.console.target", "System.err") + pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout") + pro.put("log4j.appender.console.layout.ConversionPattern", + "%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n") + PropertyConfigurator.configure(pro) + } + } /** diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index a73e1ef0288a5..4b1d280624c57 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark import java.io.File -import org.apache.log4j.Logger -import org.apache.log4j.Level - import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.prop.TableDrivenPropertyChecks._ @@ -54,7 +51,7 @@ class DriverSuite extends FunSuite with Timeouts { */ object DriverWithoutCleanup { def main(args: Array[String]) { - Logger.getRootLogger().setLevel(Level.WARN) + Utils.configTestLog4j("INFO") val sc = new SparkContext(args(0), "DriverWithoutCleanup") sc.parallelize(1 to 100, 4).count() } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 7e1ef80c84561..22b369a829418 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -317,6 +317,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { object JarCreationTest { def main(args: Array[String]) { + Utils.configTestLog4j("INFO") val conf = new SparkConf() val sc = new SparkContext(conf) val result = sc.makeRDD(1 to 100, 10).mapPartitions { x => @@ -338,6 +339,7 @@ object JarCreationTest { object SimpleApplicationTest { def main(args: Array[String]) { + Utils.configTestLog4j("INFO") val conf = new SparkConf() val sc = new SparkContext(conf) val configs = Seq("spark.master", "spark.app.name") From 02b5ac7191c66a866ffedde313eb10f2adfc9b58 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 9 Sep 2014 14:42:28 -0700 Subject: [PATCH 0643/1492] Minor - Fix trivial compilation warnings. Author: Prashant Sharma Closes #2331 from ScrapCodes/compilation-warn and squashes the following commits: 44c1e76 [Prashant Sharma] Minor - Fix trivial compilation warnings. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 - .../org/apache/spark/examples/graphx/LiveJournalPageRank.scala | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 24d1a8f9eceae..c6c5b8f22b549 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -49,7 +49,6 @@ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkD import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ -import org.apache.spark.SPARK_VERSION import org.apache.spark.ui.SparkUI import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala index bdc8fa7f99f2e..e809a65b79975 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/LiveJournalPageRank.scala @@ -20,7 +20,7 @@ package org.apache.spark.examples.graphx import org.apache.spark.SparkContext._ import org.apache.spark._ import org.apache.spark.graphx._ -import org.apache.spark.examples.graphx.Analytics + /** * Uses GraphX to run PageRank on a LiveJournal social network graph. Download the dataset from From 07ee4a28c3a502121770f301316cb2256e8f0ce2 Mon Sep 17 00:00:00 2001 From: xinyunh Date: Tue, 9 Sep 2014 16:55:39 -0700 Subject: [PATCH 0644/1492] [SPARK-3176] Implement 'ABS and 'LAST' for sql Add support for the mathematical function"ABS" and the analytic function "last" to return a subset of the rows satisfying a query within spark sql. Test-cases included. Author: xinyunh Author: bomeng Closes #2099 from xinyunh/sqlTest and squashes the following commits: 71d15e7 [xinyunh] remove POWER part 8843643 [xinyunh] fix the code style issue 39f0309 [bomeng] Modify the code of POWER and ABS. Move them to the file arithmetic ff8e51e [bomeng] add abs() function support 7f6980a [xinyunh] fix the bug in 'Last' component b3df91b [xinyunh] add 'Last' component --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +++ .../spark/sql/catalyst/dsl/package.scala | 1 + .../sql/catalyst/expressions/aggregates.scala | 28 +++++++++++++++++++ .../sql/catalyst/expressions/arithmetic.scala | 15 ++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 23 +++++++++++++-- 5 files changed, 69 insertions(+), 2 deletions(-) mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala mode change 100644 => 100755 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala old mode 100644 new mode 100755 index bfc197cf7a938..a04b4a938da64 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -82,6 +82,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val DISTINCT = Keyword("DISTINCT") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") + protected val LAST = Keyword("LAST") protected val FROM = Keyword("FROM") protected val FULL = Keyword("FULL") protected val GROUP = Keyword("GROUP") @@ -125,6 +126,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val SUBSTR = Keyword("SUBSTR") protected val SUBSTRING = Keyword("SUBSTRING") protected val SQRT = Keyword("SQRT") + protected val ABS = Keyword("ABS") // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -315,6 +317,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { case s ~ _ ~ _ ~ _ ~ _ ~ e => ApproxCountDistinct(e, s.toDouble) } | FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | + LAST ~> "(" ~> expression <~ ")" ^^ { case exp => Last(exp) } | AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | MAX ~> "(" ~> expression <~ ")" ^^ { case exp => Max(exp) } | @@ -330,6 +333,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { case s ~ "," ~ p ~ "," ~ l => Substring(s,p,l) } | SQRT ~> "(" ~> expression <~ ")" ^^ { case exp => Sqrt(exp) } | + ABS ~> "(" ~> expression <~ ")" ^^ { case exp => Abs(exp) } | ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ { case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala old mode 100644 new mode 100755 index f44521d6381c9..deb622c39faf5 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -132,6 +132,7 @@ package object dsl { def approxCountDistinct(e: Expression, rsd: Double = 0.05) = ApproxCountDistinct(e, rsd) def avg(e: Expression) = Average(e) def first(e: Expression) = First(e) + def last(e: Expression) = Last(e) def min(e: Expression) = Min(e) def max(e: Expression) = Max(e) def upper(e: Expression) = Upper(e) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala old mode 100644 new mode 100755 index 15560a2a933ad..1b4d892625dbb --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -344,6 +344,21 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod override def newInstance() = new FirstFunction(child, this) } +case class Last(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { + override def references = child.references + override def nullable = true + override def dataType = child.dataType + override def toString = s"LAST($child)" + + override def asPartial: SplitEvaluation = { + val partialLast = Alias(Last(child), "PartialLast")() + SplitEvaluation( + Last(partialLast.toAttribute), + partialLast :: Nil) + } + override def newInstance() = new LastFunction(child, this) +} + case class AverageFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -489,3 +504,16 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag override def eval(input: Row): Any = result } + +case class LastFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var result: Any = null + + override def update(input: Row): Unit = { + result = input + } + + override def eval(input: Row): Any = if (result != null) expr.eval(result.asInstanceOf[Row]) + else null +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index f988fb010b107..fe825fdcdae37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types._ +import scala.math.pow case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any @@ -129,3 +130,17 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def toString = s"MaxOf($left, $right)" } + +/** + * A function that get the absolute value of the numeric value. + */ +case class Abs(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def dataType = child.dataType + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"Abs($child)" + + override def eval(input: Row): Any = n1(child, input, _.abs(_)) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 739c12f338f34..514ac543df92a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -41,6 +41,25 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } + test("SPARK-3176 Added Parser of SQL ABS()") { + checkAnswer( + sql("SELECT ABS(-1.3)"), + 1.3) + checkAnswer( + sql("SELECT ABS(0.0)"), + 0.0) + checkAnswer( + sql("SELECT ABS(2.5)"), + 2.5) + } + + test("SPARK-3176 Added Parser of SQL LAST()") { + checkAnswer( + sql("SELECT LAST(n) FROM lowerCaseData"), + 4) + } + + test("SPARK-2041 column name equals tablename") { checkAnswer( sql("SELECT tableName FROM tableName"), @@ -53,14 +72,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq ) } - + test("SQRT with automatic string casts") { checkAnswer( sql("SELECT SQRT(CAST(key AS STRING)) FROM testData"), (1 to 100).map(x => Row(math.sqrt(x.toDouble))).toSeq ) } - + test("SPARK-2407 Added Parser of SQL SUBSTR()") { checkAnswer( sql("SELECT substr(tableName, 1, 2) FROM tableName"), From c110614b33a690a3db6ccb1a920fb6a3795aa5a0 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 9 Sep 2014 18:39:33 -0700 Subject: [PATCH 0645/1492] [SPARK-3448][SQL] Check for null in SpecificMutableRow.update `SpecificMutableRow.update` doesn't check for null, and breaks existing `MutableRow` contract. The tricky part here is that for performance considerations, the `update` method of all subclasses of `MutableValue` doesn't check for null and sets the null bit to false. Author: Cheng Lian Closes #2325 from liancheng/check-for-null and squashes the following commits: 9366c44 [Cheng Lian] Check for null in SpecificMutableRow.update --- .../spark/sql/catalyst/expressions/SpecificRow.scala | 4 +++- .../src/test/scala/org/apache/spark/sql/RowSuite.scala | 8 +++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala index 75ea0e8459df8..088f11ee4aa53 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -227,7 +227,9 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR new SpecificMutableRow(newValues) } - override def update(ordinal: Int, value: Any): Unit = values(ordinal).update(value) + override def update(ordinal: Int, value: Any): Unit = { + if (value == null) setNullAt(ordinal) else values(ordinal).update(value) + } override def iterator: Iterator[Any] = values.map(_.boxed).iterator diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 651cb735ab7d9..811319e0a6601 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, SpecificMutableRow} class RowSuite extends FunSuite { @@ -43,4 +43,10 @@ class RowSuite extends FunSuite { assert(expected.getBoolean(2) === actual2.getBoolean(2)) assert(expected(3) === actual2(3)) } + + test("SpecificMutableRow.update with null") { + val row = new SpecificMutableRow(Seq(IntegerType)) + row(0) = null + assert(row.isNullAt(0)) + } } From 25b5b867d5e18bac1c5bcdc6f8c63d97858194c7 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Tue, 9 Sep 2014 18:54:54 -0700 Subject: [PATCH 0646/1492] [SPARK-3458] enable python "with" statements for SparkContext allow for best practice code, ``` try: sc = SparkContext() app(sc) finally: sc.stop() ``` to be written using a "with" statement, ``` with SparkContext() as sc: app(sc) ``` Author: Matthew Farrellee Closes #2335 from mattf/SPARK-3458 and squashes the following commits: 5b4e37c [Matthew Farrellee] [SPARK-3458] enable python "with" statements for SparkContext --- python/pyspark/context.py | 14 ++++++++++++++ python/pyspark/tests.py | 29 +++++++++++++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5a30431568b16..84bc0a3b7ccd0 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -232,6 +232,20 @@ def _ensure_initialized(cls, instance=None, gateway=None): else: SparkContext._active_spark_context = instance + def __enter__(self): + """ + Enable 'with SparkContext(...) as sc: app(sc)' syntax. + """ + return self + + def __exit__(self, type, value, trace): + """ + Enable 'with SparkContext(...) as sc: app' syntax. + + Specifically stop the context on exit of the with block. + """ + self.stop() + @classmethod def setSystemProperty(cls, key, value): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0bd2a9e6c507d..bb84ebe72cb24 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1254,6 +1254,35 @@ def test_single_script_on_cluster(self): self.assertIn("[2, 4, 6]", out) +class ContextStopTests(unittest.TestCase): + + def test_stop(self): + sc = SparkContext() + self.assertNotEqual(SparkContext._active_spark_context, None) + sc.stop() + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with(self): + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with_exception(self): + try: + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + raise Exception() + except: + pass + self.assertEqual(SparkContext._active_spark_context, None) + + def test_with_stop(self): + with SparkContext() as sc: + self.assertNotEqual(SparkContext._active_spark_context, None) + sc.stop() + self.assertEqual(SparkContext._active_spark_context, None) + + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): From b734ed0c229373dbc589b9eca7327537ca458138 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Tue, 9 Sep 2014 23:47:12 -0700 Subject: [PATCH 0647/1492] [SPARK-3395] [SQL] DSL sometimes incorrectly reuses attribute ids, breaking queries This resolves https://issues.apache.org/jira/browse/SPARK-3395 Author: Eric Liang Closes #2266 from ericl/spark-3395 and squashes the following commits: 7f2b6f0 [Eric Liang] add regression test 05bd1e4 [Eric Liang] in the dsl, create a new schema instance in each applySchema --- .../scala/org/apache/spark/sql/SchemaRDD.scala | 3 ++- .../scala/org/apache/spark/sql/DslQuerySuite.scala | 14 ++++++++++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 33b2ed1b3a399..d2ceb4a2b0b25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -428,7 +428,8 @@ class SchemaRDD( */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { new SchemaRDD(sqlContext, - SparkLogicalPlan(ExistingRdd(queryExecution.analyzed.output, rdd))(sqlContext)) + SparkLogicalPlan( + ExistingRdd(queryExecution.analyzed.output.map(_.newInstance), rdd))(sqlContext)) } // ======================================================================= diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 1a6a6c17473a3..d001abb7e1fcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.test._ /* Implicits */ @@ -133,6 +135,18 @@ class DslQuerySuite extends QueryTest { mapData.take(1).toSeq) } + test("SPARK-3395 limit distinct") { + val filtered = TestData.testData2 + .distinct() + .orderBy(SortOrder('a, Ascending), SortOrder('b, Ascending)) + .limit(1) + .registerTempTable("onerow") + checkAnswer( + sql("select * from onerow inner join testData2 on onerow.a = testData2.a"), + (1, 1, 1, 1) :: + (1, 1, 1, 2) :: Nil) + } + test("average") { checkAnswer( testData2.groupBy()(avg('a)), From 6f7a76838f15687583e3b0ab43309a3c079368c4 Mon Sep 17 00:00:00 2001 From: Benoy Antony Date: Wed, 10 Sep 2014 11:59:39 -0500 Subject: [PATCH 0648/1492] =?UTF-8?q?[SPARK-3286]=20-=20Cannot=20view=20Ap?= =?UTF-8?q?plicationMaster=20UI=20when=20Yarn=E2=80=99s=20url=20scheme=20i?= =?UTF-8?q?...?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ...s https Author: Benoy Antony Closes #2276 from benoyantony/SPARK-3286 and squashes the following commits: c3d51ee [Benoy Antony] Use address with scheme, but Allpha version removes the scheme e82f94e [Benoy Antony] Use address with scheme, but Allpha version removes the scheme 92127c9 [Benoy Antony] rebasing from master 450c536 [Benoy Antony] [SPARK-3286] - Cannot view ApplicationMaster UI when Yarn’s url scheme is https f060c02 [Benoy Antony] [SPARK-3286] - Cannot view ApplicationMaster UI when Yarn’s url scheme is https --- .../scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 4 +++- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index ad27a9ab781d2..fc30953011812 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import scala.collection.{Map, Set} +import java.net.URI; import org.apache.hadoop.net.NetUtils import org.apache.hadoop.yarn.api._ @@ -97,7 +98,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) - appMasterRequest.setTrackingUrl(uiAddress) + //remove the scheme from the url if it exists since Hadoop does not expect scheme + appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) resourceManager.registerApplicationMaster(appMasterRequest) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a879c833a014f..5756263e89e21 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -189,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIHostPort, securityMgr) + registerAM(sc.ui.appUIAddress, securityMgr) try { userThread.join() } finally { From a0283300c4af5e64a1dc06193245daa1e746b5f4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Sep 2014 10:45:15 -0700 Subject: [PATCH 0649/1492] [SPARK-3362][SQL] Fix resolution for casewhen with nulls. Current implementation will ignore else val type. Author: Daoyuan Wang Closes #2245 from adrian-wang/casewhenbug and squashes the following commits: 3332f6e [Daoyuan Wang] remove wrong comment 83b536c [Daoyuan Wang] a comment to trigger retest d7315b3 [Daoyuan Wang] code improve eed35fc [Daoyuan Wang] bug in casewhen resolve --- .../apache/spark/sql/catalyst/expressions/predicates.scala | 5 +++-- ...then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c | 1 + ...en 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 | 1 + ...hen 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb | 1 + ...hen 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 | 1 + ...hen 1Y else null end -0-589982a400d86157791c7216b10b6b5d | 1 + ...then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 | 1 + ...en null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 | 1 + ...hen null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 | 1 + ...hen null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 | 1 + ...hen null else 1Y end -0-7f4c32299c3738739b678ece62752a7b | 1 + .../spark/sql/hive/execution/HiveTypeCoercionSuite.scala | 6 ++++++ 12 files changed, 19 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c create mode 100644 sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 create mode 100644 sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb create mode 100644 sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 create mode 100644 sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 create mode 100644 sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 1313ccd120c1f..329af332d0fa1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -265,12 +265,13 @@ case class CaseWhen(branches: Seq[Expression]) extends Expression { false } else { val allCondBooleans = predicates.forall(_.dataType == BooleanType) - val dataTypesEqual = values.map(_.dataType).distinct.size <= 1 + // both then and else val should be considered. + val dataTypesEqual = (values ++ elseValue).map(_.dataType).distinct.size <= 1 allCondBooleans && dataTypesEqual } } - /** Written in imperative fashion for performance considerations. Same for CaseKeyWhen. */ + /** Written in imperative fashion for performance considerations. */ override def eval(input: Row): Any = { val len = branchesArr.length var i = 0 diff --git a/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c b/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1 else null end -0-f7c7fdd35c084bc797890aa08d33693c @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 b/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1.0 else null end -0-aeb1f906bfe92f2d406f84109301afe0 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb b/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1L else null end -0-763ae85e7a52b4cf4162d6a8931716bb @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 b/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1S else null end -0-6f5f3b3dbe9f1d1eb98443aef315b982 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d b/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then 1Y else null end -0-589982a400d86157791c7216b10b6b5d @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 b/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1 end -0-48bd83660cf3ba93cdbdc24559092171 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 b/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1.0 end -0-7f5ce763801781cf568c6a31dd80b623 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 b/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1L end -0-a7f1305ea4f86e596c368e35e45cc4e5 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 b/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1S end -0-dfb61969e6cb6e6dbe89225b538c8d98 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b b/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/case when then null else 1Y end -0-7f4c32299c3738739b678ece62752a7b @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index c3c18cf8ccac3..48fffe53cf2ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -33,6 +33,12 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { } } + val nullVal = "null" + baseTypes.init.foreach { i => + createQueryTest(s"case when then $i else $nullVal end ", s"SELECT case when true then $i else $nullVal end FROM src limit 1") + createQueryTest(s"case when then $nullVal else $i end ", s"SELECT case when true then $nullVal else $i end FROM src limit 1") + } + test("[SPARK-2210] boolean cast on boolean value should be removed") { val q = "select cast(cast(key=0 as boolean) as boolean) from src" val project = TestHive.sql(q).queryExecution.executedPlan.collect { case e: Project => e }.head From f0c87dc86ae65a39cd19370d8d960b4a60854517 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Sep 2014 10:48:33 -0700 Subject: [PATCH 0650/1492] [SPARK-3363][SQL] Type Coercion should promote null to all other types. Type Coercion should support every type to have null value Author: Daoyuan Wang Author: Michael Armbrust Closes #2246 from adrian-wang/spark3363-0 and squashes the following commits: c6241de [Daoyuan Wang] minor code clean 595b417 [Daoyuan Wang] Merge pull request #2 from marmbrus/pr/2246 832e640 [Michael Armbrust] reduce code duplication ef6f986 [Daoyuan Wang] make double boolean miss in jsonRDD compatibleType c619f0a [Daoyuan Wang] Type Coercion should support every type to have null value --- .../catalyst/analysis/HiveTypeCoercion.scala | 38 +++++++------- .../analysis/HiveTypeCoercionSuite.scala | 32 +++++++++--- .../org/apache/spark/sql/json/JsonRDD.scala | 51 ++++++++----------- 3 files changed, 67 insertions(+), 54 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index d6758eb5b6a32..bd8131c9af6e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -26,10 +26,22 @@ object HiveTypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: val numericPrecedence = - Seq(NullType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) - // Boolean is only wider than Void - val booleanPrecedence = Seq(NullType, BooleanType) - val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: booleanPrecedence :: Nil + Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) + val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: Nil + + def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { + val valueTypes = Seq(t1, t2).filter(t => t != NullType) + if (valueTypes.distinct.size > 1) { + // Try and find a promotion rule that contains both types in question. + val applicableConversion = + HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) + + // If found return the widest common type, otherwise None + applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + } else { + Some(if (valueTypes.size == 0) NullType else valueTypes.head) + } + } } /** @@ -53,17 +65,6 @@ trait HiveTypeCoercion { Division :: Nil - trait TypeWidening { - def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - } - } - /** * Applies any changes to [[AttributeReference]] data types that are made by other rules to * instances higher in the query tree. @@ -144,7 +145,8 @@ trait HiveTypeCoercion { * - LongType to FloatType * - LongType to DoubleType */ - object WidenTypes extends Rule[LogicalPlan] with TypeWidening { + object WidenTypes extends Rule[LogicalPlan] { + import HiveTypeCoercion._ def apply(plan: LogicalPlan): LogicalPlan = plan transform { case u @ Union(left, right) if u.childrenResolved && !u.resolved => @@ -352,7 +354,9 @@ trait HiveTypeCoercion { /** * Coerces the type of different branches of a CASE WHEN statement to a common type. */ - object CaseWhenCoercion extends Rule[LogicalPlan] with TypeWidening { + object CaseWhenCoercion extends Rule[LogicalPlan] { + import HiveTypeCoercion._ + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case cw @ CaseWhen(branches) if !cw.resolved && !branches.exists(!_.resolved) => val valueTypes = branches.sliding(2, 2).map { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index b9e0f8e9dcc5f..ba8b853b6f99e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -23,20 +23,20 @@ import org.apache.spark.sql.catalyst.types._ class HiveTypeCoercionSuite extends FunSuite { - val rules = new HiveTypeCoercion { } - import rules._ - - test("tightest common bound for numeric and boolean types") { + test("tightest common bound for types") { def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) { - var found = WidenTypes.findTightestCommonType(t1, t2) + var found = HiveTypeCoercion.findTightestCommonType(t1, t2) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found") // Test both directions to make sure the widening is symmetric. - found = WidenTypes.findTightestCommonType(t2, t1) + found = HiveTypeCoercion.findTightestCommonType(t2, t1) assert(found == tightestCommon, s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found") } + // Null + widenTest(NullType, NullType, Some(NullType)) + // Boolean widenTest(NullType, BooleanType, Some(BooleanType)) widenTest(BooleanType, BooleanType, Some(BooleanType)) @@ -60,12 +60,28 @@ class HiveTypeCoercionSuite extends FunSuite { widenTest(DoubleType, DoubleType, Some(DoubleType)) // Integral mixed with floating point. - widenTest(NullType, FloatType, Some(FloatType)) - widenTest(NullType, DoubleType, Some(DoubleType)) widenTest(IntegerType, FloatType, Some(FloatType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) widenTest(IntegerType, DoubleType, Some(DoubleType)) widenTest(LongType, FloatType, Some(FloatType)) widenTest(LongType, DoubleType, Some(DoubleType)) + + // StringType + widenTest(NullType, StringType, Some(StringType)) + widenTest(StringType, StringType, Some(StringType)) + widenTest(IntegerType, StringType, None) + widenTest(LongType, StringType, None) + + // TimestampType + widenTest(NullType, TimestampType, Some(TimestampType)) + widenTest(TimestampType, TimestampType, Some(TimestampType)) + widenTest(IntegerType, TimestampType, None) + widenTest(StringType, TimestampType, None) + + // ComplexType + widenTest(NullType, MapType(IntegerType, StringType, false), Some(MapType(IntegerType, StringType, false))) + widenTest(NullType, StructType(Seq()), Some(StructType(Seq()))) + widenTest(StringType, MapType(IntegerType, StringType, true), None) + widenTest(ArrayType(IntegerType), StructType(Seq()), None) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 1c0b03c684f10..70062eae3b7ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -125,38 +125,31 @@ private[sql] object JsonRDD extends Logging { * Returns the most general data type for two given data types. */ private[json] def compatibleType(t1: DataType, t2: DataType): DataType = { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p - .contains(t2)) - - // If found return the widest common type, otherwise None - val returnType = applicableConversion.map(_.filter(t => t == t1 || t == t2).last) - - if (returnType.isDefined) { - returnType.get - } else { - // t1 or t2 is a StructType, ArrayType, or an unexpected type. - (t1, t2) match { - case (other: DataType, NullType) => other - case (NullType, other: DataType) => other - case (StructType(fields1), StructType(fields2)) => { - val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { - case (name, fieldTypes) => { - val dataType = fieldTypes.map(field => field.dataType).reduce( - (type1: DataType, type2: DataType) => compatibleType(type1, type2)) - StructField(name, dataType, true) + HiveTypeCoercion.findTightestCommonType(t1, t2) match { + case Some(commonType) => commonType + case None => + // t1 or t2 is a StructType, ArrayType, or an unexpected type. + (t1, t2) match { + case (other: DataType, NullType) => other + case (NullType, other: DataType) => other + case (StructType(fields1), StructType(fields2)) => { + val newFields = (fields1 ++ fields2).groupBy(field => field.name).map { + case (name, fieldTypes) => { + val dataType = fieldTypes.map(field => field.dataType).reduce( + (type1: DataType, type2: DataType) => compatibleType(type1, type2)) + StructField(name, dataType, true) + } } + StructType(newFields.toSeq.sortBy { + case StructField(name, _, _) => name + }) } - StructType(newFields.toSeq.sortBy { - case StructField(name, _, _) => name - }) + case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => + ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) + // TODO: We should use JsonObjectStringType to mark that values of field will be + // strings and every string is a Json object. + case (_, _) => StringType } - case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => - ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) - // TODO: We should use JsonObjectStringType to mark that values of field will be - // strings and every string is a Json object. - case (_, _) => StringType - } } } From 26503fdf20f4181a2b390c88b83f364e6a4ccc21 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 Sep 2014 12:02:23 -0700 Subject: [PATCH 0651/1492] [HOTFIX] Fix scala style issue introduced by #2276. --- .../scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index fc30953011812..acf26505e4cf9 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -98,7 +98,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC // Users can then monitor stderr/stdout on that node if required. appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) - //remove the scheme from the url if it exists since Hadoop does not expect scheme + // remove the scheme from the url if it exists since Hadoop does not expect scheme appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) resourceManager.registerApplicationMaster(appMasterRequest) } From 1f4a648d4e30e837d6cf3ea8de1808e2254ad70b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 10 Sep 2014 14:34:24 -0500 Subject: [PATCH 0652/1492] SPARK-1713. Use a thread pool for launching executors. This patch copies the approach used in the MapReduce application master for launching containers. Author: Sandy Ryza Closes #663 from sryza/sandy-spark-1713 and squashes the following commits: 036550d [Sandy Ryza] SPARK-1713. [YARN] Use a threadpool for launching executor containers --- docs/running-on-yarn.md | 7 +++++++ .../apache/spark/deploy/yarn/YarnAllocator.scala | 14 ++++++++++++-- 2 files changed, 19 insertions(+), 2 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 943f06b114cb9..d8b22f3663d08 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -125,6 +125,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes the environment of the executor launcher. + + spark.yarn.containerLauncherMaxThreads + 25 + + The maximum number of threads to use in the application master for launching executor containers. + + # Launching Spark on YARN diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 02b9a81bf6b50..0b8744f4b8bdf 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import java.util.{List => JList} -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ @@ -32,6 +32,8 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import com.google.common.util.concurrent.ThreadFactoryBuilder + object AllocationType extends Enumeration { type AllocationType = Value val HOST, RACK, ANY = Value @@ -95,6 +97,14 @@ private[yarn] abstract class YarnAllocator( protected val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, preferredNodes) + private val launcherPool = new ThreadPoolExecutor( + // max pool size of Integer.MAX_VALUE is ignored because we use an unbounded queue + sparkConf.getInt("spark.yarn.containerLauncherMaxThreads", 25), Integer.MAX_VALUE, + 1, TimeUnit.MINUTES, + new LinkedBlockingQueue[Runnable](), + new ThreadFactoryBuilder().setNameFormat("ContainerLauncher #%d").setDaemon(true).build()) + launcherPool.allowCoreThreadTimeOut(true) + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue def getNumExecutorsFailed: Int = numExecutorsFailed.intValue @@ -283,7 +293,7 @@ private[yarn] abstract class YarnAllocator( executorMemory, executorCores, securityMgr) - new Thread(executorRunnable).start() + launcherPool.execute(executorRunnable) } } logDebug(""" From e4f4886d7148bf48f9e3462b83bfb1ecc7edbe31 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 10 Sep 2014 12:56:59 -0700 Subject: [PATCH 0653/1492] [SPARK-2096][SQL] Correctly parse dot notations First let me write down the current `projections` grammar of spark sql: expression : orExpression orExpression : andExpression {"or" andExpression} andExpression : comparisonExpression {"and" comparisonExpression} comparisonExpression : termExpression | termExpression "=" termExpression | termExpression ">" termExpression | ... termExpression : productExpression {"+"|"-" productExpression} productExpression : baseExpression {"*"|"/"|"%" baseExpression} baseExpression : expression "[" expression "]" | ... | ident | ... ident : identChar {identChar | digit} | delimiters | ... identChar : letter | "_" | "." delimiters : "," | ";" | "(" | ")" | "[" | "]" | ... projection : expression [["AS"] ident] projections : projection { "," projection} For something like `a.b.c[1]`, it will be parsed as: But for something like `a[1].b`, the current grammar can't parse it correctly. A simple solution is written in `ParquetQuerySuite#NestedSqlParser`, changed grammars are: delimiters : "." | "," | ";" | "(" | ")" | "[" | "]" | ... identChar : letter | "_" baseExpression : expression "[" expression "]" | expression "." ident | ... | ident | ... This works well, but can't cover some corner case like `select t.a.b from table as t`: `t.a.b` parsed as `GetField(GetField(UnResolved("t"), "a"), "b")` instead of `GetField(UnResolved("t.a"), "b")` using this new grammar. However, we can't resolve `t` as it's not a filed, but the whole table.(if we could do this, then `select t from table as t` is legal, which is unexpected) My solution is: dotExpressionHeader : ident "." ident baseExpression : expression "[" expression "]" | expression "." ident | ... | dotExpressionHeader | ident | ... I passed all test cases under sql locally and add a more complex case. "arrayOfStruct.field1 to access all values of field1" is not supported yet. Since this PR has changed a lot of code, I will open another PR for it. I'm not familiar with the latter optimize phase, please correct me if I missed something. Author: Wenchen Fan Author: Michael Armbrust Closes #2230 from cloud-fan/dot and squashes the following commits: e1a8898 [Wenchen Fan] remove support for arbitrary nested arrays ee8a724 [Wenchen Fan] rollback LogicalPlan, support dot operation on nested array type a58df40 [Michael Armbrust] add regression test for doubly nested data 16bc4c6 [Wenchen Fan] some enhance 95d733f [Wenchen Fan] split long line dc31698 [Wenchen Fan] SPARK-2096 Correctly parse dot notations --- .../apache/spark/sql/catalyst/SqlParser.scala | 13 ++- .../catalyst/plans/logical/LogicalPlan.scala | 6 +- .../org/apache/spark/sql/json/JsonSuite.scala | 14 +++ .../apache/spark/sql/json/TestJsonData.scala | 26 +++++ .../spark/sql/parquet/ParquetQuerySuite.scala | 102 +++++------------- .../sql/hive/execution/SQLQuerySuite.scala | 17 ++- 6 files changed, 88 insertions(+), 90 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a04b4a938da64..ca69531c69a77 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -357,16 +357,25 @@ class SqlParser extends StandardTokenParsers with PackratParsers { expression ~ "[" ~ expression <~ "]" ^^ { case base ~ _ ~ ordinal => GetItem(base, ordinal) } | + (expression <~ ".") ~ ident ^^ { + case base ~ fieldName => GetField(base, fieldName) + } | TRUE ^^^ Literal(true, BooleanType) | FALSE ^^^ Literal(false, BooleanType) | cast | "(" ~> expression <~ ")" | function | "-" ~> literal ^^ UnaryMinus | + dotExpressionHeader | ident ^^ UnresolvedAttribute | "*" ^^^ Star(None) | literal + protected lazy val dotExpressionHeader: Parser[Expression] = + (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { + case i1 ~ i2 ~ rest => UnresolvedAttribute(i1 + "." + i2 + rest.mkString(".", ".", "")) + } + protected lazy val dataType: Parser[DataType] = STRING ^^^ StringType | TIMESTAMP ^^^ TimestampType } @@ -380,7 +389,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":", "[", "]" + ",", ";", "%", "{", "}", ":", "[", "]", "." ) override lazy val token: Parser[Token] = ( @@ -401,7 +410,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { | failure("illegal character") ) - override def identChar = letter | elem('_') | elem('.') + override def identChar = letter | elem('_') override def whitespace: Parser[Any] = rep( whitespaceChar diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index f81d9111945f5..bae491f07c13f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -104,11 +104,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { case Seq((a, Nil)) => Some(a) // One match, no nested fields, use it. // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - a.dataType match { - case StructType(fields) => - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - case _ => None // Don't know how to resolve these field references - } + Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) case Seq() => None // No matches. case ambiguousReferences => throw new TreeNodeException( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 05513a127150c..301d482d27d86 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -581,4 +581,18 @@ class JsonSuite extends QueryTest { "this is a simple string.") :: Nil ) } + + test("SPARK-2096 Correctly parse dot notations") { + val jsonSchemaRDD = jsonRDD(complexFieldAndType2) + jsonSchemaRDD.registerTempTable("jsonTable") + + checkAnswer( + sql("select arrayOfStruct[0].field1, arrayOfStruct[0].field2 from jsonTable"), + (true, "str1") :: Nil + ) + checkAnswer( + sql("select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1] from jsonTable"), + ("str2", 6) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index a88310b5f1b46..b3f95f08e8044 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -82,4 +82,30 @@ object TestJsonData { """{"c":[33, 44]}""" :: """{"d":{"field":true}}""" :: """{"e":"str"}""" :: Nil) + + val complexFieldAndType2 = + TestSQLContext.sparkContext.parallelize( + """{"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], + "complexArrayOfStruct": [ + { + "field1": [ + { + "inner1": "str1" + }, + { + "inner2": ["str2", "str22"] + }], + "field2": [[1, 2], [3, 4]] + }, + { + "field1": [ + { + "inner2": ["str3", "str33"] + }, + { + "inner1": "str4" + }], + "field2": [[5, 6], [7, 8]] + }] + }""" :: Nil) } 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 42923b6a288d9..b0a06cd3ca090 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 @@ -17,19 +17,14 @@ package org.apache.spark.sql.parquet +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapreduce.Job import org.scalatest.{BeforeAndAfterAll, FunSuiteLike} - import parquet.hadoop.ParquetFileWriter import parquet.hadoop.util.ContextUtil -import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.hadoop.mapreduce.Job - -import org.apache.spark.SparkContext import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser} -import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAttribute} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.types.{BooleanType, IntegerType} +import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ @@ -87,11 +82,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA var testRDD: SchemaRDD = null - // TODO: remove this once SqlParser can parse nested select statements - var nestedParserSqlContext: NestedParserSQLContext = null - override def beforeAll() { - nestedParserSqlContext = new NestedParserSQLContext(TestSQLContext.sparkContext) ParquetTestData.writeFile() ParquetTestData.writeFilterFile() ParquetTestData.writeNestedFile1() @@ -718,11 +709,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Projection in addressbook") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir1.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir1.toString).toSchemaRDD data.registerTempTable("data") - val query = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM data") + val query = sql("SELECT owner, contacts[1].name FROM data") val tmp = query.collect() assert(tmp.size === 2) assert(tmp(0).size === 2) @@ -733,21 +722,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Simple query on nested int data") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir2.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir2.toString).toSchemaRDD data.registerTempTable("data") - val result1 = nestedParserSqlContext.sql("SELECT entries[0].value FROM data").collect() + val result1 = sql("SELECT entries[0].value FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === 2.5) - val result2 = nestedParserSqlContext.sql("SELECT entries[0] FROM data").collect() + val result2 = sql("SELECT entries[0] FROM data").collect() assert(result2.size === 1) val subresult1 = result2(0)(0).asInstanceOf[CatalystConverter.StructScalaType[_]] assert(subresult1.size === 2) assert(subresult1(0) === 2.5) assert(subresult1(1) === false) - val result3 = nestedParserSqlContext.sql("SELECT outerouter FROM data").collect() + val result3 = sql("SELECT outerouter FROM data").collect() val subresult2 = result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] @@ -760,19 +747,18 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("nested structs") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir3.toString) + val data = parquetFile(ParquetTestData.testNestedDir3.toString) .toSchemaRDD data.registerTempTable("data") - val result1 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() + val result1 = sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() assert(result1.size === 1) assert(result1(0).size === 1) assert(result1(0)(0) === false) - val result2 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() + val result2 = sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() assert(result2.size === 1) assert(result2(0).size === 1) assert(result2(0)(0) === true) - val result3 = nestedParserSqlContext.sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() + val result3 = sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() assert(result3.size === 1) assert(result3(0).size === 1) assert(result3(0)(0) === false) @@ -796,11 +782,9 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("map with struct values") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir4.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD data.registerTempTable("mapTable") - val result1 = nestedParserSqlContext.sql("SELECT data2 FROM mapTable").collect() + val result1 = sql("SELECT data2 FROM mapTable").collect() assert(result1.size === 1) val entry1 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] @@ -814,7 +798,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result2 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() + val result2 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() assert(result2.size === 1) assert(result2(0)(0) === 42.toLong) assert(result2(0)(1) === "the answer") @@ -825,15 +809,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // has no effect in this test case val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) - val result = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir1.toString) - .toSchemaRDD + val result = parquetFile(ParquetTestData.testNestedDir1.toString).toSchemaRDD result.saveAsParquetFile(tmpdir.toString) - nestedParserSqlContext - .parquetFile(tmpdir.toString) + parquetFile(tmpdir.toString) .toSchemaRDD .registerTempTable("tmpcopy") - val tmpdata = nestedParserSqlContext.sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() + val tmpdata = sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() assert(tmpdata.size === 2) assert(tmpdata(0).size === 2) assert(tmpdata(0)(0) === "Julien Le Dem") @@ -844,20 +825,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } test("Writing out Map and reading it back in") { - val data = nestedParserSqlContext - .parquetFile(ParquetTestData.testNestedDir4.toString) - .toSchemaRDD + val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) data.saveAsParquetFile(tmpdir.toString) - nestedParserSqlContext - .parquetFile(tmpdir.toString) + parquetFile(tmpdir.toString) .toSchemaRDD .registerTempTable("tmpmapcopy") - val result1 = nestedParserSqlContext.sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() + val result1 = sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() assert(result1.size === 1) assert(result1(0)(0) === 2) - val result2 = nestedParserSqlContext.sql("SELECT data2 FROM tmpmapcopy").collect() + val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() assert(result2.size === 1) val entry1 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] @@ -871,42 +849,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(entry2 != null) assert(entry2(0) === 49) assert(entry2(1) === null) - val result3 = nestedParserSqlContext.sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() + val result3 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() assert(result3.size === 1) assert(result3(0)(0) === 42.toLong) assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } } - -// TODO: the code below is needed temporarily until the standard parser is able to parse -// nested field expressions correctly -class NestedParserSQLContext(@transient override val sparkContext: SparkContext) extends SQLContext(sparkContext) { - override protected[sql] val parser = new NestedSqlParser() -} - -class NestedSqlLexical(override val keywords: Seq[String]) extends SqlLexical(keywords) { - override def identChar = letter | elem('_') - delimiters += (".") -} - -class NestedSqlParser extends SqlParser { - override val lexical = new NestedSqlLexical(reservedWords) - - override protected lazy val baseExpression: PackratParser[Expression] = - expression ~ "[" ~ expression <~ "]" ^^ { - case base ~ _ ~ ordinal => GetItem(base, ordinal) - } | - expression ~ "." ~ ident ^^ { - case base ~ _ ~ fieldName => GetField(base, fieldName) - } | - TRUE ^^^ Literal(true, BooleanType) | - FALSE ^^^ Literal(false, BooleanType) | - cast | - "(" ~> expression <~ ")" | - function | - "-" ~> literal ^^ UnaryMinus | - ident ^^ UnresolvedAttribute | - "*" ^^^ Star(None) | - literal -} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 635a9fb0d56cb..b99caf77bce28 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.hive.execution -import scala.reflect.ClassTag - -import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.QueryTest import org.apache.spark.sql.hive.test.TestHive._ +case class Nested1(f1: Nested2) +case class Nested2(f2: Nested3) +case class Nested3(f3: Int) + /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is @@ -47,4 +47,11 @@ class SQLQuerySuite extends QueryTest { GROUP BY key, value ORDER BY value) a""").collect().toSeq) } + + test("double nested data") { + sparkContext.parallelize(Nested1(Nested2(Nested3(1))) :: Nil).registerTempTable("nested") + checkAnswer( + sql("SELECT f1.f2.f3 FROM nested"), + 1) + } } From 558962a83fb0758ab5c13ff4ea58cc96c29cbbcc Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Wed, 10 Sep 2014 13:06:47 -0700 Subject: [PATCH 0654/1492] [SPARK-3411] Improve load-balancing of concurrently-submitted drivers across workers If the waiting driver array is too big, the drivers in it will be dispatched to the first worker we get(if it has enough resources), with or without the Randomization. We should do randomization every time we dispatch a driver, in order to better balance drivers. Author: WangTaoTheTonic Author: WangTao Closes #1106 from WangTaoTheTonic/fixBalanceDrivers and squashes the following commits: d1a928b [WangTaoTheTonic] Minor adjustment b6560cf [WangTaoTheTonic] solve the shuffle problem for HashSet f674e59 [WangTaoTheTonic] add comment and minor fix 2835929 [WangTao] solve the failed test and avoid filtering 2ca3091 [WangTao] fix checkstyle bc91bb1 [WangTao] Avoid shuffle every time we schedule the driver using round robin bbc7087 [WangTaoTheTonic] Optimize the schedule in Master --- .../apache/spark/deploy/master/Master.scala | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index a3909d6ea95c0..2a3bd6ba0b9dc 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 @@ -487,13 +487,25 @@ private[spark] class Master( if (state != RecoveryState.ALIVE) { return } // First schedule drivers, they take strict precedence over applications - val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers - for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { - for (driver <- List(waitingDrivers: _*)) { // iterate over a copy of waitingDrivers + // Randomization helps balance drivers + val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) + val aliveWorkerNum = shuffledAliveWorkers.size + var curPos = 0 + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers + // We assign workers to each waiting driver in a round-robin fashion. For each driver, we + // start from the last worker that was assigned a driver, and continue onwards until we have + // explored all alive workers. + curPos = (curPos + 1) % aliveWorkerNum + val startPos = curPos + var launched = false + while (curPos != startPos && !launched) { + val worker = shuffledAliveWorkers(curPos) if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver + launched = true } + curPos = (curPos + 1) % aliveWorkerNum } } From 79cdb9b64ad2fa3ab7f2c221766d36658b917c40 Mon Sep 17 00:00:00 2001 From: "qiping.lqp" Date: Wed, 10 Sep 2014 15:37:10 -0700 Subject: [PATCH 0655/1492] [SPARK-2207][SPARK-3272][MLLib]Add minimum information gain and minimum instances per node as training parameters for decision tree. These two parameters can act as early stop rules to do pre-pruning. When a split cause cause left or right child to have less than `minInstancesPerNode` or has less information gain than `minInfoGain`, current node will not be split by this split. When there is no possible splits that satisfy requirements, there is no useful information gain stats, but we still need to calculate the predict value for current node. So I separated calculation of predict from calculation of information gain, which can also save computation when the number of possible splits is large. Please see [SPARK-3272](https://issues.apache.org/jira/browse/SPARK-3272) for more details. CC: mengxr manishamde jkbradley, please help me review this, thanks. Author: qiping.lqp Author: chouqin Closes #2332 from chouqin/dt-preprune and squashes the following commits: f1d11d1 [chouqin] fix typo c7ebaf1 [chouqin] fix typo 39f9b60 [chouqin] change edge `minInstancesPerNode` to 2 and add one more test 0278a11 [chouqin] remove `noSplit` and set `Predict` private to tree d593ec7 [chouqin] fix docs and change minInstancesPerNode to 1 efcc736 [qiping.lqp] fix bug 10b8012 [qiping.lqp] fix style 6728fad [qiping.lqp] minor fix: remove empty lines bb465ca [qiping.lqp] Merge branch 'master' of https://github.com/apache/spark into dt-preprune cadd569 [qiping.lqp] add api docs 46b891f [qiping.lqp] fix bug e72c7e4 [qiping.lqp] add comments 845c6fa [qiping.lqp] fix style f195e83 [qiping.lqp] fix style 987cbf4 [qiping.lqp] fix bug ff34845 [qiping.lqp] separate calculation of predict of node from calculation of info gain ac42378 [qiping.lqp] add min info gain and min instances per node parameters in decision tree --- .../spark/mllib/tree/DecisionTree.scala | 72 ++++++++---- .../mllib/tree/configuration/Strategy.scala | 9 ++ .../tree/impl/DecisionTreeMetadata.scala | 7 +- .../tree/model/InformationGainStats.scala | 20 ++-- .../spark/mllib/tree/model/Predict.scala | 36 ++++++ .../apache/spark/mllib/tree/model/Split.scala | 2 + .../spark/mllib/tree/DecisionTreeSuite.scala | 103 ++++++++++++++++-- 7 files changed, 213 insertions(+), 36 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index d1309b2b20f54..98596569b8c95 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -130,7 +130,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Find best split for all nodes at a level. timer.start("findBestSplits") - val splitsStatsForLevel: Array[(Split, InformationGainStats)] = + val splitsStatsForLevel: Array[(Split, InformationGainStats, Predict)] = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") @@ -143,8 +143,9 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo timer.start("extractNodeInfo") val split = nodeSplitStats._1 val stats = nodeSplitStats._2 + val predict = nodeSplitStats._3.predict val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) - val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) + val node = new Node(nodeIndex, predict, isLeaf, Some(split), None, None, Some(stats)) logDebug("Node = " + node) nodes(nodeIndex) = node timer.stop("extractNodeInfo") @@ -425,7 +426,7 @@ object DecisionTree extends Serializable with Logging { splits: Array[Array[Split]], bins: Array[Array[Bin]], maxLevelForSingleGroup: Int, - timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats)] = { + timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats, Predict)] = { // split into groups to avoid memory overflow during aggregation if (level > maxLevelForSingleGroup) { // When information for all nodes at a given level cannot be stored in memory, @@ -434,7 +435,7 @@ object DecisionTree extends Serializable with Logging { // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. val numGroups = 1 << level - maxLevelForSingleGroup logDebug("numGroups = " + numGroups) - var bestSplits = new Array[(Split, InformationGainStats)](0) + var bestSplits = new Array[(Split, InformationGainStats, Predict)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 while (groupIndex < numGroups) { @@ -605,7 +606,7 @@ object DecisionTree extends Serializable with Logging { bins: Array[Array[Bin]], timer: TimeTracker, numGroups: Int = 1, - groupIndex: Int = 0): Array[(Split, InformationGainStats)] = { + groupIndex: Int = 0): Array[(Split, InformationGainStats, Predict)] = { /* * The high-level descriptions of the best split optimizations are noted here. @@ -705,7 +706,7 @@ object DecisionTree extends Serializable with Logging { // Calculate best splits for all nodes at a given level timer.start("chooseSplits") - val bestSplits = new Array[(Split, InformationGainStats)](numNodes) + val bestSplits = new Array[(Split, InformationGainStats, Predict)](numNodes) // Iterating over all nodes at this level var nodeIndex = 0 while (nodeIndex < numNodes) { @@ -734,28 +735,27 @@ object DecisionTree extends Serializable with Logging { topImpurity: Double, level: Int, metadata: DecisionTreeMetadata): InformationGainStats = { - val leftCount = leftImpurityCalculator.count val rightCount = rightImpurityCalculator.count - val totalCount = leftCount + rightCount - if (totalCount == 0) { - // Return arbitrary prediction. - return new InformationGainStats(0, topImpurity, topImpurity, topImpurity, 0) + // If left child or right child doesn't satisfy minimum instances per node, + // then this split is invalid, return invalid information gain stats. + if ((leftCount < metadata.minInstancesPerNode) || + (rightCount < metadata.minInstancesPerNode)) { + return InformationGainStats.invalidInformationGainStats } - val parentNodeAgg = leftImpurityCalculator.copy - parentNodeAgg.add(rightImpurityCalculator) + val totalCount = leftCount + rightCount + // impurity of parent node val impurity = if (level > 0) { topImpurity } else { + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) parentNodeAgg.calculate() } - val predict = parentNodeAgg.predict - val prob = parentNodeAgg.prob(predict) - val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 val rightImpurity = rightImpurityCalculator.calculate() @@ -764,7 +764,31 @@ object DecisionTree extends Serializable with Logging { val gain = impurity - leftWeight * leftImpurity - rightWeight * rightImpurity - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict, prob) + // if information gain doesn't satisfy minimum information gain, + // then this split is invalid, return invalid information gain stats. + if (gain < metadata.minInfoGain) { + return InformationGainStats.invalidInformationGainStats + } + + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity) + } + + /** + * Calculate predict value for current node, given stats of any split. + * Note that this function is called only once for each node. + * @param leftImpurityCalculator left node aggregates for a split + * @param rightImpurityCalculator right node aggregates for a node + * @return predict value for current node + */ + private def calculatePredict( + leftImpurityCalculator: ImpurityCalculator, + rightImpurityCalculator: ImpurityCalculator): Predict = { + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) + val predict = parentNodeAgg.predict + val prob = parentNodeAgg.prob(predict) + + new Predict(predict, prob) } /** @@ -780,12 +804,15 @@ object DecisionTree extends Serializable with Logging { nodeImpurity: Double, level: Int, metadata: DecisionTreeMetadata, - splits: Array[Array[Split]]): (Split, InformationGainStats) = { + splits: Array[Array[Split]]): (Split, InformationGainStats, Predict) = { logDebug("node impurity = " + nodeImpurity) + // calculate predict only once + var predict: Option[Predict] = None + // For each (feature, split), calculate the gain, and select the best (feature, split). - Range(0, metadata.numFeatures).map { featureIndex => + val (bestSplit, bestSplitStats) = Range(0, metadata.numFeatures).map { featureIndex => val numSplits = metadata.numSplits(featureIndex) if (metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. @@ -803,6 +830,7 @@ object DecisionTree extends Serializable with Logging { val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) + predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) (splitIdx, gainStats) @@ -816,6 +844,7 @@ object DecisionTree extends Serializable with Logging { Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) + predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) (splitIndex, gainStats) @@ -887,6 +916,7 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) + predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) (splitIndex, gainStats) @@ -898,6 +928,10 @@ object DecisionTree extends Serializable with Logging { (bestFeatureSplit, bestFeatureGainStats) } }.maxBy(_._2.gain) + + require(predict.isDefined, "must calculate predict for each node") + + (bestSplit, bestSplitStats, predict.get) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 23f74d5360fe5..987fe632c91ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -49,6 +49,13 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * k) implies the feature n is categorical with k categories 0, * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. + * @param minInstancesPerNode Minimum number of instances each child must have after split. + * Default value is 1. If a split cause left or right child + * to have less than minInstancesPerNode, + * this split will not be considered as a valid split. + * @param minInfoGain Minimum information gain a split must get. Default value is 0.0. + * If a split has less information gain than minInfoGain, + * this split will not be considered as a valid split. * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is * 256 MB. */ @@ -61,6 +68,8 @@ class Strategy ( val maxBins: Int = 32, val quantileCalculationStrategy: QuantileStrategy = Sort, val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), + val minInstancesPerNode: Int = 1, + val minInfoGain: Double = 0.0, val maxMemoryInMB: Int = 256) extends Serializable { if (algo == Classification) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index e95add7558bcf..5ceaa8154d11a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -45,7 +45,9 @@ private[tree] class DecisionTreeMetadata( val unorderedFeatures: Set[Int], val numBins: Array[Int], val impurity: Impurity, - val quantileStrategy: QuantileStrategy) extends Serializable { + val quantileStrategy: QuantileStrategy, + val minInstancesPerNode: Int, + val minInfoGain: Double) extends Serializable { def isUnordered(featureIndex: Int): Boolean = unorderedFeatures.contains(featureIndex) @@ -127,7 +129,8 @@ private[tree] object DecisionTreeMetadata { new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, - strategy.impurity, strategy.quantileCalculationStrategy) + strategy.impurity, strategy.quantileCalculationStrategy, + strategy.minInstancesPerNode, strategy.minInfoGain) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index fb12298e0f5d3..f3e2619bd8ba0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -26,20 +26,26 @@ import org.apache.spark.annotation.DeveloperApi * @param impurity current node impurity * @param leftImpurity left node impurity * @param rightImpurity right node impurity - * @param predict predicted value - * @param prob probability of the label (classification only) */ @DeveloperApi class InformationGainStats( val gain: Double, val impurity: Double, val leftImpurity: Double, - val rightImpurity: Double, - val predict: Double, - val prob: Double = 0.0) extends Serializable { + val rightImpurity: Double) extends Serializable { override def toString = { - "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f, prob = %f" - .format(gain, impurity, leftImpurity, rightImpurity, predict, prob) + "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" + .format(gain, impurity, leftImpurity, rightImpurity) } } + + +private[tree] object InformationGainStats { + /** + * An [[org.apache.spark.mllib.tree.model.InformationGainStats]] object to + * denote that current split doesn't satisfies minimum info gain or + * minimum number of instances per node. + */ + val invalidInformationGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala new file mode 100644 index 0000000000000..6fac2be2797bc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.model + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Predicted value for a node + * @param predict predicted value + * @param prob probability of the label (classification only) + */ +@DeveloperApi +private[tree] class Predict( + val predict: Double, + val prob: Double = 0.0) extends Serializable{ + + override def toString = { + "predict = %f, prob = %f".format(predict, prob) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 50fb48b40de3d..b7a85f58544a3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType +import org.apache.spark.mllib.tree.configuration.FeatureType +import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** * :: DeveloperApi :: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 69482f2acbb40..fd8547c1660fc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} -import org.apache.spark.mllib.tree.model.{DecisionTreeModel, Node} +import org.apache.spark.mllib.tree.model.{InformationGainStats, DecisionTreeModel, Node} import org.apache.spark.mllib.util.LocalSparkContext class DecisionTreeSuite extends FunSuite with LocalSparkContext { @@ -279,9 +279,10 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(split.threshold === Double.MinValue) val stats = bestSplits(0)._2 + val predict = bestSplits(0)._3 assert(stats.gain > 0) - assert(stats.predict === 1) - assert(stats.prob === 0.6) + assert(predict.predict === 1) + assert(predict.prob === 0.6) assert(stats.impurity > 0.2) } @@ -312,8 +313,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(split.threshold === Double.MinValue) val stats = bestSplits(0)._2 + val predict = bestSplits(0)._3.predict assert(stats.gain > 0) - assert(stats.predict === 0.6) + assert(predict === 0.6) assert(stats.impurity > 0.2) } @@ -387,7 +389,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._2.predict === 1) + assert(bestSplits(0)._3.predict === 1) } test("Binary classification stump with fixed label 0 for Entropy") { @@ -414,7 +416,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._2.predict === 0) + assert(bestSplits(0)._3.predict === 0) } test("Binary classification stump with fixed label 1 for Entropy") { @@ -441,7 +443,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(0)._2.gain === 0) assert(bestSplits(0)._2.leftImpurity === 0) assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._2.predict === 1) + assert(bestSplits(0)._3.predict === 1) } test("Second level node building with vs. without groups") { @@ -490,7 +492,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bestSplits(i)._2.impurity === bestSplitsWithGroups(i)._2.impurity) assert(bestSplits(i)._2.leftImpurity === bestSplitsWithGroups(i)._2.leftImpurity) assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) - assert(bestSplits(i)._2.predict === bestSplitsWithGroups(i)._2.predict) + assert(bestSplits(i)._3.predict === bestSplitsWithGroups(i)._3.predict) } } @@ -674,6 +676,91 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { validateClassifier(model, arr, 0.6) } + test("split must satisfy min instances per node requirements") { + val arr = new Array[LabeledPoint](3) + arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) + arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, + maxDepth = 2, numClassesForClassification = 2, minInstancesPerNode = 2) + + val model = DecisionTree.train(input, strategy) + assert(model.topNode.isLeaf) + assert(model.topNode.predict == 0.0) + val predicts = input.map(p => model.predict(p.features)).collect() + predicts.foreach { predict => + assert(predict == 0.0) + } + + // test for findBestSplits when no valid split can be found + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length == 1) + val bestInfoStats = bestSplits(0)._2 + assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + } + + test("don't choose split that doesn't satisfy min instance per node requirements") { + // if a split doesn't satisfy min instances per node requirements, + // this split is invalid, even though the information gain of split is large. + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0, 1.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(1.0, 1.0)) + arr(2) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) + arr(3) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, + maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2), + numClassesForClassification = 2, minInstancesPerNode = 2) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length == 1) + val bestSplit = bestSplits(0)._1 + val bestSplitStats = bestSplits(0)._1 + assert(bestSplit.feature == 1) + assert(bestSplitStats != InformationGainStats.invalidInformationGainStats) + } + + test("split must satisfy min info gain requirements") { + val arr = new Array[LabeledPoint](3) + arr(0) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))) + arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) + arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) + + val input = sc.parallelize(arr) + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, minInfoGain = 1.0) + + val model = DecisionTree.train(input, strategy) + assert(model.topNode.isLeaf) + assert(model.topNode.predict == 0.0) + val predicts = input.map(p => model.predict(p.features)).collect() + predicts.foreach { predict => + assert(predict == 0.0) + } + + // test for findBestSplits when no valid split can be found + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, + new Array[Node](0), splits, bins, 10) + + assert(bestSplits.length == 1) + val bestInfoStats = bestSplits(0)._2 + assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + } } object DecisionTreeSuite { From 84e2c8bfe41837baf2aeffa9741e4dbd14351981 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 10 Sep 2014 20:57:38 -0700 Subject: [PATCH 0656/1492] [SQL] Add test case with workaround for reading partitioned Avro files In order to read from partitioned Avro files we need to also set the `SERDEPROPERTIES` since `TBLPROPERTIES` are not passed to the initialization. This PR simply adds a test to make sure we don't break this workaround. Author: Michael Armbrust Closes #2340 from marmbrus/avroPartitioned and squashes the following commits: 6b969d6 [Michael Armbrust] fix style fea2124 [Michael Armbrust] Add test case with workaround for reading partitioned avro files. --- .../org/apache/spark/sql/hive/TestHive.scala | 69 ++++++++++++++++++- ...roSerDe-0-e4501461c855cc9071a872a64186c3de | 8 +++ .../sql/hive/execution/HiveSerDeSuite.scala | 2 + 3 files changed, 78 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a013f3f7a805f..6974f3e581b97 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -269,7 +269,74 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { |) """.stripMargin.cmd, s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/episodes.avro")}' INTO TABLE episodes".cmd - ) + ), + // THIS TABLE IS NOT THE SAME AS THE HIVE TEST TABLE episodes_partitioned AS DYNAMIC PARITIONING + // IS NOT YET SUPPORTED + TestTable("episodes_part", + s"""CREATE TABLE episodes_part (title STRING, air_date STRING, doctor INT) + |PARTITIONED BY (doctor_pt INT) + |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' + |STORED AS + |INPUTFORMAT '${classOf[AvroContainerInputFormat].getCanonicalName}' + |OUTPUTFORMAT '${classOf[AvroContainerOutputFormat].getCanonicalName}' + |TBLPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + // WORKAROUND: Required to pass schema to SerDe for partitioned tables. + // TODO: Pass this automatically from the table to partitions. + s""" + |ALTER TABLE episodes_part SET SERDEPROPERTIES ( + | 'avro.schema.literal'='{ + | "type": "record", + | "name": "episodes", + | "namespace": "testing.hive.avro.serde", + | "fields": [ + | { + | "name": "title", + | "type": "string", + | "doc": "episode title" + | }, + | { + | "name": "air_date", + | "type": "string", + | "doc": "initial date" + | }, + | { + | "name": "doctor", + | "type": "int", + | "doc": "main actor playing the Doctor in episode" + | } + | ] + | }' + |) + """.stripMargin.cmd, + s""" + INSERT OVERWRITE TABLE episodes_part PARTITION (doctor_pt=1) + SELECT title, air_date, doctor FROM episodes + """.cmd + ) ) hiveQTestUtilTables.foreach(registerTestTable) diff --git a/sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de b/sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de new file mode 100644 index 0000000000000..49c8434730ffa --- /dev/null +++ b/sql/hive/src/test/resources/golden/Read Partitioned with AvroSerDe-0-e4501461c855cc9071a872a64186c3de @@ -0,0 +1,8 @@ +The Eleventh Hour 3 April 2010 11 1 +The Doctor's Wife 14 May 2011 11 1 +Horror of Fang Rock 3 September 1977 4 1 +An Unearthly Child 23 November 1963 1 1 +The Mysterious Planet 6 September 1986 6 1 +Rose 26 March 2005 9 1 +The Power of the Daleks 5 November 1966 2 1 +Castrolava 4 January 1982 5 1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 8bc72384a64ee..7486bfa82b00b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -37,4 +37,6 @@ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") + + createQueryTest("Read Partitioned with AvroSerDe", "SELECT * FROM episodes_part") } From f92cde24e8f30dddd5bcec71bb3687498c1406da Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 10 Sep 2014 20:59:40 -0700 Subject: [PATCH 0657/1492] [SPARK-3447][SQL] Remove explicit conversion with JListWrapper to avoid NPE Author: Michael Armbrust Closes #2323 from marmbrus/kryoJListNPE and squashes the following commits: 9634f11 [Michael Armbrust] Rollback JSON RDD changes 4d4d93c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into kryoJListNPE 646976b [Michael Armbrust] Fix JSON RDD Conversion too 59065bc [Michael Armbrust] Remove explicit conversion to avoid NPE --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a2f334aab9fdf..c551c7c9877e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -460,7 +460,6 @@ class SQLContext(@transient val sparkContext: SparkContext) rdd: RDD[Array[Any]], schema: StructType): SchemaRDD = { import scala.collection.JavaConversions._ - import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} def needsConversion(dataType: DataType): Boolean = dataType match { case ByteType => true @@ -482,8 +481,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case (null, _) => null case (c: java.util.List[_], ArrayType(elementType, _)) => - val converted = c.map { e => convert(e, elementType)} - JListWrapper(converted) + c.map { e => convert(e, elementType)}: Seq[Any] case (c, ArrayType(elementType, _)) if c.getClass.isArray => c.asInstanceOf[Array[_]].map(e => convert(e, elementType)): Seq[Any] From c27718f376483dbe6290de612094c8d4ce9b16b4 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Wed, 10 Sep 2014 21:01:53 -0700 Subject: [PATCH 0658/1492] [SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit LogicalPlan contains a ‘resolved’ attribute indicating that all of its execution requirements have been resolved. This attribute is not checked before query execution. The analyzer contains a step to check that all Expressions are resolved, but this is not equivalent to checking all LogicalPlans. In particular, the Union plan’s implementation of ‘resolved’ verifies that the types of its children’s columns are compatible. Because the analyzer does not check that a Union plan is resolved, it is possible to execute a Union plan that outputs different types in the same column. See SPARK-2781 for an example. This patch adds two checks to the analyzer’s CheckResolution rule. First, each logical plan is checked to see if it is not resolved despite its children being resolved. This allows the ‘problem’ unresolved plan to be included in the TreeNodeException for reporting. Then as a backstop the root plan is checked to see if it is resolved, which recursively checks that the entire plan tree is resolved. Note that the resolved attribute is implemented recursively, and this patch also explicitly checks the resolved attribute on each logical plan in the tree. I assume the query plan trees will not be large enough for this redundant checking to meaningfully impact performance. Because this patch starts validating that LogicalPlans are resolved before execution, I had to fix some cases where unresolved plans were passing through the analyzer as part of the implementation of the hive query system. In particular, HiveContext applies the CreateTables and PreInsertionCasts, and ExtractPythonUdfs rules manually after the analyzer runs. I moved these rules to the analyzer stage (for hive queries only), in the process completing a code TODO indicating the rules should be moved to the analyzer. It’s worth noting that moving the CreateTables rule means introducing an analyzer rule with a significant side effect - in this case the side effect is creating a hive table. The rule will only attempt to create a table once even if its batch is executed multiple times, because it converts the InsertIntoCreatedTable plan it matches against into an InsertIntoTable. Additionally, these hive rules must be added to the Resolution batch rather than as a separate batch because hive rules rules may be needed to resolve non-root nodes, leaving the root to be resolved on a subsequent batch iteration. For example, the hive compatibility test auto_smb_mapjoin_14, and others, make use of a query plan where the root is a Union and its children are each a hive InsertIntoTable. Mixing the custom hive rules with standard analyzer rules initially resulted in an additional failure because of policy differences between spark sql and hive when casting a boolean to a string. Hive casts booleans to strings as “true” / “false” while spark sql casts booleans to strings as “1” / “0” (causing the cast1.q test to fail). This behavior is a result of the BooleanCasts rule in HiveTypeCoercion.scala, and from looking at the implementation of BooleanCasts I think converting to to “1”/“0” is potentially a programming mistake. (If the BooleanCasts rule is disabled, casting produces “true”/“false” instead.) I believe “true” / “false” should be the behavior for spark sql - I changed the behavior so bools are converted to “true”/“false” to be consistent with hive, and none of the existing spark tests failed. Finally, in some initial testing with hive it appears that an implicit type coercion of boolean to string results in a lowercase string, e.g. CONCAT( TRUE, “” ) -> “true” while an explicit cast produces an all caps string, e.g. CAST( TRUE AS STRING ) -> “TRUE”. The change I’ve made just converts to lowercase strings in all cases. I believe it is at least more correct than the existing spark sql implementation where all Cast expressions become “1” / “0”. Author: Aaron Staple Closes #1706 from staple/SPARK-2781 and squashes the following commits: 32683c4 [Aaron Staple] Fix compilation failure due to merge. 7c77fda [Aaron Staple] Move ExtractPythonUdfs to Analyzer's extendedRules in HiveContext. d49bfb3 [Aaron Staple] Address review comments. 915b690 [Aaron Staple] Fix merge issue causing compilation failure. 701dcd2 [Aaron Staple] [SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer. --- .../sql/catalyst/analysis/Analyzer.scala | 21 +++++++-- .../catalyst/analysis/HiveTypeCoercion.scala | 4 ++ .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 13 +++++- .../analysis/HiveTypeCoercionSuite.scala | 15 +++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 45 +++++++++++++++++-- .../apache/spark/sql/hive/HiveContext.scala | 11 +++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 16 ++++--- 8 files changed, 107 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4a9524074132e..574d96d92942b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -40,7 +40,12 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool // TODO: pass this in as a parameter. val fixedPoint = FixedPoint(100) - val batches: Seq[Batch] = Seq( + /** + * Override to provide additional rules for the "Resolution" batch. + */ + val extendedRules: Seq[Rule[LogicalPlan]] = Nil + + lazy val batches: Seq[Batch] = Seq( Batch("MultiInstanceRelations", Once, NewRelationInstances), Batch("CaseInsensitiveAttributeReferences", Once, @@ -54,8 +59,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool StarExpansion :: ResolveFunctions :: GlobalAggregates :: - UnresolvedHavingClauseAttributes :: - typeCoercionRules :_*), + UnresolvedHavingClauseAttributes :: + typeCoercionRules ++ + extendedRules : _*), Batch("Check Analysis", Once, CheckResolution), Batch("AnalysisOperators", fixedPoint, @@ -63,7 +69,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) /** - * Makes sure all attributes have been resolved. + * Makes sure all attributes and logical plans have been resolved. */ object CheckResolution extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = { @@ -71,6 +77,13 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case p if p.expressions.exists(!_.resolved) => throw new TreeNodeException(p, s"Unresolved attributes: ${p.expressions.filterNot(_.resolved).mkString(",")}") + case p if !p.resolved && p.childrenResolved => + throw new TreeNodeException(p, "Unresolved plan found") + } match { + // As a backstop, use the root node to check that the entire plan tree is resolved. + case p if !p.resolved => + throw new TreeNodeException(p, "Unresolved plan in tree") + case p => p } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index bd8131c9af6e0..79e5283e86a37 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -286,6 +286,10 @@ trait HiveTypeCoercion { // If the data type is not boolean and is being cast boolean, turn it into a comparison // with the numeric value, i.e. x != 0. This will coerce the type into numeric type. case Cast(e, BooleanType) if e.dataType != BooleanType => Not(EqualTo(e, Literal(0))) + // Stringify boolean if casting to StringType. + // TODO Ensure true/false string letter casing is consistent with Hive in all cases. + case Cast(e, StringType) if e.dataType == BooleanType => + If(e, Literal("true"), Literal("false")) // Turn true into 1, and false into 0 if casting boolean into other types. case Cast(e, dataType) if e.dataType == BooleanType => Cast(If(e, Literal(1), Literal(0)), dataType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index bae491f07c13f..ede431ad4ab27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -58,7 +58,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { /** * Returns true if this expression and all its children have been resolved to a specific schema - * and false if it is still contains any unresolved placeholders. Implementations of LogicalPlan + * and false if it still contains any unresolved placeholders. Implementations of LogicalPlan * can override this (e.g. * [[org.apache.spark.sql.catalyst.analysis.UnresolvedRelation UnresolvedRelation]] * should return `false`). diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 0a4fde3de7752..5809a108ff62e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -93,6 +93,17 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { val e = intercept[TreeNodeException[_]] { caseSensitiveAnalyze(Project(Seq(UnresolvedAttribute("abcd")), testRelation)) } - assert(e.getMessage().toLowerCase.contains("unresolved")) + assert(e.getMessage().toLowerCase.contains("unresolved attribute")) + } + + test("throw errors for unresolved plans during analysis") { + case class UnresolvedTestPlan() extends LeafNode { + override lazy val resolved = false + override def output = Nil + } + val e = intercept[TreeNodeException[_]] { + caseSensitiveAnalyze(UnresolvedTestPlan()) + } + assert(e.getMessage().toLowerCase.contains("unresolved plan")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index ba8b853b6f99e..baeb9b0cf5964 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} import org.apache.spark.sql.catalyst.types._ class HiveTypeCoercionSuite extends FunSuite { @@ -84,4 +86,17 @@ class HiveTypeCoercionSuite extends FunSuite { widenTest(StringType, MapType(IntegerType, StringType, true), None) widenTest(ArrayType(IntegerType), StructType(Seq()), None) } + + test("boolean casts") { + val booleanCasts = new HiveTypeCoercion { }.BooleanCasts + def ruleTest(initial: Expression, transformed: Expression) { + val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) + assert(booleanCasts(Project(Seq(Alias(initial, "a")()), testRelation)) == + Project(Seq(Alias(transformed, "a")()), testRelation)) + } + // Remove superflous boolean -> boolean casts. + ruleTest(Cast(Literal(true), BooleanType), Literal(true)) + // Stringify boolean when casting to string. + ruleTest(Cast(Literal(false), StringType), If(Literal(false), Literal("true"), Literal("false"))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 514ac543df92a..67563b6c55f4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll @@ -477,18 +478,48 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null))) } - test("EXCEPT") { + test("UNION") { + checkAnswer( + sql("SELECT * FROM lowerCaseData UNION SELECT * FROM upperCaseData"), + (1, "A") :: (1, "a") :: (2, "B") :: (2, "b") :: (3, "C") :: (3, "c") :: + (4, "D") :: (4, "d") :: (5, "E") :: (6, "F") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData UNION SELECT * FROM lowerCaseData"), + (1, "a") :: (2, "b") :: (3, "c") :: (4, "d") :: Nil) + checkAnswer( + sql("SELECT * FROM lowerCaseData UNION ALL SELECT * FROM lowerCaseData"), + (1, "a") :: (1, "a") :: (2, "b") :: (2, "b") :: (3, "c") :: (3, "c") :: + (4, "d") :: (4, "d") :: Nil) + } + test("UNION with column mismatches") { + // Column name mismatches are allowed. + checkAnswer( + sql("SELECT n,l FROM lowerCaseData UNION SELECT N as x1, L as x2 FROM upperCaseData"), + (1, "A") :: (1, "a") :: (2, "B") :: (2, "b") :: (3, "C") :: (3, "c") :: + (4, "D") :: (4, "d") :: (5, "E") :: (6, "F") :: Nil) + // Column type mismatches are not allowed, forcing a type coercion. checkAnswer( - sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData "), + sql("SELECT n FROM lowerCaseData UNION SELECT L FROM upperCaseData"), + ("1" :: "2" :: "3" :: "4" :: "A" :: "B" :: "C" :: "D" :: "E" :: "F" :: Nil).map(Tuple1(_))) + // Column type mismatches where a coercion is not possible, in this case between integer + // and array types, trigger a TreeNodeException. + intercept[TreeNodeException[_]] { + sql("SELECT data FROM arrayData UNION SELECT 1 FROM arrayData").collect() + } + } + + test("EXCEPT") { + checkAnswer( + sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM upperCaseData"), (1, "a") :: (2, "b") :: (3, "c") :: (4, "d") :: Nil) checkAnswer( - sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData "), Nil) + sql("SELECT * FROM lowerCaseData EXCEPT SELECT * FROM lowerCaseData"), Nil) checkAnswer( - sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData "), Nil) + sql("SELECT * FROM upperCaseData EXCEPT SELECT * FROM upperCaseData"), Nil) } test("INTERSECT") { @@ -634,6 +665,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), Seq() ) + } + test("cast boolean to string") { + // TODO Ensure true/false string letter casing is consistent with Hive in all cases. + checkAnswer( + sql("SELECT CAST(TRUE AS STRING), CAST(FALSE AS STRING) FROM testData LIMIT 1"), + ("true", "false") :: Nil) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ced8397972fbd..e0be09e6793ea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -262,7 +262,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* An analyzer that uses the Hive metastore. */ @transient override protected[sql] lazy val analyzer = - new Analyzer(catalog, functionRegistry, caseSensitive = false) + new Analyzer(catalog, functionRegistry, caseSensitive = false) { + override val extendedRules = + catalog.CreateTables :: + catalog.PreInsertionCasts :: + ExtractPythonUdfs :: + Nil + } /** * Runs the specified SQL query using Hive. @@ -353,9 +359,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /** Extends QueryExecution with hive specific features. */ protected[sql] abstract class QueryExecution extends super.QueryExecution { - // TODO: Create mixin for the analyzer instead of overriding things here. - override lazy val optimizedPlan = - optimizer(ExtractPythonUdfs(catalog.PreInsertionCasts(catalog.CreateTables(analyzed)))) override lazy val toRdd: RDD[Row] = executedPlan.execute().map(_.copy()) 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 6571c35499ef4..dfa2a7a9d28bb 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 @@ -109,6 +109,9 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with */ object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Wait until children are resolved. + case p: LogicalPlan if !p.childrenResolved => p + case InsertIntoCreatedTable(db, tableName, child) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) @@ -116,8 +119,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with createTable(databaseName, tblName, child.output) InsertIntoTable( - EliminateAnalysisOperators( - lookupRelation(Some(databaseName), tblName, None)), + lookupRelation(Some(databaseName), tblName, None), Map.empty, child, overwrite = false) @@ -130,15 +132,17 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with */ object PreInsertionCasts extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan.transform { - // Wait until children are resolved + // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => + case p @ InsertIntoTable( + LowerCaseSchema(table: MetastoreRelation), _, child, _) => castChildOutput(p, table, child) case p @ logical.InsertIntoTable( - InMemoryRelation(_, _, _, - HiveTableScan(_, table, _)), _, child, _) => + LowerCaseSchema( + InMemoryRelation(_, _, _, + HiveTableScan(_, table, _))), _, child, _) => castChildOutput(p, table, child) } From ed1980ffa9ccb87d76694ba910ef22df034bca49 Mon Sep 17 00:00:00 2001 From: Chris Cope Date: Thu, 11 Sep 2014 08:13:07 -0500 Subject: [PATCH 0659/1492] [SPARK-2140] Updating heap memory calculation for YARN stable and alpha. Updated pull request, reflecting YARN stable and alpha states. I am getting intermittent test failures on my own test infrastructure. Is that tracked anywhere yet? Author: Chris Cope Closes #2253 from copester/master and squashes the following commits: 5ad89da [Chris Cope] [SPARK-2140] Removing calculateAMMemory functions since they are no longer needed. 52b4e45 [Chris Cope] [SPARK-2140] Updating heap memory calculation for YARN stable and alpha. --- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 8 -------- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 4 +--- .../org/apache/spark/deploy/yarn/ClientBaseSuite.scala | 3 --- .../main/scala/org/apache/spark/deploy/yarn/Client.scala | 9 --------- 4 files changed, 1 insertion(+), 23 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 10fc39bba87d1..aff9ab71f0937 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -103,14 +103,6 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa appContext } - def calculateAMMemory(newApp: GetNewApplicationResponse): Int = { - val minResMemory = newApp.getMinimumResourceCapability().getMemory() - val amMemory = ((args.amMemory / minResMemory) * minResMemory) + - ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - - memoryOverhead) - amMemory - } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { // Setup security tokens. val dob = new DataOutputBuffer() diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8075b7a7fb837..c96f731923d22 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -300,8 +300,6 @@ trait ClientBase extends Logging { retval.toString } - def calculateAMMemory(newApp: GetNewApplicationResponse): Int - def setupSecurityToken(amContainer: ContainerLaunchContext) def createContainerLaunchContext( @@ -346,7 +344,7 @@ trait ClientBase extends Logging { } amContainer.setEnvironment(env) - val amMemory = calculateAMMemory(newApp) + val amMemory = args.amMemory val javaOpts = ListBuffer[String]() diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 68cc2890f3a22..5480eca7c832c 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -238,9 +238,6 @@ class ClientBaseSuite extends FunSuite with Matchers { val sparkConf: SparkConf, val yarnConf: YarnConfiguration) extends ClientBase { - override def calculateAMMemory(newApp: GetNewApplicationResponse): Int = - throw new UnsupportedOperationException() - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = throw new UnsupportedOperationException() diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 313a0d21ce181..82e45e3e7ad54 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -103,15 +103,6 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa clusterMetrics.getNumNodeManagers) } - def calculateAMMemory(newApp: GetNewApplicationResponse) :Int = { - // TODO: Need a replacement for the following code to fix -Xmx? - // val minResMemory: Int = newApp.getMinimumResourceCapability().getMemory() - // var amMemory = ((args.amMemory / minResMemory) * minResMemory) + - // ((if ((args.amMemory % minResMemory) == 0) 0 else minResMemory) - - // memoryOverhead ) - args.amMemory - } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { // Setup security tokens. val dob = new DataOutputBuffer() From 1ef656ea85b4b93c7b0f3cf8042b63a0de0901cb Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 11 Sep 2014 11:50:36 -0700 Subject: [PATCH 0660/1492] [SPARK-3047] [PySpark] add an option to use str in textFileRDD str is much efficient than unicode (both CPU and memory), it'e better to use str in textFileRDD. In order to keep compatibility, use unicode by default. (Maybe change it in the future). use_unicode=True: daviesliudm:~/work/spark$ time python wc.py (u'./universe/spark/sql/core/target/java/org/apache/spark/sql/execution/ExplainCommand$.java', 7776) real 2m8.298s user 0m0.185s sys 0m0.064s use_unicode=False daviesliudm:~/work/spark$ time python wc.py ('./universe/spark/sql/core/target/java/org/apache/spark/sql/execution/ExplainCommand$.java', 7776) real 1m26.402s user 0m0.182s sys 0m0.062s We can see that it got 32% improvement! Author: Davies Liu Closes #1951 from davies/unicode and squashes the following commits: 8352d57 [Davies Liu] update version number a286f2f [Davies Liu] rollback loads() 85246e5 [Davies Liu] add docs for use_unicode a0295e1 [Davies Liu] add an option to use str in textFile() --- python/pyspark/context.py | 16 ++++++++++++---- python/pyspark/serializers.py | 18 +++++++++++------- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 84bc0a3b7ccd0..3ab98e262df31 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -331,12 +331,16 @@ def pickleFile(self, name, minPartitions=None): return RDD(self._jsc.objectFile(name, minPartitions), self, BatchedSerializer(PickleSerializer())) - def textFile(self, name, minPartitions=None): + def textFile(self, name, minPartitions=None, use_unicode=True): """ Read a text file from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI, and return it as an RDD of Strings. + If use_unicode is False, the strings will be kept as `str` (encoding + as `utf-8`), which is faster and smaller than unicode. (Added in + Spark 1.2) + >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: ... testFile.write("Hello world!") @@ -346,9 +350,9 @@ def textFile(self, name, minPartitions=None): """ minPartitions = minPartitions or min(self.defaultParallelism, 2) return RDD(self._jsc.textFile(name, minPartitions), self, - UTF8Deserializer()) + UTF8Deserializer(use_unicode)) - def wholeTextFiles(self, path, minPartitions=None): + def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): """ Read a directory of text files from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system @@ -356,6 +360,10 @@ def wholeTextFiles(self, path, minPartitions=None): key-value pair, where the key is the path of each file, the value is the content of each file. + If use_unicode is False, the strings will be kept as `str` (encoding + as `utf-8`), which is faster and smaller than unicode. (Added in + Spark 1.2) + For example, if you have the following files:: hdfs://a-hdfs-path/part-00000 @@ -386,7 +394,7 @@ def wholeTextFiles(self, path, minPartitions=None): """ minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.wholeTextFiles(path, minPartitions), self, - PairDeserializer(UTF8Deserializer(), UTF8Deserializer())) + PairDeserializer(UTF8Deserializer(use_unicode), UTF8Deserializer(use_unicode))) def _dictToJavaMap(self, d): jm = self._jvm.java.util.HashMap() diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 55e6cf3308611..7b2710b913128 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -429,18 +429,22 @@ class UTF8Deserializer(Serializer): Deserializes streams written by String.getBytes. """ + def __init__(self, use_unicode=False): + self.use_unicode = use_unicode + def loads(self, stream): length = read_int(stream) - return stream.read(length).decode('utf8') + s = stream.read(length) + return s.decode("utf-8") if self.use_unicode else s def load_stream(self, stream): - while True: - try: + try: + while True: yield self.loads(stream) - except struct.error: - return - except EOFError: - return + except struct.error: + return + except EOFError: + return def read_long(stream): From ca83f1e2c4dfa519e44b837b6815cba3b4526d92 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 11 Sep 2014 11:57:01 -0700 Subject: [PATCH 0661/1492] [SPARK-2917] [SQL] Avoid table creation in logical plan analyzing for CTAS Author: Cheng Hao Closes #1846 from chenghao-intel/ctas and squashes the following commits: 56a0578 [Cheng Hao] remove the unused imports 9a57abc [Cheng Hao] Avoid table creation in logical plan analyzing --- .../plans/logical/basicOperators.scala | 3 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 4 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 14 +--- .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../spark/sql/hive/HiveStrategies.scala | 10 +++ .../hive/execution/CreateTableAsSelect.scala | 73 +++++++++++++++++++ .../hive/execution/InsertIntoHiveTable.scala | 6 +- .../sql/hive/execution/SQLQuerySuite.scala | 9 +++ 8 files changed, 104 insertions(+), 17 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 4adfb189372d6..5d10754c7b028 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -114,11 +114,12 @@ case class InsertIntoTable( } } -case class InsertIntoCreatedTable( +case class CreateTableAsSelect( databaseName: Option[String], tableName: String, child: LogicalPlan) extends UnaryNode { override def output = child.output + override lazy val resolved = (databaseName != None && childrenResolved) } case class WriteToFile( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 2f3033a5f94f0..e52eeb3e1c47e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -54,7 +54,7 @@ private[sql] trait SchemaRDDLike { @transient protected[spark] val logicalPlan: LogicalPlan = baseLogicalPlan match { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. - case _: Command | _: InsertIntoTable | _: InsertIntoCreatedTable | _: WriteToFile => + case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToFile => queryExecution.toRdd SparkLogicalPlan(queryExecution.executedPlan)(sqlContext) case _ => @@ -124,7 +124,7 @@ private[sql] trait SchemaRDDLike { */ @Experimental def saveAsTable(tableName: String): Unit = - sqlContext.executePlan(InsertIntoCreatedTable(None, tableName, logicalPlan)).toRdd + sqlContext.executePlan(CreateTableAsSelect(None, tableName, logicalPlan)).toRdd /** Returns the schema as a string in the tree format. * 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 dfa2a7a9d28bb..2c0db9be57e54 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 @@ -54,8 +54,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with db: Option[String], tableName: String, alias: Option[String]): LogicalPlan = synchronized { - val (dbName, tblName) = processDatabaseAndTableName(db, tableName) - val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) + val (databaseName, tblName) = processDatabaseAndTableName( + db.getOrElse(hive.sessionState.getCurrentDatabase), tableName) val table = client.getTable(databaseName, tblName) val partitions: Seq[Partition] = if (table.isPartitioned) { @@ -112,17 +112,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case InsertIntoCreatedTable(db, tableName, child) => + case CreateTableAsSelect(db, tableName, child) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - createTable(databaseName, tblName, child.output) - - InsertIntoTable( - lookupRelation(Some(databaseName), tblName, None), - Map.empty, - child, - overwrite = false) + CreateTableAsSelect(Some(databaseName), tableName, child) } } 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 c98287c6aa662..21ecf17028dbc 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 @@ -489,7 +489,7 @@ private[hive] object HiveQl { val (db, tableName) = extractDbNameTableName(tableNameParts) - InsertIntoCreatedTable(db, tableName, nodeToPlan(query)) + CreateTableAsSelect(db, tableName, nodeToPlan(query)) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 72cc01cdf4c84..43dd3d234f73a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -165,6 +165,16 @@ private[hive] trait HiveStrategies { InMemoryRelation(_, _, _, HiveTableScan(_, table, _)), partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil + case logical.CreateTableAsSelect(database, tableName, child) => + val query = planLater(child) + CreateTableAsSelect( + database.get, + tableName, + query, + InsertIntoHiveTable(_: MetastoreRelation, + Map(), + query, + true)(hiveContext)) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala new file mode 100644 index 0000000000000..71ea774d77795 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.annotation.Experimental +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LowerCaseSchema +import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.MetastoreRelation + +/** + * :: Experimental :: + * Create table and insert the query result into it. + * @param database the database name of the new relation + * @param tableName the table name of the new relation + * @param insertIntoRelation function of creating the `InsertIntoHiveTable` + * by specifying the `MetaStoreRelation`, the data will be inserted into that table. + * TODO Add more table creating properties, e.g. SerDe, StorageHandler, in-memory cache etc. + */ +@Experimental +case class CreateTableAsSelect( + database: String, + tableName: String, + query: SparkPlan, + insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) + extends LeafNode with Command { + + def output = Seq.empty + + // A lazy computing of the metastoreRelation + private[this] lazy val metastoreRelation: MetastoreRelation = { + // Create the table + val sc = sqlContext.asInstanceOf[HiveContext] + sc.catalog.createTable(database, tableName, query.output, false) + // Get the Metastore Relation + sc.catalog.lookupRelation(Some(database), tableName, None) match { + case LowerCaseSchema(r: MetastoreRelation) => r + case o: MetastoreRelation => o + } + } + + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + insertIntoRelation(metastoreRelation).execute + Seq.empty[Row] + } + + override def execute(): RDD[Row] = { + sideEffectResult + sparkContext.emptyRDD[Row] + } + + override def argString: String = { + s"[Database:$database, TableName: $tableName, InsertIntoHiveTable]\n" + query.toString + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 39033bdeac4b0..a284a91a91e31 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -53,9 +53,9 @@ case class InsertIntoHiveTable( (@transient sc: HiveContext) extends UnaryNode { - val outputClass = newSerializer(table.tableDesc).getSerializedClass - @transient private val hiveContext = new Context(sc.hiveconf) - @transient private val db = Hive.get(sc.hiveconf) + @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass + @transient private lazy val hiveContext = new Context(sc.hiveconf) + @transient private lazy val db = Hive.get(sc.hiveconf) private def newSerializer(tableDesc: TableDesc): Serializer = { val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index b99caf77bce28..679efe082f2a0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.QueryTest + +import org.apache.spark.sql.Row import org.apache.spark.sql.hive.test.TestHive._ case class Nested1(f1: Nested2) @@ -54,4 +56,11 @@ class SQLQuerySuite extends QueryTest { sql("SELECT f1.f2.f3 FROM nested"), 1) } + + test("test CTAS") { + checkAnswer(sql("CREATE TABLE test_ctas_123 AS SELECT key, value FROM src"), Seq.empty[Row]) + checkAnswer( + sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), + sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) + } } From 4bc9e046cb8922923dff254e3e621fb4de656f98 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 11 Sep 2014 15:23:33 -0700 Subject: [PATCH 0662/1492] [SPARK-3390][SQL] sqlContext.jsonRDD fails on a complex structure of JSON array and JSON object nesting This PR aims to correctly handle JSON arrays in the type of `ArrayType(...(ArrayType(StructType)))`. JIRA: https://issues.apache.org/jira/browse/SPARK-3390. Author: Yin Huai Closes #2364 from yhuai/SPARK-3390 and squashes the following commits: 46db418 [Yin Huai] Handle JSON arrays in the type of ArrayType(...(ArrayType(StructType))). --- .../org/apache/spark/sql/json/JsonRDD.scala | 66 +++++++++++-------- .../org/apache/spark/sql/json/JsonSuite.scala | 29 +++++++- .../apache/spark/sql/json/TestJsonData.scala | 30 ++++++++- 3 files changed, 96 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 70062eae3b7ce..873221835daf8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -68,8 +68,15 @@ private[sql] object JsonRDD extends Logging { val (topLevel, structLike) = values.partition(_.size == 1) val topLevelFields = topLevel.filter { name => resolved.get(prefix ++ name).get match { - case ArrayType(StructType(Nil), _) => false - case ArrayType(_, _) => true + case ArrayType(elementType, _) => { + def hasInnerStruct(t: DataType): Boolean = t match { + case s: StructType => false + case ArrayType(t1, _) => hasInnerStruct(t1) + case o => true + } + + hasInnerStruct(elementType) + } case struct: StructType => false case _ => true } @@ -84,7 +91,18 @@ private[sql] object JsonRDD extends Logging { val dataType = resolved.get(prefix :+ name).get dataType match { case array: ArrayType => - Some(StructField(name, ArrayType(structType, array.containsNull), nullable = true)) + // The pattern of this array is ArrayType(...(ArrayType(StructType))). + // Since the inner struct of array is a placeholder (StructType(Nil)), + // we need to replace this placeholder with the actual StructType (structType). + def getActualArrayType( + innerStruct: StructType, + currentArray: ArrayType): ArrayType = currentArray match { + case ArrayType(s: StructType, containsNull) => + ArrayType(innerStruct, containsNull) + case ArrayType(a: ArrayType, containsNull) => + ArrayType(getActualArrayType(innerStruct, a), containsNull) + } + Some(StructField(name, getActualArrayType(structType, array), nullable = true)) case struct: StructType => Some(StructField(name, structType, nullable = true)) // dataType is StringType means that we have resolved type conflicts involving // primitive types and complex types. So, the type of name has been relaxed to @@ -168,8 +186,7 @@ private[sql] object JsonRDD extends Logging { /** * Returns the element type of an JSON array. We go through all elements of this array * to detect any possible type conflict. We use [[compatibleType]] to resolve - * type conflicts. Right now, when the element of an array is another array, we - * treat the element as String. + * type conflicts. */ private def typeOfArray(l: Seq[Any]): ArrayType = { val containsNull = l.exists(v => v == null) @@ -216,18 +233,24 @@ private[sql] object JsonRDD extends Logging { } case (key: String, array: Seq[_]) => { // The value associated with the key is an array. - typeOfArray(array) match { + // Handle inner structs of an array. + def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match { case ArrayType(StructType(Nil), containsNull) => { // The elements of this arrays are structs. - array.asInstanceOf[Seq[Map[String, Any]]].flatMap { + v.asInstanceOf[Seq[Map[String, Any]]].flatMap { element => allKeysWithValueTypes(element) }.map { - case (k, dataType) => (s"$key.$k", dataType) - } :+ (key, ArrayType(StructType(Nil), containsNull)) + case (k, t) => (s"$key.$k", t) + } } - case ArrayType(elementType, containsNull) => - (key, ArrayType(elementType, containsNull)) :: Nil + case ArrayType(t1, containsNull) => + v.asInstanceOf[Seq[Any]].flatMap { + element => buildKeyPathForInnerStructs(element, t1) + } + case other => Nil } + val elementType = typeOfArray(array) + buildKeyPathForInnerStructs(array, elementType) :+ (key, elementType) } case (key: String, value) => (key, typeOfPrimitiveValue(value)) :: Nil } @@ -339,8 +362,6 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case ArrayType(elementType, _) => - value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case StringType => toString(value) case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) @@ -348,6 +369,10 @@ private[sql] object JsonRDD extends Logging { case DecimalType => toDecimal(value) case BooleanType => value.asInstanceOf[BooleanType.JvmType] case NullType => null + + case ArrayType(elementType, _) => + value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) + case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], struct) } } } @@ -356,22 +381,9 @@ private[sql] object JsonRDD extends Logging { // TODO: Reuse the row instead of creating a new one for every record. val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { - // StructType - case (StructField(name, fields: StructType, _), i) => - row.update(i, json.get(name).flatMap(v => Option(v)).map( - v => asRow(v.asInstanceOf[Map[String, Any]], fields)).orNull) - - // ArrayType(StructType) - case (StructField(name, ArrayType(structType: StructType, _), _), i) => - row.update(i, - json.get(name).flatMap(v => Option(v)).map( - v => v.asInstanceOf[Seq[Any]].map( - e => asRow(e.asInstanceOf[Map[String, Any]], structType))).orNull) - - // Other cases case (StructField(name, dataType, _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( - enforceCorrectType(_, dataType)).getOrElse(null)) + enforceCorrectType(_, dataType)).orNull) } row diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 301d482d27d86..b50d93855405a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -591,8 +591,35 @@ class JsonSuite extends QueryTest { (true, "str1") :: Nil ) checkAnswer( - sql("select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1] from jsonTable"), + sql( + """ + |select complexArrayOfStruct[0].field1[1].inner2[0], complexArrayOfStruct[1].field2[0][1] + |from jsonTable + """.stripMargin), ("str2", 6) :: Nil ) } + + test("SPARK-3390 Complex arrays") { + val jsonSchemaRDD = jsonRDD(complexFieldAndType2) + jsonSchemaRDD.registerTempTable("jsonTable") + + checkAnswer( + sql( + """ + |select arrayOfArray1[0][0][0], arrayOfArray1[1][0][1], arrayOfArray1[1][1][0] + |from jsonTable + """.stripMargin), + (5, 7, 8) :: Nil + ) + checkAnswer( + sql( + """ + |select arrayOfArray2[0][0][0].inner1, arrayOfArray2[1][0], + |arrayOfArray2[1][1][1].inner2[0], arrayOfArray2[2][0][0].inner3[0][0].inner4 + |from jsonTable + """.stripMargin), + ("str1", Nil, "str4", 2) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index b3f95f08e8044..5f0b3959a63ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -106,6 +106,34 @@ object TestJsonData { "inner1": "str4" }], "field2": [[5, 6], [7, 8]] - }] + }], + "arrayOfArray1": [ + [ + [5] + ], + [ + [6, 7], + [8] + ]], + "arrayOfArray2": [ + [ + [ + { + "inner1": "str1" + } + ] + ], + [ + [], + [ + {"inner2": ["str3", "str33"]}, + {"inner2": ["str4"], "inner1": "str11"} + ] + ], + [ + [ + {"inner3": [[{"inner4": 2}]]} + ] + ]] }""" :: Nil) } From 6324eb7b5b0ae005cb2e913e36b1508bd6f1b9b8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 11 Sep 2014 17:18:46 -0700 Subject: [PATCH 0663/1492] [Spark-3490] Disable SparkUI for tests We currently open many ephemeral ports during the tests, and as a result we occasionally can't bind to new ones. This has caused the `DriverSuite` and the `SparkSubmitSuite` to fail intermittently. By disabling the `SparkUI` when it's not needed, we already cut down on the number of ports opened significantly, on the order of the number of `SparkContexts` ever created. We must keep it enabled for a few tests for the UI itself, however. Author: Andrew Or Closes #2363 from andrewor14/disable-ui-for-tests and squashes the following commits: 332a7d5 [Andrew Or] No need to set spark.ui.port to 0 anymore 30c93a2 [Andrew Or] Simplify streaming UISuite a431b84 [Andrew Or] Fix streaming test failures 8f5ae53 [Andrew Or] Fix no new line at the end 29c9b5b [Andrew Or] Disable SparkUI for tests --- .../scala/org/apache/spark/SparkContext.scala | 12 +++-- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../cluster/SimrSchedulerBackend.scala | 6 +-- .../cluster/SparkDeploySchedulerBackend.scala | 4 +- .../scala/org/apache/spark/ui/UISuite.scala | 44 +++++++++++++------ pom.xml | 2 +- project/SparkBuild.scala | 2 +- .../spark/streaming/StreamingContext.scala | 11 ++++- .../spark/streaming/StreamingSource.scala | 2 +- .../spark/streaming/ui/StreamingTab.scala | 25 ++++++++--- .../org/apache/spark/streaming/UISuite.scala | 16 +++++-- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../cluster/YarnClientSchedulerBackend.scala | 2 +- 13 files changed, 92 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c6c5b8f22b549..218b353dd9d49 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -220,8 +220,14 @@ class SparkContext(config: SparkConf) extends Logging { new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) // Initialize the Spark UI, registering all associated listeners - private[spark] val ui = new SparkUI(this) - ui.bind() + private[spark] val ui: Option[SparkUI] = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(new SparkUI(this)) + } else { + // For tests, do not enable the UI + None + } + ui.foreach(_.bind()) /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) @@ -990,7 +996,7 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { postApplicationEnd() - ui.stop() + ui.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler 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 5b5257269d92f..9a0cb1c6c6ccd 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 @@ -292,7 +292,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A 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) + scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala index 513d74a08a47f..ee10aa061f4e9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SimrSchedulerBackend.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.spark.{Logging, SparkContext, SparkEnv} @@ -47,16 +46,17 @@ private[spark] class SimrSchedulerBackend( val conf = SparkHadoopUtil.get.newConfiguration(sc.conf) val fs = FileSystem.get(conf) + val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") logInfo("Writing to HDFS file: " + driverFilePath) logInfo("Writing Akka address: " + driverUrl) - logInfo("Writing Spark UI Address: " + sc.ui.appUIAddress) + logInfo("Writing Spark UI Address: " + appUIAddress) // Create temporary file to prevent race condition where executors get empty driverUrl file val temp = fs.create(tmpPath, true) temp.writeUTF(driverUrl) temp.writeInt(maxCores) - temp.writeUTF(sc.ui.appUIAddress) + temp.writeUTF(appUIAddress) temp.close() // "Atomic" rename 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 06872ace2ecf4..2f45d192e1d4d 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 @@ -67,8 +67,10 @@ private[spark] class SparkDeploySchedulerBackend( val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) + val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") + val eventLogDir = sc.eventLogger.map(_.logDir) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) + appUIAddress, eventLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 038746d2eda4b..2f56642956701 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -36,11 +36,25 @@ import scala.xml.Node class UISuite extends FunSuite { + /** + * Create a test SparkContext with the SparkUI enabled. + * It is safe to `get` the SparkUI directly from the SparkContext returned here. + */ + private def newSparkContext(): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val sc = new SparkContext(conf) + assert(sc.ui.isDefined) + sc + } + ignore("basic ui visibility") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark(newSparkContext()) { sc => // test if the ui is visible, and all the expected tabs are visible eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.appUIAddress).mkString + val html = Source.fromURL(sc.ui.get.appUIAddress).mkString assert(!html.contains("random data that should not be present")) assert(html.toLowerCase.contains("stages")) assert(html.toLowerCase.contains("storage")) @@ -51,7 +65,7 @@ class UISuite extends FunSuite { } ignore("visibility at localhost:4040") { - withSpark(new SparkContext("local", "test")) { sc => + withSpark(newSparkContext()) { sc => // test if visible from http://localhost:4040 eventually(timeout(10 seconds), interval(50 milliseconds)) { val html = Source.fromURL("http://localhost:4040").mkString @@ -61,8 +75,8 @@ class UISuite extends FunSuite { } ignore("attaching a new tab") { - withSpark(new SparkContext("local", "test")) { sc => - val sparkUI = sc.ui + withSpark(newSparkContext()) { sc => + val sparkUI = sc.ui.get val newTab = new WebUITab(sparkUI, "foo") { attachPage(new WebUIPage("") { @@ -73,7 +87,7 @@ class UISuite extends FunSuite { } sparkUI.attachTab(newTab) eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.appUIAddress).mkString + val html = Source.fromURL(sparkUI.appUIAddress).mkString assert(!html.contains("random data that should not be present")) // check whether new page exists @@ -87,7 +101,7 @@ class UISuite extends FunSuite { } eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString + val html = Source.fromURL(sparkUI.appUIAddress.stripSuffix("/") + "/foo").mkString // check whether new page exists assert(html.contains("magic")) } @@ -129,16 +143,20 @@ class UISuite extends FunSuite { } test("verify appUIAddress contains the scheme") { - withSpark(new SparkContext("local", "test")) { sc => - val uiAddress = sc.ui.appUIAddress - assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + withSpark(newSparkContext()) { sc => + val ui = sc.ui.get + val uiAddress = ui.appUIAddress + val uiHostPort = ui.appUIHostPort + assert(uiAddress.equals("http://" + uiHostPort)) } } test("verify appUIAddress contains the port") { - withSpark(new SparkContext("local", "test")) { sc => - val splitUIAddress = sc.ui.appUIAddress.split(':') - assert(splitUIAddress(2).toInt == sc.ui.boundPort) + withSpark(newSparkContext()) { sc => + val ui = sc.ui.get + val splitUIAddress = ui.appUIAddress.split(':') + val boundPort = ui.boundPort + assert(splitUIAddress(2).toInt == boundPort) } } } diff --git a/pom.xml b/pom.xml index 64fb1e57e30e0..e5f863e85445c 100644 --- a/pom.xml +++ b/pom.xml @@ -899,7 +899,7 @@ true ${session.executionRootDirectory} 1 - 0 + false diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 45f6d2973ea90..c07ea313f1228 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -337,7 +337,7 @@ object TestSettings { javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.ports.maxRetries=100", - javaOptions in Test += "-Dspark.ui.port=0", + javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") .map { case (k,v) => s"-D$k=$v" }.toSeq, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 457e8ab28ed82..f63560dcb5b89 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -37,7 +37,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver} import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.ui.StreamingTab +import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} import org.apache.spark.util.MetadataCleaner /** @@ -158,7 +158,14 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter - private[streaming] val uiTab = new StreamingTab(this) + private[streaming] val progressListener = new StreamingJobProgressListener(this) + + private[streaming] val uiTab: Option[StreamingTab] = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(new StreamingTab(this)) + } else { + None + } /** Register streaming source to metrics system */ private val streamingSource = new StreamingSource(this) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala index 75f0e8716dc7e..e35a568ddf115 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -26,7 +26,7 @@ private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { override val metricRegistry = new MetricRegistry override val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) - private val streamingListener = ssc.uiTab.listener + private val streamingListener = ssc.progressListener private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, defaultValue: T) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index 34ac254f337eb..d9d04cd706a04 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -17,18 +17,31 @@ package org.apache.spark.streaming.ui -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} import org.apache.spark.streaming.StreamingContext -import org.apache.spark.ui.SparkUITab +import org.apache.spark.ui.{SparkUI, SparkUITab} -/** Spark Web UI tab that shows statistics of a streaming job */ +import StreamingTab._ + +/** + * Spark Web UI tab that shows statistics of a streaming job. + * This assumes the given SparkContext has enabled its SparkUI. + */ private[spark] class StreamingTab(ssc: StreamingContext) - extends SparkUITab(ssc.sc.ui, "streaming") with Logging { + extends SparkUITab(getSparkUI(ssc), "streaming") with Logging { - val parent = ssc.sc.ui - val listener = new StreamingJobProgressListener(ssc) + val parent = getSparkUI(ssc) + val listener = ssc.progressListener ssc.addStreamingListener(listener) attachPage(new StreamingPage(this)) parent.attachTab(this) } + +private object StreamingTab { + def getSparkUI(ssc: StreamingContext): SparkUI = { + ssc.sc.ui.getOrElse { + throw new SparkException("Parent SparkUI to attach this tab to not found!") + } + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 2a0db7564915d..4c7e43c2943c9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -24,13 +24,22 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ +import org.apache.spark.SparkConf + class UISuite extends FunSuite { // Ignored: See SPARK-1530 ignore("streaming tab in spark UI") { - val ssc = new StreamingContext("local", "test", Seconds(1)) + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val ssc = new StreamingContext(conf, Seconds(1)) + assert(ssc.sc.ui.isDefined, "Spark UI is not started!") + val ui = ssc.sc.ui.get + eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL(ssc.sparkContext.ui.appUIAddress).mkString + val html = Source.fromURL(ui.appUIAddress).mkString assert(!html.contains("random data that should not be present")) // test if streaming tab exist assert(html.toLowerCase.contains("streaming")) @@ -39,8 +48,7 @@ class UISuite extends FunSuite { } eventually(timeout(10 seconds), interval(50 milliseconds)) { - val html = Source.fromURL( - ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString + val html = Source.fromURL(ui.appUIAddress.stripSuffix("/") + "/streaming").mkString assert(html.toLowerCase.contains("batch")) assert(html.toLowerCase.contains("network")) } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 5756263e89e21..878b6db546032 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -189,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, if (sc == null) { finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") } else { - registerAM(sc.ui.appUIAddress, securityMgr) + registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) try { userThread.join() } finally { 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 41c662cd7a6de..6aa6475fe4a18 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 @@ -55,7 +55,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) + sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIHostPort) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( From ce59725b8703d18988e495dbaaf86ddde4bdfc5a Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 11 Sep 2014 17:28:36 -0700 Subject: [PATCH 0664/1492] [SPARK-3429] Don't include the empty string "" as a defaultAclUser Changes logging from ``` 14/09/05 02:01:08 INFO SecurityManager: Changing view acls to: aash, 14/09/05 02:01:08 INFO SecurityManager: Changing modify acls to: aash, 14/09/05 02:01:08 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(aash, ); users with modify permissions: Set(aash, ) ``` to ``` 14/09/05 02:28:28 INFO SecurityManager: Changing view acls to: aash 14/09/05 02:28:28 INFO SecurityManager: Changing modify acls to: aash 14/09/05 02:28:28 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(aash); users with modify permissions: Set(aash) ``` Note that the first set of logs have a Set of size 2 containing "aash" and the empty string "" cc tgravescs Author: Andrew Ash Closes #2286 from ash211/empty-default-acl and squashes the following commits: 18cc612 [Andrew Ash] Use .isEmpty instead of =="" cf973a1 [Andrew Ash] Don't include the empty string "" as a defaultAclUser --- core/src/main/scala/org/apache/spark/SecurityManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 12b15fe0815be..3832a780ec4bc 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -162,7 +162,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { // always add the current user and SPARK_USER to the viewAcls private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), - Option(System.getenv("SPARK_USER")).getOrElse("")) + Option(System.getenv("SPARK_USER")).getOrElse("")).filter(!_.isEmpty) setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) From f858f466862541c3faad76a1fa2391f1c17ec9dd Mon Sep 17 00:00:00 2001 From: Cody Koeninger Date: Thu, 11 Sep 2014 17:49:36 -0700 Subject: [PATCH 0665/1492] SPARK-3462 push down filters and projections into Unions Author: Cody Koeninger Closes #2345 from koeninger/SPARK-3462 and squashes the following commits: 5c8d24d [Cody Koeninger] SPARK-3462 remove now-unused parameter 0788691 [Cody Koeninger] SPARK-3462 add tests, handle compatible schema with different aliases, per marmbrus feedback ef47b3b [Cody Koeninger] SPARK-3462 push down filters and projections into Unions --- .../sql/catalyst/optimizer/Optimizer.scala | 48 ++++++++++++++ .../optimizer/UnionPushdownSuite.scala | 62 +++++++++++++++++++ 2 files changed, 110 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala 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 ddd4b3755d629..a4133feae8166 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 @@ -40,12 +40,60 @@ object Optimizer extends RuleExecutor[LogicalPlan] { SimplifyCasts, SimplifyCaseConversionExpressions) :: Batch("Filter Pushdown", FixedPoint(100), + UnionPushdown, CombineFilters, PushPredicateThroughProject, PushPredicateThroughJoin, ColumnPruning) :: Nil } +/** + * Pushes operations to either side of a Union. + */ +object UnionPushdown extends Rule[LogicalPlan] { + + /** + * Maps Attributes from the left side to the corresponding Attribute on the right side. + */ + def buildRewrites(union: Union): AttributeMap[Attribute] = { + assert(union.left.output.size == union.right.output.size) + + AttributeMap(union.left.output.zip(union.right.output)) + } + + /** + * Rewrites an expression so that it can be pushed to the right side of a Union operator. + * This method relies on the fact that the output attributes of a union are always equal + * to the left child's output. + */ + def pushToRight[A <: Expression](e: A, rewrites: AttributeMap[Attribute]): A = { + val result = e transform { + case a: Attribute => rewrites(a) + } + + // We must promise the compiler that we did not discard the names in the case of project + // expressions. This is safe since the only transformation is from Attribute => Attribute. + result.asInstanceOf[A] + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Push down filter into union + case Filter(condition, u @ Union(left, right)) => + val rewrites = buildRewrites(u) + Union( + Filter(condition, left), + Filter(pushToRight(condition, rewrites), right)) + + // Push down projection into union + case Project(projectList, u @ Union(left, right)) => + val rewrites = buildRewrites(u) + Union( + Project(projectList, left), + Project(projectList.map(pushToRight(_, rewrites)), right)) + } +} + + /** * Attempts to eliminate the reading of unneeded columns from the query plan using the following * transformations: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala new file mode 100644 index 0000000000000..dfef87bd9133d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UnionPushdownSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.{PlanTest, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ + +class UnionPushdownSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateAnalysisOperators) :: + Batch("Union Pushdown", Once, + UnionPushdown) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int) + val testUnion = Union(testRelation, testRelation2) + + test("union: filter to each side") { + val query = testUnion.where('a === 1) + + val optimized = Optimize(query.analyze) + + val correctAnswer = + Union(testRelation.where('a === 1), testRelation2.where('d === 1)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("union: project to each side") { + val query = testUnion.select('b) + + val optimized = Optimize(query.analyze) + + val correctAnswer = + Union(testRelation.select('b), testRelation2.select('e)).analyze + + comparePlans(optimized, correctAnswer) + } +} From 33c7a738ae9f2d12425afad6f08a4fe0b7a5c6ab Mon Sep 17 00:00:00 2001 From: witgo Date: Thu, 11 Sep 2014 18:44:35 -0700 Subject: [PATCH 0666/1492] SPARK-2482: Resolve sbt warnings during build At the same time, import the `scala.language.postfixOps` and ` org.scalatest.time.SpanSugar._` cause `scala.language.postfixOps` doesn't work Author: witgo Closes #1330 from witgo/sbt_warnings3 and squashes the following commits: 179ba61 [witgo] Resolve sbt warnings during build --- .../scala/org/apache/spark/bagel/BagelSuite.scala | 2 -- .../scala/org/apache/spark/ContextCleanerSuite.scala | 1 - .../test/scala/org/apache/spark/DriverSuite.scala | 2 -- .../org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 1 - .../src/test/scala/org/apache/spark/ui/UISuite.scala | 1 - pom.xml | 1 - .../scala/org/apache/spark/repl/SparkILoop.scala | 12 ++++++------ .../spark/streaming/NetworkReceiverSuite.scala | 1 - .../spark/streaming/StreamingContextSuite.scala | 2 -- .../spark/streaming/StreamingListenerSuite.scala | 1 - .../scala/org/apache/spark/streaming/UISuite.scala | 1 - 11 files changed, 6 insertions(+), 19 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 55241d33cd3f0..ccb262a4ee02a 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -24,8 +24,6 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.storage.StorageLevel -import scala.language.postfixOps - class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 2744894277ae8..2e3fc5ef0e336 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -21,7 +21,6 @@ import java.lang.ref.WeakReference import scala.collection.mutable.{HashSet, SynchronizedSet} import scala.language.existentials -import scala.language.postfixOps import scala.util.Random import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 4b1d280624c57..5265ba904032f 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -26,8 +26,6 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils -import scala.language.postfixOps - class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 28197657e9bad..3b833f2e41867 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -22,7 +22,6 @@ import java.util.concurrent.Semaphore import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global -import scala.language.postfixOps import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2f56642956701..48790b59e7fbd 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -21,7 +21,6 @@ import java.net.ServerSocket import javax.servlet.http.HttpServletRequest import scala.io.Source -import scala.language.postfixOps import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server diff --git a/pom.xml b/pom.xml index e5f863e85445c..28763476f8313 100644 --- a/pom.xml +++ b/pom.xml @@ -839,7 +839,6 @@ -unchecked -deprecation -feature - -language:postfixOps -Xms1024m diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index d9eeffa86016a..e56b74edba88c 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -15,15 +15,15 @@ import scala.tools.nsc._ import scala.tools.nsc.backend.JavaPlatform import scala.tools.nsc.interpreter._ -import scala.tools.nsc.interpreter.{ Results => IR } -import Predef.{ println => _, _ } -import java.io.{ BufferedReader, FileReader } +import scala.tools.nsc.interpreter.{Results => IR} +import Predef.{println => _, _} +import java.io.{BufferedReader, FileReader} import java.net.URI import java.util.concurrent.locks.ReentrantLock import scala.sys.process.Process import scala.tools.nsc.interpreter.session._ -import scala.util.Properties.{ jdkHome, javaVersion } -import scala.tools.util.{ Javap } +import scala.util.Properties.{jdkHome, javaVersion} +import scala.tools.util.{Javap} import scala.annotation.tailrec import scala.collection.mutable.ListBuffer import scala.concurrent.ops @@ -33,7 +33,7 @@ import scala.tools.nsc.io.{File, Directory} import scala.reflect.NameTransformer._ import scala.tools.nsc.util.ScalaClassLoader._ import scala.tools.util._ -import scala.language.{implicitConversions, existentials} +import scala.language.{implicitConversions, existentials, postfixOps} import scala.reflect.{ClassTag, classTag} import scala.tools.reflect.StdRuntimeTags._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index f4e11f975de94..99c8d13231aac 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.language.postfixOps import org.apache.spark.SparkConf import org.apache.spark.storage.{StorageLevel, StreamBlockId} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 7b33d3b235466..a3cabd6be02fe 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -29,8 +29,6 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ -import scala.language.postfixOps - class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { val master = "local[2]" diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 2861f5335ae36..84fed95a75e67 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global -import scala.language.postfixOps import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 4c7e43c2943c9..8e30118266855 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.streaming import scala.io.Source -import scala.language.postfixOps import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ From 42904b8d013e71d03e301c3da62e33b4cc2eb54e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 11 Sep 2014 18:53:26 -0700 Subject: [PATCH 0667/1492] [SPARK-3465] fix task metrics aggregation in local mode Before overwrite t.taskMetrics, take a deepcopy of it. Author: Davies Liu Closes #2338 from davies/fix_metric and squashes the following commits: a5cdb63 [Davies Liu] Merge branch 'master' into fix_metric 7c879e0 [Davies Liu] add more comments 754b5b8 [Davies Liu] copy taskMetrics only when isLocal is true 5ca26dc [Davies Liu] fix task metrics aggregation in local mode --- .../scala/org/apache/spark/executor/Executor.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index dd903dc65d204..acae448a9c66f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -360,7 +360,16 @@ private[spark] class Executor( if (!taskRunner.attemptedTask.isEmpty) { Option(taskRunner.task).flatMap(_.metrics).foreach { metrics => metrics.updateShuffleReadMetrics - tasksMetrics += ((taskRunner.taskId, metrics)) + if (isLocal) { + // JobProgressListener will hold an reference of it during + // onExecutorMetricsUpdate(), then JobProgressListener can not see + // the changes of metrics any more, so make a deep copy of it + val copiedMetrics = Utils.deserialize[TaskMetrics](Utils.serialize(metrics)) + tasksMetrics += ((taskRunner.taskId, copiedMetrics)) + } else { + // It will be copied by serialization + tasksMetrics += ((taskRunner.taskId, metrics)) + } } } } From b8634df1f1eb6ce909bec779522c9c9912c7d06a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 12 Sep 2014 01:37:59 -0700 Subject: [PATCH 0668/1492] [SPARK-3160] [SPARK-3494] [mllib] DecisionTree: eliminate pre-allocated nodes, parentImpurities arrays. Memory calc bug fix. This PR includes some code simplifications and re-organization which will be helpful for implementing random forests. The main changes are that the nodes and parentImpurities arrays are no longer pre-allocated in the main train() method. Also added 2 bug fixes: * maxMemoryUsage calculation * over-allocation of space for bins in DTStatsAggregator for unordered features. Relation to RFs: * Since RFs will be deeper and will therefore be more likely sparse (not full trees), it could be a cost savings to avoid pre-allocating a full tree. * The associated re-organization also reduces bookkeeping, which will make RFs easier to implement. * The return code doneTraining may be generalized to include cases such as nodes ready for local training. Details: No longer pre-allocate parentImpurities array in main train() method. * parentImpurities values are now stored in individual nodes (in Node.stats.impurity). * These were not really needed. They were used in calculateGainForSplit(), but they can be calculated anyways using parentNodeAgg. No longer using Node.build since tree structure is constructed on-the-fly. * Did not eliminate since it is public (Developer) API. Marked as deprecated. Eliminated pre-allocated nodes array in main train() method. * Nodes are constructed and added to the tree structure as needed during training. * Moved tree construction from main train() method into findBestSplitsPerGroup() since there is no need to keep the (split, gain) array for an entire level of nodes. Only one element of that array is needed at a time, so we do not the array. findBestSplits() now returns 2 items: * rootNode (newly created root node on first iteration, same root node on later iterations) * doneTraining (indicating if all nodes at that level were leafs) Updated DecisionTreeSuite. Notes: * Improved test "Second level node building with vs. without groups" ** generateOrderedLabeledPoints() modified so that it really does require 2 levels of internal nodes. * Related update: Added Node.deepCopy (private[tree]), used for test suite CC: mengxr Author: Joseph K. Bradley Closes #2341 from jkbradley/dt-spark-3160 and squashes the following commits: 07dd1ee [Joseph K. Bradley] Fixed overflow bug with computing maxMemoryUsage in DecisionTree. Also fixed bug with over-allocating space in DTStatsAggregator for unordered features. debe072 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 5c4ac33 [Joseph K. Bradley] Added check in Strategy to make sure minInstancesPerNode >= 1 0dd4d87 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 306120f [Joseph K. Bradley] Fixed typo in DecisionTreeModel.scala doc eaa1dcf [Joseph K. Bradley] Added topNode doc in DecisionTree and scalastyle fix d4d7864 [Joseph K. Bradley] Marked Node.build as deprecated d4dbb99 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 1a8f0ad [Joseph K. Bradley] Eliminated pre-allocated nodes array in main train() method. * Nodes are constructed and added to the tree structure as needed during training. 2ab763b [Joseph K. Bradley] Simplifications to DecisionTree code: --- .../spark/mllib/tree/DecisionTree.scala | 191 +++++------- .../mllib/tree/configuration/Strategy.scala | 3 + .../mllib/tree/impl/DTStatsAggregator.scala | 11 +- .../tree/impl/DecisionTreeMetadata.scala | 3 +- .../mllib/tree/model/DecisionTreeModel.scala | 2 +- .../apache/spark/mllib/tree/model/Node.scala | 37 +++ .../spark/mllib/tree/DecisionTreeSuite.scala | 277 +++++++++--------- 7 files changed, 268 insertions(+), 256 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 98596569b8c95..56bb8812100a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -87,17 +87,11 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo val maxDepth = strategy.maxDepth require(maxDepth <= 30, s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") - // Number of nodes to allocate: max number of nodes possible given the depth of the tree, plus 1 - val maxNumNodesPlus1 = Node.startIndexInLevel(maxDepth + 1) - // Initialize an array to hold parent impurity calculations for each node. - val parentImpurities = new Array[Double](maxNumNodesPlus1) - // dummy value for top node (updated during first split calculation) - val nodes = new Array[Node](maxNumNodesPlus1) // Calculate level for single group construction // Max memory usage for aggregates - val maxMemoryUsage = strategy.maxMemoryInMB * 1024 * 1024 + val maxMemoryUsage = strategy.maxMemoryInMB * 1024L * 1024L logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") // TODO: Calculate memory usage more precisely. val numElementsPerNode = DecisionTree.getElementsPerNode(metadata) @@ -120,81 +114,35 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * beforehand and is not used in later levels. */ + var topNode: Node = null // set on first iteration var level = 0 var break = false while (level <= maxDepth && !break) { - logDebug("#####################################") logDebug("level = " + level) logDebug("#####################################") // Find best split for all nodes at a level. timer.start("findBestSplits") - val splitsStatsForLevel: Array[(Split, InformationGainStats, Predict)] = - DecisionTree.findBestSplits(treeInput, parentImpurities, - metadata, level, nodes, splits, bins, maxLevelForSingleGroup, timer) + val (tmpTopNode: Node, doneTraining: Boolean) = DecisionTree.findBestSplits(treeInput, + metadata, level, topNode, splits, bins, maxLevelForSingleGroup, timer) timer.stop("findBestSplits") - val levelNodeIndexOffset = Node.startIndexInLevel(level) - for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { - val nodeIndex = levelNodeIndexOffset + index - - // Extract info for this node (index) at the current level. - timer.start("extractNodeInfo") - val split = nodeSplitStats._1 - val stats = nodeSplitStats._2 - val predict = nodeSplitStats._3.predict - val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth) - val node = new Node(nodeIndex, predict, isLeaf, Some(split), None, None, Some(stats)) - logDebug("Node = " + node) - nodes(nodeIndex) = node - timer.stop("extractNodeInfo") - - if (level != 0) { - // Set parent. - val parentNodeIndex = Node.parentIndex(nodeIndex) - if (Node.isLeftChild(nodeIndex)) { - nodes(parentNodeIndex).leftNode = Some(nodes(nodeIndex)) - } else { - nodes(parentNodeIndex).rightNode = Some(nodes(nodeIndex)) - } - } - // Extract info for nodes at the next lower level. - timer.start("extractInfoForLowerLevels") - if (level < maxDepth) { - val leftChildIndex = Node.leftChildIndex(nodeIndex) - val leftImpurity = stats.leftImpurity - logDebug("leftChildIndex = " + leftChildIndex + ", impurity = " + leftImpurity) - parentImpurities(leftChildIndex) = leftImpurity - - val rightChildIndex = Node.rightChildIndex(nodeIndex) - val rightImpurity = stats.rightImpurity - logDebug("rightChildIndex = " + rightChildIndex + ", impurity = " + rightImpurity) - parentImpurities(rightChildIndex) = rightImpurity - } - timer.stop("extractInfoForLowerLevels") - logDebug("final best split = " + split) + if (level == 0) { + topNode = tmpTopNode } - require(Node.maxNodesInLevel(level) == splitsStatsForLevel.length) - // Check whether all the nodes at the current level at leaves. - val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) - logDebug("all leaf = " + allLeaf) - if (allLeaf) { - break = true // no more tree construction - } else { - level += 1 + if (doneTraining) { + break = true + logDebug("done training") } + + level += 1 } logDebug("#####################################") logDebug("Extracting tree model") logDebug("#####################################") - // Initialize the top or root node of the tree. - val topNode = nodes(1) - // Build the full tree using the node info calculated in the level-wise best split calculations. - topNode.build(nodes) - timer.stop("total") logInfo("Internal timing for DecisionTree:") @@ -409,24 +357,26 @@ object DecisionTree extends Serializable with Logging { * multiple groups if the level-wise training task could lead to memory overflow. * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] - * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree + * @param topNode Root node of the tree (or invalid node when training first level). * @param splits possible splits for all features, indexed (numFeatures)(numSplits) * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. - * @return array (over nodes) of splits with best split for each node at a given level. + * @return (root, doneTraining) where: + * root = Root node (which is newly created on the first iteration), + * doneTraining = true if no more internal nodes were created. */ private[tree] def findBestSplits( input: RDD[TreePoint], - parentImpurities: Array[Double], metadata: DecisionTreeMetadata, level: Int, - nodes: Array[Node], + topNode: Node, splits: Array[Array[Split]], bins: Array[Array[Bin]], maxLevelForSingleGroup: Int, - timer: TimeTracker = new TimeTracker): Array[(Split, InformationGainStats, Predict)] = { + timer: TimeTracker = new TimeTracker): (Node, Boolean) = { + // split into groups to avoid memory overflow during aggregation if (level > maxLevelForSingleGroup) { // When information for all nodes at a given level cannot be stored in memory, @@ -435,18 +385,18 @@ object DecisionTree extends Serializable with Logging { // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. val numGroups = 1 << level - maxLevelForSingleGroup logDebug("numGroups = " + numGroups) - var bestSplits = new Array[(Split, InformationGainStats, Predict)](0) // Iterate over each group of nodes at a level. var groupIndex = 0 + var doneTraining = true while (groupIndex < numGroups) { - val bestSplitsForGroup = findBestSplitsPerGroup(input, parentImpurities, metadata, level, - nodes, splits, bins, timer, numGroups, groupIndex) - bestSplits = Array.concat(bestSplits, bestSplitsForGroup) + val (tmpRoot, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, + topNode, splits, bins, timer, numGroups, groupIndex) + doneTraining = doneTraining && doneTrainingGroup groupIndex += 1 } - bestSplits + (topNode, doneTraining) // Not first iteration, so topNode was already set. } else { - findBestSplitsPerGroup(input, parentImpurities, metadata, level, nodes, splits, bins, timer) + findBestSplitsPerGroup(input, metadata, level, topNode, splits, bins, timer) } } @@ -586,27 +536,27 @@ object DecisionTree extends Serializable with Logging { * Returns an array of optimal splits for a group of nodes at a given level * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] - * @param parentImpurities Impurities for all parent nodes for the current level * @param metadata Learning and dataset metadata * @param level Level of the tree - * @param nodes Array of all nodes in the tree. Used for matching data points to nodes. + * @param topNode Root node of the tree (or invalid node when training first level). * @param splits possible splits for all features, indexed (numFeatures)(numSplits) * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param numGroups total number of node groups at the current level. Default value is set to 1. * @param groupIndex index of the node group being processed. Default value is set to 0. - * @return array of splits with best splits for all nodes at a given level. + * @return (root, doneTraining) where: + * root = Root node (which is newly created on the first iteration), + * doneTraining = true if no more internal nodes were created. */ private def findBestSplitsPerGroup( input: RDD[TreePoint], - parentImpurities: Array[Double], metadata: DecisionTreeMetadata, level: Int, - nodes: Array[Node], + topNode: Node, splits: Array[Array[Split]], bins: Array[Array[Bin]], timer: TimeTracker, numGroups: Int = 1, - groupIndex: Int = 0): Array[(Split, InformationGainStats, Predict)] = { + groupIndex: Int = 0): (Node, Boolean) = { /* * The high-level descriptions of the best split optimizations are noted here. @@ -663,7 +613,7 @@ object DecisionTree extends Serializable with Logging { 0 } else { val globalNodeIndex = - predictNodeIndex(nodes(1), treePoint.binnedFeatures, bins, metadata.unorderedFeatures) + predictNodeIndex(topNode, treePoint.binnedFeatures, bins, metadata.unorderedFeatures) globalNodeIndex - globalNodeIndexOffset } } @@ -706,33 +656,63 @@ object DecisionTree extends Serializable with Logging { // Calculate best splits for all nodes at a given level timer.start("chooseSplits") - val bestSplits = new Array[(Split, InformationGainStats, Predict)](numNodes) - // Iterating over all nodes at this level + // On the first iteration, we need to get and return the newly created root node. + var newTopNode: Node = topNode + + // Iterate over all nodes at this level var nodeIndex = 0 + var internalNodeCount = 0 while (nodeIndex < numNodes) { - val nodeImpurity = parentImpurities(globalNodeIndexOffset + nodeIndex) - logDebug("node impurity = " + nodeImpurity) - bestSplits(nodeIndex) = - binsToBestSplit(binAggregates, nodeIndex, nodeImpurity, level, metadata, splits) - logDebug("best split = " + bestSplits(nodeIndex)._1) + val (split: Split, stats: InformationGainStats, predict: Predict) = + binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits) + logDebug("best split = " + split) + + val globalNodeIndex = globalNodeIndexOffset + nodeIndex + + // Extract info for this node at the current level. + val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth) + val node = + new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats)) + logDebug("Node = " + node) + + if (!isLeaf) { + internalNodeCount += 1 + } + if (level == 0) { + newTopNode = node + } else { + // Set parent. + val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode) + if (Node.isLeftChild(globalNodeIndex)) { + parentNode.leftNode = Some(node) + } else { + parentNode.rightNode = Some(node) + } + } + if (level < metadata.maxDepth) { + logDebug("leftChildIndex = " + Node.leftChildIndex(globalNodeIndex) + + ", impurity = " + stats.leftImpurity) + logDebug("rightChildIndex = " + Node.rightChildIndex(globalNodeIndex) + + ", impurity = " + stats.rightImpurity) + } + nodeIndex += 1 } timer.stop("chooseSplits") - bestSplits + val doneTraining = internalNodeCount == 0 + (newTopNode, doneTraining) } /** * Calculate the information gain for a given (feature, split) based upon left/right aggregates. * @param leftImpurityCalculator left node aggregates for this (feature, split) * @param rightImpurityCalculator right node aggregate for this (feature, split) - * @param topImpurity impurity of the parent node * @return information gain and statistics for all splits */ private def calculateGainForSplit( leftImpurityCalculator: ImpurityCalculator, rightImpurityCalculator: ImpurityCalculator, - topImpurity: Double, level: Int, metadata: DecisionTreeMetadata): InformationGainStats = { val leftCount = leftImpurityCalculator.count @@ -747,14 +727,10 @@ object DecisionTree extends Serializable with Logging { val totalCount = leftCount + rightCount - // impurity of parent node - val impurity = if (level > 0) { - topImpurity - } else { - val parentNodeAgg = leftImpurityCalculator.copy - parentNodeAgg.add(rightImpurityCalculator) - parentNodeAgg.calculate() - } + val parentNodeAgg = leftImpurityCalculator.copy + parentNodeAgg.add(rightImpurityCalculator) + + val impurity = parentNodeAgg.calculate() val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 val rightImpurity = rightImpurityCalculator.calculate() @@ -795,19 +771,15 @@ object DecisionTree extends Serializable with Logging { * Find the best split for a node. * @param binAggregates Bin statistics. * @param nodeIndex Index for node to split in this (level, group). - * @param nodeImpurity Impurity of the node (nodeIndex). * @return tuple for best split: (Split, information gain) */ private def binsToBestSplit( binAggregates: DTStatsAggregator, nodeIndex: Int, - nodeImpurity: Double, level: Int, metadata: DecisionTreeMetadata, splits: Array[Array[Split]]): (Split, InformationGainStats, Predict) = { - logDebug("node impurity = " + nodeImpurity) - // calculate predict only once var predict: Option[Predict] = None @@ -831,8 +803,7 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = - calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) (splitIdx, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) @@ -845,8 +816,7 @@ object DecisionTree extends Serializable with Logging { val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = - calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) @@ -917,8 +887,7 @@ object DecisionTree extends Serializable with Logging { binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = - calculateGainForSplit(leftChildStats, rightChildStats, nodeImpurity, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) val categoriesForSplit = @@ -937,8 +906,8 @@ object DecisionTree extends Serializable with Logging { /** * Get the number of values to be stored per node in the bin aggregates. */ - private def getElementsPerNode(metadata: DecisionTreeMetadata): Int = { - val totalBins = metadata.numBins.sum + private def getElementsPerNode(metadata: DecisionTreeMetadata): Long = { + val totalBins = metadata.numBins.map(_.toLong).sum if (metadata.isClassification) { metadata.numClasses * totalBins } else { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 987fe632c91ed..31d1e8ac30eea 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -75,6 +75,9 @@ class Strategy ( if (algo == Classification) { require(numClassesForClassification >= 2) } + require(minInstancesPerNode >= 1, + s"DecisionTree Strategy requires minInstancesPerNode >= 1 but was given $minInstancesPerNode") + val isMulticlassClassification = algo == Classification && numClassesForClassification > 2 val isMulticlassWithCategoricalFeatures diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index 866d85a79bea1..61a94246711bf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -65,14 +65,7 @@ private[tree] class DTStatsAggregator( * Offset for each feature for calculating indices into the [[allStats]] array. */ private val featureOffsets: Array[Int] = { - def featureOffsetsCalc(total: Int, featureIndex: Int): Int = { - if (isUnordered(featureIndex)) { - total + 2 * numBins(featureIndex) - } else { - total + numBins(featureIndex) - } - } - Range(0, numFeatures).scanLeft(0)(featureOffsetsCalc).map(statsSize * _).toArray + numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) } /** @@ -149,7 +142,7 @@ private[tree] class DTStatsAggregator( s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + s" but was called for ordered feature $featureIndex.") val baseOffset = nodeIndex * nodeStride + featureOffsets(featureIndex) - (baseOffset, baseOffset + numBins(featureIndex) * statsSize) + (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index 5ceaa8154d11a..b6d49e5555b1a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -46,6 +46,7 @@ private[tree] class DecisionTreeMetadata( val numBins: Array[Int], val impurity: Impurity, val quantileStrategy: QuantileStrategy, + val maxDepth: Int, val minInstancesPerNode: Int, val minInfoGain: Double) extends Serializable { @@ -129,7 +130,7 @@ private[tree] object DecisionTreeMetadata { new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, - strategy.impurity, strategy.quantileCalculationStrategy, + strategy.impurity, strategy.quantileCalculationStrategy, strategy.maxDepth, strategy.minInstancesPerNode, strategy.minInfoGain) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 0594fd0749d21..271b2c4ad813e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -46,7 +46,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * Predict values for the given data set using the model trained. * * @param features RDD representing data points to be predicted - * @return RDD[Int] where each entry contains the corresponding prediction + * @return RDD of predictions for each of the given data points */ def predict(features: RDD[Vector]): RDD[Double] = { features.map(x => predict(x)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 5b8a4cbed2306..5f0095d23c7ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -55,6 +55,8 @@ class Node ( * build the left node and right nodes if not leaf * @param nodes array of nodes */ + @deprecated("build should no longer be used since trees are constructed on-the-fly in training", + "1.2.0") def build(nodes: Array[Node]): Unit = { logDebug("building node " + id + " at level " + Node.indexToLevel(id)) logDebug("id = " + id + ", split = " + split) @@ -93,6 +95,23 @@ class Node ( } } + /** + * Returns a deep copy of the subtree rooted at this node. + */ + private[tree] def deepCopy(): Node = { + val leftNodeCopy = if (leftNode.isEmpty) { + None + } else { + Some(leftNode.get.deepCopy()) + } + val rightNodeCopy = if (rightNode.isEmpty) { + None + } else { + Some(rightNode.get.deepCopy()) + } + new Node(id, predict, isLeaf, split, leftNodeCopy, rightNodeCopy, stats) + } + /** * Get the number of nodes in tree below this node, including leaf nodes. * E.g., if this is a leaf, returns 0. If both children are leaves, returns 2. @@ -190,4 +209,22 @@ private[tree] object Node { */ def startIndexInLevel(level: Int): Int = 1 << level + /** + * Traces down from a root node to get the node with the given node index. + * This assumes the node exists. + */ + def getNode(nodeIndex: Int, rootNode: Node): Node = { + var tmpNode: Node = rootNode + var levelsToGo = indexToLevel(nodeIndex) + while (levelsToGo > 0) { + if ((nodeIndex & (1 << levelsToGo - 1)) == 0) { + tmpNode = tmpNode.leftNode.get + } else { + tmpNode = tmpNode.rightNode.get + } + levelsToGo -= 1 + } + tmpNode + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index fd8547c1660fc..1bd7ea05c46c8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -270,19 +270,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 0) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode: Node, doneTraining: Boolean) = + DecisionTree.findBestSplits(treeInput, metadata, 0, null, splits, bins, 10) - val split = bestSplits(0)._1 + val split = rootNode.split.get assert(split.categories === List(1.0)) assert(split.featureType === Categorical) assert(split.threshold === Double.MinValue) - val stats = bestSplits(0)._2 - val predict = bestSplits(0)._3 + val stats = rootNode.stats.get assert(stats.gain > 0) - assert(predict.predict === 1) - assert(predict.prob === 0.6) + assert(rootNode.predict === 1) assert(stats.impurity > 0.2) } @@ -303,19 +301,18 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - val split = bestSplits(0)._1 + val split = rootNode.split.get assert(split.categories.length === 1) assert(split.categories.contains(1.0)) assert(split.featureType === Categorical) assert(split.threshold === Double.MinValue) - val stats = bestSplits(0)._2 - val predict = bestSplits(0)._3.predict + val stats = rootNode.stats.get assert(stats.gain > 0) - assert(predict === 0.6) + assert(rootNode.predict === 0.6) assert(stats.impurity > 0.2) } @@ -356,13 +353,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) } test("Binary classification stump with fixed label 1 for Gini") { @@ -382,14 +382,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._3.predict === 1) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) + assert(rootNode.predict === 1) } test("Binary classification stump with fixed label 0 for Entropy") { @@ -409,14 +412,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._3.predict === 0) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) + assert(rootNode.predict === 0) } test("Binary classification stump with fixed label 1 for Entropy") { @@ -436,14 +442,17 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(2), metadata, 0, - new Array[Node](0), splits, bins, 10) - assert(bestSplits.length === 1) - assert(bestSplits(0)._1.feature === 0) - assert(bestSplits(0)._2.gain === 0) - assert(bestSplits(0)._2.leftImpurity === 0) - assert(bestSplits(0)._2.rightImpurity === 0) - assert(bestSplits(0)._3.predict === 1) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + + val stats = rootNode.stats.get + assert(stats.gain === 0) + assert(stats.leftImpurity === 0) + assert(stats.rightImpurity === 0) + assert(rootNode.predict === 1) } test("Second level node building with vs. without groups") { @@ -459,40 +468,46 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins(0).length === 100) // Train a 1-node model - val strategyOneNode = new Strategy(Classification, Entropy, 1, 2, 100) + val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, + numClassesForClassification = 2, maxBins = 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val nodes: Array[Node] = new Array[Node](8) - nodes(1) = modelOneNode.topNode - nodes(1).leftNode = None - nodes(1).rightNode = None - - val parentImpurities = Array(0, 0.5, 0.5, 0.5) + val rootNodeCopy1 = modelOneNode.topNode.deepCopy() + val rootNodeCopy2 = modelOneNode.topNode.deepCopy() // Single group second level tree construction. val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, nodes, - splits, bins, 10) - assert(bestSplits.length === 2) - assert(bestSplits(0)._2.gain > 0) - assert(bestSplits(1)._2.gain > 0) + val (rootNode, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, + rootNodeCopy1, splits, bins, 10) + assert(rootNode.leftNode.nonEmpty) + assert(rootNode.rightNode.nonEmpty) + val children1 = new Array[Node](2) + children1(0) = rootNode.leftNode.get + children1(1) = rootNode.rightNode.get // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second // level tree construction. - val bestSplitsWithGroups = DecisionTree.findBestSplits(treeInput, parentImpurities, metadata, 1, - nodes, splits, bins, 0) - assert(bestSplitsWithGroups.length === 2) - assert(bestSplitsWithGroups(0)._2.gain > 0) - assert(bestSplitsWithGroups(1)._2.gain > 0) + val (rootNode2, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, + rootNodeCopy2, splits, bins, 0) + assert(rootNode2.leftNode.nonEmpty) + assert(rootNode2.rightNode.nonEmpty) + val children2 = new Array[Node](2) + children2(0) = rootNode2.leftNode.get + children2(1) = rootNode2.rightNode.get // Verify whether the splits obtained using single group and multiple group level // construction strategies are the same. - for (i <- 0 until bestSplits.length) { - assert(bestSplits(i)._1 === bestSplitsWithGroups(i)._1) - assert(bestSplits(i)._2.gain === bestSplitsWithGroups(i)._2.gain) - assert(bestSplits(i)._2.impurity === bestSplitsWithGroups(i)._2.impurity) - assert(bestSplits(i)._2.leftImpurity === bestSplitsWithGroups(i)._2.leftImpurity) - assert(bestSplits(i)._2.rightImpurity === bestSplitsWithGroups(i)._2.rightImpurity) - assert(bestSplits(i)._3.predict === bestSplitsWithGroups(i)._3.predict) + for (i <- 0 until 2) { + assert(children1(i).stats.nonEmpty && children1(i).stats.get.gain > 0) + assert(children2(i).stats.nonEmpty && children2(i).stats.get.gain > 0) + assert(children1(i).split === children2(i).split) + assert(children1(i).stats.nonEmpty && children2(i).stats.nonEmpty) + val stats1 = children1(i).stats.get + val stats2 = children2(i).stats.get + assert(stats1.gain === stats2.gain) + assert(stats1.impurity === stats2.impurity) + assert(stats1.leftImpurity === stats2.leftImpurity) + assert(stats1.rightImpurity === stats2.rightImpurity) + assert(children1(i).predict === children2(i).predict) } } @@ -508,15 +523,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - assert(bestSplit.feature === 0) - assert(bestSplit.categories.length === 1) - assert(bestSplit.categories.contains(1)) - assert(bestSplit.featureType === Categorical) + val split = rootNode.split.get + assert(split.feature === 0) + assert(split.categories.length === 1) + assert(split.categories.contains(1)) + assert(split.featureType === Categorical) } test("Binary classification stump with 1 continuous feature, to check off-by-1 error") { @@ -573,16 +587,16 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) - - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - assert(bestSplit.feature === 0) - assert(bestSplit.categories.length === 1) - assert(bestSplit.categories.contains(1)) - assert(bestSplit.featureType === Categorical) - val gain = bestSplits(0)._2 + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) + + val split = rootNode.split.get + assert(split.feature === 0) + assert(split.categories.length === 1) + assert(split.categories.contains(1)) + assert(split.featureType === Categorical) + + val gain = rootNode.stats.get assert(gain.leftImpurity === 0) assert(gain.rightImpurity === 0) } @@ -600,16 +614,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) - - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplit.feature === 1) - assert(bestSplit.featureType === Continuous) - assert(bestSplit.threshold > 1980) - assert(bestSplit.threshold < 2020) + val split = rootNode.split.get + assert(split.feature === 1) + assert(split.featureType === Continuous) + assert(split.threshold > 1980) + assert(split.threshold < 2020) } @@ -627,16 +639,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - - assert(bestSplit.feature === 1) - assert(bestSplit.featureType === Continuous) - assert(bestSplit.threshold > 1980) - assert(bestSplit.threshold < 2020) + val split = rootNode.split.get + assert(split.feature === 1) + assert(split.featureType === Continuous) + assert(split.threshold > 1980) + assert(split.threshold < 2020) } test("Multiclass classification stump with 10-ary (ordered) categorical features") { @@ -652,15 +662,14 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(32), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length === 1) - val bestSplit = bestSplits(0)._1 - assert(bestSplit.feature === 0) - assert(bestSplit.categories.length === 1) - assert(bestSplit.categories.contains(1.0)) - assert(bestSplit.featureType === Categorical) + val split = rootNode.split.get + assert(split.feature === 0) + assert(split.categories.length === 1) + assert(split.categories.contains(1.0)) + assert(split.featureType === Categorical) } test("Multiclass classification tree with 10-ary (ordered) categorical features," + @@ -698,12 +707,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length == 1) - val bestInfoStats = bestSplits(0)._2 - assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + val gain = rootNode.stats.get + assert(gain == InformationGainStats.invalidInformationGainStats) } test("don't choose split that doesn't satisfy min instance per node requirements") { @@ -722,14 +730,13 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length == 1) - val bestSplit = bestSplits(0)._1 - val bestSplitStats = bestSplits(0)._1 - assert(bestSplit.feature == 1) - assert(bestSplitStats != InformationGainStats.invalidInformationGainStats) + val split = rootNode.split.get + val gain = rootNode.stats.get + assert(split.feature == 1) + assert(gain != InformationGainStats.invalidInformationGainStats) } test("split must satisfy min info gain requirements") { @@ -754,12 +761,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val bestSplits = DecisionTree.findBestSplits(treeInput, new Array(8), metadata, 0, - new Array[Node](0), splits, bins, 10) + val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, + null, splits, bins, 10) - assert(bestSplits.length == 1) - val bestInfoStats = bestSplits(0)._2 - assert(bestInfoStats == InformationGainStats.invalidInformationGainStats) + val gain = rootNode.stats.get + assert(gain == InformationGainStats.invalidInformationGainStats) } } @@ -786,13 +792,16 @@ object DecisionTreeSuite { def generateOrderedLabeledPoints(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000) { - if (i < 600) { - val lp = new LabeledPoint(0.0, Vectors.dense(i.toDouble, 1000.0 - i)) - arr(i) = lp + val label = if (i < 100) { + 0.0 + } else if (i < 500) { + 1.0 + } else if (i < 900) { + 0.0 } else { - val lp = new LabeledPoint(1.0, Vectors.dense(i.toDouble, 1000.0 - i)) - arr(i) = lp + 1.0 } + arr(i) = new LabeledPoint(label, Vectors.dense(i.toDouble, 1000.0 - i)) } arr } From f116f76bf1f1610905ca094c8edc53151a78d2f4 Mon Sep 17 00:00:00 2001 From: "Mark G. Whitney" Date: Fri, 12 Sep 2014 08:08:58 -0500 Subject: [PATCH 0669/1492] [SPARK-2558][DOCS] Add --queue example to YARN doc Put original YARN queue spark-submit arg description in running-on-yarn html table and example command line Author: Mark G. Whitney Closes #2218 from kramimus/2258-yarndoc and squashes the following commits: 4b5d808 [Mark G. Whitney] remove yarn queue config f8cda0d [Mark G. Whitney] [SPARK-2558][DOCS] Add spark.yarn.queue description to YARN doc --- docs/running-on-yarn.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d8b22f3663d08..212248bcce1c1 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -155,6 +155,7 @@ For example: --driver-memory 4g \ --executor-memory 2g \ --executor-cores 1 \ + --queue thequeue \ lib/spark-examples*.jar \ 10 From 533377621f1e178e18fa0b79d653a11b66e4e250 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Fri, 12 Sep 2014 09:46:21 -0700 Subject: [PATCH 0670/1492] [PySpark] Add blank line so that Python RDD.top() docstring renders correctly Author: RJ Nowling Closes #2370 from rnowling/python_rdd_docstrings and squashes the following commits: 5230574 [RJ Nowling] Add blank line so that Python RDD.top() docstring renders correctly --- python/pyspark/rdd.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 5667154cb84a8..6ad5ab2a2d1ae 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1060,6 +1060,7 @@ def top(self, num, key=None): Get the top N elements from a RDD. Note: It returns the list sorted in descending order. + >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) [12] >>> sc.parallelize([2, 3, 4, 5, 6], 2).top(2) From 8194fc662c08eb445444c207264e22361def54ea Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 12 Sep 2014 11:29:30 -0700 Subject: [PATCH 0671/1492] [SPARK-3481] [SQL] Eliminate the error log in local Hive comparison test Logically, we should remove the Hive Table/Database first and then reset the Hive configuration, repoint to the new data warehouse directory etc. Otherwise it raised exceptions like "Database doesn't not exists: default" in the local testing. Author: Cheng Hao Closes #2352 from chenghao-intel/test_hive and squashes the following commits: 74fd76b [Cheng Hao] eliminate the error log --- .../org/apache/spark/sql/hive/TestHive.scala | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 6974f3e581b97..a3bfd3a8f1fd2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -376,15 +376,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } - // It is important that we RESET first as broken hooks that might have been set could break - // other sql exec here. - runSqlHive("RESET") - // For some reason, RESET does not reset the following variables... - runSqlHive("set datanucleus.cache.collections=true") - runSqlHive("set datanucleus.cache.collections.lazy=true") - // Lots of tests fail if we do not change the partition whitelist from the default. - runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") - loadedTables.clear() catalog.client.getAllTables("default").foreach { t => logDebug(s"Deleting table $t") @@ -410,6 +401,14 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { FunctionRegistry.unregisterTemporaryUDF(udfName) } + // It is important that we RESET first as broken hooks that might have been set could break + // other sql exec here. + runSqlHive("RESET") + // For some reason, RESET does not reset the following variables... + runSqlHive("set datanucleus.cache.collections=true") + runSqlHive("set datanucleus.cache.collections.lazy=true") + // Lots of tests fail if we do not change the partition whitelist from the default. + runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") configure() runSqlHive("USE default") From eae81b0bfdf3159be90f507a03853800aec1874a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 12 Sep 2014 13:43:29 -0700 Subject: [PATCH 0672/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #930 (close requested by 'andrewor14') Closes #867 (close requested by 'marmbrus') Closes #1829 (close requested by 'marmbrus') Closes #1131 (close requested by 'JoshRosen') Closes #1571 (close requested by 'andrewor14') Closes #2359 (close requested by 'andrewor14') From 15a564598fe63003652b1e24527c432080b5976c Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Fri, 12 Sep 2014 14:08:38 -0700 Subject: [PATCH 0673/1492] [SPARK-3427] [GraphX] Avoid active vertex tracking in static PageRank GraphX's current implementation of static (fixed iteration count) PageRank uses the Pregel API. This unnecessarily tracks active vertices, even though in static PageRank all vertices are always active. Active vertex tracking incurs the following costs: 1. A shuffle per iteration to ship the active sets to the edge partitions. 2. A hash table creation per iteration at each partition to index the active sets for lookup. 3. A hash lookup per edge to check whether the source vertex is active. I reimplemented static PageRank using the lower-level GraphX API instead of the Pregel API. In benchmarks on a 16-node m2.4xlarge cluster, this provided a 23% speedup (from 514 s to 397 s, mean over 3 trials) for 10 iterations of PageRank on a synthetic graph with 10M vertices and 1.27B edges. Author: Ankur Dave Closes #2308 from ankurdave/SPARK-3427 and squashes the following commits: 449996a [Ankur Dave] Avoid unnecessary active vertex tracking in static PageRank --- .../apache/spark/graphx/lib/PageRank.scala | 45 ++++++++++++------- 1 file changed, 29 insertions(+), 16 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 614555a054dfb..257e2f3a36115 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -79,30 +79,43 @@ object PageRank extends Logging { def run[VD: ClassTag, ED: ClassTag]( graph: Graph[VD, ED], numIter: Int, resetProb: Double = 0.15): Graph[Double, Double] = { - // Initialize the pagerankGraph with each edge attribute having + // Initialize the PageRank graph with each edge attribute having // weight 1/outDegree and each vertex with attribute 1.0. - val pagerankGraph: Graph[Double, Double] = graph + var rankGraph: Graph[Double, Double] = graph // Associate the degree with each vertex .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } // Set the weight on the edges based on the degree .mapTriplets( e => 1.0 / e.srcAttr ) // Set the vertex attributes to the initial pagerank values - .mapVertices( (id, attr) => 1.0 ) - .cache() + .mapVertices( (id, attr) => resetProb ) - // Define the three functions needed to implement PageRank in the GraphX - // version of Pregel - def vertexProgram(id: VertexId, attr: Double, msgSum: Double): Double = - resetProb + (1.0 - resetProb) * msgSum - def sendMessage(edge: EdgeTriplet[Double, Double]) = - Iterator((edge.dstId, edge.srcAttr * edge.attr)) - def messageCombiner(a: Double, b: Double): Double = a + b - // The initial message received by all vertices in PageRank - val initialMessage = 0.0 + var iteration = 0 + var prevRankGraph: Graph[Double, Double] = null + while (iteration < numIter) { + rankGraph.cache() - // Execute pregel for a fixed number of iterations. - Pregel(pagerankGraph, initialMessage, numIter, activeDirection = EdgeDirection.Out)( - vertexProgram, sendMessage, messageCombiner) + // Compute the outgoing rank contributions of each vertex, perform local preaggregation, and + // do the final aggregation at the receiving vertices. Requires a shuffle for aggregation. + val rankUpdates = rankGraph.mapReduceTriplets[Double]( + e => Iterator((e.dstId, e.srcAttr * e.attr)), _ + _) + + // Apply the final rank updates to get the new ranks, using join to preserve ranks of vertices + // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the + // edge partitions. + prevRankGraph = rankGraph + rankGraph = rankGraph.joinVertices(rankUpdates) { + (id, oldRank, msgSum) => resetProb + (1.0 - resetProb) * msgSum + }.cache() + + rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices + logInfo(s"PageRank finished iteration $iteration.") + prevRankGraph.vertices.unpersist(false) + prevRankGraph.edges.unpersist(false) + + iteration += 1 + } + + rankGraph } /** From 1d767967e925f1d727957c2d43383ef6ad2c5d5e Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 12 Sep 2014 16:48:28 -0500 Subject: [PATCH 0674/1492] SPARK-3014. Log a more informative messages in a couple failure scenario... ...s Author: Sandy Ryza Closes #1934 from sryza/sandy-spark-3014 and squashes the following commits: ae19cc1 [Sandy Ryza] SPARK-3014. Log a more informative messages in a couple failure scenarios --- .../main/scala/org/apache/spark/deploy/SparkSubmit.scala | 6 ++++-- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 ++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 0fdb5ae3c2e40..5ed3575816a38 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy import java.io.{File, PrintStream} -import java.lang.reflect.InvocationTargetException +import java.lang.reflect.{Modifier, InvocationTargetException} import java.net.URL import scala.collection.mutable.{ArrayBuffer, HashMap, Map} @@ -323,7 +323,9 @@ object SparkSubmit { } val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) - + if (!Modifier.isStatic(mainMethod.getModifiers)) { + throw new IllegalStateException("The main method in the given main class must be static") + } try { mainMethod.invoke(null, childArgs.toArray) } catch { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 878b6db546032..735d7723b0ce6 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -283,11 +283,9 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } val sparkContext = sparkContextRef.get() - assert(sparkContext != null || count >= numTries) if (sparkContext == null) { - logError( - "Unable to retrieve sparkContext inspite of waiting for %d, numTries = %d".format( - count * waitTime, numTries)) + logError(("SparkContext did not initialize after waiting for %d ms. Please check earlier" + + " log output for errors. Failing the application.").format(numTries * waitTime)) } sparkContext } From af2583826c15d2a4e2732017ea20feeff0fb79f6 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 12 Sep 2014 14:54:42 -0700 Subject: [PATCH 0675/1492] [SPARK-3217] Add Guava to classpath when SPARK_PREPEND_CLASSES is set. When that option is used, the compiled classes from the build directory are prepended to the classpath. Now that we avoid packaging Guava, that means we have classes referencing the original Guava location in the app's classpath, so errors happen. For that case, add Guava manually to the classpath. Note: if Spark is compiled with "-Phadoop-provided", it's tricky to make things work with SPARK_PREPEND_CLASSES, because you need to add the Hadoop classpath using SPARK_CLASSPATH and that means the older Hadoop Guava overrides the newer one Spark needs. So someone using SPARK_PREPEND_CLASSES needs to remember to not use that profile. Author: Marcelo Vanzin Closes #2141 from vanzin/SPARK-3217 and squashes the following commits: b967324 [Marcelo Vanzin] [SPARK-3217] Add Guava to classpath when SPARK_PREPEND_CLASSES is set. --- bin/compute-classpath.sh | 1 + core/pom.xml | 27 +++++++++++++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 15c6779402994..0f63e36d8aeca 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -43,6 +43,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\ "classes ahead of assembly." >&2 CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes" + CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*" CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes" diff --git a/core/pom.xml b/core/pom.xml index b2b788a4bc13b..2a81f6df289c0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -351,6 +351,33 @@ + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + ${project.build.directory} + false + false + true + true + guava + true + + + + From 25311c2c545a60eb9dcf704814d4600987852155 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 12 Sep 2014 20:31:11 -0500 Subject: [PATCH 0676/1492] [SPARK-3456] YarnAllocator on alpha can lose container requests to RM Author: Thomas Graves Closes #2373 from tgravescs/SPARK-3456 and squashes the following commits: 77e9532 [Thomas Graves] [SPARK-3456] YarnAllocator on alpha can lose container requests to RM --- .../spark/deploy/yarn/YarnAllocationHandler.scala | 11 ++++++----- .../org/apache/spark/deploy/yarn/YarnAllocator.scala | 8 ++++++-- .../spark/deploy/yarn/YarnAllocationHandler.scala | 3 ++- 3 files changed, 14 insertions(+), 8 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 5a1b42c1e17d5..6c93d8582330b 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -48,16 +48,17 @@ private[yarn] class YarnAllocationHandler( private val lastResponseId = new AtomicInteger() private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() - override protected def allocateContainers(count: Int): YarnAllocateResponse = { + override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { var resourceRequests: List[ResourceRequest] = null - logDebug("numExecutors: " + count) + logDebug("asking for additional executors: " + count + " with already pending: " + pending) + val totalNumAsk = count + pending if (count <= 0) { resourceRequests = List() } else if (preferredHostToCount.isEmpty) { logDebug("host preferences is empty") resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, count, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) + AllocationType.ANY, null, totalNumAsk, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) } else { // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. @@ -80,7 +81,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests: ResourceRequest = createResourceRequest( AllocationType.ANY, resource = null, - count, + totalNumAsk, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( @@ -103,7 +104,7 @@ private[yarn] class YarnAllocationHandler( req.addAllReleases(releasedContainerList) if (count > 0) { - logInfo("Allocating %d executor containers with %d of memory each.".format(count, + logInfo("Allocating %d executor containers with %d of memory each.".format(totalNumAsk, executorMemory + memoryOverhead)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 0b8744f4b8bdf..299e38a5eb9c0 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -112,6 +112,9 @@ private[yarn] abstract class YarnAllocator( def allocateResources() = { val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() + // this is needed by alpha, do it here since we add numPending right after this + val executorsPending = numPendingAllocate.get() + if (missing > 0) { numPendingAllocate.addAndGet(missing) logInfo("Will Allocate %d executor containers, each with %d memory".format( @@ -121,7 +124,7 @@ private[yarn] abstract class YarnAllocator( logDebug("Empty allocation request ...") } - val allocateResponse = allocateContainers(missing) + val allocateResponse = allocateContainers(missing, executorsPending) val allocatedContainers = allocateResponse.getAllocatedContainers() if (allocatedContainers.size > 0) { @@ -435,9 +438,10 @@ private[yarn] abstract class YarnAllocator( * * @param count Number of containers to allocate. * If zero, should still contact RM (as a heartbeat). + * @param pending Number of containers pending allocate. Only used on alpha. * @return Response to the allocation request. */ - protected def allocateContainers(count: Int): YarnAllocateResponse + protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse /** Called to release a previously allocated container. */ protected def releaseContainer(container: Container): Unit diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 5438f151ac0ad..e44a8db41b97e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -47,7 +47,8 @@ private[yarn] class YarnAllocationHandler( amClient.releaseAssignedContainer(container.getId()) } - override protected def allocateContainers(count: Int): YarnAllocateResponse = { + // pending isn't used on stable as the AMRMClient handles incremental asks + override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { addResourceRequests(count) // We have already set the container request. Poll the ResourceManager for a response. From 71af030b46a89aaa9a87f18f56b9e1f1cd8ce2e7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 12 Sep 2014 18:42:50 -0700 Subject: [PATCH 0677/1492] [SPARK-3094] [PySpark] compatitable with PyPy After this patch, we can run PySpark in PyPy (testing with PyPy 2.3.1 in Mac 10.9), for example: ``` PYSPARK_PYTHON=pypy ./bin/spark-submit wordcount.py ``` The performance speed up will depend on work load (from 20% to 3000%). Here are some benchmarks: Job | CPython 2.7 | PyPy 2.3.1 | Speed up ------- | ------------ | ------------- | ------- Word Count | 41s | 15s | 2.7x Sort | 46s | 44s | 1.05x Stats | 174s | 3.6s | 48x Here is the code used for benchmark: ```python rdd = sc.textFile("text") def wordcount(): rdd.flatMap(lambda x:x.split('/'))\ .map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y).collectAsMap() def sort(): rdd.sortBy(lambda x:x, 1).count() def stats(): sc.parallelize(range(1024), 20).flatMap(lambda x: xrange(5024)).stats() ``` Author: Davies Liu Closes #2144 from davies/pypy and squashes the following commits: 9aed6c5 [Davies Liu] use protocol 2 in CloudPickle 4bc1f04 [Davies Liu] refactor b20ab3a [Davies Liu] pickle sys.stdout and stderr in portable way 3ca2351 [Davies Liu] Merge branch 'master' into pypy fae8b19 [Davies Liu] improve attrgetter, add tests 591f830 [Davies Liu] try to run tests with PyPy in run-tests c8d62ba [Davies Liu] cleanup f651fd0 [Davies Liu] fix tests using array with PyPy 1b98fb3 [Davies Liu] serialize itemgetter/attrgetter in portable ways 3c1dbfe [Davies Liu] Merge branch 'master' into pypy 42fb5fa [Davies Liu] Merge branch 'master' into pypy cb2d724 [Davies Liu] fix tests 9986692 [Davies Liu] Merge branch 'master' into pypy 25b4ca7 [Davies Liu] support PyPy --- python/pyspark/cloudpickle.py | 168 ++++++++++++++-------------------- python/pyspark/daemon.py | 6 +- python/pyspark/serializers.py | 10 +- python/pyspark/tests.py | 85 +++++++++++++++-- python/run-tests | 21 +++++ 5 files changed, 172 insertions(+), 118 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 80e51d1a583a0..32dda3888c62d 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -52,35 +52,19 @@ import itertools from copy_reg import _extension_registry, _inverted_registry, _extension_cache import new -import dis import traceback +import platform -#relevant opcodes -STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) -DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) -LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) -GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] +PyImp = platform.python_implementation() -HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) -EXTENDED_ARG = chr(dis.EXTENDED_ARG) import logging cloudLog = logging.getLogger("Cloud.Transport") -try: - import ctypes -except (MemoryError, ImportError): - logging.warning('Exception raised on importing ctypes. Likely python bug.. some functionality will be disabled', exc_info = True) - ctypes = None - PyObject_HEAD = None -else: - - # for reading internal structures - PyObject_HEAD = [ - ('ob_refcnt', ctypes.c_size_t), - ('ob_type', ctypes.c_void_p), - ] +if PyImp == "PyPy": + # register builtin type in `new` + new.method = types.MethodType try: from cStringIO import StringIO @@ -225,6 +209,8 @@ def save_function(self, obj, name=None, pack=struct.pack): if themodule: self.modules.add(themodule) + if getattr(themodule, name, None) is obj: + return self.save_global(obj, name) if not self.savedDjangoEnv: #hack for django - if we detect the settings module, we transport it @@ -306,44 +292,28 @@ def save_function_tuple(self, func, forced_imports): # create a skeleton function object and memoize it save(_make_skel_func) - save((code, len(closure), base_globals)) + save((code, closure, base_globals)) write(pickle.REDUCE) self.memoize(func) # save the rest of the func data needed by _fill_function save(f_globals) save(defaults) - save(closure) save(dct) write(pickle.TUPLE) write(pickle.REDUCE) # applies _fill_function on the tuple @staticmethod - def extract_code_globals(co): + def extract_code_globals(code): """ Find all globals names read or written to by codeblock co """ - code = co.co_code - names = co.co_names - out_names = set() - - n = len(code) - i = 0 - extended_arg = 0 - while i < n: - op = code[i] - - i = i+1 - if op >= HAVE_ARGUMENT: - oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg - extended_arg = 0 - i = i+2 - if op == EXTENDED_ARG: - extended_arg = oparg*65536L - if op in GLOBAL_OPS: - out_names.add(names[oparg]) - #print 'extracted', out_names, ' from ', names - return out_names + names = set(code.co_names) + if code.co_consts: # see if nested function have any global refs + for const in code.co_consts: + if type(const) is types.CodeType: + names |= CloudPickler.extract_code_globals(const) + return names def extract_func_data(self, func): """ @@ -354,10 +324,7 @@ def extract_func_data(self, func): # extract all global ref's func_global_refs = CloudPickler.extract_code_globals(code) - if code.co_consts: # see if nested function have any global refs - for const in code.co_consts: - if type(const) is types.CodeType and const.co_names: - func_global_refs = func_global_refs.union( CloudPickler.extract_code_globals(const)) + # process all variables referenced by global environment f_globals = {} for var in func_global_refs: @@ -396,6 +363,12 @@ def get_contents(cell): return (code, f_globals, defaults, closure, dct, base_globals) + def save_builtin_function(self, obj): + if obj.__module__ is "__builtin__": + return self.save_global(obj) + return self.save_function(obj) + dispatch[types.BuiltinFunctionType] = save_builtin_function + def save_global(self, obj, name=None, pack=struct.pack): write = self.write memo = self.memo @@ -435,7 +408,7 @@ def save_global(self, obj, name=None, pack=struct.pack): try: klass = getattr(themodule, name) except AttributeError, a: - #print themodule, name, obj, type(obj) + # print themodule, name, obj, type(obj) raise pickle.PicklingError("Can't pickle builtin %s" % obj) else: raise @@ -480,7 +453,6 @@ def save_global(self, obj, name=None, pack=struct.pack): write(pickle.GLOBAL + modname + '\n' + name + '\n') self.memoize(obj) dispatch[types.ClassType] = save_global - dispatch[types.BuiltinFunctionType] = save_global dispatch[types.TypeType] = save_global def save_instancemethod(self, obj): @@ -551,23 +523,39 @@ def save_property(self, obj): dispatch[property] = save_property def save_itemgetter(self, obj): - """itemgetter serializer (needed for namedtuple support) - a bit of a pain as we need to read ctypes internals""" - class ItemGetterType(ctypes.Structure): - _fields_ = PyObject_HEAD + [ - ('nitems', ctypes.c_size_t), - ('item', ctypes.py_object) - ] - - - obj = ctypes.cast(ctypes.c_void_p(id(obj)), ctypes.POINTER(ItemGetterType)).contents - return self.save_reduce(operator.itemgetter, - obj.item if obj.nitems > 1 else (obj.item,)) - - if PyObject_HEAD: + """itemgetter serializer (needed for namedtuple support)""" + class Dummy: + def __getitem__(self, item): + return item + items = obj(Dummy()) + if not isinstance(items, tuple): + items = (items, ) + return self.save_reduce(operator.itemgetter, items) + + if type(operator.itemgetter) is type: dispatch[operator.itemgetter] = save_itemgetter + def save_attrgetter(self, obj): + """attrgetter serializer""" + class Dummy(object): + def __init__(self, attrs, index=None): + self.attrs = attrs + self.index = index + def __getattribute__(self, item): + attrs = object.__getattribute__(self, "attrs") + index = object.__getattribute__(self, "index") + if index is None: + index = len(attrs) + attrs.append(item) + else: + attrs[index] = ".".join([attrs[index], item]) + return type(self)(attrs, index) + attrs = [] + obj(Dummy(attrs)) + return self.save_reduce(operator.attrgetter, tuple(attrs)) + if type(operator.attrgetter) is type: + dispatch[operator.attrgetter] = save_attrgetter def save_reduce(self, func, args, state=None, listitems=None, dictitems=None, obj=None): @@ -660,11 +648,11 @@ def save_file(self, obj): if not hasattr(obj, 'name') or not hasattr(obj, 'mode'): raise pickle.PicklingError("Cannot pickle files that do not map to an actual file") - if obj.name == '': + if obj is sys.stdout: return self.save_reduce(getattr, (sys,'stdout'), obj=obj) - if obj.name == '': + if obj is sys.stderr: return self.save_reduce(getattr, (sys,'stderr'), obj=obj) - if obj.name == '': + if obj is sys.stdin: raise pickle.PicklingError("Cannot pickle standard input") if hasattr(obj, 'isatty') and obj.isatty(): raise pickle.PicklingError("Cannot pickle files that map to tty objects") @@ -873,8 +861,7 @@ def _genpartial(func, args, kwds): kwds = {} return partial(func, *args, **kwds) - -def _fill_function(func, globals, defaults, closure, dict): +def _fill_function(func, globals, defaults, dict): """ Fills in the rest of function data into the skeleton function object that were created via _make_skel_func(). """ @@ -882,49 +869,28 @@ def _fill_function(func, globals, defaults, closure, dict): func.func_defaults = defaults func.func_dict = dict - if len(closure) != len(func.func_closure): - raise pickle.UnpicklingError("closure lengths don't match up") - for i in range(len(closure)): - _change_cell_value(func.func_closure[i], closure[i]) - return func -def _make_skel_func(code, num_closures, base_globals = None): +def _make_cell(value): + return (lambda: value).func_closure[0] + +def _reconstruct_closure(values): + return tuple([_make_cell(v) for v in values]) + +def _make_skel_func(code, closures, base_globals = None): """ Creates a skeleton function object that contains just the provided code and the correct number of cells in func_closure. All other func attributes (e.g. func_globals) are empty. """ - #build closure (cells): - if not ctypes: - raise Exception('ctypes failed to import; cannot build function') - - cellnew = ctypes.pythonapi.PyCell_New - cellnew.restype = ctypes.py_object - cellnew.argtypes = (ctypes.py_object,) - dummy_closure = tuple(map(lambda i: cellnew(None), range(num_closures))) + closure = _reconstruct_closure(closures) if closures else None if base_globals is None: base_globals = {} base_globals['__builtins__'] = __builtins__ return types.FunctionType(code, base_globals, - None, None, dummy_closure) - -# this piece of opaque code is needed below to modify 'cell' contents -cell_changer_code = new.code( - 1, 1, 2, 0, - ''.join([ - chr(dis.opmap['LOAD_FAST']), '\x00\x00', - chr(dis.opmap['DUP_TOP']), - chr(dis.opmap['STORE_DEREF']), '\x00\x00', - chr(dis.opmap['RETURN_VALUE']) - ]), - (), (), ('newval',), '', 'cell_changer', 1, '', ('c',), () -) - -def _change_cell_value(cell, newval): - """ Changes the contents of 'cell' object to newval """ - return new.function(cell_changer_code, {}, None, (), (cell,))(newval) + None, None, closure) + """Constructors for 3rd party libraries Note: These can never be renamed due to client compatibility issues""" diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 22ab8d30c0ae3..15445abf67147 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -42,10 +42,6 @@ def worker(sock): """ Called by a worker process after the fork(). """ - # Redirect stdout to stderr - os.dup2(2, 1) - sys.stdout = sys.stderr # The sys.stdout object is different from file descriptor 1 - signal.signal(SIGHUP, SIG_DFL) signal.signal(SIGCHLD, SIG_DFL) signal.signal(SIGTERM, SIG_DFL) @@ -102,6 +98,7 @@ def manager(): listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() write_int(listen_port, sys.stdout) + sys.stdout.flush() def shutdown(code): signal.signal(SIGTERM, SIG_DFL) @@ -115,7 +112,6 @@ def handle_sigterm(*args): signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP # Initialization complete - sys.stdout.close() try: while True: try: diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 7b2710b913128..a5f9341e819a9 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -355,7 +355,8 @@ class PickleSerializer(FramedSerializer): def dumps(self, obj): return cPickle.dumps(obj, 2) - loads = cPickle.loads + def loads(self, obj): + return cPickle.loads(obj) class CloudPickleSerializer(PickleSerializer): @@ -374,8 +375,11 @@ class MarshalSerializer(FramedSerializer): This serializer is faster than PickleSerializer but supports fewer datatypes. """ - dumps = marshal.dumps - loads = marshal.loads + def dumps(self, obj): + return marshal.dumps(obj) + + def loads(self, obj): + return marshal.loads(obj) class AutoSerializer(FramedSerializer): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index bb84ebe72cb24..2e7c2750a8bb6 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -31,6 +31,7 @@ import time import zipfile import random +from platform import python_implementation if sys.version_info[:2] <= (2, 6): import unittest2 as unittest @@ -41,7 +42,8 @@ from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.files import SparkFiles -from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer +from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ + CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType @@ -168,15 +170,46 @@ def test_namedtuple(self): p2 = loads(dumps(p1, 2)) self.assertEquals(p1, p2) - -# Regression test for SPARK-3415 -class CloudPickleTest(unittest.TestCase): + def test_itemgetter(self): + from operator import itemgetter + ser = CloudPickleSerializer() + d = range(10) + getter = itemgetter(1) + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + getter = itemgetter(0, 3) + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + def test_attrgetter(self): + from operator import attrgetter + ser = CloudPickleSerializer() + + class C(object): + def __getattr__(self, item): + return item + d = C() + getter = attrgetter("a") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + getter = attrgetter("a", "b") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + d.e = C() + getter = attrgetter("e.a") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + getter = attrgetter("e.a", "e.b") + getter2 = ser.loads(ser.dumps(getter)) + self.assertEqual(getter(d), getter2(d)) + + # Regression test for SPARK-3415 def test_pickling_file_handles(self): - from pyspark.cloudpickle import dumps - from StringIO import StringIO - from pickle import load + ser = CloudPickleSerializer() out1 = sys.stderr - out2 = load(StringIO(dumps(out1))) + out2 = ser.loads(ser.dumps(out1)) self.assertEquals(out1, out2) @@ -861,8 +894,42 @@ def test_oldhadoop(self): conf=input_conf).collect()) self.assertEqual(old_dataset, dict_data) - @unittest.skipIf(sys.version_info[:2] <= (2, 6), "Skipped on 2.6 until SPARK-2951 is fixed") def test_newhadoop(self): + basepath = self.tempdir.name + data = [(1, ""), + (1, "a"), + (2, "bcdf")] + self.sc.parallelize(data).saveAsNewAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text") + result = sorted(self.sc.newAPIHadoopFile( + basepath + "/newhadoop/", + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text").collect()) + self.assertEqual(result, data) + + conf = { + "mapreduce.outputformat.class": + "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat", + "mapred.output.key.class": "org.apache.hadoop.io.IntWritable", + "mapred.output.value.class": "org.apache.hadoop.io.Text", + "mapred.output.dir": basepath + "/newdataset/" + } + self.sc.parallelize(data).saveAsNewAPIHadoopDataset(conf) + input_conf = {"mapred.input.dir": basepath + "/newdataset/"} + new_dataset = sorted(self.sc.newAPIHadoopRDD( + "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.Text", + conf=input_conf).collect()) + self.assertEqual(new_dataset, data) + + @unittest.skipIf(sys.version_info[:2] <= (2, 6) or python_implementation() == "PyPy", + "Skipped on 2.6 and PyPy until SPARK-2951 is fixed") + def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays array_data = [(1, array('d')), diff --git a/python/run-tests b/python/run-tests index d98840de59d2c..a67e5a99fbdcc 100755 --- a/python/run-tests +++ b/python/run-tests @@ -85,6 +85,27 @@ run_test "pyspark/mllib/tests.py" run_test "pyspark/mllib/tree.py" run_test "pyspark/mllib/util.py" +# Try to test with PyPy +if [ $(which pypy) ]; then + export PYSPARK_PYTHON="pypy" + echo "Testing with PyPy version:" + $PYSPARK_PYTHON --version + + run_test "pyspark/rdd.py" + run_test "pyspark/context.py" + run_test "pyspark/conf.py" + run_test "pyspark/sql.py" + # These tests are included in the module-level docs, and so must + # be handled on a higher level rather than within the python file. + export PYSPARK_DOC_TEST=1 + run_test "pyspark/broadcast.py" + run_test "pyspark/accumulators.py" + run_test "pyspark/serializers.py" + unset PYSPARK_DOC_TEST + run_test "pyspark/shuffle.py" + run_test "pyspark/tests.py" +fi + if [[ $FAILED == 0 ]]; then echo -en "\033[32m" # Green echo "Tests passed." From 885d1621bc06bc1f009c9707c3452eac26baf828 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 12 Sep 2014 19:05:39 -0700 Subject: [PATCH 0678/1492] [SPARK-3500] [SQL] use JavaSchemaRDD as SchemaRDD._jschema_rdd Currently, SchemaRDD._jschema_rdd is SchemaRDD, the Scala API (coalesce(), repartition()) can not been called in Python easily, there is no way to specify the implicit parameter `ord`. The _jrdd is an JavaRDD, so _jschema_rdd should also be JavaSchemaRDD. In this patch, change _schema_rdd to JavaSchemaRDD, also added an assert for it. If some methods are missing from JavaSchemaRDD, then it's called by _schema_rdd.baseSchemaRDD().xxx(). BTW, Do we need JavaSQLContext? Author: Davies Liu Closes #2369 from davies/fix_schemardd and squashes the following commits: abee159 [Davies Liu] use JavaSchemaRDD as SchemaRDD._jschema_rdd --- python/pyspark/sql.py | 38 ++++++++++++++++++-------------------- python/pyspark/tests.py | 28 ++++++++++++++++++++++++++++ 2 files changed, 46 insertions(+), 20 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 53eea6d6cf3ba..fc9310fef318c 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1122,7 +1122,7 @@ def applySchema(self, rdd, schema): batched = isinstance(rdd._jrdd_deserializer, BatchedSerializer) jrdd = self._pythonToJava(rdd._jrdd, batched) srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), str(schema)) - return SchemaRDD(srdd, self) + return SchemaRDD(srdd.toJavaSchemaRDD(), self) def registerRDDAsTable(self, rdd, tableName): """Registers the given RDD as a temporary table in the catalog. @@ -1134,8 +1134,8 @@ def registerRDDAsTable(self, rdd, tableName): >>> sqlCtx.registerRDDAsTable(srdd, "table1") """ if (rdd.__class__ is SchemaRDD): - jschema_rdd = rdd._jschema_rdd - self._ssql_ctx.registerRDDAsTable(jschema_rdd, tableName) + srdd = rdd._jschema_rdd.baseSchemaRDD() + self._ssql_ctx.registerRDDAsTable(srdd, tableName) else: raise ValueError("Can only register SchemaRDD as table") @@ -1151,7 +1151,7 @@ def parquetFile(self, path): >>> sorted(srdd.collect()) == sorted(srdd2.collect()) True """ - jschema_rdd = self._ssql_ctx.parquetFile(path) + jschema_rdd = self._ssql_ctx.parquetFile(path).toJavaSchemaRDD() return SchemaRDD(jschema_rdd, self) def jsonFile(self, path, schema=None): @@ -1207,11 +1207,11 @@ def jsonFile(self, path, schema=None): [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] """ if schema is None: - jschema_rdd = self._ssql_ctx.jsonFile(path) + srdd = self._ssql_ctx.jsonFile(path) else: scala_datatype = self._ssql_ctx.parseDataType(str(schema)) - jschema_rdd = self._ssql_ctx.jsonFile(path, scala_datatype) - return SchemaRDD(jschema_rdd, self) + srdd = self._ssql_ctx.jsonFile(path, scala_datatype) + return SchemaRDD(srdd.toJavaSchemaRDD(), self) def jsonRDD(self, rdd, schema=None): """Loads an RDD storing one JSON object per string as a L{SchemaRDD}. @@ -1275,11 +1275,11 @@ def func(iterator): keyed._bypass_serializer = True jrdd = keyed._jrdd.map(self._jvm.BytesToString()) if schema is None: - jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + srdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) else: scala_datatype = self._ssql_ctx.parseDataType(str(schema)) - jschema_rdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) - return SchemaRDD(jschema_rdd, self) + srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) + return SchemaRDD(srdd.toJavaSchemaRDD(), self) def sql(self, sqlQuery): """Return a L{SchemaRDD} representing the result of the given query. @@ -1290,7 +1290,7 @@ def sql(self, sqlQuery): >>> srdd2.collect() [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] """ - return SchemaRDD(self._ssql_ctx.sql(sqlQuery), self) + return SchemaRDD(self._ssql_ctx.sql(sqlQuery).toJavaSchemaRDD(), self) def table(self, tableName): """Returns the specified table as a L{SchemaRDD}. @@ -1301,7 +1301,7 @@ def table(self, tableName): >>> sorted(srdd.collect()) == sorted(srdd2.collect()) True """ - return SchemaRDD(self._ssql_ctx.table(tableName), self) + return SchemaRDD(self._ssql_ctx.table(tableName).toJavaSchemaRDD(), self) def cacheTable(self, tableName): """Caches the specified table in-memory.""" @@ -1353,7 +1353,7 @@ def hiveql(self, hqlQuery): warnings.warn("hiveql() is deprecated as the sql function now parses using HiveQL by" + "default. The SQL dialect for parsing can be set using 'spark.sql.dialect'", DeprecationWarning) - return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery), self) + return SchemaRDD(self._ssql_ctx.hiveql(hqlQuery).toJavaSchemaRDD(), self) def hql(self, hqlQuery): """ @@ -1524,6 +1524,8 @@ class SchemaRDD(RDD): def __init__(self, jschema_rdd, sql_ctx): self.sql_ctx = sql_ctx self._sc = sql_ctx._sc + clsName = jschema_rdd.getClass().getName() + assert clsName.endswith("JavaSchemaRDD"), "jschema_rdd must be JavaSchemaRDD" self._jschema_rdd = jschema_rdd self._id = None self.is_cached = False @@ -1540,7 +1542,7 @@ def _jrdd(self): L{pyspark.rdd.RDD} super class (map, filter, etc.). """ if not hasattr(self, '_lazy_jrdd'): - self._lazy_jrdd = self._jschema_rdd.javaToPython() + self._lazy_jrdd = self._jschema_rdd.baseSchemaRDD().javaToPython() return self._lazy_jrdd def id(self): @@ -1598,7 +1600,7 @@ def saveAsTable(self, tableName): def schema(self): """Returns the schema of this SchemaRDD (represented by a L{StructType}).""" - return _parse_datatype_string(self._jschema_rdd.schema().toString()) + return _parse_datatype_string(self._jschema_rdd.baseSchemaRDD().schema().toString()) def schemaString(self): """Returns the output schema in the tree format.""" @@ -1649,8 +1651,6 @@ def mapPartitionsWithIndex(self, f, preservesPartitioning=False): rdd = RDD(self._jrdd, self._sc, self._jrdd_deserializer) schema = self.schema() - import pickle - pickle.loads(pickle.dumps(schema)) def applySchema(_, it): cls = _create_cls(schema) @@ -1687,10 +1687,8 @@ def isCheckpointed(self): def getCheckpointFile(self): checkpointFile = self._jschema_rdd.getCheckpointFile() - if checkpointFile.isDefined(): + if checkpointFile.isPresent(): return checkpointFile.get() - else: - return None def coalesce(self, numPartitions, shuffle=False): rdd = self._jschema_rdd.coalesce(numPartitions, shuffle) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 2e7c2750a8bb6..b687d695b01c4 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -607,6 +607,34 @@ def test_broadcast_in_udf(self): [res] = self.sqlCtx.sql("SELECT MYUDF('')").collect() self.assertEqual("", res[0]) + def test_basic_functions(self): + rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) + srdd = self.sqlCtx.jsonRDD(rdd) + srdd.count() + srdd.collect() + srdd.schemaString() + srdd.schema() + + # cache and checkpoint + self.assertFalse(srdd.is_cached) + srdd.persist() + srdd.unpersist() + srdd.cache() + self.assertTrue(srdd.is_cached) + self.assertFalse(srdd.isCheckpointed()) + self.assertEqual(None, srdd.getCheckpointFile()) + + srdd = srdd.coalesce(2, True) + srdd = srdd.repartition(3) + srdd = srdd.distinct() + srdd.intersection(srdd) + self.assertEqual(2, srdd.count()) + + srdd.registerTempTable("temp") + srdd = self.sqlCtx.sql("select foo from temp") + srdd.count() + srdd.collect() + class TestIO(PySparkTestCase): From 6d887db7891be643f0131b136e82191b5f6eb407 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 12 Sep 2014 20:14:09 -0700 Subject: [PATCH 0679/1492] [SPARK-3515][SQL] Moves test suite setup code to beforeAll rather than in constructor Please refer to the JIRA ticket for details. **NOTE** We should check all test suites that do similar initialization-like side effects in their constructors. This PR only fixes `ParquetMetastoreSuite` because it breaks our Jenkins Maven build. Author: Cheng Lian Closes #2375 from liancheng/say-no-to-constructor and squashes the following commits: 0ceb75b [Cheng Lian] Moves test suite setup code to beforeAll rather than in constructor --- .../sql/parquet/ParquetMetastoreSuite.scala | 53 +++++++++---------- 1 file changed, 24 insertions(+), 29 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala index 0723be7298e15..e380280f301c1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala @@ -20,14 +20,10 @@ package org.apache.spark.sql.parquet import java.io.File -import org.apache.spark.sql.hive.execution.HiveTableScan import org.scalatest.BeforeAndAfterAll -import scala.reflect.ClassTag - -import org.apache.spark.sql.{SQLConf, QueryTest} -import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.sql.hive.test.TestHive._ case class ParquetData(intField: Int, stringField: String) @@ -36,27 +32,19 @@ case class ParquetData(intField: Int, stringField: String) * Tests for our SerDe -> Native parquet scan conversion. */ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { - override def beforeAll(): Unit = { - setConf("spark.sql.hive.convertMetastoreParquet", "true") - } - - override def afterAll(): Unit = { - setConf("spark.sql.hive.convertMetastoreParquet", "false") - } - - val partitionedTableDir = File.createTempFile("parquettests", "sparksql") - partitionedTableDir.delete() - partitionedTableDir.mkdir() - - (1 to 10).foreach { p => - val partDir = new File(partitionedTableDir, s"p=$p") - sparkContext.makeRDD(1 to 10) - .map(i => ParquetData(i, s"part-$p")) - .saveAsParquetFile(partDir.getCanonicalPath) - } - - sql(s""" + val partitionedTableDir = File.createTempFile("parquettests", "sparksql") + partitionedTableDir.delete() + partitionedTableDir.mkdir() + + (1 to 10).foreach { p => + val partDir = new File(partitionedTableDir, s"p=$p") + sparkContext.makeRDD(1 to 10) + .map(i => ParquetData(i, s"part-$p")) + .saveAsParquetFile(partDir.getCanonicalPath) + } + + sql(s""" create external table partitioned_parquet ( intField INT, @@ -70,7 +58,7 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { location '${partitionedTableDir.getCanonicalPath}' """) - sql(s""" + sql(s""" create external table normal_parquet ( intField INT, @@ -83,8 +71,15 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { location '${new File(partitionedTableDir, "p=1").getCanonicalPath}' """) - (1 to 10).foreach { p => - sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") + (1 to 10).foreach { p => + sql(s"ALTER TABLE partitioned_parquet ADD PARTITION (p=$p)") + } + + setConf("spark.sql.hive.convertMetastoreParquet", "true") + } + + override def afterAll(): Unit = { + setConf("spark.sql.hive.convertMetastoreParquet", "false") } test("project the partitioning column") { From 2584ea5b23b1c5a4df9549b94bfc9b8e0900532e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 12 Sep 2014 21:55:39 -0700 Subject: [PATCH 0680/1492] [SPARK-3469] Make sure all TaskCompletionListener are called even with failures This is necessary because we rely on this callback interface to clean resources up. The old behavior would lead to resource leaks. Note that this also changes the fault semantics of TaskCompletionListener. Previously failures in TaskCompletionListeners would result in the task being reported immediately. With this change, we report the exception at the end, and the reported exception is a TaskCompletionListenerException that contains all the exception messages. Author: Reynold Xin Closes #2343 from rxin/taskcontext-callback and squashes the following commits: a3845b2 [Reynold Xin] Mark TaskCompletionListenerException as private[spark]. ac5baea [Reynold Xin] Removed obsolete comment. aa68ea4 [Reynold Xin] Throw an exception if task completion callback fails. 29b6162 [Reynold Xin] oops compilation failed. 1cb444d [Reynold Xin] [SPARK-3469] Call all TaskCompletionListeners even if some fail. --- .../scala/org/apache/spark/TaskContext.scala | 18 ++++++++-- .../TaskCompletionListenerException.scala | 34 +++++++++++++++++++ .../spark/scheduler/TaskContextSuite.scala | 22 ++++++++++-- 3 files changed, 69 insertions(+), 5 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 2b99b8a5af250..51b3e4d5e0936 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.TaskCompletionListener +import org.apache.spark.util.{TaskCompletionListenerException, TaskCompletionListener} /** @@ -41,7 +41,7 @@ class TaskContext( val attemptId: Long, val runningLocally: Boolean = false, private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty) - extends Serializable { + extends Serializable with Logging { @deprecated("use partitionId", "0.8.1") def splitId = partitionId @@ -103,8 +103,20 @@ class TaskContext( /** Marks the task as completed and triggers the listeners. */ private[spark] def markTaskCompleted(): Unit = { completed = true + val errorMsgs = new ArrayBuffer[String](2) // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach { _.onTaskCompletion(this) } + onCompleteCallbacks.reverse.foreach { listener => + try { + listener.onTaskCompletion(this) + } catch { + case e: Throwable => + errorMsgs += e.getMessage + logError("Error in TaskCompletionListener", e) + } + } + if (errorMsgs.nonEmpty) { + throw new TaskCompletionListenerException(errorMsgs) + } } /** Marks the task for interruption, i.e. cancellation. */ diff --git a/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala b/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala new file mode 100644 index 0000000000000..f64e069cd1724 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/TaskCompletionListenerException.scala @@ -0,0 +1,34 @@ +/* + * 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.util + +/** + * Exception thrown when there is an exception in + * executing the callback in TaskCompletionListener. + */ +private[spark] +class TaskCompletionListenerException(errorMessages: Seq[String]) extends Exception { + + override def getMessage: String = { + if (errorMessages.size == 1) { + errorMessages.head + } else { + errorMessages.zipWithIndex.map { case (msg, i) => s"Exception $i: $msg" }.mkString("\n") + } + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index db2ad829a48f9..faba5508c906c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.scheduler +import org.mockito.Mockito._ +import org.mockito.Matchers.any + import org.scalatest.FunSuite import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.rdd.RDD -import org.apache.spark.util.Utils +import org.apache.spark.util.{TaskCompletionListenerException, TaskCompletionListener} + class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { - test("Calls executeOnCompleteCallbacks after failure") { + test("calls TaskCompletionListener after failure") { TaskContextSuite.completed = false sc = new SparkContext("local", "test") val rdd = new RDD[String](sc, List()) { @@ -45,6 +49,20 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte } assert(TaskContextSuite.completed === true) } + + test("all TaskCompletionListeners should be called even if some fail") { + val context = new TaskContext(0, 0, 0) + val listener = mock(classOf[TaskCompletionListener]) + context.addTaskCompletionListener(_ => throw new Exception("blah")) + context.addTaskCompletionListener(listener) + context.addTaskCompletionListener(_ => throw new Exception("blah")) + + intercept[TaskCompletionListenerException] { + context.markTaskCompleted() + } + + verify(listener, times(1)).onTaskCompletion(any()) + } } private object TaskContextSuite { From e11eeb71fa3a5fe7ddacb94d5b93b173d4d901a8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Fri, 12 Sep 2014 21:58:02 -0700 Subject: [PATCH 0681/1492] [SQL][Docs] Update SQL programming guide to show the correct default value of containsNull in an ArrayType After #1889, the default value of `containsNull` in an `ArrayType` is `true`. Author: Yin Huai Closes #2374 from yhuai/containsNull and squashes the following commits: dc609a3 [Yin Huai] Update the SQL programming guide to show the correct default value of containsNull in an ArrayType (the default value is true instead of false). --- docs/sql-programming-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d83efa4bab324..3159d52787d5a 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1232,7 +1232,7 @@ import org.apache.spark.sql._ scala.collection.Seq ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is false. + Note: The default value of containsNull is true. @@ -1358,7 +1358,7 @@ please use factory methods provided in java.util.List DataType.createArrayType(elementType)
    - Note: The value of containsNull will be false
    + Note: The value of containsNull will be true
    DataType.createArrayType(elementType, containsNull). @@ -1505,7 +1505,7 @@ from pyspark.sql import * list, tuple, or array ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is False. + Note: The default value of containsNull is True. From feaa3706f17e44efcdac9f0a543a5b91232771ce Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 12 Sep 2014 22:50:37 -0700 Subject: [PATCH 0682/1492] SPARK-3470 [CORE] [STREAMING] Add Closeable / close() to Java context objects ... that expose a stop() lifecycle method. This doesn't add `AutoCloseable`, which is Java 7+ only. But it should be possible to use try-with-resources on a `Closeable` in Java 7, as long as the `close()` does not throw a checked exception, and these don't. Q.E.D. Author: Sean Owen Closes #2346 from srowen/SPARK-3470 and squashes the following commits: 612c21d [Sean Owen] Add Closeable / close() to Java context objects that expose a stop() lifecycle method --- .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 7 ++++++- .../spark/streaming/api/java/JavaStreamingContext.scala | 7 +++++-- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8e178bc8480f7..23f7e6be81a90 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -17,6 +17,7 @@ package org.apache.spark.api.java +import java.io.Closeable import java.util import java.util.{Map => JMap} @@ -40,7 +41,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. */ -class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround { +class JavaSparkContext(val sc: SparkContext) + extends JavaSparkContextVarargsWorkaround with Closeable { + /** * Create a JavaSparkContext that loads settings from system properties (for instance, when * launching with ./bin/spark-submit). @@ -534,6 +537,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork sc.stop() } + override def close(): Unit = stop() + /** * Get Spark's home location from either a value set through the constructor, * or the spark.home Java property, or the SPARK_HOME environment variable diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 18605cac7006c..9dc26dc6b32a1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -21,7 +21,7 @@ package org.apache.spark.streaming.api.java import scala.collection.JavaConversions._ import scala.reflect.ClassTag -import java.io.InputStream +import java.io.{Closeable, InputStream} import java.util.{List => JList, Map => JMap} import akka.actor.{Props, SupervisorStrategy} @@ -49,7 +49,7 @@ import org.apache.spark.streaming.receiver.Receiver * respectively. `context.awaitTransformation()` allows the current thread to wait for the * termination of a context by `stop()` or by an exception. */ -class JavaStreamingContext(val ssc: StreamingContext) { +class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Create a StreamingContext. @@ -540,6 +540,9 @@ class JavaStreamingContext(val ssc: StreamingContext) { def stop(stopSparkContext: Boolean, stopGracefully: Boolean) = { ssc.stop(stopSparkContext, stopGracefully) } + + override def close(): Unit = stop() + } /** From b4dded40fbecb485f1ddfd8316b44d42a1554d64 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 12 Sep 2014 22:51:25 -0700 Subject: [PATCH 0683/1492] Proper indent for the previous commit. --- .../main/scala/org/apache/spark/api/java/JavaSparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 23f7e6be81a90..791d853a015a1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -42,7 +42,7 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. */ class JavaSparkContext(val sc: SparkContext) - extends JavaSparkContextVarargsWorkaround with Closeable { + extends JavaSparkContextVarargsWorkaround with Closeable { /** * Create a JavaSparkContext that loads settings from system properties (for instance, when From a523ceaf159733dabcef84c7adc1463546679f65 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sat, 13 Sep 2014 12:34:20 -0700 Subject: [PATCH 0684/1492] [SQL] [Docs] typo fixes * Fixed random typo * Added in missing description for DecimalType Author: Nicholas Chammas Closes #2367 from nchammas/patch-1 and squashes the following commits: aa528be [Nicholas Chammas] doc fix for SQL DecimalType 3247ac1 [Nicholas Chammas] [SQL] [Docs] typo fixes --- docs/sql-programming-guide.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3159d52787d5a..8d41fdec699e9 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -918,7 +918,6 @@ options. ## Migration Guide for Shark User ### Scheduling -s To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, users can set the `spark.sql.thriftserver.scheduler.pool` variable: @@ -1110,7 +1109,7 @@ evaluated by the SQL execution engine. A full list of the functions supported c The range of numbers is from `-9223372036854775808` to `9223372036854775807`. - `FloatType`: Represents 4-byte single-precision floating point numbers. - `DoubleType`: Represents 8-byte double-precision floating point numbers. - - `DecimalType`: + - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. * String type - `StringType`: Represents character string values. * Binary type From 184cd51c4207c23726da97f907f2d912a5a44845 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 13 Sep 2014 12:35:40 -0700 Subject: [PATCH 0685/1492] [SPARK-3481][SQL] Removes the evil MINOR HACK This is a follow up of #2352. Now we can finally remove the evil "MINOR HACK", which covered up the eldest bug in the history of Spark SQL (see details [here](https://github.com/apache/spark/pull/2352#issuecomment-55440621)). Author: Cheng Lian Closes #2377 from liancheng/remove-evil-minor-hack and squashes the following commits: 0869c78 [Cheng Lian] Removes the evil MINOR HACK --- .../spark/sql/hive/execution/HiveComparisonTest.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 671c3b162f875..79cc7a3fcc7d6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -250,9 +250,9 @@ abstract class HiveComparisonTest } try { - // MINOR HACK: You must run a query before calling reset the first time. - TestHive.sql("SHOW TABLES") - if (reset) { TestHive.reset() } + if (reset) { + TestHive.reset() + } val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => From 74049249abb952ad061c0e221c22ff894a9e9c8d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 13 Sep 2014 15:08:30 -0700 Subject: [PATCH 0686/1492] [SPARK-3294][SQL] Eliminates boxing costs from in-memory columnar storage This is a major refactoring of the in-memory columnar storage implementation, aims to eliminate boxing costs from critical paths (building/accessing column buffers) as much as possible. The basic idea is to refactor all major interfaces into a row-based form and use them together with `SpecificMutableRow`. The difficult part is how to adapt all compression schemes, esp. `RunLengthEncoding` and `DictionaryEncoding`, to this design. Since in-memory compression is disabled by default for now, and this PR should be strictly better than before no matter in-memory compression is enabled or not, maybe I'll finish that part in another PR. **UPDATE** This PR also took the chance to optimize `HiveTableScan` by 1. leveraging `SpecificMutableRow` to avoid boxing cost, and 1. building specific `Writable` unwrapper functions a head of time to avoid per row pattern matching and branching costs. TODO - [x] Benchmark - [ ] ~~Eliminate boxing costs in `RunLengthEncoding`~~ (left to future PRs) - [ ] ~~Eliminate boxing costs in `DictionaryEncoding` (seems not easy to do without specializing `DictionaryEncoding` for every supported column type)~~ (left to future PRs) ## Micro benchmark The benchmark uses a 10 million line CSV table consists of bytes, shorts, integers, longs, floats and doubles, measures the time to build the in-memory version of this table, and the time to scan the whole in-memory table. Benchmark code can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-hivetablescanbenchmark-scala). Script used to generate the input table can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-tablegen-scala). Speedup: - Hive table scanning + column buffer building: **18.74%** The original benchmark uses 1K as in-memory batch size, when increased to 10K, it can be 28.32% faster. - In-memory table scanning: **7.95%** Before: | Building | Scanning ------- | -------- | -------- 1 | 16472 | 525 2 | 16168 | 530 3 | 16386 | 529 4 | 16184 | 538 5 | 16209 | 521 Average | 16283.8 | 528.6 After: | Building | Scanning ------- | -------- | -------- 1 | 13124 | 458 2 | 13260 | 529 3 | 12981 | 463 4 | 13214 | 483 5 | 13583 | 500 Average | 13232.4 | 486.6 Author: Cheng Lian Closes #2327 from liancheng/prevent-boxing/unboxing and squashes the following commits: 4419fe4 [Cheng Lian] Addressing comments e5d2cf2 [Cheng Lian] Bug fix: should call setNullAt when field value is null to avoid NPE 8b8552b [Cheng Lian] Only checks for partition batch pruning flag once 489f97b [Cheng Lian] Bug fix: TableReader.fillObject uses wrong ordinals 97bbc4e [Cheng Lian] Optimizes hive.TableReader by by providing specific Writable unwrappers a head of time 3dc1f94 [Cheng Lian] Minor changes to eliminate row object creation 5b39cb9 [Cheng Lian] Lowers log level of compression scheme details f2a7890 [Cheng Lian] Use SpecificMutableRow in InMemoryColumnarTableScan to avoid boxing 9cf30b0 [Cheng Lian] Added row based ColumnType.append/extract 456c366 [Cheng Lian] Made compression decoder row based edac3cd [Cheng Lian] Makes ColumnAccessor.extractSingle row based 8216936 [Cheng Lian] Removes boxing cost in IntDelta and LongDelta by providing specialized implementations b70d519 [Cheng Lian] Made some in-memory columnar storage interfaces row-based --- .../catalyst/expressions/SpecificRow.scala | 2 +- .../spark/sql/columnar/ColumnAccessor.scala | 8 +- .../spark/sql/columnar/ColumnBuilder.scala | 27 +- .../spark/sql/columnar/ColumnStats.scala | 16 +- .../spark/sql/columnar/ColumnType.scala | 178 ++++++++++-- .../columnar/InMemoryColumnarTableScan.scala | 92 +++--- .../sql/columnar/NullableColumnAccessor.scala | 4 +- .../sql/columnar/NullableColumnBuilder.scala | 8 +- .../CompressibleColumnAccessor.scala | 7 +- .../CompressibleColumnBuilder.scala | 24 +- .../compression/CompressionScheme.scala | 16 +- .../compression/compressionSchemes.scala | 264 +++++++++++------- .../spark/sql/columnar/ColumnStatsSuite.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 11 +- .../columnar/InMemoryColumnarQuerySuite.scala | 2 +- .../NullableColumnAccessorSuite.scala | 4 +- .../columnar/NullableColumnBuilderSuite.scala | 4 +- .../columnar/PartitionBatchPruningSuite.scala | 6 +- .../compression/BooleanBitSetSuite.scala | 7 +- .../compression/DictionaryEncodingSuite.scala | 9 +- .../compression/IntegralDeltaSuite.scala | 9 +- .../compression/RunLengthEncodingSuite.scala | 9 +- .../apache/spark/sql/hive/TableReader.scala | 119 ++++---- .../sql/hive/execution/HiveQuerySuite.scala | 18 +- 24 files changed, 554 insertions(+), 292 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala index 088f11ee4aa53..9cbab3d5d0d0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala @@ -171,7 +171,7 @@ final class MutableByte extends MutableValue { } final class MutableAny extends MutableValue { - var value: Any = 0 + var value: Any = _ def boxed = if (isNull) null else value def update(v: Any) = value = { isNull = false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index 42a5a9a84f362..c9faf0852142a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -50,11 +50,13 @@ private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( def hasNext = buffer.hasRemaining - def extractTo(row: MutableRow, ordinal: Int) { - columnType.setField(row, ordinal, extractSingle(buffer)) + def extractTo(row: MutableRow, ordinal: Int): Unit = { + extractSingle(row, ordinal) } - def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer) + def extractSingle(row: MutableRow, ordinal: Int): Unit = { + columnType.extract(buffer, row, ordinal) + } protected def underlyingBuffer = buffer } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index b3ec5ded22422..2e61a981375aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -68,10 +68,9 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId) } - override def appendFrom(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - buffer = ensureFreeSpace(buffer, columnType.actualSize(field)) - columnType.append(field, buffer) + override def appendFrom(row: Row, ordinal: Int): Unit = { + buffer = ensureFreeSpace(buffer, columnType.actualSize(row, ordinal)) + columnType.append(row, ordinal, buffer) } override def build() = { @@ -142,16 +141,16 @@ private[sql] object ColumnBuilder { useCompression: Boolean = false): ColumnBuilder = { val builder = (typeId match { - case INT.typeId => new IntColumnBuilder - case LONG.typeId => new LongColumnBuilder - case FLOAT.typeId => new FloatColumnBuilder - case DOUBLE.typeId => new DoubleColumnBuilder - case BOOLEAN.typeId => new BooleanColumnBuilder - case BYTE.typeId => new ByteColumnBuilder - case SHORT.typeId => new ShortColumnBuilder - case STRING.typeId => new StringColumnBuilder - case BINARY.typeId => new BinaryColumnBuilder - case GENERIC.typeId => new GenericColumnBuilder + case INT.typeId => new IntColumnBuilder + case LONG.typeId => new LongColumnBuilder + case FLOAT.typeId => new FloatColumnBuilder + case DOUBLE.typeId => new DoubleColumnBuilder + case BOOLEAN.typeId => new BooleanColumnBuilder + case BYTE.typeId => new ByteColumnBuilder + case SHORT.typeId => new ShortColumnBuilder + case STRING.typeId => new StringColumnBuilder + case BINARY.typeId => new BinaryColumnBuilder + case GENERIC.typeId => new GenericColumnBuilder case TIMESTAMP.typeId => new TimestampColumnBuilder }).asInstanceOf[ColumnBuilder] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index fc343ccb995c2..203a714e03c97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -69,7 +69,7 @@ private[sql] class ByteColumnStats extends ColumnStats { var lower = Byte.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getByte(ordinal) if (value > upper) upper = value @@ -87,7 +87,7 @@ private[sql] class ShortColumnStats extends ColumnStats { var lower = Short.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getShort(ordinal) if (value > upper) upper = value @@ -105,7 +105,7 @@ private[sql] class LongColumnStats extends ColumnStats { var lower = Long.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getLong(ordinal) if (value > upper) upper = value @@ -123,7 +123,7 @@ private[sql] class DoubleColumnStats extends ColumnStats { var lower = Double.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getDouble(ordinal) if (value > upper) upper = value @@ -141,7 +141,7 @@ private[sql] class FloatColumnStats extends ColumnStats { var lower = Float.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getFloat(ordinal) if (value > upper) upper = value @@ -159,7 +159,7 @@ private[sql] class IntColumnStats extends ColumnStats { var lower = Int.MaxValue var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getInt(ordinal) if (value > upper) upper = value @@ -177,7 +177,7 @@ private[sql] class StringColumnStats extends ColumnStats { var lower: String = null var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row.getString(ordinal) if (upper == null || value.compareTo(upper) > 0) upper = value @@ -195,7 +195,7 @@ private[sql] class TimestampColumnStats extends ColumnStats { var lower: Timestamp = null var nullCount = 0 - override def gatherStats(row: Row, ordinal: Int) { + override def gatherStats(row: Row, ordinal: Int): Unit = { if (!row.isNullAt(ordinal)) { val value = row(ordinal).asInstanceOf[Timestamp] if (upper == null || value.compareTo(upper) > 0) upper = value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 9a61600115872..198b5756676aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,11 +18,10 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import java.sql.Timestamp import scala.reflect.runtime.universe.TypeTag -import java.sql.Timestamp - import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types._ @@ -46,16 +45,33 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( */ def extract(buffer: ByteBuffer): JvmType + /** + * Extracts a value out of the buffer at the buffer's current position and stores in + * `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs whenever + * possible. + */ + def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + setField(row, ordinal, extract(buffer)) + } + /** * Appends the given value v of type T into the given ByteBuffer. */ - def append(v: JvmType, buffer: ByteBuffer) + def append(v: JvmType, buffer: ByteBuffer): Unit + + /** + * Appends `row(ordinal)` of type T into the given ByteBuffer. Subclasses should override this + * method to avoid boxing/unboxing costs whenever possible. + */ + def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + append(getField(row, ordinal), buffer) + } /** - * Returns the size of the value. This is used to calculate the size of variable length types - * such as byte arrays and strings. + * Returns the size of the value `row(ordinal)`. This is used to calculate the size of variable + * length types such as byte arrays and strings. */ - def actualSize(v: JvmType): Int = defaultSize + def actualSize(row: Row, ordinal: Int): Int = defaultSize /** * Returns `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs @@ -67,7 +83,15 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( * Sets `row(ordinal)` to `field`. Subclasses should override this method to avoid boxing/unboxing * costs whenever possible. */ - def setField(row: MutableRow, ordinal: Int, value: JvmType) + def setField(row: MutableRow, ordinal: Int, value: JvmType): Unit + + /** + * Copies `from(fromOrdinal)` to `to(toOrdinal)`. Subclasses should override this method to avoid + * boxing/unboxing costs whenever possible. + */ + def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to(toOrdinal) = from(fromOrdinal) + } /** * Creates a duplicated copy of the value. @@ -90,119 +114,205 @@ private[sql] abstract class NativeColumnType[T <: NativeType]( } private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { - def append(v: Int, buffer: ByteBuffer) { + def append(v: Int, buffer: ByteBuffer): Unit = { buffer.putInt(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putInt(row.getInt(ordinal)) + } + def extract(buffer: ByteBuffer) = { buffer.getInt() } - override def setField(row: MutableRow, ordinal: Int, value: Int) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setInt(ordinal, buffer.getInt()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Int): Unit = { row.setInt(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getInt(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setInt(toOrdinal, from.getInt(fromOrdinal)) + } } private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { - override def append(v: Long, buffer: ByteBuffer) { + override def append(v: Long, buffer: ByteBuffer): Unit = { buffer.putLong(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putLong(row.getLong(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getLong() } - override def setField(row: MutableRow, ordinal: Int, value: Long) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setLong(ordinal, buffer.getLong()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Long): Unit = { row.setLong(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getLong(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setLong(toOrdinal, from.getLong(fromOrdinal)) + } } private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { - override def append(v: Float, buffer: ByteBuffer) { + override def append(v: Float, buffer: ByteBuffer): Unit = { buffer.putFloat(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putFloat(row.getFloat(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getFloat() } - override def setField(row: MutableRow, ordinal: Int, value: Float) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setFloat(ordinal, buffer.getFloat()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Float): Unit = { row.setFloat(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getFloat(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setFloat(toOrdinal, from.getFloat(fromOrdinal)) + } } private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { - override def append(v: Double, buffer: ByteBuffer) { + override def append(v: Double, buffer: ByteBuffer): Unit = { buffer.putDouble(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putDouble(row.getDouble(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getDouble() } - override def setField(row: MutableRow, ordinal: Int, value: Double) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setDouble(ordinal, buffer.getDouble()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Double): Unit = { row.setDouble(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getDouble(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setDouble(toOrdinal, from.getDouble(fromOrdinal)) + } } private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { - override def append(v: Boolean, buffer: ByteBuffer) { - buffer.put(if (v) 1.toByte else 0.toByte) + override def append(v: Boolean, buffer: ByteBuffer): Unit = { + buffer.put(if (v) 1: Byte else 0: Byte) + } + + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.put(if (row.getBoolean(ordinal)) 1: Byte else 0: Byte) } override def extract(buffer: ByteBuffer) = buffer.get() == 1 - override def setField(row: MutableRow, ordinal: Int, value: Boolean) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setBoolean(ordinal, buffer.get() == 1) + } + + override def setField(row: MutableRow, ordinal: Int, value: Boolean): Unit = { row.setBoolean(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getBoolean(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setBoolean(toOrdinal, from.getBoolean(fromOrdinal)) + } } private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { - override def append(v: Byte, buffer: ByteBuffer) { + override def append(v: Byte, buffer: ByteBuffer): Unit = { buffer.put(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.put(row.getByte(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.get() } - override def setField(row: MutableRow, ordinal: Int, value: Byte) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setByte(ordinal, buffer.get()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Byte): Unit = { row.setByte(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getByte(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setByte(toOrdinal, from.getByte(fromOrdinal)) + } } private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { - override def append(v: Short, buffer: ByteBuffer) { + override def append(v: Short, buffer: ByteBuffer): Unit = { buffer.putShort(v) } + override def append(row: Row, ordinal: Int, buffer: ByteBuffer): Unit = { + buffer.putShort(row.getShort(ordinal)) + } + override def extract(buffer: ByteBuffer) = { buffer.getShort() } - override def setField(row: MutableRow, ordinal: Int, value: Short) { + override def extract(buffer: ByteBuffer, row: MutableRow, ordinal: Int): Unit = { + row.setShort(ordinal, buffer.getShort()) + } + + override def setField(row: MutableRow, ordinal: Int, value: Short): Unit = { row.setShort(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getShort(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setShort(toOrdinal, from.getShort(fromOrdinal)) + } } private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { - override def actualSize(v: String): Int = v.getBytes("utf-8").length + 4 + override def actualSize(row: Row, ordinal: Int): Int = { + row.getString(ordinal).getBytes("utf-8").length + 4 + } - override def append(v: String, buffer: ByteBuffer) { + override def append(v: String, buffer: ByteBuffer): Unit = { val stringBytes = v.getBytes("utf-8") buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } @@ -214,11 +324,15 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { new String(stringBytes, "utf-8") } - override def setField(row: MutableRow, ordinal: Int, value: String) { + override def setField(row: MutableRow, ordinal: Int, value: String): Unit = { row.setString(ordinal, value) } override def getField(row: Row, ordinal: Int) = row.getString(ordinal) + + override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { + to.setString(toOrdinal, from.getString(fromOrdinal)) + } } private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { @@ -228,7 +342,7 @@ private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { timestamp } - override def append(v: Timestamp, buffer: ByteBuffer) { + override def append(v: Timestamp, buffer: ByteBuffer): Unit = { buffer.putLong(v.getTime).putInt(v.getNanos) } @@ -236,7 +350,7 @@ private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { row(ordinal).asInstanceOf[Timestamp] } - override def setField(row: MutableRow, ordinal: Int, value: Timestamp) { + override def setField(row: MutableRow, ordinal: Int, value: Timestamp): Unit = { row(ordinal) = value } } @@ -246,9 +360,11 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( defaultSize: Int) extends ColumnType[T, Array[Byte]](typeId, defaultSize) { - override def actualSize(v: Array[Byte]) = v.length + 4 + override def actualSize(row: Row, ordinal: Int) = { + getField(row, ordinal).length + 4 + } - override def append(v: Array[Byte], buffer: ByteBuffer) { + override def append(v: Array[Byte], buffer: ByteBuffer): Unit = { buffer.putInt(v.length).put(v, 0, v.length) } @@ -261,7 +377,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { - override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = value } @@ -272,7 +388,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. private[sql] object GENERIC extends ByteArrayColumnType[DataType](10, 16) { - override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } 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 6eab2f23c18e1..8a3612cdf19be 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 @@ -52,7 +52,7 @@ private[sql] case class InMemoryRelation( // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { val output = child.output - val cached = child.execute().mapPartitions { baseIterator => + val cached = child.execute().mapPartitions { rowIterator => new Iterator[CachedBatch] { def next() = { val columnBuilders = output.map { attribute => @@ -61,11 +61,9 @@ private[sql] case class InMemoryRelation( ColumnBuilder(columnType.typeId, initialBufferSize, attribute.name, useCompression) }.toArray - var row: Row = null var rowCount = 0 - - while (baseIterator.hasNext && rowCount < batchSize) { - row = baseIterator.next() + while (rowIterator.hasNext && rowCount < batchSize) { + val row = rowIterator.next() var i = 0 while (i < row.length) { columnBuilders(i).appendFrom(row, i) @@ -80,7 +78,7 @@ private[sql] case class InMemoryRelation( CachedBatch(columnBuilders.map(_.build()), stats) } - def hasNext = baseIterator.hasNext + def hasNext = rowIterator.hasNext } }.cache() @@ -182,6 +180,7 @@ private[sql] case class InMemoryColumnarTableScan( } } + // Accumulators used for testing purposes val readPartitions = sparkContext.accumulator(0) val readBatches = sparkContext.accumulator(0) @@ -191,40 +190,36 @@ private[sql] case class InMemoryColumnarTableScan( readPartitions.setValue(0) readBatches.setValue(0) - relation.cachedColumnBuffers.mapPartitions { iterator => + relation.cachedColumnBuffers.mapPartitions { cachedBatchIterator => val partitionFilter = newPredicate( partitionFilters.reduceOption(And).getOrElse(Literal(true)), relation.partitionStatistics.schema) - // Find the ordinals of the requested columns. If none are requested, use the first. - val requestedColumns = if (attributes.isEmpty) { - Seq(0) + // Find the ordinals and data types of the requested columns. If none are requested, use the + // narrowest (the field with minimum default element size). + val (requestedColumnIndices, requestedColumnDataTypes) = if (attributes.isEmpty) { + val (narrowestOrdinal, narrowestDataType) = + relation.output.zipWithIndex.map { case (a, ordinal) => + ordinal -> a.dataType + } minBy { case (_, dataType) => + ColumnType(dataType).defaultSize + } + Seq(narrowestOrdinal) -> Seq(narrowestDataType) } else { - attributes.map(a => relation.output.indexWhere(_.exprId == a.exprId)) + attributes.map { a => + relation.output.indexWhere(_.exprId == a.exprId) -> a.dataType + }.unzip } - val rows = iterator - // Skip pruned batches - .filter { cachedBatch => - if (inMemoryPartitionPruningEnabled && !partitionFilter(cachedBatch.stats)) { - def statsString = relation.partitionStatistics.schema - .zip(cachedBatch.stats) - .map { case (a, s) => s"${a.name}: $s" } - .mkString(", ") - logInfo(s"Skipping partition based on stats $statsString") - false - } else { - readBatches += 1 - true - } - } - // Build column accessors - .map { cachedBatch => - requestedColumns.map(cachedBatch.buffers(_)).map(ColumnAccessor(_)) - } - // Extract rows via column accessors - .flatMap { columnAccessors => - val nextRow = new GenericMutableRow(columnAccessors.length) + val nextRow = new SpecificMutableRow(requestedColumnDataTypes) + + def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]) = { + val rows = cacheBatches.flatMap { cachedBatch => + // Build column accessors + val columnAccessors = + requestedColumnIndices.map(cachedBatch.buffers(_)).map(ColumnAccessor(_)) + + // Extract rows via column accessors new Iterator[Row] { override def next() = { var i = 0 @@ -235,15 +230,38 @@ private[sql] case class InMemoryColumnarTableScan( nextRow } - override def hasNext = columnAccessors.head.hasNext + override def hasNext = columnAccessors(0).hasNext } } - if (rows.hasNext) { - readPartitions += 1 + if (rows.hasNext) { + readPartitions += 1 + } + + rows } - rows + // Do partition batch pruning if enabled + val cachedBatchesToScan = + if (inMemoryPartitionPruningEnabled) { + cachedBatchIterator.filter { cachedBatch => + if (!partitionFilter(cachedBatch.stats)) { + def statsString = relation.partitionStatistics.schema + .zip(cachedBatch.stats) + .map { case (a, s) => s"${a.name}: $s" } + .mkString(", ") + logInfo(s"Skipping partition based on stats $statsString") + false + } else { + readBatches += 1 + true + } + } + } else { + cachedBatchIterator + } + + cachedBatchesToRows(cachedBatchesToScan) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index b7f8826861a2c..965782a40031b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -29,7 +29,7 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { private var nextNullIndex: Int = _ private var pos: Int = 0 - abstract override protected def initialize() { + abstract override protected def initialize(): Unit = { nullsBuffer = underlyingBuffer.duplicate().order(ByteOrder.nativeOrder()) nullCount = nullsBuffer.getInt() nextNullIndex = if (nullCount > 0) nullsBuffer.getInt() else -1 @@ -39,7 +39,7 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { super.initialize() } - abstract override def extractTo(row: MutableRow, ordinal: Int) { + abstract override def extractTo(row: MutableRow, ordinal: Int): Unit = { if (pos == nextNullIndex) { seenNulls += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index a72970eef7aa4..f1f494ac26d0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -40,7 +40,11 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { protected var nullCount: Int = _ private var pos: Int = _ - abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { + abstract override def initialize( + initialSize: Int, + columnName: String, + useCompression: Boolean): Unit = { + nulls = ByteBuffer.allocate(1024) nulls.order(ByteOrder.nativeOrder()) pos = 0 @@ -48,7 +52,7 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { super.initialize(initialSize, columnName, useCompression) } - abstract override def appendFrom(row: Row, ordinal: Int) { + abstract override def appendFrom(row: Row, ordinal: Int): Unit = { columnStats.gatherStats(row, ordinal) if (row.isNullAt(ordinal)) { nulls = ColumnBuilder.ensureFreeSpace(nulls, 4) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala index b4120a3d4368b..27ac5f4dbdbbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.columnar.compression -import java.nio.ByteBuffer - +import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} @@ -34,5 +33,7 @@ private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAcc abstract override def hasNext = super.hasNext || decoder.hasNext - override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() + override def extractSingle(row: MutableRow, ordinal: Int): Unit = { + decoder.next(row, ordinal) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index a5826bb033e41..628d9cec41d6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -48,12 +48,16 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] var compressionEncoders: Seq[Encoder[T]] = _ - abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) { + abstract override def initialize( + initialSize: Int, + columnName: String, + useCompression: Boolean): Unit = { + compressionEncoders = if (useCompression) { - schemes.filter(_.supports(columnType)).map(_.encoder[T]) + schemes.filter(_.supports(columnType)).map(_.encoder[T](columnType)) } else { - Seq(PassThrough.encoder) + Seq(PassThrough.encoder(columnType)) } super.initialize(initialSize, columnName, useCompression) } @@ -62,17 +66,15 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] encoder.compressionRatio < 0.8 } - private def gatherCompressibilityStats(row: Row, ordinal: Int) { - val field = columnType.getField(row, ordinal) - + private def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { var i = 0 while (i < compressionEncoders.length) { - compressionEncoders(i).gatherCompressibilityStats(field, columnType) + compressionEncoders(i).gatherCompressibilityStats(row, ordinal) i += 1 } } - abstract override def appendFrom(row: Row, ordinal: Int) { + abstract override def appendFrom(row: Row, ordinal: Int): Unit = { super.appendFrom(row, ordinal) if (!row.isNullAt(ordinal)) { gatherCompressibilityStats(row, ordinal) @@ -84,7 +86,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] val typeId = nonNullBuffer.getInt() val encoder: Encoder[T] = { val candidate = compressionEncoders.minBy(_.compressionRatio) - if (isWorthCompressing(candidate)) candidate else PassThrough.encoder + if (isWorthCompressing(candidate)) candidate else PassThrough.encoder(columnType) } // Header = column type ID + null count + null positions @@ -104,7 +106,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] .putInt(nullCount) .put(nulls) - logInfo(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") - encoder.compress(nonNullBuffer, compressedBuffer, columnType) + logDebug(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + encoder.compress(nonNullBuffer, compressedBuffer) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index 7797f75177893..acb06cb5376b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.columnar.compression -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} private[sql] trait Encoder[T <: NativeType] { - def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) {} + def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = {} def compressedSize: Int @@ -33,17 +35,21 @@ private[sql] trait Encoder[T <: NativeType] { if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0 } - def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]): ByteBuffer + def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer } -private[sql] trait Decoder[T <: NativeType] extends Iterator[T#JvmType] +private[sql] trait Decoder[T <: NativeType] { + def next(row: MutableRow, ordinal: Int): Unit + + def hasNext: Boolean +} private[sql] trait CompressionScheme { def typeId: Int def supports(columnType: ColumnType[_, _]): Boolean - def encoder[T <: NativeType]: Encoder[T] + def encoder[T <: NativeType](columnType: NativeColumnType[T]): Encoder[T] def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index 8cf9ec74ca2de..29edcf17242c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -23,7 +23,8 @@ import scala.collection.mutable import scala.reflect.ClassTag import scala.reflect.runtime.universe.runtimeMirror -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.{MutableRow, SpecificMutableRow} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ import org.apache.spark.util.Utils @@ -33,18 +34,20 @@ private[sql] case object PassThrough extends CompressionScheme { override def supports(columnType: ColumnType[_, _]) = true - override def encoder[T <: NativeType] = new this.Encoder[T] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + new this.Encoder[T](columnType) + } override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) } - class Encoder[T <: NativeType] extends compression.Encoder[T] { + class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { override def uncompressedSize = 0 override def compressedSize = 0 - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { + override def compress(from: ByteBuffer, to: ByteBuffer) = { // Writes compression type ID and copies raw contents to.putInt(PassThrough.typeId).put(from).rewind() to @@ -54,7 +57,9 @@ private[sql] case object PassThrough extends CompressionScheme { class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) extends compression.Decoder[T] { - override def next() = columnType.extract(buffer) + override def next(row: MutableRow, ordinal: Int): Unit = { + columnType.extract(buffer, row, ordinal) + } override def hasNext = buffer.hasRemaining } @@ -63,7 +68,9 @@ private[sql] case object PassThrough extends CompressionScheme { private[sql] case object RunLengthEncoding extends CompressionScheme { override val typeId = 1 - override def encoder[T <: NativeType] = new this.Encoder[T] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + new this.Encoder[T](columnType) + } override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) @@ -74,24 +81,25 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { case _ => false } - class Encoder[T <: NativeType] extends compression.Encoder[T] { + class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { private var _uncompressedSize = 0 private var _compressedSize = 0 // Using `MutableRow` to store the last value to avoid boxing/unboxing cost. - private val lastValue = new GenericMutableRow(1) + private val lastValue = new SpecificMutableRow(Seq(columnType.dataType)) private var lastRun = 0 override def uncompressedSize = _uncompressedSize override def compressedSize = _compressedSize - override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { - val actualSize = columnType.actualSize(value) + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = columnType.getField(row, ordinal) + val actualSize = columnType.actualSize(row, ordinal) _uncompressedSize += actualSize if (lastValue.isNullAt(0)) { - columnType.setField(lastValue, 0, value) + columnType.copyField(row, ordinal, lastValue, 0) lastRun = 1 _compressedSize += actualSize + 4 } else { @@ -99,37 +107,40 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { lastRun += 1 } else { _compressedSize += actualSize + 4 - columnType.setField(lastValue, 0, value) + columnType.copyField(row, ordinal, lastValue, 0) lastRun = 1 } } } - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { + override def compress(from: ByteBuffer, to: ByteBuffer) = { to.putInt(RunLengthEncoding.typeId) if (from.hasRemaining) { - var currentValue = columnType.extract(from) + val currentValue = new SpecificMutableRow(Seq(columnType.dataType)) var currentRun = 1 + val value = new SpecificMutableRow(Seq(columnType.dataType)) + + columnType.extract(from, currentValue, 0) while (from.hasRemaining) { - val value = columnType.extract(from) + columnType.extract(from, value, 0) - if (value == currentValue) { + if (value.head == currentValue.head) { currentRun += 1 } else { // Writes current run - columnType.append(currentValue, to) + columnType.append(currentValue, 0, to) to.putInt(currentRun) // Resets current run - currentValue = value + columnType.copyField(value, 0, currentValue, 0) currentRun = 1 } } // Writes the last run - columnType.append(currentValue, to) + columnType.append(currentValue, 0, to) to.putInt(currentRun) } @@ -145,7 +156,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { private var valueCount = 0 private var currentValue: T#JvmType = _ - override def next() = { + override def next(row: MutableRow, ordinal: Int): Unit = { if (valueCount == run) { currentValue = columnType.extract(buffer) run = buffer.getInt() @@ -154,7 +165,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { valueCount += 1 } - currentValue + columnType.setField(row, ordinal, currentValue) } override def hasNext = valueCount < run || buffer.hasRemaining @@ -171,14 +182,16 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { new this.Decoder(buffer, columnType) } - override def encoder[T <: NativeType] = new this.Encoder[T] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + new this.Encoder[T](columnType) + } override def supports(columnType: ColumnType[_, _]) = columnType match { case INT | LONG | STRING => true case _ => false } - class Encoder[T <: NativeType] extends compression.Encoder[T] { + class Encoder[T <: NativeType](columnType: NativeColumnType[T]) extends compression.Encoder[T] { // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary // overflows. private var _uncompressedSize = 0 @@ -200,9 +213,11 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { // to store dictionary element count. private var dictionarySize = 4 - override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = columnType.getField(row, ordinal) + if (!overflow) { - val actualSize = columnType.actualSize(value) + val actualSize = columnType.actualSize(row, ordinal) count += 1 _uncompressedSize += actualSize @@ -221,7 +236,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { } } - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { + override def compress(from: ByteBuffer, to: ByteBuffer) = { if (overflow) { throw new IllegalStateException( "Dictionary encoding should not be used because of dictionary overflow.") @@ -264,7 +279,9 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { } } - override def next() = dictionary(buffer.getShort()) + override def next(row: MutableRow, ordinal: Int): Unit = { + columnType.setField(row, ordinal, dictionary(buffer.getShort())) + } override def hasNext = buffer.hasRemaining } @@ -279,25 +296,20 @@ private[sql] case object BooleanBitSet extends CompressionScheme { new this.Decoder(buffer).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + (new this.Encoder).asInstanceOf[compression.Encoder[T]] + } override def supports(columnType: ColumnType[_, _]) = columnType == BOOLEAN class Encoder extends compression.Encoder[BooleanType.type] { private var _uncompressedSize = 0 - override def gatherCompressibilityStats( - value: Boolean, - columnType: NativeColumnType[BooleanType.type]) { - + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { _uncompressedSize += BOOLEAN.defaultSize } - override def compress( - from: ByteBuffer, - to: ByteBuffer, - columnType: NativeColumnType[BooleanType.type]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer) = { to.putInt(BooleanBitSet.typeId) // Total element count (1 byte per Boolean value) .putInt(from.remaining) @@ -349,7 +361,7 @@ private[sql] case object BooleanBitSet extends CompressionScheme { private var visited: Int = 0 - override def next(): Boolean = { + override def next(row: MutableRow, ordinal: Int): Unit = { val bit = visited % BITS_PER_LONG visited += 1 @@ -357,123 +369,167 @@ private[sql] case object BooleanBitSet extends CompressionScheme { currentWord = buffer.getLong() } - ((currentWord >> bit) & 1) != 0 + row.setBoolean(ordinal, ((currentWord >> bit) & 1) != 0) } override def hasNext: Boolean = visited < count } } -private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends CompressionScheme { +private[sql] case object IntDelta extends CompressionScheme { + override def typeId: Int = 4 + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { - new this.Decoder(buffer, columnType.asInstanceOf[NativeColumnType[I]]) - .asInstanceOf[compression.Decoder[T]] + new Decoder(buffer, INT).asInstanceOf[compression.Decoder[T]] } - override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] - - /** - * Computes `delta = x - y`, returns `(true, delta)` if `delta` can fit into a single byte, or - * `(false, 0: Byte)` otherwise. - */ - protected def byteSizedDelta(x: I#JvmType, y: I#JvmType): (Boolean, Byte) + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + (new Encoder).asInstanceOf[compression.Encoder[T]] + } - /** - * Simply computes `x + delta` - */ - protected def addDelta(x: I#JvmType, delta: Byte): I#JvmType + override def supports(columnType: ColumnType[_, _]) = columnType == INT - class Encoder extends compression.Encoder[I] { - private var _compressedSize: Int = 0 + class Encoder extends compression.Encoder[IntegerType.type] { + protected var _compressedSize: Int = 0 + protected var _uncompressedSize: Int = 0 - private var _uncompressedSize: Int = 0 + override def compressedSize = _compressedSize + override def uncompressedSize = _uncompressedSize - private var prev: I#JvmType = _ + private var prevValue: Int = _ - private var initial = true + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = row.getInt(ordinal) + val delta = value - prevValue - override def gatherCompressibilityStats(value: I#JvmType, columnType: NativeColumnType[I]) { - _uncompressedSize += columnType.defaultSize + _compressedSize += 1 - if (initial) { - initial = false - _compressedSize += 1 + columnType.defaultSize - } else { - val (smallEnough, _) = byteSizedDelta(value, prev) - _compressedSize += (if (smallEnough) 1 else 1 + columnType.defaultSize) + // If this is the first integer to be compressed, or the delta is out of byte range, then give + // up compressing this integer. + if (_uncompressedSize == 0 || delta <= Byte.MinValue || delta > Byte.MaxValue) { + _compressedSize += INT.defaultSize } - prev = value + _uncompressedSize += INT.defaultSize + prevValue = value } - override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[I]) = { + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { to.putInt(typeId) if (from.hasRemaining) { - var prev = columnType.extract(from) + var prev = from.getInt() to.put(Byte.MinValue) - columnType.append(prev, to) + to.putInt(prev) while (from.hasRemaining) { - val current = columnType.extract(from) - val (smallEnough, delta) = byteSizedDelta(current, prev) + val current = from.getInt() + val delta = current - prev prev = current - if (smallEnough) { - to.put(delta) + if (Byte.MinValue < delta && delta <= Byte.MaxValue) { + to.put(delta.toByte) } else { to.put(Byte.MinValue) - columnType.append(current, to) + to.putInt(current) } } } - to.rewind() - to + to.rewind().asInstanceOf[ByteBuffer] } - - override def uncompressedSize = _uncompressedSize - - override def compressedSize = _compressedSize } - class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[I]) - extends compression.Decoder[I] { + class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[IntegerType.type]) + extends compression.Decoder[IntegerType.type] { + + private var prev: Int = _ - private var prev: I#JvmType = _ + override def hasNext: Boolean = buffer.hasRemaining - override def next() = { + override def next(row: MutableRow, ordinal: Int): Unit = { val delta = buffer.get() - prev = if (delta > Byte.MinValue) addDelta(prev, delta) else columnType.extract(buffer) - prev + prev = if (delta > Byte.MinValue) prev + delta else buffer.getInt() + row.setInt(ordinal, prev) } - - override def hasNext = buffer.hasRemaining } } -private[sql] case object IntDelta extends IntegralDelta[IntegerType.type] { - override val typeId = 4 +private[sql] case object LongDelta extends CompressionScheme { + override def typeId: Int = 5 - override def supports(columnType: ColumnType[_, _]) = columnType == INT + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new Decoder(buffer, LONG).asInstanceOf[compression.Decoder[T]] + } + + override def encoder[T <: NativeType](columnType: NativeColumnType[T]) = { + (new Encoder).asInstanceOf[compression.Encoder[T]] + } - override protected def addDelta(x: Int, delta: Byte) = x + delta + override def supports(columnType: ColumnType[_, _]) = columnType == LONG + + class Encoder extends compression.Encoder[LongType.type] { + protected var _compressedSize: Int = 0 + protected var _uncompressedSize: Int = 0 + + override def compressedSize = _compressedSize + override def uncompressedSize = _uncompressedSize + + private var prevValue: Long = _ + + override def gatherCompressibilityStats(row: Row, ordinal: Int): Unit = { + val value = row.getLong(ordinal) + val delta = value - prevValue + + _compressedSize += 1 - override protected def byteSizedDelta(x: Int, y: Int): (Boolean, Byte) = { - val delta = x - y - if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + // If this is the first long integer to be compressed, or the delta is out of byte range, then + // give up compressing this long integer. + if (_uncompressedSize == 0 || delta <= Byte.MinValue || delta > Byte.MaxValue) { + _compressedSize += LONG.defaultSize + } + + _uncompressedSize += LONG.defaultSize + prevValue = value + } + + override def compress(from: ByteBuffer, to: ByteBuffer): ByteBuffer = { + to.putInt(typeId) + + if (from.hasRemaining) { + var prev = from.getLong() + to.put(Byte.MinValue) + to.putLong(prev) + + while (from.hasRemaining) { + val current = from.getLong() + val delta = current - prev + prev = current + + if (Byte.MinValue < delta && delta <= Byte.MaxValue) { + to.put(delta.toByte) + } else { + to.put(Byte.MinValue) + to.putLong(current) + } + } + } + + to.rewind().asInstanceOf[ByteBuffer] + } } -} -private[sql] case object LongDelta extends IntegralDelta[LongType.type] { - override val typeId = 5 + class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[LongType.type]) + extends compression.Decoder[LongType.type] { - override def supports(columnType: ColumnType[_, _]) = columnType == LONG + private var prev: Long = _ - override protected def addDelta(x: Long, delta: Byte) = x + delta + override def hasNext: Boolean = buffer.hasRemaining - override protected def byteSizedDelta(x: Long, y: Long): (Boolean, Byte) = { - val delta = x - y - if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + override def next(row: MutableRow, ordinal: Int): Unit = { + val delta = buffer.get() + prev = if (delta > Byte.MinValue) prev + delta else buffer.getLong() + row.setLong(ordinal, prev) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index cde91ceb68c98..0cdbb3167ce36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -35,7 +35,7 @@ class ColumnStatsSuite extends FunSuite { def testColumnStats[T <: NativeType, U <: ColumnStats]( columnStatsClass: Class[U], columnType: NativeColumnType[T], - initialStatistics: Row) { + initialStatistics: Row): Unit = { val columnStatsName = columnStatsClass.getSimpleName diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 75f653f3280bd..4fb1ecf1d532b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -23,6 +23,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -46,10 +47,12 @@ class ColumnTypeSuite extends FunSuite with Logging { def checkActualSize[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType], value: JvmType, - expected: Int) { + expected: Int): Unit = { assertResult(expected, s"Wrong actualSize for $columnType") { - columnType.actualSize(value) + val row = new GenericMutableRow(1) + columnType.setField(row, 0, value) + columnType.actualSize(row, 0) } } @@ -147,7 +150,7 @@ class ColumnTypeSuite extends FunSuite with Logging { def testNativeColumnType[T <: NativeType]( columnType: NativeColumnType[T], putter: (ByteBuffer, T#JvmType) => Unit, - getter: (ByteBuffer) => T#JvmType) { + getter: (ByteBuffer) => T#JvmType): Unit = { testColumnType[T, T#JvmType](columnType, putter, getter) } @@ -155,7 +158,7 @@ class ColumnTypeSuite extends FunSuite with Logging { def testColumnType[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType], putter: (ByteBuffer, JvmType) => Unit, - getter: (ByteBuffer) => JvmType) { + getter: (ByteBuffer) => JvmType): Unit = { val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE) val seq = (0 until 4).map(_ => makeRandomValue(columnType)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 0e3c67f5eed29..c1278248ef655 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.{SQLConf, QueryTest, TestData} +import org.apache.spark.sql.{QueryTest, TestData} class InMemoryColumnarQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 3baa6f8ec0c83..6c9a9ab6c3418 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -45,7 +45,9 @@ class NullableColumnAccessorSuite extends FunSuite { testNullableColumnAccessor(_) } - def testNullableColumnAccessor[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { + def testNullableColumnAccessor[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType]): Unit = { + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") val nullRow = makeNullRow(1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index a77262534a352..f54a21eb4fbb1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -41,7 +41,9 @@ class NullableColumnBuilderSuite extends FunSuite { testNullableColumnBuilder(_) } - def testNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { + def testNullableColumnBuilder[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType]): Unit = { + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") test(s"$typeName column builder: empty column") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 5d2fd4959197c..69e0adbd3ee0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -28,7 +28,7 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be val originalColumnBatchSize = columnBatchSize val originalInMemoryPartitionPruning = inMemoryPartitionPruning - override protected def beforeAll() { + override protected def beforeAll(): Unit = { // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch setConf(SQLConf.COLUMN_BATCH_SIZE, "10") val rawData = sparkContext.makeRDD(1 to 100, 5).map(IntegerData) @@ -38,7 +38,7 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") } - override protected def afterAll() { + override protected def afterAll(): Unit = { setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize.toString) setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning.toString) } @@ -76,7 +76,7 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be filter: String, expectedQueryResult: Seq[Int], expectedReadPartitions: Int, - expectedReadBatches: Int) { + expectedReadBatches: Int): Unit = { test(filter) { val query = sql(s"SELECT * FROM intData WHERE $filter") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala index e01cc8b4d20f2..d9e488e0ffd16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar.{NoopColumnStats, BOOLEAN} import org.apache.spark.sql.columnar.ColumnarTestUtils._ @@ -72,10 +73,14 @@ class BooleanBitSetSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) + val mutableRow = new GenericMutableRow(1) if (values.nonEmpty) { values.foreach { assert(decoder.hasNext) - assertResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value") { + decoder.next(mutableRow, 0) + mutableRow.getBoolean(0) + } } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index d2969d906c943..1cdb909146d57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ @@ -67,7 +68,7 @@ class DictionaryEncodingSuite extends FunSuite { val buffer = builder.build() val headerSize = CompressionScheme.columnHeaderSize(buffer) // 4 extra bytes for dictionary size - val dictionarySize = 4 + values.map(columnType.actualSize).sum + val dictionarySize = 4 + rows.map(columnType.actualSize(_, 0)).sum // 2 bytes for each `Short` val compressedSize = 4 + dictionarySize + 2 * inputSeq.length // 4 extra bytes for compression scheme type ID @@ -97,11 +98,15 @@ class DictionaryEncodingSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = DictionaryEncoding.decoder(buffer, columnType) + val mutableRow = new GenericMutableRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - assertResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 322f447c24840..73f31c0233343 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -31,7 +31,7 @@ class IntegralDeltaSuite extends FunSuite { def testIntegralDelta[I <: IntegralType]( columnStats: ColumnStats, columnType: NativeColumnType[I], - scheme: IntegralDelta[I]) { + scheme: CompressionScheme) { def skeleton(input: Seq[I#JvmType]) { // ------------- @@ -96,10 +96,15 @@ class IntegralDeltaSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = scheme.decoder(buffer, columnType) + val mutableRow = new GenericMutableRow(1) + if (input.nonEmpty) { input.foreach{ assert(decoder.hasNext) - assertResult(_, "Wrong decoded value")(decoder.next()) + assertResult(_, "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } } } assert(!decoder.hasNext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 218c09ac26362..4ce2552112c92 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.columnar.compression import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ @@ -57,7 +58,7 @@ class RunLengthEncodingSuite extends FunSuite { // Compression scheme ID + compressed contents val compressedSize = 4 + inputRuns.map { case (index, _) => // 4 extra bytes each run for run length - columnType.actualSize(values(index)) + 4 + columnType.actualSize(rows(index), 0) + 4 }.sum // 4 extra bytes for compression scheme type ID @@ -80,11 +81,15 @@ class RunLengthEncodingSuite extends FunSuite { buffer.rewind().position(headerSize + 4) val decoder = RunLengthEncoding.decoder(buffer, columnType) + val mutableRow = new GenericMutableRow(1) if (inputSeq.nonEmpty) { inputSeq.foreach { i => assert(decoder.hasNext) - assertResult(values(i), "Wrong decoded value")(decoder.next()) + assertResult(values(i), "Wrong decoded value") { + decoder.next(mutableRow, 0) + columnType.getField(mutableRow, 0) + } } } 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 329f80cad471e..84fafcde63d05 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 @@ -25,16 +25,14 @@ import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector - +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} - -import org.apache.spark.sql.catalyst.expressions.{Attribute, Row, GenericMutableRow, Literal, Cast} -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.expressions._ /** * A trait for subclasses that handle table scans. @@ -108,12 +106,12 @@ class HadoopTableReader( val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) val attrsWithIndex = attributes.zipWithIndex - val mutableRow = new GenericMutableRow(attrsWithIndex.length) + val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = deserializerClass.newInstance() deserializer.initialize(hconf, tableDesc.getProperties) - HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow) } @@ -164,33 +162,32 @@ class HadoopTableReader( val tableDesc = relation.tableDesc val broadcastedHiveConf = _broadcastedHiveConf val localDeserializer = partDeserializer - val mutableRow = new GenericMutableRow(attributes.length) - - // split the attributes (output schema) into 2 categories: - // (partition keys, ordinal), (normal attributes, ordinal), the ordinal mean the - // index of the attribute in the output Row. - val (partitionKeys, attrs) = attributes.zipWithIndex.partition(attr => { - relation.partitionKeys.indexOf(attr._1) >= 0 - }) - - def fillPartitionKeys(parts: Array[String], row: GenericMutableRow) = { - partitionKeys.foreach { case (attr, ordinal) => - // get partition key ordinal for a given attribute - val partOridinal = relation.partitionKeys.indexOf(attr) - row(ordinal) = Cast(Literal(parts(partOridinal)), attr.dataType).eval(null) + val mutableRow = new SpecificMutableRow(attributes.map(_.dataType)) + + // Splits all attributes into two groups, partition key attributes and those that are not. + // Attached indices indicate the position of each attribute in the output schema. + val (partitionKeyAttrs, nonPartitionKeyAttrs) = + attributes.zipWithIndex.partition { case (attr, _) => + relation.partitionKeys.contains(attr) + } + + def fillPartitionKeys(rawPartValues: Array[String], row: MutableRow) = { + partitionKeyAttrs.foreach { case (attr, ordinal) => + val partOrdinal = relation.partitionKeys.indexOf(attr) + row(ordinal) = Cast(Literal(rawPartValues(partOrdinal)), attr.dataType).eval(null) } } - // fill the partition key for the given MutableRow Object + + // Fill all partition keys to the given MutableRow object fillPartitionKeys(partValues, mutableRow) - val hivePartitionRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) - hivePartitionRDD.mapPartitions { iter => + createHadoopRdd(tableDesc, inputPathStr, ifc).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value val deserializer = localDeserializer.newInstance() deserializer.initialize(hconf, partProps) - // fill the non partition key attributes - HadoopTableReader.fillObject(iter, deserializer, attrs, mutableRow) + // fill the non partition key attributes + HadoopTableReader.fillObject(iter, deserializer, nonPartitionKeyAttrs, mutableRow) } }.toSeq @@ -257,38 +254,64 @@ private[hive] object HadoopTableReader extends HiveInspectors { } /** - * Transform the raw data(Writable object) into the Row object for an iterable input - * @param iter Iterable input which represented as Writable object - * @param deserializer Deserializer associated with the input writable object - * @param attrs Represents the row attribute names and its zero-based position in the MutableRow - * @param row reusable MutableRow object - * - * @return Iterable Row object that transformed from the given iterable input. + * Transform all given raw `Writable`s into `Row`s. + * + * @param iterator Iterator of all `Writable`s to be transformed + * @param deserializer The `Deserializer` associated with the input `Writable` + * @param nonPartitionKeyAttrs Attributes that should be filled together with their corresponding + * positions in the output schema + * @param mutableRow A reusable `MutableRow` that should be filled + * @return An `Iterator[Row]` transformed from `iterator` */ def fillObject( - iter: Iterator[Writable], + iterator: Iterator[Writable], deserializer: Deserializer, - attrs: Seq[(Attribute, Int)], - row: GenericMutableRow): Iterator[Row] = { + nonPartitionKeyAttrs: Seq[(Attribute, Int)], + mutableRow: MutableRow): Iterator[Row] = { + val soi = deserializer.getObjectInspector().asInstanceOf[StructObjectInspector] - // get the field references according to the attributes(output of the reader) required - val fieldRefs = attrs.map { case (attr, idx) => (soi.getStructFieldRef(attr.name), idx) } + val (fieldRefs, fieldOrdinals) = nonPartitionKeyAttrs.map { case (attr, ordinal) => + soi.getStructFieldRef(attr.name) -> ordinal + }.unzip + + // Builds specific unwrappers ahead of time according to object inspector types to avoid pattern + // matching and branching costs per row. + val unwrappers: Seq[(Any, MutableRow, Int) => Unit] = fieldRefs.map { + _.getFieldObjectInspector match { + case oi: BooleanObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setBoolean(ordinal, oi.get(value)) + case oi: ByteObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setByte(ordinal, oi.get(value)) + case oi: ShortObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setShort(ordinal, oi.get(value)) + case oi: IntObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setInt(ordinal, oi.get(value)) + case oi: LongObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setLong(ordinal, oi.get(value)) + case oi: FloatObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setFloat(ordinal, oi.get(value)) + case oi: DoubleObjectInspector => + (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) + case oi => + (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrapData(value, oi) + } + } // Map each tuple to a row object - iter.map { value => + iterator.map { value => val raw = deserializer.deserialize(value) - var idx = 0; - while (idx < fieldRefs.length) { - val fieldRef = fieldRefs(idx)._1 - val fieldIdx = fieldRefs(idx)._2 - val fieldValue = soi.getStructFieldData(raw, fieldRef) - - row(fieldIdx) = unwrapData(fieldValue, fieldRef.getFieldObjectInspector()) - - idx += 1 + var i = 0 + while (i < fieldRefs.length) { + val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) + if (fieldValue == null) { + mutableRow.setNullAt(fieldOrdinals(i)) + } else { + unwrappers(i)(fieldValue, mutableRow, fieldOrdinals(i)) + } + i += 1 } - row: Row + mutableRow: Row } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 6bf8d18a5c32c..8c8a8b124ac69 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -295,8 +295,16 @@ class HiveQuerySuite extends HiveComparisonTest { "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15") test("implement identity function using case statement") { - val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src").collect().toSet - val expected = sql("SELECT key FROM src").collect().toSet + val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src") + .map { case Row(i: Int) => i } + .collect() + .toSet + + val expected = sql("SELECT key FROM src") + .map { case Row(i: Int) => i } + .collect() + .toSet + assert(actual === expected) } @@ -559,9 +567,9 @@ class HiveQuerySuite extends HiveComparisonTest { val testVal = "test.val.0" val nonexistentKey = "nonexistent" val KV = "([^=]+)=([^=]*)".r - def collectResults(rdd: SchemaRDD): Set[(String, String)] = - rdd.collect().map { - case Row(key: String, value: String) => key -> value + def collectResults(rdd: SchemaRDD): Set[(String, String)] = + rdd.collect().map { + case Row(key: String, value: String) => key -> value case Row(KV(key, value)) => key -> value }.toSet clear() From 0f8c4edf4e750e3d11da27cc22c40b0489da7f37 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 13 Sep 2014 16:08:04 -0700 Subject: [PATCH 0687/1492] [SQL] Decrease partitions when testing Author: Michael Armbrust Closes #2164 from marmbrus/shufflePartitions and squashes the following commits: 0da1e8c [Michael Armbrust] test hax ef2d985 [Michael Armbrust] more test hacks. 2dabae3 [Michael Armbrust] more test fixes 0bdbf21 [Michael Armbrust] Make parquet tests less order dependent b42eeab [Michael Armbrust] increase test parallelism 80453d5 [Michael Armbrust] Decrease partitions when testing --- .../spark/sql/test/TestSQLContext.scala | 9 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 142 +++++------------- .../org/apache/spark/sql/hive/TestHive.scala | 7 +- 3 files changed, 51 insertions(+), 107 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index f2389f8f0591e..265b67737c475 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -18,8 +18,13 @@ package org.apache.spark.sql.test import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SQLConf, SQLContext} /** A SQLContext that can be used for local testing. */ object TestSQLContext - extends SQLContext(new SparkContext("local", "TestSQLContext", new SparkConf())) + extends SQLContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) { + + /** Fewer partitions to speed up testing. */ + override private[spark] def numShufflePartitions: Int = + getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt +} 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 b0a06cd3ca090..08f7358446b29 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 @@ -58,8 +58,7 @@ case class AllDataTypes( doubleField: Double, shortField: Short, byteField: Byte, - booleanField: Boolean, - binaryField: Array[Byte]) + booleanField: Boolean) case class AllDataTypesWithNonPrimitiveType( stringField: String, @@ -70,13 +69,14 @@ case class AllDataTypesWithNonPrimitiveType( shortField: Short, byteField: Byte, booleanField: Boolean, - binaryField: Array[Byte], array: Seq[Int], arrayContainsNull: Seq[Option[Int]], map: Map[Int, Long], mapValueContainsNull: Map[Int, Option[Long]], data: Data) +case class BinaryData(binaryData: Array[Byte]) + class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. @@ -108,26 +108,26 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Read/Write All Types") { 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, - (0 to x).map(_.toByte).toArray)) - .saveAsParquetFile(tempDir) - val result = parquetFile(tempDir).collect() - range.foreach { - i => - assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") - assert(result(i).getInt(1) === i) - assert(result(i).getLong(2) === i.toLong) - assert(result(i).getFloat(3) === i.toFloat) - assert(result(i).getDouble(4) === i.toDouble) - 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) - } + val data = sparkContext.parallelize(range) + .map(x => AllDataTypes(s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0)) + + data.saveAsParquetFile(tempDir) + + checkAnswer( + parquetFile(tempDir), + data.toSchemaRDD.collect().toSeq) } - test("Treat binary as string") { + test("read/write binary data") { + // Since equality for Array[Byte] is broken we test this separately. + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + sparkContext.parallelize(BinaryData("test".getBytes("utf8")) :: Nil).saveAsParquetFile(tempDir) + parquetFile(tempDir) + .map(r => new String(r(0).asInstanceOf[Array[Byte]], "utf8")) + .collect().toSeq == Seq("test") + } + + ignore("Treat binary as string") { val oldIsParquetBinaryAsString = TestSQLContext.isParquetBinaryAsString // Create the test file. @@ -142,37 +142,16 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA StructField("c2", BinaryType, false) :: Nil) val schemaRDD1 = applySchema(rowRDD, schema) schemaRDD1.saveAsParquetFile(path) - val resultWithBinary = parquetFile(path).collect - range.foreach { - i => - assert(resultWithBinary(i).getInt(0) === i) - assert(resultWithBinary(i)(1) === s"val_$i".getBytes) - } - - TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") - // This ParquetRelation always use Parquet types to derive output. - val parquetRelation = new ParquetRelation( - path.toString, - Some(TestSQLContext.sparkContext.hadoopConfiguration), - TestSQLContext) { - override val output = - ParquetTypesConverter.convertToAttributes( - ParquetTypesConverter.readMetaData(new Path(path), conf).getFileMetaData.getSchema, - TestSQLContext.isParquetBinaryAsString) - } - val schemaRDD = new SchemaRDD(TestSQLContext, parquetRelation) - val resultWithString = schemaRDD.collect - range.foreach { - i => - assert(resultWithString(i).getInt(0) === i) - assert(resultWithString(i)(1) === s"val_$i") - } + checkAnswer( + parquetFile(path).select('c1, 'c2.cast(StringType)), + schemaRDD1.select('c1, 'c2.cast(StringType)).collect().toSeq) - schemaRDD.registerTempTable("tmp") + setConf(SQLConf.PARQUET_BINARY_AS_STRING, "true") + parquetFile(path).printSchema() checkAnswer( - sql("SELECT c1, c2 FROM tmp WHERE c2 = 'val_5' OR c2 = 'val_7'"), - (5, "val_5") :: - (7, "val_7") :: Nil) + parquetFile(path), + schemaRDD1.select('c1, 'c2.cast(StringType)).collect().toSeq) + // Set it back. TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) @@ -275,34 +254,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) - TestSQLContext.sparkContext.parallelize(range) + val data = 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(Option(_).filter(_ % 3 == 0)), (0 until x).map(i => i -> i.toLong).toMap, (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), Data((0 until x), Nested(x, s"$x")))) - .saveAsParquetFile(tempDir) - val result = parquetFile(tempDir).collect() - range.foreach { - i => - assert(result(i).getString(0) == s"$i", s"row $i String field did not match, got ${result(i).getString(0)}") - assert(result(i).getInt(1) === i) - assert(result(i).getLong(2) === i.toLong) - assert(result(i).getFloat(3) === i.toFloat) - assert(result(i).getDouble(4) === i.toDouble) - 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) - assert(result(i)(9) === (0 until i)) - assert(result(i)(10) === (0 until i).map(i => if (i % 3 == 0) i else null)) - assert(result(i)(11) === (0 until i).map(i => i -> i.toLong).toMap) - assert(result(i)(12) === (0 until i).map(i => i -> i.toLong).toMap + (i -> null)) - assert(result(i)(13) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) - } + data.saveAsParquetFile(tempDir) + + checkAnswer( + parquetFile(tempDir), + data.toSchemaRDD.collect().toSeq) } test("self-join parquet files") { @@ -399,23 +363,6 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } } - test("Saving case class RDD table to file and reading it back in") { - val file = getTempFilePath("parquet") - val path = file.toString - val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) - .map(i => TestRDDEntry(i, s"val_$i")) - rdd.saveAsParquetFile(path) - val readFile = parquetFile(path) - readFile.registerTempTable("tmpx") - val rdd_copy = sql("SELECT * FROM tmpx").collect() - val rdd_orig = rdd.collect() - for(i <- 0 to 99) { - assert(rdd_copy(i).apply(0) === rdd_orig(i).key, s"key error in line $i") - assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value error in line $i") - } - Utils.deleteRecursively(file) - } - test("Read a parquet file instead of a directory") { val file = getTempFilePath("parquet") val path = file.toString @@ -448,32 +395,19 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() assert(rdd_copy1.size === 100) - assert(rdd_copy1(0).apply(0) === 1) - assert(rdd_copy1(0).apply(1) === "val_1") - // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is - // executed twice otherwise?! + sql("INSERT INTO dest SELECT * FROM source") - val rdd_copy2 = sql("SELECT * FROM dest").collect() + val rdd_copy2 = sql("SELECT * FROM dest").collect().sortBy(_.getInt(0)) assert(rdd_copy2.size === 200) - assert(rdd_copy2(0).apply(0) === 1) - assert(rdd_copy2(0).apply(1) === "val_1") - assert(rdd_copy2(99).apply(0) === 100) - assert(rdd_copy2(99).apply(1) === "val_100") - assert(rdd_copy2(100).apply(0) === 1) - assert(rdd_copy2(100).apply(1) === "val_1") Utils.deleteRecursively(dirname) } test("Insert (appending) to same table via Scala API") { - // TODO: why does collecting break things? It seems InsertIntoParquet::execute() is - // executed twice otherwise?! sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) assert(double_rdd.size === 30) - for(i <- (0 to 14)) { - assert(double_rdd(i) === double_rdd(i+15), s"error: lines $i and ${i+15} to not match") - } + // let's restore the original test data Utils.deleteRecursively(ParquetTestData.testDir) ParquetTestData.writeFile() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a3bfd3a8f1fd2..70fb15259e7d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -35,12 +35,13 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{CacheCommand, LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.hive._ +import org.apache.spark.sql.SQLConf /* Implicit conversions */ import scala.collection.JavaConversions._ object TestHive - extends TestHiveContext(new SparkContext("local", "TestSQLContext", new SparkConf())) + extends TestHiveContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) /** * A locally running test instance of Spark's Hive execution engine. @@ -90,6 +91,10 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } + /** Fewer partitions to speed up testing. */ + override private[spark] def numShufflePartitions: Int = + getConf(SQLConf.SHUFFLE_PARTITIONS, "5").toInt + /** * Returns the value of specified environmental variable as a [[java.io.File]] after checking * to ensure it exists From 2aea0da84c58a179917311290083456dfa043db7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 13 Sep 2014 16:22:04 -0700 Subject: [PATCH 0688/1492] [SPARK-3030] [PySpark] Reuse Python worker Reuse Python worker to avoid the overhead of fork() Python process for each tasks. It also tracks the broadcasts for each worker, avoid sending repeated broadcasts. This can reduce the time for dummy task from 22ms to 13ms (-40%). It can help to reduce the latency for Spark Streaming. For a job with broadcast (43M after compress): ``` b = sc.broadcast(set(range(30000000))) print sc.parallelize(range(24000), 100).filter(lambda x: x in b.value).count() ``` It will finish in 281s without reused worker, and it will finish in 65s with reused worker(4 CPUs). After reusing the worker, it can save about 9 seconds for transfer and deserialize the broadcast for each tasks. It's enabled by default, could be disabled by `spark.python.worker.reuse = false`. Author: Davies Liu Closes #2259 from davies/reuse-worker and squashes the following commits: f11f617 [Davies Liu] Merge branch 'master' into reuse-worker 3939f20 [Davies Liu] fix bug in serializer in mllib cf1c55e [Davies Liu] address comments 3133a60 [Davies Liu] fix accumulator with reused worker 760ab1f [Davies Liu] do not reuse worker if there are any exceptions 7abb224 [Davies Liu] refactor: sychronized with itself ac3206e [Davies Liu] renaming 8911f44 [Davies Liu] synchronized getWorkerBroadcasts() 6325fc1 [Davies Liu] bugfix: bid >= 0 e0131a2 [Davies Liu] fix name of config 583716e [Davies Liu] only reuse completed and not interrupted worker ace2917 [Davies Liu] kill python worker after timeout 6123d0f [Davies Liu] track broadcasts for each worker 8d2f08c [Davies Liu] reuse python worker --- .../scala/org/apache/spark/SparkEnv.scala | 8 ++ .../apache/spark/api/python/PythonRDD.scala | 58 ++++++++++--- .../api/python/PythonWorkerFactory.scala | 85 ++++++++++++++++--- docs/configuration.md | 10 +++ python/pyspark/daemon.py | 38 ++++----- python/pyspark/mllib/_common.py | 12 ++- python/pyspark/serializers.py | 4 + python/pyspark/tests.py | 35 ++++++++ python/pyspark/worker.py | 9 +- 9 files changed, 208 insertions(+), 51 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index dd95e406f2a8e..009ed64775844 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -108,6 +108,14 @@ class SparkEnv ( pythonWorkers.get(key).foreach(_.stopWorker(worker)) } } + + private[spark] + def releasePythonWorker(pythonExec: String, envVars: Map[String, String], worker: Socket) { + synchronized { + val key = (pythonExec, envVars) + pythonWorkers.get(key).foreach(_.releaseWorker(worker)) + } + } } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ae8010300a500..ca8eef5f99edf 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -23,6 +23,7 @@ import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ +import scala.collection.mutable import scala.language.existentials import scala.reflect.ClassTag import scala.util.{Try, Success, Failure} @@ -52,6 +53,7 @@ private[spark] class PythonRDD( extends RDD[Array[Byte]](parent) { val bufferSize = conf.getInt("spark.buffer.size", 65536) + val reuse_worker = conf.getBoolean("spark.python.worker.reuse", true) override def getPartitions = parent.partitions @@ -63,19 +65,26 @@ private[spark] class PythonRDD( val localdir = env.blockManager.diskBlockManager.localDirs.map( f => f.getPath()).mkString(",") envVars += ("SPARK_LOCAL_DIRS" -> localdir) // it's also used in monitor thread + if (reuse_worker) { + envVars += ("SPARK_REUSE_WORKER" -> "1") + } val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap) // Start a thread to feed the process input from our parent's iterator val writerThread = new WriterThread(env, worker, split, context) + var complete_cleanly = false context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() - - // Cleanup the worker socket. This will also cause the Python worker to exit. - try { - worker.close() - } catch { - case e: Exception => logWarning("Failed to close worker socket", e) + if (reuse_worker && complete_cleanly) { + env.releasePythonWorker(pythonExec, envVars.toMap, worker) + } else { + try { + worker.close() + } catch { + case e: Exception => + logWarning("Failed to close worker socket", e) + } } } @@ -133,6 +142,7 @@ private[spark] class PythonRDD( stream.readFully(update) accumulator += Collections.singletonList(update) } + complete_cleanly = true null } } catch { @@ -195,11 +205,26 @@ private[spark] class PythonRDD( PythonRDD.writeUTF(include, dataOut) } // Broadcast variables - dataOut.writeInt(broadcastVars.length) + val oldBids = PythonRDD.getWorkerBroadcasts(worker) + val newBids = broadcastVars.map(_.id).toSet + // number of different broadcasts + val cnt = oldBids.diff(newBids).size + newBids.diff(oldBids).size + dataOut.writeInt(cnt) + for (bid <- oldBids) { + if (!newBids.contains(bid)) { + // remove the broadcast from worker + dataOut.writeLong(- bid - 1) // bid >= 0 + oldBids.remove(bid) + } + } for (broadcast <- broadcastVars) { - dataOut.writeLong(broadcast.id) - dataOut.writeInt(broadcast.value.length) - dataOut.write(broadcast.value) + if (!oldBids.contains(broadcast.id)) { + // send new broadcast + dataOut.writeLong(broadcast.id) + dataOut.writeInt(broadcast.value.length) + dataOut.write(broadcast.value) + oldBids.add(broadcast.id) + } } dataOut.flush() // Serialized command: @@ -207,17 +232,18 @@ private[spark] class PythonRDD( dataOut.write(command) // Data values PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) + dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) dataOut.flush() } catch { case e: Exception if context.isCompleted || context.isInterrupted => logDebug("Exception thrown after task completion (likely due to cleanup)", e) + worker.shutdownOutput() case e: Exception => // We must avoid throwing exceptions here, because the thread uncaught exception handler // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e - } finally { - Try(worker.shutdownOutput()) // kill Python worker process + worker.shutdownOutput() } } } @@ -278,6 +304,14 @@ private object SpecialLengths { private[spark] object PythonRDD extends Logging { val UTF8 = Charset.forName("UTF-8") + // remember the broadcasts sent to each worker + private val workerBroadcasts = new mutable.WeakHashMap[Socket, mutable.Set[Long]]() + private def getWorkerBroadcasts(worker: Socket) = { + synchronized { + workerBroadcasts.getOrElseUpdate(worker, new mutable.HashSet[Long]()) + } + } + /** * Adapter for calling SparkContext#runJob from Python. * diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 4c4796f6c59ba..71bdf0fe1b917 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -40,7 +40,10 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) var daemonPort: Int = 0 - var daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + val daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + val idleWorkers = new mutable.Queue[Socket]() + var lastActivity = 0L + new MonitorThread().start() var simpleWorkers = new mutable.WeakHashMap[Socket, Process]() @@ -51,6 +54,11 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String def create(): Socket = { if (useDaemon) { + synchronized { + if (idleWorkers.size > 0) { + return idleWorkers.dequeue() + } + } createThroughDaemon() } else { createSimpleWorker() @@ -199,9 +207,44 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } } + /** + * Monitor all the idle workers, kill them after timeout. + */ + private class MonitorThread extends Thread(s"Idle Worker Monitor for $pythonExec") { + + setDaemon(true) + + override def run() { + while (true) { + synchronized { + if (lastActivity + IDLE_WORKER_TIMEOUT_MS < System.currentTimeMillis()) { + cleanupIdleWorkers() + lastActivity = System.currentTimeMillis() + } + } + Thread.sleep(10000) + } + } + } + + private def cleanupIdleWorkers() { + while (idleWorkers.length > 0) { + val worker = idleWorkers.dequeue() + try { + // the worker will exit after closing the socket + worker.close() + } catch { + case e: Exception => + logWarning("Failed to close worker socket", e) + } + } + } + private def stopDaemon() { synchronized { if (useDaemon) { + cleanupIdleWorkers() + // Request shutdown of existing daemon by sending SIGTERM if (daemon != null) { daemon.destroy() @@ -220,23 +263,43 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } def stopWorker(worker: Socket) { - if (useDaemon) { - if (daemon != null) { - daemonWorkers.get(worker).foreach { pid => - // tell daemon to kill worker by pid - val output = new DataOutputStream(daemon.getOutputStream) - output.writeInt(pid) - output.flush() - daemon.getOutputStream.flush() + synchronized { + if (useDaemon) { + if (daemon != null) { + daemonWorkers.get(worker).foreach { pid => + // tell daemon to kill worker by pid + val output = new DataOutputStream(daemon.getOutputStream) + output.writeInt(pid) + output.flush() + daemon.getOutputStream.flush() + } } + } else { + simpleWorkers.get(worker).foreach(_.destroy()) } - } else { - simpleWorkers.get(worker).foreach(_.destroy()) } worker.close() } + + def releaseWorker(worker: Socket) { + if (useDaemon) { + synchronized { + lastActivity = System.currentTimeMillis() + idleWorkers.enqueue(worker) + } + } else { + // Cleanup the worker socket. This will also cause the Python worker to exit. + try { + worker.close() + } catch { + case e: Exception => + logWarning("Failed to close worker socket", e) + } + } + } } private object PythonWorkerFactory { val PROCESS_WAIT_TIMEOUT_MS = 10000 + val IDLE_WORKER_TIMEOUT_MS = 60000 // kill idle workers after 1 minute } diff --git a/docs/configuration.md b/docs/configuration.md index 36178efb97103..af16489a44281 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -206,6 +206,16 @@ Apart from these, the following properties are also available, and may be useful used during aggregation goes above this amount, it will spill the data into disks. + + spark.python.worker.reuse + true + + Reuse Python worker or not. If yes, it will use a fixed number of Python workers, + does not need to fork() a Python process for every tasks. It will be very useful + if there is large broadcast, then the broadcast will not be needed to transfered + from JVM to Python worker for every task. + + spark.executorEnv.[EnvironmentVariableName] (none) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 15445abf67147..64d6202acb27d 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -23,6 +23,7 @@ import sys import traceback import time +import gc from errno import EINTR, ECHILD, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN @@ -46,17 +47,6 @@ def worker(sock): signal.signal(SIGCHLD, SIG_DFL) signal.signal(SIGTERM, SIG_DFL) - # Blocks until the socket is closed by draining the input stream - # until it raises an exception or returns EOF. - def waitSocketClose(sock): - try: - while True: - # Empty string is returned upon EOF (and only then). - if sock.recv(4096) == '': - return - except: - pass - # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because # otherwise writes also cause a seek that makes us miss data on the read side. @@ -64,17 +54,13 @@ def waitSocketClose(sock): outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) exit_code = 0 try: - # Acknowledge that the fork was successful - write_int(os.getpid(), outfile) - outfile.flush() worker_main(infile, outfile) except SystemExit as exc: - exit_code = exc.code + exit_code = compute_real_exit_code(exc.code) finally: outfile.flush() - # The Scala side will close the socket upon task completion. - waitSocketClose(sock) - os._exit(compute_real_exit_code(exit_code)) + if exit_code: + os._exit(exit_code) # Cleanup zombie children @@ -111,6 +97,8 @@ def handle_sigterm(*args): signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP + reuse = os.environ.get("SPARK_REUSE_WORKER") + # Initialization complete try: while True: @@ -163,7 +151,19 @@ def handle_sigterm(*args): # in child process listen_sock.close() try: - worker(sock) + # Acknowledge that the fork was successful + outfile = sock.makefile("w") + write_int(os.getpid(), outfile) + outfile.flush() + outfile.close() + while True: + worker(sock) + if not reuse: + # wait for closing + while sock.recv(1024): + pass + break + gc.collect() except: traceback.print_exc() os._exit(1) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index bb60d3d0c8463..68f6033616726 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -21,7 +21,7 @@ from numpy import ndarray, float64, int64, int32, array_equal, array from pyspark import SparkContext, RDD from pyspark.mllib.linalg import SparseVector -from pyspark.serializers import Serializer +from pyspark.serializers import FramedSerializer """ @@ -451,18 +451,16 @@ def _serialize_rating(r): return ba -class RatingDeserializer(Serializer): +class RatingDeserializer(FramedSerializer): - def loads(self, stream): - length = struct.unpack("!i", stream.read(4))[0] - ba = stream.read(length) - res = ndarray(shape=(3, ), buffer=ba, dtype=float64, offset=4) + def loads(self, string): + res = ndarray(shape=(3, ), buffer=string, dtype=float64, offset=4) return int(res[0]), int(res[1]), res[2] def load_stream(self, stream): while True: try: - yield self.loads(stream) + yield self._read_with_length(stream) except struct.error: return except EOFError: diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index a5f9341e819a9..ec3c6f055441d 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -144,6 +144,8 @@ def _write_with_length(self, obj, stream): def _read_with_length(self, stream): length = read_int(stream) + if length == SpecialLengths.END_OF_DATA_SECTION: + raise EOFError obj = stream.read(length) if obj == "": raise EOFError @@ -438,6 +440,8 @@ def __init__(self, use_unicode=False): def loads(self, stream): length = read_int(stream) + if length == SpecialLengths.END_OF_DATA_SECTION: + raise EOFError s = stream.read(length) return s.decode("utf-8") if self.use_unicode else s diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b687d695b01c4..747cd1767de7b 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1222,11 +1222,46 @@ def run(): except OSError: self.fail("daemon had been killed") + # run a normal job + rdd = self.sc.parallelize(range(100), 1) + self.assertEqual(100, rdd.map(str).count()) + def test_fd_leak(self): N = 1100 # fd limit is 1024 by default rdd = self.sc.parallelize(range(N), N) self.assertEquals(N, rdd.count()) + def test_after_exception(self): + def raise_exception(_): + raise Exception() + rdd = self.sc.parallelize(range(100), 1) + self.assertRaises(Exception, lambda: rdd.foreach(raise_exception)) + self.assertEqual(100, rdd.map(str).count()) + + def test_after_jvm_exception(self): + tempFile = tempfile.NamedTemporaryFile(delete=False) + tempFile.write("Hello World!") + tempFile.close() + data = self.sc.textFile(tempFile.name, 1) + filtered_data = data.filter(lambda x: True) + self.assertEqual(1, filtered_data.count()) + os.unlink(tempFile.name) + self.assertRaises(Exception, lambda: filtered_data.count()) + + rdd = self.sc.parallelize(range(100), 1) + self.assertEqual(100, rdd.map(str).count()) + + def test_accumulator_when_reuse_worker(self): + from pyspark.accumulators import INT_ACCUMULATOR_PARAM + acc1 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM) + self.sc.parallelize(range(100), 20).foreach(lambda x: acc1.add(x)) + self.assertEqual(sum(range(100)), acc1.value) + + acc2 = self.sc.accumulator(0, INT_ACCUMULATOR_PARAM) + self.sc.parallelize(range(100), 20).foreach(lambda x: acc2.add(x)) + self.assertEqual(sum(range(100)), acc2.value) + self.assertEqual(sum(range(100)), acc1.value) + class TestSparkSubmit(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 6805063e06798..61b8a74d060e8 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -69,9 +69,14 @@ def main(infile, outfile): ser = CompressedSerializer(pickleSer) for _ in range(num_broadcast_variables): bid = read_long(infile) - value = ser._read_with_length(infile) - _broadcastRegistry[bid] = Broadcast(bid, value) + if bid >= 0: + value = ser._read_with_length(infile) + _broadcastRegistry[bid] = Broadcast(bid, value) + else: + bid = - bid - 1 + _broadcastRegistry.remove(bid) + _accumulatorRegistry.clear() command = pickleSer._read_with_length(infile) (func, deserializer, serializer) = command init_time = time.time() From 4e3fbe8cdb6c6291e219195abb272f3c81f0ed63 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 13 Sep 2014 22:31:21 -0700 Subject: [PATCH 0689/1492] [SPARK-3463] [PySpark] aggregate and show spilled bytes in Python Aggregate the number of bytes spilled into disks during aggregation or sorting, show them in Web UI. ![spilled](https://cloud.githubusercontent.com/assets/40902/4209758/4b995562-386d-11e4-97c1-8e838ee1d4e3.png) This patch is blocked by SPARK-3465. (It includes a fix for that). Author: Davies Liu Closes #2336 from davies/metrics and squashes the following commits: e37df38 [Davies Liu] remove outdated comments 1245eb7 [Davies Liu] remove the temporary fix ebd2f43 [Davies Liu] Merge branch 'master' into metrics 7e4ad04 [Davies Liu] Merge branch 'master' into metrics fbe9029 [Davies Liu] show spilled bytes in Python in web ui --- .../apache/spark/api/python/PythonRDD.scala | 4 ++++ python/pyspark/shuffle.py | 19 ++++++++++++++++--- python/pyspark/tests.py | 15 ++++++++------- python/pyspark/worker.py | 14 ++++++++++---- 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ca8eef5f99edf..d5002fa02992b 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -124,6 +124,10 @@ private[spark] class PythonRDD( val total = finishTime - startTime logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, init, finish)) + val memoryBytesSpilled = stream.readLong() + val diskBytesSpilled = stream.readLong() + context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled + context.taskMetrics.diskBytesSpilled += diskBytesSpilled read() case SpecialLengths.PYTHON_EXCEPTION_THROWN => // Signals that an exception has been thrown in python diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 49829f5280a5f..ce597cbe91e15 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -68,6 +68,11 @@ def _get_local_dirs(sub): return [os.path.join(d, "python", str(os.getpid()), sub) for d in dirs] +# global stats +MemoryBytesSpilled = 0L +DiskBytesSpilled = 0L + + class Aggregator(object): """ @@ -313,10 +318,12 @@ def _spill(self): It will dump the data in batch for better performance. """ + global MemoryBytesSpilled, DiskBytesSpilled path = self._get_spill_dir(self.spills) if not os.path.exists(path): os.makedirs(path) + used_memory = get_used_memory() if not self.pdata: # The data has not been partitioned, it will iterator the # dataset once, write them into different files, has no @@ -334,6 +341,7 @@ def _spill(self): self.serializer.dump_stream([(k, v)], streams[h]) for s in streams: + DiskBytesSpilled += s.tell() s.close() self.data.clear() @@ -346,9 +354,11 @@ def _spill(self): # dump items in batch self.serializer.dump_stream(self.pdata[i].iteritems(), f) self.pdata[i].clear() + DiskBytesSpilled += os.path.getsize(p) self.spills += 1 gc.collect() # release the memory as much as possible + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 def iteritems(self): """ Return all merged items as iterator """ @@ -462,7 +472,6 @@ def __init__(self, memory_limit, serializer=None): self.memory_limit = memory_limit self.local_dirs = _get_local_dirs("sort") self.serializer = serializer or BatchedSerializer(PickleSerializer(), 1024) - self._spilled_bytes = 0 def _get_path(self, n): """ Choose one directory for spill by number n """ @@ -476,6 +485,7 @@ def sorted(self, iterator, key=None, reverse=False): Sort the elements in iterator, do external sort when the memory goes above the limit. """ + global MemoryBytesSpilled, DiskBytesSpilled batch = 10 chunks, current_chunk = [], [] iterator = iter(iterator) @@ -486,15 +496,18 @@ def sorted(self, iterator, key=None, reverse=False): if len(chunk) < batch: break - if get_used_memory() > self.memory_limit: + used_memory = get_used_memory() + if used_memory > self.memory_limit: # sort them inplace will save memory current_chunk.sort(key=key, reverse=reverse) path = self._get_path(len(chunks)) with open(path, 'w') as f: self.serializer.dump_stream(current_chunk, f) - self._spilled_bytes += os.path.getsize(path) chunks.append(self.serializer.load_stream(open(path))) current_chunk = [] + gc.collect() + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + DiskBytesSpilled += os.path.getsize(path) elif not chunks: batch = min(batch * 2, 10000) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 747cd1767de7b..f3309a20fcffb 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -46,6 +46,7 @@ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType +from pyspark import shuffle _have_scipy = False _have_numpy = False @@ -138,17 +139,17 @@ def test_external_sort(self): random.shuffle(l) sorter = ExternalSorter(1) self.assertEquals(sorted(l), list(sorter.sorted(l))) - self.assertGreater(sorter._spilled_bytes, 0) - last = sorter._spilled_bytes + self.assertGreater(shuffle.DiskBytesSpilled, 0) + last = shuffle.DiskBytesSpilled self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) - self.assertGreater(sorter._spilled_bytes, last) - last = sorter._spilled_bytes + self.assertGreater(shuffle.DiskBytesSpilled, last) + last = shuffle.DiskBytesSpilled self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) - self.assertGreater(sorter._spilled_bytes, last) - last = sorter._spilled_bytes + self.assertGreater(shuffle.DiskBytesSpilled, last) + last = shuffle.DiskBytesSpilled self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), list(sorter.sorted(l, key=lambda x: -x, reverse=True))) - self.assertGreater(sorter._spilled_bytes, last) + self.assertGreater(shuffle.DiskBytesSpilled, last) def test_external_sort_in_rdd(self): conf = SparkConf().set("spark.python.worker.memory", "1m") diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 61b8a74d060e8..252176ac65fec 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -23,16 +23,14 @@ import time import socket import traceback -# CloudPickler needs to be imported so that depicklers are registered using the -# copy_reg module. + from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry -from pyspark.cloudpickle import CloudPickler from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, write_int, read_long, \ write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ CompressedSerializer - +from pyspark import shuffle pickleSer = PickleSerializer() utf8_deserializer = UTF8Deserializer() @@ -52,6 +50,11 @@ def main(infile, outfile): if split_index == -1: # for unit tests return + # initialize global state + shuffle.MemoryBytesSpilled = 0 + shuffle.DiskBytesSpilled = 0 + _accumulatorRegistry.clear() + # fetch name of workdir spark_files_dir = utf8_deserializer.loads(infile) SparkFiles._root_directory = spark_files_dir @@ -97,6 +100,9 @@ def main(infile, outfile): exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) + write_long(shuffle.MemoryBytesSpilled, outfile) + write_long(shuffle.DiskBytesSpilled, outfile) + # Mark the beginning of the accumulators section of the output write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) write_int(len(_accumulatorRegistry), outfile) From c243b21a8ba2610266702e00d7d4b5443cb1f687 Mon Sep 17 00:00:00 2001 From: Bertrand Bossy Date: Sun, 14 Sep 2014 21:10:17 -0700 Subject: [PATCH 0690/1492] SPARK-3039: Allow spark to be built using avro-mapred for hadoop2 SPARK-3039: Adds the maven property "avro.mapred.classifier" to build spark-assembly with avro-mapred with support for the new Hadoop API. Sets this property to hadoop2 for Hadoop 2 profiles. I am not very familiar with maven, nor do I know whether this potentially breaks something in the hive part of spark. There might be a more elegant way of doing this. Author: Bertrand Bossy Closes #1945 from bbossy/SPARK-3039 and squashes the following commits: c32ce59 [Bertrand Bossy] SPARK-3039: Allow spark to be built using avro-mapred for hadoop2 --- pom.xml | 5 +++++ sql/hive/pom.xml | 9 +++++++++ 2 files changed, 14 insertions(+) diff --git a/pom.xml b/pom.xml index 28763476f8313..520aed3806937 100644 --- a/pom.xml +++ b/pom.xml @@ -134,6 +134,7 @@ 0.3.6 3.0.0 1.7.6 + 0.7.1 1.8.3 1.1.0 @@ -621,6 +622,7 @@ org.apache.avro avro-mapred ${avro.version} + ${avro.mapred.classifier} io.netty @@ -1108,6 +1110,7 @@ 2.2.0 2.5.0 + hadoop2 @@ -1117,6 +1120,7 @@ 2.3.0 2.5.0 0.9.0 + hadoop2 @@ -1126,6 +1130,7 @@ 2.4.0 2.5.0 0.9.0 + hadoop2 diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 45a4c6dc98da0..9d7a02bf7b0b7 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -95,6 +95,15 @@ org.apache.avro avro + ${avro.version} + + + + org.apache.avro + avro-mapred + ${avro.version} + ${avro.mapred.classifier} org.scalatest From f493f7982b50e3c99e78b649e7c6c5b4313c5ffa Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 14 Sep 2014 21:17:29 -0700 Subject: [PATCH 0691/1492] [SPARK-3452] Maven build should skip publishing artifacts people shouldn... ...'t depend on Publish local in maven term is `install` and publish otherwise is `deploy` So disabled both for following projects. Author: Prashant Sharma Closes #2329 from ScrapCodes/SPARK-3452/maven-skip-install and squashes the following commits: 257b79a [Prashant Sharma] [SPARK-3452] Maven build should skip publishing artifacts people shouldn't depend on --- assembly/pom.xml | 14 ++++++++++++++ examples/pom.xml | 14 ++++++++++++++ extras/java8-tests/pom.xml | 14 ++++++++++++++ repl/pom.xml | 14 ++++++++++++++ tools/pom.xml | 14 ++++++++++++++ yarn/pom.xml | 14 ++++++++++++++ 6 files changed, 84 insertions(+) diff --git a/assembly/pom.xml b/assembly/pom.xml index 4146168fc804b..604b1ab3de6a8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -88,6 +88,20 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins diff --git a/examples/pom.xml b/examples/pom.xml index 3f46c40464d3b..2b561857f9f33 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -203,6 +203,20 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins maven-shade-plugin diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml index 8658ecf5abfab..7e478bed62da7 100644 --- a/extras/java8-tests/pom.xml +++ b/extras/java8-tests/pom.xml @@ -74,6 +74,20 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins maven-surefire-plugin diff --git a/repl/pom.xml b/repl/pom.xml index fcc5f90d870e8..af528c8914335 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -99,6 +99,20 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.scalatest scalatest-maven-plugin diff --git a/tools/pom.xml b/tools/pom.xml index f36674476770c..b90eb0ca250c5 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -63,6 +63,20 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.apache.maven.plugins maven-source-plugin diff --git a/yarn/pom.xml b/yarn/pom.xml index 7fcd7ee0d4547..815a736c2e8fd 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -88,6 +88,20 @@ + + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-install-plugin + + true + + org.codehaus.mojo build-helper-maven-plugin From cc14644460872efb344e8d895859d70213a40840 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 15 Sep 2014 08:53:58 -0500 Subject: [PATCH 0692/1492] [SPARK-3410] The priority of shutdownhook for ApplicationMaster should not be integer literal I think, it need to keep the priority of shutdown hook for ApplicationMaster than the priority of shutdown hook for o.a.h.FileSystem depending on changing the priority for FileSystem. Author: Kousuke Saruta Closes #2283 from sarutak/SPARK-3410 and squashes the following commits: 1d44fef [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3410 bd6cc53 [Kousuke Saruta] Modified style ee6f1aa [Kousuke Saruta] Added constant "SHUTDOWN_HOOK_PRIORITY" to ApplicationMaster 54eb68f [Kousuke Saruta] Changed Shutdown hook priority to 20 2f0aee3 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3410 4c5cb93 [Kousuke Saruta] Modified the priority for AM's shutdown hook 217d1a4 [Kousuke Saruta] Removed unused import statements 717aba2 [Kousuke Saruta] Modified ApplicationMaster to make to keep the priority of shutdown hook for ApplicationMaster higher than the priority of shutdown hook for HDFS --- .../spark/deploy/yarn/ApplicationMaster.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 735d7723b0ce6..cde5fff637a39 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -21,12 +21,8 @@ import java.io.IOException import java.net.Socket import java.util.concurrent.atomic.AtomicReference -import scala.collection.JavaConversions._ -import scala.util.Try - import akka.actor._ import akka.remote._ -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ @@ -107,8 +103,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } } - // Use priority 30 as it's higher than HDFS. It's the same priority MapReduce is using. - ShutdownHookManager.get().addShutdownHook(cleanupHook, 30) + + // Use higher priority than FileSystem. + assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) + ShutdownHookManager + .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) // Call this to force generation of secret so it gets populated into the // Hadoop UGI. This has to happen before the startUserClass which does a @@ -407,6 +406,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, object ApplicationMaster extends Logging { + val SHUTDOWN_HOOK_PRIORITY: Int = 30 + private var master: ApplicationMaster = _ def main(args: Array[String]) = { From fe2b1d6a209db9fe96b1c6630677955b94bd48c9 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Mon, 15 Sep 2014 10:57:53 -0700 Subject: [PATCH 0693/1492] [SPARK-3425] do not set MaxPermSize for OpenJDK 1.8 Closes #2387 Author: Matthew Farrellee Closes #2301 from mattf/SPARK-3425 and squashes the following commits: 20f3c09 [Matthew Farrellee] [SPARK-3425] do not set MaxPermSize for OpenJDK 1.8 --- bin/spark-class | 2 +- dev/merge_spark_pr.py | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index 5f5f9ea74888d..613dc9c4566f2 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -105,7 +105,7 @@ else exit 1 fi fi -JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') +JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size if [ "$JAVA_VERSION" -ge 18 ]; then diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index d48c8bde12905..a8e92e36fe0d8 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -44,9 +44,9 @@ # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") # ASF JIRA username -JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") # ASF JIRA password -JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "") +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "35500") GITHUB_BASE = "https://github.com/apache/spark/pull" GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" From e59fac1f97c3fbeeb6defd12625a49763a353156 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 15 Sep 2014 16:11:41 -0700 Subject: [PATCH 0694/1492] [SPARK-3518] Remove wasted statement in JsonProtocol Author: Kousuke Saruta Closes #2380 from sarutak/SPARK-3518 and squashes the following commits: 8a1464e [Kousuke Saruta] Replaced a variable with simple field reference c660fbc [Kousuke Saruta] Removed useless statement in JsonProtocol.scala --- core/src/main/scala/org/apache/spark/util/JsonProtocol.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index b0754e3ce10db..c4dddb2d1037e 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -205,7 +205,6 @@ private[spark] object JsonProtocol { } def taskInfoToJson(taskInfo: TaskInfo): JValue = { - val accumUpdateMap = taskInfo.accumulables ("Task ID" -> taskInfo.taskId) ~ ("Index" -> taskInfo.index) ~ ("Attempt" -> taskInfo.attempt) ~ From 37d925280cdfdda8f6f7174c67a614056eea5d69 Mon Sep 17 00:00:00 2001 From: yantangzhai Date: Mon, 15 Sep 2014 16:57:38 -0700 Subject: [PATCH 0695/1492] [SPARK-2714] DAGScheduler logs jobid when runJob finishes DAGScheduler logs jobid when runJob finishes Author: yantangzhai Closes #1617 from YanTangZhai/SPARK-2714 and squashes the following commits: 0a0243f [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes fbb1150 [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes 7aec2a9 [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes fb42f0f [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes 090d908 [yantangzhai] [SPARK-2714] DAGScheduler logs jobid when runJob finishes --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 9 +++++++-- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 218b353dd9d49..428f019b02a23 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1072,11 +1072,8 @@ class SparkContext(config: SparkConf) extends Logging { val callSite = getCallSite val cleanedFunc = clean(func) logInfo("Starting job: " + callSite.shortForm) - val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, resultHandler, localProperties.get) - logInfo( - "Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s") rdd.doCheckpoint() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6fcf9e31543ed..b2774dfc47553 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -507,11 +507,16 @@ class DAGScheduler( resultHandler: (Int, U) => Unit, properties: Properties = null) { + val start = System.nanoTime val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { - case JobSucceeded => {} + case JobSucceeded => { + logInfo("Job %d finished: %s, took %f s".format + (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) + } case JobFailed(exception: Exception) => - logInfo("Failed to run " + callSite.shortForm) + logInfo("Job %d failed: %s, took %f s".format + (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) throw exception } } From 3b93128139e8d303f1d7bfd04e9a99a11a5b6404 Mon Sep 17 00:00:00 2001 From: Christoph Sawade Date: Mon, 15 Sep 2014 17:39:31 -0700 Subject: [PATCH 0696/1492] [SPARK-3396][MLLIB] Use SquaredL2Updater in LogisticRegressionWithSGD SimpleUpdater ignores the regularizer, which leads to an unregularized LogReg. To enable the common L2 regularizer (and the corresponding regularization parameter) for logistic regression the SquaredL2Updater has to be used in SGD (see, e.g., [SVMWithSGD]) Author: Christoph Sawade Closes #2398 from BigCrunsh/fix-regparam-logreg and squashes the following commits: 0820c04 [Christoph Sawade] Use SquaredL2Updater in LogisticRegressionWithSGD --- .../classification/LogisticRegression.scala | 2 +- .../LogisticRegressionSuite.scala | 44 +++++++++++++++++-- 2 files changed, 42 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 486bdbfa9cb47..84d3c7cebd7c8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -84,7 +84,7 @@ class LogisticRegressionWithSGD private ( extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { private val gradient = new LogisticGradient() - private val updater = new SimpleUpdater() + private val updater = new SquaredL2Updater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 862178694a50e..e954baaf7d91e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -43,7 +43,7 @@ object LogisticRegressionSuite { offset: Double, scale: Double, nPoints: Int, - seed: Int): Seq[LabeledPoint] = { + seed: Int): Seq[LabeledPoint] = { val rnd = new Random(seed) val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian()) @@ -58,12 +58,15 @@ object LogisticRegressionSuite { } class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Matchers { - def validatePrediction(predictions: Seq[Double], input: Seq[LabeledPoint]) { + def validatePrediction( + predictions: Seq[Double], + input: Seq[LabeledPoint], + expectedAcc: Double = 0.83) { val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => prediction != expected.label } // At least 83% of the predictions should be on. - ((input.length - numOffPredictions).toDouble / input.length) should be > 0.83 + ((input.length - numOffPredictions).toDouble / input.length) should be > expectedAcc } // Test if we can correctly learn A, B where Y = logistic(A + B*X) @@ -155,6 +158,41 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Match validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + test("logistic regression with initial weights and non-default regularization parameter") { + val nPoints = 10000 + val A = 2.0 + val B = -1.5 + + val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) + + val initialB = -1.0 + val initialWeights = Vectors.dense(initialB) + + val testRDD = sc.parallelize(testData, 2) + testRDD.cache() + + // Use half as many iterations as the previous test. + val lr = new LogisticRegressionWithSGD().setIntercept(true) + lr.optimizer. + setStepSize(10.0). + setNumIterations(10). + setRegParam(1.0) + + val model = lr.run(testRDD, initialWeights) + + // Test the weights + assert(model.weights(0) ~== -430000.0 relTol 20000.0) + assert(model.intercept ~== 370000.0 relTol 20000.0) + + val validationData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 17) + val validationRDD = sc.parallelize(validationData, 2) + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData, 0.8) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData, 0.8) + } + test("logistic regression with initial weights with LBFGS") { val nPoints = 10000 val A = 2.0 From 983d6a9c48b69c5f0542922aa8b133f69eb1034d Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Mon, 15 Sep 2014 17:41:15 -0700 Subject: [PATCH 0697/1492] [MLlib] Update SVD documentation in IndexedRowMatrix Updating this to reflect the newest SVD via ARPACK Author: Reza Zadeh Closes #2389 from rezazadeh/irmdocs and squashes the following commits: 7fa1313 [Reza Zadeh] Update svd docs 715da25 [Reza Zadeh] Updated computeSVD documentation IndexedRowMatrix --- .../mllib/linalg/distributed/IndexedRowMatrix.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index ac6eaea3f43ad..5c1acca0ec532 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -76,16 +76,12 @@ class IndexedRowMatrix( } /** - * Computes the singular value decomposition of this matrix. + * Computes the singular value decomposition of this IndexedRowMatrix. * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. * - * There is no restriction on m, but we require `n^2` doubles to fit in memory. - * Further, n should be less than m. - - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), from which we recover S and V. - * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). - * Note that this approach requires `O(n^3)` time on the master node. + * The cost and implementation of this method is identical to that in + * [[org.apache.spark.mllib.linalg.distributed.RowMatrix]] + * With the addition of indices. * * At most k largest non-zero singular values and associated vectors are returned. * If there are k such values, then the dimensions of the return will be: From fdb302f49c021227026909bdcdade7496059013f Mon Sep 17 00:00:00 2001 From: "qiping.lqp" Date: Mon, 15 Sep 2014 17:43:26 -0700 Subject: [PATCH 0698/1492] [SPARK-3516] [mllib] DecisionTree: Add minInstancesPerNode, minInfoGain params to example and Python API Added minInstancesPerNode, minInfoGain params to: * DecisionTreeRunner.scala example * Python API (tree.py) Also: * Fixed typo in tree suite test "do not choose split that does not satisfy min instance per node requirements" * small style fixes CC: mengxr Author: qiping.lqp Author: Joseph K. Bradley Author: chouqin Closes #2349 from jkbradley/chouqin-dt-preprune and squashes the following commits: 61b2e72 [Joseph K. Bradley] Added max of 10GB for maxMemoryInMB in Strategy. a95e7c8 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into chouqin-dt-preprune 95c479d [Joseph K. Bradley] * Fixed typo in tree suite test "do not choose split that does not satisfy min instance per node requirements" * small style fixes e2628b6 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into chouqin-dt-preprune 19b01af [Joseph K. Bradley] Merge remote-tracking branch 'chouqin/dt-preprune' into chouqin-dt-preprune f1d11d1 [chouqin] fix typo c7ebaf1 [chouqin] fix typo 39f9b60 [chouqin] change edge `minInstancesPerNode` to 2 and add one more test c6e2dfc [Joseph K. Bradley] Added minInstancesPerNode and minInfoGain parameters to DecisionTreeRunner.scala and to Python API in tree.py 0278a11 [chouqin] remove `noSplit` and set `Predict` private to tree d593ec7 [chouqin] fix docs and change minInstancesPerNode to 1 efcc736 [qiping.lqp] fix bug 10b8012 [qiping.lqp] fix style 6728fad [qiping.lqp] minor fix: remove empty lines bb465ca [qiping.lqp] Merge branch 'master' of https://github.com/apache/spark into dt-preprune cadd569 [qiping.lqp] add api docs 46b891f [qiping.lqp] fix bug e72c7e4 [qiping.lqp] add comments 845c6fa [qiping.lqp] fix style f195e83 [qiping.lqp] fix style 987cbf4 [qiping.lqp] fix bug ff34845 [qiping.lqp] separate calculation of predict of node from calculation of info gain ac42378 [qiping.lqp] add min info gain and min instances per node parameters in decision tree --- .../examples/mllib/DecisionTreeRunner.scala | 13 ++++++++++++- .../spark/mllib/api/python/PythonMLLibAPI.scala | 8 ++++++-- .../apache/spark/mllib/tree/DecisionTree.scala | 4 ++-- .../mllib/tree/configuration/Strategy.scala | 2 ++ .../apache/spark/mllib/tree/model/Predict.scala | 6 +----- .../spark/mllib/tree/DecisionTreeSuite.scala | 4 ++-- python/pyspark/mllib/tree.py | 16 ++++++++++++---- 7 files changed, 37 insertions(+), 16 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 72c3ab475b61f..4683e6eb966be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -55,6 +55,8 @@ object DecisionTreeRunner { maxDepth: Int = 5, impurity: ImpurityType = Gini, maxBins: Int = 32, + minInstancesPerNode: Int = 1, + minInfoGain: Double = 0.0, fracTest: Double = 0.2) def main(args: Array[String]) { @@ -75,6 +77,13 @@ object DecisionTreeRunner { opt[Int]("maxBins") .text(s"max number of bins, default: ${defaultParams.maxBins}") .action((x, c) => c.copy(maxBins = x)) + opt[Int]("minInstancesPerNode") + .text(s"min number of instances required at child nodes to create the parent split," + + s" default: ${defaultParams.minInstancesPerNode}") + .action((x, c) => c.copy(minInstancesPerNode = x)) + opt[Double]("minInfoGain") + .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") + .action((x, c) => c.copy(minInfoGain = x)) opt[Double]("fracTest") .text(s"fraction of data to hold out for testing, default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) @@ -179,7 +188,9 @@ object DecisionTreeRunner { impurity = impurityCalculator, maxDepth = params.maxDepth, maxBins = params.maxBins, - numClassesForClassification = numClasses) + numClassesForClassification = numClasses, + minInstancesPerNode = params.minInstancesPerNode, + minInfoGain = params.minInfoGain) val model = DecisionTree.train(training, strategy) println(model) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 4343124f102a0..fa0fa69f38634 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -303,7 +303,9 @@ class PythonMLLibAPI extends Serializable { categoricalFeaturesInfoJMap: java.util.Map[Int, Int], impurityStr: String, maxDepth: Int, - maxBins: Int): DecisionTreeModel = { + maxBins: Int, + minInstancesPerNode: Int, + minInfoGain: Double): DecisionTreeModel = { val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) @@ -316,7 +318,9 @@ class PythonMLLibAPI extends Serializable { maxDepth = maxDepth, numClassesForClassification = numClasses, maxBins = maxBins, - categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap) + categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap, + minInstancesPerNode = minInstancesPerNode, + minInfoGain = minInfoGain) DecisionTree.train(data, strategy) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 56bb8812100a7..c7f2576c822b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -389,7 +389,7 @@ object DecisionTree extends Serializable with Logging { var groupIndex = 0 var doneTraining = true while (groupIndex < numGroups) { - val (tmpRoot, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, + val (_, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, topNode, splits, bins, timer, numGroups, groupIndex) doneTraining = doneTraining && doneTrainingGroup groupIndex += 1 @@ -898,7 +898,7 @@ object DecisionTree extends Serializable with Logging { } }.maxBy(_._2.gain) - require(predict.isDefined, "must calculate predict for each node") + assert(predict.isDefined, "must calculate predict for each node") (bestSplit, bestSplitStats, predict.get) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 31d1e8ac30eea..caaccbfb8ad16 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -77,6 +77,8 @@ class Strategy ( } require(minInstancesPerNode >= 1, s"DecisionTree Strategy requires minInstancesPerNode >= 1 but was given $minInstancesPerNode") + require(maxMemoryInMB <= 10240, + s"DecisionTree Strategy requires maxMemoryInMB <= 10240, but was given $maxMemoryInMB") val isMulticlassClassification = algo == Classification && numClassesForClassification > 2 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index 6fac2be2797bc..d8476b5cd7bc7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -17,18 +17,14 @@ package org.apache.spark.mllib.tree.model -import org.apache.spark.annotation.DeveloperApi - /** - * :: DeveloperApi :: * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) */ -@DeveloperApi private[tree] class Predict( val predict: Double, - val prob: Double = 0.0) extends Serializable{ + val prob: Double = 0.0) extends Serializable { override def toString = { "predict = %f, prob = %f".format(predict, prob) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 1bd7ea05c46c8..2b2e579b992f6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -714,8 +714,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(gain == InformationGainStats.invalidInformationGainStats) } - test("don't choose split that doesn't satisfy min instance per node requirements") { - // if a split doesn't satisfy min instances per node requirements, + test("do not choose split that does not satisfy min instance per node requirements") { + // if a split does not satisfy min instances per node requirements, // this split is invalid, even though the information gain of split is large. val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.dense(0.0, 1.0)) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index ccc000ac70ba6..5b13ab682bbfc 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -138,7 +138,8 @@ class DecisionTree(object): @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, - impurity="gini", maxDepth=5, maxBins=32): + impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, + minInfoGain=0.0): """ Train a DecisionTreeModel for classification. @@ -154,6 +155,9 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, E.g., depth 0 means 1 leaf node. Depth 1 means 1 internal node + 2 leaf nodes. :param maxBins: Number of bins used for finding splits at each node. + :param minInstancesPerNode: Min number of instances required at child nodes to create + the parent split + :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel """ sc = data.context @@ -164,13 +168,14 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( dataBytes._jrdd, "classification", numClasses, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins) + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) dataBytes.unpersist() return DecisionTreeModel(sc, model) @staticmethod def trainRegressor(data, categoricalFeaturesInfo, - impurity="variance", maxDepth=5, maxBins=32): + impurity="variance", maxDepth=5, maxBins=32, minInstancesPerNode=1, + minInfoGain=0.0): """ Train a DecisionTreeModel for regression. @@ -185,6 +190,9 @@ def trainRegressor(data, categoricalFeaturesInfo, E.g., depth 0 means 1 leaf node. Depth 1 means 1 internal node + 2 leaf nodes. :param maxBins: Number of bins used for finding splits at each node. + :param minInstancesPerNode: Min number of instances required at child nodes to create + the parent split + :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel """ sc = data.context @@ -195,7 +203,7 @@ def trainRegressor(data, categoricalFeaturesInfo, model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( dataBytes._jrdd, "regression", 0, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins) + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) dataBytes.unpersist() return DecisionTreeModel(sc, model) From da33acb8b681eca5e787d546fe922af76a151398 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 15 Sep 2014 18:57:25 -0700 Subject: [PATCH 0699/1492] [SPARK-2951] [PySpark] support unpickle array.array for Python 2.6 Pyrolite can not unpickle array.array which pickled by Python 2.6, this patch fix it by extend Pyrolite. There is a bug in Pyrolite when unpickle array of float/double, this patch workaround it by reverse the endianness for float/double. This workaround should be removed after Pyrolite have a new release to fix this issue. I had send an PR to Pyrolite to fix it: https://github.com/irmen/Pyrolite/pull/11 Author: Davies Liu Closes #2365 from davies/pickle and squashes the following commits: f44f771 [Davies Liu] enable tests about array 3908f5c [Davies Liu] Merge branch 'master' into pickle c77c87b [Davies Liu] cleanup debugging code 60e4e2f [Davies Liu] support unpickle array.array for Python 2.6 --- .../apache/spark/api/python/SerDeUtil.scala | 51 +++++++++++++++++++ python/pyspark/context.py | 1 + python/pyspark/tests.py | 2 - 3 files changed, 52 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index efc9009c088a8..6668797f5f8be 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -17,6 +17,8 @@ package org.apache.spark.api.python +import java.nio.ByteOrder + import scala.collection.JavaConversions._ import scala.util.Failure import scala.util.Try @@ -28,6 +30,55 @@ import org.apache.spark.rdd.RDD /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ private[python] object SerDeUtil extends Logging { + // Unpickle array.array generated by Python 2.6 + class ArrayConstructor extends net.razorvine.pickle.objects.ArrayConstructor { + // /* Description of types */ + // static struct arraydescr descriptors[] = { + // {'c', sizeof(char), c_getitem, c_setitem}, + // {'b', sizeof(char), b_getitem, b_setitem}, + // {'B', sizeof(char), BB_getitem, BB_setitem}, + // #ifdef Py_USING_UNICODE + // {'u', sizeof(Py_UNICODE), u_getitem, u_setitem}, + // #endif + // {'h', sizeof(short), h_getitem, h_setitem}, + // {'H', sizeof(short), HH_getitem, HH_setitem}, + // {'i', sizeof(int), i_getitem, i_setitem}, + // {'I', sizeof(int), II_getitem, II_setitem}, + // {'l', sizeof(long), l_getitem, l_setitem}, + // {'L', sizeof(long), LL_getitem, LL_setitem}, + // {'f', sizeof(float), f_getitem, f_setitem}, + // {'d', sizeof(double), d_getitem, d_setitem}, + // {'\0', 0, 0, 0} /* Sentinel */ + // }; + // TODO: support Py_UNICODE with 2 bytes + // FIXME: unpickle array of float is wrong in Pyrolite, so we reverse the + // machine code for float/double here to workaround it. + // we should fix this after Pyrolite fix them + val machineCodes: Map[Char, Int] = if (ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN)) { + Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 3, 'h' -> 5, 'I' -> 7, 'i' -> 9, + 'L' -> 11, 'l' -> 13, 'f' -> 14, 'd' -> 16, 'u' -> 21 + ) + } else { + Map('c' -> 1, 'B' -> 0, 'b' -> 1, 'H' -> 2, 'h' -> 4, 'I' -> 6, 'i' -> 8, + 'L' -> 10, 'l' -> 12, 'f' -> 15, 'd' -> 17, 'u' -> 20 + ) + } + override def construct(args: Array[Object]): Object = { + if (args.length == 1) { + construct(args ++ Array("")) + } else if (args.length == 2 && args(1).isInstanceOf[String]) { + val typecode = args(0).asInstanceOf[String].charAt(0) + val data: String = args(1).asInstanceOf[String] + construct(typecode, machineCodes(typecode), data.getBytes("ISO-8859-1")) + } else { + super.construct(args) + } + } + } + + def initialize() = { + Unpickler.registerConstructor("array", "array", new ArrayConstructor()) + } private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { val pickle = new Pickler diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 3ab98e262df31..ea28e8cd8c89f 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -214,6 +214,7 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile + SparkContext._jvm.SerDeUtil.initialize() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f3309a20fcffb..f255b44359fec 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -956,8 +956,6 @@ def test_newhadoop(self): conf=input_conf).collect()) self.assertEqual(new_dataset, data) - @unittest.skipIf(sys.version_info[:2] <= (2, 6) or python_implementation() == "PyPy", - "Skipped on 2.6 and PyPy until SPARK-2951 is fixed") def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays From 60050f42885582a699fc7a6fa0529964162bb8a3 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Mon, 15 Sep 2014 19:28:17 -0700 Subject: [PATCH 0700/1492] [SPARK-1087] Move python traceback utilities into new traceback_utils.py file. Also made some cosmetic cleanups. Author: Aaron Staple Closes #2385 from staple/SPARK-1087 and squashes the following commits: 7b3bb13 [Aaron Staple] Address review comments, cosmetic cleanups. 10ba6e1 [Aaron Staple] [SPARK-1087] Move python traceback utilities into new traceback_utils.py file. --- python/pyspark/context.py | 8 +--- python/pyspark/rdd.py | 58 ++--------------------- python/pyspark/traceback_utils.py | 78 +++++++++++++++++++++++++++++++ 3 files changed, 83 insertions(+), 61 deletions(-) create mode 100644 python/pyspark/traceback_utils.py diff --git a/python/pyspark/context.py b/python/pyspark/context.py index ea28e8cd8c89f..a33aae87f65e8 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -20,7 +20,6 @@ import sys from threading import Lock from tempfile import NamedTemporaryFile -from collections import namedtuple from pyspark import accumulators from pyspark.accumulators import Accumulator @@ -33,6 +32,7 @@ from pyspark.storagelevel import StorageLevel from pyspark import rdd from pyspark.rdd import RDD +from pyspark.traceback_utils import CallSite, first_spark_call from py4j.java_collections import ListConverter @@ -99,11 +99,7 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ... ValueError:... """ - if rdd._extract_concise_traceback() is not None: - self._callsite = rdd._extract_concise_traceback() - else: - tempNamedTuple = namedtuple("Callsite", "function file linenum") - self._callsite = tempNamedTuple(function=None, file=None, linenum=None) + self._callsite = first_spark_call() or CallSite(None, None, None) SparkContext._ensure_initialized(self, gateway=gateway) try: self._do_init(master, appName, sparkHome, pyFiles, environment, batchSize, serializer, diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6ad5ab2a2d1ae..21f182b0ff137 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -18,13 +18,11 @@ from base64 import standard_b64encode as b64enc import copy from collections import defaultdict -from collections import namedtuple from itertools import chain, ifilter, imap import operator import os import sys import shlex -import traceback from subprocess import Popen, PIPE from tempfile import NamedTemporaryFile from threading import Thread @@ -45,6 +43,7 @@ from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ get_used_memory, ExternalSorter +from pyspark.traceback_utils import SCCallSiteSync from py4j.java_collections import ListConverter, MapConverter @@ -81,57 +80,6 @@ def portable_hash(x): return hash(x) -def _extract_concise_traceback(): - """ - This function returns the traceback info for a callsite, returns a dict - with function name, file name and line number - """ - tb = traceback.extract_stack() - callsite = namedtuple("Callsite", "function file linenum") - if len(tb) == 0: - return None - file, line, module, what = tb[len(tb) - 1] - sparkpath = os.path.dirname(file) - first_spark_frame = len(tb) - 1 - for i in range(0, len(tb)): - file, line, fun, what = tb[i] - if file.startswith(sparkpath): - first_spark_frame = i - break - if first_spark_frame == 0: - 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] - 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) - else: - self._traceback = "Error! Could not extract traceback info" - self._context = sc - - def __enter__(self): - global _spark_stack_depth - if _spark_stack_depth == 0: - self._context._jsc.setCallSite(self._traceback) - _spark_stack_depth += 1 - - def __exit__(self, type, value, tb): - global _spark_stack_depth - _spark_stack_depth -= 1 - if _spark_stack_depth == 0: - self._context._jsc.setCallSite(None) - - class BoundedFloat(float): """ Bounded value is generated by approximate job, with confidence and low @@ -704,7 +652,7 @@ def collect(self): """ Return a list that contains all of the elements in this RDD. """ - with _JavaStackTrace(self.context) as st: + with SCCallSiteSync(self.context) as css: bytesInJava = self._jrdd.collect().iterator() return list(self._collect_iterator_through_file(bytesInJava)) @@ -1515,7 +1463,7 @@ def add_shuffle_key(split, iterator): keyed = self.mapPartitionsWithIndex(add_shuffle_key) keyed._bypass_serializer = True - with _JavaStackTrace(self.context) as st: + with SCCallSiteSync(self.context) as css: pairRDD = self.ctx._jvm.PairwiseRDD( keyed._jrdd.rdd()).asJavaPairRDD() partitioner = self.ctx._jvm.PythonPartitioner(numPartitions, diff --git a/python/pyspark/traceback_utils.py b/python/pyspark/traceback_utils.py new file mode 100644 index 0000000000000..bb8646df2b0bf --- /dev/null +++ b/python/pyspark/traceback_utils.py @@ -0,0 +1,78 @@ +# +# 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. +# + +from collections import namedtuple +import os +import traceback + + +CallSite = namedtuple("CallSite", "function file linenum") + + +def first_spark_call(): + """ + Return a CallSite representing the first Spark call in the current call stack. + """ + tb = traceback.extract_stack() + if len(tb) == 0: + return None + file, line, module, what = tb[len(tb) - 1] + sparkpath = os.path.dirname(file) + first_spark_frame = len(tb) - 1 + for i in range(0, len(tb)): + file, line, fun, what = tb[i] + if file.startswith(sparkpath): + first_spark_frame = i + break + if first_spark_frame == 0: + 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] + return CallSite(function=sfun, file=ufile, linenum=uline) + + +class SCCallSiteSync(object): + """ + Helper for setting the spark context call site. + + Example usage: + from pyspark.context import SCCallSiteSync + with SCCallSiteSync() as css: +
    + """ + + _spark_stack_depth = 0 + + def __init__(self, sc): + call_site = first_spark_call() + if call_site is not None: + self._call_site = "%s at %s:%s" % ( + call_site.function, call_site.file, call_site.linenum) + else: + self._call_site = "Error! Could not extract traceback info" + self._context = sc + + def __enter__(self): + if SCCallSiteSync._spark_stack_depth == 0: + self._context._jsc.setCallSite(self._call_site) + SCCallSiteSync._spark_stack_depth += 1 + + def __exit__(self, type, value, tb): + SCCallSiteSync._spark_stack_depth -= 1 + if SCCallSiteSync._spark_stack_depth == 0: + self._context._jsc.setCallSite(None) From d428ac6a221d2dce19c43442abf197f2ade6658f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 15 Sep 2014 21:09:58 -0700 Subject: [PATCH 0701/1492] [SPARK-3540] Add reboot-slaves functionality to the ec2 script Tested on a real cluster. Author: Reynold Xin Closes #2404 from rxin/ec2-reboot-slaves and squashes the following commits: 00a2dbd [Reynold Xin] Allow rebooting slaves. --- ec2/spark_ec2.py | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index bfd07593b92ed..5682e96aa8770 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -52,7 +52,7 @@ class UsageError(Exception): def parse_args(): parser = OptionParser( usage="spark-ec2 [options] " - + "\n\n can be: launch, destroy, login, stop, start, get-master", + + "\n\n can be: launch, destroy, login, stop, start, get-master, reboot-slaves", add_help_option=False) parser.add_option( "-h", "--help", action="help", @@ -950,6 +950,20 @@ def real_main(): subprocess.check_call( ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) + elif action == "reboot-slaves": + response = raw_input( + "Are you sure you want to reboot the cluster " + + cluster_name + " slaves?\n" + + "Reboot cluster slaves " + cluster_name + " (y/N): ") + if response == "y": + (master_nodes, slave_nodes) = get_existing_cluster( + conn, opts, cluster_name, die_on_error=False) + print "Rebooting slaves..." + for inst in slave_nodes: + if inst.state not in ["shutting-down", "terminated"]: + print "Rebooting " + inst.id + inst.reboot() + elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) print master_nodes[0].public_dns_name From ecf0c02935815f0d4018c0e30ec4c784e60a5db0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Sep 2014 21:14:00 -0700 Subject: [PATCH 0702/1492] [SPARK-3433][BUILD] Fix for Mima false-positives with @DeveloperAPI and @Experimental annotations. Actually false positive reported was due to mima generator not picking up the new jars in presence of old jars(theoretically this should not have happened.). So as a workaround, ran them both separately and just append them together. Author: Prashant Sharma Author: Prashant Sharma Closes #2285 from ScrapCodes/mima-fix and squashes the following commits: 093c76f [Prashant Sharma] Update mima 59012a8 [Prashant Sharma] Update mima 35b6c71 [Prashant Sharma] SPARK-3433 Fix for Mima false-positives with @DeveloperAPI and @Experimental annotations. --- dev/mima | 8 ++++++++ project/MimaBuild.scala | 6 ++++++ project/MimaExcludes.scala | 8 +------- project/SparkBuild.scala | 2 +- .../apache/spark/tools/GenerateMIMAIgnore.scala | 14 ++++++++++---- 5 files changed, 26 insertions(+), 12 deletions(-) diff --git a/dev/mima b/dev/mima index f9b9b03538f15..40603166c21ae 100755 --- a/dev/mima +++ b/dev/mima @@ -25,11 +25,19 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)" cd "$FWDIR" echo -e "q\n" | sbt/sbt oldDeps/update +rm -f .generated-mima* + +# Generate Mima Ignore is called twice, first with latest built jars +# on the classpath and then again with previous version jars on the classpath. +# Because of a bug in GenerateMIMAIgnore that when old jars are ahead on classpath +# it did not process the new classes (which are in assembly jar). +./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore export SPARK_CLASSPATH="`find lib_managed \( -name '*spark*jar' -a -type f \) | tr "\\n" ":"`" echo "SPARK_CLASSPATH=$SPARK_CLASSPATH" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore + echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" ret_val=$? diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 0f5d71afcf616..39f8ba4745737 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -30,6 +30,12 @@ object MimaBuild { def excludeMember(fullName: String) = Seq( ProblemFilters.exclude[MissingMethodProblem](fullName), + // Sometimes excluded methods have default arguments and + // they are translated into public methods/fields($default$) in generated + // bytecode. It is not possible to exhustively list everything. + // But this should be okay. + ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$2"), + ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$1"), ProblemFilters.exclude[MissingFieldProblem](fullName), ProblemFilters.exclude[IncompatibleResultTypeProblem](fullName), ProblemFilters.exclude[IncompatibleMethTypeProblem](fullName), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 46b78bd5c7061..2f1e05dfcc7b1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -37,14 +37,8 @@ object MimaExcludes { Seq( MimaBuild.excludeSparkPackage("deploy"), MimaBuild.excludeSparkPackage("graphx") - ) ++ - // This is @DeveloperAPI, but Mima still gives false-positives: - MimaBuild.excludeSparkClass("scheduler.SparkListenerApplicationStart") ++ - Seq( - // This is @Experimental, but Mima still gives false-positives: - ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.api.java.JavaRDDLike.foreachAsync") ) + case v if v.startsWith("1.1") => Seq( MimaBuild.excludeSparkPackage("deploy"), diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c07ea313f1228..ab9f8ba120e83 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -187,7 +187,7 @@ object OldDeps { Some("org.apache.spark" % fullId % "1.1.0") } - def oldDepsSettings() = Defaults.defaultSettings ++ Seq( + def oldDepsSettings() = Defaults.coreDefaultSettings ++ Seq( name := "old-deps", scalaVersion := "2.10.4", retrieveManaged := true, diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index bcf6d43ab34eb..595ded6ae67fa 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import scala.collection.JavaConversions._ import scala.reflect.runtime.universe.runtimeMirror import scala.reflect.runtime.{universe => unv} +import scala.util.Try /** * A tool for generating classes to be excluded during binary checking with MIMA. It is expected @@ -121,12 +122,17 @@ object GenerateMIMAIgnore { } def main(args: Array[String]) { + import scala.tools.nsc.io.File val (privateClasses, privateMembers) = privateWithin("org.apache.spark") - scala.tools.nsc.io.File(".generated-mima-class-excludes"). - writeAll(privateClasses.mkString("\n")) + val previousContents = Try(File(".generated-mima-class-excludes").lines()). + getOrElse(Iterator.empty).mkString("\n") + File(".generated-mima-class-excludes") + .writeAll(previousContents + privateClasses.mkString("\n")) println("Created : .generated-mima-class-excludes in current directory.") - scala.tools.nsc.io.File(".generated-mima-member-excludes"). - writeAll(privateMembers.mkString("\n")) + val previousMembersContents = Try(File(".generated-mima-member-excludes").lines) + .getOrElse(Iterator.empty).mkString("\n") + File(".generated-mima-member-excludes").writeAll(previousMembersContents + + privateMembers.mkString("\n")) println("Created : .generated-mima-member-excludes in current directory.") } From febafefa5aaee3b3eda5e1b45a75bc6d8e7fb13f Mon Sep 17 00:00:00 2001 From: Ye Xianjin Date: Mon, 15 Sep 2014 21:53:38 -0700 Subject: [PATCH 0703/1492] [SPARK-3040] pick up a more proper local ip address for Utils.findLocalIpAddress method Short version: NetworkInterface.getNetworkInterfaces returns ifs in reverse order compared to ifconfig output. It may pick up ip address associated with tun0 or virtual network interface. See [SPARK_3040](https://issues.apache.org/jira/browse/SPARK-3040) for more detail Author: Ye Xianjin Closes #1946 from advancedxy/SPARK-3040 and squashes the following commits: f33f6b2 [Ye Xianjin] add windows support 087a785 [Ye Xianjin] reverse the Networkinterface.getNetworkInterfaces output order to get a more proper local ip address. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 7 ++++++- 1 file changed, 6 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 79943766d0f0f..c76b7af18481d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -530,7 +530,12 @@ private[spark] object Utils extends Logging { if (address.isLoopbackAddress) { // Address resolves to something like 127.0.1.1, which happens on Debian; try to find // a better address using the local network interfaces - for (ni <- NetworkInterface.getNetworkInterfaces) { + // getNetworkInterfaces returns ifs in reverse order compared to ifconfig output order + // on unix-like system. On windows, it returns in index order. + // It's more proper to pick ip address following system output order. + val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.toList + val reOrderedNetworkIFs = if (isWindows) activeNetworkIFs else activeNetworkIFs.reverse + for (ni <- reOrderedNetworkIFs) { for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && !addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) { // We've found an address that looks reasonable! From 61e21fe7f478e7b06b72851f26b87d99cbbdf117 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 16 Sep 2014 09:18:03 -0700 Subject: [PATCH 0704/1492] SPARK-3069 [DOCS] Build instructions in README are outdated Here's my crack at Bertrand's suggestion. The Github `README.md` contains build info that's outdated. It should just point to the current online docs, and reflect that Maven is the primary build now. (Incidentally, the stanza at the end about contributions of original work should go in https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark too. It won't hurt to be crystal clear about the agreement to license, given that ICLAs are not required of anyone here.) Author: Sean Owen Closes #2014 from srowen/SPARK-3069 and squashes the following commits: 501507e [Sean Owen] Note that Zinc is for Maven builds too db2bd97 [Sean Owen] sbt -> sbt/sbt and add note about zinc be82027 [Sean Owen] Fix additional occurrences of building-with-maven -> building-spark 91c921f [Sean Owen] Move building-with-maven to building-spark and create a redirect. Update doc links to building-spark.html Add jekyll-redirect-from plugin and make associated config changes (including fixing pygments deprecation). Add example of SBT to README.md 999544e [Sean Owen] Change "Building Spark with Maven" title to "Building Spark"; reinstate tl;dr info about dev/run-tests in README.md; add brief note about building with SBT c18d140 [Sean Owen] Optionally, remove the copy of contributing text from main README.md 8e83934 [Sean Owen] Add CONTRIBUTING.md to trigger notice on new pull request page b1c04a1 [Sean Owen] Refer to current online documentation for building, and remove slightly outdated copy in README.md --- CONTRIBUTING.md | 12 +++ README.md | 78 ++++--------------- docs/README.md | 5 +- docs/_config.yml | 4 +- docs/_layouts/global.html | 2 +- ...ilding-with-maven.md => building-spark.md} | 20 ++++- docs/hadoop-third-party-distributions.md | 2 +- docs/index.md | 4 +- docs/running-on-yarn.md | 2 +- docs/streaming-kinesis-integration.md | 2 +- make-distribution.sh | 2 +- 11 files changed, 60 insertions(+), 73 deletions(-) create mode 100644 CONTRIBUTING.md rename docs/{building-with-maven.md => building-spark.md} (87%) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md new file mode 100644 index 0000000000000..c6b4aa5344757 --- /dev/null +++ b/CONTRIBUTING.md @@ -0,0 +1,12 @@ +## Contributing to Spark + +Contributions via GitHub pull requests are gladly accepted from their original +author. Along with any pull requests, please state that the contribution is +your original work and that you license the work to the project under the +project's open source license. Whether or not you state this explicitly, by +submitting any copyrighted material via pull request, email, or other means +you agree to license the material under the project's open source license and +warrant that you have the legal authority to do so. + +Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) +for more information. diff --git a/README.md b/README.md index 5b09ad86849e7..b05bbfb5a594c 100644 --- a/README.md +++ b/README.md @@ -13,16 +13,19 @@ and Spark Streaming for stream processing. ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the project webpage at . +guide, on the [project web page](http://spark.apache.org/documentation.html). This README file only contains basic setup instructions. ## Building Spark -Spark is built on Scala 2.10. To build Spark and its example programs, run: +Spark is built using [Apache Maven](http://maven.apache.org/). +To build Spark and its example programs, run: - ./sbt/sbt assembly + mvn -DskipTests clean package (You do not need to do this if you downloaded a pre-built package.) +More detailed documentation is available from the project site, at +["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). ## Interactive Scala Shell @@ -71,73 +74,24 @@ can be run using: ./dev/run-tests +Please see the guidance on how to +[run all automated tests](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting) + ## A Note About Hadoop Versions Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs. -You can change the version by setting `-Dhadoop.version` when building Spark. - -For Apache Hadoop versions 1.x, Cloudera CDH MRv1, and other Hadoop -versions without YARN, use: - - # Apache Hadoop 1.2.1 - $ sbt/sbt -Dhadoop.version=1.2.1 assembly - - # Cloudera CDH 4.2.0 with MapReduce v1 - $ sbt/sbt -Dhadoop.version=2.0.0-mr1-cdh4.2.0 assembly - -For Apache Hadoop 2.2.X, 2.1.X, 2.0.X, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions -with YARN, also set `-Pyarn`: - - # Apache Hadoop 2.0.5-alpha - $ sbt/sbt -Dhadoop.version=2.0.5-alpha -Pyarn assembly - - # Cloudera CDH 4.2.0 with MapReduce v2 - $ sbt/sbt -Dhadoop.version=2.0.0-cdh4.2.0 -Pyarn assembly - - # Apache Hadoop 2.2.X and newer - $ sbt/sbt -Dhadoop.version=2.2.0 -Pyarn assembly - -When developing a Spark application, specify the Hadoop version by adding the -"hadoop-client" artifact to your project's dependencies. For example, if you're -using Hadoop 1.2.1 and build your application using SBT, add this entry to -`libraryDependencies`: - - "org.apache.hadoop" % "hadoop-client" % "1.2.1" -If your project is built with Maven, add this to your POM file's `` section: - - - org.apache.hadoop - hadoop-client - 1.2.1 - - - -## A Note About Thrift JDBC server and CLI for Spark SQL - -Spark SQL supports Thrift JDBC server and CLI. -See sql-programming-guide.md for more information about using the JDBC server and CLI. -You can use those features by setting `-Phive` when building Spark as follows. - - $ sbt/sbt -Phive assembly +Please refer to the build documentation at +["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version) +for detailed guidance on building for a particular distribution of Hadoop, including +building for particular Hive and Hive Thriftserver distributions. See also +["Third Party Hadoop Distributions"](http://spark.apache.org/docs/latest/hadoop-third-party-distributions.html) +for guidance on building a Spark application that works with a particular +distribution. ## Configuration Please refer to the [Configuration guide](http://spark.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. - - -## Contributing to Spark - -Contributions via GitHub pull requests are gladly accepted from their original -author. Along with any pull requests, please state that the contribution is -your original work and that you license the work to the project under the -project's open source license. Whether or not you state this explicitly, by -submitting any copyrighted material via pull request, email, or other means -you agree to license the material under the project's open source license and -warrant that you have the legal authority to do so. - -Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) -for more information. diff --git a/docs/README.md b/docs/README.md index 0a0126c5747d1..fdc89d2eb767a 100644 --- a/docs/README.md +++ b/docs/README.md @@ -23,8 +23,9 @@ The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllr To use the `jekyll` command, you will need to have Jekyll installed. The easiest way to do this is via a Ruby Gem, see the [jekyll installation instructions](http://jekyllrb.com/docs/installation). -If not already installed, you need to install `kramdown` with `sudo gem install kramdown`. -Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory +If not already installed, you need to install `kramdown` and `jekyll-redirect-from` Gems +with `sudo gem install kramdown jekyll-redirect-from`. +Execute `jekyll build` from the `docs/` directory. Compiling the site with Jekyll will create a directory called `_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: diff --git a/docs/_config.yml b/docs/_config.yml index 45b78fe724a50..d3ea2625c7448 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -1,5 +1,7 @@ -pygments: true +highlighter: pygments markdown: kramdown +gems: + - jekyll-redirect-from # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index b30ab1e5218c0..a53e8a775b71f 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -109,7 +109,7 @@
  • Hardware Provisioning
  • 3rd-Party Hadoop Distros
  • -
  • Building Spark with Maven
  • +
  • Building Spark
  • Contributing to Spark
  • diff --git a/docs/building-with-maven.md b/docs/building-spark.md similarity index 87% rename from docs/building-with-maven.md rename to docs/building-spark.md index bce7412c7d4c9..2378092d4a1a8 100644 --- a/docs/building-with-maven.md +++ b/docs/building-spark.md @@ -1,6 +1,7 @@ --- layout: global -title: Building Spark with Maven +title: Building Spark +redirect_from: "building-with-maven.html" --- * This will become a table of contents (this text will be scraped). @@ -159,4 +160,21 @@ then ship it over to the cluster. We are investigating the exact cause for this. The assembly jar produced by `mvn package` will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The `hadoop-provided` profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. +# Building with SBT +Maven is the official recommendation for packaging Spark, and is the "build of reference". +But SBT is supported for day-to-day development since it can provide much faster iterative +compilation. More advanced developers may wish to use SBT. + +The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables +can be set to control the SBT build. For example: + + sbt/sbt -Pyarn -Phadoop-2.3 compile + +# Speeding up Compilation with Zinc + +[Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental +compiler. When run locally as a background process, it speeds up builds of Scala-based projects +like Spark. Developers who regularly recompile Spark with Maven will be the most interested in +Zinc. The project site gives instructions for building and running `zinc`; OS X users can +install it using `brew install zinc`. \ No newline at end of file diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index ab1023b8f1842..dd73e9dc54440 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -11,7 +11,7 @@ with these distributions: When compiling Spark, you'll need to specify the Hadoop version by defining the `hadoop.version` property. For certain versions, you will need to specify additional profiles. For more detail, -see the guide on [building with maven](building-with-maven.html#specifying-the-hadoop-version): +see the guide on [building with maven](building-spark.html#specifying-the-hadoop-version): mvn -Dhadoop.version=1.0.4 -DskipTests clean package mvn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package diff --git a/docs/index.md b/docs/index.md index 7fe6b43d32af7..e8ebadbd4e427 100644 --- a/docs/index.md +++ b/docs/index.md @@ -12,7 +12,7 @@ It also supports a rich set of higher-level tools including [Spark SQL](sql-prog Get Spark from the [downloads page](http://spark.apache.org/downloads.html) of the project website. This documentation is for Spark version {{site.SPARK_VERSION}}. The downloads page contains Spark packages for many popular HDFS versions. If you'd like to build Spark from -scratch, visit [building Spark with Maven](building-with-maven.html). +scratch, visit [Building Spark](building-spark.html). Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy to run locally on one machine --- all you need is to have `java` installed on your system `PATH`, @@ -105,7 +105,7 @@ options for deployment: * [3rd Party Hadoop Distributions](hadoop-third-party-distributions.html): using common Hadoop distributions * Integration with other storage systems: * [OpenStack Swift](storage-openstack-swift.html) -* [Building Spark with Maven](building-with-maven.html): build Spark using the Maven system +* [Building Spark](building-spark.html): build Spark using the Maven system * [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) **External Resources:** diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 212248bcce1c1..74bcc2eeb65f6 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -11,7 +11,7 @@ was added to Spark in version 0.6.0, and improved in subsequent releases. Running Spark-on-YARN requires a binary distribution of Spark which is built with YARN support. Binary distributions can be downloaded from the Spark project website. -To build Spark yourself, refer to the [building with Maven guide](building-with-maven.html). +To build Spark yourself, refer to [Building Spark](building-spark.html). # Configuration diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index c6090d9ec30c7..379eb513d521e 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -108,7 +108,7 @@ A Kinesis stream can be set up at one of the valid Kinesis endpoints with 1 or m #### Running the Example To run the example, -- Download Spark source and follow the [instructions](building-with-maven.html) to build Spark with profile *-Pkinesis-asl*. +- Download Spark source and follow the [instructions](building-spark.html) to build Spark with profile *-Pkinesis-asl*. mvn -Pkinesis-asl -DskipTests clean package diff --git a/make-distribution.sh b/make-distribution.sh index 9b012b9222db4..884659954a491 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -40,7 +40,7 @@ function exit_with_usage { echo "" echo "usage:" echo "./make-distribution.sh [--name] [--tgz] [--with-tachyon] " - echo "See Spark's \"Building with Maven\" doc for correct Maven options." + echo "See Spark's \"Building Spark\" doc for correct Maven options." echo "" exit 1 } From 7b8008f5a4d413b61aa88fbc60959e98e59f17dd Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 16 Sep 2014 09:21:03 -0700 Subject: [PATCH 0705/1492] [SPARK-2182] Scalastyle rule blocking non ascii characters. ...erators. Author: Prashant Sharma Closes #2358 from ScrapCodes/scalastyle-unicode and squashes the following commits: 12a20f2 [Prashant Sharma] [SPARK-2182] Scalastyle rule blocking (non keyboard typeable) unicode operators. --- .../scalastyle/NonASCIICharacterChecker.scala | 39 +++++++++++++++++++ scalastyle-config.xml | 1 + 2 files changed, 40 insertions(+) create mode 100644 project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala new file mode 100644 index 0000000000000..3d43c35299555 --- /dev/null +++ b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/NonASCIICharacterChecker.scala @@ -0,0 +1,39 @@ +/* + * 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.scalastyle + +import java.util.regex.Pattern + +import org.scalastyle.{PositionError, ScalariformChecker, ScalastyleError} + +import scalariform.lexer.Token +import scalariform.parser.CompilationUnit + +class NonASCIICharacterChecker extends ScalariformChecker { + val errorKey: String = "non.ascii.character.disallowed" + + override def verify(ast: CompilationUnit): List[ScalastyleError] = { + ast.tokens.filter(hasNonAsciiChars).map(x => PositionError(x.offset)).toList + } + + private def hasNonAsciiChars(x: Token) = + x.rawText.trim.nonEmpty && !Pattern.compile( """\p{ASCII}+""", Pattern.DOTALL) + .matcher(x.text.trim).matches() + +} diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 76ba1ecca33ab..c54f8b72ebf42 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -140,5 +140,6 @@ + From 86d253ec4e2ed94c68687d575f9e2dfbb44463e1 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 16 Sep 2014 11:21:30 -0700 Subject: [PATCH 0706/1492] [SPARK-3527] [SQL] Strip the string message Author: Cheng Hao Closes #2392 from chenghao-intel/trim and squashes the following commits: e52024f [Cheng Hao] trim the string message --- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c551c7c9877e8..7dbaf7faff0c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -414,7 +414,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def simpleString: String = s"""== Physical Plan == |${stringOrError(executedPlan)} - """ + """.stripMargin.trim override def toString: String = // TODO previously will output RDD details by run (${stringOrError(toRdd.toDebugString)}) From 9d5fa763d8559ac412a18d7a2f43c4368a0af897 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Tue, 16 Sep 2014 11:39:57 -0700 Subject: [PATCH 0707/1492] [SPARK-3519] add distinct(n) to PySpark Added missing rdd.distinct(numPartitions) and associated tests Author: Matthew Farrellee Closes #2383 from mattf/SPARK-3519 and squashes the following commits: 30b837a [Matthew Farrellee] Combine test cases to save on JVM startups 6bc4a2c [Matthew Farrellee] [SPARK-3519] add distinct(n) to SchemaRDD in PySpark 7a17f2b [Matthew Farrellee] [SPARK-3519] add distinct(n) to PySpark --- python/pyspark/rdd.py | 4 ++-- python/pyspark/sql.py | 7 +++++-- python/pyspark/tests.py | 17 +++++++++++++++++ 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 21f182b0ff137..cb09c191bed71 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -301,7 +301,7 @@ def func(iterator): return ifilter(f, iterator) return self.mapPartitions(func, True) - def distinct(self): + def distinct(self, numPartitions=None): """ Return a new RDD containing the distinct elements in this RDD. @@ -309,7 +309,7 @@ def distinct(self): [1, 2, 3] """ return self.map(lambda x: (x, None)) \ - .reduceByKey(lambda x, _: x) \ + .reduceByKey(lambda x, _: x, numPartitions) \ .map(lambda (x, _): x) def sample(self, withReplacement, fraction, seed=None): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index fc9310fef318c..eac55cbe15193 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1694,8 +1694,11 @@ def coalesce(self, numPartitions, shuffle=False): rdd = self._jschema_rdd.coalesce(numPartitions, shuffle) return SchemaRDD(rdd, self.sql_ctx) - def distinct(self): - rdd = self._jschema_rdd.distinct() + def distinct(self, numPartitions=None): + if numPartitions is None: + rdd = self._jschema_rdd.distinct() + else: + rdd = self._jschema_rdd.distinct(numPartitions) return SchemaRDD(rdd, self.sql_ctx) def intersection(self, other): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f255b44359fec..0b3854347ad2e 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -587,6 +587,14 @@ def test_repartitionAndSortWithinPartitions(self): self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + def test_distinct(self): + rdd = self.sc.parallelize((1, 2, 3)*10, 10) + self.assertEquals(rdd.getNumPartitions(), 10) + self.assertEquals(rdd.distinct().count(), 3) + result = rdd.distinct(5) + self.assertEquals(result.getNumPartitions(), 5) + self.assertEquals(result.count(), 3) + class TestSQL(PySparkTestCase): @@ -636,6 +644,15 @@ def test_basic_functions(self): srdd.count() srdd.collect() + def test_distinct(self): + rdd = self.sc.parallelize(['{"a": 1}', '{"b": 2}', '{"c": 3}']*10, 10) + srdd = self.sqlCtx.jsonRDD(rdd) + self.assertEquals(srdd.getNumPartitions(), 10) + self.assertEquals(srdd.distinct().count(), 3) + result = srdd.distinct(5) + self.assertEquals(result.getNumPartitions(), 5) + self.assertEquals(result.count(), 3) + class TestIO(PySparkTestCase): From 7583699873fb4f252c6ce65db1096783ef438731 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Tue, 16 Sep 2014 11:40:28 -0700 Subject: [PATCH 0708/1492] [SPARK-3308][SQL] Ability to read JSON Arrays as tables This PR aims to support reading top level JSON arrays and take every element in such an array as a row (an empty array will not generate a row). JIRA: https://issues.apache.org/jira/browse/SPARK-3308 Author: Yin Huai Closes #2400 from yhuai/SPARK-3308 and squashes the following commits: 990077a [Yin Huai] Handle top level JSON arrays. --- .../org/apache/spark/sql/json/JsonRDD.scala | 10 +++++++--- .../org/apache/spark/sql/json/JsonSuite.scala | 17 +++++++++++++++++ .../apache/spark/sql/json/TestJsonData.scala | 7 +++++++ 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 873221835daf8..0f27fd13e7379 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -287,9 +287,13 @@ private[sql] object JsonRDD extends Logging { // the ObjectMapper will take the last value associated with this duplicate key. // For example: for {"key": 1, "key":2}, we will get "key"->2. val mapper = new ObjectMapper() - iter.map { record => - val parsed = scalafy(mapper.readValue(record, classOf[java.util.Map[String, Any]])) - parsed.asInstanceOf[Map[String, Any]] + iter.flatMap { record => + val parsed = mapper.readValue(record, classOf[Object]) match { + case map: java.util.Map[_, _] => scalafy(map).asInstanceOf[Map[String, Any]] :: Nil + case list: java.util.List[_] => scalafy(list).asInstanceOf[Seq[Map[String, Any]]] + } + + parsed } }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index b50d93855405a..685e788207725 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -622,4 +622,21 @@ class JsonSuite extends QueryTest { ("str1", Nil, "str4", 2) :: Nil ) } + + test("SPARK-3308 Read top level JSON arrays") { + val jsonSchemaRDD = jsonRDD(jsonArray) + jsonSchemaRDD.registerTempTable("jsonTable") + + checkAnswer( + sql( + """ + |select a, b, c + |from jsonTable + """.stripMargin), + ("str_a_1", null, null) :: + ("str_a_2", null, null) :: + (null, "str_b_3", null) :: + ("str_a_4", "str_b_4", "str_c_4") ::Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index 5f0b3959a63ad..fc833b8b54e4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -136,4 +136,11 @@ object TestJsonData { ] ]] }""" :: Nil) + + val jsonArray = + TestSQLContext.sparkContext.parallelize( + """[{"a":"str_a_1"}]""" :: + """[{"a":"str_a_2"}, {"b":"str_b_3"}]""" :: + """{"b":"str_b_4", "a":"str_a_4", "c":"str_c_4"}""" :: + """[]""" :: Nil) } From 30f288ae34a67307aa45b7aecbd0d02a0a14fe69 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 16 Sep 2014 11:42:26 -0700 Subject: [PATCH 0709/1492] [SPARK-2890][SQL] Allow reading of data when case insensitive resolution could cause possible ambiguity. Throwing an error in the constructor makes it possible to run queries, even when there is no actual ambiguity. Remove this check in favor of throwing an error in analysis when they query is actually is ambiguous. Also took the opportunity to add test cases that would have caught a subtle bug in my first attempt at fixing this and refactor some other test code. Author: Michael Armbrust Closes #2209 from marmbrus/sameNameStruct and squashes the following commits: 729cca4 [Michael Armbrust] Better tests. a003aeb [Michael Armbrust] Remove error (it'll be caught in analysis). --- .../spark/sql/catalyst/types/dataTypes.scala | 4 -- .../sql/hive/execution/HiveUdfSuite.scala | 67 ++++++++++++------- 2 files changed, 44 insertions(+), 27 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 70c6d06cf2534..49520b7678e90 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 @@ -308,13 +308,9 @@ case class StructField(name: String, dataType: DataType, nullable: Boolean) { object StructType { protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) - - private def validateFields(fields: Seq[StructField]): Boolean = - fields.map(field => field.name).distinct.size == fields.size } case class StructType(fields: Seq[StructField]) extends DataType { - require(StructType.validateFields(fields), "Found fields with the same name.") /** * Returns all field names in a [[Seq]]. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index b6b8592344ef5..cc125d539c3c2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -17,47 +17,68 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.test.TestHive -import org.apache.hadoop.conf.Configuration -import org.apache.spark.SparkContext._ +import java.io.{DataOutput, DataInput} import java.util -import org.apache.hadoop.fs.{FileSystem, Path} +import java.util.Properties + +import org.apache.spark.util.Utils + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.serde2.{SerDeStats, AbstractSerDe} -import org.apache.hadoop.io.{NullWritable, Writable} +import org.apache.hadoop.io.Writable import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorFactory, ObjectInspector} -import java.util.Properties + import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import scala.collection.JavaConversions._ -import java.io.{DataOutput, DataInput} import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject +import org.apache.spark.sql.Row +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ + +case class Fields(f1: Int, f2: Int, f3: Int, f4: Int, f5: Int) + /** * A test suite for Hive custom UDFs. */ class HiveUdfSuite extends HiveComparisonTest { - TestHive.sql( - """ + test("spark sql udf test that returns a struct") { + registerFunction("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) + assert(sql( + """ + |SELECT getStruct(1).f1, + | getStruct(1).f2, + | getStruct(1).f3, + | getStruct(1).f4, + | getStruct(1).f5 FROM src LIMIT 1 + """.stripMargin).first() === Row(1, 2, 3, 4, 5)) + } + + test("hive struct udf") { + sql( + """ |CREATE EXTERNAL TABLE hiveUdfTestTable ( | pair STRUCT |) |PARTITIONED BY (partition STRING) |ROW FORMAT SERDE '%s' |STORED AS SEQUENCEFILE - """.stripMargin.format(classOf[PairSerDe].getName) - ) - - TestHive.sql( - "ALTER TABLE hiveUdfTestTable ADD IF NOT EXISTS PARTITION(partition='testUdf') LOCATION '%s'" - .format(this.getClass.getClassLoader.getResource("data/files/testUdf").getFile) - ) - - TestHive.sql("CREATE TEMPORARY FUNCTION testUdf AS '%s'".format(classOf[PairUdf].getName)) - - TestHive.sql("SELECT testUdf(pair) FROM hiveUdfTestTable") - - TestHive.sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") + """. + stripMargin.format(classOf[PairSerDe].getName)) + + val location = Utils.getSparkClassLoader.getResource("data/files/testUdf").getFile + sql(s""" + ALTER TABLE hiveUdfTestTable + ADD IF NOT EXISTS PARTITION(partition='testUdf') + LOCATION '$location'""") + + sql(s"CREATE TEMPORARY FUNCTION testUdf AS '${classOf[PairUdf].getName}'") + sql("SELECT testUdf(pair) FROM hiveUdfTestTable") + sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") + } } class TestPair(x: Int, y: Int) extends Writable with Serializable { From 8e7ae477ba40a064d27cf149aa211ff6108fe239 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Tue, 16 Sep 2014 11:45:35 -0700 Subject: [PATCH 0710/1492] [SPARK-2314][SQL] Override collect and take in python library, and count in java library, with optimized versions. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit SchemaRDD overrides RDD functions, including collect, count, and take, with optimized versions making use of the query optimizer. The java and python interface classes wrapping SchemaRDD need to ensure the optimized versions are called as well. This patch overrides relevant calls in the python and java interfaces with optimized versions. Adds a new Row serialization pathway between python and java, based on JList[Array[Byte]] versus the existing RDD[Array[Byte]]. I wasn’t overjoyed about doing this, but I noticed that some QueryPlans implement optimizations in executeCollect(), which outputs an Array[Row] rather than the typical RDD[Row] that can be shipped to python using the existing serialization code. To me it made sense to ship the Array[Row] over to python directly instead of converting it back to an RDD[Row] just for the purpose of sending the Rows to python using the existing serialization code. Author: Aaron Staple Closes #1592 from staple/SPARK-2314 and squashes the following commits: 89ff550 [Aaron Staple] Merge with master. 6bb7b6c [Aaron Staple] Fix typo. b56d0ac [Aaron Staple] [SPARK-2314][SQL] Override count in JavaSchemaRDD, forwarding to SchemaRDD's count. 0fc9d40 [Aaron Staple] Fix comment typos. f03cdfa [Aaron Staple] [SPARK-2314][SQL] Override collect and take in sql.py, forwarding to SchemaRDD's collect. --- .../apache/spark/api/python/PythonRDD.scala | 2 +- python/pyspark/sql.py | 47 +++++++++++++++++-- .../org/apache/spark/sql/SchemaRDD.scala | 37 ++++++++++----- .../spark/sql/api/java/JavaSchemaRDD.scala | 2 + 4 files changed, 71 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index d5002fa02992b..12b345a8fa7c3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -776,7 +776,7 @@ private[spark] object PythonRDD extends Logging { } /** - * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by + * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by * PySpark. */ def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index eac55cbe15193..621a556ec6356 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -30,6 +30,7 @@ from pyspark.rdd import RDD, PipelinedRDD from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer from pyspark.storagelevel import StorageLevel +from pyspark.traceback_utils import SCCallSiteSync from itertools import chain, ifilter, imap @@ -1550,6 +1551,18 @@ def id(self): self._id = self._jrdd.id() return self._id + def limit(self, num): + """Limit the result count to the number specified. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.limit(2).collect() + [Row(field1=1, field2=u'row1'), Row(field1=2, field2=u'row2')] + >>> srdd.limit(0).collect() + [] + """ + rdd = self._jschema_rdd.baseSchemaRDD().limit(num).toJavaSchemaRDD() + return SchemaRDD(rdd, self.sql_ctx) + def saveAsParquetFile(self, path): """Save the contents as a Parquet file, preserving the schema. @@ -1626,15 +1639,39 @@ def count(self): return self._jschema_rdd.count() def collect(self): - """ - Return a list that contains all of the rows in this RDD. + """Return a list that contains all of the rows in this RDD. - Each object in the list is on Row, the fields can be accessed as + Each object in the list is a Row, the fields can be accessed as attributes. + + Unlike the base RDD implementation of collect, this implementation + leverages the query optimizer to perform a collect on the SchemaRDD, + which supports features such as filter pushdown. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.collect() + [Row(field1=1, field2=u'row1'), ..., Row(field1=3, field2=u'row3')] """ - rows = RDD.collect(self) + with SCCallSiteSync(self.context) as css: + bytesInJava = self._jschema_rdd.baseSchemaRDD().collectToPython().iterator() cls = _create_cls(self.schema()) - return map(cls, rows) + return map(cls, self._collect_iterator_through_file(bytesInJava)) + + def take(self, num): + """Take the first num rows of the RDD. + + Each object in the list is a Row, the fields can be accessed as + attributes. + + Unlike the base RDD implementation of take, this implementation + leverages the query optimizer to perform a collect on a SchemaRDD, + which supports features such as filter pushdown. + + >>> srdd = sqlCtx.inferSchema(rdd) + >>> srdd.take(2) + [Row(field1=1, field2=u'row1'), Row(field1=2, field2=u'row2')] + """ + return self.limit(num).collect() # Convert each object in the RDD to a Row with the right class # for this SchemaRDD, so that fields can be accessed as attributes. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index d2ceb4a2b0b25..3bc5dce095511 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -377,15 +377,15 @@ class SchemaRDD( def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan) /** - * Converts a JavaRDD to a PythonRDD. It is used by pyspark. + * Helper for converting a Row to a simple Array suitable for pyspark serialization. */ - private[sql] def javaToPython: JavaRDD[Array[Byte]] = { + private def rowToJArray(row: Row, structType: StructType): Array[Any] = { import scala.collection.Map def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { case (null, _) => null - case (obj: Row, struct: StructType) => rowToArray(obj, struct) + case (obj: Row, struct: StructType) => rowToJArray(obj, struct) case (seq: Seq[Any], array: ArrayType) => seq.map(x => toJava(x, array.elementType)).asJava @@ -402,22 +402,37 @@ class SchemaRDD( case (other, _) => other } - def rowToArray(row: Row, structType: StructType): Array[Any] = { - val fields = structType.fields.map(field => field.dataType) - row.zip(fields).map { - case (obj, dataType) => toJava(obj, dataType) - }.toArray - } + val fields = structType.fields.map(field => field.dataType) + row.zip(fields).map { + case (obj, dataType) => toJava(obj, dataType) + }.toArray + } + /** + * Converts a JavaRDD to a PythonRDD. It is used by pyspark. + */ + private[sql] def javaToPython: JavaRDD[Array[Byte]] = { val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => - rowToArray(row, rowSchema) + rowToJArray(row, rowSchema) }.grouped(100).map(batched => pickle.dumps(batched.toArray)) } } + /** + * Serializes the Array[Row] returned by SchemaRDD's optimized collect(), using the same + * format as javaToPython. It is used by pyspark. + */ + private[sql] def collectToPython: JList[Array[Byte]] = { + val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) + val pickle = new Pickler + new java.util.ArrayList(collect().map { row => + rowToJArray(row, rowSchema) + }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable) + } + /** * Creates SchemaRDD by applying own schema to derived RDD. Typically used to wrap return value * of base RDD functions that do not change schema. @@ -433,7 +448,7 @@ class SchemaRDD( } // ======================================================================= - // Overriden RDD actions + // Overridden RDD actions // ======================================================================= override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() 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 4d799b4038fdd..e7faba0c7f620 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 @@ -112,6 +112,8 @@ class JavaSchemaRDD( new java.util.ArrayList(arr) } + override def count(): Long = baseSchemaRDD.count + override def take(num: Int): JList[Row] = { import scala.collection.JavaConversions._ val arr: java.util.Collection[Row] = baseSchemaRDD.take(num).toSeq.map(new Row(_)) From df90e81fd383c0d89dee6db16d5520def9190c56 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Tue, 16 Sep 2014 11:48:20 -0700 Subject: [PATCH 0711/1492] [Docs] minor punctuation fix Author: Nicholas Chammas Closes #2414 from nchammas/patch-1 and squashes the following commits: 14664bf [Nicholas Chammas] [Docs] minor punctuation fix --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index b05bbfb5a594c..8dd8b70696aa2 100644 --- a/README.md +++ b/README.md @@ -75,7 +75,7 @@ can be run using: ./dev/run-tests Please see the guidance on how to -[run all automated tests](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting) +[run all automated tests](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting). ## A Note About Hadoop Versions From 84073eb1172dc959936149265378f6e24d303685 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 16 Sep 2014 11:51:46 -0700 Subject: [PATCH 0712/1492] [SQL][DOCS] Improve section on thrift-server Taken from liancheng's updates. Merged conflicts with #2316. Author: Michael Armbrust Closes #2384 from marmbrus/sqlDocUpdate and squashes the following commits: 2db6319 [Michael Armbrust] @liancheng's updates --- docs/sql-programming-guide.md | 58 ++++++++++++++++++++++++----------- 1 file changed, 40 insertions(+), 18 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 8d41fdec699e9..c498b41c43380 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -128,7 +128,7 @@ feature parity with a HiveContext.
    -The specific variant of SQL that is used to parse queries can also be selected using the +The specific variant of SQL that is used to parse queries can also be selected using the `spark.sql.dialect` option. This parameter can be changed using either the `setConf` method on a SQLContext or by using a `SET key=value` command in SQL. For a SQLContext, the only dialect available is "sql" which uses a simple SQL parser provided by Spark SQL. In a HiveContext, the @@ -139,7 +139,7 @@ default is "hiveql", though "sql" is also available. Since the HiveQL parser is Spark SQL supports operating on a variety of data sources through the `SchemaRDD` interface. A SchemaRDD can be operated on as normal RDDs and can also be registered as a temporary table. -Registering a SchemaRDD as a table allows you to run SQL queries over its data. This section +Registering a SchemaRDD as a table allows you to run SQL queries over its data. This section describes the various methods for loading data into a SchemaRDD. ## RDDs @@ -152,7 +152,7 @@ while writing your Spark application. The second method for creating SchemaRDDs is through a programmatic interface that allows you to construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows you to construct SchemaRDDs when the columns and their types are not known until runtime. - + ### Inferring the Schema Using Reflection
    @@ -193,7 +193,7 @@ teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
    Spark SQL supports automatically converting an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) -into a Schema RDD. The BeanInfo, obtained using reflection, defines the schema of the table. +into a Schema RDD. The BeanInfo, obtained using reflection, defines the schema of the table. Currently, Spark SQL does not support JavaBeans that contain nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a class that implements Serializable and has getters and setters for all of its fields. @@ -480,7 +480,7 @@ for name in names.collect(): [Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. +of the original data. ### Loading Data Programmatically @@ -562,7 +562,7 @@ for teenName in teenNames.collect():
    -
    +
    ### Configuration @@ -808,7 +808,7 @@ memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove Note that if you call `cache` rather than `cacheTable`, tables will _not_ be cached using the in-memory columnar format, and therefore `cacheTable` is strongly recommended for this use case. -Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running +Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -881,10 +881,32 @@ To start the JDBC server, run the following in the Spark directory: ./sbin/start-thriftserver.sh -The default port the server listens on is 10000. To listen on customized host and port, please set -the `HIVE_SERVER2_THRIFT_PORT` and `HIVE_SERVER2_THRIFT_BIND_HOST` environment variables. You may -run `./sbin/start-thriftserver.sh --help` for a complete list of all available options. Now you can -use beeline to test the Thrift JDBC server: +This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to +specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of +all available options. By default, the server listens on localhost:10000. You may override this +bahaviour via either environment variables, i.e.: + +{% highlight bash %} +export HIVE_SERVER2_THRIFT_PORT= +export HIVE_SERVER2_THRIFT_BIND_HOST= +./sbin/start-thriftserver.sh \ + --master \ + ... +``` +{% endhighlight %} + +or system properties: + +{% highlight bash %} +./sbin/start-thriftserver.sh \ + --hiveconf hive.server2.thrift.port= \ + --hiveconf hive.server2.thrift.bind.host= \ + --master + ... +``` +{% endhighlight %} + +Now you can use beeline to test the Thrift JDBC server: ./bin/beeline @@ -930,7 +952,7 @@ SQL deprecates this property in favor of `spark.sql.shuffle.partitions`, whose d is 200. Users may customize this property via `SET`: SET spark.sql.shuffle.partitions=10; - SELECT page, count(*) c + SELECT page, count(*) c FROM logs_last_month_cached GROUP BY page ORDER BY c DESC LIMIT 10; @@ -1139,7 +1161,7 @@ evaluated by the SQL execution engine. A full list of the functions supported c
    All data types of Spark SQL are located in the package `org.apache.spark.sql`. -You can access them by doing +You can access them by doing {% highlight scala %} import org.apache.spark.sql._ {% endhighlight %} @@ -1245,7 +1267,7 @@ import org.apache.spark.sql._
    - - + + + + + + From 14f222f7f76cc93633aae27a94c0e556e289ec56 Mon Sep 17 00:00:00 2001 From: Qiping Li Date: Thu, 9 Oct 2014 01:36:58 -0700 Subject: [PATCH 0941/1492] [SPARK-3158][MLLIB]Avoid 1 extra aggregation for DecisionTree training Currently, the implementation does one unnecessary aggregation step. The aggregation step for level L (to choose splits) gives enough information to set the predictions of any leaf nodes at level L+1. We can use that info and skip the aggregation step for the last level of the tree (which only has leaf nodes). ### Implementation Details Each node now has a `impurity` field and the `predict` is changed from type `Double` to type `Predict`(this can be used to compute predict probability in the future) When compute best splits for each node, we also compute impurity and predict for the child nodes, which is used to constructed newly allocated child nodes. So at level L, we have set impurity and predict for nodes at level L +1. If level L+1 is the last level, then we can avoid aggregation. What's more, calculation of parent impurity in Top nodes for each tree needs to be treated differently because we have to compute impurity and predict for them first. In `binsToBestSplit`, if current node is top node(level == 0), we calculate impurity and predict first. after finding best split, top node's predict and impurity is set to the calculated value. Non-top nodes's impurity and predict are already calculated and don't need to be recalculated again. I have considered to add a initialization step to set top nodes' impurity and predict and then we can treat all nodes in the same way, but this will need a lot of duplication of code(all the code to do seq operation(BinSeqOp) needs to be duplicated), so I choose the current way. CC mengxr manishamde jkbradley, please help me review this, thanks. Author: Qiping Li Closes #2708 from chouqin/avoid-agg and squashes the following commits: 8e269ea [Qiping Li] adjust code and comments eefeef1 [Qiping Li] adjust comments and check child nodes' impurity c41b1b6 [Qiping Li] fix pyspark unit test 7ad7a71 [Qiping Li] fix unit test 822c912 [Qiping Li] add comments and unit test e41d715 [Qiping Li] fix bug in test suite 6cc0333 [Qiping Li] SPARK-3158: Avoid 1 extra aggregation for DecisionTree training --- .../spark/mllib/tree/DecisionTree.scala | 97 +++++++++++------ .../tree/model/InformationGainStats.scala | 9 +- .../apache/spark/mllib/tree/model/Node.scala | 37 +++++-- .../spark/mllib/tree/DecisionTreeSuite.scala | 102 ++++++++++++++++-- 4 files changed, 197 insertions(+), 48 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index b311d10023894..03eeaa707715b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -532,6 +532,14 @@ object DecisionTree extends Serializable with Logging { Some(mutableNodeToFeatures.toMap) } + // array of nodes to train indexed by node index in group + val nodes = new Array[Node](numNodes) + nodesForGroup.foreach { case (treeIndex, nodesForTree) => + nodesForTree.foreach { node => + nodes(treeToNodeToIndexInfo(treeIndex)(node.id).nodeIndexInGroup) = node + } + } + // Calculate best splits for all nodes in the group timer.start("chooseSplits") @@ -568,7 +576,7 @@ object DecisionTree extends Serializable with Logging { // find best split for each node val (split: Split, stats: InformationGainStats, predict: Predict) = - binsToBestSplit(aggStats, splits, featuresForNode) + binsToBestSplit(aggStats, splits, featuresForNode, nodes(nodeIndex)) (nodeIndex, (split, stats, predict)) }.collectAsMap() @@ -587,17 +595,30 @@ object DecisionTree extends Serializable with Logging { // Extract info for this node. Create children if not leaf. val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth) assert(node.id == nodeIndex) - node.predict = predict.predict + node.predict = predict node.isLeaf = isLeaf node.stats = Some(stats) + node.impurity = stats.impurity logDebug("Node = " + node) if (!isLeaf) { node.split = Some(split) - node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex))) - node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex))) - nodeQueue.enqueue((treeIndex, node.leftNode.get)) - nodeQueue.enqueue((treeIndex, node.rightNode.get)) + val childIsLeaf = (Node.indexToLevel(nodeIndex) + 1) == metadata.maxDepth + val leftChildIsLeaf = childIsLeaf || (stats.leftImpurity == 0.0) + val rightChildIsLeaf = childIsLeaf || (stats.rightImpurity == 0.0) + node.leftNode = Some(Node(Node.leftChildIndex(nodeIndex), + stats.leftPredict, stats.leftImpurity, leftChildIsLeaf)) + node.rightNode = Some(Node(Node.rightChildIndex(nodeIndex), + stats.rightPredict, stats.rightImpurity, rightChildIsLeaf)) + + // enqueue left child and right child if they are not leaves + if (!leftChildIsLeaf) { + nodeQueue.enqueue((treeIndex, node.leftNode.get)) + } + if (!rightChildIsLeaf) { + nodeQueue.enqueue((treeIndex, node.rightNode.get)) + } + logDebug("leftChildIndex = " + node.leftNode.get.id + ", impurity = " + stats.leftImpurity) logDebug("rightChildIndex = " + node.rightNode.get.id + @@ -617,7 +638,8 @@ object DecisionTree extends Serializable with Logging { private def calculateGainForSplit( leftImpurityCalculator: ImpurityCalculator, rightImpurityCalculator: ImpurityCalculator, - metadata: DecisionTreeMetadata): InformationGainStats = { + metadata: DecisionTreeMetadata, + impurity: Double): InformationGainStats = { val leftCount = leftImpurityCalculator.count val rightCount = rightImpurityCalculator.count @@ -630,11 +652,6 @@ object DecisionTree extends Serializable with Logging { val totalCount = leftCount + rightCount - val parentNodeAgg = leftImpurityCalculator.copy - parentNodeAgg.add(rightImpurityCalculator) - - val impurity = parentNodeAgg.calculate() - val leftImpurity = leftImpurityCalculator.calculate() // Note: This equals 0 if count = 0 val rightImpurity = rightImpurityCalculator.calculate() @@ -649,7 +666,18 @@ object DecisionTree extends Serializable with Logging { return InformationGainStats.invalidInformationGainStats } - new InformationGainStats(gain, impurity, leftImpurity, rightImpurity) + // calculate left and right predict + val leftPredict = calculatePredict(leftImpurityCalculator) + val rightPredict = calculatePredict(rightImpurityCalculator) + + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, + leftPredict, rightPredict) + } + + private def calculatePredict(impurityCalculator: ImpurityCalculator): Predict = { + val predict = impurityCalculator.predict + val prob = impurityCalculator.prob(predict) + new Predict(predict, prob) } /** @@ -657,17 +685,17 @@ object DecisionTree extends Serializable with Logging { * Note that this function is called only once for each node. * @param leftImpurityCalculator left node aggregates for a split * @param rightImpurityCalculator right node aggregates for a split - * @return predict value for current node + * @return predict value and impurity for current node */ - private def calculatePredict( + private def calculatePredictImpurity( leftImpurityCalculator: ImpurityCalculator, - rightImpurityCalculator: ImpurityCalculator): Predict = { + rightImpurityCalculator: ImpurityCalculator): (Predict, Double) = { val parentNodeAgg = leftImpurityCalculator.copy parentNodeAgg.add(rightImpurityCalculator) - val predict = parentNodeAgg.predict - val prob = parentNodeAgg.prob(predict) + val predict = calculatePredict(parentNodeAgg) + val impurity = parentNodeAgg.calculate() - new Predict(predict, prob) + (predict, impurity) } /** @@ -678,10 +706,16 @@ object DecisionTree extends Serializable with Logging { private def binsToBestSplit( binAggregates: DTStatsAggregator, splits: Array[Array[Split]], - featuresForNode: Option[Array[Int]]): (Split, InformationGainStats, Predict) = { + featuresForNode: Option[Array[Int]], + node: Node): (Split, InformationGainStats, Predict) = { - // calculate predict only once - var predict: Option[Predict] = None + // calculate predict and impurity if current node is top node + val level = Node.indexToLevel(node.id) + var predictWithImpurity: Option[(Predict, Double)] = if (level == 0) { + None + } else { + Some((node.predict, node.impurity)) + } // For each (feature, split), calculate the gain, and select the best (feature, split). val (bestSplit, bestSplitStats) = @@ -708,9 +742,10 @@ object DecisionTree extends Serializable with Logging { val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) - predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) + predictWithImpurity = Some(predictWithImpurity.getOrElse( + calculatePredictImpurity(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, - rightChildStats, binAggregates.metadata) + rightChildStats, binAggregates.metadata, predictWithImpurity.get._2) (splitIdx, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) @@ -722,9 +757,10 @@ object DecisionTree extends Serializable with Logging { Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) - predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) + predictWithImpurity = Some(predictWithImpurity.getOrElse( + calculatePredictImpurity(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, - rightChildStats, binAggregates.metadata) + rightChildStats, binAggregates.metadata, predictWithImpurity.get._2) (splitIndex, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) @@ -794,9 +830,10 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) - predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) + predictWithImpurity = Some(predictWithImpurity.getOrElse( + calculatePredictImpurity(leftChildStats, rightChildStats))) val gainStats = calculateGainForSplit(leftChildStats, - rightChildStats, binAggregates.metadata) + rightChildStats, binAggregates.metadata, predictWithImpurity.get._2) (splitIndex, gainStats) }.maxBy(_._2.gain) val categoriesForSplit = @@ -807,9 +844,7 @@ object DecisionTree extends Serializable with Logging { } }.maxBy(_._2.gain) - assert(predict.isDefined, "must calculate predict for each node") - - (bestSplit, bestSplitStats, predict.get) + (bestSplit, bestSplitStats, predictWithImpurity.get._1) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index a89e71e115806..9a50ecb550c38 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -26,13 +26,17 @@ import org.apache.spark.annotation.DeveloperApi * @param impurity current node impurity * @param leftImpurity left node impurity * @param rightImpurity right node impurity + * @param leftPredict left node predict + * @param rightPredict right node predict */ @DeveloperApi class InformationGainStats( val gain: Double, val impurity: Double, val leftImpurity: Double, - val rightImpurity: Double) extends Serializable { + val rightImpurity: Double, + val leftPredict: Predict, + val rightPredict: Predict) extends Serializable { override def toString = { "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" @@ -58,5 +62,6 @@ private[tree] object InformationGainStats { * denote that current split doesn't satisfies minimum info gain or * minimum number of instances per node. */ - val invalidInformationGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0) + val invalidInformationGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0, + new Predict(0.0, 0.0), new Predict(0.0, 0.0)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 56c3e25d9285f..2179da8dbe03e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -32,7 +32,8 @@ import org.apache.spark.mllib.linalg.Vector * * @param id integer node id, from 1 * @param predict predicted value at the node - * @param isLeaf whether the leaf is a node + * @param impurity current node impurity + * @param isLeaf whether the node is a leaf * @param split split to calculate left and right nodes * @param leftNode left child * @param rightNode right child @@ -41,7 +42,8 @@ import org.apache.spark.mllib.linalg.Vector @DeveloperApi class Node ( val id: Int, - var predict: Double, + var predict: Predict, + var impurity: Double, var isLeaf: Boolean, var split: Option[Split], var leftNode: Option[Node], @@ -49,7 +51,7 @@ class Node ( var stats: Option[InformationGainStats]) extends Serializable with Logging { override def toString = "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + - "split = " + split + ", stats = " + stats + "impurity = " + impurity + "split = " + split + ", stats = " + stats /** * build the left node and right nodes if not leaf @@ -62,6 +64,7 @@ class Node ( logDebug("id = " + id + ", split = " + split) logDebug("stats = " + stats) logDebug("predict = " + predict) + logDebug("impurity = " + impurity) if (!isLeaf) { leftNode = Some(nodes(Node.leftChildIndex(id))) rightNode = Some(nodes(Node.rightChildIndex(id))) @@ -77,7 +80,7 @@ class Node ( */ def predict(features: Vector) : Double = { if (isLeaf) { - predict + predict.predict } else{ if (split.get.featureType == Continuous) { if (features(split.get.feature) <= split.get.threshold) { @@ -109,7 +112,7 @@ class Node ( } else { Some(rightNode.get.deepCopy()) } - new Node(id, predict, isLeaf, split, leftNodeCopy, rightNodeCopy, stats) + new Node(id, predict, impurity, isLeaf, split, leftNodeCopy, rightNodeCopy, stats) } /** @@ -154,7 +157,7 @@ class Node ( } val prefix: String = " " * indentFactor if (isLeaf) { - prefix + s"Predict: $predict\n" + prefix + s"Predict: ${predict.predict}\n" } else { prefix + s"If ${splitToString(split.get, left=true)}\n" + leftNode.get.subtreeToString(indentFactor + 1) + @@ -170,7 +173,27 @@ private[tree] object Node { /** * Return a node with the given node id (but nothing else set). */ - def emptyNode(nodeIndex: Int): Node = new Node(nodeIndex, 0, false, None, None, None, None) + def emptyNode(nodeIndex: Int): Node = new Node(nodeIndex, new Predict(Double.MinValue), -1.0, + false, None, None, None, None) + + /** + * Construct a node with nodeIndex, predict, impurity and isLeaf parameters. + * This is used in `DecisionTree.findBestSplits` to construct child nodes + * after finding the best splits for parent nodes. + * Other fields are set at next level. + * @param nodeIndex integer node id, from 1 + * @param predict predicted value at the node + * @param impurity current node impurity + * @param isLeaf whether the node is a leaf + * @return new node instance + */ + def apply( + nodeIndex: Int, + predict: Predict, + impurity: Double, + isLeaf: Boolean): Node = { + new Node(nodeIndex, predict, impurity, isLeaf, None, None, None, None) + } /** * Return the index of the left child of this node. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index a48ed71a1c5fc..98a72b0c4d750 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -253,7 +253,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = rootNode.stats.get assert(stats.gain > 0) - assert(rootNode.predict === 1) + assert(rootNode.predict.predict === 1) assert(stats.impurity > 0.2) } @@ -282,7 +282,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val stats = rootNode.stats.get assert(stats.gain > 0) - assert(rootNode.predict === 0.6) + assert(rootNode.predict.predict === 0.6) assert(stats.impurity > 0.2) } @@ -352,7 +352,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.gain === 0) assert(stats.leftImpurity === 0) assert(stats.rightImpurity === 0) - assert(rootNode.predict === 1) + assert(rootNode.predict.predict === 1) } test("Binary classification stump with fixed label 0 for Entropy") { @@ -377,7 +377,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.gain === 0) assert(stats.leftImpurity === 0) assert(stats.rightImpurity === 0) - assert(rootNode.predict === 0) + assert(rootNode.predict.predict === 0) } test("Binary classification stump with fixed label 1 for Entropy") { @@ -402,7 +402,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats.gain === 0) assert(stats.leftImpurity === 0) assert(stats.rightImpurity === 0) - assert(rootNode.predict === 1) + assert(rootNode.predict.predict === 1) } test("Second level node building with vs. without groups") { @@ -471,7 +471,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(stats1.impurity === stats2.impurity) assert(stats1.leftImpurity === stats2.leftImpurity) assert(stats1.rightImpurity === stats2.rightImpurity) - assert(children1(i).predict === children2(i).predict) + assert(children1(i).predict.predict === children2(i).predict.predict) } } @@ -646,7 +646,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val model = DecisionTree.train(rdd, strategy) assert(model.topNode.isLeaf) - assert(model.topNode.predict == 0.0) + assert(model.topNode.predict.predict == 0.0) val predicts = rdd.map(p => model.predict(p.features)).collect() predicts.foreach { predict => assert(predict == 0.0) @@ -693,7 +693,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val model = DecisionTree.train(input, strategy) assert(model.topNode.isLeaf) - assert(model.topNode.predict == 0.0) + assert(model.topNode.predict.predict == 0.0) val predicts = input.map(p => model.predict(p.features)).collect() predicts.foreach { predict => assert(predict == 0.0) @@ -705,6 +705,92 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val gain = rootNode.stats.get assert(gain == InformationGainStats.invalidInformationGainStats) } + + test("Avoid aggregation on the last level") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) + arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) + arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, + numClassesForClassification = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue leaf nodes into node queue + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } + + test("Avoid aggregation if impurity is 0.0") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)) + arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)) + arr(3) = new LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0)) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClassesForClassification = 2, categoricalFeaturesInfo = Map(0 -> 3)) + val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) + val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) + + val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) + val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + + val topNode = Node.emptyNode(nodeIndex = 1) + assert(topNode.predict.predict === Double.MinValue) + assert(topNode.impurity === -1.0) + assert(topNode.isLeaf === false) + + val nodesForGroup = Map((0, Array(topNode))) + val treeToNodeToIndexInfo = Map((0, Map( + (topNode.id, new RandomForest.NodeIndexInfo(0, None)) + ))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(topNode), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) + + // don't enqueue a node into node queue if its impurity is 0.0 + assert(nodeQueue.isEmpty) + + // set impurity and predict for topNode + assert(topNode.predict.predict !== Double.MinValue) + assert(topNode.impurity !== -1.0) + + // set impurity and predict for child nodes + assert(topNode.leftNode.get.predict.predict === 0.0) + assert(topNode.rightNode.get.predict.predict === 1.0) + assert(topNode.leftNode.get.impurity === 0.0) + assert(topNode.rightNode.get.impurity === 0.0) + } } object DecisionTreeSuite { From 1e0aa4deba65aa1241b9a30edb82665eae27242f Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Thu, 9 Oct 2014 09:22:32 -0700 Subject: [PATCH 0942/1492] [Minor] use norm operator after breeze 0.10 upgrade cc mengxr Author: GuoQiang Li Closes #2730 from witgo/SPARK-3856 and squashes the following commits: 2cffce1 [GuoQiang Li] use norm operator after breeze 0.10 upgrade --- .../spark/mllib/feature/NormalizerSuite.scala | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala index fb76dccfdf79e..2bf9d9816ae45 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/NormalizerSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.feature import org.scalatest.FunSuite +import breeze.linalg.{norm => brzNorm} + import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors} import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -50,10 +52,10 @@ class NormalizerSuite extends FunSuite with LocalSparkContext { assert((data1, data1RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(data1(0).toBreeze.norm(1) ~== 1.0 absTol 1E-5) - assert(data1(2).toBreeze.norm(1) ~== 1.0 absTol 1E-5) - assert(data1(3).toBreeze.norm(1) ~== 1.0 absTol 1E-5) - assert(data1(4).toBreeze.norm(1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(0).toBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(2).toBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(3).toBreeze, 1) ~== 1.0 absTol 1E-5) + assert(brzNorm(data1(4).toBreeze, 1) ~== 1.0 absTol 1E-5) assert(data1(0) ~== Vectors.sparse(3, Seq((0, -0.465116279), (1, 0.53488372))) absTol 1E-5) assert(data1(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) @@ -77,10 +79,10 @@ class NormalizerSuite extends FunSuite with LocalSparkContext { assert((data2, data2RDD.collect()).zipped.forall((v1, v2) => v1 ~== v2 absTol 1E-5)) - assert(data2(0).toBreeze.norm(2) ~== 1.0 absTol 1E-5) - assert(data2(2).toBreeze.norm(2) ~== 1.0 absTol 1E-5) - assert(data2(3).toBreeze.norm(2) ~== 1.0 absTol 1E-5) - assert(data2(4).toBreeze.norm(2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(0).toBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(2).toBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(3).toBreeze, 2) ~== 1.0 absTol 1E-5) + assert(brzNorm(data2(4).toBreeze, 2) ~== 1.0 absTol 1E-5) assert(data2(0) ~== Vectors.sparse(3, Seq((0, -0.65617871), (1, 0.75460552))) absTol 1E-5) assert(data2(1) ~== Vectors.dense(0.0, 0.0, 0.0) absTol 1E-5) From 73bf3f2e0c03216aa29c25fea2d97205b5977903 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 9 Oct 2014 11:27:21 -0700 Subject: [PATCH 0943/1492] [SPARK-3741] Make ConnectionManager propagate errors properly and add mo... ...re logs to avoid Executors swallowing errors This PR made the following changes: * Register a callback to `Connection` so that the error will be propagated properly. * Add more logs so that the errors won't be swallowed by Executors. * Use trySuccess/tryFailure because `Promise` doesn't allow to call success/failure more than once. Author: zsxwing Closes #2593 from zsxwing/SPARK-3741 and squashes the following commits: 1d5aed5 [zsxwing] Fix naming 0b8a61c [zsxwing] Merge branch 'master' into SPARK-3741 764aec5 [zsxwing] [SPARK-3741] Make ConnectionManager propagate errors properly and add more logs to avoid Executors swallowing errors --- .../apache/spark/network/nio/Connection.scala | 35 +-- .../spark/network/nio/ConnectionManager.scala | 206 +++++++++++++----- 2 files changed, 172 insertions(+), 69 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index f368209980f93..4f6f5e235811d 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -20,11 +20,14 @@ package org.apache.spark.network.nio import java.net._ import java.nio._ import java.nio.channels._ +import java.util.concurrent.ConcurrentLinkedQueue import java.util.LinkedList import org.apache.spark._ +import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.util.control.NonFatal private[nio] abstract class Connection(val channel: SocketChannel, val selector: Selector, @@ -51,7 +54,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, @volatile private var closed = false var onCloseCallback: Connection => Unit = null - var onExceptionCallback: (Connection, Exception) => Unit = null + val onExceptionCallbacks = new ConcurrentLinkedQueue[(Connection, Throwable) => Unit] var onKeyInterestChangeCallback: (Connection, Int) => Unit = null val remoteAddress = getRemoteAddress() @@ -130,20 +133,24 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, onCloseCallback = callback } - def onException(callback: (Connection, Exception) => Unit) { - onExceptionCallback = callback + def onException(callback: (Connection, Throwable) => Unit) { + onExceptionCallbacks.add(callback) } def onKeyInterestChange(callback: (Connection, Int) => Unit) { onKeyInterestChangeCallback = callback } - def callOnExceptionCallback(e: Exception) { - if (onExceptionCallback != null) { - onExceptionCallback(this, e) - } else { - logError("Error in connection to " + getRemoteConnectionManagerId() + - " and OnExceptionCallback not registered", e) + def callOnExceptionCallbacks(e: Throwable) { + onExceptionCallbacks foreach { + callback => + try { + callback(this, e) + } catch { + case NonFatal(e) => { + logWarning("Ignored error in onExceptionCallback", e) + } + } } } @@ -323,7 +330,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } catch { case e: Exception => { logError("Error connecting to " + address, e) - callOnExceptionCallback(e) + callOnExceptionCallbacks(e) } } } @@ -348,7 +355,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } catch { case e: Exception => { logWarning("Error finishing connection to " + address, e) - callOnExceptionCallback(e) + callOnExceptionCallbacks(e) } } true @@ -393,7 +400,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } catch { case e: Exception => { logWarning("Error writing in connection to " + getRemoteConnectionManagerId(), e) - callOnExceptionCallback(e) + callOnExceptionCallbacks(e) close() return false } @@ -420,7 +427,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, case e: Exception => logError("Exception while reading SendingConnection to " + getRemoteConnectionManagerId(), e) - callOnExceptionCallback(e) + callOnExceptionCallbacks(e) close() } @@ -577,7 +584,7 @@ private[spark] class ReceivingConnection( } catch { case e: Exception => { logWarning("Error reading from connection to " + getRemoteConnectionManagerId(), e) - callOnExceptionCallback(e) + callOnExceptionCallbacks(e) close() return false } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 01cd27a907eea..6b00190c5eccc 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -34,6 +34,8 @@ import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.Utils +import scala.util.Try +import scala.util.control.NonFatal private[nio] class ConnectionManager( port: Int, @@ -51,14 +53,23 @@ private[nio] class ConnectionManager( class MessageStatus( val message: Message, val connectionManagerId: ConnectionManagerId, - completionHandler: MessageStatus => Unit) { + completionHandler: Try[Message] => Unit) { - /** This is non-None if message has been ack'd */ - var ackMessage: Option[Message] = None + def success(ackMessage: Message) { + if (ackMessage == null) { + failure(new NullPointerException) + } + else { + completionHandler(scala.util.Success(ackMessage)) + } + } - def markDone(ackMessage: Option[Message]) { - this.ackMessage = ackMessage - completionHandler(this) + def failWithoutAck() { + completionHandler(scala.util.Failure(new IOException("Failed without being ACK'd"))) + } + + def failure(e: Throwable) { + completionHandler(scala.util.Failure(e)) } } @@ -72,14 +83,32 @@ private[nio] class ConnectionManager( conf.getInt("spark.core.connection.handler.threads.max", 60), conf.getInt("spark.core.connection.handler.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-message-executor")) + Utils.namedThreadFactory("handle-message-executor")) { + + override def afterExecute(r: Runnable, t: Throwable): Unit = { + super.afterExecute(r, t) + if (t != null && NonFatal(t)) { + logError("Error in handleMessageExecutor is not handled properly", t) + } + } + + } private val handleReadWriteExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.io.threads.min", 4), conf.getInt("spark.core.connection.io.threads.max", 32), conf.getInt("spark.core.connection.io.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-read-write-executor")) + Utils.namedThreadFactory("handle-read-write-executor")) { + + override def afterExecute(r: Runnable, t: Throwable): Unit = { + super.afterExecute(r, t) + if (t != null && NonFatal(t)) { + logError("Error in handleReadWriteExecutor is not handled properly", t) + } + } + + } // Use a different, yet smaller, thread pool - infrequently used with very short lived tasks : // which should be executed asap @@ -153,17 +182,24 @@ private[nio] class ConnectionManager( } handleReadWriteExecutor.execute(new Runnable { override def run() { - var register: Boolean = false try { - register = conn.write() - } finally { - writeRunnableStarted.synchronized { - writeRunnableStarted -= key - val needReregister = register || conn.resetForceReregister() - if (needReregister && conn.changeInterestForWrite()) { - conn.registerInterest() + var register: Boolean = false + try { + register = conn.write() + } finally { + writeRunnableStarted.synchronized { + writeRunnableStarted -= key + val needReregister = register || conn.resetForceReregister() + if (needReregister && conn.changeInterestForWrite()) { + conn.registerInterest() + } } } + } catch { + case NonFatal(e) => { + logError("Error when writing to " + conn.getRemoteConnectionManagerId(), e) + conn.callOnExceptionCallbacks(e) + } } } } ) @@ -187,16 +223,23 @@ private[nio] class ConnectionManager( } handleReadWriteExecutor.execute(new Runnable { override def run() { - var register: Boolean = false try { - register = conn.read() - } finally { - readRunnableStarted.synchronized { - readRunnableStarted -= key - if (register && conn.changeInterestForRead()) { - conn.registerInterest() + var register: Boolean = false + try { + register = conn.read() + } finally { + readRunnableStarted.synchronized { + readRunnableStarted -= key + if (register && conn.changeInterestForRead()) { + conn.registerInterest() + } } } + } catch { + case NonFatal(e) => { + logError("Error when reading from " + conn.getRemoteConnectionManagerId(), e) + conn.callOnExceptionCallbacks(e) + } } } } ) @@ -213,19 +256,25 @@ private[nio] class ConnectionManager( handleConnectExecutor.execute(new Runnable { override def run() { + try { + var tries: Int = 10 + while (tries >= 0) { + if (conn.finishConnect(false)) return + // Sleep ? + Thread.sleep(1) + tries -= 1 + } - var tries: Int = 10 - while (tries >= 0) { - if (conn.finishConnect(false)) return - // Sleep ? - Thread.sleep(1) - tries -= 1 + // fallback to previous behavior : we should not really come here since this method was + // triggered since channel became connectable : but at times, the first finishConnect need + // not succeed : hence the loop to retry a few 'times'. + conn.finishConnect(true) + } catch { + case NonFatal(e) => { + logError("Error when finishConnect for " + conn.getRemoteConnectionManagerId(), e) + conn.callOnExceptionCallbacks(e) + } } - - // fallback to previous behavior : we should not really come here since this method was - // triggered since channel became connectable : but at times, the first finishConnect need - // not succeed : hence the loop to retry a few 'times'. - conn.finishConnect(true) } } ) } @@ -246,16 +295,16 @@ private[nio] class ConnectionManager( handleConnectExecutor.execute(new Runnable { override def run() { try { - conn.callOnExceptionCallback(e) + conn.callOnExceptionCallbacks(e) } catch { // ignore exceptions - case e: Exception => logDebug("Ignoring exception", e) + case NonFatal(e) => logDebug("Ignoring exception", e) } try { conn.close() } catch { // ignore exceptions - case e: Exception => logDebug("Ignoring exception", e) + case NonFatal(e) => logDebug("Ignoring exception", e) } } }) @@ -448,7 +497,7 @@ private[nio] class ConnectionManager( messageStatuses.values.filter(_.connectionManagerId == sendingConnectionManagerId) .foreach(status => { logInfo("Notifying " + status) - status.markDone(None) + status.failWithoutAck() }) messageStatuses.retain((i, status) => { @@ -477,7 +526,7 @@ private[nio] class ConnectionManager( for (s <- messageStatuses.values if s.connectionManagerId == sendingConnectionManagerId) { logInfo("Notifying " + s) - s.markDone(None) + s.failWithoutAck() } messageStatuses.retain((i, status) => { @@ -492,7 +541,7 @@ private[nio] class ConnectionManager( } } - def handleConnectionError(connection: Connection, e: Exception) { + def handleConnectionError(connection: Connection, e: Throwable) { logInfo("Handling connection error on connection to " + connection.getRemoteConnectionManagerId()) removeConnection(connection) @@ -510,9 +559,17 @@ private[nio] class ConnectionManager( val runnable = new Runnable() { val creationTime = System.currentTimeMillis def run() { - logDebug("Handler thread delay is " + (System.currentTimeMillis - creationTime) + " ms") - handleMessage(connectionManagerId, message, connection) - logDebug("Handling delay is " + (System.currentTimeMillis - creationTime) + " ms") + try { + logDebug("Handler thread delay is " + (System.currentTimeMillis - creationTime) + " ms") + handleMessage(connectionManagerId, message, connection) + logDebug("Handling delay is " + (System.currentTimeMillis - creationTime) + " ms") + } catch { + case NonFatal(e) => { + logError("Error when handling messages from " + + connection.getRemoteConnectionManagerId(), e) + connection.callOnExceptionCallbacks(e) + } + } } } handleMessageExecutor.execute(runnable) @@ -651,7 +708,7 @@ private[nio] class ConnectionManager( messageStatuses.get(bufferMessage.ackId) match { case Some(status) => { messageStatuses -= bufferMessage.ackId - status.markDone(Some(message)) + status.success(message) } case None => { /** @@ -770,6 +827,12 @@ private[nio] class ConnectionManager( val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId, newConnectionId, securityManager) + newConnection.onException { + case (conn, e) => { + logError("Exception while sending message.", e) + reportSendingMessageFailure(message.id, e) + } + } logTrace("creating new sending connection: " + newConnectionId) registerRequests.enqueue(newConnection) @@ -782,13 +845,36 @@ private[nio] class ConnectionManager( "connectionid: " + connection.connectionId) if (authEnabled) { - checkSendAuthFirst(connectionManagerId, connection) + try { + checkSendAuthFirst(connectionManagerId, connection) + } catch { + case NonFatal(e) => { + reportSendingMessageFailure(message.id, e) + } + } } logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") connection.send(message) wakeupSelector() } + private def reportSendingMessageFailure(messageId: Int, e: Throwable): Unit = { + // need to tell sender it failed + messageStatuses.synchronized { + val s = messageStatuses.get(messageId) + s match { + case Some(msgStatus) => { + messageStatuses -= messageId + logInfo("Notifying " + msgStatus.connectionManagerId) + msgStatus.failure(e) + } + case None => { + logError("no messageStatus for failed message id: " + messageId) + } + } + } + } + private def wakeupSelector() { selector.wakeup() } @@ -807,9 +893,11 @@ private[nio] class ConnectionManager( override def run(): Unit = { messageStatuses.synchronized { messageStatuses.remove(message.id).foreach ( s => { - promise.failure( - new IOException("sendMessageReliably failed because ack " + - s"was not received within $ackTimeout sec")) + val e = new IOException("sendMessageReliably failed because ack " + + s"was not received within $ackTimeout sec") + if (!promise.tryFailure(e)) { + logWarning("Ignore error because promise is completed", e) + } }) } } @@ -817,15 +905,23 @@ private[nio] class ConnectionManager( val status = new MessageStatus(message, connectionManagerId, s => { timeoutTask.cancel() - s.ackMessage match { - case None => // Indicates a failure where we either never sent or never got ACK'd - promise.failure(new IOException("sendMessageReliably failed without being ACK'd")) - case Some(ackMessage) => + s match { + case scala.util.Failure(e) => + // Indicates a failure where we either never sent or never got ACK'd + if (!promise.tryFailure(e)) { + logWarning("Ignore error because promise is completed", e) + } + case scala.util.Success(ackMessage) => if (ackMessage.hasError) { - promise.failure( - new IOException("sendMessageReliably failed with ACK that signalled a remote error")) + val e = new IOException( + "sendMessageReliably failed with ACK that signalled a remote error") + if (!promise.tryFailure(e)) { + logWarning("Ignore error because promise is completed", e) + } } else { - promise.success(ackMessage) + if (!promise.trySuccess(ackMessage)) { + logWarning("Drop ackMessage because promise is completed") + } } } }) From b77a02f41c60d869f48b65e72ed696c05b30bc48 Mon Sep 17 00:00:00 2001 From: Vida Ha Date: Thu, 9 Oct 2014 13:13:31 -0700 Subject: [PATCH 0944/1492] [SPARK-3752][SQL]: Add tests for different UDF's Author: Vida Ha Closes #2621 from vidaha/vida/SPARK-3752 and squashes the following commits: d7fdbbc [Vida Ha] Add tests for different UDF's --- .../hive/execution/UDFIntegerToString.java | 26 ++++ .../sql/hive/execution/UDFListListInt.java | 51 ++++++++ .../sql/hive/execution/UDFListString.java | 38 ++++++ .../sql/hive/execution/UDFStringString.java | 26 ++++ .../sql/hive/execution/UDFTwoListList.java | 28 +++++ .../sql/hive/execution/HiveUdfSuite.scala | 111 +++++++++++++++--- 6 files changed, 265 insertions(+), 15 deletions(-) create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFIntegerToString.java create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFStringString.java create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFTwoListList.java diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFIntegerToString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFIntegerToString.java new file mode 100644 index 0000000000000..6c4f378bc5471 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFIntegerToString.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +public class UDFIntegerToString extends UDF { + public String evaluate(Integer i) { + return i.toString(); + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java new file mode 100644 index 0000000000000..d2d39a8c4dc28 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListListInt.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +import java.util.List; + +public class UDFListListInt extends UDF { + /** + * + * @param obj + * SQL schema: array> + * Java Type: List> + * @return + */ + public long evaluate(Object obj) { + if (obj == null) { + return 0l; + } + List listList = (List) obj; + long retVal = 0; + for (List aList : listList) { + @SuppressWarnings("unchecked") + List list = (List) aList; + @SuppressWarnings("unchecked") + Integer someInt = (Integer) list.get(1); + try { + retVal += (long) (someInt.intValue()); + } catch (NullPointerException e) { + System.out.println(e); + } + } + return retVal; + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java new file mode 100644 index 0000000000000..efd34df293c88 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +import java.util.List; +import org.apache.commons.lang.StringUtils; + +public class UDFListString extends UDF { + + public String evaluate(Object a) { + if (a == null) { + return null; + } + @SuppressWarnings("unchecked") + List s = (List) a; + + return StringUtils.join(s, ','); + } + + +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFStringString.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFStringString.java new file mode 100644 index 0000000000000..a369188d471e8 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFStringString.java @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +public class UDFStringString extends UDF { + public String evaluate(String s1, String s2) { + return s1 + " " + s2; + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFTwoListList.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFTwoListList.java new file mode 100644 index 0000000000000..0165591a7ce78 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFTwoListList.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +public class UDFTwoListList extends UDF { + public String evaluate(Object o1, Object o2) { + UDFListListInt udf = new UDFListListInt(); + + return String.format("%s, %s", udf.evaluate(o1), udf.evaluate(o2)); + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index e4324e9528f9b..872f28d514efe 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -17,33 +17,37 @@ package org.apache.spark.sql.hive.execution -import java.io.{DataOutput, DataInput} +import java.io.{DataInput, DataOutput} import java.util import java.util.Properties -import org.apache.spark.util.Utils - -import scala.collection.JavaConversions._ - import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.serde2.{SerDeStats, AbstractSerDe} -import org.apache.hadoop.io.Writable -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorFactory, ObjectInspector} - -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory import org.apache.hadoop.hive.ql.udf.generic.GenericUDF import org.apache.hadoop.hive.ql.udf.generic.GenericUDF.DeferredObject - -import org.apache.spark.sql.Row +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.{AbstractSerDe, SerDeStats} +import org.apache.hadoop.io.Writable +import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.hive.test.TestHive._ + +import org.apache.spark.util.Utils + +import scala.collection.JavaConversions._ case class Fields(f1: Int, f2: Int, f3: Int, f4: Int, f5: Int) +// Case classes for the custom UDF's. +case class IntegerCaseClass(i: Int) +case class ListListIntCaseClass(lli: Seq[(Int, Int, Int)]) +case class StringCaseClass(s: String) +case class ListStringCaseClass(l: Seq[String]) + /** * A test suite for Hive custom UDFs. */ -class HiveUdfSuite extends HiveComparisonTest { +class HiveUdfSuite extends QueryTest { + import TestHive._ test("spark sql udf test that returns a struct") { registerFunction("getStruct", (_: Int) => Fields(1, 2, 3, 4, 5)) @@ -81,7 +85,84 @@ class HiveUdfSuite extends HiveComparisonTest { } test("SPARK-2693 udaf aggregates test") { - assert(sql("SELECT percentile(key,1) FROM src").first === sql("SELECT max(key) FROM src").first) + checkAnswer(sql("SELECT percentile(key,1) FROM src LIMIT 1"), + sql("SELECT max(key) FROM src").collect().toSeq) + } + + test("UDFIntegerToString") { + val testData = TestHive.sparkContext.parallelize( + IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil) + testData.registerTempTable("integerTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '${classOf[UDFIntegerToString].getName}'") + checkAnswer( + sql("SELECT testUDFIntegerToString(i) FROM integerTable"), //.collect(), + Seq(Seq("1"), Seq("2"))) + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFIntegerToString") + + TestHive.reset() + } + + test("UDFListListInt") { + val testData = TestHive.sparkContext.parallelize( + ListListIntCaseClass(Nil) :: + ListListIntCaseClass(Seq((1, 2, 3))) :: + ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: Nil) + testData.registerTempTable("listListIntTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") + checkAnswer( + sql("SELECT testUDFListListInt(lli) FROM listListIntTable"), //.collect(), + Seq(Seq(0), Seq(2), Seq(13))) + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListListInt") + + TestHive.reset() + } + + test("UDFListString") { + val testData = TestHive.sparkContext.parallelize( + ListStringCaseClass(Seq("a", "b", "c")) :: + ListStringCaseClass(Seq("d", "e")) :: Nil) + testData.registerTempTable("listStringTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") + checkAnswer( + sql("SELECT testUDFListString(l) FROM listStringTable"), //.collect(), + Seq(Seq("a,b,c"), Seq("d,e"))) + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListString") + + TestHive.reset() + } + + test("UDFStringString") { + val testData = TestHive.sparkContext.parallelize( + StringCaseClass("world") :: StringCaseClass("goodbye") :: Nil) + testData.registerTempTable("stringTable") + + sql(s"CREATE TEMPORARY FUNCTION testStringStringUdf AS '${classOf[UDFStringString].getName}'") + checkAnswer( + sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"), //.collect(), + Seq(Seq("hello world"), Seq("hello goodbye"))) + sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUdf") + + TestHive.reset() + } + + test("UDFTwoListList") { + val testData = TestHive.sparkContext.parallelize( + ListListIntCaseClass(Nil) :: + ListListIntCaseClass(Seq((1, 2, 3))) :: + ListListIntCaseClass(Seq((4, 5, 6), (7, 8, 9))) :: + Nil) + testData.registerTempTable("TwoListTable") + + sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") + checkAnswer( + sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"), //.collect(), + Seq(Seq("0, 0"), Seq("2, 2"), Seq("13, 13"))) + sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList") + + TestHive.reset() } } From 752e90f15e0bb82d283f05eff08df874b48caed9 Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Thu, 9 Oct 2014 12:59:14 -0700 Subject: [PATCH 0945/1492] [SPARK-3711][SQL] Optimize where in clause filter queries The In case class is replaced by a InSet class in case all the filters are literals, which uses a hashset instead of Sequence, thereby giving significant performance improvement (earlier the seq was using a worst case linear match (exists method) since expressions were assumed in the filter list) . Maximum improvement should be visible in case small percentage of large data matches the filter list. Author: Yash Datta Closes #2561 from saucam/branch-1.1 and squashes the following commits: 4bf2d19 [Yash Datta] SPARK-3711: 1. Fix code style and import order 2. Fix optimization condition 3. Add tests for null in filter list 4. Add test case that optimization is not triggered in case of attributes in filter list afedbcd [Yash Datta] SPARK-3711: 1. Add test cases for InSet class in ExpressionEvaluationSuite 2. Add class OptimizedInSuite on the lines of ConstantFoldingSuite, for the optimized In clause 0fc902f [Yash Datta] SPARK-3711: UnaryMinus will be handled by constantFolding bd84c67 [Yash Datta] SPARK-3711: Incorporate review comments. Move optimization of In clause to Optimizer.scala by adding a rule. Add appropriate comments 430f5d1 [Yash Datta] SPARK-3711: Optimize the filter list in case of negative values as well bee98aa [Yash Datta] SPARK-3711: Optimize where in clause filter queries --- .../sql/catalyst/expressions/predicates.scala | 19 ++++- .../sql/catalyst/optimizer/Optimizer.scala | 18 ++++- .../ExpressionEvaluationSuite.scala | 21 +++++ .../catalyst/optimizer/OptimizeInSuite.scala | 76 +++++++++++++++++++ 4 files changed, 132 insertions(+), 2 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 329af332d0fa1..1e22b2d03c672 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.catalyst.expressions +import scala.collection.immutable.HashSet import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.BooleanType - object InterpretedPredicate { def apply(expression: Expression, inputSchema: Seq[Attribute]): (Row => Boolean) = apply(BindReferences.bindReference(expression, inputSchema)) @@ -95,6 +95,23 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { } } +/** + * Optimized version of In clause, when all filter values of In clause are + * static. + */ +case class InSet(value: Expression, hset: HashSet[Any], child: Seq[Expression]) + extends Predicate { + + def children = child + + def nullable = true // TODO: Figure out correct nullability semantics of IN. + override def toString = s"$value INSET ${hset.mkString("(", ",", ")")}" + + override def eval(input: Row): Any = { + hset.contains(value.eval(input)) + } +} + case class And(left: Expression, right: Expression) extends BinaryPredicate { def symbol = "&&" 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 636d0b95583e4..3693b41404fd6 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer +import scala.collection.immutable.HashSet import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner import org.apache.spark.sql.catalyst.plans.FullOuter @@ -38,7 +39,8 @@ object Optimizer extends RuleExecutor[LogicalPlan] { BooleanSimplification, SimplifyFilters, SimplifyCasts, - SimplifyCaseConversionExpressions) :: + SimplifyCaseConversionExpressions, + OptimizeIn) :: Batch("Filter Pushdown", FixedPoint(100), UnionPushdown, CombineFilters, @@ -273,6 +275,20 @@ object ConstantFolding extends Rule[LogicalPlan] { } } +/** + * Replaces [[In (value, seq[Literal])]] with optimized version[[InSet (value, HashSet[Literal])]] + * which is much faster + */ +object OptimizeIn extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => q transformExpressionsDown { + case In(v, list) if !list.exists(!_.isInstanceOf[Literal]) => + val hSet = list.map(e => e.eval(null)) + InSet(v, HashSet() ++ hSet, v +: list) + } + } +} + /** * Simplifies boolean expressions where the answer can be determined without evaluating both sides. * Note that this rule can eliminate expressions that might otherwise have been evaluated and thus 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 63931af4bac3d..692ed78a7292c 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 @@ -19,12 +19,15 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp +import scala.collection.immutable.HashSet + import org.scalatest.FunSuite import org.scalatest.Matchers._ import org.scalautils.TripleEqualsSupport.Spread import org.apache.spark.sql.catalyst.types._ + /* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -145,6 +148,24 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) } + test("INSET") { + val hS = HashSet[Any]() + 1 + 2 + val nS = HashSet[Any]() + 1 + 2 + null + val one = Literal(1) + val two = Literal(2) + val three = Literal(3) + val nl = Literal(null) + val s = Seq(one, two) + val nullS = Seq(one, two, null) + checkEvaluation(InSet(one, hS, one +: s), true) + checkEvaluation(InSet(two, hS, two +: s), true) + checkEvaluation(InSet(two, nS, two +: nullS), true) + checkEvaluation(InSet(nl, nS, nl +: nullS), true) + checkEvaluation(InSet(three, hS, three +: s), false) + checkEvaluation(InSet(three, nS, three +: nullS), false) + checkEvaluation(InSet(one, hS, one +: s) && InSet(two, hS, two +: s), true) + } + test("MaxOf") { checkEvaluation(MaxOf(1, 2), 2) checkEvaluation(MaxOf(2, 1), 2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala new file mode 100644 index 0000000000000..97a78ec971c39 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import scala.collection.immutable.HashSet +import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.types._ + +// For implicit conversions +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ + +class OptimizeInSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("AnalysisNodes", Once, + EliminateAnalysisOperators) :: + Batch("ConstantFolding", Once, + ConstantFolding, + BooleanSimplification, + OptimizeIn) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + + test("OptimizedIn test: In clause optimized to InSet") { + val originalQuery = + testRelation + .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2)))) + .analyze + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .where(InSet(UnresolvedAttribute("a"), HashSet[Any]()+1+2, + UnresolvedAttribute("a") +: Seq(Literal(1),Literal(2)))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("OptimizedIn test: In clause not optimized in case filter has attributes") { + val originalQuery = + testRelation + .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2), UnresolvedAttribute("b")))) + .analyze + + val optimized = Optimize(originalQuery.analyze) + val correctAnswer = + testRelation + .where(In(UnresolvedAttribute("a"), Seq(Literal(1),Literal(2), UnresolvedAttribute("b")))) + .analyze + + comparePlans(optimized, correctAnswer) + } +} From 2c8851343a2e4d1d5b3a2b959eaa651a92982a72 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 9 Oct 2014 13:22:36 -0700 Subject: [PATCH 0946/1492] [SPARK-3806][SQL] Minor fix for CliSuite To fix two issues in CliSuite 1 CliSuite throw IndexOutOfBoundsException: Exception in thread "Thread-6" java.lang.IndexOutOfBoundsException: 6 at scala.collection.mutable.ResizableArray$class.apply(ResizableArray.scala:43) at scala.collection.mutable.ArrayBuffer.apply(ArrayBuffer.scala:47) at org.apache.spark.sql.hive.thriftserver.CliSuite.org$apache$spark$sql$hive$thriftserver$CliSuite$$captureOutput$1(CliSuite.scala:67) at org.apache.spark.sql.hive.thriftserver.CliSuite$$anonfun$4.apply(CliSuite.scala:78) at org.apache.spark.sql.hive.thriftserver.CliSuite$$anonfun$4.apply(CliSuite.scala:78) at scala.sys.process.ProcessLogger$$anon$1.out(ProcessLogger.scala:96) at scala.sys.process.BasicIO$$anonfun$processOutFully$1.apply(BasicIO.scala:135) at scala.sys.process.BasicIO$$anonfun$processOutFully$1.apply(BasicIO.scala:135) at scala.sys.process.BasicIO$.readFully$1(BasicIO.scala:175) at scala.sys.process.BasicIO$.processLinesFully(BasicIO.scala:179) at scala.sys.process.BasicIO$$anonfun$processFully$1.apply(BasicIO.scala:164) at scala.sys.process.BasicIO$$anonfun$processFully$1.apply(BasicIO.scala:162) at scala.sys.process.ProcessBuilderImpl$Simple$$anonfun$3.apply$mcV$sp(ProcessBuilderImpl.scala:73) at scala.sys.process.ProcessImpl$Spawn$$anon$1.run(ProcessImpl.scala:22) Actually, it is the Mutil-Threads lead to this problem. 2 Using ```line.startsWith``` instead ```line.contains``` to assert expected answer. This is a tiny bug in CliSuite, for test case "Simple commands", there is a expected answers "5", if we use ```contains``` that means output like "14/10/06 11:```5```4:36 INFO CliDriver: Time taken: 1.078 seconds" or "14/10/06 11:54:36 INFO StatsReportListener: 0% ```5```% 10% 25% 50% 75% 90% 95% 100%" will make the assert true. Author: scwf Closes #2666 from scwf/clisuite and squashes the following commits: 11430db [scwf] fix-clisuite --- .../org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 3475c2c9db080..d68dd090b5e6c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -62,9 +62,11 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { def captureOutput(source: String)(line: String) { buffer += s"$source> $line" - if (line.contains(expectedAnswers(next.get()))) { - if (next.incrementAndGet() == expectedAnswers.size) { - foundAllExpectedAnswers.trySuccess(()) + if (next.get() < expectedAnswers.size) { + if (line.startsWith(expectedAnswers(next.get()))) { + if (next.incrementAndGet() == expectedAnswers.size) { + foundAllExpectedAnswers.trySuccess(()) + } } } } From e7edb723d22869f228b838fd242bf8e6fe73ee19 Mon Sep 17 00:00:00 2001 From: cocoatomo Date: Thu, 9 Oct 2014 13:46:26 -0700 Subject: [PATCH 0947/1492] [SPARK-3868][PySpark] Hard to recognize which module is tested from unit-tests.log ./python/run-tests script display messages about which test it is running currently on stdout but not write them on unit-tests.log. It is harder for us to recognize what test programs were executed and which test was failed. Author: cocoatomo Closes #2724 from cocoatomo/issues/3868-display-testing-module-name and squashes the following commits: c63d9fa [cocoatomo] [SPARK-3868][PySpark] Hard to recognize which module is tested from unit-tests.log --- python/run-tests | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/python/run-tests b/python/run-tests index 63395f72788f9..f6a96841175e8 100755 --- a/python/run-tests +++ b/python/run-tests @@ -25,16 +25,17 @@ FWDIR="$(cd "`dirname "$0"`"; cd ../; pwd)" cd "$FWDIR/python" FAILED=0 +LOG_FILE=unit-tests.log -rm -f unit-tests.log +rm -f $LOG_FILE # Remove the metastore and warehouse directory created by the HiveContext tests in Spark SQL rm -rf metastore warehouse function run_test() { - echo "Running test: $1" + echo "Running test: $1" | tee -a $LOG_FILE - SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log + SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 2>&1 | tee -a $LOG_FILE FAILED=$((PIPESTATUS[0]||$FAILED)) From ec4d40e48186af18e25517e0474020720645f583 Mon Sep 17 00:00:00 2001 From: Mike Timper Date: Thu, 9 Oct 2014 14:02:27 -0700 Subject: [PATCH 0948/1492] [SPARK-3853][SQL] JSON Schema support for Timestamp fields In JSONRDD.scala, add 'case TimestampType' in the enforceCorrectType function and a toTimestamp function. Author: Mike Timper Closes #2720 from mtimper/master and squashes the following commits: 9386ab8 [Mike Timper] Fix and tests for SPARK-3853 --- .../main/scala/org/apache/spark/sql/json/JsonRDD.scala | 10 ++++++++++ .../scala/org/apache/spark/sql/json/JsonSuite.scala | 8 ++++++++ 2 files changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 0f27fd13e7379..fbc2965e61e92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.json import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal +import java.sql.Timestamp import com.fasterxml.jackson.databind.ObjectMapper @@ -361,6 +362,14 @@ private[sql] object JsonRDD extends Logging { } } + private def toTimestamp(value: Any): Timestamp = { + value match { + case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) + case value: java.lang.Long => new Timestamp(value) + case value: java.lang.String => Timestamp.valueOf(value) + } + } + private[json] def enforceCorrectType(value: Any, desiredType: DataType): Any ={ if (value == null) { null @@ -377,6 +386,7 @@ private[sql] object JsonRDD extends Logging { case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], struct) + case TimestampType => toTimestamp(value) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 685e788207725..3cfcb2b1aa993 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -23,6 +23,8 @@ import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} import org.apache.spark.sql.QueryTest import org.apache.spark.sql.test.TestSQLContext._ +import java.sql.Timestamp + class JsonSuite extends QueryTest { import TestJsonData._ TestJsonData @@ -50,6 +52,12 @@ class JsonSuite extends QueryTest { val doubleNumber: Double = 1.7976931348623157E308d checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) checkTypePromotion(BigDecimal(doubleNumber), enforceCorrectType(doubleNumber, DecimalType)) + + checkTypePromotion(new Timestamp(intNumber), enforceCorrectType(intNumber, TimestampType)) + checkTypePromotion(new Timestamp(intNumber.toLong), + enforceCorrectType(intNumber.toLong, TimestampType)) + val strDate = "2014-09-30 12:34:56" + checkTypePromotion(Timestamp.valueOf(strDate), enforceCorrectType(strDate, TimestampType)) } test("Get compatible type") { From 1faa1135a3fc0acd89f934f01a4a2edefcb93d33 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 9 Oct 2014 14:50:36 -0700 Subject: [PATCH 0949/1492] Revert "[SPARK-2805] Upgrade to akka 2.3.4" This reverts commit b9df8af62e8d7b263a668dfb6e9668ab4294ea37. --- .../org/apache/spark/deploy/Client.scala | 2 +- .../spark/deploy/client/AppClient.scala | 2 +- .../spark/deploy/worker/WorkerWatcher.scala | 2 +- .../apache/spark/MapOutputTrackerSuite.scala | 4 +- pom.xml | 2 +- .../spark/streaming/InputStreamsSuite.scala | 71 +++++++++++++++++++ 6 files changed, 77 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index f2687ce6b42b4..065ddda50e65e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -130,7 +130,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") System.exit(-1) - case AssociationErrorEvent(cause, _, remoteAddress, _, _) => + case AssociationErrorEvent(cause, _, remoteAddress, _) => println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") println(s"Cause was: $cause") System.exit(-1) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 98a93d1fcb2a3..32790053a6be8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -154,7 +154,7 @@ private[spark] class AppClient( logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() - case AssociationErrorEvent(cause, _, address, _, _) if isPossibleMaster(address) => + case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) => logWarning(s"Could not connect to $address: $cause") case StopAppClient => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 63a8ac817b618..6d0d0bbe5ecec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -54,7 +54,7 @@ private[spark] class WorkerWatcher(workerUrl: String) case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => logInfo(s"Successfully connected to $workerUrl") - case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) + case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => // These logs may not be seen if the worker (and associated pipe) has died logError(s"Could not initialize connection to worker $workerUrl. Exiting.") diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index cbc0bd178d894..1fef79ad1001f 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -146,7 +146,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) val actorSystem = ActorSystem("test") val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) + new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) val masterActor = actorRef.underlyingActor // Frame size should be ~123B, and no exception should be thrown @@ -164,7 +164,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) val actorSystem = ActorSystem("test") val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) + new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) val masterActor = actorRef.underlyingActor // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. diff --git a/pom.xml b/pom.xml index 3b6d4ecbae2c1..7756c89b00cad 100644 --- a/pom.xml +++ b/pom.xml @@ -118,7 +118,7 @@ 0.18.1 shaded-protobuf org.spark-project.akka - 2.3.4-spark + 2.2.3-shaded-protobuf 1.7.5 1.2.17 1.0.4 diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 6107fcdc447b6..952a74fd5f6de 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.streaming import akka.actor.Actor +import akka.actor.IO +import akka.actor.IOManager import akka.actor.Props import akka.util.ByteString @@ -142,6 +144,59 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } + // TODO: This test works in IntelliJ but not through SBT + ignore("actor input stream") { + // Start the server + val testServer = new TestServer() + val port = testServer.port + testServer.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", + // Had to pass the local value of port to prevent from closing over entire scope + StorageLevel.MEMORY_AND_DISK) + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(networkStream, outputBuffer) + def output = outputBuffer.flatMap(x => x) + outputStream.register() + ssc.start() + + // Feed data to the server to send to the network receiver + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = 1 to 9 + val expectedOutput = input.map(x => x.toString) + Thread.sleep(1000) + for (i <- 0 until input.size) { + testServer.send(input(i).toString) + Thread.sleep(500) + clock.addToTime(batchDuration.milliseconds) + } + Thread.sleep(1000) + logInfo("Stopping server") + testServer.stop() + logInfo("Stopping context") + ssc.stop() + + // Verify whether data received was as expected + logInfo("--------------------------------") + logInfo("output.size = " + outputBuffer.size) + logInfo("output") + outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("expected output.size = " + expectedOutput.size) + logInfo("expected output") + expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) + logInfo("--------------------------------") + + // Verify whether all the elements received are as expected + // (whether the elements were received one in each interval is not verified) + assert(output.size === expectedOutput.size) + for (i <- 0 until output.size) { + assert(output(i) === expectedOutput(i)) + } + } + + test("multi-thread receiver") { // set up the test receiver val numThreads = 10 @@ -323,6 +378,22 @@ class TestServer(portToBind: Int = 0) extends Logging { def port = serverSocket.getLocalPort } +/** This is an actor for testing actor input stream */ +class TestActor(port: Int) extends Actor with ActorHelper { + + def bytesToString(byteString: ByteString) = byteString.utf8String + + override def preStart(): Unit = { + @deprecated("suppress compile time deprecation warning", "1.0.0") + val unit = IOManager(context.system).connect(new InetSocketAddress(port)) + } + + def receive = { + case IO.Read(socket, bytes) => + store(bytesToString(bytes)) + } +} + /** This is a receiver to test multiple threads inserting data using block generator */ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) extends Receiver[Int](StorageLevel.MEMORY_ONLY_SER) with Logging { From 1c7f0ab302de9f82b1bd6da852d133823bc67c66 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 9 Oct 2014 14:57:27 -0700 Subject: [PATCH 0950/1492] [SPARK-3339][SQL] Support for skipping json lines that fail to parse This PR aims to provide a way to skip/query corrupt JSON records. To do so, we introduce an internal column to hold corrupt records (the default name is `_corrupt_record`. This name can be changed by setting the value of `spark.sql.columnNameOfCorruptRecord`). When there is a parsing error, we will put the corrupt record in its unparsed format to the internal column. Users can skip/query this column through SQL. * To query those corrupt records ``` -- For Hive parser SELECT `_corrupt_record` FROM jsonTable WHERE `_corrupt_record` IS NOT NULL -- For our SQL parser SELECT _corrupt_record FROM jsonTable WHERE _corrupt_record IS NOT NULL ``` * To skip corrupt records and query regular records ``` -- For Hive parser SELECT field1, field2 FROM jsonTable WHERE `_corrupt_record` IS NULL -- For our SQL parser SELECT field1, field2 FROM jsonTable WHERE _corrupt_record IS NULL ``` Generally, it is not recommended to change the name of the internal column. If the name has to be changed to avoid possible name conflicts, you can use `sqlContext.setConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD, )` or `sqlContext.sql(SET spark.sql.columnNameOfCorruptRecord=)`. Author: Yin Huai Closes #2680 from yhuai/corruptJsonRecord and squashes the following commits: 4c9828e [Yin Huai] Merge remote-tracking branch 'upstream/master' into corruptJsonRecord 309616a [Yin Huai] Change the default name of corrupt record to "_corrupt_record". b4a3632 [Yin Huai] Merge remote-tracking branch 'upstream/master' into corruptJsonRecord 9375ae9 [Yin Huai] Set the column name of corrupt json record back to the default one after the unit test. ee584c0 [Yin Huai] Provide a way to query corrupt json records as unparsed strings. --- .../scala/org/apache/spark/sql/SQLConf.scala | 4 ++ .../org/apache/spark/sql/SQLContext.scala | 14 +++-- .../spark/sql/api/java/JavaSQLContext.scala | 16 +++-- .../org/apache/spark/sql/json/JsonRDD.scala | 30 ++++++--- .../org/apache/spark/sql/json/JsonSuite.scala | 62 ++++++++++++++++++- .../apache/spark/sql/json/TestJsonData.scala | 9 +++ 6 files changed, 116 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index f6f4cf3b80d41..07e6e2eccddf4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -35,6 +35,7 @@ private[spark] object SQLConf { val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" + val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -131,6 +132,9 @@ private[sql] trait SQLConf { private[spark] def inMemoryPartitionPruning: Boolean = getConf(IN_MEMORY_PARTITION_PRUNING, "false").toBoolean + private[spark] def columnNameOfCorruptRecord: String = + getConf(COLUMN_NAME_OF_CORRUPT_RECORD, "_corrupt_record") + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 35561cac3e5e1..014e1e2826724 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -195,9 +195,12 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def jsonRDD(json: RDD[String], schema: StructType): SchemaRDD = { + val columnNameOfCorruptJsonRecord = columnNameOfCorruptRecord val appliedSchema = - Option(schema).getOrElse(JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, 1.0))) - val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema) + Option(schema).getOrElse( + JsonRDD.nullTypeToStringType( + JsonRDD.inferSchema(json, 1.0, columnNameOfCorruptJsonRecord))) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) applySchema(rowRDD, appliedSchema) } @@ -206,8 +209,11 @@ class SQLContext(@transient val sparkContext: SparkContext) */ @Experimental def jsonRDD(json: RDD[String], samplingRatio: Double): SchemaRDD = { - val appliedSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json, samplingRatio)) - val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema) + val columnNameOfCorruptJsonRecord = columnNameOfCorruptRecord + val appliedSchema = + JsonRDD.nullTypeToStringType( + JsonRDD.inferSchema(json, samplingRatio, columnNameOfCorruptJsonRecord)) + val rowRDD = JsonRDD.jsonStringToRow(json, appliedSchema, columnNameOfCorruptJsonRecord) applySchema(rowRDD, appliedSchema) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index c006c4330ff66..f8171c3be3207 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -148,8 +148,12 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { * It goes through the entire dataset once to determine the schema. */ def jsonRDD(json: JavaRDD[String]): JavaSchemaRDD = { - val appliedScalaSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0)) - val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) + val columnNameOfCorruptJsonRecord = sqlContext.columnNameOfCorruptRecord + val appliedScalaSchema = + JsonRDD.nullTypeToStringType( + JsonRDD.inferSchema(json.rdd, 1.0, columnNameOfCorruptJsonRecord)) + val scalaRowRDD = + JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema, columnNameOfCorruptJsonRecord) val logicalPlan = LogicalRDD(appliedScalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) @@ -162,10 +166,14 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { */ @Experimental def jsonRDD(json: JavaRDD[String], schema: StructType): JavaSchemaRDD = { + val columnNameOfCorruptJsonRecord = sqlContext.columnNameOfCorruptRecord val appliedScalaSchema = Option(asScalaDataType(schema)).getOrElse( - JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[SStructType] - val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) + JsonRDD.nullTypeToStringType( + JsonRDD.inferSchema( + json.rdd, 1.0, columnNameOfCorruptJsonRecord))).asInstanceOf[SStructType] + val scalaRowRDD = JsonRDD.jsonStringToRow( + json.rdd, appliedScalaSchema, columnNameOfCorruptJsonRecord) val logicalPlan = LogicalRDD(appliedScalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index fbc2965e61e92..61ee960aad9d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -22,6 +22,7 @@ import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal import java.sql.Timestamp +import com.fasterxml.jackson.core.JsonProcessingException import com.fasterxml.jackson.databind.ObjectMapper import org.apache.spark.rdd.RDD @@ -35,16 +36,19 @@ private[sql] object JsonRDD extends Logging { private[sql] def jsonStringToRow( json: RDD[String], - schema: StructType): RDD[Row] = { - parseJson(json).map(parsed => asRow(parsed, schema)) + schema: StructType, + columnNameOfCorruptRecords: String): RDD[Row] = { + parseJson(json, columnNameOfCorruptRecords).map(parsed => asRow(parsed, schema)) } private[sql] def inferSchema( json: RDD[String], - samplingRatio: Double = 1.0): StructType = { + samplingRatio: Double = 1.0, + columnNameOfCorruptRecords: String): StructType = { require(samplingRatio > 0, s"samplingRatio ($samplingRatio) should be greater than 0") val schemaData = if (samplingRatio > 0.99) json else json.sample(false, samplingRatio, 1) - val allKeys = parseJson(schemaData).map(allKeysWithValueTypes).reduce(_ ++ _) + val allKeys = + parseJson(schemaData, columnNameOfCorruptRecords).map(allKeysWithValueTypes).reduce(_ ++ _) createSchema(allKeys) } @@ -274,7 +278,9 @@ private[sql] object JsonRDD extends Logging { case atom => atom } - private def parseJson(json: RDD[String]): RDD[Map[String, Any]] = { + private def parseJson( + json: RDD[String], + columnNameOfCorruptRecords: String): RDD[Map[String, Any]] = { // According to [Jackson-72: https://jira.codehaus.org/browse/JACKSON-72], // ObjectMapper will not return BigDecimal when // "DeserializationFeature.USE_BIG_DECIMAL_FOR_FLOATS" is disabled @@ -289,12 +295,16 @@ private[sql] object JsonRDD extends Logging { // For example: for {"key": 1, "key":2}, we will get "key"->2. val mapper = new ObjectMapper() iter.flatMap { record => - val parsed = mapper.readValue(record, classOf[Object]) match { - case map: java.util.Map[_, _] => scalafy(map).asInstanceOf[Map[String, Any]] :: Nil - case list: java.util.List[_] => scalafy(list).asInstanceOf[Seq[Map[String, Any]]] - } + try { + val parsed = mapper.readValue(record, classOf[Object]) match { + case map: java.util.Map[_, _] => scalafy(map).asInstanceOf[Map[String, Any]] :: Nil + case list: java.util.List[_] => scalafy(list).asInstanceOf[Seq[Map[String, Any]]] + } - parsed + parsed + } catch { + case e: JsonProcessingException => Map(columnNameOfCorruptRecords -> record) :: Nil + } } }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 3cfcb2b1aa993..7bb08f1b513ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -21,6 +21,8 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import java.sql.Timestamp @@ -644,7 +646,65 @@ class JsonSuite extends QueryTest { ("str_a_1", null, null) :: ("str_a_2", null, null) :: (null, "str_b_3", null) :: - ("str_a_4", "str_b_4", "str_c_4") ::Nil + ("str_a_4", "str_b_4", "str_c_4") :: Nil ) } + + test("Corrupt records") { + // Test if we can query corrupt records. + val oldColumnNameOfCorruptRecord = TestSQLContext.columnNameOfCorruptRecord + TestSQLContext.setConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD, "_unparsed") + + val jsonSchemaRDD = jsonRDD(corruptRecords) + jsonSchemaRDD.registerTempTable("jsonTable") + + val schema = StructType( + StructField("_unparsed", StringType, true) :: + StructField("a", StringType, true) :: + StructField("b", StringType, true) :: + StructField("c", StringType, true) :: Nil) + + assert(schema === jsonSchemaRDD.schema) + + // In HiveContext, backticks should be used to access columns starting with a underscore. + checkAnswer( + sql( + """ + |SELECT a, b, c, _unparsed + |FROM jsonTable + """.stripMargin), + (null, null, null, "{") :: + (null, null, null, "") :: + (null, null, null, """{"a":1, b:2}""") :: + (null, null, null, """{"a":{, b:3}""") :: + ("str_a_4", "str_b_4", "str_c_4", null) :: + (null, null, null, "]") :: Nil + ) + + checkAnswer( + sql( + """ + |SELECT a, b, c + |FROM jsonTable + |WHERE _unparsed IS NULL + """.stripMargin), + ("str_a_4", "str_b_4", "str_c_4") :: Nil + ) + + checkAnswer( + sql( + """ + |SELECT _unparsed + |FROM jsonTable + |WHERE _unparsed IS NOT NULL + """.stripMargin), + Seq("{") :: + Seq("") :: + Seq("""{"a":1, b:2}""") :: + Seq("""{"a":{, b:3}""") :: + Seq("]") :: Nil + ) + + TestSQLContext.setConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD, oldColumnNameOfCorruptRecord) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index fc833b8b54e4c..eaca9f0508a12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -143,4 +143,13 @@ object TestJsonData { """[{"a":"str_a_2"}, {"b":"str_b_3"}]""" :: """{"b":"str_b_4", "a":"str_a_4", "c":"str_c_4"}""" :: """[]""" :: Nil) + + val corruptRecords = + TestSQLContext.sparkContext.parallelize( + """{""" :: + """""" :: + """{"a":1, b:2}""" :: + """{"a":{, b:3}""" :: + """{"b":"str_b_4", "a":"str_a_4", "c":"str_c_4"}""" :: + """]""" :: Nil) } From 0c0e09f567deb775ee378f5385a16884f68b332d Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Thu, 9 Oct 2014 14:59:03 -0700 Subject: [PATCH 0951/1492] [SPARK-3412][SQL]add missing row api chenghao-intel assigned this to me, check PR #2284 for previous discussion Author: Daoyuan Wang Closes #2529 from adrian-wang/rowapi and squashes the following commits: c6594b2 [Daoyuan Wang] using boxed 7b7e6e3 [Daoyuan Wang] update pattern match 7a39456 [Daoyuan Wang] rename file and refresh getAs[T] 4c18c29 [Daoyuan Wang] remove setAs[T] and null judge 1614493 [Daoyuan Wang] add missing row api --- .../sql/catalyst/expressions/Projection.scala | 15 ++++++++++++++ .../spark/sql/catalyst/expressions/Row.scala | 20 ++++++++++--------- ...ificRow.scala => SpecificMutableRow.scala} | 8 ++++++-- 3 files changed, 32 insertions(+), 11 deletions(-) rename sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/{SpecificRow.scala => SpecificMutableRow.scala} (97%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index ef1d12531f109..204904ecf04db 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -137,6 +137,9 @@ class JoinedRow extends Row { def getString(i: Int): String = if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + override def getAs[T](i: Int): T = + if (i < row1.size) row1.getAs[T](i) else row2.getAs[T](i - row1.size) + def copy() = { val totalSize = row1.size + row2.size val copiedValues = new Array[Any](totalSize) @@ -226,6 +229,9 @@ class JoinedRow2 extends Row { def getString(i: Int): String = if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + override def getAs[T](i: Int): T = + if (i < row1.size) row1.getAs[T](i) else row2.getAs[T](i - row1.size) + def copy() = { val totalSize = row1.size + row2.size val copiedValues = new Array[Any](totalSize) @@ -309,6 +315,9 @@ class JoinedRow3 extends Row { def getString(i: Int): String = if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + override def getAs[T](i: Int): T = + if (i < row1.size) row1.getAs[T](i) else row2.getAs[T](i - row1.size) + def copy() = { val totalSize = row1.size + row2.size val copiedValues = new Array[Any](totalSize) @@ -392,6 +401,9 @@ class JoinedRow4 extends Row { def getString(i: Int): String = if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + override def getAs[T](i: Int): T = + if (i < row1.size) row1.getAs[T](i) else row2.getAs[T](i - row1.size) + def copy() = { val totalSize = row1.size + row2.size val copiedValues = new Array[Any](totalSize) @@ -475,6 +487,9 @@ class JoinedRow5 extends Row { def getString(i: Int): String = if (i < row1.size) row1.getString(i) else row2.getString(i - row1.size) + override def getAs[T](i: Int): T = + if (i < row1.size) row1.getAs[T](i) else row2.getAs[T](i - row1.size) + def copy() = { val totalSize = row1.size + row2.size val copiedValues = new Array[Any](totalSize) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala index d68a4fabeac77..d00ec39774c35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -64,6 +64,7 @@ trait Row extends Seq[Any] with Serializable { def getShort(i: Int): Short def getByte(i: Int): Byte def getString(i: Int): String + def getAs[T](i: Int): T = apply(i).asInstanceOf[T] override def toString() = s"[${this.mkString(",")}]" @@ -118,6 +119,7 @@ object EmptyRow extends Row { def getShort(i: Int): Short = throw new UnsupportedOperationException def getByte(i: Int): Byte = throw new UnsupportedOperationException def getString(i: Int): String = throw new UnsupportedOperationException + override def getAs[T](i: Int): T = throw new UnsupportedOperationException def copy() = this } @@ -217,19 +219,19 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { /** No-arg constructor for serialization. */ def this() = this(0) - override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value } - override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value } - override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value } - override def setFloat(ordinal: Int,value: Float): Unit = { values(ordinal) = value } - override def setInt(ordinal: Int,value: Int): Unit = { values(ordinal) = value } - override def setLong(ordinal: Int,value: Long): Unit = { values(ordinal) = value } - override def setString(ordinal: Int,value: String): Unit = { values(ordinal) = value } + override def setBoolean(ordinal: Int, value: Boolean): Unit = { values(ordinal) = value } + override def setByte(ordinal: Int, value: Byte): Unit = { values(ordinal) = value } + override def setDouble(ordinal: Int, value: Double): Unit = { values(ordinal) = value } + override def setFloat(ordinal: Int, value: Float): Unit = { values(ordinal) = value } + override def setInt(ordinal: Int, value: Int): Unit = { values(ordinal) = value } + override def setLong(ordinal: Int, value: Long): Unit = { values(ordinal) = value } + override def setString(ordinal: Int, value: String): Unit = { values(ordinal) = value } override def setNullAt(i: Int): Unit = { values(i) = null } - override def setShort(ordinal: Int,value: Short): Unit = { values(ordinal) = value } + override def setShort(ordinal: Int, value: Short): Unit = { values(ordinal) = value } - override def update(ordinal: Int,value: Any): Unit = { values(ordinal) = value } + override def update(ordinal: Int, value: Any): Unit = { values(ordinal) = value } override def copy() = new GenericRow(values.clone()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala similarity index 97% rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 9cbab3d5d0d0d..570379c533e1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -233,9 +233,9 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR override def iterator: Iterator[Any] = values.map(_.boxed).iterator - def setString(ordinal: Int, value: String) = update(ordinal, value) + override def setString(ordinal: Int, value: String) = update(ordinal, value) - def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String] + override def getString(ordinal: Int) = apply(ordinal).asInstanceOf[String] override def setInt(ordinal: Int, value: Int): Unit = { val currentValue = values(ordinal).asInstanceOf[MutableInt] @@ -306,4 +306,8 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR override def getByte(i: Int): Byte = { values(i).asInstanceOf[MutableByte].value } + + override def getAs[T](i: Int): T = { + values(i).boxed.asInstanceOf[T] + } } From bc3b6cb06153d6b05f311dd78459768b6cf6a404 Mon Sep 17 00:00:00 2001 From: Nathan Howell Date: Thu, 9 Oct 2014 15:03:01 -0700 Subject: [PATCH 0952/1492] [SPARK-3858][SQL] Pass the generator alias into logical plan node The alias parameter is being ignored, which makes it more difficult to specify a qualifier for Generator expressions. Author: Nathan Howell Closes #2721 from NathanHowell/SPARK-3858 and squashes the following commits: 8aa0f43 [Nathan Howell] [SPARK-3858][SQL] Pass the generator alias into logical plan node --- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- .../test/scala/org/apache/spark/sql/DslQuerySuite.scala | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 594bf8ffc20e1..948122d42f0e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -360,7 +360,7 @@ class SchemaRDD( join: Boolean = false, outer: Boolean = false, alias: Option[String] = None) = - new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) + new SchemaRDD(sqlContext, Generate(generator, join, outer, alias, logicalPlan)) /** * Returns this RDD as a SchemaRDD. Intended primarily to force the invocation of the implicit diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index d001abb7e1fcc..45e58afe9d9a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -147,6 +147,14 @@ class DslQuerySuite extends QueryTest { (1, 1, 1, 2) :: Nil) } + test("SPARK-3858 generator qualifiers are discarded") { + checkAnswer( + arrayData.as('ad) + .generate(Explode("data" :: Nil, 'data), alias = Some("ex")) + .select("ex.data".attr), + Seq(1, 2, 3, 2, 3, 4).map(Seq(_))) + } + test("average") { checkAnswer( testData2.groupBy()(avg('a)), From ac302052870a650d56f2d3131c27755bb2960ad7 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Thu, 9 Oct 2014 15:14:58 -0700 Subject: [PATCH 0953/1492] [SPARK-3813][SQL] Support "case when" conditional functions in Spark SQL. "case when" conditional function is already supported in Spark SQL but there is no support in SqlParser. So added parser support to it. Author : ravipesala ravindra.pesalahuawei.com Author: ravipesala Closes #2678 from ravipesala/SPARK-3813 and squashes the following commits: 70c75a7 [ravipesala] Fixed styles 713ea84 [ravipesala] Updated as per admin comments 709684f [ravipesala] Changed parser to support case when function. --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 14 ++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 15 +++++++++++++-- 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 854b5b461bdc8..4662f585cfe15 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -77,10 +77,13 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val BETWEEN = Keyword("BETWEEN") protected val BY = Keyword("BY") protected val CACHE = Keyword("CACHE") + protected val CASE = Keyword("CASE") protected val CAST = Keyword("CAST") protected val COUNT = Keyword("COUNT") protected val DESC = Keyword("DESC") protected val DISTINCT = Keyword("DISTINCT") + protected val ELSE = Keyword("ELSE") + protected val END = Keyword("END") protected val EXCEPT = Keyword("EXCEPT") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") @@ -122,11 +125,13 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val SUBSTRING = Keyword("SUBSTRING") protected val SUM = Keyword("SUM") protected val TABLE = Keyword("TABLE") + protected val THEN = Keyword("THEN") protected val TIMESTAMP = Keyword("TIMESTAMP") protected val TRUE = Keyword("TRUE") protected val UNCACHE = Keyword("UNCACHE") protected val UNION = Keyword("UNION") protected val UPPER = Keyword("UPPER") + protected val WHEN = Keyword("WHEN") protected val WHERE = Keyword("WHERE") // Use reflection to find the reserved words defined in this class. @@ -333,6 +338,15 @@ class SqlParser extends StandardTokenParsers with PackratParsers { IF ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { case c ~ "," ~ t ~ "," ~ f => If(c,t,f) } | + CASE ~> expression.? ~ (WHEN ~> expression ~ (THEN ~> expression)).* ~ + (ELSE ~> expression).? <~ END ^^ { + case casePart ~ altPart ~ elsePart => + val altExprs = altPart.flatMap { + case we ~ te => + Seq(casePart.fold(we)(EqualTo(_, we)), te) + } + CaseWhen(altExprs ++ elsePart.toList) + } | (SUBSTR | SUBSTRING) ~> "(" ~> expression ~ "," ~ expression <~ ")" ^^ { case s ~ "," ~ p => Substring(s,p,Literal(Integer.MAX_VALUE)) } | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index b9b196ea5a46a..79de1bb855dbe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -680,9 +680,20 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT CAST(TRUE AS STRING), CAST(FALSE AS STRING) FROM testData LIMIT 1"), ("true", "false") :: Nil) } - + test("SPARK-3371 Renaming a function expression with group by gives error") { registerFunction("len", (s: String) => s.length) checkAnswer( - sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"), 1)} + sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"), 1) + } + + test("SPARK-3813 CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END") { + checkAnswer( + sql("SELECT CASE key WHEN 1 THEN 1 ELSE 0 END FROM testData WHERE key = 1 group by key"), 1) + } + + test("SPARK-3813 CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END") { + checkAnswer( + sql("SELECT CASE WHEN key=1 THEN 1 ELSE 2 END FROM testData WHERE key = 1 group by key"), 1) + } } From 4e9b551a0b807f5a2cc6679165c8be4e88a3d077 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 9 Oct 2014 16:08:07 -0700 Subject: [PATCH 0954/1492] [SPARK-3772] Allow `ipython` to be used by Pyspark workers; IPython support improvements: This pull request addresses a few issues related to PySpark's IPython support: - Fix the remaining uses of the '-u' flag, which IPython doesn't support (see SPARK-3772). - Change PYSPARK_PYTHON_OPTS to PYSPARK_DRIVER_PYTHON_OPTS, so that the old name is reserved in case we ever want to allow the worker Python options to be customized (this variable was introduced in #2554 and hasn't landed in a release yet, so this doesn't break any compatibility). - Introduce a PYSPARK_DRIVER_PYTHON option that allows the driver to use `ipython` while the workers use a different Python version. - Attempt to use Python 2.7 by default if PYSPARK_PYTHON is not specified. - Retain the old semantics for IPYTHON=1 and IPYTHON_OPTS (to avoid breaking existing example programs). There are more details in a block comment in `bin/pyspark`. Author: Josh Rosen Closes #2651 from JoshRosen/SPARK-3772 and squashes the following commits: 7b8eb86 [Josh Rosen] More changes to PySpark python executable configuration: c4f5778 [Josh Rosen] [SPARK-3772] Allow ipython to be used by Pyspark workers; IPython fixes: --- bin/pyspark | 51 ++++++++++++++----- .../api/python/PythonWorkerFactory.scala | 8 ++- .../apache/spark/deploy/PythonRunner.scala | 4 +- docs/programming-guide.md | 8 +-- 4 files changed, 51 insertions(+), 20 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 6655725ef8e8e..96f30a260a09e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -50,22 +50,47 @@ fi . "$FWDIR"/bin/load-spark-env.sh -# Figure out which Python executable to use +# In Spark <= 1.1, setting IPYTHON=1 would cause the driver to be launched using the `ipython` +# executable, while the worker would still be launched using PYSPARK_PYTHON. +# +# In Spark 1.2, we removed the documentation of the IPYTHON and IPYTHON_OPTS variables and added +# PYSPARK_DRIVER_PYTHON and PYSPARK_DRIVER_PYTHON_OPTS to allow IPython to be used for the driver. +# Now, users can simply set PYSPARK_DRIVER_PYTHON=ipython to use IPython and set +# PYSPARK_DRIVER_PYTHON_OPTS to pass options when starting the Python driver +# (e.g. PYSPARK_DRIVER_PYTHON_OPTS='notebook'). This supports full customization of the IPython +# and executor Python executables. +# +# For backwards-compatibility, we retain the old IPYTHON and IPYTHON_OPTS variables. + +# Determine the Python executable to use if PYSPARK_PYTHON or PYSPARK_DRIVER_PYTHON isn't set: +if hash python2.7 2>/dev/null; then + # Attempt to use Python 2.7, if installed: + DEFAULT_PYTHON="python2.7" +else + DEFAULT_PYTHON="python" +fi + +# Determine the Python executable to use for the driver: +if [[ -n "$IPYTHON_OPTS" || "$IPYTHON" == "1" ]]; then + # If IPython options are specified, assume user wants to run IPython + # (for backwards-compatibility) + PYSPARK_DRIVER_PYTHON_OPTS="$PYSPARK_DRIVER_PYTHON_OPTS $IPYTHON_OPTS" + PYSPARK_DRIVER_PYTHON="ipython" +elif [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then + PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"$DEFAULT_PYTHON"}" +fi + +# Determine the Python executable to use for the executors: if [[ -z "$PYSPARK_PYTHON" ]]; then - if [[ "$IPYTHON" = "1" || -n "$IPYTHON_OPTS" ]]; then - # for backward compatibility - PYSPARK_PYTHON="ipython" + if [[ $PYSPARK_DRIVER_PYTHON == *ipython* && $DEFAULT_PYTHON != "python2.7" ]]; then + echo "IPython requires Python 2.7+; please install python2.7 or set PYSPARK_PYTHON" 1>&2 + exit 1 else - PYSPARK_PYTHON="python" + PYSPARK_PYTHON="$DEFAULT_PYTHON" fi fi export PYSPARK_PYTHON -if [[ -z "$PYSPARK_PYTHON_OPTS" && -n "$IPYTHON_OPTS" ]]; then - # for backward compatibility - PYSPARK_PYTHON_OPTS="$IPYTHON_OPTS" -fi - # Add the PySpark classes to the Python path: export PYTHONPATH="$SPARK_HOME/python/:$PYTHONPATH" export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" @@ -93,9 +118,9 @@ if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR if [[ -n "$PYSPARK_DOC_TEST" ]]; then - exec "$PYSPARK_PYTHON" -m doctest $1 + exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else - exec "$PYSPARK_PYTHON" $1 + exec "$PYSPARK_DRIVER_PYTHON" $1 fi exit fi @@ -111,5 +136,5 @@ if [[ "$1" =~ \.py$ ]]; then else # PySpark shell requires special handling downstream export PYSPARK_SHELL=1 - exec "$PYSPARK_PYTHON" $PYSPARK_PYTHON_OPTS + exec "$PYSPARK_DRIVER_PYTHON" $PYSPARK_DRIVER_PYTHON_OPTS fi diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 71bdf0fe1b917..e314408c067e9 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -108,10 +108,12 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) // Create and start the worker - val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.worker")) + val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.worker")) val workerEnv = pb.environment() workerEnv.putAll(envVars) workerEnv.put("PYTHONPATH", pythonPath) + // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u: + workerEnv.put("PYTHONUNBUFFERED", "YES") val worker = pb.start() // Redirect worker stdout and stderr @@ -149,10 +151,12 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { // Create and start the daemon - val pb = new ProcessBuilder(Seq(pythonExec, "-u", "-m", "pyspark.daemon")) + val pb = new ProcessBuilder(Seq(pythonExec, "-m", "pyspark.daemon")) val workerEnv = pb.environment() workerEnv.putAll(envVars) workerEnv.put("PYTHONPATH", pythonPath) + // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u: + workerEnv.put("PYTHONUNBUFFERED", "YES") daemon = pb.start() val in = new DataInputStream(daemon.getInputStream) diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index 79b4d7ea41a33..af94b05ce3847 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -34,7 +34,8 @@ object PythonRunner { val pythonFile = args(0) val pyFiles = args(1) val otherArgs = args.slice(2, args.length) - val pythonExec = sys.env.get("PYSPARK_PYTHON").getOrElse("python") // TODO: get this from conf + val pythonExec = + sys.env.getOrElse("PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python")) // Format python file paths before adding them to the PYTHONPATH val formattedPythonFile = formatPath(pythonFile) @@ -57,6 +58,7 @@ object PythonRunner { val builder = new ProcessBuilder(Seq(pythonExec, formattedPythonFile) ++ otherArgs) val env = builder.environment() env.put("PYTHONPATH", pythonPath) + // This is equivalent to setting the -u flag; we use it because ipython doesn't support -u: env.put("PYTHONUNBUFFERED", "YES") // value is needed to be set to a non-empty string env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 8e8cc1dd983f8..18420afb27e3c 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -211,17 +211,17 @@ For a complete list of options, run `pyspark --help`. Behind the scenes, It is also possible to launch the PySpark shell in [IPython](http://ipython.org), the enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To -use IPython, set the `PYSPARK_PYTHON` variable to `ipython` when running `bin/pyspark`: +use IPython, set the `PYSPARK_DRIVER_PYTHON` variable to `ipython` when running `bin/pyspark`: {% highlight bash %} -$ PYSPARK_PYTHON=ipython ./bin/pyspark +$ PYSPARK_DRIVER_PYTHON=ipython ./bin/pyspark {% endhighlight %} -You can customize the `ipython` command by setting `PYSPARK_PYTHON_OPTS`. For example, to launch +You can customize the `ipython` command by setting `PYSPARK_DRIVER_PYTHON_OPTS`. For example, to launch the [IPython Notebook](http://ipython.org/notebook.html) with PyLab plot support: {% highlight bash %} -$ PYSPARK_PYTHON=ipython PYSPARK_PYTHON_OPTS="notebook --pylab inline" ./bin/pyspark +$ PYSPARK_DRIVER_PYTHON=ipython PYSPARK_DRIVER_PYTHON_OPTS="notebook --pylab inline" ./bin/pyspark {% endhighlight %} From 2837bf8548db7e9d43f6eefedf5a73feb22daedb Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 9 Oct 2014 17:54:02 -0700 Subject: [PATCH 0955/1492] [SPARK-3798][SQL] Store the output of a generator in a val This prevents it from changing during serialization, leading to corrupted results. Author: Michael Armbrust Closes #2656 from marmbrus/generateBug and squashes the following commits: efa32eb [Michael Armbrust] Store the output of a generator in a val. This prevents it from changing during serialization. --- .../main/scala/org/apache/spark/sql/execution/Generate.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala index c386fd121c5de..38877c28de3a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -39,7 +39,8 @@ case class Generate( child: SparkPlan) extends UnaryNode { - protected def generatorOutput: Seq[Attribute] = { + // This must be a val since the generator output expr ids are not preserved by serialization. + protected val generatorOutput: Seq[Attribute] = { if (join && outer) { generator.output.map(_.withNullability(true)) } else { @@ -62,7 +63,7 @@ case class Generate( newProjection(child.output ++ nullValues, child.output) val joinProjection = - newProjection(child.output ++ generator.output, child.output ++ generator.output) + newProjection(child.output ++ generatorOutput, child.output ++ generatorOutput) val joinedRow = new JoinedRow iter.flatMap {row => From 363baacaded56047bcc63276d729ab911e0336cf Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 9 Oct 2014 18:21:59 -0700 Subject: [PATCH 0956/1492] SPARK-3811 [CORE] More robust / standard Utils.deleteRecursively, Utils.createTempDir I noticed a few issues with how temp directories are created and deleted: *Minor* * Guava's `Files.createTempDir()` plus `File.deleteOnExit()` is used in many tests to make a temp dir, but `Utils.createTempDir()` seems to be the standard Spark mechanism * Call to `File.deleteOnExit()` could be pushed into `Utils.createTempDir()` as well, along with this replacement * _I messed up the message in an exception in `Utils` in SPARK-3794; fixed here_ *Bit Less Minor* * `Utils.deleteRecursively()` fails immediately if any `IOException` occurs, instead of trying to delete any remaining files and subdirectories. I've observed this leave temp dirs around. I suggest changing it to continue in the face of an exception and throw one of the possibly several exceptions that occur at the end. * `Utils.createTempDir()` will add a JVM shutdown hook every time the method is called. Even if the subdir is the parent of another parent dir, since this check is inside the hook. However `Utils` manages a set of all dirs to delete on shutdown already, called `shutdownDeletePaths`. A single hook can be registered to delete all of these on exit. This is how Tachyon temp paths are cleaned up in `TachyonBlockManager`. I noticed a few other things that might be changed but wanted to ask first: * Shouldn't the set of dirs to delete be `File`, not just `String` paths? * `Utils` manages the set of `TachyonFile` that have been registered for deletion, but the shutdown hook is managed in `TachyonBlockManager`. Should this logic not live together, and not in `Utils`? it's more specific to Tachyon, and looks a slight bit odd to import in such a generic place. Author: Sean Owen Closes #2670 from srowen/SPARK-3811 and squashes the following commits: 071ae60 [Sean Owen] Update per @vanzin's review da0146d [Sean Owen] Make Utils.deleteRecursively try to delete all paths even when an exception occurs; use one shutdown hook instead of one per method call to delete temp dirs 3a0faa4 [Sean Owen] Standardize on Utils.createTempDir instead of Files.createTempDir --- .../scala/org/apache/spark/TestUtils.scala | 5 +- .../scala/org/apache/spark/util/Utils.scala | 55 +++++++++++++------ .../org/apache/spark/FileServerSuite.scala | 4 +- .../scala/org/apache/spark/FileSuite.scala | 4 +- .../spark/deploy/SparkSubmitSuite.scala | 3 +- .../WholeTextFileRecordReaderSuite.scala | 6 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 21 ++++--- .../scheduler/EventLoggingListenerSuite.scala | 4 +- .../spark/scheduler/ReplayListenerSuite.scala | 4 +- .../spark/storage/DiskBlockManagerSuite.scala | 17 +----- .../apache/spark/util/FileLoggerSuite.scala | 3 +- .../org/apache/spark/util/UtilsSuite.scala | 28 +++++++++- .../spark/mllib/util/MLUtilsSuite.scala | 9 ++- .../spark/repl/ExecutorClassLoaderSuite.scala | 8 +-- .../org/apache/spark/repl/ReplSuite.scala | 4 +- .../spark/streaming/CheckpointSuite.scala | 3 +- .../spark/streaming/InputStreamsSuite.scala | 3 +- .../spark/streaming/MasterFailureTest.scala | 3 +- .../spark/streaming/TestSuiteBase.scala | 5 +- .../spark/deploy/yarn/ClientBaseSuite.scala | 5 +- 20 files changed, 102 insertions(+), 92 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 8ca731038e528..e72826dc25f41 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -26,6 +26,8 @@ import scala.collection.JavaConversions._ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import com.google.common.io.Files +import org.apache.spark.util.Utils + /** * Utilities for tests. Included in main codebase since it's used by multiple * projects. @@ -42,8 +44,7 @@ private[spark] object TestUtils { * in order to avoid interference between tests. */ def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { - val tempDir = Files.createTempDir() - tempDir.deleteOnExit() + val tempDir = Utils.createTempDir() val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) 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 3d307b3c16d3e..07477dd460a4b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -168,6 +168,20 @@ private[spark] object Utils extends Logging { private val shutdownDeletePaths = new scala.collection.mutable.HashSet[String]() private val shutdownDeleteTachyonPaths = new scala.collection.mutable.HashSet[String]() + // Add a shutdown hook to delete the temp dirs when the JVM exits + Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dirs") { + override def run(): Unit = Utils.logUncaughtExceptions { + logDebug("Shutdown hook called") + shutdownDeletePaths.foreach { dirPath => + try { + Utils.deleteRecursively(new File(dirPath)) + } catch { + case e: Exception => logError(s"Exception while deleting Spark temp dir: $dirPath", e) + } + } + } + }) + // Register the path to be deleted via shutdown hook def registerShutdownDeleteDir(file: File) { val absolutePath = file.getAbsolutePath() @@ -252,14 +266,6 @@ private[spark] object Utils extends Logging { } registerShutdownDeleteDir(dir) - - // Add a shutdown hook to delete the temp dir when the JVM exits - Runtime.getRuntime.addShutdownHook(new Thread("delete Spark temp dir " + dir) { - override def run() { - // Attempt to delete if some patch which is parent of this is not already registered. - if (! hasRootAsShutdownDeleteDir(dir)) Utils.deleteRecursively(dir) - } - }) dir } @@ -666,15 +672,30 @@ private[spark] object Utils extends Logging { */ def deleteRecursively(file: File) { if (file != null) { - if (file.isDirectory() && !isSymlink(file)) { - for (child <- listFilesSafely(file)) { - deleteRecursively(child) + try { + if (file.isDirectory && !isSymlink(file)) { + var savedIOException: IOException = null + for (child <- listFilesSafely(file)) { + try { + deleteRecursively(child) + } catch { + // In case of multiple exceptions, only last one will be thrown + case ioe: IOException => savedIOException = ioe + } + } + if (savedIOException != null) { + throw savedIOException + } + shutdownDeletePaths.synchronized { + shutdownDeletePaths.remove(file.getAbsolutePath) + } } - } - if (!file.delete()) { - // Delete can also fail if the file simply did not exist - if (file.exists()) { - throw new IOException("Failed to delete: " + file.getAbsolutePath) + } finally { + if (!file.delete()) { + // Delete can also fail if the file simply did not exist + if (file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath) + } } } } @@ -713,7 +734,7 @@ private[spark] object Utils extends Logging { */ def doesDirectoryContainAnyNewFiles(dir: File, cutoff: Long): Boolean = { if (!dir.isDirectory) { - throw new IllegalArgumentException("$dir is not a directory!") + throw new IllegalArgumentException(s"$dir is not a directory!") } val filesAndDirs = dir.listFiles() val cutoffTimeInMillis = System.currentTimeMillis - (cutoff * 1000) diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 7e18f45de7b5b..a8867020e457d 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark import java.io._ import java.util.jar.{JarEntry, JarOutputStream} -import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext._ @@ -41,8 +40,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { override def beforeAll() { super.beforeAll() - tmpDir = Files.createTempDir() - tmpDir.deleteOnExit() + tmpDir = Utils.createTempDir() val testTempDir = new File(tmpDir, "test") testTempDir.mkdir() diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 4a53d25012ad9..a2b74c4419d46 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -21,7 +21,6 @@ import java.io.{File, FileWriter} import scala.io.Source -import com.google.common.io.Files import org.apache.hadoop.io._ import org.apache.hadoop.io.compress.DefaultCodec import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, FileSplit, TextInputFormat, TextOutputFormat} @@ -39,8 +38,7 @@ class FileSuite extends FunSuite with LocalSparkContext { override def beforeEach() { super.beforeEach() - tempDir = Files.createTempDir() - tempDir.deleteOnExit() + tempDir = Utils.createTempDir() } override def afterEach() { diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 4cba90e8f2afe..1cdf50d5c08c7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -26,7 +26,6 @@ import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.util.Utils import org.scalatest.FunSuite import org.scalatest.Matchers -import com.google.common.io.Files class SparkSubmitSuite extends FunSuite with Matchers { def beforeAll() { @@ -332,7 +331,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { } def forConfDir(defaults: Map[String, String]) (f: String => Unit) = { - val tmpDir = Files.createTempDir() + val tmpDir = Utils.createTempDir() val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf)) diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index d5ebfb3f3fae1..12d1c7b2faba6 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -23,8 +23,6 @@ import java.io.FileOutputStream import scala.collection.immutable.IndexedSeq -import com.google.common.io.Files - import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite @@ -66,9 +64,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { * 3) Does the contents be the same. */ test("Correctness of WholeTextFileRecordReader.") { - - val dir = Files.createTempDir() - dir.deleteOnExit() + val dir = Utils.createTempDir() println(s"Local disk address is ${dir.toString}.") WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 75b01191901b8..3620e251cc139 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -24,13 +24,14 @@ import org.apache.hadoop.util.Progressable import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.util.Random -import com.google.common.io.Files import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter => NewOutputCommitter, OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext} import org.apache.spark.{Partitioner, SharedSparkContext} import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils + import org.scalatest.FunSuite class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { @@ -381,14 +382,16 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } test("zero-partition RDD") { - val emptyDir = Files.createTempDir() - emptyDir.deleteOnExit() - val file = sc.textFile(emptyDir.getAbsolutePath) - assert(file.partitions.size == 0) - assert(file.collect().toList === Nil) - // Test that a shuffle on the file works, because this used to be a bug - assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - emptyDir.delete() + val emptyDir = Utils.createTempDir() + try { + val file = sc.textFile(emptyDir.getAbsolutePath) + assert(file.partitions.isEmpty) + assert(file.collect().toList === Nil) + // Test that a shuffle on the file works, because this used to be a bug + assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) + } finally { + Utils.deleteRecursively(emptyDir) + } } test("keys and values") { diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 3efa85431876b..abc300fcffaf9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.scheduler import scala.collection.mutable import scala.io.Source -import com.google.common.io.Files import org.apache.hadoop.fs.{FileStatus, Path} import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -51,8 +50,7 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { private var logDirPath: Path = _ before { - testDir = Files.createTempDir() - testDir.deleteOnExit() + testDir = Utils.createTempDir() logDirPath = Utils.getFilePath(testDir, "spark-events") } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 48114feee6233..e05f373392d4a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.scheduler import java.io.{File, PrintWriter} -import com.google.common.io.Files import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} @@ -39,8 +38,7 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { private var testDir: File = _ before { - testDir = Files.createTempDir() - testDir.deleteOnExit() + testDir = Utils.createTempDir() } after { diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index e4522e00a622d..bc5c74c126b74 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -19,22 +19,13 @@ package org.apache.spark.storage import java.io.{File, FileWriter} -import org.apache.spark.network.nio.NioBlockTransferService -import org.apache.spark.shuffle.hash.HashShuffleManager - -import scala.collection.mutable import scala.language.reflectiveCalls -import akka.actor.Props -import com.google.common.io.Files import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.SparkConf -import org.apache.spark.scheduler.LiveListenerBus -import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.util.{AkkaUtils, Utils} -import org.apache.spark.executor.ShuffleWriteMetrics +import org.apache.spark.util.Utils class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll { private val testConf = new SparkConf(false) @@ -48,10 +39,8 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach with Before override def beforeAll() { super.beforeAll() - rootDir0 = Files.createTempDir() - rootDir0.deleteOnExit() - rootDir1 = Files.createTempDir() - rootDir1.deleteOnExit() + rootDir0 = Utils.createTempDir() + rootDir1 = Utils.createTempDir() rootDirs = rootDir0.getAbsolutePath + "," + rootDir1.getAbsolutePath } diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala index c3dd156b40514..dc2a05631d83d 100644 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -21,7 +21,6 @@ import java.io.{File, IOException} import scala.io.Source -import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.scalatest.{BeforeAndAfter, FunSuite} @@ -44,7 +43,7 @@ class FileLoggerSuite extends FunSuite with BeforeAndAfter { private var logDirPathString: String = _ before { - testDir = Files.createTempDir() + testDir = Utils.createTempDir() logDirPath = Utils.getFilePath(testDir, "test-file-logger") logDirPathString = logDirPath.toString } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index e63d9d085e385..0344da60dae66 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -112,7 +112,7 @@ class UtilsSuite extends FunSuite { } test("reading offset bytes of a file") { - val tmpDir2 = Files.createTempDir() + val tmpDir2 = Utils.createTempDir() tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) @@ -141,7 +141,7 @@ class UtilsSuite extends FunSuite { } test("reading offset bytes across multiple files") { - val tmpDir = Files.createTempDir() + val tmpDir = Utils.createTempDir() tmpDir.deleteOnExit() val files = (1 to 3).map(i => new File(tmpDir, i.toString)) Files.write("0123456789", files(0), Charsets.UTF_8) @@ -308,4 +308,28 @@ class UtilsSuite extends FunSuite { } } + test("deleteRecursively") { + val tempDir1 = Utils.createTempDir() + assert(tempDir1.exists()) + Utils.deleteRecursively(tempDir1) + assert(!tempDir1.exists()) + + val tempDir2 = Utils.createTempDir() + val tempFile1 = new File(tempDir2, "foo.txt") + Files.touch(tempFile1) + assert(tempFile1.exists()) + Utils.deleteRecursively(tempFile1) + assert(!tempFile1.exists()) + + val tempDir3 = new File(tempDir2, "subdir") + assert(tempDir3.mkdir()) + val tempFile2 = new File(tempDir3, "bar.txt") + Files.touch(tempFile2) + assert(tempFile2.exists()) + Utils.deleteRecursively(tempDir2) + assert(!tempDir2.exists()) + assert(!tempDir3.exists()) + assert(!tempFile2.exists()) + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 8ef2bb1bf6a78..0dbe766b4d917 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -67,8 +67,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { |0 |0 2:4.0 4:5.0 6:6.0 """.stripMargin - val tempDir = Files.createTempDir() - tempDir.deleteOnExit() + val tempDir = Utils.createTempDir() val file = new File(tempDir.getPath, "part-00000") Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString @@ -100,7 +99,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { LabeledPoint(1.1, Vectors.sparse(3, Seq((0, 1.23), (2, 4.56)))), LabeledPoint(0.0, Vectors.dense(1.01, 2.02, 3.03)) ), 2) - val tempDir = Files.createTempDir() + val tempDir = Utils.createTempDir() val outputDir = new File(tempDir, "output") MLUtils.saveAsLibSVMFile(examples, outputDir.toURI.toString) val lines = outputDir.listFiles() @@ -166,7 +165,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Vectors.sparse(2, Array(1), Array(-1.0)), Vectors.dense(0.0, 1.0) ), 2) - val tempDir = Files.createTempDir() + val tempDir = Utils.createTempDir() val outputDir = new File(tempDir, "vectors") val path = outputDir.toURI.toString vectors.saveAsTextFile(path) @@ -181,7 +180,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { LabeledPoint(0.0, Vectors.sparse(2, Array(1), Array(-1.0))), LabeledPoint(1.0, Vectors.dense(0.0, 1.0)) ), 2) - val tempDir = Files.createTempDir() + val tempDir = Utils.createTempDir() val outputDir = new File(tempDir, "points") val path = outputDir.toURI.toString points.saveAsTextFile(path) diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index 3e2ee7541f40d..6a79e76a34db8 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -23,8 +23,6 @@ import java.net.{URL, URLClassLoader} import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import com.google.common.io.Files - import org.apache.spark.{SparkConf, TestUtils} import org.apache.spark.util.Utils @@ -39,10 +37,8 @@ class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { super.beforeAll() - tempDir1 = Files.createTempDir() - tempDir1.deleteOnExit() - tempDir2 = Files.createTempDir() - tempDir2.deleteOnExit() + tempDir1 = Utils.createTempDir() + tempDir2 = Utils.createTempDir() url1 = "file://" + tempDir1 urls2 = List(tempDir2.toURI.toURL).toArray childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1")) diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index c8763eb277052..91c9c52c3c98a 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -22,7 +22,6 @@ import java.net.URLClassLoader import scala.collection.mutable.ArrayBuffer -import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext import org.apache.commons.lang3.StringEscapeUtils @@ -190,8 +189,7 @@ class ReplSuite extends FunSuite { } test("interacting with files") { - val tempDir = Files.createTempDir() - tempDir.deleteOnExit() + val tempDir = Utils.createTempDir() val out = new FileWriter(tempDir + "/input") out.write("Hello world!\n") out.write("What's up?\n") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 8511390cb1ad5..e5592e52b0d2d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -231,8 +231,7 @@ class CheckpointSuite extends TestSuiteBase { // failure, are re-processed or not. test("recovery with file input stream") { // Set up the streaming context and input streams - val testDir = Files.createTempDir() - testDir.deleteOnExit() + val testDir = Utils.createTempDir() var ssc = new StreamingContext(master, framework, Seconds(1)) ssc.checkpoint(checkpointDir) val fileStream = ssc.textFileStream(testDir.toString) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 952a74fd5f6de..a44a45a3e9bd6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -98,8 +98,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") // Set up the streaming context and input streams - val testDir = Files.createTempDir() - testDir.deleteOnExit() + val testDir = Utils.createTempDir() val ssc = new StreamingContext(conf, batchDuration) val fileStream = ssc.textFileStream(testDir.toString) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index c53c01706083a..5dbb7232009eb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -352,8 +352,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) extends Thread with Logging { override def run() { - val localTestDir = Files.createTempDir() - localTestDir.deleteOnExit() + val localTestDir = Utils.createTempDir() var fs = testDir.getFileSystem(new Configuration()) val maxTries = 3 try { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 759baacaa4308..9327ff4822699 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -24,12 +24,12 @@ import scala.collection.mutable.SynchronizedBuffer import scala.reflect.ClassTag import org.scalatest.{BeforeAndAfter, FunSuite} -import com.google.common.io.Files import org.apache.spark.streaming.dstream.{DStream, InputDStream, ForEachDStream} import org.apache.spark.streaming.util.ManualClock import org.apache.spark.{SparkConf, Logging} import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** * This is a input stream just for the testsuites. This is equivalent to a checkpointable, @@ -120,9 +120,8 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { // Directory where the checkpoint data will be saved lazy val checkpointDir = { - val dir = Files.createTempDir() + val dir = Utils.createTempDir() logDebug(s"checkpointDir: $dir") - dir.deleteOnExit() dir.toString } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 9bd916100dd2c..17b79ae1d82c4 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -20,13 +20,10 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI -import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ @@ -117,7 +114,7 @@ class ClientBaseSuite extends FunSuite with Matchers { doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), any(classOf[Path]), anyShort(), anyBoolean()) - val tempDir = Files.createTempDir() + val tempDir = Utils.createTempDir() try { client.prepareLocalResources(tempDir.getAbsolutePath()) sparkConf.getOption(ClientBase.CONF_SPARK_USER_JAR) should be (Some(USER)) From edf02da389f75df5a42465d41f035d6b65599848 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 9 Oct 2014 18:25:06 -0700 Subject: [PATCH 0957/1492] [SPARK-3654][SQL] Unifies SQL and HiveQL parsers This PR is a follow up of #2590, and tries to introduce a top level SQL parser entry point for all SQL dialects supported by Spark SQL. A top level parser `SparkSQLParser` is introduced to handle the syntaxes that all SQL dialects should recognize (e.g. `CACHE TABLE`, `UNCACHE TABLE` and `SET`, etc.). For all the syntaxes this parser doesn't recognize directly, it fallbacks to a specified function that tries to parse arbitrary input to a `LogicalPlan`. This function is typically another parser combinator like `SqlParser`. DDL syntaxes introduced in #2475 can be moved to here. The `ExtendedHiveQlParser` now only handle Hive specific extensions. Also took the chance to refactor/reformat `SqlParser` for better readability. Author: Cheng Lian Closes #2698 from liancheng/gen-sql-parser and squashes the following commits: ceada76 [Cheng Lian] Minor styling fixes 9738934 [Cheng Lian] Minor refactoring, removes optional trailing ";" in the parser bb2ab12 [Cheng Lian] SET property value can be empty string ce8860b [Cheng Lian] Passes test suites e86968e [Cheng Lian] Removes debugging code 8bcace5 [Cheng Lian] Replaces digit.+ to rep1(digit) (Scala style checking doesn't like it) d15d54f [Cheng Lian] Unifies SQL and HiveQL parsers --- .../spark/sql/catalyst/SparkSQLParser.scala | 186 ++++++++ .../apache/spark/sql/catalyst/SqlParser.scala | 426 +++++++----------- .../sql/catalyst/plans/logical/commands.scala | 15 +- .../org/apache/spark/sql/SQLContext.scala | 9 +- .../spark/sql/execution/SparkStrategies.scala | 4 +- .../apache/spark/sql/execution/commands.scala | 34 +- .../apache/spark/sql/CachedTableSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 4 +- .../server/SparkSQLOperationManager.scala | 2 +- .../spark/sql/hive/ExtendedHiveQlParser.scala | 110 +---- .../org/apache/spark/sql/hive/HiveQl.scala | 15 +- .../spark/sql/hive/HiveStrategies.scala | 8 +- 12 files changed, 414 insertions(+), 401 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala new file mode 100644 index 0000000000000..04467342e6ab5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -0,0 +1,186 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import scala.language.implicitConversions +import scala.util.parsing.combinator.lexical.StdLexical +import scala.util.parsing.combinator.syntactical.StandardTokenParsers +import scala.util.parsing.combinator.{PackratParsers, RegexParsers} +import scala.util.parsing.input.CharArrayReader.EofCh + +import org.apache.spark.sql.catalyst.plans.logical._ + +private[sql] abstract class AbstractSparkSQLParser + extends StandardTokenParsers with PackratParsers { + + def apply(input: String): LogicalPlan = phrase(start)(new lexical.Scanner(input)) match { + case Success(plan, _) => plan + case failureOrError => sys.error(failureOrError.toString) + } + + protected case class Keyword(str: String) + + protected def start: Parser[LogicalPlan] + + // Returns the whole input string + protected lazy val wholeInput: Parser[String] = new Parser[String] { + def apply(in: Input): ParseResult[String] = + Success(in.source.toString, in.drop(in.source.length())) + } + + // Returns the rest of the input string that are not parsed yet + protected lazy val restInput: Parser[String] = new Parser[String] { + def apply(in: Input): ParseResult[String] = + Success( + in.source.subSequence(in.offset, in.source.length()).toString, + in.drop(in.source.length())) + } +} + +class SqlLexical(val keywords: Seq[String]) extends StdLexical { + case class FloatLit(chars: String) extends Token { + override def toString = chars + } + + reserved ++= keywords.flatMap(w => allCaseVersions(w)) + + delimiters += ( + "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", + ",", ";", "%", "{", "}", ":", "[", "]", "." + ) + + override lazy val token: Parser[Token] = + ( identChar ~ (identChar | digit).* ^^ + { case first ~ rest => processIdent((first :: rest).mkString) } + | rep1(digit) ~ ('.' ~> digit.*).? ^^ { + case i ~ None => NumericLit(i.mkString) + case i ~ Some(d) => FloatLit(i.mkString + "." + d.mkString) + } + | '\'' ~> chrExcept('\'', '\n', EofCh).* <~ '\'' ^^ + { case chars => StringLit(chars mkString "") } + | '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^ + { case chars => StringLit(chars mkString "") } + | EofCh ^^^ EOF + | '\'' ~> failure("unclosed string literal") + | '"' ~> failure("unclosed string literal") + | delim + | failure("illegal character") + ) + + override def identChar = letter | elem('_') + + override def whitespace: Parser[Any] = + ( whitespaceChar + | '/' ~ '*' ~ comment + | '/' ~ '/' ~ chrExcept(EofCh, '\n').* + | '#' ~ chrExcept(EofCh, '\n').* + | '-' ~ '-' ~ chrExcept(EofCh, '\n').* + | '/' ~ '*' ~ failure("unclosed comment") + ).* + + /** Generate all variations of upper and lower case of a given string */ + def allCaseVersions(s: String, prefix: String = ""): Stream[String] = { + if (s == "") { + Stream(prefix) + } else { + allCaseVersions(s.tail, prefix + s.head.toLower) ++ + allCaseVersions(s.tail, prefix + s.head.toUpper) + } + } +} + +/** + * The top level Spark SQL parser. This parser recognizes syntaxes that are available for all SQL + * dialects supported by Spark SQL, and delegates all the other syntaxes to the `fallback` parser. + * + * @param fallback A function that parses an input string to a logical plan + */ +private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends AbstractSparkSQLParser { + + // A parser for the key-value part of the "SET [key = [value ]]" syntax + private object SetCommandParser extends RegexParsers { + private val key: Parser[String] = "(?m)[^=]+".r + + private val value: Parser[String] = "(?m).*$".r + + private val pair: Parser[LogicalPlan] = + (key ~ ("=".r ~> value).?).? ^^ { + case None => SetCommand(None) + case Some(k ~ v) => SetCommand(Some(k.trim -> v.map(_.trim))) + } + + def apply(input: String): LogicalPlan = parseAll(pair, input) match { + case Success(plan, _) => plan + case x => sys.error(x.toString) + } + } + + protected val AS = Keyword("AS") + protected val CACHE = Keyword("CACHE") + protected val LAZY = Keyword("LAZY") + protected val SET = Keyword("SET") + protected val TABLE = Keyword("TABLE") + protected val SOURCE = Keyword("SOURCE") + protected val UNCACHE = Keyword("UNCACHE") + + protected implicit def asParser(k: Keyword): Parser[String] = + lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) + + private val reservedWords: Seq[String] = + this + .getClass + .getMethods + .filter(_.getReturnType == classOf[Keyword]) + .map(_.invoke(this).asInstanceOf[Keyword].str) + + override val lexical = new SqlLexical(reservedWords) + + override protected lazy val start: Parser[LogicalPlan] = + cache | uncache | set | shell | source | others + + private lazy val cache: Parser[LogicalPlan] = + CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ { + case isLazy ~ tableName ~ plan => + CacheTableCommand(tableName, plan.map(fallback), isLazy.isDefined) + } + + private lazy val uncache: Parser[LogicalPlan] = + UNCACHE ~ TABLE ~> ident ^^ { + case tableName => UncacheTableCommand(tableName) + } + + private lazy val set: Parser[LogicalPlan] = + SET ~> restInput ^^ { + case input => SetCommandParser(input) + } + + private lazy val shell: Parser[LogicalPlan] = + "!" ~> restInput ^^ { + case input => ShellCommand(input.trim) + } + + private lazy val source: Parser[LogicalPlan] = + SOURCE ~> restInput ^^ { + case input => SourceCommand(input.trim) + } + + private lazy val others: Parser[LogicalPlan] = + wholeInput ^^ { + case input => fallback(input) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 4662f585cfe15..b4d606d37e732 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -18,10 +18,6 @@ package org.apache.spark.sql.catalyst import scala.language.implicitConversions -import scala.util.parsing.combinator.lexical.StdLexical -import scala.util.parsing.combinator.syntactical.StandardTokenParsers -import scala.util.parsing.combinator.PackratParsers -import scala.util.parsing.input.CharArrayReader.EofCh import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -39,31 +35,7 @@ import org.apache.spark.sql.catalyst.types._ * This is currently included mostly for illustrative purposes. Users wanting more complete support * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. */ -class SqlParser extends StandardTokenParsers with PackratParsers { - - def apply(input: String): LogicalPlan = { - // Special-case out set commands since the value fields can be - // complex to handle without RegexParsers. Also this approach - // is clearer for the several possible cases of set commands. - if (input.trim.toLowerCase.startsWith("set")) { - input.trim.drop(3).split("=", 2).map(_.trim) match { - case Array("") => // "set" - SetCommand(None, None) - case Array(key) => // "set key" - SetCommand(Some(key), None) - case Array(key, value) => // "set key=value" - SetCommand(Some(key), Some(value)) - } - } else { - phrase(query)(new lexical.Scanner(input)) match { - case Success(r, x) => r - case x => sys.error(x.toString) - } - } - } - - protected case class Keyword(str: String) - +class SqlParser extends AbstractSparkSQLParser { protected implicit def asParser(k: Keyword): Parser[String] = lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) @@ -100,7 +72,6 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val IS = Keyword("IS") protected val JOIN = Keyword("JOIN") protected val LAST = Keyword("LAST") - protected val LAZY = Keyword("LAZY") protected val LEFT = Keyword("LEFT") protected val LIKE = Keyword("LIKE") protected val LIMIT = Keyword("LIMIT") @@ -128,7 +99,6 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val THEN = Keyword("THEN") protected val TIMESTAMP = Keyword("TIMESTAMP") protected val TRUE = Keyword("TRUE") - protected val UNCACHE = Keyword("UNCACHE") protected val UNION = Keyword("UNION") protected val UPPER = Keyword("UPPER") protected val WHEN = Keyword("WHEN") @@ -136,7 +106,8 @@ class SqlParser extends StandardTokenParsers with PackratParsers { // Use reflection to find the reserved words defined in this class. protected val reservedWords = - this.getClass + this + .getClass .getMethods .filter(_.getReturnType == classOf[Keyword]) .map(_.invoke(this).asInstanceOf[Keyword].str) @@ -150,86 +121,68 @@ class SqlParser extends StandardTokenParsers with PackratParsers { } } - protected lazy val query: Parser[LogicalPlan] = ( - select * ( - UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } | - INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } | - EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} | - UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } + protected lazy val start: Parser[LogicalPlan] = + ( select * + ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } + | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } + | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} + | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) - | insert | cache | unCache - ) + | insert + ) protected lazy val select: Parser[LogicalPlan] = - SELECT ~> opt(DISTINCT) ~ projections ~ - opt(from) ~ opt(filter) ~ - opt(grouping) ~ - opt(having) ~ - opt(orderBy) ~ - opt(limit) <~ opt(";") ^^ { - case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => - val base = r.getOrElse(NoRelation) - val withFilter = f.map(f => Filter(f, base)).getOrElse(base) - val withProjection = - g.map {g => - Aggregate(g, assignAliases(p), withFilter) - }.getOrElse(Project(assignAliases(p), withFilter)) - val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection) - val withHaving = h.map(h => Filter(h, withDistinct)).getOrElse(withDistinct) - val withOrder = o.map(o => Sort(o, withHaving)).getOrElse(withHaving) - val withLimit = l.map { l => Limit(l, withOrder) }.getOrElse(withOrder) - withLimit - } + SELECT ~> DISTINCT.? ~ + repsep(projection, ",") ~ + (FROM ~> relations).? ~ + (WHERE ~> expression).? ~ + (GROUP ~ BY ~> rep1sep(expression, ",")).? ~ + (HAVING ~> expression).? ~ + (ORDER ~ BY ~> ordering).? ~ + (LIMIT ~> expression).? ^^ { + case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => + val base = r.getOrElse(NoRelation) + val withFilter = f.map(f => Filter(f, base)).getOrElse(base) + val withProjection = g + .map(Aggregate(_, assignAliases(p), withFilter)) + .getOrElse(Project(assignAliases(p), withFilter)) + val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection) + val withHaving = h.map(Filter(_, withDistinct)).getOrElse(withDistinct) + val withOrder = o.map(Sort(_, withHaving)).getOrElse(withHaving) + val withLimit = l.map(Limit(_, withOrder)).getOrElse(withOrder) + withLimit + } protected lazy val insert: Parser[LogicalPlan] = - INSERT ~> opt(OVERWRITE) ~ inTo ~ select <~ opt(";") ^^ { - case o ~ r ~ s => - val overwrite: Boolean = o.getOrElse("") == "OVERWRITE" - InsertIntoTable(r, Map[String, Option[String]](), s, overwrite) - } - - protected lazy val cache: Parser[LogicalPlan] = - CACHE ~> opt(LAZY) ~ (TABLE ~> ident) ~ opt(AS ~> select) <~ opt(";") ^^ { - case isLazy ~ tableName ~ plan => - CacheTableCommand(tableName, plan, isLazy.isDefined) - } - - protected lazy val unCache: Parser[LogicalPlan] = - UNCACHE ~ TABLE ~> ident <~ opt(";") ^^ { - case tableName => UncacheTableCommand(tableName) + INSERT ~> OVERWRITE.? ~ (INTO ~> relation) ~ select ^^ { + case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o.isDefined) } - protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",") - protected lazy val projection: Parser[Expression] = - expression ~ (opt(AS) ~> opt(ident)) ^^ { - case e ~ None => e - case e ~ Some(a) => Alias(e, a)() + expression ~ (AS.? ~> ident.?) ^^ { + case e ~ a => a.fold(e)(Alias(e, _)()) } - protected lazy val from: Parser[LogicalPlan] = FROM ~> relations - - protected lazy val inTo: Parser[LogicalPlan] = INTO ~> relation - // Based very loosely on the MySQL Grammar. // http://dev.mysql.com/doc/refman/5.0/en/join.html protected lazy val relations: Parser[LogicalPlan] = - relation ~ "," ~ relation ^^ { case r1 ~ _ ~ r2 => Join(r1, r2, Inner, None) } | - relation + ( relation ~ ("," ~> relation) ^^ { case r1 ~ r2 => Join(r1, r2, Inner, None) } + | relation + ) protected lazy val relation: Parser[LogicalPlan] = - joinedRelation | - relationFactor + joinedRelation | relationFactor protected lazy val relationFactor: Parser[LogicalPlan] = - ident ~ (opt(AS) ~> opt(ident)) ^^ { - case tableName ~ alias => UnresolvedRelation(None, tableName, alias) - } | - "(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) } + ( ident ~ (opt(AS) ~> opt(ident)) ^^ { + case tableName ~ alias => UnresolvedRelation(None, tableName, alias) + } + | ("(" ~> start <~ ")") ~ (AS.? ~> ident) ^^ { case s ~ a => Subquery(a, s) } + ) protected lazy val joinedRelation: Parser[LogicalPlan] = - relationFactor ~ opt(joinType) ~ JOIN ~ relationFactor ~ opt(joinConditions) ^^ { - case r1 ~ jt ~ _ ~ r2 ~ cond => + relationFactor ~ joinType.? ~ (JOIN ~> relationFactor) ~ joinConditions.? ^^ { + case r1 ~ jt ~ r2 ~ cond => Join(r1, r2, joinType = jt.getOrElse(Inner), cond) } @@ -237,160 +190,145 @@ class SqlParser extends StandardTokenParsers with PackratParsers { ON ~> expression protected lazy val joinType: Parser[JoinType] = - INNER ^^^ Inner | - LEFT ~ SEMI ^^^ LeftSemi | - LEFT ~ opt(OUTER) ^^^ LeftOuter | - RIGHT ~ opt(OUTER) ^^^ RightOuter | - FULL ~ opt(OUTER) ^^^ FullOuter - - protected lazy val filter: Parser[Expression] = WHERE ~ expression ^^ { case _ ~ e => e } - - protected lazy val orderBy: Parser[Seq[SortOrder]] = - ORDER ~> BY ~> ordering + ( INNER ^^^ Inner + | LEFT ~ SEMI ^^^ LeftSemi + | LEFT ~ OUTER.? ^^^ LeftOuter + | RIGHT ~ OUTER.? ^^^ RightOuter + | FULL ~ OUTER.? ^^^ FullOuter + ) protected lazy val ordering: Parser[Seq[SortOrder]] = - rep1sep(singleOrder, ",") | - rep1sep(expression, ",") ~ opt(direction) ^^ { - case exps ~ None => exps.map(SortOrder(_, Ascending)) - case exps ~ Some(d) => exps.map(SortOrder(_, d)) - } + ( rep1sep(singleOrder, ",") + | rep1sep(expression, ",") ~ direction.? ^^ { + case exps ~ d => exps.map(SortOrder(_, d.getOrElse(Ascending))) + } + ) protected lazy val singleOrder: Parser[SortOrder] = - expression ~ direction ^^ { case e ~ o => SortOrder(e,o) } + expression ~ direction ^^ { case e ~ o => SortOrder(e, o) } protected lazy val direction: Parser[SortDirection] = - ASC ^^^ Ascending | - DESC ^^^ Descending - - protected lazy val grouping: Parser[Seq[Expression]] = - GROUP ~> BY ~> rep1sep(expression, ",") - - protected lazy val having: Parser[Expression] = - HAVING ~> expression - - protected lazy val limit: Parser[Expression] = - LIMIT ~> expression + ( ASC ^^^ Ascending + | DESC ^^^ Descending + ) - protected lazy val expression: Parser[Expression] = orExpression + protected lazy val expression: Parser[Expression] = + orExpression protected lazy val orExpression: Parser[Expression] = - andExpression * (OR ^^^ { (e1: Expression, e2: Expression) => Or(e1,e2) }) + andExpression * (OR ^^^ { (e1: Expression, e2: Expression) => Or(e1, e2) }) protected lazy val andExpression: Parser[Expression] = - comparisonExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) }) + comparisonExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1, e2) }) protected lazy val comparisonExpression: Parser[Expression] = - termExpression ~ "=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => EqualTo(e1, e2) } | - termExpression ~ "<" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThan(e1, e2) } | - termExpression ~ "<=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThanOrEqual(e1, e2) } | - termExpression ~ ">" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThan(e1, e2) } | - termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | - termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | - termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | - termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { - case e ~ _ ~ el ~ _ ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) - } | - termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | - termExpression ~ REGEXP ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | - termExpression ~ LIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => Like(e1, e2) } | - termExpression ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ { - case e1 ~ _ ~ _ ~ e2 => In(e1, e2) - } | - termExpression ~ NOT ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ { - case e1 ~ _ ~ _ ~ _ ~ e2 => Not(In(e1, e2)) - } | - termExpression <~ IS ~ NULL ^^ { case e => IsNull(e) } | - termExpression <~ IS ~ NOT ~ NULL ^^ { case e => IsNotNull(e) } | - NOT ~> termExpression ^^ {e => Not(e)} | - termExpression + ( termExpression ~ ("=" ~> termExpression) ^^ { case e1 ~ e2 => EqualTo(e1, e2) } + | termExpression ~ ("<" ~> termExpression) ^^ { case e1 ~ e2 => LessThan(e1, e2) } + | termExpression ~ ("<=" ~> termExpression) ^^ { case e1 ~ e2 => LessThanOrEqual(e1, e2) } + | termExpression ~ (">" ~> termExpression) ^^ { case e1 ~ e2 => GreaterThan(e1, e2) } + | termExpression ~ (">=" ~> termExpression) ^^ { case e1 ~ e2 => GreaterThanOrEqual(e1, e2) } + | termExpression ~ ("!=" ~> termExpression) ^^ { case e1 ~ e2 => Not(EqualTo(e1, e2)) } + | termExpression ~ ("<>" ~> termExpression) ^^ { case e1 ~ e2 => Not(EqualTo(e1, e2)) } + | termExpression ~ (BETWEEN ~> termExpression) ~ (AND ~> termExpression) ^^ { + case e ~ el ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + } + | termExpression ~ (RLIKE ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } + | termExpression ~ (REGEXP ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } + | termExpression ~ (LIKE ~> termExpression) ^^ { case e1 ~ e2 => Like(e1, e2) } + | termExpression ~ (IN ~ "(" ~> rep1sep(termExpression, ",")) <~ ")" ^^ { + case e1 ~ e2 => In(e1, e2) + } + | termExpression ~ (NOT ~ IN ~ "(" ~> rep1sep(termExpression, ",")) <~ ")" ^^ { + case e1 ~ e2 => Not(In(e1, e2)) + } + | termExpression <~ IS ~ NULL ^^ { case e => IsNull(e) } + | termExpression <~ IS ~ NOT ~ NULL ^^ { case e => IsNotNull(e) } + | NOT ~> termExpression ^^ {e => Not(e)} + | termExpression + ) protected lazy val termExpression: Parser[Expression] = - productExpression * ( - "+" ^^^ { (e1: Expression, e2: Expression) => Add(e1,e2) } | - "-" ^^^ { (e1: Expression, e2: Expression) => Subtract(e1,e2) } ) + productExpression * + ( "+" ^^^ { (e1: Expression, e2: Expression) => Add(e1, e2) } + | "-" ^^^ { (e1: Expression, e2: Expression) => Subtract(e1, e2) } + ) protected lazy val productExpression: Parser[Expression] = - baseExpression * ( - "*" ^^^ { (e1: Expression, e2: Expression) => Multiply(e1,e2) } | - "/" ^^^ { (e1: Expression, e2: Expression) => Divide(e1,e2) } | - "%" ^^^ { (e1: Expression, e2: Expression) => Remainder(e1,e2) } - ) + baseExpression * + ( "*" ^^^ { (e1: Expression, e2: Expression) => Multiply(e1, e2) } + | "/" ^^^ { (e1: Expression, e2: Expression) => Divide(e1, e2) } + | "%" ^^^ { (e1: Expression, e2: Expression) => Remainder(e1, e2) } + ) protected lazy val function: Parser[Expression] = - SUM ~> "(" ~> expression <~ ")" ^^ { case exp => Sum(exp) } | - SUM ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => SumDistinct(exp) } | - COUNT ~> "(" ~ "*" <~ ")" ^^ { case _ => Count(Literal(1)) } | - COUNT ~> "(" ~ expression <~ ")" ^^ { case dist ~ exp => Count(exp) } | - COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } | - APPROXIMATE ~> COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { - case exp => ApproxCountDistinct(exp) - } | - APPROXIMATE ~> "(" ~> floatLit ~ ")" ~ COUNT ~ "(" ~ DISTINCT ~ expression <~ ")" ^^ { - case s ~ _ ~ _ ~ _ ~ _ ~ e => ApproxCountDistinct(e, s.toDouble) - } | - FIRST ~> "(" ~> expression <~ ")" ^^ { case exp => First(exp) } | - LAST ~> "(" ~> expression <~ ")" ^^ { case exp => Last(exp) } | - AVG ~> "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } | - MIN ~> "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } | - MAX ~> "(" ~> expression <~ ")" ^^ { case exp => Max(exp) } | - UPPER ~> "(" ~> expression <~ ")" ^^ { case exp => Upper(exp) } | - LOWER ~> "(" ~> expression <~ ")" ^^ { case exp => Lower(exp) } | - IF ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { - case c ~ "," ~ t ~ "," ~ f => If(c,t,f) - } | - CASE ~> expression.? ~ (WHEN ~> expression ~ (THEN ~> expression)).* ~ - (ELSE ~> expression).? <~ END ^^ { - case casePart ~ altPart ~ elsePart => - val altExprs = altPart.flatMap { - case we ~ te => - Seq(casePart.fold(we)(EqualTo(_, we)), te) + ( SUM ~> "(" ~> expression <~ ")" ^^ { case exp => Sum(exp) } + | SUM ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => SumDistinct(exp) } + | COUNT ~ "(" ~> "*" <~ ")" ^^ { case _ => Count(Literal(1)) } + | COUNT ~ "(" ~> expression <~ ")" ^^ { case exp => Count(exp) } + | COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } + | APPROXIMATE ~ COUNT ~ "(" ~ DISTINCT ~> expression <~ ")" ^^ + { case exp => ApproxCountDistinct(exp) } + | APPROXIMATE ~> "(" ~> floatLit ~ ")" ~ COUNT ~ "(" ~ DISTINCT ~ expression <~ ")" ^^ + { case s ~ _ ~ _ ~ _ ~ _ ~ e => ApproxCountDistinct(e, s.toDouble) } + | FIRST ~ "(" ~> expression <~ ")" ^^ { case exp => First(exp) } + | LAST ~ "(" ~> expression <~ ")" ^^ { case exp => Last(exp) } + | AVG ~ "(" ~> expression <~ ")" ^^ { case exp => Average(exp) } + | MIN ~ "(" ~> expression <~ ")" ^^ { case exp => Min(exp) } + | MAX ~ "(" ~> expression <~ ")" ^^ { case exp => Max(exp) } + | UPPER ~ "(" ~> expression <~ ")" ^^ { case exp => Upper(exp) } + | LOWER ~ "(" ~> expression <~ ")" ^^ { case exp => Lower(exp) } + | IF ~ "(" ~> expression ~ ("," ~> expression) ~ ("," ~> expression) <~ ")" ^^ + { case c ~ t ~ f => If(c, t, f) } + | CASE ~> expression.? ~ (WHEN ~> expression ~ (THEN ~> expression)).* ~ + (ELSE ~> expression).? <~ END ^^ { + case casePart ~ altPart ~ elsePart => + val altExprs = altPart.flatMap { case whenExpr ~ thenExpr => + Seq(casePart.fold(whenExpr)(EqualTo(_, whenExpr)), thenExpr) + } + CaseWhen(altExprs ++ elsePart.toList) } - CaseWhen(altExprs ++ elsePart.toList) - } | - (SUBSTR | SUBSTRING) ~> "(" ~> expression ~ "," ~ expression <~ ")" ^^ { - case s ~ "," ~ p => Substring(s,p,Literal(Integer.MAX_VALUE)) - } | - (SUBSTR | SUBSTRING) ~> "(" ~> expression ~ "," ~ expression ~ "," ~ expression <~ ")" ^^ { - case s ~ "," ~ p ~ "," ~ l => Substring(s,p,l) - } | - SQRT ~> "(" ~> expression <~ ")" ^^ { case exp => Sqrt(exp) } | - ABS ~> "(" ~> expression <~ ")" ^^ { case exp => Abs(exp) } | - ident ~ "(" ~ repsep(expression, ",") <~ ")" ^^ { - case udfName ~ _ ~ exprs => UnresolvedFunction(udfName, exprs) - } + | (SUBSTR | SUBSTRING) ~ "(" ~> expression ~ ("," ~> expression) <~ ")" ^^ + { case s ~ p => Substring(s, p, Literal(Integer.MAX_VALUE)) } + | (SUBSTR | SUBSTRING) ~ "(" ~> expression ~ ("," ~> expression) ~ ("," ~> expression) <~ ")" ^^ + { case s ~ p ~ l => Substring(s, p, l) } + | SQRT ~ "(" ~> expression <~ ")" ^^ { case exp => Sqrt(exp) } + | ABS ~ "(" ~> expression <~ ")" ^^ { case exp => Abs(exp) } + | ident ~ ("(" ~> repsep(expression, ",")) <~ ")" ^^ + { case udfName ~ exprs => UnresolvedFunction(udfName, exprs) } + ) protected lazy val cast: Parser[Expression] = - CAST ~> "(" ~> expression ~ AS ~ dataType <~ ")" ^^ { case exp ~ _ ~ t => Cast(exp, t) } + CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { case exp ~ t => Cast(exp, t) } protected lazy val literal: Parser[Literal] = - numericLit ^^ { - case i if i.toLong > Int.MaxValue => Literal(i.toLong) - case i => Literal(i.toInt) - } | - NULL ^^^ Literal(null, NullType) | - floatLit ^^ {case f => Literal(f.toDouble) } | - stringLit ^^ {case s => Literal(s, StringType) } + ( numericLit ^^ { + case i if i.toLong > Int.MaxValue => Literal(i.toLong) + case i => Literal(i.toInt) + } + | NULL ^^^ Literal(null, NullType) + | floatLit ^^ {case f => Literal(f.toDouble) } + | stringLit ^^ {case s => Literal(s, StringType) } + ) protected lazy val floatLit: Parser[String] = elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) protected lazy val baseExpression: PackratParser[Expression] = - expression ~ "[" ~ expression <~ "]" ^^ { - case base ~ _ ~ ordinal => GetItem(base, ordinal) - } | - (expression <~ ".") ~ ident ^^ { - case base ~ fieldName => GetField(base, fieldName) - } | - TRUE ^^^ Literal(true, BooleanType) | - FALSE ^^^ Literal(false, BooleanType) | - cast | - "(" ~> expression <~ ")" | - function | - "-" ~> literal ^^ UnaryMinus | - dotExpressionHeader | - ident ^^ UnresolvedAttribute | - "*" ^^^ Star(None) | - literal + ( expression ~ ("[" ~> expression <~ "]") ^^ + { case base ~ ordinal => GetItem(base, ordinal) } + | (expression <~ ".") ~ ident ^^ + { case base ~ fieldName => GetField(base, fieldName) } + | TRUE ^^^ Literal(true, BooleanType) + | FALSE ^^^ Literal(false, BooleanType) + | cast + | "(" ~> expression <~ ")" + | function + | "-" ~> literal ^^ UnaryMinus + | dotExpressionHeader + | ident ^^ UnresolvedAttribute + | "*" ^^^ Star(None) + | literal + ) protected lazy val dotExpressionHeader: Parser[Expression] = (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { @@ -400,55 +338,3 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected lazy val dataType: Parser[DataType] = STRING ^^^ StringType | TIMESTAMP ^^^ TimestampType } - -class SqlLexical(val keywords: Seq[String]) extends StdLexical { - case class FloatLit(chars: String) extends Token { - override def toString = chars - } - - reserved ++= keywords.flatMap(w => allCaseVersions(w)) - - delimiters += ( - "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":", "[", "]", "." - ) - - override lazy val token: Parser[Token] = ( - identChar ~ rep( identChar | digit ) ^^ - { case first ~ rest => processIdent(first :: rest mkString "") } - | rep1(digit) ~ opt('.' ~> rep(digit)) ^^ { - case i ~ None => NumericLit(i mkString "") - case i ~ Some(d) => FloatLit(i.mkString("") + "." + d.mkString("")) - } - | '\'' ~ rep( chrExcept('\'', '\n', EofCh) ) ~ '\'' ^^ - { case '\'' ~ chars ~ '\'' => StringLit(chars mkString "") } - | '\"' ~ rep( chrExcept('\"', '\n', EofCh) ) ~ '\"' ^^ - { case '\"' ~ chars ~ '\"' => StringLit(chars mkString "") } - | EofCh ^^^ EOF - | '\'' ~> failure("unclosed string literal") - | '\"' ~> failure("unclosed string literal") - | delim - | failure("illegal character") - ) - - override def identChar = letter | elem('_') - - override def whitespace: Parser[Any] = rep( - whitespaceChar - | '/' ~ '*' ~ comment - | '/' ~ '/' ~ rep( chrExcept(EofCh, '\n') ) - | '#' ~ rep( chrExcept(EofCh, '\n') ) - | '-' ~ '-' ~ rep( chrExcept(EofCh, '\n') ) - | '/' ~ '*' ~ failure("unclosed comment") - ) - - /** Generate all variations of upper and lower case of a given string */ - def allCaseVersions(s: String, prefix: String = ""): Stream[String] = { - if (s == "") { - Stream(prefix) - } else { - allCaseVersions(s.tail, prefix + s.head.toLower) ++ - allCaseVersions(s.tail, prefix + s.head.toUpper) - } - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 9a3848cfc6b62..b8ba2ee428a20 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -39,9 +39,9 @@ case class NativeCommand(cmd: String) extends Command { } /** - * Commands of the form "SET (key) (= value)". + * Commands of the form "SET [key [= value] ]". */ -case class SetCommand(key: Option[String], value: Option[String]) extends Command { +case class SetCommand(kv: Option[(String, Option[String])]) extends Command { override def output = Seq( AttributeReference("", StringType, nullable = false)()) } @@ -81,3 +81,14 @@ case class DescribeCommand( AttributeReference("data_type", StringType, nullable = false)(), AttributeReference("comment", StringType, nullable = false)()) } + +/** + * Returned for the "! shellCommand" command + */ +case class ShellCommand(cmd: String) extends Command + + +/** + * Returned for the "SOURCE file" command + */ +case class SourceCommand(filePath: String) extends Command diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 014e1e2826724..23e7b2d270777 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -66,12 +66,17 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] lazy val analyzer: Analyzer = new Analyzer(catalog, functionRegistry, caseSensitive = true) + @transient protected[sql] val optimizer = Optimizer + @transient - protected[sql] val parser = new catalyst.SqlParser + protected[sql] val sqlParser = { + val fallback = new catalyst.SqlParser + new catalyst.SparkSQLParser(fallback(_)) + } - protected[sql] def parseSql(sql: String): LogicalPlan = parser(sql) + protected[sql] def parseSql(sql: String): LogicalPlan = sqlParser(sql) protected[sql] def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index bbf17b9fadf86..4f1af7234d551 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -304,8 +304,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case class CommandStrategy(context: SQLContext) extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.SetCommand(key, value) => - Seq(execution.SetCommand(key, value, plan.output)(context)) + case logical.SetCommand(kv) => + Seq(execution.SetCommand(kv, plan.output)(context)) case logical.ExplainCommand(logicalPlan, extended) => Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) case logical.CacheTableCommand(tableName, optPlan, isLazy) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index d49633c24ad4d..5859eba408ee1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -48,29 +48,28 @@ trait Command { * :: DeveloperApi :: */ @DeveloperApi -case class SetCommand( - key: Option[String], value: Option[String], output: Seq[Attribute])( +case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribute])( @transient context: SQLContext) extends LeafNode with Command with Logging { - override protected lazy val sideEffectResult: Seq[Row] = (key, value) match { - // Set value for key k. - case (Some(k), Some(v)) => - if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + override protected lazy val sideEffectResult: Seq[Row] = kv match { + // Set value for the key. + case Some((key, Some(value))) => + if (key == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - context.setConf(SQLConf.SHUFFLE_PARTITIONS, v) - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$v")) + context.setConf(SQLConf.SHUFFLE_PARTITIONS, value) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) } else { - context.setConf(k, v) - Seq(Row(s"$k=$v")) + context.setConf(key, value) + Seq(Row(s"$key=$value")) } - // Query the value bound to key k. - case (Some(k), _) => + // Query the value bound to the key. + case Some((key, None)) => // TODO (lian) This is just a workaround to make the Simba ODBC driver work. // Should remove this once we get the ODBC driver updated. - if (k == "-v") { + if (key == "-v") { val hiveJars = Seq( "hive-exec-0.12.0.jar", "hive-service-0.12.0.jar", @@ -84,23 +83,20 @@ case class SetCommand( Row("system:java.class.path=" + hiveJars), Row("system:sun.java.command=shark.SharkServer2")) } else { - if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { + if (key == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${context.numShufflePartitions}")) } else { - Seq(Row(s"$k=${context.getConf(k, "")}")) + Seq(Row(s"$key=${context.getConf(key, "")}")) } } // Query all key-value pairs that are set in the SQLConf of the context. - case (None, None) => + case _ => context.getAllConfs.map { case (k, v) => Row(s"$k=$v") }.toSeq - - case _ => - throw new IllegalArgumentException() } override def otherCopyArgs = context :: Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 1e624f97004f5..c87ded81fdc27 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -69,7 +69,7 @@ class CachedTableSuite extends QueryTest { test("calling .unpersist() should drop in-memory columnar cache") { table("testData").cache() table("testData").count() - table("testData").unpersist(true) + table("testData").unpersist(blocking = true) assertCached(table("testData"), 0) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 79de1bb855dbe..a94022c0cf6e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -42,7 +42,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { TimeZone.setDefault(origZone) } - test("SPARK-3176 Added Parser of SQL ABS()") { checkAnswer( sql("SELECT ABS(-1.3)"), @@ -61,7 +60,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { 4) } - test("SPARK-2041 column name equals tablename") { checkAnswer( sql("SELECT tableName FROM tableName"), @@ -694,6 +692,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("SPARK-3813 CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END") { checkAnswer( - sql("SELECT CASE WHEN key=1 THEN 1 ELSE 2 END FROM testData WHERE key = 1 group by key"), 1) + sql("SELECT CASE WHEN key = 1 THEN 1 ELSE 2 END FROM testData WHERE key = 1 group by key"), 1) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 910174a153768..accf61576b804 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -172,7 +172,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) result = hiveContext.sql(statement) logDebug(result.queryExecution.toString()) result.queryExecution.logical match { - case SetCommand(Some(key), Some(value)) if (key == SQLConf.THRIFTSERVER_POOL) => + case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => sessionToActivePool(parentSession) = value logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") case _ => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala index c5844e92eaaa9..430ffb29989ea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala @@ -18,118 +18,50 @@ package org.apache.spark.sql.hive import scala.language.implicitConversions -import scala.util.parsing.combinator.syntactical.StandardTokenParsers -import scala.util.parsing.combinator.PackratParsers + import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.SqlLexical +import org.apache.spark.sql.catalyst.{AbstractSparkSQLParser, SqlLexical} /** - * A parser that recognizes all HiveQL constructs together with several Spark SQL specific - * extensions like CACHE TABLE and UNCACHE TABLE. + * A parser that recognizes all HiveQL constructs together with Spark SQL specific extensions. */ -private[hive] class ExtendedHiveQlParser extends StandardTokenParsers with PackratParsers { - - def apply(input: String): LogicalPlan = { - // Special-case out set commands since the value fields can be - // complex to handle without RegexParsers. Also this approach - // is clearer for the several possible cases of set commands. - if (input.trim.toLowerCase.startsWith("set")) { - input.trim.drop(3).split("=", 2).map(_.trim) match { - case Array("") => // "set" - SetCommand(None, None) - case Array(key) => // "set key" - SetCommand(Some(key), None) - case Array(key, value) => // "set key=value" - SetCommand(Some(key), Some(value)) - } - } else if (input.trim.startsWith("!")) { - ShellCommand(input.drop(1)) - } else { - phrase(query)(new lexical.Scanner(input)) match { - case Success(r, x) => r - case x => sys.error(x.toString) - } - } - } - - protected case class Keyword(str: String) - - protected val ADD = Keyword("ADD") - protected val AS = Keyword("AS") - protected val CACHE = Keyword("CACHE") - protected val DFS = Keyword("DFS") - protected val FILE = Keyword("FILE") - protected val JAR = Keyword("JAR") - protected val LAZY = Keyword("LAZY") - protected val SET = Keyword("SET") - protected val SOURCE = Keyword("SOURCE") - protected val TABLE = Keyword("TABLE") - protected val UNCACHE = Keyword("UNCACHE") - +private[hive] class ExtendedHiveQlParser extends AbstractSparkSQLParser { protected implicit def asParser(k: Keyword): Parser[String] = lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) - protected def allCaseConverse(k: String): Parser[String] = - lexical.allCaseVersions(k).map(x => x : Parser[String]).reduce(_ | _) + protected val ADD = Keyword("ADD") + protected val DFS = Keyword("DFS") + protected val FILE = Keyword("FILE") + protected val JAR = Keyword("JAR") - protected val reservedWords = - this.getClass + private val reservedWords = + this + .getClass .getMethods .filter(_.getReturnType == classOf[Keyword]) .map(_.invoke(this).asInstanceOf[Keyword].str) override val lexical = new SqlLexical(reservedWords) - protected lazy val query: Parser[LogicalPlan] = - cache | uncache | addJar | addFile | dfs | source | hiveQl + protected lazy val start: Parser[LogicalPlan] = dfs | addJar | addFile | hiveQl protected lazy val hiveQl: Parser[LogicalPlan] = restInput ^^ { - case statement => HiveQl.createPlan(statement.trim()) - } - - // Returns the whole input string - protected lazy val wholeInput: Parser[String] = new Parser[String] { - def apply(in: Input) = - Success(in.source.toString, in.drop(in.source.length())) - } - - // Returns the rest of the input string that are not parsed yet - protected lazy val restInput: Parser[String] = new Parser[String] { - def apply(in: Input) = - Success( - in.source.subSequence(in.offset, in.source.length).toString, - in.drop(in.source.length())) - } - - protected lazy val cache: Parser[LogicalPlan] = - CACHE ~> opt(LAZY) ~ (TABLE ~> ident) ~ opt(AS ~> hiveQl) ^^ { - case isLazy ~ tableName ~ plan => - CacheTableCommand(tableName, plan, isLazy.isDefined) - } - - protected lazy val uncache: Parser[LogicalPlan] = - UNCACHE ~ TABLE ~> ident ^^ { - case tableName => UncacheTableCommand(tableName) + case statement => HiveQl.createPlan(statement.trim) } - protected lazy val addJar: Parser[LogicalPlan] = - ADD ~ JAR ~> restInput ^^ { - case jar => AddJar(jar.trim()) + protected lazy val dfs: Parser[LogicalPlan] = + DFS ~> wholeInput ^^ { + case command => NativeCommand(command.trim) } - protected lazy val addFile: Parser[LogicalPlan] = + private lazy val addFile: Parser[LogicalPlan] = ADD ~ FILE ~> restInput ^^ { - case file => AddFile(file.trim()) + case input => AddFile(input.trim) } - protected lazy val dfs: Parser[LogicalPlan] = - DFS ~> wholeInput ^^ { - case command => NativeCommand(command.trim()) - } - - protected lazy val source: Parser[LogicalPlan] = - SOURCE ~> restInput ^^ { - case file => SourceCommand(file.trim()) + private lazy val addJar: Parser[LogicalPlan] = + ADD ~ JAR ~> restInput ^^ { + case input => AddJar(input.trim) } } 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 32c9175f181bb..98a46a31e1ffd 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 @@ -21,6 +21,7 @@ import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils +import org.apache.spark.sql.catalyst.SparkSQLParser import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ @@ -38,10 +39,6 @@ import scala.collection.JavaConversions._ */ private[hive] case object NativePlaceholder extends Command -private[hive] case class ShellCommand(cmd: String) extends Command - -private[hive] case class SourceCommand(filePath: String) extends Command - private[hive] case class AddFile(filePath: String) extends Command private[hive] case class AddJar(path: String) extends Command @@ -126,9 +123,11 @@ private[hive] object HiveQl { "TOK_CREATETABLE", "TOK_DESCTABLE" ) ++ nativeCommands - - // It parses hive sql query along with with several Spark SQL specific extensions - protected val hiveSqlParser = new ExtendedHiveQlParser + + protected val hqlParser = { + val fallback = new ExtendedHiveQlParser + new SparkSQLParser(fallback(_)) + } /** * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations @@ -218,7 +217,7 @@ private[hive] object HiveQl { def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql)) /** Returns a LogicalPlan for a given HiveQL string. */ - def parseSql(sql: String): LogicalPlan = hiveSqlParser(sql) + def parseSql(sql: String): LogicalPlan = hqlParser(sql) /** Creates LogicalPlan for a given HiveQL string. */ def createPlan(sql: String) = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 508d8239c7628..5c66322f1ed99 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -167,10 +167,10 @@ private[hive] trait HiveStrategies { database.get, tableName, query, - InsertIntoHiveTable(_: MetastoreRelation, - Map(), - query, - true)(hiveContext)) :: Nil + InsertIntoHiveTable(_: MetastoreRelation, + Map(), + query, + overwrite = true)(hiveContext)) :: Nil case _ => Nil } } From 421382d0e728940caa3e61bc11237c61f256378a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 9 Oct 2014 18:26:43 -0700 Subject: [PATCH 0958/1492] [SPARK-3824][SQL] Sets in-memory table default storage level to MEMORY_AND_DISK Using `MEMORY_AND_DISK` as default storage level for in-memory table caching. Due to the in-memory columnar representation, recomputing an in-memory cached table partitions can be very expensive. Author: Cheng Lian Closes #2686 from liancheng/spark-3824 and squashes the following commits: 35d2ed0 [Cheng Lian] Removes extra space 1ab7967 [Cheng Lian] Reduces test data size to fit DiskStore.getBytes() ba565f0 [Cheng Lian] Maks CachedBatch serializable 07f0204 [Cheng Lian] Sets in-memory table default storage level to MEMORY_AND_DISK --- .../main/scala/org/apache/spark/sql/CacheManager.scala | 10 +++++++--- .../spark/sql/columnar/InMemoryColumnarTableScan.scala | 9 +++++---- .../scala/org/apache/spark/sql/CachedTableSuite.scala | 10 +++++----- 3 files changed, 17 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index 3bf7382ac67a6..5ab2b5316ab10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -22,7 +22,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.storage.StorageLevel -import org.apache.spark.storage.StorageLevel.MEMORY_ONLY +import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK /** Holds a cached logical plan and its data */ private case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) @@ -74,10 +74,14 @@ private[sql] trait CacheManager { cachedData.clear() } - /** Caches the data produced by the logical representation of the given schema rdd. */ + /** + * Caches the data produced by the logical representation of the given schema rdd. Unlike + * `RDD.cache()`, the default storage level is set to be `MEMORY_AND_DISK` because recomputing + * the in-memory columnar representation of the underlying table is expensive. + */ private[sql] def cacheQuery( query: SchemaRDD, - storageLevel: StorageLevel = MEMORY_ONLY): Unit = writeLock { + storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock { val planToCache = query.queryExecution.optimizedPlan if (lookupCachedData(planToCache).nonEmpty) { logWarning("Asked to cache already cached data.") 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 4f79173a26f88..22ab0e2613f21 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 @@ -38,7 +38,7 @@ private[sql] object InMemoryRelation { new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child)() } -private[sql] case class CachedBatch(buffers: Array[ByteBuffer], stats: Row) +private[sql] case class CachedBatch(buffers: Array[Array[Byte]], stats: Row) private[sql] case class InMemoryRelation( output: Seq[Attribute], @@ -91,7 +91,7 @@ private[sql] case class InMemoryRelation( val stats = Row.fromSeq( columnBuilders.map(_.columnStats.collectedStatistics).foldLeft(Seq.empty[Any])(_ ++ _)) - CachedBatch(columnBuilders.map(_.build()), stats) + CachedBatch(columnBuilders.map(_.build().array()), stats) } def hasNext = rowIterator.hasNext @@ -238,8 +238,9 @@ private[sql] case class InMemoryColumnarTableScan( def cachedBatchesToRows(cacheBatches: Iterator[CachedBatch]) = { val rows = cacheBatches.flatMap { cachedBatch => // Build column accessors - val columnAccessors = - requestedColumnIndices.map(cachedBatch.buffers(_)).map(ColumnAccessor(_)) + val columnAccessors = requestedColumnIndices.map { batch => + ColumnAccessor(ByteBuffer.wrap(cachedBatch.buffers(batch))) + } // Extract rows via column accessors new Iterator[Row] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index c87ded81fdc27..444bc95009c31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.test.TestSQLContext._ -import org.apache.spark.storage.RDDBlockId +import org.apache.spark.storage.{StorageLevel, RDDBlockId} case class BigData(s: String) @@ -55,10 +55,10 @@ class CachedTableSuite extends QueryTest { test("too big for memory") { val data = "*" * 10000 - sparkContext.parallelize(1 to 1000000, 1).map(_ => BigData(data)).registerTempTable("bigData") - cacheTable("bigData") - assert(table("bigData").count() === 1000000L) - uncacheTable("bigData") + sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).registerTempTable("bigData") + table("bigData").persist(StorageLevel.MEMORY_AND_DISK) + assert(table("bigData").count() === 200000L) + table("bigData").unpersist() } test("calling .cache() should use in-memory columnar caching") { From 6f98902a3d7749e543bc493a8c62b1e3a7b924cc Mon Sep 17 00:00:00 2001 From: ravipesala Date: Thu, 9 Oct 2014 18:41:36 -0700 Subject: [PATCH 0959/1492] [SPARK-3834][SQL] Backticks not correctly handled in subquery aliases The queries like SELECT a.key FROM (SELECT key FROM src) \`a\` does not work as backticks in subquery aliases are not handled properly. This PR fixes that. Author : ravipesala ravindra.pesalahuawei.com Author: ravipesala Closes #2737 from ravipesala/SPARK-3834 and squashes the following commits: 0e0ab98 [ravipesala] Fixing issue in backtick handling for subquery aliases --- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 8 +++++++- 2 files changed, 8 insertions(+), 2 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 98a46a31e1ffd..7cc14dc7a9c9e 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 @@ -638,7 +638,7 @@ private[hive] object HiveQl { def nodeToRelation(node: Node): LogicalPlan = node match { case Token("TOK_SUBQUERY", query :: Token(alias, Nil) :: Nil) => - Subquery(alias, nodeToPlan(query)) + Subquery(cleanIdentifier(alias), nodeToPlan(query)) case Token(laterViewToken(isOuter), selectClause :: relationClause :: Nil) => val Token("TOK_SELECT", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 3647bb1c4ce7d..fbe6ac765c009 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -68,5 +68,11 @@ class SQLQuerySuite extends QueryTest { checkAnswer( sql("SELECT k FROM (SELECT `key` AS `k` FROM src) a"), sql("SELECT `key` FROM src").collect().toSeq) - } + } + + test("SPARK-3834 Backticks not correctly handled in subquery aliases") { + checkAnswer( + sql("SELECT a.key FROM (SELECT key FROM src) `a`"), + sql("SELECT `key` FROM src").collect().toSeq) + } } From 411cf29fff011561f0093bb6101af87842828369 Mon Sep 17 00:00:00 2001 From: Anand Avati Date: Fri, 10 Oct 2014 00:46:56 -0700 Subject: [PATCH 0960/1492] [SPARK-2805] Upgrade Akka to 2.3.4 This is a second rev of the Akka upgrade (earlier merged, but reverted). I made a slight modification which is that I also upgrade Hive to deal with a compatibility issue related to the protocol buffers library. Author: Anand Avati Author: Patrick Wendell Closes #2752 from pwendell/akka-upgrade and squashes the following commits: 4c7ca3f [Patrick Wendell] Upgrading to new hive->protobuf version 57a2315 [Anand Avati] SPARK-1812: streaming - remove tests which depend on akka.actor.IO 2a551d3 [Anand Avati] SPARK-1812: core - upgrade to akka 2.3.4 --- .../org/apache/spark/deploy/Client.scala | 2 +- .../spark/deploy/client/AppClient.scala | 2 +- .../spark/deploy/worker/WorkerWatcher.scala | 2 +- .../apache/spark/MapOutputTrackerSuite.scala | 4 +- pom.xml | 4 +- .../spark/streaming/InputStreamsSuite.scala | 71 ------------------- 6 files changed, 7 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 065ddda50e65e..f2687ce6b42b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -130,7 +130,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") System.exit(-1) - case AssociationErrorEvent(cause, _, remoteAddress, _) => + case AssociationErrorEvent(cause, _, remoteAddress, _, _) => println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") println(s"Cause was: $cause") System.exit(-1) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 32790053a6be8..98a93d1fcb2a3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -154,7 +154,7 @@ private[spark] class AppClient( logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() - case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) => + case AssociationErrorEvent(cause, _, address, _, _) if isPossibleMaster(address) => logWarning(s"Could not connect to $address: $cause") case StopAppClient => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 6d0d0bbe5ecec..63a8ac817b618 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -54,7 +54,7 @@ private[spark] class WorkerWatcher(workerUrl: String) case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => logInfo(s"Successfully connected to $workerUrl") - case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound) + case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) if isWorker(remoteAddress) => // These logs may not be seen if the worker (and associated pipe) has died logError(s"Could not initialize connection to worker $workerUrl. Exiting.") diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 1fef79ad1001f..cbc0bd178d894 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -146,7 +146,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) val actorSystem = ActorSystem("test") val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) val masterActor = actorRef.underlyingActor // Frame size should be ~123B, and no exception should be thrown @@ -164,7 +164,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) val actorSystem = ActorSystem("test") val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) val masterActor = actorRef.underlyingActor // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. diff --git a/pom.xml b/pom.xml index 7756c89b00cad..d047b9e307d4b 100644 --- a/pom.xml +++ b/pom.xml @@ -118,7 +118,7 @@ 0.18.1 shaded-protobuf org.spark-project.akka - 2.2.3-shaded-protobuf + 2.3.4-spark 1.7.5 1.2.17 1.0.4 @@ -127,7 +127,7 @@ 0.94.6 1.4.0 3.4.5 - 0.12.0 + 0.12.0-protobuf 1.4.3 1.2.3 8.1.14.v20131031 diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index a44a45a3e9bd6..fa04fa326e370 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -18,8 +18,6 @@ package org.apache.spark.streaming import akka.actor.Actor -import akka.actor.IO -import akka.actor.IOManager import akka.actor.Props import akka.util.ByteString @@ -143,59 +141,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - // TODO: This test works in IntelliJ but not through SBT - ignore("actor input stream") { - // Start the server - val testServer = new TestServer() - val port = testServer.port - testServer.start() - - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", - // Had to pass the local value of port to prevent from closing over entire scope - StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(networkStream, outputBuffer) - def output = outputBuffer.flatMap(x => x) - outputStream.register() - ssc.start() - - // Feed data to the server to send to the network receiver - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val input = 1 to 9 - val expectedOutput = input.map(x => x.toString) - Thread.sleep(1000) - for (i <- 0 until input.size) { - testServer.send(input(i).toString) - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - } - Thread.sleep(1000) - logInfo("Stopping server") - testServer.stop() - logInfo("Stopping context") - ssc.stop() - - // Verify whether data received was as expected - logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) - logInfo("output") - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output.size = " + expectedOutput.size) - logInfo("expected output") - expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - // (whether the elements were received one in each interval is not verified) - assert(output.size === expectedOutput.size) - for (i <- 0 until output.size) { - assert(output(i) === expectedOutput(i)) - } - } - - test("multi-thread receiver") { // set up the test receiver val numThreads = 10 @@ -377,22 +322,6 @@ class TestServer(portToBind: Int = 0) extends Logging { def port = serverSocket.getLocalPort } -/** This is an actor for testing actor input stream */ -class TestActor(port: Int) extends Actor with ActorHelper { - - def bytesToString(byteString: ByteString) = byteString.utf8String - - override def preStart(): Unit = { - @deprecated("suppress compile time deprecation warning", "1.0.0") - val unit = IOManager(context.system).connect(new InetSocketAddress(port)) - } - - def receive = { - case IO.Read(socket, bytes) => - store(bytesToString(bytes)) - } -} - /** This is a receiver to test multiple threads inserting data using block generator */ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) extends Receiver[Int](StorageLevel.MEMORY_ONLY_SER) with Logging { From 90f73fcc47c7bf881f808653d46a9936f37c3c31 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 10 Oct 2014 01:44:36 -0700 Subject: [PATCH 0961/1492] [SPARK-3889] Attempt to avoid SIGBUS by not mmapping files in ConnectionManager In general, individual shuffle blocks are frequently small, so mmapping them often creates a lot of waste. It may not be bad to mmap the larger ones, but it is pretty inconvenient to get configuration into ManagedBuffer, and besides it is unlikely to help all that much. Author: Aaron Davidson Closes #2742 from aarondav/mmap and squashes the following commits: a152065 [Aaron Davidson] Add other pathway back 52b6cd2 [Aaron Davidson] [SPARK-3889] Attempt to avoid SIGBUS by not mmapping files in ConnectionManager --- .../org/apache/spark/network/ManagedBuffer.scala | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala index a4409181ec907..4c9ca97a2a6b7 100644 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -66,13 +66,27 @@ sealed abstract class ManagedBuffer { final class FileSegmentManagedBuffer(val file: File, val offset: Long, val length: Long) extends ManagedBuffer { + /** + * Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889). + * Avoid unless there's a good reason not to. + */ + private val MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024; + override def size: Long = length override def nioByteBuffer(): ByteBuffer = { var channel: FileChannel = null try { channel = new RandomAccessFile(file, "r").getChannel - channel.map(MapMode.READ_ONLY, offset, length) + // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. + if (length < MIN_MEMORY_MAP_BYTES) { + val buf = ByteBuffer.allocate(length.toInt) + channel.read(buf, offset) + buf.flip() + buf + } else { + channel.map(MapMode.READ_ONLY, offset, length) + } } catch { case e: IOException => Try(channel.size).toOption match { From 72f36ee571ad27c7c7c70bb9aecc7e6ef51dfd44 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 10 Oct 2014 14:14:05 -0700 Subject: [PATCH 0962/1492] [SPARK-3886] [PySpark] use AutoBatchedSerializer by default Use AutoBatchedSerializer by default, which will choose the proper batch size based on size of serialized objects, let the size of serialized batch fall in into [64k - 640k]. In JVM, the serializer will also track the objects in batch to figure out duplicated objects, larger batch may cause OOM in JVM. Author: Davies Liu Closes #2740 from davies/batchsize and squashes the following commits: 52cdb88 [Davies Liu] update docs 185f2b9 [Davies Liu] use AutoBatchedSerializer by default --- python/pyspark/context.py | 11 +++++++---- python/pyspark/serializers.py | 4 ++-- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6fb30d65c5edd..85c04624da4a6 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer, CompressedSerializer + PairDeserializer, CompressedSerializer, AutoBatchedSerializer from pyspark.storagelevel import StorageLevel from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -67,7 +67,7 @@ class SparkContext(object): _default_batch_size_for_serialized_input = 10 def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, - environment=None, batchSize=1024, serializer=PickleSerializer(), conf=None, + environment=None, batchSize=0, serializer=PickleSerializer(), conf=None, gateway=None): """ Create a new SparkContext. At least the master and app name should be set, @@ -83,8 +83,9 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, :param environment: A dictionary of environment variables to set on worker nodes. :param batchSize: The number of Python objects represented as a single - Java object. Set 1 to disable batching or -1 to use an - unlimited batch size. + Java object. Set 1 to disable batching, 0 to automatically choose + the batch size based on object sizes, or -1 to use an unlimited + batch size :param serializer: The serializer for RDDs. :param conf: A L{SparkConf} object setting Spark properties. :param gateway: Use an existing gateway and JVM, otherwise a new JVM @@ -117,6 +118,8 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self._unbatched_serializer = serializer if batchSize == 1: self.serializer = self._unbatched_serializer + elif batchSize == 0: + self.serializer = AutoBatchedSerializer(self._unbatched_serializer) else: self.serializer = BatchedSerializer(self._unbatched_serializer, batchSize) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 099fa54cf2bd7..3d1a34b281acc 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -220,7 +220,7 @@ class AutoBatchedSerializer(BatchedSerializer): Choose the size of batch automatically based on the size of object """ - def __init__(self, serializer, bestSize=1 << 20): + def __init__(self, serializer, bestSize=1 << 16): BatchedSerializer.__init__(self, serializer, -1) self.bestSize = bestSize @@ -247,7 +247,7 @@ def __eq__(self, other): other.serializer == self.serializer) def __str__(self): - return "BatchedSerializer<%s>" % str(self.serializer) + return "AutoBatchedSerializer<%s>" % str(self.serializer) class CartesianDeserializer(FramedSerializer): From 1d72a30874a88bdbab75217f001cf2af409016e7 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 10 Oct 2014 16:49:19 -0700 Subject: [PATCH 0963/1492] HOTFIX: Fix build issue with Akka 2.3.4 upgrade. We had to upgrade our Hive 0.12 version as well to deal with a protobuf conflict (both hive and akka have been using a shaded protobuf version). This is testing a correctly patched version of Hive 0.12. Author: Patrick Wendell Closes #2756 from pwendell/hotfix and squashes the following commits: cc979d0 [Patrick Wendell] HOTFIX: Fix build issue with Akka 2.3.4 upgrade. --- pom.xml | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d047b9e307d4b..288bbf1114bea 100644 --- a/pom.xml +++ b/pom.xml @@ -127,7 +127,7 @@ 0.94.6 1.4.0 3.4.5 - 0.12.0-protobuf + 0.12.0-protobuf-2.5 1.4.3 1.2.3 8.1.14.v20131031 @@ -223,6 +223,18 @@ false + + + spark-staging + Spring Staging Repository + https://oss.sonatype.org/content/repositories/orgspark-project-1085 + + true + + + false + + From 0e8203f4fb721158fb27897680da476174d24c4b Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 10 Oct 2014 18:39:55 -0700 Subject: [PATCH 0964/1492] [SPARK-2924] Required by scala 2.11, only one fun/ctor amongst overriden alternatives, can have default argument(s). ...riden alternatives, can have default argument. Author: Prashant Sharma Closes #2750 from ScrapCodes/SPARK-2924/default-args-removed and squashes the following commits: d9785c3 [Prashant Sharma] [SPARK-2924] Required by scala 2.11, only one function/ctor amongst overriden alternatives, can have default argument. --- .../org/apache/spark/util/FileLogger.scala | 19 +++++++++++++++++-- .../apache/spark/util/FileLoggerSuite.scala | 8 ++++---- 2 files changed, 21 insertions(+), 6 deletions(-) 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 6d1fc05a15d2c..fdc73f08261a6 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -51,12 +51,27 @@ private[spark] class FileLogger( def this( logDir: String, sparkConf: SparkConf, - compress: Boolean = false, - overwrite: Boolean = true) = { + compress: Boolean, + overwrite: Boolean) = { this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = compress, overwrite = overwrite) } + def this( + logDir: String, + sparkConf: SparkConf, + compress: Boolean) = { + this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = compress, + overwrite = true) + } + + def this( + logDir: String, + sparkConf: SparkConf) = { + this(logDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf), compress = false, + overwrite = true) + } + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } diff --git a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala index dc2a05631d83d..72466a3aa1130 100644 --- a/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileLoggerSuite.scala @@ -74,13 +74,13 @@ class FileLoggerSuite extends FunSuite with BeforeAndAfter { test("Logging when directory already exists") { // Create the logging directory multiple times - new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() - new FileLogger(logDirPathString, new SparkConf, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() + new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = true).start() // If overwrite is not enabled, an exception should be thrown intercept[IOException] { - new FileLogger(logDirPathString, new SparkConf, overwrite = false).start() + new FileLogger(logDirPathString, new SparkConf, compress = false, overwrite = false).start() } } From 81015a2ba49583d730ce65b2262f50f1f2451a79 Mon Sep 17 00:00:00 2001 From: cocoatomo Date: Sat, 11 Oct 2014 11:26:17 -0700 Subject: [PATCH 0965/1492] [SPARK-3867][PySpark] ./python/run-tests failed when it run with Python 2.6 and unittest2 is not installed ./python/run-tests search a Python 2.6 executable on PATH and use it if available. When using Python 2.6, it is going to import unittest2 module which is not a standard library in Python 2.6, so it fails with ImportError. Author: cocoatomo Closes #2759 from cocoatomo/issues/3867-unittest2-import-error and squashes the following commits: f068eb5 [cocoatomo] [SPARK-3867] ./python/run-tests failed when it run with Python 2.6 and unittest2 is not installed --- python/pyspark/mllib/tests.py | 6 +++++- python/pyspark/tests.py | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 5c20e100e144f..463faf7b6f520 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -25,7 +25,11 @@ from numpy import array, array_equal if sys.version_info[:2] <= (2, 6): - import unittest2 as unittest + try: + import unittest2 as unittest + except ImportError: + sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier') + sys.exit(1) else: import unittest diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 7f05d48ade2b3..ceab57464f013 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -34,7 +34,11 @@ from platform import python_implementation if sys.version_info[:2] <= (2, 6): - import unittest2 as unittest + try: + import unittest2 as unittest + except ImportError: + sys.stderr.write('Please install unittest2 to test with Python 2.6 or earlier') + sys.exit(1) else: import unittest From 7a3f589ef86200f99624fea8322e5af0cad774a7 Mon Sep 17 00:00:00 2001 From: cocoatomo Date: Sat, 11 Oct 2014 11:51:59 -0700 Subject: [PATCH 0966/1492] [SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and building warnings Sphinx documents contains a corrupted ReST format and have some warnings. The purpose of this issue is same as https://issues.apache.org/jira/browse/SPARK-3773. commit: 0e8203f4fb721158fb27897680da476174d24c4b output ``` $ cd ./python/docs $ make clean html rm -rf _build/* sphinx-build -b html -d _build/doctrees . _build/html Making output directory... Running Sphinx v1.2.3 loading pickled environment... not yet created building [html]: targets for 4 source files that are out of date updating environment: 4 added, 0 changed, 0 removed reading sources... [100%] pyspark.sql /Users//MyRepos/Scala/spark/python/pyspark/mllib/feature.py:docstring of pyspark.mllib.feature.Word2VecModel.findSynonyms:4: WARNING: Field list ends without a blank line; unexpected unindent. /Users//MyRepos/Scala/spark/python/pyspark/mllib/feature.py:docstring of pyspark.mllib.feature.Word2VecModel.transform:3: WARNING: Field list ends without a blank line; unexpected unindent. /Users//MyRepos/Scala/spark/python/pyspark/sql.py:docstring of pyspark.sql:4: WARNING: Bullet list ends without a blank line; unexpected unindent. looking for now-outdated files... none found pickling environment... done checking consistency... done preparing documents... done writing output... [100%] pyspark.sql writing additional files... (12 module code pages) _modules/index search copying static files... WARNING: html_static_path entry u'/Users//MyRepos/Scala/spark/python/docs/_static' does not exist done copying extra files... done dumping search index... done dumping object inventory... done build succeeded, 4 warnings. Build finished. The HTML pages are in _build/html. ``` Author: cocoatomo Closes #2766 from cocoatomo/issues/3909-sphinx-build-warnings and squashes the following commits: 2c7faa8 [cocoatomo] [SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and building warnings --- python/docs/conf.py | 2 +- python/pyspark/mllib/feature.py | 2 ++ python/pyspark/rdd.py | 2 +- python/pyspark/sql.py | 10 +++++----- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/python/docs/conf.py b/python/docs/conf.py index 8e6324f058251..e58d97ae6a746 100644 --- a/python/docs/conf.py +++ b/python/docs/conf.py @@ -131,7 +131,7 @@ # Add any paths that contain custom static files (such as style sheets) here, # relative to this directory. They are copied after the builtin static files, # so a file named "default.css" will overwrite the builtin "default.css". -html_static_path = ['_static'] +#html_static_path = ['_static'] # Add any extra paths that contain custom files (such as robots.txt or # .htaccess) here, relative to this directory. These files are copied diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index a44a27fd3b6a6..f4cbf31b94fe2 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -44,6 +44,7 @@ def transform(self, word): """ :param word: a word :return: vector representation of word + Transforms a word to its vector representation Note: local use only @@ -57,6 +58,7 @@ def findSynonyms(self, x, num): :param x: a word or a vector representation of word :param num: number of synonyms to find :return: array of (word, cosineSimilarity) + Find synonyms of a word Note: local use only diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6797d50659a92..e13bab946c44a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2009,7 +2009,7 @@ def countApproxDistinct(self, relativeSD=0.05): of The Art Cardinality Estimation Algorithm", available here. - :param relativeSD Relative accuracy. Smaller values create + :param relativeSD: Relative accuracy. Smaller values create counters that require more space. It must be greater than 0.000017. diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d3d36eb995ab6..b31a82f9b19ac 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -19,14 +19,14 @@ public classes of Spark SQL: - L{SQLContext} - Main entry point for SQL functionality. + Main entry point for SQL functionality. - L{SchemaRDD} - A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In - addition to normal RDD operations, SchemaRDDs also support SQL. + A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In + addition to normal RDD operations, SchemaRDDs also support SQL. - L{Row} - A Row of data returned by a Spark SQL query. + A Row of data returned by a Spark SQL query. - L{HiveContext} - Main entry point for accessing data stored in Apache Hive.. + Main entry point for accessing data stored in Apache Hive.. """ import itertools From 69c67abaa9d4bb4b95792d1862bc65efc764c194 Mon Sep 17 00:00:00 2001 From: giwa Date: Sun, 12 Oct 2014 02:46:56 -0700 Subject: [PATCH 0967/1492] [SPARK-2377] Python API for Streaming This patch brings Python API for Streaming. This patch is based on work from @giwa Author: giwa Author: Ken Takagiwa Author: Davies Liu Author: Ken Takagiwa Author: Tathagata Das Author: Ken Author: Ken Takagiwa Author: Matthew Farrellee Closes #2538 from davies/streaming and squashes the following commits: 64561e4 [Davies Liu] fix tests 331ecce [Davies Liu] fix example 3e2492b [Davies Liu] change updateStateByKey() to easy API 182be73 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 02d0575 [Davies Liu] add wrapper for foreachRDD() bebeb4a [Davies Liu] address all comments 6db00da [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 8380064 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 52c535b [Davies Liu] remove fix for sum() e108ec1 [Davies Liu] address comments 37fe06f [Davies Liu] use random port for callback server d05871e [Davies Liu] remove reuse of PythonRDD be5e5ff [Davies Liu] merge branch of env, make tests stable. 8071541 [Davies Liu] Merge branch 'env' into streaming c7bbbce [Davies Liu] fix sphinx docs 6bb9d91 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 4d0ea8b [Davies Liu] clear reference of SparkEnv after stop 54bd92b [Davies Liu] improve tests c2b31cb [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming 7a88f9f [Davies Liu] rollback RDD.setContext(), use textFileStream() to test checkpointing bd8a4c2 [Davies Liu] fix scala style 7797c70 [Davies Liu] refactor ff88bec [Davies Liu] rename RDDFunction to TransformFunction d328aca [Davies Liu] fix serializer in queueStream 6f0da2f [Davies Liu] recover from checkpoint fa7261b [Davies Liu] refactor a13ff34 [Davies Liu] address comments 8466916 [Davies Liu] support checkpoint 9a16bd1 [Davies Liu] change number of partitions during tests b98d63f [Davies Liu] change private[spark] to private[python] eed6e2a [Davies Liu] rollback not needed changes e00136b [Davies Liu] address comments 069a94c [Davies Liu] fix the number of partitions during window() 338580a [Davies Liu] change _first(), _take(), _collect() as private API 19797f9 [Davies Liu] clean up 6ebceca [Davies Liu] add more tests c40c52d [Davies Liu] change first(), take(n) to has the same behavior as RDD 98ac6c2 [Davies Liu] support ssc.transform() b983f0f [Davies Liu] address comments 847f9b9 [Davies Liu] add more docs, add first(), take() e059ca2 [Davies Liu] move check of window into Python fce0ef5 [Davies Liu] rafactor of foreachRDD() 7001b51 [Davies Liu] refactor of queueStream() 26ea396 [Davies Liu] refactor 74df565 [Davies Liu] fix print and docs b32774c [Davies Liu] move java_import into streaming 604323f [Davies Liu] enable streaming tests c499ba0 [Davies Liu] remove Time and Duration 3f0fb4b [Davies Liu] refactor fix tests c28f520 [Davies Liu] support updateStateByKey d357b70 [Davies Liu] support windowed dstream bd13026 [Davies Liu] fix examples eec401e [Davies Liu] refactor, combine TransformedRDD, fix reuse PythonRDD, fix union 9a57685 [Davies Liu] fix python style bd27874 [Davies Liu] fix scala style 7339be0 [Davies Liu] delete tests 7f53086 [Davies Liu] support transform(), refactor and cleanup df098fc [Davies Liu] Merge branch 'master' into giwa 550dfd9 [giwa] WIP fixing 1.1 merge 5cdb6fa [giwa] changed for SCCallSiteSync e685853 [giwa] meged with rebased 1.1 branch 2d32a74 [giwa] added some StreamingContextTestSuite 4a59e1e [giwa] WIP:added more test for StreamingContext 8ffdbf1 [giwa] added atexit to handle callback server d5f5fcb [giwa] added comment for StreamingContext.sparkContext 63c881a [giwa] added StreamingContext.sparkContext d39f102 [giwa] added StreamingContext.remember d542743 [giwa] clean up code 2fdf0de [Matthew Farrellee] Fix scalastyle errors c0a06bc [giwa] delete not implemented functions f385976 [giwa] delete inproper comments b0f2015 [giwa] added comment in dstream._test_output bebb3f3 [giwa] remove the last brank line fbed8da [giwa] revert pom.xml 8ed93af [giwa] fixed explanaiton 066ba90 [giwa] revert pom.xml fa4af88 [giwa] remove duplicated import 6ae3caa [giwa] revert pom.xml 7dc7391 [giwa] fixed typo 62dc7a3 [giwa] clean up exmples f04882c [giwa] clen up examples b171ec3 [giwa] fixed pep8 violation f198d14 [giwa] clean up code 3166d31 [giwa] clean up c00e091 [giwa] change test case not to use awaitTermination e80647e [giwa] adopted the latest compression way of python command 58e41ff [giwa] merge with master 455e5af [giwa] removed wasted print in DStream af336b7 [giwa] add comments ddd4ee1 [giwa] added TODO coments 99ce042 [giwa] added saveAsTextFiles and saveAsPickledFiles 2a06cdb [giwa] remove waste duplicated code c5ecfc1 [giwa] basic function test cases are passed 8dcda84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 795b2cd [giwa] broke something 1e126bf [giwa] WIP: solved partitioned and None is not recognized f67cf57 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test 953deb0 [giwa] edited the comment to add more precise description af610d3 [giwa] removed unnesessary changes c1d546e [giwa] fixed PEP-008 violation 99410be [giwa] delete waste file b3b0362 [giwa] added basic operation test cases 9cde7c9 [giwa] WIP added test case bd3ba53 [giwa] WIP 5c04a5f [giwa] WIP: added PythonTestInputStream 019ef38 [giwa] WIP 1934726 [giwa] update comment 376e3ac [giwa] WIP 932372a [giwa] clean up dstream.py 0b09cff [giwa] added stop in StreamingContext 92e333e [giwa] implemented reduce and count function in Dstream 1b83354 [giwa] Removed the waste line 88f7506 [Ken Takagiwa] Kill py4j callback server properly 54b5358 [Ken Takagiwa] tried to restart callback server 4f07163 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. fe02547 [Ken Takagiwa] remove waste file 2ad7bd3 [Ken Takagiwa] clean up codes 6197a11 [Ken Takagiwa] clean up code eb4bf48 [Ken Takagiwa] fix map function 98c2a00 [Ken Takagiwa] added count operation but this implementation need double check 58591d2 [Ken Takagiwa] reduceByKey is working 0df7111 [Ken Takagiwa] delete old file f485b1d [Ken Takagiwa] fied input of socketTextDStream dd6de81 [Ken Takagiwa] initial commit for socketTextStream 247fd74 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 4bcb318 [Ken Takagiwa] implementing transform function in Python 38adf95 [Ken Takagiwa] added reducedByKey not working yet 66fcfff [Ken Takagiwa] modify dstream.py to fix indent error 41886c2 [Ken Takagiwa] comment PythonDStream.PairwiseDStream 0b99bec [Ken] initial commit for pySparkStreaming c214199 [giwa] added testcase for combineByKey 5625bdc [giwa] added gorupByKey testcase 10ab87b [giwa] added sparkContext as input parameter in StreamingContext 10b5b04 [giwa] removed wasted print in DStream e54f986 [giwa] add comments 16aa64f [giwa] added TODO coments 74535d4 [giwa] added saveAsTextFiles and saveAsPickledFiles f76c182 [giwa] remove waste duplicated code 18c8723 [giwa] modified streaming test case to add coment 13fb44c [giwa] basic function test cases are passed 3000b2b [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 ff14070 [giwa] broke something bcdec33 [giwa] WIP: solved partitioned and None is not recognized 270a9e1 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test bb10956 [giwa] edited the comment to add more precise description 253a863 [giwa] removed unnesessary changes 3d37822 [giwa] fixed PEP-008 violation f21cab3 [giwa] delete waste file 878bad7 [giwa] added basic operation test cases ce2acd2 [giwa] WIP added test case 9ad6855 [giwa] WIP 1df77f5 [giwa] WIP: added PythonTestInputStream 1523b66 [giwa] WIP 8a0fbbc [giwa] update comment fe648e3 [giwa] WIP 29c2bc5 [giwa] initial commit for testcase 4d40d63 [giwa] clean up dstream.py c462bb3 [giwa] added stop in StreamingContext d2c01ba [giwa] clean up examples 3c45cd2 [giwa] implemented reduce and count function in Dstream b349649 [giwa] Removed the waste line 3b498e1 [Ken Takagiwa] Kill py4j callback server properly 84a9668 [Ken Takagiwa] tried to restart callback server 9ab8952 [Tathagata Das] Added extra line. 05e991b [Tathagata Das] Added missing file b1d2a30 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. 678e854 [Ken Takagiwa] remove waste file 0a8bbbb [Ken Takagiwa] clean up codes bab31c1 [Ken Takagiwa] clean up code 72b9738 [Ken Takagiwa] fix map function d3ee86a [Ken Takagiwa] added count operation but this implementation need double check 15feea9 [Ken Takagiwa] edit python sparkstreaming example 6f98e50 [Ken Takagiwa] reduceByKey is working c455c8d [Ken Takagiwa] added reducedByKey not working yet dc6995d [Ken Takagiwa] delete old file b31446a [Ken Takagiwa] fixed typo of network_workdcount.py ccfd214 [Ken Takagiwa] added doctest for pyspark.streaming.duration 0d1b954 [Ken Takagiwa] fied input of socketTextDStream f746109 [Ken Takagiwa] initial commit for socketTextStream bb7ccf3 [Ken Takagiwa] remove unused import in python 224fc5e [Ken Takagiwa] add empty line d2099d8 [Ken Takagiwa] sorted the import following Spark coding convention 5bac7ec [Ken Takagiwa] revert streaming/pom.xml e1df940 [Ken Takagiwa] revert pom.xml 494cae5 [Ken Takagiwa] remove not implemented DStream functions in python 17a74c6 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 1a0f065 [Ken Takagiwa] implementing transform function in Python d7b4d6f [Ken Takagiwa] added reducedByKey not working yet 87438e2 [Ken Takagiwa] modify dstream.py to fix indent error b406252 [Ken Takagiwa] comment PythonDStream.PairwiseDStream 454981d [Ken] initial commit for pySparkStreaming 150b94c [giwa] added some StreamingContextTestSuite f7bc8f9 [giwa] WIP:added more test for StreamingContext ee50c5a [giwa] added atexit to handle callback server fdc9125 [giwa] added comment for StreamingContext.sparkContext f5bfb70 [giwa] added StreamingContext.sparkContext da09768 [giwa] added StreamingContext.remember d68b568 [giwa] clean up code 4afa390 [giwa] clean up code 1fd6bc7 [Ken Takagiwa] Merge pull request #2 from mattf/giwa-master d9d59fe [Matthew Farrellee] Fix scalastyle errors 67473a9 [giwa] delete not implemented functions c97377c [giwa] delete inproper comments 2ea769e [giwa] added comment in dstream._test_output 3b27bd4 [giwa] remove the last brank line acfcaeb [giwa] revert pom.xml 93f7637 [giwa] fixed explanaiton 50fd6f9 [giwa] revert pom.xml 4f82c89 [giwa] remove duplicated import 9d1de23 [giwa] revert pom.xml 7339df2 [giwa] fixed typo 9c85e48 [giwa] clean up exmples 24f95db [giwa] clen up examples 0d30109 [giwa] fixed pep8 violation b7dab85 [giwa] improve test case 583e66d [giwa] move tests for streaming inside streaming directory 1d84142 [giwa] remove unimplement test f0ea311 [giwa] clean up code 171edeb [giwa] clean up 4dedd2d [giwa] change test case not to use awaitTermination 268a6a5 [giwa] Changed awaitTermination not to call awaitTermincation in Scala. Just use time.sleep instread 09a28bf [giwa] improve testcases 58150f5 [giwa] Changed the test case to focus the test operation 199e37f [giwa] adopted the latest compression way of python command 185fdbf [giwa] merge with master f1798c4 [giwa] merge with master e70f706 [giwa] added testcase for combineByKey e162822 [giwa] added gorupByKey testcase 97742fe [giwa] added sparkContext as input parameter in StreamingContext 14d4c0e [giwa] removed wasted print in DStream 6d8190a [giwa] add comments 4aa99e4 [giwa] added TODO coments e9fab72 [giwa] added saveAsTextFiles and saveAsPickledFiles 94f2b65 [giwa] remove waste duplicated code 580fbc2 [giwa] modified streaming test case to add coment 99e4bb3 [giwa] basic function test cases are passed 7051a84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 35933e1 [giwa] broke something 9767712 [giwa] WIP: solved partitioned and None is not recognized 4f2d7e6 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test 33c0f94d [giwa] edited the comment to add more precise description 774f18d [giwa] removed unnesessary changes 3a671cc [giwa] remove export PYSPARK_PYTHON in spark submit 8efa266 [giwa] fixed PEP-008 violation fa75d71 [giwa] delete waste file 7f96294 [giwa] added basic operation test cases 3dda31a [giwa] WIP added test case 1f68b78 [giwa] WIP c05922c [giwa] WIP: added PythonTestInputStream 1fd12ae [giwa] WIP c880a33 [giwa] update comment 5d22c92 [giwa] WIP ea4b06b [giwa] initial commit for testcase 5a9b525 [giwa] clean up dstream.py 79c5809 [giwa] added stop in StreamingContext 189dcea [giwa] clean up examples b8d7d24 [giwa] implemented reduce and count function in Dstream b6468e6 [giwa] Removed the waste line b47b5fd [Ken Takagiwa] Kill py4j callback server properly 19ddcdd [Ken Takagiwa] tried to restart callback server c9fc124 [Tathagata Das] Added extra line. 4caae3f [Tathagata Das] Added missing file 4eff053 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. 5e822d4 [Ken Takagiwa] remove waste file aeaf8a5 [Ken Takagiwa] clean up codes 9fa249b [Ken Takagiwa] clean up code 05459c6 [Ken Takagiwa] fix map function a9f4ecb [Ken Takagiwa] added count operation but this implementation need double check d1ee6ca [Ken Takagiwa] edit python sparkstreaming example 0b8b7d0 [Ken Takagiwa] reduceByKey is working d25d5cf [Ken Takagiwa] added reducedByKey not working yet 7f7c5d1 [Ken Takagiwa] delete old file 967dc26 [Ken Takagiwa] fixed typo of network_workdcount.py 57fb740 [Ken Takagiwa] added doctest for pyspark.streaming.duration 4b69fb1 [Ken Takagiwa] fied input of socketTextDStream 02f618a [Ken Takagiwa] initial commit for socketTextStream 4ce4058 [Ken Takagiwa] remove unused import in python 856d98e [Ken Takagiwa] add empty line 490e338 [Ken Takagiwa] sorted the import following Spark coding convention 5594bd4 [Ken Takagiwa] revert pom.xml 2adca84 [Ken Takagiwa] remove not implemented DStream functions in python e551e13 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit 3758175 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit c5518b4 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 dcf243f [Ken Takagiwa] implementing transform function in Python 9af03f4 [Ken Takagiwa] added reducedByKey not working yet 6e0d9c7 [Ken Takagiwa] modify dstream.py to fix indent error e497b9b [Ken Takagiwa] comment PythonDStream.PairwiseDStream 5c3a683 [Ken] initial commit for pySparkStreaming 665bfdb [giwa] added testcase for combineByKey a3d2379 [giwa] added gorupByKey testcase 636090a [giwa] added sparkContext as input parameter in StreamingContext e7ebb08 [giwa] removed wasted print in DStream d8b593b [giwa] add comments ea9c873 [giwa] added TODO coments 89ae38a [giwa] added saveAsTextFiles and saveAsPickledFiles e3033fc [giwa] remove waste duplicated code a14c7e1 [giwa] modified streaming test case to add coment 536def4 [giwa] basic function test cases are passed 2112638 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4 080541a [giwa] broke something 0704b86 [giwa] WIP: solved partitioned and None is not recognized 90a6484 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test a65f302 [giwa] edited the comment to add more precise description bdde697 [giwa] removed unnesessary changes e8c7bfc [giwa] remove export PYSPARK_PYTHON in spark submit 3334169 [giwa] fixed PEP-008 violation db0a303 [giwa] delete waste file 2cfd3a0 [giwa] added basic operation test cases 90ae568 [giwa] WIP added test case a120d07 [giwa] WIP f671cdb [giwa] WIP: added PythonTestInputStream 56fae45 [giwa] WIP e35e101 [giwa] Merge branch 'master' into testcase ba5112d [giwa] update comment 28aa56d [giwa] WIP fb08559 [giwa] initial commit for testcase a613b85 [giwa] clean up dstream.py c40c0ef [giwa] added stop in StreamingContext 31e4260 [giwa] clean up examples d2127d6 [giwa] implemented reduce and count function in Dstream 48f7746 [giwa] Removed the waste line 0f83eaa [Ken Takagiwa] delete py4j 0.8.1 1679808 [Ken Takagiwa] Kill py4j callback server properly f96cd4e [Ken Takagiwa] tried to restart callback server fe86198 [Ken Takagiwa] add py4j 0.8.2.1 but server is not launched 1064fe0 [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark 28c6620 [Ken Takagiwa] Implemented DStream.foreachRDD in the Python API using Py4J callback server 85b0fe1 [Ken Takagiwa] Merge pull request #1 from tdas/python-foreach 54e2e8c [Tathagata Das] Added extra line. e185338 [Tathagata Das] Added missing file a778d4b [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server. cc2092b [Ken Takagiwa] remove waste file d042ac6 [Ken Takagiwa] clean up codes 84a021f [Ken Takagiwa] clean up code bd20e17 [Ken Takagiwa] fix map function d01a125 [Ken Takagiwa] added count operation but this implementation need double check 7d05109 [Ken Takagiwa] merge with remote branch ae464e0 [Ken Takagiwa] edit python sparkstreaming example 04af046 [Ken Takagiwa] reduceByKey is working 3b6d7b0 [Ken Takagiwa] implementing transform function in Python 571d52d [Ken Takagiwa] added reducedByKey not working yet 5720979 [Ken Takagiwa] delete old file e604fcb [Ken Takagiwa] fixed typo of network_workdcount.py 4b7c08b [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark ce7d426 [Ken Takagiwa] added doctest for pyspark.streaming.duration a8c9fd5 [Ken Takagiwa] fixed for socketTextStream a61fa9e [Ken Takagiwa] fied input of socketTextDStream 1e84f41 [Ken Takagiwa] initial commit for socketTextStream 6d012f7 [Ken Takagiwa] remove unused import in python 25d30d5 [Ken Takagiwa] add empty line 6e0a64a [Ken Takagiwa] sorted the import following Spark coding convention fa4a7fc [Ken Takagiwa] revert streaming/pom.xml 8f8202b [Ken Takagiwa] revert streaming pom.xml c9d79dd [Ken Takagiwa] revert pom.xml 57e3e52 [Ken Takagiwa] remove not implemented DStream functions in python 0a516f5 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit a7a0b5c [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit 72bfc66 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10 69e9cd3 [Ken Takagiwa] implementing transform function in Python 94a0787 [Ken Takagiwa] added reducedByKey not working yet 88068cf [Ken Takagiwa] modify dstream.py to fix indent error 1367be5 [Ken Takagiwa] comment PythonDStream.PairwiseDStream eb2b3ba [Ken] Merge remote-tracking branch 'upstream/master' d8e51f9 [Ken] initial commit for pySparkStreaming --- .../apache/spark/api/python/PythonRDD.scala | 10 +- .../main/python/streaming/hdfs_wordcount.py | 49 ++ .../python/streaming/network_wordcount.py | 48 ++ .../streaming/stateful_network_wordcount.py | 57 ++ python/docs/epytext.py | 2 +- python/docs/index.rst | 1 + python/docs/pyspark.rst | 3 +- python/pyspark/context.py | 8 +- python/pyspark/serializers.py | 3 + python/pyspark/streaming/__init__.py | 21 + python/pyspark/streaming/context.py | 325 +++++++++ python/pyspark/streaming/dstream.py | 621 ++++++++++++++++++ python/pyspark/streaming/tests.py | 545 +++++++++++++++ python/pyspark/streaming/util.py | 128 ++++ python/run-tests | 7 + .../streaming/api/java/JavaDStreamLike.scala | 2 +- .../streaming/api/python/PythonDStream.scala | 316 +++++++++ 17 files changed, 2133 insertions(+), 13 deletions(-) create mode 100644 examples/src/main/python/streaming/hdfs_wordcount.py create mode 100644 examples/src/main/python/streaming/network_wordcount.py create mode 100644 examples/src/main/python/streaming/stateful_network_wordcount.py create mode 100644 python/pyspark/streaming/__init__.py create mode 100644 python/pyspark/streaming/context.py create mode 100644 python/pyspark/streaming/dstream.py create mode 100644 python/pyspark/streaming/tests.py create mode 100644 python/pyspark/streaming/util.py create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index c74f86548ef85..4acbdf9d5e25f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -25,8 +25,6 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.collection.mutable import scala.language.existentials -import scala.reflect.ClassTag -import scala.util.{Try, Success, Failure} import net.razorvine.pickle.{Pickler, Unpickler} @@ -42,7 +40,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils private[spark] class PythonRDD( - parent: RDD[_], + @transient parent: RDD[_], command: Array[Byte], envVars: JMap[String, String], pythonIncludes: JList[String], @@ -55,9 +53,9 @@ private[spark] class PythonRDD( val bufferSize = conf.getInt("spark.buffer.size", 65536) val reuse_worker = conf.getBoolean("spark.python.worker.reuse", true) - override def getPartitions = parent.partitions + override def getPartitions = firstParent.partitions - override val partitioner = if (preservePartitoning) parent.partitioner else None + override val partitioner = if (preservePartitoning) firstParent.partitioner else None override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = { val startTime = System.currentTimeMillis @@ -234,7 +232,7 @@ private[spark] class PythonRDD( dataOut.writeInt(command.length) dataOut.write(command) // Data values - PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut) + PythonRDD.writeIteratorToStream(firstParent.iterator(split, context), dataOut) dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) dataOut.flush() } catch { diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py new file mode 100644 index 0000000000000..40faff0ccc7db --- /dev/null +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -0,0 +1,49 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Counts words in new text files created in the given directory + Usage: hdfs_wordcount.py + is the directory that Spark Streaming will use to find and read new text files. + + To run this on your local machine on directory `localdir`, run this example + $ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localdir + + Then create a text file in `localdir` and the words in the file will get counted. +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + +if __name__ == "__main__": + if len(sys.argv) != 2: + print >> sys.stderr, "Usage: hdfs_wordcount.py " + exit(-1) + + sc = SparkContext(appName="PythonStreamingHDFSWordCount") + ssc = StreamingContext(sc, 1) + + lines = ssc.textFileStream(sys.argv[1]) + counts = lines.flatMap(lambda line: line.split(" "))\ + .map(lambda x: (x, 1))\ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/network_wordcount.py b/examples/src/main/python/streaming/network_wordcount.py new file mode 100644 index 0000000000000..cfa9c1ff5bfbc --- /dev/null +++ b/examples/src/main/python/streaming/network_wordcount.py @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the network every second. + Usage: network_wordcount.py + and describe the TCP server that Spark Streaming would connect to receive data. + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999` +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: network_wordcount.py " + exit(-1) + sc = SparkContext(appName="PythonStreamingNetworkWordCount") + ssc = StreamingContext(sc, 1) + + lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2])) + counts = lines.flatMap(lambda line: line.split(" "))\ + .map(lambda word: (word, 1))\ + .reduceByKey(lambda a, b: a+b) + counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/stateful_network_wordcount.py b/examples/src/main/python/streaming/stateful_network_wordcount.py new file mode 100644 index 0000000000000..18a9a5a452ffb --- /dev/null +++ b/examples/src/main/python/streaming/stateful_network_wordcount.py @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Counts words in UTF8 encoded, '\n' delimited text received from the + network every second. + + Usage: stateful_network_wordcount.py + and describe the TCP server that Spark Streaming + would connect to receive data. + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + and then run the example + `$ bin/spark-submit examples/src/main/python/streaming/stateful_network_wordcount.py \ + localhost 9999` +""" + +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + +if __name__ == "__main__": + if len(sys.argv) != 3: + print >> sys.stderr, "Usage: stateful_network_wordcount.py " + exit(-1) + sc = SparkContext(appName="PythonStreamingStatefulNetworkWordCount") + ssc = StreamingContext(sc, 1) + ssc.checkpoint("checkpoint") + + def updateFunc(new_values, last_sum): + return sum(new_values) + (last_sum or 0) + + lines = ssc.socketTextStream(sys.argv[1], int(sys.argv[2])) + running_counts = lines.flatMap(lambda line: line.split(" "))\ + .map(lambda word: (word, 1))\ + .updateStateByKey(updateFunc) + + running_counts.pprint() + + ssc.start() + ssc.awaitTermination() diff --git a/python/docs/epytext.py b/python/docs/epytext.py index 61d731bff570d..19fefbfc057a4 100644 --- a/python/docs/epytext.py +++ b/python/docs/epytext.py @@ -5,7 +5,7 @@ (r"L{([\w.()]+)}", r":class:`\1`"), (r"[LC]{(\w+\.\w+)\(\)}", r":func:`\1`"), (r"C{([\w.()]+)}", r":class:`\1`"), - (r"[IBCM]{(.+)}", r"`\1`"), + (r"[IBCM]{([^}]+)}", r"`\1`"), ('pyspark.rdd.RDD', 'RDD'), ) diff --git a/python/docs/index.rst b/python/docs/index.rst index d66e051b15371..703bef644de28 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -13,6 +13,7 @@ Contents: pyspark pyspark.sql + pyspark.streaming pyspark.mllib diff --git a/python/docs/pyspark.rst b/python/docs/pyspark.rst index a68bd62433085..e81be3b6cb796 100644 --- a/python/docs/pyspark.rst +++ b/python/docs/pyspark.rst @@ -7,8 +7,9 @@ Subpackages .. toctree:: :maxdepth: 1 - pyspark.mllib pyspark.sql + pyspark.streaming + pyspark.mllib Contents -------- diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 85c04624da4a6..89d2e2e5b4a8e 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -68,7 +68,7 @@ class SparkContext(object): def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, environment=None, batchSize=0, serializer=PickleSerializer(), conf=None, - gateway=None): + gateway=None, jsc=None): """ Create a new SparkContext. At least the master and app name should be set, either through the named parameters here or through C{conf}. @@ -104,14 +104,14 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, SparkContext._ensure_initialized(self, gateway=gateway) try: self._do_init(master, appName, sparkHome, pyFiles, environment, batchSize, serializer, - conf) + conf, jsc) except: # If an error occurs, clean up in order to allow future SparkContext creation: self.stop() raise def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, serializer, - conf): + conf, jsc): self.environment = environment or {} self._conf = conf or SparkConf(_jvm=self._jvm) self._batchSize = batchSize # -1 represents an unlimited batch size @@ -154,7 +154,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self.environment[varName] = v # Create the Java SparkContext through Py4J - self._jsc = self._initialize_context(self._conf._jconf) + self._jsc = jsc or self._initialize_context(self._conf._jconf) # Create a single Accumulator in Java that we'll send all our updates through; # they will be passed back to us through a TCP server diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 3d1a34b281acc..08a0f0d8ffb3e 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -114,6 +114,9 @@ def __ne__(self, other): def __repr__(self): return "<%s object>" % self.__class__.__name__ + def __hash__(self): + return hash(str(self)) + class FramedSerializer(Serializer): diff --git a/python/pyspark/streaming/__init__.py b/python/pyspark/streaming/__init__.py new file mode 100644 index 0000000000000..d2644a1d4ffab --- /dev/null +++ b/python/pyspark/streaming/__init__.py @@ -0,0 +1,21 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark.streaming.context import StreamingContext +from pyspark.streaming.dstream import DStream + +__all__ = ['StreamingContext', 'DStream'] diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py new file mode 100644 index 0000000000000..dc9dc41121935 --- /dev/null +++ b/python/pyspark/streaming/context.py @@ -0,0 +1,325 @@ +# +# 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. +# +import os +import sys + +from py4j.java_collections import ListConverter +from py4j.java_gateway import java_import, JavaObject + +from pyspark import RDD, SparkConf +from pyspark.serializers import UTF8Deserializer, CloudPickleSerializer +from pyspark.context import SparkContext +from pyspark.storagelevel import StorageLevel +from pyspark.streaming.dstream import DStream +from pyspark.streaming.util import TransformFunction, TransformFunctionSerializer + +__all__ = ["StreamingContext"] + + +def _daemonize_callback_server(): + """ + Hack Py4J to daemonize callback server + + The thread of callback server has daemon=False, it will block the driver + from exiting if it's not shutdown. The following code replace `start()` + of CallbackServer with a new version, which set daemon=True for this + thread. + + Also, it will update the port number (0) with real port + """ + # TODO: create a patch for Py4J + import socket + import py4j.java_gateway + logger = py4j.java_gateway.logger + from py4j.java_gateway import Py4JNetworkError + from threading import Thread + + def start(self): + """Starts the CallbackServer. This method should be called by the + client instead of run().""" + self.server_socket = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + self.server_socket.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, + 1) + try: + self.server_socket.bind((self.address, self.port)) + if not self.port: + # update port with real port + self.port = self.server_socket.getsockname()[1] + except Exception as e: + msg = 'An error occurred while trying to start the callback server: %s' % e + logger.exception(msg) + raise Py4JNetworkError(msg) + + # Maybe thread needs to be cleanup up? + self.thread = Thread(target=self.run) + self.thread.daemon = True + self.thread.start() + + py4j.java_gateway.CallbackServer.start = start + + +class StreamingContext(object): + """ + Main entry point for Spark Streaming functionality. A StreamingContext + represents the connection to a Spark cluster, and can be used to create + L{DStream} various input sources. It can be from an existing L{SparkContext}. + After creating and transforming DStreams, the streaming computation can + be started and stopped using `context.start()` and `context.stop()`, + respectively. `context.awaitTransformation()` allows the current thread + to wait for the termination of the context by `stop()` or by an exception. + """ + _transformerSerializer = None + + def __init__(self, sparkContext, batchDuration=None, jssc=None): + """ + Create a new StreamingContext. + + @param sparkContext: L{SparkContext} object. + @param batchDuration: the time interval (in seconds) at which streaming + data will be divided into batches + """ + + self._sc = sparkContext + self._jvm = self._sc._jvm + self._jssc = jssc or self._initialize_context(self._sc, batchDuration) + + def _initialize_context(self, sc, duration): + self._ensure_initialized() + return self._jvm.JavaStreamingContext(sc._jsc, self._jduration(duration)) + + def _jduration(self, seconds): + """ + Create Duration object given number of seconds + """ + return self._jvm.Duration(int(seconds * 1000)) + + @classmethod + def _ensure_initialized(cls): + SparkContext._ensure_initialized() + gw = SparkContext._gateway + + java_import(gw.jvm, "org.apache.spark.streaming.*") + java_import(gw.jvm, "org.apache.spark.streaming.api.java.*") + java_import(gw.jvm, "org.apache.spark.streaming.api.python.*") + + # start callback server + # getattr will fallback to JVM, so we cannot test by hasattr() + if "_callback_server" not in gw.__dict__: + _daemonize_callback_server() + # use random port + gw._start_callback_server(0) + # gateway with real port + gw._python_proxy_port = gw._callback_server.port + # get the GatewayServer object in JVM by ID + jgws = JavaObject("GATEWAY_SERVER", gw._gateway_client) + # update the port of CallbackClient with real port + gw.jvm.PythonDStream.updatePythonGatewayPort(jgws, gw._python_proxy_port) + + # register serializer for TransformFunction + # it happens before creating SparkContext when loading from checkpointing + cls._transformerSerializer = TransformFunctionSerializer( + SparkContext._active_spark_context, CloudPickleSerializer(), gw) + + @classmethod + def getOrCreate(cls, checkpointPath, setupFunc): + """ + Either recreate a StreamingContext from checkpoint data or create a new StreamingContext. + If checkpoint data exists in the provided `checkpointPath`, then StreamingContext will be + recreated from the checkpoint data. If the data does not exist, then the provided setupFunc + will be used to create a JavaStreamingContext. + + @param checkpointPath Checkpoint directory used in an earlier JavaStreamingContext program + @param setupFunc Function to create a new JavaStreamingContext and setup DStreams + """ + # TODO: support checkpoint in HDFS + if not os.path.exists(checkpointPath) or not os.listdir(checkpointPath): + ssc = setupFunc() + ssc.checkpoint(checkpointPath) + return ssc + + cls._ensure_initialized() + gw = SparkContext._gateway + + try: + jssc = gw.jvm.JavaStreamingContext(checkpointPath) + except Exception: + print >>sys.stderr, "failed to load StreamingContext from checkpoint" + raise + + jsc = jssc.sparkContext() + conf = SparkConf(_jconf=jsc.getConf()) + sc = SparkContext(conf=conf, gateway=gw, jsc=jsc) + # update ctx in serializer + SparkContext._active_spark_context = sc + cls._transformerSerializer.ctx = sc + return StreamingContext(sc, None, jssc) + + @property + def sparkContext(self): + """ + Return SparkContext which is associated with this StreamingContext. + """ + return self._sc + + def start(self): + """ + Start the execution of the streams. + """ + self._jssc.start() + + def awaitTermination(self, timeout=None): + """ + Wait for the execution to stop. + @param timeout: time to wait in seconds + """ + if timeout is None: + self._jssc.awaitTermination() + else: + self._jssc.awaitTermination(int(timeout * 1000)) + + def stop(self, stopSparkContext=True, stopGraceFully=False): + """ + Stop the execution of the streams, with option of ensuring all + received data has been processed. + + @param stopSparkContext: Stop the associated SparkContext or not + @param stopGracefully: Stop gracefully by waiting for the processing + of all received data to be completed + """ + self._jssc.stop(stopSparkContext, stopGraceFully) + if stopSparkContext: + self._sc.stop() + + def remember(self, duration): + """ + Set each DStreams in this context to remember RDDs it generated + in the last given duration. DStreams remember RDDs only for a + limited duration of time and releases them for garbage collection. + This method allows the developer to specify how to long to remember + the RDDs (if the developer wishes to query old data outside the + DStream computation). + + @param duration: Minimum duration (in seconds) that each DStream + should remember its RDDs + """ + self._jssc.remember(self._jduration(duration)) + + def checkpoint(self, directory): + """ + Sets the context to periodically checkpoint the DStream operations for master + fault-tolerance. The graph will be checkpointed every batch interval. + + @param directory: HDFS-compatible directory where the checkpoint data + will be reliably stored + """ + self._jssc.checkpoint(directory) + + def socketTextStream(self, hostname, port, storageLevel=StorageLevel.MEMORY_AND_DISK_SER_2): + """ + Create an input from TCP source hostname:port. Data is received using + a TCP socket and receive byte is interpreted as UTF8 encoded ``\\n`` delimited + lines. + + @param hostname: Hostname to connect to for receiving data + @param port: Port to connect to for receiving data + @param storageLevel: Storage level to use for storing the received objects + """ + jlevel = self._sc._getJavaStorageLevel(storageLevel) + return DStream(self._jssc.socketTextStream(hostname, port, jlevel), self, + UTF8Deserializer()) + + def textFileStream(self, directory): + """ + Create an input stream that monitors a Hadoop-compatible file system + for new files and reads them as text files. Files must be wrriten to the + monitored directory by "moving" them from another location within the same + file system. File names starting with . are ignored. + """ + return DStream(self._jssc.textFileStream(directory), self, UTF8Deserializer()) + + def _check_serializers(self, rdds): + # make sure they have same serializer + if len(set(rdd._jrdd_deserializer for rdd in rdds)) > 1: + for i in range(len(rdds)): + # reset them to sc.serializer + rdds[i] = rdds[i]._reserialize() + + def queueStream(self, rdds, oneAtATime=True, default=None): + """ + Create an input stream from an queue of RDDs or list. In each batch, + it will process either one or all of the RDDs returned by the queue. + + NOTE: changes to the queue after the stream is created will not be recognized. + + @param rdds: Queue of RDDs + @param oneAtATime: pick one rdd each time or pick all of them once. + @param default: The default rdd if no more in rdds + """ + if default and not isinstance(default, RDD): + default = self._sc.parallelize(default) + + if not rdds and default: + rdds = [rdds] + + if rdds and not isinstance(rdds[0], RDD): + rdds = [self._sc.parallelize(input) for input in rdds] + self._check_serializers(rdds) + + jrdds = ListConverter().convert([r._jrdd for r in rdds], + SparkContext._gateway._gateway_client) + queue = self._jvm.PythonDStream.toRDDQueue(jrdds) + if default: + default = default._reserialize(rdds[0]._jrdd_deserializer) + jdstream = self._jssc.queueStream(queue, oneAtATime, default._jrdd) + else: + jdstream = self._jssc.queueStream(queue, oneAtATime) + return DStream(jdstream, self, rdds[0]._jrdd_deserializer) + + def transform(self, dstreams, transformFunc): + """ + Create a new DStream in which each RDD is generated by applying + a function on RDDs of the DStreams. The order of the JavaRDDs in + the transform function parameter will be the same as the order + of corresponding DStreams in the list. + """ + jdstreams = ListConverter().convert([d._jdstream for d in dstreams], + SparkContext._gateway._gateway_client) + # change the final serializer to sc.serializer + func = TransformFunction(self._sc, + lambda t, *rdds: transformFunc(rdds).map(lambda x: x), + *[d._jrdd_deserializer for d in dstreams]) + jfunc = self._jvm.TransformFunction(func) + jdstream = self._jssc.transform(jdstreams, jfunc) + return DStream(jdstream, self, self._sc.serializer) + + def union(self, *dstreams): + """ + Create a unified DStream from multiple DStreams of the same + type and same slide duration. + """ + if not dstreams: + raise ValueError("should have at least one DStream to union") + if len(dstreams) == 1: + return dstreams[0] + if len(set(s._jrdd_deserializer for s in dstreams)) > 1: + raise ValueError("All DStreams should have same serializer") + if len(set(s._slideDuration for s in dstreams)) > 1: + raise ValueError("All DStreams should have same slide duration") + first = dstreams[0] + jrest = ListConverter().convert([d._jdstream for d in dstreams[1:]], + SparkContext._gateway._gateway_client) + return DStream(self._jssc.union(first._jdstream, jrest), self, first._jrdd_deserializer) diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py new file mode 100644 index 0000000000000..5ae5cf07f0137 --- /dev/null +++ b/python/pyspark/streaming/dstream.py @@ -0,0 +1,621 @@ +# +# 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. +# + +from itertools import chain, ifilter, imap +import operator +import time +from datetime import datetime + +from py4j.protocol import Py4JJavaError + +from pyspark import RDD +from pyspark.storagelevel import StorageLevel +from pyspark.streaming.util import rddToFileName, TransformFunction +from pyspark.rdd import portable_hash +from pyspark.resultiterable import ResultIterable + +__all__ = ["DStream"] + + +class DStream(object): + """ + A Discretized Stream (DStream), the basic abstraction in Spark Streaming, + is a continuous sequence of RDDs (of the same type) representing a + continuous stream of data (see L{RDD} in the Spark core documentation + for more details on RDDs). + + DStreams can either be created from live data (such as, data from TCP + sockets, Kafka, Flume, etc.) using a L{StreamingContext} or it can be + generated by transforming existing DStreams using operations such as + `map`, `window` and `reduceByKeyAndWindow`. While a Spark Streaming + program is running, each DStream periodically generates a RDD, either + from live data or by transforming the RDD generated by a parent DStream. + + DStreams internally is characterized by a few basic properties: + - A list of other DStreams that the DStream depends on + - A time interval at which the DStream generates an RDD + - A function that is used to generate an RDD after each time interval + """ + def __init__(self, jdstream, ssc, jrdd_deserializer): + self._jdstream = jdstream + self._ssc = ssc + self._sc = ssc._sc + self._jrdd_deserializer = jrdd_deserializer + self.is_cached = False + self.is_checkpointed = False + + def context(self): + """ + Return the StreamingContext associated with this DStream + """ + return self._ssc + + def count(self): + """ + Return a new DStream in which each RDD has a single element + generated by counting each RDD of this DStream. + """ + return self.mapPartitions(lambda i: [sum(1 for _ in i)]).reduce(operator.add) + + def filter(self, f): + """ + Return a new DStream containing only the elements that satisfy predicate. + """ + def func(iterator): + return ifilter(f, iterator) + return self.mapPartitions(func, True) + + def flatMap(self, f, preservesPartitioning=False): + """ + Return a new DStream by applying a function to all elements of + this DStream, and then flattening the results + """ + def func(s, iterator): + return chain.from_iterable(imap(f, iterator)) + return self.mapPartitionsWithIndex(func, preservesPartitioning) + + def map(self, f, preservesPartitioning=False): + """ + Return a new DStream by applying a function to each element of DStream. + """ + def func(iterator): + return imap(f, iterator) + return self.mapPartitions(func, preservesPartitioning) + + def mapPartitions(self, f, preservesPartitioning=False): + """ + Return a new DStream in which each RDD is generated by applying + mapPartitions() to each RDDs of this DStream. + """ + def func(s, iterator): + return f(iterator) + return self.mapPartitionsWithIndex(func, preservesPartitioning) + + def mapPartitionsWithIndex(self, f, preservesPartitioning=False): + """ + Return a new DStream in which each RDD is generated by applying + mapPartitionsWithIndex() to each RDDs of this DStream. + """ + return self.transform(lambda rdd: rdd.mapPartitionsWithIndex(f, preservesPartitioning)) + + def reduce(self, func): + """ + Return a new DStream in which each RDD has a single element + generated by reducing each RDD of this DStream. + """ + return self.map(lambda x: (None, x)).reduceByKey(func, 1).map(lambda x: x[1]) + + def reduceByKey(self, func, numPartitions=None): + """ + Return a new DStream by applying reduceByKey to each RDD. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + return self.combineByKey(lambda x: x, func, func, numPartitions) + + def combineByKey(self, createCombiner, mergeValue, mergeCombiners, + numPartitions=None): + """ + Return a new DStream by applying combineByKey to each RDD. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + + def func(rdd): + return rdd.combineByKey(createCombiner, mergeValue, mergeCombiners, numPartitions) + return self.transform(func) + + def partitionBy(self, numPartitions, partitionFunc=portable_hash): + """ + Return a copy of the DStream in which each RDD are partitioned + using the specified partitioner. + """ + return self.transform(lambda rdd: rdd.partitionBy(numPartitions, partitionFunc)) + + def foreachRDD(self, func): + """ + Apply a function to each RDD in this DStream. + """ + if func.func_code.co_argcount == 1: + old_func = func + func = lambda t, rdd: old_func(rdd) + jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer) + api = self._ssc._jvm.PythonDStream + api.callForeachRDD(self._jdstream, jfunc) + + def pprint(self): + """ + Print the first ten elements of each RDD generated in this DStream. + """ + def takeAndPrint(time, rdd): + taken = rdd.take(11) + print "-------------------------------------------" + print "Time: %s" % time + print "-------------------------------------------" + for record in taken[:10]: + print record + if len(taken) > 10: + print "..." + print + + self.foreachRDD(takeAndPrint) + + def mapValues(self, f): + """ + Return a new DStream by applying a map function to the value of + each key-value pairs in this DStream without changing the key. + """ + map_values_fn = lambda (k, v): (k, f(v)) + return self.map(map_values_fn, preservesPartitioning=True) + + def flatMapValues(self, f): + """ + Return a new DStream by applying a flatmap function to the value + of each key-value pairs in this DStream without changing the key. + """ + flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + return self.flatMap(flat_map_fn, preservesPartitioning=True) + + def glom(self): + """ + Return a new DStream in which RDD is generated by applying glom() + to RDD of this DStream. + """ + def func(iterator): + yield list(iterator) + return self.mapPartitions(func) + + def cache(self): + """ + Persist the RDDs of this DStream with the default storage level + (C{MEMORY_ONLY_SER}). + """ + self.is_cached = True + self.persist(StorageLevel.MEMORY_ONLY_SER) + return self + + def persist(self, storageLevel): + """ + Persist the RDDs of this DStream with the given storage level + """ + self.is_cached = True + javaStorageLevel = self._sc._getJavaStorageLevel(storageLevel) + self._jdstream.persist(javaStorageLevel) + return self + + def checkpoint(self, interval): + """ + Enable periodic checkpointing of RDDs of this DStream + + @param interval: time in seconds, after each period of that, generated + RDD will be checkpointed + """ + self.is_checkpointed = True + self._jdstream.checkpoint(self._ssc._jduration(interval)) + return self + + def groupByKey(self, numPartitions=None): + """ + Return a new DStream by applying groupByKey on each RDD. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + return self.transform(lambda rdd: rdd.groupByKey(numPartitions)) + + def countByValue(self): + """ + Return a new DStream in which each RDD contains the counts of each + distinct value in each RDD of this DStream. + """ + return self.map(lambda x: (x, None)).reduceByKey(lambda x, y: None).count() + + def saveAsTextFiles(self, prefix, suffix=None): + """ + Save each RDD in this DStream as at text file, using string + representation of elements. + """ + def saveAsTextFile(t, rdd): + path = rddToFileName(prefix, suffix, t) + try: + rdd.saveAsTextFile(path) + except Py4JJavaError as e: + # after recovered from checkpointing, the foreachRDD may + # be called twice + if 'FileAlreadyExistsException' not in str(e): + raise + return self.foreachRDD(saveAsTextFile) + + # TODO: uncomment this until we have ssc.pickleFileStream() + # def saveAsPickleFiles(self, prefix, suffix=None): + # """ + # Save each RDD in this DStream as at binary file, the elements are + # serialized by pickle. + # """ + # def saveAsPickleFile(t, rdd): + # path = rddToFileName(prefix, suffix, t) + # try: + # rdd.saveAsPickleFile(path) + # except Py4JJavaError as e: + # # after recovered from checkpointing, the foreachRDD may + # # be called twice + # if 'FileAlreadyExistsException' not in str(e): + # raise + # return self.foreachRDD(saveAsPickleFile) + + def transform(self, func): + """ + Return a new DStream in which each RDD is generated by applying a function + on each RDD of this DStream. + + `func` can have one argument of `rdd`, or have two arguments of + (`time`, `rdd`) + """ + if func.func_code.co_argcount == 1: + oldfunc = func + func = lambda t, rdd: oldfunc(rdd) + assert func.func_code.co_argcount == 2, "func should take one or two arguments" + return TransformedDStream(self, func) + + def transformWith(self, func, other, keepSerializer=False): + """ + Return a new DStream in which each RDD is generated by applying a function + on each RDD of this DStream and 'other' DStream. + + `func` can have two arguments of (`rdd_a`, `rdd_b`) or have three + arguments of (`time`, `rdd_a`, `rdd_b`) + """ + if func.func_code.co_argcount == 2: + oldfunc = func + func = lambda t, a, b: oldfunc(a, b) + assert func.func_code.co_argcount == 3, "func should take two or three arguments" + jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer, other._jrdd_deserializer) + dstream = self._sc._jvm.PythonTransformed2DStream(self._jdstream.dstream(), + other._jdstream.dstream(), jfunc) + jrdd_serializer = self._jrdd_deserializer if keepSerializer else self._sc.serializer + return DStream(dstream.asJavaDStream(), self._ssc, jrdd_serializer) + + def repartition(self, numPartitions): + """ + Return a new DStream with an increased or decreased level of parallelism. + """ + return self.transform(lambda rdd: rdd.repartition(numPartitions)) + + @property + def _slideDuration(self): + """ + Return the slideDuration in seconds of this DStream + """ + return self._jdstream.dstream().slideDuration().milliseconds() / 1000.0 + + def union(self, other): + """ + Return a new DStream by unifying data of another DStream with this DStream. + + @param other: Another DStream having the same interval (i.e., slideDuration) + as this DStream. + """ + if self._slideDuration != other._slideDuration: + raise ValueError("the two DStream should have same slide duration") + return self.transformWith(lambda a, b: a.union(b), other, True) + + def cogroup(self, other, numPartitions=None): + """ + Return a new DStream by applying 'cogroup' between RDDs of this + DStream and `other` DStream. + + Hash partitioning is used to generate the RDDs with `numPartitions` partitions. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + return self.transformWith(lambda a, b: a.cogroup(b, numPartitions), other) + + def join(self, other, numPartitions=None): + """ + Return a new DStream by applying 'join' between RDDs of this DStream and + `other` DStream. + + Hash partitioning is used to generate the RDDs with `numPartitions` + partitions. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + return self.transformWith(lambda a, b: a.join(b, numPartitions), other) + + def leftOuterJoin(self, other, numPartitions=None): + """ + Return a new DStream by applying 'left outer join' between RDDs of this DStream and + `other` DStream. + + Hash partitioning is used to generate the RDDs with `numPartitions` + partitions. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + return self.transformWith(lambda a, b: a.leftOuterJoin(b, numPartitions), other) + + def rightOuterJoin(self, other, numPartitions=None): + """ + Return a new DStream by applying 'right outer join' between RDDs of this DStream and + `other` DStream. + + Hash partitioning is used to generate the RDDs with `numPartitions` + partitions. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + return self.transformWith(lambda a, b: a.rightOuterJoin(b, numPartitions), other) + + def fullOuterJoin(self, other, numPartitions=None): + """ + Return a new DStream by applying 'full outer join' between RDDs of this DStream and + `other` DStream. + + Hash partitioning is used to generate the RDDs with `numPartitions` + partitions. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + return self.transformWith(lambda a, b: a.fullOuterJoin(b, numPartitions), other) + + def _jtime(self, timestamp): + """ Convert datetime or unix_timestamp into Time + """ + if isinstance(timestamp, datetime): + timestamp = time.mktime(timestamp.timetuple()) + return self._sc._jvm.Time(long(timestamp * 1000)) + + def slice(self, begin, end): + """ + Return all the RDDs between 'begin' to 'end' (both included) + + `begin`, `end` could be datetime.datetime() or unix_timestamp + """ + jrdds = self._jdstream.slice(self._jtime(begin), self._jtime(end)) + return [RDD(jrdd, self._sc, self._jrdd_deserializer) for jrdd in jrdds] + + def _validate_window_param(self, window, slide): + duration = self._jdstream.dstream().slideDuration().milliseconds() + if int(window * 1000) % duration != 0: + raise ValueError("windowDuration must be multiple of the slide duration (%d ms)" + % duration) + if slide and int(slide * 1000) % duration != 0: + raise ValueError("slideDuration must be multiple of the slide duration (%d ms)" + % duration) + + def window(self, windowDuration, slideDuration=None): + """ + Return a new DStream in which each RDD contains all the elements in seen in a + sliding window of time over this DStream. + + @param windowDuration: width of the window; must be a multiple of this DStream's + batching interval + @param slideDuration: sliding interval of the window (i.e., the interval after which + the new DStream will generate RDDs); must be a multiple of this + DStream's batching interval + """ + self._validate_window_param(windowDuration, slideDuration) + d = self._ssc._jduration(windowDuration) + if slideDuration is None: + return DStream(self._jdstream.window(d), self._ssc, self._jrdd_deserializer) + s = self._ssc._jduration(slideDuration) + return DStream(self._jdstream.window(d, s), self._ssc, self._jrdd_deserializer) + + def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuration): + """ + Return a new DStream in which each RDD has a single element generated by reducing all + elements in a sliding window over this DStream. + + if `invReduceFunc` is not None, the reduction is done incrementally + using the old window's reduced value : + 1. reduce the new values that entered the window (e.g., adding new counts) + 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + This is more efficient than `invReduceFunc` is None. + + @param reduceFunc: associative reduce function + @param invReduceFunc: inverse reduce function of `reduceFunc` + @param windowDuration: width of the window; must be a multiple of this DStream's + batching interval + @param slideDuration: sliding interval of the window (i.e., the interval after which + the new DStream will generate RDDs); must be a multiple of this + DStream's batching interval + """ + keyed = self.map(lambda x: (1, x)) + reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc, + windowDuration, slideDuration, 1) + return reduced.map(lambda (k, v): v) + + def countByWindow(self, windowDuration, slideDuration): + """ + Return a new DStream in which each RDD has a single element generated + by counting the number of elements in a window over this DStream. + windowDuration and slideDuration are as defined in the window() operation. + + This is equivalent to window(windowDuration, slideDuration).count(), + but will be more efficient if window is large. + """ + return self.map(lambda x: 1).reduceByWindow(operator.add, operator.sub, + windowDuration, slideDuration) + + def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=None): + """ + Return a new DStream in which each RDD contains the count of distinct elements in + RDDs in a sliding window over this DStream. + + @param windowDuration: width of the window; must be a multiple of this DStream's + batching interval + @param slideDuration: sliding interval of the window (i.e., the interval after which + the new DStream will generate RDDs); must be a multiple of this + DStream's batching interval + @param numPartitions: number of partitions of each RDD in the new DStream. + """ + keyed = self.map(lambda x: (x, 1)) + counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub, + windowDuration, slideDuration, numPartitions) + return counted.filter(lambda (k, v): v > 0).count() + + def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None): + """ + Return a new DStream by applying `groupByKey` over a sliding window. + Similar to `DStream.groupByKey()`, but applies it over a sliding window. + + @param windowDuration: width of the window; must be a multiple of this DStream's + batching interval + @param slideDuration: sliding interval of the window (i.e., the interval after which + the new DStream will generate RDDs); must be a multiple of this + DStream's batching interval + @param numPartitions: Number of partitions of each RDD in the new DStream. + """ + ls = self.mapValues(lambda x: [x]) + grouped = ls.reduceByKeyAndWindow(lambda a, b: a.extend(b) or a, lambda a, b: a[len(b):], + windowDuration, slideDuration, numPartitions) + return grouped.mapValues(ResultIterable) + + def reduceByKeyAndWindow(self, func, invFunc, windowDuration, slideDuration=None, + numPartitions=None, filterFunc=None): + """ + Return a new DStream by applying incremental `reduceByKey` over a sliding window. + + The reduced value of over a new window is calculated using the old window's reduce value : + 1. reduce the new values that entered the window (e.g., adding new counts) + 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + + `invFunc` can be None, then it will reduce all the RDDs in window, could be slower + than having `invFunc`. + + @param reduceFunc: associative reduce function + @param invReduceFunc: inverse function of `reduceFunc` + @param windowDuration: width of the window; must be a multiple of this DStream's + batching interval + @param slideDuration: sliding interval of the window (i.e., the interval after which + the new DStream will generate RDDs); must be a multiple of this + DStream's batching interval + @param numPartitions: number of partitions of each RDD in the new DStream. + @param filterFunc: function to filter expired key-value pairs; + only pairs that satisfy the function are retained + set this to null if you do not want to filter + """ + self._validate_window_param(windowDuration, slideDuration) + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + + reduced = self.reduceByKey(func, numPartitions) + + def reduceFunc(t, a, b): + b = b.reduceByKey(func, numPartitions) + r = a.union(b).reduceByKey(func, numPartitions) if a else b + if filterFunc: + r = r.filter(filterFunc) + return r + + def invReduceFunc(t, a, b): + b = b.reduceByKey(func, numPartitions) + joined = a.leftOuterJoin(b, numPartitions) + return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1) + + jreduceFunc = TransformFunction(self._sc, reduceFunc, reduced._jrdd_deserializer) + if invReduceFunc: + jinvReduceFunc = TransformFunction(self._sc, invReduceFunc, reduced._jrdd_deserializer) + else: + jinvReduceFunc = None + if slideDuration is None: + slideDuration = self._slideDuration + dstream = self._sc._jvm.PythonReducedWindowedDStream(reduced._jdstream.dstream(), + jreduceFunc, jinvReduceFunc, + self._ssc._jduration(windowDuration), + self._ssc._jduration(slideDuration)) + return DStream(dstream.asJavaDStream(), self._ssc, self._sc.serializer) + + def updateStateByKey(self, updateFunc, numPartitions=None): + """ + Return a new "state" DStream where the state for each key is updated by applying + the given function on the previous state of the key and the new values of the key. + + @param updateFunc: State update function. If this function returns None, then + corresponding state key-value pair will be eliminated. + """ + if numPartitions is None: + numPartitions = self._sc.defaultParallelism + + def reduceFunc(t, a, b): + if a is None: + g = b.groupByKey(numPartitions).mapValues(lambda vs: (list(vs), None)) + else: + g = a.cogroup(b, numPartitions) + g = g.mapValues(lambda (va, vb): (list(vb), list(va)[0] if len(va) else None)) + state = g.mapValues(lambda (vs, s): updateFunc(vs, s)) + return state.filter(lambda (k, v): v is not None) + + jreduceFunc = TransformFunction(self._sc, reduceFunc, + self._sc.serializer, self._jrdd_deserializer) + dstream = self._sc._jvm.PythonStateDStream(self._jdstream.dstream(), jreduceFunc) + return DStream(dstream.asJavaDStream(), self._ssc, self._sc.serializer) + + +class TransformedDStream(DStream): + """ + TransformedDStream is an DStream generated by an Python function + transforming each RDD of an DStream to another RDDs. + + Multiple continuous transformations of DStream can be combined into + one transformation. + """ + def __init__(self, prev, func): + self._ssc = prev._ssc + self._sc = self._ssc._sc + self._jrdd_deserializer = self._sc.serializer + self.is_cached = False + self.is_checkpointed = False + self._jdstream_val = None + + if (isinstance(prev, TransformedDStream) and + not prev.is_cached and not prev.is_checkpointed): + prev_func = prev.func + self.func = lambda t, rdd: func(t, prev_func(t, rdd)) + self.prev = prev.prev + else: + self.prev = prev + self.func = func + + @property + def _jdstream(self): + if self._jdstream_val is not None: + return self._jdstream_val + + jfunc = TransformFunction(self._sc, self.func, self.prev._jrdd_deserializer) + dstream = self._sc._jvm.PythonTransformedDStream(self.prev._jdstream.dstream(), jfunc) + self._jdstream_val = dstream.asJavaDStream() + return self._jdstream_val diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py new file mode 100644 index 0000000000000..a8d876d0fa3b3 --- /dev/null +++ b/python/pyspark/streaming/tests.py @@ -0,0 +1,545 @@ +# +# 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. +# + +import os +from itertools import chain +import time +import operator +import unittest +import tempfile + +from pyspark.context import SparkConf, SparkContext, RDD +from pyspark.streaming.context import StreamingContext + + +class PySparkStreamingTestCase(unittest.TestCase): + + timeout = 10 # seconds + duration = 1 + + def setUp(self): + class_name = self.__class__.__name__ + conf = SparkConf().set("spark.default.parallelism", 1) + self.sc = SparkContext(appName=class_name, conf=conf) + self.sc.setCheckpointDir("/tmp") + # TODO: decrease duration to speed up tests + self.ssc = StreamingContext(self.sc, self.duration) + + def tearDown(self): + self.ssc.stop() + + def wait_for(self, result, n): + start_time = time.time() + while len(result) < n and time.time() - start_time < self.timeout: + time.sleep(0.01) + if len(result) < n: + print "timeout after", self.timeout + + def _take(self, dstream, n): + """ + Return the first `n` elements in the stream (will start and stop). + """ + results = [] + + def take(_, rdd): + if rdd and len(results) < n: + results.extend(rdd.take(n - len(results))) + + dstream.foreachRDD(take) + + self.ssc.start() + self.wait_for(results, n) + return results + + def _collect(self, dstream, n, block=True): + """ + Collect each RDDs into the returned list. + + :return: list, which will have the collected items. + """ + result = [] + + def get_output(_, rdd): + if rdd and len(result) < n: + r = rdd.collect() + if r: + result.append(r) + + dstream.foreachRDD(get_output) + + if not block: + return result + + self.ssc.start() + self.wait_for(result, n) + return result + + def _test_func(self, input, func, expected, sort=False, input2=None): + """ + @param input: dataset for the test. This should be list of lists. + @param func: wrapped function. This function should return PythonDStream object. + @param expected: expected output for this testcase. + """ + if not isinstance(input[0], RDD): + input = [self.sc.parallelize(d, 1) for d in input] + input_stream = self.ssc.queueStream(input) + if input2 and not isinstance(input2[0], RDD): + input2 = [self.sc.parallelize(d, 1) for d in input2] + input_stream2 = self.ssc.queueStream(input2) if input2 is not None else None + + # Apply test function to stream. + if input2: + stream = func(input_stream, input_stream2) + else: + stream = func(input_stream) + + result = self._collect(stream, len(expected)) + if sort: + self._sort_result_based_on_key(result) + self._sort_result_based_on_key(expected) + self.assertEqual(expected, result) + + def _sort_result_based_on_key(self, outputs): + """Sort the list based on first value.""" + for output in outputs: + output.sort(key=lambda x: x[0]) + + +class BasicOperationTests(PySparkStreamingTestCase): + + def test_map(self): + """Basic operation test for DStream.map.""" + input = [range(1, 5), range(5, 9), range(9, 13)] + + def func(dstream): + return dstream.map(str) + expected = map(lambda x: map(str, x), input) + self._test_func(input, func, expected) + + def test_flatMap(self): + """Basic operation test for DStream.faltMap.""" + input = [range(1, 5), range(5, 9), range(9, 13)] + + def func(dstream): + return dstream.flatMap(lambda x: (x, x * 2)) + expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))), + input) + self._test_func(input, func, expected) + + def test_filter(self): + """Basic operation test for DStream.filter.""" + input = [range(1, 5), range(5, 9), range(9, 13)] + + def func(dstream): + return dstream.filter(lambda x: x % 2 == 0) + expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input) + self._test_func(input, func, expected) + + def test_count(self): + """Basic operation test for DStream.count.""" + input = [range(5), range(10), range(20)] + + def func(dstream): + return dstream.count() + expected = map(lambda x: [len(x)], input) + self._test_func(input, func, expected) + + def test_reduce(self): + """Basic operation test for DStream.reduce.""" + input = [range(1, 5), range(5, 9), range(9, 13)] + + def func(dstream): + return dstream.reduce(operator.add) + expected = map(lambda x: [reduce(operator.add, x)], input) + self._test_func(input, func, expected) + + def test_reduceByKey(self): + """Basic operation test for DStream.reduceByKey.""" + input = [[("a", 1), ("a", 1), ("b", 1), ("b", 1)], + [("", 1), ("", 1), ("", 1), ("", 1)], + [(1, 1), (1, 1), (2, 1), (2, 1), (3, 1)]] + + def func(dstream): + return dstream.reduceByKey(operator.add) + expected = [[("a", 2), ("b", 2)], [("", 4)], [(1, 2), (2, 2), (3, 1)]] + self._test_func(input, func, expected, sort=True) + + def test_mapValues(self): + """Basic operation test for DStream.mapValues.""" + input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], + [("", 4), (1, 1), (2, 2), (3, 3)], + [(1, 1), (2, 1), (3, 1), (4, 1)]] + + def func(dstream): + return dstream.mapValues(lambda x: x + 10) + expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)], + [("", 14), (1, 11), (2, 12), (3, 13)], + [(1, 11), (2, 11), (3, 11), (4, 11)]] + self._test_func(input, func, expected, sort=True) + + def test_flatMapValues(self): + """Basic operation test for DStream.flatMapValues.""" + input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], + [("", 4), (1, 1), (2, 1), (3, 1)], + [(1, 1), (2, 1), (3, 1), (4, 1)]] + + def func(dstream): + return dstream.flatMapValues(lambda x: (x, x + 10)) + expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12), + ("c", 1), ("c", 11), ("d", 1), ("d", 11)], + [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], + [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]] + self._test_func(input, func, expected) + + def test_glom(self): + """Basic operation test for DStream.glom.""" + input = [range(1, 5), range(5, 9), range(9, 13)] + rdds = [self.sc.parallelize(r, 2) for r in input] + + def func(dstream): + return dstream.glom() + expected = [[[1, 2], [3, 4]], [[5, 6], [7, 8]], [[9, 10], [11, 12]]] + self._test_func(rdds, func, expected) + + def test_mapPartitions(self): + """Basic operation test for DStream.mapPartitions.""" + input = [range(1, 5), range(5, 9), range(9, 13)] + rdds = [self.sc.parallelize(r, 2) for r in input] + + def func(dstream): + def f(iterator): + yield sum(iterator) + return dstream.mapPartitions(f) + expected = [[3, 7], [11, 15], [19, 23]] + self._test_func(rdds, func, expected) + + def test_countByValue(self): + """Basic operation test for DStream.countByValue.""" + input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]] + + def func(dstream): + return dstream.countByValue() + expected = [[4], [4], [3]] + self._test_func(input, func, expected) + + def test_groupByKey(self): + """Basic operation test for DStream.groupByKey.""" + input = [[(1, 1), (2, 1), (3, 1), (4, 1)], + [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)], + [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]] + + def func(dstream): + return dstream.groupByKey().mapValues(list) + + expected = [[(1, [1]), (2, [1]), (3, [1]), (4, [1])], + [(1, [1, 1, 1]), (2, [1, 1]), (3, [1])], + [("a", [1, 1]), ("b", [1]), ("", [1, 1, 1])]] + self._test_func(input, func, expected, sort=True) + + def test_combineByKey(self): + """Basic operation test for DStream.combineByKey.""" + input = [[(1, 1), (2, 1), (3, 1), (4, 1)], + [(1, 1), (1, 1), (1, 1), (2, 1), (2, 1), (3, 1)], + [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1), ("", 1)]] + + def func(dstream): + def add(a, b): + return a + str(b) + return dstream.combineByKey(str, add, add) + expected = [[(1, "1"), (2, "1"), (3, "1"), (4, "1")], + [(1, "111"), (2, "11"), (3, "1")], + [("a", "11"), ("b", "1"), ("", "111")]] + self._test_func(input, func, expected, sort=True) + + def test_repartition(self): + input = [range(1, 5), range(5, 9)] + rdds = [self.sc.parallelize(r, 2) for r in input] + + def func(dstream): + return dstream.repartition(1).glom() + expected = [[[1, 2, 3, 4]], [[5, 6, 7, 8]]] + self._test_func(rdds, func, expected) + + def test_union(self): + input1 = [range(3), range(5), range(6)] + input2 = [range(3, 6), range(5, 6)] + + def func(d1, d2): + return d1.union(d2) + + expected = [range(6), range(6), range(6)] + self._test_func(input1, func, expected, input2=input2) + + def test_cogroup(self): + input = [[(1, 1), (2, 1), (3, 1)], + [(1, 1), (1, 1), (1, 1), (2, 1)], + [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 1)]] + input2 = [[(1, 2)], + [(4, 1)], + [("a", 1), ("a", 1), ("b", 1), ("", 1), ("", 2)]] + + def func(d1, d2): + return d1.cogroup(d2).mapValues(lambda vs: tuple(map(list, vs))) + + expected = [[(1, ([1], [2])), (2, ([1], [])), (3, ([1], []))], + [(1, ([1, 1, 1], [])), (2, ([1], [])), (4, ([], [1]))], + [("a", ([1, 1], [1, 1])), ("b", ([1], [1])), ("", ([1, 1], [1, 2]))]] + self._test_func(input, func, expected, sort=True, input2=input2) + + def test_join(self): + input = [[('a', 1), ('b', 2)]] + input2 = [[('b', 3), ('c', 4)]] + + def func(a, b): + return a.join(b) + + expected = [[('b', (2, 3))]] + self._test_func(input, func, expected, True, input2) + + def test_left_outer_join(self): + input = [[('a', 1), ('b', 2)]] + input2 = [[('b', 3), ('c', 4)]] + + def func(a, b): + return a.leftOuterJoin(b) + + expected = [[('a', (1, None)), ('b', (2, 3))]] + self._test_func(input, func, expected, True, input2) + + def test_right_outer_join(self): + input = [[('a', 1), ('b', 2)]] + input2 = [[('b', 3), ('c', 4)]] + + def func(a, b): + return a.rightOuterJoin(b) + + expected = [[('b', (2, 3)), ('c', (None, 4))]] + self._test_func(input, func, expected, True, input2) + + def test_full_outer_join(self): + input = [[('a', 1), ('b', 2)]] + input2 = [[('b', 3), ('c', 4)]] + + def func(a, b): + return a.fullOuterJoin(b) + + expected = [[('a', (1, None)), ('b', (2, 3)), ('c', (None, 4))]] + self._test_func(input, func, expected, True, input2) + + def test_update_state_by_key(self): + + def updater(vs, s): + if not s: + s = [] + s.extend(vs) + return s + + input = [[('k', i)] for i in range(5)] + + def func(dstream): + return dstream.updateStateByKey(updater) + + expected = [[0], [0, 1], [0, 1, 2], [0, 1, 2, 3], [0, 1, 2, 3, 4]] + expected = [[('k', v)] for v in expected] + self._test_func(input, func, expected) + + +class WindowFunctionTests(PySparkStreamingTestCase): + + timeout = 20 + + def test_window(self): + input = [range(1), range(2), range(3), range(4), range(5)] + + def func(dstream): + return dstream.window(3, 1).count() + + expected = [[1], [3], [6], [9], [12], [9], [5]] + self._test_func(input, func, expected) + + def test_count_by_window(self): + input = [range(1), range(2), range(3), range(4), range(5)] + + def func(dstream): + return dstream.countByWindow(3, 1) + + expected = [[1], [3], [6], [9], [12], [9], [5]] + self._test_func(input, func, expected) + + def test_count_by_window_large(self): + input = [range(1), range(2), range(3), range(4), range(5), range(6)] + + def func(dstream): + return dstream.countByWindow(5, 1) + + expected = [[1], [3], [6], [10], [15], [20], [18], [15], [11], [6]] + self._test_func(input, func, expected) + + def test_count_by_value_and_window(self): + input = [range(1), range(2), range(3), range(4), range(5), range(6)] + + def func(dstream): + return dstream.countByValueAndWindow(5, 1) + + expected = [[1], [2], [3], [4], [5], [6], [6], [6], [6], [6]] + self._test_func(input, func, expected) + + def test_group_by_key_and_window(self): + input = [[('a', i)] for i in range(5)] + + def func(dstream): + return dstream.groupByKeyAndWindow(3, 1).mapValues(list) + + expected = [[('a', [0])], [('a', [0, 1])], [('a', [0, 1, 2])], [('a', [1, 2, 3])], + [('a', [2, 3, 4])], [('a', [3, 4])], [('a', [4])]] + self._test_func(input, func, expected) + + def test_reduce_by_invalid_window(self): + input1 = [range(3), range(5), range(1), range(6)] + d1 = self.ssc.queueStream(input1) + self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 0.1, 0.1)) + self.assertRaises(ValueError, lambda: d1.reduceByKeyAndWindow(None, None, 1, 0.1)) + + +class StreamingContextTests(PySparkStreamingTestCase): + + duration = 0.1 + + def _add_input_stream(self): + inputs = map(lambda x: range(1, x), range(101)) + stream = self.ssc.queueStream(inputs) + self._collect(stream, 1, block=False) + + def test_stop_only_streaming_context(self): + self._add_input_stream() + self.ssc.start() + self.ssc.stop(False) + self.assertEqual(len(self.sc.parallelize(range(5), 5).glom().collect()), 5) + + def test_stop_multiple_times(self): + self._add_input_stream() + self.ssc.start() + self.ssc.stop() + self.ssc.stop() + + def test_queue_stream(self): + input = [range(i + 1) for i in range(3)] + dstream = self.ssc.queueStream(input) + result = self._collect(dstream, 3) + self.assertEqual(input, result) + + def test_text_file_stream(self): + d = tempfile.mkdtemp() + self.ssc = StreamingContext(self.sc, self.duration) + dstream2 = self.ssc.textFileStream(d).map(int) + result = self._collect(dstream2, 2, block=False) + self.ssc.start() + for name in ('a', 'b'): + time.sleep(1) + with open(os.path.join(d, name), "w") as f: + f.writelines(["%d\n" % i for i in range(10)]) + self.wait_for(result, 2) + self.assertEqual([range(10), range(10)], result) + + def test_union(self): + input = [range(i + 1) for i in range(3)] + dstream = self.ssc.queueStream(input) + dstream2 = self.ssc.queueStream(input) + dstream3 = self.ssc.union(dstream, dstream2) + result = self._collect(dstream3, 3) + expected = [i * 2 for i in input] + self.assertEqual(expected, result) + + def test_transform(self): + dstream1 = self.ssc.queueStream([[1]]) + dstream2 = self.ssc.queueStream([[2]]) + dstream3 = self.ssc.queueStream([[3]]) + + def func(rdds): + rdd1, rdd2, rdd3 = rdds + return rdd2.union(rdd3).union(rdd1) + + dstream = self.ssc.transform([dstream1, dstream2, dstream3], func) + + self.assertEqual([2, 3, 1], self._take(dstream, 3)) + + +class CheckpointTests(PySparkStreamingTestCase): + + def setUp(self): + pass + + def test_get_or_create(self): + inputd = tempfile.mkdtemp() + outputd = tempfile.mkdtemp() + "/" + + def updater(vs, s): + return sum(vs, s or 0) + + def setup(): + conf = SparkConf().set("spark.default.parallelism", 1) + sc = SparkContext(conf=conf) + ssc = StreamingContext(sc, 0.5) + dstream = ssc.textFileStream(inputd).map(lambda x: (x, 1)) + wc = dstream.updateStateByKey(updater) + wc.map(lambda x: "%s,%d" % x).saveAsTextFiles(outputd + "test") + wc.checkpoint(.5) + return ssc + + cpd = tempfile.mkdtemp("test_streaming_cps") + self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup) + ssc.start() + + def check_output(n): + while not os.listdir(outputd): + time.sleep(0.1) + time.sleep(1) # make sure mtime is larger than the previous one + with open(os.path.join(inputd, str(n)), 'w') as f: + f.writelines(["%d\n" % i for i in range(10)]) + + while True: + p = os.path.join(outputd, max(os.listdir(outputd))) + if '_SUCCESS' not in os.listdir(p): + # not finished + time.sleep(0.01) + continue + ordd = ssc.sparkContext.textFile(p).map(lambda line: line.split(",")) + d = ordd.values().map(int).collect() + if not d: + time.sleep(0.01) + continue + self.assertEqual(10, len(d)) + s = set(d) + self.assertEqual(1, len(s)) + m = s.pop() + if n > m: + continue + self.assertEqual(n, m) + break + + check_output(1) + check_output(2) + ssc.stop(True, True) + + time.sleep(1) + self.ssc = ssc = StreamingContext.getOrCreate(cpd, setup) + ssc.start() + check_output(3) + + +if __name__ == "__main__": + unittest.main() diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py new file mode 100644 index 0000000000000..86ee5aa04f252 --- /dev/null +++ b/python/pyspark/streaming/util.py @@ -0,0 +1,128 @@ +# +# 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. +# + +import time +from datetime import datetime +import traceback + +from pyspark import SparkContext, RDD + + +class TransformFunction(object): + """ + This class wraps a function RDD[X] -> RDD[Y] that was passed to + DStream.transform(), allowing it to be called from Java via Py4J's + callback server. + + Java calls this function with a sequence of JavaRDDs and this function + returns a single JavaRDD pointer back to Java. + """ + _emptyRDD = None + + def __init__(self, ctx, func, *deserializers): + self.ctx = ctx + self.func = func + self.deserializers = deserializers + + def call(self, milliseconds, jrdds): + try: + if self.ctx is None: + self.ctx = SparkContext._active_spark_context + if not self.ctx or not self.ctx._jsc: + # stopped + return + + # extend deserializers with the first one + sers = self.deserializers + if len(sers) < len(jrdds): + sers += (sers[0],) * (len(jrdds) - len(sers)) + + rdds = [RDD(jrdd, self.ctx, ser) if jrdd else None + for jrdd, ser in zip(jrdds, sers)] + t = datetime.fromtimestamp(milliseconds / 1000.0) + r = self.func(t, *rdds) + if r: + return r._jrdd + except Exception: + traceback.print_exc() + + def __repr__(self): + return "TransformFunction(%s)" % self.func + + class Java: + implements = ['org.apache.spark.streaming.api.python.PythonTransformFunction'] + + +class TransformFunctionSerializer(object): + """ + This class implements a serializer for PythonTransformFunction Java + objects. + + This is necessary because the Java PythonTransformFunction objects are + actually Py4J references to Python objects and thus are not directly + serializable. When Java needs to serialize a PythonTransformFunction, + it uses this class to invoke Python, which returns the serialized function + as a byte array. + """ + def __init__(self, ctx, serializer, gateway=None): + self.ctx = ctx + self.serializer = serializer + self.gateway = gateway or self.ctx._gateway + self.gateway.jvm.PythonDStream.registerSerializer(self) + + def dumps(self, id): + try: + func = self.gateway.gateway_property.pool[id] + return bytearray(self.serializer.dumps((func.func, func.deserializers))) + except Exception: + traceback.print_exc() + + def loads(self, bytes): + try: + f, deserializers = self.serializer.loads(str(bytes)) + return TransformFunction(self.ctx, f, *deserializers) + except Exception: + traceback.print_exc() + + def __repr__(self): + return "TransformFunctionSerializer(%s)" % self.serializer + + class Java: + implements = ['org.apache.spark.streaming.api.python.PythonTransformFunctionSerializer'] + + +def rddToFileName(prefix, suffix, timestamp): + """ + Return string prefix-time(.suffix) + + >>> rddToFileName("spark", None, 12345678910) + 'spark-12345678910' + >>> rddToFileName("spark", "tmp", 12345678910) + 'spark-12345678910.tmp' + """ + if isinstance(timestamp, datetime): + seconds = time.mktime(timestamp.timetuple()) + timestamp = long(seconds * 1000) + timestamp.microsecond / 1000 + if suffix is None: + return prefix + "-" + str(timestamp) + else: + return prefix + "-" + str(timestamp) + "." + suffix + + +if __name__ == "__main__": + import doctest + doctest.testmod() diff --git a/python/run-tests b/python/run-tests index f6a96841175e8..2f98443c30aef 100755 --- a/python/run-tests +++ b/python/run-tests @@ -81,6 +81,11 @@ function run_mllib_tests() { run_test "pyspark/mllib/tests.py" } +function run_streaming_tests() { + run_test "pyspark/streaming/util.py" + run_test "pyspark/streaming/tests.py" +} + echo "Running PySpark tests. Output is in python/unit-tests.log." export PYSPARK_PYTHON="python" @@ -96,6 +101,7 @@ $PYSPARK_PYTHON --version run_core_tests run_sql_tests run_mllib_tests +run_streaming_tests # Try to test with PyPy if [ $(which pypy) ]; then @@ -105,6 +111,7 @@ if [ $(which pypy) ]; then run_core_tests run_sql_tests + run_streaming_tests fi if [[ $FAILED == 0 ]]; then diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index a6184de4e83c1..2a7004e56ef53 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -167,7 +167,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T new JavaPairDStream(dstream.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } - /** + /** * Return a new DStream in which each RDD is generated by applying mapPartitions() to each RDDs * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition * of the RDD. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala new file mode 100644 index 0000000000000..213dff6a76354 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -0,0 +1,316 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.api.python + +import java.io.{ObjectInputStream, ObjectOutputStream} +import java.lang.reflect.Proxy +import java.util.{ArrayList => JArrayList, List => JList} +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.language.existentials + +import py4j.GatewayServer + +import org.apache.spark.api.java._ +import org.apache.spark.api.python._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Interval, Duration, Time} +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.api.java._ + + +/** + * Interface for Python callback function which is used to transform RDDs + */ +private[python] trait PythonTransformFunction { + def call(time: Long, rdds: JList[_]): JavaRDD[Array[Byte]] +} + +/** + * Interface for Python Serializer to serialize PythonTransformFunction + */ +private[python] trait PythonTransformFunctionSerializer { + def dumps(id: String): Array[Byte] + def loads(bytes: Array[Byte]): PythonTransformFunction +} + +/** + * Wraps a PythonTransformFunction (which is a Python object accessed through Py4J) + * so that it looks like a Scala function and can be transparently serialized and + * deserialized by Java. + */ +private[python] class TransformFunction(@transient var pfunc: PythonTransformFunction) + extends function.Function2[JList[JavaRDD[_]], Time, JavaRDD[Array[Byte]]] { + + def apply(rdd: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = { + Option(pfunc.call(time.milliseconds, List(rdd.map(JavaRDD.fromRDD(_)).orNull).asJava)) + .map(_.rdd) + } + + def apply(rdd: Option[RDD[_]], rdd2: Option[RDD[_]], time: Time): Option[RDD[Array[Byte]]] = { + val rdds = List(rdd.map(JavaRDD.fromRDD(_)).orNull, rdd2.map(JavaRDD.fromRDD(_)).orNull).asJava + Option(pfunc.call(time.milliseconds, rdds)).map(_.rdd) + } + + // for function.Function2 + def call(rdds: JList[JavaRDD[_]], time: Time): JavaRDD[Array[Byte]] = { + pfunc.call(time.milliseconds, rdds) + } + + private def writeObject(out: ObjectOutputStream): Unit = { + val bytes = PythonTransformFunctionSerializer.serialize(pfunc) + out.writeInt(bytes.length) + out.write(bytes) + } + + private def readObject(in: ObjectInputStream): Unit = { + val length = in.readInt() + val bytes = new Array[Byte](length) + in.readFully(bytes) + pfunc = PythonTransformFunctionSerializer.deserialize(bytes) + } +} + +/** + * Helpers for PythonTransformFunctionSerializer + * + * PythonTransformFunctionSerializer is logically a singleton that's happens to be + * implemented as a Python object. + */ +private[python] object PythonTransformFunctionSerializer { + + /** + * A serializer in Python, used to serialize PythonTransformFunction + */ + private var serializer: PythonTransformFunctionSerializer = _ + + /* + * Register a serializer from Python, should be called during initialization + */ + def register(ser: PythonTransformFunctionSerializer): Unit = { + serializer = ser + } + + def serialize(func: PythonTransformFunction): Array[Byte] = { + assert(serializer != null, "Serializer has not been registered!") + // get the id of PythonTransformFunction in py4j + val h = Proxy.getInvocationHandler(func.asInstanceOf[Proxy]) + val f = h.getClass().getDeclaredField("id") + f.setAccessible(true) + val id = f.get(h).asInstanceOf[String] + serializer.dumps(id) + } + + def deserialize(bytes: Array[Byte]): PythonTransformFunction = { + assert(serializer != null, "Serializer has not been registered!") + serializer.loads(bytes) + } +} + +/** + * Helper functions, which are called from Python via Py4J. + */ +private[python] object PythonDStream { + + /** + * can not access PythonTransformFunctionSerializer.register() via Py4j + * Py4JError: PythonTransformFunctionSerializerregister does not exist in the JVM + */ + def registerSerializer(ser: PythonTransformFunctionSerializer): Unit = { + PythonTransformFunctionSerializer.register(ser) + } + + /** + * Update the port of callback client to `port` + */ + def updatePythonGatewayPort(gws: GatewayServer, port: Int): Unit = { + val cl = gws.getCallbackClient + val f = cl.getClass.getDeclaredField("port") + f.setAccessible(true) + f.setInt(cl, port) + } + + /** + * helper function for DStream.foreachRDD(), + * cannot be `foreachRDD`, it will confusing py4j + */ + def callForeachRDD(jdstream: JavaDStream[Array[Byte]], pfunc: PythonTransformFunction) { + val func = new TransformFunction((pfunc)) + jdstream.dstream.foreachRDD((rdd, time) => func(Some(rdd), time)) + } + + /** + * convert list of RDD into queue of RDDs, for ssc.queueStream() + */ + def toRDDQueue(rdds: JArrayList[JavaRDD[Array[Byte]]]): java.util.Queue[JavaRDD[Array[Byte]]] = { + val queue = new java.util.LinkedList[JavaRDD[Array[Byte]]] + rdds.forall(queue.add(_)) + queue + } +} + +/** + * Base class for PythonDStream with some common methods + */ +private[python] abstract class PythonDStream( + parent: DStream[_], + @transient pfunc: PythonTransformFunction) + extends DStream[Array[Byte]] (parent.ssc) { + + val func = new TransformFunction(pfunc) + + override def dependencies = List(parent) + + override def slideDuration: Duration = parent.slideDuration + + val asJavaDStream = JavaDStream.fromDStream(this) +} + +/** + * Transformed DStream in Python. + */ +private[python] class PythonTransformedDStream ( + parent: DStream[_], + @transient pfunc: PythonTransformFunction) + extends PythonDStream(parent, pfunc) { + + override def compute(validTime: Time): Option[RDD[Array[Byte]]] = { + val rdd = parent.getOrCompute(validTime) + if (rdd.isDefined) { + func(rdd, validTime) + } else { + None + } + } +} + +/** + * Transformed from two DStreams in Python. + */ +private[python] class PythonTransformed2DStream( + parent: DStream[_], + parent2: DStream[_], + @transient pfunc: PythonTransformFunction) + extends DStream[Array[Byte]] (parent.ssc) { + + val func = new TransformFunction(pfunc) + + override def dependencies = List(parent, parent2) + + override def slideDuration: Duration = parent.slideDuration + + override def compute(validTime: Time): Option[RDD[Array[Byte]]] = { + val empty: RDD[_] = ssc.sparkContext.emptyRDD + val rdd1 = parent.getOrCompute(validTime).getOrElse(empty) + val rdd2 = parent2.getOrCompute(validTime).getOrElse(empty) + func(Some(rdd1), Some(rdd2), validTime) + } + + val asJavaDStream = JavaDStream.fromDStream(this) +} + +/** + * similar to StateDStream + */ +private[python] class PythonStateDStream( + parent: DStream[Array[Byte]], + @transient reduceFunc: PythonTransformFunction) + extends PythonDStream(parent, reduceFunc) { + + super.persist(StorageLevel.MEMORY_ONLY) + override val mustCheckpoint = true + + override def compute(validTime: Time): Option[RDD[Array[Byte]]] = { + val lastState = getOrCompute(validTime - slideDuration) + val rdd = parent.getOrCompute(validTime) + if (rdd.isDefined) { + func(lastState, rdd, validTime) + } else { + lastState + } + } +} + +/** + * similar to ReducedWindowedDStream + */ +private[python] class PythonReducedWindowedDStream( + parent: DStream[Array[Byte]], + @transient preduceFunc: PythonTransformFunction, + @transient pinvReduceFunc: PythonTransformFunction, + _windowDuration: Duration, + _slideDuration: Duration) + extends PythonDStream(parent, preduceFunc) { + + super.persist(StorageLevel.MEMORY_ONLY) + override val mustCheckpoint = true + + val invReduceFunc = new TransformFunction(pinvReduceFunc) + + def windowDuration: Duration = _windowDuration + override def slideDuration: Duration = _slideDuration + override def parentRememberDuration: Duration = rememberDuration + windowDuration + + override def compute(validTime: Time): Option[RDD[Array[Byte]]] = { + val currentTime = validTime + val current = new Interval(currentTime - windowDuration, currentTime) + val previous = current - slideDuration + + // _____________________________ + // | previous window _________|___________________ + // |___________________| current window | --------------> Time + // |_____________________________| + // + // |________ _________| |________ _________| + // | | + // V V + // old RDDs new RDDs + // + val previousRDD = getOrCompute(previous.endTime) + + // for small window, reduce once will be better than twice + if (pinvReduceFunc != null && previousRDD.isDefined + && windowDuration >= slideDuration * 5) { + + // subtract the values from old RDDs + val oldRDDs = parent.slice(previous.beginTime + parent.slideDuration, current.beginTime) + val subtracted = if (oldRDDs.size > 0) { + invReduceFunc(previousRDD, Some(ssc.sc.union(oldRDDs)), validTime) + } else { + previousRDD + } + + // add the RDDs of the reduced values in "new time steps" + val newRDDs = parent.slice(previous.endTime + parent.slideDuration, current.endTime) + if (newRDDs.size > 0) { + func(subtracted, Some(ssc.sc.union(newRDDs)), validTime) + } else { + subtracted + } + } else { + // Get the RDDs of the reduced values in current window + val currentRDDs = parent.slice(current.beginTime + parent.slideDuration, current.endTime) + if (currentRDDs.size > 0) { + func(None, Some(ssc.sc.union(currentRDDs)), validTime) + } else { + None + } + } + } +} From 18bd67c24b081f113b34455692451571c466df92 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 12 Oct 2014 13:08:42 -0700 Subject: [PATCH 0968/1492] [SPARK-3887] Send stracktrace in ConnectionManager error replies When reporting that a remote error occurred, the ConnectionManager should also log the stacktrace of the remote exception. This PR accomplishes this by sending the remote exception's stacktrace as the payload in the "negative ACK / error message." Author: Josh Rosen Closes #2741 from JoshRosen/propagate-cm-exceptions-to-sender and squashes the following commits: b5366cc [Josh Rosen] Explicitly encode error messages using UTF-8. cef18b3 [Josh Rosen] [SPARK-3887] Send stracktrace in ConnectionManager error messages. --- .../spark/network/nio/ConnectionManager.scala | 10 ++++++---- .../org/apache/spark/network/nio/Message.scala | 14 ++++++++++++++ .../network/nio/NioBlockTransferService.scala | 11 ++++------- .../spark/network/nio/ConnectionManagerSuite.scala | 6 ++++-- 4 files changed, 28 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 6b00190c5eccc..9396b6ba84e7e 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -748,9 +748,7 @@ private[nio] class ConnectionManager( } catch { case e: Exception => { logError(s"Exception was thrown while processing message", e) - val m = Message.createBufferMessage(bufferMessage.id) - m.hasError = true - ackMessage = Some(m) + ackMessage = Some(Message.createErrorMessage(e, bufferMessage.id)) } } finally { sendMessage(connectionManagerId, ackMessage.getOrElse { @@ -913,8 +911,12 @@ private[nio] class ConnectionManager( } case scala.util.Success(ackMessage) => if (ackMessage.hasError) { + val errorMsgByteBuf = ackMessage.asInstanceOf[BufferMessage].buffers.head + val errorMsgBytes = new Array[Byte](errorMsgByteBuf.limit()) + errorMsgByteBuf.get(errorMsgBytes) + val errorMsg = new String(errorMsgBytes, "utf-8") val e = new IOException( - "sendMessageReliably failed with ACK that signalled a remote error") + s"sendMessageReliably failed with ACK that signalled a remote error: $errorMsg") if (!promise.tryFailure(e)) { logWarning("Ignore error because promise is completed", e) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala index 0b874c2891255..3ad04591da658 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import org.apache.spark.util.Utils private[nio] abstract class Message(val typ: Long, val id: Int) { var senderAddress: InetSocketAddress = null @@ -84,6 +85,19 @@ private[nio] object Message { createBufferMessage(new Array[ByteBuffer](0), ackId) } + /** + * Create a "negative acknowledgment" to notify a sender that an error occurred + * while processing its message. The exception's stacktrace will be formatted + * as a string, serialized into a byte array, and sent as the message payload. + */ + def createErrorMessage(exception: Exception, ackId: Int): BufferMessage = { + val exceptionString = Utils.exceptionString(exception) + val serializedExceptionString = ByteBuffer.wrap(exceptionString.getBytes("utf-8")) + val errorMessage = createBufferMessage(serializedExceptionString, ackId) + errorMessage.hasError = true + errorMessage + } + def create(header: MessageChunkHeader): Message = { val newMessage: Message = header.typ match { case BUFFER_MESSAGE => new BufferMessage(header.id, diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index b389b9a2022c6..5add4fc433fb3 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -151,17 +151,14 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa } catch { case e: Exception => { logError("Exception handling buffer message", e) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) + Some(Message.createErrorMessage(e, msg.id)) } } case otherMessage: Any => - logError("Unknown type message received: " + otherMessage) - val errorMessage = Message.createBufferMessage(msg.id) - errorMessage.hasError = true - Some(errorMessage) + val errorMsg = s"Received unknown message type: ${otherMessage.getClass.getName}" + logError(errorMsg) + Some(Message.createErrorMessage(new UnsupportedOperationException(errorMsg), msg.id)) } } diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index 9f49587cdc670..b70734dfe37cf 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -27,6 +27,7 @@ import scala.language.postfixOps import org.scalatest.FunSuite import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.util.Utils /** * Test the ConnectionManager with various security settings. @@ -236,7 +237,7 @@ class ConnectionManagerSuite extends FunSuite { val manager = new ConnectionManager(0, conf, securityManager) val managerServer = new ConnectionManager(0, conf, securityManager) managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - throw new Exception + throw new Exception("Custom exception text") }) val size = 10 * 1024 * 1024 @@ -246,9 +247,10 @@ class ConnectionManagerSuite extends FunSuite { val future = manager.sendMessageReliably(managerServer.id, bufferMessage) - intercept[IOException] { + val exception = intercept[IOException] { Await.result(future, 1 second) } + assert(Utils.exceptionString(exception).contains("Custom exception text")) manager.stop() managerServer.stop() From e5be4de7bcf5aa7afc856fc665427ff2b22a0fcd Mon Sep 17 00:00:00 2001 From: NamelessAnalyst Date: Sun, 12 Oct 2014 14:18:55 -0700 Subject: [PATCH 0969/1492] SPARK-3716 [GraphX] Update Analytics.scala for partitionStrategy assignment Previously, when the val partitionStrategy was created it called a function in the Analytics object which was a copy of the PartitionStrategy.fromString() method. This function has been removed, and the assignment of partitionStrategy now uses the PartitionStrategy.fromString method instead. In this way, it better matches the declarations of edge/vertex StorageLevel variables. Author: NamelessAnalyst Closes #2569 from NamelessAnalyst/branch-1.1 and squashes the following commits: c24ff51 [NamelessAnalyst] Update Analytics.scala (cherry picked from commit 5a21e3e7e97f135c81c664098a723434b910f09d) Signed-off-by: Ankur Dave --- .../spark/examples/graphx/Analytics.scala | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index c4317a6aec798..45527d9382fd0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -46,17 +46,6 @@ object Analytics extends Logging { } val options = mutable.Map(optionsList: _*) - def pickPartitioner(v: String): PartitionStrategy = { - // TODO: Use reflection rather than listing all the partitioning strategies here. - v match { - case "RandomVertexCut" => RandomVertexCut - case "EdgePartition1D" => EdgePartition1D - case "EdgePartition2D" => EdgePartition2D - case "CanonicalRandomVertexCut" => CanonicalRandomVertexCut - case _ => throw new IllegalArgumentException("Invalid PartitionStrategy: " + v) - } - } - val conf = new SparkConf() .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") @@ -67,7 +56,7 @@ object Analytics extends Logging { sys.exit(1) } val partitionStrategy: Option[PartitionStrategy] = options.remove("partStrategy") - .map(pickPartitioner(_)) + .map(PartitionStrategy.fromString(_)) val edgeStorageLevel = options.remove("edgeStorageLevel") .map(StorageLevel.fromString(_)).getOrElse(StorageLevel.MEMORY_ONLY) val vertexStorageLevel = options.remove("vertexStorageLevel") @@ -107,7 +96,7 @@ object Analytics extends Logging { if (!outFname.isEmpty) { logWarning("Saving pageranks of pages to " + outFname) - pr.map{case (id, r) => id + "\t" + r}.saveAsTextFile(outFname) + pr.map { case (id, r) => id + "\t" + r }.saveAsTextFile(outFname) } sc.stop() @@ -129,7 +118,7 @@ object Analytics extends Logging { val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) val cc = ConnectedComponents.run(graph) - println("Components: " + cc.vertices.map{ case (vid,data) => data}.distinct()) + println("Components: " + cc.vertices.map { case (vid, data) => data }.distinct()) sc.stop() case "triangles" => @@ -147,7 +136,7 @@ object Analytics extends Logging { minEdgePartitions = numEPart, edgeStorageLevel = edgeStorageLevel, vertexStorageLevel = vertexStorageLevel) - // TriangleCount requires the graph to be partitioned + // TriangleCount requires the graph to be partitioned .partitionBy(partitionStrategy.getOrElse(RandomVertexCut)).cache() val triangles = TriangleCount.run(graph) println("Triangles: " + triangles.vertices.map { From c86c9760374f331ab7ed173b0a022250635485d3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sun, 12 Oct 2014 15:41:27 -0700 Subject: [PATCH 0970/1492] [HOTFIX] Fix compilation error for Yarn 2.0.*-alpha This was reported in https://issues.apache.org/jira/browse/SPARK-3445. There are API differences between the 0.23.* and the 2.0.*-alpha branches that are not accounted for when this code was introduced. Author: Andrew Or Closes #2776 from andrewor14/fix-yarn-alpha and squashes the following commits: ec94752 [Andrew Or] Fix compilation error for 2.0.*-alpha --- .../src/main/scala/org/apache/spark/deploy/yarn/Client.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 5a20532315e59..5c7bca4541222 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -122,7 +122,7 @@ private[spark] class Client( * ApplicationReport#getClientToken is renamed `getClientToAMToken` in the stable API. */ override def getClientToken(report: ApplicationReport): String = - Option(report.getClientToken).getOrElse("") + Option(report.getClientToken).map(_.toString).getOrElse("") } object Client { From fc616d51a510f82627b5be949a5941419834cf70 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jakub=20Dubovsk=C3=BD?= Date: Sun, 12 Oct 2014 22:03:26 -0700 Subject: [PATCH 0971/1492] [SPARK-3121] Wrong implementation of implicit bytesWritableConverter MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit val path = ... //path to seq file with BytesWritable as type of both key and value val file = sc.sequenceFile[Array[Byte],Array[Byte]](path) file.take(1)(0)._1 This prints incorrect content of byte array. Actual content starts with correct one and some "random" bytes and zeros are appended. BytesWritable has two methods: getBytes() - return content of all internal array which is often longer then actual value stored. It usually contains the rest of previous longer values copyBytes() - return just begining of internal array determined by internal length property It looks like in implicit conversion between BytesWritable and Array[byte] getBytes is used instead of correct copyBytes. dbtsai Author: Jakub Dubovský Author: Dubovsky Jakub Closes #2712 from james64/3121-bugfix and squashes the following commits: f85d24c [Jakub Dubovský] Test name changed, comments added 1b20d51 [Jakub Dubovský] Import placed correctly 406e26c [Jakub Dubovský] Scala style fixed f92ffa6 [Dubovsky Jakub] performance tuning 480f9cd [Dubovsky Jakub] Bug 3121 fixed --- .../scala/org/apache/spark/SparkContext.scala | 6 ++- .../org/apache/spark/SparkContextSuite.scala | 40 +++++++++++++++++++ 2 files changed, 45 insertions(+), 1 deletion(-) create mode 100644 core/src/test/scala/org/apache/spark/SparkContextSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 396cdd1247e07..b709b8880ba76 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -21,6 +21,7 @@ import scala.language.implicitConversions import java.io._ import java.net.URI +import java.util.Arrays import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} import java.util.UUID.randomUUID @@ -1429,7 +1430,10 @@ object SparkContext extends Logging { simpleWritableConverter[Boolean, BooleanWritable](_.get) implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { - simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes) + simpleWritableConverter[Array[Byte], BytesWritable](bw => + // getBytes method returns array which is longer then data to be returned + Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) + ) } implicit def stringWritableConverter(): WritableConverter[String] = diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala new file mode 100644 index 0000000000000..31edad1c56c73 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.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 + +import org.scalatest.FunSuite + +import org.apache.hadoop.io.BytesWritable + +class SparkContextSuite extends FunSuite { + //Regression test for SPARK-3121 + test("BytesWritable implicit conversion is correct") { + val bytesWritable = new BytesWritable() + val inputArray = (1 to 10).map(_.toByte).toArray + bytesWritable.set(inputArray, 0, 10) + bytesWritable.set(inputArray, 0, 5) + + val converter = SparkContext.bytesWritableConverter() + val byteArray = converter.convert(bytesWritable) + assert(byteArray.length === 5) + + bytesWritable.set(inputArray, 0, 0) + val byteArray2 = converter.convert(bytesWritable) + assert(byteArray2.length === 0) + } +} From b4a7fa7a663c462bf537ca9d63af0dba6b4a8033 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sun, 12 Oct 2014 22:48:54 -0700 Subject: [PATCH 0972/1492] [SPARK-3905][Web UI]The keys for sorting the columns of Executor page ,Stage page Storage page are incorrect Author: GuoQiang Li Closes #2763 from witgo/SPARK-3905 and squashes the following commits: 17d7990 [GuoQiang Li] The keys for sorting the columns of Executor page ,Stage page Storage page are incorrect --- .../org/apache/spark/ui/jobs/ExecutorTable.scala | 12 ++++++------ .../scala/org/apache/spark/ui/jobs/StageTable.scala | 6 +++--- .../org/apache/spark/ui/storage/StoragePage.scala | 6 +++--- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 2987dc04494a5..f0e43fbf70976 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -71,19 +71,19 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobPr - + - - - - - } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2e67310594784..4ee7f08ab47a2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -176,9 +176,9 @@ private[ui] class StageTableBase( {makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, stageData.numFailedTasks, s.numTasks)} - - - + + + } /** Render an HTML row that represents a stage */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 716591c9ed449..83489ca0679ee 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -58,9 +58,9 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { - - - + + + // scalastyle:on } From d8b8c210786dfb905d06ea0a21d633f7772d5d1a Mon Sep 17 00:00:00 2001 From: Ken Takagiwa Date: Sun, 12 Oct 2014 23:05:14 -0700 Subject: [PATCH 0973/1492] Add echo "Run streaming tests ..." Author: Ken Takagiwa Closes #2778 from giwa/patch-2 and squashes the following commits: a59f9a1 [Ken Takagiwa] Add echo "Run streaming tests ..." --- python/run-tests | 1 + 1 file changed, 1 insertion(+) diff --git a/python/run-tests b/python/run-tests index 2f98443c30aef..80acd002ab7eb 100755 --- a/python/run-tests +++ b/python/run-tests @@ -82,6 +82,7 @@ function run_mllib_tests() { } function run_streaming_tests() { + echo "Run streaming tests ..." run_test "pyspark/streaming/util.py" run_test "pyspark/streaming/tests.py" } From 92e017fb894be1e8e2b2b5274fec4c31a7a4412e Mon Sep 17 00:00:00 2001 From: w00228970 Date: Sun, 12 Oct 2014 23:35:50 -0700 Subject: [PATCH 0974/1492] [SPARK-3899][Doc]fix wrong links in streaming doc There are three [Custom Receiver Guide] links in streaming doc, the first is wrong. Author: w00228970 Author: wangfei Closes #2749 from scwf/streaming-doc and squashes the following commits: 0cd76b7 [wangfei] update link tojump to the Akka-specific section 45b0646 [w00228970] wrong link in streaming doc --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 5c21e912ea160..738309c668387 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -494,7 +494,7 @@ methods for creating DStreams from files and Akka actors as input sources. For simple text files, there is an easier method `streamingContext.textFileStream(dataDirectory)`. And file streams do not require running a receiver, hence does not require allocating cores. -- **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver Guide](#implementing-and-using-a-custom-actor-based-receiver) for more details. +- **Streams based on Custom Actors:** DStreams can be created with data streams received through Akka actors by using `streamingContext.actorStream(actorProps, actor-name)`. See the [Custom Receiver Guide](streaming-custom-receivers.html#implementing-and-using-a-custom-actor-based-receiver) for more details. - **Queue of RDDs as a Stream:** For testing a Spark Streaming application with test data, one can also create a DStream based on a queue of RDDs, using `streamingContext.queueStream(queueOfRDDs)`. Each RDD pushed into the queue will be treated as a batch of data in the DStream, and processed like a stream. From 942847fd94c920f7954ddf01f97263926e512b0e Mon Sep 17 00:00:00 2001 From: omgteam Date: Mon, 13 Oct 2014 09:59:41 -0700 Subject: [PATCH 0975/1492] Bug Fix: without unpersist method in RandomForest.scala During trainning Gradient Boosting Decision Tree on large-scale sparse data, spark spill hundreds of data onto disk. And find the bug below: In version 1.1.0 DecisionTree.scala, train Method, treeInput has been persisted in Memory, but without unpersist. It caused heavy DISK usage. In github version(1.2.0 maybe), RandomForest.scala, train Method, baggedInput has been persisted but without unpersisted too. After added unpersist, it works right. https://issues.apache.org/jira/browse/SPARK-3918 Author: omgteam Closes #2775 from omgteam/master and squashes the following commits: 815d543 [omgteam] adjust tab to spaces 1a36f83 [omgteam] Bug: fix without unpersist baggedInput in RandomForest.scala --- .../main/scala/org/apache/spark/mllib/tree/RandomForest.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index fa7a26f17c3ca..ebbd8e0257209 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -176,6 +176,8 @@ private class RandomForest ( timer.stop("findBestSplits") } + baggedInput.unpersist() + timer.stop("total") logInfo("Internal timing for DecisionTree:") From 39ccabacf11abdd9afc8f9895084c6707ff35c85 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Oct 2014 11:50:42 -0700 Subject: [PATCH 0976/1492] [SPARK-3861][SQL] Avoid rebuilding hash tables for broadcast joins on each partition Author: Reynold Xin Closes #2727 from rxin/SPARK-3861-broadcast-hash-2 and squashes the following commits: 9c7b1a2 [Reynold Xin] Revert "Reuse CompactBuffer in UniqueKeyHashedRelation." 97626a1 [Reynold Xin] Reuse CompactBuffer in UniqueKeyHashedRelation. 7fcffb5 [Reynold Xin] Make UniqueKeyHashedRelation private[joins]. 18eb214 [Reynold Xin] Merge branch 'SPARK-3861-broadcast-hash' into SPARK-3861-broadcast-hash-1 4b9d0c9 [Reynold Xin] UniqueKeyHashedRelation.get should return null if the value is null. e0ebdd1 [Reynold Xin] Added a test case. 90b58c0 [Reynold Xin] [SPARK-3861] Avoid rebuilding hash tables on each partition 0c0082b [Reynold Xin] Fix line length. cbc664c [Reynold Xin] Rename join -> joins package. a070d44 [Reynold Xin] Fix line length in HashJoin a39be8c [Reynold Xin] [SPARK-3857] Create a join package for various join operators. --- .../execution/joins/BroadcastHashJoin.scala | 8 +- .../spark/sql/execution/joins/HashJoin.scala | 34 ++---- .../sql/execution/joins/HashedRelation.scala | 109 ++++++++++++++++++ .../execution/joins/ShuffledHashJoin.scala | 5 +- .../execution/joins/HashedRelationSuite.scala | 63 ++++++++++ 5 files changed, 187 insertions(+), 32 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index d88ab6367a1b3..8fd35880eedfe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -22,7 +22,7 @@ import scala.concurrent.duration._ import scala.concurrent.ExecutionContext.Implicits.global import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.{Row, Expression} import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnspecifiedDistribution} import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} @@ -49,14 +49,16 @@ case class BroadcastHashJoin( @transient private val broadcastFuture = future { - sparkContext.broadcast(buildPlan.executeCollect()) + val input: Array[Row] = buildPlan.executeCollect() + val hashed = HashedRelation(input.iterator, buildSideKeyGenerator, input.length) + sparkContext.broadcast(hashed) } override def execute() = { val broadcastRelation = Await.result(broadcastFuture, 5.minute) streamedPlan.execute().mapPartitions { streamedIter => - joinIterators(broadcastRelation.value.iterator, streamedIter) + hashJoin(streamedIter, broadcastRelation.value) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index 472b2e6ca6b4a..4012d757d5f9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow2, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.collection.CompactBuffer @@ -43,34 +43,14 @@ trait HashJoin { override def output = left.output ++ right.output - @transient protected lazy val buildSideKeyGenerator = newProjection(buildKeys, buildPlan.output) - @transient protected lazy val streamSideKeyGenerator = + @transient protected lazy val buildSideKeyGenerator: Projection = + newProjection(buildKeys, buildPlan.output) + + @transient protected lazy val streamSideKeyGenerator: () => MutableProjection = newMutableProjection(streamedKeys, streamedPlan.output) - protected def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = + protected def hashJoin(streamIter: Iterator[Row], hashedRelation: HashedRelation): Iterator[Row] = { - // TODO: Use Spark's HashMap implementation. - - val hashTable = new java.util.HashMap[Row, CompactBuffer[Row]]() - var currentRow: Row = null - - // Create a mapping of buildKeys -> rows - while (buildIter.hasNext) { - currentRow = buildIter.next() - val rowKey = buildSideKeyGenerator(currentRow) - if (!rowKey.anyNull) { - val existingMatchList = hashTable.get(rowKey) - val matchList = if (existingMatchList == null) { - val newMatchList = new CompactBuffer[Row]() - hashTable.put(rowKey, newMatchList) - newMatchList - } else { - existingMatchList - } - matchList += currentRow.copy() - } - } - new Iterator[Row] { private[this] var currentStreamedRow: Row = _ private[this] var currentHashMatches: CompactBuffer[Row] = _ @@ -107,7 +87,7 @@ trait HashJoin { while (currentHashMatches == null && streamIter.hasNext) { currentStreamedRow = streamIter.next() if (!joinKeys(currentStreamedRow).anyNull) { - currentHashMatches = hashTable.get(joinKeys.currentValue) + currentHashMatches = hashedRelation.get(joinKeys.currentValue) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala new file mode 100644 index 0000000000000..38b8993b03f82 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import java.util.{HashMap => JavaHashMap} + +import org.apache.spark.sql.catalyst.expressions.{Projection, Row} +import org.apache.spark.util.collection.CompactBuffer + + +/** + * Interface for a hashed relation by some key. Use [[HashedRelation.apply]] to create a concrete + * object. + */ +private[joins] sealed trait HashedRelation { + def get(key: Row): CompactBuffer[Row] +} + + +/** + * A general [[HashedRelation]] backed by a hash map that maps the key into a sequence of values. + */ +private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, CompactBuffer[Row]]) + extends HashedRelation with Serializable { + + override def get(key: Row) = hashTable.get(key) +} + + +/** + * A specialized [[HashedRelation]] that maps key into a single value. This implementation + * assumes the key is unique. + */ +private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, Row]) + extends HashedRelation with Serializable { + + override def get(key: Row) = { + val v = hashTable.get(key) + if (v eq null) null else CompactBuffer(v) + } + + def getValue(key: Row): Row = hashTable.get(key) +} + + +// TODO(rxin): a version of [[HashedRelation]] backed by arrays for consecutive integer keys. + + +private[joins] object HashedRelation { + + def apply( + input: Iterator[Row], + keyGenerator: Projection, + sizeEstimate: Int = 64): HashedRelation = { + + // TODO: Use Spark's HashMap implementation. + val hashTable = new JavaHashMap[Row, CompactBuffer[Row]](sizeEstimate) + var currentRow: Row = null + + // Whether the join key is unique. If the key is unique, we can convert the underlying + // hash map into one specialized for this. + var keyIsUnique = true + + // Create a mapping of buildKeys -> rows + while (input.hasNext) { + currentRow = input.next() + val rowKey = keyGenerator(currentRow) + if (!rowKey.anyNull) { + val existingMatchList = hashTable.get(rowKey) + val matchList = if (existingMatchList == null) { + val newMatchList = new CompactBuffer[Row]() + hashTable.put(rowKey, newMatchList) + newMatchList + } else { + keyIsUnique = false + existingMatchList + } + matchList += currentRow.copy() + } + } + + if (keyIsUnique) { + val uniqHashTable = new JavaHashMap[Row, Row](hashTable.size) + val iter = hashTable.entrySet().iterator() + while (iter.hasNext) { + val entry = iter.next() + uniqHashTable.put(entry.getKey, entry.getValue()(0)) + } + new UniqueKeyHashedRelation(uniqHashTable) + } else { + new GeneralHashedRelation(hashTable) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala index 8247304c1dc2c..418c1c23e5546 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -42,8 +42,9 @@ case class ShuffledHashJoin( ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil override def execute() = { - buildPlan.execute().zipPartitions(streamedPlan.execute()) { - (buildIter, streamIter) => joinIterators(buildIter, streamIter) + buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => + val hashed = HashedRelation(buildIter, buildSideKeyGenerator) + hashJoin(streamIter, hashed) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala new file mode 100644 index 0000000000000..2aad01ded1acf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions.{Projection, Row} +import org.apache.spark.util.collection.CompactBuffer + + +class HashedRelationSuite extends FunSuite { + + // Key is simply the record itself + private val keyProjection = new Projection { + override def apply(row: Row): Row = row + } + + test("GeneralHashedRelation") { + val data = Array(Row(0), Row(1), Row(2), Row(2)) + val hashed = HashedRelation(data.iterator, keyProjection) + assert(hashed.isInstanceOf[GeneralHashedRelation]) + + assert(hashed.get(data(0)) == CompactBuffer[Row](data(0))) + assert(hashed.get(data(1)) == CompactBuffer[Row](data(1))) + assert(hashed.get(Row(10)) === null) + + val data2 = CompactBuffer[Row](data(2)) + data2 += data(2) + assert(hashed.get(data(2)) == data2) + } + + test("UniqueKeyHashedRelation") { + val data = Array(Row(0), Row(1), Row(2)) + val hashed = HashedRelation(data.iterator, keyProjection) + assert(hashed.isInstanceOf[UniqueKeyHashedRelation]) + + assert(hashed.get(data(0)) == CompactBuffer[Row](data(0))) + assert(hashed.get(data(1)) == CompactBuffer[Row](data(1))) + assert(hashed.get(data(2)) == CompactBuffer[Row](data(2))) + assert(hashed.get(Row(10)) === null) + + val uniqHashed = hashed.asInstanceOf[UniqueKeyHashedRelation] + assert(uniqHashed.getValue(data(0)) == data(0)) + assert(uniqHashed.getValue(data(1)) == data(1)) + assert(uniqHashed.getValue(data(2)) == data(2)) + assert(uniqHashed.getValue(Row(10)) == null) + } +} From 49bbdcb660edff7522430b329a300765164ccc44 Mon Sep 17 00:00:00 2001 From: yingjieMiao Date: Mon, 13 Oct 2014 13:11:55 -0700 Subject: [PATCH 0977/1492] [Spark] RDD take() method: overestimate too much In the comment (Line 1083), it says: "Otherwise, interpolate the number of partitions we need to try, but overestimate it by 50%." `(1.5 * num * partsScanned / buf.size).toInt` is the guess of "num of total partitions needed". In every iteration, we should consider the increment `(1.5 * num * partsScanned / buf.size).toInt - partsScanned` Existing implementation 'exponentially' grows `partsScanned ` ( roughly: `x_{n+1} >= (1.5 + 1) x_n`) This could be a performance problem. (unless this is the intended behavior) Author: yingjieMiao Closes #2648 from yingjieMiao/rdd_take and squashes the following commits: d758218 [yingjieMiao] scala style fix a8e74bb [yingjieMiao] python style fix 4b6e777 [yingjieMiao] infix operator style fix 4391d3b [yingjieMiao] typo fix. 692f4e6 [yingjieMiao] cap numPartsToTry c4483dc [yingjieMiao] style fix 1d2c410 [yingjieMiao] also change in rdd.py and AsyncRDD d31ff7e [yingjieMiao] handle the edge case after 1 iteration a2aa36b [yingjieMiao] RDD take method: overestimate too much --- .../scala/org/apache/spark/rdd/AsyncRDDActions.scala | 12 +++++++----- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 8 +++++--- python/pyspark/rdd.py | 5 ++++- 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index b62f3fbdc4a15..ede5568493cc0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -78,16 +78,18 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 1 if (partsScanned > 0) { - // If we didn't find any rows after the first iteration, just try all partitions next. + // If we didn't find any rows after the previous iteration, quadruple and retry. // Otherwise, interpolate the number of partitions we need to try, but overestimate it - // by 50%. + // by 50%. We also cap the estimation in the end. if (results.size == 0) { - numPartsToTry = totalParts - 1 + numPartsToTry = partsScanned * 4 } else { - numPartsToTry = (1.5 * num * partsScanned / results.size).toInt + // the left side of max is >=1 whenever partsScanned >= 2 + numPartsToTry = Math.max(1, + (1.5 * num * partsScanned / results.size).toInt - partsScanned) + numPartsToTry = Math.min(numPartsToTry, partsScanned * 4) } } - numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions val left = num - results.size val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) 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 2aba40d152e3e..71cabf61d4ee0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1079,15 +1079,17 @@ abstract class RDD[T: ClassTag]( // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 1 if (partsScanned > 0) { - // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise, + // If we didn't find any rows after the previous iteration, quadruple and retry. Otherwise, // interpolate the number of partitions we need to try, but overestimate it by 50%. + // We also cap the estimation in the end. if (buf.size == 0) { numPartsToTry = partsScanned * 4 } else { - numPartsToTry = (1.5 * num * partsScanned / buf.size).toInt + // the left side of max is >=1 whenever partsScanned >= 2 + numPartsToTry = Math.max((1.5 * num * partsScanned / buf.size).toInt - partsScanned, 1) + numPartsToTry = Math.min(numPartsToTry, partsScanned * 4) } } - numPartsToTry = math.max(0, numPartsToTry) // guard against negative num of partitions val left = num - buf.size val p = partsScanned until math.min(partsScanned + numPartsToTry, totalParts) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index e13bab946c44a..15be4bfec92f9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1070,10 +1070,13 @@ def take(self, num): # If we didn't find any rows after the previous iteration, # quadruple and retry. Otherwise, interpolate the number of # partitions we need to try, but overestimate it by 50%. + # We also cap the estimation in the end. if len(items) == 0: numPartsToTry = partsScanned * 4 else: - numPartsToTry = int(1.5 * num * partsScanned / len(items)) + # the first paramter of max is >=1 whenever partsScanned >= 2 + numPartsToTry = int(1.5 * num * partsScanned / len(items)) - partsScanned + numPartsToTry = min(max(numPartsToTry, 1), partsScanned * 4) left = num - len(items) From 46db277cc14bf3c1e4c4779baa8a40189b332d89 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Oct 2014 13:31:14 -0700 Subject: [PATCH 0978/1492] [SPARK-3892][SQL] remove redundant type name Author: Daoyuan Wang Closes #2747 from adrian-wang/typename and squashes the following commits: 2824216 [Daoyuan Wang] remove redundant typeName fbaf340 [Daoyuan Wang] typename --- .../org/apache/spark/sql/catalyst/types/dataTypes.scala | 5 ----- 1 file changed, 5 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 1d375b8754182..5bdacab664f8b 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 @@ -349,7 +349,6 @@ case object FloatType extends FractionalType { object ArrayType { /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) - def typeName: String = "array" } /** @@ -395,8 +394,6 @@ case class StructField(name: String, dataType: DataType, nullable: Boolean) { object StructType { protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) - - def typeName = "struct" } case class StructType(fields: Seq[StructField]) extends DataType { @@ -459,8 +456,6 @@ object MapType { */ def apply(keyType: DataType, valueType: DataType): MapType = MapType(keyType: DataType, valueType: DataType, true) - - def simpleName = "map" } /** From 2ac40da3f9fa6d45a59bb45b41606f1931ac5e81 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Oct 2014 13:33:12 -0700 Subject: [PATCH 0979/1492] [SPARK-3407][SQL]Add Date type support Author: Daoyuan Wang Closes #2344 from adrian-wang/date and squashes the following commits: f15074a [Daoyuan Wang] remove outdated lines 2038085 [Daoyuan Wang] update return type 00fe81f [Daoyuan Wang] address lian cheng's comments 0df6ea1 [Daoyuan Wang] rebase and remove simple string bb1b1ef [Daoyuan Wang] remove failing test aa96735 [Daoyuan Wang] not cast for same type compare 30bf48b [Daoyuan Wang] resolve rebase conflict 617d1a8 [Daoyuan Wang] add date_udf case to white list c37e848 [Daoyuan Wang] comment update 5429212 [Daoyuan Wang] change to long f8f219f [Daoyuan Wang] revise according to Cheng Hao 0e0a4f5 [Daoyuan Wang] minor format 4ddcb92 [Daoyuan Wang] add java api for date 0e3110e [Daoyuan Wang] try to fix timezone issue 17fda35 [Daoyuan Wang] set test list 2dfbb5b [Daoyuan Wang] support date type --- .../spark/sql/catalyst/ScalaReflection.scala | 5 +- .../catalyst/analysis/HiveTypeCoercion.scala | 29 +++- .../spark/sql/catalyst/dsl/package.scala | 6 +- .../spark/sql/catalyst/expressions/Cast.scala | 98 +++++++++++-- .../sql/catalyst/expressions/literals.scala | 3 +- .../spark/sql/catalyst/types/dataTypes.scala | 12 +- .../ExpressionEvaluationSuite.scala | 35 ++++- .../apache/spark/sql/api/java/DataType.java | 5 + .../apache/spark/sql/api/java/DateType.java | 27 ++++ .../spark/sql/columnar/ColumnAccessor.scala | 4 + .../spark/sql/columnar/ColumnBuilder.scala | 3 + .../spark/sql/columnar/ColumnStats.scala | 20 ++- .../spark/sql/columnar/ColumnType.scala | 28 +++- .../scala/org/apache/spark/sql/package.scala | 10 ++ .../sql/types/util/DataTypeConversions.scala | 3 + .../sql/ScalaReflectionRelationSuite.scala | 5 +- .../spark/sql/columnar/ColumnStatsSuite.scala | 1 + .../spark/sql/columnar/ColumnTypeSuite.scala | 7 +- .../sql/columnar/ColumnarTestUtils.scala | 3 +- .../NullableColumnAccessorSuite.scala | 4 +- .../columnar/NullableColumnBuilderSuite.scala | 4 +- .../execution/HiveCompatibilitySuite.scala | 10 ++ .../apache/spark/sql/hive/HiveContext.scala | 6 +- .../spark/sql/hive/HiveInspectors.scala | 9 ++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 + .../org/apache/spark/sql/hive/HiveQl.scala | 8 + .../date_1-0-23edf29bf7376c70d5ecf12720f4b1eb | 0 .../date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b | 0 ...date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 | 1 + ...date_1-11-480c5f024a28232b7857be327c992509 | 1 + ...date_1-12-4c0ed7fcb75770d8790575b586bf14f4 | 1 + .../date_1-13-44fc74c1993062c0a9522199ff27fea | 1 + ...date_1-14-4855a66124b16d1d0d003235995ac06b | 1 + ...date_1-15-8bc190dba0f641840b5e1e198a14c55b | 1 + ...date_1-16-23edf29bf7376c70d5ecf12720f4b1eb | 0 .../date_1-2-abdce0c0d14d3fc7441b7c134b02f99a | 0 .../date_1-3-df16364a220ff96a6ea1cd478cbc1d0b | 1 + .../date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 | 1 + .../date_1-5-5e70fc74158fbfca38134174360de12d | 0 .../date_1-6-df16364a220ff96a6ea1cd478cbc1d0b | 1 + .../date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 | 1 + .../date_1-8-1d5c58095cd52ea539d869f2ab1ab67d | 0 .../date_1-9-df16364a220ff96a6ea1cd478cbc1d0b | 1 + .../date_2-3-eedb73e0a622c2ab760b524f395dd4ba | 137 ++++++++++++++++++ .../date_2-4-3618dfde8da7c26f03bca72970db9ef7 | 137 ++++++++++++++++++ .../date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c | 12 ++ .../date_2-6-f4edce7cb20f325e8b69e787b2ae8882 | 0 .../date_3-3-4cf49e71b636df754871a675f9e4e24 | 0 .../date_3-4-e009f358964f6d1236cfc03283e2b06f | 1 + .../date_3-5-c26de4559926ddb0127d2dc5ea154774 | 0 .../date_4-0-b84f7e931d710dcbe3c5126d998285a8 | 0 .../date_4-1-6272f5e518f6a20bc96a5870ff315c4f | 0 .../date_4-2-4a0e7bde447ef616b98e0f55d2886de0 | 0 .../date_4-3-a23faa56b5d3ca9063a21f72b4278b00 | 0 .../date_4-4-bee09a7384666043621f68297cee2e68 | 1 + .../date_4-5-b84f7e931d710dcbe3c5126d998285a8 | 0 ...parison-0-69eec445bd045c9dc899fafa348d8495 | 1 + ...parison-1-fcc400871a502009c8680509e3869ec1 | 1 + ...arison-10-a9f2560c273163e11306d4f1dd1d9d54 | 1 + ...arison-11-4a7bac9ddcf40db6329faaec8e426543 | 1 + ...parison-2-b8598a4d0c948c2ddcf3eeef0abf2264 | 1 + ...parison-3-14d35f266be9cceb11a2ae09ec8b3835 | 1 + ...parison-4-c8865b14d53f2c2496fb69ee8191bf37 | 1 + ...parison-5-f2c907e64da8166a731ddc0ed19bad6c | 1 + ...parison-6-5606505a92bad10023ad9a3ef77eacc9 | 1 + ...mparison-7-47913d4aaf0d468ab3764cc3bfd68eb | 1 + ...parison-8-1e5ce4f833b6fba45618437c8fb7643c | 1 + ...parison-9-bcd987341fc1c38047a27d29dac6ae7c | 1 + ...e_join1-3-f71c7be760fb4de4eff8225f2c6614b2 | 22 +++ ...te_join1-4-70b9b49c55699fe94cfde069f5d197c | 0 ..._serde-10-d80e681519dcd8f5078c5602bb5befa9 | 0 ..._serde-11-29540200936bba47f17553547b409af7 | 0 ..._serde-12-c3c3275658b89d31fc504db31ae9f99c | 0 ..._serde-13-6c546456c81e635b6753e1552fac9129 | 1 + ..._serde-14-f8ba18cc7b0225b4022299c44d435101 | 1 + ..._serde-15-66fadc9bcea7d107a610758aa6f50ff3 | 0 ..._serde-16-1bd3345b46f77e17810978e56f9f7c6b | 0 ..._serde-17-a0df43062f8ab676ef728c9968443f12 | 0 ..._serde-18-b50ecc72ce9018ab12fb17568fef038a | 1 + ..._serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 | 1 + ..._serde-20-588516368d8c1533cb7bfb2157fd58c1 | 0 ..._serde-21-dfe166fe053468e738dca23ebe043091 | 0 ..._serde-22-45240a488fb708e432d2f45b74ef7e63 | 0 ..._serde-23-1742a51e4967a8d263572d890cd8d4a8 | 1 + ...e_serde-24-14fd49bd6fee907c1699f7b4e26685b | 1 + ..._serde-25-a199cf185184a25190d65c123d0694ee | 0 ..._serde-26-c5fa68d9aff36f22e5edc1b54332d0ab | 0 ..._serde-27-4d86c79f858866acec3c37f6598c2638 | 0 ..._serde-28-16a41fc9e0f51eb417c763bae8e9cadb | 1 + ..._serde-29-bd1cb09aacd906527b0bbf43bbded812 | 1 + ..._serde-30-7c80741f9f485729afc68609c55423a0 | 0 ...e_serde-31-da36cd1654aee055cb3650133c9d11f | 0 ..._serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 | 0 ..._serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 | 1 + ..._serde-34-6485841336c097895ad5b34f42c0745f | 1 + ..._serde-35-8651a7c351cbc07fb1af6193f6885de8 | 0 ..._serde-36-36e6041f53433482631018410bb62a99 | 0 ..._serde-37-3ddfd8ecb28991aeed588f1ea852c427 | 0 ..._serde-38-e6167e27465514356c557a77d956ea46 | 0 ..._serde-39-c1e17c93582656c12970c37bac153bf2 | 0 ..._serde-40-4a17944b9ec8999bb20c5ba5d4cb877c | 0 ...e_serde-8-cace4f60a08342f58fbe816a9c3a73cf | 137 ++++++++++++++++++ ...e_serde-9-436c3c61cc4278b54ac79c53c88ff422 | 12 ++ ...ate_udf-0-84604a42a5d7f2842f1eec10c689d447 | 0 ...ate_udf-1-5e8136f6a6503ae9bef9beca80fada13 | 0 ...te_udf-10-988ad9744096a29a3672a2d4c121299b | 1 + ...te_udf-11-a5100dd42201b5bc035a9d684cc21bdc | 1 + ...te_udf-12-eb7280a1f191344a99eaa0f805e8faff | 1 + ...te_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 | 1 + ...ate_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 | 1 + ...te_udf-15-d031ee50c119d7c6acafd53543dbd0c4 | 1 + ...te_udf-16-dc59f69e1685e8d923b187ec50d80f06 | 1 + ...te_udf-17-7d046d4efc568049cf3792470b6feab9 | 1 + ...te_udf-18-84604a42a5d7f2842f1eec10c689d447 | 0 ...te_udf-19-5e8136f6a6503ae9bef9beca80fada13 | 0 ...ate_udf-2-10e337c34d1e82a360b8599988f4b266 | 0 ...te_udf-20-10e337c34d1e82a360b8599988f4b266 | 0 ...ate_udf-3-29e406e613c0284b3e16a8943a4d31bd | 0 ...ate_udf-4-23653315213f578856ab5c3bd80c0264 | 0 ...ate_udf-5-891fd92a4787b9789f6d1f51c1eddc8a | 0 ...ate_udf-6-3473c118d20783eafb456043a2ee5d5b | 0 ...ate_udf-7-9fb5165824e161074565e7500959c1b2 | 0 ...ate_udf-8-badfe833681362092fc6345f888b1c21 | 1 + ...ate_udf-9-a8cbb039661d796beaa0d1564c58c563 | 1 + ...on_date-0-7ec1f3a845e2c49191460e15af30aa30 | 0 ...on_date-1-916193405ce5e020dcd32c58325db6fe | 0 ...n_date-10-a8dde9c0b5746dd770c9c262d23ffb10 | 1 + ...n_date-11-fdface2fb6eef67f15bb7d0de2294957 | 1 + ...n_date-12-9b945f8ece6e09ad28c866ff3a10cc24 | 1 + ...on_date-13-b7cb91c7c459798078a79071d329dbf | 1 + ...n_date-14-e4366325f3a0c4a8e92be59f4de73fce | 1 + ...n_date-15-a062a6e87867d8c8cfbdad97bedcbe5f | 1 + ...n_date-16-22a5627d9ac112665eae01d07a91c89c | 1 + ...on_date-17-b9ce94ef93cb16d629af7d7f8ee637e | 1 + ...n_date-18-72c6e9a4e0b434cef67144825346c687 | 1 + ...n_date-19-44e5165eb210559e420105073bc96125 | 1 + ...on_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 | 0 ...n_date-20-7ec1f3a845e2c49191460e15af30aa30 | 0 ...on_date-3-c938b08f57d588926a5d5fbfa4531012 | 0 ...on_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 | 0 ...on_date-5-a855aba47876561fd4fb095e09580686 | 0 ...on_date-6-1405c311915f27b0cc616c83d39eaacc | 2 + ...on_date-7-2ac950d8d5656549dd453e5464cb8530 | 5 + ...on_date-8-a425c11c12c9ce4c9c43d4fbccee5347 | 1 + ...on_date-9-aad6078a09b7bd8f5141437e86bb229f | 1 + ..._check-12-7e053ba4f9dea1e74c1d04c557c3adac | 6 + ..._check-13-45fb706ff448da1fe609c7ff76a80d4d | 0 ...on_date-6-f4d5c71145a9b7464685aa7d09cd4dfd | 40 +++++ ...on_date-7-a0bade1c77338d4f72962389a1f5bea2 | 0 ...on_date-8-21306adbd8be8ad75174ad9d3e42b73c | 0 150 files changed, 872 insertions(+), 42 deletions(-) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java create mode 100644 sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb create mode 100644 sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b create mode 100644 sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 create mode 100644 sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 create mode 100644 sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 create mode 100644 sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea create mode 100644 sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b create mode 100644 sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b create mode 100644 sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb create mode 100644 sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a create mode 100644 sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b create mode 100644 sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 create mode 100644 sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d create mode 100644 sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b create mode 100644 sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 create mode 100644 sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d create mode 100644 sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b create mode 100644 sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba create mode 100644 sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 create mode 100644 sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c create mode 100644 sql/hive/src/test/resources/golden/date_2-6-f4edce7cb20f325e8b69e787b2ae8882 create mode 100644 sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 create mode 100644 sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f create mode 100644 sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 create mode 100644 sql/hive/src/test/resources/golden/date_4-0-b84f7e931d710dcbe3c5126d998285a8 create mode 100644 sql/hive/src/test/resources/golden/date_4-1-6272f5e518f6a20bc96a5870ff315c4f create mode 100644 sql/hive/src/test/resources/golden/date_4-2-4a0e7bde447ef616b98e0f55d2886de0 create mode 100644 sql/hive/src/test/resources/golden/date_4-3-a23faa56b5d3ca9063a21f72b4278b00 create mode 100644 sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 create mode 100644 sql/hive/src/test/resources/golden/date_4-5-b84f7e931d710dcbe3c5126d998285a8 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c create mode 100644 sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 create mode 100644 sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb create mode 100644 sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c create mode 100644 sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c create mode 100644 sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 create mode 100644 sql/hive/src/test/resources/golden/date_join1-4-70b9b49c55699fe94cfde069f5d197c create mode 100644 sql/hive/src/test/resources/golden/date_serde-10-d80e681519dcd8f5078c5602bb5befa9 create mode 100644 sql/hive/src/test/resources/golden/date_serde-11-29540200936bba47f17553547b409af7 create mode 100644 sql/hive/src/test/resources/golden/date_serde-12-c3c3275658b89d31fc504db31ae9f99c create mode 100644 sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 create mode 100644 sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 create mode 100644 sql/hive/src/test/resources/golden/date_serde-15-66fadc9bcea7d107a610758aa6f50ff3 create mode 100644 sql/hive/src/test/resources/golden/date_serde-16-1bd3345b46f77e17810978e56f9f7c6b create mode 100644 sql/hive/src/test/resources/golden/date_serde-17-a0df43062f8ab676ef728c9968443f12 create mode 100644 sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a create mode 100644 sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 create mode 100644 sql/hive/src/test/resources/golden/date_serde-20-588516368d8c1533cb7bfb2157fd58c1 create mode 100644 sql/hive/src/test/resources/golden/date_serde-21-dfe166fe053468e738dca23ebe043091 create mode 100644 sql/hive/src/test/resources/golden/date_serde-22-45240a488fb708e432d2f45b74ef7e63 create mode 100644 sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 create mode 100644 sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b create mode 100644 sql/hive/src/test/resources/golden/date_serde-25-a199cf185184a25190d65c123d0694ee create mode 100644 sql/hive/src/test/resources/golden/date_serde-26-c5fa68d9aff36f22e5edc1b54332d0ab create mode 100644 sql/hive/src/test/resources/golden/date_serde-27-4d86c79f858866acec3c37f6598c2638 create mode 100644 sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb create mode 100644 sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 create mode 100644 sql/hive/src/test/resources/golden/date_serde-30-7c80741f9f485729afc68609c55423a0 create mode 100644 sql/hive/src/test/resources/golden/date_serde-31-da36cd1654aee055cb3650133c9d11f create mode 100644 sql/hive/src/test/resources/golden/date_serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 create mode 100644 sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 create mode 100644 sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f create mode 100644 sql/hive/src/test/resources/golden/date_serde-35-8651a7c351cbc07fb1af6193f6885de8 create mode 100644 sql/hive/src/test/resources/golden/date_serde-36-36e6041f53433482631018410bb62a99 create mode 100644 sql/hive/src/test/resources/golden/date_serde-37-3ddfd8ecb28991aeed588f1ea852c427 create mode 100644 sql/hive/src/test/resources/golden/date_serde-38-e6167e27465514356c557a77d956ea46 create mode 100644 sql/hive/src/test/resources/golden/date_serde-39-c1e17c93582656c12970c37bac153bf2 create mode 100644 sql/hive/src/test/resources/golden/date_serde-40-4a17944b9ec8999bb20c5ba5d4cb877c create mode 100644 sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf create mode 100644 sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 create mode 100644 sql/hive/src/test/resources/golden/date_udf-0-84604a42a5d7f2842f1eec10c689d447 create mode 100644 sql/hive/src/test/resources/golden/date_udf-1-5e8136f6a6503ae9bef9beca80fada13 create mode 100644 sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b create mode 100644 sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc create mode 100644 sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff create mode 100644 sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 create mode 100644 sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 create mode 100644 sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 create mode 100644 sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 create mode 100644 sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 create mode 100644 sql/hive/src/test/resources/golden/date_udf-18-84604a42a5d7f2842f1eec10c689d447 create mode 100644 sql/hive/src/test/resources/golden/date_udf-19-5e8136f6a6503ae9bef9beca80fada13 create mode 100644 sql/hive/src/test/resources/golden/date_udf-2-10e337c34d1e82a360b8599988f4b266 create mode 100644 sql/hive/src/test/resources/golden/date_udf-20-10e337c34d1e82a360b8599988f4b266 create mode 100644 sql/hive/src/test/resources/golden/date_udf-3-29e406e613c0284b3e16a8943a4d31bd create mode 100644 sql/hive/src/test/resources/golden/date_udf-4-23653315213f578856ab5c3bd80c0264 create mode 100644 sql/hive/src/test/resources/golden/date_udf-5-891fd92a4787b9789f6d1f51c1eddc8a create mode 100644 sql/hive/src/test/resources/golden/date_udf-6-3473c118d20783eafb456043a2ee5d5b create mode 100644 sql/hive/src/test/resources/golden/date_udf-7-9fb5165824e161074565e7500959c1b2 create mode 100644 sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 create mode 100644 sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 create mode 100644 sql/hive/src/test/resources/golden/partition_date-0-7ec1f3a845e2c49191460e15af30aa30 create mode 100644 sql/hive/src/test/resources/golden/partition_date-1-916193405ce5e020dcd32c58325db6fe create mode 100644 sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 create mode 100644 sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 create mode 100644 sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 create mode 100644 sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf create mode 100644 sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce create mode 100644 sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f create mode 100644 sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c create mode 100644 sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e create mode 100644 sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 create mode 100644 sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 create mode 100644 sql/hive/src/test/resources/golden/partition_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 create mode 100644 sql/hive/src/test/resources/golden/partition_date-20-7ec1f3a845e2c49191460e15af30aa30 create mode 100644 sql/hive/src/test/resources/golden/partition_date-3-c938b08f57d588926a5d5fbfa4531012 create mode 100644 sql/hive/src/test/resources/golden/partition_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 create mode 100644 sql/hive/src/test/resources/golden/partition_date-5-a855aba47876561fd4fb095e09580686 create mode 100644 sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc create mode 100644 sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 create mode 100644 sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 create mode 100644 sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-13-45fb706ff448da1fe609c7ff76a80d4d create mode 100644 sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd create mode 100644 sql/hive/src/test/resources/golden/union_date-7-a0bade1c77338d4f72962389a1f5bea2 create mode 100644 sql/hive/src/test/resources/golden/union_date-8-21306adbd8be8ad75174ad9d3e42b73c diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index b3ae8e6779700..3d4296f9d7068 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation @@ -77,8 +77,9 @@ object ScalaReflection { val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< typeOf[Date] => Schema(DateType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 64881854df7a5..7c480de107e7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -220,20 +220,39 @@ trait HiveTypeCoercion { case a: BinaryArithmetic if a.right.dataType == StringType => a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) + // we should cast all timestamp/date/string compare into string compare + case p: BinaryPredicate if p.left.dataType == StringType + && p.right.dataType == DateType => + p.makeCopy(Array(p.left, Cast(p.right, StringType))) + case p: BinaryPredicate if p.left.dataType == DateType + && p.right.dataType == StringType => + p.makeCopy(Array(Cast(p.left, StringType), p.right)) case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType == TimestampType => - p.makeCopy(Array(Cast(p.left, TimestampType), p.right)) + p.makeCopy(Array(p.left, Cast(p.right, StringType))) case p: BinaryPredicate if p.left.dataType == TimestampType && p.right.dataType == StringType => - p.makeCopy(Array(p.left, Cast(p.right, TimestampType))) + p.makeCopy(Array(Cast(p.left, StringType), p.right)) + case p: BinaryPredicate if p.left.dataType == TimestampType + && p.right.dataType == DateType => + p.makeCopy(Array(Cast(p.left, StringType), Cast(p.right, StringType))) + case p: BinaryPredicate if p.left.dataType == DateType + && p.right.dataType == TimestampType => + p.makeCopy(Array(Cast(p.left, StringType), Cast(p.right, StringType))) case p: BinaryPredicate if p.left.dataType == StringType && p.right.dataType != StringType => p.makeCopy(Array(Cast(p.left, DoubleType), p.right)) case p: BinaryPredicate if p.left.dataType != StringType && p.right.dataType == StringType => p.makeCopy(Array(p.left, Cast(p.right, DoubleType))) - case i @ In(a,b) if a.dataType == TimestampType && b.forall(_.dataType == StringType) => - i.makeCopy(Array(a,b.map(Cast(_,TimestampType)))) + case i @ In(a, b) if a.dataType == DateType && b.forall(_.dataType == StringType) => + i.makeCopy(Array(Cast(a, StringType), b)) + case i @ In(a, b) if a.dataType == TimestampType && b.forall(_.dataType == StringType) => + i.makeCopy(Array(Cast(a, StringType), b)) + case i @ In(a, b) if a.dataType == DateType && b.forall(_.dataType == TimestampType) => + i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) + case i @ In(a, b) if a.dataType == TimestampType && b.forall(_.dataType == DateType) => + i.makeCopy(Array(Cast(a, StringType), b.map(Cast(_, StringType)))) case Sum(e) if e.dataType == StringType => Sum(Cast(e, DoubleType)) @@ -283,6 +302,8 @@ trait HiveTypeCoercion { // Skip if the type is boolean type already. Note that this extra cast should be removed // by optimizer.SimplifyCasts. case Cast(e, BooleanType) if e.dataType == BooleanType => e + // DateType should be null if be cast to boolean. + case Cast(e, BooleanType) if e.dataType == DateType => Cast(e, BooleanType) // If the data type is not boolean and is being cast boolean, turn it into a comparison // with the numeric value, i.e. x != 0. This will coerce the type into numeric type. case Cast(e, BooleanType) if e.dataType != BooleanType => Not(EqualTo(e, Literal(0))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index deb622c39faf5..75b6e37c2a1f9 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.language.implicitConversions @@ -119,6 +119,7 @@ package object dsl { implicit def floatToLiteral(f: Float) = Literal(f) implicit def doubleToLiteral(d: Double) = Literal(d) implicit def stringToLiteral(s: String) = Literal(s) + implicit def dateToLiteral(d: Date) = Literal(d) implicit def decimalToLiteral(d: BigDecimal) = Literal(d) implicit def timestampToLiteral(t: Timestamp) = Literal(t) implicit def binaryToLiteral(a: Array[Byte]) = Literal(a) @@ -174,6 +175,9 @@ package object dsl { /** Creates a new AttributeReference of type string */ def string = AttributeReference(s, StringType, nullable = true)() + /** Creates a new AttributeReference of type date */ + def date = AttributeReference(s, DateType, nullable = true)() + /** Creates a new AttributeReference of type decimal */ def decimal = AttributeReference(s, DecimalType, nullable = true)() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index f626d09f037bc..8e5ee12e314bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -17,18 +17,21 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.types._ /** Cast the child expression to the target data type. */ -case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { +case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with Logging { override def foldable = child.foldable override def nullable = (child.dataType, dataType) match { case (StringType, _: NumericType) => true case (StringType, TimestampType) => true + case (StringType, DateType) => true case _ => child.nullable } @@ -42,6 +45,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // UDFToString private[this] def castToString: Any => Any = child.dataType match { case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) + case DateType => buildCast[Date](_, dateToString) case TimestampType => buildCast[Timestamp](_, timestampToString) case _ => buildCast[Any](_, _.toString) } @@ -56,7 +60,10 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => buildCast[String](_, _.length() != 0) case TimestampType => - buildCast[Timestamp](_, b => b.getTime() != 0 || b.getNanos() != 0) + buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) + case DateType => + // Hive would return null when cast from date to boolean + buildCast[Date](_, d => null) case LongType => buildCast[Long](_, _ != 0) case IntegerType => @@ -95,6 +102,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { buildCast[Short](_, s => new Timestamp(s)) case ByteType => buildCast[Byte](_, b => new Timestamp(b)) + case DateType => + buildCast[Date](_, d => new Timestamp(d.getTime)) // TimestampWritable.decimalToTimestamp case DecimalType => buildCast[BigDecimal](_, d => decimalToTimestamp(d)) @@ -130,7 +139,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // Converts Timestamp to string according to Hive TimestampWritable convention private[this] def timestampToString(ts: Timestamp): String = { val timestampString = ts.toString - val formatted = Cast.threadLocalDateFormat.get.format(ts) + val formatted = Cast.threadLocalTimestampFormat.get.format(ts) if (timestampString.length > 19 && timestampString.substring(19) != ".0") { formatted + timestampString.substring(19) @@ -139,6 +148,39 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } } + // Converts Timestamp to string according to Hive TimestampWritable convention + private[this] def timestampToDateString(ts: Timestamp): String = { + Cast.threadLocalDateFormat.get.format(ts) + } + + // DateConverter + private[this] def castToDate: Any => Any = child.dataType match { + case StringType => + buildCast[String](_, s => + try Date.valueOf(s) catch { case _: java.lang.IllegalArgumentException => null } + ) + case TimestampType => + // throw valid precision more than seconds, according to Hive. + // Timestamp.nanos is in 0 to 999,999,999, no more than a second. + buildCast[Timestamp](_, t => new Date(Math.floor(t.getTime / 1000.0).toLong * 1000)) + // Hive throws this exception as a Semantic Exception + // It is never possible to compare result when hive return with exception, so we can return null + // NULL is more reasonable here, since the query itself obeys the grammar. + case _ => _ => null + } + + // Date cannot be cast to long, according to hive + private[this] def dateToLong(d: Date) = null + + // Date cannot be cast to double, according to hive + private[this] def dateToDouble(d: Date) = null + + // Converts Date to string according to Hive DateWritable convention + private[this] def dateToString(d: Date): String = { + Cast.threadLocalDateFormat.get.format(d) + } + + // LongConverter private[this] def castToLong: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toLong catch { @@ -146,6 +188,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0L) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t)) case DecimalType => @@ -154,6 +198,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) } + // IntConverter private[this] def castToInt: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toInt catch { @@ -161,6 +206,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1 else 0) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toInt) case DecimalType => @@ -169,6 +216,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) } + // ShortConverter private[this] def castToShort: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toShort catch { @@ -176,6 +224,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1.toShort else 0.toShort) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toShort) case DecimalType => @@ -184,6 +234,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort } + // ByteConverter private[this] def castToByte: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toByte catch { @@ -191,6 +242,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1.toByte else 0.toByte) + case DateType => + buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toByte) case DecimalType => @@ -199,6 +252,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } + // DecimalConverter private[this] def castToDecimal: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try BigDecimal(s.toDouble) catch { @@ -206,6 +260,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) BigDecimal(1) else BigDecimal(0)) + case DateType => + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => // Note that we lose precision here. buildCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) @@ -213,6 +269,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => BigDecimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)) } + // DoubleConverter private[this] def castToDouble: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toDouble catch { @@ -220,6 +277,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1d else 0d) + case DateType => + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t)) case DecimalType => @@ -228,6 +287,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) } + // FloatConverter private[this] def castToFloat: Any => Any = child.dataType match { case StringType => buildCast[String](_, s => try s.toFloat catch { @@ -235,6 +295,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { }) case BooleanType => buildCast[Boolean](_, b => if (b) 1f else 0f) + case DateType => + buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t).toFloat) case DecimalType => @@ -245,17 +307,18 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { private[this] lazy val cast: Any => Any = dataType match { case dt if dt == child.dataType => identity[Any] - case StringType => castToString - case BinaryType => castToBinary - case DecimalType => castToDecimal + case StringType => castToString + case BinaryType => castToBinary + case DecimalType => castToDecimal + case DateType => castToDate case TimestampType => castToTimestamp - case BooleanType => castToBoolean - case ByteType => castToByte - case ShortType => castToShort - case IntegerType => castToInt - case FloatType => castToFloat - case LongType => castToLong - case DoubleType => castToDouble + case BooleanType => castToBoolean + case ByteType => castToByte + case ShortType => castToShort + case IntegerType => castToInt + case FloatType => castToFloat + case LongType => castToLong + case DoubleType => castToDouble } override def eval(input: Row): Any = { @@ -267,6 +330,13 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { object Cast { // `SimpleDateFormat` is not thread-safe. private[sql] val threadLocalDateFormat = new ThreadLocal[DateFormat] { + override def initialValue() = { + new SimpleDateFormat("yyyy-MM-dd") + } + } + + // `SimpleDateFormat` is not thread-safe. + private[sql] val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { override def initialValue() = { new SimpleDateFormat("yyyy-MM-dd HH:mm:ss") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 78a0c55e4bbe5..ba240233cae61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.types._ @@ -33,6 +33,7 @@ object Literal { case b: Boolean => Literal(b, BooleanType) case d: BigDecimal => Literal(d, DecimalType) case t: Timestamp => Literal(t, TimestampType) + case d: Date => Literal(d, DateType) case a: Array[Byte] => Literal(a, BinaryType) case null => Literal(null, NullType) } 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 5bdacab664f8b..0cf139ebde417 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.types -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.math.Numeric.{BigDecimalAsIfIntegral, DoubleAsIfIntegral, FloatAsIfIntegral} import scala.reflect.ClassTag @@ -250,6 +250,16 @@ case object TimestampType extends NativeType { } } +case object DateType extends NativeType { + private[sql] type JvmType = Date + + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Date, y: Date) = x.compareTo(y) + } +} + abstract class NumericType extends NativeType with PrimitiveType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a 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 692ed78a7292c..6dc5942023f9e 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.collection.immutable.HashSet @@ -252,8 +252,11 @@ class ExpressionEvaluationSuite extends FunSuite { test("data type casting") { - val sts = "1970-01-01 00:00:01.1" - val ts = Timestamp.valueOf(sts) + val sd = "1970-01-01" + val d = Date.valueOf(sd) + val sts = sd + " 00:00:02" + val nts = sts + ".1" + val ts = Timestamp.valueOf(nts) checkEvaluation("abdef" cast StringType, "abdef") checkEvaluation("abdef" cast DecimalType, null) @@ -266,8 +269,15 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) - checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts) + checkEvaluation(Cast(Literal(sd) cast DateType, StringType), sd) + checkEvaluation(Cast(Literal(d) cast StringType, DateType), d) + checkEvaluation(Cast(Literal(nts) cast TimestampType, StringType), nts) checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts) + // all convert to string type to check + checkEvaluation( + Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd) + checkEvaluation( + Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), sts) checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef") @@ -316,6 +326,12 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) } + test("date") { + val d1 = Date.valueOf("1970-01-01") + val d2 = Date.valueOf("1970-01-02") + checkEvaluation(Literal(d1) < Literal(d2), true) + } + test("timestamp") { val ts1 = new Timestamp(12) val ts2 = new Timestamp(123) @@ -323,6 +339,17 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(ts1) < Literal(ts2), true) } + test("date casting") { + val d = Date.valueOf("1970-01-01") + checkEvaluation(Cast(d, ShortType), null) + checkEvaluation(Cast(d, IntegerType), null) + checkEvaluation(Cast(d, LongType), null) + checkEvaluation(Cast(d, FloatType), null) + checkEvaluation(Cast(d, DoubleType), null) + checkEvaluation(Cast(d, StringType), "1970-01-01") + checkEvaluation(Cast(Cast(d, TimestampType), StringType), "1970-01-01 00:00:00") + } + test("timestamp casting") { val millis = 15 * 1000 + 2 val seconds = millis * 1000 + 2 diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index 37b4c8ffcba0b..37e88d72b9172 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -44,6 +44,11 @@ public abstract class DataType { */ public static final BooleanType BooleanType = new BooleanType(); + /** + * Gets the DateType object. + */ + public static final DateType DateType = new DateType(); + /** * Gets the TimestampType object. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java new file mode 100644 index 0000000000000..6677793baa365 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DateType.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * The data type representing java.sql.Date values. + * + * {@code DateType} is represented by the singleton object {@link DataType#DateType}. + */ +public class DateType extends DataType { + protected DateType() {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index c9faf0852142a..538dd5b734664 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -92,6 +92,9 @@ private[sql] class FloatColumnAccessor(buffer: ByteBuffer) private[sql] class StringColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, STRING) +private[sql] class DateColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, DATE) + private[sql] class TimestampColumnAccessor(buffer: ByteBuffer) extends NativeColumnAccessor(buffer, TIMESTAMP) @@ -118,6 +121,7 @@ private[sql] object ColumnAccessor { case BYTE.typeId => new ByteColumnAccessor(dup) case SHORT.typeId => new ShortColumnAccessor(dup) case STRING.typeId => new StringColumnAccessor(dup) + case DATE.typeId => new DateColumnAccessor(dup) case TIMESTAMP.typeId => new TimestampColumnAccessor(dup) case BINARY.typeId => new BinaryColumnAccessor(dup) case GENERIC.typeId => new GenericColumnAccessor(dup) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 2e61a981375aa..300cef15bf8a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -107,6 +107,8 @@ private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColum private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) +private[sql] class DateColumnBuilder extends NativeColumnBuilder(new DateColumnStats, DATE) + private[sql] class TimestampColumnBuilder extends NativeColumnBuilder(new TimestampColumnStats, TIMESTAMP) @@ -151,6 +153,7 @@ private[sql] object ColumnBuilder { case STRING.typeId => new StringColumnBuilder case BINARY.typeId => new BinaryColumnBuilder case GENERIC.typeId => new GenericColumnBuilder + case DATE.typeId => new DateColumnBuilder case TIMESTAMP.typeId => new TimestampColumnBuilder }).asInstanceOf[ColumnBuilder] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 203a714e03c97..b34ab255d084a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, AttributeReference} @@ -190,6 +190,24 @@ private[sql] class StringColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount) } +private[sql] class DateColumnStats extends ColumnStats { + var upper: Date = null + var lower: Date = null + var nullCount = 0 + + override def gatherStats(row: Row, ordinal: Int) { + if (!row.isNullAt(ordinal)) { + val value = row(ordinal).asInstanceOf[Date] + if (upper == null || value.compareTo(upper) > 0) upper = value + if (lower == null || value.compareTo(lower) < 0) lower = value + } else { + nullCount += 1 + } + } + + def collectedStatistics = Row(lower, upper, nullCount) +} + private[sql] class TimestampColumnStats extends ColumnStats { var upper: Timestamp = null var lower: Timestamp = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 198b5756676aa..ab66c85c4f242 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import scala.reflect.runtime.universe.TypeTag @@ -335,7 +335,26 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { } } -private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 8, 12) { +private[sql] object DATE extends NativeColumnType(DateType, 8, 8) { + override def extract(buffer: ByteBuffer) = { + val date = new Date(buffer.getLong()) + date + } + + override def append(v: Date, buffer: ByteBuffer): Unit = { + buffer.putLong(v.getTime) + } + + override def getField(row: Row, ordinal: Int) = { + row(ordinal).asInstanceOf[Date] + } + + override def setField(row: MutableRow, ordinal: Int, value: Date): Unit = { + row(ordinal) = value + } +} + +private[sql] object TIMESTAMP extends NativeColumnType(TimestampType, 9, 12) { override def extract(buffer: ByteBuffer) = { val timestamp = new Timestamp(buffer.getLong()) timestamp.setNanos(buffer.getInt()) @@ -376,7 +395,7 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](10, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = value } @@ -387,7 +406,7 @@ private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](9, 16) { // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](10, 16) { +private[sql] object GENERIC extends ByteArrayColumnType[DataType](11, 16) { override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]): Unit = { row(ordinal) = SparkSqlSerializer.deserialize[Any](value) } @@ -407,6 +426,7 @@ private[sql] object ColumnType { case ShortType => SHORT case StringType => STRING case BinaryType => BINARY + case DateType => DATE case TimestampType => TIMESTAMP case _ => GENERIC } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index f513eae9c2d13..e98d151286818 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -165,6 +165,16 @@ package object sql { @DeveloperApi val TimestampType = catalyst.types.TimestampType + /** + * :: DeveloperApi :: + * + * The data type representing `java.sql.Date` values. + * + * @group dataType + */ + @DeveloperApi + val DateType = catalyst.types.DateType + /** * :: DeveloperApi :: * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 77353f4eb0227..e44cb08309523 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -41,6 +41,7 @@ protected[sql] object DataTypeConversions { case StringType => JDataType.StringType case BinaryType => JDataType.BinaryType case BooleanType => JDataType.BooleanType + case DateType => JDataType.DateType case TimestampType => JDataType.TimestampType case DecimalType => JDataType.DecimalType case DoubleType => JDataType.DoubleType @@ -80,6 +81,8 @@ protected[sql] object DataTypeConversions { BinaryType case booleanType: org.apache.spark.sql.api.java.BooleanType => BooleanType + case dateType: org.apache.spark.sql.api.java.DateType => + DateType case timestampType: org.apache.spark.sql.api.java.TimestampType => TimestampType case decimalType: org.apache.spark.sql.api.java.DecimalType => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index e24c521d24c7a..bfa9ea416266d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.scalatest.FunSuite @@ -34,6 +34,7 @@ case class ReflectData( byteField: Byte, booleanField: Boolean, decimalField: BigDecimal, + date: Date, timestampField: Timestamp, seqInt: Seq[Int]) @@ -76,7 +77,7 @@ case class ComplexReflectData( class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - BigDecimal(1), new Timestamp(12345), Seq(1,2,3)) + BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) rdd.registerTempTable("reflectData") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 0cdbb3167ce36..6bdf741134e2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -30,6 +30,7 @@ class ColumnStatsSuite extends FunSuite { testColumnStats(classOf[FloatColumnStats], FLOAT, Row(Float.MaxValue, Float.MinValue, 0)) testColumnStats(classOf[DoubleColumnStats], DOUBLE, Row(Double.MaxValue, Double.MinValue, 0)) testColumnStats(classOf[StringColumnStats], STRING, Row(null, null, 0)) + testColumnStats(classOf[DateColumnStats], DATE, Row(null, null, 0)) testColumnStats(classOf[TimestampColumnStats], TIMESTAMP, Row(null, null, 0)) def testColumnStats[T <: NativeType, U <: ColumnStats]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 4fb1ecf1d532b..3f3f35d50188b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.scalatest.FunSuite @@ -33,8 +33,8 @@ class ColumnTypeSuite extends FunSuite with Logging { test("defaultSize") { val checks = Map( - INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, - BOOLEAN -> 1, STRING -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) + INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, BOOLEAN -> 1, + STRING -> 8, DATE -> 8, TIMESTAMP -> 12, BINARY -> 16, GENERIC -> 16) checks.foreach { case (columnType, expectedSize) => assertResult(expectedSize, s"Wrong defaultSize for $columnType") { @@ -64,6 +64,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(DATE, new Date(0L), 8) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) val binary = Array.fill[Byte](4)(0: Byte) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index 38b04dd959f70..a1f21219eaf2f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.columnar import scala.collection.immutable.HashSet import scala.util.Random -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow @@ -50,6 +50,7 @@ object ColumnarTestUtils { case STRING => Random.nextString(Random.nextInt(32)) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) + case DATE => new Date(Random.nextLong()) case TIMESTAMP => val timestamp = new Timestamp(Random.nextLong()) timestamp.setNanos(Random.nextInt(999999999)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 6c9a9ab6c3418..21906e3fdcc6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -41,7 +41,9 @@ object TestNullableColumnAccessor { class NullableColumnAccessorSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { + Seq( + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + ).foreach { testNullableColumnAccessor(_) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index f54a21eb4fbb1..cb73f3da81e24 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -37,7 +37,9 @@ object TestNullableColumnBuilder { class NullableColumnBuilderSuite extends FunSuite { import ColumnarTestUtils._ - Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, TIMESTAMP).foreach { + Seq( + INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC, DATE, TIMESTAMP + ).foreach { testNullableColumnBuilder(_) } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 35e9c9939d4b7..463888551a359 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -343,6 +343,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ct_case_insensitive", "database_location", "database_properties", + "date_2", + "date_3", + "date_4", + "date_comparison", + "date_join1", + "date_serde", + "date_udf", "decimal_1", "decimal_4", "decimal_join", @@ -604,8 +611,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", + "partition_date", "partition_schema1", "partition_serde_format", + "partition_type_check", "partition_varchar1", "partition_wise_fileformat4", "partition_wise_fileformat5", @@ -904,6 +913,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "union7", "union8", "union9", + "union_date", "union_lateralview", "union_ppr", "union_remove_11", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index fad3b39f81413..8b5a90159e1bb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive import java.io.{BufferedReader, File, InputStreamReader, PrintStream} -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.util.{ArrayList => JArrayList} import scala.collection.JavaConversions._ @@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.io.TimestampWritable +import org.apache.hadoop.hive.serde2.io.DateWritable import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD @@ -357,7 +358,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DecimalType, TimestampType, BinaryType) + ShortType, DecimalType, DateType, TimestampType, BinaryType) protected[sql] def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => @@ -372,6 +373,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "NULL" + case (d: Date, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (other, tpe) if primitiveTypes contains tpe => other.toString diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index d633c42c6bd67..1977618b4c9f2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -39,6 +39,7 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType + case c: Class[_] if c == classOf[hiveIo.DateWritable] => DateType case c: Class[_] if c == classOf[hiveIo.TimestampWritable] => TimestampType case c: Class[_] if c == classOf[hadoopIo.Text] => StringType case c: Class[_] if c == classOf[hadoopIo.IntWritable] => IntegerType @@ -49,6 +50,7 @@ private[hive] trait HiveInspectors { // java class case c: Class[_] if c == classOf[java.lang.String] => StringType + case c: Class[_] if c == classOf[java.sql.Date] => DateType case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType case c: Class[_] if c == classOf[HiveDecimal] => DecimalType case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType @@ -93,6 +95,7 @@ private[hive] trait HiveInspectors { System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) bytes } + case d: hiveIo.DateWritable => d.get case t: hiveIo.TimestampWritable => t.getTimestamp case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) case list: java.util.List[_] => list.map(unwrap) @@ -108,6 +111,7 @@ private[hive] trait HiveInspectors { case str: String => str case p: java.math.BigDecimal => p case p: Array[Byte] => p + case p: java.sql.Date => p case p: java.sql.Timestamp => p } @@ -147,6 +151,7 @@ private[hive] trait HiveInspectors { case l: Byte => l: java.lang.Byte case b: BigDecimal => new HiveDecimal(b.underlying()) case b: Array[Byte] => b + case d: java.sql.Date => d case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) case m: Map[_,_] => @@ -173,6 +178,7 @@ private[hive] trait HiveInspectors { case ByteType => PrimitiveObjectInspectorFactory.javaByteObjectInspector case NullType => PrimitiveObjectInspectorFactory.javaVoidObjectInspector case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + case DateType => PrimitiveObjectInspectorFactory.javaDateObjectInspector case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector case StructType(fields) => @@ -211,6 +217,8 @@ private[hive] trait HiveInspectors { case _: JavaBinaryObjectInspector => BinaryType case _: WritableHiveDecimalObjectInspector => DecimalType case _: JavaHiveDecimalObjectInspector => DecimalType + case _: WritableDateObjectInspector => DateType + case _: JavaDateObjectInspector => DateType case _: WritableTimestampObjectInspector => TimestampType case _: JavaTimestampObjectInspector => TimestampType case _: WritableVoidObjectInspector => NullType @@ -238,6 +246,7 @@ private[hive] trait HiveInspectors { case ShortType => shortTypeInfo case StringType => stringTypeInfo case DecimalType => decimalTypeInfo + case DateType => dateTypeInfo case TimestampType => timestampTypeInfo case NullType => voidTypeInfo } 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 addd5bed8426d..c5fee5e4702f6 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 @@ -186,6 +186,7 @@ object HiveMetastoreTypes extends RegexParsers { "binary" ^^^ BinaryType | "boolean" ^^^ BooleanType | "decimal" ^^^ DecimalType | + "date" ^^^ DateType | "timestamp" ^^^ TimestampType | "varchar\\((\\d+)\\)".r ^^^ StringType @@ -235,6 +236,7 @@ object HiveMetastoreTypes extends RegexParsers { case LongType => "bigint" case BinaryType => "binary" case BooleanType => "boolean" + case DateType => "date" case DecimalType => "decimal" case TimestampType => "timestamp" case NullType => "void" 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 7cc14dc7a9c9e..2b599157d15d3 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.sql.Date + import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils @@ -317,6 +319,7 @@ private[hive] object HiveQl { case Token("TOK_STRING", Nil) => StringType case Token("TOK_FLOAT", Nil) => FloatType case Token("TOK_DOUBLE", Nil) => DoubleType + case Token("TOK_DATE", Nil) => DateType case Token("TOK_TIMESTAMP", Nil) => TimestampType case Token("TOK_BINARY", Nil) => BinaryType case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) @@ -924,6 +927,8 @@ private[hive] object HiveQl { Cast(nodeToExpr(arg), DecimalType) case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), TimestampType) + case Token("TOK_FUNCTION", Token("TOK_DATE", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DateType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) @@ -1047,6 +1052,9 @@ private[hive] object HiveQl { case ast: ASTNode if ast.getType == HiveParser.StringLiteral => Literal(BaseSemanticAnalyzer.unescapeSQLString(ast.getText)) + case ast: ASTNode if ast.getType == HiveParser.TOK_DATELITERAL => + Literal(Date.valueOf(ast.getText.substring(1, ast.getText.length - 1))) + case a: ASTNode => throw new NotImplementedError( s"""No parse rules for ASTNode type: ${a.getType}, text: ${a.getText} : diff --git a/sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-0-23edf29bf7376c70d5ecf12720f4b1eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b b/sql/hive/src/test/resources/golden/date_1-1-4ebe3571c13a8b0c03096fbd972b7f1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 new file mode 100644 index 0000000000000..8fb5edae63c6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-10-d964bec7e5632091ab5cb6f6786dbbf9 @@ -0,0 +1 @@ +2011-01-01 1 diff --git a/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 b/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 new file mode 100644 index 0000000000000..5a368ab170261 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-11-480c5f024a28232b7857be327c992509 @@ -0,0 +1 @@ +2012-01-01 2011-01-01 2011-01-01 00:00:00 2011-01-01 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 b/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 new file mode 100644 index 0000000000000..edb4b1f84001b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-12-4c0ed7fcb75770d8790575b586bf14f4 @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea b/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea new file mode 100644 index 0000000000000..2af0b9ed3a68c --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-13-44fc74c1993062c0a9522199ff27fea @@ -0,0 +1 @@ +true true true true true true true true true true diff --git a/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b b/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b new file mode 100644 index 0000000000000..d8dfbf60007bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-14-4855a66124b16d1d0d003235995ac06b @@ -0,0 +1 @@ +2001-01-28 2001-02-28 2001-03-28 2001-04-28 2001-05-28 2001-06-28 2001-07-28 2001-08-28 2001-09-28 2001-10-28 2001-11-28 2001-12-28 diff --git a/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b b/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b new file mode 100644 index 0000000000000..4f6a1bc4273e0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-15-8bc190dba0f641840b5e1e198a14c55b @@ -0,0 +1 @@ +true true true true true true true true true true true true diff --git a/sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb b/sql/hive/src/test/resources/golden/date_1-16-23edf29bf7376c70d5ecf12720f4b1eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a b/sql/hive/src/test/resources/golden/date_1-2-abdce0c0d14d3fc7441b7c134b02f99a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b new file mode 100644 index 0000000000000..963bc42fdee07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-3-df16364a220ff96a6ea1cd478cbc1d0b @@ -0,0 +1 @@ +2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 new file mode 100644 index 0000000000000..8fb5edae63c6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-4-d964bec7e5632091ab5cb6f6786dbbf9 @@ -0,0 +1 @@ +2011-01-01 1 diff --git a/sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d b/sql/hive/src/test/resources/golden/date_1-5-5e70fc74158fbfca38134174360de12d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b new file mode 100644 index 0000000000000..963bc42fdee07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-6-df16364a220ff96a6ea1cd478cbc1d0b @@ -0,0 +1 @@ +2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 b/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 new file mode 100644 index 0000000000000..8fb5edae63c6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-7-d964bec7e5632091ab5cb6f6786dbbf9 @@ -0,0 +1 @@ +2011-01-01 1 diff --git a/sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d b/sql/hive/src/test/resources/golden/date_1-8-1d5c58095cd52ea539d869f2ab1ab67d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b b/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b new file mode 100644 index 0000000000000..963bc42fdee07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_1-9-df16364a220ff96a6ea1cd478cbc1d0b @@ -0,0 +1 @@ +2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba b/sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba new file mode 100644 index 0000000000000..db973ab292d5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_2-3-eedb73e0a622c2ab760b524f395dd4ba @@ -0,0 +1,137 @@ +2010-10-20 7291 +2010-10-20 3198 +2010-10-20 3014 +2010-10-20 2630 +2010-10-20 1610 +2010-10-20 1599 +2010-10-20 1531 +2010-10-20 1142 +2010-10-20 1064 +2010-10-20 897 +2010-10-20 361 +2010-10-21 7291 +2010-10-21 3198 +2010-10-21 3014 +2010-10-21 2646 +2010-10-21 2630 +2010-10-21 1610 +2010-10-21 1599 +2010-10-21 1531 +2010-10-21 1142 +2010-10-21 1064 +2010-10-21 897 +2010-10-21 361 +2010-10-22 3198 +2010-10-22 3014 +2010-10-22 2646 +2010-10-22 2630 +2010-10-22 1610 +2010-10-22 1599 +2010-10-22 1531 +2010-10-22 1142 +2010-10-22 1064 +2010-10-22 897 +2010-10-22 361 +2010-10-23 7274 +2010-10-23 5917 +2010-10-23 5904 +2010-10-23 5832 +2010-10-23 3171 +2010-10-23 3085 +2010-10-23 2932 +2010-10-23 1805 +2010-10-23 650 +2010-10-23 426 +2010-10-23 384 +2010-10-23 272 +2010-10-24 7282 +2010-10-24 3198 +2010-10-24 3014 +2010-10-24 2646 +2010-10-24 2630 +2010-10-24 2571 +2010-10-24 2254 +2010-10-24 1610 +2010-10-24 1599 +2010-10-24 1531 +2010-10-24 897 +2010-10-24 361 +2010-10-25 7291 +2010-10-25 3198 +2010-10-25 3014 +2010-10-25 2646 +2010-10-25 2630 +2010-10-25 1610 +2010-10-25 1599 +2010-10-25 1531 +2010-10-25 1142 +2010-10-25 1064 +2010-10-25 897 +2010-10-25 361 +2010-10-26 7291 +2010-10-26 3198 +2010-10-26 3014 +2010-10-26 2662 +2010-10-26 2646 +2010-10-26 2630 +2010-10-26 1610 +2010-10-26 1599 +2010-10-26 1531 +2010-10-26 1142 +2010-10-26 1064 +2010-10-26 897 +2010-10-26 361 +2010-10-27 7291 +2010-10-27 3198 +2010-10-27 3014 +2010-10-27 2630 +2010-10-27 1610 +2010-10-27 1599 +2010-10-27 1531 +2010-10-27 1142 +2010-10-27 1064 +2010-10-27 897 +2010-10-27 361 +2010-10-28 7291 +2010-10-28 3198 +2010-10-28 3014 +2010-10-28 2646 +2010-10-28 2630 +2010-10-28 1610 +2010-10-28 1599 +2010-10-28 1531 +2010-10-28 1142 +2010-10-28 1064 +2010-10-28 897 +2010-10-28 361 +2010-10-29 7291 +2010-10-29 3198 +2010-10-29 3014 +2010-10-29 2646 +2010-10-29 2630 +2010-10-29 1610 +2010-10-29 1599 +2010-10-29 1531 +2010-10-29 1142 +2010-10-29 1064 +2010-10-29 897 +2010-10-29 361 +2010-10-30 5917 +2010-10-30 5904 +2010-10-30 3171 +2010-10-30 3085 +2010-10-30 2932 +2010-10-30 2018 +2010-10-30 1805 +2010-10-30 650 +2010-10-30 426 +2010-10-30 384 +2010-10-30 272 +2010-10-31 7282 +2010-10-31 3198 +2010-10-31 2571 +2010-10-31 1610 +2010-10-31 1599 +2010-10-31 1531 +2010-10-31 897 +2010-10-31 361 diff --git a/sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 b/sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 new file mode 100644 index 0000000000000..1b0ea7b9eec84 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_2-4-3618dfde8da7c26f03bca72970db9ef7 @@ -0,0 +1,137 @@ +2010-10-31 361 +2010-10-31 897 +2010-10-31 1531 +2010-10-31 1599 +2010-10-31 1610 +2010-10-31 2571 +2010-10-31 3198 +2010-10-31 7282 +2010-10-30 272 +2010-10-30 384 +2010-10-30 426 +2010-10-30 650 +2010-10-30 1805 +2010-10-30 2018 +2010-10-30 2932 +2010-10-30 3085 +2010-10-30 3171 +2010-10-30 5904 +2010-10-30 5917 +2010-10-29 361 +2010-10-29 897 +2010-10-29 1064 +2010-10-29 1142 +2010-10-29 1531 +2010-10-29 1599 +2010-10-29 1610 +2010-10-29 2630 +2010-10-29 2646 +2010-10-29 3014 +2010-10-29 3198 +2010-10-29 7291 +2010-10-28 361 +2010-10-28 897 +2010-10-28 1064 +2010-10-28 1142 +2010-10-28 1531 +2010-10-28 1599 +2010-10-28 1610 +2010-10-28 2630 +2010-10-28 2646 +2010-10-28 3014 +2010-10-28 3198 +2010-10-28 7291 +2010-10-27 361 +2010-10-27 897 +2010-10-27 1064 +2010-10-27 1142 +2010-10-27 1531 +2010-10-27 1599 +2010-10-27 1610 +2010-10-27 2630 +2010-10-27 3014 +2010-10-27 3198 +2010-10-27 7291 +2010-10-26 361 +2010-10-26 897 +2010-10-26 1064 +2010-10-26 1142 +2010-10-26 1531 +2010-10-26 1599 +2010-10-26 1610 +2010-10-26 2630 +2010-10-26 2646 +2010-10-26 2662 +2010-10-26 3014 +2010-10-26 3198 +2010-10-26 7291 +2010-10-25 361 +2010-10-25 897 +2010-10-25 1064 +2010-10-25 1142 +2010-10-25 1531 +2010-10-25 1599 +2010-10-25 1610 +2010-10-25 2630 +2010-10-25 2646 +2010-10-25 3014 +2010-10-25 3198 +2010-10-25 7291 +2010-10-24 361 +2010-10-24 897 +2010-10-24 1531 +2010-10-24 1599 +2010-10-24 1610 +2010-10-24 2254 +2010-10-24 2571 +2010-10-24 2630 +2010-10-24 2646 +2010-10-24 3014 +2010-10-24 3198 +2010-10-24 7282 +2010-10-23 272 +2010-10-23 384 +2010-10-23 426 +2010-10-23 650 +2010-10-23 1805 +2010-10-23 2932 +2010-10-23 3085 +2010-10-23 3171 +2010-10-23 5832 +2010-10-23 5904 +2010-10-23 5917 +2010-10-23 7274 +2010-10-22 361 +2010-10-22 897 +2010-10-22 1064 +2010-10-22 1142 +2010-10-22 1531 +2010-10-22 1599 +2010-10-22 1610 +2010-10-22 2630 +2010-10-22 2646 +2010-10-22 3014 +2010-10-22 3198 +2010-10-21 361 +2010-10-21 897 +2010-10-21 1064 +2010-10-21 1142 +2010-10-21 1531 +2010-10-21 1599 +2010-10-21 1610 +2010-10-21 2630 +2010-10-21 2646 +2010-10-21 3014 +2010-10-21 3198 +2010-10-21 7291 +2010-10-20 361 +2010-10-20 897 +2010-10-20 1064 +2010-10-20 1142 +2010-10-20 1531 +2010-10-20 1599 +2010-10-20 1610 +2010-10-20 2630 +2010-10-20 3014 +2010-10-20 3198 +2010-10-20 7291 diff --git a/sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c b/sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c new file mode 100644 index 0000000000000..0f2a6f7a99237 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_2-5-fe9bebfc8994ddd8d7cd0208c1f0af3c @@ -0,0 +1,12 @@ +2010-10-20 11 +2010-10-21 12 +2010-10-22 11 +2010-10-23 12 +2010-10-24 12 +2010-10-25 12 +2010-10-26 13 +2010-10-27 11 +2010-10-28 12 +2010-10-29 12 +2010-10-30 11 +2010-10-31 8 diff --git a/sql/hive/src/test/resources/golden/date_2-6-f4edce7cb20f325e8b69e787b2ae8882 b/sql/hive/src/test/resources/golden/date_2-6-f4edce7cb20f325e8b69e787b2ae8882 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 b/sql/hive/src/test/resources/golden/date_3-3-4cf49e71b636df754871a675f9e4e24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f b/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f new file mode 100644 index 0000000000000..66d2220d06de2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_3-4-e009f358964f6d1236cfc03283e2b06f @@ -0,0 +1 @@ +1 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 b/sql/hive/src/test/resources/golden/date_3-5-c26de4559926ddb0127d2dc5ea154774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-0-b84f7e931d710dcbe3c5126d998285a8 b/sql/hive/src/test/resources/golden/date_4-0-b84f7e931d710dcbe3c5126d998285a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-1-6272f5e518f6a20bc96a5870ff315c4f b/sql/hive/src/test/resources/golden/date_4-1-6272f5e518f6a20bc96a5870ff315c4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-2-4a0e7bde447ef616b98e0f55d2886de0 b/sql/hive/src/test/resources/golden/date_4-2-4a0e7bde447ef616b98e0f55d2886de0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-3-a23faa56b5d3ca9063a21f72b4278b00 b/sql/hive/src/test/resources/golden/date_4-3-a23faa56b5d3ca9063a21f72b4278b00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 b/sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 new file mode 100644 index 0000000000000..b61affde4ffce --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_4-4-bee09a7384666043621f68297cee2e68 @@ -0,0 +1 @@ +2011-01-01 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/date_4-5-b84f7e931d710dcbe3c5126d998285a8 b/sql/hive/src/test/resources/golden/date_4-5-b84f7e931d710dcbe3c5126d998285a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 b/sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-0-69eec445bd045c9dc899fafa348d8495 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 b/sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-1-fcc400871a502009c8680509e3869ec1 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 b/sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-10-a9f2560c273163e11306d4f1dd1d9d54 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 b/sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-11-4a7bac9ddcf40db6329faaec8e426543 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 b/sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-2-b8598a4d0c948c2ddcf3eeef0abf2264 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 b/sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-3-14d35f266be9cceb11a2ae09ec8b3835 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 b/sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-4-c8865b14d53f2c2496fb69ee8191bf37 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c b/sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-5-f2c907e64da8166a731ddc0ed19bad6c @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 b/sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-6-5606505a92bad10023ad9a3ef77eacc9 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb b/sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-7-47913d4aaf0d468ab3764cc3bfd68eb @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c b/sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-8-1e5ce4f833b6fba45618437c8fb7643c @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c b/sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_comparison-9-bcd987341fc1c38047a27d29dac6ae7c @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 b/sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 new file mode 100644 index 0000000000000..b7305b903edca --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_join1-3-f71c7be760fb4de4eff8225f2c6614b2 @@ -0,0 +1,22 @@ +1064 2010-10-20 1064 2010-10-20 +1142 2010-10-21 1142 2010-10-21 +1599 2010-10-22 1599 2010-10-22 +361 2010-10-23 361 2010-10-23 +897 2010-10-24 897 2010-10-24 +1531 2010-10-25 1531 2010-10-25 +1610 2010-10-26 1610 2010-10-26 +3198 2010-10-27 3198 2010-10-27 +1064 2010-10-28 1064 2010-10-28 +1142 2010-10-29 1142 2010-10-29 +1064 2000-11-20 1064 2000-11-20 +1142 2000-11-21 1142 2000-11-21 +1599 2000-11-22 1599 2000-11-22 +361 2000-11-23 361 2000-11-23 +897 2000-11-24 897 2000-11-24 +1531 2000-11-25 1531 2000-11-25 +1610 2000-11-26 1610 2000-11-26 +3198 2000-11-27 3198 2000-11-27 +1064 2000-11-28 1064 2000-11-28 +1142 2000-11-28 1064 2000-11-28 +1064 2000-11-28 1142 2000-11-28 +1142 2000-11-28 1142 2000-11-28 diff --git a/sql/hive/src/test/resources/golden/date_join1-4-70b9b49c55699fe94cfde069f5d197c b/sql/hive/src/test/resources/golden/date_join1-4-70b9b49c55699fe94cfde069f5d197c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-10-d80e681519dcd8f5078c5602bb5befa9 b/sql/hive/src/test/resources/golden/date_serde-10-d80e681519dcd8f5078c5602bb5befa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-11-29540200936bba47f17553547b409af7 b/sql/hive/src/test/resources/golden/date_serde-11-29540200936bba47f17553547b409af7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-12-c3c3275658b89d31fc504db31ae9f99c b/sql/hive/src/test/resources/golden/date_serde-12-c3c3275658b89d31fc504db31ae9f99c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 b/sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-13-6c546456c81e635b6753e1552fac9129 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 b/sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-14-f8ba18cc7b0225b4022299c44d435101 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-15-66fadc9bcea7d107a610758aa6f50ff3 b/sql/hive/src/test/resources/golden/date_serde-15-66fadc9bcea7d107a610758aa6f50ff3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-16-1bd3345b46f77e17810978e56f9f7c6b b/sql/hive/src/test/resources/golden/date_serde-16-1bd3345b46f77e17810978e56f9f7c6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-17-a0df43062f8ab676ef728c9968443f12 b/sql/hive/src/test/resources/golden/date_serde-17-a0df43062f8ab676ef728c9968443f12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a b/sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-18-b50ecc72ce9018ab12fb17568fef038a @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 b/sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-19-28f1cf92bdd6b2e5d328cd9d10f828b6 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-20-588516368d8c1533cb7bfb2157fd58c1 b/sql/hive/src/test/resources/golden/date_serde-20-588516368d8c1533cb7bfb2157fd58c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-21-dfe166fe053468e738dca23ebe043091 b/sql/hive/src/test/resources/golden/date_serde-21-dfe166fe053468e738dca23ebe043091 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-22-45240a488fb708e432d2f45b74ef7e63 b/sql/hive/src/test/resources/golden/date_serde-22-45240a488fb708e432d2f45b74ef7e63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 b/sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-23-1742a51e4967a8d263572d890cd8d4a8 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b b/sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-24-14fd49bd6fee907c1699f7b4e26685b @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-25-a199cf185184a25190d65c123d0694ee b/sql/hive/src/test/resources/golden/date_serde-25-a199cf185184a25190d65c123d0694ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-26-c5fa68d9aff36f22e5edc1b54332d0ab b/sql/hive/src/test/resources/golden/date_serde-26-c5fa68d9aff36f22e5edc1b54332d0ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-27-4d86c79f858866acec3c37f6598c2638 b/sql/hive/src/test/resources/golden/date_serde-27-4d86c79f858866acec3c37f6598c2638 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb b/sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-28-16a41fc9e0f51eb417c763bae8e9cadb @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 b/sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-29-bd1cb09aacd906527b0bbf43bbded812 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-30-7c80741f9f485729afc68609c55423a0 b/sql/hive/src/test/resources/golden/date_serde-30-7c80741f9f485729afc68609c55423a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-31-da36cd1654aee055cb3650133c9d11f b/sql/hive/src/test/resources/golden/date_serde-31-da36cd1654aee055cb3650133c9d11f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 b/sql/hive/src/test/resources/golden/date_serde-32-bb2f76bd307ed616a3c797f8dd45a8d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 b/sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-33-a742813b024e6dcfb4a358aa4e9fcdb6 @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f b/sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f new file mode 100644 index 0000000000000..9f2238d57d6f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-34-6485841336c097895ad5b34f42c0745f @@ -0,0 +1 @@ +2010-10-20 1064 diff --git a/sql/hive/src/test/resources/golden/date_serde-35-8651a7c351cbc07fb1af6193f6885de8 b/sql/hive/src/test/resources/golden/date_serde-35-8651a7c351cbc07fb1af6193f6885de8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-36-36e6041f53433482631018410bb62a99 b/sql/hive/src/test/resources/golden/date_serde-36-36e6041f53433482631018410bb62a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-37-3ddfd8ecb28991aeed588f1ea852c427 b/sql/hive/src/test/resources/golden/date_serde-37-3ddfd8ecb28991aeed588f1ea852c427 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-38-e6167e27465514356c557a77d956ea46 b/sql/hive/src/test/resources/golden/date_serde-38-e6167e27465514356c557a77d956ea46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-39-c1e17c93582656c12970c37bac153bf2 b/sql/hive/src/test/resources/golden/date_serde-39-c1e17c93582656c12970c37bac153bf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-40-4a17944b9ec8999bb20c5ba5d4cb877c b/sql/hive/src/test/resources/golden/date_serde-40-4a17944b9ec8999bb20c5ba5d4cb877c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf b/sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf new file mode 100644 index 0000000000000..16c03e7276fec --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-8-cace4f60a08342f58fbe816a9c3a73cf @@ -0,0 +1,137 @@ +Baltimore New York 2010-10-20 -30.0 1064 +Baltimore New York 2010-10-20 23.0 1142 +Baltimore New York 2010-10-20 6.0 1599 +Chicago New York 2010-10-20 42.0 361 +Chicago New York 2010-10-20 24.0 897 +Chicago New York 2010-10-20 15.0 1531 +Chicago New York 2010-10-20 -6.0 1610 +Chicago New York 2010-10-20 -2.0 3198 +Baltimore New York 2010-10-21 17.0 1064 +Baltimore New York 2010-10-21 105.0 1142 +Baltimore New York 2010-10-21 28.0 1599 +Chicago New York 2010-10-21 142.0 361 +Chicago New York 2010-10-21 77.0 897 +Chicago New York 2010-10-21 53.0 1531 +Chicago New York 2010-10-21 -5.0 1610 +Chicago New York 2010-10-21 51.0 3198 +Baltimore New York 2010-10-22 -12.0 1064 +Baltimore New York 2010-10-22 54.0 1142 +Baltimore New York 2010-10-22 18.0 1599 +Chicago New York 2010-10-22 2.0 361 +Chicago New York 2010-10-22 24.0 897 +Chicago New York 2010-10-22 16.0 1531 +Chicago New York 2010-10-22 -6.0 1610 +Chicago New York 2010-10-22 -11.0 3198 +Baltimore New York 2010-10-23 18.0 272 +Baltimore New York 2010-10-23 -10.0 1805 +Baltimore New York 2010-10-23 6.0 3171 +Chicago New York 2010-10-23 3.0 384 +Chicago New York 2010-10-23 32.0 426 +Chicago New York 2010-10-23 1.0 650 +Chicago New York 2010-10-23 11.0 3085 +Baltimore New York 2010-10-24 12.0 1599 +Baltimore New York 2010-10-24 20.0 2571 +Chicago New York 2010-10-24 10.0 361 +Chicago New York 2010-10-24 113.0 897 +Chicago New York 2010-10-24 -5.0 1531 +Chicago New York 2010-10-24 -17.0 1610 +Chicago New York 2010-10-24 -3.0 3198 +Baltimore New York 2010-10-25 -25.0 1064 +Baltimore New York 2010-10-25 92.0 1142 +Baltimore New York 2010-10-25 106.0 1599 +Chicago New York 2010-10-25 31.0 361 +Chicago New York 2010-10-25 -1.0 897 +Chicago New York 2010-10-25 43.0 1531 +Chicago New York 2010-10-25 6.0 1610 +Chicago New York 2010-10-25 -16.0 3198 +Baltimore New York 2010-10-26 -22.0 1064 +Baltimore New York 2010-10-26 123.0 1142 +Baltimore New York 2010-10-26 90.0 1599 +Chicago New York 2010-10-26 12.0 361 +Chicago New York 2010-10-26 0.0 897 +Chicago New York 2010-10-26 29.0 1531 +Chicago New York 2010-10-26 -17.0 1610 +Chicago New York 2010-10-26 6.0 3198 +Baltimore New York 2010-10-27 -18.0 1064 +Baltimore New York 2010-10-27 49.0 1142 +Baltimore New York 2010-10-27 92.0 1599 +Chicago New York 2010-10-27 148.0 361 +Chicago New York 2010-10-27 -11.0 897 +Chicago New York 2010-10-27 70.0 1531 +Chicago New York 2010-10-27 8.0 1610 +Chicago New York 2010-10-27 21.0 3198 +Baltimore New York 2010-10-28 -4.0 1064 +Baltimore New York 2010-10-28 -14.0 1142 +Baltimore New York 2010-10-28 -14.0 1599 +Chicago New York 2010-10-28 2.0 361 +Chicago New York 2010-10-28 2.0 897 +Chicago New York 2010-10-28 -11.0 1531 +Chicago New York 2010-10-28 3.0 1610 +Chicago New York 2010-10-28 -18.0 3198 +Baltimore New York 2010-10-29 -24.0 1064 +Baltimore New York 2010-10-29 21.0 1142 +Baltimore New York 2010-10-29 -2.0 1599 +Chicago New York 2010-10-29 -12.0 361 +Chicago New York 2010-10-29 -11.0 897 +Chicago New York 2010-10-29 15.0 1531 +Chicago New York 2010-10-29 -18.0 1610 +Chicago New York 2010-10-29 -4.0 3198 +Baltimore New York 2010-10-30 14.0 272 +Baltimore New York 2010-10-30 -1.0 1805 +Baltimore New York 2010-10-30 5.0 3171 +Chicago New York 2010-10-30 -6.0 384 +Chicago New York 2010-10-30 -10.0 426 +Chicago New York 2010-10-30 -5.0 650 +Chicago New York 2010-10-30 -5.0 3085 +Baltimore New York 2010-10-31 -1.0 1599 +Baltimore New York 2010-10-31 -14.0 2571 +Chicago New York 2010-10-31 -25.0 361 +Chicago New York 2010-10-31 -18.0 897 +Chicago New York 2010-10-31 -4.0 1531 +Chicago New York 2010-10-31 -22.0 1610 +Chicago New York 2010-10-31 -15.0 3198 +Cleveland New York 2010-10-30 -23.0 2018 +Cleveland New York 2010-10-30 -12.0 2932 +Cleveland New York 2010-10-29 -4.0 2630 +Cleveland New York 2010-10-29 -19.0 2646 +Cleveland New York 2010-10-29 -12.0 3014 +Cleveland New York 2010-10-28 3.0 2630 +Cleveland New York 2010-10-28 -6.0 2646 +Cleveland New York 2010-10-28 1.0 3014 +Cleveland New York 2010-10-27 16.0 2630 +Cleveland New York 2010-10-27 27.0 3014 +Cleveland New York 2010-10-26 4.0 2630 +Cleveland New York 2010-10-26 -27.0 2646 +Cleveland New York 2010-10-26 -11.0 2662 +Cleveland New York 2010-10-26 13.0 3014 +Cleveland New York 2010-10-25 -4.0 2630 +Cleveland New York 2010-10-25 81.0 2646 +Cleveland New York 2010-10-25 42.0 3014 +Cleveland New York 2010-10-24 5.0 2254 +Cleveland New York 2010-10-24 -11.0 2630 +Cleveland New York 2010-10-24 -20.0 2646 +Cleveland New York 2010-10-24 -9.0 3014 +Cleveland New York 2010-10-23 -21.0 2932 +Cleveland New York 2010-10-22 1.0 2630 +Cleveland New York 2010-10-22 -25.0 2646 +Cleveland New York 2010-10-22 -3.0 3014 +Cleveland New York 2010-10-21 3.0 2630 +Cleveland New York 2010-10-21 29.0 2646 +Cleveland New York 2010-10-21 72.0 3014 +Cleveland New York 2010-10-20 -8.0 2630 +Cleveland New York 2010-10-20 -15.0 3014 +Washington New York 2010-10-23 -25.0 5832 +Washington New York 2010-10-23 -21.0 5904 +Washington New York 2010-10-23 -18.0 5917 +Washington New York 2010-10-30 -27.0 5904 +Washington New York 2010-10-30 -16.0 5917 +Washington New York 2010-10-20 -2.0 7291 +Washington New York 2010-10-21 22.0 7291 +Washington New York 2010-10-23 -16.0 7274 +Washington New York 2010-10-24 -26.0 7282 +Washington New York 2010-10-25 9.0 7291 +Washington New York 2010-10-26 4.0 7291 +Washington New York 2010-10-27 26.0 7291 +Washington New York 2010-10-28 45.0 7291 +Washington New York 2010-10-29 1.0 7291 +Washington New York 2010-10-31 -18.0 7282 diff --git a/sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 b/sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 new file mode 100644 index 0000000000000..0f2a6f7a99237 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_serde-9-436c3c61cc4278b54ac79c53c88ff422 @@ -0,0 +1,12 @@ +2010-10-20 11 +2010-10-21 12 +2010-10-22 11 +2010-10-23 12 +2010-10-24 12 +2010-10-25 12 +2010-10-26 13 +2010-10-27 11 +2010-10-28 12 +2010-10-29 12 +2010-10-30 11 +2010-10-31 8 diff --git a/sql/hive/src/test/resources/golden/date_udf-0-84604a42a5d7f2842f1eec10c689d447 b/sql/hive/src/test/resources/golden/date_udf-0-84604a42a5d7f2842f1eec10c689d447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-1-5e8136f6a6503ae9bef9beca80fada13 b/sql/hive/src/test/resources/golden/date_udf-1-5e8136f6a6503ae9bef9beca80fada13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b b/sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b new file mode 100644 index 0000000000000..83c33400edb47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-10-988ad9744096a29a3672a2d4c121299b @@ -0,0 +1 @@ +0 3333 -3333 -3332 3332 diff --git a/sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc b/sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc new file mode 100644 index 0000000000000..4a2462bb3929b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-11-a5100dd42201b5bc035a9d684cc21bdc @@ -0,0 +1 @@ +NULL 2011 5 6 6 18 2011-05-06 diff --git a/sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff b/sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff new file mode 100644 index 0000000000000..19497254f8f7e --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-12-eb7280a1f191344a99eaa0f805e8faff @@ -0,0 +1 @@ +2011-05-11 2011-04-26 diff --git a/sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 b/sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 new file mode 100644 index 0000000000000..977f0d24c58cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-13-cc99e4f14fd092994b006ee7ebe4fc92 @@ -0,0 +1 @@ +0 3333 -3333 -3333 3333 diff --git a/sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 b/sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 new file mode 100644 index 0000000000000..44d1f45e4eb73 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-14-a6a5ce5134cc1125355a4bdf0a73d97 @@ -0,0 +1 @@ +1970-01-01 08:00:00 1969-12-31 16:00:00 2013-06-19 07:00:00 2013-06-18 17:00:00 diff --git a/sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 b/sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 new file mode 100644 index 0000000000000..645b71d8d61e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-15-d031ee50c119d7c6acafd53543dbd0c4 @@ -0,0 +1 @@ +true true true true diff --git a/sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 b/sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 new file mode 100644 index 0000000000000..51863e9a14e4b --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-16-dc59f69e1685e8d923b187ec50d80f06 @@ -0,0 +1 @@ +2010-10-20 diff --git a/sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 b/sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 new file mode 100644 index 0000000000000..4043ee1cbdd40 --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-17-7d046d4efc568049cf3792470b6feab9 @@ -0,0 +1 @@ +2010-10-31 diff --git a/sql/hive/src/test/resources/golden/date_udf-18-84604a42a5d7f2842f1eec10c689d447 b/sql/hive/src/test/resources/golden/date_udf-18-84604a42a5d7f2842f1eec10c689d447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-19-5e8136f6a6503ae9bef9beca80fada13 b/sql/hive/src/test/resources/golden/date_udf-19-5e8136f6a6503ae9bef9beca80fada13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-2-10e337c34d1e82a360b8599988f4b266 b/sql/hive/src/test/resources/golden/date_udf-2-10e337c34d1e82a360b8599988f4b266 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-20-10e337c34d1e82a360b8599988f4b266 b/sql/hive/src/test/resources/golden/date_udf-20-10e337c34d1e82a360b8599988f4b266 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-3-29e406e613c0284b3e16a8943a4d31bd b/sql/hive/src/test/resources/golden/date_udf-3-29e406e613c0284b3e16a8943a4d31bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-4-23653315213f578856ab5c3bd80c0264 b/sql/hive/src/test/resources/golden/date_udf-4-23653315213f578856ab5c3bd80c0264 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-5-891fd92a4787b9789f6d1f51c1eddc8a b/sql/hive/src/test/resources/golden/date_udf-5-891fd92a4787b9789f6d1f51c1eddc8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-6-3473c118d20783eafb456043a2ee5d5b b/sql/hive/src/test/resources/golden/date_udf-6-3473c118d20783eafb456043a2ee5d5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-7-9fb5165824e161074565e7500959c1b2 b/sql/hive/src/test/resources/golden/date_udf-7-9fb5165824e161074565e7500959c1b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 b/sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 new file mode 100644 index 0000000000000..18d17ea11b53e --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-8-badfe833681362092fc6345f888b1c21 @@ -0,0 +1 @@ +1304665200 2011 5 6 6 18 2011-05-06 diff --git a/sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 b/sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 new file mode 100644 index 0000000000000..19497254f8f7e --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_udf-9-a8cbb039661d796beaa0d1564c58c563 @@ -0,0 +1 @@ +2011-05-11 2011-04-26 diff --git a/sql/hive/src/test/resources/golden/partition_date-0-7ec1f3a845e2c49191460e15af30aa30 b/sql/hive/src/test/resources/golden/partition_date-0-7ec1f3a845e2c49191460e15af30aa30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-1-916193405ce5e020dcd32c58325db6fe b/sql/hive/src/test/resources/golden/partition_date-1-916193405ce5e020dcd32c58325db6fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 b/sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-10-a8dde9c0b5746dd770c9c262d23ffb10 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 b/sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 new file mode 100644 index 0000000000000..b4de394767536 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-11-fdface2fb6eef67f15bb7d0de2294957 @@ -0,0 +1 @@ +11 diff --git a/sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 b/sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 new file mode 100644 index 0000000000000..64bb6b746dcea --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-12-9b945f8ece6e09ad28c866ff3a10cc24 @@ -0,0 +1 @@ +30 diff --git a/sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf b/sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-13-b7cb91c7c459798078a79071d329dbf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce b/sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-14-e4366325f3a0c4a8e92be59f4de73fce @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f b/sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-15-a062a6e87867d8c8cfbdad97bedcbe5f @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c b/sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-16-22a5627d9ac112665eae01d07a91c89c @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e b/sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-17-b9ce94ef93cb16d629af7d7f8ee637e @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 b/sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-18-72c6e9a4e0b434cef67144825346c687 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 b/sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-19-44e5165eb210559e420105073bc96125 @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 b/sql/hive/src/test/resources/golden/partition_date-2-e2e70ac0f4e0ea987b49b86f73d819c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-20-7ec1f3a845e2c49191460e15af30aa30 b/sql/hive/src/test/resources/golden/partition_date-20-7ec1f3a845e2c49191460e15af30aa30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-3-c938b08f57d588926a5d5fbfa4531012 b/sql/hive/src/test/resources/golden/partition_date-3-c938b08f57d588926a5d5fbfa4531012 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 b/sql/hive/src/test/resources/golden/partition_date-4-a93eff99ce43bb939ec1d6464c0ef0b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-5-a855aba47876561fd4fb095e09580686 b/sql/hive/src/test/resources/golden/partition_date-5-a855aba47876561fd4fb095e09580686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc b/sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc new file mode 100644 index 0000000000000..051ca3d3c28e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-6-1405c311915f27b0cc616c83d39eaacc @@ -0,0 +1,2 @@ +2000-01-01 +2013-08-08 diff --git a/sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 b/sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 new file mode 100644 index 0000000000000..24192eefd2caf --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-7-2ac950d8d5656549dd453e5464cb8530 @@ -0,0 +1,5 @@ +165 val_165 2000-01-01 2 +238 val_238 2000-01-01 2 +27 val_27 2000-01-01 2 +311 val_311 2000-01-01 2 +86 val_86 2000-01-01 2 diff --git a/sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 b/sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-8-a425c11c12c9ce4c9c43d4fbccee5347 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f b/sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-9-aad6078a09b7bd8f5141437e86bb229f @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac b/sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac new file mode 100644 index 0000000000000..91ba621412d72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_type_check-12-7e053ba4f9dea1e74c1d04c557c3adac @@ -0,0 +1,6 @@ +1 11 2008-01-01 +2 12 2008-01-01 +3 13 2008-01-01 +7 17 2008-01-01 +8 18 2008-01-01 +8 28 2008-01-01 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-13-45fb706ff448da1fe609c7ff76a80d4d b/sql/hive/src/test/resources/golden/partition_type_check-13-45fb706ff448da1fe609c7ff76a80d4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd b/sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd new file mode 100644 index 0000000000000..7941f53d8d4c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_date-6-f4d5c71145a9b7464685aa7d09cd4dfd @@ -0,0 +1,40 @@ +1064 2000-11-20 +1064 2000-11-20 +1142 2000-11-21 +1142 2000-11-21 +1599 2000-11-22 +1599 2000-11-22 +361 2000-11-23 +361 2000-11-23 +897 2000-11-24 +897 2000-11-24 +1531 2000-11-25 +1531 2000-11-25 +1610 2000-11-26 +1610 2000-11-26 +3198 2000-11-27 +3198 2000-11-27 +1064 2000-11-28 +1064 2000-11-28 +1142 2000-11-28 +1142 2000-11-28 +1064 2010-10-20 +1064 2010-10-20 +1142 2010-10-21 +1142 2010-10-21 +1599 2010-10-22 +1599 2010-10-22 +361 2010-10-23 +361 2010-10-23 +897 2010-10-24 +897 2010-10-24 +1531 2010-10-25 +1531 2010-10-25 +1610 2010-10-26 +1610 2010-10-26 +3198 2010-10-27 +3198 2010-10-27 +1064 2010-10-28 +1064 2010-10-28 +1142 2010-10-29 +1142 2010-10-29 diff --git a/sql/hive/src/test/resources/golden/union_date-7-a0bade1c77338d4f72962389a1f5bea2 b/sql/hive/src/test/resources/golden/union_date-7-a0bade1c77338d4f72962389a1f5bea2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-8-21306adbd8be8ad75174ad9d3e42b73c b/sql/hive/src/test/resources/golden/union_date-8-21306adbd8be8ad75174ad9d3e42b73c new file mode 100644 index 0000000000000..e69de29bb2d1d From 56102dc2d849c221f325a7888cd66abb640ec887 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 13 Oct 2014 13:36:39 -0700 Subject: [PATCH 0980/1492] [SPARK-2066][SQL] Adds checks for non-aggregate attributes with aggregation This PR adds a new rule `CheckAggregation` to the analyzer to provide better error message for non-aggregate attributes with aggregation. Author: Cheng Lian Closes #2774 from liancheng/non-aggregate-attr and squashes the following commits: 5246004 [Cheng Lian] Passes test suites bf1878d [Cheng Lian] Adds checks for non-aggregate attributes with aggregation --- .../sql/catalyst/analysis/Analyzer.scala | 36 ++++++++++++++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 26 ++++++++++++++ 2 files changed, 57 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fe83eb12502dc..82553063145b8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -63,7 +63,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool typeCoercionRules ++ extendedRules : _*), Batch("Check Analysis", Once, - CheckResolution), + CheckResolution, + CheckAggregation), Batch("AnalysisOperators", fixedPoint, EliminateAnalysisOperators) ) @@ -88,6 +89,32 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } + /** + * Checks for non-aggregated attributes with aggregation + */ + object CheckAggregation extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + plan.transform { + case aggregatePlan @ Aggregate(groupingExprs, aggregateExprs, child) => + def isValidAggregateExpression(expr: Expression): Boolean = expr match { + case _: AggregateExpression => true + case e: Attribute => groupingExprs.contains(e) + case e if groupingExprs.contains(e) => true + case e if e.references.isEmpty => true + case e => e.children.forall(isValidAggregateExpression) + } + + aggregateExprs.foreach { e => + if (!isValidAggregateExpression(e)) { + throw new TreeNodeException(plan, s"Expression not in GROUP BY: $e") + } + } + + aggregatePlan + } + } + } + /** * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ @@ -204,18 +231,17 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object UnresolvedHavingClauseAttributes extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) + case filter @ Filter(havingCondition, aggregate @ Aggregate(_, originalAggExprs, _)) if aggregate.resolved && containsAggregate(havingCondition) => { val evaluatedCondition = Alias(havingCondition, "havingCondition")() val aggExprsWithHaving = evaluatedCondition +: originalAggExprs - + Project(aggregate.output, Filter(evaluatedCondition.toAttribute, aggregate.copy(aggregateExpressions = aggExprsWithHaving))) } - } - + protected def containsAggregate(condition: Expression): Boolean = condition .collect { case ae: AggregateExpression => ae } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index a94022c0cf6e3..15f6ba4f72bbd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.joins.BroadcastHashJoin import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll @@ -694,4 +695,29 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAnswer( sql("SELECT CASE WHEN key = 1 THEN 1 ELSE 2 END FROM testData WHERE key = 1 group by key"), 1) } + + test("throw errors for non-aggregate attributes with aggregation") { + def checkAggregation(query: String, isInvalidQuery: Boolean = true) { + val logicalPlan = sql(query).queryExecution.logical + + if (isInvalidQuery) { + val e = intercept[TreeNodeException[LogicalPlan]](sql(query).queryExecution.analyzed) + assert( + e.getMessage.startsWith("Expression not in GROUP BY"), + "Non-aggregate attribute(s) not detected\n" + logicalPlan) + } else { + // Should not throw + sql(query).queryExecution.analyzed + } + } + + checkAggregation("SELECT key, COUNT(*) FROM testData") + checkAggregation("SELECT COUNT(key), COUNT(*) FROM testData", false) + + checkAggregation("SELECT value, COUNT(*) FROM testData GROUP BY key") + checkAggregation("SELECT COUNT(value), SUM(key) FROM testData GROUP BY key", false) + + checkAggregation("SELECT key + 2, COUNT(*) FROM testData GROUP BY key + 1") + checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) + } } From d3cdf9128ae828dc7f1893439f66a0de68c6e527 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 13 Oct 2014 13:40:20 -0700 Subject: [PATCH 0981/1492] [SPARK-3529] [SQL] Delete the temp files after test exit There are lots of temporal files created by TestHive under the /tmp by default, which may cause potential performance issue for testing. This PR will automatically delete them after test exit. Author: Cheng Hao Closes #2393 from chenghao-intel/delete_temp_on_exit and squashes the following commits: 3a6511f [Cheng Hao] Remove the temp dir after text exit --- .../main/scala/org/apache/spark/sql/hive/TestHive.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a4354c1379c63..9a9e2eda6bcd4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ @@ -71,11 +72,14 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { setConf("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastorePath;create=true") setConf("hive.metastore.warehouse.dir", warehousePath) + Utils.registerShutdownDeleteDir(new File(warehousePath)) + Utils.registerShutdownDeleteDir(new File(metastorePath)) } val testTempDir = File.createTempFile("testTempFiles", "spark.hive.tmp") testTempDir.delete() testTempDir.mkdir() + Utils.registerShutdownDeleteDir(testTempDir) // For some hive test case which contain ${system:test.tmp.dir} System.setProperty("test.tmp.dir", testTempDir.getCanonicalPath) @@ -121,8 +125,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") hiveFilesTemp.delete() hiveFilesTemp.mkdir() - hiveFilesTemp.deleteOnExit() - + Utils.registerShutdownDeleteDir(hiveFilesTemp) val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) From 73da9c26b0e2e8bf0ab055906211727a7097c963 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 13 Oct 2014 13:43:41 -0700 Subject: [PATCH 0982/1492] [SPARK-3771][SQL] AppendingParquetOutputFormat should use reflection to prevent from breaking binary-compatibility. Original problem is [SPARK-3764](https://issues.apache.org/jira/browse/SPARK-3764). `AppendingParquetOutputFormat` uses a binary-incompatible method `context.getTaskAttemptID`. This causes binary-incompatible of Spark itself, i.e. if Spark itself is built against hadoop-1, the artifact is for only hadoop-1, and vice versa. Author: Takuya UESHIN Closes #2638 from ueshin/issues/SPARK-3771 and squashes the following commits: efd3784 [Takuya UESHIN] Add a comment to explain the reason to use reflection. ec213c1 [Takuya UESHIN] Use reflection to prevent breaking binary-compatibility. --- .../spark/sql/parquet/ParquetTableOperations.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index ffb732347d30a..1f4237d7ede65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -331,13 +331,21 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) // override to choose output filename so not overwrite existing ones override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val taskId: TaskID = context.getTaskAttemptID.getTaskID + val taskId: TaskID = getTaskAttemptID(context).getTaskID val partition: Int = taskId.getId val filename = s"part-r-${partition + offset}.parquet" val committer: FileOutputCommitter = getOutputCommitter(context).asInstanceOf[FileOutputCommitter] new Path(committer.getWorkPath, filename) } + + // The TaskAttemptContext is a class in hadoop-1 but is an interface in hadoop-2. + // The signatures of the method TaskAttemptContext.getTaskAttemptID for the both versions + // are the same, so the method calls are source-compatible but NOT binary-compatible because + // the opcode of method call for class is INVOKEVIRTUAL and for interface is INVOKEINTERFACE. + private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { + context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] + } } /** From e10d71e7e58bf2ec0f1942cb2f0602396ab866b4 Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Mon, 13 Oct 2014 13:45:34 -0700 Subject: [PATCH 0983/1492] [SPARK-3559][SQL] Remove unnecessary columns from List of needed Column Ids in Hive Conf Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #2713 from gvramana/remove_unnecessary_columns and squashes the following commits: b7ba768 [Venkata Ramana Gollamudi] Added comment and checkstyle fix 6a93459 [Venkata Ramana Gollamudi] cloned hiveconf for each TableScanOperators so that only required columns are added --- .../scala/org/apache/spark/sql/hive/TableReader.scala | 6 ++++-- .../spark/sql/hive/execution/HiveTableScan.scala | 10 ++++++++-- 2 files changed, 12 insertions(+), 4 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 84fafcde63d05..0de29d5cffd0e 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} @@ -52,7 +53,8 @@ private[hive] class HadoopTableReader( @transient attributes: Seq[Attribute], @transient relation: MetastoreRelation, - @transient sc: HiveContext) + @transient sc: HiveContext, + @transient hiveExtraConf: HiveConf) extends TableReader { // Choose the minimum number of splits. If mapred.map.tasks is set, then use that unless @@ -63,7 +65,7 @@ class HadoopTableReader( // TODO: set aws s3 credentials. private val _broadcastedHiveConf = - sc.sparkContext.broadcast(new SerializableWritable(sc.hiveconf)) + sc.sparkContext.broadcast(new SerializableWritable(hiveExtraConf)) def broadcastedHiveConf = _broadcastedHiveConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 577ca928b43b6..a32147584f6f4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -64,8 +64,14 @@ case class HiveTableScan( BindReferences.bindReference(pred, relation.partitionKeys) } + // Create a local copy of hiveconf,so that scan specific modifications should not impact + // other queries @transient - private[this] val hadoopReader = new HadoopTableReader(attributes, relation, context) + private[this] val hiveExtraConf = new HiveConf(context.hiveconf) + + @transient + private[this] val hadoopReader = + new HadoopTableReader(attributes, relation, context, hiveExtraConf) private[this] def castFromString(value: String, dataType: DataType) = { Cast(Literal(value), dataType).eval(null) @@ -97,7 +103,7 @@ case class HiveTableScan( hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(",")) } - addColumnMetadataToConf(context.hiveconf) + addColumnMetadataToConf(hiveExtraConf) /** * Prunes partitions not involve the query plan. From 371321cadee7df39258bd374eb59c1e32451d96b Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 13 Oct 2014 13:46:34 -0700 Subject: [PATCH 0984/1492] [SQL] Add type checking debugging functions Adds some functions that were very useful when trying to track down the bug from #2656. This change also changes the tree output for query plans to include the `'` prefix to unresolved nodes and `!` prefix to nodes that refer to non-existent attributes. Author: Michael Armbrust Closes #2657 from marmbrus/debugging and squashes the following commits: 654b926 [Michael Armbrust] Clean-up, add tests 763af15 [Michael Armbrust] Add typeChecking debugging functions 8c69303 [Michael Armbrust] Add inputSet, references to QueryPlan. Improve tree string with a prefix to denote invalid or unresolved nodes. fbeab54 [Michael Armbrust] Better toString, factories for AttributeSet. --- .../catalyst/expressions/AttributeSet.scala | 23 +++-- .../sql/catalyst/expressions/Projection.scala | 2 + .../expressions/namedExpressions.scala | 4 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 23 +++++ .../catalyst/plans/logical/LogicalPlan.scala | 8 +- .../plans/logical/basicOperators.scala | 5 -- .../spark/sql/execution/debug/package.scala | 85 +++++++++++++++++++ .../sql/execution/debug/DebuggingSuite.scala | 33 +++++++ 8 files changed, 163 insertions(+), 20 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index c3a08bbdb6bc7..2b4969b7cfec0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -17,19 +17,26 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.analysis.Star + protected class AttributeEquals(val a: Attribute) { override def hashCode() = a.exprId.hashCode() - override def equals(other: Any) = other match { - case otherReference: AttributeEquals => a.exprId == otherReference.a.exprId - case otherAttribute => false + override def equals(other: Any) = (a, other.asInstanceOf[AttributeEquals].a) match { + case (a1: AttributeReference, a2: AttributeReference) => a1.exprId == a2.exprId + case (a1, a2) => a1 == a2 } } object AttributeSet { - /** Constructs a new [[AttributeSet]] given a sequence of [[Attribute Attributes]]. */ - def apply(baseSet: Seq[Attribute]) = { - new AttributeSet(baseSet.map(new AttributeEquals(_)).toSet) - } + def apply(a: Attribute) = + new AttributeSet(Set(new AttributeEquals(a))) + + /** Constructs a new [[AttributeSet]] given a sequence of [[Expression Expressions]]. */ + def apply(baseSet: Seq[Expression]) = + new AttributeSet( + baseSet + .flatMap(_.references) + .map(new AttributeEquals(_)).toSet) } /** @@ -103,4 +110,6 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) // We must force toSeq to not be strict otherwise we end up with a [[Stream]] that captures all // sorts of things in its closure. override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq + + override def toString = "{" + baseSet.map(_.a).mkString(", ") + "}" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 204904ecf04db..e7e81a21fdf03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -39,6 +39,8 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { } new GenericRow(outputArray) } + + override def toString = s"Row => [${exprArray.mkString(",")}]" } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index e5a958d599393..d023db44d8543 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -57,6 +57,8 @@ abstract class NamedExpression extends Expression { abstract class Attribute extends NamedExpression { self: Product => + override def references = AttributeSet(this) + def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute def withName(newName: String): Attribute @@ -116,8 +118,6 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { - override def references = AttributeSet(this :: Nil) - override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index af9e4d86e995a..dcbbb62c0aca4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -31,6 +31,25 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy */ def outputSet: AttributeSet = AttributeSet(output) + /** + * All Attributes that appear in expressions from this operator. Note that this set does not + * include attributes that are implicitly referenced by being passed through to the output tuple. + */ + def references: AttributeSet = AttributeSet(expressions.flatMap(_.references)) + + /** + * The set of all attributes that are input to this operator by its children. + */ + def inputSet: AttributeSet = + AttributeSet(children.flatMap(_.asInstanceOf[QueryPlan[PlanType]].output)) + + /** + * Attributes that are referenced by expressions but not provided by this nodes children. + * Subclasses should override this method if they produce attributes internally as it is used by + * assertions designed to prevent the construction of invalid plans. + */ + def missingInput: AttributeSet = references -- inputSet + /** * Runs [[transform]] with `rule` on all expressions present in this query operator. * Users should not expect a specific directionality. If a specific directionality is needed, @@ -132,4 +151,8 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy /** Prints out the schema in the tree format */ def printSchema(): Unit = println(schemaString) + + protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" + + override def simpleString = statePrefix + super.simpleString } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 4f8ad8a7e0223..882e9c6110089 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -53,12 +53,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product) } - /** - * Returns the set of attributes that this node takes as - * input from its children. - */ - lazy val inputSet: AttributeSet = AttributeSet(children.flatMap(_.output)) - /** * Returns true if this expression and all its children have been resolved to a specific schema * and false if it still contains any unresolved placeholders. Implementations of LogicalPlan @@ -68,6 +62,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { */ lazy val resolved: Boolean = !expressions.exists(!_.resolved) && childrenResolved + override protected def statePrefix = if (!resolved) "'" else super.statePrefix + /** * Returns true if all its children of this query plan have been resolved. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index f8e9930ac270d..14b03c7445c13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -138,11 +138,6 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { - /** The set of all AttributeReferences required for this aggregation. */ - def references = - AttributeSet( - groupingExpressions.flatMap(_.references) ++ aggregateExpressions.flatMap(_.references)) - override def output = aggregateExpressions.map(_.toAttribute) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index a9535a750bcd7..61be5ed2db65c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -24,6 +24,7 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext._ import org.apache.spark.sql.{SchemaRDD, Row} import org.apache.spark.sql.catalyst.trees.TreeNodeRef +import org.apache.spark.sql.catalyst.types._ /** * :: DeveloperApi :: @@ -56,6 +57,23 @@ package object debug { case _ => } } + + def typeCheck(): Unit = { + val plan = query.queryExecution.executedPlan + val visited = new collection.mutable.HashSet[TreeNodeRef]() + val debugPlan = plan transform { + case s: SparkPlan if !visited.contains(new TreeNodeRef(s)) => + visited += new TreeNodeRef(s) + TypeCheck(s) + } + try { + println(s"Results returned: ${debugPlan.execute().count()}") + } catch { + case e: Exception => + def unwrap(e: Throwable): Throwable = if (e.getCause == null) e else unwrap(e.getCause) + println(s"Deepest Error: ${unwrap(e)}") + } + } } private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { @@ -115,4 +133,71 @@ package object debug { } } } + + /** + * :: DeveloperApi :: + * Helper functions for checking that runtime types match a given schema. + */ + @DeveloperApi + object TypeCheck { + def typeCheck(data: Any, schema: DataType): Unit = (data, schema) match { + case (null, _) => + + case (row: Row, StructType(fields)) => + row.zip(fields.map(_.dataType)).foreach { case(d,t) => typeCheck(d,t) } + case (s: Seq[_], ArrayType(elemType, _)) => + s.foreach(typeCheck(_, elemType)) + case (m: Map[_, _], MapType(keyType, valueType, _)) => + m.keys.foreach(typeCheck(_, keyType)) + m.values.foreach(typeCheck(_, valueType)) + + case (_: Long, LongType) => + case (_: Int, IntegerType) => + case (_: String, StringType) => + case (_: Float, FloatType) => + case (_: Byte, ByteType) => + case (_: Short, ShortType) => + case (_: Boolean, BooleanType) => + case (_: Double, DoubleType) => + + case (d, t) => sys.error(s"Invalid data found: got $d (${d.getClass}) expected $t") + } + } + + /** + * :: DeveloperApi :: + * Augments SchemaRDDs with debug methods. + */ + @DeveloperApi + private[sql] case class TypeCheck(child: SparkPlan) extends SparkPlan { + import TypeCheck._ + + override def nodeName = "" + + /* Only required when defining this class in a REPL. + override def makeCopy(args: Array[Object]): this.type = + TypeCheck(args(0).asInstanceOf[SparkPlan]).asInstanceOf[this.type] + */ + + def output = child.output + + def children = child :: Nil + + def execute() = { + child.execute().map { row => + try typeCheck(row, child.schema) catch { + case e: Exception => + sys.error( + s""" + |ERROR WHEN TYPE CHECKING QUERY + |============================== + |$e + |======== BAD TREE ============ + |$child + """.stripMargin) + } + row + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala new file mode 100644 index 0000000000000..87c28c334d228 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.debug + +import org.scalatest.FunSuite + +import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext._ + +class DebuggingSuite extends FunSuite { + test("SchemaRDD.debug()") { + testData.debug() + } + + test("SchemaRDD.typeCheck()") { + testData.typeCheck() + } +} \ No newline at end of file From e6e37701f12be82fa77dc28d825ddd36a1ab7594 Mon Sep 17 00:00:00 2001 From: chirag Date: Mon, 13 Oct 2014 13:47:26 -0700 Subject: [PATCH 0985/1492] SPARK-3807: SparkSql does not work for tables created using custom serde SparkSql crashes on selecting tables using custom serde. Example: ---------------- CREATE EXTERNAL TABLE table_name PARTITIONED BY ( a int) ROW FORMAT 'SERDE "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer" with serdeproperties("serialization.format"="org.apache.thrift.protocol.TBinaryProtocol","serialization.class"="ser_class") STORED AS SEQUENCEFILE; The following exception is seen on running a query like 'select * from table_name limit 1': ERROR CliDriver: org.apache.hadoop.hive.serde2.SerDeException: java.lang.NullPointerException at org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer.initialize(ThriftDeserializer.java:68) at org.apache.hadoop.hive.ql.plan.TableDesc.getDeserializer(TableDesc.java:80) at org.apache.spark.sql.hive.execution.HiveTableScan.addColumnMetadataToConf(HiveTableScan.scala:86) at org.apache.spark.sql.hive.execution.HiveTableScan.(HiveTableScan.scala:100) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188) at org.apache.spark.sql.SQLContext$SparkPlanner.pruneFilterProject(SQLContext.scala:364) at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$.apply(HiveStrategies.scala:184) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54) at org.apache.spark.sql.execution.SparkStrategies$BasicOperators$.apply(SparkStrategies.scala:280) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58) at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371) at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59) at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:402) at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:400) at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:406) at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:406) at org.apache.spark.sql.hive.HiveContext$QueryExecution.stringResult(HiveContext.scala:406) at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:59) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:291) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:226) at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source) at java.lang.reflect.Method.invoke(Unknown Source) at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:328) at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:75) at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.lang.NullPointerException Author: chirag Closes #2674 from chiragaggarwal/branch-1.1 and squashes the following commits: 370c31b [chirag] SPARK-3807: Add a test case to validate the fix. 1f26805 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde (Incorporated Review Comments) ba4bc0c [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde 5c73b72 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde (cherry picked from commit 925e22d3132b983a2fcee31e3878b680c7ff92da) Signed-off-by: Michael Armbrust --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 2 +- .../org/apache/spark/sql/hive/execution/HiveTableScan.scala | 6 +++++- .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 3 +++ 3 files changed, 9 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 c5fee5e4702f6..75a19656af110 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 @@ -305,7 +305,7 @@ private[hive] case class MetastoreRelation val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) /** Non-partitionKey attributes */ - val attributes = table.getSd.getCols.map(_.toAttribute) + val attributes = hiveQlTable.getCols.map(_.toAttribute) val output = attributes ++ partitionKeys } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index a32147584f6f4..5b83b77d80a22 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -86,10 +86,14 @@ case class HiveTableScan( ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) + val tableDesc = relation.tableDesc + val deserializer = tableDesc.getDeserializerClass.newInstance + deserializer.initialize(hiveConf, tableDesc.getProperties) + // Specifies types and object inspectors of columns to be scanned. val structOI = ObjectInspectorUtils .getStandardObjectInspector( - relation.tableDesc.getDeserializer.getObjectInspector, + deserializer.getObjectInspector, ObjectInspectorCopyOption.JAVA) .asInstanceOf[StructObjectInspector] diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2829105f43716..3e100775e4981 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -802,6 +802,9 @@ class HiveQuerySuite extends HiveComparisonTest { clear() } + createQueryTest("select from thrift based table", + "SELECT * from src_thrift") + // Put tests that depend on specific Hive settings before these last two test, // since they modify /clear stuff. } From 9d9ca91fef70eca6fc576be9c99aed5d8ce6e68b Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Mon, 13 Oct 2014 13:49:11 -0700 Subject: [PATCH 0986/1492] [SQL]Small bug in unresolved.scala name should throw exception with name instead of exprId. Author: Liquan Pei Closes #2758 from Ishiihara/SparkSQL-bug and squashes the following commits: aa36a3b [Liquan Pei] small bug --- .../org/apache/spark/sql/catalyst/analysis/unresolved.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 67570a6f73c36..77d84e1687e1b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -88,7 +88,7 @@ case class Star( mapFunction: Attribute => Expression = identity[Attribute]) extends Attribute with trees.LeafNode[Expression] { - override def name = throw new UnresolvedException(this, "exprId") + override def name = throw new UnresolvedException(this, "name") override def exprId = throw new UnresolvedException(this, "exprId") override def dataType = throw new UnresolvedException(this, "dataType") override def nullable = throw new UnresolvedException(this, "nullable") From 9eb49d4134e23a15142fb592d54d920e89bd8786 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 13 Oct 2014 13:50:27 -0700 Subject: [PATCH 0987/1492] [SPARK-3809][SQL] Fixes test suites in hive-thriftserver As scwf pointed out, `HiveThriftServer2Suite` isn't effective anymore after the Thrift server was made a daemon. On the other hand, these test suites were known flaky, PR #2214 tried to fix them but failed because of unknown Jenkins build error. This PR fixes both sets of issues. In this PR, instead of watching `start-thriftserver.sh` output, the test code start a `tail` process to watch the log file. A `Thread.sleep` has to be introduced because the `kill` command used in `stop-thriftserver.sh` is not synchronous. As for the root cause of the mysterious Jenkins build failure. Please refer to [this comment](https://github.com/apache/spark/pull/2675#issuecomment-58464189) below for details. ---- (Copied from PR description of #2214) This PR fixes two issues of `HiveThriftServer2Suite` and brings 1 enhancement: 1. Although metastore, warehouse directories and listening port are randomly chosen, all test cases share the same configuration. Due to parallel test execution, one of the two test case is doomed to fail 2. We caught any exceptions thrown from a test case and print diagnosis information, but forgot to re-throw the exception... 3. When the forked server process ends prematurely (e.g., fails to start), the `serverRunning` promise is completed with a failure, preventing the test code to keep waiting until timeout. So, embarrassingly, this test suite was failing continuously for several days but no one had ever noticed it... Fortunately no bugs in the production code were covered under the hood. Author: Cheng Lian Author: wangfei Closes #2675 from liancheng/fix-thriftserver-tests and squashes the following commits: 1c384b7 [Cheng Lian] Minor code cleanup, restore the logging level hack in TestHive.scala 7805c33 [wangfei] reset SPARK_TESTING to avoid loading Log4J configurations in testing class paths af2b5a9 [Cheng Lian] Removes log level hacks from TestHiveContext d116405 [wangfei] make sure that log4j level is INFO ee92a82 [Cheng Lian] Relaxes timeout 7fd6757 [Cheng Lian] Fixes test suites in hive-thriftserver --- .../sql/hive/thriftserver/CliSuite.scala | 13 ++- .../thriftserver/HiveThriftServer2Suite.scala | 86 +++++++++++-------- 2 files changed, 60 insertions(+), 39 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index d68dd090b5e6c..fc97a25be34be 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.apache.spark.Logging +import org.apache.spark.{SparkException, Logging} import org.apache.spark.sql.catalyst.util.getTempFilePath class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { @@ -62,8 +62,11 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { def captureOutput(source: String)(line: String) { buffer += s"$source> $line" + // If we haven't found all expected answers... if (next.get() < expectedAnswers.size) { + // If another expected answer is found... if (line.startsWith(expectedAnswers(next.get()))) { + // If all expected answers have been found... if (next.incrementAndGet() == expectedAnswers.size) { foundAllExpectedAnswers.trySuccess(()) } @@ -77,7 +80,8 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { Future { val exitValue = process.exitValue() - logInfo(s"Spark SQL CLI process exit value: $exitValue") + foundAllExpectedAnswers.tryFailure( + new SparkException(s"Spark SQL CLI process exit value: $exitValue")) } try { @@ -98,6 +102,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { |End CliSuite failure output |=========================== """.stripMargin, cause) + throw cause } finally { warehousePath.delete() metastorePath.delete() @@ -109,7 +114,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") - runCliWithin(1.minute)( + runCliWithin(3.minute)( "CREATE TABLE hive_test(key INT, val STRING);" -> "OK", "SHOW TABLES;" @@ -120,7 +125,7 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { -> "Time taken: ", "SELECT COUNT(*) FROM hive_test;" -> "5", - "DROP TABLE hive_test" + "DROP TABLE hive_test;" -> "Time taken: " ) } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index 38977ff162097..e3b4e45a3d68c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -17,17 +17,17 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.duration._ -import scala.concurrent.{Await, Future, Promise} -import scala.sys.process.{Process, ProcessLogger} - import java.io.File import java.net.ServerSocket import java.sql.{DriverManager, Statement} import java.util.concurrent.TimeoutException +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.concurrent.{Await, Promise} +import scala.sys.process.{Process, ProcessLogger} +import scala.util.Try + import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver import org.scalatest.FunSuite @@ -41,25 +41,25 @@ import org.apache.spark.sql.catalyst.util.getTempFilePath class HiveThriftServer2Suite extends FunSuite with Logging { Class.forName(classOf[HiveDriver].getCanonicalName) - private val listeningHost = "localhost" - private val listeningPort = { - // Let the system to choose a random available port to avoid collision with other parallel - // builds. - val socket = new ServerSocket(0) - val port = socket.getLocalPort - socket.close() - port - } - - private val warehousePath = getTempFilePath("warehouse") - private val metastorePath = getTempFilePath("metastore") - private val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" - - def startThriftServerWithin(timeout: FiniteDuration = 30.seconds)(f: Statement => Unit) { - val serverScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + def startThriftServerWithin(timeout: FiniteDuration = 1.minute)(f: Statement => Unit) { + val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) + val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) + + val warehousePath = getTempFilePath("warehouse") + val metastorePath = getTempFilePath("metastore") + val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" + val listeningHost = "localhost" + val listeningPort = { + // Let the system to choose a random available port to avoid collision with other parallel + // builds. + val socket = new ServerSocket(0) + val port = socket.getLocalPort + socket.close() + port + } val command = - s"""$serverScript + s"""$startScript | --master local | --hiveconf hive.root.logger=INFO,console | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri @@ -68,29 +68,40 @@ class HiveThriftServer2Suite extends FunSuite with Logging { | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$listeningPort """.stripMargin.split("\\s+").toSeq - val serverStarted = Promise[Unit]() + val serverRunning = Promise[Unit]() val buffer = new ArrayBuffer[String]() + val LOGGING_MARK = + s"starting ${HiveThriftServer2.getClass.getCanonicalName.stripSuffix("$")}, logging to " + var logTailingProcess: Process = null + var logFilePath: String = null - def captureOutput(source: String)(line: String) { - buffer += s"$source> $line" + def captureLogOutput(line: String): Unit = { + buffer += line if (line.contains("ThriftBinaryCLIService listening on")) { - serverStarted.success(()) + serverRunning.success(()) } } - val process = Process(command).run( - ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) - - Future { - val exitValue = process.exitValue() - logInfo(s"Spark SQL Thrift server process exit value: $exitValue") + def captureThriftServerOutput(source: String)(line: String): Unit = { + if (line.startsWith(LOGGING_MARK)) { + logFilePath = line.drop(LOGGING_MARK.length).trim + // Ensure that the log file is created so that the `tail' command won't fail + Try(new File(logFilePath).createNewFile()) + logTailingProcess = Process(s"/usr/bin/env tail -f $logFilePath") + .run(ProcessLogger(captureLogOutput, _ => ())) + } } + // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths + Process(command, None, "SPARK_TESTING" -> "0").run(ProcessLogger( + captureThriftServerOutput("stdout"), + captureThriftServerOutput("stderr"))) + val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/" val user = System.getProperty("user.name") try { - Await.result(serverStarted.future, timeout) + Await.result(serverRunning.future, timeout) val connection = DriverManager.getConnection(jdbcUri, user, "") val statement = connection.createStatement() @@ -122,10 +133,15 @@ class HiveThriftServer2Suite extends FunSuite with Logging { |End HiveThriftServer2Suite failure output |========================================= """.stripMargin, cause) + throw cause } finally { warehousePath.delete() metastorePath.delete() - process.destroy() + Process(stopScript).run().exitValue() + // The `spark-daemon.sh' script uses kill, which is not synchronous, have to wait for a while. + Thread.sleep(3.seconds.toMillis) + Option(logTailingProcess).map(_.destroy()) + Option(logFilePath).map(new File(_).delete()) } } From 4d26aca770f7dd50eee1ed7855e9eda68b5a7ffa Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 13 Oct 2014 22:46:49 -0700 Subject: [PATCH 0988/1492] [SPARK-3912][Streaming] Fixed flakyFlumeStreamSuite @harishreedharan @pwendell See JIRA for diagnosis of the problem https://issues.apache.org/jira/browse/SPARK-3912 The solution was to reimplement it. 1. Find a free port (by binding and releasing a server-scoket), and then use that port 2. Remove thread.sleep()s, instead repeatedly try to create a sender and send data and check whether data was sent. Use eventually() to minimize waiting time. 3. Check whether all the data was received, without caring about batches. Author: Tathagata Das Closes #2773 from tdas/flume-test-fix and squashes the following commits: 93cd7f6 [Tathagata Das] Reimplimented FlumeStreamSuite to be more robust. --- .../streaming/flume/FlumeStreamSuite.scala | 166 +++++++++++------- 1 file changed, 102 insertions(+), 64 deletions(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 33235d150b4a5..13943ed5442b9 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -17,103 +17,141 @@ package org.apache.spark.streaming.flume -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} - -import java.net.InetSocketAddress +import java.net.{InetSocketAddress, ServerSocket} import java.nio.ByteBuffer import java.nio.charset.Charset +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} +import scala.concurrent.duration._ +import scala.language.postfixOps + import org.apache.avro.ipc.NettyTransceiver import org.apache.avro.ipc.specific.SpecificRequestor +import org.apache.flume.source.avro import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} +import org.jboss.netty.channel.ChannelPipeline +import org.jboss.netty.channel.socket.SocketChannel +import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.jboss.netty.handler.codec.compression._ +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.scalatest.concurrent.Eventually._ +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} -import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream} +import org.apache.spark.streaming.scheduler.{StreamingListener, StreamingListenerReceiverStarted} import org.apache.spark.util.Utils -import org.jboss.netty.channel.ChannelPipeline -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import org.jboss.netty.channel.socket.SocketChannel -import org.jboss.netty.handler.codec.compression._ +class FlumeStreamSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { + val conf = new SparkConf().setMaster("local[4]").setAppName("FlumeStreamSuite") + + var ssc: StreamingContext = null + var transceiver: NettyTransceiver = null -class FlumeStreamSuite extends TestSuiteBase { + after { + if (ssc != null) { + ssc.stop() + } + if (transceiver != null) { + transceiver.close() + } + } test("flume input stream") { - runFlumeStreamTest(false) + testFlumeStream(testCompression = false) } test("flume input compressed stream") { - runFlumeStreamTest(true) + testFlumeStream(testCompression = true) + } + + /** Run test on flume stream */ + private def testFlumeStream(testCompression: Boolean): Unit = { + val input = (1 to 100).map { _.toString } + val testPort = findFreePort() + val outputBuffer = startContext(testPort, testCompression) + writeAndVerify(input, testPort, outputBuffer, testCompression) + } + + /** Find a free port */ + private def findFreePort(): Int = { + Utils.startServiceOnPort(23456, (trialPort: Int) => { + val socket = new ServerSocket(trialPort) + socket.close() + (null, trialPort) + })._2 } - - def runFlumeStreamTest(enableDecompression: Boolean) { - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val (flumeStream, testPort) = - Utils.startServiceOnPort(9997, (trialPort: Int) => { - val dstream = FlumeUtils.createStream( - ssc, "localhost", trialPort, StorageLevel.MEMORY_AND_DISK, enableDecompression) - (dstream, trialPort) - }) + /** Setup and start the streaming context */ + private def startContext( + testPort: Int, testCompression: Boolean): (ArrayBuffer[Seq[SparkFlumeEvent]]) = { + ssc = new StreamingContext(conf, Milliseconds(200)) + val flumeStream = FlumeUtils.createStream( + ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, testCompression) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] val outputStream = new TestOutputStream(flumeStream, outputBuffer) outputStream.register() ssc.start() + outputBuffer + } - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val input = Seq(1, 2, 3, 4, 5) - Thread.sleep(1000) - val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) - var client: AvroSourceProtocol = null - - if (enableDecompression) { - client = SpecificRequestor.getClient( - classOf[AvroSourceProtocol], - new NettyTransceiver(new InetSocketAddress("localhost", testPort), - new CompressionChannelFactory(6))) - } else { - client = SpecificRequestor.getClient( - classOf[AvroSourceProtocol], transceiver) - } + /** Send data to the flume receiver and verify whether the data was received */ + private def writeAndVerify( + input: Seq[String], + testPort: Int, + outputBuffer: ArrayBuffer[Seq[SparkFlumeEvent]], + enableCompression: Boolean + ) { + val testAddress = new InetSocketAddress("localhost", testPort) - for (i <- 0 until input.size) { + val inputEvents = input.map { item => val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(input(i).toString.getBytes("utf-8"))) + event.setBody(ByteBuffer.wrap(item.getBytes("UTF-8"))) event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) - client.append(event) - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) + event } - Thread.sleep(1000) - - val startTime = System.currentTimeMillis() - while (outputBuffer.size < input.size && System.currentTimeMillis() - startTime < maxWaitTimeMillis) { - logInfo("output.size = " + outputBuffer.size + ", input.size = " + input.size) - Thread.sleep(100) + eventually(timeout(10 seconds), interval(100 milliseconds)) { + // if last attempted transceiver had succeeded, close it + if (transceiver != null) { + transceiver.close() + transceiver = null + } + + // Create transceiver + transceiver = { + if (enableCompression) { + new NettyTransceiver(testAddress, new CompressionChannelFactory(6)) + } else { + new NettyTransceiver(testAddress) + } + } + + // Create Avro client with the transceiver + val client = SpecificRequestor.getClient(classOf[AvroSourceProtocol], transceiver) + client should not be null + + // Send data + val status = client.appendBatch(inputEvents.toList) + status should be (avro.Status.OK) } - Thread.sleep(1000) - val timeTaken = System.currentTimeMillis() - startTime - assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") - logInfo("Stopping context") - ssc.stop() - - val decoder = Charset.forName("UTF-8").newDecoder() - - assert(outputBuffer.size === input.length) - for (i <- 0 until outputBuffer.size) { - assert(outputBuffer(i).size === 1) - val str = decoder.decode(outputBuffer(i).head.event.getBody) - assert(str.toString === input(i).toString) - assert(outputBuffer(i).head.event.getHeaders.get("test") === "header") + + val decoder = Charset.forName("UTF-8").newDecoder() + eventually(timeout(10 seconds), interval(100 milliseconds)) { + val outputEvents = outputBuffer.flatten.map { _.event } + outputEvents.foreach { + event => + event.getHeaders.get("test") should be("header") + } + val output = outputEvents.map(event => decoder.decode(event.getBody()).toString) + output should be (input) } } - class CompressionChannelFactory(compressionLevel: Int) extends NioClientSocketChannelFactory { + /** Class to create socket channel with compression */ + private class CompressionChannelFactory(compressionLevel: Int) extends NioClientSocketChannelFactory { override def newChannel(pipeline: ChannelPipeline): SocketChannel = { val encoder = new ZlibEncoder(compressionLevel) pipeline.addFirst("deflater", encoder) From 186b497c945cc7bbe7a21fef56a948dd1fd10774 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 13 Oct 2014 23:31:37 -0700 Subject: [PATCH 0989/1492] [SPARK-3921] Fix CoarseGrainedExecutorBackend's arguments for Standalone mode The goal of this patch is to fix the swapped arguments in standalone mode, which was caused by https://github.com/apache/spark/commit/79e45c9323455a51f25ed9acd0edd8682b4bbb88#diff-79391110e9f26657e415aa169a004998R153. More details can be found in the JIRA: [SPARK-3921](https://issues.apache.org/jira/browse/SPARK-3921) Tested in Standalone mode, but not in Mesos. Author: Aaron Davidson Closes #2779 from aarondav/fix-standalone and squashes the following commits: 725227a [Aaron Davidson] Fix ExecutorRunnerTest 9d703fe [Aaron Davidson] [SPARK-3921] Fix CoarseGrainedExecutorBackend's arguments for Standalone mode --- .../apache/spark/deploy/worker/ExecutorRunner.scala | 3 ++- .../spark/executor/CoarseGrainedExecutorBackend.scala | 3 +++ .../cluster/SparkDeploySchedulerBackend.scala | 3 ++- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 8 ++++---- .../spark/deploy/worker/ExecutorRunnerTest.scala | 10 ++++------ 5 files changed, 15 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 71650cd773bcf..71d7385b08eb9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -111,13 +111,14 @@ private[spark] class ExecutorRunner( case "{{EXECUTOR_ID}}" => execId.toString case "{{HOSTNAME}}" => host case "{{CORES}}" => cores.toString + case "{{APP_ID}}" => appId case other => other } def getCommandSeq = { val command = Command( appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), + appDesc.command.arguments.map(substituteVariables), appDesc.command.environment, appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 06061edfc0844..c40a3e16675ad 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -152,6 +152,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { "Usage: CoarseGrainedExecutorBackend " + " [] ") System.exit(1) + + // NB: These arguments are provided by SparkDeploySchedulerBackend (for standalone mode) + // and CoarseMesosSchedulerBackend (for mesos mode). case 5 => run(args(0), args(1), args(2), args(3).toInt, args(4), None) case x if x > 5 => 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 ed209d195ec9d..8c7de75600b5f 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 @@ -51,7 +51,8 @@ private[spark] class SparkDeploySchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{APP_ID}}", + "{{WORKER_URL}}") val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") .map(Utils.splitCommandString).getOrElse(Seq.empty) val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath").toSeq.flatMap { cp => diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 90828578cd88f..d7f88de4b40aa 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -150,17 +150,17 @@ private[spark] class CoarseMesosSchedulerBackend( if (uri == null) { val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s".format( + runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores, appId)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( ("cd %s*; " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s") .format(basename, driverUrl, offer.getSlaveId.getValue, - offer.getHostname, numCores)) + offer.getHostname, numCores, appId)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 39ab53cf0b5b1..5e2592e8d2e8d 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -26,14 +26,12 @@ import org.apache.spark.SparkConf class ExecutorRunnerTest extends FunSuite { test("command includes appId") { - def f(s:String) = new File(s) + val appId = "12345-worker321-9876" val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val appDesc = new ApplicationDescription("app name", Some(8), 500, - Command("foo", Seq(), Map(), Seq(), Seq(), Seq()), "appUiUrl") - val appId = "12345-worker321-9876" - val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome), - f("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) - + Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") + val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", + new File(sparkHome), new File("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) assert(er.getCommandSeq.last === appId) } } From 9b6de6fbc00b184d81fc28ac160d03451fad80ec Mon Sep 17 00:00:00 2001 From: Bill Bejeck Date: Tue, 14 Oct 2014 12:12:38 -0700 Subject: [PATCH 0990/1492] SPARK-3178 setting SPARK_WORKER_MEMORY to a value without a label (m or g) sets the worker memory limit to zero Validate the memory is greater than zero when set from the SPARK_WORKER_MEMORY environment variable or command line without a g or m label. Added unit tests. If memory is 0 an IllegalStateException is thrown. Updated unit tests to mock environment variables by subclassing SparkConf (tip provided by Josh Rosen). Updated WorkerArguments to use SparkConf.getenv instead of System.getenv for reading the SPARK_WORKER_MEMORY environment variable. Author: Bill Bejeck Closes #2309 from bbejeck/spark-memory-worker and squashes the following commits: 51cf915 [Bill Bejeck] SPARK-3178 - Validate the memory is greater than zero when set from the SPARK_WORKER_MEMORY environment variable or command line without a g or m label. Added unit tests. If memory is 0 an IllegalStateException is thrown. Updated unit tests to mock environment variables by subclassing SparkConf (tip provided by Josh Rosen). Updated WorkerArguments to use SparkConf.getenv instead of System.getenv for reading the SPARK_WORKER_MEMORY environment variable. --- .../spark/deploy/worker/WorkerArguments.scala | 13 ++- .../deploy/worker/WorkerArgumentsTest.scala | 82 +++++++++++++++++++ 2 files changed, 93 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 1e295aaa48c30..54e3937edde6b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -41,8 +41,8 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { if (System.getenv("SPARK_WORKER_CORES") != null) { cores = System.getenv("SPARK_WORKER_CORES").toInt } - if (System.getenv("SPARK_WORKER_MEMORY") != null) { - memory = Utils.memoryStringToMb(System.getenv("SPARK_WORKER_MEMORY")) + if (conf.getenv("SPARK_WORKER_MEMORY") != null) { + memory = Utils.memoryStringToMb(conf.getenv("SPARK_WORKER_MEMORY")) } if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt @@ -56,6 +56,8 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { parse(args.toList) + checkWorkerMemory() + def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) @@ -153,4 +155,11 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { // Leave out 1 GB for the operating system, but don't return a negative memory size math.max(totalMb - 1024, 512) } + + def checkWorkerMemory(): Unit = { + if (memory <= 0) { + val message = "Memory can't be 0, missing a M or G on the end of the memory specification?" + throw new IllegalStateException(message) + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala new file mode 100644 index 0000000000000..1a28a9a187cd7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.deploy.worker + +import org.apache.spark.SparkConf +import org.scalatest.FunSuite + + +class WorkerArgumentsTest extends FunSuite { + + test("Memory can't be set to 0 when cmd line args leave off M or G") { + val conf = new SparkConf + val args = Array("-m", "10000", "spark://localhost:0000 ") + intercept[IllegalStateException] { + new WorkerArguments(args, conf) + } + } + + + test("Memory can't be set to 0 when SPARK_WORKER_MEMORY env property leaves off M or G") { + val args = Array("spark://localhost:0000 ") + + class MySparkConf extends SparkConf(false) { + override def getenv(name: String) = { + if (name == "SPARK_WORKER_MEMORY") "50000" + else super.getenv(name) + } + + override def clone: SparkConf = { + new MySparkConf().setAll(settings) + } + } + val conf = new MySparkConf() + intercept[IllegalStateException] { + new WorkerArguments(args, conf) + } + } + + test("Memory correctly set when SPARK_WORKER_MEMORY env property appends G") { + val args = Array("spark://localhost:0000 ") + + class MySparkConf extends SparkConf(false) { + override def getenv(name: String) = { + if (name == "SPARK_WORKER_MEMORY") "5G" + else super.getenv(name) + } + + override def clone: SparkConf = { + new MySparkConf().setAll(settings) + } + } + val conf = new MySparkConf() + val workerArgs = new WorkerArguments(args, conf) + assert(workerArgs.memory === 5120) + } + + test("Memory correctly set from args with M appended to memory value") { + val conf = new SparkConf + val args = Array("-m", "10000M", "spark://localhost:0000 ") + + val workerArgs = new WorkerArguments(args, conf) + assert(workerArgs.memory === 10000) + + } + +} From 7ced88b0d6b4d90c262f19afa99c02b51c0ea5ea Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Tue, 14 Oct 2014 14:09:39 -0700 Subject: [PATCH 0991/1492] [SPARK-3946] gitignore in /python includes wrong directory Modified to ignore not the docs/ directory, but only the docs/_build/ which is the output directory of sphinx build. Author: Masayoshi TSUZUKI Closes #2796 from tsudukim/feature/SPARK-3946 and squashes the following commits: 2bea6a9 [Masayoshi TSUZUKI] [SPARK-3946] gitignore in /python includes wrong directory --- python/.gitignore | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/.gitignore b/python/.gitignore index 80b361ffbd51c..52128cf844a79 100644 --- a/python/.gitignore +++ b/python/.gitignore @@ -1,5 +1,5 @@ *.pyc -docs/ +docs/_build/ pyspark.egg-info build/ dist/ From 24b818b971ba715b6796518e4c6afdecb1b16f15 Mon Sep 17 00:00:00 2001 From: shitis Date: Tue, 14 Oct 2014 14:16:45 -0700 Subject: [PATCH 0992/1492] [SPARK-3944][Core] Using Option[String] where value of String can be null Author: shitis Closes #2795 from Shiti/master and squashes the following commits: 46897d7 [shitis] Using Option Wrapper to convert String with value null to None --- .../scala/org/apache/spark/util/Utils.scala | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 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 07477dd460a4b..aad901620f53e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -340,8 +340,8 @@ private[spark] object Utils extends Logging { val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) - uri.getScheme match { - case "http" | "https" | "ftp" => + Option(uri.getScheme) match { + case Some("http") | Some("https") | Some("ftp") => logInfo("Fetching " + url + " to " + tempFile) var uc: URLConnection = null @@ -374,7 +374,7 @@ private[spark] object Utils extends Logging { } } Files.move(tempFile, targetFile) - case "file" | null => + case Some("file") | None => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) @@ -403,7 +403,7 @@ private[spark] object Utils extends Logging { logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) Files.copy(sourceFile, targetFile) } - case _ => + case Some(other) => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) @@ -1368,16 +1368,17 @@ private[spark] object Utils extends Logging { if (uri.getPath == null) { throw new IllegalArgumentException(s"Given path is malformed: $uri") } - uri.getScheme match { - case windowsDrive(d) if windows => + + Option(uri.getScheme) match { + case Some(windowsDrive(d)) if windows => new URI("file:/" + uri.toString.stripPrefix("/")) - case null => + case None => // Preserve fragments for HDFS file name substitution (denoted by "#") // For instance, in "abc.py#xyz.py", "xyz.py" is the name observed by the application val fragment = uri.getFragment val part = new File(uri.getPath).toURI new URI(part.getScheme, part.getPath, fragment) - case _ => + case Some(other) => uri } } @@ -1399,10 +1400,11 @@ private[spark] object Utils extends Logging { } else { paths.split(",").filter { p => val formattedPath = if (windows) formatWindowsPath(p) else p - new URI(formattedPath).getScheme match { - case windowsDrive(d) if windows => false - case "local" | "file" | null => false - case _ => true + val uri = new URI(formattedPath) + Option(uri.getScheme) match { + case Some(windowsDrive(d)) if windows => false + case Some("local") | Some("file") | None => false + case Some(other) => true } } } From 56096dbaa8cb3ab39bfc2ce5827192313613b010 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 14 Oct 2014 14:42:09 -0700 Subject: [PATCH 0993/1492] SPARK-3803 [MLLIB] ArrayIndexOutOfBoundsException found in executing computePrincipalComponents Avoid overflow in computing n*(n+1)/2 as much as possible; throw explicit error when Gramian computation will fail due to negative array size; warn about large result when computing Gramian too Author: Sean Owen Closes #2801 from srowen/SPARK-3803 and squashes the following commits: b4e6d92 [Sean Owen] Avoid overflow in computing n*(n+1)/2 as much as possible; throw explicit error when Gramian computation will fail due to negative array size; warn about large result when computing Gramian too --- .../mllib/linalg/distributed/RowMatrix.scala | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 8380058cf9b41..ec2d481dccc22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -111,7 +111,10 @@ class RowMatrix( */ def computeGramianMatrix(): Matrix = { val n = numCols().toInt - val nt: Int = n * (n + 1) / 2 + checkNumColumns(n) + // Computes n*(n+1)/2, avoiding overflow in the multiplication. + // This succeeds when n <= 65535, which is checked above + val nt: Int = if (n % 2 == 0) ((n / 2) * (n + 1)) else (n * ((n + 1) / 2)) // Compute the upper triangular part of the gram matrix. val GU = rows.treeAggregate(new BDV[Double](new Array[Double](nt)))( @@ -123,6 +126,16 @@ class RowMatrix( RowMatrix.triuToFull(n, GU.data) } + private def checkNumColumns(cols: Int): Unit = { + if (cols > 65535) { + throw new IllegalArgumentException(s"Argument with more than 65535 cols: $cols") + } + if (cols > 10000) { + val mem = cols * cols * 8 + logWarning(s"$cols columns will require at least $mem bytes of memory!") + } + } + /** * Computes singular value decomposition of this matrix. Denote this matrix by A (m x n). This * will compute matrices U, S, V such that A ~= U * S * V', where S contains the leading k @@ -301,12 +314,7 @@ class RowMatrix( */ def computeCovariance(): Matrix = { val n = numCols().toInt - - if (n > 10000) { - val mem = n * n * java.lang.Double.SIZE / java.lang.Byte.SIZE - logWarning(s"The number of columns $n is greater than 10000! " + - s"We need at least $mem bytes of memory.") - } + checkNumColumns(n) val (m, mean) = rows.treeAggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), From 7b4f39f647da1f7b1b57e38827a8639243c661cb Mon Sep 17 00:00:00 2001 From: cocoatomo Date: Tue, 14 Oct 2014 15:09:51 -0700 Subject: [PATCH 0994/1492] [SPARK-3869] ./bin/spark-class miss Java version with _JAVA_OPTIONS set When _JAVA_OPTIONS environment variable is set, a command "java -version" outputs a message like "Picked up _JAVA_OPTIONS: -Dfile.encoding=UTF-8". ./bin/spark-class knows java version from the first line of "java -version" output, so it mistakes java version with _JAVA_OPTIONS set. Author: cocoatomo Closes #2725 from cocoatomo/issues/3869-mistake-java-version and squashes the following commits: f894ebd [cocoatomo] [SPARK-3869] ./bin/spark-class miss Java version with _JAVA_OPTIONS set --- bin/spark-class | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/spark-class b/bin/spark-class index e8201c18d52de..91d858bc063d0 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -105,7 +105,7 @@ else exit 1 fi fi -JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') +JAVA_VERSION=$("$RUNNER" -version 2>&1 | grep 'version' | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') # Set JAVA_OPTS to be able to load native libraries and to set heap size if [ "$JAVA_VERSION" -ge 18 ]; then From 66af8e2508bfe9c9d4aecc17a19f297c98e9661d Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Tue, 14 Oct 2014 18:50:14 -0700 Subject: [PATCH 0995/1492] [SPARK-3943] Some scripts bin\*.cmd pollutes environment variables in Windows Modified not to pollute environment variables. Just moved the main logic into `XXX2.cmd` from `XXX.cmd`, and call `XXX2.cmd` with cmd command in `XXX.cmd`. `pyspark.cmd` and `spark-class.cmd` are already using the same way, but `spark-shell.cmd`, `spark-submit.cmd` and `/python/docs/make.bat` are not. Author: Masayoshi TSUZUKI Closes #2797 from tsudukim/feature/SPARK-3943 and squashes the following commits: b397a7d [Masayoshi TSUZUKI] [SPARK-3943] Some scripts bin\*.cmd pollutes environment variables in Windows --- bin/spark-shell.cmd | 5 +- bin/spark-shell2.cmd | 22 ++++ bin/spark-submit.cmd | 51 +-------- bin/spark-submit2.cmd | 68 ++++++++++++ python/docs/make.bat | 242 +---------------------------------------- python/docs/make2.bat | 243 ++++++++++++++++++++++++++++++++++++++++++ 6 files changed, 342 insertions(+), 289 deletions(-) create mode 100644 bin/spark-shell2.cmd create mode 100644 bin/spark-submit2.cmd create mode 100644 python/docs/make2.bat diff --git a/bin/spark-shell.cmd b/bin/spark-shell.cmd index 2ee60b4e2a2b3..8f90ba5a0b3b8 100755 --- a/bin/spark-shell.cmd +++ b/bin/spark-shell.cmd @@ -17,6 +17,7 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -set SPARK_HOME=%~dp0.. +rem This is the entry point for running Spark shell. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. -cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %* spark-shell +cmd /V /E /C %~dp0spark-shell2.cmd %* diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd new file mode 100644 index 0000000000000..2ee60b4e2a2b3 --- /dev/null +++ b/bin/spark-shell2.cmd @@ -0,0 +1,22 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +set SPARK_HOME=%~dp0.. + +cmd /V /E /C %SPARK_HOME%\bin\spark-submit.cmd --class org.apache.spark.repl.Main %* spark-shell diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index cf6046d1547ad..8f3b84c7b971d 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -17,52 +17,7 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem -rem NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! +rem This is the entry point for running Spark submit. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. -set SPARK_HOME=%~dp0.. -set ORIG_ARGS=%* - -rem Reset the values of all variables used -set SPARK_SUBMIT_DEPLOY_MODE=client -set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_HOME%\conf\spark-defaults.conf -set SPARK_SUBMIT_DRIVER_MEMORY= -set SPARK_SUBMIT_LIBRARY_PATH= -set SPARK_SUBMIT_CLASSPATH= -set SPARK_SUBMIT_OPTS= -set SPARK_SUBMIT_BOOTSTRAP_DRIVER= - -:loop -if [%1] == [] goto continue - if [%1] == [--deploy-mode] ( - set SPARK_SUBMIT_DEPLOY_MODE=%2 - ) else if [%1] == [--properties-file] ( - set SPARK_SUBMIT_PROPERTIES_FILE=%2 - ) else if [%1] == [--driver-memory] ( - set SPARK_SUBMIT_DRIVER_MEMORY=%2 - ) else if [%1] == [--driver-library-path] ( - set SPARK_SUBMIT_LIBRARY_PATH=%2 - ) else if [%1] == [--driver-class-path] ( - set SPARK_SUBMIT_CLASSPATH=%2 - ) else if [%1] == [--driver-java-options] ( - set SPARK_SUBMIT_OPTS=%2 - ) - shift -goto loop -:continue - -rem For client mode, the driver will be launched in the same JVM that launches -rem SparkSubmit, so we may need to read the properties file for any extra class -rem paths, library paths, java options and memory early on. Otherwise, it will -rem be too late by the time the driver JVM has started. - -if [%SPARK_SUBMIT_DEPLOY_MODE%] == [client] ( - if exist %SPARK_SUBMIT_PROPERTIES_FILE% ( - rem Parse the properties file only if the special configs exist - for /f %%i in ('findstr /r /c:"^[\t ]*spark.driver.memory" /c:"^[\t ]*spark.driver.extra" ^ - %SPARK_SUBMIT_PROPERTIES_FILE%') do ( - set SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 - ) - ) -) - -cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% +cmd /V /E /C %~dp0spark-submit2.cmd %* diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd new file mode 100644 index 0000000000000..cf6046d1547ad --- /dev/null +++ b/bin/spark-submit2.cmd @@ -0,0 +1,68 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! + +set SPARK_HOME=%~dp0.. +set ORIG_ARGS=%* + +rem Reset the values of all variables used +set SPARK_SUBMIT_DEPLOY_MODE=client +set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_HOME%\conf\spark-defaults.conf +set SPARK_SUBMIT_DRIVER_MEMORY= +set SPARK_SUBMIT_LIBRARY_PATH= +set SPARK_SUBMIT_CLASSPATH= +set SPARK_SUBMIT_OPTS= +set SPARK_SUBMIT_BOOTSTRAP_DRIVER= + +:loop +if [%1] == [] goto continue + if [%1] == [--deploy-mode] ( + set SPARK_SUBMIT_DEPLOY_MODE=%2 + ) else if [%1] == [--properties-file] ( + set SPARK_SUBMIT_PROPERTIES_FILE=%2 + ) else if [%1] == [--driver-memory] ( + set SPARK_SUBMIT_DRIVER_MEMORY=%2 + ) else if [%1] == [--driver-library-path] ( + set SPARK_SUBMIT_LIBRARY_PATH=%2 + ) else if [%1] == [--driver-class-path] ( + set SPARK_SUBMIT_CLASSPATH=%2 + ) else if [%1] == [--driver-java-options] ( + set SPARK_SUBMIT_OPTS=%2 + ) + shift +goto loop +:continue + +rem For client mode, the driver will be launched in the same JVM that launches +rem SparkSubmit, so we may need to read the properties file for any extra class +rem paths, library paths, java options and memory early on. Otherwise, it will +rem be too late by the time the driver JVM has started. + +if [%SPARK_SUBMIT_DEPLOY_MODE%] == [client] ( + if exist %SPARK_SUBMIT_PROPERTIES_FILE% ( + rem Parse the properties file only if the special configs exist + for /f %%i in ('findstr /r /c:"^[\t ]*spark.driver.memory" /c:"^[\t ]*spark.driver.extra" ^ + %SPARK_SUBMIT_PROPERTIES_FILE%') do ( + set SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 + ) + ) +) + +cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% diff --git a/python/docs/make.bat b/python/docs/make.bat index adad44fd7536a..c011e82b4a35a 100644 --- a/python/docs/make.bat +++ b/python/docs/make.bat @@ -1,242 +1,6 @@ @ECHO OFF -REM Command file for Sphinx documentation +rem This is the entry point for running Sphinx documentation. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. -if "%SPHINXBUILD%" == "" ( - set SPHINXBUILD=sphinx-build -) -set BUILDDIR=_build -set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% . -set I18NSPHINXOPTS=%SPHINXOPTS% . -if NOT "%PAPER%" == "" ( - set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% - set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% -) - -if "%1" == "" goto help - -if "%1" == "help" ( - :help - echo.Please use `make ^` where ^ is one of - echo. html to make standalone HTML files - echo. dirhtml to make HTML files named index.html in directories - echo. singlehtml to make a single large HTML file - echo. pickle to make pickle files - echo. json to make JSON files - echo. htmlhelp to make HTML files and a HTML help project - echo. qthelp to make HTML files and a qthelp project - echo. devhelp to make HTML files and a Devhelp project - echo. epub to make an epub - echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter - echo. text to make text files - echo. man to make manual pages - echo. texinfo to make Texinfo files - echo. gettext to make PO message catalogs - echo. changes to make an overview over all changed/added/deprecated items - echo. xml to make Docutils-native XML files - echo. pseudoxml to make pseudoxml-XML files for display purposes - echo. linkcheck to check all external links for integrity - echo. doctest to run all doctests embedded in the documentation if enabled - goto end -) - -if "%1" == "clean" ( - for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i - del /q /s %BUILDDIR%\* - goto end -) - - -%SPHINXBUILD% 2> nul -if errorlevel 9009 ( - echo. - echo.The 'sphinx-build' command was not found. Make sure you have Sphinx - echo.installed, then set the SPHINXBUILD environment variable to point - echo.to the full path of the 'sphinx-build' executable. Alternatively you - echo.may add the Sphinx directory to PATH. - echo. - echo.If you don't have Sphinx installed, grab it from - echo.http://sphinx-doc.org/ - exit /b 1 -) - -if "%1" == "html" ( - %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/html. - goto end -) - -if "%1" == "dirhtml" ( - %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. - goto end -) - -if "%1" == "singlehtml" ( - %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. - goto end -) - -if "%1" == "pickle" ( - %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the pickle files. - goto end -) - -if "%1" == "json" ( - %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the JSON files. - goto end -) - -if "%1" == "htmlhelp" ( - %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run HTML Help Workshop with the ^ -.hhp project file in %BUILDDIR%/htmlhelp. - goto end -) - -if "%1" == "qthelp" ( - %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run "qcollectiongenerator" with the ^ -.qhcp project file in %BUILDDIR%/qthelp, like this: - echo.^> qcollectiongenerator %BUILDDIR%\qthelp\pyspark.qhcp - echo.To view the help file: - echo.^> assistant -collectionFile %BUILDDIR%\qthelp\pyspark.ghc - goto end -) - -if "%1" == "devhelp" ( - %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. - goto end -) - -if "%1" == "epub" ( - %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The epub file is in %BUILDDIR%/epub. - goto end -) - -if "%1" == "latex" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdf" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf - cd %BUILDDIR%/.. - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdfja" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf-ja - cd %BUILDDIR%/.. - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "text" ( - %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The text files are in %BUILDDIR%/text. - goto end -) - -if "%1" == "man" ( - %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The manual pages are in %BUILDDIR%/man. - goto end -) - -if "%1" == "texinfo" ( - %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. - goto end -) - -if "%1" == "gettext" ( - %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The message catalogs are in %BUILDDIR%/locale. - goto end -) - -if "%1" == "changes" ( - %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes - if errorlevel 1 exit /b 1 - echo. - echo.The overview file is in %BUILDDIR%/changes. - goto end -) - -if "%1" == "linkcheck" ( - %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck - if errorlevel 1 exit /b 1 - echo. - echo.Link check complete; look for any errors in the above output ^ -or in %BUILDDIR%/linkcheck/output.txt. - goto end -) - -if "%1" == "doctest" ( - %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest - if errorlevel 1 exit /b 1 - echo. - echo.Testing of doctests in the sources finished, look at the ^ -results in %BUILDDIR%/doctest/output.txt. - goto end -) - -if "%1" == "xml" ( - %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The XML files are in %BUILDDIR%/xml. - goto end -) - -if "%1" == "pseudoxml" ( - %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. - goto end -) - -:end +cmd /V /E /C %~dp0make2.bat %* diff --git a/python/docs/make2.bat b/python/docs/make2.bat new file mode 100644 index 0000000000000..7bcaeafad13d7 --- /dev/null +++ b/python/docs/make2.bat @@ -0,0 +1,243 @@ +@ECHO OFF + +REM Command file for Sphinx documentation + + +if "%SPHINXBUILD%" == "" ( + set SPHINXBUILD=sphinx-build +) +set BUILDDIR=_build +set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% . +set I18NSPHINXOPTS=%SPHINXOPTS% . +if NOT "%PAPER%" == "" ( + set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% + set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% +) + +if "%1" == "" goto help + +if "%1" == "help" ( + :help + echo.Please use `make ^` where ^ is one of + echo. html to make standalone HTML files + echo. dirhtml to make HTML files named index.html in directories + echo. singlehtml to make a single large HTML file + echo. pickle to make pickle files + echo. json to make JSON files + echo. htmlhelp to make HTML files and a HTML help project + echo. qthelp to make HTML files and a qthelp project + echo. devhelp to make HTML files and a Devhelp project + echo. epub to make an epub + echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter + echo. text to make text files + echo. man to make manual pages + echo. texinfo to make Texinfo files + echo. gettext to make PO message catalogs + echo. changes to make an overview over all changed/added/deprecated items + echo. xml to make Docutils-native XML files + echo. pseudoxml to make pseudoxml-XML files for display purposes + echo. linkcheck to check all external links for integrity + echo. doctest to run all doctests embedded in the documentation if enabled + goto end +) + +if "%1" == "clean" ( + for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i + del /q /s %BUILDDIR%\* + goto end +) + + +%SPHINXBUILD% 2> nul +if errorlevel 9009 ( + echo. + echo.The 'sphinx-build' command was not found. Make sure you have Sphinx + echo.installed, then set the SPHINXBUILD environment variable to point + echo.to the full path of the 'sphinx-build' executable. Alternatively you + echo.may add the Sphinx directory to PATH. + echo. + echo.If you don't have Sphinx installed, grab it from + echo.http://sphinx-doc.org/ + exit /b 1 +) + +if "%1" == "html" ( + %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/html. + goto end +) + +if "%1" == "dirhtml" ( + %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. + goto end +) + +if "%1" == "singlehtml" ( + %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. + goto end +) + +if "%1" == "pickle" ( + %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the pickle files. + goto end +) + +if "%1" == "json" ( + %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the JSON files. + goto end +) + +if "%1" == "htmlhelp" ( + %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run HTML Help Workshop with the ^ +.hhp project file in %BUILDDIR%/htmlhelp. + goto end +) + +if "%1" == "qthelp" ( + %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run "qcollectiongenerator" with the ^ +.qhcp project file in %BUILDDIR%/qthelp, like this: + echo.^> qcollectiongenerator %BUILDDIR%\qthelp\pyspark.qhcp + echo.To view the help file: + echo.^> assistant -collectionFile %BUILDDIR%\qthelp\pyspark.ghc + goto end +) + +if "%1" == "devhelp" ( + %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. + goto end +) + +if "%1" == "epub" ( + %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The epub file is in %BUILDDIR%/epub. + goto end +) + +if "%1" == "latex" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdf" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdfja" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf-ja + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "text" ( + %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The text files are in %BUILDDIR%/text. + goto end +) + +if "%1" == "man" ( + %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The manual pages are in %BUILDDIR%/man. + goto end +) + +if "%1" == "texinfo" ( + %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. + goto end +) + +if "%1" == "gettext" ( + %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The message catalogs are in %BUILDDIR%/locale. + goto end +) + +if "%1" == "changes" ( + %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes + if errorlevel 1 exit /b 1 + echo. + echo.The overview file is in %BUILDDIR%/changes. + goto end +) + +if "%1" == "linkcheck" ( + %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck + if errorlevel 1 exit /b 1 + echo. + echo.Link check complete; look for any errors in the above output ^ +or in %BUILDDIR%/linkcheck/output.txt. + goto end +) + +if "%1" == "doctest" ( + %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest + if errorlevel 1 exit /b 1 + echo. + echo.Testing of doctests in the sources finished, look at the ^ +results in %BUILDDIR%/doctest/output.txt. + goto end +) + +if "%1" == "xml" ( + %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The XML files are in %BUILDDIR%/xml. + goto end +) + +if "%1" == "pseudoxml" ( + %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. + goto end +) + +:end From 18ab6bd709bb9fcae290ffc43294d13f06670d55 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 14 Oct 2014 21:37:51 -0700 Subject: [PATCH 0996/1492] SPARK-1307 [DOCS] Don't use term 'standalone' to refer to a Spark Application HT to Diana, just proposing an implementation of her suggestion, which I rather agreed with. Is there a second/third for the motion? Refer to "self-contained" rather than "standalone" apps to avoid confusion with standalone deployment mode. And fix placement of reference to this in MLlib docs. Author: Sean Owen Closes #2787 from srowen/SPARK-1307 and squashes the following commits: b5b82e2 [Sean Owen] Refer to "self-contained" rather than "standalone" apps to avoid confusion with standalone deployment mode. And fix placement of reference to this in MLlib docs. --- docs/mllib-clustering.md | 14 +++++++------- docs/mllib-collaborative-filtering.md | 14 +++++++------- docs/mllib-dimensionality-reduction.md | 17 +++++++++-------- docs/mllib-linear-methods.md | 20 ++++++++++---------- docs/quick-start.md | 8 ++++---- 5 files changed, 37 insertions(+), 36 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index d10bd63746629..7978e934fb36b 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -69,7 +69,7 @@ println("Within Set Sum of Squared Errors = " + WSSSE) All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. A standalone application example +calling `.rdd()` on your `JavaRDD` object. A self-contained application example that is equivalent to the provided example in Scala is given below: {% highlight java %} @@ -113,12 +113,6 @@ public class KMeansExample { } } {% endhighlight %} - -In order to run the above standalone application, follow the instructions -provided in the [Standalone -Applications](quick-start.html#standalone-applications) section of the Spark -quick-start guide. Be sure to also include *spark-mllib* to your build file as -a dependency.
    @@ -153,3 +147,9 @@ print("Within Set Sum of Squared Error = " + str(WSSSE))
    + +In order to run the above application, follow the instructions +provided in the [Self-Contained Applications](quick-start.html#self-contained-applications) +section of the Spark +Quick Start guide. Be sure to also include *spark-mllib* to your build file as +a dependency. diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index d5c539db791be..2094963392295 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -110,7 +110,7 @@ val model = ALS.trainImplicit(ratings, rank, numIterations, alpha) All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. A standalone application example +calling `.rdd()` on your `JavaRDD` object. A self-contained application example that is equivalent to the provided example in Scala is given bellow: {% highlight java %} @@ -184,12 +184,6 @@ public class CollaborativeFiltering { } } {% endhighlight %} - -In order to run the above standalone application, follow the instructions -provided in the [Standalone -Applications](quick-start.html#standalone-applications) section of the Spark -quick-start guide. Be sure to also include *spark-mllib* to your build file as -a dependency.
    @@ -229,6 +223,12 @@ model = ALS.trainImplicit(ratings, rank, numIterations, alpha = 0.01)
    +In order to run the above application, follow the instructions +provided in the [Self-Contained Applications](quick-start.html#self-contained-applications) +section of the Spark +Quick Start guide. Be sure to also include *spark-mllib* to your build file as +a dependency. + ## Tutorial The [training exercises](https://databricks-training.s3.amazonaws.com/index.html) from the Spark Summit 2014 include a hands-on tutorial for diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 21cb35b4270ca..870fed6cc5024 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -121,9 +121,9 @@ public class SVD { The same code applies to `IndexedRowMatrix` if `U` is defined as an `IndexedRowMatrix`. -In order to run the above standalone application, follow the instructions -provided in the [Standalone -Applications](quick-start.html#standalone-applications) section of the Spark +In order to run the above application, follow the instructions +provided in the [Self-Contained +Applications](quick-start.html#self-contained-applications) section of the Spark quick-start guide. Be sure to also include *spark-mllib* to your build file as a dependency. @@ -200,10 +200,11 @@ public class PCA { } {% endhighlight %} -In order to run the above standalone application, follow the instructions -provided in the [Standalone -Applications](quick-start.html#standalone-applications) section of the Spark -quick-start guide. Be sure to also include *spark-mllib* to your build file as -a dependency. + +In order to run the above application, follow the instructions +provided in the [Self-Contained Applications](quick-start.html#self-contained-applications) +section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency. diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index d31bec3e1bd01..bc914a1899801 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -247,7 +247,7 @@ val modelL1 = svmAlg.run(training) All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. A standalone application example +calling `.rdd()` on your `JavaRDD` object. A self-contained application example that is equivalent to the provided example in Scala is given bellow: {% highlight java %} @@ -323,9 +323,9 @@ svmAlg.optimizer() final SVMModel modelL1 = svmAlg.run(training.rdd()); {% endhighlight %} -In order to run the above standalone application, follow the instructions -provided in the [Standalone -Applications](quick-start.html#standalone-applications) section of the Spark +In order to run the above application, follow the instructions +provided in the [Self-Contained +Applications](quick-start.html#self-contained-applications) section of the Spark quick-start guide. Be sure to also include *spark-mllib* to your build file as a dependency. @@ -482,12 +482,6 @@ public class LinearRegression { } } {% endhighlight %} - -In order to run the above standalone application, follow the instructions -provided in the [Standalone -Applications](quick-start.html#standalone-applications) section of the Spark -quick-start guide. Be sure to also include *spark-mllib* to your build file as -a dependency.
    @@ -519,6 +513,12 @@ print("Mean Squared Error = " + str(MSE))
    +In order to run the above application, follow the instructions +provided in the [Self-Contained Applications](quick-start.html#self-contained-applications) +section of the Spark +quick-start guide. Be sure to also include *spark-mllib* to your build file as +a dependency. + ## Streaming linear regression When data arrive in a streaming fashion, it is useful to fit regression models online, diff --git a/docs/quick-start.md b/docs/quick-start.md index 23313d8aa6152..6236de0e1f2c4 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -8,7 +8,7 @@ title: Quick Start This tutorial provides a quick introduction to using Spark. We will first introduce the API through Spark's interactive shell (in Python or Scala), -then show how to write standalone applications in Java, Scala, and Python. +then show how to write applications in Java, Scala, and Python. See the [programming guide](programming-guide.html) for a more complete reference. To follow along with this guide, first download a packaged release of Spark from the @@ -215,8 +215,8 @@ a cluster, as described in the [programming guide](programming-guide.html#initia -# Standalone Applications -Now say we wanted to write a standalone application using the Spark API. We will walk through a +# Self-Contained Applications +Now say we wanted to write a self-contained application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python.
    @@ -387,7 +387,7 @@ Lines with a: 46, Lines with b: 23
    -Now we will show how to write a standalone application using the Python API (PySpark). +Now we will show how to write an application using the Python API (PySpark). As an example, we'll create a simple Spark application, `SimpleApp.py`: From 293a0b5dbba0474832dc7e9d387f3b10f6c452ea Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Tue, 14 Oct 2014 22:16:38 -0700 Subject: [PATCH 0997/1492] [SPARK-2098] All Spark processes should support spark-defaults.conf, config file This is another implementation about #1256 cc andrewor14 vanzin Author: GuoQiang Li Closes #2379 from witgo/SPARK-2098-new and squashes the following commits: 4ef1cbd [GuoQiang Li] review commit 49ef70e [GuoQiang Li] Refactor getDefaultPropertiesFile c45d20c [GuoQiang Li] All Spark processes should support spark-defaults.conf, config file --- .../spark/deploy/SparkSubmitArguments.scala | 42 ++-------------- .../SparkSubmitDriverBootstrapper.scala | 2 +- .../history/HistoryServerArguments.scala | 16 ++++++- .../spark/deploy/master/MasterArguments.scala | 19 ++++++-- .../spark/deploy/worker/WorkerArguments.scala | 21 ++++++-- .../scala/org/apache/spark/util/Utils.scala | 48 +++++++++++++++++++ .../org/apache/spark/util/UtilsSuite.scala | 19 ++++++++ docs/monitoring.md | 7 +++ 8 files changed, 124 insertions(+), 50 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 57b251ff47714..72a452e0aefb5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,14 +17,11 @@ package org.apache.spark.deploy -import java.io.{File, FileInputStream, IOException} -import java.util.Properties import java.util.jar.JarFile import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} -import org.apache.spark.SparkException import org.apache.spark.util.Utils /** @@ -63,9 +60,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St val defaultProperties = new HashMap[String, String]() if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") Option(propertiesFile).foreach { filename => - val file = new File(filename) - SparkSubmitArguments.getPropertiesFromFile(file).foreach { case (k, v) => - if (k.startsWith("spark")) { + Utils.getPropertiesFromFile(filename).foreach { case (k, v) => + if (k.startsWith("spark.")) { defaultProperties(k) = v if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") } else { @@ -90,19 +86,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St */ private def mergeSparkProperties(): Unit = { // Use common defaults file, if not specified by user - if (propertiesFile == null) { - val sep = File.separator - val sparkHomeConfig = env.get("SPARK_HOME").map(sparkHome => s"${sparkHome}${sep}conf") - val confDir = env.get("SPARK_CONF_DIR").orElse(sparkHomeConfig) - - confDir.foreach { sparkConfDir => - val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" - val file = new File(defaultPath) - if (file.exists()) { - propertiesFile = file.getAbsolutePath - } - } - } + propertiesFile = Option(propertiesFile).getOrElse(Utils.getDefaultPropertiesFile(env)) val properties = HashMap[String, String]() properties.putAll(defaultSparkProperties) @@ -397,23 +381,3 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St SparkSubmit.exitFn() } } - -object SparkSubmitArguments { - /** Load properties present in the given file. */ - def getPropertiesFromFile(file: File): Seq[(String, String)] = { - require(file.exists(), s"Properties file $file does not exist") - require(file.isFile(), s"Properties file $file is not a normal file") - val inputStream = new FileInputStream(file) - try { - val properties = new Properties() - properties.load(inputStream) - properties.stringPropertyNames().toSeq.map(k => (k, properties(k).trim)) - } catch { - case e: IOException => - val message = s"Failed when loading Spark properties file $file" - throw new SparkException(message, e) - } finally { - inputStream.close() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index a64170a47bc1c..0125330589da5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -68,7 +68,7 @@ private[spark] object SparkSubmitDriverBootstrapper { assume(bootstrapDriver != null, "SPARK_SUBMIT_BOOTSTRAP_DRIVER must be set") // Parse the properties file for the equivalent spark.driver.* configs - val properties = SparkSubmitArguments.getPropertiesFromFile(new File(propertiesFile)).toMap + val properties = Utils.getPropertiesFromFile(propertiesFile) val confDriverMemory = properties.get("spark.driver.memory") val confLibraryPath = properties.get("spark.driver.extraLibraryPath") val confClasspath = properties.get("spark.driver.extraClassPath") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 25fc76c23e0fb..5bce32a04d16d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -18,12 +18,14 @@ package org.apache.spark.deploy.history import org.apache.spark.SparkConf +import org.apache.spark.util.Utils /** * Command-line parser for the master. */ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) { private var logDir: String = null + private var propertiesFile: String = null parse(args.toList) @@ -32,11 +34,16 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] case ("--dir" | "-d") :: value :: tail => logDir = value conf.set("spark.history.fs.logDirectory", value) + System.setProperty("spark.history.fs.logDirectory", value) parse(tail) case ("--help" | "-h") :: tail => printUsageAndExit(0) + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + case Nil => case _ => @@ -44,10 +51,17 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] } } + // This mutates the SparkConf, so all accesses to it must be made after this line + Utils.loadDefaultSparkProperties(conf, propertiesFile) + private def printUsageAndExit(exitCode: Int) { System.err.println( """ - |Usage: HistoryServer + |Usage: HistoryServer [options] + | + |Options: + | --properties-file FILE Path to a custom Spark properties file. + | Default is conf/spark-defaults.conf. | |Configuration options can be set by setting the corresponding JVM system property. |History Server options are always available; additional options depend on the provider. diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 4b0dbbe543d3f..e34bee7854292 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -27,6 +27,7 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { var host = Utils.localHostName() var port = 7077 var webUiPort = 8080 + var propertiesFile: String = null // Check for settings in environment variables if (System.getenv("SPARK_MASTER_HOST") != null) { @@ -38,12 +39,16 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { if (System.getenv("SPARK_MASTER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_MASTER_WEBUI_PORT").toInt } + + parse(args.toList) + + // This mutates the SparkConf, so all accesses to it must be made after this line + propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) + if (conf.contains("spark.master.ui.port")) { webUiPort = conf.get("spark.master.ui.port").toInt } - parse(args.toList) - def parse(args: List[String]): Unit = args match { case ("--ip" | "-i") :: value :: tail => Utils.checkHost(value, "ip no longer supported, please use hostname " + value) @@ -63,7 +68,11 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { webUiPort = value parse(tail) - case ("--help" | "-h") :: tail => + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--help") :: tail => printUsageAndExit(0) case Nil => {} @@ -83,7 +92,9 @@ private[spark] class MasterArguments(args: Array[String], conf: SparkConf) { " -i HOST, --ip HOST Hostname to listen on (deprecated, please use --host or -h) \n" + " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: 7077)\n" + - " --webui-port PORT Port for web UI (default: 8080)") + " --webui-port PORT Port for web UI (default: 8080)\n" + + " --properties-file FILE Path to a custom Spark properties file.\n" + + " Default is conf/spark-defaults.conf.") System.exit(exitCode) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 54e3937edde6b..019cd70f2a229 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -33,6 +33,7 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { var memory = inferDefaultMemory() var masters: Array[String] = null var workDir: String = null + var propertiesFile: String = null // Check for settings in environment variables if (System.getenv("SPARK_WORKER_PORT") != null) { @@ -47,15 +48,19 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt } - if (conf.contains("spark.worker.ui.port")) { - webUiPort = conf.get("spark.worker.ui.port").toInt - } if (System.getenv("SPARK_WORKER_DIR") != null) { workDir = System.getenv("SPARK_WORKER_DIR") } parse(args.toList) + // This mutates the SparkConf, so all accesses to it must be made after this line + propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) + + if (conf.contains("spark.worker.ui.port")) { + webUiPort = conf.get("spark.worker.ui.port").toInt + } + checkWorkerMemory() def parse(args: List[String]): Unit = args match { @@ -89,7 +94,11 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { webUiPort = value parse(tail) - case ("--help" | "-h") :: tail => + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--help") :: tail => printUsageAndExit(0) case value :: tail => @@ -124,7 +133,9 @@ private[spark] class WorkerArguments(args: Array[String], conf: SparkConf) { " -i HOST, --ip IP Hostname to listen on (deprecated, please use --host or -h)\n" + " -h HOST, --host HOST Hostname to listen on\n" + " -p PORT, --port PORT Port to listen on (default: random)\n" + - " --webui-port PORT Port for web UI (default: 8081)") + " --webui-port PORT Port for web UI (default: 8081)\n" + + " --properties-file FILE Path to a custom Spark properties file.\n" + + " Default is conf/spark-defaults.conf.") System.exit(exitCode) } 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 aad901620f53e..cbc4095065a19 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1410,6 +1410,54 @@ private[spark] object Utils extends Logging { } } + /** + * Load default Spark properties from the given file. If no file is provided, + * use the common defaults file. This mutates state in the given SparkConf and + * in this JVM's system properties if the config specified in the file is not + * already set. Return the path of the properties file used. + */ + def loadDefaultSparkProperties(conf: SparkConf, filePath: String = null): String = { + val path = Option(filePath).getOrElse(getDefaultPropertiesFile()) + Option(path).foreach { confFile => + getPropertiesFromFile(confFile).filter { case (k, v) => + k.startsWith("spark.") + }.foreach { case (k, v) => + conf.setIfMissing(k, v) + sys.props.getOrElseUpdate(k, v) + } + } + path + } + + /** Load properties present in the given file. */ + def getPropertiesFromFile(filename: String): Map[String, String] = { + val file = new File(filename) + require(file.exists(), s"Properties file $file does not exist") + require(file.isFile(), s"Properties file $file is not a normal file") + + val inReader = new InputStreamReader(new FileInputStream(file), "UTF-8") + try { + val properties = new Properties() + properties.load(inReader) + properties.stringPropertyNames().map(k => (k, properties(k).trim)).toMap + } catch { + case e: IOException => + throw new SparkException(s"Failed when loading Spark properties from $filename", e) + } finally { + inReader.close() + } + } + + /** Return the path of the default Spark properties file. */ + def getDefaultPropertiesFile(env: Map[String, String] = sys.env): String = { + env.get("SPARK_CONF_DIR") + .orElse(env.get("SPARK_HOME").map { t => s"$t${File.separator}conf" }) + .map { t => new File(s"$t${File.separator}spark-defaults.conf")} + .filter(_.isFile) + .map(_.getAbsolutePath) + .orNull + } + /** Return a nice string representation of the exception, including the stack trace. */ def exceptionString(e: Exception): String = { if (e == null) "" else exceptionString(getFormattedClassName(e), e.getMessage, e.getStackTrace) diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 0344da60dae66..ea7ef0524d1e1 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -27,6 +27,8 @@ import com.google.common.base.Charsets import com.google.common.io.Files import org.scalatest.FunSuite +import org.apache.spark.SparkConf + class UtilsSuite extends FunSuite { test("bytesToString") { @@ -332,4 +334,21 @@ class UtilsSuite extends FunSuite { assert(!tempFile2.exists()) } + test("loading properties from file") { + val outFile = File.createTempFile("test-load-spark-properties", "test") + try { + System.setProperty("spark.test.fileNameLoadB", "2") + Files.write("spark.test.fileNameLoadA true\n" + + "spark.test.fileNameLoadB 1\n", outFile, Charsets.UTF_8) + val properties = Utils.getPropertiesFromFile(outFile.getAbsolutePath) + properties + .filter { case (k, v) => k.startsWith("spark.")} + .foreach { case (k, v) => sys.props.getOrElseUpdate(k, v)} + val sparkConf = new SparkConf + assert(sparkConf.getBoolean("spark.test.fileNameLoadA", false) === true) + assert(sparkConf.getInt("spark.test.fileNameLoadB", 1) === 2) + } finally { + outFile.delete() + } + } } diff --git a/docs/monitoring.md b/docs/monitoring.md index d07ec4a57a2cc..e3f81a76acdbb 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -77,6 +77,13 @@ follows: one implementation, provided by Spark, which looks for application logs stored in the file system. +
    + + + + From 044583a241203e7fe759366b273ad32fd9bf7c05 Mon Sep 17 00:00:00 2001 From: prudhvi Date: Thu, 16 Oct 2014 02:05:44 -0400 Subject: [PATCH 0998/1492] [Core] Upgrading ScalaStyle version to 0.5 and removing SparkSpaceAfterCommentStartChecker. Author: prudhvi Closes #2799 from prudhvije/ScalaStyle/space-after-comment-start and squashes the following commits: fc263a1 [prudhvi] [Core] Using scalastyle to check the space after comment start --- project/plugins.sbt | 2 +- .../SparkSpaceAfterCommentStartChecker.scala | 58 ------------------- scalastyle-config.xml | 2 +- 3 files changed, 2 insertions(+), 60 deletions(-) delete mode 100644 project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala diff --git a/project/plugins.sbt b/project/plugins.sbt index 8096c61414660..678f5ed1ba610 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -17,7 +17,7 @@ addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0") addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") -addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.5.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala deleted file mode 100644 index 80d3faa3fe749..0000000000000 --- a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - - -package org.apache.spark.scalastyle - -import java.util.regex.Pattern - -import org.scalastyle.{PositionError, ScalariformChecker, ScalastyleError} -import scalariform.lexer.{MultiLineComment, ScalaDocComment, SingleLineComment, Token} -import scalariform.parser.CompilationUnit - -class SparkSpaceAfterCommentStartChecker extends ScalariformChecker { - val errorKey: String = "insert.a.single.space.after.comment.start.and.before.end" - - private def multiLineCommentRegex(comment: Token) = - Pattern.compile( """/\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() || - Pattern.compile( """/\*.*\S\*/""", Pattern.DOTALL).matcher(comment.text.trim).matches() - - private def scalaDocPatternRegex(comment: Token) = - Pattern.compile( """/\*\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() || - Pattern.compile( """/\*\*.*\S\*/""", Pattern.DOTALL).matcher(comment.text.trim).matches() - - private def singleLineCommentRegex(comment: Token): Boolean = - comment.text.trim.matches( """//\S+.*""") && !comment.text.trim.matches( """///+""") - - override def verify(ast: CompilationUnit): List[ScalastyleError] = { - ast.tokens - .filter(hasComment) - .map { - _.associatedWhitespaceAndComments.comments.map { - case x: SingleLineComment if singleLineCommentRegex(x.token) => Some(x.token.offset) - case x: MultiLineComment if multiLineCommentRegex(x.token) => Some(x.token.offset) - case x: ScalaDocComment if scalaDocPatternRegex(x.token) => Some(x.token.offset) - case _ => None - }.flatten - }.flatten.map(PositionError(_)) - } - - - private def hasComment(x: Token) = - x.associatedWhitespaceAndComments != null && !x.associatedWhitespaceAndComments.comments.isEmpty - -} diff --git a/scalastyle-config.xml b/scalastyle-config.xml index c54f8b72ebf42..0ff521706c71a 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -141,5 +141,5 @@ - + From 4c589cac4496c6a4bb8485a340bd0641dca13847 Mon Sep 17 00:00:00 2001 From: Shiti Date: Thu, 16 Oct 2014 10:52:06 -0700 Subject: [PATCH 0999/1492] [SPARK-3944][Core] Code re-factored as suggested Author: Shiti Closes #2810 from Shiti/master and squashes the following commits: 051d82f [Shiti] setting the default value of uri scheme to "file" where matching "file" or None yields the same result --- .../main/scala/org/apache/spark/util/Utils.scala | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index cbc4095065a19..53a7512edd852 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -340,8 +340,8 @@ private[spark] object Utils extends Logging { val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) - Option(uri.getScheme) match { - case Some("http") | Some("https") | Some("ftp") => + Option(uri.getScheme).getOrElse("file") match { + case "http" | "https" | "ftp" => logInfo("Fetching " + url + " to " + tempFile) var uc: URLConnection = null @@ -374,7 +374,7 @@ private[spark] object Utils extends Logging { } } Files.move(tempFile, targetFile) - case Some("file") | None => + case "file" => // In the case of a local file, copy the local file to the target directory. // Note the difference between uri vs url. val sourceFile = if (uri.isAbsolute) new File(uri) else new File(url) @@ -403,7 +403,7 @@ private[spark] object Utils extends Logging { logInfo("Copying " + sourceFile.getAbsolutePath + " to " + targetFile.getAbsolutePath) Files.copy(sourceFile, targetFile) } - case Some(other) => + case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) @@ -1401,10 +1401,10 @@ private[spark] object Utils extends Logging { paths.split(",").filter { p => val formattedPath = if (windows) formatWindowsPath(p) else p val uri = new URI(formattedPath) - Option(uri.getScheme) match { - case Some(windowsDrive(d)) if windows => false - case Some("local") | Some("file") | None => false - case Some(other) => true + Option(uri.getScheme).getOrElse("file") match { + case windowsDrive(d) if windows => false + case "local" | "file" => false + case _ => true } } } From 091d32c52e9d73da95896016c1d920e89858abfa Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Oct 2014 14:56:50 -0700 Subject: [PATCH 1000/1492] [SPARK-3971] [MLLib] [PySpark] hotfix: Customized pickler should work in cluster mode Customized pickler should be registered before unpickling, but in executor, there is no way to register the picklers before run the tasks. So, we need to register the picklers in the tasks itself, duplicate the javaToPython() and pythonToJava() in MLlib, call SerDe.initialize() before pickling or unpickling. Author: Davies Liu Closes #2830 from davies/fix_pickle and squashes the following commits: 0c85fb9 [Davies Liu] revert the privacy change 6b94e15 [Davies Liu] use JavaConverters instead of JavaConversions 0f02050 [Davies Liu] hotfix: Customized pickler does not work in cluster --- .../apache/spark/api/python/PythonRDD.scala | 7 ++- .../apache/spark/api/python/SerDeUtil.scala | 14 ++++- .../mllib/api/python/PythonMLLibAPI.scala | 52 +++++++++++++++++-- python/pyspark/context.py | 2 - python/pyspark/mllib/classification.py | 4 +- python/pyspark/mllib/clustering.py | 4 +- python/pyspark/mllib/feature.py | 5 +- python/pyspark/mllib/linalg.py | 13 +++++ python/pyspark/mllib/random.py | 2 +- python/pyspark/mllib/recommendation.py | 7 +-- python/pyspark/mllib/regression.py | 4 +- python/pyspark/mllib/stat.py | 7 +-- python/pyspark/mllib/tree.py | 8 +-- python/pyspark/mllib/util.py | 6 +-- 14 files changed, 101 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 4acbdf9d5e25f..29ca751519abd 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -23,6 +23,7 @@ import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.existentials @@ -746,6 +747,7 @@ private[spark] object PythonRDD extends Logging { def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { pyRDD.rdd.mapPartitions { iter => val unpickle = new Unpickler + SerDeUtil.initialize() iter.flatMap { row => unpickle.loads(row) match { // in case of objects are pickled in batch mode @@ -785,7 +787,7 @@ private[spark] object PythonRDD extends Logging { }.toJavaRDD() } - private class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { + private[spark] class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { private val pickle = new Pickler() private var batch = 1 private val buffer = new mutable.ArrayBuffer[Any] @@ -822,11 +824,12 @@ private[spark] object PythonRDD extends Logging { */ def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { pyRDD.rdd.mapPartitions { iter => + SerDeUtil.initialize() val unpickle = new Unpickler iter.flatMap { row => val obj = unpickle.loads(row) if (batched) { - obj.asInstanceOf[JArrayList[_]] + obj.asInstanceOf[JArrayList[_]].asScala } else { Seq(obj) } diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 7903457b17e13..ebdc3533e0992 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -29,7 +29,7 @@ import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ -private[python] object SerDeUtil extends Logging { +private[spark] object SerDeUtil extends Logging { // Unpickle array.array generated by Python 2.6 class ArrayConstructor extends net.razorvine.pickle.objects.ArrayConstructor { // /* Description of types */ @@ -76,9 +76,18 @@ private[python] object SerDeUtil extends Logging { } } + private var initialized = false + // This should be called before trying to unpickle array.array from Python + // In cluster mode, this should be put in closure def initialize() = { - Unpickler.registerConstructor("array", "array", new ArrayConstructor()) + synchronized{ + if (!initialized) { + Unpickler.registerConstructor("array", "array", new ArrayConstructor()) + initialized = true + } + } } + initialize() private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { val pickle = new Pickler @@ -143,6 +152,7 @@ private[python] object SerDeUtil extends Logging { obj.asInstanceOf[Array[_]].length == 2 } pyRDD.mapPartitions { iter => + initialize() val unpickle = new Unpickler val unpickled = if (batchSerialized) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index f7251e65e04f1..9a100170b75c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream +import java.util.{ArrayList => JArrayList} import scala.collection.JavaConverters._ import scala.language.existentials @@ -27,6 +28,7 @@ import net.razorvine.pickle._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.feature.Word2Vec @@ -639,13 +641,24 @@ private[spark] object SerDe extends Serializable { } } + var initialized = false + // This should be called before trying to serialize any above classes + // In cluster mode, this should be put in the closure def initialize(): Unit = { - new DenseVectorPickler().register() - new DenseMatrixPickler().register() - new SparseVectorPickler().register() - new LabeledPointPickler().register() - new RatingPickler().register() + SerDeUtil.initialize() + synchronized { + if (!initialized) { + new DenseVectorPickler().register() + new DenseMatrixPickler().register() + new SparseVectorPickler().register() + new LabeledPointPickler().register() + new RatingPickler().register() + initialized = true + } + } } + // will not called in Executor automatically + initialize() def dumps(obj: AnyRef): Array[Byte] = { new Pickler().dumps(obj) @@ -659,4 +672,33 @@ private[spark] object SerDe extends Serializable { def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) } + + /** + * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + new PythonRDD.AutoBatchedPickler(iter) + } + } + + /** + * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. + */ + def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { + pyRDD.rdd.mapPartitions { iter => + initialize() // let it called in executor + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj.asInstanceOf[JArrayList[_]].asScala + } else { + Seq(obj) + } + } + }.toJavaRDD() + } } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 89d2e2e5b4a8e..8d27ccb95f82c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -215,8 +215,6 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._gateway = gateway or launch_gateway() SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile - SparkContext._jvm.SerDeUtil.initialize() - SparkContext._jvm.SerDe.initialize() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index cd43982191702..e295c9d0954d9 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -21,7 +21,7 @@ from numpy import array from pyspark import SparkContext, PickleSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper @@ -244,7 +244,7 @@ def train(cls, data, lambda_=1.0): :param lambda_: The smoothing parameter """ sc = data.context - jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(data._to_java_object_rdd(), lambda_) + jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(_to_java_object_rdd(data), lambda_) labels, pi, theta = PickleSerializer().loads(str(sc._jvm.SerDe.dumps(jlist))) return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 12c56022717a5..5ee7997104d21 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -17,7 +17,7 @@ from pyspark import SparkContext from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd __all__ = ['KMeansModel', 'KMeans'] @@ -85,7 +85,7 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" # cache serialized data to avoid objects over head in JVM cached = rdd.map(_convert_to_vector)._reserialize(AutoBatchedSerializer(ser)).cache() model = sc._jvm.PythonMLLibAPI().trainKMeansModel( - cached._to_java_object_rdd(), k, maxIterations, runs, initializationMode) + _to_java_object_rdd(cached), k, maxIterations, runs, initializationMode) bytes = sc._jvm.SerDe.dumps(model.clusterCenters()) centers = ser.loads(str(bytes)) return KMeansModel([c.toArray() for c in centers]) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index f4cbf31b94fe2..b5a3f22c6907e 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -19,8 +19,7 @@ Python package for feature in MLlib. """ from pyspark.serializers import PickleSerializer, AutoBatchedSerializer - -from pyspark.mllib.linalg import _convert_to_vector +from pyspark.mllib.linalg import _convert_to_vector, _to_java_object_rdd __all__ = ['Word2Vec', 'Word2VecModel'] @@ -176,7 +175,7 @@ def fit(self, data): seed = self.seed model = sc._jvm.PythonMLLibAPI().trainWord2Vec( - data._to_java_object_rdd(), vectorSize, + _to_java_object_rdd(data), vectorSize, learningRate, numPartitions, numIterations, seed) return Word2VecModel(sc, model) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 24c5480b2f753..773d8d393805d 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -29,6 +29,8 @@ import numpy as np +from pyspark.serializers import AutoBatchedSerializer, PickleSerializer + __all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] @@ -50,6 +52,17 @@ def fast_pickle_array(ar): _have_scipy = False +# this will call the MLlib version of pythonToJava() +def _to_java_object_rdd(rdd): + """ Return an JavaRDD of Object by unpickling + + It will convert each Python object into Java object by Pyrolite, whenever the + RDD is serialized in batch or not. + """ + rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) + return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) + + def _convert_to_vector(l): if isinstance(l, Vector): return l diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index a787e4dea2c55..73baba4ace5f6 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -32,7 +32,7 @@ def serialize(f): @wraps(f) def func(sc, *a, **kw): jrdd = f(sc, *a, **kw) - return RDD(sc._jvm.PythonRDD.javaToPython(jrdd), sc, + return RDD(sc._jvm.SerDe.javaToPython(jrdd), sc, BatchedSerializer(PickleSerializer(), 1024)) return func diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 59c1c5ff0ced0..17f96b8700bd7 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -18,6 +18,7 @@ from pyspark import SparkContext from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.rdd import RDD +from pyspark.mllib.linalg import _to_java_object_rdd __all__ = ['MatrixFactorizationModel', 'ALS'] @@ -77,9 +78,9 @@ def predictAll(self, user_product): first = tuple(map(int, first)) assert all(type(x) is int for x in first), "user and product in user_product shoul be int" sc = self._context - tuplerdd = sc._jvm.SerDe.asTupleRDD(user_product._to_java_object_rdd().rdd()) + tuplerdd = sc._jvm.SerDe.asTupleRDD(_to_java_object_rdd(user_product).rdd()) jresult = self._java_model.predict(tuplerdd).toJavaRDD() - return RDD(sc._jvm.PythonRDD.javaToPython(jresult), sc, + return RDD(sc._jvm.SerDe.javaToPython(jresult), sc, AutoBatchedSerializer(PickleSerializer())) @@ -97,7 +98,7 @@ def _prepare(cls, ratings): # serialize them by AutoBatchedSerializer before cache to reduce the # objects overhead in JVM cached = ratings._reserialize(AutoBatchedSerializer(PickleSerializer())).cache() - return cached._to_java_object_rdd() + return _to_java_object_rdd(cached) @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 12b322aaae796..93e17faf5cd51 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -19,8 +19,8 @@ from numpy import array from pyspark import SparkContext -from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.serializers import PickleSerializer, AutoBatchedSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -131,7 +131,7 @@ def _regression_train_wrapper(sc, train_func, modelClass, data, initial_weights) # use AutoBatchedSerializer before cache to reduce the memory # overhead in JVM cached = data._reserialize(AutoBatchedSerializer(ser)).cache() - ans = train_func(cached._to_java_object_rdd(), initial_bytes) + ans = train_func(_to_java_object_rdd(cached), initial_bytes) assert len(ans) == 2, "JVM call result had unexpected length" weights = ser.loads(str(ans[0])) return modelClass(weights, ans[1]) diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index b9de0909a6fb1..a6019dadf781c 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -22,6 +22,7 @@ from functools import wraps from pyspark import PickleSerializer +from pyspark.mllib.linalg import _to_java_object_rdd __all__ = ['MultivariateStatisticalSummary', 'Statistics'] @@ -106,7 +107,7 @@ def colStats(rdd): array([ 2., 0., 0., -2.]) """ sc = rdd.ctx - jrdd = rdd._to_java_object_rdd() + jrdd = _to_java_object_rdd(rdd) cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) return MultivariateStatisticalSummary(sc, cStats) @@ -162,14 +163,14 @@ def corr(x, y=None, method=None): if type(y) == str: raise TypeError("Use 'method=' to specify method name.") - jx = x._to_java_object_rdd() + jx = _to_java_object_rdd(x) if not y: resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) bytes = sc._jvm.SerDe.dumps(resultMat) ser = PickleSerializer() return ser.loads(str(bytes)).toArray() else: - jy = y._to_java_object_rdd() + jy = _to_java_object_rdd(y) return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 5d7abfb96b7fe..0938eebd3a548 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -19,7 +19,7 @@ from pyspark import SparkContext, RDD from pyspark.serializers import BatchedSerializer, PickleSerializer -from pyspark.mllib.linalg import Vector, _convert_to_vector +from pyspark.mllib.linalg import Vector, _convert_to_vector, _to_java_object_rdd from pyspark.mllib.regression import LabeledPoint __all__ = ['DecisionTreeModel', 'DecisionTree'] @@ -61,8 +61,8 @@ def predict(self, x): return self._sc.parallelize([]) if not isinstance(first[0], Vector): x = x.map(_convert_to_vector) - jPred = self._java_model.predict(x._to_java_object_rdd()).toJavaRDD() - jpyrdd = self._sc._jvm.PythonRDD.javaToPython(jPred) + jPred = self._java_model.predict(_to_java_object_rdd(x)).toJavaRDD() + jpyrdd = self._sc._jvm.SerDe.javaToPython(jPred) return RDD(jpyrdd, self._sc, BatchedSerializer(ser, 1024)) else: @@ -104,7 +104,7 @@ def _train(data, type, numClasses, categoricalFeaturesInfo, first = data.first() assert isinstance(first, LabeledPoint), "the data should be RDD of LabeledPoint" sc = data.context - jrdd = data._to_java_object_rdd() + jrdd = _to_java_object_rdd(data) cfiMap = MapConverter().convert(categoricalFeaturesInfo, sc._gateway._gateway_client) model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 1357fd4fbc8aa..84b39a48619d2 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -19,7 +19,7 @@ import warnings from pyspark.rdd import RDD -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.serializers import AutoBatchedSerializer, PickleSerializer from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -174,8 +174,8 @@ def loadLabeledPoints(sc, path, minPartitions=None): """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) jrdd = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) - jpyrdd = sc._jvm.PythonRDD.javaToPython(jrdd) - return RDD(jpyrdd, sc, BatchedSerializer(PickleSerializer())) + jpyrdd = sc._jvm.SerDe.javaToPython(jrdd) + return RDD(jpyrdd, sc, AutoBatchedSerializer(PickleSerializer())) def _test(): From 99e416b6d64402a5432a265797a1c155a38f4e6f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 16 Oct 2014 16:15:55 -0700 Subject: [PATCH 1001/1492] [SQL] Fixes the race condition that may cause test failure The removed `Future` was used to end the test case as soon as the Spark SQL CLI process exits. When the process exits prematurely, this mechanism prevents the test case to wait until timeout. But it also creates a race condition: when `foundAllExpectedAnswers.tryFailure` is called, there are chances that the last expected output line of the CLI process hasn't been caught by the main logics of the test code, thus fails the test case. Removing this `Future` doesn't affect correctness. Author: Cheng Lian Closes #2823 from liancheng/clean-clisuite and squashes the following commits: 489a97c [Cheng Lian] Fixes the race condition that may cause test failure --- .../org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 6 ------ 1 file changed, 6 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index fc97a25be34be..8a72e9d2aef57 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -78,12 +78,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { val process = (Process(command) #< queryStream).run( ProcessLogger(captureOutput("stdout"), captureOutput("stderr"))) - Future { - val exitValue = process.exitValue() - foundAllExpectedAnswers.tryFailure( - new SparkException(s"Spark SQL CLI process exit value: $exitValue")) - } - try { Await.result(foundAllExpectedAnswers.future, timeout) } catch { case cause: Throwable => From 2fe0ba95616bb3860736b6b426635a5d2a0e9bd9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 16 Oct 2014 21:38:45 -0400 Subject: [PATCH 1002/1492] SPARK-3874: Provide stable TaskContext API This is a small number of clean-up changes on top of #2782. Closes #2782. Author: Prashant Sharma Author: Patrick Wendell Closes #2803 from pwendell/pr-2782 and squashes the following commits: 56d5b7a [Patrick Wendell] Minor clean-up 44089ec [Patrick Wendell] Clean-up the TaskContext API. ed551ce [Prashant Sharma] Fixed a typo df261d0 [Prashant Sharma] Josh's suggestion facf3b1 [Prashant Sharma] Fixed the mima issue. 7ecc2fe [Prashant Sharma] CR, Moved implementations to TaskContextImpl bbd9e05 [Prashant Sharma] adding missed out files to git. ef633f5 [Prashant Sharma] SPARK-3874, Provide stable TaskContext API --- .../java/org/apache/spark/TaskContext.java | 225 +++--------------- .../org/apache/spark/TaskContextHelper.scala | 29 +++ .../org/apache/spark/TaskContextImpl.scala | 91 +++++++ .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 8 +- .../apache/spark/scheduler/DAGScheduler.scala | 6 +- .../org/apache/spark/scheduler/Task.scala | 10 +- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../util/JavaTaskCompletionListenerImpl.java | 4 +- .../org/apache/spark/CacheManagerSuite.scala | 8 +- .../org/apache/spark/rdd/PipedRDDSuite.scala | 2 +- .../spark/scheduler/TaskContextSuite.scala | 2 +- .../ShuffleBlockFetcherIteratorSuite.scala | 8 +- project/MimaBuild.scala | 2 +- project/MimaExcludes.scala | 6 +- .../sql/parquet/ParquetTableOperations.scala | 4 +- 16 files changed, 186 insertions(+), 223 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/TaskContextHelper.scala create mode 100644 core/src/main/scala/org/apache/spark/TaskContextImpl.scala diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java index 4e6d708af0ea7..2d998d4c7a5d9 100644 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -18,131 +18,55 @@ package org.apache.spark; import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; import scala.Function0; import scala.Function1; import scala.Unit; -import scala.collection.JavaConversions; import org.apache.spark.annotation.DeveloperApi; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.util.TaskCompletionListener; -import org.apache.spark.util.TaskCompletionListenerException; /** -* :: DeveloperApi :: -* Contextual information about a task which can be read or mutated during execution. -*/ -@DeveloperApi -public class TaskContext implements Serializable { - - private int stageId; - private int partitionId; - private long attemptId; - private boolean runningLocally; - private TaskMetrics taskMetrics; - - /** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task - * @param runningLocally whether the task is running locally in the driver JVM - * @param taskMetrics performance metrics of the task - */ - @DeveloperApi - public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally, - TaskMetrics taskMetrics) { - this.attemptId = attemptId; - this.partitionId = partitionId; - this.runningLocally = runningLocally; - this.stageId = stageId; - this.taskMetrics = taskMetrics; - } - - /** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task - * @param runningLocally whether the task is running locally in the driver JVM - */ - @DeveloperApi - public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally) { - this.attemptId = attemptId; - this.partitionId = partitionId; - this.runningLocally = runningLocally; - this.stageId = stageId; - this.taskMetrics = TaskMetrics.empty(); - } - + * Contextual information about a task which can be read or mutated during + * execution. To access the TaskContext for a running task use + * TaskContext.get(). + */ +public abstract class TaskContext implements Serializable { /** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task + * Return the currently active TaskContext. This can be called inside of + * user functions to access contextual information about running tasks. */ - @DeveloperApi - public TaskContext(int stageId, int partitionId, long attemptId) { - this.attemptId = attemptId; - this.partitionId = partitionId; - this.runningLocally = false; - this.stageId = stageId; - this.taskMetrics = TaskMetrics.empty(); + public static TaskContext get() { + return taskContext.get(); } private static ThreadLocal taskContext = new ThreadLocal(); - /** - * :: Internal API :: - * This is spark internal API, not intended to be called from user programs. - */ - public static void setTaskContext(TaskContext tc) { + static void setTaskContext(TaskContext tc) { taskContext.set(tc); } - public static TaskContext get() { - return taskContext.get(); - } - - /** :: Internal API :: */ - public static void unset() { + static void unset() { taskContext.remove(); } - // List of callback functions to execute when the task completes. - private transient List onCompleteCallbacks = - new ArrayList(); - - // Whether the corresponding task has been killed. - private volatile boolean interrupted = false; - - // Whether the task has completed. - private volatile boolean completed = false; - /** - * Checks whether the task has completed. + * Whether the task has completed. */ - public boolean isCompleted() { - return completed; - } + public abstract boolean isCompleted(); /** - * Checks whether the task has been killed. + * Whether the task has been killed. */ - public boolean isInterrupted() { - return interrupted; - } + public abstract boolean isInterrupted(); + + /** @deprecated: use isRunningLocally() */ + @Deprecated + public abstract boolean runningLocally(); + + public abstract boolean isRunningLocally(); /** * Add a (Java friendly) listener to be executed on task completion. @@ -150,10 +74,7 @@ public boolean isInterrupted() { *

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

    * An example use is for HadoopRDD to register a callback to close the input stream. */ - public TaskContext addTaskCompletionListener(final Function1 f) { - onCompleteCallbacks.add(new TaskCompletionListener() { - @Override - public void onTaskCompletion(TaskContext context) { - f.apply(context); - } - }); - return this; - } + public abstract TaskContext addTaskCompletionListener(final Function1 f); /** * Add a callback function to be executed on task completion. An example use * is for HadoopRDD to register a callback to close the input stream. * Will be called in any situation - success, failure, or cancellation. * - * Deprecated: use addTaskCompletionListener - * + * @deprecated: use addTaskCompletionListener + * * @param f Callback function. */ @Deprecated - public void addOnCompleteCallback(final Function0 f) { - onCompleteCallbacks.add(new TaskCompletionListener() { - @Override - public void onTaskCompletion(TaskContext context) { - f.apply(); - } - }); - } - - /** - * ::Internal API:: - * Marks the task as completed and triggers the listeners. - */ - public void markTaskCompleted() throws TaskCompletionListenerException { - completed = true; - List errorMsgs = new ArrayList(2); - // Process complete callbacks in the reverse order of registration - List revlist = - new ArrayList(onCompleteCallbacks); - Collections.reverse(revlist); - for (TaskCompletionListener tcl: revlist) { - try { - tcl.onTaskCompletion(this); - } catch (Throwable e) { - errorMsgs.add(e.getMessage()); - } - } - - if (!errorMsgs.isEmpty()) { - throw new TaskCompletionListenerException(JavaConversions.asScalaBuffer(errorMsgs)); - } - } - - /** - * ::Internal API:: - * Marks the task for interruption, i.e. cancellation. - */ - public void markInterrupted() { - interrupted = true; - } - - @Deprecated - /** Deprecated: use getStageId() */ - public int stageId() { - return stageId; - } - - @Deprecated - /** Deprecated: use getPartitionId() */ - public int partitionId() { - return partitionId; - } - - @Deprecated - /** Deprecated: use getAttemptId() */ - public long attemptId() { - return attemptId; - } - - @Deprecated - /** Deprecated: use isRunningLocally() */ - public boolean runningLocally() { - return runningLocally; - } - - public boolean isRunningLocally() { - return runningLocally; - } + public abstract void addOnCompleteCallback(final Function0 f); - public int getStageId() { - return stageId; - } + public abstract int stageId(); - public int getPartitionId() { - return partitionId; - } + public abstract int partitionId(); - public long getAttemptId() { - return attemptId; - } + public abstract long attemptId(); - /** ::Internal API:: */ - public TaskMetrics taskMetrics() { - return taskMetrics; - } + /** ::DeveloperApi:: */ + @DeveloperApi + public abstract TaskMetrics taskMetrics(); } diff --git a/core/src/main/scala/org/apache/spark/TaskContextHelper.scala b/core/src/main/scala/org/apache/spark/TaskContextHelper.scala new file mode 100644 index 0000000000000..4636c4600a01a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskContextHelper.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +/** + * This class exists to restrict the visibility of TaskContext setters. + */ +private [spark] object TaskContextHelper { + + def setTaskContext(tc: TaskContext): Unit = TaskContext.setTaskContext(tc) + + def unset(): Unit = TaskContext.unset() + +} diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala new file mode 100644 index 0000000000000..afd2b85d33a77 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.util.{TaskCompletionListener, TaskCompletionListenerException} + +import scala.collection.mutable.ArrayBuffer + +private[spark] class TaskContextImpl(val stageId: Int, + val partitionId: Int, + val attemptId: Long, + val runningLocally: Boolean = false, + val taskMetrics: TaskMetrics = TaskMetrics.empty) + extends TaskContext + with Logging { + + // List of callback functions to execute when the task completes. + @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] + + // Whether the corresponding task has been killed. + @volatile private var interrupted: Boolean = false + + // Whether the task has completed. + @volatile private var completed: Boolean = false + + override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { + onCompleteCallbacks += listener + this + } + + override def addTaskCompletionListener(f: TaskContext => Unit): this.type = { + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f(context) + } + this + } + + @deprecated("use addTaskCompletionListener", "1.1.0") + override def addOnCompleteCallback(f: () => Unit) { + onCompleteCallbacks += new TaskCompletionListener { + override def onTaskCompletion(context: TaskContext): Unit = f() + } + } + + /** Marks the task as completed and triggers the listeners. */ + private[spark] def markTaskCompleted(): Unit = { + completed = true + val errorMsgs = new ArrayBuffer[String](2) + // Process complete callbacks in the reverse order of registration + onCompleteCallbacks.reverse.foreach { listener => + try { + listener.onTaskCompletion(this) + } catch { + case e: Throwable => + errorMsgs += e.getMessage + logError("Error in TaskCompletionListener", e) + } + } + if (errorMsgs.nonEmpty) { + throw new TaskCompletionListenerException(errorMsgs) + } + } + + /** Marks the task for interruption, i.e. cancellation. */ + private[spark] def markInterrupted(): Unit = { + interrupted = true + } + + override def isCompleted: Boolean = completed + + override def isRunningLocally: Boolean = runningLocally + + override def isInterrupted: Boolean = interrupted +} + diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 6b63eb23e9ee1..8010dd90082f8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -196,7 +196,7 @@ class HadoopRDD[K, V]( val jobConf = getJobConf() val inputFormat = getInputFormat(jobConf) HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), - context.getStageId, theSplit.index, context.getAttemptId.toInt, jobConf) + context.stageId, theSplit.index, context.attemptId.toInt, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 0d97506450a7f..929ded58a3bd5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -956,9 +956,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt + val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outfmt.newInstance @@ -1027,9 +1027,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt + val attemptNumber = (context.attemptId % Int.MaxValue).toInt - writer.setup(context.getStageId, context.getPartitionId, attemptNumber) + writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() try { var count = 0 diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 788eb1ff4e455..f81fa6d8089fc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -633,14 +633,14 @@ class DAGScheduler( val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) val taskContext = - new TaskContext(job.finalStage.id, job.partitions(0), 0, true) - TaskContext.setTaskContext(taskContext) + new TaskContextImpl(job.finalStage.id, job.partitions(0), 0, true) + TaskContextHelper.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { taskContext.markTaskCompleted() - TaskContext.unset() + TaskContextHelper.unset() } } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index c6e47c84a0cb2..2552d03d18d06 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import org.apache.spark.TaskContext +import org.apache.spark.{TaskContextHelper, TaskContextImpl, TaskContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.ByteBufferInputStream @@ -45,8 +45,8 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { final def run(attemptId: Long): T = { - context = new TaskContext(stageId, partitionId, attemptId, false) - TaskContext.setTaskContext(context) + context = new TaskContextImpl(stageId, partitionId, attemptId, false) + TaskContextHelper.setTaskContext(context) context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() if (_killed) { @@ -56,7 +56,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex runTask(context) } finally { context.markTaskCompleted() - TaskContext.unset() + TaskContextHelper.unset() } } @@ -70,7 +70,7 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex var metrics: Option[TaskMetrics] = None // Task context, to be initialized in run(). - @transient protected var context: TaskContext = _ + @transient protected var context: TaskContextImpl = _ // The actual Thread on which the task is running, if any. Initialized in run(). @volatile @transient private var taskThread: Thread = _ diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 4a078435447e5..b8fa822ae4bd8 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -776,7 +776,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0L, false, new TaskMetrics()); + TaskContext context = new TaskContextImpl(0, 0, 0L, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java index 0944bf8cd5c71..e9ec700e32e15 100644 --- a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java +++ b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java @@ -30,8 +30,8 @@ public class JavaTaskCompletionListenerImpl implements TaskCompletionListener { public void onTaskCompletion(TaskContext context) { context.isCompleted(); context.isInterrupted(); - context.getStageId(); - context.getPartitionId(); + context.stageId(); + context.partitionId(); context.isRunningLocally(); context.addTaskCompletionListener(this); } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index d735010d7c9d5..c0735f448d193 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -66,7 +66,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar // in blockManager.put is a losing battle. You have been warned. blockManager = sc.env.blockManager cacheManager = sc.env.cacheManager - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) val computeValue = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) val getValue = blockManager.get(RDDBlockId(rdd.id, split.index)) assert(computeValue.toList === List(1, 2, 3, 4)) @@ -81,7 +81,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -94,7 +94,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, true) + val context = new TaskContextImpl(0, 0, 0, true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -102,7 +102,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("verify task metrics updated correctly") { cacheManager = sc.env.cacheManager - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) cacheManager.getOrCompute(rdd3, split, context, StorageLevel.MEMORY_ONLY) assert(context.taskMetrics.updatedBlocks.getOrElse(Seq()).size === 2) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index be972c5e97a7e..271a90c6646bb 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -174,7 +174,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) - val tContext = new TaskContext(0, 0, 0) + val tContext = new TaskContextImpl(0, 0, 0) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index faba5508c906c..561a5e9cd90c4 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -51,7 +51,7 @@ class TaskContextSuite extends FunSuite with BeforeAndAfter with LocalSparkConte } test("all TaskCompletionListeners should be called even if some fail") { - val context = new TaskContext(0, 0, 0) + val context = new TaskContextImpl(0, 0, 0) val listener = mock(classOf[TaskCompletionListener]) context.addTaskCompletionListener(_ => throw new Exception("blah")) context.addTaskCompletionListener(listener) diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 809bd70929656..a8c049d749015 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import org.apache.spark.TaskContext +import org.apache.spark.{TaskContextImpl, TaskContext} import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} import org.mockito.Mockito._ @@ -62,7 +62,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { ) val iterator = new ShuffleBlockFetcherIterator( - new TaskContext(0, 0, 0), + new TaskContextImpl(0, 0, 0), transfer, blockManager, blocksByAddress, @@ -120,7 +120,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { ) val iterator = new ShuffleBlockFetcherIterator( - new TaskContext(0, 0, 0), + new TaskContextImpl(0, 0, 0), transfer, blockManager, blocksByAddress, @@ -169,7 +169,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { (bmId, Seq((blId1, 1L), (blId2, 1L)))) val iterator = new ShuffleBlockFetcherIterator( - new TaskContext(0, 0, 0), + new TaskContextImpl(0, 0, 0), transfer, blockManager, blocksByAddress, diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 39f8ba4745737..d919b18e09855 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -32,7 +32,7 @@ object MimaBuild { ProblemFilters.exclude[MissingMethodProblem](fullName), // Sometimes excluded methods have default arguments and // they are translated into public methods/fields($default$) in generated - // bytecode. It is not possible to exhustively list everything. + // bytecode. It is not possible to exhaustively list everything. // But this should be okay. ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$2"), ProblemFilters.exclude[MissingMethodProblem](fullName+"$default$1"), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index d499302124461..350aad47735e4 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -50,7 +50,11 @@ object MimaExcludes { "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL2"), // MapStatus should be private[spark] ProblemFilters.exclude[IncompatibleTemplateDefProblem]( - "org.apache.spark.scheduler.MapStatus") + "org.apache.spark.scheduler.MapStatus"), + // TaskContext was promoted to Abstract class + ProblemFilters.exclude[AbstractClassProblem]( + "org.apache.spark.TaskContext") + ) case v if v.startsWith("1.1") => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 1f4237d7ede65..5c6fa78ae3895 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -289,9 +289,9 @@ case class InsertIntoParquetTable( def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt + val attemptNumber = (context.attemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = new AppendingParquetOutputFormat(taskIdOffset) From 7f7b50ed9d4ffdd6b23e0faa56b068a049da67f7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 16 Oct 2014 18:58:18 -0700 Subject: [PATCH 1003/1492] [SPARK-3923] Increase Akka heartbeat pause above heartbeat interval Something about the 2.3.4 upgrade seems to have made the issue manifest where all the services disconnect from each other after exactly 1000 seconds (which is the heartbeat interval). [This post](https://groups.google.com/forum/#!topic/akka-user/X3xzpTCbEFs) suggests that heartbeat pause should be greater than heartbeat interval, and increasing the pause from 600s to 6000s seems to have rectified the issue. My current cluster has now exceeded 1400s of uptime without failure! I do not know why this fixed it, because the threshold we have set for the failure detector is the exponent of a timeout, and 300 is extremely large. Perhaps the default failure detector changed in 2.3.4 and now ignores threshold. Author: Aaron Davidson Closes #2784 from aarondav/fix-timeout and squashes the following commits: bd1151a [Aaron Davidson] Increase pause, don't decrease interval 9cb0372 [Aaron Davidson] [SPARK-3923] Decrease Akka heartbeat interval below heartbeat pause --- core/src/main/scala/org/apache/spark/util/AkkaUtils.scala | 2 +- docs/configuration.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index e2d32c859bbda..f41c8d0315cb3 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -77,7 +77,7 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 600) + val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) val akkaFailureDetector = conf.getDouble("spark.akka.failure-detector.threshold", 300.0) val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) diff --git a/docs/configuration.md b/docs/configuration.md index f311f0d2a6206..8515ee045177f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -725,7 +725,7 @@ Apart from these, the following properties are also available, and may be useful

    - + - - - + + + - + } } From 2f54543815c0905dc958d444ad638c23a29507c6 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:44:29 -0700 Subject: [PATCH 1147/1492] [SPARK-3661] Respect spark.*.memory in cluster mode This also includes minor re-organization of the code. Tested locally in both client and deploy modes. Author: Andrew Or Author: Andrew Or Closes #2697 from andrewor14/memory-cluster-mode and squashes the following commits: 01d78bc [Andrew Or] Merge branch 'master' of github.com:apache/spark into memory-cluster-mode ccd468b [Andrew Or] Add some comments per Patrick c956577 [Andrew Or] Tweak wording 2b4afa0 [Andrew Or] Unused import 47a5a88 [Andrew Or] Correct Spark properties precedence order bf64717 [Andrew Or] Merge branch 'master' of github.com:apache/spark into memory-cluster-mode dd452d0 [Andrew Or] Respect spark.*.memory in cluster mode --- .../org/apache/spark/deploy/SparkSubmit.scala | 8 +- .../spark/deploy/SparkSubmitArguments.scala | 74 +++++++++++-------- 2 files changed, 45 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 0379adeb07b83..b43e68e40f791 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -274,17 +274,11 @@ object SparkSubmit { } } - // Properties given with --conf are superceded by other options, but take precedence over - // properties in the defaults file. + // Load any properties specified through --conf and the default properties file for ((k, v) <- args.sparkProperties) { sysProps.getOrElseUpdate(k, v) } - // Read from default spark properties, if any - for ((k, v) <- args.defaultSparkProperties) { - sysProps.getOrElseUpdate(k, v) - } - // Resolve paths in certain spark properties val pathConfigs = Seq( "spark.jars", diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 72a452e0aefb5..f0e9ee67f6a67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy import java.util.jar.JarFile -import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.util.Utils @@ -72,39 +71,54 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St defaultProperties } - // Respect SPARK_*_MEMORY for cluster mode - driverMemory = sys.env.get("SPARK_DRIVER_MEMORY").orNull - executorMemory = sys.env.get("SPARK_EXECUTOR_MEMORY").orNull - + // Set parameters from command line arguments parseOpts(args.toList) - mergeSparkProperties() + // Populate `sparkProperties` map from properties file + mergeDefaultSparkProperties() + // Use `sparkProperties` map along with env vars to fill in any missing parameters + loadEnvironmentArguments() + checkRequiredArguments() /** - * Fill in any undefined values based on the default properties file or options passed in through - * the '--conf' flag. + * Merge values from the default properties file with those specified through --conf. + * When this is called, `sparkProperties` is already filled with configs from the latter. */ - private def mergeSparkProperties(): Unit = { + private def mergeDefaultSparkProperties(): Unit = { // Use common defaults file, if not specified by user propertiesFile = Option(propertiesFile).getOrElse(Utils.getDefaultPropertiesFile(env)) + // Honor --conf before the defaults file + defaultSparkProperties.foreach { case (k, v) => + if (!sparkProperties.contains(k)) { + sparkProperties(k) = v + } + } + } - val properties = HashMap[String, String]() - properties.putAll(defaultSparkProperties) - properties.putAll(sparkProperties) - - // Use properties file as fallback for values which have a direct analog to - // arguments in this script. - master = Option(master).orElse(properties.get("spark.master")).orNull - executorMemory = Option(executorMemory).orElse(properties.get("spark.executor.memory")).orNull - executorCores = Option(executorCores).orElse(properties.get("spark.executor.cores")).orNull + /** + * Load arguments from environment variables, Spark properties etc. + */ + private def loadEnvironmentArguments(): Unit = { + master = Option(master) + .orElse(sparkProperties.get("spark.master")) + .orElse(env.get("MASTER")) + .orNull + driverMemory = Option(driverMemory) + .orElse(sparkProperties.get("spark.driver.memory")) + .orElse(env.get("SPARK_DRIVER_MEMORY")) + .orNull + executorMemory = Option(executorMemory) + .orElse(sparkProperties.get("spark.executor.memory")) + .orElse(env.get("SPARK_EXECUTOR_MEMORY")) + .orNull + executorCores = Option(executorCores) + .orElse(sparkProperties.get("spark.executor.cores")) + .orNull totalExecutorCores = Option(totalExecutorCores) - .orElse(properties.get("spark.cores.max")) + .orElse(sparkProperties.get("spark.cores.max")) .orNull - name = Option(name).orElse(properties.get("spark.app.name")).orNull - jars = Option(jars).orElse(properties.get("spark.jars")).orNull - - // This supports env vars in older versions of Spark - master = Option(master).orElse(env.get("MASTER")).orNull + name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull + jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull // Try to set main class from JAR if no --class argument is given @@ -131,7 +145,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } /** Ensure that required fields exists. Call this only once all defaults are loaded. */ - private def checkRequiredArguments() = { + private def checkRequiredArguments(): Unit = { if (args.length == 0) { printUsageAndExit(-1) } @@ -166,7 +180,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } } - override def toString = { + override def toString = { s"""Parsed arguments: | master $master | deployMode $deployMode @@ -174,7 +188,6 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | executorCores $executorCores | totalExecutorCores $totalExecutorCores | propertiesFile $propertiesFile - | extraSparkProperties $sparkProperties | driverMemory $driverMemory | driverCores $driverCores | driverExtraClassPath $driverExtraClassPath @@ -193,8 +206,9 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St | jars $jars | verbose $verbose | - |Default properties from $propertiesFile: - |${defaultSparkProperties.mkString(" ", "\n ", "\n")} + |Spark properties used, including those specified through + | --conf and those from the properties file $propertiesFile: + |${sparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } @@ -327,7 +341,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, St } } - private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + private def printUsageAndExit(exitCode: Int, unknownParam: Any = null): Unit = { val outStream = SparkSubmit.printStream if (unknownParam != null) { outStream.println("Unknown/unsupported param " + unknownParam) From 68cb69daf3022e973422e496ccf827ca3806ff30 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 30 Oct 2014 15:54:53 -0700 Subject: [PATCH 1148/1492] SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop (This is just a look at what completely moving the classes would look like. I know Patrick flagged that as maybe not OK, although, it's private?) Author: Sean Owen Closes #2814 from srowen/SPARK-1209 and squashes the following commits: ead1115 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though? 2d42c1d [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark --- .../main/scala/org/apache/spark/SparkHadoopWriter.scala | 1 + .../{hadoop => spark}/mapred/SparkHadoopMapRedUtil.scala | 6 ++++-- .../mapreduce/SparkHadoopMapReduceUtil.scala | 5 +++-- .../main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 + .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 ++- project/MimaExcludes.scala | 8 ++++++++ .../apache/spark/sql/parquet/ParquetTableOperations.scala | 1 + .../org/apache/spark/sql/hive/hiveWriterContainers.scala | 1 + 8 files changed, 21 insertions(+), 5 deletions(-) rename core/src/main/scala/org/apache/{hadoop => spark}/mapred/SparkHadoopMapRedUtil.scala (93%) rename core/src/main/scala/org/apache/{hadoop => spark}/mapreduce/SparkHadoopMapReduceUtil.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 376e69cd997d5..40237596570de 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD /** diff --git a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala similarity index 93% rename from core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala rename to core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala index 0c47afae54c8b..fe2bc65f89ab4 100644 --- a/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala @@ -15,9 +15,11 @@ * limitations under the License. */ -package org.apache.hadoop.mapred +package org.apache.spark.mapred -private[apache] +import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} + +private[spark] trait SparkHadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = { val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", diff --git a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala similarity index 96% rename from core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala rename to core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala index 1fca5729c6092..3340673f91156 100644 --- a/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala @@ -15,13 +15,14 @@ * limitations under the License. */ -package org.apache.hadoop.mapreduce +package org.apache.spark.mapreduce import java.lang.{Boolean => JBoolean, Integer => JInteger} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} -private[apache] +private[spark] trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 324563248793c..e7b11707551df 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil 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 da89f634abaea..462f0d6268a86 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -33,13 +33,14 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, -RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} +RecordWriter => NewRecordWriter} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 6a0495f8fd540..a94d09be3bec6 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -77,6 +77,14 @@ object MimaExcludes { // SPARK-3822 ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") + ) ++ Seq( + // SPARK-1209 + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.hadoop.mapred.SparkHadoopMapRedUtil"), + ProblemFilters.exclude[MissingTypesProblem]( + "org.apache.spark.rdd.PairRDDFunctions") ) case v if v.startsWith("1.1") => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 5c6fa78ae3895..ef995b3d1a6b4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -42,6 +42,7 @@ import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType +import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index bf2ce9df67c58..cc8bb3e172c6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -31,6 +31,7 @@ import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ +import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc} From 9b6ebe33db27be38c3036ffeda17096043fb0fb9 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Thu, 30 Oct 2014 17:15:45 -0700 Subject: [PATCH 1149/1492] [SPARK-4120][SQL] Join of multiple tables with syntax like SELECT .. FROM T1,T2,T3.. does not work in SparkSQL Right now it works for only 2 tables like below query. sql("SELECT * FROM records1 as a,records2 as b where a.key=b.key ") But it does not work for more than 2 tables like below query sql("SELECT * FROM records1 as a,records2 as b,records3 as c where a.key=b.key and a.key=c.key"). Author: ravipesala Closes #2987 from ravipesala/multijoin and squashes the following commits: 429b005 [ravipesala] Support multiple joins --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 3 ++- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 10 ++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 0acf7252ba3f0..942b843104958 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -166,7 +166,8 @@ class SqlParser extends AbstractSparkSQLParser { // Based very loosely on the MySQL Grammar. // http://dev.mysql.com/doc/refman/5.0/en/join.html protected lazy val relations: Parser[LogicalPlan] = - ( relation ~ ("," ~> relation) ^^ { case r1 ~ r2 => Join(r1, r2, Inner, None) } + ( relation ~ rep1("," ~> relation) ^^ { + case r1 ~ joins => joins.foldLeft(r1) { case(lhs, r) => Join(lhs, r, Inner, None) } } | relation ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 1034c2d05f8cf..4c36ca0f41174 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -899,4 +899,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("SPARK-3814 Support Bitwise ~ operator") { checkAnswer(sql("SELECT ~key FROM testData WHERE key = 1 "), -2) } + + test("SPARK-4120 Join of multiple tables does not work in SparkSQL") { + checkAnswer( + sql( + """SELECT a.key, b.key, c.key + |FROM testData a,testData b,testData c + |where a.key = b.key and a.key = c.key + """.stripMargin), + (1 to 100).map(i => Seq(i, i, i))) + } } From 2e35e24294ad8a5e76c89ea888fe330052dabd5a Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Thu, 30 Oct 2014 17:17:24 -0700 Subject: [PATCH 1150/1492] [SPARK-3968][SQL] Use parquet-mr filter2 api The parquet-mr project has introduced a new filter api (https://github.com/apache/incubator-parquet-mr/pull/4), along with several fixes . It can also eliminate entire RowGroups depending on certain statistics like min/max We can leverage that to further improve performance of queries with filters. Also filter2 api introduces ability to create custom filters. We can create a custom filter for the optimized In clause (InSet) , so that elimination happens in the ParquetRecordReader itself Author: Yash Datta Closes #2841 from saucam/master and squashes the following commits: 8282ba0 [Yash Datta] SPARK-3968: fix scala code style and add some more tests for filtering on optional columns 515df1c [Yash Datta] SPARK-3968: Add a test case for filter pushdown on optional column 5f4530e [Yash Datta] SPARK-3968: Fix scala code style f304667 [Yash Datta] SPARK-3968: Using task metadata strategy for row group filtering ec53e92 [Yash Datta] SPARK-3968: No push down should result in case we are unable to create a record filter 48163c3 [Yash Datta] SPARK-3968: Code cleanup cc7b596 [Yash Datta] SPARK-3968: 1. Fix RowGroupFiltering not working 2. Use the serialization/deserialization from Parquet library for filter pushdown caed851 [Yash Datta] Revert "SPARK-3968: Not pushing the filters in case of OPTIONAL columns" since filtering on optional columns is now supported in filter2 api 49703c9 [Yash Datta] SPARK-3968: Not pushing the filters in case of OPTIONAL columns 9d09741 [Yash Datta] SPARK-3968: Change parquet filter pushdown to use filter2 api of parquet-mr --- pom.xml | 2 +- .../spark/sql/parquet/ParquetFilters.scala | 230 +++++++----------- .../sql/parquet/ParquetTableOperations.scala | 179 +++++++++++--- .../spark/sql/parquet/ParquetTestData.scala | 19 ++ .../spark/sql/parquet/ParquetQuerySuite.scala | 57 +++++ 5 files changed, 308 insertions(+), 179 deletions(-) diff --git a/pom.xml b/pom.xml index e4c92470fc03e..379274d0b16e3 100644 --- a/pom.xml +++ b/pom.xml @@ -133,7 +133,7 @@ 0.13.1 10.10.1.1 - 1.4.3 + 1.6.0rc3 1.2.3 8.1.14.v20131031 0.3.6 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 7c83f1cad7d71..517a5cf0029ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -21,8 +21,12 @@ import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration -import parquet.filter._ -import parquet.filter.ColumnPredicates._ +import parquet.filter2.compat.FilterCompat +import parquet.filter2.compat.FilterCompat._ +import parquet.filter2.predicate.FilterPredicate +import parquet.filter2.predicate.FilterApi +import parquet.filter2.predicate.FilterApi._ +import parquet.io.api.Binary import parquet.column.ColumnReader import com.google.common.io.BaseEncoding @@ -38,67 +42,74 @@ private[sql] object ParquetFilters { // set this to false if pushdown should be disabled val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.hints.parquetFilterPushdown" - def createRecordFilter(filterExpressions: Seq[Expression]): UnboundRecordFilter = { + def createRecordFilter(filterExpressions: Seq[Expression]): Filter = { val filters: Seq[CatalystFilter] = filterExpressions.collect { case (expression: Expression) if createFilter(expression).isDefined => createFilter(expression).get } - if (filters.length > 0) filters.reduce(AndRecordFilter.and) else null + if (filters.length > 0) FilterCompat.get(filters.reduce(FilterApi.and)) else null } - def createFilter(expression: Expression): Option[CatalystFilter] = { + def createFilter(expression: Expression): Option[CatalystFilter] ={ def createEqualityFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case BooleanType => - ComparisonFilter.createBooleanFilter(name, literal.value.asInstanceOf[Boolean], predicate) + ComparisonFilter.createBooleanFilter( + name, + literal.value.asInstanceOf[Boolean], + predicate) case IntegerType => - ComparisonFilter.createIntFilter( + new ComparisonFilter( name, - (x: Int) => x == literal.value.asInstanceOf[Int], + FilterApi.eq(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x == literal.value.asInstanceOf[Long], + FilterApi.eq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x == literal.value.asInstanceOf[Double], + FilterApi.eq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x == literal.value.asInstanceOf[Float], + FilterApi.eq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) case StringType => - ComparisonFilter.createStringFilter(name, literal.value.asInstanceOf[String], predicate) + ComparisonFilter.createStringFilter( + name, + literal.value.asInstanceOf[String], + predicate) } + def createLessThanFilter( name: String, literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( - name, - (x: Int) => x < literal.value.asInstanceOf[Int], + new ComparisonFilter( + name, + FilterApi.lt(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x < literal.value.asInstanceOf[Long], + FilterApi.lt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x < literal.value.asInstanceOf[Double], + FilterApi.lt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x < literal.value.asInstanceOf[Float], + FilterApi.lt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } def createLessThanOrEqualFilter( @@ -106,24 +117,24 @@ private[sql] object ParquetFilters { literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( + new ComparisonFilter( name, - (x: Int) => x <= literal.value.asInstanceOf[Int], + FilterApi.ltEq(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x <= literal.value.asInstanceOf[Long], + FilterApi.ltEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x <= literal.value.asInstanceOf[Double], + FilterApi.ltEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x <= literal.value.asInstanceOf[Float], + FilterApi.ltEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } // TODO: combine these two types somehow? @@ -132,24 +143,24 @@ private[sql] object ParquetFilters { literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( + new ComparisonFilter( name, - (x: Int) => x > literal.value.asInstanceOf[Int], + FilterApi.gt(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x > literal.value.asInstanceOf[Long], + FilterApi.gt(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x > literal.value.asInstanceOf[Double], + FilterApi.gt(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x > literal.value.asInstanceOf[Float], + FilterApi.gt(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } def createGreaterThanOrEqualFilter( @@ -157,23 +168,24 @@ private[sql] object ParquetFilters { literal: Literal, predicate: CatalystPredicate) = literal.dataType match { case IntegerType => - ComparisonFilter.createIntFilter( - name, (x: Int) => x >= literal.value.asInstanceOf[Int], + new ComparisonFilter( + name, + FilterApi.gtEq(intColumn(name), literal.value.asInstanceOf[Integer]), predicate) case LongType => - ComparisonFilter.createLongFilter( + new ComparisonFilter( name, - (x: Long) => x >= literal.value.asInstanceOf[Long], + FilterApi.gtEq(longColumn(name), literal.value.asInstanceOf[java.lang.Long]), predicate) case DoubleType => - ComparisonFilter.createDoubleFilter( + new ComparisonFilter( name, - (x: Double) => x >= literal.value.asInstanceOf[Double], + FilterApi.gtEq(doubleColumn(name), literal.value.asInstanceOf[java.lang.Double]), predicate) case FloatType => - ComparisonFilter.createFloatFilter( + new ComparisonFilter( name, - (x: Float) => x >= literal.value.asInstanceOf[Float], + FilterApi.gtEq(floatColumn(name), literal.value.asInstanceOf[java.lang.Float]), predicate) } @@ -209,25 +221,25 @@ private[sql] object ParquetFilters { case _ => None } } - case p @ EqualTo(left: Literal, right: NamedExpression) if !right.nullable => + case p @ EqualTo(left: Literal, right: NamedExpression) => Some(createEqualityFilter(right.name, left, p)) - case p @ EqualTo(left: NamedExpression, right: Literal) if !left.nullable => + case p @ EqualTo(left: NamedExpression, right: Literal) => Some(createEqualityFilter(left.name, right, p)) - case p @ LessThan(left: Literal, right: NamedExpression) if !right.nullable => + case p @ LessThan(left: Literal, right: NamedExpression) => Some(createLessThanFilter(right.name, left, p)) - case p @ LessThan(left: NamedExpression, right: Literal) if !left.nullable => + case p @ LessThan(left: NamedExpression, right: Literal) => Some(createLessThanFilter(left.name, right, p)) - case p @ LessThanOrEqual(left: Literal, right: NamedExpression) if !right.nullable => + case p @ LessThanOrEqual(left: Literal, right: NamedExpression) => Some(createLessThanOrEqualFilter(right.name, left, p)) - case p @ LessThanOrEqual(left: NamedExpression, right: Literal) if !left.nullable => + case p @ LessThanOrEqual(left: NamedExpression, right: Literal) => Some(createLessThanOrEqualFilter(left.name, right, p)) - case p @ GreaterThan(left: Literal, right: NamedExpression) if !right.nullable => + case p @ GreaterThan(left: Literal, right: NamedExpression) => Some(createGreaterThanFilter(right.name, left, p)) - case p @ GreaterThan(left: NamedExpression, right: Literal) if !left.nullable => + case p @ GreaterThan(left: NamedExpression, right: Literal) => Some(createGreaterThanFilter(left.name, right, p)) - case p @ GreaterThanOrEqual(left: Literal, right: NamedExpression) if !right.nullable => + case p @ GreaterThanOrEqual(left: Literal, right: NamedExpression) => Some(createGreaterThanOrEqualFilter(right.name, left, p)) - case p @ GreaterThanOrEqual(left: NamedExpression, right: Literal) if !left.nullable => + case p @ GreaterThanOrEqual(left: NamedExpression, right: Literal) => Some(createGreaterThanOrEqualFilter(left.name, right, p)) case _ => None } @@ -300,52 +312,54 @@ private[sql] object ParquetFilters { } abstract private[parquet] class CatalystFilter( - @transient val predicate: CatalystPredicate) extends UnboundRecordFilter + @transient val predicate: CatalystPredicate) extends FilterPredicate private[parquet] case class ComparisonFilter( val columnName: String, - private var filter: UnboundRecordFilter, + private var filter: FilterPredicate, @transient override val predicate: CatalystPredicate) extends CatalystFilter(predicate) { - override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { - filter.bind(readers) + override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { + filter.accept(visitor) } } private[parquet] case class OrFilter( - private var filter: UnboundRecordFilter, + private var filter: FilterPredicate, @transient val left: CatalystFilter, @transient val right: CatalystFilter, @transient override val predicate: Or) extends CatalystFilter(predicate) { def this(l: CatalystFilter, r: CatalystFilter) = this( - OrRecordFilter.or(l, r), + FilterApi.or(l, r), l, r, Or(l.predicate, r.predicate)) - override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { - filter.bind(readers) + override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { + filter.accept(visitor); } + } private[parquet] case class AndFilter( - private var filter: UnboundRecordFilter, + private var filter: FilterPredicate, @transient val left: CatalystFilter, @transient val right: CatalystFilter, @transient override val predicate: And) extends CatalystFilter(predicate) { def this(l: CatalystFilter, r: CatalystFilter) = this( - AndRecordFilter.and(l, r), + FilterApi.and(l, r), l, r, And(l.predicate, r.predicate)) - override def bind(readers: java.lang.Iterable[ColumnReader]): RecordFilter = { - filter.bind(readers) + override def accept[R](visitor: FilterPredicate.Visitor[R]): R = { + filter.accept(visitor); } + } private[parquet] object ComparisonFilter { @@ -355,13 +369,7 @@ private[parquet] object ComparisonFilter { predicate: CatalystPredicate): CatalystFilter = new ComparisonFilter( columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToBoolean( - new BooleanPredicateFunction { - def functionToApply(input: Boolean): Boolean = input == value - } - )), + FilterApi.eq(booleanColumn(columnName), value.asInstanceOf[java.lang.Boolean]), predicate) def createStringFilter( @@ -370,72 +378,6 @@ private[parquet] object ComparisonFilter { predicate: CatalystPredicate): CatalystFilter = new ComparisonFilter( columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToString ( - new ColumnPredicates.PredicateFunction[String] { - def functionToApply(input: String): Boolean = input == value - } - )), - predicate) - - def createIntFilter( - columnName: String, - func: Int => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToInteger( - new IntegerPredicateFunction { - def functionToApply(input: Int) = func(input) - } - )), - predicate) - - def createLongFilter( - columnName: String, - func: Long => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToLong( - new LongPredicateFunction { - def functionToApply(input: Long) = func(input) - } - )), - predicate) - - def createDoubleFilter( - columnName: String, - func: Double => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToDouble( - new DoublePredicateFunction { - def functionToApply(input: Double) = func(input) - } - )), - predicate) - - def createFloatFilter( - columnName: String, - func: Float => Boolean, - predicate: CatalystPredicate): CatalystFilter = - new ComparisonFilter( - columnName, - ColumnRecordFilter.column( - columnName, - ColumnPredicates.applyFunctionToFloat( - new FloatPredicateFunction { - def functionToApply(input: Float) = func(input) - } - )), + FilterApi.eq(binaryColumn(columnName), Binary.fromString(value)), predicate) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index ef995b3d1a6b4..416bf5614437f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -38,6 +38,7 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import parquet.hadoop._ import parquet.hadoop.api.{InitContext, ReadSupport} import parquet.hadoop.metadata.GlobalMetaData +import parquet.hadoop.api.ReadSupport.ReadContext import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType @@ -77,6 +78,10 @@ case class ParquetTableScan( s"$normalOutput + $partOutput != $attributes, ${relation.output}") override def execute(): RDD[Row] = { + import parquet.filter2.compat.FilterCompat.FilterPredicateCompat + import parquet.filter2.compat.FilterCompat.Filter + import parquet.filter2.predicate.FilterPredicate + val sc = sqlContext.sparkContext val job = new Job(sc.hadoopConfiguration) ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) @@ -107,7 +112,13 @@ case class ParquetTableScan( // "spark.sql.hints.parquetFilterPushdown" to false inside SparkConf. if (columnPruningPred.length > 0 && sc.conf.getBoolean(ParquetFilters.PARQUET_FILTER_PUSHDOWN_ENABLED, true)) { - ParquetFilters.serializeFilterExpressions(columnPruningPred, conf) + + // Set this in configuration of ParquetInputFormat, needed for RowGroupFiltering + val filter: Filter = ParquetFilters.createRecordFilter(columnPruningPred) + if (filter != null){ + val filterPredicate = filter.asInstanceOf[FilterPredicateCompat].getFilterPredicate() + ParquetInputFormat.setFilterPredicate(conf, filterPredicate) + } } // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata @@ -363,15 +374,17 @@ private[parquet] class FilteringParquetRowInputFormat override def createRecordReader( inputSplit: InputSplit, taskAttemptContext: TaskAttemptContext): RecordReader[Void, Row] = { + + import parquet.filter2.compat.FilterCompat.NoOpFilter + import parquet.filter2.compat.FilterCompat.Filter + val readSupport: ReadSupport[Row] = new RowReadSupport() - val filterExpressions = - ParquetFilters.deserializeFilterExpressions(ContextUtil.getConfiguration(taskAttemptContext)) - if (filterExpressions.length > 0) { - logInfo(s"Pushing down predicates for RecordFilter: ${filterExpressions.mkString(", ")}") + val filter = ParquetInputFormat.getFilter(ContextUtil.getConfiguration(taskAttemptContext)) + if (!filter.isInstanceOf[NoOpFilter]) { new ParquetRecordReader[Row]( readSupport, - ParquetFilters.createRecordFilter(filterExpressions)) + filter) } else { new ParquetRecordReader[Row](readSupport) } @@ -424,10 +437,8 @@ private[parquet] class FilteringParquetRowInputFormat configuration: Configuration, footers: JList[Footer]): JList[ParquetInputSplit] = { - import FilteringParquetRowInputFormat.blockLocationCache - - val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) - + // Use task side strategy by default + val taskSideMetaData = configuration.getBoolean(ParquetInputFormat.TASK_SIDE_METADATA, true) val maxSplitSize: JLong = configuration.getLong("mapred.max.split.size", Long.MaxValue) val minSplitSize: JLong = Math.max(getFormatMinSplitSize(), configuration.getLong("mapred.min.split.size", 0L)) @@ -436,23 +447,67 @@ private[parquet] class FilteringParquetRowInputFormat s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" + s" minSplitSize = $minSplitSize") } - val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + + // Uses strict type checking by default val getGlobalMetaData = classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]]) getGlobalMetaData.setAccessible(true) val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData] - // if parquet file is empty, return empty splits. - if (globalMetaData == null) { - return splits - } + if (globalMetaData == null) { + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + return splits + } + val readContext = getReadSupport(configuration).init( new InitContext(configuration, globalMetaData.getKeyValueMetaData(), globalMetaData.getSchema())) + + if (taskSideMetaData){ + logInfo("Using Task Side Metadata Split Strategy") + return getTaskSideSplits(configuration, + footers, + maxSplitSize, + minSplitSize, + readContext) + } else { + logInfo("Using Client Side Metadata Split Strategy") + return getClientSideSplits(configuration, + footers, + maxSplitSize, + minSplitSize, + readContext) + } + + } + + def getClientSideSplits( + configuration: Configuration, + footers: JList[Footer], + maxSplitSize: JLong, + minSplitSize: JLong, + readContext: ReadContext): JList[ParquetInputSplit] = { + + import FilteringParquetRowInputFormat.blockLocationCache + import parquet.filter2.compat.FilterCompat; + import parquet.filter2.compat.FilterCompat.Filter; + import parquet.filter2.compat.RowGroupFilter; + + val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + val filter: Filter = ParquetInputFormat.getFilter(configuration) + var rowGroupsDropped: Long = 0 + var totalRowGroups: Long = 0 + + // Ugly hack, stuck with it until PR: + // https://github.com/apache/incubator-parquet-mr/pull/17 + // is resolved val generateSplits = - classOf[ParquetInputFormat[_]].getDeclaredMethods.find(_.getName == "generateSplits").get + Class.forName("parquet.hadoop.ClientSideMetadataSplitStrategy") + .getDeclaredMethods.find(_.getName == "generateSplits").getOrElse( + sys.error(s"Failed to reflectively invoke ClientSideMetadataSplitStrategy.generateSplits")) generateSplits.setAccessible(true) for (footer <- footers) { @@ -461,29 +516,85 @@ private[parquet] class FilteringParquetRowInputFormat val status = fileStatuses.getOrElse(file, fs.getFileStatus(file)) val parquetMetaData = footer.getParquetMetadata val blocks = parquetMetaData.getBlocks - var blockLocations: Array[BlockLocation] = null - if (!cacheMetadata) { - blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) - } else { - blockLocations = blockLocationCache.get(status, new Callable[Array[BlockLocation]] { - def call(): Array[BlockLocation] = fs.getFileBlockLocations(status, 0, status.getLen) - }) - } + totalRowGroups = totalRowGroups + blocks.size + val filteredBlocks = RowGroupFilter.filterRowGroups( + filter, + blocks, + parquetMetaData.getFileMetaData.getSchema) + rowGroupsDropped = rowGroupsDropped + (blocks.size - filteredBlocks.size) + + if (!filteredBlocks.isEmpty){ + var blockLocations: Array[BlockLocation] = null + if (!cacheMetadata) { + blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) + } else { + blockLocations = blockLocationCache.get(status, new Callable[Array[BlockLocation]] { + def call(): Array[BlockLocation] = fs.getFileBlockLocations(status, 0, status.getLen) + }) + } + splits.addAll( + generateSplits.invoke( + null, + filteredBlocks, + blockLocations, + status, + readContext.getRequestedSchema.toString, + readContext.getReadSupportMetadata, + minSplitSize, + maxSplitSize).asInstanceOf[JList[ParquetInputSplit]]) + } + } + + if (rowGroupsDropped > 0 && totalRowGroups > 0){ + val percentDropped = ((rowGroupsDropped/totalRowGroups.toDouble) * 100).toInt + logInfo(s"Dropping $rowGroupsDropped row groups that do not pass filter predicate " + + s"($percentDropped %) !") + } + else { + logInfo("There were no row groups that could be dropped due to filter predicates") + } + splits + + } + + def getTaskSideSplits( + configuration: Configuration, + footers: JList[Footer], + maxSplitSize: JLong, + minSplitSize: JLong, + readContext: ReadContext): JList[ParquetInputSplit] = { + + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] + + // Ugly hack, stuck with it until PR: + // https://github.com/apache/incubator-parquet-mr/pull/17 + // is resolved + val generateSplits = + Class.forName("parquet.hadoop.TaskSideMetadataSplitStrategy") + .getDeclaredMethods.find(_.getName == "generateTaskSideMDSplits").getOrElse( + sys.error( + s"Failed to reflectively invoke TaskSideMetadataSplitStrategy.generateTaskSideMDSplits")) + generateSplits.setAccessible(true) + + for (footer <- footers) { + val file = footer.getFile + val fs = file.getFileSystem(configuration) + val status = fileStatuses.getOrElse(file, fs.getFileStatus(file)) + val blockLocations = fs.getFileBlockLocations(status, 0, status.getLen) splits.addAll( generateSplits.invoke( - null, - blocks, - blockLocations, - status, - parquetMetaData.getFileMetaData, - readContext.getRequestedSchema.toString, - readContext.getReadSupportMetadata, - minSplitSize, - maxSplitSize).asInstanceOf[JList[ParquetInputSplit]]) + null, + blockLocations, + status, + readContext.getRequestedSchema.toString, + readContext.getReadSupportMetadata, + minSplitSize, + maxSplitSize).asInstanceOf[JList[ParquetInputSplit]]) } splits - } + } + } private[parquet] object FilteringParquetRowInputFormat { 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 837ea7695dbb3..c0918a40d136f 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 @@ -92,6 +92,12 @@ private[sql] object ParquetTestData { required int64 mylong; required float myfloat; required double mydouble; + optional boolean myoptboolean; + optional int32 myoptint; + optional binary myoptstring (UTF8); + optional int64 myoptlong; + optional float myoptfloat; + optional double myoptdouble; } """ @@ -255,6 +261,19 @@ private[sql] object ParquetTestData { record.add(3, i.toLong) record.add(4, i.toFloat + 0.5f) record.add(5, i.toDouble + 0.5d) + if (i % 2 == 0) { + if (i % 3 == 0) { + record.add(6, true) + } else { + record.add(6, false) + } + record.add(7, i) + record.add(8, i.toString) + record.add(9, i.toLong) + record.add(10, i.toFloat + 0.5f) + record.add(11, i.toDouble + 0.5d) + } + writer.write(record) } writer.close() 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 25e41ecf28e2e..9979ab446d8b5 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 @@ -560,6 +560,63 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(stringResult.size === 1) assert(stringResult(0).getString(2) == "100", "stringvalue incorrect") assert(stringResult(0).getInt(1) === 100) + + val query7 = sql(s"SELECT * FROM testfiltersource WHERE myoptint < 40") + assert( + query7.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val optResult = query7.collect() + assert(optResult.size === 20) + for(i <- 0 until 20) { + if (optResult(i)(7) != i * 2) { + fail(s"optional Int value in result row $i should be ${2*4*i}") + } + } + for(myval <- Seq("myoptint", "myoptlong", "myoptdouble", "myoptfloat")) { + val query8 = sql(s"SELECT * FROM testfiltersource WHERE $myval < 150 AND $myval >= 100") + assert( + query8.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result8 = query8.collect() + assert(result8.size === 25) + assert(result8(0)(7) === 100) + assert(result8(24)(7) === 148) + val query9 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 150 AND $myval <= 200") + assert( + query9.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result9 = query9.collect() + assert(result9.size === 25) + if (myval == "myoptint" || myval == "myoptlong") { + assert(result9(0)(7) === 152) + assert(result9(24)(7) === 200) + } else { + assert(result9(0)(7) === 150) + assert(result9(24)(7) === 198) + } + } + val query10 = sql("SELECT * FROM testfiltersource WHERE myoptstring = \"100\"") + assert( + query10.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result10 = query10.collect() + assert(result10.size === 1) + assert(result10(0).getString(8) == "100", "stringvalue incorrect") + assert(result10(0).getInt(7) === 100) + val query11 = sql(s"SELECT * FROM testfiltersource WHERE myoptboolean = true AND myoptint < 40") + assert( + query11.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], + "Top operator should be ParquetTableScan after pushdown") + val result11 = query11.collect() + assert(result11.size === 7) + for(i <- 0 until 6) { + if (!result11(i).getBoolean(6)) { + fail(s"optional Boolean value in result row $i not true") + } + if (result11(i).getInt(7) != i * 6) { + fail(s"optional Int value in result row $i should be ${6*i}") + } + } } test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { From 26d31d15fda3f63707a28d1a1115770ad127cf8f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 17:56:10 -0700 Subject: [PATCH 1151/1492] Revert "SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop" This reverts commit 68cb69daf3022e973422e496ccf827ca3806ff30. --- .../{spark => hadoop}/mapred/SparkHadoopMapRedUtil.scala | 6 ++---- .../mapreduce/SparkHadoopMapReduceUtil.scala | 5 ++--- .../main/scala/org/apache/spark/SparkHadoopWriter.scala | 1 - .../main/scala/org/apache/spark/rdd/NewHadoopRDD.scala | 1 - .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 +-- project/MimaExcludes.scala | 8 -------- .../apache/spark/sql/parquet/ParquetTableOperations.scala | 1 - .../org/apache/spark/sql/hive/hiveWriterContainers.scala | 1 - 8 files changed, 5 insertions(+), 21 deletions(-) rename core/src/main/scala/org/apache/{spark => hadoop}/mapred/SparkHadoopMapRedUtil.scala (93%) rename core/src/main/scala/org/apache/{spark => hadoop}/mapreduce/SparkHadoopMapReduceUtil.scala (96%) diff --git a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala similarity index 93% rename from core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala index fe2bc65f89ab4..0c47afae54c8b 100644 --- a/core/src/main/scala/org/apache/spark/mapred/SparkHadoopMapRedUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapred/SparkHadoopMapRedUtil.scala @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.mapred +package org.apache.hadoop.mapred -import org.apache.hadoop.mapred.{TaskAttemptID, JobID, JobConf, JobContext, TaskAttemptContext} - -private[spark] +private[apache] trait SparkHadoopMapRedUtil { def newJobContext(conf: JobConf, jobId: JobID): JobContext = { val klass = firstAvailableClass("org.apache.hadoop.mapred.JobContextImpl", diff --git a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala similarity index 96% rename from core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala rename to core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala index 3340673f91156..1fca5729c6092 100644 --- a/core/src/main/scala/org/apache/spark/mapreduce/SparkHadoopMapReduceUtil.scala +++ b/core/src/main/scala/org/apache/hadoop/mapreduce/SparkHadoopMapReduceUtil.scala @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.mapreduce +package org.apache.hadoop.mapreduce import java.lang.{Boolean => JBoolean, Integer => JInteger} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapreduce.{JobContext, JobID, TaskAttemptContext, TaskAttemptID} -private[spark] +private[apache] trait SparkHadoopMapReduceUtil { def newJobContext(conf: Configuration, jobId: JobID): JobContext = { val klass = firstAvailableClass( diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 40237596570de..376e69cd997d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.rdd.HadoopRDD /** diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index e7b11707551df..324563248793c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -35,7 +35,6 @@ import org.apache.spark.Partition import org.apache.spark.SerializableWritable import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils import org.apache.spark.deploy.SparkHadoopUtil 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 462f0d6268a86..da89f634abaea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -33,14 +33,13 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob, OutputFormat => NewOutputFormat, -RecordWriter => NewRecordWriter} +RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a94d09be3bec6..6a0495f8fd540 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -77,14 +77,6 @@ object MimaExcludes { // SPARK-3822 ProblemFilters.exclude[IncompatibleResultTypeProblem]( "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") - ) ++ Seq( - // SPARK-1209 - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.hadoop.mapred.SparkHadoopMapRedUtil"), - ProblemFilters.exclude[MissingTypesProblem]( - "org.apache.spark.rdd.PairRDDFunctions") ) case v if v.startsWith("1.1") => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 416bf5614437f..9664c565a0b86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -43,7 +43,6 @@ import parquet.hadoop.util.ContextUtil import parquet.io.ParquetDecodingException import parquet.schema.MessageType -import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.SQLConf diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index cc8bb3e172c6e..bf2ce9df67c58 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -31,7 +31,6 @@ import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ -import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc} From 0734d09320fe37edd3a02718511cda0bda852478 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 30 Oct 2014 20:15:36 -0700 Subject: [PATCH 1152/1492] HOTFIX: Clean up build in network module. This is currently breaking the package build for some people (including me). This patch does some general clean-up which also fixes the current issue. - Uses consistent artifact naming - Adds sbt support for this module - Changes tests to use scalatest (fixes the original issue[1]) One thing to note, it turns out that scalatest when invoked in the Maven build doesn't succesfully detect JUnit Java tests. This is a long standing issue, I noticed it applies to all of our current test suites as well. I've created SPARK-4159 to fix this. [1] The original issue is that we need to allocate extra memory for the tests, happens by default in our scalatest configuration. Author: Patrick Wendell Closes #3025 from pwendell/hotfix and squashes the following commits: faa9053 [Patrick Wendell] HOTFIX: Clean up build in network module. --- core/pom.xml | 2 +- network/common/pom.xml | 34 +++++++++++++++++----------------- project/SparkBuild.scala | 8 +++++--- 3 files changed, 23 insertions(+), 21 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 8020a2daf81ec..6963ce4777e6f 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -46,7 +46,7 @@ org.apache.spark - network + spark-network-common_2.10 ${project.version} diff --git a/network/common/pom.xml b/network/common/pom.xml index e3b7e328701b4..a33e44b63d981 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -27,12 +27,12 @@ org.apache.spark - network + spark-network-common_2.10 jar - Shuffle Streaming Service + Spark Project Common Network Code http://spark.apache.org/ - network + network-common @@ -59,6 +59,11 @@ junit test + + com.novocode + junit-interface + test + log4j log4j @@ -69,25 +74,20 @@ mockito-all test + + org.scalatest + scalatest_${scala.binary.version} + test + - - target/java/classes - target/java/test-classes + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes - org.apache.maven.plugins - maven-surefire-plugin - 2.17 - - false - - **/Test*.java - **/*Test.java - **/*Suite.java - - + org.scalatest + scalatest-maven-plugin diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6d5eb681c6131..77083518bbab3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -31,10 +31,10 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, - sql, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, + sql, networkCommon, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, streamingTwitter, streamingZeromq) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "sql", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", + "sql", "network-common", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = @@ -142,7 +142,9 @@ object SparkBuild extends PomBuild { // TODO: Add Sql to mima checks allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - streamingFlumeSink).contains(x)).foreach(x => enable(MimaBuild.mimaSettings(sparkHome, x))(x)) + streamingFlumeSink, networkCommon).contains(x)).foreach { + x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) + } /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) From 872fc669b497fb255db3212568f2a14c2ba0d5db Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 30 Oct 2014 22:25:18 -0700 Subject: [PATCH 1153/1492] [SPARK-4124] [MLlib] [PySpark] simplify serialization in MLlib Python API Create several helper functions to call MLlib Java API, convert the arguments to Java type and convert return value to Python object automatically, this simplify serialization in MLlib Python API very much. After this, the MLlib Python API does not need to deal with serialization details anymore, it's easier to add new API. cc mengxr Author: Davies Liu Closes #2995 from davies/cleanup and squashes the following commits: 8fa6ec6 [Davies Liu] address comments 16b85a0 [Davies Liu] Merge branch 'master' of github.com:apache/spark into cleanup 43743e5 [Davies Liu] bugfix 731331f [Davies Liu] simplify serialization in MLlib Python API --- .../mllib/api/python/PythonMLLibAPI.scala | 84 ++++++----- python/pyspark/mllib/classification.py | 30 ++-- python/pyspark/mllib/clustering.py | 15 +- python/pyspark/mllib/common.py | 135 ++++++++++++++++++ python/pyspark/mllib/feature.py | 122 +++------------- python/pyspark/mllib/linalg.py | 12 -- python/pyspark/mllib/random.py | 34 ++--- python/pyspark/mllib/recommendation.py | 62 ++------ python/pyspark/mllib/regression.py | 52 +++---- python/pyspark/mllib/stat.py | 65 ++------- python/pyspark/mllib/tree.py | 55 ++----- python/pyspark/mllib/util.py | 7 +- 12 files changed, 287 insertions(+), 386 deletions(-) create mode 100644 python/pyspark/mllib/common.py diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 485abe272326c..acdc67ddc660a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream -import java.util.{ArrayList => JArrayList} +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.language.existentials @@ -72,15 +72,11 @@ class PythonMLLibAPI extends Serializable { private def trainRegressionModel( learner: GeneralizedLinearAlgorithm[_ <: GeneralizedLinearModel], data: JavaRDD[LabeledPoint], - initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { - val initialWeights = SerDe.loads(initialWeightsBA).asInstanceOf[Vector] + initialWeights: Vector): JList[Object] = { // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. learner.disableUncachedWarning() val model = learner.run(data.rdd, initialWeights) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.dumps(model.weights)) - ret.add(model.intercept: java.lang.Double) - ret + List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava } /** @@ -91,10 +87,10 @@ class PythonMLLibAPI extends Serializable { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte], + initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): java.util.List[java.lang.Object] = { + intercept: Boolean): JList[Object] = { val lrAlg = new LinearRegressionWithSGD() lrAlg.setIntercept(intercept) lrAlg.optimizer @@ -113,7 +109,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( lrAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -125,7 +121,7 @@ class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + initialWeights: Vector): JList[Object] = { val lassoAlg = new LassoWithSGD() lassoAlg.optimizer .setNumIterations(numIterations) @@ -135,7 +131,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( lassoAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -147,7 +143,7 @@ class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + initialWeights: Vector): JList[Object] = { val ridgeAlg = new RidgeRegressionWithSGD() ridgeAlg.optimizer .setNumIterations(numIterations) @@ -157,7 +153,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( ridgeAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -169,9 +165,9 @@ class PythonMLLibAPI extends Serializable { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte], + initialWeights: Vector, regType: String, - intercept: Boolean): java.util.List[java.lang.Object] = { + intercept: Boolean): JList[Object] = { val SVMAlg = new SVMWithSGD() SVMAlg.setIntercept(intercept) SVMAlg.optimizer @@ -190,7 +186,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( SVMAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -201,10 +197,10 @@ class PythonMLLibAPI extends Serializable { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeightsBA: Array[Byte], + initialWeights: Vector, regParam: Double, regType: String, - intercept: Boolean): java.util.List[java.lang.Object] = { + intercept: Boolean): JList[Object] = { val LogRegAlg = new LogisticRegressionWithSGD() LogRegAlg.setIntercept(intercept) LogRegAlg.optimizer @@ -223,7 +219,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( LogRegAlg, data, - initialWeightsBA) + initialWeights) } /** @@ -231,13 +227,10 @@ class PythonMLLibAPI extends Serializable { */ def trainNaiveBayes( data: JavaRDD[LabeledPoint], - lambda: Double): java.util.List[java.lang.Object] = { + lambda: Double): JList[Object] = { val model = NaiveBayes.train(data.rdd, lambda) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(Vectors.dense(model.labels)) - ret.add(Vectors.dense(model.pi)) - ret.add(model.theta) - ret + List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta). + map(_.asInstanceOf[Object]).asJava } /** @@ -259,6 +252,21 @@ class PythonMLLibAPI extends Serializable { return kMeansAlg.run(data.rdd) } + /** + * A Wrapper of MatrixFactorizationModel to provide helpfer method for Python + */ + private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorizationModel) + extends MatrixFactorizationModel(model.rank, model.userFeatures, model.productFeatures) { + + def predict(userAndProducts: JavaRDD[Array[Any]]): RDD[Rating] = + predict(SerDe.asTupleRDD(userAndProducts.rdd)) + + def getUserFeatures = SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + + def getProductFeatures = SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + + } + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -271,7 +279,7 @@ class PythonMLLibAPI extends Serializable { iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { - ALS.train(ratings.rdd, rank, iterations, lambda, blocks) + new MatrixFactorizationModelWrapper(ALS.train(ratings.rdd, rank, iterations, lambda, blocks)) } /** @@ -287,7 +295,8 @@ class PythonMLLibAPI extends Serializable { lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { - ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) + new MatrixFactorizationModelWrapper( + ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha)) } /** @@ -373,19 +382,16 @@ class PythonMLLibAPI extends Serializable { rdd.rdd.map(model.transform) } - def findSynonyms(word: String, num: Int): java.util.List[java.lang.Object] = { + def findSynonyms(word: String, num: Int): JList[Object] = { val vec = transform(word) findSynonyms(vec, num) } - def findSynonyms(vector: Vector, num: Int): java.util.List[java.lang.Object] = { + def findSynonyms(vector: Vector, num: Int): JList[Object] = { val result = model.findSynonyms(vector, num) val similarity = Vectors.dense(result.map(_._2)) val words = result.map(_._1) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(words) - ret.add(similarity) - ret + List(words, similarity).map(_.asInstanceOf[Object]).asJava } } @@ -395,13 +401,13 @@ class PythonMLLibAPI extends Serializable { * Extra care needs to be taken in the Python code to ensure it gets freed on exit; * see the Py4J documentation. * @param data Training data - * @param categoricalFeaturesInfoJMap Categorical features info, as Java map + * @param categoricalFeaturesInfo Categorical features info, as Java map */ def trainDecisionTreeModel( data: JavaRDD[LabeledPoint], algoStr: String, numClasses: Int, - categoricalFeaturesInfoJMap: java.util.Map[Int, Int], + categoricalFeaturesInfo: JMap[Int, Int], impurityStr: String, maxDepth: Int, maxBins: Int, @@ -417,7 +423,7 @@ class PythonMLLibAPI extends Serializable { maxDepth = maxDepth, numClassesForClassification = numClasses, maxBins = maxBins, - categoricalFeaturesInfo = categoricalFeaturesInfoJMap.asScala.toMap, + categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap, minInstancesPerNode = minInstancesPerNode, minInfoGain = minInfoGain) @@ -589,7 +595,7 @@ private[spark] object SerDe extends Serializable { if (objects.length == 0 || objects.length > 3) { out.write(Opcodes.MARK) } - objects.foreach(pickler.save(_)) + objects.foreach(pickler.save) val code = objects.length match { case 1 => Opcodes.TUPLE1 case 2 => Opcodes.TUPLE2 @@ -719,7 +725,7 @@ private[spark] object SerDe extends Serializable { } /* convert RDD[Tuple2[,]] to RDD[Array[Any]] */ - def fromTuple2RDD(rdd: RDD[Tuple2[Any, Any]]): RDD[Array[Any]] = { + def fromTuple2RDD(rdd: RDD[(Any, Any)]): RDD[Array[Any]] = { rdd.map(x => Array(x._1, x._2)) } diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index e295c9d0954d9..297a2bf37d2cf 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -20,8 +20,8 @@ import numpy from numpy import array -from pyspark import SparkContext, PickleSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc +from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper @@ -102,14 +102,11 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, training data (i.e. whether bias features are activated or not). """ - sc = data.context + def train(rdd, i): + return callMLlibFunc("trainLogisticRegressionModelWithSGD", rdd, iterations, step, + miniBatchFraction, i, regParam, regType, intercept) - def train(jdata, i): - return sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( - jdata, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - - return _regression_train_wrapper(sc, train, LogisticRegressionModel, data, - initialWeights) + return _regression_train_wrapper(train, LogisticRegressionModel, data, initialWeights) class SVMModel(LinearModel): @@ -174,13 +171,11 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, training data (i.e. whether bias features are activated or not). """ - sc = data.context - - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( - jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) + def train(rdd, i): + return callMLlibFunc("trainSVMModelWithSGD", rdd, iterations, step, regParam, + miniBatchFraction, i, regType, intercept) - return _regression_train_wrapper(sc, train, SVMModel, data, initialWeights) + return _regression_train_wrapper(train, SVMModel, data, initialWeights) class NaiveBayesModel(object): @@ -243,14 +238,13 @@ def train(cls, data, lambda_=1.0): (e.g. a count vector). :param lambda_: The smoothing parameter """ - sc = data.context - jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(_to_java_object_rdd(data), lambda_) - labels, pi, theta = PickleSerializer().loads(str(sc._jvm.SerDe.dumps(jlist))) + labels, pi, theta = callMLlibFunc("trainNaiveBayes", data, lambda_) return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) def _test(): import doctest + from pyspark import SparkContext globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 5ee7997104d21..fe4c4cc5094d8 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -16,8 +16,8 @@ # from pyspark import SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _to_java_object_rdd +from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['KMeansModel', 'KMeans'] @@ -80,14 +80,11 @@ class KMeans(object): @classmethod def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" - sc = rdd.context - ser = PickleSerializer() # cache serialized data to avoid objects over head in JVM - cached = rdd.map(_convert_to_vector)._reserialize(AutoBatchedSerializer(ser)).cache() - model = sc._jvm.PythonMLLibAPI().trainKMeansModel( - _to_java_object_rdd(cached), k, maxIterations, runs, initializationMode) - bytes = sc._jvm.SerDe.dumps(model.clusterCenters()) - centers = ser.loads(str(bytes)) + jcached = _to_java_object_rdd(rdd.map(_convert_to_vector), cache=True) + model = callMLlibFunc("trainKMeansModel", jcached, k, maxIterations, runs, + initializationMode) + centers = callJavaFunc(rdd.context, model.clusterCenters) return KMeansModel([c.toArray() for c in centers]) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py new file mode 100644 index 0000000000000..76864d8163586 --- /dev/null +++ b/python/pyspark/mllib/common.py @@ -0,0 +1,135 @@ +# +# 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. +# + +import py4j.protocol +from py4j.protocol import Py4JJavaError +from py4j.java_gateway import JavaObject +from py4j.java_collections import MapConverter, ListConverter, JavaArray, JavaList + +from pyspark import RDD, SparkContext +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer + + +# Hack for support float('inf') in Py4j +_old_smart_decode = py4j.protocol.smart_decode + +_float_str_mapping = { + 'nan': 'NaN', + 'inf': 'Infinity', + '-inf': '-Infinity', +} + + +def _new_smart_decode(obj): + if isinstance(obj, float): + s = unicode(obj) + return _float_str_mapping.get(s, s) + return _old_smart_decode(obj) + +py4j.protocol.smart_decode = _new_smart_decode + + +_picklable_classes = [ + 'LinkedList', + 'SparseVector', + 'DenseVector', + 'DenseMatrix', + 'Rating', + 'LabeledPoint', +] + + +# this will call the MLlib version of pythonToJava() +def _to_java_object_rdd(rdd, cache=False): + """ Return an JavaRDD of Object by unpickling + + It will convert each Python object into Java object by Pyrolite, whenever the + RDD is serialized in batch or not. + """ + rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) + if cache: + rdd.cache() + return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) + + +def _py2java(sc, obj): + """ Convert Python object into Java """ + if isinstance(obj, RDD): + obj = _to_java_object_rdd(obj) + elif isinstance(obj, SparkContext): + obj = obj._jsc + elif isinstance(obj, dict): + obj = MapConverter().convert(obj, sc._gateway._gateway_client) + elif isinstance(obj, (list, tuple)): + obj = ListConverter().convert(obj, sc._gateway._gateway_client) + elif isinstance(obj, JavaObject): + pass + elif isinstance(obj, (int, long, float, bool, basestring)): + pass + else: + bytes = bytearray(PickleSerializer().dumps(obj)) + obj = sc._jvm.SerDe.loads(bytes) + return obj + + +def _java2py(sc, r): + if isinstance(r, JavaObject): + clsName = r.getClass().getSimpleName() + # convert RDD into JavaRDD + if clsName != 'JavaRDD' and clsName.endswith("RDD"): + r = r.toJavaRDD() + clsName = 'JavaRDD' + + if clsName == 'JavaRDD': + jrdd = sc._jvm.SerDe.javaToPython(r) + return RDD(jrdd, sc, AutoBatchedSerializer(PickleSerializer())) + + elif isinstance(r, (JavaArray, JavaList)) or clsName in _picklable_classes: + r = sc._jvm.SerDe.dumps(r) + + if isinstance(r, bytearray): + r = PickleSerializer().loads(str(r)) + return r + + +def callJavaFunc(sc, func, *args): + """ Call Java Function """ + args = [_py2java(sc, a) for a in args] + return _java2py(sc, func(*args)) + + +def callMLlibFunc(name, *args): + """ Call API in PythonMLLibAPI """ + sc = SparkContext._active_spark_context + api = getattr(sc._jvm.PythonMLLibAPI(), name) + return callJavaFunc(sc, api, *args) + + +class JavaModelWrapper(object): + """ + Wrapper for the model in JVM + """ + def __init__(self, java_model): + self._sc = SparkContext._active_spark_context + self._java_model = java_model + + def __del__(self): + self._sc._gateway.detach(self._java_model) + + def call(self, name, *a): + """Call method of java_model""" + return callJavaFunc(self._sc, getattr(self._java_model, name), *a) diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 324343443ebdb..44bf6f269d7a3 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -21,89 +21,16 @@ import sys import warnings -import py4j.protocol from py4j.protocol import Py4JJavaError -from py4j.java_gateway import JavaObject from pyspark import RDD, SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import Vectors, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper +from pyspark.mllib.linalg import Vectors __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] -# Hack for support float('inf') in Py4j -_old_smart_decode = py4j.protocol.smart_decode - -_float_str_mapping = { - u'nan': u'NaN', - u'inf': u'Infinity', - u'-inf': u'-Infinity', -} - - -def _new_smart_decode(obj): - if isinstance(obj, float): - s = unicode(obj) - return _float_str_mapping.get(s, s) - return _old_smart_decode(obj) - -py4j.protocol.smart_decode = _new_smart_decode - - -# TODO: move these helper functions into utils -_picklable_classes = [ - 'LinkedList', - 'SparseVector', - 'DenseVector', - 'DenseMatrix', - 'Rating', - 'LabeledPoint', -] - - -def _py2java(sc, a): - """ Convert Python object into Java """ - if isinstance(a, RDD): - a = _to_java_object_rdd(a) - elif not isinstance(a, (int, long, float, bool, basestring)): - bytes = bytearray(PickleSerializer().dumps(a)) - a = sc._jvm.SerDe.loads(bytes) - return a - - -def _java2py(sc, r): - if isinstance(r, JavaObject): - clsName = r.getClass().getSimpleName() - if clsName in ("RDD", "JavaRDD"): - if clsName == "RDD": - r = r.toJavaRDD() - jrdd = sc._jvm.SerDe.javaToPython(r) - return RDD(jrdd, sc, AutoBatchedSerializer(PickleSerializer())) - - elif clsName in _picklable_classes: - r = sc._jvm.SerDe.dumps(r) - - if isinstance(r, bytearray): - r = PickleSerializer().loads(str(r)) - return r - - -def _callJavaFunc(sc, func, *args): - """ Call Java Function - """ - args = [_py2java(sc, a) for a in args] - return _java2py(sc, func(*args)) - - -def _callAPI(sc, name, *args): - """ Call API in PythonMLLibAPI - """ - api = getattr(sc._jvm.PythonMLLibAPI(), name) - return _callJavaFunc(sc, api, *args) - - class VectorTransformer(object): """ :: DeveloperApi :: @@ -160,25 +87,19 @@ def transform(self, vector): """ sc = SparkContext._active_spark_context assert sc is not None, "SparkContext should be initialized first" - return _callAPI(sc, "normalizeVector", self.p, vector) + return callMLlibFunc("normalizeVector", self.p, vector) -class JavaModelWrapper(VectorTransformer): +class JavaVectorTransformer(JavaModelWrapper, VectorTransformer): """ Wrapper for the model in JVM """ - def __init__(self, sc, java_model): - self._sc = sc - self._java_model = java_model - - def __del__(self): - self._sc._gateway.detach(self._java_model) def transform(self, dataset): - return _callJavaFunc(self._sc, self._java_model.transform, dataset) + return self.call("transform", dataset) -class StandardScalerModel(JavaModelWrapper): +class StandardScalerModel(JavaVectorTransformer): """ :: Experimental :: @@ -192,7 +113,7 @@ def transform(self, vector): :return: Standardized vector. If the variance of a column is zero, it will return default `0.0` for the column with zero variance. """ - return JavaModelWrapper.transform(self, vector) + return JavaVectorTransformer.transform(self, vector) class StandardScaler(object): @@ -233,9 +154,8 @@ def fit(self, dataset): the transformation model. :return: a StandardScalarModel """ - sc = dataset.context - jmodel = _callAPI(sc, "fitStandardScaler", self.withMean, self.withStd, dataset) - return StandardScalerModel(sc, jmodel) + jmodel = callMLlibFunc("fitStandardScaler", self.withMean, self.withStd, dataset) + return StandardScalerModel(jmodel) class HashingTF(object): @@ -276,7 +196,7 @@ def transform(self, document): return Vectors.sparse(self.numFeatures, freq.items()) -class IDFModel(JavaModelWrapper): +class IDFModel(JavaVectorTransformer): """ Represents an IDF model that can transform term frequency vectors. """ @@ -291,7 +211,7 @@ def transform(self, dataset): :param dataset: an RDD of term frequency vectors :return: an RDD of TF-IDF vectors """ - return JavaModelWrapper.transform(self, dataset) + return JavaVectorTransformer.transform(self, dataset) class IDF(object): @@ -335,12 +255,11 @@ def fit(self, dataset): :param dataset: an RDD of term frequency vectors """ - sc = dataset.context - jmodel = _callAPI(sc, "fitIDF", self.minDocFreq, dataset) - return IDFModel(sc, jmodel) + jmodel = callMLlibFunc("fitIDF", self.minDocFreq, dataset) + return IDFModel(jmodel) -class Word2VecModel(JavaModelWrapper): +class Word2VecModel(JavaVectorTransformer): """ class for Word2Vec model """ @@ -354,7 +273,7 @@ def transform(self, word): :return: vector representation of word(s) """ try: - return _callJavaFunc(self._sc, self._java_model.transform, word) + return self.call("transform", word) except Py4JJavaError: raise ValueError("%s not found" % word) @@ -368,7 +287,7 @@ def findSynonyms(self, word, num): Note: local use only """ - words, similarity = _callJavaFunc(self._sc, self._java_model.findSynonyms, word, num) + words, similarity = self.call("findSynonyms", word, num) return zip(words, similarity) @@ -458,11 +377,10 @@ def fit(self, data): :param data: training data. RDD of subtype of Iterable[String] :return: Word2VecModel instance """ - sc = data.context - jmodel = _callAPI(sc, "trainWord2Vec", data, int(self.vectorSize), - float(self.learningRate), int(self.numPartitions), - int(self.numIterations), long(self.seed)) - return Word2VecModel(sc, jmodel) + jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), + float(self.learningRate), int(self.numPartitions), + int(self.numIterations), long(self.seed)) + return Word2VecModel(jmodel) def _test(): diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 1b9bf596242df..d0a0e102a1a07 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -29,7 +29,6 @@ import numpy as np -from pyspark.serializers import AutoBatchedSerializer, PickleSerializer __all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] @@ -52,17 +51,6 @@ def fast_pickle_array(ar): _have_scipy = False -# this will call the MLlib version of pythonToJava() -def _to_java_object_rdd(rdd): - """ Return an JavaRDD of Object by unpickling - - It will convert each Python object into Java object by Pyrolite, whenever the - RDD is serialized in batch or not. - """ - rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) - return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) - - def _convert_to_vector(l): if isinstance(l, Vector): return l diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 2202c51ab9c06..7eebfc6bcd894 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -21,22 +21,12 @@ from functools import wraps -from pyspark.rdd import RDD -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.common import callMLlibFunc __all__ = ['RandomRDDs', ] -def serialize(f): - @wraps(f) - def func(sc, *a, **kw): - jrdd = f(sc, *a, **kw) - return RDD(sc._jvm.SerDe.javaToPython(jrdd), sc, - BatchedSerializer(PickleSerializer(), 1024)) - return func - - def toArray(f): @wraps(f) def func(sc, *a, **kw): @@ -52,7 +42,6 @@ class RandomRDDs(object): """ @staticmethod - @serialize def uniformRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the @@ -74,10 +63,9 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): >>> parts == sc.defaultParallelism True """ - return sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) + return callMLlibFunc("uniformRDD", sc._jsc, size, numPartitions, seed) @staticmethod - @serialize def normalRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the standard normal @@ -97,10 +85,9 @@ def normalRDD(sc, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - 1.0) < 0.1 True """ - return sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) + return callMLlibFunc("normalRDD", sc._jsc, size, numPartitions, seed) @staticmethod - @serialize def poissonRDD(sc, mean, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the Poisson @@ -117,11 +104,10 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - sqrt(mean)) < 0.5 True """ - return sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) + return callMLlibFunc("poissonRDD", sc._jsc, mean, size, numPartitions, seed) @staticmethod @toArray - @serialize def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -136,12 +122,10 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 """ - return sc._jvm.PythonMLLibAPI() \ - .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + return callMLlibFunc("uniformVectorRDD", sc._jsc, numRows, numCols, numPartitions, seed) @staticmethod @toArray - @serialize def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -156,12 +140,10 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - 1.0) < 0.1 True """ - return sc._jvm.PythonMLLibAPI() \ - .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) + return callMLlibFunc("normalVectorRDD", sc._jsc, numRows, numCols, numPartitions, seed) @staticmethod @toArray - @serialize def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -179,8 +161,8 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - sqrt(mean)) < 0.5 True """ - return sc._jvm.PythonMLLibAPI() \ - .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) + return callMLlibFunc("poissonVectorRDD", sc._jsc, mean, numRows, numCols, + numPartitions, seed) def _test(): diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 22872dbbe3b55..6b32af07c9be2 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -16,9 +16,8 @@ # from pyspark import SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.rdd import RDD -from pyspark.mllib.linalg import _to_java_object_rdd +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, _to_java_object_rdd __all__ = ['MatrixFactorizationModel', 'ALS'] @@ -36,7 +35,7 @@ def __repr__(self): return "Rating(%d, %d, %d)" % (self.user, self.product, self.rating) -class MatrixFactorizationModel(object): +class MatrixFactorizationModel(JavaModelWrapper): """A matrix factorisation model trained by regularized alternating least-squares. @@ -71,48 +70,21 @@ class MatrixFactorizationModel(object): >>> len(latents) == 4 True """ - - def __init__(self, sc, java_model): - self._context = sc - self._java_model = java_model - - def __del__(self): - self._context._gateway.detach(self._java_model) - def predict(self, user, product): return self._java_model.predict(user, product) def predictAll(self, user_product): assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" first = user_product.first() - if isinstance(first, list): - user_product = user_product.map(tuple) - first = tuple(first) - assert type(first) is tuple and len(first) == 2, \ - "user_product should be RDD of (user, product)" - if any(isinstance(x, str) for x in first): - user_product = user_product.map(lambda (u, p): (int(x), int(p))) - first = tuple(map(int, first)) - assert all(type(x) is int for x in first), "user and product in user_product shoul be int" - sc = self._context - tuplerdd = sc._jvm.SerDe.asTupleRDD(_to_java_object_rdd(user_product).rdd()) - jresult = self._java_model.predict(tuplerdd).toJavaRDD() - return RDD(sc._jvm.SerDe.javaToPython(jresult), sc, - AutoBatchedSerializer(PickleSerializer())) + assert len(first) == 2, "user_product should be RDD of (user, product)" + user_product = user_product.map(lambda (u, p): (int(u), int(p))) + return self.call("predict", user_product) def userFeatures(self): - sc = self._context - juf = self._java_model.userFeatures() - juf = sc._jvm.SerDe.fromTuple2RDD(juf).toJavaRDD() - return RDD(sc._jvm.PythonRDD.javaToPython(juf), sc, - AutoBatchedSerializer(PickleSerializer())) + return self.call("getUserFeatures") def productFeatures(self): - sc = self._context - jpf = self._java_model.productFeatures() - jpf = sc._jvm.SerDe.fromTuple2RDD(jpf).toJavaRDD() - return RDD(sc._jvm.PythonRDD.javaToPython(jpf), sc, - AutoBatchedSerializer(PickleSerializer())) + return self.call("getProductFeatures") class ALS(object): @@ -126,25 +98,19 @@ def _prepare(cls, ratings): ratings = ratings.map(lambda x: Rating(*x)) else: raise ValueError("rating should be RDD of Rating or tuple/list") - # serialize them by AutoBatchedSerializer before cache to reduce the - # objects overhead in JVM - cached = ratings._reserialize(AutoBatchedSerializer(PickleSerializer())).cache() - return _to_java_object_rdd(cached) + return _to_java_object_rdd(ratings, True) @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): - sc = ratings.context - jrating = cls._prepare(ratings) - mod = sc._jvm.PythonMLLibAPI().trainALSModel(jrating, rank, iterations, lambda_, blocks) - return MatrixFactorizationModel(sc, mod) + model = callMLlibFunc("trainALSModel", cls._prepare(ratings), rank, iterations, + lambda_, blocks) + return MatrixFactorizationModel(model) @classmethod def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01): - sc = ratings.context - jrating = cls._prepare(ratings) - mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel( - jrating, rank, iterations, lambda_, blocks, alpha) - return MatrixFactorizationModel(sc, mod) + model = callMLlibFunc("trainImplicitALSModel", cls._prepare(ratings), rank, + iterations, lambda_, blocks, alpha) + return MatrixFactorizationModel(model) def _test(): diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 93e17faf5cd51..43c1a2fc101dd 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,9 +18,8 @@ import numpy as np from numpy import array -from pyspark import SparkContext -from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import SparseVector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, _to_java_object_rdd +from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -124,17 +123,11 @@ class LinearRegressionModel(LinearRegressionModelBase): # train_func should take two parameters, namely data and initial_weights, and # return the result of a call to the appropriate JVM stub. # _regression_train_wrapper is responsible for setup and error checking. -def _regression_train_wrapper(sc, train_func, modelClass, data, initial_weights): +def _regression_train_wrapper(train_func, modelClass, data, initial_weights): initial_weights = initial_weights or [0.0] * len(data.first().features) - ser = PickleSerializer() - initial_bytes = bytearray(ser.dumps(_convert_to_vector(initial_weights))) - # use AutoBatchedSerializer before cache to reduce the memory - # overhead in JVM - cached = data._reserialize(AutoBatchedSerializer(ser)).cache() - ans = train_func(_to_java_object_rdd(cached), initial_bytes) - assert len(ans) == 2, "JVM call result had unexpected length" - weights = ser.loads(str(ans[0])) - return modelClass(weights, ans[1]) + weights, intercept = train_func(_to_java_object_rdd(data, cache=True), + _convert_to_vector(initial_weights)) + return modelClass(weights, intercept) class LinearRegressionWithSGD(object): @@ -168,13 +161,12 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, training data (i.e. whether bias features are activated or not). """ - sc = data.context + def train(rdd, i): + return callMLlibFunc("trainLinearRegressionModelWithSGD", rdd, iterations, step, + miniBatchFraction, i, regParam, regType, intercept) - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - - return _regression_train_wrapper(sc, train, LinearRegressionModel, data, initialWeights) + return _regression_train_wrapper(train, LinearRegressionModel, + data, initialWeights) class LassoModel(LinearRegressionModelBase): @@ -216,12 +208,10 @@ class LassoWithSGD(object): def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a Lasso regression model on the given data.""" - sc = data.context - - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( - jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train, LassoModel, data, initialWeights) + def train(rdd, i): + return callMLlibFunc("trainLassoModelWithSGD", rdd, iterations, step, regParam, + miniBatchFraction, i) + return _regression_train_wrapper(train, LassoModel, data, initialWeights) class RidgeRegressionModel(LinearRegressionModelBase): @@ -263,17 +253,17 @@ class RidgeRegressionWithSGD(object): def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a ridge regression model on the given data.""" - sc = data.context - - def train(jrdd, i): - return sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( - jrdd, iterations, step, regParam, miniBatchFraction, i) + def train(rdd, i): + return callMLlibFunc("trainRidgeModelWithSGD", rdd, iterations, step, regParam, + miniBatchFraction, i) - return _regression_train_wrapper(sc, train, RidgeRegressionModel, data, initialWeights) + return _regression_train_wrapper(train, RidgeRegressionModel, + data, initialWeights) def _test(): import doctest + from pyspark import SparkContext globs = globals().copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 84baf12b906df..15f0652f833d7 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -19,66 +19,36 @@ Python package for statistical functions in MLlib. """ -from functools import wraps - -from pyspark import PickleSerializer -from pyspark.mllib.linalg import _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper +from pyspark.mllib.linalg import _convert_to_vector __all__ = ['MultivariateStatisticalSummary', 'Statistics'] -def serialize(f): - ser = PickleSerializer() - - @wraps(f) - def func(self): - jvec = f(self) - bytes = self._sc._jvm.SerDe.dumps(jvec) - return ser.loads(str(bytes)).toArray() - - return func - - -class MultivariateStatisticalSummary(object): +class MultivariateStatisticalSummary(JavaModelWrapper): """ Trait for multivariate statistical summary of a data matrix. """ - def __init__(self, sc, java_summary): - """ - :param sc: Spark context - :param java_summary: Handle to Java summary object - """ - self._sc = sc - self._java_summary = java_summary - - def __del__(self): - self._sc._gateway.detach(self._java_summary) - - @serialize def mean(self): - return self._java_summary.mean() + return self.call("mean").toArray() - @serialize def variance(self): - return self._java_summary.variance() + return self.call("variance").toArray() def count(self): - return self._java_summary.count() + return self.call("count") - @serialize def numNonzeros(self): - return self._java_summary.numNonzeros() + return self.call("numNonzeros").toArray() - @serialize def max(self): - return self._java_summary.max() + return self.call("max").toArray() - @serialize def min(self): - return self._java_summary.min() + return self.call("min").toArray() class Statistics(object): @@ -106,10 +76,8 @@ def colStats(rdd): >>> cStats.min() array([ 2., 0., 0., -2.]) """ - sc = rdd.ctx - jrdd = _to_java_object_rdd(rdd.map(_convert_to_vector)) - cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) - return MultivariateStatisticalSummary(sc, cStats) + cStats = callMLlibFunc("colStats", rdd.map(_convert_to_vector)) + return MultivariateStatisticalSummary(cStats) @staticmethod def corr(x, y=None, method=None): @@ -156,7 +124,6 @@ def corr(x, y=None, method=None): ... except TypeError: ... pass """ - sc = x.ctx # Check inputs to determine whether a single value or a matrix is needed for output. # Since it's legal for users to use the method name as the second argument, we need to # check if y is used to specify the method name instead. @@ -164,15 +131,9 @@ def corr(x, y=None, method=None): raise TypeError("Use 'method=' to specify method name.") if not y: - jx = _to_java_object_rdd(x.map(_convert_to_vector)) - resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) - bytes = sc._jvm.SerDe.dumps(resultMat) - ser = PickleSerializer() - return ser.loads(str(bytes)).toArray() + return callMLlibFunc("corr", x.map(_convert_to_vector), method).toArray() else: - jx = _to_java_object_rdd(x.map(float)) - jy = _to_java_object_rdd(y.map(float)) - return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) + return callMLlibFunc("corr", x.map(float), y.map(float), method) def _test(): diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 64ee79d83e849..5d1a3c0962796 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -15,36 +15,22 @@ # limitations under the License. # -from py4j.java_collections import MapConverter - from pyspark import SparkContext, RDD -from pyspark.serializers import BatchedSerializer, PickleSerializer -from pyspark.mllib.linalg import Vector, _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper +from pyspark.mllib.linalg import _convert_to_vector from pyspark.mllib.regression import LabeledPoint __all__ = ['DecisionTreeModel', 'DecisionTree'] -class DecisionTreeModel(object): +class DecisionTreeModel(JavaModelWrapper): """ A decision tree model for classification or regression. EXPERIMENTAL: This is an experimental API. - It will probably be modified for Spark v1.2. + It will probably be modified in future. """ - - def __init__(self, sc, java_model): - """ - :param sc: Spark context - :param java_model: Handle to Java model object - """ - self._sc = sc - self._java_model = java_model - - def __del__(self): - self._sc._gateway.detach(self._java_model) - def predict(self, x): """ Predict the label of one or more examples. @@ -52,24 +38,11 @@ def predict(self, x): :param x: Data point (feature vector), or an RDD of data points (feature vectors). """ - SerDe = self._sc._jvm.SerDe - ser = PickleSerializer() if isinstance(x, RDD): - # Bulk prediction - first = x.take(1) - if not first: - return self._sc.parallelize([]) - if not isinstance(first[0], Vector): - x = x.map(_convert_to_vector) - jPred = self._java_model.predict(_to_java_object_rdd(x)).toJavaRDD() - jpyrdd = self._sc._jvm.SerDe.javaToPython(jPred) - return RDD(jpyrdd, self._sc, BatchedSerializer(ser, 1024)) + return self.call("predict", x.map(_convert_to_vector)) else: - # Assume x is a single data point. - bytes = bytearray(ser.dumps(_convert_to_vector(x))) - vec = self._sc._jvm.SerDe.loads(bytes) - return self._java_model.predict(vec) + return self.call("predict", _convert_to_vector(x)) def numNodes(self): return self._java_model.numNodes() @@ -98,19 +71,13 @@ class DecisionTree(object): """ @staticmethod - def _train(data, type, numClasses, categoricalFeaturesInfo, - impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, - minInfoGain=0.0): + def _train(data, type, numClasses, features, impurity="gini", maxDepth=5, maxBins=32, + minInstancesPerNode=1, minInfoGain=0.0): first = data.first() assert isinstance(first, LabeledPoint), "the data should be RDD of LabeledPoint" - sc = data.context - jrdd = _to_java_object_rdd(data) - cfiMap = MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - jrdd, type, numClasses, cfiMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - return DecisionTreeModel(sc, model) + model = callMLlibFunc("trainDecisionTreeModel", data, type, numClasses, features, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) + return DecisionTreeModel(model) @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 84b39a48619d2..96aef8f510fa6 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -18,8 +18,7 @@ import numpy as np import warnings -from pyspark.rdd import RDD -from pyspark.serializers import AutoBatchedSerializer, PickleSerializer +from pyspark.mllib.common import callMLlibFunc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -173,9 +172,7 @@ def loadLabeledPoints(sc, path, minPartitions=None): (0.0,[1.01,2.02,3.03]) """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) - jrdd = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) - jpyrdd = sc._jvm.SerDe.javaToPython(jrdd) - return RDD(jpyrdd, sc, AutoBatchedSerializer(PickleSerializer())) + return callMLlibFunc("loadLabeledPoints", sc, path, minPartitions) def _test(): From ad3bd0dff8997861c5a04438145ba6f91c57a849 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Thu, 30 Oct 2014 22:30:52 -0700 Subject: [PATCH 1154/1492] [SPARK-3250] Implement Gap Sampling optimization for random sampling More efficient sampling, based on Gap Sampling optimization: http://erikerlandson.github.io/blog/2014/09/11/faster-random-samples-with-gap-sampling/ Author: Erik Erlandson Closes #2455 from erikerlandson/spark-3250-pr and squashes the following commits: 72496bc [Erik Erlandson] [SPARK-3250] Implement Gap Sampling optimization for random sampling --- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../spark/util/random/RandomSampler.scala | 286 ++++++++- .../java/org/apache/spark/JavaAPISuite.java | 9 +- .../util/random/RandomSamplerSuite.scala | 606 +++++++++++++++--- .../org/apache/spark/mllib/util/MLUtils.scala | 4 +- 5 files changed, 790 insertions(+), 121 deletions(-) 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 b7f125d01dfaf..c169b2d3fe97f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -43,7 +43,8 @@ import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite} import org.apache.spark.util.collection.OpenHashMap -import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} +import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, BernoulliCellSampler, + SamplingUtils} /** * A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Represents an immutable, @@ -375,7 +376,8 @@ abstract class RDD[T: ClassTag]( val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => - new PartitionwiseSampledRDD[T, T](this, new BernoulliSampler[T](x(0), x(1)), true, seed) + new PartitionwiseSampledRDD[T, T]( + this, new BernoulliCellSampler[T](x(0), x(1)), true, seed) }.toArray } diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index ee389def20c8c..76e7a2760bcd1 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -19,6 +19,9 @@ package org.apache.spark.util.random import java.util.Random +import scala.reflect.ClassTag +import scala.collection.mutable.ArrayBuffer + import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.annotation.DeveloperApi @@ -38,13 +41,47 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable /** take a random sample */ def sample(items: Iterator[T]): Iterator[U] + /** return a copy of the RandomSampler object */ override def clone: RandomSampler[T, U] = throw new NotImplementedError("clone() is not implemented.") } +private[spark] +object RandomSampler { + /** Default random number generator used by random samplers. */ + def newDefaultRNG: Random = new XORShiftRandom + + /** + * Default maximum gap-sampling fraction. + * For sampling fractions <= this value, the gap sampling optimization will be applied. + * Above this value, it is assumed that "tradtional" Bernoulli sampling is faster. The + * optimal value for this will depend on the RNG. More expensive RNGs will tend to make + * the optimal value higher. The most reliable way to determine this value for a new RNG + * is to experiment. When tuning for a new RNG, I would expect a value of 0.5 to be close + * in most cases, as an initial guess. + */ + val defaultMaxGapSamplingFraction = 0.4 + + /** + * Default epsilon for floating point numbers sampled from the RNG. + * The gap-sampling compute logic requires taking log(x), where x is sampled from an RNG. + * To guard against errors from taking log(0), a positive epsilon lower bound is applied. + * A good value for this parameter is at or near the minimum positive floating + * point value returned by "nextDouble()" (or equivalent), for the RNG being used. + */ + val rngEpsilon = 5e-11 + + /** + * Sampling fraction arguments may be results of computation, and subject to floating + * point jitter. I check the arguments with this epsilon slop factor to prevent spurious + * warnings for cases such as summing some numbers to get a sampling fraction of 1.000000001 + */ + val roundingEpsilon = 1e-6 +} + /** * :: DeveloperApi :: - * A sampler based on Bernoulli trials. + * A sampler based on Bernoulli trials for partitioning a data sequence. * * @param lb lower bound of the acceptance range * @param ub upper bound of the acceptance range @@ -52,57 +89,262 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @tparam T item type */ @DeveloperApi -class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) +class BernoulliCellSampler[T](lb: Double, ub: Double, complement: Boolean = false) extends RandomSampler[T, T] { - private[random] var rng: Random = new XORShiftRandom + /** epsilon slop to avoid failure from floating point jitter. */ + require( + lb <= (ub + RandomSampler.roundingEpsilon), + s"Lower bound ($lb) must be <= upper bound ($ub)") + require( + lb >= (0.0 - RandomSampler.roundingEpsilon), + s"Lower bound ($lb) must be >= 0.0") + require( + ub <= (1.0 + RandomSampler.roundingEpsilon), + s"Upper bound ($ub) must be <= 1.0") - def this(ratio: Double) = this(0.0d, ratio) + private val rng: Random = new XORShiftRandom override def setSeed(seed: Long) = rng.setSeed(seed) override def sample(items: Iterator[T]): Iterator[T] = { - items.filter { item => - val x = rng.nextDouble() - (x >= lb && x < ub) ^ complement + if (ub - lb <= 0.0) { + if (complement) items else Iterator.empty + } else { + if (complement) { + items.filter { item => { + val x = rng.nextDouble() + (x < lb) || (x >= ub) + }} + } else { + items.filter { item => { + val x = rng.nextDouble() + (x >= lb) && (x < ub) + }} + } } } /** * Return a sampler that is the complement of the range specified of the current sampler. */ - def cloneComplement(): BernoulliSampler[T] = new BernoulliSampler[T](lb, ub, !complement) + def cloneComplement(): BernoulliCellSampler[T] = + new BernoulliCellSampler[T](lb, ub, !complement) + + override def clone = new BernoulliCellSampler[T](lb, ub, complement) +} + + +/** + * :: DeveloperApi :: + * A sampler based on Bernoulli trials. + * + * @param fraction the sampling fraction, aka Bernoulli sampling probability + * @tparam T item type + */ +@DeveloperApi +class BernoulliSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T] { + + /** epsilon slop to avoid failure from floating point jitter */ + require( + fraction >= (0.0 - RandomSampler.roundingEpsilon) + && fraction <= (1.0 + RandomSampler.roundingEpsilon), + s"Sampling fraction ($fraction) must be on interval [0, 1]") - override def clone = new BernoulliSampler[T](lb, ub, complement) + private val rng: Random = RandomSampler.newDefaultRNG + + override def setSeed(seed: Long) = rng.setSeed(seed) + + override def sample(items: Iterator[T]): Iterator[T] = { + if (fraction <= 0.0) { + Iterator.empty + } else if (fraction >= 1.0) { + items + } else if (fraction <= RandomSampler.defaultMaxGapSamplingFraction) { + new GapSamplingIterator(items, fraction, rng, RandomSampler.rngEpsilon) + } else { + items.filter { _ => rng.nextDouble() <= fraction } + } + } + + override def clone = new BernoulliSampler[T](fraction) } + /** * :: DeveloperApi :: - * A sampler based on values drawn from Poisson distribution. + * A sampler for sampling with replacement, based on values drawn from Poisson distribution. * - * @param mean Poisson mean + * @param fraction the sampling fraction (with replacement) * @tparam T item type */ @DeveloperApi -class PoissonSampler[T](mean: Double) extends RandomSampler[T, T] { +class PoissonSampler[T: ClassTag](fraction: Double) extends RandomSampler[T, T] { + + /** Epsilon slop to avoid failure from floating point jitter. */ + require( + fraction >= (0.0 - RandomSampler.roundingEpsilon), + s"Sampling fraction ($fraction) must be >= 0") - private[random] var rng = new PoissonDistribution(mean) + // PoissonDistribution throws an exception when fraction <= 0 + // If fraction is <= 0, Iterator.empty is used below, so we can use any placeholder value. + private val rng = new PoissonDistribution(if (fraction > 0.0) fraction else 1.0) + private val rngGap = RandomSampler.newDefaultRNG override def setSeed(seed: Long) { - rng = new PoissonDistribution(mean) rng.reseedRandomGenerator(seed) + rngGap.setSeed(seed) } override def sample(items: Iterator[T]): Iterator[T] = { - items.flatMap { item => - val count = rng.sample() - if (count == 0) { - Iterator.empty - } else { - Iterator.fill(count)(item) - } + if (fraction <= 0.0) { + Iterator.empty + } else if (fraction <= RandomSampler.defaultMaxGapSamplingFraction) { + new GapSamplingReplacementIterator(items, fraction, rngGap, RandomSampler.rngEpsilon) + } else { + items.flatMap { item => { + val count = rng.sample() + if (count == 0) Iterator.empty else Iterator.fill(count)(item) + }} + } + } + + override def clone = new PoissonSampler[T](fraction) +} + + +private[spark] +class GapSamplingIterator[T: ClassTag]( + var data: Iterator[T], + f: Double, + rng: Random = RandomSampler.newDefaultRNG, + epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { + + require(f > 0.0 && f < 1.0, s"Sampling fraction ($f) must reside on open interval (0, 1)") + require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") + + /** implement efficient linear-sequence drop until Scala includes fix for jira SI-8835. */ + private val iterDrop: Int => Unit = { + val arrayClass = Array.empty[T].iterator.getClass + val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass + data.getClass match { + case `arrayClass` => ((n: Int) => { data = data.drop(n) }) + case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) + case _ => ((n: Int) => { + var j = 0 + while (j < n && data.hasNext) { + data.next() + j += 1 + } + }) + } + } + + override def hasNext: Boolean = data.hasNext + + override def next(): T = { + val r = data.next() + advance + r + } + + private val lnq = math.log1p(-f) + + /** skip elements that won't be sampled, according to geometric dist P(k) = (f)(1-f)^k. */ + private def advance: Unit = { + val u = math.max(rng.nextDouble(), epsilon) + val k = (math.log(u) / lnq).toInt + iterDrop(k) + } + + /** advance to first sample as part of object construction. */ + advance + // Attempting to invoke this closer to the top with other object initialization + // was causing it to break in strange ways, so I'm invoking it last, which seems to + // work reliably. +} + +private[spark] +class GapSamplingReplacementIterator[T: ClassTag]( + var data: Iterator[T], + f: Double, + rng: Random = RandomSampler.newDefaultRNG, + epsilon: Double = RandomSampler.rngEpsilon) extends Iterator[T] { + + require(f > 0.0, s"Sampling fraction ($f) must be > 0") + require(epsilon > 0.0, s"epsilon ($epsilon) must be > 0") + + /** implement efficient linear-sequence drop until scala includes fix for jira SI-8835. */ + private val iterDrop: Int => Unit = { + val arrayClass = Array.empty[T].iterator.getClass + val arrayBufferClass = ArrayBuffer.empty[T].iterator.getClass + data.getClass match { + case `arrayClass` => ((n: Int) => { data = data.drop(n) }) + case `arrayBufferClass` => ((n: Int) => { data = data.drop(n) }) + case _ => ((n: Int) => { + var j = 0 + while (j < n && data.hasNext) { + data.next() + j += 1 + } + }) + } + } + + /** current sampling value, and its replication factor, as we are sampling with replacement. */ + private var v: T = _ + private var rep: Int = 0 + + override def hasNext: Boolean = data.hasNext || rep > 0 + + override def next(): T = { + val r = v + rep -= 1 + if (rep <= 0) advance + r + } + + /** + * Skip elements with replication factor zero (i.e. elements that won't be sampled). + * Samples 'k' from geometric distribution P(k) = (1-q)(q)^k, where q = e^(-f), that is + * q is the probabililty of Poisson(0; f) + */ + private def advance: Unit = { + val u = math.max(rng.nextDouble(), epsilon) + val k = (math.log(u) / (-f)).toInt + iterDrop(k) + // set the value and replication factor for the next value + if (data.hasNext) { + v = data.next() + rep = poissonGE1 + } + } + + private val q = math.exp(-f) + + /** + * Sample from Poisson distribution, conditioned such that the sampled value is >= 1. + * This is an adaptation from the algorithm for Generating Poisson distributed random variables: + * http://en.wikipedia.org/wiki/Poisson_distribution + */ + private def poissonGE1: Int = { + // simulate that the standard poisson sampling + // gave us at least one iteration, for a sample of >= 1 + var pp = q + ((1.0 - q) * rng.nextDouble()) + var r = 1 + + // now continue with standard poisson sampling algorithm + pp *= rng.nextDouble() + while (pp > q) { + r += 1 + pp *= rng.nextDouble() } + r } - override def clone = new PoissonSampler[T](mean) + /** advance to first sample as part of object construction. */ + advance + // Attempting to invoke this closer to the top with other object initialization + // was causing it to break in strange ways, so I'm invoking it last, which seems to + // work reliably. } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 0172876a264b8..c21a4b30d7726 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -140,11 +140,10 @@ public void intersection() { public void sample() { List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); JavaRDD rdd = sc.parallelize(ints); - JavaRDD sample20 = rdd.sample(true, 0.2, 11); - // expected 2 but of course result varies randomly a bit - Assert.assertEquals(1, sample20.count()); - JavaRDD sample20NoReplacement = rdd.sample(false, 0.2, 11); - Assert.assertEquals(2, sample20NoReplacement.count()); + JavaRDD sample20 = rdd.sample(true, 0.2, 3); + Assert.assertEquals(2, sample20.count()); + JavaRDD sample20WithoutReplacement = rdd.sample(false, 0.2, 5); + Assert.assertEquals(2, sample20WithoutReplacement.count()); } @Test diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index ba67d766a775a..20944b62473c5 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -18,97 +18,523 @@ package org.apache.spark.util.random import java.util.Random - +import scala.collection.mutable.ArrayBuffer import org.apache.commons.math3.distribution.PoissonDistribution -import org.scalatest.{BeforeAndAfter, FunSuite} -import org.scalatest.mock.EasyMockSugar - -class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { - - val a = List(1, 2, 3, 4, 5, 6, 7, 8, 9) - - var random: Random = _ - var poisson: PoissonDistribution = _ - - before { - random = mock[Random] - poisson = mock[PoissonDistribution] - } - - test("BernoulliSamplerWithRange") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55) - sampler.rng = random - assert(sampler.sample(a.iterator).toList == List(3, 4, 5)) - } - } - - test("BernoulliSamplerWithRangeInverse") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55, true) - sampler.rng = random - assert(sampler.sample(a.iterator).toList === List(1, 2, 6, 7, 8, 9)) - } - } - - test("BernoulliSamplerWithRatio") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.35) - sampler.rng = random - assert(sampler.sample(a.iterator).toList == List(1, 2, 3)) - } - } - - test("BernoulliSamplerWithComplement") { - expecting { - for(x <- Seq(0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9)) { - random.nextDouble().andReturn(x) - } - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.25, 0.55, true) - sampler.rng = random - assert(sampler.sample(a.iterator).toList == List(1, 2, 6, 7, 8, 9)) - } - } - - test("BernoulliSamplerSetSeed") { - expecting { - random.setSeed(10L) - } - whenExecuting(random) { - val sampler = new BernoulliSampler[Int](0.2) - sampler.rng = random - sampler.setSeed(10L) - } - } - - test("PoissonSampler") { - expecting { - for(x <- Seq(0, 1, 2, 0, 1, 1, 0, 0, 0)) { - poisson.sample().andReturn(x) - } - } - whenExecuting(poisson) { - val sampler = new PoissonSampler[Int](0.2) - sampler.rng = poisson - assert(sampler.sample(a.iterator).toList == List(2, 3, 3, 5, 6)) - } +import org.scalatest.{FunSuite, Matchers} + +class RandomSamplerSuite extends FunSuite with Matchers { + /** + * My statistical testing methodology is to run a Kolmogorov-Smirnov (KS) test + * between the random samplers and simple reference samplers (known to work correctly). + * The sampling gap sizes between chosen samples should show up as having the same + * distributions between test and reference, if things are working properly. That is, + * the KS test will fail to strongly reject the null hypothesis that the distributions of + * sampling gaps are the same. + * There are no actual KS tests implemented for scala (that I can find) - and so what I + * have done here is pre-compute "D" - the KS statistic - that corresponds to a "weak" + * p-value for a particular sample size. I can then test that my measured KS stats + * are less than D. Computing D-values is easy, and implemented below. + * + * I used the scipy 'kstwobign' distribution to pre-compute my D value: + * + * def ksdval(q=0.1, n=1000): + * en = np.sqrt(float(n) / 2.0) + * return stats.kstwobign.isf(float(q)) / (en + 0.12 + 0.11 / en) + * + * When comparing KS stats I take the median of a small number of independent test runs + * to compensate for the issue that any sampled statistic will show "false positive" with + * some probability. Even when two distributions are the same, they will register as + * different 10% of the time at a p-value of 0.1 + */ + + // This D value is the precomputed KS statistic for p-value 0.1, sample size 1000: + val sampleSize = 1000 + val D = 0.0544280747619 + + // I'm not a big fan of fixing seeds, but unit testing based on running statistical tests + // will always fail with some nonzero probability, so I'll fix the seed to prevent these + // tests from generating random failure noise in CI testing, etc. + val rngSeed: Random = RandomSampler.newDefaultRNG + rngSeed.setSeed(235711) + + // Reference implementation of sampling without replacement (bernoulli) + def sample[T](data: Iterator[T], f: Double): Iterator[T] = { + val rng: Random = RandomSampler.newDefaultRNG + rng.setSeed(rngSeed.nextLong) + data.filter(_ => (rng.nextDouble <= f)) + } + + // Reference implementation of sampling with replacement + def sampleWR[T](data: Iterator[T], f: Double): Iterator[T] = { + val rng = new PoissonDistribution(f) + rng.reseedRandomGenerator(rngSeed.nextLong) + data.flatMap { v => { + val rep = rng.sample() + if (rep == 0) Iterator.empty else Iterator.fill(rep)(v) + }} + } + + // Returns iterator over gap lengths between samples. + // This function assumes input data is integers sampled from the sequence of + // increasing integers: {0, 1, 2, ...}. This works because that is how I generate them, + // and the samplers preserve their input order + def gaps(data: Iterator[Int]): Iterator[Int] = { + data.sliding(2).withPartial(false).map { x => x(1) - x(0) } + } + + // Returns the cumulative distribution from a histogram + def cumulativeDist(hist: Array[Int]): Array[Double] = { + val n = hist.sum.toDouble + assert(n > 0.0) + hist.scanLeft(0)(_ + _).drop(1).map { _.toDouble / n } + } + + // Returns aligned cumulative distributions from two arrays of data + def cumulants(d1: Array[Int], d2: Array[Int], + ss: Int = sampleSize): (Array[Double], Array[Double]) = { + assert(math.min(d1.length, d2.length) > 0) + assert(math.min(d1.min, d2.min) >= 0) + val m = 1 + math.max(d1.max, d2.max) + val h1 = Array.fill[Int](m)(0) + val h2 = Array.fill[Int](m)(0) + for (v <- d1) { h1(v) += 1 } + for (v <- d2) { h2(v) += 1 } + assert(h1.sum == h2.sum) + assert(h1.sum == ss) + (cumulativeDist(h1), cumulativeDist(h2)) + } + + // Computes the Kolmogorov-Smirnov 'D' statistic from two cumulative distributions + def KSD(cdf1: Array[Double], cdf2: Array[Double]): Double = { + assert(cdf1.length == cdf2.length) + val n = cdf1.length + assert(n > 0) + assert(cdf1(n-1) == 1.0) + assert(cdf2(n-1) == 1.0) + cdf1.zip(cdf2).map { x => Math.abs(x._1 - x._2) }.max + } + + // Returns the median KS 'D' statistic between two samples, over (m) sampling trials + def medianKSD(data1: => Iterator[Int], data2: => Iterator[Int], m: Int = 5): Double = { + val t = Array.fill[Double](m) { + val (c1, c2) = cumulants(data1.take(sampleSize).toArray, + data2.take(sampleSize).toArray) + KSD(c1, c2) + }.sorted + // return the median KS statistic + t(m / 2) + } + + test("utilities") { + val s1 = Array(0, 1, 1, 0, 2) + val s2 = Array(1, 0, 3, 2, 1) + val (c1, c2) = cumulants(s1, s2, ss = 5) + c1 should be (Array(0.4, 0.8, 1.0, 1.0)) + c2 should be (Array(0.2, 0.6, 0.8, 1.0)) + KSD(c1, c2) should be (0.2 +- 0.000001) + KSD(c2, c1) should be (KSD(c1, c2)) + gaps(List(0, 1, 1, 2, 4, 11).iterator).toArray should be (Array(1, 0, 1, 2, 7)) + } + + test("sanity check medianKSD against references") { + var d: Double = 0.0 + + // should be statistically same, i.e. fail to reject null hypothesis strongly + d = medianKSD(gaps(sample(Iterator.from(0), 0.5)), gaps(sample(Iterator.from(0), 0.5))) + d should be < D + + // should be statistically different - null hypothesis will have high D value, + // corresponding to low p-value that rejects the null hypothesis + d = medianKSD(gaps(sample(Iterator.from(0), 0.4)), gaps(sample(Iterator.from(0), 0.5))) + d should be > D + + // same! + d = medianKSD(gaps(sampleWR(Iterator.from(0), 0.5)), gaps(sampleWR(Iterator.from(0), 0.5))) + d should be < D + + // different! + d = medianKSD(gaps(sampleWR(Iterator.from(0), 0.5)), gaps(sampleWR(Iterator.from(0), 0.6))) + d should be > D + } + + test("bernoulli sampling") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.5))) + d should be < D + + sampler = new BernoulliSampler[Int](0.7) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.7))) + d should be < D + + sampler = new BernoulliSampler[Int](0.9) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new BernoulliSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.6))) + d should be > D + } + + test("bernoulli sampling with gap sampling optimization") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler: RandomSampler[Int, Int] = new BernoulliSampler[Int](0.01) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.01))) + d should be < D + + sampler = new BernoulliSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.3))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new BernoulliSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.4))) + d should be > D + } + + test("bernoulli boundary cases") { + val data = (1 to 100).toArray + + var sampler = new BernoulliSampler[Int](0.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliSampler[Int](1.0) + sampler.sample(data.iterator).toArray should be (data) + + sampler = new BernoulliSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0)) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliSampler[Int](1.0 + (RandomSampler.roundingEpsilon / 2.0)) + sampler.sample(data.iterator).toArray should be (data) + } + + test("bernoulli data types") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler = new BernoulliSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + + // Array iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toArray.iterator)), + gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + // ArrayBuffer iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).to[ArrayBuffer].iterator)), + gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + // List iterator (non-indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toList.iterator)), + gaps(sample(Iterator.from(0), 0.1))) + d should be < D + } + + test("bernoulli clone") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d = 0.0 + var sampler = new BernoulliSampler[Int](0.1).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliSampler[Int](0.9).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + } + + test("bernoulli set seed") { + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler1 = new BernoulliSampler[Int](0.2) + var sampler2 = new BernoulliSampler[Int](0.2) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + + sampler1 = new BernoulliSampler[Int](0.8) + sampler2 = new BernoulliSampler[Int](0.8) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + } + + test("replacement sampling") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler = new PoissonSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.5))) + d should be < D + + sampler = new PoissonSampler[Int](0.7) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.7))) + d should be < D + + sampler = new PoissonSampler[Int](0.9) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.9))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new PoissonSampler[Int](0.5) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.6))) + d should be > D + } + + test("replacement sampling with gap sampling") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + + var sampler = new PoissonSampler[Int](0.01) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.01))) + d should be < D + + sampler = new PoissonSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + sampler = new PoissonSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.3))) + d should be < D + + // sampling at different frequencies should show up as statistically different: + sampler = new PoissonSampler[Int](0.3) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.4))) + d should be > D + } + + test("replacement boundary cases") { + val data = (1 to 100).toArray + + var sampler = new PoissonSampler[Int](0.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new PoissonSampler[Int](0.0 - (RandomSampler.roundingEpsilon / 2.0)) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + // sampling with replacement has no upper bound on sampling fraction + sampler = new PoissonSampler[Int](2.0) + sampler.sample(data.iterator).length should be > (data.length) + } + + test("replacement data types") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler = new PoissonSampler[Int](0.1) + sampler.setSeed(rngSeed.nextLong) + + // Array iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toArray.iterator)), + gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + // ArrayBuffer iterator (indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).to[ArrayBuffer].iterator)), + gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + // List iterator (non-indexable type) + d = medianKSD( + gaps(sampler.sample(Iterator.from(0).take(20*sampleSize).toList.iterator)), + gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + } + + test("replacement clone") { + // Tests expect maximum gap sampling fraction to be this value + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d = 0.0 + var sampler = new PoissonSampler[Int](0.1).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.1))) + d should be < D + + sampler = new PoissonSampler[Int](0.9).clone + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sampleWR(Iterator.from(0), 0.9))) + d should be < D + } + + test("replacement set seed") { + RandomSampler.defaultMaxGapSamplingFraction should be (0.4) + + var d: Double = 0.0 + var sampler1 = new PoissonSampler[Int](0.2) + var sampler2 = new PoissonSampler[Int](0.2) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + + sampler1 = new PoissonSampler[Int](0.8) + sampler2 = new PoissonSampler[Int](0.8) + + // distributions should be identical if seeds are set same + sampler1.setSeed(73) + sampler2.setSeed(73) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be (0.0) + + // should be different for different seeds + sampler1.setSeed(73) + sampler2.setSeed(37) + d = medianKSD(gaps(sampler1.sample(Iterator.from(0))), gaps(sampler2.sample(Iterator.from(0)))) + d should be > 0.0 + d should be < D + } + + test("bernoulli partitioning sampling") { + var d: Double = 0.0 + + var sampler = new BernoulliCellSampler[Int](0.1, 0.2) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.1))) + d should be < D + + sampler = new BernoulliCellSampler[Int](0.1, 0.2, true) + sampler.setSeed(rngSeed.nextLong) + d = medianKSD(gaps(sampler.sample(Iterator.from(0))), gaps(sample(Iterator.from(0), 0.9))) + d should be < D + } + + test("bernoulli partitioning boundary cases") { + val data = (1 to 100).toArray + val d = RandomSampler.roundingEpsilon / 2.0 + + var sampler = new BernoulliCellSampler[Int](0.0, 0.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](1.0, 1.0) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.0, 1.0) + sampler.sample(data.iterator).toArray should be (data) + + sampler = new BernoulliCellSampler[Int](0.0 - d, 1.0 + d) + sampler.sample(data.iterator).toArray should be (data) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5 - d) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + } + + test("bernoulli partitioning data") { + val seed = rngSeed.nextLong + val data = (1 to 100).toArray + + var sampler = new BernoulliCellSampler[Int](0.4, 0.6) + sampler.setSeed(seed) + val s1 = sampler.sample(data.iterator).toArray + s1.length should be > 0 + + sampler = new BernoulliCellSampler[Int](0.4, 0.6, true) + sampler.setSeed(seed) + val s2 = sampler.sample(data.iterator).toArray + s2.length should be > 0 + + (s1 ++ s2).sorted should be (data) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5) + sampler.sample(data.iterator).toArray should be (Array.empty[Int]) + + sampler = new BernoulliCellSampler[Int](0.5, 0.5, true) + sampler.sample(data.iterator).toArray should be (data) + } + + test("bernoulli partitioning clone") { + val seed = rngSeed.nextLong + val data = (1 to 100).toArray + val base = new BernoulliCellSampler[Int](0.35, 0.65) + + var sampler = base.clone + sampler.setSeed(seed) + val s1 = sampler.sample(data.iterator).toArray + s1.length should be > 0 + + sampler = base.cloneComplement + sampler.setSeed(seed) + val s2 = sampler.sample(data.iterator).toArray + s2.length should be > 0 + + (s1 ++ s2).sorted should be (data) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index b88e08bf148ae..9353351af72a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -26,7 +26,7 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD -import org.apache.spark.util.random.BernoulliSampler +import org.apache.spark.util.random.BernoulliCellSampler import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.storage.StorageLevel @@ -244,7 +244,7 @@ object MLUtils { def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Int): Array[(RDD[T], RDD[T])] = { val numFoldsF = numFolds.toFloat (1 to numFolds).map { fold => - val sampler = new BernoulliSampler[T]((fold - 1) / numFoldsF, fold / numFoldsF, + val sampler = new BernoulliCellSampler[T]((fold - 1) / numFoldsF, fold / numFoldsF, complement = false) val validation = new PartitionwiseSampledRDD(rdd, sampler, true, seed) val training = new PartitionwiseSampledRDD(rdd, sampler.cloneComplement(), true, seed) From d31517a3cd6f887cc66ffd3c8e0c12bace3a948d Mon Sep 17 00:00:00 2001 From: Anant Date: Thu, 30 Oct 2014 23:02:42 -0700 Subject: [PATCH 1155/1492] [SPARK-4108][SQL] Fixed usage of deprecated in sql/catalyst/types/datatypes Fixed usage of deprecated in sql/catalyst/types/datatypes to have versio...n parameter Author: Anant Closes #2970 from anantasty/SPARK-4108 and squashes the following commits: e92cb01 [Anant] Fixed usage of deprecated in sql/catalyst/types/datatypes to have version parameter --- .../scala/org/apache/spark/sql/catalyst/types/dataTypes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b9cf37d53ffd2..4e6e1166bfffb 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 @@ -76,7 +76,7 @@ object DataType { StructField(name, parseDataType(dataType), nullable) } - @deprecated("Use DataType.fromJson instead") + @deprecated("Use DataType.fromJson instead", "1.2.0") def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) private object CaseClassStringParser extends RegexParsers { From 58a6077e56f0f8f75fdd57d3aaad7485aeb07615 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 30 Oct 2014 23:59:46 -0700 Subject: [PATCH 1156/1492] [SPARK-4143] [SQL] Move inner class DeferredObjectAdapter to top level The class DeferredObjectAdapter is the inner class of HiveGenericUdf, which may cause some overhead in closure ser/de-ser. Move it to top level. Author: Cheng Hao Closes #3007 from chenghao-intel/move_deferred and squashes the following commits: 3a139b1 [Cheng Hao] Move inner class DeferredObjectAdapter to top level --- .../org/apache/spark/sql/hive/hiveUdfs.scala | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index aff4ddce92272..86f7eea5dfd69 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.analysis @@ -134,11 +135,19 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ } } +// Adapter from Catalyst ExpressionResult to Hive DeferredObject +private[hive] class DeferredObjectAdapter(oi: ObjectInspector) + extends DeferredObject with HiveInspectors { + private var func: () => Any = _ + def set(func: () => Any) { + this.func = func + } + override def prepare(i: Int) = {} + override def get(): AnyRef = wrap(func(), oi) +} + private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq[Expression]) extends HiveUdf with HiveInspectors { - - import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ - type UDFType = GenericUDF @transient @@ -161,16 +170,6 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq protected lazy val deferedObjects = argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] - // Adapter from Catalyst ExpressionResult to Hive DeferredObject - class DeferredObjectAdapter(oi: ObjectInspector) extends DeferredObject { - private var func: () => Any = _ - def set(func: () => Any) { - this.func = func - } - override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(func(), oi) - } - lazy val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { From acd4ac7c9a503445e27739708cf36e19119b8ddc Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 31 Oct 2014 08:43:06 -0500 Subject: [PATCH 1157/1492] SPARK-3837. Warn when YARN kills containers for exceeding memory limits I triggered the issue and verified the message gets printed on a pseudo-distributed cluster. Author: Sandy Ryza Closes #2744 from sryza/sandy-spark-3837 and squashes the following commits: 858a268 [Sandy Ryza] Review feedback c937f00 [Sandy Ryza] SPARK-3837. Warn when YARN kills containers for exceeding memory limits --- .../spark/deploy/yarn/YarnAllocator.scala | 30 ++++++++++++++-- .../deploy/yarn/YarnAllocatorSuite.scala | 34 +++++++++++++++++++ 2 files changed, 61 insertions(+), 3 deletions(-) create mode 100644 yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 7ae8ef237ff89..e6196194acbb4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.yarn import java.util.{List => JList} import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger +import java.util.regex.Pattern import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} @@ -375,12 +376,22 @@ private[yarn] abstract class YarnAllocator( logInfo("Completed container %s (state: %s, exit status: %s)".format( containerId, completedContainer.getState, - completedContainer.getExitStatus())) + completedContainer.getExitStatus)) // Hadoop 2.2.X added a ContainerExitStatus we should switch to use // there are some exit status' we shouldn't necessarily count against us, but for // now I think its ok as none of the containers are expected to exit - if (completedContainer.getExitStatus() != 0) { - logInfo("Container marked as failed: " + containerId) + if (completedContainer.getExitStatus == -103) { // vmem limit exceeded + logWarning(memLimitExceededLogMessage( + completedContainer.getDiagnostics, + VMEM_EXCEEDED_PATTERN)) + } else if (completedContainer.getExitStatus == -104) { // pmem limit exceeded + logWarning(memLimitExceededLogMessage( + completedContainer.getDiagnostics, + PMEM_EXCEEDED_PATTERN)) + } else if (completedContainer.getExitStatus != 0) { + logInfo("Container marked as failed: " + containerId + + ". Exit status: " + completedContainer.getExitStatus + + ". Diagnostics: " + completedContainer.getDiagnostics) numExecutorsFailed.incrementAndGet() } } @@ -428,6 +439,19 @@ private[yarn] abstract class YarnAllocator( } } + private val MEM_REGEX = "[0-9.]+ [KMG]B" + private val PMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") + private val VMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") + + def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { + val matcher = pattern.matcher(diagnostics) + val diag = if (matcher.find()) " " + matcher.group() + "." else "" + ("Container killed by YARN for exceeding memory limits." + diag + + " Consider boosting spark.yarn.executor.memoryOverhead.") + } + protected def allocatedContainersOnHost(host: String): Int = { var retval = 0 allocatedHostToContainersMap.synchronized { diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala new file mode 100644 index 0000000000000..9fff63fb25156 --- /dev/null +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import org.apache.spark.deploy.yarn.MemLimitLogger._ +import org.scalatest.FunSuite + +class YarnAllocatorSuite extends FunSuite { + test("memory exceeded diagnostic regexes") { + val diagnostics = + "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " + + "beyond physical memory limits. Current usage: 2.1 MB of 2 GB physical memory used; " + + "5.8 GB of 4.2 GB virtual memory used. Killing container." + val vmemMsg = memLimitExceededLogMessage(diagnostics, VMEM_EXCEEDED_PATTERN) + val pmemMsg = memLimitExceededLogMessage(diagnostics, PMEM_EXCEEDED_PATTERN) + assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used.")) + assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used.")) + } +} \ No newline at end of file From adb6415c1d65d466a10c50e8dc6cb3bf2805ebdf Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Fri, 31 Oct 2014 10:28:19 -0700 Subject: [PATCH 1158/1492] [SPARK-4016] Allow user to show/hide UI metrics. This commit adds a set of checkboxes to the stage detail page that the user can use to show additional task metrics, including the GC time, result serialization time, result fetch time, and scheduler delay. All of these metrics are now hidden by default. This allows advanced users to look at more detailed metrics, without distracting the average user. This change also cleans up the stage detail page so that metrics are shown in the same order in the summary table as in the task table, and updates the metrics in both tables such that they contain the same set of metrics. The ability to remember a user's preferences for which metrics should be shown has been filed as SPARK-4024. Here's what the stage detail page looks like by default: ![image](https://cloud.githubusercontent.com/assets/1108612/4744322/3ebe319e-5a2f-11e4-891f-c792be79caa2.png) and once a user clicks "Show additional metrics" (note that all the metrics get checked by default): ![image](https://cloud.githubusercontent.com/assets/1108612/4744332/51e5abda-5a2f-11e4-8994-d0d3705ee05d.png) cc shivaram andrewor14 Author: Kay Ousterhout Closes #2867 from kayousterhout/SPARK-checkboxes and squashes the following commits: 6015913 [Kay Ousterhout] Added comment 08dee73 [Kay Ousterhout] Josh's usability comments 0940d61 [Kay Ousterhout] Style updates based on Andrew's review ef05ccd [Kay Ousterhout] Added tooltips d7cfaaf [Kay Ousterhout] Made list of add'l metrics collapsible. 70c1fb5 [Kay Ousterhout] [SPARK-4016] Allow user to show/hide UI metrics. --- .../spark/ui/static/additional-metrics.js | 53 ++++ .../org/apache/spark/ui/static/table.js | 35 +++ .../org/apache/spark/ui/static/webui.css | 30 +++ .../scala/org/apache/spark/ui/ToolTips.scala | 12 + .../scala/org/apache/spark/ui/UIUtils.scala | 44 ++-- .../org/apache/spark/ui/jobs/StagePage.scala | 242 ++++++++++++------ .../spark/ui/jobs/TaskDetailsClassNames.scala | 29 +++ 7 files changed, 350 insertions(+), 95 deletions(-) create mode 100644 core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js create mode 100644 core/src/main/resources/org/apache/spark/ui/static/table.js create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js new file mode 100644 index 0000000000000..c5936b5038ac9 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -0,0 +1,53 @@ +/* + * 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. + */ + +/* Register functions to show/hide columns based on checkboxes. These need + * to be registered after the page loads. */ +$(function() { + $("span.expand-additional-metrics").click(function(){ + // Expand the list of additional metrics. + var additionalMetricsDiv = $(this).parent().find('.additional-metrics'); + $(additionalMetricsDiv).toggleClass('collapsed'); + + // Switch the class of the arrow from open to closed. + $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-open'); + $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed'); + + // If clicking caused the metrics to expand, automatically check all options for additional + // metrics (don't trigger a click when collapsing metrics, because it leads to weird + // toggling behavior). + if (!$(additionalMetricsDiv).hasClass('collapsed')) { + $(this).parent().find('input:checkbox:not(:checked)').trigger('click'); + } + }); + + $("input:checkbox:not(:checked)").each(function() { + var column = "table ." + $(this).attr("name"); + $(column).hide(); + }); + + $("input:checkbox").click(function() { + var column = "table ." + $(this).attr("name"); + $(column).toggle(); + stripeTables(); + }); + + // Trigger a click on the checkbox if a user clicks the label next to it. + $("span.additional-metric-title").click(function() { + $(this).parent().find('input:checkbox').trigger('click'); + }); +}); diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js new file mode 100644 index 0000000000000..32187ba6e8df0 --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -0,0 +1,35 @@ +/* + * 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. + */ + +/* Adds background colors to stripe table rows. This is necessary (instead of using css or the + * table striping provided by bootstrap) to appropriately stripe tables with hidden rows. */ +function stripeTables() { + $("table.table-striped-custom").each(function() { + $(this).find("tr:not(:hidden)").each(function (index) { + if (index % 2 == 1) { + $(this).css("background-color", "#f9f9f9"); + } else { + $(this).css("background-color", "#ffffff"); + } + }); + }); +} + +/* Stripe all tables after pages finish loading. */ +$(function() { + stripeTables(); +}); diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 152bde5f6994f..a2220e761ac98 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -120,7 +120,37 @@ pre { border: none; } +span.expand-additional-metrics { + cursor: pointer; +} + +span.additional-metric-title { + cursor: pointer; +} + +.additional-metrics.collapsed { + display: none; +} + .tooltip { font-weight: normal; } +.arrow-open { + width: 0; + height: 0; + border-left: 5px solid transparent; + border-right: 5px solid transparent; + border-top: 5px solid black; + float: left; + margin-top: 6px; +} + +.arrow-closed { + width: 0; + height: 0; + border-top: 5px solid transparent; + border-bottom: 5px solid transparent; + border-left: 5px solid black; + display: inline-block; +} diff --git a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala index 9ced9b8107ebf..f02904df31fcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/ToolTips.scala +++ b/core/src/main/scala/org/apache/spark/ui/ToolTips.scala @@ -31,4 +31,16 @@ private[spark] object ToolTips { val SHUFFLE_READ = """Bytes read from remote executors. Typically less than shuffle write bytes because this does not include shuffle data read locally.""" + + val GETTING_RESULT_TIME = + """Time that the driver spends fetching task results from workers. If this is large, consider + decreasing the amount of data returned from each task.""" + + val RESULT_SERIALIZATION_TIME = + """Time spent serializing the task result on the executor before sending it back to the + driver.""" + + val GC_TIME = + """Time that the executor spent paused for Java garbage collection while the task was + running.""" } 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 76714b1e6964f..3312671b6f885 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -20,13 +20,13 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.{Locale, Date} -import scala.xml.{Text, Node} +import scala.xml.{Node, Text} import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils extends Logging { - val TABLE_CLASS = "table table-bordered table-striped table-condensed sortable" + val TABLE_CLASS = "table table-bordered table-striped-custom table-condensed sortable" // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -160,6 +160,8 @@ private[spark] object UIUtils extends Logging { + + } /** Returns a spark page with correctly formatted headers */ @@ -240,7 +242,8 @@ private[spark] object UIUtils extends Logging { generateDataRow: T => Seq[Node], data: Iterable[T], fixedWidth: Boolean = false, - id: Option[String] = None): Seq[Node] = { + id: Option[String] = None, + headerClasses: Seq[String] = Seq.empty): Seq[Node] = { var listingTableClass = TABLE_CLASS if (fixedWidth) { @@ -248,20 +251,29 @@ private[spark] object UIUtils extends Logging { } val colWidth = 100.toDouble / headers.size val colWidthAttr = if (fixedWidth) colWidth + "%" else "" - val headerRow: Seq[Node] = { - // if none of the headers have "\n" in them - if (headers.forall(!_.contains("\n"))) { - // represent header as simple text - headers.map(h => ) + + def getClass(index: Int): String = { + if (index < headerClasses.size) { + headerClasses(index) } else { - // represent header text as list while respecting "\n" - headers.map { case h => - - } + "" + } + } + + val newlinesInHeader = headers.exists(_.contains("\n")) + def getHeaderContent(header: String): Seq[Node] = { + if (newlinesInHeader) { +
      + { header.split("\n").map { case t =>
    • {t}
    • } } +
    + } else { + Text(header) + } + } + + val headerRow: Seq[Node] = { + headers.view.zipWithIndex.map { x => + } }
    StructType org.apache.spark.sql.Row + StructType(fields)
    Note: fields is a Seq of StructFields. Also, two fields with the same name are not allowed. @@ -1267,7 +1289,7 @@ import org.apache.spark.sql._ All data types of Spark SQL are located in the package of `org.apache.spark.sql.api.java`. To access or create a data type, -please use factory methods provided in +please use factory methods provided in `org.apache.spark.sql.api.java.DataType`. @@ -1373,7 +1395,7 @@ please use factory methods provided in - - - + From 7d1a37239c50394025d9f16acf5dcd05cfbe7250 Mon Sep 17 00:00:00 2001 From: chesterxgchen Date: Wed, 17 Sep 2014 10:25:52 -0500 Subject: [PATCH 0720/1492] SPARK-3177 (on Master Branch) The JIRA and PR was original created for branch-1.1, and move to master branch now. Chester The Issue is due to that yarn-alpha and yarn have different APIs for certain class fields. In this particular case, the ClientBase using reflection to to address this issue, and we need to different way to test the ClientBase's method. Original ClientBaseSuite using getFieldValue() method to do this. But it doesn't work for yarn-alpha as the API returns an array of String instead of just String (which is the case for Yarn-stable API). To fix the test, I add a new method def getFieldValue2[A: ClassTag, A1: ClassTag, B](clazz: Class[_], field: String, defaults: => B) (mapTo: A => B)(mapTo1: A1 => B) : B = Try(clazz.getField(field)).map(_.get(null)).map { case v: A => mapTo(v) case v1: A1 => mapTo1(v1) case _ => defaults }.toOption.getOrElse(defaults) to handle the cases where the field type can be either type A or A1. In this new method the type A or A1 is pattern matched and corresponding mapTo function (mapTo or mapTo1) is used. Author: chesterxgchen Closes #2204 from chesterxgchen/SPARK-3177-master and squashes the following commits: e72a6ea [chesterxgchen] The Issue is due to that yarn-alpha and yarn have different APIs for certain class fields. In this particular case, the ClientBase using reflection to to address this issue, and we need to different way to test the ClientBase's method. Original ClientBaseSuite using getFieldValue() method to do this. But it doesn't work for yarn-alpha as the API returns an array of String instead of just String (which is the case for Yarn-stable API). --- .../spark/deploy/yarn/ClientBaseSuite.scala | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 5480eca7c832c..c3b7a2c8f02e5 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -38,6 +38,7 @@ import org.scalatest.Matchers import scala.collection.JavaConversions._ import scala.collection.mutable.{ HashMap => MutableHashMap } +import scala.reflect.ClassTag import scala.util.Try import org.apache.spark.{SparkException, SparkConf} @@ -200,9 +201,10 @@ class ClientBaseSuite extends FunSuite with Matchers { val knownDefMRAppCP: Seq[String] = - getFieldValue[String, Seq[String]](classOf[MRJobConfig], - "DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH", - Seq[String]())(a => a.split(",")) + getFieldValue2[String, Array[String], Seq[String]]( + classOf[MRJobConfig], + "DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH", + Seq[String]())(a => a.split(","))(a => a.toSeq) val knownYARNAppCP = Some(Seq("/known/yarn/path")) @@ -232,6 +234,17 @@ class ClientBaseSuite extends FunSuite with Matchers { def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = Try(clazz.getField(field)).map(_.get(null).asInstanceOf[A]).toOption.map(mapTo).getOrElse(defaults) + def getFieldValue2[A: ClassTag, A1: ClassTag, B]( + clazz: Class[_], + field: String, + defaults: => B)(mapTo: A => B)(mapTo1: A1 => B) : B = { + Try(clazz.getField(field)).map(_.get(null)).map { + case v: A => mapTo(v) + case v1: A1 => mapTo1(v1) + case _ => defaults + }.toOption.getOrElse(defaults) + } + private class DummyClient( val args: ClientArguments, val conf: Configuration, From 8fbd5f4a90f92e064aa057adbd3f8c58dd0087fa Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 17 Sep 2014 12:33:09 -0700 Subject: [PATCH 0721/1492] [Docs] minor grammar fix Author: Nicholas Chammas Closes #2430 from nchammas/patch-2 and squashes the following commits: d476bfb [Nicholas Chammas] [Docs] minor grammar fix --- CONTRIBUTING.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index c6b4aa5344757..b6c6b050fa331 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -8,5 +8,5 @@ submitting any copyrighted material via pull request, email, or other means you agree to license the material under the project's open source license and warrant that you have the legal authority to do so. -Please see [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) +Please see the [Contributing to Spark wiki page](https://cwiki.apache.org/SPARK/Contributing+to+Spark) for more information. From cbf983bb4a550ff26756ed7308fb03db42cffcff Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 17 Sep 2014 12:41:49 -0700 Subject: [PATCH 0722/1492] [SQL][DOCS] Improve table caching section Author: Michael Armbrust Closes #2434 from marmbrus/patch-1 and squashes the following commits: 67215be [Michael Armbrust] [SQL][DOCS] Improve table caching section --- docs/sql-programming-guide.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c498b41c43380..5212e19c41349 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -801,12 +801,12 @@ turning on some experimental options. ## Caching Data In Memory -Spark SQL can cache tables using an in-memory columnar format by calling `cacheTable("tableName")`. +Spark SQL can cache tables using an in-memory columnar format by calling `sqlContext.cacheTable("tableName")`. Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `uncacheTable("tableName")` to remove the table from memory. +memory usage and GC pressure. You can call `sqlContext.uncacheTable("tableName")` to remove the table from memory. -Note that if you call `cache` rather than `cacheTable`, tables will _not_ be cached using -the in-memory columnar format, and therefore `cacheTable` is strongly recommended for this use case. +Note that if you call `schemaRDD.cache()` rather than `sqlContext.cacheTable(...)`, tables will _not_ be cached using +the in-memory columnar format, and therefore `sqlContext.cacheTable(...)` is strongly recommended for this use case. Configuration of in-memory caching can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. From 5044e4953a1744593d83fe90628fb4893e5463f1 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 17 Sep 2014 12:44:44 -0700 Subject: [PATCH 0723/1492] [SPARK-1455] [SPARK-3534] [Build] When possible, run SQL tests only. If the only files changed are related to SQL, then only run the SQL tests. This patch includes some cosmetic/maintainability refactoring. I would be more than happy to undo some of these changes if they are inappropriate. We can accept this patch mostly as-is and address the immediate need documented in [SPARK-3534](https://issues.apache.org/jira/browse/SPARK-3534), or we can keep it open until a satisfactory solution along the lines [discussed here](https://issues.apache.org/jira/browse/SPARK-1455?focusedCommentId=14136424&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14136424) is reached. Note: I had to hack this patch up to test it locally, so what I'm submitting here and what I tested are technically different. Author: Nicholas Chammas Closes #2420 from nchammas/selective-testing and squashes the following commits: db3fa2d [Nicholas Chammas] diff against master! f9e23f6 [Nicholas Chammas] when possible, run SQL tests only --- dev/run-tests | 156 ++++++++++++++++++++++++++++++++++---------------- 1 file changed, 106 insertions(+), 50 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 79401213a7fa2..53148d23f385f 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,44 +21,73 @@ FWDIR="$(cd "`dirname $0`"/..; pwd)" cd "$FWDIR" -if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then - if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=1.0.4" - elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" - elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Dhadoop.version=2.2.0" - elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" +# Remove work directory +rm -rf ./work + +# Build against the right verison of Hadoop. +{ + if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then + if [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop1.0" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=1.0.4" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Dhadoop.version=2.2.0" + elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" + fi fi -fi -if [ -z "$SBT_MAVEN_PROFILES_ARGS" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" -fi + if [ -z "$SBT_MAVEN_PROFILES_ARGS" ]; then + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" + fi +} export SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Pkinesis-asl" -echo "SBT_MAVEN_PROFILES_ARGS=\"$SBT_MAVEN_PROFILES_ARGS\"" - -# Remove work directory -rm -rf ./work - -if test -x "$JAVA_HOME/bin/java"; then - declare java_cmd="$JAVA_HOME/bin/java" -else - declare java_cmd=java -fi -JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') -[ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." +# Determine Java path and version. +{ + if test -x "$JAVA_HOME/bin/java"; then + declare java_cmd="$JAVA_HOME/bin/java" + else + declare java_cmd=java + fi + + # We can't use sed -r -e due to OS X / BSD compatibility; hence, all the parentheses. + JAVA_VERSION=$( + $java_cmd -version 2>&1 \ + | grep -e "^java version" --max-count=1 \ + | sed "s/java version \"\(.*\)\.\(.*\)\.\(.*\)\"/\1\2/" + ) + + if [ "$JAVA_VERSION" -lt 18 ]; then + echo "[warn] Java 8 tests will not run because JDK version is < 1.8." + fi +} -# Partial solution for SPARK-1455. Only run Hive tests if there are sql changes. +# Only run Hive tests if there are sql changes. +# Partial solution for SPARK-1455. if [ -n "$AMPLAB_JENKINS" ]; then git fetch origin master:master - diffs=`git diff --name-only master | grep "^\(sql/\)\|\(bin/spark-sql\)\|\(sbin/start-thriftserver.sh\)"` - if [ -n "$diffs" ]; then - echo "Detected changes in SQL. Will run Hive test suite." + + sql_diffs=$( + git diff --name-only master \ + | grep -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh" + ) + + non_sql_diffs=$( + git diff --name-only master \ + | grep -v -e "^sql/" -e "^bin/spark-sql" -e "^sbin/start-thriftserver.sh" + ) + + if [ -n "$sql_diffs" ]; then + echo "[info] Detected changes in SQL. Will run Hive test suite." _RUN_SQL_TESTS=true + + if [ -z "$non_sql_diffs" ]; then + echo "[info] Detected no changes except in SQL. Will only run SQL tests." + _SQL_TESTS_ONLY=true + fi fi fi @@ -70,42 +99,69 @@ echo "" echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" -dev/check-license +./dev/check-license echo "" echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" -dev/lint-scala +./dev/lint-scala echo "" echo "=========================================================================" echo "Running Python style checks" echo "=========================================================================" -dev/lint-python +./dev/lint-python + +echo "" +echo "=========================================================================" +echo "Building Spark" +echo "=========================================================================" + +{ + # We always build with Hive because the PySpark Spark SQL tests need it. + BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + + echo "[info] Building Spark with these arguments: $BUILD_MVN_PROFILE_ARGS" + + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + #+ (either resolution or compilation) prompts the user for input either q, r, etc + #+ to quit or retry. This echo is there to make it not block. + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +} echo "" echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -# Build Spark; we always build with Hive because the PySpark Spark SQL tests need it. -# echo "q" is needed because sbt on encountering a build file with failure -# (either resolution or compilation) prompts the user for input either q, r, -# etc to quit or retry. This echo is there to make it not block. -BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive " -echo -e "q\n" | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly | \ - grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" - -# If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled: -if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" -fi -# echo "q" is needed because sbt on encountering a build file with failure -# (either resolution or compilation) prompts the user for input either q, r, -# etc to quit or retry. This echo is there to make it not block. -echo -e "q\n" | sbt/sbt $SBT_MAVEN_PROFILES_ARGS test | \ - grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +{ + # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. + if [ -n "$_RUN_SQL_TESTS" ]; then + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + fi + + if [ -n "$_SQL_TESTS_ONLY" ]; then + SBT_MAVEN_TEST_ARGS="catalyst/test sql/test hive/test" + else + SBT_MAVEN_TEST_ARGS="test" + fi + + echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS $SBT_MAVEN_TEST_ARGS" + + # NOTE: echo "q" is needed because sbt on encountering a build file with failure + #+ (either resolution or compilation) prompts the user for input either q, r, etc + #+ to quit or retry. This echo is there to make it not block. + # QUESTION: Why doesn't 'yes "q"' work? + # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + echo -e "q\n" \ + | sbt/sbt "$SBT_MAVEN_PROFILES_ARGS" "$SBT_MAVEN_TEST_ARGS" \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +} echo "" echo "=========================================================================" @@ -117,4 +173,4 @@ echo "" echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" -dev/mima +./dev/mima From b3830b28f8a70224d87c89d8491c514c4c191d23 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 17 Sep 2014 15:07:57 -0700 Subject: [PATCH 0724/1492] Docs: move HA subsections to a deeper indentation level Makes the table of contents read better Author: Andrew Ash Closes #2402 from ash211/docs/better-indentation and squashes the following commits: ea0e130 [Andrew Ash] Move HA subsections to a deeper indentation level --- docs/spark-standalone.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index c791c81f8bfd0..99a8e43a6b489 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -307,7 +307,7 @@ tight firewall settings. For a complete list of ports to configure, see the By default, standalone scheduling clusters are resilient to Worker failures (insofar as Spark itself is resilient to losing work by moving it to other workers). However, the scheduler uses a Master to make scheduling decisions, and this (by default) creates a single point of failure: if the Master crashes, no new applications can be created. In order to circumvent this, we have two high availability schemes, detailed below. -# Standby Masters with ZooKeeper +## Standby Masters with ZooKeeper **Overview** @@ -347,7 +347,7 @@ There's an important distinction to be made between "registering with a Master" Due to this property, new Masters can be created at any time, and the only thing you need to worry about is that _new_ applications and Workers can find it to register with in case it becomes the leader. Once registered, you're taken care of. -# Single-Node Recovery with Local File System +## Single-Node Recovery with Local File System **Overview** From 7fc3bb7c88a6bf5348d52ffee37a220a47c5a398 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 17 Sep 2014 15:14:04 -0700 Subject: [PATCH 0725/1492] [SPARK-3534] Fix expansion of testing arguments to sbt Testing arguments to `sbt` need to be passed as an array, not a single, long string. Fixes a bug introduced in #2420. Author: Nicholas Chammas Closes #2437 from nchammas/selective-testing and squashes the following commits: a9f9c1c [Nicholas Chammas] fix printing of sbt test arguments cf57cbf [Nicholas Chammas] fix sbt test arguments e33b978 [Nicholas Chammas] Merge pull request #2 from apache/master 0b47ca4 [Nicholas Chammas] Merge branch 'master' of github.com:nchammas/spark 8051486 [Nicholas Chammas] Merge pull request #1 from apache/master 03180a4 [Nicholas Chammas] Merge branch 'master' of github.com:nchammas/spark d4c5f43 [Nicholas Chammas] Merge pull request #6 from apache/master --- dev/run-tests | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/dev/run-tests b/dev/run-tests index 53148d23f385f..7c002160c3a4a 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -141,17 +141,20 @@ echo "=========================================================================" { # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. + # This must be a single argument, as it is. if [ -n "$_RUN_SQL_TESTS" ]; then SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" fi if [ -n "$_SQL_TESTS_ONLY" ]; then - SBT_MAVEN_TEST_ARGS="catalyst/test sql/test hive/test" + # This must be an array of individual arguments. Otherwise, having one long string + #+ will be interpreted as a single test, which doesn't work. + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test") else - SBT_MAVEN_TEST_ARGS="test" + SBT_MAVEN_TEST_ARGS=("test") fi - echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS $SBT_MAVEN_TEST_ARGS" + echo "[info] Running Spark tests with these arguments: $SBT_MAVEN_PROFILES_ARGS ${SBT_MAVEN_TEST_ARGS[@]}" # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc @@ -159,7 +162,7 @@ echo "=========================================================================" # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ - | sbt/sbt "$SBT_MAVEN_PROFILES_ARGS" "$SBT_MAVEN_TEST_ARGS" \ + | sbt/sbt "$SBT_MAVEN_PROFILES_ARGS" "${SBT_MAVEN_TEST_ARGS[@]}" \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } From cbc065039f5176acc49899462bfab2521da26701 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 17 Sep 2014 16:23:50 -0700 Subject: [PATCH 0726/1492] [SPARK-3571] Spark standalone cluster mode doesn't work. I think, this issue is caused by #1106 Author: Kousuke Saruta Closes #2436 from sarutak/SPARK-3571 and squashes the following commits: 7a4deea [Kousuke Saruta] Modified Master.scala to use numWorkersVisited and numWorkersAlive instead of stopPos 4e51e35 [Kousuke Saruta] Modified Master to prevent from 0 divide 4817ecd [Kousuke Saruta] Brushed up previous change 71e84b6 [Kousuke Saruta] Modified Master to enable schedule normally --- .../scala/org/apache/spark/deploy/master/Master.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2a3bd6ba0b9dc..432b552c58cd8 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 @@ -489,23 +489,24 @@ private[spark] class Master( // First schedule drivers, they take strict precedence over applications // Randomization helps balance drivers val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) - val aliveWorkerNum = shuffledAliveWorkers.size + val numWorkersAlive = shuffledAliveWorkers.size var curPos = 0 + for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers // We assign workers to each waiting driver in a round-robin fashion. For each driver, we // start from the last worker that was assigned a driver, and continue onwards until we have // explored all alive workers. - curPos = (curPos + 1) % aliveWorkerNum - val startPos = curPos var launched = false - while (curPos != startPos && !launched) { + var numWorkersVisited = 0 + while (numWorkersVisited < numWorkersAlive && !launched) { val worker = shuffledAliveWorkers(curPos) + numWorkersVisited += 1 if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { launchDriver(worker, driver) waitingDrivers -= driver launched = true } - curPos = (curPos + 1) % aliveWorkerNum + curPos = (curPos + 1) % numWorkersAlive } } From 6688a266f2cb84c2d43b8e4d27f710718c4cc4a0 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 17 Sep 2014 16:31:58 -0700 Subject: [PATCH 0727/1492] [SPARK-3564][WebUI] Display App ID on HistoryPage Author: Kousuke Saruta Closes #2424 from sarutak/display-appid-on-webui and squashes the following commits: 417fe90 [Kousuke Saruta] Added "App ID column" to HistoryPage --- .../scala/org/apache/spark/deploy/history/HistoryPage.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index c4ef8b63b0071..d25c29113d6da 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -67,6 +67,7 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { } private val appHeader = Seq( + "App ID", "App Name", "Started", "Completed", @@ -81,7 +82,8 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { val duration = UIUtils.formatDuration(info.endTime - info.startTime) val lastUpdated = UIUtils.formatDate(info.lastUpdated) - + + From 1147973f1c7713013c7c0ca414482b511a730475 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 17 Sep 2014 16:52:27 -0700 Subject: [PATCH 0728/1492] [SPARK-3567] appId field in SparkDeploySchedulerBackend should be volatile Author: Kousuke Saruta Closes #2428 from sarutak/appid-volatile-modification and squashes the following commits: c7d890d [Kousuke Saruta] Added volatile modifier to appId field in SparkDeploySchedulerBackend --- .../spark/scheduler/cluster/SparkDeploySchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 2f45d192e1d4d..5c5ecc8434d78 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 @@ -34,7 +34,7 @@ private[spark] class SparkDeploySchedulerBackend( var client: AppClient = null var stopping = false var shutdownCallback : (SparkDeploySchedulerBackend) => Unit = _ - var appId: String = _ + @volatile var appId: String = _ val registrationLock = new Object() var registrationDone = false From 3f169bfe3c322bf4344e13276dbbe34279b59ad0 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Wed, 17 Sep 2014 21:59:23 -0700 Subject: [PATCH 0729/1492] [SPARK-3565]Fix configuration item not consistent with document https://issues.apache.org/jira/browse/SPARK-3565 "spark.ports.maxRetries" should be "spark.port.maxRetries". Make the configuration keys in document and code consistent. Author: WangTaoTheTonic Closes #2427 from WangTaoTheTonic/fixPortRetries and squashes the following commits: c178813 [WangTaoTheTonic] Use blank lines trigger Jenkins 646f3fe [WangTaoTheTonic] also in SparkBuild.scala 3700dba [WangTaoTheTonic] Fix configuration item not consistent with document --- core/src/main/scala/org/apache/spark/util/Utils.scala | 6 +++--- .../scala/org/apache/spark/deploy/JsonProtocolSuite.scala | 2 ++ docs/configuration.md | 2 +- project/SparkBuild.scala | 2 +- 4 files changed, 7 insertions(+), 5 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 c76b7af18481d..ed063844323af 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1382,15 +1382,15 @@ private[spark] object Utils extends Logging { } /** - * Default number of retries in binding to a port. + * Default maximum number of retries when binding to a port before giving up. */ val portMaxRetries: Int = { if (sys.props.contains("spark.testing")) { // Set a higher number of retries for tests... - sys.props.get("spark.ports.maxRetries").map(_.toInt).getOrElse(100) + sys.props.get("spark.port.maxRetries").map(_.toInt).getOrElse(100) } else { Option(SparkEnv.get) - .flatMap(_.conf.getOption("spark.ports.maxRetries")) + .flatMap(_.conf.getOption("spark.port.maxRetries")) .map(_.toInt) .getOrElse(16) } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 2a58c6a40d8e4..3f1cd0752e766 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -115,11 +115,13 @@ class JsonProtocolSuite extends FunSuite { workerInfo.lastHeartbeat = JsonConstants.currTimeInMillis workerInfo } + def createExecutorRunner(): ExecutorRunner = { new ExecutorRunner("appId", 123, createAppDesc(), 4, 1234, null, "workerId", "host", new File("sparkHome"), new File("workDir"), "akka://worker", new SparkConf, ExecutorState.RUNNING) } + def createDriverRunner(): DriverRunner = { new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), createDriverDesc(), null, "akka://worker") diff --git a/docs/configuration.md b/docs/configuration.md index 99faf51c6f3db..a6dd7245e1552 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -657,7 +657,7 @@ Apart from these, the following properties are also available, and may be useful diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ab9f8ba120e83..12ac82293df76 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -336,7 +336,7 @@ object TestSettings { fork := true, javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", - javaOptions in Test += "-Dspark.ports.maxRetries=100", + javaOptions in Test += "-Dspark.port.maxRetries=100", javaOptions in Test += "-Dspark.ui.enabled=false", javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", javaOptions in Test ++= System.getProperties.filter(_._1 startsWith "spark") From 5547fa1ee98bf166061804bd64df4cb51a656a3f Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 17 Sep 2014 22:37:11 -0700 Subject: [PATCH 0730/1492] [SPARK-3534] Add hive-thriftserver to SQL tests Addresses the problem pointed out in [this comment](https://github.com/apache/spark/pull/2441#issuecomment-55990116). Author: Nicholas Chammas Closes #2442 from nchammas/patch-1 and squashes the following commits: 7e68b60 [Nicholas Chammas] [SPARK-3534] Add hive-thriftserver to SQL tests --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 7c002160c3a4a..5f6df17b509a3 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -149,7 +149,7 @@ echo "=========================================================================" if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string #+ will be interpreted as a single test, which doesn't work. - SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test") + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test") else SBT_MAVEN_TEST_ARGS=("test") fi From 6772afec2f57360bd886ba3c8487e6140869d8f0 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Wed, 17 Sep 2014 22:54:34 -0700 Subject: [PATCH 0731/1492] [Minor] rat exclude dependency-reduced-pom.xml Author: GuoQiang Li Closes #2326 from witgo/rat-excludes and squashes the following commits: 860904e [GuoQiang Li] rat exclude dependency-reduced-pom.xml --- .rat-excludes | 1 + 1 file changed, 1 insertion(+) diff --git a/.rat-excludes b/.rat-excludes index fb6323daf9211..1897ec8f747ca 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -58,3 +58,4 @@ dist/* .*iws logs .*scalastyle-output.xml +.*dependency-reduced-pom.xml From 3447d100900af15a7340a2f6a5430ffb6d9c6c23 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 18 Sep 2014 10:17:18 -0700 Subject: [PATCH 0732/1492] [SPARK-3547]Using a special exit code instead of 1 to represent ClassNotFoundExcepti... ...on As improvement of https://github.com/apache/spark/pull/1944, we should use more special exit code to represent ClassNotFoundException. Author: WangTaoTheTonic Closes #2421 from WangTaoTheTonic/classnotfoundExitCode and squashes the following commits: 645a22a [WangTaoTheTonic] Serveral typos to trigger Jenkins d6ae559 [WangTaoTheTonic] use 101 instead a2d6465 [WangTaoTheTonic] use 127 instead fbb232f [WangTaoTheTonic] Using a special exit code instead of 1 to represent ClassNotFoundException --- bin/spark-sql | 2 +- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../main/scala/org/apache/spark/network/nio/Connection.scala | 2 +- .../org/apache/spark/network/nio/ConnectionManager.scala | 4 ++-- sbin/start-thriftserver.sh | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/bin/spark-sql b/bin/spark-sql index ae096530cad04..9d66140b6aa17 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,7 +24,7 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -CLASS_NOT_FOUND_EXIT_STATUS=1 +CLASS_NOT_FOUND_EXIT_STATUS=101 # Figure out where Spark is installed FWDIR="$(cd "`dirname "$0"`"/..; pwd)" diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 5ed3575816a38..5d15af1326ef0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -54,7 +54,7 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" - private val CLASS_NOT_FOUND_EXIT_STATUS = 1 + private val CLASS_NOT_FOUND_EXIT_STATUS = 101 // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(-1) diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 74074a8dcbfff..18172d359cb35 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -460,7 +460,7 @@ private[spark] class ReceivingConnection( if (currId != null) currId else super.getRemoteConnectionManagerId() } - // The reciever's remote address is the local socket on remote side : which is NOT + // The receiver's remote address is the local socket on remote side : which is NOT // the connection manager id of the receiver. // We infer that from the messages we receive on the receiver socket. private def processConnectionManagerId(header: MessageChunkHeader) { diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 09d3ea306515b..5aa7e94943561 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -501,7 +501,7 @@ private[nio] class ConnectionManager( def changeConnectionKeyInterest(connection: Connection, ops: Int) { keyInterestChangeRequests += ((connection.key, ops)) - // so that registerations happen ! + // so that registrations happen ! wakeupSelector() } @@ -832,7 +832,7 @@ private[nio] class ConnectionManager( } /** - * Send a message and block until an acknowldgment is received or an error occurs. + * Send a message and block until an acknowledgment is received or an error occurs. * @param connectionManagerId the message's destination * @param message the message being sent * @return a Future that either returns the acknowledgment message or captures an exception. diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 4ce40fe750384..ba953e763faab 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,7 +27,7 @@ set -o posix FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -CLASS_NOT_FOUND_EXIT_STATUS=1 +CLASS_NOT_FOUND_EXIT_STATUS=101 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" From 3ad4176cf980591469997a8a612bf422c90f86fd Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 18 Sep 2014 10:30:17 -0700 Subject: [PATCH 0733/1492] SPARK-3579 Jekyll doc generation is different across environments. This patch makes some small changes to fix this problem: 1. We document specific versions of Jekyll/Kramdown to use that match those used when building the upstream docs. 2. We add a configuration for a property that for some reason varies across packages of Jekyll/Kramdown even with the same version. Author: Patrick Wendell Closes #2443 from pwendell/jekyll and squashes the following commits: 54ee2ab [Patrick Wendell] SPARK-3579 Jekyll doc generation is different across environments. --- docs/README.md | 16 ++++++++++------ docs/_config.yml | 5 +++++ 2 files changed, 15 insertions(+), 6 deletions(-) diff --git a/docs/README.md b/docs/README.md index fdc89d2eb767a..79708c3df9106 100644 --- a/docs/README.md +++ b/docs/README.md @@ -20,12 +20,16 @@ In this directory you will find textfiles formatted using Markdown, with an ".md read those text files directly if you want. Start with index.md. The markdown code can be compiled to HTML using the [Jekyll tool](http://jekyllrb.com). -To use the `jekyll` command, you will need to have Jekyll installed. -The easiest way to do this is via a Ruby Gem, see the -[jekyll installation instructions](http://jekyllrb.com/docs/installation). -If not already installed, you need to install `kramdown` and `jekyll-redirect-from` Gems -with `sudo gem install kramdown jekyll-redirect-from`. -Execute `jekyll build` from the `docs/` directory. Compiling the site with Jekyll will create a directory +`Jekyll` and a few dependencies must be installed for this to work. We recommend +installing via the Ruby Gem dependency manager. Since the exact HTML output +varies between versions of Jekyll and its dependencies, we list specific versions here +in some cases: + + $ sudo gem install jekyll -v 1.4.3 + $ sudo gem uninstall kramdown -v 1.4.1 + $ sudo gem install jekyll-redirect-from + +Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory called `_site` containing index.html as well as the rest of the compiled files. You can modify the default Jekyll build as follows: diff --git a/docs/_config.yml b/docs/_config.yml index d3ea2625c7448..7bc3a78e2d265 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -3,6 +3,11 @@ markdown: kramdown gems: - jekyll-redirect-from +# For some reason kramdown seems to behave differently on different +# OS/packages wrt encoding. So we hard code this config. +kramdown: + entity_output: numeric + # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. SPARK_VERSION: 1.0.0-SNAPSHOT From 6cab838b9803e3294c07bbf731c47154ec57afc0 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 18 Sep 2014 12:04:32 -0700 Subject: [PATCH 0734/1492] [SPARK-3566] [BUILD] .gitignore and .rat-excludes should consider Windows cmd file and Emacs' backup files Author: Kousuke Saruta Closes #2426 from sarutak/emacs-metafiles-ignore and squashes the following commits: a306020 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into emacs-metafiles-ignore 6a0a5eb [Kousuke Saruta] Added cmd file entry to .rat-excludes and .gitignore 897da63 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into emacs-metafiles-ignore 8cade06 [Kousuke Saruta] Modified .gitignore to ignore emacs lock file and backup file --- .gitignore | 3 +++ .rat-excludes | 1 + 2 files changed, 4 insertions(+) diff --git a/.gitignore b/.gitignore index a31bf7e0091f4..1bcd0165761ac 100644 --- a/.gitignore +++ b/.gitignore @@ -1,4 +1,6 @@ *~ +*.#* +*#*# *.swp *.ipr *.iml @@ -16,6 +18,7 @@ third_party/libmesos.so third_party/libmesos.dylib conf/java-opts conf/*.sh +conf/*.cmd conf/*.properties conf/*.conf conf/*.xml diff --git a/.rat-excludes b/.rat-excludes index 1897ec8f747ca..9fc99d7fca35d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -20,6 +20,7 @@ log4j.properties.template metrics.properties.template slaves spark-env.sh +spark-env.cmd spark-env.sh.template log4j-defaults.properties bootstrap-tooltip.js From 471e6a3a47bd4b94878798f6f6fc93e2e672efff Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 18 Sep 2014 12:07:24 -0700 Subject: [PATCH 0735/1492] [SPARK-3589][Minor]remove redundant code https://issues.apache.org/jira/browse/SPARK-3589 "export CLASSPATH" in spark-class is redundant since same variable is exported before. We could reuse defined value "isYarnCluster" in SparkSubmit.scala. Author: WangTaoTheTonic Closes #2445 from WangTaoTheTonic/removeRedundant and squashes the following commits: 6fb6872 [WangTaoTheTonic] remove redundant code --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 5d15af1326ef0..3dd1dd5b82fe8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -261,7 +261,7 @@ object SparkSubmit { } // In yarn-cluster mode, use yarn.Client as a wrapper around the user class - if (clusterManager == YARN && deployMode == CLUSTER) { + if (isYarnCluster) { childMainClass = "org.apache.spark.deploy.yarn.Client" if (args.primaryResource != SPARK_INTERNAL) { childArgs += ("--jar", args.primaryResource) From b3ed37e5bad15d56db90c2b25fe11c1f758d3a97 Mon Sep 17 00:00:00 2001 From: Victsm Date: Thu, 18 Sep 2014 15:58:14 -0700 Subject: [PATCH 0736/1492] [SPARK-3560] Fixed setting spark.jars system property in yarn-cluster mode Author: Victsm Author: Min Shen Closes #2449 from Victsm/SPARK-3560 and squashes the following commits: 918405a [Victsm] Removed the additional space 4502a2a [Min Shen] [SPARK-3560] Fixed setting spark.jars system property in yarn-cluster mode. (cherry picked from commit 832dff64ddb1240a4c8e22fcdc0e993cc8c808de) Signed-off-by: Andrew Or --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 3 ++- .../test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 3dd1dd5b82fe8..ec0324e24915a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -172,7 +172,7 @@ object SparkSubmit { // All cluster managers OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.master"), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.app.name"), - OptionAssigner(args.jars, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, sysProp = "spark.jars"), + OptionAssigner(args.jars, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.jars"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, sysProp = "spark.driver.memory"), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, @@ -205,6 +205,7 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), // Other options + OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 22b369a829418..0c324d8bdf6a4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -154,6 +154,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { sysProps("spark.app.name") should be ("beauty") sysProps("spark.shuffle.spill") should be ("false") sysProps("SPARK_SUBMIT") should be ("true") + sysProps.keys should not contain ("spark.jars") } test("handles YARN client mode") { From 9306297d1d888d0430f79b2133ee7377871a3a18 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 18 Sep 2014 17:49:28 -0700 Subject: [PATCH 0737/1492] [Minor Hot Fix] Move a line in SparkSubmit to the right place This was introduced in #2449 Author: Andrew Or Closes #2452 from andrewor14/standalone-hot-fix and squashes the following commits: d5190ca [Andrew Or] Put that line in the right place --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index ec0324e24915a..d132ecb3f9989 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -183,6 +183,7 @@ object SparkSubmit { sysProp = "spark.driver.extraLibraryPath"), // Standalone cluster only + OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.driverMemory, STANDALONE, CLUSTER, clOption = "--memory"), OptionAssigner(args.driverCores, STANDALONE, CLUSTER, clOption = "--cores"), @@ -205,7 +206,6 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), // Other options - OptionAssigner(args.jars, STANDALONE, CLUSTER, sysProp = "spark.jars"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, From e77fa81a61798c89d5a9b6c9dc067d11785254b7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 18 Sep 2014 18:11:48 -0700 Subject: [PATCH 0738/1492] [SPARK-3554] [PySpark] use broadcast automatically for large closure Py4j can not handle large string efficiently, so we should use broadcast for large closure automatically. (Broadcast use local filesystem to pass through data). Author: Davies Liu Closes #2417 from davies/command and squashes the following commits: fbf4e97 [Davies Liu] bugfix aefd508 [Davies Liu] use broadcast automatically for large closure --- python/pyspark/rdd.py | 4 ++++ python/pyspark/sql.py | 8 ++++++-- python/pyspark/tests.py | 6 ++++++ python/pyspark/worker.py | 4 +++- 4 files changed, 19 insertions(+), 3 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index cb09c191bed71..b43606b7304c5 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2061,8 +2061,12 @@ def _jrdd(self): self._jrdd_deserializer = NoOpSerializer() command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) + # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() pickled_command = ser.dumps(command) + if pickled_command > (1 << 20): # 1M + broadcast = self.ctx.broadcast(pickled_command) + pickled_command = ser.dumps(broadcast) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 8f6dbab240c7b..42a9920f10e6f 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -27,7 +27,7 @@ from array import array from operator import itemgetter -from pyspark.rdd import RDD, PipelinedRDD +from pyspark.rdd import RDD from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -975,7 +975,11 @@ def registerFunction(self, name, f, returnType=StringType()): command = (func, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) - pickled_command = CloudPickleSerializer().dumps(command) + ser = CloudPickleSerializer() + pickled_command = ser.dumps(command) + if pickled_command > (1 << 20): # 1M + broadcast = self._sc.broadcast(pickled_command) + pickled_command = ser.dumps(broadcast) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self._sc._pickled_broadcast_vars], self._sc._gateway._gateway_client) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0b3854347ad2e..7301966e48045 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -434,6 +434,12 @@ def test_large_broadcast(self): m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() self.assertEquals(N, m) + def test_large_closure(self): + N = 1000000 + data = [float(i) for i in xrange(N)] + m = self.sc.parallelize(range(1), 1).map(lambda x: len(data)).sum() + self.assertEquals(N, m) + def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) b = self.sc.parallelize(range(100, 105)) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 252176ac65fec..d6c06e2dbef62 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -77,10 +77,12 @@ def main(infile, outfile): _broadcastRegistry[bid] = Broadcast(bid, value) else: bid = - bid - 1 - _broadcastRegistry.remove(bid) + _broadcastRegistry.pop(bid) _accumulatorRegistry.clear() command = pickleSer._read_with_length(infile) + if isinstance(command, Broadcast): + command = pickleSer.loads(command.value) (func, deserializer, serializer) = command init_time = time.time() iterator = deserializer.load_stream(infile) From e76ef5cb8eed6b78fb722b3d6fbeb9466a0e3499 Mon Sep 17 00:00:00 2001 From: Burak Date: Thu, 18 Sep 2014 22:18:51 -0700 Subject: [PATCH 0739/1492] [SPARK-3418] Sparse Matrix support (CCS) and additional native BLAS operations added Local `SparseMatrix` support added in Compressed Column Storage (CCS) format in addition to Level-2 and Level-3 BLAS operations such as dgemv and dgemm respectively. BLAS doesn't support sparse matrix operations, therefore support for `SparseMatrix`-`DenseMatrix` multiplication and `SparseMatrix`-`DenseVector` implementations have been added. I will post performance comparisons in the comments momentarily. Author: Burak Closes #2294 from brkyvz/SPARK-3418 and squashes the following commits: 88814ed [Burak] Hopefully fixed MiMa this time 47e49d5 [Burak] really fixed MiMa issue f0bae57 [Burak] [SPARK-3418] Fixed MiMa compatibility issues (excluded from check) 4b7dbec [Burak] 9/17 comments addressed 7af2f83 [Burak] sealed traits Vector and Matrix d3a8a16 [Burak] [SPARK-3418] Squashed missing alpha bug. 421045f [Burak] [SPARK-3418] New code review comments addressed f35a161 [Burak] [SPARK-3418] Code review comments addressed and multiplication further optimized 2508577 [Burak] [SPARK-3418] Fixed one more style issue d16e8a0 [Burak] [SPARK-3418] Fixed style issues and added documentation for methods 204a3f7 [Burak] [SPARK-3418] Fixed failing Matrix unit test 6025297 [Burak] [SPARK-3418] Fixed Scala-style errors dc7be71 [Burak] [SPARK-3418][MLlib] Matrix unit tests expanded with indexing and updating d2d5851 [Burak] [SPARK-3418][MLlib] Sparse Matrix support and additional native BLAS operations added --- .../org/apache/spark/mllib/linalg/BLAS.scala | 330 +++++++++++++++++- .../apache/spark/mllib/linalg/Matrices.scala | 232 +++++++++++- .../apache/spark/mllib/linalg/Vectors.scala | 2 +- .../apache/spark/mllib/linalg/BLASSuite.scala | 111 ++++++ .../linalg/BreezeMatrixConversionSuite.scala | 24 +- .../spark/mllib/linalg/MatricesSuite.scala | 76 ++++ .../spark/mllib/util/TestingUtils.scala | 65 +++- project/MimaExcludes.scala | 4 +- 8 files changed, 834 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 70e23033c8754..54ee930d61003 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -18,13 +18,17 @@ package org.apache.spark.mllib.linalg import com.github.fommil.netlib.{BLAS => NetlibBLAS, F2jBLAS} +import com.github.fommil.netlib.BLAS.{getInstance => NativeBLAS} + +import org.apache.spark.Logging /** * BLAS routines for MLlib's vectors and matrices. */ -private[mllib] object BLAS extends Serializable { +private[mllib] object BLAS extends Serializable with Logging { @transient private var _f2jBLAS: NetlibBLAS = _ + @transient private var _nativeBLAS: NetlibBLAS = _ // For level-1 routines, we use Java implementation. private def f2jBLAS: NetlibBLAS = { @@ -197,4 +201,328 @@ private[mllib] object BLAS extends Serializable { throw new IllegalArgumentException(s"scal doesn't support vector type ${x.getClass}.") } } + + // For level-3 routines, we use the native BLAS. + private def nativeBLAS: NetlibBLAS = { + if (_nativeBLAS == null) { + _nativeBLAS = NativeBLAS + } + _nativeBLAS + } + + /** + * C := alpha * A * B + beta * C + * @param transA whether to use the transpose of matrix A (true), or A itself (false). + * @param transB whether to use the transpose of matrix B (true), or B itself (false). + * @param alpha a scalar to scale the multiplication A * B. + * @param A the matrix A that will be left multiplied to B. Size of m x k. + * @param B the matrix B that will be left multiplied by A. Size of k x n. + * @param beta a scalar that can be used to scale matrix C. + * @param C the resulting matrix C. Size of m x n. + */ + def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: Matrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + if (alpha == 0.0) { + logDebug("gemm: alpha is equal to 0. Returning C.") + } else { + A match { + case sparse: SparseMatrix => + gemm(transA, transB, alpha, sparse, B, beta, C) + case dense: DenseMatrix => + gemm(transA, transB, alpha, dense, B, beta, C) + case _ => + throw new IllegalArgumentException(s"gemm doesn't support matrix type ${A.getClass}.") + } + } + } + + /** + * C := alpha * A * B + beta * C + * + * @param alpha a scalar to scale the multiplication A * B. + * @param A the matrix A that will be left multiplied to B. Size of m x k. + * @param B the matrix B that will be left multiplied by A. Size of k x n. + * @param beta a scalar that can be used to scale matrix C. + * @param C the resulting matrix C. Size of m x n. + */ + def gemm( + alpha: Double, + A: Matrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + gemm(false, false, alpha, A, B, beta, C) + } + + /** + * C := alpha * A * B + beta * C + * For `DenseMatrix` A. + */ + private def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: DenseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val mA: Int = if (!transA) A.numRows else A.numCols + val nB: Int = if (!transB) B.numCols else B.numRows + val kA: Int = if (!transA) A.numCols else A.numRows + val kB: Int = if (!transB) B.numRows else B.numCols + val tAstr = if (!transA) "N" else "T" + val tBstr = if (!transB) "N" else "T" + + require(kA == kB, s"The columns of A don't match the rows of B. A: $kA, B: $kB") + require(mA == C.numRows, s"The rows of C don't match the rows of A. C: ${C.numRows}, A: $mA") + require(nB == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") + + nativeBLAS.dgemm(tAstr, tBstr, mA, nB, kA, alpha, A.values, A.numRows, B.values, B.numRows, + beta, C.values, C.numRows) + } + + /** + * C := alpha * A * B + beta * C + * For `SparseMatrix` A. + */ + private def gemm( + transA: Boolean, + transB: Boolean, + alpha: Double, + A: SparseMatrix, + B: DenseMatrix, + beta: Double, + C: DenseMatrix): Unit = { + val mA: Int = if (!transA) A.numRows else A.numCols + val nB: Int = if (!transB) B.numCols else B.numRows + val kA: Int = if (!transA) A.numCols else A.numRows + val kB: Int = if (!transB) B.numRows else B.numCols + + require(kA == kB, s"The columns of A don't match the rows of B. A: $kA, B: $kB") + require(mA == C.numRows, s"The rows of C don't match the rows of A. C: ${C.numRows}, A: $mA") + require(nB == C.numCols, + s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") + + val Avals = A.values + val Arows = if (!transA) A.rowIndices else A.colPtrs + val Acols = if (!transA) A.colPtrs else A.rowIndices + + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (transA){ + var colCounterForB = 0 + if (!transB) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var rowCounterForA = 0 + val Cstart = colCounterForB * mA + val Bstart = colCounterForB * kA + while (rowCounterForA < mA) { + var i = Arows(rowCounterForA) + val indEnd = Arows(rowCounterForA + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * B.values(Bstart + Acols(i)) + i += 1 + } + val Cindex = Cstart + rowCounterForA + C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + rowCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var rowCounter = 0 + val Cstart = colCounterForB * mA + while (rowCounter < mA) { + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + while (i < indEnd) { + sum += Avals(i) * B(colCounterForB, Acols(i)) + i += 1 + } + val Cindex = Cstart + rowCounter + C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + rowCounter += 1 + } + colCounterForB += 1 + } + } + } else { + // Scale matrix first if `beta` is not equal to 0.0 + if (beta != 0.0){ + f2jBLAS.dscal(C.values.length, beta, C.values, 1) + } + // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of + // B, and added to C. + var colCounterForB = 0 // the column to be updated in C + if (!transB) { // Expensive to put the check inside the loop + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Bstart = colCounterForB * kB + val Cstart = colCounterForB * mA + while (colCounterForA < kA) { + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val Bval = B.values(Bstart + colCounterForA) * alpha + while (i < indEnd){ + C.values(Cstart + Arows(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } else { + while (colCounterForB < nB) { + var colCounterForA = 0 // The column of A to multiply with the row of B + val Cstart = colCounterForB * mA + while (colCounterForA < kA){ + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val Bval = B(colCounterForB, colCounterForA) * alpha + while (i < indEnd){ + C.values(Cstart + Arows(i)) += Avals(i) * Bval + i += 1 + } + colCounterForA += 1 + } + colCounterForB += 1 + } + } + } + } + + /** + * y := alpha * A * x + beta * y + * @param trans whether to use the transpose of matrix A (true), or A itself (false). + * @param alpha a scalar to scale the multiplication A * x. + * @param A the matrix A that will be left multiplied to x. Size of m x n. + * @param x the vector x that will be left multiplied by A. Size of n x 1. + * @param beta a scalar that can be used to scale vector y. + * @param y the resulting vector y. Size of m x 1. + */ + def gemv( + trans: Boolean, + alpha: Double, + A: Matrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + + val mA: Int = if (!trans) A.numRows else A.numCols + val nx: Int = x.size + val nA: Int = if (!trans) A.numCols else A.numRows + + require(nA == nx, s"The columns of A don't match the number of elements of x. A: $nA, x: $nx") + require(mA == y.size, + s"The rows of A don't match the number of elements of y. A: $mA, y:${y.size}}") + if (alpha == 0.0) { + logDebug("gemv: alpha is equal to 0. Returning y.") + } else { + A match { + case sparse: SparseMatrix => + gemv(trans, alpha, sparse, x, beta, y) + case dense: DenseMatrix => + gemv(trans, alpha, dense, x, beta, y) + case _ => + throw new IllegalArgumentException(s"gemv doesn't support matrix type ${A.getClass}.") + } + } + } + + /** + * y := alpha * A * x + beta * y + * + * @param alpha a scalar to scale the multiplication A * x. + * @param A the matrix A that will be left multiplied to x. Size of m x n. + * @param x the vector x that will be left multiplied by A. Size of n x 1. + * @param beta a scalar that can be used to scale vector y. + * @param y the resulting vector y. Size of m x 1. + */ + def gemv( + alpha: Double, + A: Matrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + gemv(false, alpha, A, x, beta, y) + } + + /** + * y := alpha * A * x + beta * y + * For `DenseMatrix` A. + */ + private def gemv( + trans: Boolean, + alpha: Double, + A: DenseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + val tStrA = if (!trans) "N" else "T" + nativeBLAS.dgemv(tStrA, A.numRows, A.numCols, alpha, A.values, A.numRows, x.values, 1, beta, + y.values, 1) + } + + /** + * y := alpha * A * x + beta * y + * For `SparseMatrix` A. + */ + private def gemv( + trans: Boolean, + alpha: Double, + A: SparseMatrix, + x: DenseVector, + beta: Double, + y: DenseVector): Unit = { + + val mA: Int = if(!trans) A.numRows else A.numCols + val nA: Int = if(!trans) A.numCols else A.numRows + + val Avals = A.values + val Arows = if (!trans) A.rowIndices else A.colPtrs + val Acols = if (!trans) A.colPtrs else A.rowIndices + + // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices + if (trans){ + var rowCounter = 0 + while (rowCounter < mA){ + var i = Arows(rowCounter) + val indEnd = Arows(rowCounter + 1) + var sum = 0.0 + while(i < indEnd){ + sum += Avals(i) * x.values(Acols(i)) + i += 1 + } + y.values(rowCounter) = beta * y.values(rowCounter) + sum * alpha + rowCounter += 1 + } + } else { + // Scale vector first if `beta` is not equal to 0.0 + if (beta != 0.0){ + scal(beta, y) + } + // Perform matrix-vector multiplication and add to y + var colCounterForA = 0 + while (colCounterForA < nA){ + var i = Acols(colCounterForA) + val indEnd = Acols(colCounterForA + 1) + val xVal = x.values(colCounterForA) * alpha + while (i < indEnd){ + val rowIndex = Arows(i) + y.values(rowIndex) += Avals(i) * xVal + i += 1 + } + colCounterForA += 1 + } + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index b11ba5d30fbd3..5711532abcf80 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -17,12 +17,16 @@ package org.apache.spark.mllib.linalg -import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} +import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} + +import org.apache.spark.util.random.XORShiftRandom + +import java.util.Arrays /** * Trait for a local matrix. */ -trait Matrix extends Serializable { +sealed trait Matrix extends Serializable { /** Number of rows. */ def numRows: Int @@ -37,8 +41,46 @@ trait Matrix extends Serializable { private[mllib] def toBreeze: BM[Double] /** Gets the (i, j)-th element. */ - private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) + private[mllib] def apply(i: Int, j: Int): Double + + /** Return the index for the (i, j)-th element in the backing array. */ + private[mllib] def index(i: Int, j: Int): Int + + /** Update element at (i, j) */ + private[mllib] def update(i: Int, j: Int, v: Double): Unit + + /** Get a deep copy of the matrix. */ + def copy: Matrix + /** Convenience method for `Matrix`-`DenseMatrix` multiplication. */ + def multiply(y: DenseMatrix): DenseMatrix = { + val C: DenseMatrix = Matrices.zeros(numRows, y.numCols).asInstanceOf[DenseMatrix] + BLAS.gemm(false, false, 1.0, this, y, 0.0, C) + C + } + + /** Convenience method for `Matrix`-`DenseVector` multiplication. */ + def multiply(y: DenseVector): DenseVector = { + val output = new DenseVector(new Array[Double](numRows)) + BLAS.gemv(1.0, this, y, 0.0, output) + output + } + + /** Convenience method for `Matrix`^T^-`DenseMatrix` multiplication. */ + def transposeMultiply(y: DenseMatrix): DenseMatrix = { + val C: DenseMatrix = Matrices.zeros(numCols, y.numCols).asInstanceOf[DenseMatrix] + BLAS.gemm(true, false, 1.0, this, y, 0.0, C) + C + } + + /** Convenience method for `Matrix`^T^-`DenseVector` multiplication. */ + def transposeMultiply(y: DenseVector): DenseVector = { + val output = new DenseVector(new Array[Double](numCols)) + BLAS.gemv(true, 1.0, this, y, 0.0, output) + output + } + + /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() } @@ -59,11 +101,98 @@ trait Matrix extends Serializable { */ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) extends Matrix { - require(values.length == numRows * numCols) + require(values.length == numRows * numCols, "The number of values supplied doesn't match the " + + s"size of the matrix! values.length: ${values.length}, numRows * numCols: ${numRows * numCols}") override def toArray: Array[Double] = values - private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) + private[mllib] def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) + + private[mllib] def apply(i: Int): Double = values(i) + + private[mllib] def apply(i: Int, j: Int): Double = values(index(i, j)) + + private[mllib] def index(i: Int, j: Int): Int = i + numRows * j + + private[mllib] def update(i: Int, j: Int, v: Double): Unit = { + values(index(i, j)) = v + } + + override def copy = new DenseMatrix(numRows, numCols, values.clone()) +} + +/** + * Column-majored sparse matrix. + * The entry values are stored in Compressed Sparse Column (CSC) format. + * For example, the following matrix + * {{{ + * 1.0 0.0 4.0 + * 0.0 3.0 5.0 + * 2.0 0.0 6.0 + * }}} + * is stored as `values: [1.0, 2.0, 3.0, 4.0, 5.0, 6.0]`, + * `rowIndices=[0, 2, 1, 0, 1, 2]`, `colPointers=[0, 2, 3, 6]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry. They must be in strictly increasing order for each + * column + * @param values non-zero matrix entries in column major + */ +class SparseMatrix( + val numRows: Int, + val numCols: Int, + val colPtrs: Array[Int], + val rowIndices: Array[Int], + val values: Array[Double]) extends Matrix { + + require(values.length == rowIndices.length, "The number of row indices and values don't match! " + + s"values.length: ${values.length}, rowIndices.length: ${rowIndices.length}") + require(colPtrs.length == numCols + 1, "The length of the column indices should be the " + + s"number of columns + 1. Currently, colPointers.length: ${colPtrs.length}, " + + s"numCols: $numCols") + + override def toArray: Array[Double] = { + val arr = new Array[Double](numRows * numCols) + var j = 0 + while (j < numCols) { + var i = colPtrs(j) + val indEnd = colPtrs(j + 1) + val offset = j * numRows + while (i < indEnd) { + val rowIndex = rowIndices(i) + arr(offset + rowIndex) = values(i) + i += 1 + } + j += 1 + } + arr + } + + private[mllib] def toBreeze: BM[Double] = + new BSM[Double](values, numRows, numCols, colPtrs, rowIndices) + + private[mllib] def apply(i: Int, j: Int): Double = { + val ind = index(i, j) + if (ind < 0) 0.0 else values(ind) + } + + private[mllib] def index(i: Int, j: Int): Int = { + Arrays.binarySearch(rowIndices, colPtrs(j), colPtrs(j + 1), i) + } + + private[mllib] def update(i: Int, j: Int, v: Double): Unit = { + val ind = index(i, j) + if (ind == -1){ + throw new NoSuchElementException("The given row and column indices correspond to a zero " + + "value. Only non-zero elements in Sparse Matrices can be updated.") + } else { + values(index(i, j)) = v + } + } + + override def copy = new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values.clone()) } /** @@ -82,6 +211,24 @@ object Matrices { new DenseMatrix(numRows, numCols, values) } + /** + * Creates a column-majored sparse matrix in Compressed Sparse Column (CSC) format. + * + * @param numRows number of rows + * @param numCols number of columns + * @param colPtrs the index corresponding to the start of a new column + * @param rowIndices the row index of the entry + * @param values non-zero matrix entries in column major + */ + def sparse( + numRows: Int, + numCols: Int, + colPtrs: Array[Int], + rowIndices: Array[Int], + values: Array[Double]): Matrix = { + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + } + /** * Creates a Matrix instance from a breeze matrix. * @param breeze a breeze matrix @@ -93,9 +240,84 @@ object Matrices { require(dm.majorStride == dm.rows, "Do not support stride size different from the number of rows.") new DenseMatrix(dm.rows, dm.cols, dm.data) + case sm: BSM[Double] => + new SparseMatrix(sm.rows, sm.cols, sm.colPtrs, sm.rowIndices, sm.data) case _ => throw new UnsupportedOperationException( s"Do not support conversion from type ${breeze.getClass.getName}.") } } + + /** + * Generate a `DenseMatrix` consisting of zeros. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of zeros + */ + def zeros(numRows: Int, numCols: Int): Matrix = + new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) + + /** + * Generate a `DenseMatrix` consisting of ones. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values of ones + */ + def ones(numRows: Int, numCols: Int): Matrix = + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) + + /** + * Generate an Identity Matrix in `DenseMatrix` format. + * @param n number of rows and columns of the matrix + * @return `DenseMatrix` with size `n` x `n` and values of ones on the diagonal + */ + def eye(n: Int): Matrix = { + val identity = Matrices.zeros(n, n) + var i = 0 + while (i < n){ + identity.update(i, i, 1.0) + i += 1 + } + identity + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) + */ + def rand(numRows: Int, numCols: Int): Matrix = { + val rand = new XORShiftRandom + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextDouble())) + } + + /** + * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. + * @param numRows number of rows of the matrix + * @param numCols number of columns of the matrix + * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) + */ + def randn(numRows: Int, numCols: Int): Matrix = { + val rand = new XORShiftRandom + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextGaussian())) + } + + /** + * Generate a diagonal matrix in `DenseMatrix` format from the supplied values. + * @param vector a `Vector` tat will form the values on the diagonal of the matrix + * @return Square `DenseMatrix` with size `values.length` x `values.length` and `values` + * on the diagonal + */ + def diag(vector: Vector): Matrix = { + val n = vector.size + val matrix = Matrices.eye(n) + val values = vector.toArray + var i = 0 + while (i < n) { + matrix.update(i, i, values(i)) + i += 1 + } + matrix + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index a45781d12e41e..6af225b7f49f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -33,7 +33,7 @@ import org.apache.spark.SparkException * * Note: Users should not implement this interface. */ -trait Vector extends Serializable { +sealed trait Vector extends Serializable { /** * Size of the vector. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala index 1952e6734ecf7..5d70c914f14b0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BLASSuite.scala @@ -126,4 +126,115 @@ class BLASSuite extends FunSuite { } } } + + test("gemm") { + + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val B = new DenseMatrix(3, 2, Array(1.0, 0.0, 0.0, 0.0, 2.0, 1.0)) + val expected = new DenseMatrix(4, 2, Array(0.0, 1.0, 0.0, 0.0, 4.0, 0.0, 2.0, 3.0)) + + assert(dA multiply B ~== expected absTol 1e-15) + assert(sA multiply B ~== expected absTol 1e-15) + + val C1 = new DenseMatrix(4, 2, Array(1.0, 0.0, 2.0, 1.0, 0.0, 0.0, 1.0, 0.0)) + val C2 = C1.copy + val C3 = C1.copy + val C4 = C1.copy + val C5 = C1.copy + val C6 = C1.copy + val C7 = C1.copy + val C8 = C1.copy + val expected2 = new DenseMatrix(4, 2, Array(2.0, 1.0, 4.0, 2.0, 4.0, 0.0, 4.0, 3.0)) + val expected3 = new DenseMatrix(4, 2, Array(2.0, 2.0, 4.0, 2.0, 8.0, 0.0, 6.0, 6.0)) + + gemm(1.0, dA, B, 2.0, C1) + gemm(1.0, sA, B, 2.0, C2) + gemm(2.0, dA, B, 2.0, C3) + gemm(2.0, sA, B, 2.0, C4) + assert(C1 ~== expected2 absTol 1e-15) + assert(C2 ~== expected2 absTol 1e-15) + assert(C3 ~== expected3 absTol 1e-15) + assert(C4 ~== expected3 absTol 1e-15) + + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemm(true, false, 1.0, dA, B, 2.0, C1) + } + } + + val dAT = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sAT = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + assert(dAT transposeMultiply B ~== expected absTol 1e-15) + assert(sAT transposeMultiply B ~== expected absTol 1e-15) + + gemm(true, false, 1.0, dAT, B, 2.0, C5) + gemm(true, false, 1.0, sAT, B, 2.0, C6) + gemm(true, false, 2.0, dAT, B, 2.0, C7) + gemm(true, false, 2.0, sAT, B, 2.0, C8) + assert(C5 ~== expected2 absTol 1e-15) + assert(C6 ~== expected2 absTol 1e-15) + assert(C7 ~== expected3 absTol 1e-15) + assert(C8 ~== expected3 absTol 1e-15) + } + + test("gemv") { + + val dA = + new DenseMatrix(4, 3, Array(0.0, 1.0, 0.0, 0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 0.0, 3.0)) + val sA = new SparseMatrix(4, 3, Array(0, 1, 3, 4), Array(1, 0, 2, 3), Array(1.0, 2.0, 1.0, 3.0)) + + val x = new DenseVector(Array(1.0, 2.0, 3.0)) + val expected = new DenseVector(Array(4.0, 1.0, 2.0, 9.0)) + + assert(dA multiply x ~== expected absTol 1e-15) + assert(sA multiply x ~== expected absTol 1e-15) + + val y1 = new DenseVector(Array(1.0, 3.0, 1.0, 0.0)) + val y2 = y1.copy + val y3 = y1.copy + val y4 = y1.copy + val y5 = y1.copy + val y6 = y1.copy + val y7 = y1.copy + val y8 = y1.copy + val expected2 = new DenseVector(Array(6.0, 7.0, 4.0, 9.0)) + val expected3 = new DenseVector(Array(10.0, 8.0, 6.0, 18.0)) + + gemv(1.0, dA, x, 2.0, y1) + gemv(1.0, sA, x, 2.0, y2) + gemv(2.0, dA, x, 2.0, y3) + gemv(2.0, sA, x, 2.0, y4) + assert(y1 ~== expected2 absTol 1e-15) + assert(y2 ~== expected2 absTol 1e-15) + assert(y3 ~== expected3 absTol 1e-15) + assert(y4 ~== expected3 absTol 1e-15) + withClue("columns of A don't match the rows of B") { + intercept[Exception] { + gemv(true, 1.0, dA, x, 2.0, y1) + } + } + + val dAT = + new DenseMatrix(3, 4, Array(0.0, 2.0, 0.0, 1.0, 0.0, 0.0, 0.0, 1.0, 0.0, 0.0, 0.0, 3.0)) + val sAT = + new SparseMatrix(3, 4, Array(0, 1, 2, 3, 4), Array(1, 0, 1, 2), Array(2.0, 1.0, 1.0, 3.0)) + + assert(dAT transposeMultiply x ~== expected absTol 1e-15) + assert(sAT transposeMultiply x ~== expected absTol 1e-15) + + gemv(true, 1.0, dAT, x, 2.0, y5) + gemv(true, 1.0, sAT, x, 2.0, y6) + gemv(true, 2.0, dAT, x, 2.0, y7) + gemv(true, 2.0, sAT, x, 2.0, y8) + assert(y5 ~== expected2 absTol 1e-15) + assert(y6 ~== expected2 absTol 1e-15) + assert(y7 ~== expected3 absTol 1e-15) + assert(y8 ~== expected3 absTol 1e-15) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index 82d49c76ed02b..73a6d3a27d868 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg import org.scalatest.FunSuite -import breeze.linalg.{DenseMatrix => BDM} +import breeze.linalg.{DenseMatrix => BDM, CSCMatrix => BSM} class BreezeMatrixConversionSuite extends FunSuite { test("dense matrix to breeze") { @@ -37,4 +37,26 @@ class BreezeMatrixConversionSuite extends FunSuite { assert(mat.numCols === breeze.cols) assert(mat.values.eq(breeze.data), "should not copy data") } + + test("sparse matrix to breeze") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(3, 2, colPtrs, rowIndices, values) + val breeze = mat.toBreeze.asInstanceOf[BSM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[SparseMatrix].values), "should not copy data") + } + + test("sparse breeze matrix to sparse matrix") { + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val breeze = new BSM[Double](values, 3, 2, colPtrs, rowIndices) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[SparseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 9c66b4db9f16b..5f8b8c4b72697 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -36,4 +36,80 @@ class MatricesSuite extends FunSuite { Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) } } + + test("sparse matrix construction") { + val m = 3 + val n = 2 + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val mat = Matrices.sparse(m, n, colPtrs, rowIndices, values).asInstanceOf[SparseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.colPtrs.eq(colPtrs), "should not copy data") + assert(mat.rowIndices.eq(rowIndices), "should not copy data") + } + + test("sparse matrix construction with wrong number of elements") { + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1), Array(1, 2, 1), Array(0.0, 1.0, 2.0)) + } + + intercept[IllegalArgumentException] { + Matrices.sparse(3, 2, Array(0, 1, 2), Array(1, 2), Array(0.0, 1.0, 2.0)) + } + } + + test("matrix copies are deep copies") { + val m = 3 + val n = 2 + + val denseMat = Matrices.dense(m, n, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val denseCopy = denseMat.copy + + assert(!denseMat.toArray.eq(denseCopy.toArray)) + + val values = Array(1.0, 2.0, 4.0, 5.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 1, 2) + val sparseMat = Matrices.sparse(m, n, colPtrs, rowIndices, values) + val sparseCopy = sparseMat.copy + + assert(!sparseMat.toArray.eq(sparseCopy.toArray)) + } + + test("matrix indexing and updating") { + val m = 3 + val n = 2 + val allValues = Array(0.0, 1.0, 2.0, 3.0, 4.0, 0.0) + + val denseMat = new DenseMatrix(m, n, allValues) + + assert(denseMat(0, 1) === 3.0) + assert(denseMat(0, 1) === denseMat.values(3)) + assert(denseMat(0, 1) === denseMat(3)) + assert(denseMat(0, 0) === 0.0) + + denseMat.update(0, 0, 10.0) + assert(denseMat(0, 0) === 10.0) + assert(denseMat.values(0) === 10.0) + + val sparseValues = Array(1.0, 2.0, 3.0, 4.0) + val colPtrs = Array(0, 2, 4) + val rowIndices = Array(1, 2, 0, 1) + val sparseMat = new SparseMatrix(m, n, colPtrs, rowIndices, sparseValues) + + assert(sparseMat(0, 1) === 3.0) + assert(sparseMat(0, 1) === sparseMat.values(2)) + assert(sparseMat(0, 0) === 0.0) + + intercept[NoSuchElementException] { + sparseMat.update(0, 0, 10.0) + } + + sparseMat.update(0, 1, 10.0) + assert(sparseMat(0, 1) === 10.0) + assert(sparseMat.values(2) === 10.0) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index 29cc42d8cbea7..30b906aaa3ba4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.util -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{Matrix, Vector} import org.scalatest.exceptions.TestFailedException object TestingUtils { @@ -169,4 +169,67 @@ object TestingUtils { override def toString = x.toString } + case class CompareMatrixRightSide( + fun: (Matrix, Matrix, Double) => Boolean, y: Matrix, eps: Double, method: String) + + /** + * Implicit class for comparing two matrices using relative tolerance or absolute tolerance. + */ + implicit class MatrixWithAlmostEquals(val x: Matrix) { + + /** + * When the difference of two vectors are within eps, returns true; otherwise, returns false. + */ + def ~=(r: CompareMatrixRightSide): Boolean = r.fun(x, r.y, r.eps) + + /** + * When the difference of two vectors are within eps, returns false; otherwise, returns true. + */ + def !~=(r: CompareMatrixRightSide): Boolean = !r.fun(x, r.y, r.eps) + + /** + * Throws exception when the difference of two vectors are NOT within eps; + * otherwise, returns true. + */ + def ~==(r: CompareMatrixRightSide): Boolean = { + if (!r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Expected \n$x\n and \n${r.y}\n to be within ${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Throws exception when the difference of two matrices are within eps; otherwise, returns true. + */ + def !~==(r: CompareMatrixRightSide): Boolean = { + if (r.fun(x, r.y, r.eps)) { + throw new TestFailedException( + s"Did not expect \n$x\n and \n${r.y}\n to be within " + + "${r.eps}${r.method} for all elements.", 0) + } + true + } + + /** + * Comparison using absolute tolerance. + */ + def absTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 absTol eps) + }, x, eps, ABS_TOL_MSG) + + /** + * Comparison using relative tolerance. Note that comparing against sparse vector + * with elements having value of zero will raise exception because it involves with + * comparing against zero. + */ + def relTol(eps: Double): CompareMatrixRightSide = CompareMatrixRightSide( + (x: Matrix, y: Matrix, eps: Double) => { + x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) + }, x, eps, REL_TOL_MSG) + + override def toString = x.toString + } + } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2f1e05dfcc7b1..3280e662fa0b1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -37,7 +37,9 @@ object MimaExcludes { Seq( MimaBuild.excludeSparkPackage("deploy"), MimaBuild.excludeSparkPackage("graphx") - ) + ) ++ + MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ + MimaBuild.excludeSparkClass("mllib.linalg.Vector") case v if v.startsWith("1.1") => Seq( From 3bbbdd8180cf316c6f8dde0e879410b6b29f8cc3 Mon Sep 17 00:00:00 2001 From: Larry Xiao Date: Thu, 18 Sep 2014 23:32:32 -0700 Subject: [PATCH 0740/1492] [SPARK-2062][GraphX] VertexRDD.apply does not use the mergeFunc VertexRDD.apply had a bug where it ignored the merge function for duplicate vertices and instead used whichever vertex attribute occurred first. This commit fixes the bug by passing the merge function through to ShippableVertexPartition.apply, which merges any duplicates using the merge function and then fills in missing vertices using the specified default vertex attribute. This commit also adds a unit test for VertexRDD.apply. Author: Larry Xiao Author: Blie Arkansol Author: Ankur Dave Closes #1903 from larryxiao/2062 and squashes the following commits: 625aa9d [Blie Arkansol] Merge pull request #1 from ankurdave/SPARK-2062 476770b [Ankur Dave] ShippableVertexPartition.initFrom: Don't run mergeFunc on default values 614059f [Larry Xiao] doc update: note about the default null value vertices construction dfdb3c9 [Larry Xiao] minor fix 1c70366 [Larry Xiao] scalastyle check: wrap line, parameter list indent 4 spaces e4ca697 [Larry Xiao] [TEST] VertexRDD.apply mergeFunc 6a35ea8 [Larry Xiao] [TEST] VertexRDD.apply mergeFunc 4fbc29c [Blie Arkansol] undo unnecessary change efae765 [Larry Xiao] fix mistakes: should be able to call with or without mergeFunc b2422f9 [Larry Xiao] Merge branch '2062' of github.com:larryxiao/spark into 2062 52dc7f7 [Larry Xiao] pass mergeFunc to VertexPartitionBase, where merge is handled 581e9ee [Larry Xiao] TODO: VertexRDDSuite 20d80a3 [Larry Xiao] [SPARK-2062][GraphX] VertexRDD.apply does not use the mergeFunc --- .../org/apache/spark/graphx/VertexRDD.scala | 4 +-- .../impl/ShippableVertexPartition.scala | 28 +++++++++++++++---- .../apache/spark/graphx/VertexRDDSuite.scala | 11 ++++++++ 3 files changed, 36 insertions(+), 7 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 04fbc9dbab8d1..2c8b245955d12 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -392,7 +392,7 @@ object VertexRDD { */ def apply[VD: ClassTag]( vertices: RDD[(VertexId, VD)], edges: EdgeRDD[_, _], defaultVal: VD): VertexRDD[VD] = { - VertexRDD(vertices, edges, defaultVal, (a, b) => b) + VertexRDD(vertices, edges, defaultVal, (a, b) => a) } /** @@ -419,7 +419,7 @@ object VertexRDD { (vertexIter, routingTableIter) => val routingTable = if (routingTableIter.hasNext) routingTableIter.next() else RoutingTablePartition.empty - Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal)) + Iterator(ShippableVertexPartition(vertexIter, routingTable, defaultVal, mergeFunc)) } new VertexRDD(vertexPartitions) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index dca54b8a7da86..5412d720475dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -36,7 +36,7 @@ private[graphx] object ShippableVertexPartition { /** Construct a `ShippableVertexPartition` from the given vertices without any routing table. */ def apply[VD: ClassTag](iter: Iterator[(VertexId, VD)]): ShippableVertexPartition[VD] = - apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD]) + apply(iter, RoutingTablePartition.empty, null.asInstanceOf[VD], (a, b) => a) /** * Construct a `ShippableVertexPartition` from the given vertices with the specified routing @@ -44,10 +44,28 @@ object ShippableVertexPartition { */ def apply[VD: ClassTag]( iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD) - : ShippableVertexPartition[VD] = { - val fullIter = iter ++ routingTable.iterator.map(vid => (vid, defaultVal)) - val (index, values, mask) = VertexPartitionBase.initFrom(fullIter, (a: VD, b: VD) => a) - new ShippableVertexPartition(index, values, mask, routingTable) + : ShippableVertexPartition[VD] = + apply(iter, routingTable, defaultVal, (a, b) => a) + + /** + * Construct a `ShippableVertexPartition` from the given vertices with the specified routing + * table, filling in missing vertices mentioned in the routing table using `defaultVal`, + * and merging duplicate vertex atrribute with mergeFunc. + */ + def apply[VD: ClassTag]( + iter: Iterator[(VertexId, VD)], routingTable: RoutingTablePartition, defaultVal: VD, + mergeFunc: (VD, VD) => VD): ShippableVertexPartition[VD] = { + val map = new GraphXPrimitiveKeyOpenHashMap[VertexId, VD] + // Merge the given vertices using mergeFunc + iter.foreach { pair => + map.setMerge(pair._1, pair._2, mergeFunc) + } + // Fill in missing vertices mentioned in the routing table + routingTable.iterator.foreach { vid => + map.changeValue(vid, defaultVal, identity) + } + + new ShippableVertexPartition(map.keySet, map._values, map.keySet.getBitSet, routingTable) } import scala.language.implicitConversions diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index cc86bafd2d644..42d3f21dbae98 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -99,4 +99,15 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { } } + test("mergeFunc") { + // test to see if the mergeFunc is working correctly + withSpark { sc => + val verts = sc.parallelize(List((0L, 0), (1L, 1), (1L, 2), (2L, 3), (2L, 3), (2L, 3))) + val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) + val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) + // test merge function + assert(rdd.collect.toSet == Set((0L, 0), (1L, 3), (2L, 9))) + } + } + } From a48956f5825d2255736eee50de79fba79bcb7e39 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 19 Sep 2014 10:49:42 -0700 Subject: [PATCH 0741/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #726 (close requested by 'pwendell') Closes #151 (close requested by 'pwendell') From be0c7563ea001a59469dbba219d2a8ef5785afa3 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Fri, 19 Sep 2014 14:31:50 -0700 Subject: [PATCH 0742/1492] [SPARK-1701] Clarify slice vs partition in the programming guide This is a partial solution to SPARK-1701, only addressing the documentation confusion. Additional work can be to actually change the numSlices parameter name across languages, with care required for scala & python to maintain backward compatibility for named parameters. Author: Matthew Farrellee Closes #2305 from mattf/SPARK-1701 and squashes the following commits: c0af05d [Matthew Farrellee] Further tweak 06f80fc [Matthew Farrellee] Wording tweak from Josh Rosen's review 7b045e0 [Matthew Farrellee] [SPARK-1701] Clarify slice vs partition in the programming guide --- docs/programming-guide.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 624cc744dfd51..01d378af574b5 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -286,7 +286,7 @@ We describe operations on distributed datasets later on. -One important parameter for parallel collections is the number of *slices* to cut the dataset into. Spark will run one task for each slice of the cluster. Typically you want 2-4 slices for each CPU in your cluster. Normally, Spark tries to set the number of slices automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). +One important parameter for parallel collections is the number of *partitions* to cut the dataset into. Spark will run one task for each partition of the cluster. Typically you want 2-4 partitions for each CPU in your cluster. Normally, Spark tries to set the number of partitions automatically based on your cluster. However, you can also set it manually by passing it as a second parameter to `parallelize` (e.g. `sc.parallelize(data, 10)`). Note: some places in the code use the term slices (a synonym for partitions) to maintain backward compatibility. ## External Datasets @@ -311,7 +311,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Scala API also supports several other data formats: @@ -343,7 +343,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Java API also supports several other data formats: @@ -375,7 +375,7 @@ Some notes on reading files with Spark: * All of Spark's file-based input methods, including `textFile`, support running on directories, compressed files, and wildcards as well. For example, you can use `textFile("/my/directory")`, `textFile("/my/directory/*.txt")`, and `textFile("/my/directory/*.gz")`. -* The `textFile` method also takes an optional second argument for controlling the number of slices of the file. By default, Spark creates one slice for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of slices by passing a larger value. Note that you cannot have fewer slices than blocks. +* The `textFile` method also takes an optional second argument for controlling the number of partitions of the file. By default, Spark creates one partition for each block of the file (blocks being 64MB by default in HDFS), but you can also ask for a higher number of partitions by passing a larger value. Note that you cannot have fewer partitions than blocks. Apart from text files, Spark's Python API also supports several other data formats: From a03e5b81e91d9d792b6a2e01d1505394ea303dd8 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Fri, 19 Sep 2014 14:35:22 -0700 Subject: [PATCH 0743/1492] [SPARK-1701] [PySpark] remove slice terminology from python examples Author: Matthew Farrellee Closes #2304 from mattf/SPARK-1701-partition-over-slice-for-python-examples and squashes the following commits: 928a581 [Matthew Farrellee] [SPARK-1701] [PySpark] remove slice terminology from python examples --- examples/src/main/python/als.py | 12 ++++++------ examples/src/main/python/pi.py | 8 ++++---- examples/src/main/python/transitive_closure.py | 6 +++--- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 5b1fa4d997eeb..70b6146e39a87 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -54,7 +54,7 @@ def update(i, vec, mat, ratings): if __name__ == "__main__": """ - Usage: als [M] [U] [F] [iterations] [slices]" + Usage: als [M] [U] [F] [iterations] [partitions]" """ print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an @@ -66,10 +66,10 @@ def update(i, vec, mat, ratings): U = int(sys.argv[2]) if len(sys.argv) > 2 else 500 F = int(sys.argv[3]) if len(sys.argv) > 3 else 10 ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 - slices = int(sys.argv[5]) if len(sys.argv) > 5 else 2 + partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2 - print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \ - (M, U, F, ITERATIONS, slices) + print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \ + (M, U, F, ITERATIONS, partitions) R = matrix(rand(M, F)) * matrix(rand(U, F).T) ms = matrix(rand(M, F)) @@ -80,7 +80,7 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) for i in range(ITERATIONS): - ms = sc.parallelize(range(M), slices) \ + ms = sc.parallelize(range(M), partitions) \ .map(lambda x: update(x, msb.value[x, :], usb.value, Rb.value)) \ .collect() # collect() returns a list, so array ends up being @@ -88,7 +88,7 @@ def update(i, vec, mat, ratings): ms = matrix(np.array(ms)[:, :, 0]) msb = sc.broadcast(ms) - us = sc.parallelize(range(U), slices) \ + us = sc.parallelize(range(U), partitions) \ .map(lambda x: update(x, usb.value[x, :], msb.value, Rb.value.T)) \ .collect() us = matrix(np.array(us)[:, :, 0]) diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index ee9036adfa281..a7c74e969cdb9 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -24,18 +24,18 @@ if __name__ == "__main__": """ - Usage: pi [slices] + Usage: pi [partitions] """ sc = SparkContext(appName="PythonPi") - slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - n = 100000 * slices + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + n = 100000 * partitions def f(_): x = random() * 2 - 1 y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n + 1), slices).map(f).reduce(add) + count = sc.parallelize(xrange(1, n + 1), partitions).map(f).reduce(add) print "Pi is roughly %f" % (4.0 * count / n) sc.stop() diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index bf331b542c438..00a281bfb6506 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -37,11 +37,11 @@ def generateGraph(): if __name__ == "__main__": """ - Usage: transitive_closure [slices] + Usage: transitive_closure [partitions] """ sc = SparkContext(appName="PythonTransitiveClosure") - slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2 - tc = sc.parallelize(generateGraph(), slices).cache() + partitions = int(sys.argv[1]) if len(sys.argv) > 1 else 2 + tc = sc.parallelize(generateGraph(), partitions).cache() # Linear transitive closure: each round grows paths by one edge, # by joining the graph's edges with the already-discovered paths. From fce5e251d636c788cda91345867e0294280c074d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 19 Sep 2014 15:01:11 -0700 Subject: [PATCH 0744/1492] [SPARK-3491] [MLlib] [PySpark] use pickle to serialize data in MLlib Currently, we serialize the data between JVM and Python case by case manually, this cannot scale to support so many APIs in MLlib. This patch will try to address this problem by serialize the data using pickle protocol, using Pyrolite library to serialize/deserialize in JVM. Pickle protocol can be easily extended to support customized class. All the modules are refactored to use this protocol. Known issues: There will be some performance regression (both CPU and memory, the serialized data increased) Author: Davies Liu Closes #2378 from davies/pickle_mllib and squashes the following commits: dffbba2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into pickle_mllib 810f97f [Davies Liu] fix equal of matrix 032cd62 [Davies Liu] add more type check and conversion for user_product bd738ab [Davies Liu] address comments e431377 [Davies Liu] fix cache of rdd, refactor 19d0967 [Davies Liu] refactor Picklers 2511e76 [Davies Liu] cleanup 1fccf1a [Davies Liu] address comments a2cc855 [Davies Liu] fix tests 9ceff73 [Davies Liu] test size of serialized Rating 44e0551 [Davies Liu] fix cache a379a81 [Davies Liu] fix pickle array in python2.7 df625c7 [Davies Liu] Merge commit '154d141' into pickle_mllib 154d141 [Davies Liu] fix autobatchedpickler 44736d7 [Davies Liu] speed up pickling array in Python 2.7 e1d1bfc [Davies Liu] refactor 708dc02 [Davies Liu] fix tests 9dcfb63 [Davies Liu] fix style 88034f0 [Davies Liu] rafactor, address comments 46a501e [Davies Liu] choose batch size automatically df19464 [Davies Liu] memorize the module and class name during pickleing f3506c5 [Davies Liu] Merge branch 'master' into pickle_mllib 722dd96 [Davies Liu] cleanup _common.py 0ee1525 [Davies Liu] remove outdated tests b02e34f [Davies Liu] remove _common.py 84c721d [Davies Liu] Merge branch 'master' into pickle_mllib 4d7963e [Davies Liu] remove muanlly serialization 6d26b03 [Davies Liu] fix tests c383544 [Davies Liu] classification f2a0856 [Davies Liu] mllib/regression d9f691f [Davies Liu] mllib/util cccb8b1 [Davies Liu] mllib/tree 8fe166a [Davies Liu] Merge branch 'pickle' into pickle_mllib aa2287e [Davies Liu] random f1544c4 [Davies Liu] refactor clustering 52d1350 [Davies Liu] use new protocol in mllib/stat b30ef35 [Davies Liu] use pickle to serialize data for mllib/recommendation f44f771 [Davies Liu] enable tests about array 3908f5c [Davies Liu] Merge branch 'master' into pickle c77c87b [Davies Liu] cleanup debugging code 60e4e2f [Davies Liu] support unpickle array.array for Python 2.6 --- .../apache/spark/api/python/PythonRDD.scala | 31 +- .../apache/spark/api/python/SerDeUtil.scala | 4 +- .../mllib/api/python/PythonMLLibAPI.scala | 487 ++++++--------- .../apache/spark/mllib/linalg/Matrices.scala | 10 +- .../MatrixFactorizationModel.scala | 15 - .../api/python/PythonMLLibAPISuite.scala | 44 +- python/epydoc.conf | 2 +- python/pyspark/context.py | 1 + python/pyspark/mllib/_common.py | 562 ------------------ python/pyspark/mllib/classification.py | 61 +- python/pyspark/mllib/clustering.py | 38 +- python/pyspark/mllib/linalg.py | 256 ++++++-- python/pyspark/mllib/random.py | 54 +- python/pyspark/mllib/recommendation.py | 69 ++- python/pyspark/mllib/regression.py | 105 ++-- python/pyspark/mllib/stat.py | 63 +- python/pyspark/mllib/tests.py | 99 +-- python/pyspark/mllib/tree.py | 167 +++--- python/pyspark/mllib/util.py | 43 +- python/pyspark/rdd.py | 10 +- python/pyspark/serializers.py | 36 ++ python/run-tests | 1 - 22 files changed, 891 insertions(+), 1267 deletions(-) delete mode 100644 python/pyspark/mllib/_common.py diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 12b345a8fa7c3..f9ff4ea6ca157 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -775,17 +775,36 @@ private[spark] object PythonRDD extends Logging { }.toJavaRDD() } + private class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { + private val pickle = new Pickler() + private var batch = 1 + private val buffer = new mutable.ArrayBuffer[Any] + + override def hasNext(): Boolean = iter.hasNext + + override def next(): Array[Byte] = { + while (iter.hasNext && buffer.length < batch) { + buffer += iter.next() + } + val bytes = pickle.dumps(buffer.toArray) + val size = bytes.length + // let 1M < size < 10M + if (size < 1024 * 1024) { + batch *= 2 + } else if (size > 1024 * 1024 * 10 && batch > 1) { + batch /= 2 + } + buffer.clear() + bytes + } + } + /** * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by * PySpark. */ def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { - jRDD.rdd.mapPartitions { iter => - val pickle = new Pickler - iter.map { row => - pickle.dumps(row) - } - } + jRDD.rdd.mapPartitions { iter => new AutoBatchedPickler(iter) } } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 6668797f5f8be..7903457b17e13 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -68,8 +68,8 @@ private[python] object SerDeUtil extends Logging { construct(args ++ Array("")) } else if (args.length == 2 && args(1).isInstanceOf[String]) { val typecode = args(0).asInstanceOf[String].charAt(0) - val data: String = args(1).asInstanceOf[String] - construct(typecode, machineCodes(typecode), data.getBytes("ISO-8859-1")) + val data: Array[Byte] = args(1).asInstanceOf[String].getBytes("ISO-8859-1") + construct(typecode, machineCodes(typecode), data) } else { super.construct(args) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index fa0fa69f38634..9164c294ac7b8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -17,16 +17,20 @@ package org.apache.spark.mllib.api.python -import java.nio.{ByteBuffer, ByteOrder} +import java.io.OutputStream import scala.collection.JavaConverters._ +import scala.language.existentials +import scala.reflect.ClassTag + +import net.razorvine.pickle._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.optimization._ -import org.apache.spark.mllib.linalg.{Matrix, SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ @@ -40,11 +44,10 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils + /** * :: DeveloperApi :: * The Java stubs necessary for the Python mllib bindings. - * - * See python/pyspark/mllib/_common.py for the mutually agreed upon data format. */ @DeveloperApi class PythonMLLibAPI extends Serializable { @@ -60,18 +63,17 @@ class PythonMLLibAPI extends Serializable { def loadLabeledPoints( jsc: JavaSparkContext, path: String, - minPartitions: Int): JavaRDD[Array[Byte]] = - MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions).map(SerDe.serializeLabeledPoint) + minPartitions: Int): JavaRDD[LabeledPoint] = + MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions) private def trainRegressionModel( trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val initialWeights = SerDe.deserializeDoubleVector(initialWeightsBA) - val model = trainFunc(data, initialWeights) + val initialWeights = SerDe.loads(initialWeightsBA).asInstanceOf[Vector] + val model = trainFunc(data.rdd, initialWeights) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleVector(model.weights)) + ret.add(SerDe.dumps(model.weights)) ret.add(model.intercept: java.lang.Double) ret } @@ -80,7 +82,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LinearRegressionWithSGD.train() */ def trainLinearRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, miniBatchFraction: Double, @@ -106,7 +108,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( (data, initialWeights) => lrAlg.run(data, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -114,7 +116,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LassoWithSGD.train() */ def trainLassoModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, @@ -129,7 +131,7 @@ class PythonMLLibAPI extends Serializable { regParam, miniBatchFraction, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -137,7 +139,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib RidgeRegressionWithSGD.train() */ def trainRidgeModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, @@ -152,7 +154,7 @@ class PythonMLLibAPI extends Serializable { regParam, miniBatchFraction, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -160,7 +162,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib SVMWithSGD.train() */ def trainSVMModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, regParam: Double, @@ -186,7 +188,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( (data, initialWeights) => SVMAlg.run(data, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -194,7 +196,7 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib LogisticRegressionWithSGD.train() */ def trainLogisticRegressionModelWithSGD( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], numIterations: Int, stepSize: Double, miniBatchFraction: Double, @@ -220,7 +222,7 @@ class PythonMLLibAPI extends Serializable { trainRegressionModel( (data, initialWeights) => LogRegAlg.run(data, initialWeights), - dataBytesJRDD, + data, initialWeightsBA) } @@ -228,14 +230,13 @@ class PythonMLLibAPI extends Serializable { * Java stub for NaiveBayes.train() */ def trainNaiveBayes( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], lambda: Double): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val model = NaiveBayes.train(data, lambda) + val model = NaiveBayes.train(data.rdd, lambda) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.labels))) - ret.add(SerDe.serializeDoubleVector(Vectors.dense(model.pi))) - ret.add(SerDe.serializeDoubleMatrix(model.theta)) + ret.add(Vectors.dense(model.labels)) + ret.add(Vectors.dense(model.pi)) + ret.add(model.theta) ret } @@ -243,16 +244,12 @@ class PythonMLLibAPI extends Serializable { * Java stub for Python mllib KMeans.train() */ def trainKMeansModel( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[Vector], k: Int, maxIterations: Int, runs: Int, - initializationMode: String): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(bytes => SerDe.deserializeDoubleVector(bytes)) - val model = KMeans.train(data, k, maxIterations, runs, initializationMode) - val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(SerDe.serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) - ret + initializationMode: String): KMeansModel = { + KMeans.train(data.rdd, k, maxIterations, runs, initializationMode) } /** @@ -262,13 +259,12 @@ class PythonMLLibAPI extends Serializable { * the Py4J documentation. */ def trainALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], + ratings: JavaRDD[Rating], rank: Int, iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) - ALS.train(ratings, rank, iterations, lambda, blocks) + ALS.train(ratings.rdd, rank, iterations, lambda, blocks) } /** @@ -278,14 +274,13 @@ class PythonMLLibAPI extends Serializable { * exit; see the Py4J documentation. */ def trainImplicitALSModel( - ratingsBytesJRDD: JavaRDD[Array[Byte]], + ratingsJRDD: JavaRDD[Rating], rank: Int, iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { - val ratings = ratingsBytesJRDD.rdd.map(SerDe.unpackRating) - ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) + ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) } /** @@ -293,11 +288,11 @@ class PythonMLLibAPI extends Serializable { * This stub returns a handle to the Java object instead of the content of the Java object. * Extra care needs to be taken in the Python code to ensure it gets freed on exit; * see the Py4J documentation. - * @param dataBytesJRDD Training data + * @param data Training data * @param categoricalFeaturesInfoJMap Categorical features info, as Java map */ def trainDecisionTreeModel( - dataBytesJRDD: JavaRDD[Array[Byte]], + data: JavaRDD[LabeledPoint], algoStr: String, numClasses: Int, categoricalFeaturesInfoJMap: java.util.Map[Int, Int], @@ -307,8 +302,6 @@ class PythonMLLibAPI extends Serializable { minInstancesPerNode: Int, minInfoGain: Double): DecisionTreeModel = { - val data = dataBytesJRDD.rdd.map(SerDe.deserializeLabeledPoint) - val algo = Algo.fromString(algoStr) val impurity = Impurities.fromString(impurityStr) @@ -322,44 +315,15 @@ class PythonMLLibAPI extends Serializable { minInstancesPerNode = minInstancesPerNode, minInfoGain = minInfoGain) - DecisionTree.train(data, strategy) - } - - /** - * Predict the label of the given data point. - * This is a Java stub for python DecisionTreeModel.predict() - * - * @param featuresBytes Serialized feature vector for data point - * @return predicted label - */ - def predictDecisionTreeModel( - model: DecisionTreeModel, - featuresBytes: Array[Byte]): Double = { - val features: Vector = SerDe.deserializeDoubleVector(featuresBytes) - model.predict(features) - } - - /** - * Predict the labels of the given data points. - * This is a Java stub for python DecisionTreeModel.predict() - * - * @param dataJRDD A JavaRDD with serialized feature vectors - * @return JavaRDD of serialized predictions - */ - def predictDecisionTreeModel( - model: DecisionTreeModel, - dataJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val data = dataJRDD.rdd.map(xBytes => SerDe.deserializeDoubleVector(xBytes)) - model.predict(data).map(SerDe.serializeDouble) + DecisionTree.train(data.rdd, strategy) } /** * Java stub for mllib Statistics.colStats(X: RDD[Vector]). * TODO figure out return type. */ - def colStats(X: JavaRDD[Array[Byte]]): MultivariateStatisticalSummarySerialized = { - val cStats = Statistics.colStats(X.rdd.map(SerDe.deserializeDoubleVector(_))) - new MultivariateStatisticalSummarySerialized(cStats) + def colStats(rdd: JavaRDD[Vector]): MultivariateStatisticalSummary = { + Statistics.colStats(rdd.rdd) } /** @@ -367,19 +331,15 @@ class PythonMLLibAPI extends Serializable { * Returns the correlation matrix serialized into a byte array understood by deserializers in * pyspark. */ - def corr(X: JavaRDD[Array[Byte]], method: String): Array[Byte] = { - val inputMatrix = X.rdd.map(SerDe.deserializeDoubleVector(_)) - val result = Statistics.corr(inputMatrix, getCorrNameOrDefault(method)) - SerDe.serializeDoubleMatrix(SerDe.to2dArray(result)) + def corr(x: JavaRDD[Vector], method: String): Matrix = { + Statistics.corr(x.rdd, getCorrNameOrDefault(method)) } /** * Java stub for mllib Statistics.corr(x: RDD[Double], y: RDD[Double], method: String). */ - def corr(x: JavaRDD[Array[Byte]], y: JavaRDD[Array[Byte]], method: String): Double = { - val xDeser = x.rdd.map(SerDe.deserializeDouble(_)) - val yDeser = y.rdd.map(SerDe.deserializeDouble(_)) - Statistics.corr(xDeser, yDeser, getCorrNameOrDefault(method)) + def corr(x: JavaRDD[Double], y: JavaRDD[Double], method: String): Double = { + Statistics.corr(x.rdd, y.rdd, getCorrNameOrDefault(method)) } // used by the corr methods to retrieve the name of the correlation method passed in via pyspark @@ -411,10 +371,10 @@ class PythonMLLibAPI extends Serializable { def uniformRDD(jsc: JavaSparkContext, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + RG.uniformRDD(jsc.sc, size, parts, s) } /** @@ -423,10 +383,10 @@ class PythonMLLibAPI extends Serializable { def normalRDD(jsc: JavaSparkContext, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalRDD(jsc.sc, size, parts, s).map(SerDe.serializeDouble) + RG.normalRDD(jsc.sc, size, parts, s) } /** @@ -436,10 +396,10 @@ class PythonMLLibAPI extends Serializable { mean: Double, size: Long, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Double] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonRDD(jsc.sc, mean, size, parts, s).map(SerDe.serializeDouble) + RG.poissonRDD(jsc.sc, mean, size, parts, s) } /** @@ -449,10 +409,10 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.uniformVectorRDD(jsc.sc, numRows, numCols, parts, s) } /** @@ -462,10 +422,10 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.normalVectorRDD(jsc.sc, numRows, numCols, parts, s) } /** @@ -476,259 +436,168 @@ class PythonMLLibAPI extends Serializable { numRows: Long, numCols: Int, numPartitions: java.lang.Integer, - seed: java.lang.Long): JavaRDD[Array[Byte]] = { + seed: java.lang.Long): JavaRDD[Vector] = { val parts = getNumPartitionsOrDefault(numPartitions, jsc) val s = getSeedOrDefault(seed) - RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s).map(SerDe.serializeDoubleVector) + RG.poissonVectorRDD(jsc.sc, mean, numRows, numCols, parts, s) } } /** - * :: DeveloperApi :: - * MultivariateStatisticalSummary with Vector fields serialized. + * SerDe utility functions for PythonMLLibAPI. */ -@DeveloperApi -class MultivariateStatisticalSummarySerialized(val summary: MultivariateStatisticalSummary) - extends Serializable { +private[spark] object SerDe extends Serializable { - def mean: Array[Byte] = SerDe.serializeDoubleVector(summary.mean) + val PYSPARK_PACKAGE = "pyspark.mllib" - def variance: Array[Byte] = SerDe.serializeDoubleVector(summary.variance) + /** + * Base class used for pickle + */ + private[python] abstract class BasePickler[T: ClassTag] + extends IObjectPickler with IObjectConstructor { + + private val cls = implicitly[ClassTag[T]].runtimeClass + private val module = PYSPARK_PACKAGE + "." + cls.getName.split('.')(4) + private val name = cls.getSimpleName + + // register this to Pickler and Unpickler + def register(): Unit = { + Pickler.registerCustomPickler(this.getClass, this) + Pickler.registerCustomPickler(cls, this) + Unpickler.registerConstructor(module, name, this) + } - def count: Long = summary.count + def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { + if (obj == this) { + out.write(Opcodes.GLOBAL) + out.write((module + "\n" + name + "\n").getBytes()) + } else { + pickler.save(this) // it will be memorized by Pickler + saveState(obj, out, pickler) + out.write(Opcodes.REDUCE) + } + } + + private[python] def saveObjects(out: OutputStream, pickler: Pickler, objects: Any*) = { + if (objects.length == 0 || objects.length > 3) { + out.write(Opcodes.MARK) + } + objects.foreach(pickler.save(_)) + val code = objects.length match { + case 1 => Opcodes.TUPLE1 + case 2 => Opcodes.TUPLE2 + case 3 => Opcodes.TUPLE3 + case _ => Opcodes.TUPLE + } + out.write(code) + } - def numNonzeros: Array[Byte] = SerDe.serializeDoubleVector(summary.numNonzeros) + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) + } - def max: Array[Byte] = SerDe.serializeDoubleVector(summary.max) + // Pickler for DenseVector + private[python] class DenseVectorPickler extends BasePickler[DenseVector] { - def min: Array[Byte] = SerDe.serializeDoubleVector(summary.min) -} + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val vector: DenseVector = obj.asInstanceOf[DenseVector] + saveObjects(out, pickler, vector.toArray) + } -/** - * SerDe utility functions for PythonMLLibAPI. - */ -private[spark] object SerDe extends Serializable { - private val DENSE_VECTOR_MAGIC: Byte = 1 - private val SPARSE_VECTOR_MAGIC: Byte = 2 - private val DENSE_MATRIX_MAGIC: Byte = 3 - private val LABELED_POINT_MAGIC: Byte = 4 - - private[python] def deserializeDoubleVector(bytes: Array[Byte], offset: Int = 0): Vector = { - require(bytes.length - offset >= 5, "Byte array too short") - val magic = bytes(offset) - if (magic == DENSE_VECTOR_MAGIC) { - deserializeDenseVector(bytes, offset) - } else if (magic == SPARSE_VECTOR_MAGIC) { - deserializeSparseVector(bytes, offset) - } else { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + def construct(args: Array[Object]): Object = { + require(args.length == 1) + if (args.length != 1) { + throw new PickleException("should be 1") + } + new DenseVector(args(0).asInstanceOf[Array[Double]]) } } - private[python] def deserializeDouble(bytes: Array[Byte], offset: Int = 0): Double = { - require(bytes.length - offset == 8, "Wrong size byte array for Double") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - bb.getDouble - } - - private[python] def deserializeDenseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 5, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == DENSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val length = bb.getInt() - require (packetLength == 5 + 8 * length, "Invalid packet length: " + packetLength) - val db = bb.asDoubleBuffer() - val ans = new Array[Double](length.toInt) - db.get(ans) - Vectors.dense(ans) - } - - private[python] def deserializeSparseVector(bytes: Array[Byte], offset: Int = 0): Vector = { - val packetLength = bytes.length - offset - require(packetLength >= 9, "Byte array too short") - val bb = ByteBuffer.wrap(bytes, offset, bytes.length - offset) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - require(magic == SPARSE_VECTOR_MAGIC, "Invalid magic: " + magic) - val size = bb.getInt() - val nonZeros = bb.getInt() - require (packetLength == 9 + 12 * nonZeros, "Invalid packet length: " + packetLength) - val ib = bb.asIntBuffer() - val indices = new Array[Int](nonZeros) - ib.get(indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - val values = new Array[Double](nonZeros) - db.get(values) - Vectors.sparse(size, indices, values) - } + // Pickler for DenseMatrix + private[python] class DenseMatrixPickler extends BasePickler[DenseMatrix] { - /** - * Returns an 8-byte array for the input Double. - * - * Note: we currently do not use a magic byte for double for storage efficiency. - * This should be reconsidered when we add Ser/De for other 8-byte types (e.g. Long), for safety. - * The corresponding deserializer, deserializeDouble, needs to be modified as well if the - * serialization scheme changes. - */ - private[python] def serializeDouble(double: Double): Array[Byte] = { - val bytes = new Array[Byte](8) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putDouble(double) - bytes - } - - private[python] def serializeDenseVector(doubles: Array[Double]): Array[Byte] = { - val len = doubles.length - val bytes = new Array[Byte](5 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_VECTOR_MAGIC) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(doubles) - bytes - } - - private[python] def serializeSparseVector(vector: SparseVector): Array[Byte] = { - val nonZeros = vector.indices.length - val bytes = new Array[Byte](9 + 12 * nonZeros) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(SPARSE_VECTOR_MAGIC) - bb.putInt(vector.size) - bb.putInt(nonZeros) - val ib = bb.asIntBuffer() - ib.put(vector.indices) - bb.position(bb.position() + 4 * nonZeros) - val db = bb.asDoubleBuffer() - db.put(vector.values) - bytes - } - - private[python] def serializeDoubleVector(vector: Vector): Array[Byte] = vector match { - case s: SparseVector => - serializeSparseVector(s) - case _ => - serializeDenseVector(vector.toArray) - } - - private[python] def deserializeDoubleMatrix(bytes: Array[Byte]): Array[Array[Double]] = { - val packetLength = bytes.length - if (packetLength < 9) { - throw new IllegalArgumentException("Byte array too short.") + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] + saveObjects(out, pickler, m.numRows, m.numCols, m.values) } - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - val magic = bb.get() - if (magic != DENSE_MATRIX_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], + args(2).asInstanceOf[Array[Double]]) } - val rows = bb.getInt() - val cols = bb.getInt() - if (packetLength != 9 + 8 * rows * cols) { - throw new IllegalArgumentException("Size " + rows + "x" + cols + " is wrong.") + } + + // Pickler for SparseVector + private[python] class SparseVectorPickler extends BasePickler[SparseVector] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val v: SparseVector = obj.asInstanceOf[SparseVector] + saveObjects(out, pickler, v.size, v.indices, v.values) } - val db = bb.asDoubleBuffer() - val ans = new Array[Array[Double]](rows.toInt) - for (i <- 0 until rows.toInt) { - ans(i) = new Array[Double](cols.toInt) - db.get(ans(i)) + + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new SparseVector(args(0).asInstanceOf[Int], args(1).asInstanceOf[Array[Int]], + args(2).asInstanceOf[Array[Double]]) } - ans } - private[python] def serializeDoubleMatrix(doubles: Array[Array[Double]]): Array[Byte] = { - val rows = doubles.length - var cols = 0 - if (rows > 0) { - cols = doubles(0).length + // Pickler for LabeledPoint + private[python] class LabeledPointPickler extends BasePickler[LabeledPoint] { + + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val point: LabeledPoint = obj.asInstanceOf[LabeledPoint] + saveObjects(out, pickler, point.label, point.features) } - val bytes = new Array[Byte](9 + 8 * rows * cols) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(DENSE_MATRIX_MAGIC) - bb.putInt(rows) - bb.putInt(cols) - val db = bb.asDoubleBuffer() - for (i <- 0 until rows) { - db.put(doubles(i)) + + def construct(args: Array[Object]): Object = { + if (args.length != 2) { + throw new PickleException("should be 2") + } + new LabeledPoint(args(0).asInstanceOf[Double], args(1).asInstanceOf[Vector]) } - bytes } - private[python] def serializeLabeledPoint(p: LabeledPoint): Array[Byte] = { - val fb = serializeDoubleVector(p.features) - val bytes = new Array[Byte](1 + 8 + fb.length) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.put(LABELED_POINT_MAGIC) - bb.putDouble(p.label) - bb.put(fb) - bytes - } + // Pickler for Rating + private[python] class RatingPickler extends BasePickler[Rating] { - private[python] def deserializeLabeledPoint(bytes: Array[Byte]): LabeledPoint = { - require(bytes.length >= 9, "Byte array too short") - val magic = bytes(0) - if (magic != LABELED_POINT_MAGIC) { - throw new IllegalArgumentException("Magic " + magic + " is wrong.") + def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { + val rating: Rating = obj.asInstanceOf[Rating] + saveObjects(out, pickler, rating.user, rating.product, rating.rating) } - val labelBytes = ByteBuffer.wrap(bytes, 1, 8) - labelBytes.order(ByteOrder.nativeOrder()) - val label = labelBytes.asDoubleBuffer().get(0) - LabeledPoint(label, deserializeDoubleVector(bytes, 9)) - } - // Reformat a Matrix into Array[Array[Double]] for serialization - private[python] def to2dArray(matrix: Matrix): Array[Array[Double]] = { - val values = matrix.toArray - Array.tabulate(matrix.numRows, matrix.numCols)((i, j) => values(i + j * matrix.numRows)) + def construct(args: Array[Object]): Object = { + if (args.length != 3) { + throw new PickleException("should be 3") + } + new Rating(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], + args(2).asInstanceOf[Double]) + } } + def initialize(): Unit = { + new DenseVectorPickler().register() + new DenseMatrixPickler().register() + new SparseVectorPickler().register() + new LabeledPointPickler().register() + new RatingPickler().register() + } - /** Unpack a Rating object from an array of bytes */ - private[python] def unpackRating(ratingBytes: Array[Byte]): Rating = { - val bb = ByteBuffer.wrap(ratingBytes) - bb.order(ByteOrder.nativeOrder()) - val user = bb.getInt() - val product = bb.getInt() - val rating = bb.getDouble() - new Rating(user, product, rating) + def dumps(obj: AnyRef): Array[Byte] = { + new Pickler().dumps(obj) } - /** Unpack a tuple of Ints from an array of bytes */ - def unpackTuple(tupleBytes: Array[Byte]): (Int, Int) = { - val bb = ByteBuffer.wrap(tupleBytes) - bb.order(ByteOrder.nativeOrder()) - val v1 = bb.getInt() - val v2 = bb.getInt() - (v1, v2) + def loads(bytes: Array[Byte]): AnyRef = { + new Unpickler().loads(bytes) } - /** - * Serialize a Rating object into an array of bytes. - * It can be deserialized using RatingDeserializer(). - * - * @param rate the Rating object to serialize - * @return - */ - def serializeRating(rate: Rating): Array[Byte] = { - val len = 3 - val bytes = new Array[Byte](4 + 8 * len) - val bb = ByteBuffer.wrap(bytes) - bb.order(ByteOrder.nativeOrder()) - bb.putInt(len) - val db = bb.asDoubleBuffer() - db.put(rate.user.toDouble) - db.put(rate.product.toDouble) - db.put(rate.rating) - bytes + /* convert object into Tuple */ + def asTupleRDD(rdd: RDD[Array[Any]]): RDD[(Int, Int)] = { + rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 5711532abcf80..4e87fe088ecc5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -17,12 +17,12 @@ package org.apache.spark.mllib.linalg +import java.util.Arrays + import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} import org.apache.spark.util.random.XORShiftRandom -import java.util.Arrays - /** * Trait for a local matrix. */ @@ -106,6 +106,12 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) override def toArray: Array[Double] = values + override def equals(o: Any) = o match { + case m: DenseMatrix => + m.numRows == numRows && m.numCols == numCols && Arrays.equals(toArray, m.toArray) + case _ => false + } + private[mllib] def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) private[mllib] def apply(i: Int): Double = values(i) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 478c6485052b6..66b58ba770160 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -106,19 +106,4 @@ class MatrixFactorizationModel private[mllib] ( } scored.top(num)(Ordering.by(_._2)) } - - /** - * :: DeveloperApi :: - * Predict the rating of many users for many products. - * This is a Java stub for python predictAll() - * - * @param usersProductsJRDD A JavaRDD with serialized tuples (user, product) - * @return JavaRDD of serialized Rating objects. - */ - @DeveloperApi - def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = { - val usersProducts = usersProductsJRDD.rdd.map(xBytes => SerDe.unpackTuple(xBytes)) - predict(usersProducts).map(rate => SerDe.serializeRating(rate)) - } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala index 092d67bbc5238..db8ed62fa46ce 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/api/python/PythonMLLibAPISuite.scala @@ -19,12 +19,15 @@ package org.apache.spark.mllib.api.python import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{Matrices, Vectors} +import org.apache.spark.mllib.linalg.{DenseMatrix, Matrices, Vectors} import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.recommendation.Rating class PythonMLLibAPISuite extends FunSuite { - test("vector serialization") { + SerDe.initialize() + + test("pickle vector") { val vectors = Seq( Vectors.dense(Array.empty[Double]), Vectors.dense(0.0), @@ -33,14 +36,13 @@ class PythonMLLibAPISuite extends FunSuite { Vectors.sparse(1, Array.empty[Int], Array.empty[Double]), Vectors.sparse(2, Array(1), Array(-2.0))) vectors.foreach { v => - val bytes = SerDe.serializeDoubleVector(v) - val u = SerDe.deserializeDoubleVector(bytes) + val u = SerDe.loads(SerDe.dumps(v)) assert(u.getClass === v.getClass) assert(u === v) } } - test("labeled point serialization") { + test("pickle labeled point") { val points = Seq( LabeledPoint(0.0, Vectors.dense(Array.empty[Double])), LabeledPoint(1.0, Vectors.dense(0.0)), @@ -49,34 +51,44 @@ class PythonMLLibAPISuite extends FunSuite { LabeledPoint(1.0, Vectors.sparse(1, Array.empty[Int], Array.empty[Double])), LabeledPoint(-0.5, Vectors.sparse(2, Array(1), Array(-2.0)))) points.foreach { p => - val bytes = SerDe.serializeLabeledPoint(p) - val q = SerDe.deserializeLabeledPoint(bytes) + val q = SerDe.loads(SerDe.dumps(p)).asInstanceOf[LabeledPoint] assert(q.label === p.label) assert(q.features.getClass === p.features.getClass) assert(q.features === p.features) } } - test("double serialization") { + test("pickle double") { for (x <- List(123.0, -10.0, 0.0, Double.MaxValue, Double.MinValue, Double.NaN)) { - val bytes = SerDe.serializeDouble(x) - val deser = SerDe.deserializeDouble(bytes) + val deser = SerDe.loads(SerDe.dumps(x.asInstanceOf[AnyRef])).asInstanceOf[Double] // We use `equals` here for comparison because we cannot use `==` for NaN assert(x.equals(deser)) } } - test("matrix to 2D array") { + test("pickle matrix") { val values = Array[Double](0, 1.2, 3, 4.56, 7, 8) val matrix = Matrices.dense(2, 3, values) - val arr = SerDe.to2dArray(matrix) - val expected = Array(Array[Double](0, 3, 7), Array[Double](1.2, 4.56, 8)) - assert(arr === expected) + val nm = SerDe.loads(SerDe.dumps(matrix)).asInstanceOf[DenseMatrix] + assert(matrix === nm) // Test conversion for empty matrix val empty = Array[Double]() val emptyMatrix = Matrices.dense(0, 0, empty) - val empty2D = SerDe.to2dArray(emptyMatrix) - assert(empty2D === Array[Array[Double]]()) + val ne = SerDe.loads(SerDe.dumps(emptyMatrix)).asInstanceOf[DenseMatrix] + assert(emptyMatrix == ne) + } + + test("pickle rating") { + val rat = new Rating(1, 2, 3.0) + val rat2 = SerDe.loads(SerDe.dumps(rat)).asInstanceOf[Rating] + assert(rat == rat2) + + // Test name of class only occur once + val rats = (1 to 10).map(x => new Rating(x, x + 1, x + 3.0)).toArray + val bytes = SerDe.dumps(rats) + assert(bytes.toString.split("Rating").length == 1) + assert(bytes.length / 10 < 25) // 25 bytes per rating + } } diff --git a/python/epydoc.conf b/python/epydoc.conf index 51c0faf359939..8593e08deda19 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -34,5 +34,5 @@ private: no exclude: pyspark.cloudpickle pyspark.worker pyspark.join pyspark.java_gateway pyspark.examples pyspark.shell pyspark.tests - pyspark.rddsampler pyspark.daemon pyspark.mllib._common + pyspark.rddsampler pyspark.daemon pyspark.mllib.tests pyspark.shuffle diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a17f2c1203d36..064a24bff539c 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -211,6 +211,7 @@ def _ensure_initialized(cls, instance=None, gateway=None): SparkContext._jvm = SparkContext._gateway.jvm SparkContext._writeToFile = SparkContext._jvm.PythonRDD.writeToFile SparkContext._jvm.SerDeUtil.initialize() + SparkContext._jvm.SerDe.initialize() if instance: if (SparkContext._active_spark_context and diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py deleted file mode 100644 index 68f6033616726..0000000000000 --- a/python/pyspark/mllib/_common.py +++ /dev/null @@ -1,562 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -import struct -import sys -import numpy -from numpy import ndarray, float64, int64, int32, array_equal, array -from pyspark import SparkContext, RDD -from pyspark.mllib.linalg import SparseVector -from pyspark.serializers import FramedSerializer - - -""" -Common utilities shared throughout MLlib, primarily for dealing with -different data types. These include: -- Serialization utilities to / from byte arrays that Java can handle -- Serializers for other data types, like ALS Rating objects -- Common methods for linear models -- Methods to deal with the different vector types we support, such as - SparseVector and scipy.sparse matrices. -""" - - -# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, -# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. - -_have_scipy = False -_scipy_issparse = None -try: - import scipy.sparse - _have_scipy = True - _scipy_issparse = scipy.sparse.issparse -except: - # No SciPy in environment, but that's okay - pass - - -# Serialization functions to and from Scala. These use the following formats, understood -# by the PythonMLLibAPI class in Scala: -# -# Dense double vector format: -# -# [1-byte 1] [4-byte length] [length*8 bytes of data] -# -# Sparse double vector format: -# -# [1-byte 2] [4-byte length] [4-byte nonzeros] [nonzeros*4 bytes of indices] \ -# [nonzeros*8 bytes of values] -# -# Double matrix format: -# -# [1-byte 3] [4-byte rows] [4-byte cols] [rows*cols*8 bytes of data] -# -# LabeledPoint format: -# -# [1-byte 4] [8-byte label] [dense or sparse vector] -# -# This is all in machine-endian. That means that the Java interpreter and the -# Python interpreter must agree on what endian the machine is. - - -DENSE_VECTOR_MAGIC = 1 -SPARSE_VECTOR_MAGIC = 2 -DENSE_MATRIX_MAGIC = 3 -LABELED_POINT_MAGIC = 4 - - -# Workaround for SPARK-2954: before Python 2.7, struct.unpack couldn't unpack bytearray()s. -if sys.version_info[:2] <= (2, 6): - def _unpack(fmt, string): - return struct.unpack(fmt, buffer(string)) -else: - _unpack = struct.unpack - - -def _deserialize_numpy_array(shape, ba, offset, dtype=float64): - """ - Deserialize a numpy array of the given type from an offset in - bytearray ba, assigning it the given shape. - - >>> x = array([1.0, 2.0, 3.0, 4.0, 5.0]) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0)) - True - >>> x = array([1.0, 2.0, 3.0, 4.0]).reshape(2,2) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0)) - True - >>> x = array([1, 2, 3], dtype=int32) - >>> array_equal(x, _deserialize_numpy_array(x.shape, x.data, 0, dtype=int32)) - True - """ - ar = ndarray(shape=shape, buffer=ba, offset=offset, dtype=dtype, order='C') - return ar.copy() - - -def _serialize_double(d): - """ - Serialize a double (float or numpy.float64) into a mutually understood format. - """ - if type(d) == float or type(d) == float64 or type(d) == int or type(d) == long: - d = float64(d) - ba = bytearray(8) - _copyto(d, buffer=ba, offset=0, shape=[1], dtype=float64) - return ba - else: - raise TypeError("_serialize_double called on non-float input") - - -def _serialize_double_vector(v): - """ - Serialize a double vector into a mutually understood format. - - Note: we currently do not use a magic byte for double for storage - efficiency. This should be reconsidered when we add Ser/De for other - 8-byte types (e.g. Long), for safety. The corresponding deserializer, - _deserialize_double, needs to be modified as well if the serialization - scheme changes. - - >>> x = array([1,2,3]) - >>> y = _deserialize_double_vector(_serialize_double_vector(x)) - >>> array_equal(y, array([1.0, 2.0, 3.0])) - True - """ - v = _convert_vector(v) - if type(v) == ndarray: - return _serialize_dense_vector(v) - elif type(v) == SparseVector: - return _serialize_sparse_vector(v) - else: - raise TypeError("_serialize_double_vector called on a %s; " - "wanted ndarray or SparseVector" % type(v)) - - -def _serialize_dense_vector(v): - """Serialize a dense vector given as a NumPy array.""" - if v.ndim != 1: - raise TypeError("_serialize_double_vector called on a %ddarray; " - "wanted a 1darray" % v.ndim) - if v.dtype != float64: - if numpy.issubdtype(v.dtype, numpy.complex): - raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) - v = v.astype(float64) - length = v.shape[0] - ba = bytearray(5 + 8 * length) - ba[0] = DENSE_VECTOR_MAGIC - length_bytes = ndarray(shape=[1], buffer=ba, offset=1, dtype=int32) - length_bytes[0] = length - _copyto(v, buffer=ba, offset=5, shape=[length], dtype=float64) - return ba - - -def _serialize_sparse_vector(v): - """Serialize a pyspark.mllib.linalg.SparseVector.""" - nonzeros = len(v.indices) - ba = bytearray(9 + 12 * nonzeros) - ba[0] = SPARSE_VECTOR_MAGIC - header = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) - header[0] = v.size - header[1] = nonzeros - _copyto(v.indices, buffer=ba, offset=9, shape=[nonzeros], dtype=int32) - values_offset = 9 + 4 * nonzeros - _copyto(v.values, buffer=ba, offset=values_offset, shape=[nonzeros], dtype=float64) - return ba - - -def _deserialize_double(ba, offset=0): - """Deserialize a double from a mutually understood format. - - >>> import sys - >>> _deserialize_double(_serialize_double(123.0)) == 123.0 - True - >>> _deserialize_double(_serialize_double(float64(0.0))) == 0.0 - True - >>> _deserialize_double(_serialize_double(1)) == 1.0 - True - >>> _deserialize_double(_serialize_double(1L)) == 1.0 - True - >>> x = sys.float_info.max - >>> _deserialize_double(_serialize_double(sys.float_info.max)) == x - True - >>> y = float64(sys.float_info.max) - >>> _deserialize_double(_serialize_double(sys.float_info.max)) == y - True - """ - if type(ba) != bytearray: - raise TypeError("_deserialize_double called on a %s; wanted bytearray" % type(ba)) - if len(ba) - offset != 8: - raise TypeError("_deserialize_double called on a %d-byte array; wanted 8 bytes." % nb) - return _unpack("d", ba[offset:])[0] - - -def _deserialize_double_vector(ba, offset=0): - """Deserialize a double vector from a mutually understood format. - - >>> x = array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0]) - >>> array_equal(x, _deserialize_double_vector(_serialize_double_vector(x))) - True - >>> s = SparseVector(4, [1, 3], [3.0, 5.5]) - >>> s == _deserialize_double_vector(_serialize_double_vector(s)) - True - """ - if type(ba) != bytearray: - raise TypeError("_deserialize_double_vector called on a %s; " - "wanted bytearray" % type(ba)) - nb = len(ba) - offset - if nb < 5: - raise TypeError("_deserialize_double_vector called on a %d-byte array, " - "which is too short" % nb) - if ba[offset] == DENSE_VECTOR_MAGIC: - return _deserialize_dense_vector(ba, offset) - elif ba[offset] == SPARSE_VECTOR_MAGIC: - return _deserialize_sparse_vector(ba, offset) - else: - raise TypeError("_deserialize_double_vector called on bytearray " - "with wrong magic") - - -def _deserialize_dense_vector(ba, offset=0): - """Deserialize a dense vector into a numpy array.""" - nb = len(ba) - offset - if nb < 5: - raise TypeError("_deserialize_dense_vector called on a %d-byte array, " - "which is too short" % nb) - length = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=int32)[0] - if nb < 8 * length + 5: - raise TypeError("_deserialize_dense_vector called on bytearray " - "with wrong length") - return _deserialize_numpy_array([length], ba, offset + 5) - - -def _deserialize_sparse_vector(ba, offset=0): - """Deserialize a sparse vector into a MLlib SparseVector object.""" - nb = len(ba) - offset - if nb < 9: - raise TypeError("_deserialize_sparse_vector called on a %d-byte array, " - "which is too short" % nb) - header = ndarray(shape=[2], buffer=ba, offset=offset + 1, dtype=int32) - size = header[0] - nonzeros = header[1] - if nb < 9 + 12 * nonzeros: - raise TypeError("_deserialize_sparse_vector called on bytearray " - "with wrong length") - indices = _deserialize_numpy_array([nonzeros], ba, offset + 9, dtype=int32) - values = _deserialize_numpy_array([nonzeros], ba, offset + 9 + 4 * nonzeros, dtype=float64) - return SparseVector(int(size), indices, values) - - -def _serialize_double_matrix(m): - """Serialize a double matrix into a mutually understood format.""" - if (type(m) == ndarray and m.ndim == 2): - if m.dtype != float64: - if numpy.issubdtype(m.dtype, numpy.complex): - raise TypeError("_serialize_double_matrix called on an ndarray of %s; " - "wanted ndarray of float64" % m.dtype) - m = m.astype(float64) - rows = m.shape[0] - cols = m.shape[1] - ba = bytearray(9 + 8 * rows * cols) - ba[0] = DENSE_MATRIX_MAGIC - lengths = ndarray(shape=[3], buffer=ba, offset=1, dtype=int32) - lengths[0] = rows - lengths[1] = cols - _copyto(m, buffer=ba, offset=9, shape=[rows, cols], dtype=float64) - return ba - else: - raise TypeError("_serialize_double_matrix called on a " - "non-double-matrix") - - -def _deserialize_double_matrix(ba): - """Deserialize a double matrix from a mutually understood format.""" - if type(ba) != bytearray: - raise TypeError("_deserialize_double_matrix called on a %s; " - "wanted bytearray" % type(ba)) - if len(ba) < 9: - raise TypeError("_deserialize_double_matrix called on a %d-byte array, " - "which is too short" % len(ba)) - if ba[0] != DENSE_MATRIX_MAGIC: - raise TypeError("_deserialize_double_matrix called on bytearray " - "with wrong magic") - lengths = ndarray(shape=[2], buffer=ba, offset=1, dtype=int32) - rows = lengths[0] - cols = lengths[1] - if (len(ba) != 8 * rows * cols + 9): - raise TypeError("_deserialize_double_matrix called on bytearray " - "with wrong length") - return _deserialize_numpy_array([rows, cols], ba, 9) - - -def _serialize_labeled_point(p): - """ - Serialize a LabeledPoint with a features vector of any type. - - >>> from pyspark.mllib.regression import LabeledPoint - >>> dp0 = LabeledPoint(0.5, array([1.0, 2.0, 3.0, 4.0, -1.0, 0.0, -0.0])) - >>> dp1 = _deserialize_labeled_point(_serialize_labeled_point(dp0)) - >>> dp1.label == dp0.label - True - >>> array_equal(dp1.features, dp0.features) - True - >>> sp0 = LabeledPoint(0.0, SparseVector(4, [1, 3], [3.0, 5.5])) - >>> sp1 = _deserialize_labeled_point(_serialize_labeled_point(sp0)) - >>> sp1.label == sp1.label - True - >>> sp1.features == sp0.features - True - """ - from pyspark.mllib.regression import LabeledPoint - serialized_features = _serialize_double_vector(p.features) - header = bytearray(9) - header[0] = LABELED_POINT_MAGIC - header_float = ndarray(shape=[1], buffer=header, offset=1, dtype=float64) - header_float[0] = p.label - return header + serialized_features - - -def _deserialize_labeled_point(ba, offset=0): - """Deserialize a LabeledPoint from a mutually understood format.""" - from pyspark.mllib.regression import LabeledPoint - if type(ba) != bytearray: - raise TypeError("Expecting a bytearray but got %s" % type(ba)) - if ba[offset] != LABELED_POINT_MAGIC: - raise TypeError("Expecting magic number %d but got %d" % (LABELED_POINT_MAGIC, ba[0])) - label = ndarray(shape=[1], buffer=ba, offset=offset + 1, dtype=float64)[0] - features = _deserialize_double_vector(ba, offset + 9) - return LabeledPoint(label, features) - - -def _copyto(array, buffer, offset, shape, dtype): - """ - Copy the contents of a vector to a destination bytearray at the - given offset. - - TODO: In the future this could use numpy.copyto on NumPy 1.7+, but - we should benchmark that to see whether it provides a benefit. - """ - temp_array = ndarray(shape=shape, buffer=buffer, offset=offset, dtype=dtype, order='C') - temp_array[...] = array - - -def _get_unmangled_rdd(data, serializer, cache=True): - """ - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - dataBytes = data.map(serializer) - dataBytes._bypass_serializer = True - if cache: - dataBytes.cache() - return dataBytes - - -def _get_unmangled_double_vector_rdd(data, cache=True): - """ - Map a pickled Python RDD of Python dense or sparse vectors to a Java RDD of - _serialized_double_vectors. - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - return _get_unmangled_rdd(data, _serialize_double_vector, cache) - - -def _get_unmangled_labeled_point_rdd(data, cache=True): - """ - Map a pickled Python RDD of LabeledPoint to a Java RDD of _serialized_labeled_points. - :param cache: If True, the serialized RDD is cached. (default = True) - WARNING: Users should unpersist() this later! - """ - return _get_unmangled_rdd(data, _serialize_labeled_point, cache) - - -# Common functions for dealing with and training linear models - -def _linear_predictor_typecheck(x, coeffs): - """ - Check that x is a one-dimensional vector of the right shape. - This is a temporary hackaround until we actually implement bulk predict. - """ - x = _convert_vector(x) - if type(x) == ndarray: - if x.ndim == 1: - if x.shape != coeffs.shape: - raise RuntimeError("Got array of %d elements; wanted %d" % ( - numpy.shape(x)[0], coeffs.shape[0])) - else: - raise RuntimeError("Bulk predict not yet supported.") - elif type(x) == SparseVector: - if x.size != coeffs.shape[0]: - raise RuntimeError("Got sparse vector of size %d; wanted %d" % ( - x.size, coeffs.shape[0])) - elif isinstance(x, RDD): - raise RuntimeError("Bulk predict not yet supported.") - else: - raise TypeError("Argument of type " + type(x).__name__ + " unsupported") - - -# If we weren't given initial weights, take a zero vector of the appropriate -# length. -def _get_initial_weights(initial_weights, data): - if initial_weights is None: - initial_weights = _convert_vector(data.first().features) - if type(initial_weights) == ndarray: - if initial_weights.ndim != 1: - raise TypeError("At least one data element has " - + initial_weights.ndim + " dimensions, which is not 1") - initial_weights = numpy.zeros([initial_weights.shape[0]]) - elif type(initial_weights) == SparseVector: - initial_weights = numpy.zeros([initial_weights.size]) - return initial_weights - - -# train_func should take two parameters, namely data and initial_weights, and -# return the result of a call to the appropriate JVM stub. -# _regression_train_wrapper is responsible for setup and error checking. -def _regression_train_wrapper(sc, train_func, klass, data, initial_weights): - initial_weights = _get_initial_weights(initial_weights, data) - dataBytes = _get_unmangled_labeled_point_rdd(data) - ans = train_func(dataBytes, _serialize_double_vector(initial_weights)) - if len(ans) != 2: - raise RuntimeError("JVM call result had unexpected length") - elif type(ans[0]) != bytearray: - raise RuntimeError("JVM call result had first element of type " - + type(ans[0]).__name__ + " which is not bytearray") - elif type(ans[1]) != float: - raise RuntimeError("JVM call result had second element of type " - + type(ans[0]).__name__ + " which is not float") - return klass(_deserialize_double_vector(ans[0]), ans[1]) - - -# Functions for serializing ALS Rating objects and tuples - -def _serialize_rating(r): - ba = bytearray(16) - intpart = ndarray(shape=[2], buffer=ba, dtype=int32) - doublepart = ndarray(shape=[1], buffer=ba, dtype=float64, offset=8) - intpart[0], intpart[1], doublepart[0] = r - return ba - - -class RatingDeserializer(FramedSerializer): - - def loads(self, string): - res = ndarray(shape=(3, ), buffer=string, dtype=float64, offset=4) - return int(res[0]), int(res[1]), res[2] - - def load_stream(self, stream): - while True: - try: - yield self._read_with_length(stream) - except struct.error: - return - except EOFError: - return - - -def _serialize_tuple(t): - ba = bytearray(8) - intpart = ndarray(shape=[2], buffer=ba, dtype=int32) - intpart[0], intpart[1] = t - return ba - - -# Vector math functions that support all of our vector types - -def _convert_vector(vec): - """ - Convert a vector to a format we support internally. This does - the following: - - * For dense NumPy vectors (ndarray), returns them as is - * For our SparseVector class, returns that as is - * For Python lists, converts them to NumPy vectors - * For scipy.sparse.*_matrix column vectors, converts them to - our own SparseVector type. - - This should be called before passing any data to our algorithms - or attempting to serialize it to Java. - """ - if type(vec) == ndarray or type(vec) == SparseVector: - return vec - elif type(vec) == list: - return array(vec, dtype=float64) - elif _have_scipy: - if _scipy_issparse(vec): - assert vec.shape[1] == 1, "Expected column vector" - csc = vec.tocsc() - return SparseVector(vec.shape[0], csc.indices, csc.data) - raise TypeError("Expected NumPy array, SparseVector, or scipy.sparse matrix") - - -def _squared_distance(v1, v2): - """ - Squared distance of two NumPy or sparse vectors. - - >>> dense1 = array([1., 2.]) - >>> sparse1 = SparseVector(2, [0, 1], [1., 2.]) - >>> dense2 = array([2., 1.]) - >>> sparse2 = SparseVector(2, [0, 1], [2., 1.]) - >>> _squared_distance(dense1, dense2) - 2.0 - >>> _squared_distance(dense1, sparse2) - 2.0 - >>> _squared_distance(sparse1, dense2) - 2.0 - >>> _squared_distance(sparse1, sparse2) - 2.0 - """ - v1 = _convert_vector(v1) - v2 = _convert_vector(v2) - if type(v1) == ndarray and type(v2) == ndarray: - diff = v1 - v2 - return numpy.dot(diff, diff) - elif type(v1) == ndarray: - return v2.squared_distance(v1) - else: - return v1.squared_distance(v2) - - -def _dot(vec, target): - """ - Compute the dot product of a vector of the types we support - (Numpy array, list, SparseVector, or SciPy sparse) and a target - NumPy array that is either 1- or 2-dimensional. Equivalent to - calling numpy.dot of the two vectors, but for SciPy ones, we - have to transpose them because they're column vectors. - """ - if type(vec) == ndarray: - return numpy.dot(vec, target) - elif type(vec) == SparseVector: - return vec.dot(target) - elif type(vec) == list: - return numpy.dot(_convert_vector(vec), target) - else: - return vec.transpose().dot(target)[0] - - -def _test(): - import doctest - globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) - (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) - globs['sc'].stop() - if failure_count: - exit(-1) - - -if __name__ == "__main__": - _test() diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 71ab46b61d7fa..ac142fb49a90c 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -15,19 +15,14 @@ # limitations under the License. # +from math import exp + import numpy +from numpy import array -from numpy import array, shape -from pyspark import SparkContext -from pyspark.mllib._common import \ - _dot, _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ - _linear_predictor_typecheck, _get_unmangled_labeled_point_rdd -from pyspark.mllib.linalg import SparseVector -from pyspark.mllib.regression import LabeledPoint, LinearModel -from math import exp, log +from pyspark import SparkContext, PickleSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.mllib.regression import LabeledPoint, LinearModel, _regression_train_wrapper __all__ = ['LogisticRegressionModel', 'LogisticRegressionWithSGD', 'SVMModel', @@ -67,8 +62,7 @@ class LogisticRegressionModel(LinearModel): """ def predict(self, x): - _linear_predictor_typecheck(x, self._coeff) - margin = _dot(x, self._coeff) + self._intercept + margin = self.weights.dot(x) + self._intercept if margin > 0: prob = 1 / (1 + exp(-margin)) else: @@ -81,7 +75,7 @@ class LogisticRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=1.0, regType=None, intercept=False): + initialWeights=None, regParam=1.0, regType="none", intercept=False): """ Train a logistic regression model on the given data. @@ -106,11 +100,12 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - return _regression_train_wrapper(sc, train_func, LogisticRegressionModel, data, + + def train(jdata, i): + return sc._jvm.PythonMLLibAPI().trainLogisticRegressionModelWithSGD( + jdata, iterations, step, miniBatchFraction, i, regParam, regType, intercept) + + return _regression_train_wrapper(sc, train, LogisticRegressionModel, data, initialWeights) @@ -141,8 +136,7 @@ class SVMModel(LinearModel): """ def predict(self, x): - _linear_predictor_typecheck(x, self._coeff) - margin = _dot(x, self._coeff) + self._intercept + margin = self.weights.dot(x) + self.intercept return 1 if margin >= 0 else 0 @@ -150,7 +144,7 @@ class SVMWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, regParam=1.0, - miniBatchFraction=1.0, initialWeights=None, regType=None, intercept=False): + miniBatchFraction=1.0, initialWeights=None, regType="none", intercept=False): """ Train a support vector machine on the given data. @@ -175,11 +169,12 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) - return _regression_train_wrapper(sc, train_func, SVMModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainSVMModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i, regType, intercept) + + return _regression_train_wrapper(sc, train, SVMModel, data, initialWeights) class NaiveBayesModel(object): @@ -220,7 +215,8 @@ def __init__(self, labels, pi, theta): def predict(self, x): """Return the most likely class for a data vector x""" - return self.labels[numpy.argmax(self.pi + _dot(x, self.theta.transpose()))] + x = _convert_to_vector(x) + return self.labels[numpy.argmax(self.pi + x.dot(self.theta.transpose()))] class NaiveBayes(object): @@ -242,12 +238,9 @@ def train(cls, data, lambda_=1.0): @param lambda_: The smoothing parameter """ sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - ans = sc._jvm.PythonMLLibAPI().trainNaiveBayes(dataBytes._jrdd, lambda_) - return NaiveBayesModel( - _deserialize_double_vector(ans[0]), - _deserialize_double_vector(ans[1]), - _deserialize_double_matrix(ans[2])) + jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(data._to_java_object_rdd(), lambda_) + labels, pi, theta = PickleSerializer().loads(str(sc._jvm.SerDe.dumps(jlist))) + return NaiveBayesModel(labels.toArray(), pi.toArray(), numpy.array(theta)) def _test(): diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index f3e952a1d842a..12c56022717a5 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,15 +15,9 @@ # limitations under the License. # -from numpy import array, dot -from math import sqrt from pyspark import SparkContext -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _squared_distance, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper -from pyspark.mllib.linalg import SparseVector +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer +from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['KMeansModel', 'KMeans'] @@ -32,6 +26,7 @@ class KMeansModel(object): """A clustering model derived from the k-means method. + >>> from numpy import array >>> data = array([0.0,0.0, 1.0,1.0, 9.0,8.0, 8.0,9.0]).reshape(4,2) >>> model = KMeans.train( ... sc.parallelize(data), 2, maxIterations=10, runs=30, initializationMode="random") @@ -71,8 +66,9 @@ def predict(self, x): """Find the cluster to which x belongs in this model.""" best = 0 best_distance = float("inf") - for i in range(0, len(self.centers)): - distance = _squared_distance(x, self.centers[i]) + x = _convert_to_vector(x) + for i in xrange(len(self.centers)): + distance = x.squared_distance(self.centers[i]) if distance < best_distance: best = i best_distance = distance @@ -82,19 +78,17 @@ def predict(self, x): class KMeans(object): @classmethod - def train(cls, data, k, maxIterations=100, runs=1, initializationMode="k-means||"): + def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" - sc = data.context - dataBytes = _get_unmangled_double_vector_rdd(data) - ans = sc._jvm.PythonMLLibAPI().trainKMeansModel( - dataBytes._jrdd, k, maxIterations, runs, initializationMode) - if len(ans) != 1: - raise RuntimeError("JVM call result had unexpected length") - elif type(ans[0]) != bytearray: - raise RuntimeError("JVM call result had first element of type " - + type(ans[0]) + " which is not bytearray") - matrix = _deserialize_double_matrix(ans[0]) - return KMeansModel([row for row in matrix]) + sc = rdd.context + ser = PickleSerializer() + # cache serialized data to avoid objects over head in JVM + cached = rdd.map(_convert_to_vector)._reserialize(AutoBatchedSerializer(ser)).cache() + model = sc._jvm.PythonMLLibAPI().trainKMeansModel( + cached._to_java_object_rdd(), k, maxIterations, runs, initializationMode) + bytes = sc._jvm.SerDe.dumps(model.clusterCenters()) + centers = ser.loads(str(bytes)) + return KMeansModel([c.toArray() for c in centers]) def _test(): diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index e69051c104e37..0a5dcaac55e46 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -23,14 +23,148 @@ SciPy is available in their environment. """ -import numpy -from numpy import array, array_equal, ndarray, float64, int32 +import sys +import array +import copy_reg +import numpy as np -__all__ = ['SparseVector', 'Vectors'] +__all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] -class SparseVector(object): +if sys.version_info[:2] == (2, 7): + # speed up pickling array in Python 2.7 + def fast_pickle_array(ar): + return array.array, (ar.typecode, ar.tostring()) + copy_reg.pickle(array.array, fast_pickle_array) + + +# Check whether we have SciPy. MLlib works without it too, but if we have it, some methods, +# such as _dot and _serialize_double_vector, start to support scipy.sparse matrices. + +try: + import scipy.sparse + _have_scipy = True +except: + # No SciPy in environment, but that's okay + _have_scipy = False + + +def _convert_to_vector(l): + if isinstance(l, Vector): + return l + elif type(l) in (array.array, np.array, np.ndarray, list, tuple): + return DenseVector(l) + elif _have_scipy and scipy.sparse.issparse(l): + assert l.shape[1] == 1, "Expected column vector" + csc = l.tocsc() + return SparseVector(l.shape[0], csc.indices, csc.data) + else: + raise TypeError("Cannot convert type %s into Vector" % type(l)) + + +class Vector(object): + """ + Abstract class for DenseVector and SparseVector + """ + def toArray(self): + """ + Convert the vector into an numpy.ndarray + :return: numpy.ndarray + """ + raise NotImplementedError + + +class DenseVector(Vector): + def __init__(self, ar): + if not isinstance(ar, array.array): + ar = array.array('d', ar) + self.array = ar + + def __reduce__(self): + return DenseVector, (self.array,) + + def dot(self, other): + """ + Compute the dot product of two Vectors. We support + (Numpy array, list, SparseVector, or SciPy sparse) + and a target NumPy array that is either 1- or 2-dimensional. + Equivalent to calling numpy.dot of the two vectors. + + >>> dense = DenseVector(array.array('d', [1., 2.])) + >>> dense.dot(dense) + 5.0 + >>> dense.dot(SparseVector(2, [0, 1], [2., 1.])) + 4.0 + >>> dense.dot(range(1, 3)) + 5.0 + >>> dense.dot(np.array(range(1, 3))) + 5.0 + """ + if isinstance(other, SparseVector): + return other.dot(self) + elif _have_scipy and scipy.sparse.issparse(other): + return other.transpose().dot(self.toArray())[0] + elif isinstance(other, Vector): + return np.dot(self.toArray(), other.toArray()) + else: + return np.dot(self.toArray(), other) + + def squared_distance(self, other): + """ + Squared distance of two Vectors. + + >>> dense1 = DenseVector(array.array('d', [1., 2.])) + >>> dense1.squared_distance(dense1) + 0.0 + >>> dense2 = np.array([2., 1.]) + >>> dense1.squared_distance(dense2) + 2.0 + >>> dense3 = [2., 1.] + >>> dense1.squared_distance(dense3) + 2.0 + >>> sparse1 = SparseVector(2, [0, 1], [2., 1.]) + >>> dense1.squared_distance(sparse1) + 2.0 + """ + if isinstance(other, SparseVector): + return other.squared_distance(self) + elif _have_scipy and scipy.sparse.issparse(other): + return _convert_to_vector(other).squared_distance(self) + + if isinstance(other, Vector): + other = other.toArray() + elif not isinstance(other, np.ndarray): + other = np.array(other) + diff = self.toArray() - other + return np.dot(diff, diff) + + def toArray(self): + return np.array(self.array) + + def __getitem__(self, item): + return self.array[item] + + def __len__(self): + return len(self.array) + + def __str__(self): + return "[" + ",".join([str(v) for v in self.array]) + "]" + + def __repr__(self): + return "DenseVector(%r)" % self.array + + def __eq__(self, other): + return isinstance(other, DenseVector) and self.array == other.array + + def __ne__(self, other): + return not self == other + + def __getattr__(self, item): + return getattr(self.array, item) + + +class SparseVector(Vector): """ A simple sparse vector class for passing data to MLlib. Users may @@ -61,16 +195,19 @@ def __init__(self, size, *args): if type(pairs) == dict: pairs = pairs.items() pairs = sorted(pairs) - self.indices = array([p[0] for p in pairs], dtype=int32) - self.values = array([p[1] for p in pairs], dtype=float64) + self.indices = array.array('i', [p[0] for p in pairs]) + self.values = array.array('d', [p[1] for p in pairs]) else: assert len(args[0]) == len(args[1]), "index and value arrays not same length" - self.indices = array(args[0], dtype=int32) - self.values = array(args[1], dtype=float64) + self.indices = array.array('i', args[0]) + self.values = array.array('d', args[1]) for i in xrange(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: raise TypeError("indices array must be sorted") + def __reduce__(self): + return (SparseVector, (self.size, self.indices, self.values)) + def dot(self, other): """ Dot product with a SparseVector or 1- or 2-dimensional Numpy array. @@ -78,15 +215,15 @@ def dot(self, other): >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) >>> a.dot(a) 25.0 - >>> a.dot(array([1., 2., 3., 4.])) + >>> a.dot(array.array('d', [1., 2., 3., 4.])) 22.0 >>> b = SparseVector(4, [2, 4], [1.0, 2.0]) >>> a.dot(b) 0.0 - >>> a.dot(array([[1, 1], [2, 2], [3, 3], [4, 4]])) + >>> a.dot(np.array([[1, 1], [2, 2], [3, 3], [4, 4]])) array([ 22., 22.]) """ - if type(other) == ndarray: + if type(other) == np.ndarray: if other.ndim == 1: result = 0.0 for i in xrange(len(self.indices)): @@ -94,10 +231,17 @@ def dot(self, other): return result elif other.ndim == 2: results = [self.dot(other[:, i]) for i in xrange(other.shape[1])] - return array(results) + return np.array(results) else: raise Exception("Cannot call dot with %d-dimensional array" % other.ndim) - else: + + elif type(other) in (array.array, DenseVector): + result = 0.0 + for i in xrange(len(self.indices)): + result += self.values[i] * other[self.indices[i]] + return result + + elif type(other) is SparseVector: result = 0.0 i, j = 0, 0 while i < len(self.indices) and j < len(other.indices): @@ -110,6 +254,8 @@ def dot(self, other): else: j += 1 return result + else: + return self.dot(_convert_to_vector(other)) def squared_distance(self, other): """ @@ -118,7 +264,9 @@ def squared_distance(self, other): >>> a = SparseVector(4, [1, 3], [3.0, 4.0]) >>> a.squared_distance(a) 0.0 - >>> a.squared_distance(array([1., 2., 3., 4.])) + >>> a.squared_distance(array.array('d', [1., 2., 3., 4.])) + 11.0 + >>> a.squared_distance(np.array([1., 2., 3., 4.])) 11.0 >>> b = SparseVector(4, [2, 4], [1.0, 2.0]) >>> a.squared_distance(b) @@ -126,22 +274,22 @@ def squared_distance(self, other): >>> b.squared_distance(a) 30.0 """ - if type(other) == ndarray: - if other.ndim == 1: - result = 0.0 - j = 0 # index into our own array - for i in xrange(other.shape[0]): - if j < len(self.indices) and self.indices[j] == i: - diff = self.values[j] - other[i] - result += diff * diff - j += 1 - else: - result += other[i] * other[i] - return result - else: + if type(other) in (list, array.array, DenseVector, np.array, np.ndarray): + if type(other) is np.array and other.ndim != 1: raise Exception("Cannot call squared_distance with %d-dimensional array" % other.ndim) - else: + result = 0.0 + j = 0 # index into our own array + for i in xrange(len(other)): + if j < len(self.indices) and self.indices[j] == i: + diff = self.values[j] - other[i] + result += diff * diff + j += 1 + else: + result += other[i] * other[i] + return result + + elif type(other) is SparseVector: result = 0.0 i, j = 0, 0 while i < len(self.indices) and j < len(other.indices): @@ -163,16 +311,21 @@ def squared_distance(self, other): result += other.values[j] * other.values[j] j += 1 return result + else: + return self.squared_distance(_convert_to_vector(other)) def toArray(self): """ Returns a copy of this SparseVector as a 1-dimensional NumPy array. """ - arr = numpy.zeros(self.size) + arr = np.zeros((self.size,), dtype=np.float64) for i in xrange(self.indices.size): arr[self.indices[i]] = self.values[i] return arr + def __len__(self): + return self.size + def __str__(self): inds = "[" + ",".join([str(i) for i in self.indices]) + "]" vals = "[" + ",".join([str(v) for v in self.values]) + "]" @@ -198,8 +351,8 @@ def __eq__(self, other): return (isinstance(other, self.__class__) and other.size == self.size - and array_equal(other.indices, self.indices) - and array_equal(other.values, self.values)) + and other.indices == self.indices + and other.values == self.values) def __ne__(self, other): return not self.__eq__(other) @@ -242,9 +395,9 @@ def dense(elements): returns a NumPy array. >>> Vectors.dense([1, 2, 3]) - array([ 1., 2., 3.]) + DenseVector(array('d', [1.0, 2.0, 3.0])) """ - return array(elements, dtype=float64) + return DenseVector(elements) @staticmethod def stringify(vector): @@ -257,10 +410,39 @@ def stringify(vector): >>> Vectors.stringify(Vectors.dense([0.0, 1.0])) '[0.0,1.0]' """ - if type(vector) == SparseVector: - return str(vector) - else: - return "[" + ",".join([str(v) for v in vector]) + "]" + return str(vector) + + +class Matrix(object): + """ the Matrix """ + def __init__(self, nRow, nCol): + self.nRow = nRow + self.nCol = nCol + + def toArray(self): + raise NotImplementedError + + +class DenseMatrix(Matrix): + def __init__(self, nRow, nCol, values): + Matrix.__init__(self, nRow, nCol) + assert len(values) == nRow * nCol + self.values = values + + def __reduce__(self): + return DenseMatrix, (self.nRow, self.nCol, self.values) + + def toArray(self): + """ + Return an numpy.ndarray + + >>> arr = array.array('d', [float(i) for i in range(4)]) + >>> m = DenseMatrix(2, 2, arr) + >>> m.toArray() + array([[ 0., 1.], + [ 2., 3.]]) + """ + return np.ndarray((self.nRow, self.nCol), np.float64, buffer=self.values.tostring()) def _test(): diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index d53c95fd59c25..a787e4dea2c55 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -19,15 +19,32 @@ Python package for random data generation. """ +from functools import wraps from pyspark.rdd import RDD -from pyspark.mllib._common import _deserialize_double, _deserialize_double_vector -from pyspark.serializers import NoOpSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer __all__ = ['RandomRDDs', ] +def serialize(f): + @wraps(f) + def func(sc, *a, **kw): + jrdd = f(sc, *a, **kw) + return RDD(sc._jvm.PythonRDD.javaToPython(jrdd), sc, + BatchedSerializer(PickleSerializer(), 1024)) + return func + + +def toArray(f): + @wraps(f) + def func(sc, *a, **kw): + rdd = f(sc, *a, **kw) + return rdd.map(lambda vec: vec.toArray()) + return func + + class RandomRDDs(object): """ Generator methods for creating RDDs comprised of i.i.d samples from @@ -35,6 +52,7 @@ class RandomRDDs(object): """ @staticmethod + @serialize def uniformRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the @@ -56,11 +74,10 @@ def uniformRDD(sc, size, numPartitions=None, seed=None): >>> parts == sc.defaultParallelism True """ - jrdd = sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) - return uniform.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().uniformRDD(sc._jsc, size, numPartitions, seed) @staticmethod + @serialize def normalRDD(sc, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the standard normal @@ -80,11 +97,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - 1.0) < 0.1 True """ - jrdd = sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) - return normal.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().normalRDD(sc._jsc, size, numPartitions, seed) @staticmethod + @serialize def poissonRDD(sc, mean, size, numPartitions=None, seed=None): """ Generates an RDD comprised of i.i.d. samples from the Poisson @@ -101,11 +117,11 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): >>> abs(stats.stdev() - sqrt(mean)) < 0.5 True """ - jrdd = sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) - return poisson.map(lambda bytes: _deserialize_double(bytearray(bytes))) + return sc._jvm.PythonMLLibAPI().poissonRDD(sc._jsc, mean, size, numPartitions, seed) @staticmethod + @toArray + @serialize def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -120,12 +136,12 @@ def uniformVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> RandomRDDs.uniformVectorRDD(sc, 10, 10, 4).getNumPartitions() 4 """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .uniformVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - uniform = RDD(jrdd, sc, NoOpSerializer()) - return uniform.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod + @toArray + @serialize def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -140,12 +156,12 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - 1.0) < 0.1 True """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .normalVectorRDD(sc._jsc, numRows, numCols, numPartitions, seed) - normal = RDD(jrdd, sc, NoOpSerializer()) - return normal.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) @staticmethod + @toArray + @serialize def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): """ Generates an RDD comprised of vectors containing i.i.d. samples drawn @@ -163,10 +179,8 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> abs(mat.std() - sqrt(mean)) < 0.5 True """ - jrdd = sc._jvm.PythonMLLibAPI() \ + return sc._jvm.PythonMLLibAPI() \ .poissonVectorRDD(sc._jsc, mean, numRows, numCols, numPartitions, seed) - poisson = RDD(jrdd, sc, NoOpSerializer()) - return poisson.map(lambda bytes: _deserialize_double_vector(bytearray(bytes))) def _test(): diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 2df23394da6f8..59c1c5ff0ced0 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -16,17 +16,25 @@ # from pyspark import SparkContext -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, \ - _serialize_double_matrix, _deserialize_double_matrix, \ - _serialize_double_vector, _deserialize_double_vector, \ - _get_initial_weights, _serialize_rating, _regression_train_wrapper, \ - _serialize_tuple, RatingDeserializer +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer from pyspark.rdd import RDD __all__ = ['MatrixFactorizationModel', 'ALS'] +class Rating(object): + def __init__(self, user, product, rating): + self.user = int(user) + self.product = int(product) + self.rating = float(rating) + + def __reduce__(self): + return Rating, (self.user, self.product, self.rating) + + def __repr__(self): + return "Rating(%d, %d, %d)" % (self.user, self.product, self.rating) + + class MatrixFactorizationModel(object): """A matrix factorisation model trained by regularized alternating @@ -39,7 +47,9 @@ class MatrixFactorizationModel(object): >>> model = ALS.trainImplicit(ratings, 1) >>> model.predict(2,2) is not None True + >>> testset = sc.parallelize([(1, 2), (1, 1)]) + >>> model = ALS.train(ratings, 1) >>> model.predictAll(testset).count() == 2 True """ @@ -54,34 +64,61 @@ def __del__(self): def predict(self, user, product): return self._java_model.predict(user, product) - def predictAll(self, usersProducts): - usersProductsJRDD = _get_unmangled_rdd(usersProducts, _serialize_tuple) - return RDD(self._java_model.predict(usersProductsJRDD._jrdd), - self._context, RatingDeserializer()) + def predictAll(self, user_product): + assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" + first = user_product.first() + if isinstance(first, list): + user_product = user_product.map(tuple) + first = tuple(first) + assert type(first) is tuple and len(first) == 2, \ + "user_product should be RDD of (user, product)" + if any(isinstance(x, str) for x in first): + user_product = user_product.map(lambda (u, p): (int(x), int(p))) + first = tuple(map(int, first)) + assert all(type(x) is int for x in first), "user and product in user_product shoul be int" + sc = self._context + tuplerdd = sc._jvm.SerDe.asTupleRDD(user_product._to_java_object_rdd().rdd()) + jresult = self._java_model.predict(tuplerdd).toJavaRDD() + return RDD(sc._jvm.PythonRDD.javaToPython(jresult), sc, + AutoBatchedSerializer(PickleSerializer())) class ALS(object): + @classmethod + def _prepare(cls, ratings): + assert isinstance(ratings, RDD), "ratings should be RDD" + first = ratings.first() + if not isinstance(first, Rating): + if isinstance(first, (tuple, list)): + ratings = ratings.map(lambda x: Rating(*x)) + else: + raise ValueError("rating should be RDD of Rating or tuple/list") + # serialize them by AutoBatchedSerializer before cache to reduce the + # objects overhead in JVM + cached = ratings._reserialize(AutoBatchedSerializer(PickleSerializer())).cache() + return cached._to_java_object_rdd() + @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1): sc = ratings.context - ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) - mod = sc._jvm.PythonMLLibAPI().trainALSModel( - ratingBytes._jrdd, rank, iterations, lambda_, blocks) + jrating = cls._prepare(ratings) + mod = sc._jvm.PythonMLLibAPI().trainALSModel(jrating, rank, iterations, lambda_, blocks) return MatrixFactorizationModel(sc, mod) @classmethod def trainImplicit(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, alpha=0.01): sc = ratings.context - ratingBytes = _get_unmangled_rdd(ratings, _serialize_rating) + jrating = cls._prepare(ratings) mod = sc._jvm.PythonMLLibAPI().trainImplicitALSModel( - ratingBytes._jrdd, rank, iterations, lambda_, blocks, alpha) + jrating, rank, iterations, lambda_, blocks, alpha) return MatrixFactorizationModel(sc, mod) def _test(): import doctest - globs = globals().copy() + import pyspark.mllib.recommendation + globs = pyspark.mllib.recommendation.__dict__.copy() globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index f572dcfb840b6..cbdbc09858013 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -15,12 +15,12 @@ # limitations under the License. # -from numpy import array, ndarray -from pyspark import SparkContext -from pyspark.mllib._common import _dot, _regression_train_wrapper, \ - _linear_predictor_typecheck, _have_scipy, _scipy_issparse -from pyspark.mllib.linalg import SparseVector, Vectors +import numpy as np +from numpy import array +from pyspark import SparkContext +from pyspark.mllib.linalg import SparseVector, _convert_to_vector +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel' 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -38,16 +38,16 @@ class LabeledPoint(object): def __init__(self, label, features): self.label = label - if (type(features) == ndarray or type(features) == SparseVector - or (_have_scipy and _scipy_issparse(features))): - self.features = features - elif type(features) == list: - self.features = array(features) - else: - raise TypeError("Expected NumPy array, list, SparseVector, or scipy.sparse matrix") + self.features = _convert_to_vector(features) + + def __reduce__(self): + return (LabeledPoint, (self.label, self.features)) def __str__(self): - return "(" + ",".join((str(self.label), Vectors.stringify(self.features))) + ")" + return "(" + ",".join((str(self.label), str(self.features))) + ")" + + def __repr__(self): + return "LabeledPoint(" + ",".join((repr(self.label), repr(self.features))) + ")" class LinearModel(object): @@ -55,7 +55,7 @@ class LinearModel(object): """A linear model that has a vector of coefficients and an intercept.""" def __init__(self, weights, intercept): - self._coeff = weights + self._coeff = _convert_to_vector(weights) self._intercept = intercept @property @@ -71,18 +71,19 @@ class LinearRegressionModelBase(LinearModel): """A linear regression model. - >>> lrmb = LinearRegressionModelBase(array([1.0, 2.0]), 0.1) - >>> abs(lrmb.predict(array([-1.03, 7.777])) - 14.624) < 1e-6 + >>> lrmb = LinearRegressionModelBase(np.array([1.0, 2.0]), 0.1) + >>> abs(lrmb.predict(np.array([-1.03, 7.777])) - 14.624) < 1e-6 True >>> abs(lrmb.predict(SparseVector(2, {0: -1.03, 1: 7.777})) - 14.624) < 1e-6 True """ def predict(self, x): - """Predict the value of the dependent variable given a vector x""" - """containing values for the independent variables.""" - _linear_predictor_typecheck(x, self._coeff) - return _dot(x, self._coeff) + self._intercept + """ + Predict the value of the dependent variable given a vector x + containing values for the independent variables. + """ + return self.weights.dot(x) + self.intercept class LinearRegressionModel(LinearRegressionModelBase): @@ -96,10 +97,10 @@ class LinearRegressionModel(LinearRegressionModelBase): ... LabeledPoint(3.0, [2.0]), ... LabeledPoint(2.0, [3.0]) ... ] - >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=np.array([1.0])) + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -117,11 +118,27 @@ class LinearRegressionModel(LinearRegressionModelBase): """ +# train_func should take two parameters, namely data and initial_weights, and +# return the result of a call to the appropriate JVM stub. +# _regression_train_wrapper is responsible for setup and error checking. +def _regression_train_wrapper(sc, train_func, modelClass, data, initial_weights): + initial_weights = initial_weights or [0.0] * len(data.first().features) + ser = PickleSerializer() + initial_bytes = bytearray(ser.dumps(_convert_to_vector(initial_weights))) + # use AutoBatchedSerializer before cache to reduce the memory + # overhead in JVM + cached = data._reserialize(AutoBatchedSerializer(ser)).cache() + ans = train_func(cached._to_java_object_rdd(), initial_bytes) + assert len(ans) == 2, "JVM call result had unexpected length" + weights = ser.loads(str(ans[0])) + return modelClass(weights, ans[1]) + + class LinearRegressionWithSGD(object): @classmethod def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, - initialWeights=None, regParam=1.0, regType=None, intercept=False): + initialWeights=None, regParam=1.0, regType="none", intercept=False): """ Train a linear regression model on the given data. @@ -146,11 +163,12 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, are activated or not). """ sc = data.context - if regType is None: - regType = "none" - train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( - d._jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) - return _regression_train_wrapper(sc, train_f, LinearRegressionModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainLinearRegressionModelWithSGD( + jrdd, iterations, step, miniBatchFraction, i, regParam, regType, intercept) + + return _regression_train_wrapper(sc, train, LinearRegressionModel, data, initialWeights) class LassoModel(LinearRegressionModelBase): @@ -166,9 +184,9 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(2.0, [3.0]) ... ] >>> lrm = LassoWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -179,7 +197,7 @@ class LassoModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -193,9 +211,11 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a Lasso regression model on the given data.""" sc = data.context - train_f = lambda d, i: sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train_f, LassoModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainLassoModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i) + return _regression_train_wrapper(sc, train, LassoModel, data, initialWeights) class RidgeRegressionModel(LinearRegressionModelBase): @@ -211,9 +231,9 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, [3.0]) ... ] >>> lrm = RidgeRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True - >>> abs(lrm.predict(array([1.0])) - 1) < 0.5 + >>> abs(lrm.predict(np.array([1.0])) - 1) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -224,7 +244,7 @@ class RidgeRegressionModel(LinearRegressionModelBase): ... LabeledPoint(2.0, SparseVector(1, {0: 3.0})) ... ] >>> lrm = LinearRegressionWithSGD.train(sc.parallelize(data), initialWeights=array([1.0])) - >>> abs(lrm.predict(array([0.0])) - 0) < 0.5 + >>> abs(lrm.predict(np.array([0.0])) - 0) < 0.5 True >>> abs(lrm.predict(SparseVector(1, {0: 1.0})) - 1) < 0.5 True @@ -238,9 +258,12 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, miniBatchFraction=1.0, initialWeights=None): """Train a ridge regression model on the given data.""" sc = data.context - train_func = lambda d, i: sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( - d._jrdd, iterations, step, regParam, miniBatchFraction, i) - return _regression_train_wrapper(sc, train_func, RidgeRegressionModel, data, initialWeights) + + def train(jrdd, i): + return sc._jvm.PythonMLLibAPI().trainRidgeModelWithSGD( + jrdd, iterations, step, regParam, miniBatchFraction, i) + + return _regression_train_wrapper(sc, train, RidgeRegressionModel, data, initialWeights) def _test(): diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index 8c726f171c978..b9de0909a6fb1 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -19,14 +19,26 @@ Python package for statistical functions in MLlib. """ -from pyspark.mllib._common import \ - _get_unmangled_double_vector_rdd, _get_unmangled_rdd, \ - _serialize_double, _deserialize_double_matrix, _deserialize_double_vector +from functools import wraps + +from pyspark import PickleSerializer __all__ = ['MultivariateStatisticalSummary', 'Statistics'] +def serialize(f): + ser = PickleSerializer() + + @wraps(f) + def func(self): + jvec = f(self) + bytes = self._sc._jvm.SerDe.dumps(jvec) + return ser.loads(str(bytes)).toArray() + + return func + + class MultivariateStatisticalSummary(object): """ @@ -44,33 +56,38 @@ def __init__(self, sc, java_summary): def __del__(self): self._sc._gateway.detach(self._java_summary) + @serialize def mean(self): - return _deserialize_double_vector(self._java_summary.mean()) + return self._java_summary.mean() + @serialize def variance(self): - return _deserialize_double_vector(self._java_summary.variance()) + return self._java_summary.variance() def count(self): return self._java_summary.count() + @serialize def numNonzeros(self): - return _deserialize_double_vector(self._java_summary.numNonzeros()) + return self._java_summary.numNonzeros() + @serialize def max(self): - return _deserialize_double_vector(self._java_summary.max()) + return self._java_summary.max() + @serialize def min(self): - return _deserialize_double_vector(self._java_summary.min()) + return self._java_summary.min() class Statistics(object): @staticmethod - def colStats(X): + def colStats(rdd): """ Computes column-wise summary statistics for the input RDD[Vector]. - >>> from linalg import Vectors + >>> from pyspark.mllib.linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([2, 0, 0, -2]), ... Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8])]) @@ -88,9 +105,9 @@ def colStats(X): >>> cStats.min() array([ 2., 0., 0., -2.]) """ - sc = X.ctx - Xser = _get_unmangled_double_vector_rdd(X) - cStats = sc._jvm.PythonMLLibAPI().colStats(Xser._jrdd) + sc = rdd.ctx + jrdd = rdd._to_java_object_rdd() + cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) return MultivariateStatisticalSummary(sc, cStats) @staticmethod @@ -117,7 +134,7 @@ def corr(x, y=None, method=None): >>> from math import isnan >>> isnan(Statistics.corr(x, zeros)) True - >>> from linalg import Vectors + >>> from pyspark.mllib.linalg import Vectors >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) >>> pearsonCorr = Statistics.corr(rdd) @@ -144,18 +161,16 @@ def corr(x, y=None, method=None): # check if y is used to specify the method name instead. if type(y) == str: raise TypeError("Use 'method=' to specify method name.") + + jx = x._to_java_object_rdd() if not y: - try: - Xser = _get_unmangled_double_vector_rdd(x) - except TypeError: - raise TypeError("corr called on a single RDD not consisted of Vectors.") - resultMat = sc._jvm.PythonMLLibAPI().corr(Xser._jrdd, method) - return _deserialize_double_matrix(resultMat) + resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) + bytes = sc._jvm.SerDe.dumps(resultMat) + ser = PickleSerializer() + return ser.loads(str(bytes)).toArray() else: - xSer = _get_unmangled_rdd(x, _serialize_double) - ySer = _get_unmangled_rdd(y, _serialize_double) - result = sc._jvm.PythonMLLibAPI().corr(xSer._jrdd, ySer._jrdd, method) - return result + jy = y._to_java_object_rdd() + return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) def _test(): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8a851bd35c0e8..f72e88ba6e2ba 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -20,6 +20,8 @@ """ import sys +import array as pyarray + from numpy import array, array_equal if sys.version_info[:2] <= (2, 6): @@ -27,9 +29,8 @@ else: import unittest -from pyspark.mllib._common import _convert_vector, _serialize_double_vector, \ - _deserialize_double_vector, _dot, _squared_distance -from pyspark.mllib.linalg import SparseVector +from pyspark.serializers import PickleSerializer +from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint from pyspark.tests import PySparkTestCase @@ -42,39 +43,52 @@ # No SciPy, but that's okay, we'll skip those tests pass +ser = PickleSerializer() + + +def _squared_distance(a, b): + if isinstance(a, Vector): + return a.squared_distance(b) + else: + return b.squared_distance(a) -class VectorTests(unittest.TestCase): + +class VectorTests(PySparkTestCase): + + def _test_serialize(self, v): + jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) + nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) + self.assertEqual(v, nv) + vs = [v] * 100 + jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) + nvs = ser.loads(str(self.sc._jvm.SerDe.dumps(jvecs))) + self.assertEqual(vs, nvs) def test_serialize(self): - sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [1, 2, 3, 4] - self.assertTrue(sv is _convert_vector(sv)) - self.assertTrue(dv is _convert_vector(dv)) - self.assertTrue(array_equal(dv, _convert_vector(lst))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(sv))) - self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(dv)))) - self.assertTrue(array_equal(dv, _deserialize_double_vector(_serialize_double_vector(lst)))) + self._test_serialize(DenseVector(range(10))) + self._test_serialize(DenseVector(array([1., 2., 3., 4.]))) + self._test_serialize(DenseVector(pyarray.array('d', range(10)))) + self._test_serialize(SparseVector(4, {1: 1, 3: 2})) def test_dot(self): sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [1, 2, 3, 4] + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([1, 2, 3, 4]) mat = array([[1., 2., 3., 4.], [1., 2., 3., 4.], [1., 2., 3., 4.], [1., 2., 3., 4.]]) - self.assertEquals(10.0, _dot(sv, dv)) - self.assertTrue(array_equal(array([3., 6., 9., 12.]), _dot(sv, mat))) - self.assertEquals(30.0, _dot(dv, dv)) - self.assertTrue(array_equal(array([10., 20., 30., 40.]), _dot(dv, mat))) - self.assertEquals(30.0, _dot(lst, dv)) - self.assertTrue(array_equal(array([10., 20., 30., 40.]), _dot(lst, mat))) + self.assertEquals(10.0, sv.dot(dv)) + self.assertTrue(array_equal(array([3., 6., 9., 12.]), sv.dot(mat))) + self.assertEquals(30.0, dv.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), dv.dot(mat))) + self.assertEquals(30.0, lst.dot(dv)) + self.assertTrue(array_equal(array([10., 20., 30., 40.]), lst.dot(mat))) def test_squared_distance(self): sv = SparseVector(4, {1: 1, 3: 2}) - dv = array([1., 2., 3., 4.]) - lst = [4, 3, 2, 1] + dv = DenseVector(array([1., 2., 3., 4.])) + lst = DenseVector([4, 3, 2, 1]) self.assertEquals(15.0, _squared_distance(sv, dv)) self.assertEquals(25.0, _squared_distance(sv, lst)) self.assertEquals(20.0, _squared_distance(dv, lst)) @@ -198,41 +212,36 @@ def test_serialize(self): lil[1, 0] = 1 lil[3, 0] = 2 sv = SparseVector(4, {1: 1, 3: 2}) - self.assertEquals(sv, _convert_vector(lil)) - self.assertEquals(sv, _convert_vector(lil.tocsc())) - self.assertEquals(sv, _convert_vector(lil.tocoo())) - self.assertEquals(sv, _convert_vector(lil.tocsr())) - self.assertEquals(sv, _convert_vector(lil.todok())) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsc()))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.tocsr()))) - self.assertEquals(sv, _deserialize_double_vector(_serialize_double_vector(lil.todok()))) + self.assertEquals(sv, _convert_to_vector(lil)) + self.assertEquals(sv, _convert_to_vector(lil.tocsc())) + self.assertEquals(sv, _convert_to_vector(lil.tocoo())) + self.assertEquals(sv, _convert_to_vector(lil.tocsr())) + self.assertEquals(sv, _convert_to_vector(lil.todok())) + + def serialize(l): + return ser.loads(ser.dumps(_convert_to_vector(l))) + self.assertEquals(sv, serialize(lil)) + self.assertEquals(sv, serialize(lil.tocsc())) + self.assertEquals(sv, serialize(lil.tocsr())) + self.assertEquals(sv, serialize(lil.todok())) def test_dot(self): from scipy.sparse import lil_matrix lil = lil_matrix((4, 1)) lil[1, 0] = 1 lil[3, 0] = 2 - dv = array([1., 2., 3., 4.]) - sv = SparseVector(4, {0: 1, 1: 2, 2: 3, 3: 4}) - mat = array([[1., 2., 3., 4.], - [1., 2., 3., 4.], - [1., 2., 3., 4.], - [1., 2., 3., 4.]]) - self.assertEquals(10.0, _dot(lil, dv)) - self.assertTrue(array_equal(array([3., 6., 9., 12.]), _dot(lil, mat))) + dv = DenseVector(array([1., 2., 3., 4.])) + self.assertEquals(10.0, dv.dot(lil)) def test_squared_distance(self): from scipy.sparse import lil_matrix lil = lil_matrix((4, 1)) lil[1, 0] = 3 lil[3, 0] = 2 - dv = array([1., 2., 3., 4.]) + dv = DenseVector(array([1., 2., 3., 4.])) sv = SparseVector(4, {0: 1, 1: 2, 2: 3, 3: 4}) - self.assertEquals(15.0, _squared_distance(lil, dv)) - self.assertEquals(15.0, _squared_distance(lil, sv)) - self.assertEquals(15.0, _squared_distance(dv, lil)) - self.assertEquals(15.0, _squared_distance(sv, lil)) + self.assertEquals(15.0, dv.squared_distance(lil)) + self.assertEquals(15.0, sv.squared_distance(lil)) def scipy_matrix(self, size, values): """Create a column SciPy matrix from a dictionary of values""" diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 5b13ab682bbfc..f59a818a6e74d 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -18,13 +18,9 @@ from py4j.java_collections import MapConverter from pyspark import SparkContext, RDD -from pyspark.mllib._common import \ - _get_unmangled_rdd, _get_unmangled_double_vector_rdd, _serialize_double_vector, \ - _deserialize_labeled_point, _get_unmangled_labeled_point_rdd, \ - _deserialize_double +from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.linalg import Vector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint -from pyspark.serializers import NoOpSerializer - __all__ = ['DecisionTreeModel', 'DecisionTree'] @@ -55,21 +51,24 @@ def predict(self, x): :param x: Data point (feature vector), or an RDD of data points (feature vectors). """ - pythonAPI = self._sc._jvm.PythonMLLibAPI() + SerDe = self._sc._jvm.SerDe + ser = PickleSerializer() if isinstance(x, RDD): # Bulk prediction - if x.count() == 0: + first = x.take(1) + if not first: return self._sc.parallelize([]) - dataBytes = _get_unmangled_double_vector_rdd(x, cache=False) - jSerializedPreds = \ - pythonAPI.predictDecisionTreeModel(self._java_model, - dataBytes._jrdd) - serializedPreds = RDD(jSerializedPreds, self._sc, NoOpSerializer()) - return serializedPreds.map(lambda bytes: _deserialize_double(bytearray(bytes))) + if not isinstance(first[0], Vector): + x = x.map(_convert_to_vector) + jPred = self._java_model.predict(x._to_java_object_rdd()).toJavaRDD() + jpyrdd = self._sc._jvm.PythonRDD.javaToPython(jPred) + return RDD(jpyrdd, self._sc, BatchedSerializer(ser, 1024)) + else: # Assume x is a single data point. - x_ = _serialize_double_vector(x) - return pythonAPI.predictDecisionTreeModel(self._java_model, x_) + bytes = bytearray(ser.dumps(_convert_to_vector(x))) + vec = self._sc._jvm.SerDe.loads(bytes) + return self._java_model.predict(vec) def numNodes(self): return self._java_model.numNodes() @@ -77,7 +76,7 @@ def numNodes(self): def depth(self): return self._java_model.depth() - def __str__(self): + def __repr__(self): return self._java_model.toString() @@ -90,52 +89,23 @@ class DecisionTree(object): EXPERIMENTAL: This is an experimental API. It will probably be modified for Spark v1.2. - Example usage: - - >>> from numpy import array - >>> import sys - >>> from pyspark.mllib.regression import LabeledPoint - >>> from pyspark.mllib.tree import DecisionTree - >>> from pyspark.mllib.linalg import SparseVector - >>> - >>> data = [ - ... LabeledPoint(0.0, [0.0]), - ... LabeledPoint(1.0, [1.0]), - ... LabeledPoint(1.0, [2.0]), - ... LabeledPoint(1.0, [3.0]) - ... ] - >>> categoricalFeaturesInfo = {} # no categorical features - >>> model = DecisionTree.trainClassifier(sc.parallelize(data), numClasses=2, - ... categoricalFeaturesInfo=categoricalFeaturesInfo) - >>> sys.stdout.write(model) - DecisionTreeModel classifier - If (feature 0 <= 0.5) - Predict: 0.0 - Else (feature 0 > 0.5) - Predict: 1.0 - >>> model.predict(array([1.0])) > 0 - True - >>> model.predict(array([0.0])) == 0 - True - >>> sparse_data = [ - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), - ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), - ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), - ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) - ... ] - >>> - >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), - ... categoricalFeaturesInfo=categoricalFeaturesInfo) - >>> model.predict(array([0.0, 1.0])) == 1 - True - >>> model.predict(array([0.0, 0.0])) == 0 - True - >>> model.predict(SparseVector(2, {1: 1.0})) == 1 - True - >>> model.predict(SparseVector(2, {1: 0.0})) == 0 - True """ + @staticmethod + def _train(data, type, numClasses, categoricalFeaturesInfo, + impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, + minInfoGain=0.0): + first = data.first() + assert isinstance(first, LabeledPoint), "the data should be RDD of LabeledPoint" + sc = data.context + jrdd = data._to_java_object_rdd() + cfiMap = MapConverter().convert(categoricalFeaturesInfo, + sc._gateway._gateway_client) + model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( + jrdd, type, numClasses, cfiMap, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) + return DecisionTreeModel(sc, model) + @staticmethod def trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity="gini", maxDepth=5, maxBins=32, minInstancesPerNode=1, @@ -159,18 +129,34 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, the parent split :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel + + Example usage: + + >>> from numpy import array + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> from pyspark.mllib.linalg import SparseVector + >>> + >>> data = [ + ... LabeledPoint(0.0, [0.0]), + ... LabeledPoint(1.0, [1.0]), + ... LabeledPoint(1.0, [2.0]), + ... LabeledPoint(1.0, [3.0]) + ... ] + >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) + >>> print model, # it already has newline + DecisionTreeModel classifier + If (feature 0 <= 0.5) + Predict: 0.0 + Else (feature 0 > 0.5) + Predict: 1.0 + >>> model.predict(array([1.0])) > 0 + True + >>> model.predict(array([0.0])) == 0 + True """ - sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - categoricalFeaturesInfoJMap = \ - MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - dataBytes._jrdd, "classification", - numClasses, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - dataBytes.unpersist() - return DecisionTreeModel(sc, model) + return DecisionTree._train(data, "classification", numClasses, categoricalFeaturesInfo, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) @staticmethod def trainRegressor(data, categoricalFeaturesInfo, @@ -194,18 +180,33 @@ def trainRegressor(data, categoricalFeaturesInfo, the parent split :param minInfoGain: Min info gain required to create a split :return: DecisionTreeModel + + Example usage: + + >>> from numpy import array + >>> from pyspark.mllib.regression import LabeledPoint + >>> from pyspark.mllib.tree import DecisionTree + >>> from pyspark.mllib.linalg import SparseVector + >>> + >>> sparse_data = [ + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 1.0})), + ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})), + ... LabeledPoint(1.0, SparseVector(2, {1: 2.0})) + ... ] + >>> + >>> model = DecisionTree.trainRegressor(sc.parallelize(sparse_data), {}) + >>> model.predict(array([0.0, 1.0])) == 1 + True + >>> model.predict(array([0.0, 0.0])) == 0 + True + >>> model.predict(SparseVector(2, {1: 1.0})) == 1 + True + >>> model.predict(SparseVector(2, {1: 0.0})) == 0 + True """ - sc = data.context - dataBytes = _get_unmangled_labeled_point_rdd(data) - categoricalFeaturesInfoJMap = \ - MapConverter().convert(categoricalFeaturesInfo, - sc._gateway._gateway_client) - model = sc._jvm.PythonMLLibAPI().trainDecisionTreeModel( - dataBytes._jrdd, "regression", - 0, categoricalFeaturesInfoJMap, - impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) - dataBytes.unpersist() - return DecisionTreeModel(sc, model) + return DecisionTree._train(data, "regression", 0, categoricalFeaturesInfo, + impurity, maxDepth, maxBins, minInstancesPerNode, minInfoGain) def _test(): diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 1c7b8c809ab5b..8233d4e81f1ca 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -18,11 +18,10 @@ import numpy as np import warnings -from pyspark.mllib.linalg import Vectors, SparseVector -from pyspark.mllib.regression import LabeledPoint -from pyspark.mllib._common import _convert_vector, _deserialize_labeled_point from pyspark.rdd import RDD -from pyspark.serializers import NoOpSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector +from pyspark.mllib.regression import LabeledPoint class MLUtils(object): @@ -32,15 +31,12 @@ class MLUtils(object): """ @staticmethod - def _parse_libsvm_line(line, multiclass): - warnings.warn("deprecated", DeprecationWarning) - return _parse_libsvm_line(line) - - @staticmethod - def _parse_libsvm_line(line): + def _parse_libsvm_line(line, multiclass=None): """ Parses a line in LIBSVM format into (label, indices, values). """ + if multiclass is not None: + warnings.warn("deprecated", DeprecationWarning) items = line.split(None) label = float(items[0]) nnz = len(items) - 1 @@ -55,27 +51,20 @@ def _parse_libsvm_line(line): @staticmethod def _convert_labeled_point_to_libsvm(p): """Converts a LabeledPoint to a string in LIBSVM format.""" + assert isinstance(p, LabeledPoint) items = [str(p.label)] - v = _convert_vector(p.features) - if type(v) == np.ndarray: - for i in xrange(len(v)): - items.append(str(i + 1) + ":" + str(v[i])) - elif type(v) == SparseVector: + v = _convert_to_vector(p.features) + if isinstance(v, SparseVector): nnz = len(v.indices) for i in xrange(nnz): items.append(str(v.indices[i] + 1) + ":" + str(v.values[i])) else: - raise TypeError("_convert_labeled_point_to_libsvm needs either ndarray or SparseVector" - " but got " % type(v)) + for i in xrange(len(v)): + items.append(str(i + 1) + ":" + str(v[i])) return " ".join(items) @staticmethod - def loadLibSVMFile(sc, path, multiclass=False, numFeatures=-1, minPartitions=None): - warnings.warn("deprecated", DeprecationWarning) - return loadLibSVMFile(sc, path, numFeatures, minPartitions) - - @staticmethod - def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): + def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None): """ Loads labeled data in the LIBSVM format into an RDD of LabeledPoint. The LIBSVM format is a text-based format used by @@ -122,6 +111,8 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None): >>> print examples[2] (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) """ + if multiclass is not None: + warnings.warn("deprecated", DeprecationWarning) lines = sc.textFile(path, minPartitions) parsed = lines.map(lambda l: MLUtils._parse_libsvm_line(l)) @@ -182,9 +173,9 @@ def loadLabeledPoints(sc, path, minPartitions=None): (0.0,[1.01,2.02,3.03]) """ minPartitions = minPartitions or min(sc.defaultParallelism, 2) - jSerialized = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) - serialized = RDD(jSerialized, sc, NoOpSerializer()) - return serialized.map(lambda bytes: _deserialize_labeled_point(bytearray(bytes))) + jrdd = sc._jvm.PythonMLLibAPI().loadLabeledPoints(sc._jsc, path, minPartitions) + jpyrdd = sc._jvm.PythonRDD.javaToPython(jrdd) + return RDD(jpyrdd, sc, BatchedSerializer(PickleSerializer())) def _test(): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index b43606b7304c5..8ef233bc80c5c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -34,7 +34,7 @@ from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ - PickleSerializer, pack_long, CompressedSerializer + PickleSerializer, pack_long, AutoBatchedSerializer from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -1927,10 +1927,10 @@ def _to_java_object_rdd(self): It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. """ - if not self._is_pickled(): - self = self._reserialize(BatchedSerializer(PickleSerializer(), 1024)) - batched = isinstance(self._jrdd_deserializer, BatchedSerializer) - return self.ctx._jvm.PythonRDD.pythonToJava(self._jrdd, batched) + rdd = self._reserialize(AutoBatchedSerializer(PickleSerializer())) \ + if not self._is_pickled() else self + is_batch = isinstance(rdd._jrdd_deserializer, BatchedSerializer) + return self.ctx._jvm.PythonRDD.pythonToJava(rdd._jrdd, is_batch) def countApprox(self, timeout, confidence=0.95): """ diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 44ac5642836e0..2672da36c1f50 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -68,6 +68,7 @@ import types import collections import zlib +import itertools from pyspark import cloudpickle @@ -214,6 +215,41 @@ def __str__(self): return "BatchedSerializer<%s>" % str(self.serializer) +class AutoBatchedSerializer(BatchedSerializer): + """ + Choose the size of batch automatically based on the size of object + """ + + def __init__(self, serializer, bestSize=1 << 20): + BatchedSerializer.__init__(self, serializer, -1) + self.bestSize = bestSize + + def dump_stream(self, iterator, stream): + batch, best = 1, self.bestSize + iterator = iter(iterator) + while True: + vs = list(itertools.islice(iterator, batch)) + if not vs: + break + + bytes = self.serializer.dumps(vs) + write_int(len(bytes), stream) + stream.write(bytes) + + size = len(bytes) + if size < best: + batch *= 2 + elif size > best * 10 and batch > 1: + batch /= 2 + + def __eq__(self, other): + return (isinstance(other, AutoBatchedSerializer) and + other.serializer == self.serializer) + + def __str__(self): + return "BatchedSerializer<%s>" % str(self.serializer) + + class CartesianDeserializer(FramedSerializer): """ diff --git a/python/run-tests b/python/run-tests index a67e5a99fbdcc..a7ec270c7da21 100755 --- a/python/run-tests +++ b/python/run-tests @@ -73,7 +73,6 @@ run_test "pyspark/serializers.py" unset PYSPARK_DOC_TEST run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" -run_test "pyspark/mllib/_common.py" run_test "pyspark/mllib/classification.py" run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/linalg.py" From 2c3cc7641d86fa5196406955325a042890f77563 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 19 Sep 2014 15:29:22 -0700 Subject: [PATCH 0745/1492] [SPARK-3501] [SQL] Fix the bug of Hive SimpleUDF creates unnecessary type cast When do the query like: ``` select datediff(cast(value as timestamp), cast('2002-03-21 00:00:00' as timestamp)) from src; ``` SparkSQL will raise exception: ``` [info] scala.MatchError: TimestampType (of class org.apache.spark.sql.catalyst.types.TimestampType$) [info] at org.apache.spark.sql.catalyst.expressions.Cast.castToTimestamp(Cast.scala:77) [info] at org.apache.spark.sql.catalyst.expressions.Cast.cast$lzycompute(Cast.scala:251) [info] at org.apache.spark.sql.catalyst.expressions.Cast.cast(Cast.scala:247) [info] at org.apache.spark.sql.catalyst.expressions.Cast.eval(Cast.scala:263) [info] at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$5$$anonfun$applyOrElse$2.applyOrElse(Optimizer.scala:217) [info] at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$5$$anonfun$applyOrElse$2.applyOrElse(Optimizer.scala:210) [info] at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:144) [info] at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4$$anonfun$apply$2.apply(TreeNode.scala:180) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) [info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244) ``` Author: Cheng Hao Closes #2368 from chenghao-intel/cast_exception and squashes the following commits: 5c9c3a5 [Cheng Hao] make more clear code 49dfc50 [Cheng Hao] Add no-op for Cast and revert the position of SimplifyCasts b804abd [Cheng Hao] Add unit test to show the failure in identical data type casting 330a5c8 [Cheng Hao] Update Code based on comments b834ed4 [Cheng Hao] Fix bug of HiveSimpleUDF with unnecessary type cast which cause exception in constant folding --- .../apache/spark/sql/catalyst/expressions/Cast.scala | 1 + .../scala/org/apache/spark/sql/hive/hiveUdfs.scala | 3 ++- ...imestamp in UDF-0-66952a3949d7544716fd1a675498b1fa | 1 + .../spark/sql/hive/execution/HiveQuerySuite.scala | 11 ++++++++++- 4 files changed, 14 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 0ad2b30cf9c1f..0379275121bf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -245,6 +245,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } private[this] lazy val cast: Any => Any = dataType match { + case dt if dt == child.dataType => identity[Any] case StringType => castToString case BinaryType => castToBinary case DecimalType => castToDecimal diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 7d1ad53d8bdb3..7cda0dd302c86 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -51,12 +51,13 @@ private[hive] abstract class HiveFunctionRegistry val function = functionInfo.getFunctionClass.newInstance().asInstanceOf[UDF] val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) - lazy val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) + val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) HiveSimpleUdf( functionClassName, children.zip(expectedDataTypes).map { case (e, NullType) => e + case (e, t) if (e.dataType == t) => e case (e, t) => Cast(e, t) } ) diff --git a/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa b/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 8c8a8b124ac69..56bcd95eab4bc 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -142,16 +142,25 @@ class HiveQuerySuite extends HiveComparisonTest { setConf("spark.sql.dialect", "sql") assert(sql("SELECT 1").collect() === Array(Seq(1))) setConf("spark.sql.dialect", "hiveql") - } test("Query expressed in HiveQL") { sql("FROM src SELECT key").collect() } + test("Query with constant folding the CAST") { + sql("SELECT CAST(CAST('123' AS binary) AS binary) FROM src LIMIT 1").collect() + } + createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT", "SELECT AVG(0), SUM(0), COUNT(null), COUNT(value) FROM src GROUP BY key") + createQueryTest("Cast Timestamp to Timestamp in UDF", + """ + | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) + | FROM src LIMIT 1 + """.stripMargin) + createQueryTest("Simple Average", "SELECT AVG(key) FROM src") From 5522151eb14f4208798901f5c090868edd8e8dde Mon Sep 17 00:00:00 2001 From: ravipesala Date: Fri, 19 Sep 2014 15:31:57 -0700 Subject: [PATCH 0746/1492] [SPARK-2594][SQL] Support CACHE TABLE AS SELECT ... This feature allows user to add cache table from the select query. Example : ```CACHE TABLE testCacheTable AS SELECT * FROM TEST_TABLE``` Spark takes this type of SQL as command and it does lazy caching just like ```SQLContext.cacheTable```, ```CACHE TABLE ``` does. It can be executed from both SQLContext and HiveContext. Recreated the pull request after rebasing with master.And fixed all the comments raised in previous pull requests. https://github.com/apache/spark/pull/2381 https://github.com/apache/spark/pull/2390 Author : ravipesala ravindra.pesalahuawei.com Author: ravipesala Closes #2397 from ravipesala/SPARK-2594 and squashes the following commits: a5f0beb [ravipesala] Simplified the code as per Admin comment. 8059cd2 [ravipesala] Changed the behaviour from eager caching to lazy caching. d6e469d [ravipesala] Code review comments by Admin are handled. c18aa38 [ravipesala] Merge remote-tracking branch 'remotes/ravipesala/Add-Cache-table-as' into SPARK-2594 394d5ca [ravipesala] Changed style fb1759b [ravipesala] Updated as per Admin comments 8c9993c [ravipesala] Changed the style d8b37b2 [ravipesala] Updated as per the comments by Admin bc0bffc [ravipesala] Merge remote-tracking branch 'ravipesala/Add-Cache-table-as' into Add-Cache-table-as e3265d0 [ravipesala] Updated the code as per the comments by Admin in pull request. 724b9db [ravipesala] Changed style aaf5b59 [ravipesala] Added comment dc33895 [ravipesala] Updated parser to support add cache table command b5276b2 [ravipesala] Updated parser to support add cache table command eebc0c1 [ravipesala] Add CACHE TABLE AS SELECT ... 6758f80 [ravipesala] Changed style 7459ce3 [ravipesala] Added comment 13c8e27 [ravipesala] Updated parser to support add cache table command 4e858d8 [ravipesala] Updated parser to support add cache table command b803fc8 [ravipesala] Add CACHE TABLE AS SELECT ... --- .../apache/spark/sql/catalyst/SqlParser.scala | 14 +++++++-- .../sql/catalyst/plans/logical/commands.scala | 5 ++++ .../spark/sql/execution/SparkStrategies.scala | 2 ++ .../apache/spark/sql/execution/commands.scala | 18 +++++++++++ .../apache/spark/sql/CachedTableSuite.scala | 13 ++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 30 ++++++++++++------- 6 files changed, 69 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index ca69531c69a77..862f78702c4e6 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -151,7 +151,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} | UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) - | insert | cache + | insert | cache | unCache ) protected lazy val select: Parser[LogicalPlan] = @@ -183,9 +183,17 @@ class SqlParser extends StandardTokenParsers with PackratParsers { } protected lazy val cache: Parser[LogicalPlan] = - (CACHE ^^^ true | UNCACHE ^^^ false) ~ TABLE ~ ident ^^ { - case doCache ~ _ ~ tableName => CacheCommand(tableName, doCache) + CACHE ~ TABLE ~> ident ~ opt(AS ~> select) <~ opt(";") ^^ { + case tableName ~ None => + CacheCommand(tableName, true) + case tableName ~ Some(plan) => + CacheTableAsSelectCommand(tableName, plan) } + + protected lazy val unCache: Parser[LogicalPlan] = + UNCACHE ~ TABLE ~> ident <~ opt(";") ^^ { + case tableName => CacheCommand(tableName, false) + } protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index a01809c1fc5e2..8366639fa0e8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -75,3 +75,8 @@ case class DescribeCommand( AttributeReference("data_type", StringType, nullable = false)(), AttributeReference("comment", StringType, nullable = false)()) } + +/** + * Returned for the "CACHE TABLE tableName AS SELECT .." command. + */ +case class CacheTableAsSelectCommand(tableName: String, plan: LogicalPlan) extends Command diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 7943d6e1b6fb5..45687d960404c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -305,6 +305,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) case logical.CacheCommand(tableName, cache) => Seq(execution.CacheCommand(tableName, cache)(context)) + case logical.CacheTableAsSelectCommand(tableName, plan) => + Seq(execution.CacheTableAsSelectCommand(tableName, plan)) case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 94543fc95b470..c2f48a902a3e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -166,3 +166,21 @@ case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( child.output.map(field => Row(field.name, field.dataType.toString, null)) } } + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class CacheTableAsSelectCommand(tableName: String, logicalPlan: LogicalPlan) + extends LeafNode with Command { + + override protected[sql] lazy val sideEffectResult = { + import sqlContext._ + logicalPlan.registerTempTable(tableName) + cacheTable(tableName) + Seq.empty[Row] + } + + override def output: Seq[Attribute] = Seq.empty + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index befef46d93973..591592841e9fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -119,4 +119,17 @@ class CachedTableSuite extends QueryTest { } assert(!TestSQLContext.isCached("testData"), "Table 'testData' should not be cached") } + + test("CACHE TABLE tableName AS SELECT Star Table") { + TestSQLContext.sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + TestSQLContext.sql("SELECT * FROM testCacheTable WHERE key = 1").collect() + assert(TestSQLContext.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + TestSQLContext.uncacheTable("testCacheTable") + } + + test("'CACHE TABLE tableName AS SELECT ..'") { + TestSQLContext.sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + assert(TestSQLContext.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + TestSQLContext.uncacheTable("testCacheTable") + } } 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 21ecf17028dbc..0aa6292c0184e 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 @@ -229,7 +229,12 @@ private[hive] object HiveQl { SetCommand(Some(key), Some(value)) } } else if (sql.trim.toLowerCase.startsWith("cache table")) { - CacheCommand(sql.trim.drop(12).trim, true) + sql.trim.drop(12).trim.split(" ").toSeq match { + case Seq(tableName) => + CacheCommand(tableName, true) + case Seq(tableName, _, select @ _*) => + CacheTableAsSelectCommand(tableName, createPlan(select.mkString(" ").trim)) + } } else if (sql.trim.toLowerCase.startsWith("uncache table")) { CacheCommand(sql.trim.drop(14).trim, false) } else if (sql.trim.toLowerCase.startsWith("add jar")) { @@ -243,15 +248,7 @@ private[hive] object HiveQl { } else if (sql.trim.startsWith("!")) { ShellCommand(sql.drop(1)) } else { - val tree = getAst(sql) - if (nativeCommands contains tree.getText) { - NativeCommand(sql) - } else { - nodeToPlan(tree) match { - case NativePlaceholder => NativeCommand(sql) - case other => other - } - } + createPlan(sql) } } catch { case e: Exception => throw new ParseException(sql, e) @@ -262,6 +259,19 @@ private[hive] object HiveQl { """.stripMargin) } } + + /** Creates LogicalPlan for a given HiveQL string. */ + def createPlan(sql: String) = { + val tree = getAst(sql) + if (nativeCommands contains tree.getText) { + NativeCommand(sql) + } else { + nodeToPlan(tree) match { + case NativePlaceholder => NativeCommand(sql) + case other => other + } + } + } def parseDdl(ddl: String): Seq[Attribute] = { val tree = From a95ad99e31c2d5980a3b8cd8e36ff968b1e6b201 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 19 Sep 2014 15:33:42 -0700 Subject: [PATCH 0747/1492] [SPARK-3592] [SQL] [PySpark] support applySchema to RDD of Row Fix the issue when applySchema() to an RDD of Row. Also add type mapping for BinaryType. Author: Davies Liu Closes #2448 from davies/row and squashes the following commits: dd220cf [Davies Liu] fix test 3f3f188 [Davies Liu] add more test f559746 [Davies Liu] add tests, fix serialization 9688fd2 [Davies Liu] support applySchema to RDD of Row --- python/pyspark/sql.py | 13 ++++++++++--- python/pyspark/tests.py | 11 ++++++++++- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 42a9920f10e6f..653195ea438cf 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -440,6 +440,7 @@ def _parse_datatype_string(datatype_string): float: DoubleType, str: StringType, unicode: StringType, + bytearray: BinaryType, decimal.Decimal: DecimalType, datetime.datetime: TimestampType, datetime.date: TimestampType, @@ -690,11 +691,12 @@ def _infer_schema_type(obj, dataType): ByteType: (int, long), ShortType: (int, long), IntegerType: (int, long), - LongType: (long,), + LongType: (int, long), FloatType: (float,), DoubleType: (float,), DecimalType: (decimal.Decimal,), StringType: (str, unicode), + BinaryType: (bytearray,), TimestampType: (datetime.datetime,), ArrayType: (list, tuple, array), MapType: (dict,), @@ -728,9 +730,9 @@ def _verify_type(obj, dataType): return _type = type(dataType) - if _type not in _acceptable_types: - return + assert _type in _acceptable_types, "unkown datatype: %s" % dataType + # subclass of them can not be deserialized in JVM if type(obj) not in _acceptable_types[_type]: raise TypeError("%s can not accept abject in type %s" % (dataType, type(obj))) @@ -1121,6 +1123,11 @@ def applySchema(self, rdd, schema): # take the first few rows to verify schema rows = rdd.take(10) + # Row() cannot been deserialized by Pyrolite + if rows and isinstance(rows[0], tuple) and rows[0].__class__.__name__ == 'Row': + rdd = rdd.map(tuple) + rows = rdd.take(10) + for row in rows: _verify_type(row, schema) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 7301966e48045..a94eb0f429e0a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -45,7 +45,7 @@ from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter -from pyspark.sql import SQLContext, IntegerType +from pyspark.sql import SQLContext, IntegerType, Row from pyspark import shuffle _have_scipy = False @@ -659,6 +659,15 @@ def test_distinct(self): self.assertEquals(result.getNumPartitions(), 5) self.assertEquals(result.count(), 3) + def test_apply_schema_to_row(self): + srdd = self.sqlCtx.jsonRDD(self.sc.parallelize(["""{"a":2}"""])) + srdd2 = self.sqlCtx.applySchema(srdd.map(lambda x: x), srdd.schema()) + self.assertEqual(srdd.collect(), srdd2.collect()) + + rdd = self.sc.parallelize(range(10)).map(lambda x: Row(a=x)) + srdd3 = self.sqlCtx.applySchema(rdd, srdd.schema()) + self.assertEqual(10, srdd3.count()) + class TestIO(PySparkTestCase): From 3b9cd13ebc108c7c6d518a760333cd992667126c Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 19 Sep 2014 15:34:48 -0700 Subject: [PATCH 0748/1492] SPARK-3605. Fix typo in SchemaRDD. Author: Sandy Ryza Closes #2460 from sryza/sandy-spark-3605 and squashes the following commits: 09d940b [Sandy Ryza] SPARK-3605. Fix typo in SchemaRDD. --- sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 3bc5dce095511..3b873f7c62cb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -45,9 +45,8 @@ import org.apache.spark.api.java.JavaRDD * explicitly using the `createSchemaRDD` function on a [[SQLContext]]. * * A `SchemaRDD` can also be created by loading data in from external sources. - * Examples are loading data from Parquet files by using by using the - * `parquetFile` method on [[SQLContext]], and loading JSON datasets - * by using `jsonFile` and `jsonRDD` methods on [[SQLContext]]. + * Examples are loading data from Parquet files by using the `parquetFile` method on [[SQLContext]] + * and loading JSON datasets by using `jsonFile` and `jsonRDD` methods on [[SQLContext]]. * * == SQL Queries == * A SchemaRDD can be registered as a table in the [[SQLContext]] that was used to create it. Once From ba68a51c407197d478b330403af8fe24a176bef3 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 19 Sep 2014 15:39:31 -0700 Subject: [PATCH 0749/1492] [SPARK-3485][SQL] Use GenericUDFUtils.ConversionHelper for Simple UDF type conversions This is just another solution to SPARK-3485, in addition to PR #2355 In this patch, we will use ConventionHelper and FunctionRegistry to invoke a simple udf evaluation, which rely more on hive, but much cleaner and safer. We can discuss which one is better. Author: Daoyuan Wang Closes #2407 from adrian-wang/simpleudf and squashes the following commits: 15762d2 [Daoyuan Wang] add posmod test which would fail the test but now ok 0d69eb4 [Daoyuan Wang] another way to pass to hive simple udf --- .../execution/HiveCompatibilitySuite.scala | 1 + .../org/apache/spark/sql/hive/hiveUdfs.scala | 55 ++++++------------- 2 files changed, 17 insertions(+), 39 deletions(-) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index ab487d673e813..556c984ad392b 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -801,6 +801,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_or", "udf_parse_url", "udf_PI", + "udf_pmod", "udf_positive", "udf_pow", "udf_power", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 7cda0dd302c86..5a0e6c5cc1bba 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper + import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.common.`type`.HiveDecimal @@ -105,52 +107,27 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) @transient - lazy val dataType = javaClassToDataType(method.getReturnType) + protected lazy val arguments = children.map(c => toInspector(c.dataType)).toArray - protected lazy val wrappers: Array[(Any) => AnyRef] = method.getParameterTypes.map { argClass => - val primitiveClasses = Seq( - Integer.TYPE, classOf[java.lang.Integer], classOf[java.lang.String], java.lang.Double.TYPE, - classOf[java.lang.Double], java.lang.Long.TYPE, classOf[java.lang.Long], - classOf[HiveDecimal], java.lang.Byte.TYPE, classOf[java.lang.Byte], - classOf[java.sql.Timestamp] - ) - val matchingConstructor = argClass.getConstructors.find { c => - c.getParameterTypes.size == 1 && primitiveClasses.contains(c.getParameterTypes.head) - } + // Create parameter converters + @transient + protected lazy val conversionHelper = new ConversionHelper(method, arguments) - matchingConstructor match { - case Some(constructor) => - (a: Any) => { - logDebug( - s"Wrapping $a of type ${if (a == null) "null" else a.getClass.getName} $constructor.") - // We must make sure that primitives get boxed java style. - if (a == null) { - null - } else { - constructor.newInstance(a match { - case i: Int => i: java.lang.Integer - case bd: BigDecimal => new HiveDecimal(bd.underlying()) - case other: AnyRef => other - }).asInstanceOf[AnyRef] - } - } - case None => - (a: Any) => a match { - case wrapper => wrap(wrapper) - } - } + @transient + lazy val dataType = javaClassToDataType(method.getReturnType) + + def catalystToHive(value: Any): Object = value match { + // TODO need more types here? or can we use wrap() + case bd: BigDecimal => new HiveDecimal(bd.underlying()) + case d => d.asInstanceOf[Object] } // TODO: Finish input output types. override def eval(input: Row): Any = { - val evaluatedChildren = children.map(_.eval(input)) - // Wrap the function arguments in the expected types. - val args = evaluatedChildren.zip(wrappers).map { - case (arg, wrapper) => wrapper(arg) - } + val evaluatedChildren = children.map(c => catalystToHive(c.eval(input))) - // Invoke the udf and unwrap the result. - unwrap(method.invoke(function, args: _*)) + unwrap(FunctionRegistry.invoke(method, function, conversionHelper + .convertIfNecessary(evaluatedChildren: _*): _*)) } } From 99b06b6fd2d79403ef4307ac6f3fa84176e7a622 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Fri, 19 Sep 2014 15:44:47 -0700 Subject: [PATCH 0750/1492] [Build] Fix passing of args to sbt Simple mistake, simple fix: ```shell args="arg1 arg2 arg3" sbt $args # sbt sees 3 arguments sbt "$args" # sbt sees 1 argument ``` Should fix the problems we are seeing [here](https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/694/AMPLAB_JENKINS_BUILD_PROFILE=hadoop1.0,label=centos/console), for example. Author: Nicholas Chammas Closes #2462 from nchammas/fix-sbt-master-build and squashes the following commits: 4500c86 [Nicholas Chammas] warn about quoting 10018a6 [Nicholas Chammas] Revert "test hadoop1 build" 7d5356c [Nicholas Chammas] Revert "re-add bad quoting for testing" 061600c [Nicholas Chammas] re-add bad quoting for testing b2de56c [Nicholas Chammas] test hadoop1 build 43fb854 [Nicholas Chammas] unquote profile args --- dev/run-tests | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 5f6df17b509a3..c3d8f49cdd993 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -127,6 +127,8 @@ echo "=========================================================================" # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc #+ to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a + #+ single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ @@ -159,10 +161,13 @@ echo "=========================================================================" # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc #+ to quit or retry. This echo is there to make it not block. + # NOTE: Do not quote $SBT_MAVEN_PROFILES_ARGS or else it will be interpreted as a + #+ single argument! + #+ "${SBT_MAVEN_TEST_ARGS[@]}" is cool because it's an array. # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? echo -e "q\n" \ - | sbt/sbt "$SBT_MAVEN_PROFILES_ARGS" "${SBT_MAVEN_TEST_ARGS[@]}" \ + | sbt/sbt $SBT_MAVEN_PROFILES_ARGS "${SBT_MAVEN_TEST_ARGS[@]}" \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } From 8af2370619a8a6bb1af7df43b8329ab319348ad8 Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Fri, 19 Sep 2014 16:02:38 -0700 Subject: [PATCH 0751/1492] [Docs] Fix outdated docs for standalone cluster This is now supported! Author: andrewor14 Author: Andrew Or Closes #2461 from andrewor14/document-standalone-cluster and squashes the following commits: 85c8b9e [andrewor14] Wording change per Patrick 35e30ee [Andrew Or] Fix outdated docs for standalone cluster --- docs/spark-standalone.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 99a8e43a6b489..29b5491861bf3 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -248,8 +248,10 @@ You can also pass an option `--cores ` to control the number of cores The [`spark-submit` script](submitting-applications.html) provides the most straightforward way to submit a compiled Spark application to the cluster. For standalone clusters, Spark currently -only supports deploying the driver inside the client process that is submitting the application -(`client` deploy mode). +supports two deploy modes. In `client` mode, the driver is launched in the same process as the +client that submits the application. In `cluster` mode, however, the driver is launched from one +of the Worker processes inside the cluster, and the client process exits as soon as it fulfills +its responsibility of submitting the application without waiting for the application to finish. If your application is launched through Spark submit, then the application jar is automatically distributed to all worker nodes. For any additional jars that your application depends on, you From 78d4220fa0bf2f9ee663e34bbf3544a5313b02f0 Mon Sep 17 00:00:00 2001 From: Vida Ha Date: Sat, 20 Sep 2014 01:24:49 -0700 Subject: [PATCH 0752/1492] SPARK-3608 Break if the instance tag naming succeeds Author: Vida Ha Closes #2466 from vidaha/vida/spark-3608 and squashes the following commits: 9509776 [Vida Ha] Break if the instance tag naming succeeds --- ec2/spark_ec2.py | 1 + 1 file changed, 1 insertion(+) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index abac71eaca595..fbeccd89b43b3 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -505,6 +505,7 @@ def tag_instance(instance, name): for i in range(0, 5): try: instance.add_tag(key='Name', value=name) + break except: print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) if (i == 5): From c32c8538efca2124924920614e4dbe7ce90938f4 Mon Sep 17 00:00:00 2001 From: "Santiago M. Mola" Date: Sat, 20 Sep 2014 15:05:03 -0700 Subject: [PATCH 0753/1492] Fix Java example in Streaming Programming Guide "val conf" was used instead of "SparkConf conf" in Java snippet. Author: Santiago M. Mola Closes #2472 from smola/patch-1 and squashes the following commits: 5bfeb9b [Santiago M. Mola] Fix Java example in Streaming Programming Guide --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 41f170580f452..5c21e912ea160 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -138,7 +138,7 @@ import org.apache.spark.streaming.api.java.*; import scala.Tuple2; // Create a local StreamingContext with two working thread and batch interval of 1 second -val conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") +SparkConf conf = new SparkConf().setMaster("local[2]").setAppName("NetworkWordCount") JavaStreamingContext jssc = new JavaStreamingContext(conf, new Duration(1000)) {% endhighlight %} From 5f8833c672ab64aa5886a8239ae2ff2a8ea42363 Mon Sep 17 00:00:00 2001 From: Matthew Farrellee Date: Sat, 20 Sep 2014 15:09:35 -0700 Subject: [PATCH 0754/1492] [PySpark] remove unnecessary use of numSlices from pyspark tests Author: Matthew Farrellee Closes #2467 from mattf/master-pyspark-remove-numslices-from-tests and squashes the following commits: c49a87b [Matthew Farrellee] [PySpark] remove unnecessary use of numSlices from pyspark tests --- python/pyspark/tests.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index a94eb0f429e0a..1b8afb763b26a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -1107,7 +1107,7 @@ def test_reserialization(self): def test_unbatched_save_and_read(self): basepath = self.tempdir.name ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] - self.sc.parallelize(ei, numSlices=len(ei)).saveAsSequenceFile( + self.sc.parallelize(ei, len(ei)).saveAsSequenceFile( basepath + "/unbatched/") unbatched_sequence = sorted(self.sc.sequenceFile( @@ -1153,7 +1153,7 @@ def test_malformed_RDD(self): basepath = self.tempdir.name # non-batch-serialized RDD[[(K, V)]] should be rejected data = [[(1, "a")], [(2, "aa")], [(3, "aaa")]] - rdd = self.sc.parallelize(data, numSlices=len(data)) + rdd = self.sc.parallelize(data, len(data)) self.assertRaises(Exception, lambda: rdd.saveAsSequenceFile( basepath + "/malformed/sequence")) From 7c8ad1c0838762f5b632f683834c88a711aef4dd Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sat, 20 Sep 2014 16:03:17 -0700 Subject: [PATCH 0755/1492] SPARK-3574. Shuffle finish time always reported as -1 The included test waits 100 ms after job completion for task completion events to come in so it can verify they have reasonable finish times. Does anyone know a better way to wait on listener events that are expected to come in? Author: Sandy Ryza Closes #2440 from sryza/sandy-spark-3574 and squashes the following commits: c81439b [Sandy Ryza] Fix test failure b340956 [Sandy Ryza] SPARK-3574. Remove shuffleFinishTime metric --- .../main/scala/org/apache/spark/executor/TaskMetrics.scala | 6 ------ .../main/scala/org/apache/spark/scheduler/JobLogger.scala | 1 - .../src/main/scala/org/apache/spark/util/JsonProtocol.scala | 2 -- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 3 --- 4 files changed, 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 99a88c13456df..3e49b6235aff3 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -137,7 +137,6 @@ class TaskMetrics extends Serializable { merged.localBlocksFetched += depMetrics.localBlocksFetched merged.remoteBlocksFetched += depMetrics.remoteBlocksFetched merged.remoteBytesRead += depMetrics.remoteBytesRead - merged.shuffleFinishTime = math.max(merged.shuffleFinishTime, depMetrics.shuffleFinishTime) } _shuffleReadMetrics = Some(merged) } @@ -177,11 +176,6 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { */ @DeveloperApi class ShuffleReadMetrics extends Serializable { - /** - * Absolute time when this task finished reading shuffle data - */ - var shuffleFinishTime: Long = -1 - /** * Number of blocks fetched in this shuffle by this task (remote or local) */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 4d6b5c81883b6..ceb434feb6ca1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -171,7 +171,6 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener } val shuffleReadMetrics = taskMetrics.shuffleReadMetrics match { case Some(metrics) => - " SHUFFLE_FINISH_TIME=" + metrics.shuffleFinishTime + " BLOCK_FETCHED_TOTAL=" + metrics.totalBlocksFetched + " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index c4dddb2d1037e..6a48f673c4e78 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -255,7 +255,6 @@ private[spark] object JsonProtocol { } def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { - ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ @@ -590,7 +589,6 @@ private[spark] object JsonProtocol { def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { val metrics = new ShuffleReadMetrics - metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 2b45d8b695853..f1f88c5fd3634 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -356,7 +356,6 @@ class JsonProtocolSuite extends FunSuite { } private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { - assert(metrics1.shuffleFinishTime === metrics2.shuffleFinishTime) assert(metrics1.remoteBlocksFetched === metrics2.remoteBlocksFetched) assert(metrics1.localBlocksFetched === metrics2.localBlocksFetched) assert(metrics1.fetchWaitTime === metrics2.fetchWaitTime) @@ -568,7 +567,6 @@ class JsonProtocolSuite extends FunSuite { t.inputMetrics = Some(inputMetrics) } else { val sr = new ShuffleReadMetrics - sr.shuffleFinishTime = b + c sr.remoteBytesRead = b + d sr.localBlocksFetched = e sr.fetchWaitTime = a + d @@ -806,7 +804,6 @@ class JsonProtocolSuite extends FunSuite { | "Memory Bytes Spilled": 800, | "Disk Bytes Spilled": 0, | "Shuffle Read Metrics": { - | "Shuffle Finish Time": 900, | "Remote Blocks Fetched": 800, | "Local Blocks Fetched": 700, | "Fetch Wait Time": 900, From 7f54580c4503d8b6bfcf7d4cbc83b83458140926 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 20 Sep 2014 16:30:49 -0700 Subject: [PATCH 0756/1492] [SPARK-3609][SQL] Adds sizeInBytes statistics for Limit operator when all output attributes are of native data types This helps to replace shuffled hash joins with broadcast hash joins in some cases. Author: Cheng Lian Closes #2468 from liancheng/more-stats and squashes the following commits: 32687dc [Cheng Lian] Moved the test case to PlannerSuite 5595a91 [Cheng Lian] Removes debugging code 73faf69 [Cheng Lian] Test case for auto choosing broadcast hash join f30fe1d [Cheng Lian] Adds sizeInBytes estimation for Limit when all output types are native types --- .../plans/logical/basicOperators.scala | 11 ++++++++++ .../spark/sql/catalyst/types/dataTypes.scala | 10 ++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 9 +++++---- .../spark/sql/execution/PlannerSuite.scala | 20 ++++++++++++++++++- 4 files changed, 45 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 5d10754c7b028..8e8259cae6670 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -148,6 +148,17 @@ case class Aggregate( case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output + + override lazy val statistics: Statistics = + if (output.forall(_.dataType.isInstanceOf[NativeType])) { + val limit = limitExpr.eval(null).asInstanceOf[Int] + val sizeInBytes = (limit: Long) * output.map { a => + NativeType.defaultSizeOf(a.dataType.asInstanceOf[NativeType]) + }.sum + Statistics(sizeInBytes = sizeInBytes) + } else { + Statistics(sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product) + } } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { 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 49520b7678e90..e3050e5397937 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 @@ -122,6 +122,16 @@ object NativeType { IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) def unapply(dt: DataType): Boolean = all.contains(dt) + + val defaultSizeOf: Map[NativeType, Int] = Map( + IntegerType -> 4, + BooleanType -> 1, + LongType -> 8, + DoubleType -> 8, + FloatType -> 4, + ShortType -> 2, + ByteType -> 1, + StringType -> 4096) } trait PrimitiveType extends DataType { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 67563b6c55f4b..15f6bcef93886 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.{ShuffledHashJoin, BroadcastHashJoin} import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll import java.util.TimeZone @@ -649,24 +650,24 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { (3, null) :: (4, 2147483644) :: Nil) } - + test("SPARK-3423 BETWEEN") { checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 5 and 7"), Seq((5, "5"), (6, "6"), (7, "7")) ) - + checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 7 and 7"), Seq((7, "7")) ) - + checkAnswer( sql("SELECT key, value FROM testData WHERE key BETWEEN 9 and 7"), Seq() ) } - + test("cast boolean to string") { // TODO Ensure true/false string letter casing is consistent with Hive in all cases. checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 37d64f0de7bab..bfbf431a11913 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -22,7 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.execution +import org.apache.spark.sql.{SQLConf, execution} import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ @@ -57,4 +57,22 @@ class PlannerSuite extends FunSuite { val planned = HashAggregation(query) assert(planned.nonEmpty) } + + test("sizeInBytes estimation of limit operator for broadcast hash join optimization") { + val origThreshold = autoBroadcastJoinThreshold + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, 81920.toString) + + // Using a threshold that is definitely larger than the small testing table (b) below + val a = testData.as('a) + val b = testData.limit(3).as('b) + val planned = a.join(b, Inner, Some("a.key".attr === "b.key".attr)).queryExecution.executedPlan + + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } + val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } + + assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") + assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") + + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) + } } From 293ce85145d7a37f7cb329831cbf921be571c2f5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 20 Sep 2014 16:41:14 -0700 Subject: [PATCH 0757/1492] [SPARK-3414][SQL] Replace LowerCaseSchema with Resolver **This PR introduces a subtle change in semantics for HiveContext when using the results in Python or Scala. Specifically, while resolution remains case insensitive, it is now case preserving.** _This PR is a follow up to #2293 (and to a lesser extent #2262 #2334)._ In #2293 the catalog was changed to store analyzed logical plans instead of unresolved ones. While this change fixed the reported bug (which was caused by yet another instance of us forgetting to put in a `LowerCaseSchema` operator) it had the consequence of breaking assumptions made by `MultiInstanceRelation`. Specifically, we can't replace swap out leaf operators in a tree without rewriting changed expression ids (which happens when you self join the same RDD that has been registered as a temp table). In this PR, I instead remove the need to insert `LowerCaseSchema` operators at all, by moving the concern of matching up identifiers completely into analysis. Doing so allows the test cases from both #2293 and #2262 to pass at the same time (and likely fixes a slew of other "unknown unknown" bugs). While it is rolled back in this PR, storing the analyzed plan might actually be a good idea. For instance, it is kind of confusing if you register a temporary table, change the case sensitivity of resolution and now you can't query that table anymore. This can be addressed in a follow up PR. Follow-ups: - Configurable case sensitivity - Consider storing analyzed plans for temp tables Author: Michael Armbrust Closes #2382 from marmbrus/lowercase and squashes the following commits: c21171e [Michael Armbrust] Ensure the resolver is used for field lookups and ensure that case insensitive resolution is still case preserving. d4320f1 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into lowercase 2de881e [Michael Armbrust] Address comments. 219805a [Michael Armbrust] style 5b93711 [Michael Armbrust] Replace LowerCaseSchema with Resolver. --- .../sql/catalyst/analysis/Analyzer.scala | 38 +++------ .../spark/sql/catalyst/analysis/package.scala | 12 ++- .../sql/catalyst/analysis/unresolved.scala | 6 +- .../expressions/namedExpressions.scala | 10 ++- .../catalyst/plans/logical/LogicalPlan.scala | 77 ++++++++++++++++--- .../plans/logical/basicOperators.scala | 26 ------- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 2 - .../apache/spark/sql/hive/HiveContext.scala | 10 +-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 +- .../spark/sql/hive/HiveStrategies.scala | 4 +- .../hive/execution/CreateTableAsSelect.scala | 4 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 7 +- ...nsitive-0-98b2e34c9134208e9fe7c62d33010005 | 1 + .../hive/execution/HiveResolutionSuite.scala | 19 +++-- 15 files changed, 125 insertions(+), 99 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 574d96d92942b..71810b798bd04 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -37,6 +37,8 @@ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Boolean) extends RuleExecutor[LogicalPlan] with HiveTypeCoercion { + val resolver = if (caseSensitive) caseSensitiveResolution else caseInsensitiveResolution + // TODO: pass this in as a parameter. val fixedPoint = FixedPoint(100) @@ -48,8 +50,6 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool lazy val batches: Seq[Batch] = Seq( Batch("MultiInstanceRelations", Once, NewRelationInstances), - Batch("CaseInsensitiveAttributeReferences", Once, - (if (caseSensitive) Nil else LowercaseAttributeReferences :: Nil) : _*), Batch("Resolution", fixedPoint, ResolveReferences :: ResolveRelations :: @@ -98,23 +98,6 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } - /** - * Makes attribute naming case insensitive by turning all UnresolvedAttributes to lowercase. - */ - object LowercaseAttributeReferences extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case UnresolvedRelation(databaseName, name, alias) => - UnresolvedRelation(databaseName, name, alias.map(_.toLowerCase)) - case Subquery(alias, child) => Subquery(alias.toLowerCase, child) - case q: LogicalPlan => q transformExpressions { - case s: Star => s.copy(table = s.table.map(_.toLowerCase)) - case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) - case Alias(c, name) => Alias(c, name.toLowerCase)() - case GetField(c, name) => GetField(c, name.toLowerCase) - } - } - } - /** * Replaces [[UnresolvedAttribute]]s with concrete * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's @@ -127,7 +110,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool q transformExpressions { case u @ UnresolvedAttribute(name) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = q.resolveChildren(name).getOrElse(u) + val result = q.resolveChildren(name, resolver).getOrElse(u) logDebug(s"Resolving $u to $result") result } @@ -144,7 +127,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) - val resolved = unresolved.flatMap(child.resolveChildren) + val resolved = unresolved.flatMap(child.resolve(_, resolver)) val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) val missingInProject = requiredAttributes -- p.output @@ -154,6 +137,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool Sort(ordering, Project(projectList ++ missingInProject, child))) } else { + logDebug(s"Failed to find $missingInProject in ${p.output.mkString(", ")}") s // Nothing we can do here. Return original plan. } case s @ Sort(ordering, a @ Aggregate(grouping, aggs, child)) if !s.resolved && a.resolved => @@ -165,7 +149,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve) + val resolved = unresolved.flatMap(groupingRelation.resolve(_, resolver)) val missingInAggs = resolved.filterNot(a.outputSet.contains) logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") if (missingInAggs.nonEmpty) { @@ -258,14 +242,14 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case p @ Project(projectList, child) if containsStar(projectList) => Project( projectList.flatMap { - case s: Star => s.expand(child.output) + case s: Star => s.expand(child.output, resolver) case o => o :: Nil }, child) case t: ScriptTransformation if containsStar(t.input) => t.copy( input = t.input.flatMap { - case s: Star => s.expand(t.child.output) + case s: Star => s.expand(t.child.output, resolver) case o => o :: Nil } ) @@ -273,7 +257,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case a: Aggregate if containsStar(a.aggregateExpressions) => a.copy( aggregateExpressions = a.aggregateExpressions.flatMap { - case s: Star => s.expand(a.child.output) + case s: Star => s.expand(a.child.output, resolver) case o => o :: Nil } ) @@ -290,13 +274,11 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool /** * Removes [[catalyst.plans.logical.Subquery Subquery]] operators from the plan. Subqueries are * only required to provide scoping information for attributes and can be removed once analysis is - * complete. Similarly, this node also removes - * [[catalyst.plans.logical.LowerCaseSchema LowerCaseSchema]] operators. + * complete. */ object EliminateAnalysisOperators extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Subquery(_, child) => child - case LowerCaseSchema(child) => child } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala index 9f37ca904ffeb..3f672a3e0fd91 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala @@ -22,4 +22,14 @@ package org.apache.spark.sql.catalyst * Analysis consists of translating [[UnresolvedAttribute]]s and [[UnresolvedRelation]]s * into fully typed objects using information in a schema [[Catalog]]. */ -package object analysis +package object analysis { + + /** + * Responsible for resolving which identifiers refer to the same entity. For example, by using + * case insensitive equality. + */ + type Resolver = (String, String) => Boolean + + val caseInsensitiveResolution = (a: String, b: String) => a.equalsIgnoreCase(b) + val caseSensitiveResolution = (a: String, b: String) => a == b +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a2c61c65487cb..67570a6f73c36 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -54,6 +54,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def newInstance = this override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this + override def withName(newName: String) = UnresolvedAttribute(name) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -97,13 +98,14 @@ case class Star( override def newInstance = this override def withNullability(newNullability: Boolean) = this override def withQualifiers(newQualifiers: Seq[String]) = this + override def withName(newName: String) = this - def expand(input: Seq[Attribute]): Seq[NamedExpression] = { + def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] = { val expandedAttributes: Seq[Attribute] = table match { // If there is no table specified, use all input attributes. case None => input // If there is a table, pick out attributes that are part of this table. - case Some(t) => input.filter(_.qualifiers contains t) + case Some(t) => input.filter(_.qualifiers.filter(resolver(_, t)).nonEmpty) } val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map { case (n: NamedExpression, _) => n diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 7c4b9d4847e26..59fb0311a9c44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -59,6 +59,7 @@ abstract class Attribute extends NamedExpression { def withNullability(newNullability: Boolean): Attribute def withQualifiers(newQualifiers: Seq[String]): Attribute + def withName(newName: String): Attribute def toAttribute = this def newInstance: Attribute @@ -86,7 +87,6 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable - override def toAttribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) @@ -144,6 +144,14 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea } } + override def withName(newName: String): AttributeReference = { + if (name == newName) { + this + } else { + AttributeReference(newName, dataType, nullable)(exprId, qualifiers) + } + } + /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index ede431ad4ab27..28d863e58beca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.trees -abstract class LogicalPlan extends QueryPlan[LogicalPlan] { +abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { self: Product => /** @@ -75,20 +77,25 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { * nodes of this LogicalPlan. The attribute is expressed as * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ - def resolveChildren(name: String): Option[NamedExpression] = - resolve(name, children.flatMap(_.output)) + def resolveChildren(name: String, resolver: Resolver): Option[NamedExpression] = + resolve(name, children.flatMap(_.output), resolver) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this * LogicalPlan. The attribute is expressed as string in the following form: * `[scope].AttributeName.[nested].[fields]...`. */ - def resolve(name: String): Option[NamedExpression] = - resolve(name, output) + def resolve(name: String, resolver: Resolver): Option[NamedExpression] = + resolve(name, output, resolver) /** Performs attribute resolution given a name and a sequence of possible attributes. */ - protected def resolve(name: String, input: Seq[Attribute]): Option[NamedExpression] = { + protected def resolve( + name: String, + input: Seq[Attribute], + resolver: Resolver): Option[NamedExpression] = { + val parts = name.split("\\.") + // Collect all attributes that are output by this nodes children where either the first part // matches the name or where the first part matches the scope and the second part matches the // name. Return these matches along with any remaining parts, which represent dotted access to @@ -96,21 +103,69 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { val options = input.flatMap { option => // If the first part of the desired name matches a qualifier for this possible match, drop it. val remainingParts = - if (option.qualifiers.contains(parts.head) && parts.size > 1) parts.drop(1) else parts - if (option.name == remainingParts.head) (option, remainingParts.tail.toList) :: Nil else Nil + if (option.qualifiers.find(resolver(_, parts.head)).nonEmpty && parts.size > 1) { + parts.drop(1) + } else { + parts + } + + if (resolver(option.name, remainingParts.head)) { + // Preserve the case of the user's attribute reference. + (option.withName(remainingParts.head), remainingParts.tail.toList) :: Nil + } else { + Nil + } } options.distinct match { - case Seq((a, Nil)) => Some(a) // One match, no nested fields, use it. + // One match, no nested fields, use it. + case Seq((a, Nil)) => Some(a) + // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => - Some(Alias(nestedFields.foldLeft(a: Expression)(GetField), nestedFields.last)()) - case Seq() => None // No matches. + val aliased = + Alias( + resolveNesting(nestedFields, a, resolver), + nestedFields.last)() // Preserve the case of the user's field access. + Some(aliased) + + // No matches. + case Seq() => + logTrace(s"Could not find $name in ${input.mkString(", ")}") + None + + // More than one match. case ambiguousReferences => throw new TreeNodeException( this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") } } + + /** + * Given a list of successive nested field accesses, and a based expression, attempt to resolve + * the actual field lookups on this expression. + */ + private def resolveNesting( + nestedFields: List[String], + expression: Expression, + resolver: Resolver): Expression = { + + (nestedFields, expression.dataType) match { + case (Nil, _) => expression + case (requestedField :: rest, StructType(fields)) => + val actualField = fields.filter(f => resolver(f.name, requestedField)) + actualField match { + case Seq() => + sys.error( + s"No such struct field $requestedField in ${fields.map(_.name).mkString(", ")}") + case Seq(singleMatch) => + resolveNesting(rest, GetField(expression, singleMatch.name), resolver) + case multipleMatches => + sys.error(s"Ambiguous reference to fields ${multipleMatches.mkString(", ")}") + } + case (_, dt) => sys.error(s"Can't access nested field in type $dt") + } + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 8e8259cae6670..391508279bb80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -165,32 +165,6 @@ case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { override def output = child.output.map(_.withQualifiers(alias :: Nil)) } -/** - * Converts the schema of `child` to all lowercase, together with LowercaseAttributeReferences - * this allows for optional case insensitive attribute resolution. This node can be elided after - * analysis. - */ -case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { - protected def lowerCaseSchema(dataType: DataType): DataType = dataType match { - case StructType(fields) => - StructType(fields.map(f => - StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable))) - case ArrayType(elemType, containsNull) => ArrayType(lowerCaseSchema(elemType), containsNull) - case otherType => otherType - } - - override val output = child.output.map { - case a: AttributeReference => - AttributeReference( - a.name.toLowerCase, - lowerCaseSchema(a.dataType), - a.nullable)( - a.exprId, - a.qualifiers) - case other => other - } -} - case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7dbaf7faff0c0..b245e1a863cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -246,7 +246,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * @group userf */ def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { - catalog.registerTable(None, tableName, rdd.queryExecution.analyzed) + catalog.registerTable(None, tableName, rdd.queryExecution.logical) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 15f6bcef93886..08376eb5e5c4e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -381,7 +381,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3349 partitioning after limit") { - /* sql("SELECT DISTINCT n FROM lowerCaseData ORDER BY n DESC") .limit(2) .registerTempTable("subset1") @@ -396,7 +395,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM lowerCaseData INNER JOIN subset2 ON subset2.n = lowerCaseData.n"), (1, "a", 1) :: (2, "b", 2) :: Nil) - */ } test("mixed-case keywords") { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index e0be09e6793ea..3e1a7b71528e0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -244,15 +244,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { - override def lookupRelation( - databaseName: Option[String], - tableName: String, - alias: Option[String] = None): LogicalPlan = { - - LowerCaseSchema(super.lookupRelation(databaseName, tableName, alias)) - } - } + override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog // Note that HiveUDFs will be overridden by functions registered in this context. @transient 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 2c0db9be57e54..6b4399e852c7b 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 @@ -129,14 +129,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable( - LowerCaseSchema(table: MetastoreRelation), _, child, _) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => castChildOutput(p, table, child) case p @ logical.InsertIntoTable( - LowerCaseSchema( InMemoryRelation(_, _, _, - HiveTableScan(_, table, _))), _, child, _) => + HiveTableScan(_, table, _)), _, child, _) => castChildOutput(p, table, child) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 43dd3d234f73a..8ac17f37201a8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} @@ -55,7 +55,7 @@ private[hive] trait HiveStrategies { object ParquetConversion extends Strategy { implicit class LogicalPlanHacks(s: SchemaRDD) { def lowerCase = - new SchemaRDD(s.sqlContext, LowerCaseSchema(s.logicalPlan)) + new SchemaRDD(s.sqlContext, s.logicalPlan) def addPartitioningAttributes(attrs: Seq[Attribute]) = new SchemaRDD( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 71ea774d77795..1017fe6d5396d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.LowerCaseSchema import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.MetastoreRelation @@ -52,8 +51,7 @@ case class CreateTableAsSelect( sc.catalog.createTable(database, tableName, query.output, false) // Get the Metastore Relation sc.catalog.lookupRelation(Some(database), tableName, None) match { - case LowerCaseSchema(r: MetastoreRelation) => r - case o: MetastoreRelation => o + case r: MetastoreRelation => r } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 5a0e6c5cc1bba..19ff3b66ad7ed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -44,10 +44,11 @@ private[hive] abstract class HiveFunctionRegistry def lookupFunction(name: String, children: Seq[Expression]): Expression = { // We only look it up to see if it exists, but do not include it in the HiveUDF since it is // not always serializable. - val functionInfo: FunctionInfo = Option(FunctionRegistry.getFunctionInfo(name)).getOrElse( - sys.error(s"Couldn't find function $name")) + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(name.toLowerCase)).getOrElse( + sys.error(s"Couldn't find function $name")) - val functionClassName = functionInfo.getFunctionClass.getName() + val functionClassName = functionInfo.getFunctionClass.getName if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { val function = functionInfo.getFunctionClass.newInstance().asInstanceOf[UDF] diff --git a/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 b/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/database.table table.attr case insensitive-0-98b2e34c9134208e9fe7c62d33010005 @@ -0,0 +1 @@ +0 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 b6be6bc1bfefe..ee9d08ff75450 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 @@ -36,6 +36,9 @@ class HiveResolutionSuite extends HiveComparisonTest { createQueryTest("database.table table.attr", "SELECT src.key FROM default.src ORDER BY key LIMIT 1") + createQueryTest("database.table table.attr case insensitive", + "SELECT SRC.Key FROM Default.Src ORDER BY key LIMIT 1") + createQueryTest("alias.attr", "SELECT a.key FROM src a ORDER BY key LIMIT 1") @@ -56,14 +59,18 @@ class HiveResolutionSuite extends HiveComparisonTest { TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) .registerTempTable("caseSensitivityTest") - sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") - - println(sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").queryExecution) - - sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").collect() + val query = sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + assert(query.schema.fields.map(_.name) === Seq("a", "b", "A", "B", "a", "b", "A", "B"), + "The output schema did not preserve the case of the query.") + query.collect() + } - // TODO: sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a") + ignore("case insensitivity with scala reflection joins") { + // Test resolution with Scala Reflection + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2), Seq(Nested(1,2))) :: Nil) + .registerTempTable("caseSensitivityTest") + sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a").collect() } test("nested repeated resolution") { From 8e875d2aff5f30a5f7a4bf694fc89a8b852fdcdc Mon Sep 17 00:00:00 2001 From: WangTao Date: Sat, 20 Sep 2014 19:07:07 -0700 Subject: [PATCH 0758/1492] [SPARK-3599]Avoid loading properties file frequently https://issues.apache.org/jira/browse/SPARK-3599 Author: WangTao Author: WangTaoTheTonic Closes #2454 from WangTaoTheTonic/avoidLoadingFrequently and squashes the following commits: 3681182 [WangTao] do not use clone 7dca036 [WangTao] use lazy val instead 2a79f26 [WangTaoTheTonic] Avoid loaing properties file frequently --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../spark/deploy/SparkSubmitArguments.scala | 17 +++++++++-------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index d132ecb3f9989..580a439c9a892 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -280,7 +280,7 @@ object SparkSubmit { } // Read from default spark properties, if any - for ((k, v) <- args.getDefaultSparkProperties) { + for ((k, v) <- args.defaultSparkProperties) { sysProps.getOrElseUpdate(k, v) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index d545f58c5da7e..92e0917743ed1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -57,12 +57,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { var pyFiles: String = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() - parseOpts(args.toList) - mergeSparkProperties() - checkRequiredArguments() - - /** Return default present in the currently defined defaults file. */ - def getDefaultSparkProperties = { + /** Default properties present in the currently defined defaults file. */ + lazy val defaultSparkProperties: HashMap[String, String] = { val defaultProperties = new HashMap[String, String]() if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") Option(propertiesFile).foreach { filename => @@ -79,6 +75,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { defaultProperties } + parseOpts(args.toList) + mergeSparkProperties() + checkRequiredArguments() + /** * Fill in any undefined values based on the default properties file or options passed in through * the '--conf' flag. @@ -107,7 +107,8 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } } - val properties = getDefaultSparkProperties + val properties = HashMap[String, String]() + properties.putAll(defaultSparkProperties) properties.putAll(sparkProperties) // Use properties file as fallback for values which have a direct analog to @@ -213,7 +214,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { | verbose $verbose | |Default properties from $propertiesFile: - |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} + |${defaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } From d112a6c79dee7b5d8459696f97d329190e8d09a5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 20 Sep 2014 23:11:05 -0700 Subject: [PATCH 0759/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #1328 (close requested by 'pwendell') Closes #2314 (close requested by 'pwendell') Closes #997 (close requested by 'pwendell') Closes #550 (close requested by 'pwendell') Closes #1506 (close requested by 'pwendell') Closes #2423 (close requested by 'mengxr') Closes #554 (close requested by 'joshrosen') From a0454efe21e5c7ffe1b9bb7b18021a5580952e69 Mon Sep 17 00:00:00 2001 From: Ian Hummel Date: Sun, 21 Sep 2014 13:04:36 -0700 Subject: [PATCH 0760/1492] [SPARK-3595] Respect configured OutputCommitters when calling saveAsHadoopFile Addresses the issue in https://issues.apache.org/jira/browse/SPARK-3595, namely saveAsHadoopFile hardcoding the OutputCommitter. This is not ideal when running Spark jobs that write to S3, especially when running them from an EMR cluster where the default OutputCommitter is a DirectOutputCommitter. Author: Ian Hummel Closes #2450 from themodernlife/spark-3595 and squashes the following commits: f37a0e5 [Ian Hummel] Update based on comments from pwendell a11d9f3 [Ian Hummel] Fix formatting 4359664 [Ian Hummel] Add an example showing usage 8b6be94 [Ian Hummel] Add ability to specify OutputCommitter, espcially useful when writing to an S3 bucket from an EMR cluster --- .../org/apache/spark/SparkHadoopWriter.scala | 2 +- .../apache/spark/rdd/PairRDDFunctions.scala | 7 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 107 ++++++++++++++---- 3 files changed, 91 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index f6703986bdf11..376e69cd997d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -116,7 +116,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } } else { - logWarning ("No need to commit output of task: " + taID.value) + logInfo ("No need to commit output of task: " + taID.value) } } 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 f6d9d12fe9006..51ba8c2d17834 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -872,7 +872,12 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) hadoopConf.set("mapred.output.compression.codec", c.getCanonicalName) hadoopConf.set("mapred.output.compression.type", CompressionType.BLOCK.toString) } - hadoopConf.setOutputCommitter(classOf[FileOutputCommitter]) + + // Use configured output committer if already set + if (conf.getOutputCommitter == null) { + hadoopConf.setOutputCommitter(classOf[FileOutputCommitter]) + } + FileOutputFormat.setOutputPath(hadoopConf, SparkHadoopWriter.createPathFromString(path, hadoopConf)) saveAsHadoopDataset(hadoopConf) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 63d3ddb4af98a..e84cc69592339 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -17,17 +17,21 @@ package org.apache.spark.rdd -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.mapred._ +import org.apache.hadoop.util.Progressable + +import scala.collection.mutable.{ArrayBuffer, HashSet} import scala.util.Random -import org.scalatest.FunSuite import com.google.common.io.Files -import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.conf.{Configuration, Configurable} - -import org.apache.spark.SparkContext._ +import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter => NewOutputCommitter, +OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, +TaskAttemptContext => NewTaskAttempContext} import org.apache.spark.{Partitioner, SharedSparkContext} +import org.apache.spark.SparkContext._ +import org.scalatest.FunSuite class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("aggregateByKey") { @@ -467,7 +471,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) // No error, non-configurable formats still work - pairs.saveAsNewAPIHadoopFile[FakeFormat]("ignored") + pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored") /* Check that configurable formats get configured: @@ -478,6 +482,17 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") } + test("saveAsHadoopFile should respect configured output committers") { + val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) + val conf = new JobConf() + conf.setOutputCommitter(classOf[FakeOutputCommitter]) + + FakeOutputCommitter.ran = false + pairs.saveAsHadoopFile("ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) + + assert(FakeOutputCommitter.ran, "OutputCommitter was never called") + } + test("lookup") { val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) @@ -621,40 +636,86 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { and the test will therefore throw InstantiationException when saveAsNewAPIHadoopFile tries to instantiate them with Class.newInstance. */ + +/* + * Original Hadoop API + */ class FakeWriter extends RecordWriter[Integer, Integer] { + override def write(key: Integer, value: Integer): Unit = () - def close(p1: TaskAttemptContext) = () + override def close(reporter: Reporter): Unit = () +} + +class FakeOutputCommitter() extends OutputCommitter() { + override def setupJob(jobContext: JobContext): Unit = () + + override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true + + override def setupTask(taskContext: TaskAttemptContext): Unit = () + + override def commitTask(taskContext: TaskAttemptContext): Unit = { + FakeOutputCommitter.ran = true + () + } + + override def abortTask(taskContext: TaskAttemptContext): Unit = () +} + +/* + * Used to communicate state between the test harness and the OutputCommitter. + */ +object FakeOutputCommitter { + var ran = false +} + +class FakeOutputFormat() extends OutputFormat[Integer, Integer]() { + override def getRecordWriter( + ignored: FileSystem, + job: JobConf, name: String, + progress: Progressable): RecordWriter[Integer, Integer] = { + new FakeWriter() + } + + override def checkOutputSpecs(ignored: FileSystem, job: JobConf): Unit = () +} + +/* + * New-style Hadoop API + */ +class NewFakeWriter extends NewRecordWriter[Integer, Integer] { + + def close(p1: NewTaskAttempContext) = () def write(p1: Integer, p2: Integer) = () } -class FakeCommitter extends OutputCommitter { - def setupJob(p1: JobContext) = () +class NewFakeCommitter extends NewOutputCommitter { + def setupJob(p1: NewJobContext) = () - def needsTaskCommit(p1: TaskAttemptContext): Boolean = false + def needsTaskCommit(p1: NewTaskAttempContext): Boolean = false - def setupTask(p1: TaskAttemptContext) = () + def setupTask(p1: NewTaskAttempContext) = () - def commitTask(p1: TaskAttemptContext) = () + def commitTask(p1: NewTaskAttempContext) = () - def abortTask(p1: TaskAttemptContext) = () + def abortTask(p1: NewTaskAttempContext) = () } -class FakeFormat() extends OutputFormat[Integer, Integer]() { +class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { - def checkOutputSpecs(p1: JobContext) = () + def checkOutputSpecs(p1: NewJobContext) = () - def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { - new FakeWriter() + def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { + new NewFakeWriter() } - def getOutputCommitter(p1: TaskAttemptContext): OutputCommitter = { - new FakeCommitter() + def getOutputCommitter(p1: NewTaskAttempContext): NewOutputCommitter = { + new NewFakeCommitter() } } -class ConfigTestFormat() extends FakeFormat() with Configurable { +class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false def setConf(p1: Configuration) = { @@ -664,7 +725,7 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { def getConf: Configuration = null - override def getRecordWriter(p1: TaskAttemptContext): RecordWriter[Integer, Integer] = { + override def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { assert(setConfCalled, "setConf was never called") super.getRecordWriter(p1) } From fd0b32c520e3d1088b2fe9228be114932e6c3a0c Mon Sep 17 00:00:00 2001 From: wangfei Date: Sun, 21 Sep 2014 13:09:36 -0700 Subject: [PATCH 0761/1492] [Minor]ignore .idea_modules ignore .idea_modules , ```sbt/sbt gen-idea``` generate this dir. Author: wangfei Closes #2476 from scwf/patch-4 and squashes the following commits: e6ab88a [wangfei] ignore .idea_modules --- .gitignore | 1 + 1 file changed, 1 insertion(+) diff --git a/.gitignore b/.gitignore index 1bcd0165761ac..7779980b74a22 100644 --- a/.gitignore +++ b/.gitignore @@ -6,6 +6,7 @@ *.iml *.iws .idea/ +.idea_modules/ sbt/*.jar .settings .cache From fec921552ffccc36937214406b3e4a050eb0d8e0 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Mon, 22 Sep 2014 09:10:41 -0700 Subject: [PATCH 0762/1492] [MLLib] Fix example code variable name misspelling in MLLib Feature Extraction guide Author: RJ Nowling Closes #2459 from rnowling/tfidf-fix and squashes the following commits: b370a91 [RJ Nowling] Fix variable name misspelling in MLLib Feature Extraction guide --- docs/mllib-feature-extraction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 44f0f76220b6e..41a27f6208d1b 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -68,7 +68,7 @@ val sc: SparkContext = ... val documents: RDD[Seq[String]] = sc.textFile("...").map(_.split(" ").toSeq) val hashingTF = new HashingTF() -val tf: RDD[Vector] = hasingTF.transform(documents) +val tf: RDD[Vector] = hashingTF.transform(documents) {% endhighlight %} While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: From 56dae30ca70489a62686cb245728b09b2179bb5a Mon Sep 17 00:00:00 2001 From: Grega Kespret Date: Mon, 22 Sep 2014 10:13:44 -0700 Subject: [PATCH 0763/1492] Update docs to use jsonRDD instead of wrong jsonRdd. Author: Grega Kespret Closes #2479 from gregakespret/patch-1 and squashes the following commits: dd6b90a [Grega Kespret] Update docs to use jsonRDD instead of wrong jsonRdd. --- docs/sql-programming-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 5212e19c41349..c1f80544bf0af 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -605,7 +605,7 @@ Spark SQL can automatically infer the schema of a JSON dataset and load it as a This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. -* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight scala %} // sc is an existing SparkContext. @@ -643,7 +643,7 @@ Spark SQL can automatically infer the schema of a JSON dataset and load it as a This conversion can be done using one of two methods in a JavaSQLContext : * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. -* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight java %} // sc is an existing JavaSparkContext. @@ -681,7 +681,7 @@ Spark SQL can automatically infer the schema of a JSON dataset and load it as a This conversion can be done using one of two methods in a SQLContext: * `jsonFile` - loads data from a directory of JSON files where each line of the files is a JSON object. -* `jsonRdd` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. +* `jsonRDD` - loads data from an existing RDD where each element of the RDD is a string containing a JSON object. {% highlight python %} # sc is an existing SparkContext. From f9d6220c792b779be385f3022d146911a22c2130 Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Mon, 22 Sep 2014 13:47:43 -0700 Subject: [PATCH 0764/1492] [SPARK-3578] Fix upper bound in GraphGenerators.sampleLogNormal GraphGenerators.sampleLogNormal is supposed to return an integer strictly less than maxVal. However, it violates this guarantee. It generates its return value as follows: ```scala var X: Double = maxVal while (X >= maxVal) { val Z = rand.nextGaussian() X = math.exp(mu + sigma*Z) } math.round(X.toFloat) ``` When X is sampled to be close to (but less than) maxVal, then it will pass the while loop condition, but the rounded result will be equal to maxVal, which will violate the guarantee. For example, if maxVal is 5 and X is 4.9, then X < maxVal, but `math.round(X.toFloat)` is 5. This PR instead rounds X before checking the loop condition, guaranteeing that the condition will hold for the return value. Author: Ankur Dave Closes #2439 from ankurdave/SPARK-3578 and squashes the following commits: f6655e5 [Ankur Dave] Go back to math.floor 5900c22 [Ankur Dave] Round X in loop condition 6fd5fb1 [Ankur Dave] Run sampleLogNormal bounds check 1000 times 1638598 [Ankur Dave] Round down in sampleLogNormal to guarantee upper bound --- .../org/apache/spark/graphx/util/GraphGenerators.scala | 2 +- .../apache/spark/graphx/util/GraphGeneratorsSuite.scala | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index b8309289fe475..8a13c74221546 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -118,7 +118,7 @@ object GraphGenerators { val Z = rand.nextGaussian() X = math.exp(mu + sigma*Z) } - math.round(X.toFloat) + math.floor(X).toInt } /** diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala index b346d4db2ef96..3abefbe52fa8a 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/util/GraphGeneratorsSuite.scala @@ -64,8 +64,11 @@ class GraphGeneratorsSuite extends FunSuite with LocalSparkContext { val sigma = 1.3 val maxVal = 100 - val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) - assert(dstId < maxVal) + val trials = 1000 + for (i <- 1 to trials) { + val dstId = GraphGenerators.sampleLogNormal(mu, sigma, maxVal) + assert(dstId < maxVal) + } val dstId_round1 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) val dstId_round2 = GraphGenerators.sampleLogNormal(mu, sigma, maxVal, 12345) From 14f8c340402366cb998c563b3f7d9ff7d9940271 Mon Sep 17 00:00:00 2001 From: "peng.zhang" Date: Tue, 23 Sep 2014 08:45:56 -0500 Subject: [PATCH 0765/1492] [YARN] SPARK-2668: Add variable of yarn log directory for reference from the log4j configuration MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Assign value of yarn container log directory to java opts "spark.yarn.app.container.log.dir", So user defined log4j.properties can reference this value and write log to YARN container's log directory. Otherwise, user defined file appender will only write to container's CWD, and log files in CWD will not be displayed on YARN UI,and either cannot be aggregated to HDFS log directory after job finished. User defined log4j.properties reference example: log4j.appender.rolling_file.File = ${spark.yarn.app.container.log.dir}/spark.log Author: peng.zhang Closes #1573 from renozhang/yarn-log-dir and squashes the following commits: 16c5cb8 [peng.zhang] Update doc f2b5e2a [peng.zhang] Change variable's name, and update running-on-yarn.md 503ea2d [peng.zhang] Support log4j log to yarn container dir --- docs/running-on-yarn.md | 2 ++ .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 +++ .../org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala | 3 +++ 3 files changed, 8 insertions(+) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 74bcc2eeb65f6..4b3a49eca7007 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -205,6 +205,8 @@ Note that for the first option, both executors and the application master will s log4j configuration, which may cause issues when they run on the same node (e.g. trying to write to the same log file). +If you need a reference to the proper location to put log files in the YARN so that YARN can properly display and aggregate them, use "${spark.yarn.app.container.log.dir}" in your log4j.properties. For example, log4j.appender.file_appender.File=${spark.yarn.app.container.log.dir}/spark.log. For streaming application, configuring RollingFileAppender and setting file location to YARN's log directory will avoid disk overflow caused by large log file, and logs can be accessed using YARN's log utility. + # Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index c96f731923d22..6ae4d496220a5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -388,6 +388,9 @@ trait ClientBase extends Logging { .foreach(p => javaOpts += s"-Djava.library.path=$p") } + // For log4j configuration to reference + javaOpts += "-D=spark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + val userClass = if (args.userClass != null) { Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 312d82a649792..f56f72cafe50e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -98,6 +98,9 @@ trait ExecutorRunnableUtil extends Logging { } */ + // For log4j configuration to reference + javaOpts += "-D=spark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server", // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. From c4022dd52b4827323ff956632dc7623f546da937 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 23 Sep 2014 11:20:52 -0500 Subject: [PATCH 0766/1492] [SPARK-3477] Clean up code in Yarn Client / ClientBase This is part of a broader effort to clean up the Yarn integration code after #2020. The high-level changes in this PR include: - Removing duplicate code, especially across the alpha and stable APIs - Simplify unnecessarily complex method signatures and hierarchies - Rename unclear variable and method names - Organize logging output produced when the user runs Spark on Yarn - Extensively add documentation - Privatize classes where possible I have tested the stable API on a Hadoop 2.4 cluster. I tested submitting a jar that references classes in other jars in both client and cluster mode. I also made changes in the alpha API, though I do not have access to an alpha cluster. I have verified that it compiles, but it would be ideal if others can help test it. For those interested in some examples in detail, please read on. -------------------------------------------------------------------------------------------------------- ***Appendix*** - The loop to `getApplicationReport` from the RM is duplicated in 4 places: in the stable `Client`, alpha `Client`, and twice in `YarnClientSchedulerBackend`. We should not have different loops for client and cluster deploy modes. - There are many fragmented small helper methods that are only used once and should just be inlined. For instance, `ClientBase#getLocalPath` returns `null` on certain conditions, and its only caller `ClientBase#addFileToClasspath` checks whether the value returned is `null`. We could just have the caller check on that same condition to avoid passing `null`s around. - In `YarnSparkHadoopUtil#addToEnvironment`, we take in an argument `classpathSeparator` that always has the same value upstream (i.e. `File.pathSeparator`). This argument is now removed from the signature and all callers of this method upstream. - `ClientBase#copyRemoteFile` is now renamed to `copyFileToRemote`. It was unclear whether we are copying a remote file to our local file system, or copying a locally visible file to a remote file system. Also, even the content of the method has inaccurately named variables. We use `val remoteFs` to signify the file system of the locally visible file and `val fs` to signify the remote, destination file system. These are now renamed `srcFs` and `destFs` respectively. - We currently log the AM container's environment and resource mappings directly as Scala collections. This is incredibly hard to read and probably too verbose for the average Spark user. In other modes (e.g. standalone), we also don't log the launch commands by default, so the logging level of these information is now set to `DEBUG`. - None of these classes (`Client`, `ClientBase`, `YarnSparkHadoopUtil` etc.) is intended to be used by a Spark application (the user should go through Spark submit instead). At the very least they should be `private[spark]`. Author: Andrew Or Closes #2350 from andrewor14/yarn-cleanup and squashes the following commits: 39e8c7b [Andrew Or] Address review comments 6619f9b [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-cleanup 2ca6d64 [Andrew Or] Improve logging in application monitor a3b9693 [Andrew Or] Minor changes 7dd6298 [Andrew Or] Simplify ClientBase#monitorApplication 547487c [Andrew Or] Provide default values for null application report entries a0ad1e9 [Andrew Or] Fix class not found error 1590141 [Andrew Or] Address review comments 45ccdea [Andrew Or] Remove usages of getAMMemory d8e33b6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-cleanup ed0b42d [Andrew Or] Fix alpha compilation error c0587b4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-cleanup 6d74888 [Andrew Or] Minor comment changes 6573c1d [Andrew Or] Clean up, simplify and document code for setting classpaths e4779b6 [Andrew Or] Clean up log messages + variable naming in ClientBase 8766d37 [Andrew Or] Heavily add documentation to Client* classes + various clean-ups 6c94d79 [Andrew Or] Various cleanups in ClientBase and ClientArguments ef7069a [Andrew Or] Clean up YarnClientSchedulerBackend more 6de9072 [Andrew Or] Guard against potential NPE in debug logging mode fabe4c4 [Andrew Or] Reuse more code in YarnClientSchedulerBackend 3f941dc [Andrew Or] First cut at simplifying the Client (stable and alpha) --- .../org/apache/spark/deploy/yarn/Client.scala | 145 ++-- .../spark/deploy/yarn/ClientArguments.scala | 67 +- .../apache/spark/deploy/yarn/ClientBase.scala | 682 +++++++++++------- .../yarn/ClientDistributedCacheManager.scala | 97 +-- .../deploy/yarn/ExecutorRunnableUtil.scala | 16 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 63 +- .../cluster/YarnClientSchedulerBackend.scala | 145 ++-- .../spark/deploy/yarn/ClientBaseSuite.scala | 18 +- .../org/apache/spark/deploy/yarn/Client.scala | 167 ++--- 9 files changed, 738 insertions(+), 662 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index aff9ab71f0937..5a20532315e59 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -23,13 +23,11 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.YarnClientImpl import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, Records} +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil @@ -37,7 +35,10 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. */ -class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) +private[spark] class Client( + val args: ClientArguments, + val hadoopConf: Configuration, + val sparkConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { def this(clientArgs: ClientArguments, spConf: SparkConf) = @@ -45,112 +46,86 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - val args = clientArgs - val conf = hadoopConf - val sparkConf = spConf - var rpc: YarnRPC = YarnRPC.create(conf) - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) + val yarnConf: YarnConfiguration = new YarnConfiguration(hadoopConf) + /* ------------------------------------------------------------------------------------- * + | The following methods have much in common in the stable and alpha versions of Client, | + | but cannot be implemented in the parent trait due to subtle API differences across | + | hadoop versions. | + * ------------------------------------------------------------------------------------- */ - // for client user who want to monitor app status by itself. - def runApp() = { - validateArgs() - + /** Submit an application running our ApplicationMaster to the ResourceManager. */ + override def submitApplication(): ApplicationId = { init(yarnConf) start() - logClusterResourceDetails() - val newApp = super.getNewApplication() - val appId = newApp.getApplicationId() + logInfo("Requesting a new application from cluster with %d NodeManagers" + .format(getYarnClusterMetrics.getNumNodeManagers)) - verifyClusterResources(newApp) - val appContext = createApplicationSubmissionContext(appId) - val appStagingDir = getAppStagingDir(appId) - val localResources = prepareLocalResources(appStagingDir) - val env = setupLaunchEnv(localResources, appStagingDir) - val amContainer = createContainerLaunchContext(newApp, localResources, env) + // Get a new application from our RM + val newAppResponse = getNewApplication() + val appId = newAppResponse.getApplicationId() - val capability = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - // Memory for the ApplicationMaster. - capability.setMemory(args.amMemory + memoryOverhead) - amContainer.setResource(capability) + // Verify whether the cluster has enough resources for our AM + verifyClusterResources(newAppResponse) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(amContainer) - appContext.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) + // Set up the appropriate contexts to launch our AM + val containerContext = createContainerLaunchContext(newAppResponse) + val appContext = createApplicationSubmissionContext(appId, containerContext) - submitApp(appContext) + // Finally, submit and monitor the application + logInfo(s"Submitting application ${appId.getId} to ResourceManager") + submitApplication(appContext) appId } - def run() { - val appId = runApp() - monitorApplication(appId) - } - - def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = super.getYarnClusterMetrics - logInfo("Got cluster metric info from ASM, numNodeManagers = " + - clusterMetrics.getNumNodeManagers) + /** + * Set up a context for launching our ApplicationMaster container. + * In the Yarn alpha API, the memory requirements of this container must be set in + * the ContainerLaunchContext instead of the ApplicationSubmissionContext. + */ + override def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + : ContainerLaunchContext = { + val containerContext = super.createContainerLaunchContext(newAppResponse) + val capability = Records.newRecord(classOf[Resource]) + capability.setMemory(args.amMemory + amMemoryOverhead) + containerContext.setResource(capability) + containerContext } - - def createApplicationSubmissionContext(appId: ApplicationId): ApplicationSubmissionContext = { - logInfo("Setting up application submission context for ASM") + /** Set up the context for submitting our ApplicationMaster. */ + def createApplicationSubmissionContext( + appId: ApplicationId, + containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) appContext.setApplicationId(appId) appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(containerContext) + appContext.setUser(UserGroupInformation.getCurrentUser.getShortUserName) appContext } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { - // Setup security tokens. + /** + * Set up security tokens for launching our ApplicationMaster container. + * ContainerLaunchContext#setContainerTokens is renamed `setTokens` in the stable API. + */ + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { val dob = new DataOutputBuffer() credentials.writeTokenStorageToStream(dob) amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) } - def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager. - logInfo("Submitting application to ASM") - super.submitApplication(appContext) - } - - def monitorApplication(appId: ApplicationId): Boolean = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - - while (true) { - Thread.sleep(interval) - val report = super.getApplicationReport(appId) - - logInfo("Application report from ASM: \n" + - "\t application identifier: " + appId.toString() + "\n" + - "\t appId: " + appId.getId() + "\n" + - "\t clientToken: " + report.getClientToken() + "\n" + - "\t appDiagnostics: " + report.getDiagnostics() + "\n" + - "\t appMasterHost: " + report.getHost() + "\n" + - "\t appQueue: " + report.getQueue() + "\n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + - "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + - "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + - "\t appUser: " + report.getUser() - ) - - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - return true - } - } - true - } + /** + * Return the security token used by this client to communicate with the ApplicationMaster. + * If no security is enabled, the token returned by the report is null. + * ApplicationReport#getClientToken is renamed `getClientToAMToken` in the stable API. + */ + override def getClientToken(report: ApplicationReport): String = + Option(report.getClientToken).getOrElse("") } object Client { - def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -158,19 +133,17 @@ object Client { } // Set an env variable indicating we are running in YARN mode. - // Note that anything with SPARK prefix gets propagated to all (remote) processes + // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf try { val args = new ClientArguments(argStrings, sparkConf) new Client(args, sparkConf).run() } catch { - case e: Exception => { + case e: Exception => Console.err.println(e.getMessage) System.exit(1) - } } System.exit(0) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 40d8d6d6e6961..201b742736c6e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -17,15 +17,14 @@ package org.apache.spark.deploy.yarn -import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf -import org.apache.spark.scheduler.InputFormatInfo import org.apache.spark.util.{Utils, IntParam, MemoryParam} // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! -class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { +private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) { var addJars: String = null var files: String = null var archives: String = null @@ -35,28 +34,56 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var executorMemory = 1024 // MB var executorCores = 1 var numExecutors = 2 - var amQueue = sparkConf.get("QUEUE", "default") + var amQueue = sparkConf.get("spark.yarn.queue", "default") var amMemory: Int = 512 // MB var appName: String = "Spark" var priority = 0 - parseArgs(args.toList) + // Additional memory to allocate to containers + // For now, use driver's memory overhead as our AM container's memory overhead + val amMemoryOverhead = sparkConf.getInt( + "spark.yarn.driver.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) + val executorMemoryOverhead = sparkConf.getInt( + "spark.yarn.executor.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - // env variable SPARK_YARN_DIST_ARCHIVES/SPARK_YARN_DIST_FILES set in yarn-client then - // it should default to hdfs:// - files = Option(files).getOrElse(sys.env.get("SPARK_YARN_DIST_FILES").orNull) - archives = Option(archives).getOrElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES").orNull) + parseArgs(args.toList) + loadEnvironmentArgs() + validateArgs() + + /** Load any default arguments provided through environment variables and Spark properties. */ + private def loadEnvironmentArgs(): Unit = { + // For backward compatibility, SPARK_YARN_DIST_{ARCHIVES/FILES} should be resolved to hdfs://, + // while spark.yarn.dist.{archives/files} should be resolved to file:// (SPARK-2051). + files = Option(files) + .orElse(sys.env.get("SPARK_YARN_DIST_FILES")) + .orElse(sparkConf.getOption("spark.yarn.dist.files").map(p => Utils.resolveURIs(p))) + .orNull + archives = Option(archives) + .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) + .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) + .orNull + } - // spark.yarn.dist.archives/spark.yarn.dist.files defaults to use file:// if not specified, - // for both yarn-client and yarn-cluster - files = Option(files).getOrElse(sparkConf.getOption("spark.yarn.dist.files"). - map(p => Utils.resolveURIs(p)).orNull) - archives = Option(archives).getOrElse(sparkConf.getOption("spark.yarn.dist.archives"). - map(p => Utils.resolveURIs(p)).orNull) + /** + * Fail fast if any arguments provided are invalid. + * This is intended to be called only after the provided arguments have been parsed. + */ + private def validateArgs(): Unit = { + // TODO: memory checks are outdated (SPARK-3476) + Map[Boolean, String]( + (numExecutors <= 0) -> "You must specify at least 1 executor!", + (amMemory <= amMemoryOverhead) -> s"AM memory must be > $amMemoryOverhead MB", + (executorMemory <= executorMemoryOverhead) -> + s"Executor memory must be > $executorMemoryOverhead MB" + ).foreach { case (errorCondition, errorMessage) => + if (errorCondition) { + throw new IllegalArgumentException(errorMessage + "\n" + getUsageMessage()) + } + } + } private def parseArgs(inputArgs: List[String]): Unit = { - val userArgsBuffer: ArrayBuffer[String] = new ArrayBuffer[String]() - + val userArgsBuffer = new ArrayBuffer[String]() var args = inputArgs while (!args.isEmpty) { @@ -138,16 +165,14 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { userArgs = userArgsBuffer.readOnly } - - def getUsageMessage(unknownParam: Any = null): String = { + private def getUsageMessage(unknownParam: List[String] = null): String = { val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" - message + "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + " --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" + " --class CLASS_NAME Name of your application's main class (required)\n" + - " --arg ARGS Argument to be passed to your application's main class.\n" + + " --arg ARG Argument to be passed to your application's main class.\n" + " Multiple invocations are possible, each will be passed in order.\n" + " --num-executors NUM Number of executors to start (Default: 2)\n" + " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 6ae4d496220a5..4870b0cb3ddaf 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.io.File import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import scala.collection.JavaConversions._ @@ -37,154 +36,107 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records + import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} /** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the YARN ResourceManager. + * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. + * The Client submits an application to the YARN ResourceManager. */ -trait ClientBase extends Logging { - val args: ClientArguments - val conf: Configuration - val sparkConf: SparkConf - val yarnConf: YarnConfiguration - val credentials = UserGroupInformation.getCurrentUser().getCredentials() - private val SPARK_STAGING: String = ".sparkStaging" +private[spark] trait ClientBase extends Logging { + import ClientBase._ + + protected val args: ClientArguments + protected val hadoopConf: Configuration + protected val sparkConf: SparkConf + protected val yarnConf: YarnConfiguration + protected val credentials = UserGroupInformation.getCurrentUser.getCredentials + protected val amMemoryOverhead = args.amMemoryOverhead // MB + protected val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() - // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) - // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) - - // Additional memory overhead - in mb. - protected def memoryOverhead: Int = sparkConf.getInt("spark.yarn.driver.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - - // TODO(harvey): This could just go in ClientArguments. - def validateArgs() = { - Map( - (args.numExecutors <= 0) -> "Error: You must specify at least 1 executor!", - (args.amMemory <= memoryOverhead) -> ("Error: AM memory size must be" + - "greater than: " + memoryOverhead), - (args.executorMemory <= memoryOverhead) -> ("Error: Executor memory size" + - "must be greater than: " + memoryOverhead.toString) - ).foreach { case(cond, errStr) => - if (cond) { - logError(errStr) - throw new IllegalArgumentException(args.getUsageMessage()) - } - } - } - - def getAppStagingDir(appId: ApplicationId): String = { - SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR - } - - def verifyClusterResources(app: GetNewApplicationResponse) = { - val maxMem = app.getMaximumResourceCapability().getMemory() - logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) - - // If we have requested more then the clusters max for a single resource then exit. - if (args.executorMemory > maxMem) { - val errorMessage = - "Required executor memory (%d MB), is above the max threshold (%d MB) of this cluster." - .format(args.executorMemory, maxMem) - - logError(errorMessage) - throw new IllegalArgumentException(errorMessage) - } - val amMem = args.amMemory + memoryOverhead + /** + * Fail fast if we have requested more resources per container than is available in the cluster. + */ + protected def verifyClusterResources(newAppResponse: GetNewApplicationResponse): Unit = { + val maxMem = newAppResponse.getMaximumResourceCapability().getMemory() + logInfo("Verifying our application has not requested more than the maximum " + + s"memory capability of the cluster ($maxMem MB per container)") + val executorMem = args.executorMemory + executorMemoryOverhead + if (executorMem > maxMem) { + throw new IllegalArgumentException(s"Required executor memory ($executorMem MB) " + + s"is above the max threshold ($maxMem MB) of this cluster!") + } + val amMem = args.amMemory + amMemoryOverhead if (amMem > maxMem) { - - val errorMessage = "Required AM memory (%d) is above the max threshold (%d) of this cluster." - .format(amMem, maxMem) - logError(errorMessage) - throw new IllegalArgumentException(errorMessage) + throw new IllegalArgumentException(s"Required AM memory ($amMem MB) " + + s"is above the max threshold ($maxMem MB) of this cluster!") } - // We could add checks to make sure the entire cluster has enough resources but that involves // getting all the node reports and computing ourselves. } - /** See if two file systems are the same or not. */ - private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { - val srcUri = srcFs.getUri() - val dstUri = destFs.getUri() - if (srcUri.getScheme() == null) { - return false - } - if (!srcUri.getScheme().equals(dstUri.getScheme())) { - return false - } - var srcHost = srcUri.getHost() - var dstHost = dstUri.getHost() - if ((srcHost != null) && (dstHost != null)) { - try { - srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() - dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() - } catch { - case e: UnknownHostException => - return false - } - if (!srcHost.equals(dstHost)) { - return false - } - } else if (srcHost == null && dstHost != null) { - return false - } else if (srcHost != null && dstHost == null) { - return false - } - if (srcUri.getPort() != dstUri.getPort()) { - false - } else { - true - } - } - - /** Copy the file into HDFS if needed. */ - private[yarn] def copyRemoteFile( - dstDir: Path, - originalPath: Path, + /** + * Copy the given file to a remote file system (e.g. HDFS) if needed. + * The file is only copied if the source and destination file systems are different. This is used + * for preparing resources for launching the ApplicationMaster container. Exposed for testing. + */ + def copyFileToRemote( + destDir: Path, + srcPath: Path, replication: Short, setPerms: Boolean = false): Path = { - val fs = FileSystem.get(conf) - val remoteFs = originalPath.getFileSystem(conf) - var newPath = originalPath - if (!compareFs(remoteFs, fs)) { - newPath = new Path(dstDir, originalPath.getName()) - logInfo("Uploading " + originalPath + " to " + newPath) - FileUtil.copy(remoteFs, originalPath, fs, newPath, false, conf) - fs.setReplication(newPath, replication) - if (setPerms) fs.setPermission(newPath, new FsPermission(APP_FILE_PERMISSION)) + val destFs = destDir.getFileSystem(hadoopConf) + val srcFs = srcPath.getFileSystem(hadoopConf) + var destPath = srcPath + if (!compareFs(srcFs, destFs)) { + destPath = new Path(destDir, srcPath.getName()) + logInfo(s"Uploading resource $srcPath -> $destPath") + FileUtil.copy(srcFs, srcPath, destFs, destPath, false, hadoopConf) + destFs.setReplication(destPath, replication) + if (setPerms) { + destFs.setPermission(destPath, new FsPermission(APP_FILE_PERMISSION)) + } + } else { + logInfo(s"Source and destination file systems are the same. Not copying $srcPath") } // Resolve any symlinks in the URI path so using a "current" symlink to point to a specific // version shows the specific version in the distributed cache configuration - val qualPath = fs.makeQualified(newPath) - val fc = FileContext.getFileContext(qualPath.toUri(), conf) - val destPath = fc.resolvePath(qualPath) - destPath + val qualifiedDestPath = destFs.makeQualified(destPath) + val fc = FileContext.getFileContext(qualifiedDestPath.toUri(), hadoopConf) + fc.resolvePath(qualifiedDestPath) } - private def qualifyForLocal(localURI: URI): Path = { - var qualifiedURI = localURI - // If not specified, assume these are in the local filesystem to keep behavior like Hadoop - if (qualifiedURI.getScheme() == null) { - qualifiedURI = new URI(FileSystem.getLocal(conf).makeQualified(new Path(qualifiedURI)).toString) - } + /** + * Given a local URI, resolve it and return a qualified local path that corresponds to the URI. + * This is used for preparing local resources to be included in the container launch context. + */ + private def getQualifiedLocalPath(localURI: URI): Path = { + val qualifiedURI = + if (localURI.getScheme == null) { + // If not specified, assume this is in the local filesystem to keep the behavior + // consistent with that of Hadoop + new URI(FileSystem.getLocal(hadoopConf).makeQualified(new Path(localURI)).toString) + } else { + localURI + } new Path(qualifiedURI) } + /** + * Upload any resources to the distributed cache if needed. If a resource is intended to be + * consumed locally, set up the appropriate config for downstream code to handle it properly. + * This is used for setting up a container launch context for our ApplicationMaster. + * Exposed for testing. + */ def prepareLocalResources(appStagingDir: String): HashMap[String, LocalResource] = { - logInfo("Preparing Local resources") - // Upload Spark and the application JAR to the remote file system if necessary. Add them as - // local resources to the application master. - val fs = FileSystem.get(conf) + logInfo("Preparing resources for our AM container") + // Upload Spark and the application JAR to the remote file system if necessary, + // and add them as local resources to the application master. + val fs = FileSystem.get(hadoopConf) val dst = new Path(fs.getHomeDirectory(), appStagingDir) - val nns = ClientBase.getNameNodesToAccess(sparkConf) + dst - ClientBase.obtainTokensForNamenodes(nns, conf, credentials) + val nns = getNameNodesToAccess(sparkConf) + dst + obtainTokensForNamenodes(nns, hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort val localResources = HashMap[String, LocalResource]() @@ -200,73 +152,84 @@ trait ClientBase extends Logging { "for alternatives.") } + /** + * Copy the given main resource to the distributed cache if the scheme is not "local". + * Otherwise, set the corresponding key in our SparkConf to handle it downstream. + * Each resource is represented by a 4-tuple of: + * (1) destination resource name, + * (2) local path to the resource, + * (3) Spark property key to set if the scheme is not local, and + * (4) whether to set permissions for this resource + */ List( - (ClientBase.SPARK_JAR, ClientBase.sparkJar(sparkConf), ClientBase.CONF_SPARK_JAR), - (ClientBase.APP_JAR, args.userJar, ClientBase.CONF_SPARK_USER_JAR), - ("log4j.properties", oldLog4jConf.getOrElse(null), null) - ).foreach { case(destName, _localPath, confKey) => + (SPARK_JAR, sparkJar(sparkConf), CONF_SPARK_JAR, false), + (APP_JAR, args.userJar, CONF_SPARK_USER_JAR, true), + ("log4j.properties", oldLog4jConf.orNull, null, false) + ).foreach { case (destName, _localPath, confKey, setPermissions) => val localPath: String = if (_localPath != null) _localPath.trim() else "" - if (! localPath.isEmpty()) { + if (!localPath.isEmpty()) { val localURI = new URI(localPath) - if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { - val setPermissions = destName.equals(ClientBase.APP_JAR) - val destPath = copyRemoteFile(dst, qualifyForLocal(localURI), replication, setPermissions) - val destFs = FileSystem.get(destPath.toUri(), conf) - distCacheMgr.addResource(destFs, conf, destPath, localResources, LocalResourceType.FILE, - destName, statCache) + if (localURI.getScheme != LOCAL_SCHEME) { + val src = getQualifiedLocalPath(localURI) + val destPath = copyFileToRemote(dst, src, replication, setPermissions) + val destFs = FileSystem.get(destPath.toUri(), hadoopConf) + distCacheMgr.addResource(destFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, destName, statCache) } else if (confKey != null) { + // If the resource is intended for local use only, handle this downstream + // by setting the appropriate property sparkConf.set(confKey, localPath) } } } + /** + * Do the same for any additional resources passed in through ClientArguments. + * Each resource category is represented by a 3-tuple of: + * (1) comma separated list of resources in this category, + * (2) resource type, and + * (3) whether to add these resources to the classpath + */ val cachedSecondaryJarLinks = ListBuffer.empty[String] - val fileLists = List( (args.addJars, LocalResourceType.FILE, true), + List( + (args.addJars, LocalResourceType.FILE, true), (args.files, LocalResourceType.FILE, false), - (args.archives, LocalResourceType.ARCHIVE, false) ) - fileLists.foreach { case (flist, resType, addToClasspath) => + (args.archives, LocalResourceType.ARCHIVE, false) + ).foreach { case (flist, resType, addToClasspath) => if (flist != null && !flist.isEmpty()) { - flist.split(',').foreach { case file: String => + flist.split(',').foreach { file => val localURI = new URI(file.trim()) - if (!ClientBase.LOCAL_SCHEME.equals(localURI.getScheme())) { + if (localURI.getScheme != LOCAL_SCHEME) { val localPath = new Path(localURI) val linkname = Option(localURI.getFragment()).getOrElse(localPath.getName()) - val destPath = copyRemoteFile(dst, localPath, replication) - distCacheMgr.addResource(fs, conf, destPath, localResources, resType, - linkname, statCache) + val destPath = copyFileToRemote(dst, localPath, replication) + distCacheMgr.addResource( + fs, hadoopConf, destPath, localResources, resType, linkname, statCache) if (addToClasspath) { cachedSecondaryJarLinks += linkname } } else if (addToClasspath) { + // Resource is intended for local use only and should be added to the class path cachedSecondaryJarLinks += file.trim() } } } } - logInfo("Prepared Local resources " + localResources) - sparkConf.set(ClientBase.CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) + if (cachedSecondaryJarLinks.nonEmpty) { + sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) + } - UserGroupInformation.getCurrentUser().addCredentials(credentials) localResources } - /** Get all application master environment variables set on this SparkConf */ - def getAppMasterEnv: Seq[(String, String)] = { - val prefix = "spark.yarn.appMasterEnv." - sparkConf.getAll.filter{case (k, v) => k.startsWith(prefix)} - .map{case (k, v) => (k.substring(prefix.length), v)} - } - - - def setupLaunchEnv( - localResources: HashMap[String, LocalResource], - stagingDir: String): HashMap[String, String] = { - logInfo("Setting up the launch environment") - + /** + * Set up the environment for launching our ApplicationMaster container. + */ + private def setupLaunchEnv(stagingDir: String): HashMap[String, String] = { + logInfo("Setting up the launch environment for our AM container") val env = new HashMap[String, String]() - val extraCp = sparkConf.getOption("spark.driver.extraClassPath") - ClientBase.populateClasspath(args, yarnConf, sparkConf, env, extraCp) + populateClasspath(args, yarnConf, sparkConf, env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -275,42 +238,20 @@ trait ClientBase extends Logging { distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) - getAppMasterEnv.foreach { case (key, value) => - YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) - } + // Pick up any environment variables for the AM provided through spark.yarn.appMasterEnv.* + val amEnvPrefix = "spark.yarn.appMasterEnv." + sparkConf.getAll + .filter { case (k, v) => k.startsWith(amEnvPrefix) } + .map { case (k, v) => (k.substring(amEnvPrefix.length), v) } + .foreach { case (k, v) => YarnSparkHadoopUtil.addPathToEnvironment(env, k, v) } // Keep this for backwards compatibility but users should move to the config sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => // Allow users to specify some environment variables. - YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator) - + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments. env("SPARK_YARN_USER_ENV") = userEnvs } - env - } - - def userArgsToString(clientArgs: ClientArguments): String = { - val prefix = " --arg " - val args = clientArgs.userArgs - val retval = new StringBuilder() - for (arg <- args) { - retval.append(prefix).append(" ").append(YarnSparkHadoopUtil.escapeForShell(arg)) - } - retval.toString - } - - def setupSecurityToken(amContainer: ContainerLaunchContext) - - def createContainerLaunchContext( - newApp: GetNewApplicationResponse, - localResources: HashMap[String, LocalResource], - env: HashMap[String, String]): ContainerLaunchContext = { - logInfo("Setting up container launch context") - val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) - amContainer.setLocalResources(localResources) - - val isLaunchingDriver = args.userClass != null // In cluster mode, if the deprecated SPARK_JAVA_OPTS is set, we need to propagate it to // executors. But we can't just set spark.executor.extraJavaOptions, because the driver's @@ -320,6 +261,7 @@ trait ClientBase extends Logging { // Note that to warn the user about the deprecation in cluster mode, some code from // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). + val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = @@ -342,14 +284,30 @@ trait ClientBase extends Logging { env("SPARK_JAVA_OPTS") = value } } - amContainer.setEnvironment(env) - val amMemory = args.amMemory + env + } + + /** + * Set up a ContainerLaunchContext to launch our ApplicationMaster container. + * This sets up the launch environment, java options, and the command for launching the AM. + */ + protected def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) + : ContainerLaunchContext = { + logInfo("Setting up container launch context for our AM") + + val appId = newAppResponse.getApplicationId + val appStagingDir = getAppStagingDir(appId) + val localResources = prepareLocalResources(appStagingDir) + val launchEnv = setupLaunchEnv(appStagingDir) + val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) + amContainer.setLocalResources(localResources) + amContainer.setEnvironment(launchEnv) val javaOpts = ListBuffer[String]() // Add Xmx for AM memory - javaOpts += "-Xmx" + amMemory + "m" + javaOpts += "-Xmx" + args.amMemory + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) javaOpts += "-Djava.io.tmpdir=" + tmpDir @@ -361,8 +319,7 @@ trait ClientBase extends Logging { // Instead of using this, rely on cpusets by YARN to enforce "proper" Spark behavior in // multi-tenant environments. Not sure how default Java GC behaves if it is limited to subset // of cores on a node. - val useConcurrentAndIncrementalGC = env.isDefinedAt("SPARK_USE_CONC_INCR_GC") && - java.lang.Boolean.parseBoolean(env("SPARK_USE_CONC_INCR_GC")) + val useConcurrentAndIncrementalGC = launchEnv.get("SPARK_USE_CONC_INCR_GC").exists(_.toBoolean) if (useConcurrentAndIncrementalGC) { // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines @@ -380,6 +337,8 @@ trait ClientBase extends Logging { javaOpts += YarnSparkHadoopUtil.escapeForShell(s"-D$k=$v") } + // Include driver-specific java options if we are launching a driver + val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) @@ -397,19 +356,27 @@ trait ClientBase extends Logging { } else { Nil } + val userJar = + if (args.userJar != null) { + Seq("--jar", args.userJar) + } else { + Nil + } val amClass = if (isLaunchingDriver) { - classOf[ApplicationMaster].getName() + Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName } else { - classOf[ApplicationMaster].getName().replace("ApplicationMaster", "ExecutorLauncher") + Class.forName("org.apache.spark.deploy.yarn.ExecutorLauncher").getName } + val userArgs = args.userArgs.flatMap { arg => + Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) + } val amArgs = - Seq(amClass) ++ userClass ++ - (if (args.userJar != null) Seq("--jar", args.userJar) else Nil) ++ - Seq("--executor-memory", args.executorMemory.toString, + Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ + Seq( + "--executor-memory", args.executorMemory.toString, "--executor-cores", args.executorCores.toString, - "--num-executors ", args.numExecutors.toString, - userArgsToString(args)) + "--num-executors ", args.numExecutors.toString) // Command for the ApplicationMaster val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ @@ -418,41 +385,153 @@ trait ClientBase extends Logging { "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - logInfo("Yarn AM launch context:") - logInfo(s" user class: ${args.userClass}") - logInfo(s" env: $env") - logInfo(s" command: ${commands.mkString(" ")}") - // TODO: it would be nicer to just make sure there are no null commands here val printableCommands = commands.map(s => if (s == null) "null" else s).toList amContainer.setCommands(printableCommands) - setupSecurityToken(amContainer) + logDebug("===============================================================================") + logDebug("Yarn AM launch context:") + logDebug(s" user class: ${Option(args.userClass).getOrElse("N/A")}") + logDebug(" env:") + launchEnv.foreach { case (k, v) => logDebug(s" $k -> $v") } + logDebug(" resources:") + localResources.foreach { case (k, v) => logDebug(s" $k -> $v")} + logDebug(" command:") + logDebug(s" ${printableCommands.mkString(" ")}") + logDebug("===============================================================================") // send the acl settings into YARN to control who has access via YARN interfaces val securityManager = new SecurityManager(sparkConf) amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager)) + setupSecurityToken(amContainer) + UserGroupInformation.getCurrentUser().addCredentials(credentials) amContainer } + + /** + * Report the state of an application until it has exited, either successfully or + * due to some failure, then return the application state. + * + * @param appId ID of the application to monitor. + * @param returnOnRunning Whether to also return the application state when it is RUNNING. + * @param logApplicationReport Whether to log details of the application report every iteration. + * @return state of the application, one of FINISHED, FAILED, KILLED, and RUNNING. + */ + def monitorApplication( + appId: ApplicationId, + returnOnRunning: Boolean = false, + logApplicationReport: Boolean = true): YarnApplicationState = { + val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) + var lastState: YarnApplicationState = null + while (true) { + Thread.sleep(interval) + val report = getApplicationReport(appId) + val state = report.getYarnApplicationState + + if (logApplicationReport) { + logInfo(s"Application report for $appId (state: $state)") + val details = Seq[(String, String)]( + ("client token", getClientToken(report)), + ("diagnostics", report.getDiagnostics), + ("ApplicationMaster host", report.getHost), + ("ApplicationMaster RPC port", report.getRpcPort.toString), + ("queue", report.getQueue), + ("start time", report.getStartTime.toString), + ("final status", report.getFinalApplicationStatus.toString), + ("tracking URL", report.getTrackingUrl), + ("user", report.getUser) + ) + + // Use more loggable format if value is null or empty + val formattedDetails = details + .map { case (k, v) => + val newValue = Option(v).filter(_.nonEmpty).getOrElse("N/A") + s"\n\t $k: $newValue" } + .mkString("") + + // If DEBUG is enabled, log report details every iteration + // Otherwise, log them every time the application changes state + if (log.isDebugEnabled) { + logDebug(formattedDetails) + } else if (lastState != state) { + logInfo(formattedDetails) + } + } + + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + return state + } + + if (returnOnRunning && state == YarnApplicationState.RUNNING) { + return state + } + + lastState = state + } + + // Never reached, but keeps compiler happy + throw new SparkException("While loop is depleted! This should never happen...") + } + + /** + * Submit an application to the ResourceManager and monitor its state. + * This continues until the application has exited for any reason. + */ + def run(): Unit = monitorApplication(submitApplication()) + + /* --------------------------------------------------------------------------------------- * + | Methods that cannot be implemented here due to API differences across hadoop versions | + * --------------------------------------------------------------------------------------- */ + + /** Submit an application running our ApplicationMaster to the ResourceManager. */ + def submitApplication(): ApplicationId + + /** Set up security tokens for launching our ApplicationMaster container. */ + protected def setupSecurityToken(containerContext: ContainerLaunchContext): Unit + + /** Get the application report from the ResourceManager for an application we have submitted. */ + protected def getApplicationReport(appId: ApplicationId): ApplicationReport + + /** + * Return the security token used by this client to communicate with the ApplicationMaster. + * If no security is enabled, the token returned by the report is null. + */ + protected def getClientToken(report: ApplicationReport): String } -object ClientBase extends Logging { +private[spark] object ClientBase extends Logging { + + // Alias for the Spark assembly jar and the user jar val SPARK_JAR: String = "__spark__.jar" val APP_JAR: String = "__app__.jar" + + // URI scheme that identifies local resources val LOCAL_SCHEME = "local" + + // Staging directory for any temporary jars or files + val SPARK_STAGING: String = ".sparkStaging" + + // Location of any user-defined Spark jars val CONF_SPARK_JAR = "spark.yarn.jar" - /** - * This is an internal config used to propagate the location of the user's jar file to the - * driver/executors. - */ + val ENV_SPARK_JAR = "SPARK_JAR" + + // Internal config to propagate the location of the user's jar to the driver/executors val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" - /** - * This is an internal config used to propagate the list of extra jars to add to the classpath - * of executors. - */ + + // Internal config to propagate the locations of any extra jars to add to the classpath + // of the executors val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" - val ENV_SPARK_JAR = "SPARK_JAR" + + // Staging directory is private! -> rwx-------- + val STAGING_DIR_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) + + // App files are world-wide readable and owner writable -> rw-r--r-- + val APP_FILE_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) /** * Find the user-defined Spark jar if configured, or return the jar containing this @@ -461,7 +540,7 @@ object ClientBase extends Logging { * This method first looks in the SparkConf object for the CONF_SPARK_JAR key, and in the * user environment if that is not found (for backwards compatibility). */ - def sparkJar(conf: SparkConf) = { + private def sparkJar(conf: SparkConf): String = { if (conf.contains(CONF_SPARK_JAR)) { conf.get(CONF_SPARK_JAR) } else if (System.getenv(ENV_SPARK_JAR) != null) { @@ -474,16 +553,22 @@ object ClientBase extends Logging { } } - def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) = { + /** + * Return the path to the given application's staging directory. + */ + private def getAppStagingDir(appId: ApplicationId): String = { + SPARK_STAGING + Path.SEPARATOR + appId.toString() + Path.SEPARATOR + } + + /** + * Populate the classpath entry in the given environment map with any application + * classpath specified through the Hadoop and Yarn configurations. + */ + def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]): Unit = { val classPathElementsToAdd = getYarnAppClasspath(conf) ++ getMRAppClasspath(conf) for (c <- classPathElementsToAdd.flatten) { - YarnSparkHadoopUtil.addToEnvironment( - env, - Environment.CLASSPATH.name, - c.trim, - File.pathSeparator) + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, c.trim) } - classPathElementsToAdd } private def getYarnAppClasspath(conf: Configuration): Option[Seq[String]] = @@ -519,7 +604,7 @@ object ClientBase extends Logging { /** * In Hadoop 0.23, the MR application classpath comes with the YARN application - * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. + * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. * So we need to use reflection to retrieve it. */ def getDefaultMRApplicationClasspath: Option[Seq[String]] = { @@ -545,8 +630,16 @@ object ClientBase extends Logging { triedDefault.toOption } - def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf, - env: HashMap[String, String], extraClassPath: Option[String] = None) { + /** + * Populate the classpath entry in the given environment map. + * This includes the user jar, Spark jar, and any extra application jars. + */ + def populateClasspath( + args: ClientArguments, + conf: Configuration, + sparkConf: SparkConf, + env: HashMap[String, String], + extraClassPath: Option[String] = None): Unit = { extraClassPath.foreach(addClasspathEntry(_, env)) addClasspathEntry(Environment.PWD.$(), env) @@ -554,36 +647,40 @@ object ClientBase extends Logging { if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { addUserClasspath(args, sparkConf, env) addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - ClientBase.populateHadoopClasspath(conf, env) + populateHadoopClasspath(conf, env) } else { addFileToClasspath(sparkJar(sparkConf), SPARK_JAR, env) - ClientBase.populateHadoopClasspath(conf, env) + populateHadoopClasspath(conf, env) addUserClasspath(args, sparkConf, env) } // Append all jar files under the working directory to the classpath. - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env); + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) } /** * Adds the user jars which have local: URIs (or alternate names, such as APP_JAR) explicitly * to the classpath. */ - private def addUserClasspath(args: ClientArguments, conf: SparkConf, - env: HashMap[String, String]) = { - if (args != null) { - addFileToClasspath(args.userJar, APP_JAR, env) - if (args.addJars != null) { - args.addJars.split(",").foreach { case file: String => - addFileToClasspath(file, null, env) - } + private def addUserClasspath( + args: ClientArguments, + conf: SparkConf, + env: HashMap[String, String]): Unit = { + + // If `args` is not null, we are launching an AM container. + // Otherwise, we are launching executor containers. + val (mainJar, secondaryJars) = + if (args != null) { + (args.userJar, args.addJars) + } else { + (conf.get(CONF_SPARK_USER_JAR, null), conf.get(CONF_SPARK_YARN_SECONDARY_JARS, null)) } - } else { - val userJar = conf.get(CONF_SPARK_USER_JAR, null) - addFileToClasspath(userJar, APP_JAR, env) - val cachedSecondaryJarLinks = conf.get(CONF_SPARK_YARN_SECONDARY_JARS, "").split(",") - cachedSecondaryJarLinks.foreach(jar => addFileToClasspath(jar, null, env)) + addFileToClasspath(mainJar, APP_JAR, env) + if (secondaryJars != null) { + secondaryJars.split(",").filter(_.nonEmpty).foreach { jar => + addFileToClasspath(jar, null, env) + } } } @@ -599,46 +696,44 @@ object ClientBase extends Logging { * @param fileName Alternate name for the file (optional). * @param env Map holding the environment variables. */ - private def addFileToClasspath(path: String, fileName: String, - env: HashMap[String, String]) : Unit = { + private def addFileToClasspath( + path: String, + fileName: String, + env: HashMap[String, String]): Unit = { if (path != null) { scala.util.control.Exception.ignoring(classOf[URISyntaxException]) { - val localPath = getLocalPath(path) - if (localPath != null) { - addClasspathEntry(localPath, env) + val uri = new URI(path) + if (uri.getScheme == LOCAL_SCHEME) { + addClasspathEntry(uri.getPath, env) return } } } if (fileName != null) { - addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env); + addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + fileName, env) } } /** - * Returns the local path if the URI is a "local:" URI, or null otherwise. + * Add the given path to the classpath entry of the given environment map. + * If the classpath is already set, this appends the new path to the existing classpath. */ - private def getLocalPath(resource: String): String = { - val uri = new URI(resource) - if (LOCAL_SCHEME.equals(uri.getScheme())) { - return uri.getPath() - } - null - } - - private def addClasspathEntry(path: String, env: HashMap[String, String]) = - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, path, - File.pathSeparator) + private def addClasspathEntry(path: String, env: HashMap[String, String]): Unit = + YarnSparkHadoopUtil.addPathToEnvironment(env, Environment.CLASSPATH.name, path) /** * Get the list of namenodes the user may access. */ - private[yarn] def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { - sparkConf.get("spark.yarn.access.namenodes", "").split(",").map(_.trim()).filter(!_.isEmpty) - .map(new Path(_)).toSet + def getNameNodesToAccess(sparkConf: SparkConf): Set[Path] = { + sparkConf.get("spark.yarn.access.namenodes", "") + .split(",") + .map(_.trim()) + .filter(!_.isEmpty) + .map(new Path(_)) + .toSet } - private[yarn] def getTokenRenewer(conf: Configuration): String = { + def getTokenRenewer(conf: Configuration): String = { val delegTokenRenewer = Master.getMasterPrincipal(conf) logDebug("delegation token renewer is: " + delegTokenRenewer) if (delegTokenRenewer == null || delegTokenRenewer.length() == 0) { @@ -652,17 +747,54 @@ object ClientBase extends Logging { /** * Obtains tokens for the namenodes passed in and adds them to the credentials. */ - private[yarn] def obtainTokensForNamenodes(paths: Set[Path], conf: Configuration, - creds: Credentials) { + def obtainTokensForNamenodes( + paths: Set[Path], + conf: Configuration, + creds: Credentials): Unit = { if (UserGroupInformation.isSecurityEnabled()) { val delegTokenRenewer = getTokenRenewer(conf) + paths.foreach { dst => + val dstFs = dst.getFileSystem(conf) + logDebug("getting token for namenode: " + dst) + dstFs.addDelegationTokens(delegTokenRenewer, creds) + } + } + } - paths.foreach { - dst => - val dstFs = dst.getFileSystem(conf) - logDebug("getting token for namenode: " + dst) - dstFs.addDelegationTokens(delegTokenRenewer, creds) + /** + * Return whether the two file systems are the same. + */ + private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { + val srcUri = srcFs.getUri() + val dstUri = destFs.getUri() + if (srcUri.getScheme() == null) { + return false + } + if (!srcUri.getScheme().equals(dstUri.getScheme())) { + return false + } + var srcHost = srcUri.getHost() + var dstHost = dstUri.getHost() + if ((srcHost != null) && (dstHost != null)) { + try { + srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() + dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() + } catch { + case e: UnknownHostException => + return false } + if (!srcHost.equals(dstHost)) { + return false + } + } else if (srcHost == null && dstHost != null) { + return false + } else if (srcHost != null && dstHost == null) { + return false + } + if (srcUri.getPort() != dstUri.getPort()) { + false + } else { + true } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 9b7f1fca96c6d..c592ecfdfce06 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -19,29 +19,24 @@ package org.apache.spark.deploy.yarn import java.net.URI +import scala.collection.mutable.{HashMap, LinkedHashMap, Map} + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileStatus -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.permission.FsAction -import org.apache.hadoop.yarn.api.records.LocalResource -import org.apache.hadoop.yarn.api.records.LocalResourceVisibility -import org.apache.hadoop.yarn.api.records.LocalResourceType +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.util.{Records, ConverterUtils} -import org.apache.spark.Logging - -import scala.collection.mutable.HashMap -import scala.collection.mutable.LinkedHashMap -import scala.collection.mutable.Map - +import org.apache.spark.Logging /** Client side methods to setup the Hadoop distributed cache */ -class ClientDistributedCacheManager() extends Logging { - private val distCacheFiles: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() - private val distCacheArchives: Map[String, Tuple3[String, String, String]] = - LinkedHashMap[String, Tuple3[String, String, String]]() +private[spark] class ClientDistributedCacheManager() extends Logging { + + // Mappings from remote URI to (file status, modification time, visibility) + private val distCacheFiles: Map[String, (String, String, String)] = + LinkedHashMap[String, (String, String, String)]() + private val distCacheArchives: Map[String, (String, String, String)] = + LinkedHashMap[String, (String, String, String)]() /** @@ -68,9 +63,9 @@ class ClientDistributedCacheManager() extends Logging { resourceType: LocalResourceType, link: String, statCache: Map[URI, FileStatus], - appMasterOnly: Boolean = false) = { + appMasterOnly: Boolean = false): Unit = { val destStatus = fs.getFileStatus(destPath) - val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + val amJarRsrc = Records.newRecord(classOf[LocalResource]) amJarRsrc.setType(resourceType) val visibility = getVisibility(conf, destPath.toUri(), statCache) amJarRsrc.setVisibility(visibility) @@ -80,7 +75,7 @@ class ClientDistributedCacheManager() extends Logging { if (link == null || link.isEmpty()) throw new Exception("You must specify a valid link name") localResources(link) = amJarRsrc - if (appMasterOnly == false) { + if (!appMasterOnly) { val uri = destPath.toUri() val pathURI = new URI(uri.getScheme(), uri.getAuthority(), uri.getPath(), null, link) if (resourceType == LocalResourceType.FILE) { @@ -95,12 +90,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Adds the necessary cache file env variables to the env passed in - * @param env */ - def setDistFilesEnv(env: Map[String, String]) = { + def setDistFilesEnv(env: Map[String, String]): Unit = { val (keys, tupleValues) = distCacheFiles.unzip val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { env("SPARK_YARN_CACHE_FILES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } env("SPARK_YARN_CACHE_FILES_TIME_STAMPS") = @@ -114,12 +107,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Adds the necessary cache archive env variables to the env passed in - * @param env */ - def setDistArchivesEnv(env: Map[String, String]) = { + def setDistArchivesEnv(env: Map[String, String]): Unit = { val (keys, tupleValues) = distCacheArchives.unzip val (sizes, timeStamps, visibilities) = tupleValues.unzip3 - if (keys.size > 0) { env("SPARK_YARN_CACHE_ARCHIVES") = keys.reduceLeft[String] { (acc,n) => acc + "," + n } env("SPARK_YARN_CACHE_ARCHIVES_TIME_STAMPS") = @@ -133,25 +124,21 @@ class ClientDistributedCacheManager() extends Logging { /** * Returns the local resource visibility depending on the cache file permissions - * @param conf - * @param uri - * @param statCache * @return LocalResourceVisibility */ - def getVisibility(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): - LocalResourceVisibility = { + def getVisibility( + conf: Configuration, + uri: URI, + statCache: Map[URI, FileStatus]): LocalResourceVisibility = { if (isPublic(conf, uri, statCache)) { - return LocalResourceVisibility.PUBLIC - } - LocalResourceVisibility.PRIVATE + LocalResourceVisibility.PUBLIC + } else { + LocalResourceVisibility.PRIVATE + } } /** - * Returns a boolean to denote whether a cache file is visible to all(public) - * or not - * @param conf - * @param uri - * @param statCache + * Returns a boolean to denote whether a cache file is visible to all (public) * @return true if the path in the uri is visible to all, false otherwise */ def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { @@ -167,13 +154,12 @@ class ClientDistributedCacheManager() extends Logging { /** * Returns true if all ancestors of the specified path have the 'execute' * permission set for all users (i.e. that other users can traverse - * the directory heirarchy to the given path) - * @param fs - * @param path - * @param statCache + * the directory hierarchy to the given path) * @return true if all ancestors have the 'execute' permission set for all users */ - def ancestorsHaveExecutePermissions(fs: FileSystem, path: Path, + def ancestorsHaveExecutePermissions( + fs: FileSystem, + path: Path, statCache: Map[URI, FileStatus]): Boolean = { var current = path while (current != null) { @@ -187,32 +173,25 @@ class ClientDistributedCacheManager() extends Logging { } /** - * Checks for a given path whether the Other permissions on it + * Checks for a given path whether the Other permissions on it * imply the permission in the passed FsAction - * @param fs - * @param path - * @param action - * @param statCache * @return true if the path in the uri is visible to all, false otherwise */ - def checkPermissionOfOther(fs: FileSystem, path: Path, - action: FsAction, statCache: Map[URI, FileStatus]): Boolean = { + def checkPermissionOfOther( + fs: FileSystem, + path: Path, + action: FsAction, + statCache: Map[URI, FileStatus]): Boolean = { val status = getFileStatus(fs, path.toUri(), statCache) val perms = status.getPermission() val otherAction = perms.getOtherAction() - if (otherAction.implies(action)) { - return true - } - false + otherAction.implies(action) } /** - * Checks to see if the given uri exists in the cache, if it does it + * Checks to see if the given uri exists in the cache, if it does it * returns the existing FileStatus, otherwise it stats the uri, stores * it in the cache, and returns the FileStatus. - * @param fs - * @param uri - * @param statCache * @return FileStatus */ def getFileStatus(fs: FileSystem, uri: URI, statCache: Map[URI, FileStatus]): FileStatus = { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index f56f72cafe50e..bbbf615510762 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -17,7 +17,6 @@ package org.apache.spark.deploy.yarn -import java.io.File import java.net.URI import scala.collection.JavaConversions._ @@ -128,9 +127,9 @@ trait ExecutorRunnableUtil extends Logging { localResources: HashMap[String, LocalResource], timestamp: String, size: String, - vis: String) = { + vis: String): Unit = { val uri = new URI(file) - val amJarRsrc = Records.newRecord(classOf[LocalResource]).asInstanceOf[LocalResource] + val amJarRsrc = Records.newRecord(classOf[LocalResource]) amJarRsrc.setType(rtype) amJarRsrc.setVisibility(LocalResourceVisibility.valueOf(vis)) amJarRsrc.setResource(ConverterUtils.getYarnUrlFromURI(uri)) @@ -175,14 +174,17 @@ trait ExecutorRunnableUtil extends Logging { ClientBase.populateClasspath(null, yarnConf, sparkConf, env, extraCp) sparkConf.getExecutorEnv.foreach { case (key, value) => - YarnSparkHadoopUtil.addToEnvironment(env, key, value, File.pathSeparator) + // This assumes each executor environment variable set here is a path + // This is kept for backward compatibility and consistency with hadoop + YarnSparkHadoopUtil.addPathToEnvironment(env, key, value) } // Keep this for backwards compatibility but users should move to the config - YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), - File.pathSeparator) + sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs => + YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) + } - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } + System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k, v) => env(k) = v } env } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 4a33e34c3bfc7..0b712c201904a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.lang.{Boolean => JBoolean} +import java.io.File import java.util.{Collections, Set => JSet} import java.util.regex.Matcher import java.util.regex.Pattern @@ -29,14 +30,12 @@ import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.util.StringInterner import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.conf.Configuration -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -100,30 +99,26 @@ object YarnSparkHadoopUtil { private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() - def addToEnvironment( - env: HashMap[String, String], - variable: String, - value: String, - classPathSeparator: String) = { - var envVariable = "" - if (env.get(variable) == None) { - envVariable = value - } else { - envVariable = env.get(variable).get + classPathSeparator + value - } - env put (StringInterner.weakIntern(variable), StringInterner.weakIntern(envVariable)) + /** + * Add a path variable to the given environment map. + * If the map already contains this key, append the value to the existing value instead. + */ + def addPathToEnvironment(env: HashMap[String, String], key: String, value: String): Unit = { + val newValue = if (env.contains(key)) { env(key) + File.pathSeparator + value } else value + env.put(key, newValue) } - def setEnvFromInputString( - env: HashMap[String, String], - envString: String, - classPathSeparator: String) = { - if (envString != null && envString.length() > 0) { - var childEnvs = envString.split(",") - var p = Pattern.compile(getEnvironmentVariableRegex()) + /** + * Set zero or more environment variables specified by the given input string. + * The input string is expected to take the form "KEY1=VAL1,KEY2=VAL2,KEY3=VAL3". + */ + def setEnvFromInputString(env: HashMap[String, String], inputString: String): Unit = { + if (inputString != null && inputString.length() > 0) { + val childEnvs = inputString.split(",") + val p = Pattern.compile(environmentVariableRegex) for (cEnv <- childEnvs) { - var parts = cEnv.split("=") // split on '=' - var m = p.matcher(parts(1)) + val parts = cEnv.split("=") // split on '=' + val m = p.matcher(parts(1)) val sb = new StringBuffer while (m.find()) { val variable = m.group(1) @@ -131,8 +126,7 @@ object YarnSparkHadoopUtil { if (env.get(variable) != None) { replace = env.get(variable).get } else { - // if this key is not configured for the child .. get it - // from the env + // if this key is not configured for the child .. get it from the env replace = System.getenv(variable) if (replace == null) { // the env key is note present anywhere .. simply set it @@ -142,14 +136,15 @@ object YarnSparkHadoopUtil { m.appendReplacement(sb, Matcher.quoteReplacement(replace)) } m.appendTail(sb) - addToEnvironment(env, parts(0), sb.toString(), classPathSeparator) + // This treats the environment variable as path variable delimited by `File.pathSeparator` + // This is kept for backward compatibility and consistency with Hadoop's behavior + addPathToEnvironment(env, parts(0), sb.toString) } } } - private def getEnvironmentVariableRegex() : String = { - val osName = System.getProperty("os.name") - if (osName startsWith "Windows") { + private val environmentVariableRegex: String = { + if (Utils.isWindows) { "%([A-Za-z_][A-Za-z0-9_]*?)%" } else { "\\$([A-Za-z_][A-Za-z0-9_]*)" @@ -181,14 +176,14 @@ object YarnSparkHadoopUtil { } } - private[spark] def lookupRack(conf: Configuration, host: String): String = { + def lookupRack(conf: Configuration, host: String): String = { if (!hostToRack.contains(host)) { populateRackInfo(conf, host) } hostToRack.get(host) } - private[spark] def populateRackInfo(conf: Configuration, hostname: String) { + def populateRackInfo(conf: Configuration, hostname: String) { Utils.checkHost(hostname) if (!hostToRack.containsKey(hostname)) { @@ -212,8 +207,8 @@ object YarnSparkHadoopUtil { } } - private[spark] def getApplicationAclsForYarn(securityMgr: SecurityManager): - Map[ApplicationAccessType, String] = { + def getApplicationAclsForYarn(securityMgr: SecurityManager) + : Map[ApplicationAccessType, String] = { Map[ApplicationAccessType, String] ( ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls, ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls 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 6aa6475fe4a18..200a30899290b 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 @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnSparkHadoopUtil} +import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -34,115 +34,120 @@ private[spark] class YarnClientSchedulerBackend( minRegisteredRatio = 0.8 } - var client: Client = null - var appId: ApplicationId = null - var checkerThread: Thread = null - var stopping: Boolean = false - var totalExpectedExecutors = 0 - - private[spark] def addArg(optionName: String, envVar: String, sysProp: String, - arrayBuf: ArrayBuffer[String]) { - if (System.getenv(envVar) != null) { - arrayBuf += (optionName, System.getenv(envVar)) - } else if (sc.getConf.contains(sysProp)) { - arrayBuf += (optionName, sc.getConf.get(sysProp)) - } - } + private var client: Client = null + private var appId: ApplicationId = null + private var stopping: Boolean = false + private var totalExpectedExecutors = 0 + /** + * Create a Yarn client to submit an application to the ResourceManager. + * This waits until the application is running. + */ override def start() { super.start() - val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIHostPort) } val argsArrayBuf = new ArrayBuffer[String]() - argsArrayBuf += ( - "--args", hostport - ) - - // process any optional arguments, given either as environment variables - // or system properties. use the defaults already defined in ClientArguments - // if things aren't specified. system properties override environment - // variables. - List(("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), - ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), - ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.executor.instances"), - ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), - ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), - ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), - ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), - ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), - ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), - ("--name", "SPARK_YARN_APP_NAME", "spark.app.name")) - .foreach { case (optName, envVar, sysProp) => addArg(optName, envVar, sysProp, argsArrayBuf) } - - logDebug("ClientArguments called with: " + argsArrayBuf) + argsArrayBuf += ("--arg", hostport) + argsArrayBuf ++= getExtraClientArguments + + logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" ")) val args = new ClientArguments(argsArrayBuf.toArray, conf) totalExpectedExecutors = args.numExecutors client = new Client(args, conf) - appId = client.runApp() - waitForApp() - checkerThread = yarnApplicationStateCheckerThread() + appId = client.submitApplication() + waitForApplication() + asyncMonitorApplication() } - def waitForApp() { - - // TODO : need a better way to find out whether the executors are ready or not - // maybe by resource usage report? - while(true) { - val report = client.getApplicationReport(appId) - - logInfo("Application report from ASM: \n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + /** + * Return any extra command line arguments to be passed to Client provided in the form of + * environment variables or Spark properties. + */ + private def getExtraClientArguments: Seq[String] = { + val extraArgs = new ArrayBuffer[String] + val optionTuples = // List of (target Client argument, environment variable, Spark property) + List( + ("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), + ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), + ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.executor.instances"), + ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), + ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), + ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), + ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), + ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), + ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), + ("--name", "SPARK_YARN_APP_NAME", "spark.app.name") ) - - // Ready to go, or already gone. - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.RUNNING) { - return - } else if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - throw new SparkException("Yarn application already ended," + - "might be killed or not able to launch application master.") + optionTuples.foreach { case (optionName, envVar, sparkProp) => + if (System.getenv(envVar) != null) { + extraArgs += (optionName, System.getenv(envVar)) + } else if (sc.getConf.contains(sparkProp)) { + extraArgs += (optionName, sc.getConf.get(sparkProp)) } + } + extraArgs + } - Thread.sleep(1000) + /** + * Report the state of the application until it is running. + * If the application has finished, failed or been killed in the process, throw an exception. + * This assumes both `client` and `appId` have already been set. + */ + private def waitForApplication(): Unit = { + assert(client != null && appId != null, "Application has not been submitted yet!") + val state = client.monitorApplication(appId, returnOnRunning = true) // blocking + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.FAILED || + state == YarnApplicationState.KILLED) { + throw new SparkException("Yarn application has already ended! " + + "It might have been killed or unable to launch application master.") + } + if (state == YarnApplicationState.RUNNING) { + logInfo(s"Application $appId has started running.") } } - private def yarnApplicationStateCheckerThread(): Thread = { + /** + * Monitor the application state in a separate thread. + * If the application has exited for any reason, stop the SparkContext. + * This assumes both `client` and `appId` have already been set. + */ + private def asyncMonitorApplication(): Unit = { + assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { while (!stopping) { val report = client.getApplicationReport(appId) val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.KILLED - || state == YarnApplicationState.FAILED) { - logError(s"Yarn application already ended: $state") + if (state == YarnApplicationState.FINISHED || + state == YarnApplicationState.KILLED || + state == YarnApplicationState.FAILED) { + logError(s"Yarn application has already exited with state $state!") sc.stop() stopping = true } Thread.sleep(1000L) } - checkerThread = null Thread.currentThread().interrupt() } } - t.setName("Yarn Application State Checker") + t.setName("Yarn application state monitor") t.setDaemon(true) t.start() - t } + /** + * Stop the scheduler. This assumes `start()` has already been called. + */ override def stop() { + assert(client != null, "Attempted to stop this scheduler before starting it!") stopping = true super.stop() - client.stop + client.stop() logInfo("Stopped") } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index c3b7a2c8f02e5..9bd916100dd2c 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse -import org.apache.hadoop.yarn.api.records.ContainerLaunchContext +import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.mockito.Matchers._ import org.mockito.Mockito._ @@ -90,7 +90,7 @@ class ClientBaseSuite extends FunSuite with Matchers { val env = new MutableHashMap[String, String]() val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - ClientBase.populateClasspath(args, conf, sparkConf, env, None) + ClientBase.populateClasspath(args, conf, sparkConf, env) val cp = env("CLASSPATH").split(File.pathSeparator) s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => @@ -114,10 +114,10 @@ class ClientBaseSuite extends FunSuite with Matchers { val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) - doReturn(new Path("/")).when(client).copyRemoteFile(any(classOf[Path]), + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), any(classOf[Path]), anyShort(), anyBoolean()) - var tempDir = Files.createTempDir(); + val tempDir = Files.createTempDir() try { client.prepareLocalResources(tempDir.getAbsolutePath()) sparkConf.getOption(ClientBase.CONF_SPARK_USER_JAR) should be (Some(USER)) @@ -247,13 +247,13 @@ class ClientBaseSuite extends FunSuite with Matchers { private class DummyClient( val args: ClientArguments, - val conf: Configuration, + val hadoopConf: Configuration, val sparkConf: SparkConf, val yarnConf: YarnConfiguration) extends ClientBase { - - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = - throw new UnsupportedOperationException() - + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = ??? + override def submitApplication(): ApplicationId = ??? + override def getApplicationReport(appId: ApplicationId): ApplicationReport = ??? + override def getClientToken(report: ApplicationReport): String = ??? } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 82e45e3e7ad54..0b43e6ee20538 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -21,11 +21,9 @@ import java.nio.ByteBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.api.YarnClient +import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SparkConf} @@ -34,128 +32,98 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. */ -class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) +private[spark] class Client( + val args: ClientArguments, + val hadoopConf: Configuration, + val sparkConf: SparkConf) extends ClientBase with Logging { - val yarnClient = YarnClient.createYarnClient - def this(clientArgs: ClientArguments, spConf: SparkConf) = this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - val args = clientArgs - val conf = hadoopConf - val sparkConf = spConf - var rpc: YarnRPC = YarnRPC.create(conf) - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - - def runApp(): ApplicationId = { - validateArgs() - // Initialize and start the client service. + val yarnClient = YarnClient.createYarnClient + val yarnConf = new YarnConfiguration(hadoopConf) + + def stop(): Unit = yarnClient.stop() + + /* ------------------------------------------------------------------------------------- * + | The following methods have much in common in the stable and alpha versions of Client, | + | but cannot be implemented in the parent trait due to subtle API differences across | + | hadoop versions. | + * ------------------------------------------------------------------------------------- */ + + /** + * Submit an application running our ApplicationMaster to the ResourceManager. + * + * The stable Yarn API provides a convenience method (YarnClient#createApplication) for + * creating applications and setting up the application submission context. This was not + * available in the alpha API. + */ + override def submitApplication(): ApplicationId = { yarnClient.init(yarnConf) yarnClient.start() - // Log details about this YARN cluster (e.g, the number of slave machines/NodeManagers). - logClusterResourceDetails() - - // Prepare to submit a request to the ResourcManager (specifically its ApplicationsManager (ASM) - // interface). + logInfo("Requesting a new application from cluster with %d NodeManagers" + .format(yarnClient.getYarnClusterMetrics.getNumNodeManagers)) - // Get a new client application. + // Get a new application from our RM val newApp = yarnClient.createApplication() val newAppResponse = newApp.getNewApplicationResponse() val appId = newAppResponse.getApplicationId() + // Verify whether the cluster has enough resources for our AM verifyClusterResources(newAppResponse) - // Set up resource and environment variables. - val appStagingDir = getAppStagingDir(appId) - val localResources = prepareLocalResources(appStagingDir) - val launchEnv = setupLaunchEnv(localResources, appStagingDir) - val amContainer = createContainerLaunchContext(newAppResponse, localResources, launchEnv) + // Set up the appropriate contexts to launch our AM + val containerContext = createContainerLaunchContext(newAppResponse) + val appContext = createApplicationSubmissionContext(newApp, containerContext) - // Set up an application submission context. - val appContext = newApp.getApplicationSubmissionContext() - appContext.setApplicationName(args.appName) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(amContainer) - appContext.setApplicationType("SPARK") - - // Memory for the ApplicationMaster. - val memoryResource = Records.newRecord(classOf[Resource]).asInstanceOf[Resource] - memoryResource.setMemory(args.amMemory + memoryOverhead) - appContext.setResource(memoryResource) - - // Finally, submit and monitor the application. - submitApp(appContext) + // Finally, submit and monitor the application + logInfo(s"Submitting application ${appId.getId} to ResourceManager") + yarnClient.submitApplication(appContext) appId } - def run() { - val appId = runApp() - monitorApplication(appId) - } - - def logClusterResourceDetails() { - val clusterMetrics: YarnClusterMetrics = yarnClient.getYarnClusterMetrics - logInfo("Got cluster metric info from ResourceManager, number of NodeManagers: " + - clusterMetrics.getNumNodeManagers) + /** + * Set up the context for submitting our ApplicationMaster. + * This uses the YarnClientApplication not available in the Yarn alpha API. + */ + def createApplicationSubmissionContext( + newApp: YarnClientApplication, + containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { + val appContext = newApp.getApplicationSubmissionContext + appContext.setApplicationName(args.appName) + appContext.setQueue(args.amQueue) + appContext.setAMContainerSpec(containerContext) + appContext.setApplicationType("SPARK") + val capability = Records.newRecord(classOf[Resource]) + capability.setMemory(args.amMemory + amMemoryOverhead) + appContext.setResource(capability) + appContext } - def setupSecurityToken(amContainer: ContainerLaunchContext) = { - // Setup security tokens. - val dob = new DataOutputBuffer() + /** Set up security tokens for launching our ApplicationMaster container. */ + override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { + val dob = new DataOutputBuffer credentials.writeTokenStorageToStream(dob) - amContainer.setTokens(ByteBuffer.wrap(dob.getData())) + amContainer.setTokens(ByteBuffer.wrap(dob.getData)) } - def submitApp(appContext: ApplicationSubmissionContext) = { - // Submit the application to the applications manager. - logInfo("Submitting application to ResourceManager") - yarnClient.submitApplication(appContext) - } + /** Get the application report from the ResourceManager for an application we have submitted. */ + override def getApplicationReport(appId: ApplicationId): ApplicationReport = + yarnClient.getApplicationReport(appId) - def getApplicationReport(appId: ApplicationId) = - yarnClient.getApplicationReport(appId) - - def stop = yarnClient.stop - - def monitorApplication(appId: ApplicationId): Boolean = { - val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) - - while (true) { - Thread.sleep(interval) - val report = yarnClient.getApplicationReport(appId) - - logInfo("Application report from ResourceManager: \n" + - "\t application identifier: " + appId.toString() + "\n" + - "\t appId: " + appId.getId() + "\n" + - "\t clientToAMToken: " + report.getClientToAMToken() + "\n" + - "\t appDiagnostics: " + report.getDiagnostics() + "\n" + - "\t appMasterHost: " + report.getHost() + "\n" + - "\t appQueue: " + report.getQueue() + "\n" + - "\t appMasterRpcPort: " + report.getRpcPort() + "\n" + - "\t appStartTime: " + report.getStartTime() + "\n" + - "\t yarnAppState: " + report.getYarnApplicationState() + "\n" + - "\t distributedFinalState: " + report.getFinalApplicationStatus() + "\n" + - "\t appTrackingUrl: " + report.getTrackingUrl() + "\n" + - "\t appUser: " + report.getUser() - ) - - val state = report.getYarnApplicationState() - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.FAILED || - state == YarnApplicationState.KILLED) { - return true - } - } - true - } + /** + * Return the security token used by this client to communicate with the ApplicationMaster. + * If no security is enabled, the token returned by the report is null. + */ + override def getClientToken(report: ApplicationReport): String = + Option(report.getClientToAMToken).map(_.toString).getOrElse("") } object Client { - def main(argStrings: Array[String]) { if (!sys.props.contains("SPARK_SUBMIT")) { println("WARNING: This client is deprecated and will be removed in a " + @@ -163,22 +131,19 @@ object Client { } // Set an env variable indicating we are running in YARN mode. - // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - - // see Client#setupLaunchEnv(). + // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf() + val sparkConf = new SparkConf try { val args = new ClientArguments(argStrings, sparkConf) new Client(args, sparkConf).run() } catch { - case e: Exception => { + case e: Exception => Console.err.println(e.getMessage) System.exit(1) - } } System.exit(0) } - } From 11c10df825419372df61a8d23c51e8c3cc78047f Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 23 Sep 2014 11:40:14 -0500 Subject: [PATCH 0767/1492] [SPARK-3304] [YARN] ApplicationMaster's Finish status is wrong when uncaught exception is thrown from ReporterThread Author: Kousuke Saruta Closes #2198 from sarutak/SPARK-3304 and squashes the following commits: 2696237 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 5b80363 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 4eb0a3e [Kousuke Saruta] Remoed the description about spark.yarn.scheduler.reporterThread.maxFailure 9741597 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 f7538d4 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 358ef8d [Kousuke Saruta] Merge branch 'SPARK-3304' of github.com:sarutak/spark into SPARK-3304 0d138c6 [Kousuke Saruta] Revert "tmp" f8da10a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 b6e9879 [Kousuke Saruta] tmp 8d256ed [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 13b2652 [Kousuke Saruta] Merge branch 'SPARK-3304' of github.com:sarutak/spark into SPARK-3304 2711e15 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 c081f8e [Kousuke Saruta] Modified ApplicationMaster to handle exception in ReporterThread itself 0bbd3a6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3304 a6982ad [Kousuke Saruta] Added ability handling uncaught exception thrown from Reporter thread --- .../spark/deploy/yarn/ApplicationMaster.scala | 66 +++++++++++++++---- 1 file changed, 54 insertions(+), 12 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index cde5fff637a39..9050808157257 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -17,7 +17,10 @@ package org.apache.spark.deploy.yarn +import scala.util.control.NonFatal + import java.io.IOException +import java.lang.reflect.InvocationTargetException import java.net.Socket import java.util.concurrent.atomic.AtomicReference @@ -55,6 +58,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, @volatile private var finished = false @volatile private var finalStatus = FinalApplicationStatus.UNDEFINED + @volatile private var userClassThread: Thread = _ private var reporterThread: Thread = _ private var allocator: YarnAllocator = _ @@ -221,18 +225,48 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) + // The number of failures in a row until Reporter thread give up + val reporterMaxFailures = sparkConf.getInt("spark.yarn.scheduler.reporterThread.maxFailures", 5) + val t = new Thread { override def run() { + var failureCount = 0 + while (!finished) { - checkNumExecutorsFailed() - if (!finished) { - logDebug("Sending progress") - allocator.allocateResources() - try { - Thread.sleep(interval) - } catch { - case e: InterruptedException => + try { + checkNumExecutorsFailed() + if (!finished) { + logDebug("Sending progress") + allocator.allocateResources() } + failureCount = 0 + } catch { + case e: Throwable => { + failureCount += 1 + if (!NonFatal(e) || failureCount >= reporterMaxFailures) { + logError("Exception was thrown from Reporter thread.", e) + finish(FinalApplicationStatus.FAILED, "Exception was thrown" + + s"${failureCount} time(s) from Reporter thread.") + + /** + * If exception is thrown from ReporterThread, + * interrupt user class to stop. + * Without this interrupting, if exception is + * thrown before allocating enough executors, + * YarnClusterScheduler waits until timeout even though + * we cannot allocate executors. + */ + logInfo("Interrupting user class to stop.") + userClassThread.interrupt + } else { + logWarning(s"Reporter thread fails ${failureCount} time(s) in a row.", e) + } + } + } + try { + Thread.sleep(interval) + } catch { + case e: InterruptedException => } } } @@ -355,7 +389,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val mainMethod = Class.forName(args.userClass, false, Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) - val t = new Thread { + userClassThread = new Thread { override def run() { var status = FinalApplicationStatus.FAILED try { @@ -366,15 +400,23 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Some apps have "System.exit(0)" at the end. The user thread will stop here unless // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. status = FinalApplicationStatus.SUCCEEDED + } catch { + case e: InvocationTargetException => { + e.getCause match { + case _: InterruptedException => { + // Reporter thread can interrupt to stop user class + } + } + } } finally { logDebug("Finishing main") } finalStatus = status } } - t.setName("Driver") - t.start() - t + userClassThread.setName("Driver") + userClassThread.start() + userClassThread } // Actor used to monitor the driver when running in client deploy mode. From 66bc0f2d675d06cdd48638f124a1ff32be2bf456 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 23 Sep 2014 11:45:44 -0700 Subject: [PATCH 0768/1492] [SPARK-3598][SQL]cast to timestamp should be the same as hive this patch fixes timestamp smaller than 0 and cast int as timestamp select cast(1000 as timestamp) from src limit 1; should return 1970-01-01 00:00:01, but we now take it as 1000 seconds. also, current implementation has bug when the time is before 1970-01-01 00:00:00. rxin marmbrus chenghao-intel Author: Daoyuan Wang Closes #2458 from adrian-wang/timestamp and squashes the following commits: 4274b1d [Daoyuan Wang] set test not related to timezone 1234f66 [Daoyuan Wang] fix timestamp smaller than 0 and cast int as timestamp --- .../spark/sql/catalyst/expressions/Cast.scala | 17 +++++++------ .../ExpressionEvaluationSuite.scala | 16 ++++++++----- ...cast #1-0-69fc614ccea92bbe39f4decc299edcc6 | 1 + ...cast #2-0-732ed232ac592c5e7f7c913a88874fd2 | 1 + ... cast #3-0-76ee270337f664b36cacfc6528ac109 | 1 + ...cast #4-0-732ed232ac592c5e7f7c913a88874fd2 | 1 + ...cast #5-0-dbd7bcd167d322d6617b884c02c7f247 | 1 + ...cast #6-0-6d2da5cfada03605834e38bc4075bc79 | 1 + ...cast #7-0-1d70654217035f8ce5f64344f4c5a80f | 1 + ...cast #8-0-6d2da5cfada03605834e38bc4075bc79 | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 24 +++++++++++++++++++ 11 files changed, 50 insertions(+), 15 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f create mode 100644 sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 0379275121bf2..f626d09f037bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -86,15 +86,15 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { try Timestamp.valueOf(n) catch { case _: java.lang.IllegalArgumentException => null } }) case BooleanType => - buildCast[Boolean](_, b => new Timestamp((if (b) 1 else 0) * 1000)) + buildCast[Boolean](_, b => new Timestamp((if (b) 1 else 0))) case LongType => - buildCast[Long](_, l => new Timestamp(l * 1000)) + buildCast[Long](_, l => new Timestamp(l)) case IntegerType => - buildCast[Int](_, i => new Timestamp(i * 1000)) + buildCast[Int](_, i => new Timestamp(i)) case ShortType => - buildCast[Short](_, s => new Timestamp(s * 1000)) + buildCast[Short](_, s => new Timestamp(s)) case ByteType => - buildCast[Byte](_, b => new Timestamp(b * 1000)) + buildCast[Byte](_, b => new Timestamp(b)) // TimestampWritable.decimalToTimestamp case DecimalType => buildCast[BigDecimal](_, d => decimalToTimestamp(d)) @@ -107,11 +107,10 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } private[this] def decimalToTimestamp(d: BigDecimal) = { - val seconds = d.longValue() + val seconds = Math.floor(d.toDouble).toLong val bd = (d - seconds) * 1000000000 val nanos = bd.intValue() - // Convert to millis val millis = seconds * 1000 val t = new Timestamp(millis) @@ -121,11 +120,11 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { } // Timestamp to long, converting milliseconds to seconds - private[this] def timestampToLong(ts: Timestamp) = ts.getTime / 1000 + private[this] def timestampToLong(ts: Timestamp) = Math.floor(ts.getTime / 1000.0).toLong private[this] def timestampToDouble(ts: Timestamp) = { // First part is the seconds since the beginning of time, followed by nanosecs. - ts.getTime / 1000 + ts.getNanos.toDouble / 1000000000 + Math.floor(ts.getTime / 1000.0).toLong + ts.getNanos.toDouble / 1000000000 } // Converts Timestamp to string according to Hive TimestampWritable convention 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 b961346dfc995..8b6721d5d8125 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 @@ -231,7 +231,9 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("12.65" cast DecimalType, BigDecimal(12.65)) checkEvaluation(Literal(1) cast LongType, 1) - checkEvaluation(Cast(Literal(1) cast TimestampType, LongType), 1) + checkEvaluation(Cast(Literal(1000) cast TimestampType, LongType), 1.toLong) + checkEvaluation(Cast(Literal(-1200) cast TimestampType, LongType), -2.toLong) + checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts) @@ -242,11 +244,11 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Cast(Cast(Cast( Cast("5" cast ByteType, ShortType), IntegerType), FloatType), DoubleType), LongType), 5) checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast ByteType, TimestampType), DecimalType), LongType), StringType), ShortType), 5) + Cast("5" cast ByteType, TimestampType), DecimalType), LongType), StringType), ShortType), 0) checkEvaluation(Cast(Cast(Cast(Cast( Cast("5" cast TimestampType, ByteType), DecimalType), LongType), StringType), ShortType), null) checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast DecimalType, ByteType), TimestampType), LongType), StringType), ShortType), 5) + Cast("5" cast DecimalType, ByteType), TimestampType), LongType), StringType), ShortType), 0) checkEvaluation(Literal(true) cast IntegerType, 1) checkEvaluation(Literal(false) cast IntegerType, 0) checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1) @@ -293,16 +295,18 @@ class ExpressionEvaluationSuite extends FunSuite { test("timestamp casting") { val millis = 15 * 1000 + 2 + val seconds = millis * 1000 + 2 val ts = new Timestamp(millis) val ts1 = new Timestamp(15 * 1000) // a timestamp without the milliseconds part + val tss = new Timestamp(seconds) checkEvaluation(Cast(ts, ShortType), 15) checkEvaluation(Cast(ts, IntegerType), 15) checkEvaluation(Cast(ts, LongType), 15) checkEvaluation(Cast(ts, FloatType), 15.002f) checkEvaluation(Cast(ts, DoubleType), 15.002) - checkEvaluation(Cast(Cast(ts, ShortType), TimestampType), ts1) - checkEvaluation(Cast(Cast(ts, IntegerType), TimestampType), ts1) - checkEvaluation(Cast(Cast(ts, LongType), TimestampType), ts1) + checkEvaluation(Cast(Cast(tss, ShortType), TimestampType), ts) + checkEvaluation(Cast(Cast(tss, IntegerType), TimestampType), ts) + checkEvaluation(Cast(Cast(tss, LongType), TimestampType), ts) checkEvaluation(Cast(Cast(millis.toFloat / 1000, TimestampType), FloatType), millis.toFloat / 1000) checkEvaluation(Cast(Cast(millis.toDouble / 1000, TimestampType), DoubleType), diff --git a/sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 b/sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 new file mode 100644 index 0000000000000..8ebf695ba7d20 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 @@ -0,0 +1 @@ +0.001 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 new file mode 100644 index 0000000000000..5625e59da8873 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #2-0-732ed232ac592c5e7f7c913a88874fd2 @@ -0,0 +1 @@ +1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #3-0-76ee270337f664b36cacfc6528ac109 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 new file mode 100644 index 0000000000000..5625e59da8873 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #4-0-732ed232ac592c5e7f7c913a88874fd2 @@ -0,0 +1 @@ +1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 b/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 new file mode 100644 index 0000000000000..27de46fdf22ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #5-0-dbd7bcd167d322d6617b884c02c7f247 @@ -0,0 +1 @@ +-0.0010000000000000009 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 new file mode 100644 index 0000000000000..1d94c8a014fb4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #6-0-6d2da5cfada03605834e38bc4075bc79 @@ -0,0 +1 @@ +-1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f b/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f new file mode 100644 index 0000000000000..3fbedf693b51d --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #7-0-1d70654217035f8ce5f64344f4c5a80f @@ -0,0 +1 @@ +-2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 new file mode 100644 index 0000000000000..1d94c8a014fb4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp cast #8-0-6d2da5cfada03605834e38bc4075bc79 @@ -0,0 +1 @@ +-1.2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 56bcd95eab4bc..6fc891ba4cca5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -303,6 +303,30 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("case statements WITHOUT key #4", "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15") + createQueryTest("timestamp cast #1", + "SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #2", + "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #3", + "SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1") + + createQueryTest("timestamp cast #4", + "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #5", + "SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #6", + "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + + createQueryTest("timestamp cast #7", + "SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1") + + createQueryTest("timestamp cast #8", + "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + test("implement identity function using case statement") { val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src") .map { case Row(i: Int) => i } From 116016b481cecbd8ad6e9717d92f977a164a6653 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 23 Sep 2014 11:47:53 -0700 Subject: [PATCH 0769/1492] [SPARK-3582][SQL] not limit argument type for hive simple udf Since we have moved to `ConventionHelper`, it is quite easy to avoid call `javaClassToDataType` in hive simple udf. This will solve SPARK-3582. Author: Daoyuan Wang Closes #2506 from adrian-wang/spark3582 and squashes the following commits: 450c28e [Daoyuan Wang] not limit argument type for hive simple udf --- .../spark/sql/hive/HiveInspectors.scala | 4 ++-- .../org/apache/spark/sql/hive/hiveUdfs.scala | 22 ++----------------- 2 files changed, 4 insertions(+), 22 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 943bbaa8ce25e..fa889ec104c6e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -137,7 +137,7 @@ private[hive] trait HiveInspectors { /** Converts native catalyst types to the types expected by Hive */ def wrap(a: Any): AnyRef = a match { - case s: String => new hadoopIo.Text(s) // TODO why should be Text? + case s: String => s: java.lang.String case i: Int => i: java.lang.Integer case b: Boolean => b: java.lang.Boolean case f: Float => f: java.lang.Float @@ -145,7 +145,7 @@ private[hive] trait HiveInspectors { case l: Long => l: java.lang.Long case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte - case b: BigDecimal => b.bigDecimal + case b: BigDecimal => new HiveDecimal(b.underlying()) case b: Array[Byte] => b case t: java.sql.Timestamp => t case s: Seq[_] => seqAsJavaList(s.map(wrap)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 19ff3b66ad7ed..68944ed4ef21d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -51,19 +51,7 @@ private[hive] abstract class HiveFunctionRegistry val functionClassName = functionInfo.getFunctionClass.getName if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - val function = functionInfo.getFunctionClass.newInstance().asInstanceOf[UDF] - val method = function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) - - val expectedDataTypes = method.getParameterTypes.map(javaClassToDataType) - - HiveSimpleUdf( - functionClassName, - children.zip(expectedDataTypes).map { - case (e, NullType) => e - case (e, t) if (e.dataType == t) => e - case (e, t) => Cast(e, t) - } - ) + HiveSimpleUdf(functionClassName, children) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdf(functionClassName, children) } else if ( @@ -117,15 +105,9 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ @transient lazy val dataType = javaClassToDataType(method.getReturnType) - def catalystToHive(value: Any): Object = value match { - // TODO need more types here? or can we use wrap() - case bd: BigDecimal => new HiveDecimal(bd.underlying()) - case d => d.asInstanceOf[Object] - } - // TODO: Finish input output types. override def eval(input: Row): Any = { - val evaluatedChildren = children.map(c => catalystToHive(c.eval(input))) + val evaluatedChildren = children.map(c => wrap(c.eval(input))) unwrap(FunctionRegistry.invoke(method, function, conversionHelper .convertIfNecessary(evaluatedChildren: _*): _*)) From 3b8eefa9b843c7f1e0e8dda6023272bc9f011c5c Mon Sep 17 00:00:00 2001 From: ravipesala Date: Tue, 23 Sep 2014 11:52:13 -0700 Subject: [PATCH 0770/1492] [SPARK-3536][SQL] SELECT on empty parquet table throws exception It returns null metadata from parquet if querying on empty parquet file while calculating splits.So added null check and returns the empty splits. Author : ravipesala ravindra.pesalahuawei.com Author: ravipesala Closes #2456 from ravipesala/SPARK-3536 and squashes the following commits: 1e81a50 [ravipesala] Fixed the issue when querying on empty parquet file. --- .../spark/sql/parquet/ParquetTableOperations.scala | 7 +++++-- .../org/apache/spark/sql/parquet/ParquetQuerySuite.scala | 9 +++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index a5a5d139a65cb..d39e31a7fa195 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -427,11 +427,15 @@ private[parquet] class FilteringParquetRowInputFormat s"maxSplitSize or minSplitSie should not be negative: maxSplitSize = $maxSplitSize;" + s" minSplitSize = $minSplitSize") } - + val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] val getGlobalMetaData = classOf[ParquetFileWriter].getDeclaredMethod("getGlobalMetaData", classOf[JList[Footer]]) getGlobalMetaData.setAccessible(true) val globalMetaData = getGlobalMetaData.invoke(null, footers).asInstanceOf[GlobalMetaData] + // if parquet file is empty, return empty splits. + if (globalMetaData == null) { + return splits + } val readContext = getReadSupport(configuration).init( new InitContext(configuration, @@ -442,7 +446,6 @@ private[parquet] class FilteringParquetRowInputFormat classOf[ParquetInputFormat[_]].getDeclaredMethods.find(_.getName == "generateSplits").get generateSplits.setAccessible(true) - val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] for (footer <- footers) { val fs = footer.getFile.getFileSystem(configuration) val file = footer.getFile 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 08f7358446b29..07adf731405af 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 @@ -789,4 +789,13 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } + + test("Querying on empty parquet throws exception (SPARK-3536)") { + val tmpdir = Utils.createTempDir() + Utils.deleteRecursively(tmpdir) + createParquetFile[TestRDDEntry](tmpdir.toString()).registerTempTable("tmpemptytable") + val result1 = sql("SELECT * FROM tmpemptytable").collect() + assert(result1.size === 0) + Utils.deleteRecursively(tmpdir) + } } From e73b48ace0a7e0f249221240140235d33eeac36b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 23 Sep 2014 11:58:05 -0700 Subject: [PATCH 0771/1492] SPARK-2745 [STREAMING] Add Java friendly methods to Duration class tdas is this what you had in mind for this JIRA? I saw this one and thought it would be easy to take care of, and helpful as I use streaming from Java. I could do the same for `Time`? Happy to do so. Author: Sean Owen Closes #2403 from srowen/SPARK-2745 and squashes the following commits: 5a9e706 [Sean Owen] Change "Duration" to "Durations" to avoid changing Duration case class API bda301c [Sean Owen] Just delegate to Scala binary operator syntax to avoid scalastyle warning 7dde949 [Sean Owen] Disable scalastyle for false positives. Add Java static factory methods seconds(), minutes() to Duration. Add Java-friendly methods to Time too, and unit tests. Remove unnecessary math.floor from Time.floor() 4dee32e [Sean Owen] Add named methods to Duration in parallel to symbolic methods for Java-friendliness. Also add unit tests for Duration, in Scala and Java. --- .../org/apache/spark/streaming/Duration.scala | 39 ++++++ .../org/apache/spark/streaming/Time.scala | 20 +++- .../spark/streaming/JavaDurationSuite.java | 84 +++++++++++++ .../apache/spark/streaming/JavaTimeSuite.java | 63 ++++++++++ .../spark/streaming/DurationSuite.scala | 110 +++++++++++++++++ .../apache/spark/streaming/TimeSuite.scala | 111 ++++++++++++++++++ 6 files changed, 425 insertions(+), 2 deletions(-) create mode 100644 streaming/src/test/java/org/apache/spark/streaming/JavaDurationSuite.java create mode 100644 streaming/src/test/java/org/apache/spark/streaming/JavaTimeSuite.java create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/DurationSuite.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala index 6bf275f5afcb2..a0d8fb5ab93ec 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Duration.scala @@ -37,6 +37,25 @@ case class Duration (private val millis: Long) { def / (that: Duration): Double = millis.toDouble / that.millis.toDouble + // Java-friendlier versions of the above. + + def less(that: Duration): Boolean = this < that + + def lessEq(that: Duration): Boolean = this <= that + + def greater(that: Duration): Boolean = this > that + + def greaterEq(that: Duration): Boolean = this >= that + + def plus(that: Duration): Duration = this + that + + def minus(that: Duration): Duration = this - that + + def times(times: Int): Duration = this * times + + def div(that: Duration): Double = this / that + + def isMultipleOf(that: Duration): Boolean = (this.millis % that.millis == 0) @@ -80,4 +99,24 @@ object Minutes { def apply(minutes: Long) = new Duration(minutes * 60000) } +// Java-friendlier versions of the objects above. +// Named "Durations" instead of "Duration" to avoid changing the case class's implied API. + +object Durations { + + /** + * @return [[org.apache.spark.streaming.Duration]] representing given number of milliseconds. + */ + def milliseconds(milliseconds: Long) = Milliseconds(milliseconds) + /** + * @return [[org.apache.spark.streaming.Duration]] representing given number of seconds. + */ + def seconds(seconds: Long) = Seconds(seconds) + + /** + * @return [[org.apache.spark.streaming.Duration]] representing given number of minutes. + */ + def minutes(minutes: Long) = Minutes(minutes) + +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 37b3b28fa01cb..42c49678d24f0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -41,10 +41,26 @@ case class Time(private val millis: Long) { def - (that: Duration): Time = new Time(millis - that.milliseconds) + // Java-friendlier versions of the above. + + def less(that: Time): Boolean = this < that + + def lessEq(that: Time): Boolean = this <= that + + def greater(that: Time): Boolean = this > that + + def greaterEq(that: Time): Boolean = this >= that + + def plus(that: Duration): Time = this + that + + def minus(that: Time): Duration = this - that + + def minus(that: Duration): Time = this - that + + def floor(that: Duration): Time = { val t = that.milliseconds - val m = math.floor(this.millis / t).toLong - new Time(m * t) + new Time((this.millis / t) * t) } def isMultipleOf(that: Duration): Boolean = diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaDurationSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaDurationSuite.java new file mode 100644 index 0000000000000..76425fe2aa2d3 --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaDurationSuite.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.streaming; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaDurationSuite { + + // Just testing the methods that are specially exposed for Java. + // This does not repeat all tests found in the Scala suite. + + @Test + public void testLess() { + Assert.assertTrue(new Duration(999).less(new Duration(1000))); + } + + @Test + public void testLessEq() { + Assert.assertTrue(new Duration(1000).lessEq(new Duration(1000))); + } + + @Test + public void testGreater() { + Assert.assertTrue(new Duration(1000).greater(new Duration(999))); + } + + @Test + public void testGreaterEq() { + Assert.assertTrue(new Duration(1000).greaterEq(new Duration(1000))); + } + + @Test + public void testPlus() { + Assert.assertEquals(new Duration(1100), new Duration(1000).plus(new Duration(100))); + } + + @Test + public void testMinus() { + Assert.assertEquals(new Duration(900), new Duration(1000).minus(new Duration(100))); + } + + @Test + public void testTimes() { + Assert.assertEquals(new Duration(200), new Duration(100).times(2)); + } + + @Test + public void testDiv() { + Assert.assertEquals(200.0, new Duration(1000).div(new Duration(5)), 1.0e-12); + } + + @Test + public void testMilliseconds() { + Assert.assertEquals(new Duration(100), Durations.milliseconds(100)); + } + + @Test + public void testSeconds() { + Assert.assertEquals(new Duration(30 * 1000), Durations.seconds(30)); + } + + @Test + public void testMinutes() { + Assert.assertEquals(new Duration(2 * 60 * 1000), Durations.minutes(2)); + } + +} diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaTimeSuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaTimeSuite.java new file mode 100644 index 0000000000000..ad6b1853e3d12 --- /dev/null +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaTimeSuite.java @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaTimeSuite { + + // Just testing the methods that are specially exposed for Java. + // This does not repeat all tests found in the Scala suite. + + @Test + public void testLess() { + Assert.assertTrue(new Time(999).less(new Time(1000))); + } + + @Test + public void testLessEq() { + Assert.assertTrue(new Time(1000).lessEq(new Time(1000))); + } + + @Test + public void testGreater() { + Assert.assertTrue(new Time(1000).greater(new Time(999))); + } + + @Test + public void testGreaterEq() { + Assert.assertTrue(new Time(1000).greaterEq(new Time(1000))); + } + + @Test + public void testPlus() { + Assert.assertEquals(new Time(1100), new Time(1000).plus(new Duration(100))); + } + + @Test + public void testMinusTime() { + Assert.assertEquals(new Duration(900), new Time(1000).minus(new Time(100))); + } + + @Test + public void testMinusDuration() { + Assert.assertEquals(new Time(900), new Time(1000).minus(new Duration(100))); + } + +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/DurationSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/DurationSuite.scala new file mode 100644 index 0000000000000..6202250e897f2 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/DurationSuite.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +class DurationSuite extends TestSuiteBase { + + test("less") { + assert(new Duration(999) < new Duration(1000)) + assert(new Duration(0) < new Duration(1)) + assert(!(new Duration(1000) < new Duration(999))) + assert(!(new Duration(1000) < new Duration(1000))) + } + + test("lessEq") { + assert(new Duration(999) <= new Duration(1000)) + assert(new Duration(0) <= new Duration(1)) + assert(!(new Duration(1000) <= new Duration(999))) + assert(new Duration(1000) <= new Duration(1000)) + } + + test("greater") { + assert(!(new Duration(999) > new Duration(1000))) + assert(!(new Duration(0) > new Duration(1))) + assert(new Duration(1000) > new Duration(999)) + assert(!(new Duration(1000) > new Duration(1000))) + } + + test("greaterEq") { + assert(!(new Duration(999) >= new Duration(1000))) + assert(!(new Duration(0) >= new Duration(1))) + assert(new Duration(1000) >= new Duration(999)) + assert(new Duration(1000) >= new Duration(1000)) + } + + test("plus") { + assert((new Duration(1000) + new Duration(100)) == new Duration(1100)) + assert((new Duration(1000) + new Duration(0)) == new Duration(1000)) + } + + test("minus") { + assert((new Duration(1000) - new Duration(100)) == new Duration(900)) + assert((new Duration(1000) - new Duration(0)) == new Duration(1000)) + assert((new Duration(1000) - new Duration(1000)) == new Duration(0)) + } + + test("times") { + assert((new Duration(100) * 2) == new Duration(200)) + assert((new Duration(100) * 1) == new Duration(100)) + assert((new Duration(100) * 0) == new Duration(0)) + } + + test("div") { + assert((new Duration(1000) / new Duration(5)) == 200.0) + assert((new Duration(1000) / new Duration(1)) == 1000.0) + assert((new Duration(1000) / new Duration(1000)) == 1.0) + assert((new Duration(1000) / new Duration(2000)) == 0.5) + } + + test("isMultipleOf") { + assert(new Duration(1000).isMultipleOf(new Duration(5))) + assert(new Duration(1000).isMultipleOf(new Duration(1000))) + assert(new Duration(1000).isMultipleOf(new Duration(1))) + assert(!new Duration(1000).isMultipleOf(new Duration(6))) + } + + test("min") { + assert(new Duration(999).min(new Duration(1000)) == new Duration(999)) + assert(new Duration(1000).min(new Duration(999)) == new Duration(999)) + assert(new Duration(1000).min(new Duration(1000)) == new Duration(1000)) + } + + test("max") { + assert(new Duration(999).max(new Duration(1000)) == new Duration(1000)) + assert(new Duration(1000).max(new Duration(999)) == new Duration(1000)) + assert(new Duration(1000).max(new Duration(1000)) == new Duration(1000)) + } + + test("isZero") { + assert(new Duration(0).isZero) + assert(!(new Duration(1).isZero)) + } + + test("Milliseconds") { + assert(new Duration(100) == Milliseconds(100)) + } + + test("Seconds") { + assert(new Duration(30 * 1000) == Seconds(30)) + } + + test("Minutes") { + assert(new Duration(2 * 60 * 1000) == Minutes(2)) + } + +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala new file mode 100644 index 0000000000000..5579ac364346c --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/TimeSuite.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +class TimeSuite extends TestSuiteBase { + + test("less") { + assert(new Time(999) < new Time(1000)) + assert(new Time(0) < new Time(1)) + assert(!(new Time(1000) < new Time(999))) + assert(!(new Time(1000) < new Time(1000))) + } + + test("lessEq") { + assert(new Time(999) <= new Time(1000)) + assert(new Time(0) <= new Time(1)) + assert(!(new Time(1000) <= new Time(999))) + assert(new Time(1000) <= new Time(1000)) + } + + test("greater") { + assert(!(new Time(999) > new Time(1000))) + assert(!(new Time(0) > new Time(1))) + assert(new Time(1000) > new Time(999)) + assert(!(new Time(1000) > new Time(1000))) + } + + test("greaterEq") { + assert(!(new Time(999) >= new Time(1000))) + assert(!(new Time(0) >= new Time(1))) + assert(new Time(1000) >= new Time(999)) + assert(new Time(1000) >= new Time(1000)) + } + + test("plus") { + assert((new Time(1000) + new Duration(100)) == new Time(1100)) + assert((new Time(1000) + new Duration(0)) == new Time(1000)) + } + + test("minus Time") { + assert((new Time(1000) - new Time(100)) == new Duration(900)) + assert((new Time(1000) - new Time(0)) == new Duration(1000)) + assert((new Time(1000) - new Time(1000)) == new Duration(0)) + } + + test("minus Duration") { + assert((new Time(1000) - new Duration(100)) == new Time(900)) + assert((new Time(1000) - new Duration(0)) == new Time(1000)) + assert((new Time(1000) - new Duration(1000)) == new Time(0)) + } + + test("floor") { + assert(new Time(1350).floor(new Duration(200)) == new Time(1200)) + assert(new Time(1200).floor(new Duration(200)) == new Time(1200)) + assert(new Time(199).floor(new Duration(200)) == new Time(0)) + assert(new Time(1).floor(new Duration(1)) == new Time(1)) + } + + test("isMultipleOf") { + assert(new Time(1000).isMultipleOf(new Duration(5))) + assert(new Time(1000).isMultipleOf(new Duration(1000))) + assert(new Time(1000).isMultipleOf(new Duration(1))) + assert(!new Time(1000).isMultipleOf(new Duration(6))) + } + + test("min") { + assert(new Time(999).min(new Time(1000)) == new Time(999)) + assert(new Time(1000).min(new Time(999)) == new Time(999)) + assert(new Time(1000).min(new Time(1000)) == new Time(1000)) + } + + test("max") { + assert(new Time(999).max(new Time(1000)) == new Time(1000)) + assert(new Time(1000).max(new Time(999)) == new Time(1000)) + assert(new Time(1000).max(new Time(1000)) == new Time(1000)) + } + + test("until") { + assert(new Time(1000).until(new Time(1100), new Duration(100)) == + Seq(Time(1000))) + assert(new Time(1000).until(new Time(1000), new Duration(100)) == + Seq()) + assert(new Time(1000).until(new Time(1100), new Duration(30)) == + Seq(Time(1000), Time(1030), Time(1060), Time(1090))) + } + + test("to") { + assert(new Time(1000).to(new Time(1100), new Duration(100)) == + Seq(Time(1000), Time(1100))) + assert(new Time(1000).to(new Time(1000), new Duration(100)) == + Seq(Time(1000))) + assert(new Time(1000).to(new Time(1100), new Duration(30)) == + Seq(Time(1000), Time(1030), Time(1060), Time(1090))) + } + +} From ae60f8fb2d879ee1ebc0746bcbe05b89ab6ed3c9 Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 23 Sep 2014 11:59:44 -0700 Subject: [PATCH 0772/1492] [SPARK-3481][SQL] removes the evil MINOR HACK a follow up of https://github.com/apache/spark/pull/2377 and https://github.com/apache/spark/pull/2352, see detail there. Author: wangfei Closes #2505 from scwf/patch-6 and squashes the following commits: 4874ec8 [wangfei] removes the evil MINOR HACK --- .../org/apache/spark/sql/hive/execution/PruningSuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 8275e2d3bcce3..8474d850c9c6c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -28,8 +28,6 @@ import scala.collection.JavaConversions._ * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { - // MINOR HACK: You must run a query before calling reset the first time. - TestHive.sql("SHOW TABLES") TestHive.cacheTables = false // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, need to reset From 1c62f97e94de96ca3dc6daf778f008176e92888a Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Tue, 23 Sep 2014 12:17:47 -0700 Subject: [PATCH 0773/1492] [SPARK-3268][SQL] DoubleType, FloatType and DecimalType modulus support Supported modulus operation using % operator on fractional datatypes FloatType, DoubleType and DecimalType Example: SELECT 1388632775.0 % 60 from tablename LIMIT 1 Author : Venkata Ramana Gollamudi ramana.gollamudihuawei.com Author: Venkata Ramana Gollamudi Closes #2457 from gvramana/double_modulus_support and squashes the following commits: 79172a8 [Venkata Ramana Gollamudi] Add hive cache to testcase c09bd5b [Venkata Ramana Gollamudi] Added a HiveQuerySuite testcase 193fa81 [Venkata Ramana Gollamudi] corrected testcase 3624471 [Venkata Ramana Gollamudi] modified testcase e112c09 [Venkata Ramana Gollamudi] corrected the testcase 513d0e0 [Venkata Ramana Gollamudi] modified to add modulus support to fractional types float,double,decimal 296d253 [Venkata Ramana Gollamudi] modified to add modulus support to fractional types float,double,decimal --- .../sql/catalyst/expressions/Expression.scala | 3 ++ .../spark/sql/catalyst/types/dataTypes.scala | 5 +++ .../ExpressionEvaluationSuite.scala | 32 +++++++++++++++++++ ...modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 3 ++ 5 files changed, 44 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 70507e7ee2be8..1eb260efa6387 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -179,6 +179,9 @@ abstract class Expression extends TreeNode[Expression] { case i: IntegralType => f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( i.integral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) + case i: FractionalType => + f.asInstanceOf[(Integral[i.JvmType], i.JvmType, i.JvmType) => i.JvmType]( + i.asIntegral, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) case other => sys.error(s"Type $other does not support numeric operations") } } 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 e3050e5397937..c7d73d3990c3a 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,6 +19,7 @@ package org.apache.spark.sql.catalyst.types import java.sql.Timestamp +import scala.math.Numeric.{FloatAsIfIntegral, BigDecimalAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} import scala.util.parsing.combinator.RegexParsers @@ -250,6 +251,7 @@ object FractionalType { } abstract class FractionalType extends NumericType { private[sql] val fractional: Fractional[JvmType] + private[sql] val asIntegral: Integral[JvmType] } case object DecimalType extends FractionalType { @@ -258,6 +260,7 @@ case object DecimalType extends FractionalType { private[sql] val numeric = implicitly[Numeric[BigDecimal]] private[sql] val fractional = implicitly[Fractional[BigDecimal]] private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = BigDecimalAsIfIntegral def simpleString: String = "decimal" } @@ -267,6 +270,7 @@ case object DoubleType extends FractionalType { private[sql] val numeric = implicitly[Numeric[Double]] private[sql] val fractional = implicitly[Fractional[Double]] private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = DoubleAsIfIntegral def simpleString: String = "double" } @@ -276,6 +280,7 @@ case object FloatType extends FractionalType { private[sql] val numeric = implicitly[Numeric[Float]] private[sql] val fractional = implicitly[Fractional[Float]] private[sql] val ordering = implicitly[Ordering[JvmType]] + private[sql] val asIntegral = FloatAsIfIntegral def simpleString: String = "float" } 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 8b6721d5d8125..63931af4bac3d 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp import org.scalatest.FunSuite +import org.scalatest.Matchers._ +import org.scalautils.TripleEqualsSupport.Spread import org.apache.spark.sql.catalyst.types._ @@ -129,6 +131,13 @@ class ExpressionEvaluationSuite extends FunSuite { } } + def checkDoubleEvaluation(expression: Expression, expected: Spread[Double], inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + actual.asInstanceOf[Double] shouldBe expected + } + test("IN") { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) @@ -471,6 +480,29 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 % c2, 1, row) } + test("fractional arithmetic") { + val row = new GenericRow(Array[Any](1.1, 2.0, 3.1, null)) + val c1 = 'a.double.at(0) + val c2 = 'a.double.at(1) + val c3 = 'a.double.at(2) + val c4 = 'a.double.at(3) + + checkEvaluation(UnaryMinus(c1), -1.1, row) + checkEvaluation(UnaryMinus(Literal(100.0, DoubleType)), -100.0) + checkEvaluation(Add(c1, c4), null, row) + checkEvaluation(Add(c1, c2), 3.1, row) + checkEvaluation(Add(c1, Literal(null, DoubleType)), null, row) + checkEvaluation(Add(Literal(null, DoubleType), c2), null, row) + checkEvaluation(Add(Literal(null, DoubleType), Literal(null, DoubleType)), null, row) + + checkEvaluation(-c1, -1.1, row) + checkEvaluation(c1 + c2, 3.1, row) + checkDoubleEvaluation(c1 - c2, (-0.9 +- 0.001), row) + checkDoubleEvaluation(c1 * c2, (2.2 +- 0.001), row) + checkDoubleEvaluation(c1 / c2, (0.55 +- 0.001), row) + checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row) + } + test("BinaryComparison") { val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) val c1 = 'a.int.at(0) diff --git a/sql/hive/src/test/resources/golden/modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e b/sql/hive/src/test/resources/golden/modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e new file mode 100644 index 0000000000000..52eab0653c505 --- /dev/null +++ b/sql/hive/src/test/resources/golden/modulus-0-6afd4a359a478cfa3ebd9ad00ae3868e @@ -0,0 +1 @@ +1 true 0.5 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 6fc891ba4cca5..426f5fcee6157 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -138,6 +138,9 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("division", "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1") + createQueryTest("modulus", + "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), (101 / 2) % 10 FROM src LIMIT 1") + test("Query expressed in SQL") { setConf("spark.sql.dialect", "sql") assert(sql("SELECT 1").collect() === Array(Seq(1))) From a08153f8a3e7bad81bae330ec4152651da5e7804 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 23 Sep 2014 12:27:12 -0700 Subject: [PATCH 0774/1492] [SPARK-3646][SQL] Copy SQL configuration from SparkConf when a SQLContext is created. This will allow us to take advantage of things like the spark.defaults file. Author: Michael Armbrust Closes #2493 from marmbrus/copySparkConf and squashes the following commits: 0bd1377 [Michael Armbrust] Copy SQL configuration from SparkConf when a SQLContext is created. --- .../main/scala/org/apache/spark/sql/SQLContext.scala | 5 +++++ .../org/apache/spark/sql/test/TestSQLContext.scala | 6 +++++- .../scala/org/apache/spark/sql/SQLConfSuite.scala | 11 ++++++++++- 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index b245e1a863cc3..a42bedbe6c04e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -75,6 +75,11 @@ class SQLContext(@transient val sparkContext: SparkContext) protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } + sparkContext.getConf.getAll.foreach { + case (key, value) if key.startsWith("spark.sql") => setConf(key, value) + case _ => + } + /** * :: DeveloperApi :: * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 265b67737c475..6bb81c76ed8bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -22,7 +22,11 @@ import org.apache.spark.sql.{SQLConf, SQLContext} /** A SQLContext that can be used for local testing. */ object TestSQLContext - extends SQLContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) { + extends SQLContext( + new SparkContext( + "local[2]", + "TestSQLContext", + new SparkConf().set("spark.sql.testkey", "true"))) { /** Fewer partitions to speed up testing. */ override private[spark] def numShufflePartitions: Int = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 584f71b3c13d5..60701f0e154f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -17,16 +17,25 @@ package org.apache.spark.sql +import org.scalatest.FunSuiteLike + import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ -class SQLConfSuite extends QueryTest { +class SQLConfSuite extends QueryTest with FunSuiteLike { val testKey = "test.key.0" val testVal = "test.val.0" + test("propagate from spark conf") { + // We create a new context here to avoid order dependence with other tests that might call + // clear(). + val newContext = new SQLContext(TestSQLContext.sparkContext) + assert(newContext.getConf("spark.sql.testkey", "false") == "true") + } + test("programmatic ways of basic setting and getting") { clear() assert(getAllConfs.size === 0) From 8dfe79ffb204807945e3c09b75c7255b09ad2a97 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 23 Sep 2014 13:42:00 -0700 Subject: [PATCH 0775/1492] [SPARK-3647] Add more exceptions to Guava relocation. Guava's Optional refers to some package private classes / methods, and when those are relocated the code stops working, throwing exceptions. So add the affected classes to the exception list too, and add a unit test. (Note that this unit test only really makes sense in maven, since we don't relocate in the sbt build. Also, JavaAPISuite doesn't seem to be run by "mvn test" - I had to manually add command line options to enable it.) Author: Marcelo Vanzin Closes #2496 from vanzin/SPARK-3647 and squashes the following commits: 84f58d7 [Marcelo Vanzin] [SPARK-3647] Add more exceptions to Guava relocation. --- assembly/pom.xml | 4 ++- core/pom.xml | 2 ++ .../java/org/apache/spark/JavaAPISuite.java | 26 +++++++++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 604b1ab3de6a8..5ec9da22ae83f 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -141,7 +141,9 @@ com.google.common.** - com.google.common.base.Optional** + com/google/common/base/Absent* + com/google/common/base/Optional* + com/google/common/base/Present* diff --git a/core/pom.xml b/core/pom.xml index 2a81f6df289c0..e012c5e673b74 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -343,7 +343,9 @@ com.google.guava:guava + com/google/common/base/Absent* com/google/common/base/Optional* + com/google/common/base/Present* diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b8574dfb42e6b..b8c23d524e00b 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1307,4 +1307,30 @@ public void collectUnderlyingScalaRDD() { SomeCustomClass[] collected = (SomeCustomClass[]) rdd.rdd().retag(SomeCustomClass.class).collect(); Assert.assertEquals(data.size(), collected.length); } + + /** + * Test for SPARK-3647. This test needs to use the maven-built assembly to trigger the issue, + * since that's the only artifact where Guava classes have been relocated. + */ + @Test + public void testGuavaOptional() { + // Stop the context created in setUp() and start a local-cluster one, to force usage of the + // assembly. + sc.stop(); + JavaSparkContext localCluster = new JavaSparkContext("local-cluster[1,1,512]", "JavaAPISuite"); + try { + JavaRDD rdd1 = localCluster.parallelize(Arrays.asList(1, 2, null), 3); + JavaRDD> rdd2 = rdd1.map( + new Function>() { + @Override + public Optional call(Integer i) { + return Optional.fromNullable(i); + } + }); + rdd2.collect(); + } finally { + localCluster.stop(); + } + } + } From d79238d03a2ffe0cf5fc6166543d67768693ddbe Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 23 Sep 2014 13:44:18 -0700 Subject: [PATCH 0776/1492] SPARK-3612. Executor shouldn't quit if heartbeat message fails to reach ... ...the driver Author: Sandy Ryza Closes #2487 from sryza/sandy-spark-3612 and squashes the following commits: 2b7353d [Sandy Ryza] SPARK-3612. Executor shouldn't quit if heartbeat message fails to reach the driver --- .../org/apache/spark/executor/Executor.scala | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index acae448a9c66f..d7211ae465902 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -24,6 +24,7 @@ import java.util.concurrent._ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil @@ -375,12 +376,17 @@ private[spark] class Executor( } val message = Heartbeat(executorId, tasksMetrics.toArray, env.blockManager.blockManagerId) - val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, - retryAttempts, retryIntervalMs, timeout) - if (response.reregisterBlockManager) { - logWarning("Told to re-register on heartbeat") - env.blockManager.reregister() + try { + val response = AkkaUtils.askWithReply[HeartbeatResponse](message, heartbeatReceiverRef, + retryAttempts, retryIntervalMs, timeout) + if (response.reregisterBlockManager) { + logWarning("Told to re-register on heartbeat") + env.blockManager.reregister() + } + } catch { + case NonFatal(t) => logWarning("Issue communicating with driver in heartbeater", t) } + Thread.sleep(interval) } } From b3fef50e22fb3fe499f627179d17836a92dcb33a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 23 Sep 2014 14:00:33 -0700 Subject: [PATCH 0777/1492] [SPARK-3653] Respect SPARK_*_MEMORY for cluster mode `SPARK_DRIVER_MEMORY` was only used to start the `SparkSubmit` JVM, which becomes the driver only in client mode but not cluster mode. In cluster mode, this property is simply not propagated to the worker nodes. `SPARK_EXECUTOR_MEMORY` is picked up from `SparkContext`, but in cluster mode the driver runs on one of the worker machines, where this environment variable may not be set. Author: Andrew Or Closes #2500 from andrewor14/memory-env-vars and squashes the following commits: 6217b38 [Andrew Or] Respect SPARK_*_MEMORY for cluster mode --- .../scala/org/apache/spark/deploy/SparkSubmitArguments.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 92e0917743ed1..2b72c61cc8177 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -75,6 +75,10 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { defaultProperties } + // Respect SPARK_*_MEMORY for cluster mode + driverMemory = sys.env.get("SPARK_DRIVER_MEMORY").orNull + executorMemory = sys.env.get("SPARK_EXECUTOR_MEMORY").orNull + parseOpts(args.toList) mergeSparkProperties() checkRequiredArguments() From 729952a5efce755387c76cdf29280ee6f49fdb72 Mon Sep 17 00:00:00 2001 From: Mubarak Seyed Date: Tue, 23 Sep 2014 15:09:12 -0700 Subject: [PATCH 0778/1492] [SPARK-1853] Show Streaming application code context (file, line number) in Spark Stages UI This is a refactored version of the original PR https://github.com/apache/spark/pull/1723 my mubarak Please take a look andrewor14, mubarak Author: Mubarak Seyed Author: Tathagata Das Closes #2464 from tdas/streaming-callsite and squashes the following commits: dc54c71 [Tathagata Das] Made changes based on PR comments. 390b45d [Tathagata Das] Fixed minor bugs. 904cd92 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-callsite 7baa427 [Tathagata Das] Refactored getCallSite and setCallSite to make it simpler. Also added unit test for DStream creation site. b9ed945 [Mubarak Seyed] Adding streaming utils c461cf4 [Mubarak Seyed] Merge remote-tracking branch 'upstream/master' ceb43da [Mubarak Seyed] Changing default regex function name 8c5d443 [Mubarak Seyed] Merge remote-tracking branch 'upstream/master' 196121b [Mubarak Seyed] Merge remote-tracking branch 'upstream/master' 491a1eb [Mubarak Seyed] Removing streaming visibility from getRDDCreationCallSite in DStream 33a7295 [Mubarak Seyed] Fixing review comments: Merging both setCallSite methods c26d933 [Mubarak Seyed] Merge remote-tracking branch 'upstream/master' f51fd9f [Mubarak Seyed] Fixing scalastyle, Regex for Utils.getCallSite, and changing method names in DStream 5051c58 [Mubarak Seyed] Getting return value of compute() into variable and call setCallSite(prevCallSite) only once. Adding return for other code paths (for None) a207eb7 [Mubarak Seyed] Fixing code review comments ccde038 [Mubarak Seyed] Removing Utils import from MappedDStream 2a09ad6 [Mubarak Seyed] Changes in Utils.scala for SPARK-1853 1d90cc3 [Mubarak Seyed] Changes for SPARK-1853 5f3105a [Mubarak Seyed] Merge remote-tracking branch 'upstream/master' 70f494f [Mubarak Seyed] Changes for SPARK-1853 1500deb [Mubarak Seyed] Changes in Spark Streaming UI 9d38d3c [Mubarak Seyed] [SPARK-1853] Show Streaming application code context (file, line number) in Spark Stages UI d466d75 [Mubarak Seyed] Changes for spark streaming UI --- .../scala/org/apache/spark/SparkContext.scala | 32 +++++-- .../main/scala/org/apache/spark/rdd/RDD.scala | 7 +- .../scala/org/apache/spark/util/Utils.scala | 27 ++++-- .../spark/streaming/StreamingContext.scala | 4 +- .../spark/streaming/dstream/DStream.scala | 96 ++++++++++++------- .../streaming/StreamingContextSuite.scala | 45 ++++++++- 6 files changed, 153 insertions(+), 58 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 428f019b02a23..979d178c35969 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1030,28 +1030,40 @@ class SparkContext(config: SparkConf) extends Logging { } /** - * Support function for API backtraces. + * Set the thread-local property for overriding the call sites + * of actions and RDDs. */ - def setCallSite(site: String) { - setLocalProperty("externalCallSite", site) + def setCallSite(shortCallSite: String) { + setLocalProperty(CallSite.SHORT_FORM, shortCallSite) } /** - * Support function for API backtraces. + * Set the thread-local property for overriding the call sites + * of actions and RDDs. + */ + private[spark] def setCallSite(callSite: CallSite) { + setLocalProperty(CallSite.SHORT_FORM, callSite.shortForm) + setLocalProperty(CallSite.LONG_FORM, callSite.longForm) + } + + /** + * Clear the thread-local property for overriding the call sites + * of actions and RDDs. */ def clearCallSite() { - setLocalProperty("externalCallSite", null) + setLocalProperty(CallSite.SHORT_FORM, null) + setLocalProperty(CallSite.LONG_FORM, null) } /** * Capture the current user callsite and return a formatted version for printing. If the user - * has overridden the call site, this will return the user's version. + * has overridden the call site using `setCallSite()`, this will return the user's version. */ private[spark] def getCallSite(): CallSite = { - Option(getLocalProperty("externalCallSite")) match { - case Some(callSite) => CallSite(callSite, longForm = "") - case None => Utils.getCallSite - } + Option(getLocalProperty(CallSite.SHORT_FORM)).map { case shortCallSite => + val longCallSite = Option(getLocalProperty(CallSite.LONG_FORM)).getOrElse("") + CallSite(shortCallSite, longCallSite) + }.getOrElse(Utils.getCallSite()) } /** 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 a9b905b0d1a63..0e90caa5c9ca7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.util.Random +import java.util.{Properties, Random} import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer @@ -41,7 +41,7 @@ import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils} @@ -1224,7 +1224,8 @@ abstract class RDD[T: ClassTag]( private var storageLevel: StorageLevel = StorageLevel.NONE /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ - @transient private[spark] val creationSite = Utils.getCallSite + @transient private[spark] val creationSite = sc.getCallSite() + private[spark] def getCreationSite: String = Option(creationSite).map(_.shortForm).getOrElse("") private[spark] def elementClassTag: ClassTag[T] = classTag[T] 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 ed063844323af..2755887feeeff 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -49,6 +49,11 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, /** CallSite represents a place in user code. It can have a short and a long form. */ private[spark] case class CallSite(shortForm: String, longForm: String) +private[spark] object CallSite { + val SHORT_FORM = "callSite.short" + val LONG_FORM = "callSite.long" +} + /** * Various utility methods used by Spark. */ @@ -859,18 +864,26 @@ private[spark] object Utils extends Logging { } } - /** - * A regular expression to match classes of the "core" Spark API that we want to skip when - * finding the call site of a method. - */ - private val SPARK_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r + /** Default filtering function for finding call sites using `getCallSite`. */ + private def coreExclusionFunction(className: String): Boolean = { + // A regular expression to match classes of the "core" Spark API that we want to skip when + // finding the call site of a method. + val SPARK_CORE_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r + val SCALA_CLASS_REGEX = """^scala""".r + val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined + val isScalaClass = SCALA_CLASS_REGEX.findFirstIn(className).isDefined + // If the class is a Spark internal class or a Scala class, then exclude. + isSparkCoreClass || isScalaClass + } /** * When called inside a class in the spark package, returns the name of the user code class * (outside the spark package) that called into Spark, as well as which Spark method they called. * This is used, for example, to tell users where in their code each RDD got created. + * + * @param skipClass Function that is used to exclude non-user-code classes. */ - def getCallSite: CallSite = { + def getCallSite(skipClass: String => Boolean = coreExclusionFunction): CallSite = { val trace = Thread.currentThread.getStackTrace() .filterNot { ste:StackTraceElement => // When running under some profilers, the current stack trace might contain some bogus @@ -891,7 +904,7 @@ private[spark] object Utils extends Logging { for (el <- trace) { if (insideSpark) { - if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName).isDefined) { + if (skipClass(el.getClassName)) { lastSparkMethod = if (el.getMethodName == "") { // Spark method is a constructor; get its class name el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index f63560dcb5b89..5a8eef1372e23 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -35,10 +35,9 @@ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver} +import org.apache.spark.streaming.receiver.{ActorReceiver, ActorSupervisorStrategy, Receiver} import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab} -import org.apache.spark.util.MetadataCleaner /** * Main entry point for Spark Streaming functionality. It provides methods used to create @@ -448,6 +447,7 @@ class StreamingContext private[streaming] ( throw new SparkException("StreamingContext has already been stopped") } validate() + sparkContext.setCallSite(DStream.getCreationSite()) scheduler.start() state = Started } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index e05db236addca..65f7ccd318684 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -23,6 +23,7 @@ import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import scala.deprecated import scala.collection.mutable.HashMap import scala.reflect.ClassTag +import scala.util.matching.Regex import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.{BlockRDD, RDD} @@ -30,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.util.MetadataCleaner +import org.apache.spark.util.{CallSite, MetadataCleaner} /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -106,6 +107,9 @@ abstract class DStream[T: ClassTag] ( /** Return the StreamingContext associated with this DStream */ def context = ssc + /* Set the creation call site */ + private[streaming] val creationSite = DStream.getCreationSite() + /** Persist the RDDs of this DStream with the given storage level */ def persist(level: StorageLevel): DStream[T] = { if (this.isInitialized) { @@ -272,43 +276,41 @@ abstract class DStream[T: ClassTag] ( } /** - * Retrieve a precomputed RDD of this DStream, or computes the RDD. This is an internal - * method that should not be called directly. + * Get the RDD corresponding to the given time; either retrieve it from cache + * or compute-and-cache it. */ private[streaming] def getOrCompute(time: Time): Option[RDD[T]] = { - // If this DStream was not initialized (i.e., zeroTime not set), then do it - // If RDD was already generated, then retrieve it from HashMap - generatedRDDs.get(time) match { - - // If an RDD was already generated and is being reused, then - // probably all RDDs in this DStream will be reused and hence should be cached - case Some(oldRDD) => Some(oldRDD) - - // if RDD was not generated, and if the time is valid - // (based on sliding time of this DStream), then generate the RDD - case None => { - if (isTimeValid(time)) { - compute(time) match { - case Some(newRDD) => - if (storageLevel != StorageLevel.NONE) { - newRDD.persist(storageLevel) - logInfo("Persisting RDD " + newRDD.id + " for time " + - time + " to " + storageLevel + " at time " + time) - } - if (checkpointDuration != null && - (time - zeroTime).isMultipleOf(checkpointDuration)) { - newRDD.checkpoint() - logInfo("Marking RDD " + newRDD.id + " for time " + time + - " for checkpointing at time " + time) - } - generatedRDDs.put(time, newRDD) - Some(newRDD) - case None => - None + // If RDD was already generated, then retrieve it from HashMap, + // or else compute the RDD + generatedRDDs.get(time).orElse { + // Compute the RDD if time is valid (e.g. correct time in a sliding window) + // of RDD generation, else generate nothing. + if (isTimeValid(time)) { + // Set the thread-local property for call sites to this DStream's creation site + // such that RDDs generated by compute gets that as their creation site. + // Note that this `getOrCompute` may get called from another DStream which may have + // set its own call site. So we store its call site in a temporary variable, + // set this DStream's creation site, generate RDDs and then restore the previous call site. + val prevCallSite = ssc.sparkContext.getCallSite() + ssc.sparkContext.setCallSite(creationSite) + val rddOption = compute(time) + ssc.sparkContext.setCallSite(prevCallSite) + + rddOption.foreach { case newRDD => + // Register the generated RDD for caching and checkpointing + if (storageLevel != StorageLevel.NONE) { + newRDD.persist(storageLevel) + logDebug(s"Persisting RDD ${newRDD.id} for time $time to $storageLevel") } - } else { - None + if (checkpointDuration != null && (time - zeroTime).isMultipleOf(checkpointDuration)) { + newRDD.checkpoint() + logInfo(s"Marking RDD ${newRDD.id} for time $time for checkpointing") + } + generatedRDDs.put(time, newRDD) } + rddOption + } else { + None } } } @@ -799,3 +801,29 @@ abstract class DStream[T: ClassTag] ( this } } + +private[streaming] object DStream { + + /** Get the creation site of a DStream from the stack trace of when the DStream is created. */ + def getCreationSite(): CallSite = { + val SPARK_CLASS_REGEX = """^org\.apache\.spark""".r + val SPARK_STREAMING_TESTCLASS_REGEX = """^org\.apache\.spark\.streaming\.test""".r + val SPARK_EXAMPLES_CLASS_REGEX = """^org\.apache\.spark\.examples""".r + val SCALA_CLASS_REGEX = """^scala""".r + + /** Filtering function that excludes non-user classes for a streaming application */ + def streamingExclustionFunction(className: String): Boolean = { + def doesMatch(r: Regex) = r.findFirstIn(className).isDefined + val isSparkClass = doesMatch(SPARK_CLASS_REGEX) + val isSparkExampleClass = doesMatch(SPARK_EXAMPLES_CLASS_REGEX) + val isSparkStreamingTestClass = doesMatch(SPARK_STREAMING_TESTCLASS_REGEX) + val isScalaClass = doesMatch(SCALA_CLASS_REGEX) + + // If the class is a spark example class or a streaming test class then it is considered + // as a streaming application class and don't exclude. Otherwise, exclude any + // non-Spark and non-Scala class, as the rest would streaming application classes. + (isSparkClass || isScalaClass) && !isSparkExampleClass && !isSparkStreamingTestClass + } + org.apache.spark.util.Utils.getCallSite(streamingExclustionFunction) + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index a3cabd6be02fe..ebf83748ffa28 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -19,13 +19,16 @@ package org.apache.spark.streaming import java.util.concurrent.atomic.AtomicInteger +import scala.language.postfixOps + import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.{MetadataCleaner, Utils} -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.util.Utils +import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.Eventually._ import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ @@ -257,6 +260,10 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w assert(exception.getMessage.contains("transform"), "Expected exception not thrown") } + test("DStream and generated RDD creation sites") { + testPackage.test() + } + def addInputStream(s: StreamingContext): DStream[Int] = { val input = (1 to 100).map(i => (1 to i)) val inputStream = new TestInputStream(s, input, 1) @@ -293,3 +300,37 @@ class TestReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging object TestReceiver { val counter = new AtomicInteger(1) } + +/** Streaming application for testing DStream and RDD creation sites */ +package object testPackage extends Assertions { + def test() { + val conf = new SparkConf().setMaster("local").setAppName("CreationSite test") + val ssc = new StreamingContext(conf , Milliseconds(100)) + try { + val inputStream = ssc.receiverStream(new TestReceiver) + + // Verify creation site of DStream + val creationSite = inputStream.creationSite + assert(creationSite.shortForm.contains("receiverStream") && + creationSite.shortForm.contains("StreamingContextSuite") + ) + assert(creationSite.longForm.contains("testPackage")) + + // Verify creation site of generated RDDs + var rddGenerated = false + var rddCreationSiteCorrect = true + + inputStream.foreachRDD { rdd => + rddCreationSiteCorrect = rdd.creationSite == creationSite + rddGenerated = true + } + ssc.start() + + eventually(timeout(10000 millis), interval(10 millis)) { + assert(rddGenerated && rddCreationSiteCorrect, "RDD creation site was not correct") + } + } finally { + ssc.stop() + } + } +} From c429126066f766396b706894b6942f1ca7fcb528 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Wed, 24 Sep 2014 11:33:58 -0700 Subject: [PATCH 0779/1492] [Build] Diff from branch point Sometimes Jenkins posts [spurious reports of new classes being added](https://github.com/apache/spark/pull/2339#issuecomment-56570170). I believe this stems from diffing the patch against `master`, as opposed to against `master...`, which starts from the commit the PR was branched from. This patch fixes that behavior. Author: Nicholas Chammas Closes #2512 from nchammas/diff-only-commits-ahead and squashes the following commits: c065599 [Nicholas Chammas] comment typo fix a453c67 [Nicholas Chammas] diff from branch point --- dev/run-tests-jenkins | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 06c3781eb3ccf..a6ecf3196d7d4 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -92,13 +92,13 @@ function post_message () { merge_note=" * This patch merges cleanly." source_files=$( - git diff master --name-only \ + git diff master... --name-only `# diff patch against master from branch point` \ | grep -v -e "\/test" `# ignore files in test directories` \ | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ | tr "\n" " " ) new_public_classes=$( - git diff master ${source_files} `# diff this patch against master and...` \ + git diff master... ${source_files} `# diff patch against master from branch point` \ | grep "^\+" `# filter in only added lines` \ | sed -r -e "s/^\+//g" `# remove the leading +` \ | grep -e "trait " -e "class " `# filter in lines with these key words` \ From 50f863365348d52a9285fc779efbedbf1567ea11 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 24 Sep 2014 11:34:39 -0700 Subject: [PATCH 0780/1492] [SPARK-3659] Set EC2 version to 1.1.0 and update version map This brings the master branch in sync with branch-1.1 Author: Shivaram Venkataraman Closes #2510 from shivaram/spark-ec2-version and squashes the following commits: bb0dd16 [Shivaram Venkataraman] Set EC2 version to 1.1.0 and update version map --- ec2/spark_ec2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index fbeccd89b43b3..7f2cd7d94de39 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -38,7 +38,7 @@ from boto.ec2.blockdevicemapping import BlockDeviceMapping, BlockDeviceType, EBSBlockDeviceType from boto import ec2 -DEFAULT_SPARK_VERSION = "1.0.0" +DEFAULT_SPARK_VERSION = "1.1.0" # A URL prefix from which to fetch AMI information AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" @@ -218,7 +218,7 @@ def is_active(instance): def get_spark_shark_version(opts): spark_shark_map = { "0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1", - "1.0.0": "1.0.0" + "1.0.0": "1.0.0", "1.0.1": "1.0.1", "1.0.2": "1.0.2", "1.1.0": "1.1.0" } version = opts.spark_version.replace("v", "") if version not in spark_shark_map: From c854b9fcb5595b1d70b6ce257fc7574602ac5e49 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 24 Sep 2014 12:10:09 -0700 Subject: [PATCH 0781/1492] [SPARK-3634] [PySpark] User's module should take precedence over system modules Python modules added through addPyFile should take precedence over system modules. This patch put the path for user added module in the front of sys.path (just after ''). Author: Davies Liu Closes #2492 from davies/path and squashes the following commits: 4a2af78 [Davies Liu] fix tests f7ff4da [Davies Liu] ad license header 6b0002f [Davies Liu] add tests c16c392 [Davies Liu] put addPyFile in front of sys.path --- python/pyspark/context.py | 11 +++++------ python/pyspark/tests.py | 12 ++++++++++++ python/pyspark/worker.py | 11 +++++++++-- python/test_support/SimpleHTTPServer.py | 22 ++++++++++++++++++++++ 4 files changed, 48 insertions(+), 8 deletions(-) create mode 100644 python/test_support/SimpleHTTPServer.py diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 064a24bff539c..8e7b00469e246 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -171,7 +171,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, SparkFiles._sc = self root_dir = SparkFiles.getRootDirectory() - sys.path.append(root_dir) + sys.path.insert(1, root_dir) # Deploy any code dependencies specified in the constructor self._python_includes = list() @@ -183,10 +183,9 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, for path in self._conf.get("spark.submit.pyFiles", "").split(","): if path != "": (dirname, filename) = os.path.split(path) - self._python_includes.append(filename) - sys.path.append(path) - if dirname not in sys.path: - sys.path.append(dirname) + if filename.lower().endswith("zip") or filename.lower().endswith("egg"): + self._python_includes.append(filename) + sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) # Create a temporary directory inside spark.local.dir: local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf()) @@ -667,7 +666,7 @@ def addPyFile(self, path): if filename.endswith('.zip') or filename.endswith('.ZIP') or filename.endswith('.egg'): self._python_includes.append(filename) # for tests in local mode - sys.path.append(os.path.join(SparkFiles.getRootDirectory(), filename)) + sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) def setCheckpointDir(self, dirName): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1b8afb763b26a..4483bf80dbe06 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -323,6 +323,18 @@ def func(): from userlib import UserClass self.assertEqual("Hello World from inside a package!", UserClass().hello()) + def test_overwrite_system_module(self): + self.sc.addPyFile(os.path.join(SPARK_HOME, "python/test_support/SimpleHTTPServer.py")) + + import SimpleHTTPServer + self.assertEqual("My Server", SimpleHTTPServer.__name__) + + def func(x): + import SimpleHTTPServer + return SimpleHTTPServer.__name__ + + self.assertEqual(["My Server"], self.sc.parallelize(range(1)).map(func).collect()) + class TestRDDFunctions(PySparkTestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index d6c06e2dbef62..c1f6e3e4a1f40 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -43,6 +43,13 @@ def report_times(outfile, boot, init, finish): write_long(1000 * finish, outfile) +def add_path(path): + # worker can be used, so donot add path multiple times + if path not in sys.path: + # overwrite system packages + sys.path.insert(1, path) + + def main(infile, outfile): try: boot_time = time.time() @@ -61,11 +68,11 @@ def main(infile, outfile): SparkFiles._is_running_on_worker = True # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH - sys.path.append(spark_files_dir) # *.py files that were added will be copied here + add_path(spark_files_dir) # *.py files that were added will be copied here num_python_includes = read_int(infile) for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) - sys.path.append(os.path.join(spark_files_dir, filename)) + add_path(os.path.join(spark_files_dir, filename)) # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) diff --git a/python/test_support/SimpleHTTPServer.py b/python/test_support/SimpleHTTPServer.py new file mode 100644 index 0000000000000..eddbd588e02dc --- /dev/null +++ b/python/test_support/SimpleHTTPServer.py @@ -0,0 +1,22 @@ +# +# 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. +# + +""" +Used to test override standard SimpleHTTPServer module. +""" + +__name__ = "My Server" From bb96012b7360b099a19fecc80f0209b30f118ada Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 24 Sep 2014 13:00:05 -0700 Subject: [PATCH 0782/1492] [SPARK-3679] [PySpark] pickle the exact globals of functions function.func_code.co_names has all the names used in the function, including name of attributes. It will pickle some unnecessary globals if there is a global having the same name with attribute (in co_names). There is a regression introduced by #2144, revert part of changes in that PR. cc JoshRosen Author: Davies Liu Closes #2522 from davies/globals and squashes the following commits: dfbccf5 [Davies Liu] fix bug while pickle globals of function --- python/pyspark/cloudpickle.py | 42 ++++++++++++++++++++++++++++++----- python/pyspark/tests.py | 18 +++++++++++++++ 2 files changed, 54 insertions(+), 6 deletions(-) diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 32dda3888c62d..bb0783555aa77 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -52,6 +52,7 @@ import itertools from copy_reg import _extension_registry, _inverted_registry, _extension_cache import new +import dis import traceback import platform @@ -61,6 +62,14 @@ import logging cloudLog = logging.getLogger("Cloud.Transport") +#relevant opcodes +STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) +DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) +LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) +GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] + +HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) +EXTENDED_ARG = chr(dis.EXTENDED_ARG) if PyImp == "PyPy": # register builtin type in `new` @@ -304,16 +313,37 @@ def save_function_tuple(self, func, forced_imports): write(pickle.REDUCE) # applies _fill_function on the tuple @staticmethod - def extract_code_globals(code): + def extract_code_globals(co): """ Find all globals names read or written to by codeblock co """ - names = set(code.co_names) - if code.co_consts: # see if nested function have any global refs - for const in code.co_consts: + code = co.co_code + names = co.co_names + out_names = set() + + n = len(code) + i = 0 + extended_arg = 0 + while i < n: + op = code[i] + + i = i+1 + if op >= HAVE_ARGUMENT: + oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg + extended_arg = 0 + i = i+2 + if op == EXTENDED_ARG: + extended_arg = oparg*65536L + if op in GLOBAL_OPS: + out_names.add(names[oparg]) + #print 'extracted', out_names, ' from ', names + + if co.co_consts: # see if nested function have any global refs + for const in co.co_consts: if type(const) is types.CodeType: - names |= CloudPickler.extract_code_globals(const) - return names + out_names |= CloudPickler.extract_code_globals(const) + + return out_names def extract_func_data(self, func): """ diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 4483bf80dbe06..d1bb2033b7a16 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -213,6 +213,24 @@ def test_pickling_file_handles(self): out2 = ser.loads(ser.dumps(out1)) self.assertEquals(out1, out2) + def test_func_globals(self): + + class Unpicklable(object): + def __reduce__(self): + raise Exception("not picklable") + + global exit + exit = Unpicklable() + + ser = CloudPickleSerializer() + self.assertRaises(Exception, lambda: ser.dumps(exit)) + + def foo(): + sys.exit(0) + + self.assertTrue("exit" in foo.func_code.co_names) + ser.dumps(foo) + class PySparkTestCase(unittest.TestCase): From 74fb2ecf7afc2d314f6477f8f2e6134614387453 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Wed, 24 Sep 2014 17:18:55 -0700 Subject: [PATCH 0783/1492] [SPARK-3615][Streaming]Fix Kafka unit test hard coded Zookeeper port issue Details can be seen in [SPARK-3615](https://issues.apache.org/jira/browse/SPARK-3615). Author: jerryshao Closes #2483 from jerryshao/SPARK_3615 and squashes the following commits: 8555563 [jerryshao] Fix Kafka unit test hard coded Zookeeper port issue --- .../streaming/kafka/JavaKafkaStreamSuite.java | 2 +- .../streaming/kafka/KafkaStreamSuite.scala | 46 +++++++++++++------ 2 files changed, 34 insertions(+), 14 deletions(-) diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 0571454c01dae..efb0099c7c850 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -81,7 +81,7 @@ public void testKafkaStream() throws InterruptedException { Predef.>conforms())); HashMap kafkaParams = new HashMap(); - kafkaParams.put("zookeeper.connect", testSuite.zkConnect()); + kafkaParams.put("zookeeper.connect", testSuite.zkHost() + ":" + testSuite.zkPort()); kafkaParams.put("group.id", "test-consumer-" + KafkaTestUtils.random().nextInt(10000)); kafkaParams.put("auto.offset.reset", "smallest"); diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index c0b55e9340253..6943326eb750e 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -24,7 +24,7 @@ import java.util.{Properties, Random} import scala.collection.mutable import kafka.admin.CreateTopicCommand -import kafka.common.TopicAndPartition +import kafka.common.{KafkaException, TopicAndPartition} import kafka.producer.{KeyedMessage, ProducerConfig, Producer} import kafka.utils.ZKStringSerializer import kafka.serializer.{StringDecoder, StringEncoder} @@ -42,14 +42,13 @@ import org.apache.spark.util.Utils class KafkaStreamSuite extends TestSuiteBase { import KafkaTestUtils._ - val zkConnect = "localhost:2181" + val zkHost = "localhost" + var zkPort: Int = 0 val zkConnectionTimeout = 6000 val zkSessionTimeout = 6000 - val brokerPort = 9092 - val brokerProps = getBrokerConfig(brokerPort, zkConnect) - val brokerConf = new KafkaConfig(brokerProps) - + protected var brokerPort = 9092 + protected var brokerConf: KafkaConfig = _ protected var zookeeper: EmbeddedZookeeper = _ protected var zkClient: ZkClient = _ protected var server: KafkaServer = _ @@ -59,16 +58,35 @@ class KafkaStreamSuite extends TestSuiteBase { override def beforeFunction() { // Zookeeper server startup - zookeeper = new EmbeddedZookeeper(zkConnect) + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort logInfo("==================== 0 ====================") - zkClient = new ZkClient(zkConnect, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) + + zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + ZKStringSerializer) logInfo("==================== 1 ====================") // Kafka broker startup - server = new KafkaServer(brokerConf) - logInfo("==================== 2 ====================") - server.startup() - logInfo("==================== 3 ====================") + var bindSuccess: Boolean = false + while(!bindSuccess) { + try { + val brokerProps = getBrokerConfig(brokerPort, s"$zkHost:$zkPort") + brokerConf = new KafkaConfig(brokerProps) + server = new KafkaServer(brokerConf) + logInfo("==================== 2 ====================") + server.startup() + logInfo("==================== 3 ====================") + bindSuccess = true + } catch { + case e: KafkaException => + if (e.getMessage != null && e.getMessage.contains("Socket server failed to bind to")) { + brokerPort += 1 + } + case e: Exception => throw new Exception("Kafka server create failed", e) + } + } + Thread.sleep(2000) logInfo("==================== 4 ====================") super.beforeFunction() @@ -92,7 +110,7 @@ class KafkaStreamSuite extends TestSuiteBase { createTopic(topic) produceAndSendMessage(topic, sent) - val kafkaParams = Map("zookeeper.connect" -> zkConnect, + val kafkaParams = Map("zookeeper.connect" -> s"$zkHost:$zkPort", "group.id" -> s"test-consumer-${random.nextInt(10000)}", "auto.offset.reset" -> "smallest") @@ -200,6 +218,8 @@ object KafkaTestUtils { factory.configure(new InetSocketAddress(ip, port), 16) factory.startup(zookeeper) + val actualPort = factory.getLocalPort + def shutdown() { factory.shutdown() Utils.deleteRecursively(snapshotDir) From 8ca4ecb6a56b96bae21b33e27f6abdb53676683a Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Wed, 24 Sep 2014 20:39:09 -0700 Subject: [PATCH 0784/1492] [SPARK-546] Add full outer join to RDD and DStream. leftOuterJoin and rightOuterJoin are already implemented. This patch adds fullOuterJoin. Author: Aaron Staple Closes #1395 from staple/SPARK-546 and squashes the following commits: 1f5595c [Aaron Staple] Fix python style 7ac0aa9 [Aaron Staple] [SPARK-546] Add full outer join to RDD and DStream. 3b5d137 [Aaron Staple] In JavaPairDStream, make class tag specification in rightOuterJoin consistent with other functions. 31f2956 [Aaron Staple] Fix left outer join documentation comments. --- .../apache/spark/api/java/JavaPairRDD.scala | 48 +++++++++++++++++ .../apache/spark/rdd/PairRDDFunctions.scala | 42 +++++++++++++++ .../org/apache/spark/PartitioningSuite.scala | 3 ++ .../spark/rdd/PairRDDFunctionsSuite.scala | 15 ++++++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 1 + docs/programming-guide.md | 2 +- python/pyspark/join.py | 16 ++++++ python/pyspark/rdd.py | 25 ++++++++- .../streaming/api/java/JavaPairDStream.scala | 54 +++++++++++++++++-- .../dstream/PairDStreamFunctions.scala | 36 +++++++++++++ .../streaming/BasicOperationsSuite.scala | 15 ++++++ 11 files changed, 250 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 880f61c49726e..0846225e4f992 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -469,6 +469,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Uses the given Partitioner to partition the output RDD. + */ + def fullOuterJoin[W](other: JavaPairRDD[K, W], partitioner: Partitioner) + : JavaPairRDD[K, (Optional[V], Optional[W])] = { + val joinResult = rdd.fullOuterJoin(other, partitioner) + fromRDD(joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + }) + } + /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the existing * partitioner/parallelism level. @@ -563,6 +579,38 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) fromRDD(joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)}) } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/ + * parallelism level. + */ + def fullOuterJoin[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (Optional[V], Optional[W])] = { + val joinResult = rdd.fullOuterJoin(other) + fromRDD(joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + }) + } + + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions. + */ + def fullOuterJoin[W](other: JavaPairRDD[K, W], numPartitions: Int) + : JavaPairRDD[K, (Optional[V], Optional[W])] = { + val joinResult = rdd.fullOuterJoin(other, numPartitions) + fromRDD(joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + }) + } + /** * Return the key-value pairs in this RDD to the master as a Map. */ 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 51ba8c2d17834..7f578bc5dac39 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -506,6 +506,23 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Uses the given Partitioner to partition the output RDD. + */ + def fullOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Option[V], Option[W]))] = { + this.cogroup(other, partitioner).flatMapValues { + case (vs, Seq()) => vs.map(v => (Some(v), None)) + case (Seq(), ws) => ws.map(w => (None, Some(w))) + case (vs, ws) => for (v <- vs; w <- ws) yield (Some(v), Some(w)) + } + } + /** * Simplified version of combineByKey that hash-partitions the resulting RDD using the * existing partitioner/parallelism level. @@ -585,6 +602,31 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) rightOuterJoin(other, new HashPartitioner(numPartitions)) } + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD using the existing partitioner/ + * parallelism level. + */ + def fullOuterJoin[W](other: RDD[(K, W)]): RDD[(K, (Option[V], Option[W]))] = { + fullOuterJoin(other, defaultPartitioner(self, other)) + } + + /** + * Perform a full outer join of `this` and `other`. For each element (k, v) in `this`, the + * resulting RDD will either contain all pairs (k, (Some(v), Some(w))) for w in `other`, or + * the pair (k, (Some(v), None)) if no elements in `other` have key k. Similarly, for each + * element (k, w) in `other`, the resulting RDD will either contain all pairs + * (k, (Some(v), Some(w))) for v in `this`, or the pair (k, (None, Some(w))) if no elements + * in `this` have key k. Hash-partitions the resulting RDD into the given number of partitions. + */ + def fullOuterJoin[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Option[V], Option[W]))] = { + fullOuterJoin(other, new HashPartitioner(numPartitions)) + } + /** * Return the key-value pairs in this RDD to the master as a Map. * diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index fc0cee3e8749d..646ede30ae6ff 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -193,11 +193,13 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(grouped2.join(grouped4).partitioner === grouped4.partitioner) assert(grouped2.leftOuterJoin(grouped4).partitioner === grouped4.partitioner) assert(grouped2.rightOuterJoin(grouped4).partitioner === grouped4.partitioner) + assert(grouped2.fullOuterJoin(grouped4).partitioner === grouped4.partitioner) assert(grouped2.cogroup(grouped4).partitioner === grouped4.partitioner) assert(grouped2.join(reduced2).partitioner === grouped2.partitioner) assert(grouped2.leftOuterJoin(reduced2).partitioner === grouped2.partitioner) assert(grouped2.rightOuterJoin(reduced2).partitioner === grouped2.partitioner) + assert(grouped2.fullOuterJoin(reduced2).partitioner === grouped2.partitioner) assert(grouped2.cogroup(reduced2).partitioner === grouped2.partitioner) assert(grouped2.map(_ => 1).partitioner === None) @@ -218,6 +220,7 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(intercept[SparkException]{ arrPairs.join(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.leftOuterJoin(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.rightOuterJoin(arrPairs) }.getMessage.contains("array")) + assert(intercept[SparkException]{ arrPairs.fullOuterJoin(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.groupByKey() }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.countByKey() }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.countByKeyApprox(1) }.getMessage.contains("array")) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index e84cc69592339..75b01191901b8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -298,6 +298,21 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { )) } + test("fullOuterJoin") { + val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) + val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) + val joined = rdd1.fullOuterJoin(rdd2).collect() + assert(joined.size === 6) + assert(joined.toSet === Set( + (1, (Some(1), Some('x'))), + (1, (Some(2), Some('x'))), + (2, (Some(1), Some('y'))), + (2, (Some(1), Some('z'))), + (3, (Some(1), None)), + (4, (None, Some('w'))) + )) + } + test("join with no matches") { val rdd1 = sc.parallelize(Array((1, 1), (1, 2), (2, 1), (3, 1))) val rdd2 = sc.parallelize(Array((4, 'x'), (5, 'y'), (5, 'z'), (6, 'w'))) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index c1b501a75c8b8..465c1a8a43a79 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -193,6 +193,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(rdd.join(emptyKv).collect().size === 0) assert(rdd.rightOuterJoin(emptyKv).collect().size === 0) assert(rdd.leftOuterJoin(emptyKv).collect().size === 2) + assert(rdd.fullOuterJoin(emptyKv).collect().size === 2) assert(rdd.cogroup(emptyKv).collect().size === 2) assert(rdd.union(emptyKv).collect().size === 2) } diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 01d378af574b5..510b47a2aaad1 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -906,7 +906,7 @@ for details. diff --git a/python/pyspark/join.py b/python/pyspark/join.py index b0f1cc1927066..b4a844713745a 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -80,6 +80,22 @@ def dispatch(seq): return _do_python_join(rdd, other, numPartitions, dispatch) +def python_full_outer_join(rdd, other, numPartitions): + def dispatch(seq): + vbuf, wbuf = [], [] + for (n, v) in seq: + if n == 1: + vbuf.append(v) + elif n == 2: + wbuf.append(v) + if not vbuf: + vbuf.append(None) + if not wbuf: + wbuf.append(None) + return [(v, w) for v in vbuf for w in wbuf] + return _do_python_join(rdd, other, numPartitions, dispatch) + + def python_cogroup(rdds, numPartitions): def make_mapper(i): return lambda (k, v): (k, (i, v)) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8ef233bc80c5c..680140d72d03c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -36,7 +36,7 @@ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer from pyspark.join import python_join, python_left_outer_join, \ - python_right_outer_join, python_cogroup + python_right_outer_join, python_full_outer_join, python_cogroup from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler, RDDStratifiedSampler from pyspark.storagelevel import StorageLevel @@ -1375,7 +1375,7 @@ def leftOuterJoin(self, other, numPartitions=None): For each element (k, v) in C{self}, the resulting RDD will either contain all pairs (k, (v, w)) for w in C{other}, or the pair - (k, (v, None)) if no elements in other have key k. + (k, (v, None)) if no elements in C{other} have key k. Hash-partitions the resulting RDD into the given number of partitions. @@ -1403,6 +1403,27 @@ def rightOuterJoin(self, other, numPartitions=None): """ return python_right_outer_join(self, other, numPartitions) + def fullOuterJoin(self, other, numPartitions=None): + """ + Perform a right outer join of C{self} and C{other}. + + For each element (k, v) in C{self}, the resulting RDD will either + contain all pairs (k, (v, w)) for w in C{other}, or the pair + (k, (v, None)) if no elements in C{other} have key k. + + Similarly, for each element (k, w) in C{other}, the resulting RDD will + either contain all pairs (k, (v, w)) for v in C{self}, or the pair + (k, (None, w)) if no elements in C{self} have key k. + + Hash-partitions the resulting RDD into the given number of partitions. + + >>> x = sc.parallelize([("a", 1), ("b", 4)]) + >>> y = sc.parallelize([("a", 2), ("c", 8)]) + >>> sorted(x.fullOuterJoin(y).collect()) + [('a', (1, 2)), ('b', (4, None)), ('c', (None, 8))] + """ + return python_full_outer_join(self, other, numPartitions) + # TODO: add option to control map-side combining # portable_hash is used as default, because builtin hash of None is different # cross machines. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index c00e11d11910f..59d4423086ef0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -606,8 +606,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( } /** - * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. + * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. */ def leftOuterJoin[W]( other: JavaPairDStream[K, W], @@ -624,8 +625,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * number of partitions. */ def rightOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (Optional[V], W)] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag val joinResult = dstream.rightOuterJoin(other.dstream) joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)} } @@ -658,6 +658,52 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)} } + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default + * number of partitions. + */ + def fullOuterJoin[W](other: JavaPairDStream[K, W]) + : JavaPairDStream[K, (Optional[V], Optional[W])] = { + implicit val cm: ClassTag[W] = fakeClassTag + val joinResult = dstream.fullOuterJoin(other.dstream) + joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + } + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ + def fullOuterJoin[W]( + other: JavaPairDStream[K, W], + numPartitions: Int + ): JavaPairDStream[K, (Optional[V], Optional[W])] = { + implicit val cm: ClassTag[W] = fakeClassTag + val joinResult = dstream.fullOuterJoin(other.dstream, numPartitions) + joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + } + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ + def fullOuterJoin[W]( + other: JavaPairDStream[K, W], + partitioner: Partitioner + ): JavaPairDStream[K, (Optional[V], Optional[W])] = { + implicit val cm: ClassTag[W] = fakeClassTag + val joinResult = dstream.fullOuterJoin(other.dstream, partitioner) + joinResult.mapValues{ case (v, w) => + (JavaUtils.optionToOptional(v), JavaUtils.optionToOptional(w)) + } + } + /** * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval is * generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix". diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 826bf39e860e1..9467595d307a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -568,6 +568,42 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) ) } + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with Spark's default + * number of partitions. + */ + def fullOuterJoin[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Option[V], Option[W]))] = { + fullOuterJoin[W](other, defaultPartitioner()) + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. Hash partitioning is used to generate the RDDs with `numPartitions` + * partitions. + */ + def fullOuterJoin[W: ClassTag]( + other: DStream[(K, W)], + numPartitions: Int + ): DStream[(K, (Option[V], Option[W]))] = { + fullOuterJoin[W](other, defaultPartitioner(numPartitions)) + } + + /** + * Return a new DStream by applying 'full outer join' between RDDs of `this` DStream and + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control + * the partitioning of each RDD. + */ + def fullOuterJoin[W: ClassTag]( + other: DStream[(K, W)], + partitioner: Partitioner + ): DStream[(K, (Option[V], Option[W]))] = { + self.transformWith( + other, + (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.fullOuterJoin(rdd2, partitioner) + ) + } + /** * Save each RDD in `this` DStream as a Hadoop file. The file name at each batch interval * is generated based on `prefix` and `suffix`: "prefix-TIME_IN_MS.suffix" diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 059ac6c2dbee2..6c8bb50145367 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -303,6 +303,21 @@ class BasicOperationsSuite extends TestSuiteBase { testOperation(inputData1, inputData2, operation, outputData, true) } + test("fullOuterJoin") { + val inputData1 = Seq( Seq("a", "b"), Seq("a", ""), Seq(""), Seq() ) + val inputData2 = Seq( Seq("a", "b"), Seq("b", ""), Seq(), Seq("") ) + val outputData = Seq( + Seq( ("a", (Some(1), Some("x"))), ("b", (Some(1), Some("x"))) ), + Seq( ("", (Some(1), Some("x"))), ("a", (Some(1), None)), ("b", (None, Some("x"))) ), + Seq( ("", (Some(1), None)) ), + Seq( ("", (None, Some("x"))) ) + ) + val operation = (s1: DStream[String], s2: DStream[String]) => { + s1.map(x => (x, 1)).fullOuterJoin(s2.map(x => (x, "x"))) + } + testOperation(inputData1, inputData2, operation, outputData, true) + } + test("updateStateByKey") { val inputData = Seq( From b8487713d3bf288a4f6fc149e6ee4cc8196d6e7d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 24 Sep 2014 23:10:26 -0700 Subject: [PATCH 0785/1492] [SPARK-2778] [yarn] Add yarn integration tests. This patch adds a couple of, currently, very simple integration tests to make sure both client and cluster modes are working. The tests don't do much yet other than run a simple job, but the plan is to enhance them after we get the framework in. The cluster tests are noisy, so redirect all log output to a file like other tests do. Copying the conf around sucks but it's less work than messing with maven/sbt and having to clean up other projects. Note the test is only added for yarn-stable. The code compiles against yarn-alpha but there are two issues I ran into that I could not overcome: - an old netty dependency kept creeping into the classpath and causing akka to not work, when using sbt; the old netty was correctly suppressed under maven. - MiniYARNCluster kept failing to execute containers because it did not create the NM's local dir itself; this is apparently a known behavior, but I'm not sure how to work around it. None of those issues are present with the stable Yarn. Also, these tests are a little slow to run. Apparently Spark doesn't yet tag tests (so that these could be isolated in a "slow" batch), so this is something to keep in mind. Author: Marcelo Vanzin Closes #2257 from vanzin/yarn-tests and squashes the following commits: 6d5b84e [Marcelo Vanzin] Fix wrong system property being set. 8b0933d [Marcelo Vanzin] Merge branch 'master' into yarn-tests 5c2b56f [Marcelo Vanzin] Use custom log4j conf for Yarn containers. ec73f17 [Marcelo Vanzin] More review feedback. 67f5b02 [Marcelo Vanzin] Review feedback. f01517c [Marcelo Vanzin] Review feedback. 68fbbbf [Marcelo Vanzin] Use older constructor available in older Hadoop releases. d07ef9a [Marcelo Vanzin] Merge branch 'master' into yarn-tests add8416 [Marcelo Vanzin] [SPARK-2778] [yarn] Add yarn integration tests. --- pom.xml | 31 +++- .../spark/deploy/yarn/ApplicationMaster.scala | 10 +- .../apache/spark/deploy/yarn/ClientBase.scala | 2 +- .../deploy/yarn/ExecutorRunnableUtil.scala | 2 +- yarn/pom.xml | 3 +- yarn/stable/pom.xml | 9 + .../src/test/resources/log4j.properties | 28 ++++ .../spark/deploy/yarn/YarnClusterSuite.scala | 154 ++++++++++++++++++ 8 files changed, 229 insertions(+), 10 deletions(-) create mode 100644 yarn/stable/src/test/resources/log4j.properties create mode 100644 yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala diff --git a/pom.xml b/pom.xml index 520aed3806937..f3de097b9cb32 100644 --- a/pom.xml +++ b/pom.xml @@ -712,6 +712,35 @@ + + org.apache.hadoop + hadoop-yarn-server-tests + ${yarn.version} + tests + test + + + asm + asm + + + org.ow2.asm + asm + + + org.jboss.netty + netty + + + javax.servlet + servlet-api + + + commons-logging + commons-logging + + + org.apache.hadoop hadoop-yarn-server-web-proxy @@ -1187,7 +1216,7 @@ org.apache.zookeeper zookeeper - 3.4.5-mapr-1406 + 3.4.5-mapr-1406 diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 9050808157257..b51daeb437516 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -401,17 +401,17 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. status = FinalApplicationStatus.SUCCEEDED } catch { - case e: InvocationTargetException => { + case e: InvocationTargetException => e.getCause match { - case _: InterruptedException => { + case _: InterruptedException => // Reporter thread can interrupt to stop user class - } + + case e => throw e } - } } finally { logDebug("Finishing main") + finalStatus = status } - finalStatus = status } } userClassThread.setName("Driver") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 4870b0cb3ddaf..1cf19c198509c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -348,7 +348,7 @@ private[spark] trait ClientBase extends Logging { } // For log4j configuration to reference - javaOpts += "-D=spark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) val userClass = if (args.userClass != null) { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index bbbf615510762..d7a7175d5e578 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -98,7 +98,7 @@ trait ExecutorRunnableUtil extends Logging { */ // For log4j configuration to reference - javaOpts += "-D=spark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR + javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server", diff --git a/yarn/pom.xml b/yarn/pom.xml index 815a736c2e8fd..8a7035c85e9f1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -140,7 +140,6 @@ ${basedir}/../.. - ${spark.classpath} @@ -148,7 +147,7 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - + ../common/src/main/resources diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index fd934b7726181..97eb0548e77c3 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -32,4 +32,13 @@ jar Spark Project YARN Stable API + + + org.apache.hadoop + hadoop-yarn-server-tests + tests + test + + + diff --git a/yarn/stable/src/test/resources/log4j.properties b/yarn/stable/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..26b73a1b39744 --- /dev/null +++ b/yarn/stable/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file core/target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala new file mode 100644 index 0000000000000..857a4447dd738 --- /dev/null +++ b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import java.io.File + +import scala.collection.JavaConversions._ + +import com.google.common.base.Charsets +import com.google.common.io.Files +import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} + +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.server.MiniYARNCluster + +import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.util.Utils + +class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { + + // log4j configuration for the Yarn containers, so that their output is collected + // by Yarn instead of trying to overwrite unit-tests.log. + private val LOG4J_CONF = """ + |log4j.rootCategory=DEBUG, console + |log4j.appender.console=org.apache.log4j.ConsoleAppender + |log4j.appender.console.target=System.err + |log4j.appender.console.layout=org.apache.log4j.PatternLayout + |log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n + """.stripMargin + + private var yarnCluster: MiniYARNCluster = _ + private var tempDir: File = _ + private var fakeSparkJar: File = _ + private var oldConf: Map[String, String] = _ + + override def beforeAll() { + tempDir = Utils.createTempDir() + + val logConfDir = new File(tempDir, "log4j") + logConfDir.mkdir() + + val logConfFile = new File(logConfDir, "log4j.properties") + Files.write(LOG4J_CONF, logConfFile, Charsets.UTF_8) + + val childClasspath = logConfDir.getAbsolutePath() + File.pathSeparator + + sys.props("java.class.path") + + oldConf = sys.props.filter { case (k, v) => k.startsWith("spark.") }.toMap + + yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) + yarnCluster.init(new YarnConfiguration()) + yarnCluster.start() + yarnCluster.getConfig().foreach { e => + sys.props += ("spark.hadoop." + e.getKey() -> e.getValue()) + } + + fakeSparkJar = File.createTempFile("sparkJar", null, tempDir) + sys.props += ("spark.yarn.jar" -> ("local:" + fakeSparkJar.getAbsolutePath())) + sys.props += ("spark.executor.instances" -> "1") + sys.props += ("spark.driver.extraClassPath" -> childClasspath) + sys.props += ("spark.executor.extraClassPath" -> childClasspath) + + super.beforeAll() + } + + override def afterAll() { + yarnCluster.stop() + sys.props.retain { case (k, v) => !k.startsWith("spark.") } + sys.props ++= oldConf + super.afterAll() + } + + test("run Spark in yarn-client mode") { + var result = File.createTempFile("result", null, tempDir) + YarnClusterDriver.main(Array("yarn-client", result.getAbsolutePath())) + checkResult(result) + } + + test("run Spark in yarn-cluster mode") { + val main = YarnClusterDriver.getClass.getName().stripSuffix("$") + var result = File.createTempFile("result", null, tempDir) + + // The Client object will call System.exit() after the job is done, and we don't want + // that because it messes up the scalatest monitoring. So replicate some of what main() + // does here. + val args = Array("--class", main, + "--jar", "file:" + fakeSparkJar.getAbsolutePath(), + "--arg", "yarn-cluster", + "--arg", result.getAbsolutePath(), + "--num-executors", "1") + val sparkConf = new SparkConf() + val yarnConf = SparkHadoopUtil.get.newConfiguration(sparkConf) + val clientArgs = new ClientArguments(args, sparkConf) + new Client(clientArgs, yarnConf, sparkConf).run() + checkResult(result) + } + + /** + * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide + * any sort of error when the job process finishes successfully, but the job itself fails. So + * the tests enforce that something is written to a file after everything is ok to indicate + * that the job succeeded. + */ + private def checkResult(result: File) = { + var resultString = Files.toString(result, Charsets.UTF_8) + resultString should be ("success") + } + +} + +private object YarnClusterDriver extends Logging with Matchers { + + def main(args: Array[String]) = { + if (args.length != 2) { + System.err.println( + s""" + |Invalid command line: ${args.mkString(" ")} + | + |Usage: YarnClusterDriver [master] [result file] + """.stripMargin) + System.exit(1) + } + + val sc = new SparkContext(new SparkConf().setMaster(args(0)) + .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) + val status = new File(args(1)) + var result = "failure" + try { + val data = sc.parallelize(1 to 4, 4).collect().toSet + data should be (Set(1, 2, 3, 4)) + result = "success" + } finally { + sc.stop() + Files.write(result, status, Charsets.UTF_8) + } + } + +} From c3f2a8588e19aab814ac5cdd86575bb5558d5e46 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 25 Sep 2014 23:20:17 +0530 Subject: [PATCH 0786/1492] SPARK-2932 [STREAMING] Move MasterFailureTest out of "main" source directory (HT @vanzin) Whatever the reason was for having this test class in `main`, if there is one, appear to be moot. This may have been a result of earlier streaming test reorganization. This simply puts `MasterFailureTest` back under `test/`, removes some redundant copied code, and touches up a few tiny inspection warnings along the way. Author: Sean Owen Closes #2399 from srowen/SPARK-2932 and squashes the following commits: 3909411 [Sean Owen] Move MasterFailureTest to src/test, and remove redundant TestOutputStream --- .../apache/spark/streaming/FailureSuite.scala | 1 - .../spark/streaming}/MasterFailureTest.scala | 43 ++++--------------- 2 files changed, 8 insertions(+), 36 deletions(-) rename streaming/src/{main/scala/org/apache/spark/streaming/util => test/scala/org/apache/spark/streaming}/MasterFailureTest.scala (91%) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index 92e1b76d28301..40434b1f9b709 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.streaming import org.apache.spark.Logging -import org.apache.spark.streaming.util.MasterFailureTest import org.apache.spark.util.Utils import java.io.File diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala similarity index 91% rename from streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala rename to streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index 98e17ff92e205..c53c01706083a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -15,20 +15,18 @@ * limitations under the License. */ -package org.apache.spark.streaming.util +package org.apache.spark.streaming import org.apache.spark.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} +import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -import StreamingContext._ +import org.apache.spark.streaming.StreamingContext._ import scala.util.Random -import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import java.io.{File, ObjectInputStream, IOException} +import java.io.{File, IOException} import java.nio.charset.Charset import java.util.UUID @@ -91,7 +89,7 @@ object MasterFailureTest extends Logging { // Input: time=1 ==> [ a ] , time=2 ==> [ a, a ] , time=3 ==> [ a, a, a ] , ... val input = (1 to numBatches).map(i => (1 to i).map(_ => "a").mkString(" ")).toSeq // Expected output: time=1 ==> [ (a, 1) ] , time=2 ==> [ (a, 3) ] , time=3 ==> [ (a,6) ] , ... - val expectedOutput = (1L to numBatches).map(i => (1L to i).reduce(_ + _)).map(j => ("a", j)) + val expectedOutput = (1L to numBatches).map(i => (1L to i).sum).map(j => ("a", j)) val operation = (st: DStream[String]) => { val updateFunc = (values: Seq[Long], state: Option[Long]) => { @@ -218,7 +216,7 @@ object MasterFailureTest extends Logging { while(!isLastOutputGenerated && !isTimedOut) { // Get the output buffer - val outputBuffer = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[T]].output + val outputBuffer = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[T]].output def output = outputBuffer.flatMap(x => x) // Start the thread to kill the streaming after some time @@ -239,7 +237,7 @@ object MasterFailureTest extends Logging { while (!killed && !isLastOutputGenerated && !isTimedOut) { Thread.sleep(100) timeRan = System.currentTimeMillis() - startTime - isLastOutputGenerated = (!output.isEmpty && output.last == lastExpectedOutput) + isLastOutputGenerated = (output.nonEmpty && output.last == lastExpectedOutput) isTimedOut = (timeRan + totalTimeRan > maxTimeToRun) } } catch { @@ -313,31 +311,6 @@ object MasterFailureTest extends Logging { } } -/** - * This is a output stream just for testing. All the output is collected into a - * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. - */ -private[streaming] -class TestOutputStream[T: ClassTag]( - parent: DStream[T], - val output: ArrayBuffer[Seq[T]] = new ArrayBuffer[Seq[T]] with SynchronizedBuffer[Seq[T]] - ) extends ForEachDStream[T]( - parent, - (rdd: RDD[T], t: Time) => { - val collected = rdd.collect() - output += collected - } - ) { - - // This is to clear the output buffer every it is read from a checkpoint - @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { - ois.defaultReadObject() - output.clear() - } -} - - /** * Thread to kill streaming context after a random period of time. */ From 9b56e249e09d8da20f703b9381c5c3c8a1a1d4a9 Mon Sep 17 00:00:00 2001 From: epahomov Date: Thu, 25 Sep 2014 14:50:12 -0700 Subject: [PATCH 0787/1492] [SPARK-3690] Closing shuffle writers we swallow more important exception Author: epahomov Closes #2537 from epahomov/SPARK-3690 and squashes the following commits: a0b7de4 [epahomov] [SPARK-3690] Closing shuffle writers we swallow more important exception --- .../org/apache/spark/scheduler/ShuffleMapTask.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 381eff2147e95..a98ee118254a3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -69,8 +69,13 @@ private[spark] class ShuffleMapTask( return writer.stop(success = true).get } catch { case e: Exception => - if (writer != null) { - writer.stop(success = false) + try { + if (writer != null) { + writer.stop(success = false) + } + } catch { + case e: Exception => + log.debug("Could not stop writer", e) } throw e } finally { From ff637c9380a6342fd0a4dde0710ec23856751dd4 Mon Sep 17 00:00:00 2001 From: Aaron Staple Date: Thu, 25 Sep 2014 16:11:00 -0700 Subject: [PATCH 0788/1492] [SPARK-1484][MLLIB] Warn when running an iterative algorithm on uncached data. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Add warnings to KMeans, GeneralizedLinearAlgorithm, and computeSVD when called with input data that is not cached. KMeans is implemented iteratively, and I believe that GeneralizedLinearAlgorithm’s current optimizers are iterative and its future optimizers are also likely to be iterative. RowMatrix’s computeSVD is iterative against an RDD when run in DistARPACK mode. ALS and DecisionTree are iterative as well, but they implement RDD caching internally so do not require a warning. I added a warning to GeneralizedLinearAlgorithm rather than inside its optimizers, where the iteration actually occurs, because internally GeneralizedLinearAlgorithm maps its input data to an uncached RDD before passing it to an optimizer. (In other words, the warning would be printed for every GeneralizedLinearAlgorithm run, regardless of whether its input is cached, if the warning were in GradientDescent or other optimizer.) I assume that use of an uncached RDD by GeneralizedLinearAlgorithm is intentional, and that the mapping there (adding label, intercepts and scaling) is a lightweight operation. Arguably a user calling an optimizer such as GradientDescent will be knowledgable enough to cache their data without needing a log warning, so lack of a warning in the optimizers may be ok. Some of the documentation examples making use of these iterative algorithms did not cache their training RDDs (while others did). I updated the examples to always cache. I also fixed some (unrelated) minor errors in the documentation examples. Author: Aaron Staple Closes #2347 from staple/SPARK-1484 and squashes the following commits: bd49701 [Aaron Staple] Address review comments. ab2d4a4 [Aaron Staple] Disable warnings on python code path. a7a0f99 [Aaron Staple] Change code comments per review comments. 7cca1dc [Aaron Staple] Change warning message text. c77e939 [Aaron Staple] [SPARK-1484][MLLIB] Warn when running an iterative algorithm on uncached data. 3b6c511 [Aaron Staple] Minor doc example fixes. --- docs/mllib-clustering.md | 3 +- docs/mllib-linear-methods.md | 9 ++-- docs/mllib-optimization.md | 1 + .../mllib/api/python/PythonMLLibAPI.scala | 54 ++++++++++--------- .../spark/mllib/clustering/KMeans.scala | 22 ++++++++ .../mllib/linalg/distributed/RowMatrix.scala | 11 ++++ .../GeneralizedLinearAlgorithm.scala | 21 ++++++++ 7 files changed, 91 insertions(+), 30 deletions(-) diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index dfd9cd572888c..d10bd63746629 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -52,7 +52,7 @@ import org.apache.spark.mllib.linalg.Vectors // Load and parse the data val data = sc.textFile("data/mllib/kmeans_data.txt") -val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))) +val parsedData = data.map(s => Vectors.dense(s.split(' ').map(_.toDouble))).cache() // Cluster the data into two classes using KMeans val numClusters = 2 @@ -100,6 +100,7 @@ public class KMeansExample { } } ); + parsedData.cache(); // Cluster the data into two classes using KMeans int numClusters = 2; diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 9137f9dc1b692..d31bec3e1bd01 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -396,7 +396,7 @@ 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))) -} +}.cache() // Building the model val numIterations = 100 @@ -455,6 +455,7 @@ public class LinearRegression { } } ); + parsedData.cache(); // Building the model int numIterations = 100; @@ -470,7 +471,7 @@ public class LinearRegression { } } ); - JavaRDD MSE = new JavaDoubleRDD(valuesAndPreds.map( + double MSE = new JavaDoubleRDD(valuesAndPreds.map( new Function, Object>() { public Object call(Tuple2 pair) { return Math.pow(pair._1() - pair._2(), 2.0); @@ -553,8 +554,8 @@ but in practice you will likely want to use unlabeled vectors for test data. {% highlight scala %} -val trainingData = ssc.textFileStream('/training/data/dir').map(LabeledPoint.parse) -val testData = ssc.textFileStream('/testing/data/dir').map(LabeledPoint.parse) +val trainingData = ssc.textFileStream("/training/data/dir").map(LabeledPoint.parse).cache() +val testData = ssc.textFileStream("/testing/data/dir").map(LabeledPoint.parse) {% endhighlight %} diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 26ce5f3c501ff..45141c235be90 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -217,6 +217,7 @@ import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLUtils import org.apache.spark.mllib.classification.LogisticRegressionModel +import org.apache.spark.mllib.optimization.{LBFGS, LogisticGradient, SquaredL2Updater} val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") val numFeatures = data.take(1)(0).features.size diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 9164c294ac7b8..e9f41758581e3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -67,11 +67,13 @@ class PythonMLLibAPI extends Serializable { MLUtils.loadLabeledPoints(jsc.sc, path, minPartitions) private def trainRegressionModel( - trainFunc: (RDD[LabeledPoint], Vector) => GeneralizedLinearModel, + learner: GeneralizedLinearAlgorithm[_ <: GeneralizedLinearModel], data: JavaRDD[LabeledPoint], initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { val initialWeights = SerDe.loads(initialWeightsBA).asInstanceOf[Vector] - val model = trainFunc(data.rdd, initialWeights) + // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. + learner.disableUncachedWarning() + val model = learner.run(data.rdd, initialWeights) val ret = new java.util.LinkedList[java.lang.Object]() ret.add(SerDe.dumps(model.weights)) ret.add(model.intercept: java.lang.Double) @@ -106,8 +108,7 @@ class PythonMLLibAPI extends Serializable { + " Can only be initialized using the following string values: [l1, l2, none].") } trainRegressionModel( - (data, initialWeights) => - lrAlg.run(data, initialWeights), + lrAlg, data, initialWeightsBA) } @@ -122,15 +123,14 @@ class PythonMLLibAPI extends Serializable { regParam: Double, miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + val lassoAlg = new LassoWithSGD() + lassoAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + .setMiniBatchFraction(miniBatchFraction) trainRegressionModel( - (data, initialWeights) => - LassoWithSGD.train( - data, - numIterations, - stepSize, - regParam, - miniBatchFraction, - initialWeights), + lassoAlg, data, initialWeightsBA) } @@ -145,15 +145,14 @@ class PythonMLLibAPI extends Serializable { regParam: Double, miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { + val ridgeAlg = new RidgeRegressionWithSGD() + ridgeAlg.optimizer + .setNumIterations(numIterations) + .setRegParam(regParam) + .setStepSize(stepSize) + .setMiniBatchFraction(miniBatchFraction) trainRegressionModel( - (data, initialWeights) => - RidgeRegressionWithSGD.train( - data, - numIterations, - stepSize, - regParam, - miniBatchFraction, - initialWeights), + ridgeAlg, data, initialWeightsBA) } @@ -186,8 +185,7 @@ class PythonMLLibAPI extends Serializable { + " Can only be initialized using the following string values: [l1, l2, none].") } trainRegressionModel( - (data, initialWeights) => - SVMAlg.run(data, initialWeights), + SVMAlg, data, initialWeightsBA) } @@ -220,8 +218,7 @@ class PythonMLLibAPI extends Serializable { + " Can only be initialized using the following string values: [l1, l2, none].") } trainRegressionModel( - (data, initialWeights) => - LogRegAlg.run(data, initialWeights), + LogRegAlg, data, initialWeightsBA) } @@ -249,7 +246,14 @@ class PythonMLLibAPI extends Serializable { maxIterations: Int, runs: Int, initializationMode: String): KMeansModel = { - KMeans.train(data.rdd, k, maxIterations, runs, initializationMode) + val kMeansAlg = new KMeans() + .setK(k) + .setMaxIterations(maxIterations) + .setRuns(runs) + .setInitializationMode(initializationMode) + // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. + .disableUncachedWarning() + return kMeansAlg.run(data.rdd) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index fce8fe29f6e40..7443f232ec3e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -27,6 +27,7 @@ import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom /** @@ -112,11 +113,26 @@ class KMeans private ( this } + /** Whether a warning should be logged if the input RDD is uncached. */ + private var warnOnUncachedInput = true + + /** Disable warnings about uncached input. */ + private[spark] def disableUncachedWarning(): this.type = { + warnOnUncachedInput = false + this + } + /** * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. */ def run(data: RDD[Vector]): KMeansModel = { + + if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data is not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + // Compute squared norms and cache them. val norms = data.map(v => breezeNorm(v.toBreeze, 2.0)) norms.persist() @@ -125,6 +141,12 @@ class KMeans private ( } val model = runBreeze(breezeData) norms.unpersist() + + // Warn at the end of the run as well, for increased visibility. + if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data was not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } model } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 2e414a73be8e0..4174f45d231c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -30,6 +30,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.Logging import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} +import org.apache.spark.storage.StorageLevel /** * :: Experimental :: @@ -231,6 +232,10 @@ class RowMatrix( val brzSvd.SVD(uFull: BDM[Double], sigmaSquaresFull: BDV[Double], _) = brzSvd(G) (sigmaSquaresFull, uFull) case SVDMode.DistARPACK => + if (rows.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data is not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } require(k < n, s"k must be smaller than n in dist-eigs mode but got k=$k and n=$n.") EigenValueDecomposition.symmetricEigs(multiplyGramianMatrixBy, n, k, tol, maxIter) } @@ -256,6 +261,12 @@ class RowMatrix( logWarning(s"Requested $k singular values but only found $sk nonzeros.") } + // Warn at the end of the run as well, for increased visibility. + if (computeMode == SVDMode.DistARPACK && rows.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data was not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + val s = Vectors.dense(Arrays.copyOfRange(sigmas.data, 0, sk)) val V = Matrices.dense(n, sk, Arrays.copyOfRange(u.data, 0, n * sk)) 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 20c1fdd2269ce..d0fe4179685ca 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 @@ -24,6 +24,7 @@ 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._ +import org.apache.spark.storage.StorageLevel /** * :: DeveloperApi :: @@ -133,6 +134,15 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] this } + /** Whether a warning should be logged if the input RDD is uncached. */ + private var warnOnUncachedInput = true + + /** Disable warnings about uncached input. */ + private[spark] def disableUncachedWarning(): this.type = { + warnOnUncachedInput = false + this + } + /** * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. @@ -149,6 +159,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { + if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data is not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + // Check the data properties before running the optimizer if (validateData && !validators.forall(func => func(input))) { throw new SparkException("Input validation failed.") @@ -223,6 +238,12 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] weights = scaler.transform(weights) } + // Warn at the end of the run as well, for increased visibility. + if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data was not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } + createModel(weights, intercept) } } From 0dc868e787a3bc69c1b8e90d916a6dcea8dbcd6d Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 25 Sep 2014 16:49:15 -0700 Subject: [PATCH 0789/1492] [SPARK-3584] sbin/slaves doesn't work when we use password authentication for SSH Author: Kousuke Saruta Closes #2444 from sarutak/slaves-scripts-modification and squashes the following commits: eff7394 [Kousuke Saruta] Improve the description about Cluster Launch Script in docs/spark-standalone.md 7858225 [Kousuke Saruta] Modified sbin/slaves to use the environment variable "SPARK_SSH_FOREGROUND" as a flag 53d7121 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into slaves-scripts-modification e570431 [Kousuke Saruta] Added a description for SPARK_SSH_FOREGROUND variable 7120a0c [Kousuke Saruta] Added a description about default host for sbin/slaves 1bba8a9 [Kousuke Saruta] Added SPARK_SSH_FOREGROUND flag to sbin/slaves 88e2f17 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into slaves-scripts-modification 297e75d [Kousuke Saruta] Modified sbin/slaves not to export HOSTLIST --- .gitignore | 1 + .rat-excludes | 1 + conf/{slaves => slaves.template} | 0 docs/spark-standalone.md | 7 ++++++- sbin/slaves.sh | 31 ++++++++++++++++++++++--------- 5 files changed, 30 insertions(+), 10 deletions(-) rename conf/{slaves => slaves.template} (100%) diff --git a/.gitignore b/.gitignore index 7779980b74a22..34939e3a97aaa 100644 --- a/.gitignore +++ b/.gitignore @@ -23,6 +23,7 @@ conf/*.cmd conf/*.properties conf/*.conf conf/*.xml +conf/slaves docs/_site docs/api target/ diff --git a/.rat-excludes b/.rat-excludes index 9fc99d7fca35d..b14ad53720f32 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -19,6 +19,7 @@ log4j.properties log4j.properties.template metrics.properties.template slaves +slaves.template spark-env.sh spark-env.cmd spark-env.sh.template diff --git a/conf/slaves b/conf/slaves.template similarity index 100% rename from conf/slaves rename to conf/slaves.template diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 29b5491861bf3..58103fab20819 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -62,7 +62,12 @@ Finally, the following configuration options can be passed to the master and wor # Cluster Launch Scripts -To launch a Spark standalone cluster with the launch scripts, you need to create a file called `conf/slaves` in your Spark directory, which should contain the hostnames of all the machines where you would like to start Spark workers, one per line. The master machine must be able to access each of the slave machines via password-less `ssh` (using a private key). For testing, you can just put `localhost` in this file. +To launch a Spark standalone cluster with the launch scripts, you should create a file called conf/slaves in your Spark directory, +which must contain the hostnames of all the machines where you intend to start Spark workers, one per line. +If conf/slaves does not exist, the launch scripts defaults to a single machine (localhost), which is useful for testing. +Note, the master machine accesses each of the worker machines via ssh. By default, ssh is run in parallel and requires password-less (using a private key) access to be setup. +If you do not have a password-less setup, you can set the environment variable SPARK_SSH_FOREGROUND and serially provide a password for each worker. + Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/bin`: diff --git a/sbin/slaves.sh b/sbin/slaves.sh index 1d4dc5edf9858..cdad47ee2e594 100755 --- a/sbin/slaves.sh +++ b/sbin/slaves.sh @@ -44,7 +44,9 @@ sbin="`cd "$sbin"; pwd`" # If the slaves file is specified in the command line, # then it takes precedence over the definition in # spark-env.sh. Save it here. -HOSTLIST="$SPARK_SLAVES" +if [ -f "$SPARK_SLAVES" ]; then + HOSTLIST=`cat "$SPARK_SLAVES"` +fi # Check if --config is passed as an argument. It is an optional parameter. # Exit if the argument is not a directory. @@ -67,23 +69,34 @@ fi if [ "$HOSTLIST" = "" ]; then if [ "$SPARK_SLAVES" = "" ]; then - export HOSTLIST="${SPARK_CONF_DIR}/slaves" + if [ -f "${SPARK_CONF_DIR}/slaves" ]; then + HOSTLIST=`cat "${SPARK_CONF_DIR}/slaves"` + else + HOSTLIST=localhost + fi else - export HOSTLIST="${SPARK_SLAVES}" + HOSTLIST=`cat "${SPARK_SLAVES}"` fi fi + + # By default disable strict host key checking if [ "$SPARK_SSH_OPTS" = "" ]; then SPARK_SSH_OPTS="-o StrictHostKeyChecking=no" fi -for slave in `cat "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do - ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ - 2>&1 | sed "s/^/$slave: /" & - if [ "$SPARK_SLAVE_SLEEP" != "" ]; then - sleep $SPARK_SLAVE_SLEEP - fi +for slave in `echo "$HOSTLIST"|sed "s/#.*$//;/^$/d"`; do + if [ -n "${SPARK_SSH_FOREGROUND}" ]; then + ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$slave: /" + else + ssh $SPARK_SSH_OPTS "$slave" $"${@// /\\ }" \ + 2>&1 | sed "s/^/$slave: /" & + fi + if [ "$SPARK_SLAVE_SLEEP" != "" ]; then + sleep $SPARK_SLAVE_SLEEP + fi done wait From 86bce764983f2b14e1bd87fc3f4f938f7a217e1b Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 25 Sep 2014 18:24:01 -0700 Subject: [PATCH 0790/1492] SPARK-2634: Change MapOutputTrackerWorker.mapStatuses to ConcurrentHashMap MapOutputTrackerWorker.mapStatuses is used concurrently, it should be thread-safe. This bug has already been fixed in #1328. Nevertheless, considering #1328 won't be merged soon, I send this trivial fix and hope this issue can be solved soon. Author: zsxwing Closes #1541 from zsxwing/SPARK-2634 and squashes the following commits: d450053 [zsxwing] SPARK-2634: Change MapOutputTrackerWorker.mapStatuses to ConcurrentHashMap --- .../main/scala/org/apache/spark/MapOutputTracker.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 51705c895a55c..f92189b707fb5 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -18,10 +18,12 @@ package org.apache.spark import java.io._ +import java.util.concurrent.ConcurrentHashMap import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.{HashSet, HashMap, Map} import scala.concurrent.Await +import scala.collection.JavaConversions._ import akka.actor._ import akka.pattern.ask @@ -84,6 +86,9 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks. * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the * master's corresponding HashMap. + * + * Note: because mapStatuses is accessed concurrently, subclasses should make sure it's a + * thread-safe map. */ protected val mapStatuses: Map[Int, Array[MapStatus]] @@ -339,7 +344,8 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) * MapOutputTrackerMaster. */ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { - protected val mapStatuses = new HashMap[Int, Array[MapStatus]] + protected val mapStatuses: Map[Int, Array[MapStatus]] = + new ConcurrentHashMap[Int, Array[MapStatus]] } private[spark] object MapOutputTracker { From b235e013638685758885842dc3268e9800af3678 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Thu, 25 Sep 2014 22:56:43 -0700 Subject: [PATCH 0791/1492] [SPARK-3686][STREAMING] Wait for sink to commit the channel before check... ...ing for the channel size. Author: Hari Shreedharan Closes #2531 from harishreedharan/sparksinksuite-fix and squashes the following commits: 30393c1 [Hari Shreedharan] Use more deterministic method to figure out when batches come in. 6ce9d8b [Hari Shreedharan] [SPARK-3686][STREAMING] Wait for sink to commit the channel before checking for the channel size. --- .../flume/sink/SparkAvroCallbackHandler.scala | 14 +++++++++++- .../streaming/flume/sink/SparkSink.scala | 10 +++++++++ .../flume/sink/TransactionProcessor.scala | 12 ++++++++++ .../streaming/flume/sink/SparkSinkSuite.scala | 22 +++++++++++-------- 4 files changed, 48 insertions(+), 10 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index e77cf7bfa54d0..3c656a381bd9b 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.streaming.flume.sink -import java.util.concurrent.{ConcurrentHashMap, Executors} +import java.util.concurrent.{CountDownLatch, ConcurrentHashMap, Executors} import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConversions._ @@ -58,8 +58,12 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha private val seqBase = RandomStringUtils.randomAlphanumeric(8) private val seqCounter = new AtomicLong(0) + @volatile private var stopped = false + @volatile private var isTest = false + private var testLatch: CountDownLatch = null + /** * Returns a bunch of events to Spark over Avro RPC. * @param n Maximum number of events to return in a batch @@ -90,6 +94,9 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha val processor = new TransactionProcessor( channel, seq, n, transactionTimeout, backOffInterval, this) sequenceNumberToProcessor.put(seq, processor) + if (isTest) { + processor.countDownWhenBatchAcked(testLatch) + } Some(processor) } else { None @@ -141,6 +148,11 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha } } + private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { + testLatch = latch + isTest = true + } + /** * Shuts down the executor used to process transactions. */ diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala index 98ae7d783aec8..14dffb15fef98 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala @@ -138,6 +138,16 @@ class SparkSink extends AbstractSink with Logging with Configurable { throw new RuntimeException("Server was not started!") ) } + + /** + * Pass in a [[CountDownLatch]] for testing purposes. This batch is counted down when each + * batch is received. The test can simply call await on this latch till the expected number of + * batches are received. + * @param latch + */ + private[flume] def countdownWhenBatchReceived(latch: CountDownLatch) { + handler.foreach(_.countDownWhenBatchAcked(latch)) + } } /** diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala index 13f3aa94be414..ea45b14294df9 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala @@ -62,6 +62,10 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, @volatile private var stopped = false + @volatile private var isTest = false + + private var testLatch: CountDownLatch = null + // The transaction that this processor would handle var txOpt: Option[Transaction] = None @@ -182,6 +186,9 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, rollbackAndClose(tx, close = false) // tx will be closed later anyway } finally { tx.close() + if (isTest) { + testLatch.countDown() + } } } else { logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") @@ -237,4 +244,9 @@ private class TransactionProcessor(val channel: Channel, val seqNum: String, processAckOrNack() null } + + private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { + testLatch = latch + isTest = true + } } diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala index 75a6668c6210b..a2b2cc6149d95 100644 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala @@ -38,7 +38,7 @@ class SparkSinkSuite extends FunSuite { val channelCapacity = 5000 test("Success with ack") { - val (channel, sink) = initializeChannelAndSink() + val (channel, sink, latch) = initializeChannelAndSink() channel.start() sink.start() @@ -51,6 +51,7 @@ class SparkSinkSuite extends FunSuite { val events = client.getEventBatch(1000) client.ack(events.getSequenceNumber) assert(events.getEvents.size() === 1000) + latch.await(1, TimeUnit.SECONDS) assertChannelIsEmpty(channel) sink.stop() channel.stop() @@ -58,7 +59,7 @@ class SparkSinkSuite extends FunSuite { } test("Failure with nack") { - val (channel, sink) = initializeChannelAndSink() + val (channel, sink, latch) = initializeChannelAndSink() channel.start() sink.start() putEvents(channel, eventsPerBatch) @@ -70,6 +71,7 @@ class SparkSinkSuite extends FunSuite { val events = client.getEventBatch(1000) assert(events.getEvents.size() === 1000) client.nack(events.getSequenceNumber) + latch.await(1, TimeUnit.SECONDS) assert(availableChannelSlots(channel) === 4000) sink.stop() channel.stop() @@ -77,7 +79,7 @@ class SparkSinkSuite extends FunSuite { } test("Failure with timeout") { - val (channel, sink) = initializeChannelAndSink(Map(SparkSinkConfig + val (channel, sink, latch) = initializeChannelAndSink(Map(SparkSinkConfig .CONF_TRANSACTION_TIMEOUT -> 1.toString)) channel.start() sink.start() @@ -88,7 +90,7 @@ class SparkSinkSuite extends FunSuite { val (transceiver, client) = getTransceiverAndClient(address, 1)(0) val events = client.getEventBatch(1000) assert(events.getEvents.size() === 1000) - Thread.sleep(1000) + latch.await(1, TimeUnit.SECONDS) assert(availableChannelSlots(channel) === 4000) sink.stop() channel.stop() @@ -106,7 +108,7 @@ class SparkSinkSuite extends FunSuite { def testMultipleConsumers(failSome: Boolean): Unit = { implicit val executorContext = ExecutionContext .fromExecutorService(Executors.newFixedThreadPool(5)) - val (channel, sink) = initializeChannelAndSink() + val (channel, sink, latch) = initializeChannelAndSink(Map.empty, 5) channel.start() sink.start() (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) @@ -136,7 +138,7 @@ class SparkSinkSuite extends FunSuite { } }) batchCounter.await() - TimeUnit.SECONDS.sleep(1) // Allow the sink to commit the transactions. + latch.await(1, TimeUnit.SECONDS) executorContext.shutdown() if(failSome) { assert(availableChannelSlots(channel) === 3000) @@ -148,8 +150,8 @@ class SparkSinkSuite extends FunSuite { transceiversAndClients.foreach(x => x._1.close()) } - private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty): (MemoryChannel, - SparkSink) = { + private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty, + batchCounter: Int = 1): (MemoryChannel, SparkSink, CountDownLatch) = { val channel = new MemoryChannel() val channelContext = new Context() @@ -165,7 +167,9 @@ class SparkSinkSuite extends FunSuite { sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) sink.configure(sinkContext) sink.setChannel(channel) - (channel, sink) + val latch = new CountDownLatch(batchCounter) + sink.countdownWhenBatchReceived(latch) + (channel, sink, latch) } private def putEvents(ch: MemoryChannel, count: Int): Unit = { From 1aa549ba9839565274a12c52fa1075b424f138a6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 26 Sep 2014 09:27:42 -0700 Subject: [PATCH 0792/1492] [SPARK-3478] [PySpark] Profile the Python tasks This patch add profiling support for PySpark, it will show the profiling results before the driver exits, here is one example: ``` ============================================================ Profile of RDD ============================================================ 5146507 function calls (5146487 primitive calls) in 71.094 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge) 20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__) 20 0.017 0.001 0.017 0.001 {cPickle.dumps} 1024 0.003 0.000 0.003 0.000 t.py:16() 20 0.001 0.000 0.001 0.000 {reduce} 21 0.001 0.000 0.001 0.000 {cPickle.loads} 20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames) 41 0.001 0.000 0.001 0.000 serializers.py:461(read_int) 40 0.001 0.000 0.002 0.000 serializers.py:179(_batched) 62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects} 20 0.000 0.000 71.072 3.554 rdd.py:863() 20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream) 40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func) 41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream) 40 0.000 0.000 71.072 1.777 rdd.py:304(func) 20 0.000 0.000 71.094 3.555 worker.py:82(process) ``` Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk by `sc.dump_profiles(path)`, such as ```python >>> sc._conf.set("spark.python.profile", "true") >>> rdd = sc.parallelize(range(100)).map(str) >>> rdd.count() 100 >>> sc.show_profiles() ============================================================ Profile of RDD ============================================================ 284 function calls (276 primitive calls) in 0.001 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream) 4 0.000 0.000 0.000 0.000 {reduce} 12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func) 4 0.000 0.000 0.000 0.000 {cPickle.loads} 4 0.000 0.000 0.000 0.000 {cPickle.dumps} 104 0.000 0.000 0.000 0.000 rdd.py:852() 8 0.000 0.000 0.000 0.000 serializers.py:461(read_int) 12 0.000 0.000 0.000 0.000 rdd.py:303(func) ``` The profiling is disabled by default, can be enabled by "spark.python.profile=true". Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump" Author: Davies Liu Closes #2351 from davies/profiler and squashes the following commits: 7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles() 2b0daf2 [Davies Liu] fix docs 7a56c24 [Davies Liu] bugfix cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 09d02c3 [Davies Liu] Merge branch 'master' into profiler c23865c [Davies Liu] Merge branch 'master' into profiler 15d6f18 [Davies Liu] add docs for two configs dadee1a [Davies Liu] add docs string and clear profiles after show or dump 4f8309d [Davies Liu] address comment, add tests 0a5b6eb [Davies Liu] fix Python UDF 4b20494 [Davies Liu] add profile for python --- docs/configuration.md | 19 +++++++++++++++++ python/pyspark/accumulators.py | 15 +++++++++++++ python/pyspark/context.py | 39 +++++++++++++++++++++++++++++++++- python/pyspark/rdd.py | 10 +++++++-- python/pyspark/sql.py | 2 +- python/pyspark/tests.py | 30 ++++++++++++++++++++++++++ python/pyspark/worker.py | 19 ++++++++++++++--- 7 files changed, 127 insertions(+), 7 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index a6dd7245e1552..791b6f2aa3261 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -206,6 +206,25 @@ Apart from these, the following properties are also available, and may be useful used during aggregation goes above this amount, it will spill the data into disks. + + + + + + + + + diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index ccbca67656c8d..b8cdbbe3cf2b6 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -215,6 +215,21 @@ def addInPlace(self, value1, value2): COMPLEX_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0j) +class PStatsParam(AccumulatorParam): + """PStatsParam is used to merge pstats.Stats""" + + @staticmethod + def zero(value): + return None + + @staticmethod + def addInPlace(value1, value2): + if value1 is None: + return value2 + value1.add(value2) + return value1 + + class _UpdateRequestHandler(SocketServer.StreamRequestHandler): """ diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 8e7b00469e246..abeda19b77d8b 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -20,6 +20,7 @@ import sys from threading import Lock from tempfile import NamedTemporaryFile +import atexit from pyspark import accumulators from pyspark.accumulators import Accumulator @@ -30,7 +31,6 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, CompressedSerializer from pyspark.storagelevel import StorageLevel -from pyspark import rdd from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -192,6 +192,9 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self._temp_dir = \ self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() + # profiling stats collected for each PythonRDD + self._profile_stats = [] + def _initialize_context(self, jconf): """ Initialize SparkContext in function to allow subclass specific initialization @@ -792,6 +795,40 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal) return list(mappedRDD._collect_iterator_through_file(it)) + def _add_profile(self, id, profileAcc): + if not self._profile_stats: + dump_path = self._conf.get("spark.python.profile.dump") + if dump_path: + atexit.register(self.dump_profiles, dump_path) + else: + atexit.register(self.show_profiles) + + self._profile_stats.append([id, profileAcc, False]) + + def show_profiles(self): + """ Print the profile stats to stdout """ + for i, (id, acc, showed) in enumerate(self._profile_stats): + stats = acc.value + if not showed and stats: + print "=" * 60 + print "Profile of RDD" % id + print "=" * 60 + stats.sort_stats("tottime", "cumtime").print_stats() + # mark it as showed + self._profile_stats[i][2] = True + + def dump_profiles(self, path): + """ Dump the profile stats into directory `path` + """ + if not os.path.exists(path): + os.makedirs(path) + for id, acc, _ in self._profile_stats: + stats = acc.value + if stats: + p = os.path.join(path, "rdd_%d.pstats" % id) + stats.dump_stats(p) + self._profile_stats = [] + def _test(): import atexit diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 680140d72d03c..8ed89e2f9769f 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -15,7 +15,6 @@ # limitations under the License. # -from base64 import standard_b64encode as b64enc import copy from collections import defaultdict from itertools import chain, ifilter, imap @@ -32,6 +31,7 @@ from random import Random from math import sqrt, log, isinf, isnan +from pyspark.accumulators import PStatsParam from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer @@ -2080,7 +2080,9 @@ def _jrdd(self): return self._jrdd_val if self._bypass_serializer: self._jrdd_deserializer = NoOpSerializer() - command = (self.func, self._prev_jrdd_deserializer, + enable_profile = self.ctx._conf.get("spark.python.profile", "false") == "true" + profileStats = self.ctx.accumulator(None, PStatsParam) if enable_profile else None + command = (self.func, profileStats, self._prev_jrdd_deserializer, self._jrdd_deserializer) # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() @@ -2102,6 +2104,10 @@ def _jrdd(self): self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() + + if enable_profile: + self._id = self._jrdd_val.id() + self.ctx._add_profile(self._id, profileStats) return self._jrdd_val def id(self): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 653195ea438cf..ee5bda8bb43d5 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -974,7 +974,7 @@ def registerFunction(self, name, f, returnType=StringType()): [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) - command = (func, + command = (func, None, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) ser = CloudPickleSerializer() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index d1bb2033b7a16..e6002afa9c70d 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -632,6 +632,36 @@ def test_distinct(self): self.assertEquals(result.count(), 3) +class TestProfiler(PySparkTestCase): + + def setUp(self): + self._old_sys_path = list(sys.path) + class_name = self.__class__.__name__ + conf = SparkConf().set("spark.python.profile", "true") + self.sc = SparkContext('local[4]', class_name, batchSize=2, conf=conf) + + def test_profiler(self): + + def heavy_foo(x): + for i in range(1 << 20): + x = 1 + rdd = self.sc.parallelize(range(100)) + rdd.foreach(heavy_foo) + profiles = self.sc._profile_stats + self.assertEqual(1, len(profiles)) + id, acc, _ = profiles[0] + stats = acc.value + self.assertTrue(stats is not None) + width, stat_list = stats.get_print_list([]) + func_names = [func_name for fname, n, func_name in stat_list] + self.assertTrue("heavy_foo" in func_names) + + self.sc.show_profiles() + d = tempfile.gettempdir() + self.sc.dump_profiles(d) + self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) + + class TestSQL(PySparkTestCase): def setUp(self): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index c1f6e3e4a1f40..8257dddfee1c3 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -23,6 +23,8 @@ import time import socket import traceback +import cProfile +import pstats from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry @@ -90,10 +92,21 @@ def main(infile, outfile): command = pickleSer._read_with_length(infile) if isinstance(command, Broadcast): command = pickleSer.loads(command.value) - (func, deserializer, serializer) = command + (func, stats, deserializer, serializer) = command init_time = time.time() - iterator = deserializer.load_stream(infile) - serializer.dump_stream(func(split_index, iterator), outfile) + + def process(): + iterator = deserializer.load_stream(infile) + serializer.dump_stream(func(split_index, iterator), outfile) + + if stats: + p = cProfile.Profile() + p.runcall(process) + st = pstats.Stats(p) + st.stream = None # make it picklable + stats.add(st.strip_dirs()) + else: + process() except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) From d16e161d744b27291fd2ee7e3578917ee14d83f9 Mon Sep 17 00:00:00 2001 From: aniketbhatnagar Date: Fri, 26 Sep 2014 09:47:58 -0700 Subject: [PATCH 0793/1492] SPARK-3639 | Removed settings master in examples This patch removes setting of master as local in Kinesis examples so that users can set it using submit-job. Author: aniketbhatnagar Closes #2536 from aniketbhatnagar/Kinesis-Examples-Master-Unset and squashes the following commits: c9723ac [aniketbhatnagar] Merge remote-tracking branch 'origin/Kinesis-Examples-Master-Unset' into Kinesis-Examples-Master-Unset fec8ead [aniketbhatnagar] SPARK-3639 | Removed settings master in examples 31cdc59 [aniketbhatnagar] SPARK-3639 | Removed settings master in examples --- .../examples/streaming/JavaKinesisWordCountASL.java | 9 ++++----- .../examples/streaming/KinesisWordCountASL.scala | 13 +++++-------- 2 files changed, 9 insertions(+), 13 deletions(-) diff --git a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index aa917d0575c4c..b0bff27a61c19 100644 --- a/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/extras/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -71,6 +71,9 @@ * org.apache.spark.examples.streaming.JavaKinesisWordCountASL mySparkStream \ * https://kinesis.us-east-1.amazonaws.com * + * Note that number of workers/threads should be 1 more than the number of receivers. + * This leaves one thread available for actually processing the data. + * * There is a companion helper class called KinesisWordCountProducerASL which puts dummy data * onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in the class definition. @@ -114,12 +117,8 @@ public static void main(String[] args) { /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard */ int numStreams = numShards; - /* Must add 1 more thread than the number of receivers or the output won't show properly from the driver */ - int numSparkThreads = numStreams + 1; - /* Setup the Spark config. */ - SparkConf sparkConfig = new SparkConf().setAppName("KinesisWordCount").setMaster( - "local[" + numSparkThreads + "]"); + SparkConf sparkConfig = new SparkConf().setAppName("KinesisWordCount"); /* Kinesis checkpoint interval. Same as batchInterval for this example. */ Duration checkpointInterval = batchInterval; diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala index fffd90de08240..32da0858d1a1d 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/examples/streaming/KinesisWordCountASL.scala @@ -65,6 +65,10 @@ import org.apache.log4j.Level * org.apache.spark.examples.streaming.KinesisWordCountASL mySparkStream \ * https://kinesis.us-east-1.amazonaws.com * + * + * Note that number of workers/threads should be 1 more than the number of receivers. + * This leaves one thread available for actually processing the data. + * * There is a companion helper class below called KinesisWordCountProducerASL which puts * dummy data onto the Kinesis stream. * Usage instructions for KinesisWordCountProducerASL are provided in that class definition. @@ -97,17 +101,10 @@ private object KinesisWordCountASL extends Logging { /* In this example, we're going to create 1 Kinesis Worker/Receiver/DStream for each shard. */ val numStreams = numShards - /* - * numSparkThreads should be 1 more thread than the number of receivers. - * This leaves one thread available for actually processing the data. - */ - val numSparkThreads = numStreams + 1 - /* Setup the and SparkConfig and StreamingContext */ /* Spark Streaming batch interval */ - val batchInterval = Milliseconds(2000) + val batchInterval = Milliseconds(2000) val sparkConfig = new SparkConf().setAppName("KinesisWordCount") - .setMaster(s"local[$numSparkThreads]") val ssc = new StreamingContext(sparkConfig, batchInterval) /* Kinesis checkpoint interval. Same as batchInterval for this example. */ From ec9df6a765701fa41390083df12e1dc1fee50662 Mon Sep 17 00:00:00 2001 From: RJ Nowling Date: Fri, 26 Sep 2014 09:58:47 -0700 Subject: [PATCH 0794/1492] [SPARK-3614][MLLIB] Add minimumOccurence filtering to IDF This PR for [SPARK-3614](https://issues.apache.org/jira/browse/SPARK-3614) adds functionality for filtering out terms which do not appear in at least a minimum number of documents. This is implemented using a minimumOccurence parameter (default 0). When terms' document frequencies are less than minimumOccurence, their IDFs are set to 0, just like when the DF is 0. As a result, the TF-IDFs for the terms are found to be 0, as if the terms were not present in the documents. This PR makes the following changes: * Add a minimumOccurence parameter to the IDF and DocumentFrequencyAggregator classes. * Create a parameter-less constructor for IDF with a default minimumOccurence value of 0 to remain backwards-compatibility with the original IDF API. * Sets the IDFs to 0 for terms which DFs are less than minimumOccurence * Add tests to the Spark IDFSuite and Java JavaTfIdfSuite test suites * Updated the MLLib Feature Extraction programming guide to describe the new feature Author: RJ Nowling Closes #2494 from rnowling/spark-3614-idf-filter and squashes the following commits: 0aa3c63 [RJ Nowling] Fix identation e6523a8 [RJ Nowling] Remove unnecessary toDouble's from IDFSuite bfa82ec [RJ Nowling] Add space after if 30d20b3 [RJ Nowling] Add spaces around equals signs 9013447 [RJ Nowling] Add space before division operator 79978fc [RJ Nowling] Remove unnecessary semi-colon 40fd70c [RJ Nowling] Change minimumOccurence to minDocFreq in code and docs 47850ab [RJ Nowling] Changed minimumOccurence to Int from Long 9fb4093 [RJ Nowling] Remove unnecessary lines from IDF class docs 1fc09d8 [RJ Nowling] Add backwards-compatible constructor to DocumentFrequencyAggregator 1801fd2 [RJ Nowling] Fix style errors in IDF.scala 6897252 [RJ Nowling] Preface minimumOccurence members with val to make them final and immutable a200bab [RJ Nowling] Remove unnecessary else statement 4b974f5 [RJ Nowling] Remove accidentally-added import from testing c0cc643 [RJ Nowling] Add minimumOccurence filtering to IDF --- docs/mllib-feature-extraction.md | 15 ++++++++ .../org/apache/spark/mllib/feature/IDF.scala | 37 +++++++++++++++++-- .../spark/mllib/feature/JavaTfIdfSuite.java | 20 ++++++++++ .../apache/spark/mllib/feature/IDFSuite.scala | 36 +++++++++++++++++- 4 files changed, 103 insertions(+), 5 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 41a27f6208d1b..1511ae6dda4ed 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -82,6 +82,21 @@ tf.cache() val idf = new IDF().fit(tf) val tfidf: RDD[Vector] = idf.transform(tf) {% endhighlight %} + +MLLib's IDF implementation provides an option for ignoring terms which occur in less than a +minimum number of documents. In such cases, the IDF for these terms is set to 0. This feature +can be used by passing the `minDocFreq` value to the IDF constructor. + +{% highlight scala %} +import org.apache.spark.mllib.feature.IDF + +// ... continue from the previous example +tf.cache() +val idf = new IDF(minDocFreq = 2).fit(tf) +val tfidf: RDD[Vector] = idf.transform(tf) +{% endhighlight %} + + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index d40d5553c1d21..720bb70b08dbf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -30,9 +30,18 @@ import org.apache.spark.rdd.RDD * Inverse document frequency (IDF). * The standard formulation is used: `idf = log((m + 1) / (d(t) + 1))`, where `m` is the total * number of documents and `d(t)` is the number of documents that contain term `t`. + * + * This implementation supports filtering out terms which do not appear in a minimum number + * of documents (controlled by the variable `minDocFreq`). For terms that are not in + * at least `minDocFreq` documents, the IDF is found as 0, resulting in TF-IDFs of 0. + * + * @param minDocFreq minimum of documents in which a term + * should appear for filtering */ @Experimental -class IDF { +class IDF(val minDocFreq: Int) { + + def this() = this(0) // TODO: Allow different IDF formulations. @@ -41,7 +50,8 @@ class IDF { * @param dataset an RDD of term frequency vectors */ def fit(dataset: RDD[Vector]): IDFModel = { - val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator)( + val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator( + minDocFreq = minDocFreq))( seqOp = (df, v) => df.add(v), combOp = (df1, df2) => df1.merge(df2) ).idf() @@ -60,13 +70,16 @@ class IDF { private object IDF { /** Document frequency aggregator. */ - class DocumentFrequencyAggregator extends Serializable { + class DocumentFrequencyAggregator(val minDocFreq: Int) extends Serializable { /** number of documents */ private var m = 0L /** document frequency vector */ private var df: BDV[Long] = _ + + def this() = this(0) + /** Adds a new document. */ def add(doc: Vector): this.type = { if (isEmpty) { @@ -123,7 +136,18 @@ private object IDF { val inv = new Array[Double](n) var j = 0 while (j < n) { - inv(j) = math.log((m + 1.0)/ (df(j) + 1.0)) + /* + * If the term is not present in the minimum + * number of documents, set IDF to 0. This + * will cause multiplication in IDFModel to + * set TF-IDF to 0. + * + * Since arrays are initialized to 0 by default, + * we just omit changing those entries. + */ + if(df(j) >= minDocFreq) { + inv(j) = math.log((m + 1.0) / (df(j) + 1.0)) + } j += 1 } Vectors.dense(inv) @@ -140,6 +164,11 @@ class IDFModel private[mllib] (val idf: Vector) extends Serializable { /** * Transforms term frequency (TF) vectors to TF-IDF vectors. + * + * If `minDocFreq` was set for the IDF calculation, + * the terms which occur in fewer than `minDocFreq` + * documents will have an entry of 0. + * * @param dataset an RDD of term frequency vectors * @return an RDD of TF-IDF vectors */ diff --git a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java index e8d99f4ae43ae..064263e02cd11 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/feature/JavaTfIdfSuite.java @@ -63,4 +63,24 @@ public void tfIdf() { Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); } } + + @Test + public void tfIdfMinimumDocumentFrequency() { + // The tests are to check Java compatibility. + HashingTF tf = new HashingTF(); + JavaRDD> documents = sc.parallelize(Lists.newArrayList( + Lists.newArrayList("this is a sentence".split(" ")), + Lists.newArrayList("this is another sentence".split(" ")), + Lists.newArrayList("this is still a sentence".split(" "))), 2); + JavaRDD termFreqs = tf.transform(documents); + termFreqs.collect(); + IDF idf = new IDF(2); + JavaRDD tfIdfs = idf.fit(termFreqs).transform(termFreqs); + List localTfIdfs = tfIdfs.collect(); + int indexOfThis = tf.indexOf("this"); + for (Vector v: localTfIdfs) { + Assert.assertEquals(0.0, v.apply(indexOfThis), 1e-15); + } + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala index 53d9c0c640b98..43974f84e3ca8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala @@ -38,7 +38,7 @@ class IDFSuite extends FunSuite with LocalSparkContext { val idf = new IDF val model = idf.fit(termFrequencies) val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => - math.log((m.toDouble + 1.0) / (x + 1.0)) + math.log((m + 1.0) / (x + 1.0)) }) assert(model.idf ~== expected absTol 1e-12) val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() @@ -54,4 +54,38 @@ class IDFSuite extends FunSuite with LocalSparkContext { assert(tfidf2.indices === Array(1)) assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) } + + test("idf minimum document frequency filtering") { + val n = 4 + val localTermFrequencies = Seq( + Vectors.sparse(n, Array(1, 3), Array(1.0, 2.0)), + Vectors.dense(0.0, 1.0, 2.0, 3.0), + Vectors.sparse(n, Array(1), Array(1.0)) + ) + val m = localTermFrequencies.size + val termFrequencies = sc.parallelize(localTermFrequencies, 2) + val idf = new IDF(minDocFreq = 1) + val model = idf.fit(termFrequencies) + val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => + if (x > 0) { + math.log((m + 1.0) / (x + 1.0)) + } else { + 0 + } + }) + assert(model.idf ~== expected absTol 1e-12) + val tfidf = model.transform(termFrequencies).cache().zipWithIndex().map(_.swap).collectAsMap() + assert(tfidf.size === 3) + val tfidf0 = tfidf(0L).asInstanceOf[SparseVector] + assert(tfidf0.indices === Array(1, 3)) + assert(Vectors.dense(tfidf0.values) ~== + Vectors.dense(1.0 * expected(1), 2.0 * expected(3)) absTol 1e-12) + val tfidf1 = tfidf(1L).asInstanceOf[DenseVector] + assert(Vectors.dense(tfidf1.values) ~== + Vectors.dense(0.0, 1.0 * expected(1), 2.0 * expected(2), 3.0 * expected(3)) absTol 1e-12) + val tfidf2 = tfidf(2L).asInstanceOf[SparseVector] + assert(tfidf2.indices === Array(1)) + assert(tfidf2.values(0) ~== (1.0 * expected(1)) absTol 1e-12) + } + } From 30461c6ac3dcfb05dc1891494ec161601c0fb59f Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 26 Sep 2014 11:26:53 -0700 Subject: [PATCH 0795/1492] [SPARK-3695]shuffle fetch fail output should output detailed host and port in error message Author: Daoyuan Wang Closes #2539 from adrian-wang/fetchfail and squashes the following commits: 6c1b1e0 [Daoyuan Wang] shuffle fetch fail output --- .../org/apache/spark/storage/ShuffleBlockFetcherIterator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index d868758a7f549..71b276b5f18e4 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -121,7 +121,7 @@ final class ShuffleBlockFetcherIterator( } override def onBlockFetchFailure(e: Throwable): Unit = { - logError("Failed to get block(s) from ${req.address.host}:${req.address.port}", e) + logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) // Note that there is a chance that some blocks have been fetched successfully, but we // still add them to the failed queue. This is fine because when the caller see a // FetchFailedException, it is going to fail the entire task anyway. From 8da10bf14660f1d5b1dab692cb56b9832ab10d40 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 26 Sep 2014 11:50:48 -0700 Subject: [PATCH 0796/1492] [SPARK-3476] Remove outdated memory checks in Yarn See description in [JIRA](https://issues.apache.org/jira/browse/SPARK-3476). Author: Andrew Or Closes #2528 from andrewor14/yarn-memory-checks and squashes the following commits: c5400cd [Andrew Or] Simplify checks e30ffac [Andrew Or] Remove outdated memory checks --- .../apache/spark/deploy/yarn/ClientArguments.scala | 13 +++---------- 1 file changed, 3 insertions(+), 10 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 201b742736c6e..26dbd6237c6b8 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -69,16 +69,9 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) * This is intended to be called only after the provided arguments have been parsed. */ private def validateArgs(): Unit = { - // TODO: memory checks are outdated (SPARK-3476) - Map[Boolean, String]( - (numExecutors <= 0) -> "You must specify at least 1 executor!", - (amMemory <= amMemoryOverhead) -> s"AM memory must be > $amMemoryOverhead MB", - (executorMemory <= executorMemoryOverhead) -> - s"Executor memory must be > $executorMemoryOverhead MB" - ).foreach { case (errorCondition, errorMessage) => - if (errorCondition) { - throw new IllegalArgumentException(errorMessage + "\n" + getUsageMessage()) - } + if (numExecutors <= 0) { + throw new IllegalArgumentException( + "You must specify at least 1 executor!\n" + getUsageMessage()) } } From 0ec2d2e8f0c0dc61a7ed6377898846661d2424cd Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 26 Sep 2014 12:04:37 -0700 Subject: [PATCH 0797/1492] [SPARK-3531][SQL]select null from table would throw a MatchError Author: Daoyuan Wang Closes #2396 from adrian-wang/selectnull and squashes the following commits: 2458229 [Daoyuan Wang] rebase solution --- .../scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 1 + .../select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 | 1 + .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 3 +++ 3 files changed, 5 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 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 6b4399e852c7b..9a0b9b46ac4ee 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 @@ -244,6 +244,7 @@ object HiveMetastoreTypes extends RegexParsers { case BooleanType => "boolean" case DecimalType => "decimal" case TimestampType => "timestamp" + case NullType => "void" } } diff --git a/sql/hive/src/test/resources/golden/select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 b/sql/hive/src/test/resources/golden/select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/select null from table-0-5bb53cca754cc8afe9cd22feb8c586d1 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 426f5fcee6157..2f876cafaf218 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -330,6 +330,9 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("timestamp cast #8", "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + createQueryTest("select null from table", + "SELECT null FROM src LIMIT 1") + test("implement identity function using case statement") { val actual = sql("SELECT (CASE key WHEN key THEN key END) FROM src") .map { case Row(i: Int) => i } From 7364fa5a176da69e425bca0e3e137ee73275c78c Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Fri, 26 Sep 2014 12:06:01 -0700 Subject: [PATCH 0798/1492] [SPARK-3393] [SQL] Align the log4j configuration for Spark & SparkSQLCLI User may be confused for the HQL logging & configurations, we'd better provide a default templates. Both files are copied from Hive. Author: Cheng Hao Closes #2263 from chenghao-intel/hive_template and squashes the following commits: 53bffa9 [Cheng Hao] Remove the hive-log4j.properties initialization --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 17 ----------------- 1 file changed, 17 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index b092f42372171..7ba4564602ecd 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -73,18 +73,6 @@ private[hive] object SparkSQLCLIDriver { System.exit(1) } - // NOTE: It is critical to do this here so that log4j is reinitialized - // before any of the other core hive classes are loaded - var logInitFailed = false - var logInitDetailMessage: String = null - try { - logInitDetailMessage = LogUtils.initHiveLog4j() - } catch { - case e: LogInitializationException => - logInitFailed = true - logInitDetailMessage = e.getMessage - } - val sessionState = new CliSessionState(new HiveConf(classOf[SessionState])) sessionState.in = System.in @@ -100,11 +88,6 @@ private[hive] object SparkSQLCLIDriver { System.exit(2) } - if (!sessionState.getIsSilent) { - if (logInitFailed) System.err.println(logInitDetailMessage) - else SessionState.getConsole.printInfo(logInitDetailMessage) - } - // Set all properties specified via command line. val conf: HiveConf = sessionState.getConf sessionState.cmdProperties.entrySet().foreach { item: java.util.Map.Entry[Object, Object] => From f872e4fb80b8429800daa9c44c0cac620c1ff303 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 26 Sep 2014 14:47:14 -0700 Subject: [PATCH 0799/1492] Revert "[SPARK-3478] [PySpark] Profile the Python tasks" This reverts commit 1aa549ba9839565274a12c52fa1075b424f138a6. --- docs/configuration.md | 19 ----------------- python/pyspark/accumulators.py | 15 ------------- python/pyspark/context.py | 39 +--------------------------------- python/pyspark/rdd.py | 10 ++------- python/pyspark/sql.py | 2 +- python/pyspark/tests.py | 30 -------------------------- python/pyspark/worker.py | 19 +++-------------- 7 files changed, 7 insertions(+), 127 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 791b6f2aa3261..a6dd7245e1552 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -206,25 +206,6 @@ Apart from these, the following properties are also available, and may be useful used during aggregation goes above this amount, it will spill the data into disks. - - - - - - - - - diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index b8cdbbe3cf2b6..ccbca67656c8d 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -215,21 +215,6 @@ def addInPlace(self, value1, value2): COMPLEX_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0j) -class PStatsParam(AccumulatorParam): - """PStatsParam is used to merge pstats.Stats""" - - @staticmethod - def zero(value): - return None - - @staticmethod - def addInPlace(value1, value2): - if value1 is None: - return value2 - value1.add(value2) - return value1 - - class _UpdateRequestHandler(SocketServer.StreamRequestHandler): """ diff --git a/python/pyspark/context.py b/python/pyspark/context.py index abeda19b77d8b..8e7b00469e246 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -20,7 +20,6 @@ import sys from threading import Lock from tempfile import NamedTemporaryFile -import atexit from pyspark import accumulators from pyspark.accumulators import Accumulator @@ -31,6 +30,7 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, CompressedSerializer from pyspark.storagelevel import StorageLevel +from pyspark import rdd from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -192,9 +192,6 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self._temp_dir = \ self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() - # profiling stats collected for each PythonRDD - self._profile_stats = [] - def _initialize_context(self, jconf): """ Initialize SparkContext in function to allow subclass specific initialization @@ -795,40 +792,6 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal) return list(mappedRDD._collect_iterator_through_file(it)) - def _add_profile(self, id, profileAcc): - if not self._profile_stats: - dump_path = self._conf.get("spark.python.profile.dump") - if dump_path: - atexit.register(self.dump_profiles, dump_path) - else: - atexit.register(self.show_profiles) - - self._profile_stats.append([id, profileAcc, False]) - - def show_profiles(self): - """ Print the profile stats to stdout """ - for i, (id, acc, showed) in enumerate(self._profile_stats): - stats = acc.value - if not showed and stats: - print "=" * 60 - print "Profile of RDD" % id - print "=" * 60 - stats.sort_stats("tottime", "cumtime").print_stats() - # mark it as showed - self._profile_stats[i][2] = True - - def dump_profiles(self, path): - """ Dump the profile stats into directory `path` - """ - if not os.path.exists(path): - os.makedirs(path) - for id, acc, _ in self._profile_stats: - stats = acc.value - if stats: - p = os.path.join(path, "rdd_%d.pstats" % id) - stats.dump_stats(p) - self._profile_stats = [] - def _test(): import atexit diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8ed89e2f9769f..680140d72d03c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -15,6 +15,7 @@ # limitations under the License. # +from base64 import standard_b64encode as b64enc import copy from collections import defaultdict from itertools import chain, ifilter, imap @@ -31,7 +32,6 @@ from random import Random from math import sqrt, log, isinf, isnan -from pyspark.accumulators import PStatsParam from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer @@ -2080,9 +2080,7 @@ def _jrdd(self): return self._jrdd_val if self._bypass_serializer: self._jrdd_deserializer = NoOpSerializer() - enable_profile = self.ctx._conf.get("spark.python.profile", "false") == "true" - profileStats = self.ctx.accumulator(None, PStatsParam) if enable_profile else None - command = (self.func, profileStats, self._prev_jrdd_deserializer, + command = (self.func, self._prev_jrdd_deserializer, self._jrdd_deserializer) # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() @@ -2104,10 +2102,6 @@ def _jrdd(self): self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() - - if enable_profile: - self._id = self._jrdd_val.id() - self.ctx._add_profile(self._id, profileStats) return self._jrdd_val def id(self): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index ee5bda8bb43d5..653195ea438cf 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -974,7 +974,7 @@ def registerFunction(self, name, f, returnType=StringType()): [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) - command = (func, None, + command = (func, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) ser = CloudPickleSerializer() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index e6002afa9c70d..d1bb2033b7a16 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -632,36 +632,6 @@ def test_distinct(self): self.assertEquals(result.count(), 3) -class TestProfiler(PySparkTestCase): - - def setUp(self): - self._old_sys_path = list(sys.path) - class_name = self.__class__.__name__ - conf = SparkConf().set("spark.python.profile", "true") - self.sc = SparkContext('local[4]', class_name, batchSize=2, conf=conf) - - def test_profiler(self): - - def heavy_foo(x): - for i in range(1 << 20): - x = 1 - rdd = self.sc.parallelize(range(100)) - rdd.foreach(heavy_foo) - profiles = self.sc._profile_stats - self.assertEqual(1, len(profiles)) - id, acc, _ = profiles[0] - stats = acc.value - self.assertTrue(stats is not None) - width, stat_list = stats.get_print_list([]) - func_names = [func_name for fname, n, func_name in stat_list] - self.assertTrue("heavy_foo" in func_names) - - self.sc.show_profiles() - d = tempfile.gettempdir() - self.sc.dump_profiles(d) - self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) - - class TestSQL(PySparkTestCase): def setUp(self): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8257dddfee1c3..c1f6e3e4a1f40 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -23,8 +23,6 @@ import time import socket import traceback -import cProfile -import pstats from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry @@ -92,21 +90,10 @@ def main(infile, outfile): command = pickleSer._read_with_length(infile) if isinstance(command, Broadcast): command = pickleSer.loads(command.value) - (func, stats, deserializer, serializer) = command + (func, deserializer, serializer) = command init_time = time.time() - - def process(): - iterator = deserializer.load_stream(infile) - serializer.dump_stream(func(split_index, iterator), outfile) - - if stats: - p = cProfile.Profile() - p.runcall(process) - st = pstats.Stats(p) - st.stream = None # make it picklable - stats.add(st.strip_dirs()) - else: - process() + iterator = deserializer.load_stream(infile) + serializer.dump_stream(func(split_index, iterator), outfile) except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) From 5e34855cf04145cc3b7bae996c2a6e668f144a11 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 26 Sep 2014 21:29:54 -0700 Subject: [PATCH 0800/1492] [SPARK-3543] Write TaskContext in Java and expose it through a static accessor. Author: Prashant Sharma Author: Shashank Sharma Closes #2425 from ScrapCodes/SPARK-3543/withTaskContext and squashes the following commits: 8ae414c [Shashank Sharma] CR ee8bd00 [Prashant Sharma] Added internal API in docs comments. ddb8cbe [Prashant Sharma] Moved setting the thread local to where TaskContext is instantiated. a7d5e23 [Prashant Sharma] Added doc comments. edf945e [Prashant Sharma] Code review git add -A f716fd1 [Prashant Sharma] introduced thread local for getting the task context. 333c7d6 [Prashant Sharma] Translated Task context from scala to java. --- .../java/org/apache/spark/TaskContext.java | 274 ++++++++++++++++++ .../scala/org/apache/spark/TaskContext.scala | 126 -------- .../main/scala/org/apache/spark/rdd/RDD.scala | 1 + .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../org/apache/spark/scheduler/Task.scala | 6 +- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../org/apache/spark/CacheManagerSuite.scala | 2 +- 7 files changed, 284 insertions(+), 131 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/TaskContext.java delete mode 100644 core/src/main/scala/org/apache/spark/TaskContext.scala diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java new file mode 100644 index 0000000000000..09b8ce02bd3d8 --- /dev/null +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -0,0 +1,274 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import scala.Function0; +import scala.Function1; +import scala.Unit; +import scala.collection.JavaConversions; + +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.executor.TaskMetrics; +import org.apache.spark.util.TaskCompletionListener; +import org.apache.spark.util.TaskCompletionListenerException; + +/** +* :: DeveloperApi :: +* Contextual information about a task which can be read or mutated during execution. +*/ +@DeveloperApi +public class TaskContext implements Serializable { + + private int stageId; + private int partitionId; + private long attemptId; + private boolean runningLocally; + private TaskMetrics taskMetrics; + + /** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + * @param runningLocally whether the task is running locally in the driver JVM + * @param taskMetrics performance metrics of the task + */ + @DeveloperApi + public TaskContext(Integer stageId, Integer partitionId, Long attemptId, Boolean runningLocally, + TaskMetrics taskMetrics) { + this.attemptId = attemptId; + this.partitionId = partitionId; + this.runningLocally = runningLocally; + this.stageId = stageId; + this.taskMetrics = taskMetrics; + } + + + /** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + * @param runningLocally whether the task is running locally in the driver JVM + */ + @DeveloperApi + public TaskContext(Integer stageId, Integer partitionId, Long attemptId, + Boolean runningLocally) { + this.attemptId = attemptId; + this.partitionId = partitionId; + this.runningLocally = runningLocally; + this.stageId = stageId; + this.taskMetrics = TaskMetrics.empty(); + } + + + /** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + * + * @param stageId stage id + * @param partitionId index of the partition + * @param attemptId the number of attempts to execute this task + */ + @DeveloperApi + public TaskContext(Integer stageId, Integer partitionId, Long attemptId) { + this.attemptId = attemptId; + this.partitionId = partitionId; + this.runningLocally = false; + this.stageId = stageId; + this.taskMetrics = TaskMetrics.empty(); + } + + private static ThreadLocal taskContext = + new ThreadLocal(); + + /** + * :: Internal API :: + * This is spark internal API, not intended to be called from user programs. + */ + public static void setTaskContext(TaskContext tc) { + taskContext.set(tc); + } + + public static TaskContext get() { + return taskContext.get(); + } + + /** + * :: Internal API :: + */ + public static void remove() { + taskContext.remove(); + } + + // List of callback functions to execute when the task completes. + private transient List onCompleteCallbacks = + new ArrayList(); + + // Whether the corresponding task has been killed. + private volatile Boolean interrupted = false; + + // Whether the task has completed. + private volatile Boolean completed = false; + + /** + * Checks whether the task has completed. + */ + public Boolean isCompleted() { + return completed; + } + + /** + * Checks whether the task has been killed. + */ + public Boolean isInterrupted() { + return interrupted; + } + + /** + * Add a (Java friendly) listener to be executed on task completion. + * This will be called in all situation - success, failure, or cancellation. + *

    + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + public TaskContext addTaskCompletionListener(TaskCompletionListener listener) { + onCompleteCallbacks.add(listener); + return this; + } + + /** + * Add a listener in the form of a Scala closure to be executed on task completion. + * This will be called in all situations - success, failure, or cancellation. + *

    + * An example use is for HadoopRDD to register a callback to close the input stream. + */ + public TaskContext addTaskCompletionListener(final Function1 f) { + onCompleteCallbacks.add(new TaskCompletionListener() { + @Override + public void onTaskCompletion(TaskContext context) { + f.apply(context); + } + }); + return this; + } + + /** + * Add a callback function to be executed on task completion. An example use + * is for HadoopRDD to register a callback to close the input stream. + * Will be called in any situation - success, failure, or cancellation. + * + * Deprecated: use addTaskCompletionListener + * + * @param f Callback function. + */ + @Deprecated + public void addOnCompleteCallback(final Function0 f) { + onCompleteCallbacks.add(new TaskCompletionListener() { + @Override + public void onTaskCompletion(TaskContext context) { + f.apply(); + } + }); + } + + /** + * ::Internal API:: + * Marks the task as completed and triggers the listeners. + */ + public void markTaskCompleted() throws TaskCompletionListenerException { + completed = true; + List errorMsgs = new ArrayList(2); + // Process complete callbacks in the reverse order of registration + List revlist = + new ArrayList(onCompleteCallbacks); + Collections.reverse(revlist); + for (TaskCompletionListener tcl: revlist) { + try { + tcl.onTaskCompletion(this); + } catch (Throwable e) { + errorMsgs.add(e.getMessage()); + } + } + + if (!errorMsgs.isEmpty()) { + throw new TaskCompletionListenerException(JavaConversions.asScalaBuffer(errorMsgs)); + } + } + + /** + * ::Internal API:: + * Marks the task for interruption, i.e. cancellation. + */ + public void markInterrupted() { + interrupted = true; + } + + @Deprecated + /** Deprecated: use getStageId() */ + public int stageId() { + return stageId; + } + + @Deprecated + /** Deprecated: use getPartitionId() */ + public int partitionId() { + return partitionId; + } + + @Deprecated + /** Deprecated: use getAttemptId() */ + public long attemptId() { + return attemptId; + } + + @Deprecated + /** Deprecated: use getRunningLocally() */ + public boolean runningLocally() { + return runningLocally; + } + + public boolean getRunningLocally() { + return runningLocally; + } + + public int getStageId() { + return stageId; + } + + public int getPartitionId() { + return partitionId; + } + + public long getAttemptId() { + return attemptId; + } + + /** ::Internal API:: */ + public TaskMetrics taskMetrics() { + return taskMetrics; + } +} diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala deleted file mode 100644 index 51b3e4d5e0936..0000000000000 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ /dev/null @@ -1,126 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.{TaskCompletionListenerException, TaskCompletionListener} - - -/** - * :: DeveloperApi :: - * Contextual information about a task which can be read or mutated during execution. - * - * @param stageId stage id - * @param partitionId index of the partition - * @param attemptId the number of attempts to execute this task - * @param runningLocally whether the task is running locally in the driver JVM - * @param taskMetrics performance metrics of the task - */ -@DeveloperApi -class TaskContext( - val stageId: Int, - val partitionId: Int, - val attemptId: Long, - val runningLocally: Boolean = false, - private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty) - extends Serializable with Logging { - - @deprecated("use partitionId", "0.8.1") - def splitId = partitionId - - // List of callback functions to execute when the task completes. - @transient private val onCompleteCallbacks = new ArrayBuffer[TaskCompletionListener] - - // Whether the corresponding task has been killed. - @volatile private var interrupted: Boolean = false - - // Whether the task has completed. - @volatile private var completed: Boolean = false - - /** Checks whether the task has completed. */ - def isCompleted: Boolean = completed - - /** Checks whether the task has been killed. */ - def isInterrupted: Boolean = interrupted - - // TODO: Also track whether the task has completed successfully or with exception. - - /** - * Add a (Java friendly) listener to be executed on task completion. - * This will be called in all situation - success, failure, or cancellation. - * - * An example use is for HadoopRDD to register a callback to close the input stream. - */ - def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { - onCompleteCallbacks += listener - this - } - - /** - * Add a listener in the form of a Scala closure to be executed on task completion. - * This will be called in all situation - success, failure, or cancellation. - * - * An example use is for HadoopRDD to register a callback to close the input stream. - */ - def addTaskCompletionListener(f: TaskContext => Unit): this.type = { - onCompleteCallbacks += new TaskCompletionListener { - override def onTaskCompletion(context: TaskContext): Unit = f(context) - } - this - } - - /** - * Add a callback function to be executed on task completion. An example use - * is for HadoopRDD to register a callback to close the input stream. - * Will be called in any situation - success, failure, or cancellation. - * @param f Callback function. - */ - @deprecated("use addTaskCompletionListener", "1.1.0") - def addOnCompleteCallback(f: () => Unit) { - onCompleteCallbacks += new TaskCompletionListener { - override def onTaskCompletion(context: TaskContext): Unit = f() - } - } - - /** Marks the task as completed and triggers the listeners. */ - private[spark] def markTaskCompleted(): Unit = { - completed = true - val errorMsgs = new ArrayBuffer[String](2) - // Process complete callbacks in the reverse order of registration - onCompleteCallbacks.reverse.foreach { listener => - try { - listener.onTaskCompletion(this) - } catch { - case e: Throwable => - errorMsgs += e.getMessage - logError("Error in TaskCompletionListener", e) - } - } - if (errorMsgs.nonEmpty) { - throw new TaskCompletionListenerException(errorMsgs) - } - } - - /** Marks the task for interruption, i.e. cancellation. */ - private[spark] def markInterrupted(): Unit = { - interrupted = true - } -} 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 0e90caa5c9ca7..ba712c9d7776f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -619,6 +619,7 @@ abstract class RDD[T: ClassTag]( * should be `false` unless this is a pair RDD and the input function doesn't modify the keys. */ @DeveloperApi + @deprecated("use TaskContext.get", "1.2.0") def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index b2774dfc47553..32cf29ed140e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -634,12 +634,14 @@ class DAGScheduler( val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) val taskContext = - new TaskContext(job.finalStage.id, job.partitions(0), 0, runningLocally = true) + new TaskContext(job.finalStage.id, job.partitions(0), 0, true) + TaskContext.setTaskContext(taskContext) try { val result = job.func(taskContext, rdd.iterator(split, taskContext)) job.listener.taskSucceeded(0, result) } finally { taskContext.markTaskCompleted() + TaskContext.remove() } } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 6aa0cca06878d..bf73f6f7bd0e1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -45,7 +45,8 @@ import org.apache.spark.util.Utils private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) extends Serializable { final def run(attemptId: Long): T = { - context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + context = new TaskContext(stageId, partitionId, attemptId, false) + TaskContext.setTaskContext(context) context.taskMetrics.hostname = Utils.localHostName() taskThread = Thread.currentThread() if (_killed) { @@ -92,7 +93,8 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex if (interruptThread && taskThread != null) { taskThread.interrupt() } - } + TaskContext.remove() + } } /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b8c23d524e00b..4a078435447e5 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -776,7 +776,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0, false, new TaskMetrics()); + TaskContext context = new TaskContext(0, 0, 0L, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.partitions().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 90dcadcffd091..d735010d7c9d5 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -94,7 +94,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar } whenExecuting(blockManager) { - val context = new TaskContext(0, 0, 0, runningLocally = true) + val context = new TaskContext(0, 0, 0, true) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } From a3feaf04dc35069b80233fe7cccd62fc3072fc1f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 26 Sep 2014 21:44:10 -0700 Subject: [PATCH 0801/1492] Close #2194. From e976ca236f3c5578d8d7639b788774b1053b65f7 Mon Sep 17 00:00:00 2001 From: Sarah Gerweck Date: Fri, 26 Sep 2014 22:21:50 -0700 Subject: [PATCH 0802/1492] Slaves file is now a template. Change 0dc868e removed the `conf/slaves` file and made it a template like most of the other configuration files. This means you can no longer run `make-distribution.sh` unless you manually create a slaves file to be statically bundled in your distribution, which seems at odds with making it a template file. Author: Sarah Gerweck Closes #2549 from sarahgerweck/noMoreSlaves and squashes the following commits: d11d99a [Sarah Gerweck] Slaves file is now a template. --- make-distribution.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 884659954a491..0bc839e1dbe4d 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -201,7 +201,6 @@ fi # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf -cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp "$FWDIR/README.md" "$DISTDIR" cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" From 0cdcdd2c9df98fb64d9d16ebace992fbba9c16b4 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 26 Sep 2014 22:23:49 -0700 Subject: [PATCH 0803/1492] [Build]remove spark-staging-1030 Since 1.1.0 has published, remove spark-staging-1030. Author: wangfei Closes #2532 from scwf/patch-2 and squashes the following commits: bc9e00b [wangfei] remove spark-staging-1030 --- pom.xml | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/pom.xml b/pom.xml index f3de097b9cb32..70cb9729ff6d3 100644 --- a/pom.xml +++ b/pom.xml @@ -222,18 +222,6 @@ false - - - spark-staging-1030 - Spark 1.1.0 Staging (1030) - https://repository.apache.org/content/repositories/orgapachespark-1030/ - - true - - - false - - From f0eea76d941c487763febbd9162600f89cedbd5c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 26 Sep 2014 22:24:34 -0700 Subject: [PATCH 0804/1492] [SQL][DOCS] Clarify that the server is for JDBC and ODBC Author: Michael Armbrust Closes #2527 from marmbrus/patch-1 and squashes the following commits: a0f9f1c [Michael Armbrust] [SQL][DOCS] Clarify that the server is for JDBC and ODBC --- docs/sql-programming-guide.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c1f80544bf0af..65249808fae3e 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -872,12 +872,12 @@ that these options will be deprecated in future release as more optimizations ar Spark SQL also supports interfaces for running SQL queries directly without the need to write any code. -## Running the Thrift JDBC server +## Running the Thrift JDBC/ODBC server -The Thrift JDBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) +The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) in Hive 0.12. You can test the JDBC server with the beeline script that comes with either Spark or Hive 0.12. -To start the JDBC server, run the following in the Spark directory: +To start the JDBC/ODBC server, run the following in the Spark directory: ./sbin/start-thriftserver.sh @@ -906,11 +906,11 @@ or system properties: ``` {% endhighlight %} -Now you can use beeline to test the Thrift JDBC server: +Now you can use beeline to test the Thrift JDBC/ODBC server: ./bin/beeline -Connect to the JDBC server in beeline with: +Connect to the JDBC/ODBC server in beeline with: beeline> !connect jdbc:hive2://localhost:10000 From d8a9d1d442dd5612f82edaf2a780579c4d43dcfd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 26 Sep 2014 22:30:12 -0700 Subject: [PATCH 0805/1492] [SPARK-3675][SQL] Allow starting a JDBC server on an existing context Author: Michael Armbrust Closes #2515 from marmbrus/jdbcExistingContext and squashes the following commits: 7866fad [Michael Armbrust] Allows starting a JDBC server on an existing context. --- .../sql/hive/thriftserver/HiveThriftServer2.scala | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index cadf7aaf42157..3d468d804622c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -26,6 +26,7 @@ import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ @@ -33,9 +34,21 @@ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a * `HiveThriftServer2` thrift server. */ -private[hive] object HiveThriftServer2 extends Logging { +object HiveThriftServer2 extends Logging { var LOG = LogFactory.getLog(classOf[HiveServer2]) + /** + * :: DeveloperApi :: + * Starts a new thrift server with the given context. + */ + @DeveloperApi + def startWithContext(sqlContext: HiveContext): Unit = { + val server = new HiveThriftServer2(sqlContext) + server.init(sqlContext.hiveconf) + server.start() + } + + def main(args: Array[String]) { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") From 9e8ced7847d84d63f0da08b15623d558a2407583 Mon Sep 17 00:00:00 2001 From: Jeff Steinmetz Date: Fri, 26 Sep 2014 23:00:40 -0700 Subject: [PATCH 0806/1492] stop, start and destroy require the EC2_REGION i.e ./spark-ec2 --region=us-west-1 stop yourclustername Author: Jeff Steinmetz Closes #2473 from jeffsteinmetz/master and squashes the following commits: 7491f2c [Jeff Steinmetz] fix case in EC2 cluster setup documentation bd3d777 [Jeff Steinmetz] standardized ec2 documenation to use sample args 2bf4a57 [Jeff Steinmetz] standardized ec2 documenation to use sample args 68d8372 [Jeff Steinmetz] standardized ec2 documenation to use sample args d2ab6e2 [Jeff Steinmetz] standardized ec2 documenation to use sample args 520e6dc [Jeff Steinmetz] standardized ec2 documenation to use sample args 37fc876 [Jeff Steinmetz] stop, start and destroy require the EC2_REGION --- docs/ec2-scripts.md | 29 +++++++++++++++++++---------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index b2ca6a9b48f32..530798f2b8022 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -48,6 +48,15 @@ by looking for the "Name" tag of the instance in the Amazon EC2 Console. key pair, `` is the number of slave nodes to launch (try 1 at first), and `` is the name to give to your cluster. + + For example: + + ```bash + export AWS_SECRET_ACCESS_KEY=AaBbCcDdEeFGgHhIiJjKkLlMmNnOoPpQqRrSsTtU +export AWS_ACCESS_KEY_ID=ABCDEFG1234567890123 +./spark-ec2 --key-pair=awskey --identity-file=awskey.pem --region=us-west-1 --zone=us-west-1a --spark-version=1.1.0 launch my-spark-cluster + ``` + - After everything launches, check that the cluster scheduler is up and sees all the slaves by going to its web UI, which will be printed at the end of the script (typically `http://:8080`). @@ -55,27 +64,27 @@ by looking for the "Name" tag of the instance in the Amazon EC2 Console. You can also run `./spark-ec2 --help` to see more usage options. The following options are worth pointing out: -- `--instance-type=` can be used to specify an EC2 +- `--instance-type=` can be used to specify an EC2 instance type to use. For now, the script only supports 64-bit instance types, and the default type is `m1.large` (which has 2 cores and 7.5 GB RAM). Refer to the Amazon pages about [EC2 instance types](http://aws.amazon.com/ec2/instance-types) and [EC2 pricing](http://aws.amazon.com/ec2/#pricing) for information about other instance types. -- `--region=` specifies an EC2 region in which to launch +- `--region=` specifies an EC2 region in which to launch instances. The default region is `us-east-1`. -- `--zone=` can be used to specify an EC2 availability zone +- `--zone=` can be used to specify an EC2 availability zone to launch instances in. Sometimes, you will get an error because there is not enough capacity in one zone, and you should try to launch in another. -- `--ebs-vol-size=GB` will attach an EBS volume with a given amount +- `--ebs-vol-size=` will attach an EBS volume with a given amount of space to each node so that you can have a persistent HDFS cluster on your nodes across cluster restarts (see below). -- `--spot-price=PRICE` will launch the worker nodes as +- `--spot-price=` will launch the worker nodes as [Spot Instances](http://aws.amazon.com/ec2/spot-instances/), bidding for the given maximum price (in dollars). -- `--spark-version=VERSION` will pre-load the cluster with the - specified version of Spark. VERSION can be a version number +- `--spark-version=` will pre-load the cluster with the + specified version of Spark. The `` can be a version number (e.g. "0.7.3") or a specific git hash. By default, a recent version will be used. - If one of your launches fails due to e.g. not having the right @@ -137,11 +146,11 @@ cost you any EC2 cycles, but ***will*** continue to cost money for EBS storage. - To stop one of your clusters, go into the `ec2` directory and run -`./spark-ec2 stop `. +`./spark-ec2 --region= stop `. - To restart it later, run -`./spark-ec2 -i start `. +`./spark-ec2 -i --region= start `. - To ultimately destroy the cluster and stop consuming EBS space, run -`./spark-ec2 destroy ` as described in the previous +`./spark-ec2 --region= destroy ` as described in the previous section. # Limitations From 2d972fd84ac54a89e416442508a6d4eaeff452c1 Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Fri, 26 Sep 2014 23:15:10 -0700 Subject: [PATCH 0807/1492] [SPARK-1021] Defer the data-driven computation of partition bounds in so... ...rtByKey() until evaluation. Author: Erik Erlandson Closes #1689 from erikerlandson/spark-1021-pr and squashes the following commits: 50b6da6 [Erik Erlandson] use standard getIteratorSize in countAsync 4e334a9 [Erik Erlandson] exception mystery fixed by fixing bug in ComplexFutureAction b88b5d4 [Erik Erlandson] tweak async actions to use ComplexFutureAction[T] so they handle RangePartitioner sampling job properly b2b20e8 [Erik Erlandson] Fix bug in exception passing with ComplexFutureAction[T] ca8913e [Erik Erlandson] RangePartition sampling job -> FutureAction 7143f97 [Erik Erlandson] [SPARK-1021] modify range bounds variable to be thread safe ac67195 [Erik Erlandson] [SPARK-1021] Defer the data-driven computation of partition bounds in sortByKey() until evaluation. --- .../scala/org/apache/spark/FutureAction.scala | 7 +- .../scala/org/apache/spark/Partitioner.scala | 29 +++++++-- .../apache/spark/rdd/AsyncRDDActions.scala | 64 +++++++++++-------- 3 files changed, 66 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 75ea535f2f57b..c277c3a47d421 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -208,7 +208,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { processPartition: Iterator[T] => U, partitions: Seq[Int], resultHandler: (Int, U) => Unit, - resultFunc: => R) { + resultFunc: => R): R = { // If the action hasn't been cancelled yet, submit the job. The check and the submitJob // command need to be in an atomic block. val job = this.synchronized { @@ -223,7 +223,10 @@ class ComplexFutureAction[T] extends FutureAction[T] { // cancel the job and stop the execution. This is not in a synchronized block because // Await.ready eventually waits on the monitor in FutureJob.jobWaiter. try { - Await.ready(job, Duration.Inf) + Await.ready(job, Duration.Inf).value.get match { + case scala.util.Failure(e) => throw e + case scala.util.Success(v) => v + } } catch { case e: InterruptedException => job.cancel() diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 37053bb6f37ad..d40b152d221c5 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -29,6 +29,10 @@ import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{CollectionsUtils, Utils} import org.apache.spark.util.random.{XORShiftRandom, SamplingUtils} +import org.apache.spark.SparkContext.rddToAsyncRDDActions +import scala.concurrent.Await +import scala.concurrent.duration.Duration + /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. @@ -113,8 +117,12 @@ class RangePartitioner[K : Ordering : ClassTag, V]( private var ordering = implicitly[Ordering[K]] // An array of upper bounds for the first (partitions - 1) partitions - private var rangeBounds: Array[K] = { - if (partitions <= 1) { + @volatile private var valRB: Array[K] = null + + private def rangeBounds: Array[K] = this.synchronized { + if (valRB != null) return valRB + + valRB = if (partitions <= 1) { Array.empty } else { // This is the sample size we need to have roughly balanced output partitions, capped at 1M. @@ -152,6 +160,8 @@ class RangePartitioner[K : Ordering : ClassTag, V]( RangePartitioner.determineBounds(candidates, partitions) } } + + valRB } def numPartitions = rangeBounds.length + 1 @@ -222,7 +232,8 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = this.synchronized { + if (valRB != null) return val sfactory = SparkEnv.get.serializer sfactory match { case js: JavaSerializer => in.defaultReadObject() @@ -234,7 +245,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( val ser = sfactory.newInstance() Utils.deserializeViaNestedStream(in, ser) { ds => implicit val classTag = ds.readObject[ClassTag[Array[K]]]() - rangeBounds = ds.readObject[Array[K]]() + valRB = ds.readObject[Array[K]]() } } } @@ -254,12 +265,18 @@ private[spark] object RangePartitioner { sampleSizePerPartition: Int): (Long, Array[(Int, Int, Array[K])]) = { val shift = rdd.id // val classTagK = classTag[K] // to avoid serializing the entire partitioner object - val sketched = rdd.mapPartitionsWithIndex { (idx, iter) => + // use collectAsync here to run this job as a future, which is cancellable + val sketchFuture = rdd.mapPartitionsWithIndex { (idx, iter) => val seed = byteswap32(idx ^ (shift << 16)) val (sample, n) = SamplingUtils.reservoirSampleAndCount( iter, sampleSizePerPartition, seed) Iterator((idx, n, sample)) - }.collect() + }.collectAsync() + // We do need the future's value to continue any further + val sketched = Await.ready(sketchFuture, Duration.Inf).value.get match { + case scala.util.Success(v) => v.toArray + case scala.util.Failure(e) => throw e + } val numItems = sketched.map(_._2.toLong).sum (numItems, sketched) } diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index b62f3fbdc4a15..7a68b3afa8158 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag +import org.apache.spark.util.Utils import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} import org.apache.spark.annotation.Experimental @@ -38,29 +39,30 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi * Returns a future for counting the number of elements in the RDD. */ def countAsync(): FutureAction[Long] = { - val totalCount = new AtomicLong - self.context.submitJob( - self, - (iter: Iterator[T]) => { - var result = 0L - while (iter.hasNext) { - result += 1L - iter.next() - } - result - }, - Range(0, self.partitions.size), - (index: Int, data: Long) => totalCount.addAndGet(data), - totalCount.get()) + val f = new ComplexFutureAction[Long] + f.run { + val totalCount = new AtomicLong + f.runJob(self, + (iter: Iterator[T]) => Utils.getIteratorSize(iter), + Range(0, self.partitions.size), + (index: Int, data: Long) => totalCount.addAndGet(data), + totalCount.get()) + } } /** * Returns a future for retrieving all elements of this RDD. */ def collectAsync(): FutureAction[Seq[T]] = { - val results = new Array[Array[T]](self.partitions.size) - self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size), - (index, data) => results(index) = data, results.flatten.toSeq) + val f = new ComplexFutureAction[Seq[T]] + f.run { + val results = new Array[Array[T]](self.partitions.size) + f.runJob(self, + (iter: Iterator[T]) => iter.toArray, + Range(0, self.partitions.size), + (index: Int, data: Array[T]) => results(index) = data, + results.flatten.toSeq) + } } /** @@ -104,24 +106,34 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi } results.toSeq } - - f } /** * Applies a function f to all elements of this RDD. */ - def foreachAsync(f: T => Unit): FutureAction[Unit] = { - val cleanF = self.context.clean(f) - self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.size), - (index, data) => Unit, Unit) + def foreachAsync(expr: T => Unit): FutureAction[Unit] = { + val f = new ComplexFutureAction[Unit] + val exprClean = self.context.clean(expr) + f.run { + f.runJob(self, + (iter: Iterator[T]) => iter.foreach(exprClean), + Range(0, self.partitions.size), + (index: Int, data: Unit) => Unit, + Unit) + } } /** * Applies a function f to each partition of this RDD. */ - def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = { - self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size), - (index, data) => Unit, Unit) + def foreachPartitionAsync(expr: Iterator[T] => Unit): FutureAction[Unit] = { + val f = new ComplexFutureAction[Unit] + f.run { + f.runJob(self, + expr, + Range(0, self.partitions.size), + (index: Int, data: Unit) => Unit, + Unit) + } } } From 436a7730b6e7067f74b3739a3a412490003f7c4c Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 27 Sep 2014 00:57:26 -0700 Subject: [PATCH 0808/1492] Minor cleanup to tighten visibility and remove compilation warning. Author: Reynold Xin Closes #2555 from rxin/cleanup and squashes the following commits: 6add199 [Reynold Xin] Minor cleanup to tighten visibility and remove compilation warning. --- .../input/WholeTextFileRecordReader.scala | 24 +++++----- .../apache/spark/metrics/MetricsSystem.scala | 28 ++++++----- .../spark/metrics/MetricsSystemSuite.scala | 33 +++++++------ .../streaming/StreamingContextSuite.scala | 47 ++++++++++--------- 4 files changed, 70 insertions(+), 62 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala index c3dabd2e79995..3564ab2e2a162 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -36,33 +36,31 @@ private[spark] class WholeTextFileRecordReader( index: Integer) extends RecordReader[String, String] { - private val path = split.getPath(index) - private val fs = path.getFileSystem(context.getConfiguration) + private[this] val path = split.getPath(index) + private[this] val fs = path.getFileSystem(context.getConfiguration) // True means the current file has been processed, then skip it. - private var processed = false + private[this] var processed = false - private val key = path.toString - private var value: String = null + private[this] val key = path.toString + private[this] var value: String = null - override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + override def initialize(split: InputSplit, context: TaskAttemptContext): Unit = {} - override def close() = {} + override def close(): Unit = {} - override def getProgress = if (processed) 1.0f else 0.0f + override def getProgress: Float = if (processed) 1.0f else 0.0f - override def getCurrentKey = key + override def getCurrentKey: String = key - override def getCurrentValue = value + override def getCurrentValue: String = value - override def nextKeyValue = { + override def nextKeyValue(): Boolean = { if (!processed) { val fileIn = fs.open(path) val innerBuffer = ByteStreams.toByteArray(fileIn) - value = new Text(innerBuffer).toString Closeables.close(fileIn, false) - processed = true true } else { diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 6ef817d0e587e..fd316a89a1a10 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -63,15 +63,18 @@ import org.apache.spark.metrics.source.Source * * [options] is the specific property of this source or sink. */ -private[spark] class MetricsSystem private (val instance: String, - conf: SparkConf, securityMgr: SecurityManager) extends Logging { +private[spark] class MetricsSystem private ( + val instance: String, + conf: SparkConf, + securityMgr: SecurityManager) + extends Logging { - val confFile = conf.get("spark.metrics.conf", null) - val metricsConfig = new MetricsConfig(Option(confFile)) + private[this] val confFile = conf.get("spark.metrics.conf", null) + private[this] val metricsConfig = new MetricsConfig(Option(confFile)) - val sinks = new mutable.ArrayBuffer[Sink] - val sources = new mutable.ArrayBuffer[Source] - val registry = new MetricRegistry() + private val sinks = new mutable.ArrayBuffer[Sink] + private val sources = new mutable.ArrayBuffer[Source] + private val registry = new MetricRegistry() // Treat MetricsServlet as a special sink as it should be exposed to add handlers to web ui private var metricsServlet: Option[MetricsServlet] = None @@ -91,7 +94,7 @@ private[spark] class MetricsSystem private (val instance: String, sinks.foreach(_.stop) } - def report(): Unit = { + def report() { sinks.foreach(_.report()) } @@ -155,8 +158,8 @@ private[spark] object MetricsSystem { val SINK_REGEX = "^sink\\.(.+)\\.(.+)".r val SOURCE_REGEX = "^source\\.(.+)\\.(.+)".r - val MINIMAL_POLL_UNIT = TimeUnit.SECONDS - val MINIMAL_POLL_PERIOD = 1 + private[this] val MINIMAL_POLL_UNIT = TimeUnit.SECONDS + private[this] val MINIMAL_POLL_PERIOD = 1 def checkMinimalPollingPeriod(pollUnit: TimeUnit, pollPeriod: Int) { val period = MINIMAL_POLL_UNIT.convert(pollPeriod, pollUnit) @@ -166,7 +169,8 @@ private[spark] object MetricsSystem { } } - def createMetricsSystem(instance: String, conf: SparkConf, - securityMgr: SecurityManager): MetricsSystem = + def createMetricsSystem( + instance: String, conf: SparkConf, securityMgr: SecurityManager): MetricsSystem = { new MetricsSystem(instance, conf, securityMgr) + } } diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 96a5a1231813e..e42b181194727 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -17,42 +17,47 @@ package org.apache.spark.metrics -import org.scalatest.{BeforeAndAfter, FunSuite} +import org.apache.spark.metrics.source.Source +import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} + import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.master.MasterSource -class MetricsSystemSuite extends FunSuite with BeforeAndAfter { +import scala.collection.mutable.ArrayBuffer + + +class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester{ var filePath: String = _ var conf: SparkConf = null var securityMgr: SecurityManager = null before { - filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() + filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile conf = new SparkConf(false).set("spark.metrics.conf", filePath) securityMgr = new SecurityManager(conf) } test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default", conf, securityMgr) - val sources = metricsSystem.sources - val sinks = metricsSystem.sinks + val sources = PrivateMethod[ArrayBuffer[Source]]('sources) + val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(sources.length === 0) - assert(sinks.length === 0) - assert(!metricsSystem.getServletHandlers.isEmpty) + assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sinks()).length === 0) + assert(metricsSystem.getServletHandlers.nonEmpty) } test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test", conf, securityMgr) - val sources = metricsSystem.sources - val sinks = metricsSystem.sinks + val sources = PrivateMethod[ArrayBuffer[Source]]('sources) + val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) - assert(sources.length === 0) - assert(sinks.length === 1) - assert(!metricsSystem.getServletHandlers.isEmpty) + assert(metricsSystem.invokePrivate(sources()).length === 0) + assert(metricsSystem.invokePrivate(sinks()).length === 1) + assert(metricsSystem.getServletHandlers.nonEmpty) val source = new MasterSource(null) metricsSystem.registerSource(source) - assert(sources.length === 1) + assert(metricsSystem.invokePrivate(sources()).length === 1) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index ebf83748ffa28..655cec1573f58 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -19,18 +19,18 @@ package org.apache.spark.streaming import java.util.concurrent.atomic.AtomicInteger -import scala.language.postfixOps +import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.Eventually._ +import org.scalatest.exceptions.TestFailedDueToTimeoutException +import org.scalatest.time.SpanSugar._ import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.util.Utils -import org.scalatest.{Assertions, BeforeAndAfter, FunSuite} -import org.scalatest.concurrent.Timeouts -import org.scalatest.concurrent.Eventually._ -import org.scalatest.exceptions.TestFailedDueToTimeoutException -import org.scalatest.time.SpanSugar._ + class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { @@ -68,7 +68,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from no conf + spark home + env") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil, Map(envPair)) - assert(ssc.conf.getExecutorEnv.exists(_ == envPair)) + assert(ssc.conf.getExecutorEnv.contains(envPair)) } test("from conf with settings") { @@ -94,7 +94,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", "10") val ssc1 = new StreamingContext(myConf, batchDuration) - addInputStream(ssc1).register + addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") @@ -107,7 +107,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("start and stop state check") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() assert(ssc.state === ssc.StreamingContextState.Initialized) ssc.start() @@ -118,7 +118,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("start multiple times") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() intercept[SparkException] { ssc.start() @@ -127,7 +127,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop multiple times") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() ssc.stop() ssc.stop() @@ -135,7 +135,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop before start and start after stop") { ssc = new StreamingContext(master, appName, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.stop() // stop before start should not throw exception ssc.start() ssc.stop() @@ -147,12 +147,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop only streaming context") { ssc = new StreamingContext(master, appName, batchDuration) sc = ssc.sparkContext - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() - ssc.stop(false) + ssc.stop(stopSparkContext = false) assert(sc.makeRDD(1 to 100).collect().size === 100) ssc = new StreamingContext(sc, batchDuration) - addInputStream(ssc).register + addInputStream(ssc).register() ssc.start() ssc.stop() } @@ -167,11 +167,11 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w var runningCount = 0 TestReceiver.counter.set(1) val input = ssc.receiverStream(new TestReceiver) - input.count.foreachRDD(rdd => { + input.count().foreachRDD { rdd => val count = rdd.first() runningCount += count.toInt logInfo("Count = " + count + ", Running count = " + runningCount) - }) + } ssc.start() ssc.awaitTermination(500) ssc.stop(stopSparkContext = false, stopGracefully = true) @@ -191,7 +191,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("awaitTermination") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.map(x => x).register + inputStream.map(x => x).register() // test whether start() blocks indefinitely or not failAfter(2000 millis) { @@ -215,7 +215,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w // test whether wait exits if context is stopped failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown new Thread() { - override def run { + override def run() { Thread.sleep(500) ssc.stop() } @@ -239,8 +239,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("awaitTermination with error in task") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.map(x => { throw new TestException("error in map task"); x}) - .foreachRDD(_.count) + inputStream + .map { x => throw new TestException("error in map task"); x } + .foreachRDD(_.count()) val exception = intercept[Exception] { ssc.start() @@ -252,7 +253,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("awaitTermination with error in job generation") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.transform(rdd => { throw new TestException("error in transform"); rdd }).register + inputStream.transform { rdd => throw new TestException("error in transform"); rdd }.register() val exception = intercept[TestException] { ssc.start() ssc.awaitTermination(5000) @@ -265,7 +266,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } def addInputStream(s: StreamingContext): DStream[Int] = { - val input = (1 to 100).map(i => (1 to i)) + val input = (1 to 100).map(i => 1 to i) val inputStream = new TestInputStream(s, input, 1) inputStream } From 66107f46f374f83729cd79ab260eb59fa123c041 Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Sat, 27 Sep 2014 09:41:04 -0700 Subject: [PATCH 0809/1492] Docs : use "--total-executor-cores" rather than "--cores" after spark-shell Author: CrazyJvm Closes #2540 from CrazyJvm/standalone-core and squashes the following commits: 66d9fc6 [CrazyJvm] use "--total-executor-cores" rather than "--cores" after spark-shell --- docs/spark-standalone.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 58103fab20819..a3028aa86dc45 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -247,7 +247,7 @@ To run an interactive Spark shell against the cluster, run the following command ./bin/spark-shell --master spark://IP:PORT -You can also pass an option `--cores ` to control the number of cores that spark-shell uses on the cluster. +You can also pass an option `--total-executor-cores ` to control the number of cores that spark-shell uses on the cluster. # Launching Compiled Spark Applications From 0800881051df8029afb22a4ec17970e316a85855 Mon Sep 17 00:00:00 2001 From: w00228970 Date: Sat, 27 Sep 2014 12:06:06 -0700 Subject: [PATCH 0810/1492] [SPARK-3676][SQL] Fix hive test suite failure due to diffs in JDK 1.6/1.7 This is a bug in JDK6: http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4428022 this is because jdk get different result to operate ```double```, ```System.out.println(1/500d)``` in different jdk get different result jdk 1.6.0(_31) ---- 0.0020 jdk 1.7.0(_05) ---- 0.002 this leads to HiveQuerySuite failed when generate golden answer in jdk 1.7 and run tests in jdk 1.6, result did not match Author: w00228970 Closes #2517 from scwf/HiveQuerySuite and squashes the following commits: 0cb5e8d [w00228970] delete golden answer of division-0 and timestamp cast #1 1df3964 [w00228970] Jdk version leads to different query output for Double, this make HiveQuerySuite failed --- .../division-0-63b19f8a22471c8ba0415c1d3bc276f7 | 1 - ...amp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 | 1 - .../spark/sql/hive/execution/HiveQuerySuite.scala | 15 +++++++++++---- 3 files changed, 11 insertions(+), 6 deletions(-) delete mode 100644 sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 diff --git a/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 b/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 deleted file mode 100644 index 7b7a9175114ce..0000000000000 --- a/sql/hive/src/test/resources/golden/division-0-63b19f8a22471c8ba0415c1d3bc276f7 +++ /dev/null @@ -1 +0,0 @@ -2.0 0.5 0.3333333333333333 0.002 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 b/sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 deleted file mode 100644 index 8ebf695ba7d20..0000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #1-0-69fc614ccea92bbe39f4decc299edcc6 +++ /dev/null @@ -1 +0,0 @@ -0.001 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2f876cafaf218..2da8a6fac3d99 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -135,8 +135,12 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("div", "SELECT 1 DIV 2, 1 div 2, 1 dIv 2, 100 DIV 51, 100 DIV 49 FROM src LIMIT 1") - createQueryTest("division", - "SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1") + // Jdk version leads to different query output for double, so not use createQueryTest here + test("division") { + val res = sql("SELECT 2 / 1, 1 / 2, 1 / 3, 1 / COUNT(*) FROM src LIMIT 1").collect().head + Seq(2.0, 0.5, 0.3333333333333333, 0.002).zip(res).foreach( x => + assert(x._1 == x._2.asInstanceOf[Double])) + } createQueryTest("modulus", "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), (101 / 2) % 10 FROM src LIMIT 1") @@ -306,8 +310,11 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("case statements WITHOUT key #4", "SELECT (CASE WHEN key > 2 THEN 3 WHEN 2 > key THEN 2 ELSE 0 END) FROM src WHERE key < 15") - createQueryTest("timestamp cast #1", - "SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") + // Jdk version leads to different query output for double, so not use createQueryTest here + test("timestamp cast #1") { + val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + assert(0.001 == res.getDouble(0)) + } createQueryTest("timestamp cast #2", "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") From f0c7e19550d46f81a0a3ff272bbf66ce4bafead6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 27 Sep 2014 12:10:16 -0700 Subject: [PATCH 0811/1492] [SPARK-3680][SQL] Fix bug caused by eager typing of HiveGenericUDFs Typing of UDFs should be lazy as it is often not valid to call `dataType` on an expression until after all of its children are `resolved`. Author: Michael Armbrust Closes #2525 from marmbrus/concatBug and squashes the following commits: 5b8efe7 [Michael Armbrust] fix bug with eager typing of udfs --- .../org/apache/spark/sql/hive/hiveUdfs.scala | 2 +- .../spark/sql/parquet/ParquetMetastoreSuite.scala | 15 +++++++++++---- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 68944ed4ef21d..732e4976f6843 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -151,7 +151,7 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq override def get(): AnyRef = wrap(func()) } - val dataType: DataType = inspectorToDataType(returnInspector) + lazy val dataType: DataType = inspectorToDataType(returnInspector) override def eval(input: Row): Any = { returnInspector // Make sure initialized. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala index e380280f301c1..86adbbf3ad2d8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/ParquetMetastoreSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.parquet import java.io.File +import org.apache.spark.sql.catalyst.expressions.Row import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.QueryTest @@ -142,15 +143,21 @@ class ParquetMetastoreSuite extends QueryTest with BeforeAndAfterAll { test("sum") { checkAnswer( sql("SELECT SUM(intField) FROM partitioned_parquet WHERE intField IN (1,2,3) AND p = 1"), - 1 + 2 + 3 - ) + 1 + 2 + 3) + } + + test("hive udfs") { + checkAnswer( + sql("SELECT concat(stringField, stringField) FROM partitioned_parquet"), + sql("SELECT stringField FROM partitioned_parquet").map { + case Row(s: String) => Row(s + s) + }.collect().toSeq) } test("non-part select(*)") { checkAnswer( sql("SELECT COUNT(*) FROM normal_parquet"), - 10 - ) + 10) } test("conversion is working") { From 0d8cdf0ede908f6c488a075170f1563815009e29 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 27 Sep 2014 12:21:37 -0700 Subject: [PATCH 0812/1492] [SPARK-3681] [SQL] [PySpark] fix serialization of List and Map in SchemaRDD Currently, the schema of object in ArrayType or MapType is attached lazily, it will have better performance but introduce issues while serialization or accessing nested objects. This patch will apply schema to the objects of ArrayType or MapType immediately when accessing them, will be a little bit slower, but much robust. Author: Davies Liu Closes #2526 from davies/nested and squashes the following commits: 2399ae5 [Davies Liu] fix serialization of List and Map in SchemaRDD --- python/pyspark/sql.py | 40 +++++++++++++--------------------------- python/pyspark/tests.py | 21 +++++++++++++++++++++ 2 files changed, 34 insertions(+), 27 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 653195ea438cf..f71d24c470dc9 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -838,43 +838,29 @@ def _create_cls(dataType): >>> obj = _create_cls(schema)(row) >>> pickle.loads(pickle.dumps(obj)) Row(a=[1], b={'key': Row(c=1, d=2.0)}) + >>> pickle.loads(pickle.dumps(obj.a)) + [1] + >>> pickle.loads(pickle.dumps(obj.b)) + {'key': Row(c=1, d=2.0)} """ if isinstance(dataType, ArrayType): cls = _create_cls(dataType.elementType) - class List(list): - - def __getitem__(self, i): - # create object with datetype - return _create_object(cls, list.__getitem__(self, i)) - - def __repr__(self): - # call collect __repr__ for nested objects - return "[%s]" % (", ".join(repr(self[i]) - for i in range(len(self)))) - - def __reduce__(self): - return list.__reduce__(self) + def List(l): + if l is None: + return + return [_create_object(cls, v) for v in l] return List elif isinstance(dataType, MapType): - vcls = _create_cls(dataType.valueType) - - class Dict(dict): - - def __getitem__(self, k): - # create object with datetype - return _create_object(vcls, dict.__getitem__(self, k)) - - def __repr__(self): - # call collect __repr__ for nested objects - return "{%s}" % (", ".join("%r: %r" % (k, self[k]) - for k in self)) + cls = _create_cls(dataType.valueType) - def __reduce__(self): - return dict.__reduce__(self) + def Dict(d): + if d is None: + return + return dict((k, _create_object(cls, v)) for k, v in d.items()) return Dict diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index d1bb2033b7a16..29df754c6fd29 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -698,6 +698,27 @@ def test_apply_schema_to_row(self): srdd3 = self.sqlCtx.applySchema(rdd, srdd.schema()) self.assertEqual(10, srdd3.count()) + def test_serialize_nested_array_and_map(self): + d = [Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")})] + rdd = self.sc.parallelize(d) + srdd = self.sqlCtx.inferSchema(rdd) + row = srdd.first() + self.assertEqual(1, len(row.l)) + self.assertEqual(1, row.l[0].a) + self.assertEqual("2", row.d["key"].d) + + l = srdd.map(lambda x: x.l).first() + self.assertEqual(1, len(l)) + self.assertEqual('s', l[0].b) + + d = srdd.map(lambda x: x.d).first() + self.assertEqual(1, len(d)) + self.assertEqual(1.0, d["key"].c) + + row = srdd.map(lambda x: x.d["key"]).first() + self.assertEqual(1.0, row.c) + self.assertEqual("2", row.d) + class TestIO(PySparkTestCase): From 5b922bb458e863f5be0ae68167de882743f70b86 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 27 Sep 2014 14:46:00 -0700 Subject: [PATCH 0813/1492] [SPARK-3543] Clean up Java TaskContext implementation. This addresses some minor issues in https://github.com/apache/spark/pull/2425 Author: Reynold Xin Closes #2557 from rxin/TaskContext and squashes the following commits: a51e5f6 [Reynold Xin] [SPARK-3543] Clean up Java TaskContext implementation. --- .../java/org/apache/spark/TaskContext.java | 33 ++++++++----------- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/ResultTask.scala | 6 +--- .../spark/scheduler/ShuffleMapTask.scala | 2 -- .../org/apache/spark/scheduler/Task.scala | 8 +++-- 5 files changed, 22 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java index 09b8ce02bd3d8..4e6d708af0ea7 100644 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -56,7 +56,7 @@ public class TaskContext implements Serializable { * @param taskMetrics performance metrics of the task */ @DeveloperApi - public TaskContext(Integer stageId, Integer partitionId, Long attemptId, Boolean runningLocally, + public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally, TaskMetrics taskMetrics) { this.attemptId = attemptId; this.partitionId = partitionId; @@ -65,7 +65,6 @@ public TaskContext(Integer stageId, Integer partitionId, Long attemptId, Boolean this.taskMetrics = taskMetrics; } - /** * :: DeveloperApi :: * Contextual information about a task which can be read or mutated during execution. @@ -76,8 +75,7 @@ public TaskContext(Integer stageId, Integer partitionId, Long attemptId, Boolean * @param runningLocally whether the task is running locally in the driver JVM */ @DeveloperApi - public TaskContext(Integer stageId, Integer partitionId, Long attemptId, - Boolean runningLocally) { + public TaskContext(int stageId, int partitionId, long attemptId, boolean runningLocally) { this.attemptId = attemptId; this.partitionId = partitionId; this.runningLocally = runningLocally; @@ -85,7 +83,6 @@ public TaskContext(Integer stageId, Integer partitionId, Long attemptId, this.taskMetrics = TaskMetrics.empty(); } - /** * :: DeveloperApi :: * Contextual information about a task which can be read or mutated during execution. @@ -95,7 +92,7 @@ public TaskContext(Integer stageId, Integer partitionId, Long attemptId, * @param attemptId the number of attempts to execute this task */ @DeveloperApi - public TaskContext(Integer stageId, Integer partitionId, Long attemptId) { + public TaskContext(int stageId, int partitionId, long attemptId) { this.attemptId = attemptId; this.partitionId = partitionId; this.runningLocally = false; @@ -107,9 +104,9 @@ public TaskContext(Integer stageId, Integer partitionId, Long attemptId) { new ThreadLocal(); /** - * :: Internal API :: - * This is spark internal API, not intended to be called from user programs. - */ + * :: Internal API :: + * This is spark internal API, not intended to be called from user programs. + */ public static void setTaskContext(TaskContext tc) { taskContext.set(tc); } @@ -118,10 +115,8 @@ public static TaskContext get() { return taskContext.get(); } - /** - * :: Internal API :: - */ - public static void remove() { + /** :: Internal API :: */ + public static void unset() { taskContext.remove(); } @@ -130,22 +125,22 @@ public static void remove() { new ArrayList(); // Whether the corresponding task has been killed. - private volatile Boolean interrupted = false; + private volatile boolean interrupted = false; // Whether the task has completed. - private volatile Boolean completed = false; + private volatile boolean completed = false; /** * Checks whether the task has completed. */ - public Boolean isCompleted() { + public boolean isCompleted() { return completed; } /** * Checks whether the task has been killed. */ - public Boolean isInterrupted() { + public boolean isInterrupted() { return interrupted; } @@ -246,12 +241,12 @@ public long attemptId() { } @Deprecated - /** Deprecated: use getRunningLocally() */ + /** Deprecated: use isRunningLocally() */ public boolean runningLocally() { return runningLocally; } - public boolean getRunningLocally() { + public boolean isRunningLocally() { return runningLocally; } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 32cf29ed140e6..70c235dffff70 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -641,7 +641,7 @@ class DAGScheduler( job.listener.taskSucceeded(0, result) } finally { taskContext.markTaskCompleted() - TaskContext.remove() + TaskContext.unset() } } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 2ccbd8edeb028..4a9ff918afe25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -58,11 +58,7 @@ private[spark] class ResultTask[T, U]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) metrics = Some(context.taskMetrics) - try { - func(context, rdd.iterator(partition, context)) - } finally { - context.markTaskCompleted() - } + func(context, rdd.iterator(partition, context)) } // This is only callable on the driver side. diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index a98ee118254a3..79709089c0da4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -78,8 +78,6 @@ private[spark] class ShuffleMapTask( log.debug("Could not stop writer", e) } throw e - } finally { - context.markTaskCompleted() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index bf73f6f7bd0e1..c6e47c84a0cb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -52,7 +52,12 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex if (_killed) { kill(interruptThread = false) } - runTask(context) + try { + runTask(context) + } finally { + context.markTaskCompleted() + TaskContext.unset() + } } def runTask(context: TaskContext): T @@ -93,7 +98,6 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex if (interruptThread && taskThread != null) { taskThread.interrupt() } - TaskContext.remove() } } From 248232936e1bead7f102e59eb8faf3126c582d9d Mon Sep 17 00:00:00 2001 From: Uri Laserson Date: Sat, 27 Sep 2014 21:48:05 -0700 Subject: [PATCH 0814/1492] [SPARK-3389] Add Converter for ease of Parquet reading in PySpark https://issues.apache.org/jira/browse/SPARK-3389 Author: Uri Laserson Closes #2256 from laserson/SPARK-3389 and squashes the following commits: 0ed363e [Uri Laserson] PEP8'd the python file 0b4b380 [Uri Laserson] Moved converter to examples and added python example eecf4dc [Uri Laserson] [SPARK-3389] Add Converter for ease of Parquet reading in PySpark --- .../src/main/python/parquet_inputformat.py | 59 ++++++++++++++ examples/src/main/resources/full_user.avsc | 1 + examples/src/main/resources/users.parquet | Bin 0 -> 615 bytes .../pythonconverters/AvroConverters.scala | 76 +++++++++++------- 4 files changed, 106 insertions(+), 30 deletions(-) create mode 100644 examples/src/main/python/parquet_inputformat.py create mode 100644 examples/src/main/resources/full_user.avsc create mode 100644 examples/src/main/resources/users.parquet diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py new file mode 100644 index 0000000000000..c9b08f878a1e6 --- /dev/null +++ b/examples/src/main/python/parquet_inputformat.py @@ -0,0 +1,59 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import sys + +from pyspark import SparkContext + +""" +Read data file users.parquet in local Spark distro: + +$ cd $SPARK_HOME +$ export AVRO_PARQUET_JARS=/path/to/parquet-avro-1.5.0.jar +$ ./bin/spark-submit --driver-class-path /path/to/example/jar \\ + --jars $AVRO_PARQUET_JARS \\ + ./examples/src/main/python/parquet_inputformat.py \\ + examples/src/main/resources/users.parquet +<...lots of log output...> +{u'favorite_color': None, u'name': u'Alyssa', u'favorite_numbers': [3, 9, 15, 20]} +{u'favorite_color': u'red', u'name': u'Ben', u'favorite_numbers': []} +<...more log output...> +""" +if __name__ == "__main__": + if len(sys.argv) != 2: + print >> sys.stderr, """ + Usage: parquet_inputformat.py + + Run with example jar: + ./bin/spark-submit --driver-class-path /path/to/example/jar \\ + /path/to/examples/parquet_inputformat.py + Assumes you have Parquet data stored in . + """ + exit(-1) + + path = sys.argv[1] + sc = SparkContext(appName="ParquetInputFormat") + + parquet_rdd = sc.newAPIHadoopFile( + path, + 'parquet.avro.AvroParquetInputFormat', + 'java.lang.Void', + 'org.apache.avro.generic.IndexedRecord', + valueConverter='org.apache.spark.examples.pythonconverters.IndexedRecordToJavaConverter') + output = parquet_rdd.map(lambda x: x[1]).collect() + for k in output: + print k diff --git a/examples/src/main/resources/full_user.avsc b/examples/src/main/resources/full_user.avsc new file mode 100644 index 0000000000000..04e7ba2dca4f6 --- /dev/null +++ b/examples/src/main/resources/full_user.avsc @@ -0,0 +1 @@ +{"type": "record", "namespace": "example.avro", "name": "User", "fields": [{"type": "string", "name": "name"}, {"type": ["string", "null"], "name": "favorite_color"}, {"type": {"items": "int", "type": "array"}, "name": "favorite_numbers"}]} \ No newline at end of file diff --git a/examples/src/main/resources/users.parquet b/examples/src/main/resources/users.parquet new file mode 100644 index 0000000000000000000000000000000000000000..aa527338c43a8400fd56e549cb28aa1e6a9ccccf GIT binary patch literal 615 zcmZuv%WA?v6dhv>skOF(GbAMx8A#|N4V6|9aiOIPms03PD`l!<8_27ZC>8M^`h9*v zzoIu$LutDhxO2}r_i<*1{f8z-m}1MuG6X7C5vuhRgizmG#W5>FbjJgL&hf>LqokaZ zYYC8|l;VQV0B_^Ajmr=y801Dh!>c8wcbamJ;GDv#!@-jNG^p_p=0_fP*iwYfW6TA} zaK%KL95A1oK&zONR-LnDDBOfUPeU&hkZvLEEKddt|AmVfOQ~2gWv#@7U@Jsq-O#(1 zYT$})sz~1z#S)RpJsDWAfHh39mWmYpcax0PB|U2hw9kS8^O``r{L^;V4CrMtA|s$8 zM79MYBi+!Bv%TW!8~2&EEv#v>ia701!Ka~^QJbb)!ad!5e~TkFO;bOe0ch@WZx++e zczw`hQu|ObPJ|o0(v6+txjmU@P-546O!ri1zVJLc`A@QUG#BNAXU0Mr-ol4zs2e17 jvzcs=rbSG=FL-k0i^dXO!wrK*)46qS&=)u|gfI3D{z{mb literal 0 HcmV?d00001 diff --git a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala index 1b25983a38453..a11890d6f2b1c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala +++ b/examples/src/main/scala/org/apache/spark/examples/pythonconverters/AvroConverters.scala @@ -30,21 +30,28 @@ import org.apache.spark.api.python.Converter import org.apache.spark.SparkException -/** - * Implementation of [[org.apache.spark.api.python.Converter]] that converts - * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries - * to work with all 3 Avro data mappings (Generic, Specific and Reflect). - */ -class AvroWrapperToJavaConverter extends Converter[Any, Any] { - override def convert(obj: Any): Any = { +object AvroConversionUtil extends Serializable { + def fromAvro(obj: Any, schema: Schema): Any = { if (obj == null) { return null } - obj.asInstanceOf[AvroWrapper[_]].datum() match { - case null => null - case record: IndexedRecord => unpackRecord(record) - case other => throw new SparkException( - s"Unsupported top-level Avro data type ${other.getClass.getName}") + schema.getType match { + case UNION => unpackUnion(obj, schema) + case ARRAY => unpackArray(obj, schema) + case FIXED => unpackFixed(obj, schema) + case MAP => unpackMap(obj, schema) + case BYTES => unpackBytes(obj) + case RECORD => unpackRecord(obj) + case STRING => obj.toString + case ENUM => obj.toString + case NULL => obj + case BOOLEAN => obj + case DOUBLE => obj + case FLOAT => obj + case INT => obj + case LONG => obj + case other => throw new SparkException( + s"Unknown Avro schema type ${other.getName}") } } @@ -103,28 +110,37 @@ class AvroWrapperToJavaConverter extends Converter[Any, Any] { "Unions may only consist of a concrete type and null") } } +} - def fromAvro(obj: Any, schema: Schema): Any = { +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro IndexedRecord (e.g., derived from AvroParquetInputFormat) to a Java Map. + */ +class IndexedRecordToJavaConverter extends Converter[IndexedRecord, JMap[String, Any]]{ + override def convert(record: IndexedRecord): JMap[String, Any] = { + if (record == null) { + return null + } + val map = new java.util.HashMap[String, Any] + AvroConversionUtil.unpackRecord(record) + } +} + +/** + * Implementation of [[org.apache.spark.api.python.Converter]] that converts + * an Avro Record wrapped in an AvroKey (or AvroValue) to a Java Map. It tries + * to work with all 3 Avro data mappings (Generic, Specific and Reflect). + */ +class AvroWrapperToJavaConverter extends Converter[Any, Any] { + override def convert(obj: Any): Any = { if (obj == null) { return null } - schema.getType match { - case UNION => unpackUnion(obj, schema) - case ARRAY => unpackArray(obj, schema) - case FIXED => unpackFixed(obj, schema) - case MAP => unpackMap(obj, schema) - case BYTES => unpackBytes(obj) - case RECORD => unpackRecord(obj) - case STRING => obj.toString - case ENUM => obj.toString - case NULL => obj - case BOOLEAN => obj - case DOUBLE => obj - case FLOAT => obj - case INT => obj - case LONG => obj - case other => throw new SparkException( - s"Unknown Avro schema type ${other.getName}") + obj.asInstanceOf[AvroWrapper[_]].datum() match { + case null => null + case record: IndexedRecord => AvroConversionUtil.unpackRecord(record) + case other => throw new SparkException( + s"Unsupported top-level Avro data type ${other.getClass.getName}") } } } From 9966d1a8aaed3d8cfed93855959705ea3c677215 Mon Sep 17 00:00:00 2001 From: Dale Date: Sat, 27 Sep 2014 22:08:10 -0700 Subject: [PATCH 0815/1492] SPARK-CORE [SPARK-3651] Group common CoarseGrainedSchedulerBackend variables together from [SPARK-3651] In CoarseGrainedSchedulerBackend, we have: private val executorActor = new HashMap[String, ActorRef] private val executorAddress = new HashMap[String, Address] private val executorHost = new HashMap[String, String] private val freeCores = new HashMap[String, Int] private val totalCores = new HashMap[String, Int] We only ever put / remove stuff from these maps together. It would simplify the code if we consolidate these all into one map as we have done in JobProgressListener in https://issues.apache.org/jira/browse/SPARK-2299. Author: Dale Closes #2533 from tigerquoll/SPARK-3651 and squashes the following commits: d1be0a9 [Dale] [SPARK-3651] implemented suggested changes. Changed a reference from executorInfo to executorData to be consistent with other usages 6890663 [Dale] [SPARK-3651] implemented suggested changes 7d671cf [Dale] [SPARK-3651] Grouped variables under a ExecutorDataObject, and reference them via a map entry as they are all retrieved under the same key --- .../CoarseGrainedSchedulerBackend.scala | 68 ++++++++----------- .../scheduler/cluster/ExecutorData.scala | 38 +++++++++++ 2 files changed, 68 insertions(+), 38 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala 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 9a0cb1c6c6ccd..59e15edc75f5a 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 @@ -62,15 +62,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A val createTime = System.currentTimeMillis() class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { - override protected def log = CoarseGrainedSchedulerBackend.this.log - - private val executorActor = new HashMap[String, ActorRef] - private val executorAddress = new HashMap[String, Address] - private val executorHost = new HashMap[String, String] - private val freeCores = new HashMap[String, Int] - private val totalCores = new HashMap[String, Int] private val addressToExecutorId = new HashMap[Address, String] + private val executorDataMap = new HashMap[String, ExecutorData] override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() @@ -85,16 +79,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A def receiveWithLogging = { case RegisterExecutor(executorId, hostPort, cores) => Utils.checkHostPort(hostPort, "Host port expected " + hostPort) - if (executorActor.contains(executorId)) { + if (executorDataMap.contains(executorId)) { sender ! RegisterExecutorFailed("Duplicate executor ID: " + executorId) } else { logInfo("Registered executor: " + sender + " with ID " + executorId) sender ! RegisteredExecutor - executorActor(executorId) = sender - executorHost(executorId) = Utils.parseHostPort(hostPort)._1 - totalCores(executorId) = cores - freeCores(executorId) = cores - executorAddress(executorId) = sender.path.address + executorDataMap.put(executorId, new ExecutorData(sender, sender.path.address, + Utils.parseHostPort(hostPort)._1, cores, cores)) + addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) @@ -104,13 +96,14 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case StatusUpdate(executorId, taskId, state, data) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { - if (executorActor.contains(executorId)) { - freeCores(executorId) += scheduler.CPUS_PER_TASK - makeOffers(executorId) - } else { - // Ignoring the update since we don't know about the executor. - val msg = "Ignored task status update (%d state %s) from unknown executor %s with ID %s" - logWarning(msg.format(taskId, state, sender, executorId)) + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorInfo.freeCores += scheduler.CPUS_PER_TASK + makeOffers(executorId) + case None => + // Ignoring the update since we don't know about the executor. + logWarning(s"Ignored task status update ($taskId state $state) " + + "from unknown executor $sender with ID $executorId") } } @@ -118,7 +111,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A makeOffers() case KillTask(taskId, executorId, interruptThread) => - executorActor(executorId) ! KillTask(taskId, executorId, interruptThread) + executorDataMap(executorId).executorActor ! KillTask(taskId, executorId, interruptThread) case StopDriver => sender ! true @@ -126,8 +119,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case StopExecutors => logInfo("Asking each executor to shut down") - for (executor <- executorActor.values) { - executor ! StopExecutor + for ((_, executorData) <- executorDataMap) { + executorData.executorActor ! StopExecutor } sender ! true @@ -138,6 +131,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case AddWebUIFilter(filterName, filterParams, proxyBase) => addWebUIFilter(filterName, filterParams, proxyBase) sender ! true + case DisassociatedEvent(_, address, _) => addressToExecutorId.get(address).foreach(removeExecutor(_, "remote Akka client disassociated")) @@ -149,13 +143,15 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Make fake resource offers on all executors def makeOffers() { launchTasks(scheduler.resourceOffers( - executorHost.toArray.map {case (id, host) => new WorkerOffer(id, host, freeCores(id))})) + executorDataMap.map {case (id, executorData) => + new WorkerOffer(id, executorData.executorHost, executorData.freeCores)}.toSeq)) } // Make fake resource offers on just one executor def makeOffers(executorId: String) { + val executorData = executorDataMap(executorId) launchTasks(scheduler.resourceOffers( - Seq(new WorkerOffer(executorId, executorHost(executorId), freeCores(executorId))))) + Seq(new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores)))) } // Launch tasks returned by a set of resource offers @@ -179,25 +175,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } } else { - freeCores(task.executorId) -= scheduler.CPUS_PER_TASK - executorActor(task.executorId) ! LaunchTask(new SerializableBuffer(serializedTask)) + val executorData = executorDataMap(task.executorId) + executorData.freeCores -= scheduler.CPUS_PER_TASK + executorData.executorActor ! LaunchTask(new SerializableBuffer(serializedTask)) } } } // Remove a disconnected slave from the cluster def removeExecutor(executorId: String, reason: String) { - if (executorActor.contains(executorId)) { - logInfo("Executor " + executorId + " disconnected, so removing it") - val numCores = totalCores(executorId) - executorActor -= executorId - executorHost -= executorId - addressToExecutorId -= executorAddress(executorId) - executorAddress -= executorId - totalCores -= executorId - freeCores -= executorId - totalCoreCount.addAndGet(-numCores) - scheduler.executorLost(executorId, SlaveLost(reason)) + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorDataMap -= executorId + totalCoreCount.addAndGet(-executorInfo.totalCores) + scheduler.executorLost(executorId, SlaveLost(reason)) + case None => logError(s"Asked to remove non existant executor $executorId") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala new file mode 100644 index 0000000000000..74a92985b6629 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -0,0 +1,38 @@ +/* + * 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 akka.actor.{Address, ActorRef} + +/** + * Grouping of data that is accessed by a CourseGrainedScheduler. This class + * is stored in a Map keyed by an executorID + * + * @param executorActor The actorRef representing this executor + * @param executorAddress The network address of this executor + * @param executorHost The hostname that this executor is running on + * @param freeCores The current number of cores available for work on the executor + * @param totalCores The total number of cores available to the executor + */ +private[cluster] class ExecutorData( + val executorActor: ActorRef, + val executorAddress: Address, + val executorHost: String , + var freeCores: Int, + val totalCores: Int +) From 66e1c40c67f40dc4a5519812bc84877751933e7a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 27 Sep 2014 22:18:02 -0700 Subject: [PATCH 0816/1492] Minor fix for the previous commit. --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 6 +++--- .../org/apache/spark/scheduler/cluster/ExecutorData.scala | 5 ++--- 2 files changed, 5 insertions(+), 6 deletions(-) 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 59e15edc75f5a..89089e7d6f8a8 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 @@ -142,9 +142,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Make fake resource offers on all executors def makeOffers() { - launchTasks(scheduler.resourceOffers( - executorDataMap.map {case (id, executorData) => - new WorkerOffer(id, executorData.executorHost, executorData.freeCores)}.toSeq)) + launchTasks(scheduler.resourceOffers(executorDataMap.map { case (id, executorData) => + new WorkerOffer(id, executorData.executorHost, executorData.freeCores) + }.toSeq)) } // Make fake resource offers on just one executor diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index 74a92985b6629..b71bd5783d6df 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -20,10 +20,9 @@ package org.apache.spark.scheduler.cluster import akka.actor.{Address, ActorRef} /** - * Grouping of data that is accessed by a CourseGrainedScheduler. This class - * is stored in a Map keyed by an executorID + * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. * - * @param executorActor The actorRef representing this executor + * @param executorActor The ActorRef representing this executor * @param executorAddress The network address of this executor * @param executorHost The hostname that this executor is running on * @param freeCores The current number of cores available for work on the executor From 6918012d0f4841c5422b5827879a952428ec3a62 Mon Sep 17 00:00:00 2001 From: William Benton Date: Sun, 28 Sep 2014 01:01:27 -0700 Subject: [PATCH 0817/1492] SPARK-3699: SQL and Hive console tasks now clean up appropriately The sbt tasks sql/console and hive/console will now `stop()` the `SparkContext` upon exit. Previously, they left an ugly stack trace when quitting. Author: William Benton Closes #2547 from willb/consoleCleanup and squashes the following commits: d5e431f [William Benton] SQL and Hive console tasks now clean up. --- project/SparkBuild.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 12ac82293df76..01a5b20e7c51d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -221,7 +221,8 @@ object SQL { |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 + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + cleanupCommands in console := "sparkContext.stop()" ) } @@ -249,7 +250,8 @@ object Hive { |import org.apache.spark.sql.execution |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ - |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, + cleanupCommands in console := "sparkContext.stop()" ) } From 1f13a40ccd5a869aec62788a1e345dc24fa648c8 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Sun, 28 Sep 2014 18:30:13 -0700 Subject: [PATCH 0818/1492] [SPARK-3715][Docs]minor typo https://issues.apache.org/jira/browse/SPARK-3715 Author: WangTaoTheTonic Closes #2567 from WangTaoTheTonic/minortypo and squashes the following commits: 9cc3f7a [WangTaoTheTonic] minor typo --- docs/sql-programming-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 65249808fae3e..818fd5ab80af8 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -732,7 +732,7 @@ Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.

    When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do +adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do not have an existing Hive deployment can still create a HiveContext. When not configured by the hive-site.xml, the context automatically creates `metastore_db` and `warehouse` in the current directory. @@ -753,7 +753,7 @@ sqlContext.sql("FROM src SELECT key, value").collect().foreach(println)
    When working with Hive one must construct a `JavaHiveContext`, which inherits from `JavaSQLContext`, and -adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to the `sql` method a `JavaHiveContext` also provides an `hql` methods, which allows queries to be expressed in HiveQL. @@ -774,7 +774,7 @@ Row[] results = sqlContext.sql("FROM src SELECT key, value").collect();
    When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and -adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +adds support for finding tables in the MetaStore and writing queries using HiveQL. In addition to the `sql` method a `HiveContext` also provides an `hql` methods, which allows queries to be expressed in HiveQL. From 8e874185ed9efae8a1dc6b61d56ff401d72bb087 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 28 Sep 2014 18:33:11 -0700 Subject: [PATCH 0819/1492] Revert "[SPARK-1021] Defer the data-driven computation of partition bounds in so..." This reverts commit 2d972fd84ac54a89e416442508a6d4eaeff452c1. The commit was hanging correlationoptimizer14. --- .../scala/org/apache/spark/FutureAction.scala | 7 +- .../scala/org/apache/spark/Partitioner.scala | 29 ++------- .../apache/spark/rdd/AsyncRDDActions.scala | 64 ++++++++----------- 3 files changed, 34 insertions(+), 66 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index c277c3a47d421..75ea535f2f57b 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -208,7 +208,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { processPartition: Iterator[T] => U, partitions: Seq[Int], resultHandler: (Int, U) => Unit, - resultFunc: => R): R = { + resultFunc: => R) { // If the action hasn't been cancelled yet, submit the job. The check and the submitJob // command need to be in an atomic block. val job = this.synchronized { @@ -223,10 +223,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { // cancel the job and stop the execution. This is not in a synchronized block because // Await.ready eventually waits on the monitor in FutureJob.jobWaiter. try { - Await.ready(job, Duration.Inf).value.get match { - case scala.util.Failure(e) => throw e - case scala.util.Success(v) => v - } + Await.ready(job, Duration.Inf) } catch { case e: InterruptedException => job.cancel() diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index d40b152d221c5..37053bb6f37ad 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -29,10 +29,6 @@ import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{CollectionsUtils, Utils} import org.apache.spark.util.random.{XORShiftRandom, SamplingUtils} -import org.apache.spark.SparkContext.rddToAsyncRDDActions -import scala.concurrent.Await -import scala.concurrent.duration.Duration - /** * An object that defines how the elements in a key-value pair RDD are partitioned by key. * Maps each key to a partition ID, from 0 to `numPartitions - 1`. @@ -117,12 +113,8 @@ class RangePartitioner[K : Ordering : ClassTag, V]( private var ordering = implicitly[Ordering[K]] // An array of upper bounds for the first (partitions - 1) partitions - @volatile private var valRB: Array[K] = null - - private def rangeBounds: Array[K] = this.synchronized { - if (valRB != null) return valRB - - valRB = if (partitions <= 1) { + private var rangeBounds: Array[K] = { + if (partitions <= 1) { Array.empty } else { // This is the sample size we need to have roughly balanced output partitions, capped at 1M. @@ -160,8 +152,6 @@ class RangePartitioner[K : Ordering : ClassTag, V]( RangePartitioner.determineBounds(candidates, partitions) } } - - valRB } def numPartitions = rangeBounds.length + 1 @@ -232,8 +222,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream): Unit = this.synchronized { - if (valRB != null) return + private def readObject(in: ObjectInputStream) { val sfactory = SparkEnv.get.serializer sfactory match { case js: JavaSerializer => in.defaultReadObject() @@ -245,7 +234,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( val ser = sfactory.newInstance() Utils.deserializeViaNestedStream(in, ser) { ds => implicit val classTag = ds.readObject[ClassTag[Array[K]]]() - valRB = ds.readObject[Array[K]]() + rangeBounds = ds.readObject[Array[K]]() } } } @@ -265,18 +254,12 @@ private[spark] object RangePartitioner { sampleSizePerPartition: Int): (Long, Array[(Int, Int, Array[K])]) = { val shift = rdd.id // val classTagK = classTag[K] // to avoid serializing the entire partitioner object - // use collectAsync here to run this job as a future, which is cancellable - val sketchFuture = rdd.mapPartitionsWithIndex { (idx, iter) => + val sketched = rdd.mapPartitionsWithIndex { (idx, iter) => val seed = byteswap32(idx ^ (shift << 16)) val (sample, n) = SamplingUtils.reservoirSampleAndCount( iter, sampleSizePerPartition, seed) Iterator((idx, n, sample)) - }.collectAsync() - // We do need the future's value to continue any further - val sketched = Await.ready(sketchFuture, Duration.Inf).value.get match { - case scala.util.Success(v) => v.toArray - case scala.util.Failure(e) => throw e - } + }.collect() val numItems = sketched.map(_._2.toLong).sum (numItems, sketched) } diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 7a68b3afa8158..b62f3fbdc4a15 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -23,7 +23,6 @@ import scala.collection.mutable.ArrayBuffer import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag -import org.apache.spark.util.Utils import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} import org.apache.spark.annotation.Experimental @@ -39,30 +38,29 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi * Returns a future for counting the number of elements in the RDD. */ def countAsync(): FutureAction[Long] = { - val f = new ComplexFutureAction[Long] - f.run { - val totalCount = new AtomicLong - f.runJob(self, - (iter: Iterator[T]) => Utils.getIteratorSize(iter), - Range(0, self.partitions.size), - (index: Int, data: Long) => totalCount.addAndGet(data), - totalCount.get()) - } + val totalCount = new AtomicLong + self.context.submitJob( + self, + (iter: Iterator[T]) => { + var result = 0L + while (iter.hasNext) { + result += 1L + iter.next() + } + result + }, + Range(0, self.partitions.size), + (index: Int, data: Long) => totalCount.addAndGet(data), + totalCount.get()) } /** * Returns a future for retrieving all elements of this RDD. */ def collectAsync(): FutureAction[Seq[T]] = { - val f = new ComplexFutureAction[Seq[T]] - f.run { - val results = new Array[Array[T]](self.partitions.size) - f.runJob(self, - (iter: Iterator[T]) => iter.toArray, - Range(0, self.partitions.size), - (index: Int, data: Array[T]) => results(index) = data, - results.flatten.toSeq) - } + val results = new Array[Array[T]](self.partitions.size) + self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.size), + (index, data) => results(index) = data, results.flatten.toSeq) } /** @@ -106,34 +104,24 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi } results.toSeq } + + f } /** * Applies a function f to all elements of this RDD. */ - def foreachAsync(expr: T => Unit): FutureAction[Unit] = { - val f = new ComplexFutureAction[Unit] - val exprClean = self.context.clean(expr) - f.run { - f.runJob(self, - (iter: Iterator[T]) => iter.foreach(exprClean), - Range(0, self.partitions.size), - (index: Int, data: Unit) => Unit, - Unit) - } + def foreachAsync(f: T => Unit): FutureAction[Unit] = { + val cleanF = self.context.clean(f) + self.context.submitJob[T, Unit, Unit](self, _.foreach(cleanF), Range(0, self.partitions.size), + (index, data) => Unit, Unit) } /** * Applies a function f to each partition of this RDD. */ - def foreachPartitionAsync(expr: Iterator[T] => Unit): FutureAction[Unit] = { - val f = new ComplexFutureAction[Unit] - f.run { - f.runJob(self, - expr, - Range(0, self.partitions.size), - (index: Int, data: Unit) => Unit, - Unit) - } + def foreachPartitionAsync(f: Iterator[T] => Unit): FutureAction[Unit] = { + self.context.submitJob[T, Unit, Unit](self, f, Range(0, self.partitions.size), + (index, data) => Unit, Unit) } } From 25164a89dd32eef58d9b6823ae259439f796e81a Mon Sep 17 00:00:00 2001 From: Jim Lim Date: Sun, 28 Sep 2014 19:04:24 -0700 Subject: [PATCH 0820/1492] SPARK-2761 refactor #maybeSpill into Spillable Moved `#maybeSpill` in ExternalSorter and EAOM into `Spillable`. Author: Jim Lim Closes #2416 from jimjh/SPARK-2761 and squashes the following commits: cf8be9a [Jim Lim] SPARK-2761 fix documentation, reorder code f94d522 [Jim Lim] SPARK-2761 refactor Spillable to simplify sig e75a24e [Jim Lim] SPARK-2761 use protected over protected[this] 7270e0d [Jim Lim] SPARK-2761 refactor #maybeSpill into Spillable --- .../collection/ExternalAppendOnlyMap.scala | 46 ++------ .../util/collection/ExternalSorter.scala | 68 +++-------- .../spark/util/collection/Spillable.scala | 111 ++++++++++++++++++ 3 files changed, 133 insertions(+), 92 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/Spillable.scala diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 8a015c1d26a96..0c088da46aa5e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -66,23 +66,19 @@ class ExternalAppendOnlyMap[K, V, C]( mergeCombiners: (C, C) => C, serializer: Serializer = SparkEnv.get.serializer, blockManager: BlockManager = SparkEnv.get.blockManager) - extends Iterable[(K, C)] with Serializable with Logging { + extends Iterable[(K, C)] + with Serializable + with Logging + with Spillable[SizeTracker] { private var currentMap = new SizeTrackingAppendOnlyMap[K, C] private val spilledMaps = new ArrayBuffer[DiskMapIterator] private val sparkConf = SparkEnv.get.conf private val diskBlockManager = blockManager.diskBlockManager - private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager // Number of pairs inserted since last spill; note that we count them even if a value is merged // with a previous key in case we're doing something like groupBy where the result grows - private var elementsRead = 0L - - // Number of in-memory pairs inserted before tracking the map's shuffle memory usage - private val trackMemoryThreshold = 1000 - - // How much of the shared memory pool this collection has claimed - private var myMemoryThreshold = 0L + protected[this] var elementsRead = 0L /** * Size of object batches when reading/writing from serializers. @@ -95,11 +91,7 @@ class ExternalAppendOnlyMap[K, V, C]( */ private val serializerBatchSize = sparkConf.getLong("spark.shuffle.spill.batchSize", 10000) - // How many times we have spilled so far - private var spillCount = 0 - // Number of bytes spilled in total - private var _memoryBytesSpilled = 0L private var _diskBytesSpilled = 0L private val fileBufferSize = sparkConf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 @@ -136,19 +128,8 @@ class ExternalAppendOnlyMap[K, V, C]( while (entries.hasNext) { curEntry = entries.next() - if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && - currentMap.estimateSize() >= myMemoryThreshold) - { - // Claim up to double our current memory from the shuffle memory pool - val currentMemory = currentMap.estimateSize() - val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) - myMemoryThreshold += granted - if (myMemoryThreshold <= currentMemory) { - // We were granted too little memory to grow further (either tryToAcquire returned 0, - // or we already had more memory than myMemoryThreshold); spill the current collection - spill(currentMemory) // Will also release memory back to ShuffleMemoryManager - } + if (maybeSpill(currentMap, currentMap.estimateSize())) { + currentMap = new SizeTrackingAppendOnlyMap[K, C] } currentMap.changeValue(curEntry._1, update) elementsRead += 1 @@ -171,11 +152,7 @@ class ExternalAppendOnlyMap[K, V, C]( /** * Sort the existing contents of the in-memory map and spill them to a temporary file on disk. */ - private def spill(mapSize: Long): Unit = { - spillCount += 1 - val threadId = Thread.currentThread().getId - logInfo("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" - .format(threadId, mapSize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) + override protected[this] def spill(collection: SizeTracker): Unit = { val (blockId, file) = diskBlockManager.createTempBlock() curWriteMetrics = new ShuffleWriteMetrics() var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, @@ -231,18 +208,11 @@ class ExternalAppendOnlyMap[K, V, C]( } } - currentMap = new SizeTrackingAppendOnlyMap[K, C] spilledMaps.append(new DiskMapIterator(file, blockId, batchSizes)) - // Release our memory back to the shuffle pool so that other threads can grab it - shuffleMemoryManager.release(myMemoryThreshold) - myMemoryThreshold = 0L - elementsRead = 0 - _memoryBytesSpilled += mapSize } - def memoryBytesSpilled: Long = _memoryBytesSpilled def diskBytesSpilled: Long = _diskBytesSpilled /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 782b979e2e93d..0a152cb97ad9e 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -79,14 +79,14 @@ private[spark] class ExternalSorter[K, V, C]( aggregator: Option[Aggregator[K, V, C]] = None, partitioner: Option[Partitioner] = None, ordering: Option[Ordering[K]] = None, - serializer: Option[Serializer] = None) extends Logging { + serializer: Option[Serializer] = None) + extends Logging with Spillable[SizeTrackingPairCollection[(Int, K), C]] { private val numPartitions = partitioner.map(_.numPartitions).getOrElse(1) private val shouldPartition = numPartitions > 1 private val blockManager = SparkEnv.get.blockManager private val diskBlockManager = blockManager.diskBlockManager - private val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager private val ser = Serializer.getSerializer(serializer) private val serInstance = ser.newInstance() @@ -115,22 +115,14 @@ private[spark] class ExternalSorter[K, V, C]( // Number of pairs read from input since last spill; note that we count them even if a value is // merged with a previous key in case we're doing something like groupBy where the result grows - private var elementsRead = 0L - - // What threshold of elementsRead we start estimating map size at. - private val trackMemoryThreshold = 1000 + protected[this] var elementsRead = 0L // Total spilling statistics - private var spillCount = 0 - private var _memoryBytesSpilled = 0L private var _diskBytesSpilled = 0L // Write metrics for current spill private var curWriteMetrics: ShuffleWriteMetrics = _ - // How much of the shared memory pool this collection has claimed - private var myMemoryThreshold = 0L - // If there are fewer than spark.shuffle.sort.bypassMergeThreshold partitions and we don't need // local aggregation and sorting, write numPartitions files directly and just concatenate them // at the end. This avoids doing serialization and deserialization twice to merge together the @@ -209,7 +201,7 @@ private[spark] class ExternalSorter[K, V, C]( elementsRead += 1 kv = records.next() map.changeValue((getPartition(kv._1), kv._1), update) - maybeSpill(usingMap = true) + maybeSpillCollection(usingMap = true) } } else { // Stick values into our buffer @@ -217,7 +209,7 @@ private[spark] class ExternalSorter[K, V, C]( elementsRead += 1 val kv = records.next() buffer.insert((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) - maybeSpill(usingMap = false) + maybeSpillCollection(usingMap = false) } } } @@ -227,61 +219,31 @@ private[spark] class ExternalSorter[K, V, C]( * * @param usingMap whether we're using a map or buffer as our current in-memory collection */ - private def maybeSpill(usingMap: Boolean): Unit = { + private def maybeSpillCollection(usingMap: Boolean): Unit = { if (!spillingEnabled) { return } - val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer - - // TODO: factor this out of both here and ExternalAppendOnlyMap - if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && - collection.estimateSize() >= myMemoryThreshold) - { - // Claim up to double our current memory from the shuffle memory pool - val currentMemory = collection.estimateSize() - val amountToRequest = 2 * currentMemory - myMemoryThreshold - val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) - myMemoryThreshold += granted - if (myMemoryThreshold <= currentMemory) { - // We were granted too little memory to grow further (either tryToAcquire returned 0, - // or we already had more memory than myMemoryThreshold); spill the current collection - spill(currentMemory, usingMap) // Will also release memory back to ShuffleMemoryManager + if (usingMap) { + if (maybeSpill(map, map.estimateSize())) { + map = new SizeTrackingAppendOnlyMap[(Int, K), C] + } + } else { + if (maybeSpill(buffer, buffer.estimateSize())) { + buffer = new SizeTrackingPairBuffer[(Int, K), C] } } } /** * Spill the current in-memory collection to disk, adding a new file to spills, and clear it. - * - * @param usingMap whether we're using a map or buffer as our current in-memory collection */ - private def spill(memorySize: Long, usingMap: Boolean): Unit = { - val collection: SizeTrackingPairCollection[(Int, K), C] = if (usingMap) map else buffer - val memorySize = collection.estimateSize() - - spillCount += 1 - val threadId = Thread.currentThread().getId - logInfo("Thread %d spilling in-memory batch of %d MB to disk (%d spill%s so far)" - .format(threadId, memorySize / (1024 * 1024), spillCount, if (spillCount > 1) "s" else "")) - + override protected[this] def spill(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { if (bypassMergeSort) { spillToPartitionFiles(collection) } else { spillToMergeableFile(collection) } - - if (usingMap) { - map = new SizeTrackingAppendOnlyMap[(Int, K), C] - } else { - buffer = new SizeTrackingPairBuffer[(Int, K), C] - } - - // Release our memory back to the shuffle pool so that other threads can grab it - shuffleMemoryManager.release(myMemoryThreshold) - myMemoryThreshold = 0 - - _memoryBytesSpilled += memorySize } /** @@ -804,8 +766,6 @@ private[spark] class ExternalSorter[K, V, C]( } } - def memoryBytesSpilled: Long = _memoryBytesSpilled - def diskBytesSpilled: Long = _diskBytesSpilled /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala new file mode 100644 index 0000000000000..d7dccd4af8c6e --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -0,0 +1,111 @@ +/* + * 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.util.collection + +import org.apache.spark.Logging +import org.apache.spark.SparkEnv + +/** + * Spills contents of an in-memory collection to disk when the memory threshold + * has been exceeded. + */ +private[spark] trait Spillable[C] { + + this: Logging => + + /** + * Spills the current in-memory collection to disk, and releases the memory. + * + * @param collection collection to spill to disk + */ + protected def spill(collection: C): Unit + + // Number of elements read from input since last spill + protected var elementsRead: Long + + // Memory manager that can be used to acquire/release memory + private[this] val shuffleMemoryManager = SparkEnv.get.shuffleMemoryManager + + // What threshold of elementsRead we start estimating collection size at + private[this] val trackMemoryThreshold = 1000 + + // How much of the shared memory pool this collection has claimed + private[this] var myMemoryThreshold = 0L + + // Number of bytes spilled in total + private[this] var _memoryBytesSpilled = 0L + + // Number of spills + private[this] var _spillCount = 0 + + /** + * Spills the current in-memory collection to disk if needed. Attempts to acquire more + * memory before spilling. + * + * @param collection collection to spill to disk + * @param currentMemory estimated size of the collection in bytes + * @return true if `collection` was spilled to disk; false otherwise + */ + protected def maybeSpill(collection: C, currentMemory: Long): Boolean = { + if (elementsRead > trackMemoryThreshold && elementsRead % 32 == 0 && + currentMemory >= myMemoryThreshold) { + // Claim up to double our current memory from the shuffle memory pool + val amountToRequest = 2 * currentMemory - myMemoryThreshold + val granted = shuffleMemoryManager.tryToAcquire(amountToRequest) + myMemoryThreshold += granted + if (myMemoryThreshold <= currentMemory) { + // We were granted too little memory to grow further (either tryToAcquire returned 0, + // or we already had more memory than myMemoryThreshold); spill the current collection + _spillCount += 1 + logSpillage(currentMemory) + + spill(collection) + + // Keep track of spills, and release memory + _memoryBytesSpilled += currentMemory + releaseMemoryForThisThread() + return true + } + } + false + } + + /** + * @return number of bytes spilled in total + */ + def memoryBytesSpilled: Long = _memoryBytesSpilled + + /** + * Release our memory back to the shuffle pool so that other threads can grab it. + */ + private def releaseMemoryForThisThread(): Unit = { + shuffleMemoryManager.release(myMemoryThreshold) + myMemoryThreshold = 0L + } + + /** + * Prints a standard log message detailing spillage. + * + * @param size number of bytes spilled + */ + @inline private def logSpillage(size: Long) { + val threadId = Thread.currentThread().getId + logInfo("Thread %d spilling in-memory map of %d MB to disk (%d time%s so far)" + .format(threadId, size / (1024 * 1024), _spillCount, if (_spillCount > 1) "s" else "")) + } +} From f350cd307045c2c02e713225d8f1247f18ba123e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 28 Sep 2014 20:32:54 -0700 Subject: [PATCH 0821/1492] [SPARK-3543] TaskContext remaining cleanup work. Author: Reynold Xin Closes #2560 from rxin/TaskContext and squashes the following commits: 9eff95a [Reynold Xin] [SPARK-3543] remaining cleanup work. --- core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 3 ++- .../apache/spark/util/JavaTaskCompletionListenerImpl.java | 7 +++---- .../serializer/ProactiveClosureSerializationSuite.scala | 6 +----- .../apache/spark/sql/parquet/ParquetTableOperations.scala | 4 ++-- 5 files changed, 9 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 036dcc49664ef..21d0cc7b5cbaa 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -194,7 +194,7 @@ class HadoopRDD[K, V]( val jobConf = getJobConf() val inputFormat = getInputFormat(jobConf) HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), - context.stageId, theSplit.index, context.attemptId.toInt, jobConf) + context.getStageId, theSplit.index, context.getAttemptId.toInt, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7f578bc5dac39..67833743f3a98 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -86,7 +86,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) } val aggregator = new Aggregator[K, V, C](createCombiner, mergeValue, mergeCombiners) if (self.partitioner == Some(partitioner)) { - self.mapPartitionsWithContext((context, iter) => { + self.mapPartitions(iter => { + val context = TaskContext.get() new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) } else { diff --git a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java index af34cdb03e4d1..0944bf8cd5c71 100644 --- a/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java +++ b/core/src/test/java/org/apache/spark/util/JavaTaskCompletionListenerImpl.java @@ -30,10 +30,9 @@ public class JavaTaskCompletionListenerImpl implements TaskCompletionListener { public void onTaskCompletion(TaskContext context) { context.isCompleted(); context.isInterrupted(); - context.stageId(); - context.partitionId(); - context.runningLocally(); - context.taskMetrics(); + context.getStageId(); + context.getPartitionId(); + context.isRunningLocally(); context.addTaskCompletionListener(this); } } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala index aad6599589420..d037e2c19a64d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -50,8 +50,7 @@ class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContex "flatMap" -> xflatMap _, "filter" -> xfilter _, "mapPartitions" -> xmapPartitions _, - "mapPartitionsWithIndex" -> xmapPartitionsWithIndex _, - "mapPartitionsWithContext" -> xmapPartitionsWithContext _)) { + "mapPartitionsWithIndex" -> xmapPartitionsWithIndex _)) { val (name, xf) = transformation test(s"$name transformations throw proactive serialization exceptions") { @@ -78,8 +77,5 @@ class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContex private def xmapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = x.mapPartitionsWithIndex((_, it) => it.map(y=>uc.op(y))) - - private def xmapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithContext((_, it) => it.map(y=>uc.op(y))) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index d39e31a7fa195..ffb732347d30a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -289,9 +289,9 @@ case class InsertIntoParquetTable( def writeShard(context: TaskContext, iter: Iterator[Row]): Int = { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = new AppendingParquetOutputFormat(taskIdOffset) From 0dc2b6361d61b7d94cba3dc83da2abb7e08ba6fe Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 28 Sep 2014 21:44:50 -0700 Subject: [PATCH 0822/1492] [SPARK-1545] [mllib] Add Random Forests This PR adds RandomForest to MLlib. The implementation is basic, and future performance optimizations will be important. (Note: RFs = Random Forests.) # Overview ## RandomForest * trains multiple trees at once to reduce the number of passes over the data * allows feature subsets at each node * uses a queue of nodes instead of fixed groups for each level This implementation is based an implementation by manishamde and the [Alpine Labs Sequoia Forest](https://github.com/AlpineNow/SparkML2) by codedeft (in particular, the TreePoint, BaggedPoint, and node queue implementations). Thank you for your inputs! ## Testing Correctness: This has been tested for correctness with the test suites and with DecisionTreeRunner on example datasets. Performance: This has been performance tested using [this branch of spark-perf](https://github.com/jkbradley/spark-perf/tree/rfs). Results below. ### Regression tests for DecisionTree Summary: For training 1 tree, there are small regressions, especially from feature subsampling. In the table below, each row is a single (random) dataset. The 2 different sets of result columns are for 2 different RF implementations: * (numTrees): This is from an earlier commit, after implementing RandomForest to train multiple trees at once. It does not include any code for feature subsampling. * (feature subsets): This is from this current PR's code, after implementing feature subsampling. These tests were to identify regressions in DecisionTree, so they are training 1 tree with all of the features (i.e., no feature subsampling). These were run on an EC2 cluster with 15 workers, training 1 tree with maxDepth = 5 (= 6 levels). Speedup values < 1 indicate slowdowns from the old DecisionTree implementation. numInstances | numFeatures | runtime (sec) | speedup | runtime (sec) | speedup ---- | ---- | ---- | ---- | ---- | ---- | | (numTrees) | (numTrees) | (feature subsets) | (feature subsets) 20000 | 100 | 4.051 | 1.044433473 | 4.478 | 0.9448414471 20000 | 500 | 8.472 | 1.104461756 | 9.315 | 1.004508857 20000 | 1500 | 19.354 | 1.05854087 | 20.863 | 0.9819776638 20000 | 3500 | 43.674 | 1.072033704 | 45.887 | 1.020332556 200000 | 100 | 4.196 | 1.171830315 | 4.848 | 1.014232673 200000 | 500 | 8.926 | 1.082791844 | 9.771 | 0.989151571 200000 | 1500 | 20.58 | 1.068415938 | 22.134 | 0.9934038131 200000 | 3500 | 48.043 | 1.075203464 | 52.249 | 0.9886505005 2000000 | 100 | 4.944 | 1.01355178 | 5.796 | 0.8645617667 2000000 | 500 | 11.11 | 1.016831683 | 12.482 | 0.9050632911 2000000 | 1500 | 31.144 | 1.017852556 | 35.274 | 0.8986789136 2000000 | 3500 | 79.981 | 1.085382778 | 101.105 | 0.8586123337 20000000 | 100 | 8.304 | 0.9270231214 | 9.073 | 0.8484514494 20000000 | 500 | 28.174 | 1.083268262 | 34.236 | 0.8914592826 20000000 | 1500 | 143.97 | 0.9579634646 | 159.275 | 0.8659111599 ### Tests for forests I have run other tests with numTrees=10 and with sqrt(numFeatures), and those indicate that multi-model training and feature subsets can speed up training for forests, especially when training deeper trees. # Details on specific classes ## Changes to DecisionTree * Main train() method is now in RandomForest. * findBestSplits() is no longer needed. (It split levels into groups, but we now use a queue of nodes.) * Many small changes to support RFs. (Note: These methods should be moved to RandomForest.scala in a later PR, but are in DecisionTree.scala to make code comparison easier.) ## RandomForest * Main train() method is from old DecisionTree. * selectNodesToSplit: Note that it selects nodes and feature subsets jointly to track memory usage. ## RandomForestModel * Stores an Array[DecisionTreeModel] * Prediction: * For classification, most common label. For regression, mean. * We could support other methods later. ## examples/.../DecisionTreeRunner * This now takes numTrees and featureSubsetStrategy, to support RFs. ## DTStatsAggregator * 2 types of functionality (w/ and w/o subsampling features): These require different indexing methods. (We could treat both as subsampling, but this is less efficient DTStatsAggregator is now abstract, and 2 child classes implement these 2 types of functionality. ## impurities * These now take instance weights. ## Node * Some vals changed to vars. * This is unfortunately a public API change (DeveloperApi). This could be avoided by creating a LearningNode struct, but would be awkward. ## RandomForestSuite Please let me know if there are missing tests! ## BaggedPoint This wraps TreePoint and holds bootstrap weights/counts. # Design decisions * BaggedPoint: BaggedPoint is separate from TreePoint since it may be useful for other bagging algorithms later on. * RandomForest public API: What options should be easily supported by the train* methods? Should ALL options be in the Java-friendly constructors? Should there be a constructor taking Strategy? * Feature subsampling options: What options should be supported? scikit-learn supports the same options, except for "onethird." One option would be to allow users to specific fractions ("0.1"): the current options could be supported, and any unrecognized values would be parsed as Doubles in [0,1]. * Splits and bins are computed before bootstrapping, so all trees use the same discretization. * One queue, instead of one queue per tree. CC: mengxr manishamde codedeft chouqin Please let me know if you have suggestions---thanks! Author: Joseph K. Bradley Author: qiping.lqp Author: chouqin Closes #2435 from jkbradley/rfs-new and squashes the following commits: c694174 [Joseph K. Bradley] Fixed typo cc59d78 [Joseph K. Bradley] fixed imports e25909f [Joseph K. Bradley] Simplified node group maps. Specifically, created NodeIndexInfo to store node index in agg and feature subsets, and no longer create extra maps in findBestSplits fbe9a1e [Joseph K. Bradley] Changed default featureSubsetStrategy to be sqrt for classification, onethird for regression. Updated docs with references. ef7c293 [Joseph K. Bradley] Updates based on code review. Most substantial changes: * Simplified DTStatsAggregator * Made RandomForestModel.trees public * Added test for regression to RandomForestSuite 593b13c [Joseph K. Bradley] Fixed bug in metadata for computing log2(num features). Now it checks >= 1. a1a08df [Joseph K. Bradley] Removed old comments 866e766 [Joseph K. Bradley] Changed RandomForestSuite randomized tests to use multiple fixed random seeds. ff8bb96 [Joseph K. Bradley] removed usage of null from RandomForest and replaced with Option bf1a4c5 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into rfs-new 6b79c07 [Joseph K. Bradley] Added RandomForestSuite, and fixed small bugs, style issues. d7753d4 [Joseph K. Bradley] Added numTrees and featureSubsetStrategy to DecisionTreeRunner (to support RandomForest). Fixed bugs so that RandomForest now runs. 746d43c [Joseph K. Bradley] Implemented feature subsampling. Tested DecisionTree but not RandomForest. 6309d1d [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into rfs-new. Added RandomForestModel.toString b7ae594 [Joseph K. Bradley] Updated docs. Small fix for bug which does not cause errors: No longer allocate unused child nodes for leaf nodes. 121c74e [Joseph K. Bradley] Basic random forests are implemented. Random features per node not yet implemented. Test suite not implemented. 325d18a [Joseph K. Bradley] Merge branch 'chouqin-dt-preprune' into rfs-new 4ef9bf1 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into rfs-new 61b2e72 [Joseph K. Bradley] Added max of 10GB for maxMemoryInMB in Strategy. a95e7c8 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into chouqin-dt-preprune 6da8571 [Joseph K. Bradley] RFs partly implemented, not done yet eddd1eb [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into rfs-new 5c4ac33 [Joseph K. Bradley] Added check in Strategy to make sure minInstancesPerNode >= 1 0dd4d87 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 95c479d [Joseph K. Bradley] * Fixed typo in tree suite test "do not choose split that does not satisfy min instance per node requirements" * small style fixes e2628b6 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into chouqin-dt-preprune 19b01af [Joseph K. Bradley] Merge remote-tracking branch 'chouqin/dt-preprune' into chouqin-dt-preprune f1d11d1 [chouqin] fix typo c7ebaf1 [chouqin] fix typo 39f9b60 [chouqin] change edge `minInstancesPerNode` to 2 and add one more test c6e2dfc [Joseph K. Bradley] Added minInstancesPerNode and minInfoGain parameters to DecisionTreeRunner.scala and to Python API in tree.py 306120f [Joseph K. Bradley] Fixed typo in DecisionTreeModel.scala doc eaa1dcf [Joseph K. Bradley] Added topNode doc in DecisionTree and scalastyle fix d4d7864 [Joseph K. Bradley] Marked Node.build as deprecated d4dbb99 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dt-spark-3160 1a8f0ad [Joseph K. Bradley] Eliminated pre-allocated nodes array in main train() method. * Nodes are constructed and added to the tree structure as needed during training. 0278a11 [chouqin] remove `noSplit` and set `Predict` private to tree d593ec7 [chouqin] fix docs and change minInstancesPerNode to 1 2ab763b [Joseph K. Bradley] Simplifications to DecisionTree code: efcc736 [qiping.lqp] fix bug 10b8012 [qiping.lqp] fix style 6728fad [qiping.lqp] minor fix: remove empty lines bb465ca [qiping.lqp] Merge branch 'master' of https://github.com/apache/spark into dt-preprune cadd569 [qiping.lqp] add api docs 46b891f [qiping.lqp] fix bug e72c7e4 [qiping.lqp] add comments 845c6fa [qiping.lqp] fix style f195e83 [qiping.lqp] fix style 987cbf4 [qiping.lqp] fix bug ff34845 [qiping.lqp] separate calculation of predict of node from calculation of info gain ac42378 [qiping.lqp] add min info gain and min instances per node parameters in decision tree --- .../examples/mllib/DecisionTreeRunner.scala | 76 ++- .../spark/mllib/tree/DecisionTree.scala | 457 ++++++------------ .../spark/mllib/tree/RandomForest.scala | 451 +++++++++++++++++ .../spark/mllib/tree/impl/BaggedPoint.scala | 80 +++ .../mllib/tree/impl/DTStatsAggregator.scala | 219 +++++++-- .../tree/impl/DecisionTreeMetadata.scala | 47 +- .../spark/mllib/tree/impurity/Entropy.scala | 4 +- .../spark/mllib/tree/impurity/Gini.scala | 4 +- .../spark/mllib/tree/impurity/Impurity.scala | 2 +- .../spark/mllib/tree/impurity/Variance.scala | 8 +- .../apache/spark/mllib/tree/model/Node.scala | 13 +- .../mllib/tree/model/RandomForestModel.scala | 105 ++++ .../spark/mllib/tree/DecisionTreeSuite.scala | 210 ++++---- .../spark/mllib/tree/RandomForestSuite.scala | 245 ++++++++++ 14 files changed, 1410 insertions(+), 511 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 4683e6eb966be..96fb068e9e126 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -21,16 +21,18 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.tree.{DecisionTree, impurity} +import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity} import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.model.DecisionTreeModel +import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils /** - * An example runner for decision tree. Run with + * An example runner for decision trees and random forests. Run with * {{{ * ./bin/run-example org.apache.spark.examples.mllib.DecisionTreeRunner [options] * }}} @@ -57,6 +59,8 @@ object DecisionTreeRunner { maxBins: Int = 32, minInstancesPerNode: Int = 1, minInfoGain: Double = 0.0, + numTrees: Int = 1, + featureSubsetStrategy: String = "auto", fracTest: Double = 0.2) def main(args: Array[String]) { @@ -79,11 +83,20 @@ object DecisionTreeRunner { .action((x, c) => c.copy(maxBins = x)) opt[Int]("minInstancesPerNode") .text(s"min number of instances required at child nodes to create the parent split," + - s" default: ${defaultParams.minInstancesPerNode}") + s" default: ${defaultParams.minInstancesPerNode}") .action((x, c) => c.copy(minInstancesPerNode = x)) opt[Double]("minInfoGain") .text(s"min info gain required to create a split, default: ${defaultParams.minInfoGain}") .action((x, c) => c.copy(minInfoGain = x)) + opt[Int]("numTrees") + .text(s"number of trees (1 = decision tree, 2+ = random forest)," + + s" default: ${defaultParams.numTrees}") + .action((x, c) => c.copy(numTrees = x)) + opt[String]("featureSubsetStrategy") + .text(s"feature subset sampling strategy" + + s" (${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}}), " + + s"default: ${defaultParams.featureSubsetStrategy}") + .action((x, c) => c.copy(featureSubsetStrategy = x)) opt[Double]("fracTest") .text(s"fraction of data to hold out for testing, default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) @@ -191,18 +204,35 @@ object DecisionTreeRunner { numClassesForClassification = numClasses, minInstancesPerNode = params.minInstancesPerNode, minInfoGain = params.minInfoGain) - val model = DecisionTree.train(training, strategy) - - println(model) - - if (params.algo == Classification) { - val accuracy = accuracyScore(model, test) - println(s"Test accuracy = $accuracy") - } - - if (params.algo == Regression) { - val mse = meanSquaredError(model, test) - println(s"Test mean squared error = $mse") + if (params.numTrees == 1) { + val model = DecisionTree.train(training, strategy) + println(model) + if (params.algo == Classification) { + val accuracy = + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + println(s"Test accuracy = $accuracy") + } + if (params.algo == Regression) { + val mse = meanSquaredError(model, test) + println(s"Test mean squared error = $mse") + } + } else { + val randomSeed = Utils.random.nextInt() + if (params.algo == Classification) { + val model = RandomForest.trainClassifier(training, strategy, params.numTrees, + params.featureSubsetStrategy, randomSeed) + println(model) + val accuracy = + new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision + println(s"Test accuracy = $accuracy") + } + if (params.algo == Regression) { + val model = RandomForest.trainRegressor(training, strategy, params.numTrees, + params.featureSubsetStrategy, randomSeed) + println(model) + val mse = meanSquaredError(model, test) + println(s"Test mean squared error = $mse") + } } sc.stop() @@ -211,9 +241,7 @@ object DecisionTreeRunner { /** * Calculates the classifier accuracy. */ - private def accuracyScore( - model: DecisionTreeModel, - data: RDD[LabeledPoint]): Double = { + private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { val correctCount = data.filter(y => model.predict(y.features) == y.label).count() val count = data.count() correctCount.toDouble / count @@ -228,4 +256,14 @@ object DecisionTreeRunner { err * err }.mean() } + + /** + * Calculates the mean squared error for regression. + */ + private def meanSquaredError(tree: RandomForestModel, data: RDD[LabeledPoint]): Double = { + data.map { y => + val err = tree.predict(y.features) - y.label + err * err + }.mean() + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index c7f2576c822b1..b7dc373ebd9cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -18,12 +18,14 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ +import scala.collection.mutable import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ @@ -33,7 +35,6 @@ import org.apache.spark.mllib.tree.impurity.{Impurities, Impurity} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel import org.apache.spark.util.random.XORShiftRandom @@ -56,99 +57,10 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * @return DecisionTreeModel that can be used for prediction */ def train(input: RDD[LabeledPoint]): DecisionTreeModel = { - - val timer = new TimeTracker() - - timer.start("total") - - timer.start("init") - - val retaggedInput = input.retag(classOf[LabeledPoint]) - val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy) - logDebug("algo = " + strategy.algo) - logDebug("maxBins = " + metadata.maxBins) - - // Find the splits and the corresponding bins (interval between the splits) using a sample - // of the input data. - timer.start("findSplitsBins") - val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) - timer.stop("findSplitsBins") - logDebug("numBins: feature: number of bins") - logDebug(Range(0, metadata.numFeatures).map { featureIndex => - s"\t$featureIndex\t${metadata.numBins(featureIndex)}" - }.mkString("\n")) - - // Bin feature values (TreePoint representation). - // Cache input RDD for speedup during multiple passes. - val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) - .persist(StorageLevel.MEMORY_AND_DISK) - - // depth of the decision tree - val maxDepth = strategy.maxDepth - require(maxDepth <= 30, - s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") - - // Calculate level for single group construction - - // Max memory usage for aggregates - val maxMemoryUsage = strategy.maxMemoryInMB * 1024L * 1024L - logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") - // TODO: Calculate memory usage more precisely. - val numElementsPerNode = DecisionTree.getElementsPerNode(metadata) - - logDebug("numElementsPerNode = " + numElementsPerNode) - val arraySizePerNode = 8 * numElementsPerNode // approx. memory usage for bin aggregate array - val maxNumberOfNodesPerGroup = math.max(maxMemoryUsage / arraySizePerNode, 1) - logDebug("maxNumberOfNodesPerGroup = " + maxNumberOfNodesPerGroup) - // nodes at a level is 2^level. level is zero indexed. - val maxLevelForSingleGroup = math.max( - (math.log(maxNumberOfNodesPerGroup) / math.log(2)).floor.toInt, 0) - logDebug("max level for single group = " + maxLevelForSingleGroup) - - timer.stop("init") - - /* - * The main idea here is to perform level-wise training of the decision tree nodes thus - * reducing the passes over the data from l to log2(l) where l is the total number of nodes. - * Each data sample is handled by a particular node at that level (or it reaches a leaf - * beforehand and is not used in later levels. - */ - - var topNode: Node = null // set on first iteration - var level = 0 - var break = false - while (level <= maxDepth && !break) { - logDebug("#####################################") - logDebug("level = " + level) - logDebug("#####################################") - - // Find best split for all nodes at a level. - timer.start("findBestSplits") - val (tmpTopNode: Node, doneTraining: Boolean) = DecisionTree.findBestSplits(treeInput, - metadata, level, topNode, splits, bins, maxLevelForSingleGroup, timer) - timer.stop("findBestSplits") - - if (level == 0) { - topNode = tmpTopNode - } - if (doneTraining) { - break = true - logDebug("done training") - } - - level += 1 - } - - logDebug("#####################################") - logDebug("Extracting tree model") - logDebug("#####################################") - - timer.stop("total") - - logInfo("Internal timing for DecisionTree:") - logInfo(s"$timer") - - new DecisionTreeModel(topNode, strategy.algo) + // Note: random seed will not be used since numTrees = 1. + val rf = new RandomForest(strategy, numTrees = 1, featureSubsetStrategy = "all", seed = 0) + val rfModel = rf.train(input) + rfModel.trees(0) } } @@ -352,58 +264,10 @@ object DecisionTree extends Serializable with Logging { impurity, maxDepth, maxBins) } - /** - * Returns an array of optimal splits for all nodes at a given level. Splits the task into - * multiple groups if the level-wise training task could lead to memory overflow. - * - * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] - * @param metadata Learning and dataset metadata - * @param level Level of the tree - * @param topNode Root node of the tree (or invalid node when training first level). - * @param splits possible splits for all features, indexed (numFeatures)(numSplits) - * @param bins possible bins for all features, indexed (numFeatures)(numBins) - * @param maxLevelForSingleGroup the deepest level for single-group level-wise computation. - * @return (root, doneTraining) where: - * root = Root node (which is newly created on the first iteration), - * doneTraining = true if no more internal nodes were created. - */ - private[tree] def findBestSplits( - input: RDD[TreePoint], - metadata: DecisionTreeMetadata, - level: Int, - topNode: Node, - splits: Array[Array[Split]], - bins: Array[Array[Bin]], - maxLevelForSingleGroup: Int, - timer: TimeTracker = new TimeTracker): (Node, Boolean) = { - - // split into groups to avoid memory overflow during aggregation - if (level > maxLevelForSingleGroup) { - // When information for all nodes at a given level cannot be stored in memory, - // the nodes are divided into multiple groups at each level with the number of groups - // increasing exponentially per level. For example, if maxLevelForSingleGroup is 10, - // numGroups is equal to 2 at level 11 and 4 at level 12, respectively. - val numGroups = 1 << level - maxLevelForSingleGroup - logDebug("numGroups = " + numGroups) - // Iterate over each group of nodes at a level. - var groupIndex = 0 - var doneTraining = true - while (groupIndex < numGroups) { - val (_, doneTrainingGroup) = findBestSplitsPerGroup(input, metadata, level, - topNode, splits, bins, timer, numGroups, groupIndex) - doneTraining = doneTraining && doneTrainingGroup - groupIndex += 1 - } - (topNode, doneTraining) // Not first iteration, so topNode was already set. - } else { - findBestSplitsPerGroup(input, metadata, level, topNode, splits, bins, timer) - } - } - /** * Get the node index corresponding to this data point. - * This function mimics prediction, passing an example from the root node down to a node - * at the current level being trained; that node's index is returned. + * This function mimics prediction, passing an example from the root node down to a leaf + * or unsplit node; that node's index is returned. * * @param node Node in tree from which to classify the given data point. * @param binnedFeatures Binned feature vector for data point. @@ -413,14 +277,15 @@ object DecisionTree extends Serializable with Logging { * Otherwise, last node reachable in tree matching this example. * Note: This is the global node index, i.e., the index used in the tree. * This index is different from the index used during training a particular - * set of nodes in a (level, group). + * group of nodes on one call to [[findBestSplits()]]. */ private def predictNodeIndex( node: Node, binnedFeatures: Array[Int], bins: Array[Array[Bin]], unorderedFeatures: Set[Int]): Int = { - if (node.isLeaf) { + if (node.isLeaf || node.split.isEmpty) { + // Node is either leaf, or has not yet been split. node.id } else { val featureIndex = node.split.get.feature @@ -465,43 +330,60 @@ object DecisionTree extends Serializable with Logging { * @param agg Array storing aggregate calculation, with a set of sufficient statistics for * each (node, feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). + * @param nodeIndex Node corresponding to treePoint. agg is indexed in [0, numNodes). * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param unorderedFeatures Set of indices of unordered features. + * @param instanceWeight Weight (importance) of instance in dataset. */ private def mixedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, nodeIndex: Int, bins: Array[Array[Bin]], - unorderedFeatures: Set[Int]): Unit = { - // Iterate over all features. - val numFeatures = treePoint.binnedFeatures.size + unorderedFeatures: Set[Int], + instanceWeight: Double, + featuresForNode: Option[Array[Int]]): Unit = { + val numFeaturesPerNode = if (featuresForNode.nonEmpty) { + // Use subsampled features + featuresForNode.get.size + } else { + // Use all features + agg.metadata.numFeatures + } val nodeOffset = agg.getNodeOffset(nodeIndex) - var featureIndex = 0 - while (featureIndex < numFeatures) { + // Iterate over features. + var featureIndexIdx = 0 + while (featureIndexIdx < numFeaturesPerNode) { + val featureIndex = if (featuresForNode.nonEmpty) { + featuresForNode.get.apply(featureIndexIdx) + } else { + featureIndexIdx + } if (unorderedFeatures.contains(featureIndex)) { // Unordered feature val featureValue = treePoint.binnedFeatures(featureIndex) val (leftNodeFeatureOffset, rightNodeFeatureOffset) = - agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndexIdx) // Update the left or right bin for each split. - val numSplits = agg.numSplits(featureIndex) + val numSplits = agg.metadata.numSplits(featureIndex) var splitIndex = 0 while (splitIndex < numSplits) { if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { - agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label) + agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, + instanceWeight) } else { - agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label) + agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, + instanceWeight) } splitIndex += 1 } } else { // Ordered feature val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, featureIndex, binIndex, treePoint.label) + agg.nodeUpdate(nodeOffset, nodeIndex, featureIndexIdx, binIndex, treePoint.label, + instanceWeight) } - featureIndex += 1 + featureIndexIdx += 1 } } @@ -513,66 +395,77 @@ object DecisionTree extends Serializable with Logging { * @param agg Array storing aggregate calculation, with a set of sufficient statistics for * each (node, feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. Indexed from 0 at start of (level, group). - * @return agg + * @param nodeIndex Node corresponding to treePoint. agg is indexed in [0, numNodes). + * @param instanceWeight Weight (importance) of instance in dataset. */ private def orderedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - nodeIndex: Int): Unit = { + nodeIndex: Int, + instanceWeight: Double, + featuresForNode: Option[Array[Int]]): Unit = { val label = treePoint.label val nodeOffset = agg.getNodeOffset(nodeIndex) - // Iterate over all features. - val numFeatures = agg.numFeatures - var featureIndex = 0 - while (featureIndex < numFeatures) { - val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, featureIndex, binIndex, label) - featureIndex += 1 + // Iterate over features. + if (featuresForNode.nonEmpty) { + // Use subsampled features + var featureIndexIdx = 0 + while (featureIndexIdx < featuresForNode.get.size) { + val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx)) + agg.nodeUpdate(nodeOffset, nodeIndex, featureIndexIdx, binIndex, label, instanceWeight) + featureIndexIdx += 1 + } + } else { + // Use all features + val numFeatures = agg.metadata.numFeatures + var featureIndex = 0 + while (featureIndex < numFeatures) { + val binIndex = treePoint.binnedFeatures(featureIndex) + agg.nodeUpdate(nodeOffset, nodeIndex, featureIndex, binIndex, label, instanceWeight) + featureIndex += 1 + } } } /** - * Returns an array of optimal splits for a group of nodes at a given level + * Given a group of nodes, this finds the best split for each node. * * @param input Training data: RDD of [[org.apache.spark.mllib.tree.impl.TreePoint]] * @param metadata Learning and dataset metadata - * @param level Level of the tree - * @param topNode Root node of the tree (or invalid node when training first level). + * @param topNodes Root node for each tree. Used for matching instances with nodes. + * @param nodesForGroup Mapping: treeIndex --> nodes to be split in tree + * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo, + * where nodeIndexInfo stores the index in the group and the + * feature subsets (if using feature subsets). * @param splits possible splits for all features, indexed (numFeatures)(numSplits) * @param bins possible bins for all features, indexed (numFeatures)(numBins) - * @param numGroups total number of node groups at the current level. Default value is set to 1. - * @param groupIndex index of the node group being processed. Default value is set to 0. - * @return (root, doneTraining) where: - * root = Root node (which is newly created on the first iteration), - * doneTraining = true if no more internal nodes were created. + * @param nodeQueue Queue of nodes to split, with values (treeIndex, node). + * Updated with new non-leaf nodes which are created. */ - private def findBestSplitsPerGroup( - input: RDD[TreePoint], + private[tree] def findBestSplits( + input: RDD[BaggedPoint[TreePoint]], metadata: DecisionTreeMetadata, - level: Int, - topNode: Node, + topNodes: Array[Node], + nodesForGroup: Map[Int, Array[Node]], + treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]], splits: Array[Array[Split]], bins: Array[Array[Bin]], - timer: TimeTracker, - numGroups: Int = 1, - groupIndex: Int = 0): (Node, Boolean) = { + nodeQueue: mutable.Queue[(Int, Node)], + timer: TimeTracker = new TimeTracker): Unit = { /* * The high-level descriptions of the best split optimizations are noted here. * - * *Level-wise training* - * We perform bin calculations for all nodes at the given level to avoid making multiple - * passes over the data. Thus, for a slightly increased computation and storage cost we save - * several iterations over the data especially at higher levels of the decision tree. + * *Group-wise training* + * We perform bin calculations for groups of nodes to reduce the number of + * passes over the data. Each iteration requires more computation and storage, + * but saves several iterations over the data. * * *Bin-wise computation* * We use a bin-wise best split computation strategy instead of a straightforward best split * computation strategy. Instead of analyzing each sample for contribution to the left/right * child node impurity of every split, we first categorize each feature of a sample into a - * bin. Each bin is an interval between a low and high split. Since each split, and thus bin, - * is ordered (read ordering for categorical variables in the findSplitsBins method), - * we exploit this structure to calculate aggregates for bins and then use these aggregates + * bin. We exploit this structure to calculate aggregates for bins and then use these aggregates * to calculate information gain for each split. * * *Aggregation over partitions* @@ -582,42 +475,15 @@ object DecisionTree extends Serializable with Logging { * drastically reduce the communication overhead. */ - // Common calculations for multiple nested methods: - - // numNodes: Number of nodes in this (level of tree, group), - // where nodes at deeper (larger) levels may be divided into groups. - val numNodes = Node.maxNodesInLevel(level) / numGroups + // numNodes: Number of nodes in this group + val numNodes = nodesForGroup.values.map(_.size).sum logDebug("numNodes = " + numNodes) - logDebug("numFeatures = " + metadata.numFeatures) logDebug("numClasses = " + metadata.numClasses) logDebug("isMulticlass = " + metadata.isMulticlass) logDebug("isMulticlassWithCategoricalFeatures = " + metadata.isMulticlassWithCategoricalFeatures) - // shift when more than one group is used at deep tree level - val groupShift = numNodes * groupIndex - - // Used for treePointToNodeIndex to get an index for this (level, group). - // - Node.startIndexInLevel(level) gives the global index offset for nodes at this level. - // - groupShift corrects for groups in this level before the current group. - val globalNodeIndexOffset = Node.startIndexInLevel(level) + groupShift - - /** - * Find the node index for the given example. - * Nodes are indexed from 0 at the start of this (level, group). - * If the example does not reach this level, returns a value < 0. - */ - def treePointToNodeIndex(treePoint: TreePoint): Int = { - if (level == 0) { - 0 - } else { - val globalNodeIndex = - predictNodeIndex(topNode, treePoint.binnedFeatures, bins, metadata.unorderedFeatures) - globalNodeIndex - globalNodeIndexOffset - } - } - /** * Performs a sequential aggregation over a partition. * @@ -626,21 +492,27 @@ object DecisionTree extends Serializable with Logging { * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for * each (node, feature, bin). - * @param treePoint Data point being aggregated. + * @param baggedPoint Data point being aggregated. * @return agg */ def binSeqOp( agg: DTStatsAggregator, - treePoint: TreePoint): DTStatsAggregator = { - val nodeIndex = treePointToNodeIndex(treePoint) - // If the example does not reach this level, then nodeIndex < 0. - // If the example reaches this level but is handled in a different group, - // then either nodeIndex < 0 (previous group) or nodeIndex >= numNodes (later group). - if (nodeIndex >= 0 && nodeIndex < numNodes) { - if (metadata.unorderedFeatures.isEmpty) { - orderedBinSeqOp(agg, treePoint, nodeIndex) - } else { - mixedBinSeqOp(agg, treePoint, nodeIndex, bins, metadata.unorderedFeatures) + baggedPoint: BaggedPoint[TreePoint]): DTStatsAggregator = { + treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => + val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures, + bins, metadata.unorderedFeatures) + val nodeInfo = nodeIndexToInfo.getOrElse(nodeIndex, null) + // If the example does not reach a node in this group, then nodeIndex = null. + if (nodeInfo != null) { + val aggNodeIndex = nodeInfo.nodeIndexInGroup + val featuresForNode = nodeInfo.featureSubset + val instanceWeight = baggedPoint.subsampleWeights(treeIndex) + if (metadata.unorderedFeatures.isEmpty) { + orderedBinSeqOp(agg, baggedPoint.datum, aggNodeIndex, instanceWeight, featuresForNode) + } else { + mixedBinSeqOp(agg, baggedPoint.datum, aggNodeIndex, bins, metadata.unorderedFeatures, + instanceWeight, featuresForNode) + } } } agg @@ -649,71 +521,62 @@ object DecisionTree extends Serializable with Logging { // Calculate bin aggregates. timer.start("aggregation") val binAggregates: DTStatsAggregator = { - val initAgg = new DTStatsAggregator(metadata, numNodes) + val initAgg = if (metadata.subsamplingFeatures) { + new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo) + } else { + new DTStatsAggregatorFixedFeatures(metadata, numNodes) + } input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp) } timer.stop("aggregation") - // Calculate best splits for all nodes at a given level + // Calculate best splits for all nodes in the group timer.start("chooseSplits") - // On the first iteration, we need to get and return the newly created root node. - var newTopNode: Node = topNode - - // Iterate over all nodes at this level - var nodeIndex = 0 - var internalNodeCount = 0 - while (nodeIndex < numNodes) { - val (split: Split, stats: InformationGainStats, predict: Predict) = - binsToBestSplit(binAggregates, nodeIndex, level, metadata, splits) - logDebug("best split = " + split) - - val globalNodeIndex = globalNodeIndexOffset + nodeIndex - // Extract info for this node at the current level. - val isLeaf = (stats.gain <= 0) || (level == metadata.maxDepth) - val node = - new Node(globalNodeIndex, predict.predict, isLeaf, Some(split), None, None, Some(stats)) - logDebug("Node = " + node) - - if (!isLeaf) { - internalNodeCount += 1 - } - if (level == 0) { - newTopNode = node - } else { - // Set parent. - val parentNode = Node.getNode(Node.parentIndex(globalNodeIndex), topNode) - if (Node.isLeftChild(globalNodeIndex)) { - parentNode.leftNode = Some(node) - } else { - parentNode.rightNode = Some(node) + // Iterate over all nodes in this group. + nodesForGroup.foreach { case (treeIndex, nodesForTree) => + nodesForTree.foreach { node => + val nodeIndex = node.id + val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex) + val aggNodeIndex = nodeInfo.nodeIndexInGroup + val featuresForNode = nodeInfo.featureSubset + val (split: Split, stats: InformationGainStats, predict: Predict) = + binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode) + logDebug("best split = " + split) + + // Extract info for this node. Create children if not leaf. + val isLeaf = (stats.gain <= 0) || (Node.indexToLevel(nodeIndex) == metadata.maxDepth) + assert(node.id == nodeIndex) + node.predict = predict.predict + node.isLeaf = isLeaf + node.stats = Some(stats) + logDebug("Node = " + node) + + if (!isLeaf) { + node.split = Some(split) + node.leftNode = Some(Node.emptyNode(Node.leftChildIndex(nodeIndex))) + node.rightNode = Some(Node.emptyNode(Node.rightChildIndex(nodeIndex))) + nodeQueue.enqueue((treeIndex, node.leftNode.get)) + nodeQueue.enqueue((treeIndex, node.rightNode.get)) + logDebug("leftChildIndex = " + node.leftNode.get.id + + ", impurity = " + stats.leftImpurity) + logDebug("rightChildIndex = " + node.rightNode.get.id + + ", impurity = " + stats.rightImpurity) } } - if (level < metadata.maxDepth) { - logDebug("leftChildIndex = " + Node.leftChildIndex(globalNodeIndex) + - ", impurity = " + stats.leftImpurity) - logDebug("rightChildIndex = " + Node.rightChildIndex(globalNodeIndex) + - ", impurity = " + stats.rightImpurity) - } - - nodeIndex += 1 } timer.stop("chooseSplits") - - val doneTraining = internalNodeCount == 0 - (newTopNode, doneTraining) } /** * Calculate the information gain for a given (feature, split) based upon left/right aggregates. * @param leftImpurityCalculator left node aggregates for this (feature, split) * @param rightImpurityCalculator right node aggregate for this (feature, split) - * @return information gain and statistics for all splits + * @return information gain and statistics for split */ private def calculateGainForSplit( leftImpurityCalculator: ImpurityCalculator, rightImpurityCalculator: ImpurityCalculator, - level: Int, metadata: DecisionTreeMetadata): InformationGainStats = { val leftCount = leftImpurityCalculator.count val rightCount = rightImpurityCalculator.count @@ -753,7 +616,7 @@ object DecisionTree extends Serializable with Logging { * Calculate predict value for current node, given stats of any split. * Note that this function is called only once for each node. * @param leftImpurityCalculator left node aggregates for a split - * @param rightImpurityCalculator right node aggregates for a node + * @param rightImpurityCalculator right node aggregates for a split * @return predict value for current node */ private def calculatePredict( @@ -770,27 +633,33 @@ object DecisionTree extends Serializable with Logging { /** * Find the best split for a node. * @param binAggregates Bin statistics. - * @param nodeIndex Index for node to split in this (level, group). - * @return tuple for best split: (Split, information gain) + * @param nodeIndex Index into aggregates for node to split in this group. + * @return tuple for best split: (Split, information gain, prediction at node) */ private def binsToBestSplit( binAggregates: DTStatsAggregator, nodeIndex: Int, - level: Int, - metadata: DecisionTreeMetadata, - splits: Array[Array[Split]]): (Split, InformationGainStats, Predict) = { + splits: Array[Array[Split]], + featuresForNode: Option[Array[Int]]): (Split, InformationGainStats, Predict) = { + + val metadata: DecisionTreeMetadata = binAggregates.metadata // calculate predict only once var predict: Option[Predict] = None // For each (feature, split), calculate the gain, and select the best (feature, split). - val (bestSplit, bestSplitStats) = Range(0, metadata.numFeatures).map { featureIndex => + val (bestSplit, bestSplitStats) = Range(0, metadata.numFeaturesPerNode).map { featureIndexIdx => + val featureIndex = if (featuresForNode.nonEmpty) { + featuresForNode.get.apply(featureIndexIdx) + } else { + featureIndexIdx + } val numSplits = metadata.numSplits(featureIndex) if (metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. // Afterwards, binAggregates for a bin is the sum of aggregates for // that bin + all preceding bins. - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndexIdx) var splitIndex = 0 while (splitIndex < numSplits) { binAggregates.mergeForNodeFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) @@ -803,26 +672,26 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) (splitIdx, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else if (metadata.isUnordered(featureIndex)) { // Unordered categorical feature val (leftChildOffset, rightChildOffset) = - binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndex) + binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndexIdx) val (bestFeatureSplitIndex, bestFeatureGainStats) = Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else { // Ordered categorical feature - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndex) + val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndexIdx) val numBins = metadata.numBins(featureIndex) /* Each bin is one category (feature value). @@ -887,7 +756,7 @@ object DecisionTree extends Serializable with Logging { binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, level, metadata) + val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) val categoriesForSplit = @@ -903,18 +772,6 @@ object DecisionTree extends Serializable with Logging { (bestSplit, bestSplitStats, predict.get) } - /** - * Get the number of values to be stored per node in the bin aggregates. - */ - private def getElementsPerNode(metadata: DecisionTreeMetadata): Long = { - val totalBins = metadata.numBins.map(_.toLong).sum - if (metadata.isClassification) { - metadata.numClasses * totalBins - } else { - 3 * totalBins - } - } - /** * Returns splits and bins for decision tree calculation. * Continuous and categorical features are handled differently. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala new file mode 100644 index 0000000000000..7fa7725e79e46 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -0,0 +1,451 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.Logging +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker} +import org.apache.spark.mllib.tree.impurity.Impurities +import org.apache.spark.mllib.tree.model._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils + +/** + * :: Experimental :: + * A class which implements a random forest learning algorithm for classification and regression. + * It supports both continuous and categorical features. + * + * The settings for featureSubsetStrategy are based on the following references: + * - log2: tested in Breiman (2001) + * - sqrt: recommended by Breiman manual for random forests + * - The defaults of sqrt (classification) and onethird (regression) match the R randomForest + * package. + * @see [[http://www.stat.berkeley.edu/~breiman/randomforest2001.pdf Breiman (2001)]] + * @see [[http://www.stat.berkeley.edu/~breiman/Using_random_forests_V3.1.pdf Breiman manual for + * random forests]] + * + * @param strategy The configuration parameters for the random forest algorithm which specify + * the type of algorithm (classification, regression, etc.), feature type + * (continuous, categorical), depth of the tree, quantile calculation strategy, + * etc. + * @param numTrees If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param seed Random seed for bootstrapping and choosing feature subsets. + */ +@Experimental +private class RandomForest ( + private val strategy: Strategy, + private val numTrees: Int, + featureSubsetStrategy: String, + private val seed: Int) + extends Serializable with Logging { + + strategy.assertValid() + require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.") + require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy), + s"RandomForest given invalid featureSubsetStrategy: $featureSubsetStrategy." + + s" Supported values: ${RandomForest.supportedFeatureSubsetStrategies.mkString(", ")}.") + + /** + * Method to train a decision tree model over an RDD + * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] + * @return RandomForestModel that can be used for prediction + */ + def train(input: RDD[LabeledPoint]): RandomForestModel = { + + val timer = new TimeTracker() + + timer.start("total") + + timer.start("init") + + val retaggedInput = input.retag(classOf[LabeledPoint]) + val metadata = + DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy) + logDebug("algo = " + strategy.algo) + logDebug("numTrees = " + numTrees) + logDebug("seed = " + seed) + logDebug("maxBins = " + metadata.maxBins) + logDebug("featureSubsetStrategy = " + featureSubsetStrategy) + logDebug("numFeaturesPerNode = " + metadata.numFeaturesPerNode) + + // Find the splits and the corresponding bins (interval between the splits) using a sample + // of the input data. + timer.start("findSplitsBins") + val (splits, bins) = DecisionTree.findSplitsBins(retaggedInput, metadata) + timer.stop("findSplitsBins") + logDebug("numBins: feature: number of bins") + logDebug(Range(0, metadata.numFeatures).map { featureIndex => + s"\t$featureIndex\t${metadata.numBins(featureIndex)}" + }.mkString("\n")) + + // Bin feature values (TreePoint representation). + // Cache input RDD for speedup during multiple passes. + val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) + val baggedInput = if (numTrees > 1) { + BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed) + } else { + BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + }.persist(StorageLevel.MEMORY_AND_DISK) + + // depth of the decision tree + val maxDepth = strategy.maxDepth + require(maxDepth <= 30, + s"DecisionTree currently only supports maxDepth <= 30, but was given maxDepth = $maxDepth.") + + // Max memory usage for aggregates + // TODO: Calculate memory usage more precisely. + val maxMemoryUsage: Long = strategy.maxMemoryInMB * 1024L * 1024L + logDebug("max memory usage for aggregates = " + maxMemoryUsage + " bytes.") + val maxMemoryPerNode = { + val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { + // Find numFeaturesPerNode largest bins to get an upper bound on memory usage. + Some(metadata.numBins.zipWithIndex.sortBy(- _._1) + .take(metadata.numFeaturesPerNode).map(_._2)) + } else { + None + } + RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L + } + require(maxMemoryPerNode <= maxMemoryUsage, + s"RandomForest/DecisionTree given maxMemoryInMB = ${strategy.maxMemoryInMB}," + + " which is too small for the given features." + + s" Minimum value = ${maxMemoryPerNode / (1024L * 1024L)}") + + timer.stop("init") + + /* + * The main idea here is to perform group-wise training of the decision tree nodes thus + * reducing the passes over the data from (# nodes) to (# nodes / maxNumberOfNodesPerGroup). + * Each data sample is handled by a particular node (or it reaches a leaf and is not used + * in lower levels). + */ + + // FIFO queue of nodes to train: (treeIndex, node) + val nodeQueue = new mutable.Queue[(Int, Node)]() + + val rng = new scala.util.Random() + rng.setSeed(seed) + + // Allocate and queue root nodes. + val topNodes: Array[Node] = Array.fill[Node](numTrees)(Node.emptyNode(nodeIndex = 1)) + Range(0, numTrees).foreach(treeIndex => nodeQueue.enqueue((treeIndex, topNodes(treeIndex)))) + + while (nodeQueue.nonEmpty) { + // Collect some nodes to split, and choose features for each node (if subsampling). + // Each group of nodes may come from one or multiple trees, and at multiple levels. + val (nodesForGroup, treeToNodeToIndexInfo) = + RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng) + // Sanity check (should never occur): + assert(nodesForGroup.size > 0, + s"RandomForest selected empty nodesForGroup. Error for unknown reason.") + + // Choose node splits, and enqueue new nodes as needed. + timer.start("findBestSplits") + DecisionTree.findBestSplits(baggedInput, + metadata, topNodes, nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue, timer) + timer.stop("findBestSplits") + } + + timer.stop("total") + + logInfo("Internal timing for DecisionTree:") + logInfo(s"$timer") + + val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo)) + RandomForestModel.build(trees) + } + +} + +object RandomForest extends Serializable with Logging { + + /** + * Method to train a decision tree model for binary or multiclass classification. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels should take values {0, 1, ..., numClasses-1}. + * @param strategy Parameters for training each tree in the forest. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + strategy: Strategy, + numTrees: Int, + featureSubsetStrategy: String, + seed: Int): RandomForestModel = { + require(strategy.algo == Classification, + s"RandomForest.trainClassifier given Strategy with invalid algo: ${strategy.algo}") + val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed) + rf.train(input) + } + + /** + * Method to train a decision tree model for binary or multiclass classification. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels should take values {0, 1, ..., numClasses-1}. + * @param numClassesForClassification number of classes for classification. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param impurity Criterion used for information gain calculation. + * Supported values: "gini" (recommended) or "entropy". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: Map[Int, Int], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int = Utils.random.nextInt()): RandomForestModel = { + val impurityType = Impurities.fromString(impurity) + val strategy = new Strategy(Classification, impurityType, maxDepth, + numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo) + trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainClassifier]] + */ + def trainClassifier( + input: JavaRDD[LabeledPoint], + numClassesForClassification: Int, + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int): RandomForestModel = { + trainClassifier(input.rdd, numClassesForClassification, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) + } + + /** + * Method to train a decision tree model for regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels are real numbers. + * @param strategy Parameters for training each tree in the forest. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + strategy: Strategy, + numTrees: Int, + featureSubsetStrategy: String, + seed: Int): RandomForestModel = { + require(strategy.algo == Regression, + s"RandomForest.trainRegressor given Strategy with invalid algo: ${strategy.algo}") + val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed) + rf.train(input) + } + + /** + * Method to train a decision tree model for regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * Labels are real numbers. + * @param categoricalFeaturesInfo Map storing arity of categorical features. + * E.g., an entry (n -> k) indicates that feature n is categorical + * with k categories indexed from 0: {0, 1, ..., k-1}. + * @param numTrees Number of trees in the random forest. + * @param featureSubsetStrategy Number of features to consider for splits at each node. + * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * If "auto" is set, this parameter is set based on numTrees: + * if numTrees == 1, set to "all"; + * if numTrees > 1 (forest) set to "sqrt" for classification and + * to "onethird" for regression. + * @param impurity Criterion used for information gain calculation. + * Supported values: "variance". + * @param maxDepth Maximum depth of the tree. + * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. + * (suggested value: 4) + * @param maxBins maximum number of bins used for splitting features + * (suggested value: 100) + * @param seed Random seed for bootstrapping and choosing feature subsets. + * @return RandomForestModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + categoricalFeaturesInfo: Map[Int, Int], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int = Utils.random.nextInt()): RandomForestModel = { + val impurityType = Impurities.fromString(impurity) + val strategy = new Strategy(Regression, impurityType, maxDepth, + 0, maxBins, Sort, categoricalFeaturesInfo) + trainRegressor(input, strategy, numTrees, featureSubsetStrategy, seed) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.RandomForest$#trainRegressor]] + */ + def trainRegressor( + input: JavaRDD[LabeledPoint], + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + numTrees: Int, + featureSubsetStrategy: String, + impurity: String, + maxDepth: Int, + maxBins: Int, + seed: Int): RandomForestModel = { + trainRegressor(input.rdd, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) + } + + /** + * List of supported feature subset sampling strategies. + */ + val supportedFeatureSubsetStrategies: Array[String] = + Array("auto", "all", "sqrt", "log2", "onethird") + + private[tree] class NodeIndexInfo( + val nodeIndexInGroup: Int, + val featureSubset: Option[Array[Int]]) extends Serializable + + /** + * Pull nodes off of the queue, and collect a group of nodes to be split on this iteration. + * This tracks the memory usage for aggregates and stops adding nodes when too much memory + * will be needed; this allows an adaptive number of nodes since different nodes may require + * different amounts of memory (if featureSubsetStrategy is not "all"). + * + * @param nodeQueue Queue of nodes to split. + * @param maxMemoryUsage Bound on size of aggregate statistics. + * @return (nodesForGroup, treeToNodeToIndexInfo). + * nodesForGroup holds the nodes to split: treeIndex --> nodes in tree. + * treeToNodeToIndexInfo holds indices selected features for each node: + * treeIndex --> (global) node index --> (node index in group, feature indices). + * The (global) node index is the index in the tree; the node index in group is the + * index in [0, numNodesInGroup) of the node in this group. + * The feature indices are None if not subsampling features. + */ + private[tree] def selectNodesToSplit( + nodeQueue: mutable.Queue[(Int, Node)], + maxMemoryUsage: Long, + metadata: DecisionTreeMetadata, + rng: scala.util.Random): (Map[Int, Array[Node]], Map[Int, Map[Int, NodeIndexInfo]]) = { + // Collect some nodes to split: + // nodesForGroup(treeIndex) = nodes to split + val mutableNodesForGroup = new mutable.HashMap[Int, mutable.ArrayBuffer[Node]]() + val mutableTreeToNodeToIndexInfo = + new mutable.HashMap[Int, mutable.HashMap[Int, NodeIndexInfo]]() + var memUsage: Long = 0L + var numNodesInGroup = 0 + while (nodeQueue.nonEmpty && memUsage < maxMemoryUsage) { + val (treeIndex, node) = nodeQueue.head + // Choose subset of features for node (if subsampling). + val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { + // TODO: Use more efficient subsampling? (use selection-and-rejection or reservoir) + Some(rng.shuffle(Range(0, metadata.numFeatures).toList) + .take(metadata.numFeaturesPerNode).toArray) + } else { + None + } + // Check if enough memory remains to add this node to the group. + val nodeMemUsage = RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L + if (memUsage + nodeMemUsage <= maxMemoryUsage) { + nodeQueue.dequeue() + mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[Node]()) += node + mutableTreeToNodeToIndexInfo + .getOrElseUpdate(treeIndex, new mutable.HashMap[Int, NodeIndexInfo]())(node.id) + = new NodeIndexInfo(numNodesInGroup, featureSubset) + } + numNodesInGroup += 1 + memUsage += nodeMemUsage + } + // Convert mutable maps to immutable ones. + val nodesForGroup: Map[Int, Array[Node]] = mutableNodesForGroup.mapValues(_.toArray).toMap + val treeToNodeToIndexInfo = mutableTreeToNodeToIndexInfo.mapValues(_.toMap).toMap + (nodesForGroup, treeToNodeToIndexInfo) + } + + /** + * Get the number of values to be stored for this node in the bin aggregates. + * @param featureSubset Indices of features which may be split at this node. + * If None, then use all features. + */ + private[tree] def aggregateSizeForNode( + metadata: DecisionTreeMetadata, + featureSubset: Option[Array[Int]]): Long = { + val totalBins = if (featureSubset.nonEmpty) { + featureSubset.get.map(featureIndex => metadata.numBins(featureIndex).toLong).sum + } else { + metadata.numBins.map(_.toLong).sum + } + if (metadata.isClassification) { + metadata.numClasses * totalBins + } else { + 3 * totalBins + } + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala new file mode 100644 index 0000000000000..937c8a2ac5836 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import cern.jet.random.Poisson +import cern.jet.random.engine.DRand + +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +/** + * Internal representation of a datapoint which belongs to several subsamples of the same dataset, + * particularly for bagging (e.g., for random forests). + * + * This holds one instance, as well as an array of weights which represent the (weighted) + * number of times which this instance appears in each subsample. + * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that + * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively. + * + * @param datum Data instance + * @param subsampleWeights Weight of this instance in each subsampled dataset. + * + * TODO: This does not currently support (Double) weighted instances. Once MLlib has weighted + * dataset support, update. (We store subsampleWeights as Double for this future extension.) + */ +private[tree] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double]) + extends Serializable + +private[tree] object BaggedPoint { + + /** + * Convert an input dataset into its BaggedPoint representation, + * choosing subsample counts for each instance. + * Each subsample has the same number of instances as the original dataset, + * and is created by subsampling with replacement. + * @param input Input dataset. + * @param numSubsamples Number of subsamples of this RDD to take. + * @param seed Random seed. + * @return BaggedPoint dataset representation + */ + def convertToBaggedRDD[Datum]( + input: RDD[Datum], + numSubsamples: Int, + seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = { + input.mapPartitionsWithIndex { (partitionIndex, instances) => + // TODO: Support different sampling rates, and sampling without replacement. + // Use random seed = seed + partitionIndex + 1 to make generation reproducible. + val poisson = new Poisson(1.0, new DRand(seed + partitionIndex + 1)) + instances.map { instance => + val subsampleWeights = new Array[Double](numSubsamples) + var subsampleIndex = 0 + while (subsampleIndex < numSubsamples) { + subsampleWeights(subsampleIndex) = poisson.nextInt() + subsampleIndex += 1 + } + new BaggedPoint(instance, subsampleWeights) + } + } + } + + def convertToBaggedRDDWithoutSampling[Datum](input: RDD[Datum]): RDD[BaggedPoint[Datum]] = { + input.map(datum => new BaggedPoint(datum, Array(1.0))) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index 61a94246711bf..d49df7a016375 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -17,16 +17,17 @@ package org.apache.spark.mllib.tree.impl +import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.impurity._ /** * DecisionTree statistics aggregator. * This holds a flat array of statistics for a set of (nodes, features, bins) * and helps with indexing. + * This class is abstract to support learning with and without feature subsampling. */ -private[tree] class DTStatsAggregator( - val metadata: DecisionTreeMetadata, - val numNodes: Int) extends Serializable { +private[tree] abstract class DTStatsAggregator( + val metadata: DecisionTreeMetadata) extends Serializable { /** * [[ImpurityAggregator]] instance specifying the impurity type. @@ -43,49 +44,21 @@ private[tree] class DTStatsAggregator( */ val statsSize: Int = impurityAggregator.statsSize - val numFeatures: Int = metadata.numFeatures - - /** - * Number of bins for each feature. This is indexed by the feature index. - */ - val numBins: Array[Int] = metadata.numBins - - /** - * Number of splits for the given feature. - */ - def numSplits(featureIndex: Int): Int = metadata.numSplits(featureIndex) - /** * Indicator for each feature of whether that feature is an unordered feature. * TODO: Is Array[Boolean] any faster? */ def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) - /** - * Offset for each feature for calculating indices into the [[allStats]] array. - */ - private val featureOffsets: Array[Int] = { - numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) - } - - /** - * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. - */ - private val nodeStride: Int = featureOffsets.last - /** * Total number of elements stored in this aggregator. */ - val allStatsSize: Int = numNodes * nodeStride + def allStatsSize: Int /** - * Flat array of elements. - * Index for start of stats for a (node, feature, bin) is: - * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize - * Note: For unordered features, the left child stats have binIndex in [0, numBins(featureIndex)) - * and the right child stats in [numBins(featureIndex), 2 * numBins(featureIndex)) + * Get flat array of elements stored in this aggregator. */ - val allStats: Array[Double] = new Array[Double](allStatsSize) + protected def allStats: Array[Double] /** * Get an [[ImpurityCalculator]] for a given (node, feature, bin). @@ -102,36 +75,39 @@ private[tree] class DTStatsAggregator( /** * Update the stats for a given (node, feature, bin) for ordered features, using the given label. */ - def update(nodeIndex: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { - val i = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label) + def update( + nodeIndex: Int, + featureIndex: Int, + binIndex: Int, + label: Double, + instanceWeight: Double): Unit = { + val i = getNodeFeatureOffset(nodeIndex, featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label, instanceWeight) } /** * Pre-compute node offset for use with [[nodeUpdate]]. */ - def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride + def getNodeOffset(nodeIndex: Int): Int /** * Faster version of [[update]]. * Update the stats for a given (node, feature, bin) for ordered features, using the given label. * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. */ - def nodeUpdate(nodeOffset: Int, featureIndex: Int, binIndex: Int, label: Double): Unit = { - val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label) - } + def nodeUpdate( + nodeOffset: Int, + nodeIndex: Int, + featureIndex: Int, + binIndex: Int, + label: Double, + instanceWeight: Double): Unit /** * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. * For ordered features only. */ - def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { - require(!isUnordered(featureIndex), - s"DTStatsAggregator.getNodeFeatureOffset is for ordered features only, but was called" + - s" for unordered feature $featureIndex.") - nodeIndex * nodeStride + featureOffsets(featureIndex) - } + def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int /** * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. @@ -140,9 +116,9 @@ private[tree] class DTStatsAggregator( def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = { require(isUnordered(featureIndex), s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + - s" but was called for ordered feature $featureIndex.") - val baseOffset = nodeIndex * nodeStride + featureOffsets(featureIndex) - (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) + s" but was called for ordered feature $featureIndex.") + val baseOffset = getNodeFeatureOffset(nodeIndex, featureIndex) + (baseOffset, baseOffset + (metadata.numBins(featureIndex) >> 1) * statsSize) } /** @@ -154,8 +130,13 @@ private[tree] class DTStatsAggregator( * (node, feature, left/right child) offset from * [[getLeftRightNodeFeatureOffsets]]. */ - def nodeFeatureUpdate(nodeFeatureOffset: Int, binIndex: Int, label: Double): Unit = { - impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label) + def nodeFeatureUpdate( + nodeFeatureOffset: Int, + binIndex: Int, + label: Double, + instanceWeight: Double): Unit = { + impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label, + instanceWeight) } /** @@ -189,7 +170,139 @@ private[tree] class DTStatsAggregator( } this } +} + +/** + * DecisionTree statistics aggregator. + * This holds a flat array of statistics for a set of (nodes, features, bins) + * and helps with indexing. + * + * This instance of [[DTStatsAggregator]] is used when not subsampling features. + * + * @param numNodes Number of nodes to collect statistics for. + */ +private[tree] class DTStatsAggregatorFixedFeatures( + metadata: DecisionTreeMetadata, + numNodes: Int) extends DTStatsAggregator(metadata) { + + /** + * Offset for each feature for calculating indices into the [[allStats]] array. + * Mapping: featureIndex --> offset + */ + private val featureOffsets: Array[Int] = { + metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) + } + + /** + * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. + */ + private val nodeStride: Int = featureOffsets.last + override val allStatsSize: Int = numNodes * nodeStride + + /** + * Flat array of elements. + * Index for start of stats for a (node, feature, bin) is: + * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, the left child stats precede the right child stats + * in the binIndex order. + */ + override protected val allStats: Array[Double] = new Array[Double](allStatsSize) + + override def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride + + override def nodeUpdate( + nodeOffset: Int, + nodeIndex: Int, + featureIndex: Int, + binIndex: Int, + label: Double, + instanceWeight: Double): Unit = { + val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label, instanceWeight) + } + + override def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { + nodeIndex * nodeStride + featureOffsets(featureIndex) + } +} + +/** + * DecisionTree statistics aggregator. + * This holds a flat array of statistics for a set of (nodes, features, bins) + * and helps with indexing. + * + * This instance of [[DTStatsAggregator]] is used when subsampling features. + * + * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo, + * where nodeIndexInfo stores the index in the group and the + * feature subsets (if using feature subsets). + */ +private[tree] class DTStatsAggregatorSubsampledFeatures( + metadata: DecisionTreeMetadata, + treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]]) extends DTStatsAggregator(metadata) { + + /** + * For each node, offset for each feature for calculating indices into the [[allStats]] array. + * Mapping: nodeIndex --> featureIndex --> offset + */ + private val featureOffsets: Array[Array[Int]] = { + val numNodes: Int = treeToNodeToIndexInfo.values.map(_.size).sum + val offsets = new Array[Array[Int]](numNodes) + treeToNodeToIndexInfo.foreach { case (treeIndex, nodeToIndexInfo) => + nodeToIndexInfo.foreach { case (globalNodeIndex, nodeInfo) => + offsets(nodeInfo.nodeIndexInGroup) = nodeInfo.featureSubset.get.map(metadata.numBins(_)) + .scanLeft(0)((total, nBins) => total + statsSize * nBins) + } + } + offsets + } + + /** + * For each node, offset for each feature for calculating indices into the [[allStats]] array. + */ + protected val nodeOffsets: Array[Int] = featureOffsets.map(_.last).scanLeft(0)(_ + _) + + override val allStatsSize: Int = nodeOffsets.last + + /** + * Flat array of elements. + * Index for start of stats for a (node, feature, bin) is: + * index = nodeOffsets(nodeIndex) + featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, the left child stats precede the right child stats + * in the binIndex order. + */ + override protected val allStats: Array[Double] = new Array[Double](allStatsSize) + + override def getNodeOffset(nodeIndex: Int): Int = nodeOffsets(nodeIndex) + + /** + * Faster version of [[update]]. + * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. + * @param featureIndex Index of feature in featuresForNodes(nodeIndex). + * Note: This is NOT the original feature index. + */ + override def nodeUpdate( + nodeOffset: Int, + nodeIndex: Int, + featureIndex: Int, + binIndex: Int, + label: Double, + instanceWeight: Double): Unit = { + val i = nodeOffset + featureOffsets(nodeIndex)(featureIndex) + binIndex * statsSize + impurityAggregator.update(allStats, i, label, instanceWeight) + } + + /** + * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * For ordered features only. + * @param featureIndex Index of feature in featuresForNodes(nodeIndex). + * Note: This is NOT the original feature index. + */ + override def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { + nodeOffsets(nodeIndex) + featureOffsets(nodeIndex)(featureIndex) + } } private[tree] object DTStatsAggregator extends Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index b6d49e5555b1a..212dce25236e0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -48,7 +48,9 @@ private[tree] class DecisionTreeMetadata( val quantileStrategy: QuantileStrategy, val maxDepth: Int, val minInstancesPerNode: Int, - val minInfoGain: Double) extends Serializable { + val minInfoGain: Double, + val numTrees: Int, + val numFeaturesPerNode: Int) extends Serializable { def isUnordered(featureIndex: Int): Boolean = unorderedFeatures.contains(featureIndex) @@ -73,6 +75,11 @@ private[tree] class DecisionTreeMetadata( numBins(featureIndex) - 1 } + /** + * Indicates if feature subsampling is being used. + */ + def subsamplingFeatures: Boolean = numFeatures != numFeaturesPerNode + } private[tree] object DecisionTreeMetadata { @@ -82,7 +89,11 @@ private[tree] object DecisionTreeMetadata { * This computes which categorical features will be ordered vs. unordered, * as well as the number of splits and bins for each feature. */ - def buildMetadata(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeMetadata = { + def buildMetadata( + input: RDD[LabeledPoint], + strategy: Strategy, + numTrees: Int, + featureSubsetStrategy: String): DecisionTreeMetadata = { val numFeatures = input.take(1)(0).features.size val numExamples = input.count() @@ -128,13 +139,43 @@ private[tree] object DecisionTreeMetadata { } } + // Set number of features to use per node (for random forests). + val _featureSubsetStrategy = featureSubsetStrategy match { + case "auto" => + if (numTrees == 1) { + "all" + } else { + if (strategy.algo == Classification) { + "sqrt" + } else { + "onethird" + } + } + case _ => featureSubsetStrategy + } + val numFeaturesPerNode: Int = _featureSubsetStrategy match { + case "all" => numFeatures + case "sqrt" => math.sqrt(numFeatures).ceil.toInt + case "log2" => math.max(1, (math.log(numFeatures) / math.log(2)).ceil.toInt) + case "onethird" => (numFeatures / 3.0).ceil.toInt + } + new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, strategy.impurity, strategy.quantileCalculationStrategy, strategy.maxDepth, - strategy.minInstancesPerNode, strategy.minInfoGain) + strategy.minInstancesPerNode, strategy.minInfoGain, numTrees, numFeaturesPerNode) } /** + * Version of [[buildMetadata()]] for DecisionTree. + */ + def buildMetadata( + input: RDD[LabeledPoint], + strategy: Strategy): DecisionTreeMetadata = { + buildMetadata(input, strategy, numTrees = 1, featureSubsetStrategy = "all") + } + + /** * Given the arity of a categorical feature (arity = number of categories), * return the number of bins for the feature if it is to be treated as an unordered feature. * There is 1 split for every partitioning of categories into 2 disjoint, non-empty sets; diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 1c8afc2d0f4bc..0e02345aa3774 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -89,12 +89,12 @@ private[tree] class EntropyAggregator(numClasses: Int) * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { if (label >= statsSize) { throw new IllegalArgumentException(s"EntropyAggregator given label $label" + s" but requires label < numClasses (= $statsSize).") } - allStats(offset + label.toInt) += 1 + allStats(offset + label.toInt) += instanceWeight } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index 5cfdf345d163c..7c83cd48e16a0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -85,12 +85,12 @@ private[tree] class GiniAggregator(numClasses: Int) * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit = { + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { if (label >= statsSize) { throw new IllegalArgumentException(s"GiniAggregator given label $label" + s" but requires label < numClasses (= $statsSize).") } - allStats(offset + label.toInt) += 1 + allStats(offset + label.toInt) += instanceWeight } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index 5a047d6cb5480..60e2ab2bb829e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -78,7 +78,7 @@ private[tree] abstract class ImpurityAggregator(val statsSize: Int) extends Seri * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit /** * Get an [[ImpurityCalculator]] for a (node, feature, bin). diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index e9ccecb1b8067..df9eafa5da16a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -75,10 +75,10 @@ private[tree] class VarianceAggregator() * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double): Unit = { - allStats(offset) += 1 - allStats(offset + 1) += label - allStats(offset + 2) += label * label + def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { + allStats(offset) += instanceWeight + allStats(offset + 1) += instanceWeight * label + allStats(offset + 2) += instanceWeight * label * label } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 5f0095d23c7ed..56c3e25d9285f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -41,12 +41,12 @@ import org.apache.spark.mllib.linalg.Vector @DeveloperApi class Node ( val id: Int, - val predict: Double, - val isLeaf: Boolean, - val split: Option[Split], + var predict: Double, + var isLeaf: Boolean, + var split: Option[Split], var leftNode: Option[Node], var rightNode: Option[Node], - val stats: Option[InformationGainStats]) extends Serializable with Logging { + var stats: Option[InformationGainStats]) extends Serializable with Logging { override def toString = "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + "split = " + split + ", stats = " + stats @@ -167,6 +167,11 @@ class Node ( private[tree] object Node { + /** + * Return a node with the given node id (but nothing else set). + */ + def emptyNode(nodeIndex: Int): Node = new Node(nodeIndex, 0, false, None, None, None, None) + /** * Return the index of the left child of this node. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala new file mode 100644 index 0000000000000..538c0e233202a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.model + +import scala.collection.mutable + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.rdd.RDD + +/** + * :: Experimental :: + * Random forest model for classification or regression. + * This model stores a collection of [[DecisionTreeModel]] instances and uses them to make + * aggregate predictions. + * @param trees Trees which make up this forest. This cannot be empty. + * @param algo algorithm type -- classification or regression + */ +@Experimental +class RandomForestModel(val trees: Array[DecisionTreeModel], val algo: Algo) extends Serializable { + + require(trees.size > 0, s"RandomForestModel cannot be created with empty trees collection.") + + /** + * Predict values for a single data point. + * + * @param features array representing a single data point + * @return Double prediction from the trained model + */ + def predict(features: Vector): Double = { + algo match { + case Classification => + val predictionToCount = new mutable.HashMap[Int, Int]() + trees.foreach { tree => + val prediction = tree.predict(features).toInt + predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1 + } + predictionToCount.maxBy(_._2)._1 + case Regression => + trees.map(_.predict(features)).sum / trees.size + } + } + + /** + * Predict values for the given data set. + * + * @param features RDD representing data points to be predicted + * @return RDD[Double] where each entry contains the corresponding prediction + */ + def predict(features: RDD[Vector]): RDD[Double] = { + features.map(x => predict(x)) + } + + /** + * Get number of trees in forest. + */ + def numTrees: Int = trees.size + + /** + * Print full model. + */ + override def toString: String = { + val header = algo match { + case Classification => + s"RandomForestModel classifier with $numTrees trees\n" + case Regression => + s"RandomForestModel regressor with $numTrees trees\n" + case _ => throw new IllegalArgumentException( + s"RandomForestModel given unknown algo parameter: $algo.") + } + header + trees.zipWithIndex.map { case (tree, treeIndex) => + s" Tree $treeIndex:\n" + tree.topNode.subtreeToString(4) + }.fold("")(_ + _) + } + +} + +private[tree] object RandomForestModel { + + def build(trees: Array[DecisionTreeModel]): RandomForestModel = { + require(trees.size > 0, s"RandomForestModel cannot be created with empty trees collection.") + val algo: Algo = trees(0).algo + require(trees.forall(_.algo == algo), + "RandomForestModel cannot combine trees which have different output types" + + " (classification/regression).") + new RandomForestModel(trees, algo) + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 2b2e579b992f6..a48ed71a1c5fc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ +import scala.collection.mutable import org.scalatest.FunSuite @@ -26,39 +27,13 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.tree.configuration.Strategy -import org.apache.spark.mllib.tree.impl.{DecisionTreeMetadata, TreePoint} +import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{InformationGainStats, DecisionTreeModel, Node} import org.apache.spark.mllib.util.LocalSparkContext class DecisionTreeSuite extends FunSuite with LocalSparkContext { - def validateClassifier( - model: DecisionTreeModel, - input: Seq[LabeledPoint], - requiredAccuracy: Double) { - val predictions = input.map(x => model.predict(x.features)) - val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => - prediction != expected.label - } - val accuracy = (input.length - numOffPredictions).toDouble / input.length - assert(accuracy >= requiredAccuracy, - s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") - } - - def validateRegressor( - model: DecisionTreeModel, - input: Seq[LabeledPoint], - requiredMSE: Double) { - val predictions = input.map(x => model.predict(x.features)) - val squaredError = predictions.zip(input).map { case (prediction, expected) => - val err = prediction - expected.label - err * err - }.sum - val mse = squaredError / input.length - assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") - } - test("Binary classification with continuous features: split and bin calculation") { val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() assert(arr.length === 1000) @@ -233,7 +208,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) - // 2^10 - 1 > 100, so categorical features will be ordered + // 2^(10-1) - 1 > 100, so categorical features will be ordered val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) @@ -269,9 +244,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(splits(0).length === 0) assert(bins(0).length === 0) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode: Node, doneTraining: Boolean) = - DecisionTree.findBestSplits(treeInput, metadata, 0, null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.categories === List(1.0)) @@ -299,10 +272,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.categories.length === 1) @@ -331,7 +301,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(!metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) - validateRegressor(model, arr, 0.0) + DecisionTreeSuite.validateRegressor(model, arr, 0.0) assert(model.numNodes === 3) assert(model.depth === 1) } @@ -352,12 +322,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -381,12 +346,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -411,12 +371,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -441,12 +396,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(bins.length === 2) assert(bins(0).length === 100) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) - - val split = rootNode.split.get - assert(split.feature === 0) + val rootNode = DecisionTree.train(rdd, strategy).topNode val stats = rootNode.stats.get assert(stats.gain === 0) @@ -471,25 +421,39 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, numClassesForClassification = 2, maxBins = 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) - val rootNodeCopy1 = modelOneNode.topNode.deepCopy() - val rootNodeCopy2 = modelOneNode.topNode.deepCopy() + val rootNode1 = modelOneNode.topNode.deepCopy() + val rootNode2 = modelOneNode.topNode.deepCopy() + assert(rootNode1.leftNode.nonEmpty) + assert(rootNode1.rightNode.nonEmpty) - // Single group second level tree construction. val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, - rootNodeCopy1, splits, bins, 10) - assert(rootNode.leftNode.nonEmpty) - assert(rootNode.rightNode.nonEmpty) + val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + + // Single group second level tree construction. + val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) + val treeToNodeToIndexInfo = Map((0, Map( + (rootNode1.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None)), + (rootNode1.rightNode.get.id, new RandomForest.NodeIndexInfo(1, None))))) + val nodeQueue = new mutable.Queue[(Int, Node)]() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode1), + nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue) val children1 = new Array[Node](2) - children1(0) = rootNode.leftNode.get - children1(1) = rootNode.rightNode.get - - // maxLevelForSingleGroup parameter is set to 0 to force splitting into groups for second - // level tree construction. - val (rootNode2, _) = DecisionTree.findBestSplits(treeInput, metadata, 1, - rootNodeCopy2, splits, bins, 0) - assert(rootNode2.leftNode.nonEmpty) - assert(rootNode2.rightNode.nonEmpty) + children1(0) = rootNode1.leftNode.get + children1(1) = rootNode1.rightNode.get + + // Train one second-level node at a time. + val nodesForGroupA = Map((0, Array(rootNode2.leftNode.get))) + val treeToNodeToIndexInfoA = Map((0, Map( + (rootNode2.leftNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupA, treeToNodeToIndexInfoA, splits, bins, nodeQueue) + val nodesForGroupB = Map((0, Array(rootNode2.rightNode.get))) + val treeToNodeToIndexInfoB = Map((0, Map( + (rootNode2.rightNode.get.id, new RandomForest.NodeIndexInfo(0, None))))) + nodeQueue.clear() + DecisionTree.findBestSplits(baggedInput, metadata, Array(rootNode2), + nodesForGroupB, treeToNodeToIndexInfoB, splits, bins, nodeQueue) val children2 = new Array[Node](2) children2(0) = rootNode2.leftNode.get children2(1) = rootNode2.rightNode.get @@ -521,10 +485,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.feature === 0) @@ -544,7 +505,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 1.0) + DecisionTreeSuite.validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) } @@ -561,7 +522,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 2) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 1.0) + DecisionTreeSuite.validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) assert(model.topNode.split.get.feature === 1) @@ -581,14 +542,11 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(metadata.isUnordered(featureIndex = 1)) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 1.0) + DecisionTreeSuite.validateClassifier(model, arr, 1.0) assert(model.numNodes === 3) assert(model.depth === 1) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = model.topNode val split = rootNode.split.get assert(split.feature === 0) @@ -610,12 +568,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 0.9) + DecisionTreeSuite.validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = model.topNode val split = rootNode.split.get assert(split.feature === 1) @@ -635,12 +590,9 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 0.9) + DecisionTreeSuite.validateClassifier(model, arr, 0.9) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = model.topNode val split = rootNode.split.get assert(split.feature === 1) @@ -660,10 +612,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) - val (splits, bins) = DecisionTree.findSplitsBins(rdd, metadata) - val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get assert(split.feature === 0) @@ -682,7 +631,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(strategy.isMulticlassClassification) val model = DecisionTree.train(rdd, strategy) - validateClassifier(model, arr, 0.6) + DecisionTreeSuite.validateClassifier(model, arr, 0.6) } test("split must satisfy min instances per node requirements") { @@ -691,24 +640,20 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { arr(1) = new LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))) arr(2) = new LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0)))) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, numClassesForClassification = 2, minInstancesPerNode = 2) - val model = DecisionTree.train(input, strategy) + val model = DecisionTree.train(rdd, strategy) assert(model.topNode.isLeaf) assert(model.topNode.predict == 0.0) - val predicts = input.map(p => model.predict(p.features)).collect() + val predicts = rdd.map(p => model.predict(p.features)).collect() predicts.foreach { predict => assert(predict == 0.0) } - // test for findBestSplits when no valid split can be found - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + // test when no valid split can be found + val rootNode = model.topNode val gain = rootNode.stats.get assert(gain == InformationGainStats.invalidInformationGainStats) @@ -723,15 +668,12 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { arr(2) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) arr(3) = new LabeledPoint(0.0, Vectors.dense(0.0, 0.0)) - val input = sc.parallelize(arr) + val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2), numClassesForClassification = 2, minInstancesPerNode = 2) - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + + val rootNode = DecisionTree.train(rdd, strategy).topNode val split = rootNode.split.get val gain = rootNode.stats.get @@ -757,12 +699,8 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(predict == 0.0) } - // test for findBestSplits when no valid split can be found - val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) - val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) - val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val (rootNode, doneTraining) = DecisionTree.findBestSplits(treeInput, metadata, 0, - null, splits, bins, 10) + // test when no valid split can be found + val rootNode = model.topNode val gain = rootNode.stats.get assert(gain == InformationGainStats.invalidInformationGainStats) @@ -771,6 +709,32 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { object DecisionTreeSuite { + def validateClassifier( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredAccuracy: Double) { + val predictions = input.map(x => model.predict(x.features)) + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + prediction != expected.label + } + val accuracy = (input.length - numOffPredictions).toDouble / input.length + assert(accuracy >= requiredAccuracy, + s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") + } + + def validateRegressor( + model: DecisionTreeModel, + input: Seq[LabeledPoint], + requiredMSE: Double) { + val predictions = input.map(x => model.predict(x.features)) + val squaredError = predictions.zip(input).map { case (prediction, expected) => + val err = prediction - expected.label + err * err + }.sum + val mse = squaredError / input.length + assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") + } + def generateOrderedLabeledPointsWithLabel0(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala new file mode 100644 index 0000000000000..30669fcd1c75b --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -0,0 +1,245 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import scala.collection.mutable + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata} +import org.apache.spark.mllib.tree.impurity.{Gini, Variance} +import org.apache.spark.mllib.tree.model.{Node, RandomForestModel} +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.util.StatCounter + +/** + * Test suite for [[RandomForest]]. + */ +class RandomForestSuite extends FunSuite with LocalSparkContext { + + test("BaggedPoint RDD: without subsampling") { + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1) + val rdd = sc.parallelize(arr) + val baggedRDD = BaggedPoint.convertToBaggedRDDWithoutSampling(rdd) + baggedRDD.collect().foreach { baggedPoint => + assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1) + } + } + + test("BaggedPoint RDD: with subsampling") { + val numSubsamples = 100 + val (expectedMean, expectedStddev) = (1.0, 1.0) + + val seeds = Array(123, 5354, 230, 349867, 23987) + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1) + val rdd = sc.parallelize(arr) + seeds.foreach { seed => + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, numSubsamples, seed = seed) + val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + RandomForestSuite.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, + expectedStddev, epsilon = 0.01) + } + } + + test("Binary classification with continuous features:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 50) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + val numTrees = 1 + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + + val rf = RandomForest.trainClassifier(rdd, strategy, numTrees = numTrees, + featureSubsetStrategy = "auto", seed = 123) + assert(rf.trees.size === 1) + val rfTree = rf.trees(0) + + val dt = DecisionTree.train(rdd, strategy) + + RandomForestSuite.validateClassifier(rf, arr, 0.9) + DecisionTreeSuite.validateClassifier(dt, arr, 0.9) + + // Make sure trees are the same. + assert(rfTree.toString == dt.toString) + } + + test("Regression with continuous features:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 50) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + val numTrees = 1 + + val strategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + + val rf = RandomForest.trainRegressor(rdd, strategy, numTrees = numTrees, + featureSubsetStrategy = "auto", seed = 123) + assert(rf.trees.size === 1) + val rfTree = rf.trees(0) + + val dt = DecisionTree.train(rdd, strategy) + + RandomForestSuite.validateRegressor(rf, arr, 0.01) + DecisionTreeSuite.validateRegressor(dt, arr, 0.01) + + // Make sure trees are the same. + assert(rfTree.toString == dt.toString) + } + + test("Binary classification with continuous features: subsampling features") { + val numFeatures = 50 + val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + + // Select feature subset for top nodes. Return true if OK. + def checkFeatureSubsetStrategy( + numTrees: Int, + featureSubsetStrategy: String, + numFeaturesPerNode: Int): Unit = { + val seeds = Array(123, 5354, 230, 349867, 23987) + val maxMemoryUsage: Long = 128 * 1024L * 1024L + val metadata = + DecisionTreeMetadata.buildMetadata(rdd, strategy, numTrees, featureSubsetStrategy) + seeds.foreach { seed => + val failString = s"Failed on test with:" + + s"numTrees=$numTrees, featureSubsetStrategy=$featureSubsetStrategy," + + s" numFeaturesPerNode=$numFeaturesPerNode, seed=$seed" + val nodeQueue = new mutable.Queue[(Int, Node)]() + val topNodes: Array[Node] = new Array[Node](numTrees) + Range(0, numTrees).foreach { treeIndex => + topNodes(treeIndex) = Node.emptyNode(nodeIndex = 1) + nodeQueue.enqueue((treeIndex, topNodes(treeIndex))) + } + val rng = new scala.util.Random(seed = seed) + val (nodesForGroup: Map[Int, Array[Node]], + treeToNodeToIndexInfo: Map[Int, Map[Int, RandomForest.NodeIndexInfo]]) = + RandomForest.selectNodesToSplit(nodeQueue, maxMemoryUsage, metadata, rng) + + assert(nodesForGroup.size === numTrees, failString) + assert(nodesForGroup.values.forall(_.size == 1), failString) // 1 node per tree + if (numFeaturesPerNode == numFeatures) { + // featureSubset values should all be None + assert(treeToNodeToIndexInfo.values.forall(_.values.forall(_.featureSubset.isEmpty)), + failString) + } else { + // Check number of features. + assert(treeToNodeToIndexInfo.values.forall(_.values.forall( + _.featureSubset.get.size === numFeaturesPerNode)), failString) + } + } + } + + checkFeatureSubsetStrategy(numTrees = 1, "auto", numFeatures) + checkFeatureSubsetStrategy(numTrees = 1, "all", numFeatures) + checkFeatureSubsetStrategy(numTrees = 1, "sqrt", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 1, "log2", + (math.log(numFeatures) / math.log(2)).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 1, "onethird", (numFeatures / 3.0).ceil.toInt) + + checkFeatureSubsetStrategy(numTrees = 2, "all", numFeatures) + checkFeatureSubsetStrategy(numTrees = 2, "auto", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "sqrt", math.sqrt(numFeatures).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "log2", + (math.log(numFeatures) / math.log(2)).ceil.toInt) + checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt) + } + +} + +object RandomForestSuite { + + /** + * Aggregates all values in data, and tests whether the empirical mean and stddev are within + * epsilon of the expected values. + * @param data Every element of the data should be an i.i.d. sample from some distribution. + */ + def testRandomArrays( + data: Array[Array[Double]], + numCols: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double) { + val values = new mutable.ArrayBuffer[Double]() + data.foreach { row => + assert(row.size == numCols) + values ++= row + } + val stats = new StatCounter(values) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + def validateClassifier( + model: RandomForestModel, + input: Seq[LabeledPoint], + requiredAccuracy: Double) { + val predictions = input.map(x => model.predict(x.features)) + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + prediction != expected.label + } + val accuracy = (input.length - numOffPredictions).toDouble / input.length + assert(accuracy >= requiredAccuracy, + s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") + } + + def validateRegressor( + model: RandomForestModel, + input: Seq[LabeledPoint], + requiredMSE: Double) { + val predictions = input.map(x => model.predict(x.features)) + val squaredError = predictions.zip(input).map { case (prediction, expected) => + val err = prediction - expected.label + err * err + }.sum + val mse = squaredError / input.length + assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") + } + + def generateOrderedLabeledPoints(numFeatures: Int): Array[LabeledPoint] = { + val numInstances = 1000 + val arr = new Array[LabeledPoint](numInstances) + for (i <- 0 until numInstances) { + val label = if (i < numInstances / 10) { + 0.0 + } else if (i < numInstances / 2) { + 1.0 + } else if (i < numInstances * 0.9) { + 0.0 + } else { + 1.0 + } + val features = Array.fill[Double](numFeatures)(i.toDouble) + arr(i) = new LabeledPoint(label, Vectors.dense(features)) + } + arr + } + +} From 1651cc117d73f0af6ec9f55b0c6c9b2bd565906c Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Sun, 28 Sep 2014 21:55:09 -0700 Subject: [PATCH 0823/1492] [EC2] Cleanup Python parens and disk dict Minor fixes: * Remove unnecessary parens (Python style) * Sort `disks_by_instance` dict and remove duplicate `t1.micro` key Author: Nicholas Chammas Closes #2571 from nchammas/ec2-polish and squashes the following commits: 9d203d5 [Nicholas Chammas] paren and dict cleanup --- ec2/spark_ec2.py | 60 ++++++++++++++++++++++++------------------------ 1 file changed, 30 insertions(+), 30 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 7f2cd7d94de39..5776d0b519309 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -508,7 +508,7 @@ def tag_instance(instance, name): break except: print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) - if (i == 5): + if i == 5: raise "Error - failed max attempts to add name tag" time.sleep(5) @@ -530,7 +530,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): for res in reservations: active = [i for i in res.instances if is_active(i)] for instance in active: - if (instance.tags.get(u'Name') is None): + if instance.tags.get(u'Name') is None: tag_instance(instance, name) # Now proceed to detect master and slaves instances. reservations = conn.get_all_instances() @@ -545,7 +545,7 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): elif name.startswith(cluster_name + "-slave"): slave_nodes.append(inst) if any((master_nodes, slave_nodes)): - print ("Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes))) + print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes)) if master_nodes != [] or not die_on_error: return (master_nodes, slave_nodes) else: @@ -626,43 +626,43 @@ def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes): def get_num_disks(instance_type): # From http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html # Updated 2014-6-20 + # For easy maintainability, please keep this manually-inputted dictionary sorted by key. disks_by_instance = { - "m1.small": 1, - "m1.medium": 1, - "m1.large": 2, - "m1.xlarge": 4, - "t1.micro": 1, "c1.medium": 1, "c1.xlarge": 4, - "m2.xlarge": 1, - "m2.2xlarge": 1, - "m2.4xlarge": 2, + "c3.2xlarge": 2, + "c3.4xlarge": 2, + "c3.8xlarge": 2, + "c3.large": 2, + "c3.xlarge": 2, "cc1.4xlarge": 2, "cc2.8xlarge": 4, "cg1.4xlarge": 2, - "hs1.8xlarge": 24, "cr1.8xlarge": 2, + "g2.2xlarge": 1, "hi1.4xlarge": 2, - "m3.medium": 1, - "m3.large": 1, - "m3.xlarge": 2, - "m3.2xlarge": 2, - "i2.xlarge": 1, + "hs1.8xlarge": 24, "i2.2xlarge": 2, "i2.4xlarge": 4, "i2.8xlarge": 8, - "c3.large": 2, - "c3.xlarge": 2, - "c3.2xlarge": 2, - "c3.4xlarge": 2, - "c3.8xlarge": 2, - "r3.large": 1, - "r3.xlarge": 1, + "i2.xlarge": 1, + "m1.large": 2, + "m1.medium": 1, + "m1.small": 1, + "m1.xlarge": 4, + "m2.2xlarge": 1, + "m2.4xlarge": 2, + "m2.xlarge": 1, + "m3.2xlarge": 2, + "m3.large": 1, + "m3.medium": 1, + "m3.xlarge": 2, "r3.2xlarge": 1, "r3.4xlarge": 1, "r3.8xlarge": 2, - "g2.2xlarge": 1, - "t1.micro": 0 + "r3.large": 1, + "r3.xlarge": 1, + "t1.micro": 0, } if instance_type in disks_by_instance: return disks_by_instance[instance_type] @@ -785,7 +785,7 @@ def ssh(host, opts, command): ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), stringify_command(command)]) except subprocess.CalledProcessError as e: - if (tries > 5): + if tries > 5: # If this was an ssh failure, provide the user with hints. if e.returncode == 255: raise UsageError( @@ -820,18 +820,18 @@ def ssh_read(host, opts, command): ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)]) -def ssh_write(host, opts, command, input): +def ssh_write(host, opts, command, arguments): tries = 0 while True: proc = subprocess.Popen( ssh_command(opts) + ['%s@%s' % (opts.user, host), stringify_command(command)], stdin=subprocess.PIPE) - proc.stdin.write(input) + proc.stdin.write(arguments) proc.stdin.close() status = proc.wait() if status == 0: break - elif (tries > 5): + elif tries > 5: raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: print >> stderr, \ From 657bdff41a27568a981b3e342ad380fe92aa08a0 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 29 Sep 2014 01:13:15 -0700 Subject: [PATCH 0824/1492] [CORE] Bugfix: LogErr format in DAGScheduler.scala Author: Zhang, Liye Closes #2572 from liyezhang556520/DAGLogErr and squashes the following commits: 5be2491 [Zhang, Liye] Bugfix: LogErr format in DAGScheduler.scala --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 70c235dffff70..5a96f52a10cd4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1209,7 +1209,7 @@ class DAGScheduler( .format(job.jobId, stageId)) } else if (jobsForStage.get.size == 1) { if (!stageIdToStage.contains(stageId)) { - logError("Missing Stage for stage with id $stageId") + logError(s"Missing Stage for stage with id $stageId") } else { // This is the only job that uses this stage, so fail the stage if it is running. val stage = stageIdToStage(stageId) From aedd251c54fd130fe6e2f28d7587d39136e7ad1c Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Mon, 29 Sep 2014 10:45:08 -0700 Subject: [PATCH 0825/1492] [EC2] Sort long, manually-inputted dictionaries Similar to the work done in #2571, this PR just sorts the remaining manually-inputted dicts in the EC2 script so they are easier to maintain. Author: Nicholas Chammas Closes #2578 from nchammas/ec2-dict-sort and squashes the following commits: f55c692 [Nicholas Chammas] sort long dictionaries --- ec2/spark_ec2.py | 69 ++++++++++++++++++++++++++---------------------- 1 file changed, 38 insertions(+), 31 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 5776d0b519309..941dfb988b9fb 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -217,8 +217,15 @@ def is_active(instance): # Return correct versions of Spark and Shark, given the supplied Spark version def get_spark_shark_version(opts): spark_shark_map = { - "0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1", - "1.0.0": "1.0.0", "1.0.1": "1.0.1", "1.0.2": "1.0.2", "1.1.0": "1.1.0" + "0.7.3": "0.7.1", + "0.8.0": "0.8.0", + "0.8.1": "0.8.1", + "0.9.0": "0.9.0", + "0.9.1": "0.9.1", + "1.0.0": "1.0.0", + "1.0.1": "1.0.1", + "1.0.2": "1.0.2", + "1.1.0": "1.1.0", } version = opts.spark_version.replace("v", "") if version not in spark_shark_map: @@ -227,49 +234,49 @@ def get_spark_shark_version(opts): return (version, spark_shark_map[version]) -# Attempt to resolve an appropriate AMI given the architecture and -# region of the request. -# Information regarding Amazon Linux AMI instance type was update on 2014-6-20: -# http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ +# Attempt to resolve an appropriate AMI given the architecture and region of the request. +# Source: http://aws.amazon.com/amazon-linux-ami/instance-type-matrix/ +# Last Updated: 2014-06-20 +# For easy maintainability, please keep this manually-inputted dictionary sorted by key. def get_spark_ami(opts): instance_types = { - "m1.small": "pvm", - "m1.medium": "pvm", - "m1.large": "pvm", - "m1.xlarge": "pvm", - "t1.micro": "pvm", "c1.medium": "pvm", "c1.xlarge": "pvm", - "m2.xlarge": "pvm", - "m2.2xlarge": "pvm", - "m2.4xlarge": "pvm", + "c3.2xlarge": "pvm", + "c3.4xlarge": "pvm", + "c3.8xlarge": "pvm", + "c3.large": "pvm", + "c3.xlarge": "pvm", "cc1.4xlarge": "hvm", "cc2.8xlarge": "hvm", "cg1.4xlarge": "hvm", - "hs1.8xlarge": "pvm", - "hi1.4xlarge": "pvm", - "m3.medium": "hvm", - "m3.large": "hvm", - "m3.xlarge": "hvm", - "m3.2xlarge": "hvm", "cr1.8xlarge": "hvm", - "i2.xlarge": "hvm", + "hi1.4xlarge": "pvm", + "hs1.8xlarge": "pvm", "i2.2xlarge": "hvm", "i2.4xlarge": "hvm", "i2.8xlarge": "hvm", - "c3.large": "pvm", - "c3.xlarge": "pvm", - "c3.2xlarge": "pvm", - "c3.4xlarge": "pvm", - "c3.8xlarge": "pvm", - "r3.large": "hvm", - "r3.xlarge": "hvm", + "i2.xlarge": "hvm", + "m1.large": "pvm", + "m1.medium": "pvm", + "m1.small": "pvm", + "m1.xlarge": "pvm", + "m2.2xlarge": "pvm", + "m2.4xlarge": "pvm", + "m2.xlarge": "pvm", + "m3.2xlarge": "hvm", + "m3.large": "hvm", + "m3.medium": "hvm", + "m3.xlarge": "hvm", "r3.2xlarge": "hvm", "r3.4xlarge": "hvm", "r3.8xlarge": "hvm", + "r3.large": "hvm", + "r3.xlarge": "hvm", + "t1.micro": "pvm", + "t2.medium": "hvm", "t2.micro": "hvm", "t2.small": "hvm", - "t2.medium": "hvm" } if opts.instance_type in instance_types: instance_type = instance_types[opts.instance_type] @@ -624,8 +631,8 @@ def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes): # Get number of local disks available for a given EC2 instance type. def get_num_disks(instance_type): - # From http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html - # Updated 2014-6-20 + # Source: http://docs.aws.amazon.com/AWSEC2/latest/UserGuide/InstanceStorage.html + # Last Updated: 2014-06-20 # For easy maintainability, please keep this manually-inputted dictionary sorted by key. disks_by_instance = { "c1.medium": 1, From 587a0cd7ed964ebfca2c97924c4f1e363f1fd3cb Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Mon, 29 Sep 2014 11:15:09 -0700 Subject: [PATCH 0826/1492] [MLlib] [SPARK-2885] DIMSUM: All-pairs similarity MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit # All-pairs similarity via DIMSUM Compute all pairs of similar vectors using brute force approach, and also DIMSUM sampling approach. Laying down some notation: we are looking for all pairs of similar columns in an m x n RowMatrix whose entries are denoted a_ij, with the i’th row denoted r_i and the j’th column denoted c_j. There is an oversampling parameter labeled ɣ that should be set to 4 log(n)/s to get provably correct results (with high probability), where s is the similarity threshold. The algorithm is stated with a Map and Reduce, with proofs of correctness and efficiency in published papers [1] [2]. The reducer is simply the summation reducer. The mapper is more interesting, and is also the heart of the scheme. As an exercise, you should try to see why in expectation, the map-reduce below outputs cosine similarities. ![dimsumv2](https://cloud.githubusercontent.com/assets/3220351/3807272/d1d9514e-1c62-11e4-9f12-3cfdb1d78b3a.png) [1] Bosagh-Zadeh, Reza and Carlsson, Gunnar (2013), Dimension Independent Matrix Square using MapReduce, arXiv:1304.1467 http://arxiv.org/abs/1304.1467 [2] Bosagh-Zadeh, Reza and Goel, Ashish (2012), Dimension Independent Similarity Computation, arXiv:1206.2082 http://arxiv.org/abs/1206.2082 # Testing Tests for all invocations included. Added L1 and L2 norm computation to MultivariateStatisticalSummary since it was needed. Added tests for both of them. Author: Reza Zadeh Author: Xiangrui Meng Closes #1778 from rezazadeh/dimsumv2 and squashes the following commits: 404c64c [Reza Zadeh] Merge remote-tracking branch 'upstream/master' into dimsumv2 4eb71c6 [Reza Zadeh] Add excludes for normL1 and normL2 ee8bd65 [Reza Zadeh] Merge remote-tracking branch 'upstream/master' into dimsumv2 976ddd4 [Reza Zadeh] Broadcast colMags. Avoid div by zero. 3467cff [Reza Zadeh] Merge remote-tracking branch 'upstream/master' into dimsumv2 aea0247 [Reza Zadeh] Allow large thresholds to promote sparsity 9fe17c0 [Xiangrui Meng] organize imports 2196ba5 [Xiangrui Meng] Merge branch 'rezazadeh-dimsumv2' into dimsumv2 254ca08 [Reza Zadeh] Merge remote-tracking branch 'upstream/master' into dimsumv2 f2947e4 [Xiangrui Meng] some optimization 3c4cf41 [Xiangrui Meng] Merge branch 'master' into rezazadeh-dimsumv2 0e4eda4 [Reza Zadeh] Use partition index for RNG 251bb9c [Reza Zadeh] Documentation 25e9d0d [Reza Zadeh] Line length for style fb296f6 [Reza Zadeh] renamed to normL1 and normL2 3764983 [Reza Zadeh] Documentation e9c6791 [Reza Zadeh] New interface and documentation 613f261 [Reza Zadeh] Column magnitude summary 75a0b51 [Reza Zadeh] Use Ints instead of Longs in the shuffle 0f12ade [Reza Zadeh] Style changes eb1dc20 [Reza Zadeh] Use Double.PositiveInfinity instead of Double.Max f56a882 [Reza Zadeh] Remove changes to MultivariateOnlineSummarizer dbc55ba [Reza Zadeh] Make colMagnitudes a method in RowMatrix 41e8ece [Reza Zadeh] style changes 139c8e1 [Reza Zadeh] Syntax changes 029aa9c [Reza Zadeh] javadoc and new test 75edb25 [Reza Zadeh] All tests passing! 05e59b8 [Reza Zadeh] Add test 502ce52 [Reza Zadeh] new interface 654c4fb [Reza Zadeh] default methods 3726ca9 [Reza Zadeh] Remove MatrixAlgebra 6bebabb [Reza Zadeh] remove changes to MatrixSuite 5b8cd7d [Reza Zadeh] Initial files --- .../mllib/linalg/distributed/RowMatrix.scala | 171 +++++++++++++++++- .../stat/MultivariateOnlineSummarizer.scala | 38 +++- .../stat/MultivariateStatisticalSummary.scala | 10 + .../linalg/distributed/RowMatrixSuite.scala | 37 ++++ project/MimaExcludes.scala | 9 +- 5 files changed, 259 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 4174f45d231c7..8380058cf9b41 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -19,17 +19,21 @@ package org.apache.spark.mllib.linalg.distributed import java.util.Arrays -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV} -import breeze.linalg.{svd => brzSvd, axpy => brzAxpy} +import scala.collection.mutable.ListBuffer + +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, SparseVector => BSV, axpy => brzAxpy, + svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ -import org.apache.spark.rdd.RDD -import org.apache.spark.Logging import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} +import org.apache.spark.rdd.RDD +import org.apache.spark.util.random.XORShiftRandom import org.apache.spark.storage.StorageLevel /** @@ -411,6 +415,165 @@ class RowMatrix( new RowMatrix(AB, nRows, B.numCols) } + /** + * Compute all cosine similarities between columns of this matrix using the brute-force + * approach of computing normalized dot products. + * + * @return An n x n sparse upper-triangular matrix of cosine similarities between + * columns of this matrix. + */ + def columnSimilarities(): CoordinateMatrix = { + columnSimilarities(0.0) + } + + /** + * Compute similarities between columns of this matrix using a sampling approach. + * + * The threshold parameter is a trade-off knob between estimate quality and computational cost. + * + * Setting a threshold of 0 guarantees deterministic correct results, but comes at exactly + * the same cost as the brute-force approach. Setting the threshold to positive values + * incurs strictly less computational cost than the brute-force approach, however the + * similarities computed will be estimates. + * + * The sampling guarantees relative-error correctness for those pairs of columns that have + * similarity greater than the given similarity threshold. + * + * To describe the guarantee, we set some notation: + * Let A be the smallest in magnitude non-zero element of this matrix. + * Let B be the largest in magnitude non-zero element of this matrix. + * Let L be the maximum number of non-zeros per row. + * + * For example, for {0,1} matrices: A=B=1. + * Another example, for the Netflix matrix: A=1, B=5 + * + * For those column pairs that are above the threshold, + * the computed similarity is correct to within 20% relative error with probability + * at least 1 - (0.981)^10/B^ + * + * The shuffle size is bounded by the *smaller* of the following two expressions: + * + * O(n log(n) L / (threshold * A)) + * O(m L^2^) + * + * The latter is the cost of the brute-force approach, so for non-zero thresholds, + * the cost is always cheaper than the brute-force approach. + * + * @param threshold Set to 0 for deterministic guaranteed correctness. + * Similarities above this threshold are estimated + * with the cost vs estimate quality trade-off described above. + * @return An n x n sparse upper-triangular matrix of cosine similarities + * between columns of this matrix. + */ + def columnSimilarities(threshold: Double): CoordinateMatrix = { + require(threshold >= 0, s"Threshold cannot be negative: $threshold") + + if (threshold > 1) { + logWarning(s"Threshold is greater than 1: $threshold " + + "Computation will be more efficient with promoted sparsity, " + + " however there is no correctness guarantee.") + } + + val gamma = if (threshold < 1e-6) { + Double.PositiveInfinity + } else { + 10 * math.log(numCols()) / threshold + } + + columnSimilaritiesDIMSUM(computeColumnSummaryStatistics().normL2.toArray, gamma) + } + + /** + * Find all similar columns using the DIMSUM sampling algorithm, described in two papers + * + * http://arxiv.org/abs/1206.2082 + * http://arxiv.org/abs/1304.1467 + * + * @param colMags A vector of column magnitudes + * @param gamma The oversampling parameter. For provable results, set to 10 * log(n) / s, + * where s is the smallest similarity score to be estimated, + * and n is the number of columns + * @return An n x n sparse upper-triangular matrix of cosine similarities + * between columns of this matrix. + */ + private[mllib] def columnSimilaritiesDIMSUM( + colMags: Array[Double], + gamma: Double): CoordinateMatrix = { + require(gamma > 1.0, s"Oversampling should be greater than 1: $gamma") + require(colMags.size == this.numCols(), "Number of magnitudes didn't match column dimension") + val sg = math.sqrt(gamma) // sqrt(gamma) used many times + + // Don't divide by zero for those columns with zero magnitude + val colMagsCorrected = colMags.map(x => if (x == 0) 1.0 else x) + + val sc = rows.context + val pBV = sc.broadcast(colMagsCorrected.map(c => sg / c)) + val qBV = sc.broadcast(colMagsCorrected.map(c => math.min(sg, c))) + + val sims = rows.mapPartitionsWithIndex { (indx, iter) => + val p = pBV.value + val q = qBV.value + + val rand = new XORShiftRandom(indx) + val scaled = new Array[Double](p.size) + iter.flatMap { row => + val buf = new ListBuffer[((Int, Int), Double)]() + row match { + case sv: SparseVector => + val nnz = sv.indices.size + var k = 0 + while (k < nnz) { + scaled(k) = sv.values(k) / q(sv.indices(k)) + k += 1 + } + k = 0 + while (k < nnz) { + val i = sv.indices(k) + val iVal = scaled(k) + if (iVal != 0 && rand.nextDouble() < p(i)) { + var l = k + 1 + while (l < nnz) { + val j = sv.indices(l) + val jVal = scaled(l) + if (jVal != 0 && rand.nextDouble() < p(j)) { + buf += (((i, j), iVal * jVal)) + } + l += 1 + } + } + k += 1 + } + case dv: DenseVector => + val n = dv.values.size + var i = 0 + while (i < n) { + scaled(i) = dv.values(i) / q(i) + i += 1 + } + i = 0 + while (i < n) { + val iVal = scaled(i) + if (iVal != 0 && rand.nextDouble() < p(i)) { + var j = i + 1 + while (j < n) { + val jVal = scaled(j) + if (jVal != 0 && rand.nextDouble() < p(j)) { + buf += (((i, j), iVal * jVal)) + } + j += 1 + } + } + i += 1 + } + } + buf + } + }.reduceByKey(_ + _).map { case ((i, j), sim) => + MatrixEntry(i.toLong, j.toLong, sim) + } + new CoordinateMatrix(sims, numCols(), numCols()) + } + private[mllib] override def toBreeze(): BDM[Double] = { val m = numRows().toInt val n = numCols().toInt diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 7d845c44365dd..3025d4837cab4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -42,6 +42,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S private var n = 0 private var currMean: BDV[Double] = _ private var currM2n: BDV[Double] = _ + private var currM2: BDV[Double] = _ + private var currL1: BDV[Double] = _ private var totalCnt: Long = 0 private var nnz: BDV[Double] = _ private var currMax: BDV[Double] = _ @@ -60,6 +62,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currMean = BDV.zeros[Double](n) currM2n = BDV.zeros[Double](n) + currM2 = BDV.zeros[Double](n) + currL1 = BDV.zeros[Double](n) nnz = BDV.zeros[Double](n) currMax = BDV.fill(n)(Double.MinValue) currMin = BDV.fill(n)(Double.MaxValue) @@ -81,6 +85,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val tmpPrevMean = currMean(i) currMean(i) = (currMean(i) * nnz(i) + value) / (nnz(i) + 1.0) currM2n(i) += (value - currMean(i)) * (value - tmpPrevMean) + currM2(i) += value * value + currL1(i) += math.abs(value) nnz(i) += 1.0 } @@ -97,7 +103,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S * @return This MultivariateOnlineSummarizer object. */ def merge(other: MultivariateOnlineSummarizer): this.type = { - if (this.totalCnt != 0 && other.totalCnt != 0) { + if (this.totalCnt != 0 && other.totalCnt != 0) { require(n == other.n, s"Dimensions mismatch when merging with another summarizer. " + s"Expecting $n but got ${other.n}.") totalCnt += other.totalCnt @@ -114,6 +120,15 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / (nnz(i) + other.nnz(i)) } + // merge m2 together + if (nnz(i) + other.nnz(i) != 0.0) { + currM2(i) += other.currM2(i) + } + // merge l1 together + if (nnz(i) + other.nnz(i) != 0.0) { + currL1(i) += other.currL1(i) + } + if (currMax(i) < other.currMax(i)) { currMax(i) = other.currMax(i) } @@ -127,6 +142,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S this.n = other.n this.currMean = other.currMean.copy this.currM2n = other.currM2n.copy + this.currM2 = other.currM2.copy + this.currL1 = other.currL1.copy this.totalCnt = other.totalCnt this.nnz = other.nnz.copy this.currMax = other.currMax.copy @@ -198,4 +215,23 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S } Vectors.fromBreeze(currMin) } + + override def normL2: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + + val realMagnitude = BDV.zeros[Double](n) + + var i = 0 + while (i < currM2.size) { + realMagnitude(i) = math.sqrt(currM2(i)) + i += 1 + } + + Vectors.fromBreeze(realMagnitude) + } + + override def normL1: Vector = { + require(totalCnt > 0, s"Nothing has been added to this summarizer.") + Vectors.fromBreeze(currL1) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala index f9eb343da2b82..6a364c93284af 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -53,4 +53,14 @@ trait MultivariateStatisticalSummary { * Minimum value of each column. */ def min: Vector + + /** + * Euclidean magnitude of each column + */ + def normL2: Vector + + /** + * L1 norm of each column + */ + def normL1: Vector } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 1d3a3221365cc..63f3ed58c0d4d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -95,6 +95,40 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { } } + test("similar columns") { + val colMags = Vectors.dense(Math.sqrt(126), Math.sqrt(66), Math.sqrt(94)) + val expected = BDM( + (0.0, 54.0, 72.0), + (0.0, 0.0, 78.0), + (0.0, 0.0, 0.0)) + + for (i <- 0 until n; j <- 0 until n) { + expected(i, j) /= (colMags(i) * colMags(j)) + } + + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.columnSimilarities(0.11).toBreeze() + for (i <- 0 until n; j <- 0 until n) { + if (expected(i, j) > 0) { + val actual = expected(i, j) + val estimate = G(i, j) + assert(math.abs(actual - estimate) / actual < 0.2, + s"Similarities not close enough: $actual vs $estimate") + } + } + } + + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.columnSimilarities() + assert(closeToZero(G.toBreeze() - expected)) + } + + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.columnSimilaritiesDIMSUM(colMags.toArray, 150.0) + assert(closeToZero(G.toBreeze() - expected)) + } + } + test("svd of a full-rank matrix") { for (mat <- Seq(denseMat, sparseMat)) { for (mode <- Seq("auto", "local-svd", "local-eigs", "dist-eigs")) { @@ -190,6 +224,9 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { assert(summary.numNonzeros === Vectors.dense(3.0, 3.0, 4.0), "nnz mismatch") assert(summary.max === Vectors.dense(9.0, 7.0, 8.0), "max mismatch") assert(summary.min === Vectors.dense(0.0, 0.0, 1.0), "column mismatch.") + assert(summary.normL2 === Vectors.dense(Math.sqrt(126), Math.sqrt(66), Math.sqrt(94)), + "magnitude mismatch.") + assert(summary.normL1 === Vectors.dense(18.0, 12.0, 16.0), "L1 norm mismatch") } } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 3280e662fa0b1..1adfaa18c6202 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -39,7 +39,14 @@ object MimaExcludes { MimaBuild.excludeSparkPackage("graphx") ) ++ MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ - MimaBuild.excludeSparkClass("mllib.linalg.Vector") + MimaBuild.excludeSparkClass("mllib.linalg.Vector") ++ + Seq( + // Added normL1 and normL2 to trait MultivariateStatisticalSummary + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL1"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL2") + ) case v if v.startsWith("1.1") => Seq( From dab1b0ae29a6d3017bdca23464f22a51d51eaae1 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 29 Sep 2014 11:25:32 -0700 Subject: [PATCH 0827/1492] [SPARK-3032][Shuffle] Fix key comparison integer overflow introduced sorting exception Previous key comparison in `ExternalSorter` will get wrong sorting result or exception when key comparison overflows, details can be seen in [SPARK-3032](https://issues.apache.org/jira/browse/SPARK-3032). Here fix this and add a unit test to prove it. Author: jerryshao Closes #2514 from jerryshao/SPARK-3032 and squashes the following commits: 6f3c302 [jerryshao] Improve the unit test according to comments 01911e6 [jerryshao] Change the test to show the contract violate exception 83acb38 [jerryshao] Minor changes according to comments fa2a08f [jerryshao] Fix key comparison integer overflow introduced sorting exception --- .../util/collection/ExternalSorter.scala | 2 +- .../util/collection/ExternalSorterSuite.scala | 55 +++++++++++++++++++ 2 files changed, 56 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 0a152cb97ad9e..644fa36818647 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -144,7 +144,7 @@ private[spark] class ExternalSorter[K, V, C]( override def compare(a: K, b: K): Int = { val h1 = if (a == null) 0 else a.hashCode() val h2 = if (b == null) 0 else b.hashCode() - h1 - h2 + if (h1 < h2) -1 else if (h1 == h2) 0 else 1 } }) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 706faed980f31..f26e40fbd4b36 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.{PrivateMethodTester, FunSuite} import org.apache.spark._ import org.apache.spark.SparkContext._ +import scala.util.Random + class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMethodTester { private def createSparkConf(loadDefaults: Boolean): SparkConf = { val conf = new SparkConf(loadDefaults) @@ -707,4 +709,57 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe Some(agg), Some(new HashPartitioner(FEW_PARTITIONS)), None, None) assertDidNotBypassMergeSort(sorter4) } + + test("sort without breaking sorting contracts") { + val conf = createSparkConf(true) + conf.set("spark.shuffle.memoryFraction", "0.01") + conf.set("spark.shuffle.manager", "sort") + sc = new SparkContext("local-cluster[1,1,512]", "test", conf) + + // Using wrongOrdering to show integer overflow introduced exception. + val rand = new Random(100L) + val wrongOrdering = new Ordering[String] { + override def compare(a: String, b: String) = { + val h1 = if (a == null) 0 else a.hashCode() + val h2 = if (b == null) 0 else b.hashCode() + h1 - h2 + } + } + + val testData = Array.tabulate(100000) { _ => rand.nextInt().toString } + + val sorter1 = new ExternalSorter[String, String, String]( + None, None, Some(wrongOrdering), None) + val thrown = intercept[IllegalArgumentException] { + sorter1.insertAll(testData.iterator.map(i => (i, i))) + sorter1.iterator + } + + assert(thrown.getClass() === classOf[IllegalArgumentException]) + assert(thrown.getMessage().contains("Comparison method violates its general contract")) + sorter1.stop() + + // Using aggregation and external spill to make sure ExternalSorter using + // partitionKeyComparator. + def createCombiner(i: String) = ArrayBuffer(i) + def mergeValue(c: ArrayBuffer[String], i: String) = c += i + def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]) = c1 ++= c2 + + val agg = new Aggregator[String, String, ArrayBuffer[String]]( + createCombiner, mergeValue, mergeCombiners) + + val sorter2 = new ExternalSorter[String, String, ArrayBuffer[String]]( + Some(agg), None, None, None) + sorter2.insertAll(testData.iterator.map(i => (i, i))) + + // To validate the hash ordering of key + var minKey = Int.MinValue + sorter2.iterator.foreach { case (k, v) => + val h = k.hashCode() + assert(h >= minKey) + minKey = h + } + + sorter2.stop() + } } From e43c72fe04d4fbf2a108b456d533e641b71b0a2a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Sep 2014 12:38:24 -0700 Subject: [PATCH 0828/1492] Add more debug message for ManagedBuffer This is to help debug the error reported at http://apache-spark-user-list.1001560.n3.nabble.com/SQL-queries-fail-in-1-2-0-SNAPSHOT-td15327.html Author: Reynold Xin Closes #2580 from rxin/buffer-debug and squashes the following commits: 5814292 [Reynold Xin] Logging close() in case close() fails. 323dfec [Reynold Xin] Add more debug message. --- .../apache/spark/network/ManagedBuffer.scala | 43 ++++++++++++++++--- .../scala/org/apache/spark/util/Utils.scala | 14 ++++++ 2 files changed, 51 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala index e990c1da6730f..a4409181ec907 100644 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -17,15 +17,17 @@ package org.apache.spark.network -import java.io.{FileInputStream, RandomAccessFile, File, InputStream} +import java.io._ import java.nio.ByteBuffer import java.nio.channels.FileChannel import java.nio.channels.FileChannel.MapMode +import scala.util.Try + import com.google.common.io.ByteStreams import io.netty.buffer.{ByteBufInputStream, ByteBuf} -import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.{ByteBufferInputStream, Utils} /** @@ -71,18 +73,47 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt try { channel = new RandomAccessFile(file, "r").getChannel channel.map(MapMode.READ_ONLY, offset, length) + } catch { + case e: IOException => + Try(channel.size).toOption match { + case Some(fileLen) => + throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) + case None => + throw new IOException(s"Error in opening $this", e) + } } finally { if (channel != null) { - channel.close() + Utils.tryLog(channel.close()) } } } override def inputStream(): InputStream = { - val is = new FileInputStream(file) - is.skip(offset) - ByteStreams.limit(is, length) + var is: FileInputStream = null + try { + is = new FileInputStream(file) + is.skip(offset) + ByteStreams.limit(is, length) + } catch { + case e: IOException => + if (is != null) { + Utils.tryLog(is.close()) + } + Try(file.length).toOption match { + case Some(fileLen) => + throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) + case None => + throw new IOException(s"Error in opening $this", e) + } + case e: Throwable => + if (is != null) { + Utils.tryLog(is.close()) + } + throw e + } } + + override def toString: String = s"${getClass.getName}($file, $offset, $length)" } 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 2755887feeeff..10d440828e323 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1304,6 +1304,20 @@ private[spark] object Utils extends Logging { } } + /** Executes the given block in a Try, logging any uncaught exceptions. */ + def tryLog[T](f: => T): Try[T] = { + try { + val res = f + scala.util.Success(res) + } catch { + case ct: ControlThrowable => + throw ct + case t: Throwable => + logError(s"Uncaught exception in thread ${Thread.currentThread().getName}", t) + scala.util.Failure(t) + } + } + /** Returns true if the given exception was fatal. See docs for scala.util.control.NonFatal. */ def isFatalError(e: Throwable): Boolean = { e match { From 0bbe7faeffa17577ae8a33dfcd8c4c783db5c909 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Mon, 29 Sep 2014 15:51:55 -0700 Subject: [PATCH 0829/1492] [SPARK-3007][SQL]Add Dynamic Partition support to Spark Sql hive MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit a new PR base on new master. changes are the same as https://github.com/apache/spark/pull/1919 Author: baishuo(白硕) Author: baishuo Author: Cheng Lian Closes #2226 from baishuo/patch-3007 and squashes the following commits: e69ce88 [Cheng Lian] Adds tests to verify dynamic partitioning folder layout b20a3dc [Cheng Lian] Addresses @yhuai's comments 096bbbc [baishuo(白硕)] Merge pull request #1 from liancheng/refactor-dp 1093c20 [Cheng Lian] Adds more tests 5004542 [Cheng Lian] Minor refactoring fae9eff [Cheng Lian] Refactors InsertIntoHiveTable to a Command 528e84c [Cheng Lian] Fixes typo in test name, regenerated golden answer files c464b26 [Cheng Lian] Refactors dynamic partitioning support 5033928 [baishuo] pass check style 2201c75 [baishuo] use HiveConf.DEFAULTPARTITIONNAME to replace hive.exec.default.partition.name b47c9bf [baishuo] modify according micheal's advice c3ab36d [baishuo] modify for some bad indentation 7ce2d9f [baishuo] modify code to pass scala style checks 37c1c43 [baishuo] delete a empty else branch 66e33fc [baishuo] do a little modify 88d0110 [baishuo] update file after test a3961d9 [baishuo(白硕)] Update Cast.scala f7467d0 [baishuo(白硕)] Update InsertIntoHiveTable.scala c1a59dd [baishuo(白硕)] Update Cast.scala 0e18496 [baishuo(白硕)] Update HiveQuerySuite.scala 60f70aa [baishuo(白硕)] Update InsertIntoHiveTable.scala 0a50db9 [baishuo(白硕)] Update HiveCompatibilitySuite.scala 491c7d0 [baishuo(白硕)] Update InsertIntoHiveTable.scala a2374a8 [baishuo(白硕)] Update InsertIntoHiveTable.scala 701a814 [baishuo(白硕)] Update SparkHadoopWriter.scala dc24c41 [baishuo(白硕)] Update HiveQl.scala --- .../execution/HiveCompatibilitySuite.scala | 17 ++ .../org/apache/spark/SparkHadoopWriter.scala | 195 ---------------- .../org/apache/spark/sql/hive/HiveQl.scala | 5 - .../hive/execution/InsertIntoHiveTable.scala | 207 +++++++++-------- .../spark/sql/hive/hiveWriterContainers.scala | 217 ++++++++++++++++++ ...rtition-0-be33aaa7253c8f248ff3921cd7dae340 | 0 ...rtition-1-640552dd462707563fd255a713f83b41 | 0 ...rtition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 | 1 + ...rtition-3-b7f7fa7ebf666f4fee27e149d8c6961f | 0 ...rtition-4-8bdb71ad8cb3cc3026043def2525de3a | 0 ...rtition-5-c630dce438f3792e7fb0f523fbbb3e1e | 0 ...rtition-6-7abc9ec8a36cdc5e89e955265a7fd7cf | 0 ...rtition-7-be33aaa7253c8f248ff3921cd7dae340 | 0 .../sql/hive/execution/HiveQuerySuite.scala | 100 +++++++- 14 files changed, 443 insertions(+), 299 deletions(-) delete mode 100644 sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 556c984ad392b..35e9c9939d4b7 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -220,6 +220,23 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", + "dynamic_partition_skip_default", + "infer_bucket_sort_dyn_part", + "load_dyn_part1", + "load_dyn_part2", + "load_dyn_part3", + "load_dyn_part4", + "load_dyn_part5", + "load_dyn_part6", + "load_dyn_part7", + "load_dyn_part8", + "load_dyn_part9", + "load_dyn_part10", + "load_dyn_part11", + "load_dyn_part12", + "load_dyn_part13", + "load_dyn_part14", + "load_dyn_part14_win", "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala deleted file mode 100644 index ab7862f4f9e06..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.io.IOException -import java.text.NumberFormat -import java.util.Date - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} -import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.FileSinkDesc -import org.apache.hadoop.mapred._ -import org.apache.hadoop.io.Writable - -import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} - -/** - * Internal helper class that saves an RDD using a Hive OutputFormat. - * It is based on [[SparkHadoopWriter]]. - */ -private[hive] class SparkHiveHadoopWriter( - @transient jobConf: JobConf, - fileSinkConf: FileSinkDesc) - extends Logging - with SparkHadoopMapRedUtil - with Serializable { - - private val now = new Date() - private val conf = new SerializableWritable(jobConf) - - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null - - @transient private var writer: FileSinkOperator.RecordWriter = null - @transient private var format: HiveOutputFormat[AnyRef, Writable] = null - @transient private var committer: OutputCommitter = null - @transient private var jobContext: JobContext = null - @transient private var taskContext: TaskAttemptContext = null - - def preSetup() { - setIDs(0, 0, 0) - setConfParams() - - val jCtxt = getJobContext() - getOutputCommitter().setupJob(jCtxt) - } - - - def setup(jobid: Int, splitid: Int, attemptid: Int) { - setIDs(jobid, splitid, attemptid) - setConfParams() - } - - def open() { - val numfmt = NumberFormat.getInstance() - numfmt.setMinimumIntegerDigits(5) - numfmt.setGroupingUsed(false) - - val extension = Utilities.getFileExtension( - conf.value, - fileSinkConf.getCompressed, - getOutputFormat()) - - val outputName = "part-" + numfmt.format(splitID) + extension - val path = FileOutputFormat.getTaskOutputPath(conf.value, outputName) - - getOutputCommitter().setupTask(getTaskContext()) - writer = HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - fileSinkConf, - path, - null) - } - - def write(value: Writable) { - if (writer != null) { - writer.write(value) - } else { - throw new IOException("Writer is null, open() has not been called") - } - } - - def close() { - // Seems the boolean value passed into close does not matter. - writer.close(false) - } - - def commit() { - val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() - if (cmtr.needsTaskCommit(taCtxt)) { - try { - cmtr.commitTask(taCtxt) - logInfo (taID + ": Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - cmtr.abortTask(taCtxt) - throw e - } - } else { - logWarning ("No need to commit output of task: " + taID.value) - } - } - - def commitJob() { - // always ? Or if cmtr.needsTaskCommit ? - val cmtr = getOutputCommitter() - cmtr.commitJob(getJobContext()) - } - - // ********* Private Functions ********* - - private def getOutputFormat(): HiveOutputFormat[AnyRef,Writable] = { - if (format == null) { - format = conf.value.getOutputFormat() - .asInstanceOf[HiveOutputFormat[AnyRef,Writable]] - } - format - } - - private def getOutputCommitter(): OutputCommitter = { - if (committer == null) { - committer = conf.value.getOutputCommitter - } - committer - } - - private def getJobContext(): JobContext = { - if (jobContext == null) { - jobContext = newJobContext(conf.value, jID.value) - } - jobContext - } - - private def getTaskContext(): TaskAttemptContext = { - if (taskContext == null) { - taskContext = newTaskAttemptContext(conf.value, taID.value) - } - taskContext - } - - private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { - jobID = jobId - splitID = splitId - attemptID = attemptId - - jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) - } - - private def setConfParams() { - conf.value.set("mapred.job.id", jID.value.toString) - conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) - conf.value.set("mapred.task.id", taID.value.toString) - conf.value.setBoolean("mapred.task.is.map", true) - conf.value.setInt("mapred.task.partition", splitID) - } -} - -private[hive] object SparkHiveHadoopWriter { - def createPathFromString(path: String, conf: JobConf): Path = { - if (path == null) { - throw new IllegalArgumentException("Output path is null") - } - val outputPath = new Path(path) - val fs = outputPath.getFileSystem(conf) - if (outputPath == null || fs == null) { - throw new IllegalArgumentException("Incorrectly formatted output path") - } - outputPath.makeQualified(fs) - } -} 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 0aa6292c0184e..4e30e6e06fe21 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 @@ -837,11 +837,6 @@ private[hive] object HiveQl { cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) - if (partitionKeys.values.exists(p => p.isEmpty)) { - throw new NotImplementedError(s"Do not support INSERT INTO/OVERWRITE with" + - s"dynamic partitioning.") - } - InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query, overwrite) case a: ASTNode => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index a284a91a91e31..3d2ee010696f6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -19,27 +19,25 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConversions._ -import java.util.{HashMap => JHashMap} - import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils -import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector -import org.apache.hadoop.io.Writable +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} -import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} -import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, SparkHiveHadoopWriter} +import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} +import org.apache.spark.sql.hive._ +import org.apache.spark.{SerializableWritable, SparkException, TaskContext} /** * :: DeveloperApi :: @@ -51,7 +49,7 @@ case class InsertIntoHiveTable( child: SparkPlan, overwrite: Boolean) (@transient sc: HiveContext) - extends UnaryNode { + extends UnaryNode with Command { @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) @@ -101,66 +99,74 @@ case class InsertIntoHiveTable( } def saveAsHiveFile( - rdd: RDD[Writable], + rdd: RDD[Row], valueClass: Class[_], fileSinkConf: FileSinkDesc, - conf: JobConf, - isCompressed: Boolean) { - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - conf.setOutputValueClass(valueClass) - if (fileSinkConf.getTableInfo.getOutputFileFormatClassName == null) { - throw new SparkException("Output format class not set") - } - // Doesn't work in Scala 2.9 due to what may be a generics bug - // TODO: Should we uncomment this for Scala 2.10? - // conf.setOutputFormat(outputFormatClass) - conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) + conf: SerializableWritable[JobConf], + writerContainer: SparkHiveWriterContainer) { + assert(valueClass != null, "Output value class not set") + conf.value.setOutputValueClass(valueClass) + + val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName + assert(outputFileFormatClassName != null, "Output format class not set") + conf.value.set("mapred.output.format.class", outputFileFormatClassName) + + val isCompressed = conf.value.getBoolean( + ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) + if (isCompressed) { // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", // and "mapred.output.compression.type" have no impact on ORC because it uses table properties // to store compression information. - conf.set("mapred.output.compress", "true") + conf.value.set("mapred.output.compress", "true") fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) + fileSinkConf.setCompressCodec(conf.value.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(conf.value.get("mapred.output.compression.type")) } - conf.setOutputCommitter(classOf[FileOutputCommitter]) - FileOutputFormat.setOutputPath( - conf, - SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) + conf.value.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath( + conf.value, + SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName, conf.value)) log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) - writer.preSetup() + writerContainer.driverSideSetup() + sc.sparkContext.runJob(rdd, writeToFile _) + writerContainer.commitJob() + + // Note that this function is executed on executor side + def writeToFile(context: TaskContext, iterator: Iterator[Row]) { + val serializer = newSerializer(fileSinkConf.getTableInfo) + val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val outputData = new Array[Any](fieldOIs.length) - def writeToFile(context: TaskContext, iter: Iterator[Writable]) { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt + writerContainer.executorSideSetup(context.stageId, context.partitionId, attemptNumber) - writer.setup(context.stageId, context.partitionId, attemptNumber) - writer.open() + iterator.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + // TODO (lian) avoid per row dynamic dispatching and pattern matching cost in `wrap` + outputData(i) = wrap(row(i), fieldOIs(i)) + i += 1 + } - var count = 0 - while(iter.hasNext) { - val record = iter.next() - count += 1 - writer.write(record) + val writer = writerContainer.getLocalFileWriter(row) + writer.write(serializer.serialize(outputData, standardOI)) } - writer.close() - writer.commit() + writerContainer.close() } - - sc.sparkContext.runJob(rdd, writeToFile _) - writer.commitJob() } - override def execute() = result - /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the @@ -168,50 +174,57 @@ case class InsertIntoHiveTable( * * Note: this is run once and then kept to avoid double insertions. */ - private lazy val result: RDD[Row] = { - val childRdd = child.execute() - assert(childRdd != null) - + override protected[sql] lazy val sideEffectResult: Seq[Row] = { // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc val tableLocation = table.hiveQlTable.getDataLocation val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val rdd = childRdd.mapPartitions { iter => - val serializer = newSerializer(fileSinkConf.getTableInfo) - val standardOI = ObjectInspectorUtils - .getStandardObjectInspector( - fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] + val numDynamicPartitions = partition.values.count(_.isEmpty) + val numStaticPartitions = partition.values.count(_.nonEmpty) + val partitionSpec = partition.map { + case (key, Some(value)) => key -> value + case (key, None) => key -> "" + } - val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray - val outputData = new Array[Any](fieldOIs.length) - iter.map { row => - var i = 0 - while (i < row.length) { - // Casts Strings to HiveVarchars when necessary. - outputData(i) = wrap(row(i), fieldOIs(i)) - i += 1 - } + // All partition column names in the format of "//..." + val partitionColumns = fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns") + val partitionColumnNames = Option(partitionColumns).map(_.split("/")).orNull - serializer.serialize(outputData, standardOI) + // Validate partition spec if there exist any dynamic partitions + if (numDynamicPartitions > 0) { + // Report error if dynamic partitioning is not enabled + if (!sc.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { + throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg) + } + + // Report error if dynamic partition strict mode is on but no static partition is found + if (numStaticPartitions == 0 && + sc.hiveconf.getVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { + throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg) + } + + // Report error if any static partition appears after a dynamic partition + val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) + isDynamic.init.zip(isDynamic.tail).find(_ == (true, false)).foreach { _ => + throw new SparkException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } - // ORC stores compression information in table properties. While, there are other formats - // (e.g. RCFile) that rely on hadoop configurations to store compression information. val jobConf = new JobConf(sc.hiveconf) - saveAsHiveFile( - rdd, - outputClass, - fileSinkConf, - jobConf, - sc.hiveconf.getBoolean("hive.exec.compress.output", false)) - - // TODO: Handle dynamic partitioning. + val jobConfSer = new SerializableWritable(jobConf) + + val writerContainer = if (numDynamicPartitions > 0) { + val dynamicPartColNames = partitionColumnNames.takeRight(numDynamicPartitions) + new SparkHiveDynamicPartitionWriterContainer(jobConf, fileSinkConf, dynamicPartColNames) + } else { + new SparkHiveWriterContainer(jobConf, fileSinkConf) + } + + saveAsHiveFile(child.execute(), outputClass, fileSinkConf, jobConfSer, writerContainer) + val outputPath = FileOutputFormat.getOutputPath(jobConf) // Have to construct the format of dbname.tablename. val qualifiedTableName = s"${table.databaseName}.${table.tableName}" @@ -220,10 +233,6 @@ case class InsertIntoHiveTable( // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. val holdDDLTime = false if (partition.nonEmpty) { - val partitionSpec = partition.map { - case (key, Some(value)) => key -> value - case (key, None) => key -> "" // Should not reach here right now. - } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) db.validatePartitionNameCharacters(partVals) // inheritTableSpecs is set to true. It should be set to false for a IMPORT query @@ -231,14 +240,26 @@ case class InsertIntoHiveTable( val inheritTableSpecs = true // TODO: Correctly set isSkewedStoreAsSubdir. val isSkewedStoreAsSubdir = false - db.loadPartition( - outputPath, - qualifiedTableName, - partitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + if (numDynamicPartitions > 0) { + db.loadDynamicPartitions( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + numDynamicPartitions, + holdDDLTime, + isSkewedStoreAsSubdir + ) + } else { + db.loadPartition( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } } else { db.loadTable( outputPath, @@ -251,6 +272,6 @@ case class InsertIntoHiveTable( // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. // TODO: implement hive compatibility as rules. - sc.sparkContext.makeRDD(Nil, 1) + Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala new file mode 100644 index 0000000000000..a667188fa53bd --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.IOException +import java.text.NumberFormat +import java.util.Date + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred._ + +import org.apache.spark.sql.Row +import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} + +/** + * Internal helper class that saves an RDD using a Hive OutputFormat. + * It is based on [[SparkHadoopWriter]]. + */ +private[hive] class SparkHiveWriterContainer( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc) + extends Logging + with SparkHadoopMapRedUtil + with Serializable { + + private val now = new Date() + protected val conf = new SerializableWritable(jobConf) + + private var jobID = 0 + private var splitID = 0 + private var attemptID = 0 + private var jID: SerializableWritable[JobID] = null + private var taID: SerializableWritable[TaskAttemptID] = null + + @transient private var writer: FileSinkOperator.RecordWriter = null + @transient private lazy val committer = conf.value.getOutputCommitter + @transient private lazy val jobContext = newJobContext(conf.value, jID.value) + @transient private lazy val taskContext = newTaskAttemptContext(conf.value, taID.value) + @transient private lazy val outputFormat = + conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef,Writable]] + + def driverSideSetup() { + setIDs(0, 0, 0) + setConfParams() + committer.setupJob(jobContext) + } + + def executorSideSetup(jobId: Int, splitId: Int, attemptId: Int) { + setIDs(jobId, splitId, attemptId) + setConfParams() + committer.setupTask(taskContext) + initWriters() + } + + protected def getOutputName: String = { + val numberFormat = NumberFormat.getInstance() + numberFormat.setMinimumIntegerDigits(5) + numberFormat.setGroupingUsed(false) + val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat) + "part-" + numberFormat.format(splitID) + extension + } + + def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = writer + + def close() { + // Seems the boolean value passed into close does not matter. + writer.close(false) + commit() + } + + def commitJob() { + committer.commitJob(jobContext) + } + + protected def initWriters() { + // NOTE this method is executed at the executor side. + // For Hive tables without partitions or with only static partitions, only 1 writer is needed. + writer = HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + fileSinkConf, + FileOutputFormat.getTaskOutputPath(conf.value, getOutputName), + Reporter.NULL) + } + + protected def commit() { + if (committer.needsTaskCommit(taskContext)) { + try { + committer.commitTask(taskContext) + logInfo (taID + ": Committed") + } catch { + case e: IOException => + logError("Error committing the output of task: " + taID.value, e) + committer.abortTask(taskContext) + throw e + } + } else { + logInfo("No need to commit output of task: " + taID.value) + } + } + + // ********* Private Functions ********* + + private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { + jobID = jobId + splitID = splitId + attemptID = attemptId + + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) + taID = new SerializableWritable[TaskAttemptID]( + new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) + } + + private def setConfParams() { + conf.value.set("mapred.job.id", jID.value.toString) + conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) + conf.value.set("mapred.task.id", taID.value.toString) + conf.value.setBoolean("mapred.task.is.map", true) + conf.value.setInt("mapred.task.partition", splitID) + } +} + +private[hive] object SparkHiveWriterContainer { + def createPathFromString(path: String, conf: JobConf): Path = { + if (path == null) { + throw new IllegalArgumentException("Output path is null") + } + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(conf) + if (outputPath == null || fs == null) { + throw new IllegalArgumentException("Incorrectly formatted output path") + } + outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + } +} + +private[spark] class SparkHiveDynamicPartitionWriterContainer( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc, + dynamicPartColNames: Array[String]) + extends SparkHiveWriterContainer(jobConf, fileSinkConf) { + + private val defaultPartName = jobConf.get( + ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultVal) + + @transient private var writers: mutable.HashMap[String, FileSinkOperator.RecordWriter] = _ + + override protected def initWriters(): Unit = { + // NOTE: This method is executed at the executor side. + // Actual writers are created for each dynamic partition on the fly. + writers = mutable.HashMap.empty[String, FileSinkOperator.RecordWriter] + } + + override def close(): Unit = { + writers.values.foreach(_.close(false)) + commit() + } + + override def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = { + val dynamicPartPath = dynamicPartColNames + .zip(row.takeRight(dynamicPartColNames.length)) + .map { case (col, rawVal) => + val string = String.valueOf(rawVal) + s"/$col=${if (rawVal == null || string.isEmpty) defaultPartName else string}" + } + .mkString + + def newWriter = { + val newFileSinkDesc = new FileSinkDesc( + fileSinkConf.getDirName + dynamicPartPath, + fileSinkConf.getTableInfo, + fileSinkConf.getCompressed) + newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec) + newFileSinkDesc.setCompressType(fileSinkConf.getCompressType) + + val path = { + val outputPath = FileOutputFormat.getOutputPath(conf.value) + assert(outputPath != null, "Undefined job output-path") + val workPath = new Path(outputPath, dynamicPartPath.stripPrefix("/")) + new Path(workPath, getOutputName) + } + + HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + newFileSinkDesc, + path, + Reporter.NULL) + } + + writers.getOrElseUpdate(dynamicPartPath, newWriter) + } +} diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 b/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 b/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f b/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a b/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e b/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf b/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2da8a6fac3d99..5d743a51b47c5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.hive.execution import scala.util.Try +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -380,7 +383,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.exists(_ == "== Physical Plan ==") + explanation.contains("== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { @@ -568,6 +571,91 @@ class HiveQuerySuite extends HiveComparisonTest { case class LogEntry(filename: String, message: String) case class LogFile(name: String) + createQueryTest("dynamic_partition", + """ + |DROP TABLE IF EXISTS dynamic_part_table; + |CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT); + | + |SET hive.exec.dynamic.partition.mode=nonstrict; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, 1, 1 FROM src WHERE key=150; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, NULL, 1 FROM src WHERE key=150; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, 1, NULL FROM src WHERE key=150; + | + |INSERT INTO TABLe dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, NULL, NULL FROM src WHERE key=150; + | + |DROP TABLE IF EXISTS dynamic_part_table; + """.stripMargin) + + test("Dynamic partition folder layout") { + sql("DROP TABLE IF EXISTS dynamic_part_table") + sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + val data = Map( + Seq("1", "1") -> 1, + Seq("1", "NULL") -> 2, + Seq("NULL", "1") -> 3, + Seq("NULL", "NULL") -> 4) + + data.foreach { case (parts, value) => + sql( + s"""INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT $value, ${parts.mkString(", ")} FROM src WHERE key=150 + """.stripMargin) + + val partFolder = Seq("partcol1", "partcol2") + .zip(parts) + .map { case (k, v) => + if (v == "NULL") { + s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultVal}" + } else { + s"$k=$v" + } + } + .mkString("/") + + // Loads partition data to a temporary table to verify contents + val path = s"$warehousePath/dynamic_part_table/$partFolder/part-00000" + + sql("DROP TABLE IF EXISTS dp_verify") + sql("CREATE TABLE dp_verify(intcol INT)") + sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") + + assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) + } + } + + test("Partition spec validation") { + sql("DROP TABLE IF EXISTS dp_test") + sql("CREATE TABLE dp_test(key INT, value STRING) PARTITIONED BY (dp INT, sp INT)") + sql("SET hive.exec.dynamic.partition.mode=strict") + + // Should throw when using strict dynamic partition mode without any static partition + intercept[SparkException] { + sql( + """INSERT INTO TABLE dp_test PARTITION(dp) + |SELECT key, value, key % 5 FROM src + """.stripMargin) + } + + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + // Should throw when a static partition appears after a dynamic partition + intercept[SparkException] { + sql( + """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1) + |SELECT key, value, key % 5 FROM src + """.stripMargin) + } + } + test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") @@ -625,27 +713,27 @@ class HiveQuerySuite extends HiveComparisonTest { assert(sql("SET").collect().size == 0) assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey=$testVal")) + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) assertResult(Set(testKey -> testVal)) { - collectResults(hql("SET")) + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(hql("SET")) + collectResults(sql("SET")) } // "set key" assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey")) + collectResults(sql(s"SET $testKey")) } assertResult(Set(nonexistentKey -> "")) { - collectResults(hql(s"SET $nonexistentKey")) + collectResults(sql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as sql() by repeating the above using sql(). From 51229ff7f4d3517706a1cdc1a2943ede1c605089 Mon Sep 17 00:00:00 2001 From: yingjieMiao Date: Mon, 29 Sep 2014 18:01:27 -0700 Subject: [PATCH 0830/1492] [graphX] GraphOps: random pick vertex bug When `numVertices > 50`, probability is set to 0. This would cause infinite loop. Author: yingjieMiao Closes #2553 from yingjieMiao/graphx and squashes the following commits: 6adf3c8 [yingjieMiao] [graphX] GraphOps: random pick vertex bug --- graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 02afaa987d40d..d0dd45dba618e 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -254,7 +254,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali * Picks a random vertex from the graph and returns its ID. */ def pickRandomVertex(): VertexId = { - val probability = 50 / graph.numVertices + val probability = 50.0 / graph.numVertices var found = false var retVal: VertexId = null.asInstanceOf[VertexId] while (!found) { From dc30e4504abcda1774f5f09a08bba73d29a2898b Mon Sep 17 00:00:00 2001 From: oded Date: Mon, 29 Sep 2014 18:05:53 -0700 Subject: [PATCH 0831/1492] Fixed the condition in StronglyConnectedComponents Issue: SPARK-3635 Author: oded Closes #2486 from odedz/master and squashes the following commits: dd7890a [oded] Fixed the condition in StronglyConnectedComponents Issue: SPARK-3635 --- .../apache/spark/graphx/lib/StronglyConnectedComponents.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala index 46da38eeb725a..8dd958033b338 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -75,7 +75,7 @@ object StronglyConnectedComponents { sccWorkGraph, Long.MaxValue, activeDirection = EdgeDirection.Out)( (vid, myScc, neighborScc) => (math.min(myScc._1, neighborScc), myScc._2), e => { - if (e.srcId < e.dstId) { + if (e.srcAttr._1 < e.dstAttr._1) { Iterator((e.dstId, e.srcAttr._1)) } else { Iterator() From 210404a56197ad347f1e621ed53ef01327fba2bd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Sep 2014 21:53:21 -0700 Subject: [PATCH 0832/1492] Minor cleanup of code. Author: Reynold Xin Closes #2581 from rxin/minor-cleanup and squashes the following commits: 736a91b [Reynold Xin] Minor cleanup of code. --- .../apache/spark/scheduler/JobLogger.scala | 17 +----- .../org/apache/spark/util/JsonProtocol.scala | 1 - .../scala/org/apache/spark/util/Utils.scala | 60 +++++++++---------- 3 files changed, 31 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index ceb434feb6ca1..54904bffdf10b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -20,15 +20,12 @@ package org.apache.spark.scheduler import java.io.{File, FileNotFoundException, IOException, PrintWriter} import java.text.SimpleDateFormat import java.util.{Date, Properties} -import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.executor.{DataReadMethod, TaskMetrics} -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel +import org.apache.spark.executor.TaskMetrics /** * :: DeveloperApi :: @@ -62,24 +59,16 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener private val dateFormat = new ThreadLocal[SimpleDateFormat]() { override def initialValue() = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") } - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] createLogDir() - // The following 5 functions are used only in testing. - private[scheduler] def getLogDir = logDir - private[scheduler] def getJobIdToPrintWriter = jobIdToPrintWriter - private[scheduler] def getStageIdToJobId = stageIdToJobId - private[scheduler] def getJobIdToStageIds = jobIdToStageIds - private[scheduler] def getEventQueue = eventQueue - /** Create a folder for log files, the folder's name is the creation time of jobLogger */ protected def createLogDir() { val dir = new File(logDir + "/" + logDirName + "/") if (dir.exists()) { return } - if (dir.mkdirs() == false) { + if (!dir.mkdirs()) { // JobLogger should throw a exception rather than continue to construct this object. throw new IOException("create log directory error:" + logDir + "/" + logDirName + "/") } @@ -261,7 +250,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener protected def recordJobProperties(jobId: Int, properties: Properties) { if (properties != null) { val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobId, description, false) + jobLogInfo(jobId, description, withTime = false) } } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 6a48f673c4e78..5b2e7d3a7edb9 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -25,7 +25,6 @@ import scala.collection.Map import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.JsonAST._ -import org.json4s.jackson.JsonMethods._ import org.apache.spark.executor.{DataReadMethod, InputMetrics, ShuffleReadMetrics, 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 10d440828e323..dbe0cfa2b8ff9 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -23,8 +23,6 @@ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} -import org.apache.log4j.PropertyConfigurator - import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -37,12 +35,12 @@ import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration +import org.apache.log4j.PropertyConfigurator import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark._ -import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.ExecutorUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} @@ -86,7 +84,7 @@ private[spark] object Utils extends Logging { ois.readObject.asInstanceOf[T] } - /** Deserialize a Long value (used for {@link org.apache.spark.api.python.PythonPartitioner}) */ + /** Deserialize a Long value (used for [[org.apache.spark.api.python.PythonPartitioner]]) */ def deserializeLongValue(bytes: Array[Byte]) : Long = { // Note: we assume that we are given a Long value encoded in network (big-endian) byte order var result = bytes(7) & 0xFFL @@ -153,7 +151,7 @@ private[spark] object Utils extends Logging { def classForName(className: String) = Class.forName(className, true, getContextOrSparkClassLoader) /** - * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. + * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]] */ def writeByteBuffer(bb: ByteBuffer, out: ObjectOutput) = { if (bb.hasArray) { @@ -333,7 +331,7 @@ private[spark] object Utils extends Logging { val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) val targetFile = new File(targetDir, filename) val uri = new URI(url) - val fileOverwrite = conf.getBoolean("spark.files.overwrite", false) + val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) uri.getScheme match { case "http" | "https" | "ftp" => logInfo("Fetching " + url + " to " + tempFile) @@ -355,7 +353,7 @@ private[spark] object Utils extends Logging { uc.connect() val in = uc.getInputStream() val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, true) + Utils.copyStream(in, out, closeStreams = true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { if (fileOverwrite) { targetFile.delete() @@ -402,7 +400,7 @@ private[spark] object Utils extends Logging { val fs = getHadoopFileSystem(uri, hadoopConf) val in = fs.open(new Path(uri)) val out = new FileOutputStream(tempFile) - Utils.copyStream(in, out, true) + Utils.copyStream(in, out, closeStreams = true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { if (fileOverwrite) { targetFile.delete() @@ -666,7 +664,7 @@ private[spark] object Utils extends Logging { */ def deleteRecursively(file: File) { if (file != null) { - if ((file.isDirectory) && !isSymlink(file)) { + if (file.isDirectory() && !isSymlink(file)) { for (child <- listFilesSafely(file)) { deleteRecursively(child) } @@ -701,11 +699,7 @@ private[spark] object Utils extends Logging { new File(file.getParentFile().getCanonicalFile(), file.getName()) } - if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { - return false - } else { - return true - } + !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()) } /** @@ -804,7 +798,7 @@ private[spark] object Utils extends Logging { .start() new Thread("read stdout for " + command(0)) { override def run() { - for (line <- Source.fromInputStream(process.getInputStream).getLines) { + for (line <- Source.fromInputStream(process.getInputStream).getLines()) { System.err.println(line) } } @@ -818,8 +812,10 @@ private[spark] object Utils extends Logging { /** * Execute a command and get its output, throwing an exception if it yields a code other than 0. */ - def executeAndGetOutput(command: Seq[String], workingDir: File = new File("."), - extraEnvironment: Map[String, String] = Map.empty): String = { + def executeAndGetOutput( + command: Seq[String], + workingDir: File = new File("."), + extraEnvironment: Map[String, String] = Map.empty): String = { val builder = new ProcessBuilder(command: _*) .directory(workingDir) val environment = builder.environment() @@ -829,7 +825,7 @@ private[spark] object Utils extends Logging { val process = builder.start() new Thread("read stderr for " + command(0)) { override def run() { - for (line <- Source.fromInputStream(process.getErrorStream).getLines) { + for (line <- Source.fromInputStream(process.getErrorStream).getLines()) { System.err.println(line) } } @@ -837,7 +833,7 @@ private[spark] object Utils extends Logging { val output = new StringBuffer val stdoutThread = new Thread("read stdout for " + command(0)) { override def run() { - for (line <- Source.fromInputStream(process.getInputStream).getLines) { + for (line <- Source.fromInputStream(process.getInputStream).getLines()) { output.append(line) } } @@ -846,8 +842,8 @@ private[spark] object Utils extends Logging { val exitCode = process.waitFor() stdoutThread.join() // Wait for it to finish reading output if (exitCode != 0) { - logError(s"Process $command exited with code $exitCode: ${output}") - throw new SparkException("Process " + command + " exited with code " + exitCode) + logError(s"Process $command exited with code $exitCode: $output") + throw new SparkException(s"Process $command exited with code $exitCode") } output.toString } @@ -860,6 +856,7 @@ private[spark] object Utils extends Logging { try { block } catch { + case e: ControlThrowable => throw e case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) } } @@ -884,13 +881,12 @@ private[spark] object Utils extends Logging { * @param skipClass Function that is used to exclude non-user-code classes. */ def getCallSite(skipClass: String => Boolean = coreExclusionFunction): CallSite = { - val trace = Thread.currentThread.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")) - } + val trace = Thread.currentThread.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 @@ -924,7 +920,7 @@ private[spark] object Utils extends Logging { } val callStackDepth = System.getProperty("spark.callstack.depth", "20").toInt CallSite( - shortForm = "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine), + shortForm = s"$lastSparkMethod at $firstUserFile:$firstUserLine", longForm = callStack.take(callStackDepth).mkString("\n")) } @@ -1027,7 +1023,7 @@ private[spark] object Utils extends Logging { false } - def isSpace(c: Char): Boolean = { + private def isSpace(c: Char): Boolean = { " \t\r\n".indexOf(c) != -1 } @@ -1179,7 +1175,7 @@ private[spark] object Utils extends Logging { } import scala.sys.process._ (linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! - ProcessLogger(line => (logInfo(line))) + ProcessLogger(line => logInfo(line)) } @@ -1260,7 +1256,7 @@ private[spark] object Utils extends Logging { val startTime = System.currentTimeMillis while (!terminated) { try { - process.exitValue + process.exitValue() terminated = true } catch { case e: IllegalThreadStateException => From 6b79bfb42580b6bd4c4cd99fb521534a94150693 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Sep 2014 22:56:22 -0700 Subject: [PATCH 0833/1492] [SPARK-3613] Record only average block size in MapStatus for large stages This changes the way we send MapStatus from executors back to driver for large stages (>2000 tasks). For large stages, we no longer send one byte per block. Instead, we just send the average block size. This makes large jobs (tens of thousands of tasks) much more reliable since the driver no longer sends huge amount of data. Author: Reynold Xin Closes #2470 from rxin/mapstatus and squashes the following commits: 822ff54 [Reynold Xin] Code review feedback. 3b86f56 [Reynold Xin] Added MimaExclude. f89d182 [Reynold Xin] Fixed a bug in MapStatus 6a0401c [Reynold Xin] [SPARK-3613] Record only average block size in MapStatus for large stages. --- .../org/apache/spark/MapOutputTracker.scala | 29 +---- .../apache/spark/scheduler/MapStatus.scala | 119 ++++++++++++++++-- .../shuffle/hash/HashShuffleWriter.scala | 8 +- .../shuffle/sort/SortShuffleWriter.scala | 3 +- .../apache/spark/MapOutputTrackerSuite.scala | 66 ++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- .../spark/scheduler/MapStatusSuite.scala | 92 ++++++++++++++ .../apache/spark/util/AkkaUtilsSuite.scala | 14 +-- project/MimaExcludes.scala | 5 +- 9 files changed, 240 insertions(+), 98 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index f92189b707fb5..4cb0bd4142435 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -349,7 +349,6 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } private[spark] object MapOutputTracker { - private val LOG_BASE = 1.1 // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will @@ -385,34 +384,8 @@ private[spark] object MapOutputTracker { throw new MetadataFetchFailedException( shuffleId, reduceId, "Missing an output location for shuffle " + shuffleId) } else { - (status.location, decompressSize(status.compressedSizes(reduceId))) + (status.location, status.getSizeForBlock(reduceId)) } } } - - /** - * Compress a size in bytes to 8 bits for efficient reporting of map output sizes. - * We do this by encoding the log base 1.1 of the size as an integer, which can support - * sizes up to 35 GB with at most 10% error. - */ - def compressSize(size: Long): Byte = { - if (size == 0) { - 0 - } else if (size <= 1L) { - 1 - } else { - math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte - } - } - - /** - * Decompress an 8-bit encoded block size, using the reverse operation of compressSize. - */ - def decompressSize(compressedSize: Byte): Long = { - if (compressedSize == 0) { - 0 - } else { - math.pow(LOG_BASE, compressedSize & 0xFF).toLong - } - } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index d3f63ff92ac6f..e25096ea92d70 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -24,22 +24,123 @@ import org.apache.spark.storage.BlockManagerId /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. - * The map output sizes are compressed using MapOutputTracker.compressSize. */ -private[spark] class MapStatus(var location: BlockManagerId, var compressedSizes: Array[Byte]) - extends Externalizable { +private[spark] sealed trait MapStatus { + /** Location where this task was run. */ + def location: BlockManagerId - def this() = this(null, null) // For deserialization only + /** Estimated size for the reduce block, in bytes. */ + def getSizeForBlock(reduceId: Int): Long +} + + +private[spark] object MapStatus { + + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { + if (uncompressedSizes.length > 2000) { + new HighlyCompressedMapStatus(loc, uncompressedSizes) + } else { + new CompressedMapStatus(loc, uncompressedSizes) + } + } + + private[this] val LOG_BASE = 1.1 + + /** + * Compress a size in bytes to 8 bits for efficient reporting of map output sizes. + * We do this by encoding the log base 1.1 of the size as an integer, which can support + * sizes up to 35 GB with at most 10% error. + */ + def compressSize(size: Long): Byte = { + if (size == 0) { + 0 + } else if (size <= 1L) { + 1 + } else { + math.min(255, math.ceil(math.log(size) / math.log(LOG_BASE)).toInt).toByte + } + } + + /** + * Decompress an 8-bit encoded block size, using the reverse operation of compressSize. + */ + def decompressSize(compressedSize: Byte): Long = { + if (compressedSize == 0) { + 0 + } else { + math.pow(LOG_BASE, compressedSize & 0xFF).toLong + } + } +} + + +/** + * A [[MapStatus]] implementation that tracks the size of each block. Size for each block is + * represented using a single byte. + * + * @param loc location where the task is being executed. + * @param compressedSizes size of the blocks, indexed by reduce partition id. + */ +private[spark] class CompressedMapStatus( + private[this] var loc: BlockManagerId, + private[this] var compressedSizes: Array[Byte]) + extends MapStatus with Externalizable { + + protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only + + def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { + this(loc, uncompressedSizes.map(MapStatus.compressSize)) + } - def writeExternal(out: ObjectOutput) { - location.writeExternal(out) + override def location: BlockManagerId = loc + + override def getSizeForBlock(reduceId: Int): Long = { + MapStatus.decompressSize(compressedSizes(reduceId)) + } + + override def writeExternal(out: ObjectOutput): Unit = { + loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) } - def readExternal(in: ObjectInput) { - location = BlockManagerId(in) - compressedSizes = new Array[Byte](in.readInt()) + override def readExternal(in: ObjectInput): Unit = { + loc = BlockManagerId(in) + val len = in.readInt() + compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) } } + + +/** + * A [[MapStatus]] implementation that only stores the average size of the blocks. + * + * @param loc location where the task is being executed. + * @param avgSize average size of all the blocks + */ +private[spark] class HighlyCompressedMapStatus( + private[this] var loc: BlockManagerId, + private[this] var avgSize: Long) + extends MapStatus with Externalizable { + + def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { + this(loc, uncompressedSizes.sum / uncompressedSizes.length) + } + + protected def this() = this(null, 0L) // For deserialization only + + override def location: BlockManagerId = loc + + override def getSizeForBlock(reduceId: Int): Long = avgSize + + override def writeExternal(out: ObjectOutput): Unit = { + loc.writeExternal(out) + out.writeLong(avgSize) + } + + override def readExternal(in: ObjectInput): Unit = { + loc = BlockManagerId(in) + avgSize = in.readLong() + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 4b9454d75abb7..746ed33b54c00 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -103,13 +103,11 @@ private[spark] class HashShuffleWriter[K, V]( private def commitWritesAndBuildStatus(): MapStatus = { // Commit the writes. Get the size of each bucket block (total block size). - val compressedSizes = shuffle.writers.map { writer: BlockObjectWriter => + val sizes: Array[Long] = shuffle.writers.map { writer: BlockObjectWriter => writer.commitAndClose() - val size = writer.fileSegment().length - MapOutputTracker.compressSize(size) + writer.fileSegment().length } - - new MapStatus(blockManager.blockManagerId, compressedSizes) + MapStatus(blockManager.blockManagerId, sizes) } private def revertWrites(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 89a78d6982ba0..927481b72cf4f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -70,8 +70,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths) - mapStatus = new MapStatus(blockManager.blockManagerId, - partitionLengths.map(MapOutputTracker.compressSize)) + mapStatus = MapStatus(blockManager.blockManagerId, partitionLengths) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 5369169811f81..1fef79ad1001f 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -23,32 +23,13 @@ import akka.actor._ import akka.testkit.TestActorRef import org.scalatest.FunSuite -import org.apache.spark.scheduler.MapStatus +import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { private val conf = new SparkConf - test("compressSize") { - assert(MapOutputTracker.compressSize(0L) === 0) - assert(MapOutputTracker.compressSize(1L) === 1) - assert(MapOutputTracker.compressSize(2L) === 8) - assert(MapOutputTracker.compressSize(10L) === 25) - assert((MapOutputTracker.compressSize(1000000L) & 0xFF) === 145) - assert((MapOutputTracker.compressSize(1000000000L) & 0xFF) === 218) - // This last size is bigger than we can encode in a byte, so check that we just return 255 - assert((MapOutputTracker.compressSize(1000000000000000000L) & 0xFF) === 255) - } - - test("decompressSize") { - assert(MapOutputTracker.decompressSize(0) === 0) - for (size <- Seq(2L, 10L, 100L, 50000L, 1000000L, 1000000000L)) { - val size2 = MapOutputTracker.decompressSize(MapOutputTracker.compressSize(size)) - assert(size2 >= 0.99 * size && size2 <= 1.11 * size, - "size " + size + " decompressed to " + size2 + ", which is out of range") - } - } test("master start and stop") { val actorSystem = ActorSystem("test") @@ -65,14 +46,12 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) assert(tracker.containsShuffle(10)) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val compressedSize10000 = MapOutputTracker.compressSize(10000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - val size10000 = MapOutputTracker.decompressSize(compressedSize10000) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), + Array(1000L, 10000L))) + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), + Array(10000L, 1000L))) val statuses = tracker.getServerStatuses(10, 0) assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), (BlockManagerId("b", "hostB", 1000), size10000))) @@ -84,11 +63,11 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val compressedSize10000 = MapOutputTracker.compressSize(10000L) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + val compressedSize1000 = MapStatus.compressSize(1000L) + val compressedSize10000 = MapStatus.compressSize(10000L) + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), Array(compressedSize1000, compressedSize10000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) assert(tracker.getServerStatuses(10, 0).nonEmpty) @@ -103,11 +82,11 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val compressedSize10000 = MapOutputTracker.compressSize(10000L) - tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000), + val compressedSize1000 = MapStatus.compressSize(1000L) + val compressedSize10000 = MapStatus.compressSize(10000L) + tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), Array(compressedSize1000, compressedSize1000, compressedSize1000))) - tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000), + tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), Array(compressedSize10000, compressedSize1000, compressedSize1000))) // As if we had two simultaneous fetch failures @@ -142,10 +121,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { slaveTracker.updateEpoch(masterTracker.getEpoch) intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, MapStatus( + BlockManagerId("a", "hostA", 1000), Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === @@ -173,8 +151,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // Frame size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("88", "mph", 1000), Array.fill[Byte](10)(0))) + masterTracker.registerMapOutput(10, 0, MapStatus( + BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) masterActor.receive(GetMapOutputStatuses(10)) } @@ -194,8 +172,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // being sent. masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => - masterTracker.registerMapOutput(20, i, new MapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Byte](4000000)(0))) + masterTracker.registerMapOutput(20, i, new CompressedMapStatus( + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index aa73469b6acd8..a2e4f712db55b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -740,7 +740,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F } private def makeMapStatus(host: String, reduces: Int): MapStatus = - new MapStatus(makeBlockManagerId(host), Array.fill[Byte](reduces)(2)) + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(2)) private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala new file mode 100644 index 0000000000000..79e04f046e4c4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import org.apache.spark.storage.BlockManagerId +import org.scalatest.FunSuite + +import org.apache.spark.SparkConf +import org.apache.spark.serializer.JavaSerializer + + +class MapStatusSuite extends FunSuite { + + test("compressSize") { + assert(MapStatus.compressSize(0L) === 0) + assert(MapStatus.compressSize(1L) === 1) + assert(MapStatus.compressSize(2L) === 8) + assert(MapStatus.compressSize(10L) === 25) + assert((MapStatus.compressSize(1000000L) & 0xFF) === 145) + assert((MapStatus.compressSize(1000000000L) & 0xFF) === 218) + // This last size is bigger than we can encode in a byte, so check that we just return 255 + assert((MapStatus.compressSize(1000000000000000000L) & 0xFF) === 255) + } + + test("decompressSize") { + assert(MapStatus.decompressSize(0) === 0) + for (size <- Seq(2L, 10L, 100L, 50000L, 1000000L, 1000000000L)) { + val size2 = MapStatus.decompressSize(MapStatus.compressSize(size)) + assert(size2 >= 0.99 * size && size2 <= 1.11 * size, + "size " + size + " decompressed to " + size2 + ", which is out of range") + } + } + + test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { + val sizes = Array.fill[Long](2001)(150L) + val status = MapStatus(null, sizes) + assert(status.isInstanceOf[HighlyCompressedMapStatus]) + assert(status.getSizeForBlock(10) === 150L) + assert(status.getSizeForBlock(50) === 150L) + assert(status.getSizeForBlock(99) === 150L) + assert(status.getSizeForBlock(2000) === 150L) + } + + test(classOf[HighlyCompressedMapStatus].getName + ": estimated size is within 10%") { + val sizes = Array.tabulate[Long](50) { i => i.toLong } + val loc = BlockManagerId("a", "b", 10) + val status = MapStatus(loc, sizes) + val ser = new JavaSerializer(new SparkConf) + val buf = ser.newInstance().serialize(status) + val status1 = ser.newInstance().deserialize[MapStatus](buf) + assert(status1.location == loc) + for (i <- 0 until sizes.length) { + // make sure the estimated size is within 10% of the input; note that we skip the very small + // sizes because the compression is very lossy there. + val estimate = status1.getSizeForBlock(i) + if (estimate > 100) { + assert(math.abs(estimate - sizes(i)) * 10 <= sizes(i), + s"incorrect estimated size $estimate, original was ${sizes(i)}") + } + } + } + + test(classOf[HighlyCompressedMapStatus].getName + ": estimated size should be the average size") { + val sizes = Array.tabulate[Long](3000) { i => i.toLong } + val avg = sizes.sum / sizes.length + val loc = BlockManagerId("a", "b", 10) + val status = MapStatus(loc, sizes) + val ser = new JavaSerializer(new SparkConf) + val buf = ser.newInstance().serialize(status) + val status1 = ser.newInstance().deserialize[MapStatus](buf) + assert(status1.location == loc) + for (i <- 0 until 3000) { + val estimate = status1.getSizeForBlock(i) + assert(estimate === avg) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala index 76bf4cfd11267..7bca1711ae226 100644 --- a/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/AkkaUtilsSuite.scala @@ -106,10 +106,9 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, + MapStatus(BlockManagerId("a", "hostA", 1000), Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) @@ -157,10 +156,9 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) - val compressedSize1000 = MapOutputTracker.compressSize(1000L) - val size1000 = MapOutputTracker.decompressSize(compressedSize1000) - masterTracker.registerMapOutput(10, 0, new MapStatus( - BlockManagerId("a", "hostA", 1000), Array(compressedSize1000))) + val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) + masterTracker.registerMapOutput(10, 0, MapStatus( + BlockManagerId("a", "hostA", 1000), Array(1000L))) masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 1adfaa18c6202..4076ebc6fc8d5 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -45,7 +45,10 @@ object MimaExcludes { ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL1"), ProblemFilters.exclude[MissingMethodProblem]( - "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL2") + "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL2"), + // MapStatus should be private[spark] + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.scheduler.MapStatus") ) case v if v.startsWith("1.1") => From de700d31778eb68807183cf32be8034abdc0120e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 29 Sep 2014 23:17:53 -0700 Subject: [PATCH 0834/1492] [SPARK-3709] Executors don't always report broadcast block removal properly back to the driver The problem was that the 2nd argument in RemoveBroadcast is not tellMaster! It is "removeFromDriver". Basically when removeFromDriver is not true, we don't report broadcast block removal back to the driver, and then other executors mistakenly think that the executor would still have the block, and try to fetch from it. cc @tdas Author: Reynold Xin Closes #2588 from rxin/debug and squashes the following commits: 6dab2e3 [Reynold Xin] Don't log random messages. f430686 [Reynold Xin] Always report broadcast removal back to master. 2a13f70 [Reynold Xin] iii --- .../apache/spark/network/nio/NioBlockTransferService.scala | 2 +- .../org/apache/spark/storage/BlockManagerSlaveActor.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index 59958ee894230..b389b9a2022c6 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -200,6 +200,6 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa val buffer = blockDataManager.getBlockData(blockId).orNull logDebug("GetBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + " and got buffer " + buffer) - buffer.nioByteBuffer() + if (buffer == null) null else buffer.nioByteBuffer() } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index 14ae2f38c5670..8462871e798a5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -58,9 +58,9 @@ class BlockManagerSlaveActor( SparkEnv.get.shuffleManager.unregisterShuffle(shuffleId) } - case RemoveBroadcast(broadcastId, tellMaster) => + case RemoveBroadcast(broadcastId, _) => doAsync[Int]("removing broadcast " + broadcastId, sender) { - blockManager.removeBroadcast(broadcastId, tellMaster) + blockManager.removeBroadcast(broadcastId, tellMaster = true) } case GetBlockStatus(blockId, _) => From b167a8c7e75d9e816784bd655bce1feb6c447210 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 29 Sep 2014 23:36:10 -0700 Subject: [PATCH 0835/1492] [SPARK-3734] DriverRunner should not read SPARK_HOME from submitter's environment When using spark-submit in `cluster` mode to submit a job to a Spark Standalone cluster, if the JAVA_HOME environment variable was set on the submitting machine then DriverRunner would attempt to use the submitter's JAVA_HOME to launch the driver process (instead of the worker's JAVA_HOME), causing the driver to fail unless the submitter and worker had the same Java location. This commit fixes this by reading JAVA_HOME from sys.env instead of command.environment. Author: Josh Rosen Closes #2586 from JoshRosen/SPARK-3734 and squashes the following commits: e9513d9 [Josh Rosen] [SPARK-3734] DriverRunner should not read SPARK_HOME from submitter's environment. --- .../scala/org/apache/spark/deploy/worker/CommandUtils.scala | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 12e98fd40d6c9..2e9be2a180c68 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -30,7 +30,7 @@ import org.apache.spark.util.Utils private[spark] object CommandUtils extends Logging { def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val runner = getEnv("JAVA_HOME", command).map(_ + "/bin/java").getOrElse("java") + val runner = sys.env.get("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() // fails to kill a process tree on Windows @@ -38,9 +38,6 @@ object CommandUtils extends Logging { command.arguments } - private def getEnv(key: String, command: Command): Option[String] = - command.environment.get(key).orElse(Option(System.getenv(key))) - /** * Attention: this must always be aligned with the environment variables in the run scripts and * the way the JAVA_OPTS are assembled there. From b64fcbd2dcec3418397328399c58f98d990a54f1 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 30 Sep 2014 09:43:46 -0700 Subject: [PATCH 0836/1492] Revert "[SPARK-3007][SQL]Add Dynamic Partition support to Spark Sql hive" This reverts commit 0bbe7faeffa17577ae8a33dfcd8c4c783db5c909. --- .../execution/HiveCompatibilitySuite.scala | 17 -- .../org/apache/spark/SparkHadoopWriter.scala | 195 ++++++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 5 + .../hive/execution/InsertIntoHiveTable.scala | 207 ++++++++--------- .../spark/sql/hive/hiveWriterContainers.scala | 217 ------------------ ...rtition-0-be33aaa7253c8f248ff3921cd7dae340 | 0 ...rtition-1-640552dd462707563fd255a713f83b41 | 0 ...rtition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 | 1 - ...rtition-3-b7f7fa7ebf666f4fee27e149d8c6961f | 0 ...rtition-4-8bdb71ad8cb3cc3026043def2525de3a | 0 ...rtition-5-c630dce438f3792e7fb0f523fbbb3e1e | 0 ...rtition-6-7abc9ec8a36cdc5e89e955265a7fd7cf | 0 ...rtition-7-be33aaa7253c8f248ff3921cd7dae340 | 0 .../sql/hive/execution/HiveQuerySuite.scala | 100 +------- 14 files changed, 299 insertions(+), 443 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf delete mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 35e9c9939d4b7..556c984ad392b 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -220,23 +220,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", - "dynamic_partition_skip_default", - "infer_bucket_sort_dyn_part", - "load_dyn_part1", - "load_dyn_part2", - "load_dyn_part3", - "load_dyn_part4", - "load_dyn_part5", - "load_dyn_part6", - "load_dyn_part7", - "load_dyn_part8", - "load_dyn_part9", - "load_dyn_part10", - "load_dyn_part11", - "load_dyn_part12", - "load_dyn_part13", - "load_dyn_part14", - "load_dyn_part14_win", "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala new file mode 100644 index 0000000000000..ab7862f4f9e06 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -0,0 +1,195 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.IOException +import java.text.NumberFormat +import java.util.Date + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.mapred._ +import org.apache.hadoop.io.Writable + +import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} + +/** + * Internal helper class that saves an RDD using a Hive OutputFormat. + * It is based on [[SparkHadoopWriter]]. + */ +private[hive] class SparkHiveHadoopWriter( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc) + extends Logging + with SparkHadoopMapRedUtil + with Serializable { + + private val now = new Date() + private val conf = new SerializableWritable(jobConf) + + private var jobID = 0 + private var splitID = 0 + private var attemptID = 0 + private var jID: SerializableWritable[JobID] = null + private var taID: SerializableWritable[TaskAttemptID] = null + + @transient private var writer: FileSinkOperator.RecordWriter = null + @transient private var format: HiveOutputFormat[AnyRef, Writable] = null + @transient private var committer: OutputCommitter = null + @transient private var jobContext: JobContext = null + @transient private var taskContext: TaskAttemptContext = null + + def preSetup() { + setIDs(0, 0, 0) + setConfParams() + + val jCtxt = getJobContext() + getOutputCommitter().setupJob(jCtxt) + } + + + def setup(jobid: Int, splitid: Int, attemptid: Int) { + setIDs(jobid, splitid, attemptid) + setConfParams() + } + + def open() { + val numfmt = NumberFormat.getInstance() + numfmt.setMinimumIntegerDigits(5) + numfmt.setGroupingUsed(false) + + val extension = Utilities.getFileExtension( + conf.value, + fileSinkConf.getCompressed, + getOutputFormat()) + + val outputName = "part-" + numfmt.format(splitID) + extension + val path = FileOutputFormat.getTaskOutputPath(conf.value, outputName) + + getOutputCommitter().setupTask(getTaskContext()) + writer = HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + fileSinkConf, + path, + null) + } + + def write(value: Writable) { + if (writer != null) { + writer.write(value) + } else { + throw new IOException("Writer is null, open() has not been called") + } + } + + def close() { + // Seems the boolean value passed into close does not matter. + writer.close(false) + } + + def commit() { + val taCtxt = getTaskContext() + val cmtr = getOutputCommitter() + if (cmtr.needsTaskCommit(taCtxt)) { + try { + cmtr.commitTask(taCtxt) + logInfo (taID + ": Committed") + } catch { + case e: IOException => + logError("Error committing the output of task: " + taID.value, e) + cmtr.abortTask(taCtxt) + throw e + } + } else { + logWarning ("No need to commit output of task: " + taID.value) + } + } + + def commitJob() { + // always ? Or if cmtr.needsTaskCommit ? + val cmtr = getOutputCommitter() + cmtr.commitJob(getJobContext()) + } + + // ********* Private Functions ********* + + private def getOutputFormat(): HiveOutputFormat[AnyRef,Writable] = { + if (format == null) { + format = conf.value.getOutputFormat() + .asInstanceOf[HiveOutputFormat[AnyRef,Writable]] + } + format + } + + private def getOutputCommitter(): OutputCommitter = { + if (committer == null) { + committer = conf.value.getOutputCommitter + } + committer + } + + private def getJobContext(): JobContext = { + if (jobContext == null) { + jobContext = newJobContext(conf.value, jID.value) + } + jobContext + } + + private def getTaskContext(): TaskAttemptContext = { + if (taskContext == null) { + taskContext = newTaskAttemptContext(conf.value, taID.value) + } + taskContext + } + + private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { + jobID = jobId + splitID = splitId + attemptID = attemptId + + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) + taID = new SerializableWritable[TaskAttemptID]( + new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) + } + + private def setConfParams() { + conf.value.set("mapred.job.id", jID.value.toString) + conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) + conf.value.set("mapred.task.id", taID.value.toString) + conf.value.setBoolean("mapred.task.is.map", true) + conf.value.setInt("mapred.task.partition", splitID) + } +} + +private[hive] object SparkHiveHadoopWriter { + def createPathFromString(path: String, conf: JobConf): Path = { + if (path == null) { + throw new IllegalArgumentException("Output path is null") + } + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(conf) + if (outputPath == null || fs == null) { + throw new IllegalArgumentException("Incorrectly formatted output path") + } + outputPath.makeQualified(fs) + } +} 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 4e30e6e06fe21..0aa6292c0184e 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 @@ -837,6 +837,11 @@ private[hive] object HiveQl { cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) + if (partitionKeys.values.exists(p => p.isEmpty)) { + throw new NotImplementedError(s"Do not support INSERT INTO/OVERWRITE with" + + s"dynamic partitioning.") + } + InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query, overwrite) case a: ASTNode => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 3d2ee010696f6..a284a91a91e31 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -19,25 +19,27 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConversions._ +import java.util.{HashMap => JHashMap} + import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils +import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} -import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.objectinspector._ -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector +import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} +import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} -import org.apache.spark.sql.hive._ -import org.apache.spark.{SerializableWritable, SparkException, TaskContext} +import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} +import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, SparkHiveHadoopWriter} /** * :: DeveloperApi :: @@ -49,7 +51,7 @@ case class InsertIntoHiveTable( child: SparkPlan, overwrite: Boolean) (@transient sc: HiveContext) - extends UnaryNode with Command { + extends UnaryNode { @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) @@ -99,74 +101,66 @@ case class InsertIntoHiveTable( } def saveAsHiveFile( - rdd: RDD[Row], + rdd: RDD[Writable], valueClass: Class[_], fileSinkConf: FileSinkDesc, - conf: SerializableWritable[JobConf], - writerContainer: SparkHiveWriterContainer) { - assert(valueClass != null, "Output value class not set") - conf.value.setOutputValueClass(valueClass) - - val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName - assert(outputFileFormatClassName != null, "Output format class not set") - conf.value.set("mapred.output.format.class", outputFileFormatClassName) - - val isCompressed = conf.value.getBoolean( - ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) - + conf: JobConf, + isCompressed: Boolean) { + if (valueClass == null) { + throw new SparkException("Output value class not set") + } + conf.setOutputValueClass(valueClass) + if (fileSinkConf.getTableInfo.getOutputFileFormatClassName == null) { + throw new SparkException("Output format class not set") + } + // Doesn't work in Scala 2.9 due to what may be a generics bug + // TODO: Should we uncomment this for Scala 2.10? + // conf.setOutputFormat(outputFormatClass) + conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) if (isCompressed) { // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", // and "mapred.output.compression.type" have no impact on ORC because it uses table properties // to store compression information. - conf.value.set("mapred.output.compress", "true") + conf.set("mapred.output.compress", "true") fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(conf.value.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(conf.value.get("mapred.output.compression.type")) + fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) } - conf.value.setOutputCommitter(classOf[FileOutputCommitter]) - + conf.setOutputCommitter(classOf[FileOutputCommitter]) FileOutputFormat.setOutputPath( - conf.value, - SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName, conf.value)) - log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) + conf, + SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) - writerContainer.driverSideSetup() - sc.sparkContext.runJob(rdd, writeToFile _) - writerContainer.commitJob() - - // Note that this function is executed on executor side - def writeToFile(context: TaskContext, iterator: Iterator[Row]) { - val serializer = newSerializer(fileSinkConf.getTableInfo) - val standardOI = ObjectInspectorUtils - .getStandardObjectInspector( - fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] + log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray - val outputData = new Array[Any](fieldOIs.length) + val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) + writer.preSetup() + def writeToFile(context: TaskContext, iter: Iterator[Writable]) { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt - writerContainer.executorSideSetup(context.stageId, context.partitionId, attemptNumber) - iterator.foreach { row => - var i = 0 - while (i < fieldOIs.length) { - // TODO (lian) avoid per row dynamic dispatching and pattern matching cost in `wrap` - outputData(i) = wrap(row(i), fieldOIs(i)) - i += 1 - } + writer.setup(context.stageId, context.partitionId, attemptNumber) + writer.open() - val writer = writerContainer.getLocalFileWriter(row) - writer.write(serializer.serialize(outputData, standardOI)) + var count = 0 + while(iter.hasNext) { + val record = iter.next() + count += 1 + writer.write(record) } - writerContainer.close() + writer.close() + writer.commit() } + + sc.sparkContext.runJob(rdd, writeToFile _) + writer.commitJob() } + override def execute() = result + /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the @@ -174,57 +168,50 @@ case class InsertIntoHiveTable( * * Note: this is run once and then kept to avoid double insertions. */ - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + private lazy val result: RDD[Row] = { + val childRdd = child.execute() + assert(childRdd != null) + // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc val tableLocation = table.hiveQlTable.getDataLocation val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) + val rdd = childRdd.mapPartitions { iter => + val serializer = newSerializer(fileSinkConf.getTableInfo) + val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] - val numDynamicPartitions = partition.values.count(_.isEmpty) - val numStaticPartitions = partition.values.count(_.nonEmpty) - val partitionSpec = partition.map { - case (key, Some(value)) => key -> value - case (key, None) => key -> "" - } - - // All partition column names in the format of "//..." - val partitionColumns = fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns") - val partitionColumnNames = Option(partitionColumns).map(_.split("/")).orNull - - // Validate partition spec if there exist any dynamic partitions - if (numDynamicPartitions > 0) { - // Report error if dynamic partitioning is not enabled - if (!sc.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { - throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg) - } - // Report error if dynamic partition strict mode is on but no static partition is found - if (numStaticPartitions == 0 && - sc.hiveconf.getVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { - throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg) - } + val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val outputData = new Array[Any](fieldOIs.length) + iter.map { row => + var i = 0 + while (i < row.length) { + // Casts Strings to HiveVarchars when necessary. + outputData(i) = wrap(row(i), fieldOIs(i)) + i += 1 + } - // Report error if any static partition appears after a dynamic partition - val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) - isDynamic.init.zip(isDynamic.tail).find(_ == (true, false)).foreach { _ => - throw new SparkException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) + serializer.serialize(outputData, standardOI) } } + // ORC stores compression information in table properties. While, there are other formats + // (e.g. RCFile) that rely on hadoop configurations to store compression information. val jobConf = new JobConf(sc.hiveconf) - val jobConfSer = new SerializableWritable(jobConf) - - val writerContainer = if (numDynamicPartitions > 0) { - val dynamicPartColNames = partitionColumnNames.takeRight(numDynamicPartitions) - new SparkHiveDynamicPartitionWriterContainer(jobConf, fileSinkConf, dynamicPartColNames) - } else { - new SparkHiveWriterContainer(jobConf, fileSinkConf) - } - - saveAsHiveFile(child.execute(), outputClass, fileSinkConf, jobConfSer, writerContainer) - + saveAsHiveFile( + rdd, + outputClass, + fileSinkConf, + jobConf, + sc.hiveconf.getBoolean("hive.exec.compress.output", false)) + + // TODO: Handle dynamic partitioning. val outputPath = FileOutputFormat.getOutputPath(jobConf) // Have to construct the format of dbname.tablename. val qualifiedTableName = s"${table.databaseName}.${table.tableName}" @@ -233,6 +220,10 @@ case class InsertIntoHiveTable( // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. val holdDDLTime = false if (partition.nonEmpty) { + val partitionSpec = partition.map { + case (key, Some(value)) => key -> value + case (key, None) => key -> "" // Should not reach here right now. + } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) db.validatePartitionNameCharacters(partVals) // inheritTableSpecs is set to true. It should be set to false for a IMPORT query @@ -240,26 +231,14 @@ case class InsertIntoHiveTable( val inheritTableSpecs = true // TODO: Correctly set isSkewedStoreAsSubdir. val isSkewedStoreAsSubdir = false - if (numDynamicPartitions > 0) { - db.loadDynamicPartitions( - outputPath, - qualifiedTableName, - partitionSpec, - overwrite, - numDynamicPartitions, - holdDDLTime, - isSkewedStoreAsSubdir - ) - } else { - db.loadPartition( - outputPath, - qualifiedTableName, - partitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) - } + db.loadPartition( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) } else { db.loadTable( outputPath, @@ -272,6 +251,6 @@ case class InsertIntoHiveTable( // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. // TODO: implement hive compatibility as rules. - Seq.empty[Row] + sc.sparkContext.makeRDD(Nil, 1) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala deleted file mode 100644 index a667188fa53bd..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.io.IOException -import java.text.NumberFormat -import java.util.Date - -import scala.collection.mutable - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} -import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.FileSinkDesc -import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred._ - -import org.apache.spark.sql.Row -import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} - -/** - * Internal helper class that saves an RDD using a Hive OutputFormat. - * It is based on [[SparkHadoopWriter]]. - */ -private[hive] class SparkHiveWriterContainer( - @transient jobConf: JobConf, - fileSinkConf: FileSinkDesc) - extends Logging - with SparkHadoopMapRedUtil - with Serializable { - - private val now = new Date() - protected val conf = new SerializableWritable(jobConf) - - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null - - @transient private var writer: FileSinkOperator.RecordWriter = null - @transient private lazy val committer = conf.value.getOutputCommitter - @transient private lazy val jobContext = newJobContext(conf.value, jID.value) - @transient private lazy val taskContext = newTaskAttemptContext(conf.value, taID.value) - @transient private lazy val outputFormat = - conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef,Writable]] - - def driverSideSetup() { - setIDs(0, 0, 0) - setConfParams() - committer.setupJob(jobContext) - } - - def executorSideSetup(jobId: Int, splitId: Int, attemptId: Int) { - setIDs(jobId, splitId, attemptId) - setConfParams() - committer.setupTask(taskContext) - initWriters() - } - - protected def getOutputName: String = { - val numberFormat = NumberFormat.getInstance() - numberFormat.setMinimumIntegerDigits(5) - numberFormat.setGroupingUsed(false) - val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat) - "part-" + numberFormat.format(splitID) + extension - } - - def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = writer - - def close() { - // Seems the boolean value passed into close does not matter. - writer.close(false) - commit() - } - - def commitJob() { - committer.commitJob(jobContext) - } - - protected def initWriters() { - // NOTE this method is executed at the executor side. - // For Hive tables without partitions or with only static partitions, only 1 writer is needed. - writer = HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - fileSinkConf, - FileOutputFormat.getTaskOutputPath(conf.value, getOutputName), - Reporter.NULL) - } - - protected def commit() { - if (committer.needsTaskCommit(taskContext)) { - try { - committer.commitTask(taskContext) - logInfo (taID + ": Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - committer.abortTask(taskContext) - throw e - } - } else { - logInfo("No need to commit output of task: " + taID.value) - } - } - - // ********* Private Functions ********* - - private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { - jobID = jobId - splitID = splitId - attemptID = attemptId - - jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) - } - - private def setConfParams() { - conf.value.set("mapred.job.id", jID.value.toString) - conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) - conf.value.set("mapred.task.id", taID.value.toString) - conf.value.setBoolean("mapred.task.is.map", true) - conf.value.setInt("mapred.task.partition", splitID) - } -} - -private[hive] object SparkHiveWriterContainer { - def createPathFromString(path: String, conf: JobConf): Path = { - if (path == null) { - throw new IllegalArgumentException("Output path is null") - } - val outputPath = new Path(path) - val fs = outputPath.getFileSystem(conf) - if (outputPath == null || fs == null) { - throw new IllegalArgumentException("Incorrectly formatted output path") - } - outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - } -} - -private[spark] class SparkHiveDynamicPartitionWriterContainer( - @transient jobConf: JobConf, - fileSinkConf: FileSinkDesc, - dynamicPartColNames: Array[String]) - extends SparkHiveWriterContainer(jobConf, fileSinkConf) { - - private val defaultPartName = jobConf.get( - ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultVal) - - @transient private var writers: mutable.HashMap[String, FileSinkOperator.RecordWriter] = _ - - override protected def initWriters(): Unit = { - // NOTE: This method is executed at the executor side. - // Actual writers are created for each dynamic partition on the fly. - writers = mutable.HashMap.empty[String, FileSinkOperator.RecordWriter] - } - - override def close(): Unit = { - writers.values.foreach(_.close(false)) - commit() - } - - override def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = { - val dynamicPartPath = dynamicPartColNames - .zip(row.takeRight(dynamicPartColNames.length)) - .map { case (col, rawVal) => - val string = String.valueOf(rawVal) - s"/$col=${if (rawVal == null || string.isEmpty) defaultPartName else string}" - } - .mkString - - def newWriter = { - val newFileSinkDesc = new FileSinkDesc( - fileSinkConf.getDirName + dynamicPartPath, - fileSinkConf.getTableInfo, - fileSinkConf.getCompressed) - newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec) - newFileSinkDesc.setCompressType(fileSinkConf.getCompressType) - - val path = { - val outputPath = FileOutputFormat.getOutputPath(conf.value) - assert(outputPath != null, "Undefined job output-path") - val workPath = new Path(outputPath, dynamicPartPath.stripPrefix("/")) - new Path(workPath, getOutputName) - } - - HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - newFileSinkDesc, - path, - Reporter.NULL) - } - - writers.getOrElseUpdate(dynamicPartPath, newWriter) - } -} diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 b/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 b/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f b/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a b/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e b/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf b/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 5d743a51b47c5..2da8a6fac3d99 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,9 +19,6 @@ package org.apache.spark.sql.hive.execution import scala.util.Try -import org.apache.hadoop.hive.conf.HiveConf.ConfVars - -import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -383,7 +380,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.contains("== Physical Plan ==") + explanation.exists(_ == "== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { @@ -571,91 +568,6 @@ class HiveQuerySuite extends HiveComparisonTest { case class LogEntry(filename: String, message: String) case class LogFile(name: String) - createQueryTest("dynamic_partition", - """ - |DROP TABLE IF EXISTS dynamic_part_table; - |CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT); - | - |SET hive.exec.dynamic.partition.mode=nonstrict; - | - |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) - |SELECT 1, 1, 1 FROM src WHERE key=150; - | - |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) - |SELECT 1, NULL, 1 FROM src WHERE key=150; - | - |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) - |SELECT 1, 1, NULL FROM src WHERE key=150; - | - |INSERT INTO TABLe dynamic_part_table PARTITION(partcol1, partcol2) - |SELECT 1, NULL, NULL FROM src WHERE key=150; - | - |DROP TABLE IF EXISTS dynamic_part_table; - """.stripMargin) - - test("Dynamic partition folder layout") { - sql("DROP TABLE IF EXISTS dynamic_part_table") - sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") - sql("SET hive.exec.dynamic.partition.mode=nonstrict") - - val data = Map( - Seq("1", "1") -> 1, - Seq("1", "NULL") -> 2, - Seq("NULL", "1") -> 3, - Seq("NULL", "NULL") -> 4) - - data.foreach { case (parts, value) => - sql( - s"""INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) - |SELECT $value, ${parts.mkString(", ")} FROM src WHERE key=150 - """.stripMargin) - - val partFolder = Seq("partcol1", "partcol2") - .zip(parts) - .map { case (k, v) => - if (v == "NULL") { - s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultVal}" - } else { - s"$k=$v" - } - } - .mkString("/") - - // Loads partition data to a temporary table to verify contents - val path = s"$warehousePath/dynamic_part_table/$partFolder/part-00000" - - sql("DROP TABLE IF EXISTS dp_verify") - sql("CREATE TABLE dp_verify(intcol INT)") - sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") - - assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) - } - } - - test("Partition spec validation") { - sql("DROP TABLE IF EXISTS dp_test") - sql("CREATE TABLE dp_test(key INT, value STRING) PARTITIONED BY (dp INT, sp INT)") - sql("SET hive.exec.dynamic.partition.mode=strict") - - // Should throw when using strict dynamic partition mode without any static partition - intercept[SparkException] { - sql( - """INSERT INTO TABLE dp_test PARTITION(dp) - |SELECT key, value, key % 5 FROM src - """.stripMargin) - } - - sql("SET hive.exec.dynamic.partition.mode=nonstrict") - - // Should throw when a static partition appears after a dynamic partition - intercept[SparkException] { - sql( - """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1) - |SELECT key, value, key % 5 FROM src - """.stripMargin) - } - } - test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") @@ -713,27 +625,27 @@ class HiveQuerySuite extends HiveComparisonTest { assert(sql("SET").collect().size == 0) assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey=$testVal")) + collectResults(hql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) assertResult(Set(testKey -> testVal)) { - collectResults(sql("SET")) + collectResults(hql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(sql("SET")) + collectResults(hql("SET")) } // "set key" assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey")) + collectResults(hql(s"SET $testKey")) } assertResult(Set(nonexistentKey -> "")) { - collectResults(sql(s"SET $nonexistentKey")) + collectResults(hql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as sql() by repeating the above using sql(). From 157e7d0f62eaf016a0c3749065ddcec170540a36 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 30 Sep 2014 09:46:58 -0700 Subject: [PATCH 0837/1492] HOTFIX: Ignore flaky tests in YARN --- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 857a4447dd738..4b6635679f053 100644 --- a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -86,13 +86,13 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { super.afterAll() } - test("run Spark in yarn-client mode") { + ignore("run Spark in yarn-client mode") { var result = File.createTempFile("result", null, tempDir) YarnClusterDriver.main(Array("yarn-client", result.getAbsolutePath())) checkResult(result) } - test("run Spark in yarn-cluster mode") { + ignore("run Spark in yarn-cluster mode") { val main = YarnClusterDriver.getClass.getName().stripSuffix("$") var result = File.createTempFile("result", null, tempDir) From ab6dd80ba0f7e1042ea270d10400109a467fe40e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 30 Sep 2014 11:15:38 -0700 Subject: [PATCH 0838/1492] [SPARK-3356] [DOCS] Document when RDD elements' ordering within partitions is nondeterministic As suggested by mateiz , and because it came up on the mailing list again last week, this attempts to document that ordering of elements is not guaranteed across RDD evaluations in groupBy, zip, and partition-wise RDD methods. Suggestions welcome about the wording, or other methods that need a note. Author: Sean Owen Closes #2508 from srowen/SPARK-3356 and squashes the following commits: b7c96fd [Sean Owen] Undo change to programming guide ad4aeec [Sean Owen] Don't mention ordering in partition-wise methods, reword description of ordering for zip methods per review, and add similar note to programming guide, which mentions groupByKey (but not zip methods) fce943b [Sean Owen] Note that ordering of elements is not guaranteed across RDD evaluations in groupBy, zip, and partition-wise RDD methods --- .../apache/spark/rdd/PairRDDFunctions.scala | 9 +++++++-- .../main/scala/org/apache/spark/rdd/RDD.scala | 20 ++++++++++++++++--- docs/programming-guide.md | 2 +- 3 files changed, 25 insertions(+), 6 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 67833743f3a98..929ded58a3bd5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -420,6 +420,8 @@ 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. + * The ordering of elements within each group is not guaranteed, and may even differ + * each time the resulting RDD is evaluated. * * 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]] @@ -439,7 +441,8 @@ 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. + * resulting RDD with into `numPartitions` partitions. The ordering of elements within + * each group is not guaranteed, and may even differ each time the resulting RDD is evaluated. * * 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]] @@ -535,7 +538,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. + * resulting RDD with the existing partitioner/parallelism level. The ordering of elements + * within each group is not guaranteed, and may even differ each time the resulting RDD is + * evaluated. * * 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]] 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 ba712c9d7776f..ab9e97c8fe409 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -509,7 +509,8 @@ 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. + * mapping to that key. The ordering of elements within each group is not guaranteed, and + * may even differ each time the resulting RDD is evaluated. * * 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]] @@ -520,7 +521,8 @@ 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. + * mapping to that key. The ordering of elements within each group is not guaranteed, and + * may even differ each time the resulting RDD is evaluated. * * 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]] @@ -531,7 +533,8 @@ 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. + * mapping to that key. The ordering of elements within each group is not guaranteed, and + * may even differ each time the resulting RDD is evaluated. * * 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]] @@ -1028,8 +1031,14 @@ abstract class RDD[T: ClassTag]( * Zips this RDD with its element indices. The ordering is first based on the partition index * and then the ordering of items within each partition. So the first item in the first * partition gets index 0, and the last item in the last partition receives the largest index. + * * This is similar to Scala's zipWithIndex but it uses Long instead of Int as the index type. * This method needs to trigger a spark job when this RDD contains more than one partitions. + * + * Note that some RDDs, such as those returned by groupBy(), do not guarantee order of + * elements in a partition. The index assigned to each element is therefore not guaranteed, + * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee + * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. */ def zipWithIndex(): RDD[(T, Long)] = new ZippedWithIndexRDD(this) @@ -1037,6 +1046,11 @@ abstract class RDD[T: ClassTag]( * Zips this RDD with generated unique Long ids. Items in the kth partition will get ids k, n+k, * 2*n+k, ..., where n is the number of partitions. So there may exist gaps, but this method * won't trigger a spark job, which is different from [[org.apache.spark.rdd.RDD#zipWithIndex]]. + * + * Note that some RDDs, such as those returned by groupBy(), do not guarantee order of + * elements in a partition. The unique ID assigned to each element is therefore not guaranteed, + * and may even change if the RDD is reevaluated. If a fixed ordering is required to guarantee + * the same index assignments, you should sort the RDD with sortByKey() or save it to a file. */ def zipWithUniqueId(): RDD[(T, Long)] = { val n = this.partitions.size.toLong diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 510b47a2aaad1..1d61a3c555eaf 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -883,7 +883,7 @@ for details.
    + + + + + + + + + diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index ccbca67656c8d..b8cdbbe3cf2b6 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -215,6 +215,21 @@ def addInPlace(self, value1, value2): COMPLEX_ACCUMULATOR_PARAM = AddingAccumulatorParam(0.0j) +class PStatsParam(AccumulatorParam): + """PStatsParam is used to merge pstats.Stats""" + + @staticmethod + def zero(value): + return None + + @staticmethod + def addInPlace(value1, value2): + if value1 is None: + return value2 + value1.add(value2) + return value1 + + class _UpdateRequestHandler(SocketServer.StreamRequestHandler): """ diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 8e7b00469e246..e9418320ff781 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -20,6 +20,7 @@ import sys from threading import Lock from tempfile import NamedTemporaryFile +import atexit from pyspark import accumulators from pyspark.accumulators import Accumulator @@ -30,7 +31,6 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, CompressedSerializer from pyspark.storagelevel import StorageLevel -from pyspark import rdd from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -192,6 +192,9 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self._temp_dir = \ self._jvm.org.apache.spark.util.Utils.createTempDir(local_dir).getAbsolutePath() + # profiling stats collected for each PythonRDD + self._profile_stats = [] + def _initialize_context(self, jconf): """ Initialize SparkContext in function to allow subclass specific initialization @@ -792,6 +795,40 @@ def runJob(self, rdd, partitionFunc, partitions=None, allowLocal=False): it = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, javaPartitions, allowLocal) return list(mappedRDD._collect_iterator_through_file(it)) + def _add_profile(self, id, profileAcc): + if not self._profile_stats: + dump_path = self._conf.get("spark.python.profile.dump") + if dump_path: + atexit.register(self.dump_profiles, dump_path) + else: + atexit.register(self.show_profiles) + + self._profile_stats.append([id, profileAcc, False]) + + def show_profiles(self): + """ Print the profile stats to stdout """ + for i, (id, acc, showed) in enumerate(self._profile_stats): + stats = acc.value + if not showed and stats: + print "=" * 60 + print "Profile of RDD" % id + print "=" * 60 + stats.sort_stats("time", "cumulative").print_stats() + # mark it as showed + self._profile_stats[i][2] = True + + def dump_profiles(self, path): + """ Dump the profile stats into directory `path` + """ + if not os.path.exists(path): + os.makedirs(path) + for id, acc, _ in self._profile_stats: + stats = acc.value + if stats: + p = os.path.join(path, "rdd_%d.pstats" % id) + stats.dump_stats(p) + self._profile_stats = [] + def _test(): import atexit diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 680140d72d03c..8ed89e2f9769f 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -15,7 +15,6 @@ # limitations under the License. # -from base64 import standard_b64encode as b64enc import copy from collections import defaultdict from itertools import chain, ifilter, imap @@ -32,6 +31,7 @@ from random import Random from math import sqrt, log, isinf, isnan +from pyspark.accumulators import PStatsParam from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer @@ -2080,7 +2080,9 @@ def _jrdd(self): return self._jrdd_val if self._bypass_serializer: self._jrdd_deserializer = NoOpSerializer() - command = (self.func, self._prev_jrdd_deserializer, + enable_profile = self.ctx._conf.get("spark.python.profile", "false") == "true" + profileStats = self.ctx.accumulator(None, PStatsParam) if enable_profile else None + command = (self.func, profileStats, self._prev_jrdd_deserializer, self._jrdd_deserializer) # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() @@ -2102,6 +2104,10 @@ def _jrdd(self): self.ctx.pythonExec, broadcast_vars, self.ctx._javaAccumulator) self._jrdd_val = python_rdd.asJavaRDD() + + if enable_profile: + self._id = self._jrdd_val.id() + self.ctx._add_profile(self._id, profileStats) return self._jrdd_val def id(self): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index f71d24c470dc9..d8bdf22355ec8 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -960,7 +960,7 @@ def registerFunction(self, name, f, returnType=StringType()): [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) - command = (func, + command = (func, None, BatchedSerializer(PickleSerializer(), 1024), BatchedSerializer(PickleSerializer(), 1024)) ser = CloudPickleSerializer() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 29df754c6fd29..7e2bbc9cb617f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -632,6 +632,36 @@ def test_distinct(self): self.assertEquals(result.count(), 3) +class TestProfiler(PySparkTestCase): + + def setUp(self): + self._old_sys_path = list(sys.path) + class_name = self.__class__.__name__ + conf = SparkConf().set("spark.python.profile", "true") + self.sc = SparkContext('local[4]', class_name, batchSize=2, conf=conf) + + def test_profiler(self): + + def heavy_foo(x): + for i in range(1 << 20): + x = 1 + rdd = self.sc.parallelize(range(100)) + rdd.foreach(heavy_foo) + profiles = self.sc._profile_stats + self.assertEqual(1, len(profiles)) + id, acc, _ = profiles[0] + stats = acc.value + self.assertTrue(stats is not None) + width, stat_list = stats.get_print_list([]) + func_names = [func_name for fname, n, func_name in stat_list] + self.assertTrue("heavy_foo" in func_names) + + self.sc.show_profiles() + d = tempfile.gettempdir() + self.sc.dump_profiles(d) + self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) + + class TestSQL(PySparkTestCase): def setUp(self): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index c1f6e3e4a1f40..8257dddfee1c3 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -23,6 +23,8 @@ import time import socket import traceback +import cProfile +import pstats from pyspark.accumulators import _accumulatorRegistry from pyspark.broadcast import Broadcast, _broadcastRegistry @@ -90,10 +92,21 @@ def main(infile, outfile): command = pickleSer._read_with_length(infile) if isinstance(command, Broadcast): command = pickleSer.loads(command.value) - (func, deserializer, serializer) = command + (func, stats, deserializer, serializer) = command init_time = time.time() - iterator = deserializer.load_stream(infile) - serializer.dump_stream(func(split_index, iterator), outfile) + + def process(): + iterator = deserializer.load_stream(infile) + serializer.dump_stream(func(split_index, iterator), outfile) + + if stats: + p = cProfile.Profile() + p.runcall(process) + st = pstats.Stats(p) + st.stream = None # make it picklable + stats.add(st.strip_dirs()) + else: + process() except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) From eb43043f411b87b7b412ee31e858246bd93fdd04 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 Oct 2014 00:29:14 -0700 Subject: [PATCH 0845/1492] [SPARK-3747] TaskResultGetter could incorrectly abort a stage if it cannot get result for a specific task Author: Reynold Xin Closes #2599 from rxin/SPARK-3747 and squashes the following commits: a74c04d [Reynold Xin] Added a line of comment explaining NonFatal 0e8d44c [Reynold Xin] [SPARK-3747] TaskResultGetter could incorrectly abort a stage if it cannot get result for a specific task --- .../org/apache/spark/scheduler/TaskResultGetter.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index df59f444b7a0e..3f345ceeaaf7a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -19,6 +19,8 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer +import scala.util.control.NonFatal + import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.serializer.SerializerInstance @@ -32,7 +34,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul private val THREADS = sparkEnv.conf.getInt("spark.resultGetter.threads", 4) private val getTaskResultExecutor = Utils.newDaemonFixedThreadPool( - THREADS, "Result resolver thread") + THREADS, "task-result-getter") protected val serializer = new ThreadLocal[SerializerInstance] { override def initialValue(): SerializerInstance = { @@ -70,7 +72,8 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul case cnf: ClassNotFoundException => val loader = Thread.currentThread.getContextClassLoader taskSetManager.abort("ClassNotFound with classloader: " + loader) - case ex: Exception => + // Matching NonFatal so we don't catch the ControlThrowable from the "return" above. + case NonFatal(ex) => logError("Exception while getting task result", ex) taskSetManager.abort("Exception while getting task result: %s".format(ex)) } From 7bf6cc9701cbb0f77fb85a412e387fb92274fca5 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 1 Oct 2014 01:03:24 -0700 Subject: [PATCH 0846/1492] [SPARK-3751] [mllib] DecisionTree: example update + print options DecisionTreeRunner functionality additions: * Allow user to pass in a test dataset * Do not print full model if the model is too large. As part of this, modify DecisionTreeModel and RandomForestModel to allow printing less info. Proposed updates: * toString: prints model summary * toDebugString: prints full model (named after RDD.toDebugString) Similar update to Python API: * __repr__() now prints a model summary * toDebugString() now prints the full model CC: mengxr chouqin manishamde codedeft Small update (whomever can take a look). Thanks! Author: Joseph K. Bradley Closes #2604 from jkbradley/dtrunner-update and squashes the following commits: b2b3c60 [Joseph K. Bradley] re-added python sql doc test, temporarily removed before 07b1fae [Joseph K. Bradley] repr() now prints a model summary toDebugString() now prints the full model 1d0d93d [Joseph K. Bradley] Updated DT and RF to print less when toString is called. Added toDebugString for verbose printing. 22eac8c [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update e007a95 [Joseph K. Bradley] Updated DecisionTreeRunner to accept a test dataset. --- .../examples/mllib/DecisionTreeRunner.scala | 99 ++++++++++++++----- .../mllib/tree/model/DecisionTreeModel.scala | 14 ++- .../mllib/tree/model/RandomForestModel.scala | 30 ++++-- python/pyspark/mllib/tree.py | 10 +- 4 files changed, 111 insertions(+), 42 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 96fb068e9e126..4adc91d2fbe65 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -52,6 +52,7 @@ object DecisionTreeRunner { case class Params( input: String = null, + testInput: String = "", dataFormat: String = "libsvm", algo: Algo = Classification, maxDepth: Int = 5, @@ -98,13 +99,18 @@ object DecisionTreeRunner { s"default: ${defaultParams.featureSubsetStrategy}") .action((x, c) => c.copy(featureSubsetStrategy = x)) opt[Double]("fracTest") - .text(s"fraction of data to hold out for testing, default: ${defaultParams.fracTest}") + .text(s"fraction of data to hold out for testing. If given option testInput, " + + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) + opt[String]("testInput") + .text(s"input path to test dataset. If given, option fracTest is ignored." + + s" default: ${defaultParams.testInput}") + .action((x, c) => c.copy(testInput = x)) opt[String]("") .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") .action((x, c) => c.copy(dataFormat = x)) arg[String]("") - .text("input paths to labeled examples in dense format (label,f0 f1 f2 ...)") + .text("input path to labeled examples") .required() .action((x, c) => c.copy(input = x)) checkConfig { params => @@ -141,7 +147,7 @@ object DecisionTreeRunner { case "libsvm" => MLUtils.loadLibSVMFile(sc, params.input).cache() } // For classification, re-index classes if needed. - val (examples, numClasses) = params.algo match { + val (examples, classIndexMap, numClasses) = params.algo match { case Classification => { // classCounts: class --> # examples in class val classCounts = origExamples.map(_.label).countByValue() @@ -170,16 +176,40 @@ object DecisionTreeRunner { val frac = classCounts(c) / numExamples.toDouble println(s"$c\t$frac\t${classCounts(c)}") } - (examples, numClasses) + (examples, classIndexMap, numClasses) } case Regression => - (origExamples, 0) + (origExamples, null, 0) case _ => throw new IllegalArgumentException("Algo ${params.algo} not supported.") } - // Split into training, test. - val splits = examples.randomSplit(Array(1.0 - params.fracTest, params.fracTest)) + // Create training, test sets. + val splits = if (params.testInput != "") { + // Load testInput. + val origTestExamples = params.dataFormat match { + case "dense" => MLUtils.loadLabeledPoints(sc, params.testInput) + case "libsvm" => MLUtils.loadLibSVMFile(sc, params.testInput) + } + params.algo match { + case Classification => { + // classCounts: class --> # examples in class + val testExamples = { + if (classIndexMap.isEmpty) { + origTestExamples + } else { + origTestExamples.map(lp => LabeledPoint(classIndexMap(lp.label), lp.features)) + } + } + Array(examples, testExamples) + } + case Regression => + Array(examples, origTestExamples) + } + } else { + // Split input into training, test. + examples.randomSplit(Array(1.0 - params.fracTest, params.fracTest)) + } val training = splits(0).cache() val test = splits(1).cache() val numTraining = training.count() @@ -206,47 +236,62 @@ object DecisionTreeRunner { minInfoGain = params.minInfoGain) if (params.numTrees == 1) { val model = DecisionTree.train(training, strategy) - println(model) + if (model.numNodes < 20) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } if (params.algo == Classification) { - val accuracy = + val trainAccuracy = + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) + .precision + println(s"Train accuracy = $trainAccuracy") + val testAccuracy = new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision - println(s"Test accuracy = $accuracy") + println(s"Test accuracy = $testAccuracy") } if (params.algo == Regression) { - val mse = meanSquaredError(model, test) - println(s"Test mean squared error = $mse") + val trainMSE = meanSquaredError(model, training) + println(s"Train mean squared error = $trainMSE") + val testMSE = meanSquaredError(model, test) + println(s"Test mean squared error = $testMSE") } } else { val randomSeed = Utils.random.nextInt() if (params.algo == Classification) { val model = RandomForest.trainClassifier(training, strategy, params.numTrees, params.featureSubsetStrategy, randomSeed) - println(model) - val accuracy = + if (model.totalNumNodes < 30) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } + val trainAccuracy = + new MulticlassMetrics(training.map(lp => (model.predict(lp.features), lp.label))) + .precision + println(s"Train accuracy = $trainAccuracy") + val testAccuracy = new MulticlassMetrics(test.map(lp => (model.predict(lp.features), lp.label))).precision - println(s"Test accuracy = $accuracy") + println(s"Test accuracy = $testAccuracy") } if (params.algo == Regression) { val model = RandomForest.trainRegressor(training, strategy, params.numTrees, params.featureSubsetStrategy, randomSeed) - println(model) - val mse = meanSquaredError(model, test) - println(s"Test mean squared error = $mse") + if (model.totalNumNodes < 30) { + println(model.toDebugString) // Print full model. + } else { + println(model) // Print model summary. + } + val trainMSE = meanSquaredError(model, training) + println(s"Train mean squared error = $trainMSE") + val testMSE = meanSquaredError(model, test) + println(s"Test mean squared error = $testMSE") } } sc.stop() } - /** - * Calculates the classifier accuracy. - */ - private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { - val correctCount = data.filter(y => model.predict(y.features) == y.label).count() - val count = data.count() - correctCount.toDouble / count - } - /** * Calculates the mean squared error for regression. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 271b2c4ad813e..ec1d99ab26f9c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -68,15 +68,23 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable } /** - * Print full model. + * Print a summary of the model. */ override def toString: String = algo match { case Classification => - s"DecisionTreeModel classifier\n" + topNode.subtreeToString(2) + s"DecisionTreeModel classifier of depth $depth with $numNodes nodes" case Regression => - s"DecisionTreeModel regressor\n" + topNode.subtreeToString(2) + s"DecisionTreeModel regressor of depth $depth with $numNodes nodes" case _ => throw new IllegalArgumentException( s"DecisionTreeModel given unknown algo parameter: $algo.") } + /** + * Print the full model to a string. + */ + def toDebugString: String = { + val header = toString + "\n" + header + topNode.subtreeToString(2) + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala index 538c0e233202a..4d66d6d81caa5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala @@ -73,17 +73,27 @@ class RandomForestModel(val trees: Array[DecisionTreeModel], val algo: Algo) ext def numTrees: Int = trees.size /** - * Print full model. + * Get total number of nodes, summed over all trees in the forest. */ - override def toString: String = { - val header = algo match { - case Classification => - s"RandomForestModel classifier with $numTrees trees\n" - case Regression => - s"RandomForestModel regressor with $numTrees trees\n" - case _ => throw new IllegalArgumentException( - s"RandomForestModel given unknown algo parameter: $algo.") - } + def totalNumNodes: Int = trees.map(tree => tree.numNodes).sum + + /** + * Print a summary of the model. + */ + override def toString: String = algo match { + case Classification => + s"RandomForestModel classifier with $numTrees trees" + case Regression => + s"RandomForestModel regressor with $numTrees trees" + case _ => throw new IllegalArgumentException( + s"RandomForestModel given unknown algo parameter: $algo.") + } + + /** + * Print the full model to a string. + */ + def toDebugString: String = { + val header = toString + "\n" header + trees.zipWithIndex.map { case (tree, treeIndex) => s" Tree $treeIndex:\n" + tree.topNode.subtreeToString(4) }.fold("")(_ + _) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index f59a818a6e74d..afdcdbdf3ae01 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -77,8 +77,13 @@ def depth(self): return self._java_model.depth() def __repr__(self): + """ Print summary of model. """ return self._java_model.toString() + def toDebugString(self): + """ Print full model. """ + return self._java_model.toDebugString() + class DecisionTree(object): @@ -135,7 +140,6 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, >>> from numpy import array >>> from pyspark.mllib.regression import LabeledPoint >>> from pyspark.mllib.tree import DecisionTree - >>> from pyspark.mllib.linalg import SparseVector >>> >>> data = [ ... LabeledPoint(0.0, [0.0]), @@ -145,7 +149,9 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, ... ] >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) >>> print model, # it already has newline - DecisionTreeModel classifier + DecisionTreeModel classifier of depth 1 with 3 nodes + >>> print model.toDebugString(), # it already has newline + DecisionTreeModel classifier of depth 1 with 3 nodes If (feature 0 <= 0.5) Predict: 0.0 Else (feature 0 > 0.5) From 3888ee2f3875f7053f63f70190670247e5c77383 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 1 Oct 2014 01:03:49 -0700 Subject: [PATCH 0847/1492] [SPARK-3748] Log thread name in unit test logs Thread names are useful for correlating failures. Author: Reynold Xin Closes #2600 from rxin/log4j and squashes the following commits: 83ffe88 [Reynold Xin] [SPARK-3748] Log thread name in unit test logs --- bagel/src/test/resources/log4j.properties | 2 +- core/src/test/resources/log4j.properties | 2 +- external/flume/src/test/resources/log4j.properties | 2 +- external/kafka/src/test/resources/log4j.properties | 2 +- external/mqtt/src/test/resources/log4j.properties | 2 +- external/twitter/src/test/resources/log4j.properties | 2 +- external/zeromq/src/test/resources/log4j.properties | 2 +- extras/java8-tests/src/test/resources/log4j.properties | 2 +- extras/kinesis-asl/src/test/resources/log4j.properties | 2 +- graphx/src/test/resources/log4j.properties | 2 +- mllib/src/test/resources/log4j.properties | 2 +- repl/src/test/resources/log4j.properties | 2 +- sql/core/src/test/resources/log4j.properties | 2 +- sql/hive/src/test/resources/log4j.properties | 2 +- streaming/src/test/resources/log4j.properties | 2 +- yarn/stable/src/test/resources/log4j.properties | 2 +- 16 files changed, 16 insertions(+), 16 deletions(-) diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 30b4baa4d714a..789869f72e3b0 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 26b73a1b39744..9dd05f17f012b 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties index 180beaa8cc5a7..bb0ab319a0080 100644 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ b/extras/java8-tests/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties index e01e049595475..d9d08f68687d3 100644 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -20,7 +20,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 26b73a1b39744..9dd05f17f012b 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index ddfc4ac6b23ed..a469badf603c6 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index 9c4896e49698c..52098993f5c3c 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index c7e0ff1cf6494..fbed0a782dd3e 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -30,7 +30,7 @@ log4j.appender.FA=org.apache.log4j.FileAppender log4j.appender.FA.append=false log4j.appender.FA.file=target/unit-tests.log log4j.appender.FA.layout=org.apache.log4j.PatternLayout -log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = INFO diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index c07d8fedf1993..9fdb526d945e0 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -30,7 +30,7 @@ log4j.appender.FA=org.apache.log4j.FileAppender log4j.appender.FA.append=false log4j.appender.FA.file=target/unit-tests.log log4j.appender.FA.layout=org.apache.log4j.PatternLayout -log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n # Set the logger level of File Appender to WARN log4j.appender.FA.Threshold = INFO diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 45d2ec676df66..4411d6e20c52a 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -22,7 +22,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN diff --git a/yarn/stable/src/test/resources/log4j.properties b/yarn/stable/src/test/resources/log4j.properties index 26b73a1b39744..9dd05f17f012b 100644 --- a/yarn/stable/src/test/resources/log4j.properties +++ b/yarn/stable/src/test/resources/log4j.properties @@ -21,7 +21,7 @@ log4j.appender.file=org.apache.log4j.FileAppender log4j.appender.file.append=false log4j.appender.file.file=target/unit-tests.log log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %p %c{1}: %m%n +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose log4j.logger.org.eclipse.jetty=WARN From 0bfd3afb00936b0f46ba613be0982e38bc7032b5 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Wed, 1 Oct 2014 08:55:04 -0700 Subject: [PATCH 0848/1492] [SPARK-3757] mvn clean doesn't delete some files Added directory to be deleted into maven-clean-plugin in pom.xml. Author: Masayoshi TSUZUKI Closes #2613 from tsudukim/feature/SPARK-3757 and squashes the following commits: 8804bfc [Masayoshi TSUZUKI] Modified indent. 67c7171 [Masayoshi TSUZUKI] [SPARK-3757] mvn clean doesn't delete some files --- core/pom.xml | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index e012c5e673b74..a5a178079bc57 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -322,6 +322,17 @@ + + maven-clean-plugin + + + + ${basedir}/../python/build + + + true + + org.apache.maven.plugins maven-shade-plugin From abf588f47a26d0066f0b75d52b200a87bb085064 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 1 Oct 2014 11:21:34 -0700 Subject: [PATCH 0849/1492] [SPARK-3749] [PySpark] fix bugs in broadcast large closure of RDD 1. broadcast is triggle unexpected 2. fd is leaked in JVM (also leak in parallelize()) 3. broadcast is not unpersisted in JVM after RDD is not be used any more. cc JoshRosen , sorry for these stupid bugs. Author: Davies Liu Closes #2603 from davies/fix_broadcast and squashes the following commits: 080a743 [Davies Liu] fix bugs in broadcast large closure of RDD --- .../apache/spark/api/python/PythonRDD.scala | 34 ++++++++++++------- python/pyspark/rdd.py | 12 +++++-- python/pyspark/sql.py | 2 +- python/pyspark/tests.py | 8 +++-- 4 files changed, 37 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index f9ff4ea6ca157..924141475383d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -339,26 +339,34 @@ private[spark] object PythonRDD extends Logging { def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) - val objs = new collection.mutable.ArrayBuffer[Array[Byte]] try { - while (true) { - val length = file.readInt() - val obj = new Array[Byte](length) - file.readFully(obj) - objs.append(obj) + val objs = new collection.mutable.ArrayBuffer[Array[Byte]] + try { + while (true) { + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + objs.append(obj) + } + } catch { + case eof: EOFException => {} } - } catch { - case eof: EOFException => {} + JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) + } finally { + file.close() } - JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) } def readBroadcastFromFile(sc: JavaSparkContext, filename: String): Broadcast[Array[Byte]] = { val file = new DataInputStream(new FileInputStream(filename)) - val length = file.readInt() - val obj = new Array[Byte](length) - file.readFully(obj) - sc.broadcast(obj) + try { + val length = file.readInt() + val obj = new Array[Byte](length) + file.readFully(obj) + sc.broadcast(obj) + } finally { + file.close() + } } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 8ed89e2f9769f..dc6497772e502 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2073,6 +2073,12 @@ def pipeline_func(split, iterator): self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self._partitionFunc = prev._partitionFunc if self.preservesPartitioning else None + self._broadcast = None + + def __del__(self): + if self._broadcast: + self._broadcast.unpersist() + self._broadcast = None @property def _jrdd(self): @@ -2087,9 +2093,9 @@ def _jrdd(self): # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() pickled_command = ser.dumps(command) - if pickled_command > (1 << 20): # 1M - broadcast = self.ctx.broadcast(pickled_command) - pickled_command = ser.dumps(broadcast) + if len(pickled_command) > (1 << 20): # 1M + self._broadcast = self.ctx.broadcast(pickled_command) + pickled_command = ser.dumps(self._broadcast) broadcast_vars = ListConverter().convert( [x._jbroadcast for x in self.ctx._pickled_broadcast_vars], self.ctx._gateway._gateway_client) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d8bdf22355ec8..974b5e287bc00 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -965,7 +965,7 @@ def registerFunction(self, name, f, returnType=StringType()): BatchedSerializer(PickleSerializer(), 1024)) ser = CloudPickleSerializer() pickled_command = ser.dumps(command) - if pickled_command > (1 << 20): # 1M + if len(pickled_command) > (1 << 20): # 1M broadcast = self._sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) broadcast_vars = ListConverter().convert( diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 7e2bbc9cb617f..6fb6bc998c752 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -467,8 +467,12 @@ def test_large_broadcast(self): def test_large_closure(self): N = 1000000 data = [float(i) for i in xrange(N)] - m = self.sc.parallelize(range(1), 1).map(lambda x: len(data)).sum() - self.assertEquals(N, m) + rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) + self.assertEquals(N, rdd.first()) + self.assertTrue(rdd._broadcast is not None) + rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1) + self.assertEqual(1, rdd.first()) + self.assertTrue(rdd._broadcast is None) def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) From dcb2f73f1cf1f6efd5175267e135ad6cf4bf6e3d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 1 Oct 2014 11:28:22 -0700 Subject: [PATCH 0850/1492] SPARK-2626 [DOCS] Stop SparkContext in all examples Call SparkContext.stop() in all examples (and touch up minor nearby code style issues while at it) Author: Sean Owen Closes #2575 from srowen/SPARK-2626 and squashes the following commits: 5b2baae [Sean Owen] Call SparkContext.stop() in all examples (and touch up minor nearby code style issues while at it) --- .../main/java/org/apache/spark/examples/JavaSparkPi.java | 3 ++- .../java/org/apache/spark/examples/sql/JavaSparkSQL.java | 9 ++++++++- examples/src/main/python/avro_inputformat.py | 2 ++ examples/src/main/python/parquet_inputformat.py | 2 ++ .../org/apache/spark/examples/CassandraCQLTest.scala | 2 ++ .../scala/org/apache/spark/examples/CassandraTest.scala | 2 ++ .../scala/org/apache/spark/examples/GroupByTest.scala | 6 +++--- .../main/scala/org/apache/spark/examples/LogQuery.scala | 2 ++ .../apache/spark/examples/bagel/WikipediaPageRank.scala | 9 +++++---- .../org/apache/spark/examples/sql/RDDRelation.scala | 4 +++- .../apache/spark/examples/sql/hive/HiveFromSpark.scala | 4 +++- 11 files changed, 34 insertions(+), 11 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java index 11157d7573fae..0f07cb4098325 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaSparkPi.java @@ -31,7 +31,6 @@ * Usage: JavaSparkPi [slices] */ public final class JavaSparkPi { - public static void main(String[] args) throws Exception { SparkConf sparkConf = new SparkConf().setAppName("JavaSparkPi"); @@ -61,5 +60,7 @@ public Integer call(Integer integer, Integer integer2) { }); System.out.println("Pi is roughly " + 4.0 * count / n); + + jsc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index 898297dc658ba..01c77bd44337e 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -61,7 +61,8 @@ public static void main(String[] args) throws Exception { // Load a text file and convert each line to a Java Bean. JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( new Function() { - public Person call(String line) throws Exception { + @Override + public Person call(String line) { String[] parts = line.split(","); Person person = new Person(); @@ -82,6 +83,7 @@ public Person call(String line) throws Exception { // The results of SQL queries are SchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. List teenagerNames = teenagers.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0); } @@ -104,6 +106,7 @@ public String call(Row row) { JavaSchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0); } @@ -136,6 +139,7 @@ public String call(Row row) { // The results of SQL queries are JavaSchemaRDDs and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. teenagerNames = teenagers3.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0); } }).collect(); for (String name: teenagerNames) { @@ -162,6 +166,7 @@ public String call(Row row) { JavaSchemaRDD peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.map(new Function() { + @Override public String call(Row row) { return "Name: " + row.getString(0) + ", City: " + row.getString(1); } @@ -169,5 +174,7 @@ public String call(Row row) { for (String name: nameAndCity) { System.out.println(name); } + + ctx.stop(); } } diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index cfda8d8327aa3..4626bbb7e3b02 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -78,3 +78,5 @@ output = avro_rdd.map(lambda x: x[0]).collect() for k in output: print k + + sc.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index c9b08f878a1e6..fa4c20ab20281 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -57,3 +57,5 @@ output = parquet_rdd.map(lambda x: x[1]).collect() for k in output: print k + + sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 71f53af68f4d3..11d5c92c5952d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -136,5 +136,7 @@ object CassandraCQLTest { classOf[CqlOutputFormat], job.getConfiguration() ) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala index 91ba364a346a5..ec689474aecb0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraTest.scala @@ -126,6 +126,8 @@ object CassandraTest { } }.saveAsNewAPIHadoopFile("casDemo", classOf[ByteBuffer], classOf[List[Mutation]], classOf[ColumnFamilyOutputFormat], job.getConfiguration) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index efd91bb054981..15f6678648b29 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -44,11 +44,11 @@ object GroupByTest { arr1(i) = (ranGen.nextInt(Int.MaxValue), byteArr) } arr1 - }.cache + }.cache() // Enforce that everything has been calculated and in cache - pairs1.count + pairs1.count() - println(pairs1.groupByKey(numReducers).count) + println(pairs1.groupByKey(numReducers).count()) sc.stop() } diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 4c655b84fde2e..74620ad007d83 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -79,5 +79,7 @@ object LogQuery { .reduceByKey((a, b) => a.merge(b)) .collect().foreach{ case (user, query) => println("%s\t%s".format(user, query))} + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 235c3bf820244..e4db3ec51313d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -21,7 +21,6 @@ import org.apache.spark._ import org.apache.spark.SparkContext._ import org.apache.spark.bagel._ -import org.apache.spark.bagel.Bagel._ import scala.xml.{XML,NodeSeq} @@ -78,9 +77,9 @@ object WikipediaPageRank { (id, new PRVertex(1.0 / numVertices, outEdges)) }) if (usePartitioner) { - vertices = vertices.partitionBy(new HashPartitioner(sc.defaultParallelism)).cache + vertices = vertices.partitionBy(new HashPartitioner(sc.defaultParallelism)).cache() } else { - vertices = vertices.cache + vertices = vertices.cache() } println("Done parsing input file.") @@ -100,7 +99,9 @@ object WikipediaPageRank { (result .filter { case (id, vertex) => vertex.value >= threshold } .map { case (id, vertex) => "%s\t%s\n".format(id, vertex.value) } - .collect.mkString) + .collect().mkString) println(top) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala index d56d64c564200..2e98b2dc30b80 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala @@ -51,7 +51,7 @@ object RDDRelation { val rddFromSql = sql("SELECT key, value FROM records WHERE key < 10") println("Result of RDD.map:") - rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect.foreach(println) + rddFromSql.map(row => s"Key: ${row(0)}, Value: ${row(1)}").collect().foreach(println) // Queries can also be written using a LINQ-like Scala DSL. rdd.where('key === 1).orderBy('value.asc).select('key).collect().foreach(println) @@ -68,5 +68,7 @@ object RDDRelation { // These files can also be registered as tables. parquetFile.registerTempTable("parquetFile") sql("SELECT * FROM parquetFile").collect().foreach(println) + + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 3423fac0ad303..e26f213e8afa8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -39,7 +39,7 @@ object HiveFromSpark { // Queries are expressed in HiveQL println("Result of 'SELECT *': ") - sql("SELECT * FROM src").collect.foreach(println) + sql("SELECT * FROM src").collect().foreach(println) // Aggregation queries are also supported. val count = sql("SELECT COUNT(*) FROM src").collect().head.getLong(0) @@ -61,5 +61,7 @@ object HiveFromSpark { // Queries can then join RDD data with data stored in Hive. println("Result of SELECT *:") sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) + + sc.stop() } } From 6390aae4eacbabfb1c53fb828b824f6a6518beff Mon Sep 17 00:00:00 2001 From: scwf Date: Wed, 1 Oct 2014 11:30:29 -0700 Subject: [PATCH 0851/1492] [SPARK-3755][Core] Do not bind port 1 - 1024 to server in spark Non-root user use port 1- 1024 to start jetty server will get the exception " java.net.SocketException: Permission denied", so not use these ports Author: scwf Closes #2610 from scwf/1-1024 and squashes the following commits: cb8cc76 [scwf] do not use port 1 - 1024 --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 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 dbe0cfa2b8ff9..1d80012dc142d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1437,7 +1437,7 @@ private[spark] object Utils extends Logging { val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" for (offset <- 0 to maxRetries) { // Do not increment port if startPort is 0, which is treated as a special port - val tryPort = if (startPort == 0) startPort else (startPort + offset) % 65536 + val tryPort = if (startPort == 0) startPort else (startPort + offset) % (65536 - 1024) + 1024 try { val (service, port) = startService(tryPort) logInfo(s"Successfully started service$serviceString on port $port.") From 2fedb5dddcc10d3186f49fc4996a7bb5b68bbc85 Mon Sep 17 00:00:00 2001 From: scwf Date: Wed, 1 Oct 2014 11:51:30 -0700 Subject: [PATCH 0852/1492] [SPARK-3756] [Core]check exception is caused by an address-port collision properly Jetty server use MultiException to handle exceptions when start server refer https://github.com/eclipse/jetty.project/blob/jetty-8.1.14.v20131031/jetty-server/src/main/java/org/eclipse/jetty/server/Server.java So in ```isBindCollision``` add the logical to cover MultiException Author: scwf Closes #2611 from scwf/fix-isBindCollision and squashes the following commits: 984cb12 [scwf] optimize the fix 3a6c849 [scwf] fix bug in isBindCollision --- core/src/main/scala/org/apache/spark/util/Utils.scala | 3 +++ 1 file changed, 3 insertions(+) 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 1d80012dc142d..e5b83c069d961 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -23,6 +23,8 @@ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import org.eclipse.jetty.util.MultiException + import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -1470,6 +1472,7 @@ private[spark] object Utils extends Logging { return true } isBindCollision(e.getCause) + case e: MultiException => e.getThrowables.exists(isBindCollision) case e: Exception => isBindCollision(e.getCause) case _ => false } From 8cc70e7e15fd800f31b94e9102069506360289db Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 1 Oct 2014 12:40:37 -0700 Subject: [PATCH 0853/1492] [SQL] Kill dangerous trailing space in query string MD5 of query strings in `createQueryTest` calls are used to generate golden files, leaving trailing spaces there can be really dangerous. Got bitten by this while working on #2616: my "smart" IDE automatically removed a trailing space and makes Jenkins fail. (Really should add "no trailing space" to our coding style guidelines!) Author: Cheng Lian Closes #2619 from liancheng/kill-trailing-space and squashes the following commits: 034f119 [Cheng Lian] Kill dangerous trailing space in query string --- ...tamp to Timestamp in UDF-0-db6d4503454e4dbb9edcbab9a8718d7f} | 0 .../org/apache/spark/sql/hive/execution/HiveQuerySuite.scala | 2 +- 2 files changed, 1 insertion(+), 1 deletion(-) rename sql/hive/src/test/resources/golden/{Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa => Cast Timestamp to Timestamp in UDF-0-db6d4503454e4dbb9edcbab9a8718d7f} (100%) diff --git a/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa b/sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-db6d4503454e4dbb9edcbab9a8718d7f similarity index 100% rename from sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-66952a3949d7544716fd1a675498b1fa rename to sql/hive/src/test/resources/golden/Cast Timestamp to Timestamp in UDF-0-db6d4503454e4dbb9edcbab9a8718d7f diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2da8a6fac3d99..f5868bff22f13 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -164,7 +164,7 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("Cast Timestamp to Timestamp in UDF", """ - | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) + | SELECT DATEDIFF(CAST(value AS timestamp), CAST('2002-03-21 00:00:00' AS timestamp)) | FROM src LIMIT 1 """.stripMargin) From b81ee0b46d63c2122b88941696654100fd736942 Mon Sep 17 00:00:00 2001 From: Gaspar Munoz Date: Wed, 1 Oct 2014 13:47:22 -0700 Subject: [PATCH 0854/1492] Typo error in KafkaWordCount example topicpMap to topicMap Author: Gaspar Munoz Closes #2614 from gasparms/patch-1 and squashes the following commits: 00aab2c [Gaspar Munoz] Typo error in KafkaWordCount example --- .../org/apache/spark/examples/streaming/KafkaWordCount.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 566ba6f911e02..c9e1511278ede 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -53,8 +53,8 @@ object KafkaWordCount { val ssc = new StreamingContext(sparkConf, Seconds(2)) ssc.checkpoint("checkpoint") - val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap - val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2) + val topicMap = topics.split(",").map((_,numThreads.toInt)).toMap + val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicMap).map(_._2) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1L)) .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2) From 17333c7a3c26ca6d28e8f3ca097da37d6b655217 Mon Sep 17 00:00:00 2001 From: jyotiska Date: Wed, 1 Oct 2014 13:52:50 -0700 Subject: [PATCH 0855/1492] Python SQL Example Code SQL example code for Python, as shown on [SQL Programming Guide](https://spark.apache.org/docs/1.0.2/sql-programming-guide.html) Author: jyotiska Closes #2521 from jyotiska/sql_example and squashes the following commits: 1471dcb [jyotiska] added imports for sql b25e436 [jyotiska] pep 8 compliance 43fd10a [jyotiska] lines broken to maintain 80 char limit b4fdf4e [jyotiska] removed blank lines 83d5ab7 [jyotiska] added inferschema and applyschema to the demo 306667e [jyotiska] replaced blank line with end line c90502a [jyotiska] fixed new line 4939a70 [jyotiska] added new line at end for python style 0b46148 [jyotiska] fixed appname for python sql example 8f67b5b [jyotiska] added python sql example --- examples/src/main/python/sql.py | 73 +++++++++++++++++++++++++++++++++ 1 file changed, 73 insertions(+) create mode 100644 examples/src/main/python/sql.py diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py new file mode 100644 index 0000000000000..eefa022f1927c --- /dev/null +++ b/examples/src/main/python/sql.py @@ -0,0 +1,73 @@ +# +# 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. +# + +import os + +from pyspark import SparkContext +from pyspark.sql import SQLContext +from pyspark.sql import Row, StructField, StructType, StringType, IntegerType + + +if __name__ == "__main__": + sc = SparkContext(appName="PythonSQL") + sqlContext = SQLContext(sc) + + # RDD is created from a list of rows + some_rdd = sc.parallelize([Row(name="John", age=19), + Row(name="Smith", age=23), + Row(name="Sarah", age=18)]) + # Infer schema from the first row, create a SchemaRDD and print the schema + some_schemardd = sqlContext.inferSchema(some_rdd) + some_schemardd.printSchema() + + # Another RDD is created from a list of tuples + another_rdd = sc.parallelize([("John", 19), ("Smith", 23), ("Sarah", 18)]) + # Schema with two fields - person_name and person_age + schema = StructType([StructField("person_name", StringType(), False), + StructField("person_age", IntegerType(), False)]) + # Create a SchemaRDD by applying the schema to the RDD and print the schema + another_schemardd = sqlContext.applySchema(another_rdd, schema) + another_schemardd.printSchema() + # root + # |-- age: integer (nullable = true) + # |-- name: string (nullable = true) + + # A JSON dataset is pointed to by path. + # The path can be either a single text file or a directory storing text files. + path = os.environ['SPARK_HOME'] + "examples/src/main/resources/people.json" + # Create a SchemaRDD from the file(s) pointed to by path + people = sqlContext.jsonFile(path) + # root + # |-- person_name: string (nullable = false) + # |-- person_age: integer (nullable = false) + + # The inferred schema can be visualized using the printSchema() method. + people.printSchema() + # root + # |-- age: IntegerType + # |-- name: StringType + + # Register this SchemaRDD as a table. + people.registerAsTable("people") + + # SQL statements can be run by using the sql methods provided by sqlContext + teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + + for each in teenagers.collect(): + print each[0] + + sc.stop() From fcad3fae6135bc2c9bdaf9e4c3cfe24838e63eae Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Oct 2014 14:37:27 -0700 Subject: [PATCH 0856/1492] [SPARK-3746][SQL] Lock hive client when creating tables Author: Michael Armbrust Closes #2598 from marmbrus/hiveClientLock and squashes the following commits: ca89fe8 [Michael Armbrust] Lock hive client when creating tables --- .../apache/spark/sql/hive/HiveMetastoreCatalog.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 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 9a0b9b46ac4ee..06b1446ccbd39 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 @@ -96,10 +96,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with serDeInfo.setParameters(Map[String, String]()) sd.setSerdeInfo(serDeInfo) - try client.createTable(table) catch { - case e: org.apache.hadoop.hive.ql.metadata.HiveException - if e.getCause.isInstanceOf[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] && - allowExisting => // Do nothing. + synchronized { + try client.createTable(table) catch { + case e: org.apache.hadoop.hive.ql.metadata.HiveException + if e.getCause.isInstanceOf[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] && + allowExisting => // Do nothing. + } } } From d61f2c15bb22253bfdda77462b1bd383987d2f5a Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Wed, 1 Oct 2014 15:15:09 -0700 Subject: [PATCH 0857/1492] [SPARK-3658][SQL] Start thrift server as a daemon https://issues.apache.org/jira/browse/SPARK-3658 And keep the `CLASS_NOT_FOUND_EXIT_STATUS` and exit message in `SparkSubmit.scala`. Author: WangTaoTheTonic Author: WangTao Closes #2509 from WangTaoTheTonic/thriftserver and squashes the following commits: 5dcaab2 [WangTaoTheTonic] issue about coupling 8ad9f95 [WangTaoTheTonic] generalization 598e21e [WangTao] take thrift server as a daemon --- bin/spark-sql | 12 +-------- .../org/apache/spark/deploy/SparkSubmit.scala | 4 +++ sbin/spark-daemon.sh | 16 ++++++++---- sbin/start-thriftserver.sh | 16 ++---------- sbin/stop-thriftserver.sh | 25 +++++++++++++++++++ 5 files changed, 43 insertions(+), 30 deletions(-) create mode 100755 sbin/stop-thriftserver.sh diff --git a/bin/spark-sql b/bin/spark-sql index 9d66140b6aa17..63d00437d508d 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,7 +24,6 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" -CLASS_NOT_FOUND_EXIT_STATUS=101 # Figure out where Spark is installed FWDIR="$(cd "`dirname "$0"`"/..; pwd)" @@ -53,13 +52,4 @@ source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" -"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" -exit_status=$? - -if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then - echo - echo "Failed to load Spark SQL CLI main class $CLASS." - echo "You need to build Spark with -Phive." -fi - -exit $exit_status +exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 580a439c9a892..f97bf67fa5a3b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -320,6 +320,10 @@ object SparkSubmit { } catch { case e: ClassNotFoundException => e.printStackTrace(printStream) + if (childMainClass.contains("thriftserver")) { + println(s"Failed to load main class $childMainClass.") + println("You need to build Spark with -Phive.") + } System.exit(CLASS_NOT_FOUND_EXIT_STATUS) } diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index bd476b400e1c3..cba475e2dd8c8 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -62,7 +62,7 @@ then shift fi -startStop=$1 +option=$1 shift command=$1 shift @@ -122,9 +122,9 @@ if [ "$SPARK_NICENESS" = "" ]; then fi -case $startStop in +case $option in - (start) + (start|spark-submit) mkdir -p "$SPARK_PID_DIR" @@ -142,8 +142,14 @@ case $startStop in spark_rotate_log "$log" echo starting $command, logging to $log - cd "$SPARK_PREFIX" - nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + if [ $option == spark-submit ]; then + source "$SPARK_HOME"/bin/utils.sh + gatherSparkSubmitOpts "$@" + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-submit --class $command \ + "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" >> "$log" 2>&1 < /dev/null & + else + nohup nice -n $SPARK_NICENESS "$SPARK_PREFIX"/bin/spark-class $command "$@" >> "$log" 2>&1 < /dev/null & + fi newpid=$! echo $newpid > $pid sleep 2 diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index ba953e763faab..50e8e06418b07 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,7 +27,6 @@ set -o posix FWDIR="$(cd "`dirname "$0"`"/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" -CLASS_NOT_FOUND_EXIT_STATUS=101 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" @@ -49,17 +48,6 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then exit 0 fi -source "$FWDIR"/bin/utils.sh -SUBMIT_USAGE_FUNCTION=usage -gatherSparkSubmitOpts "$@" +export SUBMIT_USAGE_FUNCTION=usage -"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" -exit_status=$? - -if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then - echo - echo "Failed to load Hive Thrift server main class $CLASS." - echo "You need to build Spark with -Phive." -fi - -exit $exit_status +exec "$FWDIR"/sbin/spark-daemon.sh spark-submit $CLASS 1 "$@" diff --git a/sbin/stop-thriftserver.sh b/sbin/stop-thriftserver.sh new file mode 100755 index 0000000000000..4031a00d4a689 --- /dev/null +++ b/sbin/stop-thriftserver.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# Stops the thrift server on the machine this script is executed on. + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +"$sbin"/spark-daemon.sh stop org.apache.spark.sql.hive.thriftserver.HiveThriftServer2 1 From 3508ce8a5a05d6cb122ad59ba33c3cc18e17e2a4 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Wed, 1 Oct 2014 15:44:41 -0700 Subject: [PATCH 0858/1492] [SPARK-3708][SQL] Backticks aren't handled correctly is aliases The below query gives error sql("SELECT k FROM (SELECT \`key\` AS \`k\` FROM src) a") It gives error because the aliases are not cleaned so it could not be resolved in further processing. Author: ravipesala Closes #2594 from ravipesala/SPARK-3708 and squashes the following commits: d55db54 [ravipesala] Fixed SPARK-3708 (Backticks aren't handled correctly is aliases) --- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../org/apache/spark/sql/hive/execution/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) 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 0aa6292c0184e..4f3f808c93dc8 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 @@ -855,7 +855,7 @@ private[hive] object HiveQl { case Token("TOK_SELEXPR", e :: Token(alias, Nil) :: Nil) => - Some(Alias(nodeToExpr(e), alias)()) + Some(Alias(nodeToExpr(e), cleanIdentifier(alias))()) /* Hints are ignored */ case Token("TOK_HINTLIST", _) => None diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 679efe082f2a0..3647bb1c4ce7d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -63,4 +63,10 @@ class SQLQuerySuite extends QueryTest { sql("SELECT key, value FROM test_ctas_123 ORDER BY key"), sql("SELECT key, value FROM src ORDER BY key").collect().toSeq) } + + test("SPARK-3708 Backticks aren't handled correctly is aliases") { + checkAnswer( + sql("SELECT k FROM (SELECT `key` AS `k` FROM src) a"), + sql("SELECT `key` FROM src").collect().toSeq) + } } From f315fb7efc95afb2cc1208159b48359ba56a010d Mon Sep 17 00:00:00 2001 From: scwf Date: Wed, 1 Oct 2014 15:55:09 -0700 Subject: [PATCH 0859/1492] [SPARK-3705][SQL] Add case for VoidObjectInspector to cover NullType add case for VoidObjectInspector in ```inspectorToDataType``` Author: scwf Closes #2552 from scwf/inspectorToDataType and squashes the following commits: 453d892 [scwf] add case for VoidObjectInspector --- .../main/scala/org/apache/spark/sql/hive/HiveInspectors.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index fa889ec104c6e..d633c42c6bd67 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -213,6 +213,8 @@ private[hive] trait HiveInspectors { case _: JavaHiveDecimalObjectInspector => DecimalType case _: WritableTimestampObjectInspector => TimestampType case _: JavaTimestampObjectInspector => TimestampType + case _: WritableVoidObjectInspector => NullType + case _: JavaVoidObjectInspector => NullType } implicit class typeInfoConversions(dt: DataType) { From f84b228c4002073ee4ff53be50462a63f48bd508 Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Wed, 1 Oct 2014 15:57:06 -0700 Subject: [PATCH 0860/1492] [SPARK-3593][SQL] Add support for sorting BinaryType BinaryType is derived from NativeType and added Ordering support. Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #2617 from gvramana/binarytype_sort and squashes the following commits: 1cf26f3 [Venkata Ramana Gollamudi] Supported Sorting of BinaryType --- .../apache/spark/sql/catalyst/types/dataTypes.scala | 12 +++++++++++- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 8 ++++++++ .../test/scala/org/apache/spark/sql/TestData.scala | 10 ++++++++++ 3 files changed, 29 insertions(+), 1 deletion(-) 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 c7d73d3990c3a..ac043d4dd8eb9 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 @@ -157,8 +157,18 @@ case object StringType extends NativeType with PrimitiveType { def simpleString: String = "string" } -case object BinaryType extends DataType with PrimitiveType { +case object BinaryType extends NativeType with PrimitiveType { private[sql] type JvmType = Array[Byte] + @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } + private[sql] val ordering = new Ordering[JvmType] { + def compare(x: Array[Byte], y: Array[Byte]): Int = { + for (i <- 0 until x.length; if i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + } + return x.length - y.length + } + } def simpleString: String = "binary" } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 08376eb5e5c4e..fdf3a229a796e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -190,6 +190,14 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"), Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + checkAnswer( + sql("SELECT b FROM binaryData ORDER BY a ASC"), + (1 to 5).map(Row(_)).toSeq) + + checkAnswer( + sql("SELECT b FROM binaryData ORDER BY a DESC"), + (1 to 5).map(Row(_)).toSeq.reverse) + checkAnswer( sql("SELECT * FROM arrayData ORDER BY data[0] ASC"), arrayData.collect().sortBy(_.data(0)).toSeq) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index eb33a61c6e811..10b7979df7375 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -54,6 +54,16 @@ object TestData { TestData2(3, 2) :: Nil) testData2.registerTempTable("testData2") + case class BinaryData(a: Array[Byte], b: Int) + val binaryData: SchemaRDD = + TestSQLContext.sparkContext.parallelize( + BinaryData("12".getBytes(), 1) :: + BinaryData("22".getBytes(), 5) :: + BinaryData("122".getBytes(), 3) :: + BinaryData("121".getBytes(), 2) :: + BinaryData("123".getBytes(), 4) :: Nil) + binaryData.registerTempTable("binaryData") + // TODO: There is no way to express null primitives as case classes currently... val testData3 = logical.LocalRelation('a.int, 'b.int).loadData( From a31f4ff22f98c01f0d9b7d1240080ff2689c1270 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 1 Oct 2014 16:00:29 -0700 Subject: [PATCH 0861/1492] [SQL] Made Command.sideEffectResult protected Considering `Command.executeCollect()` simply delegates to `Command.sideEffectResult`, we no longer need to leave the latter `protected[sql]`. Author: Cheng Lian Closes #2431 from liancheng/narrow-scope and squashes the following commits: 1bfc16a [Cheng Lian] Made Command.sideEffectResult protected --- .../apache/spark/sql/execution/commands.scala | 10 +++++----- .../org/apache/spark/sql/hive/HiveContext.scala | 2 +- .../sql/hive/execution/CreateTableAsSelect.scala | 16 ++++++++-------- .../execution/DescribeHiveTableCommand.scala | 2 +- .../spark/sql/hive/execution/NativeCommand.scala | 2 +- .../spark/sql/hive/execution/commands.scala | 6 +++--- 6 files changed, 19 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index c2f48a902a3e9..f88099ec0761e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -37,7 +37,7 @@ trait Command { * The `execute()` method of all the physical command classes should reference `sideEffectResult` * so that the command can be executed eagerly right after the command query is created. */ - protected[sql] lazy val sideEffectResult: Seq[Row] = Seq.empty[Row] + protected lazy val sideEffectResult: Seq[Row] = Seq.empty[Row] override def executeCollect(): Array[Row] = sideEffectResult.toArray @@ -53,7 +53,7 @@ case class SetCommand( @transient context: SQLContext) extends LeafNode with Command with Logging { - override protected[sql] lazy val sideEffectResult: Seq[Row] = (key, value) match { + override protected lazy val sideEffectResult: Seq[Row] = (key, value) match { // Set value for key k. case (Some(k), Some(v)) => if (k == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { @@ -121,7 +121,7 @@ case class ExplainCommand( extends LeafNode with Command { // Run through the optimizer to generate the physical plan. - override protected[sql] lazy val sideEffectResult: Seq[Row] = try { + override protected lazy val sideEffectResult: Seq[Row] = try { // TODO in Hive, the "extended" ExplainCommand prints the AST as well, and detailed properties. val queryExecution = context.executePlan(logicalPlan) val outputString = if (extended) queryExecution.toString else queryExecution.simpleString @@ -141,7 +141,7 @@ case class ExplainCommand( case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: SQLContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult = { + override protected lazy val sideEffectResult = { if (doCache) { context.cacheTable(tableName) } else { @@ -161,7 +161,7 @@ case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( @transient context: SQLContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { Row("# Registered as a temporary table", null, null) +: child.output.map(field => Row(field.name, field.dataType.toString, null)) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 3e1a7b71528e0..20ebe4996c207 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -404,7 +404,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { // be similar with Hive. describeHiveTableCommand.hiveString case command: PhysicalCommand => - command.sideEffectResult.map(_.head.toString) + command.executeCollect().map(_.head.toString) case other => val result: Seq[Seq[Any]] = toRdd.collect().toSeq diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 1017fe6d5396d..3625708d03175 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -30,23 +30,23 @@ import org.apache.spark.sql.hive.MetastoreRelation * Create table and insert the query result into it. * @param database the database name of the new relation * @param tableName the table name of the new relation - * @param insertIntoRelation function of creating the `InsertIntoHiveTable` + * @param insertIntoRelation function of creating the `InsertIntoHiveTable` * by specifying the `MetaStoreRelation`, the data will be inserted into that table. * TODO Add more table creating properties, e.g. SerDe, StorageHandler, in-memory cache etc. */ @Experimental case class CreateTableAsSelect( - database: String, - tableName: String, - query: SparkPlan, - insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) - extends LeafNode with Command { + database: String, + tableName: String, + query: SparkPlan, + insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) + extends LeafNode with Command { def output = Seq.empty // A lazy computing of the metastoreRelation private[this] lazy val metastoreRelation: MetastoreRelation = { - // Create the table + // Create the table val sc = sqlContext.asInstanceOf[HiveContext] sc.catalog.createTable(database, tableName, query.output, false) // Get the Metastore Relation @@ -55,7 +55,7 @@ case class CreateTableAsSelect( } } - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { insertIntoRelation(metastoreRelation).execute Seq.empty[Row] } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index 317801001c7a4..106cede9788ec 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -48,7 +48,7 @@ case class DescribeHiveTableCommand( .mkString("\t") } - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { // Trying to mimic the format of Hive's output. But not exactly the same. var results: Seq[(String, String, String)] = Nil diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala index 8f10e1ba7f426..6930c2babd117 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/NativeCommand.scala @@ -32,7 +32,7 @@ case class NativeCommand( @transient context: HiveContext) extends LeafNode with Command { - override protected[sql] lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_)) + override protected lazy val sideEffectResult: Seq[Row] = context.runSqlHive(sql).map(Row(_)) override def otherCopyArgs = context :: Nil } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index d61c5e274a596..0fc674af31885 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -37,7 +37,7 @@ case class AnalyzeTable(tableName: String) extends LeafNode with Command { def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { hiveContext.analyze(tableName) Seq.empty[Row] } @@ -53,7 +53,7 @@ case class DropTable(tableName: String, ifExists: Boolean) extends LeafNode with def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { val ifExistsClause = if (ifExists) "IF EXISTS " else "" hiveContext.runSqlHive(s"DROP TABLE $ifExistsClause$tableName") hiveContext.catalog.unregisterTable(None, tableName) @@ -70,7 +70,7 @@ case class AddJar(path: String) extends LeafNode with Command { override def output = Seq.empty - override protected[sql] lazy val sideEffectResult: Seq[Row] = { + override protected lazy val sideEffectResult: Seq[Row] = { hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) Seq.empty[Row] From 4e79970d32f9b917590dba8319bdc677e3bdd63a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 1 Oct 2014 16:03:00 -0700 Subject: [PATCH 0862/1492] Revert "[SPARK-3755][Core] Do not bind port 1 - 1024 to server in spark" This reverts commit 6390aae4eacbabfb1c53fb828b824f6a6518beff. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 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 e5b83c069d961..b3025c6ec3364 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1439,7 +1439,7 @@ private[spark] object Utils extends Logging { val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" for (offset <- 0 to maxRetries) { // Do not increment port if startPort is 0, which is treated as a special port - val tryPort = if (startPort == 0) startPort else (startPort + offset) % (65536 - 1024) + 1024 + val tryPort = if (startPort == 0) startPort else (startPort + offset) % 65536 try { val (service, port) = startService(tryPort) logInfo(s"Successfully started service$serviceString on port $port.") From 45e058ca4babbe3cef6524b6a0f48b466a5139bf Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 1 Oct 2014 16:30:28 -0700 Subject: [PATCH 0863/1492] [SPARK-3729][SQL] Do all hive session state initialization in lazy val This change avoids a NPE during context initialization when settings are present. Author: Michael Armbrust Closes #2583 from marmbrus/configNPE and squashes the following commits: da2ec57 [Michael Armbrust] Do all hive session state initilialization in lazy val --- .../scala/org/apache/spark/sql/hive/HiveContext.scala | 8 ++++---- .../main/scala/org/apache/spark/sql/hive/TestHive.scala | 4 +++- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 20ebe4996c207..fdb56901f9ddb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -231,12 +231,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { @transient protected[hive] lazy val sessionState = { val ss = new SessionState(hiveconf) setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. + SessionState.start(ss) + ss.err = new PrintStream(outputBuffer, true, "UTF-8") + ss.out = new PrintStream(outputBuffer, true, "UTF-8") + ss } - sessionState.err = new PrintStream(outputBuffer, true, "UTF-8") - sessionState.out = new PrintStream(outputBuffer, true, "UTF-8") - override def setConf(key: String, value: String): Unit = { super.setConf(key, value) runSqlHive(s"SET $key=$value") @@ -273,7 +274,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { results } - SessionState.start(sessionState) /** * Execute the command using Hive and return the results as a sequence. Each element diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 70fb15259e7d7..4a999b98ad92b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -40,8 +40,10 @@ import org.apache.spark.sql.SQLConf /* Implicit conversions */ import scala.collection.JavaConversions._ +// SPARK-3729: Test key required to check for initialization errors with config. object TestHive - extends TestHiveContext(new SparkContext("local[2]", "TestSQLContext", new SparkConf())) + extends TestHiveContext( + new SparkContext("local[2]", "TestSQLContext", new SparkConf().set("spark.sql.test", ""))) /** * A locally running test instance of Spark's Hive execution engine. From 1b9f0d67f28011cdff316042b344c9891f986aaa Mon Sep 17 00:00:00 2001 From: scwf Date: Wed, 1 Oct 2014 16:38:10 -0700 Subject: [PATCH 0864/1492] [SPARK-3704][SQL] Fix ColumnValue type for Short values in thrift server case ```ShortType```, we should add short value to hive row. Int value may lead to some problems. Author: scwf Closes #2551 from scwf/fix-addColumnValue and squashes the following commits: 08bcc59 [scwf] ColumnValue.shortValue for short type --- .../hive/thriftserver/server/SparkSQLOperationManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index bd3f68d92d8c7..910174a153768 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -113,7 +113,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) case ByteType => to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) case ShortType => - to.addColumnValue(ColumnValue.intValue(from.getShort(ordinal))) + to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) case TimestampType => to.addColumnValue( ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) @@ -145,7 +145,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) case ByteType => to.addColumnValue(ColumnValue.byteValue(null)) case ShortType => - to.addColumnValue(ColumnValue.intValue(null)) + to.addColumnValue(ColumnValue.shortValue(null)) case TimestampType => to.addColumnValue(ColumnValue.timestampValue(null)) case BinaryType | _: ArrayType | _: StructType | _: MapType => From 93861a5e876fa57f509cce82768656ddf8d4ef00 Mon Sep 17 00:00:00 2001 From: aniketbhatnagar Date: Wed, 1 Oct 2014 18:31:18 -0700 Subject: [PATCH 0865/1492] SPARK-3638 | Forced a compatible version of http client in kinesis-asl profile This patch forces use of commons http client 4.2 in Kinesis-asl profile so that the AWS SDK does not run into dependency conflicts Author: aniketbhatnagar Closes #2535 from aniketbhatnagar/Kinesis-HttpClient-Dep-Fix and squashes the following commits: aa2079f [aniketbhatnagar] Merge branch 'Kinesis-HttpClient-Dep-Fix' of https://github.com/aniketbhatnagar/spark into Kinesis-HttpClient-Dep-Fix 73f55f6 [aniketbhatnagar] SPARK-3638 | Forced a compatible version of http client in kinesis-asl profile 70cc75b [aniketbhatnagar] deleted merge files 725dbc9 [aniketbhatnagar] Merge remote-tracking branch 'origin/Kinesis-HttpClient-Dep-Fix' into Kinesis-HttpClient-Dep-Fix 4ed61d8 [aniketbhatnagar] SPARK-3638 | Forced a compatible version of http client in kinesis-asl profile 9cd6103 [aniketbhatnagar] SPARK-3638 | Forced a compatible version of http client in kinesis-asl profile --- assembly/pom.xml | 10 ++++++++++ examples/pom.xml | 5 +++++ pom.xml | 1 + 3 files changed, 16 insertions(+) diff --git a/assembly/pom.xml b/assembly/pom.xml index 5ec9da22ae83f..31a01e4d8e1de 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -349,5 +349,15 @@ + + kinesis-asl + + + org.apache.httpcomponents + httpclient + ${commons.httpclient.version} + + + diff --git a/examples/pom.xml b/examples/pom.xml index 2b561857f9f33..eb49a0e5af22d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -43,6 +43,11 @@ spark-streaming-kinesis-asl_${scala.binary.version} ${project.version} + + org.apache.httpcomponents + httpclient + ${commons.httpclient.version} + diff --git a/pom.xml b/pom.xml index 70cb9729ff6d3..7756c89b00cad 100644 --- a/pom.xml +++ b/pom.xml @@ -138,6 +138,7 @@ 0.7.1 1.8.3 1.1.0 + 4.2.6 64m 512m From 29c3513203218af33bea2f6d99d622cf263d55dd Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 1 Oct 2014 19:24:22 -0700 Subject: [PATCH 0866/1492] [SPARK-3446] Expose underlying job ids in FutureAction. FutureAction is the only type exposed through the async APIs, so for job IDs to be useful they need to be exposed there. The complication is that some async jobs run more than one job (e.g. takeAsync), so the exposed ID has to actually be a list of IDs that can actually change over time. So the interface doesn't look very nice, but... Change is actually small, I just added a basic test to make sure it works. Author: Marcelo Vanzin Closes #2337 from vanzin/SPARK-3446 and squashes the following commits: e166a68 [Marcelo Vanzin] Fix comment. 1fed2bc [Marcelo Vanzin] [SPARK-3446] Expose underlying job ids in FutureAction. --- .../scala/org/apache/spark/FutureAction.scala | 19 ++++++- .../org/apache/spark/FutureActionSuite.scala | 49 +++++++++++++++++++ 2 files changed, 66 insertions(+), 2 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/FutureActionSuite.scala diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 75ea535f2f57b..e8f761eaa5799 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -83,6 +83,15 @@ trait FutureAction[T] extends Future[T] { */ @throws(classOf[Exception]) def get(): T = Await.result(this, Duration.Inf) + + /** + * Returns the job IDs run by the underlying async operation. + * + * This returns the current snapshot of the job list. Certain operations may run multiple + * jobs, so multiple calls to this method may return different lists. + */ + def jobIds: Seq[Int] + } @@ -150,8 +159,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: } } - /** Get the corresponding job id for this action. */ - def jobId = jobWaiter.jobId + def jobIds = Seq(jobWaiter.jobId) } @@ -171,6 +179,8 @@ class ComplexFutureAction[T] extends FutureAction[T] { // is cancelled before the action was even run (and thus we have no thread to interrupt). @volatile private var _cancelled: Boolean = false + @volatile private var jobs: Seq[Int] = Nil + // A promise used to signal the future. private val p = promise[T]() @@ -219,6 +229,8 @@ class ComplexFutureAction[T] extends FutureAction[T] { } } + this.jobs = jobs ++ job.jobIds + // Wait for the job to complete. If the action is cancelled (with an interrupt), // cancel the job and stop the execution. This is not in a synchronized block because // Await.ready eventually waits on the monitor in FutureJob.jobWaiter. @@ -255,4 +267,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { override def isCompleted: Boolean = p.isCompleted override def value: Option[Try[T]] = p.future.value + + def jobIds = jobs + } diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala new file mode 100644 index 0000000000000..db9c25fc457a4 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import scala.concurrent.Await +import scala.concurrent.duration.Duration + +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} + +import org.apache.spark.SparkContext._ + +class FutureActionSuite extends FunSuite with BeforeAndAfter with Matchers with LocalSparkContext { + + before { + sc = new SparkContext("local", "FutureActionSuite") + } + + test("simple async action") { + val rdd = sc.parallelize(1 to 10, 2) + val job = rdd.countAsync() + val res = Await.result(job, Duration.Inf) + res should be (10) + job.jobIds.size should be (1) + } + + test("complex async action") { + val rdd = sc.parallelize(1 to 15, 3) + val job = rdd.takeAsync(10) + val res = Await.result(job, Duration.Inf) + res should be (1 to 10) + job.jobIds.size should be (2) + } + +} From f341e1c8a284b55cceb367a432c1fa5203692155 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 1 Oct 2014 23:08:51 -0700 Subject: [PATCH 0867/1492] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #1375 (close requested by 'pwendell') Closes #476 (close requested by 'mengxr') Closes #2502 (close requested by 'pwendell') Closes #2391 (close requested by 'andrewor14') From bbdf1de84ffdd3bd172f17975d2f1422a9bcf2c6 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Wed, 1 Oct 2014 23:53:21 -0700 Subject: [PATCH 0868/1492] [SPARK-3371][SQL] Renaming a function expression with group by gives error The following code gives error. ``` sqlContext.registerFunction("len", (s: String) => s.length) sqlContext.sql("select len(foo) as a, count(1) from t1 group by len(foo)").collect() ``` Because SQl parser creates the aliases to the functions in grouping expressions with generated alias names. So if user gives the alias names to the functions inside projection then it does not match the generated alias name of grouping expression. This kind of queries are working in Hive. So the fix I have given that if user provides alias to the function in projection then don't generate alias in grouping expression,use the same alias. Author: ravipesala Closes #2511 from ravipesala/SPARK-3371 and squashes the following commits: 9fb973f [ravipesala] Removed aliases to grouping expressions. f8ace79 [ravipesala] Fixed the testcase issue bad2fd0 [ravipesala] SPARK-3371 : Fixed Renaming a function expression with group by gives error --- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 862f78702c4e6..26336332c05a2 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -166,7 +166,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { val withFilter = f.map(f => Filter(f, base)).getOrElse(base) val withProjection = g.map {g => - Aggregate(assignAliases(g), assignAliases(p), withFilter) + Aggregate(g, assignAliases(p), withFilter) }.getOrElse(Project(assignAliases(p), withFilter)) val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection) val withHaving = h.map(h => Filter(h, withDistinct)).getOrElse(withDistinct) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index fdf3a229a796e..6fb6cb8db0c8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -680,4 +680,9 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT CAST(TRUE AS STRING), CAST(FALSE AS STRING) FROM testData LIMIT 1"), ("true", "false") :: Nil) } + + test("SPARK-3371 Renaming a function expression with group by gives error") { + registerFunction("len", (s: String) => s.length) + checkAnswer( + sql("SELECT len(value) as temp FROM testData WHERE key = 1 group by len(value)"), 1)} } From 6e27cb630de69fa5acb510b4e2f6b980742b1957 Mon Sep 17 00:00:00 2001 From: Colin Patrick Mccabe Date: Thu, 2 Oct 2014 00:29:31 -0700 Subject: [PATCH 0869/1492] SPARK-1767: Prefer HDFS-cached replicas when scheduling data-local tasks This change reorders the replicas returned by HadoopRDD#getPreferredLocations so that replicas cached by HDFS are at the start of the list. This requires Hadoop 2.5 or higher; previous versions of Hadoop do not expose the information needed to determine whether a replica is cached. Author: Colin Patrick Mccabe Closes #1486 from cmccabe/SPARK-1767 and squashes the following commits: 338d4f8 [Colin Patrick Mccabe] SPARK-1767: Prefer HDFS-cached replicas when scheduling data-local tasks --- .../org/apache/spark/rdd/HadoopRDD.scala | 60 +++++++++++++++++-- .../org/apache/spark/rdd/NewHadoopRDD.scala | 18 +++++- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../apache/spark/scheduler/TaskLocation.scala | 48 +++++++++++++-- .../spark/scheduler/TaskSetManager.scala | 25 +++++++- .../spark/scheduler/TaskSetManagerSuite.scala | 22 +++++++ project/MimaExcludes.scala | 2 + 8 files changed, 162 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 21d0cc7b5cbaa..6b63eb23e9ee1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -23,6 +23,7 @@ import java.io.EOFException import scala.collection.immutable.Map import scala.reflect.ClassTag +import scala.collection.mutable.ListBuffer import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.mapred.FileSplit @@ -43,6 +44,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.{NextIterator, Utils} +import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} /** @@ -249,9 +251,21 @@ class HadoopRDD[K, V]( } override def getPreferredLocations(split: Partition): Seq[String] = { - // TODO: Filtering out "localhost" in case of file:// URLs - val hadoopSplit = split.asInstanceOf[HadoopPartition] - hadoopSplit.inputSplit.value.getLocations.filter(_ != "localhost") + val hsplit = split.asInstanceOf[HadoopPartition].inputSplit.value + val locs: Option[Seq[String]] = HadoopRDD.SPLIT_INFO_REFLECTIONS match { + case Some(c) => + try { + val lsplit = c.inputSplitWithLocationInfo.cast(hsplit) + val infos = c.getLocationInfo.invoke(lsplit).asInstanceOf[Array[AnyRef]] + Some(HadoopRDD.convertSplitLocationInfo(infos)) + } catch { + case e: Exception => + logDebug("Failed to use InputSplitWithLocations.", e) + None + } + case None => None + } + locs.getOrElse(hsplit.getLocations.filter(_ != "localhost")) } override def checkpoint() { @@ -261,7 +275,7 @@ class HadoopRDD[K, V]( def getConf: Configuration = getJobConf() } -private[spark] object HadoopRDD { +private[spark] object HadoopRDD extends Logging { /** Constructing Configuration objects is not threadsafe, use this lock to serialize. */ val CONFIGURATION_INSTANTIATION_LOCK = new Object() @@ -309,4 +323,42 @@ private[spark] object HadoopRDD { f(inputSplit, firstParent[T].iterator(split, context)) } } + + private[spark] class SplitInfoReflections { + val inputSplitWithLocationInfo = + Class.forName("org.apache.hadoop.mapred.InputSplitWithLocationInfo") + val getLocationInfo = inputSplitWithLocationInfo.getMethod("getLocationInfo") + val newInputSplit = Class.forName("org.apache.hadoop.mapreduce.InputSplit") + val newGetLocationInfo = newInputSplit.getMethod("getLocationInfo") + val splitLocationInfo = Class.forName("org.apache.hadoop.mapred.SplitLocationInfo") + val isInMemory = splitLocationInfo.getMethod("isInMemory") + val getLocation = splitLocationInfo.getMethod("getLocation") + } + + private[spark] val SPLIT_INFO_REFLECTIONS: Option[SplitInfoReflections] = try { + Some(new SplitInfoReflections) + } catch { + case e: Exception => + logDebug("SplitLocationInfo and other new Hadoop classes are " + + "unavailable. Using the older Hadoop location info code.", e) + None + } + + private[spark] def convertSplitLocationInfo(infos: Array[AnyRef]): Seq[String] = { + val out = ListBuffer[String]() + infos.foreach { loc => { + val locationStr = HadoopRDD.SPLIT_INFO_REFLECTIONS.get. + getLocation.invoke(loc).asInstanceOf[String] + if (locationStr != "localhost") { + if (HadoopRDD.SPLIT_INFO_REFLECTIONS.get.isInMemory. + invoke(loc).asInstanceOf[Boolean]) { + logDebug("Partition " + locationStr + " is cached by Hadoop.") + out += new HDFSCacheTaskLocation(locationStr).toString + } else { + out += new HostTaskLocation(locationStr).toString + } + } + }} + out.seq + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 4c84b3f62354d..0cccdefc5ee09 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -173,9 +173,21 @@ class NewHadoopRDD[K, V]( new NewHadoopMapPartitionsWithSplitRDD(this, f, preservesPartitioning) } - override def getPreferredLocations(split: Partition): Seq[String] = { - val theSplit = split.asInstanceOf[NewHadoopPartition] - theSplit.serializableHadoopSplit.value.getLocations.filter(_ != "localhost") + override def getPreferredLocations(hsplit: Partition): Seq[String] = { + val split = hsplit.asInstanceOf[NewHadoopPartition].serializableHadoopSplit.value + val locs = HadoopRDD.SPLIT_INFO_REFLECTIONS match { + case Some(c) => + try { + val infos = c.newGetLocationInfo.invoke(split).asInstanceOf[Array[AnyRef]] + Some(HadoopRDD.convertSplitLocationInfo(infos)) + } catch { + case e : Exception => + logDebug("Failed to use InputSplit#getLocationInfo.", e) + None + } + case None => None + } + locs.getOrElse(split.getLocations.filter(_ != "localhost")) } def getConf: Configuration = confBroadcast.value.value 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 ab9e97c8fe409..2aba40d152e3e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -208,7 +208,7 @@ abstract class RDD[T: ClassTag]( } /** - * Get the preferred locations of a partition (as hostnames), taking into account whether the + * Get the preferred locations of a partition, taking into account whether the * RDD is checkpointed. */ final def preferredLocations(split: Partition): Seq[String] = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 5a96f52a10cd4..8135cdbb4c31f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1303,7 +1303,7 @@ class DAGScheduler( // If the RDD has some placement preferences (as is the case for input RDDs), get those val rddPrefs = rdd.preferredLocations(rdd.partitions(partition)).toList if (!rddPrefs.isEmpty) { - return rddPrefs.map(host => TaskLocation(host)) + return rddPrefs.map(TaskLocation(_)) } // If the RDD has narrow dependencies, pick the first partition of the first narrow dep // that has any placement preferences. Ideally we would choose based on transfer sizes, diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala index 67c9a6760b1b3..10c685f29d3ac 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocation.scala @@ -22,13 +22,51 @@ package org.apache.spark.scheduler * In the latter case, we will prefer to launch the task on that executorID, but our next level * of preference will be executors on the same host if this is not possible. */ -private[spark] -class TaskLocation private (val host: String, val executorId: Option[String]) extends Serializable { - override def toString: String = "TaskLocation(" + host + ", " + executorId + ")" +private[spark] sealed trait TaskLocation { + def host: String +} + +/** + * A location that includes both a host and an executor id on that host. + */ +private [spark] case class ExecutorCacheTaskLocation(override val host: String, + val executorId: String) extends TaskLocation { +} + +/** + * A location on a host. + */ +private [spark] case class HostTaskLocation(override val host: String) extends TaskLocation { + override def toString = host +} + +/** + * A location on a host that is cached by HDFS. + */ +private [spark] case class HDFSCacheTaskLocation(override val host: String) + extends TaskLocation { + override def toString = TaskLocation.inMemoryLocationTag + host } private[spark] object TaskLocation { - def apply(host: String, executorId: String) = new TaskLocation(host, Some(executorId)) + // We identify hosts on which the block is cached with this prefix. Because this prefix contains + // underscores, which are not legal characters in hostnames, there should be no potential for + // confusion. See RFC 952 and RFC 1123 for information about the format of hostnames. + val inMemoryLocationTag = "hdfs_cache_" + + def apply(host: String, executorId: String) = new ExecutorCacheTaskLocation(host, executorId) - def apply(host: String) = new TaskLocation(host, None) + /** + * Create a TaskLocation from a string returned by getPreferredLocations. + * These strings have the form [hostname] or hdfs_cache_[hostname], depending on whether the + * location is cached. + */ + def apply(str: String) = { + val hstr = str.stripPrefix(inMemoryLocationTag) + if (hstr.equals(str)) { + new HostTaskLocation(str) + } else { + new HostTaskLocation(hstr) + } + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d9d53faf843ff..a6c23fc85a1b0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -181,8 +181,24 @@ private[spark] class TaskSetManager( } for (loc <- tasks(index).preferredLocations) { - for (execId <- loc.executorId) { - addTo(pendingTasksForExecutor.getOrElseUpdate(execId, new ArrayBuffer)) + loc match { + case e: ExecutorCacheTaskLocation => + addTo(pendingTasksForExecutor.getOrElseUpdate(e.executorId, new ArrayBuffer)) + case e: HDFSCacheTaskLocation => { + val exe = sched.getExecutorsAliveOnHost(loc.host) + exe match { + case Some(set) => { + for (e <- set) { + addTo(pendingTasksForExecutor.getOrElseUpdate(e, new ArrayBuffer)) + } + logInfo(s"Pending task $index has a cached location at ${e.host} " + + ", where there are executors " + set.mkString(",")) + } + case None => logDebug(s"Pending task $index has a cached location at ${e.host} " + + ", but there are no executors alive there.") + } + } + case _ => Unit } addTo(pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer)) for (rack <- sched.getRackForHost(loc.host)) { @@ -283,7 +299,10 @@ private[spark] class TaskSetManager( // on multiple nodes when we replicate cached blocks, as in Spark Streaming for (index <- speculatableTasks if canRunOnHost(index)) { val prefs = tasks(index).preferredLocations - val executors = prefs.flatMap(_.executorId) + val executors = prefs.flatMap(_ match { + case e: ExecutorCacheTaskLocation => Some(e.executorId) + case _ => None + }); if (executors.contains(execId)) { speculatableTasks -= index return Some((index, TaskLocality.PROCESS_LOCAL)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 93e8ddacf8865..c0b07649eb6dd 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -642,6 +642,28 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execC", "host3", ANY) !== None) } + test("Test that locations with HDFSCacheTaskLocation are treated as PROCESS_LOCAL.") { + // Regression test for SPARK-2931 + sc = new SparkContext("local", "test") + val sched = new FakeTaskScheduler(sc, + ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val taskSet = FakeTask.createTaskSet(3, + Seq(HostTaskLocation("host1")), + Seq(HostTaskLocation("host2")), + Seq(HDFSCacheTaskLocation("host3"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + sched.removeExecutor("execA") + manager.executorAdded() + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + sched.removeExecutor("execB") + manager.executorAdded() + assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) + sched.removeExecutor("execC") + manager.executorAdded() + assert(manager.myLocalityLevels.sameElements(Array(ANY))) + } def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 4076ebc6fc8d5..d499302124461 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -41,6 +41,8 @@ object MimaExcludes { MimaBuild.excludeSparkClass("mllib.linalg.Matrix") ++ MimaBuild.excludeSparkClass("mllib.linalg.Vector") ++ Seq( + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.scheduler.TaskLocation"), // Added normL1 and normL2 to trait MultivariateStatisticalSummary ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.stat.MultivariateStatisticalSummary.normL1"), From 5b4a5b1acdc439a58aa2a3561ac0e3fb09f529d6 Mon Sep 17 00:00:00 2001 From: cocoatomo Date: Thu, 2 Oct 2014 11:13:19 -0700 Subject: [PATCH 0870/1492] [SPARK-3706][PySpark] Cannot run IPython REPL with IPYTHON set to "1" and PYSPARK_PYTHON unset MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### Problem The section "Using the shell" in Spark Programming Guide (https://spark.apache.org/docs/latest/programming-guide.html#using-the-shell) says that we can run pyspark REPL through IPython. But a folloing command does not run IPython but a default Python executable. ``` $ IPYTHON=1 ./bin/pyspark Python 2.7.8 (default, Jul 2 2014, 10:14:46) ... ``` the spark/bin/pyspark script on the commit b235e013638685758885842dc3268e9800af3678 decides which executable and options it use folloing way. 1. if PYSPARK_PYTHON unset * → defaulting to "python" 2. if IPYTHON_OPTS set * → set IPYTHON "1" 3. some python scripts passed to ./bin/pyspak → run it with ./bin/spark-submit * out of this issues scope 4. if IPYTHON set as "1" * → execute $PYSPARK_PYTHON (default: ipython) with arguments $IPYTHON_OPTS * otherwise execute $PYSPARK_PYTHON Therefore, when PYSPARK_PYTHON is unset, python is executed though IPYTHON is "1". In other word, when PYSPARK_PYTHON is unset, IPYTHON_OPS and IPYTHON has no effect on decide which command to use. PYSPARK_PYTHON | IPYTHON_OPTS | IPYTHON | resulting command | expected command ---- | ---- | ----- | ----- | ----- (unset → defaults to python) | (unset) | (unset) | python | (same) (unset → defaults to python) | (unset) | 1 | python | ipython (unset → defaults to python) | an_option | (unset → set to 1) | python an_option | ipython an_option (unset → defaults to python) | an_option | 1 | python an_option | ipython an_option ipython | (unset) | (unset) | ipython | (same) ipython | (unset) | 1 | ipython | (same) ipython | an_option | (unset → set to 1) | ipython an_option | (same) ipython | an_option | 1 | ipython an_option | (same) ### Suggestion The pyspark script should determine firstly whether a user wants to run IPython or other executables. 1. if IPYTHON_OPTS set * set IPYTHON "1" 2. if IPYTHON has a value "1" * PYSPARK_PYTHON defaults to "ipython" if not set 3. PYSPARK_PYTHON defaults to "python" if not set See the pull request for more detailed modification. Author: cocoatomo Closes #2554 from cocoatomo/issues/cannot-run-ipython-without-options and squashes the following commits: d2a9b06 [cocoatomo] [SPARK-3706][PySpark] Use PYTHONUNBUFFERED environment variable instead of -u option 264114c [cocoatomo] [SPARK-3706][PySpark] Remove the sentence about deprecated environment variables 42e02d5 [cocoatomo] [SPARK-3706][PySpark] Replace environment variables used to customize execution of PySpark REPL 10d56fb [cocoatomo] [SPARK-3706][PySpark] Cannot run IPython REPL with IPYTHON set to "1" and PYSPARK_PYTHON unset --- bin/pyspark | 24 +++++++++---------- .../apache/spark/deploy/PythonRunner.scala | 3 ++- docs/programming-guide.md | 8 +++---- 3 files changed, 18 insertions(+), 17 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 5142411e36974..6655725ef8e8e 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -52,10 +52,20 @@ fi # Figure out which Python executable to use if [[ -z "$PYSPARK_PYTHON" ]]; then - PYSPARK_PYTHON="python" + if [[ "$IPYTHON" = "1" || -n "$IPYTHON_OPTS" ]]; then + # for backward compatibility + PYSPARK_PYTHON="ipython" + else + PYSPARK_PYTHON="python" + fi fi export PYSPARK_PYTHON +if [[ -z "$PYSPARK_PYTHON_OPTS" && -n "$IPYTHON_OPTS" ]]; then + # for backward compatibility + PYSPARK_PYTHON_OPTS="$IPYTHON_OPTS" +fi + # Add the PySpark classes to the Python path: export PYTHONPATH="$SPARK_HOME/python/:$PYTHONPATH" export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" @@ -64,11 +74,6 @@ export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" export PYTHONSTARTUP="$FWDIR/python/pyspark/shell.py" -# If IPython options are specified, assume user wants to run IPython -if [[ -n "$IPYTHON_OPTS" ]]; then - IPYTHON=1 -fi - # Build up arguments list manually to preserve quotes and backslashes. # We export Spark submit arguments as an environment variable because shell.py must run as a # PYTHONSTARTUP script, which does not take in arguments. This is required for IPython notebooks. @@ -106,10 +111,5 @@ if [[ "$1" =~ \.py$ ]]; then else # PySpark shell requires special handling downstream export PYSPARK_SHELL=1 - # Only use ipython if no command line arguments were provided [SPARK-1134] - if [[ "$IPYTHON" = "1" ]]; then - exec ${PYSPARK_PYTHON:-ipython} $IPYTHON_OPTS - else - exec "$PYSPARK_PYTHON" - fi + exec "$PYSPARK_PYTHON" $PYSPARK_PYTHON_OPTS fi diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index b66c3ba4d5fb0..79b4d7ea41a33 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -54,9 +54,10 @@ object PythonRunner { val pythonPath = PythonUtils.mergePythonPaths(pathElements: _*) // Launch Python process - val builder = new ProcessBuilder(Seq(pythonExec, "-u", formattedPythonFile) ++ otherArgs) + val builder = new ProcessBuilder(Seq(pythonExec, formattedPythonFile) ++ otherArgs) val env = builder.environment() env.put("PYTHONPATH", pythonPath) + env.put("PYTHONUNBUFFERED", "YES") // value is needed to be set to a non-empty string env.put("PYSPARK_GATEWAY_PORT", "" + gatewayServer.getListeningPort) builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize val process = builder.start() diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 1d61a3c555eaf..8e8cc1dd983f8 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -211,17 +211,17 @@ For a complete list of options, run `pyspark --help`. Behind the scenes, It is also possible to launch the PySpark shell in [IPython](http://ipython.org), the enhanced Python interpreter. PySpark works with IPython 1.0.0 and later. To -use IPython, set the `IPYTHON` variable to `1` when running `bin/pyspark`: +use IPython, set the `PYSPARK_PYTHON` variable to `ipython` when running `bin/pyspark`: {% highlight bash %} -$ IPYTHON=1 ./bin/pyspark +$ PYSPARK_PYTHON=ipython ./bin/pyspark {% endhighlight %} -You can customize the `ipython` command by setting `IPYTHON_OPTS`. For example, to launch +You can customize the `ipython` command by setting `PYSPARK_PYTHON_OPTS`. For example, to launch the [IPython Notebook](http://ipython.org/notebook.html) with PyLab plot support: {% highlight bash %} -$ IPYTHON_OPTS="notebook --pylab inline" ./bin/pyspark +$ PYSPARK_PYTHON=ipython PYSPARK_PYTHON_OPTS="notebook --pylab inline" ./bin/pyspark {% endhighlight %} From 82a6a083a485140858bcd93d73adec59bb5cca64 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Thu, 2 Oct 2014 11:37:24 -0700 Subject: [PATCH 0871/1492] [SQL][Docs] Update the output of printSchema and fix a typo in SQL programming guide. We have changed the output format of `printSchema`. This PR will update our SQL programming guide to show the updated format. Also, it fixes a typo (the value type of `StructType` in Java API). Author: Yin Huai Closes #2630 from yhuai/sqlDoc and squashes the following commits: 267d63e [Yin Huai] Update the output of printSchema and fix a typo. --- docs/sql-programming-guide.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 818fd5ab80af8..368c3d0008b07 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -620,8 +620,8 @@ val people = sqlContext.jsonFile(path) // The inferred schema can be visualized using the printSchema() method. people.printSchema() // root -// |-- age: IntegerType -// |-- name: StringType +// |-- age: integer (nullable = true) +// |-- name: string (nullable = true) // Register this SchemaRDD as a table. people.registerTempTable("people") @@ -658,8 +658,8 @@ JavaSchemaRDD people = sqlContext.jsonFile(path); // The inferred schema can be visualized using the printSchema() method. people.printSchema(); // root -// |-- age: IntegerType -// |-- name: StringType +// |-- age: integer (nullable = true) +// |-- name: string (nullable = true) // Register this JavaSchemaRDD as a table. people.registerTempTable("people"); @@ -697,8 +697,8 @@ people = sqlContext.jsonFile(path) # The inferred schema can be visualized using the printSchema() method. people.printSchema() # root -# |-- age: IntegerType -# |-- name: StringType +# |-- age: integer (nullable = true) +# |-- name: string (nullable = true) # Register this SchemaRDD as a table. people.registerTempTable("people") @@ -1394,7 +1394,7 @@ please use factory methods provided in - + - + - + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 26dbd6237c6b8..a12f82d2fbe70 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf import org.apache.spark.util.{Utils, IntParam, MemoryParam} - +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) { @@ -39,15 +39,17 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var appName: String = "Spark" var priority = 0 + parseArgs(args.toList) + loadEnvironmentArgs() + // Additional memory to allocate to containers // For now, use driver's memory overhead as our AM container's memory overhead - val amMemoryOverhead = sparkConf.getInt( - "spark.yarn.driver.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - val executorMemoryOverhead = sparkConf.getInt( - "spark.yarn.executor.memoryOverhead", YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) + val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", + math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toInt, MEMORY_OVERHEAD_MIN)) + + val executorMemoryOverhead = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) - parseArgs(args.toList) - loadEnvironmentArgs() validateArgs() /** Load any default arguments provided through environment variables and Spark properties. */ diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 1cf19c198509c..6ecac6eae6e03 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -64,14 +64,18 @@ private[spark] trait ClientBase extends Logging { s"memory capability of the cluster ($maxMem MB per container)") val executorMem = args.executorMemory + executorMemoryOverhead if (executorMem > maxMem) { - throw new IllegalArgumentException(s"Required executor memory ($executorMem MB) " + - s"is above the max threshold ($maxMem MB) of this cluster!") + throw new IllegalArgumentException(s"Required executor memory (${args.executorMemory}" + + s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") } val amMem = args.amMemory + amMemoryOverhead if (amMem > maxMem) { - throw new IllegalArgumentException(s"Required AM memory ($amMem MB) " + - s"is above the max threshold ($maxMem MB) of this cluster!") + throw new IllegalArgumentException(s"Required AM memory (${args.amMemory}" + + s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") } + logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format( + amMem, + amMemoryOverhead)) + // We could add checks to make sure the entire cluster has enough resources but that involves // getting all the node reports and computing ourselves. } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 299e38a5eb9c0..4f4f1d2aaaade 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -33,6 +33,7 @@ import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ object AllocationType extends Enumeration { type AllocationType = Value @@ -78,10 +79,6 @@ private[yarn] abstract class YarnAllocator( // Containers to be released in next request to RM private val releasedContainers = new ConcurrentHashMap[ContainerId, Boolean] - // Additional memory overhead - in mb. - protected val memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", - YarnSparkHadoopUtil.DEFAULT_MEMORY_OVERHEAD) - // Number of container requests that have been sent to, but not yet allocated by the // ApplicationMaster. private val numPendingAllocate = new AtomicInteger() @@ -97,6 +94,10 @@ private[yarn] abstract class YarnAllocator( protected val (preferredHostToCount, preferredRackToCount) = generateNodeToWeight(conf, preferredNodes) + // Additional memory overhead - in mb. + protected val memoryOverhead: Int = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) + private val launcherPool = new ThreadPoolExecutor( // max pool size of Integer.MAX_VALUE is ignored because we use an unbounded queue sparkConf.getInt("spark.yarn.containerLauncherMaxThreads", 25), Integer.MAX_VALUE, @@ -114,12 +115,11 @@ private[yarn] abstract class YarnAllocator( // this is needed by alpha, do it here since we add numPending right after this val executorsPending = numPendingAllocate.get() - if (missing > 0) { + val totalExecutorMemory = executorMemory + memoryOverhead numPendingAllocate.addAndGet(missing) - logInfo("Will Allocate %d executor containers, each with %d memory".format( - missing, - (executorMemory + memoryOverhead))) + logInfo(s"Will allocate $missing executor containers, each with $totalExecutorMemory MB " + + s"memory including $memoryOverhead MB overhead") } else { logDebug("Empty allocation request ...") } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 0b712c201904a..e1e0144f46fe9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -84,8 +84,12 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } object YarnSparkHadoopUtil { - // Additional memory overhead - in mb. - val DEFAULT_MEMORY_OVERHEAD = 384 + // Additional memory overhead + // 7% was arrived at experimentally. In the interest of minimizing memory waste while covering + // the common cases. Memory overhead tends to grow with container size. + + val MEMORY_OVERHEAD_FACTOR = 0.07 + val MEMORY_OVERHEAD_MIN = 384 val ANY_HOST = "*" From c6469a02f14e8c23e9b4e1336768f8bbfc15f5d8 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 2 Oct 2014 13:47:30 -0700 Subject: [PATCH 0873/1492] [SPARK-3766][Doc]Snappy is also the default compress codec for broadcast variables Author: scwf Closes #2632 from scwf/compress-doc and squashes the following commits: 7983a1a [scwf] snappy is the default compression codec for broadcast --- docs/configuration.md | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 791b6f2aa3261..316490f0f43fc 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -413,10 +413,11 @@ Apart from these, the following properties are also available, and may be useful From 5db78e6b87d33ac2d48a997e69b46e9be3b63137 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 2 Oct 2014 13:49:47 -0700 Subject: [PATCH 0874/1492] [SPARK-3495] Block replication fails continuously when the replication target node is dead AND [SPARK-3496] Block replication by mistake chooses driver as target If a block manager (say, A) wants to replicate a block and the node chosen for replication (say, B) is dead, then the attempt to send the block to B fails. However, this continues to fail indefinitely. Even if the driver learns about the demise of the B, A continues to try replicating to B and failing miserably. The reason behind this bug is that A initially fetches a list of peers from the driver (when B was active), but never updates it after B is dead. This affects Spark Streaming as its receiver uses block replication. The solution in this patch adds the following. - Changed BlockManagerMaster to return all the peers of a block manager, rather than the requested number. It also filters out driver BlockManager. - Refactored BlockManager's replication code to handle peer caching correctly. + The peer for replication is randomly selected. This is different from past behavior where for a node A, a node B was deterministically chosen for the lifetime of the application. + If replication fails to one node, the peers are refetched. + The peer cached has a TTL of 1 second to enable discovery of new peers and using them for replication. - Refactored use of \ in BlockManager into a new method `BlockManagerId.isDriver` - Added replication unit tests (replication was not tested till now, duh!) This should not make a difference in performance of Spark workloads where replication is not used. @andrewor14 @JoshRosen Author: Tathagata Das Closes #2366 from tdas/replication-fix and squashes the following commits: 9690f57 [Tathagata Das] Moved replication tests to a new BlockManagerReplicationSuite. 0661773 [Tathagata Das] Minor changes based on PR comments. a55a65c [Tathagata Das] Added a unit test to test replication behavior. 012afa3 [Tathagata Das] Bug fix 89f91a0 [Tathagata Das] Minor change. 68e2c72 [Tathagata Das] Made replication peer selection logic more efficient. 08afaa9 [Tathagata Das] Made peer selection for replication deterministic to block id 3821ab9 [Tathagata Das] Fixes based on PR comments. 08e5646 [Tathagata Das] More minor changes. d402506 [Tathagata Das] Fixed imports. 4a20531 [Tathagata Das] Filtered driver block manager from peer list, and also consolidated the use of in BlockManager. 7598f91 [Tathagata Das] Minor changes. 03de02d [Tathagata Das] Change replication logic to correctly refetch peers from master on failure and on new worker addition. d081bf6 [Tathagata Das] Fixed bug in get peers and unit tests to test get-peers and replication under executor churn. 9f0ac9f [Tathagata Das] Modified replication tests to fail on replication bug. af0c1da [Tathagata Das] Added replication unit tests to BlockManagerSuite --- .../apache/spark/storage/BlockManager.scala | 122 ++++- .../apache/spark/storage/BlockManagerId.scala | 2 + .../spark/storage/BlockManagerMaster.scala | 9 +- .../storage/BlockManagerMasterActor.scala | 29 +- .../spark/storage/BlockManagerMessages.scala | 2 +- .../spark/broadcast/BroadcastSuite.scala | 2 +- .../BlockManagerReplicationSuite.scala | 418 ++++++++++++++++++ .../spark/storage/BlockManagerSuite.scala | 9 +- 8 files changed, 544 insertions(+), 49 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index d1bee3d2c033c..3f5d06e1aeee7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -22,6 +22,7 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import scala.concurrent.ExecutionContext.Implicits.global +import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ @@ -112,6 +113,11 @@ private[spark] class BlockManager( private val broadcastCleaner = new MetadataCleaner( MetadataCleanerType.BROADCAST_VARS, this.dropOldBroadcastBlocks, conf) + // Field related to peer block managers that are necessary for block replication + @volatile private var cachedPeers: Seq[BlockManagerId] = _ + private val peerFetchLock = new Object + private var lastPeerFetchTime = 0L + initialize() /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay @@ -787,31 +793,111 @@ private[spark] class BlockManager( } /** - * Replicate block to another node. + * Get peer block managers in the system. + */ + private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { + peerFetchLock.synchronized { + val cachedPeersTtl = conf.getInt("spark.storage.cachedPeersTtl", 60 * 1000) // milliseconds + val timeout = System.currentTimeMillis - lastPeerFetchTime > cachedPeersTtl + if (cachedPeers == null || forceFetch || timeout) { + cachedPeers = master.getPeers(blockManagerId).sortBy(_.hashCode) + lastPeerFetchTime = System.currentTimeMillis + logDebug("Fetched peers from master: " + cachedPeers.mkString("[", ",", "]")) + } + cachedPeers + } + } + + /** + * Replicate block to another node. Not that this is a blocking call that returns after + * the block has been replicated. */ - @volatile var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel): Unit = { + val maxReplicationFailures = conf.getInt("spark.storage.maxReplicationFailures", 1) + val numPeersToReplicateTo = level.replication - 1 + val peersForReplication = new ArrayBuffer[BlockManagerId] + val peersReplicatedTo = new ArrayBuffer[BlockManagerId] + val peersFailedToReplicateTo = new ArrayBuffer[BlockManagerId] val tLevel = StorageLevel( level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) - if (cachedPeers == null) { - cachedPeers = master.getPeers(blockManagerId, level.replication - 1) + val startTime = System.currentTimeMillis + val random = new Random(blockId.hashCode) + + var replicationFailed = false + var failures = 0 + var done = false + + // Get cached list of peers + peersForReplication ++= getPeers(forceFetch = false) + + // Get a random peer. Note that this selection of a peer is deterministic on the block id. + // So assuming the list of peers does not change and no replication failures, + // if there are multiple attempts in the same node to replicate the same block, + // the same set of peers will be selected. + def getRandomPeer(): Option[BlockManagerId] = { + // If replication had failed, then force update the cached list of peers and remove the peers + // that have been already used + if (replicationFailed) { + peersForReplication.clear() + peersForReplication ++= getPeers(forceFetch = true) + peersForReplication --= peersReplicatedTo + peersForReplication --= peersFailedToReplicateTo + } + if (!peersForReplication.isEmpty) { + Some(peersForReplication(random.nextInt(peersForReplication.size))) + } else { + None + } } - for (peer: BlockManagerId <- cachedPeers) { - val start = System.nanoTime - data.rewind() - logDebug(s"Try to replicate $blockId once; The size of the data is ${data.limit()} Bytes. " + - s"To node: $peer") - try { - blockTransferService.uploadBlockSync( - peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) - } catch { - case e: Exception => - logError(s"Failed to replicate block to $peer", e) + // One by one choose a random peer and try uploading the block to it + // If replication fails (e.g., target peer is down), force the list of cached peers + // to be re-fetched from driver and then pick another random peer for replication. Also + // temporarily black list the peer for which replication failed. + // + // This selection of a peer and replication is continued in a loop until one of the + // following 3 conditions is fulfilled: + // (i) specified number of peers have been replicated to + // (ii) too many failures in replicating to peers + // (iii) no peer left to replicate to + // + while (!done) { + getRandomPeer() match { + case Some(peer) => + try { + val onePeerStartTime = System.currentTimeMillis + data.rewind() + logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") + blockTransferService.uploadBlockSync( + peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) + logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %f ms" + .format((System.currentTimeMillis - onePeerStartTime))) + peersReplicatedTo += peer + peersForReplication -= peer + replicationFailed = false + if (peersReplicatedTo.size == numPeersToReplicateTo) { + done = true // specified number of peers have been replicated to + } + } catch { + case e: Exception => + logWarning(s"Failed to replicate $blockId to $peer, failure #$failures", e) + failures += 1 + replicationFailed = true + peersFailedToReplicateTo += peer + if (failures > maxReplicationFailures) { // too many failures in replcating to peers + done = true + } + } + case None => // no peer left to replicate to + done = true } - - logDebug("Replicating BlockId %s once used %fs; The size of the data is %d bytes." - .format(blockId, (System.nanoTime - start) / 1e6, data.limit())) + } + val timeTakeMs = (System.currentTimeMillis - startTime) + logDebug(s"Replicating $blockId of ${data.limit()} bytes to " + + s"${peersReplicatedTo.size} peer(s) took $timeTakeMs ms") + if (peersReplicatedTo.size < numPeersToReplicateTo) { + logWarning(s"Block $blockId replicated to only " + + s"${peersReplicatedTo.size} peer(s) instead of $numPeersToReplicateTo peers") } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index d4487fce49ab6..142285094342c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -59,6 +59,8 @@ class BlockManagerId private ( def port: Int = port_ + def isDriver: Boolean = (executorId == "") + override def writeExternal(out: ObjectOutput) { out.writeUTF(executorId_) out.writeUTF(host_) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 2e262594b3538..d08e1419e3e41 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -84,13 +84,8 @@ class BlockManagerMaster( } /** Get ids of other nodes in the cluster from the driver */ - def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { - val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) - if (result.length != numPeers) { - throw new SparkException( - "Error getting peers, only got " + result.size + " instead of " + numPeers) - } - result + def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { + askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId)) } /** 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 1a6c7cb24f9ac..6a06257ed0c08 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -83,8 +83,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case GetLocationsMultipleBlockIds(blockIds) => sender ! getLocationsMultipleBlockIds(blockIds) - case GetPeers(blockManagerId, size) => - sender ! getPeers(blockManagerId, size) + case GetPeers(blockManagerId) => + sender ! getPeers(blockManagerId) case GetMemoryStatus => sender ! memoryStatus @@ -173,11 +173,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus * from the executors, but not from the driver. */ private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { - // TODO: Consolidate usages of import context.dispatcher val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) val requiredBlockManagers = blockManagerInfo.values.filter { info => - removeFromDriver || info.blockManagerId.executorId != "" + removeFromDriver || !info.blockManagerId.isDriver } Future.sequence( requiredBlockManagers.map { bm => @@ -212,7 +211,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val minSeenTime = now - slaveTimeout val toRemove = new mutable.HashSet[BlockManagerId] for (info <- blockManagerInfo.values) { - if (info.lastSeenMs < minSeenTime && info.blockManagerId.executorId != "") { + if (info.lastSeenMs < minSeenTime && !info.blockManagerId.isDriver) { logWarning("Removing BlockManager " + info.blockManagerId + " with no recent heart beats: " + (now - info.lastSeenMs) + "ms exceeds " + slaveTimeout + "ms") toRemove += info.blockManagerId @@ -232,7 +231,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus */ private def heartbeatReceived(blockManagerId: BlockManagerId): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { - blockManagerId.executorId == "" && !isLocal + blockManagerId.isDriver && !isLocal } else { blockManagerInfo(blockManagerId).updateLastSeenMs() true @@ -355,7 +354,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus tachyonSize: Long) { if (!blockManagerInfo.contains(blockManagerId)) { - if (blockManagerId.executorId == "" && !isLocal) { + if (blockManagerId.isDriver && !isLocal) { // We intentionally do not register the master (except in local mode), // so we should not indicate failure. sender ! true @@ -403,16 +402,14 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockIds.map(blockId => getLocations(blockId)) } - private def getPeers(blockManagerId: BlockManagerId, size: Int): Seq[BlockManagerId] = { - val peers: Array[BlockManagerId] = blockManagerInfo.keySet.toArray - - val selfIndex = peers.indexOf(blockManagerId) - if (selfIndex == -1) { - throw new SparkException("Self index for " + blockManagerId + " not found") + /** Get the list of the peers of the given block manager */ + private def getPeers(blockManagerId: BlockManagerId): Seq[BlockManagerId] = { + val blockManagerIds = blockManagerInfo.keySet + if (blockManagerIds.contains(blockManagerId)) { + blockManagerIds.filterNot { _.isDriver }.filterNot { _ == blockManagerId }.toSeq + } else { + Seq.empty } - - // Note that this logic will select the same node multiple times if there aren't enough peers - Array.tabulate[BlockManagerId](size) { i => peers((selfIndex + i + 1) % peers.length) }.toSeq } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 2ba16b8476600..3db5dd9774ae8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -88,7 +88,7 @@ private[spark] object BlockManagerMessages { case class GetLocationsMultipleBlockIds(blockIds: Array[BlockId]) extends ToBlockManagerMaster - case class GetPeers(blockManagerId: BlockManagerId, size: Int) extends ToBlockManagerMaster + case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster case class RemoveExecutor(execId: String) extends ToBlockManagerMaster diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 978a6ded80829..acaf321de52fb 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -132,7 +132,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { val statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === 1) statuses.head match { case (bm, status) => - assert(bm.executorId === "", "Block should only be on the driver") + assert(bm.isDriver, "Block should only be on the driver") assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) assert(status.memSize > 0, "Block should be in memory store on the driver") assert(status.diskSize === 0, "Block should not be in disk store on the driver") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala new file mode 100644 index 0000000000000..1f1d53a1ee3b0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -0,0 +1,418 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + +import akka.actor.{ActorSystem, Props} +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.storage.StorageLevel._ +import org.apache.spark.util.{AkkaUtils, SizeEstimator} + +/** Testsuite that tests block replication in BlockManager */ +class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAndAfter { + + private val conf = new SparkConf(false) + var actorSystem: ActorSystem = null + var master: BlockManagerMaster = null + val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) + + // List of block manager created during an unit test, so that all of the them can be stopped + // after the unit test. + val allStores = new ArrayBuffer[BlockManager] + + // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test + conf.set("spark.kryoserializer.buffer.mb", "1") + val serializer = new KryoSerializer(conf) + + // Implicitly convert strings to BlockIds for test clarity. + implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) + + private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + val transfer = new NioBlockTransferService(conf, securityMgr) + val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, + mapOutputTracker, shuffleManager, transfer) + allStores += store + store + } + + before { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) + this.actorSystem = actorSystem + + conf.set("spark.authenticate", "false") + conf.set("spark.driver.port", boundPort.toString) + conf.set("spark.storage.unrollFraction", "0.4") + conf.set("spark.storage.unrollMemoryThreshold", "512") + + // to make a replication attempt to inactive store fail fast + conf.set("spark.core.connection.ack.wait.timeout", "1") + // to make cached peers refresh frequently + conf.set("spark.storage.cachedPeersTtl", "10") + + master = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf, true) + allStores.clear() + } + + after { + allStores.foreach { _.stop() } + allStores.clear() + actorSystem.shutdown() + actorSystem.awaitTermination() + actorSystem = null + master = null + } + + + test("get peers with addition and removal of block managers") { + val numStores = 4 + val stores = (1 to numStores - 1).map { i => makeBlockManager(1000, s"store$i") } + val storeIds = stores.map { _.blockManagerId }.toSet + assert(master.getPeers(stores(0).blockManagerId).toSet === + storeIds.filterNot { _ == stores(0).blockManagerId }) + assert(master.getPeers(stores(1).blockManagerId).toSet === + storeIds.filterNot { _ == stores(1).blockManagerId }) + assert(master.getPeers(stores(2).blockManagerId).toSet === + storeIds.filterNot { _ == stores(2).blockManagerId }) + + // Add driver store and test whether it is filtered out + val driverStore = makeBlockManager(1000, "") + assert(master.getPeers(stores(0).blockManagerId).forall(!_.isDriver)) + assert(master.getPeers(stores(1).blockManagerId).forall(!_.isDriver)) + assert(master.getPeers(stores(2).blockManagerId).forall(!_.isDriver)) + + // Add a new store and test whether get peers returns it + val newStore = makeBlockManager(1000, s"store$numStores") + assert(master.getPeers(stores(0).blockManagerId).toSet === + storeIds.filterNot { _ == stores(0).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(stores(1).blockManagerId).toSet === + storeIds.filterNot { _ == stores(1).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(stores(2).blockManagerId).toSet === + storeIds.filterNot { _ == stores(2).blockManagerId } + newStore.blockManagerId) + assert(master.getPeers(newStore.blockManagerId).toSet === storeIds) + + // Remove a store and test whether get peers returns it + val storeIdToRemove = stores(0).blockManagerId + master.removeExecutor(storeIdToRemove.executorId) + assert(!master.getPeers(stores(1).blockManagerId).contains(storeIdToRemove)) + assert(!master.getPeers(stores(2).blockManagerId).contains(storeIdToRemove)) + assert(!master.getPeers(newStore.blockManagerId).contains(storeIdToRemove)) + + // Test whether asking for peers of a unregistered block manager id returns empty list + assert(master.getPeers(stores(0).blockManagerId).isEmpty) + assert(master.getPeers(BlockManagerId("", "", 1)).isEmpty) + } + + + test("block replication - 2x replication") { + testReplication(2, + Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK_2, MEMORY_AND_DISK_SER_2) + ) + } + + test("block replication - 3x replication") { + // Generate storage levels with 3x replication + val storageLevels = { + Seq(MEMORY_ONLY, MEMORY_ONLY_SER, DISK_ONLY, MEMORY_AND_DISK, MEMORY_AND_DISK_SER).map { + level => StorageLevel( + level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 3) + } + } + testReplication(3, storageLevels) + } + + test("block replication - mixed between 1x to 5x") { + // Generate storage levels with varying replication + val storageLevels = Seq( + MEMORY_ONLY, + MEMORY_ONLY_SER_2, + StorageLevel(true, false, false, false, 3), + StorageLevel(true, true, false, true, 4), + StorageLevel(true, true, false, false, 5), + StorageLevel(true, true, false, true, 4), + StorageLevel(true, false, false, false, 3), + MEMORY_ONLY_SER_2, + MEMORY_ONLY + ) + testReplication(5, storageLevels) + } + + test("block replication - 2x replication without peers") { + intercept[org.scalatest.exceptions.TestFailedException] { + testReplication(1, + Seq(StorageLevel.MEMORY_AND_DISK_2, StorageLevel(true, false, false, false, 3))) + } + } + + test("block replication - deterministic node selection") { + val blockSize = 1000 + val storeSize = 10000 + val stores = (1 to 5).map { + i => makeBlockManager(storeSize, s"store$i") + } + val storageLevel2x = StorageLevel.MEMORY_AND_DISK_2 + val storageLevel3x = StorageLevel(true, true, false, true, 3) + val storageLevel4x = StorageLevel(true, true, false, true, 4) + + def putBlockAndGetLocations(blockId: String, level: StorageLevel): Set[BlockManagerId] = { + stores.head.putSingle(blockId, new Array[Byte](blockSize), level) + val locations = master.getLocations(blockId).sortBy { _.executorId }.toSet + stores.foreach { _.removeBlock(blockId) } + master.removeBlock(blockId) + locations + } + + // Test if two attempts to 2x replication returns same set of locations + val a1Locs = putBlockAndGetLocations("a1", storageLevel2x) + assert(putBlockAndGetLocations("a1", storageLevel2x) === a1Locs, + "Inserting a 2x replicated block second time gave different locations from the first") + + // Test if two attempts to 3x replication returns same set of locations + val a2Locs3x = putBlockAndGetLocations("a2", storageLevel3x) + assert(putBlockAndGetLocations("a2", storageLevel3x) === a2Locs3x, + "Inserting a 3x replicated block second time gave different locations from the first") + + // Test if 2x replication of a2 returns a strict subset of the locations of 3x replication + val a2Locs2x = putBlockAndGetLocations("a2", storageLevel2x) + assert( + a2Locs2x.subsetOf(a2Locs3x), + "Inserting a with 2x replication gave locations that are not a subset of locations" + + s" with 3x replication [3x: ${a2Locs3x.mkString(",")}; 2x: ${a2Locs2x.mkString(",")}" + ) + + // Test if 4x replication of a2 returns a strict superset of the locations of 3x replication + val a2Locs4x = putBlockAndGetLocations("a2", storageLevel4x) + assert( + a2Locs3x.subsetOf(a2Locs4x), + "Inserting a with 4x replication gave locations that are not a superset of locations " + + s"with 3x replication [3x: ${a2Locs3x.mkString(",")}; 4x: ${a2Locs4x.mkString(",")}" + ) + + // Test if 3x replication of two different blocks gives two different sets of locations + val a3Locs3x = putBlockAndGetLocations("a3", storageLevel3x) + assert(a3Locs3x !== a2Locs3x, "Two blocks gave same locations with 3x replication") + } + + test("block replication - replication failures") { + /* + Create a system of three block managers / stores. One of them (say, failableStore) + cannot receive blocks. So attempts to use that as replication target fails. + + +-----------/fails/-----------> failableStore + | + normalStore + | + +-----------/works/-----------> anotherNormalStore + + We are first going to add a normal block manager (i.e. normalStore) and the failable block + manager (i.e. failableStore), and test whether 2x replication fails to create two + copies of a block. Then we are going to add another normal block manager + (i.e., anotherNormalStore), and test that now 2x replication works as the + new store will be used for replication. + */ + + // Add a normal block manager + val store = makeBlockManager(10000, "store") + + // Insert a block with 2x replication and return the number of copies of the block + def replicateAndGetNumCopies(blockId: String): Int = { + store.putSingle(blockId, new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK_2) + val numLocations = master.getLocations(blockId).size + allStores.foreach { _.removeBlock(blockId) } + numLocations + } + + // Add a failable block manager with a mock transfer service that does not + // allow receiving of blocks. So attempts to use it as a replication target will fail. + val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work + when(failableTransfer.hostName).thenReturn("some-hostname") + when(failableTransfer.port).thenReturn(1000) + val failableStore = new BlockManager("failable-store", actorSystem, master, serializer, + 10000, conf, mapOutputTracker, shuffleManager, failableTransfer) + allStores += failableStore // so that this gets stopped after test + assert(master.getPeers(store.blockManagerId).toSet === Set(failableStore.blockManagerId)) + + // Test that 2x replication fails by creating only one copy of the block + assert(replicateAndGetNumCopies("a1") === 1) + + // Add another normal block manager and test that 2x replication works + makeBlockManager(10000, "anotherStore") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a2") === 2) + } + } + + test("block replication - addition and deletion of block managers") { + val blockSize = 1000 + val storeSize = 10000 + val initialStores = (1 to 2).map { i => makeBlockManager(storeSize, s"store$i") } + + // Insert a block with given replication factor and return the number of copies of the block\ + def replicateAndGetNumCopies(blockId: String, replicationFactor: Int): Int = { + val storageLevel = StorageLevel(true, true, false, true, replicationFactor) + initialStores.head.putSingle(blockId, new Array[Byte](blockSize), storageLevel) + val numLocations = master.getLocations(blockId).size + allStores.foreach { _.removeBlock(blockId) } + numLocations + } + + // 2x replication should work, 3x replication should only replicate 2x + assert(replicateAndGetNumCopies("a1", 2) === 2) + assert(replicateAndGetNumCopies("a2", 3) === 2) + + // Add another store, 3x replication should work now, 4x replication should only replicate 3x + val newStore1 = makeBlockManager(storeSize, s"newstore1") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a3", 3) === 3) + } + assert(replicateAndGetNumCopies("a4", 4) === 3) + + // Add another store, 4x replication should work now + val newStore2 = makeBlockManager(storeSize, s"newstore2") + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a5", 4) === 4) + } + + // Remove all but the 1st store, 2x replication should fail + (initialStores.tail ++ Seq(newStore1, newStore2)).foreach { + store => + master.removeExecutor(store.blockManagerId.executorId) + store.stop() + } + assert(replicateAndGetNumCopies("a6", 2) === 1) + + // Add new stores, 3x replication should work + val newStores = (3 to 5).map { + i => makeBlockManager(storeSize, s"newstore$i") + } + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + assert(replicateAndGetNumCopies("a7", 3) === 3) + } + } + + /** + * Test replication of blocks with different storage levels (various combinations of + * memory, disk & serialization). For each storage level, this function tests every store + * whether the block is present and also tests the master whether its knowledge of blocks + * is correct. Then it also drops the block from memory of each store (using LRU) and + * again checks whether the master's knowledge gets updated. + */ + private def testReplication(maxReplication: Int, storageLevels: Seq[StorageLevel]) { + import org.apache.spark.storage.StorageLevel._ + + assert(maxReplication > 1, + s"Cannot test replication factor $maxReplication") + + // storage levels to test with the given replication factor + + val storeSize = 10000 + val blockSize = 1000 + + // As many stores as the replication factor + val stores = (1 to maxReplication).map { + i => makeBlockManager(storeSize, s"store$i") + } + + storageLevels.foreach { storageLevel => + // Put the block into one of the stores + val blockId = new TestBlockId( + "block-with-" + storageLevel.description.replace(" ", "-").toLowerCase) + stores(0).putSingle(blockId, new Array[Byte](blockSize), storageLevel) + + // Assert that master know two locations for the block + val blockLocations = master.getLocations(blockId).map(_.executorId).toSet + assert(blockLocations.size === storageLevel.replication, + s"master did not have ${storageLevel.replication} locations for $blockId") + + // Test state of the stores that contain the block + stores.filter { + testStore => blockLocations.contains(testStore.blockManagerId.executorId) + }.foreach { testStore => + val testStoreName = testStore.blockManagerId.executorId + assert(testStore.getLocal(blockId).isDefined, s"$blockId was not found in $testStoreName") + assert(master.getLocations(blockId).map(_.executorId).toSet.contains(testStoreName), + s"master does not have status for ${blockId.name} in $testStoreName") + + val blockStatus = master.getBlockStatus(blockId)(testStore.blockManagerId) + + // Assert that block status in the master for this store has expected storage level + assert( + blockStatus.storageLevel.useDisk === storageLevel.useDisk && + blockStatus.storageLevel.useMemory === storageLevel.useMemory && + blockStatus.storageLevel.useOffHeap === storageLevel.useOffHeap && + blockStatus.storageLevel.deserialized === storageLevel.deserialized, + s"master does not know correct storage level for ${blockId.name} in $testStoreName") + + // Assert that the block status in the master for this store has correct memory usage info + assert(!blockStatus.storageLevel.useMemory || blockStatus.memSize >= blockSize, + s"master does not know size of ${blockId.name} stored in memory of $testStoreName") + + + // If the block is supposed to be in memory, then drop the copy of the block in + // this store test whether master is updated with zero memory usage this store + if (storageLevel.useMemory) { + // Force the block to be dropped by adding a number of dummy blocks + (1 to 10).foreach { + i => + testStore.putSingle(s"dummy-block-$i", new Array[Byte](1000), MEMORY_ONLY_SER) + } + (1 to 10).foreach { + i => testStore.removeBlock(s"dummy-block-$i") + } + + val newBlockStatusOption = master.getBlockStatus(blockId).get(testStore.blockManagerId) + + // Assert that the block status in the master either does not exist (block removed + // from every store) or has zero memory usage for this store + assert( + newBlockStatusOption.isEmpty || newBlockStatusOption.get.memSize === 0, + s"after dropping, master does not know size of ${blockId.name} " + + s"stored in memory of $testStoreName" + ) + } + + // If the block is supposed to be in disk (after dropping or otherwise, then + // test whether master has correct disk usage for this store + if (storageLevel.useDisk) { + assert(master.getBlockStatus(blockId)(testStore.blockManagerId).diskSize >= blockSize, + s"after dropping, master does not know size of ${blockId.name} " + + s"stored in disk of $testStoreName" + ) + } + } + master.removeBlock(blockId) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index e251660dae5de..9d96202a3e7ac 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -21,8 +21,6 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import java.util.Arrays import java.util.concurrent.TimeUnit -import org.apache.spark.network.nio.NioBlockTransferService - import scala.collection.mutable.ArrayBuffer import scala.concurrent.Await import scala.concurrent.duration._ @@ -35,13 +33,13 @@ import akka.util.Timeout import org.mockito.Mockito.{mock, when} -import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ -import org.scalatest.Matchers import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.executor.DataReadMethod +import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.hash.HashShuffleManager @@ -189,7 +187,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter store = makeBlockManager(2000, "exec1") store2 = makeBlockManager(2000, "exec2") - val peers = master.getPeers(store.blockManagerId, 1) + val peers = master.getPeers(store.blockManagerId) assert(peers.size === 1, "master did not return the other manager as a peer") assert(peers.head === store2.blockManagerId, "peer returned by master is not the other manager") @@ -448,7 +446,6 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter val list2DiskGet = store.get("list2disk") assert(list2DiskGet.isDefined, "list2memory expected to be in store") assert(list2DiskGet.get.data.size === 3) - System.out.println(list2DiskGet) // We don't know the exact size of the data on disk, but it should certainly be > 0. assert(list2DiskGet.get.inputMetrics.bytesRead > 0) assert(list2DiskGet.get.inputMetrics.readMethod === DataReadMethod.Disk) From 127e97bee1e6aae7b70263bc5944b7be6f4e6fea Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 2 Oct 2014 13:52:54 -0700 Subject: [PATCH 0875/1492] [SPARK-3632] ConnectionManager can run out of receive threads with authentication on If you turn authentication on and you are using a lot of executors. There is a chance that all the of the threads in the handleMessageExecutor could be waiting to send a message because they are blocked waiting on authentication to happen. This can cause a temporary deadlock until the connection times out. To fix it, I got rid of the wait/notify and use a single outbox but only send security messages from it until authentication has completed. Author: Thomas Graves Closes #2484 from tgravescs/cm_threads_auth and squashes the following commits: a0a961d [Thomas Graves] give it a type b6bc80b [Thomas Graves] Rework comments d6d4175 [Thomas Graves] update from comments 081b765 [Thomas Graves] cleanup 4d7f8f5 [Thomas Graves] Change to not use wait/notify while waiting for authentication --- .../org/apache/spark/SecurityManager.scala | 5 +- .../apache/spark/network/nio/Connection.scala | 65 +++++++++++------ .../spark/network/nio/ConnectionManager.scala | 72 +++++-------------- 3 files changed, 63 insertions(+), 79 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 3832a780ec4bc..0e0f1a7b2377e 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -103,10 +103,9 @@ import org.apache.spark.deploy.SparkHadoopUtil * and a Server, so for a particular connection is has to determine what to do. * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. - * If its acting as a client and trying to send a message to another ConnectionManager, - * it blocks the thread calling sendMessage until the SASL negotiation has occurred. * The ConnectionManager tracks all the sendingConnections using the ConnectionId - * and waits for the response from the server and does the handshake. + * and waits for the response from the server and does the handshake before sending + * the real message. * * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work diff --git a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala index 18172d359cb35..f368209980f93 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Connection.scala @@ -20,23 +20,27 @@ package org.apache.spark.network.nio import java.net._ import java.nio._ import java.nio.channels._ +import java.util.LinkedList import org.apache.spark._ -import scala.collection.mutable.{ArrayBuffer, HashMap, Queue} +import scala.collection.mutable.{ArrayBuffer, HashMap} private[nio] abstract class Connection(val channel: SocketChannel, val selector: Selector, - val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId) + val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId, + val securityMgr: SecurityManager) extends Logging { var sparkSaslServer: SparkSaslServer = null var sparkSaslClient: SparkSaslClient = null - def this(channel_ : SocketChannel, selector_ : Selector, id_ : ConnectionId) = { + def this(channel_ : SocketChannel, selector_ : Selector, id_ : ConnectionId, + securityMgr_ : SecurityManager) = { this(channel_, selector_, ConnectionManagerId.fromSocketAddress( - channel_.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress]), id_) + channel_.socket.getRemoteSocketAddress.asInstanceOf[InetSocketAddress]), + id_, securityMgr_) } channel.configureBlocking(false) @@ -52,14 +56,6 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, val remoteAddress = getRemoteAddress() - /** - * Used to synchronize client requests: client's work-related requests must - * wait until SASL authentication completes. - */ - private val authenticated = new Object() - - def getAuthenticated(): Object = authenticated - def isSaslComplete(): Boolean def resetForceReregister(): Boolean @@ -192,22 +188,22 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, private[nio] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, - remoteId_ : ConnectionManagerId, id_ : ConnectionId) - extends Connection(SocketChannel.open, selector_, remoteId_, id_) { + remoteId_ : ConnectionManagerId, id_ : ConnectionId, + securityMgr_ : SecurityManager) + extends Connection(SocketChannel.open, selector_, remoteId_, id_, securityMgr_) { def isSaslComplete(): Boolean = { if (sparkSaslClient != null) sparkSaslClient.isComplete() else false } private class Outbox { - val messages = new Queue[Message]() + val messages = new LinkedList[Message]() val defaultChunkSize = 65536 var nextMessageToBeUsed = 0 def addMessage(message: Message) { messages.synchronized { - /* messages += message */ - messages.enqueue(message) + messages.add(message) logDebug("Added [" + message + "] to outbox for sending to " + "[" + getRemoteConnectionManagerId() + "]") } @@ -218,10 +214,27 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, while (!messages.isEmpty) { /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ /* val message = messages(nextMessageToBeUsed) */ - val message = messages.dequeue() + + val message = if (securityMgr.isAuthenticationEnabled() && !isSaslComplete()) { + // only allow sending of security messages until sasl is complete + var pos = 0 + var securityMsg: Message = null + while (pos < messages.size() && securityMsg == null) { + if (messages.get(pos).isSecurityNeg) { + securityMsg = messages.remove(pos) + } + pos = pos + 1 + } + // didn't find any security messages and auth isn't completed so return + if (securityMsg == null) return None + securityMsg + } else { + messages.removeFirst() + } + val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { - messages.enqueue(message) + messages.add(message) nextMessageToBeUsed = nextMessageToBeUsed + 1 if (!message.started) { logDebug( @@ -273,6 +286,15 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, changeConnectionKeyInterest(DEFAULT_INTEREST) } + def registerAfterAuth(): Unit = { + outbox.synchronized { + needForceReregister = true + } + if (channel.isConnected) { + registerInterest() + } + } + def send(message: Message) { outbox.synchronized { outbox.addMessage(message) @@ -415,8 +437,9 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, private[spark] class ReceivingConnection( channel_ : SocketChannel, selector_ : Selector, - id_ : ConnectionId) - extends Connection(channel_, selector_, id_) { + id_ : ConnectionId, + securityMgr_ : SecurityManager) + extends Connection(channel_, selector_, id_, securityMgr_) { def isSaslComplete(): Boolean = { if (sparkSaslServer != null) sparkSaslServer.isComplete() else false diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 5aa7e94943561..01cd27a907eea 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -32,7 +32,7 @@ import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.language.postfixOps import org.apache.spark._ -import org.apache.spark.util.{SystemClock, Utils} +import org.apache.spark.util.Utils private[nio] class ConnectionManager( @@ -65,8 +65,6 @@ private[nio] class ConnectionManager( private val selector = SelectorProvider.provider.openSelector() private val ackTimeoutMonitor = new Timer("AckTimeoutMonitor", true) - // default to 30 second timeout waiting for authentication - private val authTimeout = conf.getInt("spark.core.connection.auth.wait.timeout", 30) private val ackTimeout = conf.getInt("spark.core.connection.ack.wait.timeout", 60) private val handleMessageExecutor = new ThreadPoolExecutor( @@ -409,7 +407,8 @@ private[nio] class ConnectionManager( while (newChannel != null) { try { val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) - val newConnection = new ReceivingConnection(newChannel, selector, newConnectionId) + val newConnection = new ReceivingConnection(newChannel, selector, newConnectionId, + securityManager) newConnection.onReceive(receiveMessage) addListeners(newConnection) addConnection(newConnection) @@ -527,9 +526,8 @@ private[nio] class ConnectionManager( if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed for id: " + waitingConn.connectionId) connectionsAwaitingSasl -= waitingConn.connectionId - waitingConn.getAuthenticated().synchronized { - waitingConn.getAuthenticated().notifyAll() - } + waitingConn.registerAfterAuth() + wakeupSelector() return } else { var replyToken : Array[Byte] = null @@ -538,9 +536,8 @@ private[nio] class ConnectionManager( if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed after evaluate for id: " + waitingConn.connectionId) connectionsAwaitingSasl -= waitingConn.connectionId - waitingConn.getAuthenticated().synchronized { - waitingConn.getAuthenticated().notifyAll() - } + waitingConn.registerAfterAuth() + wakeupSelector() return } val securityMsgResp = SecurityMessage.fromResponse(replyToken, @@ -574,9 +571,11 @@ private[nio] class ConnectionManager( } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) if (connection.isSaslComplete()) { - logDebug("Server sasl completed: " + connection.connectionId) + logDebug("Server sasl completed: " + connection.connectionId + + " for: " + connectionId) } else { - logDebug("Server sasl not completed: " + connection.connectionId) + logDebug("Server sasl not completed: " + connection.connectionId + + " for: " + connectionId) } if (replyToken != null) { val securityMsgResp = SecurityMessage.fromResponse(replyToken, @@ -723,7 +722,8 @@ private[nio] class ConnectionManager( if (message == null) throw new Exception("Error creating security message") connectionsAwaitingSasl += ((conn.connectionId, conn)) sendSecurityMessage(connManagerId, message) - logDebug("adding connectionsAwaitingSasl id: " + conn.connectionId) + logDebug("adding connectionsAwaitingSasl id: " + conn.connectionId + + " to: " + connManagerId) } catch { case e: Exception => { logError("Error getting first response from the SaslClient.", e) @@ -744,7 +744,7 @@ private[nio] class ConnectionManager( val inetSocketAddress = new InetSocketAddress(connManagerId.host, connManagerId.port) val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) val newConnection = new SendingConnection(inetSocketAddress, selector, connManagerId, - newConnectionId) + newConnectionId, securityManager) logInfo("creating new sending connection for security! " + newConnectionId ) registerRequests.enqueue(newConnection) @@ -769,61 +769,23 @@ private[nio] class ConnectionManager( connectionManagerId.port) val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId, - newConnectionId) + newConnectionId, securityManager) logTrace("creating new sending connection: " + newConnectionId) registerRequests.enqueue(newConnection) newConnection } val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection()) - if (authEnabled) { - checkSendAuthFirst(connectionManagerId, connection) - } + message.senderAddress = id.toSocketAddress() logDebug("Before Sending [" + message + "] to [" + connectionManagerId + "]" + " " + "connectionid: " + connection.connectionId) if (authEnabled) { - // if we aren't authenticated yet lets block the senders until authentication completes - try { - connection.getAuthenticated().synchronized { - val clock = SystemClock - val startTime = clock.getTime() - - while (!connection.isSaslComplete()) { - logDebug("getAuthenticated wait connectionid: " + connection.connectionId) - // have timeout in case remote side never responds - connection.getAuthenticated().wait(500) - if (((clock.getTime() - startTime) >= (authTimeout * 1000)) - && (!connection.isSaslComplete())) { - // took to long to authenticate the connection, something probably went wrong - throw new Exception("Took to long for authentication to " + connectionManagerId + - ", waited " + authTimeout + "seconds, failing.") - } - } - } - } catch { - case e: Exception => logError("Exception while waiting for authentication.", e) - - // need to tell sender it failed - messageStatuses.synchronized { - val s = messageStatuses.get(message.id) - s match { - case Some(msgStatus) => { - messageStatuses -= message.id - logInfo("Notifying " + msgStatus.connectionManagerId) - msgStatus.markDone(None) - } - case None => { - logError("no messageStatus for failed message id: " + message.id) - } - } - } - } + checkSendAuthFirst(connectionManagerId, connection) } logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") connection.send(message) - wakeupSelector() } From 8081ce8bd111923db143abc55bb6ef9793eece35 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 2 Oct 2014 17:47:56 -0700 Subject: [PATCH 0876/1492] [SPARK-3755][Core] avoid trying privileged port when request a non-privileged port pwendell, ```tryPort``` is not compatible with old code in last PR, this is to fix it. And after discuss with srowen renamed the title to "avoid trying privileged port when request a non-privileged port". Plz refer to the discuss for detail. Author: scwf Closes #2623 from scwf/1-1024 and squashes the following commits: 10a4437 [scwf] add comment de3fd17 [scwf] do not try privileged port when request a non-privileged port 42cb0fa [scwf] make tryPort compatible with old code cb8cc76 [scwf] do not use port 1 - 1024 --- core/src/main/scala/org/apache/spark/util/Utils.scala | 7 ++++++- 1 file changed, 6 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 b3025c6ec3364..9399ddab76331 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1439,7 +1439,12 @@ private[spark] object Utils extends Logging { val serviceString = if (serviceName.isEmpty) "" else s" '$serviceName'" for (offset <- 0 to maxRetries) { // Do not increment port if startPort is 0, which is treated as a special port - val tryPort = if (startPort == 0) startPort else (startPort + offset) % 65536 + val tryPort = if (startPort == 0) { + startPort + } else { + // If the new port wraps around, do not try a privilege port + ((startPort + offset - 1024) % (65536 - 1024)) + 1024 + } try { val (service, port) = startService(tryPort) logInfo(s"Successfully started service$serviceString on port $port.") From 42d5077fd3f2c37d1cd23f4c81aa89286a74cb40 Mon Sep 17 00:00:00 2001 From: Eric Eijkelenboom Date: Thu, 2 Oct 2014 18:04:38 -0700 Subject: [PATCH 0877/1492] [DEPLOY] SPARK-3759: Return the exit code of the driver process SparkSubmitDriverBootstrapper.scala now returns the exit code of the driver process, instead of always returning 0. Author: Eric Eijkelenboom Closes #2628 from ericeijkelenboom/master and squashes the following commits: cc4a571 [Eric Eijkelenboom] Return the exit code of the driver process --- .../apache/spark/deploy/SparkSubmitDriverBootstrapper.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index 38b5d8e1739d0..a64170a47bc1c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -154,7 +154,8 @@ private[spark] object SparkSubmitDriverBootstrapper { process.destroy() } } - process.waitFor() + val returnCode = process.waitFor() + sys.exit(returnCode) } } From 7de4e50a01e90bcf88e0b721b2b15a5162373d56 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 2 Oct 2014 19:32:21 -0700 Subject: [PATCH 0878/1492] [SQL] Initilize session state before creating CommandProcessor With the old ordering it was possible for commands in the HiveDriver to NPE due to the lack of configuration in the threadlocal session state. Author: Michael Armbrust Closes #2635 from marmbrus/initOrder and squashes the following commits: 9749850 [Michael Armbrust] Initilize session state before creating CommandProcessor --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index fdb56901f9ddb..8bcc098bbb620 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -281,13 +281,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { */ protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { try { + // Session state must be initilized before the CommandProcessor is created . + SessionState.start(sessionState) + val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) - SessionState.start(sessionState) - proc match { case driver: Driver => driver.init() From 1c90347a4bba12df7b76d282a7dbac8e555e049f Mon Sep 17 00:00:00 2001 From: ravipesala Date: Thu, 2 Oct 2014 20:04:33 -0700 Subject: [PATCH 0879/1492] [SPARK-3654][SQL] Implement all extended HiveQL statements/commands with a separate parser combinator Created separate parser for hql. It preparses the commands like cache,uncache,add jar etc.. and then parses with HiveQl Author: ravipesala Closes #2590 from ravipesala/SPARK-3654 and squashes the following commits: bbca7dd [ravipesala] Fixed code as per admin comments. ae9290a [ravipesala] Fixed style issues as per Admin comments 898ed81 [ravipesala] Removed spaces fb24edf [ravipesala] Updated the code as per admin comments 8947d37 [ravipesala] Removed duplicate code ba26cd1 [ravipesala] Created seperate parser for hql.It pre parses the commands like cache,uncache,add jar etc.. and then parses with HiveQl --- .../spark/sql/hive/ExtendedHiveQlParser.scala | 135 ++++++++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 57 ++------ .../spark/sql/hive/CachedTableSuite.scala | 6 + 3 files changed, 154 insertions(+), 44 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala new file mode 100644 index 0000000000000..e7e1cb980c2ae --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import scala.language.implicitConversions +import scala.util.parsing.combinator.syntactical.StandardTokenParsers +import scala.util.parsing.combinator.PackratParsers +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.SqlLexical + +/** + * A parser that recognizes all HiveQL constructs together with several Spark SQL specific + * extensions like CACHE TABLE and UNCACHE TABLE. + */ +private[hive] class ExtendedHiveQlParser extends StandardTokenParsers with PackratParsers { + + def apply(input: String): LogicalPlan = { + // Special-case out set commands since the value fields can be + // complex to handle without RegexParsers. Also this approach + // is clearer for the several possible cases of set commands. + if (input.trim.toLowerCase.startsWith("set")) { + input.trim.drop(3).split("=", 2).map(_.trim) match { + case Array("") => // "set" + SetCommand(None, None) + case Array(key) => // "set key" + SetCommand(Some(key), None) + case Array(key, value) => // "set key=value" + SetCommand(Some(key), Some(value)) + } + } else if (input.trim.startsWith("!")) { + ShellCommand(input.drop(1)) + } else { + phrase(query)(new lexical.Scanner(input)) match { + case Success(r, x) => r + case x => sys.error(x.toString) + } + } + } + + protected case class Keyword(str: String) + + protected val CACHE = Keyword("CACHE") + protected val SET = Keyword("SET") + protected val ADD = Keyword("ADD") + protected val JAR = Keyword("JAR") + protected val TABLE = Keyword("TABLE") + protected val AS = Keyword("AS") + protected val UNCACHE = Keyword("UNCACHE") + protected val FILE = Keyword("FILE") + protected val DFS = Keyword("DFS") + protected val SOURCE = Keyword("SOURCE") + + protected implicit def asParser(k: Keyword): Parser[String] = + lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) + + protected def allCaseConverse(k: String): Parser[String] = + lexical.allCaseVersions(k).map(x => x : Parser[String]).reduce(_ | _) + + protected val reservedWords = + this.getClass + .getMethods + .filter(_.getReturnType == classOf[Keyword]) + .map(_.invoke(this).asInstanceOf[Keyword].str) + + override val lexical = new SqlLexical(reservedWords) + + protected lazy val query: Parser[LogicalPlan] = + cache | uncache | addJar | addFile | dfs | source | hiveQl + + protected lazy val hiveQl: Parser[LogicalPlan] = + remainingQuery ^^ { + case r => HiveQl.createPlan(r.trim()) + } + + /** It returns all remaining query */ + protected lazy val remainingQuery: Parser[String] = new Parser[String] { + def apply(in: Input) = + Success( + in.source.subSequence(in.offset, in.source.length).toString, + in.drop(in.source.length())) + } + + /** It returns all query */ + protected lazy val allQuery: Parser[String] = new Parser[String] { + def apply(in: Input) = + Success(in.source.toString, in.drop(in.source.length())) + } + + protected lazy val cache: Parser[LogicalPlan] = + CACHE ~ TABLE ~> ident ~ opt(AS ~> hiveQl) ^^ { + case tableName ~ None => CacheCommand(tableName, true) + case tableName ~ Some(plan) => + CacheTableAsSelectCommand(tableName, plan) + } + + protected lazy val uncache: Parser[LogicalPlan] = + UNCACHE ~ TABLE ~> ident ^^ { + case tableName => CacheCommand(tableName, false) + } + + protected lazy val addJar: Parser[LogicalPlan] = + ADD ~ JAR ~> remainingQuery ^^ { + case rq => AddJar(rq.trim()) + } + + protected lazy val addFile: Parser[LogicalPlan] = + ADD ~ FILE ~> remainingQuery ^^ { + case rq => AddFile(rq.trim()) + } + + protected lazy val dfs: Parser[LogicalPlan] = + DFS ~> allQuery ^^ { + case aq => NativeCommand(aq.trim()) + } + + protected lazy val source: Parser[LogicalPlan] = + SOURCE ~> remainingQuery ^^ { + case rq => SourceCommand(rq.trim()) + } +} 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 4f3f808c93dc8..6bb42eeb0550d 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 @@ -126,6 +126,9 @@ private[hive] object HiveQl { "TOK_CREATETABLE", "TOK_DESCTABLE" ) ++ nativeCommands + + // It parses hive sql query along with with several Spark SQL specific extensions + protected val hiveSqlParser = new ExtendedHiveQlParser /** * A set of implicit transformations that allow Hive ASTNodes to be rewritten by transformations @@ -215,40 +218,19 @@ private[hive] object HiveQl { def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql)) /** Returns a LogicalPlan for a given HiveQL string. */ - def parseSql(sql: String): LogicalPlan = { + def parseSql(sql: String): LogicalPlan = hiveSqlParser(sql) + + /** Creates LogicalPlan for a given HiveQL string. */ + def createPlan(sql: String) = { try { - if (sql.trim.toLowerCase.startsWith("set")) { - // Split in two parts since we treat the part before the first "=" - // as key, and the part after as value, which may contain other "=" signs. - sql.trim.drop(3).split("=", 2).map(_.trim) match { - case Array("") => // "set" - SetCommand(None, None) - case Array(key) => // "set key" - SetCommand(Some(key), None) - case Array(key, value) => // "set key=value" - SetCommand(Some(key), Some(value)) - } - } else if (sql.trim.toLowerCase.startsWith("cache table")) { - sql.trim.drop(12).trim.split(" ").toSeq match { - case Seq(tableName) => - CacheCommand(tableName, true) - case Seq(tableName, _, select @ _*) => - CacheTableAsSelectCommand(tableName, createPlan(select.mkString(" ").trim)) - } - } else if (sql.trim.toLowerCase.startsWith("uncache table")) { - CacheCommand(sql.trim.drop(14).trim, false) - } else if (sql.trim.toLowerCase.startsWith("add jar")) { - AddJar(sql.trim.drop(8).trim) - } else if (sql.trim.toLowerCase.startsWith("add file")) { - AddFile(sql.trim.drop(9)) - } else if (sql.trim.toLowerCase.startsWith("dfs")) { + val tree = getAst(sql) + if (nativeCommands contains tree.getText) { NativeCommand(sql) - } else if (sql.trim.startsWith("source")) { - SourceCommand(sql.split(" ").toSeq match { case Seq("source", filePath) => filePath }) - } else if (sql.trim.startsWith("!")) { - ShellCommand(sql.drop(1)) } else { - createPlan(sql) + nodeToPlan(tree) match { + case NativePlaceholder => NativeCommand(sql) + case other => other + } } } catch { case e: Exception => throw new ParseException(sql, e) @@ -259,19 +241,6 @@ private[hive] object HiveQl { """.stripMargin) } } - - /** Creates LogicalPlan for a given HiveQL string. */ - def createPlan(sql: String) = { - val tree = getAst(sql) - if (nativeCommands contains tree.getText) { - NativeCommand(sql) - } else { - nodeToPlan(tree) match { - case NativePlaceholder => NativeCommand(sql) - case other => other - } - } - } def parseDdl(ddl: String): Seq[Attribute] = { val tree = diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 188579edd7bdd..b3057cd618c66 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -88,4 +88,10 @@ class CachedTableSuite extends HiveComparisonTest { } assert(!TestHive.isCached("src"), "Table 'src' should not be cached") } + + test("'CACHE TABLE tableName AS SELECT ..'") { + TestHive.sql("CACHE TABLE testCacheTable AS SELECT * FROM src") + assert(TestHive.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + TestHive.uncacheTable("testCacheTable") + } } From 2e4eae3a52e3d04895b00447d1ac56ae3c1b98ae Mon Sep 17 00:00:00 2001 From: "qiping.lqp" Date: Fri, 3 Oct 2014 03:26:17 -0700 Subject: [PATCH 0880/1492] [SPARK-3366][MLLIB]Compute best splits distributively in decision tree Currently, all best splits are computed on the driver, which makes the driver a bottleneck for both communication and computation. This PR fix this problem by computed best splits on executors. Instead of send all aggregate stats to the driver node, we can send aggregate stats for a node to a particular executor, using `reduceByKey` operation, then we can compute best split for this node there. Implementation details: Each node now has a nodeStatsAggregator, which save aggregate stats for all features and bins. First use mapPartition to compute node aggregate stats for all nodes in each partition. Then transform node aggregate stats to (nodeIndex, nodeStatsAggregator) pairs and use to `reduceByKey` operation to combine nodeStatsAggregator for the same node. After all stats have been combined, best splits can be computed for each node based on the node aggregate stats. Best split result is collected to driver to construct the decision tree. CC: mengxr manishamde jkbradley, please help me review this, thanks. Author: qiping.lqp Author: chouqin Closes #2595 from chouqin/dt-dist-agg and squashes the following commits: db0d24a [chouqin] fix a minor bug and adjust code a0d9de3 [chouqin] adjust code based on comments 9f201a6 [chouqin] fix bug: statsSize -> allStatsSize a8a7ed0 [chouqin] Merge branch 'master' of https://github.com/apache/spark into dt-dist-agg f13b346 [chouqin] adjust randomforest comments c32636e [chouqin] adjust code based on comments ac6a505 [chouqin] adjust code based on comments 7bbb787 [chouqin] add comments bdd2a63 [qiping.lqp] fix test suite a75df27 [qiping.lqp] fix test suite b5b0bc2 [qiping.lqp] fix style e76414f [qiping.lqp] fix testsuite 748bd45 [qiping.lqp] fix type-mismatch bug 24eacd8 [qiping.lqp] fix type-mismatch bug 5f63d6c [qiping.lqp] add multiclassification using One-Vs-All strategy 4f56496 [qiping.lqp] fix bug f00fc22 [qiping.lqp] fix bug 532993a [qiping.lqp] Compute best splits distributively in decision tree --- .../spark/mllib/tree/DecisionTree.scala | 140 ++++++--- .../spark/mllib/tree/RandomForest.scala | 5 +- .../mllib/tree/impl/DTStatsAggregator.scala | 292 +++++------------- .../tree/model/InformationGainStats.scala | 11 + .../spark/mllib/tree/RandomForestSuite.scala | 1 + 5 files changed, 182 insertions(+), 267 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index b7dc373ebd9cc..b311d10023894 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -23,7 +23,6 @@ import scala.collection.mutable import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.Logging -import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.configuration.Strategy @@ -36,6 +35,7 @@ import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.SparkContext._ /** @@ -328,9 +328,8 @@ object DecisionTree extends Serializable with Logging { * for each subset is updated. * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (node, feature, bin). + * each (feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. agg is indexed in [0, numNodes). * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param unorderedFeatures Set of indices of unordered features. * @param instanceWeight Weight (importance) of instance in dataset. @@ -338,7 +337,6 @@ object DecisionTree extends Serializable with Logging { private def mixedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - nodeIndex: Int, bins: Array[Array[Bin]], unorderedFeatures: Set[Int], instanceWeight: Double, @@ -350,7 +348,6 @@ object DecisionTree extends Serializable with Logging { // Use all features agg.metadata.numFeatures } - val nodeOffset = agg.getNodeOffset(nodeIndex) // Iterate over features. var featureIndexIdx = 0 while (featureIndexIdx < numFeaturesPerNode) { @@ -363,16 +360,16 @@ object DecisionTree extends Serializable with Logging { // Unordered feature val featureValue = treePoint.binnedFeatures(featureIndex) val (leftNodeFeatureOffset, rightNodeFeatureOffset) = - agg.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndexIdx) + agg.getLeftRightFeatureOffsets(featureIndexIdx) // Update the left or right bin for each split. val numSplits = agg.metadata.numSplits(featureIndex) var splitIndex = 0 while (splitIndex < numSplits) { if (bins(featureIndex)(splitIndex).highSplit.categories.contains(featureValue)) { - agg.nodeFeatureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, + agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) } else { - agg.nodeFeatureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, + agg.featureUpdate(rightNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) } splitIndex += 1 @@ -380,8 +377,7 @@ object DecisionTree extends Serializable with Logging { } else { // Ordered feature val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, nodeIndex, featureIndexIdx, binIndex, treePoint.label, - instanceWeight) + agg.update(featureIndexIdx, binIndex, treePoint.label, instanceWeight) } featureIndexIdx += 1 } @@ -393,26 +389,24 @@ object DecisionTree extends Serializable with Logging { * For each feature, the sufficient statistics of one bin are updated. * * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (node, feature, bin). + * each (feature, bin). * @param treePoint Data point being aggregated. - * @param nodeIndex Node corresponding to treePoint. agg is indexed in [0, numNodes). * @param instanceWeight Weight (importance) of instance in dataset. */ private def orderedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - nodeIndex: Int, instanceWeight: Double, featuresForNode: Option[Array[Int]]): Unit = { val label = treePoint.label - val nodeOffset = agg.getNodeOffset(nodeIndex) + // Iterate over features. if (featuresForNode.nonEmpty) { // Use subsampled features var featureIndexIdx = 0 while (featureIndexIdx < featuresForNode.get.size) { val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx)) - agg.nodeUpdate(nodeOffset, nodeIndex, featureIndexIdx, binIndex, label, instanceWeight) + agg.update(featureIndexIdx, binIndex, label, instanceWeight) featureIndexIdx += 1 } } else { @@ -421,7 +415,7 @@ object DecisionTree extends Serializable with Logging { var featureIndex = 0 while (featureIndex < numFeatures) { val binIndex = treePoint.binnedFeatures(featureIndex) - agg.nodeUpdate(nodeOffset, nodeIndex, featureIndex, binIndex, label, instanceWeight) + agg.update(featureIndex, binIndex, label, instanceWeight) featureIndex += 1 } } @@ -496,8 +490,8 @@ object DecisionTree extends Serializable with Logging { * @return agg */ def binSeqOp( - agg: DTStatsAggregator, - baggedPoint: BaggedPoint[TreePoint]): DTStatsAggregator = { + agg: Array[DTStatsAggregator], + baggedPoint: BaggedPoint[TreePoint]): Array[DTStatsAggregator] = { treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures, bins, metadata.unorderedFeatures) @@ -508,9 +502,9 @@ object DecisionTree extends Serializable with Logging { val featuresForNode = nodeInfo.featureSubset val instanceWeight = baggedPoint.subsampleWeights(treeIndex) if (metadata.unorderedFeatures.isEmpty) { - orderedBinSeqOp(agg, baggedPoint.datum, aggNodeIndex, instanceWeight, featuresForNode) + orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) } else { - mixedBinSeqOp(agg, baggedPoint.datum, aggNodeIndex, bins, metadata.unorderedFeatures, + mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, bins, metadata.unorderedFeatures, instanceWeight, featuresForNode) } } @@ -518,30 +512,76 @@ object DecisionTree extends Serializable with Logging { agg } - // Calculate bin aggregates. - timer.start("aggregation") - val binAggregates: DTStatsAggregator = { - val initAgg = if (metadata.subsamplingFeatures) { - new DTStatsAggregatorSubsampledFeatures(metadata, treeToNodeToIndexInfo) - } else { - new DTStatsAggregatorFixedFeatures(metadata, numNodes) + /** + * Get node index in group --> features indices map, + * which is a short cut to find feature indices for a node given node index in group + * @param treeToNodeToIndexInfo + * @return + */ + def getNodeToFeatures(treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]]) + : Option[Map[Int, Array[Int]]] = if (!metadata.subsamplingFeatures) { + None + } else { + val mutableNodeToFeatures = new mutable.HashMap[Int, Array[Int]]() + treeToNodeToIndexInfo.values.foreach { nodeIdToNodeInfo => + nodeIdToNodeInfo.values.foreach { nodeIndexInfo => + assert(nodeIndexInfo.featureSubset.isDefined) + mutableNodeToFeatures(nodeIndexInfo.nodeIndexInGroup) = nodeIndexInfo.featureSubset.get + } } - input.treeAggregate(initAgg)(binSeqOp, DTStatsAggregator.binCombOp) + Some(mutableNodeToFeatures.toMap) } - timer.stop("aggregation") // Calculate best splits for all nodes in the group timer.start("chooseSplits") + // In each partition, iterate all instances and compute aggregate stats for each node, + // yield an (nodeIndex, nodeAggregateStats) pair for each node. + // After a `reduceByKey` operation, + // stats of a node will be shuffled to a particular partition and be combined together, + // then best splits for nodes are found there. + // Finally, only best Splits for nodes are collected to driver to construct decision tree. + val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) + val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) + val nodeToBestSplits = + input.mapPartitions { points => + // Construct a nodeStatsAggregators array to hold node aggregate stats, + // each node will have a nodeStatsAggregator + val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex => + val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => + Some(nodeToFeatures(nodeIndex)) + } + new DTStatsAggregator(metadata, featuresForNode) + } + + // iterator all instances in current partition and update aggregate stats + points.foreach(binSeqOp(nodeStatsAggregators, _)) + + // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs, + // which can be combined with other partition using `reduceByKey` + nodeStatsAggregators.view.zipWithIndex.map(_.swap).iterator + }.reduceByKey((a, b) => a.merge(b)) + .map { case (nodeIndex, aggStats) => + val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => + Some(nodeToFeatures(nodeIndex)) + } + + // find best split for each node + val (split: Split, stats: InformationGainStats, predict: Predict) = + binsToBestSplit(aggStats, splits, featuresForNode) + (nodeIndex, (split, stats, predict)) + }.collectAsMap() + + timer.stop("chooseSplits") + // Iterate over all nodes in this group. nodesForGroup.foreach { case (treeIndex, nodesForTree) => nodesForTree.foreach { node => val nodeIndex = node.id val nodeInfo = treeToNodeToIndexInfo(treeIndex)(nodeIndex) val aggNodeIndex = nodeInfo.nodeIndexInGroup - val featuresForNode = nodeInfo.featureSubset val (split: Split, stats: InformationGainStats, predict: Predict) = - binsToBestSplit(binAggregates, aggNodeIndex, splits, featuresForNode) + nodeToBestSplits(aggNodeIndex) logDebug("best split = " + split) // Extract info for this node. Create children if not leaf. @@ -565,7 +605,7 @@ object DecisionTree extends Serializable with Logging { } } } - timer.stop("chooseSplits") + } /** @@ -633,36 +673,33 @@ object DecisionTree extends Serializable with Logging { /** * Find the best split for a node. * @param binAggregates Bin statistics. - * @param nodeIndex Index into aggregates for node to split in this group. * @return tuple for best split: (Split, information gain, prediction at node) */ private def binsToBestSplit( binAggregates: DTStatsAggregator, - nodeIndex: Int, splits: Array[Array[Split]], featuresForNode: Option[Array[Int]]): (Split, InformationGainStats, Predict) = { - val metadata: DecisionTreeMetadata = binAggregates.metadata - // calculate predict only once var predict: Option[Predict] = None // For each (feature, split), calculate the gain, and select the best (feature, split). - val (bestSplit, bestSplitStats) = Range(0, metadata.numFeaturesPerNode).map { featureIndexIdx => + val (bestSplit, bestSplitStats) = + Range(0, binAggregates.metadata.numFeaturesPerNode).map { featureIndexIdx => val featureIndex = if (featuresForNode.nonEmpty) { featuresForNode.get.apply(featureIndexIdx) } else { featureIndexIdx } - val numSplits = metadata.numSplits(featureIndex) - if (metadata.isContinuous(featureIndex)) { + val numSplits = binAggregates.metadata.numSplits(featureIndex) + if (binAggregates.metadata.isContinuous(featureIndex)) { // Cumulative sum (scanLeft) of bin statistics. // Afterwards, binAggregates for a bin is the sum of aggregates for // that bin + all preceding bins. - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndexIdx) + val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) var splitIndex = 0 while (splitIndex < numSplits) { - binAggregates.mergeForNodeFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) + binAggregates.mergeForFeature(nodeFeatureOffset, splitIndex + 1, splitIndex) splitIndex += 1 } // Find best split. @@ -672,27 +709,29 @@ object DecisionTree extends Serializable with Logging { val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIdx, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) - } else if (metadata.isUnordered(featureIndex)) { + } else if (binAggregates.metadata.isUnordered(featureIndex)) { // Unordered categorical feature val (leftChildOffset, rightChildOffset) = - binAggregates.getLeftRightNodeFeatureOffsets(nodeIndex, featureIndexIdx) + binAggregates.getLeftRightFeatureOffsets(featureIndexIdx) val (bestFeatureSplitIndex, bestFeatureGainStats) = Range(0, numSplits).map { splitIndex => val leftChildStats = binAggregates.getImpurityCalculator(leftChildOffset, splitIndex) val rightChildStats = binAggregates.getImpurityCalculator(rightChildOffset, splitIndex) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) (splits(featureIndex)(bestFeatureSplitIndex), bestFeatureGainStats) } else { // Ordered categorical feature - val nodeFeatureOffset = binAggregates.getNodeFeatureOffset(nodeIndex, featureIndexIdx) - val numBins = metadata.numBins(featureIndex) + val nodeFeatureOffset = binAggregates.getFeatureOffset(featureIndexIdx) + val numBins = binAggregates.metadata.numBins(featureIndex) /* Each bin is one category (feature value). * The bins are ordered based on centroidForCategories, and this ordering determines which @@ -700,7 +739,7 @@ object DecisionTree extends Serializable with Logging { * * centroidForCategories is a list: (category, centroid) */ - val centroidForCategories = if (metadata.isMulticlass) { + val centroidForCategories = if (binAggregates.metadata.isMulticlass) { // For categorical variables in multiclass classification, // the bins are ordered by the impurity of their corresponding labels. Range(0, numBins).map { case featureValue => @@ -741,7 +780,7 @@ object DecisionTree extends Serializable with Logging { while (splitIndex < numSplits) { val currentCategory = categoriesSortedByCentroid(splitIndex)._1 val nextCategory = categoriesSortedByCentroid(splitIndex + 1)._1 - binAggregates.mergeForNodeFeature(nodeFeatureOffset, nextCategory, currentCategory) + binAggregates.mergeForFeature(nodeFeatureOffset, nextCategory, currentCategory) splitIndex += 1 } // lastCategory = index of bin with total aggregates for this (node, feature) @@ -756,7 +795,8 @@ object DecisionTree extends Serializable with Logging { binAggregates.getImpurityCalculator(nodeFeatureOffset, lastCategory) rightChildStats.subtract(leftChildStats) predict = Some(predict.getOrElse(calculatePredict(leftChildStats, rightChildStats))) - val gainStats = calculateGainForSplit(leftChildStats, rightChildStats, metadata) + val gainStats = calculateGainForSplit(leftChildStats, + rightChildStats, binAggregates.metadata) (splitIndex, gainStats) }.maxBy(_._2.gain) val categoriesForSplit = diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 7fa7725e79e46..fa7a26f17c3ca 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -171,8 +171,8 @@ private class RandomForest ( // Choose node splits, and enqueue new nodes as needed. timer.start("findBestSplits") - DecisionTree.findBestSplits(baggedInput, - metadata, topNodes, nodesForGroup, treeToNodeToIndexInfo, splits, bins, nodeQueue, timer) + DecisionTree.findBestSplits(baggedInput, metadata, topNodes, nodesForGroup, + treeToNodeToIndexInfo, splits, bins, nodeQueue, timer) timer.stop("findBestSplits") } @@ -382,6 +382,7 @@ object RandomForest extends Serializable with Logging { * @param maxMemoryUsage Bound on size of aggregate statistics. * @return (nodesForGroup, treeToNodeToIndexInfo). * nodesForGroup holds the nodes to split: treeIndex --> nodes in tree. + * * treeToNodeToIndexInfo holds indices selected features for each node: * treeIndex --> (global) node index --> (node index in group, feature indices). * The (global) node index is the index in the tree; the node index in group is the diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index d49df7a016375..55f422dff0d71 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -17,17 +17,19 @@ package org.apache.spark.mllib.tree.impl -import org.apache.spark.mllib.tree.RandomForest.NodeIndexInfo import org.apache.spark.mllib.tree.impurity._ + + /** - * DecisionTree statistics aggregator. - * This holds a flat array of statistics for a set of (nodes, features, bins) + * DecisionTree statistics aggregator for a node. + * This holds a flat array of statistics for a set of (features, bins) * and helps with indexing. * This class is abstract to support learning with and without feature subsampling. */ -private[tree] abstract class DTStatsAggregator( - val metadata: DecisionTreeMetadata) extends Serializable { +private[tree] class DTStatsAggregator( + val metadata: DecisionTreeMetadata, + featureSubset: Option[Array[Int]]) extends Serializable { /** * [[ImpurityAggregator]] instance specifying the impurity type. @@ -42,7 +44,25 @@ private[tree] abstract class DTStatsAggregator( /** * Number of elements (Double values) used for the sufficient statistics of each bin. */ - val statsSize: Int = impurityAggregator.statsSize + private val statsSize: Int = impurityAggregator.statsSize + + /** + * Number of bins for each feature. This is indexed by the feature index. + */ + private val numBins: Array[Int] = { + if (featureSubset.isDefined) { + featureSubset.get.map(metadata.numBins(_)) + } else { + metadata.numBins + } + } + + /** + * Offset for each feature for calculating indices into the [[allStats]] array. + */ + private val featureOffsets: Array[Int] = { + numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) + } /** * Indicator for each feature of whether that feature is an unordered feature. @@ -51,107 +71,95 @@ private[tree] abstract class DTStatsAggregator( def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) /** - * Total number of elements stored in this aggregator. + * Total number of elements stored in this aggregator */ - def allStatsSize: Int + private val allStatsSize: Int = featureOffsets.last /** - * Get flat array of elements stored in this aggregator. + * Flat array of elements. + * Index for start of stats for a (feature, bin) is: + * index = featureOffsets(featureIndex) + binIndex * statsSize + * Note: For unordered features, + * the left child stats have binIndex in [0, numBins(featureIndex) / 2)) + * and the right child stats in [numBins(featureIndex) / 2), numBins(featureIndex)) */ - protected def allStats: Array[Double] + private val allStats: Array[Double] = new Array[Double](allStatsSize) + /** * Get an [[ImpurityCalculator]] for a given (node, feature, bin). - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. + * @param featureOffset For ordered features, this is a pre-computed (node, feature) offset + * from [[getFeatureOffset]]. * For unordered features, this is a pre-computed * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. + * [[getLeftRightFeatureOffsets]]. */ - def getImpurityCalculator(nodeFeatureOffset: Int, binIndex: Int): ImpurityCalculator = { - impurityAggregator.getCalculator(allStats, nodeFeatureOffset + binIndex * statsSize) + def getImpurityCalculator(featureOffset: Int, binIndex: Int): ImpurityCalculator = { + impurityAggregator.getCalculator(allStats, featureOffset + binIndex * statsSize) } /** - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. + * Update the stats for a given (feature, bin) for ordered features, using the given label. */ - def update( - nodeIndex: Int, - featureIndex: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - val i = getNodeFeatureOffset(nodeIndex, featureIndex) + binIndex * statsSize + def update(featureIndex: Int, binIndex: Int, label: Double, instanceWeight: Double): Unit = { + val i = featureOffsets(featureIndex) + binIndex * statsSize impurityAggregator.update(allStats, i, label, instanceWeight) } - /** - * Pre-compute node offset for use with [[nodeUpdate]]. - */ - def getNodeOffset(nodeIndex: Int): Int - /** * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. - * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. + * Update the stats for a given (feature, bin), using the given label. + * @param featureOffset For ordered features, this is a pre-computed feature offset + * from [[getFeatureOffset]]. + * For unordered features, this is a pre-computed + * (feature, left/right child) offset from + * [[getLeftRightFeatureOffsets]]. */ - def nodeUpdate( - nodeOffset: Int, - nodeIndex: Int, - featureIndex: Int, + def featureUpdate( + featureOffset: Int, binIndex: Int, label: Double, - instanceWeight: Double): Unit + instanceWeight: Double): Unit = { + impurityAggregator.update(allStats, featureOffset + binIndex * statsSize, + label, instanceWeight) + } /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * Pre-compute feature offset for use with [[featureUpdate]]. * For ordered features only. */ - def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int + def getFeatureOffset(featureIndex: Int): Int = { + require(!isUnordered(featureIndex), + s"DTStatsAggregator.getFeatureOffset is for ordered features only, but was called" + + s" for unordered feature $featureIndex.") + featureOffsets(featureIndex) + } /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. + * Pre-compute feature offset for use with [[featureUpdate]]. * For unordered features only. */ - def getLeftRightNodeFeatureOffsets(nodeIndex: Int, featureIndex: Int): (Int, Int) = { + def getLeftRightFeatureOffsets(featureIndex: Int): (Int, Int) = { require(isUnordered(featureIndex), - s"DTStatsAggregator.getLeftRightNodeFeatureOffsets is for unordered features only," + + s"DTStatsAggregator.getLeftRightFeatureOffsets is for unordered features only," + s" but was called for ordered feature $featureIndex.") - val baseOffset = getNodeFeatureOffset(nodeIndex, featureIndex) - (baseOffset, baseOffset + (metadata.numBins(featureIndex) >> 1) * statsSize) - } - - /** - * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin), using the given label. - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. - * For unordered features, this is a pre-computed - * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. - */ - def nodeFeatureUpdate( - nodeFeatureOffset: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - impurityAggregator.update(allStats, nodeFeatureOffset + binIndex * statsSize, label, - instanceWeight) + val baseOffset = featureOffsets(featureIndex) + (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) } /** - * For a given (node, feature), merge the stats for two bins. - * @param nodeFeatureOffset For ordered features, this is a pre-computed (node, feature) offset - * from [[getNodeFeatureOffset]]. + * For a given feature, merge the stats for two bins. + * @param featureOffset For ordered features, this is a pre-computed feature offset + * from [[getFeatureOffset]]. * For unordered features, this is a pre-computed - * (node, feature, left/right child) offset from - * [[getLeftRightNodeFeatureOffsets]]. + * (feature, left/right child) offset from + * [[getLeftRightFeatureOffsets]]. * @param binIndex The other bin is merged into this bin. * @param otherBinIndex This bin is not modified. */ - def mergeForNodeFeature(nodeFeatureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { - impurityAggregator.merge(allStats, nodeFeatureOffset + binIndex * statsSize, - nodeFeatureOffset + otherBinIndex * statsSize) + def mergeForFeature(featureOffset: Int, binIndex: Int, otherBinIndex: Int): Unit = { + impurityAggregator.merge(allStats, featureOffset + binIndex * statsSize, + featureOffset + otherBinIndex * statsSize) } /** @@ -161,7 +169,7 @@ private[tree] abstract class DTStatsAggregator( def merge(other: DTStatsAggregator): DTStatsAggregator = { require(allStatsSize == other.allStatsSize, s"DTStatsAggregator.merge requires that both aggregators have the same length stats vectors." - + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") + + s" This aggregator is of length $allStatsSize, but the other is ${other.allStatsSize}.") var i = 0 // TODO: Test BLAS.axpy while (i < allStatsSize) { @@ -171,149 +179,3 @@ private[tree] abstract class DTStatsAggregator( this } } - -/** - * DecisionTree statistics aggregator. - * This holds a flat array of statistics for a set of (nodes, features, bins) - * and helps with indexing. - * - * This instance of [[DTStatsAggregator]] is used when not subsampling features. - * - * @param numNodes Number of nodes to collect statistics for. - */ -private[tree] class DTStatsAggregatorFixedFeatures( - metadata: DecisionTreeMetadata, - numNodes: Int) extends DTStatsAggregator(metadata) { - - /** - * Offset for each feature for calculating indices into the [[allStats]] array. - * Mapping: featureIndex --> offset - */ - private val featureOffsets: Array[Int] = { - metadata.numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) - } - - /** - * Number of elements for each node, corresponding to stride between nodes in [[allStats]]. - */ - private val nodeStride: Int = featureOffsets.last - - override val allStatsSize: Int = numNodes * nodeStride - - /** - * Flat array of elements. - * Index for start of stats for a (node, feature, bin) is: - * index = nodeIndex * nodeStride + featureOffsets(featureIndex) + binIndex * statsSize - * Note: For unordered features, the left child stats precede the right child stats - * in the binIndex order. - */ - override protected val allStats: Array[Double] = new Array[Double](allStatsSize) - - override def getNodeOffset(nodeIndex: Int): Int = nodeIndex * nodeStride - - override def nodeUpdate( - nodeOffset: Int, - nodeIndex: Int, - featureIndex: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - val i = nodeOffset + featureOffsets(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label, instanceWeight) - } - - override def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { - nodeIndex * nodeStride + featureOffsets(featureIndex) - } -} - -/** - * DecisionTree statistics aggregator. - * This holds a flat array of statistics for a set of (nodes, features, bins) - * and helps with indexing. - * - * This instance of [[DTStatsAggregator]] is used when subsampling features. - * - * @param treeToNodeToIndexInfo Mapping: treeIndex --> nodeIndex --> nodeIndexInfo, - * where nodeIndexInfo stores the index in the group and the - * feature subsets (if using feature subsets). - */ -private[tree] class DTStatsAggregatorSubsampledFeatures( - metadata: DecisionTreeMetadata, - treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]]) extends DTStatsAggregator(metadata) { - - /** - * For each node, offset for each feature for calculating indices into the [[allStats]] array. - * Mapping: nodeIndex --> featureIndex --> offset - */ - private val featureOffsets: Array[Array[Int]] = { - val numNodes: Int = treeToNodeToIndexInfo.values.map(_.size).sum - val offsets = new Array[Array[Int]](numNodes) - treeToNodeToIndexInfo.foreach { case (treeIndex, nodeToIndexInfo) => - nodeToIndexInfo.foreach { case (globalNodeIndex, nodeInfo) => - offsets(nodeInfo.nodeIndexInGroup) = nodeInfo.featureSubset.get.map(metadata.numBins(_)) - .scanLeft(0)((total, nBins) => total + statsSize * nBins) - } - } - offsets - } - - /** - * For each node, offset for each feature for calculating indices into the [[allStats]] array. - */ - protected val nodeOffsets: Array[Int] = featureOffsets.map(_.last).scanLeft(0)(_ + _) - - override val allStatsSize: Int = nodeOffsets.last - - /** - * Flat array of elements. - * Index for start of stats for a (node, feature, bin) is: - * index = nodeOffsets(nodeIndex) + featureOffsets(featureIndex) + binIndex * statsSize - * Note: For unordered features, the left child stats precede the right child stats - * in the binIndex order. - */ - override protected val allStats: Array[Double] = new Array[Double](allStatsSize) - - override def getNodeOffset(nodeIndex: Int): Int = nodeOffsets(nodeIndex) - - /** - * Faster version of [[update]]. - * Update the stats for a given (node, feature, bin) for ordered features, using the given label. - * @param nodeOffset Pre-computed node offset from [[getNodeOffset]]. - * @param featureIndex Index of feature in featuresForNodes(nodeIndex). - * Note: This is NOT the original feature index. - */ - override def nodeUpdate( - nodeOffset: Int, - nodeIndex: Int, - featureIndex: Int, - binIndex: Int, - label: Double, - instanceWeight: Double): Unit = { - val i = nodeOffset + featureOffsets(nodeIndex)(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label, instanceWeight) - } - - /** - * Pre-compute (node, feature) offset for use with [[nodeFeatureUpdate]]. - * For ordered features only. - * @param featureIndex Index of feature in featuresForNodes(nodeIndex). - * Note: This is NOT the original feature index. - */ - override def getNodeFeatureOffset(nodeIndex: Int, featureIndex: Int): Int = { - nodeOffsets(nodeIndex) + featureOffsets(nodeIndex)(featureIndex) - } -} - -private[tree] object DTStatsAggregator extends Serializable { - - /** - * Combines two aggregates (modifying the first) and returns the combination. - */ - def binCombOp( - agg1: DTStatsAggregator, - agg2: DTStatsAggregator): DTStatsAggregator = { - agg1.merge(agg2) - } - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index f3e2619bd8ba0..a89e71e115806 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -38,6 +38,17 @@ class InformationGainStats( "gain = %f, impurity = %f, left impurity = %f, right impurity = %f" .format(gain, impurity, leftImpurity, rightImpurity) } + + override def equals(o: Any) = + o match { + case other: InformationGainStats => { + gain == other.gain && + impurity == other.impurity && + leftImpurity == other.leftImpurity && + rightImpurity == other.rightImpurity + } + case _ => false + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 30669fcd1c75b..20d372dc1d3ca 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -145,6 +145,7 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { assert(nodesForGroup.size === numTrees, failString) assert(nodesForGroup.values.forall(_.size == 1), failString) // 1 node per tree + if (numFeaturesPerNode == numFeatures) { // featureSubset values should all be None assert(treeToNodeToIndexInfo.values.forall(_.values.forall(_.featureSubset.isEmpty)), From f0811f928e5b608e1a2cba3b6828ba0ed03b701d Mon Sep 17 00:00:00 2001 From: EugenCepoi Date: Fri, 3 Oct 2014 10:03:15 -0700 Subject: [PATCH 0881/1492] SPARK-2058: Overriding SPARK_HOME/conf with SPARK_CONF_DIR Update of PR #997. With this PR, setting SPARK_CONF_DIR overrides SPARK_HOME/conf (not only spark-defaults.conf and spark-env). Author: EugenCepoi Closes #2481 from EugenCepoi/SPARK-2058 and squashes the following commits: 0bb32c2 [EugenCepoi] use orElse orNull and fixing trailing percent in compute-classpath.cmd 77f35d7 [EugenCepoi] SPARK-2058: Overriding SPARK_HOME/conf with SPARK_CONF_DIR --- bin/compute-classpath.cmd | 8 +++- bin/compute-classpath.sh | 8 +++- .../spark/deploy/SparkSubmitArguments.scala | 42 ++++++++----------- .../spark/deploy/SparkSubmitSuite.scala | 34 ++++++++++++++- docs/configuration.md | 7 ++++ 5 files changed, 71 insertions(+), 28 deletions(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 5ad52452a5c98..9b9e40321ea93 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -36,7 +36,13 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH%;%FWDIR%conf +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% + +if "x%SPARK_CONF_DIR%"!="x" ( + set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% +) else ( + set CLASSPATH=%CLASSPATH%;%FWDIR%conf +) if exist "%FWDIR%RELEASE" ( for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 0f63e36d8aeca..905bbaf99b374 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -27,8 +27,14 @@ FWDIR="$(cd "`dirname "$0"`"/..; pwd)" . "$FWDIR"/bin/load-spark-env.sh +CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH" + # Build up classpath -CLASSPATH="$SPARK_CLASSPATH:$SPARK_SUBMIT_CLASSPATH:$FWDIR/conf" +if [ -n "$SPARK_CONF_DIR" ]; then + CLASSPATH="$CLASSPATH:$SPARK_CONF_DIR" +else + CLASSPATH="$CLASSPATH:$FWDIR/conf" +fi ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 2b72c61cc8177..57b251ff47714 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -29,8 +29,9 @@ import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. + * The env argument is used for testing. */ -private[spark] class SparkSubmitArguments(args: Seq[String]) { +private[spark] class SparkSubmitArguments(args: Seq[String], env: Map[String, String] = sys.env) { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -90,20 +91,12 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { private def mergeSparkProperties(): Unit = { // Use common defaults file, if not specified by user if (propertiesFile == null) { - sys.env.get("SPARK_CONF_DIR").foreach { sparkConfDir => - val sep = File.separator - val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" - val file = new File(defaultPath) - if (file.exists()) { - propertiesFile = file.getAbsolutePath - } - } - } + val sep = File.separator + val sparkHomeConfig = env.get("SPARK_HOME").map(sparkHome => s"${sparkHome}${sep}conf") + val confDir = env.get("SPARK_CONF_DIR").orElse(sparkHomeConfig) - if (propertiesFile == null) { - sys.env.get("SPARK_HOME").foreach { sparkHome => - val sep = File.separator - val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" + confDir.foreach { sparkConfDir => + val defaultPath = s"${sparkConfDir}${sep}spark-defaults.conf" val file = new File(defaultPath) if (file.exists()) { propertiesFile = file.getAbsolutePath @@ -117,19 +110,18 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { // Use properties file as fallback for values which have a direct analog to // arguments in this script. - master = Option(master).getOrElse(properties.get("spark.master").orNull) - executorMemory = Option(executorMemory) - .getOrElse(properties.get("spark.executor.memory").orNull) - executorCores = Option(executorCores) - .getOrElse(properties.get("spark.executor.cores").orNull) + master = Option(master).orElse(properties.get("spark.master")).orNull + executorMemory = Option(executorMemory).orElse(properties.get("spark.executor.memory")).orNull + executorCores = Option(executorCores).orElse(properties.get("spark.executor.cores")).orNull totalExecutorCores = Option(totalExecutorCores) - .getOrElse(properties.get("spark.cores.max").orNull) - name = Option(name).getOrElse(properties.get("spark.app.name").orNull) - jars = Option(jars).getOrElse(properties.get("spark.jars").orNull) + .orElse(properties.get("spark.cores.max")) + .orNull + name = Option(name).orElse(properties.get("spark.app.name")).orNull + jars = Option(jars).orElse(properties.get("spark.jars")).orNull // This supports env vars in older versions of Spark - master = Option(master).getOrElse(System.getenv("MASTER")) - deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + master = Option(master).orElse(env.get("MASTER")).orNull + deployMode = Option(deployMode).orElse(env.get("DEPLOY_MODE")).orNull // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && primaryResource != null) { @@ -182,7 +174,7 @@ private[spark] class SparkSubmitArguments(args: Seq[String]) { } if (master.startsWith("yarn")) { - val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") + val hasHadoopEnv = env.contains("HADOOP_CONF_DIR") || env.contains("YARN_CONF_DIR") if (!hasHadoopEnv && !Utils.isTesting) { throw new Exception(s"When running with master '$master' " + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 0c324d8bdf6a4..4cba90e8f2afe 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.io.{File, OutputStream, PrintStream} +import java.io._ import scala.collection.mutable.ArrayBuffer @@ -26,6 +26,7 @@ import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.util.Utils import org.scalatest.FunSuite import org.scalatest.Matchers +import com.google.common.io.Files class SparkSubmitSuite extends FunSuite with Matchers { def beforeAll() { @@ -306,6 +307,21 @@ class SparkSubmitSuite extends FunSuite with Matchers { runSparkSubmit(args) } + test("SPARK_CONF_DIR overrides spark-defaults.conf") { + forConfDir(Map("spark.executor.memory" -> "2.3g")) { path => + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + val args = Seq( + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local", + unusedJar.toString) + val appArgs = new SparkSubmitArguments(args, Map("SPARK_CONF_DIR" -> path)) + assert(appArgs.propertiesFile != null) + assert(appArgs.propertiesFile.startsWith(path)) + appArgs.executorMemory should be ("2.3g") + } + } + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. def runSparkSubmit(args: Seq[String]): String = { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) @@ -314,6 +330,22 @@ class SparkSubmitSuite extends FunSuite with Matchers { new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } + + def forConfDir(defaults: Map[String, String]) (f: String => Unit) = { + val tmpDir = Files.createTempDir() + + val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") + val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf)) + for ((key, value) <- defaults) writer.write(s"$key $value\n") + + writer.close() + + try { + f(tmpDir.getAbsolutePath) + } finally { + Utils.deleteRecursively(tmpDir) + } + } } object JarCreationTest { diff --git a/docs/configuration.md b/docs/configuration.md index 316490f0f43fc..a782809a55ec0 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1108,3 +1108,10 @@ compute `SPARK_LOCAL_IP` by looking up the IP of a specific network interface. Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can configure it by adding a `log4j.properties` file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there. + +# Overriding configuration directory + +To specify a different configuration directory other than the default "SPARK_HOME/conf", +you can set SPARK_CONF_DIR. Spark will use the the configuration files (spark-defaults.conf, spark-env.sh, log4j.properties, etc) +from this directory. + From 9d320e222c221e5bb827cddf01a83e64a16d74ff Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Fri, 3 Oct 2014 10:42:41 -0700 Subject: [PATCH 0882/1492] [SPARK-3696]Do not override the user-difined conf_dir https://issues.apache.org/jira/browse/SPARK-3696 We see if SPARK_CONF_DIR is already defined before assignment. Author: WangTaoTheTonic Closes #2541 from WangTaoTheTonic/confdir and squashes the following commits: c3f31e0 [WangTaoTheTonic] Do not override the user-difined conf_dir --- sbin/spark-config.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 2718d6cba1c9a..1d154e62ed5b6 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -33,7 +33,7 @@ this="$config_bin/$script" export SPARK_PREFIX="`dirname "$this"`"/.. export SPARK_HOME="${SPARK_PREFIX}" -export SPARK_CONF_DIR="$SPARK_HOME/conf" +export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"$SPARK_HOME/conf"}" # Add the PySpark classes to the PYTHONPATH: export PYTHONPATH="$SPARK_HOME/python:$PYTHONPATH" export PYTHONPATH="$SPARK_HOME/python/lib/py4j-0.8.2.1-src.zip:$PYTHONPATH" From 22f8e1ee7c4ea7b3bd4c6faaf0fe5b88a134ae12 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Fri, 3 Oct 2014 11:25:18 -0700 Subject: [PATCH 0883/1492] [SPARK-2693][SQL] Supported for UDAF Hive Aggregates like PERCENTILE Implemented UDAF Hive aggregates by adding wrapper to Spark Hive. Author: ravipesala Closes #2620 from ravipesala/SPARK-2693 and squashes the following commits: a8df326 [ravipesala] Removed resolver from constructor arguments caf25c6 [ravipesala] Fixed style issues 5786200 [ravipesala] Supported for UDAF Hive Aggregates like PERCENTILE --- .../org/apache/spark/sql/hive/hiveUdfs.scala | 46 +++++++++++++++++-- .../sql/hive/execution/HiveUdfSuite.scala | 4 ++ 2 files changed, 46 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 732e4976f6843..68f93f247d9bb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.ql.exec.UDF +import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} import org.apache.hadoop.hive.ql.udf.generic._ @@ -57,7 +57,8 @@ private[hive] abstract class HiveFunctionRegistry } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdaf(functionClassName, children) - + } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { + HiveUdaf(functionClassName, children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { HiveGenericUdtf(functionClassName, Nil, children) } else { @@ -194,6 +195,37 @@ private[hive] case class HiveGenericUdaf( def newInstance() = new HiveUdafFunction(functionClassName, children, this) } +/** It is used as a wrapper for the hive functions which uses UDAF interface */ +private[hive] case class HiveUdaf( + functionClassName: String, + children: Seq[Expression]) extends AggregateExpression + with HiveInspectors + with HiveFunctionFactory { + + type UDFType = UDAF + + @transient + protected lazy val resolver: AbstractGenericUDAFResolver = new GenericUDAFBridge(createFunction()) + + @transient + protected lazy val objectInspector = { + resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) + .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) + } + + @transient + protected lazy val inspectors = children.map(_.dataType).map(toInspector) + + def dataType: DataType = inspectorToDataType(objectInspector) + + def nullable: Boolean = true + + override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + + def newInstance() = + new HiveUdafFunction(functionClassName, children, this, true) +} + /** * Converts a Hive Generic User Defined Table Generating Function (UDTF) to a * [[catalyst.expressions.Generator Generator]]. Note that the semantics of Generators do not allow @@ -275,14 +307,20 @@ private[hive] case class HiveGenericUdtf( private[hive] case class HiveUdafFunction( functionClassName: String, exprs: Seq[Expression], - base: AggregateExpression) + base: AggregateExpression, + isUDAFBridgeRequired: Boolean = false) extends AggregateFunction with HiveInspectors with HiveFunctionFactory { def this() = this(null, null, null) - private val resolver = createFunction[AbstractGenericUDAFResolver]() + private val resolver = + if (isUDAFBridgeRequired) { + new GenericUDAFBridge(createFunction[UDAF]()) + } else { + createFunction[AbstractGenericUDAFResolver]() + } private val inspectors = exprs.map(_.dataType).map(toInspector).toArray diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index cc125d539c3c2..e4324e9528f9b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -79,6 +79,10 @@ class HiveUdfSuite extends HiveComparisonTest { sql("SELECT testUdf(pair) FROM hiveUdfTestTable") sql("DROP TEMPORARY FUNCTION IF EXISTS testUdf") } + + test("SPARK-2693 udaf aggregates test") { + assert(sql("SELECT percentile(key,1) FROM src").first === sql("SELECT max(key) FROM src").first) + } } class TestPair(x: Int, y: Int) extends Writable with Serializable { From fbe8e9856b23262193105e7bf86075f516f0db25 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 3 Oct 2014 11:36:24 -0700 Subject: [PATCH 0884/1492] [SPARK-2778] [yarn] Add workaround for race in MiniYARNCluster. Sometimes the cluster's start() method returns before the configuration having been updated, which is done by ClientRMService in, I assume, a separate thread (otherwise there would be no race). That can cause tests to fail if the old configuration data is read, since it will contain the wrong RM address. Author: Marcelo Vanzin Closes #2605 from vanzin/SPARK-2778 and squashes the following commits: 8d02ce0 [Marcelo Vanzin] Minor cleanup. 5bebee7 [Marcelo Vanzin] [SPARK-2778] [yarn] Add workaround for race in MiniYARNCluster. --- .../spark/deploy/yarn/YarnClusterSuite.scala | 35 ++++++++++++++++--- 1 file changed, 31 insertions(+), 4 deletions(-) diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 4b6635679f053..a826b2a78a8f5 100644 --- a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.io.File +import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ @@ -32,7 +33,7 @@ import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils -class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { +class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers with Logging { // log4j configuration for the Yarn containers, so that their output is collected // by Yarn instead of trying to overwrite unit-tests.log. @@ -66,7 +67,33 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { yarnCluster = new MiniYARNCluster(getClass().getName(), 1, 1, 1) yarnCluster.init(new YarnConfiguration()) yarnCluster.start() - yarnCluster.getConfig().foreach { e => + + // There's a race in MiniYARNCluster in which start() may return before the RM has updated + // its address in the configuration. You can see this in the logs by noticing that when + // MiniYARNCluster prints the address, it still has port "0" assigned, although later the + // test works sometimes: + // + // INFO MiniYARNCluster: MiniYARN ResourceManager address: blah:0 + // + // That log message prints the contents of the RM_ADDRESS config variable. If you check it + // later on, it looks something like this: + // + // INFO YarnClusterSuite: RM address in configuration is blah:42631 + // + // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't + // done so in a timely manner (defined to be 10 seconds). + val config = yarnCluster.getConfig() + val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) + while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { + if (System.currentTimeMillis() > deadline) { + throw new IllegalStateException("Timed out waiting for RM to come up.") + } + logDebug("RM address still not set in configuration, waiting...") + TimeUnit.MILLISECONDS.sleep(100) + } + + logInfo(s"RM address in configuration is ${config.get(YarnConfiguration.RM_ADDRESS)}") + config.foreach { e => sys.props += ("spark.hadoop." + e.getKey() -> e.getValue()) } @@ -86,13 +113,13 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers { super.afterAll() } - ignore("run Spark in yarn-client mode") { + test("run Spark in yarn-client mode") { var result = File.createTempFile("result", null, tempDir) YarnClusterDriver.main(Array("yarn-client", result.getAbsolutePath())) checkResult(result) } - ignore("run Spark in yarn-cluster mode") { + test("run Spark in yarn-cluster mode") { val main = YarnClusterDriver.getClass.getName().stripSuffix("$") var result = File.createTempFile("result", null, tempDir) From bec0d0eaa33811fde72b84f7d53a6f6031e7b5d3 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 3 Oct 2014 12:26:02 -0700 Subject: [PATCH 0885/1492] [SPARK-3007][SQL] Adds dynamic partitioning support MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit PR #2226 was reverted because it broke Jenkins builds for unknown reason. This debugging PR aims to fix the Jenkins build. This PR also fixes two bugs: 1. Compression configurations in `InsertIntoHiveTable` are disabled by mistake The `FileSinkDesc` object passed to the writer container doesn't have compression related configurations. These configurations are not taken care of until `saveAsHiveFile` is called. This PR moves compression code forward, right after instantiation of the `FileSinkDesc` object. 1. `PreInsertionCasts` doesn't take table partitions into account In `castChildOutput`, `table.attributes` only contains non-partition columns, thus for partitioned table `childOutputDataTypes` never equals to `tableOutputDataTypes`. This results funny analyzed plan like this: ``` == Analyzed Logical Plan == InsertIntoTable Map(partcol1 -> None, partcol2 -> None), false MetastoreRelation default, dynamic_part_table, None Project [c_0#1164,c_1#1165,c_2#1166] Project [c_0#1164,c_1#1165,c_2#1166] Project [c_0#1164,c_1#1165,c_2#1166] ... (repeats 99 times) ... Project [c_0#1164,c_1#1165,c_2#1166] Project [c_0#1164,c_1#1165,c_2#1166] Project [1 AS c_0#1164,1 AS c_1#1165,1 AS c_2#1166] Filter (key#1170 = 150) MetastoreRelation default, src, None ``` Awful though this logical plan looks, it's harmless because all projects will be eliminated by optimizer. Guess that's why this issue hasn't been caught before. Author: Cheng Lian Author: baishuo(白硕) Author: baishuo Closes #2616 from liancheng/dp-fix and squashes the following commits: 21935b6 [Cheng Lian] Adds back deleted trailing space f471c4b [Cheng Lian] PreInsertionCasts should take table partitions into account a132c80 [Cheng Lian] Fixes output compression 9c6eb2d [Cheng Lian] Adds tests to verify dynamic partitioning folder layout 0eed349 [Cheng Lian] Addresses @yhuai's comments 26632c3 [Cheng Lian] Adds more tests 9227181 [Cheng Lian] Minor refactoring c47470e [Cheng Lian] Refactors InsertIntoHiveTable to a Command 6fb16d7 [Cheng Lian] Fixes typo in test name, regenerated golden answer files d53daa5 [Cheng Lian] Refactors dynamic partitioning support b821611 [baishuo] pass check style 997c990 [baishuo] use HiveConf.DEFAULTPARTITIONNAME to replace hive.exec.default.partition.name 761ecf2 [baishuo] modify according micheal's advice 207c6ac [baishuo] modify for some bad indentation caea6fb [baishuo] modify code to pass scala style checks b660e74 [baishuo] delete a empty else branch cd822f0 [baishuo] do a little modify 8e7268c [baishuo] update file after test 3f91665 [baishuo(白硕)] Update Cast.scala 8ad173c [baishuo(白硕)] Update InsertIntoHiveTable.scala 051ba91 [baishuo(白硕)] Update Cast.scala d452eb3 [baishuo(白硕)] Update HiveQuerySuite.scala 37c603b [baishuo(白硕)] Update InsertIntoHiveTable.scala 98cfb1f [baishuo(白硕)] Update HiveCompatibilitySuite.scala 6af73f4 [baishuo(白硕)] Update InsertIntoHiveTable.scala adf02f1 [baishuo(白硕)] Update InsertIntoHiveTable.scala 1867e23 [baishuo(白硕)] Update SparkHadoopWriter.scala 6bb5880 [baishuo(白硕)] Update HiveQl.scala --- .../execution/HiveCompatibilitySuite.scala | 17 ++ .../org/apache/spark/SparkHadoopWriter.scala | 195 ---------------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 3 +- .../org/apache/spark/sql/hive/HiveQl.scala | 5 - .../hive/execution/InsertIntoHiveTable.scala | 218 ++++++++++-------- .../spark/sql/hive/hiveWriterContainers.scala | 217 +++++++++++++++++ ...rtition-0-be33aaa7253c8f248ff3921cd7dae340 | 0 ...rtition-1-640552dd462707563fd255a713f83b41 | 0 ...rtition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 | 1 + ...rtition-3-b7f7fa7ebf666f4fee27e149d8c6961f | 0 ...rtition-4-8bdb71ad8cb3cc3026043def2525de3a | 0 ...rtition-5-c630dce438f3792e7fb0f523fbbb3e1e | 0 ...rtition-6-7abc9ec8a36cdc5e89e955265a7fd7cf | 0 ...rtition-7-be33aaa7253c8f248ff3921cd7dae340 | 0 .../sql/hive/execution/HiveQuerySuite.scala | 100 +++++++- 15 files changed, 450 insertions(+), 306 deletions(-) delete mode 100644 sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 556c984ad392b..35e9c9939d4b7 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -220,6 +220,23 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", + "dynamic_partition_skip_default", + "infer_bucket_sort_dyn_part", + "load_dyn_part1", + "load_dyn_part2", + "load_dyn_part3", + "load_dyn_part4", + "load_dyn_part5", + "load_dyn_part6", + "load_dyn_part7", + "load_dyn_part8", + "load_dyn_part9", + "load_dyn_part10", + "load_dyn_part11", + "load_dyn_part12", + "load_dyn_part13", + "load_dyn_part14", + "load_dyn_part14_win", "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala deleted file mode 100644 index ab7862f4f9e06..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ /dev/null @@ -1,195 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.io.IOException -import java.text.NumberFormat -import java.util.Date - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} -import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.FileSinkDesc -import org.apache.hadoop.mapred._ -import org.apache.hadoop.io.Writable - -import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} - -/** - * Internal helper class that saves an RDD using a Hive OutputFormat. - * It is based on [[SparkHadoopWriter]]. - */ -private[hive] class SparkHiveHadoopWriter( - @transient jobConf: JobConf, - fileSinkConf: FileSinkDesc) - extends Logging - with SparkHadoopMapRedUtil - with Serializable { - - private val now = new Date() - private val conf = new SerializableWritable(jobConf) - - private var jobID = 0 - private var splitID = 0 - private var attemptID = 0 - private var jID: SerializableWritable[JobID] = null - private var taID: SerializableWritable[TaskAttemptID] = null - - @transient private var writer: FileSinkOperator.RecordWriter = null - @transient private var format: HiveOutputFormat[AnyRef, Writable] = null - @transient private var committer: OutputCommitter = null - @transient private var jobContext: JobContext = null - @transient private var taskContext: TaskAttemptContext = null - - def preSetup() { - setIDs(0, 0, 0) - setConfParams() - - val jCtxt = getJobContext() - getOutputCommitter().setupJob(jCtxt) - } - - - def setup(jobid: Int, splitid: Int, attemptid: Int) { - setIDs(jobid, splitid, attemptid) - setConfParams() - } - - def open() { - val numfmt = NumberFormat.getInstance() - numfmt.setMinimumIntegerDigits(5) - numfmt.setGroupingUsed(false) - - val extension = Utilities.getFileExtension( - conf.value, - fileSinkConf.getCompressed, - getOutputFormat()) - - val outputName = "part-" + numfmt.format(splitID) + extension - val path = FileOutputFormat.getTaskOutputPath(conf.value, outputName) - - getOutputCommitter().setupTask(getTaskContext()) - writer = HiveFileFormatUtils.getHiveRecordWriter( - conf.value, - fileSinkConf.getTableInfo, - conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], - fileSinkConf, - path, - null) - } - - def write(value: Writable) { - if (writer != null) { - writer.write(value) - } else { - throw new IOException("Writer is null, open() has not been called") - } - } - - def close() { - // Seems the boolean value passed into close does not matter. - writer.close(false) - } - - def commit() { - val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() - if (cmtr.needsTaskCommit(taCtxt)) { - try { - cmtr.commitTask(taCtxt) - logInfo (taID + ": Committed") - } catch { - case e: IOException => - logError("Error committing the output of task: " + taID.value, e) - cmtr.abortTask(taCtxt) - throw e - } - } else { - logWarning ("No need to commit output of task: " + taID.value) - } - } - - def commitJob() { - // always ? Or if cmtr.needsTaskCommit ? - val cmtr = getOutputCommitter() - cmtr.commitJob(getJobContext()) - } - - // ********* Private Functions ********* - - private def getOutputFormat(): HiveOutputFormat[AnyRef,Writable] = { - if (format == null) { - format = conf.value.getOutputFormat() - .asInstanceOf[HiveOutputFormat[AnyRef,Writable]] - } - format - } - - private def getOutputCommitter(): OutputCommitter = { - if (committer == null) { - committer = conf.value.getOutputCommitter - } - committer - } - - private def getJobContext(): JobContext = { - if (jobContext == null) { - jobContext = newJobContext(conf.value, jID.value) - } - jobContext - } - - private def getTaskContext(): TaskAttemptContext = { - if (taskContext == null) { - taskContext = newTaskAttemptContext(conf.value, taID.value) - } - taskContext - } - - private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { - jobID = jobId - splitID = splitId - attemptID = attemptId - - jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) - taID = new SerializableWritable[TaskAttemptID]( - new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) - } - - private def setConfParams() { - conf.value.set("mapred.job.id", jID.value.toString) - conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) - conf.value.set("mapred.task.id", taID.value.toString) - conf.value.setBoolean("mapred.task.is.map", true) - conf.value.setInt("mapred.task.partition", splitID) - } -} - -private[hive] object SparkHiveHadoopWriter { - def createPathFromString(path: String, conf: JobConf): Path = { - if (path == null) { - throw new IllegalArgumentException("Output path is null") - } - val outputPath = new Path(path) - val fs = outputPath.getFileSystem(conf) - if (outputPath == null || fs == null) { - throw new IllegalArgumentException("Incorrectly formatted output path") - } - outputPath.makeQualified(fs) - } -} 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 06b1446ccbd39..989a9784a438d 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 @@ -144,7 +144,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val childOutputDataTypes = child.output.map(_.dataType) // Only check attributes, not partitionKeys since they are always strings. // TODO: Fully support inserting into partitioned tables. - val tableOutputDataTypes = table.attributes.map(_.dataType) + val tableOutputDataTypes = + table.attributes.map(_.dataType) ++ table.partitionKeys.map(_.dataType) if (childOutputDataTypes == tableOutputDataTypes) { p 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 6bb42eeb0550d..32c9175f181bb 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 @@ -806,11 +806,6 @@ private[hive] object HiveQl { cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) - if (partitionKeys.values.exists(p => p.isEmpty)) { - throw new NotImplementedError(s"Do not support INSERT INTO/OVERWRITE with" + - s"dynamic partitioning.") - } - InsertIntoTable(UnresolvedRelation(db, tableName, None), partitionKeys, query, overwrite) case a: ASTNode => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index a284a91a91e31..16a8c782acdfa 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -19,27 +19,25 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConversions._ -import java.util.{HashMap => JHashMap} - import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils -import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.Hive import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer -import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector -import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector -import org.apache.hadoop.io.Writable +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{JavaHiveDecimalObjectInspector, JavaHiveVarcharObjectInspector} import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} -import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} -import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation, SparkHiveHadoopWriter} +import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} +import org.apache.spark.sql.hive._ +import org.apache.spark.{SerializableWritable, SparkException, TaskContext} /** * :: DeveloperApi :: @@ -51,7 +49,7 @@ case class InsertIntoHiveTable( child: SparkPlan, overwrite: Boolean) (@transient sc: HiveContext) - extends UnaryNode { + extends UnaryNode with Command { @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) @@ -101,66 +99,61 @@ case class InsertIntoHiveTable( } def saveAsHiveFile( - rdd: RDD[Writable], + rdd: RDD[Row], valueClass: Class[_], fileSinkConf: FileSinkDesc, - conf: JobConf, - isCompressed: Boolean) { - if (valueClass == null) { - throw new SparkException("Output value class not set") - } - conf.setOutputValueClass(valueClass) - if (fileSinkConf.getTableInfo.getOutputFileFormatClassName == null) { - throw new SparkException("Output format class not set") - } - // Doesn't work in Scala 2.9 due to what may be a generics bug - // TODO: Should we uncomment this for Scala 2.10? - // conf.setOutputFormat(outputFormatClass) - conf.set("mapred.output.format.class", fileSinkConf.getTableInfo.getOutputFileFormatClassName) - if (isCompressed) { - // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", - // and "mapred.output.compression.type" have no impact on ORC because it uses table properties - // to store compression information. - conf.set("mapred.output.compress", "true") - fileSinkConf.setCompressed(true) - fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) - fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) - } - conf.setOutputCommitter(classOf[FileOutputCommitter]) - FileOutputFormat.setOutputPath( - conf, - SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) + conf: SerializableWritable[JobConf], + writerContainer: SparkHiveWriterContainer) { + assert(valueClass != null, "Output value class not set") + conf.value.setOutputValueClass(valueClass) + + val outputFileFormatClassName = fileSinkConf.getTableInfo.getOutputFileFormatClassName + assert(outputFileFormatClassName != null, "Output format class not set") + conf.value.set("mapred.output.format.class", outputFileFormatClassName) + conf.value.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath( + conf.value, + SparkHiveWriterContainer.createPathFromString(fileSinkConf.getDirName, conf.value)) log.debug("Saving as hadoop file of type " + valueClass.getSimpleName) - val writer = new SparkHiveHadoopWriter(conf, fileSinkConf) - writer.preSetup() + writerContainer.driverSideSetup() + sc.sparkContext.runJob(rdd, writeToFile _) + writerContainer.commitJob() + + // Note that this function is executed on executor side + def writeToFile(context: TaskContext, iterator: Iterator[Row]) { + val serializer = newSerializer(fileSinkConf.getTableInfo) + val standardOI = ObjectInspectorUtils + .getStandardObjectInspector( + fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val outputData = new Array[Any](fieldOIs.length) - def writeToFile(context: TaskContext, iter: Iterator[Writable]) { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. val attemptNumber = (context.attemptId % Int.MaxValue).toInt + writerContainer.executorSideSetup(context.stageId, context.partitionId, attemptNumber) - writer.setup(context.stageId, context.partitionId, attemptNumber) - writer.open() + iterator.foreach { row => + var i = 0 + while (i < fieldOIs.length) { + // TODO (lian) avoid per row dynamic dispatching and pattern matching cost in `wrap` + outputData(i) = wrap(row(i), fieldOIs(i)) + i += 1 + } - var count = 0 - while(iter.hasNext) { - val record = iter.next() - count += 1 - writer.write(record) + val writer = writerContainer.getLocalFileWriter(row) + writer.write(serializer.serialize(outputData, standardOI)) } - writer.close() - writer.commit() + writerContainer.close() } - - sc.sparkContext.runJob(rdd, writeToFile _) - writer.commitJob() } - override def execute() = result - /** * Inserts all the rows in the table into Hive. Row objects are properly serialized with the * `org.apache.hadoop.hive.serde2.SerDe` and the @@ -168,50 +161,69 @@ case class InsertIntoHiveTable( * * Note: this is run once and then kept to avoid double insertions. */ - private lazy val result: RDD[Row] = { - val childRdd = child.execute() - assert(childRdd != null) - + override protected[sql] lazy val sideEffectResult: Seq[Row] = { // Have to pass the TableDesc object to RDD.mapPartitions and then instantiate new serializer // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc val tableLocation = table.hiveQlTable.getDataLocation val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) - val rdd = childRdd.mapPartitions { iter => - val serializer = newSerializer(fileSinkConf.getTableInfo) - val standardOI = ObjectInspectorUtils - .getStandardObjectInspector( - fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, - ObjectInspectorCopyOption.JAVA) - .asInstanceOf[StructObjectInspector] + val isCompressed = sc.hiveconf.getBoolean( + ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) + if (isCompressed) { + // Please note that isCompressed, "mapred.output.compress", "mapred.output.compression.codec", + // and "mapred.output.compression.type" have no impact on ORC because it uses table properties + // to store compression information. + sc.hiveconf.set("mapred.output.compress", "true") + fileSinkConf.setCompressed(true) + fileSinkConf.setCompressCodec(sc.hiveconf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(sc.hiveconf.get("mapred.output.compression.type")) + } - val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray - val outputData = new Array[Any](fieldOIs.length) - iter.map { row => - var i = 0 - while (i < row.length) { - // Casts Strings to HiveVarchars when necessary. - outputData(i) = wrap(row(i), fieldOIs(i)) - i += 1 - } + val numDynamicPartitions = partition.values.count(_.isEmpty) + val numStaticPartitions = partition.values.count(_.nonEmpty) + val partitionSpec = partition.map { + case (key, Some(value)) => key -> value + case (key, None) => key -> "" + } + + // All partition column names in the format of "//..." + val partitionColumns = fileSinkConf.getTableInfo.getProperties.getProperty("partition_columns") + val partitionColumnNames = Option(partitionColumns).map(_.split("/")).orNull + + // Validate partition spec if there exist any dynamic partitions + if (numDynamicPartitions > 0) { + // Report error if dynamic partitioning is not enabled + if (!sc.hiveconf.getBoolVar(HiveConf.ConfVars.DYNAMICPARTITIONING)) { + throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_DISABLED.getMsg) + } - serializer.serialize(outputData, standardOI) + // Report error if dynamic partition strict mode is on but no static partition is found + if (numStaticPartitions == 0 && + sc.hiveconf.getVar(HiveConf.ConfVars.DYNAMICPARTITIONINGMODE).equalsIgnoreCase("strict")) { + throw new SparkException(ErrorMsg.DYNAMIC_PARTITION_STRICT_MODE.getMsg) + } + + // Report error if any static partition appears after a dynamic partition + val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) + isDynamic.init.zip(isDynamic.tail).find(_ == (true, false)).foreach { _ => + throw new SparkException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } - // ORC stores compression information in table properties. While, there are other formats - // (e.g. RCFile) that rely on hadoop configurations to store compression information. val jobConf = new JobConf(sc.hiveconf) - saveAsHiveFile( - rdd, - outputClass, - fileSinkConf, - jobConf, - sc.hiveconf.getBoolean("hive.exec.compress.output", false)) - - // TODO: Handle dynamic partitioning. + val jobConfSer = new SerializableWritable(jobConf) + + val writerContainer = if (numDynamicPartitions > 0) { + val dynamicPartColNames = partitionColumnNames.takeRight(numDynamicPartitions) + new SparkHiveDynamicPartitionWriterContainer(jobConf, fileSinkConf, dynamicPartColNames) + } else { + new SparkHiveWriterContainer(jobConf, fileSinkConf) + } + + saveAsHiveFile(child.execute(), outputClass, fileSinkConf, jobConfSer, writerContainer) + val outputPath = FileOutputFormat.getOutputPath(jobConf) // Have to construct the format of dbname.tablename. val qualifiedTableName = s"${table.databaseName}.${table.tableName}" @@ -220,10 +232,6 @@ case class InsertIntoHiveTable( // holdDDLTime will be true when TOK_HOLD_DDLTIME presents in the query as a hint. val holdDDLTime = false if (partition.nonEmpty) { - val partitionSpec = partition.map { - case (key, Some(value)) => key -> value - case (key, None) => key -> "" // Should not reach here right now. - } val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) db.validatePartitionNameCharacters(partVals) // inheritTableSpecs is set to true. It should be set to false for a IMPORT query @@ -231,14 +239,26 @@ case class InsertIntoHiveTable( val inheritTableSpecs = true // TODO: Correctly set isSkewedStoreAsSubdir. val isSkewedStoreAsSubdir = false - db.loadPartition( - outputPath, - qualifiedTableName, - partitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + if (numDynamicPartitions > 0) { + db.loadDynamicPartitions( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + numDynamicPartitions, + holdDDLTime, + isSkewedStoreAsSubdir + ) + } else { + db.loadPartition( + outputPath, + qualifiedTableName, + partitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } } else { db.loadTable( outputPath, @@ -251,6 +271,6 @@ case class InsertIntoHiveTable( // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. // TODO: implement hive compatibility as rules. - sc.sparkContext.makeRDD(Nil, 1) + Seq.empty[Row] } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala new file mode 100644 index 0000000000000..ac5c7a8220296 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.IOException +import java.text.NumberFormat +import java.util.Date + +import scala.collection.mutable + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred._ + +import org.apache.spark.sql.Row +import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} + +/** + * Internal helper class that saves an RDD using a Hive OutputFormat. + * It is based on [[SparkHadoopWriter]]. + */ +private[hive] class SparkHiveWriterContainer( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc) + extends Logging + with SparkHadoopMapRedUtil + with Serializable { + + private val now = new Date() + protected val conf = new SerializableWritable(jobConf) + + private var jobID = 0 + private var splitID = 0 + private var attemptID = 0 + private var jID: SerializableWritable[JobID] = null + private var taID: SerializableWritable[TaskAttemptID] = null + + @transient private var writer: FileSinkOperator.RecordWriter = null + @transient private lazy val committer = conf.value.getOutputCommitter + @transient private lazy val jobContext = newJobContext(conf.value, jID.value) + @transient private lazy val taskContext = newTaskAttemptContext(conf.value, taID.value) + @transient private lazy val outputFormat = + conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef,Writable]] + + def driverSideSetup() { + setIDs(0, 0, 0) + setConfParams() + committer.setupJob(jobContext) + } + + def executorSideSetup(jobId: Int, splitId: Int, attemptId: Int) { + setIDs(jobId, splitId, attemptId) + setConfParams() + committer.setupTask(taskContext) + initWriters() + } + + protected def getOutputName: String = { + val numberFormat = NumberFormat.getInstance() + numberFormat.setMinimumIntegerDigits(5) + numberFormat.setGroupingUsed(false) + val extension = Utilities.getFileExtension(conf.value, fileSinkConf.getCompressed, outputFormat) + "part-" + numberFormat.format(splitID) + extension + } + + def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = writer + + def close() { + // Seems the boolean value passed into close does not matter. + writer.close(false) + commit() + } + + def commitJob() { + committer.commitJob(jobContext) + } + + protected def initWriters() { + // NOTE this method is executed at the executor side. + // For Hive tables without partitions or with only static partitions, only 1 writer is needed. + writer = HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + fileSinkConf, + FileOutputFormat.getTaskOutputPath(conf.value, getOutputName), + Reporter.NULL) + } + + protected def commit() { + if (committer.needsTaskCommit(taskContext)) { + try { + committer.commitTask(taskContext) + logInfo (taID + ": Committed") + } catch { + case e: IOException => + logError("Error committing the output of task: " + taID.value, e) + committer.abortTask(taskContext) + throw e + } + } else { + logInfo("No need to commit output of task: " + taID.value) + } + } + + // ********* Private Functions ********* + + private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { + jobID = jobId + splitID = splitId + attemptID = attemptId + + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) + taID = new SerializableWritable[TaskAttemptID]( + new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) + } + + private def setConfParams() { + conf.value.set("mapred.job.id", jID.value.toString) + conf.value.set("mapred.tip.id", taID.value.getTaskID.toString) + conf.value.set("mapred.task.id", taID.value.toString) + conf.value.setBoolean("mapred.task.is.map", true) + conf.value.setInt("mapred.task.partition", splitID) + } +} + +private[hive] object SparkHiveWriterContainer { + def createPathFromString(path: String, conf: JobConf): Path = { + if (path == null) { + throw new IllegalArgumentException("Output path is null") + } + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(conf) + if (outputPath == null || fs == null) { + throw new IllegalArgumentException("Incorrectly formatted output path") + } + outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + } +} + +private[spark] class SparkHiveDynamicPartitionWriterContainer( + @transient jobConf: JobConf, + fileSinkConf: FileSinkDesc, + dynamicPartColNames: Array[String]) + extends SparkHiveWriterContainer(jobConf, fileSinkConf) { + + private val defaultPartName = jobConf.get( + ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultVal) + + @transient private var writers: mutable.HashMap[String, FileSinkOperator.RecordWriter] = _ + + override protected def initWriters(): Unit = { + // NOTE: This method is executed at the executor side. + // Actual writers are created for each dynamic partition on the fly. + writers = mutable.HashMap.empty[String, FileSinkOperator.RecordWriter] + } + + override def close(): Unit = { + writers.values.foreach(_.close(false)) + commit() + } + + override def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = { + val dynamicPartPath = dynamicPartColNames + .zip(row.takeRight(dynamicPartColNames.length)) + .map { case (col, rawVal) => + val string = if (rawVal == null) null else String.valueOf(rawVal) + s"/$col=${if (string == null || string.isEmpty) defaultPartName else string}" + } + .mkString + + def newWriter = { + val newFileSinkDesc = new FileSinkDesc( + fileSinkConf.getDirName + dynamicPartPath, + fileSinkConf.getTableInfo, + fileSinkConf.getCompressed) + newFileSinkDesc.setCompressCodec(fileSinkConf.getCompressCodec) + newFileSinkDesc.setCompressType(fileSinkConf.getCompressType) + + val path = { + val outputPath = FileOutputFormat.getOutputPath(conf.value) + assert(outputPath != null, "Undefined job output-path") + val workPath = new Path(outputPath, dynamicPartPath.stripPrefix("/")) + new Path(workPath, getOutputName) + } + + HiveFileFormatUtils.getHiveRecordWriter( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + newFileSinkDesc, + path, + Reporter.NULL) + } + + writers.getOrElseUpdate(dynamicPartPath, newWriter) + } +} diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-0-be33aaa7253c8f248ff3921cd7dae340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 b/sql/hive/src/test/resources/golden/dynamic_partition-1-640552dd462707563fd255a713f83b41 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 b/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/dynamic_partition-2-36456c9d0d2e3ef72ab5ba9ba48e5493 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f b/sql/hive/src/test/resources/golden/dynamic_partition-3-b7f7fa7ebf666f4fee27e149d8c6961f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a b/sql/hive/src/test/resources/golden/dynamic_partition-4-8bdb71ad8cb3cc3026043def2525de3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e b/sql/hive/src/test/resources/golden/dynamic_partition-5-c630dce438f3792e7fb0f523fbbb3e1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf b/sql/hive/src/test/resources/golden/dynamic_partition-6-7abc9ec8a36cdc5e89e955265a7fd7cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 b/sql/hive/src/test/resources/golden/dynamic_partition-7-be33aaa7253c8f248ff3921cd7dae340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index f5868bff22f13..2e282a9ade40c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,6 +19,9 @@ package org.apache.spark.sql.hive.execution import scala.util.Try +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.SparkException import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -380,7 +383,7 @@ class HiveQuerySuite extends HiveComparisonTest { def isExplanation(result: SchemaRDD) = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } - explanation.exists(_ == "== Physical Plan ==") + explanation.contains("== Physical Plan ==") } test("SPARK-1704: Explain commands as a SchemaRDD") { @@ -568,6 +571,91 @@ class HiveQuerySuite extends HiveComparisonTest { case class LogEntry(filename: String, message: String) case class LogFile(name: String) + createQueryTest("dynamic_partition", + """ + |DROP TABLE IF EXISTS dynamic_part_table; + |CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT); + | + |SET hive.exec.dynamic.partition.mode=nonstrict; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, 1, 1 FROM src WHERE key=150; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, NULL, 1 FROM src WHERE key=150; + | + |INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, 1, NULL FROM src WHERE key=150; + | + |INSERT INTO TABLe dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT 1, NULL, NULL FROM src WHERE key=150; + | + |DROP TABLE IF EXISTS dynamic_part_table; + """.stripMargin) + + test("Dynamic partition folder layout") { + sql("DROP TABLE IF EXISTS dynamic_part_table") + sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + val data = Map( + Seq("1", "1") -> 1, + Seq("1", "NULL") -> 2, + Seq("NULL", "1") -> 3, + Seq("NULL", "NULL") -> 4) + + data.foreach { case (parts, value) => + sql( + s"""INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT $value, ${parts.mkString(", ")} FROM src WHERE key=150 + """.stripMargin) + + val partFolder = Seq("partcol1", "partcol2") + .zip(parts) + .map { case (k, v) => + if (v == "NULL") { + s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultVal}" + } else { + s"$k=$v" + } + } + .mkString("/") + + // Loads partition data to a temporary table to verify contents + val path = s"$warehousePath/dynamic_part_table/$partFolder/part-00000" + + sql("DROP TABLE IF EXISTS dp_verify") + sql("CREATE TABLE dp_verify(intcol INT)") + sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") + + assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) + } + } + + test("Partition spec validation") { + sql("DROP TABLE IF EXISTS dp_test") + sql("CREATE TABLE dp_test(key INT, value STRING) PARTITIONED BY (dp INT, sp INT)") + sql("SET hive.exec.dynamic.partition.mode=strict") + + // Should throw when using strict dynamic partition mode without any static partition + intercept[SparkException] { + sql( + """INSERT INTO TABLE dp_test PARTITION(dp) + |SELECT key, value, key % 5 FROM src + """.stripMargin) + } + + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + // Should throw when a static partition appears after a dynamic partition + intercept[SparkException] { + sql( + """INSERT INTO TABLE dp_test PARTITION(dp, sp = 1) + |SELECT key, value, key % 5 FROM src + """.stripMargin) + } + } + test("SPARK-3414 regression: should store analyzed logical plan when registering a temp table") { sparkContext.makeRDD(Seq.empty[LogEntry]).registerTempTable("rawLogs") sparkContext.makeRDD(Seq.empty[LogFile]).registerTempTable("logFiles") @@ -625,27 +713,27 @@ class HiveQuerySuite extends HiveComparisonTest { assert(sql("SET").collect().size == 0) assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey=$testVal")) + collectResults(sql(s"SET $testKey=$testVal")) } assert(hiveconf.get(testKey, "") == testVal) assertResult(Set(testKey -> testVal)) { - collectResults(hql("SET")) + collectResults(sql("SET")) } sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(hql("SET")) + collectResults(sql("SET")) } // "set key" assertResult(Set(testKey -> testVal)) { - collectResults(hql(s"SET $testKey")) + collectResults(sql(s"SET $testKey")) } assertResult(Set(nonexistentKey -> "")) { - collectResults(hql(s"SET $nonexistentKey")) + collectResults(sql(s"SET $nonexistentKey")) } // Assert that sql() should have the same effects as sql() by repeating the above using sql(). From 6a1d48f4f02c4498b64439c3dd5f671286a90e30 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 3 Oct 2014 12:34:27 -0700 Subject: [PATCH 0886/1492] [SPARK-3212][SQL] Use logical plan matching instead of temporary tables for table caching _Also addresses: SPARK-1671, SPARK-1379 and SPARK-3641_ This PR introduces a new trait, `CacheManger`, which replaces the previous temporary table based caching system. Instead of creating a temporary table that shadows an existing table with and equivalent cached representation, the cached manager maintains a separate list of logical plans and their cached data. After optimization, this list is searched for any matching plan fragments. When a matching plan fragment is found it is replaced with the cached data. There are several advantages to this approach: - Calling .cache() on a SchemaRDD now works as you would expect, and uses the more efficient columnar representation. - Its now possible to provide a list of temporary tables, without having to decide if a given table is actually just a cached persistent table. (To be done in a follow-up PR) - In some cases it is possible that cached data will be used, even if a cached table was not explicitly requested. This is because we now look at the logical structure instead of the table name. - We now correctly invalidate when data is inserted into a hive table. Author: Michael Armbrust Closes #2501 from marmbrus/caching and squashes the following commits: 63fbc2c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into caching. 0ea889e [Michael Armbrust] Address comments. 1e23287 [Michael Armbrust] Add support for cache invalidation for hive inserts. 65ed04a [Michael Armbrust] fix tests. bdf9a3f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into caching b4b77f2 [Michael Armbrust] Address comments 6923c9d [Michael Armbrust] More comments / tests 80f26ac [Michael Armbrust] First draft of improved semantics for Spark SQL caching. --- .../sql/catalyst/analysis/Analyzer.scala | 3 + .../expressions/namedExpressions.scala | 4 +- .../catalyst/plans/logical/LogicalPlan.scala | 42 ++++++ .../catalyst/plans/logical/TestRelation.scala | 6 + .../plans/logical/basicOperators.scala | 4 +- .../sql/catalyst/plans/SameResultSuite.scala | 62 ++++++++ .../org/apache/spark/sql/CacheManager.scala | 139 ++++++++++++++++++ .../org/apache/spark/sql/SQLContext.scala | 51 +------ .../org/apache/spark/sql/SchemaRDD.scala | 23 ++- .../org/apache/spark/sql/SchemaRDDLike.scala | 5 +- .../spark/sql/api/java/JavaSQLContext.scala | 10 +- .../columnar/InMemoryColumnarTableScan.scala | 28 +++- .../spark/sql/execution/ExistingRDD.scala | 119 +++++++++++++++ .../spark/sql/execution/SparkPlan.scala | 33 ----- .../spark/sql/execution/SparkStrategies.scala | 9 +- .../spark/sql/execution/basicOperators.scala | 39 ----- .../apache/spark/sql/CachedTableSuite.scala | 103 +++++++------ .../columnar/InMemoryColumnarQuerySuite.scala | 7 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 +- .../spark/sql/hive/HiveStrategies.scala | 6 +- .../org/apache/spark/sql/hive/TestHive.scala | 5 +- .../hive/execution/InsertIntoHiveTable.scala | 3 + .../spark/sql/hive/CachedTableSuite.scala | 100 ++++++++----- 23 files changed, 567 insertions(+), 241 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 71810b798bd04..fe83eb12502dc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -93,6 +93,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool */ object ResolveRelations extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case i @ InsertIntoTable(UnresolvedRelation(databaseName, name, alias), _, _, _) => + i.copy( + table = EliminateAnalysisOperators(catalog.lookupRelation(databaseName, name, alias))) case UnresolvedRelation(databaseName, name, alias) => catalog.lookupRelation(databaseName, name, alias) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 59fb0311a9c44..e5a958d599393 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -62,7 +62,7 @@ abstract class Attribute extends NamedExpression { def withName(newName: String): Attribute def toAttribute = this - def newInstance: Attribute + def newInstance(): Attribute } @@ -131,7 +131,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea h } - override def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) + override def newInstance() = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) /** * Returns a copy of this [[AttributeReference]] with changed nullability. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 28d863e58beca..4f8ad8a7e0223 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.trees @@ -72,6 +73,47 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { */ def childrenResolved: Boolean = !children.exists(!_.resolved) + /** + * Returns true when the given logical plan will return the same results as this logical plan. + * + * Since its likely undecideable to generally determine if two given plans will produce the same + * results, it is okay for this function to return false, even if the results are actually + * the same. Such behavior will not affect correctness, only the application of performance + * enhancements like caching. However, it is not acceptable to return true if the results could + * possibly be different. + * + * By default this function performs a modified version of equality that is tolerant of cosmetic + * differences like attribute naming and or expression id differences. Logical operators that + * can do better should override this function. + */ + def sameResult(plan: LogicalPlan): Boolean = { + plan.getClass == this.getClass && + plan.children.size == children.size && { + logDebug(s"[${cleanArgs.mkString(", ")}] == [${plan.cleanArgs.mkString(", ")}]") + cleanArgs == plan.cleanArgs + } && + (plan.children, children).zipped.forall(_ sameResult _) + } + + /** Args that have cleaned such that differences in expression id should not affect equality */ + protected lazy val cleanArgs: Seq[Any] = { + val input = children.flatMap(_.output) + productIterator.map { + // Children are checked using sameResult above. + case tn: TreeNode[_] if children contains tn => null + case e: Expression => BindReferences.bindReference(e, input, allowFailures = true) + case s: Option[_] => s.map { + case e: Expression => BindReferences.bindReference(e, input, allowFailures = true) + case other => other + } + case s: Seq[_] => s.map { + case e: Expression => BindReferences.bindReference(e, input, allowFailures = true) + case other => other + } + case other => other + }.toSeq + } + /** * Optionally resolves the given string to a [[NamedExpression]] using the input from all child * nodes of this LogicalPlan. The attribute is expressed as diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala index f8fe558511bfd..19769986ef58c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala @@ -41,4 +41,10 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil) } override protected def stringArgs = Iterator(output) + + override def sameResult(plan: LogicalPlan): Boolean = plan match { + case LocalRelation(otherOutput, otherData) => + otherOutput.map(_.dataType) == output.map(_.dataType) && otherData == data + case _ => false + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 391508279bb80..f8e9930ac270d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -105,8 +105,8 @@ case class InsertIntoTable( child: LogicalPlan, overwrite: Boolean) extends LogicalPlan { - // The table being inserted into is a child for the purposes of transformations. - override def children = table :: child :: Nil + + override def children = child :: Nil override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala new file mode 100644 index 0000000000000..e8a793d107451 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.util._ + +/** + * Provides helper methods for comparing plans. + */ +class SameResultSuite extends FunSuite { + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) + + def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true) = { + val aAnalyzed = a.analyze + val bAnalyzed = b.analyze + + if (aAnalyzed.sameResult(bAnalyzed) != result) { + val comparison = sideBySide(aAnalyzed.toString, bAnalyzed.toString).mkString("\n") + fail(s"Plans should return sameResult = $result\n$comparison") + } + } + + test("relations") { + assertSameResult(testRelation, testRelation2) + } + + test("projections") { + assertSameResult(testRelation.select('a), testRelation2.select('a)) + assertSameResult(testRelation.select('b), testRelation2.select('b)) + assertSameResult(testRelation.select('a, 'b), testRelation2.select('a, 'b)) + assertSameResult(testRelation.select('b, 'a), testRelation2.select('b, 'a)) + + assertSameResult(testRelation, testRelation2.select('a), false) + assertSameResult(testRelation.select('b, 'a), testRelation2.select('a, 'b), false) + } + + test("filters") { + assertSameResult(testRelation.where('a === 'b), testRelation2.where('a === 'b)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala new file mode 100644 index 0000000000000..aebdbb68e49b8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import java.util.concurrent.locks.ReentrantReadWriteLock + +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.columnar.InMemoryRelation +import org.apache.spark.storage.StorageLevel +import org.apache.spark.storage.StorageLevel.MEMORY_ONLY + +/** Holds a cached logical plan and its data */ +private case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) + +/** + * Provides support in a SQLContext for caching query results and automatically using these cached + * results when subsequent queries are executed. Data is cached using byte buffers stored in an + * InMemoryRelation. This relation is automatically substituted query plans that return the + * `sameResult` as the originally cached query. + */ +private[sql] trait CacheManager { + self: SQLContext => + + @transient + private val cachedData = new scala.collection.mutable.ArrayBuffer[CachedData] + + @transient + private val cacheLock = new ReentrantReadWriteLock + + /** Returns true if the table is currently cached in-memory. */ + def isCached(tableName: String): Boolean = lookupCachedData(table(tableName)).nonEmpty + + /** Caches the specified table in-memory. */ + def cacheTable(tableName: String): Unit = cacheQuery(table(tableName)) + + /** Removes the specified table from the in-memory cache. */ + def uncacheTable(tableName: String): Unit = uncacheQuery(table(tableName)) + + /** Acquires a read lock on the cache for the duration of `f`. */ + private def readLock[A](f: => A): A = { + val lock = cacheLock.readLock() + lock.lock() + try f finally { + lock.unlock() + } + } + + /** Acquires a write lock on the cache for the duration of `f`. */ + private def writeLock[A](f: => A): A = { + val lock = cacheLock.writeLock() + lock.lock() + try f finally { + lock.unlock() + } + } + + private[sql] def clearCache(): Unit = writeLock { + cachedData.foreach(_.cachedRepresentation.cachedColumnBuffers.unpersist()) + cachedData.clear() + } + + /** Caches the data produced by the logical representation of the given schema rdd. */ + private[sql] def cacheQuery( + query: SchemaRDD, + storageLevel: StorageLevel = MEMORY_ONLY): Unit = writeLock { + val planToCache = query.queryExecution.optimizedPlan + if (lookupCachedData(planToCache).nonEmpty) { + logWarning("Asked to cache already cached data.") + } else { + cachedData += + CachedData( + planToCache, + InMemoryRelation( + useCompression, columnBatchSize, storageLevel, query.queryExecution.executedPlan)) + } + } + + /** Removes the data for the given SchemaRDD from the cache */ + private[sql] def uncacheQuery(query: SchemaRDD, blocking: Boolean = false): Unit = writeLock { + val planToCache = query.queryExecution.optimizedPlan + val dataIndex = cachedData.indexWhere(_.plan.sameResult(planToCache)) + + if (dataIndex < 0) { + throw new IllegalArgumentException(s"Table $query is not cached.") + } + + cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData.remove(dataIndex) + } + + + /** Optionally returns cached data for the given SchemaRDD */ + private[sql] def lookupCachedData(query: SchemaRDD): Option[CachedData] = readLock { + lookupCachedData(query.queryExecution.optimizedPlan) + } + + /** Optionally returns cached data for the given LogicalPlan. */ + private[sql] def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { + cachedData.find(_.plan.sameResult(plan)) + } + + /** Replaces segments of the given logical plan with cached versions where possible. */ + private[sql] def useCachedData(plan: LogicalPlan): LogicalPlan = { + plan transformDown { + case currentFragment => + lookupCachedData(currentFragment) + .map(_.cachedRepresentation.withOutput(currentFragment.output)) + .getOrElse(currentFragment) + } + } + + /** + * Invalidates the cache of any data that contains `plan`. Note that it is possible that this + * function will over invalidate. + */ + private[sql] def invalidateCache(plan: LogicalPlan): Unit = writeLock { + cachedData.foreach { + case data if data.plan.collect { case p if p.sameResult(plan) => p }.nonEmpty => + data.cachedRepresentation.recache() + case _ => + } + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a42bedbe6c04e..7a55c5bf97a71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -50,6 +50,7 @@ import org.apache.spark.{Logging, SparkContext} class SQLContext(@transient val sparkContext: SparkContext) extends org.apache.spark.Logging with SQLConf + with CacheManager with ExpressionConversions with UDFRegistration with Serializable { @@ -96,7 +97,8 @@ class SQLContext(@transient val sparkContext: SparkContext) */ implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = { SparkPlan.currentContext.set(self) - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))(self)) + new SchemaRDD(this, + LogicalRDD(ScalaReflection.attributesFor[A], RDDConversions.productToRowRdd(rdd))(self)) } /** @@ -133,7 +135,7 @@ class SQLContext(@transient val sparkContext: SparkContext) def applySchema(rowRDD: RDD[Row], schema: StructType): SchemaRDD = { // TODO: use MutableProjection when rowRDD is another SchemaRDD and the applied // schema differs from the existing schema on any field data type. - val logicalPlan = SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRDD))(self) + val logicalPlan = LogicalRDD(schema.toAttributes, rowRDD)(self) new SchemaRDD(this, logicalPlan) } @@ -272,45 +274,6 @@ class SQLContext(@transient val sparkContext: SparkContext) def table(tableName: String): SchemaRDD = new SchemaRDD(this, catalog.lookupRelation(None, tableName)) - /** Caches the specified table in-memory. */ - def cacheTable(tableName: String): Unit = { - val currentTable = table(tableName).queryExecution.analyzed - val asInMemoryRelation = currentTable match { - case _: InMemoryRelation => - currentTable - - case _ => - InMemoryRelation(useCompression, columnBatchSize, executePlan(currentTable).executedPlan) - } - - catalog.registerTable(None, tableName, asInMemoryRelation) - } - - /** Removes the specified table from the in-memory cache. */ - def uncacheTable(tableName: String): Unit = { - table(tableName).queryExecution.analyzed match { - // This is kind of a hack to make sure that if this was just an RDD registered as a table, - // we reregister the RDD as a table. - case inMem @ InMemoryRelation(_, _, _, e: ExistingRdd) => - inMem.cachedColumnBuffers.unpersist() - catalog.unregisterTable(None, tableName) - catalog.registerTable(None, tableName, SparkLogicalPlan(e)(self)) - case inMem: InMemoryRelation => - inMem.cachedColumnBuffers.unpersist() - catalog.unregisterTable(None, tableName) - case plan => throw new IllegalArgumentException(s"Table $tableName is not cached: $plan") - } - } - - /** Returns true if the table is currently cached in-memory. */ - def isCached(tableName: String): Boolean = { - val relation = table(tableName).queryExecution.analyzed - relation match { - case _: InMemoryRelation => true - case _ => false - } - } - protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext @@ -401,10 +364,12 @@ class SQLContext(@transient val sparkContext: SparkContext) lazy val analyzed = ExtractPythonUdfs(analyzer(logical)) lazy val optimizedPlan = optimizer(analyzed) + lazy val withCachedData = useCachedData(optimizedPlan) + // TODO: Don't just pick the first one... lazy val sparkPlan = { SparkPlan.currentContext.set(self) - planner(optimizedPlan).next() + planner(withCachedData).next() } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. @@ -526,6 +491,6 @@ class SQLContext(@transient val sparkContext: SparkContext) iter.map { m => new GenericRow(m): Row} } - new SchemaRDD(this, SparkLogicalPlan(ExistingRdd(schema.toAttributes, rowRdd))(self)) + new SchemaRDD(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 3b873f7c62cb6..594bf8ffc20e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import java.util.{Map => JMap, List => JList} +import org.apache.spark.storage.StorageLevel + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -32,7 +34,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.api.java.JavaRDD /** @@ -442,8 +444,7 @@ class SchemaRDD( */ private def applySchema(rdd: RDD[Row]): SchemaRDD = { new SchemaRDD(sqlContext, - SparkLogicalPlan( - ExistingRdd(queryExecution.analyzed.output.map(_.newInstance), rdd))(sqlContext)) + LogicalRDD(queryExecution.analyzed.output.map(_.newInstance()), rdd)(sqlContext)) } // ======================================================================= @@ -497,4 +498,20 @@ class SchemaRDD( override def subtract(other: RDD[Row], p: Partitioner) (implicit ord: Ordering[Row] = null): SchemaRDD = applySchema(super.subtract(other, p)(ord)) + + /** Overridden cache function will always use the in-memory columnar caching. */ + override def cache(): this.type = { + sqlContext.cacheQuery(this) + this + } + + override def persist(newLevel: StorageLevel): this.type = { + sqlContext.cacheQuery(this, newLevel) + this + } + + override def unpersist(blocking: Boolean): this.type = { + sqlContext.uncacheQuery(this, blocking) + this + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index e52eeb3e1c47e..25ba7d88ba538 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.execution.LogicalRDD /** * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) @@ -55,8 +55,7 @@ private[sql] trait SchemaRDDLike { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToFile => - queryExecution.toRdd - SparkLogicalPlan(queryExecution.executedPlan)(sqlContext) + LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => baseLogicalPlan } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 150ff8a42063d..c006c4330ff66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.json.JsonRDD import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.sql.execution.LogicalRDD import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType import org.apache.spark.util.Utils @@ -100,7 +100,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow } } - new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))(sqlContext)) + new JavaSchemaRDD(sqlContext, LogicalRDD(schema, rowRdd)(sqlContext)) } /** @@ -114,7 +114,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { val scalaRowRDD = rowRDD.rdd.map(r => r.row) val scalaSchema = asScalaDataType(schema).asInstanceOf[SStructType] val logicalPlan = - SparkLogicalPlan(ExistingRdd(scalaSchema.toAttributes, scalaRowRDD))(sqlContext) + LogicalRDD(scalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } @@ -151,7 +151,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { val appliedScalaSchema = JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0)) val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) val logicalPlan = - SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + LogicalRDD(appliedScalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } @@ -167,7 +167,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { JsonRDD.nullTypeToStringType(JsonRDD.inferSchema(json.rdd, 1.0))).asInstanceOf[SStructType] val scalaRowRDD = JsonRDD.jsonStringToRow(json.rdd, appliedScalaSchema) val logicalPlan = - SparkLogicalPlan(ExistingRdd(appliedScalaSchema.toAttributes, scalaRowRDD))(sqlContext) + LogicalRDD(appliedScalaSchema.toAttributes, scalaRowRDD)(sqlContext) new JavaSchemaRDD(sqlContext, logicalPlan) } 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 8a3612cdf19be..cec82a7f2df94 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 @@ -27,10 +27,15 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.{LeafNode, SparkPlan} +import org.apache.spark.storage.StorageLevel private[sql] object InMemoryRelation { - def apply(useCompression: Boolean, batchSize: Int, child: SparkPlan): InMemoryRelation = - new InMemoryRelation(child.output, useCompression, batchSize, child)() + def apply( + useCompression: Boolean, + batchSize: Int, + storageLevel: StorageLevel, + child: SparkPlan): InMemoryRelation = + new InMemoryRelation(child.output, useCompression, batchSize, storageLevel, child)() } private[sql] case class CachedBatch(buffers: Array[ByteBuffer], stats: Row) @@ -39,6 +44,7 @@ private[sql] case class InMemoryRelation( output: Seq[Attribute], useCompression: Boolean, batchSize: Int, + storageLevel: StorageLevel, child: SparkPlan) (private var _cachedColumnBuffers: RDD[CachedBatch] = null) extends LogicalPlan with MultiInstanceRelation { @@ -51,6 +57,16 @@ private[sql] case class InMemoryRelation( // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { + buildBuffers() + } + + def recache() = { + _cachedColumnBuffers.unpersist() + _cachedColumnBuffers = null + buildBuffers() + } + + private def buildBuffers(): Unit = { val output = child.output val cached = child.execute().mapPartitions { rowIterator => new Iterator[CachedBatch] { @@ -80,12 +96,17 @@ private[sql] case class InMemoryRelation( def hasNext = rowIterator.hasNext } - }.cache() + }.persist(storageLevel) cached.setName(child.toString) _cachedColumnBuffers = cached } + def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { + InMemoryRelation( + newOutput, useCompression, batchSize, storageLevel, child)(_cachedColumnBuffers) + } + override def children = Seq.empty override def newInstance() = { @@ -93,6 +114,7 @@ private[sql] case class InMemoryRelation( output.map(_.newInstance), useCompression, batchSize, + storageLevel, child)( _cachedColumnBuffers).asInstanceOf[this.type] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala new file mode 100644 index 0000000000000..2ddf513b6fc98 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -0,0 +1,119 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{SQLContext, Row} +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +object RDDConversions { + def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { + data.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val bufferedIterator = iterator.buffered + val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) + + bufferedIterator.map { r => + var i = 0 + while (i < mutableRow.length) { + mutableRow(i) = ScalaReflection.convertToCatalyst(r.productElement(i)) + i += 1 + } + + mutableRow + } + } + } + } + + /* + def toLogicalPlan[A <: Product : TypeTag](productRdd: RDD[A]): LogicalPlan = { + LogicalRDD(ScalaReflection.attributesFor[A], productToRowRdd(productRdd)) + } + */ +} + +case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext) + extends LogicalPlan with MultiInstanceRelation { + + def children = Nil + + def newInstance() = + LogicalRDD(output.map(_.newInstance()), rdd)(sqlContext).asInstanceOf[this.type] + + override def sameResult(plan: LogicalPlan) = plan match { + case LogicalRDD(_, otherRDD) => rdd.id == otherRDD.id + case _ => false + } + + @transient override lazy val statistics = Statistics( + // TODO: Instead of returning a default value here, find a way to return a meaningful size + // estimate for RDDs. See PR 1238 for more discussions. + sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + ) +} + +case class PhysicalRDD(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { + override def execute() = rdd +} + +@deprecated("Use LogicalRDD", "1.2.0") +case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { + override def execute() = rdd +} + +@deprecated("Use LogicalRDD", "1.2.0") +case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQLContext) + extends LogicalPlan with MultiInstanceRelation { + + def output = alreadyPlanned.output + override def children = Nil + + override final def newInstance(): this.type = { + SparkLogicalPlan( + alreadyPlanned match { + case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) + case _ => sys.error("Multiple instance of the same relation detected.") + })(sqlContext).asInstanceOf[this.type] + } + + override def sameResult(plan: LogicalPlan) = plan match { + case SparkLogicalPlan(ExistingRdd(_, rdd)) => + rdd.id == alreadyPlanned.asInstanceOf[ExistingRdd].rdd.id + case _ => false + } + + @transient override lazy val statistics = Statistics( + // TODO: Instead of returning a default value here, find a way to return a meaningful size + // estimate for RDDs. See PR 1238 for more discussions. + sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) + ) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 2b8913985b028..b1a7948b66cb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -126,39 +126,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } } -/** - * :: DeveloperApi :: - * Allows already planned SparkQueries to be linked into logical query plans. - * - * Note that in general it is not valid to use this class to link multiple copies of the same - * physical operator into the same query plan as this violates the uniqueness of expression ids. - * Special handling exists for ExistingRdd as these are already leaf operators and thus we can just - * replace the output attributes with new copies of themselves without breaking any attribute - * linking. - */ -@DeveloperApi -case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQLContext) - extends LogicalPlan with MultiInstanceRelation { - - def output = alreadyPlanned.output - override def children = Nil - - override final def newInstance(): this.type = { - SparkLogicalPlan( - alreadyPlanned match { - case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) - case _ => sys.error("Multiple instance of the same relation detected.") - })(sqlContext).asInstanceOf[this.type] - } - - @transient override lazy val statistics = Statistics( - // TODO: Instead of returning a default value here, find a way to return a meaningful size - // estimate for RDDs. See PR 1238 for more discussions. - sizeInBytes = BigInt(sqlContext.defaultSizeInBytes) - ) - -} - private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { self: Product => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 45687d960404c..cf93d5ad7b503 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -272,10 +272,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Aggregate(partial = false, group, agg, planLater(child)) :: Nil case logical.Sample(fraction, withReplacement, seed, child) => execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil + case SparkLogicalPlan(alreadyPlanned) => alreadyPlanned :: Nil case logical.LocalRelation(output, data) => - ExistingRdd( + PhysicalRDD( output, - ExistingRdd.productToRowRdd(sparkContext.parallelize(data, numPartitions))) :: Nil + RDDConversions.productToRowRdd(sparkContext.parallelize(data, numPartitions))) :: Nil case logical.Limit(IntegerLiteral(limit), child) => execution.Limit(limit, planLater(child)) :: Nil case Unions(unionChildren) => @@ -287,12 +288,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Generate(generator, join, outer, _, child) => execution.Generate(generator, join = join, outer = outer, planLater(child)) :: Nil case logical.NoRelation => - execution.ExistingRdd(Nil, singleRowRdd) :: Nil + execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil case e @ EvaluatePython(udf, child) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil - case SparkLogicalPlan(existingPlan) => existingPlan :: Nil + case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index cac376608be29..977f3c9f32096 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -210,45 +210,6 @@ case class Sort( override def output = child.output } -/** - * :: DeveloperApi :: - */ -@DeveloperApi -object ExistingRdd { - def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { - data.mapPartitions { iterator => - if (iterator.isEmpty) { - Iterator.empty - } else { - val bufferedIterator = iterator.buffered - val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) - - bufferedIterator.map { r => - var i = 0 - while (i < mutableRow.length) { - mutableRow(i) = ScalaReflection.convertToCatalyst(r.productElement(i)) - i += 1 - } - - mutableRow - } - } - } - } - - def fromProductRdd[A <: Product : TypeTag](productRdd: RDD[A]) = { - ExistingRdd(ScalaReflection.attributesFor[A], productToRowRdd(productRdd)) - } -} - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { - override def execute() = rdd -} - /** * :: DeveloperApi :: * Computes the set of distinct input rows using a HashSet. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 591592841e9fe..957388e99bd85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -20,13 +20,30 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.test.TestSQLContext._ case class BigData(s: String) class CachedTableSuite extends QueryTest { + import TestSQLContext._ TestData // Load test tables. + /** + * Throws a test failed exception when the number of cached tables differs from the expected + * number. + */ + def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val cachedData = planWithCaching collect { + case cached: InMemoryRelation => cached + } + + if (cachedData.size != numCachedTables) { + fail( + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } + } + test("too big for memory") { val data = "*" * 10000 sparkContext.parallelize(1 to 1000000, 1).map(_ => BigData(data)).registerTempTable("bigData") @@ -35,19 +52,21 @@ class CachedTableSuite extends QueryTest { uncacheTable("bigData") } + test("calling .cache() should use inmemory columnar caching") { + table("testData").cache() + + assertCached(table("testData")) + } + test("SPARK-1669: cacheTable should be idempotent") { assume(!table("testData").logicalPlan.isInstanceOf[InMemoryRelation]) cacheTable("testData") - table("testData").queryExecution.analyzed match { - case _: InMemoryRelation => - case _ => - fail("testData should be cached") - } + assertCached(table("testData")) cacheTable("testData") table("testData").queryExecution.analyzed match { - case InMemoryRelation(_, _, _, _: InMemoryColumnarTableScan) => + case InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan) => fail("cacheTable is not idempotent") case _ => @@ -55,81 +74,69 @@ class CachedTableSuite extends QueryTest { } test("read from cached table and uncache") { - TestSQLContext.cacheTable("testData") + cacheTable("testData") checkAnswer( - TestSQLContext.table("testData"), + table("testData"), testData.collect().toSeq ) - TestSQLContext.table("testData").queryExecution.analyzed match { - case _ : InMemoryRelation => // Found evidence of caching - case noCache => fail(s"No cache node found in plan $noCache") - } + assertCached(table("testData")) - TestSQLContext.uncacheTable("testData") + uncacheTable("testData") checkAnswer( - TestSQLContext.table("testData"), + table("testData"), testData.collect().toSeq ) - TestSQLContext.table("testData").queryExecution.analyzed match { - case cachePlan: InMemoryRelation => - fail(s"Table still cached after uncache: $cachePlan") - case noCache => // Table uncached successfully - } + assertCached(table("testData"), 0) } test("correct error on uncache of non-cached table") { intercept[IllegalArgumentException] { - TestSQLContext.uncacheTable("testData") + uncacheTable("testData") } } test("SELECT Star Cached Table") { - TestSQLContext.sql("SELECT * FROM testData").registerTempTable("selectStar") - TestSQLContext.cacheTable("selectStar") - TestSQLContext.sql("SELECT * FROM selectStar WHERE key = 1").collect() - TestSQLContext.uncacheTable("selectStar") + sql("SELECT * FROM testData").registerTempTable("selectStar") + cacheTable("selectStar") + sql("SELECT * FROM selectStar WHERE key = 1").collect() + uncacheTable("selectStar") } test("Self-join cached") { val unCachedAnswer = - TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect() - TestSQLContext.cacheTable("testData") + sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key").collect() + cacheTable("testData") checkAnswer( - TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key"), + sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key"), unCachedAnswer.toSeq) - TestSQLContext.uncacheTable("testData") + uncacheTable("testData") } test("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { - TestSQLContext.sql("CACHE TABLE testData") - TestSQLContext.table("testData").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => // Found evidence of caching - case _ => fail(s"Table 'testData' should be cached") - } - assert(TestSQLContext.isCached("testData"), "Table 'testData' should be cached") + sql("CACHE TABLE testData") + assertCached(table("testData")) - TestSQLContext.sql("UNCACHE TABLE testData") - TestSQLContext.table("testData").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => fail(s"Table 'testData' should not be cached") - case _ => // Found evidence of uncaching - } - assert(!TestSQLContext.isCached("testData"), "Table 'testData' should not be cached") + assert(isCached("testData"), "Table 'testData' should be cached") + + sql("UNCACHE TABLE testData") + assertCached(table("testData"), 0) + assert(!isCached("testData"), "Table 'testData' should not be cached") } test("CACHE TABLE tableName AS SELECT Star Table") { - TestSQLContext.sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") - TestSQLContext.sql("SELECT * FROM testCacheTable WHERE key = 1").collect() - assert(TestSQLContext.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") - TestSQLContext.uncacheTable("testCacheTable") + sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + sql("SELECT * FROM testCacheTable WHERE key = 1").collect() + assert(isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + uncacheTable("testCacheTable") } test("'CACHE TABLE tableName AS SELECT ..'") { - TestSQLContext.sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") - assert(TestSQLContext.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") - TestSQLContext.uncacheTable("testCacheTable") + sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") + assert(isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + uncacheTable("testCacheTable") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index c1278248ef655..9775dd26b7773 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.{QueryTest, TestData} +import org.apache.spark.storage.StorageLevel.MEMORY_ONLY class InMemoryColumnarQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ @@ -27,7 +28,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("simple columnar query") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, plan) + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().toSeq) } @@ -42,7 +43,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("projection") { val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, plan) + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().map { case Row(key: Int, value: String) => value -> key @@ -51,7 +52,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan - val scan = InMemoryRelation(useCompression = true, 5, plan) + val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().toSeq) checkAnswer(scan, testData.collect().toSeq) 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 989a9784a438d..cc0605b0adb35 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 @@ -133,11 +133,6 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => castChildOutput(p, table, child) - - case p @ logical.InsertIntoTable( - InMemoryRelation(_, _, _, - HiveTableScan(_, table, _)), _, child, _) => - castChildOutput(p, table, child) } def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) = { @@ -306,7 +301,7 @@ private[hive] case class MetastoreRelation HiveMetastoreTypes.toDataType(f.getType), // Since data can be dumped in randomly with no validation, everything is nullable. nullable = true - )(qualifiers = tableName +: alias.toSeq) + )(qualifiers = Seq(alias.getOrElse(tableName))) } // Must be a stable value since new attributes are born here. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 8ac17f37201a8..508d8239c7628 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType -import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ @@ -161,10 +160,7 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil - case logical.InsertIntoTable( - InMemoryRelation(_, _, _, - HiveTableScan(_, table, _)), partition, child, overwrite) => - InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil + case logical.CreateTableAsSelect(database, tableName, child) => val query = planLater(child) CreateTableAsSelect( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 4a999b98ad92b..c0e69393cc2e3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -353,7 +353,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { var cacheTables: Boolean = false def loadTestTable(name: String) { if (!(loadedTables contains name)) { - // Marks the table as loaded first to prevent infite mutually recursive table loading. + // Marks the table as loaded first to prevent infinite mutually recursive table loading. loadedTables += name logInfo(s"Loading test table $name") val createCmds = @@ -383,6 +383,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { log.asInstanceOf[org.apache.log4j.Logger].setLevel(org.apache.log4j.Level.WARN) } + clearCache() loadedTables.clear() catalog.client.getAllTables("default").foreach { t => logDebug(s"Deleting table $t") @@ -428,7 +429,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { loadTestTable("srcpart") } catch { case e: Exception => - logError(s"FATAL ERROR: Failed to reset TestDB state. $e") + logError("FATAL ERROR: Failed to reset TestDB state.", e) // At this point there is really no reason to continue, but the test framework traps exits. // So instead we just pause forever so that at least the developer can see where things // started to go wrong. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 16a8c782acdfa..f8b4e898ec41d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -267,6 +267,9 @@ case class InsertIntoHiveTable( holdDDLTime) } + // Invalidate the cache. + sqlContext.invalidateCache(table) + // It would be nice to just return the childRdd unchanged so insert operations could be chained, // however for now we return an empty list to simplify compatibility checks with hive, which // does not return anything for insert operations. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index b3057cd618c66..158cfb5bbee7c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -17,22 +17,60 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.{QueryTest, SchemaRDD} import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} -import org.apache.spark.sql.hive.execution.HiveComparisonTest import org.apache.spark.sql.hive.test.TestHive -class CachedTableSuite extends HiveComparisonTest { +class CachedTableSuite extends QueryTest { import TestHive._ - TestHive.loadTestTable("src") + /** + * Throws a test failed exception when the number of cached tables differs from the expected + * number. + */ + def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val cachedData = planWithCaching collect { + case cached: InMemoryRelation => cached + } + + if (cachedData.size != numCachedTables) { + fail( + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } + } test("cache table") { - TestHive.cacheTable("src") + val preCacheResults = sql("SELECT * FROM src").collect().toSeq + + cacheTable("src") + assertCached(sql("SELECT * FROM src")) + + checkAnswer( + sql("SELECT * FROM src"), + preCacheResults) + + uncacheTable("src") + assertCached(sql("SELECT * FROM src"), 0) } - createQueryTest("read from cached table", - "SELECT * FROM src LIMIT 1", reset = false) + test("cache invalidation") { + sql("CREATE TABLE cachedTable(key INT, value STRING)") + + sql("INSERT INTO TABLE cachedTable SELECT * FROM src") + checkAnswer(sql("SELECT * FROM cachedTable"), table("src").collect().toSeq) + + cacheTable("cachedTable") + checkAnswer(sql("SELECT * FROM cachedTable"), table("src").collect().toSeq) + + sql("INSERT INTO TABLE cachedTable SELECT * FROM src") + checkAnswer( + sql("SELECT * FROM cachedTable"), + table("src").collect().toSeq ++ table("src").collect().toSeq) + + sql("DROP TABLE cachedTable") + } test("Drop cached table") { sql("CREATE TABLE test(a INT)") @@ -48,25 +86,6 @@ class CachedTableSuite extends HiveComparisonTest { sql("DROP TABLE IF EXISTS nonexistantTable") } - test("check that table is cached and uncache") { - TestHive.table("src").queryExecution.analyzed match { - case _ : InMemoryRelation => // Found evidence of caching - case noCache => fail(s"No cache node found in plan $noCache") - } - TestHive.uncacheTable("src") - } - - createQueryTest("read from uncached table", - "SELECT * FROM src LIMIT 1", reset = false) - - test("make sure table is uncached") { - TestHive.table("src").queryExecution.analyzed match { - case cachePlan: InMemoryRelation => - fail(s"Table still cached after uncache: $cachePlan") - case noCache => // Table uncached successfully - } - } - test("correct error on uncache of non-cached table") { intercept[IllegalArgumentException] { TestHive.uncacheTable("src") @@ -75,23 +94,24 @@ class CachedTableSuite extends HiveComparisonTest { test("'CACHE TABLE' and 'UNCACHE TABLE' HiveQL statement") { TestHive.sql("CACHE TABLE src") - TestHive.table("src").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => // Found evidence of caching - case _ => fail(s"Table 'src' should be cached") - } + assertCached(table("src")) assert(TestHive.isCached("src"), "Table 'src' should be cached") TestHive.sql("UNCACHE TABLE src") - TestHive.table("src").queryExecution.executedPlan match { - case _: InMemoryColumnarTableScan => fail(s"Table 'src' should not be cached") - case _ => // Found evidence of uncaching - } + assertCached(table("src"), 0) assert(!TestHive.isCached("src"), "Table 'src' should not be cached") } - - test("'CACHE TABLE tableName AS SELECT ..'") { - TestHive.sql("CACHE TABLE testCacheTable AS SELECT * FROM src") - assert(TestHive.isCached("testCacheTable"), "Table 'testCacheTable' should be cached") - TestHive.uncacheTable("testCacheTable") - } + + test("CACHE TABLE AS SELECT") { + assertCached(sql("SELECT * FROM src"), 0) + sql("CACHE TABLE test AS SELECT key FROM src") + + checkAnswer( + sql("SELECT * FROM test"), + sql("SELECT key FROM src").collect().toSeq) + + assertCached(sql("SELECT * FROM test")) + + assertCached(sql("SELECT * FROM test JOIN test"), 2) + } } From a8c52d5343e19731909e73db5de151a324d31cd5 Mon Sep 17 00:00:00 2001 From: Brenden Matthews Date: Fri, 3 Oct 2014 12:58:04 -0700 Subject: [PATCH 0887/1492] [SPARK-3535][Mesos] Fix resource handling. Author: Brenden Matthews Closes #2401 from brndnmtthws/master and squashes the following commits: 4abaa5d [Brenden Matthews] [SPARK-3535][Mesos] Fix resource handling. --- .../mesos/CoarseMesosSchedulerBackend.scala | 7 ++-- .../scheduler/cluster/mesos/MemoryUtils.scala | 35 +++++++++++++++++++ .../cluster/mesos/MesosSchedulerBackend.scala | 34 ++++++++++++++---- docs/configuration.md | 11 ++++++ 4 files changed, 79 insertions(+), 8 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 64568409dbafd..3161f1ee9fa8a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -198,7 +198,9 @@ private[spark] class CoarseMesosSchedulerBackend( val slaveId = offer.getSlaveId.toString val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (totalCoresAcquired < maxCores && mem >= sc.executorMemory && cpus >= 1 && + if (totalCoresAcquired < maxCores && + mem >= MemoryUtils.calculateTotalMemory(sc) && + cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && !slaveIdsWithExecutors.contains(slaveId)) { // Launch an executor on the slave @@ -214,7 +216,8 @@ private[spark] class CoarseMesosSchedulerBackend( .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave)) .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) - .addResources(createResource("mem", sc.executorMemory)) + .addResources(createResource("mem", + MemoryUtils.calculateTotalMemory(sc))) .build() d.launchTasks( Collections.singleton(offer.getId), Collections.singletonList(task), filters) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala new file mode 100644 index 0000000000000..5101ec8352e79 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala @@ -0,0 +1,35 @@ +/* + * 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.mesos + +import org.apache.spark.SparkContext + +private[spark] object MemoryUtils { + // These defaults copied from YARN + val OVERHEAD_FRACTION = 1.07 + val OVERHEAD_MINIMUM = 384 + + def calculateTotalMemory(sc: SparkContext) = { + math.max( + sc.conf.getOption("spark.mesos.executor.memoryOverhead") + .getOrElse(OVERHEAD_MINIMUM.toString) + .toInt + sc.executorMemory, + OVERHEAD_FRACTION * sc.executorMemory + ) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index a9ef126f5de0e..4c49aa074ebc0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -124,15 +124,24 @@ private[spark] class MesosSchedulerBackend( command.setValue("cd %s*; ./sbin/spark-executor".format(basename)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } + val cpus = Resource.newBuilder() + .setName("cpus") + .setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder() + .setValue(scheduler.CPUS_PER_TASK).build()) + .build() val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(sc.executorMemory).build()) + .setScalar( + Value.Scalar.newBuilder() + .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) .build() ExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) .setCommand(command) .setData(ByteString.copyFrom(createExecArg())) + .addResources(cpus) .addResources(memory) .build() } @@ -204,18 +213,31 @@ private[spark] class MesosSchedulerBackend( val offerableWorkers = new ArrayBuffer[WorkerOffer] val offerableIndices = new HashMap[String, Int] - def enoughMemory(o: Offer) = { + def sufficientOffer(o: Offer) = { val mem = getResource(o.getResourcesList, "mem") + val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - mem >= sc.executorMemory || slaveIdsWithExecutors.contains(slaveId) + (mem >= MemoryUtils.calculateTotalMemory(sc) && + // need at least 1 for executor, 1 for task + cpus >= 2 * scheduler.CPUS_PER_TASK) || + (slaveIdsWithExecutors.contains(slaveId) && + cpus >= scheduler.CPUS_PER_TASK) } - for ((offer, index) <- offers.zipWithIndex if enoughMemory(offer)) { - offerableIndices.put(offer.getSlaveId.getValue, index) + for ((offer, index) <- offers.zipWithIndex if sufficientOffer(offer)) { + val slaveId = offer.getSlaveId.getValue + offerableIndices.put(slaveId, index) + val cpus = if (slaveIdsWithExecutors.contains(slaveId)) { + getResource(offer.getResourcesList, "cpus").toInt + } else { + // If the executor doesn't exist yet, subtract CPU for executor + getResource(offer.getResourcesList, "cpus").toInt - + scheduler.CPUS_PER_TASK + } offerableWorkers += new WorkerOffer( offer.getSlaveId.getValue, offer.getHostname, - getResource(offer.getResourcesList, "cpus").toInt) + cpus) } // Call into the TaskSchedulerImpl diff --git a/docs/configuration.md b/docs/configuration.md index a782809a55ec0..1c33855365170 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -253,6 +253,17 @@ Apart from these, the following properties are also available, and may be useful spark.executor.uri. + + + + +
    StructType org.apache.spark.sql.api.java + DataType.createStructType(fields)
    Note: fields is a List or an array of StructFields. Also, two fields with the same name are not allowed. @@ -1394,7 +1416,7 @@ please use factory methods provided in
    All data types of Spark SQL are located in the package of `pyspark.sql`. -You can access them by doing +You can access them by doing {% highlight python %} from pyspark.sql import * {% endhighlight %} @@ -1518,7 +1540,7 @@ from pyspark.sql import *
    StructType list or tuple + StructType(fields)
    Note: fields is a Seq of StructFields. Also, two fields with the same name are not allowed. From a9e910430fb6bb4ef1f6ae20761c43b96bb018df Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 16 Sep 2014 12:41:45 -0700 Subject: [PATCH 0713/1492] [SPARK-3546] InputStream of ManagedBuffer is not closed and causes running out of file descriptor Author: Kousuke Saruta Closes #2408 from sarutak/resolve-resource-leak-issue and squashes the following commits: 074781d [Kousuke Saruta] Modified SuffleBlockFetcherIterator 5f63f67 [Kousuke Saruta] Move metrics increment logic and debug logging outside try block b37231a [Kousuke Saruta] Modified FileSegmentManagedBuffer#nioByteBuffer to check null or not before invoking channel.close bf29d4a [Kousuke Saruta] Modified FileSegment to close channel --- .../org/apache/spark/network/ManagedBuffer.scala | 12 ++++++++++-- .../spark/storage/ShuffleBlockFetcherIterator.scala | 2 +- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala index dcecb6beeea9b..e990c1da6730f 100644 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -19,6 +19,7 @@ package org.apache.spark.network import java.io.{FileInputStream, RandomAccessFile, File, InputStream} import java.nio.ByteBuffer +import java.nio.channels.FileChannel import java.nio.channels.FileChannel.MapMode import com.google.common.io.ByteStreams @@ -66,8 +67,15 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt override def size: Long = length override def nioByteBuffer(): ByteBuffer = { - val channel = new RandomAccessFile(file, "r").getChannel - channel.map(MapMode.READ_ONLY, offset, length) + var channel: FileChannel = null + try { + channel = new RandomAccessFile(file, "r").getChannel + channel.map(MapMode.READ_ONLY, offset, length) + } finally { + if (channel != null) { + channel.close() + } + } } override def inputStream(): InputStream = { diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index c8e708aa6b1bc..d868758a7f549 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -23,7 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashSet import scala.collection.mutable.Queue -import org.apache.spark.{TaskContext, Logging, SparkException} +import org.apache.spark.{TaskContext, Logging} import org.apache.spark.network.{ManagedBuffer, BlockFetchingListener, BlockTransferService} import org.apache.spark.serializer.Serializer import org.apache.spark.util.Utils From ec1adecbb72d291d7ef122fb0505bae53116e0e6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 16 Sep 2014 12:51:58 -0700 Subject: [PATCH 0714/1492] [SPARK-3430] [PySpark] [Doc] generate PySpark API docs using Sphinx Using Sphinx to generate API docs for PySpark. requirement: Sphinx ``` $ cd python/docs/ $ make html ``` The generated API docs will be located at python/docs/_build/html/index.html It can co-exists with those generated by Epydoc. This is the first working version, after merging in, then we can continue to improve it and replace the epydoc finally. Author: Davies Liu Closes #2292 from davies/sphinx and squashes the following commits: 425a3b1 [Davies Liu] cleanup 1573298 [Davies Liu] move docs to python/docs/ 5fe3903 [Davies Liu] Merge branch 'master' into sphinx 9468ab0 [Davies Liu] fix makefile b408f38 [Davies Liu] address all comments e2ccb1b [Davies Liu] update name and version 9081ead [Davies Liu] generate PySpark API docs using Sphinx --- python/docs/Makefile | 179 ++++++++++++++++++ python/docs/conf.py | 332 ++++++++++++++++++++++++++++++++++ python/docs/epytext.py | 27 +++ python/docs/index.rst | 37 ++++ python/docs/make.bat | 242 +++++++++++++++++++++++++ python/docs/modules.rst | 7 + python/docs/pyspark.mllib.rst | 77 ++++++++ python/docs/pyspark.rst | 18 ++ python/docs/pyspark.sql.rst | 10 + python/pyspark/broadcast.py | 3 + python/pyspark/context.py | 2 +- python/pyspark/serializers.py | 3 + python/pyspark/sql.py | 12 +- 13 files changed, 944 insertions(+), 5 deletions(-) create mode 100644 python/docs/Makefile create mode 100644 python/docs/conf.py create mode 100644 python/docs/epytext.py create mode 100644 python/docs/index.rst create mode 100644 python/docs/make.bat create mode 100644 python/docs/modules.rst create mode 100644 python/docs/pyspark.mllib.rst create mode 100644 python/docs/pyspark.rst create mode 100644 python/docs/pyspark.sql.rst diff --git a/python/docs/Makefile b/python/docs/Makefile new file mode 100644 index 0000000000000..8a1324eecd325 --- /dev/null +++ b/python/docs/Makefile @@ -0,0 +1,179 @@ +# Makefile for Sphinx documentation +# + +# You can set these variables from the command line. +SPHINXOPTS = +SPHINXBUILD = sphinx-build +PAPER = +BUILDDIR = _build + +export PYTHONPATH=$(realpath ..):$(realpath ../lib/py4j-0.8.2.1-src.zip) + +# User-friendly check for sphinx-build +ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) +$(error The '$(SPHINXBUILD)' command was not found. Make sure you have Sphinx installed, then set the SPHINXBUILD environment variable to point to the full path of the '$(SPHINXBUILD)' executable. Alternatively you can add the directory with the executable to your PATH. If you don't have Sphinx installed, grab it from http://sphinx-doc.org/) +endif + +# Internal variables. +PAPEROPT_a4 = -D latex_paper_size=a4 +PAPEROPT_letter = -D latex_paper_size=letter +ALLSPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) . +# the i18n builder cannot share the environment and doctrees with the others +I18NSPHINXOPTS = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) . + +.PHONY: help clean html dirhtml singlehtml pickle json htmlhelp qthelp devhelp epub latex latexpdf text man changes linkcheck doctest gettext + +help: + @echo "Please use \`make ' where is one of" + @echo " html to make standalone HTML files" + @echo " dirhtml to make HTML files named index.html in directories" + @echo " singlehtml to make a single large HTML file" + @echo " pickle to make pickle files" + @echo " json to make JSON files" + @echo " htmlhelp to make HTML files and a HTML help project" + @echo " qthelp to make HTML files and a qthelp project" + @echo " devhelp to make HTML files and a Devhelp project" + @echo " epub to make an epub" + @echo " latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter" + @echo " latexpdf to make LaTeX files and run them through pdflatex" + @echo " latexpdfja to make LaTeX files and run them through platex/dvipdfmx" + @echo " text to make text files" + @echo " man to make manual pages" + @echo " texinfo to make Texinfo files" + @echo " info to make Texinfo files and run them through makeinfo" + @echo " gettext to make PO message catalogs" + @echo " changes to make an overview of all changed/added/deprecated items" + @echo " xml to make Docutils-native XML files" + @echo " pseudoxml to make pseudoxml-XML files for display purposes" + @echo " linkcheck to check all external links for integrity" + @echo " doctest to run all doctests embedded in the documentation (if enabled)" + +clean: + rm -rf $(BUILDDIR)/* + +html: + $(SPHINXBUILD) -b html $(ALLSPHINXOPTS) $(BUILDDIR)/html + @echo + @echo "Build finished. The HTML pages are in $(BUILDDIR)/html." + +dirhtml: + $(SPHINXBUILD) -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml + @echo + @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." + +singlehtml: + $(SPHINXBUILD) -b singlehtml $(ALLSPHINXOPTS) $(BUILDDIR)/singlehtml + @echo + @echo "Build finished. The HTML page is in $(BUILDDIR)/singlehtml." + +pickle: + $(SPHINXBUILD) -b pickle $(ALLSPHINXOPTS) $(BUILDDIR)/pickle + @echo + @echo "Build finished; now you can process the pickle files." + +json: + $(SPHINXBUILD) -b json $(ALLSPHINXOPTS) $(BUILDDIR)/json + @echo + @echo "Build finished; now you can process the JSON files." + +htmlhelp: + $(SPHINXBUILD) -b htmlhelp $(ALLSPHINXOPTS) $(BUILDDIR)/htmlhelp + @echo + @echo "Build finished; now you can run HTML Help Workshop with the" \ + ".hhp project file in $(BUILDDIR)/htmlhelp." + +qthelp: + $(SPHINXBUILD) -b qthelp $(ALLSPHINXOPTS) $(BUILDDIR)/qthelp + @echo + @echo "Build finished; now you can run "qcollectiongenerator" with the" \ + ".qhcp project file in $(BUILDDIR)/qthelp, like this:" + @echo "# qcollectiongenerator $(BUILDDIR)/qthelp/pyspark.qhcp" + @echo "To view the help file:" + @echo "# assistant -collectionFile $(BUILDDIR)/qthelp/pyspark.qhc" + +devhelp: + $(SPHINXBUILD) -b devhelp $(ALLSPHINXOPTS) $(BUILDDIR)/devhelp + @echo + @echo "Build finished." + @echo "To view the help file:" + @echo "# mkdir -p $$HOME/.local/share/devhelp/pyspark" + @echo "# ln -s $(BUILDDIR)/devhelp $$HOME/.local/share/devhelp/pyspark" + @echo "# devhelp" + +epub: + $(SPHINXBUILD) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub + @echo + @echo "Build finished. The epub file is in $(BUILDDIR)/epub." + +latex: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo + @echo "Build finished; the LaTeX files are in $(BUILDDIR)/latex." + @echo "Run \`make' in that directory to run these through (pdf)latex" \ + "(use \`make latexpdf' here to do that automatically)." + +latexpdf: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo "Running LaTeX files through pdflatex..." + $(MAKE) -C $(BUILDDIR)/latex all-pdf + @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." + +latexpdfja: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo "Running LaTeX files through platex and dvipdfmx..." + $(MAKE) -C $(BUILDDIR)/latex all-pdf-ja + @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." + +text: + $(SPHINXBUILD) -b text $(ALLSPHINXOPTS) $(BUILDDIR)/text + @echo + @echo "Build finished. The text files are in $(BUILDDIR)/text." + +man: + $(SPHINXBUILD) -b man $(ALLSPHINXOPTS) $(BUILDDIR)/man + @echo + @echo "Build finished. The manual pages are in $(BUILDDIR)/man." + +texinfo: + $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo + @echo + @echo "Build finished. The Texinfo files are in $(BUILDDIR)/texinfo." + @echo "Run \`make' in that directory to run these through makeinfo" \ + "(use \`make info' here to do that automatically)." + +info: + $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo + @echo "Running Texinfo files through makeinfo..." + make -C $(BUILDDIR)/texinfo info + @echo "makeinfo finished; the Info files are in $(BUILDDIR)/texinfo." + +gettext: + $(SPHINXBUILD) -b gettext $(I18NSPHINXOPTS) $(BUILDDIR)/locale + @echo + @echo "Build finished. The message catalogs are in $(BUILDDIR)/locale." + +changes: + $(SPHINXBUILD) -b changes $(ALLSPHINXOPTS) $(BUILDDIR)/changes + @echo + @echo "The overview file is in $(BUILDDIR)/changes." + +linkcheck: + $(SPHINXBUILD) -b linkcheck $(ALLSPHINXOPTS) $(BUILDDIR)/linkcheck + @echo + @echo "Link check complete; look for any errors in the above output " \ + "or in $(BUILDDIR)/linkcheck/output.txt." + +doctest: + $(SPHINXBUILD) -b doctest $(ALLSPHINXOPTS) $(BUILDDIR)/doctest + @echo "Testing of doctests in the sources finished, look at the " \ + "results in $(BUILDDIR)/doctest/output.txt." + +xml: + $(SPHINXBUILD) -b xml $(ALLSPHINXOPTS) $(BUILDDIR)/xml + @echo + @echo "Build finished. The XML files are in $(BUILDDIR)/xml." + +pseudoxml: + $(SPHINXBUILD) -b pseudoxml $(ALLSPHINXOPTS) $(BUILDDIR)/pseudoxml + @echo + @echo "Build finished. The pseudo-XML files are in $(BUILDDIR)/pseudoxml." diff --git a/python/docs/conf.py b/python/docs/conf.py new file mode 100644 index 0000000000000..c368cf81a003b --- /dev/null +++ b/python/docs/conf.py @@ -0,0 +1,332 @@ +# -*- coding: utf-8 -*- +# +# pyspark documentation build configuration file, created by +# sphinx-quickstart on Thu Aug 28 15:17:47 2014. +# +# This file is execfile()d with the current directory set to its +# containing dir. +# +# Note that not all possible configuration values are present in this +# autogenerated file. +# +# All configuration values have a default; values that are commented out +# serve to show the default. + +import sys +import os + +# If extensions (or modules to document with autodoc) are in another directory, +# add these directories to sys.path here. If the directory is relative to the +# documentation root, use os.path.abspath to make it absolute, like shown here. +sys.path.insert(0, os.path.abspath('.')) + +# -- General configuration ------------------------------------------------ + +# If your documentation needs a minimal Sphinx version, state it here. +#needs_sphinx = '1.0' + +# Add any Sphinx extension module names here, as strings. They can be +# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom +# ones. +extensions = [ + 'sphinx.ext.autodoc', + 'sphinx.ext.viewcode', + 'epytext', +] + +# Add any paths that contain templates here, relative to this directory. +templates_path = ['_templates'] + +# The suffix of source filenames. +source_suffix = '.rst' + +# The encoding of source files. +#source_encoding = 'utf-8-sig' + +# The master toctree document. +master_doc = 'index' + +# General information about the project. +project = u'PySpark' +copyright = u'2014, Author' + +# The version info for the project you're documenting, acts as replacement for +# |version| and |release|, also used in various other places throughout the +# built documents. +# +# The short X.Y version. +version = '1.1' +# The full version, including alpha/beta/rc tags. +release = '' + +# The language for content autogenerated by Sphinx. Refer to documentation +# for a list of supported languages. +#language = None + +# There are two options for replacing |today|: either, you set today to some +# non-false value, then it is used: +#today = '' +# Else, today_fmt is used as the format for a strftime call. +#today_fmt = '%B %d, %Y' + +# List of patterns, relative to source directory, that match files and +# directories to ignore when looking for source files. +exclude_patterns = ['_build'] + +# The reST default role (used for this markup: `text`) to use for all +# documents. +#default_role = None + +# If true, '()' will be appended to :func: etc. cross-reference text. +#add_function_parentheses = True + +# If true, the current module name will be prepended to all description +# unit titles (such as .. function::). +#add_module_names = True + +# If true, sectionauthor and moduleauthor directives will be shown in the +# output. They are ignored by default. +#show_authors = False + +# The name of the Pygments (syntax highlighting) style to use. +pygments_style = 'sphinx' + +# A list of ignored prefixes for module index sorting. +#modindex_common_prefix = [] + +# If true, keep warnings as "system message" paragraphs in the built documents. +#keep_warnings = False + + +# -- Options for HTML output ---------------------------------------------- + +# The theme to use for HTML and HTML Help pages. See the documentation for +# a list of builtin themes. +html_theme = 'default' + +# Theme options are theme-specific and customize the look and feel of a theme +# further. For a list of options available for each theme, see the +# documentation. +#html_theme_options = {} + +# Add any paths that contain custom themes here, relative to this directory. +#html_theme_path = [] + +# The name for this set of Sphinx documents. If None, it defaults to +# " v documentation". +#html_title = None + +# A shorter title for the navigation bar. Default is the same as html_title. +#html_short_title = None + +# The name of an image file (relative to this directory) to place at the top +# of the sidebar. +#html_logo = None + +# The name of an image file (within the static path) to use as favicon of the +# docs. This file should be a Windows icon file (.ico) being 16x16 or 32x32 +# pixels large. +#html_favicon = None + +# Add any paths that contain custom static files (such as style sheets) here, +# relative to this directory. They are copied after the builtin static files, +# so a file named "default.css" will overwrite the builtin "default.css". +html_static_path = ['_static'] + +# Add any extra paths that contain custom files (such as robots.txt or +# .htaccess) here, relative to this directory. These files are copied +# directly to the root of the documentation. +#html_extra_path = [] + +# If not '', a 'Last updated on:' timestamp is inserted at every page bottom, +# using the given strftime format. +#html_last_updated_fmt = '%b %d, %Y' + +# If true, SmartyPants will be used to convert quotes and dashes to +# typographically correct entities. +#html_use_smartypants = True + +# Custom sidebar templates, maps document names to template names. +#html_sidebars = {} + +# Additional templates that should be rendered to pages, maps page names to +# template names. +#html_additional_pages = {} + +# If false, no module index is generated. +#html_domain_indices = True + +# If false, no index is generated. +#html_use_index = True + +# If true, the index is split into individual pages for each letter. +#html_split_index = False + +# If true, links to the reST sources are added to the pages. +#html_show_sourcelink = True + +# If true, "Created using Sphinx" is shown in the HTML footer. Default is True. +#html_show_sphinx = True + +# If true, "(C) Copyright ..." is shown in the HTML footer. Default is True. +#html_show_copyright = True + +# If true, an OpenSearch description file will be output, and all pages will +# contain a tag referring to it. The value of this option must be the +# base URL from which the finished HTML is served. +#html_use_opensearch = '' + +# This is the file name suffix for HTML files (e.g. ".xhtml"). +#html_file_suffix = None + +# Output file base name for HTML help builder. +htmlhelp_basename = 'pysparkdoc' + + +# -- Options for LaTeX output --------------------------------------------- + +latex_elements = { +# The paper size ('letterpaper' or 'a4paper'). +#'papersize': 'letterpaper', + +# The font size ('10pt', '11pt' or '12pt'). +#'pointsize': '10pt', + +# Additional stuff for the LaTeX preamble. +#'preamble': '', +} + +# Grouping the document tree into LaTeX files. List of tuples +# (source start file, target name, title, +# author, documentclass [howto, manual, or own class]). +latex_documents = [ + ('index', 'pyspark.tex', u'pyspark Documentation', + u'Author', 'manual'), +] + +# The name of an image file (relative to this directory) to place at the top of +# the title page. +#latex_logo = None + +# For "manual" documents, if this is true, then toplevel headings are parts, +# not chapters. +#latex_use_parts = False + +# If true, show page references after internal links. +#latex_show_pagerefs = False + +# If true, show URL addresses after external links. +#latex_show_urls = False + +# Documents to append as an appendix to all manuals. +#latex_appendices = [] + +# If false, no module index is generated. +#latex_domain_indices = True + + +# -- Options for manual page output --------------------------------------- + +# One entry per manual page. List of tuples +# (source start file, name, description, authors, manual section). +man_pages = [ + ('index', 'pyspark', u'pyspark Documentation', + [u'Author'], 1) +] + +# If true, show URL addresses after external links. +#man_show_urls = False + + +# -- Options for Texinfo output ------------------------------------------- + +# Grouping the document tree into Texinfo files. List of tuples +# (source start file, target name, title, author, +# dir menu entry, description, category) +texinfo_documents = [ + ('index', 'pyspark', u'pyspark Documentation', + u'Author', 'pyspark', 'One line description of project.', + 'Miscellaneous'), +] + +# Documents to append as an appendix to all manuals. +#texinfo_appendices = [] + +# If false, no module index is generated. +#texinfo_domain_indices = True + +# How to display URL addresses: 'footnote', 'no', or 'inline'. +#texinfo_show_urls = 'footnote' + +# If true, do not generate a @detailmenu in the "Top" node's menu. +#texinfo_no_detailmenu = False + + +# -- Options for Epub output ---------------------------------------------- + +# Bibliographic Dublin Core info. +epub_title = u'pyspark' +epub_author = u'Author' +epub_publisher = u'Author' +epub_copyright = u'2014, Author' + +# The basename for the epub file. It defaults to the project name. +#epub_basename = u'pyspark' + +# The HTML theme for the epub output. Since the default themes are not optimized +# for small screen space, using the same theme for HTML and epub output is +# usually not wise. This defaults to 'epub', a theme designed to save visual +# space. +#epub_theme = 'epub' + +# The language of the text. It defaults to the language option +# or en if the language is not set. +#epub_language = '' + +# The scheme of the identifier. Typical schemes are ISBN or URL. +#epub_scheme = '' + +# The unique identifier of the text. This can be a ISBN number +# or the project homepage. +#epub_identifier = '' + +# A unique identification for the text. +#epub_uid = '' + +# A tuple containing the cover image and cover page html template filenames. +#epub_cover = () + +# A sequence of (type, uri, title) tuples for the guide element of content.opf. +#epub_guide = () + +# HTML files that should be inserted before the pages created by sphinx. +# The format is a list of tuples containing the path and title. +#epub_pre_files = [] + +# HTML files shat should be inserted after the pages created by sphinx. +# The format is a list of tuples containing the path and title. +#epub_post_files = [] + +# A list of files that should not be packed into the epub file. +epub_exclude_files = ['search.html'] + +# The depth of the table of contents in toc.ncx. +#epub_tocdepth = 3 + +# Allow duplicate toc entries. +#epub_tocdup = True + +# Choose between 'default' and 'includehidden'. +#epub_tocscope = 'default' + +# Fix unsupported image types using the PIL. +#epub_fix_images = False + +# Scale large images. +#epub_max_image_width = 0 + +# How to display URL addresses: 'footnote', 'no', or 'inline'. +#epub_show_urls = 'inline' + +# If false, no index is generated. +#epub_use_index = True diff --git a/python/docs/epytext.py b/python/docs/epytext.py new file mode 100644 index 0000000000000..61d731bff570d --- /dev/null +++ b/python/docs/epytext.py @@ -0,0 +1,27 @@ +import re + +RULES = ( + (r"<[\w.]+>", r""), + (r"L{([\w.()]+)}", r":class:`\1`"), + (r"[LC]{(\w+\.\w+)\(\)}", r":func:`\1`"), + (r"C{([\w.()]+)}", r":class:`\1`"), + (r"[IBCM]{(.+)}", r"`\1`"), + ('pyspark.rdd.RDD', 'RDD'), +) + +def _convert_epytext(line): + """ + >>> _convert_epytext("L{A}") + :class:`A` + """ + line = line.replace('@', ':') + for p, sub in RULES: + line = re.sub(p, sub, line) + return line + +def _process_docstring(app, what, name, obj, options, lines): + for i in range(len(lines)): + lines[i] = _convert_epytext(lines[i]) + +def setup(app): + app.connect("autodoc-process-docstring", _process_docstring) diff --git a/python/docs/index.rst b/python/docs/index.rst new file mode 100644 index 0000000000000..25b3f9bd93e63 --- /dev/null +++ b/python/docs/index.rst @@ -0,0 +1,37 @@ +.. pyspark documentation master file, created by + sphinx-quickstart on Thu Aug 28 15:17:47 2014. + You can adapt this file completely to your liking, but it should at least + contain the root `toctree` directive. + +Welcome to PySpark API reference! +=================================== + +Contents: + +.. toctree:: + :maxdepth: 2 + + pyspark + pyspark.sql + pyspark.mllib + + +Core classes: +--------------- + + :class:`pyspark.SparkContext` + + Main entry point for Spark functionality. + + :class:`pyspark.RDD` + + A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. + + +Indices and tables +================== + +* :ref:`genindex` +* :ref:`modindex` +* :ref:`search` + diff --git a/python/docs/make.bat b/python/docs/make.bat new file mode 100644 index 0000000000000..adad44fd7536a --- /dev/null +++ b/python/docs/make.bat @@ -0,0 +1,242 @@ +@ECHO OFF + +REM Command file for Sphinx documentation + +if "%SPHINXBUILD%" == "" ( + set SPHINXBUILD=sphinx-build +) +set BUILDDIR=_build +set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% . +set I18NSPHINXOPTS=%SPHINXOPTS% . +if NOT "%PAPER%" == "" ( + set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% + set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% +) + +if "%1" == "" goto help + +if "%1" == "help" ( + :help + echo.Please use `make ^` where ^ is one of + echo. html to make standalone HTML files + echo. dirhtml to make HTML files named index.html in directories + echo. singlehtml to make a single large HTML file + echo. pickle to make pickle files + echo. json to make JSON files + echo. htmlhelp to make HTML files and a HTML help project + echo. qthelp to make HTML files and a qthelp project + echo. devhelp to make HTML files and a Devhelp project + echo. epub to make an epub + echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter + echo. text to make text files + echo. man to make manual pages + echo. texinfo to make Texinfo files + echo. gettext to make PO message catalogs + echo. changes to make an overview over all changed/added/deprecated items + echo. xml to make Docutils-native XML files + echo. pseudoxml to make pseudoxml-XML files for display purposes + echo. linkcheck to check all external links for integrity + echo. doctest to run all doctests embedded in the documentation if enabled + goto end +) + +if "%1" == "clean" ( + for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i + del /q /s %BUILDDIR%\* + goto end +) + + +%SPHINXBUILD% 2> nul +if errorlevel 9009 ( + echo. + echo.The 'sphinx-build' command was not found. Make sure you have Sphinx + echo.installed, then set the SPHINXBUILD environment variable to point + echo.to the full path of the 'sphinx-build' executable. Alternatively you + echo.may add the Sphinx directory to PATH. + echo. + echo.If you don't have Sphinx installed, grab it from + echo.http://sphinx-doc.org/ + exit /b 1 +) + +if "%1" == "html" ( + %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/html. + goto end +) + +if "%1" == "dirhtml" ( + %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. + goto end +) + +if "%1" == "singlehtml" ( + %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. + goto end +) + +if "%1" == "pickle" ( + %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the pickle files. + goto end +) + +if "%1" == "json" ( + %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the JSON files. + goto end +) + +if "%1" == "htmlhelp" ( + %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run HTML Help Workshop with the ^ +.hhp project file in %BUILDDIR%/htmlhelp. + goto end +) + +if "%1" == "qthelp" ( + %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run "qcollectiongenerator" with the ^ +.qhcp project file in %BUILDDIR%/qthelp, like this: + echo.^> qcollectiongenerator %BUILDDIR%\qthelp\pyspark.qhcp + echo.To view the help file: + echo.^> assistant -collectionFile %BUILDDIR%\qthelp\pyspark.ghc + goto end +) + +if "%1" == "devhelp" ( + %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. + goto end +) + +if "%1" == "epub" ( + %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The epub file is in %BUILDDIR%/epub. + goto end +) + +if "%1" == "latex" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdf" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdfja" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf-ja + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "text" ( + %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The text files are in %BUILDDIR%/text. + goto end +) + +if "%1" == "man" ( + %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The manual pages are in %BUILDDIR%/man. + goto end +) + +if "%1" == "texinfo" ( + %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. + goto end +) + +if "%1" == "gettext" ( + %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The message catalogs are in %BUILDDIR%/locale. + goto end +) + +if "%1" == "changes" ( + %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes + if errorlevel 1 exit /b 1 + echo. + echo.The overview file is in %BUILDDIR%/changes. + goto end +) + +if "%1" == "linkcheck" ( + %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck + if errorlevel 1 exit /b 1 + echo. + echo.Link check complete; look for any errors in the above output ^ +or in %BUILDDIR%/linkcheck/output.txt. + goto end +) + +if "%1" == "doctest" ( + %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest + if errorlevel 1 exit /b 1 + echo. + echo.Testing of doctests in the sources finished, look at the ^ +results in %BUILDDIR%/doctest/output.txt. + goto end +) + +if "%1" == "xml" ( + %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The XML files are in %BUILDDIR%/xml. + goto end +) + +if "%1" == "pseudoxml" ( + %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. + goto end +) + +:end diff --git a/python/docs/modules.rst b/python/docs/modules.rst new file mode 100644 index 0000000000000..183564659fbcf --- /dev/null +++ b/python/docs/modules.rst @@ -0,0 +1,7 @@ +. += + +.. toctree:: + :maxdepth: 4 + + pyspark diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst new file mode 100644 index 0000000000000..e95d19e97f151 --- /dev/null +++ b/python/docs/pyspark.mllib.rst @@ -0,0 +1,77 @@ +pyspark.mllib package +===================== + +Submodules +---------- + +pyspark.mllib.classification module +----------------------------------- + +.. automodule:: pyspark.mllib.classification + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.clustering module +------------------------------- + +.. automodule:: pyspark.mllib.clustering + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.linalg module +--------------------------- + +.. automodule:: pyspark.mllib.linalg + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.random module +--------------------------- + +.. automodule:: pyspark.mllib.random + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.recommendation module +----------------------------------- + +.. automodule:: pyspark.mllib.recommendation + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.regression module +------------------------------- + +.. automodule:: pyspark.mllib.regression + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.stat module +------------------------- + +.. automodule:: pyspark.mllib.stat + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.tree module +------------------------- + +.. automodule:: pyspark.mllib.tree + :members: + :undoc-members: + :show-inheritance: + +pyspark.mllib.util module +------------------------- + +.. automodule:: pyspark.mllib.util + :members: + :undoc-members: + :show-inheritance: diff --git a/python/docs/pyspark.rst b/python/docs/pyspark.rst new file mode 100644 index 0000000000000..a68bd62433085 --- /dev/null +++ b/python/docs/pyspark.rst @@ -0,0 +1,18 @@ +pyspark package +=============== + +Subpackages +----------- + +.. toctree:: + :maxdepth: 1 + + pyspark.mllib + pyspark.sql + +Contents +-------- + +.. automodule:: pyspark + :members: + :undoc-members: diff --git a/python/docs/pyspark.sql.rst b/python/docs/pyspark.sql.rst new file mode 100644 index 0000000000000..65b3650ae10ab --- /dev/null +++ b/python/docs/pyspark.sql.rst @@ -0,0 +1,10 @@ +pyspark.sql module +================== + +Module contents +--------------- + +.. automodule:: pyspark.sql + :members: + :undoc-members: + :show-inheritance: diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 5c7c9cc161dff..f124dc6c07575 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -78,6 +78,9 @@ def value(self): return self._value def unpersist(self, blocking=False): + """ + Delete cached copies of this broadcast on the executors. + """ self._jbroadcast.unpersist(blocking) os.unlink(self.path) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a33aae87f65e8..a17f2c1203d36 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -53,7 +53,7 @@ class SparkContext(object): """ Main entry point for Spark functionality. A SparkContext represents the - connection to a Spark cluster, and can be used to create L{RDD}s and + connection to a Spark cluster, and can be used to create L{RDD} and broadcast variables on that cluster. """ diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index ec3c6f055441d..44ac5642836e0 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -110,6 +110,9 @@ def __eq__(self, other): def __ne__(self, other): return not self.__eq__(other) + def __repr__(self): + return "<%s object>" % self.__class__.__name__ + class FramedSerializer(Serializer): diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 621a556ec6356..8f6dbab240c7b 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -289,7 +289,7 @@ class StructType(DataType): """Spark SQL StructType The data type representing rows. - A StructType object comprises a list of L{StructField}s. + A StructType object comprises a list of L{StructField}. """ @@ -904,7 +904,7 @@ class SQLContext(object): """Main entry point for Spark SQL functionality. - A SQLContext can be used create L{SchemaRDD}s, register L{SchemaRDD}s as + A SQLContext can be used create L{SchemaRDD}, register L{SchemaRDD} as tables, execute SQL over tables, cache tables, and read parquet files. """ @@ -994,7 +994,7 @@ def registerFunction(self, name, f, returnType=StringType()): str(returnType)) def inferSchema(self, rdd): - """Infer and apply a schema to an RDD of L{Row}s. + """Infer and apply a schema to an RDD of L{Row}. We peek at the first row of the RDD to determine the fields' names and types. Nested collections are supported, which include array, @@ -1047,7 +1047,7 @@ def inferSchema(self, rdd): def applySchema(self, rdd, schema): """ - Applies the given schema to the given RDD of L{tuple} or L{list}s. + Applies the given schema to the given RDD of L{tuple} or L{list}. These tuples or lists can contain complex nested structures like lists, maps or nested rows. @@ -1183,6 +1183,7 @@ def jsonFile(self, path, schema=None): Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> srdd3 = sqlCtx.jsonFile(jsonFile, srdd1.schema()) >>> sqlCtx.registerRDDAsTable(srdd3, "table2") >>> srdd4 = sqlCtx.sql( @@ -1193,6 +1194,7 @@ def jsonFile(self, path, schema=None): Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) Row(f1=2, f2=None, f3=Row(field4=22,..., f4=[Row(field7=u'row2')]) Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> schema = StructType([ ... StructField("field2", StringType(), True), ... StructField("field3", @@ -1233,6 +1235,7 @@ def jsonRDD(self, rdd, schema=None): Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> srdd3 = sqlCtx.jsonRDD(json, srdd1.schema()) >>> sqlCtx.registerRDDAsTable(srdd3, "table2") >>> srdd4 = sqlCtx.sql( @@ -1243,6 +1246,7 @@ def jsonRDD(self, rdd, schema=None): Row(f1=1, f2=u'row1', f3=Row(field4=11, field5=None), f4=None) Row(f1=2, f2=None, f3=Row(field4=22..., f4=[Row(field7=u'row2')]) Row(f1=None, f2=u'row3', f3=Row(field4=33, field5=[]), f4=None) + >>> schema = StructType([ ... StructField("field2", StringType(), True), ... StructField("field3", From b20171267d610715d5b0a86b474c903e9bc3a1a3 Mon Sep 17 00:00:00 2001 From: Dan Osipov Date: Tue, 16 Sep 2014 13:40:16 -0700 Subject: [PATCH 0715/1492] [SPARK-787] Add S3 configuration parameters to the EC2 deploy scripts When deploying to AWS, there is additional configuration that is required to read S3 files. EMR creates it automatically, there is no reason that the Spark EC2 script shouldn't. This PR requires a corresponding PR to the mesos/spark-ec2 to be merged, as it gets cloned in the process of setting up machines: https://github.com/mesos/spark-ec2/pull/58 Author: Dan Osipov Closes #1120 from danosipov/s3_credentials and squashes the following commits: 758da8b [Dan Osipov] Modify documentation to include the new parameter 71fab14 [Dan Osipov] Use a parameter --copy-aws-credentials to enable S3 credential deployment 7e0da26 [Dan Osipov] Get AWS credentials out of boto connection instance 39bdf30 [Dan Osipov] Add S3 configuration parameters to the EC2 deploy scripts --- docs/ec2-scripts.md | 2 +- ec2/deploy.generic/root/spark-ec2/ec2-variables.sh | 2 ++ ec2/spark_ec2.py | 10 ++++++++++ 3 files changed, 13 insertions(+), 1 deletion(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index f5ac6d894e1eb..b2ca6a9b48f32 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -156,6 +156,6 @@ If you have a patch or suggestion for one of these limitations, feel free to # Accessing Data in S3 -Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n:///path`. You will also need to set your Amazon security credentials, either by setting the environment variables `AWS_ACCESS_KEY_ID` and `AWS_SECRET_ACCESS_KEY` before your program or through `SparkContext.hadoopConfiguration`. Full instructions on S3 access using the Hadoop input libraries can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3). +Spark's file interface allows it to process data in Amazon S3 using the same URI formats that are supported for Hadoop. You can specify a path in S3 as input through a URI of the form `s3n:///path`. To provide AWS credentials for S3 access, launch the Spark cluster with the option `--copy-aws-credentials`. Full instructions on S3 access using the Hadoop input libraries can be found on the [Hadoop S3 page](http://wiki.apache.org/hadoop/AmazonS3). In addition to using a single input file, you can also use a directory of files as input by simply giving the path to the directory. diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh index 3570891be804e..740c267fd9866 100644 --- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh +++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh @@ -30,3 +30,5 @@ export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}" export SWAP_MB="{{swap}}" export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}" export SPARK_MASTER_OPTS="{{spark_master_opts}}" +export AWS_ACCESS_KEY_ID="{{aws_access_key_id}}" +export AWS_SECRET_ACCESS_KEY="{{aws_secret_access_key}}" \ No newline at end of file diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 5682e96aa8770..abac71eaca595 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -158,6 +158,9 @@ def parse_args(): parser.add_option( "--additional-security-group", type="string", default="", help="Additional security group to place the machines in") + parser.add_option( + "--copy-aws-credentials", action="store_true", default=False, + help="Add AWS credentials to hadoop configuration to allow Spark to access S3") (opts, args) = parser.parse_args() if len(args) != 2: @@ -714,6 +717,13 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): "spark_master_opts": opts.master_opts } + if opts.copy_aws_credentials: + template_vars["aws_access_key_id"] = conn.aws_access_key_id + template_vars["aws_secret_access_key"] = conn.aws_secret_access_key + else: + template_vars["aws_access_key_id"] = "" + template_vars["aws_secret_access_key"] = "" + # Create a temp directory in which we will place all the files to be # deployed after we substitue template parameters in them tmp_dir = tempfile.mkdtemp() From a6e1712f1e9c36deb24c5073aa8edcfc047d76eb Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Tue, 16 Sep 2014 13:46:06 -0700 Subject: [PATCH 0716/1492] Add a Community Projects page This adds a new page to the docs listing community projects -- those created outside of Apache Spark that are of interest to the community of Spark users. Anybody can add to it just by submitting a PR. There was a discussion thread about alternatives: * Creating a Github organization for Spark projects - we could not find any sponsors for this, and it would be difficult to organize since many folks just create repos in their company organization or personal accounts * Apache has some place for storing community projects, but it was deemed difficult to work with, and again would be some permissions issues -- not everyone could update it. Author: Evan Chan Closes #2219 from velvia/community-projects-page and squashes the following commits: 7316822 [Evan Chan] Point to Spark wiki: supplemental projects page 613b021 [Evan Chan] Add a few more projects a85eaaf [Evan Chan] Add a Community Projects page --- docs/_layouts/global.html | 3 ++- docs/index.md | 1 + 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index a53e8a775b71f..627ed37de4a9c 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -111,6 +111,7 @@
  • Building Spark
  • Contributing to Spark
  • +
  • Supplemental Projects
  • @@ -151,7 +152,7 @@

    {{ page.title }}

    MathJax.Hub.Config({ tex2jax: { inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ], - displayMath: [ ["$$","$$"], ["\\[", "\\]"] ], + displayMath: [ ["$$","$$"], ["\\[", "\\]"] ], processEscapes: true, skipTags: ['script', 'noscript', 'style', 'textarea', 'pre'] } diff --git a/docs/index.md b/docs/index.md index e8ebadbd4e427..edd622ec90f64 100644 --- a/docs/index.md +++ b/docs/index.md @@ -107,6 +107,7 @@ options for deployment: * [OpenStack Swift](storage-openstack-swift.html) * [Building Spark](building-spark.html): build Spark using the Maven system * [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) +* [Supplemental Projects](https://cwiki.apache.org/confluence/display/SPARK/Supplemental+Spark+Projects): related third party Spark projects **External Resources:** From 0a7091e689a4c8b1e7b61e9f0873e6557f40d952 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 16 Sep 2014 16:03:20 -0700 Subject: [PATCH 0717/1492] [SPARK-3555] Fix UISuite race condition The test "jetty selects different port under contention" is flaky. If another process binds to 4040 before the test starts, then the first server we start there will fail, and the subsequent servers we start thereafter may successfully bind to 4040 if it was released between the servers starting. Instead, we should just let Java find a random free port for us and hold onto it for the duration of the test. Author: Andrew Or Closes #2418 from andrewor14/fix-port-contention and squashes the following commits: 0cd4974 [Andrew Or] Stop them servers a7071fe [Andrew Or] Pick random port instead of 4040 --- .../test/scala/org/apache/spark/ui/UISuite.scala | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 48790b59e7fbd..92a21f82f3c21 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -23,7 +23,6 @@ import javax.servlet.http.HttpServletRequest import scala.io.Source import scala.util.{Failure, Success, Try} -import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ @@ -108,14 +107,8 @@ class UISuite extends FunSuite { } test("jetty selects different port under contention") { - val startPort = 4040 - val server = new Server(startPort) - - Try { server.start() } match { - case Success(s) => - case Failure(e) => - // Either case server port is busy hence setup for test complete - } + val server = new ServerSocket(0) + val startPort = server.getLocalPort val serverInfo1 = JettyUtils.startJettyServer( "0.0.0.0", startPort, Seq[ServletContextHandler](), new SparkConf) val serverInfo2 = JettyUtils.startJettyServer( @@ -126,6 +119,9 @@ class UISuite extends FunSuite { assert(boundPort1 != startPort) assert(boundPort2 != startPort) assert(boundPort1 != boundPort2) + serverInfo1.server.stop() + serverInfo2.server.stop() + server.close() } test("jetty binds to port 0 correctly") { From 008a5ed4808d1467b47c1d6fa4d950cc6c4976b7 Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 16 Sep 2014 21:57:33 -0700 Subject: [PATCH 0718/1492] [Minor]ignore all config files in conf Some config files in ```conf``` should ignore, such as conf/fairscheduler.xml conf/hive-log4j.properties conf/metrics.properties ... So ignore all ```sh```/```properties```/```conf```/```xml``` files Author: wangfei Closes #2395 from scwf/patch-2 and squashes the following commits: 3dc53f2 [wangfei] duplicate ```conf/*.conf``` 3c2986f [wangfei] ignore all config files --- .gitignore | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/.gitignore b/.gitignore index 7ec8d45e12c6b..a31bf7e0091f4 100644 --- a/.gitignore +++ b/.gitignore @@ -15,11 +15,10 @@ out/ third_party/libmesos.so third_party/libmesos.dylib conf/java-opts -conf/spark-env.sh -conf/streaming-env.sh -conf/log4j.properties -conf/spark-defaults.conf -conf/hive-site.xml +conf/*.sh +conf/*.properties +conf/*.conf +conf/*.xml docs/_site docs/api target/ @@ -50,7 +49,6 @@ unit-tests.log /lib/ rat-results.txt scalastyle.txt -conf/*.conf scalastyle-output.xml # For Hive From 983609a4dd83e25598455bfce93fa1c1fa9f2c51 Mon Sep 17 00:00:00 2001 From: viper-kun Date: Wed, 17 Sep 2014 00:09:57 -0700 Subject: [PATCH 0719/1492] [Docs] Correct spark.files.fetchTimeout default value change the value of spark.files.fetchTimeout Author: viper-kun Closes #2406 from viper-kun/master and squashes the following commits: ecb0d46 [viper-kun] [Docs] Correct spark.files.fetchTimeout default value 7cf4c7a [viper-kun] Update configuration.md --- docs/configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index af16489a44281..99faf51c6f3db 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -520,10 +520,10 @@ Apart from these, the following properties are also available, and may be useful
    spark.files.fetchTimeoutfalse60 Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver. + the driver, in seconds.
    {info.name}{info.id}{info.name} {startTime} {endTime} {duration}spark.port.maxRetries 16 - Maximum number of retries when binding to a port before giving up. + Default maximum number of retries when binding to a port before giving up.
    join(otherDataset, [numTasks]) When called on datasets of type (K, V) and (K, W), returns a dataset of (K, (V, W)) pairs with all pairs of elements for each key. - Outer joins are also supported through leftOuterJoin and rightOuterJoin. + Outer joins are supported through leftOuterJoin, rightOuterJoin, and fullOuterJoin.
    spark.python.profilefalse + Enable profiling in Python worker, the profile result will show up by `sc.show_profiles()`, + or it will be displayed before the driver exiting. It also can be dumped into disk by + `sc.dump_profiles(path)`. If some of the profile results had been displayed maually, + they will not be displayed automatically before driver exiting. +
    spark.python.profile.dump(none) + The directory which is used to dump the profile result before driver exiting. + The results will be dumped as separated file for each RDD. They can be loaded + by ptats.Stats(). If this is specified, the profile result will not be displayed + automatically. +
    spark.python.worker.reuse true
    spark.python.profilefalse - Enable profiling in Python worker, the profile result will show up by `sc.show_profiles()`, - or it will be displayed before the driver exiting. It also can be dumped into disk by - `sc.dump_profiles(path)`. If some of the profile results had been displayed maually, - they will not be displayed automatically before driver exiting. -
    spark.python.profile.dump(none) - The directory which is used to dump the profile result before driver exiting. - The results will be dumped as separated file for each RDD. They can be loaded - by ptats.Stats(). If this is specified, the profile result will not be displayed - automatically. -
    spark.python.worker.reuse true
    groupByKey([numTasks]) When called on a dataset of (K, V) pairs, returns a dataset of (K, Iterable<V>) pairs.
    - Note: If you are grouping in order to perform an aggregation (such as a sum or + Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or combineByKey will yield much better performance.
    From a01a30927d107a8d9496f749eb9d89eda6dda9d7 Mon Sep 17 00:00:00 2001 From: shane knapp Date: Tue, 30 Sep 2014 13:11:25 -0700 Subject: [PATCH 0839/1492] SPARK-3745 - fix check-license to properly download and check jar for details, see: https://issues.apache.org/jira/browse/SPARK-3745 Author: shane knapp Closes #2596 from shaneknapp/SPARK-3745 and squashes the following commits: c95eea9 [shane knapp] SPARK-3745 - fix check-license to properly download and check jar --- dev/check-license | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/dev/check-license b/dev/check-license index 9ff0929e9a5e8..72b1013479964 100755 --- a/dev/check-license +++ b/dev/check-license @@ -20,11 +20,10 @@ acquire_rat_jar () { - URL1="http://search.maven.org/remotecontent?filepath=org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" - URL2="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" + URL="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" JAR="$rat_jar" - + if [[ ! -f "$rat_jar" ]]; then # Download rat launch jar if it hasn't been downloaded yet if [ ! -f "$JAR" ]; then @@ -32,15 +31,17 @@ acquire_rat_jar () { printf "Attempting to fetch rat\n" JAR_DL="${JAR}.part" if hash curl 2>/dev/null; then - (curl --silent "${URL1}" > "$JAR_DL" || curl --silent "${URL2}" > "$JAR_DL") && mv "$JAR_DL" "$JAR" + curl --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" elif hash wget 2>/dev/null; then - (wget --quiet ${URL1} -O "$JAR_DL" || wget --quiet ${URL2} -O "$JAR_DL") && mv "$JAR_DL" "$JAR" + wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" else printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 fi fi - if [ ! -f "$JAR" ]; then + + unzip -tq $JAR &> /dev/null + if [ $? -ne 0 ]; then # We failed to download printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" exit -1 @@ -55,7 +56,7 @@ cd "$FWDIR" if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" -else +else declare java_cmd=java fi From d3a3840e077802647aced1ceace1494605dda1db Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Tue, 30 Sep 2014 13:28:41 -0700 Subject: [PATCH 0840/1492] [Build] Post commit hash with timeout messages [By request](https://github.com/apache/spark/pull/2588#issuecomment-57266871), and because it also makes sense. Author: Nicholas Chammas Closes #2597 from nchammas/timeout-commit-hash and squashes the following commits: 3d90714 [Nicholas Chammas] Revert "testing: making timeout 1 minute" 2353c95 [Nicholas Chammas] testing: making timeout 1 minute e3a477e [Nicholas Chammas] post commit hash with timeout --- dev/run-tests-jenkins | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index a6ecf3196d7d4..0b1e31b9413cf 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -141,8 +141,10 @@ function post_message () { test_result="$?" if [ "$test_result" -eq "124" ]; then - fail_message="**[Tests timed out](${BUILD_URL}consoleFull)** after \ - a configured wait of \`${TESTS_TIMEOUT}\`." + fail_message="**[Tests timed out](${BUILD_URL}consoleFull)** \ + for PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL}) \ + after a configured wait of \`${TESTS_TIMEOUT}\`." + post_message "$fail_message" exit $test_result else From 8764fe368bbd72fe76ed318faad0e97a7279e2fe Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 30 Sep 2014 15:18:51 -0700 Subject: [PATCH 0841/1492] SPARK-3744 [STREAMING] FlumeStreamSuite will fail during port contention Since it looked quite easy, I took the liberty of making a quick PR that just uses `Utils.startServiceOnPort` to fix this. It works locally for me. Author: Sean Owen Closes #2601 from srowen/SPARK-3744 and squashes the following commits: ddc9319 [Sean Owen] Avoid port contention in tests by retrying several ports for Flume stream --- .../streaming/flume/FlumeStreamSuite.scala | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 6ee7ac974b4a0..33235d150b4a5 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -31,7 +31,7 @@ import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} import org.apache.spark.streaming.util.ManualClock -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream +import org.apache.spark.util.Utils import org.jboss.netty.channel.ChannelPipeline import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory @@ -41,21 +41,26 @@ import org.jboss.netty.handler.codec.compression._ class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { - runFlumeStreamTest(false, 9998) + runFlumeStreamTest(false) } test("flume input compressed stream") { - runFlumeStreamTest(true, 9997) + runFlumeStreamTest(true) } - def runFlumeStreamTest(enableDecompression: Boolean, testPort: Int) { + def runFlumeStreamTest(enableDecompression: Boolean) { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream: JavaReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, enableDecompression) + val (flumeStream, testPort) = + Utils.startServiceOnPort(9997, (trialPort: Int) => { + val dstream = FlumeUtils.createStream( + ssc, "localhost", trialPort, StorageLevel.MEMORY_AND_DISK, enableDecompression) + (dstream, trialPort) + }) + val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream.receiverInputDStream, outputBuffer) + val outputStream = new TestOutputStream(flumeStream, outputBuffer) outputStream.register() ssc.start() @@ -63,13 +68,13 @@ class FlumeStreamSuite extends TestSuiteBase { val input = Seq(1, 2, 3, 4, 5) Thread.sleep(1000) val transceiver = new NettyTransceiver(new InetSocketAddress("localhost", testPort)) - var client: AvroSourceProtocol = null; - + var client: AvroSourceProtocol = null + if (enableDecompression) { client = SpecificRequestor.getClient( classOf[AvroSourceProtocol], new NettyTransceiver(new InetSocketAddress("localhost", testPort), - new CompressionChannelFactory(6))); + new CompressionChannelFactory(6))) } else { client = SpecificRequestor.getClient( classOf[AvroSourceProtocol], transceiver) From 6c696d7da64e764111b680b1eee040a61f944c26 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 30 Sep 2014 15:55:04 -0700 Subject: [PATCH 0842/1492] Remove compiler warning from TaskContext change. Author: Reynold Xin Closes #2602 from rxin/warning and squashes the following commits: 130186b [Reynold Xin] Remove compiler warning from TaskContext change. --- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 929ded58a3bd5..0d97506450a7f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -956,9 +956,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeShard = (context: TaskContext, iter: Iterator[(K,V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt /* "reduce task" */ - val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, + val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.getPartitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) val format = outfmt.newInstance @@ -1027,9 +1027,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) val writeToFile = (context: TaskContext, iter: Iterator[(K, V)]) => { // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it // around by taking a mod. We expect that no task will be attempted 2 billion times. - val attemptNumber = (context.attemptId % Int.MaxValue).toInt + val attemptNumber = (context.getAttemptId % Int.MaxValue).toInt - writer.setup(context.stageId, context.partitionId, attemptNumber) + writer.setup(context.getStageId, context.getPartitionId, attemptNumber) writer.open() try { var count = 0 From d75496b1898dace4da1cf95e53c38093f8f95221 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 30 Sep 2014 17:10:36 -0700 Subject: [PATCH 0843/1492] [SPARK-3701][MLLIB] update python linalg api and small fixes 1. doc updates 2. simple checks on vector dimensions 3. use column major for matrices davies jkbradley Author: Xiangrui Meng Closes #2548 from mengxr/mllib-py-clean and squashes the following commits: 6dce2df [Xiangrui Meng] address comments 116b5db [Xiangrui Meng] use np.dot instead of array.dot 75f2fcc [Xiangrui Meng] fix python style fefce00 [Xiangrui Meng] better check of vector size with more tests 067ef71 [Xiangrui Meng] majored -> major ef853f9 [Xiangrui Meng] update python linalg api and small fixes --- .../apache/spark/mllib/linalg/Matrices.scala | 8 +- python/pyspark/mllib/linalg.py | 150 ++++++++++++++---- 2 files changed, 125 insertions(+), 33 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 4e87fe088ecc5..2cc52e94282ba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -85,7 +85,7 @@ sealed trait Matrix extends Serializable { } /** - * Column-majored dense matrix. + * Column-major dense matrix. * The entry values are stored in a single array of doubles with columns listed in sequence. * For example, the following matrix * {{{ @@ -128,7 +128,7 @@ class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) } /** - * Column-majored sparse matrix. + * Column-major sparse matrix. * The entry values are stored in Compressed Sparse Column (CSC) format. * For example, the following matrix * {{{ @@ -207,7 +207,7 @@ class SparseMatrix( object Matrices { /** - * Creates a column-majored dense matrix. + * Creates a column-major dense matrix. * * @param numRows number of rows * @param numCols number of columns @@ -218,7 +218,7 @@ object Matrices { } /** - * Creates a column-majored sparse matrix in Compressed Sparse Column (CSC) format. + * Creates a column-major sparse matrix in Compressed Sparse Column (CSC) format. * * @param numRows number of rows * @param numCols number of columns diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 0a5dcaac55e46..51014a8ceb785 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -63,6 +63,41 @@ def _convert_to_vector(l): raise TypeError("Cannot convert type %s into Vector" % type(l)) +def _vector_size(v): + """ + Returns the size of the vector. + + >>> _vector_size([1., 2., 3.]) + 3 + >>> _vector_size((1., 2., 3.)) + 3 + >>> _vector_size(array.array('d', [1., 2., 3.])) + 3 + >>> _vector_size(np.zeros(3)) + 3 + >>> _vector_size(np.zeros((3, 1))) + 3 + >>> _vector_size(np.zeros((1, 3))) + Traceback (most recent call last): + ... + ValueError: Cannot treat an ndarray of shape (1, 3) as a vector + """ + if isinstance(v, Vector): + return len(v) + elif type(v) in (array.array, list, tuple): + return len(v) + elif type(v) == np.ndarray: + if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): + return len(v) + else: + raise ValueError("Cannot treat an ndarray of shape %s as a vector" % str(v.shape)) + elif _have_scipy and scipy.sparse.issparse(v): + assert v.shape[1] == 1, "Expected column vector" + return v.shape[0] + else: + raise TypeError("Cannot treat type %s as a vector" % type(v)) + + class Vector(object): """ Abstract class for DenseVector and SparseVector @@ -76,6 +111,9 @@ def toArray(self): class DenseVector(Vector): + """ + A dense vector represented by a value array. + """ def __init__(self, ar): if not isinstance(ar, array.array): ar = array.array('d', ar) @@ -100,15 +138,31 @@ def dot(self, other): 5.0 >>> dense.dot(np.array(range(1, 3))) 5.0 + >>> dense.dot([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense.dot(np.reshape([1., 2., 3., 4.], (2, 2), order='F')) + array([ 5., 11.]) + >>> dense.dot(np.reshape([1., 2., 3.], (3, 1), order='F')) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch """ - if isinstance(other, SparseVector): - return other.dot(self) + if type(other) == np.ndarray and other.ndim > 1: + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.toArray(), other) elif _have_scipy and scipy.sparse.issparse(other): - return other.transpose().dot(self.toArray())[0] - elif isinstance(other, Vector): - return np.dot(self.toArray(), other.toArray()) + assert len(self) == other.shape[0], "dimension mismatch" + return other.transpose().dot(self.toArray()) else: - return np.dot(self.toArray(), other) + assert len(self) == _vector_size(other), "dimension mismatch" + if isinstance(other, SparseVector): + return other.dot(self) + elif isinstance(other, Vector): + return np.dot(self.toArray(), other.toArray()) + else: + return np.dot(self.toArray(), other) def squared_distance(self, other): """ @@ -126,7 +180,16 @@ def squared_distance(self, other): >>> sparse1 = SparseVector(2, [0, 1], [2., 1.]) >>> dense1.squared_distance(sparse1) 2.0 + >>> dense1.squared_distance([1.,]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> dense1.squared_distance(SparseVector(1, [0,], [1.,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch """ + assert len(self) == _vector_size(other), "dimension mismatch" if isinstance(other, SparseVector): return other.squared_distance(self) elif _have_scipy and scipy.sparse.issparse(other): @@ -165,12 +228,10 @@ def __getattr__(self, item): class SparseVector(Vector): - """ A simple sparse vector class for passing data to MLlib. Users may alternatively pass SciPy's {scipy.sparse} data types. """ - def __init__(self, size, *args): """ Create a sparse vector, using either a dictionary, a list of @@ -222,20 +283,33 @@ def dot(self, other): 0.0 >>> a.dot(np.array([[1, 1], [2, 2], [3, 3], [4, 4]])) array([ 22., 22.]) + >>> a.dot([1., 2., 3.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.array([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(DenseVector([1., 2.])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> a.dot(np.zeros((3, 2))) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch """ if type(other) == np.ndarray: - if other.ndim == 1: - result = 0.0 - for i in xrange(len(self.indices)): - result += self.values[i] * other[self.indices[i]] - return result - elif other.ndim == 2: + if other.ndim == 2: results = [self.dot(other[:, i]) for i in xrange(other.shape[1])] return np.array(results) - else: - raise Exception("Cannot call dot with %d-dimensional array" % other.ndim) + elif other.ndim > 2: + raise ValueError("Cannot call dot with %d-dimensional array" % other.ndim) + + assert len(self) == _vector_size(other), "dimension mismatch" - elif type(other) in (array.array, DenseVector): + if type(other) in (np.ndarray, array.array, DenseVector): result = 0.0 for i in xrange(len(self.indices)): result += self.values[i] * other[self.indices[i]] @@ -254,6 +328,7 @@ def dot(self, other): else: j += 1 return result + else: return self.dot(_convert_to_vector(other)) @@ -273,7 +348,16 @@ def squared_distance(self, other): 30.0 >>> b.squared_distance(a) 30.0 + >>> b.squared_distance([1., 2.]) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch + >>> b.squared_distance(SparseVector(3, [1,], [1.0,])) + Traceback (most recent call last): + ... + AssertionError: dimension mismatch """ + assert len(self) == _vector_size(other), "dimension mismatch" if type(other) in (list, array.array, DenseVector, np.array, np.ndarray): if type(other) is np.array and other.ndim != 1: raise Exception("Cannot call squared_distance with %d-dimensional array" % @@ -348,7 +432,6 @@ def __eq__(self, other): >>> v1 != v2 False """ - return (isinstance(other, self.__class__) and other.size == self.size and other.indices == self.indices @@ -414,23 +497,32 @@ def stringify(vector): class Matrix(object): - """ the Matrix """ - def __init__(self, nRow, nCol): - self.nRow = nRow - self.nCol = nCol + """ + Represents a local matrix. + """ + + def __init__(self, numRows, numCols): + self.numRows = numRows + self.numCols = numCols def toArray(self): + """ + Returns its elements in a NumPy ndarray. + """ raise NotImplementedError class DenseMatrix(Matrix): - def __init__(self, nRow, nCol, values): - Matrix.__init__(self, nRow, nCol) - assert len(values) == nRow * nCol + """ + Column-major dense matrix. + """ + def __init__(self, numRows, numCols, values): + Matrix.__init__(self, numRows, numCols) + assert len(values) == numRows * numCols self.values = values def __reduce__(self): - return DenseMatrix, (self.nRow, self.nCol, self.values) + return DenseMatrix, (self.numRows, self.numCols, self.values) def toArray(self): """ @@ -439,10 +531,10 @@ def toArray(self): >>> arr = array.array('d', [float(i) for i in range(4)]) >>> m = DenseMatrix(2, 2, arr) >>> m.toArray() - array([[ 0., 1.], - [ 2., 3.]]) + array([[ 0., 2.], + [ 1., 3.]]) """ - return np.ndarray((self.nRow, self.nCol), np.float64, buffer=self.values.tostring()) + return np.reshape(self.values, (self.numRows, self.numCols), order='F') def _test(): From c5414b681868a0a11cc5a94184116e66e8d3e9c0 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 30 Sep 2014 18:24:57 -0700 Subject: [PATCH 0844/1492] [SPARK-3478] [PySpark] Profile the Python tasks This patch add profiling support for PySpark, it will show the profiling results before the driver exits, here is one example: ``` ============================================================ Profile of RDD ============================================================ 5146507 function calls (5146487 primitive calls) in 71.094 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 5144576 68.331 0.000 68.331 0.000 statcounter.py:44(merge) 20 2.735 0.137 71.071 3.554 statcounter.py:33(__init__) 20 0.017 0.001 0.017 0.001 {cPickle.dumps} 1024 0.003 0.000 0.003 0.000 t.py:16() 20 0.001 0.000 0.001 0.000 {reduce} 21 0.001 0.000 0.001 0.000 {cPickle.loads} 20 0.001 0.000 0.001 0.000 copy_reg.py:95(_slotnames) 41 0.001 0.000 0.001 0.000 serializers.py:461(read_int) 40 0.001 0.000 0.002 0.000 serializers.py:179(_batched) 62 0.000 0.000 0.000 0.000 {method 'read' of 'file' objects} 20 0.000 0.000 71.072 3.554 rdd.py:863() 20 0.000 0.000 0.001 0.000 serializers.py:198(load_stream) 40/20 0.000 0.000 71.072 3.554 rdd.py:2093(pipeline_func) 41 0.000 0.000 0.002 0.000 serializers.py:130(load_stream) 40 0.000 0.000 71.072 1.777 rdd.py:304(func) 20 0.000 0.000 71.094 3.555 worker.py:82(process) ``` Also, use can show profile result manually by `sc.show_profiles()` or dump it into disk by `sc.dump_profiles(path)`, such as ```python >>> sc._conf.set("spark.python.profile", "true") >>> rdd = sc.parallelize(range(100)).map(str) >>> rdd.count() 100 >>> sc.show_profiles() ============================================================ Profile of RDD ============================================================ 284 function calls (276 primitive calls) in 0.001 seconds Ordered by: internal time, cumulative time ncalls tottime percall cumtime percall filename:lineno(function) 4 0.000 0.000 0.000 0.000 serializers.py:198(load_stream) 4 0.000 0.000 0.000 0.000 {reduce} 12/4 0.000 0.000 0.001 0.000 rdd.py:2092(pipeline_func) 4 0.000 0.000 0.000 0.000 {cPickle.loads} 4 0.000 0.000 0.000 0.000 {cPickle.dumps} 104 0.000 0.000 0.000 0.000 rdd.py:852() 8 0.000 0.000 0.000 0.000 serializers.py:461(read_int) 12 0.000 0.000 0.000 0.000 rdd.py:303(func) ``` The profiling is disabled by default, can be enabled by "spark.python.profile=true". Also, users can dump the results into disks automatically for future analysis, by "spark.python.profile.dump=path_to_dump" This is bugfix of #2351 cc JoshRosen Author: Davies Liu Closes #2556 from davies/profiler and squashes the following commits: e68df5a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 858e74c [Davies Liu] compatitable with python 2.6 7ef2aa0 [Davies Liu] bugfix, add tests for show_profiles and dump_profiles() 2b0daf2 [Davies Liu] fix docs 7a56c24 [Davies Liu] bugfix cba9463 [Davies Liu] move show_profiles and dump_profiles to SparkContext fb9565b [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 116d52a [Davies Liu] Merge branch 'master' of github.com:apache/spark into profiler 09d02c3 [Davies Liu] Merge branch 'master' into profiler c23865c [Davies Liu] Merge branch 'master' into profiler 15d6f18 [Davies Liu] add docs for two configs dadee1a [Davies Liu] add docs string and clear profiles after show or dump 4f8309d [Davies Liu] address comment, add tests 0a5b6eb [Davies Liu] fix Python UDF 4b20494 [Davies Liu] add profile for python --- docs/configuration.md | 19 +++++++++++++++++ python/pyspark/accumulators.py | 15 +++++++++++++ python/pyspark/context.py | 39 +++++++++++++++++++++++++++++++++- python/pyspark/rdd.py | 10 +++++++-- python/pyspark/sql.py | 2 +- python/pyspark/tests.py | 30 ++++++++++++++++++++++++++ python/pyspark/worker.py | 19 ++++++++++++++--- 7 files changed, 127 insertions(+), 7 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index a6dd7245e1552..791b6f2aa3261 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -206,6 +206,25 @@ Apart from these, the following properties are also available, and may be useful used during aggregation goes above this amount, it will spill the data into disks.
    spark.python.profilefalse + Enable profiling in Python worker, the profile result will show up by `sc.show_profiles()`, + or it will be displayed before the driver exiting. It also can be dumped into disk by + `sc.dump_profiles(path)`. If some of the profile results had been displayed maually, + they will not be displayed automatically before driver exiting. +
    spark.python.profile.dump(none) + The directory which is used to dump the profile result before driver exiting. + The results will be dumped as separated file for each RDD. They can be loaded + by ptats.Stats(). If this is specified, the profile result will not be displayed + automatically. +
    spark.python.worker.reuse true
    StructType org.apache.spark.sql.api.java org.apache.spark.sql.api.java.Row DataType.createStructType(fields)
    Note: fields is a List or an array of StructFields. From b4fb7b80a0d863500943d788ad3e34d502a6dafa Mon Sep 17 00:00:00 2001 From: Nishkam Ravi Date: Thu, 2 Oct 2014 13:48:35 -0500 Subject: [PATCH 0872/1492] Modify default YARN memory_overhead-- from an additive constant to a multiplier Redone against the recent master branch (https://github.com/apache/spark/pull/1391) Author: Nishkam Ravi Author: nravi Author: nishkamravi2 Closes #2485 from nishkamravi2/master_nravi and squashes the following commits: 636a9ff [nishkamravi2] Update YarnAllocator.scala 8f76c8b [Nishkam Ravi] Doc change for yarn memory overhead 35daa64 [Nishkam Ravi] Slight change in the doc for yarn memory overhead 5ac2ec1 [Nishkam Ravi] Remove out dac1047 [Nishkam Ravi] Additional documentation for yarn memory overhead issue 42c2c3d [Nishkam Ravi] Additional changes for yarn memory overhead issue 362da5e [Nishkam Ravi] Additional changes for yarn memory overhead c726bd9 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi f00fa31 [Nishkam Ravi] Improving logging for AM memoryOverhead 1cf2d1e [nishkamravi2] Update YarnAllocator.scala ebcde10 [Nishkam Ravi] Modify default YARN memory_overhead-- from an additive constant to a multiplier (redone to resolve merge conflicts) 2e69f11 [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark into master_nravi efd688a [Nishkam Ravi] Merge branch 'master' of https://github.com/apache/spark 2b630f9 [nravi] Accept memory input as "30g", "512M" instead of an int value, to be consistent with rest of Spark 3bf8fad [nravi] Merge branch 'master' of https://github.com/apache/spark 5423a03 [nravi] Merge branch 'master' of https://github.com/apache/spark eb663ca [nravi] Merge branch 'master' of https://github.com/apache/spark df2aeb1 [nravi] Improved fix for ConcurrentModificationIssue (Spark-1097, Hadoop-10456) 6b840f0 [nravi] Undo the fix for SPARK-1758 (the problem is fixed) 5108700 [nravi] Fix in Spark for the Concurrent thread modification issue (SPARK-1097, HADOOP-10456) 681b36f [nravi] Fix for SPARK-1758: failing test org.apache.spark.JavaAPISuite.wholeTextFiles --- docs/running-on-yarn.md | 8 ++++---- .../spark/deploy/yarn/ClientArguments.scala | 16 +++++++++------- .../apache/spark/deploy/yarn/ClientBase.scala | 12 ++++++++---- .../apache/spark/deploy/yarn/YarnAllocator.scala | 16 ++++++++-------- .../spark/deploy/yarn/YarnSparkHadoopUtil.scala | 8 ++++++-- 5 files changed, 35 insertions(+), 25 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 4b3a49eca7007..695813a2ba881 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -79,16 +79,16 @@ Most of the configs are the same for Spark on YARN as for other deployment modes
    spark.yarn.executor.memoryOverhead384executorMemory * 0.07, with minimum of 384 - The amount of off heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. + The amount of off heap memory (in megabytes) to be allocated per executor. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the executor size (typically 6-10%).
    spark.yarn.driver.memoryOverhead384driverMemory * 0.07, with minimum of 384 - The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. + The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%).
    spark.io.compression.codec snappy - The codec used to compress internal data such as RDD partitions and shuffle outputs. By default, - Spark provides three codecs: lz4, lzf, and snappy. You - can also use fully qualified class names to specify the codec, e.g. - org.apache.spark.io.LZ4CompressionCodec, + The codec used to compress internal data such as RDD partitions, broadcast variables and + shuffle outputs. By default, Spark provides three codecs: lz4, lzf, + and snappy. You can also use fully qualified class names to specify the codec, + e.g. + org.apache.spark.io.LZ4CompressionCodec, org.apache.spark.io.LZFCompressionCodec, and org.apache.spark.io.SnappyCompressionCodec.
    spark.mesos.executor.memoryOverheadexecutor memory * 0.07, with minimum of 384 + This value is an additive for spark.executor.memory, specified in MiB, + which is used to calculate the total Mesos task memory. A value of 384 + implies a 384MiB overhead. Additionally, there is a hard-coded 7% minimum + overhead. The final overhead will be the larger of either + `spark.mesos.executor.memoryOverhead` or 7% of `spark.executor.memory`. +
    #### Shuffle Behavior From 358d7ffd01b4a3fbae313890522cf662c71af6e5 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Fri, 3 Oct 2014 13:09:48 -0700 Subject: [PATCH 0888/1492] [SPARK-3775] Not suitable error message in spark-shell.cmd Modified some sentence of error message in bin\*.cmd. Author: Masayoshi TSUZUKI Closes #2640 from tsudukim/feature/SPARK-3775 and squashes the following commits: 3458afb [Masayoshi TSUZUKI] [SPARK-3775] Not suitable error message in spark-shell.cmd --- bin/pyspark2.cmd | 2 +- bin/run-example2.cmd | 2 +- bin/spark-class | 2 +- bin/spark-class2.cmd | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 2c4b08af8d4c3..a0e66abcc26c9 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -33,7 +33,7 @@ for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop* ) if [%FOUND_JAR%] == [0] ( echo Failed to find Spark assembly JAR. - echo You need to build Spark with sbt\sbt assembly before running this program. + echo You need to build Spark before running this program. goto exit ) :skip_build_test diff --git a/bin/run-example2.cmd b/bin/run-example2.cmd index b29bf90c64e90..b49d0dcb4ff2d 100644 --- a/bin/run-example2.cmd +++ b/bin/run-example2.cmd @@ -52,7 +52,7 @@ if exist "%FWDIR%RELEASE" ( ) if "x%SPARK_EXAMPLES_JAR%"=="x" ( echo Failed to find Spark examples assembly JAR. - echo You need to build Spark with sbt\sbt assembly before running this program. + echo You need to build Spark before running this program. goto exit ) diff --git a/bin/spark-class b/bin/spark-class index 613dc9c4566f2..e8201c18d52de 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -146,7 +146,7 @@ fi if [[ "$1" =~ org.apache.spark.tools.* ]]; then if test -z "$SPARK_TOOLS_JAR"; then echo "Failed to find Spark Tools Jar in $FWDIR/tools/target/scala-$SCALA_VERSION/" 1>&2 - echo "You need to build spark before running $1." 1>&2 + echo "You need to build Spark before running $1." 1>&2 exit 1 fi CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 6c5672819172b..da46543647efd 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -104,7 +104,7 @@ for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop* ) if "%FOUND_JAR%"=="0" ( echo Failed to find Spark assembly JAR. - echo You need to build Spark with sbt\sbt assembly before running this program. + echo You need to build Spark before running this program. goto exit ) :skip_build_test From e5566e05b1ac99aa6caf1701e47ebcdb68a002c6 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Fri, 3 Oct 2014 13:12:37 -0700 Subject: [PATCH 0889/1492] [SPARK-3774] typo comment in bin/utils.sh Modified the comment of bin/utils.sh. Author: Masayoshi TSUZUKI Closes #2639 from tsudukim/feature/SPARK-3774 and squashes the following commits: 707b779 [Masayoshi TSUZUKI] [SPARK-3774] typo comment in bin/utils.sh --- bin/utils.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/utils.sh b/bin/utils.sh index 0804b1ed9f231..22ea2b9a6d586 100755 --- a/bin/utils.sh +++ b/bin/utils.sh @@ -17,7 +17,7 @@ # limitations under the License. # -# Gather all all spark-submit options into SUBMISSION_OPTS +# Gather all spark-submit options into SUBMISSION_OPTS function gatherSparkSubmitOpts() { if [ -z "$SUBMIT_USAGE_FUNCTION" ]; then From 30abef154768e5c4c6062f3341933dbda990f6cc Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 3 Oct 2014 13:18:35 -0700 Subject: [PATCH 0890/1492] [SPARK-3606] [yarn] Correctly configure AmIpFilter for Yarn HA. The existing code only considered one of the RMs when running in Yarn HA mode, so it was possible to get errors if the active RM was not registered in the filter. The change makes use of a new API added to Yarn that returns all proxy addresses, and falls back to the old behavior if the API is not present. While there, I also made a change to look for the scheme (http or https) being used by Yarn when building the proxy URIs. Since, in the case of multiple RMs, Yarn uses commas as a separator, it was not possible anymore to use spark.filter.params to propagate this information (which used commas to delimit different config params). Instead, I added a new param (spark.filter.jsonParams) which expects a JSON string containing a map with the config data. I chose not to add it to the documentation at this point since I don't believe users will use it directly. Author: Marcelo Vanzin Closes #2469 from vanzin/SPARK-3606 and squashes the following commits: aeb458a [Marcelo Vanzin] Undelete needed import. 65e400d [Marcelo Vanzin] Remove unused import. d121883 [Marcelo Vanzin] Use separate config for each param instead of json. 04bc156 [Marcelo Vanzin] Review feedback. 4d4d6b9 [Marcelo Vanzin] [SPARK-3606] [yarn] Correctly configure AmIpFilter for Yarn HA. --- .../cluster/CoarseGrainedClusterMessage.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 8 ++++-- .../org/apache/spark/ui/JettyUtils.scala | 14 ++++++---- .../spark/deploy/yarn/YarnRMClientImpl.scala | 8 ++++-- .../spark/deploy/yarn/ApplicationMaster.scala | 12 +++----- .../spark/deploy/yarn/YarnRMClient.scala | 4 +-- .../spark/deploy/yarn/YarnRMClientImpl.scala | 28 ++++++++++++++++++- 7 files changed, 53 insertions(+), 23 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 6abf6d930c155..fb8160abc59db 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,7 +66,7 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage - case class AddWebUIFilter(filterName:String, filterParams: String, proxyBase :String) + case class AddWebUIFilter(filterName:String, filterParams: Map[String, 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 89089e7d6f8a8..59aed6b72fe42 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 @@ -275,15 +275,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } // Add filters to the SparkUI - def addWebUIFilter(filterName: String, filterParams: String, proxyBase: String) { + def addWebUIFilter(filterName: String, filterParams: Map[String, String], proxyBase: String) { if (proxyBase != null && proxyBase.nonEmpty) { System.setProperty("spark.ui.proxyBase", proxyBase) } - if (Seq(filterName, filterParams).forall(t => t != null && t.nonEmpty)) { + val hasFilter = (filterName != null && filterName.nonEmpty && + filterParams != null && filterParams.nonEmpty) + if (hasFilter) { logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") conf.set("spark.ui.filters", filterName) - conf.set(s"spark.$filterName.params", filterParams) + filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 6b4689291097f..2a27d49d2de05 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -21,9 +21,7 @@ import java.net.{InetSocketAddress, URL} import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} -import scala.annotation.tailrec import scala.language.implicitConversions -import scala.util.{Failure, Success, Try} import scala.xml.Node import org.eclipse.jetty.server.Server @@ -147,15 +145,19 @@ private[spark] object JettyUtils extends Logging { val holder : FilterHolder = new FilterHolder() holder.setClassName(filter) // Get any parameters for each filter - val paramName = "spark." + filter + ".params" - val params = conf.get(paramName, "").split(',').map(_.trim()).toSet - params.foreach { - case param : String => + conf.get("spark." + filter + ".params", "").split(',').map(_.trim()).toSet.foreach { + param: String => if (!param.isEmpty) { val parts = param.split("=") if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) } } + + val prefix = s"spark.$filter.param." + conf.getAll + .filter { case (k, v) => k.length() > prefix.length() && k.startsWith(prefix) } + .foreach { case (k, v) => holder.setInitParameter(k.substring(prefix.length()), v) } + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index acf26505e4cf9..9bd1719cb1808 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -76,8 +76,12 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC resourceManager.finishApplicationMaster(finishReq) } - override def getProxyHostAndPort(conf: YarnConfiguration) = - YarnConfiguration.getProxyHostAndPort(conf) + override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { + val proxy = YarnConfiguration.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val uriBase = "http://" + proxy + proxyBase + Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) + } override def getMaxRegAttempts(conf: YarnConfiguration) = conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index b51daeb437516..caceef5d4b5b0 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -368,18 +368,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, /** Add the Yarn IP filter that is required for properly securing the UI. */ private def addAmIpFilter() = { - val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" - val proxy = client.getProxyHostAndPort(yarnConf) - val parts = proxy.split(":") val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) - val uriBase = "http://" + proxy + proxyBase - val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase - + val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + val params = client.getAmIpFilterParams(yarnConf, proxyBase) if (isDriver) { System.setProperty("spark.ui.filters", amFilter) - System.setProperty(s"spark.$amFilter.params", params) + params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } else { - actor ! AddWebUIFilter(amFilter, params, proxyBase) + actor ! AddWebUIFilter(amFilter, params.toMap, proxyBase) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index ed65e56b3e413..943dc56202a37 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -59,8 +59,8 @@ trait YarnRMClient { /** Returns the attempt ID. */ def getAttemptId(): ApplicationAttemptId - /** Returns the RM's proxy host and port. */ - def getProxyHostAndPort(conf: YarnConfiguration): String + /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ + def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] /** Returns the maximum number of attempts to register the AM. */ def getMaxRegAttempts(conf: YarnConfiguration): Int diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index 54bc6b14c44ce..b581790e158ac 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -17,8 +17,13 @@ package org.apache.spark.deploy.yarn +import java.util.{List => JList} + import scala.collection.{Map, Set} +import scala.collection.JavaConversions._ +import scala.util._ +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ @@ -69,7 +74,28 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC appAttemptId } - override def getProxyHostAndPort(conf: YarnConfiguration) = WebAppUtils.getProxyHostAndPort(conf) + override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { + // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, + // so not all stable releases have it. + val prefix = Try(classOf[WebAppUtils].getMethod("getHttpSchemePrefix", classOf[Configuration]) + .invoke(null, conf).asInstanceOf[String]).getOrElse("http://") + + // If running a new enough Yarn, use the HA-aware API for retrieving the RM addresses. + try { + val method = classOf[WebAppUtils].getMethod("getProxyHostsAndPortsForAmFilter", + classOf[Configuration]) + val proxies = method.invoke(null, conf).asInstanceOf[JList[String]] + val hosts = proxies.map { proxy => proxy.split(":")(0) } + val uriBases = proxies.map { proxy => prefix + proxy + proxyBase } + Map("PROXY_HOSTS" -> hosts.mkString(","), "PROXY_URI_BASES" -> uriBases.mkString(",")) + } catch { + case e: NoSuchMethodException => + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts = proxy.split(":") + val uriBase = prefix + proxy + proxyBase + Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) + } + } override def getMaxRegAttempts(conf: YarnConfiguration) = conf.getInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS) From 1eb8389cb4ad40a405149b16e2719e12367d667a Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 3 Oct 2014 13:26:30 -0700 Subject: [PATCH 0891/1492] [SPARK-3763] The example of building with sbt should be "sbt assembly" instead of "sbt compile" In building-spark.md, there are some examples for making assembled package with maven but the example for building with sbt is only about for compiling. Author: Kousuke Saruta Closes #2627 from sarutak/SPARK-3763 and squashes the following commits: fadb990 [Kousuke Saruta] Modified the example to build with sbt in building-spark.md --- docs/building-spark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 2378092d4a1a8..901c157162fee 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -169,7 +169,7 @@ compilation. More advanced developers may wish to use SBT. The SBT build is derived from the Maven POM files, and so the same Maven profiles and variables can be set to control the SBT build. For example: - sbt/sbt -Pyarn -Phadoop-2.3 compile + sbt/sbt -Pyarn -Phadoop-2.3 assembly # Speeding up Compilation with Zinc From 79e45c9323455a51f25ed9acd0edd8682b4bbb88 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 3 Oct 2014 13:48:56 -0700 Subject: [PATCH 0892/1492] [SPARK-3377] [SPARK-3610] Metrics can be accidentally aggregated / History server log name should not be based on user input This PR is another solution for #2250 I'm using codahale base MetricsSystem of Spark with JMX or Graphite, and I saw following 2 problems. (1) When applications which have same spark.app.name run on cluster at the same time, some metrics names are mixed. For instance, if 2+ application is running on the cluster at the same time, each application emits the same named metric like "SparkPi.DAGScheduler.stage.failedStages" and Graphite cannot distinguish the metrics is for which application. (2) When 2+ executors run on the same machine, JVM metrics of each executors are mixed. For instance, 2+ executors running on the same node can emit the same named metric "jvm.memory" and Graphite cannot distinguish the metrics is from which application. And there is an similar issue. The directory for event logs is named using application name. Application name is defined by user and the name can includes illegal character for path names. Further more, the directory name consists of application name and System.currentTimeMillis even though each application has unique Application ID so if we run jobs which have same name, it's difficult to identify which directory is for which application. Closes #2250 Closes #1067 Author: Kousuke Saruta Closes #2432 from sarutak/metrics-structure-improvement2 and squashes the following commits: 3288b2b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 39169e4 [Kousuke Saruta] Fixed style 6570494 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 817e4f0 [Kousuke Saruta] Simplified MetricsSystem#buildRegistryName 67fa5eb [Kousuke Saruta] Unified MetricsSystem#registerSources and registerSinks in start 10be654 [Kousuke Saruta] Fixed style. 990c078 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 f0c7fba [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 59cc2cd [Kousuke Saruta] Modified SparkContextSchedulerCreationSuite f9b6fb3 [Kousuke Saruta] Modified style. 2cf8a0f [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 389090d [Kousuke Saruta] Replaced taskScheduler.applicationId() with getApplicationId in SparkContext#postApplicationStart ff45c89 [Kousuke Saruta] Added some test cases to MetricsSystemSuite 69c46a6 [Kousuke Saruta] Added warning logging logic to MetricsSystem#buildRegistryName 5cca0d2 [Kousuke Saruta] Added Javadoc comment to SparkContext#getApplicationId 16a9f01 [Kousuke Saruta] Added data types to be returned to some methods 6434b06 [Kousuke Saruta] Reverted changes related to ApplicationId 0413b90 [Kousuke Saruta] Deleted ApplicationId.java and ApplicationIdSuite.java a42300c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 0fc1b09 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 42bea55 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 248935d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 f6af132 [Kousuke Saruta] Modified SchedulerBackend and TaskScheduler to return System.currentTimeMillis as an unique Application Id 1b8b53e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 97cb85c [Kousuke Saruta] Modified confliction of MimExcludes 2cdd009 [Kousuke Saruta] Modified defailt implementation of applicationId 9aadb0b [Kousuke Saruta] Modified NetworkReceiverSuite to ensure "executor.start()" is finished in test "network receiver life cycle" 3011efc [Kousuke Saruta] Added ApplicationIdSuite.scala d009c55 [Kousuke Saruta] Modified ApplicationId#equals to compare appIds dfc83fd [Kousuke Saruta] Modified ApplicationId to implement Serializable 9ff4851 [Kousuke Saruta] Modified MimaExcludes.scala to ignore createTaskScheduler method in SparkContext 4567ffc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 6a91b14 [Kousuke Saruta] Modified SparkContextSchedulerCreationSuite, ExecutorRunnerTest and EventLoggingListenerSuite 0325caf [Kousuke Saruta] Added ApplicationId.scala 0a2fc14 [Kousuke Saruta] Modified style eabda80 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 0f890e6 [Kousuke Saruta] Modified SparkDeploySchedulerBackend and Master to pass baseLogDir instead f eventLogDir bcf25bf [Kousuke Saruta] Modified directory name for EventLogs 28d4d93 [Kousuke Saruta] Modified SparkContext and EventLoggingListener so that the directory for EventLogs is named same for Application ID 203634e [Kousuke Saruta] Modified comment in SchedulerBackend#applicationId and TaskScheduler#applicationId 424fea4 [Kousuke Saruta] Modified the subclasses of TaskScheduler and SchedulerBackend so that they can return non-optional Unique Application ID b311806 [Kousuke Saruta] Swapped last 2 arguments passed to CoarseGrainedExecutorBackend 8a2b6ec [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 086ee25 [Kousuke Saruta] Merge branch 'metrics-structure-improvement2' of github.com:sarutak/spark into metrics-structure-improvement2 e705386 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 36d2f7a [Kousuke Saruta] Added warning message for the situation we cannot get application id for the prefix for the name of metrics eea6e19 [Kousuke Saruta] Modified CoarseGrainedMesosSchedulerBackend and MesosSchedulerBackend so that we can get Application ID c229fbe [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 e719c39 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 4a93c7f [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement2 4776f9e [Kousuke Saruta] Modified MetricsSystemSuite.scala efcb6e1 [Kousuke Saruta] Modified to add application id to metrics name 2ec848a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 3ea7896 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement ead8966 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 08e627e [Kousuke Saruta] Revert "tmp" 7b67f5a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 45bd33d [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 93e263a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 848819c [Kousuke Saruta] Merge branch 'metrics-structure-improvement' of github.com:sarutak/spark into metrics-structure-improvement 912a637 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement e4a4593 [Kousuke Saruta] tmp 3e098d8 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 4603a39 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement fa7175b [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 15f88a3 [Kousuke Saruta] Modified MetricsSystem#buildRegistryName because conf.get does not return null when correspondin entry is absent 6f7dcd4 [Kousuke Saruta] Modified constructor of DAGSchedulerSource and BlockManagerSource because the instance of SparkContext is no longer used 6fc5560 [Kousuke Saruta] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource 4e057c9 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into metrics-structure-improvement 85ffc02 [Kousuke Saruta] Revert "Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource" 868e326 [Kousuke Saruta] Modified MetricsSystem to set registry name with unique application-id and driver/executor-id 71609f5 [Kousuke Saruta] Modified sourceName of ExecutorSource, DAGSchedulerSource and BlockManagerSource 55debab [Kousuke Saruta] Modified SparkContext and Executor to set spark.executor.id to identifiers 4180993 [Kousuke Saruta] Modified SparkContext to retain spark.unique.app.name property in SparkConf --- .../scala/org/apache/spark/SparkContext.scala | 52 ++++--- .../scala/org/apache/spark/SparkEnv.scala | 8 +- .../apache/spark/deploy/master/Master.scala | 12 +- .../CoarseGrainedExecutorBackend.scala | 16 ++- .../org/apache/spark/executor/Executor.scala | 1 + .../spark/executor/ExecutorSource.scala | 3 +- .../spark/executor/MesosExecutorBackend.scala | 3 +- .../apache/spark/metrics/MetricsSystem.scala | 40 +++++- .../spark/scheduler/DAGSchedulerSource.scala | 4 +- .../scheduler/EventLoggingListener.scala | 33 +++-- .../spark/scheduler/SchedulerBackend.scala | 8 +- .../spark/scheduler/TaskScheduler.scala | 8 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../cluster/SparkDeploySchedulerBackend.scala | 9 +- .../mesos/CoarseMesosSchedulerBackend.scala | 11 +- .../cluster/mesos/MesosSchedulerBackend.scala | 13 +- .../spark/scheduler/local/LocalBackend.scala | 3 + .../spark/storage/BlockManagerSource.scala | 4 +- .../spark/metrics/MetricsSystemSuite.scala | 128 +++++++++++++++++- .../scheduler/EventLoggingListenerSuite.scala | 14 +- .../spark/scheduler/ReplayListenerSuite.scala | 3 +- .../streaming/NetworkReceiverSuite.scala | 14 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 3 +- .../deploy/yarn/ExecutorRunnableUtil.scala | 2 + .../spark/deploy/yarn/YarnAllocator.scala | 2 + .../cluster/YarnClientSchedulerBackend.scala | 6 +- .../cluster/YarnClusterSchedulerBackend.scala | 9 +- .../spark/deploy/yarn/ExecutorRunnable.scala | 3 +- .../deploy/yarn/YarnAllocationHandler.scala | 2 +- 29 files changed, 331 insertions(+), 85 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 979d178c35969..97109b9f41b60 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -187,6 +187,15 @@ class SparkContext(config: SparkConf) extends Logging { val master = conf.get("spark.master") val appName = conf.get("spark.app.name") + private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) + private[spark] val eventLogDir: Option[String] = { + if (isEventLogEnabled) { + Some(conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR).stripSuffix("/")) + } else { + None + } + } + // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() @@ -200,6 +209,7 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val listenerBus = new LiveListenerBus // Create the Spark execution environment (cache, map output tracker, etc) + conf.set("spark.executor.id", "driver") private[spark] val env = SparkEnv.create( conf, "", @@ -232,19 +242,6 @@ class SparkContext(config: SparkConf) extends Logging { /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) - // Optionally log Spark events - private[spark] val eventLogger: Option[EventLoggingListener] = { - if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf, hadoopConfiguration) - logger.start() - listenerBus.addListener(logger) - Some(logger) - } else None - } - - // At this point, all relevant SparkListeners have been registered, so begin releasing events - listenerBus.start() - val startTime = System.currentTimeMillis() // Add each JAR given through the constructor @@ -309,6 +306,29 @@ class SparkContext(config: SparkConf) extends Logging { // constructor taskScheduler.start() + val applicationId: String = taskScheduler.applicationId() + conf.set("spark.app.id", applicationId) + + val metricsSystem = env.metricsSystem + + // The metrics system for Driver need to be set spark.app.id to app ID. + // So it should start after we get app ID from the task scheduler and set spark.app.id. + metricsSystem.start() + + // Optionally log Spark events + private[spark] val eventLogger: Option[EventLoggingListener] = { + if (isEventLogEnabled) { + val logger = + new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) + logger.start() + listenerBus.addListener(logger) + Some(logger) + } else None + } + + // At this point, all relevant SparkListeners have been registered, so begin releasing events + listenerBus.start() + private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { Some(new ContextCleaner(this)) @@ -411,8 +431,8 @@ class SparkContext(config: SparkConf) extends Logging { // Post init taskScheduler.postStartHook() - private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this) - private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this) + private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) + private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) private def initDriverMetrics() { SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) @@ -1278,7 +1298,7 @@ class SparkContext(config: SparkConf) extends Logging { private def postApplicationStart() { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). - listenerBus.post(SparkListenerApplicationStart(appName, taskScheduler.applicationId(), + listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), startTime, sparkUser)) } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 009ed64775844..72cac42cd2b2b 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -259,11 +259,15 @@ object SparkEnv extends Logging { } val metricsSystem = if (isDriver) { + // Don't start metrics system right now for Driver. + // We need to wait for the task scheduler to give us an app ID. + // Then we can start the metrics system. MetricsSystem.createMetricsSystem("driver", conf, securityManager) } else { - MetricsSystem.createMetricsSystem("executor", conf, securityManager) + val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) + ms.start() + ms } - metricsSystem.start() // Set the sparkFiles directory, used when downloading dependencies. In local mode, // this is a temporary directory; in distributed mode, this is the executor's current working 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 432b552c58cd8..f98b531316a3d 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 @@ -33,8 +33,8 @@ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState, - SparkHadoopUtil} +import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, + ExecutorState, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.master.DriverState.DriverState @@ -693,16 +693,18 @@ private[spark] class Master( app.desc.appUiUrl = notFoundBasePath return false } - val fileSystem = Utils.getHadoopFileSystem(eventLogDir, + + val appEventLogDir = EventLoggingListener.getLogDirPath(eventLogDir, app.id) + val fileSystem = Utils.getHadoopFileSystem(appEventLogDir, SparkHadoopUtil.get.newConfiguration(conf)) - val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(appEventLogDir, fileSystem) val eventLogPaths = eventLogInfo.logPaths val compressionCodec = eventLogInfo.compressionCodec if (eventLogPaths.isEmpty) { // Event logging is enabled for this application, but no event logs are found val title = s"Application history not found (${app.id})" - var msg = s"No event logs found for application $appName in $eventLogDir." + var msg = s"No event logs found for application $appName in $appEventLogDir." logWarning(msg) msg += " Did you specify the correct logging directory?" msg = URLEncoder.encode(msg, "UTF-8") diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 13af5b6f5812d..06061edfc0844 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -106,6 +106,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { executorId: String, hostname: String, cores: Int, + appId: String, workerUrl: Option[String]) { SignalLogger.register(log) @@ -122,7 +123,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val driver = fetcher.actorSelection(driverUrl) val timeout = AkkaUtils.askTimeout(executorConf) val fut = Patterns.ask(driver, RetrieveSparkProps, timeout) - val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] + val props = Await.result(fut, timeout).asInstanceOf[Seq[(String, String)]] ++ + Seq[(String, String)](("spark.app.id", appId)) fetcher.shutdown() // Create a new ActorSystem using driver's Spark properties to run the backend. @@ -144,16 +146,16 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { def main(args: Array[String]) { args.length match { - case x if x < 4 => + case x if x < 5 => System.err.println( // Worker url is used in spark standalone mode to enforce fate-sharing with worker "Usage: CoarseGrainedExecutorBackend " + - " []") + " [] ") System.exit(1) - case 4 => - run(args(0), args(1), args(2), args(3).toInt, None) - case x if x > 4 => - run(args(0), args(1), args(2), args(3).toInt, Some(args(4))) + case 5 => + run(args(0), args(1), args(2), args(3).toInt, args(4), None) + case x if x > 5 => + run(args(0), args(1), args(2), args(3).toInt, args(4), Some(args(5))) } } } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index d7211ae465902..9bbfcdc4a0b6e 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -74,6 +74,7 @@ private[spark] class Executor( val executorSource = new ExecutorSource(this, executorId) // Initialize Spark environment (using system properties read above) + conf.set("spark.executor.id", "executor." + executorId) private val env = { if (!isLocal) { val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index d6721586566c2..c4d73622c4727 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -37,8 +37,7 @@ private[spark] class ExecutorSource(val executor: Executor, executorId: String) override val metricRegistry = new MetricRegistry() - // TODO: It would be nice to pass the application name here - override val sourceName = "executor.%s".format(executorId) + override val sourceName = "executor" // Gauge for executor thread pool's actively executing task counts metricRegistry.register(MetricRegistry.name("threadpool", "activeTasks"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index a42c8b43bbf7f..bca0b152268ad 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -52,7 +52,8 @@ private[spark] class MesosExecutorBackend slaveInfo: SlaveInfo) { logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++ + Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue)) executor = new Executor( executorInfo.getExecutorId.getValue, slaveInfo.getHostname, diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index fd316a89a1a10..5dd67b0cbf683 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -83,10 +83,10 @@ private[spark] class MetricsSystem private ( def getServletHandlers = metricsServlet.map(_.getHandlers).getOrElse(Array()) metricsConfig.initialize() - registerSources() - registerSinks() def start() { + registerSources() + registerSinks() sinks.foreach(_.start) } @@ -98,10 +98,39 @@ private[spark] class MetricsSystem private ( sinks.foreach(_.report()) } + /** + * Build a name that uniquely identifies each metric source. + * The name is structured as follows: ... + * If either ID is not available, this defaults to just using . + * + * @param source Metric source to be named by this method. + * @return An unique metric name for each combination of + * application, executor/driver and metric source. + */ + def buildRegistryName(source: Source): String = { + val appId = conf.getOption("spark.app.id") + val executorId = conf.getOption("spark.executor.id") + val defaultName = MetricRegistry.name(source.sourceName) + + if (instance == "driver" || instance == "executor") { + if (appId.isDefined && executorId.isDefined) { + MetricRegistry.name(appId.get, executorId.get, source.sourceName) + } else { + // Only Driver and Executor are set spark.app.id and spark.executor.id. + // For instance, Master and Worker are not related to a specific application. + val warningMsg = s"Using default name $defaultName for source because %s is not set." + if (appId.isEmpty) { logWarning(warningMsg.format("spark.app.id")) } + if (executorId.isEmpty) { logWarning(warningMsg.format("spark.executor.id")) } + defaultName + } + } else { defaultName } + } + def registerSource(source: Source) { sources += source try { - registry.register(source.sourceName, source.metricRegistry) + val regName = buildRegistryName(source) + registry.register(regName, source.metricRegistry) } catch { case e: IllegalArgumentException => logInfo("Metrics already registered", e) } @@ -109,8 +138,9 @@ private[spark] class MetricsSystem private ( def removeSource(source: Source) { sources -= source + val regName = buildRegistryName(source) registry.removeMatching(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name.startsWith(source.sourceName) + def matches(name: String, metric: Metric): Boolean = name.startsWith(regName) }) } @@ -125,7 +155,7 @@ private[spark] class MetricsSystem private ( val source = Class.forName(classPath).newInstance() registerSource(source.asInstanceOf[Source]) } catch { - case e: Exception => logError("Source class " + classPath + " cannot be instantialized", e) + case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index 94944399b134a..12668b6c0988e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -22,10 +22,10 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import org.apache.spark.SparkContext import org.apache.spark.metrics.source.Source -private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: SparkContext) +private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler) extends Source { override val metricRegistry = new MetricRegistry() - override val sourceName = "%s.DAGScheduler".format(sc.appName) + override val sourceName = "DAGScheduler" metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { override def getValue: Int = dagScheduler.failedStages.size diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 64b32ae0edaac..100c9ba9b7809 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -43,38 +43,29 @@ import org.apache.spark.util.{FileLogger, JsonProtocol, Utils} * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ private[spark] class EventLoggingListener( - appName: String, + appId: String, + logBaseDir: String, sparkConf: SparkConf, hadoopConf: Configuration) extends SparkListener with Logging { import EventLoggingListener._ - def this(appName: String, sparkConf: SparkConf) = - this(appName, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) + def this(appId: String, logBaseDir: String, sparkConf: SparkConf) = + this(appId, logBaseDir, sparkConf, SparkHadoopUtil.get.newConfiguration(sparkConf)) private val shouldCompress = sparkConf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = sparkConf.getBoolean("spark.eventLog.overwrite", false) private val testing = sparkConf.getBoolean("spark.eventLog.testing", false) private val outputBufferSize = sparkConf.getInt("spark.eventLog.buffer.kb", 100) * 1024 - private val logBaseDir = sparkConf.get("spark.eventLog.dir", DEFAULT_LOG_DIR).stripSuffix("/") - private val name = appName.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_") - .toLowerCase + "-" + System.currentTimeMillis - val logDir = Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") - + val logDir = EventLoggingListener.getLogDirPath(logBaseDir, appId) + val logDirName: String = logDir.split("/").last protected val logger = new FileLogger(logDir, sparkConf, hadoopConf, outputBufferSize, shouldCompress, shouldOverwrite, Some(LOG_FILE_PERMISSIONS)) // For testing. Keep track of all JSON serialized events that have been logged. private[scheduler] val loggedEvents = new ArrayBuffer[JValue] - /** - * Return only the unique application directory without the base directory. - */ - def getApplicationLogDir(): String = { - name - } - /** * Begin logging events. * If compression is used, log a file that indicates which compression library is used. @@ -184,6 +175,18 @@ private[spark] object EventLoggingListener extends Logging { } else "" } + /** + * Return a file-system-safe path to the log directory for the given application. + * + * @param logBaseDir A base directory for the path to the log directory for given application. + * @param appId A unique app ID. + * @return A path which consists of file-system-safe characters. + */ + def getLogDirPath(logBaseDir: String, appId: String): String = { + val name = appId.replaceAll("[ :/]", "-").replaceAll("[${}'\"]", "_").toLowerCase + Utils.resolveURI(logBaseDir) + "/" + name.stripSuffix("/") + } + /** * Parse the event logging information associated with the logs in the given directory. * 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 a0be8307eff27..992c477493d8e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -23,6 +23,8 @@ package org.apache.spark.scheduler * machines become available and can launch tasks on them. */ private[spark] trait SchedulerBackend { + private val appId = "spark-application-" + System.currentTimeMillis + def start(): Unit def stop(): Unit def reviveOffers(): Unit @@ -33,10 +35,10 @@ private[spark] trait SchedulerBackend { def isReady(): Boolean = true /** - * The application ID associated with the job, if any. + * Get an application ID associated with the job. * - * @return The application ID, or None if the backend does not provide an ID. + * @return An application ID */ - def applicationId(): Option[String] = None + def applicationId(): String = appId } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 1c1ce666eab0f..a129a434c9a1a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -31,6 +31,8 @@ import org.apache.spark.storage.BlockManagerId */ private[spark] trait TaskScheduler { + private val appId = "spark-application-" + System.currentTimeMillis + def rootPool: Pool def schedulingMode: SchedulingMode @@ -66,10 +68,10 @@ private[spark] trait TaskScheduler { blockManagerId: BlockManagerId): Boolean /** - * The application ID associated with the job, if any. + * Get an application ID associated with the job. * - * @return The application ID, or None if the backend does not provide an ID. + * @return An application ID */ - def applicationId(): Option[String] = None + def applicationId(): String = appId } 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 633e892554c50..4dc550413c13c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -492,7 +492,7 @@ private[spark] class TaskSchedulerImpl( } } - override def applicationId(): Option[String] = backend.applicationId() + override def applicationId(): String = backend.applicationId() } 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 5c5ecc8434d78..ed209d195ec9d 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 @@ -68,9 +68,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - val eventLogDir = sc.eventLogger.map(_.logDir) val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, eventLogDir) + appUIAddress, sc.eventLogDir) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() @@ -129,7 +128,11 @@ private[spark] class SparkDeploySchedulerBackend( totalCoreCount.get() >= totalExpectedCores * minRegisteredRatio } - override def applicationId(): Option[String] = Option(appId) + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } private def waitForRegistration() = { registrationLock.synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 3161f1ee9fa8a..90828578cd88f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -76,6 +76,8 @@ private[spark] class CoarseMesosSchedulerBackend( var nextMesosTaskId = 0 + @volatile var appId: String = _ + def newMesosTaskId(): Int = { val id = nextMesosTaskId nextMesosTaskId += 1 @@ -167,7 +169,8 @@ private[spark] class CoarseMesosSchedulerBackend( override def offerRescinded(d: SchedulerDriver, o: OfferID) {} override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { - logInfo("Registered as framework ID " + frameworkId.getValue) + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() @@ -313,4 +316,10 @@ private[spark] class CoarseMesosSchedulerBackend( slaveLost(d, s) } + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 4c49aa074ebc0..b11786368e661 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -30,7 +30,7 @@ import org.apache.mesos._ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTaskState, _} import org.apache.spark.{Logging, SparkContext, SparkException, TaskState} -import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SchedulerBackend, SlaveLost, TaskDescription, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.scheduler._ import org.apache.spark.util.Utils /** @@ -62,6 +62,8 @@ private[spark] class MesosSchedulerBackend( var classLoader: ClassLoader = null + @volatile var appId: String = _ + override def start() { synchronized { classLoader = Thread.currentThread.getContextClassLoader @@ -177,7 +179,8 @@ private[spark] class MesosSchedulerBackend( override def registered(d: SchedulerDriver, frameworkId: FrameworkID, masterInfo: MasterInfo) { val oldClassLoader = setClassLoader() try { - logInfo("Registered as framework ID " + frameworkId.getValue) + appId = frameworkId.getValue + logInfo("Registered as framework ID " + appId) registeredLock.synchronized { isRegistered = true registeredLock.notifyAll() @@ -372,4 +375,10 @@ private[spark] class MesosSchedulerBackend( // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) + override def applicationId(): String = + Option(appId).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } + } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 9ea25c2bc7090..58b78f041cd85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -88,6 +88,7 @@ private[spark] class LocalActor( private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) extends SchedulerBackend with ExecutorBackend { + private val appId = "local-" + System.currentTimeMillis var localActor: ActorRef = null override def start() { @@ -115,4 +116,6 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: localActor ! StatusUpdate(taskId, state, serializedData) } + override def applicationId(): String = appId + } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala index 49fea6d9e2a76..8569c6f3cbbc3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSource.scala @@ -22,10 +22,10 @@ import com.codahale.metrics.{Gauge,MetricRegistry} import org.apache.spark.SparkContext import org.apache.spark.metrics.source.Source -private[spark] class BlockManagerSource(val blockManager: BlockManager, sc: SparkContext) +private[spark] class BlockManagerSource(val blockManager: BlockManager) extends Source { override val metricRegistry = new MetricRegistry() - override val sourceName = "%s.BlockManager".format(sc.appName) + override val sourceName = "BlockManager" metricRegistry.register(MetricRegistry.name("memory", "maxMem_MB"), new Gauge[Long] { override def getValue: Long = { diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index e42b181194727..3925f0ccbdbf0 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -17,14 +17,15 @@ package org.apache.spark.metrics -import org.apache.spark.metrics.source.Source import org.scalatest.{BeforeAndAfter, FunSuite, PrivateMethodTester} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.master.MasterSource +import org.apache.spark.metrics.source.Source -import scala.collection.mutable.ArrayBuffer +import com.codahale.metrics.MetricRegistry +import scala.collection.mutable.ArrayBuffer class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester{ var filePath: String = _ @@ -39,6 +40,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod test("MetricsSystem with default config") { val metricsSystem = MetricsSystem.createMetricsSystem("default", conf, securityMgr) + metricsSystem.start() val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) @@ -49,6 +51,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod test("MetricsSystem with sources add") { val metricsSystem = MetricsSystem.createMetricsSystem("test", conf, securityMgr) + metricsSystem.start() val sources = PrivateMethod[ArrayBuffer[Source]]('sources) val sinks = PrivateMethod[ArrayBuffer[Source]]('sinks) @@ -60,4 +63,125 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod metricsSystem.registerSource(source) assert(metricsSystem.invokePrivate(sources()).length === 1) } + + test("MetricsSystem with Driver instance") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "driver" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === s"$appId.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Driver instance and spark.app.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val executorId = "driver" + conf.set("spark.executor.id", executorId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with Driver instance and spark.executor.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + conf.set("spark.app.id", appId) + + val instanceName = "driver" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with Executor instance") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "executor.1" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === s"$appId.$executorId.${source.sourceName}") + } + + test("MetricsSystem with Executor instance and spark.app.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val executorId = "executor.1" + conf.set("spark.executor.id", executorId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with Executor instance and spark.executor.id is not set") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + conf.set("spark.app.id", appId) + + val instanceName = "executor" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + assert(metricName === source.sourceName) + } + + test("MetricsSystem with instance which is neither Driver nor Executor") { + val source = new Source { + override val sourceName = "dummySource" + override val metricRegistry = new MetricRegistry() + } + + val appId = "testId" + val executorId = "dummyExecutorId" + conf.set("spark.app.id", appId) + conf.set("spark.executor.id", executorId) + + val instanceName = "testInstance" + val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) + + val metricName = driverMetricsSystem.buildRegistryName(source) + + // Even if spark.app.id and spark.executor.id are set, they are not used for the metric name. + assert(metricName != s"$appId.$executorId.${source.sourceName}") + assert(metricName === source.sourceName) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index e5315bc93e217..3efa85431876b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -169,7 +169,9 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Verify logging directory exists val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val logBaseDir = conf.get("spark.eventLog.dir") + val appId = EventLoggingListenerSuite.getUniqueApplicationId + val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) eventLogger.start() val logPath = new Path(eventLogger.logDir) assert(fileSystem.exists(logPath)) @@ -209,7 +211,9 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { // Verify that all information is correctly parsed before stop() val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val logBaseDir = conf.get("spark.eventLog.dir") + val appId = EventLoggingListenerSuite.getUniqueApplicationId + val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) eventLogger.start() var eventLoggingInfo = EventLoggingListener.parseLoggingInfo(eventLogger.logDir, fileSystem) assertInfoCorrect(eventLoggingInfo, loggerStopped = false) @@ -228,7 +232,9 @@ class EventLoggingListenerSuite extends FunSuite with BeforeAndAfter { */ private def testEventLogging(compressionCodec: Option[String] = None) { val conf = getLoggingConf(logDirPath, compressionCodec) - val eventLogger = new EventLoggingListener("test", conf) + val logBaseDir = conf.get("spark.eventLog.dir") + val appId = EventLoggingListenerSuite.getUniqueApplicationId + val eventLogger = new EventLoggingListener(appId, logBaseDir, conf) val listenerBus = new LiveListenerBus val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey") @@ -408,4 +414,6 @@ object EventLoggingListenerSuite { } conf } + + def getUniqueApplicationId = "test-" + System.currentTimeMillis } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index 7ab351d1b4d24..48114feee6233 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -155,7 +155,8 @@ class ReplayListenerSuite extends FunSuite with BeforeAndAfter { * This child listener inherits only the event buffering functionality, but does not actually * log the events. */ - private class EventMonster(conf: SparkConf) extends EventLoggingListener("test", conf) { + private class EventMonster(conf: SparkConf) + extends EventLoggingListener("test", "testdir", conf) { logger.close() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index 99c8d13231aac..eb6e88cf5520d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import java.nio.ByteBuffer +import java.util.concurrent.Semaphore import scala.collection.mutable.ArrayBuffer @@ -36,6 +37,7 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { val receiver = new FakeReceiver val executor = new FakeReceiverSupervisor(receiver) + val executorStarted = new Semaphore(0) assert(executor.isAllEmpty) @@ -43,6 +45,7 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { val executingThread = new Thread() { override def run() { executor.start() + executorStarted.release(1) executor.awaitTermination() } } @@ -57,6 +60,9 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { } } + // Ensure executor is started + executorStarted.acquire() + // Verify that receiver was started assert(receiver.onStartCalled) assert(executor.isReceiverStarted) @@ -186,10 +192,10 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. */ class FakeReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) { - var otherThread: Thread = null - var receiving = false - var onStartCalled = false - var onStopCalled = false + @volatile var otherThread: Thread = null + @volatile var receiving = false + @volatile var onStartCalled = false + @volatile var onStopCalled = false def onStart() { otherThread = new Thread() { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 10cbeb8b94325..229b7a09f456b 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -47,6 +47,7 @@ class ExecutorRunnable( hostname: String, executorMemory: Int, executorCores: Int, + appAttemptId: String, securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { @@ -83,7 +84,7 @@ class ExecutorRunnable( ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - localResources) + appAttemptId, localResources) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index d7a7175d5e578..5cb4753de2e84 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -43,6 +43,7 @@ trait ExecutorRunnableUtil extends Logging { hostname: String, executorMemory: Int, executorCores: Int, + appId: String, localResources: HashMap[String, LocalResource]): List[String] = { // Extra options for the JVM val javaOpts = ListBuffer[String]() @@ -114,6 +115,7 @@ trait ExecutorRunnableUtil extends Logging { slaveId.toString, hostname.toString, executorCores.toString, + appId, "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 4f4f1d2aaaade..e1af8d5a74cb1 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -57,6 +57,7 @@ object AllocationType extends Enumeration { private[yarn] abstract class YarnAllocator( conf: Configuration, sparkConf: SparkConf, + appAttemptId: ApplicationAttemptId, args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], securityMgr: SecurityManager) @@ -295,6 +296,7 @@ private[yarn] abstract class YarnAllocator( executorHostname, executorMemory, executorCores, + appAttemptId.getApplicationId.toString, securityMgr) launcherPool.execute(executorRunnable) } 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 200a30899290b..6bb4b82316ad4 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 @@ -155,6 +155,10 @@ private[spark] class YarnClientSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): Option[String] = Option(appId).map(_.toString()) + override def applicationId(): String = + Option(appId).map(_.toString).getOrElse { + logWarning("Application ID is not initialized yet.") + super.applicationId + } } 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 index 39436d0999663..3a186cfeb4eeb 100644 --- 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 @@ -48,6 +48,13 @@ private[spark] class YarnClusterSchedulerBackend( totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio } - override def applicationId(): Option[String] = sc.getConf.getOption("spark.yarn.app.id") + override def applicationId(): String = + // In YARN Cluster mode, spark.yarn.app.id is expect to be set + // before user application is launched. + // So, if spark.yarn.app.id is not set, it is something wrong. + sc.getConf.getOption("spark.yarn.app.id").getOrElse { + logError("Application ID is not set.") + super.applicationId + } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 833be12982e71..0b5a92d87d722 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -47,6 +47,7 @@ class ExecutorRunnable( hostname: String, executorMemory: Int, executorCores: Int, + appId: String, securityMgr: SecurityManager) extends Runnable with ExecutorRunnableUtil with Logging { @@ -80,7 +81,7 @@ class ExecutorRunnable( ctx.setTokens(ByteBuffer.wrap(dob.getData())) val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - localResources) + appId, localResources) logInfo(s"Setting up executor with environment: $env") logInfo("Setting up executor with commands: " + commands) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index e44a8db41b97e..2bbf5d7db8668 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -41,7 +41,7 @@ private[yarn] class YarnAllocationHandler( args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { + extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { override protected def releaseContainer(container: Container) = { amClient.releaseAssignedContainer(container.getId()) From cf1d32e3e1071829b152d4b597bf0a0d7a5629a2 Mon Sep 17 00:00:00 2001 From: mcheah Date: Fri, 3 Oct 2014 14:22:11 -0700 Subject: [PATCH 0893/1492] [SPARK-1860] More conservative app directory cleanup. First contribution to the project, so apologize for any significant errors. This PR addresses [SPARK-1860]. The application directories are now cleaned up in a more conservative manner. Previously, app-* directories were cleaned up if the directory's timestamp was older than a given time. However, the timestamp on a directory does not reflect the modification times of the files in that directory. Therefore, app-* directories were wiped out even if the files inside them were created recently and possibly being used by Executor tasks. The solution is to change the cleanup logic to inspect all files within the app-* directory and only eliminate the app-* directory if all files in the directory are stale. Author: mcheah Closes #2609 from mccheah/worker-better-app-dir-cleanup and squashes the following commits: 87b5d03 [mcheah] [SPARK-1860] Using more string interpolation. Better error logging. 802473e [mcheah] [SPARK-1860] Cleaning up the logs generated when cleaning directories. e0a1f2e [mcheah] [SPARK-1860] Fixing broken unit test. 77a9de0 [mcheah] [SPARK-1860] More conservative app directory cleanup. --- .../spark/deploy/worker/ExecutorRunner.scala | 8 +--- .../apache/spark/deploy/worker/Worker.scala | 37 ++++++++++++++++--- .../scala/org/apache/spark/util/Utils.scala | 21 +++++++---- .../org/apache/spark/util/UtilsSuite.scala | 23 +++++++++--- 4 files changed, 62 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 00a43673e5cd3..71650cd773bcf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -42,7 +42,7 @@ private[spark] class ExecutorRunner( val workerId: String, val host: String, val sparkHome: File, - val workDir: File, + val executorDir: File, val workerUrl: String, val conf: SparkConf, var state: ExecutorState.Value) @@ -130,12 +130,6 @@ private[spark] class ExecutorRunner( */ def fetchAndRunExecutor() { try { - // Create the executor's working directory - val executorDir = new File(workDir, appId + "/" + execId) - if (!executorDir.mkdirs()) { - throw new IOException("Failed to create directory " + executorDir) - } - // Launch the process val command = getCommandSeq logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 0c454e4138c96..3b13f43a1868c 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -18,15 +18,18 @@ package org.apache.spark.deploy.worker import java.io.File +import java.io.IOException import java.text.SimpleDateFormat import java.util.Date +import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import scala.language.postfixOps import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import org.apache.commons.io.FileUtils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} @@ -191,6 +194,7 @@ private[spark] class Worker( changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) if (CLEANUP_ENABLED) { + logInfo(s"Worker cleanup enabled; old application directories will be deleted in: $workDir") context.system.scheduler.schedule(CLEANUP_INTERVAL_MILLIS millis, CLEANUP_INTERVAL_MILLIS millis, self, WorkDirCleanup) } @@ -201,10 +205,23 @@ private[spark] class Worker( case WorkDirCleanup => // Spin up a separate thread (in a future) to do the dir cleanup; don't tie up worker actor val cleanupFuture = concurrent.future { - logInfo("Cleaning up oldest application directories in " + workDir + " ...") - Utils.findOldFiles(workDir, APP_DATA_RETENTION_SECS) - .foreach(Utils.deleteRecursively) + val appDirs = workDir.listFiles() + if (appDirs == null) { + throw new IOException("ERROR: Failed to list files in " + appDirs) + } + appDirs.filter { dir => + // the directory is used by an application - check that the application is not running + // when cleaning up + val appIdFromDir = dir.getName + val isAppStillRunning = executors.values.map(_.appId).contains(appIdFromDir) + dir.isDirectory && !isAppStillRunning && + !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECS) + }.foreach { dir => + logInfo(s"Removing directory: ${dir.getPath}") + Utils.deleteRecursively(dir) + } } + cleanupFuture onFailure { case e: Throwable => logError("App dir cleanup failed: " + e.getMessage, e) @@ -233,8 +250,15 @@ private[spark] class Worker( } else { try { logInfo("Asked to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) + + // Create the executor's working directory + val executorDir = new File(workDir, appId + "/" + execId) + if (!executorDir.mkdirs()) { + throw new IOException("Failed to create directory " + executorDir) + } + val manager = new ExecutorRunner(appId, execId, appDesc, cores_, memory_, - self, workerId, host, sparkHome, workDir, akkaUrl, conf, ExecutorState.LOADING) + self, workerId, host, sparkHome, executorDir, akkaUrl, conf, ExecutorState.LOADING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ @@ -242,12 +266,13 @@ private[spark] class Worker( master ! ExecutorStateChanged(appId, execId, manager.state, None, None) } catch { case e: Exception => { - logError("Failed to launch executor %s/%d for %s".format(appId, execId, appDesc.name)) + logError(s"Failed to launch executor $appId/$execId for ${appDesc.name}.", e) if (executors.contains(appId + "/" + execId)) { executors(appId + "/" + execId).kill() executors -= appId + "/" + execId } - master ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, None, None) + master ! ExecutorStateChanged(appId, execId, ExecutorState.FAILED, + Some(e.toString), None) } } } 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 9399ddab76331..a67124140f9da 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -35,6 +35,8 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder +import org.apache.commons.io.FileUtils +import org.apache.commons.io.filefilter.TrueFileFilter import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.log4j.PropertyConfigurator @@ -705,17 +707,20 @@ private[spark] object Utils extends Logging { } /** - * Finds all the files in a directory whose last modified time is older than cutoff seconds. - * @param dir must be the path to a directory, or IllegalArgumentException is thrown - * @param cutoff measured in seconds. Files older than this are returned. + * Determines if a directory contains any files newer than cutoff seconds. + * + * @param dir must be the path to a directory, or IllegalArgumentException is thrown + * @param cutoff measured in seconds. Returns true if there are any files in dir newer than this. */ - def findOldFiles(dir: File, cutoff: Long): Seq[File] = { + def doesDirectoryContainAnyNewFiles(dir: File, cutoff: Long): Boolean = { val currentTimeMillis = System.currentTimeMillis - if (dir.isDirectory) { - val files = listFilesSafely(dir) - files.filter { file => file.lastModified < (currentTimeMillis - cutoff * 1000) } + if (!dir.isDirectory) { + throw new IllegalArgumentException (dir + " is not a directory!") } else { - throw new IllegalArgumentException(dir + " is not a directory!") + val files = FileUtils.listFilesAndDirs(dir, TrueFileFilter.TRUE, TrueFileFilter.TRUE) + val cutoffTimeInMillis = (currentTimeMillis - (cutoff * 1000)) + val newFiles = files.filter { _.lastModified > cutoffTimeInMillis } + newFiles.nonEmpty } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 70d423ba8a04d..e63d9d085e385 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -189,17 +189,28 @@ class UtilsSuite extends FunSuite { assert(Utils.getIteratorSize(iterator) === 5L) } - test("findOldFiles") { + test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files val parent: File = Utils.createTempDir() val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories val child2: File = Utils.createTempDir(parent.getCanonicalPath) - // set the last modified time of child1 to 10 secs old - child1.setLastModified(System.currentTimeMillis() - (1000 * 10)) + val child3: File = Utils.createTempDir(child1.getCanonicalPath) + // set the last modified time of child1 to 30 secs old + child1.setLastModified(System.currentTimeMillis() - (1000 * 30)) - val result = Utils.findOldFiles(parent, 5) // find files older than 5 secs - assert(result.size.equals(1)) - assert(result(0).getCanonicalPath.equals(child1.getCanonicalPath)) + // although child1 is old, child2 is still new so return true + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + child2.setLastModified(System.currentTimeMillis - (1000 * 30)) + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + parent.setLastModified(System.currentTimeMillis - (1000 * 30)) + // although parent and its immediate children are new, child3 is still old + // we expect a full recursive search for new files. + assert(Utils.doesDirectoryContainAnyNewFiles(parent, 5)) + + child3.setLastModified(System.currentTimeMillis - (1000 * 30)) + assert(!Utils.doesDirectoryContainAnyNewFiles(parent, 5)) } test("resolveURI") { From 32fad4233f353814496c84e15ba64326730b7ae7 Mon Sep 17 00:00:00 2001 From: Brenden Matthews Date: Sun, 5 Oct 2014 09:49:24 -0700 Subject: [PATCH 0894/1492] [SPARK-3597][Mesos] Implement `killTask`. The MesosSchedulerBackend did not previously implement `killTask`, resulting in an exception. Author: Brenden Matthews Closes #2453 from brndnmtthws/implement-killtask and squashes the following commits: 23ddcdc [Brenden Matthews] [SPARK-3597][Mesos] Implement `killTask`. --- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index b11786368e661..e0f2fd622f54c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -372,6 +372,13 @@ private[spark] class MesosSchedulerBackend( recordSlaveLost(d, slaveId, ExecutorExited(status)) } + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = { + driver.killTask( + TaskID.newBuilder() + .setValue(taskId.toString).build() + ) + } + // TODO: query Mesos for number of cores override def defaultParallelism() = sc.conf.getInt("spark.default.parallelism", 8) From a7c73130f1b6b0b8b19a7b0a0de5c713b673cd7b Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 5 Oct 2014 09:55:17 -0700 Subject: [PATCH 0895/1492] SPARK-1656: Fix potential resource leaks JIRA: https://issues.apache.org/jira/browse/SPARK-1656 Author: zsxwing Closes #577 from zsxwing/SPARK-1656 and squashes the following commits: c431095 [zsxwing] Add a comment and fix the code style 2de96e5 [zsxwing] Make sure file will be deleted if exception happens 28b90dc [zsxwing] Update to follow the code style 4521d6e [zsxwing] Merge branch 'master' into SPARK-1656 afc3383 [zsxwing] Update to follow the code style 071fdd1 [zsxwing] SPARK-1656: Fix potential resource leaks --- .../spark/broadcast/HttpBroadcast.scala | 25 +++++++++++-------- .../master/FileSystemPersistenceEngine.scala | 14 ++++++++--- .../org/apache/spark/storage/DiskStore.scala | 16 +++++++++++- 3 files changed, 40 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 942dc7d7eac87..4cd4f4f96fd16 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -163,18 +163,23 @@ private[broadcast] object HttpBroadcast extends Logging { private def write(id: Long, value: Any) { val file = getFile(id) - val out: OutputStream = { - if (compress) { - compressionCodec.compressedOutputStream(new FileOutputStream(file)) - } else { - new BufferedOutputStream(new FileOutputStream(file), bufferSize) + val fileOutputStream = new FileOutputStream(file) + try { + val out: OutputStream = { + if (compress) { + compressionCodec.compressedOutputStream(fileOutputStream) + } else { + new BufferedOutputStream(fileOutputStream, bufferSize) + } } + val ser = SparkEnv.get.serializer.newInstance() + val serOut = ser.serializeStream(out) + serOut.writeObject(value) + serOut.close() + files += file + } finally { + fileOutputStream.close() } - val ser = SparkEnv.get.serializer.newInstance() - val serOut = ser.serializeStream(out) - serOut.writeObject(value) - serOut.close() - files += file } private def read[T: ClassTag](id: Long): T = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index aa85aa060d9c1..08a99bbe68578 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -83,15 +83,21 @@ private[spark] class FileSystemPersistenceEngine( val serialized = serializer.toBinary(value) val out = new FileOutputStream(file) - out.write(serialized) - out.close() + try { + out.write(serialized) + } finally { + out.close() + } } def deserializeFromFile[T](file: File)(implicit m: Manifest[T]): T = { val fileData = new Array[Byte](file.length().asInstanceOf[Int]) val dis = new DataInputStream(new FileInputStream(file)) - dis.readFully(fileData) - dis.close() + try { + dis.readFully(fileData) + } finally { + dis.close() + } val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index e9304f6bb45d0..bac459e835a3f 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -73,7 +73,21 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val outputStream = new FileOutputStream(file) - blockManager.dataSerializeStream(blockId, outputStream, values) + try { + try { + blockManager.dataSerializeStream(blockId, outputStream, values) + } finally { + // Close outputStream here because it should be closed before file is deleted. + outputStream.close() + } + } catch { + case e: Throwable => + if (file.exists()) { + file.delete() + } + throw e + } + val length = file.length val timeTaken = System.currentTimeMillis - startTime From 1b97a941a09a2f63d442f435c1b444d857cd6956 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Oct 2014 11:19:17 -0700 Subject: [PATCH 0896/1492] [SPARK-3007][SQL] Fixes dynamic partitioning support for lower Hadoop versions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This is a follow up of #2226 and #2616 to fix Jenkins master SBT build failures for lower Hadoop versions (1.0.x and 2.0.x). The root cause is the semantics difference of `FileSystem.globStatus()` between different versions of Hadoop, as illustrated by the following test code: ```scala object GlobExperiments extends App { val conf = new Configuration() val fs = FileSystem.getLocal(conf) fs.globStatus(new Path("/tmp/wh/*/*/*")).foreach { status => println(status.getPath) } } ``` Target directory structure: ``` /tmp/wh ├── dir0 │   ├── dir1 │   │   └── level2 │   └── level1 └── level0 ``` Hadoop 2.4.1 result: ``` file:/tmp/wh/dir0/dir1/level2 ``` Hadoop 1.0.4 resuet: ``` file:/tmp/wh/dir0/dir1/level2 file:/tmp/wh/dir0/level1 file:/tmp/wh/level0 ``` In #2226 and #2616, we call `FileOutputCommitter.commitJob()` at the end of the job, and the `_SUCCESS` mark file is written. When working with lower Hadoop versions, due to the `globStatus()` semantics issue, `_SUCCESS` is included as a separate partition data file by `Hive.loadDynamicPartitions()`, and fails partition spec checking. The fix introduced in this PR is kind of a hack: when inserting data with dynamic partitioning, we intentionally avoid writing the `_SUCCESS` marker to workaround this issue. Hive doesn't suffer this issue because `FileSinkOperator` doesn't call `FileOutputCommitter.commitJob()`, instead, it calls `Utilities.mvFileToFinalPath()` to cleanup the output directory and then loads it into Hive warehouse by with `loadDynamicPartitions()`/`loadPartition()`/`loadTable()`. This approach is better because it handles failed job and speculative tasks properly. We should add this step to `InsertIntoHiveTable` in another PR. Author: Cheng Lian Closes #2663 from liancheng/dp-hadoop-1-fix and squashes the following commits: 0177dae [Cheng Lian] Fixes dynamic partitioning support for lower Hadoop versions --- .../spark/sql/hive/hiveWriterContainers.scala | 26 ++++++++++++++++--- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index ac5c7a8220296..6ccbc22a4acfb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -55,8 +55,8 @@ private[hive] class SparkHiveWriterContainer( private var taID: SerializableWritable[TaskAttemptID] = null @transient private var writer: FileSinkOperator.RecordWriter = null - @transient private lazy val committer = conf.value.getOutputCommitter - @transient private lazy val jobContext = newJobContext(conf.value, jID.value) + @transient protected lazy val committer = conf.value.getOutputCommitter + @transient protected lazy val jobContext = newJobContext(conf.value, jID.value) @transient private lazy val taskContext = newTaskAttemptContext(conf.value, taID.value) @transient private lazy val outputFormat = conf.value.getOutputFormat.asInstanceOf[HiveOutputFormat[AnyRef,Writable]] @@ -122,8 +122,6 @@ private[hive] class SparkHiveWriterContainer( } } - // ********* Private Functions ********* - private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { jobID = jobId splitID = splitId @@ -157,12 +155,18 @@ private[hive] object SparkHiveWriterContainer { } } +private[spark] object SparkHiveDynamicPartitionWriterContainer { + val SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = "mapreduce.fileoutputcommitter.marksuccessfuljobs" +} + private[spark] class SparkHiveDynamicPartitionWriterContainer( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc, dynamicPartColNames: Array[String]) extends SparkHiveWriterContainer(jobConf, fileSinkConf) { + import SparkHiveDynamicPartitionWriterContainer._ + private val defaultPartName = jobConf.get( ConfVars.DEFAULTPARTITIONNAME.varname, ConfVars.DEFAULTPARTITIONNAME.defaultVal) @@ -179,6 +183,20 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( commit() } + override def commitJob(): Unit = { + // This is a hack to avoid writing _SUCCESS mark file. In lower versions of Hadoop (e.g. 1.0.4), + // semantics of FileSystem.globStatus() is different from higher versions (e.g. 2.4.1) and will + // include _SUCCESS file when glob'ing for dynamic partition data files. + // + // Better solution is to add a step similar to what Hive FileSinkOperator.jobCloseOp does: + // calling something like Utilities.mvFileToFinalPath to cleanup the output directory and then + // load it with loadDynamicPartitions/loadPartition/loadTable. + val oldMarker = jobConf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, true) + jobConf.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, false) + super.commitJob() + jobConf.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, oldMarker) + } + override def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = { val dynamicPartPath = dynamicPartColNames .zip(row.takeRight(dynamicPartColNames.length)) From e222221e24c122300bbde6d5ec4002a7c42b2e24 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 5 Oct 2014 13:22:40 -0700 Subject: [PATCH 0897/1492] HOTFIX: Fix unicode error in merge script. The merge script builds up a big command array and sometimes this contains both unicode and ascii strings. This doesn't work if you try to join them into a single string. Longer term a solution is to go and make sure the source of all strings is unicode. This patch provides a simpler solution... just print the array rather than joining. I actually prefer printing an array here anyways since joining on spaces is lossy in the case of arguments that themselves contain spaces. Author: Patrick Wendell Closes #2645 from pwendell/merge-script and squashes the following commits: 167b792 [Patrick Wendell] HOTFIX: Fix unicode error in merge script. --- dev/merge_spark_pr.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index a8e92e36fe0d8..02ac20984add9 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -73,11 +73,10 @@ def fail(msg): def run_cmd(cmd): + print cmd if isinstance(cmd, list): - print " ".join(cmd) return subprocess.check_output(cmd) else: - print cmd return subprocess.check_output(cmd.split(" ")) From 79b2108de30bf91c8e58bb36405d334aeb2a00ad Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 5 Oct 2014 17:44:38 -0700 Subject: [PATCH 0898/1492] [Minor] Trivial fix to make codes more readable It should just use `maxResults` there. Author: Liang-Chi Hsieh Closes #2654 from viirya/trivial_fix and squashes the following commits: 1362289 [Liang-Chi Hsieh] Trivial fix to make codes more readable. --- .../src/main/scala/org/apache/spark/sql/hive/HiveContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 8bcc098bbb620..fad3b39f81413 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -268,7 +268,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { */ protected[sql] def runSqlHive(sql: String): Seq[String] = { val maxResults = 100000 - val results = runHive(sql, 100000) + val results = runHive(sql, maxResults) // It is very confusing when you only get back some of the results... if (results.size == maxResults) sys.error("RESULTS POSSIBLY TRUNCATED") results From 58f5361caaa2f898e38ae4b3794167881e20a818 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 5 Oct 2014 17:47:20 -0700 Subject: [PATCH 0899/1492] [SPARK-3792][SQL] Enable JavaHiveQLSuite Do not use TestSQLContext in JavaHiveQLSuite, that may lead to two SparkContexts in one jvm and enable JavaHiveQLSuite Author: scwf Closes #2652 from scwf/fix-JavaHiveQLSuite and squashes the following commits: be35c91 [scwf] enable JavaHiveQLSuite --- .../sql/hive/api/java/JavaHiveQLSuite.scala | 27 +++++++------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala index 9644b707eb1a0..46b11b582b26d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala @@ -25,34 +25,30 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.api.java.JavaSchemaRDD import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.sql.test.TestSQLContext // Implicits import scala.collection.JavaConversions._ class JavaHiveQLSuite extends FunSuite { - lazy val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext) + lazy val javaCtx = new JavaSparkContext(TestHive.sparkContext) // There is a little trickery here to avoid instantiating two HiveContexts in the same JVM lazy val javaHiveCtx = new JavaHiveContext(javaCtx) { override val sqlContext = TestHive } - ignore("SELECT * FROM src") { + test("SELECT * FROM src") { assert( javaHiveCtx.sql("SELECT * FROM src").collect().map(_.getInt(0)) === TestHive.sql("SELECT * FROM src").collect().map(_.getInt(0)).toSeq) } - private val explainCommandClassName = - classOf[ExplainCommand].getSimpleName.stripSuffix("$") - def isExplanation(result: JavaSchemaRDD) = { val explanation = result.collect().map(_.getString(0)) - explanation.size > 1 && explanation.head.startsWith(explainCommandClassName) + explanation.size > 1 && explanation.head.startsWith("== Physical Plan ==") } - ignore("Query Hive native command execution result") { + test("Query Hive native command execution result") { val tableName = "test_native_commands" assertResult(0) { @@ -63,23 +59,18 @@ class JavaHiveQLSuite extends FunSuite { javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)").count() } - javaHiveCtx.sql("SHOW TABLES").registerTempTable("show_tables") - assert( javaHiveCtx - .sql("SELECT result FROM show_tables") + .sql("SHOW TABLES") .collect() .map(_.getString(0)) .contains(tableName)) - assertResult(Array(Array("key", "int", "None"), Array("value", "string", "None"))) { - javaHiveCtx.sql(s"DESCRIBE $tableName").registerTempTable("describe_table") - - + assertResult(Array(Array("key", "int"), Array("value", "string"))) { javaHiveCtx - .sql("SELECT result FROM describe_table") + .sql(s"describe $tableName") .collect() - .map(_.getString(0).split("\t").map(_.trim)) + .map(row => Array(row.get(0).asInstanceOf[String], row.get(1).asInstanceOf[String])) .toArray } @@ -89,7 +80,7 @@ class JavaHiveQLSuite extends FunSuite { TestHive.reset() } - ignore("Exactly once semantics for DDL and command statements") { + test("Exactly once semantics for DDL and command statements") { val tableName = "test_exactly_once" val q0 = javaHiveCtx.sql(s"CREATE TABLE $tableName(key INT, value STRING)") From 34b97a067d1b370fbed8ecafab2f48501a35d783 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 5 Oct 2014 17:51:59 -0700 Subject: [PATCH 0900/1492] [SPARK-3645][SQL] Makes table caching eager by default and adds syntax for lazy caching Although lazy caching for in-memory table seems consistent with the `RDD.cache()` API, it's relatively confusing for users who mainly work with SQL and not familiar with Spark internals. The `CACHE TABLE t; SELECT COUNT(*) FROM t;` pattern is also commonly seen just to ensure predictable performance. This PR makes both the `CACHE TABLE t [AS SELECT ...]` statement and the `SQLContext.cacheTable()` API eager by default, and adds a new `CACHE LAZY TABLE t [AS SELECT ...]` syntax to provide lazy in-memory table caching. Also, took the chance to make some refactoring: `CacheCommand` and `CacheTableAsSelectCommand` are now merged and renamed to `CacheTableCommand` since the former is strictly a special case of the latter. A new `UncacheTableCommand` is added for the `UNCACHE TABLE t` statement. Author: Cheng Lian Closes #2513 from liancheng/eager-caching and squashes the following commits: fe92287 [Cheng Lian] Makes table caching eager by default and adds syntax for lazy caching --- .../apache/spark/sql/catalyst/SqlParser.scala | 45 +++--- .../spark/sql/catalyst/analysis/Catalog.scala | 2 +- .../sql/catalyst/plans/logical/commands.scala | 15 +- .../org/apache/spark/sql/CacheManager.scala | 9 +- .../columnar/InMemoryColumnarTableScan.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 8 +- .../apache/spark/sql/execution/commands.scala | 47 +++--- .../apache/spark/sql/CachedTableSuite.scala | 145 +++++++++++++----- .../spark/sql/hive/ExtendedHiveQlParser.scala | 66 ++++---- .../org/apache/spark/sql/hive/TestHive.scala | 6 +- .../spark/sql/hive/CachedTableSuite.scala | 78 ++++++++-- 11 files changed, 265 insertions(+), 158 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 26336332c05a2..854b5b461bdc8 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -67,11 +67,12 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected implicit def asParser(k: Keyword): Parser[String] = lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) + protected val ABS = Keyword("ABS") protected val ALL = Keyword("ALL") protected val AND = Keyword("AND") + protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AS = Keyword("AS") protected val ASC = Keyword("ASC") - protected val APPROXIMATE = Keyword("APPROXIMATE") protected val AVG = Keyword("AVG") protected val BETWEEN = Keyword("BETWEEN") protected val BY = Keyword("BY") @@ -80,9 +81,9 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val COUNT = Keyword("COUNT") protected val DESC = Keyword("DESC") protected val DISTINCT = Keyword("DISTINCT") + protected val EXCEPT = Keyword("EXCEPT") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") - protected val LAST = Keyword("LAST") protected val FROM = Keyword("FROM") protected val FULL = Keyword("FULL") protected val GROUP = Keyword("GROUP") @@ -91,42 +92,42 @@ class SqlParser extends StandardTokenParsers with PackratParsers { protected val IN = Keyword("IN") protected val INNER = Keyword("INNER") protected val INSERT = Keyword("INSERT") + protected val INTERSECT = Keyword("INTERSECT") protected val INTO = Keyword("INTO") protected val IS = Keyword("IS") protected val JOIN = Keyword("JOIN") + protected val LAST = Keyword("LAST") + protected val LAZY = Keyword("LAZY") protected val LEFT = Keyword("LEFT") + protected val LIKE = Keyword("LIKE") protected val LIMIT = Keyword("LIMIT") + protected val LOWER = Keyword("LOWER") protected val MAX = Keyword("MAX") protected val MIN = Keyword("MIN") protected val NOT = Keyword("NOT") protected val NULL = Keyword("NULL") protected val ON = Keyword("ON") protected val OR = Keyword("OR") - protected val OVERWRITE = Keyword("OVERWRITE") - protected val LIKE = Keyword("LIKE") - protected val RLIKE = Keyword("RLIKE") - protected val UPPER = Keyword("UPPER") - protected val LOWER = Keyword("LOWER") - protected val REGEXP = Keyword("REGEXP") protected val ORDER = Keyword("ORDER") protected val OUTER = Keyword("OUTER") + protected val OVERWRITE = Keyword("OVERWRITE") + protected val REGEXP = Keyword("REGEXP") protected val RIGHT = Keyword("RIGHT") + protected val RLIKE = Keyword("RLIKE") protected val SELECT = Keyword("SELECT") protected val SEMI = Keyword("SEMI") + protected val SQRT = Keyword("SQRT") protected val STRING = Keyword("STRING") + protected val SUBSTR = Keyword("SUBSTR") + protected val SUBSTRING = Keyword("SUBSTRING") protected val SUM = Keyword("SUM") protected val TABLE = Keyword("TABLE") protected val TIMESTAMP = Keyword("TIMESTAMP") protected val TRUE = Keyword("TRUE") protected val UNCACHE = Keyword("UNCACHE") protected val UNION = Keyword("UNION") + protected val UPPER = Keyword("UPPER") protected val WHERE = Keyword("WHERE") - protected val INTERSECT = Keyword("INTERSECT") - protected val EXCEPT = Keyword("EXCEPT") - protected val SUBSTR = Keyword("SUBSTR") - protected val SUBSTRING = Keyword("SUBSTRING") - protected val SQRT = Keyword("SQRT") - protected val ABS = Keyword("ABS") // Use reflection to find the reserved words defined in this class. protected val reservedWords = @@ -183,17 +184,15 @@ class SqlParser extends StandardTokenParsers with PackratParsers { } protected lazy val cache: Parser[LogicalPlan] = - CACHE ~ TABLE ~> ident ~ opt(AS ~> select) <~ opt(";") ^^ { - case tableName ~ None => - CacheCommand(tableName, true) - case tableName ~ Some(plan) => - CacheTableAsSelectCommand(tableName, plan) + CACHE ~> opt(LAZY) ~ (TABLE ~> ident) ~ opt(AS ~> select) <~ opt(";") ^^ { + case isLazy ~ tableName ~ plan => + CacheTableCommand(tableName, plan, isLazy.isDefined) } - + protected lazy val unCache: Parser[LogicalPlan] = UNCACHE ~ TABLE ~> ident <~ opt(";") ^^ { - case tableName => CacheCommand(tableName, false) - } + case tableName => UncacheTableCommand(tableName) + } protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",") @@ -283,7 +282,7 @@ class SqlParser extends StandardTokenParsers with PackratParsers { termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } | - termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { + termExpression ~ BETWEEN ~ termExpression ~ AND ~ termExpression ^^ { case e ~ _ ~ el ~ _ ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) } | termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | 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 616f1e2ecb60f..2059a91ba0612 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 @@ -87,7 +87,7 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { tableName: String, alias: Option[String] = None): LogicalPlan = { val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) - val table = tables.get(tblName).getOrElse(sys.error(s"Table Not Found: $tableName")) + val table = tables.getOrElse(tblName, sys.error(s"Table Not Found: $tableName")) val tableWithQualifiers = Subquery(tblName, table) // If an alias was specified by the lookup, wrap the plan in a subquery so that attributes are diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index 8366639fa0e8b..9a3848cfc6b62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -56,9 +56,15 @@ case class ExplainCommand(plan: LogicalPlan, extended: Boolean = false) extends } /** - * Returned for the "CACHE TABLE tableName" and "UNCACHE TABLE tableName" command. + * Returned for the "CACHE TABLE tableName [AS SELECT ...]" command. */ -case class CacheCommand(tableName: String, doCache: Boolean) extends Command +case class CacheTableCommand(tableName: String, plan: Option[LogicalPlan], isLazy: Boolean) + extends Command + +/** + * Returned for the "UNCACHE TABLE tableName" command. + */ +case class UncacheTableCommand(tableName: String) extends Command /** * Returned for the "DESCRIBE [EXTENDED] [dbName.]tableName" command. @@ -75,8 +81,3 @@ case class DescribeCommand( AttributeReference("data_type", StringType, nullable = false)(), AttributeReference("comment", StringType, nullable = false)()) } - -/** - * Returned for the "CACHE TABLE tableName AS SELECT .." command. - */ -case class CacheTableAsSelectCommand(tableName: String, plan: LogicalPlan) extends Command diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index aebdbb68e49b8..3bf7382ac67a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -91,14 +91,10 @@ private[sql] trait CacheManager { } /** Removes the data for the given SchemaRDD from the cache */ - private[sql] def uncacheQuery(query: SchemaRDD, blocking: Boolean = false): Unit = writeLock { + private[sql] def uncacheQuery(query: SchemaRDD, blocking: Boolean = true): Unit = writeLock { val planToCache = query.queryExecution.optimizedPlan val dataIndex = cachedData.indexWhere(_.plan.sameResult(planToCache)) - - if (dataIndex < 0) { - throw new IllegalArgumentException(s"Table $query is not cached.") - } - + require(dataIndex >= 0, s"Table $query is not cached.") cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) cachedData.remove(dataIndex) } @@ -135,5 +131,4 @@ private[sql] trait CacheManager { case _ => } } - } 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 cec82a7f2df94..4f79173a26f88 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 @@ -111,7 +111,7 @@ private[sql] case class InMemoryRelation( override def newInstance() = { new InMemoryRelation( - output.map(_.newInstance), + output.map(_.newInstance()), useCompression, batchSize, storageLevel, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index cf93d5ad7b503..5c16d0c624128 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -304,10 +304,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { Seq(execution.SetCommand(key, value, plan.output)(context)) case logical.ExplainCommand(logicalPlan, extended) => Seq(execution.ExplainCommand(logicalPlan, plan.output, extended)(context)) - case logical.CacheCommand(tableName, cache) => - Seq(execution.CacheCommand(tableName, cache)(context)) - case logical.CacheTableAsSelectCommand(tableName, plan) => - Seq(execution.CacheTableAsSelectCommand(tableName, plan)) + case logical.CacheTableCommand(tableName, optPlan, isLazy) => + Seq(execution.CacheTableCommand(tableName, optPlan, isLazy)) + case logical.UncacheTableCommand(tableName) => + Seq(execution.UncacheTableCommand(tableName)) case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index f88099ec0761e..d49633c24ad4d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -138,49 +138,54 @@ case class ExplainCommand( * :: DeveloperApi :: */ @DeveloperApi -case class CacheCommand(tableName: String, doCache: Boolean)(@transient context: SQLContext) +case class CacheTableCommand( + tableName: String, + plan: Option[LogicalPlan], + isLazy: Boolean) extends LeafNode with Command { override protected lazy val sideEffectResult = { - if (doCache) { - context.cacheTable(tableName) - } else { - context.uncacheTable(tableName) + import sqlContext._ + + plan.foreach(_.registerTempTable(tableName)) + val schemaRDD = table(tableName) + schemaRDD.cache() + + if (!isLazy) { + // Performs eager caching + schemaRDD.count() } + Seq.empty[Row] } override def output: Seq[Attribute] = Seq.empty } + /** * :: DeveloperApi :: */ @DeveloperApi -case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( - @transient context: SQLContext) - extends LeafNode with Command { - +case class UncacheTableCommand(tableName: String) extends LeafNode with Command { override protected lazy val sideEffectResult: Seq[Row] = { - Row("# Registered as a temporary table", null, null) +: - child.output.map(field => Row(field.name, field.dataType.toString, null)) + sqlContext.table(tableName).unpersist() + Seq.empty[Row] } + + override def output: Seq[Attribute] = Seq.empty } /** * :: DeveloperApi :: */ @DeveloperApi -case class CacheTableAsSelectCommand(tableName: String, logicalPlan: LogicalPlan) +case class DescribeCommand(child: SparkPlan, output: Seq[Attribute])( + @transient context: SQLContext) extends LeafNode with Command { - - override protected[sql] lazy val sideEffectResult = { - import sqlContext._ - logicalPlan.registerTempTable(tableName) - cacheTable(tableName) - Seq.empty[Row] - } - override def output: Seq[Attribute] = Seq.empty - + override protected lazy val sideEffectResult: Seq[Row] = { + Row("# Registered as a temporary table", null, null) +: + child.output.map(field => Row(field.name, field.dataType.toString, null)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 957388e99bd85..1e624f97004f5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -18,30 +18,39 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} -import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.storage.RDDBlockId case class BigData(s: String) class CachedTableSuite extends QueryTest { - import TestSQLContext._ TestData // Load test tables. - /** - * Throws a test failed exception when the number of cached tables differs from the expected - * number. - */ def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = { val planWithCaching = query.queryExecution.withCachedData val cachedData = planWithCaching collect { case cached: InMemoryRelation => cached } - if (cachedData.size != numCachedTables) { - fail( - s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + assert( + cachedData.size == numCachedTables, + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + planWithCaching) - } + } + + def rddIdOf(tableName: String): Int = { + val executedPlan = table(tableName).queryExecution.executedPlan + executedPlan.collect { + case InMemoryColumnarTableScan(_, _, relation) => + relation.cachedColumnBuffers.id + case _ => + fail(s"Table $tableName is not cached\n" + executedPlan) + }.head + } + + def isMaterialized(rddId: Int): Boolean = { + sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty } test("too big for memory") { @@ -52,10 +61,33 @@ class CachedTableSuite extends QueryTest { uncacheTable("bigData") } - test("calling .cache() should use inmemory columnar caching") { + test("calling .cache() should use in-memory columnar caching") { table("testData").cache() + assertCached(table("testData")) + } + + test("calling .unpersist() should drop in-memory columnar cache") { + table("testData").cache() + table("testData").count() + table("testData").unpersist(true) + assertCached(table("testData"), 0) + } + + test("isCached") { + cacheTable("testData") assertCached(table("testData")) + assert(table("testData").queryExecution.withCachedData match { + case _: InMemoryRelation => true + case _ => false + }) + + uncacheTable("testData") + assert(!isCached("testData")) + assert(table("testData").queryExecution.withCachedData match { + case _: InMemoryRelation => false + case _ => true + }) } test("SPARK-1669: cacheTable should be idempotent") { @@ -64,32 +96,27 @@ class CachedTableSuite extends QueryTest { cacheTable("testData") assertCached(table("testData")) - cacheTable("testData") - table("testData").queryExecution.analyzed match { - case InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan) => - fail("cacheTable is not idempotent") + assertResult(1, "InMemoryRelation not found, testData should have been cached") { + table("testData").queryExecution.withCachedData.collect { + case r: InMemoryRelation => r + }.size + } - case _ => + cacheTable("testData") + assertResult(0, "Double InMemoryRelations found, cacheTable() is not idempotent") { + table("testData").queryExecution.withCachedData.collect { + case r @ InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan) => r + }.size } } test("read from cached table and uncache") { cacheTable("testData") - - checkAnswer( - table("testData"), - testData.collect().toSeq - ) - + checkAnswer(table("testData"), testData.collect().toSeq) assertCached(table("testData")) uncacheTable("testData") - - checkAnswer( - table("testData"), - testData.collect().toSeq - ) - + checkAnswer(table("testData"), testData.collect().toSeq) assertCached(table("testData"), 0) } @@ -99,10 +126,12 @@ class CachedTableSuite extends QueryTest { } } - test("SELECT Star Cached Table") { + test("SELECT star from cached table") { sql("SELECT * FROM testData").registerTempTable("selectStar") cacheTable("selectStar") - sql("SELECT * FROM selectStar WHERE key = 1").collect() + checkAnswer( + sql("SELECT * FROM selectStar WHERE key = 1"), + Seq(Row(1, "1"))) uncacheTable("selectStar") } @@ -120,23 +149,57 @@ class CachedTableSuite extends QueryTest { sql("CACHE TABLE testData") assertCached(table("testData")) - assert(isCached("testData"), "Table 'testData' should be cached") + val rddId = rddIdOf("testData") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") sql("UNCACHE TABLE testData") - assertCached(table("testData"), 0) assert(!isCached("testData"), "Table 'testData' should not be cached") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } - - test("CACHE TABLE tableName AS SELECT Star Table") { + + test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") - sql("SELECT * FROM testCacheTable WHERE key = 1").collect() - assert(isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + assertCached(table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } - - test("'CACHE TABLE tableName AS SELECT ..'") { - sql("CACHE TABLE testCacheTable AS SELECT * FROM testData") - assert(isCached("testCacheTable"), "Table 'testCacheTable' should be cached") + + test("CACHE TABLE tableName AS SELECT ...") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM testData LIMIT 10") + assertCached(table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } + + test("CACHE LAZY TABLE tableName") { + sql("CACHE LAZY TABLE testData") + assertCached(table("testData")) + + val rddId = rddIdOf("testData") + assert( + !isMaterialized(rddId), + "Lazily cached in-memory table shouldn't be materialized eagerly") + + sql("SELECT COUNT(*) FROM testData").collect() + assert( + isMaterialized(rddId), + "Lazily cached in-memory table should have been materialized") + + uncacheTable("testData") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala index e7e1cb980c2ae..c5844e92eaaa9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ExtendedHiveQlParser.scala @@ -24,11 +24,11 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.SqlLexical /** - * A parser that recognizes all HiveQL constructs together with several Spark SQL specific + * A parser that recognizes all HiveQL constructs together with several Spark SQL specific * extensions like CACHE TABLE and UNCACHE TABLE. */ -private[hive] class ExtendedHiveQlParser extends StandardTokenParsers with PackratParsers { - +private[hive] class ExtendedHiveQlParser extends StandardTokenParsers with PackratParsers { + def apply(input: String): LogicalPlan = { // Special-case out set commands since the value fields can be // complex to handle without RegexParsers. Also this approach @@ -54,16 +54,17 @@ private[hive] class ExtendedHiveQlParser extends StandardTokenParsers with Packr protected case class Keyword(str: String) - protected val CACHE = Keyword("CACHE") - protected val SET = Keyword("SET") protected val ADD = Keyword("ADD") - protected val JAR = Keyword("JAR") - protected val TABLE = Keyword("TABLE") protected val AS = Keyword("AS") - protected val UNCACHE = Keyword("UNCACHE") - protected val FILE = Keyword("FILE") + protected val CACHE = Keyword("CACHE") protected val DFS = Keyword("DFS") + protected val FILE = Keyword("FILE") + protected val JAR = Keyword("JAR") + protected val LAZY = Keyword("LAZY") + protected val SET = Keyword("SET") protected val SOURCE = Keyword("SOURCE") + protected val TABLE = Keyword("TABLE") + protected val UNCACHE = Keyword("UNCACHE") protected implicit def asParser(k: Keyword): Parser[String] = lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) @@ -79,57 +80,56 @@ private[hive] class ExtendedHiveQlParser extends StandardTokenParsers with Packr override val lexical = new SqlLexical(reservedWords) - protected lazy val query: Parser[LogicalPlan] = + protected lazy val query: Parser[LogicalPlan] = cache | uncache | addJar | addFile | dfs | source | hiveQl protected lazy val hiveQl: Parser[LogicalPlan] = - remainingQuery ^^ { - case r => HiveQl.createPlan(r.trim()) + restInput ^^ { + case statement => HiveQl.createPlan(statement.trim()) } - /** It returns all remaining query */ - protected lazy val remainingQuery: Parser[String] = new Parser[String] { + // Returns the whole input string + protected lazy val wholeInput: Parser[String] = new Parser[String] { def apply(in: Input) = - Success( - in.source.subSequence(in.offset, in.source.length).toString, - in.drop(in.source.length())) + Success(in.source.toString, in.drop(in.source.length())) } - /** It returns all query */ - protected lazy val allQuery: Parser[String] = new Parser[String] { + // Returns the rest of the input string that are not parsed yet + protected lazy val restInput: Parser[String] = new Parser[String] { def apply(in: Input) = - Success(in.source.toString, in.drop(in.source.length())) + Success( + in.source.subSequence(in.offset, in.source.length).toString, + in.drop(in.source.length())) } protected lazy val cache: Parser[LogicalPlan] = - CACHE ~ TABLE ~> ident ~ opt(AS ~> hiveQl) ^^ { - case tableName ~ None => CacheCommand(tableName, true) - case tableName ~ Some(plan) => - CacheTableAsSelectCommand(tableName, plan) + CACHE ~> opt(LAZY) ~ (TABLE ~> ident) ~ opt(AS ~> hiveQl) ^^ { + case isLazy ~ tableName ~ plan => + CacheTableCommand(tableName, plan, isLazy.isDefined) } protected lazy val uncache: Parser[LogicalPlan] = UNCACHE ~ TABLE ~> ident ^^ { - case tableName => CacheCommand(tableName, false) + case tableName => UncacheTableCommand(tableName) } protected lazy val addJar: Parser[LogicalPlan] = - ADD ~ JAR ~> remainingQuery ^^ { - case rq => AddJar(rq.trim()) + ADD ~ JAR ~> restInput ^^ { + case jar => AddJar(jar.trim()) } protected lazy val addFile: Parser[LogicalPlan] = - ADD ~ FILE ~> remainingQuery ^^ { - case rq => AddFile(rq.trim()) + ADD ~ FILE ~> restInput ^^ { + case file => AddFile(file.trim()) } protected lazy val dfs: Parser[LogicalPlan] = - DFS ~> allQuery ^^ { - case aq => NativeCommand(aq.trim()) + DFS ~> wholeInput ^^ { + case command => NativeCommand(command.trim()) } protected lazy val source: Parser[LogicalPlan] = - SOURCE ~> remainingQuery ^^ { - case rq => SourceCommand(rq.trim()) + SOURCE ~> restInput ^^ { + case file => SourceCommand(file.trim()) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index c0e69393cc2e3..a4354c1379c63 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -32,7 +32,7 @@ import org.apache.hadoop.hive.serde2.avro.AvroSerDe import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.plans.logical.{CacheCommand, LogicalPlan, NativeCommand} +import org.apache.spark.sql.catalyst.plans.logical.{CacheTableCommand, LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.hive._ import org.apache.spark.sql.SQLConf @@ -67,7 +67,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath /** Sets up the system initially or after a RESET command */ - protected def configure() { + protected def configure(): Unit = { setConf("javax.jdo.option.ConnectionURL", s"jdbc:derby:;databaseName=$metastorePath;create=true") setConf("hive.metastore.warehouse.dir", warehousePath) @@ -154,7 +154,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { override lazy val analyzed = { val describedTables = logical match { case NativeCommand(describedTable(tbl)) => tbl :: Nil - case CacheCommand(tbl, _) => tbl :: Nil + case CacheTableCommand(tbl, _, _) => tbl :: Nil case _ => Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 158cfb5bbee7c..2060e1f1a7a4b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.{QueryTest, SchemaRDD} -import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} +import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.sql.{QueryTest, SchemaRDD} +import org.apache.spark.storage.RDDBlockId class CachedTableSuite extends QueryTest { - import TestHive._ - /** * Throws a test failed exception when the number of cached tables differs from the expected * number. @@ -34,11 +34,24 @@ class CachedTableSuite extends QueryTest { case cached: InMemoryRelation => cached } - if (cachedData.size != numCachedTables) { - fail( - s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + - planWithCaching) - } + assert( + cachedData.size == numCachedTables, + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } + + def rddIdOf(tableName: String): Int = { + val executedPlan = table(tableName).queryExecution.executedPlan + executedPlan.collect { + case InMemoryColumnarTableScan(_, _, relation) => + relation.cachedColumnBuffers.id + case _ => + fail(s"Table $tableName is not cached\n" + executedPlan) + }.head + } + + def isMaterialized(rddId: Int): Boolean = { + sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty } test("cache table") { @@ -102,16 +115,47 @@ class CachedTableSuite extends QueryTest { assert(!TestHive.isCached("src"), "Table 'src' should not be cached") } - test("CACHE TABLE AS SELECT") { - assertCached(sql("SELECT * FROM src"), 0) - sql("CACHE TABLE test AS SELECT key FROM src") + test("CACHE TABLE tableName AS SELECT * FROM anotherTable") { + sql("CACHE TABLE testCacheTable AS SELECT * FROM src") + assertCached(table("testCacheTable")) - checkAnswer( - sql("SELECT * FROM test"), - sql("SELECT key FROM src").collect().toSeq) + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") - assertCached(sql("SELECT * FROM test")) + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } + + test("CACHE TABLE tableName AS SELECT ...") { + sql("CACHE TABLE testCacheTable AS SELECT key FROM src LIMIT 10") + assertCached(table("testCacheTable")) + + val rddId = rddIdOf("testCacheTable") + assert( + isMaterialized(rddId), + "Eagerly cached in-memory table should have already been materialized") + + uncacheTable("testCacheTable") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") + } - assertCached(sql("SELECT * FROM test JOIN test"), 2) + test("CACHE LAZY TABLE tableName") { + sql("CACHE LAZY TABLE src") + assertCached(table("src")) + + val rddId = rddIdOf("src") + assert( + !isMaterialized(rddId), + "Lazily cached in-memory table shouldn't be materialized eagerly") + + sql("SELECT COUNT(*) FROM src").collect() + assert( + isMaterialized(rddId), + "Lazily cached in-memory table should have been materialized") + + uncacheTable("src") + assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } } From 90897ea5f24b03c9f3455a62c7f68b3d3f0435ad Mon Sep 17 00:00:00 2001 From: Renat Yusupov Date: Sun, 5 Oct 2014 17:56:24 -0700 Subject: [PATCH 0901/1492] [SPARK-3776][SQL] Wrong conversion to Catalyst for Option[Product] Author: Renat Yusupov Closes #2641 from r3natko/feature/catalyst_option and squashes the following commits: 55d0c06 [Renat Yusupov] [SQL] SPARK-3776: Wrong conversion to Catalyst for Option[Product] --- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- .../sql/catalyst/ScalaReflectionSuite.scala | 21 ++++++++++++++++--- 2 files changed, 19 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 88a8fa7c28e0f..b3ae8e6779700 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -33,7 +33,7 @@ object ScalaReflection { /** Converts Scala objects to catalyst rows / types */ def convertToCatalyst(a: Any): Any = a match { - case o: Option[_] => o.orNull + case o: Option[_] => o.map(convertToCatalyst).orNull case s: Seq[_] => s.map(convertToCatalyst) case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 428607d8c8253..488e373854bb3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -53,7 +53,8 @@ case class OptionalData( floatField: Option[Float], shortField: Option[Short], byteField: Option[Byte], - booleanField: Option[Boolean]) + booleanField: Option[Boolean], + structField: Option[PrimitiveData]) case class ComplexData( arrayField: Seq[Int], @@ -100,7 +101,7 @@ class ScalaReflectionSuite extends FunSuite { nullable = true)) } - test("optinal data") { + test("optional data") { val schema = schemaFor[OptionalData] assert(schema === Schema( StructType(Seq( @@ -110,7 +111,8 @@ class ScalaReflectionSuite extends FunSuite { StructField("floatField", FloatType, nullable = true), StructField("shortField", ShortType, nullable = true), StructField("byteField", ByteType, nullable = true), - StructField("booleanField", BooleanType, nullable = true))), + StructField("booleanField", BooleanType, nullable = true), + StructField("structField", schemaFor[PrimitiveData].dataType, nullable = true))), nullable = true)) } @@ -228,4 +230,17 @@ class ScalaReflectionSuite extends FunSuite { assert(ArrayType(IntegerType) === typeOfObject3(Seq(1, 2, 3))) assert(ArrayType(ArrayType(IntegerType)) === typeOfObject3(Seq(Seq(1,2,3)))) } + + test("convert PrimitiveData to catalyst") { + val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) + val convertedData = Seq(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true) + assert(convertToCatalyst(data) === convertedData) + } + + test("convert Option[Product] to catalyst") { + val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true) + val data = OptionalData(Some(1), Some(1), Some(1), Some(1), Some(1), Some(1), Some(true), Some(primitiveData)) + val convertedData = Seq(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true, convertToCatalyst(primitiveData)) + assert(convertToCatalyst(data) === convertedData) + } } From 8d22dbb5ec7a0727afdfebbbc2c57ffdb384dd0b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 5 Oct 2014 18:44:12 -0700 Subject: [PATCH 0902/1492] SPARK-3794 [CORE] Building spark core fails due to inadvertent dependency on Commons IO Remove references to Commons IO FileUtils and replace with pure Java version, which doesn't need to traverse the whole directory tree first. I think this method could be refined further if it would be alright to rename it and its args and break it down into two methods. I'm starting with a simple recursive rendition. Author: Sean Owen Closes #2662 from srowen/SPARK-3794 and squashes the following commits: 4cd172f [Sean Owen] Remove references to Commons IO FileUtils and replace with pure Java version, which doesn't need to traverse the whole directory tree first --- .../apache/spark/deploy/worker/Worker.scala | 1 - .../scala/org/apache/spark/util/Utils.scala | 20 +++++++++---------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 3b13f43a1868c..9b52cb06fb6fa 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -29,7 +29,6 @@ import scala.language.postfixOps import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.commons.io.FileUtils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} 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 a67124140f9da..3d307b3c16d3e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -35,8 +35,6 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.commons.io.FileUtils -import org.apache.commons.io.filefilter.TrueFileFilter import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.log4j.PropertyConfigurator @@ -710,18 +708,20 @@ private[spark] object Utils extends Logging { * Determines if a directory contains any files newer than cutoff seconds. * * @param dir must be the path to a directory, or IllegalArgumentException is thrown - * @param cutoff measured in seconds. Returns true if there are any files in dir newer than this. + * @param cutoff measured in seconds. Returns true if there are any files or directories in the + * given directory whose last modified time is later than this many seconds ago */ def doesDirectoryContainAnyNewFiles(dir: File, cutoff: Long): Boolean = { - val currentTimeMillis = System.currentTimeMillis if (!dir.isDirectory) { - throw new IllegalArgumentException (dir + " is not a directory!") - } else { - val files = FileUtils.listFilesAndDirs(dir, TrueFileFilter.TRUE, TrueFileFilter.TRUE) - val cutoffTimeInMillis = (currentTimeMillis - (cutoff * 1000)) - val newFiles = files.filter { _.lastModified > cutoffTimeInMillis } - newFiles.nonEmpty + throw new IllegalArgumentException("$dir is not a directory!") } + val filesAndDirs = dir.listFiles() + val cutoffTimeInMillis = System.currentTimeMillis - (cutoff * 1000) + + filesAndDirs.exists(_.lastModified() > cutoffTimeInMillis) || + filesAndDirs.filter(_.isDirectory).exists( + subdir => doesDirectoryContainAnyNewFiles(subdir, cutoff) + ) } /** From fd7b15539669b14996a51610d6724ca0811f9d65 Mon Sep 17 00:00:00 2001 From: Nathan Kronenfeld Date: Sun, 5 Oct 2014 21:03:48 -0700 Subject: [PATCH 0903/1492] Rectify gereneric parameter names between SparkContext and AccumulablePa... AccumulableParam gave its generic parameters as 'R, T', whereas SparkContext labeled them 'T, R'. Trivial, but really confusing. I resolved this in favor of AccumulableParam, because it seemed to have some logic for its names. I also extended this minimal, but at least present, justification into the SparkContext comments. Author: Nathan Kronenfeld Closes #2637 from nkronenfeld/accumulators and squashes the following commits: 98d6b74 [Nathan Kronenfeld] Rectify gereneric parameter names between SparkContext and AccumulableParam --- .../main/scala/org/apache/spark/SparkContext.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 97109b9f41b60..396cdd1247e07 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -779,20 +779,20 @@ class SparkContext(config: SparkConf) extends Logging { /** * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values * with `+=`. Only the driver can access the accumuable's `value`. - * @tparam T accumulator type - * @tparam R type that can be added to the accumulator + * @tparam R accumulator result type + * @tparam T type that can be added to the accumulator */ - def accumulable[T, R](initialValue: T)(implicit param: AccumulableParam[T, R]) = + def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) = new Accumulable(initialValue, param) /** * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the * Spark UI. Tasks can add values to the accumuable using the `+=` operator. Only the driver can * access the accumuable's `value`. - * @tparam T accumulator type - * @tparam R type that can be added to the accumulator + * @tparam R accumulator result type + * @tparam T type that can be added to the accumulator */ - def accumulable[T, R](initialValue: T, name: String)(implicit param: AccumulableParam[T, R]) = + def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) = new Accumulable(initialValue, param, Some(name)) /** From c9ae79fba25cd49ca70ca398bc75434202d26a97 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 5 Oct 2014 21:36:20 -0700 Subject: [PATCH 0904/1492] [SPARK-3765][Doc] Add test information to sbt build docs Add testing with sbt to doc ```building-spark.md``` Author: scwf Closes #2629 from scwf/sbt-doc and squashes the following commits: fd9cf29 [scwf] add testing with sbt to docs --- docs/building-spark.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/docs/building-spark.md b/docs/building-spark.md index 901c157162fee..b2940ee4029e8 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -171,6 +171,21 @@ can be set to control the SBT build. For example: sbt/sbt -Pyarn -Phadoop-2.3 assembly +# Testing with SBT + +Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: + + sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly + sbt/sbt -Pyarn -Phadoop-2.3 -Phive test + +To run only a specific test suite as follows: + + sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite" + +To run test suites of a specific sub project as follows: + + sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test + # Speeding up Compilation with Zinc [Zinc](https://github.com/typesafehub/zinc) is a long-running server version of SBT's incremental From 20ea54cc7a5176ebc63bfa9393a9bf84619bfc66 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 6 Oct 2014 14:05:45 -0700 Subject: [PATCH 0905/1492] [SPARK-2461] [PySpark] Add a toString method to GeneralizedLinearModel Add a toString method to GeneralizedLinearModel, also change `__str__` to `__repr__` for some classes, to provide better message in repr. This PR is based on #1388, thanks to sryza! closes #1388 Author: Sandy Ryza Author: Davies Liu Closes #2625 from davies/string and squashes the following commits: 3544aad [Davies Liu] fix LinearModel 0bcd642 [Davies Liu] Merge branch 'sandy-spark-2461' of github.com:sryza/spark 1ce5c2d [Sandy Ryza] __repr__ back to __str__ in a couple places aa9e962 [Sandy Ryza] Switch __str__ to __repr__ a0c5041 [Sandy Ryza] Add labels back in 1aa17f5 [Sandy Ryza] Match existing conventions fac1bc4 [Sandy Ryza] Fix PEP8 error f7b58ed [Sandy Ryza] SPARK-2461. Add a toString method to GeneralizedLinearModel --- .../spark/mllib/regression/GeneralizedLinearAlgorithm.scala | 2 ++ python/pyspark/mllib/regression.py | 3 +++ python/pyspark/serializers.py | 6 +++--- python/pyspark/sql.py | 2 +- 4 files changed, 9 insertions(+), 4 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 d0fe4179685ca..00dfc86c9e0bd 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 @@ -75,6 +75,8 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double def predict(testData: Vector): Double = { predictPoint(testData, weights, intercept) } + + override def toString() = "(weights=%s, intercept=%s)".format(weights, intercept) } /** diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index cbdbc09858013..8fe8c6db2ad9c 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -66,6 +66,9 @@ def weights(self): def intercept(self): return self._intercept + def __repr__(self): + return "(weights=%s, intercept=%s)" % (self._coeff, self._intercept) + class LinearRegressionModelBase(LinearModel): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 2672da36c1f50..099fa54cf2bd7 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -211,7 +211,7 @@ def __eq__(self, other): return (isinstance(other, BatchedSerializer) and other.serializer == self.serializer) - def __str__(self): + def __repr__(self): return "BatchedSerializer<%s>" % str(self.serializer) @@ -279,7 +279,7 @@ def __eq__(self, other): return (isinstance(other, CartesianDeserializer) and self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __str__(self): + def __repr__(self): return "CartesianDeserializer<%s, %s>" % \ (str(self.key_ser), str(self.val_ser)) @@ -306,7 +306,7 @@ def __eq__(self, other): return (isinstance(other, PairDeserializer) and self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __str__(self): + def __repr__(self): return "PairDeserializer<%s, %s>" % (str(self.key_ser), str(self.val_ser)) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 974b5e287bc00..114644ab8b79d 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -201,7 +201,7 @@ def __init__(self, elementType, containsNull=True): self.elementType = elementType self.containsNull = containsNull - def __str__(self): + def __repr__(self): return "ArrayType(%s,%s)" % (self.elementType, str(self.containsNull).lower()) From 4f01265f7d62e070ba42c251255e385644c1b16c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 6 Oct 2014 14:07:53 -0700 Subject: [PATCH 0906/1492] [SPARK-3786] [PySpark] speedup tests This patch try to speed up tests of PySpark, re-use the SparkContext in tests.py and mllib/tests.py to reduce the overhead of create SparkContext, remove some test cases, which did not make sense. It also improve the performance of some cases, such as MergerTests and SortTests. before this patch: real 21m27.320s user 4m42.967s sys 0m17.343s after this patch: real 9m47.541s user 2m12.947s sys 0m14.543s It almost cut the time by half. Author: Davies Liu Closes #2646 from davies/tests and squashes the following commits: c54de60 [Davies Liu] revert change about memory limit 6a2a4b0 [Davies Liu] refactor of tests, speedup 100% --- python/pyspark/mllib/tests.py | 2 +- python/pyspark/shuffle.py | 5 +- python/pyspark/tests.py | 92 ++++++++++++++++------------------- python/run-tests | 74 ++++++++++++++-------------- 4 files changed, 82 insertions(+), 91 deletions(-) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index f72e88ba6e2ba..5c20e100e144f 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -32,7 +32,7 @@ from pyspark.serializers import PickleSerializer from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint -from pyspark.tests import PySparkTestCase +from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase _have_scipy = False diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index ce597cbe91e15..d57a802e4734a 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -396,7 +396,6 @@ def _external_items(self): for v in self.data.iteritems(): yield v self.data.clear() - gc.collect() # remove the merged partition for j in range(self.spills): @@ -428,7 +427,7 @@ def _recursive_merged_items(self, start): subdirs = [os.path.join(d, "parts", str(i)) for d in self.localdirs] m = ExternalMerger(self.agg, self.memory_limit, self.serializer, - subdirs, self.scale * self.partitions) + subdirs, self.scale * self.partitions, self.partitions) m.pdata = [{} for _ in range(self.partitions)] limit = self._next_limit() @@ -486,7 +485,7 @@ def sorted(self, iterator, key=None, reverse=False): goes above the limit. """ global MemoryBytesSpilled, DiskBytesSpilled - batch = 10 + batch = 100 chunks, current_chunk = [], [] iterator = iter(iterator) while True: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 6fb6bc998c752..7f05d48ade2b3 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -67,10 +67,10 @@ SPARK_HOME = os.environ["SPARK_HOME"] -class TestMerger(unittest.TestCase): +class MergerTests(unittest.TestCase): def setUp(self): - self.N = 1 << 16 + self.N = 1 << 14 self.l = [i for i in xrange(self.N)] self.data = zip(self.l, self.l) self.agg = Aggregator(lambda x: [x], @@ -115,7 +115,7 @@ def test_medium_dataset(self): sum(xrange(self.N)) * 3) def test_huge_dataset(self): - m = ExternalMerger(self.agg, 10) + m = ExternalMerger(self.agg, 10, partitions=3) m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) self.assertTrue(m.spills >= 1) self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), @@ -123,7 +123,7 @@ def test_huge_dataset(self): m._cleanup() -class TestSorter(unittest.TestCase): +class SorterTests(unittest.TestCase): def test_in_memory_sort(self): l = range(1024) random.shuffle(l) @@ -244,16 +244,25 @@ def tearDown(self): sys.path = self._old_sys_path -class TestCheckpoint(PySparkTestCase): +class ReusedPySparkTestCase(unittest.TestCase): + + @classmethod + def setUpClass(cls): + cls.sc = SparkContext('local[4]', cls.__name__, batchSize=2) + + @classmethod + def tearDownClass(cls): + cls.sc.stop() + + +class CheckpointTests(ReusedPySparkTestCase): def setUp(self): - PySparkTestCase.setUp(self) self.checkpointDir = tempfile.NamedTemporaryFile(delete=False) os.unlink(self.checkpointDir.name) self.sc.setCheckpointDir(self.checkpointDir.name) def tearDown(self): - PySparkTestCase.tearDown(self) shutil.rmtree(self.checkpointDir.name) def test_basic_checkpointing(self): @@ -288,7 +297,7 @@ def test_checkpoint_and_restore(self): self.assertEquals([1, 2, 3, 4], recovered.collect()) -class TestAddFile(PySparkTestCase): +class AddFileTests(PySparkTestCase): def test_add_py_file(self): # To ensure that we're actually testing addPyFile's effects, check that @@ -354,7 +363,7 @@ def func(x): self.assertEqual(["My Server"], self.sc.parallelize(range(1)).map(func).collect()) -class TestRDDFunctions(PySparkTestCase): +class RDDTests(ReusedPySparkTestCase): def test_id(self): rdd = self.sc.parallelize(range(10)) @@ -365,12 +374,6 @@ def test_id(self): self.assertEqual(id + 1, id2) self.assertEqual(id2, rdd2.id()) - def test_failed_sparkcontext_creation(self): - # Regression test for SPARK-1550 - self.sc.stop() - self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name")) - self.sc = SparkContext("local") - def test_save_as_textfile_with_unicode(self): # Regression test for SPARK-970 x = u"\u00A1Hola, mundo!" @@ -636,7 +639,7 @@ def test_distinct(self): self.assertEquals(result.count(), 3) -class TestProfiler(PySparkTestCase): +class ProfilerTests(PySparkTestCase): def setUp(self): self._old_sys_path = list(sys.path) @@ -666,10 +669,9 @@ def heavy_foo(x): self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) -class TestSQL(PySparkTestCase): +class SQLTests(ReusedPySparkTestCase): def setUp(self): - PySparkTestCase.setUp(self) self.sqlCtx = SQLContext(self.sc) def test_udf(self): @@ -754,27 +756,19 @@ def test_serialize_nested_array_and_map(self): self.assertEqual("2", row.d) -class TestIO(PySparkTestCase): - - def test_stdout_redirection(self): - import subprocess - - def func(x): - subprocess.check_call('ls', shell=True) - self.sc.parallelize([1]).foreach(func) +class InputFormatTests(ReusedPySparkTestCase): + @classmethod + def setUpClass(cls): + ReusedPySparkTestCase.setUpClass() + cls.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(cls.tempdir.name) + cls.sc._jvm.WriteInputFormatTestDataGenerator.generateData(cls.tempdir.name, cls.sc._jsc) -class TestInputFormat(PySparkTestCase): - - def setUp(self): - PySparkTestCase.setUp(self) - self.tempdir = tempfile.NamedTemporaryFile(delete=False) - os.unlink(self.tempdir.name) - self.sc._jvm.WriteInputFormatTestDataGenerator.generateData(self.tempdir.name, self.sc._jsc) - - def tearDown(self): - PySparkTestCase.tearDown(self) - shutil.rmtree(self.tempdir.name) + @classmethod + def tearDownClass(cls): + ReusedPySparkTestCase.tearDownClass() + shutil.rmtree(cls.tempdir.name) def test_sequencefiles(self): basepath = self.tempdir.name @@ -954,15 +948,13 @@ def test_converters(self): self.assertEqual(maps, em) -class TestOutputFormat(PySparkTestCase): +class OutputFormatTests(ReusedPySparkTestCase): def setUp(self): - PySparkTestCase.setUp(self) self.tempdir = tempfile.NamedTemporaryFile(delete=False) os.unlink(self.tempdir.name) def tearDown(self): - PySparkTestCase.tearDown(self) shutil.rmtree(self.tempdir.name, ignore_errors=True) def test_sequencefiles(self): @@ -1243,8 +1235,7 @@ def test_malformed_RDD(self): basepath + "/malformed/sequence")) -class TestDaemon(unittest.TestCase): - +class DaemonTests(unittest.TestCase): def connect(self, port): from socket import socket, AF_INET, SOCK_STREAM sock = socket(AF_INET, SOCK_STREAM) @@ -1290,7 +1281,7 @@ def test_termination_sigterm(self): self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM)) -class TestWorker(PySparkTestCase): +class WorkerTests(PySparkTestCase): def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) @@ -1342,11 +1333,6 @@ def run(): rdd = self.sc.parallelize(range(100), 1) self.assertEqual(100, rdd.map(str).count()) - def test_fd_leak(self): - N = 1100 # fd limit is 1024 by default - rdd = self.sc.parallelize(range(N), N) - self.assertEquals(N, rdd.count()) - def test_after_exception(self): def raise_exception(_): raise Exception() @@ -1379,7 +1365,7 @@ def test_accumulator_when_reuse_worker(self): self.assertEqual(sum(range(100)), acc1.value) -class TestSparkSubmit(unittest.TestCase): +class SparkSubmitTests(unittest.TestCase): def setUp(self): self.programDir = tempfile.mkdtemp() @@ -1492,6 +1478,8 @@ def test_single_script_on_cluster(self): |sc = SparkContext() |print sc.parallelize([1, 2, 3]).map(foo).collect() """) + # this will fail if you have different spark.executor.memory + # in conf/spark-defaults.conf proc = subprocess.Popen( [self.sparkSubmit, "--master", "local-cluster[1,1,512]", script], stdout=subprocess.PIPE) @@ -1500,7 +1488,11 @@ def test_single_script_on_cluster(self): self.assertIn("[2, 4, 6]", out) -class ContextStopTests(unittest.TestCase): +class ContextTests(unittest.TestCase): + + def test_failed_sparkcontext_creation(self): + # Regression test for SPARK-1550 + self.assertRaises(Exception, lambda: SparkContext("an-invalid-master-name")) def test_stop(self): sc = SparkContext() diff --git a/python/run-tests b/python/run-tests index a7ec270c7da21..c713861eb77bb 100755 --- a/python/run-tests +++ b/python/run-tests @@ -34,7 +34,7 @@ rm -rf metastore warehouse function run_test() { echo "Running test: $1" - SPARK_TESTING=1 "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log + SPARK_TESTING=1 time "$FWDIR"/bin/pyspark $1 2>&1 | tee -a unit-tests.log FAILED=$((PIPESTATUS[0]||$FAILED)) @@ -48,6 +48,37 @@ function run_test() { fi } +function run_core_tests() { + echo "Run core tests ..." + run_test "pyspark/rdd.py" + run_test "pyspark/context.py" + run_test "pyspark/conf.py" + PYSPARK_DOC_TEST=1 run_test "pyspark/broadcast.py" + PYSPARK_DOC_TEST=1 run_test "pyspark/accumulators.py" + PYSPARK_DOC_TEST=1 run_test "pyspark/serializers.py" + run_test "pyspark/shuffle.py" + run_test "pyspark/tests.py" +} + +function run_sql_tests() { + echo "Run sql tests ..." + run_test "pyspark/sql.py" +} + +function run_mllib_tests() { + echo "Run mllib tests ..." + run_test "pyspark/mllib/classification.py" + run_test "pyspark/mllib/clustering.py" + run_test "pyspark/mllib/linalg.py" + run_test "pyspark/mllib/random.py" + run_test "pyspark/mllib/recommendation.py" + run_test "pyspark/mllib/regression.py" + run_test "pyspark/mllib/stat.py" + run_test "pyspark/mllib/tree.py" + run_test "pyspark/mllib/util.py" + run_test "pyspark/mllib/tests.py" +} + echo "Running PySpark tests. Output is in python/unit-tests.log." export PYSPARK_PYTHON="python" @@ -60,29 +91,9 @@ fi echo "Testing with Python version:" $PYSPARK_PYTHON --version -run_test "pyspark/rdd.py" -run_test "pyspark/context.py" -run_test "pyspark/conf.py" -run_test "pyspark/sql.py" -# These tests are included in the module-level docs, and so must -# be handled on a higher level rather than within the python file. -export PYSPARK_DOC_TEST=1 -run_test "pyspark/broadcast.py" -run_test "pyspark/accumulators.py" -run_test "pyspark/serializers.py" -unset PYSPARK_DOC_TEST -run_test "pyspark/shuffle.py" -run_test "pyspark/tests.py" -run_test "pyspark/mllib/classification.py" -run_test "pyspark/mllib/clustering.py" -run_test "pyspark/mllib/linalg.py" -run_test "pyspark/mllib/random.py" -run_test "pyspark/mllib/recommendation.py" -run_test "pyspark/mllib/regression.py" -run_test "pyspark/mllib/stat.py" -run_test "pyspark/mllib/tests.py" -run_test "pyspark/mllib/tree.py" -run_test "pyspark/mllib/util.py" +run_core_tests +run_sql_tests +run_mllib_tests # Try to test with PyPy if [ $(which pypy) ]; then @@ -90,19 +101,8 @@ if [ $(which pypy) ]; then echo "Testing with PyPy version:" $PYSPARK_PYTHON --version - run_test "pyspark/rdd.py" - run_test "pyspark/context.py" - run_test "pyspark/conf.py" - run_test "pyspark/sql.py" - # These tests are included in the module-level docs, and so must - # be handled on a higher level rather than within the python file. - export PYSPARK_DOC_TEST=1 - run_test "pyspark/broadcast.py" - run_test "pyspark/accumulators.py" - run_test "pyspark/serializers.py" - unset PYSPARK_DOC_TEST - run_test "pyspark/shuffle.py" - run_test "pyspark/tests.py" + run_core_tests + run_sql_tests fi if [[ $FAILED == 0 ]]; then From 2300eb58ae79a86e65b3ff608a578f5d4c09892b Mon Sep 17 00:00:00 2001 From: cocoatomo Date: Mon, 6 Oct 2014 14:08:40 -0700 Subject: [PATCH 0907/1492] [SPARK-3773][PySpark][Doc] Sphinx build warning When building Sphinx documents for PySpark, we have 12 warnings. Their causes are almost docstrings in broken ReST format. To reproduce this issue, we should run following commands on the commit: 6e27cb630de69fa5acb510b4e2f6b980742b1957. ```bash $ cd ./python/docs $ make clean html ... /Users//MyRepos/Scala/spark/python/pyspark/__init__.py:docstring of pyspark.SparkContext.sequenceFile:4: ERROR: Unexpected indentation. /Users//MyRepos/Scala/spark/python/pyspark/__init__.py:docstring of pyspark.RDD.saveAsSequenceFile:4: ERROR: Unexpected indentation. /Users//MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.LogisticRegressionWithSGD.train:14: ERROR: Unexpected indentation. /Users//MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.LogisticRegressionWithSGD.train:16: WARNING: Definition list ends without a blank line; unexpected unindent. /Users//MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.LogisticRegressionWithSGD.train:17: WARNING: Block quote ends without a blank line; unexpected unindent. /Users//MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.SVMWithSGD.train:14: ERROR: Unexpected indentation. /Users//MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.SVMWithSGD.train:16: WARNING: Definition list ends without a blank line; unexpected unindent. /Users//MyRepos/Scala/spark/python/pyspark/mllib/classification.py:docstring of pyspark.mllib.classification.SVMWithSGD.train:17: WARNING: Block quote ends without a blank line; unexpected unindent. /Users//MyRepos/Scala/spark/python/docs/pyspark.mllib.rst:50: WARNING: missing attribute mentioned in :members: or __all__: module pyspark.mllib.regression, attribute RidgeRegressionModelLinearRegressionWithSGD /Users//MyRepos/Scala/spark/python/pyspark/mllib/tree.py:docstring of pyspark.mllib.tree.DecisionTreeModel.predict:3: ERROR: Unexpected indentation. ... checking consistency... /Users//MyRepos/Scala/spark/python/docs/modules.rst:: WARNING: document isn't included in any toctree ... copying static files... WARNING: html_static_path entry u'/Users//MyRepos/Scala/spark/python/docs/_static' does not exist ... build succeeded, 12 warnings. ``` Author: cocoatomo Closes #2653 from cocoatomo/issues/3773-sphinx-build-warnings and squashes the following commits: 6f65661 [cocoatomo] [SPARK-3773][PySpark][Doc] Sphinx build warning --- python/docs/modules.rst | 7 ------- python/pyspark/context.py | 1 + python/pyspark/mllib/classification.py | 26 ++++++++++++++++---------- python/pyspark/mllib/regression.py | 15 +++++++++------ python/pyspark/mllib/tree.py | 1 + python/pyspark/rdd.py | 1 + 6 files changed, 28 insertions(+), 23 deletions(-) delete mode 100644 python/docs/modules.rst diff --git a/python/docs/modules.rst b/python/docs/modules.rst deleted file mode 100644 index 183564659fbcf..0000000000000 --- a/python/docs/modules.rst +++ /dev/null @@ -1,7 +0,0 @@ -. -= - -.. toctree:: - :maxdepth: 4 - - pyspark diff --git a/python/pyspark/context.py b/python/pyspark/context.py index e9418320ff781..a45d79d6424c7 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -410,6 +410,7 @@ def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, Read a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. The mechanism is as follows: + 1. A Java RDD is created from the SequenceFile or other InputFormat, and the key and value Writable classes 2. Serialization is attempted via Pyrolite pickling diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index ac142fb49a90c..a765b1c4f7d87 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -89,11 +89,14 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, @param regParam: The regularizer parameter (default: 1.0). @param regType: The type of regularizer used for training our model. - Allowed values: "l1" for using L1Updater, - "l2" for using - SquaredL2Updater, - "none" for no regularizer. - (default: "none") + + :Allowed values: + - "l1" for using L1Updater + - "l2" for using SquaredL2Updater + - "none" for no regularizer + + (default: "none") + @param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features @@ -158,11 +161,14 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, @param initialWeights: The initial weights (default: None). @param regType: The type of regularizer used for training our model. - Allowed values: "l1" for using L1Updater, - "l2" for using - SquaredL2Updater, - "none" for no regularizer. - (default: "none") + + :Allowed values: + - "l1" for using L1Updater + - "l2" for using SquaredL2Updater, + - "none" for no regularizer. + + (default: "none") + @param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 8fe8c6db2ad9c..54f34a98337ca 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -22,7 +22,7 @@ from pyspark.mllib.linalg import SparseVector, _convert_to_vector from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -__all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel' +__all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', 'LinearRegressionWithSGD', 'LassoWithSGD', 'RidgeRegressionWithSGD'] @@ -155,11 +155,14 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, @param regParam: The regularizer parameter (default: 1.0). @param regType: The type of regularizer used for training our model. - Allowed values: "l1" for using L1Updater, - "l2" for using - SquaredL2Updater, - "none" for no regularizer. - (default: "none") + + :Allowed values: + - "l1" for using L1Updater, + - "l2" for using SquaredL2Updater, + - "none" for no regularizer. + + (default: "none") + @param intercept: Boolean parameter which indicates the use or not of the augmented representation for training data (i.e. whether bias features diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index afdcdbdf3ae01..5d7abfb96b7fe 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -48,6 +48,7 @@ def __del__(self): def predict(self, x): """ Predict the label of one or more examples. + :param x: Data point (feature vector), or an RDD of data points (feature vectors). """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index dc6497772e502..e77669aad76b6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1208,6 +1208,7 @@ def saveAsSequenceFile(self, path, compressionCodecClass=None): Output a Python RDD of key-value pairs (of form C{RDD[(K, V)]}) to any Hadoop file system, using the L{org.apache.hadoop.io.Writable} types that we convert from the RDD's key and value types. The mechanism is as follows: + 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects. 2. Keys and values of this Java RDD are converted to Writables and written out. From 69c3f441a9b6e942d6c08afecd59a0349d61cc7b Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Mon, 6 Oct 2014 14:19:06 -0700 Subject: [PATCH 0908/1492] [SPARK-3479] [Build] Report failed test category This PR allows SparkQA (i.e. Jenkins) to report in its posts to GitHub what category of test failed, if one can be determined. The failure categories are: * general failure * RAT checks failed * Scala style checks failed * Python style checks failed * Build failed * Spark unit tests failed * PySpark unit tests failed * MiMa checks failed This PR also fixes the diffing logic used to determine if a patch introduces new classes. Author: Nicholas Chammas Closes #2606 from nchammas/report-failed-test-category and squashes the following commits: d67df03 [Nicholas Chammas] report what test category failed --- dev/run-tests | 32 ++++++++++++- dev/run-tests-codes.sh | 27 +++++++++++ dev/run-tests-jenkins | 102 ++++++++++++++++++++++++++++------------- 3 files changed, 126 insertions(+), 35 deletions(-) create mode 100644 dev/run-tests-codes.sh diff --git a/dev/run-tests b/dev/run-tests index c3d8f49cdd993..4be2baaf48cd1 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -24,6 +24,16 @@ cd "$FWDIR" # Remove work directory rm -rf ./work +source "$FWDIR/dev/run-tests-codes.sh" + +CURRENT_BLOCK=$BLOCK_GENERAL + +function handle_error () { + echo "[error] Got a return code of $? on line $1 of the run-tests script." + exit $CURRENT_BLOCK +} + + # Build against the right verison of Hadoop. { if [ -n "$AMPLAB_JENKINS_BUILD_PROFILE" ]; then @@ -91,26 +101,34 @@ if [ -n "$AMPLAB_JENKINS" ]; then fi fi -# Fail fast -set -e set -o pipefail +trap 'handle_error $LINENO' ERR echo "" echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_RAT + ./dev/check-license echo "" echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_SCALA_STYLE + ./dev/lint-scala echo "" echo "=========================================================================" echo "Running Python style checks" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_PYTHON_STYLE + ./dev/lint-python echo "" @@ -118,6 +136,8 @@ echo "=========================================================================" echo "Building Spark" echo "=========================================================================" +CURRENT_BLOCK=$BLOCK_BUILD + { # We always build with Hive because the PySpark Spark SQL tests need it. BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" @@ -141,6 +161,8 @@ echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" +CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS + { # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. # This must be a single argument, as it is. @@ -175,10 +197,16 @@ echo "" echo "=========================================================================" echo "Running PySpark tests" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS + ./python/run-tests echo "" echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_MIMA + ./dev/mima diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh new file mode 100644 index 0000000000000..1348e0609dda4 --- /dev/null +++ b/dev/run-tests-codes.sh @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +# +# 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. +# + +readonly BLOCK_GENERAL=10 +readonly BLOCK_RAT=11 +readonly BLOCK_SCALA_STYLE=12 +readonly BLOCK_PYTHON_STYLE=13 +readonly BLOCK_BUILD=14 +readonly BLOCK_SPARK_UNIT_TESTS=15 +readonly BLOCK_PYSPARK_UNIT_TESTS=16 +readonly BLOCK_MIMA=17 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 0b1e31b9413cf..451f3b771cc76 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -26,9 +26,23 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd "$FWDIR" +source "$FWDIR/dev/run-tests-codes.sh" + COMMENTS_URL="https://api.github.com/repos/apache/spark/issues/$ghprbPullId/comments" PULL_REQUEST_URL="https://github.com/apache/spark/pull/$ghprbPullId" +# Important Environment Variables +# --- +# $ghprbActualCommit +#+ This is the hash of the most recent commit in the PR. +#+ The merge-base of this and master is the commit from which the PR was branched. +# $sha1 +#+ If the patch merges cleanly, this is a reference to the merge commit hash +#+ (e.g. "origin/pr/2606/merge"). +#+ If the patch does not merge cleanly, it is equal to $ghprbActualCommit. +#+ The merge-base of this and master in the case of a clean merge is the most recent commit +#+ against master. + COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" @@ -84,42 +98,46 @@ function post_message () { fi } + +# We diff master...$ghprbActualCommit because that gets us changes introduced in the PR +#+ and not anything else added to master since the PR was branched. + # check PR merge-ability and check for new public classes { if [ "$sha1" == "$ghprbActualCommit" ]; then - merge_note=" * This patch **does not** merge cleanly!" + merge_note=" * This patch **does not merge cleanly**." else merge_note=" * This patch merges cleanly." + fi + + source_files=$( + git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ + | grep -v -e "\/test" `# ignore files in test directories` \ + | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ + | tr "\n" " " + ) + new_public_classes=$( + git diff master...$ghprbActualCommit ${source_files} `# diff patch against master from branch point` \ + | grep "^\+" `# filter in only added lines` \ + | sed -r -e "s/^\+//g" `# remove the leading +` \ + | grep -e "trait " -e "class " `# filter in lines with these key words` \ + | grep -e "{" -e "(" `# filter in lines with these key words, too` \ + | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ + | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ + | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ + | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ + | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ + | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ + | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ + | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ + | tr -d "\n" `# remove actual LF characters` + ) - source_files=$( - git diff master... --name-only `# diff patch against master from branch point` \ - | grep -v -e "\/test" `# ignore files in test directories` \ - | grep -e "\.py$" -e "\.java$" -e "\.scala$" `# include only code files` \ - | tr "\n" " " - ) - new_public_classes=$( - git diff master... ${source_files} `# diff patch against master from branch point` \ - | grep "^\+" `# filter in only added lines` \ - | sed -r -e "s/^\+//g" `# remove the leading +` \ - | grep -e "trait " -e "class " `# filter in lines with these key words` \ - | grep -e "{" -e "(" `# filter in lines with these key words, too` \ - | grep -v -e "\@\@" -e "private" `# exclude lines with these words` \ - | grep -v -e "^// " -e "^/\*" -e "^ \* " `# exclude comment lines` \ - | sed -r -e "s/\{.*//g" `# remove from the { onwards` \ - | sed -r -e "s/\}//g" `# just in case, remove }; they mess the JSON` \ - | sed -r -e "s/\"/\\\\\"/g" `# escape double quotes; they mess the JSON` \ - | sed -r -e "s/^(.*)$/\`\1\`/g" `# surround with backticks for style` \ - | sed -r -e "s/^/ \* /g" `# prepend ' *' to start of line` \ - | sed -r -e "s/$/\\\n/g" `# append newline to end of line` \ - | tr -d "\n" `# remove actual LF characters` - ) - - if [ "$new_public_classes" == "" ]; then - public_classes_note=" * This patch adds no public classes." - else - public_classes_note=" * This patch adds the following public classes _(experimental)_:" - public_classes_note="${public_classes_note}\n${new_public_classes}" - fi + if [ -z "$new_public_classes" ]; then + public_classes_note=" * This patch adds no public classes." + else + public_classes_note=" * This patch adds the following public classes _(experimental)_:" + public_classes_note="${public_classes_note}\n${new_public_classes}" fi } @@ -147,12 +165,30 @@ function post_message () { post_message "$fail_message" exit $test_result + elif [ "$test_result" -eq "0" ]; then + test_result_note=" * This patch **passes all tests**." else - if [ "$test_result" -eq "0" ]; then - test_result_note=" * This patch **passes** unit tests." + if [ "$test_result" -eq "$BLOCK_GENERAL" ]; then + failing_test="some tests" + elif [ "$test_result" -eq "$BLOCK_RAT" ]; then + failing_test="RAT tests" + elif [ "$test_result" -eq "$BLOCK_SCALA_STYLE" ]; then + failing_test="Scala style tests" + elif [ "$test_result" -eq "$BLOCK_PYTHON_STYLE" ]; then + failing_test="Python style tests" + elif [ "$test_result" -eq "$BLOCK_BUILD" ]; then + failing_test="to build" + elif [ "$test_result" -eq "$BLOCK_SPARK_UNIT_TESTS" ]; then + failing_test="Spark unit tests" + elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then + failing_test="PySpark unit tests" + elif [ "$test_result" -eq "$BLOCK_MIMA" ]; then + failing_test="MiMa tests" else - test_result_note=" * This patch **fails** unit tests." + failing_test="some tests" fi + + test_result_note=" * This patch **fails $failing_test**." fi } From 70e824f750aa8ed446eec104ba158b0503ba58a9 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Tue, 7 Oct 2014 09:51:37 -0500 Subject: [PATCH 0909/1492] [SPARK-3627] - [yarn] - fix exit code and final status reporting to RM See the description and whats handled in the jira comment: https://issues.apache.org/jira/browse/SPARK-3627?focusedCommentId=14150013&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14150013 This does not handle yarn client mode reporting of the driver to the AM. I think that should be handled when we make it an unmanaged AM. Author: Thomas Graves Closes #2577 from tgravescs/SPARK-3627 and squashes the following commits: 9c2efbf [Thomas Graves] review comments e8cc261 [Thomas Graves] fix accidental typo during fixing comment 24c98e3 [Thomas Graves] rework 85f1901 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into SPARK-3627 fab166d [Thomas Graves] update based on review comments 32f4dfa [Thomas Graves] switch back f0b6519 [Thomas Graves] change order of cleanup staging dir d3cc800 [Thomas Graves] SPARK-3627 - yarn - fix exit code and final status reporting to RM --- .../spark/deploy/yarn/YarnRMClientImpl.scala | 26 +- .../spark/deploy/yarn/ApplicationMaster.scala | 295 +++++++++++------- .../spark/deploy/yarn/YarnRMClient.scala | 4 +- .../spark/deploy/yarn/YarnRMClientImpl.scala | 13 +- 4 files changed, 212 insertions(+), 126 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index 9bd1719cb1808..7faf55bc63372 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -40,6 +40,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC private var rpc: YarnRPC = null private var resourceManager: AMRMProtocol = _ private var uiHistoryAddress: String = _ + private var registered: Boolean = false override def register( conf: YarnConfiguration, @@ -51,8 +52,11 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC this.rpc = YarnRPC.create(conf) this.uiHistoryAddress = uiHistoryAddress - resourceManager = registerWithResourceManager(conf) - registerApplicationMaster(uiAddress) + synchronized { + resourceManager = registerWithResourceManager(conf) + registerApplicationMaster(uiAddress) + registered = true + } new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, preferredNodeLocations, securityMgr) @@ -66,14 +70,16 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC appAttemptId } - override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(getAttemptId()) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(uiHistoryAddress) - resourceManager.finishApplicationMaster(finishReq) + override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { + if (registered) { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(getAttemptId()) + finishReq.setFinishApplicationStatus(status) + finishReq.setDiagnostics(diagnostics) + finishReq.setTrackingUrl(uiHistoryAddress) + resourceManager.finishApplicationMaster(finishReq) + } } override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index caceef5d4b5b0..a3c43b43848d2 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend @@ -56,8 +57,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) + @volatile private var exitCode = 0 + @volatile private var unregistered = false @volatile private var finished = false @volatile private var finalStatus = FinalApplicationStatus.UNDEFINED + @volatile private var finalMsg: String = "" @volatile private var userClassThread: Thread = _ private var reporterThread: Thread = _ @@ -71,80 +75,107 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private val sparkContextRef = new AtomicReference[SparkContext](null) final def run(): Int = { - val appAttemptId = client.getAttemptId() + try { + val appAttemptId = client.getAttemptId() - if (isDriver) { - // Set the web ui port to be ephemeral for yarn so we don't conflict with - // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") + if (isDriver) { + // Set the web ui port to be ephemeral for yarn so we don't conflict with + // other spark processes running on the same box + System.setProperty("spark.ui.port", "0") - // Set the master property to match the requested mode. - System.setProperty("spark.master", "yarn-cluster") + // Set the master property to match the requested mode. + System.setProperty("spark.master", "yarn-cluster") - // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. - System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) - } + // Propagate the application ID so that YarnClusterSchedulerBackend can pick it up. + System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) + } - logInfo("ApplicationAttemptId: " + appAttemptId) + logInfo("ApplicationAttemptId: " + appAttemptId) - val cleanupHook = new Runnable { - override def run() { - // If the SparkContext is still registered, shut it down as a best case effort in case - // users do not call sc.stop or do System.exit(). - val sc = sparkContextRef.get() - if (sc != null) { - logInfo("Invoking sc stop from shutdown hook") - sc.stop() - finish(FinalApplicationStatus.SUCCEEDED) - } + val cleanupHook = new Runnable { + override def run() { + // If the SparkContext is still registered, shut it down as a best case effort in case + // users do not call sc.stop or do System.exit(). + val sc = sparkContextRef.get() + if (sc != null) { + logInfo("Invoking sc stop from shutdown hook") + sc.stop() + } + val maxAppAttempts = client.getMaxRegAttempts(yarnConf) + val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts + + if (!finished) { + // this shouldn't ever happen, but if it does assume weird failure + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, + "shutdown hook called without cleanly finishing") + } - // Cleanup the staging dir after the app is finished, or if it's the last attempt at - // running the AM. - val maxAppAttempts = client.getMaxRegAttempts(yarnConf) - val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts - if (finished || isLastAttempt) { - cleanupStagingDir() + if (!unregistered) { + // we only want to unregister if we don't want the RM to retry + if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { + unregister(finalStatus, finalMsg) + cleanupStagingDir() + } + } } } - } - // Use higher priority than FileSystem. - assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) - ShutdownHookManager - .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) + // Use higher priority than FileSystem. + assert(ApplicationMaster.SHUTDOWN_HOOK_PRIORITY > FileSystem.SHUTDOWN_HOOK_PRIORITY) + ShutdownHookManager + .get().addShutdownHook(cleanupHook, ApplicationMaster.SHUTDOWN_HOOK_PRIORITY) - // Call this to force generation of secret so it gets populated into the - // Hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the executor containers. - val securityMgr = new SecurityManager(sparkConf) + // Call this to force generation of secret so it gets populated into the + // Hadoop UGI. This has to happen before the startUserClass which does a + // doAs in order for the credentials to be passed on to the executor containers. + val securityMgr = new SecurityManager(sparkConf) - if (isDriver) { - runDriver(securityMgr) - } else { - runExecutorLauncher(securityMgr) + if (isDriver) { + runDriver(securityMgr) + } else { + runExecutorLauncher(securityMgr) + } + } catch { + case e: Exception => + // catch everything else if not specifically handled + logError("Uncaught exception: ", e) + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, + "Uncaught exception: " + e.getMessage()) } + exitCode + } - if (finalStatus != FinalApplicationStatus.UNDEFINED) { - finish(finalStatus) - 0 - } else { - 1 + /** + * unregister is used to completely unregister the application from the ResourceManager. + * This means the ResourceManager will not retry the application attempt on your behalf if + * a failure occurred. + */ + final def unregister(status: FinalApplicationStatus, diagnostics: String = null) = synchronized { + if (!unregistered) { + logInfo(s"Unregistering ApplicationMaster with $status" + + Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) + unregistered = true + client.unregister(status, Option(diagnostics).getOrElse("")) } } - final def finish(status: FinalApplicationStatus, diagnostics: String = null) = synchronized { + final def finish(status: FinalApplicationStatus, code: Int, msg: String = null) = synchronized { if (!finished) { - logInfo(s"Finishing ApplicationMaster with $status" + - Option(diagnostics).map(msg => s" (diag message: $msg)").getOrElse("")) - finished = true + logInfo(s"Final app status: ${status}, exitCode: ${code}" + + Option(msg).map(msg => s", (reason: $msg)").getOrElse("")) + exitCode = code finalStatus = status - try { - if (Thread.currentThread() != reporterThread) { - reporterThread.interrupt() - reporterThread.join() - } - } finally { - client.shutdown(status, Option(diagnostics).getOrElse("")) + finalMsg = msg + finished = true + if (Thread.currentThread() != reporterThread && reporterThread != null) { + logDebug("shutting down reporter thread") + reporterThread.interrupt() + } + if (Thread.currentThread() != userClassThread && userClassThread != null) { + logDebug("shutting down user thread") + userClassThread.interrupt() } } } @@ -182,7 +213,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def runDriver(securityMgr: SecurityManager): Unit = { addAmIpFilter() - val userThread = startUserClass() + setupSystemSecurityManager() + userClassThread = startUserClass() // This a bit hacky, but we need to wait until the spark.driver.port property has // been set by the Thread executing the user class. @@ -190,15 +222,12 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // If there is no SparkContext at this point, just fail the app. if (sc == null) { - finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_SC_NOT_INITED, + "Timed out waiting for SparkContext.") } else { registerAM(sc.ui.map(_.appUIAddress).getOrElse(""), securityMgr) - try { - userThread.join() - } finally { - // In cluster mode, ask the reporter thread to stop since the user app is finished. - reporterThread.interrupt() - } + userClassThread.join() } } @@ -211,7 +240,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // In client mode the actor will stop the reporter thread. reporterThread.join() - finalStatus = FinalApplicationStatus.SUCCEEDED } private def launchReporterThread(): Thread = { @@ -231,33 +259,26 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, val t = new Thread { override def run() { var failureCount = 0 - while (!finished) { try { - checkNumExecutorsFailed() - if (!finished) { + if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, + "Max number of executor failures reached") + } else { logDebug("Sending progress") allocator.allocateResources() } failureCount = 0 } catch { + case i: InterruptedException => case e: Throwable => { failureCount += 1 if (!NonFatal(e) || failureCount >= reporterMaxFailures) { - logError("Exception was thrown from Reporter thread.", e) - finish(FinalApplicationStatus.FAILED, "Exception was thrown" + - s"${failureCount} time(s) from Reporter thread.") - - /** - * If exception is thrown from ReporterThread, - * interrupt user class to stop. - * Without this interrupting, if exception is - * thrown before allocating enough executors, - * YarnClusterScheduler waits until timeout even though - * we cannot allocate executors. - */ - logInfo("Interrupting user class to stop.") - userClassThread.interrupt + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + + s"${failureCount} time(s) from Reporter thread.") + } else { logWarning(s"Reporter thread fails ${failureCount} time(s) in a row.", e) } @@ -308,7 +329,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, sparkContextRef.synchronized { var count = 0 val waitTime = 10000L - val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10) + val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10) while (sparkContextRef.get() == null && count < numTries && !finished) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 @@ -328,10 +349,19 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, private def waitForSparkDriver(): ActorRef = { logInfo("Waiting for Spark driver to be reachable.") var driverUp = false + var count = 0 val hostport = args.userArgs(0) val (driverHost, driverPort) = Utils.parseHostPort(hostport) - while (!driverUp) { + + // spark driver should already be up since it launched us, but we don't want to + // wait forever, so wait 100 seconds max to match the cluster mode setting. + // Leave this config unpublished for now. SPARK-3779 to investigating changing + // this config to be time based. + val numTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 1000) + + while (!driverUp && !finished && count < numTries) { try { + count = count + 1 val socket = new Socket(driverHost, driverPort) socket.close() logInfo("Driver now available: %s:%s".format(driverHost, driverPort)) @@ -343,6 +373,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, Thread.sleep(100) } } + + if (!driverUp) { + throw new SparkException("Failed to connect to driver!") + } + sparkConf.set("spark.driver.host", driverHost) sparkConf.set("spark.driver.port", driverPort.toString) @@ -354,18 +389,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") } - private def checkNumExecutorsFailed() = { - if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finish(FinalApplicationStatus.FAILED, "Max number of executor failures reached.") - - val sc = sparkContextRef.get() - if (sc != null) { - logInfo("Invoking sc stop from checkNumExecutorsFailed") - sc.stop() - } - } - } - /** Add the Yarn IP filter that is required for properly securing the UI. */ private def addAmIpFilter() = { val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) @@ -379,40 +402,81 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, } } + /** + * This system security manager applies to the entire process. + * It's main purpose is to handle the case if the user code does a System.exit. + * This allows us to catch that and properly set the YARN application status and + * cleanup if needed. + */ + private def setupSystemSecurityManager(): Unit = { + try { + var stopped = false + System.setSecurityManager(new java.lang.SecurityManager() { + override def checkExit(paramInt: Int) { + if (!stopped) { + logInfo("In securityManager checkExit, exit code: " + paramInt) + if (paramInt == 0) { + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + } else { + finish(FinalApplicationStatus.FAILED, + paramInt, + "User class exited with non-zero exit code") + } + stopped = true + } + } + // required for the checkExit to work properly + override def checkPermission(perm: java.security.Permission): Unit = {} + }) + } + catch { + case e: SecurityException => + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_SECURITY, + "Error in setSecurityManager") + logError("Error in setSecurityManager:", e) + } + } + + /** + * Start the user class, which contains the spark driver, in a separate Thread. + * If the main routine exits cleanly or exits with System.exit(0) we + * assume it was successful, for all other cases we assume failure. + * + * Returns the user thread that was started. + */ 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, Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) - userClassThread = new Thread { + val userThread = new Thread { override def run() { - var status = FinalApplicationStatus.FAILED try { - // Copy val mainArgs = new Array[String](args.userArgs.size) args.userArgs.copyToArray(mainArgs, 0, args.userArgs.size) mainMethod.invoke(null, mainArgs) - // Some apps have "System.exit(0)" at the end. The user thread will stop here unless - // it has an uncaught exception thrown out. It needs a shutdown hook to set SUCCEEDED. - status = FinalApplicationStatus.SUCCEEDED + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + logDebug("Done running users class") } catch { case e: InvocationTargetException => e.getCause match { case _: InterruptedException => // Reporter thread can interrupt to stop user class - - case e => throw e + case e: Exception => + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_EXCEPTION_USER_CLASS, + "User class threw exception: " + e.getMessage) + // re-throw to get it logged + throw e } - } finally { - logDebug("Finishing main") - finalStatus = status } } } - userClassThread.setName("Driver") - userClassThread.start() - userClassThread + userThread.setName("Driver") + userThread.start() + userThread } // Actor used to monitor the driver when running in client deploy mode. @@ -432,7 +496,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, override def receive = { case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") - finish(FinalApplicationStatus.SUCCEEDED) + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) case x: AddWebUIFilter => logInfo(s"Add WebUI Filter. $x") driver ! x @@ -446,6 +510,15 @@ object ApplicationMaster extends Logging { val SHUTDOWN_HOOK_PRIORITY: Int = 30 + // exit codes for different causes, no reason behind the values + private val EXIT_SUCCESS = 0 + private val EXIT_UNCAUGHT_EXCEPTION = 10 + private val EXIT_MAX_EXECUTOR_FAILURES = 11 + private val EXIT_REPORTER_FAILURE = 12 + private val EXIT_SC_NOT_INITED = 13 + private val EXIT_SECURITY = 14 + private val EXIT_EXCEPTION_USER_CLASS = 15 + private var master: ApplicationMaster = _ def main(args: Array[String]) = { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 943dc56202a37..2510b9c9cef68 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -49,12 +49,12 @@ trait YarnRMClient { securityMgr: SecurityManager): YarnAllocator /** - * Shuts down the AM. Guaranteed to only be called once. + * Unregister the AM. Guaranteed to only be called once. * * @param status The final status of the AM. * @param diagnostics Diagnostics message to include in the final status. */ - def shutdown(status: FinalApplicationStatus, diagnostics: String = ""): Unit + def unregister(status: FinalApplicationStatus, diagnostics: String = ""): Unit /** Returns the attempt ID. */ def getAttemptId(): ApplicationAttemptId diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index b581790e158ac..8d4b96ed79933 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -45,6 +45,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC private var amClient: AMRMClient[ContainerRequest] = _ private var uiHistoryAddress: String = _ + private var registered: Boolean = false override def register( conf: YarnConfiguration, @@ -59,13 +60,19 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC this.uiHistoryAddress = uiHistoryAddress logInfo("Registering the ApplicationMaster") - amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + synchronized { + amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) + registered = true + } new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, preferredNodeLocations, securityMgr) } - override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = - amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { + if (registered) { + amClient.unregisterApplicationMaster(status, diagnostics, uiHistoryAddress) + } + } override def getAttemptId() = { val containerIdString = System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) From d65fd554b4de1dbd8db3090b0e50994010d30e78 Mon Sep 17 00:00:00 2001 From: Hossein Date: Tue, 7 Oct 2014 11:46:26 -0700 Subject: [PATCH 0910/1492] [SPARK-3827] Very long RDD names are not rendered properly in web UI With Spark SQL we generate very long RDD names. These names are not properly rendered in the web UI. This PR fixes the rendering issue. [SPARK-3827] #comment Linking PR with JIRA Author: Hossein Closes #2687 from falaki/sparkTableUI and squashes the following commits: fd06409 [Hossein] Limit width of cell when RDD name is too long --- core/src/main/resources/org/apache/spark/ui/static/webui.css | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 445110d63e184..152bde5f6994f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -51,6 +51,11 @@ table.sortable thead { cursor: pointer; } +table.sortable td { + word-wrap: break-word; + max-width: 600px; +} + .progress { margin-bottom: 0px; position: relative } From 12e2551ea1773ae19559ecdada35d23608e6b0ec Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Tue, 7 Oct 2014 11:53:22 -0700 Subject: [PATCH 0911/1492] [SPARK-3808] PySpark fails to start in Windows Modified syntax error of *.cmd script. Author: Masayoshi TSUZUKI Closes #2669 from tsudukim/feature/SPARK-3808 and squashes the following commits: 7f804e6 [Masayoshi TSUZUKI] [SPARK-3808] PySpark fails to start in Windows --- bin/compute-classpath.cmd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 9b9e40321ea93..3cd0579aea8d3 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -38,7 +38,7 @@ if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% -if "x%SPARK_CONF_DIR%"!="x" ( +if not "x%SPARK_CONF_DIR%"=="x" ( set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% ) else ( set CLASSPATH=%CLASSPATH%;%FWDIR%conf From 655032965fc7e2368dff9947fc024ac720ffd19c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 7 Oct 2014 12:06:12 -0700 Subject: [PATCH 0912/1492] [SPARK-3762] clear reference of SparkEnv after stop SparkEnv is cached in ThreadLocal object, so after stop and create a new SparkContext, old SparkEnv is still used by some threads, it will trigger many problems, for example, pyspark will have problem after restart SparkContext, because py4j use thread pool for RPC. This patch will clear all the references after stop a SparkEnv. cc mateiz tdas pwendell Author: Davies Liu Closes #2624 from davies/env and squashes the following commits: a69f30c [Davies Liu] deprecate getThreadLocal ba77ca4 [Davies Liu] remove getThreadLocal(), update docs ee62bb7 [Davies Liu] cleanup ThreadLocal of SparnENV 4d0ea8b [Davies Liu] clear reference of SparkEnv after stop --- .../scala/org/apache/spark/SparkEnv.scala | 19 ++++++++----------- .../apache/spark/api/python/PythonRDD.scala | 1 - .../org/apache/spark/executor/Executor.scala | 2 -- .../scala/org/apache/spark/rdd/PipedRDD.scala | 1 - .../apache/spark/scheduler/DAGScheduler.scala | 1 - .../spark/scheduler/TaskSchedulerImpl.scala | 2 -- .../streaming/scheduler/JobGenerator.scala | 1 - .../streaming/scheduler/JobScheduler.scala | 1 - .../streaming/scheduler/ReceiverTracker.scala | 1 - 9 files changed, 8 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 72cac42cd2b2b..aba713cb4267a 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -43,9 +43,8 @@ import org.apache.spark.util.{AkkaUtils, Utils} * :: DeveloperApi :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently - * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these - * objects needs to have the right SparkEnv set. You can get the current environment with - * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. + * Spark code finds the SparkEnv through a global variable, so all the threads can access the same + * SparkEnv. It can be accessed by SparkEnv.get (e.g. after creating a SparkContext). * * NOTE: This is not intended for external use. This is exposed for Shark and may be made private * in a future release. @@ -119,30 +118,28 @@ class SparkEnv ( } object SparkEnv extends Logging { - private val env = new ThreadLocal[SparkEnv] - @volatile private var lastSetSparkEnv : SparkEnv = _ + @volatile private var env: SparkEnv = _ private[spark] val driverActorSystemName = "sparkDriver" private[spark] val executorActorSystemName = "sparkExecutor" def set(e: SparkEnv) { - lastSetSparkEnv = e - env.set(e) + env = e } /** - * Returns the ThreadLocal SparkEnv, if non-null. Else returns the SparkEnv - * previously set in any thread. + * Returns the SparkEnv. */ def get: SparkEnv = { - Option(env.get()).getOrElse(lastSetSparkEnv) + env } /** * Returns the ThreadLocal SparkEnv. */ + @deprecated("Use SparkEnv.get instead", "1.2") def getThreadLocal: SparkEnv = { - env.get() + env } private[spark] def create( diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 924141475383d..ad6eb9ef50277 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -196,7 +196,6 @@ private[spark] class PythonRDD( override def run(): Unit = Utils.logUncaughtExceptions { try { - SparkEnv.set(env) val stream = new BufferedOutputStream(worker.getOutputStream, bufferSize) val dataOut = new DataOutputStream(stream) // Partition index diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 9bbfcdc4a0b6e..616c7e6a46368 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -148,7 +148,6 @@ private[spark] class Executor( override def run() { val startTime = System.currentTimeMillis() - SparkEnv.set(env) Thread.currentThread.setContextClassLoader(replClassLoader) val ser = SparkEnv.get.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") @@ -158,7 +157,6 @@ private[spark] class Executor( val startGCTime = gcTime try { - SparkEnv.set(env) Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 5d77d37378458..56ac7a69be0d3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -131,7 +131,6 @@ private[spark] class PipedRDD[T: ClassTag]( // Start a thread to feed the process input from our parent's iterator new Thread("stdin writer for " + command) { override def run() { - SparkEnv.set(env) val out = new PrintWriter(proc.getOutputStream) // input the pipe context firstly diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 8135cdbb4c31f..788eb1ff4e455 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -630,7 +630,6 @@ class DAGScheduler( protected def runLocallyWithinThread(job: ActiveJob) { var jobResult: JobResult = JobSucceeded try { - SparkEnv.set(env) val rdd = job.finalStage.rdd val split = rdd.partitions(job.partitions(0)) val taskContext = 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 4dc550413c13c..6d697e3d003f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -216,8 +216,6 @@ private[spark] class TaskSchedulerImpl( * that tasks are balanced across the cluster. */ def resourceOffers(offers: Seq[WorkerOffer]): Seq[Seq[TaskDescription]] = synchronized { - SparkEnv.set(sc.env) - // Mark each slave as alive and remember its hostname // Also track if new executor is added var newExecAvail = false diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 374848358e700..7d73ada12d107 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -217,7 +217,6 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** Generate jobs and perform checkpoint for the given `time`. */ private def generateJobs(time: Time) { - SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => val receivedBlockInfo = graph.getReceiverInputStreams.map { stream => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 1b034b9fb187c..cfa3cd8925c80 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -138,7 +138,6 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } jobSet.handleJobStart(job) logInfo("Starting job " + job.id + " from job set of time " + jobSet.time) - SparkEnv.set(ssc.env) } private def handleJobCompletion(job: Job) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 5307fe189d717..7149dbc12a365 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -202,7 +202,6 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { @transient val thread = new Thread() { override def run() { try { - SparkEnv.set(env) startReceivers() } catch { case ie: InterruptedException => logInfo("ReceiverLauncher interrupted") From bc87cc410fae59660c13b6ae1c14204df77237b8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 7 Oct 2014 12:20:12 -0700 Subject: [PATCH 0913/1492] [SPARK-3731] [PySpark] fix memory leak in PythonRDD The parent.getOrCompute() of PythonRDD is executed in a separated thread, it should release the memory reserved for shuffle and unrolling finally. Author: Davies Liu Closes #2668 from davies/leak and squashes the following commits: ae98be2 [Davies Liu] fix memory leak in PythonRDD --- .../main/scala/org/apache/spark/api/python/PythonRDD.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index ad6eb9ef50277..c74f86548ef85 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -247,6 +247,11 @@ private[spark] class PythonRDD( // will kill the whole executor (see org.apache.spark.executor.Executor). _exception = e worker.shutdownOutput() + } finally { + // Release memory used by this thread for shuffles + env.shuffleMemoryManager.releaseMemoryForThisThread() + // Release memory used by this thread for unrolling blocks + env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() } } } From 553737c6e6d5ffa3b52a9888444f4beece5c5b1a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 7 Oct 2014 12:52:10 -0700 Subject: [PATCH 0914/1492] [SPARK-3825] Log more detail when unrolling a block fails Before: ``` 14/10/06 16:45:42 WARN CacheManager: Not enough space to cache partition rdd_0_2 in memory! Free memory is 481861527 bytes. ``` After: ``` 14/10/07 11:08:24 WARN MemoryStore: Not enough space to cache rdd_2_0 in memory! (computed 68.8 MB so far) 14/10/07 11:08:24 INFO MemoryStore: Memory use = 1088.0 B (blocks) + 445.1 MB (scratch space shared across 8 thread(s)) = 445.1 MB. Storage limit = 459.5 MB. ``` Author: Andrew Or Closes #2688 from andrewor14/cache-log-message and squashes the following commits: 28e33d6 [Andrew Or] Shy away from "unrolling" 5638c49 [Andrew Or] Grammar 39a0c28 [Andrew Or] Log more detail when unrolling a block fails --- .../scala/org/apache/spark/CacheManager.scala | 2 - .../apache/spark/storage/MemoryStore.scala | 45 ++++++++++++++++--- 2 files changed, 39 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index f8584b90cabe6..d89bb50076c9a 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -168,8 +168,6 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { arr.iterator.asInstanceOf[Iterator[T]] case Right(it) => // There is not enough space to cache this partition in memory - logWarning(s"Not enough space to cache partition $key in memory! " + - s"Free memory is ${blockManager.memoryStore.freeMemory} bytes.") val returnValues = it.asInstanceOf[Iterator[T]] if (putLevel.useDisk) { logWarning(s"Persisting partition $key to disk instead.") diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 0a09c24d61879..edbc729c17ade 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -132,8 +132,6 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) PutResult(res.size, res.data, droppedBlocks) case Right(iteratorValues) => // Not enough space to unroll this block; drop to disk if applicable - logWarning(s"Not enough space to store block $blockId in memory! " + - s"Free memory is $freeMemory bytes.") if (level.useDisk && allowPersistToDisk) { logWarning(s"Persisting block $blockId to disk instead.") val res = blockManager.diskStore.putIterator(blockId, iteratorValues, level, returnValues) @@ -265,6 +263,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) Left(vector.toArray) } else { // We ran out of space while unrolling the values for this block + logUnrollFailureMessage(blockId, vector.estimateSize()) Right(vector.iterator ++ values) } @@ -424,7 +423,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * Reserve additional memory for unrolling blocks used by this thread. * Return whether the request is granted. */ - private[spark] def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { + def reserveUnrollMemoryForThisThread(memory: Long): Boolean = { accountingLock.synchronized { val granted = freeMemory > currentUnrollMemory + memory if (granted) { @@ -439,7 +438,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) * Release memory used by this thread for unrolling blocks. * If the amount is not specified, remove the current thread's allocation altogether. */ - private[spark] def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { + def releaseUnrollMemoryForThisThread(memory: Long = -1L): Unit = { val threadId = Thread.currentThread().getId accountingLock.synchronized { if (memory < 0) { @@ -457,16 +456,50 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) /** * Return the amount of memory currently occupied for unrolling blocks across all threads. */ - private[spark] def currentUnrollMemory: Long = accountingLock.synchronized { + def currentUnrollMemory: Long = accountingLock.synchronized { unrollMemoryMap.values.sum } /** * Return the amount of memory currently occupied for unrolling blocks by this thread. */ - private[spark] def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { + def currentUnrollMemoryForThisThread: Long = accountingLock.synchronized { unrollMemoryMap.getOrElse(Thread.currentThread().getId, 0L) } + + /** + * Return the number of threads currently unrolling blocks. + */ + def numThreadsUnrolling: Int = accountingLock.synchronized { unrollMemoryMap.keys.size } + + /** + * Log information about current memory usage. + */ + def logMemoryUsage(): Unit = { + val blocksMemory = currentMemory + val unrollMemory = currentUnrollMemory + val totalMemory = blocksMemory + unrollMemory + logInfo( + s"Memory use = ${Utils.bytesToString(blocksMemory)} (blocks) + " + + s"${Utils.bytesToString(unrollMemory)} (scratch space shared across " + + s"$numThreadsUnrolling thread(s)) = ${Utils.bytesToString(totalMemory)}. " + + s"Storage limit = ${Utils.bytesToString(maxMemory)}." + ) + } + + /** + * Log a warning for failing to unroll a block. + * + * @param blockId ID of the block we are trying to unroll. + * @param finalVectorSize Final size of the vector before unrolling failed. + */ + def logUnrollFailureMessage(blockId: BlockId, finalVectorSize: Long): Unit = { + logWarning( + s"Not enough space to cache $blockId in memory! " + + s"(computed ${Utils.bytesToString(finalVectorSize)} so far)" + ) + logMemoryUsage() + } } private[spark] case class ResultWithDroppedBlocks( From 446063eca98ae56d1ac61415f4c6e89699b8db02 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 7 Oct 2014 16:00:22 -0700 Subject: [PATCH 0915/1492] [SPARK-3777] Display "Executor ID" for Tasks in Stage page Now the Stage page only displays "Executor"(host) for tasks. However, there may be more than one Executors running in the same host. Currently, when some task is hung, I only know the host of the faulty executor. Therefore I have to check all executors in the host. Adding "Executor ID" in the Tasks table. would be helpful to locate the faulty executor. Here is the new page: ![add_executor_id_for_tasks](https://cloud.githubusercontent.com/assets/1000778/4505774/acb9648c-4afa-11e4-8826-8768a0a60cc9.png) Author: zsxwing Closes #2642 from zsxwing/SPARK-3777 and squashes the following commits: 37945af [zsxwing] Put Executor ID and Host into one cell 4bbe2c7 [zsxwing] [SPARK-3777] Display "Executor ID" for Tasks in Stage page --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index db01be596e073..2414e4c65237e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -103,7 +103,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val taskHeaders: Seq[String] = Seq( - "Index", "ID", "Attempt", "Status", "Locality Level", "Executor", + "Index", "ID", "Attempt", "Status", "Locality Level", "Executor ID / Host", "Launch Time", "Duration", "GC Time", "Accumulators") ++ {if (hasInput) Seq("Input") else Nil} ++ {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ @@ -282,7 +282,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { }
    {info.status} {info.taskLocality}{info.host}{info.executorId} / {info.host} {UIUtils.formatDate(new Date(info.launchTime))} {formatDuration} From 3d7b36e0de26049e8b36b6705d8ff4224bde9eb1 Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Tue, 7 Oct 2014 16:40:16 -0700 Subject: [PATCH 0916/1492] [SPARK-3790][MLlib] CosineSimilarity Example Provide example for `RowMatrix.columnSimilarity()` Author: Reza Zadeh Closes #2622 from rezazadeh/dimsumexample and squashes the following commits: 8f20b82 [Reza Zadeh] update comment 379066d [Reza Zadeh] cache rows 792b81c [Reza Zadeh] Address review comments e573c7a [Reza Zadeh] Average absolute error b15685f [Reza Zadeh] Use scopt. Distribute evaluation. eca3dfd [Reza Zadeh] Documentation ac96fb2 [Reza Zadeh] Compute approximation error, add command line. 4533579 [Reza Zadeh] CosineSimilarity Example --- .../examples/mllib/CosineSimilarity.scala | 107 ++++++++++++++++++ 1 file changed, 107 insertions(+) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala new file mode 100644 index 0000000000000..6a3b0241ced7f --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import scopt.OptionParser + +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.distributed.{MatrixEntry, RowMatrix} +import org.apache.spark.{SparkConf, SparkContext} + +/** + * Compute the similar columns of a matrix, using cosine similarity. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + * + * Example invocation: + * + * bin/run-example mllib.CosineSimilarity \ + * --threshold 0.1 data/mllib/sample_svm_data.txt + */ +object CosineSimilarity { + case class Params(inputFile: String = null, threshold: Double = 0.1) + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("CosineSimilarity") { + head("CosineSimilarity: an example app.") + opt[Double]("threshold") + .required() + .text(s"threshold similarity: to tradeoff computation vs quality estimate") + .action((x, c) => c.copy(threshold = x)) + arg[String]("") + .required() + .text(s"input file, one row per line, space-separated") + .action((x, c) => c.copy(inputFile = x)) + note( + """ + |For example, the following command runs this app on a dataset: + | + | ./bin/spark-submit --class org.apache.spark.examples.mllib.CosineSimilarity \ + | examplesjar.jar \ + | --threshold 0.1 data/mllib/sample_svm_data.txt + """.stripMargin) + } + + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + System.exit(1) + } + } + + def run(params: Params) { + val conf = new SparkConf().setAppName("CosineSimilarity") + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(params.inputFile).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + }.cache() + val mat = new RowMatrix(rows) + + // Compute similar columns perfectly, with brute force. + val exact = mat.columnSimilarities() + + // Compute similar columns with estimation using DIMSUM + val approx = mat.columnSimilarities(params.threshold) + + val exactEntries = exact.entries.map { case MatrixEntry(i, j, u) => ((i, j), u) } + val approxEntries = approx.entries.map { case MatrixEntry(i, j, v) => ((i, j), v) } + val MAE = exactEntries.leftOuterJoin(approxEntries).values.map { + case (u, Some(v)) => + math.abs(u - v) + case (u, None) => + math.abs(u) + }.mean() + + println(s"Average absolute error in estimate is: $MAE") + + sc.stop() + } +} From 098c7344e64e69dffdcf0d95fe1c9e65a54e98f3 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Tue, 7 Oct 2014 16:43:34 -0700 Subject: [PATCH 0917/1492] [SPARK-3486][MLlib][PySpark] PySpark support for Word2Vec mengxr Added PySpark support for Word2Vec Change list (1) PySpark support for Word2Vec (2) SerDe support of string sequence both on python side and JVM side (3) Test for SerDe of string sequence on JVM side Author: Liquan Pei Closes #2356 from Ishiihara/Word2Vec-python and squashes the following commits: 476ea34 [Liquan Pei] style fixes b13a0b9 [Liquan Pei] resolve merge conflicts and minor fixes 8671eba [Liquan Pei] Merge remote-tracking branch 'upstream/master' into Word2Vec-python daf88a6 [Liquan Pei] modification according to feedback a73fa19 [Liquan Pei] clean up 3d8007b [Liquan Pei] fix findSynonyms for vector 1bdcd2e [Liquan Pei] minor fixes cdef9f4 [Liquan Pei] add missing comments b7447eb [Liquan Pei] modify according to feedback b9a7383 [Liquan Pei] cache words RDD in fit 89490bf [Liquan Pei] add tests and Word2VecModelWrapper 78bbb53 [Liquan Pei] use pickle for seq string SerDe a264b08 [Liquan Pei] Merge remote-tracking branch 'upstream/master' into Word2Vec-python ca1e5ff [Liquan Pei] fix test 68e7276 [Liquan Pei] minor style fixes 48d5e72 [Liquan Pei] Functionality improvement 0ad3ac1 [Liquan Pei] minor fix c867fdf [Liquan Pei] add Word2Vec to pyspark --- .../mllib/api/python/PythonMLLibAPI.scala | 57 +++++- .../apache/spark/mllib/feature/Word2Vec.scala | 12 +- python/docs/pyspark.mllib.rst | 8 + python/pyspark/mllib/feature.py | 193 ++++++++++++++++++ python/run-tests | 1 + 5 files changed, 264 insertions(+), 7 deletions(-) create mode 100644 python/pyspark/mllib/feature.py diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index e9f41758581e3..f7251e65e04f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -29,6 +29,8 @@ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ +import org.apache.spark.mllib.feature.Word2Vec +import org.apache.spark.mllib.feature.Word2VecModel import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.random.{RandomRDDs => RG} @@ -42,9 +44,9 @@ import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils - /** * :: DeveloperApi :: * The Java stubs necessary for the Python mllib bindings. @@ -287,6 +289,59 @@ class PythonMLLibAPI extends Serializable { ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) } + /** + * Java stub for Python mllib Word2Vec fit(). This stub returns a + * handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on + * exit; see the Py4J documentation. + * @param dataJRDD input JavaRDD + * @param vectorSize size of vector + * @param learningRate initial learning rate + * @param numPartitions number of partitions + * @param numIterations number of iterations + * @param seed initial seed for random generator + * @return A handle to java Word2VecModelWrapper instance at python side + */ + def trainWord2Vec( + dataJRDD: JavaRDD[java.util.ArrayList[String]], + vectorSize: Int, + learningRate: Double, + numPartitions: Int, + numIterations: Int, + seed: Long): Word2VecModelWrapper = { + val data = dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER) + val word2vec = new Word2Vec() + .setVectorSize(vectorSize) + .setLearningRate(learningRate) + .setNumPartitions(numPartitions) + .setNumIterations(numIterations) + .setSeed(seed) + val model = word2vec.fit(data) + data.unpersist() + new Word2VecModelWrapper(model) + } + + private[python] class Word2VecModelWrapper(model: Word2VecModel) { + def transform(word: String): Vector = { + model.transform(word) + } + + def findSynonyms(word: String, num: Int): java.util.List[java.lang.Object] = { + val vec = transform(word) + findSynonyms(vec, num) + } + + def findSynonyms(vector: Vector, num: Int): java.util.List[java.lang.Object] = { + val result = model.findSynonyms(vector, num) + val similarity = Vectors.dense(result.map(_._2)) + val words = result.map(_._1) + val ret = new java.util.LinkedList[java.lang.Object]() + ret.add(words) + ret.add(similarity) + ret + } + } + /** * Java stub for Python mllib DecisionTree.train(). * This stub returns a handle to the Java object instead of the content of the Java object. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index fc1444705364a..d321994c2a651 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -67,7 +67,7 @@ private case class VocabWord( class Word2Vec extends Serializable with Logging { private var vectorSize = 100 - private var startingAlpha = 0.025 + private var learningRate = 0.025 private var numPartitions = 1 private var numIterations = 1 private var seed = Utils.random.nextLong() @@ -84,7 +84,7 @@ class Word2Vec extends Serializable with Logging { * Sets initial learning rate (default: 0.025). */ def setLearningRate(learningRate: Double): this.type = { - this.startingAlpha = learningRate + this.learningRate = learningRate this } @@ -286,7 +286,7 @@ class Word2Vec extends Serializable with Logging { val syn0Global = Array.fill[Float](vocabSize * vectorSize)((initRandom.nextFloat() - 0.5f) / vectorSize) val syn1Global = new Array[Float](vocabSize * vectorSize) - var alpha = startingAlpha + var alpha = learningRate for (k <- 1 to numIterations) { val partial = newSentences.mapPartitionsWithIndex { case (idx, iter) => val random = new XORShiftRandom(seed ^ ((idx + 1) << 16) ^ ((-k - 1) << 8)) @@ -300,8 +300,8 @@ class Word2Vec extends Serializable with Logging { lwc = wordCount // TODO: discount by iteration? alpha = - startingAlpha * (1 - numPartitions * wordCount.toDouble / (trainWordsCount + 1)) - if (alpha < startingAlpha * 0.0001) alpha = startingAlpha * 0.0001 + learningRate * (1 - numPartitions * wordCount.toDouble / (trainWordsCount + 1)) + if (alpha < learningRate * 0.0001) alpha = learningRate * 0.0001 logInfo("wordCount = " + wordCount + ", alpha = " + alpha) } wc += sentence.size @@ -437,7 +437,7 @@ class Word2VecModel private[mllib] ( * Find synonyms of a word * @param word a word * @param num number of synonyms to find - * @return array of (word, similarity) + * @return array of (word, cosineSimilarity) */ def findSynonyms(word: String, num: Int): Array[(String, Double)] = { val vector = transform(word) diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index e95d19e97f151..4548b8739ed91 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -20,6 +20,14 @@ pyspark.mllib.clustering module :undoc-members: :show-inheritance: +pyspark.mllib.feature module +------------------------------- + +.. automodule:: pyspark.mllib.feature + :members: + :undoc-members: + :show-inheritance: + pyspark.mllib.linalg module --------------------------- diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py new file mode 100644 index 0000000000000..a44a27fd3b6a6 --- /dev/null +++ b/python/pyspark/mllib/feature.py @@ -0,0 +1,193 @@ +# +# 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. +# + +""" +Python package for feature in MLlib. +""" +from pyspark.serializers import PickleSerializer, AutoBatchedSerializer + +from pyspark.mllib.linalg import _convert_to_vector + +__all__ = ['Word2Vec', 'Word2VecModel'] + + +class Word2VecModel(object): + """ + class for Word2Vec model + """ + def __init__(self, sc, java_model): + """ + :param sc: Spark context + :param java_model: Handle to Java model object + """ + self._sc = sc + self._java_model = java_model + + def __del__(self): + self._sc._gateway.detach(self._java_model) + + def transform(self, word): + """ + :param word: a word + :return: vector representation of word + Transforms a word to its vector representation + + Note: local use only + """ + # TODO: make transform usable in RDD operations from python side + result = self._java_model.transform(word) + return PickleSerializer().loads(str(self._sc._jvm.SerDe.dumps(result))) + + def findSynonyms(self, x, num): + """ + :param x: a word or a vector representation of word + :param num: number of synonyms to find + :return: array of (word, cosineSimilarity) + Find synonyms of a word + + Note: local use only + """ + # TODO: make findSynonyms usable in RDD operations from python side + ser = PickleSerializer() + if type(x) == str: + jlist = self._java_model.findSynonyms(x, num) + else: + bytes = bytearray(ser.dumps(_convert_to_vector(x))) + vec = self._sc._jvm.SerDe.loads(bytes) + jlist = self._java_model.findSynonyms(vec, num) + words, similarity = ser.loads(str(self._sc._jvm.SerDe.dumps(jlist))) + return zip(words, similarity) + + +class Word2Vec(object): + """ + Word2Vec creates vector representation of words in a text corpus. + The algorithm first constructs a vocabulary from the corpus + and then learns vector representation of words in the vocabulary. + The vector representation can be used as features in + natural language processing and machine learning algorithms. + + We used skip-gram model in our implementation and hierarchical softmax + method to train the model. The variable names in the implementation + matches the original C implementation. + For original C implementation, see https://code.google.com/p/word2vec/ + For research papers, see + Efficient Estimation of Word Representations in Vector Space + and + Distributed Representations of Words and Phrases and their Compositionality. + + >>> sentence = "a b " * 100 + "a c " * 10 + >>> localDoc = [sentence, sentence] + >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) + >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> syms = model.findSynonyms("a", 2) + >>> str(syms[0][0]) + 'b' + >>> str(syms[1][0]) + 'c' + >>> len(syms) + 2 + >>> vec = model.transform("a") + >>> len(vec) + 10 + >>> syms = model.findSynonyms(vec, 2) + >>> str(syms[0][0]) + 'b' + >>> str(syms[1][0]) + 'c' + >>> len(syms) + 2 + """ + def __init__(self): + """ + Construct Word2Vec instance + """ + self.vectorSize = 100 + self.learningRate = 0.025 + self.numPartitions = 1 + self.numIterations = 1 + self.seed = 42L + + def setVectorSize(self, vectorSize): + """ + Sets vector size (default: 100). + """ + self.vectorSize = vectorSize + return self + + def setLearningRate(self, learningRate): + """ + Sets initial learning rate (default: 0.025). + """ + self.learningRate = learningRate + return self + + def setNumPartitions(self, numPartitions): + """ + Sets number of partitions (default: 1). Use a small number for accuracy. + """ + self.numPartitions = numPartitions + return self + + def setNumIterations(self, numIterations): + """ + Sets number of iterations (default: 1), which should be smaller than or equal to number of + partitions. + """ + self.numIterations = numIterations + return self + + def setSeed(self, seed): + """ + Sets random seed. + """ + self.seed = seed + return self + + def fit(self, data): + """ + Computes the vector representation of each word in vocabulary. + + :param data: training data. RDD of subtype of Iterable[String] + :return: python Word2VecModel instance + """ + sc = data.context + ser = PickleSerializer() + vectorSize = self.vectorSize + learningRate = self.learningRate + numPartitions = self.numPartitions + numIterations = self.numIterations + seed = self.seed + + model = sc._jvm.PythonMLLibAPI().trainWord2Vec( + data._to_java_object_rdd(), vectorSize, + learningRate, numPartitions, numIterations, seed) + return Word2VecModel(sc, model) + + +def _test(): + import doctest + from pyspark import SparkContext + globs = globals().copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + +if __name__ == "__main__": + _test() diff --git a/python/run-tests b/python/run-tests index c713861eb77bb..63395f72788f9 100755 --- a/python/run-tests +++ b/python/run-tests @@ -69,6 +69,7 @@ function run_mllib_tests() { echo "Run mllib tests ..." run_test "pyspark/mllib/classification.py" run_test "pyspark/mllib/clustering.py" + run_test "pyspark/mllib/feature.py" run_test "pyspark/mllib/linalg.py" run_test "pyspark/mllib/random.py" run_test "pyspark/mllib/recommendation.py" From b32bb72e812731d28bf05f2145314c63806f3335 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 7 Oct 2014 16:47:24 -0700 Subject: [PATCH 0918/1492] [SPARK-3832][MLlib] Upgrade Breeze dependency to 0.10 In Breeze 0.10, the L1regParam can be configured through anonymous function in OWLQN, and each component can be penalized differently. This is required for GLMNET in MLlib with L1/L2 regularization. https://github.com/scalanlp/breeze/commit/2570911026aa05aa1908ccf7370bc19cd8808a4c Author: DB Tsai Closes #2693 from dbtsai/breeze0.10 and squashes the following commits: 7a0c45c [DB Tsai] In Breeze 0.10, the L1regParam can be configured through anonymous function in OWLQN, and each component can be penalized differently. This is required for GLMNET in MLlib with L1/L2 regularization. https://github.com/scalanlp/breeze/commit/2570911026aa05aa1908ccf7370bc19cd8808a4c --- mllib/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/pom.xml b/mllib/pom.xml index a5eeef88e9d62..696e9396f627c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -57,7 +57,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.9 + 0.10 From 5912ca67140eed5dea66745aa3af4febdbd80781 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Tue, 7 Oct 2014 16:54:32 -0700 Subject: [PATCH 0919/1492] [SPARK-3398] [EC2] Have spark-ec2 intelligently wait for specific cluster states Instead of waiting arbitrary amounts of time for the cluster to reach a specific state, this patch lets `spark-ec2` explicitly wait for a cluster to reach a desired state. This is useful in a couple of situations: * The cluster is launching and you want to wait until SSH is available before installing stuff. * The cluster is being terminated and you want to wait until all the instances are terminated before trying to delete security groups. This patch removes the need for the `--wait` option and removes some of the time-based retry logic that was being used. Author: Nicholas Chammas Closes #2339 from nchammas/spark-ec2-wait-properly and squashes the following commits: 43a69f0 [Nicholas Chammas] short-circuit SSH check; linear backoff 9a9e035 [Nicholas Chammas] remove extraneous comment 26c5ed0 [Nicholas Chammas] replace print with write() bb67c06 [Nicholas Chammas] deprecate wait option; remove dead code 7969265 [Nicholas Chammas] fix long line (PEP 8) 126e4cf [Nicholas Chammas] wait for specific cluster states --- ec2/spark_ec2.py | 111 ++++++++++++++++++++++++++++++++++++----------- 1 file changed, 86 insertions(+), 25 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 941dfb988b9fb..27f468ea4f395 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -32,6 +32,7 @@ import tempfile import time import urllib2 +import warnings from optparse import OptionParser from sys import stderr import boto @@ -61,8 +62,8 @@ def parse_args(): "-s", "--slaves", type="int", default=1, help="Number of slaves to launch (default: %default)") parser.add_option( - "-w", "--wait", type="int", default=120, - help="Seconds to wait for nodes to start (default: %default)") + "-w", "--wait", type="int", + help="DEPRECATED (no longer necessary) - Seconds to wait for nodes to start") parser.add_option( "-k", "--key-pair", help="Key pair to use on instances") @@ -195,18 +196,6 @@ def get_or_make_group(conn, name): return conn.create_security_group(name, "Spark EC2 group") -# Wait for a set of launched instances to exit the "pending" state -# (i.e. either to start running or to fail and be terminated) -def wait_for_instances(conn, instances): - while True: - for i in instances: - i.update() - if len([i for i in instances if i.state == 'pending']) > 0: - time.sleep(5) - else: - return - - # Check whether a given EC2 instance object is in a state we consider active, # i.e. not terminating or terminated. We count both stopping and stopped as # active since we can restart stopped clusters. @@ -619,14 +608,64 @@ def setup_spark_cluster(master, opts): print "Ganglia started at http://%s:5080/ganglia" % master -# Wait for a whole cluster (masters, slaves and ZooKeeper) to start up -def wait_for_cluster(conn, wait_secs, master_nodes, slave_nodes): - print "Waiting for instances to start up..." - time.sleep(5) - wait_for_instances(conn, master_nodes) - wait_for_instances(conn, slave_nodes) - print "Waiting %d more seconds..." % wait_secs - time.sleep(wait_secs) +def is_ssh_available(host, opts): + "Checks if SSH is available on the host." + try: + with open(os.devnull, 'w') as devnull: + ret = subprocess.check_call( + ssh_command(opts) + ['-t', '-t', '-o', 'ConnectTimeout=3', + '%s@%s' % (opts.user, host), stringify_command('true')], + stdout=devnull, + stderr=devnull + ) + return ret == 0 + except subprocess.CalledProcessError as e: + return False + + +def is_cluster_ssh_available(cluster_instances, opts): + for i in cluster_instances: + if not is_ssh_available(host=i.ip_address, opts=opts): + return False + else: + return True + + +def wait_for_cluster_state(cluster_instances, cluster_state, opts): + """ + cluster_instances: a list of boto.ec2.instance.Instance + cluster_state: a string representing the desired state of all the instances in the cluster + value can be 'ssh-ready' or a valid value from boto.ec2.instance.InstanceState such as + 'running', 'terminated', etc. + (would be nice to replace this with a proper enum: http://stackoverflow.com/a/1695250) + """ + sys.stdout.write( + "Waiting for all instances in cluster to enter '{s}' state.".format(s=cluster_state) + ) + sys.stdout.flush() + + num_attempts = 0 + + while True: + time.sleep(3 * num_attempts) + + for i in cluster_instances: + s = i.update() # capture output to suppress print to screen in newer versions of boto + + if cluster_state == 'ssh-ready': + if all(i.state == 'running' for i in cluster_instances) and \ + is_cluster_ssh_available(cluster_instances, opts): + break + else: + if all(i.state == cluster_state for i in cluster_instances): + break + + num_attempts += 1 + + sys.stdout.write(".") + sys.stdout.flush() + + sys.stdout.write("\n") # Get number of local disks available for a given EC2 instance type. @@ -868,6 +907,16 @@ def real_main(): (opts, action, cluster_name) = parse_args() # Input parameter validation + if opts.wait is not None: + # NOTE: DeprecationWarnings are silent in 2.7+ by default. + # To show them, run Python with the -Wdefault switch. + # See: https://docs.python.org/3.5/whatsnew/2.7.html + warnings.warn( + "This option is deprecated and has no effect. " + "spark-ec2 automatically waits as long as necessary for clusters to startup.", + DeprecationWarning + ) + if opts.ebs_vol_num > 8: print >> stderr, "ebs-vol-num cannot be greater than 8" sys.exit(1) @@ -890,7 +939,11 @@ def real_main(): (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) else: (master_nodes, slave_nodes) = launch_cluster(conn, opts, cluster_name) - wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes) + wait_for_cluster_state( + cluster_instances=(master_nodes + slave_nodes), + cluster_state='ssh-ready', + opts=opts + ) setup_cluster(conn, master_nodes, slave_nodes, opts, True) elif action == "destroy": @@ -919,7 +972,11 @@ def real_main(): else: group_names = [opts.security_group_prefix + "-master", opts.security_group_prefix + "-slaves"] - + wait_for_cluster_state( + cluster_instances=(master_nodes + slave_nodes), + cluster_state='terminated', + opts=opts + ) attempt = 1 while attempt <= 3: print "Attempt %d" % attempt @@ -1019,7 +1076,11 @@ def real_main(): for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() - wait_for_cluster(conn, opts.wait, master_nodes, slave_nodes) + wait_for_cluster_state( + cluster_instances=(master_nodes + slave_nodes), + cluster_state='ssh-ready', + opts=opts + ) setup_cluster(conn, master_nodes, slave_nodes, opts, False) else: From b69c9fb6fb048509bbd8430fb697dc3a5ca4fe59 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 7 Oct 2014 16:54:49 -0700 Subject: [PATCH 0920/1492] [SPARK-3829] Make Spark logo image on the header of HistoryPage as a link to HistoryPage's page #1 There is a Spark logo on the header of HistoryPage. We can have too many HistoryPages if we run 20+ applications. So I think, it's useful if the logo is as a link to the HistoryPage's page number 1. Author: Kousuke Saruta Closes #2690 from sarutak/SPARK-3829 and squashes the following commits: 908c109 [Kousuke Saruta] Removed extra space. 00bfbd7 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3829 dd87480 [Kousuke Saruta] Made header Spark log image as a link to History Server's top page. --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 f0006b42aee4f..be69060fc3bf8 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -216,8 +216,10 @@ private[spark] object UIUtils extends Logging {

    - + + + {title}

    From 798ed22c289cf65f2249bf2f4250285685ca69e7 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 7 Oct 2014 18:09:27 -0700 Subject: [PATCH 0921/1492] [SPARK-3412] [PySpark] Replace Epydoc with Sphinx to generate Python API docs Retire Epydoc, use Sphinx to generate API docs. Refine Sphinx docs, also convert some docstrings into Sphinx style. It looks like: ![api doc](https://cloud.githubusercontent.com/assets/40902/4538272/9e2d4f10-4dec-11e4-8d96-6e45a8fe51f9.png) Author: Davies Liu Closes #2689 from davies/docs and squashes the following commits: bf4a0a5 [Davies Liu] fix links 3fb1572 [Davies Liu] fix _static in jekyll 65a287e [Davies Liu] fix scripts and logo 8524042 [Davies Liu] Merge branch 'master' of github.com:apache/spark into docs d5b874a [Davies Liu] Merge branch 'master' of github.com:apache/spark into docs 4bc1c3c [Davies Liu] refactor 746d0b6 [Davies Liu] @param -> :param 240b393 [Davies Liu] replace epydoc with sphinx doc --- docs/README.md | 8 +-- docs/_config.yml | 3 + docs/_plugins/copy_api_dirs.rb | 19 +++--- python/docs/conf.py | 12 ++-- python/docs/index.rst | 6 +- python/epydoc.conf | 38 ----------- python/pyspark/__init__.py | 26 ++------ python/pyspark/conf.py | 8 +-- python/pyspark/context.py | 92 +++++++++++++------------- python/pyspark/mllib/classification.py | 32 ++++----- python/pyspark/mllib/linalg.py | 8 +-- python/pyspark/mllib/regression.py | 18 ++--- python/pyspark/mllib/util.py | 18 ++--- python/pyspark/rdd.py | 52 +++++++-------- python/pyspark/sql.py | 33 +++++---- 15 files changed, 167 insertions(+), 206 deletions(-) delete mode 100644 python/epydoc.conf diff --git a/docs/README.md b/docs/README.md index 79708c3df9106..0facecdd5f767 100644 --- a/docs/README.md +++ b/docs/README.md @@ -54,19 +54,19 @@ phase, use the following sytax: // supported languages too. {% endhighlight %} -## API Docs (Scaladoc and Epydoc) +## API Docs (Scaladoc and Sphinx) You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. -Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the -SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as +Similarly, you can build just the PySpark docs by running `make html` from the +SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as public in `__init__.py`. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the -PySpark docs using [epydoc](http://epydoc.sourceforge.net/). +PySpark docs [Sphinx](http://sphinx-doc.org/). NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_config.yml b/docs/_config.yml index 7bc3a78e2d265..f4bf242ac191b 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -8,6 +8,9 @@ gems: kramdown: entity_output: numeric +include: + - _static + # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. SPARK_VERSION: 1.0.0-SNAPSHOT diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 3b02e090aec28..4566a2fff562b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -63,19 +63,20 @@ puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) - # Build Epydoc for Python - puts "Moving to python directory and building epydoc." - cd("../python") - puts `epydoc --config epydoc.conf` + # Build Sphinx docs for Python - puts "Moving back into docs dir." - cd("../docs") + puts "Moving to python/docs directory and building sphinx." + cd("../python/docs") + puts `make html` + + puts "Moving back into home dir." + cd("../../") puts "Making directory api/python" - mkdir_p "api/python" + mkdir_p "docs/api/python" - puts "cp -r ../python/docs/. api/python" - cp_r("../python/docs/.", "api/python") + puts "cp -r python/docs/_build/html/. docs/api/python" + cp_r("python/docs/_build/html/.", "docs/api/python") cd("..") end diff --git a/python/docs/conf.py b/python/docs/conf.py index c368cf81a003b..8e6324f058251 100644 --- a/python/docs/conf.py +++ b/python/docs/conf.py @@ -55,9 +55,9 @@ # built documents. # # The short X.Y version. -version = '1.1' +version = '1.2-SNAPSHOT' # The full version, including alpha/beta/rc tags. -release = '' +release = '1.2-SNAPSHOT' # The language for content autogenerated by Sphinx. Refer to documentation # for a list of supported languages. @@ -102,7 +102,7 @@ # The theme to use for HTML and HTML Help pages. See the documentation for # a list of builtin themes. -html_theme = 'default' +html_theme = 'nature' # Theme options are theme-specific and customize the look and feel of a theme # further. For a list of options available for each theme, see the @@ -121,7 +121,7 @@ # The name of an image file (relative to this directory) to place at the top # of the sidebar. -#html_logo = None +html_logo = "../../docs/img/spark-logo-hd.png" # The name of an image file (within the static path) to use as favicon of the # docs. This file should be a Windows icon file (.ico) being 16x16 or 32x32 @@ -154,10 +154,10 @@ #html_additional_pages = {} # If false, no module index is generated. -#html_domain_indices = True +html_domain_indices = False # If false, no index is generated. -#html_use_index = True +html_use_index = False # If true, the index is split into individual pages for each letter. #html_split_index = False diff --git a/python/docs/index.rst b/python/docs/index.rst index 25b3f9bd93e63..d66e051b15371 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -3,7 +3,7 @@ You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. -Welcome to PySpark API reference! +Welcome to Spark Python API Docs! =================================== Contents: @@ -24,14 +24,12 @@ Core classes: Main entry point for Spark functionality. :class:`pyspark.RDD` - + A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. Indices and tables ================== -* :ref:`genindex` -* :ref:`modindex` * :ref:`search` diff --git a/python/epydoc.conf b/python/epydoc.conf deleted file mode 100644 index 8593e08deda19..0000000000000 --- a/python/epydoc.conf +++ /dev/null @@ -1,38 +0,0 @@ -[epydoc] # Epydoc section marker (required by ConfigParser) - -# -# 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. -# - -# Information about the project. -name: Spark 1.0.0 Python API Docs -url: http://spark.apache.org - -# The list of modules to document. Modules can be named using -# dotted names, module filenames, or package directory names. -# This option may be repeated. -modules: pyspark - -# Write html output to the directory "apidocs" -output: html -target: docs/ - -private: no - -exclude: pyspark.cloudpickle pyspark.worker pyspark.join - pyspark.java_gateway pyspark.examples pyspark.shell pyspark.tests - pyspark.rddsampler pyspark.daemon - pyspark.mllib.tests pyspark.shuffle diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py index 1a2e774738fe7..e39e6514d77a1 100644 --- a/python/pyspark/__init__.py +++ b/python/pyspark/__init__.py @@ -20,33 +20,21 @@ Public classes: - - L{SparkContext} + - :class:`SparkContext`: Main entry point for Spark functionality. - - L{RDD} + - L{RDD} A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. - - L{Broadcast} + - L{Broadcast} A broadcast variable that gets reused across tasks. - - L{Accumulator} + - L{Accumulator} An "add-only" shared variable that tasks can only add values to. - - L{SparkConf} + - L{SparkConf} For configuring Spark. - - L{SparkFiles} + - L{SparkFiles} Access files shipped with jobs. - - L{StorageLevel} + - L{StorageLevel} Finer-grained cache persistence levels. -Spark SQL: - - L{SQLContext} - Main entry point for SQL functionality. - - L{SchemaRDD} - A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In - addition to normal RDD operations, SchemaRDDs also support SQL. - - L{Row} - A Row of data returned by a Spark SQL query. - -Hive: - - L{HiveContext} - Main entry point for accessing data stored in Apache Hive.. """ # The following block allows us to import python's random instead of mllib.random for scripts in diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index b64875a3f495a..dc7cd0bce56f3 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -83,11 +83,11 @@ def __init__(self, loadDefaults=True, _jvm=None, _jconf=None): """ Create a new Spark configuration. - @param loadDefaults: whether to load values from Java system + :param loadDefaults: whether to load values from Java system properties (True by default) - @param _jvm: internal parameter used to pass a handle to the + :param _jvm: internal parameter used to pass a handle to the Java VM; does not need to be set by users - @param _jconf: Optionally pass in an existing SparkConf handle + :param _jconf: Optionally pass in an existing SparkConf handle to use its parameters """ if _jconf: @@ -139,7 +139,7 @@ def setAll(self, pairs): """ Set multiple parameters, passed as a list of key-value pairs. - @param pairs: list of key-value pairs to set + :param pairs: list of key-value pairs to set """ for (k, v) in pairs: self._jconf.set(k, v) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index a45d79d6424c7..6fb30d65c5edd 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -73,21 +73,21 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, Create a new SparkContext. At least the master and app name should be set, either through the named parameters here or through C{conf}. - @param master: Cluster URL to connect to + :param master: Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). - @param appName: A name for your job, to display on the cluster web UI. - @param sparkHome: Location where Spark is installed on cluster nodes. - @param pyFiles: Collection of .zip or .py files to send to the cluster + :param appName: A name for your job, to display on the cluster web UI. + :param sparkHome: Location where Spark is installed on cluster nodes. + :param pyFiles: Collection of .zip or .py files to send to the cluster and add to PYTHONPATH. These can be paths on the local file system or HDFS, HTTP, HTTPS, or FTP URLs. - @param environment: A dictionary of environment variables to set on + :param environment: A dictionary of environment variables to set on worker nodes. - @param batchSize: The number of Python objects represented as a single + :param batchSize: The number of Python objects represented as a single Java object. Set 1 to disable batching or -1 to use an unlimited batch size. - @param serializer: The serializer for RDDs. - @param conf: A L{SparkConf} object setting Spark properties. - @param gateway: Use an existing gateway and JVM, otherwise a new JVM + :param serializer: The serializer for RDDs. + :param conf: A L{SparkConf} object setting Spark properties. + :param gateway: Use an existing gateway and JVM, otherwise a new JVM will be instantiated. @@ -417,16 +417,16 @@ def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, 3. If this fails, the fallback is to call 'toString' on each key and value 4. C{PickleSerializer} is used to deserialize pickled objects on the Python side - @param path: path to sequncefile - @param keyClass: fully qualified classname of key Writable class + :param path: path to sequncefile + :param keyClass: fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") - @param valueClass: fully qualified classname of value Writable class + :param valueClass: fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") - @param keyConverter: - @param valueConverter: - @param minSplits: minimum splits in dataset + :param keyConverter: + :param valueConverter: + :param minSplits: minimum splits in dataset (default min(2, sc.defaultParallelism)) - @param batchSize: The number of Python objects represented as a single + :param batchSize: The number of Python objects represented as a single Java object. (default sc._default_batch_size_for_serialized_input) """ minSplits = minSplits or min(self.defaultParallelism, 2) @@ -446,18 +446,18 @@ def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConv A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java - @param path: path to Hadoop file - @param inputFormatClass: fully qualified classname of Hadoop InputFormat + :param path: path to Hadoop file + :param inputFormatClass: fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat") - @param keyClass: fully qualified classname of key Writable class + :param keyClass: fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") - @param valueClass: fully qualified classname of value Writable class + :param valueClass: fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") - @param keyConverter: (None by default) - @param valueConverter: (None by default) - @param conf: Hadoop configuration, passed in as a dict + :param keyConverter: (None by default) + :param valueConverter: (None by default) + :param conf: Hadoop configuration, passed in as a dict (None by default) - @param batchSize: The number of Python objects represented as a single + :param batchSize: The number of Python objects represented as a single Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) @@ -476,17 +476,17 @@ def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=N This will be converted into a Configuration in Java. The mechanism is the same as for sc.sequenceFile. - @param inputFormatClass: fully qualified classname of Hadoop InputFormat + :param inputFormatClass: fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapreduce.lib.input.TextInputFormat") - @param keyClass: fully qualified classname of key Writable class + :param keyClass: fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") - @param valueClass: fully qualified classname of value Writable class + :param valueClass: fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") - @param keyConverter: (None by default) - @param valueConverter: (None by default) - @param conf: Hadoop configuration, passed in as a dict + :param keyConverter: (None by default) + :param valueConverter: (None by default) + :param conf: Hadoop configuration, passed in as a dict (None by default) - @param batchSize: The number of Python objects represented as a single + :param batchSize: The number of Python objects represented as a single Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) @@ -507,18 +507,18 @@ def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter= A Hadoop configuration can be passed in as a Python dict. This will be converted into a Configuration in Java. - @param path: path to Hadoop file - @param inputFormatClass: fully qualified classname of Hadoop InputFormat + :param path: path to Hadoop file + :param inputFormatClass: fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapred.TextInputFormat") - @param keyClass: fully qualified classname of key Writable class + :param keyClass: fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") - @param valueClass: fully qualified classname of value Writable class + :param valueClass: fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") - @param keyConverter: (None by default) - @param valueConverter: (None by default) - @param conf: Hadoop configuration, passed in as a dict + :param keyConverter: (None by default) + :param valueConverter: (None by default) + :param conf: Hadoop configuration, passed in as a dict (None by default) - @param batchSize: The number of Python objects represented as a single + :param batchSize: The number of Python objects represented as a single Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) @@ -537,17 +537,17 @@ def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, This will be converted into a Configuration in Java. The mechanism is the same as for sc.sequenceFile. - @param inputFormatClass: fully qualified classname of Hadoop InputFormat + :param inputFormatClass: fully qualified classname of Hadoop InputFormat (e.g. "org.apache.hadoop.mapred.TextInputFormat") - @param keyClass: fully qualified classname of key Writable class + :param keyClass: fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.Text") - @param valueClass: fully qualified classname of value Writable class + :param valueClass: fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.LongWritable") - @param keyConverter: (None by default) - @param valueConverter: (None by default) - @param conf: Hadoop configuration, passed in as a dict + :param keyConverter: (None by default) + :param valueConverter: (None by default) + :param conf: Hadoop configuration, passed in as a dict (None by default) - @param batchSize: The number of Python objects represented as a single + :param batchSize: The number of Python objects represented as a single Java object. (default sc._default_batch_size_for_serialized_input) """ jconf = self._dictToJavaMap(conf) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index a765b1c4f7d87..cd43982191702 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -79,15 +79,15 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, """ Train a logistic regression model on the given data. - @param data: The training data. - @param iterations: The number of iterations (default: 100). - @param step: The step parameter used in SGD + :param data: The training data. + :param iterations: The number of iterations (default: 100). + :param step: The step parameter used in SGD (default: 1.0). - @param miniBatchFraction: Fraction of data to be used for each SGD + :param miniBatchFraction: Fraction of data to be used for each SGD iteration. - @param initialWeights: The initial weights (default: None). - @param regParam: The regularizer parameter (default: 1.0). - @param regType: The type of regularizer used for training + :param initialWeights: The initial weights (default: None). + :param regParam: The regularizer parameter (default: 1.0). + :param regType: The type of regularizer used for training our model. :Allowed values: @@ -151,15 +151,15 @@ def train(cls, data, iterations=100, step=1.0, regParam=1.0, """ Train a support vector machine on the given data. - @param data: The training data. - @param iterations: The number of iterations (default: 100). - @param step: The step parameter used in SGD + :param data: The training data. + :param iterations: The number of iterations (default: 100). + :param step: The step parameter used in SGD (default: 1.0). - @param regParam: The regularizer parameter (default: 1.0). - @param miniBatchFraction: Fraction of data to be used for each SGD + :param regParam: The regularizer parameter (default: 1.0). + :param miniBatchFraction: Fraction of data to be used for each SGD iteration. - @param initialWeights: The initial weights (default: None). - @param regType: The type of regularizer used for training + :param initialWeights: The initial weights (default: None). + :param regType: The type of regularizer used for training our model. :Allowed values: @@ -238,10 +238,10 @@ def train(cls, data, lambda_=1.0): classification. By making every vector a 0-1 vector, it can also be used as Bernoulli NB (U{http://tinyurl.com/p7c96j6}). - @param data: RDD of NumPy vectors, one per element, where the first + :param data: RDD of NumPy vectors, one per element, where the first coordinate is the label and the rest is the feature vector (e.g. a count vector). - @param lambda_: The smoothing parameter + :param lambda_: The smoothing parameter """ sc = data.context jlist = sc._jvm.PythonMLLibAPI().trainNaiveBayes(data._to_java_object_rdd(), lambda_) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 51014a8ceb785..24c5480b2f753 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -238,8 +238,8 @@ def __init__(self, size, *args): (index, value) pairs, or two separate arrays of indices and values (sorted by index). - @param size: Size of the vector. - @param args: Non-zero entries, as a dictionary, list of tupes, + :param size: Size of the vector. + :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. >>> print SparseVector(4, {1: 1.0, 3: 5.5}) @@ -458,8 +458,8 @@ def sparse(size, *args): (index, value) pairs, or two separate arrays of indices and values (sorted by index). - @param size: Size of the vector. - @param args: Non-zero entries, as a dictionary, list of tupes, + :param size: Size of the vector. + :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. >>> print Vectors.sparse(4, {1: 1.0, 3: 5.5}) diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index 54f34a98337ca..12b322aaae796 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -31,8 +31,8 @@ class LabeledPoint(object): """ The features and labels of a data point. - @param label: Label for this data point. - @param features: Vector of features for this point (NumPy array, list, + :param label: Label for this data point. + :param features: Vector of features for this point (NumPy array, list, pyspark.mllib.linalg.SparseVector, or scipy.sparse column matrix) """ @@ -145,15 +145,15 @@ def train(cls, data, iterations=100, step=1.0, miniBatchFraction=1.0, """ Train a linear regression model on the given data. - @param data: The training data. - @param iterations: The number of iterations (default: 100). - @param step: The step parameter used in SGD + :param data: The training data. + :param iterations: The number of iterations (default: 100). + :param step: The step parameter used in SGD (default: 1.0). - @param miniBatchFraction: Fraction of data to be used for each SGD + :param miniBatchFraction: Fraction of data to be used for each SGD iteration. - @param initialWeights: The initial weights (default: None). - @param regParam: The regularizer parameter (default: 1.0). - @param regType: The type of regularizer used for training + :param initialWeights: The initial weights (default: None). + :param regParam: The regularizer parameter (default: 1.0). + :param regType: The type of regularizer used for training our model. :Allowed values: diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index 8233d4e81f1ca..1357fd4fbc8aa 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -77,10 +77,10 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None method parses each line into a LabeledPoint, where the feature indices are converted to zero-based. - @param sc: Spark context - @param path: file or directory path in any Hadoop-supported file + :param sc: Spark context + :param path: file or directory path in any Hadoop-supported file system URI - @param numFeatures: number of features, which will be determined + :param numFeatures: number of features, which will be determined from the input data if a nonpositive value is given. This is useful when the dataset is already split into multiple files and you @@ -88,7 +88,7 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None features may not present in certain files, which leads to inconsistent feature dimensions. - @param minPartitions: min number of partitions + :param minPartitions: min number of partitions @return: labeled data stored as an RDD of LabeledPoint >>> from tempfile import NamedTemporaryFile @@ -126,8 +126,8 @@ def saveAsLibSVMFile(data, dir): """ Save labeled data in LIBSVM format. - @param data: an RDD of LabeledPoint to be saved - @param dir: directory to save the data + :param data: an RDD of LabeledPoint to be saved + :param dir: directory to save the data >>> from tempfile import NamedTemporaryFile >>> from fileinput import input @@ -149,10 +149,10 @@ def loadLabeledPoints(sc, path, minPartitions=None): """ Load labeled points saved using RDD.saveAsTextFile. - @param sc: Spark context - @param path: file or directory path in any Hadoop-supported file + :param sc: Spark context + :param path: file or directory path in any Hadoop-supported file system URI - @param minPartitions: min number of partitions + :param minPartitions: min number of partitions @return: labeled data stored as an RDD of LabeledPoint >>> from tempfile import NamedTemporaryFile diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index e77669aad76b6..6797d50659a92 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -752,7 +752,7 @@ def max(self, key=None): """ Find the maximum item in this RDD. - @param key: A function used to generate key for comparing + :param key: A function used to generate key for comparing >>> rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0]) >>> rdd.max() @@ -768,7 +768,7 @@ def min(self, key=None): """ Find the minimum item in this RDD. - @param key: A function used to generate key for comparing + :param key: A function used to generate key for comparing >>> rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0]) >>> rdd.min() @@ -1115,9 +1115,9 @@ def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None converted for output using either user specified converters or, by default, L{org.apache.spark.api.python.JavaToWritableConverter}. - @param conf: Hadoop job configuration, passed in as a dict - @param keyConverter: (None by default) - @param valueConverter: (None by default) + :param conf: Hadoop job configuration, passed in as a dict + :param keyConverter: (None by default) + :param valueConverter: (None by default) """ jconf = self.ctx._dictToJavaMap(conf) pickledRDD = self._toPickleSerialization() @@ -1135,16 +1135,16 @@ def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueCl C{conf} is applied on top of the base Hadoop conf associated with the SparkContext of this RDD to create a merged Hadoop MapReduce job configuration for saving the data. - @param path: path to Hadoop file - @param outputFormatClass: fully qualified classname of Hadoop OutputFormat + :param path: path to Hadoop file + :param outputFormatClass: fully qualified classname of Hadoop OutputFormat (e.g. "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat") - @param keyClass: fully qualified classname of key Writable class + :param keyClass: fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.IntWritable", None by default) - @param valueClass: fully qualified classname of value Writable class + :param valueClass: fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.Text", None by default) - @param keyConverter: (None by default) - @param valueConverter: (None by default) - @param conf: Hadoop job configuration, passed in as a dict (None by default) + :param keyConverter: (None by default) + :param valueConverter: (None by default) + :param conf: Hadoop job configuration, passed in as a dict (None by default) """ jconf = self.ctx._dictToJavaMap(conf) pickledRDD = self._toPickleSerialization() @@ -1161,9 +1161,9 @@ def saveAsHadoopDataset(self, conf, keyConverter=None, valueConverter=None): converted for output using either user specified converters or, by default, L{org.apache.spark.api.python.JavaToWritableConverter}. - @param conf: Hadoop job configuration, passed in as a dict - @param keyConverter: (None by default) - @param valueConverter: (None by default) + :param conf: Hadoop job configuration, passed in as a dict + :param keyConverter: (None by default) + :param valueConverter: (None by default) """ jconf = self.ctx._dictToJavaMap(conf) pickledRDD = self._toPickleSerialization() @@ -1182,17 +1182,17 @@ def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=No C{conf} is applied on top of the base Hadoop conf associated with the SparkContext of this RDD to create a merged Hadoop MapReduce job configuration for saving the data. - @param path: path to Hadoop file - @param outputFormatClass: fully qualified classname of Hadoop OutputFormat + :param path: path to Hadoop file + :param outputFormatClass: fully qualified classname of Hadoop OutputFormat (e.g. "org.apache.hadoop.mapred.SequenceFileOutputFormat") - @param keyClass: fully qualified classname of key Writable class + :param keyClass: fully qualified classname of key Writable class (e.g. "org.apache.hadoop.io.IntWritable", None by default) - @param valueClass: fully qualified classname of value Writable class + :param valueClass: fully qualified classname of value Writable class (e.g. "org.apache.hadoop.io.Text", None by default) - @param keyConverter: (None by default) - @param valueConverter: (None by default) - @param conf: (None by default) - @param compressionCodecClass: (None by default) + :param keyConverter: (None by default) + :param valueConverter: (None by default) + :param conf: (None by default) + :param compressionCodecClass: (None by default) """ jconf = self.ctx._dictToJavaMap(conf) pickledRDD = self._toPickleSerialization() @@ -1212,8 +1212,8 @@ def saveAsSequenceFile(self, path, compressionCodecClass=None): 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects. 2. Keys and values of this Java RDD are converted to Writables and written out. - @param path: path to sequence file - @param compressionCodecClass: (None by default) + :param path: path to sequence file + :param compressionCodecClass: (None by default) """ pickledRDD = self._toPickleSerialization() batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) @@ -2009,7 +2009,7 @@ def countApproxDistinct(self, relativeSD=0.05): of The Art Cardinality Estimation Algorithm", available here. - @param relativeSD Relative accuracy. Smaller values create + :param relativeSD Relative accuracy. Smaller values create counters that require more space. It must be greater than 0.000017. diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 114644ab8b79d..3d5a281239d66 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -15,28 +15,37 @@ # limitations under the License. # +""" +public classes of Spark SQL: + + - L{SQLContext} + Main entry point for SQL functionality. + - L{SchemaRDD} + A Resilient Distributed Dataset (RDD) with Schema information for the data contained. In + addition to normal RDD operations, SchemaRDDs also support SQL. + - L{Row} + A Row of data returned by a Spark SQL query. + - L{HiveContext} + Main entry point for accessing data stored in Apache Hive.. +""" -import sys -import types import itertools -import warnings import decimal import datetime import keyword import warnings from array import array from operator import itemgetter +from itertools import imap + +from py4j.protocol import Py4JError +from py4j.java_collections import ListConverter, MapConverter from pyspark.rdd import RDD from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync -from itertools import chain, ifilter, imap - -from py4j.protocol import Py4JError -from py4j.java_collections import ListConverter, MapConverter - __all__ = [ "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", @@ -899,8 +908,8 @@ class SQLContext(object): def __init__(self, sparkContext, sqlContext=None): """Create a new SQLContext. - @param sparkContext: The SparkContext to wrap. - @param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new + :param sparkContext: The SparkContext to wrap. + :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instatiate a new SQLContext in the JVM, instead we make all calls to this object. >>> srdd = sqlCtx.inferSchema(rdd) @@ -1325,8 +1334,8 @@ class HiveContext(SQLContext): def __init__(self, sparkContext, hiveContext=None): """Create a new HiveContext. - @param sparkContext: The SparkContext to wrap. - @param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new + :param sparkContext: The SparkContext to wrap. + :param hiveContext: An optional JVM Scala HiveContext. If set, we do not instatiate a new HiveContext in the JVM, instead we make all calls to this object. """ SQLContext.__init__(self, sparkContext) From c7818434fa8ae8e02a0d66183990077a4ba1436c Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Tue, 7 Oct 2014 22:32:39 -0700 Subject: [PATCH 0922/1492] [SPARK-3836] [REPL] Spark REPL optionally propagate internal exceptions Optionally have the repl throw exceptions generated by interpreted code, instead of swallowing the exception and returning it as text output. This is useful when embedding the repl, otherwise it's not possible to know when user code threw an exception. Author: Ahir Reddy Closes #2695 from ahirreddy/repl-throw-exceptions and squashes the following commits: bad25ee [Ahir Reddy] Style Fixes f0e5b44 [Ahir Reddy] Fixed style 0d4413d [Ahir Reddy] propogate excetions from repl --- .../scala/org/apache/spark/repl/SparkIMain.scala | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 6ddb6accd696b..646c68e60c2e9 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -84,9 +84,11 @@ import org.apache.spark.util.Utils * @author Moez A. Abdel-Gawad * @author Lex Spoon */ - class SparkIMain(initialSettings: Settings, val out: JPrintWriter) - extends SparkImports with Logging { - imain => + class SparkIMain( + initialSettings: Settings, + val out: JPrintWriter, + propagateExceptions: Boolean = false) + extends SparkImports with Logging { imain => val conf = new SparkConf() @@ -816,6 +818,10 @@ import org.apache.spark.util.Utils val resultName = FixedSessionNames.resultName def bindError(t: Throwable) = { + // Immediately throw the exception if we are asked to propagate them + if (propagateExceptions) { + throw unwrap(t) + } if (!bindExceptions) // avoid looping if already binding throw t From 35afdfd624fe19ce0c009cf065bb6794ee68e181 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 7 Oct 2014 23:26:24 -0700 Subject: [PATCH 0923/1492] [SPARK-3710] Fix Yarn integration tests on Hadoop 2.2. It seems some dependencies are not declared when pulling the 2.2 test dependencies, so we need to add them manually for the Yarn cluster to come up. These don't seem to be necessary for 2.3 and beyond, so restrict them to the hadoop-2.2 profile. Author: Marcelo Vanzin Closes #2682 from vanzin/SPARK-3710 and squashes the following commits: 701d4fb [Marcelo Vanzin] Add comment. 0540bdf [Marcelo Vanzin] [SPARK-3710] Fix Yarn integration tests on Hadoop 2.2. --- yarn/stable/pom.xml | 51 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 51 insertions(+) diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 97eb0548e77c3..fe55d70ccc370 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -41,4 +41,55 @@ + + + + hadoop-2.2 + + 1.9 + + + + org.mortbay.jetty + jetty + 6.1.26 + + + org.mortbay.jetty + servlet-api + + + test + + + com.sun.jersey + jersey-core + ${jersey.version} + test + + + com.sun.jersey + jersey-json + ${jersey.version} + test + + + stax + stax-api + + + + + com.sun.jersey + jersey-server + ${jersey.version} + test + + + + + From 7fca8f41c8889a41d9ab05ad0ab39c7639f657ed Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Oct 2014 08:48:55 -0500 Subject: [PATCH 0924/1492] [SPARK-3788] [yarn] Fix compareFs to do the right thing for HDFS namespaces. HA and viewfs use namespaces instead of host names, so you can't resolve them since that will fail. So be smarter to avoid doing unnecessary work. Author: Marcelo Vanzin Closes #2649 from vanzin/SPARK-3788 and squashes the following commits: fedbc73 [Marcelo Vanzin] Update comment. c938845 [Marcelo Vanzin] Use Objects.equal() to avoid issues with ==. 9f7b571 [Marcelo Vanzin] [SPARK-3788] [yarn] Fix compareFs to do the right thing for HA, federation. --- .../apache/spark/deploy/yarn/ClientBase.scala | 31 +++++++------------ 1 file changed, 12 insertions(+), 19 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 6ecac6eae6e03..14a0386b78978 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, ListBuffer, Map} import scala.util.{Try, Success, Failure} +import com.google.common.base.Objects import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission @@ -64,12 +65,12 @@ private[spark] trait ClientBase extends Logging { s"memory capability of the cluster ($maxMem MB per container)") val executorMem = args.executorMemory + executorMemoryOverhead if (executorMem > maxMem) { - throw new IllegalArgumentException(s"Required executor memory (${args.executorMemory}" + + throw new IllegalArgumentException(s"Required executor memory (${args.executorMemory}" + s"+$executorMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") } val amMem = args.amMemory + amMemoryOverhead if (amMem > maxMem) { - throw new IllegalArgumentException(s"Required AM memory (${args.amMemory}" + + throw new IllegalArgumentException(s"Required AM memory (${args.amMemory}" + s"+$amMemoryOverhead MB) is above the max threshold ($maxMem MB) of this cluster!") } logInfo("Will allocate AM container, with %d MB memory including %d MB overhead".format( @@ -771,15 +772,17 @@ private[spark] object ClientBase extends Logging { private def compareFs(srcFs: FileSystem, destFs: FileSystem): Boolean = { val srcUri = srcFs.getUri() val dstUri = destFs.getUri() - if (srcUri.getScheme() == null) { - return false - } - if (!srcUri.getScheme().equals(dstUri.getScheme())) { + if (srcUri.getScheme() == null || srcUri.getScheme() != dstUri.getScheme()) { return false } + var srcHost = srcUri.getHost() var dstHost = dstUri.getHost() - if ((srcHost != null) && (dstHost != null)) { + + // In HA or when using viewfs, the host part of the URI may not actually be a host, but the + // name of the HDFS namespace. Those names won't resolve, so avoid even trying if they + // match. + if (srcHost != null && dstHost != null && srcHost != dstHost) { try { srcHost = InetAddress.getByName(srcHost).getCanonicalHostName() dstHost = InetAddress.getByName(dstHost).getCanonicalHostName() @@ -787,19 +790,9 @@ private[spark] object ClientBase extends Logging { case e: UnknownHostException => return false } - if (!srcHost.equals(dstHost)) { - return false - } - } else if (srcHost == null && dstHost != null) { - return false - } else if (srcHost != null && dstHost == null) { - return false - } - if (srcUri.getPort() != dstUri.getPort()) { - false - } else { - true } + + Objects.equal(srcHost, dstHost) && srcUri.getPort() == dstUri.getPort() } } From f18dd5962e4a18c3507de8147bde3a8f56380439 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 8 Oct 2014 11:53:43 -0500 Subject: [PATCH 0925/1492] [SPARK-3848] yarn alpha doesn't build on master yarn alpha build was broken by #2432 as it added an argument to YarnAllocator but not to yarn/alpha YarnAllocationHandler commit https://github.com/apache/spark/commit/79e45c9323455a51f25ed9acd0edd8682b4bbb88 Author: Kousuke Saruta Closes #2715 from sarutak/SPARK-3848 and squashes the following commits: bafb8d1 [Kousuke Saruta] Fixed parameters for the default constructor of alpha/YarnAllocatorHandler. --- .../org/apache/spark/deploy/yarn/YarnAllocationHandler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 6c93d8582330b..abd37834ed3cc 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -43,7 +43,7 @@ private[yarn] class YarnAllocationHandler( args: ApplicationMasterArguments, preferredNodes: collection.Map[String, collection.Set[SplitInfo]], securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) { + extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { private val lastResponseId = new AtomicInteger() private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() From bc4418727b40c9b6ba5194ead6e2698539272280 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 8 Oct 2014 13:33:46 -0700 Subject: [PATCH 0926/1492] HOTFIX: Use correct Hadoop profile in build --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index 4be2baaf48cd1..f47fcf66ff7e7 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -42,7 +42,7 @@ function handle_error () { elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.0" ]; then export SBT_MAVEN_PROFILES_ARGS="-Dhadoop.version=2.0.0-mr1-cdh4.1.1" elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.2" ]; then - export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Dhadoop.version=2.2.0" + export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0" elif [ "$AMPLAB_JENKINS_BUILD_PROFILE" = "hadoop2.3" ]; then export SBT_MAVEN_PROFILES_ARGS="-Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0" fi From b92bd5a2f29f7a9ce270540b6a828fa7ff205cbe Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 8 Oct 2014 14:23:21 -0700 Subject: [PATCH 0927/1492] [SPARK-3841] [mllib] Pretty-print params for ML examples Provide a parent class for the Params case classes used in many MLlib examples, where the parent class pretty-prints the case class fields: Param1Name Param1Value Param2Name Param2Value ... Using this class will make it easier to print test settings to logs. Also, updated DecisionTreeRunner to print a little more info. CC: mengxr Author: Joseph K. Bradley Closes #2700 from jkbradley/dtrunner-update and squashes the following commits: cff873f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update 7a08ae4 [Joseph K. Bradley] code review comment updates b4d2043 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update d8228a7 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update 0fc9c64 [Joseph K. Bradley] Added abstract TestParams class for mllib example parameters 12b7798 [Joseph K. Bradley] Added abstract class TestParams for pretty-printing Params values 5f84f03 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update f7441b6 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update 19eb6fc [Joseph K. Bradley] Updated DecisionTreeRunner to print training time. --- .../spark/examples/mllib/AbstractParams.scala | 53 +++++++++++++++++++ .../examples/mllib/BinaryClassification.scala | 2 +- .../spark/examples/mllib/Correlations.scala | 1 + .../examples/mllib/CosineSimilarity.scala | 1 + .../examples/mllib/DecisionTreeRunner.scala | 15 +++++- .../spark/examples/mllib/DenseKMeans.scala | 2 +- .../examples/mllib/LinearRegression.scala | 2 +- .../spark/examples/mllib/MovieLensALS.scala | 2 +- .../mllib/MultivariateSummarizer.scala | 1 + .../spark/examples/mllib/SampledRDDs.scala | 1 + .../examples/mllib/SparseNaiveBayes.scala | 2 +- 11 files changed, 75 insertions(+), 7 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala new file mode 100644 index 0000000000000..ae6057758d6fc --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/AbstractParams.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import scala.reflect.runtime.universe._ + +/** + * Abstract class for parameter case classes. + * This overrides the [[toString]] method to print all case class fields by name and value. + * @tparam T Concrete parameter class. + */ +abstract class AbstractParams[T: TypeTag] { + + private def tag: TypeTag[T] = typeTag[T] + + /** + * Finds all case class fields in concrete class instance, and outputs them in JSON-style format: + * { + * [field name]:\t[field value]\n + * [field name]:\t[field value]\n + * ... + * } + */ + override def toString: String = { + val tpe = tag.tpe + val allAccessors = tpe.declarations.collect { + case m: MethodSymbol if m.isCaseAccessor => m + } + val mirror = runtimeMirror(getClass.getClassLoader) + val instanceMirror = mirror.reflect(this) + allAccessors.map { f => + val paramName = f.name.toString + val fieldMirror = instanceMirror.reflectField(f) + val paramValue = fieldMirror.get + s" $paramName:\t$paramValue" + }.mkString("{\n", ",\n", "\n}") + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index a6f78d2441db1..1edd2432a0352 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -55,7 +55,7 @@ object BinaryClassification { stepSize: Double = 1.0, algorithm: Algorithm = LR, regType: RegType = L2, - regParam: Double = 0.1) + regParam: Double = 0.1) extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala index d6b2fe430e5a4..e49129c4e7844 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/Correlations.scala @@ -35,6 +35,7 @@ import org.apache.spark.{SparkConf, SparkContext} object Correlations { case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") + extends AbstractParams[Params] def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala index 6a3b0241ced7f..cb1abbd18fd4d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/CosineSimilarity.scala @@ -43,6 +43,7 @@ import org.apache.spark.{SparkConf, SparkContext} */ object CosineSimilarity { case class Params(inputFile: String = null, threshold: Double = 0.1) + extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 4adc91d2fbe65..837d0591478c5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -62,7 +62,7 @@ object DecisionTreeRunner { minInfoGain: Double = 0.0, numTrees: Int = 1, featureSubsetStrategy: String = "auto", - fracTest: Double = 0.2) + fracTest: Double = 0.2) extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() @@ -138,9 +138,11 @@ object DecisionTreeRunner { def run(params: Params) { - val conf = new SparkConf().setAppName("DecisionTreeRunner") + val conf = new SparkConf().setAppName(s"DecisionTreeRunner with $params") val sc = new SparkContext(conf) + println(s"DecisionTreeRunner with parameters:\n$params") + // Load training data and cache it. val origExamples = params.dataFormat match { case "dense" => MLUtils.loadLabeledPoints(sc, params.input).cache() @@ -235,7 +237,10 @@ object DecisionTreeRunner { minInstancesPerNode = params.minInstancesPerNode, minInfoGain = params.minInfoGain) if (params.numTrees == 1) { + val startTime = System.nanoTime() val model = DecisionTree.train(training, strategy) + val elapsedTime = (System.nanoTime() - startTime) / 1e9 + println(s"Training time: $elapsedTime seconds") if (model.numNodes < 20) { println(model.toDebugString) // Print full model. } else { @@ -259,8 +264,11 @@ object DecisionTreeRunner { } else { val randomSeed = Utils.random.nextInt() if (params.algo == Classification) { + val startTime = System.nanoTime() val model = RandomForest.trainClassifier(training, strategy, params.numTrees, params.featureSubsetStrategy, randomSeed) + val elapsedTime = (System.nanoTime() - startTime) / 1e9 + println(s"Training time: $elapsedTime seconds") if (model.totalNumNodes < 30) { println(model.toDebugString) // Print full model. } else { @@ -275,8 +283,11 @@ object DecisionTreeRunner { println(s"Test accuracy = $testAccuracy") } if (params.algo == Regression) { + val startTime = System.nanoTime() val model = RandomForest.trainRegressor(training, strategy, params.numTrees, params.featureSubsetStrategy, randomSeed) + val elapsedTime = (System.nanoTime() - startTime) / 1e9 + println(s"Training time: $elapsedTime seconds") if (model.totalNumNodes < 30) { println(model.toDebugString) // Print full model. } else { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala index 89dfa26c2299c..11e35598baf50 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DenseKMeans.scala @@ -44,7 +44,7 @@ object DenseKMeans { input: String = null, k: Int = -1, numIterations: Int = 10, - initializationMode: InitializationMode = Parallel) + initializationMode: InitializationMode = Parallel) extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 05b7d66f8dffd..e1f9622350135 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -47,7 +47,7 @@ object LinearRegression extends App { numIterations: Int = 100, stepSize: Double = 1.0, regType: RegType = L2, - regParam: Double = 0.1) + regParam: Double = 0.1) extends AbstractParams[Params] val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 98aaedb9d7dc9..fc6678013b932 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -55,7 +55,7 @@ object MovieLensALS { rank: Int = 10, numUserBlocks: Int = -1, numProductBlocks: Int = -1, - implicitPrefs: Boolean = false) + implicitPrefs: Boolean = false) extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala index 4532512c01f84..6e4e2d07f284b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MultivariateSummarizer.scala @@ -36,6 +36,7 @@ import org.apache.spark.{SparkConf, SparkContext} object MultivariateSummarizer { case class Params(input: String = "data/mllib/sample_linear_regression_data.txt") + extends AbstractParams[Params] def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala index f01b8266e3fe3..663c12734af68 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SampledRDDs.scala @@ -33,6 +33,7 @@ import org.apache.spark.SparkContext._ object SampledRDDs { case class Params(input: String = "data/mllib/sample_binary_classification_data.txt") + extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala index 952fa2a5109a4..f1ff4e6911f5e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala @@ -37,7 +37,7 @@ object SparseNaiveBayes { input: String = null, minPartitions: Int = 0, numFeatures: Int = -1, - lambda: Double = 1.0) + lambda: Double = 1.0) extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() From add174aa56d291bc48ef73a42c39428c923efe31 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 8 Oct 2014 15:19:19 -0700 Subject: [PATCH 0928/1492] [SPARK-3843][Minor] Cleanup scalastyle.txt at the end of running dev/scalastyle dev/scalastyle create a log file 'scalastyle.txt'. it is overwrote per running but never deleted even though dev/mima and dev/lint-python delete their log files. Author: Kousuke Saruta Closes #2702 from sarutak/scalastyle-txt-cleanup and squashes the following commits: d6e238e [Kousuke Saruta] Fixed dev/scalastyle to cleanup scalastyle.txt --- dev/scalastyle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dev/scalastyle b/dev/scalastyle index efb5f291ea3b7..c3b356bcb3c06 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -26,6 +26,8 @@ echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalasty >> scalastyle.txt ERRORS=$(cat scalastyle.txt | grep -e "\") +rm scalastyle.txt + if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" exit 1 From a85f24accd3266e0f97ee04d03c22b593d99c062 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 8 Oct 2014 17:03:47 -0700 Subject: [PATCH 0929/1492] [SPARK-3831] [SQL] Filter rule Improvement and bool expression optimization. If we write the filter which is always FALSE like SELECT * from person WHERE FALSE; 200 tasks will run. I think, 1 task is enough. And current optimizer cannot optimize the case NOT is duplicated like SELECT * from person WHERE NOT ( NOT (age > 30)); The filter rule above should be simplified Author: Kousuke Saruta Closes #2692 from sarutak/SPARK-3831 and squashes the following commits: 25f3e20 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3831 23c750c [Kousuke Saruta] Improved unsupported predicate test case a11b9f3 [Kousuke Saruta] Modified NOT predicate test case in PartitionBatchPruningSuite 8ea872b [Kousuke Saruta] Fixed the number of tasks when the data of LocalRelation is empty. --- .../spark/sql/catalyst/optimizer/Optimizer.scala | 12 ++++++++++++ .../apache/spark/sql/execution/SparkStrategies.scala | 3 ++- .../sql/columnar/PartitionBatchPruningSuite.scala | 3 ++- 3 files changed, 16 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index a4133feae8166..636d0b95583e4 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 @@ -299,6 +299,18 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (_, _) => or } + case not @ Not(exp) => + exp match { + case Literal(true, BooleanType) => Literal(false) + case Literal(false, BooleanType) => Literal(true) + case GreaterThan(l, r) => LessThanOrEqual(l, r) + case GreaterThanOrEqual(l, r) => LessThan(l, r) + case LessThan(l, r) => GreaterThanOrEqual(l, r) + case LessThanOrEqual(l, r) => GreaterThan(l, r) + case Not(e) => e + case _ => not + } + // Turn "if (true) a else b" into "a", and if (false) a else b" into "b". case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5c16d0c624128..883f2ff521e20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -274,9 +274,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.Sample(fraction, withReplacement, seed, planLater(child)) :: Nil case SparkLogicalPlan(alreadyPlanned) => alreadyPlanned :: Nil case logical.LocalRelation(output, data) => + val nPartitions = if (data.isEmpty) 1 else numPartitions PhysicalRDD( output, - RDDConversions.productToRowRdd(sparkContext.parallelize(data, numPartitions))) :: Nil + RDDConversions.productToRowRdd(sparkContext.parallelize(data, nPartitions))) :: Nil case logical.Limit(IntegerLiteral(limit), child) => execution.Limit(limit, planLater(child)) :: Nil case Unions(unionChildren) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 69e0adbd3ee0d..f53acc8c9f718 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -67,10 +67,11 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be checkBatchPruning("i > 8 AND i <= 21", 9 to 21, 2, 3) checkBatchPruning("i < 2 OR i > 99", Seq(1, 100), 2, 2) checkBatchPruning("i < 2 OR (i > 78 AND i < 92)", Seq(1) ++ (79 to 91), 3, 4) + checkBatchPruning("NOT (i < 88)", 88 to 100, 1, 2) // With unsupported predicate checkBatchPruning("i < 12 AND i IS NOT NULL", 1 to 11, 1, 2) - checkBatchPruning("NOT (i < 88)", 88 to 100, 5, 10) + checkBatchPruning(s"NOT (i in (${(1 to 30).mkString(",")}))", 31 to 100, 5, 10) def checkBatchPruning( filter: String, From a42cc08d219c579019f613faa8d310e6069c06fe Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Oct 2014 17:04:49 -0700 Subject: [PATCH 0930/1492] [SPARK-3713][SQL] Uses JSON to serialize DataType objects This PR uses JSON instead of `toString` to serialize `DataType`s. The latter is not only hard to parse but also flaky in many cases. Since we already write schema information to Parquet metadata in the old style, we have to reserve the old `DataType` parser and ensure downward compatibility. The old parser is now renamed to `CaseClassStringParser` and moved into `object DataType`. JoshRosen davies Please help review PySpark related changes, thanks! Author: Cheng Lian Closes #2563 from liancheng/datatype-to-json and squashes the following commits: fc92eb3 [Cheng Lian] Reverts debugging code, simplifies primitive type JSON representation 438c75f [Cheng Lian] Refactors PySpark DataType JSON SerDe per comments 6b6387b [Cheng Lian] Removes debugging code 6a3ee3a [Cheng Lian] Addresses per review comments dc158b5 [Cheng Lian] Addresses PEP8 issues 99ab4ee [Cheng Lian] Adds compatibility est case for Parquet type conversion a983a6c [Cheng Lian] Adds PySpark support f608c6e [Cheng Lian] De/serializes DataType objects from/to JSON --- python/pyspark/sql.py | 153 ++++++------ .../catalyst/expressions/WrapDynamic.scala | 4 +- .../spark/sql/catalyst/types/dataTypes.scala | 229 ++++++++++++------ .../org/apache/spark/sql/SQLContext.scala | 9 +- .../spark/sql/parquet/ParquetTypes.scala | 6 +- .../org/apache/spark/sql/DataTypeSuite.scala | 28 +++ .../spark/sql/parquet/ParquetQuerySuite.scala | 16 +- 7 files changed, 277 insertions(+), 168 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 3d5a281239d66..d3d36eb995ab6 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -34,6 +34,7 @@ import datetime import keyword import warnings +import json from array import array from operator import itemgetter from itertools import imap @@ -71,6 +72,18 @@ def __eq__(self, other): def __ne__(self, other): return not self.__eq__(other) + @classmethod + def typeName(cls): + return cls.__name__[:-4].lower() + + def jsonValue(self): + return self.typeName() + + def json(self): + return json.dumps(self.jsonValue(), + separators=(',', ':'), + sort_keys=True) + class PrimitiveTypeSingleton(type): @@ -214,6 +227,16 @@ def __repr__(self): return "ArrayType(%s,%s)" % (self.elementType, str(self.containsNull).lower()) + def jsonValue(self): + return {"type": self.typeName(), + "elementType": self.elementType.jsonValue(), + "containsNull": self.containsNull} + + @classmethod + def fromJson(cls, json): + return ArrayType(_parse_datatype_json_value(json["elementType"]), + json["containsNull"]) + class MapType(DataType): @@ -254,6 +277,18 @@ def __repr__(self): return "MapType(%s,%s,%s)" % (self.keyType, self.valueType, str(self.valueContainsNull).lower()) + def jsonValue(self): + return {"type": self.typeName(), + "keyType": self.keyType.jsonValue(), + "valueType": self.valueType.jsonValue(), + "valueContainsNull": self.valueContainsNull} + + @classmethod + def fromJson(cls, json): + return MapType(_parse_datatype_json_value(json["keyType"]), + _parse_datatype_json_value(json["valueType"]), + json["valueContainsNull"]) + class StructField(DataType): @@ -292,6 +327,17 @@ def __repr__(self): return "StructField(%s,%s,%s)" % (self.name, self.dataType, str(self.nullable).lower()) + def jsonValue(self): + return {"name": self.name, + "type": self.dataType.jsonValue(), + "nullable": self.nullable} + + @classmethod + def fromJson(cls, json): + return StructField(json["name"], + _parse_datatype_json_value(json["type"]), + json["nullable"]) + class StructType(DataType): @@ -321,42 +367,30 @@ def __repr__(self): return ("StructType(List(%s))" % ",".join(str(field) for field in self.fields)) + def jsonValue(self): + return {"type": self.typeName(), + "fields": [f.jsonValue() for f in self.fields]} -def _parse_datatype_list(datatype_list_string): - """Parses a list of comma separated data types.""" - index = 0 - datatype_list = [] - start = 0 - depth = 0 - while index < len(datatype_list_string): - if depth == 0 and datatype_list_string[index] == ",": - datatype_string = datatype_list_string[start:index].strip() - datatype_list.append(_parse_datatype_string(datatype_string)) - start = index + 1 - elif datatype_list_string[index] == "(": - depth += 1 - elif datatype_list_string[index] == ")": - depth -= 1 + @classmethod + def fromJson(cls, json): + return StructType([StructField.fromJson(f) for f in json["fields"]]) - index += 1 - # Handle the last data type - datatype_string = datatype_list_string[start:index].strip() - datatype_list.append(_parse_datatype_string(datatype_string)) - return datatype_list +_all_primitive_types = dict((v.typeName(), v) + for v in globals().itervalues() + if type(v) is PrimitiveTypeSingleton and + v.__base__ == PrimitiveType) -_all_primitive_types = dict((k, v) for k, v in globals().iteritems() - if type(v) is PrimitiveTypeSingleton and v.__base__ == PrimitiveType) +_all_complex_types = dict((v.typeName(), v) + for v in [ArrayType, MapType, StructType]) -def _parse_datatype_string(datatype_string): - """Parses the given data type string. - +def _parse_datatype_json_string(json_string): + """Parses the given data type JSON string. >>> def check_datatype(datatype): - ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(str(datatype)) - ... python_datatype = _parse_datatype_string( - ... scala_datatype.toString()) + ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.json()) + ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) ... return datatype == python_datatype >>> all(check_datatype(cls()) for cls in _all_primitive_types.values()) True @@ -394,51 +428,14 @@ def _parse_datatype_string(datatype_string): >>> check_datatype(complex_maptype) True """ - index = datatype_string.find("(") - if index == -1: - # It is a primitive type. - index = len(datatype_string) - type_or_field = datatype_string[:index] - rest_part = datatype_string[index + 1:len(datatype_string) - 1].strip() - - if type_or_field in _all_primitive_types: - return _all_primitive_types[type_or_field]() - - elif type_or_field == "ArrayType": - last_comma_index = rest_part.rfind(",") - containsNull = True - if rest_part[last_comma_index + 1:].strip().lower() == "false": - containsNull = False - elementType = _parse_datatype_string( - rest_part[:last_comma_index].strip()) - return ArrayType(elementType, containsNull) - - elif type_or_field == "MapType": - last_comma_index = rest_part.rfind(",") - valueContainsNull = True - if rest_part[last_comma_index + 1:].strip().lower() == "false": - valueContainsNull = False - keyType, valueType = _parse_datatype_list( - rest_part[:last_comma_index].strip()) - return MapType(keyType, valueType, valueContainsNull) - - elif type_or_field == "StructField": - first_comma_index = rest_part.find(",") - name = rest_part[:first_comma_index].strip() - last_comma_index = rest_part.rfind(",") - nullable = True - if rest_part[last_comma_index + 1:].strip().lower() == "false": - nullable = False - dataType = _parse_datatype_string( - rest_part[first_comma_index + 1:last_comma_index].strip()) - return StructField(name, dataType, nullable) - - elif type_or_field == "StructType": - # rest_part should be in the format like - # List(StructField(field1,IntegerType,false)). - field_list_string = rest_part[rest_part.find("(") + 1:-1] - fields = _parse_datatype_list(field_list_string) - return StructType(fields) + return _parse_datatype_json_value(json.loads(json_string)) + + +def _parse_datatype_json_value(json_value): + if type(json_value) is unicode and json_value in _all_primitive_types.keys(): + return _all_primitive_types[json_value]() + else: + return _all_complex_types[json_value["type"]].fromJson(json_value) # Mapping Python types to Spark SQL DateType @@ -992,7 +989,7 @@ def registerFunction(self, name, f, returnType=StringType()): self._sc.pythonExec, broadcast_vars, self._sc._javaAccumulator, - str(returnType)) + returnType.json()) def inferSchema(self, rdd): """Infer and apply a schema to an RDD of L{Row}. @@ -1128,7 +1125,7 @@ def applySchema(self, rdd, schema): batched = isinstance(rdd._jrdd_deserializer, BatchedSerializer) jrdd = self._pythonToJava(rdd._jrdd, batched) - srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), str(schema)) + srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) return SchemaRDD(srdd.toJavaSchemaRDD(), self) def registerRDDAsTable(self, rdd, tableName): @@ -1218,7 +1215,7 @@ def jsonFile(self, path, schema=None): if schema is None: srdd = self._ssql_ctx.jsonFile(path) else: - scala_datatype = self._ssql_ctx.parseDataType(str(schema)) + scala_datatype = self._ssql_ctx.parseDataType(schema.json()) srdd = self._ssql_ctx.jsonFile(path, scala_datatype) return SchemaRDD(srdd.toJavaSchemaRDD(), self) @@ -1288,7 +1285,7 @@ def func(iterator): if schema is None: srdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) else: - scala_datatype = self._ssql_ctx.parseDataType(str(schema)) + scala_datatype = self._ssql_ctx.parseDataType(schema.json()) srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) return SchemaRDD(srdd.toJavaSchemaRDD(), self) @@ -1623,7 +1620,7 @@ def saveAsTable(self, tableName): def schema(self): """Returns the schema of this SchemaRDD (represented by a L{StructType}).""" - return _parse_datatype_string(self._jschema_rdd.baseSchemaRDD().schema().toString()) + return _parse_datatype_json_string(self._jschema_rdd.baseSchemaRDD().schema().json()) def schemaString(self): """Returns the output schema in the tree format.""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index 1eb55715794a7..1a4ac06c7a79d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -24,9 +24,7 @@ import org.apache.spark.sql.catalyst.types.DataType /** * The data type representing [[DynamicRow]] values. */ -case object DynamicType extends DataType { - def simpleString: String = "dynamic" -} +case object DynamicType extends DataType /** * Wrap a [[Row]] as a [[DynamicRow]]. 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 ac043d4dd8eb9..1d375b8754182 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,71 +19,125 @@ package org.apache.spark.sql.catalyst.types import java.sql.Timestamp -import scala.math.Numeric.{FloatAsIfIntegral, BigDecimalAsIfIntegral, DoubleAsIfIntegral} +import scala.math.Numeric.{BigDecimalAsIfIntegral, DoubleAsIfIntegral, FloatAsIfIntegral} import scala.reflect.ClassTag -import scala.reflect.runtime.universe.{typeTag, TypeTag, runtimeMirror} +import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} import scala.util.parsing.combinator.RegexParsers +import org.json4s.JsonAST.JValue +import org.json4s._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.util.Utils -/** - * Utility functions for working with DataTypes. - */ -object DataType extends RegexParsers { - protected lazy val primitiveType: Parser[DataType] = - "StringType" ^^^ StringType | - "FloatType" ^^^ FloatType | - "IntegerType" ^^^ IntegerType | - "ByteType" ^^^ ByteType | - "ShortType" ^^^ ShortType | - "DoubleType" ^^^ DoubleType | - "LongType" ^^^ LongType | - "BinaryType" ^^^ BinaryType | - "BooleanType" ^^^ BooleanType | - "DecimalType" ^^^ DecimalType | - "TimestampType" ^^^ TimestampType - - protected lazy val arrayType: Parser[DataType] = - "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { - case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) - } - protected lazy val mapType: Parser[DataType] = - "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { - case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) +object DataType { + def fromJson(json: String): DataType = parseDataType(parse(json)) + + private object JSortedObject { + def unapplySeq(value: JValue): Option[List[(String, JValue)]] = value match { + case JObject(seq) => Some(seq.toList.sortBy(_._1)) + case _ => None } + } + + // NOTE: Map fields must be sorted in alphabetical order to keep consistent with the Python side. + private def parseDataType(json: JValue): DataType = json match { + case JString(name) => + PrimitiveType.nameToType(name) + + case JSortedObject( + ("containsNull", JBool(n)), + ("elementType", t: JValue), + ("type", JString("array"))) => + ArrayType(parseDataType(t), n) + + case JSortedObject( + ("keyType", k: JValue), + ("type", JString("map")), + ("valueContainsNull", JBool(n)), + ("valueType", v: JValue)) => + MapType(parseDataType(k), parseDataType(v), n) + + case JSortedObject( + ("fields", JArray(fields)), + ("type", JString("struct"))) => + StructType(fields.map(parseStructField)) + } - protected lazy val structField: Parser[StructField] = - ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { - case name ~ tpe ~ nullable => + private def parseStructField(json: JValue): StructField = json match { + case JSortedObject( + ("name", JString(name)), + ("nullable", JBool(nullable)), + ("type", dataType: JValue)) => + StructField(name, parseDataType(dataType), nullable) + } + + @deprecated("Use DataType.fromJson instead") + def fromCaseClassString(string: String): DataType = CaseClassStringParser(string) + + private object CaseClassStringParser extends RegexParsers { + protected lazy val primitiveType: Parser[DataType] = + ( "StringType" ^^^ StringType + | "FloatType" ^^^ FloatType + | "IntegerType" ^^^ IntegerType + | "ByteType" ^^^ ByteType + | "ShortType" ^^^ ShortType + | "DoubleType" ^^^ DoubleType + | "LongType" ^^^ LongType + | "BinaryType" ^^^ BinaryType + | "BooleanType" ^^^ BooleanType + | "DecimalType" ^^^ DecimalType + | "TimestampType" ^^^ TimestampType + ) + + protected lazy val arrayType: Parser[DataType] = + "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { + case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) + } + + protected lazy val mapType: Parser[DataType] = + "MapType" ~> "(" ~> dataType ~ "," ~ dataType ~ "," ~ boolVal <~ ")" ^^ { + case t1 ~ _ ~ t2 ~ _ ~ valueContainsNull => MapType(t1, t2, valueContainsNull) + } + + protected lazy val structField: Parser[StructField] = + ("StructField(" ~> "[a-zA-Z0-9_]*".r) ~ ("," ~> dataType) ~ ("," ~> boolVal <~ ")") ^^ { + case name ~ tpe ~ nullable => StructField(name, tpe, nullable = nullable) - } + } - protected lazy val boolVal: Parser[Boolean] = - "true" ^^^ true | - "false" ^^^ false + protected lazy val boolVal: Parser[Boolean] = + ( "true" ^^^ true + | "false" ^^^ false + ) - protected lazy val structType: Parser[DataType] = - "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { - case fields => new StructType(fields) - } + protected lazy val structType: Parser[DataType] = + "StructType\\([A-zA-z]*\\(".r ~> repsep(structField, ",") <~ "))" ^^ { + case fields => new StructType(fields) + } - protected lazy val dataType: Parser[DataType] = - arrayType | - mapType | - structType | - primitiveType + protected lazy val dataType: Parser[DataType] = + ( arrayType + | mapType + | structType + | primitiveType + ) + + /** + * Parses a string representation of a DataType. + * + * TODO: Generate parser as pickler... + */ + def apply(asString: String): DataType = parseAll(dataType, asString) match { + case Success(result, _) => result + case failure: NoSuccess => + throw new IllegalArgumentException(s"Unsupported dataType: $asString, $failure") + } - /** - * Parses a string representation of a DataType. - * - * TODO: Generate parser as pickler... - */ - def apply(asString: String): DataType = parseAll(dataType, asString) match { - case Success(result, _) => result - case failure: NoSuccess => sys.error(s"Unsupported dataType: $asString, $failure") } protected[types] def buildFormattedString( @@ -111,15 +165,19 @@ abstract class DataType { def isPrimitive: Boolean = false - def simpleString: String -} + def typeName: String = this.getClass.getSimpleName.stripSuffix("$").dropRight(4).toLowerCase + + private[sql] def jsonValue: JValue = typeName -case object NullType extends DataType { - def simpleString: String = "null" + def json: String = compact(render(jsonValue)) + + def prettyJson: String = pretty(render(jsonValue)) } +case object NullType extends DataType + object NativeType { - def all = Seq( + val all = Seq( IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType, StringType) def unapply(dt: DataType): Boolean = all.contains(dt) @@ -139,6 +197,12 @@ trait PrimitiveType extends DataType { override def isPrimitive = true } +object PrimitiveType { + private[sql] val all = Seq(DecimalType, TimestampType, BinaryType) ++ NativeType.all + + private[sql] val nameToType = all.map(t => t.typeName -> t).toMap +} + abstract class NativeType extends DataType { private[sql] type JvmType @transient private[sql] val tag: TypeTag[JvmType] @@ -154,7 +218,6 @@ case object StringType extends NativeType with PrimitiveType { private[sql] type JvmType = String @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] - def simpleString: String = "string" } case object BinaryType extends NativeType with PrimitiveType { @@ -166,17 +229,15 @@ case object BinaryType extends NativeType with PrimitiveType { val res = x(i).compareTo(y(i)) if (res != 0) return res } - return x.length - y.length + x.length - y.length } } - def simpleString: String = "binary" } case object BooleanType extends NativeType with PrimitiveType { private[sql] type JvmType = Boolean @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] - def simpleString: String = "boolean" } case object TimestampType extends NativeType { @@ -187,8 +248,6 @@ case object TimestampType extends NativeType { private[sql] val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) } - - def simpleString: String = "timestamp" } abstract class NumericType extends NativeType with PrimitiveType { @@ -222,7 +281,6 @@ case object LongType extends IntegralType { private[sql] val numeric = implicitly[Numeric[Long]] private[sql] val integral = implicitly[Integral[Long]] private[sql] val ordering = implicitly[Ordering[JvmType]] - def simpleString: String = "long" } case object IntegerType extends IntegralType { @@ -231,7 +289,6 @@ case object IntegerType extends IntegralType { private[sql] val numeric = implicitly[Numeric[Int]] private[sql] val integral = implicitly[Integral[Int]] private[sql] val ordering = implicitly[Ordering[JvmType]] - def simpleString: String = "integer" } case object ShortType extends IntegralType { @@ -240,7 +297,6 @@ case object ShortType extends IntegralType { private[sql] val numeric = implicitly[Numeric[Short]] private[sql] val integral = implicitly[Integral[Short]] private[sql] val ordering = implicitly[Ordering[JvmType]] - def simpleString: String = "short" } case object ByteType extends IntegralType { @@ -249,7 +305,6 @@ case object ByteType extends IntegralType { private[sql] val numeric = implicitly[Numeric[Byte]] private[sql] val integral = implicitly[Integral[Byte]] private[sql] val ordering = implicitly[Ordering[JvmType]] - def simpleString: String = "byte" } /** Matcher for any expressions that evaluate to [[FractionalType]]s */ @@ -271,7 +326,6 @@ case object DecimalType extends FractionalType { private[sql] val fractional = implicitly[Fractional[BigDecimal]] private[sql] val ordering = implicitly[Ordering[JvmType]] private[sql] val asIntegral = BigDecimalAsIfIntegral - def simpleString: String = "decimal" } case object DoubleType extends FractionalType { @@ -281,7 +335,6 @@ case object DoubleType extends FractionalType { private[sql] val fractional = implicitly[Fractional[Double]] private[sql] val ordering = implicitly[Ordering[JvmType]] private[sql] val asIntegral = DoubleAsIfIntegral - def simpleString: String = "double" } case object FloatType extends FractionalType { @@ -291,12 +344,12 @@ case object FloatType extends FractionalType { private[sql] val fractional = implicitly[Fractional[Float]] private[sql] val ordering = implicitly[Ordering[JvmType]] private[sql] val asIntegral = FloatAsIfIntegral - def simpleString: String = "float" } object ArrayType { /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) + def typeName: String = "array" } /** @@ -309,11 +362,14 @@ object ArrayType { case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { builder.append( - s"${prefix}-- element: ${elementType.simpleString} (containsNull = ${containsNull})\n") + s"$prefix-- element: ${elementType.typeName} (containsNull = $containsNull)\n") DataType.buildFormattedString(elementType, s"$prefix |", builder) } - def simpleString: String = "array" + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("elementType" -> elementType.jsonValue) ~ + ("containsNull" -> containsNull) } /** @@ -325,14 +381,22 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT case class StructField(name: String, dataType: DataType, nullable: Boolean) { private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"${prefix}-- ${name}: ${dataType.simpleString} (nullable = ${nullable})\n") + builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") DataType.buildFormattedString(dataType, s"$prefix |", builder) } + + private[sql] def jsonValue: JValue = { + ("name" -> name) ~ + ("type" -> dataType.jsonValue) ~ + ("nullable" -> nullable) + } } object StructType { protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) + + def typeName = "struct" } case class StructType(fields: Seq[StructField]) extends DataType { @@ -348,8 +412,7 @@ case class StructType(fields: Seq[StructField]) extends DataType { * have a name matching the given name, `null` will be returned. */ def apply(name: String): StructField = { - nameToField.get(name).getOrElse( - throw new IllegalArgumentException(s"Field ${name} does not exist.")) + nameToField.getOrElse(name, throw new IllegalArgumentException(s"Field $name does not exist.")) } /** @@ -358,7 +421,7 @@ case class StructType(fields: Seq[StructField]) extends DataType { */ def apply(names: Set[String]): StructType = { val nonExistFields = names -- fieldNamesSet - if (!nonExistFields.isEmpty) { + if (nonExistFields.nonEmpty) { throw new IllegalArgumentException( s"Field ${nonExistFields.mkString(",")} does not exist.") } @@ -384,7 +447,9 @@ case class StructType(fields: Seq[StructField]) extends DataType { fields.foreach(field => field.buildFormattedString(prefix, builder)) } - def simpleString: String = "struct" + override private[sql] def jsonValue = + ("type" -> typeName) ~ + ("fields" -> fields.map(_.jsonValue)) } object MapType { @@ -394,6 +459,8 @@ object MapType { */ def apply(keyType: DataType, valueType: DataType): MapType = MapType(keyType: DataType, valueType: DataType, true) + + def simpleName = "map" } /** @@ -407,12 +474,16 @@ case class MapType( valueType: DataType, valueContainsNull: Boolean) extends DataType { private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { - builder.append(s"${prefix}-- key: ${keyType.simpleString}\n") - builder.append(s"${prefix}-- value: ${valueType.simpleString} " + - s"(valueContainsNull = ${valueContainsNull})\n") + builder.append(s"$prefix-- key: ${keyType.typeName}\n") + builder.append(s"$prefix-- value: ${valueType.typeName} " + + s"(valueContainsNull = $valueContainsNull)\n") DataType.buildFormattedString(keyType, s"$prefix |", builder) DataType.buildFormattedString(valueType, s"$prefix |", builder) } - def simpleString: String = "map" + override private[sql] def jsonValue: JValue = + ("type" -> typeName) ~ + ("keyType" -> keyType.jsonValue) ~ + ("valueType" -> valueType.jsonValue) ~ + ("valueContainsNull" -> valueContainsNull) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7a55c5bf97a71..35561cac3e5e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -22,6 +22,7 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.conf.Configuration +import org.apache.spark.SparkContext import org.apache.spark.annotation.{AlphaComponent, DeveloperApi, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.ScalaReflection @@ -31,12 +32,11 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.types.DataType import org.apache.spark.sql.columnar.InMemoryRelation -import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.SparkStrategies +import org.apache.spark.sql.execution.{SparkStrategies, _} import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.{Logging, SparkContext} /** * :: AlphaComponent :: @@ -409,8 +409,7 @@ class SQLContext(@transient val sparkContext: SparkContext) * It is only used by PySpark. */ private[sql] def parseDataType(dataTypeString: String): DataType = { - val parser = org.apache.spark.sql.catalyst.types.DataType - parser(dataTypeString) + DataType.fromJson(dataTypeString) } /** 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 2941b9793597f..e6389cf77a4c9 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql.parquet import java.io.IOException +import scala.util.Try + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.Job @@ -323,14 +325,14 @@ private[parquet] object ParquetTypesConverter extends Logging { } def convertFromString(string: String): Seq[Attribute] = { - DataType(string) match { + Try(DataType.fromJson(string)).getOrElse(DataType.fromCaseClassString(string)) match { case s: StructType => s.toAttributes case other => sys.error(s"Can convert $string to row") } } def convertToString(schema: Seq[Attribute]): String = { - StructType.fromAttributes(schema).toString + StructType.fromAttributes(schema).json } def writeMetaData(attributes: Seq[Attribute], origPath: Path, conf: Configuration): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala index 8fb59c5830f6d..100ecb45e9e88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.types.DataType + class DataTypeSuite extends FunSuite { test("construct an ArrayType") { @@ -55,4 +57,30 @@ class DataTypeSuite extends FunSuite { struct(Set("b", "d", "e", "f")) } } + + def checkDataTypeJsonRepr(dataType: DataType): Unit = { + test(s"JSON - $dataType") { + assert(DataType.fromJson(dataType.json) === dataType) + } + } + + checkDataTypeJsonRepr(BooleanType) + checkDataTypeJsonRepr(ByteType) + checkDataTypeJsonRepr(ShortType) + checkDataTypeJsonRepr(IntegerType) + checkDataTypeJsonRepr(LongType) + checkDataTypeJsonRepr(FloatType) + checkDataTypeJsonRepr(DoubleType) + checkDataTypeJsonRepr(DecimalType) + checkDataTypeJsonRepr(TimestampType) + checkDataTypeJsonRepr(StringType) + checkDataTypeJsonRepr(BinaryType) + checkDataTypeJsonRepr(ArrayType(DoubleType, true)) + checkDataTypeJsonRepr(ArrayType(StringType, false)) + checkDataTypeJsonRepr(MapType(IntegerType, StringType, true)) + checkDataTypeJsonRepr(MapType(IntegerType, ArrayType(DoubleType), false)) + checkDataTypeJsonRepr( + StructType(Seq( + StructField("a", IntegerType, nullable = true), + StructField("b", ArrayType(DoubleType), nullable = false)))) } 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 07adf731405af..25e41ecf28e2e 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 @@ -789,7 +789,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } - + test("Querying on empty parquet throws exception (SPARK-3536)") { val tmpdir = Utils.createTempDir() Utils.deleteRecursively(tmpdir) @@ -798,4 +798,18 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result1.size === 0) Utils.deleteRecursively(tmpdir) } + + test("DataType string parser compatibility") { + val schema = StructType(List( + StructField("c1", IntegerType, false), + StructField("c2", BinaryType, false))) + + val fromCaseClassString = ParquetTypesConverter.convertFromString(schema.toString) + val fromJson = ParquetTypesConverter.convertFromString(schema.json) + + (fromCaseClassString, fromJson).zipped.foreach { (a, b) => + assert(a.name == b.name) + assert(a.dataType === b.dataType) + } + } } From 00b7791720e50119a98084b2e8755e1b593ca55f Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Wed, 8 Oct 2014 17:16:54 -0700 Subject: [PATCH 0931/1492] [SQL][Doc] Keep Spark SQL README.md up to date marmbrus Update README.md to be consistent with Spark 1.1 Author: Liquan Pei Closes #2706 from Ishiihara/SparkSQL-readme and squashes the following commits: 33b9d4b [Liquan Pei] keep README.md up to date --- sql/README.md | 31 +++++++++++++++---------------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/sql/README.md b/sql/README.md index 31f9152344086..c84534da9a3d3 100644 --- a/sql/README.md +++ b/sql/README.md @@ -44,38 +44,37 @@ Type in expressions to have them evaluated. Type :help for more information. scala> val query = sql("SELECT * FROM (SELECT * FROM src) a") -query: org.apache.spark.sql.ExecutedQuery = -SELECT * FROM (SELECT * FROM src) a -=== Query Plan === -Project [key#6:0.0,value#7:0.1] - HiveTableScan [key#6,value#7], (MetastoreRelation default, src, None), None +query: org.apache.spark.sql.SchemaRDD = +== Query Plan == +== Physical Plan == +HiveTableScan [key#10,value#11], (MetastoreRelation default, src, None), None ``` Query results are RDDs and can be operated as such. ``` scala> query.collect() -res8: Array[org.apache.spark.sql.execution.Row] = Array([238,val_238], [86,val_86], [311,val_311]... +res2: Array[org.apache.spark.sql.Row] = Array([238,val_238], [86,val_86], [311,val_311], [27,val_27]... ``` You can also build further queries on top of these RDDs using the query DSL. ``` -scala> query.where('key === 100).toRdd.collect() -res11: Array[org.apache.spark.sql.execution.Row] = Array([100,val_100], [100,val_100]) +scala> query.where('key === 100).collect() +res3: Array[org.apache.spark.sql.Row] = Array([100,val_100], [100,val_100]) ``` -From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](http://databricks.github.io/catalyst/latest/api/#catalyst.trees.TreeNode) objects. +From the console you can even write rules that transform query plans. For example, the above query has redundant project operators that aren't doing anything. This redundancy can be eliminated using the `transform` function that is available on all [`TreeNode`](https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala) objects. ```scala -scala> query.logicalPlan -res1: catalyst.plans.logical.LogicalPlan = -Project {key#0,value#1} - Project {key#0,value#1} +scala> query.queryExecution.analyzed +res4: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = +Project [key#10,value#11] + Project [key#10,value#11] MetastoreRelation default, src, None -scala> query.logicalPlan transform { +scala> query.queryExecution.analyzed transform { | case Project(projectList, child) if projectList == child.output => child | } -res2: catalyst.plans.logical.LogicalPlan = -Project {key#0,value#1} +res5: res17: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan = +Project [key#10,value#11] MetastoreRelation default, src, None ``` From 4ec931951fea4efbfe5db39cf581704df7d2775b Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Wed, 8 Oct 2014 17:52:27 -0700 Subject: [PATCH 0932/1492] [SPARK-3707] [SQL] Fix bug of type coercion in DIV Calling `BinaryArithmetic.dataType` will throws exception until it's resolved, but in type coercion rule `Division`, seems doesn't follow this. Author: Cheng Hao Closes #2559 from chenghao-intel/type_coercion and squashes the following commits: 199a85d [Cheng Hao] Simplify the divide rule dc55218 [Cheng Hao] fix bug of type coercion in div --- .../catalyst/analysis/HiveTypeCoercion.scala | 7 +++- .../sql/catalyst/analysis/AnalysisSuite.scala | 40 +++++++++++++++++-- 2 files changed, 42 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 79e5283e86a37..64881854df7a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -348,8 +348,11 @@ trait HiveTypeCoercion { case e if !e.childrenResolved => e // Decimal and Double remain the same - case d: Divide if d.dataType == DoubleType => d - case d: Divide if d.dataType == DecimalType => d + case d: Divide if d.resolved && d.dataType == DoubleType => d + case d: Divide if d.resolved && d.dataType == DecimalType => d + + case Divide(l, r) if l.dataType == DecimalType => Divide(l, Cast(r, DecimalType)) + case Divide(l, r) if r.dataType == DecimalType => Divide(Cast(l, DecimalType), r) case Divide(l, r) => Divide(Cast(l, DoubleType), Cast(r, DoubleType)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 5809a108ff62e..7b45738c4fc95 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.sql.catalyst.analysis import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.types.IntegerType +import org.apache.spark.sql.catalyst.types._ class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) @@ -33,6 +34,12 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { new Analyzer(caseInsensitiveCatalog, EmptyFunctionRegistry, caseSensitive = false) val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) + val testRelation2 = LocalRelation( + AttributeReference("a", StringType)(), + AttributeReference("b", StringType)(), + AttributeReference("c", DoubleType)(), + AttributeReference("d", DecimalType)(), + AttributeReference("e", ShortType)()) before { caseSensitiveCatalog.registerTable(None, "TaBlE", testRelation) @@ -74,7 +81,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { val e = intercept[RuntimeException] { caseSensitiveAnalyze(UnresolvedRelation(None, "tAbLe", None)) } - assert(e.getMessage === "Table Not Found: tAbLe") + assert(e.getMessage == "Table Not Found: tAbLe") assert( caseSensitiveAnalyze(UnresolvedRelation(None, "TaBlE", None)) === @@ -106,4 +113,31 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { } assert(e.getMessage().toLowerCase.contains("unresolved plan")) } + + test("divide should be casted into fractional types") { + val testRelation2 = LocalRelation( + AttributeReference("a", StringType)(), + AttributeReference("b", StringType)(), + AttributeReference("c", DoubleType)(), + AttributeReference("d", DecimalType)(), + AttributeReference("e", ShortType)()) + + val expr0 = 'a / 2 + val expr1 = 'a / 'b + val expr2 = 'a / 'c + val expr3 = 'a / 'd + val expr4 = 'e / 'e + val plan = caseInsensitiveAnalyze(Project( + Alias(expr0, s"Analyzer($expr0)")() :: + Alias(expr1, s"Analyzer($expr1)")() :: + Alias(expr2, s"Analyzer($expr2)")() :: + Alias(expr3, s"Analyzer($expr3)")() :: + Alias(expr4, s"Analyzer($expr4)")() :: Nil, testRelation2)) + val pl = plan.asInstanceOf[Project].projectList + assert(pl(0).dataType == DoubleType) + assert(pl(1).dataType == DoubleType) + assert(pl(2).dataType == DoubleType) + assert(pl(3).dataType == DecimalType) + assert(pl(4).dataType == DoubleType) + } } From e7033572330bd48b2438f218b0d2cd3fccdeb362 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Oct 2014 18:11:18 -0700 Subject: [PATCH 0933/1492] [SPARK-3810][SQL] Makes PreInsertionCasts handle partitions properly Includes partition keys into account when applying `PreInsertionCasts` rule. Author: Cheng Lian Closes #2672 from liancheng/fix-pre-insert-casts and squashes the following commits: def1a1a [Cheng Lian] Makes PreInsertionCasts handle partitions properly --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 15 +++----- .../sql/hive/execution/HiveQuerySuite.scala | 36 +++++++++++++++++++ 2 files changed, 41 insertions(+), 10 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 cc0605b0adb35..addd5bed8426d 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 @@ -19,31 +19,28 @@ package org.apache.spark.sql.hive import scala.util.parsing.combinator.RegexParsers -import org.apache.hadoop.hive.metastore.api.{FieldSchema, StorageDescriptor, SerDeInfo} -import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, SerDeInfo, StorageDescriptor, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.Deserializer -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} +import org.apache.spark.sql.catalyst.analysis.Catalog import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical 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.columnar.InMemoryRelation -import org.apache.spark.sql.hive.execution.HiveTableScan import org.apache.spark.util.Utils /* Implicit conversions */ import scala.collection.JavaConversions._ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { - import HiveMetastoreTypes._ + import org.apache.spark.sql.hive.HiveMetastoreTypes._ /** Connection to hive metastore. Usages should lock on `this`. */ protected[hive] val client = Hive.get(hive.hiveconf) @@ -137,10 +134,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) = { val childOutputDataTypes = child.output.map(_.dataType) - // Only check attributes, not partitionKeys since they are always strings. - // TODO: Fully support inserting into partitioned tables. val tableOutputDataTypes = - table.attributes.map(_.dataType) ++ table.partitionKeys.map(_.dataType) + (table.attributes ++ table.partitionKeys).take(child.output.length).map(_.dataType) if (childOutputDataTypes == tableOutputDataTypes) { p diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 2e282a9ade40c..2829105f43716 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -22,6 +22,7 @@ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -675,6 +676,41 @@ class HiveQuerySuite extends HiveComparisonTest { sql("SELECT * FROM boom").queryExecution.analyzed } + test("SPARK-3810: PreInsertionCasts static partitioning support") { + val analyzedPlan = { + loadTestTable("srcpart") + sql("DROP TABLE IF EXISTS withparts") + sql("CREATE TABLE withparts LIKE srcpart") + sql("INSERT INTO TABLE withparts PARTITION(ds='1', hr='2') SELECT key, value FROM src") + .queryExecution.analyzed + } + + assertResult(1, "Duplicated project detected\n" + analyzedPlan) { + analyzedPlan.collect { + case _: Project => () + }.size + } + } + + test("SPARK-3810: PreInsertionCasts dynamic partitioning support") { + val analyzedPlan = { + loadTestTable("srcpart") + sql("DROP TABLE IF EXISTS withparts") + sql("CREATE TABLE withparts LIKE srcpart") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + sql("CREATE TABLE IF NOT EXISTS withparts LIKE srcpart") + sql("INSERT INTO TABLE withparts PARTITION(ds, hr) SELECT key, value FROM src") + .queryExecution.analyzed + } + + assertResult(1, "Duplicated project detected\n" + analyzedPlan) { + analyzedPlan.collect { + case _: Project => () + }.size + } + } + test("parse HQL set commands") { // Adapted from its SQL counterpart. val testKey = "spark.sql.key.usedfortestonly" From 3e4f09d2fce9dcf45eaaca827f2cf15c9d4a6c75 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Oct 2014 18:13:22 -0700 Subject: [PATCH 0934/1492] [SQL] Prevents per row dynamic dispatching and pattern matching when inserting Hive values Builds all wrappers at first according to object inspector types to avoid per row costs. Author: Cheng Lian Closes #2592 from liancheng/hive-value-wrapper and squashes the following commits: 9696559 [Cheng Lian] Passes all tests 4998666 [Cheng Lian] Prevents per row dynamic dispatching and pattern matching when inserting Hive values --- .../hive/execution/InsertIntoHiveTable.scala | 64 ++++++++++--------- 1 file changed, 34 insertions(+), 30 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index f8b4e898ec41d..f0785d8882636 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -69,33 +69,36 @@ case class InsertIntoHiveTable( * Wraps with Hive types based on object inspector. * TODO: Consolidate all hive OI/data interface code. */ - protected def wrap(a: (Any, ObjectInspector)): Any = a match { - case (s: String, oi: JavaHiveVarcharObjectInspector) => - new HiveVarchar(s, s.size) - - case (bd: BigDecimal, oi: JavaHiveDecimalObjectInspector) => - new HiveDecimal(bd.underlying()) - - case (row: Row, oi: StandardStructObjectInspector) => - val struct = oi.create() - row.zip(oi.getAllStructFieldRefs: Seq[StructField]).foreach { - case (data, field) => - oi.setStructFieldData(struct, field, wrap(data, field.getFieldObjectInspector)) + protected def wrapperFor(oi: ObjectInspector): Any => Any = oi match { + case _: JavaHiveVarcharObjectInspector => + (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) + + case _: JavaHiveDecimalObjectInspector => + (o: Any) => new HiveDecimal(o.asInstanceOf[BigDecimal].underlying()) + + case soi: StandardStructObjectInspector => + val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) + (o: Any) => { + val struct = soi.create() + (soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[Row]).zipped.foreach { + (field, wrapper, data) => soi.setStructFieldData(struct, field, wrapper(data)) + } + struct } - struct - case (s: Seq[_], oi: ListObjectInspector) => - val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) - seqAsJavaList(wrappedSeq) + case loi: ListObjectInspector => + val wrapper = wrapperFor(loi.getListElementObjectInspector) + (o: Any) => seqAsJavaList(o.asInstanceOf[Seq[_]].map(wrapper)) - case (m: Map[_, _], oi: MapObjectInspector) => - val keyOi = oi.getMapKeyObjectInspector - val valueOi = oi.getMapValueObjectInspector - val wrappedMap = m.map { case (key, value) => wrap(key, keyOi) -> wrap(value, valueOi) } - mapAsJavaMap(wrappedMap) + case moi: MapObjectInspector => + val keyWrapper = wrapperFor(moi.getMapKeyObjectInspector) + val valueWrapper = wrapperFor(moi.getMapValueObjectInspector) + (o: Any) => mapAsJavaMap(o.asInstanceOf[Map[_, _]].map { case (key, value) => + keyWrapper(key) -> valueWrapper(value) + }) - case (obj, _) => - obj + case _ => + identity[Any] } def saveAsHiveFile( @@ -103,7 +106,7 @@ case class InsertIntoHiveTable( valueClass: Class[_], fileSinkConf: FileSinkDesc, conf: SerializableWritable[JobConf], - writerContainer: SparkHiveWriterContainer) { + writerContainer: SparkHiveWriterContainer): Unit = { assert(valueClass != null, "Output value class not set") conf.value.setOutputValueClass(valueClass) @@ -122,7 +125,7 @@ case class InsertIntoHiveTable( writerContainer.commitJob() // Note that this function is executed on executor side - def writeToFile(context: TaskContext, iterator: Iterator[Row]) { + def writeToFile(context: TaskContext, iterator: Iterator[Row]): Unit = { val serializer = newSerializer(fileSinkConf.getTableInfo) val standardOI = ObjectInspectorUtils .getStandardObjectInspector( @@ -131,6 +134,7 @@ case class InsertIntoHiveTable( .asInstanceOf[StructObjectInspector] val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector).toArray + val wrappers = fieldOIs.map(wrapperFor) val outputData = new Array[Any](fieldOIs.length) // Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it @@ -141,13 +145,13 @@ case class InsertIntoHiveTable( iterator.foreach { row => var i = 0 while (i < fieldOIs.length) { - // TODO (lian) avoid per row dynamic dispatching and pattern matching cost in `wrap` - outputData(i) = wrap(row(i), fieldOIs(i)) + outputData(i) = if (row.isNullAt(i)) null else wrappers(i)(row(i)) i += 1 } - val writer = writerContainer.getLocalFileWriter(row) - writer.write(serializer.serialize(outputData, standardOI)) + writerContainer + .getLocalFileWriter(row) + .write(serializer.serialize(outputData, standardOI)) } writerContainer.close() @@ -207,7 +211,7 @@ case class InsertIntoHiveTable( // Report error if any static partition appears after a dynamic partition val isDynamic = partitionColumnNames.map(partitionSpec(_).isEmpty) - isDynamic.init.zip(isDynamic.tail).find(_ == (true, false)).foreach { _ => + if (isDynamic.init.zip(isDynamic.tail).contains((true, false))) { throw new SparkException(ErrorMsg.PARTITION_DYN_STA_ORDER.getMsg) } } From bcb1ae049b447c37418747e0a262f54f9fc1664a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Oct 2014 18:17:01 -0700 Subject: [PATCH 0935/1492] [SPARK-3857] Create joins package for various join operators. Author: Reynold Xin Closes #2719 from rxin/sql-join-break and squashes the following commits: 0c0082b [Reynold Xin] Fix line length. cbc664c [Reynold Xin] Rename join -> joins package. a070d44 [Reynold Xin] Fix line length in HashJoin a39be8c [Reynold Xin] [SPARK-3857] Create a join package for various join operators. --- .../spark/sql/execution/SparkStrategies.scala | 41 +- .../apache/spark/sql/execution/joins.scala | 624 ------------------ .../execution/joins/BroadcastHashJoin.scala | 62 ++ .../joins/BroadcastNestedLoopJoin.scala | 144 ++++ .../execution/joins/CartesianProduct.scala | 40 ++ .../spark/sql/execution/joins/HashJoin.scala | 123 ++++ .../sql/execution/joins/HashOuterJoin.scala | 222 +++++++ .../sql/execution/joins/LeftSemiJoinBNL.scala | 73 ++ .../execution/joins/LeftSemiJoinHash.scala | 67 ++ .../execution/joins/ShuffledHashJoin.scala | 49 ++ .../spark/sql/execution/joins/package.scala | 37 ++ .../org/apache/spark/sql/JoinSuite.scala | 1 + .../org/apache/spark/sql/SQLQuerySuite.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 3 +- .../spark/sql/hive/StatisticsSuite.scala | 2 +- 15 files changed, 844 insertions(+), 646 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/package.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 883f2ff521e20..bbf17b9fadf86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.{InMemoryRelation, InMemoryColumnarTableScan} import org.apache.spark.sql.parquet._ + private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => @@ -34,13 +35,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { // Find left semi joins where at least some predicates can be evaluated by matching join keys case ExtractEquiJoinKeys(LeftSemi, leftKeys, rightKeys, condition, left, right) => - val semiJoin = execution.LeftSemiJoinHash( + val semiJoin = joins.LeftSemiJoinHash( leftKeys, rightKeys, planLater(left), planLater(right)) condition.map(Filter(_, semiJoin)).getOrElse(semiJoin) :: Nil // no predicate can be evaluated by matching hash keys case logical.Join(left, right, LeftSemi, condition) => - execution.LeftSemiJoinBNL( - planLater(left), planLater(right), condition) :: Nil + joins.LeftSemiJoinBNL(planLater(left), planLater(right), condition) :: Nil case _ => Nil } } @@ -50,13 +50,13 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * evaluated by matching hash keys. * * This strategy applies a simple optimization based on the estimates of the physical sizes of - * the two join sides. When planning a [[execution.BroadcastHashJoin]], if one side has an + * the two join sides. When planning a [[joins.BroadcastHashJoin]], if one side has an * estimated physical size smaller than the user-settable threshold * [[org.apache.spark.sql.SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]], the planner would mark it as the * ''build'' relation and mark the other relation as the ''stream'' side. The build table will be * ''broadcasted'' to all of the executors involved in the join, as a * [[org.apache.spark.broadcast.Broadcast]] object. If both estimates exceed the threshold, they - * will instead be used to decide the build side in a [[execution.ShuffledHashJoin]]. + * will instead be used to decide the build side in a [[joins.ShuffledHashJoin]]. */ object HashJoin extends Strategy with PredicateHelper { @@ -66,8 +66,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { left: LogicalPlan, right: LogicalPlan, condition: Option[Expression], - side: BuildSide) = { - val broadcastHashJoin = execution.BroadcastHashJoin( + side: joins.BuildSide) = { + val broadcastHashJoin = execution.joins.BroadcastHashJoin( leftKeys, rightKeys, side, planLater(left), planLater(right)) condition.map(Filter(_, broadcastHashJoin)).getOrElse(broadcastHashJoin) :: Nil } @@ -76,27 +76,26 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) if sqlContext.autoBroadcastJoinThreshold > 0 && right.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => - makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildRight) + makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildRight) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) if sqlContext.autoBroadcastJoinThreshold > 0 && left.statistics.sizeInBytes <= sqlContext.autoBroadcastJoinThreshold => - makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, BuildLeft) + makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildLeft) case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => val buildSide = if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { - BuildRight + joins.BuildRight } else { - BuildLeft + joins.BuildLeft } - val hashJoin = - execution.ShuffledHashJoin( - leftKeys, rightKeys, buildSide, planLater(left), planLater(right)) + val hashJoin = joins.ShuffledHashJoin( + leftKeys, rightKeys, buildSide, planLater(left), planLater(right)) condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) => - execution.HashOuterJoin( + joins.HashOuterJoin( leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil case _ => Nil @@ -164,8 +163,12 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, joinType, condition) => val buildSide = - if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) BuildRight else BuildLeft - execution.BroadcastNestedLoopJoin( + if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { + joins.BuildRight + } else { + joins.BuildLeft + } + joins.BroadcastNestedLoopJoin( planLater(left), planLater(right), buildSide, joinType, condition) :: Nil case _ => Nil } @@ -174,10 +177,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object CartesianProduct extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Join(left, right, _, None) => - execution.CartesianProduct(planLater(left), planLater(right)) :: Nil + execution.joins.CartesianProduct(planLater(left), planLater(right)) :: Nil case logical.Join(left, right, Inner, Some(condition)) => execution.Filter(condition, - execution.CartesianProduct(planLater(left), planLater(right))) :: Nil + execution.joins.CartesianProduct(planLater(left), planLater(right))) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala deleted file mode 100644 index 2890a563bed48..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala +++ /dev/null @@ -1,624 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution - -import java.util.{HashMap => JavaHashMap} - -import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent._ -import scala.concurrent.duration._ - -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.util.collection.CompactBuffer - -@DeveloperApi -sealed abstract class BuildSide - -@DeveloperApi -case object BuildLeft extends BuildSide - -@DeveloperApi -case object BuildRight extends BuildSide - -trait HashJoin { - self: SparkPlan => - - val leftKeys: Seq[Expression] - val rightKeys: Seq[Expression] - val buildSide: BuildSide - val left: SparkPlan - val right: SparkPlan - - lazy val (buildPlan, streamedPlan) = buildSide match { - case BuildLeft => (left, right) - case BuildRight => (right, left) - } - - lazy val (buildKeys, streamedKeys) = buildSide match { - case BuildLeft => (leftKeys, rightKeys) - case BuildRight => (rightKeys, leftKeys) - } - - def output = left.output ++ right.output - - @transient lazy val buildSideKeyGenerator = newProjection(buildKeys, buildPlan.output) - @transient lazy val streamSideKeyGenerator = - newMutableProjection(streamedKeys, streamedPlan.output) - - def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = { - // TODO: Use Spark's HashMap implementation. - - val hashTable = new java.util.HashMap[Row, CompactBuffer[Row]]() - var currentRow: Row = null - - // Create a mapping of buildKeys -> rows - while (buildIter.hasNext) { - currentRow = buildIter.next() - val rowKey = buildSideKeyGenerator(currentRow) - if (!rowKey.anyNull) { - val existingMatchList = hashTable.get(rowKey) - val matchList = if (existingMatchList == null) { - val newMatchList = new CompactBuffer[Row]() - hashTable.put(rowKey, newMatchList) - newMatchList - } else { - existingMatchList - } - matchList += currentRow.copy() - } - } - - new Iterator[Row] { - private[this] var currentStreamedRow: Row = _ - private[this] var currentHashMatches: CompactBuffer[Row] = _ - private[this] var currentMatchPosition: Int = -1 - - // Mutable per row objects. - private[this] val joinRow = new JoinedRow2 - - private[this] val joinKeys = streamSideKeyGenerator() - - override final def hasNext: Boolean = - (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || - (streamIter.hasNext && fetchNext()) - - override final def next() = { - val ret = buildSide match { - case BuildRight => joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) - case BuildLeft => joinRow(currentHashMatches(currentMatchPosition), currentStreamedRow) - } - currentMatchPosition += 1 - ret - } - - /** - * Searches the streamed iterator for the next row that has at least one match in hashtable. - * - * @return true if the search is successful, and false if the streamed iterator runs out of - * tuples. - */ - private final def fetchNext(): Boolean = { - currentHashMatches = null - currentMatchPosition = -1 - - while (currentHashMatches == null && streamIter.hasNext) { - currentStreamedRow = streamIter.next() - if (!joinKeys(currentStreamedRow).anyNull) { - currentHashMatches = hashTable.get(joinKeys.currentValue) - } - } - - if (currentHashMatches == null) { - false - } else { - currentMatchPosition = 0 - true - } - } - } - } -} - -/** - * :: DeveloperApi :: - * Performs a hash based outer join for two child relations by shuffling the data using - * the join keys. This operator requires loading the associated partition in both side into memory. - */ -@DeveloperApi -case class HashOuterJoin( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan) extends BinaryNode { - - override def outputPartitioning: Partitioning = joinType match { - case LeftOuter => left.outputPartitioning - case RightOuter => right.outputPartitioning - case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) - case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") - } - - override def requiredChildDistribution = - ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - - override def output = { - joinType match { - case LeftOuter => - left.output ++ right.output.map(_.withNullability(true)) - case RightOuter => - left.output.map(_.withNullability(true)) ++ right.output - case FullOuter => - left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) - case x => - throw new Exception(s"HashOuterJoin should not take $x as the JoinType") - } - } - - @transient private[this] lazy val DUMMY_LIST = Seq[Row](null) - @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row] - - // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala - // iterator for performance purpose. - - private[this] def leftOuterIterator( - key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { - val joinedRow = new JoinedRow() - val rightNullRow = new GenericRow(right.output.length) - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - - leftIter.iterator.flatMap { l => - joinedRow.withLeft(l) - var matched = false - (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => - matched = true - joinedRow.copy - } else { - Nil - }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { - // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the - // records in right side. - // If we didn't get any proper row, then append a single row with empty right - joinedRow.withRight(rightNullRow).copy - }) - } - } - - private[this] def rightOuterIterator( - key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { - val joinedRow = new JoinedRow() - val leftNullRow = new GenericRow(left.output.length) - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - - rightIter.iterator.flatMap { r => - joinedRow.withRight(r) - var matched = false - (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => - matched = true - joinedRow.copy - } else { - Nil - }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { - // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all of the - // records in left side. - // If we didn't get any proper row, then append a single row with empty left. - joinedRow.withLeft(leftNullRow).copy - }) - } - } - - private[this] def fullOuterIterator( - key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { - val joinedRow = new JoinedRow() - val leftNullRow = new GenericRow(left.output.length) - val rightNullRow = new GenericRow(right.output.length) - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - - if (!key.anyNull) { - // Store the positions of records in right, if one of its associated row satisfy - // the join condition. - val rightMatchedSet = scala.collection.mutable.Set[Int]() - leftIter.iterator.flatMap[Row] { l => - joinedRow.withLeft(l) - var matched = false - rightIter.zipWithIndex.collect { - // 1. For those matched (satisfy the join condition) records with both sides filled, - // append them directly - - case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> { - matched = true - // if the row satisfy the join condition, add its index into the matched set - rightMatchedSet.add(idx) - joinedRow.copy - } - } ++ DUMMY_LIST.filter(_ => !matched).map( _ => { - // 2. For those unmatched records in left, append additional records with empty right. - - // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, - // as we don't know whether we need to append it until finish iterating all - // of the records in right side. - // If we didn't get any proper row, then append a single row with empty right. - joinedRow.withRight(rightNullRow).copy - }) - } ++ rightIter.zipWithIndex.collect { - // 3. For those unmatched records in right, append additional records with empty left. - - // Re-visiting the records in right, and append additional row with empty left, if its not - // in the matched set. - case (r, idx) if (!rightMatchedSet.contains(idx)) => { - joinedRow(leftNullRow, r).copy - } - } - } else { - leftIter.iterator.map[Row] { l => - joinedRow(l, rightNullRow).copy - } ++ rightIter.iterator.map[Row] { r => - joinedRow(leftNullRow, r).copy - } - } - } - - private[this] def buildHashTable( - iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, CompactBuffer[Row]] = { - val hashTable = new JavaHashMap[Row, CompactBuffer[Row]]() - while (iter.hasNext) { - val currentRow = iter.next() - val rowKey = keyGenerator(currentRow) - - var existingMatchList = hashTable.get(rowKey) - if (existingMatchList == null) { - existingMatchList = new CompactBuffer[Row]() - hashTable.put(rowKey, existingMatchList) - } - - existingMatchList += currentRow.copy() - } - - hashTable - } - - def execute() = { - left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => - // TODO this probably can be replaced by external sort (sort merged join?) - // Build HashMap for current partition in left relation - val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) - // Build HashMap for current partition in right relation - val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) - - import scala.collection.JavaConversions._ - val boundCondition = - condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) - joinType match { - case LeftOuter => leftHashTable.keysIterator.flatMap { key => - leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), - rightHashTable.getOrElse(key, EMPTY_LIST)) - } - case RightOuter => rightHashTable.keysIterator.flatMap { key => - rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), - rightHashTable.getOrElse(key, EMPTY_LIST)) - } - case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => - fullOuterIterator(key, - leftHashTable.getOrElse(key, EMPTY_LIST), - rightHashTable.getOrElse(key, EMPTY_LIST)) - } - case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") - } - } - } -} - -/** - * :: DeveloperApi :: - * Performs an inner hash join of two child relations by first shuffling the data using the join - * keys. - */ -@DeveloperApi -case class ShuffledHashJoin( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - buildSide: BuildSide, - left: SparkPlan, - right: SparkPlan) extends BinaryNode with HashJoin { - - override def outputPartitioning: Partitioning = left.outputPartitioning - - override def requiredChildDistribution = - ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - - def execute() = { - buildPlan.execute().zipPartitions(streamedPlan.execute()) { - (buildIter, streamIter) => joinIterators(buildIter, streamIter) - } - } -} - -/** - * :: DeveloperApi :: - * Build the right table's join keys into a HashSet, and iteratively go through the left - * table, to find the if join keys are in the Hash set. - */ -@DeveloperApi -case class LeftSemiJoinHash( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - left: SparkPlan, - right: SparkPlan) extends BinaryNode with HashJoin { - - val buildSide = BuildRight - - override def requiredChildDistribution = - ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - - override def output = left.output - - def execute() = { - buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => - val hashSet = new java.util.HashSet[Row]() - var currentRow: Row = null - - // Create a Hash set of buildKeys - while (buildIter.hasNext) { - currentRow = buildIter.next() - val rowKey = buildSideKeyGenerator(currentRow) - if (!rowKey.anyNull) { - val keyExists = hashSet.contains(rowKey) - if (!keyExists) { - hashSet.add(rowKey) - } - } - } - - val joinKeys = streamSideKeyGenerator() - streamIter.filter(current => { - !joinKeys(current).anyNull && hashSet.contains(joinKeys.currentValue) - }) - } - } -} - - -/** - * :: DeveloperApi :: - * Performs an inner hash join of two child relations. When the output RDD of this operator is - * being constructed, a Spark job is asynchronously started to calculate the values for the - * broadcasted relation. This data is then placed in a Spark broadcast variable. The streamed - * relation is not shuffled. - */ -@DeveloperApi -case class BroadcastHashJoin( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - buildSide: BuildSide, - left: SparkPlan, - right: SparkPlan) extends BinaryNode with HashJoin { - - override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning - - override def requiredChildDistribution = - UnspecifiedDistribution :: UnspecifiedDistribution :: Nil - - @transient - val broadcastFuture = future { - sparkContext.broadcast(buildPlan.executeCollect()) - } - - def execute() = { - val broadcastRelation = Await.result(broadcastFuture, 5.minute) - - streamedPlan.execute().mapPartitions { streamedIter => - joinIterators(broadcastRelation.value.iterator, streamedIter) - } - } -} - -/** - * :: DeveloperApi :: - * Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys - * for hash join. - */ -@DeveloperApi -case class LeftSemiJoinBNL( - streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) - extends BinaryNode { - // TODO: Override requiredChildDistribution. - - override def outputPartitioning: Partitioning = streamed.outputPartitioning - - def output = left.output - - /** The Streamed Relation */ - def left = streamed - /** The Broadcast relation */ - def right = broadcast - - @transient lazy val boundCondition = - InterpretedPredicate( - condition - .map(c => BindReferences.bindReference(c, left.output ++ right.output)) - .getOrElse(Literal(true))) - - def execute() = { - val broadcastedRelation = - sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) - - streamed.execute().mapPartitions { streamedIter => - val joinedRow = new JoinedRow - - streamedIter.filter(streamedRow => { - var i = 0 - var matched = false - - while (i < broadcastedRelation.value.size && !matched) { - val broadcastedRow = broadcastedRelation.value(i) - if (boundCondition(joinedRow(streamedRow, broadcastedRow))) { - matched = true - } - i += 1 - } - matched - }) - } - } -} - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { - def output = left.output ++ right.output - - def execute() = { - val leftResults = left.execute().map(_.copy()) - val rightResults = right.execute().map(_.copy()) - - leftResults.cartesian(rightResults).mapPartitions { iter => - val joinedRow = new JoinedRow - iter.map(r => joinedRow(r._1, r._2)) - } - } -} - -/** - * :: DeveloperApi :: - */ -@DeveloperApi -case class BroadcastNestedLoopJoin( - left: SparkPlan, - right: SparkPlan, - buildSide: BuildSide, - joinType: JoinType, - condition: Option[Expression]) extends BinaryNode { - // TODO: Override requiredChildDistribution. - - /** BuildRight means the right relation <=> the broadcast relation. */ - val (streamed, broadcast) = buildSide match { - case BuildRight => (left, right) - case BuildLeft => (right, left) - } - - override def outputPartitioning: Partitioning = streamed.outputPartitioning - - override def output = { - joinType match { - case LeftOuter => - left.output ++ right.output.map(_.withNullability(true)) - case RightOuter => - left.output.map(_.withNullability(true)) ++ right.output - case FullOuter => - left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) - case _ => - left.output ++ right.output - } - } - - @transient lazy val boundCondition = - InterpretedPredicate( - condition - .map(c => BindReferences.bindReference(c, left.output ++ right.output)) - .getOrElse(Literal(true))) - - def execute() = { - val broadcastedRelation = - sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) - - /** All rows that either match both-way, or rows from streamed joined with nulls. */ - val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => - val matchedRows = new CompactBuffer[Row] - // TODO: Use Spark's BitSet. - val includedBroadcastTuples = - new scala.collection.mutable.BitSet(broadcastedRelation.value.size) - val joinedRow = new JoinedRow - val leftNulls = new GenericMutableRow(left.output.size) - val rightNulls = new GenericMutableRow(right.output.size) - - streamedIter.foreach { streamedRow => - var i = 0 - var streamRowMatched = false - - while (i < broadcastedRelation.value.size) { - // TODO: One bitset per partition instead of per row. - val broadcastedRow = broadcastedRelation.value(i) - buildSide match { - case BuildRight if boundCondition(joinedRow(streamedRow, broadcastedRow)) => - matchedRows += joinedRow(streamedRow, broadcastedRow).copy() - streamRowMatched = true - includedBroadcastTuples += i - case BuildLeft if boundCondition(joinedRow(broadcastedRow, streamedRow)) => - matchedRows += joinedRow(broadcastedRow, streamedRow).copy() - streamRowMatched = true - includedBroadcastTuples += i - case _ => - } - i += 1 - } - - (streamRowMatched, joinType, buildSide) match { - case (false, LeftOuter | FullOuter, BuildRight) => - matchedRows += joinedRow(streamedRow, rightNulls).copy() - case (false, RightOuter | FullOuter, BuildLeft) => - matchedRows += joinedRow(leftNulls, streamedRow).copy() - case _ => - } - } - Iterator((matchedRows, includedBroadcastTuples)) - } - - val includedBroadcastTuples = matchesOrStreamedRowsWithNulls.map(_._2) - val allIncludedBroadcastTuples = - if (includedBroadcastTuples.count == 0) { - new scala.collection.mutable.BitSet(broadcastedRelation.value.size) - } else { - includedBroadcastTuples.reduce(_ ++ _) - } - - val leftNulls = new GenericMutableRow(left.output.size) - val rightNulls = new GenericMutableRow(right.output.size) - /** Rows from broadcasted joined with nulls. */ - val broadcastRowsWithNulls: Seq[Row] = { - val buf: CompactBuffer[Row] = new CompactBuffer() - var i = 0 - val rel = broadcastedRelation.value - while (i < rel.length) { - if (!allIncludedBroadcastTuples.contains(i)) { - (joinType, buildSide) match { - case (RightOuter | FullOuter, BuildRight) => buf += new JoinedRow(leftNulls, rel(i)) - case (LeftOuter | FullOuter, BuildLeft) => buf += new JoinedRow(rel(i), rightNulls) - case _ => - } - } - i += 1 - } - buf.toSeq - } - - // TODO: Breaks lineage. - sparkContext.union( - matchesOrStreamedRowsWithNulls.flatMap(_._1), sparkContext.makeRDD(broadcastRowsWithNulls)) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala new file mode 100644 index 0000000000000..d88ab6367a1b3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import scala.concurrent._ +import scala.concurrent.duration._ +import scala.concurrent.ExecutionContext.Implicits.global + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnspecifiedDistribution} +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} + +/** + * :: DeveloperApi :: + * Performs an inner hash join of two child relations. When the output RDD of this operator is + * being constructed, a Spark job is asynchronously started to calculate the values for the + * broadcasted relation. This data is then placed in a Spark broadcast variable. The streamed + * relation is not shuffled. + */ +@DeveloperApi +case class BroadcastHashJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + buildSide: BuildSide, + left: SparkPlan, + right: SparkPlan) + extends BinaryNode with HashJoin { + + override def outputPartitioning: Partitioning = streamedPlan.outputPartitioning + + override def requiredChildDistribution = + UnspecifiedDistribution :: UnspecifiedDistribution :: Nil + + @transient + private val broadcastFuture = future { + sparkContext.broadcast(buildPlan.executeCollect()) + } + + override def execute() = { + val broadcastRelation = Await.result(broadcastFuture, 5.minute) + + streamedPlan.execute().mapPartitions { streamedIter => + joinIterators(broadcastRelation.value.iterator, streamedIter) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala new file mode 100644 index 0000000000000..36aad13778bd2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala @@ -0,0 +1,144 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class BroadcastNestedLoopJoin( + left: SparkPlan, + right: SparkPlan, + buildSide: BuildSide, + joinType: JoinType, + condition: Option[Expression]) extends BinaryNode { + // TODO: Override requiredChildDistribution. + + /** BuildRight means the right relation <=> the broadcast relation. */ + private val (streamed, broadcast) = buildSide match { + case BuildRight => (left, right) + case BuildLeft => (right, left) + } + + override def outputPartitioning: Partitioning = streamed.outputPartitioning + + override def output = { + joinType match { + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case FullOuter => + left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case _ => + left.output ++ right.output + } + } + + @transient private lazy val boundCondition = + InterpretedPredicate( + condition + .map(c => BindReferences.bindReference(c, left.output ++ right.output)) + .getOrElse(Literal(true))) + + override def execute() = { + val broadcastedRelation = + sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + + /** All rows that either match both-way, or rows from streamed joined with nulls. */ + val matchesOrStreamedRowsWithNulls = streamed.execute().mapPartitions { streamedIter => + val matchedRows = new CompactBuffer[Row] + // TODO: Use Spark's BitSet. + val includedBroadcastTuples = + new scala.collection.mutable.BitSet(broadcastedRelation.value.size) + val joinedRow = new JoinedRow + val leftNulls = new GenericMutableRow(left.output.size) + val rightNulls = new GenericMutableRow(right.output.size) + + streamedIter.foreach { streamedRow => + var i = 0 + var streamRowMatched = false + + while (i < broadcastedRelation.value.size) { + // TODO: One bitset per partition instead of per row. + val broadcastedRow = broadcastedRelation.value(i) + buildSide match { + case BuildRight if boundCondition(joinedRow(streamedRow, broadcastedRow)) => + matchedRows += joinedRow(streamedRow, broadcastedRow).copy() + streamRowMatched = true + includedBroadcastTuples += i + case BuildLeft if boundCondition(joinedRow(broadcastedRow, streamedRow)) => + matchedRows += joinedRow(broadcastedRow, streamedRow).copy() + streamRowMatched = true + includedBroadcastTuples += i + case _ => + } + i += 1 + } + + (streamRowMatched, joinType, buildSide) match { + case (false, LeftOuter | FullOuter, BuildRight) => + matchedRows += joinedRow(streamedRow, rightNulls).copy() + case (false, RightOuter | FullOuter, BuildLeft) => + matchedRows += joinedRow(leftNulls, streamedRow).copy() + case _ => + } + } + Iterator((matchedRows, includedBroadcastTuples)) + } + + val includedBroadcastTuples = matchesOrStreamedRowsWithNulls.map(_._2) + val allIncludedBroadcastTuples = + if (includedBroadcastTuples.count == 0) { + new scala.collection.mutable.BitSet(broadcastedRelation.value.size) + } else { + includedBroadcastTuples.reduce(_ ++ _) + } + + val leftNulls = new GenericMutableRow(left.output.size) + val rightNulls = new GenericMutableRow(right.output.size) + /** Rows from broadcasted joined with nulls. */ + val broadcastRowsWithNulls: Seq[Row] = { + val buf: CompactBuffer[Row] = new CompactBuffer() + var i = 0 + val rel = broadcastedRelation.value + while (i < rel.length) { + if (!allIncludedBroadcastTuples.contains(i)) { + (joinType, buildSide) match { + case (RightOuter | FullOuter, BuildRight) => buf += new JoinedRow(leftNulls, rel(i)) + case (LeftOuter | FullOuter, BuildLeft) => buf += new JoinedRow(rel(i), rightNulls) + case _ => + } + } + i += 1 + } + buf.toSeq + } + + // TODO: Breaks lineage. + sparkContext.union( + matchesOrStreamedRowsWithNulls.flatMap(_._1), sparkContext.makeRDD(broadcastRowsWithNulls)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.scala new file mode 100644 index 0000000000000..76c14c02aab34 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/CartesianProduct.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.sql.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { + override def output = left.output ++ right.output + + override def execute() = { + val leftResults = left.execute().map(_.copy()) + val rightResults = right.execute().map(_.copy()) + + leftResults.cartesian(rightResults).mapPartitions { iter => + val joinedRow = new JoinedRow + iter.map(r => joinedRow(r._1, r._2)) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala new file mode 100644 index 0000000000000..472b2e6ca6b4a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.sql.catalyst.expressions.{Expression, JoinedRow2, Row} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.util.collection.CompactBuffer + + +trait HashJoin { + self: SparkPlan => + + val leftKeys: Seq[Expression] + val rightKeys: Seq[Expression] + val buildSide: BuildSide + val left: SparkPlan + val right: SparkPlan + + protected lazy val (buildPlan, streamedPlan) = buildSide match { + case BuildLeft => (left, right) + case BuildRight => (right, left) + } + + protected lazy val (buildKeys, streamedKeys) = buildSide match { + case BuildLeft => (leftKeys, rightKeys) + case BuildRight => (rightKeys, leftKeys) + } + + override def output = left.output ++ right.output + + @transient protected lazy val buildSideKeyGenerator = newProjection(buildKeys, buildPlan.output) + @transient protected lazy val streamSideKeyGenerator = + newMutableProjection(streamedKeys, streamedPlan.output) + + protected def joinIterators(buildIter: Iterator[Row], streamIter: Iterator[Row]): Iterator[Row] = + { + // TODO: Use Spark's HashMap implementation. + + val hashTable = new java.util.HashMap[Row, CompactBuffer[Row]]() + var currentRow: Row = null + + // Create a mapping of buildKeys -> rows + while (buildIter.hasNext) { + currentRow = buildIter.next() + val rowKey = buildSideKeyGenerator(currentRow) + if (!rowKey.anyNull) { + val existingMatchList = hashTable.get(rowKey) + val matchList = if (existingMatchList == null) { + val newMatchList = new CompactBuffer[Row]() + hashTable.put(rowKey, newMatchList) + newMatchList + } else { + existingMatchList + } + matchList += currentRow.copy() + } + } + + new Iterator[Row] { + private[this] var currentStreamedRow: Row = _ + private[this] var currentHashMatches: CompactBuffer[Row] = _ + private[this] var currentMatchPosition: Int = -1 + + // Mutable per row objects. + private[this] val joinRow = new JoinedRow2 + + private[this] val joinKeys = streamSideKeyGenerator() + + override final def hasNext: Boolean = + (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || + (streamIter.hasNext && fetchNext()) + + override final def next() = { + val ret = buildSide match { + case BuildRight => joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) + case BuildLeft => joinRow(currentHashMatches(currentMatchPosition), currentStreamedRow) + } + currentMatchPosition += 1 + ret + } + + /** + * Searches the streamed iterator for the next row that has at least one match in hashtable. + * + * @return true if the search is successful, and false if the streamed iterator runs out of + * tuples. + */ + private final def fetchNext(): Boolean = { + currentHashMatches = null + currentMatchPosition = -1 + + while (currentHashMatches == null && streamIter.hasNext) { + currentStreamedRow = streamIter.next() + if (!joinKeys(currentStreamedRow).anyNull) { + currentHashMatches = hashTable.get(joinKeys.currentValue) + } + } + + if (currentHashMatches == null) { + false + } else { + currentMatchPosition = 0 + true + } + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala new file mode 100644 index 0000000000000..b73041d306b36 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import java.util.{HashMap => JavaHashMap} + +import scala.collection.JavaConversions._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * Performs a hash based outer join for two child relations by shuffling the data using + * the join keys. This operator requires loading the associated partition in both side into memory. + */ +@DeveloperApi +case class HashOuterJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def outputPartitioning: Partitioning = joinType match { + case LeftOuter => left.outputPartitioning + case RightOuter => right.outputPartitioning + case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) + case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } + + override def requiredChildDistribution = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + override def output = { + joinType match { + case LeftOuter => + left.output ++ right.output.map(_.withNullability(true)) + case RightOuter => + left.output.map(_.withNullability(true)) ++ right.output + case FullOuter => + left.output.map(_.withNullability(true)) ++ right.output.map(_.withNullability(true)) + case x => + throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } + } + + @transient private[this] lazy val DUMMY_LIST = Seq[Row](null) + @transient private[this] lazy val EMPTY_LIST = Seq.empty[Row] + + // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala + // iterator for performance purpose. + + private[this] def leftOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val rightNullRow = new GenericRow(right.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + leftIter.iterator.flatMap { l => + joinedRow.withLeft(l) + var matched = false + (if (!key.anyNull) rightIter.collect { case r if (boundCondition(joinedRow.withRight(r))) => + matched = true + joinedRow.copy + } else { + Nil + }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all of the + // records in right side. + // If we didn't get any proper row, then append a single row with empty right + joinedRow.withRight(rightNullRow).copy + }) + } + } + + private[this] def rightOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val leftNullRow = new GenericRow(left.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + rightIter.iterator.flatMap { r => + joinedRow.withRight(r) + var matched = false + (if (!key.anyNull) leftIter.collect { case l if (boundCondition(joinedRow.withLeft(l))) => + matched = true + joinedRow.copy + } else { + Nil + }) ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all of the + // records in left side. + // If we didn't get any proper row, then append a single row with empty left. + joinedRow.withLeft(leftNullRow).copy + }) + } + } + + private[this] def fullOuterIterator( + key: Row, leftIter: Iterable[Row], rightIter: Iterable[Row]): Iterator[Row] = { + val joinedRow = new JoinedRow() + val leftNullRow = new GenericRow(left.output.length) + val rightNullRow = new GenericRow(right.output.length) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + + if (!key.anyNull) { + // Store the positions of records in right, if one of its associated row satisfy + // the join condition. + val rightMatchedSet = scala.collection.mutable.Set[Int]() + leftIter.iterator.flatMap[Row] { l => + joinedRow.withLeft(l) + var matched = false + rightIter.zipWithIndex.collect { + // 1. For those matched (satisfy the join condition) records with both sides filled, + // append them directly + + case (r, idx) if (boundCondition(joinedRow.withRight(r)))=> { + matched = true + // if the row satisfy the join condition, add its index into the matched set + rightMatchedSet.add(idx) + joinedRow.copy + } + } ++ DUMMY_LIST.filter(_ => !matched).map( _ => { + // 2. For those unmatched records in left, append additional records with empty right. + + // DUMMY_LIST.filter(_ => !matched) is a tricky way to add additional row, + // as we don't know whether we need to append it until finish iterating all + // of the records in right side. + // If we didn't get any proper row, then append a single row with empty right. + joinedRow.withRight(rightNullRow).copy + }) + } ++ rightIter.zipWithIndex.collect { + // 3. For those unmatched records in right, append additional records with empty left. + + // Re-visiting the records in right, and append additional row with empty left, if its not + // in the matched set. + case (r, idx) if (!rightMatchedSet.contains(idx)) => { + joinedRow(leftNullRow, r).copy + } + } + } else { + leftIter.iterator.map[Row] { l => + joinedRow(l, rightNullRow).copy + } ++ rightIter.iterator.map[Row] { r => + joinedRow(leftNullRow, r).copy + } + } + } + + private[this] def buildHashTable( + iter: Iterator[Row], keyGenerator: Projection): JavaHashMap[Row, CompactBuffer[Row]] = { + val hashTable = new JavaHashMap[Row, CompactBuffer[Row]]() + while (iter.hasNext) { + val currentRow = iter.next() + val rowKey = keyGenerator(currentRow) + + var existingMatchList = hashTable.get(rowKey) + if (existingMatchList == null) { + existingMatchList = new CompactBuffer[Row]() + hashTable.put(rowKey, existingMatchList) + } + + existingMatchList += currentRow.copy() + } + + hashTable + } + + override def execute() = { + left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => + // TODO this probably can be replaced by external sort (sort merged join?) + // Build HashMap for current partition in left relation + val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) + // Build HashMap for current partition in right relation + val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + val boundCondition = + condition.map(newPredicate(_, left.output ++ right.output)).getOrElse((row: Row) => true) + joinType match { + case LeftOuter => leftHashTable.keysIterator.flatMap { key => + leftOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) + } + case RightOuter => rightHashTable.keysIterator.flatMap { key => + rightOuterIterator(key, leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) + } + case FullOuter => (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => + fullOuterIterator(key, + leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST)) + } + case x => throw new Exception(s"HashOuterJoin should not take $x as the JoinType") + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala new file mode 100644 index 0000000000000..60003d1900d85 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinBNL.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} + +/** + * :: DeveloperApi :: + * Using BroadcastNestedLoopJoin to calculate left semi join result when there's no join keys + * for hash join. + */ +@DeveloperApi +case class LeftSemiJoinBNL( + streamed: SparkPlan, broadcast: SparkPlan, condition: Option[Expression]) + extends BinaryNode { + // TODO: Override requiredChildDistribution. + + override def outputPartitioning: Partitioning = streamed.outputPartitioning + + override def output = left.output + + /** The Streamed Relation */ + override def left = streamed + /** The Broadcast relation */ + override def right = broadcast + + @transient private lazy val boundCondition = + InterpretedPredicate( + condition + .map(c => BindReferences.bindReference(c, left.output ++ right.output)) + .getOrElse(Literal(true))) + + override def execute() = { + val broadcastedRelation = + sparkContext.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq) + + streamed.execute().mapPartitions { streamedIter => + val joinedRow = new JoinedRow + + streamedIter.filter(streamedRow => { + var i = 0 + var matched = false + + while (i < broadcastedRelation.value.size && !matched) { + val broadcastedRow = broadcastedRelation.value(i) + if (boundCondition(joinedRow(streamedRow, broadcastedRow))) { + matched = true + } + i += 1 + } + matched + }) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala new file mode 100644 index 0000000000000..ea7babf3be948 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/LeftSemiJoinHash.scala @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.{Expression, Row} +import org.apache.spark.sql.catalyst.plans.physical.ClusteredDistribution +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} + +/** + * :: DeveloperApi :: + * Build the right table's join keys into a HashSet, and iteratively go through the left + * table, to find the if join keys are in the Hash set. + */ +@DeveloperApi +case class LeftSemiJoinHash( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode with HashJoin { + + override val buildSide = BuildRight + + override def requiredChildDistribution = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + override def output = left.output + + override def execute() = { + buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => + val hashSet = new java.util.HashSet[Row]() + var currentRow: Row = null + + // Create a Hash set of buildKeys + while (buildIter.hasNext) { + currentRow = buildIter.next() + val rowKey = buildSideKeyGenerator(currentRow) + if (!rowKey.anyNull) { + val keyExists = hashSet.contains(rowKey) + if (!keyExists) { + hashSet.add(rowKey) + } + } + } + + val joinKeys = streamSideKeyGenerator() + streamIter.filter(current => { + !joinKeys(current).anyNull && hashSet.contains(joinKeys.currentValue) + }) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala new file mode 100644 index 0000000000000..8247304c1dc2c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoin.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} + +/** + * :: DeveloperApi :: + * Performs an inner hash join of two child relations by first shuffling the data using the join + * keys. + */ +@DeveloperApi +case class ShuffledHashJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + buildSide: BuildSide, + left: SparkPlan, + right: SparkPlan) + extends BinaryNode with HashJoin { + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + override def execute() = { + buildPlan.execute().zipPartitions(streamedPlan.execute()) { + (buildIter, streamIter) => joinIterators(buildIter, streamIter) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/package.scala new file mode 100644 index 0000000000000..7f2ab1765b28f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/package.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Physical execution operators for join operations. + */ +package object joins { + + @DeveloperApi + sealed abstract class BuildSide + + @DeveloperApi + case object BuildRight extends BuildSide + + @DeveloperApi + case object BuildLeft extends BuildSide + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 6c7697ece8c56..07f4d2946c1b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner, LeftSemi} import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6fb6cb8db0c8f..b9b196ea5a46a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.{ShuffledHashJoin, BroadcastHashJoin} +import org.apache.spark.sql.execution.joins.BroadcastHashJoin import org.apache.spark.sql.test._ import org.scalatest.BeforeAndAfterAll import java.util.TimeZone diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index bfbf431a11913..f14ffca0e4d35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.sql.execution import org.scalatest.FunSuite +import org.apache.spark.sql.{SQLConf, execution} import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.{SQLConf, execution} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index a35c40efdc207..14e791fe0f0ee 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.NativeCommand -import org.apache.spark.sql.execution.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ From f706823b71c763fa8e8ceb9e1bd916d8dca7a639 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 8 Oct 2014 22:25:15 -0700 Subject: [PATCH 0936/1492] Fetch from branch v4 in Spark EC2 script. --- ec2/spark_ec2.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 27f468ea4f395..0d6b82b4944f3 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -583,7 +583,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git -b v3") + ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git -b v4") print "Deploying files to master..." deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, modules) From 9c439d33160ef3b31173381735dfa8cfb7d552ba Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 8 Oct 2014 22:35:14 -0700 Subject: [PATCH 0937/1492] [SPARK-3856][MLLIB] use norm operator after breeze 0.10 upgrade Got warning msg: ~~~ [warn] /Users/meng/src/spark/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala:50: method norm in trait NumericOps is deprecated: Use norm(XXX) instead of XXX.norm [warn] var norm = vector.toBreeze.norm(p) ~~~ dbtsai Author: Xiangrui Meng Closes #2718 from mengxr/SPARK-3856 and squashes the following commits: 4f38169 [Xiangrui Meng] use norm operator --- .../scala/org/apache/spark/mllib/feature/Normalizer.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index 3afb47767281c..4734251127bb4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.feature -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => brzNorm} import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -47,7 +47,7 @@ class Normalizer(p: Double) extends VectorTransformer { * @return normalized vector. If the norm of the input is zero, it will return the input vector. */ override def transform(vector: Vector): Vector = { - var norm = vector.toBreeze.norm(p) + var norm = brzNorm(vector.toBreeze, p) if (norm != 0.0) { // For dense vector, we've to allocate new memory for new output vector. From b9df8af62e8d7b263a668dfb6e9668ab4294ea37 Mon Sep 17 00:00:00 2001 From: Anand Avati Date: Wed, 8 Oct 2014 23:45:17 -0700 Subject: [PATCH 0938/1492] [SPARK-2805] Upgrade to akka 2.3.4 Upgrade to akka 2.3.4 Author: Anand Avati Closes #1685 from avati/SPARK-1812-akka-2.3 and squashes the following commits: 57a2315 [Anand Avati] SPARK-1812: streaming - remove tests which depend on akka.actor.IO 2a551d3 [Anand Avati] SPARK-1812: core - upgrade to akka 2.3.4 --- .../org/apache/spark/deploy/Client.scala | 2 +- .../spark/deploy/client/AppClient.scala | 2 +- .../spark/deploy/worker/WorkerWatcher.scala | 2 +- .../apache/spark/MapOutputTrackerSuite.scala | 4 +- pom.xml | 2 +- .../spark/streaming/InputStreamsSuite.scala | 71 ------------------- 6 files changed, 6 insertions(+), 77 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 065ddda50e65e..f2687ce6b42b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -130,7 +130,7 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") System.exit(-1) - case AssociationErrorEvent(cause, _, remoteAddress, _) => + case AssociationErrorEvent(cause, _, remoteAddress, _, _) => println(s"Error connecting to master ${driverArgs.master} ($remoteAddress), exiting.") println(s"Cause was: $cause") System.exit(-1) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 32790053a6be8..98a93d1fcb2a3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -154,7 +154,7 @@ private[spark] class AppClient( logWarning(s"Connection to $address failed; waiting for master to reconnect...") markDisconnected() - case AssociationErrorEvent(cause, _, address, _) if isPossibleMaster(address) => + case AssociationErrorEvent(cause, _, address, _, _) if isPossibleMaster(address) => logWarning(s"Could not connect to $address: $cause") case StopAppClient => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala index 6d0d0bbe5ecec..63a8ac817b618 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerWatcher.scala @@ -54,7 +54,7 @@ private[spark] class WorkerWatcher(workerUrl: String) case AssociatedEvent(localAddress, remoteAddress, inbound) if isWorker(remoteAddress) => logInfo(s"Successfully connected to $workerUrl") - case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound) + case AssociationErrorEvent(cause, localAddress, remoteAddress, inbound, _) if isWorker(remoteAddress) => // These logs may not be seen if the worker (and associated pipe) has died logError(s"Could not initialize connection to worker $workerUrl. Exiting.") diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 1fef79ad1001f..cbc0bd178d894 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -146,7 +146,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) val actorSystem = ActorSystem("test") val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) val masterActor = actorRef.underlyingActor // Frame size should be ~123B, and no exception should be thrown @@ -164,7 +164,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) val actorSystem = ActorSystem("test") val actorRef = TestActorRef[MapOutputTrackerMasterActor]( - new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + Props(new MapOutputTrackerMasterActor(masterTracker, newConf)))(actorSystem) val masterActor = actorRef.underlyingActor // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. diff --git a/pom.xml b/pom.xml index 7756c89b00cad..3b6d4ecbae2c1 100644 --- a/pom.xml +++ b/pom.xml @@ -118,7 +118,7 @@ 0.18.1 shaded-protobuf org.spark-project.akka - 2.2.3-shaded-protobuf + 2.3.4-spark 1.7.5 1.2.17 1.0.4 diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 952a74fd5f6de..6107fcdc447b6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -18,8 +18,6 @@ package org.apache.spark.streaming import akka.actor.Actor -import akka.actor.IO -import akka.actor.IOManager import akka.actor.Props import akka.util.ByteString @@ -144,59 +142,6 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - // TODO: This test works in IntelliJ but not through SBT - ignore("actor input stream") { - // Start the server - val testServer = new TestServer() - val port = testServer.port - testServer.start() - - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", - // Had to pass the local value of port to prevent from closing over entire scope - StorageLevel.MEMORY_AND_DISK) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - val outputStream = new TestOutputStream(networkStream, outputBuffer) - def output = outputBuffer.flatMap(x => x) - outputStream.register() - ssc.start() - - // Feed data to the server to send to the network receiver - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val input = 1 to 9 - val expectedOutput = input.map(x => x.toString) - Thread.sleep(1000) - for (i <- 0 until input.size) { - testServer.send(input(i).toString) - Thread.sleep(500) - clock.addToTime(batchDuration.milliseconds) - } - Thread.sleep(1000) - logInfo("Stopping server") - testServer.stop() - logInfo("Stopping context") - ssc.stop() - - // Verify whether data received was as expected - logInfo("--------------------------------") - logInfo("output.size = " + outputBuffer.size) - logInfo("output") - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output.size = " + expectedOutput.size) - logInfo("expected output") - expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - // (whether the elements were received one in each interval is not verified) - assert(output.size === expectedOutput.size) - for (i <- 0 until output.size) { - assert(output(i) === expectedOutput(i)) - } - } - - test("multi-thread receiver") { // set up the test receiver val numThreads = 10 @@ -378,22 +323,6 @@ class TestServer(portToBind: Int = 0) extends Logging { def port = serverSocket.getLocalPort } -/** This is an actor for testing actor input stream */ -class TestActor(port: Int) extends Actor with ActorHelper { - - def bytesToString(byteString: ByteString) = byteString.utf8String - - override def preStart(): Unit = { - @deprecated("suppress compile time deprecation warning", "1.0.0") - val unit = IOManager(context.system).connect(new InetSocketAddress(port)) - } - - def receive = { - case IO.Read(socket, bytes) => - store(bytesToString(bytes)) - } -} - /** This is a receiver to test multiple threads inserting data using block generator */ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) extends Receiver[Int](StorageLevel.MEMORY_ONLY_SER) with Logging { From 86b392942daf61fed2ff7490178b128107a0e856 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 9 Oct 2014 00:00:24 -0700 Subject: [PATCH 0939/1492] [SPARK-3844][UI] Truncate appName in WebUI if it is too long Truncate appName in WebUI if it is too long. Author: Xiangrui Meng Closes #2707 from mengxr/truncate-app-name and squashes the following commits: 87834ce [Xiangrui Meng] move scala import below java c7111dc [Xiangrui Meng] truncate appName in WebUI if it is too long --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) 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 be69060fc3bf8..32e6b15bb0999 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -21,6 +21,7 @@ import java.text.SimpleDateFormat import java.util.{Locale, Date} import scala.xml.Node + import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ @@ -169,6 +170,7 @@ private[spark] object UIUtils extends Logging { refreshInterval: Option[Int] = None): Seq[Node] = { val appName = activeTab.appName + val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..." val header = activeTab.headerTabs.map { tab =>
  • {tab.name} @@ -187,7 +189,9 @@ private[spark] object UIUtils extends Logging { - +
  • From 13cab5ba44e2f8d2d2204b3b0d39d7c23a819bdb Mon Sep 17 00:00:00 2001 From: nartz Date: Thu, 9 Oct 2014 00:02:11 -0700 Subject: [PATCH 0940/1492] add spark.driver.memory to config docs It took me a minute to track this down, so I thought it could be useful to have it in the docs. I'm unsure if 512mb is the default for spark.driver.memory? Also - there could be a better value for the 'description' to differentiate it from spark.executor.memory. Author: nartz Author: Nathan Artz Closes #2410 from nartz/docs/add-spark-driver-memory-to-config-docs and squashes the following commits: a2f6c62 [nartz] Update configuration.md 74521b8 [Nathan Artz] add spark.driver.memory to config docs --- docs/configuration.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 1c33855365170..f311f0d2a6206 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -103,6 +103,14 @@ of the most common options to set are: (e.g. 512m, 2g).
    spark.driver.memory512m + Amount of memory to use for the driver process, i.e. where SparkContext is initialized. + (e.g. 512m, 2g). +
    spark.serializer org.apache.spark.serializer.
    JavaSerializer
    {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")}{UIUtils.formatDuration(v.taskTime)}{UIUtils.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} + {Utils.bytesToString(v.inputBytes)} + {Utils.bytesToString(v.shuffleRead)} + {Utils.bytesToString(v.shuffleWrite)} + {Utils.bytesToString(v.memoryBytesSpilled)} + {Utils.bytesToString(v.diskBytesSpilled)}
    {inputReadWithUnit}{shuffleReadWithUnit}{shuffleWriteWithUnit}{inputReadWithUnit}{shuffleReadWithUnit}{shuffleWriteWithUnit} {rdd.numCachedPartitions} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)}{Utils.bytesToString(rdd.memSize)}{Utils.bytesToString(rdd.tachyonSize)}{Utils.bytesToString(rdd.diskSize)}{Utils.bytesToString(rdd.memSize)}{Utils.bytesToString(rdd.tachyonSize)}{Utils.bytesToString(rdd.diskSize)}
    spark.history.fs.logDirectory(none) + Directory that contains application event logs to be loaded by the history server +
    spark.history.fs.updateInterval 10
    spark.akka.heartbeat.pauses6006000 This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart beat pause From be2ec4a91d14f48e6323989fb0e0226a9d65bf7e Mon Sep 17 00:00:00 2001 From: Kun Li Date: Thu, 16 Oct 2014 19:00:10 -0700 Subject: [PATCH 1004/1492] [SQL]typo in HiveFromSpark Author: Kun Li Closes #2809 from jackylk/patch-1 and squashes the following commits: 46c926b [Kun Li] typo in HiveFromSpark --- .../org/apache/spark/examples/sql/hive/HiveFromSpark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index e26f213e8afa8..0c52ef8ed96ac 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -28,7 +28,7 @@ object HiveFromSpark { val sparkConf = new SparkConf().setAppName("HiveFromSpark") val sc = new SparkContext(sparkConf) - // A local hive context creates an instance of the Hive Metastore in process, storing the + // A local hive context creates an instance of the Hive Metastore in process, storing // the warehouse data in the current directory. This location can be overridden by // specifying a second parameter to the constructor. val hiveContext = new HiveContext(sc) From 642b246beb7879978d31f2e6e97de7e06c74dcb7 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Thu, 16 Oct 2014 19:07:37 -0700 Subject: [PATCH 1005/1492] [SPARK-3941][CORE] _remainingmem should not increase twice when updateBlockInfo In BlockManagermasterActor, _remainingMem would increase memSize for twice when updateBlockInfo if new storageLevel is invalid and old storageLevel is "useMemory". Also, _remainingMem should increase with original memory size instead of new memSize. Author: Zhang, Liye Closes #2792 from liyezhang556520/spark-3941-remainMem and squashes the following commits: 3d487cc [Zhang, Liye] make the code concise 0380a32 [Zhang, Liye] [SPARK-3941][CORE] _remainingmem should not increase twice when updateBlockInfo --- .../apache/spark/storage/BlockManagerMasterActor.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 6a06257ed0c08..088f06e389d83 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -457,16 +457,18 @@ private[spark] class BlockManagerInfo( if (_blocks.containsKey(blockId)) { // The block exists on the slave already. - val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel + val blockStatus: BlockStatus = _blocks.get(blockId) + val originalLevel: StorageLevel = blockStatus.storageLevel + val originalMemSize: Long = blockStatus.memSize if (originalLevel.useMemory) { - _remainingMem += memSize + _remainingMem += originalMemSize } } if (storageLevel.isValid) { /* isValid means it is either stored in-memory, on-disk or on-Tachyon. - * But the memSize here indicates the data size in or dropped from memory, + * The memSize here indicates the data size in or dropped from memory, * tachyonSize here indicates the data size in or dropped from Tachyon, * and the diskSize here indicates the data size in or dropped to disk. * They can be both larger than 0, when a block is dropped from memory to disk. @@ -493,7 +495,6 @@ private[spark] class BlockManagerInfo( val blockStatus: BlockStatus = _blocks.get(blockId) _blocks.remove(blockId) if (blockStatus.storageLevel.useMemory) { - _remainingMem += blockStatus.memSize logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), Utils.bytesToString(_remainingMem))) From e7f4ea8a52f0d3d56684b4f9caadce978eac4816 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 16 Oct 2014 19:12:39 -0700 Subject: [PATCH 1006/1492] [SPARK-3890][Docs]remove redundant spark.executor.memory in doc Introduced in https://github.com/pwendell/spark/commit/f7e79bc42c1635686c3af01eef147dae92de2529, I'm not sure why we need two spark.executor.memory here. Author: WangTaoTheTonic Author: WangTao Closes #2745 from WangTaoTheTonic/redundantconfig and squashes the following commits: e7564dc [WangTao] too long line fdbdb1f [WangTaoTheTonic] trivial workaround d06b6e5 [WangTaoTheTonic] remove redundant spark.executor.memory in doc --- docs/configuration.md | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8515ee045177f..f0204c640bc89 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -161,14 +161,6 @@ Apart from these, the following properties are also available, and may be useful #### Runtime Environment - - - - - @@ -365,7 +357,7 @@ Apart from these, the following properties are also available, and may be useful @@ -880,8 +872,8 @@ Apart from these, the following properties are also available, and may be useful @@ -893,7 +885,7 @@ Apart from these, the following properties are also available, and may be useful to wait for before scheduling begins. Specified as a double between 0 and 1. Regardless of whether the minimum ratio of resources has been reached, the maximum amount of time it will wait before scheduling begins is controlled by config - spark.scheduler.maxRegisteredResourcesWaitingTime + spark.scheduler.maxRegisteredResourcesWaitingTime. From 56fd34af52a18230bf3ea7b041f2a184eddc1103 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Oct 2014 19:22:02 -0700 Subject: [PATCH 1007/1492] [SPARK-3741] Add afterExecute for handleConnectExecutor Sorry. I found that I forgot to add `afterExecute` for `handleConnectExecutor` in #2593. Author: zsxwing Closes #2794 from zsxwing/SPARK-3741 and squashes the following commits: a0bc4dd [zsxwing] Add afterExecute for handleConnectExecutor --- .../apache/spark/network/nio/ConnectionManager.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 9396b6ba84e7e..bda4bf50932c3 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -117,7 +117,16 @@ private[nio] class ConnectionManager( conf.getInt("spark.core.connection.connect.threads.max", 8), conf.getInt("spark.core.connection.connect.threads.keepalive", 60), TimeUnit.SECONDS, new LinkedBlockingDeque[Runnable](), - Utils.namedThreadFactory("handle-connect-executor")) + Utils.namedThreadFactory("handle-connect-executor")) { + + override def afterExecute(r: Runnable, t: Throwable): Unit = { + super.afterExecute(r, t) + if (t != null && NonFatal(t)) { + logError("Error in handleConnectExecutor is not handled properly", t) + } + } + + } private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation From dedace83f35cba0f833d962acbd75572318948c4 Mon Sep 17 00:00:00 2001 From: yantangzhai Date: Thu, 16 Oct 2014 19:25:37 -0700 Subject: [PATCH 1008/1492] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes JobProgressPage could not show Fair Scheduler Pools section sometimes. SparkContext starts webui and then postEnvironmentUpdate. Sometimes JobProgressPage is accessed between webui starting and postEnvironmentUpdate, then the lazy val isFairScheduler will be false. The Fair Scheduler Pools section will not display any more. Author: yantangzhai Author: YanTangZhai Closes #1966 from YanTangZhai/SPARK-3067 and squashes the following commits: d4323f8 [yantangzhai] update [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes 8a00106 [YanTangZhai] Merge pull request #6 from apache/master b6391cc [yantangzhai] revert [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes d2226cd [yantangzhai] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes cbcba66 [YanTangZhai] Merge pull request #3 from apache/master aac7f7b [yantangzhai] [SPARK-3067] JobProgressPage could not show Fair Scheduler Pools section sometimes cdef539 [YanTangZhai] Merge pull request #1 from apache/master --- core/src/main/scala/org/apache/spark/SparkContext.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b709b8880ba76..354116286c77d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -238,7 +238,6 @@ class SparkContext(config: SparkConf) extends Logging { // For tests, do not enable the UI None } - ui.foreach(_.bind()) /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) @@ -342,6 +341,10 @@ class SparkContext(config: SparkConf) extends Logging { postEnvironmentUpdate() postApplicationStart() + // Bind the SparkUI after starting the task scheduler + // because certain pages and listeners depend on it + ui.foreach(_.bind()) + private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack From e678b9f02a2936b35c95e91a5f0ff388b5720261 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 16 Oct 2014 19:43:33 -0700 Subject: [PATCH 1009/1492] [SPARK-3973] Print call site information for broadcasts Its hard to debug which broadcast variables refer to what in a big codebase. Printing call site information helps in debugging. Author: Shivaram Venkataraman Closes #2829 from shivaram/spark-broadcast-print and squashes the following commits: cd6dbdf [Shivaram Venkataraman] Print call site information for broadcasts --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 354116286c77d..dd3157990ef2d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -818,6 +818,8 @@ class SparkContext(config: SparkConf) extends Logging { */ def broadcast[T: ClassTag](value: T): Broadcast[T] = { val bc = env.broadcastManager.newBroadcast[T](value, isLocal) + val callSite = getCallSite + logInfo("Created broadcast " + bc.id + " from " + callSite.shortForm) cleaner.foreach(_.registerBroadcastForCleanup(bc)) bc } From c351862064ed7d2031ea4c8bf33881e5f702ea0a Mon Sep 17 00:00:00 2001 From: likun Date: Fri, 17 Oct 2014 10:33:45 -0700 Subject: [PATCH 1010/1492] [SPARK-3935][Core] log the number of records that has been written There is a unused variable(count) in saveAsHadoopDataset in PairRDDFunctions.scala. The initial idea of this variable seems to count the number of records, so I am adding a log statement to log the number of records that has been written to the writer. Author: likun Author: jackylk Closes #2791 from jackylk/SPARK-3935 and squashes the following commits: a874047 [jackylk] removing the unused variable in PairRddFunctions.scala 3bf43c7 [likun] log the number of records has been written --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 929ded58a3bd5..ac96de86dd6d4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -1032,10 +1032,8 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) writer.setup(context.stageId, context.partitionId, attemptNumber) writer.open() try { - var count = 0 while (iter.hasNext) { val record = iter.next() - count += 1 writer.write(record._1.asInstanceOf[AnyRef], record._2.asInstanceOf[AnyRef]) } } finally { From 803e7f087797bae643754f8db88848a17282ca6e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 17 Oct 2014 13:45:10 -0500 Subject: [PATCH 1011/1492] [SPARK-3979] [yarn] Use fs's default replication. This avoids issues when HDFS is configured in a way that would not allow the hardcoded default replication of "3". Note: getDefaultReplication(Path) was added in 0.23.3, and the oldest one available on Maven Central is 0.23.7, so I chose to not add code to access that method via reflection. Author: Marcelo Vanzin Closes #2831 from vanzin/SPARK-3979 and squashes the following commits: b0e3a97 [Marcelo Vanzin] [SPARK-3979] [yarn] Use fs's default replication. --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 14a0386b78978..0efac4ea63702 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -143,7 +143,8 @@ private[spark] trait ClientBase extends Logging { val nns = getNameNodesToAccess(sparkConf) + dst obtainTokensForNamenodes(nns, hadoopConf, credentials) - val replication = sparkConf.getInt("spark.yarn.submit.file.replication", 3).toShort + val replication = sparkConf.getInt("spark.yarn.submit.file.replication", + fs.getDefaultReplication(dst)).toShort val localResources = HashMap[String, LocalResource]() FileSystem.mkdirs(fs, dst, new FsPermission(STAGING_DIR_PERMISSION)) From adcb7d3350032dda69a43de724c8bdff5fef2c67 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 17 Oct 2014 14:12:07 -0700 Subject: [PATCH 1012/1492] [SPARK-3855][SQL] Preserve the result attribute of python UDFs though transformations In the current implementation it was possible for the reference to change after analysis. Author: Michael Armbrust Closes #2717 from marmbrus/pythonUdfResults and squashes the following commits: da14879 [Michael Armbrust] Fix test 6343bcb [Michael Armbrust] add test 9533286 [Michael Armbrust] Correctly preserve the result attribute of python UDFs though transformations --- python/pyspark/tests.py | 6 ++++++ .../apache/spark/sql/execution/SparkStrategies.scala | 2 +- .../org/apache/spark/sql/execution/pythonUdfs.scala | 12 ++++++++++-- 3 files changed, 17 insertions(+), 3 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index ceab57464f013..f5ccf31abb3fa 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -683,6 +683,12 @@ def test_udf(self): [row] = self.sqlCtx.sql("SELECT twoArgs('test', 1)").collect() self.assertEqual(row[0], 5) + def test_udf2(self): + self.sqlCtx.registerFunction("strlen", lambda string: len(string)) + self.sqlCtx.inferSchema(self.sc.parallelize([Row(a="test")])).registerTempTable("test") + [res] = self.sqlCtx.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() + self.assertEqual(u"4", res[0]) + def test_broadcast_in_udf(self): bar = {"a": "aa", "b": "bb", "c": "abc"} foo = self.sc.broadcast(bar) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 4f1af7234d551..79e4ddb8c4f5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -295,7 +295,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil - case e @ EvaluatePython(udf, child) => + case e @ EvaluatePython(udf, child, _) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 0977da3e8577c..be729e5d244b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -105,13 +105,21 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { } } +object EvaluatePython { + def apply(udf: PythonUDF, child: LogicalPlan) = + new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) +} + /** * :: DeveloperApi :: * Evaluates a [[PythonUDF]], appending the result to the end of the input tuple. */ @DeveloperApi -case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { - val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() +case class EvaluatePython( + udf: PythonUDF, + child: LogicalPlan, + resultAttribute: AttributeReference) + extends logical.UnaryNode { def output = child.output :+ resultAttribute } From 23f6171d633d4347ca4aa8ec7cb7bd57342b21b5 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 17 Oct 2014 14:49:44 -0700 Subject: [PATCH 1013/1492] [SPARK-3985] [Examples] fix file path using os.path.join Author: Daoyuan Wang Closes #2834 from adrian-wang/sqlpypath and squashes the following commits: da7aa95 [Daoyuan Wang] fix file path using path.join --- examples/src/main/python/sql.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index eefa022f1927c..d2c5ca48c6cb8 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -48,7 +48,7 @@ # A JSON dataset is pointed to by path. # The path can be either a single text file or a directory storing text files. - path = os.environ['SPARK_HOME'] + "examples/src/main/resources/people.json" + path = os.path.join(os.environ['SPARK_HOME'], "examples/src/main/resources/people.json") # Create a SchemaRDD from the file(s) pointed to by path people = sqlContext.jsonFile(path) # root From 477c6481cca94b15c9c8b43e674f220a1cda1dd1 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Fri, 17 Oct 2014 15:02:57 -0700 Subject: [PATCH 1014/1492] [SPARK-3934] [SPARK-3918] [mllib] Bug fixes for RandomForest, DecisionTree SPARK-3934: When run with a mix of unordered categorical and continuous features, on multiclass classification, RandomForest fails. The bug is in the sanity checks in getFeatureOffset and getLeftRightFeatureOffsets, which use the wrong indices for checking whether features are unordered. Fix: Remove the sanity checks since they are not really needed, and since they would require DTStatsAggregator to keep track of an extra set of indices (for the feature subset). Added test to RandomForestSuite which failed with old version but now works. SPARK-3918: Added baggedInput.unpersist at end of training. Also: * I removed DTStatsAggregator.isUnordered since it is no longer used. * DecisionTreeMetadata: Added logWarning when maxBins is automatically reduced. * Updated DecisionTreeRunner to explicitly fix the test data to have the same number of features as the training data. This is a temporary fix which should eventually be replaced by pre-indexing both datasets. * RandomForestModel: Updated toString to print total number of nodes in forest. * Changed Predict class to be public DeveloperApi. This was necessary to allow users to create their own trees by hand (for testing). CC: mengxr manishamde chouqin codedeft Just notifying you of these small bug fixes. Author: Joseph K. Bradley Closes #2785 from jkbradley/dtrunner-update and squashes the following commits: 9132321 [Joseph K. Bradley] merged with master, fixed imports 9dbd000 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update e116473 [Joseph K. Bradley] Changed Predict class to be public DeveloperApi. f502e65 [Joseph K. Bradley] bug fix for SPARK-3934 7f3d60f [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into dtrunner-update ba567ab [Joseph K. Bradley] Changed DTRunner to load test data using same number of features as in training data. 4e88c1f [Joseph K. Bradley] changed RF toString to print total number of nodes --- .../examples/mllib/DecisionTreeRunner.scala | 3 ++- .../mllib/tree/impl/DTStatsAggregator.scala | 16 +--------------- .../mllib/tree/impl/DecisionTreeMetadata.scala | 7 ++++++- .../apache/spark/mllib/tree/model/Predict.scala | 5 ++++- .../mllib/tree/model/RandomForestModel.scala | 4 ++-- .../spark/mllib/tree/RandomForestSuite.scala | 16 ++++++++++++++++ 6 files changed, 31 insertions(+), 20 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 837d0591478c5..0890e6263e165 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -189,9 +189,10 @@ object DecisionTreeRunner { // Create training, test sets. val splits = if (params.testInput != "") { // Load testInput. + val numFeatures = examples.take(1)(0).features.size val origTestExamples = params.dataFormat match { case "dense" => MLUtils.loadLabeledPoints(sc, params.testInput) - case "libsvm" => MLUtils.loadLibSVMFile(sc, params.testInput) + case "libsvm" => MLUtils.loadLibSVMFile(sc, params.testInput, numFeatures) } params.algo match { case Classification => { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala index 55f422dff0d71..ce8825cc03229 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DTStatsAggregator.scala @@ -64,12 +64,6 @@ private[tree] class DTStatsAggregator( numBins.scanLeft(0)((total, nBins) => total + statsSize * nBins) } - /** - * Indicator for each feature of whether that feature is an unordered feature. - * TODO: Is Array[Boolean] any faster? - */ - def isUnordered(featureIndex: Int): Boolean = metadata.isUnordered(featureIndex) - /** * Total number of elements stored in this aggregator */ @@ -128,21 +122,13 @@ private[tree] class DTStatsAggregator( * Pre-compute feature offset for use with [[featureUpdate]]. * For ordered features only. */ - def getFeatureOffset(featureIndex: Int): Int = { - require(!isUnordered(featureIndex), - s"DTStatsAggregator.getFeatureOffset is for ordered features only, but was called" + - s" for unordered feature $featureIndex.") - featureOffsets(featureIndex) - } + def getFeatureOffset(featureIndex: Int): Int = featureOffsets(featureIndex) /** * Pre-compute feature offset for use with [[featureUpdate]]. * For unordered features only. */ def getLeftRightFeatureOffsets(featureIndex: Int): (Int, Int) = { - require(isUnordered(featureIndex), - s"DTStatsAggregator.getLeftRightFeatureOffsets is for unordered features only," + - s" but was called for ordered feature $featureIndex.") val baseOffset = featureOffsets(featureIndex) (baseOffset, baseOffset + (numBins(featureIndex) >> 1) * statsSize) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index 212dce25236e0..772c02670e541 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree.impl import scala.collection.mutable +import org.apache.spark.Logging import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ @@ -82,7 +83,7 @@ private[tree] class DecisionTreeMetadata( } -private[tree] object DecisionTreeMetadata { +private[tree] object DecisionTreeMetadata extends Logging { /** * Construct a [[DecisionTreeMetadata]] instance for this dataset and parameters. @@ -103,6 +104,10 @@ private[tree] object DecisionTreeMetadata { } val maxPossibleBins = math.min(strategy.maxBins, numExamples).toInt + if (maxPossibleBins < strategy.maxBins) { + logWarning(s"DecisionTree reducing maxBins from ${strategy.maxBins} to $maxPossibleBins" + + s" (= number of training instances)") + } // We check the number of bins here against maxPossibleBins. // This needs to be checked here instead of in Strategy since maxPossibleBins can be modified diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala index d8476b5cd7bc7..004838ee5ba0e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Predict.scala @@ -17,12 +17,15 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi + /** * Predicted value for a node * @param predict predicted value * @param prob probability of the label (classification only) */ -private[tree] class Predict( +@DeveloperApi +class Predict( val predict: Double, val prob: Double = 0.0) extends Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala index 4d66d6d81caa5..6a22e2abe59bd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala @@ -82,9 +82,9 @@ class RandomForestModel(val trees: Array[DecisionTreeModel], val algo: Algo) ext */ override def toString: String = algo match { case Classification => - s"RandomForestModel classifier with $numTrees trees" + s"RandomForestModel classifier with $numTrees trees and $totalNumNodes total nodes" case Regression => - s"RandomForestModel regressor with $numTrees trees" + s"RandomForestModel regressor with $numTrees trees and $totalNumNodes total nodes" case _ => throw new IllegalArgumentException( s"RandomForestModel given unknown algo parameter: $algo.") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 20d372dc1d3ca..fb44ceb0f57ee 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -173,6 +173,22 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt) } + test("alternating categorical and continuous features with multiclass labels to test indexing") { + val arr = new Array[LabeledPoint](4) + arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0, 3.0, 1.0)) + arr(1) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0, 1.0, 2.0)) + arr(2) = new LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0, 6.0, 3.0)) + arr(3) = new LabeledPoint(2.0, Vectors.dense(0.0, 2.0, 1.0, 3.0, 2.0)) + val categoricalFeaturesInfo = Map(0 -> 3, 2 -> 2, 4 -> 4) + val input = sc.parallelize(arr) + + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, + numClassesForClassification = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) + val model = RandomForest.trainClassifier(input, strategy, numTrees = 2, + featureSubsetStrategy = "sqrt", seed = 12345) + RandomForestSuite.validateClassifier(model, arr, 1.0) + } + } object RandomForestSuite { From f406a8391825d8866110f29a0d656c82cd064520 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 18 Oct 2014 12:33:20 -0700 Subject: [PATCH 1015/1492] SPARK-3926 [CORE] Result of JavaRDD.collectAsMap() is not Serializable Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are Author: Sean Owen Closes #2805 from srowen/SPARK-3926 and squashes the following commits: ecb78ee [Sean Owen] Fix conflict between java.io.Serializable and use of Scala's Serializable f4717f9 [Sean Owen] Oops, fix compile problem ae1b36f [Sean Owen] Expand to cover Maps returned from other Java API methods as well 51c26c2 [Sean Owen] Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are --- .../org/apache/spark/api/java/JavaPairRDD.scala | 12 +++++++----- .../org/apache/spark/api/java/JavaRDDLike.scala | 7 ++++--- .../scala/org/apache/spark/api/java/JavaUtils.scala | 10 ++++++++++ .../scala/org/apache/spark/sql/api/java/Row.scala | 3 ++- 4 files changed, 23 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 0846225e4f992..c38b96528d037 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -35,6 +35,7 @@ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} @@ -265,10 +266,10 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * before sending results to a reducer, similarly to a "combiner" in MapReduce. */ def reduceByKeyLocally(func: JFunction2[V, V, V]): java.util.Map[K, V] = - mapAsJavaMap(rdd.reduceByKeyLocally(func)) + mapAsSerializableJavaMap(rdd.reduceByKeyLocally(func)) /** Count the number of elements for each key, and return the result to the master as a Map. */ - def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) + def countByKey(): java.util.Map[K, Long] = mapAsSerializableJavaMap(rdd.countByKey()) /** * :: Experimental :: @@ -277,7 +278,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) */ @Experimental def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = - rdd.countByKeyApprox(timeout).map(mapAsJavaMap) + rdd.countByKeyApprox(timeout).map(mapAsSerializableJavaMap) /** * :: Experimental :: @@ -287,7 +288,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[java.util.Map[K, BoundedDouble]] = - rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) + rdd.countByKeyApprox(timeout, confidence).map(mapAsSerializableJavaMap) /** * Aggregate the values of each key, using given combine functions and a neutral "zero value". @@ -614,7 +615,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return the key-value pairs in this RDD to the master as a Map. */ - def collectAsMap(): java.util.Map[K, V] = mapAsJavaMap(rdd.collectAsMap()) + def collectAsMap(): java.util.Map[K, V] = mapAsSerializableJavaMap(rdd.collectAsMap()) + /** * Pass each value in the key-value pair RDD through a map function without changing the keys; diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 545bc0e9e99ed..c744399483349 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -30,6 +30,7 @@ import org.apache.spark.{FutureAction, Partition, SparkContext, TaskContext} import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD @@ -390,7 +391,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * combine step happens locally on the master, equivalent to running a single reduce task. */ def countByValue(): java.util.Map[T, java.lang.Long] = - mapAsJavaMap(rdd.countByValue().map((x => (x._1, new java.lang.Long(x._2))))) + mapAsSerializableJavaMap(rdd.countByValue().map((x => (x._1, new java.lang.Long(x._2))))) /** * (Experimental) Approximate version of countByValue(). @@ -399,13 +400,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { timeout: Long, confidence: Double ): PartialResult[java.util.Map[T, BoundedDouble]] = - rdd.countByValueApprox(timeout, confidence).map(mapAsJavaMap) + rdd.countByValueApprox(timeout, confidence).map(mapAsSerializableJavaMap) /** * (Experimental) Approximate version of countByValue(). */ def countByValueApprox(timeout: Long): PartialResult[java.util.Map[T, BoundedDouble]] = - rdd.countByValueApprox(timeout).map(mapAsJavaMap) + rdd.countByValueApprox(timeout).map(mapAsSerializableJavaMap) /** * Take the first num elements of the RDD. This currently scans the partitions *one by one*, so diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index 22810cb1c662d..b52d0a5028e84 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,10 +19,20 @@ package org.apache.spark.api.java import com.google.common.base.Optional +import scala.collection.convert.Wrappers.MapWrapper + private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) case None => Optional.absent() } + + // Workaround for SPARK-3926 / SI-8911 + def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) = + new SerializableMapWrapper(underlying) + + class SerializableMapWrapper[A, B](underlying: collection.Map[A, B]) + extends MapWrapper(underlying) with java.io.Serializable + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index e9d04ce7aae4c..df01411f60a05 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -22,6 +22,7 @@ import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} import scala.collection.JavaConversions import scala.math.BigDecimal +import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} /** @@ -114,7 +115,7 @@ object Row { // they are actually accessed. case row: ScalaRow => new Row(row) case map: scala.collection.Map[_, _] => - JavaConversions.mapAsJavaMap( + mapAsSerializableJavaMap( map.map { case (key, value) => (toJavaValue(key), toJavaValue(value)) } From 05db2da7dc256822cdb602c4821cbb9fb84dac98 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 18 Oct 2014 19:14:48 -0700 Subject: [PATCH 1016/1492] [SPARK-3952] [Streaming] [PySpark] add Python examples in Streaming Programming Guide Having Python examples in Streaming Programming Guide. Also add RecoverableNetworkWordCount example. Author: Davies Liu Author: Davies Liu Closes #2808 from davies/pyguide and squashes the following commits: 8d4bec4 [Davies Liu] update readme 26a7e37 [Davies Liu] fix format 3821c4d [Davies Liu] address comments, add missing file 7e4bb8a [Davies Liu] add Python examples in Streaming Programming Guide --- docs/README.md | 3 +- docs/streaming-programming-guide.md | 304 +++++++++++++++++- .../recoverable_network_wordcount.py | 80 +++++ python/docs/pyspark.streaming.rst | 10 + python/pyspark/streaming/dstream.py | 8 +- 5 files changed, 391 insertions(+), 14 deletions(-) create mode 100644 examples/src/main/python/streaming/recoverable_network_wordcount.py create mode 100644 python/docs/pyspark.streaming.rst diff --git a/docs/README.md b/docs/README.md index 0facecdd5f767..d2d58e435d4c4 100644 --- a/docs/README.md +++ b/docs/README.md @@ -25,8 +25,7 @@ installing via the Ruby Gem dependency manager. Since the exact HTML output varies between versions of Jekyll and its dependencies, we list specific versions here in some cases: - $ sudo gem install jekyll -v 1.4.3 - $ sudo gem uninstall kramdown -v 1.4.1 + $ sudo gem install jekyll $ sudo gem install jekyll-redirect-from Execute `jekyll` from the `docs/` directory. Compiling the site with Jekyll will create a directory diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 738309c668387..8bbba88b31978 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -212,6 +212,67 @@ The complete code can be found in the Spark Streaming example [JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
    + +
    +First, we import StreamingContext, which is the main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and batch interval of 1 second. + +{% highlight python %} +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + +# Create a local StreamingContext with two working thread and batch interval of 1 second +sc = SparkContext("local[2]", "NetworkWordCount") +ssc = StreamingContext(sc, 1) +{% endhighlight %} + +Using this context, we can create a DStream that represents streaming data from a TCP +source hostname, e.g. `localhost`, and port, e.g. `9999` + +{% highlight python %} +# Create a DStream that will connect to hostname:port, like localhost:9999 +lines = ssc.socketTextStream("localhost", 9999) +{% endhighlight %} + +This `lines` DStream represents the stream of data that will be received from the data +server. Each record in this DStream is a line of text. Next, we want to split the lines by +space into words. + +{% highlight python %} +# Split each line into words +words = lines.flatMap(lambda line: line.split(" ")) +{% endhighlight %} + +`flatMap` is a one-to-many DStream operation that creates a new DStream by +generating multiple new records from each record in the source DStream. In this case, +each line will be split into multiple words and the stream of words is represented as the +`words` DStream. Next, we want to count these words. + +{% highlight python %} +# Count each word in each batch +pairs = words.map(lambda word: (word, 1)) +wordCounts = pairs.reduceByKey(lambda x, y: x + y) + +# Print the first ten elements of each RDD generated in this DStream to the console +wordCounts.pprint() +{% endhighlight %} + +The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, +1)` pairs, which is then reduced to get the frequency of words in each batch of data. +Finally, `wordCounts.pprint()` will print a few of the counts generated every second. + +Note that when these lines are executed, Spark Streaming only sets up the computation it +will perform when it is started, and no real processing has started yet. To start the processing +after all the transformations have been setup, we finally call + +{% highlight python %} +ssc.start() # Start the computation +ssc.awaitTermination() # Wait for the computation to terminate +{% endhighlight %} + +The complete code can be found in the Spark Streaming example +[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/network_wordcount.py). +
    +
    @@ -236,6 +297,11 @@ $ ./bin/run-example streaming.NetworkWordCount localhost 9999 $ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 {% endhighlight %} +
    +{% highlight bash %} +$ ./bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999 +{% endhighlight %} +
    @@ -259,8 +325,11 @@ hello world
    Property NameDefaultMeaning
    spark.executor.memory512m - Amount of memory to use per executor process, in the same format as JVM memory strings - (e.g. 512m, 2g). -
    spark.executor.extraJavaOptions (none)spark.ui.port 4040 - Port for your application's dashboard, which shows memory and workload data + Port for your application's dashboard, which shows memory and workload data.
    spark.scheduler.revive.interval 1000 - The interval length for the scheduler to revive the worker resource offers to run tasks. - (in milliseconds) + The interval length for the scheduler to revive the worker resource offers to run tasks + (in milliseconds).
    +
    + +
    {% highlight bash %} -# TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount +# TERMINAL 2: RUNNING NetworkWordCount $ ./bin/run-example streaming.NetworkWordCount localhost 9999 ... @@ -271,6 +340,37 @@ Time: 1357008430000 ms (world,1) ... {% endhighlight %} +
    + +
    +{% highlight bash %} +# TERMINAL 2: RUNNING JavaNetworkWordCount + +$ ./bin/run-example streaming.JavaNetworkWordCount localhost 9999 +... +------------------------------------------- +Time: 1357008430000 ms +------------------------------------------- +(hello,1) +(world,1) +... +{% endhighlight %} +
    +
    +{% highlight bash %} +# TERMINAL 2: RUNNING network_wordcount.py + +$ ./bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999 +... +------------------------------------------- +Time: 2014-10-14 15:25:21 +------------------------------------------- +(hello,1) +(world,1) +... +{% endhighlight %} +
    +
    @@ -398,9 +498,34 @@ JavaSparkContext sc = ... //existing JavaSparkContext JavaStreamingContext ssc = new JavaStreamingContext(sc, new Duration(1000)); {% endhighlight %} +
    + +A [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) object can be created from a [SparkContext](api/python/pyspark.html#pyspark.SparkContext) object. + +{% highlight python %} +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + +sc = SparkContext(master, appName) +ssc = StreamingContext(sc, 1) +{% endhighlight %} + +The `appName` parameter is a name for your application to show on the cluster UI. +`master` is a [Spark, Mesos or YARN cluster URL](submitting-applications.html#master-urls), +or a special __"local[\*]"__ string to run in local mode. In practice, when running on a cluster, +you will not want to hardcode `master` in the program, +but rather [launch the application with `spark-submit`](submitting-applications.html) and +receive it there. However, for local testing and unit tests, you can pass "local[\*]" to run Spark Streaming +in-process (detects the number of cores in the local system). + +The batch interval must be set based on the latency requirements of your application +and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) +section for more details. +
    After a context is defined, you have to do the follow steps. + 1. Define the input sources. 1. Setup the streaming computations. 1. Start the receiving and procesing of data using `streamingContext.start()`. @@ -483,6 +608,9 @@ methods for creating DStreams from files and Akka actors as input sources.
    streamingContext.fileStream(dataDirectory);
    +
    + streamingContext.textFileStream(dataDirectory) +
    Spark Streaming will monitor the directory `dataDirectory` and process any files created in that directory (files written in nested directories not supported). Note that @@ -684,13 +812,30 @@ This is applied on a DStream containing words (say, the `pairs` DStream containi JavaPairDStream runningCounts = pairs.updateStateByKey(updateFunction); {% endhighlight %} + +
    + +{% highlight python %} +def updateFunction(newValues, runningCount): + if runningCount is None: + runningCount = 0 + return sum(newValues, runningCount) # add the new values with the previous running count to get the new count +{% endhighlight %} + +This is applied on a DStream containing words (say, the `pairs` DStream containing `(word, +1)` pairs in the [earlier example](#a-quick-example)). + +{% highlight python %} +runningCounts = pairs.updateStateByKey(updateFunction) +{% endhighlight %} +
    The update function will be called for each word, with `newValues` having a sequence of 1's (from the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete Scala code, take a look at the example -[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala). +[stateful_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/python/streaming/stateful_network_wordcount.py). #### Transform Operation {:.no_toc} @@ -732,6 +877,15 @@ JavaPairDStream cleanedDStream = wordCounts.transform( }); {% endhighlight %} + +
    + +{% highlight python %} +spamInfoRDD = sc.pickleFile(...) # RDD containing spam information + +# join data stream with spam information to do data cleaning +cleanedDStream = wordCounts.transform(lambda rdd: rdd.join(spamInfoRDD).filter(...)) +{% endhighlight %}
    @@ -793,6 +947,14 @@ Function2 reduceFunc = new Function2 windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000)); {% endhighlight %} + +
    + +{% highlight python %} +# Reduce last 30 seconds of data, every 10 seconds +windowedWordCounts = pairs.reduceByKeyAndWindow(lambda x, y: x + y, lambda x, y: x - y, 30, 10) +{% endhighlight %} +
    @@ -860,6 +1022,7 @@ see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). For the Java API, see [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html). +For the Python API, see [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) *** @@ -872,9 +1035,12 @@ Currently, the following output operations are defined: - + + This is useful for development and debugging. +
    + PS: called pprint() in Python) + @@ -915,17 +1081,41 @@ For this purpose, a developer may inadvertantly try creating a connection object the Spark driver, but try to use it in a Spark worker to save records in the RDDs. For example (in Scala), +
    +
    + +{% highlight scala %} dstream.foreachRDD(rdd => { val connection = createNewConnection() // executed at the driver rdd.foreach(record => { connection.send(record) // executed at the worker }) }) +{% endhighlight %} + +
    +
    + +{% highlight python %} +def sendRecord(rdd): + connection = createNewConnection() # executed at the driver + rdd.foreach(lambda record: connection.send(record)) + connection.close() + +dstream.foreachRDD(sendRecord) +{% endhighlight %} + +
    +
    - This is incorrect as this requires the connection object to be serialized and sent from the driver to the worker. Such connection objects are rarely transferrable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. + This is incorrect as this requires the connection object to be serialized and sent from the driver to the worker. Such connection objects are rarely transferrable across machines. This error may manifest as serialization errors (connection object not serializable), initialization errors (connection object needs to be initialized at the workers), etc. The correct solution is to create the connection object at the worker. - However, this can lead to another common mistake - creating a new connection for every record. For example, +
    +
    + +{% highlight scala %} dstream.foreachRDD(rdd => { rdd.foreach(record => { val connection = createNewConnection() @@ -933,9 +1123,28 @@ For example (in Scala), connection.close() }) }) +{% endhighlight %} - Typically, creating a connection object has time and resource overheads. Therefore, creating and destroying a connection object for each record can incur unnecessarily high overheads and can significantly reduce the overall throughput of the system. A better solution is to use `rdd.foreachPartition` - create a single connection object and send all the records in a RDD partition using that connection. +
    +
    + +{% highlight python %} +def sendRecord(record): + connection = createNewConnection() + connection.send(record) + connection.close() + +dstream.foreachRDD(lambda rdd: rdd.foreach(sendRecord)) +{% endhighlight %} +
    +
    + + Typically, creating a connection object has time and resource overheads. Therefore, creating and destroying a connection object for each record can incur unnecessarily high overheads and can significantly reduce the overall throughput of the system. A better solution is to use `rdd.foreachPartition` - create a single connection object and send all the records in a RDD partition using that connection. + +
    +
    +{% highlight scala %} dstream.foreachRDD(rdd => { rdd.foreachPartition(partitionOfRecords => { val connection = createNewConnection() @@ -943,13 +1152,31 @@ For example (in Scala), connection.close() }) }) +{% endhighlight %} +
    + +
    +{% highlight python %} +def sendPartition(iter): + connection = createNewConnection() + for record in iter: + connection.send(record) + connection.close() + +dstream.foreachRDD(lambda rdd: rdd.foreachPartition(sendPartition)) +{% endhighlight %} +
    +
    - This amortizes the connection creation overheads over many records. + This amortizes the connection creation overheads over many records. - Finally, this can be further optimized by reusing connection objects across multiple RDDs/batches. One can maintain a static pool of connection objects than can be reused as RDDs of multiple batches are pushed to the external system, thus further reducing the overheads. - + +
    +
    +{% highlight scala %} dstream.foreachRDD(rdd => { rdd.foreachPartition(partitionOfRecords => { // ConnectionPool is a static, lazily initialized pool of connections @@ -958,8 +1185,25 @@ For example (in Scala), ConnectionPool.returnConnection(connection) // return to the pool for future reuse }) }) +{% endhighlight %} +
    - Note that the connections in the pool should be lazily created on demand and timed out if not used for a while. This achieves the most efficient sending of data to external systems. +
    +{% highlight python %} +def sendPartition(iter): + # ConnectionPool is a static, lazily initialized pool of connections + connection = ConnectionPool.getConnection() + for record in iter: + connection.send(record) + # return to the pool for future reuse + ConnectionPool.returnConnection(connection) + +dstream.foreachRDD(lambda rdd: rdd.foreachPartition(sendPartition)) +{% endhighlight %} +
    +
    + +Note that the connections in the pool should be lazily created on demand and timed out if not used for a while. This achieves the most efficient sending of data to external systems. ##### Other points to remember: @@ -1376,6 +1620,44 @@ You can also explicitly create a `JavaStreamingContext` from the checkpoint data the computation by using `new JavaStreamingContext(checkpointDirectory)`. +
    + +This behavior is made simple by using `StreamingContext.getOrCreate`. This is used as follows. + +{% highlight python %} +# Function to create and setup a new StreamingContext +def functionToCreateContext(): + sc = SparkContext(...) # new context + ssc = new StreamingContext(...) + lines = ssc.socketTextStream(...) # create DStreams + ... + ssc.checkpoint(checkpointDirectory) # set checkpoint directory + return ssc + +# Get StreamingContext from checkpoint data or create a new one +context = StreamingContext.getOrCreate(checkpointDirectory, functionToCreateContext) + +# Do additional setup on context that needs to be done, +# irrespective of whether it is being started or restarted +context. ... + +# Start the context +context.start() +context.awaitTermination() +{% endhighlight %} + +If the `checkpointDirectory` exists, then the context will be recreated from the checkpoint data. +If the directory does not exist (i.e., running for the first time), +then the function `functionToCreateContext` will be called to create a new +context and set up the DStreams. See the Python example +[recoverable_network_wordcount.py]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/streaming/recoverable_network_wordcount.py). +This example appends the word counts of network data into a file. + +You can also explicitly create a `StreamingContext` from the checkpoint data and start the + computation by using `StreamingContext.getOrCreate(checkpointDirectory, None)`. + +
    + **Note**: If Spark Streaming and/or the Spark Streaming program is recompiled, @@ -1572,7 +1854,11 @@ package and renamed for better clarity. [TwitterUtils](api/java/index.html?org/apache/spark/streaming/twitter/TwitterUtils.html), [ZeroMQUtils](api/java/index.html?org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and [MQTTUtils](api/java/index.html?org/apache/spark/streaming/mqtt/MQTTUtils.html) + - Python docs + * [StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext) + * [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming) + and [Python] ({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/python/streaming) * [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) and [video](http://youtu.be/g171ndOHgJ0) describing Spark Streaming. diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py new file mode 100644 index 0000000000000..fc6827c82bf9b --- /dev/null +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -0,0 +1,80 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" + Counts words in text encoded with UTF8 received from the network every second. + + Usage: recoverable_network_wordcount.py + and describe the TCP server that Spark Streaming would connect to receive + data. directory to HDFS-compatible file system which checkpoint data + file to which the word counts will be appended + + To run this on your local machine, you need to first run a Netcat server + `$ nc -lk 9999` + + and then run the example + `$ bin/spark-submit examples/src/main/python/streaming/recoverable_network_wordcount.py \ + localhost 9999 ~/checkpoint/ ~/out` + + If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create + a new StreamingContext (will print "Creating new context" to the console). Otherwise, if + checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from + the checkpoint data. +""" + +import os +import sys + +from pyspark import SparkContext +from pyspark.streaming import StreamingContext + + +def createContext(host, port, outputPath): + # If you do not see this printed, that means the StreamingContext has been loaded + # from the new checkpoint + print "Creating new context" + if os.path.exists(outputPath): + os.remove(outputPath) + sc = SparkContext(appName="PythonStreamingRecoverableNetworkWordCount") + ssc = StreamingContext(sc, 1) + + # Create a socket stream on target ip:port and count the + # words in input stream of \n delimited text (eg. generated by 'nc') + lines = ssc.socketTextStream(host, port) + words = lines.flatMap(lambda line: line.split(" ")) + wordCounts = words.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y) + + def echo(time, rdd): + counts = "Counts at time %s %s" % (time, rdd.collect()) + print counts + print "Appending to " + os.path.abspath(outputPath) + with open(outputPath, 'a') as f: + f.write(counts + "\n") + + wordCounts.foreachRDD(echo) + return ssc + +if __name__ == "__main__": + if len(sys.argv) != 5: + print >> sys.stderr, "Usage: recoverable_network_wordcount.py "\ + " " + exit(-1) + host, port, checkpoint, output = sys.argv[1:] + ssc = StreamingContext.getOrCreate(checkpoint, + lambda: createContext(host, int(port), output)) + ssc.start() + ssc.awaitTermination() diff --git a/python/docs/pyspark.streaming.rst b/python/docs/pyspark.streaming.rst new file mode 100644 index 0000000000000..5024d694b668f --- /dev/null +++ b/python/docs/pyspark.streaming.rst @@ -0,0 +1,10 @@ +pyspark.streaming module +================== + +Module contents +--------------- + +.. automodule:: pyspark.streaming + :members: + :undoc-members: + :show-inheritance: diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 5ae5cf07f0137..0826ddc56e844 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -441,9 +441,11 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio if `invReduceFunc` is not None, the reduction is done incrementally using the old window's reduced value : - 1. reduce the new values that entered the window (e.g., adding new counts) - 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) - This is more efficient than `invReduceFunc` is None. + + 1. reduce the new values that entered the window (e.g., adding new counts) + + 2. "inverse reduce" the old values that left the window (e.g., subtracting old counts) + This is more efficient than `invReduceFunc` is None. @param reduceFunc: associative reduce function @param invReduceFunc: inverse reduce function of `reduceFunc` From 7e63bb49c526c3f872619ae14e4b5273f4c535e9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 19 Oct 2014 00:31:06 -0700 Subject: [PATCH 1017/1492] [SPARK-2546] Clone JobConf for each task (branch-1.0 / 1.1 backport) This patch attempts to fix SPARK-2546 in `branch-1.0` and `branch-1.1`. The underlying problem is that thread-safety issues in Hadoop Configuration objects may cause Spark tasks to get stuck in infinite loops. The approach taken here is to clone a new copy of the JobConf for each task rather than sharing a single copy between tasks. Note that there are still Configuration thread-safety issues that may affect the driver, but these seem much less likely to occur in practice and will be more complex to fix (see discussion on the SPARK-2546 ticket). This cloning is guarded by a new configuration option (`spark.hadoop.cloneConf`) and is disabled by default in order to avoid unexpected performance regressions for workloads that are unaffected by the Configuration thread-safety issues. Author: Josh Rosen Closes #2684 from JoshRosen/jobconf-fix-backport and squashes the following commits: f14f259 [Josh Rosen] Add configuration option to control cloning of Hadoop JobConf. b562451 [Josh Rosen] Remove unused jobConfCacheKey field. dd25697 [Josh Rosen] [SPARK-2546] [1.0 / 1.1 backport] Clone JobConf for each task. (cherry picked from commit 2cd40db2b3ab5ddcb323fd05c171dbd9025f9e71) Signed-off-by: Josh Rosen Conflicts: core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala --- .../org/apache/spark/rdd/HadoopRDD.scala | 53 +++++++++++++------ docs/configuration.md | 9 ++++ 2 files changed, 47 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 8010dd90082f8..775141775e06c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -132,27 +132,47 @@ class HadoopRDD[K, V]( // used to build JobTracker ID private val createTime = new Date() + private val shouldCloneJobConf = sc.conf.get("spark.hadoop.cloneConf", "false").toBoolean + // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { val conf: Configuration = broadcastedConf.value.value - if (conf.isInstanceOf[JobConf]) { - // A user-broadcasted JobConf was provided to the HadoopRDD, so always use it. - conf.asInstanceOf[JobConf] - } else if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) { - // getJobConf() has been called previously, so there is already a local cache of the JobConf - // needed by this RDD. - HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf] - } else { - // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the - // local process. The local cache is accessed through HadoopRDD.putCachedMetadata(). - // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects. - // Synchronize to prevent ConcurrentModificationException (Spark-1097, Hadoop-10456). + if (shouldCloneJobConf) { + // Hadoop Configuration objects are not thread-safe, which may lead to various problems if + // one job modifies a configuration while another reads it (SPARK-2546). This problem occurs + // somewhat rarely because most jobs treat the configuration as though it's immutable. One + // solution, implemented here, is to clone the Configuration object. Unfortunately, this + // clone can be very expensive. To avoid unexpected performance regressions for workloads and + // Hadoop versions that do not suffer from these thread-safety issues, this cloning is + // disabled by default. HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { + logDebug("Cloning Hadoop Configuration") val newJobConf = new JobConf(conf) - initLocalJobConfFuncOpt.map(f => f(newJobConf)) - HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) + if (!conf.isInstanceOf[JobConf]) { + initLocalJobConfFuncOpt.map(f => f(newJobConf)) + } newJobConf } + } else { + if (conf.isInstanceOf[JobConf]) { + logDebug("Re-using user-broadcasted JobConf") + conf.asInstanceOf[JobConf] + } else if (HadoopRDD.containsCachedMetadata(jobConfCacheKey)) { + logDebug("Re-using cached JobConf") + HadoopRDD.getCachedMetadata(jobConfCacheKey).asInstanceOf[JobConf] + } else { + // Create a JobConf that will be cached and used across this RDD's getJobConf() calls in the + // local process. The local cache is accessed through HadoopRDD.putCachedMetadata(). + // The caching helps minimize GC, since a JobConf can contain ~10KB of temporary objects. + // Synchronize to prevent ConcurrentModificationException (SPARK-1097, HADOOP-10456). + HadoopRDD.CONFIGURATION_INSTANTIATION_LOCK.synchronized { + logDebug("Creating new JobConf and caching it for later re-use") + val newJobConf = new JobConf(conf) + initLocalJobConfFuncOpt.map(f => f(newJobConf)) + HadoopRDD.putCachedMetadata(jobConfCacheKey, newJobConf) + newJobConf + } + } } } @@ -276,7 +296,10 @@ class HadoopRDD[K, V]( } private[spark] object HadoopRDD extends Logging { - /** Constructing Configuration objects is not threadsafe, use this lock to serialize. */ + /** + * Configuration's constructor is not threadsafe (see SPARK-1097 and HADOOP-10456). + * Therefore, we synchronize on this lock before calling new JobConf() or new Configuration(). + */ val CONFIGURATION_INSTANTIATION_LOCK = new Object() /** diff --git a/docs/configuration.md b/docs/configuration.md index f0204c640bc89..96fa1377ec399 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -619,6 +619,15 @@ Apart from these, the following properties are also available, and may be useful output directories. We recommend that users do not disable this except if trying to achieve compatibility with previous versions of Spark. Simply use Hadoop's FileSystem API to delete output directories by hand. + + + + + From d1966f3a8bafdcef87d10ef9db5976cf89faee4b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 19 Oct 2014 20:02:31 -0700 Subject: [PATCH 1018/1492] [SPARK-3902] [SPARK-3590] Stabilize AsynRDDActions and add Java API This PR adds a Java API for AsyncRDDActions and promotes the API from `Experimental` to stable. Author: Josh Rosen Author: Josh Rosen Closes #2760 from JoshRosen/async-rdd-actions-in-java and squashes the following commits: 0d45fbc [Josh Rosen] Whitespace fix. ad3ae53 [Josh Rosen] Merge remote-tracking branch 'origin/master' into async-rdd-actions-in-java c0153a5 [Josh Rosen] Remove unused variable. e8e2867 [Josh Rosen] Updates based on Marcelo's review feedback 7a1417f [Josh Rosen] Removed unnecessary java.util import. 6f8f6ac [Josh Rosen] Fix import ordering. ff28e49 [Josh Rosen] Add MiMa excludes and fix a scalastyle error. 346e46e [Josh Rosen] [SPARK-3902] Stabilize AsyncRDDActions; add Java API. --- .../spark/api/java/JavaFutureAction.java | 33 +++++++ .../scala/org/apache/spark/FutureAction.scala | 86 ++++++++++++++--- .../apache/spark/api/java/JavaRDDLike.scala | 53 ++++++++--- .../apache/spark/rdd/AsyncRDDActions.scala | 3 - .../java/org/apache/spark/JavaAPISuite.java | 93 ++++++++++++++++++- project/MimaExcludes.scala | 13 ++- 6 files changed, 246 insertions(+), 35 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java diff --git a/core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java b/core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java new file mode 100644 index 0000000000000..0ad189633e427 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/JavaFutureAction.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java; + + +import java.util.List; +import java.util.concurrent.Future; + +public interface JavaFutureAction extends Future { + + /** + * Returns the job IDs run by the underlying async operation. + * + * This returns the current snapshot of the job list. Certain operations may run multiple + * jobs, so multiple calls to this method may return different lists. + */ + List jobIds(); +} diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index e8f761eaa5799..d5c8f9d76c476 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -17,20 +17,21 @@ package org.apache.spark -import scala.concurrent._ -import scala.concurrent.duration.Duration -import scala.util.Try +import java.util.Collections +import java.util.concurrent.TimeUnit -import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaFutureAction import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} +import scala.concurrent._ +import scala.concurrent.duration.Duration +import scala.util.{Failure, Try} + /** - * :: Experimental :: * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ -@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -69,6 +70,11 @@ trait FutureAction[T] extends Future[T] { */ override def isCompleted: Boolean + /** + * Returns whether the action has been cancelled. + */ + def isCancelled: Boolean + /** * The value of this Future. * @@ -96,15 +102,16 @@ trait FutureAction[T] extends Future[T] { /** - * :: Experimental :: * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ -@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { + @volatile private var _cancelled: Boolean = false + override def cancel() { + _cancelled = true jobWaiter.cancel() } @@ -143,6 +150,8 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: } override def isCompleted: Boolean = jobWaiter.jobFinished + + override def isCancelled: Boolean = _cancelled override def value: Option[Try[T]] = { if (jobWaiter.jobFinished) { @@ -164,12 +173,10 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** - * :: Experimental :: * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. */ -@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. @@ -222,7 +229,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { // If the action hasn't been cancelled yet, submit the job. The check and the submitJob // command need to be in an atomic block. val job = this.synchronized { - if (!cancelled) { + if (!isCancelled) { rdd.context.submitJob(rdd, processPartition, partitions, resultHandler, resultFunc) } else { throw new SparkException("Action has been cancelled") @@ -243,10 +250,7 @@ class ComplexFutureAction[T] extends FutureAction[T] { } } - /** - * Returns whether the promise has been cancelled. - */ - def cancelled: Boolean = _cancelled + override def isCancelled: Boolean = _cancelled @throws(classOf[InterruptedException]) @throws(classOf[scala.concurrent.TimeoutException]) @@ -271,3 +275,55 @@ class ComplexFutureAction[T] extends FutureAction[T] { def jobIds = jobs } + +private[spark] +class JavaFutureActionWrapper[S, T](futureAction: FutureAction[S], converter: S => T) + extends JavaFutureAction[T] { + + import scala.collection.JavaConverters._ + + override def isCancelled: Boolean = futureAction.isCancelled + + override def isDone: Boolean = { + // According to java.util.Future's Javadoc, this returns True if the task was completed, + // whether that completion was due to successful execution, an exception, or a cancellation. + futureAction.isCancelled || futureAction.isCompleted + } + + override def jobIds(): java.util.List[java.lang.Integer] = { + Collections.unmodifiableList(futureAction.jobIds.map(Integer.valueOf).asJava) + } + + private def getImpl(timeout: Duration): T = { + // This will throw TimeoutException on timeout: + Await.ready(futureAction, timeout) + futureAction.value.get match { + case scala.util.Success(value) => converter(value) + case Failure(exception) => + if (isCancelled) { + throw new CancellationException("Job cancelled").initCause(exception) + } else { + // java.util.Future.get() wraps exceptions in ExecutionException + throw new ExecutionException("Exception thrown by job", exception) + } + } + } + + override def get(): T = getImpl(Duration.Inf) + + override def get(timeout: Long, unit: TimeUnit): T = + getImpl(Duration.fromNanos(unit.toNanos(timeout))) + + override def cancel(mayInterruptIfRunning: Boolean): Boolean = synchronized { + if (isDone) { + // According to java.util.Future's Javadoc, this should return false if the task is completed. + false + } else { + // We're limited in terms of the semantics we can provide here; our cancellation is + // asynchronous and doesn't provide a mechanism to not cancel if the job is running. + futureAction.cancel() + true + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index c744399483349..efb8978f7ce12 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -21,12 +21,14 @@ import java.util.{Comparator, List => JList, Iterator => JIterator} import java.lang.{Iterable => JIterable, Long => JLong} import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ import scala.reflect.ClassTag import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec -import org.apache.spark.{FutureAction, Partition, SparkContext, TaskContext} +import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag @@ -294,8 +296,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Applies a function f to all elements of this RDD. */ def foreach(f: VoidFunction[T]) { - val cleanF = rdd.context.clean((x: T) => f.call(x)) - rdd.foreach(cleanF) + rdd.foreach(x => f.call(x)) } /** @@ -576,16 +577,44 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def name(): String = rdd.name /** - * :: Experimental :: - * The asynchronous version of the foreach action. - * - * @param f the function to apply to all the elements of the RDD - * @return a FutureAction for the action + * The asynchronous version of `count`, which returns a + * future for counting the number of elements in this RDD. */ - @Experimental - def foreachAsync(f: VoidFunction[T]): FutureAction[Unit] = { - import org.apache.spark.SparkContext._ - rdd.foreachAsync(x => f.call(x)) + def countAsync(): JavaFutureAction[JLong] = { + new JavaFutureActionWrapper[Long, JLong](rdd.countAsync(), JLong.valueOf) + } + + /** + * The asynchronous version of `collect`, which returns a future for + * retrieving an array containing all of the elements in this RDD. + */ + def collectAsync(): JavaFutureAction[JList[T]] = { + new JavaFutureActionWrapper(rdd.collectAsync(), (x: Seq[T]) => x.asJava) + } + + /** + * The asynchronous version of the `take` action, which returns a + * future for retrieving the first `num` elements of this RDD. + */ + def takeAsync(num: Int): JavaFutureAction[JList[T]] = { + new JavaFutureActionWrapper(rdd.takeAsync(num), (x: Seq[T]) => x.asJava) } + /** + * The asynchronous version of the `foreach` action, which + * applies a function f to all the elements of this RDD. + */ + def foreachAsync(f: VoidFunction[T]): JavaFutureAction[Void] = { + new JavaFutureActionWrapper[Unit, Void](rdd.foreachAsync(x => f.call(x)), + { x => null.asInstanceOf[Void] }) + } + + /** + * The asynchronous version of the `foreachPartition` action, which + * applies a function f to each partition of this RDD. + */ + def foreachPartitionAsync(f: VoidFunction[java.util.Iterator[T]]): JavaFutureAction[Void] = { + new JavaFutureActionWrapper[Unit, Void](rdd.foreachPartitionAsync(x => f.call(x)), + { x => null.asInstanceOf[Void] }) + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index ede5568493cc0..9f9f10b7ebc3a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,14 +24,11 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} -import org.apache.spark.annotation.Experimental /** - * :: Experimental :: * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ -@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index b8fa822ae4bd8..3190148fb5f43 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -20,6 +20,7 @@ import java.io.*; import java.net.URI; import java.util.*; +import java.util.concurrent.*; import scala.Tuple2; import scala.Tuple3; @@ -29,6 +30,7 @@ import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.base.Throwables; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -43,10 +45,7 @@ import org.junit.Before; import org.junit.Test; -import org.apache.spark.api.java.JavaDoubleRDD; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.*; import org.apache.spark.api.java.function.*; import org.apache.spark.executor.TaskMetrics; import org.apache.spark.partial.BoundedDouble; @@ -1308,6 +1307,92 @@ public void collectUnderlyingScalaRDD() { Assert.assertEquals(data.size(), collected.length); } + private static final class BuggyMapFunction implements Function { + + @Override + public T call(T x) throws Exception { + throw new IllegalStateException("Custom exception!"); + } + } + + @Test + public void collectAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction> future = rdd.collectAsync(); + List result = future.get(); + Assert.assertEquals(data, result); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + + @Test + public void foreachAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.foreachAsync( + new VoidFunction() { + @Override + public void call(Integer integer) throws Exception { + // intentionally left blank. + } + } + ); + future.get(); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + + @Test + public void countAsync() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.countAsync(); + long count = future.get(); + Assert.assertEquals(data.size(), count); + Assert.assertFalse(future.isCancelled()); + Assert.assertTrue(future.isDone()); + Assert.assertEquals(1, future.jobIds().size()); + } + + @Test + public void testAsyncActionCancellation() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.foreachAsync(new VoidFunction() { + @Override + public void call(Integer integer) throws Exception { + Thread.sleep(10000); // To ensure that the job won't finish before it's cancelled. + } + }); + future.cancel(true); + Assert.assertTrue(future.isCancelled()); + Assert.assertTrue(future.isDone()); + try { + future.get(2000, TimeUnit.MILLISECONDS); + Assert.fail("Expected future.get() for cancelled job to throw CancellationException"); + } catch (CancellationException ignored) { + // pass + } + } + + @Test + public void testAsyncActionErrorWrapping() throws Exception { + List data = Arrays.asList(1, 2, 3, 4, 5); + JavaRDD rdd = sc.parallelize(data, 1); + JavaFutureAction future = rdd.map(new BuggyMapFunction()).countAsync(); + try { + future.get(2, TimeUnit.SECONDS); + Assert.fail("Expected future.get() for failed job to throw ExcecutionException"); + } catch (ExecutionException ee) { + Assert.assertTrue(Throwables.getStackTraceAsString(ee).contains("Custom exception!")); + } + Assert.assertTrue(future.isDone()); + } + + /** * Test for SPARK-3647. This test needs to use the maven-built assembly to trigger the issue, * since that's the only artifact where Guava classes have been relocated. diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 350aad47735e4..c58666af84f24 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -54,7 +54,18 @@ object MimaExcludes { // TaskContext was promoted to Abstract class ProblemFilters.exclude[AbstractClassProblem]( "org.apache.spark.TaskContext") - + ) ++ Seq( + // Adding new methods to the JavaRDDLike trait: + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.takeAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.foreachPartitionAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.countAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.foreachAsync"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.api.java.JavaRDDLike.collectAsync") ) case v if v.startsWith("1.1") => From c7aeecd08fd329085760fa89025ec0d9c04f5e3f Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 20 Oct 2014 10:20:21 -0700 Subject: [PATCH 1019/1492] [SPARK-3948][Shuffle]Fix stream corruption bug in sort-based shuffle Kernel 2.6.32 bug will lead to unexpected behavior of transferTo in copyStream, and this will corrupt the shuffle output file in sort-based shuffle, which will somehow introduce PARSING_ERROR(2), deserialization error or offset out of range. Here fix this by adding append flag, also add some position checking code. Details can be seen in [SPARK-3948](https://issues.apache.org/jira/browse/SPARK-3948). Author: jerryshao Closes #2824 from jerryshao/SPARK-3948 and squashes the following commits: be0533a [jerryshao] Address the comments a82b184 [jerryshao] add configuration to control the NIO way of copying stream e17ada2 [jerryshao] Fix kernel 2.6.32 bug led unexpected behavior of transferTo --- .../scala/org/apache/spark/util/Utils.scala | 29 ++++++++++++++++--- .../util/collection/ExternalSorter.scala | 5 ++-- 2 files changed, 28 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 53a7512edd852..0aeff6455b3fe 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -269,23 +269,44 @@ private[spark] object Utils extends Logging { dir } - /** Copy all data from an InputStream to an OutputStream */ + /** Copy all data from an InputStream to an OutputStream. NIO way of file stream to file stream + * copying is disabled by default unless explicitly set transferToEnabled as true, + * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false]. + */ def copyStream(in: InputStream, out: OutputStream, - closeStreams: Boolean = false): Long = + closeStreams: Boolean = false, + transferToEnabled: Boolean = false): Long = { var count = 0L try { - if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream]) { + if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream] + && transferToEnabled) { // When both streams are File stream, use transferTo to improve copy performance. val inChannel = in.asInstanceOf[FileInputStream].getChannel() val outChannel = out.asInstanceOf[FileOutputStream].getChannel() + val initialPos = outChannel.position() val size = inChannel.size() // In case transferTo method transferred less data than we have required. while (count < size) { count += inChannel.transferTo(count, size - count, outChannel) } + + // Check the position after transferTo loop to see if it is in the right position and + // give user information if not. + // Position will not be increased to the expected length after calling transferTo in + // kernel version 2.6.32, this issue can be seen in + // https://bugs.openjdk.java.net/browse/JDK-7052359 + // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948). + val finalPos = outChannel.position() + assert(finalPos == initialPos + size, + s""" + |Current position $finalPos do not equal to expected position ${initialPos + size} + |after transferTo, please check your kernel version to see if it is 2.6.32, + |this is a kernel bug which will lead to unexpected behavior when using transferTo. + |You can set spark.file.transferTo = false to disable this NIO feature. + """.stripMargin) } else { val buf = new Array[Byte](8192) var n = 0 @@ -727,7 +748,7 @@ private[spark] object Utils extends Logging { /** * Determines if a directory contains any files newer than cutoff seconds. - * + * * @param dir must be the path to a directory, or IllegalArgumentException is thrown * @param cutoff measured in seconds. Returns true if there are any files or directories in the * given directory whose last modified time is later than this many seconds ago diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index 644fa36818647..d1b06d14acbd2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -93,6 +93,7 @@ private[spark] class ExternalSorter[K, V, C]( private val conf = SparkEnv.get.conf private val spillingEnabled = conf.getBoolean("spark.shuffle.spill", true) private val fileBufferSize = conf.getInt("spark.shuffle.file.buffer.kb", 32) * 1024 + private val transferToEnabled = conf.getBoolean("spark.file.transferTo", true) // Size of object batches when reading/writing from serializers. // @@ -705,10 +706,10 @@ private[spark] class ExternalSorter[K, V, C]( var out: FileOutputStream = null var in: FileInputStream = null try { - out = new FileOutputStream(outputFile) + out = new FileOutputStream(outputFile, true) for (i <- 0 until numPartitions) { in = new FileInputStream(partitionWriters(i).fileSegment().file) - val size = org.apache.spark.util.Utils.copyStream(in, out, false) + val size = org.apache.spark.util.Utils.copyStream(in, out, false, transferToEnabled) in.close() in = null lengths(i) = size From 51afde9d8b8a67958c4632a13af143d7c7fd1f04 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Mon, 20 Oct 2014 11:01:26 -0700 Subject: [PATCH 1020/1492] [SPARK-4010][Web UI]Spark UI returns 500 in yarn-client mode The problem caused by #1966 CC YanTangZhai andrewor14 Author: GuoQiang Li Closes #2858 from witgo/SPARK-4010 and squashes the following commits: 9866fbf [GuoQiang Li] Spark UI returns 500 in yarn-client mode --- core/src/main/scala/org/apache/spark/SparkContext.scala | 8 ++++---- .../scala/org/apache/spark/ui/jobs/JobProgressPage.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dd3157990ef2d..ac7935b8c231e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -239,6 +239,10 @@ class SparkContext(config: SparkConf) extends Logging { None } + // Bind the UI before starting the task scheduler to communicate + // the bound port to the cluster manager properly + ui.foreach(_.bind()) + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) @@ -341,10 +345,6 @@ class SparkContext(config: SparkConf) extends Logging { postEnvironmentUpdate() postApplicationStart() - // Bind the SparkUI after starting the task scheduler - // because certain pages and listeners depend on it - ui.foreach(_.bind()) - private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index a82f71ed08475..1e02f1225d344 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -29,7 +29,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") private val live = parent.live private val sc = parent.sc private val listener = parent.listener - private lazy val isFairScheduler = parent.isFairScheduler + private def isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { From ea054e1fc70e09e0babcdae2a37f6f7aa6a035f2 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 20 Oct 2014 11:31:51 -0700 Subject: [PATCH 1021/1492] [SPARK-3986][SQL] Fix package names to fit their directory names. Package names of 2 test suites are different from their directory names. - `GeneratedEvaluationSuite` - `GeneratedMutableEvaluationSuite` Author: Takuya UESHIN Closes #2835 from ueshin/issues/SPARK-3986 and squashes the following commits: fa2cc05 [Takuya UESHIN] Fix package names to fit their directory names. --- .../sql/catalyst/expressions/GeneratedEvaluationSuite.scala | 3 +-- .../catalyst/expressions/GeneratedMutableEvaluationSuite.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala index 245a2e148030c..ef3114fd4dbab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.optimizer +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala index 887aabb1d5fb4..275ea2627ebcd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.optimizer +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ From 4afe9a4852ebeb4cc77322a14225cd3dec165f3f Mon Sep 17 00:00:00 2001 From: mcheah Date: Mon, 20 Oct 2014 11:35:18 -0700 Subject: [PATCH 1022/1492] [SPARK-3736] Workers reconnect when disassociated from the master. Before, if the master node is killed and restarted, the worker nodes would not attempt to reconnect to the Master. Therefore, when the Master node was restarted, the worker nodes needed to be restarted as well. Now, when the Master node is disconnected, the worker nodes will continuously ping the master node in attempts to reconnect to it. Once the master node restarts, it will detect one of the registration requests from its former workers. The result is that the cluster re-enters a healthy state. In addition, when the master does not receive a heartbeat from the worker, the worker was removed; however, when the worker sent a heartbeat to the master, the master used to ignore the heartbeat. Now, a master that receives a heartbeat from a worker that had been disconnected will request the worker to re-attempt the registration process, at which point the worker will send a RegisterWorker request and be re-connected accordingly. Re-connection attempts per worker are submitted every N seconds, where N is configured by the property spark.worker.reconnect.interval - this has a default of 60 seconds right now. Author: mcheah Closes #2828 from mccheah/reconnect-dead-workers and squashes the following commits: 83f8bc9 [mcheah] [SPARK-3736] More informative log message, and fixing some indentation. fe0e02f [mcheah] [SPARK-3736] Moving reconnection logic to registerWithMaster(). 94ddeca [mcheah] [SPARK-3736] Changing a log warning to a log info. a698e35 [mcheah] [SPARK-3736] Addressing PR comment to make some defs private. b9a3077 [mcheah] [SPARK-3736] Addressing PR comments related to reconnection. 2ad5ed5 [mcheah] [SPARK-3736] Cancel attempts to reconnect if the master changes. b5b34af [mcheah] [SPARK-3736] Workers reconnect when disassociated from the master. --- .../apache/spark/deploy/DeployMessage.scala | 2 + .../apache/spark/deploy/master/Master.scala | 9 ++- .../apache/spark/deploy/worker/Worker.scala | 81 ++++++++++++++----- 3 files changed, 72 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index a7368f9f3dfbe..b9dd8557ee904 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -71,6 +71,8 @@ private[deploy] object DeployMessages { case class RegisterWorkerFailed(message: String) extends DeployMessage + case class ReconnectWorker(masterUrl: String) extends DeployMessage + case class KillExecutor(masterUrl: String, appId: String, execId: Int) extends DeployMessage case class LaunchExecutor( diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index f98b531316a3d..3b6bb9fe128a4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -341,7 +341,14 @@ private[spark] class Master( case Some(workerInfo) => workerInfo.lastHeartbeat = System.currentTimeMillis() case None => - logWarning("Got heartbeat from unregistered worker " + workerId) + if (workers.map(_.id).contains(workerId)) { + logWarning(s"Got heartbeat from unregistered worker $workerId." + + " Asking it to re-register.") + sender ! ReconnectWorker(masterUrl) + } else { + logWarning(s"Got heartbeat from unregistered worker $workerId." + + " This worker was never registered, so ignoring the heartbeat.") + } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 9b52cb06fb6fa..c4a8ec2e5e7b0 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -20,12 +20,14 @@ package org.apache.spark.deploy.worker import java.io.File import java.io.IOException import java.text.SimpleDateFormat -import java.util.Date +import java.util.{UUID, Date} +import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap import scala.concurrent.duration._ import scala.language.postfixOps +import scala.util.Random import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} @@ -64,8 +66,22 @@ private[spark] class Worker( // Send a heartbeat every (heartbeat timeout) / 4 milliseconds val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 - val REGISTRATION_TIMEOUT = 20.seconds - val REGISTRATION_RETRIES = 3 + // Model retries to connect to the master, after Hadoop's model. + // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) + // Afterwards, the next 10 attempts are between 30 and 90 seconds. + // A bit of randomness is introduced so that not all of the workers attempt to reconnect at + // the same time. + val INITIAL_REGISTRATION_RETRIES = 6 + val TOTAL_REGISTRATION_RETRIES = INITIAL_REGISTRATION_RETRIES + 10 + val FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND = 0.500 + val REGISTRATION_RETRY_FUZZ_MULTIPLIER = { + val randomNumberGenerator = new Random(UUID.randomUUID.getMostSignificantBits) + randomNumberGenerator.nextDouble + FUZZ_MULTIPLIER_INTERVAL_LOWER_BOUND + } + val INITIAL_REGISTRATION_RETRY_INTERVAL = (math.round(10 * + REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds + val PROLONGED_REGISTRATION_RETRY_INTERVAL = (math.round(60 + * REGISTRATION_RETRY_FUZZ_MULTIPLIER)).seconds val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) // How often worker will clean up old app folders @@ -103,6 +119,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 + var connectionAttemptCount = 0 val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) val workerSource = new WorkerSource(this) @@ -158,7 +175,7 @@ private[spark] class Worker( connected = true } - def tryRegisterAllMasters() { + private def tryRegisterAllMasters() { for (masterUrl <- masterUrls) { logInfo("Connecting to master " + masterUrl + "...") val actor = context.actorSelection(Master.toAkkaUrl(masterUrl)) @@ -166,26 +183,47 @@ private[spark] class Worker( } } - def registerWithMaster() { - tryRegisterAllMasters() - var retries = 0 - registrationRetryTimer = Some { - context.system.scheduler.schedule(REGISTRATION_TIMEOUT, REGISTRATION_TIMEOUT) { - Utils.tryOrExit { - retries += 1 - if (registered) { - registrationRetryTimer.foreach(_.cancel()) - } else if (retries >= REGISTRATION_RETRIES) { - logError("All masters are unresponsive! Giving up.") - System.exit(1) - } else { - tryRegisterAllMasters() + private def retryConnectToMaster() { + Utils.tryOrExit { + connectionAttemptCount += 1 + logInfo(s"Attempting to connect to master (attempt # $connectionAttemptCount") + if (registered) { + registrationRetryTimer.foreach(_.cancel()) + registrationRetryTimer = None + } else if (connectionAttemptCount <= TOTAL_REGISTRATION_RETRIES) { + tryRegisterAllMasters() + if (connectionAttemptCount == INITIAL_REGISTRATION_RETRIES) { + registrationRetryTimer.foreach(_.cancel()) + registrationRetryTimer = Some { + context.system.scheduler.schedule(PROLONGED_REGISTRATION_RETRY_INTERVAL, + PROLONGED_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) } } + } else { + logError("All masters are unresponsive! Giving up.") + System.exit(1) } } } + def registerWithMaster() { + // DisassociatedEvent may be triggered multiple times, so don't attempt registration + // if there are outstanding registration attempts scheduled. + registrationRetryTimer match { + case None => + registered = false + tryRegisterAllMasters() + connectionAttemptCount = 0 + registrationRetryTimer = Some { + context.system.scheduler.schedule(INITIAL_REGISTRATION_RETRY_INTERVAL, + INITIAL_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + } + case Some(_) => + logInfo("Not spawning another attempt to register with the master, since there is an" + + " attempt scheduled already.") + } + } + override def receiveWithLogging = { case RegisteredWorker(masterUrl, masterWebUiUrl) => logInfo("Successfully registered with master " + masterUrl) @@ -243,6 +281,10 @@ private[spark] class Worker( System.exit(1) } + case ReconnectWorker(masterUrl) => + logInfo(s"Master with url $masterUrl requested this worker to reconnect.") + registerWithMaster() + case LaunchExecutor(masterUrl, appId, execId, appDesc, cores_, memory_) => if (masterUrl != activeMasterUrl) { logWarning("Invalid Master (" + masterUrl + ") attempted to launch executor.") @@ -362,9 +404,10 @@ private[spark] class Worker( } } - def masterDisconnected() { + private def masterDisconnected() { logError("Connection to master failed! Waiting for master to reconnect...") connected = false + registerWithMaster() } def generateWorkerId(): String = { From eadc4c590ee43572528da55d84ed65f09153e857 Mon Sep 17 00:00:00 2001 From: Qiping Li Date: Mon, 20 Oct 2014 13:12:26 -0700 Subject: [PATCH 1023/1492] [SPARK-3207][MLLIB]Choose splits for continuous features in DecisionTree more adaptively DecisionTree splits on continuous features by choosing an array of values from a subsample of the data. Currently, it does not check for identical values in the subsample, so it could end up having multiple copies of the same split. In this PR, we choose splits for a continuous feature in 3 steps: 1. Sort sample values for this feature 2. Get number of occurrence of each distinct value 3. Iterate the value count array computed in step 2 to choose splits. After find splits, `numSplits` and `numBins` in metadata will be updated. CC: mengxr manishamde jkbradley, please help me review this, thanks. Author: Qiping Li Author: chouqin Author: liqi Author: qiping.lqp Closes #2780 from chouqin/dt-findsplits and squashes the following commits: 18d0301 [Qiping Li] check explicitly findsplits return distinct splits 8dc28ab [chouqin] remove blank lines ffc920f [chouqin] adjust code based on comments and add more test cases 9857039 [chouqin] Merge branch 'master' of https://github.com/apache/spark into dt-findsplits d353596 [qiping.lqp] fix pyspark doc test 9e64699 [Qiping Li] fix random forest unit test 3c72913 [Qiping Li] fix random forest unit test 092efcb [Qiping Li] fix bug f69f47f [Qiping Li] fix bug ab303a4 [Qiping Li] fix bug af6dc97 [Qiping Li] fix bug 2a8267a [Qiping Li] fix bug c339a61 [Qiping Li] fix bug 369f812 [Qiping Li] fix style 8f46af6 [Qiping Li] add comments and unit test 9e7138e [Qiping Li] Merge branch 'dt-findsplits' of https://github.com/chouqin/spark into dt-findsplits 1b25a35 [Qiping Li] Merge branch 'master' of https://github.com/apache/spark into dt-findsplits 0cd744a [liqi] fix bug 3652823 [Qiping Li] fix bug af7cb79 [Qiping Li] Choose splits for continuous features in DecisionTree more adaptively --- .../spark/mllib/tree/DecisionTree.scala | 104 ++++++++++++++++-- .../tree/impl/DecisionTreeMetadata.scala | 11 ++ .../spark/mllib/tree/DecisionTreeSuite.scala | 68 +++++++++++- .../spark/mllib/tree/RandomForestSuite.scala | 5 +- python/pyspark/mllib/tree.py | 4 +- 5 files changed, 176 insertions(+), 16 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 03eeaa707715b..6737a2f4176c2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -19,6 +19,8 @@ package org.apache.spark.mllib.tree import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD @@ -909,32 +911,39 @@ object DecisionTree extends Serializable with Logging { // Iterate over all features. var featureIndex = 0 while (featureIndex < numFeatures) { - val numSplits = metadata.numSplits(featureIndex) - val numBins = metadata.numBins(featureIndex) if (metadata.isContinuous(featureIndex)) { - val numSamples = sampledInput.length + val featureSamples = sampledInput.map(lp => lp.features(featureIndex)) + val featureSplits = findSplitsForContinuousFeature(featureSamples, + metadata, featureIndex) + + val numSplits = featureSplits.length + val numBins = numSplits + 1 + logDebug(s"featureIndex = $featureIndex, numSplits = $numSplits") splits(featureIndex) = new Array[Split](numSplits) bins(featureIndex) = new Array[Bin](numBins) - val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted - val stride: Double = numSamples.toDouble / metadata.numBins(featureIndex) - logDebug("stride = " + stride) - for (splitIndex <- 0 until numSplits) { - val sampleIndex = splitIndex * stride.toInt - // Set threshold halfway in between 2 samples. - val threshold = (featureSamples(sampleIndex) + featureSamples(sampleIndex + 1)) / 2.0 + + var splitIndex = 0 + while (splitIndex < numSplits) { + val threshold = featureSplits(splitIndex) splits(featureIndex)(splitIndex) = new Split(featureIndex, threshold, Continuous, List()) + splitIndex += 1 } bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), splits(featureIndex)(0), Continuous, Double.MinValue) - for (splitIndex <- 1 until numSplits) { + + splitIndex = 1 + while (splitIndex < numSplits) { bins(featureIndex)(splitIndex) = new Bin(splits(featureIndex)(splitIndex - 1), splits(featureIndex)(splitIndex), Continuous, Double.MinValue) + splitIndex += 1 } bins(featureIndex)(numSplits) = new Bin(splits(featureIndex)(numSplits - 1), new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) } else { + val numSplits = metadata.numSplits(featureIndex) + val numBins = metadata.numBins(featureIndex) // Categorical feature val featureArity = metadata.featureArity(featureIndex) if (metadata.isUnordered(featureIndex)) { @@ -1011,4 +1020,77 @@ object DecisionTree extends Serializable with Logging { categories } + /** + * Find splits for a continuous feature + * NOTE: Returned number of splits is set based on `featureSamples` and + * could be different from the specified `numSplits`. + * The `numSplits` attribute in the `DecisionTreeMetadata` class will be set accordingly. + * @param featureSamples feature values of each sample + * @param metadata decision tree metadata + * NOTE: `metadata.numbins` will be changed accordingly + * if there are not enough splits to be found + * @param featureIndex feature index to find splits + * @return array of splits + */ + private[tree] def findSplitsForContinuousFeature( + featureSamples: Array[Double], + metadata: DecisionTreeMetadata, + featureIndex: Int): Array[Double] = { + require(metadata.isContinuous(featureIndex), + "findSplitsForContinuousFeature can only be used to find splits for a continuous feature.") + + val splits = { + val numSplits = metadata.numSplits(featureIndex) + + // get count for each distinct value + val valueCountMap = featureSamples.foldLeft(Map.empty[Double, Int]) { (m, x) => + m + ((x, m.getOrElse(x, 0) + 1)) + } + // sort distinct values + val valueCounts = valueCountMap.toSeq.sortBy(_._1).toArray + + // if possible splits is not enough or just enough, just return all possible splits + val possibleSplits = valueCounts.length + if (possibleSplits <= numSplits) { + valueCounts.map(_._1) + } else { + // stride between splits + val stride: Double = featureSamples.length.toDouble / (numSplits + 1) + logDebug("stride = " + stride) + + // iterate `valueCount` to find splits + val splits = new ArrayBuffer[Double] + var index = 1 + // currentCount: sum of counts of values that have been visited + var currentCount = valueCounts(0)._2 + // targetCount: target value for `currentCount`. + // If `currentCount` is closest value to `targetCount`, + // then current value is a split threshold. + // After finding a split threshold, `targetCount` is added by stride. + var targetCount = stride + while (index < valueCounts.length) { + val previousCount = currentCount + currentCount += valueCounts(index)._2 + val previousGap = math.abs(previousCount - targetCount) + val currentGap = math.abs(currentCount - targetCount) + // If adding count of current value to currentCount + // makes the gap between currentCount and targetCount smaller, + // previous value is a split threshold. + if (previousGap < currentGap) { + splits.append(valueCounts(index - 1)._1) + targetCount += stride + } + index += 1 + } + + splits.toArray + } + } + + assert(splits.length > 0) + // set number of splits accordingly + metadata.setNumSplits(featureIndex, splits.length) + + splits + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index 772c02670e541..5bc0f2635c6b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -76,6 +76,17 @@ private[tree] class DecisionTreeMetadata( numBins(featureIndex) - 1 } + + /** + * Set number of splits for a continuous feature. + * For a continuous feature, number of bins is number of splits plus 1. + */ + def setNumSplits(featureIndex: Int, numSplits: Int) { + require(isContinuous(featureIndex), + s"Only number of bin for a continuous feature can be set.") + numBins(featureIndex) = numSplits + 1 + } + /** * Indicates if feature subsampling is being used. */ diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 98a72b0c4d750..8fc5e111bbc17 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ -import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.configuration.{QuantileStrategy, Strategy} import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata, TreePoint} import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} import org.apache.spark.mllib.tree.model.{InformationGainStats, DecisionTreeModel, Node} @@ -102,6 +102,72 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(List(3.0, 2.0, 0.0).toSeq === l.toSeq) } + test("find splits for a continuous feature") { + // find splits for normal case + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(6), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array.fill(200000)(math.random) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 5) + assert(fakeMetadata.numSplits(0) === 5) + assert(fakeMetadata.numBins(0) === 6) + // check returned splits are distinct + assert(splits.distinct.length === splits.length) + } + + // find splits should not return identical splits + // when there are not enough split candidates, reduce the number of splits in metadata + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(5), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 3).map(_.toDouble) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 3) + assert(fakeMetadata.numSplits(0) === 3) + assert(fakeMetadata.numBins(0) === 4) + // check returned splits are distinct + assert(splits.distinct.length === splits.length) + } + + // find splits when most samples close to the minimum + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 4, 5).map(_.toDouble) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 2) + assert(fakeMetadata.numSplits(0) === 2) + assert(fakeMetadata.numBins(0) === 3) + assert(splits(0) === 2.0) + assert(splits(1) === 3.0) + } + + // find splits when most samples close to the maximum + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0, + Map(), Set(), + Array(3), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0, 0 + ) + val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2).map(_.toDouble) + val splits = DecisionTree.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) + assert(splits.length === 1) + assert(fakeMetadata.numSplits(0) === 1) + assert(fakeMetadata.numBins(0) === 2) + assert(splits(0) === 1.0) + } + } + test("Multiclass classification with unordered categorical features:" + " split and bin calculations") { val arr = DecisionTreeSuite.generateCategoricalDataPoints() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index fb44ceb0f57ee..6b13765b98f41 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -93,8 +93,9 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { val categoricalFeaturesInfo = Map.empty[Int, Int] val numTrees = 1 - val strategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + val strategy = new Strategy(algo = Regression, impurity = Variance, + maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + categoricalFeaturesInfo = categoricalFeaturesInfo) val rf = RandomForest.trainRegressor(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 0938eebd3a548..64ee79d83e849 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -153,9 +153,9 @@ def trainClassifier(data, numClasses, categoricalFeaturesInfo, DecisionTreeModel classifier of depth 1 with 3 nodes >>> print model.toDebugString(), # it already has newline DecisionTreeModel classifier of depth 1 with 3 nodes - If (feature 0 <= 0.5) + If (feature 0 <= 0.0) Predict: 0.0 - Else (feature 0 > 0.5) + Else (feature 0 > 0.0) Predict: 1.0 >>> model.predict(array([1.0])) > 0 True From 1b3ce61ce9061719d8c28d621c3200e0bff3a4e9 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 20 Oct 2014 15:29:54 -0700 Subject: [PATCH 1024/1492] [SPARK-3906][SQL] Adds multiple join support for SQLContext Author: Cheng Lian Closes #2767 from liancheng/multi-join and squashes the following commits: 9dc0d18 [Cheng Lian] Adds multiple join support for SQLContext --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 8 +++++--- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 11 +++++++++++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index b4d606d37e732..a277684f6327c 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -181,9 +181,11 @@ class SqlParser extends AbstractSparkSQLParser { ) protected lazy val joinedRelation: Parser[LogicalPlan] = - relationFactor ~ joinType.? ~ (JOIN ~> relationFactor) ~ joinConditions.? ^^ { - case r1 ~ jt ~ r2 ~ cond => - Join(r1, r2, joinType = jt.getOrElse(Inner), cond) + relationFactor ~ rep1(joinType.? ~ (JOIN ~> relationFactor) ~ joinConditions.?) ^^ { + case r1 ~ joins => + joins.foldLeft(r1) { case (lhs, jt ~ rhs ~ cond) => + Join(lhs, rhs, joinType = jt.getOrElse(Inner), cond) + } } protected lazy val joinConditions: Parser[Expression] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 15f6ba4f72bbd..beb924f1715b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -720,4 +720,15 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAggregation("SELECT key + 2, COUNT(*) FROM testData GROUP BY key + 1") checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) } + + test("Multiple join") { + checkAnswer( + sql( + """SELECT a.key, b.key, c.key + |FROM testData a + |JOIN testData b ON a.key = b.key + |JOIN testData c ON a.key = c.key + """.stripMargin), + (1 to 100).map(i => Seq(i, i, i))) + } } From e9c1afa87b8a6d9ea02aa9f7c319a36f00c0e4c3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 20 Oct 2014 15:32:17 -0700 Subject: [PATCH 1025/1492] [SPARK-3800][SQL] Clean aliases from grouping expressions Author: Michael Armbrust Closes #2658 from marmbrus/nestedAggs and squashes the following commits: 862b763 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs 3234521 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs 8b06fdc [Michael Armbrust] possible fix for grouping on nested fields --- .../spark/sql/catalyst/analysis/Analyzer.scala | 18 ++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 17 +++++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 82553063145b8..a448c794213ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -60,6 +60,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ResolveFunctions :: GlobalAggregates :: UnresolvedHavingClauseAttributes :: + TrimAliases :: typeCoercionRules ++ extendedRules : _*), Batch("Check Analysis", Once, @@ -89,6 +90,23 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } } + /** + * Removes no-op Alias expressions from the plan. + */ + object TrimAliases extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case Aggregate(groups, aggs, child) => + Aggregate( + groups.map { + _ transform { + case Alias(c, _) => c + } + }, + aggs, + child) + } + } + /** * Checks for non-aggregated attributes with aggregation */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index beb924f1715b0..3959925a2e529 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -43,6 +43,23 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { TimeZone.setDefault(origZone) } + test("grouping on nested fields") { + jsonRDD(sparkContext.parallelize("""{"nested": {"attribute": 1}, "value": 2}""" :: Nil)) + .registerTempTable("rows") + + checkAnswer( + sql( + """ + |select attribute, sum(cnt) + |from ( + | select nested.attribute, count(*) as cnt + | from rows + | group by nested.attribute) a + |group by attribute + """.stripMargin), + Row(1, 1) :: Nil) + } + test("SPARK-3176 Added Parser of SQL ABS()") { checkAnswer( sql("SELECT ABS(-1.3)"), From 364d52b707b5c49bc29ce60dbfe6e845a75d5d86 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 20 Oct 2014 15:51:05 -0700 Subject: [PATCH 1026/1492] [SPARK-3966][SQL] Fix nullabilities of Cast related to DateType. Author: Takuya UESHIN Closes #2820 from ueshin/issues/SPARK-3966 and squashes the following commits: ca4a745 [Takuya UESHIN] Fix nullabilities of Cast related to DateType. --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 8e5ee12e314bf..8e5baf0eb82d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -32,6 +32,10 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case (StringType, _: NumericType) => true case (StringType, TimestampType) => true case (StringType, DateType) => true + case (_: NumericType, DateType) => true + case (BooleanType, DateType) => true + case (DateType, _: NumericType) => true + case (DateType, BooleanType) => true case _ => child.nullable } From fce1d41611fdb27956c3394a706ed14960182a83 Mon Sep 17 00:00:00 2001 From: luogankun Date: Mon, 20 Oct 2014 16:50:51 -0700 Subject: [PATCH 1027/1492] [SPARK-3945]Properties of hive-site.xml is invalid in running the Thrift JDBC server Write properties of hive-site.xml to HiveContext when initilize session state in SparkSQLEnv.scala. The method of SparkSQLEnv.init() in HiveThriftServer2.scala can not write the properties of hive-site.xml to HiveContext. Such as: add configuration property spark.sql.shuffle.partititions in the hive-site.xml. Author: luogankun Closes #2800 from luogankun/SPARK-3945 and squashes the following commits: 3679efc [luogankun] [SPARK-3945]Write properties of hive-site.xml to HiveContext when initilize session state In SparkSQLEnv.scala --- .../apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 582264eb59f83..2136a2ea63543 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -39,7 +39,11 @@ private[hive] object SparkSQLEnv extends Logging { sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = SessionState.get() + @transient override lazy val sessionState = { + val state = SessionState.get() + setConf(state.getConf.getAllProperties) + state + } @transient override lazy val hiveconf = sessionState.getConf } } From 7586e2e67ad45007f78803179b04d199c174bd69 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 20 Oct 2014 17:09:12 -0700 Subject: [PATCH 1028/1492] [SPARK-3969][SQL] Optimizer should have a super class as an interface. Some developers want to replace `Optimizer` to fit their projects but can't do so because currently `Optimizer` is an `object`. Author: Takuya UESHIN Closes #2825 from ueshin/issues/SPARK-3969 and squashes the following commits: abbc53c [Takuya UESHIN] Re-rename Optimizer object. 4d2e1bc [Takuya UESHIN] Rename Optimizer object. 9547a23 [Takuya UESHIN] Extract abstract class from Optimizer for developers to be able to replace Optimizer. --- .../org/apache/spark/sql/catalyst/optimizer/Optimizer.scala | 4 +++- .../sql/catalyst/optimizer/ExpressionOptimizationSuite.scala | 4 ++-- sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 3693b41404fd6..9ce7c78195830 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -28,7 +28,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ -object Optimizer extends RuleExecutor[LogicalPlan] { +abstract class Optimizer extends RuleExecutor[LogicalPlan] + +object DefaultOptimizer extends Optimizer { val batches = Batch("Combine Limits", FixedPoint(100), CombineLimits) :: diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala index 890d6289b9dfb..ae99a3f9ba287 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala @@ -30,7 +30,7 @@ class ExpressionOptimizationSuite extends ExpressionEvaluationSuite { expected: Any, inputRow: Row = EmptyRow): Unit = { val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, NoRelation) - val optimizedPlan = Optimizer(plan) + val optimizedPlan = DefaultOptimizer(plan) super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow) } -} \ No newline at end of file +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 23e7b2d270777..0e4a9ca60b00d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl.ExpressionConversions import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.types.DataType @@ -68,7 +68,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new Analyzer(catalog, functionRegistry, caseSensitive = true) @transient - protected[sql] val optimizer = Optimizer + protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer @transient protected[sql] val sqlParser = { From 0fe1c093690e5af6268182801747dfa6394dfc5b Mon Sep 17 00:00:00 2001 From: wangxiaojing Date: Mon, 20 Oct 2014 17:15:24 -0700 Subject: [PATCH 1029/1492] [SPARK-3940][SQL] Avoid console printing error messages three times MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit If wrong sql,the console print error one times。 eg:
    spark-sql> show tabless;
    show tabless;
    14/10/13 21:03:48 INFO ParseDriver: Parsing command: show tabless
    ............
    	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:274)
    	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413)
    	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:209)
    	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
    Caused by: org.apache.hadoop.hive.ql.parse.ParseException: line 1:5 cannot recognize input near 'show' 'tabless' '' in ddl statement
    
    	at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:193)
    	at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:161)
    	at org.apache.spark.sql.hive.HiveQl$.getAst(HiveQl.scala:218)
    	at org.apache.spark.sql.hive.HiveQl$.createPlan(HiveQl.scala:226)
    	... 47 more
    Time taken: 4.35 seconds
    14/10/13 21:03:51 INFO CliDriver: Time taken: 4.35 seconds
    
    Author: wangxiaojing Closes #2790 from wangxiaojing/spark-3940 and squashes the following commits: e2e5c14 [wangxiaojing] sql Print the error code three times --- .../org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 7463df1f47d43..a5c457c677564 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -62,7 +62,7 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo } catch { case cause: Throwable => logError(s"Failed in [$command]", cause) - new CommandProcessorResponse(-3, ExceptionUtils.getFullStackTrace(cause), null) + new CommandProcessorResponse(0, ExceptionUtils.getFullStackTrace(cause), null) } } From 342b57db66e379c475daf5399baf680ff42b87c2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 20 Oct 2014 19:16:35 -0700 Subject: [PATCH 1030/1492] Update Building Spark link. --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8dd8b70696aa2..dbf53dcd76b2d 100644 --- a/README.md +++ b/README.md @@ -25,7 +25,7 @@ To build Spark and its example programs, run: (You do not need to do this if you downloaded a pre-built package.) More detailed documentation is available from the project site, at -["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). +["Building Spark with Maven"](http://spark.apache.org/docs/latest/building-with-maven.html). ## Interactive Scala Shell From 5a8f64f33632fbf89d16cade2e0e66c5ed60760b Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 21 Oct 2014 00:49:11 -0700 Subject: [PATCH 1031/1492] [SPARK-3958] TorrentBroadcast cleanup / debugging improvements. This PR makes several changes to TorrentBroadcast in order to make it easier to reason about, which should help when debugging SPARK-3958. The key changes: - Remove all state from the global TorrentBroadcast object. This state consisted mainly of configuration options, like the block size and compression codec, and was read by the blockify / unblockify methods. Unfortunately, the use of `lazy val` for `BLOCK_SIZE` meant that the block size was always determined by the first SparkConf that TorrentBroadast was initialized with; as a result, unit tests could not properly test TorrentBroadcast with different block sizes. Instead, blockifyObject and unBlockifyObject now accept compression codecs and blockSizes as arguments. These arguments are supplied at the call sites inside of TorrentBroadcast instances. Each TorrentBroadcast instance determines these values from SparkEnv's SparkConf. I was careful to ensure that we do not accidentally serialize CompressionCodec or SparkConf objects as part of the TorrentBroadcast object. - Remove special-case handling of local-mode in TorrentBroadcast. I don't think that broadcast implementations should know about whether we're running in local mode. If we want to optimize the performance of broadcast in local mode, then we should detect this at a higher level and use a dummy LocalBroadcastFactory implementation instead. Removing this code fixes a subtle error condition: in the old local mode code, a failure to find the broadcast in the local BlockManager would lead to an attempt to deblockify zero blocks, which could lead to confusing deserialization or decompression errors when we attempted to decompress an empty byte array. This should never have happened, though: a failure to find the block in local mode is evidence of some other error. The changes here will make it easier to debug those errors if they ever happen. - Add a check that throws an exception when attempting to deblockify an empty array. - Use ScalaCheck to add a test to check that TorrentBroadcast's blockifyObject and unBlockifyObject methods are inverses. - Misc. cleanup and logging improvements. Author: Josh Rosen Closes #2844 from JoshRosen/torrentbroadcast-bugfix and squashes the following commits: 1e8268d [Josh Rosen] Address Reynold's review comments 2a9fdfd [Josh Rosen] Address Reynold's review comments. c3b08f9 [Josh Rosen] Update TorrentBroadcast tests to reflect removal of special local-mode optimizations. 5c22782 [Josh Rosen] Store broadcast variable's value in the driver. 33fc754 [Josh Rosen] Change blockify/unblockifyObject to accept serializer as argument. 618a872 [Josh Rosen] [SPARK-3958] TorrentBroadcast cleanup / debugging improvements. --- .../spark/broadcast/TorrentBroadcast.scala | 136 +++++++++--------- .../broadcast/TorrentBroadcastFactory.scala | 11 +- .../spark/broadcast/BroadcastSuite.scala | 42 ++++-- 3 files changed, 97 insertions(+), 92 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 42d58682a1e23..99af2e9608ea7 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -26,6 +26,7 @@ import scala.util.Random import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec +import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.ByteBufferInputStream import org.apache.spark.util.io.ByteArrayChunkOutputStream @@ -46,14 +47,12 @@ import org.apache.spark.util.io.ByteArrayChunkOutputStream * This prevents the driver from being the bottleneck in sending out multiple copies of the * broadcast data (one per executor) as done by the [[org.apache.spark.broadcast.HttpBroadcast]]. * + * When initialized, TorrentBroadcast objects read SparkEnv.get.conf. + * * @param obj object to broadcast - * @param isLocal whether Spark is running in local mode (single JVM process). * @param id A unique identifier for the broadcast variable. */ -private[spark] class TorrentBroadcast[T: ClassTag]( - obj : T, - @transient private val isLocal: Boolean, - id: Long) +private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) extends Broadcast[T](id) with Logging with Serializable { /** @@ -62,6 +61,20 @@ private[spark] class TorrentBroadcast[T: ClassTag]( * blocks from the driver and/or other executors. */ @transient private var _value: T = obj + /** The compression codec to use, or None if compression is disabled */ + @transient private var compressionCodec: Option[CompressionCodec] = _ + /** Size of each block. Default value is 4MB. This value is only read by the broadcaster. */ + @transient private var blockSize: Int = _ + + private def setConf(conf: SparkConf) { + compressionCodec = if (conf.getBoolean("spark.broadcast.compress", true)) { + Some(CompressionCodec.createCodec(conf)) + } else { + None + } + blockSize = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 + } + setConf(SparkEnv.get.conf) private val broadcastId = BroadcastBlockId(id) @@ -76,23 +89,20 @@ private[spark] class TorrentBroadcast[T: ClassTag]( * @return number of blocks this broadcast variable is divided into */ private def writeBlocks(): Int = { - // For local mode, just put the object in the BlockManager so we can find it later. - SparkEnv.get.blockManager.putSingle( - broadcastId, _value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) - - if (!isLocal) { - val blocks = TorrentBroadcast.blockifyObject(_value) - blocks.zipWithIndex.foreach { case (block, i) => - SparkEnv.get.blockManager.putBytes( - BroadcastBlockId(id, "piece" + i), - block, - StorageLevel.MEMORY_AND_DISK_SER, - tellMaster = true) - } - blocks.length - } else { - 0 + // Store a copy of the broadcast variable in the driver so that tasks run on the driver + // do not create a duplicate copy of the broadcast variable's value. + SparkEnv.get.blockManager.putSingle(broadcastId, _value, StorageLevel.MEMORY_AND_DISK, + tellMaster = false) + val blocks = + TorrentBroadcast.blockifyObject(_value, blockSize, SparkEnv.get.serializer, compressionCodec) + blocks.zipWithIndex.foreach { case (block, i) => + SparkEnv.get.blockManager.putBytes( + BroadcastBlockId(id, "piece" + i), + block, + StorageLevel.MEMORY_AND_DISK_SER, + tellMaster = true) } + blocks.length } /** Fetch torrent blocks from the driver and/or other executors. */ @@ -104,29 +114,24 @@ private[spark] class TorrentBroadcast[T: ClassTag]( for (pid <- Random.shuffle(Seq.range(0, numBlocks))) { val pieceId = BroadcastBlockId(id, "piece" + pid) - - // First try getLocalBytes because there is a chance that previous attempts to fetch the + logDebug(s"Reading piece $pieceId of $broadcastId") + // First try getLocalBytes because there is a chance that previous attempts to fetch the // broadcast blocks have already fetched some of the blocks. In that case, some blocks // would be available locally (on this executor). - var blockOpt = bm.getLocalBytes(pieceId) - if (!blockOpt.isDefined) { - blockOpt = bm.getRemoteBytes(pieceId) - blockOpt match { - case Some(block) => - // If we found the block from remote executors/driver's BlockManager, put the block - // in this executor's BlockManager. - SparkEnv.get.blockManager.putBytes( - pieceId, - block, - StorageLevel.MEMORY_AND_DISK_SER, - tellMaster = true) - - case None => - throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) - } + def getLocal: Option[ByteBuffer] = bm.getLocalBytes(pieceId) + def getRemote: Option[ByteBuffer] = bm.getRemoteBytes(pieceId).map { block => + // If we found the block from remote executors/driver's BlockManager, put the block + // in this executor's BlockManager. + SparkEnv.get.blockManager.putBytes( + pieceId, + block, + StorageLevel.MEMORY_AND_DISK_SER, + tellMaster = true) + block } - // If we get here, the option is defined. - blocks(pid) = blockOpt.get + val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse( + throw new SparkException(s"Failed to get $pieceId of $broadcastId")) + blocks(pid) = block } blocks } @@ -156,6 +161,7 @@ private[spark] class TorrentBroadcast[T: ClassTag]( private def readObject(in: ObjectInputStream) { in.defaultReadObject() TorrentBroadcast.synchronized { + setConf(SparkEnv.get.conf) SparkEnv.get.blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => _value = x.asInstanceOf[T] @@ -167,7 +173,8 @@ private[spark] class TorrentBroadcast[T: ClassTag]( val time = (System.nanoTime() - start) / 1e9 logInfo("Reading broadcast variable " + id + " took " + time + " s") - _value = TorrentBroadcast.unBlockifyObject[T](blocks) + _value = + TorrentBroadcast.unBlockifyObject[T](blocks, SparkEnv.get.serializer, compressionCodec) // Store the merged copy in BlockManager so other tasks on this executor don't // need to re-fetch it. SparkEnv.get.blockManager.putSingle( @@ -179,43 +186,29 @@ private[spark] class TorrentBroadcast[T: ClassTag]( private object TorrentBroadcast extends Logging { - /** Size of each block. Default value is 4MB. */ - private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 - private var initialized = false - private var conf: SparkConf = null - private var compress: Boolean = false - private var compressionCodec: CompressionCodec = null - - def initialize(_isDriver: Boolean, conf: SparkConf) { - TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests - synchronized { - if (!initialized) { - compress = conf.getBoolean("spark.broadcast.compress", true) - compressionCodec = CompressionCodec.createCodec(conf) - initialized = true - } - } - } - def stop() { - initialized = false - } - - def blockifyObject[T: ClassTag](obj: T): Array[ByteBuffer] = { - val bos = new ByteArrayChunkOutputStream(BLOCK_SIZE) - val out: OutputStream = if (compress) compressionCodec.compressedOutputStream(bos) else bos - val ser = SparkEnv.get.serializer.newInstance() + def blockifyObject[T: ClassTag]( + obj: T, + blockSize: Int, + serializer: Serializer, + compressionCodec: Option[CompressionCodec]): Array[ByteBuffer] = { + val bos = new ByteArrayChunkOutputStream(blockSize) + val out: OutputStream = compressionCodec.map(c => c.compressedOutputStream(bos)).getOrElse(bos) + val ser = serializer.newInstance() val serOut = ser.serializeStream(out) serOut.writeObject[T](obj).close() bos.toArrays.map(ByteBuffer.wrap) } - def unBlockifyObject[T: ClassTag](blocks: Array[ByteBuffer]): T = { + def unBlockifyObject[T: ClassTag]( + blocks: Array[ByteBuffer], + serializer: Serializer, + compressionCodec: Option[CompressionCodec]): T = { + require(blocks.nonEmpty, "Cannot unblockify an empty array of blocks") val is = new SequenceInputStream( asJavaEnumeration(blocks.iterator.map(block => new ByteBufferInputStream(block)))) - val in: InputStream = if (compress) compressionCodec.compressedInputStream(is) else is - - val ser = SparkEnv.get.serializer.newInstance() + val in: InputStream = compressionCodec.map(c => c.compressedInputStream(is)).getOrElse(is) + val ser = serializer.newInstance() val serIn = ser.deserializeStream(in) val obj = serIn.readObject[T]() serIn.close() @@ -227,6 +220,7 @@ private object TorrentBroadcast extends Logging { * If removeFromDriver is true, also remove these persisted blocks on the driver. */ def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = { + logDebug(s"Unpersisting TorrentBroadcast $id") SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) } } diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala index ad0f701d7a98f..fb024c12094f2 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala @@ -28,14 +28,13 @@ import org.apache.spark.{SecurityManager, SparkConf} */ class TorrentBroadcastFactory extends BroadcastFactory { - override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { - TorrentBroadcast.initialize(isDriver, conf) - } + override def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { } - override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = - new TorrentBroadcast[T](value_, isLocal, id) + override def newBroadcast[T: ClassTag](value_ : T, isLocal: Boolean, id: Long) = { + new TorrentBroadcast[T](value_, id) + } - override def stop() { TorrentBroadcast.stop() } + override def stop() { } /** * Remove all persisted state associated with the torrent broadcast with the given ID. diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index acaf321de52fb..e096c8c3e9b46 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.broadcast +import scala.util.Random + import org.scalatest.FunSuite import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} +import org.apache.spark.io.SnappyCompressionCodec +import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ - class BroadcastSuite extends FunSuite with LocalSparkContext { private val httpConf = broadcastConf("HttpBroadcastFactory") @@ -84,6 +87,24 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) } + test("TorrentBroadcast's blockifyObject and unblockifyObject are inverses") { + import org.apache.spark.broadcast.TorrentBroadcast._ + val blockSize = 1024 + val conf = new SparkConf() + val compressionCodec = Some(new SnappyCompressionCodec(conf)) + val serializer = new JavaSerializer(conf) + val seed = 42 + val rand = new Random(seed) + for (trial <- 1 to 100) { + val size = 1 + rand.nextInt(1024 * 10) + val data: Array[Byte] = new Array[Byte](size) + rand.nextBytes(data) + val blocks = blockifyObject(data, blockSize, serializer, compressionCodec) + val unblockified = unBlockifyObject[Array[Byte]](blocks, serializer, compressionCodec) + assert(unblockified === data) + } + } + test("Unpersisting HttpBroadcast on executors only in local mode") { testUnpersistHttpBroadcast(distributed = false, removeFromDriver = false) } @@ -193,26 +214,17 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { blockId = BroadcastBlockId(broadcastId, "piece0") statuses = bmm.getBlockStatus(blockId, askSlaves = true) - assert(statuses.size === (if (distributed) 1 else 0)) + assert(statuses.size === 1) } // Verify that blocks are persisted in both the executors and the driver def afterUsingBroadcast(broadcastId: Long, bmm: BlockManagerMaster) { var blockId = BroadcastBlockId(broadcastId) - var statuses = bmm.getBlockStatus(blockId, askSlaves = true) - if (distributed) { - assert(statuses.size === numSlaves + 1) - } else { - assert(statuses.size === 1) - } + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === numSlaves + 1) blockId = BroadcastBlockId(broadcastId, "piece0") - statuses = bmm.getBlockStatus(blockId, askSlaves = true) - if (distributed) { - assert(statuses.size === numSlaves + 1) - } else { - assert(statuses.size === 0) - } + assert(statuses.size === numSlaves + 1) } // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver @@ -224,7 +236,7 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { assert(statuses.size === expectedNumBlocks) blockId = BroadcastBlockId(broadcastId, "piece0") - expectedNumBlocks = if (removeFromDriver || !distributed) 0 else 1 + expectedNumBlocks = if (removeFromDriver) 0 else 1 statuses = bmm.getBlockStatus(blockId, askSlaves = true) assert(statuses.size === expectedNumBlocks) } From 85708168341a9406c451df20af3374c0850ce166 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 21 Oct 2014 09:29:45 -0700 Subject: [PATCH 1032/1492] [SPARK-4023] [MLlib] [PySpark] convert rdd into RDD of Vector Convert the input rdd to RDD of Vector. cc mengxr Author: Davies Liu Closes #2870 from davies/fix4023 and squashes the following commits: 1eac767 [Davies Liu] address comments 0871576 [Davies Liu] convert rdd into RDD of Vector --- python/pyspark/mllib/stat.py | 9 +++++---- python/pyspark/mllib/tests.py | 19 +++++++++++++++++++ 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/stat.py b/python/pyspark/mllib/stat.py index a6019dadf781c..84baf12b906df 100644 --- a/python/pyspark/mllib/stat.py +++ b/python/pyspark/mllib/stat.py @@ -22,7 +22,7 @@ from functools import wraps from pyspark import PickleSerializer -from pyspark.mllib.linalg import _to_java_object_rdd +from pyspark.mllib.linalg import _convert_to_vector, _to_java_object_rdd __all__ = ['MultivariateStatisticalSummary', 'Statistics'] @@ -107,7 +107,7 @@ def colStats(rdd): array([ 2., 0., 0., -2.]) """ sc = rdd.ctx - jrdd = _to_java_object_rdd(rdd) + jrdd = _to_java_object_rdd(rdd.map(_convert_to_vector)) cStats = sc._jvm.PythonMLLibAPI().colStats(jrdd) return MultivariateStatisticalSummary(sc, cStats) @@ -163,14 +163,15 @@ def corr(x, y=None, method=None): if type(y) == str: raise TypeError("Use 'method=' to specify method name.") - jx = _to_java_object_rdd(x) if not y: + jx = _to_java_object_rdd(x.map(_convert_to_vector)) resultMat = sc._jvm.PythonMLLibAPI().corr(jx, method) bytes = sc._jvm.SerDe.dumps(resultMat) ser = PickleSerializer() return ser.loads(str(bytes)).toArray() else: - jy = _to_java_object_rdd(y) + jx = _to_java_object_rdd(x.map(float)) + jy = _to_java_object_rdd(y.map(float)) return sc._jvm.PythonMLLibAPI().corr(jx, jy, method) diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 463faf7b6f520..d6fb87b378b4a 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -36,6 +36,8 @@ from pyspark.serializers import PickleSerializer from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, _convert_to_vector from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.random import RandomRDDs +from pyspark.mllib.stat import Statistics from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase @@ -202,6 +204,23 @@ def test_regression(self): self.assertTrue(dt_model.predict(features[3]) > 0) +class StatTests(PySparkTestCase): + # SPARK-4023 + def test_col_with_different_rdds(self): + # numpy + data = RandomRDDs.normalVectorRDD(self.sc, 1000, 10, 10) + summary = Statistics.colStats(data) + self.assertEqual(1000, summary.count()) + # array + data = self.sc.parallelize([range(10)] * 10) + summary = Statistics.colStats(data) + self.assertEqual(10, summary.count()) + # array + data = self.sc.parallelize([pyarray.array("d", range(10))] * 10) + summary = Statistics.colStats(data) + self.assertEqual(10, summary.count()) + + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): From 2aeb84bc79b643912d26e08ec2d87e444027fefc Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 21 Oct 2014 09:37:17 -0700 Subject: [PATCH 1033/1492] replace awaitTransformation with awaitTermination in scaladoc/javadoc Author: Holden Karau Closes #2861 from holdenk/SPARK-4015-Documentation-in-the-streaming-context-references-non-existent-function and squashes the following commits: 081db8a [Holden Karau] fix pyspark streaming doc too 0e03863 [Holden Karau] replace awaitTransformation with awaitTermination --- python/pyspark/streaming/context.py | 2 +- .../scala/org/apache/spark/streaming/StreamingContext.scala | 2 +- .../apache/spark/streaming/api/java/JavaStreamingContext.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index dc9dc41121935..2f53fbd27b17a 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -79,7 +79,7 @@ class StreamingContext(object): L{DStream} various input sources. It can be from an existing L{SparkContext}. After creating and transforming DStreams, the streaming computation can be started and stopped using `context.start()` and `context.stop()`, - respectively. `context.awaitTransformation()` allows the current thread + respectively. `context.awaitTermination()` allows the current thread to wait for the termination of the context by `stop()` or by an exception. """ _transformerSerializer = None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 5a8eef1372e23..23d6d1c5e50fa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -47,7 +47,7 @@ import org.apache.spark.streaming.ui.{StreamingJobProgressListener, StreamingTab * The associated SparkContext can be accessed using `context.sparkContext`. After * creating and transforming DStreams, the streaming computation can be started and stopped * using `context.start()` and `context.stop()`, respectively. - * `context.awaitTransformation()` allows the current thread to wait for the termination + * `context.awaitTermination()` allows the current thread to wait for the termination * of the context by `stop()` or by an exception. */ class StreamingContext private[streaming] ( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 9dc26dc6b32a1..7db66c69a6d73 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -46,7 +46,7 @@ import org.apache.spark.streaming.receiver.Receiver * org.apache.spark.api.java.JavaSparkContext (see core Spark documentation) can be accessed * using `context.sparkContext`. After creating and transforming DStreams, the streaming * computation can be started and stopped using `context.start()` and `context.stop()`, - * respectively. `context.awaitTransformation()` allows the current thread to wait for the + * respectively. `context.awaitTermination()` allows the current thread to wait for the * termination of a context by `stop()` or by an exception. */ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { From c262cd5ddce75333ec936e90b81278c3992841ec Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 21 Oct 2014 10:37:13 -0700 Subject: [PATCH 1034/1492] [SPARK-4035] Fix a wrong format specifier Just found a typo. Should not use "%f" for Long. Author: zsxwing Closes #2875 from zsxwing/SPARK-4035 and squashes the following commits: ce347e2 [zsxwing] Fix a wrong format specifier --- core/src/main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 3f5d06e1aeee7..0ce2a3f631b15 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -870,7 +870,7 @@ private[spark] class BlockManager( logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") blockTransferService.uploadBlockSync( peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) - logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %f ms" + logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %d ms" .format((System.currentTimeMillis - onePeerStartTime))) peersReplicatedTo += peer peersForReplication -= peer From 61ca7742d21dd66f5a7b3bb826e3aaca6f049b68 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 21 Oct 2014 11:22:25 -0700 Subject: [PATCH 1035/1492] [SPARK-4020] Do not rely on timeouts to remove failed block managers If an executor fails without being scheduled to run any tasks, then `DAGScheduler` won't notify `BlockManagerMasterActor` that the associated block manager should be removed. Instead, the associated block manager will be expired only after a few rounds of heartbeat timeouts. In terms of removal treatment, there should really be no distinction between executors that have been scheduled tasks and those that have not. The fix, then, is to add all known executors to `TaskSchedulerImpl`'s `activeExecutorIds` whether or not it has been scheduled a task. In fact, the existing comment above `activeExecutorIds` is ``` // Which executor IDs we have executors on val activeExecutorIds = new HashSet[String] ``` not "Which executors have been scheduled tasks thus far." Author: Andrew Or Closes #2865 from andrewor14/active-executors and squashes the following commits: ff3172b [Andrew Or] Add all known executors to `activeExecutorIds` --- .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6d697e3d003f6..2b39c7fc872da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -221,6 +221,7 @@ private[spark] class TaskSchedulerImpl( var newExecAvail = false for (o <- offers) { executorIdToHost(o.executorId) = o.host + activeExecutorIds += o.executorId if (!executorsByHost.contains(o.host)) { executorsByHost(o.host) = new HashSet[String]() executorAdded(o.executorId, o.host) @@ -261,7 +262,6 @@ private[spark] class TaskSchedulerImpl( val tid = task.taskId taskIdToTaskSetId(tid) = taskSet.taskSet.id taskIdToExecutorId(tid) = execId - activeExecutorIds += execId executorsByHost(host) += execId availableCpus(i) -= CPUS_PER_TASK assert(availableCpus(i) >= 0) From 1a623b2e163da3a9112cb9b68bda22b9e398ed5c Mon Sep 17 00:00:00 2001 From: Michelangelo D'Agostino Date: Tue, 21 Oct 2014 11:49:39 -0700 Subject: [PATCH 1036/1492] SPARK-3770: Make userFeatures accessible from python https://issues.apache.org/jira/browse/SPARK-3770 We need access to the underlying latent user features from python. However, the userFeatures RDD from the MatrixFactorizationModel isn't accessible from the python bindings. I've added a method to the underlying scala class to turn the RDD[(Int, Array[Double])] to an RDD[String]. This is then accessed from the python recommendation.py Author: Michelangelo D'Agostino Closes #2636 from mdagost/mf_user_features and squashes the following commits: c98f9e2 [Michelangelo D'Agostino] Added unit tests for userFeatures and productFeatures and merged master. d5eadf8 [Michelangelo D'Agostino] Merge branch 'master' into mf_user_features 2481a2a [Michelangelo D'Agostino] Merged master and resolved conflict. a6ffb96 [Michelangelo D'Agostino] Eliminated a function from our first approach to this problem that is no longer needed now that we added the fromTuple2RDD function. 2aa1bf8 [Michelangelo D'Agostino] Implemented a function called fromTuple2RDD in PythonMLLibAPI and used it to expose the MF userFeatures and productFeatures in python. 34cb2a2 [Michelangelo D'Agostino] A couple of lint cleanups and a comment. cdd98e3 [Michelangelo D'Agostino] It's working now. e1fbe5e [Michelangelo D'Agostino] Added scala function to stringify userFeatures for access in python. --- .../mllib/api/python/PythonMLLibAPI.scala | 5 +++ python/pyspark/mllib/recommendation.py | 31 +++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 9a100170b75c6..b478c21537c2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -673,6 +673,11 @@ private[spark] object SerDe extends Serializable { rdd.map(x => (x(0).asInstanceOf[Int], x(1).asInstanceOf[Int])) } + /* convert RDD[Tuple2[,]] to RDD[Array[Any]] */ + def fromTuple2RDD(rdd: RDD[Tuple2[Any, Any]]): RDD[Array[Any]] = { + rdd.map(x => Array(x._1, x._2)) + } + /** * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by * PySpark. diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 17f96b8700bd7..22872dbbe3b55 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -53,6 +53,23 @@ class MatrixFactorizationModel(object): >>> model = ALS.train(ratings, 1) >>> model.predictAll(testset).count() == 2 True + + >>> model = ALS.train(ratings, 4) + >>> model.userFeatures().count() == 2 + True + + >>> first_user = model.userFeatures().take(1)[0] + >>> latents = first_user[1] + >>> len(latents) == 4 + True + + >>> model.productFeatures().count() == 2 + True + + >>> first_product = model.productFeatures().take(1)[0] + >>> latents = first_product[1] + >>> len(latents) == 4 + True """ def __init__(self, sc, java_model): @@ -83,6 +100,20 @@ def predictAll(self, user_product): return RDD(sc._jvm.SerDe.javaToPython(jresult), sc, AutoBatchedSerializer(PickleSerializer())) + def userFeatures(self): + sc = self._context + juf = self._java_model.userFeatures() + juf = sc._jvm.SerDe.fromTuple2RDD(juf).toJavaRDD() + return RDD(sc._jvm.PythonRDD.javaToPython(juf), sc, + AutoBatchedSerializer(PickleSerializer())) + + def productFeatures(self): + sc = self._context + jpf = self._java_model.productFeatures() + jpf = sc._jvm.SerDe.fromTuple2RDD(jpf).toJavaRDD() + return RDD(sc._jvm.PythonRDD.javaToPython(jpf), sc, + AutoBatchedSerializer(PickleSerializer())) + class ALS(object): From 5fdaf52a9df21cac69e2a4612aeb4e760e4424e7 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 21 Oct 2014 13:15:29 -0700 Subject: [PATCH 1037/1492] [SPARK-3994] Use standard Aggregator code path for countByKey and countByValue See [JIRA](https://issues.apache.org/jira/browse/SPARK-3994) for more information. Also adds a note which warns against using these methods. Author: Aaron Davidson Closes #2839 from aarondav/countByKey and squashes the following commits: d6fdb2a [Aaron Davidson] Respond to comments e1f06d3 [Aaron Davidson] [SPARK-3994] Use standard Aggregator code path for countByKey and countByValue --- .../apache/spark/rdd/PairRDDFunctions.scala | 11 +++++-- .../main/scala/org/apache/spark/rdd/RDD.scala | 31 +++++-------------- 2 files changed, 16 insertions(+), 26 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 ac96de86dd6d4..da89f634abaea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -315,8 +315,15 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) @deprecated("Use reduceByKeyLocally", "1.0.0") def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = reduceByKeyLocally(func) - /** Count the number of elements for each key, and return the result to the master as a Map. */ - def countByKey(): Map[K, Long] = self.map(_._1).countByValue() + /** + * Count the number of elements for each key, collecting the results to a local Map. + * + * Note that this method should only be used if the resulting map is expected to be small, as + * the whole thing is loaded into the driver's memory. + * To handle very large results, consider using rdd.mapValues(_ => 1L).reduceByKey(_ + _), which + * returns an RDD[T, Long] instead of a map. + */ + def countByKey(): Map[K, Long] = self.mapValues(_ => 1L).reduceByKey(_ + _).collect().toMap /** * :: Experimental :: 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 71cabf61d4ee0..b7f125d01dfaf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -927,32 +927,15 @@ abstract class RDD[T: ClassTag]( } /** - * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final - * combine step happens locally on the master, equivalent to running a single reduce task. + * Return the count of each unique value in this RDD as a local map of (value, count) pairs. + * + * Note that this method should only be used if the resulting map is expected to be small, as + * the whole thing is loaded into the driver's memory. + * To handle very large results, consider using rdd.map(x => (x, 1L)).reduceByKey(_ + _), which + * returns an RDD[T, Long] instead of a map. */ def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = { - if (elementClassTag.runtimeClass.isArray) { - throw new SparkException("countByValue() does not support arrays") - } - // TODO: This should perhaps be distributed by default. - val countPartition = (iter: Iterator[T]) => { - val map = new OpenHashMap[T,Long] - iter.foreach { - t => map.changeValue(t, 1L, _ + 1L) - } - Iterator(map) - }: Iterator[OpenHashMap[T,Long]] - val mergeMaps = (m1: OpenHashMap[T,Long], m2: OpenHashMap[T,Long]) => { - m2.foreach { case (key, value) => - m1.changeValue(key, value, _ + value) - } - m1 - }: OpenHashMap[T,Long] - val myResult = mapPartitions(countPartition).reduce(mergeMaps) - // Convert to a Scala mutable map - val mutableResult = scala.collection.mutable.Map[T,Long]() - myResult.foreach { case (k, v) => mutableResult.put(k, v) } - mutableResult + map(value => (value, null)).countByKey() } /** From 814a9cd7fabebf2a06f7e2e5d46b6a2b28b917c2 Mon Sep 17 00:00:00 2001 From: coderxiang Date: Tue, 21 Oct 2014 15:45:47 -0700 Subject: [PATCH 1038/1492] SPARK-3568 [mllib] add ranking metrics Add common metrics for ranking algorithms (http://www-nlp.stanford.edu/IR-book/), including: - Mean Average Precision - Precisionn: top-n precision - Discounted cumulative gain (DCG) and NDCG The following methods and the corresponding tests are implemented: ``` class RankingMetrics[T](predictionAndLabels: RDD[(Array[T], Array[T])]) { /* Returns the precsionk for each query */ lazy val precAtK: RDD[Array[Double]] /** * param k the position to compute the truncated precision * return the average precision at the first k ranking positions */ def precision(k: Int): Double /* Returns the average precision for each query */ lazy val avePrec: RDD[Double] /*Returns the mean average precision (MAP) of all the queries*/ lazy val meanAvePrec: Double /*Returns the normalized discounted cumulative gain for each query */ lazy val ndcgAtK: RDD[Array[Double]] /** * param k the position to compute the truncated ndcg * return the average ndcg at the first k ranking positions */ def ndcg(k: Int): Double } ``` Author: coderxiang Closes #2667 from coderxiang/rankingmetrics and squashes the following commits: d881097 [coderxiang] update doc 14d9cd9 [coderxiang] remove unexpected files d7fb93f [coderxiang] style change and remove ignored files f113ee1 [coderxiang] modify doc for displaying superscript and subscript f626896 [coderxiang] improve doc and remove unnecessary computation while labSet is empty be6645e [coderxiang] set the precision of empty labset to 0.0 d64c120 [coderxiang] add logWarning for empty ground truth set dfae292 [coderxiang] handle empty labSet for map. add test 62047c4 [coderxiang] style change and add documentation f66612d [coderxiang] add additional test of precisionAt b794cb2 [coderxiang] move private members precAtK, ndcgAtK into public methods. style change 77c9e5d [coderxiang] set precAtK and ndcgAtK as private member. Improve documentation 5f87bce [coderxiang] add API to calculate precision and ndcg at each ranking position b7851cc [coderxiang] Use generic type to represent IDs e443fee [coderxiang] change style and use alternative builtin methods 3a5a6ff [coderxiang] add ranking metrics --- .../mllib/evaluation/RankingMetrics.scala | 152 ++++++++++++++++++ .../evaluation/RankingMetricsSuite.scala | 54 +++++++ 2 files changed, 206 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala new file mode 100644 index 0000000000000..93a7353e2c070 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -0,0 +1,152 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import scala.reflect.ClassTag + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental +import org.apache.spark.rdd.RDD + +/** + * ::Experimental:: + * Evaluator for ranking algorithms. + * + * @param predictionAndLabels an RDD of (predicted ranking, ground truth set) pairs. + */ +@Experimental +class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])]) + extends Logging with Serializable { + + /** + * Compute the average precision of all the queries, truncated at ranking position k. + * + * If for a query, the ranking algorithm returns n (n < k) results, the precision value will be + * computed as #(relevant items retrieved) / k. This formula also applies when the size of the + * ground truth set is less than k. + * + * If a query has an empty ground truth set, zero will be used as precision together with + * a log warning. + * + * See the following paper for detail: + * + * IR evaluation methods for retrieving highly relevant documents. K. Jarvelin and J. Kekalainen + * + * @param k the position to compute the truncated precision, must be positive + * @return the average precision at the first k ranking positions + */ + def precisionAt(k: Int): Double = { + require(k > 0, "ranking position k should be positive") + predictionAndLabels.map { case (pred, lab) => + val labSet = lab.toSet + + if (labSet.nonEmpty) { + val n = math.min(pred.length, k) + var i = 0 + var cnt = 0 + while (i < n) { + if (labSet.contains(pred(i))) { + cnt += 1 + } + i += 1 + } + cnt.toDouble / k + } else { + logWarning("Empty ground truth set, check input data") + 0.0 + } + }.mean + } + + /** + * Returns the mean average precision (MAP) of all the queries. + * If a query has an empty ground truth set, the average precision will be zero and a log + * warining is generated. + */ + lazy val meanAveragePrecision: Double = { + predictionAndLabels.map { case (pred, lab) => + val labSet = lab.toSet + + if (labSet.nonEmpty) { + var i = 0 + var cnt = 0 + var precSum = 0.0 + val n = pred.length + while (i < n) { + if (labSet.contains(pred(i))) { + cnt += 1 + precSum += cnt.toDouble / (i + 1) + } + i += 1 + } + precSum / labSet.size + } else { + logWarning("Empty ground truth set, check input data") + 0.0 + } + }.mean + } + + /** + * Compute the average NDCG value of all the queries, truncated at ranking position k. + * The discounted cumulative gain at position k is computed as: + * sum,,i=1,,^k^ (2^{relevance of ''i''th item}^ - 1) / log(i + 1), + * and the NDCG is obtained by dividing the DCG value on the ground truth set. In the current + * implementation, the relevance value is binary. + + * If a query has an empty ground truth set, zero will be used as ndcg together with + * a log warning. + * + * See the following paper for detail: + * + * IR evaluation methods for retrieving highly relevant documents. K. Jarvelin and J. Kekalainen + * + * @param k the position to compute the truncated ndcg, must be positive + * @return the average ndcg at the first k ranking positions + */ + def ndcgAt(k: Int): Double = { + require(k > 0, "ranking position k should be positive") + predictionAndLabels.map { case (pred, lab) => + val labSet = lab.toSet + + if (labSet.nonEmpty) { + val labSetSize = labSet.size + val n = math.min(math.max(pred.length, labSetSize), k) + var maxDcg = 0.0 + var dcg = 0.0 + var i = 0 + while (i < n) { + val gain = 1.0 / math.log(i + 2) + if (labSet.contains(pred(i))) { + dcg += gain + } + if (i < labSetSize) { + maxDcg += gain + } + i += 1 + } + dcg / maxDcg + } else { + logWarning("Empty ground truth set, check input data") + 0.0 + } + }.mean + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala new file mode 100644 index 0000000000000..a2d4bb41484b8 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.mllib.util.LocalSparkContext + +class RankingMetricsSuite extends FunSuite with LocalSparkContext { + test("Ranking metrics: map, ndcg") { + val predictionAndLabels = sc.parallelize( + Seq( + (Array[Int](1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Array[Int](1, 2, 3, 4, 5)), + (Array[Int](4, 1, 5, 6, 2, 7, 3, 8, 9, 10), Array[Int](1, 2, 3)), + (Array[Int](1, 2, 3, 4, 5), Array[Int]()) + ), 2) + val eps: Double = 1E-5 + + val metrics = new RankingMetrics(predictionAndLabels) + val map = metrics.meanAveragePrecision + + assert(metrics.precisionAt(1) ~== 1.0/3 absTol eps) + assert(metrics.precisionAt(2) ~== 1.0/3 absTol eps) + assert(metrics.precisionAt(3) ~== 1.0/3 absTol eps) + assert(metrics.precisionAt(4) ~== 0.75/3 absTol eps) + assert(metrics.precisionAt(5) ~== 0.8/3 absTol eps) + assert(metrics.precisionAt(10) ~== 0.8/3 absTol eps) + assert(metrics.precisionAt(15) ~== 8.0/45 absTol eps) + + assert(map ~== 0.355026 absTol eps) + + assert(metrics.ndcgAt(3) ~== 1.0/3 absTol eps) + assert(metrics.ndcgAt(5) ~== 0.328788 absTol eps) + assert(metrics.ndcgAt(10) ~== 0.487913 absTol eps) + assert(metrics.ndcgAt(15) ~== metrics.ndcgAt(10) absTol eps) + + } +} From 856b081729057f9da31a86e4bfa0dc0013492042 Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 21 Oct 2014 16:20:58 -0700 Subject: [PATCH 1039/1492] [SQL]redundant methods for broadcast redundant methods for broadcast in ```TableReader``` Author: wangfei Closes #2862 from scwf/TableReader and squashes the following commits: 414cc24 [wangfei] unnecessary methods for broadcast --- .../main/scala/org/apache/spark/sql/hive/TableReader.scala | 4 ---- 1 file changed, 4 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 0de29d5cffd0e..fd4f65e488259 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 @@ -67,10 +67,6 @@ class HadoopTableReader( private val _broadcastedHiveConf = sc.sparkContext.broadcast(new SerializableWritable(hiveExtraConf)) - def broadcastedHiveConf = _broadcastedHiveConf - - def hiveConf = _broadcastedHiveConf.value.value - override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] = makeRDDForTable( hiveTable, From 6bb56faea8d238ea22c2de33db93b1b39f492b3a Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 21 Oct 2014 21:53:09 -0700 Subject: [PATCH 1040/1492] SPARK-1813. Add a utility to SparkConf that makes using Kryo really easy Author: Sandy Ryza Closes #789 from sryza/sandy-spark-1813 and squashes the following commits: 48b05e9 [Sandy Ryza] Simplify b824932 [Sandy Ryza] Allow both spark.kryo.classesToRegister and spark.kryo.registrator at the same time 6a15bb7 [Sandy Ryza] Small fix a2278c0 [Sandy Ryza] Respond to review comments 6ef592e [Sandy Ryza] SPARK-1813. Add a utility to SparkConf that makes using Kryo really easy --- .../scala/org/apache/spark/SparkConf.scala | 17 ++++- .../spark/serializer/KryoSerializer.scala | 43 ++++++++----- .../java/org/apache/spark/JavaAPISuite.java | 12 ++++ .../org/apache/spark/SparkConfSuite.scala | 62 +++++++++++++++++++ .../serializer/KryoSerializerSuite.scala | 6 +- docs/configuration.md | 15 ++++- docs/tuning.md | 17 +---- .../spark/examples/bagel/PageRankUtils.scala | 17 ----- .../examples/bagel/WikipediaPageRank.scala | 3 +- .../spark/examples/graphx/Analytics.scala | 6 +- .../examples/graphx/SynthBenchmark.scala | 5 +- .../spark/examples/mllib/MovieLensALS.scala | 12 +--- .../spark/graphx/GraphKryoRegistrator.scala | 2 +- .../org/apache/spark/graphx/GraphXUtils.scala | 47 ++++++++++++++ .../spark/graphx/LocalSparkContext.scala | 6 +- .../graphx/impl/EdgePartitionSuite.scala | 6 +- .../graphx/impl/VertexPartitionSuite.scala | 6 +- 17 files changed, 195 insertions(+), 87 deletions(-) create mode 100644 graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 605df0e929faa..dbbcc23305c50 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -18,7 +18,8 @@ package org.apache.spark import scala.collection.JavaConverters._ -import scala.collection.mutable.HashMap +import scala.collection.mutable.{HashMap, LinkedHashSet} +import org.apache.spark.serializer.KryoSerializer /** * Configuration for a Spark application. Used to set various Spark parameters as key-value pairs. @@ -140,6 +141,20 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { this } + /** + * Use Kryo serialization and register the given set of classes with Kryo. + * If called multiple times, this will append the classes from all calls together. + */ + def registerKryoClasses(classes: Array[Class[_]]): SparkConf = { + val allClassNames = new LinkedHashSet[String]() + allClassNames ++= get("spark.kryo.classesToRegister", "").split(',').filter(!_.isEmpty) + allClassNames ++= classes.map(_.getName) + + set("spark.kryo.classesToRegister", allClassNames.mkString(",")) + set("spark.serializer", classOf[KryoSerializer].getName) + this + } + /** Remove a parameter from the configuration */ def remove(key: String): SparkConf = { settings.remove(key) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index d6386f8c06fff..621a951c27d07 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -53,7 +53,18 @@ class KryoSerializer(conf: SparkConf) private val maxBufferSize = conf.getInt("spark.kryoserializer.buffer.max.mb", 64) * 1024 * 1024 private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) - private val registrator = conf.getOption("spark.kryo.registrator") + private val userRegistrator = conf.getOption("spark.kryo.registrator") + private val classesToRegister = conf.get("spark.kryo.classesToRegister", "") + .split(',') + .filter(!_.isEmpty) + .map { className => + try { + Class.forName(className) + } catch { + case e: Exception => + throw new SparkException("Failed to load class to register with Kryo", e) + } + } def newKryoOutput() = new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) @@ -80,22 +91,20 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) kryo.register(classOf[HttpBroadcast[_]], new KryoJavaSerializer()) - // Allow the user to register their own classes by setting spark.kryo.registrator - for (regCls <- registrator) { - logDebug("Running user registrator: " + regCls) - try { - val reg = Class.forName(regCls, true, classLoader).newInstance() - .asInstanceOf[KryoRegistrator] - - // Use the default classloader when calling the user registrator. - Thread.currentThread.setContextClassLoader(classLoader) - reg.registerClasses(kryo) - } catch { - case e: Exception => - throw new SparkException(s"Failed to invoke $regCls", e) - } finally { - Thread.currentThread.setContextClassLoader(oldClassLoader) - } + try { + // Use the default classloader when calling the user registrator. + Thread.currentThread.setContextClassLoader(classLoader) + // Register classes given through spark.kryo.classesToRegister. + classesToRegister.foreach { clazz => kryo.register(clazz) } + // Allow the user to register their own classes by setting spark.kryo.registrator. + userRegistrator + .map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]) + .foreach { reg => reg.registerClasses(kryo) } + } catch { + case e: Exception => + throw new SparkException(s"Failed to register classes with Kryo", e) + } finally { + Thread.currentThread.setContextClassLoader(oldClassLoader) } // Register Chill's classes; we do this after our ranges and the user's own classes to let diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 3190148fb5f43..814e40c4f77cc 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1418,4 +1418,16 @@ public Optional call(Integer i) { } } + static class Class1 {} + static class Class2 {} + + @Test + public void testRegisterKryoClasses() { + SparkConf conf = new SparkConf(); + conf.registerKryoClasses(new Class[]{ Class1.class, Class2.class }); + Assert.assertEquals( + Class1.class.getName() + "," + Class2.class.getName(), + conf.get("spark.kryo.classesToRegister")); + } + } diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 87e9012622456..5d018ea9868a7 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark import org.scalatest.FunSuite +import org.apache.spark.serializer.{KryoRegistrator, KryoSerializer} +import com.esotericsoftware.kryo.Kryo class SparkConfSuite extends FunSuite with LocalSparkContext { test("loading from system properties") { @@ -133,4 +135,64 @@ class SparkConfSuite extends FunSuite with LocalSparkContext { System.clearProperty("spark.test.a.b.c") } } + + test("register kryo classes through registerKryoClasses") { + val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + + conf.registerKryoClasses(Array(classOf[Class1], classOf[Class2])) + assert(conf.get("spark.kryo.classesToRegister") === + classOf[Class1].getName + "," + classOf[Class2].getName) + + conf.registerKryoClasses(Array(classOf[Class3])) + assert(conf.get("spark.kryo.classesToRegister") === + classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + + conf.registerKryoClasses(Array(classOf[Class2])) + assert(conf.get("spark.kryo.classesToRegister") === + classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + + // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't + // blow up. + val serializer = new KryoSerializer(conf) + serializer.newInstance().serialize(new Class1()) + serializer.newInstance().serialize(new Class2()) + serializer.newInstance().serialize(new Class3()) + } + + test("register kryo classes through registerKryoClasses and custom registrator") { + val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + + conf.registerKryoClasses(Array(classOf[Class1])) + assert(conf.get("spark.kryo.classesToRegister") === classOf[Class1].getName) + + conf.set("spark.kryo.registrator", classOf[CustomRegistrator].getName) + + // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't + // blow up. + val serializer = new KryoSerializer(conf) + serializer.newInstance().serialize(new Class1()) + serializer.newInstance().serialize(new Class2()) + } + + test("register kryo classes through conf") { + val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + conf.set("spark.kryo.classesToRegister", "java.lang.StringBuffer") + conf.set("spark.serializer", classOf[KryoSerializer].getName) + + // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't + // blow up. + val serializer = new KryoSerializer(conf) + serializer.newInstance().serialize(new StringBuffer()) + } + +} + +class Class1 {} +class Class2 {} +class Class3 {} + +class CustomRegistrator extends KryoRegistrator { + def registerClasses(kryo: Kryo) { + kryo.register(classOf[Class2]) + } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index e1e35b688d581..64ac6d2d920d2 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -210,13 +210,13 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { } test("kryo with nonexistent custom registrator should fail") { - import org.apache.spark.{SparkConf, SparkException} + import org.apache.spark.SparkException val conf = new SparkConf(false) conf.set("spark.kryo.registrator", "this.class.does.not.exist") - + val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance()) - assert(thrown.getMessage.contains("Failed to invoke this.class.does.not.exist")) + assert(thrown.getMessage.contains("Failed to register classes with Kryo")) } test("default class loader can be set by a different thread") { diff --git a/docs/configuration.md b/docs/configuration.md index 96fa1377ec399..66738d3ca754e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -124,12 +124,23 @@ of the most common options to set are: org.apache.spark.Serializer.
    + + + + + + + + + + diff --git a/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java b/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java new file mode 100644 index 0000000000000..430e96ab14d9d --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples; + +import org.apache.spark.SparkConf; +import org.apache.spark.SparkJobInfo; +import org.apache.spark.SparkStageInfo; +import org.apache.spark.api.java.JavaFutureAction; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +import java.util.Arrays; +import java.util.List; + +/** + * Example of using Spark's status APIs from Java. + */ +public final class JavaStatusAPIDemo { + + public static final String APP_NAME = "JavaStatusAPIDemo"; + + public static final class IdentityWithDelay implements Function { + @Override + public T call(T x) throws Exception { + Thread.sleep(2 * 1000); // 2 seconds + return x; + } + } + + public static void main(String[] args) throws Exception { + SparkConf sparkConf = new SparkConf().setAppName(APP_NAME); + final JavaSparkContext sc = new JavaSparkContext(sparkConf); + + // Example of implementing a progress reporter for a simple job. + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 5).map( + new IdentityWithDelay()); + JavaFutureAction> jobFuture = rdd.collectAsync(); + while (!jobFuture.isDone()) { + Thread.sleep(1000); // 1 second + List jobIds = jobFuture.jobIds(); + if (jobIds.isEmpty()) { + continue; + } + int currentJobId = jobIds.get(jobIds.size() - 1); + SparkJobInfo jobInfo = sc.getJobInfo(currentJobId); + SparkStageInfo stageInfo = sc.getStageInfo(jobInfo.stageIds()[0]); + System.out.println(stageInfo.numTasks() + " tasks total: " + stageInfo.numActiveTasks() + + " active, " + stageInfo.numCompletedTasks() + " complete"); + } + + System.out.println("Job results are: " + jobFuture.get()); + sc.stop(); + } +} From e41786c77482d3f9e3c01cfd583c8899815c3106 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 25 Oct 2014 01:20:39 -0700 Subject: [PATCH 1073/1492] [SPARK-4088] [PySpark] Python worker should exit after socket is closed by JVM In case of take() or exception in Python, python worker may exit before JVM read() all the response, then the write thread may raise "Connection reset" exception. Python should always wait JVM to close the socket first. cc JoshRosen This is a warm fix, or the tests will be flaky, sorry for that. Author: Davies Liu Closes #2941 from davies/fix_exit and squashes the following commits: 9d4d21e [Davies Liu] fix race --- python/pyspark/daemon.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index dbb34775d9ac5..f09587f211708 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -62,8 +62,7 @@ def worker(sock): exit_code = compute_real_exit_code(exc.code) finally: outfile.flush() - if exit_code: - os._exit(exit_code) + return exit_code # Cleanup zombie children @@ -160,10 +159,13 @@ def handle_sigterm(*args): outfile.flush() outfile.close() while True: - worker(sock) - if not reuse: + code = worker(sock) + if not reuse or code: # wait for closing - while sock.recv(1024): + try: + while sock.recv(1024): + pass + except Exception: pass break gc.collect() From 2e52e4f815b3110912ecdb8ed18aa4b2209abcfb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Oct 2014 17:07:44 -0700 Subject: [PATCH 1074/1492] Revert "[SPARK-4056] Upgrade snappy-java to 1.1.1.5" This reverts commit 898b22ab1fe90e8a3935b19566465046f2256fa6. Reverting because this may be causing OOMs. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 030bea948b5ce..a1195262614dd 100644 --- a/pom.xml +++ b/pom.xml @@ -346,7 +346,7 @@ org.xerial.snappy snappy-java - 1.1.1.5 + 1.1.1.3 net.jpountz.lz4 From c6834440085b79f6d3e011f9e55ffd672be855fe Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 25 Oct 2014 20:07:44 -0700 Subject: [PATCH 1075/1492] [SPARK-4071] Unroll fails silently if BlockManager is small In tests, we may want to have BlockManagers of size < 1MB (spark.storage.unrollMemoryThreshold). However, these BlockManagers are useless because we can't unroll anything in them ever. At the very least we need to log a warning. tdas Author: Andrew Or Closes #2917 from andrewor14/unroll-safely-logging and squashes the following commits: 38947e3 [Andrew Or] Warn against starting a block manager that's too small fd621b4 [Andrew Or] Warn against failure to reserve initial memory threshold --- .../org/apache/spark/storage/MemoryStore.scala | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index edbc729c17ade..71305a46bf570 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -56,6 +56,16 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) (maxMemory * unrollFraction).toLong } + // Initial memory to request before unrolling any block + private val unrollMemoryThreshold: Long = + conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + + if (maxMemory < unrollMemoryThreshold) { + logWarning(s"Max memory ${Utils.bytesToString(maxMemory)} is less than the initial memory " + + s"threshold ${Utils.bytesToString(unrollMemoryThreshold)} needed to store a block in " + + s"memory. Please configure Spark with more memory.") + } + logInfo("MemoryStore started with capacity %s".format(Utils.bytesToString(maxMemory))) /** Free memory not occupied by existing blocks. Note that this does not include unroll memory. */ @@ -213,7 +223,7 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Whether there is still enough memory for us to continue unrolling this block var keepUnrolling = true // Initial per-thread memory to request for unrolling blocks (bytes). Exposed for testing. - val initialMemoryThreshold = conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + val initialMemoryThreshold = unrollMemoryThreshold // How often to check whether we need to request more memory val memoryCheckPeriod = 16 // Memory currently reserved by this thread for this particular unrolling operation @@ -228,6 +238,11 @@ private[spark] class MemoryStore(blockManager: BlockManager, maxMemory: Long) // Request enough memory to begin unrolling keepUnrolling = reserveUnrollMemoryForThisThread(initialMemoryThreshold) + if (!keepUnrolling) { + logWarning(s"Failed to reserve initial memory threshold of " + + s"${Utils.bytesToString(initialMemoryThreshold)} for computing block $blockId in memory.") + } + // Unroll this block safely, checking whether we have exceeded our threshold periodically try { while (values.hasNext && keepUnrolling) { From df7974b8e59d00e8efbb61629418fa6265c1ddab Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 25 Oct 2014 23:18:02 -0700 Subject: [PATCH 1076/1492] SPARK-3359 [DOCS] sbt/sbt unidoc doesn't work with Java 8 This follows https://github.com/apache/spark/pull/2893 , but does not completely fix SPARK-3359 either. This fixes minor scaladoc/javadoc issues that Javadoc 8 will treat as errors. Author: Sean Owen Closes #2909 from srowen/SPARK-3359 and squashes the following commits: f62c347 [Sean Owen] Fix some javadoc issues that javadoc 8 considers errors. This is not all of the errors turned up when javadoc 8 runs on output of genjavadoc. --- core/src/main/java/org/apache/spark/TaskContext.java | 2 -- .../org/apache/spark/api/java/function/PairFunction.java | 3 ++- .../scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 6 +++--- .../scala/org/apache/spark/api/java/JavaPairRDD.scala | 4 ++-- .../org/apache/spark/api/java/JavaSparkContext.scala | 5 ++++- .../scala/org/apache/spark/mllib/feature/Normalizer.scala | 2 +- .../apache/spark/mllib/linalg/distributed/RowMatrix.scala | 7 ++++--- .../main/scala/org/apache/spark/mllib/util/MLUtils.scala | 8 ++++---- .../org/apache/spark/sql/api/java/JavaSchemaRDD.scala | 2 +- 9 files changed, 21 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/apache/spark/TaskContext.java b/core/src/main/java/org/apache/spark/TaskContext.java index 2d998d4c7a5d9..0d6973203eba1 100644 --- a/core/src/main/java/org/apache/spark/TaskContext.java +++ b/core/src/main/java/org/apache/spark/TaskContext.java @@ -71,7 +71,6 @@ static void unset() { /** * Add a (Java friendly) listener to be executed on task completion. * This will be called in all situation - success, failure, or cancellation. - *

    * An example use is for HadoopRDD to register a callback to close the input stream. */ public abstract TaskContext addTaskCompletionListener(TaskCompletionListener listener); @@ -79,7 +78,6 @@ static void unset() { /** * Add a listener in the form of a Scala closure to be executed on task completion. * This will be called in all situations - success, failure, or cancellation. - *

    * An example use is for HadoopRDD to register a callback to close the input stream. */ public abstract TaskContext addTaskCompletionListener(final Function1 f); diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java index abd9bcc07ac61..99bf240a17225 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java @@ -22,7 +22,8 @@ import scala.Tuple2; /** - * A function that returns key-value pairs (Tuple2), and can be used to construct PairRDDs. + * A function that returns key-value pairs (Tuple2<K, V>), and can be used to + * construct PairRDDs. */ public interface PairFunction extends Serializable { public Tuple2 call(T t) throws Exception; diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index a6123bd108c11..8e8f7f6c4fda2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -114,7 +114,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja * Return an RDD with the elements from `this` that are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtract(other: JavaDoubleRDD): JavaDoubleRDD = fromRDD(srdd.subtract(other)) @@ -233,11 +233,11 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja * to the left except for the last which is closed * e.g. for the array * [1,10,20,50] the buckets are [1,10) [10,20) [20,50] - * e.g 1<=x<10 , 10<=x<20, 20<=x<50 + * e.g 1<=x<10 , 10<=x<20, 20<=x<50 * And on the input of 1 and 50 we would have a histogram of 1,0,0 * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched - * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets + * from an O(log n) insertion to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. * buckets array must be at least two elements diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index c38b96528d037..e37f3acaf6e30 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -392,7 +392,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return an RDD with the elements from `this` that are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtract(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] = fromRDD(rdd.subtract(other)) @@ -413,7 +413,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return an RDD with the pairs from `this` whose keys are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtractByKey[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, V] = { implicit val ctag: ClassTag[W] = fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 45168ba62d3c1..0565adf4d4ead 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -215,7 +215,10 @@ class JavaSparkContext(val sc: SparkContext) * hdfs://a-hdfs-path/part-nnnnn * }}} * - * Do `JavaPairRDD rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")`, + * Do + * {{{ + * JavaPairRDD rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path") + * }}} * *

    then `rdd` contains * {{{ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index 4734251127bb4..dfad25d57c947 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} * :: Experimental :: * Normalizes samples individually to unit L^p^ norm * - * For any 1 <= p < Double.PositiveInfinity, normalizes samples using + * For any 1 <= p < Double.PositiveInfinity, normalizes samples using * sum(abs(vector).^p^)^(1/p)^ as norm. * * For p = Double.PositiveInfinity, max(abs(vector)) will be used as norm for normalization. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index ec2d481dccc22..10a515af88802 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -152,7 +152,7 @@ class RowMatrix( * storing the right singular vectors, is computed via matrix multiplication as * U = A * (V * S^-1^), if requested by user. The actual method to use is determined * automatically based on the cost: - * - If n is small (n < 100) or k is large compared with n (k > n / 2), we compute the Gramian + * - If n is small (n < 100) or k is large compared with n (k > n / 2), we compute the Gramian * matrix first and then compute its top eigenvalues and eigenvectors locally on the driver. * This requires a single pass with O(n^2^) storage on each executor and on the driver, and * O(n^2^ k) time on the driver. @@ -169,7 +169,8 @@ class RowMatrix( * @note The conditions that decide which method to use internally and the default parameters are * subject to change. * - * @param k number of leading singular values to keep (0 < k <= n). It might return less than k if + * @param k number of leading singular values to keep (0 < k <= n). + * It might return less than k if * there are numerically zero singular values or there are not enough Ritz values * converged before the maximum number of Arnoldi update iterations is reached (in case * that matrix A is ill-conditioned). @@ -192,7 +193,7 @@ class RowMatrix( /** * The actual SVD implementation, visible for testing. * - * @param k number of leading singular values to keep (0 < k <= n) + * @param k number of leading singular values to keep (0 < k <= n) * @param computeU whether to compute U * @param rCond the reciprocal condition number * @param maxIter max number of iterations (if ARPACK is used) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index ca35100aa99c6..dce0adffa6249 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -196,8 +196,8 @@ object MLUtils { /** * Load labeled data from a file. The data format used here is - * , ... - * where , are feature values in Double and is the corresponding label as Double. + * L, f1 f2 ... + * where f1, f2 are feature values in Double and L is the corresponding label as Double. * * @param sc SparkContext * @param dir Directory to the input data files. @@ -219,8 +219,8 @@ object MLUtils { /** * Save labeled data to a file. The data format used here is - * , ... - * where , are feature values in Double and is the corresponding label as Double. + * L, f1 f2 ... + * where f1, f2 are feature values in Double and L is the corresponding label as Double. * * @param data An RDD of LabeledPoints containing data to be saved. * @param dir Directory to save the data. 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 e7faba0c7f620..1e0ccb368a276 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 @@ -193,7 +193,7 @@ class JavaSchemaRDD( * Return an RDD with the elements from `this` that are not in `other`. * * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting - * RDD will be <= us. + * RDD will be <= us. */ def subtract(other: JavaSchemaRDD): JavaSchemaRDD = this.baseSchemaRDD.subtract(other.baseSchemaRDD).toJavaSchemaRDD From b75954015fba8041ff25ab6d1a53b22b8f868526 Mon Sep 17 00:00:00 2001 From: Daniel Lemire Date: Sun, 26 Oct 2014 10:03:20 -0700 Subject: [PATCH 1077/1492] Update RoaringBitmap to 0.4.3 Roaring has been updated to version 0.4.3. We fixed a rarely occurring bug with serialization. No API or format changes were made. Author: Daniel Lemire Closes #2938 from lemire/master and squashes the following commits: 431f3a0 [Daniel Lemire] Recommended bug fix release --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a1195262614dd..2faf0c7dcffd3 100644 --- a/pom.xml +++ b/pom.xml @@ -445,7 +445,7 @@ org.roaringbitmap RoaringBitmap - 0.4.1 + 0.4.3 commons-net From bf589fc717c842d1998e3c3a523bc8775cb30269 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 26 Oct 2014 11:29:27 -0700 Subject: [PATCH 1078/1492] [SPARK-3616] Add basic Selenium tests to WebUISuite This patch adds Selenium tests for Spark's web UI. To avoid adding extra dependencies to the test environment, the tests use Selenium's HtmlUnitDriver, which is pure-Java, instead of, say, ChromeDriver. I added new tests to try to reproduce a few UI bugs reported on JIRA, namely SPARK-3021, SPARK-2105, and SPARK-2527. I wasn't able to reproduce these bugs; I suspect that the older ones might have been fixed by other patches. In order to use HtmlUnitDriver, I added an explicit dependency on the org.apache.httpcomponents version of httpclient in order to prevent jets3t's older version from taking precedence on the classpath. I also upgraded ScalaTest to 2.2.1. Author: Josh Rosen Author: Josh Rosen Closes #2474 from JoshRosen/webui-selenium-tests and squashes the following commits: fcc9e83 [Josh Rosen] scalautils -> scalactic package rename 510e54a [Josh Rosen] [SPARK-3616] Add basic Selenium tests to WebUISuite. --- core/pom.xml | 5 + .../scala/org/apache/spark/ui/UIUtils.scala | 7 +- .../scala/org/apache/spark/ui/WebUI.scala | 2 +- .../org/apache/spark/ui/storage/RDDPage.scala | 6 +- .../apache/spark/ui/storage/StoragePage.scala | 2 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 112 ++++++++++++++++++ pom.xml | 8 +- .../ExpressionEvaluationSuite.scala | 2 +- 8 files changed, 135 insertions(+), 9 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala diff --git a/core/pom.xml b/core/pom.xml index 7b68dbaea4789..320d1076f7c03 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -247,6 +247,11 @@ + + org.seleniumhq.selenium + selenium-java + test + org.scalatest scalatest_${scala.binary.version} 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 32e6b15bb0999..76714b1e6964f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.{Locale, Date} -import scala.xml.Node +import scala.xml.{Text, Node} import org.apache.spark.Logging @@ -239,7 +239,8 @@ private[spark] object UIUtils extends Logging { headers: Seq[String], generateDataRow: T => Seq[Node], data: Iterable[T], - fixedWidth: Boolean = false): Seq[Node] = { + fixedWidth: Boolean = false, + id: Option[String] = None): Seq[Node] = { var listingTableClass = TABLE_CLASS if (fixedWidth) { @@ -263,7 +264,7 @@ private[spark] object UIUtils extends Logging { } } } -

    Output OperationMeaning
    print() print() Prints first ten elements of every batch of data in a DStream on the driver. - This is useful for development and debugging.
    saveAsObjectFiles(prefix, [suffix])
    spark.hadoop.cloneConffalseIf set to true, clones a new Hadoop Configuration object for each task. This + option should be enabled to work around Configuration thread-safety issues (see + SPARK-2546 for more details). + This is disabled by default in order to avoid unexpected performance regressions for jobs that + are not affected by these issues.
    spark.executor.heartbeatInterval 10000
    spark.kryo.classesToRegister(none) + If you use Kryo serialization, give a comma-separated list of custom class names to register + with Kryo. + See the tuning guide for more details. +
    spark.kryo.registrator (none) - If you use Kryo serialization, set this class to register your custom classes with Kryo. - It should be set to a class that extends + If you use Kryo serialization, set this class to register your custom classes with Kryo. This + property is useful if you need to register your classes in a custom way, e.g. to specify a custom + field serializer. Otherwise spark.kryo.classesToRegister is simpler. It should be + set to a class that extends KryoRegistrator. See the tuning guide for more details. diff --git a/docs/tuning.md b/docs/tuning.md index 8fb2a0433b1a8..9b5c9adac6a4f 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -47,24 +47,11 @@ registration requirement, but we recommend trying it in any network-intensive ap Spark automatically includes Kryo serializers for the many commonly-used core Scala classes covered in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. -To register your own custom classes with Kryo, create a public class that extends -[`org.apache.spark.serializer.KryoRegistrator`](api/scala/index.html#org.apache.spark.serializer.KryoRegistrator) and set the -`spark.kryo.registrator` config property to point to it, as follows: +To register your own custom classes with Kryo, use the `registerKryoClasses` method. {% highlight scala %} -import com.esotericsoftware.kryo.Kryo -import org.apache.spark.serializer.KryoRegistrator - -class MyRegistrator extends KryoRegistrator { - override def registerClasses(kryo: Kryo) { - kryo.register(classOf[MyClass1]) - kryo.register(classOf[MyClass2]) - } -} - val conf = new SparkConf().setMaster(...).setAppName(...) -conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") -conf.set("spark.kryo.registrator", "mypackage.MyRegistrator") +conf.registerKryoClasses(Seq(classOf[MyClass1], classOf[MyClass2])) val sc = new SparkContext(conf) {% endhighlight %} diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala index e06f4dcd54442..e322d4ce5a745 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/PageRankUtils.scala @@ -18,17 +18,7 @@ package org.apache.spark.examples.bagel import org.apache.spark._ -import org.apache.spark.SparkContext._ -import org.apache.spark.serializer.KryoRegistrator - import org.apache.spark.bagel._ -import org.apache.spark.bagel.Bagel._ - -import scala.collection.mutable.ArrayBuffer - -import java.io.{InputStream, OutputStream, DataInputStream, DataOutputStream} - -import com.esotericsoftware.kryo._ class PageRankUtils extends Serializable { def computeWithCombiner(numVertices: Long, epsilon: Double)( @@ -99,13 +89,6 @@ class PRMessage() extends Message[String] with Serializable { } } -class PRKryoRegistrator extends KryoRegistrator { - def registerClasses(kryo: Kryo) { - kryo.register(classOf[PRVertex]) - kryo.register(classOf[PRMessage]) - } -} - class CustomPartitioner(partitions: Int) extends Partitioner { def numPartitions = partitions diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index e4db3ec51313d..859abedf2a55e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -38,8 +38,7 @@ object WikipediaPageRank { } val sparkConf = new SparkConf() sparkConf.setAppName("WikipediaPageRank") - sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) + sparkConf.registerKryoClasses(Array(classOf[PRVertex], classOf[PRMessage])) val inputFile = args(0) val threshold = args(1).toDouble diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 45527d9382fd0..d70d93608a57c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -46,10 +46,8 @@ object Analytics extends Logging { } val options = mutable.Map(optionsList: _*) - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") - .set("spark.locality.wait", "100000") + val conf = new SparkConf().set("spark.locality.wait", "100000") + GraphXUtils.registerKryoClasses(conf) val numEPart = options.remove("numEPart").map(_.toInt).getOrElse { println("Set the number of edge partitions using --numEPart.") diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 5f35a5836462e..05676021718d9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -18,7 +18,7 @@ package org.apache.spark.examples.graphx import org.apache.spark.SparkContext._ -import org.apache.spark.graphx.PartitionStrategy +import org.apache.spark.graphx.{GraphXUtils, PartitionStrategy} import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.graphx.util.GraphGenerators import java.io.{PrintWriter, FileOutputStream} @@ -80,8 +80,7 @@ object SynthBenchmark { val conf = new SparkConf() .setAppName(s"GraphX Synth Benchmark (nverts = $numVertices, app = $app)") - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + GraphXUtils.registerKryoClasses(conf) val sc = new SparkContext(conf) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index fc6678013b932..8796c28db8a66 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -19,7 +19,6 @@ package org.apache.spark.examples.mllib import scala.collection.mutable -import com.esotericsoftware.kryo.Kryo import org.apache.log4j.{Level, Logger} import scopt.OptionParser @@ -27,7 +26,6 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.recommendation.{ALS, MatrixFactorizationModel, Rating} import org.apache.spark.rdd.RDD -import org.apache.spark.serializer.{KryoSerializer, KryoRegistrator} /** * An example app for ALS on MovieLens data (http://grouplens.org/datasets/movielens/). @@ -40,13 +38,6 @@ import org.apache.spark.serializer.{KryoSerializer, KryoRegistrator} */ object MovieLensALS { - class ALSRegistrator extends KryoRegistrator { - override def registerClasses(kryo: Kryo) { - kryo.register(classOf[Rating]) - kryo.register(classOf[mutable.BitSet]) - } - } - case class Params( input: String = null, kryo: Boolean = false, @@ -108,8 +99,7 @@ object MovieLensALS { def run(params: Params) { val conf = new SparkConf().setAppName(s"MovieLensALS with $params") if (params.kryo) { - conf.set("spark.serializer", classOf[KryoSerializer].getName) - .set("spark.kryo.registrator", classOf[ALSRegistrator].getName) + conf.registerKryoClasses(Array(classOf[mutable.BitSet], classOf[Rating])) .set("spark.kryoserializer.buffer.mb", "8") } val sc = new SparkContext(conf) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala index 1948c978c30bf..563c948957ecf 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphKryoRegistrator.scala @@ -27,10 +27,10 @@ import org.apache.spark.graphx.impl._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap import org.apache.spark.util.collection.OpenHashSet - /** * Registers GraphX classes with Kryo for improved performance. */ +@deprecated("Register GraphX classes with Kryo using GraphXUtils.registerKryoClasses", "1.2.0") class GraphKryoRegistrator extends KryoRegistrator { def registerClasses(kryo: Kryo) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala new file mode 100644 index 0000000000000..2cb07937eaa2a --- /dev/null +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphXUtils.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.graphx + +import org.apache.spark.SparkConf + +import org.apache.spark.graphx.impl._ +import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap + +import org.apache.spark.util.collection.{OpenHashSet, BitSet} +import org.apache.spark.util.BoundedPriorityQueue + +object GraphXUtils { + /** + * Registers classes that GraphX uses with Kryo. + */ + def registerKryoClasses(conf: SparkConf) { + conf.registerKryoClasses(Array( + classOf[Edge[Object]], + classOf[(VertexId, Object)], + classOf[EdgePartition[Object, Object]], + classOf[BitSet], + classOf[VertexIdToIndexMap], + classOf[VertexAttributeBlock[Object]], + classOf[PartitionStrategy], + classOf[BoundedPriorityQueue[Object]], + classOf[EdgeDirection], + classOf[GraphXPrimitiveKeyOpenHashMap[VertexId, Int]], + classOf[OpenHashSet[Int]], + classOf[OpenHashSet[Long]])) + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index 47594a800a3b1..a3e28efc75a98 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -17,9 +17,6 @@ package org.apache.spark.graphx -import org.scalatest.Suite -import org.scalatest.BeforeAndAfterEach - import org.apache.spark.SparkConf import org.apache.spark.SparkContext @@ -31,8 +28,7 @@ trait LocalSparkContext { /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ def withSpark[T](f: SparkContext => T) = { val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator") + GraphXUtils.registerKryoClasses(conf) val sc = new SparkContext("local", "test", conf) try { f(sc) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala index 9d00f76327e4c..db1dac6160080 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgePartitionSuite.scala @@ -129,9 +129,9 @@ class EdgePartitionSuite extends FunSuite { val aList = List((0, 1, 0), (1, 0, 0), (1, 2, 0), (5, 4, 0), (5, 5, 0)) val a: EdgePartition[Int, Int] = makeEdgePartition(aList) val javaSer = new JavaSerializer(new SparkConf()) - val kryoSer = new KryoSerializer(new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator")) + val conf = new SparkConf() + GraphXUtils.registerKryoClasses(conf) + val kryoSer = new KryoSerializer(conf) for (ser <- List(javaSer, kryoSer); s = ser.newInstance()) { val aSer: EdgePartition[Int, Int] = s.deserialize(s.serialize(a)) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala index f9e771a900013..fe8304c1cdc32 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/VertexPartitionSuite.scala @@ -125,9 +125,9 @@ class VertexPartitionSuite extends FunSuite { val verts = Set((0L, 1), (1L, 1), (2L, 1)) val vp = VertexPartition(verts.iterator) val javaSer = new JavaSerializer(new SparkConf()) - val kryoSer = new KryoSerializer(new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", "org.apache.spark.graphx.GraphKryoRegistrator")) + val conf = new SparkConf() + GraphXUtils.registerKryoClasses(conf) + val kryoSer = new KryoSerializer(conf) for (ser <- List(javaSer, kryoSer); s = ser.newInstance()) { val vpSer: VertexPartition[Int] = s.deserialize(s.serialize(vp)) From bae4ca3bbf75cf5b57a098a95848b0fd65bc241e Mon Sep 17 00:00:00 2001 From: Karthik Date: Wed, 22 Oct 2014 00:08:53 -0700 Subject: [PATCH 1041/1492] Update JavaCustomReceiver.java Changed the usage string to correctly reflect the file name. Author: Karthik Closes #2699 from namelessnerd/patch-1 and squashes the following commits: 8570e33 [Karthik] Update JavaCustomReceiver.java --- .../org/apache/spark/examples/streaming/JavaCustomReceiver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java index 5622df5ce03ff..981bc4f0613a9 100644 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java +++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java @@ -57,7 +57,7 @@ public class JavaCustomReceiver extends Receiver { public static void main(String[] args) { if (args.length < 2) { - System.err.println("Usage: JavaNetworkWordCount "); + System.err.println("Usage: JavaCustomReceiver "); System.exit(1); } From f05e09b4c95d799bdda3c3ff7fb76a4cd656415d Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Wed, 22 Oct 2014 00:51:33 -0700 Subject: [PATCH 1042/1492] use isRunningLocally rather than runningLocally runningLocally is deprecated now Author: CrazyJvm Closes #2879 from CrazyJvm/runningLocally and squashes the following commits: bec0b3e [CrazyJvm] use isRunningLocally rather than runningLocally --- core/src/main/scala/org/apache/spark/CacheManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index d89bb50076c9a..80da62c44edc5 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -61,7 +61,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val computedValues = rdd.computeOrReadCheckpoint(partition, context) // If the task is running locally, do not persist the result - if (context.runningLocally) { + if (context.isRunningLocally) { return computedValues } From 97cf19f64e924569892e0a0417de19329855f4af Mon Sep 17 00:00:00 2001 From: freeman Date: Wed, 22 Oct 2014 09:33:12 -0700 Subject: [PATCH 1043/1492] Fix for sampling error in NumPy v1.9 [SPARK-3995][PYSPARK] Change maximum value for default seed during RDD sampling so that it is strictly less than 2 ** 32. This prevents a bug in the most recent version of NumPy, which cannot accept random seeds above this bound. Adds an extra test that uses the default seed (instead of setting it manually, as in the docstrings). mengxr Author: freeman Closes #2889 from freeman-lab/pyspark-sampling and squashes the following commits: dc385ef [freeman] Change maximum value for default seed --- python/pyspark/rddsampler.py | 4 ++-- python/pyspark/tests.py | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 55e247da0e4dc..528a181e8905a 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -31,7 +31,7 @@ def __init__(self, withReplacement, seed=None): "Falling back to default random generator for sampling.") self._use_numpy = False - self._seed = seed if seed is not None else random.randint(0, sys.maxint) + self._seed = seed if seed is not None else random.randint(0, 2 ** 32 - 1) self._withReplacement = withReplacement self._random = None self._split = None @@ -47,7 +47,7 @@ def initRandomGenerator(self, split): for _ in range(0, split): # discard the next few values in the sequence to have a # different seed for the different splits - self._random.randint(0, sys.maxint) + self._random.randint(0, 2 ** 32 - 1) self._split = split self._rand_initialized = True diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index f5ccf31abb3fa..1a8e4150e63c3 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -433,6 +433,12 @@ def test_deleting_input_files(self): os.unlink(tempFile.name) self.assertRaises(Exception, lambda: filtered_data.count()) + def test_sampling_default_seed(self): + # Test for SPARK-3995 (default seed setting) + data = self.sc.parallelize(range(1000), 1) + subset = data.takeSample(False, 10) + self.assertEqual(len(subset), 10) + def testAggregateByKey(self): data = self.sc.parallelize([(1, 1), (1, 1), (3, 2), (5, 1), (5, 3)], 2) From 813effc701fc27121c6f23ab32882932016fdbe0 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 22 Oct 2014 14:49:58 -0700 Subject: [PATCH 1044/1492] [SPARK-3426] Fix sort-based shuffle error when spark.shuffle.compress and spark.shuffle.spill.compress settings are different This PR fixes SPARK-3426, an issue where sort-based shuffle crashes if the `spark.shuffle.spill.compress` and `spark.shuffle.compress` settings have different values. The problem is that sort-based shuffle's read and write paths use different settings for determining whether to apply compression. ExternalSorter writes runs to files using `TempBlockId` ids, which causes `spark.shuffle.spill.compress` to be used for enabling compression, but these spilled files end up being shuffled over the network and read as shuffle files using `ShuffleBlockId` by BlockStoreShuffleFetcher, which causes `spark.shuffle.compress` to be used for enabling decompression. As a result, this leads to errors when these settings disagree. Based on the discussions in #2247 and #2178, it sounds like we don't want to remove the `spark.shuffle.spill.compress` setting. Therefore, I've tried to come up with a fix where `spark.shuffle.spill.compress` is used to compress data that's read and written locally and `spark.shuffle.compress` is used to compress any data that will be fetched / read as shuffle blocks. To do this, I split `TempBlockId` into two new id types, `TempLocalBlockId` and `TempShuffleBlockId`, which map to `spark.shuffle.spill.compress` and `spark.shuffle.compress`, respectively. ExternalAppendOnlyMap also used temp blocks for spilling data. It looks like ExternalSorter was designed to be a generic sorter but its configuration already happens to be tied to sort-based shuffle, so I think it's fine if we use `spark.shuffle.compress` to compress its spills; we can move the compression configuration to the constructor in a later commit if we find that ExternalSorter is being used in other contexts where we want different configuration options to control compression. To summarize: **Before:** | | ExternalAppendOnlyMap | ExternalSorter | |-------|------------------------------|------------------------------| | Read | spark.shuffle.spill.compress | spark.shuffle.compress | | Write | spark.shuffle.spill.compress | spark.shuffle.spill.compress | **After:** | | ExternalAppendOnlyMap | ExternalSorter | |-------|------------------------------|------------------------| | Read | spark.shuffle.spill.compress | spark.shuffle.compress | | Write | spark.shuffle.spill.compress | spark.shuffle.compress | Thanks to andrewor14 for debugging this with me! Author: Josh Rosen Closes #2890 from JoshRosen/SPARK-3426 and squashes the following commits: 1921cf6 [Josh Rosen] Minor edit for clarity. c8dd8f2 [Josh Rosen] Add comment explaining use of createTempShuffleBlock(). 2c687b9 [Josh Rosen] Fix SPARK-3426. 91e7e40 [Josh Rosen] Combine tests into single test of all combinations 76ca65e [Josh Rosen] Add regression test for SPARK-3426. --- .../org/apache/spark/storage/BlockId.scala | 11 ++++++--- .../apache/spark/storage/BlockManager.scala | 3 ++- .../spark/storage/DiskBlockManager.scala | 17 +++++++++---- .../collection/ExternalAppendOnlyMap.scala | 2 +- .../util/collection/ExternalSorter.scala | 15 ++++++++++-- .../scala/org/apache/spark/ShuffleSuite.scala | 24 +++++++++++++++++++ 6 files changed, 61 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index a83a3f468ae5f..8df5ec6bde184 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -83,9 +83,14 @@ case class StreamBlockId(streamId: Int, uniqueId: Long) extends BlockId { def name = "input-" + streamId + "-" + uniqueId } -/** Id associated with temporary data managed as blocks. Not serializable. */ -private[spark] case class TempBlockId(id: UUID) extends BlockId { - def name = "temp_" + id +/** Id associated with temporary local data managed as blocks. Not serializable. */ +private[spark] case class TempLocalBlockId(id: UUID) extends BlockId { + def name = "temp_local_" + id +} + +/** Id associated with temporary shuffle data managed as blocks. Not serializable. */ +private[spark] case class TempShuffleBlockId(id: UUID) extends BlockId { + def name = "temp_shuffle_" + id } // Intended only for testing purposes diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0ce2a3f631b15..4cc97923658bc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1071,7 +1071,8 @@ private[spark] class BlockManager( case _: ShuffleBlockId => compressShuffle case _: BroadcastBlockId => compressBroadcast case _: RDDBlockId => compressRdds - case _: TempBlockId => compressShuffleSpill + case _: TempLocalBlockId => compressShuffleSpill + case _: TempShuffleBlockId => compressShuffle case _ => false } } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index a715594f198c2..6633a1db57e59 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -98,11 +98,20 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon getAllFiles().map(f => BlockId(f.getName)) } - /** Produces a unique block id and File suitable for intermediate results. */ - def createTempBlock(): (TempBlockId, File) = { - var blockId = new TempBlockId(UUID.randomUUID()) + /** Produces a unique block id and File suitable for storing local intermediate results. */ + def createTempLocalBlock(): (TempLocalBlockId, File) = { + var blockId = new TempLocalBlockId(UUID.randomUUID()) while (getFile(blockId).exists()) { - blockId = new TempBlockId(UUID.randomUUID()) + blockId = new TempLocalBlockId(UUID.randomUUID()) + } + (blockId, getFile(blockId)) + } + + /** Produces a unique block id and File suitable for storing shuffled intermediate results. */ + def createTempShuffleBlock(): (TempShuffleBlockId, File) = { + var blockId = new TempShuffleBlockId(UUID.randomUUID()) + while (getFile(blockId).exists()) { + blockId = new TempShuffleBlockId(UUID.randomUUID()) } (blockId, getFile(blockId)) } diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 0c088da46aa5e..26fa0cb6d7bde 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -153,7 +153,7 @@ class ExternalAppendOnlyMap[K, V, C]( * Sort the existing contents of the in-memory map and spill them to a temporary file on disk. */ override protected[this] def spill(collection: SizeTracker): Unit = { - val (blockId, file) = diskBlockManager.createTempBlock() + val (blockId, file) = diskBlockManager.createTempLocalBlock() curWriteMetrics = new ShuffleWriteMetrics() var writer = blockManager.getDiskWriter(blockId, file, serializer, fileBufferSize, curWriteMetrics) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index d1b06d14acbd2..c1ce13683b569 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -38,6 +38,11 @@ import org.apache.spark.storage.{BlockObjectWriter, BlockId} * * If combining is disabled, the type C must equal V -- we'll cast the objects at the end. * + * Note: Although ExternalSorter is a fairly generic sorter, some of its configuration is tied + * to its use in sort-based shuffle (for example, its block compression is controlled by + * `spark.shuffle.compress`). We may need to revisit this if ExternalSorter is used in other + * non-shuffle contexts where we might want to use different configuration settings. + * * @param aggregator optional Aggregator with combine functions to use for merging data * @param partitioner optional Partitioner; if given, sort by partition ID and then key * @param ordering optional Ordering to sort keys within each partition; should be a total ordering @@ -259,7 +264,10 @@ private[spark] class ExternalSorter[K, V, C]( private def spillToMergeableFile(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { assert(!bypassMergeSort) - val (blockId, file) = diskBlockManager.createTempBlock() + // Because these files may be read during shuffle, their compression must be controlled by + // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use + // createTempShuffleBlock here; see SPARK-3426 for more context. + val (blockId, file) = diskBlockManager.createTempShuffleBlock() curWriteMetrics = new ShuffleWriteMetrics() var writer = blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics) var objectsWritten = 0 // Objects written since the last flush @@ -338,7 +346,10 @@ private[spark] class ExternalSorter[K, V, C]( if (partitionWriters == null) { curWriteMetrics = new ShuffleWriteMetrics() partitionWriters = Array.fill(numPartitions) { - val (blockId, file) = diskBlockManager.createTempBlock() + // Because these files may be read during shuffle, their compression must be controlled by + // spark.shuffle.compress instead of spark.shuffle.spill.compress, so we need to use + // createTempShuffleBlock here; see SPARK-3426 for more context. + val (blockId, file) = diskBlockManager.createTempShuffleBlock() blockManager.getDiskWriter(blockId, file, ser, fileBufferSize, curWriteMetrics).open() } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 15aa4d83800fa..2bdd84ce69ab8 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -242,6 +242,30 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.toLowerCase.contains("serializable")) } + + test("shuffle with different compression settings (SPARK-3426)") { + for ( + shuffleSpillCompress <- Set(true, false); + shuffleCompress <- Set(true, false) + ) { + val conf = new SparkConf() + .setAppName("test") + .setMaster("local") + .set("spark.shuffle.spill.compress", shuffleSpillCompress.toString) + .set("spark.shuffle.compress", shuffleCompress.toString) + .set("spark.shuffle.memoryFraction", "0.001") + resetSparkContext() + sc = new SparkContext(conf) + try { + sc.parallelize(0 until 100000).map(i => (i / 4, i)).groupByKey().collect() + } catch { + case e: Exception => + val errMsg = s"Failed with spark.shuffle.spill.compress=$shuffleSpillCompress," + + s" spark.shuffle.compress=$shuffleCompress" + throw new Exception(errMsg, e) + } + } + } } object ShuffleSuite { From 137d94235383cc49ccf8a7bb7f314f578aa1dede Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 22 Oct 2014 15:04:41 -0700 Subject: [PATCH 1045/1492] [SPARK-3877][YARN] Throw an exception when application is not successful so that the exit code wil be set to 1 When an yarn application fails (yarn-cluster mode), the exit code of spark-submit is still 0. It's hard for people to write some automatic scripts to run spark jobs in yarn because the failure can not be detected in these scripts. This PR added a status checking after `monitorApplication`. If an application is not successful, `run()` will throw an `SparkException`, so that Client.scala will exit with code 1. Therefore, people can use the exit code of `spark-submit` to write some automatic scripts. Author: zsxwing Closes #2732 from zsxwing/SPARK-3877 and squashes the following commits: 1f89fa5 [zsxwing] Fix the unit test a0498e1 [zsxwing] Update the docs and the error message e1cb9ef [zsxwing] Fix the hacky way of calling Client ff16fec [zsxwing] Remove System.exit in Client.scala and add a test 6a2c103 [zsxwing] [SPARK-3877] Throw an exception when application is not successful so that the exit code wil be set to 1 --- .../org/apache/spark/deploy/yarn/Client.scala | 12 ++------ .../apache/spark/deploy/yarn/ClientBase.scala | 29 +++++++++++++++---- .../cluster/YarnClientSchedulerBackend.scala | 2 +- .../org/apache/spark/deploy/yarn/Client.scala | 12 ++------ .../spark/deploy/yarn/YarnClusterSuite.scala | 24 ++++++++++----- 5 files changed, 44 insertions(+), 35 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 5c7bca4541222..9c66c785848a5 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -137,15 +137,7 @@ object Client { System.setProperty("SPARK_YARN_MODE", "true") val sparkConf = new SparkConf - try { - val args = new ClientArguments(argStrings, sparkConf) - new Client(args, sparkConf).run() - } catch { - case e: Exception => - Console.err.println(e.getMessage) - System.exit(1) - } - - System.exit(0) + val args = new ClientArguments(argStrings, sparkConf) + new Client(args, sparkConf).run() } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 0efac4ea63702..fb0e34bf5985e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -417,17 +417,19 @@ private[spark] trait ClientBase extends Logging { /** * Report the state of an application until it has exited, either successfully or - * due to some failure, then return the application state. + * due to some failure, then return a pair of the yarn application state (FINISHED, FAILED, + * KILLED, or RUNNING) and the final application state (UNDEFINED, SUCCEEDED, FAILED, + * or KILLED). * * @param appId ID of the application to monitor. * @param returnOnRunning Whether to also return the application state when it is RUNNING. * @param logApplicationReport Whether to log details of the application report every iteration. - * @return state of the application, one of FINISHED, FAILED, KILLED, and RUNNING. + * @return A pair of the yarn application state and the final application state. */ def monitorApplication( appId: ApplicationId, returnOnRunning: Boolean = false, - logApplicationReport: Boolean = true): YarnApplicationState = { + logApplicationReport: Boolean = true): (YarnApplicationState, FinalApplicationStatus) = { val interval = sparkConf.getLong("spark.yarn.report.interval", 1000) var lastState: YarnApplicationState = null while (true) { @@ -468,11 +470,11 @@ private[spark] trait ClientBase extends Logging { if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { - return state + return (state, report.getFinalApplicationStatus) } if (returnOnRunning && state == YarnApplicationState.RUNNING) { - return state + return (state, report.getFinalApplicationStatus) } lastState = state @@ -485,8 +487,23 @@ private[spark] trait ClientBase extends Logging { /** * Submit an application to the ResourceManager and monitor its state. * This continues until the application has exited for any reason. + * If the application finishes with a failed, killed, or undefined status, + * throw an appropriate SparkException. */ - def run(): Unit = monitorApplication(submitApplication()) + def run(): Unit = { + val (yarnApplicationState, finalApplicationStatus) = monitorApplication(submitApplication()) + if (yarnApplicationState == YarnApplicationState.FAILED || + finalApplicationStatus == FinalApplicationStatus.FAILED) { + throw new SparkException("Application finished with failed status") + } + if (yarnApplicationState == YarnApplicationState.KILLED || + finalApplicationStatus == FinalApplicationStatus.KILLED) { + throw new SparkException("Application is killed") + } + if (finalApplicationStatus == FinalApplicationStatus.UNDEFINED) { + throw new SparkException("The final status of application is undefined") + } + } /* --------------------------------------------------------------------------------------- * | Methods that cannot be implemented here due to API differences across hadoop versions | 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 6bb4b82316ad4..d948a2aeedd45 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 @@ -99,7 +99,7 @@ private[spark] class YarnClientSchedulerBackend( */ private def waitForApplication(): Unit = { assert(client != null && appId != null, "Application has not been submitted yet!") - val state = client.monitorApplication(appId, returnOnRunning = true) // blocking + val (state, _) = client.monitorApplication(appId, returnOnRunning = true) // blocking if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 0b43e6ee20538..addaddb711d3c 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -135,15 +135,7 @@ object Client { System.setProperty("SPARK_YARN_MODE", "true") val sparkConf = new SparkConf - try { - val args = new ClientArguments(argStrings, sparkConf) - new Client(args, sparkConf).run() - } catch { - case e: Exception => - Console.err.println(e.getMessage) - System.exit(1) - } - - System.exit(0) + val args = new ClientArguments(argStrings, sparkConf) + new Client(args, sparkConf).run() } } diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index a826b2a78a8f5..d79b85e867fcd 100644 --- a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -29,7 +29,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.MiniYARNCluster -import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -123,21 +123,29 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit val main = YarnClusterDriver.getClass.getName().stripSuffix("$") var result = File.createTempFile("result", null, tempDir) - // The Client object will call System.exit() after the job is done, and we don't want - // that because it messes up the scalatest monitoring. So replicate some of what main() - // does here. val args = Array("--class", main, "--jar", "file:" + fakeSparkJar.getAbsolutePath(), "--arg", "yarn-cluster", "--arg", result.getAbsolutePath(), "--num-executors", "1") - val sparkConf = new SparkConf() - val yarnConf = SparkHadoopUtil.get.newConfiguration(sparkConf) - val clientArgs = new ClientArguments(args, sparkConf) - new Client(clientArgs, yarnConf, sparkConf).run() + Client.main(args) checkResult(result) } + test("run Spark in yarn-cluster mode unsuccessfully") { + val main = YarnClusterDriver.getClass.getName().stripSuffix("$") + + // Use only one argument so the driver will fail + val args = Array("--class", main, + "--jar", "file:" + fakeSparkJar.getAbsolutePath(), + "--arg", "yarn-cluster", + "--num-executors", "1") + val exception = intercept[SparkException] { + Client.main(args) + } + assert(Utils.exceptionString(exception).contains("Application finished with failed status")) + } + /** * This is a workaround for an issue with yarn-cluster mode: the Client class will not provide * any sort of error when the job process finishes successfully, but the job itself fails. So From c5882c663e054adcd3ecd9f11e91a1929dbc14a3 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 22 Oct 2014 19:44:00 -0700 Subject: [PATCH 1046/1492] [SPARK-3812] [BUILD] Adapt maven build to publish effective pom. I have tried maven help plugin first but that published all projects in top level pom. So I was left with no choice but to roll my own trivial plugin. This patch basically installs an effective pom after maven install is finished. The problem it fixes is described as follows: If you install using maven ` mvn install -DskipTests -Dhadoop.version=2.2.0 -Phadoop-2.2 ` Then without this patch the published pom(s) will have hadoop version as 1.0.4. This can be a problem at some point. Author: Prashant Sharma Closes #2673 from ScrapCodes/build-changes-effective-pom and squashes the following commits: aa7b91d [Prashant Sharma] used an unused dep. 0300dac [Prashant Sharma] improved comment messages.. 28f891e [Prashant Sharma] Added a useless dependency, so that we can shade it. And realized fake shading works for us. 553d96b [Prashant Sharma] Shaded some unused class of an unused dep, to generate effective pom(s) --- pom.xml | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 288bbf1114bea..687cc6352d5c1 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,17 @@ - + + + + org.spark-project.spark + unused + 1.0.0 + + @@ -992,6 +1002,27 @@ + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + false + + + org.spark-project.spark:unused + + + + + + org.apache.maven.plugins maven-enforcer-plugin From d6a302539213e8cdb51ca14b1769aeb33f9f435f Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 22 Oct 2014 22:13:11 -0700 Subject: [PATCH 1047/1492] [BUILD] Fixed resolver for scalastyle plugin and upgrade sbt version. Author: Prashant Sharma Closes #2877 from ScrapCodes/scalastyle-fix and squashes the following commits: a17b9fe [Prashant Sharma] [BUILD] Fixed resolver for scalastyle plugin. --- project/build.properties | 2 +- project/plugins.sbt | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/project/build.properties b/project/build.properties index c12ef652adfcb..32a3aeefaf9fb 100644 --- a/project/build.properties +++ b/project/build.properties @@ -14,4 +14,4 @@ # See the License for the specific language governing permissions and # limitations under the License. # -sbt.version=0.13.5 +sbt.version=0.13.6 diff --git a/project/plugins.sbt b/project/plugins.sbt index 678f5ed1ba610..9d50a50b109af 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -4,6 +4,8 @@ resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline. resolvers += "Typesafe Repository" at "http://repo.typesafe.com/typesafe/releases/" +resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/" + addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.11.2") addSbtPlugin("com.typesafe.sbteclipse" % "sbteclipse-plugin" % "2.2.0") From f799700eec4a5e33db9b2d6a4bee60a50fd5a099 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Thu, 23 Oct 2014 09:19:32 -0700 Subject: [PATCH 1048/1492] [SPARK-4055][MLlib] Inconsistent spelling 'MLlib' and 'MLLib' Thare are some inconsistent spellings 'MLlib' and 'MLLib' in some documents and source codes. Author: Kousuke Saruta Closes #2903 from sarutak/SPARK-4055 and squashes the following commits: b031640 [Kousuke Saruta] Fixed inconsistent spelling "MLlib and MLLib" --- docs/mllib-feature-extraction.md | 2 +- docs/mllib-statistics.md | 2 +- .../src/main/java/org/apache/spark/examples/mllib/JavaALS.java | 2 +- .../main/java/org/apache/spark/examples/mllib/JavaKMeans.java | 2 +- .../main/scala/org/apache/spark/mllib/api/python/package.scala | 2 +- 5 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 1511ae6dda4ed..11622414494e4 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -83,7 +83,7 @@ val idf = new IDF().fit(tf) val tfidf: RDD[Vector] = idf.transform(tf) {% endhighlight %} -MLLib's IDF implementation provides an option for ignoring terms which occur in less than a +MLlib's IDF implementation provides an option for ignoring terms which occur in less than a minimum number of documents. In such cases, the IDF for these terms is set to 0. This feature can be used by passing the `minDocFreq` value to the IDF constructor. diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index c4632413991f1..10a5131c07414 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -197,7 +197,7 @@ print Statistics.corr(data, method="pearson") ## Stratified sampling -Unlike the other statistics functions, which reside in MLLib, stratified sampling methods, +Unlike the other statistics functions, which reside in MLlib, stratified sampling methods, `sampleByKey` and `sampleByKeyExact`, can be performed on RDD's of key-value pairs. For stratified sampling, the keys can be thought of as a label and the value as a specific attribute. For example the key can be man or woman, or document ids, and the respective values can be the list of ages diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java index 8d381d4e0a943..95a430f1da234 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java @@ -32,7 +32,7 @@ import scala.Tuple2; /** - * Example using MLLib ALS from Java. + * Example using MLlib ALS from Java. */ public final class JavaALS { diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java index f796123a25727..e575eedeb465c 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java @@ -30,7 +30,7 @@ import org.apache.spark.mllib.linalg.Vectors; /** - * Example using MLLib KMeans from Java. + * Example using MLlib KMeans from Java. */ public final class JavaKMeans { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala index 87bdc8558aaf5..c67a6d3ae6cce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/package.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.api /** - * Internal support for MLLib Python API. + * Internal support for MLlib Python API. * * @see [[org.apache.spark.mllib.api.python.PythonMLLibAPI]] */ From 6b485225271a3c616c4fa1231c20090a95c86f32 Mon Sep 17 00:00:00 2001 From: Tal Sliwowicz Date: Thu, 23 Oct 2014 10:51:06 -0700 Subject: [PATCH 1049/1492] [SPARK-4006] In long running contexts, we encountered the situation of double registe... ...r without a remove in between. The cause for that is unknown, and assumed a temp network issue. However, since the second register is with a BlockManagerId on a different port, blockManagerInfo.contains() returns false, while blockManagerIdByExecutor returns Some. This inconsistency is caught in a conditional statement that does System.exit(1), which is a huge robustness issue for us. The fix - simply remove the old id from both maps during register when this happens. We are mimicking the behavior of expireDeadHosts(), by doing local cleanup of the maps before trying to add new ones. Also - added some logging for register and unregister. This is just like https://github.com/apache/spark/pull/2854 except it's on master Author: Tal Sliwowicz Closes #2886 from tsliwowicz/master-block-mgr-removal and squashes the following commits: 094d508 [Tal Sliwowicz] some more white space change undone 41a2217 [Tal Sliwowicz] some more whitspaces change undone 7bcfc3d [Tal Sliwowicz] whitspaces fix df9d98f [Tal Sliwowicz] Code review comments fixed f48bce9 [Tal Sliwowicz] In long running contexts, we encountered the situation of double register without a remove in between. The cause for that is unknown, and assumed a temp network issue. --- .../storage/BlockManagerMasterActor.scala | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 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 088f06e389d83..5e375a2553979 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -203,6 +203,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } } listenerBus.post(SparkListenerBlockManagerRemoved(System.currentTimeMillis(), blockManagerId)) + logInfo(s"Removing block manager $blockManagerId") } private def expireDeadHosts() { @@ -327,20 +328,20 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val time = System.currentTimeMillis() if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { - case Some(manager) => - // A block manager of the same executor already exists. - // This should never happen. Let's just quit. - logError("Got two different block manager registrations on " + id.executorId) - System.exit(1) + case Some(oldId) => + // A block manager of the same executor already exists, so remove it (assumed dead) + logError("Got two different block manager registrations on same executor - " + + s" will replace old one $oldId with new one $id") + removeExecutor(id.executorId) 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, time, maxMemSize, slaveActor) + logInfo("Registering block manager %s with %s RAM, %s".format( + id.hostPort, Utils.bytesToString(maxMemSize), id)) + + blockManagerIdByExecutor(id.executorId) = id + + blockManagerInfo(id) = new BlockManagerInfo( + id, System.currentTimeMillis(), maxMemSize, slaveActor) } listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxMemSize)) } From 293672c499911328eb27b48dbd7bdef4f4cc8adb Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 23 Oct 2014 13:46:55 -0700 Subject: [PATCH 1050/1492] specify unidocGenjavadocVersion of 0.8 Fixes an issue with being too strict generating javadoc causing errors. Author: Holden Karau Closes #2893 from holdenk/SPARK-3359-sbtunidoc-java8 and squashes the following commits: 9379a70 [Holden Karau] specify unidocGenjavadocVersion of 0.8 --- project/SparkBuild.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 01a5b20e7c51d..705937e3016e2 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -22,6 +22,7 @@ import sbt._ import sbt.Classpaths.publishTask import sbt.Keys._ import sbtunidoc.Plugin.genjavadocSettings +import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} import net.virtualvoid.sbt.graph.Plugin.graphSettings @@ -116,6 +117,7 @@ object SparkBuild extends PomBuild { retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", publishMavenStyle := true, + unidocGenjavadocVersion := "0.8", resolvers += Resolver.mavenLocal, otherResolvers <<= SbtPomKeys.mvnLocalRepository(dotM2 => Seq(Resolver.file("dotM2", dotM2))), From 222fa47f0dfd6c53aac513655a519521d9396e72 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 23 Oct 2014 16:01:38 -0700 Subject: [PATCH 1051/1492] Revert "[SPARK-3812] [BUILD] Adapt maven build to publish effective pom." This reverts commit c5882c663e054adcd3ecd9f11e91a1929dbc14a3. I am reverting this becuase it appears to cause the maven tests to hang. --- pom.xml | 33 +-------------------------------- 1 file changed, 1 insertion(+), 32 deletions(-) diff --git a/pom.xml b/pom.xml index 687cc6352d5c1..288bbf1114bea 100644 --- a/pom.xml +++ b/pom.xml @@ -248,17 +248,7 @@ - - - - org.spark-project.spark - unused - 1.0.0 - - + @@ -1002,27 +992,6 @@ - - - org.apache.maven.plugins - maven-shade-plugin - - - package - - shade - - - false - - - org.spark-project.spark:unused - - - - - - org.apache.maven.plugins maven-enforcer-plugin From 83b7a1c6503adce1826fc537b4db47e534da5cae Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 23 Oct 2014 16:39:32 -0700 Subject: [PATCH 1052/1492] [SPARK-4019] [SPARK-3740] Fix MapStatus compression bug that could lead to empty results or Snappy errors This commit fixes a bug in MapStatus that could cause jobs to wrongly return empty results if those jobs contained stages with more than 2000 partitions where most of those partitions were empty. For jobs with > 2000 partitions, MapStatus uses HighlyCompressedMapStatus, which only stores the average size of blocks. If the average block size is zero, then this will cause all blocks to be reported as empty, causing BlockFetcherIterator to mistakenly skip them. For example, this would return an empty result: sc.makeRDD(0 until 10, 1000).repartition(2001).collect() This can also lead to deserialization errors (e.g. Snappy decoding errors) for jobs with > 2000 partitions where the average block size is non-zero but there is at least one empty block. In this case, the BlockFetcher attempts to fetch empty blocks and fails when trying to deserialize them. The root problem here is that MapStatus has a (previously undocumented) correctness property that was violated by HighlyCompressedMapStatus: If a block is non-empty, then getSizeForBlock must be non-zero. I fixed this by modifying HighlyCompressedMapStatus to store the average size of _non-empty_ blocks and to use a compressed bitmap to track which blocks are empty. I also removed a test which was broken as originally written: it attempted to check that HighlyCompressedMapStatus's size estimation error was < 10%, but this was broken because HighlyCompressedMapStatus is only used for map statuses with > 2000 partitions, but the test only created 50. Author: Josh Rosen Closes #2866 from JoshRosen/spark-4019 and squashes the following commits: fc8b490 [Josh Rosen] Roll back hashset change, which didn't improve performance. 5faa0a4 [Josh Rosen] Incorporate review feedback c8b8cae [Josh Rosen] Two performance fixes: 3b892dd [Josh Rosen] Address Reynold's review comments ba2e71c [Josh Rosen] Add missing newline 609407d [Josh Rosen] Use Roaring Bitmap to track non-empty blocks. c23897a [Josh Rosen] Use sets when comparing collect() results 91276a3 [Josh Rosen] [SPARK-4019] Fix MapStatus compression bug that could lead to empty results. --- core/pom.xml | 4 + .../apache/spark/scheduler/MapStatus.scala | 76 ++++++++++++++++--- .../scala/org/apache/spark/rdd/RDDSuite.scala | 5 ++ .../spark/scheduler/MapStatusSuite.scala | 53 +++++++------ pom.xml | 5 ++ 5 files changed, 109 insertions(+), 34 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index a5a178079bc57..7b68dbaea4789 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -132,6 +132,10 @@ com.twitter chill-java + + org.roaringbitmap + RoaringBitmap + commons-net commons-net diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index e25096ea92d70..2ab5d9637b593 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -19,6 +19,8 @@ package org.apache.spark.scheduler import java.io.{Externalizable, ObjectInput, ObjectOutput} +import org.roaringbitmap.RoaringBitmap + import org.apache.spark.storage.BlockManagerId /** @@ -29,7 +31,12 @@ private[spark] sealed trait MapStatus { /** Location where this task was run. */ def location: BlockManagerId - /** Estimated size for the reduce block, in bytes. */ + /** + * Estimated size for the reduce block, in bytes. + * + * If a block is non-empty, then this method MUST return a non-zero size. This invariant is + * necessary for correctness, since block fetchers are allowed to skip zero-size blocks. + */ def getSizeForBlock(reduceId: Int): Long } @@ -38,7 +45,7 @@ private[spark] object MapStatus { def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { if (uncompressedSizes.length > 2000) { - new HighlyCompressedMapStatus(loc, uncompressedSizes) + HighlyCompressedMapStatus(loc, uncompressedSizes) } else { new CompressedMapStatus(loc, uncompressedSizes) } @@ -112,35 +119,80 @@ private[spark] class CompressedMapStatus( } } - /** - * A [[MapStatus]] implementation that only stores the average size of the blocks. + * A [[MapStatus]] implementation that only stores the average size of non-empty blocks, + * plus a bitmap for tracking which blocks are non-empty. During serialization, this bitmap + * is compressed. * - * @param loc location where the task is being executed. - * @param avgSize average size of all the blocks + * @param loc location where the task is being executed + * @param numNonEmptyBlocks the number of non-empty blocks + * @param emptyBlocks a bitmap tracking which blocks are empty + * @param avgSize average size of the non-empty blocks */ -private[spark] class HighlyCompressedMapStatus( +private[spark] class HighlyCompressedMapStatus private ( private[this] var loc: BlockManagerId, + private[this] var numNonEmptyBlocks: Int, + private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long) extends MapStatus with Externalizable { - def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { - this(loc, uncompressedSizes.sum / uncompressedSizes.length) - } + // loc could be null when the default constructor is called during deserialization + require(loc == null || avgSize > 0 || numNonEmptyBlocks == 0, + "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, 0L) // For deserialization only + protected def this() = this(null, -1, null, -1) // For deserialization only override def location: BlockManagerId = loc - override def getSizeForBlock(reduceId: Int): Long = avgSize + override def getSizeForBlock(reduceId: Int): Long = { + if (emptyBlocks.contains(reduceId)) { + 0 + } else { + avgSize + } + } override def writeExternal(out: ObjectOutput): Unit = { loc.writeExternal(out) + emptyBlocks.writeExternal(out) out.writeLong(avgSize) } override def readExternal(in: ObjectInput): Unit = { loc = BlockManagerId(in) + emptyBlocks = new RoaringBitmap() + emptyBlocks.readExternal(in) avgSize = in.readLong() } } + +private[spark] object HighlyCompressedMapStatus { + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { + // We must keep track of which blocks are empty so that we don't report a zero-sized + // block as being non-empty (or vice-versa) when using the average block size. + var i = 0 + var numNonEmptyBlocks: Int = 0 + var totalSize: Long = 0 + // From a compression standpoint, it shouldn't matter whether we track empty or non-empty + // blocks. From a performance standpoint, we benefit from tracking empty blocks because + // we expect that there will be far fewer of them, so we will perform fewer bitmap insertions. + val emptyBlocks = new RoaringBitmap() + val totalNumBlocks = uncompressedSizes.length + while (i < totalNumBlocks) { + var size = uncompressedSizes(i) + if (size > 0) { + numNonEmptyBlocks += 1 + totalSize += size + } else { + emptyBlocks.add(i) + } + i += 1 + } + val avgSize = if (numNonEmptyBlocks > 0) { + totalSize / numNonEmptyBlocks + } else { + 0 + } + new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize) + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 465c1a8a43a79..6d2e696dc2fc4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -459,6 +459,11 @@ class RDDSuite extends FunSuite with SharedSparkContext { for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) } + test("collect large number of empty partitions") { + // Regression test for SPARK-4019 + assert(sc.makeRDD(0 until 10, 1000).repartition(2001).collect().toSet === (0 until 10).toSet) + } + test("take") { var nums = sc.makeRDD(Range(1, 1000), 1) assert(nums.take(0).size === 0) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 79e04f046e4c4..950c6dc58e332 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.FunSuite import org.apache.spark.SparkConf import org.apache.spark.serializer.JavaSerializer +import scala.util.Random class MapStatusSuite extends FunSuite { @@ -46,6 +47,26 @@ class MapStatusSuite extends FunSuite { } } + test("MapStatus should never report non-empty blocks' sizes as 0") { + import Math._ + for ( + numSizes <- Seq(1, 10, 100, 1000, 10000); + mean <- Seq(0L, 100L, 10000L, Int.MaxValue.toLong); + stddev <- Seq(0.0, 0.01, 0.5, 1.0) + ) { + val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean) + val status = MapStatus(BlockManagerId("a", "b", 10), sizes) + val status1 = compressAndDecompressMapStatus(status) + for (i <- 0 until numSizes) { + if (sizes(i) != 0) { + val failureMessage = s"Failed with $numSizes sizes with mean=$mean, stddev=$stddev" + assert(status.getSizeForBlock(i) !== 0, failureMessage) + assert(status1.getSizeForBlock(i) !== 0, failureMessage) + } + } + } + } + test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { val sizes = Array.fill[Long](2001)(150L) val status = MapStatus(null, sizes) @@ -56,37 +77,25 @@ class MapStatusSuite extends FunSuite { assert(status.getSizeForBlock(2000) === 150L) } - test(classOf[HighlyCompressedMapStatus].getName + ": estimated size is within 10%") { - val sizes = Array.tabulate[Long](50) { i => i.toLong } + test("HighlyCompressedMapStatus: estimated size should be the average non-empty block size") { + val sizes = Array.tabulate[Long](3000) { i => i.toLong } + val avg = sizes.sum / sizes.filter(_ != 0).length val loc = BlockManagerId("a", "b", 10) val status = MapStatus(loc, sizes) - val ser = new JavaSerializer(new SparkConf) - val buf = ser.newInstance().serialize(status) - val status1 = ser.newInstance().deserialize[MapStatus](buf) + val status1 = compressAndDecompressMapStatus(status) + assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) - for (i <- 0 until sizes.length) { - // make sure the estimated size is within 10% of the input; note that we skip the very small - // sizes because the compression is very lossy there. + for (i <- 0 until 3000) { val estimate = status1.getSizeForBlock(i) - if (estimate > 100) { - assert(math.abs(estimate - sizes(i)) * 10 <= sizes(i), - s"incorrect estimated size $estimate, original was ${sizes(i)}") + if (sizes(i) > 0) { + assert(estimate === avg) } } } - test(classOf[HighlyCompressedMapStatus].getName + ": estimated size should be the average size") { - val sizes = Array.tabulate[Long](3000) { i => i.toLong } - val avg = sizes.sum / sizes.length - val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes) + def compressAndDecompressMapStatus(status: MapStatus): MapStatus = { val ser = new JavaSerializer(new SparkConf) val buf = ser.newInstance().serialize(status) - val status1 = ser.newInstance().deserialize[MapStatus](buf) - assert(status1.location == loc) - for (i <- 0 until 3000) { - val estimate = status1.getSizeForBlock(i) - assert(estimate === avg) - } + ser.newInstance().deserialize[MapStatus](buf) } } diff --git a/pom.xml b/pom.xml index 288bbf1114bea..a7e71f9ca5596 100644 --- a/pom.xml +++ b/pom.xml @@ -428,6 +428,11 @@ + + org.roaringbitmap + RoaringBitmap + 0.4.1 + commons-net commons-net From e595c8d08a20a122295af62d5e9cc4116f9727f6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 23 Oct 2014 17:20:00 -0700 Subject: [PATCH 1053/1492] [SPARK-3993] [PySpark] fix bug while reuse worker after take() After take(), maybe there are some garbage left in the socket, then next task assigned to this worker will hang because of corrupted data. We should make sure the socket is clean before reuse it, write END_OF_STREAM at the end, and check it after read out all result from python. Author: Davies Liu Author: Davies Liu Closes #2838 from davies/fix_reuse and squashes the following commits: 8872914 [Davies Liu] fix tests 660875b [Davies Liu] fix bug while reuse worker after take() --- .../scala/org/apache/spark/SparkEnv.scala | 2 ++ .../apache/spark/api/python/PythonRDD.scala | 11 ++++++++++- python/pyspark/daemon.py | 5 ++++- python/pyspark/serializers.py | 1 + python/pyspark/tests.py | 19 ++++++++++++++++++- python/pyspark/worker.py | 11 +++++++++-- 6 files changed, 44 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index aba713cb4267a..906a00b0bd17c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -68,6 +68,7 @@ class SparkEnv ( val shuffleMemoryManager: ShuffleMemoryManager, val conf: SparkConf) extends Logging { + private[spark] var isStopped = false private val pythonWorkers = mutable.HashMap[(String, Map[String, String]), PythonWorkerFactory]() // A general, soft-reference map for metadata needed during HadoopRDD split computation @@ -75,6 +76,7 @@ class SparkEnv ( private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]() private[spark] def stop() { + isStopped = true pythonWorkers.foreach { case(key, worker) => worker.stop() } Option(httpFileServer).foreach(_.stop()) mapOutputTracker.stop() diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 29ca751519abd..163dca6cade5a 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -75,6 +75,7 @@ private[spark] class PythonRDD( var complete_cleanly = false context.addTaskCompletionListener { context => writerThread.shutdownOnTaskCompletion() + writerThread.join() if (reuse_worker && complete_cleanly) { env.releasePythonWorker(pythonExec, envVars.toMap, worker) } else { @@ -145,7 +146,9 @@ private[spark] class PythonRDD( stream.readFully(update) accumulator += Collections.singletonList(update) } - complete_cleanly = true + if (stream.readInt() == SpecialLengths.END_OF_STREAM) { + complete_cleanly = true + } null } } catch { @@ -154,6 +157,10 @@ private[spark] class PythonRDD( logDebug("Exception thrown after task interruption", e) throw new TaskKilledException + case e: Exception if env.isStopped => + logDebug("Exception thrown after context is stopped", e) + null // exit silently + case e: Exception if writerThread.exception.isDefined => logError("Python worker exited unexpectedly (crashed)", e) logError("This may have been caused by a prior exception:", writerThread.exception.get) @@ -235,6 +242,7 @@ private[spark] class PythonRDD( // Data values PythonRDD.writeIteratorToStream(firstParent.iterator(split, context), dataOut) dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION) + dataOut.writeInt(SpecialLengths.END_OF_STREAM) dataOut.flush() } catch { case e: Exception if context.isCompleted || context.isInterrupted => @@ -306,6 +314,7 @@ private object SpecialLengths { val END_OF_DATA_SECTION = -1 val PYTHON_EXCEPTION_THROWN = -2 val TIMING_DATA = -3 + val END_OF_STREAM = -4 } private[spark] object PythonRDD extends Logging { diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 64d6202acb27d..dbb34775d9ac5 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -26,7 +26,7 @@ import gc from errno import EINTR, ECHILD, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN -from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN +from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT from pyspark.worker import main as worker_main from pyspark.serializers import read_int, write_int @@ -46,6 +46,9 @@ def worker(sock): signal.signal(SIGHUP, SIG_DFL) signal.signal(SIGCHLD, SIG_DFL) signal.signal(SIGTERM, SIG_DFL) + # restore the handler for SIGINT, + # it's useful for debugging (show the stacktrace before exit) + signal.signal(SIGINT, signal.default_int_handler) # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 08a0f0d8ffb3e..904bd9f2652d3 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -80,6 +80,7 @@ class SpecialLengths(object): END_OF_DATA_SECTION = -1 PYTHON_EXCEPTION_THROWN = -2 TIMING_DATA = -3 + END_OF_STREAM = -4 class Serializer(object): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1a8e4150e63c3..7a2107ec326ee 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -31,7 +31,7 @@ import time import zipfile import random -from platform import python_implementation +import threading if sys.version_info[:2] <= (2, 6): try: @@ -1380,6 +1380,23 @@ def test_accumulator_when_reuse_worker(self): self.assertEqual(sum(range(100)), acc2.value) self.assertEqual(sum(range(100)), acc1.value) + def test_reuse_worker_after_take(self): + rdd = self.sc.parallelize(range(100000), 1) + self.assertEqual(0, rdd.first()) + + def count(): + try: + rdd.count() + except Exception: + pass + + t = threading.Thread(target=count) + t.daemon = True + t.start() + t.join(5) + self.assertTrue(not t.isAlive()) + self.assertEqual(100000, rdd.count()) + class SparkSubmitTests(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8257dddfee1c3..2bdccb5e93f09 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -57,7 +57,7 @@ def main(infile, outfile): boot_time = time.time() split_index = read_int(infile) if split_index == -1: # for unit tests - return + exit(-1) # initialize global state shuffle.MemoryBytesSpilled = 0 @@ -111,7 +111,6 @@ def process(): try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) write_with_length(traceback.format_exc(), outfile) - outfile.flush() except IOError: # JVM close the socket pass @@ -131,6 +130,14 @@ def process(): for (aid, accum) in _accumulatorRegistry.items(): pickleSer._write_with_length((aid, accum._value), outfile) + # check end of stream + if read_int(infile) == SpecialLengths.END_OF_STREAM: + write_int(SpecialLengths.END_OF_STREAM, outfile) + else: + # write a different value to tell JVM to not reuse this worker + write_int(SpecialLengths.END_OF_DATA_SECTION, outfile) + exit(-1) + if __name__ == '__main__': # Read a local port to connect to from stdin From a29c9bd614e6d8088f759ae39218b945604d0b5b Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 23 Oct 2014 22:15:03 -0700 Subject: [PATCH 1054/1492] [SPARK-4000][BUILD] Sends archived unit tests logs to Jenkins master This PR sends archived unit tests logs to the build history directory in Jenkins master, so that we can serve it via HTTP later to help debugging Jenkins build failures. pwendell JoshRosen Please help review, thanks! Author: Cheng Lian Closes #2845 from liancheng/log-archive and squashes the following commits: ac8d9d4 [Cheng Lian] Includes build number in messages posted to GitHub 68c7010 [Cheng Lian] Logs backup should be implemented in dev/run-tests-jenkins 4b912f7 [Cheng Lian] Sends archived unit tests logs to Jenkins master --- dev/run-tests-jenkins | 53 +++++++++++++++++++++++++++++++++---------- 1 file changed, 41 insertions(+), 12 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 451f3b771cc76..87c6715153da7 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -53,9 +53,9 @@ function post_message () { local message=$1 local data="{\"body\": \"$message\"}" local HTTP_CODE_HEADER="HTTP Response Code: " - + echo "Attempting to post to Github..." - + local curl_output=$( curl `#--dump-header -` \ --silent \ @@ -75,12 +75,12 @@ function post_message () { echo " > data: ${data}" >&2 # exit $curl_status fi - + local api_response=$( echo "${curl_output}" \ | grep -v -e "^${HTTP_CODE_HEADER}" ) - + local http_code=$( echo "${curl_output}" \ | grep -e "^${HTTP_CODE_HEADER}" \ @@ -92,12 +92,39 @@ function post_message () { echo " > api_response: ${api_response}" >&2 echo " > data: ${data}" >&2 fi - + if [ "$curl_status" -eq 0 ] && [ "$http_code" -eq "201" ]; then echo " > Post successful." fi } +function send_archived_logs () { + echo "Archiving unit tests logs..." + + local log_files=$(find . -name "unit-tests.log") + + if [ -z "$log_files" ]; then + echo "> No log files found." >&2 + else + local log_archive="unit-tests-logs.tar.gz" + echo "$log_files" | xargs tar czf ${log_archive} + + local jenkins_build_dir=${JENKINS_HOME}/jobs/${JOB_NAME}/builds/${BUILD_NUMBER} + local scp_output=$(scp ${log_archive} amp-jenkins-master:${jenkins_build_dir}/${log_archive}) + local scp_status="$?" + + if [ "$scp_status" -ne 0 ]; then + echo "Failed to send archived unit tests logs to Jenkins master." >&2 + echo "> scp_status: ${scp_status}" >&2 + echo "> scp_output: ${scp_output}" >&2 + else + echo "> Send successful." + fi + + rm -f ${log_archive} + fi +} + # We diff master...$ghprbActualCommit because that gets us changes introduced in the PR #+ and not anything else added to master since the PR was branched. @@ -109,7 +136,7 @@ function post_message () { else merge_note=" * This patch merges cleanly." fi - + source_files=$( git diff master...$ghprbActualCommit --name-only `# diff patch against master from branch point` \ | grep -v -e "\/test" `# ignore files in test directories` \ @@ -144,12 +171,12 @@ function post_message () { # post start message { start_message="\ - [QA tests have started](${BUILD_URL}consoleFull) for \ + [Test build ${BUILD_DISPLAY_NAME} has started](${BUILD_URL}consoleFull) for \ PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." - + start_message="${start_message}\n${merge_note}" # start_message="${start_message}\n${public_classes_note}" - + post_message "$start_message" } @@ -159,7 +186,7 @@ function post_message () { test_result="$?" if [ "$test_result" -eq "124" ]; then - fail_message="**[Tests timed out](${BUILD_URL}consoleFull)** \ + fail_message="**[Test build ${BUILD_DISPLAY_NAME} timed out](${BUILD_URL}consoleFull)** \ for PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL}) \ after a configured wait of \`${TESTS_TIMEOUT}\`." @@ -187,15 +214,17 @@ function post_message () { else failing_test="some tests" fi - + test_result_note=" * This patch **fails $failing_test**." fi + + send_archived_logs } # post end message { result_message="\ - [QA tests have finished](${BUILD_URL}consoleFull) for \ + [Test build ${BUILD_DISPLAY_NAME} has finished](${BUILD_URL}consoleFull) for \ PR $ghprbPullId at commit [\`${SHORT_COMMIT_HASH}\`](${COMMIT_URL})." result_message="${result_message}\n${test_result_note}" From 0aea22895ce94f4f80faad5a2c41b76edac172b9 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 23 Oct 2014 23:49:50 -0700 Subject: [PATCH 1055/1492] SPARK-3812 Build changes to publish effective pom. Author: Prashant Sharma Closes #2921 from ScrapCodes/build-changes-effective-pom and squashes the following commits: 8841491 [Prashant Sharma] Fixed broken maven build. aa7b91d [Prashant Sharma] used an unused dep. 0300dac [Prashant Sharma] improved comment messages.. 28f891e [Prashant Sharma] Added a useless dependency, so that we can shade it. And realized fake shading works for us. 553d96b [Prashant Sharma] Shaded some unused class of an unused dep, to generate effective pom(s) --- pom.xml | 33 ++++++++++++++++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a7e71f9ca5596..a9897b866b036 100644 --- a/pom.xml +++ b/pom.xml @@ -248,7 +248,17 @@ - + + + + org.spark-project.spark + unused + 1.0.0 + + @@ -997,6 +1007,27 @@ + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + org.spark-project.spark:unused + + + + + + package + + shade + + + + org.apache.maven.plugins maven-enforcer-plugin From 809c785bcc33e684a68ea14240a466def864199a Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 23 Oct 2014 23:58:00 -0700 Subject: [PATCH 1056/1492] [SPARK-2652] [PySpark] donot use KyroSerializer as default serializer KyroSerializer can not serialize customized class without registered explicitly, use it as default serializer in PySpark will introduce some regression in MLlib. cc mengxr Author: Davies Liu Closes #2916 from davies/revert and squashes the following commits: 43eb6d3 [Davies Liu] donot use KyroSerializer as default serializer --- python/pyspark/context.py | 1 - 1 file changed, 1 deletion(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 8d27ccb95f82c..5f8dcedb1eea2 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -43,7 +43,6 @@ # These are special default configs for PySpark, they will overwrite # the default ones for Spark if they are not configured by user. DEFAULT_CONFIGS = { - "spark.serializer": "org.apache.spark.serializer.KryoSerializer", "spark.serializer.objectStreamReset": 100, "spark.rdd.compress": True, } From d2987e8f7a2cb3bf971f381399d8efdccb51d3d2 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 24 Oct 2014 08:51:08 -0500 Subject: [PATCH 1057/1492] [SPARK-3900][YARN] ApplicationMaster's shutdown hook fails and IllegalStateException is thrown. ApplicationMaster registers a shutdown hook and it calls ApplicationMaster#cleanupStagingDir. cleanupStagingDir invokes FileSystem.get(yarnConf) and it invokes FileSystem.getInternal. FileSystem.getInternal also registers shutdown hook. In FileSystem of hadoop 0.23, the shutdown hook registration does not consider whether shutdown is in progress or not (In 2.2, it's considered). // 0.23 if (map.isEmpty() ) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } // 2.2 if (map.isEmpty() && !ShutdownHookManager.get().isShutdownInProgress()) { ShutdownHookManager.get().addShutdownHook(clientFinalizer, SHUTDOWN_HOOK_PRIORITY); } Thus, in 0.23, another shutdown hook can be registered when ApplicationMaster's shutdown hook run. This issue cause IllegalStateException as follows. java.lang.IllegalStateException: Shutdown in progress, cannot add a shutdownHook at org.apache.hadoop.util.ShutdownHookManager.addShutdownHook(ShutdownHookManager.java:152) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2306) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2278) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:316) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:162) at org.apache.spark.deploy.yarn.ApplicationMaster.org$apache$spark$deploy$yarn$ApplicationMaster$$cleanupStagingDir(ApplicationMaster.scala:307) at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$3.run(ApplicationMaster.scala:118) at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54) Author: Kousuke Saruta Closes #2924 from sarutak/SPARK-3900-2 and squashes the following commits: 9112817 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900-2 97018fa [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 2c2850e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 ee52db2 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 a7d6c9b [Kousuke Saruta] Merge branch 'SPARK-3900' of github.com:sarutak/spark into SPARK-3900 1cdf03c [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 a5f6443 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3900 57b397d [Kousuke Saruta] Fixed IllegalStateException caused by shutdown hook registration in another shutdown hook --- .../org/apache/spark/deploy/yarn/ApplicationMaster.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index a3c43b43848d2..e6fe0265d8811 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -92,6 +92,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, logInfo("ApplicationAttemptId: " + appAttemptId) + val fs = FileSystem.get(yarnConf) val cleanupHook = new Runnable { override def run() { // If the SparkContext is still registered, shut it down as a best case effort in case @@ -115,7 +116,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // we only want to unregister if we don't want the RM to retry if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { unregister(finalStatus, finalMsg) - cleanupStagingDir() + cleanupStagingDir(fs) } } } @@ -303,8 +304,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, /** * Clean up the staging directory. */ - private def cleanupStagingDir() { - val fs = FileSystem.get(yarnConf) + private def cleanupStagingDir(fs: FileSystem) { var stagingDirPath: Path = null try { val preserveFiles = sparkConf.get("spark.yarn.preserve.staging.files", "false").toBoolean From d60a9d440b00beb107c1f1d7f42886c94f04a092 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 24 Oct 2014 10:48:03 -0700 Subject: [PATCH 1058/1492] [SPARK-4051] [SQL] [PySpark] Convert Row into dictionary Added a method to Row to turn row into dict: ``` >>> row = Row(a=1) >>> row.asDict() {'a': 1} ``` Author: Davies Liu Closes #2896 from davies/dict and squashes the following commits: 8d97366 [Davies Liu] convert Row into dict --- python/pyspark/sql.py | 12 ++++++++++++ python/pyspark/tests.py | 9 +++++++++ 2 files changed, 21 insertions(+) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index b31a82f9b19ac..7daf306f68479 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -883,6 +883,10 @@ class Row(tuple): # create property for fast access locals().update(_create_properties(dataType.fields)) + def asDict(self): + """ Return as a dict """ + return dict(zip(self.__FIELDS__, self)) + def __repr__(self): # call collect __repr__ for nested objects return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) @@ -1466,6 +1470,14 @@ def __new__(self, *args, **kwargs): else: raise ValueError("No args or kwargs") + def asDict(self): + """ + Return as an dict + """ + if not hasattr(self, "__FIELDS__"): + raise TypeError("Cannot convert a Row class into dict") + return dict(zip(self.__FIELDS__, self)) + # let obect acs like class def __call__(self, *args): """create new Row object""" diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 7a2107ec326ee..047d85783089f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -771,6 +771,15 @@ def test_serialize_nested_array_and_map(self): self.assertEqual(1.0, row.c) self.assertEqual("2", row.d) + def test_convert_row_to_dict(self): + row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) + self.assertEqual(1, row.asDict()['l'][0].a) + rdd = self.sc.parallelize([row]) + srdd = self.sqlCtx.inferSchema(rdd) + srdd.registerTempTable("test") + row = self.sqlCtx.sql("select l[0].a AS la from test").first() + self.assertEqual(1, row.asDict()["la"]) + class InputFormatTests(ReusedPySparkTestCase): From 0e886610eedd8ea24761cdcefa25ccedeca72dc8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 24 Oct 2014 10:52:25 -0700 Subject: [PATCH 1059/1492] [SPARK-4050][SQL] Fix caching of temporary tables with projections. Previously cached data was found by `sameResult` plan matching on optimized plans. This technique however fails to locate the cached data when a temporary table with a projection is queried with a further reduced projection. The failure is due to the fact that optimization will collapse the projections, producing a plan that no longer produces the sameResult as the cached data (though the cached data still subsumes the desired data). For example consider the following previously failing test case. ```scala sql("CACHE TABLE tempTable AS SELECT key FROM testData") assertCached(sql("SELECT COUNT(*) FROM tempTable")) ``` In this PR I change the matching to occur after analysis instead of optimization, so that in the case of temporary tables, the plans will always match. I think this should work generally, however, this error does raise questions about the need to do more thorough subsumption checking when locating cached data. Another question is what sort of semantics we want to provide when uncaching data from temporary tables. For example consider the following sequence of commands: ```scala testData.select('key).registerTempTable("tempTable1") testData.select('key).registerTempTable("tempTable2") cacheTable("tempTable1") // This obviously works. assertCached(sql("SELECT COUNT(*) FROM tempTable1")) // It seems good that this works ... assertCached(sql("SELECT COUNT(*) FROM tempTable2")) // ... but is this valid? uncacheTable("tempTable2") // Should this still be cached? assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0) ``` Author: Michael Armbrust Closes #2912 from marmbrus/cachingBug and squashes the following commits: 9c822d4 [Michael Armbrust] remove commented out code 5c72fb7 [Michael Armbrust] Add a test case / question about uncaching semantics. 63a23e4 [Michael Armbrust] Perform caching on analyzed instead of optimized plan. 03f1cfe [Michael Armbrust] Clean-up / add tests to SameResult suite. --- .../sql/catalyst/plans/SameResultSuite.scala | 10 ++++-- .../org/apache/spark/sql/CacheManager.scala | 10 +++--- .../org/apache/spark/sql/SQLContext.scala | 6 ++-- .../apache/spark/sql/CachedTableSuite.scala | 34 ++++++++++++++++++- 4 files changed, 48 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index e8a793d107451..11e6831b24768 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ /** - * Provides helper methods for comparing plans. + * Tests for the sameResult function of [[LogicalPlan]]. */ class SameResultSuite extends FunSuite { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) @@ -52,11 +52,15 @@ class SameResultSuite extends FunSuite { assertSameResult(testRelation.select('a, 'b), testRelation2.select('a, 'b)) assertSameResult(testRelation.select('b, 'a), testRelation2.select('b, 'a)) - assertSameResult(testRelation, testRelation2.select('a), false) - assertSameResult(testRelation.select('b, 'a), testRelation2.select('a, 'b), false) + assertSameResult(testRelation, testRelation2.select('a), result = false) + assertSameResult(testRelation.select('b, 'a), testRelation2.select('a, 'b), result = false) } test("filters") { assertSameResult(testRelation.where('a === 'b), testRelation2.where('a === 'b)) } + + test("sorts") { + assertSameResult(testRelation.orderBy('a.asc), testRelation2.orderBy('a.asc)) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index 5ab2b5316ab10..3ced11a5e6c11 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -82,7 +82,7 @@ private[sql] trait CacheManager { private[sql] def cacheQuery( query: SchemaRDD, storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock { - val planToCache = query.queryExecution.optimizedPlan + val planToCache = query.queryExecution.analyzed if (lookupCachedData(planToCache).nonEmpty) { logWarning("Asked to cache already cached data.") } else { @@ -96,8 +96,8 @@ private[sql] trait CacheManager { /** Removes the data for the given SchemaRDD from the cache */ private[sql] def uncacheQuery(query: SchemaRDD, blocking: Boolean = true): Unit = writeLock { - val planToCache = query.queryExecution.optimizedPlan - val dataIndex = cachedData.indexWhere(_.plan.sameResult(planToCache)) + val planToCache = query.queryExecution.analyzed + val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) require(dataIndex >= 0, s"Table $query is not cached.") cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) cachedData.remove(dataIndex) @@ -106,12 +106,12 @@ private[sql] trait CacheManager { /** Optionally returns cached data for the given SchemaRDD */ private[sql] def lookupCachedData(query: SchemaRDD): Option[CachedData] = readLock { - lookupCachedData(query.queryExecution.optimizedPlan) + lookupCachedData(query.queryExecution.analyzed) } /** Optionally returns cached data for the given LogicalPlan. */ private[sql] def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { - cachedData.find(_.plan.sameResult(plan)) + cachedData.find(cd => plan.sameResult(cd.plan)) } /** Replaces segments of the given logical plan with cached versions where possible. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 0e4a9ca60b00d..590dbf3cb893d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -374,13 +374,13 @@ class SQLContext(@transient val sparkContext: SparkContext) def logical: LogicalPlan lazy val analyzed = ExtractPythonUdfs(analyzer(logical)) - lazy val optimizedPlan = optimizer(analyzed) - lazy val withCachedData = useCachedData(optimizedPlan) + lazy val withCachedData = useCachedData(analyzed) + lazy val optimizedPlan = optimizer(withCachedData) // TODO: Don't just pick the first one... lazy val sparkPlan = { SparkPlan.currentContext.set(self) - planner(withCachedData).next() + planner(optimizedPlan).next() } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 444bc95009c31..da5a358df3b1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -53,17 +53,47 @@ class CachedTableSuite extends QueryTest { sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)).nonEmpty } + test("cache temp table") { + testData.select('key).registerTempTable("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) + cacheTable("tempTable") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + uncacheTable("tempTable") + } + + test("cache table as select") { + sql("CACHE TABLE tempTable AS SELECT key FROM testData") + assertCached(sql("SELECT COUNT(*) FROM tempTable")) + uncacheTable("tempTable") + } + + test("uncaching temp table") { + testData.select('key).registerTempTable("tempTable1") + testData.select('key).registerTempTable("tempTable2") + cacheTable("tempTable1") + + assertCached(sql("SELECT COUNT(*) FROM tempTable1")) + assertCached(sql("SELECT COUNT(*) FROM tempTable2")) + + // Is this valid? + uncacheTable("tempTable2") + + // Should this be cached? + assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0) + } + test("too big for memory") { val data = "*" * 10000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(table("bigData").count() === 200000L) - table("bigData").unpersist() + table("bigData").unpersist(blocking = true) } test("calling .cache() should use in-memory columnar caching") { table("testData").cache() assertCached(table("testData")) + table("testData").unpersist(blocking = true) } test("calling .unpersist() should drop in-memory columnar cache") { @@ -108,6 +138,8 @@ class CachedTableSuite extends QueryTest { case r @ InMemoryRelation(_, _, _, _, _: InMemoryColumnarTableScan) => r }.size } + + uncacheTable("testData") } test("read from cached table and uncache") { From 7c89a8f0c81ecf91dba34c1f44393f45845d438c Mon Sep 17 00:00:00 2001 From: Zhan Zhang Date: Fri, 24 Oct 2014 11:03:17 -0700 Subject: [PATCH 1060/1492] [SPARK-2706][SQL] Enable Spark to support Hive 0.13 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Given that a lot of users are trying to use hive 0.13 in spark, and the incompatibility between hive-0.12 and hive-0.13 on the API level I want to propose following approach, which has no or minimum impact on existing hive-0.12 support, but be able to jumpstart the development of hive-0.13 and future version support. Approach: Introduce “hive-version” property, and manipulate pom.xml files to support different hive version at compiling time through shim layer, e.g., hive-0.12.0 and hive-0.13.1. More specifically, 1. For each different hive version, there is a very light layer of shim code to handle API differences, sitting in sql/hive/hive-version, e.g., sql/hive/v0.12.0 or sql/hive/v0.13.1 2. Add a new profile hive-default active by default, which picks up all existing configuration and hive-0.12.0 shim (v0.12.0) if no hive.version is specified. 3. If user specifies different version (currently only 0.13.1 by -Dhive.version = 0.13.1), hive-versions profile will be activated, which pick up hive-version specific shim layer and configuration, mainly the hive jars and hive-version shim, e.g., v0.13.1. 4. With this approach, nothing is changed with current hive-0.12 support. No change by default: sbt/sbt -Phive For example: sbt/sbt -Phive -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly To enable hive-0.13: sbt/sbt -Dhive.version=0.13.1 For example: sbt/sbt -Dhive.version=0.13.1 -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly Note that in hive-0.13, hive-thriftserver is not enabled, which should be fixed by other Jira, and we don’t need -Phive with -Dhive.version in building (probably we should use -Phive -Dhive.version=xxx instead after thrift server is also supported in hive-0.13.1). Author: Zhan Zhang Author: zhzhan Author: Patrick Wendell Closes #2241 from zhzhan/spark-2706 and squashes the following commits: 3ece905 [Zhan Zhang] minor fix 410b668 [Zhan Zhang] solve review comments cbb4691 [Zhan Zhang] change run-test for new options 0d4d2ed [Zhan Zhang] rebase 497b0f4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 8fad1cf [Zhan Zhang] change the pom file and make hive-0.13.1 as the default ab028d1 [Zhan Zhang] rebase 4a2e36d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 4cb1b93 [zhzhan] Merge pull request #1 from pwendell/pr-2241 b0478c0 [Patrick Wendell] Changes to simplify the build of SPARK-2706 2b50502 [Zhan Zhang] rebase a72c0d4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark cb22863 [Zhan Zhang] correct the typo 20f6cf7 [Zhan Zhang] solve compatability issue f7912a9 [Zhan Zhang] rebase and solve review feedback 301eb4a [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 10c3565 [Zhan Zhang] address review comments 6bc9204 [Zhan Zhang] rebase and remove temparory repo d3aa3f2 [Zhan Zhang] Merge branch 'master' into spark-2706 cedcc6f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 3ced0d7 [Zhan Zhang] rebase d9b981d [Zhan Zhang] rebase and fix error due to rollback adf4924 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 3dd50e8 [Zhan Zhang] solve conflicts and remove unnecessary implicts d10bf00 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark dc7bdb3 [Zhan Zhang] solve conflicts 7e0cc36 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark d7c3e1e [Zhan Zhang] Merge branch 'master' into spark-2706 68deb11 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark d48bd18 [Zhan Zhang] address review comments 3ee3b2b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 57ea52e [Zhan Zhang] Merge branch 'master' into spark-2706 2b0d513 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 9412d24 [Zhan Zhang] address review comments f4af934 [Zhan Zhang] rebase 1ccd7cc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 128b60b [Zhan Zhang] ignore 0.12.0 test cases for the time being af9feb9 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 5f5619f [Zhan Zhang] restructure the directory and different hive version support 05d3683 [Zhan Zhang] solve conflicts e4c1982 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark 94b4fdc [Zhan Zhang] Spark-2706: hive-0.13.1 support on spark 87ebf3b [Zhan Zhang] Merge branch 'master' into spark-2706 921e914 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark f896b2a [Zhan Zhang] Merge branch 'master' into spark-2706 789ea21 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark cb53a2c [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark f6a8a40 [Zhan Zhang] revert ba14f28 [Zhan Zhang] test dbedff3 [Zhan Zhang] Merge remote-tracking branch 'upstream/master' 70964fe [Zhan Zhang] revert fe0f379 [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark 70ffd93 [Zhan Zhang] revert 42585ec [Zhan Zhang] test 7d5fce2 [Zhan Zhang] test --- assembly/pom.xml | 6 + dev/run-tests | 4 +- docs/building-spark.md | 26 ++- pom.xml | 29 ++- sql/hive/pom.xml | 37 +++- .../apache/spark/sql/hive/HiveContext.scala | 23 ++- .../spark/sql/hive/HiveInspectors.scala | 3 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 10 +- .../org/apache/spark/sql/hive/HiveQl.scala | 16 +- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/TestHive.scala | 5 + .../execution/DescribeHiveTableCommand.scala | 4 +- .../sql/hive/execution/HiveTableScan.scala | 4 +- .../hive/execution/InsertIntoHiveTable.scala | 8 +- .../spark/sql/hive/hiveWriterContainers.scala | 3 +- .../spark/sql/hive/StatisticsSuite.scala | 7 +- .../sql/hive/execution/HiveQuerySuite.scala | 22 ++- .../org/apache/spark/sql/hive/Shim.scala | 89 +++++++++ .../org/apache/spark/sql/hive/Shim.scala | 170 ++++++++++++++++++ 19 files changed, 406 insertions(+), 63 deletions(-) create mode 100644 sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala create mode 100644 sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala diff --git a/assembly/pom.xml b/assembly/pom.xml index 31a01e4d8e1de..bfef95b8deb95 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -197,6 +197,12 @@ spark-hive_${scala.binary.version} ${project.version} + + + + + hive-0.12.0 + org.apache.spark spark-hive-thriftserver_${scala.binary.version} diff --git a/dev/run-tests b/dev/run-tests index f47fcf66ff7e7..7d06c86eb4b41 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -140,7 +140,7 @@ CURRENT_BLOCK=$BLOCK_BUILD { # We always build with Hive because the PySpark Spark SQL tests need it. - BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" echo "[info] Building Spark with these arguments: $BUILD_MVN_PROFILE_ARGS" @@ -167,7 +167,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. # This must be a single argument, as it is. if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" fi if [ -n "$_SQL_TESTS_ONLY" ]; then diff --git a/docs/building-spark.md b/docs/building-spark.md index b2940ee4029e8..11fd56c145c01 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -97,12 +97,20 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package {% endhighlight %} + + # Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, -add the `-Phive` profile to your existing build options. +add the `-Phive` profile to your existing build options. By default Spark +will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using +the `-Phive-0.12.0` profile. NOTE: currently the JDBC server is only +supported for Hive 0.12.0. {% highlight bash %} -# Apache Hadoop 2.4.X with Hive support +# Apache Hadoop 2.4.X with Hive 13 support mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package + +# Apache Hadoop 2.4.X with Hive 12 support +mvn -Pyarn -Phive-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package {% endhighlight %} # Spark Tests in Maven @@ -111,8 +119,8 @@ Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.o Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive clean package - mvn -Pyarn -Phadoop-2.3 -Phive test + mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-0.12.0 clean package + mvn -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test The ScalaTest plugin also supports running only a specific test suite as follows: @@ -175,16 +183,16 @@ can be set to control the SBT build. For example: Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly - sbt/sbt -Pyarn -Phadoop-2.3 -Phive test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 assembly + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test To run only a specific test suite as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite" + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 "test-only org.apache.spark.repl.ReplSuite" To run test suites of a specific sub project as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 core/test # Speeding up Compilation with Zinc @@ -192,4 +200,4 @@ To run test suites of a specific sub project as follows: compiler. When run locally as a background process, it speeds up builds of Scala-based projects like Spark. Developers who regularly recompile Spark with Maven will be the most interested in Zinc. The project site gives instructions for building and running `zinc`; OS X users can -install it using `brew install zinc`. \ No newline at end of file +install it using `brew install zinc`. diff --git a/pom.xml b/pom.xml index a9897b866b036..a1195262614dd 100644 --- a/pom.xml +++ b/pom.xml @@ -127,7 +127,11 @@ 0.94.6 1.4.0 3.4.5 - 0.12.0-protobuf-2.5 + + 0.13.1 + + 0.13.1 + 10.10.1.1 1.4.3 1.2.3 8.1.14.v20131031 @@ -456,7 +460,7 @@ org.apache.derby derby - 10.4.2.0 + ${derby.version} com.codahale.metrics @@ -1308,16 +1312,31 @@ - - hive + hive-0.12.0 false + sql/hive-thriftserver + + 0.12.0-protobuf-2.5 + 0.12.0 + 10.4.2.0 + + + + hive-0.13.1 + + false + + + 0.13.1 + 0.13.1 + 10.10.1.1 + - diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 9d7a02bf7b0b7..db01363b4d629 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -36,11 +36,6 @@ - - com.twitter - parquet-hive-bundle - 1.5.0 - org.apache.spark spark-core_${scala.binary.version} @@ -116,7 +111,6 @@ test - hive @@ -144,6 +138,19 @@ + + hive-0.12.0 + + false + + + + com.twitter + parquet-hive-bundle + 1.5.0 + + + @@ -154,6 +161,24 @@ org.scalatest scalatest-maven-plugin + + org.codehaus.mojo + build-helper-maven-plugin + + + add-default-sources + generate-sources + + add-source + + + + v${hive.version.short}/src/main/scala + + + + + diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 8b5a90159e1bb..34ed57b001637 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -32,7 +32,6 @@ import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.metadata.Table import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.io.TimestampWritable import org.apache.hadoop.hive.serde2.io.DateWritable @@ -47,6 +46,7 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand +import org.apache.spark.sql.hive.HiveShim /** * DEPRECATED: Use HiveContext instead. @@ -171,13 +171,15 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val tableParameters = relation.hiveQlTable.getParameters val oldTotalSize = - Option(tableParameters.get(StatsSetupConst.TOTAL_SIZE)).map(_.toLong).getOrElse(0L) + Option(tableParameters.get(HiveShim.getStatsSetupConstTotalSize)) + .map(_.toLong) + .getOrElse(0L) val newTotalSize = getFileSizeForTable(hiveconf, relation.hiveQlTable) // Update the Hive metastore if the total size of the table is different than the size // recorded in the Hive metastore. // This logic is based on org.apache.hadoop.hive.ql.exec.StatsTask.aggregateStats(). if (newTotalSize > 0 && newTotalSize != oldTotalSize) { - tableParameters.put(StatsSetupConst.TOTAL_SIZE, newTotalSize.toString) + tableParameters.put(HiveShim.getStatsSetupConstTotalSize, newTotalSize.toString) val hiveTTable = relation.hiveQlTable.getTTable hiveTTable.setParameters(tableParameters) val tableFullName = @@ -282,29 +284,24 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { */ protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = { try { - // Session state must be initilized before the CommandProcessor is created . - SessionState.start(sessionState) - val cmd_trimmed: String = cmd.trim() val tokens: Array[String] = cmd_trimmed.split("\\s+") val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hiveconf) + val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hiveconf) proc match { case driver: Driver => - driver.init() - - val results = new JArrayList[String] + val results = HiveShim.createDriverResultsArray val response: CommandProcessorResponse = driver.run(cmd) // Throw an exception if there is an error in query processing. if (response.getResponseCode != 0) { - driver.destroy() + driver.close() throw new QueryExecutionException(response.getErrorMessage) } driver.setMaxRows(maxRows) driver.getResults(results) - driver.destroy() - results + driver.close() + HiveShim.processResults(results) case _ => sessionState.out.println(tokens(0) + " " + cmd_1) Seq(proc.run(cmd_1).getResponseCode.toString) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 1977618b4c9f2..deaa1a2a154f2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.hive.HiveShim /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -149,7 +150,7 @@ private[hive] trait HiveInspectors { case l: Long => l: java.lang.Long case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte - case b: BigDecimal => new HiveDecimal(b.underlying()) + case b: BigDecimal => HiveShim.createDecimal(b.underlying()) case b: Array[Byte] => b case d: java.sql.Date => d case t: java.sql.Timestamp => t 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 75a19656af110..904bb48691e35 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 @@ -22,7 +22,6 @@ import scala.util.parsing.combinator.RegexParsers import org.apache.hadoop.hive.metastore.api.{FieldSchema, SerDeInfo, StorageDescriptor, Partition => TPartition, Table => TTable} import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.Deserializer import org.apache.spark.Logging @@ -34,6 +33,7 @@ import org.apache.spark.sql.catalyst.plans.logical 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.hive.HiveShim import org.apache.spark.util.Utils /* Implicit conversions */ @@ -56,7 +56,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val table = client.getTable(databaseName, tblName) val partitions: Seq[Partition] = if (table.isPartitioned) { - client.getAllPartitionsForPruner(table).toSeq + HiveShim.getAllPartitionsOf(client, table).toSeq } else { Nil } @@ -185,7 +185,7 @@ object HiveMetastoreTypes extends RegexParsers { "bigint" ^^^ LongType | "binary" ^^^ BinaryType | "boolean" ^^^ BooleanType | - "decimal" ^^^ DecimalType | + HiveShim.metastoreDecimal ^^^ DecimalType | "date" ^^^ DateType | "timestamp" ^^^ TimestampType | "varchar\\((\\d+)\\)".r ^^^ StringType @@ -272,13 +272,13 @@ private[hive] case class MetastoreRelation // of RPCs are involved. Besides `totalSize`, there are also `numFiles`, `numRows`, // `rawDataSize` keys (see StatsSetupConst in Hive) that we can look at in the future. BigInt( - Option(hiveQlTable.getParameters.get(StatsSetupConst.TOTAL_SIZE)) + Option(hiveQlTable.getParameters.get(HiveShim.getStatsSetupConstTotalSize)) .map(_.toLong) .getOrElse(sqlContext.defaultSizeInBytes)) } ) - val tableDesc = new TableDesc( + val tableDesc = HiveShim.getTableDesc( Class.forName( hiveQlTable.getSerializationLib, true, 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 2b599157d15d3..ffcb6b505b9c6 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 @@ -18,7 +18,8 @@ package org.apache.spark.sql.hive import java.sql.Date - +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ import org.apache.hadoop.hive.ql.plan.PlanUtils @@ -216,7 +217,18 @@ private[hive] object HiveQl { /** * Returns the AST for the given SQL string. */ - def getAst(sql: String): ASTNode = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql)) + def getAst(sql: String): ASTNode = { + /* + * Context has to be passed in hive0.13.1. + * Otherwise, there will be Null pointer exception, + * when retrieving properties form HiveConf. + */ + val hContext = new Context(new HiveConf()) + val node = ParseUtils.findRootNonNullToken((new ParseDriver).parse(sql, hContext)) + hContext.clear() + node + } + /** Returns a LogicalPlan for a given HiveQL string. */ def parseSql(sql: String): LogicalPlan = hqlParser(sql) 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 fd4f65e488259..e45eb57b3debf 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 @@ -34,6 +34,7 @@ import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.hive.HiveShim /** * A trait for subclasses that handle table scans. @@ -138,7 +139,7 @@ class HadoopTableReader( filterOpt: Option[PathFilter]): RDD[Row] = { val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) - val partPath = partition.getPartitionPath + val partPath = HiveShim.getDataLocationPath(partition) val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) val ifc = partDesc.getInputFileFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 9a9e2eda6bcd4..0f74fe8943706 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -26,6 +26,7 @@ import scala.language.implicitConversions import org.apache.hadoop.hive.ql.exec.FunctionRegistry import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} import org.apache.hadoop.hive.ql.metadata.Table +import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.serde2.RegexSerDe import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.hadoop.hive.serde2.avro.AvroSerDe @@ -63,6 +64,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { // By clearing the port we force Spark to pick a new one. This allows us to rerun tests // without restarting the JVM. System.clearProperty("spark.hostPort") + CommandProcessorFactory.clean(hiveconf) lazy val warehousePath = getTempFilePath("sparkHiveWarehouse").getCanonicalPath lazy val metastorePath = getTempFilePath("sparkHiveMetastore").getCanonicalPath @@ -375,6 +377,9 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { */ protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames + // Database default may not exist in 0.13.1, create it if not exist + HiveShim.createDefaultDBIfNeeded(this) + /** * Resets the test instance by deleting any tables that have been created. * TODO: also clear out UDFs, views, etc. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index 106cede9788ec..fbd375639692f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -26,6 +26,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.execution.{Command, LeafNode} import org.apache.spark.sql.hive.{HiveContext, MetastoreRelation} +import org.apache.spark.sql.hive.HiveShim /** * Implementation for "describe [extended] table". @@ -43,7 +44,8 @@ case class DescribeHiveTableCommand( // Strings with the format like Hive. It is used for result comparison in our unit tests. lazy val hiveString: Seq[String] = sideEffectResult.map { case Row(name: String, dataType: String, comment) => - Seq(name, dataType, Option(comment.asInstanceOf[String]).getOrElse("None")) + Seq(name, dataType, + Option(comment.asInstanceOf[String]).getOrElse(HiveShim.getEmptyCommentsFieldValue)) .map(s => String.format(s"%-20s", s)) .mkString("\t") } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 5b83b77d80a22..85965a6ea095a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -23,7 +23,6 @@ import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition} import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.ColumnProjectionUtils import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption import org.apache.hadoop.hive.serde2.objectinspector.primitive._ @@ -83,8 +82,7 @@ case class HiveTableScan( attributes.map(a => relation.attributes.indexWhere(_.name == a.name): Integer).filter(index => index >= 0) - ColumnProjectionUtils.appendReadColumnIDs(hiveConf, neededColumnIDs) - ColumnProjectionUtils.appendReadColumnNames(hiveConf, attributes.map(_.name)) + HiveShim.appendReadColumns(hiveConf, neededColumnIDs, attributes.map(_.name)) val tableDesc = relation.tableDesc val deserializer = tableDesc.getDeserializerClass.newInstance diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index f0785d8882636..7db5fd804d6ef 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.{Context, ErrorMsg} import org.apache.hadoop.hive.serde2.Serializer import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.ObjectInspectorCopyOption @@ -37,6 +37,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} import org.apache.spark.sql.hive._ +import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.{SerializableWritable, SparkException, TaskContext} /** @@ -74,7 +76,7 @@ case class InsertIntoHiveTable( (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) case _: JavaHiveDecimalObjectInspector => - (o: Any) => new HiveDecimal(o.asInstanceOf[BigDecimal].underlying()) + (o: Any) => HiveShim.createDecimal(o.asInstanceOf[BigDecimal].underlying()) case soi: StandardStructObjectInspector => val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) @@ -170,7 +172,7 @@ case class InsertIntoHiveTable( // instances within the closure, since Serializer is not serializable while TableDesc is. val tableDesc = table.tableDesc val tableLocation = table.hiveQlTable.getDataLocation - val tmpLocation = hiveContext.getExternalTmpFileURI(tableLocation) + val tmpLocation = HiveShim.getExternalTmpPath(hiveContext, tableLocation) val fileSinkConf = new FileSinkDesc(tmpLocation.toString, tableDesc, false) val isCompressed = sc.hiveconf.getBoolean( ConfVars.COMPRESSRESULT.varname, ConfVars.COMPRESSRESULT.defaultBoolVal) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 6ccbc22a4acfb..981ab954da489 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -27,12 +27,13 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} -import org.apache.hadoop.hive.ql.plan.FileSinkDesc import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} +import org.apache.spark.sql.hive.{ShimFileSinkDesc => FileSinkDesc} +import org.apache.spark.sql.hive.HiveShim._ /** * Internal helper class that saves an RDD using a Hive OutputFormat. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 14e791fe0f0ee..aaefe84ce81ea 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -25,6 +25,7 @@ import scala.reflect.ClassTag import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.NativeCommand import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} +import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ @@ -80,8 +81,10 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() sql("INSERT INTO TABLE analyzeTable SELECT * FROM src").collect() - assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) - + // TODO: How does it works? needs to add it back for other hive version. + if (HiveShim.version =="0.12.0") { + assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) + } sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") assert(queryTotalSize("analyzeTable") === BigInt(11624)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 3e100775e4981..5de20175d9f57 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -508,14 +508,14 @@ class HiveQuerySuite extends HiveComparisonTest { // Describe a partition is a native command assertResult( Array( - Array("key", "int", "None"), - Array("value", "string", "None"), - Array("dt", "string", "None"), + Array("key", "int", HiveShim.getEmptyCommentsFieldValue), + Array("value", "string", HiveShim.getEmptyCommentsFieldValue), + Array("dt", "string", HiveShim.getEmptyCommentsFieldValue), Array("", "", ""), Array("# Partition Information", "", ""), Array("# col_name", "data_type", "comment"), Array("", "", ""), - Array("dt", "string", "None")) + Array("dt", "string", HiveShim.getEmptyCommentsFieldValue)) ) { sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") .select('result) @@ -561,11 +561,15 @@ class HiveQuerySuite extends HiveComparisonTest { |WITH serdeproperties('s1'='9') """.stripMargin) } - sql(s"ADD JAR $testJar") - sql( - """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' - |WITH serdeproperties('s1'='9') - """.stripMargin) + // Now only verify 0.12.0, and ignore other versions due to binary compatability + // current TestSerDe.jar is from 0.12.0 + if (HiveShim.version == "0.12.0") { + sql(s"ADD JAR $testJar") + sql( + """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' + |WITH serdeproperties('s1'='9') + """.stripMargin) + } sql("DROP TABLE alter1") } diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala new file mode 100644 index 0000000000000..6dde636965afd --- /dev/null +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.net.URI +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors._ +import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.{io => hadoopIo} +import org.apache.hadoop.mapred.InputFormat +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[hive] object HiveShim { + val version = "0.12.0" + val metastoreDecimal = "decimal" + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[String] + + def processResults(results: JArrayList[String]) = results + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { } + + /** The string used to denote an empty comments field in the schema. */ + def getEmptyCommentsFieldValue = "None" + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd(0), conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + new HiveDecimal(bd) + } + + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + ColumnProjectionUtils.appendReadColumnIDs(conf, ids) + ColumnProjectionUtils.appendReadColumnNames(conf, names) + } + + def getExternalTmpPath(context: Context, uri: URI) = { + context.getExternalTmpFileURI(uri) + } + + def getDataLocationPath(p: Partition) = p.getPartitionPath + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) + +} + +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends FileSinkDesc(dir, tableInfo, compressed) { +} diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala new file mode 100644 index 0000000000000..8678c0c475db4 --- /dev/null +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala @@ -0,0 +1,170 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.StatsSetupConst +import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} +import org.apache.hadoop.mapred.InputFormat +import org.apache.spark.Logging +import org.apache.hadoop.{io => hadoopIo} +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.13.1. + */ +private[hive] object HiveShim { + val version = "0.13.1" + /* + * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded) + * Full support of new decimal feature need to be fixed in seperate PR. + */ + val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[Object] + + def processResults(results: JArrayList[Object]) = { + results.map { r => + r match { + case s: String => s + case a: Array[Object] => a(0).asInstanceOf[String] + } + } + } + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { + context.runSqlHive("CREATE DATABASE default") + context.runSqlHive("USE default") + } + + /* The string used to denote an empty comments field in the schema. */ + def getEmptyCommentsFieldValue = "" + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd, conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + HiveDecimal.create(bd) + } + + /* + * This function in hive-0.13 become private, but we have to do this to walkaround hive bug + */ + private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) { + val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "") + val result: StringBuilder = new StringBuilder(old) + var first: Boolean = old.isEmpty + + for (col <- cols) { + if (first) { + first = false + } else { + result.append(',') + } + result.append(col) + } + conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString) + } + + /* + * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty + */ + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + if (ids != null && ids.size > 0) { + ColumnProjectionUtils.appendReadColumns(conf, ids) + } + if (names != null && names.size > 0) { + appendReadColumnNames(conf, names) + } + } + + def getExternalTmpPath(context: Context, path: Path) = { + context.getExternalTmpPath(path.toUri) + } + + def getDataLocationPath(p: Partition) = p.getDataLocation + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsOf(tbl) + + /* + * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. + * Fix it through wrapper. + * */ + implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = { + var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed) + f.setCompressCodec(w.compressCodec) + f.setCompressType(w.compressType) + f.setTableInfo(w.tableInfo) + f.setDestTableId(w.destTableId) + f + } +} + +/* + * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. + * Fix it through wrapper. + */ +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends Serializable with Logging { + var compressCodec: String = _ + var compressType: String = _ + var destTableId: Int = _ + + def setCompressed(compressed: Boolean) { + this.compressed = compressed + } + + def getDirName = dir + + def setDestTableId(destTableId: Int) { + this.destTableId = destTableId + } + + def setTableInfo(tableInfo: TableDesc) { + this.tableInfo = tableInfo + } + + def setCompressCodec(intermediateCompressorCodec: String) { + compressCodec = intermediateCompressorCodec + } + + def setCompressType(intermediateCompressType: String) { + compressType = intermediateCompressType + } +} From 6a40a76848203d7266c134a26191579138c76903 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Fri, 24 Oct 2014 11:44:48 -0700 Subject: [PATCH 1061/1492] [SPARK-4026][Streaming] Write ahead log management As part of the effort to avoid data loss on Spark Streaming driver failure, we want to implement a write ahead log that can write received data to HDFS. This allows the received data to be persist across driver failures. So when the streaming driver is restarted, it can find and reprocess all the data that were received but not processed. This was primarily implemented by @harishreedharan. This is still WIP, as he is going to improve the unitests by using HDFS mini cluster. Author: Hari Shreedharan Author: Tathagata Das Closes #2882 from tdas/driver-ha-wal and squashes the following commits: e4bee20 [Tathagata Das] Removed synchronized, Path.getFileSystem is threadsafe 55514e2 [Tathagata Das] Minor changes based on PR comments. d29fddd [Tathagata Das] Merge pull request #20 from harishreedharan/driver-ha-wal a317a4d [Hari Shreedharan] Directory deletion should not fail tests 9514dc8 [Tathagata Das] Added unit tests to test reading of corrupted data and other minor edits 3881706 [Tathagata Das] Merge pull request #19 from harishreedharan/driver-ha-wal 4705fff [Hari Shreedharan] Sort listed files by name. Use local files for WAL tests. eb356ca [Tathagata Das] Merge pull request #18 from harishreedharan/driver-ha-wal 82ce56e [Hari Shreedharan] Fix file ordering issue in WALManager tests 5ff90ee [Hari Shreedharan] Fix tests to not ignore ordering and also assert all data is present ef8db09 [Tathagata Das] Merge pull request #17 from harishreedharan/driver-ha-wal 7e40e56 [Hari Shreedharan] Restore old build directory after tests 587b876 [Hari Shreedharan] Fix broken test. Call getFileSystem only from synchronized method. b4be0c1 [Hari Shreedharan] Remove unused method edcbee1 [Hari Shreedharan] Tests reading and writing data using writers now use Minicluster. 5c70d1f [Hari Shreedharan] Remove underlying stream from the WALWriter. 4ab602a [Tathagata Das] Refactored write ahead stuff from streaming.storage to streaming.util b06be2b [Tathagata Das] Adding missing license. 5182ffb [Hari Shreedharan] Added documentation 172358d [Tathagata Das] Pulled WriteAheadLog-related stuff from tdas/spark/tree/driver-ha-working --- .../spark/streaming/util/HdfsUtils.scala | 72 ++++ .../util/WriteAheadLogFileSegment.scala | 20 + .../streaming/util/WriteAheadLogManager.scala | 224 +++++++++++ .../util/WriteAheadLogRandomReader.scala | 55 +++ .../streaming/util/WriteAheadLogReader.scala | 82 ++++ .../streaming/util/WriteAheadLogWriter.scala | 82 ++++ .../streaming/util/WriteAheadLogSuite.scala | 357 ++++++++++++++++++ 7 files changed, 892 insertions(+) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala new file mode 100644 index 0000000000000..491f1175576e6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs._ + +private[streaming] object HdfsUtils { + + def getOutputStream(path: String, conf: Configuration): FSDataOutputStream = { + val dfsPath = new Path(path) + val dfs = getFileSystemForPath(dfsPath, conf) + // If the file exists and we have append support, append instead of creating a new file + val stream: FSDataOutputStream = { + if (dfs.isFile(dfsPath)) { + if (conf.getBoolean("hdfs.append.support", false)) { + dfs.append(dfsPath) + } else { + throw new IllegalStateException("File exists and there is no append support!") + } + } else { + dfs.create(dfsPath) + } + } + stream + } + + def getInputStream(path: String, conf: Configuration): FSDataInputStream = { + val dfsPath = new Path(path) + val dfs = getFileSystemForPath(dfsPath, conf) + val instream = dfs.open(dfsPath) + instream + } + + def checkState(state: Boolean, errorMsg: => String) { + if (!state) { + throw new IllegalStateException(errorMsg) + } + } + + def getBlockLocations(path: String, conf: Configuration): Option[Array[String]] = { + val dfsPath = new Path(path) + val dfs = getFileSystemForPath(dfsPath, conf) + val fileStatus = dfs.getFileStatus(dfsPath) + val blockLocs = Option(dfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen)) + blockLocs.map(_.flatMap(_.getHosts)) + } + + def getFileSystemForPath(path: Path, conf: Configuration): FileSystem = { + // For local file systems, return the raw loca file system, such calls to flush() + // actually flushes the stream. + val fs = path.getFileSystem(conf) + fs match { + case localFs: LocalFileSystem => localFs.getRawFileSystem + case _ => fs + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala new file mode 100644 index 0000000000000..1005a2c8ec303 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogFileSegment.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +/** Class for representing a segment of data in a write ahead log file */ +private[streaming] case class WriteAheadLogFileSegment (path: String, offset: Long, length: Int) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala new file mode 100644 index 0000000000000..70d234320be7c --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogManager.scala @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.{ExecutionContext, Future} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.permission.FsPermission +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import WriteAheadLogManager._ + +/** + * This class manages write ahead log files. + * - Writes records (bytebuffers) to periodically rotating log files. + * - Recovers the log files and the reads the recovered records upon failures. + * - Cleans up old log files. + * + * Uses [[org.apache.spark.streaming.util.WriteAheadLogWriter]] to write + * and [[org.apache.spark.streaming.util.WriteAheadLogReader]] to read. + * + * @param logDirectory Directory when rotating log files will be created. + * @param hadoopConf Hadoop configuration for reading/writing log files. + * @param rollingIntervalSecs The interval in seconds with which logs will be rolled over. + * Default is one minute. + * @param maxFailures Max number of failures that is tolerated for every attempt to write to log. + * Default is three. + * @param callerName Optional name of the class who is using this manager. + * @param clock Optional clock that is used to check for rotation interval. + */ +private[streaming] class WriteAheadLogManager( + logDirectory: String, + hadoopConf: Configuration, + rollingIntervalSecs: Int = 60, + maxFailures: Int = 3, + callerName: String = "", + clock: Clock = new SystemClock + ) extends Logging { + + private val pastLogs = new ArrayBuffer[LogInfo] + private val callerNameTag = + if (callerName.nonEmpty) s" for $callerName" else "" + private val threadpoolName = s"WriteAheadLogManager $callerNameTag" + implicit private val executionContext = ExecutionContext.fromExecutorService( + Utils.newDaemonFixedThreadPool(1, threadpoolName)) + override protected val logName = s"WriteAheadLogManager $callerNameTag" + + private var currentLogPath: Option[String] = None + private var currentLogWriter: WriteAheadLogWriter = null + private var currentLogWriterStartTime: Long = -1L + private var currentLogWriterStopTime: Long = -1L + + initializeOrRecover() + + /** + * Write a byte buffer to the log file. This method synchronously writes the data in the + * ByteBuffer to HDFS. When this method returns, the data is guaranteed to have been flushed + * to HDFS, and will be available for readers to read. + */ + def writeToLog(byteBuffer: ByteBuffer): WriteAheadLogFileSegment = synchronized { + var fileSegment: WriteAheadLogFileSegment = null + var failures = 0 + var lastException: Exception = null + var succeeded = false + while (!succeeded && failures < maxFailures) { + try { + fileSegment = getLogWriter(clock.currentTime).write(byteBuffer) + succeeded = true + } catch { + case ex: Exception => + lastException = ex + logWarning("Failed to write to write ahead log") + resetWriter() + failures += 1 + } + } + if (fileSegment == null) { + logError(s"Failed to write to write ahead log after $failures failures") + throw lastException + } + fileSegment + } + + /** + * Read all the existing logs from the log directory. + * + * Note that this is typically called when the caller is initializing and wants + * to recover past state from the write ahead logs (that is, before making any writes). + * If this is called after writes have been made using this manager, then it may not return + * the latest the records. This does not deal with currently active log files, and + * hence the implementation is kept simple. + */ + def readFromLog(): Iterator[ByteBuffer] = synchronized { + val logFilesToRead = pastLogs.map{ _.path} ++ currentLogPath + logInfo("Reading from the logs: " + logFilesToRead.mkString("\n")) + logFilesToRead.iterator.map { file => + logDebug(s"Creating log reader with $file") + new WriteAheadLogReader(file, hadoopConf) + } flatMap { x => x } + } + + /** + * Delete the log files that are older than the threshold time. + * + * Its important to note that the threshold time is based on the time stamps used in the log + * files, which is usually based on the local system time. So if there is coordination necessary + * between the node calculating the threshTime (say, driver node), and the local system time + * (say, worker node), the caller has to take account of possible time skew. + */ + def cleanupOldLogs(threshTime: Long): Unit = { + val oldLogFiles = synchronized { pastLogs.filter { _.endTime < threshTime } } + logInfo(s"Attempting to clear ${oldLogFiles.size} old log files in $logDirectory " + + s"older than $threshTime: ${oldLogFiles.map { _.path }.mkString("\n")}") + + def deleteFiles() { + oldLogFiles.foreach { logInfo => + try { + val path = new Path(logInfo.path) + val fs = HdfsUtils.getFileSystemForPath(path, hadoopConf) + fs.delete(path, true) + synchronized { pastLogs -= logInfo } + logDebug(s"Cleared log file $logInfo") + } catch { + case ex: Exception => + logWarning(s"Error clearing write ahead log file $logInfo", ex) + } + } + logInfo(s"Cleared log files in $logDirectory older than $threshTime") + } + if (!executionContext.isShutdown) { + Future { deleteFiles() } + } + } + + /** Stop the manager, close any open log writer */ + def stop(): Unit = synchronized { + if (currentLogWriter != null) { + currentLogWriter.close() + } + executionContext.shutdown() + logInfo("Stopped write ahead log manager") + } + + /** Get the current log writer while taking care of rotation */ + private def getLogWriter(currentTime: Long): WriteAheadLogWriter = synchronized { + if (currentLogWriter == null || currentTime > currentLogWriterStopTime) { + resetWriter() + currentLogPath.foreach { + pastLogs += LogInfo(currentLogWriterStartTime, currentLogWriterStopTime, _) + } + currentLogWriterStartTime = currentTime + currentLogWriterStopTime = currentTime + (rollingIntervalSecs * 1000) + val newLogPath = new Path(logDirectory, + timeToLogFile(currentLogWriterStartTime, currentLogWriterStopTime)) + currentLogPath = Some(newLogPath.toString) + currentLogWriter = new WriteAheadLogWriter(currentLogPath.get, hadoopConf) + } + currentLogWriter + } + + /** Initialize the log directory or recover existing logs inside the directory */ + private def initializeOrRecover(): Unit = synchronized { + val logDirectoryPath = new Path(logDirectory) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + + if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { + val logFileInfo = logFilesTologInfo(fileSystem.listStatus(logDirectoryPath).map { _.getPath }) + pastLogs.clear() + pastLogs ++= logFileInfo + logInfo(s"Recovered ${logFileInfo.size} write ahead log files from $logDirectory") + logDebug(s"Recovered files are:\n${logFileInfo.map(_.path).mkString("\n")}") + } + } + + private def resetWriter(): Unit = synchronized { + if (currentLogWriter != null) { + currentLogWriter.close() + currentLogWriter = null + } + } +} + +private[util] object WriteAheadLogManager { + + case class LogInfo(startTime: Long, endTime: Long, path: String) + + val logFileRegex = """log-(\d+)-(\d+)""".r + + def timeToLogFile(startTime: Long, stopTime: Long): String = { + s"log-$startTime-$stopTime" + } + + /** Convert a sequence of files to a sequence of sorted LogInfo objects */ + def logFilesTologInfo(files: Seq[Path]): Seq[LogInfo] = { + files.flatMap { file => + logFileRegex.findFirstIn(file.getName()) match { + case Some(logFileRegex(startTimeStr, stopTimeStr)) => + val startTime = startTimeStr.toLong + val stopTime = stopTimeStr.toLong + Some(LogInfo(startTime, stopTime, file.toString)) + case None => + None + } + }.sortBy { _.startTime } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala new file mode 100644 index 0000000000000..92bad7a882a65 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.io.Closeable +import java.nio.ByteBuffer + +import org.apache.hadoop.conf.Configuration + +/** + * A random access reader for reading write ahead log files written using + * [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. Given the file segment info, + * this reads the record (bytebuffer) from the log file. + */ +private[streaming] class WriteAheadLogRandomReader(path: String, conf: Configuration) + extends Closeable { + + private val instream = HdfsUtils.getInputStream(path, conf) + private var closed = false + + def read(segment: WriteAheadLogFileSegment): ByteBuffer = synchronized { + assertOpen() + instream.seek(segment.offset) + val nextLength = instream.readInt() + HdfsUtils.checkState(nextLength == segment.length, + s"Expected message length to be ${segment.length}, but was $nextLength") + val buffer = new Array[Byte](nextLength) + instream.readFully(buffer) + ByteBuffer.wrap(buffer) + } + + override def close(): Unit = synchronized { + closed = true + instream.close() + } + + private def assertOpen() { + HdfsUtils.checkState(!closed, "Stream is closed. Create a new Reader to read from the file.") + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala new file mode 100644 index 0000000000000..2afc0d1551acf --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogReader.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.io.{Closeable, EOFException} +import java.nio.ByteBuffer + +import org.apache.hadoop.conf.Configuration +import org.apache.spark.Logging + +/** + * A reader for reading write ahead log files written using + * [[org.apache.spark.streaming.util.WriteAheadLogWriter]]. This reads + * the records (bytebuffers) in the log file sequentially and return them as an + * iterator of bytebuffers. + */ +private[streaming] class WriteAheadLogReader(path: String, conf: Configuration) + extends Iterator[ByteBuffer] with Closeable with Logging { + + private val instream = HdfsUtils.getInputStream(path, conf) + private var closed = false + private var nextItem: Option[ByteBuffer] = None + + override def hasNext: Boolean = synchronized { + if (closed) { + return false + } + + if (nextItem.isDefined) { // handle the case where hasNext is called without calling next + true + } else { + try { + val length = instream.readInt() + val buffer = new Array[Byte](length) + instream.readFully(buffer) + nextItem = Some(ByteBuffer.wrap(buffer)) + logTrace("Read next item " + nextItem.get) + true + } catch { + case e: EOFException => + logDebug("Error reading next item, EOF reached", e) + close() + false + case e: Exception => + logWarning("Error while trying to read data from HDFS.", e) + close() + throw e + } + } + } + + override def next(): ByteBuffer = synchronized { + val data = nextItem.getOrElse { + close() + throw new IllegalStateException( + "next called without calling hasNext or after hasNext returned false") + } + nextItem = None // Ensure the next hasNext call loads new data. + data + } + + override def close(): Unit = synchronized { + if (!closed) { + instream.close() + } + closed = true + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala new file mode 100644 index 0000000000000..679f6a6dfd7c1 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogWriter.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.io._ +import java.net.URI +import java.nio.ByteBuffer + +import scala.util.Try + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FSDataOutputStream, FileSystem} + +/** + * A writer for writing byte-buffers to a write ahead log file. + */ +private[streaming] class WriteAheadLogWriter(path: String, hadoopConf: Configuration) + extends Closeable { + + private lazy val stream = HdfsUtils.getOutputStream(path, hadoopConf) + + private lazy val hadoopFlushMethod = { + // Use reflection to get the right flush operation + val cls = classOf[FSDataOutputStream] + Try(cls.getMethod("hflush")).orElse(Try(cls.getMethod("sync"))).toOption + } + + private var nextOffset = stream.getPos() + private var closed = false + + /** Write the bytebuffer to the log file */ + def write(data: ByteBuffer): WriteAheadLogFileSegment = synchronized { + assertOpen() + data.rewind() // Rewind to ensure all data in the buffer is retrieved + val lengthToWrite = data.remaining() + val segment = new WriteAheadLogFileSegment(path, nextOffset, lengthToWrite) + stream.writeInt(lengthToWrite) + if (data.hasArray) { + stream.write(data.array()) + } else { + // If the buffer is not backed by an array, we transfer using temp array + // Note that despite the extra array copy, this should be faster than byte-by-byte copy + while (data.hasRemaining) { + val array = new Array[Byte](data.remaining) + data.get(array) + stream.write(array) + } + } + flush() + nextOffset = stream.getPos() + segment + } + + override def close(): Unit = synchronized { + closed = true + stream.close() + } + + private def flush() { + hadoopFlushMethod.foreach { _.invoke(stream) } + // Useful for local file system where hflush/sync does not work (HADOOP-7844) + stream.getWrappedStream.flush() + } + + private def assertOpen() { + HdfsUtils.checkState(!closed, "Stream is closed. Create a new Writer to write to file.") + } +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala new file mode 100644 index 0000000000000..5eba93c208c50 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -0,0 +1,357 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.util + +import java.io._ +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.{implicitConversions, postfixOps} +import scala.util.Random + +import WriteAheadLogSuite._ +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.spark.util.Utils +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually._ + +class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { + + val hadoopConf = new Configuration() + var tempDir: File = null + var testDir: String = null + var testFile: String = null + var manager: WriteAheadLogManager = null + + before { + tempDir = Files.createTempDir() + testDir = tempDir.toString + testFile = new File(tempDir, Random.nextString(10)).toString + if (manager != null) { + manager.stop() + manager = null + } + } + + after { + FileUtils.deleteQuietly(tempDir) + } + + test("WriteAheadLogWriter - writing data") { + val dataToWrite = generateRandomData() + val segments = writeDataUsingWriter(testFile, dataToWrite) + val writtenData = readDataManually(testFile, segments) + assert(writtenData === dataToWrite) + } + + test("WriteAheadLogWriter - syncing of data by writing and reading immediately") { + val dataToWrite = generateRandomData() + val writer = new WriteAheadLogWriter(testFile, hadoopConf) + dataToWrite.foreach { data => + val segment = writer.write(stringToByteBuffer(data)) + val dataRead = readDataManually(testFile, Seq(segment)).head + assert(data === dataRead) + } + writer.close() + } + + test("WriteAheadLogReader - sequentially reading data") { + val writtenData = generateRandomData() + writeDataManually(writtenData, testFile) + val reader = new WriteAheadLogReader(testFile, hadoopConf) + val readData = reader.toSeq.map(byteBufferToString) + assert(readData === writtenData) + assert(reader.hasNext === false) + intercept[Exception] { + reader.next() + } + reader.close() + } + + test("WriteAheadLogReader - sequentially reading data written with writer") { + val dataToWrite = generateRandomData() + writeDataUsingWriter(testFile, dataToWrite) + val readData = readDataUsingReader(testFile) + assert(readData === dataToWrite) + } + + test("WriteAheadLogReader - reading data written with writer after corrupted write") { + // Write data manually for testing the sequential reader + val dataToWrite = generateRandomData() + writeDataUsingWriter(testFile, dataToWrite) + val fileLength = new File(testFile).length() + + // Append some garbage data to get the effect of a corrupted write + val fw = new FileWriter(testFile, true) + fw.append("This line appended to file!") + fw.close() + + // Verify the data can be read and is same as the one correctly written + assert(readDataUsingReader(testFile) === dataToWrite) + + // Corrupt the last correctly written file + val raf = new FileOutputStream(testFile, true).getChannel() + raf.truncate(fileLength - 1) + raf.close() + + // Verify all the data except the last can be read + assert(readDataUsingReader(testFile) === (dataToWrite.dropRight(1))) + } + + test("WriteAheadLogRandomReader - reading data using random reader") { + // Write data manually for testing the random reader + val writtenData = generateRandomData() + val segments = writeDataManually(writtenData, testFile) + + // Get a random order of these segments and read them back + val writtenDataAndSegments = writtenData.zip(segments).toSeq.permutations.take(10).flatten + val reader = new WriteAheadLogRandomReader(testFile, hadoopConf) + writtenDataAndSegments.foreach { case (data, segment) => + assert(data === byteBufferToString(reader.read(segment))) + } + reader.close() + } + + test("WriteAheadLogRandomReader - reading data using random reader written with writer") { + // Write data using writer for testing the random reader + val data = generateRandomData() + val segments = writeDataUsingWriter(testFile, data) + + // Read a random sequence of segments and verify read data + val dataAndSegments = data.zip(segments).toSeq.permutations.take(10).flatten + val reader = new WriteAheadLogRandomReader(testFile, hadoopConf) + dataAndSegments.foreach { case (data, segment) => + assert(data === byteBufferToString(reader.read(segment))) + } + reader.close() + } + + test("WriteAheadLogManager - write rotating logs") { + // Write data using manager + val dataToWrite = generateRandomData() + writeDataUsingManager(testDir, dataToWrite) + + // Read data manually to verify the written data + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val writtenData = logFiles.flatMap { file => readDataManually(file)} + assert(writtenData === dataToWrite) + } + + test("WriteAheadLogManager - read rotating logs") { + // Write data manually for testing reading through manager + val writtenData = (1 to 10).map { i => + val data = generateRandomData() + val file = testDir + s"/log-$i-$i" + writeDataManually(data, file) + data + }.flatten + + val logDirectoryPath = new Path(testDir) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + assert(fileSystem.exists(logDirectoryPath) === true) + + // Read data using manager and verify + val readData = readDataUsingManager(testDir) + assert(readData === writtenData) + } + + test("WriteAheadLogManager - recover past logs when creating new manager") { + // Write data with manager, recover with new manager and verify + val dataToWrite = generateRandomData() + writeDataUsingManager(testDir, dataToWrite) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + val readData = readDataUsingManager(testDir) + assert(dataToWrite === readData) + } + + test("WriteAheadLogManager - cleanup old logs") { + // Write data with manager, recover with new manager and verify + val manualClock = new ManualClock + val dataToWrite = generateRandomData() + manager = writeDataUsingManager(testDir, dataToWrite, manualClock, stopManager = false) + val logFiles = getLogFilesInDirectory(testDir) + assert(logFiles.size > 1) + manager.cleanupOldLogs(manualClock.currentTime() / 2) + eventually(timeout(1 second), interval(10 milliseconds)) { + assert(getLogFilesInDirectory(testDir).size < logFiles.size) + } + } + + test("WriteAheadLogManager - handling file errors while reading rotating logs") { + // Generate a set of log files + val manualClock = new ManualClock + val dataToWrite1 = generateRandomData() + writeDataUsingManager(testDir, dataToWrite1, manualClock) + val logFiles1 = getLogFilesInDirectory(testDir) + assert(logFiles1.size > 1) + + + // Recover old files and generate a second set of log files + val dataToWrite2 = generateRandomData() + manualClock.addToTime(100000) + writeDataUsingManager(testDir, dataToWrite2, manualClock) + val logFiles2 = getLogFilesInDirectory(testDir) + assert(logFiles2.size > logFiles1.size) + + // Read the files and verify that all the written data can be read + val readData1 = readDataUsingManager(testDir) + assert(readData1 === (dataToWrite1 ++ dataToWrite2)) + + // Corrupt the first set of files so that they are basically unreadable + logFiles1.foreach { f => + val raf = new FileOutputStream(f, true).getChannel() + raf.truncate(1) + raf.close() + } + + // Verify that the corrupted files do not prevent reading of the second set of data + val readData = readDataUsingManager(testDir) + assert(readData === dataToWrite2) + } +} + +object WriteAheadLogSuite { + + private val hadoopConf = new Configuration() + + /** Write data to a file directly and return an array of the file segments written. */ + def writeDataManually(data: Seq[String], file: String): Seq[WriteAheadLogFileSegment] = { + val segments = new ArrayBuffer[WriteAheadLogFileSegment]() + val writer = HdfsUtils.getOutputStream(file, hadoopConf) + data.foreach { item => + val offset = writer.getPos + val bytes = Utils.serialize(item) + writer.writeInt(bytes.size) + writer.write(bytes) + segments += WriteAheadLogFileSegment(file, offset, bytes.size) + } + writer.close() + segments + } + + /** + * Write data to a file using the writer class and return an array of the file segments written. + */ + def writeDataUsingWriter(filePath: String, data: Seq[String]): Seq[WriteAheadLogFileSegment] = { + val writer = new WriteAheadLogWriter(filePath, hadoopConf) + val segments = data.map { + item => writer.write(item) + } + writer.close() + segments + } + + /** Write data to rotating files in log directory using the manager class. */ + def writeDataUsingManager( + logDirectory: String, + data: Seq[String], + manualClock: ManualClock = new ManualClock, + stopManager: Boolean = true + ): WriteAheadLogManager = { + if (manualClock.currentTime < 100000) manualClock.setTime(10000) + val manager = new WriteAheadLogManager(logDirectory, hadoopConf, + rollingIntervalSecs = 1, callerName = "WriteAheadLogSuite", clock = manualClock) + // Ensure that 500 does not get sorted after 2000, so put a high base value. + data.foreach { item => + manualClock.addToTime(500) + manager.writeToLog(item) + } + if (stopManager) manager.stop() + manager + } + + /** Read data from a segments of a log file directly and return the list of byte buffers.*/ + def readDataManually(file: String, segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { + val reader = HdfsUtils.getInputStream(file, hadoopConf) + segments.map { x => + reader.seek(x.offset) + val data = new Array[Byte](x.length) + reader.readInt() + reader.readFully(data) + Utils.deserialize[String](data) + } + } + + /** Read all the data from a log file directly and return the list of byte buffers. */ + def readDataManually(file: String): Seq[String] = { + val reader = HdfsUtils.getInputStream(file, hadoopConf) + val buffer = new ArrayBuffer[String] + try { + while (true) { + // Read till EOF is thrown + val length = reader.readInt() + val bytes = new Array[Byte](length) + reader.read(bytes) + buffer += Utils.deserialize[String](bytes) + } + } catch { + case ex: EOFException => + } finally { + reader.close() + } + buffer + } + + /** Read all the data from a log file using reader class and return the list of byte buffers. */ + def readDataUsingReader(file: String): Seq[String] = { + val reader = new WriteAheadLogReader(file, hadoopConf) + val readData = reader.toList.map(byteBufferToString) + reader.close() + readData + } + + /** Read all the data in the log file in a directory using the manager class. */ + def readDataUsingManager(logDirectory: String): Seq[String] = { + val manager = new WriteAheadLogManager(logDirectory, hadoopConf, + callerName = "WriteAheadLogSuite") + val data = manager.readFromLog().map(byteBufferToString).toSeq + manager.stop() + data + } + + /** Get the log files in a direction */ + def getLogFilesInDirectory(directory: String): Seq[String] = { + val logDirectoryPath = new Path(directory) + val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) + + if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { + fileSystem.listStatus(logDirectoryPath).map { + _.getPath.toString.stripPrefix("file:") + }.sorted + } else { + Seq.empty + } + } + + def generateRandomData(): Seq[String] = { + (1 to 100).map { _.toString } + } + + implicit def stringToByteBuffer(str: String): ByteBuffer = { + ByteBuffer.wrap(Utils.serialize(str)) + } + + implicit def byteBufferToString(byteBuffer: ByteBuffer): String = { + Utils.deserialize[String](byteBuffer.array) + } +} From 7aacb7bfad4ec73fd8f18555c72ef6962c14358f Mon Sep 17 00:00:00 2001 From: Li Zhihui Date: Fri, 24 Oct 2014 13:01:36 -0700 Subject: [PATCH 1062/1492] [SPARK-2713] Executors of same application in same host should only download files & jars once If Spark lunched multiple executors in one host for one application, every executor would download it dependent files and jars (if not using local: url) independently. It maybe result in huge latency. In my case, it result in 20 seconds latency to download dependent jars(size about 17M) when I lunched 32 executors in every host(total 4 hosts). This patch will cache downloaded files and jars for executors to reduce network throughput and download latency. In my case, the latency was reduced from 20 seconds to less than 1 second. Author: Li Zhihui Author: li-zhihui Closes #1616 from li-zhihui/cachefiles and squashes the following commits: 36940df [Li Zhihui] Close cache for local mode 935fed6 [Li Zhihui] Clean code. f9330d4 [Li Zhihui] Clean code again 7050d46 [Li Zhihui] Clean code 074a422 [Li Zhihui] Fix: deal with spark.files.overwrite 03ed3a8 [li-zhihui] rename cache file name as XXXXXXXXX_cache 2766055 [li-zhihui] Use url.hashCode + timestamp as cachedFileName 76a7b66 [Li Zhihui] Clean code & use applcation work directory as cache directory 3510eb0 [Li Zhihui] Keep fetchFile private 2ffd742 [Li Zhihui] add comment for FileLock e0ebd48 [Li Zhihui] Try and finally lock.release 7fb7c0b [Li Zhihui] Release lock before copy files 6b997bf [Li Zhihui] Executors of same application in same host should only download files & jars once --- .../scala/org/apache/spark/SparkContext.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 10 ++- .../scala/org/apache/spark/util/Utils.scala | 87 ++++++++++++++++--- 3 files changed, 82 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ac7935b8c231e..55602a90829d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -837,11 +837,12 @@ class SparkContext(config: SparkConf) extends Logging { case "local" => "file:" + uri.getPath case _ => path } - addedFiles(key) = System.currentTimeMillis + val timestamp = System.currentTimeMillis + addedFiles(key) = timestamp // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). Utils.fetchFile(path, new File(SparkFiles.getRootDirectory()), conf, env.securityManager, - hadoopConfiguration) + hadoopConfiguration, timestamp, useCache = false) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) postEnvironmentUpdate() diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 616c7e6a46368..0b75b9b21fb82 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -322,14 +322,16 @@ private[spark] class Executor( // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, env.securityManager, - hadoopConf) + // Fetch file with useCache mode, close cache for local mode. + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentFiles(name) = timestamp } for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, env.securityManager, - hadoopConf) + // Fetch file with useCache mode, close cache for local mode. + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, + env.securityManager, hadoopConf, timestamp, useCache = !isLocal) currentJars(name) = timestamp // Add it to our class loader val localName = name.split("/").last 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 0aeff6455b3fe..ccbddd985ae0a 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -347,15 +347,84 @@ private[spark] object Utils extends Logging { } /** - * Download a file requested by the executor. Supports fetching the file in a variety of ways, + * Download a file to target directory. Supports fetching the file in a variety of ways, + * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. + * + * If `useCache` is true, first attempts to fetch the file to a local cache that's shared + * across executors running the same application. `useCache` is used mainly for + * the executors, and not in local mode. + * + * Throws SparkException if the target file already exists and has different contents than + * the requested file. + */ + def fetchFile( + url: String, + targetDir: File, + conf: SparkConf, + securityMgr: SecurityManager, + hadoopConf: Configuration, + timestamp: Long, + useCache: Boolean) { + val fileName = url.split("/").last + val targetFile = new File(targetDir, fileName) + if (useCache) { + val cachedFileName = s"${url.hashCode}${timestamp}_cache" + val lockFileName = s"${url.hashCode}${timestamp}_lock" + val localDir = new File(getLocalDir(conf)) + val lockFile = new File(localDir, lockFileName) + val raf = new RandomAccessFile(lockFile, "rw") + // Only one executor entry. + // The FileLock is only used to control synchronization for executors download file, + // it's always safe regardless of lock type (mandatory or advisory). + val lock = raf.getChannel().lock() + val cachedFile = new File(localDir, cachedFileName) + try { + if (!cachedFile.exists()) { + doFetchFile(url, localDir, cachedFileName, conf, securityMgr, hadoopConf) + } + } finally { + lock.release() + } + if (targetFile.exists && !Files.equal(cachedFile, targetFile)) { + if (conf.getBoolean("spark.files.overwrite", false)) { + targetFile.delete() + logInfo((s"File $targetFile exists and does not match contents of $url, " + + s"replacing it with $url")) + } else { + throw new SparkException(s"File $targetFile exists and does not match contents of $url") + } + } + Files.copy(cachedFile, targetFile) + } else { + doFetchFile(url, targetDir, fileName, conf, securityMgr, hadoopConf) + } + + // Decompress the file if it's a .tar or .tar.gz + if (fileName.endsWith(".tar.gz") || fileName.endsWith(".tgz")) { + logInfo("Untarring " + fileName) + Utils.execute(Seq("tar", "-xzf", fileName), targetDir) + } else if (fileName.endsWith(".tar")) { + logInfo("Untarring " + fileName) + Utils.execute(Seq("tar", "-xf", fileName), targetDir) + } + // Make the file executable - That's necessary for scripts + FileUtil.chmod(targetFile.getAbsolutePath, "a+x") + } + + /** + * Download a file to target directory. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. * * Throws SparkException if the target file already exists and has different contents than * the requested file. */ - def fetchFile(url: String, targetDir: File, conf: SparkConf, securityMgr: SecurityManager, - hadoopConf: Configuration) { - val filename = url.split("/").last + private def doFetchFile( + url: String, + targetDir: File, + filename: String, + conf: SparkConf, + securityMgr: SecurityManager, + hadoopConf: Configuration) { val tempDir = getLocalDir(conf) val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) val targetFile = new File(targetDir, filename) @@ -443,16 +512,6 @@ private[spark] object Utils extends Logging { } Files.move(tempFile, targetFile) } - // Decompress the file if it's a .tar or .tar.gz - if (filename.endsWith(".tar.gz") || filename.endsWith(".tgz")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xzf", filename), targetDir) - } else if (filename.endsWith(".tar")) { - logInfo("Untarring " + filename) - Utils.execute(Seq("tar", "-xf", filename), targetDir) - } - // Make the file executable - That's necessary for scripts - FileUtil.chmod(targetFile.getAbsolutePath, "a+x") } /** From 30ea2868e7afbec20bfc83818249b6d2d7dc6aec Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 24 Oct 2014 13:04:35 -0700 Subject: [PATCH 1063/1492] [SPARK-4076] Parameter expansion in spark-config is wrong In sbin/spark-config.sh, parameter expansion is used to extract source root as follows. this="${BASH_SOURCE-$0}" I think, the parameter expansion should be ":" instead of "". If we use "-" and BASH_SOURCE="", (empty character is set, not unset), "" (empty character) is set to $this. Author: Kousuke Saruta Closes #2930 from sarutak/SPARK-4076 and squashes the following commits: 32a0370 [Kousuke Saruta] Fixed wrong parameter expansion --- sbin/spark-config.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index 1d154e62ed5b6..b0361d72d3f2c 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -20,7 +20,7 @@ # also should not be passed any arguments, since we need original $* # resolve links - $0 may be a softlink -this="${BASH_SOURCE-$0}" +this="${BASH_SOURCE:-$0}" common_bin="$(cd -P -- "$(dirname -- "$this")" && pwd -P)" script="$(basename -- "$this")" this="$common_bin/$script" From 098f83c7ccd7dad9f9228596da69fe5f55711a52 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 24 Oct 2014 13:08:21 -0700 Subject: [PATCH 1064/1492] [SPARK-4075] [Deploy] Jar url validation is not enough for Jar file In deploy.ClientArguments.isValidJarUrl, the url is checked as follows. def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") So, it allows like 'hdfs:file.jar' (no authority). Author: Kousuke Saruta Closes #2925 from sarutak/uri-syntax-check-improvement and squashes the following commits: cf06173 [Kousuke Saruta] Improved URI syntax checking --- .../org/apache/spark/deploy/ClientArguments.scala | 11 ++++++++++- .../scala/org/apache/spark/deploy/ClientSuite.scala | 6 ++++++ 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 39150deab863c..4e802e02c4149 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy +import java.net.{URI, URISyntaxException} + import scala.collection.mutable.ListBuffer import org.apache.log4j.Level @@ -114,5 +116,12 @@ private[spark] class ClientArguments(args: Array[String]) { } object ClientArguments { - def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") + def isValidJarUrl(s: String): Boolean = { + try { + val uri = new URI(s) + uri.getScheme != null && uri.getAuthority != null && s.endsWith("jar") + } catch { + case _: URISyntaxException => false + } + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 4161aede1d1d0..94a2bdd74e744 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -29,6 +29,12 @@ class ClientSuite extends FunSuite with Matchers { ClientArguments.isValidJarUrl("hdfs://someHost:1234/foo") should be (false) ClientArguments.isValidJarUrl("/missing/a/protocol/jarfile.jar") should be (false) ClientArguments.isValidJarUrl("not-even-a-path.jar") should be (false) + + // No authority + ClientArguments.isValidJarUrl("hdfs:someHost:1234/jarfile.jar") should be (false) + + // Invalid syntax + ClientArguments.isValidJarUrl("hdfs:") should be (false) } } From b563987e8dffc2aed1a834d555589a41cfb2a706 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Fri, 24 Oct 2014 13:32:23 -0700 Subject: [PATCH 1065/1492] [SPARK-4013] Do not create multiple actor systems on each executor In the existing code, each coarse-grained executor has two concurrently running actor systems. This causes many more error messages to be logged than necessary when the executor is lost or killed because we receive a disassociation event for each of these actor systems. This is blocking #2840. Author: Andrew Or Closes #2863 from andrewor14/executor-actor-system and squashes the following commits: 44ce2e0 [Andrew Or] Avoid starting two actor systems on each executor --- .../scala/org/apache/spark/SparkContext.scala | 12 ++--- .../scala/org/apache/spark/SparkEnv.scala | 49 +++++++++++++++++-- .../CoarseGrainedExecutorBackend.scala | 11 +++-- .../org/apache/spark/executor/Executor.scala | 11 +++-- 4 files changed, 61 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 55602a90829d5..4565832334420 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -209,16 +209,10 @@ class SparkContext(config: SparkConf) extends Logging { // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - // Create the Spark execution environment (cache, map output tracker, etc) conf.set("spark.executor.id", "driver") - private[spark] val env = SparkEnv.create( - conf, - "", - conf.get("spark.driver.host"), - conf.get("spark.driver.port").toInt, - isDriver = true, - isLocal = isLocal, - listenerBus = listenerBus) + + // Create the Spark execution environment (cache, map output tracker, etc) + private[spark] val env = SparkEnv.createDriverEnv(conf, isLocal, listenerBus) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 906a00b0bd17c..5c076e5f1c11d 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -144,14 +144,46 @@ object SparkEnv extends Logging { env } - private[spark] def create( + /** + * Create a SparkEnv for the driver. + */ + private[spark] def createDriverEnv( + conf: SparkConf, + isLocal: Boolean, + listenerBus: LiveListenerBus): SparkEnv = { + assert(conf.contains("spark.driver.host"), "spark.driver.host is not set on the driver!") + assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") + val hostname = conf.get("spark.driver.host") + val port = conf.get("spark.driver.port").toInt + create(conf, "", hostname, port, true, isLocal, listenerBus) + } + + /** + * Create a SparkEnv for an executor. + * In coarse-grained mode, the executor provides an actor system that is already instantiated. + */ + private[spark] def createExecutorEnv( + conf: SparkConf, + executorId: String, + hostname: String, + port: Int, + isLocal: Boolean, + actorSystem: ActorSystem = null): SparkEnv = { + create(conf, executorId, hostname, port, false, isLocal, defaultActorSystem = actorSystem) + } + + /** + * Helper method to create a SparkEnv for a driver or an executor. + */ + private def create( conf: SparkConf, executorId: String, hostname: String, port: Int, isDriver: Boolean, isLocal: Boolean, - listenerBus: LiveListenerBus = null): SparkEnv = { + listenerBus: LiveListenerBus = null, + defaultActorSystem: ActorSystem = null): SparkEnv = { // Listener bus is only used on the driver if (isDriver) { @@ -159,9 +191,16 @@ object SparkEnv extends Logging { } val securityManager = new SecurityManager(conf) - val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName - val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - actorSystemName, hostname, port, conf, securityManager) + + // If an existing actor system is already provided, use it. + // This is the case when an executor is launched in coarse-grained mode. + val (actorSystem, boundPort) = + Option(defaultActorSystem) match { + case Some(as) => (as, port) + case None => + val actorSystemName = if (isDriver) driverActorSystemName else executorActorSystemName + AkkaUtils.createActorSystem(actorSystemName, hostname, port, conf, securityManager) + } // Figure out which port Akka actually bound to in case the original port is 0 or occupied. // This is so that we tell the executors the correct port to connect to. diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index c40a3e16675ad..697154d762d41 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import scala.concurrent.Await -import akka.actor.{Actor, ActorSelection, Props} +import akka.actor.{Actor, ActorSelection, ActorSystem, Props} import akka.pattern.Patterns import akka.remote.{RemotingLifecycleEvent, DisassociatedEvent} @@ -38,7 +38,8 @@ private[spark] class CoarseGrainedExecutorBackend( executorId: String, hostPort: String, cores: Int, - sparkProperties: Seq[(String, String)]) + sparkProperties: Seq[(String, String)], + actorSystem: ActorSystem) extends Actor with ActorLogReceive with ExecutorBackend with Logging { Utils.checkHostPort(hostPort, "Expected hostport") @@ -57,8 +58,8 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, - false) + val (hostname, _) = Utils.parseHostPort(hostPort) + executor = new Executor(executorId, hostname, sparkProperties, isLocal = false, actorSystem) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -135,7 +136,7 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( Props(classOf[CoarseGrainedExecutorBackend], - driverUrl, executorId, sparkHostPort, cores, props), + driverUrl, executorId, sparkHostPort, cores, props, actorSystem), name = "Executor") workerUrl.foreach { url => actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 0b75b9b21fb82..70a46c75f42c4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -26,6 +26,8 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal +import akka.actor.ActorSystem + import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ @@ -35,12 +37,14 @@ import org.apache.spark.util.{AkkaUtils, Utils} /** * Spark executor used with Mesos, YARN, and the standalone scheduler. + * In coarse-grained mode, an existing actor system is provided. */ private[spark] class Executor( executorId: String, slaveHostname: String, properties: Seq[(String, String)], - isLocal: Boolean = false) + isLocal: Boolean = false, + actorSystem: ActorSystem = null) extends Logging { // Application dependencies (added through SparkContext) that we've fetched so far on this node. @@ -77,8 +81,9 @@ private[spark] class Executor( conf.set("spark.executor.id", "executor." + executorId) private val env = { if (!isLocal) { - val _env = SparkEnv.create(conf, executorId, slaveHostname, 0, - isDriver = false, isLocal = false) + val port = conf.getInt("spark.executor.port", 0) + val _env = SparkEnv.createExecutorEnv( + conf, executorId, slaveHostname, port, isLocal, actorSystem) SparkEnv.set(_env) _env.metricsSystem.registerSource(executorSource) _env From f80dcf2aeef762ca370e91d2c7d6e4f7894c3cd8 Mon Sep 17 00:00:00 2001 From: Nan Zhu Date: Fri, 24 Oct 2014 13:46:45 -0700 Subject: [PATCH 1066/1492] [SPARK-4067] refactor ExecutorUncaughtExceptionHandler https://issues.apache.org/jira/browse/SPARK-4067 currently , we call Utils.tryOrExit everywhere AppClient Executor TaskSchedulerImpl It makes the name of ExecutorUncaughtExceptionHandler unfit to the real case.... Author: Nan Zhu Author: Nan Zhu Closes #2913 from CodingCat/SPARK-4067 and squashes the following commits: 035ee3d [Nan Zhu] make RAT happy e62e416 [Nan Zhu] add some general Exit code a10b63f [Nan Zhu] refactor --- .../org/apache/spark/executor/Executor.scala | 6 ++-- .../spark/executor/ExecutorExitCode.scala | 12 ++----- .../org/apache/spark/util/SparkExitCode.scala | 32 +++++++++++++++++++ .../SparkUncaughtExceptionHandler.scala} | 13 ++++---- .../scala/org/apache/spark/util/Utils.scala | 4 +-- 5 files changed, 45 insertions(+), 22 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/SparkExitCode.scala rename core/src/main/scala/org/apache/spark/{executor/ExecutorUncaughtExceptionHandler.scala => util/SparkUncaughtExceptionHandler.scala} (80%) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 70a46c75f42c4..2889f59e33e84 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -33,7 +33,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.{SparkUncaughtExceptionHandler, AkkaUtils, Utils} /** * Spark executor used with Mesos, YARN, and the standalone scheduler. @@ -72,7 +72,7 @@ private[spark] class Executor( // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire // executor process to avoid surprising stalls. - Thread.setDefaultUncaughtExceptionHandler(ExecutorUncaughtExceptionHandler) + Thread.setDefaultUncaughtExceptionHandler(SparkUncaughtExceptionHandler) } val executorSource = new ExecutorSource(this, executorId) @@ -258,7 +258,7 @@ private[spark] class Executor( // Don't forcibly exit unless the exception was inherently fatal, to avoid // stopping other tasks unnecessarily. if (Utils.isFatalError(t)) { - ExecutorUncaughtExceptionHandler.uncaughtException(t) + SparkUncaughtExceptionHandler.uncaughtException(t) } } } finally { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 38be2c58b333f..52862ae0ca5e4 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -17,6 +17,8 @@ package org.apache.spark.executor +import org.apache.spark.util.SparkExitCode._ + /** * These are exit codes that executors should use to provide the master with information about * executor failures assuming that cluster management framework can capture the exit codes (but @@ -27,16 +29,6 @@ package org.apache.spark.executor */ private[spark] object ExecutorExitCode { - /** The default uncaught exception handler was reached. */ - val UNCAUGHT_EXCEPTION = 50 - - /** The default uncaught exception handler was called and an exception was encountered while - logging the exception. */ - val UNCAUGHT_EXCEPTION_TWICE = 51 - - /** The default uncaught exception handler was reached, and the uncaught exception was an - OutOfMemoryError. */ - val OOM = 52 /** DiskStore failed to create a local temporary directory after many attempts. */ val DISK_STORE_FAILED_TO_CREATE_DIR = 53 diff --git a/core/src/main/scala/org/apache/spark/util/SparkExitCode.scala b/core/src/main/scala/org/apache/spark/util/SparkExitCode.scala new file mode 100644 index 0000000000000..c93b1cca9f564 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/SparkExitCode.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +private[spark] object SparkExitCode { + /** The default uncaught exception handler was reached. */ + val UNCAUGHT_EXCEPTION = 50 + + /** The default uncaught exception handler was called and an exception was encountered while + logging the exception. */ + val UNCAUGHT_EXCEPTION_TWICE = 51 + + /** The default uncaught exception handler was reached, and the uncaught exception was an + OutOfMemoryError. */ + val OOM = 52 + +} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala similarity index 80% rename from core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala rename to core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala index b0e984c03964c..ad3db1fbb57ed 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorUncaughtExceptionHandler.scala +++ b/core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala @@ -15,17 +15,16 @@ * limitations under the License. */ -package org.apache.spark.executor +package org.apache.spark.util import org.apache.spark.Logging -import org.apache.spark.util.Utils /** * The default uncaught exception handler for Executors terminates the whole process, to avoid * getting into a bad state indefinitely. Since Executors are relatively lightweight, it's better * to fail fast when things go wrong. */ -private[spark] object ExecutorUncaughtExceptionHandler +private[spark] object SparkUncaughtExceptionHandler extends Thread.UncaughtExceptionHandler with Logging { override def uncaughtException(thread: Thread, exception: Throwable) { @@ -36,14 +35,14 @@ private[spark] object ExecutorUncaughtExceptionHandler // (If we do, we will deadlock.) if (!Utils.inShutdown()) { if (exception.isInstanceOf[OutOfMemoryError]) { - System.exit(ExecutorExitCode.OOM) + System.exit(SparkExitCode.OOM) } else { - System.exit(ExecutorExitCode.UNCAUGHT_EXCEPTION) + System.exit(SparkExitCode.UNCAUGHT_EXCEPTION) } } } catch { - case oom: OutOfMemoryError => Runtime.getRuntime.halt(ExecutorExitCode.OOM) - case t: Throwable => Runtime.getRuntime.halt(ExecutorExitCode.UNCAUGHT_EXCEPTION_TWICE) + case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM) + case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE) } } 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 ccbddd985ae0a..65bdbaae65463 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -43,7 +43,7 @@ import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark._ -import org.apache.spark.executor.ExecutorUncaughtExceptionHandler +import org.apache.spark.util.SparkUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -965,7 +965,7 @@ private[spark] object Utils extends Logging { block } catch { case e: ControlThrowable => throw e - case t: Throwable => ExecutorUncaughtExceptionHandler.uncaughtException(t) + case t: Throwable => SparkUncaughtExceptionHandler.uncaughtException(t) } } From 07e439b4fe1cea4cee8ec8e39803b8349078f119 Mon Sep 17 00:00:00 2001 From: Grace Date: Fri, 24 Oct 2014 13:48:08 -0700 Subject: [PATCH 1067/1492] [GraphX] Modify option name according to example doc in SynthBenchmark Now graphx.SynthBenchmark example has an option of iteration number named as "niter". However, in its document, it is named as "niters". The mismatch between the implementation and document causes certain IllegalArgumentException while trying that example. Author: Grace Closes #2888 from GraceH/synthbenchmark and squashes the following commits: f101ee1 [Grace] Modify option name according to example doc --- .../scala/org/apache/spark/examples/graphx/SynthBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala index 05676021718d9..3ec20d594b784 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/SynthBenchmark.scala @@ -67,7 +67,7 @@ object SynthBenchmark { options.foreach { case ("app", v) => app = v - case ("niter", v) => niter = v.toInt + case ("niters", v) => niter = v.toInt case ("nverts", v) => numVertices = v.toInt case ("numEPart", v) => numEPart = Some(v.toInt) case ("partStrategy", v) => partitionStrategy = Some(PartitionStrategy.fromString(v)) From 3a906c6631a914da8ede3111c63f89a0dac3f369 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 24 Oct 2014 14:03:03 -0700 Subject: [PATCH 1068/1492] [HOTFIX][SQL] Remove sleep on reset() failure. Author: Michael Armbrust Closes #2934 from marmbrus/patch-2 and squashes the following commits: a96dab2 [Michael Armbrust] Remove sleep on reset() failure. --- .../src/main/scala/org/apache/spark/sql/hive/TestHive.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 0f74fe8943706..c6ff4ea6de594 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -438,10 +438,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { } catch { case e: Exception => logError("FATAL ERROR: Failed to reset TestDB state.", e) - // At this point there is really no reason to continue, but the test framework traps exits. - // So instead we just pause forever so that at least the developer can see where things - // started to go wrong. - Thread.sleep(100000) } } } From 6c98c29ae0033556fd4424f41d1de005c509e511 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Oct 2014 15:06:15 -0700 Subject: [PATCH 1069/1492] [SPARK-4080] Only throw IOException from [write|read][Object|External] If classes implementing Serializable or Externalizable interfaces throw exceptions other than IOException or ClassNotFoundException from their (de)serialization methods, then this results in an unhelpful "IOException: unexpected exception type" rather than the actual exception that produced the (de)serialization error. This patch fixes this by adding a utility method that re-wraps any uncaught exceptions in IOException (unless they are already instances of IOException). Author: Josh Rosen Closes #2932 from JoshRosen/SPARK-4080 and squashes the following commits: cd3a9be [Josh Rosen] [SPARK-4080] Only throw IOException from [write|read][Object|External]. --- .../scala/org/apache/spark/Accumulators.scala | 3 ++- .../main/scala/org/apache/spark/Partitioner.scala | 4 ++-- .../org/apache/spark/SerializableWritable.scala | 5 +++-- .../apache/spark/broadcast/HttpBroadcast.scala | 4 ++-- .../apache/spark/broadcast/TorrentBroadcast.scala | 6 +++--- .../spark/deploy/master/ApplicationInfo.scala | 3 ++- .../apache/spark/deploy/master/DriverInfo.scala | 3 ++- .../apache/spark/deploy/master/WorkerInfo.scala | 2 +- .../scala/org/apache/spark/rdd/CartesianRDD.scala | 3 ++- .../scala/org/apache/spark/rdd/CoGroupedRDD.scala | 3 ++- .../scala/org/apache/spark/rdd/CoalescedRDD.scala | 3 ++- .../apache/spark/rdd/ParallelCollectionRDD.scala | 4 ++-- .../spark/rdd/PartitionerAwareUnionRDD.scala | 3 ++- .../scala/org/apache/spark/rdd/UnionRDD.scala | 3 ++- .../apache/spark/rdd/ZippedPartitionsRDD.scala | 3 ++- .../org/apache/spark/scheduler/MapStatus.scala | 9 +++++---- .../org/apache/spark/scheduler/TaskResult.scala | 4 ++-- .../apache/spark/serializer/JavaSerializer.scala | 4 ++-- .../org/apache/spark/storage/BlockManagerId.scala | 4 ++-- .../spark/storage/BlockManagerMessages.scala | 6 ++++-- .../org/apache/spark/storage/StorageLevel.scala | 5 +++-- .../apache/spark/util/SerializableBuffer.scala | 4 ++-- .../main/scala/org/apache/spark/util/Utils.scala | 15 +++++++++++++++ .../spark/streaming/flume/FlumeInputDStream.scala | 4 ++-- .../org/apache/spark/streaming/DStreamGraph.scala | 5 +++-- .../streaming/api/python/PythonDStream.scala | 5 +++-- .../apache/spark/streaming/dstream/DStream.scala | 6 +++--- .../streaming/dstream/DStreamCheckpointData.scala | 5 +++-- .../streaming/dstream/FileInputDStream.scala | 4 ++-- .../apache/spark/streaming/TestSuiteBase.scala | 4 ++-- 30 files changed, 84 insertions(+), 52 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 12f2fe031cb1d..2301caafb07ff 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.Map import scala.reflect.ClassTag import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.Utils /** * A data type that can be accumulated, ie has an commutative and associative "add" operation, @@ -126,7 +127,7 @@ class Accumulable[R, T] ( } // Called by Java when deserializing an object - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() value_ = zero deserialized = true diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 37053bb6f37ad..e53a78ead2c0e 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -204,7 +204,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } @throws(classOf[IOException]) - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer sfactory match { case js: JavaSerializer => out.defaultWriteObject() @@ -222,7 +222,7 @@ class RangePartitioner[K : Ordering : ClassTag, V]( } @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer sfactory match { case js: JavaSerializer => in.defaultReadObject() diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index e50b9ac2291f9..55cb25946c2ad 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -24,18 +24,19 @@ import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils @DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.defaultWriteObject() new ObjectWritable(t).write(out) } - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() val ow = new ObjectWritable() ow.setConf(new Configuration()) diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 4cd4f4f96fd16..7dade04273b08 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -72,13 +72,13 @@ private[spark] class HttpBroadcast[T: ClassTag]( } /** Used by the JVM when serializing this object. */ - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { assertValid() out.defaultWriteObject() } /** Used by the JVM when deserializing this object. */ - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() HttpBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(blockId) match { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 99af2e9608ea7..75e64c1bf401e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -28,7 +28,7 @@ import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} -import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.{ByteBufferInputStream, Utils} import org.apache.spark.util.io.ByteArrayChunkOutputStream /** @@ -152,13 +152,13 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } /** Used by the JVM when serializing this object. */ - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { assertValid() out.defaultWriteObject() } /** Used by the JVM when deserializing this object. */ - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() TorrentBroadcast.synchronized { setConf(SparkEnv.get.conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index c3ca43f8d0734..6ba395be1cc2c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import akka.actor.ActorRef import org.apache.spark.deploy.ApplicationDescription +import org.apache.spark.util.Utils private[spark] class ApplicationInfo( val startTime: Long, @@ -46,7 +47,7 @@ private[spark] class ApplicationInfo( init() - private def readObject(in: java.io.ObjectInputStream): Unit = { + private def readObject(in: java.io.ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() init() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala index 80b570a44af18..2ac21186881fa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/DriverInfo.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import java.util.Date import org.apache.spark.deploy.DriverDescription +import org.apache.spark.util.Utils private[spark] class DriverInfo( val startTime: Long, @@ -36,7 +37,7 @@ private[spark] class DriverInfo( init() - private def readObject(in: java.io.ObjectInputStream): Unit = { + private def readObject(in: java.io.ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() init() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index c5fa9cf7d7c2d..d221b0f6cc86b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -50,7 +50,7 @@ private[spark] class WorkerInfo( def coresFree: Int = cores - coresUsed def memoryFree: Int = memory - memoryUsed - private def readObject(in: java.io.ObjectInputStream) : Unit = { + private def readObject(in: java.io.ObjectInputStream): Unit = Utils.tryOrIOException { in.defaultReadObject() init() } diff --git a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala index 4908711d17db7..1cbd684224b7c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CartesianRDD.scala @@ -22,6 +22,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.reflect.ClassTag import org.apache.spark._ +import org.apache.spark.util.Utils private[spark] class CartesianPartition( @@ -36,7 +37,7 @@ class CartesianPartition( override val index: Int = idx @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization s1 = rdd1.partitions(s1Index) s2 = rdd2.partitions(s2Index) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index fabb882cdd4b3..ffc0a8a6d67eb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -27,6 +27,7 @@ import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap, CompactBuffer} +import org.apache.spark.util.Utils import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleHandle @@ -39,7 +40,7 @@ private[spark] case class NarrowCoGroupSplitDep( ) extends CoGroupSplitDep { @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization split = rdd.partitions(splitIndex) oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 11ebafbf6d457..9fab1d78abb04 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -25,6 +25,7 @@ import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ +import org.apache.spark.util.Utils /** * Class that captures a coalesced RDD by essentially keeping track of parent partitions @@ -42,7 +43,7 @@ private[spark] case class CoalescedRDDPartition( var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent partition at the time of task serialization parents = parentsIndices.map(rdd.partitions(_)) oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index 66c71bf7e8bb5..87b22de6ae697 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -48,7 +48,7 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( override def index: Int = slice @throws(classOf[IOException]) - private def writeObject(out: ObjectOutputStream): Unit = { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer @@ -67,7 +67,7 @@ private[spark] class ParallelCollectionPartition[T: ClassTag]( } @throws(classOf[IOException]) - private def readObject(in: ObjectInputStream): Unit = { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val sfactory = SparkEnv.get.serializer sfactory match { diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index 0c2cd7a24783b..92b0641d0fb6e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -22,6 +22,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.reflect.ClassTag import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} +import org.apache.spark.util.Utils /** * Class representing partitions of PartitionerAwareUnionRDD, which maintains the list of @@ -38,7 +39,7 @@ class PartitionerAwareUnionRDDPartition( override def hashCode(): Int = idx @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent partition at the time of task serialization parents = rdds.map(_.partitions(index)).toArray oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 0c97eb0aaa51f..aece683ff3199 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * Partition for UnionRDD. @@ -48,7 +49,7 @@ private[spark] class UnionPartition[T: ClassTag]( override val index: Int = idx @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization parentPartition = rdd.partitions(parentRddPartitionIndex) oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index f3d30f6c9b32f..996f2cd3f34a3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -22,6 +22,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.reflect.ClassTag import org.apache.spark.{OneToOneDependency, Partition, SparkContext, TaskContext} +import org.apache.spark.util.Utils private[spark] class ZippedPartitionsPartition( idx: Int, @@ -34,7 +35,7 @@ private[spark] class ZippedPartitionsPartition( def partitions = partitionValues @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { // Update the reference to parent split at the time of task serialization partitionValues = rdds.map(rdd => rdd.partitions(idx)) oos.defaultWriteObject() diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 2ab5d9637b593..01d5943d777f3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -22,6 +22,7 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import org.roaringbitmap.RoaringBitmap import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.Utils /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the @@ -105,13 +106,13 @@ private[spark] class CompressedMapStatus( MapStatus.decompressSize(compressedSizes(reduceId)) } - override def writeExternal(out: ObjectOutput): Unit = { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) out.writeInt(compressedSizes.length) out.write(compressedSizes) } - override def readExternal(in: ObjectInput): Unit = { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) val len = in.readInt() compressedSizes = new Array[Byte](len) @@ -152,13 +153,13 @@ private[spark] class HighlyCompressedMapStatus private ( } } - override def writeExternal(out: ObjectOutput): Unit = { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) emptyBlocks.writeExternal(out) out.writeLong(avgSize) } - override def readExternal(in: ObjectInput): Unit = { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index d49d8fb887007..11c19eeb6e42c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -42,7 +42,7 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long def this() = this(null.asInstanceOf[ByteBuffer], null, null) - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeInt(valueBytes.remaining); Utils.writeByteBuffer(valueBytes, out) @@ -55,7 +55,7 @@ class DirectTaskResult[T](var valueBytes: ByteBuffer, var accumUpdates: Map[Long out.writeObject(metrics) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { val blen = in.readInt() val byteVal = new Array[Byte](blen) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 554a33ce7f1a6..662a7b91248aa 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -117,11 +117,11 @@ class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { new JavaSerializerInstance(counterReset, classLoader) } - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeInt(counterReset) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { counterReset = in.readInt() } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 142285094342c..259f423c73e6b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -61,13 +61,13 @@ class BlockManagerId private ( def isDriver: Boolean = (executorId == "") - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeUTF(executorId_) out.writeUTF(host_) out.writeInt(port_) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { executorId_ = in.readUTF() host_ = in.readUTF() port_ = in.readInt() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 3db5dd9774ae8..291ddfcc113ac 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -21,6 +21,8 @@ import java.io.{Externalizable, ObjectInput, ObjectOutput} import akka.actor.ActorRef +import org.apache.spark.util.Utils + private[spark] object BlockManagerMessages { ////////////////////////////////////////////////////////////////////////////////// // Messages from the master to slaves. @@ -65,7 +67,7 @@ private[spark] object BlockManagerMessages { def this() = this(null, null, null, 0, 0, 0) // For deserialization only - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { blockManagerId.writeExternal(out) out.writeUTF(blockId.name) storageLevel.writeExternal(out) @@ -74,7 +76,7 @@ private[spark] object BlockManagerMessages { out.writeLong(tachyonSize) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { blockManagerId = BlockManagerId(in) blockId = BlockId(in.readUTF()) storageLevel = StorageLevel(in) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 1e35abaab5353..56edc4fe2e4ad 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -97,12 +98,12 @@ class StorageLevel private( ret } - override def writeExternal(out: ObjectOutput) { + override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeByte(toInt) out.writeByte(_replication) } - override def readExternal(in: ObjectInput) { + override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { val flags = in.readByte() _useDisk = (flags & 8) != 0 _useMemory = (flags & 4) != 0 diff --git a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala index 2b452ad33b021..770ff9d5ad6ae 100644 --- a/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/SerializableBuffer.scala @@ -29,7 +29,7 @@ private[spark] class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable { def value = buffer - private def readObject(in: ObjectInputStream) { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val length = in.readInt() buffer = ByteBuffer.allocate(length) var amountRead = 0 @@ -44,7 +44,7 @@ class SerializableBuffer(@transient var buffer: ByteBuffer) extends Serializable buffer.rewind() // Allow us to read it later } - private def writeObject(out: ObjectOutputStream) { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { out.writeInt(buffer.limit()) if (Channels.newChannel(out).write(buffer) != buffer.limit()) { throw new IOException("Could not fully write buffer to output stream") 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 65bdbaae65463..e1dc49238733c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -969,6 +969,21 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code that evaluates to Unit, re-throwing any non-fatal uncaught + * exceptions as IOException. This is used when implementing Externalizable and Serializable's + * read and write methods, since Java's serializer will not report non-IOExceptions properly; + * see SPARK-4080 for more context. + */ + def tryOrIOException(block: => Unit) { + try { + block + } catch { + case e: IOException => throw e + case NonFatal(t) => throw new IOException(t) + } + } + /** Default filtering function for finding call sites using `getCallSite`. */ private def coreExclusionFunction(className: String): Boolean = { // A regular expression to match classes of the "core" Spark API that we want to skip when diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 4b2ea45fb81d0..2de2a7926bfd1 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -66,7 +66,7 @@ class SparkFlumeEvent() extends Externalizable { var event : AvroFlumeEvent = new AvroFlumeEvent() /* De-serialize from bytes. */ - def readExternal(in: ObjectInput) { + def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { val bodyLength = in.readInt() val bodyBuff = new Array[Byte](bodyLength) in.readFully(bodyBuff) @@ -93,7 +93,7 @@ class SparkFlumeEvent() extends Externalizable { } /* Serialize to bytes. */ - def writeExternal(out: ObjectOutput) { + def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { val body = event.getBody.array() out.writeInt(body.length) out.write(body) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index b4adf0e9651a8..e59c24adb84af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -22,6 +22,7 @@ import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import org.apache.spark.Logging import org.apache.spark.streaming.scheduler.Job import org.apache.spark.streaming.dstream.{DStream, ReceiverInputDStream, InputDStream} +import org.apache.spark.util.Utils final private[streaming] class DStreamGraph extends Serializable with Logging { @@ -160,7 +161,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { logDebug("DStreamGraph.writeObject used") this.synchronized { checkpointInProgress = true @@ -172,7 +173,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug("DStreamGraph.readObject used") this.synchronized { checkpointInProgress = true diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala index 213dff6a76354..7053f47ec69a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/python/PythonDStream.scala @@ -33,6 +33,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Interval, Duration, Time} import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.api.java._ +import org.apache.spark.util.Utils /** @@ -73,13 +74,13 @@ private[python] class TransformFunction(@transient var pfunc: PythonTransformFun pfunc.call(time.milliseconds, rdds) } - private def writeObject(out: ObjectOutputStream): Unit = { + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { val bytes = PythonTransformFunctionSerializer.serialize(pfunc) out.writeInt(bytes.length) out.write(bytes) } - private def readObject(in: ObjectInputStream): Unit = { + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { val length = in.readInt() val bytes = new Array[Byte](length) in.readFully(bytes) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 65f7ccd318684..eabd61d713e0c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.util.{CallSite, MetadataCleaner} +import org.apache.spark.util.{CallSite, MetadataCleaner, Utils} /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -400,7 +400,7 @@ abstract class DStream[T: ClassTag] ( } @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".writeObject used") if (graph != null) { graph.synchronized { @@ -423,7 +423,7 @@ abstract class DStream[T: ClassTag] ( } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() generatedRDDs = new HashMap[Time, RDD[T]] () diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index f33c0ceafdf42..0dc72790fbdbd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.FileSystem import org.apache.spark.Logging import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils private[streaming] class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) @@ -119,7 +120,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } @throws(classOf[IOException]) - private def writeObject(oos: ObjectOutputStream) { + private def writeObject(oos: ObjectOutputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".writeObject used") if (dstream.context.graph != null) { dstream.context.graph.synchronized { @@ -142,7 +143,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() timeToOldestCheckpointFileTime = new HashMap[Time, Time] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 9eecbfaef363f..8152b7542ac57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark.rdd.RDD import org.apache.spark.rdd.UnionRDD import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.util.TimeStampedHashMap +import org.apache.spark.util.{TimeStampedHashMap, Utils} private[streaming] @@ -151,7 +151,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas } @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() generatedRDDs = new HashMap[Time, RDD[(K,V)]] () diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 9327ff4822699..2154c24abda3a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -73,7 +73,7 @@ class TestOutputStream[T: ClassTag](parent: DStream[T], // This is to clear the output buffer every it is read from a checkpoint @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { ois.defaultReadObject() output.clear() } @@ -95,7 +95,7 @@ class TestOutputStreamWithPartitions[T: ClassTag](parent: DStream[T], // This is to clear the output buffer every it is read from a checkpoint @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream) { + private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { ois.defaultReadObject() output.clear() } From 898b22ab1fe90e8a3935b19566465046f2256fa6 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Fri, 24 Oct 2014 17:21:08 -0700 Subject: [PATCH 1070/1492] [SPARK-4056] Upgrade snappy-java to 1.1.1.5 This upgrades snappy-java to 1.1.1.5, which improves error messages when attempting to deserialize empty inputs using SnappyInputStream (see https://github.com/xerial/snappy-java/issues/89). Author: Josh Rosen Author: Josh Rosen Closes #2911 from JoshRosen/upgrade-snappy-java and squashes the following commits: adec96c [Josh Rosen] Use snappy-java 1.1.1.5 cc953d6 [Josh Rosen] [SPARK-4056] Upgrade snappy-java to 1.1.1.4 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index a1195262614dd..030bea948b5ce 100644 --- a/pom.xml +++ b/pom.xml @@ -346,7 +346,7 @@ org.xerial.snappy snappy-java - 1.1.1.3 + 1.1.1.5 net.jpountz.lz4 From 3a845d3c048eebb0bddb3937128746fde3e8e4d8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 24 Oct 2014 18:36:35 -0700 Subject: [PATCH 1071/1492] [SQL] Update Hive test harness for Hive 12 and 13 As part of the upgrade I also copy the newest version of the query tests, and whitelist a bunch of new ones that are now passing. Author: Michael Armbrust Closes #2936 from marmbrus/fix13tests and squashes the following commits: d9cbdab [Michael Armbrust] Remove user specific tests 65801cd [Michael Armbrust] style and rat 8f6b09a [Michael Armbrust] Update test harness to work with both Hive 12 and 13. f044843 [Michael Armbrust] Update Hive query tests and golden files to 0.13 --- .rat-excludes | 1 + dev/run-tests | 2 +- project/SparkBuild.scala | 6 +- .../catalyst/analysis/HiveTypeCoercion.scala | 13 + .../catalyst/expressions/complexTypes.scala | 25 + .../org/apache/spark/sql/SQLContext.scala | 1 - .../execution/HiveCompatibilitySuite.scala | 104 +- .../apache/spark/sql/hive/HiveContext.scala | 3 +- .../spark/sql/hive/HiveInspectors.scala | 1 - .../spark/sql/hive/HiveMetastoreCatalog.scala | 1 - .../org/apache/spark/sql/hive/HiveQl.scala | 21 +- .../apache/spark/sql/hive/TableReader.scala | 1 - .../org/apache/spark/sql/hive/TestHive.scala | 7 +- .../sql/hive/api/java/JavaHiveContext.scala | 6 +- .../execution/DescribeHiveTableCommand.scala | 2 +- .../test/resources/data/conf/hive-site.xml | 80 +- .../test/resources/data/files/ProxyAuth.res | 15 + .../test/resources/data/files/alltypes.txt | 2 + .../test/resources/data/files/alltypes2.txt | 2 + .../src/test/resources/data/files/alltypesorc | Bin 0 -> 377237 bytes .../resources/data/files/char_varchar_udf.txt | 1 + .../test/resources/data/files/datatypes.txt | 6 +- .../src/test/resources/data/files/decimal.txt | 18 + .../src/test/resources/data/files/dept.txt | 4 + .../src/test/resources/data/files/emp.txt | 6 + .../data/files/exported_table/_metadata | 1 + .../data/files/exported_table/data/data | 2 + .../ext_test_space/folder+with space/data.txt | 3 + .../data/files/futurama_episodes.avro | Bin 0 -> 3044 bytes .../data/files/header_footer_table_1/0001.txt | 8 + .../data/files/header_footer_table_1/0002.txt | 8 + .../data/files/header_footer_table_1/0003.txt | 4 + .../header_footer_table_2/2012/01/01/0001.txt | 8 + .../header_footer_table_2/2012/01/02/0002.txt | 8 + .../header_footer_table_2/2012/01/03/0003.txt | 4 + .../files/header_footer_table_3/empty1.txt} | 0 .../files/header_footer_table_3/empty2.txt} | 0 .../src/test/resources/data/files/input.txt | 7 + .../test/resources/data/files/keystore.jks | Bin 0 -> 2248 bytes .../src/test/resources/data/files/kv9.txt | 27 + .../src/test/resources/data/files/loc.txt | 8 + .../resources/data/files/non_ascii_tbl.txt | 1 + .../data/files/orc_create_people.txt | 200 +- .../resources/data/files/orc_split_elim.orc | Bin 0 -> 246402 bytes .../resources/data/files/parquet_create.txt | 3 + .../data/files/parquet_partitioned.txt | 3 + .../resources/data/files/parquet_types.txt | 21 + .../test/resources/data/files/person age.txt | 10 +- .../test/resources/data/files/person+age.txt | 3 + .../resources/data/files/posexplode_data.txt | 4 + .../src/test/resources/data/files/sample.json | 1 + .../test/resources/data/files/symlink1.txt | 4 +- .../test/resources/data/files/symlink2.txt | 2 +- .../test/resources/data/files/truststore.jks | Bin 0 -> 958 bytes .../resources/data/scripts/input20_script.py | 30 + ...' + '1'-0-77504a9f3d712143beb52f3c25a904cb | 2 +- .../'1' + 1-0-130514c6116c311d808590a075b187b | 2 +- ...1' + 1.0-0-5db3b55120a19863d96460d399c2d0e | 2 +- ...1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 | 2 +- ...1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 | 2 +- ...1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a | 2 +- .../1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a | 2 +- .../1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 | 2 +- ...1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 | 2 +- .../1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e | 2 +- .../1 + 1S-0-2e99da48f67f588c9e632a57c713522e | 2 +- .../1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f | 2 +- ...0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 | 2 +- ...1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 | 2 +- ...0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d | 2 +- ....0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 | 2 +- ....0 + 1S-0-31fbe14d01fb532176c1689680398368 | 2 +- ....0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 | 2 +- ...L + '1'-0-6e39c7be301f3846efa9b4c939815b4a | 2 +- .../1L + 1-0-1864a260554255a09e4f28b8551eef9d | 2 +- ...L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 | 2 +- ...1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f | 2 +- ...1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 | 2 +- ...1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 | 2 +- ...S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 | 2 +- .../1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 | 2 +- ...S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e | 2 +- ...1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 | 2 +- ...1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e | 2 +- ...1S + 1Y-0-e59bc8279cd364224476ffc504c7685b | 2 +- ...Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb | 2 +- .../1Y + 1-0-a4541db51882b19503649138fbb295f | 2 +- ...Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d | 2 +- ...1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 | 2 +- ...1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 | 2 +- ...1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 | 2 +- ...M_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 | 309 -- ...M_COUNT-0-a393cfc24ad74f930f3284743254c10c | 309 -- ...M_COUNT-0-ae497f1556f548c1e2da9244397a985d | 309 -- ...quences-0-2f25c33d97c43f3276171624d988a286 | 2 +- .../LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 | 2 +- ...h group-0-f52ca483a3e5eadc1b20ba8320d029a7 | 2 +- ...age + 1-0-5e296b921c470f0f0b5d099f28bd5935 | 2 +- ...Average-0-c197ea78c4d8f85f1a317805b6da07e5 | 2 +- ..._exist-11-9c36cac1372650b703400c60dd29042c | 2 +- ..._exist-18-d824f22606f48dfca48ce241a7505f5b | 2 +- ..._exist-20-d824f22606f48dfca48ce241a7505f5b | 2 +- ..._exist-22-d824f22606f48dfca48ce241a7505f5b | 2 +- ..._exist-24-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-3-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-5-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-7-d824f22606f48dfca48ce241a7505f5b | 2 +- ...t_exist-9-d824f22606f48dfca48ce241a7505f5b | 2 +- ...ultiple-1-4d9d4efbabc9fffef8841cc049f479c1 | 29 - ...ultiple-4-7950c676506564b085b41426ed41747c | 2 +- ...itelist-0-3806584ff765bca682594008b90fc304 | 2 +- ...itelist-0-3c23ae800b2f6fb93620890da2dba196 | 2 +- ...as.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 | 2 +- .../alter2-1-aac9c2c7033fd7264c9a107a88ff591 | 10 +- .../alter2-11-aac9c2c7033fd7264c9a107a88ff591 | 10 +- .../alter2-14-aac9c2c7033fd7264c9a107a88ff591 | 10 +- ...alter2-15-75a213649242c2410ea6846f08c91d75 | 2 +- .../alter2-17-aac9c2c7033fd7264c9a107a88ff591 | 10 +- ...alter2-18-75a213649242c2410ea6846f08c91d75 | 2 +- ...alter2-20-9c36cac1372650b703400c60dd29042c | 2 +- ...alter2-25-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-28-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-29-63f13c364546ddce5d2176c6604a948f | 2 +- ...alter2-31-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-32-63f13c364546ddce5d2176c6604a948f | 2 +- ...alter2-35-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-38-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-39-63f13c364546ddce5d2176c6604a948f | 2 +- .../alter2-4-aac9c2c7033fd7264c9a107a88ff591 | 10 +- ...alter2-41-4ef75e12575453225738ea167c4617e5 | 10 +- ...alter2-42-63f13c364546ddce5d2176c6604a948f | 2 +- .../alter2-5-75a213649242c2410ea6846f08c91d75 | 2 +- .../alter2-7-aac9c2c7033fd7264c9a107a88ff591 | 10 +- .../alter2-8-75a213649242c2410ea6846f08c91d75 | 2 +- ...alter3-1-47f70392b97b94cdb425b25bde204f58} | 0 ...alter3-12-2fcb7fc251f682a584ad513fddfac506 | 12 +- ...alter3-16-9c36cac1372650b703400c60dd29042c | 2 +- ...lter3-21-91e32b3028ecc352dad8884829148311} | 0 ...alter3-25-568a59760e5d3241b63d65cce595face | 2 +- ...alter3-27-54ad133b447f67c6d1ed7d4c43803a87 | 12 +- ...alter3-28-5332228ea451105c897d0c8c3c8f2773 | 12 +- ...alter3-29-b8fba19b9009131caffbb5fe7468b67c | 2 +- ...alter3-32-327744965ee8ed630f56fa3e4a3c5c65 | 12 +- .../alter3-5-bf2a8fd1884bb584059c848332e30c97 | 2 +- .../alter3-7-30be5698ca15c1fd836686e7ad48ad8 | 12 +- .../alter3-8-8f0a466bd1d021e40690865b7ae52a43 | 12 +- .../alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 | 2 +- .../alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- ...alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- ...alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- ...alter4-16-549981e00a3d95f03dd5a9ef6044aa20 | 2 +- .../alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 | 6 +- .../alter4-5-9c36cac1372650b703400c60dd29042c | 2 +- ...alter5-1-cbad54fbd4a08fc8717708f93358ec3e} | 0 ...lter5-15-cbad54fbd4a08fc8717708f93358ec3e} | 0 ...alter5-18-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- ...alter5-21-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- .../alter5-4-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- .../alter5-7-2a9c8219c1468a1cf0534c665d1fcebf | 8 +- ...er_index-2-f36cb2eed39691ca949b25182e2dd31 | 4 +- ...er_index-4-f36cb2eed39691ca949b25182e2dd31 | 4 +- ...r_index-6-489b4ceb2f4301a7132628303f99240d | 2 +- ...erge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 | 2 +- ...merge_2-2-bf243aa10b608872b9e8286f89c5ff30 | 14 +- ...erge_2-3-bc9bb363f9a2026cfc70a31bb4551352} | 0 ...erge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224} | 0 ...erge_2-5-6319bf26f3739260b1a77e2ea89ef147} | 0 ...merge_2-6-f2eeb518a957cece4250cc7558839e02 | 2 +- ...merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 | 2 +- ...merge_2-9-f2eeb518a957cece4250cc7558839e02 | 2 +- ...oltype-12-84807e0be3e91250d4b8f2dc7938a256 | 1 - ...oltype-14-fdad866a990591083719bd45e4113f58 | 1 - ...oltype-16-b0534980e325d1fee63c2636ad3f8a4e | 75 - ...oltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c | 1 - ...oltype-19-63799ee74ccc42d9bb817e6d00a6fae3 | 1 - ..._coltype-2-17e04afbb81a724fe8c47c07b642f9a | 10 - ...coltype-21-17e04afbb81a724fe8c47c07b642f9a | 10 - ...oltype-22-639cdccdea5f4b1863f9b17b04183c93 | 10 - ...oltype-23-325be9a5d7c0277498a139c0a9fdb26a | 10 - ...oltype-24-71de9e476503e83f82baf1ab17da87f6 | 10 - ...coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 | 1 - ...coltype-6-db84ad9022cdc1828c24a0340321c8fd | 1 - ...coltype-8-42a70ae131fbb834c79074fdbd7beea0 | 1 - ...mat_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 | 6 +- ...at_loc-11-fe39b84ddc86b6bf042dc30c1b612321 | 10 +- ...at_loc-13-fe39b84ddc86b6bf042dc30c1b612321 | 10 +- ...at_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 | 10 +- ...at_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 | 10 +- ...at_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 | 10 +- ...mat_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 | 6 +- ...mat_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 | 6 +- ...mat_loc-9-fe39b84ddc86b6bf042dc30c1b612321 | 10 +- ...t_mode-1-e11f1476d2f30ab7a83f95013809f9e6} | 0 ...t_mode-2-29b4b67965f24c40bcc426043efb892d} | 0 ...t_mode-3-2100fc7853625d8c7dad5c0ee70d4690} | 0 ...t_mode-4-c3fa911536c75e87febf2874e7889879} | 0 ...ct_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 | 2 +- ...ct_mode-9-d1b12be1d01eabaf244f41e74d902d9d | 2 +- ...itelist-0-3c23ae800b2f6fb93620890da2dba196 | 2 +- ...tition-11-bc84e38ccad173f5b47474d91db244d7 | 2 +- ...tition-14-9c36cac1372650b703400c60dd29042c | 2 +- ...ition-19-d862c00e5a37ae841a6c8ec6c5d68e8c} | 0 ...rtition-2-9c36cac1372650b703400c60dd29042c | 2 +- ...tition-22-d50111b57d14f1ded1c47c773b0e0ac2 | 2 +- ...tition-24-21dd05d56ebba285a8eb5bde5904d6a3 | 2 +- ...tition-26-270655c514bb6f04acd4459df52dd77b | 2 +- ...tition-4-833254c8f7c5b1248e1a249560652627} | 0 ...rtition-7-e3d9a36d53d30de215b855095c58d0d7 | 2 +- ...rtition-9-21dd05d56ebba285a8eb5bde5904d6a3 | 2 +- ...e_serde-1-5bc931a540f0fec54e852ff10f52f879 | 8 +- ..._serde-11-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ..._serde-13-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ..._serde-15-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ..._serde-17-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ...e_serde-3-5bc931a540f0fec54e852ff10f52f879 | 8 +- ...e_serde-5-5bc931a540f0fec54e852ff10f52f879 | 8 +- ...e_serde-9-6ee4b3a60659ec5496f06347eda232a8 | 12 +- ...rchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb} | 0 ...archar2-3-fb3191f771e2396d5fc80659a8c68797 | 2 +- ...archar2-5-84e700f9dc6033c1f237fcdb95e31a0c | 2 +- ...rchar2-6-3250407f20f3766c18f44b8bfae1829d} | 0 ...rchar2-7-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...archar2-7-84e700f9dc6033c1f237fcdb95e31a0c | 1 - ...archar2-8-4c12c4c53d99338796be34e603dc612c | 1 - ...archar2-8-84e700f9dc6033c1f237fcdb95e31a0c | 1 + ...archar2-9-4c12c4c53d99338796be34e603dc612c | 1 + ..._select-2-1ac845048a8c714a36a719ea8e4f570b | 12 +- ..._select-4-1ac845048a8c714a36a719ea8e4f570b | 6 +- ..._select-6-1ac845048a8c714a36a719ea8e4f570b | 8 +- ...ew_rename-2-67e47ee2746463594d5c48b10ba1bb | 5 - ..._rename-4-19c1c00f0aa99d81b7466958c15d88e3 | 5 - ...us_col-0-e91e3e5a22029b9b979ccbbef97add66} | 0 ...us_col-1-b4fe82679efdf6a15e9ecff53baf8d8d} | 0 ...us_col-2-dadfa3854356dead14b93c5a71a5d8ab} | 0 ...us_col-3-70509ccd2765d90b98666b6dff8afe1b} | 0 ...s_join-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ...s_join-1-84e7846d50fc15e836c83911ce039871} | 0 ..._join-10-a59dc1c01d48c82d46a5beab759f618d} | 0 ..._join-11-3e0ade2476221f6396381f55a3b82e60} | 0 ..._join-12-cef44682d339a67ba765f854da21f976} | 0 ..._join-13-6292c27f0c4eaeab2a5e6c317e3afa2e} | 0 ..._join-14-4f3042b9feebd00c540ddac40e7254d1} | 0 ..._join-15-a2f3b8a636e46e5df514df46c452855f} | 0 ..._join-16-a75699a21ea36d962a1ba45bd5a12f26} | 0 ..._join-17-64380f94a431e3a1698aa2edd3d0a6b2} | 0 ..._join-18-222d1fcce7420950436e00d3a1bba957} | 0 ..._join-19-dea5f8f5c703583c7a3bdcb62cd3d589} | 0 ...s_join-2-40548ec2313af8dbdcbb8ad0477d8600} | 0 ..._join-20-2d5e186b97166e3a625a169d0d73f9c8} | 0 ..._join-21-ed73d4b01424287148347ccf721b37e2} | 0 ..._join-22-2cf93da6bb0efdafeaa989c9df236701} | 0 ..._join-23-fa90806f6137300311df032e28df3d4c} | 0 ..._join-24-4a6976344eeae35e059285ed78f9feb3} | 0 ...s_join-3-26f82fb6734d5bc6f7159b06c0949178} | 0 ...s_join-4-a598c93d86a646cfa6ea86da54bce9b8} | 0 ...s_join-5-d12ba848d20d1def95379c490068f050} | 0 ...s_join-6-96a4806e61c5d882affac57e115f285f} | 0 ...s_join-7-38be41d6328f57350a78fb2097d1ebd2} | 0 ...s_join-8-eb11e867a36e2114ef0f0f001e01984c} | 0 ...s_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ..._limit-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ..._limit-1-26f82fb6734d5bc6f7159b06c0949178} | 0 ...limit-10-a89c94fd0de0cfc96725fea890987cc0} | 0 ..._limit-2-eb11e867a36e2114ef0f0f001e01984c} | 0 ..._limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ..._limit-4-4252006172a476fbc591eebee49bffa3} | 0 ..._limit-5-a2f3b8a636e46e5df514df46c452855f} | 0 ..._limit-6-f1fd6c403a9787947877f48c772afe96} | 0 ..._limit-7-69b422026fce4e2d2cde9a478d6aaa40} | 0 ..._limit-8-72f5e55c1e244084eea255c32a6f58eb} | 0 ..._limit-9-9da67c62d5e3145d450ad7953b33361f} | 0 ...s_part-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ...ts_part-1-3c29684bfd2df7439ee0551eb42cfa0} | 0 ..._part-10-5ba0295bfe42deb678e59b3a330b14ff} | 0 ...ts_part-11-dbdbe2e04c5dad6c8af78b6386b329} | 0 ..._part-12-2856fb4f4af8932804cb238429d9de6f} | 0 ..._part-13-4fa8b0f9fb0202ac0e82fb87538d6445} | 0 ..._part-14-62c557bfb7bf70694a32ebceea329ebd} | 0 ..._part-15-f796cd035603726a5c4ce3e71194822b} | 0 ..._part-16-45eb5544124364714549f199f9c2b2ac} | 0 ..._part-17-ad61ebd912b6bef3778c4ff38c0be5ab} | 0 ..._part-18-b9f2dff7c7b57412cea44433ea810fa7} | 0 ..._part-19-10cab43c2966718bb39e2f22365cd6c1} | 0 ...s_part-2-a4fb8359a2179ec70777aad6366071b7} | 0 ..._part-20-51ec5046b50543899ed54c9fc2b885af} | 0 ..._part-21-d69cb7b7770b51bc4b99d5d0f74d4662} | 0 ..._part-22-22f430881fbfe92736988c2a4184fe0c} | 0 ..._part-23-2719ee3f0b8b07f1e5c80f8329b9f87f} | 0 ..._part-24-1f7bdd816b83f45a6d53d08b6866109f} | 0 ..._part-25-d1599e385e8bad6cd62d564a6fd0536f} | 0 ..._part-26-ec26bcb2587f4638b364ba494ac373e0} | 0 ..._part-27-7804e2e230a42a46d72546fa2381b423} | 0 ..._part-28-40f2a1f06d181ef93edf42a591cbf15e} | 0 ...s_part-3-16367c381d4b189b3640c92511244bfe} | 0 ...s_part-4-397e834a25a20b68aed5e87d55ea6174} | 0 ...s_part-5-cef44682d339a67ba765f854da21f976} | 0 ...s_part-6-f4263aca1612c05cb08242448191ad05} | 0 ...s_part-7-b222533ddda09918c4f5acc408bf1a02} | 0 ...s_part-8-84e3979946de67779a9704a3adc2184f} | 0 ...ts_part-9-c6c38e93babafe56cd4f177a17d37b8} | 0 ..._table-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ...s_table-1-3c29684bfd2df7439ee0551eb42cfa0} | 0 ...table-10-a7419af512a6c0b60ec51e556608cdad} | 0 ...table-11-4be11140c2e21b3a172224d103844785} | 0 ...table-12-84bc7cf67544461e9102820ed410dfb6} | 0 ...s_table-13-d2acfdf279aad035b31ed61e87bff6} | 0 ...table-14-60d1c44b8412c4983265e320ecfb25f0} | 0 ...table-15-7b185296b6a0c6ceff2df1f89fecc6b4} | 0 ...table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d} | 0 ...table-17-cd84d629d569a09d139f38dae01e677c} | 0 ...table-18-7012af4092c18075f4f3a00b80f491ee} | 0 ..._table-2-84e7846d50fc15e836c83911ce039871} | 0 ..._table-3-a598c93d86a646cfa6ea86da54bce9b8} | 0 ..._table-4-d12ba848d20d1def95379c490068f050} | 0 ..._table-5-4a8ad142e57c5dce2623e92f5e5ad15a} | 0 ..._table-6-a59dc1c01d48c82d46a5beab759f618d} | 0 ..._table-7-6292c27f0c4eaeab2a5e6c317e3afa2e} | 0 ..._table-8-43cb040e2fe01904bc52e198fcae9b3d} | 0 ..._table-9-eadfdc61d22bd22bbf5a69370908a82e} | 0 ..._union-0-2a0c41508e1e70eaedf9de99751c8fa9} | 0 ..._union-1-26f82fb6734d5bc6f7159b06c0949178} | 0 ...union-10-3aa3eb68d092ea0157c5426a5f2ca3f9} | 0 ...union-11-c1302094d7b4456550826535b529468b} | 0 ...union-12-3af760f960a2e0995d1bc4afef0c5aa0} | 0 ...union-13-26f82fb6734d5bc6f7159b06c0949178} | 0 ...union-14-eb11e867a36e2114ef0f0f001e01984c} | 0 ...union-15-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ...union-16-4252006172a476fbc591eebee49bffa3} | 0 ...union-17-a2f3b8a636e46e5df514df46c452855f} | 0 ...union-18-58d55f8c8b2489726232a00254499144} | 0 ...union-19-70c851c23a7ac89f0366b9c26085cc3e} | 0 ..._union-2-eb11e867a36e2114ef0f0f001e01984c} | 0 ...union-20-222d1fcce7420950436e00d3a1bba957} | 0 ...union-21-a5503df8f50e057415719bcfe4c69f13} | 0 ...union-22-a23089e5a76dc35df04d9936d60c4565} | 0 ..._union-3-e0b7cb2226234fbea8ae11ea7d9d84bd} | 0 ..._union-4-4252006172a476fbc591eebee49bffa3} | 0 ..._union-5-a2f3b8a636e46e5df514df46c452855f} | 0 ..._union-6-222d1fcce7420950436e00d3a1bba957} | 0 ..._union-7-aab1a7b7e1fe4d061a580126d67dfd0a} | 0 ..._union-8-172e0d089bd5bcbaf54775a618d826bb} | 0 ..._union-9-69b422026fce4e2d2cde9a478d6aaa40} | 0 ...archive-0-89cd75b0565e8d96910d5528db9984e7 | 1 - ...archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ...rchive-11-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...rchive-13-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...archive-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 - ...rchive-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 - ...rchive-17-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...rchive-20-530277b0fee8b05c37b26846bceef827 | 6 - ...rchive-22-530277b0fee8b05c37b26846bceef827 | 6 - ...rchive-24-530277b0fee8b05c37b26846bceef827 | 6 - ...rchive-28-188eb7912265ed8dffa5200517bbe526 | 1 - ...rchive-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 1 - ...adoop20-0-89cd75b0565e8d96910d5528db9984e7 | 1 - ...adoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ...doop20-11-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...doop20-13-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...adoop20-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 - ...doop20-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 - ...doop20-17-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...doop20-20-530277b0fee8b05c37b26846bceef827 | 6 - ...doop20-22-530277b0fee8b05c37b26846bceef827 | 6 - ...doop20-24-530277b0fee8b05c37b26846bceef827 | 6 - ...doop20-28-188eb7912265ed8dffa5200517bbe526 | 1 - ...doop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a | 1 - ...e_multi-0-89cd75b0565e8d96910d5528db9984e7 | 1 - ...e_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ..._multi-11-cf5431cd843666b95ad2a82b334ac01e | 1 - ..._multi-13-27895cbe0ee6d24d7fc866314491e1bb | 1 - ...e_multi-14-2cde1a2d3bfcec814985f498eba0fb8 | 1 - ..._multi-15-c9f39b388ae698e385e092d0ffeb3c73 | 9 - ..._multi-17-27895cbe0ee6d24d7fc866314491e1bb | 1 - .../attr-0-24e06ffd262f2a5a6eec3314445d83ba | 2 +- ...o_join0-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join0-2-7bd04899197b027d81c24e45a99ad15c | 2 +- ...o_join1-0-443afb71720bad780b5dbfb6dbf4b51a | 2 +- ...to_join1-4-ae1247a065c41ce0329ca6078ab586e | 2 +- ..._join10-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join10-2-eef4ee52e0783b15fb5fe17378806b13 | 2 +- ..._join11-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join11-2-5496e81f60ba1d8a95d8375589c71e05 | 2 +- ..._join12-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join12-2-4df549c5f0b6bff0c843008fa35b1320 | 2 +- ..._join13-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join13-2-5ff417533a1243cd6fc556960fa170c9 | 2 +- ..._join14-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join14-2-9b141c1e5917ca82c6bc36a9a2950a1e | 2 +- ...o_join14-3-2b9ccaa793eae0e73bf76335d3d6880 | 2 +- ..._join14-4-bab89dfffa77258e34a595e0e79986e3 | 2 +- ...o_join14-7-5b5ded1412301eae5f8f705a39e6832 | 2 +- ..._join15-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 | 2 +- ..._join16-0-ce1ef910fff98f174931cc641f7cef3a | 1 - ..._join16-2-66e56dcda38eb09819ac49e47e40d125 | 1 - ..._join17-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join17-4-11d706a64d44a8b0d41b290c4671c29c | 2 +- ..._join18-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join18-2-f633ade9577c8b0e89d89124194c8d0f | 2 +- ..._join19-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join19-4-eaa70da463b92e85e1796277f016c18f | 2 +- ...o_join2-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join2-4-9d8144612cb3132ad9f7c8fa93586185 | 2 +- ..._join20-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join20-2-903ee25e327188edaaf2040fec5a8e52 | 2 +- ..._join20-4-e48e08d5b94719d56a58284eaef757f2 | 2 +- ..._join21-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join21-2-3536b7d78713e86ee67f5f6c9b88958f | 2 +- ..._join22-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join22-2-a4abc288c20edee53ede45d248cf3abb | 2 +- ..._join23-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join23-2-6d2c5b58222f31658a0cf957e093a150 | 2 +- ..._join24-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join24-4-d79325ef6494aa87843fdfd78de7c812 | 2 +- ..._join26-1-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join26-4-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ..._join27-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._join27-2-fceaa1ebd63334061d2d8daf961e935e | 2 +- ..._join28-0-10a2c01dccc8980fe6aff1f9dd65042c | 2 +- ..._join28-1-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join3-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join3-4-eaa70da463b92e85e1796277f016c18f | 2 +- ..._join30-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...join30-10-820f36ed1bdf14c1deb383f508a5ed7a | 2 +- ..._join30-12-de6e26f52f6f9ea5ef9a4868e57d36d | 2 +- ...join30-14-7a248488c218919ab50e072fdbdecb73 | 2 +- ...join30-16-f4f5bc179d84baf57e14cd2f8bd39436 | 2 +- ..._join30-2-bc472f95600f47d5ea60fdeddc59dbc7 | 2 +- ...to_join30-4-f5083eca9c3df277988d8b345b8d43 | 2 +- ..._join30-6-4a9144326fc7d066c9aadb13d1b95031 | 2 +- ..._join30-8-8a27209399df7f9c4d15988b11753a61 | 2 +- ..._join31-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 | 2 +- ..._join32-0-24ca942f094b14b92086305cc125e833 | 2 +- ...oin32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c} | 0 ...oin32-13-ee2dcaae78ae900ffce8d19fbadc3735} | 0 ...oin32-14-7927c2ce644d1ce1de251405c8563e99} | 0 ...join32-15-4cd3b51861720ac06c6deb818c83670} | 0 ...oin32-20-693874ea8e06a8b155394ac27af2b1a7} | 0 ...oin32-21-bf8a1bb0baaae9fbf1c3aa656f991f42} | 0 ...oin32-22-3d14d63e996851f51a98f987995d8da6} | 0 ...oin32-23-8b183ec2c164b3b530e802ffc880a5fa} | 0 ..._join32-5-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ...o_join32-6-442e69416faaea9309bb8c2a3eb73ef | 2 +- ..._join32-7-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...o_join4-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join4-4-998c3a307b074a6505bb7fcef276be04 | 2 +- ...o_join5-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join5-4-998c3a307b074a6505bb7fcef276be04 | 2 +- ...o_join6-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join6-4-998c3a307b074a6505bb7fcef276be04 | 2 +- ...o_join7-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join7-4-30d0c1a49784347fedbac21a69c3a899 | 2 +- ...o_join9-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...o_join9-4-eaa70da463b92e85e1796277f016c18f | 2 +- ...ilters-2-bee6095f42de6a16708c2f9addc1b9bd} | 0 ...lters-31-268d8fb3cb9b04eb269fe7ec40a24dfe} | 0 ...lters-32-6dc6866a65c74d69538b776b41b06c16} | 0 ...lters-33-e884480a0f7273d3e2f2de2ba46b855c} | 0 ...lters-34-98fd86aea9cacaa82d43c7468109dd33} | 0 ...n_nulls-0-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 | 2 +- ..._nulls-11-141c550a314d15c4e200e5baeb246de2 | 2 +- ..._nulls-12-8a65225a88da0169af26848c06cb981c | 2 +- ..._nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 | 2 +- ..._nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 | 2 +- ..._nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 | 2 +- ..._nulls-16-dbe244d2c21e477c3703c4ce1903e8af | 2 +- ..._nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 | 2 +- ..._nulls-18-439a409bc50dfd86dee78c151c3de5eb | 2 +- ..._nulls-19-92641e46934ebbf3d44e6e60de1882f4 | 2 +- ..._nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f} | 0 ..._nulls-20-e34b2b210059a5f93c0a873d91859b5d | 2 +- ..._nulls-21-c7aaa831acbd959c6d1793056e3c288a | 2 +- ..._nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 | 2 +- ..._nulls-23-5805a41289a26139c06604a40bf5a6fa | 2 +- ..._nulls-24-80991af26d5d37e0864ecc2c8ab0b984 | 2 +- ...n_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 | 2 +- ...in_nulls-4-fc1128c86cd430db8cd4ff834be4562 | 2 +- ...n_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc | 2 +- ...n_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 | 2 +- ...n_nulls-7-8395fa78507105c2a018e88f717b95e2 | 2 +- ...n_nulls-8-fd992f2127a139aeb554d797e748ed54 | 2 +- ...n_nulls-9-e3a86622a437e910b7225d1e6108da9e | 2 +- ...values-1-2bfb628930d072124636d21d82e3b462} | 0 ...values-4-11af6838bb9e04152c2f9a7e2044abe0} | 0 ...values-7-99fcaa5203ed3debb52c9086028dc8c2} | 0 ..._values-8-950af86c321a67ab3ed0fa5b63ea6aed | 2 +- ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_14-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_14-12-94538bc7322522a5534cafc0551d2189 | 2 +- ...oin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 | 2 +- ...oin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 | 2 +- ...oin_14-18-21269869cd3aaf4ade2170d9017de018 | 2 +- ...pjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...oin_14-20-4e0e8cd0626a84b21ca7d2f633623578 | 2 +- ...oin_14-22-2fe7b834b341bf18e36cd79dd00ec16a | 2 +- ...oin_14-24-43ba2c72db9db1ec18d835ec978f8da1 | 2 +- ...oin_14-26-b66c416fdd98d76981f19e9c14b6a562 | 2 +- ...oin_14-28-b889b147255231f7fe44bd57e1f8ba66 | 2 +- ...oin_14-30-b9d66e78b8898a97a42d1118300fa0ce | 2 +- ...oin_14-32-b0ca9e20cd48457e6cf1c313d5505213 | 2 +- ...oin_14-37-a45927057c01fd54818b5dd50e77f60e | 2 +- ...oin_14-38-a988727daa49fb5e190f81c027bb7005 | 2 +- ...oin_14-43-a45927057c01fd54818b5dd50e77f60e | 2 +- ...oin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 | 2 +- ...join_14-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_14-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_14-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_1-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...oin_1-10-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...oin_1-11-337e909605c780d00ad8895686defa06} | 0 ...join_1-12-24ca942f094b14b92086305cc125e833 | 2 +- ...join_1-13-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_1-14-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_1-15-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_1-16-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc | 2 +- ...join_1-19-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_1-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_1-21-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_1-22-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_1-24-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_1-4-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_1-5-d964114ed76536c8e3cacd231340851c} | 0 ...join_1-6-e44aff8a860cf3965752d3e1ce725cde} | 0 ...join_1-7-ae582a255a02d91674aab500aee79e20} | 0 ..._join_1-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...join_1-9-10b03ce2526bf180faaec9310cfab290} | 0 ...join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_10-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_10-10-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_10-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_10-12-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...oin_10-14-95e18bd00f2de246efca1756681c1e87 | 2 +- ...join_10-16-caa641c820fcc5f601758c5f0385b4e | 2 +- ..._join_10-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_10-7-24ca942f094b14b92086305cc125e833 | 2 +- ...join_10-8-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_10-9-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_11-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...in_11-10-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...in_11-11-337e909605c780d00ad8895686defa06} | 0 ...oin_11-12-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_11-14-325432a220aa3ebe8b816069916924d8 | 2 +- ...oin_11-15-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_11-16-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ...join_11-17-442e69416faaea9309bb8c2a3eb73ef | 2 +- ...oin_11-19-325432a220aa3ebe8b816069916924d8 | 2 +- ...oin_11-2-f42438f3f5c266b997686ba846420ebe} | 0 ...oin_11-21-c4d55c247c9326f474d89b29b81d60aa | 2 +- ...oin_11-4-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_11-5-d964114ed76536c8e3cacd231340851c} | 0 ...oin_11-6-e44aff8a860cf3965752d3e1ce725cde} | 0 ...oin_11-7-ae582a255a02d91674aab500aee79e20} | 0 ...join_11-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_11-9-10b03ce2526bf180faaec9310cfab290} | 0 ...oin_12-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...in_12-10-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...in_12-11-337e909605c780d00ad8895686defa06} | 0 ...oin_12-12-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_12-13-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_12-14-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_12-15-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...in_12-17-97ba394ab6aad2547f399ebbf757a4b6} | 0 ...in_12-18-73ee2d7b62e2aede20ca5de577cd7b7f} | 0 ...oin_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f} | 0 ...oin_12-2-f42438f3f5c266b997686ba846420ebe} | 0 ...oin_12-21-4ecd65f0e26e981b66770b3e91e128fc | 2 +- ...oin_12-4-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_12-5-d964114ed76536c8e3cacd231340851c} | 0 ...oin_12-6-e44aff8a860cf3965752d3e1ce725cde} | 0 ...oin_12-7-ae582a255a02d91674aab500aee79e20} | 0 ...join_12-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_12-9-10b03ce2526bf180faaec9310cfab290} | 0 ...join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_13-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_13-10-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...oin_13-11-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...oin_13-12-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_13-13-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_13-14-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_13-15-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 | 2 +- ...oin_13-19-4b2ac2865384fbca7f374191d8021d51 | 2 +- ..._join_13-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...oin_13-20-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...oin_13-21-ea23403b9eb55e8b06d1c198e439569f | 2 +- ...oin_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 | 2 +- ...oin_13-25-4b2ac2865384fbca7f374191d8021d51 | 2 +- ...oin_13-26-f135547e33c01d1f543c8b1349d60348 | 2 +- ...oin_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 | 2 +- ...oin_13-30-4b2ac2865384fbca7f374191d8021d51 | 2 +- ...join_13-9-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_14-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_14-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_14-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_14-13-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_14-15-43ad2152b18d711adbdd1aeb940b662a | 2 +- ...oin_14-18-a16ff76d72ad428fb9d43ab910f259fd | 2 +- ..._join_14-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_14-7-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_14-8-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_14-9-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_15-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_15-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_15-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...oin_15-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_15-13-24ca942f094b14b92086305cc125e833 | 2 +- ..._join_15-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_15-7-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_15-8-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_15-9-f135547e33c01d1f543c8b1349d60348 | 2 +- ...oin_16-0-24ca942f094b14b92086305cc125e833} | 0 ...oin_16-1-16367c381d4b189b3640c92511244bfe} | 0 ...in_16-10-4706d21b17f993cc1cc94be6b7e04c28} | 0 ...in_16-11-5e81d0b41cc58d8cc151046c7a111411} | 0 ...in_16-12-e8a77ff790bfd6740489fc4374ec6c3d} | 0 ...in_16-13-920818d557d2525dabb5c539a80a1bbb} | 0 ...in_16-14-455dfeeba27ecf53923db0cbf0aab908} | 0 ...in_16-15-11f98f575685beedc14a88fc47a61041} | 0 ...oin_16-16-dff02d7b4c5242434d5e7449bdb67f8b | 24 + ...oin_16-2-7b4ad215fc2e75c71c6614a2b6322e8e} | 0 ...oin_16-3-365488a703b0640acda73a7d7e6efa06} | 0 ...oin_16-4-d0ec6d66ff349db09fd455eec149efdb} | 0 ...oin_16-5-3b0f76816be2c1b18a2058027a19bc9f} | 0 ...oin_16-6-86473a0498e4361e4db0b4a22f2e8571} | 0 ...oin_16-7-7e87a31677022b6a1c360922ef74754e} | 0 ...join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c} | 0 ...join_16-9-ae5880516ea2f924cfbaeb919adc86e} | 0 ...join_2-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...join_2-10-24ca942f094b14b92086305cc125e833 | 2 +- ...join_2-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_2-12-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_2-13-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_2-14-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 | 2 +- ...join_2-17-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_2-18-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 2 +- ...join_2-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_2-20-8180638a57b64557e02815c863031755 | 2 +- ...join_2-22-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_2-3-aa6ec7f17b48cf49f02ab7367453ab39} | 0 ...join_2-4-66b07c93d79ed9958b8427dad16c3ef3} | 0 ...join_2-6-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_2-7-d964114ed76536c8e3cacd231340851c} | 0 ..._join_2-8-962264967269db1d5f28a9a6c60dbf1} | 0 ...join_2-9-10b03ce2526bf180faaec9310cfab290} | 0 ...join_3-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...join_3-10-24ca942f094b14b92086305cc125e833 | 2 +- ...join_3-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_3-12-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_3-13-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_3-14-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc | 2 +- ...join_3-17-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_3-19-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_3-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_3-20-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_3-22-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_3-4-45b63361c1e5178b69a1531b238c8460} | 0 ...join_3-6-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_3-7-d964114ed76536c8e3cacd231340851c} | 0 ...join_3-8-e44aff8a860cf3965752d3e1ce725cde} | 0 ...join_3-9-ae582a255a02d91674aab500aee79e20} | 0 ...join_4-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...oin_4-10-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_4-11-d964114ed76536c8e3cacd231340851c} | 0 ...join_4-12-24ca942f094b14b92086305cc125e833 | 2 +- ...join_4-13-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_4-14-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_4-15-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_4-16-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 | 2 +- ...join_4-19-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_4-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_4-21-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_4-22-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_4-24-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_4-3-aa6ec7f17b48cf49f02ab7367453ab39} | 0 ...join_4-4-66b07c93d79ed9958b8427dad16c3ef3} | 0 ...join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_4-6-45b63361c1e5178b69a1531b238c8460} | 0 ...join_4-7-ecca12a2f377c18c53563a534e9dd5a5} | 0 ...join_4-8-65930e1b01da720cf296ca3df668b58d} | 0 ...join_5-1-fdbb11a2de2777dfdd916b59764d5c8e} | 0 ...join_5-10-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_5-11-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 | 2 +- ...join_5-14-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_5-16-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_5-17-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_5-18-24ca942f094b14b92086305cc125e833 | 2 +- ...join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed} | 0 ...join_5-20-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_5-3-9878d6ab0fe143611c88ffc0602197e3} | 0 ...join_5-4-a576657b9fd1c23aed85a7409887b2fb} | 0 ...join_5-6-350b202868590b5edaed18007fd5cbbe} | 0 ...join_5-7-1155788c7c133a73c3609e8052accfa5} | 0 ..._join_5-8-d0ec6d66ff349db09fd455eec149efdb | 2 +- ..._join_5-9-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._join_6-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_6-11-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_6-12-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_6-13-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_6-14-24ca942f094b14b92086305cc125e833 | 2 +- ...join_6-15-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...join_6-16-ea23403b9eb55e8b06d1c198e439569f | 2 +- ...join_6-17-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_6-19-cf41f7ce9478536e823107d1810ff1d7 | 2 +- ...e_join_6-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_6-21-b55506a213ec710004e6d7f3462834d0 | 2 +- ...join_6-23-4281442c87dcf6007f8bd42504eba186 | 2 +- ...join_6-25-52f0e65724d29e2b4054b59a50d2837b | 2 +- ...join_6-27-961f141836f2cc9521f681cadbc3d140 | 2 +- ...join_6-29-fd0cc412e0987569a4ed879454b53fb0 | 2 +- ...join_6-30-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_6-32-cf41f7ce9478536e823107d1810ff1d7 | 2 +- ...join_6-34-52f0e65724d29e2b4054b59a50d2837b | 2 +- ...join_6-36-961f141836f2cc9521f681cadbc3d140 | 2 +- ...join_6-38-fd0cc412e0987569a4ed879454b53fb0 | 2 +- ...join_7-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...oin_7-10-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...oin_7-11-d964114ed76536c8e3cacd231340851c} | 0 ...join_7-12-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_7-13-10b03ce2526bf180faaec9310cfab290} | 0 ...join_7-14-24ca942f094b14b92086305cc125e833 | 2 +- ...join_7-15-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_7-16-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_7-17-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_7-18-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 | 2 +- ...join_7-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_7-21-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_7-23-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_7-24-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_7-26-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_7-3-aa6ec7f17b48cf49f02ab7367453ab39} | 0 ...join_7-4-66b07c93d79ed9958b8427dad16c3ef3} | 0 ...join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_7-6-45b63361c1e5178b69a1531b238c8460} | 0 ...join_7-7-ecca12a2f377c18c53563a534e9dd5a5} | 0 ...join_7-8-65930e1b01da720cf296ca3df668b58d} | 0 ...join_8-1-4e24a5c0c67a137478e4a8be2a081872} | 0 ...join_8-10-962264967269db1d5f28a9a6c60dbf1} | 0 ...oin_8-11-10b03ce2526bf180faaec9310cfab290} | 0 ...oin_8-12-5cba470fbd02e730781a3b63fd9aa3e2} | 0 ...oin_8-13-337e909605c780d00ad8895686defa06} | 0 ...join_8-14-24ca942f094b14b92086305cc125e833 | 2 +- ...join_8-15-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_8-16-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_8-17-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_8-18-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc | 2 +- ...join_8-2-f42438f3f5c266b997686ba846420ebe} | 0 ...join_8-21-325432a220aa3ebe8b816069916924d8 | 2 +- ...join_8-23-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_8-24-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 | 2 +- ...join_8-26-8180638a57b64557e02815c863031755 | 2 +- ...join_8-28-7d0c37fc09323ce11aae0b58dc687660 | 2 +- ...join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1} | 0 ...join_8-4-45b63361c1e5178b69a1531b238c8460} | 0 ...join_8-6-9e58f8a961723c40a5d1f742251a8fa5} | 0 ...join_8-7-d964114ed76536c8e3cacd231340851c} | 0 ...join_8-8-e44aff8a860cf3965752d3e1ce725cde} | 0 ...join_8-9-ae582a255a02d91674aab500aee79e20} | 0 ..._join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._join_9-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_9-10-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...join_9-11-cda81d86d127fca0e2fbc2161e91400d | 2 +- ...join_9-13-94538bc7322522a5534cafc0551d2189 | 2 +- ...join_9-15-63261d35ddda973eeeb97b994ab7a476 | 2 +- ...join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 | 2 +- ...join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 | 2 +- ...e_join_9-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_9-21-21269869cd3aaf4ade2170d9017de018 | 2 +- ...join_9-23-4e0e8cd0626a84b21ca7d2f633623578 | 2 +- ...join_9-25-2fe7b834b341bf18e36cd79dd00ec16a | 2 +- ...join_9-27-43ba2c72db9db1ec18d835ec978f8da1 | 2 +- ...join_9-29-b66c416fdd98d76981f19e9c14b6a562 | 2 +- ...join_9-31-b889b147255231f7fe44bd57e1f8ba66 | 2 +- ...join_9-33-aa8d713ad4e19b72b5bd7628d60c295e | 2 +- ...join_9-35-b9d66e78b8898a97a42d1118300fa0ce | 2 +- ...join_9-37-b0ca9e20cd48457e6cf1c313d5505213 | 2 +- ...join_9-38-f135547e33c01d1f543c8b1349d60348 | 2 +- ...join_9-40-94538bc7322522a5534cafc0551d2189 | 2 +- ...join_9-42-63261d35ddda973eeeb97b994ab7a476 | 2 +- ...join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 | 2 +- ...join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 | 2 +- ...join_9-48-21269869cd3aaf4ade2170d9017de018 | 2 +- ...join_9-50-4e0e8cd0626a84b21ca7d2f633623578 | 2 +- ...join_9-52-2fe7b834b341bf18e36cd79dd00ec16a | 2 +- ...join_9-54-43ba2c72db9db1ec18d835ec978f8da1 | 2 +- ...join_9-56-b889b147255231f7fe44bd57e1f8ba66 | 2 +- ...join_9-58-aa8d713ad4e19b72b5bd7628d60c295e | 2 +- ...join_9-60-b9d66e78b8898a97a42d1118300fa0ce | 2 +- ...join_9-62-b0ca9e20cd48457e6cf1c313d5505213 | 2 +- ..._join_9-7-24ca942f094b14b92086305cc125e833 | 2 +- ..._join_9-8-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._join_9-9-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._table1-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ..._table2-3-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...e_union-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...between-0-df3cf89fcf2ef64199a582fae14a3321 | 1 - ...nstant-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...onstant-0-8c922b2264278dd481ef0dff2088e2b8 | 1 - ...onstant-1-604fde0397444c5c9f1d70e6287de664 | 1 + ..._format-3-84db2ef4a7f2151e26457db559b862d9 | 500 ---- ...olserde-3-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...olserde-2-7e72160489bbb59dadf24e0cc239a5f4 | 4 - ...able_1-1-aca7ae366662c9698b5d2c01a6cb3948} | 0 ...table_1-3-d6518380547e7eef338886f3bdc7bdd2 | 2 +- .../cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a | 2 +- ..._to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 | 1 - ...ar_udf-0-4de1b2fa0908a3d856474aae2bc38c08} | 0 ...har_udf-1-5b1e7c580ed5e756d30a4c557af8902} | 0 ...har_udf-2-880ce74a83bb2bfb9c9bd584b8874ac} | 0 ...cluster-1-707a2295731e0d631a6c5f71c745c8d5 | 2 +- ...luster-11-dcf78a6537ba2b4d4b828a9a27cf545e | 2 +- ...luster-13-12635b4b7f34eba5554d5b892b5b64e7 | 2 +- ...luster-15-b4c15c85c18f310f1b5bc56a78ad94b0 | 2 +- ...luster-17-62979aa9e6b4e6ffb44ec452aabbef65 | 2 +- ...luster-19-e5284c7a7c36ee55740bd127ef4bf8c7 | 2 +- ...luster-21-4787b258a786cf195bcb59cd90f6013f | 2 +- ...luster-23-b66ed6ead4deecd49f0f67de1f2bab2e | 2 +- ...luster-25-f57ce48b6a6e671b58c96535ab482b6a | 2 +- ...cluster-3-530671e2a5b8983c60cfedaf486f0f0f | 2 +- ...cluster-5-e99040f6a24c53a4c89801ff3663ff72 | 2 +- ...cluster-7-a22600d60c81a25061b1e20b6726c691 | 2 +- ...cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 | 2 +- ...hortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f | 11 - ..._partlvl-4-30d92b61681b9ae7786ed46c3c3e808 | 58 - ...partlvl-5-f37a302cb19b4fe1c8280c08153294a3 | 129 - ...partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 | 58 - ...partlvl-8-dc5682403f4154cef30860f2b4e37bce | 129 - ..._tbllvl-3-7c45bd1125420b85a0374fecbf947a95 | 73 - ..._tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 | 141 - ..._tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d | 89 - ...ombine1-0-84b74227c9f1563f530cd3ac3b333e54 | 2 +- ...ombine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 2 +- ...ombine1-2-c95dc367df88c9e5cf77157f29ba2daf | 2 +- ...ombine1-3-6e53a3ac93113f20db3a12f1dcf30e86 | 2 +- ...ombine1-4-84967075baa3e56fff2a23f8ab9ba076 | 2 +- ...ombine1-5-2ee5d706fe3a3bcc38b795f6e94970ea | 2 +- ...ombine1-6-1d1f97cce07323812de3027920b04b75 | 2 +- ...ombine1-9-e5ce23369b0ad260512a0f61c6969b73 | 2 +- ...adoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...doop20-12-cd15ffd140539cf86090814729ec4748 | 8 - ...doop20-14-4695309eb4e91ef29c9857aa8fd6130c | 12 - ...adoop20-16-557997716a68312e8cae75428e3ce31 | 1 - ...doop20-18-2af7419c1d84fe155e23f3972e049b97 | 2 - ...adoop20-2-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...adoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...adoop20-4-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...adoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...adoop20-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...adoop20-7-16367c381d4b189b3640c92511244bfe | 1 - ...adoop20-8-99d1f07b2ce904afd6a809fd1814efe9 | 1 - ...adoop20-9-30cb07965e4b5025545361b948fc83c2 | 1 - ...ne2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...ne2_win-1-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...e2_win-11-cd15ffd140539cf86090814729ec4748 | 8 - ...e2_win-13-4695309eb4e91ef29c9857aa8fd6130c | 12 - ...ne2_win-15-557997716a68312e8cae75428e3ce31 | 1 - ...e2_win-17-2af7419c1d84fe155e23f3972e049b97 | 2 - ...ne2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...ne2_win-3-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...ne2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...ne2_win-5-a4fb8359a2179ec70777aad6366071b7 | 1 - ...ne2_win-6-16367c381d4b189b3640c92511244bfe | 1 - ...ne2_win-7-99d1f07b2ce904afd6a809fd1814efe9 | 1 - ...ne2_win-8-30cb07965e4b5025545361b948fc83c2 | 1 - ...ombine3-11-30b8b79049113252bec1cbeac4018a3 | 12 - ...mbine3-12-f4baee25e0ad813258d67d707a6fc43b | 12 - ...mbine3-13-1c359bedf474e8e26f3b3562f7af6edc | 30 - ...mbine3-21-8ba8f8723c1530be062cefc2d9246e56 | 30 - ...binary-1-44b15c6c9600109e064a5ea5f9c81051} | 0 ...oolean-1-72ee4bdd5cea69136940dc40e6890e1d} | 0 ...boolean-2-fbea367698de65e22d4d660a518ea95e | 2 +- ...boolean-3-a14d8a5835c94829271f9f463d96d83d | 2 +- ...double-1-8f634b9e334fd58e71844e6283d9794d} | 0 ...s_long-1-9313f166464633b3929707d7ef11d758} | 0 ...string-1-3491ef2747a8f34899108d4ae8ebc7eb} | 0 ..._string-1-db089ff46f9826c7883198adacdfad59 | 4 +- ...izer1-26-5522db58d123d1bec48b6e71c1b258f3} | 0 ...izer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65} | 0 ...mizer1-33-7722bcc896718b584f78cecdab1fdc9f | 16 + ...izer1-35-e1d4857548e626bb31d70c096b8d0a95} | 0 ...mizer1-36-7722bcc896718b584f78cecdab1fdc9f | 16 + ...izer1-38-ef6502d6b282c8a6d228bba395b24724} | 0 ...izer1-39-ea87e76dba02a46cb958148333e397b7} | 0 ...izer1-41-b79b220859c09354e23b533c105ccbab} | 0 ...izer1-42-ea87e76dba02a46cb958148333e397b7} | 0 ...izer1-44-638e5300f4c892c2bf27bd91a8f81b64} | 0 ...izer1-45-66010469a9cdb66851da9a727ef9fdad} | 0 ...imizer1-47-3514c74c7f68f2d70cc6d51ac46c20} | 0 ...izer1-48-66010469a9cdb66851da9a727ef9fdad} | 0 ...izer1-49-b9d963d24994c47c3776dda6f7d3881f} | 0 ...izer1-50-7490df6719cd7e47aa08dbcbc3266a92} | 0 ...izer1-51-e71195e7d9f557e2abc7f03462d22dba} | 0 ...izer1-52-777edd9d575f3480ca6cebe4be57b1f6} | 0 ...izer1-53-73da9fe2b0c2ee26c021ec3f2fa27272} | 0 ...izer1-54-e71195e7d9f557e2abc7f03462d22dba} | 0 ...izer1-55-b1e2ade89ae898650f0be4f796d8947b} | 0 ...izer1-57-fcf9bcb522f542637ccdea863b408448} | 0 ...mizer1-58-3070366869308907e54797927805603} | 0 ...izer1-60-dad56e1f06c808b29e5dc8fb0c49efb2} | 0 ...mizer1-61-3070366869308907e54797927805603} | 0 ...izer1-62-b9d963d24994c47c3776dda6f7d3881f} | 0 ...izer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4} | 0 ...mizer1-64-a6bba6d9b422adb386b35c62cecb548} | 0 ...izer1-65-777edd9d575f3480ca6cebe4be57b1f6} | 0 ...izer1-66-d6bbaf0d40010159095e4cac025c50c5} | 0 ...mizer1-67-a6bba6d9b422adb386b35c62cecb548} | 0 ...mizer15-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer15-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer15-3-e149747103059314a9984235702b24b6 | 2 +- ...mizer15-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer15-5-9914f44ecb6ae7587b62e5349ff60d04 | 2 +- ...mizer15-7-e149747103059314a9984235702b24b6 | 2 +- ...imizer2-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer2-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-10-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-12-e6b368bfaea4d2838f8038b3bd29db06 | 2 +- ...mizer2-13-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-15-18f10d12e8bfa473a916c2f528500538 | 2 +- ...mizer2-16-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-18-18f10d12e8bfa473a916c2f528500538 | 2 +- ...mizer2-19-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e | 2 +- ...mizer2-22-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e | 2 +- ...mizer2-25-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-27-9b22dad2843cdc379d90687745561104 | 2 +- ...mizer2-28-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer2-3-d915fbdd493869aec42f548bdb66598d | 2 +- ...mizer2-30-9b22dad2843cdc379d90687745561104 | 2 +- ...mizer2-31-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b | 2 +- ...mizer2-34-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b | 2 +- ...imizer2-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer2-6-d915fbdd493869aec42f548bdb66598d | 2 +- ...imizer2-7-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...imizer2-9-e6b368bfaea4d2838f8038b3bd29db06 | 2 +- ...imizer3-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer3-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer3-10-e3d5ff08760b877d49c0f10f63776325 | 2 +- ...mizer3-11-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer3-12-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer3-14-1f13e03988991067d13a9f3f1b36fcf5 | 2 +- ...mizer3-15-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer3-17-1f13e03988991067d13a9f3f1b36fcf5 | 2 +- ...mizer3-18-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer3-19-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer3-21-1f13e03988991067d13a9f3f1b36fcf5 | 2 +- ...imizer3-3-e3d5ff08760b877d49c0f10f63776325 | 2 +- ...imizer3-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer3-6-e3d5ff08760b877d49c0f10f63776325 | 2 +- ...imizer3-7-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer3-8-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer4-1-c0f14def6a135cc50cba364e810ce28e} | 0 ...mizer4-10-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e | 2 +- ...mizer4-13-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-14-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e | 2 +- ...mizer4-17-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer4-18-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer4-20-6ad79a473a876923a247f0cb57bb4208 | 2 +- ...mizer4-21-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-23-6ad79a473a876923a247f0cb57bb4208 | 2 +- ...mizer4-24-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-26-62a0fd05be48759c39f3c284458dde9b | 2 +- ...mizer4-27-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf | 2 +- ...mizer4-3-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...mizer4-30-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf | 2 +- ...mizer4-33-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 | 2 +- ...mizer4-36-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer4-38-85fff71b240d0e26bab91d670c1349f2 | 2 +- ...mizer4-39-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer4-41-85fff71b240d0e26bab91d670c1349f2 | 2 +- ...mizer4-5-b5777cff7c522c4b527f77988e7f6bf1} | 0 ...imizer4-6-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer4-7-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...imizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e | 2 +- ...imizer6-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer6-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-10-25b1fe48ef86952fc7766fb74b63bf21 | 2 +- ...mizer6-11-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer6-12-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-14-84463190baec77d61b287a071c8886db | 2 +- ...mizer6-15-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-17-84463190baec77d61b287a071c8886db | 2 +- ...mizer6-18-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 2 +- ...mizer6-21-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de | 2 +- ...mizer6-24-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...mizer6-25-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-27-e149747103059314a9984235702b24b6 | 2 +- ...mizer6-28-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer6-3-25b1fe48ef86952fc7766fb74b63bf21 | 2 +- ...mizer6-30-e149747103059314a9984235702b24b6 | 2 +- ...mizer6-31-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-33-15d991127dc684513e2fff1aea3f1560 | 2 +- ...mizer6-34-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-36-15d991127dc684513e2fff1aea3f1560 | 2 +- ...mizer6-37-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-39-813d72763b5e9e3bed0f06232f55c8b8 | 2 +- ...imizer6-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-40-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-42-813d72763b5e9e3bed0f06232f55c8b8 | 2 +- ...mizer6-43-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-45-ff6c54b483726ef15c90a4c68dc659a0 | 2 +- ...mizer6-46-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-48-ff6c54b483726ef15c90a4c68dc659a0 | 2 +- ...mizer6-49-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer6-51-4746d944f4193018017984ca2df3c60d | 2 +- ...mizer6-52-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-54-4746d944f4193018017984ca2df3c60d | 2 +- ...mizer6-55-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer6-56-24ca942f094b14b92086305cc125e833 | 2 +- ...mizer6-58-4746d944f4193018017984ca2df3c60d | 2 +- ...imizer6-6-25b1fe48ef86952fc7766fb74b63bf21 | 2 +- ...imizer6-7-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer6-8-24ca942f094b14b92086305cc125e833 | 2 +- ...imizer7-0-24ca942f094b14b92086305cc125e833 | 2 +- ...imizer7-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer7-10-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer7-12-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...mizer7-13-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer7-15-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...imizer7-3-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...imizer7-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer7-6-22d71fb589c53776dabb4696b38c4a42 | 2 +- ...imizer7-7-24ca942f094b14b92086305cc125e833 | 2 +- ...imizer7-8-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...imizer7-9-4d3e60a0e9bd8c12fdba4e010493537d | 2 +- ...imizer8-0-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...imizer8-1-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...mizer8-10-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...timizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f | 2 +- ...mizer8-13-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...mizer8-15-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer8-3-932db4b9935e4bc3d21dd33a8d12c275 | 2 +- ...imizer8-4-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...imizer8-6-932db4b9935e4bc3d21dd33a8d12c275 | 2 +- ...imizer8-7-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...ptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f | 2 +- ... count-1-c47c4abedf055b4e734232fd2e274d55} | 0 .../count-10-d0f804c7d06375db74a0fcf5f17603c6 | 2 +- .../count-12-944f53db544c07a7b38a0544a21d8e13 | 2 +- .../count-2-461bad3feb7dbc25fb35d45c6876d698 | 2 +- .../count-3-dbcec232623048c7748b708123e18bf0 | 2 +- .../count-5-d0f804c7d06375db74a0fcf5f17603c6 | 2 +- .../count-7-944f53db544c07a7b38a0544a21d8e13 | 2 +- .../count-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...db name-1-417609d2bb67ba26de38e92ad834008f | 500 ---- ...able as-1-b9002c1d71895be765575b62656d1928 | 2 +- ...reate_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d | 1 - ...eate_1-10-b9c0b95624e601614ea2561b83aaf0ba | 4 - ...eate_1-12-fe8680bc5ef6fe3e64c68d2638e10785 | 2 - ...reate_1-13-5715f85456733a71fb7c83f1b254b3a | 4 - ...eate_1-15-fd9415e340cf54a9473cc4bf86673816 | 2 - ...eate_1-16-c99c700ca511b68577aae8ae513a4a32 | 4 - ...reate_1-2-ecd02bc3563cd6b60b8394956cb69084 | 2 - ...reate_1-3-c27702ff131e0ecfd71f1e1779fbe365 | 4 - ...reate_1-6-52dc9f900d7f7a559698aff9565f061a | 2 - ...reate_1-7-8564aa6cc2b0ee85292b3e8976fe9001 | 4 - ...reate_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 | 2 - ..._escape-1-ecd02bc3563cd6b60b8394956cb69084 | 2 - ..._escape-2-c27702ff131e0ecfd71f1e1779fbe365 | 4 - ...e_escape-4-7d84873a6ce03e0e408fa67ef5dd733 | 1 - ...ricudaf-0-499d36ed8229cbf74a07b59914bdf717 | 9 - ...ricudaf-3-d3b1af3e7f037de800255d9411a690e8 | 1 - ...ericudf-0-dd23fb149bb6d6937b838334559ad8d1 | 10 - ...tformat-2-8c2e4081b9d69297c35faab0a2ff86dc | 4 +- ...utformat-5-8552731917a8260c25e6df79b83bf5c | 4 +- ...tformat-8-33a4312fd617c5c9f2a560dc6ce868a5 | 4 +- ...ate_like-1-30485a2507b60b96cad3d293527e6af | 27 - ...te_like-10-7d84873a6ce03e0e408fa67ef5dd733 | 1 - ...e_like-11-ba64f0122b21f605d8b2928753784130 | 2 - ...te_like-3-eea111a209cf8895f31f64699669c705 | 27 - ...te_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 | 28 - ...e_view-10-eea111a209cf8895f31f64699669c705 | 10 +- ...e_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 | 4 +- ...e_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 | 10 +- ...e_view-25-87a663f8fd80110a9cee249535037c0d | 14 +- ...ke_view-5-ecd02bc3563cd6b60b8394956cb69084 | 4 +- ...ike_view-6-30485a2507b60b96cad3d293527e6af | 10 +- ...ke_view-9-52dc9f900d7f7a559698aff9565f061a | 4 +- ...ressed-10-d6fee0f05fa9b04cb7c557862402c929 | 14 - ...ressed-11-614c34f9e88015f21bffc4b8930bc95d | 1 - ...ressed-12-4d89cbe49f710527b54e6262472f0320 | 1 - ...pressed-2-84b74227c9f1563f530cd3ac3b333e54 | 1 - ...pressed-6-d6fee0f05fa9b04cb7c557862402c929 | 14 - ...pressed-7-614c34f9e88015f21bffc4b8930bc95d | 1 - ...pressed-8-4d89cbe49f710527b54e6262472f0320 | 1 - ...ed_type-1-ecd02bc3563cd6b60b8394956cb69084 | 8 +- ...ed_type-2-c27702ff131e0ecfd71f1e1779fbe365 | 10 +- ...d_type-3-280fd0d9876e475d7dcf5383876ebc79} | 0 ...ce_view-2-30dc3e80e3873af5115e4f5e39078a13 | 30 - ..._table1-0-cafed8ca348b243372b9114910be1557 | 2 +- ..._table1-4-f3f1642674545762a4bff5cb75634e20 | 12 +- ..._table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e | 12 +- ..._table1-6-d7a147c6b0a3609663628b43457b2cef | 14 +- ..._table-1-719851d0e8b89b51bdc6be4177455a92} | 0 ...te_udaf-0-a69c2b11dc912ef5444af32dce6aa33e | 10 - ...e_view-19-df2da604b5a8f2b236519510b5e4d34b | 1 - ...itioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 | 13 - ...itioned-5-d7a7d8592fca266745725192d3f875fc | 36 - ...heck_1-0-d782db598869f9b19e0fcf5ea2a83594} | 0 ...heck_1-1-bd8395c55fa2fc80b68eb043b7020b76} | 0 ...heck_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617} | 0 ...heck_1-3-32a3e6eb858d37f58e225d07fb323254} | 0 ...heck_1-4-36a6b6fb71ea08ff817dd40d1ffdb970} | 0 ...heck_1-5-103739f072b849d212dbc40919f92b74} | 0 ...heck_1-6-9a202e3bf15be2a310504a50920e7d25} | 0 ...heck_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac} | 0 ...heck_2-0-d782db598869f9b19e0fcf5ea2a83594} | 0 ...heck_2-1-bd8395c55fa2fc80b68eb043b7020b76} | 0 ...heck_2-2-24ca942f094b14b92086305cc125e833} | 0 ...heck_2-3-44d382ce6848d3f0b900b0808747d8e9} | 0 ...heck_2-4-c14792ccac2ca64e3e9e21af4fd12d2c} | 0 ...heck_2-5-32a3e6eb858d37f58e225d07fb323254} | 0 ...heck_2-6-36a6b6fb71ea08ff817dd40d1ffdb970} | 0 ...heck_2-7-103739f072b849d212dbc40919f92b74} | 0 ...heck_2-8-9a202e3bf15be2a310504a50920e7d25} | 0 ...heck_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac} | 0 ...varchar-7-8620d7f55849ab163b6b85f51abdaaec | 5 - ...varchar-8-c93df93c0e8688f9c7a6167589670d32 | 5 - ...le.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd | 2 +- ...e.table-0-c657beb729b6a7882309a203fc6f298e | 2 +- ..._drop-10-8db536f925bf0f5058f97897e145a661} | 0 ..._drop-11-1b0a6cff3151cfa0ef0a6f78587973a5} | 0 ..._drop-12-2ea883422b74b701711e14e61472ba06} | 0 ..._drop-13-e02a53f7e798d2741152526516f14941} | 0 ..._drop-14-8f0fe60664d020b2a42c7f5c0c7bed35} | 0 ..._drop-15-7928ac876f76c990fa21d74b6c9e14f6} | 0 ..._drop-16-f9036ff3f0a3101373bdbc9a52faf00e} | 0 ..._drop-17-95b906fd73935da8746b5277170e91e8} | 0 ..._drop-18-f65bf675b83871af7906741a60fa1318} | 0 ..._drop-19-d7fefbf585dbb67491e871ef58dca752} | 0 ..._drop-20-cbb84f0db4d55930a85cff28f7400bd0} | 0 ..._drop-21-eea49f066631be60519ae9d6b614d7d0} | 0 ..._drop-22-85833c3a68c29098827e438ff580bb94} | 0 ..._drop-23-84a5672989118a1b5792474c1469de90} | 0 ..._drop-24-441116797e8d95554c74472fa7644440} | 0 ..._drop-25-847ca97dd211889d83e5d487bfc44e23} | 0 ..._drop-26-bea20178515df24fcca04c0384f1c1b7} | 0 ..._drop-27-cb4b90a7f63c1646319ee7bb014a7750} | 0 ..._drop-28-53d67cbed634cff012dac41340bf6630} | 0 ..._drop-29-29d3232325eda40cbb0bd1786b7d070e} | 0 ...e_drop-3-db64b724719d27c7f0db4f51f5c4edaa} | 0 ..._drop-30-bbad0860316d8b9b1eed50d231f3ab5d} | 0 ..._drop-31-981f8e58caeae9cbbad3a113e3043de5} | 0 ..._drop-32-6c8e7590238b5aca1772721f0b914ece} | 0 ..._drop-33-2bc7864932f597bdf98bdc410b1a2d9c} | 0 ...e_drop-34-491138bed44a70cb783bb2b531e1d82} | 0 ..._drop-35-9e0285d0596607cdadf75a763a543866} | 0 ..._drop-36-e66471f3f1bbe2d4b3b214e47793656d} | 0 ..._drop-37-f6410721e3125a89836817136306eac4} | 0 ..._drop-38-7d45d97adebe50917a94cbe232c112a8} | 0 ..._drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1} | 0 ...e_drop-4-f21dd011aeb417043ed58c03fd5c3bf0} | 0 ..._drop-40-10073fb74a5c792322fc52389997695b} | 0 ..._drop-41-7164c585e1ef4d9036ed4db275811084} | 0 ..._drop-42-c55cffbfc4d950363be97ccdb028faf3} | 0 ..._drop-43-cc6860c125b8b62450cb858c72716dc2} | 0 ..._drop-44-de81fd80132350aedcd9f0d9a212fd94} | 0 ..._drop-45-64e22634663e57153119340583e93651} | 0 ..._drop-46-eb28b907b605e51026f9902287e1d90d} | 0 ..._drop-47-44e4d2bdab2dfa4583da47281ed00ba3} | 0 ..._drop-48-d419f4ff197d4291208c2028cd158909} | 0 ...e_drop-5-d419f4ff197d4291208c2028cd158909} | 0 ...e_drop-6-b7cf74929eabe781b0db79ed1043dc24} | 0 ...e_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9} | 0 ...e_drop-8-b3980119a4b2d8f531951a940cba3697} | 0 ...e_drop-9-97101266791d2b2c662bcde549422318} | 0 ...location-1-2b66b4554cf8ecf2ab6c325d4d89491 | 2 +- ...cation-10-c5cd9c57a13da7f345563fbd75da4e45 | 12 +- ...cation-11-9c36cac1372650b703400c60dd29042c | 2 +- ...ocation-4-be65cf649363681d54e593c42a5ecffb | 12 +- ...ocation-5-9c36cac1372650b703400c60dd29042c | 2 +- ...ocation-7-5698ac10441da07dbe3a947143c999c2 | 2 +- ...perties-0-e7bfbd9422685e9a3a6c9bd4965f828f | 2 +- ...perties-1-10de6a198e2b3f61974519ddd8623e68 | 2 +- ...erties-10-26c10ff2ec4a69b16589069ced427d23 | 2 +- ...perties-3-751417d45b8e80ee5cba2034458b5bc9 | 2 +- ...perties-5-51c0974df1125b233936f25ce709ba4a | 2 +- ...perties-6-26c10ff2ec4a69b16589069ced427d23 | 2 +- ...perties-7-e7bfbd9422685e9a3a6c9bd4965f828f | 2 +- ...perties-8-10de6a198e2b3f61974519ddd8623e68 | 2 +- ...date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1} | 0 ...date_3-3-c26f0641e7cec1093273b258e6bf7120} | 0 ...date_4-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...date_4-1-b84f7e931d710dcbe3c5126d998285a8} | 0 ...date_4-2-6272f5e518f6a20bc96a5870ff315c4f} | 0 ...date_4-3-4a0e7bde447ef616b98e0f55d2886de0} | 0 ...date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8} | 0 .../date_4-5-bee09a7384666043621f68297cee2e68 | 1 + ...date_4-6-b84f7e931d710dcbe3c5126d998285a8} | 0 ...arison-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...arison-1-69eec445bd045c9dc899fafa348d8495} | 0 ...rison-10-bcd987341fc1c38047a27d29dac6ae7c} | 0 ...rison-11-a9f2560c273163e11306d4f1dd1d9d54} | 0 ...rison-12-4a7bac9ddcf40db6329faaec8e426543} | 0 ...arison-2-fcc400871a502009c8680509e3869ec1} | 0 ...arison-3-b8598a4d0c948c2ddcf3eeef0abf2264} | 0 ...arison-4-14d35f266be9cceb11a2ae09ec8b3835} | 0 ...arison-5-c8865b14d53f2c2496fb69ee8191bf37} | 0 ...arison-6-f2c907e64da8166a731ddc0ed19bad6c} | 0 ...arison-7-5606505a92bad10023ad9a3ef77eacc9} | 0 ...parison-8-47913d4aaf0d468ab3764cc3bfd68eb} | 0 ...arison-9-1e5ce4f833b6fba45618437c8fb7643c} | 0 ..._join1-2-e967e1ef6b209dfa5bdc60021dcb1964} | 0 ..._serde-7-580096b3b48db26bea91b80e1e1b081a} | 0 ...te_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d} | 0 ...ddltime-1-426da52526f6f48c0ddeb0026fd566f1 | 4 - ...cimal_1-10-be179d261b9c42ed843dbf736b12e75 | 1 - ...imal_1-11-4c1fefa61e10a78f6406b526a60e4937 | 1 - ...imal_1-12-75b9add5e32a963cc9913c6ef4f84989 | 1 - ...cimal_1-5-cbe6b235663cf78e602673ed715a2f40 | 1 - ...cimal_1-6-91b7219bd8c67673e16cb970fcc7be1b | 1 - ...decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a | 1 - ...cimal_1-8-cdd0932288d3cc43636334439805769d | 1 - ...cimal_1-9-1504e1b00092e2c58bafcc9936ad178c | 1 - ...imal_2-10-f97d72aeb605ee18d34361c073552e92 | 1 - ...imal_2-11-58a090c30c59446d1e2b2a6c85fabf50 | 1 - ...imal_2-14-3105d1029ad981af9cf1039ad9410fc0 | 1 - ...imal_2-15-3266fde6f6ab80a8bc027de0d5574f02 | 1 - ...imal_2-16-dc565c6c5bc24fd8b32729ce91999580 | 1 - ...imal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d | 1 - ...imal_2-18-f7c34d67fd579c82c636415172ec675e | 1 - ...imal_2-19-f97d72aeb605ee18d34361c073552e92 | 1 - ...imal_2-20-58a090c30c59446d1e2b2a6c85fabf50 | 1 - ...imal_2-21-d72d68be1217c7b7a958f58456d85821 | 1 - ...imal_2-22-648e694eea042c59e8db30d067cb5bc8 | 1 - ...imal_2-23-a87b701c93a922b9e33ba16ae6a477ff | 1 - ...imal_2-27-a5ea3949eb5ab338916e4316c676c7f6 | 1 - ...imal_2-28-4a5410f96c6ef0843f12b0f593c104b1 | 1 - ...imal_2-30-26a71d79e41353830b4ada96de6e2b8a | 1 - ...cimal_2-5-3105d1029ad981af9cf1039ad9410fc0 | 1 - ...cimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 | 1 - ...cimal_2-7-dc565c6c5bc24fd8b32729ce91999580 | 1 - ...cimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d | 1 - ...cimal_2-9-f7c34d67fd579c82c636415172ec675e | 1 - ...imal_3-10-420614bb0789115e008c96a7ad822624 | 4 - ...imal_3-11-63913753553b16d6c24e063fb49fdd15 | 4 - ...cimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b | 38 - ...cimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 | 38 - ...cimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b | 38 - ...cimal_3-6-127a3a8400cae591c282dd24f8951e55 | 30 - ...cimal_3-7-9d4f27d4a4819113c5083462baa72052 | 30 - ...cimal_3-8-f65f4df6e3d971d575654ade4b4e4800 | 17 - ...cimal_3-9-b54243d38214362f9a9b1831548faac4 | 56 - ...imal_4-2-85c3185beb011f5c1e6856fc773a7484} | 0 ...imal_4-3-1451d7491441c1632fd5f751876cce6e} | 0 ...imal_4-4-1bf9ff1d72a06c33885ba695adf2511d} | 0 ...cimal_4-6-693c2e345731f9b2b547c3b75218458e | 2 +- ...cimal_4-7-f1eb45492510cb76cf6b452121af8531 | 2 +- ...l_join-0-4668e9dee2cd7a32f2b7311d7cd35508} | 0 ...l_join-1-5098974222b22a21ed847c7906df9313} | 0 ...ecision-3-42cb35d680b3caeeb22e1c4865b8264b | 75 - ...ecision-4-38aaeba3e587b4dac72e26c4b02029fc | 75 - ...ecision-5-bb27734245ecbd0511be91af21c3b9ef | 75 - ...recision-6-b2547e6ef33325b2da12ce91b57af21 | 75 - ...ecision-7-ee5b7767c7fbb8614bb4ef907e8737b7 | 75 - ...ecision-8-6e6bd4655de124dc1fc482ce0d11930e | 75 - ...ecision-9-e7b465fbeb49487b2a972a314e2c01ab | 1 - ...on_name-1-9de8e5f66c536d4ace89c61759db829c | 2 +- ...on_name-3-a7047012b4bce0158edaafe5cf0a57be | 2 +- ...imiter-1-121ff21e6931a82235de8301118cbed8} | 0 ...limiter-2-d7137294d2e53ea6edc259943e4c6069 | 2 +- ...limiter-3-176724f76343433a8f2e6131b12206d7 | 2 +- ..._indent-1-5536eb772d43014b971c6da3a0c44904 | 6 - ..._indent-2-91bffa09f4f2caaaa6864bf935c2ea94 | 32 - ...se_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 1 - ...e_json-11-b05391400dc31139998dc3abaaf86320 | 1 - ...ase_json-2-8e7cfe3e6069e796124ca940125385a | 1 - ...se_json-3-d097973152d91fa8072facb0f739e304 | 1 - ...se_json-4-549981e00a3d95f03dd5a9ef6044aa20 | 1 - ...se_json-5-a3ee372283f45479db3f2cd7ebeedc8c | 1 - ...ase_json-8-8e7cfe3e6069e796124ca940125385a | 1 - ...se_json-9-d097973152d91fa8072facb0f739e304 | 1 - ...itioned-3-b80c7ae3530bfdbc8e865d49742da826 | 8 +- ...ed_json-4-b80c7ae3530bfdbc8e865d49742da826 | 1 - ..._pretty-1-dbfaa12f26f99277b8397379189172cf | 1 - ...pretty-10-dbfaa12f26f99277b8397379189172cf | 1 - ...pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 | 12 - ...pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 | 1 - ...pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 | 24 - ...pretty-15-db4abe06aba81e685c52db6b43ba7c03 | 1 - ...pretty-16-ada55b65b72e0d65563ad6161e005f22 | 37 - ..._pretty-2-713712e0f6f18144d1f3a522e9b98861 | 23 - ..._pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 | 1 - ..._pretty-4-1546db18568697fa68a7fc781802d255 | 14 - ..._pretty-5-ce1966d8626096821b404ab8745c4914 | 1 - ..._pretty-6-1546db18568697fa68a7fc781802d255 | 44 - ..._pretty-7-db4abe06aba81e685c52db6b43ba7c03 | 1 - ..._pretty-8-1546db18568697fa68a7fc781802d255 | 31 - ...syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 | 12 - ...syntax-11-ab161e38c5d66b6c344c8372160ac74f | 33 - ...syntax-12-90c7890e1aa28e94520f35f5679560a4 | 1 - ...syntax-13-7c1216f9852d454bf93256e5a2588758 | 1 - ...syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 | 3 - ...syntax-15-e420b255509153b3326f00dcd25d11e4 | 1 - ...syntax-16-5043ee273a313d28adeca74fd33739a7 | 1 - ...syntax-17-c97a9e691cc08199678ead7f79d58b58 | 3 - ...syntax-18-69cecdc1889b7feb40b46c95524aaad4 | 1 - ...syntax-19-c1c3359705e256d7641bbffab00c43fa | 1 - ...syntax-20-781dd449a9b26dc54f3bba8e5178cd8a | 3 - ...syntax-21-5bac87eeb7e71928d01275b006720de3 | 10 - ...syntax-22-719a15ffd0018bb2898f9045be886e0f | 12 - ...syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 | 32 - ...syntax-24-ee226b42db35b219702319858e925468 | 10 - ...syntax-25-b6e10514fb473803c44bc793b9f9713e | 12 - ...syntax-26-4851dc65e26ec33c605ab1ed65b59bec | 32 - ..._syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 | 10 - ..._syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d | 12 - ...be_syntax-8-9e40f8077f34fa35fccfeae972e8c4 | 33 - ..._syntax-9-fb744775fb30d92297534d29b6eafd6b | 10 - ...le_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b | 1 - ...le_json-2-9c36cac1372650b703400c60dd29042c | 1 - ...le_json-3-576670af142149302decb9bf8662e68a | 1 - ...le_json-4-4a57591b392bb8fd18238d068d191721 | 1 - ...le_json-6-ac49d26a0211b804fee89bbe0808f430 | 1 - ...le_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 | 1 - ...le_json-9-b05391400dc31139998dc3abaaf86320 | 1 - ..._formats-4-a4890f2b20715c75e05c674d9155a5b | 2 +- ...t_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c | 2 +- ..._check-2-2d27f92dfced693fa3a68ecce5e2e838} | 0 ..._check-4-a2150709a6ff73326bdf4865dd124a23} | 0 ...ge_off-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...nge_off-0-a071dedef216e84d1cb2f0de6d34fd1a | 1 - ...ge_off-1-a071dedef216e84d1cb2f0de6d34fd1a} | 0 ...nge_off-1-ce3797dc14a603cba2a5e58c8612de5b | 1 - ...ge_off-2-ce3797dc14a603cba2a5e58c8612de5b} | 0 ...ge_off-3-f5340880d2be7b0643eb995673e89d11} | 0 ...nge_off-4-34064fd15c28dba55865cb8f3c5ba68c | 1 - ...ge_off-4-714ab8c97f4d8993680b91e1ed8f3782} | 0 ...nge_off-5-34064fd15c28dba55865cb8f3c5ba68c | 1 + ...ge_off-6-f40a07d7654573e1a8517770eb8529e7} | 0 ..._stats-0-418ec894d08c33fd712eb358f579b7a0} | 0 ..._stats-1-10987e425ba8ba8d9c01538f16eab970} | 0 ..._stats-2-a2d8f812612283b20ec3f1e92a263440} | 0 ...t_stats-3-a4397664f1f109ef0aa5ff36961b25b} | 0 ..._stats-4-e540680af4a857404a0cb7cabc1bcf31} | 0 ..._stats-5-32e9736bf27c1d2e4399a8125e14befc} | 0 ..._stats-6-297a53801744e6c4786e315e32c6189a} | 0 ..._stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d} | 0 ..._stats-8-ca16024e6f5399b1d035f5b9fd665163} | 0 ..._stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8} | 0 ...n_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee} | 0 ...n_dirs-3-46fe5bb027667f528d7179b239e3427f} | 0 ...n_dirs-4-26dcd2b2f263b5b417430efcf354663a} | 0 ...n_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea} | 0 ...n_dirs-6-7a9e67189d3d4151f23b12c22bde06b5} | 0 ...n_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd} | 0 ...n_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea} | 0 ...n_dirs-9-40110efef10f6f7b873dcd1d53463101} | 0 ...unction-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 2 +- ...p_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 2 +- ...n_dirs-0-a99af48bbcbaba062e9bc387ae2b4975} | 0 ...n_dirs-1-e67a0f7ff61a97b2c49386890ea88c54} | 0 ...n_dirs-2-19915227905aab376d918b3cada85c25} | 0 ...n_dirs-3-e32b952789a115ff02201dfa618d92b2} | 0 ...n_dirs-4-58aff7fa05fba3a7549629a17e285036} | 0 ...n_dirs-5-e32b952789a115ff02201dfa618d92b2} | 0 ...n_dirs-6-4642f8a18bf7409849f2e91d7a05f352} | 0 ...itions-10-9120e865aa132bac6e0a29c907f0b760 | 2 +- ...titions-4-9120e865aa132bac6e0a29c907f0b760 | 2 +- ...titions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 | 12 - ...titions-7-9120e865aa132bac6e0a29c907f0b760 | 2 +- ...titions-8-46a4f646bbc04f70e7ae5ed992f102f1 | 2 +- ..._filter-1-e3eda6672f6602a1b9bc93ef789662f6 | 12 +- ...filter-11-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-13-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-15-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-17-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-19-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-21-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter-22-46a4f646bbc04f70e7ae5ed992f102f1 | 2 +- ...filter-24-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter2-1-e3eda6672f6602a1b9bc93ef789662f6 | 12 +- ...ilter2-10-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...ilter2-12-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...lter2-13-8117981303487dc4c4873356931ef26a} | 0 ...ilter2-14-83e3e422cdf9403523fa60d75376d7d7 | 1 + ...lter2-15-67d75c72ea2d3982c55f3a850d93f83c} | 0 ...ilter2-7-f34625fd49a5e655cba3abb5cb8c5417} | 0 ...filter2-8-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter3-1-e3eda6672f6602a1b9bc93ef789662f6 | 12 +- ...ilter3-10-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...ilter3-12-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...filter3-8-83e3e422cdf9403523fa60d75376d7d7 | 2 +- ...otection-3-312aa26fdea6da7907e3a91f75e36f1 | 10 +- ...p_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 | 2 +- ..._table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 | 2 +- ..._table2-5-1fdd850f6c301619f91eb58c890f2ad4 | 2 +- ...n_dirs-0-97b52abf021c81b8364041c1a0bbccf3} | 0 ...n_dirs-1-f11a45c42752d06821ccd26d948d51ff} | 0 ...n_dirs-2-c0b85445b616f93c5e6d090fa35072e7} | 0 ...n_dirs-3-10a71bca930d911cc4c2022575b17299} | 0 ...n_dirs-4-b2ca31dd6cc5c32e33df700786f5b208} | 0 ...n_dirs-5-10a71bca930d911cc4c2022575b17299} | 0 ...n_dirs-6-d1c175a9d042ecd389f2f93fc867591d} | 0 ...efault-0-3cd14bc5b126ff8b337c4abc09134260} | 0 ...default-1-16367c381d4b189b3640c92511244bfe | 2 +- ...efault-2-ce929ee6a92b81d8080ca322c1c38a4b} | 0 ...efault-3-f3a5a998099b756a21cf9122a15b09d5} | 0 ...efault-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49} | 0 ...efault-5-fed732d2e7d94a4fc02e7694f9f9a39c} | 0 ...efault-6-adab6a0187003ab7ee6f217c9e409d91} | 0 ...default-7-e707e693aa61edf87768fb71f6e936e1 | 2 +- ...efault-8-adab6a0187003ab7ee6f217c9e409d91} | 0 ...efault-9-4b099f87e221b8fd5c0d0d4a97c0d146} | 0 ...e_order-2-ffe97dc8c1df3195982e38263fbe8717 | 1 - ...e_order-7-5fb418612e7c2201790d6f318c715ccf | 10 - ...e_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 | 10 - ...escape1-6-134b596abc363f0bfa7f770732ebb960 | 1 - ...escape1-7-486585cbb4de5bc908dde4c601dd7c17 | Bin 252 -> 0 bytes ...escape2-8-134b596abc363f0bfa7f770732ebb960 | 1 - ...escape2-9-486585cbb4de5bc908dde4c601dd7c17 | Bin 252 -> 0 bytes ..._compat-5-75eed21390055f8e397c81ab9d253a32 | 1 - ..._compat-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...ll_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ll_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ..._parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ..._parts-11-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...me_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...me_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ne_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ne_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...overlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...overlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ..._rename-5-75eed21390055f8e397c81ab9d253a32 | 1 - ..._rename-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...overlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...overlap-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...managed-3-be31972099603addb71187f19f7cd25d | 1 - ...managed-4-46667daf88f9c8b9b758ced6a6b28ef1 | 1 - ...xternal-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...xternal-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...ocation-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...ocation-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...ocation-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...ocation-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...xisting-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...xisting-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...al_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...al_part-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...xternal-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...xternal-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...managed-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...xternal-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...xternal-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ocation-6-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ocation-7-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ocation-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ocation-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...ocation-8-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...ocation-9-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...success-6-75eed21390055f8e397c81ab9d253a32 | 1 - ...success-7-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...success-5-75eed21390055f8e397c81ab9d253a32 | 1 - ...success-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...success-5-677ddd4b14eb6f19cfcf0c3d57f54e22 | 1 - ...success-6-a9f93b4185a714e4f6d14171d10a6c07 | 1 - ...success-5-d2ec90909f243a767be1aa299720f45d | 1 - ...success-6-2a161bb8d69da9d6e3679ca3677a0de5 | 1 - ...arrange-0-6f7c8515c354fb050829ebd66413425} | 0 ...rrange-1-378d42317b39c6519f15bd2f99c5ddc4} | 0 ...range-10-3f2680208772a0e51aefc4ef5604dddf} | 0 ...range-11-f2ca12a948fd9b5b842168e7c7d7b768} | 0 ...range-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e} | 0 ...range-13-5baad22ed7efa18d73eb8349e57cf331} | 0 ...range-14-490d6253b73064ce403e4d04a8bc18f3} | 0 ...rrange-2-24ca942f094b14b92086305cc125e833} | 0 ...rrange-3-3b0f76816be2c1b18a2058027a19bc9f} | 0 ...rrange-4-86473a0498e4361e4db0b4a22f2e8571} | 0 ...rrange-5-d0ec6d66ff349db09fd455eec149efdb} | 0 ...rrange-6-cda81d86d127fca0e2fbc2161e91400d} | 0 ...rrange-7-5b13cfa4b730e38ef2794c1532968e04} | 0 ...rrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12} | 0 ...rrange-9-73b9ac83dbc9874dc9379ad4364d40ac} | 0 ...egation-0-739356d010a919138911f295fac81607 | 2 +- ...egation-2-f015c961b6c34ac56cb8fc52fb36d7c7 | 2 +- ...mat_mix-5-c2d0da9a0f01736a2163c99fc667f279 | 2 +- ...mat_mix-6-4b658b3222b7a09ef41d023215e5b818 | 2 +- ...ncefile-2-80ec34a069bc561aa6dc87314391b131 | 6 +- ...at_text-2-80ec34a069bc561aa6dc87314391b131 | 6 +- ...eaktask-3-550e9b759fb088a81feddeff2e4be64e | 2 +- ...ktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd | 2 +- ...ktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 | 2 +- ...ktask2-12-c608f51882260caec7bb9c57a0ba3179 | 2 +- ...ktask2-4-fe8b55e4e4098d7a2662338783a50306} | 0 ...ktask2-5-cf724251613216ec54f8ac2e6b9b92fd} | 0 ...ktask2-6-8c782ae8f8245bdbe90d068a6b577d1e} | 0 ...aktask2-8-604cf64d16b9d438ee187a10d9f6352e | 2 +- ...aktask2-9-6d551990a8a745dde1cb5244947b81a1 | 2 +- ...roupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby1-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d | 2 +- ...roupby1-5-dd7bf298b8c921355edd8665c6b0c168 | 2 +- ...roupby1-7-c2c54378ffce53ade73a9dda783903e7 | 2 +- ...oupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...oupby11-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...oupby11-6-686ee4a5b2e24e51ba1d41b42215accd | 2 +- ...oupby11-7-149d359546ab38226ffeb023d7414b3d | 2 +- ...upby12-0-67e864faaff4c6b2a8e1c9fbd188bb66} | 0 ...upby12-1-13ab74a58da514fe01dbeda0c3e79883} | 0 ...upby12-2-fd150794945892f3c926a1881cd819f4} | 0 ...upby12-3-8018bb917a0706925c14421ec2761663} | 0 ...oupby12-4-adc1ec67836b26b60d8547c4996bfd8f | 309 ++ ...1_limit-0-83c59d378571a6e487aa20217bd87817 | 2 +- ..._limit-3-607512a8dd16cd9ddde561eeabfa51db} | 0 ...1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 | 2 +- ...by1_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by1_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by1_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by1_map-6-adc1ec67836b26b60d8547c4996bfd8f | 309 -- ...by1_map-6-c83b156356c269b6aef263640a4f6b7b | 309 ++ ...p_nomap-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...p_nomap-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...ap_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 | 2 +- ...p_nomap-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 309 -- ...ap_skew-6-c83b156356c269b6aef263640a4f6b7b | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-c2c54378ffce53ade73a9dda783903e7 | 309 -- ..._noskew-6-f7eda18efd187ec4bf4cb135833746cb | 309 ++ ...roupby2-0-43d53504df013e6b35f81811138a167a | 2 +- ...roupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby2-2-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby2-6-41dfb7b036cae4972b275a0262ea2e4c | 2 +- ...2_limit-0-83c59d378571a6e487aa20217bd87817 | 2 +- ..._limit-1-a56d6499aef913e11ef599ac8b4f2a25} | 0 ...2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 | 5 - ...2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a | 5 + ...by2_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by2_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by2_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by2_map-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...by2_map-6-c83b156356c269b6aef263640a4f6b7b | 10 + ...istinct-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-232957c90d04497da7fe4e599e0dbb94 | 10 + ..._noskew-6-41dfb7b036cae4972b275a0262ea2e4c | 10 - ...istinct-6-41dfb7b036cae4972b275a0262ea2e4c | 10 - ...roupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby4-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby4-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...by4_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by4_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by4_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by4_map-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...roupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby5-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby5-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...by5_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by5_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by5_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by5_map-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 309 -- ..._noskew-6-c83b156356c269b6aef263640a4f6b7b | 309 ++ ...roupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby6-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby6-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...by6_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by6_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by6_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by6_map-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ...by6_map-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ...ap_skew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-2d1fb04c7060fafe470e14061b2a5b6f | 10 + ..._noskew-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...roupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby7-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...roupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...roupby7-5-ed76c0068780120a6f23feefee303403 | 2 +- ...roupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...by7_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by7_map-1-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...y7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...y7_map-10-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...y7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...y7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...by7_map-2-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by7_map-3-83c59d378571a6e487aa20217bd87817 | 2 +- ...by7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...by7_map-7-ed76c0068780120a6f23feefee303403 | 2 +- ...reducer-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...reducer-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...educer-10-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...educer-10-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...reducer-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...reducer-6-ed76c0068780120a6f23feefee303403 | 2 +- ...reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...reducer-9-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...p_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...p_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...ap_skew-6-ed76c0068780120a6f23feefee303403 | 2 +- ...ap_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...ap_skew-9-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...noskew-10-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ..._noskew-2-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-3-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ..._noskew-7-ed76c0068780120a6f23feefee303403 | 2 +- ...reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...reducer-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...educer-10-96d0598a2a4c17041a31b908d1f065e5 | 2 +- ...reducer-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 2 +- ...reducer-6-ed76c0068780120a6f23feefee303403 | 2 +- ...educer-8-ab2390a3b0ba76907a7bee390b2924a1} | 0 ...reducer-9-652510b8dc20117c65511f06e6e73d73 | 2 +- ...roupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...roupby8-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...oupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby8-8-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...by8_map-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...by8_map-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...by8_map-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...by8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...by8_map-7-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...by8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...by8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...ap_skew-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...ap_skew-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...ap_skew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...ap_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ...ap_skew-7-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ...ap_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ...ap_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ..._noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ..._noskew-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ..._noskew-2-83c59d378571a6e487aa20217bd87817 | 2 +- ..._noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 309 -- ..._noskew-7-c1a8cf4ef8060a4703b0affe40496169 | 309 ++ ..._noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 | 309 -- ..._noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 | 309 ++ ...oupby9-10-7f98b724df05f51b3ec1f087a8da414e | 2 +- ...oupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...oupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...oupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...oupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...roupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...roupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...map_ppr-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...map_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...map_ppr-2-83c59d378571a6e487aa20217bd87817 | 2 +- ...map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...map_ppr-6-c83b156356c269b6aef263640a4f6b7b | 10 + ...istinct-6-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...istinct-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...stinct-10-6aa66df624cd32601218eee200977ce6 | 2 +- ...stinct-11-59a77127c166fef57504c011ccb427a6 | 2 +- ...istinct-5-6aa66df624cd32601218eee200977ce6 | 2 +- ...istinct-6-59a77127c166fef57504c011ccb427a6 | 2 +- ...istinct-7-293182ac89effb268855f5ac53e1ec11 | 2 +- ...educer-10-3b3ef4d975716744c85af560aa240abd | 10 - ...educer-13-38f492067df78144c272bb212633cc5e | 5 - ...educer-14-d7b3676444a9e95811184637dd0b3231 | 5 - ...educer-15-3b3ef4d975716744c85af560aa240abd | 10 - ...educer-16-b21ae88a5fc7f9b5c25ca954889ee421 | 10 - ...educer-17-7f344c2f066d74ecd135c43d39658bae | 32 - ...reducer-8-38f492067df78144c272bb212633cc5e | 5 - ...reducer-9-d7b3676444a9e95811184637dd0b3231 | 5 - ...educer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 | 2 +- ...educer2-5-376542befbcab97d864e874251720c40 | 2 +- ...educer2-6-8566e4c94d34b0fc4a71d34e89529c74 | 2 +- ...ducer3-10-73819ea1a7c0653a61652b3766afb003 | 2 +- ...ducer3-13-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ducer3-14-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...ducer3-17-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ducer3-18-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...educer3-4-521e0c1054cfa35116c02245874a4e69 | 2 +- ...educer3-5-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...educer3-8-521e0c1054cfa35116c02245874a4e69 | 2 +- ...educer3-9-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...g_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb | 2 +- ...g_float-1-4a435e268a327404f75725b82a32ee03 | 2 +- ...osition-0-422c2068a838f59324c1d9861225c824 | 1 - ...sition-10-b4c6c0aedcc924e2af65549b87b3f3f7 | 13 - ...sition-12-5583e5cfcf8083d45a3bd80434c1829f | 14 - ...sition-14-6f54558123eb5d63eeeb3e261c1dc5fb | 3 - ...osition-5-e2470670c5d709001fb17ecbc6e4f85d | 13 - ...osition-6-b4c6c0aedcc924e2af65549b87b3f3f7 | 13 - ...osition-9-e2470670c5d709001fb17ecbc6e4f85d | 13 - ...pby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...pby_ppr-1-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...pby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...istinct-5-adc1ec67836b26b60d8547c4996bfd8f | 10 - ..._sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ..._sort_1-1-ffe97dc8c1df3195982e38263fbe8717 | 1 - ...sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-14-bbdd53118f788d7bb679d094c41243c8 | 6 - ...sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ..._sort_1-2-fc30020d09151dc29be807795ad9475e | 1 - ...sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-24-d53196339980a00a619788bd799a32e7 | 5 - ...sort_1-28-c4ec0433a832ef551d70254957e3afca | 6 - ..._sort_1-3-fffea659b633b1f269b38556a7f54634 | 1 - ...sort_1-31-d53196339980a00a619788bd799a32e7 | 5 - ...sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ...sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ...sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...sort_1-52-c4ec0433a832ef551d70254957e3afca | 6 - ...sort_1-56-5373c5449884d95bc7db9dab55239a49 | 6 - ...sort_1-59-c4ec0433a832ef551d70254957e3afca | 6 - ...sort_1-62-c4ec0433a832ef551d70254957e3afca | 6 - ...sort_1-63-dbcec232623048c7748b708123e18bf0 | 1 - ...sort_1-64-7f98b724df05f51b3ec1f087a8da414e | 1 - ...sort_1-65-83c59d378571a6e487aa20217bd87817 | 1 - ...sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 - ...sort_1-69-ed76c0068780120a6f23feefee303403 | 1 - ...sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 | 5 - ...sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd | 6 - ...sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 | 1 - ...sort_1-77-6d6eac558569563cc47a657daffa8ba7 | 2 - ...sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...sort_10-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...ort_10-10-b4e225bc4787d7718bae6f00d8addfe2 | 2 +- ...sort_10-2-fc30020d09151dc29be807795ad9475e | 2 +- ...sort_10-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 | 2 +- ...ort_11-11-d9bf7e80b71121935ed4b008ae916cb1 | 6 - ...ort_11-13-c7d70702783eb90d4f53028a63c318f8 | 6 - ...ort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 | 1 - ...ort_11-20-bd99462ed878bf4bec74b3cb9132908d | 1 - ...sort_11-7-bd99462ed878bf4bec74b3cb9132908d | 1 - ...sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 | 1 - ..._sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_2-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 | 2 +- ..._sort_2-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_2-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_2-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_3-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_3-10-4bf8dba6e66e733423a3408d61897926 | 2 +- ...sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f | 2 +- ..._sort_3-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_3-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_3-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_4-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd | 2 +- ...sort_4-14-75d59344b6114c0bb20d5eac301c2170 | 2 +- ..._sort_4-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_4-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_4-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_5-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_5-10-4bf8dba6e66e733423a3408d61897926 | 2 +- ...ort_5-13-c0f14def6a135cc50cba364e810ce28e} | 0 ...sort_5-17-4bf8dba6e66e733423a3408d61897926 | 2 +- ..._sort_5-2-fc30020d09151dc29be807795ad9475e | 2 +- ...ort_5-20-c0f14def6a135cc50cba364e810ce28e} | 0 ...sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f | 2 +- ..._sort_5-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_5-5-c0f14def6a135cc50cba364e810ce28e} | 0 ..._sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_6-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd | 2 +- ..._sort_6-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_6-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_6-9-591e03d1cfc10821a601498df1ed6675} | 0 ..._sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_7-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_7-10-4bf8dba6e66e733423a3408d61897926 | 2 +- ..._sort_7-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_7-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc} | 0 ..._sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_8-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...sort_8-11-bd99462ed878bf4bec74b3cb9132908d | 2 +- ..._sort_8-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_8-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc} | 0 ..._sort_8-8-bd99462ed878bf4bec74b3cb9132908d | 2 +- ..._sort_8-9-8e06b51e940e956f14a8c7679c3d423a | 2 +- ..._sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._sort_9-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._sort_9-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._sort_9-3-fffea659b633b1f269b38556a7f54634 | 2 +- ...sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc} | 0 ...y_sort_9-9-feec69facdc973a0ff78455f766845c | 2 +- ..._skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 1 - ..._skew_1-1-ffe97dc8c1df3195982e38263fbe8717 | 1 - ...skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-15-bbdd53118f788d7bb679d094c41243c8 | 6 - ...skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ..._skew_1-2-fc30020d09151dc29be807795ad9475e | 1 - ...skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-25-d53196339980a00a619788bd799a32e7 | 5 - ...skew_1-29-c4ec0433a832ef551d70254957e3afca | 6 - ..._skew_1-3-fffea659b633b1f269b38556a7f54634 | 1 - ...skew_1-32-d53196339980a00a619788bd799a32e7 | 5 - ...skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ..._skew_1-4-c67a488530dc7e20a9e7acf02c14380f | 1 - ...skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd | 10 - ...skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd | 5 - ...skew_1-53-c4ec0433a832ef551d70254957e3afca | 6 - ...skew_1-57-5373c5449884d95bc7db9dab55239a49 | 6 - ...skew_1-60-c4ec0433a832ef551d70254957e3afca | 6 - ...skew_1-63-c4ec0433a832ef551d70254957e3afca | 6 - ...skew_1-64-dbcec232623048c7748b708123e18bf0 | 1 - ...skew_1-65-7f98b724df05f51b3ec1f087a8da414e | 1 - ...skew_1-66-83c59d378571a6e487aa20217bd87817 | 1 - ...skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c | 1 - ...skew_1-70-ed76c0068780120a6f23feefee303403 | 1 - ...skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 | 5 - ...skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd | 6 - ...skew_1-75-6f7caef1c773268350c9cf94ad85be01 | 0 ...skew_1-76-4931c5a72a5231f67317d27ca025bb97 | 0 ...skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 | 1 - ...skew_1-78-6d6eac558569563cc47a657daffa8ba7 | 2 - ..._skew_1-8-c0ea81b686236d661166912040a16ea7 | 0 ...t_skew_1-9-f0ee61903aeacb758e2eada242e5e14 | 0 ..._test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ..._test_1-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._test_1-2-fc30020d09151dc29be807795ad9475e | 2 +- ..._test_1-3-fffea659b633b1f269b38556a7f54634 | 2 +- ..._test_1-4-8e06b51e940e956f14a8c7679c3d423a | 2 +- ..._test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...test_1-6-c0f14def6a135cc50cba364e810ce28e} | 0 .../hash-0-a658b129316d666d4b01c1581eed1c1f | 2 +- ...text_cs-0-e319c8574a6cd8739e5fd5984ceed3cf | 0 ...text_cs-1-de3aa1c4674fb166b825b1e2f58d1950 | 0 ...reation-0-f880114c33c99a5f23c1465fd88f0db3 | 0 ...reation-1-a8bc76559014d9cdf07184208d582d25 | 0 ...eation-10-4bd6c86cd3f6a94964b3d13235e8e261 | 5 - ...eation-11-b5b7e4f7af5186033be12a4393dc3bb7 | 0 ...eation-12-9cc02e06c6051810c50e225bb2c66669 | 5 - ...eation-13-9c0ec67e79a0d50b46bd5c944c710fc4 | 0 ...eation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 | 5 - ...reation-15-7d7e1a479e7dcd8f9d4199775e05bc1 | 0 ...eation-16-ffa6d3fcef97b7322dd3759d4a70881d | 5 - ...eation-17-18ed0b70b0b6d076b96c9f73bfa721ad | 0 ...eation-18-bcdb19db031341c4a50264ccf49328e4 | 5 - ...eation-19-98dbf83283f9e073e88ba770ec5a707f | 0 ...creation-2-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...eation-20-68c5c98581c683b17ceaf1c0fd192871 | 5 - ...eation-21-4c8f6b48c437bf0be109fc0be1dc840e | 0 ...reation-22-f36cb2eed39691ca949b25182e2dd31 | 5 - ...eation-23-e7f21f556d3aa2bedb0717a167720741 | 0 ...eation-24-8cb102bd5addf7fece0e2691468bc3bf | 5 - ...eation-25-33c48966230b934ae8ddf74ff18bb9ca | 0 ...eation-26-f85db55b3f63ae186a1b6d5cec545939 | 0 ...eation-27-e4856f13692e63d61f72aaf75e50e5f1 | 0 ...eation-28-bd20d4b59e6489082a92fcbfcc5f8dbe | 0 ...eation-29-ee8d287111069805c41c9c0032adc46f | 0 ...reation-3-14b999fc6dfb10a3632afe14e08003e1 | 0 ...eation-30-f880114c33c99a5f23c1465fd88f0db3 | 0 ...eation-31-a8bc76559014d9cdf07184208d582d25 | 0 ...reation-32-9d5d11cb38f2e097f16d2db5693f4f1 | 0 ...eation-33-14b999fc6dfb10a3632afe14e08003e1 | 0 ...eation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...eation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...eation-36-21bcf37075b02097f16c8fc8130a83b8 | 0 ...eation-37-9334418431eca405f13206bd8db42a1b | 0 ...eation-38-f1f56119aede4f42221a68f6aaa42a26 | 0 ...eation-39-489b4ceb2f4301a7132628303f99240d | 1 - ...reation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 | 0 ...reation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 | 0 ...reation-6-21bcf37075b02097f16c8fc8130a83b8 | 0 ...reation-7-9334418431eca405f13206bd8db42a1b | 0 ...reation-8-f1f56119aede4f42221a68f6aaa42a26 | 0 ...reation-9-bf40d4d50d050f2f8342c07f5a9dcf0c | 0 ...x_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 | 0 ..._serde-0-db64b724719d27c7f0db4f51f5c4edaa} | 0 ..._serde-1-6560d12b69d55e5297a145ebc4bb0cb3} | 0 ...x_serde-1-f92d6c66d21791c11d2a822df04c1b63 | 3 - ..._serde-10-123301a057d4a46072d0431e00e20c4b | 5 - ...serde-10-c85e061ea9c5b90ca69b7450faad14b6} | 0 ..._serde-11-123301a057d4a46072d0431e00e20c4b | 5 + ..._serde-11-309e916d683a1a12ab62565697cb0046 | 0 ...serde-12-309e916d683a1a12ab62565697cb0046} | 0 ...x_serde-12-d590fd7cb9d433143de490d75686dd4 | 0 ..._serde-13-d590fd7cb9d433143de490d75686dd4} | 0 ...x_serde-2-f92d6c66d21791c11d2a822df04c1b63 | 3 + ...ex_serde-2-fd1b220f4eafb0ba9b519a156e3c87c | 0 ...x_serde-3-afcf2a156ccd4f79a0489b4593908d79 | 0 ..._serde-3-ebab588c84a7a29f03b41dcd98132229} | 0 ..._serde-4-afcf2a156ccd4f79a0489b4593908d79} | 0 ...x_serde-4-d7547751c37375a9238043bbe250e716 | 5 - ...x_serde-5-d7547751c37375a9238043bbe250e716 | 5 + ...x_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 | 0 ...x_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae | 1 - ..._serde-6-e6ff4b23b7f102e359afb4d53a1dedc3} | 0 ...x_serde-7-3b03210f94ec40db9ab02620645014d1 | 1 - ..._serde-7-c9d7dcde469d3b9a66965a64dd15e4ae} | 0 ...x_serde-8-35f48c7d6fa164bb84643657bc9280a8 | 1 - ..._serde-8-3b03210f94ec40db9ab02620645014d1} | 0 ..._serde-9-35f48c7d6fa164bb84643657bc9280a8} | 0 ...x_serde-9-c85e061ea9c5b90ca69b7450faad14b6 | 0 ...st_type-0-e231c5154b18cbc0baa082a7461dd13e | 0 ...st_type-1-c836a98522549d2a3fd43998afd8ae94 | 0 ...st_type-2-d8590c7336ae771b7a685bb544e8d2bd | 0 ...nerjoin-0-43d53504df013e6b35f81811138a167a | 2 +- ...nerjoin-4-35c7611180562dcd9dab834f41654095 | 2 +- ...tdriver-1-b2f337566a5075f3e3e81335008d95d3 | 4 +- .../input-1-6558e385bb08991302a72076dd7b7ff5 | 2 +- .../input0-1-efefd4364cd2790447fb0f908e87501f | 2 +- .../input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 | 17 - .../input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 | 4 +- ...input10-1-6970b6d2d451612b59fccbfd7ec68f74 | 17 - ...input10-2-73f00da5cfc254745d1d80f913eb6449 | 12 +- ...input11-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...1_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 | 2 +- ...input12-0-9b141c1e5917ca82c6bc36a9a2950a1e | 2 +- .../input12-1-2b9ccaa793eae0e73bf76335d3d6880 | 2 +- ...nput12-10-4d9eb316259a8e7ed6627bc27a639f7c | 2 +- ...input12-2-bab89dfffa77258e34a595e0e79986e3 | 2 +- ...input12-8-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...input12-9-3d08dc27c1a133c2497fc554c0d169bd | 2 +- ...input14-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...4_limit-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...4_limit-1-be9934fc5e6ecb9854eb7531a5929dcf | 0 ...4_limit-2-780cdc89e0e736790124b6bdac827951 | 0 ...4_limit-3-adc1ec67836b26b60d8547c4996bfd8f | 5 - ...input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 | 4 +- ...ut16_cc-1-5180e975a6babd51752706f1799e7df5 | 0 ...input17-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...input17-1-be9cde5e769f171f60f61a7739de8f17 | 0 ...input17-2-21166e268096f6ec67f4f57ec333e901 | 0 ...input17-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - .../input19-1-f2832e249ab28bb3fb8e472012c5ffc | 0 ...nput19-1-f32df514de8156b5f5b435eea2c9be40} | 0 ...input19-2-5a804e02e4419e785d15e7f39d6c5730 | 2 +- ...1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 | 2 +- ...1_limit-5-eaaf713833e28a803c798562c7d6cd23 | 2 +- .../input2-1-e0efeda558cd0194f4764a5735147b16 | 4 +- .../input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd | 4 +- .../input2-4-235f92683416fab031e6e7490487b15b | 6 +- .../input2-5-9c36cac1372650b703400c60dd29042c | 2 +- .../input2-7-9c36cac1372650b703400c60dd29042c | 2 +- .../input2-9-48bf8c06ed0a264d0863613fe79795e1 | 17 - ...input21-1-70b803742328eacc69eb1ed044a5c6b8 | 0 ...nput21-1-c45ad493e95150b580be778da6065f36} | 0 ...input21-3-9809b74435cbaedef0dc6e6b88b180fe | 2 +- ...nput22-1-b663ec84da3f9d9b9594ea2da81b1442} | 0 ...input22-1-b7f46eccd104e6ed1b29e2de45089f66 | 0 ...input22-3-8285c1934441b12f6c016e13cb410e79 | 2 +- ...input24-3-3189f3b2990de94619b9cb583d9dd3c5 | 2 +- ...input26-1-8272225744e83ba4cbe158a5c113fce3 | 2 +- ...2_limit-1-fed7e0bb996623da7dd17793e835f785 | 2 +- .../input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d | 4 +- ...input3-10-10a1a8a97f6417c3da16829f7e519475 | 8 +- ...input3-11-9c36cac1372650b703400c60dd29042c | 2 +- ...input3-12-a22d09de72e5067a0a94113cdecdaa95 | 14 - ...input3-14-efee6816e20fe61595a4a2a991071219 | 6 +- .../input3-3-1c5990b1aed2be48311810dae3019994 | 6 +- .../input3-4-9c36cac1372650b703400c60dd29042c | 2 +- .../input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 | 14 - .../input3-7-1c5990b1aed2be48311810dae3019994 | 8 +- .../input3-8-4dc0fefca4d158fd2ab40551ae9e35be | 14 - ...input30-0-582c5fcbe2fe12cc8e7b21225583d96c | 0 ...input30-1-90c0d1a75de78c405413fd627caea4ab | 0 .../input30-2-823920925ca9c8a2ca9016f52c0f4ee | 1 - ...input30-3-c21dba410fb07a098f93430a9d21df79 | 1 - ...input30-4-f0ebd08e7675b19ae831824ef4d9e223 | 0 ...input30-5-38734677c27b5f90a8df5da6e6351c76 | 0 .../input30-6-f120ac8c87db9eebb5da7ac99b48600 | 1 - ...input30-7-95d10d459c088d5fbefd00bdd8d44c3f | 1 - .../input31-2-705764f8f7cab9378964af30b83f7fe | 0 ...input31-3-50c905261882f7fd8539fdd91e68151f | 0 ...input31-4-2f886fa357df9342733551fa1b53f913 | 0 ...input32-3-b0070890240c15d647af59f41b77ba3d | 0 ...input32-4-d0e1378a30e21e0198e47d9b668ee1f6 | 0 ...input32-5-8789d32fc5b784fe2d171566732c573e | 0 ...input37-0-86e2e274650fb56651607ea10d356fc0 | 0 ...input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 | 0 ...input39-0-7bd12162381231be9d578797818957a7 | 0 ...input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 ...nput39-10-28bf1b34c04a048da339eddd4c1fd779 | 0 ...nput39-11-6d0814cbb72eb96bfc75c95d06f1e528 | 1 - ...input39-12-f120ac8c87db9eebb5da7ac99b48600 | 1 - ...nput39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 | 1 - ...nput39-14-bcc1d5fd287f81bac1092a913b09956d | 1 - ...input39-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 ...input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 ...input39-4-53453776bf062f28d371fc7336b7eae2 | 0 .../input39-5-823920925ca9c8a2ca9016f52c0f4ee | 1 - ...input39-6-763ab5853bff619e6525c01e46b2a923 | 1 - ...input39-7-9b141c1e5917ca82c6bc36a9a2950a1e | 1 - .../input39-8-2b9ccaa793eae0e73bf76335d3d6880 | 1 - ...input39-9-bab89dfffa77258e34a595e0e79986e3 | 1 - ...adoop20-0-4c33233bafd910d69c2b8845d35abffe | 0 ...adoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 | 0 ...doop20-10-6d0814cbb72eb96bfc75c95d06f1e528 | 1 - ...adoop20-11-f120ac8c87db9eebb5da7ac99b48600 | 1 - ...doop20-12-a6a77ae749a7e7f8022d71c03322fc21 | 1 - ...adoop20-2-ee667c6ab003bb83f6bf1c72153eba39 | 0 ...adoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e | 0 ...adoop20-4-53453776bf062f28d371fc7336b7eae2 | 0 ...hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee | 1 - ...adoop20-6-763ab5853bff619e6525c01e46b2a923 | 1 - ...adoop20-7-db1cd54a4cb36de2087605f32e41824f | 1 - ...adoop20-8-bab89dfffa77258e34a595e0e79986e3 | 1 - ...adoop20-9-28bf1b34c04a048da339eddd4c1fd779 | 0 ...3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 ...3_limit-1-7b46b8baf9c7628da9c190c96b917057 | 0 ...3_limit-2-c6583bdb759c8a050238a32a6ce8273d | 0 ...3_limit-3-2a87d8faa18a6311376812bd0453fece | 0 ...3_limit-4-70dad45d534146923fce88b2ffb99b0d | 0 ...3_limit-5-3664b564747487df13a5d109837219b5 | 0 ...3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 | 20 - ...input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e} | 0 .../input4-1-c139adc70f9942e527142e3be7fd2b87 | 27 - ...input4-2-b663ec84da3f9d9b9594ea2da81b1442} | 0 .../input4-2-b7f46eccd104e6ed1b29e2de45089f66 | 0 .../input4-4-271b04e0fbee2ee81bae21dcb46d55e4 | 2 +- ...nput40-1-a8adb8ae1d13607851431a1baf7578ba} | 0 ...input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 | 0 ...input40-2-e7ab3c9244fcfda76061b4153d796e93 | 2 +- ...input40-4-f241eafbb8d5da3f9c1737aed7b4f94e | 0 ...nput40-4-fdeea6b676c670b17c8d91e24a97a127} | 0 ...input40-5-ccdc9c87a8fdbeee0fa48927f9700361 | 2 +- ...nput40-6-6651f53efc5d03ed2d43b9d7aecc0002} | 0 ...input40-6-93a0c37189dfe2318ba6ad84616f0d64 | 0 ...input40-7-ccdc9c87a8fdbeee0fa48927f9700361 | 2 +- ...input41-0-763ab5853bff619e6525c01e46b2a923 | 2 +- ...input41-3-526399455dc5ecd4ea9f676b09fafeee | 2 +- ...input43-0-2baba8070f3585debc14b6bb3c83607a | 0 ...input46-0-b0cdbecce0321ac452c8e13e1bfc6924 | 0 ...input46-1-1efdd5ebfa732abdedeb10467ca71f7f | 0 ...input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca | 0 ...input46-3-c185163787977498a4b84f39f983c431 | 0 ...input46-4-7f05cde078d90c25780a7d5491b20c27 | 0 ...input46-5-f5c502e88a3dc3edb37b04af7d7955ab | 0 ..._delim-1-353d2238b781a117888a67bb7b2b2537} | 0 ...b_delim-1-5692d0e91dd0114729b8eb3aee388b72 | 0 ...b_delim-2-e52787bf798a941c854eb09d75efe617 | 2 +- ...4_limit-0-4f0124854141b8be1defa7a6d0877d8d | 0 ...t4_limit-1-c634fc723fb3aac3ce007069bdcb2af | 10 - .../input5-0-659e06570690cceeb3f37e10e855d2ea | 0 .../input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 | 0 .../input5-2-3d6eb15b4fe23d0a1aa303da818d97ad | 0 .../input5-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - .../input6-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../input7-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../input8-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...arserde-0-df919fd41f281bf7b45a2340d0c9d43e | 0 ...narserde-1-2db9e6115d209fabcb0c06e5e666fa3 | 0 ...arserde-2-ac60752a883d3204c215fa01811701be | 0 ...arserde-3-3455e6f385b0f60be5d0c842ade0f1d7 | 11 - ...arserde-4-c471b057cdb4f3c3405b311ea2b92aa3 | 11 - ...icserde-0-92c95af00fd419aa106571f72fcad67d | 0 ...icserde-1-39bc8a2aea379563a62a465cc54aecbc | 0 ...icserde-2-7bee1cc403a04b53d0a0324819e1d768 | 0 ...icserde-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - ...icserde-4-8bf42e02f82b0ac58e7d0b525a993b31 | 11 - ...zyserde-0-92c95af00fd419aa106571f72fcad67d | 0 ...zyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 | 0 ...yserde-10-d915efbc5233a49f48e81e74eef2a3c8 | 0 ...yserde-11-6aaa88142e86a9a39e980caed145e32c | 0 ...yserde-12-abde20a4a37ed330bc7128576dd18d7c | 11 - ...zyserde-2-7a165139976654640c86db8e4e5871cc | 0 ...zyserde-3-bdf93641b97ab6347ece67e2fb636e97 | 11 - ...zyserde-4-8cc058fb7986f59965976cad813267db | 11 - ...zyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ...azyserde-6-42e2838ee69484bf5301475905cee12 | 0 ...zyserde-7-b44a44827ad0dce044badd6b258eabd5 | 0 ...zyserde-8-abde20a4a37ed330bc7128576dd18d7c | 11 - ...zyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ...t_limit-1-77d7012bc901b0a9bcf9dae500ae2034 | 2 +- ...t_part0-1-1aa1486a207bedc275035acc3b37cbdb | 2 +- ...t_part1-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._part10-3-48b242bc305c9bf879e083fa11edc967 | 14 +- ..._part10-4-d0ba28297a8b73569d93605aa890aa09 | 2 +- ...t10_win-3-48b242bc305c9bf879e083fa11edc967 | 14 +- ...t10_win-4-d0ba28297a8b73569d93605aa890aa09 | 2 +- ...t_part2-4-93c97e1760e0d41b3791d6f08010a665 | 2 +- ...t_part2-5-9d0992a91951e6e4242b5b4c38d4e861 | 2 +- ...t_part3-1-ba5256285fb22a43b491253a2d519730 | 2 +- ...t_part5-3-e4419c33287ca1f48a43f61cca5b5928 | 2 +- ...t_part7-1-affad4cedcd29bb136bc477fc07e6ea0 | 2 +- ...t_part8-1-60b409a520999ba50e8b7c6e30de3474 | 2 +- ...t_part9-1-e60c60afc073367464898b8396e8f643 | 2 +- ...ncefile-0-68975193b30cb34102b380e647d8d5f4 | 2 +- ...ncefile-1-1c0f3be2d837dee49312e0a80440447e | 2 +- ...ncefile-5-3708198aac609695b22e19e89306034c | 2 +- ...ncefile-6-6716fa5aec125f0f8e9520e4318f71b9 | 2 +- ...stxpath-0-3c8a098a179d578119f75e5d7b214bd5 | 0 ...stxpath-1-ac18d29e8bd8aeed00296992edc17c0f | 0 ...estxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 | 0 ...stxpath-3-adc1ec67836b26b60d8547c4996bfd8f | 11 - ...txpath2-0-a1baa22f15f745a2dfe27ce52d363704 | 0 ...txpath2-1-27b77465d23aebf66a50c8074a75b755 | 0 ...txpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd | 0 ...txpath2-3-adc1ec67836b26b60d8547c4996bfd8f | 10 - ...txpath3-0-f05134d54292acd1f2067027889a4fac | 0 ...txpath3-1-807b097ac2f785f774db03069ebbde11 | 11 - ...txpath4-0-73819ea1a7c0653a61652b3766afb003 | 1 - ...txpath4-1-f746888141a38ba707fad01d86d41960 | 0 ...txpath4-2-e4beab1294dcac60ff36e4f2561810b9 | 10 - ...txpath4-3-ae225e86c2ae20519ffdf23190454161 | 1 - ...txpath4-4-f746888141a38ba707fad01d86d41960 | 0 ...txpath4-5-e4beab1294dcac60ff36e4f2561810b9 | 10 - ...putddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 | 12 +- ...putddl3-2-73f945a673d2e388847c317f683f160c | 4 +- ...putddl4-1-dd94576788fa50ebcf950cdf837fbcf6 | 20 +- ...putddl4-2-7fdf00ff5c22ae284728e0f035396865 | 22 +- ...utddl6-1-572c1abb70f09726d1ba77bdc884597b} | 0 ...putddl6-1-8b96b4fba4cf54c399a008d0f3be1edf | 0 ...utddl6-2-3a4def4b370f75c5fcc1174626490363} | 0 ...putddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c | 0 ...putddl6-3-e81b962986706e1c16f059b407e3f05c | 10 +- ...putddl6-4-5855e2998e26f63e927854afa86c1f03 | 10 +- ...putddl6-5-47e6a8d33932cb014830812e1f7b1f94 | 2 +- ...putddl6-7-47e6a8d33932cb014830812e1f7b1f94 | 2 +- ...putddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 | 19 - ...utddl7-1-7195712efb4910294f63303ebce24453} | 0 ...putddl7-1-7b46b8baf9c7628da9c190c96b917057 | 0 ...utddl7-10-4eec8cef401b6bba00457dfbec58cc2d | 0 ...tddl7-10-7c9248b56948716913d332bd712d69bd} | 0 ...utddl7-11-6e1f1709d770ed76aee6ff5f76856e63 | 2 +- ...putddl7-12-2c56d4a781242b0521f82bb0d2cd277 | 4 +- ...utddl7-13-45059a21c202b4658285738ee62a018a | 4 +- ...utddl7-14-30c87bc734c2afa4fea0facdf7279145 | 8 +- ...putddl7-15-fed9badc255db68c3ed1cd1446d9fbe | 8 +- ...putddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 | 2 +- ...utddl7-4-68715ba2c11220be62394c86453e6d54} | 0 ...putddl7-4-7513658e8abe9d9b72c7219321c56fa9 | 0 ...putddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 | 2 +- ...putddl7-7-48640fff8428a0dc6e90a7243adaf730 | 0 ...utddl7-7-59dd2d2556769e19bdc0a444f40f8a71} | 0 ...putddl7-8-495dc87b0bde752c890f213ff9531508 | 2 +- ...putddl8-1-c70f2d2544633366b76b92bcff18e995 | 10 +- ...itions-3-86653b3af59df59f225ee00ff5fc119f} | 0 ...titions-3-b7aaedd7d624af4e48637ff1acabe485 | 0 ...titions-4-dece2650bf0615e566cd6c84181ce026 | 0 ...itions-4-e81d45a5bec5642ec4b762f1c1a482af} | 0 ...pressed-0-ea607fbed28d20e5726f4501285d698d | 2 +- ...pressed-4-5133d2457097962811a2adf0ecd9e4ef | 2 +- ...pressed-6-5133d2457097962811a2adf0ecd9e4ef | 2 +- ...pressed-8-5133d2457097962811a2adf0ecd9e4ef | 2 +- ...t_into1-0-ae5ea07929262bde22fbe7ebe80d4992 | 0 ...t_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b | 0 ..._into1-10-c260979323c1ebdf68c6fbe003d43792 | 0 ...t_into1-11-41015d6409c5ebf670eed4999157fdb | 1 - ..._into1-12-ae5ea07929262bde22fbe7ebe80d4992 | 0 ...t_into1-2-ff6a1b25c911def274921df1bae476b7 | 0 ...t_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 | 0 ...rt_into1-4-41015d6409c5ebf670eed4999157fdb | 1 - ...t_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 | 0 ...t_into1-6-4b2e32fe57b2297d22bccb4656bdee30 | 0 ...rt_into1-7-41015d6409c5ebf670eed4999157fdb | 1 - ...t_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d | 1 - ...t_into1-9-31eb4770dc60e4765065ac7f84811d1b | 0 ...t_into2-0-957c6402df0fd7d2fccbc688e49e9661 | 0 ...t_into2-1-9828bb831fd11667b48678e5952a0941 | 0 ..._into2-10-df53336f364fe09e9591e769c13b5519 | 0 ..._into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa | 0 ..._into2-12-4cf03cb3982a457f2f72220265ecc844 | 1 - ..._into2-13-957c6402df0fd7d2fccbc688e49e9661 | 0 ...t_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 | 0 ...t_into2-3-9d5556040de01fd02d5501d141effff2 | 0 ...t_into2-4-9d5556040de01fd02d5501d141effff2 | 0 ...t_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 | 1 - ...t_into2-6-4cf03cb3982a457f2f72220265ecc844 | 1 - ...t_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 | 0 ...t_into2-8-452111285dda40205ee587de8e972896 | 0 ...t_into2-9-4cf03cb3982a457f2f72220265ecc844 | 1 - ...t_into3-0-7e35c30348a53b0748bc1bb85f75c195 | 0 ...t_into3-1-f6dd7262d45a21781d85bc343ede8fb5 | 0 ..._into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 | 1 - ..._into3-11-9cfd2d054f84262eb74a870b6365db87 | 1 - ..._into3-12-7e35c30348a53b0748bc1bb85f75c195 | 0 ..._into3-13-f6dd7262d45a21781d85bc343ede8fb5 | 0 ...t_into3-2-e51c25bae2408422a56826a263479468 | 0 ...t_into3-3-c46699c465fefe6baab35499a32b452d | 0 ...rt_into3-4-e9f4f47686fe97482b0a769a15481dd | 0 ...t_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 | 0 ...t_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 | 1 - ...t_into3-7-9cfd2d054f84262eb74a870b6365db87 | 1 - ...t_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 | 0 ...t_into3-9-19d1be183f7985f7066f86572abc82c8 | 0 ...t_into4-0-b4831621b2a02fc4e8e655b03c289310 | 1 - ...t_into4-1-3d466d45197fcf1eff55d76ef0a29720 | 0 ..._into4-10-a8016ca816f7b362cf3a0384ca91b159 | 1 - ..._into4-11-bdbfaf06bc4e323863db65fd29882eaa | 0 ..._into4-12-49b12993ebc1647b9dca8b9af19deca9 | 0 ..._into4-13-a5bb41af609f38f01d8c71334cc8d71b | 1 - ..._into4-14-3d466d45197fcf1eff55d76ef0a29720 | 0 ..._into4-15-f6ad1a49459fb6cd232fccc4a6062b25 | 0 ...t_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 | 0 ...t_into4-3-43629aaa698fb6e2db4586124561e69b | 0 ...t_into4-4-8adba808fd505f4bf0ffcc61a618480a | 0 ...t_into4-5-6bc47894aa917051abb98d0b52f43881 | 0 ...t_into4-6-8c1683bee2927da76bb0dbf44a373738 | 0 ...t_into4-7-a8016ca816f7b362cf3a0384ca91b159 | 1 - ...t_into4-8-6bc47894aa917051abb98d0b52f43881 | 0 ...t_into4-9-8c1683bee2927da76bb0dbf44a373738 | 0 ...t_into5-0-9afa473f2111cf0d9ae62041bd97f840 | 0 ...t_into5-1-8fc8db6a5564324339192f23208ffc1c | 0 ...t_into5-10-375cae396c768c1efe5d17b9f5f45f8 | 0 ..._into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d | 0 ..._into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 | 1 - ..._into5-13-e06a0b7252278141d50466e08f15b391 | 0 ..._into5-14-a3a4c16b0b723f97f654b4b5e80575c2 | 0 ..._into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 | 1 - ..._into5-16-9afa473f2111cf0d9ae62041bd97f840 | 0 ...t_into5-2-548a3a10c15c333c63fc1c239ee8b62c | 0 ...t_into5-3-a4b25f172af356ec98035329b95ddbd3 | 0 ...t_into5-4-3d5343a79ee8b680f3b74b22db6658e6 | 0 ...t_into5-5-f382c5b31afe9e0251fa9457c46c12a5 | 0 ...t_into5-6-a058ba199b9777d48b6c6595f2388533 | 1 - ...t_into5-7-2004b4ecf6ceddb033727f8d5ebab42f | 0 ...t_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 | 0 ...t_into5-9-a058ba199b9777d48b6c6595f2388533 | 1 - ...t_into6-0-16367c381d4b189b3640c92511244bfe | 1 - ...t_into6-1-a4fb8359a2179ec70777aad6366071b7 | 1 - ..._into6-10-16500f4af2c8638a670e867e59f6d457 | 0 ..._into6-11-95297f4dcc4dd6a51de4785ccd25fbba | 0 ..._into6-12-29afaab2cf10eaa65afaa6c0fcca0902 | 1 - ..._into6-13-b357a845fb5f668b09dea94c6d2e7d66 | 2 - ..._into6-14-1f00d9cc34225cb358a54c6d4008cb47 | 0 ..._into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 ...t_into6-2-1f00d9cc34225cb358a54c6d4008cb47 | 0 ...t_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d | 0 ...t_into6-4-5f12794e99c74355a23d2fda9c7c170f | 0 ...t_into6-5-de641eb41a9100e755a9ae641c752b30 | 0 ...t_into6-6-f6e7141a435922193937aa10085b0656 | 0 ...t_into6-7-6826c610b8d04ab3464712bd9cddbcd3 | 0 ...t_into6-8-33ec9514947e0b737e957bdcbbd87573 | 0 ...t_into6-9-3ccb7bc735e406ad1723c758f01bcaab | 1 - ...ternal1-0-eb0745518b859c8497506a627bfd9860 | 0 ...ternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a | 1 - ...ternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b | 1 - ...ternal1-3-f64289fb03ab105e12659fc3972ca241 | 1 - .../join0-2-52055f2dce8eac4e5249d02c42c0da87 | 2 +- .../join1-0-43d53504df013e6b35f81811138a167a | 2 +- .../join1-4-35c7611180562dcd9dab834f41654095 | 2 +- .../join10-1-73136f8e6e9ba82f75570afd15c2828d | 2 +- .../join11-1-b580f87daf1763cd8c5e59ad5b271232 | 2 +- .../join12-1-496c8aabaf3261e8fefd7b357f2ac7f | 2 +- .../join13-1-696b36d15c4358145f77c2b15b7507d5 | 2 +- .../join14-1-9b141c1e5917ca82c6bc36a9a2950a1e | 2 +- .../join14-2-2b9ccaa793eae0e73bf76335d3d6880 | 2 +- .../join14-3-bab89dfffa77258e34a595e0e79986e3 | 2 +- .../join14-4-27f1a57fbb815d169af86ae2f8305cb6 | 2 +- .../join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 2 +- ...adoop20-1-db1cd54a4cb36de2087605f32e41824f | 2 +- ...adoop20-2-bab89dfffa77258e34a595e0e79986e3 | 2 +- ...adoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 | 2 +- .../join15-1-81d76d3bf59889b07b413b6f88772667 | 2 +- .../join17-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join18-1-49f8ba8d43a6bb1d892ba66a812304f5 | 2 +- .../join2-3-cac2c9e0f8601dd56822c990774e0696 | 2 +- .../join20-1-9685f2d31ffe922d3ea217de32ca3049 | 2 +- .../join20-3-3331a020843caccf2fa32a1225c5c3a9 | 2 +- .../join21-1-3536b7d78713e86ee67f5f6c9b88958f | 2 +- .../join23-1-91b8e7fe75a7e3ba8147c56734436681 | 2 +- .../join24-2-d79325ef6494aa87843fdfd78de7c812 | 2 +- .../join25-0-8934d9b821aa4b34b760f73eff56cd06 | 2 +- .../join25-4-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join26-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join27-3-e86808fdbd54120d1e2356f8f61c02f9 | 2 +- .../join28-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join28-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join28-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join28-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join29-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join29-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join29-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join29-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join3-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join30-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join31-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join31-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join31-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join31-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join32-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join32-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join32-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join32-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ...ssSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 2 +- ...ssSize-13-ed70124968560328930327ecb108c4e9 | 2 +- ...ssSize-16-ed70124968560328930327ecb108c4e9 | 2 +- ...ssSize-19-ed70124968560328930327ecb108c4e9 | 2 +- ...essSize-2-24ca942f094b14b92086305cc125e833 | 2 +- ...ssSize-22-ed70124968560328930327ecb108c4e9 | 2 +- ...essSize-3-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...essSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 | 2 +- ...essSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a | 2 +- .../join33-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join33-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join33-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join33-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join34-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join34-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join34-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join34-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join35-1-24ca942f094b14b92086305cc125e833 | 2 +- .../join35-2-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join35-3-25fc734982956a164adde6bb1d4d8751 | 2 +- .../join35-6-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join36-0-8934d9b821aa4b34b760f73eff56cd06 | 2 +- .../join36-8-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join37-0-8934d9b821aa4b34b760f73eff56cd06 | 2 +- .../join37-4-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join38-2-eacdb3417c4a563982c488812d654c9 | 2 +- .../join38-4-53d219706847e890de1dcd369563ebef | 2 +- .../join39-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- .../join4-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join40-0-d2b5e23edec42a62e61750b110ecbaac | 2 +- ...join40-10-1d1f79e8e773d630f355c1a89d84b5aa | 2 +- ...join40-12-aaddbef9069aba3ebeb310be74671cda | 2 +- .../join40-2-507b1d9f6abbdb756a589d7bc4826251 | 2 +- .../join40-4-61fb097bda1751910de966d6a4a8f0b7 | 2 +- .../join40-6-9685f2d31ffe922d3ea217de32ca3049 | 2 +- .../join40-8-3331a020843caccf2fa32a1225c5c3a9 | 2 +- .../join41-1-25e434b6d05e08fdd5f4d9957438917 | 2 +- .../join41-3-1dc98363e7da167dc45711a87ef3a988 | 2 +- .../join41-4-44d382ce6848d3f0b900b0808747d8e9 | 2 +- .../join41-6-1dc98363e7da167dc45711a87ef3a988 | 2 +- .../join5-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join6-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join7-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join8-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../join9-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...in_1to1-1-5bab379018a4fbef12cc93658f26580a | 0 ...n_1to1-1-789b2636cfb6a08965e0bd190e419762} | 0 ...n_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e | 2 +- ...n_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-12-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-13-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-15-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 | 2 +- ...n_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-18-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-19-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-21-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 | 2 +- ...n_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-24-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-25-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-27-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e | 2 +- ...n_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-3-6228e662e573a00ed04550d049d97a3b} | 0 ...in_1to1-3-ee6db4188755bf471a12316ec7301500 | 0 ...n_1to1-30-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-31-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-33-5e48ba086f1376939535081b60f82727 | 2 +- ...n_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 | 2 +- ...n_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...n_1to1-36-6081eb976b4aef2132418510756a385b | 2 +- ...n_1to1-37-281b888188eac90c4bf670417f25cc0c | 2 +- ...n_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...n_1to1-39-5e48ba086f1376939535081b60f82727 | 2 +- ...in_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 | 2 +- ...in_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e | 2 +- ...in_1to1-6-6081eb976b4aef2132418510756a385b | 2 +- ...in_1to1-7-281b888188eac90c4bf670417f25cc0c | 2 +- ...in_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 | 2 +- ...in_1to1-9-5e48ba086f1376939535081b60f82727 | 2 +- ..._array-2-16840a0266cad03a1a0b134d105b854f} | 0 ...n_array-2-a4363f7c7e4b7d717ed90e77c37581de | 0 ..._array-3-a6ca6b64324596831033fdfe5b63a942} | 0 ...n_array-3-ddd65703cdad8959cd0cd831304b0ab9 | 0 ...n_array-4-b235265cd6bd58fd743c27b02e547d62 | 2 +- ...n_array-5-a9b9419b94631f8fe1c2297ebf103a9a | 2 +- ...n_array-6-6eded94bd39189ea6d67fe383f9b865c | 2 +- ...nsitive-1-404d691e85c7b74bad73576ee80de290 | 0 ...sitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf} | 0 ...nsitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc | 0 ...nsitive-3-93300f4a9242fa2804a5b368538d83f} | 0 ...nsitive-4-c880b2256f97413b8fe68d19d99747fd | 2 +- ...n_empty-4-df2401785dfa257de49c3ad80b0f480a | 2 +- ...n_empty-5-ce1ef910fff98f174931cc641f7cef3a | 2 +- ...filters-1-100faa7fd01bfb4390c782bb262a9403 | 0 ...ilters-1-bee6095f42de6a16708c2f9addc1b9bd} | 0 ...ilters-10-f0c5c51de4151a17a77c780be0c13e01 | 2 +- ...lters-100-a4c7cd87175222bea19fd33018890efe | 2 +- ...lters-101-f086409eb336282af5a00f4c7192ef2b | 2 +- ...lters-102-634888c4fa78ce10181c9514c6393554 | 2 +- ...lters-103-c020d3461658ae8e118281f40264ae5b | 2 +- ...lters-104-c9b79f30e1f25672ec89014f966b41b0 | 2 +- ...ilters-105-b3d9dcbb7e1caef97646eb89edf82eb | 2 +- ...lters-106-870999978978b22f21997899f1e652b8 | 2 +- ...lters-107-94824a62d882227f270a66ec7ef28cd4 | 2 +- ...lters-108-d793c1514545499f58fb1b355cbd3f0e | 2 +- ...lters-109-2709001b4aa57ed01ba975e83b556475 | 2 +- ...ilters-11-6fb35e1a65211e68de5df4a1cb9c7117 | 2 +- ...lters-110-86868ef0f28c5b42f04fb9ca64aaa3ba | 2 +- ...lters-111-fd3188d025e4c84d45cbb265d08ca569 | 2 +- ...lters-112-a0cd07949ff7dbc2287dc060bedb1942 | 2 +- ...ilters-113-48c4978768872751832149d72cdf0ba | 2 +- ...lters-114-58b8db46ea5080791c7416838d3e8f95 | 2 +- ...lters-115-98e45a3167d19d09219076a2d93afa49 | 2 +- ...ilters-116-bec2950fff2c1b3c4dc5be846653ea6 | 2 +- ...lters-117-ee8471d93c3778f4b48c5c59f473dd35 | 2 +- ...lters-118-9b395d7db8722a467d46588d0f27fc9e | 2 +- ...lters-119-7688784396db55ff148292a9755c918a | 2 +- ...ilters-12-4c29d6be8717481332cd1ee7ca17690e | 2 +- ...lters-120-6578651ffbf95c0b02825e2125e32709 | 2 +- ...lters-121-6dc4977da7f8dc7c636a03d7ad892ada | 2 +- ...lters-122-64bb9f9a0eef32d2392b80b93c9b2c98 | 2 +- ...filters-13-c6a291879bdb37f0c84f6074f257d52 | 2 +- ...ilters-14-ef8255dcad808f9128d79e6ee9f368cf | 2 +- ...ilters-15-a83678913c62249c8fdf2dac1f6e3046 | 2 +- ...ilters-16-a6511579b5e4c2d845664f0c4a6ce622 | 2 +- ...ilters-17-c22aab929016fa6f4b6d1e14cc3a4940 | 2 +- ...ilters-18-8ef51af7455eaeeaefff1c98e16dcc65 | 2 +- ...ilters-19-e164192e6538b428c8b53e008a978d3f | 2 +- ...filters-2-67fff9e6931a7320444e857e01b3d496 | 2 +- ...ilters-20-7a5da20822bf51ed69ccf640cbb816cf | 2 +- ...ilters-21-13d6d5335625fc3386a8011dc557002e | 2 +- ...ilters-22-f12ffd6cc9213d9c64cbea1b6db0632e | 2 +- ...ilters-23-a800b885358695758afdb719cdefa94f | 2 +- ...ilters-24-df3db5a91f3f4e88e18d2b1cc5b47113 | 2 +- ...ilters-25-435ecfc7f9cb5f32019fe498c21ccad3 | 2 +- ...ilters-26-da36fab3d1686aedd415a7a0f0253eca | 2 +- ...ilters-27-5f4a5437696f2a60bec9ac1443244242 | 2 +- ...ilters-28-2acf41a9f6efac0d800df557db716359 | 2 +- ...ilters-29-cf5fef4ddc313209f5ab1e5ea3763e35 | 2 +- ...filters-3-989b3dbd15ba601ae80fe454e03213d7 | 2 +- ...ilters-30-97f94f35ebc30f776a19bed59c7fb4bf | 2 +- ...ilters-31-4923f8ba429f0c71ad9b6b9499e73a7f | 2 +- ...ilters-32-5978cd7936c296493a16a31b926043ab | 2 +- ...ilters-33-607d64d50ef9aad424bd22b358efe027 | 2 +- ...ilters-34-35c2c8d74bc0ebc4710c81333cb254a9 | 2 +- ...ilters-35-a4c7cd87175222bea19fd33018890efe | 2 +- ...ilters-36-f086409eb336282af5a00f4c7192ef2b | 2 +- ...ilters-37-634888c4fa78ce10181c9514c6393554 | 2 +- ...ilters-38-c020d3461658ae8e118281f40264ae5b | 2 +- ...ilters-39-c9b79f30e1f25672ec89014f966b41b0 | 2 +- ..._filters-4-33bfcd576019d7e32683556f66e3757 | 2 +- ...filters-40-b3d9dcbb7e1caef97646eb89edf82eb | 2 +- ...ilters-41-870999978978b22f21997899f1e652b8 | 2 +- ...ilters-42-94824a62d882227f270a66ec7ef28cd4 | 2 +- ...ilters-43-d793c1514545499f58fb1b355cbd3f0e | 2 +- ...lters-46-268d8fb3cb9b04eb269fe7ec40a24dfe} | 0 ...ilters-46-aa161b0d9fe9d1aad10654fce0e3670b | 0 ...ilters-47-3c52df82c7d78501610f3f898103f753 | 0 ...lters-47-6dc6866a65c74d69538b776b41b06c16} | 0 ...ilters-48-1d85bb008e02ef4025171a4bc0866a6c | 0 ...lters-48-e884480a0f7273d3e2f2de2ba46b855c} | 0 ...lters-49-98fd86aea9cacaa82d43c7468109dd33} | 0 ..._filters-49-e79c906b894fed049ddfab4496a4e3 | 0 ...filters-5-f0c0d07019afb1bbe162e3183e18023e | 2 +- ...ilters-50-3e6612a89e9124592e790594775054b1 | 2 +- ...ilters-51-60a5f56f33fc8854a2b687005f0d96ac | 2 +- ...ilters-52-64cabe5164130a94f387288f37b62d71 | 2 +- ...ilters-53-2709001b4aa57ed01ba975e83b556475 | 2 +- ...ilters-54-86868ef0f28c5b42f04fb9ca64aaa3ba | 2 +- ...ilters-55-fd3188d025e4c84d45cbb265d08ca569 | 2 +- ...ilters-56-a0cd07949ff7dbc2287dc060bedb1942 | 2 +- ...filters-57-48c4978768872751832149d72cdf0ba | 2 +- ...ilters-58-58b8db46ea5080791c7416838d3e8f95 | 2 +- ...ilters-59-98e45a3167d19d09219076a2d93afa49 | 2 +- ...filters-6-c0c40d001cac0bc91095dddda1513ad9 | 2 +- ...filters-60-bec2950fff2c1b3c4dc5be846653ea6 | 2 +- ...ilters-61-ee8471d93c3778f4b48c5c59f473dd35 | 2 +- ...ilters-62-9b395d7db8722a467d46588d0f27fc9e | 2 +- ...ilters-63-7688784396db55ff148292a9755c918a | 2 +- ...ilters-64-6578651ffbf95c0b02825e2125e32709 | 2 +- ...ilters-65-6dc4977da7f8dc7c636a03d7ad892ada | 2 +- ...ilters-66-64bb9f9a0eef32d2392b80b93c9b2c98 | 2 +- ...ilters-67-67fff9e6931a7320444e857e01b3d496 | 2 +- ...ilters-68-989b3dbd15ba601ae80fe454e03213d7 | 2 +- ...filters-69-33bfcd576019d7e32683556f66e3757 | 2 +- ...filters-7-89963646509154a2fb1ddbbf1f55349d | 2 +- ...ilters-70-f0c0d07019afb1bbe162e3183e18023e | 2 +- ...ilters-71-c0c40d001cac0bc91095dddda1513ad9 | 2 +- ...ilters-72-89963646509154a2fb1ddbbf1f55349d | 2 +- ...ilters-73-69e0235472d7cee7d83037cd083544a5 | 2 +- ...ilters-74-b6372cc006844e8488a3b7836c67daaa | 2 +- ...ilters-75-f0c5c51de4151a17a77c780be0c13e01 | 2 +- ...ilters-76-6fb35e1a65211e68de5df4a1cb9c7117 | 2 +- ...ilters-77-4c29d6be8717481332cd1ee7ca17690e | 2 +- ...filters-78-c6a291879bdb37f0c84f6074f257d52 | 2 +- ...ilters-79-ef8255dcad808f9128d79e6ee9f368cf | 2 +- ...filters-8-69e0235472d7cee7d83037cd083544a5 | 2 +- ...ilters-80-a83678913c62249c8fdf2dac1f6e3046 | 2 +- ...ilters-81-a6511579b5e4c2d845664f0c4a6ce622 | 2 +- ...ilters-82-c22aab929016fa6f4b6d1e14cc3a4940 | 2 +- ...ilters-83-8ef51af7455eaeeaefff1c98e16dcc65 | 2 +- ...ilters-84-e164192e6538b428c8b53e008a978d3f | 2 +- ...ilters-85-7a5da20822bf51ed69ccf640cbb816cf | 2 +- ...ilters-86-13d6d5335625fc3386a8011dc557002e | 2 +- ...ilters-87-f12ffd6cc9213d9c64cbea1b6db0632e | 2 +- ...ilters-88-a800b885358695758afdb719cdefa94f | 2 +- ...ilters-89-df3db5a91f3f4e88e18d2b1cc5b47113 | 2 +- ...filters-9-b6372cc006844e8488a3b7836c67daaa | 2 +- ...ilters-90-435ecfc7f9cb5f32019fe498c21ccad3 | 2 +- ...ilters-91-da36fab3d1686aedd415a7a0f0253eca | 2 +- ...ilters-92-5f4a5437696f2a60bec9ac1443244242 | 2 +- ...ilters-93-2acf41a9f6efac0d800df557db716359 | 2 +- ...ilters-94-cf5fef4ddc313209f5ab1e5ea3763e35 | 2 +- ...ilters-95-97f94f35ebc30f776a19bed59c7fb4bf | 2 +- ...ilters-96-4923f8ba429f0c71ad9b6b9499e73a7f | 2 +- ...ilters-97-5978cd7936c296493a16a31b926043ab | 2 +- ...ilters-98-607d64d50ef9aad424bd22b358efe027 | 2 +- ...ilters-99-35c2c8d74bc0ebc4710c81333cb254a9 | 2 +- ...overlap-0-990e447b6447ced0d9684eb7db9e63ce | 0 ...overlap-1-a7336cd2d643f92715b42f9c6eb2c108 | 0 ...ive_626-3-2609f2809e0c6ae1ca93806e37960990 | 0 ...ve_626-3-4a2f2f2858540afea9a195b5322941ee} | 0 ...ive_626-4-387e3bc517b49d4e1c9752c07b72b790 | 0 ...ve_626-4-4bb73b33747da4ed852df381b7b45a71} | 0 ...ive_626-5-a0eb25c15b6ca1a04da14781b1213e11 | 0 ...ve_626-5-c6a3ae6f3539ab48b996060fb51d8ebe} | 0 ...ive_626-7-d9c817eebc533bedc3ef9172d325a2c2 | 2 +- ...ap_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ...map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa | 2 +- ..._nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f} | 0 ...n_nulls-1-97641998eb9ddf2dff56de6758486aa0 | 0 ..._nulls-10-39071d8e6b246cfd405714dbf0b5337b | 2 +- ..._nulls-11-545dff878ea0d79cdfee517572c9e0c8 | 2 +- ..._nulls-12-5e1ca8627aa685435142d8e339e77062 | 2 +- ..._nulls-13-5f670a20cbec42a34685ee0398ad7e2d | 2 +- ...n_nulls-14-e97ba69145da387a4a66635b8499077 | 2 +- ..._nulls-15-e9f9be8be6e936fb832483dfa0be5126 | 2 +- ..._nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 | 2 +- ...in_nulls-17-2b0bfe5754456475ceb6af4694165f | 2 +- ..._nulls-18-321cf9d31dac835c3def6ca3b3b860a2 | 2 +- ..._nulls-19-739bf8e440e698540d18c29226c3564c | 2 +- ...n_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 | 2 +- ..._nulls-20-fff960f1cb832bc93e3d1de519e573d5 | 2 +- ..._nulls-21-62a25fb819ae5c1ea757b6e759082a2e | 2 +- ..._nulls-22-5b2df5518994ae86c041484561857da0 | 2 +- ..._nulls-23-982c3e55235cafa3d89b5dee4366fdf8 | 2 +- ..._nulls-24-86ad66543a735d396f7336cb5bdfa495 | 2 +- ..._nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 | 2 +- ..._nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 | 2 +- ..._nulls-27-6ee7affed896b1c539628ab081842b83 | 2 +- ..._nulls-28-455aace3472c5840a885b6fab6a046cb | 2 +- ..._nulls-29-2c278a88713aef5cd30ff3720ef3eeeb | 2 +- ...n_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 | 2 +- ..._nulls-30-2c0c41da38714d1b16feffb00fa08bb1 | 2 +- ..._nulls-31-a33c48d38817ee3a7aca511dc7793486 | 2 +- ..._nulls-32-e6b104ae96622ff75d510efc6efc9352 | 2 +- ..._nulls-33-1284a11bf6aeef8ff87b471d41985f26 | 2 +- ..._nulls-34-aeb90811861431cadc5512637793afc1 | 2 +- ..._nulls-35-2d1d18d4e9775ec69333947fbd62bc82 | 2 +- ..._nulls-36-7c029c91141b36af79ba0dc1de73a257 | 2 +- ..._nulls-37-fa84731f5a6beec20d64a7981815b9bc | 2 +- ...n_nulls-4-a1f20b4863428627bae1316755cc2d94 | 2 +- ...nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe} | 0 ..._nulls-40-aa161b0d9fe9d1aad10654fce0e3670b | 0 ..._nulls-41-3c52df82c7d78501610f3f898103f753 | 0 ...nulls-41-6dc6866a65c74d69538b776b41b06c16} | 0 ..._nulls-42-1d85bb008e02ef4025171a4bc0866a6c | 0 ...nulls-42-e884480a0f7273d3e2f2de2ba46b855c} | 0 ...nulls-43-98fd86aea9cacaa82d43c7468109dd33} | 0 ...in_nulls-43-e79c906b894fed049ddfab4496a4e3 | 0 ..._nulls-44-2db30531137611e06fdba478ca7a8412 | 2 +- ..._nulls-45-e58b2754e8d9c56a473557a549d0d2b9 | 2 +- ..._nulls-46-64cabe5164130a94f387288f37b62d71 | 2 +- ..._nulls-47-ebf794e8b51be738e2d664f249869de1 | 2 +- ...n_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 | 2 +- ..._nulls-49-2d20d8f4221281a6b1cd579e761f81b7 | 2 +- ...n_nulls-5-5ec6e3df7181e0738244623cc01bf22c | 2 +- ...n_nulls-50-8b26343e74c161cf74ece5f0bd21470 | 2 +- ..._nulls-51-75339d2eb2afabf5dd088074b2563d8f | 2 +- ...n_nulls-52-caad1db99085760daaf8f96c0ce5564 | 2 +- ..._nulls-53-ec965e66e7fad403cd0ea388c3e70363 | 2 +- ...in_nulls-54-6f15c794b5587430ebb685cc61c502 | 2 +- ..._nulls-55-a1c73326f8c8d9beccda3ba365352564 | 2 +- ..._nulls-56-f7b9629093c818b6c384c79d1458d178 | 2 +- ..._nulls-57-cf353446d7f358a508f17d0984b90158 | 2 +- ..._nulls-58-5f9a59160a76f9b649102a97987ed33a | 2 +- ..._nulls-59-8753a39412ac59c7a05951aeeea73b24 | 2 +- ...n_nulls-6-7eea211c80e7f1146098e80ffb890d67 | 2 +- ...n_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 | 2 +- ...n_nulls-8-609f6bd812a44b20da0a39c827e4d870 | 2 +- ...n_nulls-9-ef4b27a877efc377cd5750af6725194b | 2 +- ...llsafe-2-5bb63fafa390b1d4c20e225a8a648dcf} | 0 ...nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 | 0 ...lsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0} | 0 ...llsafe-20-d6fc260320c577eec9a5db0d4135d224 | 0 ...lsafe-21-24332b9390108fb3379e1acc599293a1} | 0 ...llsafe-21-a60dae725ffc543f805242611d99de4e | 0 ...llsafe-22-24c80d0f9e3d72c48d947770fa184985 | 0 ...lsafe-22-4be80634a6bd916e3ebd60a124f0a48e} | 0 ...llsafe-23-3fe6ae20cab3417759dcc654a3a26746 | 0 ...lsafe-23-e4425d56be43c21124d95160653ce0ac} | 0 ...join_rc-5-1aef75afe38d512addb44dbf9a650263 | 2 +- ...order2-11-f8460b061fa00f6afbfe8eeccf4d3564 | 2 +- ...eorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...order2-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...eorder2-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...order2-5-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...eorder2-6-3fda17e4414d191f837631438a19e700 | 0 ...order2-6-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...eorder2-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...order2-7-5f4cfbbe53c5e808ee08b26514272034} | 0 ...eorder2-9-26ffed826eceda953b7124ee39ace828 | 2 +- ...order3-11-d6392b851f7dd5e3705e8ff51e1c6583 | 2 +- ...eorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...order3-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...eorder3-5-4abc4f450a58ccdd0df2e345f1276979 | 0 ...order3-5-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...eorder3-6-3fda17e4414d191f837631438a19e700 | 0 ...order3-6-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...eorder3-7-512b75ccb9459a6334da1d9699f4a5ec | 0 ...order3-7-5f4cfbbe53c5e808ee08b26514272034} | 0 ...eorder3-9-9d45e66a3990b7c53fd780f07cd52e13 | 2 +- ...order4-10-1d6b948747ac31296710a491a5652e3f | 2 +- ...eorder4-12-203aed2c4298eba4a3c51816a21a1c1 | 2 +- ...eorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...order4-3-c0f14def6a135cc50cba364e810ce28e} | 0 ...eorder4-4-4abc4f450a58ccdd0df2e345f1276979 | 0 ...order4-4-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...eorder4-5-3fda17e4414d191f837631438a19e700 | 0 ...order4-5-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...eorder4-6-24ca942f094b14b92086305cc125e833 | 2 +- ...reorder4-8-4bbb8e937245e2fa7cafbb423814754 | 2 +- ...n_star-10-57ce75f989b3b3bfd2f2eceb228e892e | 0 ..._star-10-a9e579038e3d4826fdae475d7058ab82} | 0 ..._star-11-72730ecdad9c0fd4c6ce64a0cb89fb74} | 0 ...n_star-11-eba1397e66f25cba4fd264209cc92bae | 0 ...n_star-12-89b53ae954ec88171ef87e0459f6eb82 | 0 ..._star-12-f581d6d305d652cd0f4e4fa912eb578d} | 0 ...n_star-13-342b7249c9ce1484869169b1b33191cb | 0 ..._star-13-7268564732cbb7489248f9d818f80c14} | 0 ..._star-14-2ee0fcf000f8687fc8941bf212477e57} | 0 ...n_star-14-75513308d30b781fd2e06d81963c4363 | 0 ...n_star-15-24a0b1d0257bad4f85b3a122acf6bef9 | 0 ..._star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9} | 0 ...n_star-16-24ca942f094b14b92086305cc125e833 | 2 +- ...n_star-17-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...n_star-18-1c22e451845667bd6b4eac3c49c36965 | 2 +- ...n_star-20-76473cb68a69b9408207fd43ddae9339 | 2 +- ...n_star-22-429cfd140488242d569aee6594aae76b | 2 +- ...n_star-24-e11ab68d72d874c7c6c658c1018f5a49 | 2 +- ...n_star-26-a412a0620a359a62e2ab4e45fa8e0330 | 2 +- ...n_star-28-98a3f4d8a2209e771c57462d4b723ff9 | 2 +- ...in_star-8-a957982d8981ff0a35397ca449297024 | 0 ...n_star-8-c3d53a4daab9614a09870dc8e9571f74} | 0 ...n_star-9-3f7ccccc2488de5f33a38cb3cc3eb628} | 0 ...in_star-9-904e30d8615eb411fb24c2cc08df94f4 | 0 ..._thrift-0-811b62ecbacdb26c67fa260ff3747a41 | 6 - ..._thrift-1-4f47dd107d2be1212411bda214c0d1db | 0 ..._thrift-2-d4bfb7bab262dc2898431680711dec1b | 11 - ...join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 | 0 ...in_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 ...in_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 ...in_view-2-eda40dba9678df62dde73fc5dafb2b44 | 0 ...in_view-3-eccb00d8dada8ab56a48c373e381e02b | 0 ...in_view-5-85baeea910adc4589bc3ec2ce0b1e856 | 0 ...in_view-6-c37b7962ab7371f94a9c54d61f7638ef | 0 ...in_view-7-a14cfe3eff322066e61023ec06c7735d | 0 ...in_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 | 0 ...in_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 | 0 ...noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 | 0 ...noalias-1-6d5806dd1d2511911a5de1e205523f42 | 2 - ...noalias-2-155b3cc2f5054725a9c2acca3c38c00a | 0 ...noalias-3-3b7045ace234af8e5e86d8ac23ccee56 | 2 - ..._noalias-4-e1eca4e08216897d090259d4fd1e3fe | 0 ...noalias-5-16d227442dd775615c6ecfceedc6c612 | 0 ...w_outer-0-b66c363516d1f717765df9b91da3b5b4 | 0 ...w_outer-1-8d4332785ff69bb86607700c133d0baa | 10 - ...w_outer-2-b4474ec12d042fca7a21a2cd551c1068 | 0 ...w_outer-3-57b2ffd7b60708551238c491a2a8685d | 10 - ...w_outer-4-96fe3dc4f8116e535c322437a59b5e4e | 0 ...w_outer-5-2ec3aeb923621c64da042402709e64e9 | 0 ...w_outer-6-511e4df505342e04c20e50fda8962120 | 10 - ...ew_ppd-10-a537ad7282d1c9957cdae74ad87c790b | 6 + ...ew_ppd-9-dc6fea663d875b082d38bd326d21cd95} | 0 ...emijoin-5-9c307c0559d735960ce77efa95b2b17b | 0 ...mijoin-5-aba449db0d4fe6dc9771426e102bb543} | 0 ...emijoin-6-82921fc96eef547ec0f71027ee88298c | 0 ...mijoin-6-9f50dce576b019c0be997055b8876621} | 0 ...emijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 | 0 ...mijoin-7-fff6ca40e6048d52dc2d3afc68e8353e} | 0 ...oin_mr-1-5b2e555868faa404ea09928936178181} | 0 ...join_mr-1-aa3f07f028027ffd13ab5535dc821593 | 0 ...join_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 | 0 ...oin_mr-3-c019cb2a855138da0d0b1e5c67cd6354} | 0 ...gth.udf-0-e85ebb8ce5d939964fd87bd13b326c02 | 2 +- ...ushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb | 1 - ...ushdown-1-d0a93f40892e3894460553b443c77428 | 1 - ...shdown-10-19842c7708a0787e59cc5e6b8c3a4250 | 0 ...shdown-11-b435672262fc3c02d1ed1a93ff8d856f | 20 - ...shdown-12-a3516c212d6c79986536edbd9c961098 | 0 ...shdown-13-93906f7e87e5e3dc4b9590ec13b6af78 | 20 - ...shdown-14-cfae77212d164efb18208f71332fd368 | 0 ...shdown-15-62a90d3ba54cc68055b3502c3864a3c1 | 0 ...ushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 | 0 ...shdown-17-11c349c0db0f869be88351256650fe60 | 20 - ...shdown-18-66ecdd019c936dec1ab5c9389c9071b3 | 0 ...shdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 - ...ushdown-2-4d1e292b00635298240ff909be64dce4 | 0 ...shdown-20-d4de935f7a059cce44889c6ba69cfddb | 0 ...shdown-21-f04dee0f94443ca01320657897cbb914 | 20 - ...shdown-22-4d378725e22e7e48c861983ba935bf5e | 1 - ...shdown-23-53b4dd4899cb2ba681a801a3b5ca155a | 0 ...shdown-24-cb58113b9dfd3e93d4623cba6a090976 | 100 - ...shdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a | 0 ...shdown-26-e691b284f1b830d7b83a36750105737c | 100 - ...ushdown-3-cc674af3ce71c06670e366932df43462 | 20 - ...ushdown-4-81bbb7300da27bc63f7a139677faac3f | 0 ...ushdown-5-c0664bc5bff121823ac2f37df13d6bd7 | 20 - ...pushdown-6-b722630e977a5ca509234fc417d7b30 | 0 ...ushdown-7-e8c8087a3910ea53a725b01b9d05b921 | 20 - ...ushdown-8-6194a19c1d5065731ec26ea1d5a390e1 | 0 ...ushdown-9-1446f634128a1c3e7cd224ea82452d0a | 20 - ...egative-7-fb7bf3783d4fb43673a202c4111d9092 | 0 ..._double-0-10ef1098e35d900983be3814de8f974f | 0 ...double-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...double-1-10ef1098e35d900983be3814de8f974f} | 0 ..._double-1-3863c17e03c9c1cd68452106a8721d13 | 1 - ..._double-2-3863c17e03c9c1cd68452106a8721d13 | 1 + ...l_ints-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...al_ints-0-5ffd1b49cdda4149aef2c61c53a56890 | 0 ...l_ints-1-5ffd1b49cdda4149aef2c61c53a56890} | 0 ...al_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 | 1 - ...al_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 | 1 + ...string-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...l_string-0-9b48d41e5c9e41ddc070e2fd31ace15 | 0 ..._string-1-2cf4b7268b47246afdf6c792acca379d | 1 - ..._string-1-9b48d41e5c9e41ddc070e2fd31ace15} | 0 ..._string-2-2cf4b7268b47246afdf6c792acca379d | 1 + ...ry_data-0-491edd0c42ceb79e799ba50555bc8c15 | 0 ...ry_data-1-5d72f8449b69df3c08e3f444f09428bc | 0 ...ary_data-2-242b1655c7e7325ee9f26552ea8fc25 | 0 ...ry_data-3-2a72df8d3e398d0963ef91162ce7d268 | 0 ...ry_data-4-3ebc340f7f63740f8534706d42dd37ca | Bin 114 -> 0 bytes ..._part1-10-c66fea7ad025cd1f513a98a4cc1036b1 | 1000 +++++++ ..._part1-10-fca9513ea05bfb8b7e0e6f337d184d66 | 1000 ------- ..._part1-11-8b6be78ae0527e5b7efd7db758966853 | 1000 +++++++ ..._part1-11-9c82167763a771c175c656786d545798 | 1000 ------- ...n_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 | 14 +- ...n_part10-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part10-2-151ba0c3b8317902f1235ac07d58135e | 14 +- ..._part10-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part10-4-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part10-5-16367c381d4b189b3640c92511244bfe | 2 +- ..._part10-8-245027204484e281e1cfaf74386d2967 | 2 +- ..._part10-9-a646fd106fe73b8753fe3fee86d56ebf | 1000 +++++++ ..._part10-9-eb29a5b2392bf821b2ff51b70863d531 | 1000 ------- ...n_part11-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part11-2-4301f87a8dbf9576788637386e26f9a2 | 14 +- ..._part11-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part11-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ..._part11-5-ea607fbed28d20e5726f4501285d698d | 2 +- ..._part11-6-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part11-8-9a4433518ac9ff49cb4b71812705adbb | 2 +- ..._part11-9-3889a0cba1cf3c8f8c2e67957e69406a | 2 +- ...n_part12-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b | 14 +- ..._part12-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part12-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ..._part12-5-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 | 2 +- ...n_part12-8-fd656b581b8f8fbb8ac22f444dbc345 | 2 +- ...n_part13-0-463330cf55370dbe92d6ed74ef91302 | 2 +- ..._part13-2-d52536b8ac62f6e8152e394fe135a3e0 | 14 +- ..._part13-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ..._part13-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ..._part13-5-a4fb8359a2179ec70777aad6366071b7 | 2 +- ...n_part13-8-930d8e7a94f0cdf922322dae4a02e16 | 2 +- ..._part13-9-c8de411bc094b37b59a2eb0baf6de55d | 2 +- ..._part14-1-253e2a15bfaef9aa781dc29fa324b51e | 8 +- ..._part14-2-a4fb8359a2179ec70777aad6366071b7 | 2 +- ..._part14-3-16367c381d4b189b3640c92511244bfe | 2 +- ..._part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 2 +- ..._part14-7-7c931249465f330d51ef0610f214429e | 2 +- ...t14_win-1-253e2a15bfaef9aa781dc29fa324b51e | 8 +- ...t14_win-2-a4fb8359a2179ec70777aad6366071b7 | 2 +- ...t14_win-3-16367c381d4b189b3640c92511244bfe | 2 +- ...t14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 | 2 +- ...t14_win-7-7c931249465f330d51ef0610f214429e | 2 +- ..._part15-2-1379abc2de057dc6d240a526f0dd8a3c | 0 ..._part15-3-9940fad8d8590e60bd726e9503ae4fa9 | 0 ..._part15-4-fa01199bab3e6aa47a82c9aec88aa76a | 0 ..._part15-5-9940fad8d8590e60bd726e9503ae4fa9 | 3 - ...n_part2-1-845923af04bb331c9f5995a3a3e84424 | 14 +- ...n_part2-2-dc129f70e75cd575ce8c0de288884523 | 2 +- ...n_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...n_part2-4-a4fb8359a2179ec70777aad6366071b7 | 2 +- ...n_part2-7-86ffa99b03fa88235b61bf1af7062c33 | 2 +- ...n_part2-8-a1ff8a12d94378e7e1165bd78cf724cf | 2 +- ...n_part2-9-3f29de9877006f9448272ef2422d6132 | 2 +- ...n_part3-2-dbbba335c008a61a13c1472b34470397 | 14 +- ...n_part3-8-c32770da4784bfea4d0dd77fdcba4f0a | 2000 +++++++++++++ ...n_part3-8-c3b6f0cc860b84851dd60c53b36a6437 | 2000 ------------- ..._part4-10-297cf42ec45b5aa78b80d9c35a79555a | 1500 ++++++++++ ..._part4-10-7f0c1e54518d4d1d345577a100e670e8 | 1500 ---------- ..._part4-11-1313644c9dad948bfcebd7386c309ab7 | 2500 ----------------- ..._part4-11-24618a43c4656b72f81683b45708045e | 2500 +++++++++++++++++ ...n_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd | 14 +- ...n_part5-1-9a4d1f6a14227bb66bd01557a464da8b | 8 +- ...n_part6-2-779aa345cf2875089312ec26b046415d | 14 +- ...n_part7-2-5d4c3c48f53d55e26ca142ee70d1706a | 14 +- ...n_part8-2-9e1df41acabef83f62464f52c2396c8a | 14 +- ...n_part8-9-704bd110d9aaa2ac678b7fbf645abdb9 | 2000 +++++++++++++ ...n_part8-9-a7456fceb35f62a282db750384f480db | 2000 ------------- ...n_part9-2-748ac33315295f8e55f2480f7714c27a | 14 +- ...yn_part9-9-6954f5fc8dd82ca2c076ab8bcdbc148 | 1000 +++++++ ...n_part9-9-72b1ad2231269b704130903b35ac04bb | 1000 ------- ...success-0-84028c4ca541d126baffc20d6d876810 | 0 ...success-1-c84cd1b5b491bded8ac3b0521de599c5 | 0 ...success-3-9ee887603dcba5200918ae5200afa5d5 | 0 ...he_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f | 0 ...e_name-1-d19201e2fcaee4d451292bd740e6c637} | 0 ...e_name-2-ad8795e50f5998ea1d2eb64a0c02e6e5} | 0 ...oad_fs2-0-517732da2c84ae17095b0e1d96f74d97 | 0 ...oad_fs2-1-5018c84e09be70bf663594a89f3ad731 | 0 ...oad_fs2-2-94d2317b453b3b49bb9f2b58040b4748 | 0 ...he_name-0-f99b4f29506d65c841fb1db18263fbcc | 1 - ...he_name-1-b64a19f7101a4fb3b5d08b2f6e296400 | 1 - ...he_name-2-2087e00fe000e00f64e819dca59be450 | 0 ...success-0-fa705a031ff5d97558f29c2b5b9de282 | 0 ...success-2-9ee887603dcba5200918ae5200afa5d5 | 0 ...success-0-84028c4ca541d126baffc20d6d876810 | 0 ...success-2-9ee887603dcba5200918ae5200afa5d5 | 0 ...adpart1-1-4bf1504274319c44d370b58092fe016c | 0 ...dpart1-1-6cc94d19c536a996592629f7c82c2ac9} | 0 ...adpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 | 2 +- ...adpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 | 2 +- ...art_err-2-21fe8ff9059167209647e7ea086f483e | 0 .../lock1-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../lock1-1-3e95421993ab28d18245ec2340f580a3 | 0 .../lock1-2-c0c18ac884677231a41eea8d980d0451 | 0 .../lock2-0-cd46bc635e3010cf1b990a652a584a09 | 0 .../lock2-1-3e95421993ab28d18245ec2340f580a3 | 0 .../lock2-2-c0c18ac884677231a41eea8d980d0451 | 0 .../lock2-3-27ad2962fed131f51ba802596ba37278 | 0 .../lock2-4-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock2-5-e8d1d10c308a73eef78dde414a5e40ca | 0 .../lock3-0-27ad2962fed131f51ba802596ba37278 | 0 .../lock3-1-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock3-2-b1ca816784e88f105b2fce1175340c33 | 0 .../lock3-5-8096935c5c1755f9b88583e8c72921ac | 0 .../lock3-6-598ba296ba1c6931f4161a9f50b00cbe | 0 .../lock4-1-27ad2962fed131f51ba802596ba37278 | 0 .../lock4-2-c06da7f8c1e98dc22e3171018e357f6a | 0 .../lock4-3-b1ca816784e88f105b2fce1175340c33 | 0 .../lock4-6-8096935c5c1755f9b88583e8c72921ac | 0 .../lock4-7-598ba296ba1c6931f4161a9f50b00cbe | 0 ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...oin_ppr-2-3d41e966f69a64babb783d1aad0f1b73 | 2 +- ...oin_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef | 2 +- ...join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 | 2 +- ...join_ppr-8-6fca189c46645f124d5fcb82564b703 | 2 +- ...apjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d | 1 - ...apjoin1-1-abd9364d276ec89352232da5e2237768 | 0 ...pjoin1-10-c08fefa00b89f50dd365208151593505 | 10 - ...pjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 | 0 ...pjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 | 10 - ...apjoin1-2-fe84593f006c85e68fbb797394cdccd0 | 10 - ...apjoin1-3-8439a0592619790b64d16d2506f2233d | 0 ...apjoin1-4-c08fefa00b89f50dd365208151593505 | 10 - ...apjoin1-5-72068bd4cdac40e4d18fd729f39855ba | 0 ...apjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 | 10 - ...apjoin1-7-757dfb540b8a49b3663f8caba7476ec5 | 0 ...apjoin1-8-fe84593f006c85e68fbb797394cdccd0 | 10 - ...apjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 | 0 ...istinct-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...istinct-1-a00d1791b7fa7ac5a0505d95c3d12257 | 2 +- ...stinct-11-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...stinct-12-1d351f7e821fcaf66c6f7503e42fb291 | 2 +- ...stinct-13-a7dc16cb82c595b18d4258a38a304b1e | 2 +- ...stinct-15-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...istinct-3-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...istinct-4-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...istinct-5-a7dc16cb82c595b18d4258a38a304b1e | 2 +- ...istinct-7-4489654b888efc588b13ee1cda1b6a9f | 2 +- ...istinct-8-1d351f7e821fcaf66c6f7503e42fb291 | 2 +- ...istinct-9-a00d1791b7fa7ac5a0505d95c3d12257 | 2 +- ...mapjoin-0-24ca942f094b14b92086305cc125e833 | 2 +- ...mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...mapjoin-2-25fc734982956a164adde6bb1d4d8751 | 2 +- ...apjoin-4-5166a5b9d30dfacbe33dd909c0df6310} | 0 ...mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 | 0 ...mapjoin-5-5ee898cab883074f3297198c52445ee4 | 20 + ...mapjoin-5-c47698bac140454637a999e583941ce7 | 2 - ...mapjoin-6-dca8c08a235b45d1cdcb94e363afb17} | 0 ...mapjoin-7-fddbdea343a9ddb5f8dedc18147640b7 | 2 + ...apjoin-8-2be637ed4f6146e8525ae1a863e72736} | 0 ...mapjoin-9-c47698bac140454637a999e583941ce7 | 2 + ...ubquery-0-24ca942f094b14b92086305cc125e833 | 2 +- ...ubquery-1-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...ubquery-2-25fc734982956a164adde6bb1d4d8751 | 2 +- ...ubquery-4-7df121f9774cb23edc557b98ad1e1924 | 2 +- ...ubquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b | 2 +- ...query2-10-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...query2-11-25fc734982956a164adde6bb1d4d8751 | 2 +- ...query2-13-c876a518451059f17fc15e29f6f57951 | 2 +- ...bquery2-6-5353ee601eb42d5842690d3941683be1 | 0 ...query2-6-9bf06af695892b0d7067d5b30e0b2425} | 0 ...query2-7-c6b0cdb137f13f8362c0c49c544151a4} | 0 ...bquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 | 0 ...query2-8-131ae5ecfff2733b04bdfada0108cf40} | 0 ...bquery2-8-d524906728fef9f559709fe0922ab24e | 0 ...bquery2-9-24ca942f094b14b92086305cc125e833 | 2 +- ...t_outer-0-407016bf2679fb9e9d076a2d115e859d | 2 +- ..._outer-10-ce1ef910fff98f174931cc641f7cef3a | 2 +- ..._outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 | 2 +- ..._outer-12-80993ab7f757001e1f058bf8609f8420 | 2 +- ..._outer-14-7fe52008c4a98853d086d17fc3c21906 | 2 +- ...t_outer-6-7fe52008c4a98853d086d17fc3c21906 | 2 +- ...t_outer-8-dfb08d397d3fe163d75c3b758097b68a | 2 +- ...t_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 | 2 +- ...reduce2-3-adea843673e541da8a735a5a34e7c7dc | 2 +- ...reduce3-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- .../merge1-0-593999fae618b6b38322bc9ae4e0c027 | 2 +- ...merge1-12-2e8e4adbfb21f25e7557dd86363c7138 | 2 +- .../merge1-4-3277fe538b66923cd879b45371838d2b | 2 +- .../merge2-0-b12e5c70d6d29757471b900b6160fa8a | 2 +- .../merge2-1-593999fae618b6b38322bc9ae4e0c027 | 2 +- ...merge2-17-2e8e4adbfb21f25e7557dd86363c7138 | 2 +- .../merge2-2-c95dc367df88c9e5cf77157f29ba2daf | 2 +- .../merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 | 2 +- .../merge2-4-84967075baa3e56fff2a23f8ab9ba076 | 2 +- .../merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea | 2 +- .../merge2-9-b81efaa65e1263e48278ef9062cca1dd | 2 +- ...merge4-10-692a197bd688b48f762e72978f54aa32 | 0 ...merge4-11-ca0336ac3f600cb8b4230d9904686868 | 1500 ---------- ...merge4-12-62541540a18d68a3cb8497a741061d11 | 0 ...merge4-13-ed1103f06609365b40e78d13c654cc71 | 0 ...merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 | 3 - ...merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a | 1 - .../merge4-5-3d24d877366c42030f6d9a596665720d | 0 .../merge4-6-b3a76420183795720ab3a384046e5af | 0 .../merge4-7-631a45828eae3f5f562d992efe4cd56d | 0 .../merge4-8-ca0336ac3f600cb8b4230d9904686868 | 1000 ------- .../merge4-9-ad3dc168c8b6f048717e39ab16b0a319 | 0 ...rtition-0-a4fb8359a2179ec70777aad6366071b7 | 1 - ...rtition-1-16367c381d4b189b3640c92511244bfe | 1 - ...tition-10-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...tition-11-a49c9ee01ce8858a5f00c05523329200 | 1 - ...tition-12-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...tition-13-16adcdb0e324ad233769e124b5b349da | 0 ...tition-14-79da9a1ce5c2d058b924387ac9fcde92 | 500 ---- ...tition-15-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition-16-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...tition-17-b12e5c70d6d29757471b900b6160fa8a | 1 - ...tition-18-593999fae618b6b38322bc9ae4e0c027 | 1 - ...tition-19-a49c9ee01ce8858a5f00c05523329200 | 1 - ...rtition-2-190cefc93e46906e404039de0fd5f513 | 0 ...tition-20-d295db835d4fdeea34298702295ff7c5 | 0 ...tition-21-9b9493a68ef7663e95ad86d02c45ec88 | 0 ...tition-22-79da9a1ce5c2d058b924387ac9fcde92 | 500 ---- ...tition-23-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition-24-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...tition-25-b12e5c70d6d29757471b900b6160fa8a | 1 - ...tition-26-593999fae618b6b38322bc9ae4e0c027 | 1 - ...tition-27-a49c9ee01ce8858a5f00c05523329200 | 1 - ...tition-28-ef7b35be7210f099d46448994d9dc605 | 0 ...tition-29-876c8fa15a32ac5bfcc6cb60993f6a4d | 0 ...rtition-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition-30-79da9a1ce5c2d058b924387ac9fcde92 | 500 ---- ...tition-31-d60297fed03b455c29daa4afb4d1e858 | 14 - ...rtition-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...rtition-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...rtition-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...rtition-7-777de794b7f27ea63f29a9784663393b | 0 ...rtition-8-2f1578dbc029b62daa9d47d8fa473960 | 1 - ...rtition-9-dc129f70e75cd575ce8c0de288884523 | 1 - ...tition2-0-a4fb8359a2179ec70777aad6366071b7 | 1 - ...tition2-1-16367c381d4b189b3640c92511244bfe | 1 - ...ition2-10-43d53504df013e6b35f81811138a167a | 1 - ...ition2-11-b12e5c70d6d29757471b900b6160fa8a | 1 - ...ition2-12-593999fae618b6b38322bc9ae4e0c027 | 1 - ...ition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 | 1 - ...tition2-14-3a4c24fd561f459025264baa3fb6d87 | 1 - ...ition2-15-dbf4eae8430a97a6e70b1c6222218cd3 | 0 ...ition2-16-16adcdb0e324ad233769e124b5b349da | 0 ...ition2-17-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition2-2-190cefc93e46906e404039de0fd5f513 | 0 ...tition2-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition2-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...tition2-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...tition2-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...tition2-7-777de794b7f27ea63f29a9784663393b | 0 ...tition2-8-6f7f59de1fbd607e844a2dc9394a2df8 | 0 ...tition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba | 0 ...tition3-0-a4fb8359a2179ec70777aad6366071b7 | 1 - ...tition3-1-16367c381d4b189b3640c92511244bfe | 1 - ...ition3-10-d176a1b243ac7190fbc319d73a164e2d | 0 ...ition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 | 0 ...ition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a | 0 ...ition3-13-c512eee654e7313ff9c6efb35c5b0a88 | 0 ...ition3-14-ea3e89ffe987e20dffd8388a2dbcc260 | 0 ...ition3-15-e525a096de36a3d157db1b4947e1fbb0 | 0 ...ition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 | 4 - ...ition3-17-43d53504df013e6b35f81811138a167a | 1 - ...ition3-18-b12e5c70d6d29757471b900b6160fa8a | 1 - ...ition3-19-593999fae618b6b38322bc9ae4e0c027 | 1 - ...tition3-2-190cefc93e46906e404039de0fd5f513 | 0 ...ition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 | 1 - ...tition3-21-3a4c24fd561f459025264baa3fb6d87 | 1 - ...ition3-22-bf8877c86d4b4cd7da2939cdf517acc5 | 0 ...ition3-23-3ffba3098571099bc2b13614ae3defc5 | 0 ...ition3-24-c9f91a11d9be1e981e6ec284572bbb2a | 4 - ...ition3-25-d60297fed03b455c29daa4afb4d1e858 | 14 - ...tition3-3-25401dd2c1c258e06f1b96fefd19e27f | 0 ...tition3-4-41df7b4938cff8b7ebffc3f5c701dccf | 0 ...tition3-5-8026cdd2390eec2c72a0591ae5668185 | 0 ...tition3-6-3b57c0e3fccea5322373f3725c95ec00 | 0 ...tition3-7-777de794b7f27ea63f29a9784663393b | 0 ...tition3-8-6916eceaa04091d1453a7d0d5257213c | 0 ...tition3-9-8d0305d089aa5198601cc39073fff528 | 0 ...s_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 | 0 ...s_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 | 0 ..._mixed-10-abb02d2cadc535ff51660d83e6609dc8 | 0 ..._mixed-11-725ed77dd110398f461926f190b4b5c8 | 0 ..._mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b | 0 ...s_mixed-2-de24edb80e51049d241fa0ce2e3165ff | 0 ...s_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 | 0 ...s_mixed-4-24a8048db8d561e28bcb4941498b9687 | 0 ...s_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 | 0 ...s_mixed-6-e108b1560a601946194cecaf4da12491 | 0 ...ns_mixed-7-d2068e6569b5d253932ce9d59be5221 | 0 ...s_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 | 0 ...s_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 | 0 ...taonly1-0-f0c7b56d5ad529ae6c98875501a9548d | 0 ...taonly1-1-4d93504b19d34fd360de6af7fb1f3715 | 0 ...aonly1-10-1c7bb3877b2e1e7bb9166537715d442d | 0 ...aonly1-11-30df79b460916f3f67ccf1d7b7a076f2 | 0 ...taonly1-3-7980a98d580a002b7ad7eef780039f67 | 0 ...taonly1-4-4d93504b19d34fd360de6af7fb1f3715 | 0 ...taonly1-6-537256f669bc9101d4834df67aae8cdf | 0 ...taonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 | 0 .../mi-0-a4fb8359a2179ec70777aad6366071b7 | 1 - .../mi-1-16367c381d4b189b3640c92511244bfe | 1 - .../mi-2-abf8847fb25b96e0f9477808d8378e5e | 0 .../mi-3-b66a495f7bdf106a7886b72267b8659d | 0 .../mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c | 0 .../mi-5-4aad9be12cbe4e7be5540924e6b5e7dc | 1 - .../mi-6-b331d7aa963adac3b2e1de803f90e199 | 1 - .../mi-7-fca4e06de103c3cbb675fa43e7077800 | 500 ---- .../mi-8-e946bdb81b0a831908c1c8196fdff215 | 309 -- ...pJoin1-10-24ca942f094b14b92086305cc125e833 | 2 +- ...pJoin1-12-204073e1287b0582d50e652d466f1e66 | 2 +- ...pJoin1-13-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...pJoin1-14-25fc734982956a164adde6bb1d4d8751 | 2 +- ...pJoin1-16-c14b300770b329ecb71e0275c88532d3 | 2 +- ...pJoin1-18-49bc7f430b2591978067ca8f7d181cee | 2 +- ...apJoin1-22-25e434b6d05e08fdd5f4d9957438917 | 2 +- ...pJoin1-24-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...pJoin1-25-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...pJoin1-26-25fc734982956a164adde6bb1d4d8751 | 2 +- ...pJoin1-28-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...pJoin1-29-ea23403b9eb55e8b06d1c198e439569f | 2 +- ...pJoin1-31-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...pJoin1-32-e93301ee4ba157b466d7460775f3d350 | 2 +- ...pJoin1-34-feed626e3216bcbda66b17f48305b5a1 | 2 +- ...apJoin2-0-24ca942f094b14b92086305cc125e833 | 2 +- ...apJoin2-1-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...pJoin2-10-1905c7759350b107679aef86226739f8 | 2 +- ...pJoin2-11-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...pJoin2-13-6b984427a771fe650fa875be98722cbe | 2 +- ...pJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...pJoin2-16-6b984427a771fe650fa875be98722cbe | 2 +- ...pJoin2-17-b9d963d24994c47c3776dda6f7d3881f | 2 +- ...pJoin2-19-77324702b091d514ca16d029f65d3d56 | 2 +- ...apJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 | 2 +- ...pJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 | 2 +- ...pJoin2-22-77324702b091d514ca16d029f65d3d56 | 2 +- ...pJoin2-27-d28d0f671f5d913a56d75812d24cca8e | 2 +- ...Join2-28-c14792ccac2ca64e3e9e21af4fd12d2c} | 0 ...Join2-29-b9d963d24994c47c3776dda6f7d3881f} | 0 ...Join2-30-6d1c7f7014fc709148b0f401c5f23cb3} | 0 ...pJoin2-31-2e9c4d5e2bd709e96f311488ada116b0 | 1000 +++++++ ...apJoin2-4-5ede8243cc4ba2fbd24a77578502a656 | 2 +- ...apJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 | 2 +- ...apJoin2-7-5ede8243cc4ba2fbd24a77578502a656 | 2 +- ...apJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 | 2 +- ...ert_gby-4-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ert_gby-8-521e0c1054cfa35116c02245874a4e69 | 2 +- ...ert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...rt_gby2-0-b3ee4be40513342084411c5333416d69 | 0 ...rt_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 | 0 ...rt_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 | 0 ...rt_gby2-3-538a02b95c066b307652c8d503470c8e | 0 ...rt_gby2-4-521e0c1054cfa35116c02245874a4e69 | 1 - ...rt_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 | 1 - ...t_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...rt_gby3-6-521e0c1054cfa35116c02245874a4e69 | 2 +- ...rt_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 | 2 +- ...rt_gby3-9-521e0c1054cfa35116c02245874a4e69 | 2 +- ...n_union-0-24ca942f094b14b92086305cc125e833 | 2 +- ..._complex-0-6a7c4841dab05ebae84309c9571bec6 | 0 ...complex-1-abac744dee1a4f4152781b1565fe1364 | 5 - ...complex-2-f07e9814229ed440bd0ac7e4bd924ce9 | 8 - ...complex-3-fb014bc1e9bfd919a3703cf5afb77448 | 0 ...complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d | 2 - ...virtual-0-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...virtual-1-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...irtual-10-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...irtual-11-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...irtual-12-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...irtual-13-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...irtual-14-67b834deba21676e02c155b25195a019 | 0 ...irtual-15-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...irtual-16-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...irtual-17-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...virtual-2-67b834deba21676e02c155b25195a019 | 0 ...virtual-3-52f90e6bf3f2c17b82ed34318d2c612f | 0 ...virtual-4-89696914fad2d7b7bfc5b7729a7e7c34 | 0 ...virtual-5-a299c8b1a9f8c2772989a5454574f4e5 | 0 ...virtual-6-e9a72fa546e90b8cbbd2e9470450cb56 | 0 ...virtual-7-1ea9df2d7af3c79ebef07d6087c8106f | 0 ...virtual-8-67b834deba21676e02c155b25195a019 | 0 ...virtual-9-52f90e6bf3f2c17b82ed34318d2c612f | 0 .../newline-2-4eb54a664e549614d56ca088c8867d | 0 ...s_subq1-1-f91e87acd51f7477d96620b5f9deece6 | 2 +- ...plicate-1-b1e2ade89ae898650f0be4f796d8947b | 2 +- ...plicate-3-a873d2d6991308b21ecdc46ac777c716 | 2 +- ...plicate-4-24ca942f094b14b92086305cc125e833 | 2 +- ...plicate-6-a873d2d6991308b21ecdc46ac777c716 | 2 +- ...input37-0-6ed1b2ff177492c003161ee91e982c10 | 0 ...input37-1-e3ab2563222fb8678c7c269e09e1e8d8 | 0 ...t_into1-0-d3afbe52e3a1eb246a79c1320f82c480 | 0 ...t_into1-1-5c7fc72601c9add7bc86df7e4c24af63 | 0 ..._into1-10-3c8f9dff0a12ca2b929d04b4873a4681 | 0 ..._into1-11-6f59e35684a552a855e4dc3aee667092 | 1 - ..._into1-12-d3afbe52e3a1eb246a79c1320f82c480 | 0 ...t_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ...t_into1-3-2983d09b973ea94bc701970a17fc3687 | 0 ...t_into1-4-a2f6c6c77e94833197095dc48643f9c9 | 1 - ...t_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 | 0 ...t_into1-6-2983d09b973ea94bc701970a17fc3687 | 0 ...t_into1-7-fe194a16b48b763e6efdf6fcc6116296 | 1 - ...t_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad | 1 - ...t_into1-9-461110270cec5b6d012b2c685cf2cce9 | 0 ..._alias1-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ..._alias2-3-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ll_cast-0-48a80d4fdc8009234af31ebcb6e03364 | 0 ...ll_cast-1-7257e6f8170e545962d27741353f672c | 1 - ...llgroup-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...llgroup-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...lgroup-11-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...lgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...lgroup-13-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...lgroup-15-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...llgroup-3-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...llgroup-4-dbcec232623048c7748b708123e18bf0 | 2 +- ...llgroup-5-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...llgroup-7-54a5fd76cdeff565c8c7724695aca302 | 2 +- ...llgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...llgroup-9-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...lgroup2-0-dbcec232623048c7748b708123e18bf0 | 2 +- ...lgroup2-1-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...group2-12-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...group2-13-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...lgroup2-4-dbcec232623048c7748b708123e18bf0 | 2 +- ...lgroup2-5-85c4f90b754cd88147d6b74e17d22063 | 2 +- ...lgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 | 2 +- ...lgroup2-9-c67a488530dc7e20a9e7acf02c14380f | 2 +- ...group3-1-61ead7f73d525e0d9e21beba91a3d39e} | 0 ...lgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 ...roup3-12-61ead7f73d525e0d9e21beba91a3d39e} | 0 ...group3-12-a4d6d87d12f0e353d7a3ae5c087dab44 | 0 ...roup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06} | 0 ...group3-13-f529994bef750d8edd02ede8a4323afb | 0 ...group3-15-cd90e16da918bac569e9f04faaedd280 | 2 +- ...roup3-18-113e2b587784d54d2a5b5f091389397e} | 0 ...group3-18-a24e178327b39f57ca7dfa2e69296bb2 | 0 ...roup3-19-77de4b2d65eee228848625cdd422178d} | 0 ...group3-19-b8f60d35c1c09c57efd4c339799e5404 | 0 ...group3-2-a5e12cfbc1799ce9fa9628d81b8c0b06} | 0 ...lgroup3-2-f529994bef750d8edd02ede8a4323afb | 0 ...group3-21-e09c6bf2f6112981793fbd4386d43ff6 | 2 +- ...lgroup3-4-cd90e16da918bac569e9f04faaedd280 | 2 +- ...group3-6-113e2b587784d54d2a5b5f091389397e} | 0 ...lgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 | 0 ...group3-7-77de4b2d65eee228848625cdd422178d} | 0 ...lgroup3-7-b8f60d35c1c09c57efd4c339799e5404 | 0 ...lgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 | 2 +- ...group5-1-642e12a05bf01a6716bfa15ed0012629} | 0 ...lgroup5-1-c75bafea030f127bce9a4a5dbb14f40b | 0 ...group5-3-35517978e09aa1bd1d15f51d11e08fd5} | 0 ...lgroup5-3-4492a9ce0d8502584b872860d53c449c | 0 ...lgroup5-5-60d7180632a63c79eeba47e30b854f4c | 2 +- ...linput2-2-21058230c4992a682c4adef9881fa9a2 | 2 +- ...lscript-1-3729d183a27e89a87ca9b9c9946e13a5 | 0 ...script-1-f07dfd6ef687e038083deca5941d8174} | 0 ...lscript-2-17238164053203d56d30704e2c098e80 | 0 ...script-2-bb1abcf2f4a2a5cd5c058104901627bb} | 0 ...lscript-4-472199a0c6b8f760a90863deb69e9710 | 2 +- ...pe_conv-0-c3f17f31d887479896bf454a2f6b15dc | 0 ...pe_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 | 1 - ..._create-0-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ..._create-1-b5209e6f850fc958dc9ebced21519684 | 0 ...create-10-3027edcda6a6030c6bff93681529c34d | 0 ...create-11-14ea7dcc6898979aaa61650e3fc46187 | 32 - ...create-12-dd590639ea8d5b27ccf3312be2562fc2 | 0 ...create-13-14ea7dcc6898979aaa61650e3fc46187 | 34 - ...create-14-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...create-16-3027edcda6a6030c6bff93681529c34d | 0 ...create-18-14ea7dcc6898979aaa61650e3fc46187 | 32 - ...create-19-a85f67347e5a19fc879ae83aa24c0ce4 | 0 ..._create-2-f375f322df98df65e6b1bd0d4ce8e208 | 0 ...create-20-176d469a0edba57404416535c7d48023 | 29 - ...create-21-e3accaf8ebc11bac7091db28d01fd2a7 | 0 ...create-22-31944ad765bbf262b4ebafdb06df59a6 | 3 - ...create-23-be779533ea8967231e644209114c8350 | 0 ...create-24-fe59ff341395bc347dfb9cfdee397da2 | 3 - ...create-25-c55e620d82921c40ebcdb94454ac1ead | 3 - ...create-26-aa4e0c8a3de340c1133d9fdddda8a18c | 3 - ..._create-27-2eefc0bb4283fc255409a5c41203c89 | 3 - ...create-28-cf8aa1014707dfe576820041e47436e2 | 3 - ...create-29-cee6b57822d79ce80e14e58663bf7b86 | 0 ..._create-3-8480c37b4f3a6768f459361cf8470dae | 0 ...create-30-b5e831bf4eb811ac4f06e9ac11a5283d | 0 ...create-31-c934a1ca351b76c5c4ef87cb5655f1d3 | 0 ...create-33-2151d07a07ab6b28782b7e8e4f848e36 | 0 ...create-36-ca15a32658195ecaffe11d11f487fb0a | 10 - ...create-37-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...create-38-b5209e6f850fc958dc9ebced21519684 | 0 ...create-39-f375f322df98df65e6b1bd0d4ce8e208 | 0 ..._create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 ...create-40-8480c37b4f3a6768f459361cf8470dae | 0 ...create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 | 0 ..._create-5-4f8e2d1f18399679a97c49631c4ef921 | 0 ..._create-6-2ae0fd655aa777b41015e4125f680324 | 32 - ..._create-7-27aa4a8093e80a7437266f349ea927c0 | 0 ..._create-8-14ea7dcc6898979aaa61650e3fc46187 | 32 - ..._create-9-9e7ba3cead1b5040ee3c23e8fc235d25 | 0 ...reshold-0-a399c0ef0f1578f9d2456727008dee16 | 2 +- ...hreshold-3-1a7f96f49e2992b93d97edcbdb0075d | 2 +- ...reshold-6-272f5d299289829dc22cc31f70115dd9 | 0 ...eshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3} | 0 ...reshold-7-acf39b28d4b76382acf5c56c21932ff9 | 2 +- ...reshold-9-f7b722063a6948d22aaaab0707cddde1 | 2 +- ...rt_cols-0-bfdd54175515a0557f8bd427ec23c453 | 0 ...rt_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...rt_cols-2-ab12575b9b663420d9b3702f5a32a086 | 0 ...rt_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 | 0 ...rt_cols-4-1c1eab8fc34159875afe38eb2413434e | 0 ...rt_cols-5-c0dce07a38f3c540c2b17db0a495a657 | 10 - ..._strings-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...strings-1-a67f0c90e3f99257a7364cc96e5405cf | 0 ...strings-2-a34bd419a63852c1c75f195a495ff333 | 9 - ...strings-3-3339ace17de3201296847caf29c42e99 | 0 ...strings-4-eee6b7f3a881c8f56510bbd53aeebe1e | 10 - ...th_nulls-0-a338239d2fc4e22e23ab82fa5f62139 | 0 ...h_nulls-1-2c16215823a5e6904059a48a3077da4e | 0 ...h_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 | 0 ...h_nulls-3-bbe8d353c397b795e2732bd59648d291 | 0 ...h_nulls-4-a0231a52d63534679bfab3f6a0b8effc | 0 ...h_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f | 5 - .../order-1-57d93bd7619dfc460dfa763c12142bb9 | 2 +- .../order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 | 2 +- .../order2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- .../order2-2-4162aa366dc0836eed15cc819226907f | 2 +- ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...oin_ppr-2-b60ab17f7372863504804717c4276595 | 2 +- ...join_ppr-4-be64a45a645ea5363109323a378d335 | 2 +- ...arallel-0-23a4feaede17467a8cc26e4d86ec30f9 | 2 +- ...arallel-1-851e262128626126ae1ad87869db7c54 | 2 +- ...rallel-11-6230286bc168af7b010968b543690a2a | 2 +- ...rallel-12-73a915d42e62c0e895a82602a502ee43 | 2 +- ...arallel-2-43d53504df013e6b35f81811138a167a | 2 +- ...arallel-7-6230286bc168af7b010968b543690a2a | 2 +- ...arallel-8-73a915d42e62c0e895a82602a502ee43 | 2 +- ...arallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c | 2 +- ...orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d | 0 ...orderby-1-be7423a5e77b7289153f74bc3fd97f44 | 0 ...rderby-11-39767502cbda75590c0c4b8fd089b793 | 0 ...rderby-12-7943485bcc72b2040c45c62e45ac8853 | 0 ...rderby-13-988591bf9ab008fdc4d71560aa57a736 | 32 - ...rderby-14-d93951df2ffc18dc09ab0ba2d46f1823 | 48 - ...orderby-2-be7423a5e77b7289153f74bc3fd97f44 | 0 ...orderby-6-670ec1b1f28d92d72a924c29d622aa8f | 0 ...orderby-7-7943485bcc72b2040c45c62e45ac8853 | 0 ...orderby-8-988591bf9ab008fdc4d71560aa57a736 | 32 - ...orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 | 48 - ...star_by-0-57a4ea931689f9475b687292f34abfa4 | 2 +- ...star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 | 2 +- ...star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc | 2 +- ...star_by-3-498e2973594ccf45448ba19552bfb1cd | 2 +- ...star_by-4-dc9d9d8bbb6259c89a97065902a345ec | 2 +- ...star_by-5-6888c7f7894910538d82eefa23443189 | 2 +- ...star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 | 2 +- ...star_by-7-f36c71c612ab8da4f4191980c1b61fea | 2 +- ...artInit-0-aaa3124841a8140a41d3556a4ccaa6a5 | 0 ...artInit-1-c0ec92801bec7ece0a156d407b601f7b | 1 - ...artInit-2-5db6fb2267a648ac57af8f56f91cf9a2 | 1 - ...artInit-3-878a82ca7519e3eafd3f2661b2ac1b88 | 0 ...artInit-4-5db6fb2267a648ac57af8f56f91cf9a2 | 1 - ...l_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 | 2 +- ...l_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 12 +- ...l_props-4-c04c695a6ebed215889ce75edcb33eb4 | 2 +- ...s_empty-0-c367ba7f534037ab96efc7f2273508c7 | 2 +- ...s_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 12 +- ...th_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 | 2 +- ...th_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 | 12 +- ...th_star-4-c04c695a6ebed215889ce75edcb33eb4 | 2 +- ...rtcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 | 0 ...tcols1-1-b562ff3e36de23a686d122967a1f91c8} | 0 ...n_date-1-6b0952309c3ebdd6dcb7066891d1bd74} | 0 ...n_date-10-aad6078a09b7bd8f5141437e86bb229f | 1 + ..._date-11-a01c3791f59e819c750e213b6c65d084} | 0 ...n_date-12-2662a237c86cf9e4a150a4f1856b8239 | 1 + ...n_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 | 1 + ..._date-14-3f187dad9a2fdfc6f7a3566309016f9c} | 0 ..._date-15-e4366325f3a0c4a8e92be59f4de73fce} | 0 ...n_date-16-32cf81c1a36451eccb07b20dffd930ac | 1 + ...n_date-17-8654528691598a5eef8e3c1059d24117 | 1 + ...n_date-18-a1e769630ac3caed7325a3a256713b24 | 1 + ...n_date-19-95f1399e12124840caee7492c0f3036d | 1 + ...n_date-2-220048240a7050a98411ddbc6b1f82cf} | 0 ...n_date-20-e734eb4fc8894c9a2b2b9cdac4270fba | 1 + ...n_date-21-b60c0a3677591991152a0aa500bdce68 | 1 + ..._date-22-fe277bd0a30e016826d242fd5bd61714} | 0 ...n_date-23-7f014b494185e4c2a6048efb89131344 | 1 + ..._date-24-7ec1f3a845e2c49191460e15af30aa30} | 0 ...n_date-3-f8887dd18c21bf0306e293d463b3e1d7} | 0 ...n_date-4-f7c20c208e052305c1710a5e1e80c5c8} | 0 ...n_date-5-59355f4e222dcd4f77f51d15bd896f11} | 0 ...n_date-6-416ab10ac818d432cd3a81870d6e5164} | 0 ...on_date-7-1405c311915f27b0cc616c83d39eaacc | 3 + ...on_date-8-7703adfcfdd44c9250f9eba478004714 | 5 + ...on_date-9-a425c11c12c9ce4c9c43d4fbccee5347 | 1 + ...de_name-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ...de_name-1-4de8e998198c8df484e9102f60ba05c1 | 0 ...e_name-10-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...e_name-11-94d637f6e5cee2771b9844438008a618 | 1 - ...de_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...de_name-6-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...de_name-7-94d637f6e5cee2771b9844438008a618 | 1 - ...de_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 | 8 +- ...schema1-4-9b756f83973c37236e72f6927b1c02d7 | 10 +- ...schema1-5-52a518a4f7132598998c4f6781fd7634 | 8 +- ..._format-3-54d18742b4eab85edd1946ef139771b4 | 2 +- ...al_char-0-6ae3e0805cbfa967f50ad222bc33b772 | 0 ...al_char-1-4de8e998198c8df484e9102f60ba05c1 | 0 ...al_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...al_char-5-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...al_char-6-94d637f6e5cee2771b9844438008a618 | 1 - ...al_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 | 0 ...al_char-8-f1c90c8f4afd0c2ee66019e017997eb5 | 3 - ...al_char-9-94d637f6e5cee2771b9844438008a618 | 1 - ...check-11-60aad884be613d18d3b89fca3b90dc94} | 0 ..._check-11-a1164f1770d2f787b520fbc3d345911a | 0 ..._check-2-3a6bb204d9524299f28adf1cc35d6f4d} | 0 ...e_check-2-5e857e1536264658caf0df9b7e28652f | 0 ...e_check-6-1f5f161f5e992a6aa7358dcbbffc5939 | 0 ..._check-6-663051c7106b8ee6913ca1b007157941} | 0 ...archar1-2-bca5c9edccc3a84e0a9ef92ebcbe746} | 0 ...archar1-2-fc9381cdfb786fc3b7b9b35df6003474 | 0 ...archar1-3-968759281b7d1bf0a60991ed04953b93 | 0 ...varchar1-3-9cec4b1d156f5a9cb587470b98de15} | 0 ...archar1-4-deb9b7715610152bda285a3a33b772ef | 0 ...rchar1-4-e9c4530e270db6d44cc54292e4eff680} | 0 ...rchar1-5-661aefd18c44c1eb02c2aaf8fab59f73} | 0 ...archar1-5-f77df9235261b945ca0a55e1afe2e5ce | 0 ...etadata-0-c25482149887659ec66d7534cad22f63 | 0 ...etadata-1-29f193b3e8def419733366c578e8a236 | 0 ...etadata-2-14033db9e60aa0bc4f64376c96195adf | 0 ...etadata-3-9bdf636332ca722093413523366efa86 | 0 ...etadata-4-e784348b7221bb26830cc1ebda69bdde | 1000 ------- ...eformat-0-c854b607353e810be297d3159be30da4 | 0 ...eformat-1-9fad934993b77eef15a5d10eb203a378 | 0 ...format-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 15 - ...format-11-606ad10de7caf7e65e09778f2673e712 | 25 - ...format-12-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 - ...format-13-c262e8c736b67119b9806f69eb492ef3 | 50 - ...format-14-da1b1887eb530c7e9d37667b99c9793f | 0 ...format-15-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format-16-7d619408a560b5c8d4a06dcd0ee106e5 | 15 - ...format-17-a488cb92e1388a7542d8787a22f4fb55 | 15 - ...format-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 15 - ...format-19-e3b55daf170a032dcc8ed12ee26ccf63 | 15 - ...eformat-2-7d619408a560b5c8d4a06dcd0ee106e5 | 15 - ...format-20-606ad10de7caf7e65e09778f2673e712 | 25 - ...format-21-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 - ...format-22-c56c391659f5701a9d2e8782a60f7f8a | 25 - ...format-23-c262e8c736b67119b9806f69eb492ef3 | 75 - ...format-24-fe564b4f68241ec5c5884e44a1306f4f | 75 - ...eformat-3-a488cb92e1388a7542d8787a22f4fb55 | 15 - ...eformat-4-606ad10de7caf7e65e09778f2673e712 | 25 - ...eformat-5-c262e8c736b67119b9806f69eb492ef3 | 25 - ...eformat-6-6c4f7b115f18953dcc7710fa97287459 | 0 ...eformat-7-f5f427b174dca478c14eddc371c0025a | 0 ...eformat-8-7d619408a560b5c8d4a06dcd0ee106e5 | 15 - ...eformat-9-a488cb92e1388a7542d8787a22f4fb55 | 15 - ...ormat10-0-66ee62178e3576fb38cb09800cb610bf | 1 - ...ormat10-1-198cb7d650a506ec3420b94b82a01375 | 0 ...ormat10-2-f723dedd396bd468107755b4495c1031 | 0 ...ormat10-3-c278fd699aa25809bdef310fb92f510e | 2 - ...ormat10-4-620cb6a290ef5297ac1d3a9ea776e2bf | 2 - ...ormat10-5-dd4c3f1636af9a7371edf7142abee088 | 0 ...ormat10-6-620cb6a290ef5297ac1d3a9ea776e2bf | 2 - ...ormat10-7-c278fd699aa25809bdef310fb92f510e | 2 - ...ormat11-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...rmat11-10-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...rmat11-11-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat11-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...ormat11-3-61f1abcdc66a64c11df85dded920d167 | 0 ...ormat11-4-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat11-5-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat11-6-ee7394c912991b8cd4401fb94942351f | 0 ...ormat11-7-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat11-8-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat11-9-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat12-1-b30db33650de0545cbcd53769ed7cf40 | 0 ...rmat12-10-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...rmat12-11-75a0aa2df39cbdc6a470b4c88803b42b | 4 - ...rmat12-12-3fc394a7a3d43a70968282ef1ee21dbd | 4 - ...rmat12-13-26a9a9cafa15d0223b934eba4777aea7 | 0 ...rmat12-14-d35f445501407d6fae5c3ad161fb2236 | 6 - ...rmat12-15-3fc394a7a3d43a70968282ef1ee21dbd | 6 - ...ormat12-2-92bfcf88ca528eb6c9259142bf6541e5 | 0 ...ormat12-3-61f1abcdc66a64c11df85dded920d167 | 0 ...ormat12-4-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat12-5-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat12-6-ee7394c912991b8cd4401fb94942351f | 0 ...ormat12-7-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat12-8-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat12-9-b8aed3dd7d7c151f5d96050c4058e71d | 0 ...format13-1-9a812f44c4c7a4c72b4be95139e6fd6 | 0 ...rmat13-10-5bc619cec1d519c1dc6423f841b1c0a4 | 1 - ...ormat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 | 0 ...ormat13-3-9b3e4a236550f1900a36566830b24024 | 0 ...ormat13-4-49cf189a09e11c2c635fbb574b89a2af | 0 ...ormat13-5-2ebe5fe98e830143b9571b13fe59dc0b | 0 ...ormat13-6-fa2f56078df18add8e5a77e538756488 | 0 ...ormat13-7-1d822cc037d9281ce172e2d5685b1495 | 0 ...ormat13-8-e4531456a7418952ec1d69e18bc8460b | 0 ...ormat13-9-8f6983fda271fba18d218c75b8fb2b3d | 1 - ...rmat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 | 0 ...rmat14-11-bbfb0a95274044dea4732e35c1d7ecbe | 0 ...rmat14-12-be84e8199b0a3b9f72e552018854ac15 | 1 - ...rmat14-15-dd6e9965d271bd35604059540c23d967 | 1 - ...rmat14-17-f4a3e39d5df18861e9fe67ef37af57e1 | 1 - ...rmat14-18-58080916a9f5883121bcaad719be0309 | 1 - ...ormat14-3-2683f9835169a568c1f03dae859d27d2 | 0 ...ormat14-4-2d2e1149d2f035017bb37bbfaad4def0 | 0 ...ormat14-5-61a8225e20f36e286b4d02501d2c80d9 | 0 ...ormat14-6-b5165befb75ebeed42f3e69d4d64375c | 0 ...ormat14-7-e438f7ec386b2ab19660e9da7260dd95 | 0 ...ormat14-8-f0e3ddaa1e6ea067444b1f540bfac293 | 0 ...ormat14-9-adeaa033260f16c5bc106e761e5fef8b | 0 ...ormat15-1-e1cf6c355de3ae8db7564b1676199117 | 0 ...rmat15-10-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...rmat15-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...rmat15-12-8250422b5ed16120ee33bd4fa69d3f47 | 3 - ...rmat15-13-3fc394a7a3d43a70968282ef1ee21dbd | 3 - ...ormat15-2-107d7c681b43611df056238be242127b | 0 ...ormat15-3-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat15-4-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat15-5-ee7394c912991b8cd4401fb94942351f | 0 ...ormat15-6-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat15-7-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat15-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat15-9-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat16-0-66ee62178e3576fb38cb09800cb610bf | 1 - ...ormat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 | 0 ...rmat16-10-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...rmat16-11-662bece7f71ef854ca6e25f0eef5b830 | 0 ...rmat16-12-8250422b5ed16120ee33bd4fa69d3f47 | 3 - ...rmat16-13-3fc394a7a3d43a70968282ef1ee21dbd | 3 - ...ormat16-2-107d7c681b43611df056238be242127b | 0 ...ormat16-3-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat16-4-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat16-5-ee7394c912991b8cd4401fb94942351f | 0 ...ormat16-6-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat16-7-3fc394a7a3d43a70968282ef1ee21dbd | 2 - ...ormat16-8-ed1e2dbef8eb8efbf83a50dc646485ba | 0 ...ormat16-9-75a0aa2df39cbdc6a470b4c88803b42b | 2 - ...ormat17-0-7c49277a7217a147685d30e27822d273 | 0 ...ormat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 | 1 - ...format2-0-c854b607353e810be297d3159be30da4 | 0 ...format2-1-9fad934993b77eef15a5d10eb203a378 | 0 ...ormat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ...ormat2-11-20a02894f5e9340e89b55a30bef252b7 | 75 - ...format2-2-6c4f7b115f18953dcc7710fa97287459 | 0 ...format2-3-f5f427b174dca478c14eddc371c0025a | 0 ...format2-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format2-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format2-6-32b36a6c3344c5fcca0ad6c93ffcab62 | 1 - ...format2-7-561ef0dbccfcbfbb0c75f33ebfd4203d | 0 ...format2-8-20a02894f5e9340e89b55a30bef252b7 | 75 - ...format2-9-50131c0ba7b7a6b65c789a5a8497bada | 1 - ...format3-0-c854b607353e810be297d3159be30da4 | 0 ...format3-1-6c4f7b115f18953dcc7710fa97287459 | 0 ...ormat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 | 25 - ...format3-2-f5f427b174dca478c14eddc371c0025a | 0 ...format3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 14 - ...format3-4-da1b1887eb530c7e9d37667b99c9793f | 0 ...format3-5-517aaa22478287fa80eef4a19f2cb9ff | 0 ...format3-6-e3b55daf170a032dcc8ed12ee26ccf63 | 14 - ...format3-7-c56c391659f5701a9d2e8782a60f7f8a | 25 - ...format3-8-f5f427b174dca478c14eddc371c0025a | 0 ...format3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a | 14 - ...format5-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format5-6-a0e23b26ee1777ccc8947fb5eb1e8745 | 2 +- ...format5-8-a0e23b26ee1777ccc8947fb5eb1e8745 | 2 +- ...format6-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format6-6-e95296c9f7056b0075007c61d4e5e92f | 2 +- ...format6-7-4758d41d052eba37a9acd90c2dbc58f0 | 2 +- ...format7-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format7-4-a34505bd397bb2a66e46408d1dfb6bf2 | 2 +- ...format7-5-f2c42f1f32eb3cb300420fb36cbf2362 | 2 +- ...format8-0-66ee62178e3576fb38cb09800cb610bf | 1 - ...format8-1-22e3d59a0423473051535684bca72b27 | 0 ...format8-2-55ae9fbf6daa36225dd386e34025dd38 | 0 ...format8-3-c561806d8f9ad419dc9b17ae995aab68 | 0 ...format8-4-ae71ce67b5d4a91bce1b34acde830268 | 0 ...format8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e | 0 ...format8-6-ae6a5016b6a6ace6b6f2576a4c666b4e | 0 ...format8-7-d1a5913edaaec9654edb333e8207f57b | 20 - ...format8-8-624b059dfaa86e2c78f065169de325cd | 20 - ...format9-0-66ee62178e3576fb38cb09800cb610bf | 2 +- ...format9-5-d1a5913edaaec9654edb333e8207f57b | 2 +- ...format9-6-624b059dfaa86e2c78f065169de325cd | 2 +- .../ppd1-0-ae225e86c2ae20519ffdf23190454161 | 2 +- .../ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- .../ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 | 2 +- .../ppd1-4-145c2779dadb5bd921dc2baac608b803 | 2 +- .../ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 | 2 +- .../ppd2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- .../ppd2-1-145c2779dadb5bd921dc2baac608b803 | 2 +- .../ppd2-3-fccdc7a24b626308d9ec17608f36564b | 2 +- .../ppd2-6-ae225e86c2ae20519ffdf23190454161 | 2 +- .../ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- .../ppd2-9-fccdc7a24b626308d9ec17608f36564b | 2 +- ...nt_expr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...nt_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...nt_expr-5-7da8c74586c99d96254f6f031bcaeb1c | 2 +- ...nt_expr-6-145c2779dadb5bd921dc2baac608b803 | 2 +- ...nt_expr-9-7da8c74586c99d96254f6f031bcaeb1c | 2 +- ...t_where-1-84c951641740895ca1c8ddc098805da5 | 2 +- ...ppd_gby-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d | 2 +- ...ppd_gby-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d | 2 +- ...pd_gby2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...pd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...ppd_gby2-3-321628d4c52f6992f2680a3a162f19f | 2 +- ...pd_gby2-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...ppd_gby2-6-321628d4c52f6992f2680a3a162f19f | 2 +- ...by_join-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...by_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...by_join-3-145c2779dadb5bd921dc2baac608b803 | 2 +- ...pd_join-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...pd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...pd_join-3-aab5d092ca17ed23ee71d3a6ef653998 | 2 +- ...pd_join-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...pd_join-6-aab5d092ca17ed23ee71d3a6ef653998 | 2 +- ...d_join2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...d_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...d_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 | 2 +- ...d_join2-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 | 2 +- ...d_join3-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...d_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...d_join3-3-42cd793c031af1f7961d7b5e237de76b | 2 +- ...d_join3-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_join3-6-42cd793c031af1f7961d7b5e237de76b | 2 +- ...r_join1-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join1-3-9dfd333c14f82fd71c213e1d39c83524 | 2 +- ...r_join1-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join1-6-9dfd333c14f82fd71c213e1d39c83524 | 2 +- ...r_join2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join2-3-5340cd2d88dcf41dc18119389a475e36 | 2 +- ...r_join2-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join2-6-5340cd2d88dcf41dc18119389a475e36 | 2 +- ...r_join3-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 | 2 +- ...r_join3-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 | 2 +- ...r_join4-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...r_join4-3-ac57dc2a7151f20029d6a97049d6eebe | 2 +- ...r_join4-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...r_join4-6-ac57dc2a7151f20029d6a97049d6eebe | 2 +- ...r_join5-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...r_join5-1-145c2779dadb5bd921dc2baac608b803 | 2 +- ..._random-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ..._random-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ..._random-3-145c2779dadb5bd921dc2baac608b803 | 2 +- ...ansform-0-ae225e86c2ae20519ffdf23190454161 | 1 - ...ansform-1-f18babdee8d2d4206ce4f2a93b6575f9 | 1 - ...ansform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ansform-3-5d0e4236af589d3e63a8dd84e663b745 | 84 - ...ansform-4-145c2779dadb5bd921dc2baac608b803 | 1 - ...ansform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 | 0 ...ansform-6-5d0e4236af589d3e63a8dd84e663b745 | 84 - ...udf_col-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...udf_col-6-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_union-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...d_union-1-f18babdee8d2d4206ce4f2a93b6575f9 | 2 +- ...d_union-3-678c8197f458b459171c266f7431683e | 2 +- ...d_union-4-145c2779dadb5bd921dc2baac608b803 | 2 +- ...d_union-6-678c8197f458b459171c266f7431683e | 2 +- ...on_view-0-d680c8ac7e5121c6971458a9022c00b9 | 0 ...on_view-1-bfb5ca0dd2d5f070ce359790f8b91480 | 0 ...n_view-10-a74a5ff32204d842175b3d880477178f | 0 ...n_view-11-745e750f4f4a36af27e87338a979240c | 0 ...on_view-2-ac54e78582e6bd5d5533a0f3bfb51369 | 0 ...on_view-3-f6fd9a5271e172e4a65ea57aaa41d512 | 0 ...on_view-4-19cd3ea4e274befd809e4aad21da8d50 | 0 ...on_view-5-443c0979b586f6a6dfb0dc5d28cd5124 | 0 ...on_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 | 0 ...on_view-7-aab9d79f65d6edfc1cae88a14b8e106c | 0 ...on_view-8-e3ee46daeac605b05c7ada97b3c43578 | 0 ...on_view-9-8a77c30b6b07717d1c0bee971fb0132c | 0 .../ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 | 0 .../ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 | 36 - .../ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 | 0 .../ppd_vc-3-be78760949abf728580442e9e37a3ce7 | 36 - ...shdown-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ushdown-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 ...shdown-1-855b235f9c760ba9d6d0200bfd1ad08b} | 0 ...ushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 | 0 ...shdown-10-178be64f122542983ff4384df4bb1530 | 0 ...hdown-10-2957fd9b211cee5f0372525a1de55c19} | 0 ...shdown-11-3828393aa33a55cf1aea707e1be0a452 | 0 ...hdown-11-b8d6f0ffc8294497c792b26958adee45} | 0 ...shdown-12-5affd35b94b0de3fb2a43f8729526055 | 0 ...hdown-12-ab1b9a0cdd9586c96d8856a9d632b89c} | 0 ...hdown-13-2c316c67fd494d878fbbea107d283c3b} | 0 ...shdown-13-2c64f5abe8c23957d7f0602e9c257cd1 | 0 ...hdown-14-53b4be82a1538844d03b200429efa02b} | 0 ...shdown-14-c3188230751166d9d90082cd357ecb0c | 0 ...hdown-15-71ab93d38ed2908069091c7c8cc0aba1} | 0 ...shdown-15-e52a39743f82af47902341a7ecd38afb | 0 ...shdown-16-70bef3ba441873327e188ff2ec6d68ae | 0 ...hdown-16-855cb54d28034fdb20a3615ee0918d63} | 0 ...hdown-17-d3ccf2722a8b7281fcee61b2544772c8} | 0 ...hdown-18-fcd10fbfc55a2c0aa843fe618f9613c6} | 0 ...hdown-19-ff36e3978571ac05f11e8322c024e4b6} | 0 ...ushdown-2-5eede4874e678021938683fc2f5dc900 | 0 ...shdown-2-f689aaf15b08c433d1e93ce977a9b6d5} | 0 ...hdown-20-481005cf034ef3d7b998da32eb82aa9a} | 0 ...hdown-21-9073825e8b9804331f780980d1f9fa92} | 0 ...hdown-22-9cb28f0d3a434c9d1ab039192906ec9d} | 0 ...hdown-23-678f50025924fee7b59f66e2abdb472d} | 0 ...hdown-24-4d5bfa800ba434f464b07bf9b5d39f20} | 0 ...hdown-25-22663f09ea1c1bc303756067e84df5a7} | 0 ...shdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b | 0 ...hdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b} | 0 ...hdown-27-8065c18e387fd8bffae19a80af8dc1d4} | 0 ...hdown-28-b72de558c88ae91460989938000e0d27} | 0 ...hdown-29-7217735d357770af4ffee730e4e9add4} | 0 ...shdown-3-5eede4874e678021938683fc2f5dc900} | 0 ...ushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 | 0 ...hdown-30-630e2f7918b7727fc4ca057fa21e2eea} | 0 ...hdown-31-2e062414293b643ea4d7f6df92f939e4} | 0 ...hdown-32-5eba4bf10315099129eae319d73636cf} | 0 ...hdown-33-35af0585a4f98bc222c786688cb6de6b} | 0 ...hdown-34-a5594625510703427ab8bae5d0563f73} | 0 ...shdown-4-c5c542f8ee81cd0afd44e67fc7b4d306} | 0 ...ushdown-4-f54bebec398f0fdfdbc0393123dba234 | 0 ...pushdown-5-78af12432bcbf99d4a0d41c25f964de | 0 ...shdown-5-f54bebec398f0fdfdbc0393123dba234} | 0 ...ushdown-6-58724fbe96a0b3deceef20c8cc5e318d | 0 ...ushdown-6-78af12432bcbf99d4a0d41c25f964de} | 0 ...ushdown-7-14570f946e75924d7926c809485951d1 | 0 ...shdown-7-58724fbe96a0b3deceef20c8cc5e318d} | 0 ...shdown-8-14570f946e75924d7926c809485951d1} | 0 ...ushdown-8-3ba325662296fc455f07f1c835495e4c | 0 ...shdown-9-3ba325662296fc455f07f1c835495e4c} | 0 ...pushdown-9-4375f17bed264d5554a67d119fa5dd1 | 0 ...hdown2-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...shdown2-0-855b235f9c760ba9d6d0200bfd1ad08b | 0 ...shdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 | 0 ...hdown2-1-855b235f9c760ba9d6d0200bfd1ad08b} | 0 ...hdown2-10-4fa4ba4c739b6f44975d41e4705d4389 | 1 + ...hdown2-10-ab3e46183487096745d7d90e3020e94c | 0 ...hdown2-11-680316eba447eb4649530fdc1c37d95b | 0 ...down2-11-ab3e46183487096745d7d90e3020e94c} | 0 ...hdown2-12-720582e599a974ee9ca46b653678a14a | 0 ...down2-12-b10b9e14f5a7a7a92c0c68df6dbc656a} | 0 ...down2-13-154f9859bd0822e287fbfdff12fd45ff} | 0 ...hdown2-13-bd9067aeee8884a32db921b6d478f806 | 0 ...hdown2-14-8d01597374157f2d3d066840983ba1f8 | 1 - ...down2-14-cf4a30b5c8329d8d79ddf762f318fbb3} | 0 ...shdown2-15-5614065e1b8e709f68be4fa67666f41 | 1 - ...hdown2-15-8d01597374157f2d3d066840983ba1f8 | 1 + ...hdown2-16-3a67618e47c977f58c9dd8f4b9a576eb | 1 - ...shdown2-16-5614065e1b8e709f68be4fa67666f41 | 1 + ...hdown2-17-3a67618e47c977f58c9dd8f4b9a576eb | 1 + ...hdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe | 2 - ...hdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe | 2 + ...hdown2-18-e2c7e9b01ec95dfcc685827e24d66775 | 3 - ...hdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 | 1 - ...hdown2-19-e2c7e9b01ec95dfcc685827e24d66775 | 3 + ...shdown2-2-efd3e4c333d4efb81218df8921e58f9e | 0 ...hdown2-2-ffa167b63d612a4986d02f5c0623ea7b} | 0 ...hdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 | 1 + ...hdown2-20-ece6fe0efc1e658b36ddc10f0653d229 | 1 - ...hdown2-21-ece6fe0efc1e658b36ddc10f0653d229 | 1 + ...shdown2-3-1886675984606b6c972c4a26dca6fd2c | 1 - ...hdown2-3-c7753746c190414723d66a8f876499c7} | 0 ...shdown2-4-1886675984606b6c972c4a26dca6fd2c | 1 + ...shdown2-4-33b137b28e7246ec3c2acb937c638910 | 1 - ...shdown2-5-33b137b28e7246ec3c2acb937c638910 | 1 + ...shdown2-5-b5a2518af801f95fe52a75dfc1d3e867 | 0 ...shdown2-6-96059d8042ccb2ce355541daf9893954 | 0 ...hdown2-6-b5a2518af801f95fe52a75dfc1d3e867} | 0 ...shdown2-7-93922fe8d88643e1b8aa7e3ef4195404 | 0 ...hdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b} | 0 ...hdown2-8-4507a3f200b3ce384191c91acd324dc7} | 0 ...shdown2-8-c86940e70f705f50e9091c257ee2bb40 | 1 - ...shdown2-9-4fa4ba4c739b6f44975d41e4705d4389 | 1 - ...shdown2-9-c86940e70f705f50e9091c257ee2bb40 | 1 + ...shdown3-0-df2401785dfa257de49c3ad80b0f480a | 2 +- ...shdown3-2-e879051803d0b64139e703e40fb007d0 | 2 +- ...shdown3-4-a1c18483e5f5d2fe351be09af658acbb | 2 +- ...shdown3-6-67118904e6cc8e9f5210fab88e87fb8f | 2 +- ..._header-0-860e298a0b70e7a531431e9386ddc0e7 | 1 - ..._header-1-8540676fc16ac91f3629c40f393a890a | 10 - ..._header-2-5cff10d4b561206e7e0b2e81d862ff93 | 10 - ..._header-3-e86d559aeb84a4cc017a103182c22bfb | 0 ...gress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 | 2 +- ...gress_1-2-70ba81c09588aa586e62ebaee2af685f | 0 ...ress_1-2-b6c8c8fc9df98af4dead5efabf5f162c} | 0 ...gress_1-3-43d286eebddaee26cf95f26e62a75fe4 | 2 +- ...ctmode-19-b6e156f104768706aa587b762a9d4d18 | 8 +- ...ctmode-21-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-23-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-26-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-28-a31442a7d6c6950c137856ab861d622d | 8 +- ...ctmode-32-b6e156f104768706aa587b762a9d4d18 | 8 +- ...ectmode-6-e2d1fc9c94e0d3597f393aec99850510 | 4 +- ...ectmode-8-e2d1fc9c94e0d3597f393aec99850510 | 4 +- ...push_or-5-c94def4b18b9c8c00e7a93eb19ec694a | 2 +- ...th_semi-0-3731ce715b60549c17b7993927d48436 | 2 +- ...th_semi-1-3f53ec3b276b32cf81729433e47010cb | 2 +- ...th_semi-2-bf8cb175f9b13fcc23ba46be674b5767 | 2 +- ...quote2-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 .../quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 | 0 .../quote2-1-34f3c423b2fb1f0b11457f45a60042b9 | 1 - ...quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118} | 0 .../quote2-2-34f3c423b2fb1f0b11457f45a60042b9 | 1 + ...as.attr-0-97b3c408090f758257e4bd20597a525e | 2 +- ...pruner1-0-a7e4414330751eb8ad486bb11643f64d | 0 ...pruner1-1-11cdebc422d7a0e7b257279ac9524321 | 56 - ...pruner2-0-b9598847d77e2c425423f51d755380e8 | 0 ...pruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 | 0 ...pruner2-2-db276de57ad86e8880037336886cd557 | 0 ...pruner2-3-e4419c33287ca1f48a43f61cca5b5928 | 116 - ...bigdata-0-e011be1172043c0c6d0fd2c0e89f361e | 1 - ...bigdata-1-6d0d6f4de136f56ab91987e19df8e178 | 1 - ...bigdata-2-3688b45adbdb190d58799c0b6d601055 | 1 - ...bigdata-3-fc0c054cdfbf5c130532e139d497866a | 0 ...olumnar-2-b2f56374f2ea8a967b38e77b57138d4b | 4 +- ..._merge1-0-d877ca1eefa4344eae30ad3ef2039b00 | 1 - ..._merge1-1-a4fb8359a2179ec70777aad6366071b7 | 1 - ...merge1-10-f94fdb0b79dcf73989e6fbce87355753 | 1 - ...merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 | 0 ...merge1-12-150cb190dc2343a747ea89298eb6352c | 0 ...merge1-13-af7b1e79eb40854ea82d61debaafcf40 | 1 - ..._merge1-14-4547f75ed3cb94914c2d025c1e057b5 | 1 - ...merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ...merge1-16-c198c437f48c3844d8d0ace881b3107e | 0 ..._merge1-2-690b2898f94ef32f01ded0ddb737a056 | 1 - ..._merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee | 1 - ..._merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 | 0 ..._merge1-5-c198c437f48c3844d8d0ace881b3107e | 0 ..._merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 | 0 ..._merge1-7-8c76c8d41133f29366359c308b0c9cc0 | 0 ..._merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f | 0 ..._merge1-9-780b166bd3fcc932cb217273b0636d63 | 0 ..._merge2-0-f94fdb0b79dcf73989e6fbce87355753 | 1 - ..._merge2-1-a4fb8359a2179ec70777aad6366071b7 | 1 - ...merge2-10-5b4fb8038f522877101a4e429f082f11 | 0 ..._merge2-2-690b2898f94ef32f01ded0ddb737a056 | 1 - ..._merge2-3-778b659dba30ece782a956d87b1a31eb | 1 - ..._merge2-4-5b4fb8038f522877101a4e429f082f11 | 0 ..._merge2-5-78b97f09b49452e054cba2ae461813c3 | 0 ..._merge2-6-d5a91ca1ff9039b2dc55af8735d0249a | 0 ..._merge2-7-805fc662f9b3b7a587d1043b659e2424 | 0 ..._merge2-8-afd119f412a66bdf5cd1c74ae8965685 | 1 - ..._merge2-9-351c25ee0ca991ca9e0dbafa873b7592 | 1 - ..._merge3-0-f94fdb0b79dcf73989e6fbce87355753 | 1 - ..._merge3-1-690b2898f94ef32f01ded0ddb737a056 | 1 - ...merge3-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...merge3-11-7674266b00c52a2b6755910ea0793b69 | 1 - ...merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f | 1 - ...merge3-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...merge3-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge3-2-778b659dba30ece782a956d87b1a31eb | 1 - ..._merge3-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ..._merge3-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 | 0 ..._merge3-6-e4d1c54cf6a802eef3342bec2505f69b | 0 ..._merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 ...e_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ..._merge3-9-7c38c42af102069e08a85a27a735eb61 | 0 ..._merge4-0-f94fdb0b79dcf73989e6fbce87355753 | 1 - ..._merge4-1-690b2898f94ef32f01ded0ddb737a056 | 1 - ...merge4-10-1486feb77f13bb9a0ed606fe795ef686 | 0 ...merge4-11-7674266b00c52a2b6755910ea0793b69 | 1 - ...merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f | 1 - ...merge4-13-d57efd24ca36f282a37be5d0bf6452e6 | 0 ...merge4-14-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge4-2-778b659dba30ece782a956d87b1a31eb | 1 - ..._merge4-3-d57efd24ca36f282a37be5d0bf6452e6 | 0 ..._merge4-4-5ccde6b58d36e11d6e3c02a867eced7b | 0 ..._merge4-5-18f216bf1ea33debc2576ac85ac1a449 | 0 ..._merge4-6-3e292c7212dd02d7d6000837e09a42f9 | 0 ..._merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 | 0 ...e_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 | 0 ..._merge4-9-7c38c42af102069e08a85a27a735eb61 | 0 ...l_value-2-e721b8138774fdefca7171d1051841ee | 2 +- ...ll_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 | 2 +- ...uptions-1-a9f718201b71fa855fb81a29e1d7bd3b | 2 +- ...uptions-2-4a9a9175758ef576804c8b7309b019e8 | 2 +- ...uptions-3-ea607fbed28d20e5726f4501285d698d | 2 +- ...uptions-5-6a3af12e36cec853c876a2cbae61c23a | 2 +- ...uptions-6-718032defb864225dd664b1719f3b590 | 2 +- ...d table-0-ce3797dc14a603cba2a5e58c8612de5b | 1 - ...ive_dir-0-fb096f0f4ecc530357ad76ae0353d338 | 0 ...de_join-0-aa047b3a8b40b68b93c4ad11e173c767 | 2 +- ...de_join-1-24ca942f094b14b92086305cc125e833 | 2 +- ...gex_col-0-ac78bd83c9aa538c2827598dd6007a69 | 0 ...gex_col-1-42751bfc3f1e83e7a014db9272d597db | 0 ...egex_col-2-21564f64cdfd46098e1254380490701 | 0 ...gex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 | 0 ...gex_col-4-daf9d3ca22b243870a138ba90d1593c4 | 0 ..._script-0-4e2e94d649b1792c5bd3fd879349ef92 | 1 - ..._script-1-e168f471980470d93b790702a70238fa | 1 - ..._script-2-a19a19272149c732977c37e043910505 | 1 - ...te_script-3-4eb54a664e549614d56ca088c8867d | 0 ..._column-1-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-11-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-13-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-15-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-17-9c36cac1372650b703400c60dd29042c | 2 +- ...column-21-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-23-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-25-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-27-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-29-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ..._column-3-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...column-31-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-33-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ...column-35-6a3bbeb3203ce4df35275dccc4c4e37b | 4 +- ..._column-5-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ..._column-7-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ..._column-9-6a3bbeb3203ce4df35275dccc4c4e37b | 6 +- ...ocation-0-5c73d46fb91e9d4b3dc916622df09290 | 1 - ...oin_ppr-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...oin_ppr-2-49b8b038ed8f5513405071c532967c47 | 2 +- ...oin_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 | 2 +- ...oin_ppr-6-2edf8e4ca99543089fdacbf506caa94f | 2 +- ...oin_ppr-8-b07ad21f47a74162f438abf579675f8e | 2 +- ...sample2-0-13ab74a58da514fe01dbeda0c3e79883 | 0 ...sample2-1-a1d8184eab25b242a961533cc016efd1 | 0 ...de_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...de_hook-1-2b1df88619e34f221d39598b5cd73283 | 1 - ...e_hook-10-60eadbb52f8857830a3034952c631ace | 0 ...e_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 | 0 ...e_hook-12-60018cae9a0476dc6a0ab4264310edb5 | 0 ...de_hook-2-7562d4fee13f3ba935a2e824f86a4224 | 1 - ...de_hook-3-bdb30a5d6887ee4fb089f8676313eafd | 1 - ...de_hook-4-10713b30ecb3c88acdd775bf9628c38c | 1 - ...de_hook-5-bab89dfffa77258e34a595e0e79986e3 | 1 - ...de_hook-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...de_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f | 0 ...de_hook-8-f9dd797f1c90e2108cfee585f443c132 | 0 ...de_hook-9-22fdd8380f2652de2492b34a425d46d7 | 0 ...adoop20-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...adoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 - ...doop20-10-22fdd8380f2652de2492b34a425d46d7 | 0 ...doop20-11-60eadbb52f8857830a3034952c631ace | 0 ...doop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 | 0 ...doop20-13-60018cae9a0476dc6a0ab4264310edb5 | 0 ...adoop20-2-2b1df88619e34f221d39598b5cd73283 | 1 - ...adoop20-3-7562d4fee13f3ba935a2e824f86a4224 | 1 - ...adoop20-4-bdb30a5d6887ee4fb089f8676313eafd | 1 - ...adoop20-5-10713b30ecb3c88acdd775bf9628c38c | 1 - ...adoop20-6-bab89dfffa77258e34a595e0e79986e3 | 1 - ...adoop20-7-6f53d5613262d393d82d159ec5dc16dc | 1 - ...adoop20-8-7a45282169e5a15d70ae0afb9e67ec9a | 0 ...adoop20-9-f9dd797f1c90e2108cfee585f443c132 | 0 ...nv_var1-0-16015162957e1d8e0ef586b44b276f64 | 2 - ...nv_var2-0-e5c4893b2ff412f9df0632328d404cef | 1 - ...nv_var2-1-81cb3e18ab89c533a253defff35e90f0 | 2 - ...pt_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 | 1 - ...pt_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 | 0 ...pt_pipe-2-3bf368261963ee10883e97c7746796b5 | 0 ...pt_pipe-3-afe5db164ccf986c5badd0655e009ea1 | 0 ...pt_pipe-4-7fe60c2fcca928a497252d99436b513f | 1 - ...ptfile1-0-43d53504df013e6b35f81811138a167a | 1 - ...ptfile1-1-89cf592f66b7276add70eb2c7689aa34 | 0 ...ptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 | 1 - ...d table-0-304c4992f5517febd10f43c57df4da49 | 11 + ...omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 | 2 +- ...ote_and-3-683007138a712792041ef3c8b84e914e | 2 +- ...ote_and-5-1a5a68a098bfb9c93b76a458a9faf232 | 2 +- ...ote_not-3-683007138a712792041ef3c8b84e914e | 2 +- ...ote_not-5-ceada0d3fa65bb2ec65154e797332cde | 2 +- ...uote_or-3-683007138a712792041ef3c8b84e914e | 2 +- ...uote_or-5-55cd874f705673f9de6ec8e3643c760f | 2 +- ...e_regex-0-60462d14b99bb445b02800e9fb22760e | 22 - ..._regex-10-c5b3ec90419a40660e5f83736241c429 | 4 +- ...e_regex-2-9d00484beaee46cf72b154a1351aeee9 | 0 ..._regex-2-e84d30fcc6cf11e82b54ea63e7d1d611} | 0 ..._regex-3-3ee9e78ff563d6b48741a41885f92c81} | 0 ...e_regex-3-817190d8871b70611483cd2abe2e55dc | 0 ...e_regex-7-4db287576a17c0963219ca63ee0b20e0 | 22 - ..._regex-7-bf456bcf6be7334488424dfeadf27d75} | 0 ..._regex-8-a4cf34af32b83e40e5c8b6d083938b54} | 0 ...e_regex-8-c429ee76b751e674992f61a29c95af77 | 0 ...e_regex-9-f0e8d394ad18dcbd381792fe9bd8894b | 0 ..._regex-9-f1175f3322abec6f258dd49a5905bce0} | 0 ..._schema-1-e8c6de8cd50be582d78c9a8244cd92a1 | 6 +- ..._schema-3-738e1d72a19c3db37ded87ca2fb148fa | 6 +- ...ble_sub-0-b0b2ec9de0599f8dc2eb160299a2699d | 2 +- ...le_sub-10-bf5d1e710ce0974a67b4084aaa868f67 | 2 +- ...ble_sub-3-266170978f88a14c20c3944bfb55f5c7 | 2 +- ...ble_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 | 2 +- ...ble_sub-5-ee3a31bb9bb742f90daea98b290c34cd | 2 +- ...ble_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b | 2 +- ...ble_sub-8-f321516e93eb0206453465a9b85cf67d | 2 +- ...ble_sub-9-fbb54d457caeaafce723856429bbc0b2 | 2 +- ...e_alter-1-2a91d52719cf4552ebeb867204552a26 | 10 +- ...e_alter-3-2a91d52719cf4552ebeb867204552a26 | 19 +- ...e_alter-5-2a91d52719cf4552ebeb867204552a26 | 19 +- ...e_alter-7-2a91d52719cf4552ebeb867204552a26 | 19 +- ...e_alter-9-2a91d52719cf4552ebeb867204552a26 | 19 +- ...b_table-4-b585371b624cbab2616a49f553a870a0 | 10 +- ...limited-0-97228478b9925f06726ceebb6571bf34 | 0 ...limited-1-2a91d52719cf4552ebeb867204552a26 | 12 +- ...itioned-1-2a91d52719cf4552ebeb867204552a26 | 12 +- ...e_serde-1-2a91d52719cf4552ebeb867204552a26 | 17 +- ...e_serde-4-2a91d52719cf4552ebeb867204552a26 | 10 +- ...le_view-1-1e931ea3fa6065107859ffbb29bb0ed7 | 2 +- ..._quotes-0-65fee14fcf58502241f0772b21096780 | 2 +- ..._quotes-1-26b98b2901556449d5431d731aaa642d | 2 +- ..._quotes-2-f795383fcecedf7266cd7aed8283cec3 | 2 +- ...c_quotes-3-7bc53505a4e6587132870d8d0a704d2 | 2 +- ...nctions-0-45a7762c39f1b0f26f076220e2764043 | 5 +- ...nctions-1-4a6f611305f58bdbafb2fd89ec62d797 | 4 +- ...nctions-2-97cbada21ad9efda7ce9de5891deca7c | 4 +- ...nctions-3-86945c60aed23626c43b507ee4ee6049 | 2 +- ...nctions-4-4deaa213aff83575bbaf859f79bfdd48 | 2 +- ...e_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ...e_cases-1-3f42728fb0083beb962f07c43dd9c9b7 | 0 ..._cases-10-d759a63f08c878456c3401626f253ff5 | 0 ..._cases-11-afe66851d1cdc5d8a8a0d21c5705a59e | 0 ..._cases-12-f2dd38f0a56cd2137c9e1b870271550b | 16 - ..._cases-13-a9224a7a0012e407da67041bf680d490 | 4 - ..._cases-14-9d121385c5ab32d659dc7d0374ae8d6e | 16 - ..._cases-15-b032f4869c43d1278a890328d0591d5d | 0 ..._cases-16-8b2dc53c795584e1e23a8d631c82b43f | 0 ..._cases-17-c93fd07893f47b712165725c78d95555 | 0 ..._cases-18-9acff7f5096cdafc92aa2ddb0f296f83 | 0 ..._cases-19-3f42728fb0083beb962f07c43dd9c9b7 | 0 ...e_cases-2-e90c14f9edaeab8cf4540e9a35b11546 | 0 ...e_cases-3-aa5935155586821fb35e17156c8d8460 | 0 ...e_cases-4-6eb587e2751942de625c9229872ca0dc | 0 ...e_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 | 0 ...e_cases-6-ae97a64481efe733a19007ed400925bc | 0 ...e_cases-7-34016fb262ce768de45ec1b7693fd6c8 | 0 ...e_cases-8-16d39297488db165145e1546c4cb222c | 0 ...e_cases-9-fba02256147a1a753d40f56825449471 | 0 ..._syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ..._syntax-1-fe3da04846e702e0bbe22e05b136a3b3 | 0 ...syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec | 0 ..._syntax-2-8207d7ca034ed62b9fb2c727497c22b3 | 0 ..._syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 | 0 ..._syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 | 16 - ..._syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 | 1 - ..._syntax-6-de64752733e0dcae32e692d2ad38e3d4 | 16 - ..._syntax-7-991839c8e50c5b4e490ec7faf2228d58 | 1 - ..._syntax-8-8c10f994f51bce851ecb0acee65ced7e | 16 - ..._syntax-9-f8385127c6405a2c9e48b5988184b515 | 4 - ...titions-0-73d0fdcea0bd9b828cbc3c2e88acb51a | 2 +- ...itions-1-e69b801a3c6c5f6692050bcdb0e31db9} | 0 ...titions-1-e94d4100cb64c67f1127b4e255d28ae0 | 2 - ...rtitions-2-a1bde7c2c040b4d45ddceac9983c2ca | 2 - ...titions-2-e94d4100cb64c67f1127b4e255d28ae0 | 2 + ...titions-3-9e3f80cb6ed9883c715ea8aa3f391d70 | 1 - ...titions-3-a1bde7c2c040b4d45ddceac9983c2ca} | 0 ...titions-4-9e3f80cb6ed9883c715ea8aa3f391d70 | 1 + ..._tables-0-679cc07f8475a20b49927a5bbbd3d702 | 0 ..._tables-1-ac1c8cca812353544d3f7dead5d033ce | 0 ...tables-10-643b24446d74450c0f83144b1d0ec433 | 0 ...tables-11-3f9a7f993510123059493826470f78f7 | 0 ...tables-12-c31d2f4105ec3714cfc55eef68d3f60c | 3 - ...tables-13-f72d7ab6f3f04161ce2b8f8332244657 | 4 - ...tables-14-26ca9b5509544ebac182d8aa4264ff1c | 3 - ...tables-15-72a95694f749cb3d5760a69083e9cafe | 4 - ...tables-16-dfd802554b6764b3105b6fd8dbb8e173 | 0 ...tables-17-49777c49d2627373ed5e459c4848c9ab | 0 ...tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 | 0 ...tables-19-695a68c82308540eba1d0a04e032cf39 | 0 ..._tables-2-c96604d76bcb3721d5e5a327cac0d5e5 | 18 - ...tables-20-691b4e6664e6d435233ea4e8c3b585d5 | 0 ...tables-21-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...tables-22-274454ebeb7f98690a3c152617a0e391 | 1 - ..._tables-3-a56f51be557c6f636f99fafdbbbbcd9c | 2 - ..._tables-4-743d585ec3da5fbb927581cd0683ae35 | 18 - ..._tables-5-c685b358b604bf3ef980a78d9178d87d | 2 - ..._tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 | 0 ..._tables-7-a62fc229d241303bffb29b34ad125f8c | 0 ..._tables-8-691b4e6664e6d435233ea4e8c3b585d5 | 0 ..._tables-9-64c9bf0618541518f2ba30ec24a94423 | 0 ...estatus-0-4fa957197c8678b0a3a64d8f4f6da1fa | 14 - ...estatus-1-4c31924711bdb64603a14ce57da86ab7 | 14 - ...estatus-2-ecddce523f2af516700677a051581330 | 14 - ...estatus-3-dccfbc8b5a223a9fe47120ca771ee61d | 14 - ...estatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 | 0 ...lestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 | 0 ...estatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 | 0 ...estatus-7-f7b9148c16045269344c5d74fb8a449c | 0 ...estatus-8-1cd5acb4091d916e5e18948a39979b51 | 14 - ...estatus-9-e3cc1823000abb51d2635e0c824e81a9 | 14 - ...perties-4-6c63215ea599f6533666c4d70606b139 | 12 +- ...owparts-0-593619bb962b318b82896658deaea1f1 | 17 - ...owparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a | 4 - ...emove_1-0-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...emove_1-1-114600d46ae86edcb66a500b4cac657d | 1 - ...move_1-10-fa00cf008a039908eec64ad5dd415c5b | 0 ...move_1-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...move_1-12-dd683e148baed6b27642eebacaa87a4f | 0 ...move_1-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_1-2-cafed8ca348b243372b9114910be1557 | 1 - ...remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_1-4-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...emove_1-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_1-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_1-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_1-8-daf10744f465e055b35809a528135370 | 0 ...emove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...emove_2-0-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...emove_2-1-114600d46ae86edcb66a500b4cac657d | 1 - ...move_2-10-bebf0a312f3110d0b518153543030f06 | 0 ...move_2-11-4abc4f450a58ccdd0df2e345f1276979 | 0 ...move_2-12-fa53198d9eecb9d274b09d4351b9274e | 0 ...move_2-13-3fda17e4414d191f837631438a19e700 | 0 ...move_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 | 0 ...move_2-15-43d53504df013e6b35f81811138a167a | 1 - ...emove_2-2-cafed8ca348b243372b9114910be1557 | 1 - ...remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_2-4-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...emove_2-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_2-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_2-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_2-8-c64266431d312784ebc7b9ca07ab5188 | 0 ...emove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt1-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt1-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...oinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt1-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...oinopt1-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt1-7-f66c3935651d3cc5fef7d0284e123614 | 0 ...inopt10-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt10-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt10-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 | 0 ...inopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt10-5-9abe9cb11e3336a689a4116f8804c02a | 0 ...inopt10-6-bc7008e74f5eccab48d820877d83e7e4 | 0 ...inopt10-7-f4015c993efd5fc155e2faec784718d0 | 0 ...inopt10-8-c9624d1650d395f18d9e510cab4fab79 | 0 ...inopt11-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt11-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt11-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt11-3-a079ede4f4245e62b02f624efedfb597 | 0 ...inopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt11-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt11-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt11-7-9e7e6016590d33c617cb568cbd45ef68 | 0 ...inopt12-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt12-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt12-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt12-3-da45999e75a09b27161af2c7414c1170 | 0 ...inopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt12-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 ...inopt12-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt12-7-8bbc680be8a68053db008789f335c2f0 | 0 ...inopt13-0-cafed8ca348b243372b9114910be1557 | 2 +- ...inopt13-1-16a1f74642d7ea4dac66a5ce15469c22 | 2 +- ...nopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 | 2 +- ...inopt13-2-114600d46ae86edcb66a500b4cac657d | 2 +- ...inopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...nopt13-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...inopt13-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...nopt13-6-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...inopt13-8-3fda17e4414d191f837631438a19e700 | 0 ...nopt13-8-8eb53fb8f05a43ee377aa1c927857e7c} | 0 ...inopt14-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt14-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt14-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt14-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...inopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt14-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt14-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt14-7-c329b937ad1d7cf1c838640ef5f4d135 | 0 ...inopt14-8-3fda17e4414d191f837631438a19e700 | 0 ...inopt14-9-cdf19a17f3295447b66e6e6445742a74 | 0 ...inopt15-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt15-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...nopt15-10-7df9fe6075a96bf9849848c93e449469 | 0 ...nopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 | 0 ...inopt15-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt15-3-32fed3a53e7c15b549a71c0e71d93484 | 0 ...inopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...inopt15-5-dff9d122eb83760f08d1d77814c24c91 | 0 ...inopt15-6-717b85f496a5cf006cb352f9d884608d | 0 ...oinopt15-7-ba43a86694107dd4cb754d676935408 | 0 ...inopt15-8-7381c1f36c997afac91d8f8f29e865f3 | 0 ...inopt15-9-ccb54093d518eaca1e4644686e9e776e | 0 ...inopt16-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt16-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt16-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt16-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...inopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt16-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...inopt16-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt16-7-b3208400a48223a410b76a4bd1362da5 | 0 ...inopt17-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt17-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt17-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt17-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...inopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 | 0 ...inopt17-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt17-7-373b8a6363139ca37395b1cc8c9808d9 | 0 ...inopt18-0-cafed8ca348b243372b9114910be1557 | 2 +- ...inopt18-1-16a1f74642d7ea4dac66a5ce15469c22 | 2 +- ...nopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa | 2 +- ...inopt18-2-114600d46ae86edcb66a500b4cac657d | 2 +- ...inopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 | 0 ...nopt18-4-abf4b7f158999af331d5dbfddf32fa68} | 0 ...inopt18-8-4abc4f450a58ccdd0df2e345f1276979 | 0 ...nopt18-8-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...inopt19-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt19-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt19-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt19-3-6eda8acf464a18cfd9909255ddcef37e | 0 ...inopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt19-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt19-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt19-7-1e9c17669089eea559f8fa0b6977b249 | 0 ...oinopt2-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt2-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt2-3-52247b4dd98092bf829254e17424657d | 0 ...oinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt2-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt2-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 | 0 ...inopt20-0-cafed8ca348b243372b9114910be1557 | 1 - ...inopt20-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...inopt20-2-114600d46ae86edcb66a500b4cac657d | 1 - ...inopt20-3-4420d88e35aa84327bc95153eed299e0 | 0 ...inopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt20-5-744a018c78bae6e09853dd202981e850 | 0 ...inopt20-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt20-7-e209254ae404366e6adca673d666aecb | 0 ...oinopt3-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt3-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt3-3-c64266431d312784ebc7b9ca07ab5188 | 0 ...oinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt3-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt3-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt3-7-e54bbab48fcc3c41806a101293577e9f | 0 ...oinopt4-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt4-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 | 0 ...oinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt4-5-744a018c78bae6e09853dd202981e850 | 0 ...oinopt4-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt4-7-c7e2ccaba57a581f27cfdcca76891133 | 0 ...oinopt5-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt5-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt5-3-25f94adcba19b899d1db3af93ea1c95b | 0 ...oinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt5-5-fa00cf008a039908eec64ad5dd415c5b | 0 ...oinopt5-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt5-7-f38878761b2aeeee0c04387dff60894d | 0 ...oinopt6-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt6-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt6-3-da45999e75a09b27161af2c7414c1170 | 0 ...oinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd | 0 ...oinopt6-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 | 0 ...oinopt7-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt7-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt7-3-c64266431d312784ebc7b9ca07ab5188 | 0 ...oinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt7-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt7-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt7-7-fa53198d9eecb9d274b09d4351b9274e | 0 ...oinopt7-8-3fda17e4414d191f837631438a19e700 | 0 ...joinopt7-9-b54d2a1f5d3bea81680ab06dead952c | 0 ...oinopt8-0-cafed8ca348b243372b9114910be1557 | 1 - ...oinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 | 1 - ...oinopt8-2-114600d46ae86edcb66a500b4cac657d | 1 - ...oinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 | 0 ...oinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...oinopt8-5-bebf0a312f3110d0b518153543030f06 | 0 ...oinopt8-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...oinopt8-7-fa53198d9eecb9d274b09d4351b9274e | 0 ...oinopt8-8-3fda17e4414d191f837631438a19e700 | 0 ...oinopt8-9-9b26e8e33d3109e059e7494b53aee6fd | 0 ...oinopt9-0-cafed8ca348b243372b9114910be1557 | 2 +- ...oinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 | 2 +- ...inopt9-10-5c8be465ceef8151f172b82a13e81fa8 | 2 +- ...oinopt9-2-114600d46ae86edcb66a500b4cac657d | 2 +- ...oinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b | 0 ...inopt9-4-c0f14def6a135cc50cba364e810ce28e} | 0 ...oinopt9-6-4abc4f450a58ccdd0df2e345f1276979 | 0 ...inopt9-6-ade68a23d7b1a4f328623bb5a0f07488} | 0 ...oinopt9-8-446c5e33062b109341add64a9860207d | 2 +- ...rtesian-0-e3c7f62795b0e7c14b41b0dc29c47619 | 2 +- ...join_1-10-b1114520feaa15ad7621b6a0e571c244 | 2 +- ...join_1-12-8fb6fea440e845ce23b06deed8f519fa | 2 +- ...join_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 | 2 +- ...join_1-16-c120e505c143721a36287bf992dbc1a1 | 2 +- ...join_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 | 2 +- ...join_1-20-d7979e9ab355d8856c2d12e66e7bb838 | 2 +- ...join_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b | 2 +- ...join_1-24-558e9ab6324f8082231b2fdd0e12f7ae | 2 +- ...pjoin_1-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_1-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_1-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_1-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_10-3-68d65d622e45f86d4a6c7d1d09ef823b} | 0 ...join_10-3-bfb76fa2eccda1c64a85ea3841202705 | 0 ...oin_10-4-d31ad2289181131982ef3e9cd8c6386e} | 0 ...join_10-4-d4746bf376bce0bf561450c75b83fb74 | 0 ...join_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 | 0 ...oin_10-5-f7fb003fa65cadcd0b13cbdd7b355988} | 0 ...join_10-6-1094dbf800159e1e2382ec238b2466d7 | 0 ...oin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59} | 0 ...join_10-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_10-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_10-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_11-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_11-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_11-10-3d92573eecd22525a75464ad27b1dcaa | 0 ...oin_11-11-685ec4fbbf73330e026fba9b8cc53e92 | 1 - ...oin_11-12-c05c09243793df14546e8577ee369d58 | 1 - ...oin_11-13-1e0f21b664c1940d10531b3025be7f10 | 0 ...oin_11-14-f13be826d8f11be64c5414d591f70fd6 | 0 ...oin_11-15-b62714cb184523454314d19949dba9f4 | 0 ...oin_11-16-bee943a85cd82bd199b089fbdd7450af | 1 - ...join_11-2-b89ea2173180c8ae423d856f943e061f | 1 - ...join_11-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...join_11-4-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_11-5-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_11-6-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_11-7-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_11-8-beae7266b997c97798631b9dc558534f | 0 ...join_11-9-74936eafc274242beb49bc83d7a8af30 | 0 ...join_12-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_12-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_12-10-bc8140f238cfd13ea4fc4c4173a05454 | 0 ...oin_12-11-685ec4fbbf73330e026fba9b8cc53e92 | 1 - ...oin_12-12-c05c09243793df14546e8577ee369d58 | 1 - ...join_12-13-7e7645d5ee7d79991d8fdde072c8dbb | 0 ...oin_12-14-e9715c66355e9bc54155c79a4e82b34f | 0 ...oin_12-15-42b623410c408e09153a773db91c0334 | 0 ...oin_12-16-bee943a85cd82bd199b089fbdd7450af | 1 - ...oin_12-17-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...oin_12-18-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_12-19-b89ea2173180c8ae423d856f943e061f | 1 - ...join_12-2-b89ea2173180c8ae423d856f943e061f | 1 - ...oin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b | 0 ...oin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e | 0 ...oin_12-22-b1e1754efd667aa801b194985d41eb6e | 1 - ...join_12-3-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...join_12-4-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_12-5-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_12-6-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_12-7-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_12-8-3d016b9a5b0143b7a01b34472b569fb9 | 0 ...join_12-9-74936eafc274242beb49bc83d7a8af30 | 0 ...join_13-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_13-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_13-14-b92cb43f66838319f5d607c431fe1eb3 | 2 +- ...join_13-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_13-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_13-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_13-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_13-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_13-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_14-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...oin_14-11-23d2ee09b01309b9cfcd0d625afc535d | 2 +- ...oin_14-13-95a09a0af2a77ac6b772c41a0f6a885a | 2 +- ...oin_14-15-e0f20915e07acf5ddfdbde40ef924e55 | 2 +- ...oin_14-17-5983d1e12e5a2bdd0f41549110e066ee | 2 +- ...oin_14-19-163af33279f8d08d747a00ffd1cdfac7 | 2 +- ...pjoin_14-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...oin_14-21-b6c3a3f68f212a966450286e23f59713 | 2 +- ...oin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 | 2 +- ...join_14-25-23f011143d8db18dd0f3d684adf7b8e | 2 +- ...oin_14-27-ba437062dd6661fc2fdcd41514711093 | 2 +- ...oin_14-29-d191c9ace84072ef430d7ed36ea8181b | 2 +- ...oin_14-31-fde2ad19052435bd8c155f18fe579070 | 2 +- ...oin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 | 2 +- ...join_14-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_14-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_14-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_15-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_15-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a | 2 +- ...oin_15-19-f49ef85423bb1766362f05651c9bb37f | 2 +- ...join_15-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_15-21-af3880637379684acd440830c2361f6e | 2 +- ...oin_15-23-e5e54dd17b69773812af376bfec33200 | 2 +- ...join_15-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_15-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_15-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_15-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_15-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_16-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_16-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...oin_16-12-32f3716e22149e3d0c1f3ac26d414baf | 2 +- ...join_16-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_16-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_16-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_16-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_16-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_16-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_17-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_17-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_17-2-b89ea2173180c8ae423d856f943e061f | 2 +- ...oin_17-25-f066907fca3448b27aab623d05258a9a | 2 +- ...oin_17-26-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_17-27-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_17-29-65d3ae14a785f319658812b51e4620a3 | 2 +- ...join_17-3-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...oin_17-31-783fc1b07f117cd027395cf1c09149bc | 2 +- ...join_17-4-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_17-5-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_17-6-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_17-7-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_18-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_18-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...oin_18-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...oin_18-12-27762aa3d846e8f50d97350eaa7563a1 | 1 - ...oin_18-13-91f2af0da94e5a83601d02045980f556 | 1 - ...oin_18-14-1d0b88d6e6f84e485a05c712dd185531 | 1 - ...oin_18-15-21a6e09c24697160bf70cb17254ff504 | 1 - ...oin_18-16-35dfd037075aac5a9891bf99ea01e156 | 1 - ...oin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 - ...oin_18-18-ba77d1d4a1754489e2a8d5006b0af54d | 1 - ...oin_18-19-da6eb8ded1f72f518617339f58202cc5 | 1 - ...join_18-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_18-20-940ff79d8f3b401a05b19b9417824d7a | 1 - ...oin_18-21-7ce17310f9fd362e2cc8a80211063264 | 1 - ...oin_18-22-a92f50cba6740abb3596c885a8157861 | 0 ...oin_18-23-f0675c53cddf75b012f64e797f5824c8 | 0 ...oin_18-24-a919b505c34a237ead34eea40b7d136c | 1 - ...join_18-25-d014ae435859316a1ad43548b72ecb7 | 1 - ...oin_18-26-52d3bbbbef7c580a324d991f52f44e1f | 1 - ...oin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 | 1 - ...oin_18-28-c83e75a3e18e68ef10d9970b3b8857ab | 1 - ...oin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 | 0 ...join_18-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...oin_18-30-76e467313085467a3aa929b3665f9863 | 0 ...oin_18-31-7f1e9114f4db63982985068c4bf36a29 | 1 - ...oin_18-32-b59c406dae57fa6ab656a27e310ac54c | 1 - ...oin_18-33-95b42e61bf5eed67123e30d482e0c7fe | 1 - ...oin_18-34-14ae716c8c0f47db61b89a2a17e89415 | 1 - ...oin_18-35-549b3df453bf939e1c0a38df861245e5 | 1 - ...pjoin_18-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_18-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_18-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_18-7-c248759cecf0e2c223579f5e37f6669c | 0 ...join_18-8-724d37bd4a841f1fa4062f4f3e3eb353 | 0 ...join_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_19-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_19-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e | 0 ...oin_19-11-fb15bd6eceb333302535c0bcdd2d078f | 0 ...oin_19-12-27762aa3d846e8f50d97350eaa7563a1 | 1 - ...oin_19-13-4876f6e3c0ffde24afd82ec462962f19 | 1 - ...oin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 | 1 - ...oin_19-15-44ad799a82d847054f02d37139cc6aab | 1 - ...oin_19-16-8ee972ce0d73f28e98f46361082c83dc | 1 - ...oin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec | 1 - ...oin_19-18-b2546caccb501fca356930e6cd00ea2e | 1 - ...oin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 - ...join_19-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_19-20-bceffa38b796fbc2a85daac23357da7b | 1 - ...oin_19-21-a1f4332461d1810334e7ae6d7d43f829 | 1 - ...oin_19-22-ebd323090f771227039cd21b1b8c4f3f | 1 - ...oin_19-23-90f0bad0606e63e4405eac9ec8c11deb | 1 - ...oin_19-24-3ae0277bb7a74cd6bee704313dc102e6 | 1 - ...oin_19-25-e621350131d50867015e75677cca031f | 1 - ...join_19-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_19-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_19-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_19-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_19-7-64b3fd0c215f4b8187866fa7eb55d34d | 0 ...join_19-8-ee04de92100063f9b2063baddb204864 | 0 ...join_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_2-10-1530d7565a22ace89ed16e0e1f9988ac | 2 +- ...join_2-12-48e7d8fcb2a4c31c5304045517a3bb89 | 2 +- ...join_2-14-3e7d435c7a5560abe539918adc4fa922 | 2 +- ...join_2-16-daeabb554f29b77f3c5ff7acff8c58ca | 2 +- ...join_2-18-411bc909c1c29811d184d1f4aceb25b3 | 2 +- ...join_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b | 2 +- ...pjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 | 2 +- ...join_2-24-709966d157a75ffb1c6ef70d7c72a498 | 2 +- ...pjoin_2-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_2-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_2-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_2-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_20-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_20-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 | 0 ...oin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 | 0 ...oin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 | 1 - ...oin_20-13-ba77d1d4a1754489e2a8d5006b0af54d | 1 - ...oin_20-14-da6eb8ded1f72f518617339f58202cc5 | 1 - ...oin_20-15-5acb0ec2e1abcc6d57de7529f414a75a | 0 ...oin_20-16-5b9583aecebb7480d778dc2a8605434a | 0 ...oin_20-17-e26f212ca34d303036299ba709f65522 | 0 ...oin_20-18-7fb8778a58cc27dc04d5947df15b250e | 1 - ...oin_20-19-6dd859e98c140df728f858a7a7598462 | 1 - ...join_20-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_20-20-fca1f7361c63f0ba1e9d365cf0743845 | 1 - ...oin_20-21-28d04e54c0a986079ae9d52788846519 | 1 - ...oin_20-22-1baa93dd15ab33240255c5d6d5d57366 | 1 - ...oin_20-23-5c5eaf9922e7110c6d7260c738b17457 | 0 ...join_20-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_20-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_20-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_20-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...pjoin_20-7-c9981ec081cbf54c9323e0dee977934 | 0 ...join_20-8-5c9994d48509136efd1dcb4e3f3e7aff | 0 ...join_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec | 0 ...join_21-0-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_21-1-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_21-2-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_21-3-365488a703b0640acda73a7d7e6efa06 | 2 +- ...pjoin_21-4-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_21-5-dc129f70e75cd575ce8c0de288884523 | 2 +- ...join_21-6-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...join_22-0-3b0f76816be2c1b18a2058027a19bc9f | 1 - ...join_22-1-86473a0498e4361e4db0b4a22f2e8571 | 1 - ...oin_22-10-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...oin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...oin_22-12-187c201f61c23833d0d193031926445a | 1 - ...oin_22-13-5c5c521954e5ade621fd0cbff5794c80 | 1 - ...join_22-14-b2534937758d2ff0a08e729f7b3ace4 | 1 - ...oin_22-15-83d9e3d23d6612a926e53d57a5d07307 | 1 - ...oin_22-16-abc95b36345d3e37abb196088897c7fe | 0 ...oin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b | 0 ...oin_22-18-2c29fc18b24368938f880a1bf3d5eb54 | 0 ...oin_22-19-5dc91a74bf4c43e20a8a213ad08c352e | 0 ...join_22-2-7b4ad215fc2e75c71c6614a2b6322e8e | 1 - ...oin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...oin_22-21-f0def0d77c93f6faebeca9b7a5340dbc | 0 ...oin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc | 0 ...oin_22-23-187c201f61c23833d0d193031926445a | 1 - ...oin_22-24-5c5c521954e5ade621fd0cbff5794c80 | 1 - ...join_22-25-b2534937758d2ff0a08e729f7b3ace4 | 1 - ...oin_22-26-83d9e3d23d6612a926e53d57a5d07307 | 1 - ...join_22-3-365488a703b0640acda73a7d7e6efa06 | 1 - ...pjoin_22-4-7cccbdffc32975f8935eeba14a28147 | 1 - ...join_22-5-dc129f70e75cd575ce8c0de288884523 | 1 - ...join_22-6-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...join_22-7-8317b719ffcf85da8c618e8f2379a31b | 0 ...join_22-8-8cfa26d100b454c0b0f3443f62389abb | 0 ...join_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad | 0 ...join_25-0-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ...join_25-1-365488a703b0640acda73a7d7e6efa06 | 2 +- ...oin_25-10-bd7036a4c0b57349a588b974ffaa502} | 0 ...oin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...in_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...oin_25-11-3af09654f8d38d21b5d26e6cc21210de | 0 ...join_25-12-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...in_25-12-6d835f651b099615df163be284e833de} | 0 ...oin_25-14-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ...join_25-15-442e69416faaea9309bb8c2a3eb73ef | 2 +- ...oin_25-16-beaea10543cfd753458b43d8aeb7571f | 2 +- ...oin_25-17-24ca942f094b14b92086305cc125e833 | 2 +- ...oin_25-18-d0ec6d66ff349db09fd455eec149efdb | 2 +- ...oin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 | 2 +- ...join_25-2-16367c381d4b189b3640c92511244bfe | 2 +- ...oin_25-20-44d382ce6848d3f0b900b0808747d8e9 | 2 +- ...oin_25-21-4d3e60a0e9bd8c12fdba4e010493537d | 2 +- ...oin_25-22-d0a93f40892e3894460553b443c77428 | 2 +- ...join_25-23-ae45f6382268c1035c11aa4fc8a23e2 | 2 +- ...oin_25-24-d5ad76f9178cb787cee037f25b19b270 | 2 +- ...join_25-3-bcde511b8d560ca83888265b96a781ff | 2 +- ...join_25-4-8067b7ab260021bc70af9ab47309ee35 | 2 +- ...join_25-5-545c0ea2ebd7a141526ee13059a857f6 | 2 +- ...join_25-6-dbcec232623048c7748b708123e18bf0 | 2 +- ...pjoin_3-10-efadb45f09b92d27233601394d53d79 | 2 +- ...join_3-12-77988b41de4b5a165c93942fbb2220d6 | 2 +- ...join_3-14-5456a3a43bfabcfdbb57257b633e299e | 2 +- ...join_3-16-fff1acc77150f6ea92fe8eefc04b079a | 2 +- ...join_3-18-96a658e934543dd77020ad264ec9b8c1 | 2 +- ...join_3-20-867845ed2cb38b55928f5310b4ae91bd | 2 +- ...join_3-22-5826f60da3a4b0c731c53340d50b0a06 | 2 +- ...join_3-24-c73176ea199d4d20934cf2659d42ec25 | 2 +- ...pjoin_3-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_3-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_3-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_3-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_4-14-a6efb153fe9be91c7055a5642e0f642d | 2 +- ...join_4-16-514034935af5348f67f1934cf5429d57 | 2 +- ...apjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 | 2 +- ...join_4-20-be9350a00a523e1758427a1c8e4bdf09 | 2 +- ...join_4-22-da023084bbab6727dc3823cfce500308 | 2 +- ...join_4-24-7783ab33a26d29a2c0235f52f0e4ad86 | 2 +- ...join_4-26-de14aa74d0da48dff6271410cc5dd98e | 2 +- ...pjoin_4-28-f71280b4b556515d068d074378c3a54 | 2 +- ...pjoin_4-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_4-30-a0d3abda611809460bcea4101967f21f | 2 +- ...join_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 | 2 +- ...pjoin_4-34-48869ba51ea36864e720f94b09bf6b3 | 2 +- ...join_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_4-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_4-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_4-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_5-14-e20bcd28cfd26259bcde9cffec8d2280 | 2 +- ...join_5-16-8a8dc418de3716f5112b9868f472e4b9 | 2 +- ...join_5-18-1f0dd26e6088c4f359fb691a8ef650bc | 2 +- ...join_5-20-18fe4e120d3c0d663c360b2504b4f88d | 2 +- ...join_5-22-6ed2620fe017e454459a83061f25951a | 2 +- ...join_5-24-87f758a20c15fa3b97b4ba6703ae541b | 2 +- ...join_5-26-1c1a9519e207edc3c3a927c986a37177 | 2 +- ...join_5-28-94440444fff7d2629a23a30fd778fcc7 | 2 +- ...pjoin_5-3-bd7036a4c0b57349a588b974ffaa502} | 0 ...pjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 | 0 ...join_5-30-574d26a5179e5cebbbff5bbb425a9609 | 2 +- ...pjoin_5-32-17c99f827824994cfd21c40dbf4abfc | 2 +- ...join_5-34-4db871582cf4f3038d43d0a2d5ae6895 | 2 +- ...join_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423} | 0 ...pjoin_5-4-3af09654f8d38d21b5d26e6cc21210de | 0 ...apjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 | 0 ...join_5-5-6d835f651b099615df163be284e833de} | 0 ...pjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...pjoin_5-8-b89ea2173180c8ae423d856f943e061f | 2 +- ...pjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...pjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_6-10-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_6-11-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_6-14-43123d2821871aa5b4d4a9e22e03d1ca | 2 +- ...join_6-16-449cb60e6537ba0810ea6879a7351a1e | 2 +- ...join_6-17-aa5f237005fb69b3f52808c80b4a276a | 2 +- ...apjoin_6-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_6-21-43123d2821871aa5b4d4a9e22e03d1ca | 2 +- ...join_6-23-449cb60e6537ba0810ea6879a7351a1e | 2 +- ...join_6-24-aa5f237005fb69b3f52808c80b4a276a | 2 +- ...pjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...pjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...pjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...join_7-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_7-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_7-13-b89ea2173180c8ae423d856f943e061f | 2 +- ...join_7-16-18d2a689883588ae3c24b2810663ab80 | 2 +- ...join_7-19-ab46164010b27950c293f32fb89f9f8a | 2 +- ...apjoin_7-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ...join_7-21-449cb60e6537ba0810ea6879a7351a1e | 2 +- ...join_7-22-aa5f237005fb69b3f52808c80b4a276a | 2 +- ...pjoin_7-23-1b6140f49d49eb1195c8551270b97ff | 2 +- ...pjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb | 0 ...pjoin_7-8-f983875c44b290b0884a22b6be6adc8} | 0 ...pjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f | 0 ...join_7-9-84a394d962965e38593883742cc32c0d} | 0 ...pjoin_8-0-43d53504df013e6b35f81811138a167a | 2 +- ...pjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 | 2 +- ...join_8-11-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ...join_8-12-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...join_8-13-b89ea2173180c8ae423d856f943e061f | 2 +- ...pjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ...apjoin_8-3-7cccbdffc32975f8935eeba14a28147 | 2 +- ...pjoin_8-5-a1f5562186e9e22e18ebd42208943525 | 0 ...join_8-5-eee18fc4192a4aa92a066eb66513be93} | 0 .../sort-1-10c8b0a592ed15eff488a71ec5909f45 | 2 +- ...desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 | 2 +- ..._desc_1-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_1-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_1-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_1-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...desc_2-11-47bc9cb5f049aaca33b394ea78578bdd | 2 +- ..._desc_2-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_2-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_2-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_2-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...desc_3-11-47bc9cb5f049aaca33b394ea78578bdd | 2 +- ..._desc_3-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_3-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_3-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_3-9-b89ea2173180c8ae423d856f943e061f | 2 +- ...desc_4-11-47bc9cb5f049aaca33b394ea78578bdd | 2 +- ..._desc_4-2-ffe97dc8c1df3195982e38263fbe8717 | 2 +- ..._desc_4-7-3b0f76816be2c1b18a2058027a19bc9f | 2 +- ..._desc_4-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_4-9-b89ea2173180c8ae423d856f943e061f | 2 +- ..._desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ..._desc_5-1-365488a703b0640acda73a7d7e6efa06 | 2 +- ...desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a | 2 +- ..._desc_5-7-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ..._desc_5-8-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ..._desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e | 2 +- ..._desc_6-1-365488a703b0640acda73a7d7e6efa06 | 2 +- ...desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a | 2 +- ...n_desc_6-2-7cccbdffc32975f8935eeba14a28147 | 2 +- ..._desc_6-8-c23ea191ee4d60c0a6252ce763b1beed | 2 +- ..._desc_6-9-86473a0498e4361e4db0b4a22f2e8571 | 2 +- ...stats0-14-4f18f4b06db06844920b14e2d19471a9 | 10 +- ...stats0-15-4a7ed5b7e3deb4106204f8f950357e1c | 10 +- ...stats0-23-247568f4b3ce6b97979ca9d28c2ba05c | 6 +- ...stats0-30-4f18f4b06db06844920b14e2d19471a9 | 10 +- ...stats0-31-4a7ed5b7e3deb4106204f8f950357e1c | 10 +- .../stats0-7-247568f4b3ce6b97979ca9d28c2ba05c | 6 +- .../stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats2-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e | 0 ...stats2-11-912c785dbcef3251dd1d6b7fc56eae5b | 39 - .../stats2-2-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats2-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats2-4-dc129f70e75cd575ce8c0de288884523 | 1 - .../stats2-5-6717e1026e24a38af19b6bce1951e3d7 | 0 .../stats2-6-45d4fb785fc94d266096fc29a9e90d73 | 0 .../stats2-7-6436db7a7506b560d0d1759db94a76b9 | 0 .../stats2-8-72621dba638b15d244850018e9f64d7 | 34 - .../stats2-9-6d93732dc2ca622eb60c171389caee8e | 19 - ...stats20-0-418ec894d08c33fd712eb358f579b7a0 | 1 - ...stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...stats20-2-4711e55648c25c86bb526ed50b0c3d09 | 1 - ...stats20-3-98c925a2b2c4de06e270e1b52437a98b | 0 ...stats20-4-82294461be4728b4b191414bf2fb3bd7 | 0 ...stats20-5-76509775cfe11bb98ee088188a07668a | 37 - ...stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 | 1 - ...stats20-7-82294461be4728b4b191414bf2fb3bd7 | 0 ...stats20-8-300c971de74642118d36d36349bc81aa | 37 - .../stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats3-1-418ec894d08c33fd712eb358f579b7a0 | 1 - ...stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 | 6 - ...stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 1 - .../stats3-12-892cb7ecc26e84f1c033b95a3ee3edc | 0 .../stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa | 1 - ...stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 | 6 - .../stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 | 0 ...stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 | 0 ...stats3-18-eefbb3ee8b538aec85c609351e52901b | 38 - ...stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 | 0 .../stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 | 0 ...stats3-20-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-3-ca048ad81b3df7159822073d206f0790 | 0 .../stats3-4-fa705a031ff5d97558f29c2b5b9de282 | 0 .../stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 | 43 - .../stats3-6-4bf1504274319c44d370b58092fe016c | 0 .../stats3-7-73d7d55d6e5a57aacce8618902904d | 32 - .../stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 | 0 .../stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 | 0 .../stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats4-1-418ec894d08c33fd712eb358f579b7a0 | 1 - ...stats4-10-a33b2c9d962e4921c98e62387f3989f7 | 0 ...stats4-12-30bc31441828a053d1a675b225a5d617 | 2 - ...stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 | 1000 ------- ...stats4-14-9c82167763a771c175c656786d545798 | 1000 ------- ...stats4-15-f02b95f20b526fcf2850b07ca6be4f8c | 36 - ...stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 | 36 - ...stats4-17-746b888d14c1b3fa28aa8549c174f6d9 | 36 - ...stats4-18-dbe13731de4ab2a3c23343b78525b2f7 | 36 - ...stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 | 39 - ...stats4-20-f63000f2c395b935199c9829964f98c1 | 39 - ...stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 ...stats4-22-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd | 0 .../stats4-4-f709d5c1d4c9f13b7a521867674ac54c | 0 .../stats4-5-cd849c4fe1229428da98947e3e43b46d | 0 .../stats4-6-9c0d4354b6a9be351fa32a66ff58a177 | 0 .../stats4-7-16367c381d4b189b3640c92511244bfe | 1 - .../stats4-8-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats4-9-255ad4511130fb8c9ee9d65b7c95743f | 0 .../stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats5-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - .../stats5-2-6f5d06d6100de19ec398891cb2eae161 | 0 .../stats5-3-96d9aa9c32a081518604959dcfac42df | 19 - .../stats5-4-dbf81a12f6c19c14dce831e942870744 | 0 .../stats5-5-3980dfc2e4f882dd3bf478e56929361 | 33 - .../stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats6-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...stats6-10-653f748fa2d690502ba4fda407841a20 | 32 - ...stats6-11-1c9f833953403596ad50fd32e513642c | 32 - ...stats6-12-fdbe263d925f28d598a71b7a4c577492 | 39 - .../stats6-2-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats6-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats6-4-a88c476a632cd92f748967fadb242405 | 0 .../stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats6-6-8926095434b70c83bf88c70559d38dce | 0 .../stats6-7-6615049191cfa4026a3a870c3c3749f4 | 0 .../stats6-8-e15e242124e61ede9196130cb3fb69e7 | 36 - .../stats6-9-e6b884de17a29eb476fd6cc502fc615d | 36 - .../stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats7-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...stats7-10-fdbe263d925f28d598a71b7a4c577492 | 38 - .../stats7-2-a4fb8359a2179ec70777aad6366071b7 | 1 - .../stats7-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats7-4-a88c476a632cd92f748967fadb242405 | 0 .../stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats7-6-5d0c6aa78c9786d904728ff7adab85f2 | 18 - .../stats7-7-4912a8bbc138ab97ac0983bc90951de4 | 0 .../stats7-8-e15e242124e61ede9196130cb3fb69e7 | 36 - .../stats7-9-e6b884de17a29eb476fd6cc502fc615d | 36 - .../stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats8-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - .../stats8-10-ce78d029b7764edce3a26336cfac6a8 | 19 - ...stats8-11-6615049191cfa4026a3a870c3c3749f4 | 0 ...stats8-12-e6b884de17a29eb476fd6cc502fc615d | 36 - ...stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d | 19 - ...stats8-14-ea9afc1343991ed4d410231803a174f7 | 0 ...stats8-15-653f748fa2d690502ba4fda407841a20 | 36 - ...stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 | 19 - ...stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 | 0 ...stats8-18-1c9f833953403596ad50fd32e513642c | 36 - ...stats8-19-dbf72430cff3df0e6e9405ad64531b16 | 19 - .../stats8-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...stats8-20-bff592750d1057448d2cff7694b6dad2 | 0 ...stats8-21-e15e242124e61ede9196130cb3fb69e7 | 36 - ...stats8-22-e6b884de17a29eb476fd6cc502fc615d | 36 - ...stats8-23-653f748fa2d690502ba4fda407841a20 | 36 - ...stats8-24-1c9f833953403596ad50fd32e513642c | 36 - ...stats8-25-fdbe263d925f28d598a71b7a4c577492 | 39 - .../stats8-3-16367c381d4b189b3640c92511244bfe | 1 - .../stats8-4-a88c476a632cd92f748967fadb242405 | 0 .../stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 .../stats8-6-821e1f061960081b9b379d9bfb47f267 | 19 - .../stats8-7-8926095434b70c83bf88c70559d38dce | 0 .../stats8-8-e15e242124e61ede9196130cb3fb69e7 | 36 - .../stats8-9-fdbe263d925f28d598a71b7a4c577492 | 39 - .../stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - .../stats9-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...r_error_1-0-6d1832d28f897d0049de053617bd36 | 0 ...rror_1-0-6f3df708fa339df236ec9375d2ad37fa} | 0 ..._error_1-1-36eee5cbac5c0c3228e499805b32f6} | 0 ...error_1-1-887fe99770f53e7e0a0fbdc190118612 | 1 - ...rror_1-10-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ...error_1-11-d58626190cded8d09f0457739a980eb | 2 +- ...rror_1-13-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ...rror_1-14-60a231b64a4a0e414d0ddce1c813c614 | 2 +- ...rror_1-16-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ..._error_1-2-2ca079278e0de95eecb5df315ce05c6 | 2 +- ...error_1-3-66e3e0c942759f679c270698b49bfcf1 | 2 +- ...error_1-4-d389db66cc7fd9b144445e364dac30e3 | 2 +- ...error_1-5-63abf47101c5097e66a9c3ee92b128e3 | 2 +- ...error_1-7-a31221a0c377c14e11b14484ddaa49a6 | 2 +- ...error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 | 2 +- ...yn_part-0-9505721cd28fc4ec94ef4baf07029027 | 0 ...yn_part-1-418ec894d08c33fd712eb358f579b7a0 | 1 - ...yn_part-2-c14f09f88961dbad4d800317079a9105 | 1 - ...yn_part-3-16367c381d4b189b3640c92511244bfe | 1 - ...yn_part-4-cc664530711607c530a2cd384e67a600 | 0 ...yn_part-5-76d56e06108f0c6da75aa821452fa873 | 0 ...rtition-1-418ec894d08c33fd712eb358f579b7a0 | 2 +- ...rtition-2-c14f09f88961dbad4d800317079a9105 | 2 +- ...rtition-4-aed016ae4b528521874a719a5b129a55 | 15 +- ...oscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...scan_1-10-e6b884de17a29eb476fd6cc502fc615d | 36 - ...scan_1-11-653f748fa2d690502ba4fda407841a20 | 32 - ...scan_1-12-1c9f833953403596ad50fd32e513642c | 32 - ...scan_1-13-fdbe263d925f28d598a71b7a4c577492 | 38 - ...scan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab | 0 ...scan_1-15-db563e338e4f658e5072cc60aef15480 | 0 ...scan_1-16-e446db2c3ddce173d0a51bf77a489382 | 0 ...scan_1-17-82369b182db851e06bfddb62965e03a3 | 10 - ...scan_1-18-eece38f413a0a4f6f5b63cea0751d225 | 0 ...scan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e | 36 - ...oscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...scan_1-20-9871c619bb5bf0a28f8d60e6332a614f | 36 - ...scan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 | 32 - ...scan_1-22-521b49d223a32056025fb8dbd371a72a | 32 - ...scan_1-23-fba14d8647e0c8ca214b724486af7763 | 0 ...oscan_1-3-16367c381d4b189b3640c92511244bfe | 1 - ...oscan_1-4-7938a68104e163566da69ccc70a18f2c | 0 ...oscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc | 0 ...oscan_1-6-a1fd405e5175757aaa80033162c87670 | 10 - ...oscan_1-7-497861ae04753ffbb63ee43d89eedd9e | 0 ...oscan_1-8-623f3701ead20fff786c203d23dd60ae | 0 ...oscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 | 36 - ...tscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...rtscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...scan_1-10-c06f6ce9878b7eededf8c2a085ffb380 | 32 - ...scan_1-11-418ec894d08c33fd712eb358f579b7a0 | 1 - ...scan_1-12-3e423642a5a00dc66cc709f474a3ecef | 14 - ...scan_1-13-2fe3131322b6c82e217f27e95581e681 | 0 ...scan_1-14-2c66f128acea649e8debc3c4b04fcb9c | 36 - ...scan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 | 32 - ...scan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...tscan_1-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...tscan_1-3-16367c381d4b189b3640c92511244bfe | 1 - ...tscan_1-4-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...tscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...tscan_1-6-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...tscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...tscan_1-8-29279aa84d6ba9dea9e56b493504dd30 | 0 ...tscan_1-9-90d41ae72606f9616cb7b1303f997348 | 0 ...an_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f | 1 - ...can_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...n_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 | 32 - ...n_1_23-11-418ec894d08c33fd712eb358f579b7a0 | 1 - ...n_1_23-12-3e423642a5a00dc66cc709f474a3ecef | 15 - ...n_1_23-13-2fe3131322b6c82e217f27e95581e681 | 0 ...n_1_23-14-2c66f128acea649e8debc3c4b04fcb9c | 36 - ...n_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 | 32 - ...n_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 | 0 ...an_1_23-2-a4fb8359a2179ec70777aad6366071b7 | 1 - ...an_1_23-3-16367c381d4b189b3640c92511244bfe | 1 - ...an_1_23-4-c95dc367df88c9e5cf77157f29ba2daf | 1 - ...an_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 | 1 - ...an_1_23-6-84967075baa3e56fff2a23f8ab9ba076 | 1 - ...an_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea | 1 - ...an_1_23-8-b158e24051ecb66b8af46743917771ca | 0 ...an_1_23-9-90d41ae72606f9616cb7b1303f997348 | 0 ..._error_1-1-36eee5cbac5c0c3228e499805b32f6} | 0 ...error_1-1-887fe99770f53e7e0a0fbdc190118612 | 1 - ..._to_map-0-aefd618b58ad6c37956755b6572cbc73 | 1 - ..._to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c | 2 - ...to_map-10-32997010bba305ec40812df254490730 | 0 ...to_map-11-d99f1b631bc42a6a57c581025230537a | 0 ...to_map-12-f793eb7b36a1d6379b90e241df62c72e | 3 - ...to_map-13-32997010bba305ec40812df254490730 | 0 ..._to_map-2-28d8e6677d025e1266ce95ae23dfd9ec | 0 ..._to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 | 3 - ..._to_map-4-f356516aec917fe0d34df4dc8d9d0f95 | 0 ..._to_map-5-269cfcefe2ea6c19ac6c824d3259bbda | 3 - ..._to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf | 0 ..._to_map-7-5641678c53ce6ef1dbce3994843cfcad | 3 - ..._to_map-8-84121d964faad3547f0e5cce9d268612 | 0 ..._to_map-9-903f3b3137bfa3f4594fec7b604c62bd | 3 - ...literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 | 2 +- .../subq2-1-235919a7ddb574662158503b8052e7ca | 2 +- ...as.attr-0-fc8183d758151be72b3d75d9df124504 | 2 +- ..._format-0-3b2fa9592648fc07c0d43e0d7d7f9411 | 0 ..._format-1-d498fb503b8f47db4741fdad3d266b4a | 0 ..._format-2-42119039bf8023f90b7f474f235c5dd5 | 1 - ..._format-3-77b57147024eb6b28cc9f525fdaab615 | 1 - ..._format-4-676cb274a770a6b9ca86df5dc7f912d4 | 0 ..._format-5-ef3052815ec41b5957627698ba06707b | 0 ..._format-6-891be0baec05e358a647dcca77724446 | 0 ..._format-7-208bcc9c918cbeb52907c8871be19cd5 | 0 ..._format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 | 0 ..._format-9-433d5dbbcf75ff68d6897f31baa46841 | 1 - ...le.attr-0-26c9d24eb6305ea2106c26bdca38012e | 2 +- ..._select-1-736d6a05e1fe3249a84eb58f7cd806d2 | 4 +- ..._select-3-35c08c648a66f09f2cf7cfa6019c2113 | 2 +- ...eclause-0-b38bf01368da26ec9c60e9433a9c59a1 | 0 ...eclause-1-3e38e42c5b72364c5461c626f312be8c | 0 ...eclause-2-183920d856ad75e6d1e15121d3cd7364 | 1 - ...tamp_1-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...stamp_1-0-d362501d0176855077e65f8faf067fa8 | 0 ...stamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 | 0 ...tamp_1-1-d362501d0176855077e65f8faf067fa8} | 0 ...amp_1-10-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-11-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 | 0 ...amp_1-12-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...amp_1-13-d242038c04dd4ee6075c7eebc0f75f17} | 0 ...amp_1-14-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-15-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-16-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-17-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-19-67f274bf16de625cf4e85af0c6185cac} | 0 ...tamp_1-2-1d7cf3a2512fa1876b422b79bbe05426} | 0 ...stamp_1-2-74f477a8b726f5193dd42ef378a793c4 | 0 ...amp_1-20-343c75daac6695917608c17db8bf473e} | 0 ...amp_1-21-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-21-d8fff1a6c464e50eb955babfafb0b98e | 0 ...amp_1-22-cdb04b49b836e0244f6f0857aea7da8a} | 0 ...amp_1-23-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-24-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-25-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-26-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-28-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-29-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-3-74f477a8b726f5193dd42ef378a793c4} | 0 ...stamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 | 0 ...tamp_1-30-273256141c33eb88194cad22eb940d21 | 0 ...amp_1-30-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...amp_1-31-9587976bd7e6caa5b667975c14e8dd53} | 0 ...amp_1-32-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-33-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-34-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-35-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-37-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-38-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d | 0 ...amp_1-39-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-4-d833b177fac3162215468dde991f71d1} | 0 ...amp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b} | 0 ...amp_1-41-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-42-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-43-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-44-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-46-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-47-343c75daac6695917608c17db8bf473e} | 0 ...tamp_1-48-7029255241de8e8b9710801319990044 | 0 ...amp_1-48-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-49-7a59f9f939efc4b96f8159d00b39ed3} | 0 ...tamp_1-5-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-50-90269c1e50c7ae8e75ca9cc297982135} | 0 ...amp_1-51-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...amp_1-52-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...amp_1-53-e7b398d2a8107a42419c83771bda41e6} | 0 ...amp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...amp_1-55-67f274bf16de625cf4e85af0c6185cac} | 0 ...amp_1-56-343c75daac6695917608c17db8bf473e} | 0 ...amp_1-57-cf19f7359a6d3456c4526b2c69f92d6a} | 0 ...tamp_1-57-d362501d0176855077e65f8faf067fa8 | 0 ...amp_1-58-d362501d0176855077e65f8faf067fa8} | 0 ...tamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6} | 0 ...tamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7} | 0 ...tamp_1-8-e7b398d2a8107a42419c83771bda41e6} | 0 ...tamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf} | 0 ...tamp_2-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...stamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 | 0 ...stamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 | 0 ...tamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028} | 0 ...tamp_2-10-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-11-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-12-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-12-7350308cbf49d6ebd6599d3802750acd | 0 ...amp_2-13-5f450162886ccc79af149541527f5643} | 0 ...amp_2-14-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-15-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-16-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-17-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-19-252aebfe7882335d31bfc53a8705b7a} | 0 ...tamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4} | 0 ...stamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 | 0 ...amp_2-20-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-21-240fce5f58794fa051824e8732c00c03} | 0 ...stamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b | 0 ...amp_2-22-469fe94fb60f4b00809190c303434641} | 0 ...amp_2-23-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-24-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-25-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-26-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-28-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-29-5181279a0bf8939fe46ddacae015dad8} | 0 ...tamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19} | 0 ...stamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 | 0 ...amp_2-30-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-30-ffe6b6ddaaba84152074f7781fba2243 | 0 ...amp_2-31-8f506498acf0c99c30960a00981ef460} | 0 ...amp_2-32-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-33-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-34-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-35-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-37-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-38-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-39-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-39-8236608f28681eac5503195096a34181 | 0 ...tamp_2-4-81d6d29dcb3fd12a519426dff64411d2} | 0 ...amp_2-40-972a007e54d1c09e9ac9549c19a32dbb} | 0 ...amp_2-41-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-42-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-43-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-44-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-46-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-47-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-48-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-48-654e5533ec6dc911996abc7e47af8ccb | 0 ...amp_2-49-650d2727b007638e0ed39b37c9498d66} | 0 ...tamp_2-5-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-50-25f6ec69328af6cba76899194e0dd84e} | 0 ...amp_2-51-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...amp_2-52-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...amp_2-53-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...amp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...tamp_2-55-252aebfe7882335d31bfc53a8705b7a} | 0 ...amp_2-56-5181279a0bf8939fe46ddacae015dad8} | 0 ...amp_2-57-240fce5f58794fa051824e8732c00c03} | 0 ...tamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 | 0 ...amp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028} | 0 ...tamp_2-6-93c769be4cff93bea6e62bfe4e2a8742} | 0 ...tamp_2-7-5bdbf67419cc060b82d091d80ce59bf9} | 0 ...tamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571} | 0 ...tamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef} | 0 ...stamp_3-0-165256158e3db1ce19c3c9db3c8011d2 | 0 ...tamp_3-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...tamp_3-1-165256158e3db1ce19c3c9db3c8011d2} | 0 ...stamp_3-1-81edf5107270547641586aa02b4e7d9b | 0 ...amp_3-10-ffc79abb874323e165963aa39f460a9b} | 0 ...amp_3-11-7b1ec929239ee305ea9da46ebb990c67} | 0 ...tamp_3-12-165256158e3db1ce19c3c9db3c8011d2 | 0 ...amp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a} | 0 ...tamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 | 1 + ...amp_3-14-165256158e3db1ce19c3c9db3c8011d2} | 0 ...stamp_3-2-7a012a0d98729da25b5ac374855dcee4 | 0 ...tamp_3-2-81edf5107270547641586aa02b4e7d9b} | 0 ...stamp_3-3-6143888a940bfcac1133330764f5a31a | 0 ...tamp_3-3-7a012a0d98729da25b5ac374855dcee4} | 0 ...tamp_3-4-86514381187b246a5685577c1968c559} | 0 ...tamp_3-5-935d0d2492beab99bbbba26ba62a1db4} | 0 ...tamp_3-6-8fe348d5d9b9903a26eda32d308b8e41} | 0 ...tamp_3-7-6be5fe01c502cd24db32a3781c97a703} | 0 ...tamp_3-8-6066ba0451cd0fcfac4bea6376e72add} | 0 ...tamp_3-9-22e03daa775eab145d39ec0730953f7e} | 0 ...arison-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...arison-1-4b68f7ad0f8cf337e42bf16a45e15818} | 0 ...arison-2-60557e7bd2822c89fa8b076a9d0520fc} | 0 ...arison-3-f96a9d88327951bd93f672dc2463ecd4} | 0 ...arison-4-13e17ed811165196416f777cbc162592} | 0 ...arison-5-4fa8a36edbefde4427c2ab2cf30e6399} | 0 ...arison-6-7e4fb6e8ba01df422e4c67e06a0c8453} | 0 ...arison-7-8c8e73673a950f6b3d960b08fcea076f} | 0 ...arison-8-510c0a2a57dc5df8588bd13c4152f8bc} | 0 ...arison-9-659d5b1ae8200f13f265270e52a3dd65} | 0 ...p_lazy-2-bb5a4a13274290029bd07d95c2f92563} | 0 ...mp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 | 0 ...mp_null-2-51762cf5079877abf7d81127738f4e5} | 0 ...mp_null-2-b3071984dee42c9e698e947fcbc2a1e8 | 0 ...mp_udf-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...amp_udf-0-79914c5347620c6e62a8e0b9a95984af | 0 ...amp_udf-1-59fc1842a23369235d42ed040d45fb3d | 0 ...mp_udf-1-79914c5347620c6e62a8e0b9a95984af} | 0 ...p_udf-10-287614364eaa3fb82aad08c6b62cc938} | 0 ...p_udf-11-dbc23736a61d9482d13cacada02a7a09} | 0 ...p_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35} | 0 ...mp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6} | 0 ...mp_udf-14-6ab3f356deaf807e8accc37e1f4849a} | 0 ...p_udf-15-c745a1016461403526d44928a269c1de} | 0 ...p_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe} | 0 ...p_udf-17-b36e87e17ca24d82072220bff559c718} | 0 ...p_udf-18-dad44d2d4a421286e9da080271bd2639} | 0 ...mp_udf-19-79914c5347620c6e62a8e0b9a95984af | 0 ...p_udf-19-cb033ecad964a2623bc633ac1d3f752a} | 0 ...mp_udf-2-59fc1842a23369235d42ed040d45fb3d} | 0 ...amp_udf-2-9039f474f9a96e9f15ace528faeed923 | 0 ...mp_udf-20-59fc1842a23369235d42ed040d45fb3d | 0 ...p_udf-20-79914c5347620c6e62a8e0b9a95984af} | 0 ...p_udf-21-59fc1842a23369235d42ed040d45fb3d} | 0 ...mp_udf-3-9039f474f9a96e9f15ace528faeed923} | 0 ...amp_udf-3-b0fd4ca3b22eb732a32772399331352f | 0 ...amp_udf-4-80ce02ec84ee8abcb046367ca37279cc | 0 ...mp_udf-4-b0fd4ca3b22eb732a32772399331352f} | 0 ...amp_udf-5-66868a2b075de978784011e9955483d} | 0 ...amp_udf-6-1124399033bcadf3874fb48f593392d} | 0 ...mp_udf-7-5810193ce35d38c23f4fc4b4979d60a4} | 0 ...mp_udf-8-250e640a6a818f989f3f3280b00f64f9} | 0 ...mp_udf-9-975df43df015d86422965af456f87a94} | 0 ...ansform-0-d81d055660f6ef3d9cc60dd673a8c0fe | 2 +- ...nsform1-0-b6919fc48901e388c869c84ae0211102 | 0 ...nsform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 | 0 ...nsform1-2-25d6cab86c36d65fabf5645db3126a19 | 0 ...nsform1-3-8324a70d533a06a5171c1016b1fea7c3 | 0 ...nsform1-4-65527bae8e73262255ef83082c6968f9 | 0 ...nsform1-5-e0037a3f97ce0127a40d163af4c20ad5 | 0 ...ansform1-6-3b862abd732c9e9f0db50ad0b9dae6f | 1 - ...rm_ppr1-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...rm_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 | 2 +- ...rm_ppr2-0-ae225e86c2ae20519ffdf23190454161 | 2 +- ...rm_ppr2-2-636c4938673a273299c8569295d27c99 | 2 +- ... clause-0-3b6afcbd622aa111ee260bebc763613d | 2 +- ... clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 | 2 +- ...cast_1-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d | 0 ...cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d} | 0 ..._cast_1-2-53a667981ad567b2ab977f67d65c5825 | 1 + ...dening-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...idening-0-630ac2c7e7dea4837384ccd572209229 | 0 ...dening-1-630ac2c7e7dea4837384ccd572209229} | 0 ...idening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 | 1 - ...idening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 | 0 ...dening-2-cfbdf2b6fca84c6e23d4e691d2221bd6} | 0 ...idening-3-65da8c67f6903286168acb39ac67fc04 | 1000 ------- ...dening-3-a0ba6952d9bf830d1d1ea7aebd3784a2} | 0 ...idening-4-65da8c67f6903286168acb39ac67fc04 | 1000 +++++++ ...ect_set-0-38512a3299e2390dd813e61a0f63f35e | 2 +- ...ect_set-1-78aa199d061d2ff9ba426849ea1eb449 | 2 +- ...ect_set-10-c8bc33095e1a195bb7b5e579d8d78db | 2 +- ...ect_set-11-5c3768074977ef68a1b9bb72eb9ef02 | 20 + ...ct_set-11-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...t_set-12-1d351f7e821fcaf66c6f7503e42fb291} | 0 ...ct_set-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...t_set-13-a00d1791b7fa7ac5a0505d95c3d12257} | 0 ...ect_set-13-c8bc33095e1a195bb7b5e579d8d78db | 20 - ...ect_set-14-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...t_set-15-863233ccd616401efb4bf83c4b9e3a52} | 0 ...t_set-16-a00d1791b7fa7ac5a0505d95c3d12257} | 0 ...ect_set-17-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ect_set-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...lect_set-2-4747c35670a011344982573ba31a9bb | 1 + ...ect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 | 1 + ...ect_set-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...ct_set-4-1d351f7e821fcaf66c6f7503e42fb291} | 0 ...lect_set-4-c8bc33095e1a195bb7b5e579d8d78db | 20 - ...ect_set-5-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...ct_set-5-a7dc16cb82c595b18d4258a38a304b1e} | 0 ...ect_set-6-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...lect_set-6-c8bc33095e1a195bb7b5e579d8d78db | 20 + ...ect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 | 20 + ...lect_set-7-c8bc33095e1a195bb7b5e579d8d78db | 20 - ...ect_set-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...ct_set-8-863233ccd616401efb4bf83c4b9e3a52} | 0 ...ect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...ct_set-9-a7dc16cb82c595b18d4258a38a304b1e} | 0 ...daf_corr-2-c6f2dc536bf105650a461816ae5e330 | 0 ...f_corr-2-e886f45c8f085596ffd420f89cdc2909} | 0 ...daf_corr-3-ddf417dbc3b551cc8df47d950cec03e | 2 +- ...af_corr-4-8771b2d83d14b3b641e8a77fcdc5081a | 2 +- ...af_corr-5-8abbd73784728b599f8c2a90f53da9fb | 2 +- ...af_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 | 2 +- ...af_corr-7-70e701f50c3418ff91649b2bd8287da3 | 2 +- ...af_corr-8-f2f0c7735f8b24266d5aaff96644e369 | 2 +- ...af_corr-9-e2a0fa75c43279764ebca015f62bcf16 | 2 +- ...ovar_pop-2-c6f2dc536bf105650a461816ae5e330 | 0 ...ar_pop-2-e886f45c8f085596ffd420f89cdc2909} | 0 ...var_pop-3-fe27ea6dae14250e8835baef3c2e20f6 | 2 +- ...var_pop-4-7af9231ae293d0c4b84050176e1d73ad | 2 +- ...var_pop-5-22004d36f6f3770db284644317770fcd | 2 +- ...var_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b | 2 +- ...var_pop-7-37e59e993e08216e6c69f88d6ac673ae | 2 +- ...var_pop-8-1e51388408dad651127edf940c11d91f | 2 +- ...var_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 | 2 +- ...var_samp-2-c6f2dc536bf105650a461816ae5e330 | 0 ...r_samp-2-e886f45c8f085596ffd420f89cdc2909} | 0 ...ar_samp-3-7aa25da7ccb88ba67b100888b6227aaf | 2 +- ...ar_samp-4-7e705a637950911e0a18059d8bf1fd2c | 2 +- ...var_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b | 2 +- ...ar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 | 2 +- ...ar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 | 2 +- ...ar_samp-8-44861ae58cf0951eeda28a5f778f778a | 2 +- ...ar_samp-9-234a5b02085d2228473d7ef15a6af683 | 2 +- ...numeric-0-86b9fb8bef8a5c51077623f1db3a0251 | 2 +- ...numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 | 2 +- ...numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 | 2 +- ...numeric-3-ff41f4450d6ae372633fde865ae187c6 | 2 +- ..._format-0-eff4ef3c207d14d5121368f294697964 | 0 ..._format-1-4a03c4328565c60ca99689239f07fb16 | 1 - ...prox_20-0-9ce9365f7b3f35a10b5305251c3e81ac | 0 ...prox_20-1-c7d32089880679d178dea94f1fe118e6 | 0 ...rox_20-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...rox_20-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...rox_20-12-914ba18b45a27894bd82302f07efc789 | 0 ...rox_20-13-4bd5703fa32f3283f38841acadc97adb | 0 ...rox_20-14-d861a06b90896a097901d64ab9fbec53 | 0 ...rox_20-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...rox_20-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...rox_20-17-b89ea2173180c8ae423d856f943e061f | 1 - ...rox_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 - ...prox_20-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...prox_20-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...rox_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_20-21-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_20-22-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_20-23-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_20-24-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_20-25-15f40568b41c4505841f5ad13c526f51 | 1 - ...rox_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_20-27-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_20-28-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_20-29-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...prox_20-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...rox_20-30-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...rox_20-32-dbcec232623048c7748b708123e18bf0 | 1 - ...rox_20-33-f28c7b0408737da815493741c806ff80 | 0 ...rox_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_20-35-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_20-36-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_20-37-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_20-38-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_20-39-15f40568b41c4505841f5ad13c526f51 | 1 - ...prox_20-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...rox_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_20-41-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_20-42-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_20-43-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...rox_20-44-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...prox_20-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...prox_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...prox_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...pprox_20-8-7e3cf228c457279965b7414bd05527f | 0 ...prox_20-9-5aea8aa95a85c46284f7c1f45978a228 | 0 ...prox_23-0-477a942be95c0616c72f02a0077f9ace | 0 ...prox_23-1-c7d32089880679d178dea94f1fe118e6 | 0 ...rox_23-10-b7e588217a3cd184dbbb8d419d3e33ae | 0 ...rox_23-11-5034ec132cb8b0a6bd6357a7e1abd755 | 0 ...rox_23-12-914ba18b45a27894bd82302f07efc789 | 0 ...rox_23-13-4bd5703fa32f3283f38841acadc97adb | 0 ...rox_23-14-d861a06b90896a097901d64ab9fbec53 | 0 ...rox_23-15-ca796efecd0d064e9e688a17ce75d80f | 0 ...rox_23-16-c838e13d9aafe1212a76d2cf5fe085a0 | 0 ...rox_23-17-b89ea2173180c8ae423d856f943e061f | 1 - ...rox_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 | 1 - ...prox_23-19-b931aec4add0a66c23e444cdd5c33c5 | 0 ...prox_23-2-ac53a7ba5e8a208255008d3a71fa321a | 0 ...rox_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_23-21-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_23-22-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_23-23-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_23-24-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_23-25-15f40568b41c4505841f5ad13c526f51 | 1 - ...rox_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_23-27-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_23-28-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_23-29-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...prox_23-3-1dae5b2a11507c83b0f76e677a368712 | 0 ...rox_23-30-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...rox_23-32-dbcec232623048c7748b708123e18bf0 | 1 - ...rox_23-33-f28c7b0408737da815493741c806ff80 | 0 ...rox_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c | 1 - ...rox_23-35-3cd4e1282d82d07785051a1cf0e9b4ff | 1 - ...rox_23-36-ed1aec1a908310db90c5f8667631a1df | 1 - ...rox_23-37-333d72e8bce6d11a35fc7a30418f225b | 1 - ...rox_23-38-61903781f5cd75e6f11d85e7e89c1cb3 | 1 - ...rox_23-39-15f40568b41c4505841f5ad13c526f51 | 1 - ...prox_23-4-b2e21ffa55342d4f3c243728dfe6b11f | 0 ...rox_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb | 1 - ...rox_23-41-ee4c065e8557632a9ee348dd9223c3a1 | 1 - ...rox_23-42-ba77d1a26f87385f046129b6eb7d2ec3 | 1 - ...rox_23-43-956d6b6bc69c8035f80de2e60eda65fb | 1 - ...rox_23-44-d196cc7f52bb6ae19a5e66eb2a99577c | 1 - ...rox_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 | 1 - ...prox_23-5-8ae1465266d28bc2e5da8d89617873c4 | 0 ...prox_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 | 0 ...prox_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 | 0 ...pprox_23-8-7e3cf228c457279965b7414bd05527f | 0 ...prox_23-9-5aea8aa95a85c46284f7c1f45978a228 | 0 .../udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 | 0 .../udf1-1-8281592c818ada269024ac669bec78da | 0 .../udf1-2-c7648c913ec336559fb67e3ab6938c8f | 0 .../udf1-3-adc1ec67836b26b60d8547c4996bfd8f | 1 - .../udf2-3-c5938fcdd5675b58a4ed54269b5f5591 | 2 +- .../udf3-0-66a2b926dd5439d4da9eb58743c18a8c | 0 .../udf3-1-1d04874d496d05cfe0b9d86de1111 | 0 .../udf3-2-25fe77d053e2bad8ae99757ce237052e | 0 .../udf3-3-adc1ec67836b26b60d8547c4996bfd8f | 1 - .../udf6-3-e579646b969eef49b09656114da52a73 | 2 +- .../udf6-5-fe336cd9850d6357980bd19139f76e | 2 +- .../udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 | 2 +- .../udf8-4-9f22d5a65353432826a526b1d76eb65b | 2 +- .../udf9-1-dd0981dc44ac24d445af5412e9f7fa8c | 2 +- .../udf_E-0-33251f00f840de3672f19d353fcfa66f | 0 ... udf_E-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ... udf_E-1-cad0779d18f326c8e453bf2b5fe43596} | 0 ... udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02} | 0 ... udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a} | 0 .../udf_E-4-33251f00f840de3672f19d353fcfa66f | 0 ... udf_E-4-e8924af3bf99d2e01546a965303ffd09} | 0 ... udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4} | 0 ... udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02} | 0 ... udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a} | 0 ... udf_E-8-e8924af3bf99d2e01546a965303ffd09} | 0 ...udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 .../udf_PI-0-b28e761e5564b51f98f182f561c1369f | 0 ...udf_PI-1-13fd9345fd15b654d18b2707e5274b2b} | 0 ...udf_PI-2-97a12f6967726e425469ecfa70177ff0} | 0 ...udf_PI-3-9c1476a2eab7455594e97b338ee3c188} | 0 ...udf_PI-4-890f3c276eff2c459d8dc79d5a71c866} | 0 .../udf_PI-4-b28e761e5564b51f98f182f561c1369f | 0 ...udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d} | 0 ...udf_PI-6-97a12f6967726e425469ecfa70177ff0} | 0 ...udf_PI-7-9c1476a2eab7455594e97b338ee3c188} | 0 ...udf_PI-8-890f3c276eff2c459d8dc79d5a71c866} | 0 ...df_abs-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 | 1 - ...udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 | 1 + ...udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 | 6 - ...udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 | 0 ...udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 | 6 + ...df_abs-3-50cb3c23902cd29e0dbff188c71062e5} | 0 ...udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 | 1 - .../udf_abs-4-30cd5a94c13e1619ee18b9551db879c | 0 ...udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 | 1 + ...udf_abs-5-343e899acb67c283391387f02aa7b5c4 | 1 - ...df_abs-5-5cd4c198e0de884ad436864b95fece6c} | 0 ...udf_abs-6-7aa32a019499c6464aded2e357c6843b | 1 + ...df_acos-0-4f49cb5a5c87efea534d63ed76435f06 | 1 - ...f_acos-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_acos-1-4f49cb5a5c87efea534d63ed76435f06 | 1 + ...df_acos-1-d330511cf4f626cd844b18f57f99a85f | 6 - ...df_acos-2-86fca49baf270873b46709c9eaeab87b | 1 - ...df_acos-2-d330511cf4f626cd844b18f57f99a85f | 6 + ...f_acos-3-661a0a85283df2a5c1567d60850e362b} | 0 ...df_acos-3-f7f199e5f3dde8056465d55aca29e884 | 1 - ...df_acos-4-23d588eece08fbea7431044524f1cecf | 1 + ...df_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a | 1 - ...df_acos-5-578612589fdb1ae21ee488924848fb4e | 1 + ...df_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e | 1 - ...df_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 | 1 + ...udf_add-0-5db764318a918a5f6d7c1d95d9e86045 | 2 +- ...udf_add-1-400b238f4e6cdf7120be566b0ef079c5 | 2 +- ..._array-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._array-1-2e9c2a7d9325bd9a53cf9f181b6333ee} | 0 ..._array-2-570741914bb78300b0233e5f38d7f08a} | 0 ...f_array-2-db6d3c6c67faed3ceb019cb1993502f6 | 0 ..._array-3-47818d42e5e7667d8754c3f9a4b8053a} | 0 ..._array-4-51410e4d4d679fe5a8dd7a860f4efc47} | 0 ...ntains-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ontains-0-d9a90108b052b111e8de4433e008b25a | 1 - ...ontains-1-d9a90108b052b111e8de4433e008b25a | 1 + ...ontains-1-eff16c7836252e01f3d8190cd833f79c | 4 - ...ontains-2-42d966b28e61a465d638bffc20ac7247 | 1 - ...ontains-2-eff16c7836252e01f3d8190cd833f79c | 4 + ...ontains-3-6b700cb69af298aef45b96bf5ac862d} | 0 ...ontains-3-ec0cd851fd8135dd9bc822d9a0432569 | 1 - ...ntains-4-bdb5a7e0ab81172a438145a1c406b1e8} | 0 ..._ascii-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_ascii-0-72924c23459330cca6a54c70b12a542c | 1 - ...f_ascii-1-72924c23459330cca6a54c70b12a542c | 1 + ...f_ascii-1-dab8656d7f001e85615442d60df4b6b3 | 6 - ...f_ascii-2-a9e207503f52d43903877fb998eabeaa | 0 ...f_ascii-2-dab8656d7f001e85615442d60df4b6b3 | 6 + ...f_ascii-3-28fc6497c9835c2ef331aba44576f1b1 | 1 - ..._ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34} | 0 ...udf_ascii-4-db9a06881a216f0252fa786d98c9bf | 1 + ...f_asin-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_asin-0-99da197a53767060e3fa0250254d59cd | 1 - ...df_asin-1-3d0c3fa6121f8f5158d221074f1d4129 | 6 - ...df_asin-1-99da197a53767060e3fa0250254d59cd | 1 + ...df_asin-2-3d0c3fa6121f8f5158d221074f1d4129 | 6 + ...df_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f | 1 - ...f_asin-3-4b7ee6310a49ebf784a4a712748348ac} | 0 ...df_asin-3-b67069d37df3a7fb7a3e2138d8558343 | 1 - ...df_asin-4-929dabad86ef0e564802de8f663a9e66 | 1 - ...df_asin-4-a3edf78ff8ccc629ee7d7518707b69ce | 1 + ...df_asin-5-1ee8715cce9831623d0af0031964d284 | 1 - ...df_asin-5-8dcbcf784496053e3b57c579aca809a6 | 1 + ...df_asin-6-114c8141f1e831c70d70c570f0ae778f | 1 + ...f_atan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_atan-0-c79ed30c2444c8493d0db98c33c9132b | 1 - ...df_atan-1-77e7ac1b89a4eac9102176cd73f67a62 | 4 - ...df_atan-1-c79ed30c2444c8493d0db98c33c9132b | 1 + ...df_atan-2-77e7ac1b89a4eac9102176cd73f67a62 | 4 + ...df_atan-2-bf1c7875364d5518e0ac9c1ac6943764 | 1 - ...df_atan-3-9a6252f36fe5ec1387186bf47d74a139 | 1 - ...f_atan-3-e6f97a834028a67e6c3033949f98fbf8} | 0 ...df_atan-4-c79ed30c2444c8493d0db98c33c9132b | 1 - ...df_atan-4-eacd47571ba5c67f11e025d8d4de5811 | 1 + ...df_atan-5-77e7ac1b89a4eac9102176cd73f67a62 | 4 - ...df_atan-5-c79ed30c2444c8493d0db98c33c9132b | 1 + ...df_atan-6-77e7ac1b89a4eac9102176cd73f67a62 | 4 + ...df_atan-6-bf1c7875364d5518e0ac9c1ac6943764 | 1 - ...df_atan-7-9a6252f36fe5ec1387186bf47d74a139 | 1 - ...f_atan-7-e6f97a834028a67e6c3033949f98fbf8} | 0 ...df_atan-8-eacd47571ba5c67f11e025d8d4de5811 | 1 + ...udf_avg-0-2d715528b290951fb9874f60d7e9b537 | 2 +- ...udf_avg-1-c707c56871a903e4e022b3df5c92fc3f | 2 +- ..._bigint-0-6c5b1e4b9d725caeb786bb18448a7927 | 2 +- ..._bigint-1-4636e4f0083ea54814995a03b7c81202 | 2 +- ...df_bin-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b | 1 - ...udf_bin-1-843803a1b6ada107c11397af1a2f55d6 | 5 - ...udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b | 1 + ...udf_bin-2-5ee3932ab9cd164f1005a4413a68007b | 1 - ...udf_bin-2-843803a1b6ada107c11397af1a2f55d6 | 5 + .../udf_bin-3-6fda27c8567ac896538cba3f2b230ab | 1 + .../udf_bin-3-b72fc578a7c677e15b8598248c81901 | 1 - ...udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 | 1 + ...ap_and-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...map_and-0-abea2a2780fad42422774174cbfd603d | 1 - ...map_and-1-414291f11711df40fb8362e0a0156b25 | 1 - ...map_and-1-c9e0d8424ec5f433565397b113ae4f57 | 1 + ...map_and-2-5a60dfc80bef392fa390adddab0c0f87 | 1 + ...map_and-2-d550d017f9eb9176593719ea34eaae9b | 0 ...map_and-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...ap_and-3-d550d017f9eb9176593719ea34eaae9b} | 0 ...map_and-4-6320e1c4829024f819d3883df9a782c0 | 0 ...ap_and-4-a486db1f5a06f9623a0e5abd7737b0c6} | 0 ...ap_and-5-549dbeb1293c4c49ae08bf08acdbdf23} | 0 ...map_and-5-ff2860a163cbe78d5affac8047199296 | 10 - ...map_and-6-d550d017f9eb9176593719ea34eaae9b | 0 ...map_and-6-ff2860a163cbe78d5affac8047199296 | 10 + ...ap_and-7-d550d017f9eb9176593719ea34eaae9b} | 0 ..._empty-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...p_empty-0-6c80525a4eab6761596e6ad5aa75bc90 | 1 - ...p_empty-1-a03987655a167f5b18c37b643391a0df | 1 - ..._empty-1-a174269b5d1757398ab7f89cf1c97bfa} | 0 ..._empty-2-f5d9880a3278b5632b356bbe6ecd90d3} | 0 ...tmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c | 1 - ...map_or-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...tmap_or-1-85cab84fba419b436b033e3ea07e02ef | 1 - ...tmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef | 1 + ...tmap_or-2-ab062e08acfd7e552a64ea967a0360c8 | 1 + ...tmap_or-2-d550d017f9eb9176593719ea34eaae9b | 0 ...tmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 | 0 ...map_or-3-d550d017f9eb9176593719ea34eaae9b} | 0 ...tmap_or-4-6320e1c4829024f819d3883df9a782c0 | 0 ...map_or-4-a486db1f5a06f9623a0e5abd7737b0c6} | 0 ...map_or-5-549dbeb1293c4c49ae08bf08acdbdf23} | 0 ...tmap_or-5-ea92fff4d814552b57535ed342a5dde0 | 10 - ...tmap_or-6-d550d017f9eb9176593719ea34eaae9b | 0 ...tmap_or-6-ea92fff4d814552b57535ed342a5dde0 | 10 + ...map_or-7-d550d017f9eb9176593719ea34eaae9b} | 0 ...ise_and-0-e2374700cd32add926992d5539bd463a | 2 +- ...ise_and-1-2e63ac31262106160ab043027e356a4b | 2 +- ...ise_not-0-34abab2f47f90f0f194ef44aed1cdd7f | 2 +- ...ise_not-1-ccc0c59ea3e29b6d55e1edee7029155d | 2 +- ...wise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 | 2 +- ...wise_or-1-272722c23fece2807c08191d3969c3bb | 2 +- ...ise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 | 2 +- ...ise_xor-1-8fc9b9bf1aced42754419310784e0a9f | 2 +- ...boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 | 2 +- ...boolean-1-23178b473a9d25d6be0abe378d44fb0e | 2 +- ...f_case-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_case-0-5bcbe4c0e0a75462160814a10b9449f4 | 1 - ...df_case-1-54acf006155d8822a50e47729be24004 | 1 - ...df_case-1-5bcbe4c0e0a75462160814a10b9449f4 | 1 + ...df_case-2-54acf006155d8822a50e47729be24004 | 1 + ...df_case-2-98ee676f92950375917f09d2e492253f | 0 ...f_case-3-48789112d79aeb450d9f49184fc20e1c} | 0 ...df_case-3-ec7343402fd77807842a0eaf2497a47c | 1 - ...df_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 | 1 - ...df_case-4-d39ed6ecd256fa99657f13709cb1c6e3 | 1 + ...df_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb | 1 + ...df_case-6-ff583116ba2edd78202349faf1e757dc | 1 + ...pruning-0-dd2d7a075df235f17c26bac8713e939c | 0 ..._thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 | 0 ..._thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 | 3 - ...df_ceil-0-4b40e67b8ca75729ab07df966d814e06 | 2 +- ...df_ceil-1-f410065d893a263f375fcf74072877bb | 2 +- ...ceiling-0-d5685e38400e68341e36a59671dcbdfd | 2 +- ...ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df | 2 +- ...oalesce-0-8d1c97c292c51447e58606f2cefa87c1 | 1 - ...oalesce-1-e2c086f5148f10264c09ecbd7284c809 | 4 - ...coalesce-2-bd78a25868e5598ea275e0be5e4c716 | 0 ...oalesce-3-badb02b0dfa13d6a1519e6198bb511d2 | 1 - ...oalesce-4-83f323874d7941c463ced2aee6cc5157 | 0 ...oalesce-5-4bcad31a47f4bfc3baef75b65baa8082 | 11 - ..._string-0-32b16ab99287db115e8de5214ac24b77 | 10 - ...concat-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_concat-0-7bc53505a4e6587132870d8d0a704d2 | 1 - ..._concat-1-765c520f239cdff1ea3f8d22ba83e031 | 5 - ...f_concat-1-7bc53505a4e6587132870d8d0a704d2 | 1 + ..._concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 | 1 - ..._concat-2-765c520f239cdff1ea3f8d22ba83e031 | 5 + ..._concat-3-a38183c2685e912befe6246f1b6f93b8 | 1 + ..._concat-4-a20ebbc181e5ee4a1c22ddafd212ddde | 1 + ...insert1-2-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...insert2-2-adc1ec67836b26b60d8547c4996bfd8f | 2 +- ...ncat_ws-0-4c8bd7794a527e544c838408313eeaa8 | 1 - ...cat_ws-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ncat_ws-1-4c8bd7794a527e544c838408313eeaa8 | 1 + ...ncat_ws-1-b8b80f7e9bf4348367444c73020b3cab | 4 - ...ncat_ws-2-b8b80f7e9bf4348367444c73020b3cab | 4 + ...ncat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c | 0 ...ncat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 | 0 ...cat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c} | 0 ...ncat_ws-4-a507af4996b13433b0ae100fcb32358f | 0 ...cat_ws-4-b13a1f7f63d2a54efa331c82bd635d63} | 0 ...cat_ws-5-a507af4996b13433b0ae100fcb32358f} | 0 ...ncat_ws-5-ca4f051369183cae36fc9a7bec6a9341 | 1 - ...ncat_ws-6-98276439c0605401ff89c6a5ae54be09 | 0 ...ncat_ws-6-ca4f051369183cae36fc9a7bec6a9341 | 1 + ...ncat_ws-7-8f08128276e7e50eeb5a6932c763126c | 1 - ...cat_ws-7-97071809ba7701b78e3729996f14b591} | 0 ...ncat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 | 1 + ...ncat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 | 1 - ...ncat_ws-9-8f60d81b410f4825809aa510806f2df2 | 1 + ...f_conv-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_conv-0-d552befca345f3396464529cfde9f75a | 1 - ...df_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab | 7 - ...df_conv-1-d552befca345f3396464529cfde9f75a | 1 + ...f_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 | 3 - ...f_conv-10-f9ea15984e84250494e81e25d6a401c0 | 1 + ...f_conv-11-2ce9111b47ed100bebc8d08de54efd1f | 3 + ...df_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab | 7 + ...df_conv-2-6d61a2118b54843716aef87fe539b595 | 1 - ...df_conv-3-5f43d0bec0421c86c49e2546c5ee923a | 1 + ...df_conv-3-97161f7a60851d445b23c4ebe4095a1d | 1 - ...df_conv-4-568e843076f358c404a8634b18541c55 | 1 - ...df_conv-4-5df8d45902a3537e67545e629a96328a | 1 + ...df_conv-5-3f23d98799b825a2e9594066f973d183 | 1 - ...df_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 | 1 + ...df_conv-6-4981b5b92d87fd1000fa0ac26fa6163a | 1 - ...df_conv-6-e5430adfa782ea9094e570d339626c0f | 1 + ...df_conv-7-687f9b8a09f458d771d5641eec40031b | 1 + ...df_conv-7-77bd25ad13e3697c80464e4a2682360e | 1 - ...df_conv-8-2fae52d392251be476e0c8f6071a4aeb | 1 - ...udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 | 1 + ...df_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 | 1 + ...df_conv-9-2f0098c00c10044522cd23a4a2f54957 | 1 - ...udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 | 1 - ...df_cos-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 | 4 - ...udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 | 1 + ...udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 | 4 + ...udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 | 1 - ...df_cos-3-166acc86afd6ececfe43800e38f106c9} | 0 ...udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 | 1 - ...udf_cos-4-efda2f85872c6144280970eab84ef4d4 | 1 + ...f_count-0-534a9b25b87d09e418645b1216949560 | 2 +- ...f_count-1-d566feb21bc894b97e6416b65fe5c02f | 2 +- ..._count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e | 2 +- ..._count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e | 2 +- ...f_count-13-9286bc5f08bf4db183719b1b49dc5b7 | 2 +- ...f_count-3-e43165f41597d2a1c9e8cf780b99a4a8 | 2 +- ...f_count-5-bdee61c35a27bfab974e2ba199d5dfa4 | 2 +- ...f_count-7-b975ad0d5f293508ce4832a7b19399b6 | 2 +- ...f_count-9-75b3d8a0dac332ea00af5ef8971ca643 | 2 +- ...ate_add-0-74d34471bfa0880230d8e3351eb0ab45 | 2 +- ...date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 | 2 +- ...ate_sub-0-f8fbf85026da1b0778fd325d9b5dae33 | 2 +- ...ate_sub-1-7efeb74367835ade71e5e42b22f8ced4 | 2 +- ...atediff-0-3bd040a96a4568e7ea4922faa408ada5 | 2 +- ...atediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 | 2 +- ...udf_day-0-c4c503756384ff1220222d84fd25e756 | 2 +- .../udf_day-1-87168babe1110fe4c38269843414ca4 | 2 +- ...ofmonth-0-7b2caf942528656555cf19c261a18502 | 2 +- ...ofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 | 2 +- ...egrees-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...degrees-0-85f4957cd7cd6c517f6941af1289d8ae | 0 ...egrees-1-f24ce67606944e23a4adc79f91cf0c17} | 0 ...egrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4} | 0 ...egrees-3-2fd3a55901fe765f8f154531a7f5fd6b} | 0 ...egrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96} | 0 ...degrees-4-85f4957cd7cd6c517f6941af1289d8ae | 0 ...egrees-5-3a6468b02be2605c91b31987e76fb9a8} | 0 ...egrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4} | 0 ...egrees-7-2fd3a55901fe765f8f154531a7f5fd6b} | 0 ...egrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96} | 0 ...udf_div-0-31d31c7d5c544327dabfd874c88314db | 1 - ...df_div-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_div-1-31d31c7d5c544327dabfd874c88314db | 1 + ...udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 | 4 - .../udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b | 1 - ...udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 | 4 + ...df_div-3-5111340caad64e36370d9d4bc4db5f27} | 0 ..._divide-0-1af8b249439ee5b7d4978c31669bc208 | 1 - ..._divide-1-fa932664bae88683a222b71ac45fb840 | 4 - ..._divide-2-ce54773b1babe6dde982e4e91ebaeb50 | 1 - ..._double-0-aa32d73a99587cae2f1efd9a2094d617 | 2 +- ..._double-1-79380157cbd6624d760335f8291e6fb4 | 2 +- ...df_elt-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_elt-1-b46b060da76d1772db998c26a62a608f} | 0 ...udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a | 0 ...df_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4} | 0 ...df_elt-3-c2554fac72a2a51bb33faae16aec3507} | 0 ...df_elt-4-533ad9c703c320a6556c09dd5f9ac351} | 0 ..._equal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._equal-1-36b6cdf7c5f68c91155569b1622f5876} | 0 ..._equal-2-2422b50b96502dde8b661acdfebd8892} | 0 ..._equal-3-e0faab0f5e736c24bcc5503aeac55053} | 0 ..._equal-4-39d8d6f197803de927f0af5409ec2f33} | 0 ..._equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca} | 0 ..._equal-6-878650cf21e9360a07d204c8ffb0cde7} | 0 ..._equal-7-1635ef051fecdfc7891d9f5a9a3a545e} | 0 ..._equal-8-276101b04b10b7cd6d59061a8cbf42d2} | 0 ...udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a | 2 +- ...udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 | 2 +- ...f_field-0-277b4a4dcb38cabb6df50147c77e0a33 | 1 - ..._field-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_field-1-277b4a4dcb38cabb6df50147c77e0a33 | 1 + ...f_field-1-379d8580693493f620a8f4084709324f | 2 - ..._field-10-7982ea72163dbc4cd45f53454edf66c8 | 2 - ...field-10-ca9db7e6bb687606bc273d1f6c191035} | 0 ..._field-11-7982ea72163dbc4cd45f53454edf66c8 | 2 + ...f_field-2-379d8580693493f620a8f4084709324f | 2 + ...f_field-2-d2c6583a79d77aabe388a52ec164c38b | 1 - ..._field-3-5960d42257b272f6ba043406229bbf26} | 0 ...f_field-3-fea09e934696af40bb604b40225bbc98 | 1 - ...f_field-4-212d8b2297bf6a3311d24d68de67b5c6 | 1 + ...f_field-4-b0815d34893d6cba8c07d0a0721c1d29 | 1 - ...f_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 | 0 ...f_field-5-949c2de113b214d83734c0c177f04b6b | 1 + ..._field-6-2d7c5cbe891c4a9dda34f9842f8e0828} | 0 ...f_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 | 0 ...f_field-7-3fd8b0c333acdf28c676315b03e2e10} | 0 ...f_field-7-6aa3518e9f55299754521e959e9376ef | 2 - ...f_field-8-66dc6c81db0ac9b2075783b0d8976083 | 0 ...f_field-8-6aa3518e9f55299754521e959e9376ef | 2 + ..._field-9-66dc6c81db0ac9b2075783b0d8976083} | 0 ...f_field-9-f053f2d16812aa60b6dd1cab61e90a95 | 0 ..._in_set-0-18d3e88b18c18a00598146a3307276f2 | 1 - ...in_set-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._in_set-1-18d3e88b18c18a00598146a3307276f2 | 1 + ..._in_set-1-5fb7a7a1725749dc3853eb80fba19246 | 7 - ...in_set-10-16355c6b7e169b3c0ef506c149c6853c | 1 - ...n_set-10-df21f44247d7275a292520c1605c4aab} | 0 ...in_set-11-5a8515684c458d3fffea539a3d170e3a | 1 - ...n_set-11-91f8c37820f31d0d1b16029a59a185ad} | 0 ..._in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 | 1 - ...n_set-12-692f41c998bbc6bec0f374eae4356739} | 0 ..._in_set-13-2c986a80620c9238e1f663fc591760a | 1 - ...in_set-13-45e5ae8f60de2c41f189db7922a04917 | 1 + ...in_set-14-189def133b1871ce8345a8123811a6b5 | 1 - ...n_set-14-8e410ecfad2d408ad7d2554ccd3a6621} | 0 ...in_set-15-671bff8f50feea55015a8412fc6e5ceb | 1 - ...n_set-15-c50e6ff95c05bfa854b33b03db858cd9} | 0 ...n_set-16-8e17f41ae6e8b1075af4790a8fd88e13} | 0 ...in_set-16-d5d22082588c5fc30ef502237c5797f4 | 1 - ...in_set-17-5b556a29e1685605bcc47bce60cf66c8 | 2 - ...n_set-17-fe61f992f2d971d006155bdec3143803} | 0 ...in_set-18-5b556a29e1685605bcc47bce60cf66c8 | 2 + ..._in_set-2-5fb7a7a1725749dc3853eb80fba19246 | 7 + ..._in_set-2-b3823bdc04a7f98951b55c3e30d2a772 | 0 ..._in_set-3-132b7bc7812db7683eb3bff607275d0e | 25 - ...in_set-3-b3823bdc04a7f98951b55c3e30d2a772} | 0 ..._in_set-4-132b7bc7812db7683eb3bff607275d0e | 25 + ..._in_set-4-a35471c87ba597a6d3c7c880704cac0b | 1 - ...in_set-5-6f25b5bba89e1fcae171f5d595acc4ee} | 0 ..._in_set-5-ddaa3551dffb1169b2fbb671f373b82f | 1 - ..._in_set-6-591e070365f19c65e453b98b88f5f823 | 1 - ..._in_set-6-7bf387b94afb755faca4ad73bb7c42ba | 1 + ..._in_set-7-72d05b5cf99388d539adec38c40978c3 | 1 - ...in_set-7-730d5e95ef748ad946eceefbcd633826} | 0 ..._in_set-8-780771cad9bec96a216aea8ab293c941 | 1 - ...in_set-8-ea11724531f191940e455d13878a0e69} | 0 ...in_set-9-81f9999ed1b063ce7f17d89bd0162777} | 0 ..._in_set-9-d59f5aabe1ea0963f9328065c699d175 | 1 - ...f_float-0-7987032f1c9dcad07001445f3ca1f7a7 | 2 +- ...f_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b | 2 +- ...f_floor-0-e35abe1d5534243e96d71bd0c28761d6 | 2 +- ...f_floor-1-497a1ddbcf738aead319fde4f90f5248 | 2 +- ...number-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._number-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._number-1-525f133cfff86d44afdeeda667c365a5 | 1 - ...number-1-e86d559aeb84a4cc017a103182c22bfb} | 0 ...number-10-3bddca6913ea7e281e223b0603010b77 | 1 + ..._number-2-525f133cfff86d44afdeeda667c365a5 | 1 + ..._number-2-591f302d5c1cd24e153a598fa0b352fd | 4 - ..._number-3-591f302d5c1cd24e153a598fa0b352fd | 4 + ..._number-3-c89564db1ab953e28b050b9740f2650c | 0 ..._number-4-295d41a2146a27320c2be90499343260 | 1 - ...number-4-7969ffc4e80f7214a8eead8e1084368a} | 0 ..._number-5-881f33f6727a30629bde6e4b178cf7d9 | 1 - ..._number-5-da5bf00d45d2bd758489f661caffd4dc | 1 + ..._number-6-6dfca21d142652fec9017ba828a226c8 | 1 + ..._number-6-a6720a128716e179e18933992ca899b3 | 1 - ..._number-7-31eda4b0f31406add3a61e2503590113 | 1 + ..._number-7-84a460780828b0b9a2235314cfc24766 | 1 - ..._number-8-b297476c6348209933132202030eb8ea | 1 + ..._number-8-e7eedc849c74ce7d33c559067dd9ca0e | 1 - ..._number-9-407a0a7c277bb4c5c94ce16533ce1646 | 1 - ..._number-9-a21fbe58ff475634c8ed9829c6b8c187 | 1 + ...nixtime-0-c3adaeede5c48d232473d78acf0eed7f | 2 +- ...nixtime-1-d1a511d2084c7c621b5f638908c8db65 | 2 +- ..._object-0-c08e7139c00878b98d396e65d958100f | 1 - ..._object-1-706bcfd51431ec7f2b80145837f94917 | 16 - ..._object-2-2a18d9570d9b676e240cda76df818c42 | 0 ..._object-3-f60851dc36f579e83d6848d7d3c589e6 | 0 ..._object-4-4f08101fd66fb25d7b322d47773e49f3 | 0 ...erthan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...terthan-0-99d268829a124103cb3429c53fdc4de4 | 1 - ...terthan-1-8aab8e39726a986e10e1e572939fd63c | 1 - ...terthan-1-99d268829a124103cb3429c53fdc4de4 | 1 + ...terthan-2-79ba62f35a9432647e31c6722b44fe6f | 1 - ...terthan-2-8aab8e39726a986e10e1e572939fd63c | 1 + ...terthan-3-75fcadcdc6c050f1c7e70c71dc89c800 | 1 + ...requal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...orequal-0-a7214027a91abf6501881e2af313347a | 1 - ...orequal-1-3669f2008e7f428f365efadbcb5ae451 | 1 - ...orequal-1-a7214027a91abf6501881e2af313347a | 1 + ...orequal-2-3669f2008e7f428f365efadbcb5ae451 | 1 + ...orequal-2-d2690cc7713e91201bb10ef291c95819 | 1 - ...orequal-3-631662997e0c8d24d80b5d64a17446d2 | 1 + ...f_hash-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_hash-1-b9e3a3986320d275982797140edfccf4} | 0 ...f_hash-2-a18646b51501d0b1beb967dc79afbd1a} | 0 ...df_hash-2-cc121f3c38a7a522abd824940fe04285 | 0 ...f_hash-3-2646a87ce26c383a9dafea9b56281ab7} | 0 ...f_hash-4-d1368c2e3cd113e46202156b44811987} | 0 ...df_hex-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 | 1 - ...udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 | 1 + ...udf_hex-1-d55348c0ccd133b7abb690f6949b520c | 8 - ...udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 | 1 - ...udf_hex-2-d55348c0ccd133b7abb690f6949b520c | 8 + ...udf_hex-3-3a1de5be8ce350612ee6a93303107470 | 1 + ...udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf | 1 - ...udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 | 1 + ...udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea | 1 - ...udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e | 1 + ...df_hour-0-ba1c46a403b807fe0a28b85e62d869ce | 1 - ...df_hour-1-3db41b9fe9966a45e663bc42cf182c04 | 7 - ...udf_hour-2-ace1054795b20abd5ae829814cfe15a | 0 ...df_hour-3-415b0842ab0818c82baf9fbf07180613 | 1 - ...udf_hour-4-73bfac513b993dedbe143306865a44a | 1 - ...udf_if-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_if-1-b7ffa85b5785cccef2af1b285348cc2c} | 0 ...udf_if-2-30cf7f51f92b5684e556deff3032d49a} | 0 .../udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 | 0 ...udf_if-3-59e90bb74481aaf35480076806daf365} | 0 .../udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca | 0 ...udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb} | 0 ...udf_if-5-841a8068d35a42179d3654e1a2848c43} | 0 ...udf_if-6-508f9140dd33931c7b9ad336dceb32cf} | 0 .../udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 | 1 - .../udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 | 3 - ...in_file-0-1775b929e50cae8b3e957d99c5378f59 | 1 - ...in_file-1-2f23153970a569a4643574dde8d78a58 | 0 ...f_index-0-a277ac394cae40cb55d1ef3aa5add260 | 1 - ..._index-0-e91e3e5a22029b9b979ccbbef97add66} | 0 ...f_index-1-1f5e109131b0c67ebea521fa4902a8f6 | 1 - ...f_index-1-a277ac394cae40cb55d1ef3aa5add260 | 1 + ...f_index-2-1f5e109131b0c67ebea521fa4902a8f6 | 1 + ...f_instr-0-2e76f819563dbaba4beb51e3a130b922 | 1 - ..._instr-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_instr-1-2e76f819563dbaba4beb51e3a130b922 | 1 + ...f_instr-1-32da357fc754badd6e3898dcc8989182 | 4 - ...f_instr-2-10147893f38fc08dad4fa7f2bc843fc2 | 0 ...f_instr-2-32da357fc754badd6e3898dcc8989182 | 4 + ...f_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 | 1 - ..._instr-3-c40fbd09410b11388ce7a6e9bea5846f} | 0 ...f_instr-4-7017a441a31abc235d9359440cefda49 | 1 + ...udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba | 2 +- ...udf_int-1-3f0405ff93adfe8b3402b118567867d7 | 2 +- ...notnull-0-44584503014c378bb916b38e1879bfb6 | 2 +- ...snotnull-1-6ca2ea7938c7dac331c603ad343c1a7 | 2 +- ..._isnull-0-ac8e7827d760108923509f9ea1691d53 | 2 +- ..._isnull-1-55d9d04204f30cde4aa2667db88db262 | 2 +- ...notnull-0-ac8e7827d760108923509f9ea1691d53 | 1 - ...notnull-1-55d9d04204f30cde4aa2667db88db262 | 1 - ...notnull-2-44584503014c378bb916b38e1879bfb6 | 1 - ...snotnull-3-6ca2ea7938c7dac331c603ad343c1a7 | 1 - ...notnull-4-3dd03048c0152f565b21b6d3b7b010f1 | 0 ...notnull-5-253ed8f6f8659120af927871f57d81a1 | 1 - ...notnull-6-9daf0ab0e3b8315018341d6f72bd3174 | 0 ...notnull-7-bb1030dea18d2a0c2c00a6e2de835d6b | 1 - ..._method-0-991b98a25032b21802bc2a1efde606c7 | 1 - ..._method-1-a3b94d9f2c2caf85a588b6686a64630a | 3 - ..._method-2-69e6b8725086a8fb8f55721705442112 | 0 ..._method-3-c526dfd4d9eac718ced9afb3cf9a62fd | 1 - ...f_lcase-0-649df2b742e6a03d0e0e364f5bee76ad | 2 +- ...f_lcase-1-d947c47e03bedbfd4954853cc134c66e | 2 +- ..._length-0-38364b60c3a2409f53c9aa2dae19903b | 1 - ...length-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._length-1-38364b60c3a2409f53c9aa2dae19903b | 1 + ..._length-1-f183e1f8ae516bb483132ed106289b67 | 4 - ...ength-10-de456a5765db4a06110d9483985aa4a6} | 0 ...length-10-f3a9bd30540345db0f69b6847014b333 | 1 - ...length-11-f3a9bd30540345db0f69b6847014b333 | 1 + ..._length-2-af46cb6887618240836eaf5be8afbba6 | 0 ..._length-2-f183e1f8ae516bb483132ed106289b67 | 4 + ...length-3-af46cb6887618240836eaf5be8afbba6} | 0 ..._length-3-dcd6404afce1103d5054527e6c216d6d | 0 ..._length-4-ba49ba4e6505c74bc33b5003f3930c43 | 0 ...length-4-dcd6404afce1103d5054527e6c216d6d} | 0 ..._length-5-adc1ec67836b26b60d8547c4996bfd8f | 25 - ...length-5-ba49ba4e6505c74bc33b5003f3930c43} | 0 ..._length-6-460dffb0f1ab0ac0ebc4fd545809aa9a | 0 ..._length-6-adc1ec67836b26b60d8547c4996bfd8f | 25 + ...length-7-460dffb0f1ab0ac0ebc4fd545809aa9a} | 0 ..._length-7-8f28e6c488df47d89dca670f04a7563f | 0 ..._length-8-5e0fe761b7520651c3446ce7f9179caf | 0 ...length-8-8f28e6c488df47d89dca670f04a7563f} | 0 ..._length-9-de456a5765db4a06110d9483985aa4a6 | 0 ...length-9-e41b220da98996f997b26ba7ef457a84} | 0 ...ssthan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...essthan-0-a0d9e8b51e3d13685b3889db38f22427 | 1 - ...essthan-1-952c655a1092a410e5346f1205cb8142 | 1 - ...essthan-1-a0d9e8b51e3d13685b3889db38f22427 | 1 + ...essthan-2-92fa47f7af4a03ce1a965472eaad23a7 | 1 - ...essthan-2-952c655a1092a410e5346f1205cb8142 | 1 + ...essthan-3-677a1383983c94ba8008535b5a193153 | 1 + ...requal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...orequal-0-b3b021456c85da76d1879191886a425b | 1 - ...orequal-1-869d983466744ad73b109211e1638869 | 1 - ...orequal-1-b3b021456c85da76d1879191886a425b | 1 + ...orequal-2-56775013e20ecf2287e07e83eccf2e0c | 1 - ...orequal-2-869d983466744ad73b109211e1638869 | 1 + ...orequal-3-947dd56091ae1ef399ab32ce58317667 | 1 + ...f_like-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_like-1-e0ba9a953e50554bdcbc55585cffde09} | 0 ...f_like-2-9781f89d352c506e972ad2a1d58ec03a} | 0 ...df_like-2-dbc46cb33f0dd356af03006d9492f8b7 | 0 ...f_like-3-dbc46cb33f0dd356af03006d9492f8b7} | 0 ...f_like-4-bef03784eab9d5e8404fd24960dea4fc} | 0 ...f_like-5-47bfd4d65090dab890b467ae06cf3bd5} | 0 .../udf_ln-0-779eed5722a0efaa85efe24c559072b4 | 2 +- .../udf_ln-1-60e3541b3c703d6413869d774df9b7e4 | 2 +- ...locate-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._locate-0-6e41693c9c6dceea4d7fab4c02884e4e | 1 - ..._locate-1-6e41693c9c6dceea4d7fab4c02884e4e | 1 + ..._locate-1-d9b5934457931447874d6bb7c13de478 | 4 - ..._locate-2-849e7058dbbd7d422a9f3eb08d85b15c | 0 ..._locate-2-d9b5934457931447874d6bb7c13de478 | 4 + ..._locate-3-2a260e4b8e909eb5e848bf31a07f2531 | 1 - ...locate-3-ce4a131f99dc9befa926027380b38dbb} | 0 ..._locate-4-104cbfb3b59ad563810ddd7304a58b1b | 1 + ...udf_log-0-ca773bc1afa66218f3c13dee676bd87a | 2 +- ...udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 | 2 +- ...f_log10-0-35028570b378a2c7ea25b6bf6a4fac1f | 2 +- ...f_log10-1-abf1173290ef905d24d422faf7801fe3 | 2 +- ...df_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 | 2 +- ...df_log2-1-a79f0dce2cfc000b11a3b5299f02db56 | 2 +- ...boolean-0-2e7b9484514a049bbf72a4a0af5ee127 | 10 - ...f_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 | 2 +- ...df_lower-1-550f0a6da388596a775d921b9da995c | 2 +- ...f_lower-3-61b2e3e72180c80d52cf9bed18125e08 | 2 +- ...f_lpad-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_lpad-0-995646acf1e23cea7825412915921bef | 1 - ...df_lpad-1-995646acf1e23cea7825412915921bef | 1 + ...df_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 | 6 - ...df_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 | 0 ...df_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 | 6 + ...df_lpad-3-5b04264ae9ada1304acd0410ce31f2ae | 1 - ...f_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7} | 0 ...df_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 | 1 + ...f_ltrim-0-398a623504c47bcd64fe8d200c41402f | 2 +- ...udf_ltrim-1-658d495908097792a0e33a77becac2 | 2 +- ...df_map-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_map-1-9feb9f29115f94b3bb4f6a36160bd17e} | 0 ...df_map-2-1f8cd98df9bf7b2528506551fef87dcf} | 0 ...udf_map-2-a3f90085abab46205e732b4c27b18340 | 0 ...df_map-3-be7b52baa973b8b59b7ca63fea19aa99} | 0 ...df_map-4-60cb9c30285f7a9f99377ccbd143eb06} | 0 ...ap_keys-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...ap_keys-1-9a5714f8790219e9a9708a2c45fc87aa | 1 - ...ap_keys-2-731b529a9a234473312b7d1db15be75f | 1 - ...ap_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e | 1 - ...ap_keys-4-10cb8da631c1c058dacbbe4834a5928a | 1 - ..._values-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._values-1-a1d9810d9793d4ca2d17f539ca72bd08 | 1 - ..._values-2-ed39a40cbe55bb33d2bc19f0941dae69 | 1 - ..._values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 | 1 - ..._values-4-a000d06dd3941756b4bb9ccc46f3620e | 1 - ...udf_max-0-ac7d002a46f773ab680ed8c1ac97821f | 1 - ...udf_max-1-14afa1f14687893233a662f0f32a40c9 | 1 - ...df_max-10-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...df_max-11-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...df_max-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...df_max-13-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_max-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...udf_max-5-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...udf_max-6-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 | 1 - ...udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...udf_min-0-1a67398a47b4232c3786485b451d1ff8 | 1 - ...udf_min-1-69d749d0bca0ebe56e930009e30f4f19 | 1 - ...df_min-10-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...df_min-11-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...df_min-12-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ...df_min-13-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_min-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_min-4-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...udf_min-5-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...udf_min-6-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...udf_min-7-191613d4d46d1884d0694fcd8c5fb802 | 1 - ...udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 | 1 - ..._minute-0-9a38997c1f41f4afe00faa0abc471aee | 2 +- ..._minute-1-16995573ac4f4a1b047ad6ee88699e48 | 2 +- ..._minute-3-270055c684846e87444b037226cf554c | 2 +- ..._modulo-0-4e06551d4aa9464492e0f53374a280d5 | 2 +- ..._modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 | 2 +- ...f_month-0-9a38997c1f41f4afe00faa0abc471aee | 2 +- ...f_month-1-16995573ac4f4a1b047ad6ee88699e48 | 2 +- ...negative-0-1b770ec6fb07bb771af2231a9723ec8 | 1 - ...gative-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...negative-1-1b770ec6fb07bb771af2231a9723ec8 | 1 + ...egative-1-5f64266721b1ed31cfe84ee2f2377bdf | 1 - ...ative-10-1cd28efecc0d26f463221195f5e39956} | 0 ...egative-2-5f64266721b1ed31cfe84ee2f2377bdf | 1 + ...egative-2-a6863d2c5fc8c3131fe70080a011392c | 1 - ...egative-3-a6863d2c5fc8c3131fe70080a011392c | 1 + ...egative-3-b90eec030fee9cbd177f9615b782d722 | 1 - ...egative-4-b90eec030fee9cbd177f9615b782d722 | 1 + ...egative-4-e27bf3f44ccb2e051877da8a4455f50c | 1 - ...gative-5-771e76b0acd8ddb128781da7819d0e47} | 0 ..._negative-5-93d7dd808d4af59bda601faf249a9e | 1 - ...egative-6-6758b00c5acc7aac320238accf299219 | 1 - ...gative-6-f62c4a097c592871d896a7dc47c42f61} | 0 ...negative-7-6d8783f0ed7a4b7058c95f90da3fb4b | 1 - ...gative-7-f838053f5ca5c8746dc299473dff0490} | 0 ...egative-8-634af0478ed9ed44b851cd7ef834a489 | 1 - ...gative-8-f4f23aa6f634913d194a69261af8f3f6} | 0 ...egative-9-80b4c1fe00f7997838bba64a2cb5f8aa | 1 - ...gative-9-f6a78fa3ea0f519d0e4abc5be7a960e5} | 0 ...udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc | 2 +- ...udf_not-1-efefc8302b02224d20f4bb0f159a6911 | 2 +- ...udf_not-2-7e63750d3027ced0e3452ad4eb0df117 | 2 +- ...udf_not-3-aa0c674f9ce0feba86448448a211bd2a | 2 +- ...otequal-0-27c0d57f7c7c48ceb7bc671f7160254e | 1 - ...tequal-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...otequal-1-27c0d57f7c7c48ceb7bc671f7160254e | 1 + ...otequal-1-a7f0d1064f8f398ef504008015fddf9a | 2 - ...otequal-2-7d815b1218c85e4cf69d6780cab17520 | 1 - ...otequal-2-a7f0d1064f8f398ef504008015fddf9a | 2 + ...otequal-3-7d815b1218c85e4cf69d6780cab17520 | 1 + ...otequal-3-b72baeb22fad61bb31ce2d2e69375f57 | 2 - ...otequal-4-b72baeb22fad61bb31ce2d2e69375f57 | 2 + ...otequal-4-eb04e5ee00967799c913e8a5b424a332 | 0 ...otequal-5-e361b9cf294c4aa25018b081a2c05e07 | 499 ---- ...tequal-5-eb04e5ee00967799c913e8a5b424a332} | 0 ...otequal-6-46a6514f2d7e6a097035ec1559df0096 | 0 ...otequal-6-e361b9cf294c4aa25018b081a2c05e07 | 499 ++++ ...tequal-7-46a6514f2d7e6a097035ec1559df0096} | 0 ...otequal-7-a71fea4e9514cda0da9542a7701613dd | 499 ---- ...otequal-8-a71fea4e9514cda0da9542a7701613dd | 499 ++++ ..._notop-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_notop-0-825431072651228a5a9de7f85a0498d6 | 1 - ...f_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 | 1 + ...df_nvl-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 | 1 - ...udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 | 4 - ...udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 | 1 + ...udf_nvl-2-175ed7006e8907b65e0e5357f00a0def | 0 ...udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 | 4 + ...df_nvl-3-47199a1c23cb1cc6827c601bb66513d3} | 0 ...udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 | 1 - ...udf_nvl-4-656661e80deb75729fef313d5e2bd330 | 1 + .../udf_or-0-c404aa929eb0dd87269121f8f99ada70 | 2 +- .../udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 | 2 +- ...rse_url-0-7571c0423df7bf158ea9ca98142b26b8 | 2 +- ...rse_url-1-67adfb10d4a35c4d031f26adde9f61ab | 2 +- ...rse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 | 2 +- ...centile-0-8f99f54ff944f252e47d0af1f4ed1553 | 1 - ...centile-1-c0825a744cd14917d2c904d014449a4a | 1 - ...centile-2-1d351f7e821fcaf66c6f7503e42fb291 | 1 - ...centile-3-a7dc16cb82c595b18d4258a38a304b1e | 1 - ...df_pmod-0-ed67184beaf84c0542117c26651938e1 | 1 - ...df_pmod-1-90f75e01dcee85253a501d53b8562dae | 1 - ...f_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 | 1 - ...df_pmod-3-26d9546f030281a29a50a3e8e5858234 | 1 - ...df_pmod-4-7695df16d24a821224676e6bad3d66d1 | 1 - ...df_pmod-5-cf5311d51d44afb8d73f588e27d5e029 | 1 - ...udf_pmod-6-3c09a8da2f5645e732c22a45d055125 | 1 - ...df_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 | 1 - ...df_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd | 1 - ...df_pmod-9-e7280393102077442aa1d10eb69a6d57 | 1 - ...ositive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 | 2 +- ...ositive-1-1b98434a841d2248ed985c5f6ba2cc3c | 2 +- ...ositive-2-610d421e590f035c24e29694a68b0d23 | 2 +- ...ositive-3-400b238f4e6cdf7120be566b0ef079c5 | 2 +- ...udf_pow-0-c7f5178951dd45dc2a41c16729314d81 | 2 +- ...udf_pow-1-3c22c000c35144135aedbc7052f10803 | 2 +- ...df_power-0-57001d802c281743322d28bbc520cd4 | 2 +- ...f_power-1-ebd0398b2cb03f382a16382ddac13426 | 2 +- ..._printf-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._printf-1-19c61fce27310ab2590062d643f7b26e | 1 - ..._printf-2-25aa6950cae2bb781c336378f63ceaee | 4 - ..._printf-3-9c568a0473888396bd46507e8b330c36 | 0 ..._printf-4-91728e546b450bdcbb05ef30f13be475 | 1 - ..._printf-5-3141a0421605b091ee5a9e99d7d605fb | 1 - ..._printf-6-ec37b73012f3cbbbc0422744b0db8294 | 1 - ..._printf-7-5769f3a5b3300ca1d8b861229e976126 | 0 ...adians-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...radians-0-f899daf93b02ca681e0230a792c65e86 | 0 ...radians-1-58b73fc96927d447d1225f021eaa378} | 0 ...radians-1-eaaa62dd3935ff3152323dfafd136e93 | 1 - ...radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 5 + ...radians-2-bcaca433f704f71cf9d44c238a33c7b3 | 1 - ...radians-2-cb8462f25c18b7405c41a50e52476d04 | 1 + ...radians-3-65e16c7b13de48a5d36793d0c7d35e14 | 1 - ...radians-3-bd00297cb26f599913b14a635e768be3 | 1 + ...radians-4-65e16c7b13de48a5d36793d0c7d35e14 | 1 + ..._radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 4 - ..._radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 5 + ...radians-5-f899daf93b02ca681e0230a792c65e86 | 0 ...adians-6-70c9e7199b5898e2c3a4943ec58da113} | 0 ...radians-6-eaaa62dd3935ff3152323dfafd136e93 | 1 - ...radians-7-bcaca433f704f71cf9d44c238a33c7b3 | 1 - ...radians-7-cb8462f25c18b7405c41a50e52476d04 | 1 + ...radians-8-65e16c7b13de48a5d36793d0c7d35e14 | 1 - ...radians-8-bd00297cb26f599913b14a635e768be3 | 1 + ...radians-9-65e16c7b13de48a5d36793d0c7d35e14 | 1 + ..._radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 | 4 - ...df_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 | 2 +- ...df_rand-1-c6229b8f2ca3001663229cfb8ee4763e | 2 +- ...reflect-0-904138e2a1f831c308b7f0aacc859ae1 | 1 - ...reflect-1-21ec7caa253c7f95b7cf60191140e2ee | 4 - ...reflect-2-b868357466bab2f04685c2dc73604cf0 | 0 ...eflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 | 1 - ...eflect2-1-c5a05379f482215a5a484bed0299bf19 | 3 - ...eflect2-2-effc057c78c00b0af26a4ac0f5f116ca | 0 ...regexp-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...regexp-1-19917611f74aedc0922560f7f2595948} | 0 ...regexp-2-f7f0527cd47612d7f256edd5f8963800} | 0 ...regexp-3-59aff54bae544ee620141e4e629f167a} | 0 ...extract-0-e251e1a4b1e191814f26c54b14ab6cd9 | 2 +- ...extract-1-8add879ab5904bd805412ef8723276fb | 2 +- ...replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e | 2 +- ..._replace-1-e79b45aa220d3c4c3b4523ac9c897bc | 2 +- ...repeat-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._repeat-0-fdeae3e88f27ef148281d69ac8c4b23b | 1 - ..._repeat-1-836be47190989d8975a09a545ecbfe0b | 4 - ..._repeat-1-fdeae3e88f27ef148281d69ac8c4b23b | 1 + ..._repeat-2-836be47190989d8975a09a545ecbfe0b | 4 + ..._repeat-2-e1dbea7182ec1653e1123b5b67a6d20a | 0 ...repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec} | 0 ..._repeat-3-ba9dd02f59c74d63d60d60b6231a0365 | 1 - ..._repeat-4-64c5fce0c5ad4c26680a842aa740dc57 | 1 + ...f_rlike-0-6ec6ef55ac041208627454e16b501d38 | 2 +- ...f_rlike-1-829611a596e0c87431006f7247d25eca | 2 +- ...f_round-0-10b53ca1f15fd7879365926f86512d15 | 1 - ..._round-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_round-1-10b53ca1f15fd7879365926f86512d15 | 1 + ...f_round-1-2367bcc43510dedc80bdb6707e434da8 | 4 - ...f_round-2-2367bcc43510dedc80bdb6707e434da8 | 4 + ...f_round-2-9ffa2b573360cd879338de46d91ab374 | 1 - ...f_round-3-42a221909d3f7ed51bed01a65670461c | 1 - ...f_round-3-fa4d11da8e1eba258ed191ed5f1447de | 1 + ...f_round-4-b87ccaa1e0a87c558b56d59a8a074396 | 1 + ...f_round-4-dc80ec5189a4c6ce07688df3debe7de4 | 1 - ...f_round-5-441d0075081ae87579c959d714c4922d | 1 + ...f_round-5-a2414e7845ffafc61f75465508a1850a | 1 - ...f_round-6-4658ec3bc034b43e0477bf2474939449 | 1 + ...f_round-6-48439efa5c34e7589ab5003ed916f12b | 1 - ...df_round-7-74ff5a4862c80bd8fd84bede1a0320d | 1 + ...round_2-0-91afaf77ef4061fa20005a3c87dfef32 | 0 ...round_2-1-5e44354af73865d03e9088c0232f16ce | 0 ...round_2-2-83f91f60dcb2036b61b8b21f18281298 | 1 - ...round_2-3-c62cf7b74a91f605cf669e34a9315f93 | 1 - ...round_2-4-797fa8ed05cb953327d0f6306b30d6c3 | 1 - ...round_2-5-e41b862db8cc76c1fe004cf006fad50b | 1 - ...ound_3-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...round_3-0-8415af605db167315e4d9d3c69d89e6c | 1 - ...round_3-1-15a7f123f596e28e6f238063ba4e3d6d | 1 + ...round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 | 1 - ...round_3-2-5e7ada966f726ceb748f98c20eab4f10 | 1 - ...round_3-2-a03bf4e99027d4814a32c84d89d42cca | 1 + ...round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 | 1 + ...round_3-3-e94ab3326df006c7203ead86752f16a9 | 1 - ...round_3-4-8449fbdabbb4b1e6beab89be0af498f2 | 1 + ...round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 | 1 - ..._round_3-5-3844531c8cff115d6a33636db0a26ad | 1 + ...f_rpad-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_rpad-0-56de896c5fe8b40b22a9ed55ed79889c | 1 - ...df_rpad-1-48d17e5d5d6188300d048f987fab2ca0 | 6 - ...df_rpad-1-56de896c5fe8b40b22a9ed55ed79889c | 1 + ...udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 | 0 ...df_rpad-2-48d17e5d5d6188300d048f987fab2ca0 | 6 + ...f_rpad-3-66acb969c28a8e376782ccd0d442b450} | 0 ...df_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 | 1 - ...df_rpad-4-299dee5a72aad2a2738d7841a89bb71b | 1 + ...f_rtrim-0-7acca21c725995febdf2a9c1fdf0535a | 2 +- ...f_rtrim-1-66d61255134c09d37cbfedd757ae47fd | 2 +- ...second-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._second-0-e004a6f20da3fa6db868ec847b217ff2 | 1 - ..._second-1-3525f55f4f13253c42b3abaa53d77888 | 7 - ..._second-1-e004a6f20da3fa6db868ec847b217ff2 | 1 + ..._second-2-3525f55f4f13253c42b3abaa53d77888 | 7 + ..._second-2-d678372e3837a16be245d2e33482f17f | 0 ..._second-3-2496e4d3c64ca028184431c2930d82cf | 1 - ...second-3-d678372e3837a16be245d2e33482f17f} | 0 ..._second-4-2496e4d3c64ca028184431c2930d82cf | 1 + ...ntences-0-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...ntences-1-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...tences-10-ce188a53f69129c14cbf378d2c3f6630 | 0 ...tences-11-3c8672506e23434187caf4e0064a8a80 | 31 - ...tences-12-d55c04a079ca97402509868f24921685 | 1 - ...ntences-2-1f218343f90e698fb9ed81c4f51d3d14 | 0 ...ntences-3-ce188a53f69129c14cbf378d2c3f6630 | 0 ...ntences-4-3c8672506e23434187caf4e0064a8a80 | 50 - ...ntences-5-60823f4912be26bee1a0b52a0a9588a9 | 0 ...ntences-6-27b7eeae683a87507f35e61fd4ce67de | 0 ...ntences-7-ec0dad44fa033691a731f6e4c6b5cf7f | 0 ...ntences-8-2dc07f4f0f0cb20d08c424e067ed8f69 | 0 ...ntences-9-68c61b4882802e416d5adaa2de440b59 | 0 ...df_sign-0-14f3c3034ac8039fc81681003bbf5e0e | 0 ...f_sign-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9} | 0 ...df_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 | 1 - ..._sign-10-9a5326b0bf612fed4ce0b04770bebc16} | 0 ...f_sign-10-fc7341f89c3cd3c43e436242d8aa61fa | 1 - ...f_sign-11-74237f5ecc497813cf9738b21647847a | 4 - ...f_sign-11-fc7341f89c3cd3c43e436242d8aa61fa | 1 + ...f_sign-12-74237f5ecc497813cf9738b21647847a | 4 + ...df_sign-2-85b743f0eed93904789cde4d1b5eafef | 1 - ...df_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 | 1 + ...df_sign-3-9eeb3985359429abba5d1dd702c66b0d | 1 - ...df_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 | 1 + ...f_sign-4-9a5326b0bf612fed4ce0b04770bebc16} | 0 ...df_sign-4-fc7341f89c3cd3c43e436242d8aa61fa | 1 - ...df_sign-5-74237f5ecc497813cf9738b21647847a | 4 - ...df_sign-5-fc7341f89c3cd3c43e436242d8aa61fa | 1 + ...df_sign-6-14f3c3034ac8039fc81681003bbf5e0e | 0 ...df_sign-6-74237f5ecc497813cf9738b21647847a | 4 + ...df_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 | 1 - ...f_sign-7-ed2aaa1a416c0cccc04de970424e1860} | 0 ...df_sign-8-85b743f0eed93904789cde4d1b5eafef | 1 - ...df_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 | 1 + ...df_sign-9-9eeb3985359429abba5d1dd702c66b0d | 1 - ...df_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 | 1 + ...udf_sin-0-40b50393869eb0bcde66e36fe41078ee | 1 - ...df_sin-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_sin-1-2f867f432fb322e21dce353d7eb50c63 | 4 - ...udf_sin-1-40b50393869eb0bcde66e36fe41078ee | 1 + ...udf_sin-2-2f867f432fb322e21dce353d7eb50c63 | 4 + ...udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab | 1 - ...df_sin-3-1d3a615e3aa252a317daa601811820b1} | 0 ...udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 | 1 - ...udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a | 1 + ...df_size-0-9c49a52514b1b940a7e8bb93c35eda62 | 1 - ...df_size-1-3608160636eaa7e053171bdcefc0b1a8 | 1 - ...df_size-2-96d41c9f054662827d1b6b63f5dd8db7 | 0 ...df_size-3-63df892215b6ce189d6c3e20cfc4bdbd | 1 - ...mallint-0-f28e857ef74c967303855c21dc60c042 | 2 +- ...mallint-1-37d8db74267ae370d6a076b3057c5ed6 | 2 +- ...t_array-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...t_array-1-976cd8b6b50a2748bbc768aa5e11cf82 | 1 - ...t_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 | 4 - ...t_array-3-55c4cdaf8438b06675d60848d68f35de | 0 ...t_array-4-3edb0151fae0622cb79cd04156cb4c44 | 1 - ...t_array-5-5d7dfaa9e4137938559eb536e28f8d0e | 1 - ...t_array-6-f754ac1296d16e168abb3d0ebcc35bd3 | 1 - ...t_array-7-a9c52a8684cefc095470f5a93f63d2a8 | 1 - ...t_array-8-d79f0084177230a7a2845c4791c22d25 | 0 ...t_array-9-45ef2679e195a269878527d5f264488a | 0 ..._space-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...f_space-0-91e879c9f01d90eab7bf12fcef256010 | 1 - ...f_space-1-91e879c9f01d90eab7bf12fcef256010 | 1 + ...f_space-1-e4eaf5e96807e122548cb43be9a26754 | 4 - ...f_space-2-a23a06eef37709e8587647a74bbfa7e8 | 0 ...f_space-2-e4eaf5e96807e122548cb43be9a26754 | 4 + ...f_space-3-59903e27d8188d6209e007ff643d5956 | 1 - ..._space-3-a1b9dad63547f7ba73a5230d650983b0} | 0 ...f_space-4-7adb35ad867ba3e980d937a0038ac1a5 | 1 - ...f_space-4-d9de5746edd753507c3f054e1bee7702 | 1 + ...f_space-5-ce5288dcc60f9412109930bd56752a65 | 1 + ...f_split-0-7accac7fc71ba74d61c01a69d3978338 | 1 - ...f_split-1-ebe303dfc3db504deffe5c355efd0fd1 | 4 - ...f_split-2-7bba11f8fc359f2d4b863cda11c643f9 | 0 ...f_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 | 1 - ...df_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 | 2 +- ...df_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 | 2 +- ...udf_std-0-e3613484de2b3fa707995720ec3f8a5b | 2 +- ...udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 | 2 +- ..._stddev-0-ad7627185d89a60b83ce19966eddbc92 | 2 +- ..._stddev-1-18e1d598820013453fad45852e1a303d | 2 +- ...dev_pop-0-96788538f1f20eb879a1add4bb5f9d12 | 2 +- ...dev_pop-1-6286ef94de26050617bf69c17a3b4a10 | 2 +- ...dev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 | 2 +- ...ev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 | 2 +- ...dev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 | 2 +- ...ev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 | 2 +- ..._string-0-17412ad1c1a827411caa7b5e891b6ac3 | 2 +- ..._string-1-53b00551846b7f8bb27874b3a466e68d | 2 +- ..._substr-0-20fb50d79b45264548b953e37d837fcd | 1 - ..._substr-1-2af34de7fd584c5f1ead85b3891b0920 | 10 - ..._substr-2-d0268ad73ab5d94af15d9e1703b424d1 | 1 - ..._substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 | 1 - ..._substr-4-f7933c0bb28e9a815555edfa3764524a | 1 - ..._substr-5-4f29c1273b8010ce2d05882fc44793d8 | 1 - ..._substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 | 1 - ..._substr-7-1742c72a1a8a6c6fd3eb2449d4747496 | 1 - ..._substr-8-ba6ca6bac87ca99aabd60b8e76537ade | 1 - ..._substr-9-a9aa82080133620d017160f6a644455d | 1 - ...bstring-0-8297700b238f417dea2bd60ba72a6ece | 2 +- ...bstring-1-e5df65419ecd9e837dadfcdd7f9074f8 | 2 +- ...ubtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 | 2 +- ...ubtract-1-b90eec030fee9cbd177f9615b782d722 | 2 +- ...udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 | 2 +- ...udf_sum-1-ddae1a511d3371122ab79918be5b495b | 2 +- ...udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 | 2 +- ...udf_sum-3-ddae1a511d3371122ab79918be5b495b | 2 +- ...df_tan-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...udf_tan-0-c21aa640b4edabf6613dd705d029c878 | 1 - ...udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 - ...udf_tan-1-c21aa640b4edabf6613dd705d029c878 | 1 + ...udf_tan-2-27a29c68f846824990e9e443ac95da85 | 1 - ...udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 + ...udf_tan-3-77bedd6c76bdc33428d13a894f468a97 | 1 - ...df_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b} | 0 ...udf_tan-4-769fde617744ccfaa29cefec81b8704c | 1 + ...udf_tan-4-c21aa640b4edabf6613dd705d029c878 | 1 - ...udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 - ...udf_tan-5-c21aa640b4edabf6613dd705d029c878 | 1 + ...udf_tan-6-27a29c68f846824990e9e443ac95da85 | 1 - ...udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 | 4 + ...udf_tan-7-77bedd6c76bdc33428d13a894f468a97 | 1 - ...df_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b} | 0 ...udf_tan-8-769fde617744ccfaa29cefec81b8704c | 1 + ...tlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad | 10 - ...length2-0-3da4fe901124f2bbf3d02822652c4e55 | 10 - ...tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d | 2 +- ...tinyint-1-417de1aeb44510aa9746729f9ff3b426 | 2 +- ...o_byte-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 | 1 - ...o_byte-1-94eb069fb446b7758f7e06386486bec9} | 0 ...to_byte-2-233102b562824cf38010868478e91e1} | 0 ...o_byte-3-5dc0e4c21764683d98700860d2c8ab31} | 0 ...to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 | 1 - ...o_byte-4-dafb27507b4d30fd2231680f9ea80c82} | 0 ...o_byte-5-eb6600cd2260e8e75253e7844c0d7dc2} | 0 ...o_byte-6-489cd2d26b9efde2cdbff19254289371} | 0 ...o_byte-7-1eaba393f93af1763dd761172fb78d52} | 0 ...o_byte-8-8fe36cf8fba87514744a89fe50414f79} | 0 ...o_byte-9-322163c32973ccc3a5168463db7a8589} | 0 ...to_date-0-ab5c4edc1825010642bd24f4cfc26166 | 2 +- ...to_date-1-da3c817bc5f4458078c6199390ac915e | 4 +- ..._double-2-69bf8a5a4cb378bbd54c20cb8aa97abe | 1 - ..._double-3-ab23099412d24154ff369d8bd6bde89f | 1 - ..._double-4-293a639a2b61a11da6ca798c04624f68 | 1 - ..._double-5-42d1e80bb3324030c62a23c6d1b786a8 | 1 - ..._double-6-5bac1a7db00d788fd7d82e3a78e60be6 | 1 - ..._double-7-97080ab9cd416f8acd8803291e9dc9e5 | 1 - ..._double-8-df51146f6ef960c77cd1722191e4b982 | 1 - ...o_float-2-39a67183b6d2a4da005baed849c5e971 | 1 - ...o_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce | 1 - ...o_float-4-f3e0ab8ed691a386e6be4ce6993be507 | 1 - ...o_float-5-75f364708c01b5e31f988f19e52b2201 | 1 - ...o_float-6-1d9b690354f7b04df660a9e3c448a002 | 1 - ...o_float-7-cdfefa5173854b647a76383300f8b9d1 | 1 - ...o_float-8-5b0a785185bcaa98b581c5b3dbb3e12c | 1 - ...o_long-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...to_long-1-61e6679e5a37889bc596590bde0228f0 | 1 - ...o_long-1-8aeb3ba62d4a0cecfff363741e8042f6} | 0 ...o_long-2-7e8fa1ae8d00a121ec14941a48d24947} | 0 ...o_long-3-6e5936fba8e7486beb9ab998548bbe9b} | 0 ...o_long-4-8c284b082a256abf0426d4f6f1971703} | 0 ...o_long-5-6bb29b93f6b0f1427ba93efb4e78810a} | 0 ...o_long-6-290b5a4ce01563482e81b3b532ebf9db} | 0 ...o_long-7-da20f84586dac3e50ee9d5b9078f44db} | 0 ...o_long-8-90f068b4b6275bdd1c4c431fb7fa90e2} | 0 ...o_long-9-cc8b79539085fe0e00f672b562c51cd0} | 0 ..._short-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...o_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a | 1 - ...o_short-1-32c4a61f9166c2a9d7038553d589bd9b | 1 - ..._short-1-94f15fe043839493107058a06a210cf7} | 0 ..._short-2-981e4cb6654fde7eb4634c7ad72f8570} | 0 ..._short-3-b259ee30ecf279bb4ad12d1515ca2767} | 0 ...o_short-4-40ffb132d5641645e2b8043dc056fb0} | 0 ..._short-5-5d6c46b0154d1073c035a79dbf612479} | 0 ..._short-6-6561b41835a21f973cbbc2dd80eef87f} | 0 ..._short-7-9f83813005b639a23901ca6ff87ff473} | 0 ..._short-8-885656e165feb3a674cf636dbf08716c} | 0 ..._short-9-750382fa1a1b3ed5dca0d549d3a68996} | 0 ...o_string-0-a032eb8f819689a374852c20336d5cc | 1 - ..._string-1-b461f0e6e98991aede40c7c68561dc44 | 1 - ..._string-2-bc6b37e1d5a8d28df8a957c88104c9a5 | 1 - ..._string-3-ddbb829eedc6aa1e27169303a7957d15 | 1 - ..._string-4-c20301e9bbf10143bb9bf67cd7367c21 | 1 - ...to_string-5-1040b37847d20ef29d545934316303 | 1 - ..._string-6-4181d264a7af3c6488da2f1db20dc384 | 1 - ..._string-7-567bc77f96e7dc8c89bae912e9a3af15 | 1 - ..._string-8-a70b03d79ebd989c8141f9d70dbca8ea | 1 - ..._string-9-51677fbf5d2fc5478853cec1df039e3b | 1 - ...nslate-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...anslate-0-7fe940a2b26fa19a3cfee39e56fb1241 | 1 - ...anslate-1-7fe940a2b26fa19a3cfee39e56fb1241 | 1 + ...anslate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 | 13 - ...nslate-10-2f9daada8878081cb8954880ad5a65c4 | 1 - ...nslate-10-ca7c17e78c6a3d4e19dbd66622a87eae | 1 + ...nslate-11-40c4e7adff4dde739d7797d212892c5a | 1 + ...nslate-11-76b7a339d5c62808b9f4f78816d4c55b | 1 - ...nslate-12-a5b3e5fadeec1e03371160879f060b05 | 1 - ...nslate-12-d81fd3267ec96cff31079142bf5d49bf | 1 + ...nslate-13-26085a3eba1a1b34684ec4e6c1723527 | 1 + ...anslate-2-42aba80bf1913dd7c64545831f476c58 | 0 ...anslate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 | 13 + ...anslate-3-20904c8be8fed5cbd2d66ead6248a60a | 0 ...nslate-3-42aba80bf1913dd7c64545831f476c58} | 0 ...nslate-4-20904c8be8fed5cbd2d66ead6248a60a} | 0 ...anslate-4-5d4abaf86254bacaa545c769bd7e50ba | 0 ...nslate-5-5d4abaf86254bacaa545c769bd7e50ba} | 0 ...anslate-5-f2637240d227f9732d3db76f2e9d3a59 | 0 ...anslate-6-55122cc5ea4f49e737fee58945f3f21b | 1 - ...nslate-6-f2637240d227f9732d3db76f2e9d3a59} | 0 ...anslate-7-856c9d0157c34ab85cc6c83d560bfd47 | 1 - ...anslate-7-f8de3ab54db5d6a44fddb542b3d99704 | 1 + ...anslate-8-1747ed8fbb4ef889df3db937ee51e2b0 | 1 + ...anslate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 | 1 - ...anslate-9-ee69663d7662760973b72785595be2b1 | 1 - ...anslate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 | 1 + ...df_trim-0-18aa2b7ff8d263124ea47c9e27fc672f | 2 +- ...df_trim-1-e23715e112959e6840b6feed2ecf38a7 | 2 +- ...f_ucase-0-8f8c18102eb02df524106be5ea49f23d | 2 +- ...f_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f | 2 +- ...f_unhex-0-11eb3cc5216d5446f4165007203acc47 | 1 - ...f_unhex-1-a660886085b8651852b9b77934848ae4 | 14 - ...df_unhex-2-78ba44cd7dae6619772c7620cb39b68 | 1 - ...f_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 | 1 - ...f_union-0-e3498ef95bc9d8c8ce55e75962b4a92c | 1 - ...f_union-1-f6f01250718087029165e23badc02cd6 | 4 - ...f_union-2-6af20858e3209d7cf37f736805ec5182 | 0 ...f_union-3-705d165fec6761744dd19b142c566d61 | 2 - ...estamp-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...estamp-1-d555c8cd733572bfa8cd3362da9480cb} | 0 ...mestamp-2-28c40e51e55bed62693e626efda5d9c5 | 0 ...estamp-2-8a9dbadae706047715cf5f903ff4a724} | 0 ...estamp-3-28c40e51e55bed62693e626efda5d9c5} | 0 ...mestamp-3-732b21d386f2002b87eaf02d0b9951ed | 0 ...estamp-4-6059ff48788d0fb8317fd331172ecea9} | 0 ...estamp-5-b2e42ebb75cecf09961d36587797f6d0} | 0 ...estamp-6-31243f5cb64356425b9f95ba011ac9d6} | 0 ...estamp-7-9b0f20bde1aaf9102b67a5498b167f31} | 0 ...estamp-8-47f433ff6ccce4c666440cc1a228a96d} | 0 ...f_upper-0-47dc226b0435f668df20fe0e84293ead | 2 +- ...f_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d | 2 +- ...var_pop-0-3187e740690ccc1988a19fea4202a6de | 2 +- ...var_pop-1-fd25e5226312bf54d918858511814766 | 2 +- ...ar_samp-0-b918928871d1b7f944315558c230c229 | 2 +- ...ar_samp-1-59032ed5856fd4aa17c3e8e6721eec2b | 2 +- ...ar_samp-2-b918928871d1b7f944315558c230c229 | 2 +- ...ar_samp-3-59032ed5856fd4aa17c3e8e6721eec2b | 2 +- ...ariance-0-fd23be1baa8b5ffa0d4519560d3fca87 | 2 +- ...ariance-1-c1856abae908b05bfd6183189b4fd06a | 2 +- ...ariance-2-3187e740690ccc1988a19fea4202a6de | 2 +- ...ariance-3-fd25e5226312bf54d918858511814766 | 2 +- ...ariance-4-fd23be1baa8b5ffa0d4519560d3fca87 | 2 +- ...ariance-5-c1856abae908b05bfd6183189b4fd06a | 2 +- ...ariance-6-3187e740690ccc1988a19fea4202a6de | 2 +- ...ariance-7-fd25e5226312bf54d918858511814766 | 2 +- ...ofyear-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...kofyear-0-d6b4490b549a358be375511e39627dc2 | 1 - ...kofyear-1-b7bbdfabe6054a66701250fd70065ddd | 6 - ...kofyear-1-d6b4490b549a358be375511e39627dc2 | 1 + ...kofyear-2-57ec578b3e226b6971e0fc0694b513d6 | 1 - ...kofyear-2-b7bbdfabe6054a66701250fd70065ddd | 6 + ...kofyear-3-d5dd3abb6c8c7046a85dd05f51126285 | 1 + ...f_when-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_when-0-88b97c6722176393e9b3d089559d2d11 | 1 - ...df_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 | 1 - ...df_when-1-88b97c6722176393e9b3d089559d2d11 | 1 + ...df_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 | 1 + ...df_when-2-ff1118e7d06a4725e1e98a6d70a59295 | 0 ...f_when-3-734890c41528b9d918db66b0582228a4} | 0 ...df_when-3-e63043e8d6ecf300c1fcf8654176896f | 1 - ...df_when-4-c57d6eb11efc29ce3a9c450488f3d750 | 1 + ...df_when-5-6ed21e998c0fc32c39f6375136f55de6 | 1 + ..._xpath-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...df_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 | 1 - ...f_xpath-1-5a6f10392053a4eabe62f5cabb073a71 | 8 - ...df_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 | 1 + ...f_xpath-2-5a6f10392053a4eabe62f5cabb073a71 | 8 + ...f_xpath-2-6b0a9d3874868d88d54ae133c978753d | 1 - ...f_xpath-3-5700d81a9c2a22dcca287bf8439277ea | 1 - ...f_xpath-3-f0b9adf99c68290c86d0f40f45111e18 | 1 + ...f_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a | 1 - ...f_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e | 1 + ...f_xpath-5-7395e1cd3b543316a753978f556975e0 | 1 - ...f_xpath-5-b66a64f91dd812fda2859863855988b5 | 1 + ...f_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f | 1 - ...f_xpath-6-d4f95ebc0057639138900722c74ee17a | 1 + ...f_xpath-7-6b8fceac3bd654662f067da60670e1d9 | 1 + ...oolean-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 | 1 - ...boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 | 1 + ...boolean-1-9e935539c9a3f3a118145096e7f978c4 | 6 - ...boolean-2-66148c16fde3008348a24cec5d9ba02b | 1 - ...boolean-2-9e935539c9a3f3a118145096e7f978c4 | 6 + ...oolean-3-9b97a08303a7a89e7575687f6d7ba435} | 0 ...boolean-3-c6cf7ede46f0412fe7a37ac52061b060 | 1 - ...oolean-4-284ce7d6fc6850ca853111145784286b} | 0 ...boolean-4-45d2e8baee72a0d741eb0a976af3a965 | 1 - ...oolean-5-95a0dfa897ba9050ad751a78aeb72f3d} | 0 ...boolean-5-dbc57416174a6ba26f29dfc13f91f302 | 1 - ...boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f | 1 - ...oolean-6-796c717c9d52d3efa374d12fe65259e6} | 0 ...boolean-7-824c07ed4ef40cd509fea55575e43303 | 1 - ...boolean-7-cc3ae9a7505e04a2e9b950442a81a559 | 1 + ...boolean-8-62a1af222d4e12c59cef71b979c6e58} | 0 ..._double-0-39199612969071d58b24034a2d17ca67 | 1 - ...double-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ..._double-1-2d5ceab0a14d6e837ed153e1a2235bb2 | 5 - ..._double-1-39199612969071d58b24034a2d17ca67 | 1 + ...double-10-ad0f9117f6f52590d38e619e998a6648 | 1 + ...double-10-fe9ae5013ee4f11b357128ee5ffb56c0 | 1 - ...double-11-a1960676c1b2dc76aa178ea288d801d2 | 1 - ...double-11-bf1f96ce71879c17d91ac9df44c36d29 | 1 + ...double-12-9621353ce5c583ca34216d357c5bb0eb | 1 + ..._double-2-2d5ceab0a14d6e837ed153e1a2235bb2 | 5 + ..._double-2-a4d22cea9dffaf26b485b3eb08963789 | 1 - ..._double-3-a4d22cea9dffaf26b485b3eb08963789 | 1 + ..._double-3-ea050c5b9e74d4b87b037236ef9e4fc2 | 5 - ..._double-4-93f47057c68385cff3b6f5c42307590c | 1 - ..._double-4-ea050c5b9e74d4b87b037236ef9e4fc2 | 5 + ..._double-5-1e0514d71f99da09e01a414a4e01d046 | 1 + ..._double-5-c811a2353f5baf585da8654acd13b0e5 | 1 - ..._double-6-10fcb281fd8db12dd1ac41380b8030c6 | 1 - ..._double-6-6a3985167fedd579f7bd5745133a3524 | 1 + ..._double-7-4532934141d2adb1f4ff56e65e8cf14c | 1 - ..._double-7-d97e93fb4b69522300f505e04b6674c8 | 1 + ..._double-8-547e750f5e401511db56283e66d1231d | 1 - ..._double-8-fce6cafa07b75c9843c1c1964e84fc10 | 1 + ..._double-9-b45b8b60031ac43ed2ffcd883ba2f19e | 1 - ..._double-9-f27361521c35bf4f6581dba4c185d550 | 1 + ..._float-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...h_float-0-7483dafee0dc7334eecabba31977e791 | 1 - ...h_float-1-7483dafee0dc7334eecabba31977e791 | 1 + ...h_float-1-f6ddff2daba006787aeb861ca6f9d37a | 4 - ..._float-10-89ae28cf8e6b6f422d307a0085774cab | 1 + ...h_float-2-70b7180f7bcd1d09727ad73084788d16 | 1 - ...h_float-2-f6ddff2daba006787aeb861ca6f9d37a | 4 + ...h_float-3-79b7e57efe31a0373c39f0ba79b5bd54 | 1 - ...h_float-3-b743a9cb9f8688220e0a6346856f18ce | 1 + ...h_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 | 1 + ...h_float-4-6e1f61ebe10c5fde60148e3a31706352 | 1 - ...h_float-5-410760f9560157587fbba7a677e12b9f | 1 - ...h_float-5-f5e1100f6e8de31081042413b4039fb2 | 1 + ...h_float-6-5e8457209d15467af7f14c09dfadb894 | 1 + ...h_float-6-d83a5002f52e3acb7dbb725bb434eebf | 1 - ...h_float-7-580ffe4dabef758c4fcb52050b315615 | 1 + ...h_float-7-b57c58f9343a82846f54ef60b216dfaa | 1 - ...h_float-8-3a62c730d4e5d9be6fb01307577d6f48 | 1 - ...h_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 | 1 + ...h_float-9-16793f703b552fcb24f7aea9bb8a2401 | 1 - ...h_float-9-bf66b884da717d46f0b12c3b8cf8313a | 1 + ...th_int-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...ath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 | 1 - ...ath_int-1-26ddf7e46a66065909e9e547f0459910 | 4 - ...ath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 | 1 + ...th_int-10-ea25feb474c40c6985152093d4dbb13a | 1 + ...ath_int-2-26ddf7e46a66065909e9e547f0459910 | 4 + ...ath_int-2-f10e246ebfd5f82545a3df64f51f58ba | 1 - ...th_int-3-a04ed6832ab542d6ee5903039511a826} | 0 ...ath_int-3-eaa4a790c726cfa76a247d08a6ee7aba | 1 - ...ath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 | 1 - ...th_int-4-bf5a4dbb7a98abc91111a3798b56809f} | 0 ...ath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 | 1 - ...ath_int-5-f49db0ecb889722ec68d1753c084b0e1 | 1 + ...ath_int-6-9bfa4fdc7d551742ff23efad8507ba0a | 1 - ...th_int-6-ac509f06f01c02924adef220404fc515} | 0 ...th_int-7-87ff12d650afb8f511d906778450fae7} | 0 ...ath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 | 1 - ...ath_int-8-a175811eca252aa77c16a44fbb0ee7b2 | 1 + ...ath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 | 1 - ...ath_int-9-7da88f589199f5ca873780fb22614573 | 1 + ...ath_int-9-e6c2548098879ee747a933c755c4c869 | 1 - ...h_long-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...th_long-0-d274b272251e95ec2e8139bfa471bd0b | 1 - ...th_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 | 4 - ...th_long-1-d274b272251e95ec2e8139bfa471bd0b | 1 + ...h_long-10-caeac94758a40493a5227fcdb8ec2f87 | 1 + ...th_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 | 4 + ...th_long-2-d697d943b1f7e7762d804064d11b905b | 1 - ...h_long-3-9df8d27e31d96e0b35b9b40910d4bd98} | 0 ...th_long-3-c0b20c651ae5a352322067f97bd6ae5d | 1 - ...h_long-4-3211913c56521887d30e3d1a50762b3f} | 0 ...ath_long-4-821e1cdea837bee7a8c2745bc3b85b9 | 1 - ...th_long-5-22a71b88c5bcb3db6e299a88ab791d4d | 1 - ...th_long-5-d580a8f01a546baddd939b95722e6354 | 1 + ...h_long-6-b695348ed3faec63be2c07d0d4afaaf3} | 0 ...th_long-6-f2460325cf46c830631d8bc32565c787 | 1 - ...th_long-7-b3f1d4b505151180b82fddb18cf795d0 | 1 - ...h_long-7-ed5af6d7451107a753b2c7ff130ac73b} | 0 ...th_long-8-2f952b56682969bb203fa3d9102f7015 | 1 + ...th_long-8-c21bebec7d1a4aec99fba6b0a9a03083 | 1 - ...th_long-9-7bafedc7b884df49a9f6752360969bf1 | 1 - ...th_long-9-947b6e08ba9c7defd75d00412f9bc4fd | 1 + ..._short-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...h_short-0-7d5231aed9cbbf68cd740791f9e5be17 | 1 - ...th_short-1-5d45932563b78e0b435b8cfebfe3cc2 | 4 - ...h_short-1-7d5231aed9cbbf68cd740791f9e5be17 | 1 + ..._short-10-b537709676634250e13914e76cd9a530 | 1 + ...h_short-2-10e9d4899d2fd352b58010c778c1f7a8 | 1 - ...th_short-2-5d45932563b78e0b435b8cfebfe3cc2 | 4 + ...h_short-3-5ac84f46610107b1846f4a9b26a69576 | 1 - ..._short-3-c09b4ae6886fa58dcdd728bef45e7efa} | 0 ..._short-4-84f5f6bb2fdc0987d281d52a53a4b24e} | 0 ...h_short-4-8a300079521fefbe0d2f943851c1c53c | 1 - ...h_short-5-51f5de44cf1d5289fa5892ffe16e473e | 1 - ..._short-5-c09fd0565ed041c773fee9bd0436e861} | 0 ..._short-6-16ced3de15d4ec87a4e7001376551758} | 0 ...h_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 | 1 - ..._short-7-8ffdf20c15f3ed81bb5a92c61d200ae2} | 0 ...h_short-7-e24ee123f331429c22de0a06054d0d5d | 1 - ...h_short-8-102ad2dea8d94528b402d980a45d53d4 | 1 + ...h_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 | 1 - ...h_short-9-22088ba0fb00eaa28e3460ca018b343e | 1 - ...h_short-9-d571e18b7d8ad44fef2e0b2424f34a0d | 1 + ...string-0-50131c0ba7b7a6b65c789a5a8497bada} | 0 ...string-1-e315d11e9feb29177b5cb1e221c7cfa4} | 0 ...tring-10-d87fb71039c9d2419d750a0721c5696f} | 0 ...string-2-110b583cde6cd23c486d8223c444cbe9} | 0 ...string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4} | 0 ...string-4-302630fe7dac2cc61fe7d36ead0f41ab} | 0 ...string-5-19357ba9cb87d3a5717543d2afdc96e2} | 0 ...string-6-4837e470f745487fae4f498b3a2946bc} | 0 ...string-7-8e42951d002e3c4034b4a51928442706} | 0 ...string-8-fdfa4e17d70608dcc634c9e1e8a8f288} | 0 ...string-9-84109613320bd05abccd1058044d62c3} | 0 ...f_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a | 1 - ...f_stack-1-879ca1a8453ced55a8617b390670a4e1 | 0 ...f_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 | 0 ...otation-1-3de206f543c9e1525c54547f076b99c3 | 10 +- ...otation-4-3de206f543c9e1525c54547f076b99c3 | 10 +- ...otation-7-3de206f543c9e1525c54547f076b99c3 | 10 +- ...union10-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union10-4-7f83822f19aa9b973198fe4c42c66856 | 2 +- ...union11-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union11-2-cd756f39e22e121bdbd51400662aa47f | 2 +- ...union12-0-863233ccd616401efb4bf83c4b9e3a52 | 1 - ...union12-1-9d2793d1cfd2645ac7f373a0a127e599 | 0 ...union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 | 0 ...union13-1-534d0853c5fc094404f65ca4631c1c20 | 2 +- ...union14-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union14-2-8e01b2f4a18ad41a622e0aadbe680398 | 2 +- ...union15-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- ...union15-2-a89acfb4bbc044c483b94e28152a41e0 | 2 +- ...union16-1-9f76074598f9b55d8afbb5659737a382 | 2 +- ...union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 | 2 +- ...union17-5-404392d6faff5db5f36b4aa87ac8e8c9 | 2 +- ...union18-4-1799ebb147238db6032fd6fe2fd36878 | 2 +- ...union18-5-b12dcddfa4f02a14318f6564947c98a0 | 2 +- ...union19-4-1799ebb147238db6032fd6fe2fd36878 | 2 +- ...union19-5-b12dcddfa4f02a14318f6564947c98a0 | 2 +- .../union2-1-90d739774cb96e7d0d96513c1c9968b4 | 2 +- ...union20-1-968e353589f1fddb914242beb25be94c | 2 +- ...union21-0-ecfd22e2a24ed9f113229c80a2aaee9c | 0 ...union21-1-fb1497f4c21bf7d28162f27d50320d13 | 536 ---- ...union23-1-7830963417e3535034962e2597970ddd | 2 +- ...union27-3-ab84df3813ff23be99f148449610e530 | 2 +- ...union28-3-b1d75ba0d33a452619e41f70e69616e9 | 2 +- ...union29-3-b1d75ba0d33a452619e41f70e69616e9 | 2 +- ...union30-3-b1d75ba0d33a452619e41f70e69616e9 | 2 +- ...nion31-14-c36a1d8de2713f722ec42bc4686d6125 | 2 +- ...nion31-15-5df6435aed6e0a6a6853480a027b911e | 2 +- ...nion31-24-df38c8164af7cc164c728b8178da72c5 | 2 +- ...nion31-25-1485e295a99908e1862eae397b814045 | 2 +- ...union31-8-ba92b89786ffaecd74a740705e0fa0cb | 2 +- ...union31-9-56dfdb30edd8a687f9aa9cad29b42760 | 2 +- ...union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e | 0 ...union32-1-e6c80e7d6171ae5fc428506e57dc8753 | 0 ...nion32-10-44a9b373ee7d43a4ef2bc4b8a708601b | 0 ...nion32-11-40d51bc9958b27c36ef647f0598fdee5 | 32 - ...union32-2-8e39fef33c859ef83912d0fcda319218 | 0 ...union32-3-d31e252450077ac54f4cb18a9ad95a84 | 20 - ...union32-4-79787e084ca15d479cee3a7e1ed2281e | 0 ...union32-5-51c997d0a1103b60764bbb8316a38746 | 32 - ...union32-6-96fa13d8790bbfa1d6109b7cbf890d1b | 0 ...union32-7-f936440d63f4e1027dda4de605660633 | 32 - ...union32-8-f42d83f502a7b8d3a36331e0f5621cfb | 0 ...union32-9-74fb695786df4c024288ae23ac8c00db | 32 - ...nion34-10-da2b79118c21ac45ce85001fa61b0043 | 2 +- ...union34-4-101829a66cab2efd31dcb0d86e302956 | 0 ...nion34-4-70479e10c016e5ac448394dbadb32794} | 0 ...union34-5-24ca942f094b14b92086305cc125e833 | 2 +- ...union34-7-da2b79118c21ac45ce85001fa61b0043 | 2 +- ...union34-8-b1e2ade89ae898650f0be4f796d8947b | 2 +- .../union4-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union4-4-7f83822f19aa9b973198fe4c42c66856 | 2 +- .../union5-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union5-2-2c19c8d564b010eeb42deee63d66a292 | 2 +- .../union6-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union6-4-a03959cc5aaa8f6521a73e6dae04cd15 | 2 +- .../union7-0-863233ccd616401efb4bf83c4b9e3a52 | 2 +- .../union7-2-55d6e503a281acf3289a7874c0fba3f5 | 2 +- .../union8-1-1b422e4c1c8b97775518f760b995c771 | 2 +- .../union9-1-a77ee9f723b3b17a3a02164c5d0000c1 | 2 +- ...ion_date-4-d812f7feef3b6857aeca9007f0af44c | 0 ...n_date-4-d85fe746334b430941c5db3665e744d4} | 0 ...n_date-5-82eebfded24cef08e0a881d1bcca02b1} | 0 ...on_date-5-b54839e0200bec94aa751fec8c5dbd3d | 0 ...on_null-0-27e98c4939abf1ad4445b4e715b0262a | 10 - ...remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_1-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_1-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_1-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_1-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_1-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_1-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...remove_1-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...emove_1-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_1-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_1-9-19865a08066d80cb069ae6312c465ee6 | 0 ...emove_10-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_10-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_10-10-7eaf13bc61bd4b43f4da67c347768598 | 0 ...ove_10-11-b62595b91d2d9e03a010b49ab81725d5 | 0 ...move_10-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_10-13-43d53504df013e6b35f81811138a167a | 1 - ...move_10-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_10-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_10-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_10-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_10-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_10-7-a7ba0436265932086d2a2e228356971 | 0 ...move_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...move_10-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_11-0-3c29684bfd2df7439ee0551eb42cfa0 | 2 +- ...move_11-1-16a6a293f1d2ce481b1d2482b1d5787c | 2 +- ...move_11-12-ea111d286c70e4a0c6a68a7420dc7b7 | 17 +- ...ove_11-13-43d53504df013e6b35f81811138a167a | 2 +- ...ove_11-14-e409e7032445097ace016b1876d95b3e | 2 +- ...move_11-2-cafed8ca348b243372b9114910be1557 | 2 +- ...move_11-3-b12e5c70d6d29757471b900b6160fa8a | 2 +- ...move_11-4-593999fae618b6b38322bc9ae4e0c027 | 2 +- ...move_11-5-6f53d5613262d393d82d159ec5dc16dc | 2 +- ...move_11-6-a2a411ad6620aa1ab24550ade336e785 | 2 +- ...move_11-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_11-9-94da21f150ed2c56046b80e46da8884d} | 0 ...emove_12-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_12-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_12-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_12-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...ove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...move_12-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_12-14-43d53504df013e6b35f81811138a167a | 1 - ...move_12-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_12-3-24ca942f094b14b92086305cc125e833 | 1 - ...move_12-4-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_12-5-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_12-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_12-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_12-8-9dd030d38eece4630dec5951fc8a0622 | 0 ...move_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_13-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_13-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_13-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 | 0 ...ove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af | 0 ...move_13-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_13-14-43d53504df013e6b35f81811138a167a | 1 - ...move_13-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_13-3-24ca942f094b14b92086305cc125e833 | 1 - ...move_13-4-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_13-5-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_13-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_13-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_13-8-234ecbc3f8c0e4686d3586f81cf191eb | 0 ...move_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_14-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_14-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_14-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_14-11-a667f24e26435cd2a29fef0ee45bab3c | 0 ...ove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 | 0 ...move_14-13-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_14-14-43d53504df013e6b35f81811138a167a | 1 - ...move_14-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_14-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_14-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_14-5-24ca942f094b14b92086305cc125e833 | 1 - ...move_14-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_14-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_14-8-ed33d620523b2634285698a83f433b6d | 0 ...move_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_15-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_15-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_15-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_15-11-93b7341b523213ab6e58169459bc6818 | 0 ...ove_15-12-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...move_15-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_15-14-37f9f7bc2d7456046a9f967347337e47 | 2 - ...ove_15-15-43d53504df013e6b35f81811138a167a | 1 - ...move_15-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_15-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_15-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_15-5-16367c381d4b189b3640c92511244bfe | 1 - ...move_15-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_15-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_15-8-552c4eba867e7408fc8652ff0a19170d | 0 ...move_15-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...emove_16-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_16-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_16-10-a63925fb2aa8c5df6854c248e674b0ef | 0 ...ove_16-11-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_16-12-93b7341b523213ab6e58169459bc6818 | 0 ...ove_16-13-616cc477ed00e691dbc2b310d1c6dd12 | 0 ...move_16-14-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_16-15-37f9f7bc2d7456046a9f967347337e47 | 2 - ...ove_16-16-43d53504df013e6b35f81811138a167a | 1 - ...move_16-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_16-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...move_16-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...move_16-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...move_16-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_16-7-16367c381d4b189b3640c92511244bfe | 1 - ...move_16-8-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_16-9-ec47ba0fc527a4a04d452a009d59147a | 0 ...emove_17-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_17-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_17-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 | 0 ...ove_17-12-c91289e16ad403babfc91c093ac9b86d | 0 ...move_17-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_17-14-37f9f7bc2d7456046a9f967347337e47 | 2 - ...ove_17-15-43d53504df013e6b35f81811138a167a | 1 - ...ove_17-16-626a252f75285872c1d72706f7f972c6 | 6 - ...ove_17-17-1252985379f11ae4b98d2a0e2f121b8a | 6 - ...move_17-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_17-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_17-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_17-5-16367c381d4b189b3640c92511244bfe | 1 - ...move_17-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_17-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_17-8-67e57f56d1106a57329bf75c491e3c8b | 0 ...move_17-9-a63925fb2aa8c5df6854c248e674b0ef | 0 ...emove_18-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_18-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_18-10-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...ove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad | 0 ...ove_18-12-461a24ba6c00f60d75afba62da2ac4f1 | 0 ...move_18-13-ea111d286c70e4a0c6a68a7420dc7b7 | 32 - ...ove_18-14-37f9f7bc2d7456046a9f967347337e47 | 6 - ...ove_18-15-43d53504df013e6b35f81811138a167a | 1 - ...move_18-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_18-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_18-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_18-5-16367c381d4b189b3640c92511244bfe | 1 - ...move_18-6-a4fb8359a2179ec70777aad6366071b7 | 1 - ...move_18-7-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_18-8-b1b996b2e72ca14150af7c82dbc6e139 | 0 ...move_18-9-ea467d0fee062a23c720cf47eacfef08 | 0 ...emove_19-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_19-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_19-10-8d1e5af37e7992708bf15ab7d887405b | 0 ...move_19-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_19-12-68e15b1729669c4cb2291dcabfea4387 | 10 - ...ove_19-13-b8b9df8f376df228e3c8ae65defe2801 | 0 ...ove_19-14-eed866a1ad3106a306322519f4bb52f2 | 0 ...ove_19-15-68e15b1729669c4cb2291dcabfea4387 | 2 - ...ove_19-16-471f8e794fd712dce2e40334b383e08e | 0 ...ove_19-17-ae0f7d5734ca67cbfebed70c4657e330 | 0 ...ove_19-18-43d53504df013e6b35f81811138a167a | 1 - ...move_19-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_19-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_19-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_19-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_19-6-a6c043a89a9c3456af8ee065cb17239 | 0 ...move_19-7-48f70528347f5201f387d28dae37a14a | 0 ...move_19-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_19-9-8a08edd1aa63fd3b051da82246793259 | 0 ...remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_2-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_2-10-2309570010c3e679b884c100de57d002 | 0 ...emove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_2-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_2-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_2-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_2-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_2-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...remove_2-6-1acf02741a2ff987d3e00ae9722c26e | 0 ...emove_2-7-48f70528347f5201f387d28dae37a14a | 0 ...emove_2-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...emove_20-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_20-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_20-10-89c57c91facbf54299e08955e3783ea6 | 0 ...move_20-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_20-12-43d53504df013e6b35f81811138a167a | 1 - ...move_20-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_20-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_20-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_20-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_20-6-a73143117ffec1225f6d492e5aa577e | 0 ...move_20-7-82f81adc097c247475fd29076e0cb85f | 0 ...move_20-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_20-9-683949ae07de12da0b7e7ba7f4450daa | 0 ...emove_21-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_21-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_21-10-e19469a1b70be25caaf670fe68f0a747 | 0 ...move_21-11-ea111d286c70e4a0c6a68a7420dc7b7 | 26 - ...ove_21-12-43d53504df013e6b35f81811138a167a | 1 - ...move_21-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_21-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_21-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_21-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_21-6-aeba356b56f8659963d8b2dc07a84a6f | 0 ...move_21-7-5716c408db679fb88352eaceb1703bd7 | 0 ...move_21-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_21-9-deadbce171926623b0586587fbbcd144 | 0 ...emove_22-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_22-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_22-10-403471d96c56b565272d2e4c4926d240 | 0 ...move_22-11-ea111d286c70e4a0c6a68a7420dc7b7 | 28 - ...ove_22-12-68e15b1729669c4cb2291dcabfea4387 | 10 - ...ove_22-13-a28b876b5df29140ef2bf62b4d0de3fd | 0 ...ove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af | 0 ...ove_22-15-43d53504df013e6b35f81811138a167a | 1 - ...move_22-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_22-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_22-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_22-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_22-6-722acd65700dd132dc6b4bc8c56f4ce0 | 0 ...move_22-7-46da090f5a2c73b175207cf63ff46653 | 0 ...move_22-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_22-9-b3edbbee0543ff268db4059afb9cb2cb | 0 ...emove_23-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_23-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_23-10-613ce50facecdc8d7bf8806a8ff17c13 | 0 ...move_23-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_23-12-43d53504df013e6b35f81811138a167a | 1 - ...move_23-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_23-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_23-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_23-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 | 0 ...move_23-7-48f70528347f5201f387d28dae37a14a | 0 ...move_23-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_23-9-2dca12ca51c74540e7cdbbb05e336ed5 | 0 ...emove_24-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...move_24-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...ove_24-10-6d89089b1eead05510dbccad5fcc4805 | 0 ...move_24-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...ove_24-12-43d53504df013e6b35f81811138a167a | 1 - ...move_24-2-cafed8ca348b243372b9114910be1557 | 1 - ...move_24-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...move_24-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...move_24-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...move_24-6-72ae7a9640ae611f61ac954ac1a4b682 | 0 ...move_24-7-44a7b0f14b4f5151c37498367ad7fe1e | 0 ...move_24-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_24-9-4c0550cc9c28de25993c1f98de39168f | 0 ...remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 | 2 +- ...emove_3-1-16a6a293f1d2ce481b1d2482b1d5787c | 2 +- ...emove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 | 17 +- ...move_3-12-43d53504df013e6b35f81811138a167a | 2 +- ...move_3-13-e409e7032445097ace016b1876d95b3e | 2 +- ...emove_3-2-cafed8ca348b243372b9114910be1557 | 2 +- ...emove_3-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ...emove_3-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...emove_3-5-a2a411ad6620aa1ab24550ade336e785 | 2 +- ...emove_3-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_3-8-94da21f150ed2c56046b80e46da8884d} | 0 ...remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_4-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_4-10-19865a08066d80cb069ae6312c465ee6 | 0 ...move_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_4-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_4-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_4-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...emove_4-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...emove_4-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_4-6-6f53d5613262d393d82d159ec5dc16dc | 1 - ...emove_4-7-90fcd4eb330919ad92aecb8a5bf30ead | 0 ...emove_4-8-48f70528347f5201f387d28dae37a14a | 0 ...emove_4-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_5-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_5-10-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...move_5-11-2309570010c3e679b884c100de57d002 | 0 ...emove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_5-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_5-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_5-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...emove_5-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...emove_5-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...emove_5-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_5-7-4da6ca94af4caf4426e5676a32b70375 | 0 ...emove_5-8-48f70528347f5201f387d28dae37a14a | 0 ...emove_5-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 | 2 +- ...emove_6-1-16a6a293f1d2ce481b1d2482b1d5787c | 2 +- ...move_6-12-43d53504df013e6b35f81811138a167a | 2 +- ...move_6-13-e409e7032445097ace016b1876d95b3e | 2 +- ...move_6-14-f74b28904e86047150396bc42680ca38 | 2 +- ...emove_6-2-cafed8ca348b243372b9114910be1557 | 2 +- ...emove_6-3-dc129f70e75cd575ce8c0de288884523 | 2 +- ...emove_6-4-a572a07cd60fd4607ddd7613db8a64ab | 2 +- ...emove_6-5-a2a411ad6620aa1ab24550ade336e785 | 2 +- ...emove_6-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...move_6-9-94da21f150ed2c56046b80e46da8884d} | 0 ...remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_7-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 | 0 ...emove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_7-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_7-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_7-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_7-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_7-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_7-6-3744f0ebf5c002fdfcec67fbce03dfca | 0 ...emove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_7-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_7-9-19865a08066d80cb069ae6312c465ee6 | 0 ...remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_8-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_8-10-2309570010c3e679b884c100de57d002 | 0 ...emove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_8-12-43d53504df013e6b35f81811138a167a | 1 - ...emove_8-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_8-3-dc129f70e75cd575ce8c0de288884523 | 1 - ...emove_8-4-a572a07cd60fd4607ddd7613db8a64ab | 1 - ...emove_8-5-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_8-6-895f2432b4be6fcb11641c1d063570ee | 0 ...emove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_8-8-8ff0bb1bf3da91b51d37923f1876be0e | 0 ...emove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f | 0 ...remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 | 1 - ...emove_9-1-16a6a293f1d2ce481b1d2482b1d5787c | 1 - ...move_9-10-12cf3335c756f8715a07c5a604f10f64 | 0 ...move_9-11-4e84cd589eceda668833f8f19ec28e7c | 0 ...emove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 | 27 - ...move_9-13-43d53504df013e6b35f81811138a167a | 1 - ...emove_9-2-cafed8ca348b243372b9114910be1557 | 1 - ...emove_9-3-b12e5c70d6d29757471b900b6160fa8a | 1 - ...emove_9-4-593999fae618b6b38322bc9ae4e0c027 | 1 - ...emove_9-5-6f53d5613262d393d82d159ec5dc16dc | 1 - ...emove_9-6-a2a411ad6620aa1ab24550ade336e785 | 1 - ...emove_9-7-f77ac921b27860fac94cac6b352f3eb5 | 0 ...emove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 | 0 ...emove_9-9-8ff0bb1bf3da91b51d37923f1876be0e | 0 ..._script-0-ca195b08d98d7f21fe93208499bf0ff6 | 2 +- ..._script-1-982cc6d7b98f8fb1055a10ef021e2769 | 2 +- ...on_view-0-e56367a21517656c18a5bcfeecb4327d | 0 ...on_view-1-c790d4344144460224b0f02be7e137a8 | 0 ...on_view-2-7e33b0744f57bdc8ebcd9d45348aef14 | 0 ...on_view-3-4a746bd076e063017c4d6a2f9218a6e4 | 0 ...on_view-4-d3d75f376f83b694b1dc62c46fa53f4e | 0 ...on_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 | 0 ...on_view-6-c9d7dcde469d3b9a66965a64dd15e4ae | 1 - ...on_view-7-3b03210f94ec40db9ab02620645014d1 | 1 - ...on_view-8-35f48c7d6fa164bb84643657bc9280a8 | 1 - ...rchar_2-3-a8c072e5b13997e9c79484b4af9d78da | 2 +- ...rchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f | 2 +- ...rchar_2-5-96353c24b5f2b361f72e5c26b4135519 | 2 +- ...rchar_2-6-2b62789d07b4044bc32190261bf3490f | 2 +- ..._join1-10-1958143ee083437e87662cadb48c37ce | 2 +- ...r_join1-11-a55f750032663f77066e4979dedea1c | 2 +- ...har_join1-6-6bb08c5baa913d9dc506aef65425ef | 0 ..._join1-6-ab4392aa5ff499ec43229425ff23e22f} | 0 ...r_join1-7-341118dab140b17f0b9d2c7b101d1298 | 0 ..._join1-7-a01639290aaceb4b85aa6e44319f6386} | 0 ...r_join1-8-afe7304d94450481c01ddbaf6cc3f596 | 0 ..._join1-8-c05d7b534b51cecdc2ba2de4ce57ba37} | 0 ...r_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 | 2 +- ...d_types-0-e41b0e9db7a9dbd0540e29df341933bc | 0 ...d_types-1-87a28b0e800f051525899324a064b878 | 0 ...d_types-10-92f25849990eb777ac2711f9dd2e628 | 1 - ..._types-11-9780781a92fdd992f3cee080a8717238 | 1 - ..._types-12-e1a4006971319a352280fc52eabf449f | 0 ..._types-13-d4574217a243a7d506398a819cd0eab4 | 0 ..._types-14-7c33a62195359bc89460ad65f6a5f763 | 1 - ...d_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 | 1 - ..._types-16-93811d146a429c44a2494b3aa4b2caa2 | 0 ..._types-17-5724af3985c67a0cb69919c9bbce15dc | 0 ..._types-18-d1be2ee3765a80469837ba11eb8685e1 | 1 - ..._types-19-a54f9a284228e7cdce8c34b8094f2377 | 1 - ...d_types-2-ae69b82461acc2aa366d8b1f8626d6fb | 0 ..._types-20-51a9279006f61097e68a52201daf6710 | 0 ..._types-21-1d304e09cc2a8b2824bfc04bdbc976ad | 1 - ..._types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 | 1 - ..._types-23-c7ea918777b725f2790da4fa00a3efa6 | 0 ..._types-24-5a69236334a3a1c4e771206cf547d730 | 1 - ..._types-25-61b030bb2220a533532d871ae0e08cdb | 1 - ..._types-26-e41b0e9db7a9dbd0540e29df341933bc | 0 ..._types-27-87a28b0e800f051525899324a064b878 | 0 ..._types-28-ae69b82461acc2aa366d8b1f8626d6fb | 0 ..._types-29-39d44d19c7963a9647fd3293eef670d4 | 0 ...d_types-3-39d44d19c7963a9647fd3293eef670d4 | 0 ..._types-30-162806477075d97de16dfa6f2576b751 | 0 ..._types-31-5b28e1fdb28b365ef419008a4752ed53 | 0 ...d_types-4-162806477075d97de16dfa6f2576b751 | 0 ...d_types-5-5b28e1fdb28b365ef419008a4752ed53 | 0 ...d_types-6-a47b99c355df4aad56dceb7f81a7fd5b | 0 ...d_types-7-8c483a7a0e148ca13a292a625f8702f1 | 0 ...d_types-8-94309e2b4e68ab7e25f7d9656f10b352 | 0 ...d_types-9-f248796769bc7f57cf56a75034a45520 | 0 ...r_serde-0-750a23ebdd77f32b555d4caba7ac5445 | 0 ...r_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ..._serde-10-496280d9365ee601d9b68a91495d7160 | 0 ..._serde-11-2f4d5f574f659df2bd0fb97f2d50a36e | 0 ..._serde-12-80727f22f1343407ba9200c86ed84280 | 0 ..._serde-13-4829e6e5822c73fb33ba3d619b4bd31e | 5 - ..._serde-14-4794ee93811ce3a81f64c65aed0b8b13 | 5 - ..._serde-15-6ae634e1ae1aa0730a33396bce5a6604 | 0 ..._serde-16-516202183287d734d35d8c7788d22652 | 0 ..._serde-17-8d4419222a728e2bbc6a464b5a0b5f7a | 0 ..._serde-18-b2dfd28d6a5b1e6f4556ad19755b739d | 5 - ..._serde-19-f258df2db09e9cc0e049e85e6ad950ad | 5 - ...r_serde-2-87ba3f40293b9c79fcdb3064d964232e | 0 ..._serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 | 0 ..._serde-21-64536c77ae91bfb6cf7f93f178c6200b | 0 ...r_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 | 0 ..._serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 | 5 - ..._serde-24-770169b632b76cedcd6dfb87fdc46575 | 5 - ..._serde-25-3470a259b04e126c655531491787e2fc | 0 ..._serde-26-55808e190e0ab81dcdc1feb52543ad9f | 0 ..._serde-27-8fe526fdd347c25529a383f27ad20566 | 0 ..._serde-28-5e4de93349ba89a8344bb799ad60678e | 5 - ..._serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c | 5 - ...r_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ..._serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a | 0 ..._serde-31-db1e88efcd55aaef567edaa89c1c1e12 | 0 ..._serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb | 0 ..._serde-33-620729dc7661c22f1acdc425a7cf0364 | 5 - ..._serde-34-807ee73e1cd66704dd585f7e0de954d9 | 5 - ..._serde-35-750a23ebdd77f32b555d4caba7ac5445 | 0 ..._serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d | 0 ..._serde-37-87ba3f40293b9c79fcdb3064d964232e | 0 ..._serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 | 0 ..._serde-39-8b5f81c27c41807c757522e257a8003f | 0 ...r_serde-4-8b5f81c27c41807c757522e257a8003f | 0 ..._serde-40-787193a1679a2153c037d3e4c8192bba | 0 ...r_serde-5-787193a1679a2153c037d3e4c8192bba | 0 ...r_serde-6-122f15d410249b554e12eccdfa46cc43 | 0 ...r_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 | 0 ...r_serde-8-3bfa13085b001c1a333cc72d5c9f4244 | 5 - ...r_serde-9-ec43be632e5f74057aba54c4f562c601 | 5 - ...union1-10-6ec48d5fea3e4a35275956b9b4467715 | 2 +- ...union1-11-78f6e219b974e1fdf3663e46f57892a9 | 2 +- ..._union1-6-67e66fa14dddc17757436539eca9ef64 | 0 ...union1-6-f338f341c5f86d0a44cabfb4f7bddc3b} | 0 ..._union1-7-48766d09c5ed1b6abe9ce0b8996adf36 | 0 ...union1-7-ea0d1fbae997b50dc34f7610480bbe29} | 0 ..._union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 | 0 ...union1-8-f3be9a2498927d692356c2cf871d25bf} | 0 ..._union1-9-b9455ffec62df97cfec63204ce02a110 | 2 +- .../view-0-5528e36b3b0f5b14313898cc45f9c23a | 0 .../view-1-7650b86c86dd6b1a99c86ddc5a31bd63 | 0 .../view-10-7aae4448a05e8a8a3bace7522e952cd0 | 0 .../view-11-dc95343d3e57846485dd543476391376 | 0 .../view-12-371764e1cae31ea0518c03060528d239 | 0 .../view-13-2abce88008f8a19164758ee821aaa8a6 | 0 .../view-14-deb504f4f70fd7db975950c3c47959ee | 0 .../view-15-6f2797b6f81943d3b53b8d247ae8512b | 0 .../view-2-9c529f486fa81a032bfe1253808fca8 | 0 .../view-3-89c80c0e90409d5e304775c9f420915a | 0 .../view-4-4a64d1a623ca71e515796787dbd0f904 | 0 .../view-5-f6d1bce095ecbf1aa484891392fdb07b | 0 .../view-6-47b5043f03a84695b6784682b4402ac8 | 0 .../view-7-8b1bbdadfd1e11af1b56064196164e58 | 0 .../view-8-60d2f3ee552ae7021f9fa72f0dcf2867 | 0 .../view-9-66c68babac10ae0f645fe8334c5a42d4 | 0 ...ew_cast-0-89bd46ad04f967f1f5ee17c6f201aacf | 0 ...ew_cast-1-85685a26971fb51ab6e28f9c5e8421bb | 0 ...w_cast-10-a7c865e5180df8d73dba90ede8be0d45 | 0 ...ew_cast-2-af2050aa97f0cd930cb1b8ec791007de | 0 ...iew_cast-3-2a232c31d056e6abc76f8ebe53ccd97 | 0 ...ew_cast-4-d9edb83f4cf847e141d97012314917d4 | 0 ...ew_cast-5-6db508ccd85562a9ca7841fb0a08981a | 0 ...ew_cast-6-aa5be3380ddc7104258567b406d93cc5 | 0 ...ew_cast-7-78ac3800b22682b31708b6a09b402bfb | 0 ...ew_cast-8-2cc0c576f0a008abf5bdf3308d500869 | 0 ...ew_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 | 0 ..._inputs-0-9e67dfd1d595ab8b1935b789645f76c0 | 0 ..._inputs-1-5af97e73bc3841793440105aae766bbe | 0 ..._column-0-9bacd1908e56d621913a74fe9a583d9d | 500 ---- ..._column-1-3ebad682d4ff6ca9c806db3471cf3945 | 309 -- ...column-10-2915b222a58bc994246591e536d388b4 | 0 ..._column-2-1536b365fe0a94b30a62364996529966 | 309 -- ..._column-3-c66776673c986b59b27e704664935988 | 0 ..._column-4-e47094c927b1091e31c185db0a4e69a6 | 1 - ..._column-5-d137fa7c27bc98d5f1a33f666a07f6b7 | 0 ..._column-6-68d6973677af5c9f1f1f49360c3175e7 | 1 - ..._column-7-20d7d672a4289fbd1a5de485a8353ac6 | 0 ..._column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 | 500 ---- ...l_column-9-c152da33c1517ecfc336f72b9c133d5 | 0 sql/hive/src/test/resources/log4j.properties | 6 + .../clientcompare/vectorized_math_funcs.q | 43 + .../clientcompare/vectorized_math_funcs_00.qv | 1 + .../clientcompare/vectorized_math_funcs_01.qv | 1 + .../alter_concatenate_indexed_table.q | 6 +- .../alter_partition_invalidspec.q | 4 +- .../clientnegative/alter_partition_nodrop.q | 4 +- .../alter_partition_nodrop_table.q | 4 +- .../clientnegative/alter_partition_offline.q | 4 +- .../alter_rename_partition_failure.q | 2 +- .../alter_rename_partition_failure2.q | 2 +- .../alter_rename_partition_failure3.q | 2 +- .../queries/clientnegative/ambiguous_col1.q | 1 + .../queries/clientnegative/ambiguous_col2.q | 1 + .../clientnegative/analyze_non_existent_tbl.q | 1 + .../queries/clientnegative/archive_corrupt.q | 2 +- .../clientnegative/authorization_addjar.q | 7 + .../authorization_addpartition.q | 10 + .../authorization_alter_db_owner.q | 11 + .../authorization_alter_db_owner_default.q | 8 + .../authorization_cannot_create_all_role.q | 6 + ...authorization_cannot_create_default_role.q | 6 + .../authorization_cannot_create_none_role.q | 6 + .../authorization_caseinsensitivity.q | 17 + .../authorization_create_func1.q | 7 + .../authorization_create_func2.q | 8 + .../authorization_create_macro1.q | 8 + .../authorization_create_role_no_admin.q | 3 + .../clientnegative/authorization_createview.q | 10 + .../clientnegative/authorization_ctas.q | 10 + .../authorization_desc_table_nosel.q | 14 + .../clientnegative/authorization_dfs.q | 7 + .../authorization_disallow_transform.q | 3 + .../authorization_drop_db_cascade.q | 22 + .../authorization_drop_db_empty.q | 27 + .../authorization_drop_role_no_admin.q | 10 + .../authorization_droppartition.q | 11 + .../authorization_fail_create_db.q | 5 + .../authorization_fail_drop_db.q | 5 + .../authorization_grant_table_allpriv.q | 14 + .../authorization_grant_table_dup.q | 16 + .../authorization_grant_table_fail1.q | 11 + .../authorization_grant_table_fail_nogrant.q | 14 + .../authorization_insert_noinspriv.q | 11 + .../authorization_insert_noselectpriv.q | 11 + .../authorization_invalid_priv_v1.q | 6 + .../authorization_invalid_priv_v2.q | 5 + ...authorization_not_owner_alter_tab_rename.q | 10 + ...horization_not_owner_alter_tab_serdeprop.q | 10 + .../authorization_not_owner_drop_tab.q | 11 + .../authorization_not_owner_drop_view.q | 11 + .../authorization_priv_current_role_neg.q | 29 + .../authorization_public_create.q | 1 + .../authorization_public_drop.q | 1 + .../authorization_revoke_table_fail1.q | 14 + .../authorization_revoke_table_fail2.q | 18 + .../authorization_role_cycles1.q | 12 + .../authorization_role_cycles2.q | 24 + .../clientnegative/authorization_role_grant.q | 22 + .../authorization_rolehierarchy_privs.q | 74 + .../clientnegative/authorization_select.q | 9 + .../authorization_select_view.q | 11 + .../authorization_set_role_neg1.q | 6 + .../authorization_set_role_neg2.q | 16 + .../authorization_show_parts_nosel.q | 10 + ...horization_show_role_principals_no_admin.q | 3 + .../authorization_show_role_principals_v1.q | 2 + .../authorization_show_roles_no_admin.q | 3 + .../clientnegative/authorization_truncate.q | 9 + .../authorization_uri_add_partition.q | 10 + .../authorization_uri_alterpart_loc.q | 16 + .../authorization_uri_altertab_setloc.q | 13 + .../authorization_uri_create_table1.q | 11 + .../authorization_uri_create_table_ext.q | 11 + .../authorization_uri_createdb.q | 12 + .../clientnegative/authorization_uri_export.q | 22 + .../clientnegative/authorization_uri_import.q | 25 + .../clientnegative/authorization_uri_index.q | 13 + .../clientnegative/authorization_uri_insert.q | 14 + .../authorization_uri_insert_local.q | 14 + .../authorization_uri_load_data.q | 11 + .../clientnegative/authorize_create_tbl.q | 10 + .../clientnegative/authorize_grant_public.q | 1 + .../clientnegative/authorize_revoke_public.q | 1 + .../clientnegative/bucket_mapjoin_mismatch1.q | 10 +- .../bucket_mapjoin_wrong_table_metadata_1.q | 6 +- .../bucket_mapjoin_wrong_table_metadata_2.q | 10 +- .../test/queries/clientnegative/clustern1.q | 2 - .../clientnegative/columnstats_partlvl_dp.q | 12 +- .../columnstats_partlvl_incorrect_num_keys.q | 12 +- .../columnstats_partlvl_invalid_values.q | 12 +- ...columnstats_partlvl_multiple_part_clause.q | 12 +- .../clientnegative/columnstats_tbllvl.q | 2 +- .../columnstats_tbllvl_complex_type.q | 2 +- .../columnstats_tbllvl_incorrect_column.q | 2 +- .../clientnegative/compile_processor.q | 8 + .../clientnegative/compute_stats_long.q | 7 + .../create_function_nonexistent_class.q | 1 + .../create_function_nonexistent_db.q | 1 + .../create_function_nonudf_class.q | 1 + .../queries/clientnegative/cte_recursion.q | 4 + .../clientnegative/cte_with_in_subquery.q | 1 + .../queries/clientnegative/date_literal1.q | 2 - .../clientnegative/dbtxnmgr_nodblock.q | 6 + .../clientnegative/dbtxnmgr_nodbunlock.q | 6 + .../clientnegative/dbtxnmgr_notablelock.q | 6 + .../clientnegative/dbtxnmgr_notableunlock.q | 6 + .../test/queries/clientnegative/deletejar.q | 4 +- .../clientnegative/drop_func_nonexistent.q | 3 + .../drop_partition_filter_failure2.q | 11 - .../dynamic_partitions_with_whitelist.q | 2 +- ...hange_partition_neg_incomplete_partition.q | 4 +- .../exchange_partition_neg_partition_exists.q | 2 +- ...exchange_partition_neg_partition_exists2.q | 2 +- ...exchange_partition_neg_partition_exists3.q | 2 +- ...exchange_partition_neg_partition_missing.q | 2 +- .../exim_00_unsupported_schema.q | 6 +- .../exim_01_nonpart_over_loaded.q | 10 +- .../exim_02_all_part_over_overlap.q | 16 +- .../exim_03_nonpart_noncompat_colschema.q | 8 +- .../exim_04_nonpart_noncompat_colnumber.q | 8 +- .../exim_05_nonpart_noncompat_coltype.q | 8 +- .../exim_06_nonpart_noncompat_storage.q | 8 +- .../exim_07_nonpart_noncompat_ifof.q | 8 +- .../exim_08_nonpart_noncompat_serde.q | 8 +- .../exim_09_nonpart_noncompat_serdeparam.q | 8 +- .../exim_10_nonpart_noncompat_bucketing.q | 8 +- .../exim_11_nonpart_noncompat_sorting.q | 8 +- .../clientnegative/exim_13_nonnative_import.q | 8 +- .../clientnegative/exim_14_nonpart_part.q | 8 +- .../clientnegative/exim_15_part_nonpart.q | 8 +- .../exim_16_part_noncompat_schema.q | 8 +- .../exim_17_part_spec_underspec.q | 14 +- .../exim_18_part_spec_missing.q | 14 +- .../exim_19_external_over_existing.q | 8 +- .../exim_20_managed_location_over_existing.q | 14 +- .../exim_21_part_managed_external.q | 14 +- .../clientnegative/exim_22_export_authfail.q | 4 +- .../exim_23_import_exist_authfail.q | 8 +- .../exim_24_import_part_authfail.q | 8 +- .../exim_25_import_nonexist_authfail.q | 8 +- .../clientnegative/fetchtask_ioexception.q | 2 +- .../file_with_header_footer_negative.q | 13 + .../clientnegative/illegal_partition_type.q | 2 +- .../clientnegative/illegal_partition_type3.q | 4 + .../clientnegative/illegal_partition_type4.q | 3 + .../index_compact_entry_limit.q | 1 + .../clientnegative/index_compact_size_limit.q | 1 + .../queries/clientnegative/insert_into5.q | 9 + .../queries/clientnegative/insert_into6.q | 17 + .../insertover_dynapart_ifnotexists.q | 2 +- .../clientnegative/invalid_char_length_1.q | 2 + .../clientnegative/invalid_char_length_2.q | 1 + .../clientnegative/invalid_char_length_3.q | 3 + .../queries/clientnegative/invalid_columns.q | 4 - .../clientnegative/join_alt_syntax_comma_on.q | 3 + .../join_cond_unqual_ambiguous.q | 6 + .../join_cond_unqual_ambiguous_vc.q | 5 + .../queries/clientnegative/limit_partition.q | 7 + .../clientnegative/limit_partition_stats.q | 18 + .../clientnegative/load_exist_part_authfail.q | 2 +- .../queries/clientnegative/load_non_native.q | 2 +- .../clientnegative/load_nonpart_authfail.q | 2 +- .../clientnegative/load_part_authfail.q | 2 +- .../queries/clientnegative/load_part_nospec.q | 2 +- .../clientnegative/load_stored_as_dirs.q | 2 +- .../clientnegative/load_view_failure.q | 2 +- .../clientnegative/load_wrong_fileformat.q | 2 +- .../load_wrong_fileformat_rc_seq.q | 2 +- .../load_wrong_fileformat_txt_seq.q | 2 +- .../clientnegative/load_wrong_noof_part.q | 2 +- .../clientnegative/local_mapred_error_cache.q | 2 +- .../lockneg_query_tbl_in_locked_db.q | 17 + .../lockneg_try_db_lock_conflict.q | 6 + .../lockneg_try_drop_locked_db.q | 8 + .../lockneg_try_lock_db_in_use.q | 15 + .../clientnegative/nested_complex_neg.q | 2 +- .../queries/clientnegative/nopart_insert.q | 2 +- .../test/queries/clientnegative/nopart_load.q | 2 +- .../queries/clientnegative/notable_alias3.q | 4 - .../queries/clientnegative/parquet_char.q | 3 + .../queries/clientnegative/parquet_date.q | 3 + .../queries/clientnegative/parquet_decimal.q | 3 + .../clientnegative/parquet_timestamp.q | 3 + .../queries/clientnegative/parquet_varchar.q | 3 + .../clientnegative/protectmode_part2.q | 4 +- ...ptf_negative_AggrFuncsWithNoGBYNoPartDef.q | 2 +- .../ptf_negative_AmbiguousWindowDefn.q | 2 +- .../test/queries/clientnegative/regex_col_1.q | 1 + .../test/queries/clientnegative/regex_col_2.q | 1 + .../clientnegative/regex_col_groupby.q | 1 + .../clientnegative/script_broken_pipe1.q | 3 - .../queries/clientnegative/script_error.q | 4 +- .../queries/clientnegative/serde_regex2.q | 4 +- .../clientnegative/set_hiveconf_validation2.q | 5 + .../clientnegative/stats_aggregator_error_1.q | 2 +- .../clientnegative/stats_aggregator_error_2.q | 2 +- .../clientnegative/stats_publisher_error_1.q | 2 +- .../clientnegative/stats_publisher_error_2.q | 2 +- .../subquery_exists_implicit_gby.q | 10 + .../clientnegative/subquery_in_groupby.q | 5 + .../clientnegative/subquery_in_select.q | 6 + .../subquery_multiple_cols_in_select.q | 7 + .../clientnegative/subquery_nested_subquery.q | 18 + .../subquery_notexists_implicit_gby.q | 10 + .../clientnegative/subquery_shared_alias.q | 6 + .../clientnegative/subquery_subquery_chain.q | 6 + .../subquery_unqual_corr_expr.q | 6 + .../clientnegative/subquery_windowing_corr.q | 26 + .../clientnegative/subquery_with_or_cond.q | 5 + .../clientnegative/udf_case_type_wrong.q | 6 - .../clientnegative/udf_case_type_wrong2.q | 6 - .../clientnegative/udf_case_type_wrong3.q | 6 - .../test/queries/clientnegative/udf_invalid.q | 1 + .../clientnegative/udf_local_resource.q | 1 + .../clientnegative/udf_nonexistent_resource.q | 1 + .../clientnegative/udf_qualified_name.q | 1 + .../clientnegative/udf_when_type_wrong2.q | 6 - .../clientnegative/udf_when_type_wrong3.q | 6 - .../src/test/queries/clientnegative/udfnull.q | 6 - .../src/test/queries/clientnegative/union.q | 4 - .../clientnegative/windowing_invalid_udaf.q | 1 + .../clientnegative/windowing_ll_no_neg.q | 26 + .../src/test/queries/clientpositive/alter1.q | 4 +- .../src/test/queries/clientpositive/alter3.q | 4 +- .../src/test/queries/clientpositive/alter5.q | 4 +- .../test/queries/clientpositive/alter_char1.q | 32 + .../test/queries/clientpositive/alter_char2.q | 22 + .../alter_concatenate_indexed_table.q | 12 +- .../queries/clientpositive/alter_db_owner.q | 9 + .../test/queries/clientpositive/alter_merge.q | 12 +- .../queries/clientpositive/alter_merge_2.q | 6 +- .../clientpositive/alter_merge_stats.q | 12 +- .../alter_numbuckets_partitioned_table.q | 2 +- .../alter_numbuckets_partitioned_table2.q | 1 + .../alter_numbuckets_partitioned_table2_h23.q | 85 + .../alter_numbuckets_partitioned_table_h23.q | 59 + .../clientpositive/alter_partition_coltype.q | 42 +- .../alter_partition_protect_mode.q | 8 +- .../clientpositive/alter_rename_partition.q | 4 +- .../queries/clientpositive/alter_varchar2.q | 6 +- .../queries/clientpositive/ambiguous_col.q | 1 + .../clientpositive/annotate_stats_filter.q | 76 + .../clientpositive/annotate_stats_groupby.q | 69 + .../clientpositive/annotate_stats_join.q | 81 + .../clientpositive/annotate_stats_limit.q | 30 + .../clientpositive/annotate_stats_part.q | 85 + .../clientpositive/annotate_stats_select.q | 143 + .../clientpositive/annotate_stats_table.q | 53 + .../clientpositive/annotate_stats_union.q | 55 + .../clientpositive/ansi_sql_arithmetic.q | 13 + .../queries/clientpositive/archive_corrupt.q | 2 +- .../clientpositive/archive_excludeHadoop20.q | 2 +- .../clientpositive/authorization_1_sql_std.q | 36 + .../queries/clientpositive/authorization_2.q | 2 +- .../queries/clientpositive/authorization_9.q | 17 + .../authorization_admin_almighty1.q | 17 + .../authorization_create_func1.q | 14 + .../authorization_create_macro1.q | 12 + .../authorization_create_table_owner_privs.q | 10 + .../authorization_grant_public_role.q | 18 + .../authorization_grant_table_priv.q | 43 + .../clientpositive/authorization_index.q | 12 + .../authorization_owner_actions.q | 16 + .../authorization_owner_actions_db.q | 21 + .../clientpositive/authorization_parts.q | 19 + .../authorization_revoke_table_priv.q | 61 + .../authorization_role_grant1.q | 38 + .../authorization_role_grant2.q | 34 + .../authorization_set_show_current_role.q | 21 + .../authorization_view_sqlstd.q | 66 + .../test/queries/clientpositive/auto_join25.q | 2 + .../test/queries/clientpositive/auto_join32.q | 16 +- .../clientpositive/auto_join_filters.q | 10 +- .../queries/clientpositive/auto_join_nulls.q | 2 +- .../auto_join_reordering_values.q | 6 +- .../auto_join_without_localtask.q | 29 + .../clientpositive/auto_sortmerge_join_1.q | 22 +- .../clientpositive/auto_sortmerge_join_11.q | 22 +- .../clientpositive/auto_sortmerge_join_12.q | 26 +- .../clientpositive/auto_sortmerge_join_16.q | 92 + .../clientpositive/auto_sortmerge_join_2.q | 16 +- .../clientpositive/auto_sortmerge_join_3.q | 16 +- .../clientpositive/auto_sortmerge_join_4.q | 20 +- .../clientpositive/auto_sortmerge_join_5.q | 12 +- .../clientpositive/auto_sortmerge_join_7.q | 24 +- .../clientpositive/auto_sortmerge_join_8.q | 26 +- .../clientpositive/avro_compression_enabled.q | 2 +- .../clientpositive/avro_evolved_schemas.q | 2 +- .../test/queries/clientpositive/avro_joins.q | 4 +- .../clientpositive/avro_nullable_fields.q | 2 +- .../queries/clientpositive/avro_partitioned.q | 4 +- .../queries/clientpositive/avro_sanity_test.q | 2 +- .../queries/clientpositive/binary_constant.q | 4 +- .../clientpositive/binary_table_colserde.q | 1 + .../queries/clientpositive/binarysortable_1.q | 2 +- .../bucket_if_with_path_filter.q | 15 + .../clientpositive/bucket_map_join_1.q | 4 +- .../clientpositive/bucket_map_join_2.q | 4 +- .../clientpositive/bucket_map_join_tez1.q | 85 + .../clientpositive/bucket_map_join_tez2.q | 50 + .../queries/clientpositive/bucketcontext_1.q | 20 +- .../queries/clientpositive/bucketcontext_2.q | 16 +- .../queries/clientpositive/bucketcontext_3.q | 16 +- .../queries/clientpositive/bucketcontext_4.q | 20 +- .../queries/clientpositive/bucketcontext_5.q | 12 +- .../queries/clientpositive/bucketcontext_6.q | 16 +- .../queries/clientpositive/bucketcontext_7.q | 24 +- .../queries/clientpositive/bucketcontext_8.q | 24 +- .../bucketizedhiveinputformat.q | 6 +- .../bucketizedhiveinputformat_auto.q | 20 +- .../queries/clientpositive/bucketmapjoin1.q | 16 +- .../queries/clientpositive/bucketmapjoin10.q | 20 +- .../queries/clientpositive/bucketmapjoin11.q | 24 +- .../queries/clientpositive/bucketmapjoin12.q | 12 +- .../queries/clientpositive/bucketmapjoin2.q | 16 +- .../queries/clientpositive/bucketmapjoin3.q | 16 +- .../queries/clientpositive/bucketmapjoin4.q | 16 +- .../queries/clientpositive/bucketmapjoin5.q | 28 +- .../queries/clientpositive/bucketmapjoin7.q | 8 +- .../queries/clientpositive/bucketmapjoin8.q | 8 +- .../queries/clientpositive/bucketmapjoin9.q | 14 +- .../clientpositive/bucketmapjoin_negative.q | 10 +- .../clientpositive/bucketmapjoin_negative2.q | 12 +- .../clientpositive/bucketmapjoin_negative3.q | 24 +- .../test/queries/clientpositive/cast_to_int.q | 4 +- .../src/test/queries/clientpositive/char_1.q | 32 + .../src/test/queries/clientpositive/char_2.q | 36 + .../test/queries/clientpositive/char_cast.q | 92 + .../queries/clientpositive/char_comparison.q | 40 + .../test/queries/clientpositive/char_join1.q | 35 + .../clientpositive/char_nested_types.q | 53 + .../test/queries/clientpositive/char_serde.q | 102 + .../test/queries/clientpositive/char_udf1.q | 156 + .../test/queries/clientpositive/char_union1.q | 47 + .../queries/clientpositive/char_varchar_udf.q | 9 + .../clientpositive/column_access_stats.q | 2 +- .../clientpositive/columnstats_partlvl.q | 4 +- .../clientpositive/columnstats_tbllvl.q | 2 +- .../clientpositive/compile_processor.q | 12 + .../clientpositive/compute_stats_binary.q | 2 +- .../clientpositive/compute_stats_boolean.q | 2 +- .../clientpositive/compute_stats_decimal.q | 11 + .../clientpositive/compute_stats_double.q | 2 +- .../clientpositive/compute_stats_long.q | 2 +- .../clientpositive/compute_stats_string.q | 2 +- .../queries/clientpositive/constant_prop.q | 6 +- .../clientpositive/correlationoptimizer1.q | 25 +- .../clientpositive/correlationoptimizer4.q | 6 +- .../clientpositive/correlationoptimizer5.q | 8 +- .../src/test/queries/clientpositive/count.q | 2 +- .../queries/clientpositive/create_func1.q | 30 + .../test/queries/clientpositive/create_like.q | 6 +- .../clientpositive/create_merge_compressed.q | 2 +- .../clientpositive/create_nested_type.q | 2 +- .../clientpositive/create_struct_table.q | 2 +- .../clientpositive/create_union_table.q | 2 +- .../clientpositive/create_view_translate.q | 11 + .../clientpositive/cross_product_check_1.q | 26 + .../clientpositive/cross_product_check_2.q | 27 + .../ql/src/test/queries/clientpositive/ctas.q | 2 +- .../test/queries/clientpositive/ctas_char.q | 22 + .../queries/clientpositive/ctas_hadoop20.q | 11 +- .../src/test/queries/clientpositive/cte_1.q | 28 + .../src/test/queries/clientpositive/cte_2.q | 56 + .../custom_input_output_format.q | 5 +- .../test/queries/clientpositive/database.q | 6 +- .../queries/clientpositive/database_drop.q | 17 +- .../src/test/queries/clientpositive/date_1.q | 8 +- .../src/test/queries/clientpositive/date_2.q | 2 +- .../src/test/queries/clientpositive/date_3.q | 2 +- .../src/test/queries/clientpositive/date_4.q | 4 +- .../queries/clientpositive/date_comparison.q | 2 + .../test/queries/clientpositive/date_join1.q | 2 +- .../test/queries/clientpositive/date_serde.q | 2 +- .../test/queries/clientpositive/date_udf.q | 2 +- .../clientpositive/dbtxnmgr_compact1.q | 12 + .../clientpositive/dbtxnmgr_compact2.q | 14 + .../clientpositive/dbtxnmgr_compact3.q | 15 + .../queries/clientpositive/dbtxnmgr_ddl1.q | 59 + .../queries/clientpositive/dbtxnmgr_query1.q | 17 + .../queries/clientpositive/dbtxnmgr_query2.q | 17 + .../queries/clientpositive/dbtxnmgr_query3.q | 21 + .../queries/clientpositive/dbtxnmgr_query4.q | 19 + .../queries/clientpositive/dbtxnmgr_query5.q | 24 + .../clientpositive/dbtxnmgr_showlocks.q | 11 + .../test/queries/clientpositive/decimal_1.q | 28 +- .../test/queries/clientpositive/decimal_2.q | 60 +- .../test/queries/clientpositive/decimal_3.q | 4 +- .../test/queries/clientpositive/decimal_4.q | 6 +- .../test/queries/clientpositive/decimal_5.q | 18 + .../test/queries/clientpositive/decimal_6.q | 27 + .../queries/clientpositive/decimal_join.q | 4 +- .../clientpositive/decimal_precision.q | 15 +- .../queries/clientpositive/decimal_serde.q | 2 +- .../test/queries/clientpositive/decimal_udf.q | 10 +- .../test/queries/clientpositive/delimiter.q | 2 +- .../clientpositive/desc_tbl_part_cols.q | 7 + .../disable_file_format_check.q | 4 +- .../disallow_incompatible_type_change_off.q | 4 +- .../queries/clientpositive/distinct_stats.q | 20 + .../clientpositive/drop_partitions_filter2.q | 5 +- .../clientpositive/drop_with_concurrency.q | 8 + .../dynamic_partition_skip_default.q | 16 +- .../dynpart_sort_opt_vectorization.q | 161 ++ .../dynpart_sort_optimization.q | 155 + .../src/test/queries/clientpositive/escape1.q | 2 +- .../src/test/queries/clientpositive/escape2.q | 2 +- .../clientpositive/exchange_partition.q | 2 +- .../clientpositive/exchange_partition2.q | 2 +- .../clientpositive/exchange_partition3.q | 5 +- .../clientpositive/exim_00_nonpart_empty.q | 9 +- .../queries/clientpositive/exim_01_nonpart.q | 8 +- .../clientpositive/exim_02_00_part_empty.q | 6 +- .../queries/clientpositive/exim_02_part.q | 8 +- .../exim_03_nonpart_over_compat.q | 8 +- .../queries/clientpositive/exim_04_all_part.q | 14 +- .../clientpositive/exim_04_evolved_parts.q | 6 +- .../clientpositive/exim_05_some_part.q | 14 +- .../queries/clientpositive/exim_06_one_part.q | 14 +- .../exim_07_all_part_over_nonoverlap.q | 16 +- .../clientpositive/exim_08_nonpart_rename.q | 10 +- .../exim_09_part_spec_nonoverlap.q | 18 +- .../clientpositive/exim_10_external_managed.q | 14 +- .../clientpositive/exim_11_managed_external.q | 8 +- .../exim_12_external_location.q | 14 +- .../clientpositive/exim_13_managed_location.q | 14 +- .../exim_14_managed_location_over_existing.q | 14 +- .../clientpositive/exim_15_external_part.q | 24 +- .../clientpositive/exim_16_part_external.q | 26 +- .../clientpositive/exim_17_part_managed.q | 20 +- .../clientpositive/exim_18_part_external.q | 14 +- .../exim_19_00_part_external_location.q | 16 +- .../exim_19_part_external_location.q | 20 +- .../exim_20_part_managed_location.q | 20 +- .../exim_21_export_authsuccess.q | 6 +- .../exim_22_import_exist_authsuccess.q | 8 +- .../exim_23_import_part_authsuccess.q | 8 +- .../exim_24_import_nonexist_authsuccess.q | 8 +- .../clientpositive/exim_hidden_files.q | 22 + .../clientpositive/explain_rearrange.q | 98 + ...ternal_table_with_space_in_location_path.q | 23 + .../clientpositive/file_with_header_footer.q | 39 + .../clientpositive/filter_join_breaktask2.q | 6 +- .../queries/clientpositive/filter_numeric.q | 21 + .../queries/clientpositive/global_limit.q | 14 +- .../test/queries/clientpositive/groupby10.q | 2 +- .../test/queries/clientpositive/groupby12.q | 13 + .../queries/clientpositive/groupby1_limit.q | 2 +- .../queries/clientpositive/groupby1_map.q | 2 +- .../clientpositive/groupby1_map_skew.q | 2 +- .../queries/clientpositive/groupby1_noskew.q | 2 +- .../queries/clientpositive/groupby2_limit.q | 4 +- .../queries/clientpositive/groupby2_map.q | 2 +- .../groupby2_map_multi_distinct.q | 13 +- .../queries/clientpositive/groupby2_noskew.q | 2 +- .../groupby2_noskew_multi_distinct.q | 2 +- .../queries/clientpositive/groupby4_noskew.q | 2 +- .../queries/clientpositive/groupby5_noskew.q | 2 +- .../queries/clientpositive/groupby6_map.q | 2 +- .../clientpositive/groupby6_map_skew.q | 2 +- .../queries/clientpositive/groupby6_noskew.q | 2 +- .../queries/clientpositive/groupby7_map.q | 4 +- .../groupby7_map_multi_single_reducer.q | 4 +- .../clientpositive/groupby7_map_skew.q | 4 +- .../queries/clientpositive/groupby7_noskew.q | 4 +- .../groupby7_noskew_multi_single_reducer.q | 4 +- .../queries/clientpositive/groupby8_map.q | 4 +- .../clientpositive/groupby8_map_skew.q | 4 +- .../queries/clientpositive/groupby8_noskew.q | 4 +- .../queries/clientpositive/groupby_bigdata.q | 2 +- .../queries/clientpositive/groupby_cube1.q | 2 +- .../clientpositive/groupby_grouping_id1.q | 2 +- .../clientpositive/groupby_grouping_id2.q | 2 +- .../clientpositive/groupby_grouping_sets1.q | 2 +- .../clientpositive/groupby_grouping_sets2.q | 2 +- .../clientpositive/groupby_grouping_sets3.q | 4 +- .../clientpositive/groupby_grouping_sets4.q | 2 +- .../clientpositive/groupby_grouping_sets5.q | 2 +- .../queries/clientpositive/groupby_map_ppr.q | 2 +- .../groupby_map_ppr_multi_distinct.q | 2 +- .../clientpositive/groupby_resolution.q | 61 + .../queries/clientpositive/groupby_rollup1.q | 2 +- .../queries/clientpositive/groupby_sort_1.q | 2 +- .../queries/clientpositive/groupby_sort_2.q | 2 +- .../queries/clientpositive/groupby_sort_3.q | 2 +- .../queries/clientpositive/groupby_sort_4.q | 2 +- .../queries/clientpositive/groupby_sort_5.q | 6 +- .../queries/clientpositive/groupby_sort_6.q | 2 +- .../queries/clientpositive/groupby_sort_7.q | 2 +- .../queries/clientpositive/groupby_sort_8.q | 2 +- .../queries/clientpositive/groupby_sort_9.q | 2 +- .../clientpositive/groupby_sort_skew_1.q | 2 +- .../clientpositive/groupby_sort_test_1.q | 2 +- .../clientpositive/import_exported_table.q | 13 + .../test/queries/clientpositive/index_auth.q | 13 +- .../test/queries/clientpositive/index_auto.q | 3 +- .../queries/clientpositive/index_auto_empty.q | 1 + .../clientpositive/index_auto_file_format.q | 3 +- .../clientpositive/index_auto_mult_tables.q | 1 + .../index_auto_mult_tables_compact.q | 1 + .../clientpositive/index_auto_multiple.q | 3 +- .../clientpositive/index_auto_partitioned.q | 1 + .../clientpositive/index_auto_self_join.q | 1 + .../clientpositive/index_auto_unused.q | 1 + .../queries/clientpositive/index_bitmap.q | 1 + .../queries/clientpositive/index_bitmap1.q | 1 + .../queries/clientpositive/index_bitmap2.q | 1 + .../queries/clientpositive/index_bitmap3.q | 11 +- .../clientpositive/index_bitmap_auto.q | 15 +- .../index_bitmap_auto_partitioned.q | 1 + .../clientpositive/index_bitmap_compression.q | 1 + .../queries/clientpositive/index_bitmap_rc.q | 1 + .../queries/clientpositive/index_compact.q | 1 + .../queries/clientpositive/index_compact_1.q | 1 + .../queries/clientpositive/index_compact_2.q | 3 +- .../queries/clientpositive/index_compact_3.q | 3 +- .../index_compact_binary_search.q | 4 +- .../clientpositive/index_compression.q | 1 + .../queries/clientpositive/index_creation.q | 1 + .../test/queries/clientpositive/index_serde.q | 3 +- .../test/queries/clientpositive/index_stale.q | 1 + .../clientpositive/index_stale_partitioned.q | 1 + .../infer_bucket_sort_dyn_part.q | 10 +- .../queries/clientpositive/infer_const_type.q | 2 +- .../src/test/queries/clientpositive/input13.q | 6 +- .../src/test/queries/clientpositive/input16.q | 4 +- .../test/queries/clientpositive/input16_cc.q | 4 +- .../src/test/queries/clientpositive/input19.q | 2 +- .../src/test/queries/clientpositive/input20.q | 6 +- .../src/test/queries/clientpositive/input21.q | 2 +- .../src/test/queries/clientpositive/input22.q | 2 +- .../src/test/queries/clientpositive/input33.q | 6 +- .../src/test/queries/clientpositive/input37.q | 4 +- .../queries/clientpositive/input3_limit.q | 4 +- .../src/test/queries/clientpositive/input4.q | 4 +- .../src/test/queries/clientpositive/input40.q | 6 +- .../src/test/queries/clientpositive/input43.q | 4 +- .../src/test/queries/clientpositive/input44.q | 2 +- .../src/test/queries/clientpositive/input45.q | 6 +- .../queries/clientpositive/input4_cb_delim.q | 2 +- .../test/queries/clientpositive/input_dfs.q | 2 +- .../test/queries/clientpositive/inputddl5.q | 2 +- .../test/queries/clientpositive/inputddl6.q | 4 +- .../test/queries/clientpositive/inputddl7.q | 8 +- .../insert1_overwrite_partitions.q | 4 +- .../insert2_overwrite_partitions.q | 4 +- .../queries/clientpositive/insert_into3.q | 8 +- .../insert_overwrite_local_directory_1.q | 52 +- .../test/queries/clientpositive/join_1to1.q | 4 +- .../queries/clientpositive/join_alt_syntax.q | 41 + .../test/queries/clientpositive/join_array.q | 4 +- .../clientpositive/join_casesensitive.q | 4 +- .../clientpositive/join_cond_pushdown_1.q | 30 + .../clientpositive/join_cond_pushdown_2.q | 24 + .../clientpositive/join_cond_pushdown_3.q | 34 + .../clientpositive/join_cond_pushdown_4.q | 26 + .../join_cond_pushdown_unqual1.q | 52 + .../join_cond_pushdown_unqual2.q | 47 + .../join_cond_pushdown_unqual3.q | 56 + .../join_cond_pushdown_unqual4.q | 49 + .../queries/clientpositive/join_filters.q | 10 +- .../queries/clientpositive/join_hive_626.q | 6 +- .../queries/clientpositive/join_merging.q | 25 + .../test/queries/clientpositive/join_nulls.q | 10 +- .../queries/clientpositive/join_nullsafe.q | 10 +- .../queries/clientpositive/join_reorder.q | 6 +- .../queries/clientpositive/join_reorder2.q | 8 +- .../queries/clientpositive/join_reorder3.q | 8 +- .../queries/clientpositive/join_reorder4.q | 6 +- .../test/queries/clientpositive/join_star.q | 16 +- .../clientpositive/lateral_view_noalias.q | 2 + .../queries/clientpositive/lateral_view_ppd.q | 4 + .../test/queries/clientpositive/lb_fs_stats.q | 19 + .../src/test/queries/clientpositive/leadlag.q | 2 +- .../queries/clientpositive/leadlag_queries.q | 2 +- .../queries/clientpositive/leftsemijoin.q | 6 +- .../queries/clientpositive/leftsemijoin_mr.q | 4 +- .../limit_partition_metadataonly.q | 7 + .../queries/clientpositive/limit_pushdown.q | 13 +- .../clientpositive/limit_pushdown_negative.q | 4 - .../clientpositive/list_bucket_dml_2.q | 6 +- .../clientpositive/list_bucket_dml_4.q | 2 +- .../queries/clientpositive/literal_decimal.q | 2 + .../queries/clientpositive/literal_double.q | 2 + .../queries/clientpositive/literal_ints.q | 2 + .../queries/clientpositive/literal_string.q | 2 + .../queries/clientpositive/load_binary_data.q | 2 +- .../queries/clientpositive/load_dyn_part1.q | 4 +- .../queries/clientpositive/load_dyn_part10.q | 2 +- .../queries/clientpositive/load_dyn_part3.q | 2 +- .../queries/clientpositive/load_dyn_part4.q | 4 +- .../queries/clientpositive/load_dyn_part8.q | 2 +- .../queries/clientpositive/load_dyn_part9.q | 2 +- .../load_exist_part_authsuccess.q | 2 +- .../load_file_with_space_in_the_name.q | 3 +- .../src/test/queries/clientpositive/load_fs.q | 6 +- .../test/queries/clientpositive/load_fs2.q | 6 +- .../clientpositive/load_fs_overwrite.q | 20 + .../load_hdfs_file_with_space_in_the_name.q | 9 +- .../clientpositive/load_nonpart_authsuccess.q | 2 +- .../queries/clientpositive/load_overwrite.q | 4 +- .../clientpositive/load_part_authsuccess.q | 2 +- .../test/queries/clientpositive/loadpart1.q | 2 +- .../test/queries/clientpositive/loadpart2.q | 9 + .../queries/clientpositive/loadpart_err.q | 4 +- .../src/test/queries/clientpositive/macro.q | 2 + .../queries/clientpositive/mapjoin_addjar.q | 14 + .../queries/clientpositive/mapjoin_decimal.q | 35 + .../queries/clientpositive/mapjoin_hook.q | 3 +- .../queries/clientpositive/mapjoin_mapjoin.q | 10 +- .../queries/clientpositive/mapjoin_memcheck.q | 16 + .../clientpositive/mapjoin_subquery2.q | 6 +- .../src/test/queries/clientpositive/merge3.q | 4 +- .../src/test/queries/clientpositive/merge4.q | 4 +- .../clientpositive/merge_dynamic_partition.q | 9 +- .../clientpositive/merge_dynamic_partition2.q | 13 +- .../clientpositive/merge_dynamic_partition3.q | 28 +- .../clientpositive/merge_dynamic_partition4.q | 18 +- .../clientpositive/merge_dynamic_partition5.q | 10 +- .../clientpositive/metadata_export_drop.q | 6 +- .../clientpositive/metadata_only_queries.q | 77 + .../metadata_only_queries_with_filters.q | 51 + .../ql/src/test/queries/clientpositive/mi.q | 4 +- .../ql/src/test/queries/clientpositive/mrr.q | 59 + .../queries/clientpositive/multiMapJoin1.q | 2 + .../queries/clientpositive/multiMapJoin2.q | 26 + .../queries/clientpositive/nested_complex.q | 2 +- .../src/test/queries/clientpositive/newline.q | 10 +- .../clientpositive/nonmr_fetch_threshold.q | 9 + .../nonreserved_keywords_input37.q | 4 +- .../queries/clientpositive/notable_alias3.q | 4 + .../test/queries/clientpositive/null_cast.q | 4 +- .../test/queries/clientpositive/null_column.q | 6 +- .../test/queries/clientpositive/nullformat.q | 24 + .../queries/clientpositive/nullformatCTAS.q | 24 + .../queries/clientpositive/nullformatdir.q | 21 + .../test/queries/clientpositive/nullgroup3.q | 16 +- .../test/queries/clientpositive/nullgroup5.q | 4 +- .../test/queries/clientpositive/nullscript.q | 4 +- .../queries/clientpositive/num_op_type_conv.q | 2 + .../queries/clientpositive/ops_comparison.q | 1 + .../queries/clientpositive/optrstat_groupby.q | 6 - .../test/queries/clientpositive/orc_analyze.q | 179 ++ .../test/queries/clientpositive/orc_create.q | 31 +- .../clientpositive/orc_dictionary_threshold.q | 2 +- .../clientpositive/orc_diff_part_cols.q | 2 +- .../clientpositive/orc_diff_part_cols2.q | 11 + .../clientpositive/orc_empty_strings.q | 4 +- .../clientpositive/orc_ends_with_nulls.q | 2 +- .../test/queries/clientpositive/orc_min_max.q | 32 + .../queries/clientpositive/orc_ppd_char.q | 76 + .../queries/clientpositive/orc_ppd_date.q | 97 + .../queries/clientpositive/orc_ppd_decimal.q | 151 + .../queries/clientpositive/orc_ppd_varchar.q | 76 + .../clientpositive/orc_predicate_pushdown.q | 6 +- .../clientpositive/orc_split_elimination.q | 168 ++ .../clientpositive/orc_vectorization_ppd.q | 69 + .../clientpositive/order_within_subquery.q | 19 + .../queries/clientpositive/parallel_orderby.q | 4 +- .../queries/clientpositive/parquet_create.q | 36 + .../queries/clientpositive/parquet_ctas.q | 24 + .../clientpositive/parquet_partitioned.q | 34 + .../queries/clientpositive/parquet_types.q | 38 + .../test/queries/clientpositive/partcols1.q | 2 +- .../queries/clientpositive/partition_date.q | 51 +- .../queries/clientpositive/partition_date2.q | 6 +- .../clientpositive/partition_decode_name.q | 6 +- .../clientpositive/partition_special_char.q | 6 +- .../clientpositive/partition_type_check.q | 6 +- .../clientpositive/partition_varchar1.q | 8 +- .../clientpositive/partition_varchar2.q | 10 + .../partition_wise_fileformat17.q | 10 +- .../partition_wise_fileformat18.q | 19 + .../ql/src/test/queries/clientpositive/pcr.q | 2 +- .../test/queries/clientpositive/ppd_join4.q | 22 + .../queries/clientpositive/ppd_multi_insert.q | 12 +- .../queries/clientpositive/ppd_transform.q | 9 + .../test/queries/clientpositive/ppd_udtf.q | 12 + .../queries/clientpositive/ppd_union_view.q | 12 +- .../queries/clientpositive/ppr_pushdown.q | 18 +- .../queries/clientpositive/ppr_pushdown2.q | 16 +- .../test/queries/clientpositive/progress_1.q | 2 +- .../ql/src/test/queries/clientpositive/ptf.q | 2 +- .../test/queries/clientpositive/ptf_decimal.q | 4 +- .../clientpositive/ptf_general_queries.q | 2 +- .../queries/clientpositive/ptf_matchpath.q | 2 +- .../test/queries/clientpositive/ptf_rcfile.q | 2 +- .../clientpositive/ptf_register_tblfn.q | 2 +- .../test/queries/clientpositive/ptf_seqfile.q | 2 +- .../clientpositive/ql_rewrite_gbtoidx.q | 6 +- .../src/test/queries/clientpositive/quote2.q | 2 + .../queries/clientpositive/quotedid_alter.q | 21 + .../queries/clientpositive/quotedid_basic.q | 34 + .../clientpositive/quotedid_partition.q | 24 + .../queries/clientpositive/quotedid_skew.q | 26 + .../queries/clientpositive/quotedid_smb.q | 34 + .../clientpositive/quotedid_tblproperty.q | 8 + .../queries/clientpositive/rcfile_bigdata.q | 2 +- .../test/queries/clientpositive/regex_col.q | 2 + .../queries/clientpositive/remote_script.q | 2 +- .../src/test/queries/clientpositive/repair.q | 10 +- .../clientpositive/root_dir_external_table.q | 11 + .../queries/clientpositive/schemeAuthority2.q | 4 +- .../test/queries/clientpositive/scriptfile1.q | 4 +- .../queries/clientpositive/scriptfile1_win.q | 16 + .../clientpositive/select_dummy_source.q | 33 + .../test/queries/clientpositive/serde_regex.q | 10 +- .../clientpositive/set_processor_namespaces.q | 2 +- .../clientpositive/show_indexes_edge_cases.q | 3 +- .../clientpositive/show_indexes_syntax.q | 1 + .../queries/clientpositive/show_partitions.q | 3 +- .../test/queries/clientpositive/show_roles.q | 4 + .../queries/clientpositive/show_tablestatus.q | 1 + .../test/queries/clientpositive/skewjoin.q | 8 +- .../queries/clientpositive/skewjoin_noskew.q | 9 + .../clientpositive/skewjoin_union_remove_1.q | 4 +- .../clientpositive/skewjoin_union_remove_2.q | 6 +- .../queries/clientpositive/skewjoinopt1.q | 4 +- .../queries/clientpositive/skewjoinopt10.q | 2 +- .../queries/clientpositive/skewjoinopt11.q | 4 +- .../queries/clientpositive/skewjoinopt12.q | 4 +- .../queries/clientpositive/skewjoinopt13.q | 6 +- .../queries/clientpositive/skewjoinopt14.q | 6 +- .../queries/clientpositive/skewjoinopt15.q | 4 +- .../queries/clientpositive/skewjoinopt16.q | 4 +- .../queries/clientpositive/skewjoinopt17.q | 8 +- .../queries/clientpositive/skewjoinopt18.q | 4 +- .../queries/clientpositive/skewjoinopt19.q | 4 +- .../queries/clientpositive/skewjoinopt2.q | 4 +- .../queries/clientpositive/skewjoinopt20.q | 4 +- .../queries/clientpositive/skewjoinopt3.q | 4 +- .../queries/clientpositive/skewjoinopt4.q | 4 +- .../queries/clientpositive/skewjoinopt5.q | 4 +- .../queries/clientpositive/skewjoinopt6.q | 4 +- .../queries/clientpositive/skewjoinopt7.q | 6 +- .../queries/clientpositive/skewjoinopt8.q | 6 +- .../queries/clientpositive/skewjoinopt9.q | 4 +- .../queries/clientpositive/smb_mapjoin_1.q | 6 +- .../queries/clientpositive/smb_mapjoin_10.q | 8 +- .../queries/clientpositive/smb_mapjoin_2.q | 6 +- .../queries/clientpositive/smb_mapjoin_25.q | 6 +- .../queries/clientpositive/smb_mapjoin_3.q | 6 +- .../queries/clientpositive/smb_mapjoin_4.q | 6 +- .../queries/clientpositive/smb_mapjoin_5.q | 6 +- .../queries/clientpositive/smb_mapjoin_7.q | 4 +- .../queries/clientpositive/smb_mapjoin_8.q | 2 +- .../src/test/queries/clientpositive/source.q | 2 +- .../src/test/queries/clientpositive/split.q | 8 + .../src/test/queries/clientpositive/stats1.q | 2 +- .../src/test/queries/clientpositive/stats11.q | 18 +- .../src/test/queries/clientpositive/stats18.q | 2 +- .../src/test/queries/clientpositive/stats19.q | 4 +- .../src/test/queries/clientpositive/stats3.q | 4 +- .../src/test/queries/clientpositive/stats4.q | 4 +- .../clientpositive/stats_aggregator_error_1.q | 4 +- .../queries/clientpositive/stats_counter.q | 16 + .../stats_counter_partitioned.q | 45 + .../clientpositive/stats_invalidation.q | 15 + .../clientpositive/stats_list_bucket.q | 45 + .../queries/clientpositive/stats_noscan_2.q | 10 +- .../queries/clientpositive/stats_only_null.q | 41 + .../clientpositive/stats_publisher_error_1.q | 2 +- .../src/test/queries/clientpositive/statsfs.q | 63 + .../test/queries/clientpositive/str_to_map.q | 4 +- .../ql/src/test/queries/clientpositive/subq.q | 6 +- .../clientpositive/subq_where_serialization.q | 5 + .../queries/clientpositive/subquery_alias.q | 16 + .../queries/clientpositive/subquery_exists.q | 45 + .../clientpositive/subquery_exists_having.q | 60 + .../test/queries/clientpositive/subquery_in.q | 163 ++ .../clientpositive/subquery_in_having.q | 104 + .../clientpositive/subquery_multiinsert.q | 82 + .../clientpositive/subquery_notexists.q | 41 + .../subquery_notexists_having.q | 46 + .../queries/clientpositive/subquery_notin.q | 143 + .../clientpositive/subquery_notin_having.q | 74 + .../subquery_unqualcolumnrefs.q | 83 + .../queries/clientpositive/subquery_views.q | 48 + .../symlink_text_input_format.q | 9 +- .../clientpositive/table_access_keys_stats.q | 2 +- .../clientpositive/test_boolean_whereclause.q | 2 +- .../src/test/queries/clientpositive/tez_dml.q | 40 + .../test/queries/clientpositive/tez_fsstat.q | 19 + .../tez_insert_overwrite_local_directory_1.q | 5 + .../queries/clientpositive/tez_join_tests.q | 12 + .../clientpositive/tez_joins_explain.q | 5 + .../clientpositive/tez_schema_evolution.q | 14 + .../test/queries/clientpositive/tez_union.q | 94 + .../test/queries/clientpositive/timestamp_1.q | 16 +- .../test/queries/clientpositive/timestamp_2.q | 16 +- .../test/queries/clientpositive/timestamp_3.q | 8 +- .../clientpositive/timestamp_comparison.q | 3 +- .../queries/clientpositive/timestamp_lazy.q | 2 +- .../queries/clientpositive/timestamp_null.q | 2 +- .../queries/clientpositive/timestamp_udf.q | 8 +- .../test/queries/clientpositive/transform1.q | 2 +- .../queries/clientpositive/truncate_column.q | 8 +- .../clientpositive/truncate_column_merge.q | 4 +- .../queries/clientpositive/truncate_table.q | 10 +- .../test/queries/clientpositive/type_cast_1.q | 2 + .../clientpositive/type_conversions_1.q | 1 + .../queries/clientpositive/type_widening.q | 1 + .../queries/clientpositive/udaf_collect_set.q | 11 + .../clientpositive/udaf_context_ngrams.q | 2 +- .../test/queries/clientpositive/udaf_corr.q | 2 +- .../queries/clientpositive/udaf_covar_pop.q | 2 +- .../queries/clientpositive/udaf_covar_samp.q | 2 +- .../test/queries/clientpositive/udaf_ngrams.q | 2 +- .../queries/clientpositive/udaf_percentile.q | 1 + .../udaf_percentile_approx_20.q | 8 +- .../udaf_percentile_approx_23.q | 8 +- .../queries/clientpositive/udaf_sum_list.q | 6 + .../src/test/queries/clientpositive/udf_E.q | 12 +- .../src/test/queries/clientpositive/udf_PI.q | 12 +- .../src/test/queries/clientpositive/udf_abs.q | 10 +- .../test/queries/clientpositive/udf_acos.q | 10 +- .../test/queries/clientpositive/udf_array.q | 6 +- .../clientpositive/udf_array_contains.q | 6 +- .../test/queries/clientpositive/udf_ascii.q | 6 +- .../test/queries/clientpositive/udf_asin.q | 10 +- .../test/queries/clientpositive/udf_atan.q | 10 +- .../test/queries/clientpositive/udf_between.q | 2 + .../src/test/queries/clientpositive/udf_bin.q | 6 +- .../queries/clientpositive/udf_bitmap_and.q | 8 +- .../queries/clientpositive/udf_bitmap_empty.q | 6 +- .../queries/clientpositive/udf_bitmap_or.q | 8 +- .../test/queries/clientpositive/udf_case.q | 25 +- .../queries/clientpositive/udf_case_thrift.q | 6 +- .../queries/clientpositive/udf_coalesce.q | 6 +- .../clientpositive/udf_compare_java_string.q | 2 + .../test/queries/clientpositive/udf_concat.q | 10 +- .../queries/clientpositive/udf_concat_ws.q | 8 +- .../test/queries/clientpositive/udf_conv.q | 20 +- .../src/test/queries/clientpositive/udf_cos.q | 6 +- .../clientpositive/udf_current_database.q | 26 + .../test/queries/clientpositive/udf_degrees.q | 12 +- .../src/test/queries/clientpositive/udf_div.q | 4 +- .../test/queries/clientpositive/udf_divide.q | 4 +- .../src/test/queries/clientpositive/udf_elt.q | 6 +- .../test/queries/clientpositive/udf_equal.q | 6 +- .../test/queries/clientpositive/udf_explode.q | 22 +- .../test/queries/clientpositive/udf_field.q | 12 +- .../queries/clientpositive/udf_find_in_set.q | 28 +- .../clientpositive/udf_format_number.q | 16 +- .../clientpositive/udf_get_json_object.q | 8 +- .../queries/clientpositive/udf_greaterthan.q | 4 +- .../clientpositive/udf_greaterthanorequal.q | 4 +- .../test/queries/clientpositive/udf_hash.q | 6 +- .../src/test/queries/clientpositive/udf_hex.q | 8 +- .../test/queries/clientpositive/udf_hour.q | 2 + .../src/test/queries/clientpositive/udf_if.q | 10 +- .../src/test/queries/clientpositive/udf_in.q | 4 +- .../test/queries/clientpositive/udf_in_file.q | 12 +- .../test/queries/clientpositive/udf_index.q | 1 + .../test/queries/clientpositive/udf_inline.q | 2 + .../test/queries/clientpositive/udf_instr.q | 6 +- .../clientpositive/udf_isnull_isnotnull.q | 2 + .../queries/clientpositive/udf_java_method.q | 6 +- .../test/queries/clientpositive/udf_length.q | 4 +- .../queries/clientpositive/udf_lessthan.q | 4 +- .../clientpositive/udf_lessthanorequal.q | 4 +- .../test/queries/clientpositive/udf_like.q | 4 +- .../test/queries/clientpositive/udf_locate.q | 6 +- .../clientpositive/udf_logic_java_boolean.q | 2 + .../test/queries/clientpositive/udf_lpad.q | 6 +- .../src/test/queries/clientpositive/udf_map.q | 6 +- .../queries/clientpositive/udf_map_keys.q | 6 +- .../queries/clientpositive/udf_map_values.q | 6 +- .../queries/clientpositive/udf_named_struct.q | 6 +- .../queries/clientpositive/udf_negative.q | 14 +- .../queries/clientpositive/udf_notequal.q | 2 + .../test/queries/clientpositive/udf_notop.q | 4 +- .../src/test/queries/clientpositive/udf_nvl.q | 5 +- .../test/queries/clientpositive/udf_pmod.q | 20 +- .../test/queries/clientpositive/udf_printf.q | 16 +- .../test/queries/clientpositive/udf_radians.q | 16 +- .../test/queries/clientpositive/udf_reflect.q | 6 +- .../queries/clientpositive/udf_reflect2.q | 2 + .../test/queries/clientpositive/udf_regexp.q | 4 +- .../test/queries/clientpositive/udf_repeat.q | 6 +- .../test/queries/clientpositive/udf_reverse.q | 2 +- .../test/queries/clientpositive/udf_round.q | 14 +- .../test/queries/clientpositive/udf_round_2.q | 8 +- .../test/queries/clientpositive/udf_round_3.q | 12 +- .../test/queries/clientpositive/udf_rpad.q | 6 +- .../test/queries/clientpositive/udf_second.q | 2 + .../test/queries/clientpositive/udf_sign.q | 20 +- .../src/test/queries/clientpositive/udf_sin.q | 6 +- .../test/queries/clientpositive/udf_size.q | 2 + .../queries/clientpositive/udf_sort_array.q | 14 +- .../test/queries/clientpositive/udf_space.q | 8 +- .../test/queries/clientpositive/udf_split.q | 6 +- .../test/queries/clientpositive/udf_struct.q | 6 +- .../test/queries/clientpositive/udf_substr.q | 18 +- .../src/test/queries/clientpositive/udf_tan.q | 10 +- .../queries/clientpositive/udf_testlength.q | 8 +- .../queries/clientpositive/udf_testlength2.q | 8 +- .../queries/clientpositive/udf_to_boolean.q | 60 +- .../test/queries/clientpositive/udf_to_byte.q | 22 +- .../queries/clientpositive/udf_to_double.q | 22 +- .../queries/clientpositive/udf_to_float.q | 22 +- .../test/queries/clientpositive/udf_to_long.q | 22 +- .../queries/clientpositive/udf_to_short.q | 22 +- .../queries/clientpositive/udf_to_string.q | 24 +- .../clientpositive/udf_to_unix_timestamp.q | 4 +- .../queries/clientpositive/udf_translate.q | 16 +- .../test/queries/clientpositive/udf_unhex.q | 6 +- .../test/queries/clientpositive/udf_union.q | 6 +- .../clientpositive/udf_unix_timestamp.q | 4 +- .../test/queries/clientpositive/udf_using.q | 15 + .../queries/clientpositive/udf_weekofyear.q | 4 +- .../test/queries/clientpositive/udf_when.q | 23 +- .../test/queries/clientpositive/udf_xpath.q | 12 +- .../clientpositive/udf_xpath_boolean.q | 14 +- .../queries/clientpositive/udf_xpath_double.q | 18 +- .../queries/clientpositive/udf_xpath_float.q | 18 +- .../queries/clientpositive/udf_xpath_int.q | 18 +- .../queries/clientpositive/udf_xpath_long.q | 18 +- .../queries/clientpositive/udf_xpath_short.q | 18 +- .../queries/clientpositive/udf_xpath_string.q | 18 +- .../queries/clientpositive/udtf_explode.q | 2 + .../queries/clientpositive/udtf_json_tuple.q | 14 +- .../clientpositive/udtf_parse_url_tuple.q | 12 +- .../queries/clientpositive/udtf_posexplode.q | 15 + .../src/test/queries/clientpositive/union.q | 6 +- .../src/test/queries/clientpositive/union34.q | 2 +- .../test/queries/clientpositive/union_date.q | 4 +- .../test/queries/clientpositive/union_null.q | 3 + .../queries/clientpositive/union_remove_1.q | 2 +- .../queries/clientpositive/union_remove_10.q | 2 +- .../queries/clientpositive/union_remove_11.q | 2 +- .../queries/clientpositive/union_remove_12.q | 2 +- .../queries/clientpositive/union_remove_13.q | 2 +- .../queries/clientpositive/union_remove_14.q | 2 +- .../queries/clientpositive/union_remove_15.q | 2 +- .../queries/clientpositive/union_remove_16.q | 2 +- .../queries/clientpositive/union_remove_17.q | 2 +- .../queries/clientpositive/union_remove_18.q | 2 +- .../queries/clientpositive/union_remove_19.q | 2 +- .../queries/clientpositive/union_remove_2.q | 2 +- .../queries/clientpositive/union_remove_20.q | 2 +- .../queries/clientpositive/union_remove_21.q | 2 +- .../queries/clientpositive/union_remove_22.q | 2 +- .../queries/clientpositive/union_remove_23.q | 2 +- .../queries/clientpositive/union_remove_24.q | 2 +- .../queries/clientpositive/union_remove_3.q | 2 +- .../queries/clientpositive/union_remove_4.q | 2 +- .../queries/clientpositive/union_remove_5.q | 2 +- .../queries/clientpositive/union_remove_6.q | 2 +- .../queries/clientpositive/union_remove_7.q | 2 +- .../queries/clientpositive/union_remove_8.q | 2 +- .../queries/clientpositive/union_remove_9.q | 2 +- .../queries/clientpositive/union_top_level.q | 106 + .../test/queries/clientpositive/union_view.q | 1 + .../test/queries/clientpositive/uniquejoin.q | 6 +- .../test/queries/clientpositive/varchar_1.q | 2 +- .../queries/clientpositive/varchar_cast.q | 1 + .../clientpositive/varchar_comparison.q | 1 + .../queries/clientpositive/varchar_join1.q | 6 +- .../queries/clientpositive/varchar_serde.q | 2 +- .../queries/clientpositive/varchar_union1.q | 6 +- .../clientpositive/vector_between_in.q | 35 + .../queries/clientpositive/vector_coalesce.q | 32 + .../clientpositive/vector_decimal_aggregate.q | 20 + .../clientpositive/vector_decimal_cast.q | 5 + .../vector_decimal_expressions.q | 5 + .../clientpositive/vector_decimal_mapjoin.q | 19 + .../vector_decimal_math_funcs.q | 77 + .../clientpositive/vector_left_outer_join.q | 21 + .../vector_non_string_partition.q | 17 + .../queries/clientpositive/vectorization_0.q | 27 + .../queries/clientpositive/vectorization_1.q | 21 + .../queries/clientpositive/vectorization_10.q | 24 + .../queries/clientpositive/vectorization_11.q | 15 + .../queries/clientpositive/vectorization_12.q | 32 + .../queries/clientpositive/vectorization_13.q | 31 + .../queries/clientpositive/vectorization_14.q | 33 + .../queries/clientpositive/vectorization_15.q | 31 + .../queries/clientpositive/vectorization_16.q | 20 + .../queries/clientpositive/vectorization_2.q | 23 + .../queries/clientpositive/vectorization_3.q | 25 + .../queries/clientpositive/vectorization_4.q | 23 + .../queries/clientpositive/vectorization_5.q | 20 + .../queries/clientpositive/vectorization_6.q | 21 + .../queries/clientpositive/vectorization_7.q | 25 + .../queries/clientpositive/vectorization_8.q | 23 + .../queries/clientpositive/vectorization_9.q | 24 + .../vectorization_decimal_date.q | 4 + .../clientpositive/vectorization_div0.q | 24 + .../clientpositive/vectorization_limit.q | 37 + .../clientpositive/vectorization_nested_udf.q | 3 + .../clientpositive/vectorization_not.q | 27 + .../clientpositive/vectorization_part.q | 7 + .../vectorization_part_project.q | 7 + .../clientpositive/vectorization_pushdown.q | 4 + .../vectorization_short_regress.q | 852 ++++++ .../vectorized_bucketmapjoin1.q | 46 + .../queries/clientpositive/vectorized_case.q | 37 + .../queries/clientpositive/vectorized_casts.q | 149 + .../clientpositive/vectorized_context.q | 47 + .../clientpositive/vectorized_date_funcs.q | 122 + .../clientpositive/vectorized_distinct_gby.q | 12 + .../clientpositive/vectorized_mapjoin.q | 12 + .../clientpositive/vectorized_math_funcs.q | 107 + .../vectorized_nested_mapjoin.q | 8 + .../vectorized_rcfile_columnar.q | 18 + .../clientpositive/vectorized_shufflejoin.q | 10 + .../clientpositive/vectorized_string_funcs.q | 46 + .../vectorized_timestamp_funcs.q | 124 + .../ql/src/test/queries/clientpositive/view.q | 4 +- .../test/queries/clientpositive/view_cast.q | 6 +- .../test/queries/clientpositive/windowing.q | 2 +- .../windowing_adjust_rowcontainer_sz.q | 2 +- .../clientpositive/windowing_columnPruning.q | 2 +- .../clientpositive/windowing_expressions.q | 6 +- .../windowing_multipartitioning.q | 4 +- .../queries/clientpositive/windowing_navfn.q | 6 +- .../queries/clientpositive/windowing_ntile.q | 4 +- .../queries/clientpositive/windowing_rank.q | 4 +- .../queries/clientpositive/windowing_udaf.q | 2 +- .../queries/clientpositive/windowing_udaf2.q | 4 + .../clientpositive/windowing_windowspec.q | 2 +- .../spark/sql/hive/StatisticsSuite.scala | 2 - .../sql/hive/api/java/JavaHiveQLSuite.scala | 6 +- .../hive/execution/HiveComparisonTest.scala | 19 +- .../sql/hive/execution/HiveQuerySuite.scala | 16 +- .../org/apache/spark/sql/hive/Shim.scala | 89 - .../org/apache/spark/sql/hive/Shim12.scala | 96 + .../org/apache/spark/sql/hive/Shim.scala | 170 -- .../org/apache/spark/sql/hive/Shim13.scala | 169 ++ 8166 files changed, 38307 insertions(+), 47487 deletions(-) create mode 100644 sql/hive/src/test/resources/data/files/ProxyAuth.res create mode 100644 sql/hive/src/test/resources/data/files/alltypes.txt create mode 100644 sql/hive/src/test/resources/data/files/alltypes2.txt create mode 100644 sql/hive/src/test/resources/data/files/alltypesorc create mode 100644 sql/hive/src/test/resources/data/files/char_varchar_udf.txt create mode 100644 sql/hive/src/test/resources/data/files/decimal.txt create mode 100644 sql/hive/src/test/resources/data/files/dept.txt create mode 100644 sql/hive/src/test/resources/data/files/emp.txt create mode 100644 sql/hive/src/test/resources/data/files/exported_table/_metadata create mode 100644 sql/hive/src/test/resources/data/files/exported_table/data/data create mode 100644 sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt create mode 100644 sql/hive/src/test/resources/data/files/futurama_episodes.avro create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt create mode 100644 sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt rename sql/hive/src/test/resources/{golden/alter3-1-75be487df30e301e156a22eee075633d => data/files/header_footer_table_3/empty1.txt} (100%) rename sql/hive/src/test/resources/{golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 => data/files/header_footer_table_3/empty2.txt} (100%) create mode 100644 sql/hive/src/test/resources/data/files/input.txt create mode 100644 sql/hive/src/test/resources/data/files/keystore.jks create mode 100644 sql/hive/src/test/resources/data/files/kv9.txt create mode 100644 sql/hive/src/test/resources/data/files/loc.txt create mode 100644 sql/hive/src/test/resources/data/files/non_ascii_tbl.txt create mode 100644 sql/hive/src/test/resources/data/files/orc_split_elim.orc create mode 100644 sql/hive/src/test/resources/data/files/parquet_create.txt create mode 100644 sql/hive/src/test/resources/data/files/parquet_partitioned.txt create mode 100644 sql/hive/src/test/resources/data/files/parquet_types.txt create mode 100644 sql/hive/src/test/resources/data/files/person+age.txt create mode 100644 sql/hive/src/test/resources/data/files/posexplode_data.txt create mode 100644 sql/hive/src/test/resources/data/files/sample.json create mode 100644 sql/hive/src/test/resources/data/files/truststore.jks create mode 100644 sql/hive/src/test/resources/data/scripts/input20_script.py delete mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 delete mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c delete mode 100644 sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d rename sql/hive/src/test/resources/golden/{alter5-1-b8349afaf8e62dc6608a889c04ee3d4b => alter3-1-47f70392b97b94cdb425b25bde204f58} (100%) rename sql/hive/src/test/resources/golden/{alter5-15-b8349afaf8e62dc6608a889c04ee3d4b => alter3-21-91e32b3028ecc352dad8884829148311} (100%) rename sql/hive/src/test/resources/golden/{alter_merge_2-3-cfef140167765d259320ed1e8aba718d => alter5-1-cbad54fbd4a08fc8717708f93358ec3e} (100%) rename sql/hive/src/test/resources/golden/{alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be => alter5-15-cbad54fbd4a08fc8717708f93358ec3e} (100%) rename sql/hive/src/test/resources/golden/{alter_merge_2-5-43bd090cda258e25037e7c32d500a85f => alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a => alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 => alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147} (100%) delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd delete mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 rename sql/hive/src/test/resources/golden/{alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 => alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 => alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e => alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 => alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f => alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 => alter_rename_partition-4-833254c8f7c5b1248e1a249560652627} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 => alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d => alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d} (100%) rename sql/hive/src/test/resources/golden/{combine3-0-84b74227c9f1563f530cd3ac3b333e54 => alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c delete mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c delete mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb delete mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 rename sql/hive/src/test/resources/golden/{combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c => ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d => ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 => ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 => ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b} (100%) rename sql/hive/src/test/resources/golden/{combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 => annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c => annotate_stats_join-1-84e7846d50fc15e836c83911ce039871} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 => annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 => annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60} (100%) rename sql/hive/src/test/resources/golden/{alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 => annotate_stats_join-12-cef44682d339a67ba765f854da21f976} (100%) rename sql/hive/src/test/resources/golden/{alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c => annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e} (100%) rename sql/hive/src/test/resources/golden/{alter_rename_partition-4-3479a886936571d5028971aecade705f => annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1} (100%) rename sql/hive/src/test/resources/golden/{alter_varchar2-2-325238d61f56d84c17e29033105d7b19 => annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 => annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26} (100%) rename sql/hive/src/test/resources/golden/{alter_view_rename-0-bb255b994b5207324fba6988caa937e6 => annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2} (100%) rename sql/hive/src/test/resources/golden/{alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d => annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957} (100%) rename sql/hive/src/test/resources/golden/{alter_view_rename-3-95655e33f22fc8f66549a9708812589a => annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589} (100%) rename sql/hive/src/test/resources/golden/{ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d => annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600} (100%) rename sql/hive/src/test/resources/golden/{ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab => annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8} (100%) rename sql/hive/src/test/resources/golden/{ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b => annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2} (100%) rename sql/hive/src/test/resources/golden/{archive-10-f845008104fd12eb0f13f4b113f95cf1 => annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701} (100%) rename sql/hive/src/test/resources/golden/{archive-12-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_join-23-fa90806f6137300311df032e28df3d4c} (100%) rename sql/hive/src/test/resources/golden/{archive-16-892147913578bcf60620b7dd73893dd0 => annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3} (100%) rename sql/hive/src/test/resources/golden/{archive-18-8f980275ab3a5bcfc1784f4acd46447a => annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{archive-19-f8a52a8a40141409a667a9ba2cf9630f => annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8} (100%) rename sql/hive/src/test/resources/golden/{archive-2-713efc113418b01f76ffd589840193c8 => annotate_stats_join-5-d12ba848d20d1def95379c490068f050} (100%) rename sql/hive/src/test/resources/golden/{archive-21-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_join-6-96a4806e61c5d882affac57e115f285f} (100%) rename sql/hive/src/test/resources/golden/{archive-23-892147913578bcf60620b7dd73893dd0 => annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2} (100%) rename sql/hive/src/test/resources/golden/{archive-25-56bea24ffa83d9b5932893a8ff1cb44f => annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{archive-26-44fa61c2bd0fd9acfa0d889e59880d8a => annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{combine3-15-7cccbdffc32975f8935eeba14a28147 => annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{archive-27-9ae773ebe64a3d437a035e9d94f49e5 => annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{archive-29-a0c6922e3a1dca51861c8a872dc8af19 => annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0} (100%) rename sql/hive/src/test/resources/golden/{archive-3-27ad2962fed131f51ba802596ba37278 => annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{archive-31-cd46bc635e3010cf1b990a652a584a09 => annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{archive-32-27ad2962fed131f51ba802596ba37278 => annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3} (100%) rename sql/hive/src/test/resources/golden/{archive-4-3e95421993ab28d18245ec2340f580a3 => annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{archive-5-c0c18ac884677231a41eea8d980d0451 => annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96} (100%) rename sql/hive/src/test/resources/golden/{archive-6-528ab9750a558af7f1a43b3108e793dd => annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40} (100%) rename sql/hive/src/test/resources/golden/{archive-7-e8d1d10c308a73eef78dde414a5e40ca => annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb} (100%) rename sql/hive/src/test/resources/golden/{archive-8-af459a0264559a2aeaa1341ce779ab3c => annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f} (100%) rename sql/hive/src/test/resources/golden/{combine3-2-c95dc367df88c9e5cf77157f29ba2daf => annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 => annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0} (100%) rename sql/hive/src/test/resources/golden/{archive-9-48b10f27e1459bb8e62d6c71484e2cf => annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 => annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 => annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a => annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f => annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 => annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 => annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 => annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f => annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1} (100%) rename sql/hive/src/test/resources/golden/{combine3-4-84967075baa3e56fff2a23f8ab9ba076 => annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a => annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 => annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 => annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 => annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 => annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 => annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 => annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 => annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd => annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e} (100%) rename sql/hive/src/test/resources/golden/{combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea => annotate_stats_part-3-16367c381d4b189b3640c92511244bfe} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca => annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c => annotate_stats_part-5-cef44682d339a67ba765f854da21f976} (100%) rename sql/hive/src/test/resources/golden/{archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf => annotate_stats_part-6-f4263aca1612c05cb08242448191ad05} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 => annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-12-8419ad4ed6683ebd15f993f703975b31 => annotate_stats_part-8-84e3979946de67779a9704a3adc2184f} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d => annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b => annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f => annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-2-cd46bc635e3010cf1b990a652a584a09 => annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-3-27ad2962fed131f51ba802596ba37278 => annotate_stats_table-11-4be11140c2e21b3a172224d103844785} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-4-3e95421993ab28d18245ec2340f580a3 => annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-5-c0c18ac884677231a41eea8d980d0451 => annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a => annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca => annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-8-af459a0264559a2aeaa1341ce779ab3c => annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d} (100%) rename sql/hive/src/test/resources/golden/{archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf => annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c} (100%) rename sql/hive/src/test/resources/golden/{auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf => annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 => annotate_stats_table-2-84e7846d50fc15e836c83911ce039871} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-13-63241e3791725baad8baa00fb833ef5e => annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f => annotate_stats_table-4-d12ba848d20d1def95379c490068f050} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-15-187fd938996ae7d96e60475fb69b8d35 => annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-20-e67740fb52998f1d3afcfa667505cf7 => annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b => annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-22-a1d339a0d904c3f35771192a026c7f9c => annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d} (100%) rename sql/hive/src/test/resources/golden/{auto_join32-23-1948951cc3c06cdf962d59e932a84588 => annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 => annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 => annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b => annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-32-3c52df82c7d78501610f3f898103f753 => annotate_stats_union-11-c1302094d7b4456550826535b529468b} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c => annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0} (100%) rename sql/hive/src/test/resources/golden/{auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 => annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178} (100%) rename sql/hive/src/test/resources/golden/{auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 => annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 => annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 => annotate_stats_union-16-4252006172a476fbc591eebee49bffa3} (100%) rename sql/hive/src/test/resources/golden/{auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b => annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 => annotate_stats_union-18-58d55f8c8b2489726232a00254499144} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 => annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 => annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 => annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a => annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab => annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 => annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 => annotate_stats_union-4-4252006172a476fbc591eebee49bffa3} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d => annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 => annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 => annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 => annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40} (100%) delete mode 100644 sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 delete mode 100644 sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 delete mode 100644 sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 delete mode 100644 sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 delete mode 100644 sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 delete mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 delete mode 100644 sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb delete mode 100644 sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a delete mode 100644 sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a => auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab => auto_join32-14-7927c2ce644d1ce1de251405c8563e99} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 => auto_join32-15-4cd3b51861720ac06c6deb818c83670} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 => auto_join32-20-693874ea8e06a8b155394ac27af2b1a7} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d => auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_join32-22-3d14d63e996851f51a98f987995d8da6} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 => auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 => auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 => auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a => auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 => auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 => auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a => auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab => auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 => auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 => auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a => auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 => auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 => auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 => auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a => auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 => auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 => auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 => auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 => auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 => auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c => auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 => auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 => auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833} (100%) rename sql/hive/src/test/resources/golden/{decimal_precision-14-34916eb904b8113a401ce78e6941a204 => auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 => auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a => auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041} (100%) create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b => auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e} (100%) rename sql/hive/src/test/resources/golden/{escape1-0-a4fb8359a2179ec70777aad6366071b7 => auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06} (100%) rename sql/hive/src/test/resources/golden/{escape1-1-683124e29877d2c5a96b95c8ddba97b7 => auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb} (100%) rename sql/hive/src/test/resources/golden/{escape2-0-a4fb8359a2179ec70777aad6366071b7 => auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f} (100%) rename sql/hive/src/test/resources/golden/{escape2-1-683124e29877d2c5a96b95c8ddba97b7 => auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a => auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 => auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 => auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 => auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 => auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d => auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 => auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 => auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 => auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 => auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 => auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f => auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a => auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab => auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 => auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20} (100%) rename sql/hive/src/test/resources/golden/{auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 => auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 => auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a => auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39} (100%) rename sql/hive/src/test/resources/golden/{ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 => auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3} (100%) rename sql/hive/src/test/resources/golden/{ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{ba_table3-1-28b12606c5369c783e63c17826a18d0d => auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5} (100%) rename sql/hive/src/test/resources/golden/{ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 => auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d} (100%) rename sql/hive/src/test/resources/golden/{binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 => auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e} (100%) rename sql/hive/src/test/resources/golden/{binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef => auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed} (100%) rename sql/hive/src/test/resources/golden/{binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 => auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3} (100%) rename sql/hive/src/test/resources/golden/{binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb} (100%) rename sql/hive/src/test/resources/golden/{binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 => auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe} (100%) rename sql/hive/src/test/resources/golden/{binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e => auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5} (100%) rename sql/hive/src/test/resources/golden/{binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 => auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 => auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 => auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d => auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c => auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 => auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c => auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 => auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 => auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd => auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 => auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5} (100%) rename sql/hive/src/test/resources/golden/{columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe => auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 => auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c => auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 => auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c => auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 => auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06} (100%) rename sql/hive/src/test/resources/golden/{columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f => auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 => auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 => auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 => auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 => auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 => auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde} (100%) rename sql/hive/src/test/resources/golden/{combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 => auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 rename sql/hive/src/test/resources/golden/{escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c => binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 create mode 100644 sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 delete mode 100644 sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 delete mode 100644 sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 delete mode 100644 sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 rename sql/hive/src/test/resources/golden/{combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 => binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948} (100%) delete mode 100644 sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 rename sql/hive/src/test/resources/golden/{combine2_win-12-5ac3e540fd24f94fee378e49597817b3 => char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08} (100%) rename sql/hive/src/test/resources/golden/{combine2_win-14-dd652175dac4463fed3c56aded11e6c1 => char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902} (100%) rename sql/hive/src/test/resources/golden/{combine2_win-16-8e4598e3f0701478ed12042438699ce5 => char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce delete mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 delete mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 delete mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 delete mode 100644 sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 delete mode 100644 sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 delete mode 100644 sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b delete mode 100644 sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc delete mode 100644 sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 rename sql/hive/src/test/resources/golden/{combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af => compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051} (100%) rename sql/hive/src/test/resources/golden/{combine3-10-fb173ed4483e732d367e51f88be793b1 => compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d} (100%) rename sql/hive/src/test/resources/golden/{combine3-16-6635f7f5c55557b06ad3acc321eaa739 => compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d} (100%) rename sql/hive/src/test/resources/golden/{combine3-17-8cb751103da7c909276db6bddb50ae6a => compute_stats_long-1-9313f166464633b3929707d7ef11d758} (100%) rename sql/hive/src/test/resources/golden/{combine3-18-31fae7c6be75b97d475512bd75a58a0b => compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb} (100%) rename sql/hive/src/test/resources/golden/{combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f => correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3} (100%) rename sql/hive/src/test/resources/golden/{combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd => correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65} (100%) create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f rename sql/hive/src/test/resources/golden/{combine3-22-11025483569617a9f014b5defd71e933 => correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95} (100%) create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f rename sql/hive/src/test/resources/golden/{combine3-23-4725c48df09565618cbffd05953a5f62 => correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 => correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7} (100%) rename sql/hive/src/test/resources/golden/{combine3-6-4725c48df09565618cbffd05953a5f62 => correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 => correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7} (100%) rename sql/hive/src/test/resources/golden/{combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 => correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad => correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad} (100%) rename sql/hive/src/test/resources/golden/{combine3-8-68399bc39f71ddc99ed09ed9d2fd897b => correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad => correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad} (100%) rename sql/hive/src/test/resources/golden/{escape2-3-1774adb1085f4ee6782a8dac0735399 => correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f} (100%) rename sql/hive/src/test/resources/golden/{combine3-9-b5703b76bbe99c41cbb63582a09a6e69 => correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba => correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee => correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 => correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba => correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a => correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 => correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-52-3070366869308907e54797927805603 => correlationoptimizer1-58-3070366869308907e54797927805603} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 => correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-55-3070366869308907e54797927805603 => correlationoptimizer1-61-3070366869308907e54797927805603} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 => correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_long-1-a7bc730f9862198709539e35c0208248 => correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 => correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee => correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6} (100%) rename sql/hive/src/test/resources/golden/{compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f => correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 => correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 => correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 => correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab => correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 => count-1-c47c4abedf055b4e734232fd2e274d55} (100%) delete mode 100644 sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f delete mode 100644 sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d delete mode 100644 sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba delete mode 100644 sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 delete mode 100644 sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a delete mode 100644 sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 delete mode 100644 sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 delete mode 100644 sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 delete mode 100644 sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 delete mode 100644 sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a delete mode 100644 sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 delete mode 100644 sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 delete mode 100644 sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 delete mode 100644 sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 delete mode 100644 sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 delete mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 delete mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 delete mode 100644 sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af delete mode 100644 sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 delete mode 100644 sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 delete mode 100644 sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 delete mode 100644 sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d delete mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 rename sql/hive/src/test/resources/golden/{correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 => create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79} (100%) delete mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 rename sql/hive/src/test/resources/golden/{correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 => create_struct_table-1-719851d0e8b89b51bdc6be4177455a92} (100%) delete mode 100644 sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e delete mode 100644 sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b delete mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 delete mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc rename sql/hive/src/test/resources/golden/{correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 => cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 => cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a => cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 => cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 => cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 => cross_product_check_1-5-103739f072b849d212dbc40919f92b74} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b => cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 => cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac} (100%) rename sql/hive/src/test/resources/golden/{correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 => cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594} (100%) rename sql/hive/src/test/resources/golden/{count-1-3531872d964bc2b4f07d51b9d1629df2 => cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 => cross_product_check_2-2-24ca942f094b14b92086305cc125e833} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee => cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a => cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c} (100%) rename sql/hive/src/test/resources/golden/{create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 => cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254} (100%) rename sql/hive/src/test/resources/golden/{create_1-1-c2351f011b8ea41ff7dfa8f195148da3 => cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970} (100%) rename sql/hive/src/test/resources/golden/{create_1-11-7daaeabd9c286e511e0628a32dc714d5 => cross_product_check_2-7-103739f072b849d212dbc40919f92b74} (100%) rename sql/hive/src/test/resources/golden/{create_1-14-437bd1acbae61f48e851addb769d3379 => cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25} (100%) rename sql/hive/src/test/resources/golden/{create_1-4-610b82bf7b0080d293977927e5ef780c => cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec delete mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 rename sql/hive/src/test/resources/golden/{create_1-5-c77b018276b1558c1d9462e0625e152e => database_drop-10-8db536f925bf0f5058f97897e145a661} (100%) rename sql/hive/src/test/resources/golden/{create_1-8-19331fe6a2a35f1171babfe4e1c86f59 => database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5} (100%) rename sql/hive/src/test/resources/golden/{database_drop-11-2ea883422b74b701711e14e61472ba06 => database_drop-12-2ea883422b74b701711e14e61472ba06} (100%) rename sql/hive/src/test/resources/golden/{database_drop-12-e02a53f7e798d2741152526516f14941 => database_drop-13-e02a53f7e798d2741152526516f14941} (100%) rename sql/hive/src/test/resources/golden/{create_big_view-0-bdf3c2e20793ef833f336a40791091d => database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35} (100%) rename sql/hive/src/test/resources/golden/{create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee => database_drop-15-7928ac876f76c990fa21d74b6c9e14f6} (100%) rename sql/hive/src/test/resources/golden/{create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e => database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e} (100%) rename sql/hive/src/test/resources/golden/{create_escape-3-9541399cde42210bd7ac1beb07ceb14 => database_drop-17-95b906fd73935da8746b5277170e91e8} (100%) rename sql/hive/src/test/resources/golden/{create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b => database_drop-18-f65bf675b83871af7906741a60fa1318} (100%) rename sql/hive/src/test/resources/golden/{create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 => database_drop-19-d7fefbf585dbb67491e871ef58dca752} (100%) rename sql/hive/src/test/resources/golden/{create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 => database_drop-20-cbb84f0db4d55930a85cff28f7400bd0} (100%) rename sql/hive/src/test/resources/golden/{create_like-0-c2351f011b8ea41ff7dfa8f195148da3 => database_drop-21-eea49f066631be60519ae9d6b614d7d0} (100%) rename sql/hive/src/test/resources/golden/{create_like-2-a20451f152e68606cc5e373fe5fd86a => database_drop-22-85833c3a68c29098827e438ff580bb94} (100%) rename sql/hive/src/test/resources/golden/{create_like-4-39ead53334938635b60a5ffdaa2c9f86 => database_drop-23-84a5672989118a1b5792474c1469de90} (100%) rename sql/hive/src/test/resources/golden/{create_like-5-dc9de26002604e9e436135bd4b40636d => database_drop-24-441116797e8d95554c74472fa7644440} (100%) rename sql/hive/src/test/resources/golden/{database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 => database_drop-25-847ca97dd211889d83e5d487bfc44e23} (100%) rename sql/hive/src/test/resources/golden/{database_drop-4-d419f4ff197d4291208c2028cd158909 => database_drop-26-bea20178515df24fcca04c0384f1c1b7} (100%) rename sql/hive/src/test/resources/golden/{create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 => database_drop-27-cb4b90a7f63c1646319ee7bb014a7750} (100%) rename sql/hive/src/test/resources/golden/{create_like-8-639a13da6855b974fc2e170fd49b33cb => database_drop-28-53d67cbed634cff012dac41340bf6630} (100%) rename sql/hive/src/test/resources/golden/{create_like-9-a0ce573e299b66b8ce31da2890b318cb => database_drop-29-29d3232325eda40cbb0bd1786b7d070e} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 => database_drop-3-db64b724719d27c7f0db4f51f5c4edaa} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-0-366a4de0343396b9df03277f1098722c => database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 => database_drop-31-981f8e58caeae9cbbad3a113e3043de5} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f => database_drop-32-6c8e7590238b5aca1772721f0b914ece} (100%) rename sql/hive/src/test/resources/golden/{database_drop-5-b7cf74929eabe781b0db79ed1043dc24 => database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 => database_drop-34-491138bed44a70cb783bb2b531e1d82} (100%) rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 => database_drop-35-9e0285d0596607cdadf75a763a543866} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-14-32251c08304629a3153e0b471de060c5 => database_drop-36-e66471f3f1bbe2d4b3b214e47793656d} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 => database_drop-37-f6410721e3125a89836817136306eac4} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 => database_drop-38-7d45d97adebe50917a94cbe232c112a8} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 => database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 => database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0} (100%) rename sql/hive/src/test/resources/golden/{create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 => database_drop-40-10073fb74a5c792322fc52389997695b} (100%) rename sql/hive/src/test/resources/golden/{create_nested_type-3-ac452c9279877935983c37113898e53c => database_drop-41-7164c585e1ef4d9036ed4db275811084} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d => database_drop-42-c55cffbfc4d950363be97ccdb028faf3} (100%) rename sql/hive/src/test/resources/golden/{exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 => database_drop-43-cc6860c125b8b62450cb858c72716dc2} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee => database_drop-44-de81fd80132350aedcd9f0d9a212fd94} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea => database_drop-45-64e22634663e57153119340583e93651} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d => database_drop-46-eb28b907b605e51026f9902287e1d90d} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-1-baeaf0da490037e7ada642d23013075a => database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => database_drop-48-d419f4ff197d4291208c2028cd158909} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 => database_drop-5-d419f4ff197d4291208c2028cd158909} (100%) rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 => database_drop-6-b7cf74929eabe781b0db79ed1043dc24} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de => database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9} (100%) rename sql/hive/src/test/resources/golden/{create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb => database_drop-8-b3980119a4b2d8f531951a940cba3697} (100%) rename sql/hive/src/test/resources/golden/{create_struct_table-1-2ca90a28a994405e6150c96f4a572294 => database_drop-9-97101266791d2b2c662bcde549422318} (100%) rename sql/hive/src/test/resources/golden/{create_union_table-0-8e765b54f15b948fc88392da69da283 => date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1} (100%) rename sql/hive/src/test/resources/golden/{create_union_table-1-aee4ce62fc2631423af0f569f4448353 => date_3-3-c26f0641e7cec1093273b258e6bf7120} (100%) rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee => date_4-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 => date_4-1-b84f7e931d710dcbe3c5126d998285a8} (100%) rename sql/hive/src/test/resources/golden/{create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 => date_4-2-6272f5e518f6a20bc96a5870ff315c4f} (100%) rename sql/hive/src/test/resources/golden/{create_view-1-c186ac1fe46117acb6fd452df15e0d92 => date_4-3-4a0e7bde447ef616b98e0f55d2886de0} (100%) rename sql/hive/src/test/resources/golden/{create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd => date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8} (100%) create mode 100644 sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 rename sql/hive/src/test/resources/golden/{create_view-11-2021c047203276dd2db51a56e672fcea => date_4-6-b84f7e931d710dcbe3c5126d998285a8} (100%) rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a => date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 => date_comparison-1-69eec445bd045c9dc899fafa348d8495} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 => date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc => date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 => date_comparison-12-4a7bac9ddcf40db6329faaec8e426543} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-3-13e17ed811165196416f777cbc162592 => date_comparison-2-fcc400871a502009c8680509e3869ec1} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 => date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f => date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 => date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 => date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 => date_comparison-7-5606505a92bad10023ad9a3ef77eacc9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 => date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-11-441306cae24618c49ec63445a31bf16b => date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c} (100%) rename sql/hive/src/test/resources/golden/{create_view-12-420752b11848e29bce1c8ace7d3060fc => date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964} (100%) rename sql/hive/src/test/resources/golden/{create_view-13-bff53e5df8356ac16f7b9b78b157e60a => date_serde-7-580096b3b48db26bea91b80e1e1b081a} (100%) rename sql/hive/src/test/resources/golden/{create_view-14-69162f2f22978113fea529d7fc7b78d3 => date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d delete mode 100644 sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d delete mode 100644 sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 delete mode 100644 sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 rename sql/hive/src/test/resources/golden/{create_view-15-ceebf4cb0dc23f517a444266bc8d2447 => decimal_4-2-85c3185beb011f5c1e6856fc773a7484} (100%) rename sql/hive/src/test/resources/golden/{create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 => decimal_4-3-1451d7491441c1632fd5f751876cce6e} (100%) rename sql/hive/src/test/resources/golden/{create_view-17-b3c1023d56f3439833c246e8bfd4502a => decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d} (100%) rename sql/hive/src/test/resources/golden/{create_view-18-437d0a699b73c61044ebf8539feb14f6 => decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508} (100%) rename sql/hive/src/test/resources/golden/{create_view-2-d80dcd1271ab264292e9938f3162427c => decimal_join-1-5098974222b22a21ed847c7906df9313} (100%) delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e delete mode 100644 sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab rename sql/hive/src/test/resources/golden/{create_view-20-56d203e4d0eb776bb4fa38409222b5b8 => delimiter-1-121ff21e6931a82235de8301118cbed8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 delete mode 100644 sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a delete mode 100644 sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 delete mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 delete mode 100644 sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 delete mode 100644 sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 delete mode 100644 sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 rename sql/hive/src/test/resources/golden/{create_view-21-3609711e61b5b8d241d0e839557bfd64 => disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838} (100%) rename sql/hive/src/test/resources/golden/{create_view-22-3bc364c0ee46900d2201d706d2d58d67 => disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23} (100%) rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 => disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a rename sql/hive/src/test/resources/golden/{exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee => disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b rename sql/hive/src/test/resources/golden/{read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b => disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b} (100%) rename sql/hive/src/test/resources/golden/{create_view-3-25ffe475d52d6c399acaf120dc02afe8 => disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11} (100%) delete mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c rename sql/hive/src/test/resources/golden/{create_view-4-87ed262d455e99ad45c909a2265a61b0 => disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782} (100%) create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c rename sql/hive/src/test/resources/golden/{create_view-5-391caf27ff1589ec68d5f3bc4a27e711 => disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7} (100%) rename sql/hive/src/test/resources/golden/{exim_04_all_part-1-baeaf0da490037e7ada642d23013075a => distinct_stats-0-418ec894d08c33fd712eb358f579b7a0} (100%) rename sql/hive/src/test/resources/golden/{exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => distinct_stats-1-10987e425ba8ba8d9c01538f16eab970} (100%) rename sql/hive/src/test/resources/golden/{create_view-6-d8d0e830783c383e3c00e9de3919c409 => distinct_stats-2-a2d8f812612283b20ec3f1e92a263440} (100%) rename sql/hive/src/test/resources/golden/{create_view-7-50b35b8a1772becc96cff65bba1eaee7 => distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b} (100%) rename sql/hive/src/test/resources/golden/{create_view-8-2ae18fc75eda9c3fe7d4e87829180805 => distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31} (100%) rename sql/hive/src/test/resources/golden/{create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 => distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc => distinct_stats-6-297a53801744e6c4786e315e32c6189a} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 => distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 => distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee => distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8} (100%) rename sql/hive/src/test/resources/golden/{create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 => drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-0-311fdd725609cd47ea1b859f706da41e => drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 => drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a => drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-2-3223504c97628a44b65736565c1dda32 => drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b => drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 => drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee => drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 => drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f => drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54} (100%) rename sql/hive/src/test/resources/golden/{ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 => drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-1-baeaf0da490037e7ada642d23013075a => drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2} (100%) rename sql/hive/src/test/resources/golden/{database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 => drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee => drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352} (100%) rename sql/hive/src/test/resources/golden/{database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 => drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a} (100%) create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 rename sql/hive/src/test/resources/golden/{database_drop-7-693736836ccc99f6e2449b94efcfeeba => drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c} (100%) rename sql/hive/src/test/resources/golden/{database_drop-8-97101266791d2b2c662bcde549422318 => drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417} (100%) rename sql/hive/src/test/resources/golden/{database_drop-9-8db536f925bf0f5058f97897e145a661 => drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3} (100%) rename sql/hive/src/test/resources/golden/{date_2-2-cab14d992c53c106ab257fae52001e04 => drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff} (100%) rename sql/hive/src/test/resources/golden/{date_join1-2-894b6541812ac8b0abe2a24c966817d8 => drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-1-baeaf0da490037e7ada642d23013075a => drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299} (100%) rename sql/hive/src/test/resources/golden/{date_serde-7-a34279d8ebbadb78e925e8ed9c78947d => drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee => drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d} (100%) rename sql/hive/src/test/resources/golden/{ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 => dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 => dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd => dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 => dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-2-80fc87cab17ceffea334afbb230a6653 => dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c} (100%) rename sql/hive/src/test/resources/golden/{decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 => dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-0-3c8ecb1468952afe028596c65d587bee => dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-1-868e124edc1581325bd0fd10235a126b => dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146} (100%) delete mode 100644 sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 delete mode 100644 sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf delete mode 100644 sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 delete mode 100644 sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 delete mode 100644 sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 delete mode 100644 sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 delete mode 100644 sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 delete mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d delete mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 delete mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 delete mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 delete mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 delete mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 delete mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d delete mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 rename sql/hive/src/test/resources/golden/{decimal_2-12-d63b5ea25e27852413132db4d9bfb035 => explain_rearrange-0-6f7c8515c354fb050829ebd66413425} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-2-6cc742523b3574e59ca21dad30f2d506 => explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e => explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a => explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768} (100%) rename sql/hive/src/test/resources/golden/{decimal_2-31-3c8ecb1468952afe028596c65d587bee => explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 => explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-0-90cd495a00051a0631b2021dbb9a4aef => explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3} (100%) rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee => explain_rearrange-2-24ca942f094b14b92086305cc125e833} (100%) rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a => explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f} (100%) rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 => explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee => explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a => explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 => explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 => explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12} (100%) rename sql/hive/src/test/resources/golden/{exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee => explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-12-d495d7178707ba55dcc01b9bb3398792 => filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306} (100%) rename sql/hive/src/test/resources/golden/{decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 => filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd} (100%) rename sql/hive/src/test/resources/golden/{decimal_4-2-945542ec888136afaece8d7a5e20d52d => filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e} (100%) rename sql/hive/src/test/resources/golden/{exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a => groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66} (100%) rename sql/hive/src/test/resources/golden/{decimal_4-3-399140971a10a5a0cc6a8c97a4635e => groupby12-1-13ab74a58da514fe01dbeda0c3e79883} (100%) rename sql/hive/src/test/resources/golden/{decimal_4-4-81b37675c09ed874497325ae13233e5c => groupby12-2-fd150794945892f3c926a1881cd819f4} (100%) rename sql/hive/src/test/resources/golden/{decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 => groupby12-3-8018bb917a0706925c14421ec2761663} (100%) create mode 100644 sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f rename sql/hive/src/test/resources/golden/{decimal_join-1-c1524f17ee815171055a67ddc2f9de4e => groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb rename sql/hive/src/test/resources/golden/{decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e => groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 create mode 100644 sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a delete mode 100644 sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 delete mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c delete mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f delete mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 rename sql/hive/src/test/resources/golden/{decimal_precision-1-3f4119830536c92e5ccd76be0259e110 => groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 delete mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 delete mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b delete mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e delete mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d delete mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 rename sql/hive/src/test/resources/golden/{decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 => groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{decimal_precision-15-5c49f041326bc5a9e936910094f190ce => groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c => groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{delimiter-1-d9e405c4107da8af78fcacb83a667b41 => groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb => groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 => groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-10-1afddec8522bd69f496b15980600a6e1 => groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-6-1afddec8522bd69f496b15980600a6e1 => groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc} (100%) rename sql/hive/src/test/resources/golden/{describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 => groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc} (100%) rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 => groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc} (100%) delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 delete mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa => groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf delete mode 100644 sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d delete mode 100644 sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad delete mode 100644 sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f delete mode 100644 sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e delete mode 100644 sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf delete mode 100644 sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca delete mode 100644 sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe delete mode 100644 sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f delete mode 100644 sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b delete mode 100644 sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d delete mode 100644 sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b delete mode 100644 sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 delete mode 100644 sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c delete mode 100644 sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 rename sql/hive/src/test/resources/golden/{exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 => index_serde-0-db64b724719d27c7f0db4f51f5c4edaa} (100%) rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 => index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b rename sql/hive/src/test/resources/golden/{describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 => index_serde-10-c85e061ea9c5b90ca69b7450faad14b6} (100%) create mode 100644 sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b delete mode 100644 sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 rename sql/hive/src/test/resources/golden/{describe_pretty-0-f34ca99310bf1d4793cf64423c024ad => index_serde-12-309e916d683a1a12ab62565697cb0046} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 rename sql/hive/src/test/resources/golden/{describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 => index_serde-13-d590fd7cb9d433143de490d75686dd4} (100%) create mode 100644 sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c delete mode 100644 sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 rename sql/hive/src/test/resources/golden/{describe_pretty-9-e382a994134aefcd2652b57af9195644 => index_serde-3-ebab588c84a7a29f03b41dcd98132229} (100%) rename sql/hive/src/test/resources/golden/{describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a => index_serde-4-afcf2a156ccd4f79a0489b4593908d79} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 create mode 100644 sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 delete mode 100644 sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae rename sql/hive/src/test/resources/golden/{describe_syntax-1-4f3d3497418242124113538edab45df7 => index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 rename sql/hive/src/test/resources/golden/{exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee => index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 rename sql/hive/src/test/resources/golden/{exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a => index_serde-8-3b03210f94ec40db9ab02620645014d1} (100%) rename sql/hive/src/test/resources/golden/{exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 => index_serde-9-35f48c7d6fa164bb84643657bc9280a8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 delete mode 100644 sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e delete mode 100644 sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 delete mode 100644 sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 delete mode 100644 sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 delete mode 100644 sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 delete mode 100644 sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 delete mode 100644 sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 delete mode 100644 sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc rename sql/hive/src/test/resources/golden/{describe_syntax-2-b198700c0129910d6205ef063ee83d5a => input19-1-f32df514de8156b5f5b435eea2c9be40} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 rename sql/hive/src/test/resources/golden/{describe_syntax-3-458d6aaffeee94997f67a43b88382106 => input21-1-c45ad493e95150b580be778da6065f36} (100%) rename sql/hive/src/test/resources/golden/{describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 => input22-1-b663ec84da3f9d9b9594ea2da81b1442} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 delete mode 100644 sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c delete mode 100644 sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab delete mode 100644 sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee delete mode 100644 sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 delete mode 100644 sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 delete mode 100644 sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 delete mode 100644 sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 delete mode 100644 sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f delete mode 100644 sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe delete mode 100644 sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f delete mode 100644 sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 delete mode 100644 sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d delete mode 100644 sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 delete mode 100644 sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e delete mode 100644 sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 delete mode 100644 sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 delete mode 100644 sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 delete mode 100644 sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 delete mode 100644 sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 delete mode 100644 sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 delete mode 100644 sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 delete mode 100644 sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 delete mode 100644 sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d delete mode 100644 sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 delete mode 100644 sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e delete mode 100644 sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 delete mode 100644 sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee delete mode 100644 sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 delete mode 100644 sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e delete mode 100644 sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 delete mode 100644 sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 delete mode 100644 sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 rename sql/hive/src/test/resources/golden/{describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 => input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 rename sql/hive/src/test/resources/golden/{describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 => input4-2-b663ec84da3f9d9b9594ea2da81b1442} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 rename sql/hive/src/test/resources/golden/{describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb => input40-1-a8adb8ae1d13607851431a1baf7578ba} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 delete mode 100644 sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e rename sql/hive/src/test/resources/golden/{describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 => input40-4-fdeea6b676c670b17c8d91e24a97a127} (100%) rename sql/hive/src/test/resources/golden/{disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 => input40-6-6651f53efc5d03ed2d43b9d7aecc0002} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 delete mode 100644 sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a delete mode 100644 sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 delete mode 100644 sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f delete mode 100644 sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca delete mode 100644 sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 delete mode 100644 sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 delete mode 100644 sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab rename sql/hive/src/test/resources/golden/{disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb => input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537} (100%) delete mode 100644 sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 delete mode 100644 sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d delete mode 100644 sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af delete mode 100644 sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea delete mode 100644 sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 delete mode 100644 sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad delete mode 100644 sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 delete mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c delete mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 delete mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 rename sql/hive/src/test/resources/golden/{disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 => inputddl6-1-572c1abb70f09726d1ba77bdc884597b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf rename sql/hive/src/test/resources/golden/{disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 => inputddl6-2-3a4def4b370f75c5fcc1174626490363} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c rename sql/hive/src/test/resources/golden/{disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 => inputddl7-1-7195712efb4910294f63303ebce24453} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d rename sql/hive/src/test/resources/golden/{drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c => inputddl7-10-7c9248b56948716913d332bd712d69bd} (100%) rename sql/hive/src/test/resources/golden/{drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 => inputddl7-4-68715ba2c11220be62394c86453e6d54} (100%) delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 delete mode 100644 sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 => inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71} (100%) rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 => insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 delete mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 => insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af} (100%) delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d delete mode 100644 sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 delete mode 100644 sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 delete mode 100644 sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 delete mode 100644 sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 delete mode 100644 sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 delete mode 100644 sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b delete mode 100644 sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 delete mode 100644 sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 => join_1to1-1-789b2636cfb6a08965e0bd190e419762} (100%) rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 => join_1to1-3-6228e662e573a00ed04550d049d97a3b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 => join_array-2-16840a0266cad03a1a0b134d105b854f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 => join_array-3-a6ca6b64324596831033fdfe5b63a942} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 delete mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 rename sql/hive/src/test/resources/golden/{dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 => join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc rename sql/hive/src/test/resources/golden/{enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 => join_casesensitive-3-93300f4a9242fa2804a5b368538d83f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 rename sql/hive/src/test/resources/golden/{enforce_order-1-633034e3d966737cecf2befc5df1e35d => join_filters-1-bee6095f42de6a16708c2f9addc1b9bd} (100%) rename sql/hive/src/test/resources/golden/{enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f => join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b delete mode 100644 sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 rename sql/hive/src/test/resources/golden/{enforce_order-4-3136edd49e681ea21aa35d0836eab65 => join_filters-47-6dc6866a65c74d69538b776b41b06c16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c rename sql/hive/src/test/resources/golden/{enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 => join_filters-48-e884480a0f7273d3e2f2de2ba46b855c} (100%) rename sql/hive/src/test/resources/golden/{enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 => join_filters-49-98fd86aea9cacaa82d43c7468109dd33} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 delete mode 100644 sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce delete mode 100644 sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 delete mode 100644 sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 rename sql/hive/src/test/resources/golden/{escape1-2-395d5a528c5e7235a48b4ac90938e2d6 => join_hive_626-3-4a2f2f2858540afea9a195b5322941ee} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 rename sql/hive/src/test/resources/golden/{escape1-3-4267651148da591da38737028fdbd80 => join_hive_626-4-4bb73b33747da4ed852df381b7b45a71} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 rename sql/hive/src/test/resources/golden/{escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 => join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe} (100%) rename sql/hive/src/test/resources/golden/{escape1-5-70729c3d79ded87e884c176138174645 => join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 rename sql/hive/src/test/resources/golden/{escape1-8-910536a438eec89c78bd611b3c4bb7e0 => join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 rename sql/hive/src/test/resources/golden/{escape2-10-13884d58efe80bd24862b3c54cb57c6e => join_nulls-41-6dc6866a65c74d69538b776b41b06c16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c rename sql/hive/src/test/resources/golden/{escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe => join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c} (100%) rename sql/hive/src/test/resources/golden/{escape2-5-fd0c88ad6ad131a16d1b78adbea65800 => join_nulls-43-98fd86aea9cacaa82d43c7468109dd33} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 rename sql/hive/src/test/resources/golden/{escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 => join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 rename sql/hive/src/test/resources/golden/{escape2-7-70729c3d79ded87e884c176138174645 => join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 rename sql/hive/src/test/resources/golden/{exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 => join_nullsafe-21-24332b9390108fb3379e1acc599293a1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 rename sql/hive/src/test/resources/golden/{exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 => join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 rename sql/hive/src/test/resources/golden/{exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd => join_nullsafe-23-e4425d56be43c21124d95160653ce0ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 => join_reorder2-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 => join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{exim_02_part-4-88b581725ecdd603117a1706ab9c34dc => join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 => join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd => join_reorder3-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 => join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc => join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec rename sql/hive/src/test/resources/golden/{exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 => join_reorder4-3-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 => join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 => join_star-10-a9e579038e3d4826fdae475d7058ab82} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 => join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae delete mode 100644 sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 => join_star-12-f581d6d305d652cd0f4e4fa912eb578d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 => join_star-13-7268564732cbb7489248f9d818f80c14} (100%) rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 => join_star-14-2ee0fcf000f8687fc8941bf212477e57} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 delete mode 100644 sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 rename sql/hive/src/test/resources/golden/{exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 => join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 rename sql/hive/src/test/resources/golden/{exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 => join_star-8-c3d53a4daab9614a09870dc8e9571f74} (100%) rename sql/hive/src/test/resources/golden/{exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc => join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628} (100%) delete mode 100644 sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 delete mode 100644 sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 delete mode 100644 sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db delete mode 100644 sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b delete mode 100644 sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 delete mode 100644 sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 delete mode 100644 sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 delete mode 100644 sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 delete mode 100644 sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b delete mode 100644 sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 delete mode 100644 sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef delete mode 100644 sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d delete mode 100644 sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 delete mode 100644 sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 delete mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b rename sql/hive/src/test/resources/golden/{exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b rename sql/hive/src/test/resources/golden/{exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 => leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c rename sql/hive/src/test/resources/golden/{exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => leftsemijoin-6-9f50dce576b019c0be997055b8876621} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 rename sql/hive/src/test/resources/golden/{exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 => leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc => leftsemijoin_mr-1-5b2e555868faa404ea09928936178181} (100%) delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593 delete mode 100644 sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 rename sql/hive/src/test/resources/golden/{exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354} (100%) delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a delete mode 100644 sql/hive/src/test/resources/golden/limit_pushdown_negative-7-fb7bf3783d4fb43673a202c4111d9092 delete mode 100644 sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f rename sql/hive/src/test/resources/golden/{exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee => literal_double-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 => literal_double-1-10ef1098e35d900983be3814de8f974f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 create mode 100644 sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 rename sql/hive/src/test/resources/golden/{exim_12_external_location-1-baeaf0da490037e7ada642d23013075a => literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 rename sql/hive/src/test/resources/golden/{exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 create mode 100644 sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 rename sql/hive/src/test/resources/golden/{exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => literal_string-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 delete mode 100644 sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 => literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15} (100%) create mode 100644 sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 delete mode 100644 sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-10-c66fea7ad025cd1f513a98a4cc1036b1 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-11-8b6be78ae0527e5b7efd7db758966853 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-9-a646fd106fe73b8753fe3fee86d56ebf delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-8-c32770da4784bfea4d0dd77fdcba4f0a delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-10-297cf42ec45b5aa78b80d9c35a79555a delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-11-24618a43c4656b72f81683b45708045e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-9-704bd110d9aaa2ac678b7fbf645abdb9 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-9-6954f5fc8dd82ca2c076ab8bcdbc148 delete mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb delete mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 delete mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 delete mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 delete mode 100644 sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc => load_file_with_space_in_the_name-1-d19201e2fcaee4d451292bd740e6c637} (100%) rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a => load_file_with_space_in_the_name-2-ad8795e50f5998ea1d2eb64a0c02e6e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 delete mode 100644 sql/hive/src/test/resources/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 delete mode 100644 sql/hive/src/test/resources/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 delete mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc delete mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 delete mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 delete mode 100644 sql/hive/src/test/resources/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 delete mode 100644 sql/hive/src/test/resources/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 delete mode 100644 sql/hive/src/test/resources/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 delete mode 100644 sql/hive/src/test/resources/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 delete mode 100644 sql/hive/src/test/resources/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 => loadpart1-1-6cc94d19c536a996592629f7c82c2ac9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e delete mode 100644 sql/hive/src/test/resources/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 delete mode 100644 sql/hive/src/test/resources/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 delete mode 100644 sql/hive/src/test/resources/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 delete mode 100644 sql/hive/src/test/resources/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 delete mode 100644 sql/hive/src/test/resources/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 delete mode 100644 sql/hive/src/test/resources/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 delete mode 100644 sql/hive/src/test/resources/golden/lock2-3-27ad2962fed131f51ba802596ba37278 delete mode 100644 sql/hive/src/test/resources/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a delete mode 100644 sql/hive/src/test/resources/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca delete mode 100644 sql/hive/src/test/resources/golden/lock3-0-27ad2962fed131f51ba802596ba37278 delete mode 100644 sql/hive/src/test/resources/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a delete mode 100644 sql/hive/src/test/resources/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 delete mode 100644 sql/hive/src/test/resources/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac delete mode 100644 sql/hive/src/test/resources/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe delete mode 100644 sql/hive/src/test/resources/golden/lock4-1-27ad2962fed131f51ba802596ba37278 delete mode 100644 sql/hive/src/test/resources/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a delete mode 100644 sql/hive/src/test/resources/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 delete mode 100644 sql/hive/src/test/resources/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac delete mode 100644 sql/hive/src/test/resources/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 rename sql/hive/src/test/resources/golden/{exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 => mapjoin_mapjoin-4-5166a5b9d30dfacbe33dd909c0df6310} (100%) delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-5ee898cab883074f3297198c52445ee4 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 => mapjoin_mapjoin-6-dca8c08a235b45d1cdcb94e363afb17} (100%) create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-7-fddbdea343a9ddb5f8dedc18147640b7 rename sql/hive/src/test/resources/golden/{exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee => mapjoin_mapjoin-8-2be637ed4f6146e8525ae1a863e72736} (100%) create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-9-c47698bac140454637a999e583941ce7 delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 rename sql/hive/src/test/resources/golden/{exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd => mapjoin_subquery2-6-9bf06af695892b0d7067d5b30e0b2425} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 => mapjoin_subquery2-7-c6b0cdb137f13f8362c0c49c544151a4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc => mapjoin_subquery2-8-131ae5ecfff2733b04bdfada0108cf40} (100%) delete mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e delete mode 100644 sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 delete mode 100644 sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 delete mode 100644 sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 delete mode 100644 sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 delete mode 100644 sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 delete mode 100644 sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a delete mode 100644 sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d delete mode 100644 sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af delete mode 100644 sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d delete mode 100644 sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 delete mode 100644 sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c delete mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 delete mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf delete mode 100644 sql/hive/src/test/resources/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 delete mode 100644 sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e delete mode 100644 sql/hive/src/test/resources/golden/mi-3-b66a495f7bdf106a7886b72267b8659d delete mode 100644 sql/hive/src/test/resources/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c delete mode 100644 sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc delete mode 100644 sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 delete mode 100644 sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 delete mode 100644 sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 rename sql/hive/src/test/resources/golden/{exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a => multiMapJoin2-28-c14792ccac2ca64e3e9e21af4fd12d2c} (100%) rename sql/hive/src/test/resources/golden/{exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => multiMapJoin2-29-b9d963d24994c47c3776dda6f7d3881f} (100%) rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a => multiMapJoin2-30-6d1c7f7014fc709148b0f401c5f23cb3} (100%) create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-31-2e9c4d5e2bd709e96f311488ada116b0 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 delete mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 delete mode 100644 sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 delete mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f delete mode 100644 sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad delete mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 delete mode 100644 sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 delete mode 100644 sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 => nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 rename sql/hive/src/test/resources/golden/{exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 => nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 rename sql/hive/src/test/resources/golden/{exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb rename sql/hive/src/test/resources/golden/{exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup3-18-113e2b587784d54d2a5b5f091389397e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 rename sql/hive/src/test/resources/golden/{exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup3-19-77de4b2d65eee228848625cdd422178d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 rename sql/hive/src/test/resources/golden/{exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb rename sql/hive/src/test/resources/golden/{exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup3-6-113e2b587784d54d2a5b5f091389397e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 rename sql/hive/src/test/resources/golden/{exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup3-7-77de4b2d65eee228848625cdd422178d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 => nullgroup5-1-642e12a05bf01a6716bfa15ed0012629} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd => nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c delete mode 100644 sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 rename sql/hive/src/test/resources/golden/{exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 => nullscript-1-f07dfd6ef687e038083deca5941d8174} (100%) delete mode 100644 sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 rename sql/hive/src/test/resources/golden/{exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc => nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb} (100%) delete mode 100644 sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc delete mode 100644 sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d delete mode 100644 sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d delete mode 100644 sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead delete mode 100644 sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c delete mode 100644 sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae delete mode 100644 sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d delete mode 100644 sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a delete mode 100644 sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae delete mode 100644 sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 delete mode 100644 sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 delete mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 rename sql/hive/src/test/resources/golden/{exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a => orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e delete mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 delete mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc delete mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 delete mode 100644 sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 delete mode 100644 sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 delete mode 100644 sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b delete mode 100644 sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 delete mode 100644 sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 delete mode 100644 sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 delete mode 100644 sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 rename sql/hive/src/test/resources/golden/{exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 => partcols1-1-b562ff3e36de23a686d122967a1f91c8} (100%) rename sql/hive/src/test/resources/golden/{exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 => partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f rename sql/hive/src/test/resources/golden/{orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 => partition_date-11-a01c3791f59e819c750e213b6c65d084} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 create mode 100644 sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee => partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c} (100%) rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a => partition_date-15-e4366325f3a0c4a8e92be59f4de73fce} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac create mode 100644 sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 create mode 100644 sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 create mode 100644 sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d rename sql/hive/src/test/resources/golden/{exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 => partition_date-2-220048240a7050a98411ddbc6b1f82cf} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba create mode 100644 sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 rename sql/hive/src/test/resources/golden/{type_cast_1-1-53a667981ad567b2ab977f67d65c5825 => partition_date-22-fe277bd0a30e016826d242fd5bd61714} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 rename sql/hive/src/test/resources/golden/{exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc => partition_date-24-7ec1f3a845e2c49191460e15af30aa30} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a => partition_date-3-f8887dd18c21bf0306e293d463b3e1d7} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 => partition_date-4-f7c20c208e052305c1710a5e1e80c5c8} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 => partition_date-5-59355f4e222dcd4f77f51d15bd896f11} (100%) rename sql/hive/src/test/resources/golden/{exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 => partition_date-6-416ab10ac818d432cd3a81870d6e5164} (100%) create mode 100644 sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc create mode 100644 sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 create mode 100644 sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 delete mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 delete mode 100644 sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc => partition_type_check-11-60aad884be613d18d3b89fca3b90dc94} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a rename sql/hive/src/test/resources/golden/{exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a => partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f delete mode 100644 sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 => partition_type_check-6-663051c7106b8ee6913ca1b007157941} (100%) rename sql/hive/src/test/resources/golden/{exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 => partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 rename sql/hive/src/test/resources/golden/{exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 => partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef rename sql/hive/src/test/resources/golden/{exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc => partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680} (100%) rename sql/hive/src/test/resources/golden/{exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a => partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73} (100%) delete mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 delete mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b delete mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 delete mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 delete mode 100644 sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 rename sql/hive/src/test/resources/golden/{exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 => ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b rename sql/hive/src/test/resources/golden/{exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 => ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 rename sql/hive/src/test/resources/golden/{exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 => ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c} (100%) rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 => ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown-14-53b4be82a1538844d03b200429efa02b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a => ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63} (100%) rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 => ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 => ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 => ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 => ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a => ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 => ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d => ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d => ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 => ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 => ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 => ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-27-b72de558c88ae91460989938000e0d27 => ppr_pushdown-28-b72de558c88ae91460989938000e0d27} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 => ppr_pushdown-29-7217735d357770af4ffee730e4e9add4} (100%) rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown-3-5eede4874e678021938683fc2f5dc900} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 rename sql/hive/src/test/resources/golden/{ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea => ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 => ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-31-5eba4bf10315099129eae319d73636cf => ppr_pushdown-32-5eba4bf10315099129eae319d73636cf} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b => ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b} (100%) rename sql/hive/src/test/resources/golden/{ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 => ppr_pushdown-34-a5594625510703427ab8bae5d0563f73} (100%) rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a => ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 => ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 => ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 => ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d} (100%) rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd => ppr_pushdown-8-14570f946e75924d7926c809485951d1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 => ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 rename sql/hive/src/test/resources/golden/{exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee => ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 => ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b} (100%) create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd => ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 => ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a} (100%) rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc => ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 => ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd => ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b} (100%) create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c rename sql/hive/src/test/resources/golden/{filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 => ppr_pushdown2-3-c7753746c190414723d66a8f876499c7} (100%) create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 rename sql/hive/src/test/resources/golden/{filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba => ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 rename sql/hive/src/test/resources/golden/{filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 => ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b} (100%) rename sql/hive/src/test/resources/golden/{groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 => ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7} (100%) delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 delete mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 delete mode 100644 sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 delete mode 100644 sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a delete mode 100644 sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 delete mode 100644 sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f rename sql/hive/src/test/resources/golden/{groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 => progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c} (100%) rename sql/hive/src/test/resources/golden/{exim_15_external_part-1-baeaf0da490037e7ada642d23013075a => quote2-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 delete mode 100644 sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 => quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118} (100%) create mode 100644 sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 delete mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 delete mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 delete mode 100644 sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b delete mode 100644 sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db delete mode 100644 sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 delete mode 100644 sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 delete mode 100644 sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 delete mode 100644 sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa delete mode 100644 sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 delete mode 100644 sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d delete mode 100644 sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 delete mode 100644 sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 delete mode 100644 sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a delete mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 delete mode 100644 sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 delete mode 100644 sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef delete mode 100644 sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 delete mode 100644 sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f delete mode 100644 sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 delete mode 100644 sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 create mode 100644 sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 => serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611} (100%) rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 => serde_regex-3-3ee9e78ff563d6b48741a41885f92c81} (100%) delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 => serde_regex-7-bf456bcf6be7334488424dfeadf27d75} (100%) rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 => serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54} (100%) delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 delete mode 100644 sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 => serde_regex-9-f1175f3322abec6f258dd49a5905bce0} (100%) delete mode 100644 sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e delete mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 rename sql/hive/src/test/resources/golden/{stats4-2-463330cf55370dbe92d6ed74ef91302 => show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 delete mode 100644 sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca create mode 100644 sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 delete mode 100644 sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 rename sql/hive/src/test/resources/golden/{stats4-11-ea921e0af59a4940a11c94143b1c4b32 => show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca} (100%) create mode 100644 sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce delete mode 100644 sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe delete mode 100644 sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab delete mode 100644 sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d delete mode 100644 sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c delete mode 100644 sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 delete mode 100644 sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 delete mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 delete mode 100644 sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 delete mode 100644 sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 delete mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a => skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 => skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 rename sql/hive/src/test/resources/golden/{groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 => skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 rename sql/hive/src/test/resources/golden/{groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a => skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 => skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 => skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 => skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488} (100%) rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 => smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 => smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 => smb_mapjoin_1-5-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 => smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 => smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b => smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 => smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab => smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 => smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 => smb_mapjoin_2-5-6d835f651b099615df163be284e833de} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc => smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 => smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f => smb_mapjoin_25-12-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 => smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 => smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea => smb_mapjoin_3-5-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b => smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 => smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_position-1-6b06902de5c0ca13cebe03018d86f447 => smb_mapjoin_4-5-6d835f651b099615df163be284e833de} (100%) rename sql/hive/src/test/resources/golden/{groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 => smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 rename sql/hive/src/test/resources/golden/{groupby_position-13-9ee2150594ad2eece6ee14424155d396 => smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 rename sql/hive/src/test/resources/golden/{groupby_position-2-627bb7be9c0edb5ba4c677912800d364 => smb_mapjoin_5-5-6d835f651b099615df163be284e833de} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb rename sql/hive/src/test/resources/golden/{groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac => smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f rename sql/hive/src/test/resources/golden/{groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c => smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 rename sql/hive/src/test/resources/golden/{groupby_position-7-5b32a45af11e04b46f8566bd27a28014 => smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93} (100%) delete mode 100644 sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e delete mode 100644 sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b delete mode 100644 sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 delete mode 100644 sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 delete mode 100644 sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 delete mode 100644 sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 delete mode 100644 sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e delete mode 100644 sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 delete mode 100644 sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b delete mode 100644 sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 delete mode 100644 sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a delete mode 100644 sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 delete mode 100644 sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 delete mode 100644 sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa delete mode 100644 sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 delete mode 100644 sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa delete mode 100644 sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc delete mode 100644 sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 delete mode 100644 sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa delete mode 100644 sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 delete mode 100644 sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 delete mode 100644 sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 delete mode 100644 sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b delete mode 100644 sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 delete mode 100644 sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 delete mode 100644 sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 delete mode 100644 sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 delete mode 100644 sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 delete mode 100644 sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 delete mode 100644 sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c delete mode 100644 sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d delete mode 100644 sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 delete mode 100644 sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 delete mode 100644 sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 delete mode 100644 sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 delete mode 100644 sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 delete mode 100644 sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c delete mode 100644 sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 delete mode 100644 sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 delete mode 100644 sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 delete mode 100644 sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 delete mode 100644 sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 delete mode 100644 sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd delete mode 100644 sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c delete mode 100644 sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd delete mode 100644 sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c delete mode 100644 sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d delete mode 100644 sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 delete mode 100644 sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f delete mode 100644 sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 delete mode 100644 sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df delete mode 100644 sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 delete mode 100644 sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 delete mode 100644 sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 delete mode 100644 sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce delete mode 100644 sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 delete mode 100644 sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 delete mode 100644 sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 delete mode 100644 sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 delete mode 100644 sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 delete mode 100644 sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 delete mode 100644 sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d delete mode 100644 sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 delete mode 100644 sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 delete mode 100644 sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 delete mode 100644 sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 delete mode 100644 sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 delete mode 100644 sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 delete mode 100644 sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce delete mode 100644 sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 delete mode 100644 sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 rename sql/hive/src/test/resources/golden/{groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c => stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa} (100%) rename sql/hive/src/test/resources/golden/{exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 => stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 delete mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae delete mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca delete mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 rename sql/hive/src/test/resources/golden/{exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee => stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 delete mode 100644 sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 delete mode 100644 sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 delete mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 delete mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c delete mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 rename sql/hive/src/test/resources/golden/{exim_16_part_external-1-baeaf0da490037e7ada642d23013075a => timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 => timestamp_1-1-d362501d0176855077e65f8faf067fa8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-18-67f274bf16de625cf4e85af0c6185cac => timestamp_1-10-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-10-343c75daac6695917608c17db8bf473e => timestamp_1-11-343c75daac6695917608c17db8bf473e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 rename sql/hive/src/test/resources/golden/{timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d => timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 => timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-17-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-27-67f274bf16de625cf4e85af0c6185cac => timestamp_1-19-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d => timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 rename sql/hive/src/test/resources/golden/{timestamp_1-19-343c75daac6695917608c17db8bf473e => timestamp_1-20-343c75daac6695917608c17db8bf473e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e rename sql/hive/src/test/resources/golden/{groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 => timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 => timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-26-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-36-67f274bf16de625cf4e85af0c6185cac => timestamp_1-28-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-28-343c75daac6695917608c17db8bf473e => timestamp_1-29-343c75daac6695917608c17db8bf473e} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c => timestamp_1-3-74f477a8b726f5193dd42ef378a793c4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 rename sql/hive/src/test/resources/golden/{timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 => timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 => timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-35-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-45-67f274bf16de625cf4e85af0c6185cac => timestamp_1-37-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-37-343c75daac6695917608c17db8bf473e => timestamp_1-38-343c75daac6695917608c17db8bf473e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d rename sql/hive/src/test/resources/golden/{timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea => timestamp_1-4-d833b177fac3162215468dde991f71d1} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 => timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-13-25f6ec69328af6cba76899194e0dd84e => timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-44-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-54-67f274bf16de625cf4e85af0c6185cac => timestamp_1-46-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-46-343c75daac6695917608c17db8bf473e => timestamp_1-47-343c75daac6695917608c17db8bf473e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 rename sql/hive/src/test/resources/golden/{timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 => timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-22-25f6ec69328af6cba76899194e0dd84e => timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-31-25f6ec69328af6cba76899194e0dd84e => timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-53-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-9-67f274bf16de625cf4e85af0c6185cac => timestamp_1-55-67f274bf16de625cf4e85af0c6185cac} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-55-343c75daac6695917608c17db8bf473e => timestamp_1-56-343c75daac6695917608c17db8bf473e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a => timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 rename sql/hive/src/test/resources/golden/{groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 => timestamp_1-58-d362501d0176855077e65f8faf067fa8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 => timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 => timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 => timestamp_1-8-e7b398d2a8107a42419c83771bda41e6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf => timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf} (100%) rename sql/hive/src/test/resources/golden/{exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 => timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 rename sql/hive/src/test/resources/golden/{groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 => timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-18-252aebfe7882335d31bfc53a8705b7a => timestamp_2-10-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-11-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-11-240fce5f58794fa051824e8732c00c03 => timestamp_2-12-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd rename sql/hive/src/test/resources/golden/{groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 => timestamp_2-13-5f450162886ccc79af149541527f5643} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-4-25f6ec69328af6cba76899194e0dd84e => timestamp_2-14-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-27-252aebfe7882335d31bfc53a8705b7a => timestamp_2-19-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 => timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 rename sql/hive/src/test/resources/golden/{timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-20-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-20-240fce5f58794fa051824e8732c00c03 => timestamp_2-21-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b rename sql/hive/src/test/resources/golden/{groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 => timestamp_2-22-469fe94fb60f4b00809190c303434641} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-40-25f6ec69328af6cba76899194e0dd84e => timestamp_2-23-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-36-252aebfe7882335d31bfc53a8705b7a => timestamp_2-28-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-29-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 => timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 rename sql/hive/src/test/resources/golden/{timestamp_2-29-240fce5f58794fa051824e8732c00c03 => timestamp_2-30-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 rename sql/hive/src/test/resources/golden/{groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f => timestamp_2-31-8f506498acf0c99c30960a00981ef460} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-49-25f6ec69328af6cba76899194e0dd84e => timestamp_2-32-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-45-252aebfe7882335d31bfc53a8705b7a => timestamp_2-37-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-38-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-38-240fce5f58794fa051824e8732c00c03 => timestamp_2-39-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 rename sql/hive/src/test/resources/golden/{groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 => timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 => timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 => timestamp_2-41-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-54-252aebfe7882335d31bfc53a8705b7a => timestamp_2-46-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-47-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-47-240fce5f58794fa051824e8732c00c03 => timestamp_2-48-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb rename sql/hive/src/test/resources/golden/{groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f => timestamp_2-49-650d2727b007638e0ed39b37c9498d66} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 => timestamp_2-5-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 => timestamp_2-50-25f6ec69328af6cba76899194e0dd84e} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-9-252aebfe7882335d31bfc53a8705b7a => timestamp_2-55-252aebfe7882335d31bfc53a8705b7a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 => timestamp_2-56-5181279a0bf8939fe46ddacae015dad8} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-56-240fce5f58794fa051824e8732c00c03 => timestamp_2-57-240fce5f58794fa051824e8732c00c03} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 rename sql/hive/src/test/resources/golden/{groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef => timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 => timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 => timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 => timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571} (100%) rename sql/hive/src/test/resources/golden/{timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef => timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee => timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 => timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b rename sql/hive/src/test/resources/golden/{timestamp_3-9-ffc79abb874323e165963aa39f460a9b => timestamp_3-10-ffc79abb874323e165963aa39f460a9b} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 => timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 rename sql/hive/src/test/resources/golden/{timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a => timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a} (100%) create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 rename sql/hive/src/test/resources/golden/{groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 => timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 rename sql/hive/src/test/resources/golden/{groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d => timestamp_3-2-81edf5107270547641586aa02b4e7d9b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a rename sql/hive/src/test/resources/golden/{groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b => timestamp_3-3-7a012a0d98729da25b5ac374855dcee4} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad => timestamp_3-4-86514381187b246a5685577c1968c559} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 => timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 => timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 => timestamp_3-7-6be5fe01c502cd24db32a3781c97a703} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add => timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add} (100%) rename sql/hive/src/test/resources/golden/{timestamp_3-8-22e03daa775eab145d39ec0730953f7e => timestamp_3-9-22e03daa775eab145d39ec0730953f7e} (100%) rename sql/hive/src/test/resources/golden/{exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a => timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 => timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d => timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc => timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-14-773801b833cf72d35016916b786275b5 => timestamp_comparison-4-13e17ed811165196416f777cbc162592} (100%) rename sql/hive/src/test/resources/golden/{timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 => timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 => timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 => timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 => timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 => timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 => timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 rename sql/hive/src/test/resources/golden/{groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c => timestamp_null-2-51762cf5079877abf7d81127738f4e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 rename sql/hive/src/test/resources/golden/{exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 => timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d rename sql/hive/src/test/resources/golden/{groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a => timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a => timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09 => timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35 => timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6 => timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a => timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-14-c745a1016461403526d44928a269c1de => timestamp_udf-15-c745a1016461403526d44928a269c1de} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe => timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-16-b36e87e17ca24d82072220bff559c718 => timestamp_udf-17-b36e87e17ca24d82072220bff559c718} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-17-dad44d2d4a421286e9da080271bd2639 => timestamp_udf-18-dad44d2d4a421286e9da080271bd2639} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af rename sql/hive/src/test/resources/golden/{timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938 => timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 => timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d rename sql/hive/src/test/resources/golden/{groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 => timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa => timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 => timestamp_udf-3-9039f474f9a96e9f15ace528faeed923} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f delete mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc rename sql/hive/src/test/resources/golden/{groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b => timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 => timestamp_udf-5-66868a2b075de978784011e9955483d} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-5-1124399033bcadf3874fb48f593392d => timestamp_udf-6-1124399033bcadf3874fb48f593392d} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4 => timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9 => timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9} (100%) rename sql/hive/src/test/resources/golden/{timestamp_udf-8-975df43df015d86422965af456f87a94 => timestamp_udf-9-975df43df015d86422965af456f87a94} (100%) delete mode 100644 sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 delete mode 100644 sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 delete mode 100644 sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 delete mode 100644 sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 delete mode 100644 sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 delete mode 100644 sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 delete mode 100644 sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f rename sql/hive/src/test/resources/golden/{exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee => type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d rename sql/hive/src/test/resources/golden/{groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 => type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d} (100%) create mode 100644 sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 rename sql/hive/src/test/resources/golden/{exim_18_part_external-1-baeaf0da490037e7ada642d23013075a => type_widening-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 rename sql/hive/src/test/resources/golden/{groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 => type_widening-1-630ac2c7e7dea4837384ccd572209229} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 delete mode 100644 sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 rename sql/hive/src/test/resources/golden/{exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 => type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 rename sql/hive/src/test/resources/golden/{groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c => type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2} (100%) create mode 100644 sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee => udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a => udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db rename sql/hive/src/test/resources/golden/{exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52} (100%) rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee => udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257} (100%) create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a => udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 rename sql/hive/src/test/resources/golden/{exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee => udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a => udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 rename sql/hive/src/test/resources/golden/{groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 => udaf_corr-2-e886f45c8f085596ffd420f89cdc2909} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 rename sql/hive/src/test/resources/golden/{groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 => udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 rename sql/hive/src/test/resources/golden/{groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 => udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 delete mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f delete mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 delete mode 100644 sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 delete mode 100644 sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da delete mode 100644 sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f delete mode 100644 sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c delete mode 100644 sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 delete mode 100644 sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e delete mode 100644 sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f delete mode 100644 sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f rename sql/hive/src/test/resources/golden/{exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_E-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b => udf_E-1-cad0779d18f326c8e453bf2b5fe43596} (100%) rename sql/hive/src/test/resources/golden/{udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c => udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02} (100%) rename sql/hive/src/test/resources/golden/{udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a => udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f rename sql/hive/src/test/resources/golden/{udf_E-3-e8924af3bf99d2e01546a965303ffd09 => udf_E-4-e8924af3bf99d2e01546a965303ffd09} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 => udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4} (100%) rename sql/hive/src/test/resources/golden/{udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c => udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02} (100%) rename sql/hive/src/test/resources/golden/{udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a => udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a} (100%) rename sql/hive/src/test/resources/golden/{udf_E-7-e8924af3bf99d2e01546a965303ffd09 => udf_E-8-e8924af3bf99d2e01546a965303ffd09} (100%) rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f rename sql/hive/src/test/resources/golden/{groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 => udf_PI-1-13fd9345fd15b654d18b2707e5274b2b} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 => udf_PI-2-97a12f6967726e425469ecfa70177ff0} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-2-9c1476a2eab7455594e97b338ee3c188 => udf_PI-3-9c1476a2eab7455594e97b338ee3c188} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 => udf_PI-4-890f3c276eff2c459d8dc79d5a71c866} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f rename sql/hive/src/test/resources/golden/{groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 => udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 => udf_PI-6-97a12f6967726e425469ecfa70177ff0} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-6-9c1476a2eab7455594e97b338ee3c188 => udf_PI-7-9c1476a2eab7455594e97b338ee3c188} (100%) rename sql/hive/src/test/resources/golden/{udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 => udf_PI-8-890f3c276eff2c459d8dc79d5a71c866} (100%) rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 rename sql/hive/src/test/resources/golden/{groupby_sort_1-67-b4fec0996399be2239961594897d6715 => udf_abs-3-50cb3c23902cd29e0dbff188c71062e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c create mode 100644 sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 delete mode 100644 sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 rename sql/hive/src/test/resources/golden/{groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 => udf_abs-5-5cd4c198e0de884ad436864b95fece6c} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 rename sql/hive/src/test/resources/golden/{exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e => udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b create mode 100644 sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f rename sql/hive/src/test/resources/golden/{decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 => udf_acos-3-661a0a85283df2a5c1567d60850e362b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 create mode 100644 sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a create mode 100644 sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e delete mode 100644 sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e create mode 100644 sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_array-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee => udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee} (100%) rename sql/hive/src/test/resources/golden/{udf_array-1-570741914bb78300b0233e5f38d7f08a => udf_array-2-570741914bb78300b0233e5f38d7f08a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 rename sql/hive/src/test/resources/golden/{groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc => udf_array-3-47818d42e5e7667d8754c3f9a4b8053a} (100%) rename sql/hive/src/test/resources/golden/{udf_array-3-a5d12c41277fb158e09281169c905122 => udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47} (100%) rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c rename sql/hive/src/test/resources/golden/{udf_to_boolean-3-266b9601a9154438ab95550fcd36494c => udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 rename sql/hive/src/test/resources/golden/{udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d => udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8} (100%) rename sql/hive/src/test/resources/golden/{exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 delete mode 100644 sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 rename sql/hive/src/test/resources/golden/{groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e => udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd create mode 100644 sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f rename sql/hive/src/test/resources/golden/{decimal_precision-12-18906f5c6413065621430e3fe33c7e9e => udf_asin-3-4b7ee6310a49ebf784a4a712748348ac} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce delete mode 100644 sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 rename sql/hive/src/test/resources/golden/{decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e => udf_atan-3-e6f97a834028a67e6c3033949f98fbf8} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 delete mode 100644 sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 rename sql/hive/src/test/resources/golden/{metadataonly1-2-9eadfd16be30c653625fce7b74048d9d => udf_atan-7-e6f97a834028a67e6c3033949f98fbf8} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 rename sql/hive/src/test/resources/golden/{exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b create mode 100644 sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab delete mode 100644 sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee => udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 rename sql/hive/src/test/resources/golden/{groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 => udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 rename sql/hive/src/test/resources/golden/{groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 => udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 => udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 rename sql/hive/src/test/resources/golden/{groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 => udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b} (100%) rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a => udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df rename sql/hive/src/test/resources/golden/{udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad => udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 => udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c rename sql/hive/src/test/resources/golden/{exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 => udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 rename sql/hive/src/test/resources/golden/{groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d => udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 rename sql/hive/src/test/resources/golden/{groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 => udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 => udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 delete mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 rename sql/hive/src/test/resources/golden/{groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 => udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b} (100%) rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 => udf_case-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 delete mode 100644 sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 create mode 100644 sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 create mode 100644 sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 delete mode 100644 sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f rename sql/hive/src/test/resources/golden/{groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 => udf_case-3-48789112d79aeb450d9f49184fc20e1c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c delete mode 100644 sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 create mode 100644 sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 create mode 100644 sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb create mode 100644 sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc delete mode 100644 sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c delete mode 100644 sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 delete mode 100644 sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 delete mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 delete mode 100644 sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 rename sql/hive/src/test/resources/golden/{groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 => udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 rename sql/hive/src/test/resources/golden/{groupby_sort_11-4-475d50465b23adfb70e67122425ede9e => udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f rename sql/hive/src/test/resources/golden/{groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 => udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 => udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c rename sql/hive/src/test/resources/golden/{groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c => udf_concat_ws-7-97071809ba7701b78e3729996f14b591} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 delete mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 => udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab create mode 100644 sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f create mode 100644 sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a create mode 100644 sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f create mode 100644 sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb create mode 100644 sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 delete mode 100644 sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 rename sql/hive/src/test/resources/golden/{udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 => udf_cos-3-166acc86afd6ececfe43800e38f106c9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 rename sql/hive/src/test/resources/golden/{groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 => udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae rename sql/hive/src/test/resources/golden/{groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-1-aabc6065a03b7da809376cc127af47d7 => udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b => udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 => udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae rename sql/hive/src/test/resources/golden/{groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-5-aabc6065a03b7da809376cc127af47d7 => udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b => udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b} (100%) rename sql/hive/src/test/resources/golden/{udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 => udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db rename sql/hive/src/test/resources/golden/{groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e => udf_div-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db delete mode 100644 sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 delete mode 100644 sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b create mode 100644 sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 rename sql/hive/src/test/resources/golden/{custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 => udf_div-3-5111340caad64e36370d9d4bc4db5f27} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 delete mode 100644 sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 delete mode 100644 sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 rename sql/hive/src/test/resources/golden/{groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 => udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_elt-0-b46b060da76d1772db998c26a62a608f => udf_elt-1-b46b060da76d1772db998c26a62a608f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a rename sql/hive/src/test/resources/golden/{udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 => udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_elt-3-c2554fac72a2a51bb33faae16aec3507} (100%) rename sql/hive/src/test/resources/golden/{udf_elt-3-f3be980cf4fa166f299c6ec79e981814 => udf_elt-4-533ad9c703c320a6556c09dd5f9ac351} (100%) rename sql/hive/src/test/resources/golden/{groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 => udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 => udf_equal-1-36b6cdf7c5f68c91155569b1622f5876} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-1-2422b50b96502dde8b661acdfebd8892 => udf_equal-2-2422b50b96502dde8b661acdfebd8892} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 => udf_equal-3-e0faab0f5e736c24bcc5503aeac55053} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-3-39d8d6f197803de927f0af5409ec2f33 => udf_equal-4-39d8d6f197803de927f0af5409ec2f33} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 => udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 => udf_equal-6-878650cf21e9360a07d204c8ffb0cde7} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e => udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e} (100%) rename sql/hive/src/test/resources/golden/{udf_equal-7-78f1b96c199e307714fa1b804e5bae27 => udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 => udf_field-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 delete mode 100644 sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f delete mode 100644 sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 rename sql/hive/src/test/resources/golden/{groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_field-10-ca9db7e6bb687606bc273d1f6c191035} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 create mode 100644 sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f delete mode 100644 sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b rename sql/hive/src/test/resources/golden/{div-0-5e7fc5719c7265bc1d6af52005ebad03 => udf_field-3-5960d42257b272f6ba043406229bbf26} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 create mode 100644 sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 delete mode 100644 sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 delete mode 100644 sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 create mode 100644 sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b rename sql/hive/src/test/resources/golden/{groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 rename sql/hive/src/test/resources/golden/{groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b => udf_field-7-3fd8b0c333acdf28c676315b03e2e10} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef delete mode 100644 sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 create mode 100644 sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef rename sql/hive/src/test/resources/golden/{groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 => udf_field-9-66dc6c81db0ac9b2075783b0d8976083} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c rename sql/hive/src/test/resources/golden/{udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 => udf_find_in_set-10-df21f44247d7275a292520c1605c4aab} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a rename sql/hive/src/test/resources/golden/{decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e => udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 rename sql/hive/src/test/resources/golden/{decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 => udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 rename sql/hive/src/test/resources/golden/{decimal_2-25-14face5c7104382196e65741a199c36 => udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb rename sql/hive/src/test/resources/golden/{groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 => udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9} (100%) rename sql/hive/src/test/resources/golden/{groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 => udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 => udf_find_in_set-17-fe61f992f2d971d006155bdec3143803} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e rename sql/hive/src/test/resources/golden/{groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb => udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b rename sql/hive/src/test/resources/golden/{groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 => udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 rename sql/hive/src/test/resources/golden/{decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 => udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 rename sql/hive/src/test/resources/golden/{groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 => udf_find_in_set-8-ea11724531f191940e455d13878a0e69} (100%) rename sql/hive/src/test/resources/golden/{udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 => udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 rename sql/hive/src/test/resources/golden/{groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 => udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 rename sql/hive/src/test/resources/golden/{groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb => udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 rename sql/hive/src/test/resources/golden/{groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb => udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e delete mode 100644 sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 delete mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 rename sql/hive/src/test/resources/golden/{groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 => udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 rename sql/hive/src/test/resources/golden/{groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 => udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 delete mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 rename sql/hive/src/test/resources/golden/{groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 => udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_hash-0-b9e3a3986320d275982797140edfccf4 => udf_hash-1-b9e3a3986320d275982797140edfccf4} (100%) rename sql/hive/src/test/resources/golden/{udf_hash-1-a18646b51501d0b1beb967dc79afbd1a => udf_hash-2-a18646b51501d0b1beb967dc79afbd1a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 => udf_hash-3-2646a87ce26c383a9dafea9b56281ab7} (100%) rename sql/hive/src/test/resources/golden/{udf_hash-3-23991312391d518aacf3d4469c816eae => udf_hash-4-d1368c2e3cd113e46202156b44811987} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 => udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 create mode 100644 sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 create mode 100644 sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c create mode 100644 sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf create mode 100644 sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 delete mode 100644 sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea create mode 100644 sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 delete mode 100644 sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a rename sql/hive/src/test/resources/golden/{input16_cc-0-43d53504df013e6b35f81811138a167a => udf_if-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_if-0-b7ffa85b5785cccef2af1b285348cc2c => udf_if-1-b7ffa85b5785cccef2af1b285348cc2c} (100%) rename sql/hive/src/test/resources/golden/{udf_if-1-30cf7f51f92b5684e556deff3032d49a => udf_if-2-30cf7f51f92b5684e556deff3032d49a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 => udf_if-3-59e90bb74481aaf35480076806daf365} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca rename sql/hive/src/test/resources/golden/{udf_if-3-20206f17367ff284d67044abd745ce9f => udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c => udf_if-5-841a8068d35a42179d3654e1a2848c43} (100%) rename sql/hive/src/test/resources/golden/{udf_if-5-a7db13aec05c97792f9331d63709d8cc => udf_if-6-508f9140dd33931c7b9ad336dceb32cf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 delete mode 100644 sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 delete mode 100644 sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 delete mode 100644 sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 delete mode 100644 sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 rename sql/hive/src/test/resources/golden/{input31-0-823920925ca9c8a2ca9016f52c0f4ee => udf_index-0-e91e3e5a22029b9b979ccbbef97add66} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 create mode 100644 sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 create mode 100644 sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 rename sql/hive/src/test/resources/golden/{input31-1-c21dba410fb07a098f93430a9d21df79 => udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 create mode 100644 sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 delete mode 100644 sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 => udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 delete mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 delete mode 100644 sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd delete mode 100644 sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b rename sql/hive/src/test/resources/golden/{input32-0-823920925ca9c8a2ca9016f52c0f4ee => udf_length-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b delete mode 100644 sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea => udf_length-10-de456a5765db4a06110d9483985aa4a6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 create mode 100644 sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 delete mode 100644 sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 create mode 100644 sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 => udf_length-3-af46cb6887618240836eaf5be8afbba6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d delete mode 100644 sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 => udf_length-4-dcd6404afce1103d5054527e6c216d6d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 => udf_length-5-ba49ba4e6505c74bc33b5003f3930c43} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 => udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f delete mode 100644 sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 => udf_length-8-8f28e6c488df47d89dca670f04a7563f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 => udf_length-9-e41b220da98996f997b26ba7ef457a84} (100%) rename sql/hive/src/test/resources/golden/{input32-1-c21dba410fb07a098f93430a9d21df79 => udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 rename sql/hive/src/test/resources/golden/{input32-2-1ba7748b3d2f8908c2e81771ab229316 => udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b delete mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 rename sql/hive/src/test/resources/golden/{join_view-4-763ab5853bff619e6525c01e46b2a923 => udf_like-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_like-0-e0ba9a953e50554bdcbc55585cffde09 => udf_like-1-e0ba9a953e50554bdcbc55585cffde09} (100%) rename sql/hive/src/test/resources/golden/{udf_like-1-9781f89d352c506e972ad2a1d58ec03a => udf_like-2-9781f89d352c506e972ad2a1d58ec03a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 => udf_like-3-dbc46cb33f0dd356af03006d9492f8b7} (100%) rename sql/hive/src/test/resources/golden/{udf_like-3-bef03784eab9d5e8404fd24960dea4fc => udf_like-4-bef03784eab9d5e8404fd24960dea4fc} (100%) rename sql/hive/src/test/resources/golden/{udf_like-4-af5fe5d5d176f751747bf14055d00a12 => udf_like-5-47bfd4d65090dab890b467ae06cf3bd5} (100%) rename sql/hive/src/test/resources/golden/{load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 => udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e create mode 100644 sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c create mode 100644 sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 delete mode 100644 sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 => udf_locate-3-ce4a131f99dc9befa926027380b38dbb} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b delete mode 100644 sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 rename sql/hive/src/test/resources/golden/{load_dyn_part15-1-16367c381d4b189b3640c92511244bfe => udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f => udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 rename sql/hive/src/test/resources/golden/{load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e => udf_map-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e => udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e} (100%) rename sql/hive/src/test/resources/golden/{udf_map-1-1f8cd98df9bf7b2528506551fef87dcf => udf_map-2-1f8cd98df9bf7b2528506551fef87dcf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 => udf_map-3-be7b52baa973b8b59b7ca63fea19aa99} (100%) rename sql/hive/src/test/resources/golden/{udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 => udf_map-4-60cb9c30285f7a9f99377ccbd143eb06} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e delete mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e delete mode 100644 sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f delete mode 100644 sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 rename sql/hive/src/test/resources/golden/{load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e => udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf rename sql/hive/src/test/resources/golden/{udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 => udf_negative-10-1cd28efecc0d26f463221195f5e39956} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c create mode 100644 sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 create mode 100644 sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c rename sql/hive/src/test/resources/golden/{udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a => udf_negative-5-771e76b0acd8ddb128781da7819d0e47} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 rename sql/hive/src/test/resources/golden/{udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d => udf_negative-6-f62c4a097c592871d896a7dc47c42f61} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b rename sql/hive/src/test/resources/golden/{udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 => udf_negative-7-f838053f5ca5c8746dc299473dff0490} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 rename sql/hive/src/test/resources/golden/{udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da => udf_negative-8-f4f23aa6f634913d194a69261af8f3f6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa rename sql/hive/src/test/resources/golden/{udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 => udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e rename sql/hive/src/test/resources/golden/{load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e => udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 => udf_notequal-5-eb04e5ee00967799c913e8a5b424a332} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f => udf_notequal-7-46a6514f2d7e6a097035ec1559df0096} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd rename sql/hive/src/test/resources/golden/{loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 => udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 create mode 100644 sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 rename sql/hive/src/test/resources/golden/{lock3-3-16367c381d4b189b3640c92511244bfe => udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef => udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 delete mode 100644 sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd delete mode 100644 sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 delete mode 100644 sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 rename sql/hive/src/test/resources/golden/{lock3-4-a4fb8359a2179ec70777aad6366071b7 => udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 => udf_radians-1-58b73fc96927d447d1225f021eaa378} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 => udf_radians-6-70c9e7199b5898e2c3a4943ec58da113} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 delete mode 100644 sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 delete mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca rename sql/hive/src/test/resources/golden/{lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 => udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_regexp-0-19917611f74aedc0922560f7f2595948 => udf_regexp-1-19917611f74aedc0922560f7f2595948} (100%) rename sql/hive/src/test/resources/golden/{udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 => udf_regexp-2-f7f0527cd47612d7f256edd5f8963800} (100%) rename sql/hive/src/test/resources/golden/{udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 => udf_regexp-3-59aff54bae544ee620141e4e629f167a} (100%) rename sql/hive/src/test/resources/golden/{lock4-4-16367c381d4b189b3640c92511244bfe => udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d => udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 rename sql/hive/src/test/resources/golden/{lock4-5-a4fb8359a2179ec70777aad6366071b7 => udf_round-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 create mode 100644 sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c create mode 100644 sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de create mode 100644 sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 create mode 100644 sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d delete mode 100644 sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a create mode 100644 sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 delete mode 100644 sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b create mode 100644 sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b rename sql/hive/src/test/resources/golden/{merge4-0-b12e5c70d6d29757471b900b6160fa8a => udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 delete mode 100644 sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad rename sql/hive/src/test/resources/golden/{merge4-1-593999fae618b6b38322bc9ae4e0c027 => udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b => udf_rpad-3-66acb969c28a8e376782ccd0d442b450} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b rename sql/hive/src/test/resources/golden/{merge4-2-43d53504df013e6b35f81811138a167a => udf_second-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 delete mode 100644 sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 create mode 100644 sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 create mode 100644 sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 delete mode 100644 sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f delete mode 100644 sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 => udf_second-3-d678372e3837a16be245d2e33482f17f} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 delete mode 100644 sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e rename sql/hive/src/test/resources/golden/{merge4-3-a4fb8359a2179ec70777aad6366071b7 => udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c => udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 rename sql/hive/src/test/resources/golden/{udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a => udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a create mode 100644 sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa create mode 100644 sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef create mode 100644 sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d create mode 100644 sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 rename sql/hive/src/test/resources/golden/{udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c => udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a create mode 100644 sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e create mode 100644 sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a => udf_sign-7-ed2aaa1a416c0cccc04de970424e1860} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef create mode 100644 sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 delete mode 100644 sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d create mode 100644 sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee rename sql/hive/src/test/resources/golden/{merge4-4-16367c381d4b189b3640c92511244bfe => udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 create mode 100644 sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee create mode 100644 sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab rename sql/hive/src/test/resources/golden/{udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 => udf_sin-3-1d3a615e3aa252a317daa601811820b1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 create mode 100644 sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a delete mode 100644 sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 delete mode 100644 sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 delete mode 100644 sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 delete mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a rename sql/hive/src/test/resources/golden/{metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 => udf_space-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 create mode 100644 sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 delete mode 100644 sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 delete mode 100644 sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 create mode 100644 sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 delete mode 100644 sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 => udf_space-3-a1b9dad63547f7ba73a5230d650983b0} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 create mode 100644 sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 create mode 100644 sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 delete mode 100644 sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade delete mode 100644 sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d rename sql/hive/src/test/resources/golden/{newline-1-a19a19272149c732977c37e043910505 => udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 rename sql/hive/src/test/resources/golden/{udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 => udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 delete mode 100644 sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 rename sql/hive/src/test/resources/golden/{udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 => udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c delete mode 100644 sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad delete mode 100644 sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 rename sql/hive/src/test/resources/golden/{orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 => udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 rename sql/hive/src/test/resources/golden/{udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 => udf_to_byte-1-94eb069fb446b7758f7e06386486bec9} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 => udf_to_byte-2-233102b562824cf38010868478e91e1} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc => udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 rename sql/hive/src/test/resources/golden/{udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b => udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82} (100%) rename sql/hive/src/test/resources/golden/{exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 => udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 => udf_to_byte-6-489cd2d26b9efde2cdbff19254289371} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 => udf_to_byte-7-1eaba393f93af1763dd761172fb78d52} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 => udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79} (100%) rename sql/hive/src/test/resources/golden/{udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 => udf_to_byte-9-322163c32973ccc3a5168463db7a8589} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c rename sql/hive/src/test/resources/golden/{orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb => udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 rename sql/hive/src/test/resources/golden/{udf_to_float-0-7646eca02448547eedf84a81bf42be89 => udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6} (100%) rename sql/hive/src/test/resources/golden/{input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 => udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 => udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-3-cb3318ba365833316645e1b9890f4613 => udf_to_long-4-8c284b082a256abf0426d4f6f1971703} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 => udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-5-a0d382d243a226f8d4381970b6831c3d => udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-6-47ab11eae68329cc80232fc4089479f0 => udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-7-b87c0cabb166f33984cc1b191694918e => udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2} (100%) rename sql/hive/src/test/resources/golden/{udf_to_long-8-52a24d7040db321a842f9201d245ee9 => udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0} (100%) rename sql/hive/src/test/resources/golden/{orc_create-32-16367c381d4b189b3640c92511244bfe => udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a delete mode 100644 sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b rename sql/hive/src/test/resources/golden/{udf_to_long-0-ebe447e8fb03de8844951250fe50320a => udf_to_short-1-94f15fe043839493107058a06a210cf7} (100%) rename sql/hive/src/test/resources/golden/{loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 => udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 => udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce => udf_to_short-4-40ffb132d5641645e2b8043dc056fb0} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 => udf_to_short-5-5d6c46b0154d1073c035a79dbf612479} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 => udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 => udf_to_short-7-9f83813005b639a23901ca6ff87ff473} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-7-827f3181c216fd2e990637c9a091bf0d => udf_to_short-8-885656e165feb3a674cf636dbf08716c} (100%) rename sql/hive/src/test/resources/golden/{udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 => udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea delete mode 100644 sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b rename sql/hive/src/test/resources/golden/{orc_create-34-3b03210f94ec40db9ab02620645014d1 => udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae create mode 100644 sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf create mode 100644 sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 => udf_translate-3-42aba80bf1913dd7c64545831f476c58} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa => udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 => udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad => udf_translate-6-f2637240d227f9732d3db76f2e9d3a59} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 delete mode 100644 sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 delete mode 100644 sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 delete mode 100644 sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c delete mode 100644 sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 delete mode 100644 sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 delete mode 100644 sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 rename sql/hive/src/test/resources/golden/{parallel_orderby-10-767848077fc2929ccedfd28e50564b19 => udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb => udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 => udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 => udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 => udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 => udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 => udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 => udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31} (100%) rename sql/hive/src/test/resources/golden/{udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d => udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d} (100%) rename sql/hive/src/test/resources/golden/{parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 => udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 rename sql/hive/src/test/resources/golden/{parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 => udf_when-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 delete mode 100644 sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 create mode 100644 sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 create mode 100644 sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 delete mode 100644 sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 => udf_when-3-734890c41528b9d918db66b0582228a4} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f create mode 100644 sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 create mode 100644 sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 rename sql/hive/src/test/resources/golden/{parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 => udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 rename sql/hive/src/test/resources/golden/{partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 => udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 rename sql/hive/src/test/resources/golden/{udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f => udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 rename sql/hive/src/test/resources/golden/{udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df => udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 rename sql/hive/src/test/resources/golden/{udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a => udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f rename sql/hive/src/test/resources/golden/{udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 => udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 rename sql/hive/src/test/resources/golden/{udf_to_boolean-8-37229f303635a030f6cab20e0381f51f => udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 rename sql/hive/src/test/resources/golden/{partition_decode_name-3-16367c381d4b189b3640c92511244bfe => udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 rename sql/hive/src/test/resources/golden/{partition_decode_name-4-e90740a9a52c37a964ee204379f01412 => udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a rename sql/hive/src/test/resources/golden/{partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 => udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba rename sql/hive/src/test/resources/golden/{partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 => udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 rename sql/hive/src/test/resources/golden/{partition_special_char-3-16367c381d4b189b3640c92511244bfe => udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a rename sql/hive/src/test/resources/golden/{partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_int-6-ac509f06f01c02924adef220404fc515} (100%) rename sql/hive/src/test/resources/golden/{metadataonly1-5-9eadfd16be30c653625fce7b74048d9d => udf_xpath_int-7-87ff12d650afb8f511d906778450fae7} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 rename sql/hive/src/test/resources/golden/{partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b rename sql/hive/src/test/resources/golden/{partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 => udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 => udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 rename sql/hive/src/test/resources/golden/{metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc => udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b} (100%) create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f => udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f => udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa} (100%) rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 => udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e rename sql/hive/src/test/resources/golden/{exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 => udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861} (100%) rename sql/hive/src/test/resources/golden/{partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 => udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 rename sql/hive/src/test/resources/golden/{newline-0-43392a20a8d249a279d50d96578e6a1b => udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 delete mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d rename sql/hive/src/test/resources/golden/{partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf => udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 => udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc => udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 => udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f => udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a => udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a => udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 => udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 => udf_xpath_string-7-8e42951d002e3c4034b4a51928442706} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f => udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288} (100%) rename sql/hive/src/test/resources/golden/{udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 => udf_xpath_string-9-84109613320bd05abccd1058044d62c3} (100%) delete mode 100644 sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a delete mode 100644 sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 delete mode 100644 sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 delete mode 100644 sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 delete mode 100644 sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 delete mode 100644 sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 delete mode 100644 sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c delete mode 100644 sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 delete mode 100644 sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e delete mode 100644 sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 delete mode 100644 sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b delete mode 100644 sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 delete mode 100644 sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 delete mode 100644 sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 delete mode 100644 sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e delete mode 100644 sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 delete mode 100644 sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b delete mode 100644 sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 delete mode 100644 sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb delete mode 100644 sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db delete mode 100644 sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c => union34-4-70479e10c016e5ac448394dbadb32794} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 => union_date-4-d85fe746334b430941c5db3665e744d4} (100%) rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 => union_date-5-82eebfded24cef08e0a881d1bcca02b1} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d delete mode 100644 sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 => union_remove_11-9-94da21f150ed2c56046b80e46da8884d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb delete mode 100644 sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d delete mode 100644 sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d delete mode 100644 sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b delete mode 100644 sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b => union_remove_3-8-94da21f150ed2c56046b80e46da8884d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 => union_remove_6-9-94da21f150ed2c56046b80e46da8884d} (100%) delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 delete mode 100644 sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e delete mode 100644 sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d delete mode 100644 sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 delete mode 100644 sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 delete mode 100644 sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 delete mode 100644 sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e delete mode 100644 sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 delete mode 100644 sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae delete mode 100644 sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 delete mode 100644 sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 delete mode 100644 sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 => varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 => varchar_join1-7-a01639290aaceb4b85aa6e44319f6386} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 => varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 delete mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 delete mode 100644 sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 delete mode 100644 sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b => varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc => varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29} (100%) delete mode 100644 sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 rename sql/hive/src/test/resources/golden/{groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e => varchar_union1-8-f3be9a2498927d692356c2cf871d25bf} (100%) delete mode 100644 sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a delete mode 100644 sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 delete mode 100644 sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 delete mode 100644 sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 delete mode 100644 sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 delete mode 100644 sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 delete mode 100644 sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee delete mode 100644 sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b delete mode 100644 sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 delete mode 100644 sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a delete mode 100644 sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 delete mode 100644 sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b delete mode 100644 sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 delete mode 100644 sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 delete mode 100644 sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 delete mode 100644 sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf delete mode 100644 sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb delete mode 100644 sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de delete mode 100644 sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a delete mode 100644 sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb delete mode 100644 sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 delete mode 100644 sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 delete mode 100644 sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 delete mode 100644 sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 delete mode 100644 sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q delete mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_varchar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_split_elimination.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_vectorization_ppd.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order_within_subquery.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_create.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q delete mode 100644 sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala create mode 100644 sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala delete mode 100644 sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala create mode 100644 sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala diff --git a/.rat-excludes b/.rat-excludes index b14ad53720f32..ae9745673c87d 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -48,6 +48,7 @@ sbt-launch-lib.bash plugins.sbt work .*\.q +.*\.qv golden test.out/* .*iml diff --git a/dev/run-tests b/dev/run-tests index 7d06c86eb4b41..f55497ae2bfbd 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -167,7 +167,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS # If the Spark SQL tests are enabled, run the tests with the Hive profiles enabled. # This must be a single argument, as it is. if [ -n "$_RUN_SQL_TESTS" ]; then - SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" + SBT_MAVEN_PROFILES_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive" fi if [ -n "$_SQL_TESTS_ONLY" ]; then diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 705937e3016e2..ea04473854007 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -253,7 +253,11 @@ object Hive { |import org.apache.spark.sql.hive._ |import org.apache.spark.sql.hive.test.TestHive._ |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin, - cleanupCommands in console := "sparkContext.stop()" + cleanupCommands in console := "sparkContext.stop()", + // Some of our log4j jars make it impossible to submit jobs from this JVM to Hive Map/Reduce + // in order to generate golden files. This is only required for developers who are adding new + // new query tests. + fullClasspath in Test := (fullClasspath in Test).value.filterNot { f => f.toString.contains("jcl-over") } ) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 7c480de107e7f..2b69c02b28285 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -52,6 +52,8 @@ object HiveTypeCoercion { */ trait HiveTypeCoercion { + import HiveTypeCoercion._ + val typeCoercionRules = PropagateTypes :: ConvertNaNs :: @@ -340,6 +342,13 @@ trait HiveTypeCoercion { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e + case a @ CreateArray(children) if !a.resolved => + val commonType = a.childTypes.reduce( + (a,b) => + findTightestCommonType(a,b).getOrElse(StringType)) + CreateArray( + children.map(c => if (c.dataType == commonType) c else Cast(c, commonType))) + // Promote SUM, SUM DISTINCT and AVERAGE to largest types to prevent overflows. case s @ Sum(e @ DecimalType()) => s // Decimal is already the biggest. case Sum(e @ IntegralType()) if e.dataType != LongType => Sum(Cast(e, LongType)) @@ -356,6 +365,10 @@ trait HiveTypeCoercion { Average(Cast(e, LongType)) case Average(e @ FractionalType()) if e.dataType != DoubleType => Average(Cast(e, DoubleType)) + + // Hive lets you do aggregation of timestamps... for some reason + case Sum(e @ TimestampType()) => Sum(Cast(e, DoubleType)) + case Average(e @ TimestampType()) => Average(Cast(e, DoubleType)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index dafd745ec96c6..19421e5667138 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -101,3 +101,28 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio override def toString = s"$child.$fieldName" } + +/** + * Returns an Array containing the evaluation of all children expressions. + */ +case class CreateArray(children: Seq[Expression]) extends Expression { + override type EvaluatedType = Any + + lazy val childTypes = children.map(_.dataType).distinct + + override lazy val resolved = + childrenResolved && childTypes.size <= 1 + + override def dataType: DataType = { + assert(resolved, s"Invalid dataType of mixed ArrayType ${childTypes.mkString(",")}") + ArrayType(childTypes.headOption.getOrElse(NullType)) + } + + override def nullable: Boolean = false + + override def eval(input: Row): Any = { + children.map(_.eval(input)) + } + + override def toString = s"Array(${children.mkString(",")})" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 590dbf3cb893d..c4f4ef01d78df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.types.DataType -import org.apache.spark.sql.columnar.InMemoryRelation import org.apache.spark.sql.execution.{SparkStrategies, _} import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 463888551a359..15cd62d3bf869 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -23,6 +23,7 @@ import java.util.{Locale, TimeZone} import org.scalatest.BeforeAndAfter import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.hive.test.TestHive /** @@ -102,6 +103,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf5", "udf_java_method", "create_merge_compressed", + "database_location", + "database_properties", // DFS commands "symlink_text_input_format", @@ -135,6 +138,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "stats20", "alter_merge_stats", "columnstats.*", + "annotate_stats.*", + "database_drop", + "index_serde", // Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong. @@ -211,8 +217,20 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "describe_comment_indent", // Limit clause without a ordering, which causes failure. - "orc_predicate_pushdown" - ) + "orc_predicate_pushdown", + + // Requires precision decimal support: + "decimal_1", + "udf_pmod", + "udf_when", + "udf_case", + "udf_to_double", + "udf_to_float", + + // Needs constant object inspectors + "udf_round", + "udf7" + ) ++ HiveShim.compatibilityBlackList /** * The set of tests that are believed to be working in catalyst. Tests not on whiteList or @@ -220,23 +238,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { */ override def whiteList = Seq( "add_part_exist", - "dynamic_partition_skip_default", - "infer_bucket_sort_dyn_part", - "load_dyn_part1", - "load_dyn_part2", - "load_dyn_part3", - "load_dyn_part4", - "load_dyn_part5", - "load_dyn_part6", - "load_dyn_part7", - "load_dyn_part8", - "load_dyn_part9", - "load_dyn_part10", - "load_dyn_part11", - "load_dyn_part12", - "load_dyn_part13", - "load_dyn_part14", - "load_dyn_part14_win", "add_part_multiple", "add_partition_no_whitelist", "add_partition_with_whitelist", @@ -256,6 +257,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "alter_varchar2", "alter_view_as_select", "ambiguous_col", + "annotate_stats_join", + "annotate_stats_limit", + "annotate_stats_part", + "annotate_stats_table", + "annotate_stats_union", "auto_join0", "auto_join1", "auto_join10", @@ -299,6 +305,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "auto_sortmerge_join_13", "auto_sortmerge_join_14", "auto_sortmerge_join_15", + "auto_sortmerge_join_16", "auto_sortmerge_join_2", "auto_sortmerge_join_3", "auto_sortmerge_join_4", @@ -340,7 +347,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "create_skewed_table1", "create_struct_table", "cross_join", + "cross_product_check_1", + "cross_product_check_2", "ct_case_insensitive", + "database_drop", "database_location", "database_properties", "date_2", @@ -360,8 +370,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "diff_part_input_formats", "disable_file_format_check", "disallow_incompatible_type_change_off", + "distinct_stats", + "drop_database_removes_partition_dirs", "drop_function", "drop_index", + "drop_index_removes_partition_dirs", "drop_multi_partitions", "drop_partitions_filter", "drop_partitions_filter2", @@ -369,23 +382,30 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "drop_partitions_ignore_protection", "drop_table", "drop_table2", + "drop_table_removes_partition_dirs", "drop_view", + "dynamic_partition_skip_default", "escape_clusterby1", "escape_distributeby1", "escape_orderby1", "escape_sortby1", + "explain_rearrange", "fetch_aggregation", + "fileformat_mix", "fileformat_sequencefile", "fileformat_text", "filter_join_breaktask", "filter_join_breaktask2", "groupby1", "groupby11", + "groupby12", + "groupby1_limit", "groupby1_map", "groupby1_map_nomap", "groupby1_map_skew", "groupby1_noskew", "groupby2", + "groupby2_limit", "groupby2_map", "groupby2_map_skew", "groupby2_noskew", @@ -406,6 +426,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby7_map_multi_single_reducer", "groupby7_map_skew", "groupby7_noskew", + "groupby7_noskew_multi_single_reducer", "groupby8", "groupby8_map", "groupby8_map_skew", @@ -432,6 +453,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "groupby_sort_test_1", "having", "implicit_cast1", + "index_serde", + "infer_bucket_sort_dyn_part", "innerjoin", "inoutdriver", "input", @@ -502,7 +525,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join17", "join18", "join19", - "join_1to1", "join2", "join20", "join21", @@ -534,6 +556,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "join7", "join8", "join9", + "join_1to1", "join_array", "join_casesensitive", "join_empty", @@ -557,7 +580,21 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "literal_double", "literal_ints", "literal_string", + "load_dyn_part1", + "load_dyn_part10", + "load_dyn_part11", + "load_dyn_part12", + "load_dyn_part13", + "load_dyn_part14", + "load_dyn_part14_win", + "load_dyn_part2", + "load_dyn_part3", + "load_dyn_part4", + "load_dyn_part5", + "load_dyn_part6", "load_dyn_part7", + "load_dyn_part8", + "load_dyn_part9", "load_file_with_space_in_the_name", "loadpart1", "louter_join_ppr", @@ -578,13 +615,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "merge1", "merge2", "mergejoins", - "multigroupby_singlemr", + "multiMapJoin1", + "multiMapJoin2", "multi_insert_gby", "multi_insert_gby3", "multi_insert_lateral_view", "multi_join_union", - "multiMapJoin1", - "multiMapJoin2", + "multigroupby_singlemr", "noalias_subq1", "nomore_ambiguous_table_col", "nonblock_op_deduplicate", @@ -607,10 +644,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "outer_join_ppr", "parallel", "parenthesis_star_by", - "partcols1", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", + "partcols1", "partition_date", "partition_schema1", "partition_serde_format", @@ -641,7 +678,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "ppd_outer_join5", "ppd_random", "ppd_repeated_alias", - "ppd_transform", "ppd_udf_col", "ppd_union", "ppr_allchildsarenull", @@ -674,15 +710,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "serde_regex", "serde_reported_schema", "set_variable_sub", - "show_create_table_partitioned", - "show_create_table_delimited", + "show_columns", "show_create_table_alter", - "show_create_table_view", - "show_create_table_serde", "show_create_table_db_table", + "show_create_table_delimited", "show_create_table_does_not_exist", "show_create_table_index", - "show_columns", + "show_create_table_partitioned", + "show_create_table_serde", + "show_create_table_view", "show_describe_func_quotes", "show_functions", "show_partitions", @@ -738,12 +774,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udaf_covar_pop", "udaf_covar_samp", "udaf_histogram_numeric", - "udf_10_trims", "udf2", "udf6", "udf7", "udf8", "udf9", + "udf_10_trims", + "udf_E", + "udf_PI", "udf_abs", "udf_acos", "udf_add", @@ -774,14 +812,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_cos", "udf_count", "udf_date_add", - "udf_datediff", "udf_date_sub", + "udf_datediff", "udf_day", "udf_dayofmonth", "udf_degrees", "udf_div", "udf_double", - "udf_E", "udf_elt", "udf_equal", "udf_exp", @@ -826,7 +863,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_nvl", "udf_or", "udf_parse_url", - "udf_PI", "udf_pmod", "udf_positive", "udf_pow", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 34ed57b001637..fad4091d48a89 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -46,7 +46,6 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand -import org.apache.spark.sql.hive.HiveShim /** * DEPRECATED: Use HiveContext instead. @@ -230,7 +229,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * set() or a SET command inside sql() will be set in the SQLConf *as well as* * in the HiveConf. */ - @transient protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState]) + @transient lazy val hiveconf = new HiveConf(classOf[SessionState]) @transient protected[hive] lazy val sessionState = { val ss = new SessionState(hiveconf) setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index deaa1a2a154f2..fad7373a2fa39 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -26,7 +26,6 @@ import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.HiveShim /* Implicit conversions */ import scala.collection.JavaConversions._ 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 904bb48691e35..04c48c385966e 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 @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.plans.logical 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.hive.HiveShim import org.apache.spark.util.Utils /* Implicit conversions */ 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 ffcb6b505b9c6..54c619722ee12 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 @@ -251,6 +251,8 @@ private[hive] object HiveQl { s""" |Unsupported language features in query: $sql |${dumpTree(getAst(sql))} + |$e + |${e.getStackTrace.head} """.stripMargin) } } @@ -329,6 +331,7 @@ private[hive] object HiveQl { case Token("TOK_SMALLINT", Nil) => ShortType case Token("TOK_BOOLEAN", Nil) => BooleanType case Token("TOK_STRING", Nil) => StringType + case Token("TOK_VARCHAR", Token(_, Nil) :: Nil) => StringType case Token("TOK_FLOAT", Nil) => FloatType case Token("TOK_DOUBLE", Nil) => DoubleType case Token("TOK_DATE", Nil) => DateType @@ -854,9 +857,11 @@ private[hive] object HiveQl { HiveParser.Number, HiveParser.TinyintLiteral, HiveParser.SmallintLiteral, - HiveParser.BigintLiteral) + HiveParser.BigintLiteral, + HiveParser.DecimalLiteral) /* Case insensitive matches */ + val ARRAY = "(?i)ARRAY".r val COUNT = "(?i)COUNT".r val AVG = "(?i)AVG".r val SUM = "(?i)SUM".r @@ -917,7 +922,9 @@ private[hive] object HiveQl { /* Casts */ case Token("TOK_FUNCTION", Token("TOK_STRING", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) - case Token("TOK_FUNCTION", Token("TOK_VARCHAR", Nil) :: arg :: Nil) => + case Token("TOK_FUNCTION", Token("TOK_VARCHAR", _) :: arg :: Nil) => + Cast(nodeToExpr(arg), StringType) + case Token("TOK_FUNCTION", Token("TOK_CHAR", _) :: arg :: Nil) => Cast(nodeToExpr(arg), StringType) case Token("TOK_FUNCTION", Token("TOK_INT", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), IntegerType) @@ -1009,6 +1016,8 @@ private[hive] object HiveQl { GetItem(nodeToExpr(child), nodeToExpr(ordinal)) /* Other functions */ + case Token("TOK_FUNCTION", Token(ARRAY(), Nil) :: children) => + CreateArray(children.map(nodeToExpr)) 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)) @@ -1042,10 +1051,10 @@ private[hive] object HiveQl { } else if (ast.getText.endsWith("Y")) { // Literal tinyint. v = Literal(ast.getText.substring(0, ast.getText.length() - 1).toByte, ByteType) - } else if (ast.getText.endsWith("BD")) { + } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) { // Literal decimal - val strVal = ast.getText.substring(0, ast.getText.length() - 2) - BigDecimal(strVal) + val strVal = ast.getText.stripSuffix("D").stripSuffix("B") + v = Literal(BigDecimal(strVal)) } else { v = Literal(ast.getText.toDouble, DoubleType) v = Literal(ast.getText.toLong, LongType) @@ -1056,7 +1065,7 @@ private[hive] object HiveQl { } if (v == null) { - sys.error(s"Failed to parse number ${ast.getText}") + sys.error(s"Failed to parse number '${ast.getText}'.") } else { v } 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 e45eb57b3debf..9ff7ab5a124c1 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 @@ -34,7 +34,6 @@ import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.hive.HiveShim /** * A trait for subclasses that handle table scans. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index c6ff4ea6de594..bb79ad5538046 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -20,6 +20,9 @@ package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.session.SessionState + import scala.collection.mutable import scala.language.implicitConversions @@ -119,7 +122,7 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { if (cmd.toUpperCase contains "LOAD DATA") { val testDataLocation = hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) - cmd.replaceAll("\\.\\.", testDataLocation) + cmd.replaceAll("\\.\\./\\.\\./", testDataLocation + "/") } else { cmd } @@ -417,6 +420,8 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { FunctionRegistry.unregisterTemporaryUDF(udfName) } + // Some tests corrupt this value on purpose, which breaks the RESET call below. + hiveconf.set("fs.default.name", new File(".").toURI.toString) // It is important that we RESET first as broken hooks that might have been set could break // other sql exec here. runSqlHive("RESET") diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala index a201d2349a2ef..1817c7832490e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala @@ -19,15 +19,15 @@ package org.apache.spark.sql.hive.api.java import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD} -import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.{HiveContext, HiveQl} /** * The entry point for executing Spark SQL queries from a Java program. */ -class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(sparkContext) { +class JavaHiveContext(sqlContext: SQLContext) extends JavaSQLContext(sqlContext) { - override val sqlContext = new HiveContext(sparkContext) + def this(sparkContext: JavaSparkContext) = this(new HiveContext(sparkContext)) override def sql(sqlText: String): JavaSchemaRDD = { // TODO: Create a framework for registering parsers instead of just hardcoding if statements. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala index fbd375639692f..5d98834c6fb33 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/DescribeHiveTableCommand.scala @@ -45,7 +45,7 @@ case class DescribeHiveTableCommand( lazy val hiveString: Seq[String] = sideEffectResult.map { case Row(name: String, dataType: String, comment) => Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse(HiveShim.getEmptyCommentsFieldValue)) + Option(comment.asInstanceOf[String]).getOrElse("")) .map(s => String.format(s"%-20s", s)) .mkString("\t") } diff --git a/sql/hive/src/test/resources/data/conf/hive-site.xml b/sql/hive/src/test/resources/data/conf/hive-site.xml index 4e6ff16135833..7931d6a7128fe 100644 --- a/sql/hive/src/test/resources/data/conf/hive-site.xml +++ b/sql/hive/src/test/resources/data/conf/hive-site.xml @@ -19,6 +19,12 @@ + + hive.in.test + true + Internal marker for test. Used for masking env-dependent values + + @@ -28,7 +34,7 @@ hadoop.tmp.dir - ${build.dir.hive}/test/hadoop-${user.name} + ${test.tmp.dir}/hadoop-tmp A base for other temporary directories. @@ -42,22 +48,27 @@ hive.exec.scratchdir - ${build.dir}/scratchdir + ${test.tmp.dir}/scratchdir Scratch space for Hive jobs hive.exec.local.scratchdir - ${build.dir}/localscratchdir/ + ${test.tmp.dir}/localscratchdir/ Local scratch space for Hive jobs javax.jdo.option.ConnectionURL - - jdbc:derby:;databaseName=../build/test/junit_metastore_db;create=true + jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true + + hive.stats.dbconnectionstring + jdbc:derby:;databaseName=${test.tmp.dir}/TempStatsStore;create=true + + + javax.jdo.option.ConnectionDriverName org.apache.derby.jdbc.EmbeddedDriver @@ -82,7 +93,7 @@ hive.metastore.metadb.dir - file://${build.dir}/test/data/metadb/ + file://${test.tmp.dir}/metadb/ Required by metastore server or if the uris argument below is not supplied @@ -90,32 +101,19 @@ test.log.dir - ${build.dir}/test/logs - - - - - test.src.dir - file://${build.dir}/src/test + ${test.tmp.dir}/log/ test.data.files - ${user.dir}/../data/files - - - - - test.query.file1 - file://${user.dir}/../ql/src/test/org/apache/hadoop/hive/ql/input2.q - + ${hive.root}/data/files hive.jar.path - ${build.dir.hive}/ql/hive-exec-${version}.jar + ${maven.local.repository}/org/apache/hive/hive-exec/${hive.version}/hive-exec-${hive.version}.jar @@ -127,7 +125,7 @@ hive.querylog.location - ${build.dir}/tmp + ${test.tmp.dir}/tmp Location of the structured hive logs @@ -143,18 +141,25 @@ Post Execute Hook for Tests - - hive.task.progress - false - Track progress of a task - - hive.support.concurrency true Whether hive supports concurrency or not. A zookeeper instance must be up and running for the default hive lock manager to support read-write locks. + + hive.unlock.numretries + 2 + The number of times you want to retry to do one unlock + + + + hive.lock.sleep.between.retries + 2 + The sleep time (in seconds) between various retries + + + fs.pfile.impl org.apache.hadoop.fs.ProxyLocalFileSystem @@ -194,4 +199,21 @@ The default SerDe hive will use for the rcfile format + + hive.stats.dbclass + jdbc:derby + The storage for temporary stats generated by tasks. Currently, jdbc, hbase and counter types are supported + + + + hive.stats.key.prefix.reserve.length + 0 + + + + hive.conf.restricted.list + dummy.config.value + Using dummy config value above because you cannot override config with empty value + + diff --git a/sql/hive/src/test/resources/data/files/ProxyAuth.res b/sql/hive/src/test/resources/data/files/ProxyAuth.res new file mode 100644 index 0000000000000..96eca8f61de33 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/ProxyAuth.res @@ -0,0 +1,15 @@ ++-----+-------+ +| id | name | ++-----+-------+ +| 1 | aaa | +| 2 | bbb | +| 3 | ccc | +| 4 | ddd | +| 5 | eee | ++-----+-------+ ++-------+-----+ +| name | id | ++-------+-----+ +| aaa | 1 | +| bbb | 2 | ++-------+-----+ diff --git a/sql/hive/src/test/resources/data/files/alltypes.txt b/sql/hive/src/test/resources/data/files/alltypes.txt new file mode 100644 index 0000000000000..358cf400ec815 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/alltypes.txt @@ -0,0 +1,2 @@ +true|10|100|1000|10000|4.0|20.0|2.2222|1969-12-31 15:59:58.174|1970-01-01 00:00:00|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"} +true|20|200|2000|20000|8.0|40.0|4.2222|1970-12-31 15:59:58.174|1971-01-01 00:00:00|||k3:v3,k4:v4|200,300|{20, "bar"} diff --git a/sql/hive/src/test/resources/data/files/alltypes2.txt b/sql/hive/src/test/resources/data/files/alltypes2.txt new file mode 100644 index 0000000000000..c6a05a1dad893 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/alltypes2.txt @@ -0,0 +1,2 @@ +true|10|100|1000|10000|4.0|20.0|4.2222|1969-12-31 15:59:58.174|1970-01-01|string|hello|hello|k1:v1,k2:v2|100,200|{10, "foo"} +false|20|200|2000|20000|8.0|40.0|2.2222|1970-12-31 15:59:58.174|1971-01-01|abcd|world|world|k3:v3,k4:v4|200,300|{20, "bar"} diff --git a/sql/hive/src/test/resources/data/files/alltypesorc b/sql/hive/src/test/resources/data/files/alltypesorc new file mode 100644 index 0000000000000000000000000000000000000000..95c68cabd15e0ce34d472d552f4bb60193614b6d GIT binary patch literal 377237 zcmY)URahN7*ENjd?(Qtyb>Z%?(1p9ZySo>6clSam?k>gM9SX&vSkZky?|-ttljKUq z$jr>a%#=h$O%fdf0tpri8sQrqI|Rg0>b{wE2n2;uV!N9J3CSZAH`YymGb9Ax|Dc2a zgTxb!-6VuajK6?LLjUIIUEhlFAmsjo)qejUjF?#DCNV^Uz2^8a&f0W+oAh;w73Zhz ztZR+_|I(5U|D&CL^}o)svD-dIi@sw8uR4CcaN|SB|0iwn(99eHngaGeJ=7NSF^mQv z#Q{0V_k8Q(se=^0=yUaJ@DOVM|2*UW@c>@Kdw=0!a+o?q31?N@(jD46czl1_r%*vS z{Kw@y{XY#!|2u&Exj_#coQQ8Et}g{4|7;FO!Yl(f2u>zShOjT{_jf!aJdlwt9_w$= z|EKfG|1D%lOgfY=HK>P!yAJA<5X!(EvnolXz>^u9?SvruPd?W1p;-tth1GvL|G#5n zHwfDQ;U|YCAy5>`|Lw2GY@j%E`2Wr(9sX~&@dZ)?0^+T|=<)08P3)cQTR-%jGvE_> zW9spvA3NcpwT9ivyvwB<%;400{x!4vX5qW|ai+X}>0c@phi& zJM2xuV4eqQP;wQ!FP~>`hbm&~=~7~5#c9bH3z>f88u3nSzu3|H2N^Ge0u1~ZcJ*m; zdU@a-A4&=@J_s$I4I38|Ik%2s(yBFsA10W{OEV_fedq1cE84>T!L3Z7%Xd&0walH* z$lY1Q>^L+%&}A5k7#RsymR#j$`sMt&p(j|izTFG9t{ECvk!J^~%o@(i!K*NW6we5d{mG4WvGV43blM)x;pw2Cupolqfr)}o*ZQ-VA zm&lG6+Wmv6-qT~69$l|;;g@ApeW$&Hj9HEU`!xMo{f zTNN`6q!ZGBGHd%ARKK*(89VTE(3}H_1HSPr!2nDd{ErMfoy9Ctc&-daBn|6*Riz(o zw&2q|5N$U@P=@^TN2r@1vbl%tq~w$IF9|`-ib%}QvMx>g3vT_?fBW4wM4!bmI81yN zfpW$cIVu&aTvSiDHUI39Ww6{lv+OJrcy40+Q`y;>zj^#lFKpx zXs+O2NA}bJ1h5m1?h-#Sca65Yj(LV5*y*F+FC^U(>{z^i|Hcb=1939)h6OfmiotCB z&8dzytl;@<$Y4YM-B_#12Gnpth2nP2FEiEWgxJz?WM36z(vL?`q`Nzvlp;1jvQv-; zuIC#JG5tW!_ol-g^=hOLGo_z8k>~@6-;5aQ(Bq>Rl3;JSkuNWwl1dR&LsH0$Wg>~Z^qYv5!8{=tkuE;ySes`B1>d$0{vw=f-Qj8Z?H z9&ahWDj)OO-8|DZs%>iy=TAYq;L7|bzhTql6DImJ^z_ZcWAF5P-W}B{>TOgV5gq)! z5*;T!8+QTeQcA7ru>+<0YOAw(DqqK6h889kEx5GOGOf6zV!8vo*eRheFoRQ_c`vd> zbiKonFMJjVsZJRJN_G6A#=R>KSCdEmR=`N0b-yrU$34kBltJ7{f6z0o>8u^U&Kgve z@zX&Ozp3n=iOK-Uwm;ae9(2t&@t{YSzR>eMy|37g$e>&`YiS+Ts~Yf@AN)21Ss=** z%yTgwNP`+g0R(2w24`FwxkgV=gewE}e6&QMhY0($k3wqQ-c8^PYv^Wq{6|RcNAmt~ zO(QE^D|`o6RwYtS(Qf{hEGOJS;y@tC{CP>D5v;2W9krjy4cBX3~oKWb-jTl zzSNCE4d5+0tG#i(5MecJqB7Q>g-W}mm*v^6^qNxce3l&0byu*2>Ea|tsi&*ZmZ=U3 zht2f+JXETD(FD4Ldnvj59;NDk?-yDa$_)sjrG{ zkj%X+1?M4jSjP;eG8X^R)*LJMwmkt-s@7qQ*-<%#GteK;V+P{||0U5eDLBX2f$ooE z<|2--K#h>kOpCnwTfS6aS{0>Ml@~|aUro^=RI>X09$Dz%XHg5jewrZ%f=%5K4;>8$ zm&HBB66^+YHni8ue8rDU54_T7PLd-}ZQ*I-mG)mO6}wk7^PrY*btt21)>neh8N0$f zpJ;e#X*wuZ#eRz@HaB!>BewHKPQ+UM!jzTa$7%l*^?Zx6>_I_?nqpiDVLV<)CyOVb`06*1h8@~BsD zuvErN4Q>P!Ka-Y?N>y}nhZB^QH8PZ=kF$&A(9OL(&8sz{#8$K7PvY4OP~qK-_?@}4lLMW9(-g}4)&EB!^hD~*`3bVNP4R3*EvVO zT4^WfU1w9Y4kR^&227$qdDv0*No%P8V#nR2PLHXwQiDPT3FJ}i8}2I0pL#2O6>-6P z>b{nM3_ol+XVu(uJkfPso)#fBg_UfReyf00?+E<1XR_wgYZAU)&Ul>HUr9<<*Ah9g zWk6gFH>j$hypQQ^bOht#I=+a^M$}p-8{O9kw^mTyMK}!!m2osjNXTHkYu|l+sH6D1 zR`$Mwx|Z@wo~QRniy%`<6#B=)>@gT!-A>fWW!sITUv8ve$|Azm*?o=5yd?6~Y_6P} z7(M7t&ig3{V!jwzNK8|3s9BsYH16?*jKzIOc9mN- zw0ETMceQlHtJ+UNf=T2OedwPGs_hBPBIi5||KbdYlqe?Nbz7+-S;6HcM}(}J*?+td zB5@h|vfiNm7=SY11_hZa*Z~@(z2)8AZPtTkA|Jos zW9{VfO8BJ>;0Zj5DWO3ymx#T|DcN+tV~JnE++jtO^iPk z9q#(!!zWtz=Y0$Q{276f@y~9;2Cuq&EOM37)g>oXa?tsC-qoWD0zf-Lbi7lVqFp== zmg3Ll^a0j`sdSh3*LERS%%*If!vTPL_`8AEtB8zoU)eJyei;W=eBJ{9-MUMWw9xh^ zibs``r=!fge@id1_!ldGj9wk&HzB=`%Fa#T3tetjn3AuZ8Q||Dy=9z$85sfQpNpVn z-=mVb-$;$uB(uKdUNC~&t*Yj0+)r;#j`kFlHbf@FKiK)~%in{uN6zviG&gH76orN>Hcn zQ)<5#z^0*URJsDGPcZ~K@quS^6+7F@$CD<1~{40qoCggtnr8P1#!&b)-pUsfY>!wetsZ^j#nz2z_KP^o1AW3-&O$$v$%(BjHFHN+jwOVUBLsc@H~U`o+}C7q%oSzvOOdeBLR@8p znTcv7RBAaw1?l2J~`V@0dY;C=*xSk4!4?1AQN2d|)kgscJLQP8 zU=v2ZF44zWsmvQ`n2L8N!VsjcbD;ZN=hjyiGg9_1{gO}X%CNcJ8|J3>RT=mbO?;p% zpLa5yz6q#89Dw-&hTeF5rz|&xfh%DYoCYqC^A&dWe~iKrNm(CNYT)kbwqGYM@kqwh zvSZK3Ikc_d>i!_-O9se;g^qc8vi4M|5FJ|DMjz|UAv!9gotf*I+UfWZ$TIJ$JW2hv(imb59GU zmcMl@#Hldf)G7#iIgt0 z=VnM~XzuaF6-#wf8uS(=BKogxM?aDd&THVuRQ1;-IJ~>eTFs{}W%#K+M5><5j2pDE ze(%Fg<&2u<`c*i(8FOk%m?{>LE5DNNh>sqm^acItPIDslvGw8#yE-naHNoeW~uYA-yYw@=ruf1 zs|Uo1GGA@NRKMlPIBxui9>8FBUadC#YFy_AyK7a%-U@zq$b%P0Nr3HAD?9_va^MeE zZ#;4WOr@O{a}MkJ+c5`BP)8UBYS__tYCkc_7zb+Om#q{SZ!%4p53hTRH62HuK8pX;t}zq`X*!Ty zV`*o+KQ<#rNP>S2dzP$NkCe(`M3J=`{kc(z!f$Y4GiX!wNrp8hj!1NyiXn?*muuECD2nq7r4Zb}D{MN`};2)+Z#7V`3K&%Y+eU)6&_v1)w`K| zude(4bz78~_X|C<$c`l=u{mFK$w%7@?I3hf*tmNAut@y__NV!1?$T$$BizFAJmuk;#w)uM;T3K#OLFQ$nV(H$L>6a-$HATOGHn!CBY>>HnSXhVG+-9 z>YqR11(~;qE|2$zJ5OKLm(F`9Hw42&%&av&lW(|X3`PM!x*sXlB8%^F*^KzTtIEyi zv;;}|TaMch=U|~8NE7WJS9%c|%favE7J#!D`g$W8EcTX&b`R|G(c(*0e2ch}Easkb z>&t7hkYuy@8VAWwbjTybh&Pw=Xz{VMMj(2UEf>S5?VT2T|CpKvSwxa7?P2?l1u0TU zMrb6)d6;xVU*gfsBIbLZ#$$2yIb&#s7rZvNoinfJ*S*)8ZiF6!-#lL_HI7_Mx2|FR zvq9KEyfrU|L>M)6%Z^zLi^T=Ol76&ZLMC50(Z*ipcW42g%T|k^C5&4sXVdXmBBR-~ zcXBy)#7_9KIZ4UEk5{uZx&Y+*nz?FaA3i&P$K%%Bo1R=MkNlvvz^^i{_4R6T??@@? zXt}W?CF07z*byZ^H#_Ke3=)~#hmNfhS)+dU!Uwcs&g+FtILZVd_FwhucnT$`i$R!# z7Sh3}(68`W1XN`5ToQ68Mv8V}9irY4X3`zHgox3f{p-^I z51Bn0D7T8Zo+GA?U-`W2!SRhV6d6}v6+9?2t%oDXjBm$__g4?$S^(`07ZDPo0AcVe z2^!-68pAJcW6PFkWj-n1F7Pk$6*4IkKPAK*b|NS1HFkm|3uW@{1BKE$;De72-5jqw z*p~LO96(3ReQOj;2DPh*^LF-)%|lNfa*LHdjYfIJrJJ+TZ8ZCJ{WWxF;vR-cL7Wr8 z?FN!#-8nhw`!j>h`7q3|LA?HMy2c7zK?keC*OzvH|91&ka!I$eq?t{OETe4|rt>-{ zgAOEbw@Fp!n00gmilR`LLhTj#jyyN{@YzA-7K*w2g?Ig#1tVE=dz)3* zMic4j^E5^{v(Z|mpy;Wxw(gY{{;+B_ny8`2vYmb<1|1+JHEi*f5t(IIm08ffkgi7M z@<{LTLe?A(dNY(@rm|xh`Gw+{OP{fP07a$p?L`44fDeQR+x%$*YgJLG*E2->0|DU$ zJrVt}JEsPRZrcniiwBl6q&2SNJcacSrI7%(A^eZJ*<4%tmB|u(klxKwV5KBIg^Q- z+N(_Eex^oCSX!{|6rToo7y`QI){-> zp%)s0Hw5zoQS!F8zjbaIe=|S}X6r~@C%>Y8qbs3!{Y7hKlO7+ABH`pKI#3PJI5jwVoOzk`d#$E77uy}38J4jVG^gdiM;TleC{r!ZTUR^1|i?-^z8xfV9;TOWX;NX5e(_r!hQL9EXwtk8X z6OFHzpq-vWpv3?V$X`J-aZ%y&KN>ss2nC39dE55QeSH0|f z25{A>sD6)&ifnS&P<39NbHw;OSr6oU7t?uHv1?1L8j z!ciHX&eN9stD-GdU3P-U15EErcN(4Rp9*?rSZ&|WQpfOP$O-+z;cb?5av)iG9x~n7 zu&SA}4r%x|y+qZp)K>W%Y>M_+^l*3_)9(@7+t-~p4~zw7`&_5s`wZMcyIwoXS8JJNvoI{?!NhA1M#0HKu1w zFi>~+VO~2G>-}ll*gV%KiV>(rP3keQyAsUe_yAsGVHCyhX&riQzUhHzoegbXZJAqu z4T9{#mVcfT;dsRc>#c5`k0u>twoL(|T-ROmX)DEEk!hdf`4hB`q1(Ut3-4d-y45h` zjf$@zD2i0)>Obr7?W{-$McVU5w3vF`(k{z>*VvLYw)E?Jj*bxjiJZKIX9y@Tuh8J(P#!pX#*A$xc zG~FUnIX-PSe|~O<%d=>jFi~eSb3*eLMh-NX@O$hqquMavQk9r*_lW^TTT@wWh~Cy+ z4^?TNwXLZG_((D^tC?MXT*rI^l3I`zQ2{tIjIcI8(tt`1 z2!G-g(w?{d1Gs&g#|MH|Dq!rz^VgPG{0Gm`MN9 zXt!6^4O>2TIO2}QJ1cdAke}4fI#SPGlD9(`G2<$b|GHgPL*NVT_N)SqOxua6*vmeya~fBVt9>RwN|->_n6 zgX=lxo`PFp=UW&$Dp=tM*gbe)dNquKU@#CS{T$4@tt{nT2ta3kGkrry+ks7!64k0b zWbgtkumzT<(*UhyCqCF1K{AROC}}KwhUSvwa?Ko~!+7K#SSnf_p6b?bm#dIUX2EeH zr3UVG$5#JY@II*wAQrBAu&S6@QIbW~Fi27+xQBh+!_|bl@F>)~d(Km9bjEZ`vG?QA zC|T?5mh=$6ZnbhPV0Ns>&~TyGNYDUO6+h%po=4{EDlr^=K!MOo5WP+D6*zEzv8D#4 zMK#BERPc~>;D-98rEnl}@~3jXXLRmf>el$bfB=-XR~4o8aD5uP{5c_%y&CTj9hjCT zyEB!j0#mHe?J_|dl8c;=VY!_{WsycA5)*|g)CgvO?knkI?l0L(UhMFxOXOTW&tBiF zV=I(=97u{*-bAdaIU_*=^_jP%-xU=OV6Er=-l;PueAC;fQMW4;g=6lJBHlo_IT@l( z`cY34{?{jPTAj`QaG$`@C4;sca#^H+5xYo4Mqw!IAiv;16)_KP_lyUvD^7fF%pg@! zMEDeU4RX?r+(i^*IS*KXQB`E9LF@=dNB5m%(*JSQEAymn`I5io zngdz*>!#}T3D^BP;TGHk7P^A@+eb|hgBKUm*JOh+maayc%9nOO9`u*z?wg*H)(5bX zdZmpw++fc&xxndA0inSLoR;V!xRxth4cc3uX;z782GKR;I|~J~gi^?)ZFT@3 z{CZx3k5lX*!}HjChpWaukI=qCvgT=`ioG^)ueoW@FjVc-uPj8^)u|UoO(BmGr%Sw< z$ei}`x5#hxaURX{()1i3715AJYS(IlRq$(LL)Z2=D7!lqf z87OgeKmMzXno7(m9!+FD`okxc&QL7w@>+W&i47KcK!K~vGp15LdZ!$TFBt0v@*Bm` z?JnuLFB)hkPY7jeDDdwU_#;{5fBjN2$-4?>c^dXEu)NS4)Pu~!F-_0pW$fnrE#1Eu z9Ri@OP^}7;NWvz&ZT#vHlQm(}FNPBeZR_-(CEDBzBw;k;bByQM-I^)=0qt(e;jQ!vrs72$@sTO~%8-LlzP+0b`+=(BD3CYJCqlpLVbTqg8O zjVI}swriM3jmae)tWX% z%DRRRV`lf(v}OgM@u#>J$t(^<<+zhl$cCJmaHUqu7fm01}~(Sc5AZnSuUV-QhyUQ3xqY{#tPdl zE26(?h#i37T+m>A;64qpHdVxs>8iYIAPmNx!eld zP=SYKH#Q(K#lvbuQYS@d5{1eStE%a=N3W>0C0KQBEjzSrqxmta1D{XeCBBTsj51A; z%j^NKeo(qbCCnEyUIJ0WPr3PsfzEd87CItX`l>Y7l_;hzmvInQSKe^z=34F|t;XlQ zT3+9JtI>y6>Ps-7&AfP*)lENi%hT2*#!7ti)7R$$h|vyhbBp>b>u<)X??eA*$7jpm7TlB0WH<>uN=cAg9Axlm zaEpGd{Wj&<+9bC4R{)l0k~?sAnx#3$EQn3$oFb0{QdJaz}}&>E$1u|2HK=Y zaXj$v6*+4=PlE(K-Y$PE0ie|(sG$d)84rR2x5Tn|eO90T)`+u%BG}$ck(QT7ZK$BT z6ps!j-KHP?h_n+*rMdw)^F7#uLlHXgegsYPHH%tG=m3 zY-#Y9R=3Ug&zfxs1!H(;YKzPQ3<`xul|2Xy;##Hro=LJNTYV6=8;;cflZr>U8QiHMN%sRLCAS%HBwOytcXZ`^ zx@3c3pR{O-;xwdnr4-6G&9yGiL*k-^1_Q+J3NR9>)DS8@E}He(v+4fw6Im5 z!(5`9QkV5ny+9%7HB=EtgV5_T%1uj}#`qM{(G6#WvfjMp<_n-#F6r79EiZQXM)ubS7oZm2bgF^2Y) z3j2#{J61sL3lxWd;ECn_oRo$VwI2fuTDr4>CBP(*v1W>mtP)@dIdAD@#c{H5fqVbo zdA0`{(o&L<3P?6%Q6qGRaQV2Y# zl?T9Q#x4lWI~vgkI3#6hFD+`?v^{297B^%Gj(qnE&3}4Tq%i7Bk{+m8D(qPH*^x9y ze}r*pF$%R_5^S)Sx8h`3+A`$pAHxA=8Tk4Hdoe;}b@l9-{TkY-T5(eM+ahosZDcf= z@4X5i6q5$}u<5>`JNfb^3gwW(()6Ekb{;y4Pid3 zj&#PKrNo-rhY}W-o2W#1n*A(AeuI3AS<-x7#4@Vw*|Z1^`@VRrBKh1xs_GaF{*2b9 zG|f_>l53Ji_ssaCe&x2d?O0mAkEbpk-vnJ5ZXgx7c*#7n2`^+p3rwuu>;0Ue?=jS9 z(rM=U(~>T>jD#vY|8{6`J|NtuJx4fx?7lJ_Ywvk*A%+D#R1XtB{VG#V!QUg?#3k2R z)F3tHr?C>+v+}lBga z2L|eUAK**x$5kOLzh7cXv}d7z46nl{)4g-@8Q+L!NNN*xTn4l!t=-#SP^wYT$@0=g1*IMry3LFxVi0T3Epdq%_h}N(6J;=@Q_+ zzB~%#l10k-+$UFkH}} zW3lt(N8X^fD%*CdpORq~b|FE$w%7JY{d+3LyV$C+DYsx-0IGx)bNXv=621v;l39f7 zlAHDao6G7SjgN%G3~Ck2);VXiPZNZbx)Q->E`Md<56H({ar74mUVVpc9kDf|yJ*$; z9u-1IaEzpF_t9s@SEr?~1st{wv{N980josm$j#!!)4b#Mff zjdJ}ad@{tw!Xsp{zHI1@4m2UoI@Bokqs$HLlAl;mHnPCC2wq+pm1^ql4`C1xun+dT zi{CJMBj>~zdTfF}DZYdD4i)^+qt{rsFQSneBrZ5rf;M17)CF111xc7Q(i;U5M%YAq zkBe2M&Uz%GYik3fBJ3mxYL2{XL1y$WFmKvuwbJUu&5Dv=X~q)#x@Q)#BY!B37C1HL zn^cTqFE#~q4rsOg9ohN9h$tk#Tis)zI9*UJ&yE8t$7QH2h2)W%_>e3}8t%P~ z5^Ow%r*%E7YXnpz*@EOcbnzus0kX;mbT)XZ_+$Rb)8&xurz(~)&RHrl$u8-xY)LO_M;dm=5TIaYzLd+;?!Gsrya?TE zCz<0F;+x&)X(g82-%7qb02bCOA}?hdr7U$*@jtD#P~RxXiU-O+Y|MancvY4I7!IrPX$BW6`9g}60&~!Ai8RB3m8x~QRP@`e^0>1|KNX@ zrNE2R38Ei;RJN05Ui3b(;zBtgJPYDQ=m3%&F!M}4U0C*1Ui#lTX+V?6y$It|`)m7@ z8FSxAC$coouZUu%89Hu-?*SEP&h0|5B-UT*b7iSzvTCX=SLi5p#K$Z8JYhmPF+| zv%*2eEJnj)W;f~IFErgwl?%Eobf%6cUQ}H+(#PdrHejcq9)siW+SYsJ1*>@MW1{OZ zli)_q8F1O@s8L)LDl-WVba*2DjRZZvD6%0?G|EA<*o`LGH<4#Oy3FT_mca~$yZXW5Dz-2@_3NKPVL#!+VKF3U~ zEcbmA`n4}FO-o%fyb~u4w#yF`j6aaXsI-TYM6PB@sM9!+AglpVL(E$u1!_gj53H2 z#ZF7RNdL#Q7^nM&i^I2xIxUGuLG;fg36s6V6j$ky8OLZoX|Y%53Zs5( zOywq4O#jMXiyY#8p}NYOu?k<}qX1^nEW-`j?@V2WIFfCKlA_2skRl&4)9euuAZ4_V zMJ)|=9zqkXzxujqn0$v}#lh7n#r!(5;kHX(We$S~lioG_|CAXgI#{#@TRnCgQ5`fx zp=_2*pWKv?|4-uh@^Kui$Kx9{a zBI4g1(PJvbyG(`vu%XIP%=eR8)LxiiY5L*Z%;&@fwH_@Y4*aiXFzr7KtwehTm%7t=p>=IhVxcrfBsqe=f)KR& z!LsBrBpKVjeA#f#%)Ciyigb1~H1!R5Flupiw)cnW-bMV-xSBd5lUs5#&X4Xr?ukkr{_16XBatw@>V9j5`drl zF4xMZAYMzY3)AU-rURyGg=7lAQq#p##mTA9qdc(=XV~qS2y*^OP63SebQN-%=O)Lr zR>C+ZR^I$BXkD4_NJat`u6q!b?nMmAhK5)q>-!y7n!qF3iyMU>k|n+m8vWgZm3f*s zicnD`mGbFJ-sZOaT6T?c9OgT_v(}SU32-_Ijb*&OM}O4HnuAka1wZ{x6u;+RnC1@U zl8+yUJA52bvD7a|D4g4z1KGLLQ1e)#pSq;qS^z#;!|EfE4-6C+$D%&IaICsJowU; zo~iXyiCo|Vt0b?jjgwVpjd<6GQ86e5BjrSH&~M}%>TY))hdJo*qC#3 z{{VtUvqTzhmrQRG_h=u!UTlw6;PF*c{snh^J#%NJ0=azZQ}KLE%uStl$Rucv(m02I z|CHK^{IY~)6TotVlpd)(?2=W@kcQ(WYrE6dctdRi&VXBwTe)VYIH{%0 zc|Jf=Dfa;5ft>Y|Iht#9jQY;X43eAnU&b-E3i5;?Qbz$9H1x_BMVxx0bwj>Lxc^16 zh|mxvY=ro_EH82qTe-Xtdxg7r9NvZdU=^R}BT-)h7~Rdl82W?TDRSCR7iMZ&S!9O} zb!R1JOrv>1ceuD(l)#t`1iMpbdd}*u2TtPQ7VT$dOgNaVzp8;Lw!ukLue|*hywZP9 z$^xi7H~ABMz+6Q3LJ&5C=_+R1w{C*o|3uS9is_;p8xXKCQeb%S^jQ2Kzst8oUi(%{ z1L$wORXAu3<7kU}956)X@Q`xNws0BZ5(Cq+^8?qtzbxuXhgIRBGrh@SW1y#;MwqSP zK48iCXbjH8HfM5%SRr4%h}Gf6Q>FX zMH$+^fJ!|%^^o7~9lAbcOf7Q;1_6G^f98a7ictaA;O7i_jBi_P5%7s4u>Gq#+0 zh~;$;f_xrY9$G)TjKL*V7|W6lgq)D5jE)Lxzl8N`4#rC_q2mQAh0-iM9`P~ard6{F z1^wTcQMr5)W5N4{AWMgs+aNjT0y)dHEA^EbKWD3`egwwM0R0ZhMr4gXP7FoC2;Dnm zYS2(mfDZ4Q0b(z?Dk@{+f$~q{X>4QJ8RKJQbhv>ZGa9-2#s;{&Zx9fq)`QwBLBll$<<3PmebVh5`sk!bAuT6|)UdI8#q~+{2^> z@%oC9b^CIXDF|j%#X+RdU~$R{i;*y=tLfawG4ixl#DZ-H`2-kB?|f;#I~&K*uV`=m z7XoBQx8be)TH?yKLPHSGs^(WwYBr3>kq1sX)!PUIQ#J7ub=b;OK}^*q%82Wf$6oGy zL7YRzL*!*w;*JS*`rZv=4TLk%jw%??BFaH30BotZJoQHxYg=M$X>?=I9mWMB3h7ir z6`B^Cn3$L=vR|||E|T3z5)7#8yPGWmw@GC9FA8|4?La!0?|{PLUC0{dTB!8eI2Dyy zatRx^C#zNSBZQUpF8!m&I#lhpEI6SKPePBp~+hr=hk5IXOeh zh=UnQIP)PZp(`6IJ)>Q`sWZhvh&G3VOhEd89R6jxn6@!)v4Wm8H2*9mJ{7Q+$0ZiL ztXqwv&tx&C2HILQRWEURP^9#_DK_YDhxbgX1nd!@US=$JsJChuMMD8Iv zs6W<~wVwRw#ZY=g6mD6j$+#5%N-0jO!PT%Bu*hR!8M7~Xh)t$(QQm4G`Ara| zG9(WziCd^Nd0d2Jo3Mo-0);Cr0WsN*Orz(2?f$vc8?d+QKzNlcIy**8YZ--&`MFri zP$95iK+wrxS&5P#1SMP$x?GG$lZezz8;v*%LNXR!LQRk*LJoXclfXx(BWLxZNLO}h z8lIfU0!%e2gS=4j2p_vEG@R7IzGeDUG8aGOX@t1Bl%#WKwOsI1xPZIFVu%mR4z^Q< zINgj{@J;XV0Q>tbS{f1o#*iEmDMP{ta$-UX|3_&?@E0M_2v|76Gx4eHm9nDG5Y_!g ztx`Rp<_s`^IV+e#U6|-Lu&!JoDnjA4Oxe_D5*{YSU3+ujC|Bz+;YN@$I;>=cg*1ln z+vIa`#%gKD9vE9i6a9dN8t*8=XO2wdp~Z=3uKaz;>bqRP747IwBYQ-QC5UrweOlJ7 z8T1}zhZE}$UpJ1eUh@`6MxuGn>5k+@cR*VQcZ;E8{l+8kKQD@|{Ki2!RsDjpF*6~l zgT2bM6=MVeRZJlD)%8gH;pNJ6gV3TGlJq^eu>3=_mbHzEGmKvnM;TH+^y!81;hxgr z&x2Wxc3h(qzS!CH+r1H%tL+Fn)c9Rl zqQqHklt_KBUSVTPnvXTz00V>*l{OSlRgAym6D7y%p+*>fA{9T)6rvw*CSYx3f9~u%_D}vY|8(akSj4ySKK((90M@3H)slWLL{PVm`_c?oT z-bUgT@};Tq+Wl#K+l3$`V4#!qWhg_?=Cvry#FH7kv2xy7uo!gN;l1I+%E@uK9|T3= zzIzk+N=sDYc9m_4-&kz%K&af1MnELE)>sXp8SHxh&y$LM>k{)p3D?l>2+_8FFa#@- zs@yrfiivy4%(a7L0;#kn#N;G8rp5wQ;gL}G3oWEr23le0VG;3}*vnOLQES~FQg^Uk zF?SSkou)*7)DntI;N%K#lRx|6ZQ8V{9|APIhPI;*;$g+&*EnNcl?7bnpBv7n=*@$?K>Jllf4zDqCAKAzCk-ykoOb> zf#oigP8zleQ+kKSNV6tK=Aygcc>6yzTPhm*lGg?DYQt*NJnDWG8`K1V%y|VEc6eRr zg~r}j@WEGUYa;}1m&Yzx#9?0@zemV1|!BGg8Sygt5OPF z0o&6PUfZZWSn5zmb^ChzW5p=bpq656LlP=NMN2c$FAGW=@+R>Zz>NXi;CvvIEn^*=CSH}jT!p2 zXZ#{aQ&%KP<;o}E`;(`CI&Q8MVwsHVZY?7f$b=0<+b__3iFVbLoel-gxFyk<=R45aslJDW=gd~P;ciphh6SNc9pW-HtCoo4H1RnGI{P5~r~|=x4&8zNyIe5X zQ__sO9%$vs)wpF3*F)fU%tv+K(MWC91RaT=-2k%R;8a<%Yp!Jwg}}0)e;Y(efEp94 z@<8#K+yl5|>CC`qmja8!{T=Ft!yyEw?I(4C&_y9@EldZurs$xWja9s$*?Awcgi(Jh zeq?pn_#})DeLd|&L_J`ynYJU{iY5cU`WA;_UmwycZ$D~(Y&U{Vpro$IzLcG_Tnx&M zjdJjeHl}F=)3n5VJ(T7+?k3f+Ylg;r=kxz;RNpM+9?{X_t3`E_A*xgt~lx%KRiqN}jVChdcpi^Z%pltAgTc!YBzIT!JLHC&AsB0KpxC z%LI3KonXP;9fG^Ny9Rfg!QEx>+5Edz`?j^aRbBUc=%;&I?$_r#=QNg1)*lNzPh;29 zba#LVoUkS~5eDv^ZYb7pe+-9s<2Y4Y48KTMbeDHFM!d}n95N-@oEb&2?x@-SfU|K?(-Ja*6hLdO3=cAa%HJ6n=>EQ<6C81&+Ww zixX1qdTx$oZ6F@=n>>?ii58!#8;Os(28z}8jjd4te;F?H-VRrJ4$K#9>E_Fe`%}yl zPU17R0LKp&b7LVdWNF1a&tm7LgjkNNi(^3imd2+b>07JZY+BJvr9dOzF<(5M5i6Pp`hQf@hvo4Hob@H5bj8xQ|_>v-v1>Zbq;{tH}3o_Bh2EKjm zeHJ$J3<$+pW-Hc;7XBp1w?FDP!x^tl5SEG7@0Cd_`V#{yt=mcU%4JdY>o^k59;an5 ztZVyn0THe@QLVv>hrbRPK<=OD3*!w>)6{m|$(~ zzBl84Xeid)>5#(QFs-F3B^b$)#{4(*G}ff7PY^}2KgiXXXiD-h*&Pjc&__={(kPu` z)9t<^BFrVTXS{C@8YVT^DO>@tUfxwt41#J}E$OX8P8l;j%vMfm@6HHo0{JjE`o!;L z{c;ss)N;9p^Ovu4SYkJmX-mj0P zb@4ki(na&pQ?hBZUVMh=3Lj8=tJ0ag<(ele1%KonoxNN9QCol=zgy30v>WL2pK^;X&ZiLqawhK(eC%4O_F5t70_Ge7G#{erE@M zvZ((S%3nPe!8Fwo$@O9A$S`9f(iozo7}>vfaQ3su(x>h~PCVB*jJv+21%#3Kje)NQ zs7hT>DtRXQfa&%`yvO7;00+kbuw@V$OoX>}dG`P83=HEY&Ift}hhI+k&GPa+fiQbK z7fmyVzQ=G9RE|+B?{h0(vLn^ksifxlVqK7ptQ|WqcVIpx!>6Gu<{Nzbn7Xo{Cl9^c?)cn~fP!8$pJ<&Kw;23UXT8{|+VcVc)?9wT zU{Z*_4!H7tIlFK}n%k>~s;3x9@w6*iBQsOniWh^QTnG}c_SZXslKb~ z6Eea;nnb8n=C%YPXw4Ru#N)!Tzza#KMS6EWLg<;~eBof=2no4uZ%Vt_G;1k+#w&ZC3cPpw8p_k! zMb%7pfJuxBf9KJ#K7^0=-P>bP>V9{BM|HECm1cM4Hh+*%fLh#hKUgGV{h)aF#|kxj zxSt=In0!eoa6j=}$Vd=%C|}LdB0*hb=~x_0F?dSzMfLqIF$XACCs9Mr&VXsqruGg* zO)SavAY+i@q`Hfpx%# zlEi>-&ijW+N(&LjV&iD`Sc`s64^YR|cg!_>!av?vVstx1ZUm&hZsHg2%TF6nT;Su) z*K5O(GuOm(Vyt`?)4A9Tx-|k-gfPP)bmZH76SP2wTq^&O{Cs^}==C|n5MJ>XrX8Bk zA2wJtznB}o>;C6g5KpJR8%v1SWmzeT8}y6!WCb5ixh;P+#XXwJcDbAONQSuw+=`fY zlsqfXu^Usnn0}guL5`8w@n^XRMd5mt_jEbY&*L2)`5YwR18O?*^^TZGl;4ue>Z^sn zv%j1XK6knb#h7l`n3KJ_g=9q?J*)0<*~}Ag9`a;lSBLacCG+6E4n;ckxUg^|);T)1 za)($S=fs@w4i%8#itVuydYCH`u~mi{dgdY6#VAaB>UTRC3D>gD=}$YNZkjc|#IpTx ziXIm??u7cz|9m1bf&rA8eW+XvPu>Q`>bAPnyA)$*AJYw6`#mCD!l%6Oo9P;IXkn?L zv%{|BcwzwDcI&_zw5Wbsx>MAz@ND&KcEt73qund3=x6b53GEI|*>zrJzvuio33Q6u z0; zN=BppIBbU>Nd1Sa*6^^B<7Tvlrl`LL|-{qdlq(8hG1`<9z z7zQ9-f5JKT{%E3e6lQo9zECq=q6C%46a4j?^@yhHOpNn%=vsCN?q@Nr$_7V;K{<&? zVnscxFFA@s0v`+5YcHLZeQ9sKb7!tNN931CzmVjbJv@Csb<@F!uAZj2>_VibL{4Xu z<4t6^ge-cTz9AvtpAM{ec>bSsKZQ|Ax8rlaacQVu48bw+tql&gdEQ0GABRSp#X}SS zJ$D!xzH7s0ozdG#^j{rq+Xkn{nyc#16JM{-Hi@2fPwQuzR)5LHzOB|bLp`bke##B1a?>_5Nc$a} zIym)7q$3%=rFGQRYg*JDS19ob@3S-!eu(khZ9TKbF_KqY-7O8fz_-PUx!iHD#9{{- z$lgM?b)bqatER_`dNFxqWyD)r2NtQ%W?!6qEyCT$I%}-Ygi{q3o5gkM{F*lc_1l!| z=e2t%cuF>c$h+N&EHzIi#fq%+My8N>=?^yhXZYF-545cn-`k#F1UWG-%I zv{x}Me(01I@IP952|#i(&%TG+8E+%%s%dJ+{ROCgl969oj~009hO1idj8bxVV$yNqKFfOx;Dpd7JBoxcrXZZ2@%%ahTOJaIrO--NHOci^sUfxfyEqB`lL5{)Mi!u0*91Jun+jv4i3v#Cd_?y|>CAHY?$dxfCUvAA`qQ zcZolqxKJ@PU(ZXLX7USEJB(dwCb6ls-W&G$;Ux%(9mtmo$lPnJuV3*kaM1%py4#USE9+{hVe(|y?1M(salFEw z=@eaU`}G#%=*p!Xy@rJ}8vF8t_Q}JFoK>6i@}_!m=F;EqL}3Ytq^WGAbzbs4itKA- zGoR`?V*{$AB&;jkxF2zi2k0X%ZSw4rVyt}1o8|ALh?$juEs3wCC@Z=Oa|VkOMI&w8 zY!4M;(e?R776z^!+FiO7^1w+cm7!K_60qxJ^Q$BZR|d;#0qz(ObuY3x~r? zoL);$!75sr!v+WlQ%dxDH}I{ddvK(?eZPh0zRL?+Be{tJ|HFE1F8*Ol52XkQQBpO1 z=qLzy&NI{!3pNXy8{w+`T?2g|p5|=kaxZ;|rT+Q2_)Gal6-gqhM9!VzZyNDtfVsez zy{QX8$qaro&dv;E3PLiwjFLk)Em>ep7oE4hR?S{dU>w({fEg3i867Sb8bV= z9>Ne^g!2}p4%>XOSWMx+Og7<8!Sc-Y(L$U!?%@!efgB#xo*MLN*9Qf;pN3yqLzbhC z@(Qcrr$JO%`6=znP}bns1Pwn=!~HAr_?A12Byw;>uMAgFnXzKB0GMQ_$N4LvR&Wc6 zU2Y7rfo^w&Gr$EPu?ZS2ft`GLH`az6YuGB#Z$pU>@5sL@mr-Iip(LHwMKW_Ft)Rag z4bQXdm@;f36fc$7(xxK6`0)l!;ePg&A^WF3s(*kl$U2oK*?(y(IyzZmkFPF`N;>gb zFIJSUD{As7E>&m9r0;{1M_%W;El3X*V)aPi4Y4ly z{4uAX@^Xl$RYK%ey>$p=#m)5&NAxi|RHnt7V!*$|*N5eMh^Go?U)iau)2qS>GDbX7 zlyQsKZ|iS07b*^S5SJ&1 z5tVS@c}m*6^sgS~aC>i0A1tZei`lOc@A_jJzB7i}V^5X4A5&!-(J|y0j_ztt1(02oJzzBBV^)b{hMc9?#jWAbTHt5$gdKd=?f zk&bgr!_A{L`>!$!g`jOxV-&%1Pzed`pizSSZ!8Y6C^RPOI6F6TpUc2=*$SFgs-bQz z<&~21Tgu8-{M#w@A{J8hlykKk$`yOQS1#(1Gpo_zvHGqB}1^M9J1$Hrq}vm zicZCG9dJ=Lhaw&0@#jb*Fb!-T3Y8~2YPRtTK~ai*FL4^&Yp_?AD`b>5V!k5kVD^po z2y*S0X{IqQBL)ZGY)oVw)))CcPE8EYyGc1lVb?Ig@ZM7?8gIM2ka-%-3O|*}-F|MM z5k`#x`BpjdS`TVLY@wPqe%#CpH|_V6iC`raMZ+L6|^wRr*e>dOSPVAfjyAhywL8;=R<3&+gpw+NlDIegn_T+46__8l%-(pCI) zR34w$a0F!Xjf$vfBgJgUr~GzQrJ*d7g7|atGGX?Il}_>$=gx)CbY+yGNyhj4zm}=C z)P=0UsVnN!J4(5&`Zgui0OM1oHg5cnG4Im7zj^kw(D~5rmdfH3I2BBoT+thX9|MgM z3_DlcP{*_gDA<@sS6sG4ZOAvjob3%*?AGk4<$*vTvSYL>mBo{5Aqg}1M_V%?oVaQ~r5Fn@w9Z?Cfa>wyqehY(C9i@_0j z5o9|}zvQbTI^St2$Nm&)+&80b#`!kF1^W!QY7j~W&RL|3OIz?aN z^7YeKO!7jh>^LPDUW+aj?Y+60*po9l1-EnSMfO)%t@8?}e~$?K?w3jO&kmQv0TrfV z?o{EgWqou{_I+C~e(%_ZFE1Akr-Kh%>Y{#p=i+wi7FKUXz~7sh_z>ElNu#Zg(JIsC z^f}AAF(fI))MdMY{rQu0edBOg+1uY?cZrF+ z-qF7us06O%2Jil+il_Fu$J-$ouW;(QT`{cH2pc+bIJJ4>IDz8jF6x!4Yn#vwHVWiajEY-*^Pi}@+-%f6BVI5#yh6-G z;&oiYeq-ALx-{B=Csv5Ll9>3_vYDH<{H~QlG5>Kr^5OKt5aO0cLYBGquihg_Cmf>|;DYET?=J5^L_tSXnk#8rJRB?}aXxXThQB9eya~kLesWohrGi}M(Zqj$##5e@{;9;9w zXXn!vk330(_gWJ^^TwekcGYT=vI^`g$4bmm#(w4Uxi|jmbzlE_E0V=Q8b$Ak-e%=r z@Q>s3m{iEWRy5j=CoN)JoX@i()Sq2R_>#BKciDJYRmto9)b5P^(Bp;uhm4v;wg8%T z6W}W0vtqVS7UkbN8`6C4@vp4GHWO0iOx5ICtL^dYy)XG^k`Iu1i_DH=a|1!+KvC}I zlDW?&-K1*X4q9oDyapdn3iZeqD5Lf?* zK?nL8eI|JZqsC;nfC4=W*!*6H$>H5rQ^E|Ii-?3i>f1|q+%t#CD+a>&Sj}*zMXM)< zLcBtIi5<(i?a7NzX#@uJl?aLu^B&`+?PcJwgKzHx(&(`oC#63EtY@q_)$k>!x%KND z8?GhmNfl~qNRSM{nsrx{P2wV66k++S3RcOatnd<0A#V5nHKxNwH>$+`j@cpgJAn3= zjWr*_$H+^tt*XG%{_e~gaA+O%h2Z)Sw z8*Ce>)*VcWh0URnh*XYrA7z&;XD0|wRQgIZ)Oj&_lmY~ zM$*4J+862Pb}#TtgjV03&y;328ZPXX5)-aUuA;AhH(#K(2*@m-@bz9C4U+1?L!XtI zXQc1gg4NIA)-U0e_`Y|d+VYKlh?E<>*VfO822LPhz+>j+?{%i;om?j;;9wT4_eaW4 zSmuawK8VN1eJcEPLh%XuR9d3@{#at{?;4rXZ?H|{G%{y(i9;P? zv38hSo2oj{^m?klR|}0c{W4_T-S2k#+5o)uQ2LGC_!Tph2*X5L9(FCB!1ykZ9-uv4 z8<>?0^w<<3=*U`jLDnBCpiXxxiR2{OQXEOF1^^K*I;$c5SB<_Y8Sc`ETiFKkyi%`_ z%^H^c+Ar%B$$VUKS4D1kcNnn(G&qPA#X8>~whTMp` zcQ(5gf{PUydL6!tIZ<>-RG7z^r8dwHUgWm_HWzUA`m$I!QLS~aZnA@b;_IaZpNl-o z&Jc@A80`oyNw>qrjX}X=1&#Arh`yr@hT_;(R|SBh8PUgNOynBuGKYvuKExv+BH2_R z;CWFZiK`v{q8-Anw#o1*`x?GNUTqHENFQ5Z*X2ED)Uo#MK4ddSMn-w(OfEH8P?@eq z{YPb#mTcDdfWTw5*nr(%$DtQWNL~s709QsTyVHFtR%19*l}#aNiMf|S@#7E2dASx( z()s_g-JjM`yD|dnBxiK|k`bFpEHv(NgV{Yg=KMQ6n5vYv`uHQB+rPg8LUuJiaTMW) z4fIT?9v<4?Fz?6?w6d;Y@_nEm$MFw@{Z`qU*;^-&KHGNifMj?iT6BbOMF#|I`=j>n zuh3(TEHL$PbZ6HlV^i*|!S%_h4(*&fOnNsB3Sae^n0Um;W}FoLQb)>$nuy(5EKw3j z_8B|B`vC*>B0$Xy9XH3CNxu=Wd>xigOHNl(5L+uiFfaa4trVk+KZe4tcKuc0BfY?L zWa&B%KGlYSmRR&1GcHoxhOjp#%%yrOB26=&T_6PgA76j~qmEd48Jtc^$@ptkSZxey zNA$I;ztM-{=YwFZHR4))wbTWu5uzgP+NL?9M{}Jv2~YqE2bJpR5|rX)n5JfwcC@Fh zI0vXU`(ih(KW@G>4If*%`}^l@f7Br(RM9Q7yzhlo)Gc0N{?qvXulWCUwE!Y9+*ulx&51l=FeVxr8a%njEb{ku zBqXGF?~wFgo79z+aW21!ihdLALD2q^3qkP@4mKpwS~fN9w*67*R%oSS_|wpE>nYV=TimX-h%ybrSq!yb1&^Nz@)qxtpp8LA$Hl zS`6;N?yu}8F*2};7C`u$k0?k%h08ZXh0NB@#(4o|_jF+iYcJVOQvhGs!k$aJyAIr8 zz8X@bw}TfKuNNmQPpeJroBCu|ZzcI)1^imOXEziB(BTvyG#}=xB}JO>=25Y|k~1&^ zB44w(nE_oq`T0Ev9YA_s=iZ=>WLl2C25H^|wfHotAACi?PZx91FKeK;otQUOlAVsH z(-K(r+V7UB!PA21H^QN|i)Vr3?M#nS^_lOA^A__^#|>bPMjYc==uLgMp8?AL&+(h) z#W?T)w4FVGj_)EeL*L?;sdJZEt1dIj!G;S!@9*)$4(3KY5!)=IcKU(^$J0 zOvbPhS?Mf;}rOjUwy9 z*GDfDm^U<45=@EGJKTka_{#FhTAgEgcLKC$C3zj3@{y-smRovISX6hCR{73B0KxOJ zaDlzl)@TYkJNG;kk%6artWECyJ8JB$;jX9goHv#?z$ry`#X#IL_)f@58t71H5kh+T zK?osQHRm)bQcAeTw2;VO+;XG0)8RD8^kds_^(BHwfM4^mps}L~lpC>Y?Feb~7bHTY}DlL=7pylnW`;{?UAPK9j z$L#XU*4Eg18d1q;LgpVbV>#_X};A)Q=(Zaw zWVYUzlEHQ%8luar(b6b`M)z!l7*(Y|O4oAaI>^lqu4ccL>_47B{bQi_4C*)9nD+V&Ub+1z}-oE7X{#e=ApU5%?0SS6>ZW$ z+uZ8iaGhpPXDfNeqrc*1aGVEoB?|N~=V0_k-3I!^RGYizokIoif#^+aQFiRLwm!xq zcsl0dawWL{kY_Ke#w3e7)zxY9G2Y>D4`kz8{G48o7Ka!^R+4 zBkDXK*ikyO$zFOV&lG;DuYMW4qd-AJ&j&2weaa)mUJUmhmHM0Zq8S}>9}jHWs{ARS zDZa~*plfMYhJdnaV&I!fbH&D^mAw6&%ixnfwdsoMo4W)g$Xmu4nUKglN#^OL+&-f2viyYVJC%f4m4mF!Bf@^6i; zy(YGI>Y7gIx!-v5CnE3nV00bHl)*Z&U z++%Fjq_Q>|v+KM%-Y|pp685MU)nkAAS8G5Z83lsiHct^<=;}n)g%oXRuY)4l&{>O& z-a2{%=^*mH?IlI!9re~879iAz@LFOY3jwY?k7dt2Rye!YK6i1?KY%hxk(jANGP9;aF>! z+x_P(E1#rpJ?VJ6!x)x@(vBjQefO6RV5-gN}lyMA#z22$JTA?zE zxqjVwYuEpnrL|c}ms4*R0%(sg8ZS#(w51OVpnFIfQOL_)4;A5HxaA>=NK(f5m_TUp8qQCd(MmCTo@T1i((g!&}efq3l{J|=@mFFzv zZl?9cR9G}>);pH|IFx9^wg0Krf$TTE?1AidN24%bkGR9TK;4T#1fe#9m{vdejF&3c zYO`LS9HR*S#H%gLSlagRdSUGsc&b4N6tNp-q?P>N&rL2a=YY7d_Gh64&60DZ13*EIU*}2YRtX?ZIg#&c`iV^NTr>5c@ zx4o@8i|=Y!rQa5RV}Z+irRD^=Jv$d7?$2t9|K50|LwETd&&Q^RhlCF9YBYVhi#iC` z_wKaMx|G*r_Gn+eT?{Bqc;+AhXUPpV^QxcSQrmmrJ5{G{b3}P75RMYu91toXueMyd)+5wm9@l zcnay~Q0nW63qGRjFab8KN0S)-SbTh#RPS@uF8Lim-MJ#o?5-wS0wu;Orfb+K+R?px zqJQ_VQ1T+D{`_;l&H=ZlAvq)h#8f$0LbgaW@P2=nQzJj8DeH&0aLTraHTl5R#zli- zwV3&Y^+0%X2?DW-X3bygu@{5Ke*-g!ewP9^B`>M=wno|)5>(X$&yUP^(L4DTD~2`K z*iBudU+6!*1iS9}6!9qTLG?uNDO>}=49CHq5RHl(7~w9cV5sUZe<#DmA0Daek5L?0 z{GB7seLj~XFI~9~9UTTNgMsFk{wZ!+N_r&EX|)Dv48#Y%bOtTuzrFN4DHJ&D>B(LC zVcMS6(v^M!$S)%od)I=Q4JllJDy0g1;{1>o*pKwJHVCEb_zC&LcTCWyC5`poeeG>}vo*U6OFrg6nV5s?AS0xrm{su_2E>hpb;3U$$1N#B`XSdn*2;*gwyI5R1Bzf82 z&wiV;>PwIn-s{m}-KD+3(V+)fr7!KH8&APfY<=l>(wAa9$Q}wCrAc-UpopI5vg`5X zOIAoWQg@dC%9A2v?)QF~J9LlwtN!bwuyV*0y0tqCP^t27Vfr6fuwwvpkF`;ug*T4X z`ghU}DD*4~#g(t|O%pi0bmi(p_CLc5#W3id5IWphO0_e!%!grJN*vVhi0UD^$2g2Q zp#-?CtJ$Tfi17ZQp~U{7-)huR2hdDRcH6`yov@lY^Gnp`OGn}u<3_x5Ji*%MN_?Va zZC;1xo`5d!g}3l~L!IAiEm8lw!CtirsBDZ;Y4)judNS8CgJRc&>OIY@Nl4U4(mL^- zz-tmP;kw#a+`hrhoV7Ins4U^(LvJBY3Y+ZZ7dxID8@~qj`bLc-c9SOZvkm1I=2P&> zX1MRm>IKiet-b9{$SI3$Bs#PoT>iIsz}6}I;CcSlM^lgN`UOvHY0*G7CgW9qyA^U$ zEX+zU^^r8R&;fR89MWmG&C|oSN{X35b8_Eq?QHivReFGH8n{euTWP-Iy`8EQD~)AALySqjlR2*NmKt$`DkCb+oK_TMX z_q>)@P#|DHV(VYl{e!)eU{cBM2ETUYM*KL{|8McEc?*DOa!u6O=HQ&4HD`cVZ2}UA z&4sFYGJDq~#SnHgX73Rw#Rl1c_T7y?O3n>*#Pj4|>&qm_U4dlv4knhr8#8ACKR3zh zAzi@(I2R^+vQffLe))v%Y2S%P!{|NqC4NS-EQap7)kOB3K|W32!Ru zYdMdaUFCgDExQ+UiSmsnGkf~FjC^ED*>Rjr6Ls+ifRFx&a@5&-CHfGhPBKuV&a|t( z7O%18#+qGXYz4=RDL0(8UO)eQ`0c6}xAR>F;}H^=bAk6$=*-LQr>jI&>2G~}0qana13Xyc;Vb=CuG1|J|%G$yNEl%rd=SBV~Z=L=A= zc#*vwga>AieryP^bzsOb-_s~%Z5%7tIE*`t5ox}0ln3&=`2^sMSV;t#hU8&TRetLsh(-kBUH z2nO8MdvQ(Oc5Z<@O1qo?Kxb-=@{<|wXpc)Dn=0;-9`wJ1b1A0#%eJ2|IpP(b@QLRI z2LTdTq!hzr})+f@@XmG z;>2tQ^N?_KvW_zB{JZ+?iXD~BLnLyDD{GY&0^nBtijm({! z{2S8$>%~3m|Die0-hVX-hq`@B286?9LwPE78-@mi1F58S#dX+sCKL8m#6HkbQ-?-T zp2y*cj^L&gRFztPKXjy(ip5b&ndipT$!FP$TV%`nxbvFnnf*vI^k^#le7)g)*0_0B z)d;PvNC7m@-NR6=tri(A$0o}c=oBgPMhO_WYp_`>tg%8Om%+z z**`ke6|YrDw5p&*(&}?eseX&*bJo`%pdhtbbKZkjjoD}2yXGORY)h+w=oAIk_fG5CUEc>v|vG$ty>1-W|XHykoW2dnA5(XT#vN-CR^o zSHGt;_ffEEnb8wn=oVj59&TrCj*g~Q?njSq(xp~Zmeho1} z6vA&axPir0Iivis$KfME=|-$qVt?`6`n8Mk;C3nE3&?!k-?b3>xT!1UW4s5i^r9`xv!^VsS3q0PV^sPyjyI$4gbVIDhAEN zu_KaZQCbwDcwDfTg;TU)3K1KOQx zu5s$09r?I>cdnwl8mOh+(eyk@cyL}IKK&f#y_7<71j=~!IBDu-$lOAcvNfbKR!b|o zNd${0GzGJtzG8a6z1u|3%f9N{{OvmVNArD$J7W9Op%6WZFF2r?$lsdvT1TX81?y_hem$kiK@v3@E8Fct?L zNg+Dga(xs(jOfKK_T7*h$OZGQz_^H1Va{H}rgcwVQrX28-FVdFT(747b~C(iU_6~> zoV&G6B2!HK3mzawJVyOc*#Bvt{@{rmgfaMrAC_OT5pTG1oKx^(BHI%vA<{)hnp$~l zE+KcH=eG_ITn@V!hSW;7EoKCQ=Jf9b?~funr(%b*YyPZHLM(&G!i!}Vi_aN}SmeoGrghzTJ8PWrPv^6}PLN*`5U}M7vtJZjcw;P~?=HxC zKsa|6>Vx&3Bm5;a6Bzel{xQ@aV@$!F^M-_P(S!E>rB=%p4cE3kbXX!@h;`4F>T!b59wX;3p`BJSCpPe)iaAppYyu1^&YGvoBcaRa zSqnSN8ht$gR;Z=ikV7&>CpZT>DO`UiwAo^KfU)dY`^`gwfHy%c^8J2iS0JRF1rwBd zcmOS=85v~(%RdBR6n|q2X0mOD=Jxj9z&6Nt8zL^~K|JynSR$vh9b$1?9gw41Gxzm6 zX+iO6aID&l=XtGf9-R5KhsH;N3_L}!XNEjdZ(q>z$vAB&_4?xYX^jdTlr9OCKw)8^ zr(AR4tMIVG^VOK!sDutUtS9I5U&^E0hqYKfx9<)jLPOjxdx=kl6TFttVsG8cK!R?5 z{u6q`T>}zz*JWRVi`~Y%_th>B6g&HU2Qb_>#Cu@$mpJdLN}4Nzh$}hKg@w8n-9bf@ zub3}cf!+UlsZNbkgcaxm8h|q)Q%culC&q#pLB=UX3Z%{Yb>lX16*F@*K1Kp6FcQXK}ycU+07D?!bpL$xL+uaE2hn zh{m%YC9`Yl4gCiSthq};D%unTysw_<t>23R1k%BxaUL(%qw^G?*npM;pDr5pr zmtU$&eFZhT_E6jOF5)YIIyj`s>BqlgNuLHuKBdX8o_I8SMt0@|#GsiF9Vc8t=)!!{ z`4?CqU1kW_RnIWz<1&Lw1>w`1FVNE8r$k8h0JjZ5tNl6(uV4K<7DLR&Kox+}cKM9M ziO3aFt1`Y+@7%%(6y|AY_>xg_(+UkPgdRRzgL+6E;tk9=0 zb7Aj&E;00?3$d)2o{3o>p1*i;)B1JgXUq2)2yQhxIlSO|edy!tdf?Z*%jJ)xV7gYo zx@qe-#`avY8hJBa-f}*ltL!yo-o6#y{s-ppvi2yEo^b82tBeFf%#j}ZjN7_&LkwEy zOZ^gP&mM#$y~+fiv9T$_qF+n&(2llxH-Lqo&_~?~`trWeW92x3Y=d7}4>Rczgn8?yhIccXEdguv z+VI$kxc1y0ArsU9b4bdMOWknN_8+eco(w)Yo-?}#d&~Lkcg}c^(65j6Kcg1EJGrUCeQSY4 z+bt*;@Brf!mdhcTV|Pumwji&9%7U-Vedtd%v5b$$4iv(yQ%&;b5tQBQ#aeIF0~Beo2q6OpDJA!J^Ook2;~ZQB7`tz- zTrgN{wqF{PpR^r^kFBGVUD|wdb>+VFAp*tZPe!S{aW`(obYE|3Q*?d)<5v6 z$tgXq#(op;r#lEo`TOCCGZad4S&GD>|3L00?Q}|d291Qu@xI_olP%}Ge2NzS-L|O) z*%*hoqaNrCz4B)8HTZi|Tx;(m`$CbxG1Kk23rKHzk{v*Vn6=iJCL^%tzzV^`1y+Y$ zvBBCzr!&`s+yMESR&k~mt>#672I9j}@GCTd$adf_(%G)LKoe48A}CDuiX2EgRz;ywC3Tp!cLNjoHv| zyvIR?11jt-HF2y~hB>vF`ACiqN$xPJk*G`iqPJq-j}}jxAup|Rc)jRYVE1s#!?=f*5O0Gp61MB?tA?wZlNrX}a zB79KS`r29#^zKlJ{(Sf-Ye6U652%Xb1J81?4FjeExdD`?d)1xj~76`{egZ1Kh>!3V;@jQ$o;B8>6)anuTf$) zXS3k7WJAop_1<}=+dSjaB>(a(yxq$U6tSBu!=WFgX!pSB z6sU}e>tcQ{8(q2Jx34-%@tGxqWcukwqObg8pP_o65;kf-S^?PgJJ_3~@qDesnL52u zB>KVXwlm5m+2EeakmO;L{OI-jTagj3{?oo?ih)Yg?xn`)+{n%~-sp@qE$EQQ?0*1S zK&8J@r_)XHcS}%uNoB%ue&P9@LOvhiLChV=y{+c$)DaxbW>X1CJUvfG7PaFPG zm*RcGThz5+zxFR@={rAlO{$kX8{T}=CO?Ld&mQ@0uY|NexLxR@-hTP*bOxmFyf3-! zpzJ1cP7`ru)ydv)&Re56oZobw4Gm$3^Kb9g`h}qMn}v1-{`PLwdXME9cCNwq&^F=L zZ-wWPk5vi>4(q!an}uV1TJL}l7s=nd_%VK;)+g&+tK|F?^|{S&kbVUHw1_@lw?lG( zzFdgi5&6Q=>16#&9^yhWzcdkRSpY>9Bu>?PxzSny94#|yP z+3gaaEL{C*_sWi?5@#3nj3f_>>CgNP+D}p3*2+0gFdv?z-&4}H-iMs}wd6^;!bswe zr@sm{7oCmFp&NV`u+Oxee#4@)BWX~qjS97^v{)FeFgcc`z_BeIygl7-a&k@YIAK*nM zb-pQmJE{5Q%;3K1WDX}&$L;u=Kp&kX?`PI)eHgr3%e?j}E*nn#8RW5_KADIAv$!Xx zb05y+ym0D0pZRx%e#yb#PU4BBAHAHL!TsqJOV5wK17A+SV-It7Kzy_B$bSxbt`&bQ znK+&0!XWosAYJIik9GL*S4jBMUad!Q?n>tUD&^f)s=jybbL`NkReb+3^@}C1gNn;) z$Wt7AdW|^{hMx)Kb@?XYS@ItbKhM$6v(c;Nz6j%9hyoWAPqp&sL?M4ppY(Iz`LWxp z{;;X^)Adr}2Ikm2>iJes_GdBAs+kAF>H8r49=$;5E0*2W%&$L)|C0K|7U8d7?-pl0 z{7!_=Nxb)d)IDZL@b5DI{!SfM^Df9Ej}h>rP5H9s!;(`%`aZzBrh<3WG~RV7+}m&N zlHIY?zrRvAB1`)nd>6&}qwvdDEjf!h(T+a_%!xsrZ*knCN2%un@Ti=3+y>^v1@iY9 z^Xfe3jpX}Q`eh1p$EFEOc~3q@9_yL!6Eq*4M~LT!=C?gX{T_n1D|jb-0Z*f;cOw52 zYB?{5zD>j4Sp83N@~Q6&Obgh43n7v-Ccq-89icn;RiVA{yLPSjhR8#~NR3z~O(G<|M@D*MWNXZt= zk=ZytzdLKiKYN{X&pr3-y`TN;efEoG-TwRXq~m|WTE6p+G|!Z2+%{iw=~2mng~fr% z&R@KnWozBIW5svoJ@!j3acRD{>(1D;EYV;tM{x{8zdi-zs*kc zZFxxYa)IWjmS~;NNzI46EBPPvtw;7aD7jAGH{FXhUe5U&_iDZddtWQp_>qtFJPAMU z!A_eF==rISG(IXES%aLCt8BleNmk9)`@<`3{WX6jRpUYU<0;+?0*@i9H1FgbCEFz* z07s+1xhHlzBfpyj{1JdX0*IS7g_1|G$Ahw?bpdKkr4~f z2ffOjk||kwJ`1~DIHva}Xq{;X?slO6P2%^DO1&2?JD9PYYs6+r_j$a}_Y1*=7yCN! zPZahlOP1^)KbgF8>>%8j-+4dspvIF}7t1+bu8C;9 z*T&I%+NT;md=LIOi`^f_ehrFC(*=87$KRFG-xPwkKDM7XXudV}+sAp`%8Ns9|9SnL zxb7w1HSLI>dE}94@O>)r)Q-F`pZt?cyvGSA)>ZVaiPv*Cr^dr|F4GhH^n6z`f0y3- z7~J@*ksOP@2Zciu%{~jow^rmPJuf8w^Y$=Czh>lzNA#ZAth&L(68E$5>ojoQO8&Gu z?9zM%`(EW-)_l!}lb?nbX`HTnVVzeUV&c(n&j*rO3nl$?^j-*dEK^)q1L0SXGJ9UK zK3B2?-kuL1xGP@Fdi3g!Uz@=DzkuIH;_eUfVPE__0Ni!qJAr%eNOOdBiEAb6;QuV@ z5?|TVjEApBk{5qvzpLbvJmSQi{Pi>Ol1!dl0j{z_^t|T^^arosAQ!=()&R-kshV#9 zk7;~&r|KhX5B|zTk9pJ`i#e~yPR(Dk>kjz9u6vroOWhdTeEu=>%fajKcKoP5GZVp2 zU-JJK;5?J_-$3tC*x?~?7^w4`)x2*hPOY)vEen4i0;m4U3uY(ya^rlXw`$yJot}3F z2MgeVyRla>Ja?G$hPRTa#$X3Qnh<6{}zmli(x?_*vNXf3~=Y=2b`k^2ElZ;(DlTQaF>G?bOaXxwD zQR?-SEs~YgXP4lwYQ>B3QhqjLbPm(hrFGNbk#DHG7Gakf>XUQu)^yI-L~&-l&blSM z7lwYN*s%|IxB>qs@Lr?t8`f!fXA?Ltf*&l^$!00vErOSuqt8_3yW9Ce>pMF&e=$Yl zZmc`NJu8@dRyMpb8=eXPmko*oGmdz_M*J0_&qeB#3UK~2cyxqm+ydU3gnhE2wSI_j zVItB06Y$%c{I~^P@q_2OGk+yRvSh2?-_H3XW=ZA}hX*9yENWL zJ(9AMGX7{u-H`%+z6D-=xW_(0o#?wl z^T8n+k0Rfj=e6HR-g{l~X?Ed{dd0VK17}m=k9y+GH%BrI9taS>oA20fE%kO2wD+nyU(#-GkC&cp9cK?8hl>P`;El;a>b{4f$!vT-h6PK3eLx1-`()(t31!5 zKF^|#cH=xX@sioPPnfpY-7!(~U5VR@=?`8vIwgHo z=b2^Td!|e4o0I1bfs1{dFM+!E0qTu-@LV$k{kR{@h37Nj&!>q0v)t2?*uN)r{(kV> z8hd+k-}eRg55T|AW4~;khZ6UBd^Z?oHLj{#CYy2CGcnldRfm%dczkFQIC$G zKA(dhegX%p!R_75+xK?|e7p@m#inaN#{|g)?#H=&Czv=p!#(pBb~p}?)*^$^@3in| zYQX6a)YCVJhZ)@e9>ZP-=u5;>|1{#~Y@Q#+eod(h|Dn3i?1g7Du+u{PoJ5`4m-UhO z_Z)V;ApaRJ;(Y*hWVP;@Rt9-?L7vt>4{yx^A1#QJMC>t@dO3o=#-GG%gw~lL&N&x9 z-OKZ2_%oR2^&2HeW4}P2TZtNn;MbOlW3wKf9Yeev#Sf14lB@80D{!)p_+LvNA4>f@ z7JULWHuL%ZSMW_1eA1kHzl-vec?A5l72eGnc+i)8F$aH?6EA+~T~FRhfQQF0@0cSw zh&nJ7`>sLHf$&ra@!e?muh2gl9P9u$v35K&{u=*ZXa7cUGEw=)L}Bj}*g2g1Hx9kJ z(Z8f`VMT+}Jnkp;#8cz;Aw(*0&&zYji)hlEhD@0{?r`{|bSp+fkn_vio=Vvy?m+gMD1g z8(*z2LT@knX9m4SkWXsizfJT#VzFaC@Ysp`5{5sHa6fm)j`{koiGiPIfP=a4X?%|M z{SjRJ6TRKHN&Wy&y$ByQL!Zmk(-Fk~cIw+myN^tN;Tie~PW92O0KPLzb~lrWw{MtF zpdN`=AKbJ@-$e4_EaLPW=X7LhT_O4n*`oR7)V+RA&G$#2be{X;_x+r=KYfOt;NPD< zsAHUDC*4C$3-EOff0wiV8ufD-_nuwgw+>#nuwyiOUZmd}!@5fD89g}H6!c8VkQ@)s zyhdDaq#p6(KJbN0^EWdkbI4~K;G1vhE3OBB1?25&@R7$k95EXALa!wDZNy)1a()9| zE!ou^0Jj;OFN*pv0bI@mC&#gqKmNEy-suQWq{E+mh_f@qKW`or6?nVSOwF$RARBv1Vdzs{$A7{-0zd+J4h`0}Rwe*4}o>4BaVd=hru!Fhj&=bq+!C%{(~zjJk3 zE*T2Hj3ZuBz{?x-osJ2=W+%9gME^qWBYz|B<`Z`rd~YKD^no`gQAgDhf3Jc+Kjv?t z|3uD7U&XAZez-zBc+o$YMBb}|FRrYVya)Wmkw4P0V8)C#Y~5v(!iw``&7xl zmjD0zM4E3x-P4czb|m$>C->yvxNlT}i&?}+1pFCKeyTyQEl!Q+($7gI56|Ns>jZbP z`n|y_BM*CV|F-MwXmDZwzOxKJ{FI|{823!OZ`qIa-`o5O&$WkVU*ue#*yjV{F~!kaJ>M$+(Vu}3IEQ)js^5#*U`sW1Ml~y z&Z@@WZ-VbO;A$SY%cDL?=rddEtPb4&YQcv^U#&fO*a;4! zsQ*IZi#_4*i

    b9O{w&^2BYT~@HLwFDdoJ) z!SfmX_9VY!d@!z%iLQ&W1oA$u@k?2>eT!v@VEW{3+K64 z*uOt|6W3R6jx+a(X!m-1%;!kr9yj=q~QPj6X zh#u!WqkFEM2hDE$+ptXYk79TG_lF|g@68gI#63H>H+DTA2>ycbPb_w>F`B=C-nXa+ho}!=?xL=!AP+xC9X>mU^N>f};rF|E|4Hzg z%>OARv41*tolM*vV846OzY=>-!EQm^^J>9YCHZa_dE`0fr{TX^>bJ|}{a)ZEjJRwi z{}@k~#v#-hPW)QM_lDZPGtggpiF&Jm^E#-1=20g_;XI6%GZnD!+&k8 z@_*h72JjA(S5Ll41K?TyZ~qSA_e1ZyTc|v`kf#rWxvF2?IQr-+a69!gGo_%f2D{QW z(A!I#U*u^shmbdBVDHES`%94T?#F&<0QrLvc(CeW%fWLV@Zvq}Oc}hIe%NICZ%&cF zdD@D;7qs!x&%V`v3Xz9-;qC(X@qX;KHNc}r(fgPq`%U?+YM$0LUryP*aAEgy3VE^Q zPU>y~)c^hT-+tizrm`D!h@Vuy-uvKt^dGn8_m=wllYC6V6aK%dx-B%pz08-5Q$I`c zl}|4Euc}jU2R!69JarPk2GBgyzL1J>X5aw(%YE!_Nfi-LdM&AMIJnJL(X_zS=V7`-*unK~?Zk z_Rv&l->;;B&Fq)0S3muAHv;8voM4jblO(yL411U6bwPI}jy*LyHFux&&V$O|!FVEh zdr+QX14;k@tNmZg!1n)tJ?sDf`+@jA5WffF??C(;i2nldKV0(l|8xEW023Af z!T-4n0Kxyc3jkIBfQZY;ghe;6I@ZjCO?V*@s+NQXFp+QwmXVchIT!2{4_lMxP*t(6 zHdlb)akKe!bSK)n@`PxH`%n zv<+*@7qM(+HB?dCMU@5^NXRi`3B2dn#dIyq2UL_^f}V+8;YyJz9ih!g5&_uwAwZ|w z)m1{z5Yli>N4v}~&zn_G&MU&i+v9x;J_^u382)*J{y6IV&n%Zz%WO1~wIzMu50@nQ z;$DKTf~1vt!{)c(&}sx^`NMmRb%asX(__Hs;W)U$=Zb@Fbsj=vItvD`LQT zV418sSFh|WyVmes*D#i;GBaz5%M*b-CP>Qzk`IQqdfEom0 z2CTehEPQu9%BaB|qP&1ks5Yl22YgB|%^r&4;ZS4|b#xAUY?b>|au|?W^3Od7EN*pJ z10^?KI;7!Hr7$3~jk@XQ92Ka-9XHSg)Xd=m+q`WH-i!|riqJUOZ{X5JuYCZ3GmWK+ z&hU#IC0}|c@tV;Qps(e8)b`k`bZWHhPxF`fepK%x;X50bCH9T;=M!pR-t})^^jCf14@4~mr`>hWt5(7xB1lUE%F#Y}sfnKUa?+jtQJX!I24&sJ(GfDW;09w}E3+q18YG(voF zq=8JUJ9@6NP?u?%(u2L&ivzR`aGm^0a(d{zYAk~n8?H9~Wxnq)feYgJN?>d)S~2t` z6LnDi8eD2$SjRG8a%V5uo(h($7lYC(6wSP}oRm2mNj#gnd#p#=aI)<0cn5A=@N`Rc zJ>6(sPA04F7_V4<9M28i?3+-Y@~=&^vCjZr$VT#l?ztcG@7oE0QA zQ|zsq3(>3SalpV_B0LXphHV_L7N>IECK1jl4k*ho*`DvsG=}|#FxDD$FO_;{deEnY zdQ@rDPxYX96B<5^738~@Dp2c$zp@0eEo3D{O8z?-5}f>0a^~$I)z3SvM%>o1ZjoV1 za_Bt@e@;M#@MN^(_fj|nDo6nu-SrVuKg-dvQ~i0%@}$qKM|o#m!z z;stG1aD8zJ<2Vui?$)M~i5JS-E9@p#2kro7`a(RA8~1HGGn>3&du^$&PHz^-8%gIT z!RyPjZ|P^TCG}8nYAUtn_~8D-;x=fbP? zsSor&xyvzC2SLqrWuONcL5+0~i$b#W7=L3&x4M?xliD5-F{b4U8Af#{QDj=1UV?X3 zKxuqfFANaw!65^s%!=w@=a`-!d6u@Dwqg3hRy{bmMqBOZ8Ap|u;@7^E?s0i)u2HVF z@%aPt8mu! zk5U?J5i@dhh?@#6i50(0jzvean3|b2NYilHFb|xhMCL+EDWlaZ=qT3&@`N8ljX;?e z0zrI~7x7XQk`%!^g8>w!nL9uDaVXM|Eu-c;ocSjzcG0_*94ki8z zYCkIA?-!1{shI4b`FSaIYoXZ5F`0&rI-U{SmJQlx+x#Syr~NR_O$1ymnb>FCk4?)X zmLihI01R;4k@;)lsUedS<{%`lUl=sifM7fUn075!W;#=smg6Pa3s?9O%0Ids)5R0{ zCEu`&MXnoQCd32}CkGXtw`UImo#U1u3SqH3H!m(ME30`0z2Da78A>-ddxW(KA~TkY z;KhGpsya7jiCk`@ULub9FDfhIz!t|&MjW$duxM_8I%rC^q@!65I7dfqo;D`HN~tuz z^Uh=%WIE7B9_F+@Y6eG^(m3FFDg8GWUhE@!ZT2Dx>1!P~#CrZqTwN&KDvl{4^z8uU z=!Z}`K}juq58Pkcg0IaFBo8nY)^D_?dZ4PtM!Td^t3alzj(SF`$?7+yLPT*N_P(Qo z_S4EpK30$YavgecusB8&o0joR)*WZr@Qa-NGL|XodQLsF`YmTpu(Um^a~v`aveJ%* zm^i^(S=Z(t$y$N8AI8-eA5}JzV(i#yH;T+|{ZSA|!@fimgB$xuLn|4*189PkEAErRVRls`V+v^b`4MgymT1?--9^&I%3-0%aHeF!BmyRq@X9 zMM%UkNCPuT!5a*av?*Z3%4c$mYQDR8lFbCnN?C3MTbZa&;-`S5mafs|k8&byG-NO>p60)g`*9AoF)j+& zB2V9)gcH45qh?*s`^hl@(=kn5I}+8J-|1lVXT&pTxH=*zZar#307so?PK(jr$HQY- z`XS<86p5jF8Xd}chM@|IY~Y-ZV@3kZV6bHbB*@dN2hBqS9^_(NP?i=fS?*2DjluLm zlcIA~1g0>Y*EXRYRje42+6i#~>QyH@6EA<)Ir(L_&{21insr83{dK^aUtOCdi1QH7Bg!sX#un+Rq zfUz?}8Mztk{0=588BI_=mFxI`vR|_RE1}YUGu(%+OdEEQ&i-)O*(*!9F&(auC`Z*! z4h*VhQnSrTv!anVu$T&tV16D_aJ~8qXC6TWX_mJcJUNjMdUX6{(=D}OQZy*1lOp?) zw(=Qow&Wm#*-ZyFX)v#;v~%mdayQKa+-_2EColURY2qZ@g+(1-TD3I<$KM&SfJW!- z`b#ma}0QAhG=~OhiBG{&%rE}6Pk>bNJM?Q1q1QF}3F&_vpA%=VN z>I36@G%@hO|M3I>!H4k#00{~JNvCa)b50bo%_cJQL?bs)>T+KjN_E24*wF4#n#91* za~6B025OAE9}aEI={`1Ouq56U!WTBe4Q;MiEwH?xXvJoo(B>ufyp(0x3t}8;nCNNJ zY09OJ8yTi5!eLaG7EQ{-9xHlJHX2v8i&d)?No-shRh5}*IY})GLdI8#VS>I!#T41X z(tV?arl(9Ivbe^CCj&vzk|xO=*risbF;{(48CjUa?&{$fJuf7js%|KQGR}FGdtp*k z>Qff>rRt0(6LV(v-fOc=g&AmG5___+dYvZX(~==99c@vXS}d+)Ohlz^pY4n@VVL z-19Edfn!9S$%`8kwh+Qs8Nr5pWi>s`N|?m1xJ1<$wSRNwkb1Y10$MmyA@mJmZBuwWXqo7Sc_i<|ReNS!GPJVcN<#rCa2Su5-j0 zi^U8(5R663z6%yW)|t&pwu?fflohkRBLzOC!H}v+Y{YSNUhF#LTS?7RmYZDiZEk5g zE~AviMr%E+mfo~`?;|oNli4Iauw-h2#*a(WO0n3IW~#|oxv-SsPi`QG*=LiIT9pwh z85K0pV@~wCWlfY*Vpv((iW)AwjVqTGYgp(k&9&zlx$=>V8ZyT#mZv?er0)l%tu#|- zTNJZtYt`&5O-QP)g4<+D(zYup9mic{{2oii(#n>TqvZ!NN5e~r&91ZE1gNq~;X9bH zL}xg`6v7~A9rdAHWtB2v)f>F=W06+?00u}BJf!ut$(5*jrZYU$01Zk1g)IZ(%c-YV zB@)vzm)$Tn9&Q#(rMTqc)Ehm{Qrg$T=7-_06C$=VW%{b2SMlAzK z&wc2G8ZLX|ELzEoB}u{+YcATRhAB{NHPAzkN!4<;BQ?4rRE|U5j+Eu{m9e zDh!LzWvI!Vp;pGVej$s-(SuBeysV;5S3PUNN~glqov%x@=aVM(%(Z&bTr~DN(D){F zNlq!DR4(nDUn4c;s?=jb@w!)dT~kA4EHRWsN@x`&Gw@}pSVkO)G_*S*C|UM06DTrj zLXzY@RjguV()OH}C(2fvUYAl$I%6q8lbi8Z%`7UKbl9ep=N74rJX(ptlu6%3p7>$p z>aB8IOr+xVLnP}8Mq93C(#1@UR8_@_A0!r4T^5`n(K4-bytI9#R6+796j$C+eCLcR zQ?ktFqODXTODS+n>f_+;O4T$*(`G3+x$SykiajcG4O*n?W#p*hbd8@$t_L+#nj%Y^ zr-d58!Z-jOnAe)BFcd~#DLuKRDN%7>xknniLw}0C=~jr30W2fdKG=cxM13W8hh}%;X3_Yxkh{ zT%h^D^9uv$;H1fEJkI^$Vho;m25@hwdjB+I{atRrwsX|e2%tAj>8+&!q?ou%AnTu8+b?heWa1^f^&;D}emF?LzkqD}*&~TS zY0eHn@tj0gSl(%00>%ir<$#J0$snNdN!!04zXP<1ZcuwMF`xJu&@ooXkx;;ZF}$4q zWe{6jH^BI=^OJ~gqQa=TCV(5<=BZE+2p&K#znjIU0{pwP{z_igG0sBofOjCED-w2C z%dcTOg&N6$U^rC|oVN}qucbEbBX)q80rv*PYTS2ss&)cjKckTEFXPkWm=85juGlp0 z1-&OgNi3S-G~hj|f!03EfsnLgzx0Tm7U=vHcAT2{A)X{Yg$-1Iad8Cnf$>1+!1ldi591M#~YsmI}$NZ7z1+(-0P3S(M^CyiatnGFn%BxpaQ=6 zgc#hnpH&}Y_ondhCBt*eI8N7#dhX81Q39qmA8kiaTmYOkJ)~XL)vkmAEVYH z3PBiO3>skiV0nW*2wp4YAaSlf_yBZ;{PZ><=o}E~>~%#yMhXr*P?)_2A=uitpcq2t zhA&>=tNc~e{un4)cA$t9sFm#tOJf(REZXH!LL}>h4;A|Y6GK;C$0pr=Ycb~HwZc0@ zM~ISAZGQN;K@9az^;uAZPAbHRcmepTM$Pd9Vr zFONTBj6#Y=r33G~cVCYFm|(7#bzfmdL9@SVFS z0q;ZrrFP0^5*n9mz}4dr5e!<(c|j8f6Pr+0tdoEb5QDA;;!6R1fd^7QDNG30PBLnE z@`+}cTmb&Sq;tdiL*uY!cWjeuD`56K2!ZcSpeuvL)v+HofaPpoe+fQ)*&M>^AXBly zHa;42Q2^{&1wuH|P>hjPQ%#9wtrDQYFJ2St?D28rmqL&@gI|Fka)bUV27>HCl%ZVM z*G3pFC=*=g2(G{ER1F9&#C}cF6kMEoem!VF`dkRi;rjk+^V|-{MI~@k6h@u(lAW`6 z%rJtxdre=a5r8tYh{g#P2;6qP*<`_kN2Nzv0N<4h1n$~TWE*TIaE;nAF}79(|_B~7vm5++g)zy;D^z}-jeQ15kB?%VBw z3<96OyBY$5t%04(f(#?@(W#zgM&4hlT&1zi?3B>VL}#Wlo~=-QO!pjLD0-9#Ad}D$ zIbk`d-zmVlVi3Xrq+W(x2w3$z8P$qh?cH*G-+#!BZsmcs;YsDvdvf9YM`6v8yD|st z&vTfDAz-L5t~Ep6UH+Wz6kYYXL^ps@s_-`2PC7`d@3~lN6W!oFMmL+D!EB-?ffm#z z(KZ=p)(zd_NyI47R})6I2T%~^J>Dmzp0H&xJ-@%w4n2My>*#3ANS}rFu}$=ZtCIur z1^0OOTP-CnqX0cMpxX@SrDLKOu}g%2pJ@kR)I2lLvNw8#W%qS7A=@dy9%hHk8Hyqh z-4_H@4o+M}v2TSW*xoBj`IYE_EjOjZ3Z+ld28W%ElRcF_tE1UK8Ik3ft-a!|caO9h z%`MqJqDMKF#g;!fCfVkJD{2|l43yWWd->%euWj~~gIx|(>s|W#r<*081ZY!E9rU*w zdUxA%qxTgj*q4acqGCk^m4HJawA&R79$QjW3L~*ec9VI6+|TVAbq*!>7b+98LG`m) zp3lk^cgn{DHS8OM&v@Udn8~=I3iW#^S~Q?+Z(hm=-6_k^IbLk@R)89fFbN{U(vWuP zk+#q#4>SgA{_q|VLt}yBA75kHwg&ouI+9HSAbaXw%IuegrivxFo@Jk0!z#_t>KwX6 zA40aaYgRckE!cn2w)85Pyw2>-Pe`xugZ8AQ;HT2TocA&`2O5Z90hM0ONXWB@CbD00gbrTW9!AF|e{7=uIP)>E$g?cpR?)KV z59{#_2@F=o#2Z60+}QAJFl18;L8F7bRVpqpRM=AMz(%02gW``6OfvFH13Z;eTy7ru zjq5&DN}3H*j4~A$7-wJDV1neJ+ZDRK7)%WbMTC0KFVFa182I|b#3P1`R2QC4uk{7e z@rB8iHJcB?%vgveZITg5XCa$I!0N~kg`W|(m@1FtSp-ad%`7a- z&lP_E$ZYlcJ%w3JF*gkAT@K8x!?UqUZchWyb#Uo~`3Dm#LqjLwlh5y}_QhrE80p{k za+ea622+FuY2f`@%W6>N6cdplv&xV_CdSb3Od0KJ{Ti@h-%qPzPwWcU>I$#roBKCS9k zd*hIbzq0NHK0iS1nlgr&r91`&6xh_^lP&i2=4E&`G(;`uM6g*&UzsSQ5}aH*VkWdJ zD6EfP&#l5f@%y=A*GsnZy$h7h+*6a%-*<~8rb!STVLGmYsHVdITjZn0Uy2~|hy&<_ z@aF^LI}>p`m(;p1c8dd9o0bFFH7X{=)4b7eh=U|RP|VYMV^z0NlQUNm^+e_z}1N)Or9cnX2^rMJ*D!h<94lTsS0op>rlE`!f9%= z$GZi0NYfVgw~JWW?FZ%pNzq>f1A^c|8}Ap|Bgk8zdfM+uT_?MvUQVjPJJGYO0f3%Mi zKCDmRm9UWXV@bO!NqB;5h>zscjdjf!q0W^7Qik$=9Pf!U>=D2<*{?LnWg_-nse(w_ z7(Ekky$Ly%rJw?bBU}>TyfKBV+`}qdDeHVbno@|#liL#PdJvh5g83PNRs;t(e{JK} zCWy}bEpEgy<5zma^a2PnYY(h${ahQMX9UwRxF|sx_#PuT2T~qAQ}k@lLjD0ERo9hB zS-HSp0?!yfIt3G*5WN2aPVYP`y+5^}=f3Q3A-h1iXgvoZBY_(^WW6jPIMgl{POgR-YdJhSu%S^-N#M8!{rzU%k_Rzu@?xKkjZRT|Ao-D)ozTrV zz$S|$_KpkjUQAZZ2b7+KG6FM3-hGVu6Y~+NINu3kXTFg_9K*5PRfc{sRmNvwIAObz zLIp|xmVXY04GO20LlteY!iOJtTKvv_Aq=9}gsI(lI|` z;2@nXGs-`~Px~W0eiBAH53S|l0{6Vk8&O(FdfCLMyco1~G=3==?2o`-fmW_cvJWD} zOY+l0GJo|WOCw*9dGP}qc)S*55yHn+o_r9wePpvES)R-37E!Xiy@v(KxyA%;QZu-WpLhFg#yPNQj@EuB$A0ni>9V47%cvZ>l!77c zSvH1{Aq|v)sT^2WDg0bm%s!bhYLe9((%_TD;wzkfVQR`oh7><#rW1Nzj+BKyfEFbo zIw(1Aj8ZAS(G7uBTB|>x+gBecv<0PIqUt%aSpF`G%k~9EJ1ChP*ufgc29bN9Oe!Ne zzd<(10kqK|x!lkTH7kH2gV6_Z9$WXe zGkn_01Tixd4^&X|fCr=2o=ASBHtAe^_B7q|6k<}7+IIZyOq{)f{$x-VvE@o#T0`M? zAY|*`qiv&!mT^XBsmb3UyhQFty5Y;V@=i2}Q9t`y)L$l5d05Z5-vuH`^x^adeT>KH z8@lxZ&$A&}o8D&l04Y{M#+^4uGWBI|(R*MY2^~pV-DqI&1BlO;owbc}_7WVCGkVfT zdMia}6IZw#ns4`LY=f{DeL`~QpAYF#v>mcZb$u#-H$YzlC$bFvYeXbz?cd_@?lW#8qt9C;_ED;LeJsl=Qnis%m0HWSq*q91i! zno2f)b9t(Z?6G^()*FAE>E~~X)9-e!)<^fI^N^%( zu6mPw+&6rEcbtFEbd}Y+Sz#XwPp&Q>2Kr-AD$>5<(Gsr*z4r$l;Hzr+fAj8S?;PaGpnaZG_aF<^@ z$a9{oNAyh?gKvqp5^xwS@+(~#EDbjLAu+&N$Y_|WJ93&ELs)!gglIZ~dlK%HA?tK1 z`8ibvnPE4s-`_a~@cfgE%zU7B;;~NOMC$m?QRPZ=2z+_5zdA3Aon?A?vHGYi~60uz78f~ z_kxT&0n(dQi!gh$1@=tPdv!apY^gixwtAh`Zd(8^Cn&xnc1dm_AIj+S0aV1}zy|pRni|I}6m? zV(g73XXj74-*6wY^QLY@>RCH8ITvtksw*_T36JU*zAJ-Wf==QDJ?M=-C?BQsZbFn_ zW}*FX&Kt9wPyrWY{TMJGgagNC_T5V%`59N{V#&j#G(%q>s^ zX3V@zawLT7*}LAL5En7BE3GbqU(}td`;~;4b*+lnwf(YwVc!eZhLOVx8rPIbo2}e# znLOae;CC+s3x_X(*9Q9mdvMXJ*5y-+i_M}t)*%3I7F7(rKdw;PBxMQGsZGD(1>KOD z*m#3apPpFigDCnoEWHP%=^Ku#uuNK7baiV(mTZ<1n}HK7%pXw06|pycwz$9_S5C%} z6)FK=PTSu*!mS+SB@yVvE=H=$6L>p@oW5_cvoh}XnUw?MWfG~kahL2a_rWS=w`2nt zTs^rDJ7aA~Sopbt;~E1tR42Ev82gOijHDcf#>QsWoElQO=Tu)0u7FAEP$f>Mf!FOAA-CRB)&(-0SPBN z_bDa4EHS;6npbil`MH2?h*FQ9xJ*V~sFje|Ql<`1PQ>QW_P~dDl|g38atdL|YiH;WWGv>&JwL|bllS@R`;&P&l`yY-8n&^1CIpTKdc;<85gVr@L9KU;N zRXySO(9@84`zu7DHeeUYg^^ocZoXk}}-XSNyh!rKLw#QI{YNY@TwMLoayU96y*k#UTXUYgl zPm#vM={&-~NMhwjG%9*d2bGwof{kL@rPJK=!gFK*t?;EFQr#>3ojH3>%@(n$ANsA= zUw$I?THeX1qXu7=`rXV^wiQ;tcq-r=vV9ck06BlDdV%LOtb@-cf`DWn&Z9SyGROt! zPp%Exq}3NNs9Yck4s20(jJdE7-nfuG_-RC8w=|fLBe=Lu8RcSDdwF#60#1L=xfI>N z4{Y9Zk!TrLyk{QdX~)&E88ob4)V@EDf@|(5|JXc#(eTNtO;(!4o96N~gG+O+$Y`kZ z_@nZxEqxI`SDd9(ZMphTWXMatVQ9!pn%o0*{&9x_{v|+(aL9Ty@dbDC;sgn#HHFis zH*Xj2Qb@Dli+Rv+*Nkr9$>ognRf`MZy!|P}=W{;Lo8Mpg)xh3Hov`CUR^4LMaH76= ze)A4`meKtwWR8fBrFyR#hy2N?QCEU=nP4~9LOz#e0bwhHwFuG2?;h7hwD89^n;g&(}r1Mosi(g?=Jb^O^^<0kYYE|2^(^^eQwDiG!-;|L8j`G)R5>fsd=#Q8Esc-g$^TbkmM-GDCXvvc8SaXXN-Ry}ylg!QPT z)8o+z{6|I~AMJ7Sfi0|U>=Dk}*S1ZYfO;(L1q+v;JYqj2jnTOXSE$!J1PgkBEwO+KGvPe-uf0@|SZaoCChXa(_Y* zRLUvSuEB7hNwjRj{Mdg|uBU4qpf?N~TY6hT*`45ZQT=LdpxoD2$ zuW{U*i;iH45*E9Fq5)S|bV>)}^mDlVwY-Ra#MRQgc6j~eLJ_P=eKYnfxLU*}+FjM?)2!CUU z^)JFkp1s5Y{w%-%j%@a60RiM84$q}YbVq#fB{%@@+_?ltX5eNx!=Onh*0fj|hen|{A`#r3FCi#PLV+b>*L-|^@ut{^KoUt|BQYHS#HmZ> zLu~#{;xI(3BuQPIhLXyPp&KN9XYAnU#?d&NVfD_zNuo(IEOM{^B%}J<&K3Uw!;22y z6feoLdN15QbBKF&OMT}so{*=D&?)d)vYBA79L*_%u05mCfun37l zF@IjJBAw+Ofb`f8;8Nw>i!$0ir)0?B4ISPtH%C$kgPff=fo13$lMI)?YzQAcZ`-DG zjxaE`kUV5Wc}Wi$Q($ngWo8=_onj1inMCDu|N0e~_fwO>Q|2NC$>Ug*j!F?l$dI$1 z-Iy#K)l`!&HD!?$toFFG+8y98*~FpE{GO&D9b^j==|xt2LlDF;lXd34WQ%hsfPDi) zO}Z~iRs4{xoY13AxKCiC2ihOdu1jV2lyyRCHuKq(4 zt&kI;$8vI)a4*#pL($8h?=EAfXypw1IktEHDhU01gqx6y$!Rk2!E&|H;s=2dL$r!H z>iZVaNN$kS{)&4P+*5&5;gQ~u(j3x(@d-V?%i*7x?gWSLHi*IzB{0c&+HEPe=gkz!%k?x>$B_HQ@E zwgRMt&`OkC^mV=f@c~Mcxuc>z@m)lia%FxL<#Zc3ACU1=3m+6qp;TzJe8-nNK)0tNvFIyp&}BKy*)_%78cf&ck4BRtrVpS_6H9Nu51|!3&u8C z`2(NdF)W8_y^yZ#XPo2Wy2?WEFQLlyu;~6MM<43S&pDgT5SSvs$)7)oTh88zuN*tN z6s#Q(_gxMso|Y3Snu-A^e%BQg#Ecn!uHrfcjUrUMu04^R4^1_dYzvm}DorF`B19JU z_5UX(r_-t|+XAs*mACXOZs?7+mh0m8i;@RQ>qZsd#9=rM^a?Q2{B+ek;q$bS5Zqg- z>UavDt*2{I)x*7Ksmmo={ssh|=QKjus0MOOL>wIe(G%h~;f#MC=cb138LJ7I)GS^; zZvw?jABCF1I};O(c!AaI=u|E`7hV#D0IQuU8*jR*^!5&S-1h>8HI@7Ha9miUfFGR}k}m?BTzmQ|3Z zhotHwb^A2@gIOmfc0R&#!pKoi4jy`eKvAiu+EF1(8sLdfbM0I-*qb4wk1#4{`B0wt ztHG(b6G9q|*xKw*nnuC1(l`LB@s84>q#uo^P<+v}Qny#=dH%*t6UbN^YehRlCT$TB z!-FxTNg&+A&=!a^c&n(N)9Dto6=-kb?}cfSsL|e8O2+=G zKB3x=-R6M_kU;Ab)a2Xc(cP6(?Ra79Cf;ZeDw;6iIs{3K7b-A&y?LtiRcEZ%GCIV! zMKU+D&V017pL=LL!KqI4VK30GlMa?)j?CY{Yuw_4Vj!T<>EjA4u252nTYaT)-wTjn zof&(sjA0#2OREDqViLjdj956#ob@;0$!F>m}U)56vN5j_YS0Q5l(@ z#2no?ApT)X*VD^CRCT4g-PiY<4hUIh73hJ0eu=yYraRKJucVfnqEbD7lLb7-h~={0 z>Fwhg;3rOtMsGhg{;D(62b-Jy#>6GVmwU43xe4^vpWOntAf4C-nW(!r_@qfi^w%w> z;i$hp)^i*gkjE$Edc%ADE%<6%G~O!%8z_X&!Lm+oc(NJQh<8I`VCpI@JFOc?0b0nH zpDhByz{4lj4SZIRgrIw!*VHD|i48*K^Nx^Z#&m2A@}$J32;Wt$xQ!pzpv2k;XzNSi zJ>8%W0b@GPAA=e%D)zTu8_X%lH&7{U`@%57LeRubI%5c;+655Z z!pt}HP`n2dZJ`~|PWr1zjffsQrtv=B9!a6&I05}5<@YSkPp*tn$Z3hPe) zF}XGw&Hr~Cr3MR2WA&>Uz!a|XB_8^c9ww&n)`Tg)Fn}z6CbKC*l9vn#fOAu{Z)lTd zgQC_+L4+xKGoRn0Xv<6^-#DBf=WdGoNJPp_37(@F3PdWtM7-Ihan%m<7M2FrM(YD% zmyxM*s9|=LgV>I;b>{iqFna+J9vxtgmoJ||bJs!(KyHrP7Piidn6pHYjao)l*z0%M zN>6Cs3v0s z+Xuk9(!^hWAOE@qKwy3#v#`#6{^6d*^gVtFlAz2m7x7%H#k$O_AD|+7OB^2z@g4eq zpjODU&(A8iB*Jpe+%X~Dc6N|ymSRpB+wJasTg=LjIx^giWjKu66^lm@;wnZ$LDH|B?$hwkxfkdcT{_#TXrLvCJ zayn^5;?*1NH!FI)SWSVWTZMRPAJqgoSSy-Yh5smCWWH|!7gp*0wQt+Z2y64L<5vxo z6SKQj*M7vHn~aS^Nn%;e&}4f^t!CM!M|FEhzjV?(>jn!FCH88=E2_;8^MF34*3|0> zz^{i9Skor>R9a8&&0rAyujQ_2h&ovx@*MtbMw0jK-CH|(D8sbZ)=&H@>zAUF)PJ0f zUnmcTnmFG}<&dUYr~4uZZZ^o%6vSeK_U&k&L106wARE?~;nK{$LW_rFKqTbZNL$dA zRnIGE`WGY|HYU@YVMMTvbp_}S2W*u$-{kO(C(m#x6f+V zcD9DZ&Xu4~(!8;2!O=2S50+xMg;pR7+BJ3)NQ33Jc4QBC1KT5aLSX;F%PF4nv-5zW ztS2v{J;r>rpF=C*q7#u$-#w!{wBLVR1&`l-U%RV8q=zZqF5MS<;~MzHxa7ILX;eQ3 z`?smBc>Tw8$v%l9eO*L%Vz1`MYG>S%H}V|BWCq^8#2HM!Ra-ja*FpC+s~q$-W1e9~ z32+X!DGr9=Q?z4NslNW^$?*VtLdUb-;F{s_F<%E|Dg+x{ozZCf`W^K=%p|AW-z4ar zf)QAPmK3Ik?k#X`a0w0jV0M|_)X(h?T&IlmEo2HAoI8q)yYoLVc(12)Bb)0b==O)5 ziJaDm$8t6wh&i(q>w5~A=>$S@g%Q|E!u>%7(bSp$S~`~7vc7a?zF zzhu_=@jm$eh-!9c*`afAe(%}Yj{)-dDQKldm9x0O^Ro0Q6uQnXn0m?vDCPzkBpu~b z=&tI*h@mfXk=PfPT-OpyJPhW&J}xODL%;^J)FqdvB&QFyq0 zFPhh%n`{KrLXv6V$ZiUWw!0}kx{=b(6@`Msx^VLY%dix(Te8SB*hjm|lGRA8W6~WF zxBRg305ld4Rt@g0AA=*wuiz{x@&1E{!rjd6s`jy6wiNR78d80`HSwjzzS|4B|51xm zy!f{oB*cjg)< z!pc(uR%X@y#&VRqf3dbu30y$Pes5eYYy1FkcGti=urX(*#vm#Fv()zTfDIS^`p(k6 z{TZeQ{^m5pK5}X3T@w^|_@XM7V!KZ&ZW#dhScTzsj~MxyJ^Y8bB^!(>TK8Oa)YIs;XWhg@;Q! zXiaxRe6XiF6HMfTE-Yz0-6U-2VdnkN@wO9|2Lb_4uf5x`{`ivur2OpV_4HceU zj_n`ct(vgVG6Nc4dEw@zJyL!@){N_k#UHrKUN)f4$i2hW9>`uQ{Mmvk_KK(3WYhxS z>GD!sRY0e>J-=Rgk;cf79BKd6of7Gr^37kz8$6kLm*q~2UqbdMPEz?4P}>C!M?Z< zpV#(Zy#%PfDvYn#Vk?903*1r>YlXjWSJ(*XsK7%?fj0QsUk5W(2aK{;hme$Kl?nqx z1SDWjek_C&!n`=(cN0JY{Vv%M4CLnAAD0x5SMLa*IgbN!5vcB3THe2>1{|QOvJuVBn2H z54#T7LIZw`#$%I|^J`Qeq}RIKV;gpgdw5keU6e<7^qTj*rP zU#yD6d4&kyqH_z8>UU27LW!%k4#G=q$^5&>u$2!_%#}6REx54D*wFGq>GwXmto8I zzV>)@8~zr8NhBb|-{ljv4f3&~K=-$pC;Kk*x<{R*e!*J-Ci{z><((THW_C14Y!&PP z_S(1R*NcYd=hpCOx@Iv3e;ZqQ4`SX=;Sr@u7R^#JfvwTsmZaEB2pV}i{^@{G(gRmIjYc{aIy}KFYYSg$m1|-(``!5p41BPLX@hiJN zH%i!iqx>^J^f89{-G;#GVg#0Bc*W@TL2D3`t9SdDL95l`m>s46>zlt=aGJI_Qe&aL z(=m&stfQ}3-ZVb^-Q8fkbe6>O9y*N2sv8s+EDI%2qd~rM{n$c|V)LGe96MLqGUL(< zf@qARCZcirZ0_`6z39q|$m4=Jh{TtIG#8$}fDufe>pqJ)Sg{Xh^WG?wXatYx=zAD%2_ck(RDgp*cF`CpOoMDQR; zBriEJV5LumU7Y7YoPZECCCbQgqW*-=&>j8}O>8l&601)|Vu3pQ1V1G7-t51MIn!60 zc)a~12u!CS{3*sq;+1bRiQ#WV@vDQpBF^)ZFrxcol3VajXDhh^s~S5(dlKq@@=FsD z7~0?&_~GcFpd?8v6!vB=UpRVy>ZBv{IgR@wz&B`hHR&(C@1T(kP&YOQGyc>0`6bi6(l49k4vI z7o$d}(x559X_(S3nD&L~r2D~?cbRbt|8HA+S2;_^a<1}f(0QifZLOFsOsmXUa5Z1V zn99xZ=NZeD4Y34-Uc!RE4+Vp*rcjL$hG5{U2G9*@WQvK ze5ZQ6PUxV#{=U;wSBx~okFuabOGD8{S{(kg!%-Ec7F1I!g!^Urk(In%UEb!YW-dOz<0xwb-}x$vyyW_w*S= zq#6zLnLfYO)H&(Jy7dZxl>PuKKs3}KJv4%uVGnPf;a35MjCiY06h~eJ$rxnTLjawU z*vgpjUtCojI}jNMS)lw0nZGRlmvQ_1SKh6RuNeE#NV6-pn%9U;gGy|Fvx}D_H1r77J{>4+7{0uE;a$SzPWGGCxv%`B4D+ z;~Jz&&ysi|u>yy8)>4CHeWnuV@q0GDKJ4y|*y>IB3NEL~kdw_we47_QELMKm0tD>^ zNiZ&3jK0~YA%3>4lbw@{`pD*9cAi#t(U+G466`wS=t}-M668ZoB+K>FmKvOs_`%rf z;o3ZCPWd6c`kH9ed4lujbtV8|{oXqP&qHWe;K^PNJW|E`a21rrtR`u!W`+*?Mw`w%4`PKST>pv+U7ljKnXKUFVd z!z86Nq;_6dwue#PTn`Q=;MH~Bt5`l;Qs%LmPpKj-g5rzw4*=HM}Gi7Mx7oO;^`ii)QNx{t5TF0k@0g?_iKhX>`EV_Xn9 zXC7AT#kz^}rBesb1u;jNK9#9 z)s;LAEI#A;v5vebj8GEAaz(bnWz@uW5&bc40pqVoG0*ReOT4K9%W%^hr)deH}5aYFOMe!DvNLR{iE;KgF55qx?!SG zR@TT+oc9x@!}s^c;RnI;qT%v}?^D?UcpstjrLtOf7vM%{hg!MmmpZfxta0{3$Y6#Q zuH}Bt3)*kl?Wjn38rZwo{6)HuunG?Od)mzk?ibe~g#$9OAS>+%SHzC?zlt<3h2rj- z^!8Yu75#$htXL@Q=_34$e?@^yWSIUGxMOAJGN`QM6IaeKKknke4-O;?+__hU_Jb_a zDb|&WszjIJlqY(eRsF%p(W~Y)V-crmwQ%KOm!W^afhg6D4Yi2^UFWy+zFj+v^|fjZ z_W$<|6tOR@+Ba&;s*__(tG`}dJL4nDsd}WW^HzLvKE~hblvys-63dgBy8scl1nN#ILtmU1PGMeO4+|N^lvp?*{g0r9t42*i?>wroOC?}`u4&hm z9QQTp*Myxc5;!6Z+_clSo_}>fr zT-m*g9%@*G8tJ3(saz1K9X?XnhI;h3qTuNZEzmrsL$4mgw`DIL#o*0Z_+DQD3SIhL z_0V0PpIu95v<2|fgg?9t-Ng26YlGJvctXH1#==j`;z~jg7Od(qS(>mB7XX>-9ZuU< zBpkT)BZ7pK%!f#0esE(3!R0CUgrzgMr7N;okbo&5rEtWI=s^j z8d!^`uo+-F-m9S=cm!vetjWb zKDMyZAE@#uN>&&SH0o?&&$@Eu20#!82agrVIu>oc$^8A(Y77e4CT1)6wU5Cce}xRY zoGZD@VGP0L1!9C9rk`ZzMuK4T7b!6YV5c#I3gHh1NPf1rXUzR;3jr?o8;toY<`u9W zh#267_`Saoyb&+XVgU`5wA476aF~YKi}kVa1*V+F7jCCU7R3g65+ychd!1qT%x&<9 zaz%>`cK!M)NJP=8EPjEb#f2aYHZ)}#83S{5wnrfar-VBYY;eyWFe4c!Ww=u?S+ozI$SRH+JGv3wB z`N*2C;q>JX1igEjKuAUYazx2QWkqPiMgrepC6v^6S)n)X6NFbP0Ms_tCIsFR^hZ5J zDfcnq^Ww%QmBq)W9l6Ff+|Lpul3C(9Tvt>FESO;qju=7`vKH+=`;19q* zmwSl_=;aM-g_*41tm1?h{7PSxeJ9_HjV`KFhh4%tZQ<=!81!9JE#)#k)y<_ z-<;fdV$YAPq2e06LzmoE<_xm8mP>d<0@5y=vKo#d@ zvrKY*r$}Kszdbe1f28CEQwa7_w0r^|K2WcwL=i%59C6p!?1Ud&rerdd{h9|YbV$}9 zMzVIer0gc8?`lqOBH)x2V=Xn{MnKA(zrU0AVD6dm!k|w1e4#I(QJtx|<7NbYQ=wVn zxA3)BQt=yEqGj?N7<}YbtWRdDd{*^=Qtep6OV0={e5^c5WTcp?WBO*Oi&F0Z@b*z7 zWL!zzq$C-fG#D{{<8RH<@6!pd=wzk1OlaTRI1Q^6_2du?Ycm<LUpJ(wF=Q4q z(~jZa<|9ktELrce+=d;dvmE58n@xW>(-nWIoegBxHmarTI?foq*@qy;{?hGFj}T7x zVShSToBQJQEvJD8Qbl@Z5RFB%X&GcV*2p#cZco_lm^mY})(l1*Gf}!)h8}~xs(R%& ze~B@*sWYaU$OsWJaRpR98B^E;B^(N{Z|Mq^JHt`Nf)A^1nTT57uhj&(ZF)QTu8$CA zB86;L`KWcz+bAB@H)X4%W}@_l+N^kX*#SwZA#66sSDPZOOR_6VLkDKr)y1sSE(R1R zLDA{}D^r>MePitdNT|L_pZ&c=_6$-npsF7CnI`fJ+BkCPvo|(1-aJt0ssX2$kx2VZ z)i077Z`G^3Uk7IlsBq50i8zo0>ykwugyn_PNvimgWiE-`>X)JSGpkncbpU&vp2Oc` zEi94Mf0)l^eOv}YisqsF*?BT9y7QTX3(M8_Lzjkqu2Wnze5huWBZBXOUC^o}7cBG~ z1$Pf7IZ0YQMQ3k)9->h0a#Ee8$?2mttw|vb2ub`6Mv${~?Ur#|Us^7AA}1T>_glI8 zxi+s{J3j%U7TGa{!g_2pH^?s`^?u=k;!aVrGH}~EH|{#W0zd!nQ!qUzL4BsT`76c= z#CJBmX#P6Zn;R5$K|eP1pSkZ4eBo?lyx`zh=HC@BUg0mlA&>8ODGgu#I<5)@_d9>p zdEC`VEX&8(AU1J3vjKuL&t3*Mc)H{bz~1!dktmqVZQS=PZ)V}lgPIS6+P4mfQ>kV( z%g4OJppBVSnD8g!q9pmUHYPu`)aeQU{zC6VTtVjizHXv zK(sbF7_z29{}TQ)KgAcW-x+_?$w0))Ut9Bp2d;NtSnjl)plq>Db88DLaG z3kOZOP_~OF&=v7fnUIKbKeXhX?gx z(V^aOd2KAawJ-eR9PhvK1~Z%I9T@XL?;W(|6YNz=IX8#WB>o(d z_JSusFQtY46r;2$)N4stD*Ju>fRrE39Cbz);+Njv>-9$1A8GYbCxKCSe7n;7V7v4e z9C{16oES+^+o5Q7Qm~qq}zd%;O<{TQy4v7+00>|c3A*9j2D&?9y;p{0F-N{X457}A@s{eYjRXS}^ zQR(j|ZmH62$3uII)z>JZkU*W5jNjs4mh%58B=Ahl6>^tBD)yZ(fTeeCOo zox1jfM?{qEiOS&UPTnmQb>srRig>BxKj`eVE(>heb)?L*c%+OvL0T`*2Lj;s0q&2O zW4+2l*I#DhNg>ZC*oe|M-3CDK_bO%b3&fom&qW;sYp=Y4&bc=TjK2mE@CmQs{n6kG zZu`RAAEPU88&Zw{I`970@cK`AHw-e3JvH=XG6*sJA#k=x!~b*2eM_Qi73qz8b<(Bb zW_r1^V9jM}B&czQPosco62#@gR)u%<8<4{B(J0y4HdOi5R=T9zEkhVL0dADu{`T#A zxyk$k&-U+jQ5yC8jkn+DKHd`yefr@;_)}?f*93Fhr;INKZ!{cpMjE|_;$<}EIRb%H z>jvA6|20Y0SX}ecsCdN8ZCoQrtQ^RE_sD;o%^Wm7tLLXN)WScr_0ss?%L5ACC>sU~lrm1W_T{b<=#D<`jr2GD9+ z27;Xd7!Z>)pe38XT`TT3I zmSyDlY!MM?v0L;LihvL>v&H42_!@1c4+A-%ybH{GC0dd?#zDri@N<}A)XLtSJ3fI{ zAmekwG|b)o>cJ>Z{q|3QqVxJys?uu&1*3D;UPwE&m;OTC)`EQs;WJ2bW7sO*x|-?8 z!kHEH(Nib<+{H|NT-rt6>G z9x_wA?NOC$rHICmnX7~5YY&5X$;~#Cb`HDp(;)-3`v0gZ)_X8d`2_>FLtO^CScQJo z;VzvZ3?tGpBYoDL@Gk@CL<%y-9Uz>au)CmB2;tfzNwycks4ut@FnGZ38a#7%wOcT? zAu(~?-9XHqfHXaBuHatiBH|s+yBi`Ea`U6RyWge_sPmV04`)`KR_eqTmO1?1gc$kC zJM)HNT2w#o85ix9br#TKhyn?J`Hf^^jvUZ2y>migtP?h^&L5!o`lR#&1i3$vBI=@8 zdtO)EWplKm*J_z2UAo+YDHlhcaB~N2{DH36gwi8v-Tk_U;QxNgQe!XY9XhhfKc9$` z9-c@@ehZv-e7+XOU{`na*}lkar=n|wc-^6w@#LL9zB=t<)(y{B;iI5i8p6PHFgBCz zk^Xmf>tmNsx~`SoW(tIA49rl&{Lyq<1-_lMX0j`{cMs7y_UMD6t|xCc7(zJcjs`Az z(riBjD1JYdRFV3&YkO%^<3OcyC9dS=Ninnh2G_7c2+;0^>**J zrduKncnyL4)(ie2mOl1>)YMVp*Ikvuov#iS3|&Lo|9N)j$!<;oxU449-CG)?cw;0&+xtUh3sk% zq{CC3@7ta`lsjJGNJ1Xsbub_mVB7wdlS*0XCG_{QSK!x&bZR%W$PEZG*t|Qmj{h1G z%D8Ls!C6!7J$T|zlFu8M=MWf^#eU2DR|apvEKwx-5F8hpAh1RhoTE^mVjOHDulkrr z6LHmzZvaW>J)b`Thk~`r(ctG~o!$$IEA@w_ef3C$K%tk#Wq7>|Ekz4KzpQA7K}T1v ztYzHD4Aa7zMGo4#7e`kP^N66-sY}pJr*t}946E{#xot3>JisO-Bp7zcGlf^C z=i}yrDO(?IXbfj<=4$sYEee@E5V2Efjsb))=okR;2!7%G13$BQrF%~?@@{F8Jo%$49>_3sNN;otaHnI%U%MiV0kzHR#W4*v>qNuI~y z-X&R$mJslPf4Hcgqn#J(>*S%bt}NwVSoGt!zBt<$^2B@?<7=OoKPGen;>-*BS8j?v zwv1A_1U!cEf>bV5Z*l3Jj9vbu{u@L4Jz$mHmA%sfsW|tZnzkGFnBLI17&#H|MEe#! z{_}vAoa0U~$T6)Z0JS7$5xRW(2_krzB%R*3nCG7fCA!=<{PWhgn9y20H9TR2$8fo_ ziDZ?e`i4~6ium>mq2C{Pq9jfy{+wRvB>pt+6~9+RcGkTyxJ;^H7l(S}{H^P=C>$D3 zS~W|4ZHWz;N`NFAI6Z83%WDTaRVO6Ji^m*+;%qneQi-xW%>&p|aEblFD191@!SuD9-aYSAjvD&YbX*Wf zuEFC1AJb5bpu=E~ms5hSI!#}@haI!NbvjL{4wagN=l*&4c($KvJ_9U@FYv?+bs$Y< zh9f6$`iyL4-4UKKz22f5G5GW5$|x7+iJ0-5#UIU`DVYf_N#o6geK%=aPcHPwPX0)0 z80|jIG-J#*b7utwc;Sn9XJ6TjaOOcuZ=1CrS1>F0D!H*)E3X4Ja_fAnq($D7%(@)* z?G&{Z==?exuvie@u%_O`$3@vQo}&4UP{Zxc8u4s;&eKIhv6I<8GJ--A*@mF#5!<_6UGN$A`si_ISdnr|}q zg#oyQuQ0&j_*zjpj~GrRZO)K7Lpx9SGY-}W&udOr=WFXNJ)K!Tkkvo`%AsNkT%<{v z^V*L|G4J2UjG;g$TmU8vVqSBj2Ihj%O!=i+NGLT%j;;UxQ>eR8W`_4gDmFF1(RkqZ zA11QetDU#S#UgEh8F9C;9~Kw=P;i~zZu_CVVQLo(7cF17b5QV2Ukfi0Q!WsC@aO)- z0}KnFX;LQ3h4SyBnlc^Z`*K@s#KP;Vgv(P}ZL?bOx;);o^MNlHXUHPUlSkR2GpH;J z_Q$4!p&>qte(uOb@5L*H{j-Qe@z09)CH$KV4oE@?`Uo`cVu?dURAwxR7nDo!=Eg1M zs!jhmY4Hq}B6&ItAUCjw^?k%cu-9OzKHWG&K(58oI-IU8du?aL-mouqi!X=1+YSfP zdX{sc%6=T_N_C23eNu_@@SIU{F`c@XYpzT+n&K|QhvtgR-Pt3zGPJUX4IYK4)dHx@ z#nvihepXiD*GT=yw7iPYGm{5cBn`8pJ!V~6RwMo8g)xXmz_DX!qK0kBI9RU$*(F); zR}5t<0qaC6aM$J&a4BQ(Tu0AeC)I!U;rsYR@jsBXCVEvP3@U74lWZyO4 zwO+ZC^SjVhRo3=>)}LB0%Yt1FrW3S|s2J8In{>+MLIYSg+QRwvgGJ^Br>MgLz%5>cDA1oi7|CG<(;0I(I_AsOu z%siX)YOvt}Qkid6k?!D~~ zM5;ZS6c~*jSN!kqLz%_!_J=63fzI>oPh!l#+rzy7Bso*Uw?E^1z9{+29gU>GB#-xY z2P>B;eLC6mLv*5Wg70|Kq_5<05MJBO4x7>5zA=}PaQ(~w{{_AMI&{2yIyrqLw)C;X zy5L*dkCjADrMW7KU%x?G;&{t-IfK+stXVTRDaw|(Ww6`YEB_gb{sl2P(dupU7iH-8 z0^3E(7|h_dgb9bsuY}nQQ?SI(T#}oPxtL@Bwkn0NvG(yUhUX*fja0Gv1_(@e35<7`9b;b-UChhUJUtsm#Y-m8odwmM{F`{_PM8>K>E7 zR34*3{o{^|J+-9(5gkJIlfQLELU9;bhf)D!FLqXIk4W1~WmM50JcKLq*parKy zA`z%E1o70SKh{6#UO9Q@90**HWy~L6&0A&P61@KX$;A0&Av5+Jh3S`x?|vS|;arZH z9$@D@FIg#h&6c@TVAQ}5YO#O7uf~)EGu}Yt)pHEBGkPuJc{~ofo$~XHB?8_uL;O9& z_jz#TWQ-zua5olGoWvu%+*n7}Lv#dOF^w5*jz7#ZaFGSUJi_a>2>9dagF~^4cC5>G z=U3^<-LpQPZk;F}Vta@5RQgyz!oN@YVax=bW|s7NJc@?#81N!7nz zW@FQ?t|wd_p?FLT#^+AOO0Zvj~;U+Qpiu1GV^;`Bu#15c6L3UVYKwoGH z4jaF<8J^h(vkwOWxI^71Nn64*uP-P?v)3YknIogAs4sVh&N4%7g8c1wZto3_=kW#z zqT&U)DxM;GR^rN|e@p-Xt7Z7VpY{L$`#}62h`$5zZy^2)#Q)g-|E~wqIOXgASO3pr z008|J09F4s)IL{{F6{RFa=MlY7PY}YJCfs?j&@VDKHBUt>(rx+s?e>o1kNwHIKj(+ zL++*sU~0Ap6X0+BhB)^6P~m+lN!H=(*nptDv$q0=18W9(RoUrODB2VqQ9kobV)HnK z4+|Kjs2TcyT0|D1claWAi)*#op;}Qz>V%k(fvbQR;oc!GyVFbJ0AL(otpV!jR&Ng) zO3Fr7(StmTUs7O?pt;H^O^0xS9`kt^79xN(>YxUmSMou(-RVgHL}5Y2vO|a|){u&X zq@%S(%+@@aE?^X@Z3X{=?5jfVjGc#b!v%0#_0Znw>${WPwV~*ReIbMV;mTZ7AXy85ok|Kw2wHkS|{&rgh)k=sZLu zI7n^Hp*o2}py@g8?U(_~lR@^g*#&oXDX|_IXCJTDie* zW~hA7v9VoN9)YKUefsI|Ls&`>)1M`@#SLh%ktlfA3;hwim%D^kla%|O+eSJ!sQfqL zelx(>43|Siuo2CxP=k`pzqenekwY)baEQgzLada%*JXZ7$Lx{UbgqkLuVm%tnHSrX z1VyXUCYQ|LgrxpeN=b~wsQ|y0VvCcCzNfH(3&XmIZdle()UtQsWD@pa`=Wepmwiqg z{O2O*$oY|o{a#-O%IR<~bnV0VIMHj%Vd{{%GaxF*@6M$uu75#l0{ z%m#sAEArDjqFO+@B@o+=c2o5p%B<8YEeUW2Y^L?VEO(07PR^2~Cr!Z|LU)yk{(XxV zsBzl#K!pph8KyH^2~QNO<)W5EW}uLjD6Y9VmVdF*7ucu0!@lCkyh;;Suh@pawgqbBVj=f1d!#VQ48k zjAhF{>_|;9;b(r8=;8);KUq@HyX#QHW7srCIr6&vw8;)ax#v1BT0>QF*`8umrqe)- zilD5^e0lL@eenXQ55MdmZq}GQJK>Q*Z}?P)D}4E+CucULnn zUV=N+yf{Es`=043f10!QTl&N$#c?^;=aJLSm9P9Jz2|9=imJ1P@iW7g=TM(mye6tx zG)_NXy|>d$@epbIB$V% z5f8!jXn#8uV2rMK2X?illj^i13g{d6@0qeXZjKdkSalN?I}tRDY%!g}EO>$9iosF2 z_}`#^z(7XTn71cyq(N4%6oaJmF=lO1DW-aTA4TTm%F~MG+*|0{Mp2g-wYGJUD170MJQ9!bdo3iCF>JT5vd zqvb}Sv#+YRh80xM0{FAmoG$AVEf@+9Si;VkIzGdic<~tsjpAn7`APH&N@x_>IV5?f z3y@>JV&|rZviDZ8Fsgn4TbqIJk7?Ly9MlV~r5e~%oo%@TG5c}2W>HJ30b0SD8S|E0 zf>@IGXn1*pWA);u;Q9n1IJIsVHKeCx6!@h=VJO8Ki)Ud}P7e6M3cTcP0x^gRo%xsT z?pfYJd=-D)jdyV(7J~}!3(zbqY;$tVj7vrV;0#8DC-tW`q$x<|P0_K)k;% z6Lt&Oje{!5SYy%{&3hZ_t#8*3iz!9}sNpCE{Reb%*@V-dShKbnDVc4hW&--dcfY!#L>1hI@3qEK%Z zjgjh`krvr>kOr8a@FbX;DyuH)e-%q? zigirq^-r3EU;=N-xW+<(3ry+*7Mu{Kth1}B$-6WzjI1951$?MDj6Fp5q^4+C1_`97 zx{=`_vWzq{biCv2ZSP6?P|BU&m%x8QSY5N!pQ7jii0KL@ZY(-p$Qo}yhW5dQtG=M~ z9bnMbsG50_Y0Q5Gs8pS`>vmrSJD0O&`HtdG8?}){7bBjf8ifr1Q*VbK&Dr5dRq%0T zMceLa>STWmDTwf2;<((AZ*5*9xhNDuYO|8GP&!rsV%WaRKbqxmRm0GY(K;RY+!mz4 zwWJ0vpv2Ho%02|8rYUIPI<`adg>wwHsy;^Mm!It@YgS9~2nXANtBqYhQ&qbHY^flM zb9$ql1?0$8r1{hE*xV0S_MW33;z{Wd9;3w-g-FI2+LjD7S34{k`?MVr&Qmb*skUC* zlAhvjtRT}mWLs)OO-fK@tK?wpf^dE(-PrZe!NbS)m}SkkV<%vdFC4ew52xW2-JTtX z`DiF}xFyU165d8@hJQN_>v6*X2_&m>8%$TQ|8h#Jd@gQH!4U$fXzH*ajE*^8`pP7~ zCUM%`XbWA6TdprxyBx40h=|gBgf{Og)vw`8YD$%S+RT?-GLcx@t24m32Yhcxs|`M+(lADCyOtE5vHKud*Kl6H|DwqxddKK0K{GoOphVwxFXy@nhd; z&S$xaD>5rZ>e@A4=ZgNr!M7b?TQ9u2tfFNZk{Z0JonU4?#Mc(RMZymu6cqEk8oj%GGH7EOIM<@iy)5&mIEPpxRTRa&q5)JK>4m9abuEhGHcLR-X6@bs>;|tp zj`x@Ic2M(2Ysj~klKSDgKVL7PKPI*pY*K9hp$l&b^5{o^;18h-vsZ6tKvr@3aMAqR z&zT2dP_yE9y$5C5cW>qkK|G%v5q&KeVg1}B!IeYOuCq&RJN<7SC}K_^;J@J(yslvf zIBVm8z{wxg(V~(0Kr>nX8*@E=C&l_Y5ihn!fUUYc#;NkY3=Vc+JAh+M`>%wF$pYmK zYX`?d4Pe3l+yVf>|J(uq2?_v63M-QnVRuXx02oQmMP+CzZ?jxf;jgA2BDQ9Z5lb4% zsm^CzN}Q`zxehs6d6sC2*5#$wiIl}i!h^Ai9t=g~xH*V@6In2?O{nxP@jLH0RH>vK z_f^z0BNtTjP0+jIm_8!5r7nFRD@l-gs>+sjqiD!3)1ns>wbe9~cg1v~nzJs9oYf-2 z2su>SscN-PNtX$$4Lm~cabnlWlR{3oMog?H96FDgqPrP)La_s2@sz83NY80znVD&G zpJUF~WUXjkD{YS#$>ua6gJRExA%`v#G7jg`i?)*7(_nT3E{^aH>&;B<+i3X}U^$@Ooaya&C+!aWty9Qw+P~N>yhq-s({m zL};kViB0OPB7guGN&l6SdnT`L3TL?HR9KfGCxlpaHcBm0gkm?t;zlMoMV*K!qtnU?V*l$F%eU+(!&f>%+^xSMX3fMNs`+5F6UD4V=A_FEe4_mvWs%S z$i*FVxXpTORur{WY-G{yN(%b5W=UmsB-316bFU96k8{?ArQEB`^1JM)N~W7ko8jp( zTWZYiDsI{o^Mlzo+_YLx8={Dt=`>-AieWlM6U5td9+XRvwI!34uVuWjRBVSRsd0~F zQnP8yxTx+t9%m-_WR}&YPkWnlJgBdAo*vMHj zU6kuVw!I-2T4#PJXh~AV=-VxD#Ac->ttsh5MwTn%JVFYxZ2KYOHxyttvw zI+ct74oUx3C1SY|lZ?taN@XJymhrmlX1vFxF32$wa@``@N1~Yd%*Vqa1yzaV>6UF65D3-xTRU}fmtF)ApMJLQhc zmi4js#GPa%&&8`0LQ-j}R;ctz(}z)PHPsp|6H-dY(kNosy1r9#=6N>okyd;;OfzJ9 zp|Xt@`W32H6$iN`q^^gi@WQBt#c)jSlEw{L--*tVT$0kE>!eSG$f`UU1u+wx49HEg ztE{`8aKWfqOz?&(X0eb+5^7YV&Z$97L`1Y|TmTnI8^$b~LNz-U=9SP}O4+SXk|lPc zt7|S>HF8Z(l~C-YsRhugldkF7w3{Uup*Eu-dR*+RRLnwgdM=vP4K3qymFc@B0+{y0 zT$dWTQRZpVidscM;3cp#YrRF94^6 z6kX1e`b`l4!}Gb#q65&VKCAm--}i}-WuE|atTh4fObo=<=R}bZffGUX^?RSL02Z9s zh=$V$V5Rr|^8~df9+C0(AXzAm0erdS{rP+W?6k7fLm^%-VrdIN@Nux>x7!CmL9R@C z@-hOgy;?|&n{_Ea13=MJ2?h3W|Is~jJ|`I9!Ak)85TM--bBAsZVT{i+Bqfpe*?`_d z4lq#??a2{((u|j(?i<%9z?uSiI5FzVHGE%C0AlULVUGKTDVvrvQp+sA19E}v=jFzc|ZCEPz_}nf3vB6A)tabnA#<>Z-yg(w?h7rdLLFP9?K$LKh!+~ z^b5%Gj11Zi4Pe-YbpkE#ebzL&byJ1Usk46r3D^$8%*eu0~@9fKrR%N?-eCB*Ks z^9~^3xP;)|=;0vVjsTo2O^TR5knsDEJG2%Tz!`|c0pD*iGGF5TuJ3~87Lj#F#Fi z4bY=a5!5k&q8PZy6fUmzJ)eTfa>gF{Hc;suLr?>we@r!1h^9VFmvVZpDL8S~SK?dX zSCGe5#d4}QKCs5p#lXob<@b4sx`Pnu4G6`jJiwWl$?(`l2+jsT8O2{>#H=OE+PbU; z7nCuygR4K7>j}8p>gm9BtA3mT1}xI$)h%dBOFOT(dzJ;2@A{^_GY|-2;cMKrAQD^w z_!2S@rK^XJk36v-cH|l6FA!r9i#9P3g^Jb9Sb&&m%M(4PXtC}A$L@Lf$lZ6Lnw_v! z-yu{7&!8wsL9*ixft}MBr0m?}*5wY~#-#g6zBCul!g9(hP$fCTn_tr{(&f zPy-l1+WC=1LDBwPm4Fg_=n70c&(rA~euP3nsqqz*mS`52$j5D+tWQ`Z>#0$^UlS<1 z;x?`%M;zNXLKvv=rBQ3p;6x8!z2ooT1KJhNCZ*H^6tqNvhk6q%0xctaH{TR|(F4(l zjD#Z)$gA~D0>Lp~x=$nc1m1TzgRlQ{6YCSPD!!SMhe2mB@&=zpH#kaAT-`13HNgfiQ1-k0s-%QmMJEaZS^+^6>?jkwKCXC&U=;D~ zJfV?Zqu|jK%rMnX|Mluwn9x2<#_GyxQO4~j#1KOIGKaT8iLZPN=$W!&s4SM>ankUT z&QKN8wT1AC`-7fDiZ4=V)ah6`gpWbp$w7G9;P;vIo){>xAZ#D}1^c@E*G;#OtAbs( z6~y*_03yoLb5;t>^(ebBhz{JZQq7XfEGWbXsTwh3D((=|PjQNUh;K{m@6;{O_~8QL zlp|4~9ml~9Vt!lO3R0Ltic0}0uS93fG;#r|dhZ!3ml;x3N5uy4i8%2Iu^)ysFgDj$ zlI8S*3~noA6%1{3q3?W*Vg@-p6(@FoLC(F4PvlDKLL~wbM2L{$nZZtXky&vQf5g++ zZW|Px%dy7*w`l#n?{;g$dR zZF%g_4qhrl(1RD$BiBr8L6h zVp%x2y47Ll zNt2Q(8xxop!KnhW#h>>duDJ6P+gBkZB*&&$RSTc%-Ld!q3$z@kQ!dy5OZEy?f7L$T zi3wUi=0rINgyncH@oul4X#<@SYH;<|qZC$n3{YuVS6XW)fLmrpBfp*L$;AG(J$|K#Jr_P1O_bJGq96!dI1 zZ}c4tuEHNJ(I}S*e>2+QSykY`75tS#tut~q;3$_H+yNjUyAKPT+wYdMyD8?3!_ot( zcY`Zzq9;13s5l;(2{)8vE7!qu&?~@uyM|kK48~17igp#Q5G~1U_JBJ@4uB-}!Fs1d z!Zo=2*8SUI4+bxI`1cfFuhP!O%VD+%sMmB7;cK1w=O2QQgW82)f?#b-Di?H3oQ~k| zcGFylV0T`KoGC~z2oB!2u;9<8TQ*>^!N+bOglUbiMQ}{w4&IpO`qEpU2u@m5p66({e5gnQ$Sq_ju1&|rds zohqbbqLJ|UY+vZI&b zi|l*g6N1hF@Uz~U*%MvRjxM=p$VcBeE|B{ZQ5Z-LQSICRIGODDa)+cXwsQ0od z@>=3zD+5xFjW;E&f?O0Vxb}=oWuQC1n6eS`kYp6y<2i<8<(h`Vv(zdSaVaJbVm@^g ziJ6nhW$<^uT|?_O=W=vU;5~hjhb0`Rd3byuw%sI*HP?Halqj>znD+u;3nYNu6KKtwx0^}SBblfeU2QH0U!Yta1WGW&)$4Kef@=ytYT?2~bF;|RUqk`nY9 zed4Hp+UWDRBh+3$Iad{OLG1vkcLg>7NbCA;@!F>temh;}1azLY@aA4nN6iD)kHkma z*ZKxqIO`1cH~cn=r2V2ktEOx7qS{sf<9ujM)=MxMrMXib12}Z(?7y+9LV(65hzuxw z@B@=Ua<#Oe7iQ4|?`XkySoA8Hxv=O&t zd$^#39s<}%`@l)F`?r`zuA?Knw~MoIAj8g4kyDAm*((5?ei4D-OQL&XW6<+>t!b@; zA?B5d3w6JDFNVPak_jlo9n%@n5knW-crVh?WS`?AO`8r|S7T`}d}pluXOt7c641{; zTQFuQYcbjfNBSTgX8eX(GYmIixEO2f%-fdsF01bY zZUDyi=Zb&iDC-qtFHN33_K?>H*%jEBU>u|ExyBevF)i?Az__#p?J(|qpj9g3@~_4- zN}Jd1r2Kg9%$N|fbRt~1r%xn(mB|I>zu7MQ2ik*+yurJRBAh*y%8VkR+ZMKdU69WCl|R2vwpeek9V@fHT#C^nb>-N z8Kn##j^JFbkLwJi-_@FD_g-NBcz^cgamK54qB5OepkiK2C4sqz9Go0G7FY(`$5%51 zvVWo{-G;@DMO?!CXPdSQa%8NUBraeD3pEPKMeegA;FLD}C+_i?<2TdUD~Pb7Ky<~5 zOJFe~B)wayxv8pJS1&T_E)ZDFGxg0d;3Y`jAB#28@`UsvrLi2_|@||H?)hu z<~-ulivlg|3$EwujP0)GfLR3|FUlmqFU5|w_f+c18~l8OuM|khmO*EiDP&vO-h-_@A<9$U{h-@&eM=_gnD)0bQpa`TH_ZD9bruTgF$v<$~7*ay3f z!0}OWa&=4WsZ_&0)pFDHeCxEC&$}y31^Z`p8AQi%i1N4Vq|l2#lwD8(Za9L#7C7RK z^QytJ9XPTfZsw)MvGwx4*yfRpvhqWkYI#6$edR7YaQxl$7SmEV⩔Df?{gZwi2L_ z!cf=1mqd&6*$-rFeC9qo7)hlA1H0qysLNO_)97>aY*$V@HpB7x%Ar-|z(Qt<{ zrdZDRi^PK8>e7SvJ3+mPI~ucwZiO~jPjYjOAMR$q9eDr``I!UajBUiI@Zg8OqBO8a zi~j3Rj+jJs!V^k7Sk!M+_UT6ic$&Fn6##j=#yfSQ{ZOEW@hp>GOCGl{zFZJ~Q#`*| zB4)hb?s$n|$3c!qnqEUO-Yc=%MB+vdvYe9!AN*5@ zP(sP;E#;d4rmJG?9!yRgURQS^fXi_ANq$mo8Ikyo8PEy3OGqFXPM5}k?I(~}firp6 zq*k|Itl1RI@Q~NX`e4Z8{>p!5@!(0M3L^iGKe7wKqL)yUz8bdrSih_?X|Cax6I}26 zV~Qsgi$kFENr1E91h-t3q1MtAO^&N{>n@Pid|BjzPQO*sE9AdgQ^qh}K&O@~_XuTA z4jGf->l@QUA@q*H%g(9feY(JNY&!D%8${>^Vg5#^cX(AT!Xk{8Ychp0A5V}vfarjO zSFX>5M^7GKB*%)!gA6M841~u$ddXmnY`I1v>aI3&qM-S0JrDM5su&T)lW4nmD{Zp6 z{Sl>Mg`htG6kYf()%CAMGzC+-u)%*jhDOvcmUYA>flV!WbCO!; zFOw%&)S3E~;ww@30>w7(X=jbo4nQVI+J?<{5|BKE6IvE|lmkDM6?d^z*&K}cF!k_E z$m4asHRB+&oXB965EWsP@%vqpa?402et>P0W>T4U6mtWFWZAe&ZjcgR8iT{2%^jkX zQ4aMa%TGx1XG^HqqbDHw*7D0mJz;n~SK{RRfz!`#jXaFR zDWnKyOa5S_bWlQ4NL0}=hHvq2AZ6R)-!o7csrcI^RNbfy;`;-1qt`WnQf;_}$dD!`sytN}C%HGa zAH(-^R;~j6$YSu8li9ed&AD3mQ*2j$Qpiengt7npfcn7#vR<=n#Z;kC>Zjyj{M*F^=ztz0z&g*}l4 zzivRO_yEZ#^ft0vpi)v&nt1kEwmzlL@=QE_U28$vd$5uJtjuYp98t}>q`W*Uj2Fxo zFJnbijLUtXg-I2|mblEd>P?l%Z{6}E%$LG6DKI*M?v(Hif1uRPHBes*rh1uSekONn zmfuh_4|)n9G2nk%a^d!o#w5pQuQYn9C(DKUJ3Lf+`mo@aFCMj|C0zxTX%P&E#P>lk zdX?cByokn9jHM8g6{tg>Fxb^DP>+ps_6>vAuZ6m2ZbBfx1T(PD9mwTYw{pcba&U0=4i$gc2IqAO8?*A zIS$R;hmfPVkq5<}!pATv%@3vds8?t}7YWnfaE}DaG4E;dZ6Fi$O~hS>zSfOd^rLSc z8Axk0`qmN91*{pA@o7$~%gvJ1O5e3&N-d&o##PQ0nBbLz6+hd$8QK3r$|i8G{6@PX z`$3%Ui1{Uax?e6QN%-XBaejD7`u{?R{%+ER=up6D7Z`sCJ{mrpFCif;G@JpN8uczHD%qFag2{3?{Q+TN-&rDKuO16%8b!!L=WnKA=& zPpJIR7|77EM0Zj(t5gS#p?7+IF!LjlpsZn2lqCc2!;(Rrn8$!e>CWS|kmr4Y6)@>5 zqYVDPnR#44R3|_ChndjiAlEX$t)3jw?vd}=gL2BGtB&T$0f0J2L`lGR!Wp^-k9?J8 zY{-1Fy9p&5W%F<=aAKUXo(My%PALF0L@~8K1BbyAaIo zzRDpx?o32XIYGN*Xk{Wdr_Q%IQ~f;((MFjXeuE=>b)nMais&R2QR>Omqh-Mp%A24RQJsiUBQ*C(*@j+N^BpWWPMmEzh&3O7E+z&G+0kOKZgSNmG+Fd~!L#x15S3dWl{ZO; z7Da)68zCz$bqk-Qk@d-DDIoG8(}!=x>0>fTvHFaq9q0YEujlUi?O4W-COcTxByX2J zYZILp8|kcI08NK%3(FxJtaDb*ug}${-&1;L6;{SbfRI(G0(D=xv$O?*z1S?m0Ld=9 ziJ;IhYf!eGN6GFo@=EW2UC50)8haZ@_xXjNE-m*a_D<}bCL{rEesiC5C^r(C z)3*DZ@x|f2&rNsY5G7`HUyzJ*G8E;6DD6}HBo<%3jPLFJBFzlz9L9IR@kH(pVV)&{ z9d38Wzl?}Z964Qj+c^4s8>z5V9yG@m7dEp>j-M6>;EiPI#SaPTSK2>gPgkIGVYX~> z-mndFGIox6T8ix@Wd=Wu7%+%iAtc3PY0;rmb7~Ws5>~+Qlg$Iit0`j>t?Y#}IK)kM zE}BpwlS#lOAoW#B&~;TL+nm0w>_q0*vWI)lWK&WVhcaY*z@>YZFt`F4zuNC9zA*Yv zC2qwzq%1SDW8a`6bk0pmBF>#D{WbhWuhn$j5EpKiz5Jy8CE{grak!UW=eIq6Oub16_o2*HWXHKZ3jZ zVTJ4S@rmvR3fYF>$h{wg?}YpC@^m!!7@qt#fFcr}JoLMMnl8Ct1@m5Z1(q@Do&oxq zz5kcmJj{0o@zL;UKeQg;(l=paEPM;&*{oP{<b%=D7!v8bE=Rl?Z?B(&!r{t7 zxGDSdJo#C?}HH}*={HY7qNCgC3}Q!jmX(O9;bp=u7Qfage~dpox{8~EOVA)%L|odt67eeM%E z7*@pE3%e9YCNb;6oYyoK(hQb2hwVwQj!|a0HW?l;5>m>1)}8vx?*sVA-x5Ig z@=1iE?ACq4rM;MkG`w-gExq`*CVj7=51{3<#pqS@k$|Byud=W-hxxy@PEd-jhde2GSY&^wRapDNrnA_k5 z(OZMUHq<}^QWG8FfU4zrAROuOI>s3+)(M1N&4hEm0BrAhj$VTaVf+vmB16j8DePQa zl01UVo7;jpoe)=-LEwhbtq|AQHU{AWC^@S-w{M3+6Xn9y3D~5Z#Tg!^NBxzN4D-O2 z8{xM6@urUOBGh8CFSQ}OH$GkH5*-`FUr7h(S6U*(6%*lUI#+`v607`N6I;Yzr4F-3 zq^zeVtJOfH>yXdY?9b2Te0C((Jmw8nmC&mkxSnZ zPZfb{l(6%=0$YFE=EL^H^W+7(BAx>UF}&VI_3iKCMLy4D^*g%Bu@Uu6C7NQ3B|}-U zoNy{$v1F=7^zmj_I2N&itGVjE1VBPHr;jmyzv*|D6;XCk=p|#q3H6G zeHul(oJX!TLi{8gOAijbnV3Bhzhg>YtnNkM^q{`KYlwd08y}C8=)3!mEnm@(UDNv% zGvh+S>2omxI#)&{28kM}X?n_6$e=jnUn&(#KIWJSG;2T`?{H(`B&ONCT3u_lwZfr? z;`(dV#$Q+H*U2@0FPl;?=@ zRlb%_4!kHG$1*Nt2MxdRp}fZd%M878iXJ~nWJkBfR%raLdxP~na(ZVLv<54Y+2?V` z>_pDfNX=2^vQ!`rzOfh@S@0@va3&>7D+}dI^~A&_%R`imSW{60Ww8og%CbcAFY{Pk zfMQ0umg^S(=CT5z3LQ!1$qTFi8DmPuhRIs+p0KRlD5YD{-9o&2viVjik{tjH0ptj! zT)u%Jd&)Qd+aRJ$+z5peBd$*6N0!Z=EC6`}1^vOlklBR2u>7c|OpLgMfAUKRlH0Ma z$}g4TI;RxCAG`zLf8L;!ZBKqp$_>?onHlRKau&{*eq{2;2N|{*muJMN@s#JlE!#J+ zkn{rbE|6K-^UKk=hS*9bpv(Kg?~!~0#)PUsxM#b750v-gr|Oc)_<{ct zm7zB|$(kg3H^OhwQrH*sZk2(! zncwpR%6CX_%nCWH3bp(7a;i>y)?oGIHv0m^fCkdP9;jk8#8XvY{Egr|Xt4E18CfLW z=)m<+Qrw}&8r;M0J%ByMtxPqHGF~!-B_1i`C8bY;G%}eujOuytKY0Jfuw21HH1eBX zE2xHyNx!pdwQJN})FZ*BUe^?h0QvMZWpp(vq)+t_-KBlalg?R-D4y=X6>pgT#?{~~ zzlKapbhDH&++JF8Xsed<&71(3L<+0XN3_%n04%L?W>>3TtGa-nC?NSET5GK^J*>S4 z32wn%>;1yo9&dxC*jE{~byN70`_?|=LG0Vlt9u5ye6Jxw^{fPU9e^4eTnEx1R#g-R(pjmaF%LT(E60bw?2$hXs8bx>5;dHEl@7B; zr@rY@j%S^b2wZY0CbBp91$shSz;z}{P3Psq+Tg=Fl)EleHt2G#8oFvtKxpGF`ToXE z5klc}t!rWAEANj)b$hbT*7b>e`S8Pq?H90FcdJz#Y18DWA&o2AJE!GC_nwe2Jvk=& zB8RFM>fd`;Rg-78-*q_EdQ0@e28`VIfzvxDwO)cJ=l92D$L)R6@Aq>Wdr;_)MT%5z z>#fUM{S`ix-<)5g@?!O%_XcF|%X-dEzlSa&x&kbH%M$+&VEpF;PyglZ7ZS27p#ezA z0|Sx~(z^fcK55n7;YXpeXM=H>0AF+;BS!-d^5Y8pB(=shP6xUG1W4LzuS zh5@SnW+a{B^3kXs4}7KTmmxb(j^)FgSbzMC=1%P^4sIzLVMM{KISGGeFZtuXjmPln z?4c|BjGl|3A!7l?3)IwjeJx(biGEEzq|t&=R9r}Zw-_jGhA;?{{$Y zW`>g({!BY1(+AwW2gUP;?~_?#(J=2rwo#)SHmevT@vNv>@9P?xO+GU7PF@lIT;TN! zBJ*CI_feN%i}`hM z-pa1Z8EgMgX2JOnTL?qzomrUGH^^hh0|-;dpVQN#)6zF@9{QUQIl+QdhHjmgHT(Axq&scTmRSThehmIBb5x`)DcR zaa!h%Gr%JizjJW2>U>d&<>r6($--E*+$v<1Qfp{xXzO(l?`G)IB5>vHpnPbnfe&$ZCtI#&qmohRJviZ37jpE zsiMYZFqEoI$^`Vl7VJNnO0XYHo4LGTGZF?dHDWM=+%O- zvJkS>lzp1N@13L8;(yhXtnLC|wuT&L*%!3}>f1pC^`;jR^G3e9J^%YKJ9WX|k-x?M zqBbMJv4MMJukf?6^7C-X!DrjBi^<+n$s|t%Of zEj|Fp(FO?O$6J}$-g-x*LQCx1lz&rd4Y)+upH`Sa79>Xs%J za~ih9GUf1DU1)T2R4!Pfm;O%R@|!d2S6ON7&)dzZhk6meV`u7e#c1|l%q-L@Q4I~L za4yPSt)B~6=E2)b#zb*WlnK@^hm>T@g);q$Dqw_$oS;JA5wGh~$iOce<3mUA?DvH|Va)NATBO znAlx!f1t^E>8@={7-pL(C^?myaE>lkEiD!&8}V|=9(8G ziEd}J*E4bx=0(NK6HxSL*py!MS}8xj{a-DVGRJg_yqG+etG%k5|0aH}2<#<{jSEXJ z3Ix*Ky9l|yxp@1E>{WBphbjZk2E zbGlsfH7|eHn4;`{^OlZWrH>K#>ZnY>?rrDgJl;>`yunxm8OzLyA@BG7v3l`=M1&6p zs&l46{J6!o>_ZJWNt0@~H~ZF8k3|$g=2L>8V}YezR(!!E_xaUDeJEjOC=IH80oe1m zY%}qd3wC_s8+HG?Pri+kT7pZnp=(VT`_AxfeAi=Gr7eG_D1PgqA5h8|_GO}d^X&4| z1oOok{MZxh;d_1t$$ZlxDLe#gko22_4G8^gcKb(oamvlV!uMB`rm~q@O z7lB`R7eq6P_{dJhw7`dLJi4A$Y_{5X=#XUJFh71jC31;&p&@@nY*9;r@M7lhr-p#P%lc|&0Z18!gjKC6#DsSPjp_=(6Fi(tyZ7cw#cN;?XZS1^A* zL<_!31z*=6=dB^ZC*v;jLG1+ zUlfvhB2Wju5q+nP${}q&+J>2($TdtVd^?gSa5iG=Fa`1>kU1DPore`v0ctn_mq89VALuwOtLNq-@i z3PlK&(EMUJu&Y6Ym@?oz9`1d{@6f`!!sVMZj-e=vPyGDK^g%-hGI_VbyIRhTJa{IB zsoYT(V>jV0#OqZ@@?F2k9R|vF;<%^ly^ZKuBkF_=gLcj;P}sj_F^KT9a;|^ZP?2Z7F-7~G92O4d zb3cB~#$XW+PlU9kP0DW3xN)Px4X^sgR;fDBJKXlTdYR$ADRjH7%D7^XWNvr}>BL;R zM8)7Lvd0sLCw@V#P0^4(;CEo%#1|W28lIe;)922^tBoHFnouXd0U6iuzWS;`IlI4C3j zb_v=3f=GO$??44YRX;__K|?lyjkeS-D`?Akq~6NUV24ScN|BfH45M~5%1;zP;-pcG zF}@2+?V~Eda%D;M1v#qzHZk=2Z;EO{O1ZRP)MxV6hQmp5=d*D|d$j%Czf93*0i&H5 z%K#flHET9qYrpYAu;>&F4@i%r81U?z>=Dh&F~b;qgqj!jf2+-oOteUv_rEqqH(M3*jCi?2eE=)`wr3J!xGfy@R}2d^qE{6mH-B$pES{Za zL4L7*l2r`0e`Ab~A;|i$I@PV!01+g8mzi?~=2!6@CIzQ`+=+7S2OeKgajc zEZ0e#`w-8`pLdSvYD34m@<-zLYXaOJo;b=DJuW8TF4Tvik&*`jfyt zuyT?h;YL_Ku30Vrq{Z*v?IqJDUlu$kvTszx=n(lCIvKwC6e%)PZ!axhNnp$`K=g51)p9U(wzpdU* z7Om7sgmC4a`ls*c8`|p{GZq#WK$le!Zmd356`GcetkC^GY%Q~q3&{2fM@JtvJzfTv#6eKl@IyrgAMqJhl+ z^3DjKbL#_;ywY}NIDXg{HP;O5FAgsiejI?(AnE&LN=)_=q)qN+n{eFjKY|zswL`9I9isox0h{rqsc7-VKqYP8|^9RZ% z)xZ6brY^C_4s$1^TI>Paa#8BD_Tqrfd?Wes!4|*aN0El1w5Q1}+Aj!JZcdkW?Ej~= zt3rt>{eE4naSG@u^^bIe#3~KVp(r_Ny3s#mz0K+yy&9ZuE+^A1{b{ z3O_RPQrchp0L_W~2qBj|)qDGSvkbJ@TE1Qv02%B|p=Qy1;|y}U%Mf09qamZM7GY%Q z2c8GFGisu@Gk>lgx5iM`uBtnpc?6*Ky$%ur1jCFSXyf}etE^XO#!iomj1$tB<@KVO zk?}-9~Dd^(tGd^1#w>ekL`eFb2TSs)_Wus?-!T-`rSexi; zG>O;L>sQ?djD}T`?c4h9(Lq_xBLezYOA0V&$A1ERUBz(iPJUM3$A~!UUa$Cb=wm_=*E7}Vi*C8hoVFOuV?)PHMLljLf1@~g@BN8Wq zJmUJ3r``Rc9WrQ=Sp~q@x~oZG$kY4*Z^D&2<#VgO=5L__ppI2b zrF{Ot;Pz!Hb>J((`S1CM*5SJVg4e0L<25Tlnpw`}Yv6zufg^f31-k}_6s%|OSrnXK zqxA&N;2;2sJ6m<}w!~U+qiqz&dPp=aoSD5dsJ^{Ip;T(GphU4$Ei@`L4~IKs z)~7Cv6_TG`P%x2CI@y0GDlGV5zaQ^NCU??*17aiP)Gut-P+GY3>hid^3!f26YUdZ| zB?s4~+QGPIO01vcNwmn@sCKqq%5(=X^FFEvk@j`*3*2Hgla=_<1Gm7UH`*&KS-=( zkY(F2NLV-PvEpQK>`^UW3eaO4Z!OQ~P_(Wi+8DA=y<6g9T_rjHoH2 z??@-LOr8~D-HVY1k}}ItPGx2tCsIJ)DrI@KdU5zaAJ@^BI{wsr*~H2!SR{I+D-fLp zg*qB9%w6H1=!d>nH^o*{vBHHXoYDX1*)T zM7c9c{H>4iBP;)uf+(Z>WViw4FG<643ycc5Q}Y*l<$TDbg1Kh!Y)>!H;&}#$@GCe} zo^`*@#s6XYn8k!RV7Rnj7cPpSt*(;{7^pQaS7d=!MF}cN_Hi4%5n6CF`F+W>M@184 zA#XyOf9q9HG4L8WpyQ;BX7&I~K(oL5fBji2 zh|tRY*zoEd5M6-?a8*T;Z$7F7zVVCEuWFb?2WFMNTxPnj!`X`1HAoT#f>ya#OPsl; zimJSQN8e1SrUk0}Yz6x80G3sWkPhVlRh2<`B#geg^9fY9`!g2R{k0(fuXbdf}~F-`CuP>$S-JyYTXCSRd^86)r zgijN_H>cIw|}TJ02P_XlBz%$gu4fBVkDu_!_K!#Uq68+&@! z+HNU={1QJ_U#Bs$GF^Z9?fjIxFwt)Q3Q>}9wz-qj@>g&6J^LA2NWu#hkKostZ{m?* z)(cxzJS0dk{~gKaH)Vt&^4s8W)-R+W7(wmIeE>qjnkWKsL#cWid0uLc5NxNQ9=tR{ zh(YY;BFRXEIDHhiOq^AOF(Ua3p}!Me$r8~<1Me_m_N=xa9{hzFe?%NMP>?y~g2xL5 zO;@*iv`3t--3Kb=HBzX*U>IF@z+cl9^8Q%v2J{wv z?@)I#7e^_Y!`TVm^^@&U$^y!&JXXsR-O4peV}bTB?TaY$)Y<+;1>TpNh8{z-i#hfWmTvqRxM{A8AI`+K=#e8`sdhPOLQOkU0z2&}XtHf8B;BOc53niiQ?-tu+$B5xFO+Y8xO)PKtJsT1GCCetAef~s6p>U4)1%#Qk z`5neDkyZ;{@8UAtmOa@)I{6$Kvnpia1%S-XbNN zS!E~=Sx6fP0f^I>2_`1&VfAuXcdysP2vaMbIL+>CC+0`2?=tA(sruXC>m|_2Ay@8W zd@i$yx6z4sd{oB$fnQ34Km8zh85Jc+09HUlFM9%B_LILIAE-YUlPeO>3??Xh62#Yg z>1#Ea8k(Sa4wx(_@mX_qeoxBl1i#?i-_6gE5J-W|Fn|dOR8IGV6y~mZ9QrE`UauS;gu&M#J3^xNB-B-1BHAf#L#V?h+af2Fpx_) zD1n(MUlceiNK|{=k}KR1D8jhGbaU||x!OsXXqf%L%G?Ky(3Mbg&Z1a@$F= zac{R0qeV}*4{%~mqNzaQ=?|0yl!7--orGQr6dVo=I81&TQsJikhNTvXw9r%TD zGU3bGDaX~{qV+~5Mw2D!7B5-GVuH)_v8mM5Zpi^K%;}<5?F#T#R+>D@tP7-D%}gFRJ)_K0wSfKuRl$G+Kwk41WIYS8g$r@ z2u^w-=^w$jH``K!t!OxJ!&MQGrsk4V`0+{|c0KQz-ld)emXoqN^@GM5<<5LBTx8cP zAJae`O-lRpCt4iU9x;b>yi8Y0sP3N+lFqo?ltd}u_tQA(Ql0MR@3^7%b(=B#q&s*! zuk>hPozz#tD%lNeRd(3U&#ZnB1*(MppJZd{LNt=!Nr>tgC0e~^hI z8_x$I31D6}K?6MJ+bb%|w#vaRIXQOa%HAGn@G}5Sa^U~qYXmu>mk2Abn}^tY#0F0G zca9GDhjki0^`x%_p~!J=IsQ&hb*;LtVhvS_9519|fItL4H=dJMHWNFQ8<~@zJo>mg zBN4wN%Gp-^2C8PGopXioI<2cCyX2z2yAH(HJLNe%bH&+ zd7$B1-U)sK<2`N9dKyBgoDu3^uPr6?LV$kXi=B$9Aua&%jrZPEyPpOxR$hXxY z3>1iSQeeIn=sjs-EttinsfeWD5#9m*-JAGfeBf#}(~c-F3gIIaGjvftjVYZHD_^!A znN#8!il=I`s(1!25W=YAqA(2u*ZS9HnJ8>*v#9WxEv|nrQXzR;@DFWOa|c85!a|@; z+Gz121~f*zUUQCh^KUG|d*Wlc{Ma5pPyBcf{;f!k8jXy@U@g)@yB;r6k@6>g=bTVRRL|t?qIx?gkT0t58|aH&V_J`yV+GKsd@q7 z#@+BzGV0gYLBl24q?kS_ptV(UC0tuc0iuij^DOxl==IRJc9^w>$V650h0{M4x_$VN zE(R{hE(LfKR_q=cV1-J#-k}LJGJQ(rZzED@4GsCv&y#8iD^;=n5wrY5%Ah=)cCh&}P`tjhAKNQBu}S`&yLVugu!Z5mdU6_Q$1s`jK@!SmjqRr_J0d6&W zS#d3%M6H$|;7LNnYviYDarn=6emGkobLt(lG#SaEc&N{)#XEET9H|QVoTU z$hU@OooVdX(Y0zzDQ^-v{@6p18tLCN0UDEio{TGhs(USmt3J0I*$@H&!hW^gV0W*v zs_a>!-fP<7wPvpmrJuP}a3l^Fu$mnp8@NuyINUkfYc9&Xana zcDkW}${T;{_XNN%rw|5Ro#jmYchCaq6inH=72rj4YPn&})G^j_DAzJ}nK8acL~ zDL$=lT}YetFAlG3CXEhKUlvl-{WFL4T(takvm0<-5b$f1L$uIO1AX7}xsXrZAf8>= z-qXl#U-XAcf<=Q8d`jE^oqJ@NHjE7#-}IF=bj0N5((S8Yc2`{a`u`$M!yyp!BmYiS zqtF>7smj6|&s#rMAL`n=bY}Sep!m9a3pzLh+ff3Yry4a|=xv56Gsdy=pxX7u0SqI2U4Ax7LMfE2@P(Zq}2wH_*$uX^p%d z$xZCeV|`W^|4v@5C+PQJyGjBy(otzWb6G~Fs4vxeWteC$6hMd2_7f{(-+uDu5RS~m zFU=!V35wWnV@AHMM00-s1iM5vCrc&6wB^a(?2#@$*c1c-Kw4Wxjg~6y@E!J0S{h&y zXq#6%%BQL8B2cZL`uyhYh(Qw^7pZDC892?&P*x@l+R>=p5KRdt43?uvyn?cJ{KiYz z&MNNE#^L9b&6xkBIgx+pn`b}kyO1h)neo3BXl-A5mrBSzM{FhH1;#K(T+H?8c z?cCZsFe{76pg(rV=F9Y<*y!Zb{E=Tr*s(2Rsg)eu_{VRF-ckFpOeDb9??@GKaPz>p z;}oz3YortCc2q*?1dWqUtUD|^h5RLbig?k_TmRMZJ6Su`LqMkf?lKaZCpucZ2P~wZ zb`Md%yQho)R!38k`9_B7euZ3gdPca(lI4fN*;#e6<_W7i>tGNoY?2fE8#-6?ZLoki z8`9v~%wFGAyRCJ8HL&}viAxZxB|xGSPcVV*b?G?ocA~#f0`Jd?3EM2HE*#a6nHIdM z9}Se3irZ zZwUUrki+rq!Y>xoA7KLD!RvnRbNRVzkHZw$S(FSDoYM*oABp9%>AZy8{3 z%M)Gv+nKkA`1JHNy7u`)dek$xAXWLuu);pnRI=}bl2u`-=NG`PC>IiAuTq$>=%t!| zc8&Ex4gT_?)2^mIi=ZlK{S(`mP*Vs-;ViAvqf%p7~!GMHE#aE`l zj7Q_pC3Q9!y&sgned{td44-~*P)a*F+3P-dZIR6eg?d0pAH&QCbol5F9U_6PC$64k zkmjw(Q4mbPr*kW8G{h&YFMpPDO1lL^a^XjmH<1@G$|N#b9T*eFEWUH3C+GMV*KaeNc|-+kh!Wh2m8|xhr%cuNT$K)w9XX zXSkebTJAVwjJ*;~++S+9O>A@m_V9{%Le4w*G4 zD0-hu@H+1|^HTH4HQ_23p`BiQCvkh&6SCo?+qKEZZhC)|idfLR;3PKrZ;MVTo-_}a4J6Y6fyXcDnc)f0jCiVceQ?m z5R?h>?%_M-{4>qiA&WAtJ~yVd-=*|@4tcek+e|Y2!Y^02eAA}bu-p_}RsXIDr*hk4 zns$BDWnNt$qLmq+Sq2v7Mbzq&qY%zep?}!v#wk1{AYNxUnC6T)AJL41?%Jp`wLict z{-d&h>%KKL^inX>`Sb`%ioZ}GvMw;_-(VKknM+R2CbOnuYiIFFbmsoOBdzCPRxA4( z&KaApJVR8eu76{T(R}y9pFTfyfpDS2pIFGmpk-;78xLmPa{Jkkaf&3ysIEPs3}d>^ zW^Ps1Y`JzVSu8AZ)Vh9U_jB0E)HdfXt3Kr<75*~(|B;E$75{$rxw`sTjZ;`-JXarA zlI|=?y+ypH^K_M@L~jO1z;;W~oSe_|d~SjtIej6(VyX8g*|~t4ARb_wB2|%jtXsgL zNTduNXn~Z@aKUbr&jhmy%y{;Dyx>{T1ffD3;PwZ5k+Y=FC0k8`tj*^ z${LStxp1Nk3qOJX*l1Kgump-1Y49TOu0ID5FXGdB?}io2AMmC79Ux{1TYn5cm%sey z>^}Uw{E-jT%*f|&kuxHRz6G`4G|y1d48@EI)TnpMrDhS`M-H=Lc{8Tw6=LZ6f>OU8 zm!&SzO?vO6=8wZt-=q$Dzl}a{jyWWrHU+fcix* zuEA_M)Giet0h8+!xlnnL*9z!;^NtJpt&qOL_dzj|!eqQBczx=Qvl8)9_;p3$Mz37x z?-huZ0Gq5tUrEmJJyT;ZzY=oG8*RCPl&={3>#>T#4mv(+E9cMwha8VT@RM}JL*@^#~X zU>{a@XSSUPbCq#jv{|<`oxssGw=hH5buB?OL#RNnmJCS0=(1KM5o={E!9~5V9ddXh zLw-U2@^_tMp>^c3IO_yK+v2~ClD$ZrpeFlYx7w)T*>|Ji@^jX^pioL;NHJyoOyEYFF5I&xbeS}v zbe9Ku7$?wXU1i&sjS=S5y^{ZC9?P6j9T``*k8yggnxW&P`% zzVZ5^UcY($eCr;eYJA&%KrnBw+^oKv@3mxmPe-n7@dFUN<%AK95xDTthmj$7tt;N| zt;iea0N}Wk35^B$4WD5vI^3>5+CH|PYhfQKGN{1`ahS@SxQ({6sqD5Yg@LvmC)zmI z?)jlQ-u8&jzYC+3-1U<$NldSIp^w|yiQvcf%ytR>X#Hb5`-sXJ7x=75Rk}s81A?|Y z`oO12F-ZyQ!SdY!=Y(tr+4$+dfzXH)xE&amf?qB)GV`y5h8^?+eY{y+5k?m{Wn5C& zeaF8c@0!tME!mlN>kCX}U|_+E!9Tf)5=uWk*Z(H)-Zj{a5BlmTfC=1h!}R6O{@{1E z)Xx5#Q%Po4z203!BekZoD`NLOL8t}ymu`7I zsKM+F`^%9Dbq_DK?b0~i6J9&l#UECaEg9W21EgTwa}(%B-nj=9B$Ir)H_E~#Lm*@U zWI&hb1i&7pdxylOS4_1E3E2nc@_nD}B$GCTKPik4a4FTxSlE_~VB&z!i=%qbKOEq~DL5qG4-hD|yK8I*S*d7wc%}g9u99x)H))*YNVSoR}HUNnQ;CVP9t#yv! zVMI(A43fj`=*?Bb!%v-*1^9ggg)3tzJKmwc567oNZ($rEHa!6WwVi}h4lf}+ zVojBn!-@*BltCEa_CEGyga_sUG-Ch(~3q7O2wBL1jqA9Gqd!L;hqlJ5$0Q)|a6Jx#1 zmk&O%q96)v)SWmqzLF%GvOLLzFPKo2vXk0JElD6f|Cmp?q9MyXwe0T+WJ2-IyDxLU zK?s(}jJab=SAYR-F4c&xr=bekOi=nhLjB9@PaphB_r8*h#+~i+Psv}lIQ@n#D`PE3 zspPHm_s2eAi`+9_-Jib7c<{PCdU*$IWL7_1&S7e2WK4a3&c6VKErc0|)Bq1mr*%Q+ z0tAq^wmmHpaps)|W`XjOh;nKF0Z;${`@f&{|Nr|y{2qwE1MzPl{tLwa*#7^o2hup@ z>;G5(uVo-rH{*W*1Qq~gnq7)2TM~u4kfN|fL`3YO5MO!{cL;Yv=M1TdG4#`Y?MaA; zh!lm`A|f8C*2wj(*q(k!G}E1X$A7Gd@W+4S&)fU`db?lq`!$X~f35D<;eMU(_d`9u zU%yWGYkj{q_xsuX^Q-&)*LU~Z?d5(=?%#L!Yk$A4+GE~5&$Wl{tj9Z#_1xb+?$`8w zf9c-MwMVUa+z)h3x;=m0YE7Oo5si;Cx1#^h(Rq6njW^LU6b(D=|EV<>@%Js6ylUN6 z^u6lLoAx_)&vW08P9FL0TPLG@J`>Gr$*U*%iqCxZ-tnID>xs{?WGcFnZsfa>49?x! zyuXTu@7gcF4|UF`^w!g}ukMW7sPxVLxn?RIuSCOMYox>6Z!3Bp_silv-@8h-XYqQ| zZ!e;4rDr3_7P_Q^GMU`T(-WI-F-LrN9`^DNEvP#JKvW4b067YUj5Q>uHF1*Z(ow_MtYix zr}aM`3x8sQdmY`A$;#YQ<$K-tv}PprqR!oKs+@~-g^4uhQ)BY)GFOSodv3%I<_R6| z691=I$LuG~Coa&_diZ&hSPl~VL414`t$EI;ej7_XU!wW(&R-JisiQB|Fl$`i^(-&WOMmAoaVd`8%OYu;7(LPt&sl0Tt3W%wn1 z_8a+LMB`_(ZY4RaB&!$M*idfVZ)jGCNxi&EZm-(=sC@|)N6FHXn#_j{bHr>V_ocxtsZpFNj!{1`&nxhDpGd}y(zCvV*|)#+vps7n@YDkJ-bDP zlMg-r)`iYVfum-ROCvw|YSJT{Nk2fIO38T+sr6ejyXwrPbb1!g?~*P0eGfwv>r=Lu zJS$Z?-IGe!vL*bhuRWI~>-_CQ`|QNuQZik4_TX^K`{$^5w2n&cJJ}EE0D4{<-oGTv zTQ~;geq)_A*4Z2BP(H!={#`LH^)!>Nez$4aUDAqeAH%5zKXt4hm+d5df)k{+Jf~M@ zCeKUR`LU7ru=9^xvlCt0l7ZtOx)JO$1X$@kVq(AHZ?6kD^c_zzHR&WpBmbdn2Ok?~ zU*Ma3-cUyn%aB1nW7^I|U&?VH8GTCL1D(BTcu;<{k8H&EBpi41%tcfVn0N9{eJKwZ zo=w8JsPp?D|CAH}!uv2l&yz28s-?Bjx}$akkbY?GL{cXrbAO78uKh;RWQDD$x*440 zk!uNaBLh8!lWxAYE;7^T!F^!$t;iz5g5P?L6686mUQmLyzzI;`$C(s3ZH)6Z z3U3Z%@bf4PcZ~`F7zT-!AghFp*z&wlI0>W@Q6|nqau__X!^rd8eH50Kk>@jN5e3?m z*HE8>#(@-Et*)(1VAAlhiV9fB=T&QP#&cAia+ZvQ<@23^o`ZKu#G@#%@!ssMWM}ya z*yKf!z|3!rJqg@`*TLdx&MWP6l1u@Bi!kngs$B?ZINMls5@c0Nq%F*%YOg_K3&*J1 zNp*AT5r8)gCv=1IHbG(!0I>(Je}zFfz&-c8Xbk~SK*_+p^}Tf%IO9)*hXdwvsU*ZyuFAsHIiYIN_kjeXU`?l*>u-+?W470UhdO*|GsTyesLF zcyj%sleOiqMC8B>pr-G!3KvICyZ}7>`x;e`j>E~)Nv8F)k0>BTk_A90S(#5II@i(g zMp(aulaKX;oC)i14ZnbB>}(lTtG9*Gp>${P2k`ocN?xnq_)vjoAEJ+-f(`pJ&_0xY zlZT`1;T%=eP=u;*%->g0bw2u9M->F>z2x=Q(9-~2?1lo)xj}wy{i&zrydltGaC(gD z>GaBulHOSum|1U%uTMn}@)^swx8Zp1ctN3s%;~{k-{K7!S(nQ&utRpa$D*-CVh{Qi z(01WK?`lzccCQLx*e`p{b21nI__rk5mgFiuY=!Hn(vgf#<`tYw3D2Y8_GdV=Q(pNu zApu8EG)_5Ac}0n;tPK3lvlj6LrVN-AZ4Kps_xo@+(Veo-#v_4}WGLwQBdU`(2$z8F zq*DISEu1zHqT$z;f31DUu%MunQ)AJ{&RH2?2eN3ogUM@CirYJ7XT}7L&`;MPvPxLLjtNR8#|dX>leE)fxrvr(!^5kGcsyjQ0s^}R)tfN2x1&M$$V41{_*QwDJJPzc}fbUTl zj3zB_;ek@hQpwt(V;hw?dU{Vl$RJBVGkI7wVWa`M*ma)OD_3<`q?4+Im%K43V3@Ki z8wMy*AQ*6w(CGeZ2;+eT`Wr?Q3Zj6l z1M*;4)o<)=Ci?#g6e<@80|*QA5} zJ&kE01&xqh@lWxRd#dZ~ZsvPbyppeZ5KHr;DSDlM=veN=y8t0+-C!Tc!4w9FKsS^o z07Et6C`CTKq~?7JXqq1s07?7fo=HO6gL>?_jEXg<@&}qAjt_!+2B51(=M)#~q9K0+ z)HAFmcF>*Hq{4c>7XKNqkkM72Dlj{+{e#)Z$MY`C0NnefQwNaHsL*{x&k6qe-e3Op zpTGZ&&!04ffSfdX@L8q{Og8$ZZ{mgv+(i^T2UDmx%C!`g#6=aJj+qQ|Z;dWAxOA8F z7==|-Ts!A6pErU1F8@}Vcp!kc;ZI(wrL=Sd{? zy@-lUlt45>l;hBE>TDT>XNFM|CpGH)rwRJonv!Fy?uXAxNA{z}?)`@p;tCj~DRDjF zm^V>Ce=f*1G!mP}lVj1Lp%ZL(EgNz(<ZQ{viM^(j<*(D1Ro&+#np6YZc2qVN#s&1X?)SICC0R z>Jz&g{O|SxGS&15Kk=W5;F-xO7;;=PHZ}qK&_rBDP3iC<@ zALdb0B4UAJh0E+&!(&sEwv%&kgzQmAV*V&kc(SAB8RqK zP-8IVg56Qqe3cxZ8dz%1x~WU^9vT<%KSl%qWt3ftgD@0@5s^}45D^h6()Dyj^DrxN z8C^*!We^c5BBe+To#f_AlXf0LH23e^b5C+mS$kMVy_8;^UZ?h;zVzPfHR!eI|4Q#q zy=R!I9Xw zRek?(dIQl!2a)w!XT8TkHG^ue*T(uAn{+hWbBVu_>q+-f@ZDJ7_599^i4kNu-xFQC zPA1=lU`Vl*$e~Wr=^lI`P z1O6P}Im#*F@@8WNE4_+Box0pyJ7Rb6mVtSqUPg_WxE_4u6`du-+^JbG(=5=~0uPDT zgVf$|GX`=wzAEyS;O{suxRH6F;|~7;pNyJ=*Mrys9n1LwFG+yy!F)!}cJw@fL6M^l zYPr{;hr{aQ&P?3VXHXyAN`F`8w)Hlt?=;Vykwrb>5DiLA_d2=RRsQ}8zM2}~7JX#F zOx>N&i7lAT4E9s|ddY;lj-2uP-4;&jU@klCMyq@8|C+}GPJ8eqK|jydzpRDtN4Oc^ zb?e>($NNj)BRO*VxWh*jXtuGMa`*M0mU%a1jo4M7(FWVGeZBa?rPJ=>`4Yb?b=-^; z{_!6)B>)}Q{XY{3;J|U0)HSEBJQeBjNMlrIbaqOf%G_#W4{U}M*=CfssYxXvb5!yW zzBKc&*=(Nnz(iLa6=B1M%~qY4nlbC-p}6Y%eSLrV{0r~bC*nIta3DlF0pE2?^1Lpq zfRqzpI4UD9h{icE6N)7Q3L5Ea^mO27*|=54(&ynrR ziM3CPa4Q@AxrGc$8ajdZ7|vd?+CO2KpP^n@Q>u17_k1ToHM(U%8Nc8GZ)t%N7=d## zz5n4@(rr;s8#$us1X}N;@VuEG(VP`(v^_^MO!R#Qf2xC3I_ExC18{&@@zD-gi>q}) z+TT%P*|L8#Y@TF}Gf#&z$}sz-`P4aZz%@!Jv`>;42qz&k;%aa%MxKRP4@zdW6R0~r8GWXLYjiwwIMP|n7Vpc zu=9XlSBe77#soD@Bl(VDeIt=U&y?)exO&cxRxEGQTQAmCr#BlR7>50MN2gB|1dXwc67kz_&YKmd)n6G$2YmQu?>hx-g-k7_f2Dd#<$Xi9>I~yY#fm z5k`*nX%V&8b?P*ip&rhDXs0kdPAH|U)~M0WFmpYEYtsN*-!OU@D>j%>J3OYR!5Pk!V}k|>@HqSBg0y9x3HtZ#Hu zM*CCY^pC;|=gT*RYj4|JWKBnWKhDphsEQyxh*;JGy-#O+P_GNz6hgyMt40B7Nl zK#Y&j`VuLQMC^Z4@?m*hhsN8b6Rkdt0!(4-+WGEDpCX_KjL=U-9^?6Cpxi6mGph1} zrT?$Yvq?B1uM(I2I{(FfmewZF+-suPIuNkA@CHB5-wHK zrrTiSpa*yeP*p#=c&NtbqtQOH6D{}7{(=$YA0XrWe_tOFq!L~f#l3PqjEoK*Wn9+Ob>S^O;G0M(qJ-8Ji3Tpe|oXiA**h$`eRI!q2c*O5=!(=gsF>z!Q# znMo6W0NSG#hLF&w7QgH6H{y8mWD4bW`@rc2O&ruTHnsqBy3>cx%LvQso$wS%Jei&? zq#>%e5&f-TasMk2M<78-4;Xjde3HiRawF6rJ7eTx9TGHD3~uy?k|bk6_2B)|xJp&-zLZPCO0_{o**CB|*JJIK)TsP=-w>$sITnC&H z8{RrBVoAFc^Va?_J8k^%=s>$6nX~_oa&vO*-$;6<|ItFr*Mah6DL{#?Y>umg(@mxAY3lDK#%xQ9TPR{pxtwFC_MY>8wNZ^gr_GoPU zUqwGw$DK{i3;?Fx z_jF*s?{zJ{$oNIyyS#L{0leM0YQc&M| zid&ck$LA0!z#md+x7%}AR3tYw63QMfd`G)po!{rugPv#swwu*G}{etlIX66z6#`a(bw%Rd4a%`@zr*#qUIi2f;_F<6Y*`H z;vzHqpGNsb+||hqH|Yxki{r$1aR99{>DJM^EVIvoFMc2fT*^avbjQ6*Fc2aB*kP^h zrkX5?vzY<7H?-h^cq{Yzb&i8C`TYbjdueyA1#MpBN3I^9pF)p3h8-bT#)s6P+yq4w zreTkZZBUvuLd#k_+bzN*V$vAmakmep5?*@q?7I_yLbc$v*N;3<8b^M~eh*?-n~g(_ z$U|4^|0fPpU%V1fo%MEZlW%0GD#%!5D}cGp%tn{L94qX_++v!z&x8H+;_&Ml%F$M9 zvfBgP?0|fur7*LO?=)K+TK*Gh-51*Ljj&D%d3#>=Ndx>d`cyxiIX zPt6i2tN-9t%gY~IdP+zlf0&I-`cV8c%i4|u>(txJc5lM)+p;busu6bgCPN=0k3I$& z^izJ`yR`dqQE}Dy^>b%@ce_XU4S*+^+HHsKXa4a-c`WznziiZ^vHu(-TfIRBe&1p} z%zJ|)Wen4&l|3v?Q$jSqGf$oUkYZU=_nu?Egl$*Q2MYeS2|AKI>}Uj>`69igsX zcw7rwpb;Nt9&Z8qpCf|D>T?#i3Vd2-Dgt69X62n|mlG*3@3k8(%vN7Vxj=IBRpnM<4xnsy%$X z`=RofI6Z@-#@KxwInura5;z8MT`m(hBC76GWHg|z5jMuy9iQmCZ?*Te$b@(`1acxu zTsIcY?7e8LfvYb?KpVvM)!!l$eoIp=M`h;;w=m-ml5N9QSB6VoimF=bcJN8pIweu% zEyz!;f%6G>0~o$&XfHK{OQt##GO#h~Z>7tu)@d3Ub*UV!BDPasO2}y(d=q!K(Yux+ zVTyJeFZlx=%gZOc9#lJC7tDn82aGW?D7{R<;GB5?mnTnQpn)t+ovtmj@yd8#uBTaN z@ka1cpwv;GV{*k+la=c}ln_E0Q9UVk5WuXAyJ_65iOa9RDyGG}t}K;a9_rbkTxyQP z5TEkMd%~OI|alh)k=;rS7(^;K6S=wAsFcP?=>>gSAv$k#_ zDHo$BBL-I&r-Wa3C0O7g>xcbbj$IRytw>ELN%v$?{M1aondK$_pq)qL&t72woGd3^ zC%T_3w>?4?c1~9%bQY@447=SQ+gj(~uG@S2Kctp8WoyrJi!nd~Uat?Mm5iRM{@Ohw zzYwxVy`b>Yv^mz2AcVASSaH_)j!w;{1bbFrAqxASg!p*CG%T&v{);-PY6) z0*+osWpl{=^A!Zr;gOOL$TwoqXWn!w@c?4~ArDjb*I#DtrUJ!^Z;VCg#IzmR^ooiB zx4cBbzkqe~F9btvWZ2dsr@sOZH?psk2Zj|CSOxScWW@A%#(KiOZd*{N!*=J6x{7W^ zuWHe3+`{}<&?#)JrdPXc(&_NSL0h~ErQPsUWG>P;HTeg)+F`c6P_)#LlliF$vs_mhl&W`*0gX%T1iP@y3YAG|kxQY_YU;$>g()#*0sd_r&}V17byU-DWw z5U;Dq_fDDEuvT(4a|JeG-~!|S*xJvL~R$;km|SkeR%n1$-x?p z#i1#(i@i4fc4sZSXKo&MO&TnRn2k-eT5p}t3tMedhd#&LYsqgA_$ToW;*Q;%)QUwW zpDv)JDe%I+F$a;T!{Clnos!eEikWIneo_S;nj_oYPaSM`aMgj*R*BICOI8=z4%O~G zxW3%5Zqc$<9Cv&^`V#Q_NZ|+Tyt;uw!`)z<1$Lj_Q`Z<5ASjD~NVVDbJfnm#(|KE@ z6cH|XZL^Qh|JU{0DV6z*_@h;*L{X4r8g92oR`{lf)*XjWtWK}67*EQ5NUU`<<35Sx z9@n0h+Lp5Fza-zl0I)jC8h)mAdHq zYF9E_i|>&)`s2x}po~%PUbA=e=ScJ%nYoiqepls;&fy-=@qFR%%3JaDL(&Pfc?le< zQsUSS`v?{hj1U#0UpU?p7d;bvYVCdmS_pQlt?MJR5xiGOtUw{HHQTFya5;~KUZ@d% zW-ndT-+(flqU!U!wsRe9#cRH#QT66L&yxCl7jE4NI#TO&#{>4%^C^%=PTQl{@Tb+@ z=DH7@jLM3c$ZyRw#pAE@N$hy6WpwO&^nrG&$&8vj`|wSQmKx2p2zB?0@P0I`YF^yk zTNyeBiC+D#$ktO#Cq+*e_w~fy27|1Lgl8_vH10yg$;KwQet$J6iHD zk&*)-+wiosQ@f{-4 zvJIxetM}mNP{j;kF@bR7l06%X?)8v1)LnHh6Rtg%?;8@ntIdDQwM6Tq$G+WJy2|y* zOPcVz<~dTOA2nw=y_$XCM~sv~;%%L44IYH)6yk3q>u!yk*zwote!CsX2+PjCYTm9` z=EhN6Cd1qwc$GI)g1E*Q-cR!0eA+^z=FrTqh%}v=`<^x_a)cQieZBWjd#@B9ZBzKT zW<=4~QNArNR(2HcBD)9h(YW^oOJ_SoE5HM1a03|et?)+;=|7Vl5?41zSJlwkyBz8{ z=4HePWWlse6Cd1~sE*{4HK8i7egF)9b&njh&%_Rv}2m3vn=j2 zMSCksNX+g-AVoNLMPZ{ZdCTlJX||2w#VWonpor~XJ?0L^V(TY&^(~*Q0qtnaL9AuV zSMNZo1c6f0i4ey0By_;WQDhe0@QA#+mAEg*s-iaCCuuLDvm1V79`k$aKuzOaswz+o zourg?E4E!Menfw%?Cw~?Jf4npo=W5%R08vgc1S=&1Vf@-3>J{#aiSSNpg)7MGa06L zEZ%9R-XU$GhI~S86O8`aleNAXL$>$R9?M_!?sS|Xe?jPL!v4uwL9uLtxjSN|32?zM zHMh;8jJGKw1gt>IOVv#S3yIUgdaJQmRq4q#Jdpe@VD|BF;VX*kC>||%5o3M@ld>kl z)RN&16Qs2w)T&U@-PHA72g=CH3XS5?cJc8WJbWl?WZC}$N&o<>|6l#TmVx#E|L;Kj z8;Jh`@jtfz|LcMDejvUN#P4y*=lur&3>E-gns16KSrWxZib8A=DMTC*sde-cdL6Te zeoGjbjSP1g@9S)$kV2#=L`0;peUhV*=XWBiKO|XZ-n=(%{5>ab{pi{szYh0nd%te? z>!iOo_v_mJxIN$R$NPPHzwh+?vw!#Na=*HM%x!+Zp87Gj;r&|b_hamr#ti#sx0#+T z`ZaGO&3o?uevFH??%&=fdj2BWx_+&Y`P%<}%=P#kbFZ~#S7f?>|La5h+G>r5e5x=mww%c9^-p_;;O%|nq#HC zd`YHF+2cyzKc}6n*K_f$nZ|5-J;YpF$$8X#Z?dgqddL#r+_W#*54*=V8_jVj{;<=t zZ7+9Rv)*2@?zzOkO?I0}r>8P^CtWR5m!ZeG{Aw(JzV`ME9bGgQ8HbW>tabPD@1yi0 z8^peTmtUMEKf0Uh`;FuzCguAPGsNLo_nG$nB7I!6*R|$cNViwn<30BlI@)X9rN(W` z8u%jmK5LI7>0zyX5jQU-j)~W?^obwrw8oM!x(>-zwPB)=eoc2 z`w!W)SMN=^G1mPloo;j9h^<|7ehe80+RL!mj$A#_-$z}Y{AApZ4I57Me31LH>`yK8 z`lUHn#oEi}`bNFRmv56l|0QYn{fev}BS-pDS?62!dMP{Xau?wVH)%}SCp1WOsG^cF zW8Ac5WmyzWo+Q&*-ZaT*2`zLQr}bm!MXQ$kROEh^_MeJ}jlO$YXlJAdnKf5=d+1ZK zfSyquqxvm0mt>4NmYVOi8FAF8jQ6q6e?DIP`5%dglXVKTP0M7WdEr|r@nHT2-NDQ<#!O*o#Qp2LcL(9Z{>BooDo>~^?PS0?_U8>Q63OIO|%%h_E zkCwrBE6={PCnXkCi7Q6206<}q&tj6D#wwx2nuNho&qz?6bhBv&YfhE}tvl72v!2b% z7!tY)n`IjP&2_tym;hKGEfd3&WV}d*#lvNufBcpa70ZD8z^vr&&z5+6DKVA*E#kJA zkUFtznWu4`snq^fp4FT+mzobPbGvFx{Fb{##0VpXswSG7`tjBh$5U%%&d3OGh?`tT zm?3^2r0S8}ZI!NwD;X;*?}l2(?OQej4}^Y>lAGLmDIs!58-$J*EpzuU?GbAXTJoOk z-BoNHLAM}aGcz+YGc(4_%*>7{cFfGo96M&_m}82WnPZM2W@gXLe^dYxC1w*J-AE!FPI*a;$|7=b{-otN)it}{PW1V5FbI@KUBt7T3m9v^{{4gcBLg(}CQpbOx3N-WF6x5*y|iQBU~RnpZ(0xR%=ehV z$305re9IHsHJG!iMf`0Gkk$W2p{hZ@~4KbG{_X_xK|E%t35FMB^;d$&LO-(k-nU`n9yaR^#2zZlM0mqhah z?}^{!mlTmCESR%{x~+CGAl+Qay<^0)!yVN*S;9vKqOb2zY46X}p zNUh>Eyn$_m?B*}qulXvNUSU%H#xift)=EwmbS65d+gR7|Ol`%6mYsedC4_BVB`dYi zxhg9VKS_3mb(x57cY)HE!|NlAP$XxfZi7Dr4f&xxsXn%7 z{bVFcu9(p(J|VpAAaC6F1*p|$hNfSizkXQcV&9A#RT+kp2-0@2a7qOriqjQ3LRF|w**Vz(^y8DNeg%)ks9;=5ME;A!{lP}CQWgu@?{fDE=l+%_)Sv)5-xK58 zYTyg#s5w_r)Bf@^A09P9oj9O5LZuBU0HUp!-_TX zW1-TDl!(=P?0wGUkK=$E0&X7HErktl6@!4*{g+beo1Lt$kCoJaBHEHWjzdFEMusvY za_U{sR8Ol-`1n_pi|Y`6rU^e!Ls+}G_KSi? z*jD5T?)S%m)IDYB^YNarq%6*F{mlsVD7{jFdwl_{&Y!riJB(?4gTd_bXRF^QFD-a07h-DmG0;BNX9JCy3 z6yvu}vs3wT-%XnseqQTw-UP|bhH$FD{!k5%LjMDMgtdod4aT{r2qzd|#ln_D5wT^U zO*QQh7Q-+e9RHLD{-NMlWn+35b}7VN;P9M>#`Cb=q>mYz+Wp!w9U?hxH*U+Laf`&v zeZOh9-BDkkBau7(lVG|!f;stX=8LOI@WpKzb&;RLb~2Rr7sES9b5SmxUDM1wt-zKu z(rD;hBP4@m+1WRmcdEB&cWC3$`f z3%R5fO;4god_xeZ3+=f2E?>fQ3Vz_`L#^v9>OPK<5qSp9+}!L6J~b1Ir*VLt-x zy<4e2q=_P?D{frW7#1r#ibn(Sf~NUS?z8iiyDf$j3sP~C#WqOuo8e*@jI&`1a$i~c{lMTlW zSvHjsIN=>wEs3k!se-9<%k#6)Ay)&Hj|)8p8TXJ2ygkg`FkwS=IP1M1T3sFxk`rzu zrIImBP&^)r?9}nMy-FeQxXQnG?4{NZN)yxmy?vqz`ZijZ$PPM!dPmdZr=)wse}HE2 zdGFyZzIz9On?r#qrTPC`^G6#(|98++IMWZg-@*Ny-Kx}C6;`A#O`|A{!a~>Cw7kis z2@H!K^XvA$-8*^m5+%+Yu9>fS^LQhSkB|SE#hvwqr`jU0zm5BKXmxOPYn51}d7@G3 z3H!pYGX5uiZnjqT``_gz_J^S!vgH?%&5i@&E5Xoe_s0k2zr9gPeL5P&kmXY#s0paL zD3RPtQe}pwv#SI#Z?6icHXi*d#t`9j>z>=f)Hp*-#bgy#8;`H|pDefc+`-Pp-r$Py zf18ItukNBLTFmP0(lFGF3~GUMZpzj3-)oE%J*6vtAiRFI?~68VTO2imEa*Su&U-)i(^hus4p7gWr2KxC~LVe|6GYJa>$EQAvdt)kCY6w zSQROhf=Q!8sfdX?F4fTb>sJIJL|ZMOVGBy(h%DU(Y?<_c26+E8+n1hj9fucnx0(gl zKe^wNOI(2MMBW=I+azs;%HLg)36kz8_8sEu2beW6fM6WjK$G!yLV3C9~o1kG7WJsVKG1UgpdOR@Pdh^R_K}{9y)zKu)MK z0mU0<%d(p6`3w;2BZInQ7t#VQubUjMZ3ER^J107X>b{~PM=X-TP;8HWg|S#{8NPh~ z3QT9|>d;~k3}hM=W_{&%f$%Kv{fecjY4`^Tzvuk1?(<-wh2@EJiBQEO=gn_<4_-u% zg{Cb5$aKA5mtsX|U9=R=>jXZ5hE77__$vr_|BA3lxCfWR(agpi zmO_e!h3xtv1YOcHU9507dtY~#F#QNs_e$kku_(P+_kvG_*HGTItBpA%x+Rjbj``Kz zm-lZg%(3$~4mh)yu7(hN;c`sVc6WKqDVNq2#KxPyDF>NxISi6^gb0W+Dr{rrS)o## zjs5JP9rpC_$<~}G!YUDPgt!A|xwXLJguW(UqeUKV1}V3~w=TqJLY;wVoVETzTFj2Hjy zykz7{7Ue|m{R09@(qcQ3-`)cL!>UbEB2z5HI7VGZC~=L>gP2UpjMM}PrkodruKTIR z5QTAaiFQqMzl}p|N)k7iUWw!Dn$KmD=rSV%g%e7heI25yJafl2UanDj6UV!Kr}pIu z!(3;od*yyWWCxh2ht>G%>-EB)fR6X>Pl<>p7O~?gO6{!wGXE$A${@>lv}tiSlp_WD zZj=^s0Zm&XvUH>wKki;qP^$I)8mDozw|6Ps=`%5R+WtnAXa@<&hA1&VH8<}gHflK~ zP#S>B5C2z?fs3z5IL z0jquxW6HRGJ@I}NUFhL`4OACdIr!$fUs&ExsNY}@3TF3nmN z)jK9lfh$1s0`CFoRvp}@Qq`OSLERn2jHCCT>`Tn;6R$+&b)kIS zMfx0@i;jkkRF=bsPNA&A2k;dXhW8tM-rdgeq}V;nNvDp&oUW1{9SUqSJ^q7-+)v8c zq0zAv?Su$15WM-f@ok;X4He6Cca`puOT)#=cCSs}7}4u$)$+9w*!RzY*E;!!W>KM);#Py@ z>I`sSTfmSf54fH{ef~b5+VH#R^IZLn9(5A6G-wY(!iqf3S()(5lunjh`}6LaAD1Cl zN5Ln5tnNgPi5oo(eNU{DwB3k}7oz>vcvp{!uYYGXS_Y8q~GuQEl>-V zYJ2fJylw;wdv)bjmc^!5szmr+RN9}>tq1ugj$*&Ak9G^o{@=_K(Eo{5S_q(u^;GDpoy^BXEl^KJ4_ei1NMe#)S*t)yT&hTODP3dRY0GV@*? z{*0Yi#|W+HNZIwia!EOyo%g|5Q_;JI%9t6uV>w<}^>5Rd*`bJuUH|;y($|- z+FO?q6lTef97~i7?R%y;zLMbg;Y4~T;b*d4o!i}8BJUeEr`<`DY!ou~9_euzV~xJl zM&>bG=9#OtwW;a!(qvmfxiZTBNoE{eQVpL;6`|;4S?=WRGp4&+LsvB$nwdaO?bLxG zWg4KkC=y%Klq;ezM0>}a=AADfqduOUn{bza_S`fgN4SI}8;@Fs$m>^=lAB}p>j3qy z_6hEXFgoLB8Wm$~88~bDZq}JKDtdDnq+@W2Jlu<#G4y5-%a-}hzEEoo6nmB~MXA15 zL}7ak+;y?u`%+n)$g-!Cq~$)Mw>|AOVdIAQdN3lbeF<=jLRwx!>X8m)tO3!c9R{C4FV zm-A(FTo(Q1)#6e%QXTU-xAD2TyYQ~53O8DICVYmYyzQaZM!9RyVS=TC`JnER@WN$( zzT0R%c)2;R`iq)H#Bz5B+z+>in%@=ZaCP1RUc+U)OIqNGHc7pHHr|sRH0K&aL+JrI z*;imlVJfxdF?f0CV+<5;y@>mSDhMi9zY#i^J31r6`iwkH5^E)d{ZpJGH7mITbWZNA zp-$Zke{=px{L>(6yK(DNqz*zPk2Ra}bL^|K>OKi2Vty2yEqdxYW!dt!O95S$oJtsv z+pxZ0)8&8$UF<+BP7v7vEYS)OGn^}OOt?GtuIML-YlaPh9)$A?M>yoR@(|7{aX%O{2q$Njn~wpjr_gEDvycTd%Rqt z8%LcVg;kUJNl9d~&mUjpb*aRTgw32zL?dd`J*(g8W^QCS_FpNeCeeuehBrfX=S(*ImDmU(%Ip(Jf$JsV>cE?E zmZ$u^S~p^k$_Vd%i8(2T#(k#g+A|Yc?Cn zbL4u*^O`t>zF284o!9Cy7o__|SXcWg?o?vj#<#5(X)>n7S*he}&|5!hgIDzN%evsp zvW^~jFe0EHuxgo+!j#D;qUhl3M@QI)qS=40l770l4{URLteN^)5u58QQW9T~FX5>t zqviJum|^7YUP;RP`Dv#Xxn86%9ebHNj)cZhs|=-sXqKAD?je=9;$>$GNGVNt;rc;Tjm~6s^=E#BA%R|TD4bhj|*7vV@x{z&O^)MwKdKa6> z+>VzPPx)#0nwP50sjzY}{T`P-?R&D1F8jOj7W?ep9Ifcx(Xd&N!Y+?8hT zi2Kj4Rk>fnzM}aq?-C%*_jgqY=f#*x5{KRM3c}}TO(qvnNC#*AWiiEHwsjdK`MOm$ ztF`BbcM5YP5+iRBA+RfuipU=!5kpN)pQecO$j!p8iPl;L)Qt}NE{6D1(N z!sN8X*su!*MIf`{1ObWEW34OC?Hgj&ybe?W$z9^@TM(2$X!&Fb_wO3CZxEravq?vV z*Zwt>XUS5o8)2ta?3uVWiJT)&`YNz*1cjxqg|kVR{orof+i)cB-6LCJ3nf*Sm#*IH zrq^4WViHK+!^5UXq?%M6q(_DKm31pIhp*M~-jPX##ikZ4T>c^0VO>UeZ#-(ZE1mcA ze}Cj5#C`qhJYw;hsfe84#$0pg4>7<|Sd7?EWHbarnh`y{dms1S`q|l-@P5vj9QNhT z+`-L>^@IuA8}$n3#r|TRW3uuR#GtbRpM3b{{&n(IZa%eSU77b(%Q5TkNX=bYiFJm* z|1{oW3u(Bv+O}8ma-&ysaA0*Y|LONyq1aLdyx(t|V;uK}UalM$68&W!%Vz%XVHn;) z#nYcR{p|yNe>)j}(wR=THGOiYtqN2jWTnZ~P$ z8}6WTyB4-+L+zojYqvxeRJABM_sB(FMf+FL_h|+wnmWGGU;H8N&P(SyuK>X3w7rShD_sD zb8xt?S}~KgX$p)cnV_yV*0Dc;*d;hhY$95RP~nM`xj8oAUlF-35#w!+A6H57`~z#C z7jeS-PPFWnCdQR9>ZR|Cd)xWySgbr}f&WPicsH5odjJIrpMpTI!A~M+y|8=#Cq=;U z<{sq!0SSf<0p<@1g685)h5W$G4O_oqrxKxEm`YpqM|$N>X^Y_K5sR!F?0+)Z!c+6B zMvzoE>?!Cie8-t`BesS;86xbl>mdgJ?Ia>7f1OMzPGk4M-hiMvc@Sv(%}BwOOK8B> zy$yzr;wx$WT6$7j(V>*fFpFzC4rbP%4(42(pT1dq^})siX3z-PX1UlFkuO1(e4akG z(1g~}y80W(Do#|+LFL}=xZk#AHW7J9H;Magx_C2u`3Xj6nB2OxO~y&egy2cab3w$2 zf1$WvM>e5x)tyMfH_`l;!Ylpu%Eyp5w_i91=p`H!7tkkTam>j{ynTNq4Hv0?Nph*O z`0n*N#8Op11cDqvpgzTaWL8iAj{*ua7GPk=5MV(*Q!LN}xeYL>Dr|Ln)~$!g`HK;4 zXec2P#!t*>{zOCvnTV(5VNNEdD)L>uuMb`k!hMtB@SKk{kzf`yFOTzR`{W&gS_T4E z)t_>0sPHWbvaK;sw>0&e67bp0XIrA{C<@H>Rnte58ue1;snA8(*$rgW|GE{16RMi5 zqAooLt-;?%)IIkotmwr?f5%*oh(Rr-iLWe>8C31hutnm}z$MWgg;pMr2S;H_D~+02 zc@nG^h?|DH3pW=Xo4LI5nERGqrnE~6wqB*fc-ygPSn012HtBamO48#Y%8sAfhfI<( zK-8;mqZUNM+BqP;>ScsBGGmO8!zeERJ;MqD`GP<}72se-@r}w5YG~44+ZVoqf?e=o zXl}*f(CoIes0~f`Z543A5eYDYc8*Tx&xpN2SBOZD@zIT0d1D>CoH-g7VG(yS9#-FachmtgHBH`Ck!+!@Z%gFD$Dx#n5XhC()&@%|BYx4D~FS6Lb zS)#wo44TO&zsU?d#22~FnQ}H#uQq97S30f{%X<#aAmDUR_7eA>P2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve z0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00Mvj zAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe z00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r z03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x> z2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mA zfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@ z00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve z0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00Mvj zAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe z00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r z03ZMe00MvjAOHve0)PM@00;mAfB+x>2mk_r03ZMe00MvjAOHve0)PM@00{j5D}V|H zh5(HL2^U4n1O~Q`ab#{21V&DhK7li;6^wN>_OumESwysz2+jZQ z?LRo)38YQ(>V`rvFvB!$hkaj<6|1W9iF?-%^f z&n9kQH2-P;vu7FvL9YC7H2O@2ilR{e^D1f6f3Bjo|EFJz7Z6C~6a>0F1?hn5txUsj zf}RA?Krnl~VsDA(AdnB)Ptr`kfk3bvEHI^cX-^152r;$I?=D{D7+jwaHs(zvI4`uQ zKPBy4IEO=fors>@X1JJ`hCDfvh*kZfOj0nD*(XX_UU>-cET%d?tg)*@FsH@7*3v#bTI0!oFi+0{)I+8%XfvC`96 z*;c%KkF`*WV2tHOCmKV3ZA-)CB7yQtfTUcavc^br9<#W|l%;(>#N58tjppoSsTZ zuo*ERkQld#A@nu}xrB7ny=ciHLA7E2`9h6^4>LY{)-W*`SXxT$`H~01qxdXF&ST|m z$POjXQF#-sZ`mZ%^F>B?Q|sq9i;~JptJ_$&te~@jh378;m<54?$@+C%5hc!(+0leG z5Gp==>J{Tre_Y@FzPXzjzZrGH4-23%sVq*<0 z9*_1L!BcwF(1d`RK8U`4^yR$5^nKI(_*>1)Fw@<#^=&CS0$fyEIwbS8qhY5zn2<(Gi!(`=EC&Gr zr}Qa07`f?G{wvj##+Eh;m)T0UuxdAdq-!N`=?1+Z$}kcgQ+1!Y@YUMK;3w@pRQQJ0 z4Gu)E6a#-}PV#fU?UQg7od2%Rztx?dB!Yp}BcV6ASbRl_gUEk1buimwsKN9Wr1V*Q zm40}Xw*FjoI2C-BDZH_2UN_v^rSs|6U?h=8F1d`l24<5Lqbb-^KdNi*{2g?)p-Jf% zB~o8o$4EmX%N5#gAyuxNUb^j6CrD~dJg4l)=ZQ6G^x#v$_3Qw0mDU)Y0rIi{-m8Zz3xQ z(a5f+Y>Sr6XzoPbtAqwG-*-4BI_FnMPb%pSQIpA@=clMc^6&0OywPoTFGhkR=OuXS z6|(!J2U`sl{Waarb7U1r4d<;bn~(H1wa7f_52>CAUu(yq1&T+?#kCqpz|FWKI#rvI zQsoa(!uphbcnw4*UEsUWUSeJi&Ss_~HeljZA6C(`AxFkM^QVl$e`Tm`|Gm5jdW%ZJ%Nxwm_m~)^GW0rj z1IM~`a?s|gLDM&mNcQrdlq<9*Onv@J#6J`ZFwcXJm?yK|Hlal!iI;(zSjBNE*?R62=ALAIHq2fJeN#0`?+kC3U)9bw0FQoubGqPDHWZT zd>O9|aU`;3OnV#u05;3lwad8S^^nx5M??AJW4x5AO;wCA@;^P)Mrg-xmlE0y77mq& z6tA(Qxl0Y)0S0CB^+$V#7p$^{so&NexY`=6t8O(mO4t_`d*DJd-hZJ~T`&gFA|PY- z9l`(7KO<9Ddm;%%L*z&$swj)&E=Z?h^YNqm`Ci;EaXJfOmeM;{NfY0UAHx=E8_hyhnPaiUUs!S> zF;58`?}m_>W~9H#q>tP;KCfEW7_v-6 znl&aJTzwa{xt?Pfd_@)jL1j;8%PmRPf%TDIk{luB;qKxuyMlP3{%7B=^ z)5HCO-}-r-Z0D*2-9$(K!zYj%d)3yH@i1wy?k`RrwStOB%(6bN%WEN2`+_0A#+XF< zlGJDow3o1$#FIZl<<-l@?AFugRV^Q|_Xg{fIqDN#K;3jBaQ%hVmZ)cMBh8j7vAKO3~T}?&IiSHR< z!YB`PV%Xa9TDNTATYqn8}a5yUu=a2rQVp*;v{^!_FAJ$8DcUN-;qC`)_M z{!H_LX`wTzR)4-#n=s$PYblV7u^fzS4rLRn?yVQqk_z=QSV@!$)^*e5SS}4cND9jI zVzUg)byUE*pNAV&tX7u`LLY2GOp)PKGJjV?SBV8XuA^cVQfN;%k))+#!+H4i_Vn(X zM(Zr)ueRj3Eq zwm%T)Yq+te4+=hAcgA0uJy0;tSib}+gMn3pX6}14g75#meTuyiMu8x%MO#6j)8M-R zvL4jIOV_a#P~tl(*bFI{q1WTe#%dVEFbqdL$m_xftd^`&$S7#9{F<|!l{J(&L19Nh zIQBxtS`zVGZu5trgAn(@V=6-4#E+v~!tnNa3NLvk_|KZ7#E2jLw7H*YEy8mI)+)-n zcsFB0cUM~29aIiwrfj)%G)dBlQe^(Z-_-tXsB>p!DBA;1Tq2iE`q1@P6?s}DijNa1 zx7a(Lu&?_JsgNy##K3N6H*?3X_VBwNc0~_Y7{<(^`mnmmouK1s)U&h?jdEX4hq2wE zM6WAIP9e1?XitbC&TR6_2*1rAlr#?TUouCcYo?A=YJ&qX{h+;DaIpV%HUbfK)2h4H z4QISlLg-1=TUGPuKdVK9(xGyo;Yq(P>a@J9wZCR|JnqBRl^luP4{A>td%aec%^X(j%j z%rr92d}90(+Ll`J->_dqB2n<>sWz&}?02``-hV}B@%0!fvC5+>bVS^%EIpMeZ*(em z_|Kbu{MjO$zEYH+6SJk3LJ_jv!{$BNz0F^<9y4Vr|Lld0MBvRx*+Lf8_{bbna*zIO zR~~{4+36l?r`bW95`IWyE?}|SWDbJ`;iDJ$(2*o;@2*x;IAQc@zkEeS`qdf%Gj(z6 zgrnEvqH!jyZio*qNQx(b)P8AiV7g-CkBYsa&7b)U4k162hKWY-<7l0pkh=`&pG>uv z<9i0!6&R@tgCHCJirW1^K}tf$(vQ`}MOqJ$?R61e&4JBPqwvJsjolqV(GP@fucsXe zJveBXJy1`fmx^koa2cEmL}n4&DuO0DRRwIu48z-o$ixtQ1d}CfHL6v$3L62TCk4`! zZw6bxjq11AVzbN6TBoyNm6@MPOp25PYd-ij*TQ&(m#7*awGdG1t}K*UpM3+tnv?HE zNvy@vX?>;7u{4VV`JB+izY~D$c7jmrbU=ZPv6UH~hfZXakNyzB^h($fO-kPg{zlCi zY252zw+V&dtG9;vQ!(1t?KW_z`Gg7Gw?Ar-X)S3NJ-LLItl!~c30gV0fjX}BYniLY zYQ*?Dp(zuhxC7~OT&0Qw@QgN1-NIhvkIq-6`HU|*C1SH>bQfEV7ZX|Chs0w< zFgE=vB#GI+ay#+UFBvP1%kkZ`V^yl<>>qW#$gIv7>+d>9dV%X_J#yXG)oXiJVR(|@ z3gI+IUY%w6A5VNoqvMQ*)B2=rSNy1}@Z?E~9_IXuB~vUT8y!{-BwaiIiPUUXfyWCR z+%M#NwUErgXIbWbg?n+_M<;jQRwGONCj|N{-r}=Zy%(&ur)v5OU=Bpy7 z>ml0aRmLy5LH3Z$Up1wR!2ZPK8#ITUjuw3JGfQdAMlJyd5cjlBtfO;{Q_d}L?O z$;BXkuK`EPT6e;Ds(!j*vKi2246j#vvoMg7p@5H)XdZ+Sawv0qs$)1Er9~;M@>7-7 z2fn5j3QJ+5RlkSiJZ4jL2m`Y#YP{t4XjNl|r&zm5`85i>ChUxUGUx}f?Q*m6GJ)r( zhrwQfZfAVc{)=y7P`Ev75p1k7`M-&6a60*dO`#y0yD8GC$r$?RvNk%s%)bf9#4d_n zLGDmafBm5EpPTAI)h-j0q&=_imSF?Ww1hu_kR(tFI!}Z)3SvZxXi%JjV`Pp6SIPXNK&P^f=Hy88@2C9s@!OlW)^!CP zY11l9$wp09XB2l_&xsVA5ig!F%8ALygP!zym*wT;8Zf=j>B?)Yy+=WIPFgqQZ*g5` zyxTF?Q#o~_e;mVpDlXK4>i#}6^>j{OL5woeOO$=xfV|^usjVRE(fBo6Sq?T&g+}td z88`KXWVW|Yxdh7GJnP-j1y*0+vCGSc=vb{x^A-L5@P|19OdY20bgfE2;;}cHaYT>x z_M#Z6aXnQ1Pgm|F4BA22%d=>agJAS2V?z%mQCYPiM#pGuy6{QI9CPGw$YK}%DVqa3e1=DFkq`Qr^E3u+4NY+-A$t5?gSWX( zF3WFC*dESy1k&sxAXQBo$>$4n><#Dg+?7~8!jI?PU67$INOg3khzK@_D*+P+s|0UN zH*+DMF|XjI4PhoM)@TtZMFrcNQ$8R}Ll6KJdK4i+a(*rydQx!T-~2+jqY_fHbe?#x z==88J2&X?~vf>aEoJwh)rp+{D2%jQ=h{Re)SKhN1DMJ?GnuXF^n?0C+a8B{N=w?;H z2()t)x83l#ttW^ORZ!^|GdJF@{nK~VGk+E%;5mseZq8eoKcC8LVzag0p(hr=bI+^+5et3tCC6OiM?L@19pORmCaST=k3a zNxXDqg0k$Hm{p=IS}E{Se*UW#tW;`>n^@-D>zNKY=j6;@t`Z^UD~YIU+@RM6?(BFv z&a4b?#oulq}bm7e$AHSzvx^&=oKZS$6kGViGzeLlp6GFO#T-g+*J)6YD zV0>Yi7xd#Z-PhxNKRvb2&<<`3Vm_ixuw-)9L5h9Si_?CxyTE;#kgd?V?- z8kXhB?6(#IMzZ%4YV^4Q3}Qo2Qt1x&M0)4r>MiDt=6#M`7r0hjif4^mttRR?VZ3=x zGnLC3kcafD>#k9~Z;S?Y8*bSNq(8c;8`BH6%+9t*8J!ZS9xQkY}WaATEd5C0JLngEf!$S@~JH%QP0+SZ+`(cs7jw z#0&=N%?XsxHcKcgldFU50lSahtih6vn#q}F_HUAJe5x=-*7H{l;p?+2rM@#Zk{GK4 zKi$TU#EtDB?I$Ag8r_~9lJ}Bi9yaT+velWuG{3i5>3oYXu}x#YinlB-_kZt-XRZ1Z z6xR80cB*B+#&z{Szixv+O<|dpu=Aa`#0Vj2?7l`;R~?vBWmU%o$JEk-t61kW>ci=_ zsyvFXAEiTwn=J%mU+f8Jl)%^=LBVY1hTH_7%5AEr*HaRm*dJ^`E$)+RwD4(u= z&r^xLyPwXAJeUVrrr#|zWs=70Z)C;#aAuI6Cxy`8Ve?~T%U{>1LRZR?4!j{QbX*08 zDSpi~vm2L$v>los3;?VkSdc}#1zi4Njb+KcTD#KgAta)FU%YTx(b zax}4VBb(jJ>cHsqkG*ihNd( zsqv>xl6ItbEm`!lL;P{ZODUzniVKFSN$$Al-_pSah=_=4K7l2zn`WkXK_bDqXA>=E zJf)Z)WoHz0w*1m-NGQrfqEj~a7l<~q?AZk;)DsUxdBnW$oO9ApLpCnTA6k{LXQ$T?OF{)# z^HRSu*o5-5mU|C5xd&@}QKB;1PSM{Ng$mKz1#7p)oZOV3g2j4^{a+Xb4W;o`dozpE zN*(g+<+o1n=Mom06V7}kI3Z%JDX)XU9}|6+U)dIj*-Qq;p-GvY2HfM&w7}{QR}$=q zAwO$VL${~B-3AXDO^(u)IS#pxTS>oSKd|_;>SAA>Pnw^hOYzvk2cfIq!;N}l~ z>k*$Jb!x)dek#H8B2`X{NiwK#mJS*HYNN4lL&puRd^P>;Cw`v)?v_=ToOw!De}a5J zQ#?AGe6T@eYuF#&=*i7#-l2+a-e1w^a~L5C?H}gTF$U0gmPHL3sfrhW8h(PeVuWza z#{`W2jl^8zTioH{7(EyGcGg_9syimVjdWrzT9|7`Ip_vMXKJ>y$1A9WX<3IQkrv!_ z`_Pa_PFRN?W!8g8Oky}3z`kB1=Cc?!%OLrhk^32^(h_k-G8!1_`Do&L=U>_3}K^!;0t zG9mjIc>o0pCvBk)J_bJtqXok5ZMPr%XO$|^1`KRY6s*syXYXnW3d{?HO%KH&uq1;8 zADsa!oJqn)gtH#t&#MZn^H(Ml;=+3x9XEy?jO|H-TB6SeQTXu(@?JLt+f~~VL<2&y z2@56zLyQoEqKjIUt7YN?agr31>ip1x;je5eab_kfb<1&crkv$j*;LLWZ2W9jF7$Z= zdqtd5ooB}=o{{AwF38vw8eU^tYd-BZ%g~`&yvE>kt9#ZD55xygB1(5Ap<0Prz6SXn zCvOHtwK4P};m)!&o{D>t&wRg6eGrVmCCXxTb(Ev;ct1~u^GnOE+gtr0L1X*2$p7;K z4oqHVRCybd(qn=^QdIovf(QJR7tfD1lRe9H(y5TK@QZxgml<8v2^pPb;7wt zVMnwhdbcq>WWlAl#Se&S-WC60Z~zg3XiW*Nb}TPD!)w z#h{cxNQgmvXsB@Q%&Sw~@QmKCFKVJUb3%!-z0C5}Yfq%aua~ZX*;BS@u^E#?@Dc4A zLC5W@!88UhVW1JCp056H$aiJ6+DjG}&0a7tVp0-)J9o8%AI83qcGCnyN-?yGkeW#I z=3CG+77MTeS8Pbq#{1#~#H=u+vN@kL*fk_-^FmCjB1!d{vWxMgt_-R7xpK)=bHpth zC;HUtSc+&zO9KPBm@hb{1Pf&EJSnZDr;EknpEtkwJ`_nSmVHsXV6&vdw%}r#Z1?gx zn^^SbQ7^}YmUQvGk`n zJ>Le%E$ez~-ay_%hT~6YHNJdtFv#C0%^W7PeU{hf&6^yOvaOW0Y+e7vIg(E6$1yw?t_qTJg zl}BWy-ZB`3KQqEH%C_+uxE$Fwob8y6o#K)nZpSxyy}sQywp7GbQKWiI<$b17;>Q~n zlQ#-7mfZbQS$UjmRwR@R`gwaDWfSK;vmtc^W}S*+Tf*sVuH0-vk{I$uu(kA)Y(#N{&L;BDT~W%qT^5AOb}xjIS>?$<_;mVq!@u*R3>2q=PGhopot|&XaOTho}*) zC=V;4+&u}DiU>pCOsQ}G1#29M+HxGTyk&eRpVDQkT`dF;CWCj>X@^>oI9piJREck= zq%9mGDV-`H(pPgklH6P5j6Yj2E~{L|F^@zZ{C*4g8TlRKspl>a76dpdSr4SCFFOE#KY5XcFN_-9n!7w>p29Rskm`-er zw0%7r^iwXcSm}JJT)7aW19dDIPOv+TFuc-@z|#kxOdZz%Co9S)wfheO(pGbwgT$ zB2+uf;hHZFlx|Pcj%^DbMWHkpn5kQFZ%-aSST{+z*7P{TSjR{qAPZwCH4jsFKF(Nw zV#Z4^tggz#0Kc_5w0LpS@(s7HxanOa*1X*e+$nsGmfog0oMbeD`DwsxCeM zo(_I{X3!Dx0XkZ23A8NgFv2YG-wr%=uErUX!1-rSp_K+LtU5O=FV7Pe@l`T(${I)8 zES`Gpf`(1{jiV}uofQ}vx~OwuPv8j$ZAc&pb{u$GzD84K2sQ)pZ+0=pzP*HL$+VM!@!(h^O- z#zb(b@7{2XxG`G(0gMuMMGI^jawB*DM4&M^bM7d1X_v>j9X^BtE=ah6d7GXCHF3@D zHAPeQ3g}Eq2}d!#7OmD9A7p}=gBVXy0ZHuX5 zI8;yxDt!20H(M{SLfhrIV(MrvSTgkyer$IV1i-;bHEm;m3Ukq&J;qWFWfx?dZHjm9RjrYe z001_M7XY{%D55V)Zm*foV1#vnZ18LX7W$}leV^S(A6A=dJ{})Jcn&g33TENt9sWuH z9J2w&XXCA^6U%EMM4M38A|r=fV;uf=Csrd2MUhoG8AkIqX975g8gYNtWw)Ud6=+#! zN0W(adw8`76nSn)BVySza`yQ?O&%l{Gx7nbTI2DjHhC%?G{Wq60Hh(p2S}ogcDXE2 zV*?0aRqSQkJY?E4O2a19HD@D?r{Q6m6S!J$cYZ6LEYs z05C5_A!Me||`bW$piOlb%TSU0#-dzW$VYyRVIOc{|i zTvNje5wt^&LqjigRTWQ6W>hoG2y0yhWy{y0E+Xibbpd3ETzD_GIFh%)Cd#iYd4o^C z09A6)sfD5Kbl1U>sq zZy&IKchc#FC?GY`L2C-BYh*g1Pkustbn@pAL%Gjj6!2a$}mardORGPQ#P zZ`G);5%U{AIM6I%d+OxVbnjM!H9u`1F)OLO7>$nNQ|dIULv1EtRZU*uJ{MvwC;j7t z6{KBSJ$OZ4f2L zGU7e_BZIUcYPa)UBFI8jb+C`sIr%lqXHesI73o&bDTTXBaWg92P>3-{G+Y;WLkoNi z2;XM96p${~zFvrmeQ)4E~5Z(HOAAFGm1=5Z+ZkQgY zC+D3vA8*s@IF5IkB*bV=d*rV*Q6z~_L^4t^3UkluK5c&y8>GLid+T?nE#pv_24|$i zeTGc(7g^zNBQ&#?6Dk86ClOpNHKqvz6ewlbeZs^8RxX6WO?^XUY$U4F7}p0X2J`AF zTDUSn9$f@sRxDU^JRbXyCE3}wa}+%uP?y0N85sV)G?>~i0T=|ISso&&9IY435ZbMP za5{tnZ{hr9J&$fDLD4+&T*pBcYtP8iZQYDSdK=8|Ary;`SFVQ1B$20)31E=z zB7^O)JQ3{gV_!*xUzN*O5Ap9VX8lC>76gG|H$ac#NLRSx3E_iZ`zQP2W;o!A;v7leLE~}7jZI}TCd{vILMgse$MS3XQ{+O5t;GgVFfRQDZRDa zPx^oXYha*75b?=I0t4=~K%Bc~5PooIbT|%LN&d z6A~Xl$2W9)p{4;W__lw2x%>}7p!#o-dOvt5;Bk0ST@4PuXwNa^NV z^3Qn(r~gL{KY4$pu2Mcf=xk6yz&bqMz48|vSfLuUFZwvhfdB=`h_w#uW8)(vX(3LC zam)~^`Ho)&x)TPw=89+(E}2rB=tLwSQ*|Lv@>dSEr&b+dGv;#kSzANADCrPWw`&>j zt%FfkPp$)+a?Lp940~I%-iAy9c@Izc4OVBvNa1&az#ujbG#4O?@L3QN8E_ySocY}&bMUwIQ=ZrG;%61!iz`WCn_}5(ncpXU$HyWkZvtLu5%*wCgppKP$yxN zjxjaMU^-aN5RFT~Bo<7MuQhtbOBxy2+W~H=@rNnY&nzs$_!ufBW&jffY2q9E85v#4 zGpTJ72670Hu?AtvoHY{wV^I-gTW%k=`U70xwURf0u~cPT`7Ke5zY0}gq1h{m(B4-F zCf!ZpY{+Zff8#O0hGJd?ZK?xbtzbE7QGyC&=>r+a8yjc1VLE)OcHCwV+IkUuVIVhv z6u2T4b(b9~>ryu1DPUb0Orw60Q>bJlo-IYCX?+CF*c1{HjNf}Oj+}n@(OGk5u`w66 zhF(;o7q1}?lExb@n@$t12s|5Opo%=@BB(MMnLstjj-L}c>+uJZg#mipc5z8FT?|x( zib4=D&OI9J-w7t+wiF&@2qR5`bmj^*YuIJ9&(&vrxZE|yZ!UL_WV22UZk;lyibZ@w zXrp!U@Pk2Za1Vacr{oHbvU3A!GcFjlNH!X^j-4|XYT^=d16^jpaRD&4L?uvqr(!N( z*k4gCTm2-cEkjtB`W^?agEtRo_e&|a?6hE+SCkj4Mayzye>^qpreDDktE>y$Yx@@W(jQ!{ zo3L(fGcJ1O>T(fhiL_qY!;26ayek;-(lJx_+sFu~F>eyN2oOm~6Lw|XjKUv^8v_9V z#2ZXN=p%E|xZ@VBWkfKi4}v^wR!}KxtCvFKs)c)!5H@(=b=zzo3gA0n>XB_H#Cv1i zjPfjL4;}#mcN|TZpo}JW<~Aj4k=Yium6?9hGqx{PKqLVBELmmsFakK@xQYae)wC%9 zV0?V3N3@6F274bYO;$8#T=eBn~GOlwuyfFh-YgaST zr-2@^Qr}Ujj2vtyXVrH6RrPKkqQ!Vq0vRRMvQiQ$_>wD}|1U>VEX{2&id7=bHRoI! zPx)Btx*h|tQ)>_HH5M(nR^1I{7A8NnJ2nhFOVQ%wOA1l38a(^YgS zJZV6U-S7G59;aC1Ih)z zwvIN7$uN7&6iiLt+9DkMpI{KS%=1tDTc-&tV$wUWJyvFHU644p;!kc9z%>OEcOWl~ zlDB#ph;mBTaSLvhISg(Lnx9%4N#Y5KJ^yK|l#371u>B6OJ@I#aQI$eXTzC#8^7Kxr zQD-Wa8MG)dknDG2_H!`>nL9#fMfQG3Fp^<3`#)*~yCpW#AfqfN=(PkERAu=(QS1l_?)59UMwMqwF!pe8dN39*`NFG2$LY&y;$V z31=l&+BA4p1vydtGZQmS>{LTni%oT;Ivqmgewi%Z=+rM*%VThsH)jc+94;&d&v_XX zd`>LQJsDMu7~Wbb2GVWylvF#T`~4<*Du`PIsV6M~I1FcNTi;dXtp83eOru%KH+39x z2a{}df`Wn*8ono($>jnF)|=9_asY!3!tyj}-2GSC{#`WOrR zgF`+eL-;IwR+bOQHdSHoVmcvE1#KZk+aVC$64q41EjS7Bav>kdTuBe-p(=ZuUmAMY zk{2F5o!?G@G}2(k8zd0~DFXzI6A=}%AG~{r|M*D&2ILaWE%Iv$;30QYYNLC{5nO0W z=A}A*3eF=cQb1kI_(1`bO;{5L9-d&Pe*Q+Rm;q_2hypkr^cxS*h#OZ&2Hatl7kVeS z^6M6_><<$a=~xVxSFtcsVC^OwGum=BKAm-Y79dlfuL=iELZ@P+{w+=xB8Dc;u74aX zhMFH|y`U@c$;ln7sabv7_8t?^#!7dxkYHgSun~VRTB0Dfy_`LUh*DXc+TeA(KXxtO z@oy*^#cB`Qy3r{4zY9V@0g!TMkt#i=A^S7G+OJEVU*<80SY1r-%RmHi~JQV>d^-TlZO%3|2c6@suhIMkgjRNYhNhL&6+;L+df42p5i48Om|8SUvpdFNf&6T1Dq9T7emme0Mcr zIKNl%Se8MyUrfKnQJ8D%4{zliQ%#-}GdKRy2S@#hWq@)}IYmvXZVFieZVmw;SctdZ zKMC}nXj7S3GJWJ=IJTrzTANXGI*6VKU^IZYPIbU$H~O`Wa^s_}1na#y0IWLH0{|`& z4W|f(N)x&86<&VcYCg}dPZKUVQ9}2mGaZ|8UC{R93v-IKK=(GUMt@BBTmYY7Qehc% zH95l$5j{S_4%;+yBiV0EF?)v%7!^4gACXn>6sU^s3Riik8JVHBZBI6l4gp>yQ*0r} zTCtDa6c}(aI6S#fAUK062%r5lOWiTsE1U^$RiMEuVaC_wLl=y=N*J>qCZ?$724+PF zVz+X}YV~ zEZc7APV=JQe)9MF7(^UZ0AomATV;gdS;UQ=NJC)dLrAQpZ2b3Qcgpa&WLI+GTMd5N zc6LlCEmrWFXdR;TQR>#=J5ngICPYiQ1dv)GC$`eRQ%$XQO578@RE1y~E~2L>H_v<* zI{126OW;oTL|KudR#1@3FZk5vCWP#;7mlLN9NBJ9bN#FvYr(;nZj1`NWCCEF4v@+s zHpBfd5CHg6BWtw(I42P;O_5CkXm$u}N#F2Ne_>lpaYhRWZHWA_Z;21Y1P37wK*D{F zXV8o^S<{fuD_Fz(AF>H42>DpIMzah7barJ58M*5XOA|S|bHIvnCZ!0l1XAlTBFX3~ zD>JS{J$jkLRM0Uk9M1@~83br-AVwPYN$0x$eG2VCPLV0U7PFn6d?0`WPZ%CfcM^@j zM`)43CPd-mF5XZW#V zOYrA=c8_5uO7E|xN~XDB3`=$UNrRhxFCN{%G4>oYUroXDP=5RKV&^zpY&=Gt7@^xr z7oiT*FYdLy5yoDPX~E1eH~4Rc3Q{F29lJxk4=o6zET8G0QmVFgclkY^L(25CF(*62 z2o(thPDl_gPtQ?UIgwX8A4^{YbHL{&L_owDb^~`23aJ3eej6h^ep=iq5CLAFO?4pg zU_XUiI|UZOe8$TRGQom$C8r{;4w#goUc1H~G>}^GM!p0XHbV3@d3Z423QHabT&_j) zZGTS{7DpYeW(^{OQIQvgR=NAEd6!XKHkQZSd%X`PTU%R^9UYG@Xs|j*MrZBcOy)B{ zMn+Z|VDRmVX5_hVCY5-zCX|dwG1L0%VYT>8d_3WlR|<+ye`C$R1wrTC1|^vxBK)Wz zAu!M~V*9M8XDPE0DtgP`XgB_8dQ@|C7Kqsm9UeH8OD~i@KqnBvc6bFqB1f}GVCVJ^ zH+RdIOOISee3;yoZV@GgQy)F&5(wtFV|S6TA+5aic@h|19^Us5ClHA_Bi+eEVC>Xq zOl=d(PfOez9;myEad@8wTFbq76SrNZ5vdB)D71}*9OKtR zKx7;q8^&=*N<`K-d{610W~~tsdYrY?66u0QAkyM90IT|TYoz~OX2nv@P`+Vs0d;H+ zJA8dT1q<(m5SX5JOaiCaUIz0?N8+yxe?AG*MkL@`7d}!I8Q^PCe7%-qf9mYBW_? z5ZG9ZQj>_EWvmM;F0gIwCCi0qd8QdPKB+|TZF1YAAOvUw8%efBQYUzeBR?*+bik2Y z65!+-GhPWv5S~8_J@LYfdR!!O8X-oZMfib15E*-fdb!OaT+xhLItHOwPhAq{21)Jk z6@!=-LD4qMS4U9XHvw_JSpypvAD!fzGCu9|Iu%5qYt-E>Mi@7AJ;1(H66~q*Nxw&( zC~)e{OaM?70012yZho@^b1#n{NeaZ~GAefTH1EB`DoiIVV9!W3OYf#+W~k7#fA%HV zNLvz}d|BaXGLb|fPQXH%4_`DbU>!E(4uR23SYP9{ccl>%K4i$^5)xS3C{UHn7EH~; zd{|eDea*5nH1}P|O-ovZd*>JiWX2aCHC_6IOEp53OhUaBa#~|-A2x^x7yejWAS~GQ zAwjcNNSl3kSpVlRXrjr?M`54iY3)*(X`@iA4}KTq!T-R4_3;4!!T-R4_3;4!QUCzs zVPO6L|1S{#WBdQV9!T#8;`>1S9*DmK@o!x6_5Z8?*D?^Rn&m$LAx;2y|7wSwfz_?| ztBO@~)$Huk@`TaDoyXEk0|k*^@=z;s()uF|W8Ki}y#JmLShae8GUi6zKP3&TXIyREyM&6e*OIOk!6A-BC=2UF{(7?MO;KX(6_Frj^o@0FNh>s!!{465 zSB!)s?f+|XZDK*$xE5*4c!7K#A(Q+mlFtXA(wxseXC%L}22y!V|D;IE#rn{R$*9cf z?`+jGR?RPhkvs>rkbJ>InRikjT`%$U`vZyqp8Sj|E1~{YaKWeonUcTp!*F*4ob|RO zGWy&H)jX2PG1tc(_B$tpEOx!$&SMu1k$L>v96xPVDTo$R_jRrAad}|g&eUFx$ZHE0 zh?1ly+XV(XUggN99CiDVyW1aJvBYoXIPLtQomR?MJa%|NfK;e8<^fUJ_$Wa^HyaA2 zB+?nX1{ z2Oi(ot7PNJ$lJs-M`#@*l2E9g4%~f9fI&=PEV<%9Q=<#=9~ zSLRzfL-b)dTJZuOo#%BzIDN~-p!30S%WqkoSbC(!`9P}iz)YIVy3ckWN3=PX%D-OA zem0zFeO#lWxv*}v0s)t#A(~}~z;G86xHWWQxPIgeMHL7Vxpii<7F;8rHLukcAl2lPCh)2~OB~)($E23p_a8!Ao;Yr1=o8 zh7!*CrXoe%yD0{XeETTtB#Oj*%1$DRBY-#DT++Lz%OznP3YuK3*!zFN26BQDR1Rw7 zVSWr518h@I3|1TX7WnndYE+yH7AZ)?IKa?zzb*}HOOVE9c6TnVa5j=lZYLzGJXWnC@GQP5*6Wlc92CP zxHT5=u1r(^`vs-yE#g5!O#N-XLhmkZFfC?PPSA|QX-s}R1)kaHO@Zd2+G07Tw8Rh_ zqd*6{zoN)EDsgMl9--&r(X3l5Qe#0PJ3Mt;>7`JvxG%_J&X&l{h(gYLjCR0~Z`dn3 zC>ZD)PszJ-cVED3AF+qA6mz4PYRl}P3|=Wn&9jEjT~4S(HMnzte!Hi@J#bz zomLHMu1dT%71?8&ud=m^LSgja_RxHGHkvv=6$Av&dv{x3ZuNP{T$)=-5zt5XQkc5X zC>~#iGWn#`aMcfxC=&dx>>edVU~}4L!IvZ-%mbYNo#C5Uo@#TlwW;oLssm;Q z&YVUpP5(q^uXPo_`*p15qM?#(Q|R|QJ8h~^q?c~C_^A5D6fFK?+8b{0t=y%r;HbPu zhV0hs?|46Et{?&RD#q+f8^)N2HV3f4TQsRfr1#;n`Vdzv@Jii}N={u??jW$YrpLRi z#;d+tP#K8YM@Tm3fJvXCs77;~;&;LVl%a{p`+-PP3mRFK>ue>XG71cKHWmR>2KkS& zH*vcFGI~gnFZu*C%S*X76!qLkYTML98z8AKVpPoGGN*%8RUn^=21Hrx{{WFN-fbCE z2uZ6g2iJZjm_-V+y}e;TrV)W(%5S?lCyK}TWKTJv2VGL>b3NfH5D}se1o#G(k_vt= zm(=b9NB{sbKWN|`Y_M&nqgDE(M<6$;XrU0|s^YIbt!}Q3AGZHKahnRo9kMx6FC;st z`C7lQMrG+p6B8uABBdAC!Gfr@-QvHX;bxaC1*@%V2jlR4pN&QG-6&GeVJ7C61ck^K zSfZg$E(Wbej^4NV7N^f*pNNf(CH>ICPoWg7*xiz^&+*3Ao$53g5z%mYYEEHoDMJin zZasdul`=32aJ|TR@2V0nrQ{Y*9=aIkkmZqL!?&JH0h#9c^$`4Ag#9LFJx`cn_{CLA zFWigSqPXhV;+|?zToAQAY^^F=+hl-qcOI-D zLmuXjoJs@y<=Yc==y(4o$s4$Kznza5Bd-%I&~T2$g7^EQZjSy8$iw)R9AqSwbyySX zFK27*ah!zGq*G3=&D-lYk@^>w7WFqM?}F(DZga}&+1p9bqHT7$KrGlK=&d`CS6}X{ z1`j`xNuZ*7NHmW+D=(o>nibvyClT59ejT(q!!OQC8r0>lu(9bAE=oQmVAi^O9>b}` zvbGqUO$thWu@jXCdVA@*lqn_}vV<8r(~e!`&l>wO?pcKv=t{bCwoS}Q0dxSge;)7_ zm|~~bss@%dJ(`+~kXZVT;Bf`~B(>K(##`C@zUbjr2@ANdRr1i`~ z@Uq&nTGj4&5WFz39e>#B{R0<{iZUtn_28(3?&3)pgpQk@bib}4Fh&>SkuIT0=Hu>+ioe* z{*T43yP<0$pv+)C_Z0pnA1C|L3a=Cdz(sGtQpqTT!@+qEGTBguMN(x*PCkvQ-98i6FKenYm`B*_zn3}18E9S4^GpP_WT zWYjT>CUpzpSnbMgTwl@wPhfPT+4*^}(^j`|6qR#pJDlgPzwuQP+qORPe;GdPmjc6w zYcqHHi!_rMG)BiIz>U(SC+Y-I!JB#Dck8xAPu=J^Sq)6cD2p5-5MM{0-e@aD$9p44 zHo5bmDG9>`stC-iU91j_SC?FL89VD2TdLh>E$``1kdIXe(HtNs0y!aP%-xhnSb6K* zmw4fMd7%|m)nR8dSD0n*DMj=&Xi&VPx-Qcd{Wk4K51^5b+K_y*YN4kL`Cx+jpu&{Y zog1$GzIm>muDt8g%(|Ni@7t?y=nk)4ebDmGdcAyW92fhEG7sH}y~+4WM!OMvkyK!-idKZ!nY~cm`SqWA}&CBTTqK zFYUQAC6gPuk75EZv43?uLt~7pJ8&NKdK7U09xbqP2Ew$X^ILdqra{rN5I9?{ll?1V z*z^3%3+Fkw?8j_IS`Il;XM)OJ0Tmub`kZFWa2JDcNDM=@Y^7%6=XnBf*%!w4Og4%< zB;r2ns{oKn!r7P!eu{i$Rxv*10N*t1vN0`oCnI%v>9hfS6bHGyrV)lbvfu=@yRV{r zdhEfvTSLadNA^8-kb@=nL6GUX3OyGF9_l;fU{G*;3){hLxj3=gfWV~~-5gRBNJ9&A z<+!ZoPAo0x_;&5P!|Xn(DSp_efk&FOJ?HXd6g0oJ5f=sqWp(WIs=|#Ur-d*ndo`RR zuQK*=ylhNv7TLO~zQd1+SSN!P)%N$Ea9!+KIkIKmL<;x82HpDfTH}-=&%&~>FvXC; zAXqDo;}FZXcQ0cIHldbXw(Vbcn_Hdh7+@=dB>~>Zwt%MyDG;)xIbO~4Ft&9B^JW>* zOg$pGmYC7mtNF;W-AAUBv+elTuQW{rdUyC!qV|4e1$j0nwyIjrQSqzt7%QtvkDGkM zMfW~ar%xJ(!U)~8OiYm(*M-4e(0$L%&H{m!&P$^G&-TDg!@8};rO;>IN~HfEiud~< zJO;E~23IqL;UltqU4|P*aedJ!U~X9?MCW%4$YvtF*689;qVjCE+_2v5KHes}_#~f) z+$JD7L2UIv)M--S93i7J1$GkC_WeGQe&9QAP>wt7JmuiK(<=*%o2i^Qd~$ZRxwrt~ zEW0bUdFseaw-KRWXtK%%=dR%@#9Xy?rV}J#wNeEGuS$xF8O#7{!Jw&zp>V`fIe0&3 zdH3+z+`b0&uV_#r`LfM4PJQ_=_e%eyk3Pl@lC*nPSkG#1W=&>n@Y>evy_|xD%Wga% zO5WP>CNMj^bEb9$iKija9h%QdpgI$dZq6J6VkO2ftao|b%p5sA^`P~vgp@~M3jh2m zKEHO}9Bp&^_#&2m#)&cC10O9HkK*BNqPJC-=417ZyQCRhsMWn*l$2)BxqDY`<8%Xc zwJ!FjwtBL$#Rx#e*>^6~ZiQ)MDN(YN+ZBJ(Z6?{&Zuv~ z$~|T1$bdT7VZizAHEr~Tm^ql4EgkGG1%_$2oQVFw0CKSY9xS@USR;z?(o+$ zp(nhuM@hg3x`DK4os=j=ul*?v+y6mfRs#kxvgs$T#~gNJx@qgURr5=L&Mi~{Jc?|X zd7*o(zZ-caQ`4lsOaK5I;&h`vI_X5_E>h?`;GeArbS&7FTn5i8lE5Y_xo-#A+rfTu zk{1ZPvHhkFlK}*b2jP6a2_W4If)B1g9uHkbTiw3I_M>F0_3Ec-hzmWgdjl0zp z(8a1LTarDkKSj-Ex&qx;uQ#^CrgXI}2KHi4)!W+ULJ6MY#{v#rOc^qrp3su>K~}lY zqP^hSbIGE~W7wnbhsij;>4r7Bt((lvf0S8u=@kq0W_WlggCh#F^15JKG!iMlyXJ{GO9{v|kFbJEJ~NKwYIs z2DY!Icuvp@!|fg$Ts?Ho+o>&k_ZSJw1BHE4b-ED8#5i~~TdBE$*`u+ZQ$Y*{p zxENZFIMSsxcb>7}`G+S{U9Bjn(t0?m_6XbGkq$o;N+-eUJ&8Ul-X*Yi2slLM8>cPd z!TtLY#bnUF<+Bhok+mk=PSTRp%mQ1Olc80AxBdyIOf(gr`Hnh{>GtSUm976j1qQ8f zm9%%u!1^bn`v4v-uu#Vvf{kWSa7?I%#@X$g4~zq{$J~9EYkqLHJHElM#cOp&5*ap7~QLQkYz@{=%JF zKwPZn$0xBcRuTtP79+sAb`M$m%(PUSTidJ-+o;9?qH$TKWF?;KRJvu9Fh|Q^3{EPG z!T6t9DBvZP)1;45M&&irv4E9{ljnzJOIqyos~Hs|NGdB?Wc}IpR&1zQR2Qv>EPmub z%_bnepKE8+ReL5Rxv?1wLc@4tB5}r+Ve;G4LzT}e)9KqmyuTtWkeiSAY4m`Sfl_S1md5B9uWmx)O%$|5972N6=JPjFsjqc zm#>`OClNXm7@hRZh4fn#Kg$CD9P<+3v-4aomdkF{uEDXz$7_VRC!BKzckwmC-^|CHIm2{d)e6<^GPO`k zmk(3*j$8+s*qLM}S9wtBWE8iH#(FgZ&Za^xRiRF4I3ClHFzmy}qo?v=49(XEW2R+= z&Q#~lLrbmlStJzKSBSU>2D1yQ?Me2nvZeixAzQAoqZs0Jzs|d+K$qG)}(!+$r zKw|xtSw3^gx83)x`u6A9+|--EhY0Qf4{iVQHQHsQ87b;Nyb)O{xCn9GM=xqJwu=Mm zVY#bYe?(>veEipCP1*6n!9KGB0dl6#{w%ET`H#dG%HUq_%29FhMLgBf39ybDDAJ%O z;@Q+KP3F;1oS$1hB(uqwNgI`d#1#qCzgWPb>G5THusXH_reo8(@v>s00>`HE$-yv$ zzo;B+Zp5gqP6S93!Ps7NX%L#LAg{`G zDd0+qT&@g(=zD+s-5W67tAw+>3YdUOplHKDF`VyDL|F+DNbmuK*)F2A+<|qg)0oLO zo!(ViIRG9lumZl&KP<~Rl5mSi5KPzSQ;FCd*Q@7qva_3RSVeMmyLX!_H6_FZr3?A^ zI~eBu!FTcSF=%l;p;V0ArE=1zKgy^Mx!$T#(SadJ{k5O&+m)z)f*KWnRrA}WwEH-z zZ&+m8QWkWxs|IoULDlW5DifJHj4%?xCk&;(U5(+tU!Zj{jX?t}>D2|YR)h=)AElpW zv)SLiCug?1F4AcUd4(3vj|Jo-wRzr&=~0FX+fv}Yokw_;;zyS&epo(Svm53)zm$|p zfw3O%N+7~=!xCX(tAogF?@YP+Tm`}TCI=4!UIY8EqtVd9 zE8V+#loOpZDrMP4=wf> zSj|tH1UEg7lDAh%$z2$V?!)gtmj04s6n`_23Y6v=IJlwqD20svr;1nk|D)dIvF`J0 z*o+<3Ljpl7f?%Uz*`tnKdJwmn#K{~)UJ^Lz80waO5t5*D0cH(H{=q$4JB0P2q|cbA6;W!nb04!*$#;pkHiZruGtVzh??V6a1`Ng^*| z7`i|L8W0Q5cgI%$Id4NrH1<49cI3lvc|JCIZl<&*1`#+AY95X^Q5rl+sweFqwS}C2nY%i&)n4@=v z3e0*|U!A1JVeXX8_b(^EbIKEJFp2>M>Lq?+-K>Dpn8d>ezW?}P&%p(-BF0aX+ioJelf9kes|0h`4Qggt^RBS$+?9CN=L5W{X)QH6}}ulfb~U* z_0oO@a%tz9eES!)vAUz;&eps_h;zY~-%1~HgZIsodKZ)p51v5lX~XhHqjVT00b$nD zam1uE<+}M?-O%(>j0mR;(OpH7(r`xzh2gVFNl4tq#NneZpNnvFFeE}|Gns#Lb5qMc z9wh5BHjN^()ef4n*5_@USPCdSln-VHhCJ0#>g!zrIAntLd@w}ww9US{2Cu!no?n|f z2&S+xteM=a3TQwtkOM3WR~Z%3?oH&y!;aNypx1+cvEF_Y5A}V!aG7mBFcaXB5NOh9 z;YV`@&PLQD-L&*s!Gti|?lO%z5Zs--q=pm?+s{P8LJ{Aw-tCUKL~VFJCRZ?LCD>|V zXh+Dxu88U#4#D$EXoU~hkt1>k{PZKPTD}cJhFo*zK#bvQLMPd~$E<+yjVw0VvAJN1 zGWi;JfU@qj%MNPZqeJzunO4OZbuoRnH$@U;bKC#=UkX;-t?E~M!I-%zQgX~yywm(VHgImR#Y?E|bu@30Dfufcce%(mgJ~zpk zL+xzd!-4079D3yoN4Fcxy>h&C4*h)J%?R}hjAgk-T<1%R^1V%AMcOd)Y$PAl_|MaP zU6N-}ab87{{TzW!`N`72#5T&Cf+QM?)poxqRz;+9&yf{`euG!qCafN8fpEsJ9x7%4 z0=Hu*CHCqg0-5+3py9x6>UJ}ms}dKMAcT0UXR)U~VQ-XZAhS0zu3nV@6SPlnN!r8* zOg%1|-FLiVe|pK`L6@XI#gaGT{9R>FKsb~TwcXf*Hf}lF+fGWaUesjMW@LK@Pf+Rv zWSf|l5kzlgX-f78G0*chbJ4<;L<;w5m3q`?dWF>+OF1RMlJD)%15Zc4rcM{Ve3?;h z$psKj05P8~x@>20Jit0EC0fiZxn8Bg|F@uzhzOTqU zSg6m!wLBFgUD?S7NcvJ-g}2`_kh=?14oB{hP}_O!Y)_ZAv?NO@C!V72zfgdTq+J35 zx4%SsXgyZfRFM0>od`sZ&bgJmDHv9w!X%jyBIrL5`hj_1k~{veObkLDcvkfQg-Bh# zX#gBLM)_7vf+2u~xwOJS{Ncs%8p7NwGs$8GV3UUWI!b(WP~aaGM@w>o9tK_fyjz@9 zAh`)%m;b`*4G?qw@5LTml?|CgA|=IXveHpVlFPNCPFHXqKF0MKo1sjwbyok0CP}*3 zc}iuof63l?_*Zn~s#+{SQY%j)sYd}-r=zNW zrtCV-IsJFMmqCix$Xp7dR>#SB6Izc*bl-hlhR-C<>j|#VN<@1gI8dmlOr6yw!LnH; zg@e!ygL4?-mR`MFC6-_Kus7U<@+)DyY+_ziIkIOzKLA2xS=&qJu59DNQe%(Rq?1Aj zA$Zvk*Z=@BKP{s^ci4_yjJC#Ox9bNocz)oQufX7V(St+}^|Gu}_agY^TRk3dIQ(&v z9R4@m317u7k;qOW1N%K3!K}zUt*E@2?n*&$?XnR;%_MS1BH0 zr~hG=NEJ2$3tnnwEr^=uC7hNbhlinq2e*9cqHUt+x6RK8oPbBs=c4svlkqX+lG*UQj+6qrzW{97BuY=KN8OVqJ9sctq-@;1^_a+ zCT(y0Flnd2v$bTks5=nd&%=~s^XyT4ihR%)=TsA$ZGySfV>pVhwQr%<=ZGFxybq2e zWIjz66-2)O9)Gvqxx?gaO73%>l8-tAE8(zD*3F)=V&;)+V05qT)tFJuhHkg{%LL+b z3T5V$(pB(~)+Ioxh)iK;_159@R_FkW{g?tQqQ>zLyN~9fkV{PE0Uepg99OpJ(`cx= zJD7A&&e%U69@DcabkhJ)>HA$mhXV>uFx@YGF4gRoo^`)*XdW zt7ckUn)1#!S^NiS=N0M$vQFW)Cxa?b-t&~bXG~OY^zW-8g3<*9X(L5OrFWzZB2Bu0 z^eRa2ART5XA|TR#ICKQ1Nf(hmfOG|h-eKq+28J@j)WbhHFHTNwZtmULS^L#a_D

    n9@$s5b4$R|Pw9@R z`@P}V7NB^r8QCumkdrHnhM&laBvL#61DBKcGr7=QZiBjHjOZRKNr)?#47kmy*u2kW zGIJWN)~(wRKj8_VJh;DpDC=7equT$Z-nanqEM^5?$e0IkNh`t)nZ8-nGx&rK%t&VU z*)RCfJmP*pqtwGCeqkrzn@p-DUKF@&QuNk*+FeR`300wd7ouQ%1}~wzw3lfNMp(Tz zSXTM^ef74-mxY^75Bod)AgS>!pONi%J-(k{Lc2TjOp;DwM*=sfn%QW${cQzmE*w%V ztDo2j&LF-AA1+5EW-cx2yEH!_kDP6#N{tZv(%R79a<#qon7ef_$B&`NkNjdq?xO#f zt$Ktd{i}DhLaCpf{So$LjWq&}a2`{%@u`a^j*KxIdhb^slEGRkG230Mro5=onWG4p z#oXfi_tuEWOX{x=fflvq)vs^4u*fgH&COG|a0(j59_qiPJ-cv9A+LymGxDCqx=z$OG5jt61K&75YNFu9mKpNZXLZD?vq3cTw z;fVw9ELv)KyklEeHHo9Ty*8JXuK951f7aza=x{ffEV=Nhg0s`Qke!>YX$A+JCRG>G zW@PvE%?mO*9f~gRt-tqvywirUQQwj7qi7snne3+>;`%6*xntlZ8!DaMSG(Cq=y7nj zzHi!KfAh$DeO4$sHj!p)l-XJBb`| z66ZTp$V7hvy#H9u1Lww#jr!Tk(ET4G0|y0TgS3r&T%#_Mms%R}=t$=p}_LF9^W zK;8>;D1nnKpw@!byz_dF;#~!z?|a=W!C-jA5OuEeDWq5Ew1Pdy%QZQZ?+`3;>p<5I zu=$GHX&(7aYe=B=cIO!DrI4x-N0AzCp5p+>BVotbrI#gZN+7in5K;pD%d>D9`q zue4QCqLw25xXJDrvMzLS|B8f_zG{mkM=O7dvC~$Q^$zI}fAo~%re6WK-pg78CXjdN z;J%oR<@(1+jNg}^4#GaiP?03()T>Df)--YKgkVQ-e)zbln@IN6E>t zyuNE{@$R!QD^s^ejyEVyxpjQg=$BqtjV5%CF4*P$vS=Y^{ck{&&V+cH_pMTekJv>Sd+D=n4gw%0%6p^3ch&n zG*Jrd{sv%7pN+|Ofam^-Oduy-eReN0h|Z&Efbgn|ZlehQQ|tHb*LNUS1rLkaC8eeD zz5UgaHKh(A14E|~zi!%iKiE3Rkcd`f1?GLGkjRg;_uND#>@09$*%}=v{%(^i93HiQaQ9 zhRL-y+Mrm1D{Wuy|7;Y2Pb-wkF;$;zm2O7)5}GY4@ae;KA}c4K8s zMeOqO#QNj_C@!b3g);MN7<||_Gsi||#mdIi6}jQ2pHSZ@c&i~!G%z&O(y0L~6j1eK zSnX>}L^LAGhsi#kwpRXe9ra78yQJgHke?|xC&f};Po(%52PUnRnHH)ahiXnOJjJyS zr0~F&bGLirBRk|q7Ojs3%lKNH#tqYUoA$xd9UpWh2IF|6jOOo_bZalAF0RJu zQ(Dje2?Tmkrv3N{ZU?ckOSq=wS}>V~NFI$6sH|&LLQY+{%bf~-@Bw9ae(rqqv;ML9 zQR|^&6f(a$bvEI`BwfYG*-x0enB2x$55Iv;v!4m5dsZ_d%Eo>3Y;Mld{O;k;C3d&` zNvE)O-7m`u39pDP5S(@$(gs~ zXSlD0&|jS#lf&@;KbAJ1@83AesrKHmdJso*@7BFrcit>Sx-@;Yi&R$buIm4N@7|r; z)U2=RU(?g4X5L|He^ggj=Rrl)<}o*?KJ8}u=>ZM3prBwl(jV>+Li}jVW8HE((6NID zCFgHE=+idcg#Uc{Nz<08b`Mu0{Hg3$e(904r_A1*gE12zNA^ix5ga@10ob;32cdQI??{1-h3^FCDrmCe z_X8HNv(1cossF++lFGtL>1q1q(IjSW;8M;LF>$GwgnPe|A_*#SFLT9r<6SvQI%@|@ zJ|P{kxSuJ=GvRH*)Id&#B&^JLMnex+qY`d=RiK`WYJG*-QzlW&0w5FwJ8OpSNAi5Rk!!AuOHbY^HpOM z+(8a4>lxHL*>AOa44Kc;Y>Otv!}uCWtpMCPMUtx=;8jVU%s57j|wM2&1&)|WGj{6D&^fxO^$?0DS-X6*1?D(nKeuw#s+*Q zwJ>LTDua)nwmCfAqwy_%QFcQqX*hqv;hfV!z5`Lfj0EoKWzOYKH8SBq%@lYDD zngqe%=ai>E>5!`=pnH!_el$B553(Bc*r^#)3IyzT-D*$dpOG`zFs&I0X}R-H@#0E@ z(=4i|10369QcbUv^=|3nMbXFZlV7AoTjHLAoz8IHK)i8>*Q`hZ{2i;trerqYT+Cel9Q$1 zWbdqT$(I&%$JYK)t-M$ldsPf=*REBbnR^Mvy$Ee9yAToS-9#K`b0kFwEEkAHnDgEf zQ!*-K@94T%Vjt(kt~czdhY!imrh_Ema<=Cf)#HSi)&Yx6YYXo&c`^UaP-suJ&5g=% zy4n~rQ3`g^@*k@oovH3zrAai#vZYGd^jVxx)xxm}7O`dJV!=sr>g_(T*X}7ZY1;K$ zR@?34h+h4CvRvC4O3^JcV9Sg=puXs>JskakAJs2lcN?@i@pn0G=jZWC0C?9~^7hP` zu!6;>rwZw_8B%w25}a!S%{NRw!>0h~LM@S1-SyW05G4WShDLThY>Zc<@LS9AWES70 z#dl!$bhO}V0(O5D@?rud^mTpIlRiN3YA}iVQsUP&iU8_6b_sMRcfs>5qgEqZ&SkADq(2ucPKL?dgJf&vF-+`$lw}mxjbDzP`__^p=QBeY}0gf9J2<>vd zdj9Lu{CiT?S{Xg$gva@$L!vY+;x=sY846^oZ;58qE{U71fkH$i4bC3e2W(5tx+Mox zh*V`jb}rKdf_D%Mp-FOg^;X*%;hA%5Wrl(h$i`07%{A5Jv3uaPmfPTp07j``VZF`B zlu|dZ8e%u+a-@DuHcD5qo5kh0tzJ|H-r*C~WbR!J?xqlm=5FaCZ$GuVN$X?K!haD^ zno^35W<&82OXfyLvO`WK3!TNHYMoOcK}l<9DjR_d@fO8${pNKtuc?#)EAR7itk9My zspc@;%Z;ZdL4mgO^lH=Q6+0eT%@PhQO}g;gY31lCn@`cT)AtAqh`LU-(k9R+sVjf# zj`oVaV$kOv+5d-|gcds(u#qO*3o7)E3|EH75GgyD zv65HIcFY}dOTfx+wdx`VKOQ2s#$y~vOF}V3st5p9?}{^wk-#3>VVSZ#xA;{_;k;ot zYAmc2KCmD&g3(I?oln@%D?Y(KJ0B}p)(1p*c?)#fk{)(QW953n3S(z!k=Y4JT#l|O zZox8gcoE8uZ+l}qPuE#z8I?L(9m3qylwc$_?7{h_AE5EB*KURGp~LvjTiVcS6*{Q< zSs4BrbiL3LJ_Z%syuXx9VS~bL3(JlmBXM2iH7&%UqkfKEU`Gz)tj_1mAfQhk;_>b#V5z}TQBoW)(pB#dW@sqV598(1=+Ah5RJA?cF}dYLAM&m;tN;&^OqRPxMNe{i{YwaUorL(R`59ITe?p-&@c+V zqi5uP^Z6>yG$4A#uMm8q!GKAlAh-t)P_%VZB_X}rP9dIi#bPK&Pf!WoK%*-o6iR|r zpgN*0@+yTyVQa8xDIv3l)a2;`Zc{m_lH`WWh6dXs8xUZtf~%bAzTnz|CIg^IGaW9Ectt( zb&LaajUzbiIm&z&OpIrZr#-QdgG#sbW83c=BCZ}2r-IgPngemJIumEca)6SJ9myTh zd{Xx4K*U_I7Ab6RA)pl^_d)28q(l1lp4f#ym)D!6oj8qP3}kU#DmiPAg!H$+hgEGn zOa@GvUHIbjYJ&DKib*LT4}~({7{dEbJ>R9Wjl#W^oA5HyvyqBPi5<`0S0DuLr2Z9L z7?N_e=^INEKINN)84S@89KKRkASgjd@jG3qVOGGJumU7cjVB{UFqakQxvEp6;o4r} zegHaT6v=W(k#(MW8!rnhJ@ZzvQwayUwx35WewGU`fM=V*WEd8IBrHAHe$kO!B>_iX z!!vG~?hii)+2)pS7_%X?lO|5B*|QAk>N-=Erfz?3xylye<Ld=?Wb+So+ zVoJS~;F3KT+Iw@$@UwsezbQua_sZ0131Ur`dNr{#jib26PJ3DY`5lezQO<`fS=}9$ zQn~)7>@J+Nlk#iaM0bZ5JBv2N_x4L4q~ifJ)zi2ZxyMiy-_hVf16it3yTs+BkW(wg z+eF?|3YFui$sS<18Osdn9wNvyuO|S@exdA-S=vv@S_2-ua4f{w>sHE%hNK3qs^_oH z&n=Pu(>I6Pk2F?ajmPbhb|0ak!l7FxHZhC`6wT8M`Rt1pRpSmdmH>Bngev-tEU6ZXuzDUcFkW>jx|z?=qaKf195V*3cn|8K;WUQ@r49R;w}wUp^Bv)_^C`?%r<&j zbg2X|yk%mp(v{}k2rIBg*ZWq|$*LB(i!Ba1H*0%$(S?VKld{e9rjFXA$l}_Ae2=13 zPL76i#u`>dF!YEL>8|X9p#CJ-8CO|SBNf1IS~&|UsFiN=wdtA28y_&P-d#g2tJHqbUx0Dwbl6&evNJ9i5v&S=`$_yqFt-cGm?0N>1$3{khpC z*4uA$XvODgVI5ijre=8UOzVT4Ke&?9`;6{;-#}ekqqq}_a4xJK3|t6FrD(G6MgCf@mf8H6)ZY0pT0d_isJP%CsHO_--xBUZeNMCeN94F? zV9B9%^~8j8C#a8K6jCqpN22+*ep#U?Lbmg8c^+Cs(;wHt#1$!_n7nkx1xp0!D**?p z6`pi}Jpl47-fmY;-hz1w#9nrfoghgBq6@78>5U8gK}f%6M z^tMxxS*xPY-z?V*Y@yRZqC>b7>12`5G0y+wZ^*sqq?@6%>PoeUo*r&yor8zz9!QH9 z)S_T^pSUW>35nP(Dc^X-KWOpV`J=?YxP@13N+Wmi^%ACW)HBRJ>_a%kMtK97%w{W^ zwvMHrwA9&6|IsPvw`0DkpR;er7z4&|g`(3@)&Y_f{)~z6sh2^I5 z1ngU(F8!UA6YJz6M>iD@hY&R`oFCQ-JRa1=B(&!54hWO*4mOiDn@)qr`ZsJtK8S=) zs3+ZrClLpsU?|$>T2nc*ZXguNDHTg~)lou6lQP0AT;z=ku5q9t&zJ{VGqBgc^E~ga zuCH(K`eQjGh9Z~v1%6CN*O$(Lsm3jHl^*tVul4TKr=hWzfY87zuYQU1W4f@#VRwC^ zgb0M{{ziRp40#^RR3w{4_b?Z})twPCy7`dxkN0tpGj^aN5PIRI`j zW{|gv)~dp61ugQ=oTGH3r8cK+?)KyQ^U=ROeaa8z&%g+)a=?-*6+)rl2RSAOmu(e= zzPcu9g~uI8xc_qI2tiu6%?u_r!v5Ms*uqR_;%tIEm;a$-^_B)ZGUOeR>cUf@o}V{k zj=R5wTgn;nW)hnNdC+qzbtb7)m)eM^os&yL>oG{H-2N+}-Dw}MZ^vtnSEad~x~!&G z+j5*lGQI!LHgq<3u6K+6X-_LN4V2yYhrW6Js*=Yw?dwbLBJGb^uPoG`FtcTLe1G!9 zgF%-3NXN^upBHeX=@@7u#v-yelDjIVUHT8 zP+wIC9Jn`FpjcDOt{Q+Yl{676JcYIi)y>yb0Zb8W-9I{{WRuFUM>FUG zb`_;^4aD9wx_-1KBysdv$#K@D?F!Ae!uxdF;E@Hk8D^TVJ5tUsaE^-nE9FX`18(xyOYHc1TZSCqnGXg#=D72STFDxyK&*FQfGFCZ$27V@KFaB&|YQ# zxEKX(1c{NOL}JℜQ-zJ+fC8<2X<@+u|9_ClO=;^(Zwp)bWwcRY1-AVrT{YUj9G* zWsSmc`7{(+$pP5t9OO(W=xpY^VVo24W_tR;03^G?ymXuC8I{zYroKaFMbR?H7myHn zFE^iB@7t#qw0HGFbZ(*E@>tW1G0|u9sFfu8&kER#(GOHj4coY7Z5vIBONJq@2zUz- z;=Frd*hfV$8gwm+HveM29cW3yRW5A=zv!HMY;m65A7}bWWLX64)8f)(I9_HJ6xiWn zRk51}zv_627jJUOwI9!WL-4Zadfo80u%AjoipaY&^$4-2uiA7x)*jbkfnL8k-Vm2o zsoDvb=gawlsOFdNYg)~l*O0VnCfxMaE4Ez-9e=wh>HUd8vGvt?+p?T@=4g}LwrjXg zri)yl<}-T0lNKH7z+mS#aMOBU;i{E@?Rom9WJgKUUdW5H(oEO(fb4#KaZuT-^9$bM zA>#$-iJf(Ve{bQ^BO-f0+54KI9{sJKP}fN7FGIDZfsaugRRPH; zWIh9tj0~KAZ}%~tzUar=#!$61}<;@goF1*L3%E(lqdQ_&2Fe=D=!h;-T!wf(9JW7@YWb7JqM3@DDQc`Zr+b z^<0lfH;4%~d5HXA>KBC(b59HLvVXy1y*<`@?xksn#lIIg?bV)JC+_cyFn10m)KS@j z+kl%a$=+kgh_o_-&A3wC?EOP71tYwGTCry~#0G1I?nsW?U|S9kjF|dIhgr1z<<;k0 z{B*D>4Wn2cEdIFn--%Fh(2PsFJZ90f*+p&@UN~0^0tbF(6fyy4_w}YVy*yK*UA zP}+4k4f+zlVz^>K)162y4?3~j;amt(3Z%Y?CcD>1u@Og!s%8og&2FY<=F0iD+P6BA z(@C%%H=CKZ`R4y}5DFV?yaD+&pcf@geCn+$8*W}OBf_kgwRdO)#nNw1A$ikyY4U#X zCnuUDBy$V%7ehsefJB7uuggNRf|i}ckMvLpx&-|@araL&1TV(p!cPYZztsgj1#`a0 z3+yY*O>MCL;=#YA%}zapqZdcaO(Qv2n1I`+=R-F&+HJ-|FwRPsda7JxE8Bl|gH{Z8_BjPp*HI}2+5EIBIU`6tH7 zs12sI^$`v36pb<9G9R0ybr38>ukFR_XLr|7)CzkJWLK}p`SHL?dSb~ zo>@#s1e+nUBf_ef5h?hrVmW&dA8kWBTTlb?E8jTwkGwlQ!Cxv+Hoj1+{y-9sxfO^e zZ+cnIUZz;gVse1CU$5IxAG#*p z6#EsyrKVizs+CDzR~^}d!3nJ8O+T~7Et!{wSeFB=E8N-Vf*zq_%4Vz&S9!p=dRsc| z0LhN;?k&3CiE|sn#=3XS_zK;Ik>77%BbfSl%Z(zQ{Q!=EtFVT9`GoAqH z_cGoz(mtHH2cam}m(n{w(7+q76~3Lj@EqmT;=nwDF7JfJLj`*C*Fc@OI|4J5bjKH= zvI2K(L(}3G1!Hra% zZRLe5V>sac3nP>rqpWl$QIvwjp<0kR*bsRQe2lOhC*hb}mOW2SwXq)^PGY(UfO(R1 zV4(g3#0{nSh`{SF?2O)FOsHk=O-T$J3&fCZD*OACd`8EMjv|z;QvTzZVH$E*99DGX zh=6nY{Jydv*d&Hscpm;|-&tli;4s)RQh8NY4)t%*xKu8XL|rjqy@4cjh-57`!hqs{ zH)*@DyO6O9H>{e^q78;o(hQJ5nE9N|taC(X*?#ml<;|W)%ZSg6O6q!>OQPe~WANKH z+dV<}SEyvid^H8L<*!A>G64v&f8{J4sW`z95Fk$%h)9N2gb1UE99aGgx^1aye|nau zb7fM2)-eiw?GBRUaIKRtzN?kE&=c;{yrOpLSj-|A*xnlvi+-^&5YZfGdaiKU`Z3}< zC({b^({P}j*$MgdfQRy}EN+N}Vaxo-3%1x?_4BlEX$O30u4AE@&RwIk!8p-TTh08r zo1GMj8VqHVLr|ZTUB=ZfD#7+xUz?%`$T#n5y&0_8y3{SD=4NDgPsr(KTOMKjA=P)=Y**jRjrp-^SjbF-FA2;!5}BPDiX%@)8*$sPmi4#{=jE;)mP0wzM@&O+T%sIXMYBxE zATzvU{r2f0B};XoP|jVD0#!2k#_Y;)(4=WgT;fczO$lQVY{~!cksRp3z@)^hWQa?_<}HAsWfW$MW?%OxO>5KbtC?NH!@YKTDgtDkrWtIIH&Vh_(KySIeICS3mVQ@gk_0+)t zD{d4=ER*sz7nmeKLtAbuaZt=!ysR{aj)=o7hac%3_(MVp{qH4Is$x72t>HviES|Zy zA)`H4TgjM)sF1WQc=64N=8~9-$By>9Zo<^9AI2ltqC2hQ8#oAK%+}4Zr@t~Sr(@&NeNu0-NO6?66r%M7xPKh&r zAf1t9?jZy57cPXd)rIM4wn7bp>qoHXc;}~Z?OyN1L3GREnQVyh3tOZ{{=R0r^g6zB z)%>B~fMH7ReTqvMcy&i*bB$sBoE6<|v-*nm%s#L9c`t;+cb@VpuU*2HZ!}_8#ISzA z7eid#e~PViJ_&moh$<^!L%q9oI{hU$BFA6eK5iLP5nQlqjccPMsEe#WibEq@*6XF3 zCo;qI$PT5VLHslX+p8CnfF<2zWpBo+wrLS$3`js|UG^6lRss)bi= zxWTM>0{ge=(lOxQx31*2cdaNwf?4w*=?Vh9jS45kh zv)MxmZ`8_nlQ@-1qD4&OaW_^T!+FF@{VOYlx)xc172B4j{q{}CEX?ece#4dFIGg?v_95BYFhRLbb5zukuN z#miE)pJaxYGNViv0%UQueok{29C-xJtKtx7xgCOS{hU(rKjgdIt|oLHnkqbwV3{dz zAG`A#qS#kC{UK}{xM}_5;K9BkMn7+;2%y&wOPeaJ>D>Wi3)gH;mH$u~=_zrkSl{ad zpUySjgN)W6u*up`Co53F979x{(+55VyMAalt*Dg1fDG>z2=ODmPTHV&_-4l=8p4dd zH-tcnNl+GPLsG5b7SO|p4@CAfhyr?9?mX`3XCGiDh}ZoICMdb}z6QyT8%Z=uxZ7x&d7pOyOqcf0ms7FD}xbOa6pG|*>G>aw# zi`BX-u(fM{EeKSH`(P^%&O4x|K@@n`ulg}6d*P-BFS%@riOuCy7hSX2=LtgPaZ&AM zCDHK9un>METm5P$OWtxcf}fs+^oR@gHTnI3h)_u3h#KN_bBAXO8uA%qrBz_MUg)(e zc3a_ZPx58kOSS0oi@a@$V9DA~q0@^*$W3f<;Hswt=G}U+4*HuCvM?>YR#pr4n7g+2 z1R)+iTf+zHFZO;)gs2AedUWfT%#mbe!@2M+O@^I8Yg{>9&%4_bf89^crQIw`MEVAu zXuBVxs0r<45SeeZifZl0WneCzT9FCw0ekph_WDLMY54rBNMHrLUH-IK*N>BQbAkTG zQP!2wv@vUcie8I}O2$LCtQ<@Amdp}eH7CAgHo3>YbX?8frWt0h{5>ysp%;D?|Mih6 zlN#AeRlX%*mX@E_Cz!Wn-+T~ay+*AH$z@)JXALP8Wg`h&+30qJ|7h6CRtUVM-wJ#2 z1cj$^Irm@>(bOu9f(bkNi zOK^|l%G_5<#fOT#DcL7J2ysO~3~;$Lb5i`asDnd^pD#8FM311Kto^CJ!_jx?jI|M` zS(lzw>3E-(^S0;SOgMaI<>~GG#rD_F-j~_Q>}bulc-mL{_$HlTdH)<5zQq`fXs2^J zS^b`o9{@>PVUG1g=7O>Wb-hh*LEhKQ_Af9p1abuK1^I zD~Pel^Z=V?8k>g(j<&owZE{X9l|3W7*XT7%mQHubI8)2$waD8cZ*m~x2CJ{t0yG-? zt%KfnJFNjXjqYD%cD0UFOq5~x=y8eBCrghylX+ z5m@jaBtTnR$(bQQdZ=l7HYHej+`jO?TI5a3e-8MNv1er@YLl3h~$oLwmti zbaD6$M#84{JFKqJ4j-^ooPH2o>zL9A|8H}s?L2}P0mrGL6oS`7&biiEqdJ{ZArjUT z`Y{Nv$Xfq;pdsE1T(+Vy;f(EJ{}x}HsBlPCNVC<(gK1GBwnTC|&dt8GHr;@m;NJ{{ z*sn&7hol!Sa4V?ift=e7e+mKhmlCBwu?itK!gL-TWQdS8Q7wjau!qd2KC6PRa22oc zvKHp^d?voaist_dEQP#TwVjPiS9z1NExeuLNDBEYFK>yn-Jkq(b~5@7Rr8Q*MHzCo zrx19@_wMi%=G0{fzSiD6QtGgWVo=U&e8bSngIysgFBqW`Au|pj@?Q^vf&}m=<>WC6r6XRE z%6ri@yWbSWC0CM~fA7o+<5vqyBDj<<1yjQdKnqc_gg)4p;Cwzr!E1#>>p9yLrG`HC zK-NFJVVf0mxi@UdyG#Z85~Z0S&KYn5QxN-~j;C?0dBZSZpN11L zbjT~XS_ge!Vvl#k{UhMF0X+6R&TNgqtMeg)If;C%K;J0;R-<*C60sNFWTXV#vmC-4`7XB&+wa2|4`3r5Pyds( z*!Q9&iWHBguhPK%A|Pbw8r!uVyK{gziSu%qhpUNj-GhgmvZLEQ))88qiuWLj1FiZ! z8c1Kj{7vM!VQ1sW)Q-X>+)jZ5!ZZdgeLI_fp15*y^$<%^T~R!`D(pumY4brFM$K2s zY7J&jz7t}x`86#hxF*6I}!%YVB-2nbm}_o`Do2?L@6nGkUeD?yl|Fa8K+g(db65~n-jqw;aBo#p6n zG%vjDNm?l0fNz`N_J7X?J0%M@C0cTn04+Fhgwp#mABMp21y~_wK(9N68xx>=m5DSU zy;}>gKrnfa+jy*wX;kT{Ty%I%ZZQ^pEd#@yKIrM6D~6qVH^>wWz|8F1CcO5GCB zgWHz>KKk7S%OX^vU<9OH=Ur@yEf0h=ct1t|t=Zqy~LJLFgA8^2#!Z zP*LII0N`qES$$Z;3Lx-4LD(=b%e}mo>wHaa-G&m77elfsD%7#-RA0$&;$_(=!(}xk zoFO#ZsnS1a4^jLhxeB#6l-cX^w7)%O(R*Ex_3>gtx}Am=TcnwZ%5@U9fjs=;O}eP_YvgZ@f*ct@lR==ld~bivK&{5o=0HkkS@*^TuFKNzaaF$ei1%ThUareQs zYx3^x8^)w%0h?3umd#&0q8uJk!OnOwM{nwzuJ<3G)@KFvtntHVnV+D^?|!MO`_E*x zV<#z@e01n+Ru+?kPVcV+96EfkWMd}N^|DH!WkpC@JwI+RdZ3jLrtGk@$HVoS+v zCP30L_lQg6)hkXnZ5>zWj_sHD%#i!6Ie<~ONp*)wO9kXHg}Zl_cW^yh=&_45NUyepE0G)p;cVuAjPlV>t1FfFj?;=$?891+U&u9Z;LX z!=cQ2j*Wn~&!lHlqI>Q1;GC~N?kOX8viR#hyCX8;uDGASp%}~|0V@U)3b4s4 zHRrm|4{G1vrI7U)+BOC8v?Z}y|5=P$aG|f{UqOEl{}ZEk8s#sQl3aJb`w=Nhi_S0d z1IWp}c*^8Ia4sFj_7Ml*`k4Hc_Tz`(i>GpdTb^TUTW$plU&XXhLnedM^$!yCE5|s% z3@LrW4Mq6Un+YoK!pU9IjQI>!2dt%L-Z}F^Z{Ek|CCPa{|I+>AEnNGdclDe7`jsD1 zx5FUp%VdqN%>OuUhu_V>xeUpS(IiBcPRtwpVHtC=>C%0tz7(PLB+D#RI>GOaD-HHN zfAJ=TRneaXvzOE1kKe4LsbA~qZEVz+?lb57nglo|K9oyd$0G(lL~~gRB-fsuB@6<$ zc2&0Qk}98S)uYdB>*DRwY90$`_h0R7Zu5D8RHV*WOJfe25AlCamb5*M2h}WfV%^_p zFsSOLrxm=BYH|!Hz6IcZB%6L~iSp=~)nv;PrJIwuv%sgaRNloARmLQm3FbnjKqqk9 zpmvYBgYvRbNu2S{^B|X(Wo4O0=q~~CH18&UdpSg7tNzxy1q6O%30Hh;v*ZAYhzvGv z9DKd3yPEZQHF-75W1^FPI@d~`f2Je)=l=aS_g?>}t`$bc=g8U8N|MgGdSpfs^l3q) z#(+*+W%t~my6=&hroCg|q-^UCny(!cADZGhCI`*m%yvHXy(~{#80$fDFKNad#I~2# zE4_xKh@A4xYUw0ZMr^|#e!|3ytE---*}U=JtZC1>4jdi>G)?qw{N z;^A(c>M}!I>Z5V*M)ODQ&%PTYs`w;RmjG(XSP3}NbUos;pV zG-03#j{FAfB>&$2gpT8>&mMn6U-k4#VArttOc}X)Pe`i3_iB28FvZIk^dr4%bdfws zqC=6?kA9I8C_EW6Yp;C@*Mg@gS==aeRFby&9w!NVT{S(GmcGGJ`upLZtF6rdo|&BF zH6eHKv6!?=c8sV<%<*1&_U6MG225|Q8iZ%2Pf`AvhsUR6%S1<+KS4Lu=%~IL)lTE~ zU3S*eKU)@)`G344DW%WV`5Jp+b7pKY+g1;4dSKv5e8O5DQ)8{Q;s-2uc8C{>(+I57 zVmb>{`*vn~kJ0kvRm?->xoe48v(%^ehHuOCO&VXr8xz)y8T4TRWNSz2sfvgCx*r!E zO-qiHi&-lSSjf=rU)SeYx91PFEOTu7rh&$!75aJ(t<_Vq?)__vGJ_{nK00XDgA7?y zS@_o?=_($?1EVFGF#6#31rY_}9NhE|2mcV0QA4+GS#hfI>+5$f2492?qXHsp#Z)bRaPofi&TFZ_Sohyude6xc z_Ow}Ci>r#nh*u`3#)=-6@`T)J`8hrMTR@8mMyi zc)#S-RjfaeL2zp>M4OlbbR54C&H1En@v#z|nB_(KYTo^{|Jo9oJ~3cYdU)$ax5?jS zCb%-n=;g?pLg;4 zuRhEvi<01AdgA-y?1h`j7bRE1?*(})(3TDhuh8Xuw(gJj#`F!^mZd?xGGS^J&lo7k zJ1zsst;hMiG~~4!SjLWT-+nWKd^O)nX?Ph)R*riG+hcuPy={U^8ITt4m%nH7USE1A zyE}G5=hnp{+rCO)d`_hCSj~9_P@65{l~RR)Zq^S;Y)T17lmfd?P+iES;faAi-ry( z3iqB{DAyI|a5BVr<;;kxIa&3U-`*IW`}>L5!@t2zREDZAGh{=+k9Zt;k8yQ*S$OB&%7>@ z%pPI;M)+s3d3HNyYjl1p0^&CiHLfM5`fU1e#iw^)^_?-DUtHB=?=rtHR5$V8RH(){<4x{wyz@Vn z-~Dh(u~#Dan#D;KdLD4O63lFC(m)E5!?&Al1qk^Hz7FQ&C1yXO*XFDf{r4+cQ(^O8 zjC*UXWGAB6_2wI41(?_C0-$I5mO{43Ee|^rGFuAa~ev$<6tZ93WbM)qU8&>x<38< zBQ*PN^lz*C++WwfOH3=-jyt&l?Zqq?hY*K`kYT>g^=qS^mk9zqM-8>~GPw$8WFL;M z$g^*VWQM9OM?{6e_)|=Lmyb;oD9cijtIkr9I4&NamCJ;Fj=MK$Hx7|)@|pvSZyah8 zso+%t;ncCO_WsWNlF9td*;M*jG3Dk5Q~&ym&`YQ>`K}p~eypZ-p1!}rE&Rg^kc`tq zU4chZ=8AP6s%U<&!Hga^P(oxIJegG{zD~$33f~zty#3$Q???MhW#Tl^KljQrD8E#R zFfNTJI59sp{xALwf2Elii8m^0(NAyglDFmcv45GwE-CryN$&5LdCz`uCtLXe!KEC^ zd^*ibc`C+&$;$~oCCRXyE{QuLC8to(^RuyTQo{e3id+ZAdpBN%R=t6i1_c0P#AW_e zVX6+lf9k223@>;8O=Hgy+vky4MDgQQnzLW^eo^JinWSuO6B~;1ujNIVf+8ROT)Jgt ztkE{?2}((PKDiP)POrK_rTM12f>GH(hwEp4Y(TQh%euUpZRK72Y_jjD8g44BvC8Uq z)jMyC=c<8@Vgl;WPhVZ-C}UsS!$zJi{CfJNV?WfE)_VUzRW&%JLB?g_eZ^h z7^}sHWB&xbKU&}M|FqZqvobH{C9UD-Z+%AgK$DdRj;vK$G|xQa6b_^iGEyEANwV!_ zf=f+5x5J7Tm;Vd(RxPP~kGxk(rTZ{pyy1J1u`oQ}D%)--blgO?QB|+rzV+#6Wpd%pHk|1TXq)*Xkh5KIsNIDsM_)c<@D!MSyUKfPfmrI+AMkaEf2hgW_UfWr*JzQigr==S7LJovr zx#2Lgp8~EWah}Vq>}-z%L3oqVW~mA;46Hsla~P&#$fcfCw9BGo)W*_85~mqVniLtMwru#YOa$P6LvssH$!O^PT)yVKki#tVfdD!Jab+QhJCJ&3tg*e!Q zR|r)%WQzqpQM{K(nC6OblC2g?P@9%VMP8OYNloo!wD@^p2U#yY-4}oW2}n^ba8#AD za$$#^WnqgKvS^fwiLzy4+a*5cDp+gQw~1i67O-PkNvUOusO%RQYSULWlB<9K2uZXl zuIOH{QsG=42y}}>B5_4s87fqHX52aSV$NhHQ-RBK+C@!|YfdA}4fp1Mm6 zt(3c#iAnJlBG&7f}q?o;r zvJA_s41rSll~KLw!mZ1LZzZ0ujbB+guz^(Q$qiAOdBx0nUn{~2eQSy(Ns6Y}Vzw(F zp|!KqTq6{Ap!)c7**E2;Fmmy9h-J1bkwQgmx*vIVW8mKgI}8i-NqIxB{$ zg|ZN{mJFfFiQxvp%_%WYlVWtv4ppmJ$xK7uw-wcNDOkNLS(@}JZdM>^Jf@A?N#3+O zF4rZ{s$EKy?ZYOe9>`6XCem(6UsSfRJ{HHVRiW)EUQSbKM#!a+=Xe@2P0m*!=3@&o z%=T*!Oi>K8a9ZBXb7ZO3(wrSDwpVo^S*i?M45!+3N{)$@GcCkuDCmiegWm3H*w>*n zEK*cfP(k2Zv5r}p7Tu3LQ4|@AY7$s7r$}U~t0UTpYGGHxG)vWn!bgqE6|uHkCiN+g zqt$u7ZoyovD91%@2H}%cD2ZKkao~j>rDE>u%(A@#SAtF~5{Rh)2d_vrS)c0o#mWpYx|dXx9>5r{k8MBWlNzQZ_2C;}T9bVx zDE#943d@CFz{x8Bj)qZpe}#Dj=mBui5F{R}13tf6u!0f^)hD{(0oiTEtYElN)ztw>v+v?egm(M`vDNBjA9Jb zEA_v5bWLyl+q;nGq0p5#K^>kB>YtZTnzHKF~raZIK{0y8g`-JRf~!%?XEnKJ zBXGIL9B5RqNr6{<+;%xXAn4j4SAfO%2qf zssehEnh*NZX}{ir1oWRh)w$CM(C?e3z=}@%RMQE^$)16sAh7`^>etEn>Y~8|KOvUE z%yuww4{#<}{L%|!y2$BbFlSXJ4-JQVerNz{1DjLF1^w zZ?K-YwNS-gAuDWRM{r#a*9Pl=-jrP;UOi>^I^cj1c@X^g$tKsTruO-4T;i}1zfenPg5dy-eI{e;nhCdJ!3j&596+iikGi@-MD3v`GEf!|y$vIXovytV0a9TH?A zjTg!asdl-<1Us`@El7>v4(aed-}@fu4YyI#$Caj01t^dajLD1Oz`fg$-K6BbD+Jj* zw^w>)OuJfmO`7BfIoR#q3!h9VyvZ#Fd1zq*%5s5a`u%m$Gs+kw>ufk@@|-A&7^sQ52-JVOLsc2=i~(TYL$mxn6- z(Gf!*-Si-IBH_b;Js>hnacW{1+}_|vZg(8_Ff=J*Fm%z0)q07U4sR%h(XWA#kO#Mk zpB*GLi!VKl>|V;%?R-X?l#;pokc{W>z7I81sU5)N)PulYk`2fS(>aGpn5kp^r7$Vy z#Rsvt{IIlO6~5yB!%hI_541)N%#1iO`2d4E%skM;EE~_Vx^mHYDWQ8^Z?I^IfJLv; zWIZK?-ysA`qDud4m~9V)61n-wxt$}bq)t|0)jU&KIMtE$1h9n*xsYelf0J{gJ*NBb*M8;&GnOJ%TgNBK!qb62*Q2 zZ20+z9b@%v@S}ux__d1~7;m_eMs!S-^_4-0aW2WLjVw4hX~*lcfnN(%h>ax79RxuwH5Y3UOS)JsDPc(i)=g2BMXQJ z2vwi|IG74ZxhKaZHQ*Q6&hWFQH6+4>iOQKs504Q`ntda zl^pwh&mgGScoIrUvL;d9rc^P$+&1ry4at`I5IwmBP}S-M0>$dzl&gKz?sbCyYC^xnMkfnh+6(6-Ukx<2*VL}i$v01mm<2nV_&g_deL=p9(5bHw>Z3v>3iBLB zkm^MqQ^9ma+U{yU&QTMBQ(j?Y)R~lT6Y0ZC&0ka)g~MA7$&m*EZv8l02`Z5#i?JV= zKUCWdeHvtak+dFytW~^KaS-?Sh(`*M)3(bONZTtR2&Zn)wIf5$zMt=yAz35rhWxC@ zD6pZ@cfPNZ(wR95Ur!_Yf@RB?LOD5S{Une5=8N$6>M|j-0JJDm|1KDMW!MAz8)a@+K845V^}Y4y<~rl+ z6+KXX*7~GUuy+AWo&1aoZD$=SUngH^Hc{#K^yxq4eZ(L1_Flq^C0-ZUuyFzoh% zZU#Dt0NJE<=*YJK9aX!!R1N6(V zEjnZ^CfPQ2yu{?9Nl-DY2`-q7QFX`Yfoa#urpn>j6~P3rh-11HK6Wq@c&8C(d*BgfmX{KcTUskMV?5b9Tu3q zP~MZ;Na6wSZ3j;rvr^x0iTO-b%J7Xmn?v%u#FGH?;hxEQt-{3G^TqVL zMYFa)<%qmOZm$`@S_2b&vF{E zg=d{Q@MEUWGJoUvf)$N_t8_>;1;-sYr#9vYKUiIsj7JzhFJ)xn6zo%~N-n~A{*zyV zi_ltFJ6_)(-=T_x$YSziAAeYm%Tova0e;iW4>g-F1jR_^WQ^y*0q^CQD!Dquvd|5B>-!!b{C#bk(rC+!nk5j`pu- z?Es+_BD|c!yRK1nz7-rV-MZp+qy+TO7;i0q+kE{n(vpG5bnl3<$9s{)k6^=QHFj4? z0~WG{dm8Y8Kk631o5p5q#6PIw6aO7UPz?4L|4yfe7vbLv+Omhf;K08kIPL`8!R^wr zybde-x9Vtc!)=#NG!TEsS{>aTQ~&}#f7kN~NW( zW&(Z3<>{mHDZdiAj|6`Erz35BT^XGqfyeJeko)BStn;}`YuuRx4F5uo75J=NA34F$ zxx%f#{J;fD_CufOvQEu@B{b<$6`1vO8XM(CSgKP&C6u1egd_J6${3E_c-3xmoDM4O z7~zjW&Xebs<3FtKcy0#j)reZQUw#0+jl`s|ic0cpW0TmuuU@y$U=Pjn3{n7T zK^%z*`J8_pPFPqHXX&!0JQH#~E3@jbQ`6Q%U$YlueT+MDO$AY8PVsK$4wlz`qyY&a zO_Ci%3u@_U)&zBI3sc*Ue^dy-&IM1yTbA&J&)+v6;OuF|m7;dQ4QeA>eP&V2Wy z6M5PP1=M?b;~RWfG+cFd*9CdHRH!B}-^WJ1jF4wSPEm8`;Mz)-eAU-`%t)%yE1vFv zye^hU4@Vs+A%TfVhAuY>^0A^MQ%+wi{$i3p7xvQWQJMzYh@MmClOlP8^DR1StAxhw zA;3+-t`9^0KoAa23`)8Wu+{X4<}=l$4cW!uMrj4T7MNXQ>nNjz>9ll<0#% zq%M+~vXb{!6S!Y4uuAIEzfODQr3IA|?>Y%TlLm&ux+NvGBbjA=S@MnHUtIl3Za%sQ zVZ_E$PljL+-s0q$fXB)->=(GJ@#T50M~W=5-(u^QJUql*M*8m{p@lNb9CmuAL8%f-6P{59 zE}&8MHZUZh8OtJ~_(Wv}pD1yt%<>dinW4-u#Ve#NAn?_iv!=@$VUtnW%x)-$C@G*G zrBwPf5DbAO7JZ_yCl!RGkF%6#{AFG~K8OS@$#dM8-HO@+dQ4Q9lop{@Y-& z2u%X~`jpZ5MKy^-JUopm(9pCasl87vK=4^a35&Qd)5qhTjPg1x%cwrE6fHSqk_@>c|vPtbw3}^@t zbu8+APxpgf%6V*D1GRFiB#fn&I0a2F11Pl--XwT( zX{@lw`#{B z=^FYTWVqe^A39bl(|v{*ae<=qgw~;2^kiBHtxLHra?^ky?+@+0`{yb~lCTr){n~uf zK{ao|5^_4k;`Ig|or22ESUT@tVv^olTB2JIDmzu%_==;^nzvL+*(U=QUD(j7rV5>} zkXcTVd42KWli-N1pnkm3Q;?v#Kiab9h>g08;oCESI2Yu_OSvnoju_o1b-`2&G2Qn? zmmowKz#jk&B1?f70!#g2;Emt=q?tKSf9z%s48HgqSE34HyEzzaN3&vZCBx54TJ2JP zhgScp$@9YyJdr`=`-6~7-;t4V{W0p= zbY(Z0O_=AC_n9%SKHiiDgif57o-I{Cjy>!JG3Mwr76IabkQmxba;LN57~Kwie)w#U zUJm1mG>J4#nl*pk)!P|ZcgEfI+H5i*UBe6a7>~INXskl(in&Zk$84B@EsJ8&nTepB zOCO#tq7+OayCv%0?=!PvpSFND?%qIq{)qd@SK)_QN!Mw^jag5z=7c-z;yawnKz=DP9vIkBIKCm; zy4%36T&_9%My3#j^)YbGuKid47F*0@ItSOYk16=bh(VdR_IR3%qU8jpFPaM>a>POy zFAKY21I=9OBh1H|vCWee)A0d#7n${>GTy56K^%|WDOeZoS^>MK)fe`gd{!@&-MIwk zKz0O+DSw|>EWB2NAYy?;D8KM`MOZ7T4$o#2hoJ2}uP9OHtXY zgm3kZs?B1rK})BrC8aw*-984(dn4o6BnoQ6#}L)5^jmOZl|hnyjBFve4rzIjlu+S< zRlmIVIZ|XgZQA(zFX2|wu-XNUj6K4FP<9jk=swk*?l+FjiM2Q7G`c{XSeKPNm{`}+ zM^>GBd1_`;v2GJFliNu4oYdS~JGx_p`h60ek6>@@HIH!;iu!983J&r6)oR5J!+n=ir@8{ZELF{AQcD8LV%f02zAoOR4BB)ntL- zY@USCH2pIa#NhW|l6foV*pA)|W$LsYSY}M0XPgAvL3xNmrK$+YN2J9x96i~_5Hfn%X&YWFh*h~Yd5VFlQ&T7+b z@x<8{kd2+!w@m=7m{cDQegesLH~fUg;CI(fuv%))IYnQ>kbA{3)w_M&*}}PX))we} zV1~$+_*-2?SZ)dzZup%vb(H!bd~AfD>nf2rb@TKW47Ud?ApXwqJb)(`6Q=9!_2VZ2 z*5dykHV80(w+KJkCZe32h`0g^?k-oLgGhYk&y;K4Emxvc-`{(b!(-j0n%`Imj^YUlaON{ z5}bxS=se?RDC$G_pSK^e@cR1p1SjzKo2E9NvYvq&5&s*@X zhlKjz74>d1tgP^af$kIvVBd|`s_l0ls0qqj@SKXeAu(#Q?Z%j-VctRS8Ua=s{?13> zM39e$(#%tW%;ovX*&FeR7P89$_A)*{eE{QU%$+J4@GC#UY-6FtWgH)WbV$URuaDH} zom0n@ug&13K=NMBFyWiYECD0Hj}d9;Q5%mKtPia`|HOnw0jkYCM=7el7XeATn3Sf1 z3rIK0a4euA+d*u{V}RA=#7nCPf-I0Koi zeI#+18|>ZP!IJ@pMKX6GtK?_aQ#JWMG~s9hIZqW;fSlijIe&W~&~P2rT>0^0_SYj8b3V+v&ZdnxV?J2z_lGBkWhwCR*fLHC~>4VM4UZip34A@8uzGUkfav zH8SPj$WP9=D~P$EwXw9^!PJIT40mi4<3|QM{r;5)G=h#0FsnWkvw`WIvAJ)o!ue6F z{;=l{LGej5z_@Llk1%~x_T9y&uobT&qB0Fr{fe*gPE%=%i;iiTOFIIACUZmspRLx2 z#Wd6ixM*zz72Cz7&~~p}NQDcf!ifWghxrF&IRG7+$+EIK7)4(xeKMu*d+CP2K?DRv z1mW#3{~cL5O^L~uz1ONzDp4s8cfN_EAMi6#)4whNqn1rb{s}wICBX1VB@{Nx;Tygs zVbJnQ^c7|uqGi;G<5tL>EMfBIulR%VJ+=M2gr7^q+a6Bf?kW%!M1sv^0KN+wjaahEHEg9JC%OTbke>k7m72cR)mTa0hh@@*fxbw756LmkNvp>Xb9 zcV(yF45M1cmgy+UMC4*{L+M-<^_>&G;h~zER?!g3hi&A42)s65(QTkRAcX|mD^v)^ ztE@QB8Qal_2^*>#;q$5lNb25T?h^3FzRfK6M2RW)=j>tDpZiv+N*to<0pD%fsh@#U z5T_JBm7B4@B=7ay7f860R(O?k1lh)|HoBIsDl0{rhm+k5gAnhxQoJkN=?Exwt&lr) z*%P~LRQ{NmaAx+Le|TvaCRDm(;y5gF_qo^Bqznp{!xK6tLGXpqT`0q&kCi3!1rI0s zI6swBErbTj=Q+0ndwWi;tUXBY%Hg`Rap+p?cezNstc?o5(e)tJvW8A@T-338!plEd{eWW5_NN<(Piq<&{h0Y zwiB)ajfea#$n$GxM1u~xuj*Zd8UoWmK7}UUSmZL$C{Xwx#V%#nkD$ss#5~FPd>Lea z_gK-i4_`ZNC(lLU%y4SrTKugUZUt9CH8Y{bi)gM0mfcjf08zK-5lu7pQnk>qp4eNM zma$$Xv7?z>C+Zb01$M~wfEFx!3m(_9OTW4|!f@UmE8Vqfw}9&l)L;pwzSejgYMt-W z;{l*grxf;%k~@0T+-GgTV%NgI;ie5GQJLqu#L07A!ckf zO9XTs2U^N_58wYMTM|LtKO{wFbR1%dGNBPx> zjeTUT8L{`^DS9C5jO|<(N)og|2^~Sh^m*5f=+5Tr^CY^9V4xwXR(_{18r}D1Km4Jx zg1&jmxybbZmsp(uMZ;Z5a31M_9v3-dBkNN#q|IOHC|o@B_+yn1D>$w`7g)m->$!4e z(^Rae(n~g%`}2Gnw;$LeO2}sGf6_y%`Rl~#qt@wk>VGLjPyO#WJ>KXok?*Vi07plr zq2WFml+Wnu19y90&`F;tcyPmst~`y+o1eqp7R$=?&$9-}D-!yl?YRL$-&}7j_J{$Z zznid#0j29WIkxqbo7#Qg@L@w>;EHL$dp-?PSQ!Po@S8tu&`t+*I}GM=MA>W@+_F~+ zLV-$ZkQL76RY_OmYDT)8>EWDU}Wrs-?Axy_b zrksD8h`imgf{wrjgF~~vu8xT(-E4e>!P;~}w67nj&8A*s3O|MV;{dXI z7JLORtHQjec>P0oww*V{4&UxJGEV?Scf61C0w7`=VaT=3D&_d0mMe4pXn zTzt>5y=&kPwZOno%})~5QeGgPW0-OCkXyPn9mf;1K-Cb~%=(&1qd7gc_t%169=+IU z!MbJk77DYiB#~O8u@Fi(%&%rtLQeR*rPg&teio)0xFFubKkjSJg?2JA5PNVhwJ6{M zrWZtwz+wcA82)}(eRqQ2eR7SG21|>TPZnfvlhNK{H3UypSo+su=S?H+^I2TDWbyDw zzISS@5Xm9ndSFQfYzA4pktKc6pNXPM+3AIgmh>B7leZ&A0K(@2Z86=^pJfnB zAC@hmYsf5Q*)RTU44izPq2-zaxt0&!H-dAH2e9`yA!zBOkXhvn&Tkf6@ZA_)Y(nn^ zfHkNd$Ux24YR|rR^sUX{v_>Sx6dX?q^c1HfdA;pYmaM7l{ScPGe#!U9t^cg~1e*;C z*WiwU&{MsV0@!dI{680>e_jKEHhVX&@}uf<=4ED+30pQxn1D93Lm4G1OB*24YdKW= z_6M=vD=XR zD;oRWKY&B8vsP7NTW7ZL@o~YwGnE)XP?6SlWm4Mis|axu;g0JMIKFk;W76@);Mi?{ zH_h-+xJq%Q*0b7c}@D^e|C+Vd}mmR+35X5VY4a|+(o zi&)_wjqHKz?6uhUr+l3?gWwN9GH|pU+$zk#?gVS@BTO4Ixb_79@B;@&q}iXiP5E-y z?2Q@=wtUY2R7|gsyx7CHcOwh>H={m&bdr7i{W&~*l1wMUfn7AgjcbSYaS$&KJ?3^u zYrxg;=Q_Bbv>Yc~jr)h?#%jlx#UV(P;09u}^(=A7Q*H$V4#Qxtvb)1<=vp;(_@|LEhlt32HE99L5MhLv^yxx}f;!3CZ(In2%FgRV2KO3Wl3vdPy zf4}s@YCS@}5ji8*)?FpLH=uq<=f{~}+wl>k$Gk-?a{?$V`DG@i)d1)7S|8t!vrflc zF!fd=!jtdU*%0z^+c-!2o4<+;rswz#T}Ur-p(CZ@D>rI}fQ!PFxabpFaS9jjX&+=VjyGd%5@h{Nf8}?UHx)3I~(2)LVh9N?Qqtyc#C zb8}RdfwNji*$y`TDyp`+#niY$jV=;z-KP2q-OwKL&;9G=DjnSx?swB|tFIfwr+ZJ& zs`6swzM%)xtfSrS@8qz5Ztfvt<8zp2+fMM_j(az`M`$IR_`6Tu^JDrS4_zDnW*$b) z+rtD|PeOZy=nB}o?x}3I)Wcab;g^SpK^;hAhxi;XM0$p)n;sr9zw^4@qh6C#FqAIy z7-igJ_oYZ_t9Zh+|H3Vs40^k`3lGb%I-MuV8=}w?EhgFNUqDDLNq{@u?(?4~0|XKp z+`f%?1-O^N#e%M<^2+mJ1y-KGs~8e`9f-}ffqdmM5l=KZo>S4*S32inK$EzpwYQ$< zcW}6-(hL4{N&&nZUi{L0^>)e6%eYij@os1^vAuttM}oZaYP~%~-XAzmmCc0=16lv` zkkNR5=y`%q3j z>21a;Bp7K<8ob|w+jx7c95Z^)t-@)p{P~=Xul?hlkznE%@jjgk?_6I(|K;Sh11a(^ z;$yY*MFjgW;PboR6)Wc4$AH8h`tvVd?EkcxJkIk1GoLhKOb0)Zd zy>eeOD&PyxMwc{W&yns=w)xIwVJu4mzQd_LbKfWW*u9xi9PqyL+CBWl)epZ?144`A z)8oSbBW(yFLq7;E8gI6R_;;8TWa!YH&*#k_hS@`|tdokx8_poqkxO;|S#*IyzWs>U za{>Mf%b@f>m|c+gMN?fnH)8z-gHXDM|GV^l^c0;qOE)Qx%X>qU1}H_%h3p53zGZ~Z z`9d<+uy8;+3JdpyOk3Af;{!B19# z&;W-2+vPYl{i>EZLNdYN6ML7wWE@8rzP&3@zTtkMVGdIC zfES{Z2{yNLSJMR! zoe1lZ4nOxqIU|=b>YzYpb9Bw)AB&=pj4wB#A@`HtLhVWr0T^IW{JNhHmUj8y6eWL| zzf?PH@4J&ezVwBC7Ttu24T$jvz|47iIJ`yB@bd}w=#OdO*uvey^9`H*K?IYZ*d zjIT;nmX8Im(I~Mhzs6|)i`jA&YW@8zk;f`JN5p8dF6II?8QJ~T3h&lOdiD7k3z>yp zs=S51{bHeOhXS@sg6mv~wejl&9=NxD498eo_(F^i0_J|h?88KC&`Wsk(@Ug7@gfbp ze0GZsfvkyKU=W*<({T7}_>c7dZ!$zDj2n;bRbP8t3A*lve7}h}aGxFcd~b0CeoW=S z{adH?FGP1Qjydg1YlMD}0HA5b$x202##tr4ZO|xwT%?(hfFX(Yej8c1?b}?&?KOvt z#l<(maXU%O5|lhg30wNIzGxjkf;oe*N1_g*?5|(c zMJ6R(SDrrChU1RBIc3 zzV06cQ^E|8pklK~GJo!}o{;>HYH2@nJ~qxIRvG2ZX%|GVi*K(3CpLHco^R3|L< z#)Mad=D~|60-zfc^!e8&!sJUL(;GUTkmHs0P<&XTktBlN$Y%<2E-&5r489jBYeZs_q3I_cdQyW6WOSVq5cZ5n9rYUtC-2 zgRdV}h~yKNP`F%oIx->ZWdlq{@Ykat(zOo=>kP8 z`FESRc%CMe_c!-bA82BjURHSXtJJ#&;SR>hjNa&oAIs;~TSDqz@5H?gngMU0?27?s zKN2N{ex2a!_Jwj&iUt4Lr~KhBgd7VG`fckf9=7FWM+{826}~H5nCYyMmg>UgD`%c5xT*z{NV$|};38m`!0VqBdmaNP`I2&+QUb)6-P}JDiq&q< zAZ2fVl|c)@5vbHQVs53}`0KBBUg6n0#b zWGADqOAF)!ISa$D6KoUDn6)SWDv2=O23JS+Zk`G$%)DQE*Z3-sL53R2wiNAYm~&qtc(p4Ax=$9atK%y8-r z6i*+v=N)`E;Q?6wxNX|_`O@g~zNVQRU;Qbz9OONyRzQHO&>4dI@rXs0g;_o>vdgZK zAh>?P})e=j>Iq0HA34mWpDFL2gHiC;DbDRI-So%wYv!@X(`mz8*R|LF9k@5iSv z$u$rfI>*No^$#j0O2hkEEhQo7iB3?`&)+a?R zeA`#pYl4ue_xQ` zKv%W?jXwNukfc&6S>i$65%>T_Df#uQ)VM`$NAPb>OKpwb?mphZ-V{Urh+d>L*!H%s zbdk&#>;%1){_6jhpE$XCJyNvOH{;xV97_m(esE%^yb+8af42~o#gd0cQ~q5H)r6n6 zY?Cql$@CjMewhYl`@Y)ys~j$njKzZv<+CAqxlTM$pMQ0<#2>W86DlrI;T_Kv zUPsRCFHGEuGfI9JE`IMKODZ7_q&zE;5Oj+K-WY+ASBWd@V1z2IOJnVdvVoAw6yJf| z2l{=nOO0vDUaxun@J65*eKRk?xPZR?YxZ;xFl7xDQss*0_-jl?g7zkO%@x2k!=OMPJ? zEnxVnY850kCIs%OEHb`ze;E;$Cw4$(oWkX79ac$9-S&%d128cz~5@BhrCSpih5w4WOZ$ zeKL@;@8EYNb1!>l_oxO2qZMNuqm!K) zqmKyGV8VuNqT{cI5k9Z(F@9Oi_z=}%Gjrx?bh|ygTK2G}%|D0}aMt>6;Dq+}1ITb+ z<+aZU3hTPID4lIL*j9t~wKHIc0-(kyI$-TNYq!zh?@I7}xK(ie)&ZefaOX=`hd3sU zn(YVs*0&oumS(9_cm92%I)5w#sh2~+^8HsA{+~E0z!daQd3=9$F)bgS+fn{;<7+0L zbps&jIi^I%>|OscxBbg_{g?hJLCAs_)=2vL^<5JiEuzrjeM?^Kr@{XKzB2$7na9$q zKoEsrY4Xfgkt8ZX5nTS{oa5Wyxf`pbuxNU^&-qUI$+oD_nKptu=TZIQ?5%{n|23%x zqJF6H-(NH}>wI$=xZ6y)2c4Mmi{V%F@Dig#xm;QnAX4;DU6vSQ>|x<%)9xJ)7E_f} zFlO*=-?y_Q&Lt^9AB(7M7y#B!K+=|`F9vvZV(mJFJieP=JAIun2TJC|_fD(*2hHu% z@EF%&{A4`E!R*IxeT-f4>0^&tqqWBjEb`hu4FC=C2e1i3?Y6}}GggOOFv4*FtkNUb z_;23s?B&G~eK>2W#Zk`4*eB-iR zZ}(PE=Ckq-Q#W2H?W~glL3j1q#LEL=l^!sG@%mx`Y_9iZ!;bdie$qceWF~;Fe?WDs zMtepH_`~Cl@?~yylz{v^Fpn`SE8|-1+jndQK?#JI#(~h8tK6RAQQknD;9S$nnS$VC zn$B66>K|&a2_8C;P~?(?n)D#a$QCWi7UPpJGDImY3-I@wMDB5(aKgleK5)^BnCHI; zC7KGqA6YC7ZY2W0H1^kxCwJyQbB8la6behe*{B7{<2N*vPUqq_Gpryqe0eJ_cx z&fkyt3-K!}1%SU_@*6)~DTN1;*tLHj_ylhOWS^5OSa+Rn4ofrdhs1Uy8E!mqO^Vy^ z4&+R0v655|mm&+#)JuQ(Pm^ZZ?wvLH zY+ows*h#Z)!Kqx2`eas{w(o{rjqdAY5x!|@TjZ+XoRZRyJ*Rj}*Ry)gH0|Zyptn5PzYH&o3--7fOcmZ&bfM4G0c7}We1>w- zk?;I>|J%c_5E;SI6(#IayhTQ1Ijb~n^1Fj;z%vG3ta0I86CY>E7_yv#y<79lE=@j8 zECG&yJls%r{Y;syANpEylqs1wQL#~f+U_1IVS5AT8fU66jEl`)!M)N52?fv8oO|pd zz&p+~t4`u9(}Hlr{S)3ildF$mV|6Anvw9UcS&-D(AQjm+--NI?*Mrw9U-~dh)}DU( zl~AzyCN>CWuE?_07JK5pUtB$OJF84Z3~2r}E1&l6^;=p{W*ss!fx64pZ}0rh9zoXe zTC5xTa5J0otmlObnnw>TF0%daN34W3qz~ZtIJ1K-Svq&`^2jd@EwWR9JO`Kl(1mtt zIk55#!0>&Ee6i-zn8sO~y<4@Q=W_3IBq|qSyn4oy-*st7Ia$){bt`f@uf%DI>#^#{ zIg-g@OY= z_lNcKqp1m!wTki|RVwqM%k3!gKPlpUNMjdYt!&QYUS-$%ugH`B?^QJuF+?7{zInRC zG&=F6`Mo#CR1}ig2)yPt9Xt!* zylF>@UM`;U7WS*xvByK0+jPx$IfS*UeKCD-&ynuD4RHZ9MvUTFJh`}=Jzj$P`znBL z%r_m2DdzwLHopON)gdk*^>^WyNrfzG{{nk-X#vaCI`q9S5ENcS^WR+H@h-oSTS0F4 z4-cA{1tJ`7ydbe2Z`Re(s%izC9<)pg_cHwz zp1)ZfB=kzO;%#r}xP^cQ)nUrl3keaEtV zUQOzJ(y2+^VyErjJGvFS@g-C#17a7aAJP&qlOMSUB>HeqD?tfYFYJp%9fH62bVOuH~V9;L`l2ngkbkP^AL@}zI>wRUk*XZbim|i^#0PgOhTX>w{b)n z{gnc3imRBV0@)`h6ASZ}msj#N%Nw$$_UcF5t6B@gLsO|nfGp^c(57m1H+svPS#MoS zcVv^O$h?l%zag1;i)mH9>E(y9yLQ5c{^-Td^IN_b*Ev`v-rF-S|8B(VDp)3BmwU*{ zYiLjLFP^6G=VUEDvWtWC?V5W;m4l`2biLYU6q#PLPvGQEdTbgsw)J|R#*4td9MuqHLy@P ze*#fZ{pAeeSy}dfD5i?3yO!9U8^jH-EZ4e+fB1IS>zT5531jv35Y>kWE@B)u?2{lP z#ayjEE)x{P)~UYItni&q;YFo`$wsRRz7EUndW5J=iYn~h;0d2NjO{%<&wuap`@6Kxnu9IEN#qJkTEqpFVVJnYp8wXijw#$F8#`WvJ&J{D1ppWm}7 zya~nA4(+}gi}+TJGq^6G`#{DjoT!5@@uQl_IgP(6qJOhOZ6dtJ?%&l)=w4YcgO*&gw56QN)}S1PjuWxjTu72!oB3x^u%w{)vpPI3w&3x%4;TW zPnGUG44azo>hE=q1AaGVeF33C63)d%vErEeQKvlD+=pkx@SDh%sCzqpJc4>I}?UZve%DhHBcMZW-&6p{*v^5w2Zu zCR=I7&=-5%KxmdIsap#Ub=`g!3vzTMijen@KUHRx=-$?y4Jyv-Ij@R)z0$9eDZ|~6 z*nJ_<>D8Nyq?5J~uCqJ6?H(G;YNn&?J$r_GooXx-ZSY={p(zaMVq zsZ2E3b!H2SXoD28v6fxBCKP{eEu0A1__t*?Qy0`S7c)-a@SJb-ZnaDj;otMmNRwj+Jl?7395>UQsgS0|{&@o$U`D;Ig2V_z-^H$2hj%RQIaiO;DC7w%| zsm9-78{U?;kGMig!X6m%HdaM&ZnS+g!Doe?G$t}bjDkCJ?baE1?~+S0pp46oqt6Jc!>)AI=p0!x`KKGm(wb%4NS9{BoP+R_cR3TKJ% zYb-=t+317Yy=_Vk7+I*__Plx{61gxp9s(aI}OVtX=UCJ_Vi4sje=j) z?)WV_6j0c7`0}nKk`&2nyz#jmI{f5+mq-p0-#*u0E_a(^{ODNzzyy;ZAy6{x^81R* z4I|$0j;qsQMxBZ-<1@Ex4z6~V#Iu)o_l6;YuIz6rkA!%i&%kyacg_hud1*Gi`o4(H zC04zlF2Y}5zZ2Y>egEa*?EZo-1%d$HrKr@fo4r@7?uy^&d`Bg4FX`(g-OSfVd@;MC z!tZuD!@N>qSBl|G_1|XgrLexKG zh^vM_u#XAq9ew)I{&76x7mzF7zk<%WU%30}zv8dG#r-!ja4U?+f=$dn2>%Q!`8C4m zdNnS6JhlM>E$!2cyj zA2Uk&MWmF|)ie~lBgB&1r}r}2=FhMwkRjNm*^vVJ-K>Z1Edz&S+o(T}BD3>h8UPT# z0XXH#qvMJ-lL9ipt6$gaO#}qcO0a^PU>Xp};8z#LfNxtH&!fYRXEw+uH=!XcWIj!o zj8*$5_&~1_0DJsF@u}~7F;E8~uO1j&Z3eAdP`DyuMEu3*I5m4FML=+h%Un$JK&*Ai-IYj zG}783U4Z!v432-3g8g*XQRR8=kgwy*m&8jVX}A)`^PzCnCwV?Mg_C9)N=qsQt!~{Q z<6m)zj7i0@vt&nDj+14+|EC}nzK+tas5uCanq-Hw&TdDjS+~(dH{zmd zG(}CG*?9k0Djc6$g9i(xXnTzQPof?sufWU4uj@o(WL_*+AG?#- z_6L)LhuQlHK`zVz5T{-MYbCzoQ$*DOV8N&d-UdgXp=IQPQ!5c~}S0{aX;pQRbsg${O`@$}| z07HF9HMgG(PM)B&g>nOD+SPyBs1h7Z{_~WXsaHcmNmMdde0>l!Q*gT!E2s<&NWAK` zgwzwjQ+RB5Qe>g*`;^356|F$yr;C*-<)W;~M)Z#tOobi1bah<{kqJ(NM>v%|el$R? z_Ee{QmVl;lM}pX!m9{>0B`N9Gfbg_r7)|wnzu?r(mzUh~Ofpspxrwg6n5wDAq9?EB z?F3PLv!YAQU)uuC*a_Q-P0_Sx)2M$W1PS}i_-OHimI7rC(Dp)KL zh9hsmG@nHyjVaG; zI%r^~-%zlbOMJwJBGj2n%{!U=opx{j=qzhyg?U#tC7J9TkKzyN{VX1E*#2_x;h&Yb zTV>s5#qTJBfxfj2K7TeA$1tmqirbsgSwo7K<%^UB&6~$ZHor+Ks>zuHI=saUkfwS) zJ@gI?bJ{aBm%(|P|IBr6Z5t;amY}IAw_6V$x4B>AZJt`{)i&}5M!OYY(902g<0mp# zh@nv(Sks?ji?S8owt%8zMIC33se~)qu*ILL5Mhgrab)YOTM6sbl^ofE;`yx?s&OIo zSULI@J@MGenG_=79Iu~6)~k8ne`C+Ub-h}Q|BPKD>n$7J8F-zh`F)a1Ew8KoeUqrM z8~@cEkh9fDaxlHq)u^unBkP+WzdZq7A!}gF)(=h5`n5sIUAO|8zpzUla#cvUuHi+e zuFhHperu?8{q~SnEr0r>S8rwCLHC`Ctno=bOpp8XzNT18)LNmv)Bu6Hajd;Atmul{ zgEHvC8(2sB?jIN4;ZJng02BiL9bOh2zu2z*=<&SpW&`W{2Ial`hwxk>S3Hm;i4IMvSgxnEE^}wCL8{tC%G`dkrAMQqFTFp3o zI~Z;BQevyQvi7qv%xy&@6evPwZlY6t#xi@OU+3NEcGFz88Rk{}c=s~3@gEzcctRPU z9Ikv#WqaAcx?PwePO`wL)&AbR*!;DX^==`uc5nKLNZtU37o-+u3S$pEG z4b9%8)=rs!GJGIe@&}NeMjd(-&r&Np#3sDU^q_tF`s`&;{X+=I3zOG@Tc^B(J1iD9 z27f!t^|wzr+&Q|@yz%H)cKyxwhnRkW`&{S!8L>N>*nuK^N_a^PYbV-=>^A-_(xb!2 z!1vur3`UDb@fBF6(`xs>95{-eUjF{phmgJ6zjYSa?*2zv*nS<0u}*;!u=m(rFu4&1 z?>cl8B%KmK^4W_;=X(<~;K@ua$w{yYS$lWCA7iUSJ=^>8wLym3PI}R@-YHx93=M9! z?{05?IIez4pWLiqMf^jCeeX6>2obBY*a3FmjrzWz`*{Qty!~bmQu}ugRHhw(3e8zb z39o#E!2zLxuhGa~2c(t$1m5T$utJUYqd)UW59t#Wk)n`>kLdkC!o=m5hF=+eKmLie zeLo1sU_*@bUbt-hAW1P1^-Jyt#q>&MoHlbQlzfd5!6=l%tBiDG@pz!@gjico^PvCl zy#<>&d~ZLQw&<-nL*x<|j{KsUZ+5W2f<7cqZZ^Swa5%HUks0;EZD3&P4=|QZKBQo+ z2mf6k`0q*zT`uy$hxr2s=WjWL7hfV?(lpiA<{&--K!SmGDAF2@Mw;nyyEj>PSmOIr zgX1M71AN)>;(S%JqJ?Ki*Y?{k^)r2YEaeP7T~fgMnn$#ZFm+)Gq2&t*Eqz8`il>}iX+cLXc6nS9px{TzQ9o8eC?5cavn;5qh33ZpyT zg_`KUD|d>c@@^Hr>BuOm}A>Ml%gYjBLeL{;|da=98GrQWf2 zjOBgIOY(vj4~kPEjAT$#j>WP|Co(5695Q(B$esVP%tQkrGr7QHyNaXyYc*cFVRP*7 z?Yl+CalsY@2e4^e)v{8+UpD5|37%siM@2^`XdKOIPS`faM1uK53y0R#-{KITI0)%U zgp4D0r%69LDFABMdA9vL8870ovTWju%#VQKqIJ%^^GX%URRjx?QuWmI&vEA}tp*xJ z_j^i#3e@H_+>67DOJqEu34E;F-=~q`8vqp{09XG~m(G=@W|0!SZ?IQPGq?|Cj>mkLZ!O(+Pf002;D7fCqcSw*EMbvPuT z0eUyJ^?Q)!2lxOzN=<68gn}i*Dx7{Yc4!}~)z zBqF8E4V01y)J7Hurb8h0L*Vx)z`0Wf6{s>mhm!_h8X|P?f&#GCoLC{Pm-=BRK6Mqe zYj?*9@nTK)h9J7#DG7tDM|KM{@C)$=DY}JeGl-BlYN#T4tHmRsH}Y+T*bvGk6ySMn z>Y_~a6LEm9Ac?OowyZVr|0h`nKz4Xxy5dtqs!3Ez2c^zag06fL|8GbX1B!4vlR%d> z;#qaqHiWYd;dGJ6R;y$MPGengWZ;I7;*dr;ZF%b*BRyzCdN&hGW^p`o7GaK<)0~#V zY^e~gv|OH-zg}fkF|+25GJOC#zGwxhtS7=xVof+|nu}(w5D3DH;^TT63wJm?il9~{A9O1r-Cm@<}{&c zX_fZz4@rJ9Na2by?}P$0T%~v-A^M=kC0)7HAF@dhgpY;jC2t4r0jJnNvh<#31sE5E zsB5RTxaEd3~C72o-IW$`VBA2bzRk0GLj9pl2r_ zFPdgjcLdX2>f9n@1e!5fr4Un9nZlz<4uyAVoi^q?6ebtw5C*f$8_L_3bp(}Qx1Wad z2k5g$!pJb8~KtnE{urx`~ z(I+(GYx-y9xPGWdGqvr%5y=o z&`j0D0vUIV0FrT88CnTD7d00g*%uUO1Fm`RHM#voxy^;R&Pc}fE8@R`;u;fkZ-Mpy zNP5J-LcZ zB5`S&(I82yu7NNLm4}AVkd}&nQzW?pbF4K%Co+=K7NilGVVp*iKQAc=WH4P8Gd~){ zxM-1kFPbKj_9{T+^_s6494L?N|SQI&O$I z;3aYeF6AN>z`I#Gkadx^LF4jRpdP4m6+xhESK=HK`bkR)*dvkLa@yq)02CQix;@2H ziHOl^Y8*d_V3yG?K33`!##%-O=&bmtJLC{4Ac~~$d>PqWHQ@3oiXWE<~p-VREL ztms93r^|rgDGX+j1meLIT01^>6gh@)1b2v6rrlEck9^3vZpu7OLrtJXb)32~lGo)I zT9S(Lf2Qz23O8$uB)nAMUT_z&K7-dHbN&{Fu}W3|7$~80I$a#|d}yb1J`iJPM~%?1|XO84U<)&meJL>TIQQCM7>ke(YkjA0qXW$8Ve2zD`s))K>=F*z0- zA;6)UigNgg10*E}B;FCZfRTpNbQneh39<|b3>|m6T}s}qHanymD0XDoev+Sp#QPp# zJsv_t6G>Q2WQ}}+ojUl{8X=*WU`d2=)d&Qda)L35ln$Y~Qcf8lL1+b~#_O8M8FJ=T zNnyo*kuwb{E2reAlQ3czf-9v`)oymRoZ$1Q1|~!SM-4~OnYYATB^DJ2TxiqpT6G6% zMPgr}<6)#MiS;~@qEmP_IITi%0WcMpXEzImC2@R6CNT4HtI^)oLSgcR@*%8%HSt0RU8fO(sHa}Q3T2u@WqPZNS z6r{w?S{d*ixp-7_9hbJ{Y3Kd}qxzwTqBzLZ9@SomDlQ}Ei$?eoCPSJ}GO=ATBs(HV zcy%C9)|PZ&mtJvfp+BkUa!>H)$=T0`c8ahZ#5a(E7kAK3bYsPw?=gLAE zRdneabfoNm5GJOh8Bf(sjn*q2_X81FD@0X`8S>vvfU=pFc{ydjWYBagp^G?q^aFOy zhw?3m`2a*Smjz%45MYj=fs|zAfk4xOX$Q1>;6oMHnrfOxjLOR#wfCrFbe`Dd=bwHUtQ!P9t-6`RupV*g_IE)_ZFkrRSh9q>3>4;{?!6&d= zYdUFzqah(6X;m6tXz9;!7=?XS?S_P6FKGP(kxf{Yri9ljD#ycdkQN?@$v#6bC8G{N znUI5qL7t^GjbzNGHPKPywhQRXhDrI1gO7{G+!27ne?=x3c4(*+Y*dzDV<-wYgH}B` zb$zu)!T;d{0Kxy^0{~b5J@SZn;DvV+J9^WnTL5=c=qM%tJ8J@h43=w07JgxB)n#bj zr9~17#4Va7Y;$5(j70fUvK0mLAuP8euK0v{LdbC_(xagoj8ssHl`EIPXND5iHG_Un zTFO<1NHT?1BPraS)^kFY`C)p$Alc!oCyGJ=w1hfWm-aDpr09nk$VmVrk@L}U2z`!< zjGJ+naFYCjVIK=wzI2B`E+MZ53Z8m1T1#YBIh0;-a}hIW-;? z`FcyUoU2*(JJn^OS8_PAmPS_sTy!dIfPIc7R3YXeDWdxC6OQ_NSi z0zWj4mP6n+M_{9ZL!jEHY?zz?_vEGrm}XkWZBiqY&{TWZfe*8USw#Mn={c#iWG40L zCKxk_1esK)l#6pCDRrG(XX|&`WivB%b>$pZMQB;$NLMoP44}smIW~!-PLD9&lx96M z;SNMZpa4@Egs}LkVFNipounnYibe)>`0<)VfEv^vk0Z5SS$!mSb!`PKttzY{muL`I zYy?xSS7T6<=qX*vQf5QWj5Q%Xp~-T_S(xfyS@s!)BZY$i+h}tTkCoXsDx9BL`byQl zBvjC7SI-|Xs1vi3Gr?vy5$S-}z-foy4gfX+S0w_4E?P>jrG~|)l%J?*`!2y8aoCC% z*gjjg{czFmF!BLXrzcA(C_UGa1zFodC^o8<3m&*0kXPm+Sdp4>CSXCjG9uJf@wqN| z8mG9PE>(A}_X{C->=s2R50~~Il+`d8jd)o>byw(Pbw3oM*)(&sO2tc>r7d_QD~`Eg zb3v0$aFiC|Q!(&^VFr|*1qL^Emu{5Ep%*)8`H@=o0f0tw0f4HLhee#it%1ldLWg0C zM7>*>ika^^u+O&AexlO_JV9}5_^|KYD z7a5cA=m!^=a7tehBq`9 zhEPmMD(zXBf>Ea84`3i%0p4ek@JtXKNimI5K?{f!$#tlvol`7i^!##?01mpAo3VF; zaXFxJgI{T0l#`oc)6ZWylnP^MC>K^Xp(Qh>+pDE=ap(0V2iK3`^-e>cDpi|7gzgUz zu?TVaB@_LYrDtk_I3qE6reGax0}NjHpf_ZAFsRyF!J7^_-iKBobqb)7QioZB?KWE4 zN5zB=qTD!^ZHw7#ad-=Buq--Nu4mfAA(yTr0L(`Qp@$+*5hPS{WHltT9GwE%h~S)} zqe`B^33j3oM$`eG_66aZRI@@Vn_cJYHO3ihRcSh|j82i;R zW0JfIR*6xEQ3t)RMY7LzoXJqMO3YW@u}MoLr1!a$m|hQ*5R;j0QS4H|%XTqxrqQ|D zYZi)oXmiSLJ+FqHwq+%VRpx6?gfj{{+Ql8wl&vXUN<~i=R_}ADRTeO{ab}+9m6dfq z1*2_t*=V9Ah@pnF42wlrh8FjwGla2QVvCC1W1+^v87IP0Y}B+_PI^$2mCJ!jYN<0* zDx*UT#uoHNN?PDtVkf#}s$NohVFelKvttZaPjS@b-W0TL2d%M5RO-(Lv802fu6&N? znAef30-VZpB3dCxmF8JZlRc?6S(JiG#~v9V#=hJ(`(aK*s;5*2B8eZWY@UuvR14b*j-emsRqQbQ%29EAhqcRZqb^n22a-zqxe1jS3a1ujac3Ostxq+sk9>^$TBuAPHzsu} zifW>)o*ddTRBdB+z4MH2=rHM17WygmQ!8o8j;WaonAAzuv%8OrvU8TB&uhSj#q3D^qi-IK!q}QRsCIHWtXT z3|A&>gWHB-X{m7s(XF{+)g2hRu5?iKU94DDGBi;LnI%fx=f&5H83(SBJrw9(lCyCN znrWk@E6p>C?y8ptF`Z)Y-L)dYJn%EnFy-od%SzcD2u%&x1v; zMjIu>%u8t2%GbSJHPvpa?t~SQt4`C&^f97Zo=d(9S7NX<7iZM%nN^1@xSCN&(sWy( zvZ45aT8lQw!j+6p({wJVK6AY$XeGym;$%vQIvaBy5vtou&kthj2~o2E6-hgDD5Ki; zx|d~A_{8dZ%M29r#J%J+p7+8F0bnORd3Jyhp1F?`zfA(dT> z+%6ScE_b1htL9rpmu3$4VYsFZG{~%^9@JV!Qwo|L_q*I>nh3?9=GJ$4*AyuWj0$y> z(lw2C-Y9FOiMncqGruChW+cq** zAa)?h4h)LB1w{&}K9s`Fr_i>EZ2Q*@jMGRgudr)tszq|@Z~zZU9Agx;wQQ^5OJPfq zq=KpLat@1OMOuuS;`F{3l$4IPT17yPAX}t#X0d8;%-EuYT2%{EhAE`LDvV#*o+rPuBudI zxE!TkAthkbDQzPF018REToQvP5^Q9Zr7VjrN?{XqTLUccLU3wP%8Sn1UXN6pcS=XY zRf#TaiJdc{21HdEB^Of)RV^2+)qA@aG|f_~c12YaVRD zGZy{wcO zBgyMKF?F(zq1^Z;6*QrXNi&V>he}vsq+;F4!cNsTS4G&B=%w8ekg`@NwXCx7eE+V|nfXfqn<60SJ$j z60fXbODwKjdvdcAfSCF-#Q=8va(Q4tQlDr`q!xq9g`j}+&*cJ!7lh@o7J&RRi{S;s z=?5r}2q45S1z^Oqb`am?yBjA2u;?Kg`g6N?-s>DH_xa2V3iuFyJD{b8$W*AF9|F!o z=p7Is0u0u``K^Vf|Gt3J;$Z=SUBQ4T!REcU1c+}G2LhwF780ux9sxkTA{(iQwv?VMS;0D6Nt0S2OL&_m~E7#^JqS`awkxdmqL zXdIZsZ3a1R_>SMjljsZ?v2OtTe0CFXvVc9a6s&mWjyH=Kg0n*MF95-Fhp+Gem&tL3 z!|c8j^#gtG{A>V+2dpEan1Q3}?RZo*xcb6UL59P-^+*Z`JX=-kD?`8wQb0SLQe%X{jXOSOguKBmsjqJxo;}qI zAwhii316S`MnSUerj2e8iDA~Z?F{b|h>F+NVDT7?6C_@L8nO13wHlL~ihVFypOR&#{c8o8o6DDxR&_$a`}O!7_;Xi4Ho>aY7p_r0 zP6$Q)j-oUG`Rx?=jGr^mzI#vrTy7>l5heVQ`U44s7&99bE-ynw1bp_WYg!O~LGiKJ z9DqvTg&=)5SPTY1zyldz;y$QuPLDXfrm|C~7&v&H(Apj*jfxw1{hsq}h=>54 zH-XbfywwJRaa+hDt_kB(kj`)1aYk)0d3W{()2F4YT4<1CfvxS;fHz}|W*-43Xh(}m zwQ#Z$@P2k6U_=MkJq0sVfEAFNP%E~8nH5S!SgrV0D;5XM32lmtKH%q6~jXi|3>CX(Yggp4?3-^P<3^kB#Y0cW(x;@pF-j z9>M+olKOxE*p8QIIPgfvJ#?#?v^JjG1bayD&t=Y&awQKY+8N={< z)fFQG`yvveqjFpHuUMWfpyCF;U>54@Q-E)hV2+m>;$<`eU%+=^&3+p><_Gwl(PAK^ zk~TXqI9Op|{$0V5{$~EOyu<`YI~lmJ*SgRdJpamh66X*^P*&2*pludVqdwps>DEu~ z(uLBtg(;}s_q{Tf$4%M@BtJoiaNwDKLzXWfQuHrGx}}mJhNvw%yZt)h8)Q<< z7Uja84rCOkq@nm2AY_*^T>a}t{|7z{Kg<_sAam^>#`6hA-J8ykJzn|iqpz2e!Vz+o z!T>*?gmLwCo1SkhHy`p7UxM)KE)BhHkej`p4UU)HJ;-~A-eyahx0ZPI^FbkB#GpfgE&$rtxYQ`dQV*HO!%K~B$rPs^5d52A|< z8UuyU838mobf9^yeQ16b$1Zk_pZUP+)u+97pw z`&=q8h`r%e`Ey$4hS<{Zl!9TAMKlf+Z-)MibDuH}gEc-au)(CWqSb=oq>x1mUH99^ z0KWGRjP6NROhN(dk-R?-7+>||wg8ys`5RZ)7tDS(lBgm+yNn1f6{>t11K)tVQ>Qb%`|CltL>?`1E?U0Y8NdzPUCRK=ou#P=;8uux z<8plcY!|pU5pR+xaKYvpvi{lY1-~cs^-B0ti_@~fqxw!4uU3^GFCo1H0gtZU2qG5= zCI%aBH&xyqb~8ly>5!s5$)i@ZL;{ZbY~L%(Jgz!W30TaP9YMl2tfKJgs4he(B^Q9+ zK7zsui{oWhyR49}Jw&F67h^*(rk++d{2pM2U`_gUMR2^V{E92cRtq6vdHe@7^cYJ` z*tR~52|~zuYULF8#I__V?asx(y(GBfyxM7+z7a+aM+Bc9p9FRYu^nOk9X`e`hRw5o zV0^d=XIQx)Y(1tBisfOcKh7b-@s3!~HNQ-C5BnBE>pUXdSe@Qq9ID}cQV=;UUxrA^ zHNCTqn4tv8`r1{1GwJa1;FSh0LlnqLTB75h1v-FJAKG-&EkyNe+RC+|j+h8jA*5%- zz)v{uo!XAn75_I(t%RhiM@;130T@(4EK#mMj@Ub*SMSYVri3CQ=B}GDyI&B`sQ({G z9#@38B$1pI@r-yM(dCQyM35QETM`7Gkbu86U@v}RM~-uR`5#4RD__u|-_KSQ-1A&y5o>aRayKLJLXl^J{r21N4Q-;*-_3U#N*4pV~wCszF!SjR|X)kkxtzFvp*&+;fl#;F3hN4VYjtc*Z%)`YI~>xKlBLI={{q$6$% za)Bw3dy&Zei1Q2bC^hBFN#h@yJ(&J9GA~D7=g4=+m{yZjXhzSiem|k&{mR}O%fO5x zRi>ADHENv-lwhzhfD$4;*KedwqN%holMSD5WpsbEAlJ93r@JoTFN{Sq7fg9I)nvGD$*gXpt># zCp)7|DU>0snGrB{Hzt(-#txMO9<9XYrfu6-T~|~ID|MJ10<)*O;T<(|j|6zO&io=E z5`K$n!chyJ0R#a8p(vytAdLNHNKxBq%vQ6FC;AI@N1p*yk|u#nbnfp_ne&ADg2;|b zQXM1NoPIz`^y25EDIGL?k+Vsrk)DYrMcIB*zb4bP4coY+IbxrLUFP_Rq|H<`JnJ9wP+ zH$qD<1!tyBy{c$sK-Q0Bp1Wf*w52g%YwQ@jU-iz-4*K9vw3QI~-PQ>6DdDW6Xq&Z1 ztUF(MYA;8y>K~t=?-3TjxvPTH7=6K4oCJOAvibTOg~wALqck%We)6u-inwNO!>Nv!sl*bZr+Lvs7t>F7 zmEnS97;WgdShu>9`}(6+5?`<94^h;8TQDZ)g-5J@Xc&)9aUScWQ6IXH=*qFGpMeR; z@AM&y-~J+w3Dl>vL~x-nTV`u=I#m1UHB{i>DlZi<6;psUSgha-%Gh!%!9&Ej`^VDvB$|-#1dCBEW|gxI0e$NNbF54m`=8EY%E@p#Ke7K z%>-(S#hj9YtOFLSb?$*gm<^S{*DhE|Gr~2j5k6q;q(eICb!7PMj5YP*+XOqf!a5r| zRPbv_btRiHHd=IG3o?wYOxhi`8F~U>i95-jfryi)>0WZM3(Kf19DJt1rg`XsC1GC> zHA;@)v#iNrNBFEZTTO84U?-Q`7x6E@9@yDOCV&lR{aaz9o8jBW*d->xU#!&Fy;x@z zyN~(98)V%%ZYyQPLA=u=M>L-oDz`5BO}9uEE)MRz6R#hPvlIvO07uOE+DSUgKm9kd z=SF$rFlxm^&6vQsD5}tk#EwGC9jU`GfWmYzeib)JB|mYx}@4BC2;f zg?z?Q9pV>`<7=eT8Cf{q$l7=0`NQ$iae5NB?}!r-TR)dZrx@Si2aW7w{Vqm1&L+NR zAjJ1p|bov8qT%8=^opY6=WP&!8XQUGr=y5}O5>9d^S(`ZVQI1<8 zLl>{RE~SkvH;e9)>D?W71f6bkBc+CuyOZNC0ElkihLF2ZUkUE5rWR>K)xb| zQujskE!tD9Tk7OWz_UNZ=9`pKyW5A9KtJA)etV+>XZi`fLyBbcEc}ZTU@v zL=-ZIkWo${!aE^H(91li8zJBq4&zlm^!Xb&-jH9#{gbVPI;lAX_$#(s{_IZ^CA97= zPsS;7HD!+At4*&eVf5$4wNNB)$c83Nr1Cac>3=%H!5I-U~4 zJADvk_GnC05n9ryQj4h9*cc0wZ0prti1s~xmv3)f@R*4A|2KQzPM=ZDiSw4+*t_?q z)z)&}@^&Rxkjz;nWV)y1D$Z0wkx()BAp(-b&IY0MsL|G1`6>A zyV~|ng~XE>V@Zl1StkRI4b0y{-IYsEXNCCp*sFkv&u@oe$Hp3pGocp7CSa9O~ zk5sn!?fzTjfr5@BBGnBA&)^3;XXzvHYu(`>)mW9HlDZR4DzFMeBR(KaBqJxagNZaV zfsPDH)y^S53qN~a$V(!EMkWm0Ycc7dK*_t1bpD>HqA_^=(DhAbQCcD&L58b< z(9`U`YO*w}U!-P!h}_33*~LfumN>!L7~I^j&Ui}5e!ziz`F7mVD~lSA(arW>6kSi` zU;s$jc*H(dp}yr~LdkJmZV-^8TtAmjgB|dEw*B~{z)>)l``n&9_A5*ZdM+`!2u)03 zpmiO&ekpwY=%-ONGyp4L6qRx4wJYzTe>Ly&gVJxmAjK*D$4X0XAc}@b$fMvLrlWMi z1GpcL@P*i6)LT>aBp?P@%-Qi@Hqi17AGOklRok*mf z-YgYN;6YXt@fRd7bmJ#DzL=-P4s^~^lkRHlsHVW52seRz0xW7OmB*$;GjmX#z=nj$ z@$+=@;9FTwprGzQ3Zo!@ZL2#O8LWl=!8=tPuzGweMU#MQKkKysE9@Bgez4j z7+yX_viTotj+KwJBIWs^*O~*#$HFEuESwx$L>7bVBF#aaTShhUrm3XBlMdt=mlGSv z<7G_G!C;2{;=_+Q=KWtZ3 zgnLxCZ?l*Ts;BqRJvA@-MsCyT(10MKhVrhd7-8X-hfC3_QzJ-5Y8?31ed{|l!=y(W z4sX%oHDr}NcVBtl#&zmX)Ygl`S}#oJD^5x-;!^JE2%#Wd= zXO~S4*?W499~VL|BxF=psSF+TN-mIxvIB*i=J@_lm}SB2POlzNAp3E>q~jIg`*@1! z_4)q|xCwTtzH0|pWExRSJ>x^~6SKubAh|t1U0sQN(Dt)LP3iC+tQ+-5c~cQj0-{pycQ5D_#qd~vCkXu9S6^G>BdCP^h@^uu>0jQ0eaDWEY!$Boyg zrGm0}<5C-Fa^mQz#I4LkxGG^w?0HOPT3l0itLo(Tf^nDJ+KKCTht}*?b99~El{Uy( z7}h?tiOk5_^9yb2^6PKNV_Ef8R@&;lM67o~JM~!%<-AB@c-U#NdcdGO9-fS0`iQNd zA5!{M(?VCi&T+HeNYJMR7mp6ueO}WorR8fq=OY~)#_4y+Mh=xuz^$V2v4nI;gDM&o zgpw(O%zZC(BxhZGNBTbV5xPoh3^|+yU+{&#eSR@MEv8FLG^*jtWN}@_(G~nO0gD|1 zP}1mg$U2-1!NK0&GP>PqPbL)5gUs%& zfS>lmcILk6Ky&b@(tXS~A*rBa9dtteN9IQqRehO#iSD!+bEkkSo%P-CdvZ%|pZ z`0WIuW7wsy=UDe7ukUA(fccx)4A&_}hIoGj^B8^&(xV}_O61dBZA9)$KFV`$aiN!*$@|zr(y+%q-ag3xFw|wT5?P1?#$_$5(Atd`HW~cL z%+DXytu3am+?M^SSslGI{^o!!(A;79ustS*>X*8QbQ2E0OktEq?sjg&H_TYsHL*7z z6I1C0-^~&Yxmm8Ejb309Oe5MY2jR&_yH7S27+_rd9m9lH{6GL7dMD|Q z1rz&}qNnYOvwY(o3)$E5WZGYB`2W+*6XHgz2nKj_L}@(GEKy^eQG+=f3k%D|d+d8H7djt6BZv zwm-o9cP{kFiH2Jxf46>4p`fLwc0p zDH}5vz7g`@g&7_nHYs!Z;1?OjCjWaJ`_r&~NUqZxXBLnqbYt(94hHxji64IwIRF)y z$-1gA5QJYTbFyWgIRYZER75~P0U6)^=5CY^rzG87^_OU^n)k|PVi?7SR5k~9*s?|U z!aG~}n>mk_ttR$%^!OreL2Pw4&(wg(A!BQo{GoZlzk+QL3bx_k3DI(A?BH!i-@eKY zRrKt*511U{2WZKFF4R2GwF^5M?w{CJp9Kwz@p1PsJ2v{M+8Tt#?z58v)emsI1qz0c zoyjR+-cojMo%%E;pv>*7B1!y7Y0ZJQ$J`QNO2M52Z~LI95C1}4u{MNWpOz*ZByWX- z&Z~k@bR1cjs%pi-ae$IIFcv5MdjJsMe^U)PL=;$s&is(vS`Im2H}FFqS(3hmg0G0f zU681F$A6;({rJZb9ito7K}F!q(GRli*K!OVNz*x8Zh)t8$1SpP%-jj(A98GvUwwXa zNyV)}aICu31@>6f5P{W097o?MgqI7%EmL`VeR5pynm7;%E%`y*18lY@Ee}XuH75{} zI_QC-+}0N-Dk^BiiJ!SIvxgN~J5Gwh!QtE)3?y9x5aC)S-#JhOhM90O<_{A4tiYTC zvd;1hHhD$AxGc8D=K;WQ4d2Xs3dkwT1Nmyqer#Jf9_SX_UaIZnOyhJ&YkX+&6^el3 zVqCsje;Zz$;qeeah@5g}o*=S%PADBV-zEQX?j;V)vEd_m z*@;)OLCpD}Uk-ibd@;$w1=%QyxX`ch1U+_WA?70?Ttu%Y6LB>t{grti@#YF)fMR$< zePYX!@GCDUpD$c-t(GhDL`$w*K$=9v>IJ%#_mDSC_dFkAZUdJ8%&EiU?k$Y6gVfj8q*Z#VRd#s;Y?pZ7YgJ1a$6AUp*A+yxir@Zp@is8`_g1%ghT=UD zz|DDf0b-oqM*QCQl<@nUP{5wB$nU4aA4=#8JYA^BL>kjZXa+9< zj_CAM_?tf&6lVoq9e^bgVBf%R;!<{C4o9joIXjd87Ui8+7%Qo0&Q~RP6?%O5+x!E2 zdWyOUi$OQJ#jCVcXJ_x;knpPDfRO!%$(K@vS2G^;)lC3d2wv!v*T7mVB?<9> zvo8u|i*(3Wn)!xs23MbV=J~yrGasIDz_4lY4|%cYG9Q4+#l58o@h`t9hisrC2*TO_ z0Frq8D|FCb;~j&B%NYm=XJs5;&nIm8_jejVL2Nk^FM*5)!JP4l>agjg=Fa@opXVGj zSFEWG;PZo#$6XG5HArjAJOGTJ4u^@=qX7Jee=&;QS-#2Ze!N;2Hvht9NX6IpM=2Id z=P|^ehY3m*Hg@ zKNg5Uz~}oSKyiVAbV_l()EL`mBFHH~W1$9^2Fz&$3zNc@Oaja^Gg1M`q#u8rhj79- zD!6+H^f(FodVL`Ts@YlZiu6G_1n(Gz2Mq~DUF?+*bOJLeE%%Y7oDdfYtc_^AU$*|s z&D$!7pqR&SG-|%@F904P>|8_zZMieC zA)##kTdqhvn(%!J`Z2;gqMzZ?zOAF!9@ON50R|2S2o^#=yGiM0@qo~o&3;^W4!a5N z>|7@cgY)_%fe=8=53A!@JB#LRGDd}f*&X3o_*H!Ig-A>EsH7^3vd@W5^t&`PyyI7d zg{a~%pzdQcw9x;#gsV{}6b7--(61GC5FlD#523XeEPL(An9wP5yo21NB(6L56`07A z*L_`!xi1VZW}OYidZXtV{ib1jPr@`AWtvF@2g3A-v^SE`;kz@qL*q|av`Stn0tCUh zDWsVrd_kKf9F7ROR1+pk`w(711;F@mf5>C@*IPi$(PW&z|MthvuSOB{pC7vR%mB3r zF%(oT^^^TzKFS*+RH;2+a9n=5_l;8Rd9)&Y9OmOk&axkg2(e>8GM zEROR@5&9#M&&nncSJ1ZQDo3bYUB_e}r^tXs#q|RLBv<%--Q;nzdBZ0z_qnf`FKz^X zVIrPjVJm6Ht;v0oq?SMWW*N7mQi|J;QgpOsY0@|TFsVnxS44L|Zr^5w92pV{lrQTf zB?T(#ZsF9HRsbrI=euohBq_x4rJ}%Z#Myx@m~kjAp~xu%PVPb?`>jmLMLzu=36mHi zo(0)S_$RsaxpgU+x|~e2sBqlE-5Se4RNP0fAhkH!r+ABM5X}5LncU2^>ap^YfcI(= z32W&q6tCm`13~cT+(C%AsHeU{lvs`agrj!Ms3TM~w9L&ky(7_R*ygXxvpfUQ+=GQt z!({rfK)b0_MoaPDk`wp8#)=)Wf6$*Gpps5NI0S|`;#+_Rb}Sr<52?ZQx?lhLCjrpe zkr+qLj_$1F@q)P#QbU0YdBpV z(M8M)6$*}GK#Djgi}A*)u)qCRZtT0ZVwnwTUmR5W8|J|8HLJW?tAA1x}~XvY891;NUZ} zjVUN^EDiip2JRURo(d*ioNR68W|SRDso;wj!I0_fGf~7!o9?IZ6yoh?)~?}$X_N`^&o9^y0j5pZj=;5uD7-Q4Q56{kVJzkP~tiZh@3 zo(v*X{1fpnV5^|96^BCBRtc!pfc8UNFZ^2r^QQ#&`74hLpdd0j&QxP?d?k8$72a=` zQb~Cg0H0+_`b4^ynG4?#BYzfD(pP!@=?K_5m0gjS$n;zzYy6jvD87IHpmLuK(lTVI zD=>@_yp@uGuLG50QOc%y0Os~eg=Ak<$_x8fNPY45*_E0Eqzg~CWJ>d_VvTNJ_N$fY zB6HqC3-ncb%_CO=s0}Ujd4uB0<&1@Yw7*{gu0hG?#my~(rDrNN z;8hoFsO82Nm8dk%P)*Not+EI(wC~FE!>KA3R@@h%s&WY5Yvr=(iqg7GDLflh>ubAG zRpYI{VnLBz!oZZyU8?q<0}b*@a@GH%`L1&H53~}o1kpp&S7YhZK#eV)Y;>C82A2Jr zBcxvK0PYd{3{Tz?ogp=qQe6S2-mq0XE0xb<_5%*6)n{flAGPwXXD|oC%s#BzA8Ot6 z28?{A?-&#HBfxlX4C(T_#p~AxsUMj)djfS_L2%YW{UY{ruUoxl^1WF7!QyVzjH$oq zd8M=_ip%nvYz=EYsLGkSq#U$nrvedy$VKXwc&D*~#^Idqv)9#mdo&!8O`SD!ug zCK@L13~~!=z+R0|0|DIB;a7K`Z`Ek6#WmPXo+5|0BsIGd-o`(6bnT+ z{|s-#KuCW;@rLawsaS=%_#iahs!WqnQw`$x{3{$*Mz`OtxpmA>`f?XUC#ZQIx(z!0 zepAhNV42JXj;E-zd+nlpgvPZjk=NL*#vFBC;>OLBqsS2kY0pQ=?Avoo` zw&c^(S#tpJ34Z_;e|2_yT1eZ{v3DbSVIULQ*2Gt=EYf`*B_Qdim%d|Z`?rv#p&{*I z+VBzR0MMWaYEmp~au*27xO>GHc&(3Md9oN6du@?Yu@v&eN>jd`x z^Hz4gC6$HHrWKz4bUr#!Yd8Wa@pokl{y^+P+&ZC7IToXvdqAfnN4oZeFI{-VU_qiW z13qQ<<@`)!WYU$;{sR|o^(M>AOVpXB(A8^gVOw`6;D3AQ=$9L^S=+6CE|Eb`_0Ko; zevNBbv7Q>ZfHir`v_hz-XZEE{j*S0>bUU?h_~?1j7T3Q@#AqQT$vaig&$!p`b1~?% z(?lNmdxic@e{k*or#HTL9hX0%cZ}Kkr+hsJ00G}Tst)@&&d}D!-23DQC70?cO1IKn zU!QUr<*FZP^q;>F|0(r5;`+MVN_^-WFi{8g^ssW-A+R+d04C^vh8#m{8S}zf4LCL2 z&BFxVBnA6O7*hujFp&2+DDYi=A9m9~UlO9wz}5Q=JnLJwS(d3HV#&!Ef(Gtto^w3e zJ&eOwc*gRUf(^Vl6aUh>dk+Sw$#?#<3QwITaWBxW85H_p2+K3Q+$xyrRV?Q>G@Q+u4#T^ncOa6LSVkbScmMtqEc#A5 za9_{V2yNMf4hcD^>5@L&%bqsg*SNGmI895ypAqj@CeiaQ70*aiDi{ALtX;;_t1yjH z1KcyNA;JtZQAg{8Qwh4SkwO8D^yPQHj*A2 z7EDSJav(-Vhklw=D=jy7Xg()c;*fu#ApxFx!wZAw^QlWV`u(){ZG1rOoDZC4J6CI} z{fy67bh~F72OLUc1T^<9#Tf2y(%lM7$!3g&XdSW*c*0M7$P|ivY)oc<-b4BPk>TQH z<%qdA1_^ebkum!My=F)bBQXj{jHO#-A&r&NpHx&$P&*>%voD6R_SNWfa2SWKkMxE5 zdJ%v8mXOKpU$*U{p%(y>8v=>tSw5% zM&_SE!V?^r#>o3Sv>lIY3V?$EAj7ix){A_<3j@j7WQa5Dh8iS!n%;Q z3(rvvJ5EjSd43FdE2G)@-%4hXA{L)1)7D-6Go#&$7KT-5Cg&4o7v>J;ortk@IWx^i z3Q~yJ<#Ho80W`BZ=S-U_%N)#_WY<%dOEib7l_G5gO^Khe`33#<_sCUG~H7-lBq<@@TOKXY@{lg3?0O^)cpuFN=I`;xfgP4U6aR#8;tVY|)ug629&;vTuSyi&;)+^wOO; zMrHMMW!}05&+F$8Sv=$uB}B7C^2R^K`?TR#FJz6cynR`sa3t!$8|{H5kx)E+j#Tvg zOJCz{usCyvvswaPcrIkH9-c&QZ z?VFXI?;79C30lSXnvC?eRg@cFtGv#?PyHTbc6@l2)gr6Y5KmfUN(8=)M~{JRbu28Z zI}ieKiZrW>FVq^D#}ad(e+Vu)OtIBRI_+=a`+lYfu9xG@4`$6mWRY>mY{Cl!BRPyV z)h%lM_(kum#jjIq!|dprz}omz-(wrx9A>n{2EW^Tq6&gN-0B%gOR!N!)6m`iIwAz= z7qBAFrY0|J#=v>n=A%ka!N>|f36%O8;esxOKnYEu&B3;UZ6JHXZ&1<0&PXugxE?bMR{neS06{RAAmWwwn{riq)gRJT{%ee zvo(>R>$!>8w6%8yW*SOR{$g93$%8QcVE5z>BCf@wM2i%^?~jd3D3l4`l9}~J&?{?g zdu)0Cca&{?UmqTH{WB&59{$meVT6bEOA|RSQXsEsy_D6N_@&4GOcL()Cf)Wct{Q`e6 zv8UygaJw2feA+)k3@l~xp^y@MK5?{Z@rt0lzfsd3!ymC%kwVZ;*xnGogDHh1a8+&} zj|SMMPIIy=z#`7fvWe9Jwg3O8>D4kKIe!j7#9)1~X4vXU2Ai(AVSCx&s=(Vb*p~8T7PycJBaFbWWH#Ja7Q& zuA7fp=!ju`HWtA@{CdHbF0-SU-HnzWa2p7mK}bGzl+SB&)KzEsQW9*Ap`&?>S)vk> zFB@yP8claGv&xmH;P z<%d1ebB$Bsc#=i6^luAi7{Oop`)b^OMXuvh)6WKw@{GhkC(r=xH*zU__klim=2s_P z@#-YW1=Q~ko(@Ih(zAL9L*^te_v1uGvWiN^vTaemz{&Z+3Bq1o@LKk8(dTb`)XsyU z4l5}4YjpLU@&*3T&bh3Ba$j$pyo6IP;O;stF#a-1h7qG1U^nCQaHImo*cSX^G|oV( zz!|=Xqb}?e&RduCw_0NI>U^TL*Iv~5{;U#(^X2d5L`NyIUv#d(8Pz{T8FEHlra@;K z=;HjS6z$K#K7x+I*$lvIk8@`?pv9vHP+@4iZgPdS3mgvf(0*@;a3QgE2o#GFy_Yh}oW4LnIvgUA%Sa4BTt9-q(&elB$%ykrd- zZ>Y#4RUeWucrxsAoWgr_jnk(qZWGL?!d)?f!{;nb=&0fMDfg)uuCf5d)saZzGw8B} zouLZ;*fL}H{dMJNNPkEdlE;42-d%&8ELB5{Kf)`2u5mlnO&*5@kBh4$Pt6yu8AnBJ zq9^6S&9yFr1}a{|MsU}2=^g2hs3~^)ffl&4>2J8 z|8jOJ?w|g!9ks#4jWLLF%T3}DoyMP=d?lugYSkJQe9WIB)dws7ldmsaF^=xE1QcV)N>1+zuP z;oZB@YP)cBN^jly+sla*4KQ`MechYpi@f}#JAt{TDdMYm51J-POPT)y!}C5~4F-aE z_8#}=AFfs1E2>DpKe$(MCT?eHgR}Iii7i)jl1}B`D}vqMPrSc|m>hO^BRAu_x=g8P zQ3|~&x7~R#4uHLBDsmoI1p(~SBYGuC=goGf86RKyzKxQv7V{RWd^PIrWO|Rw3MbWj z1XGg;8H{$u)*7^)=>v>*hQG>3%x=)baW~tm>Ct$6kz`x@ zDD#0QA`H09(7l}<6ZXj;9;je<#0IXQ^859f`Fi8z*3%t8gw4fm!z6`G(>M~iGm&9}g%H#8qYw(l7fzC|7{PmojvNx87NxO6A=tpLrj?*(`P z-n@#U$5NRJM%sYO2z%NFd?qfGbNJ->? zci1l(3I#_aiVi@8E)M_M6?y)pPPdyp%G&LIzx>g=|1R&SzP@h#1-ET{BWdNo*F9)# z+WUj_qJ*s)8^B|Yw~=B0tCB8Tg|+wM%oU3L@quMG^(5;3aXF}UpkSvLzb8%Nvn%7K zKNIZ4l*XAq7i}9LkbY6H)<~1O0hm9RW8iP5^R!;ihQRM#Otd1hdGo#SkLN{y_$TO8 z5diw(0k|YJ(cA2LA+JDw*PVEQoJ=c_Y}hH}2mJtLD+*A)*&Ws54Dtoy0R2clh~1+D z{@UJZy|lkAz{ro5fPe}B)^`E&1Cp5hcwG7@(d&RbsqqGJ1&}me4;H3eHe=H{iTfF4 z0eeaR-6IJkHt|Bk(gU8V#vPg3+Wn!+8{2*ZIY0rwhm;wF8I34&UU+LrLDj3F&8t7|?Z(X=Xiu8v-X3B>V*!~Z#nQtdcWs|%c4{0d% z=N8!MY{Gp>h^Yn6mEA8i@NQ9I9ZSDB8;Qce?`LG}-_m*eYVgO&d%iOjQcR92+qG~6 z9S#2GFhtvms^C&A64(wz?;{B!&`%K|KZkg~ioR;s2qN>f0Xf8nG8usksMw~En!_u& z{vH9jM1vos3@sgh$XFw@FbvZ6kNyK%IRF)yS$C>16ovm%=HyinMU;7dWVA95g7EKe z?pj?HQ)y4gIeUM*4+fLojGFUxeqK@j5M~C7u)yzQLzw;D%}mTEt~t3Pg&-Zy_xEHO zX5m%74Zqe6f8sKc9oH`07HXf_n(e z;r_=9Lj-f(P~XHKMzA?`aOLYL%o&3ol7}$<%oXcbFq*4cGtOQX{QGfXq03+nVPAz$ z&@koLpoM`q9(2Dkoff8T5Mkl|2^+IZPw|BsqSkB3f9q_~A&X2}bB`&H#Z3MX2M}Y6 z1>w|UBsAs2Ew-{W;{z7k4O>F+E#^-zXD@TI#0Rg(jS`g{^bc67?x2Fd(^dHr$j}XS zmipRa#LD8SST{rX3|V^m2r6LXAMQ@z-J?Mhf3J{Q&o9HkvXqug&E(()rpA4TmVFGN zstWb@CwJi>)JB0&5C{W*Ye^xf*4sD zral{$Oa8Dqe0H6%vOt9^6}k4bss%-Z@x$$t`Cl`=S~V2K&1$1R!XW{W0;>l_o1`VO zdJns8*{(GAYZ&P7d@{C?!8Pz98!)_kpTn0myusd&qQiR^N@Es!-C*|+& zpY=s^3# zY}@mkTe5dkd0(uq)zRR0w&R7Q1Kxvg&#Uc%xJJG0zu92w-~uE5vO@&V{$a;PeVrXqdfEQEDf3w74IR*D_OVrjf>N>})sa&(7~X{MiL_&<%!Ha0LPQ z%RIZNd2CnDmjiMJ4@}~lc3?y#|Dy;y&gT9(O_qh1MJW1esit*Vtp>U&#bqQ`$6&VJ0u+FFWAR& zSlLd2gUUJue%knK)@J+9K`RqTZT1E__;!2APB(Ei1k>4avw+Ly;P2m)umi$6(p2H# zU!yTSZSf@=3+MuPen&@EI@!HLlPV+k-*zfuaTsf<7Q9Y}z$YE)`w}>sqI{g&=YChS zZdILogTYmeuv&o!76vDm;N1w@c|5i+tma(ZvBI6lb122Pfs{DL7!&6Spu@}S8`a`* zahj1abOMeX3-Js?vNPulr>L$zp<{8VbgVHQJ5wjoamPI^)Gu4&d4!ja)>dol_@rLx zn?JsdAb8#rCqIw- zf4KLTGAUlp_882+Lby3mFVH!j-~S3LpP`RLe0PpQ9gnnfsp*{etagFeb_9ACC_OY; zVX2D_$n1h^mtAYL-LE+qzAtopqp27V7Y2L->-j^Lvsd=!!rxnW5e#O=GgxnN;9|f4 z@SuzNOO2y05J7A9sK$T10i}!kzG3~HXm_EzWY+~yvF>m2!}97wQ0@ztoR*;sxl6&{ za`mO|k(qS_mytux;|E-}1Hm8m4Y^$I3pPbEx|}%)Ln!DL>nQGOO(zQYC|rKaozE4k z3{RzhVOlkE5$SAqnYs&T7uFkAvjKsK*PJ1)T2BkiL?dr)j4YLRB-3Mbm z!6<3}BED?8`*g130h6N0AB>Og>q1`@tsITw@NM$8XYT72au8e>+I;^6UA*ORUA0Ye z%KXrP&F2&L;IReRb0Z+e_9f(qP+243}}_Z$;J5=-af6*yj==Mvm#<;*t3^X(wF({uB7dw`eU&alfc#}}A+KC+~Y z5b!`&?FEXNH|(kjl@U?XU94UNV16qvpC0vp1Nf}i-yAQ#8j~j=esHi{h-k3?6<$r~ zbJze`C4_-TAzQTkdGD^?yYpEN3}v179tfABlsZr0{hfN09T>8&lIy|&e7}0l^YWkp z2z#$d%PLd6mXqW2{3u@rglTH}fcxp zH!+5ddP|Cw*$-1j-#eawOL#>8KHhb$PphnV90mrF)MuL^RX# z5#2tS#6aRBRRqaOM6kah9f4YJ(8uEO7wtio!H+}!!dk<)xk8^bxA4}AN31~aLayRg zUH0dWeexZ|ZxdzjyjoRyvD@8LrGU}})5t_Gc;#34-p_+50-Q6(mHZW?TT zuKywW^FEZu1d5mW)oC6X4@S8hg zfxpi`+@D{6Ab;+q5zk9=&E1T{#h(mw52LWwml|F;!Yc$?srqv!y)?xX_zOT@6OHVx z<3z8M5LEN%Lt_ zLQxi=_a%q$u@|J!1N?MY6yU>p%)|%?U8jJg1C%Z<4P-VWuRo|(|T zKu|Sc9{Q8yXEaG%Gr)}+q^%>D|8)F8y6!}aSYFbLJ=yXdUvUX2{`IH5d;^|lVFAH6 z3;01IC`J76#Ul{lF`26zpdXL+PBmf50y)r?l=%;`CJ`vT7XKH7ktdVt@IW&utv~2> z%5GEqRQ=x14F>nx{(aUK@pBFdE{Ii5c3@cJM#w6g7JLtHsaRma2m*{Stq#7>6NbYz zun>5R0N1aGkD3Z-+an?2GO#q#hoNY7Mh@{DWL=gfsuTx-%5GxLo9hz(<}Q})NrE7f zX%wILR~IDN>;1tm%24xwL9sV<^G#@}Zny;R*oNTJqvGH}(}r6`(4j#3PGyS+%@i$b zB&(TH*Y4mi=yVfcMtCNDu$Y69|CUX%>l>Kw1N~{YDHyG!6E)%ioR8r5l_4j5Z+x?- z)eR28e!q&gX$W|e!^aSU1dA{U^3SCT;SFEzZ=!V?YOeVDEB|s5!dPSJX%V=}6v2Nr zCKt;!3W2vve79K`BGbtG5PM#U2qSfA9U%cdW!D1j+?nM}$bbgCiPDgvklW}LrSFi_ zJG0m2*_Zrzm{rItnZ!F}@2N^wk#*%=MB!DT0N&M9OT#;RL-5Kehwo7L2kXlBAgX%- zVyxjyLNhNkTO$6H(Dj5E5x7gq$DR{j5a}8hymg9%f?#2D|XFfW1lVztWEZbbdQ(E-5%Owv`nCH9;>i}DZCc?k&9>h`3 zaunq-d}|BM;DdQop8ro=!XA~Vof9ysI2^#zU02Aa+jtqojfHrqTog{H@&ZuOTC#Bz?m}|h8SZIHWf-EAKtOo7Z-{)vo#M1>QTP*4p(GqJih0bf z0;8IaZx{ikReb=F+3^1jABfusN~+tHW{PvQ^InEiIf9z@j&4QEhs0I&`%_CJbV=>_ z;ELLiTQR=7cLJ3aMv-(Ns6_ukqZk+Sw`n1`p$G9I{6Y;a@f_ibf44YFz&mRioh2f| zr(Z`z(y4&11hq4wH<6z>(zmNok!EA%>k-k-em#FpTs}S$WWtD(1~u3zgq`wl7xBrg zQ%v~FfG867YZOpKdPN%$z!5=10QNF-k)1Hohj1ctT$+MnzZ_W|JvIPv^qQ9TiIO^r zEcc-&6IG<-M7E4qk!y4X7RoOQIBI#oRcvwGT_8XJMZq=LWNb>lx$TfDUX!rb+uWm{nq*^ zQ7*XI2StzWWlZQE41T1lykG2dLU4~MYh4;9;;v8Hq8cCp&c}SmR6JGhz5cZ*it;3; zB~zbz%=gws{#>|awxbeXIxC7@_)G;fG6j+JLFtp9qSLrsym!eidI*JpPb|W4p9JQj$v@#r`Eo zJ}md3$*Juy4C7EtT^&akhb8L;KS=>}z#Vm_^d85=(rKK58JAC9=r8<7pv1pGHr_if zBpn7{9fK6z;&QFnMXWf9AY?~1@ADaVPF&%QKmHi_7zYedtH&Sh0zZ((%|%REXD@#7 z_H*oc1;D7Kf0fi+p zkU#*wW%^2l9QFKB9v9>>|NL9rzr@1)lfcgG)gU4Tk2nHpeoRni4&Blnfm8j zlve?upXX>-lx|U?8_b%i)O?tuac&c15#SA?i~FHPLj#TG4FD@hp8k87>ls-ClmYdw zu*4?d>4>Utp&}C*u^CAmG7REF_-FF)cYP6M%M!Ouo|XRn={%_;8_JL_f%pvI=04Tu ze*1>W`@SEu@#3HHI=J$limj8E{GE{;2+-yuI%kLQJ2aEez}@c+MrqPK)T8q$8{bI_ zocWcVFOVhUYO`cwnUl!C#<=7whORFv{?OU34d%%q`uV2VSJEc~af~~8Bq!j1#h-(H zj6N|-@3+jExzrQ@yvr=~cd(F+%BI}lV?nR`Y8m3-+Yv$BT1i|Z?(HAIBxTc}~9Me=_M$w~5 zGmxKOz_@~riwx=l&KH45nq7s|f$Pt>H^0Vv-@jIf(sC1b@c;MR?x(>f(pdH1Ao%x zukDshjv&Y^0r!$qV4)D6ft1KpqI+gp9zKagJRMdrxm%e{GCKiyJ$v6X^C^nS7<@EWh4rlxGkZ&tY@ZUU z7y-&t@JkUk^_ym&6aP4sS@WY+S${Xdz3o2D6IQL73+-@f`_L70{$h#rreWbj+ri^>qRW13Svps4&q3*1<>=jMck$#=l=H-FHH_|9;_kD(4#tX)n4%}`{LZW{KoH+SSzh> zxtnSb+=qt@J3dd@D~L$Kuel!?*FlkRv~&U(z%y@($U`lj|nQYtGTN#}4>IRNX6*nVgka@qh%s)b?EN>i8>w;yBt_wY=x97vty3@}k zAIAMJAB8}t4txk0bACssiKw zJ_=i?4^e{+pYG879)I<{u>Z7PcfPBAcLam*aV;FGO!C_c`GB|bsYYL!$BLl5wNT_~ z5d#~z=yv=}`-6+ZY;uc=^W`VJ_-o%pCQcJBMswShb}BwpQG{el(sb{Sgh_-%SA5z+ z#u<`wJ`{~9gl1reKs>)m(O5I=Ea|i6zheJ``STAh z`kQW!bVIH9dAoC+@n4ZNZ-h!f$P+U8q8Ut5#hYNpo(W3M-L4 zm+KId^0hpuD#`@)G!mpl`4@i}C4*Mt;6d^dhcQLWFNsQCDItmRXDIU$h0=HWH?EAi#3b;I|C6i*+ zk?~0LQr|9a&l61O(U(R_nTdrOhf0BmPx;J0E>8lZX%Tq_cCC2eCQ37(Uh_*W9ggW7 zsTL}y(sk)lV_bS*)=H1eiO(1XcIn>;EFs}GDkE!F3_*Vn)bC}i&8gl#&oZ|EPOxNj zRm%+Un%q*m#DDcR5?fp-h|27l)XjZX`QQE&lOrx=QS!I@^5=m{bY9jW4a<_@tpu_C z1ve?7TUqX)P?Z^wzSnx_ea<&1xSYUcUfSXBw_r+<{wI^^WIo}$vfflIWv}u{tTG#v zQjK#cK7ODq)>WtbI+h3Gz$v!Fne6#le|Wbjf#R7 zSL^`qe+EQ@{Yxz%BV5H2mC>RcW>)S@!qlIzlx7 zOZB4ad#qkJDQAqdT3ZArGIq&7f0cQrqNu#)0?jU^P2|DKhl%ZLkoiBFt49^-mM-uP zQ5nKLAlL|PvQ;spa0di_;z63>jlCs*+P*h#?iyEoSgY7ajB7H)-}`K|szU9}-ouN< zT}j@CsNRpIz5F4&pQ}Dkm4qs>@fY}UJK&`}PTbRKz_x@|7+{&`x^*%@qE;i0M*7$= z8>;aSL%KT?E>_Trh`m@R;$*6}rjDzh0t~kOLE^R7zRZt^fN%1s0eL8D4Gbv^f~4Zq z#aoVM4Yx7^x^j&ZOUr69ez`MTW8-LQxI_P5Q+c8_ts+*`{L>Yz8f$?Hj$dId$b!I# zjHp`%6WxTx=k({-LgpU-LpE37S}c}3d-2clvEgt~Uj_i6((&8L#biv$o{IV|(&;xD zLAz6Z%A@I3C2IXpkX=6lble~$;#V8XZ~axZWCU;tm@jVbRP%ek%qso7Ye(k1U=el4 z1%3uMx=nQsL?%5NmX1{y((;hX`nnUdXoT;70ImDy*8SB4)&n{JRO`d?2fo&0bl&O- zmktF-d70gMd7{Y%Wji7wE3fM17j5PU^m;{JkkGNz*q^@|Xlm*ee{%CWWK6qF`L|xz zds#ES5okLP~`khj;|la@|e|o_l5QMcK7PzPy3!5D_MT~TkA^% zAx3+B{NShnu1%JbhM4uUXk{M(2Y&vRH+AJ_rk3?b@VFczeS>BFIz#-1qt56XaZ9o< z-dSDsCMntG!#Da1P>c3OyfFoB@Bp(fnzoMNZT`hkAliILOS zS>Rhv6O2C?nr51ea@{3Ex zp9FK613g!sW=kS%DxyjeQLYLfUu)86SnGxIwvVy`xwegV^~iOHMxUBibccEVzzU;A zZY+wL8>_t(USM>k>bAF#e=^0ES#E;kgZf)cb5pXk9ycV!kcslXSjT|tcWx7|yl@(4tm;&YgcrY>o1K1(AZEMEfGH)Qg+ zllU5c)gbDSZ~$ioW<*^J{Ma5ID3y); zjj!JNWxT-W5eaQYZb5MO>6rgPjqCW)On&s>pdgI61@)kpp@L>)vFw6Pr3`CXET`>I z?o(MNP7{3#K6Q6}Q+M8EZ##fTA@G2NVareC955+_-B61P52e*^Ve&joCMn=7Bwx`V zz=$D0qy8ZzEDIe{bRhAhEG(tMsG8x$MPc3(pFW52A-(#PB8&>V$HLw-6+^%QQryD9 zR{<4HEpjQjbMQE}yYi_z8o|P4-yfsRRcpgq3KKMPz3hc6qH~iTAZE?0aBYEI*{7EH zi)rDGNTkk#H}#Aj#v4I09K7%x2u;Lygww)nAj8jLmQ4@o5litspZwx&g7?a-vljw- z!Uumv;1F09ak8X@S<3Sm_lF2p9S=p)k&UbzKH)XAU91mxtBM~)pgUiTj9&aSq!L%Z=PYsy^D!9egp}k+p>~~U2x!)9sHD_i zh!5}4qonJ%Kg*XmZaU8-@I36KWC2`C2Wnr^Brj63lo5&e0_G)m(axS9Pk=Ren19ke zvfrgZ{y-$%*~=T0)H{GYEN@N|lMr+c4woNAYI!UB9>SNikyp(yeA7}MeTbUam$g!Z zDumH?DmAEzG)SZ=Z=()#Z^59_PDaS-O;7Y@xD{rLqx*q^>QOZ!Qc?A^`Y_}= z(*yKaJy+wTde!8o!z+~G4T&nCF99)?UsNk*b z%S^e0!&>wCo%UBC(KkACf6G}draSi>!EW>ao4+$qO9FV#Tq|Ft9W~JcM9BAh{u;u+ zBvNZ-&zs2iU*e9hLoQ?VgET&C8#3~0teO9TPst%rs;RHsq;~Ladax!{_8%oal1lFH+<|u>Q zpxO0x!KU&Tg$bHaa8lCgzdB?TO0W*Sfae3LBLl(Ynt7~qb*kg9fza)*{-ef*^z$eF z%py4~QU7TYy7&Ua`b+Ut%jkif-hTVNR+r!of9$iOJO8w;Z&`IoJde}s4wNf0z(cJo zqUle=JQxz-YO<8o2j#r3ri$?Tjh4CvJ@ctyF>cnwbS+~7*C;C-?yk8Br{3(h0{3P8 z&JD2YcVoZ?qJEnZNr?y;Q`A1sJwZSN32?#*f+Rr-Pr5V~{CBlfb`-X~#u=Jj`s#q* zz|$h+-Y}~{8q&(o#YfXfm|K~nALM^ogHoQnq$v8M9sfQ?1gCdj(FVg)$I;9|q6)pi z1O|-W1%LHhAw4O1h|kGC7os7JM7Mn59qQUyLbqzM&gaGe%!K!_(zCfhG zKh|E+HiuD)k!jL9h*h_VS4}kJJ`W9b{nQlV*cy7!sX0U){QVg{qeKD?1Ak>H4g5nK z>%p$?mf%_P zMwM{60>MhkRN#Fb&`|XUaieaBwQ>u5OXs+Z{C*pHtM0f`s3;J<)4?9$=n)UBXHVQZBiNEm0RhS^fM(sx`*I8 z={Re?-n&8zB{vpH4}8iQYktU(naizGZ_1rR-Ib@4Y82vt;g6GtG2R5(K-4(xW zhRtOmaMY~yyI#5m{Ue0UPQCuNp6nNNc|SvCnI6A1Y|bYL%n5ioeYo$dvn6a2N{g(D z3hEd!6WV5A+M?tlrbq`K`8s5J7e(Kq!Q$R!g)P>$s}sgVZSl8ziZZ9%omwKSCwOo@ zAkJ(;G}6DcC9&FvcZlj99((_KR%8(Oxh2E4(3UC~Pc38G@>{D9_&ICK_}95bLCaHI zwZf@zpOf4lmA{^8Ad#+r64|22Ciyi6$;8#~9=mn&=Hu-4tna*wYqW-PyeBQG)z7iXO zG!Hu3W*pniz_Pxt4x5ufJ1$kN+*|*#A^I* z?10SVe2}`+MksfBYx1zi{Pi^cmToq}&UddAHuK?g&E~0I%Rgel0BOU^G2+ZMVdoIP zP)hvQIgAg#p24DJKJfEm zjJ&Nb{?fD{0@MPe4G1+UJU~}|Nz6rZsY1nW<$#*52QVTPppbKxkeWV4*P9hgFK%6u zcc@QYUvJi}8!lN(K=M$bhQ)quwCBTaGDmVP?xH7_%zZb1ua0=61INF=5OvRpi(T(l znY!J<^&;zz3(2X}1IFM-KtH?zc>K?hPY+M;j}L7Y*|)W4sSIkFOD}_cfn|xZ7Yr*O6Vr!60LXh`e$CB;@%o@6XFX0z%nlxcoH+aqhhW-CXCoi+Uv@ zcbMzbtJlU}N2Kl_zDs)Z@9KuV#UD07ca7nre**F4S7sG!-n;oj^j`I2sbAApefK_P zUM_^Jd(X9hhv!DdwEW$}{=GJJ=sK01yo}k()cDT&4+Vec7o<^54lbISODp`5K7tBq zhg)3r2|TY8i3;Uwwl}iV{tvJ2%JTkie|{X+*N9&&8zclP(K-|@>FZ;>5@R^)zDGi^ zSb6o!ce4%4Zz00Qh!VSBH)YA?ep?_Hz1A6%Wwxn03?P)l92u`{C(#FeHu*}L0nFtU za@K$xdfNw#VDEooxOTtqOl4Q$#J{|cl(gpQ4BJyX2&4m^sB_+I%N>Y@blRWW(nlYGi&*~n+F$az6%esPlmp4ADt`nyns_<`o$)0r3A#p1x0PU_N}cZ}=Ck{4JAA?K+*P-KU&%@yZ zuo2Z$_!u#I(f_z^Kq+fC(Bo|+PfR}|`}@Y~U-(W>?p?K8W+UAMZ(i9qA)iaqf_$WB z9WI7Q>8jA|DJ=FK8QOEm^cx4SKV{e*{DUK7Uoo=c7CR4>TxmXx|7H1Vt@n{L6f6oG zxsziWH>%u>;@*W$XN(qq-O>0W+k&CK;F7ptiP(pZuAH2G_})NUP(;4y=Octmy{vN0t* z(5|~NZNr$h=)9ZQpaNApAVeN>N@CX@%Q@mH=L%rtk!>|z#+o`)Rxdz?BSq%e;C86_ z_38V#;5;~zaS_IY`)C3S8c*5QJzLN#O+GmRFQ4ODO4IS&RZUHj@@@ckGz=Rs!=3sG zfFLMK@y2`t&R%h4x0~eped~7k<^J)s;hX3U3BZILHi{e$P$)i`@Z)Bg`gU+aE}Fgb zp!6m(MRz()R3gjxs3)@mXF5@ro8=|v{vz&cItdRdxQnGqOs(!JlMGJK?=zYdejMt? zh55i4@V!a3sZxgnkoY6m(GnEntD3Y4{P<)~r1(9>qmyc65?XK5bwW0vH9ma|)Cv9_h$?O4)C{{Yk&vT8<(0(ql%K;PEMwsbZAY^lB!$Pi45Xkd@_YMLLun1y7?x zj?mkhUx5D@Ci-Vu{wnlq!3Rc}cePY{ITaBinAUkR*p0x8S*LZ^`f}R;z;n`(MIB(W zZk=y4#7;ZSP<-Drj6tU6P^0*%xg^KpQ+Js^BiImL*ye6h1I-9gh&1}13A0^mjp5_8 zVlD$0|M5Ii#Y}`s*|nybdUm)?GxLKl1waA@HfL7T9kv;~&m0rW;1B#oPdalkF&#Zq zb~HCC>U@~_8@xYFT~TeO%S8JwyPan~^DR?9JH)d9|DU}FzZGVCgUea`NGcU$TBG@9 zA{WzHK^6hh1m=g>rMHazo9bQ6IX}~PLt^TeL(J!A4wH}^bOiIWJc0%3heGJ=nBW}{ z)Y-Qzm3TkDMl`eG-NoX|U*jSaem@_H<4vGf(-@JbOH+ng15VdI7?rC@-Luy?V^&uWmswe|_|T z%p`oybrzxPT6KUp*R5ht6Yv%}m?d}~H!ozeEx0(zlva8$uW#bCIp<&U| zSR8CEeglEH4A~mk@}h$4oYI`IP{V$(_`OafUqz;4pM>wDc>5GDp9?|Aet(Ei74G{) z((?KKU5*8B?r^SaBel~3m*6V*X!(L2x8XZu%Xg%ibOyz9`GJr-Pg?J7ZG|%>7G6rH zam(q&mxilkX|nf7iCEf(ZX7+ix^yKvbMn0e=|?1t$u3_UxAf3=HbZ{0m|nUAils+u z@$F-}jPSP`k7WvIU21b>Op)eP>B`TS?FD`Fi_Yvyr90=I?Thl?r@CO+h~+OwRSHF;HGjk?Sg_ z7I_-W+R68op^&2pRx3w=w{!A|plcn~dQ;eAl_=V&xVHWe2|mRED&1pQ)m1rvlS+E_ zh`O^ap$u0QEk+2k~3s3u^@CEm*ypwCkJx#TUDj22|D%2niaRs2Qu) z@B^~tLqA=!{(256yzyU=z72X`iwOFI#33OQyeAqS)-GC0$SGUP^B73bTWc+|Nu1SX z@&uG?bE;Z2j!a(LTdGp}EZNHt(%Mvi2=sKeMGL3D#cXAb;3z8kC%Dlui? zw+E8p)hp_+#)j#7Gg9o@UzAXAdu=%IZ1wWp>}9^`+uPE9-f(lTvQGQ!2%#I9z6wI5 zSD}n=&PHYv{bW#p(Jx+Lqm<$4_%gK7D|+uwPEQ)&-kJ#dp&< z)u5b%dDcJC_E>>H{O`?vk9PK|GqpFfj^p>ZoE|a_nqF>P%l=OrKrSHjbUxAtP6R8eZzH1MeMWc5A$L3}V8aodD-Nuk`}_TJes} z?d-o-yT0HMmcRkv-Dov7|i5fOl}m7VJcGdgLb{EgFH*zL{evQx_%GalO2Cy)Ie^28LA@!dh#IJwZ$nNzgF|}i#gkM97(wK#OJiBU7#x4=>$r=>z544ui;t9y4FvlkiPrF z9d6ccuw|3XTra^5_;dBQ?3TXV#tVxg)BJ;V`NL<0ja*npU(Svi=Dkl!0AVMEE3$~ z;^ljLa(nxU2w`$pxI#n>q^61Xb~)IO3`-K)YYk8vGxCifB3 ztXB-tHE5mi97p9YK-`JKz+#_dr|3k8y;tVoeMz51$oKvJ!*a#89ZH3Bfcv!b1IWsF zS#b;qy(sNMD)2jZ{KU5YIS&Fznuk|g$q*O*qd*2R4tkLN3h?AA{JR9D?|M*>Q7uXj z#$xY0*$0DC;uIh31B}dwwnpPQI0T4zkQ%7L>+%1|M|TkyLNYpxhU|l~>A*Y8Xt}k7N{3k{ zaQ|SD9rlZ?vK?XXLmuFMxG2dIq!sC0jEPzt6DFNc! z&gV9y$|jDC1mnr?ZojwJAL+?FM{%G|>%#6|;*TD={NUI*t{7kQ_KQBA8aUo9M!iEE zWj`7{Lq~J%qfu;%|2tYN$bQ`kYqq?ww}&6rj=m9DU;%=~&&09zJ=gCs0s>RU0iDK9 zd$-3zNtOPDyTXYt$6DtzA{ZvQeFAZU{T(9oN5#5^Gza?$`K=1A$sn|eth2a)>vnIh zx;oMCil=d@-$i&J6r32B`ibqU?s0H!n8_|A>poZGWP+wuL_Wo{W{mU{QNAtH(<`LT3B}-~s>z!P$aCFcbjL zUrJ>2N>pvF8d~vaFT+q<-M^o66wd5iIUIP(9v;t4GRp)6UM)>W8jH;7PXi$wy?$08 z4lLcf>-ty&y;H3a`55=1b*?h9$d188w50Z$d4_6QGByeCX|A6jNlhw3T=T_msM867 z3vEKeI9vikI{OMXv5!e`hyS|s0axtPAGQp{Iu{*$(Uv3*E)a^-e;3=bnomr{xhF{4 zB7(5(`b>3#k!vy8sR;1N*#M8{4Sgk}#|PpNg}gju2}wopiEx;^Ri|l~Q0B@7QZ_@= zat%_O<5%&a;>Ogt)591VNSz@YQ7}QQaM0Wb$T@X?q5hQl`{^DNH4M%kthzLaLo?fU zgrK{0k}l8-A7Gq!BA2FFb zVp!i3qwTGL&Fo*_S`@JJzsiH7JgJina`Kd$$;>$SN(%kS5B&U1~qem^g-pWmovY!i`%pF?0WyWUOUZyxL)hq`18y4nrfZBe&61{f6x0Z+xNKh^Y#9v z`5tcHQ-<$nm)lx7=0!5jwXd=E^DX00<3`fKR^yL18JIg|dzUO<+WSK5vhQ<^&)=5% zeIPwtH0Qf?bCE1(t#QzMUz+3k&UvEuW|HA3eWLG`_H~i`Yw2codzSKluXVcK|Ik{q z(r?N*(0Vi3=ux)bYtJ9L-fwodEUZ?v}``h1km=i1*vdV2m%pGUpFYq7VNedVwDd0To-yFY5ZrEKx6 z{qA)~A4`opYrbK7f2V!ybX}U?JeGZPPiM`2)bpeKbgDTNn~wXP-dW35*i3d$o8lM5 z_DHt)kbRnO<=h{#=Rz`lY7FteZu?r+xP11eH7}J@a_v{m`Kb3~tJK3qpBJ*1d@cPVZc#Z^aZjgItlHFS6&k>}A~?n(v)-zsNVv zl5x@e<)Gi!H#yU9)|yu~avo94W?VgJFXM_O=NauKe|st$+ER<5EPbvd?@;?z z{Y;sp)AZM==A#bpE5@vsjr8zUzR~xX_L=D2wVo}se&)`f7wvQ}Nhdc=Qe9KZQ`~1u zWxd?>sv^MAuMB6;n@YNiH2SUxQFcv>4%-~;>?C=}{AhUGyoahthGy5ly+>7XgpzTI z3^cFmoYtM`vphTXGSo9=)0FW=vQ0Jr9$&Z25c; zMti9z#;xQ_?_o5Rb{@5LH`4zkszRQk(1ofi$lG?CR&280)aN9sicxJhT612JpYOi3 zy&jwVWJ(aBP>1dC8Ad_)r0t6>$MxJ*Z==s3jFI;FSw2D6k#RGUe3CKuc4|6l@_Nlx z#dO+#t#?*of6q>}H$2shV7o=ww}s;N7= zQy4wwvt?B7(|70pjr1_np1!mW=x-ZFNckN*zm|+fYOl@C*mKZqaE{8;pfNg*S<^o# zX&cUQ#7@Q>9pqGFC(_47x~AHy8+P41X#P{UPOzrYIQ}QJ?Xl;I4cEm>6cQXpbuE*U zCFT7Qg~nHHol`hxrfqVr|Nci7RTWIB>A^M?ebx-P@vL1@Ilwx<@dA18m|R#WSO2AT zkKybHRF76RQQagMbS|As&vcxr6UC9Ai;c=PPjOdY>9KO2RTvML-5fSQz&F4CnH8+Z zV^qB){)o$cn3ZyU5wm7?5JnPa6TA*q7->(d(n-oiC7|cQFApWJd&8F~tR+U-IYT|9 zvwmlpV33zlJs`MEOcN}NeO(Nrvn%}yRggRa(*V;bedX9mR88JW56k8=vbz}zvNZaJ zjS}nYxWGSWQJ7^B#ip=Vaukd$wuC(#);QvpobLW)s`;iBZ)QW`1;$Yoeq6DgbH8Zc zFHtoC+|t;a^ux|LDxRFMylH*iKyfR3FPL0k&9_9>4)KCG$huxhC!CkS_ zrzo6+cfW))XO{(j(Gk0iVszy3&H`dz(von1pdxs;hu`G<&>lZqRLB~6C-Z>3#d7#g zXDOy3pK(63Y&v+8>{K(jbg?Qa>!!&B%Onp44Zs{Z<2I(ol}q^5sBT=I3%lh`mad!V z+%*q;4QCFY$y_QD-PvxV_~SuTG?S62JhLbUii%0~R7BuM?qA^8@exIy*)}>dIzD)R zB3{TiZHsS+J0&8`{*Y~m+EvAsQO%|Ha@X~LU%%RWpa~G$L%1g~UoW)iK8ET~FZCLQ zL?`8ARAJkn&X){HPt@WJ#?Ki?2#tBkfJPmhm+mTNjVj3vhq7tT#d*?D`zQZ!{)68S zqtF6qTBsQAJY`XzX7r#=9mJn<;`1HRKQ6 zewBPRiu<$DWm~(SW6>AOM)a4Qu^3_ZamWngfDe#2bl&8=Eac{D`V~e29VOVOrxok& z-#IU4-|rdkey+II9GRfqf-I1$@KKS+nGjna4^L&T-$6Pd|Hy*Pu(eAy*JU zT45D;J$z?os6HtzJo_%pXSzq@#BpEV*ZVzJGgzB#6h8^qdTAOVJHSs+RfMU?w)vfR zCQU{-+2`^ypYV>g#;&a=?{L3MhCoJe?xe|jskQHVI*`pDQCW#7`mA-0x4)wLTFx<) zp*aCe9ZH{_Gk|?_ueYps$&}6_^sn$DjrZ(lVa~wfC(;weVK_SBs23$OeVeOeac=8Bdjw>gb@)Y;jf`6}H8T%3HD ze+Rk3p|hUFsW$L2tEk*EmrtGk4_*WSO_b}(f-MwT8-1U8w-%KX-5c`u z=hw^K3%A#9-=M$nj;(3H#@DT%TTlHHJKop9?Xz1SWE!<6_P_Wnz?&%!(YvT0(VKZo z^e&hh_2scomCAUG(;Jyo@yu8F<@j~6nI0bbL@1odTLcC>P@U?(3Q5nJ6f%U2W4fE>%$N3EJ1iLjpD`YbA z@oT7u9E@-59`W7sUvNIa=Nj@X_Dt>u_AbOt@Vv0;eb#>)*gerKV7Md8 zEo}C%<80L@(Mq|`$c#iZuJp4a_RLLIqWc0_R88V{LB=}cgl8*D zQyH5+Qd=7^j8we++r-U7{?_i?^KY^GfzKw|0}eKEn9w3up#Q?R$>iJzY(4RxMN?ov zrUR#SuGMG{j|>~u^-1V9v15~LiS!$Ajsf3VJJw%Y>R^Na13WA37weOCI@AHb#JHti z>JUe)}P`Ck$S1Ay^m)23#&R9Kj`*35&jEw2ryGfhoALIqS1?9!CW zrl2KhX4#=pt$8e+C8&s8UK|OcE3>`REI{x8vF1^VLaP)jqu%%P{_^|_-zW9cGCS&& zcc}_jvjS$Iz+ffSK1>6+CV?@UQ}Cw|+AcldRZSL~B=d}EqzToL?^jDZ;$#b?P;K7C zW7A4OgT*fI4Z7R`Sz4uzziVt*x`X9>G1IA{VA`#>PprPP9rC~%OVz@V4&fX)qQ{{ z8KjV`i$rgFx@lOret1s5a0qVzDDXX$oOWz(Ruog)rm~tJecjS*v)C9}XiH)>R&E)> zvbjSCK`qHB1HiL!?gimW9{NAy`NN~1c^MghCJ}sF5EfcI*5AsK0iS= zK=hnI;PXefKc&v{-L|qJk@h2F7kl$BH{@3unh(S%$Da2hJ~z2Lzji7r--d>2R=yL& zMP>Ph7H-BvC6qkA@tK!~Vt-7Xt8_WXatHB|Io|E#4;!n~na>YqeHm4Y(Z2ssH6kUI zLk7;oENXmK%?*tf(B9|-=;mjp_G*oQ$}@C?LBCPH7LQ%OOMk|7%my1rSdE&`m>;*2 zEvE?tr9p_;q^&=()E14|tPRoThjb5{NPXe>-SDHcB0hs;^JaZ46oiEK2vdfzv7Tj3 z;LeFe2H-A(M9FqK<84at`H(!o9lo(dFVgj;p_Tw~40CN(AOhX25!WF`oBczw+h{S> zs`9#IDie9k`=8L^X}#St3TDCMu6!taDfwxGQE{*_?g1oSs2a@$?L?66KFo;_7qn8c#j zpv4()A&q3|cZz=d-@g%+QAz($>%fX#q$-zI>f^lwG4|71)`aL^&hRp)0;kKy2-JWD zUU0!YP>L>6;FV)Lm_9D;;7Vz6S|PvsV6P_i6PSDCq->YoT&LD`$a^@}nS@H!bc?ya zrKmmUNhEg??sGBFSF+-f9ZhlVnY`KVd|B?+FNh;!uA?501B+WC=Ti0C zf0`_*_<4AY2gOn2wr*+oz?la+8zW^`EBz|r6}GnYX+FCuu~&Wz!cXHW)T*}S5CJ5C zeS#hHb<#ub@;!#d7wjHZ8>h=1cCKy4>V5iRXp3;u`QMM;H(L3}kq5sWo9YnCnb!c| zzR)!{?bHLHLyO3JssD)faq8nW=V=O(BXG>KE25ZJQ{YOQ>b*{J8(!fq0i(cw>zP;H3s`jAf)iHStDU+y;V*IX2fM`D zyC{OTiqAD0Gw1s`v^p$wHL$5K_|&3(6ng86+?vWm6J?!{-%fG1&EiVl?iOQrLarR; z{*B4+1!@}0CJh-xbnf`IKZF=2&Wm*JICmTFW_3XWQ1Mr-yt;0lx+O%(M}b<|2ZxjsV2s zaE6VYs4cU?d1TWifF0XS`f@mZ?PmnDl-AKDPhUtUxB`{wb`+aOMNwooYi#Cb5oZoehkP%w&VtsJ0`uFdb%TH zMc@*Fc0F##0brXo{V&hb(A($ty<7Gha%(4KoaFi`lOn}CIaQ%#U>=&QEgh%S@r<;E zwnFIo6kK(|PwizU!OWqxgHKBzR_ENsweqLAU2#g>b+ zMO0lwg0F2gl4nez&hFiX0A=Esb2UmPWNc9uhvrY2FRo%rhi z`#P=9B_yy*Uk`kqRf3nbsZ7VDt`vq8nNxW!|7hm_dd-9e)j*Q ze!_V(a$w$*&zsFb;<-?8+i8N6Qt&D;RE`byIu`ziSPd&jd@B{v7yIILZzf7Fj7BhR z8rVwOVv@?H*$nKoTsHWIeLx|?x&Xk2=f4oHL>oS!m;-7nDQTY|os3t&!+!wC8;bt+ zlv6nFnVqeog_gKL1HI6`155rLc&?$CF`xGvCS=@u2;tsCw^`s96kWL;*(YYiuX8`o_i7{s4JANVgq__+k|_cQMxjcL`J1@%1CUQ)1`p%t zbwoy|ao)!x%pRU{RN%K>U$kM~6;Zw=FJBq#Yw3)J1 zNjK!AiJwf{0?*m2{zmCw>nFzqT}O{ztc|^EUD|dhrmhfE&6gy#`)7Fj5<%mL%<_pZ z*cL)o&IMkyvbY+-YA}UlIYrR16F1qFYj^1j9LOPWr$y)ozo51`7j6Z4Xon8L% z%L_%zQN@?jlEp568P~F87qoW0D+Rq(8%RaRUArcW6;ysJ^-scIiS{uF)>X7uW4(_3 znGH81Wpqt+U5wRmAo)-n`gYlS$3~aj`PD;sM1tJ|q>gX-oc&847&&!qTofX8=!A?E z)Ro{^^DcPd@CH&ij8-tZ{e_EXo8HS5Lje!MJB`lLF8|a|iSrU%`7>Qxl`D&n0D zgR!-!SAH&X{wyf~6Ntui|?2KuwW`O+W9IgO*?El3-qH8rLq#XKgWbQ(_@xQD^f z@P|?qlNGm-5`HG=HBvS0rxQS)>`5SkZzw*N&@@bW+dL^l`vKkps!04r3C*hABM9W+Z@lNKMQ-oBmuZAVoXaaW@B85}@I0NU%@i7yXwj;ps{ zU5RTTHS^ktARE2UYt+BO;Mc_Sr?plXScUWV<165Wp{(+19g0n{pU|6l7UbZIG^qqR z%NEsVIo1)DQgg@5%i?C9Ar40Hf}1;5H+S-1uqTews#RN*npZq(3BCeuzd0@JN*z@e zL5D2FaSv%x0_V$(oABiR&>g20+UH|nKFOK7dIYv>y0iS2TILR8k6_Ss>e@9x+TFPS zEHU69S^{cT_>^65V;BgFGfi$`ClB=K>GMwE!Na=vl0HvKJsS@p)RieS&;8yhv6ON8 zVPLM_>avh4YQDw4RaE{Wv9m*M!=(brJ28XfIP+{3rv>s)YgtysSL3f4e(q9!9E3~3 z#7|9^oV6D^=A9bTH6}DZlR4V z#o`L55%XL=isTdlwO%T2QJ2Zw{JAUH_>Y+TCA9_J^*tQTslwd`e63`&fG#unAB}lF z^-(Oglg)*D{!Eq*j3kzx4(pZsWo2I+e5JTwf*xrK-ei7q3O$u(bwMoSV-@(0Le3GK zPYs2}41UD>zDP#I70ZwgLI%na>BRpu)7^M+m83p{UiMoFM?mrNw?y5~MvHO48qB32j z#kT74o)gjb1B$69)%u1IflvBTX`OcW3c{t6F3TEdDIA+*@&c!O;`oM7rwg~G8RpdQ zgjCQ{1+x60_WqlibTOneM^Zq$iG1Fm9_=>T1gwQ=UEsKC&2|Z-V{bxc%%lqc$ZxgB z0wQM|1RcPc76ZR+f~0<0$0n5*0^t~3GLb3x^CkB9U=G|B<+7iz*|SgB$m-#kSQeG^iVM^aZu1!SuyD%1olQ-{eFBWI{r`RhmG|U-s|=UKhxN?R=p7x{q(Y;Fy=Mf zwB(oFZRs!;WNV8w?{Cyx@1;+(&mSf%B!T)U;30<~L&x#jbt>@#y-yqiYz~w{BaIA!^74urso=vBLZ!Jp{eMpiJDZ)R0P@5Ji0S%g)Om$-6Xsj zH7B;DuG`6(z(D`eqVDv9I)n6mvZz^_7wC|+JTv_w@rT0`Z~tvd^fz(m!SXUhTO z5nr9$dlm0PE^ST@?$IB}-kJ8^Pi1_)Wl$YY&@X&&cXxMpcXvOyyB*vef(N(Y5Zv80 z5F`-XA-KD{z5Ji|%Y8oFTU#|-JKeS0D?Ky4-M?;%x(`Exw0w`es18Boo;%OlwHiVP z0{cjbpHtc&L;g>Tn&B{0-(`%R3zLS*d=vH;)LBE*gS&b_Pe70jKOV<@cIN6u`t|~e zU5B8qKAEEvUDjWIb)7)Fn^nHsc+b<@Jc6x)i8-KEu>rG_AKZ7TOI|X^I)Bo?-n=BE z$g7E}*X!^JhYvsY#L@zVOhNQd*UXFD^^7<|Dfp!&PeNnu2NOsfgqjDs8f556=Y14I zHfG79$fG9LTBYCpR~~)yQ6_6qvaU_O+}QWl1qx{+?oyUDPO$aqsR4c9dpid!CArV_ z9+|7g2q5+V5MDy~2aYBHQ1v-=-fiMoWi_5%{$ z+<)$VKmgz|0EU^HG4$(bLmW|2XqkxQ`hqmHE!yOX}qonL#ge23QsF{;Z zy)S6AQk&zY;b#?a;p4FG>6AaJFA((j`WD#Z{`mIpop&X8@O}}PdodOJ+OoI49ccJ` z5$DAGczFEYyZt)z-n+A=@hW0_rz>Fg{`fAuyQcN%Pvq$*vmL_Vc^wkycXY<`VN5);Vm+-MzaJX z*h6-T;QH|7^exaT_aEJ0PS4{s|8?)ud44M{rqSW_x;uN>#`Npd#pAj0=j&t8#|O`P zkZDQk6;?&7?2}=2YuF%J*4Z@^b}A>zgb1%Z=*9mNTxNDSQ)p@WpzAIRX*K6(2*y%n z5#7YT*OJvbOwjWOpbmz)`Q<96d*dk(B^T@ldD}YVzTU*hoKt9k<10eYTZbIO|8Nzp zkH&im^LekL38Zx6HZ-PjRB-#xK6K=J(kVgUbz<{Fc=^hfCCr{~!%}(x^zwN`Swexp zw-Fw4YorO&nM3PXyA|%|eD^zzS_VO;4hHkB8M!)Ra;+p`MzyZE7+^O{tL^ ziJ!BXmTGT@sz(m(B-8tON6XvVDcIo^*~IqN6Kyv>6Feh3Fn)In62`_yr$CD{p8WCq zQ)=_E!2WoP>G_57W77Rk@eAThrz$q=GkzpQ|3*7?z4vZ@0Pj(>mBzn~t&#p5At#|L5BxW|N&jR+*oi?l2 z$k&qXbu89Xg*F?yZZ^sFu)-2jtyOob6 zt78uhB5x+vg3f|$uX8-9NQ%#uN3X$p0oArO6|8BzeRfW`;_$9}6S&k51Dl+V6_^{TWZNNmvwA-Aef@W9RuPrbOmdRSR z5R!u`gL*YC)P<5ar)JnTX-W2byPSz9-kK$0^v^h>(~JJVRB9Z018FZy_;o@0!3zx?}rlO(~Vd&!T-5B@wd72FlyB|}XGpEIet|N|?PkDGim83wM$E;M=0~R=z)49!YJIbD-S?8(Jdv1% zSy1_2q-DpRb?#1ksj}b09!bAIu4kD(1^RhUhqD+uHp}zv{Bh{BbhLVT?X3Z*mY~u_ z4hO|Dpv!vR?%1iETfji!M_>Eeh$UFzn_EahSsFvX*9BX|HeLwdF1_gURpp>rkrP4N znJl{4B?^0dQ!rs%uaLJepY5$?+^A%L?-OWHwG@G@(BKh!D$3p1%#G+` zC3%BbKW^}Ov|L=6BR)cM#_PuCK;SIHS2LYh%pRHKgkwL!_$k)`m_&>9dsI$nhx{K& z*OY$<$}=^ezReQtKJJBQPLp3X2EBHM*MQUB?oQcH_50!cnaUP^xk{abbex?>y*D*C zfQL2pL zCdng&>0972W=ijgl^ErNR3VN>VH@kqxR>^u8S3O??RCPB_K(@Q67PK`J*>D)7DYH0To$#`&{6#YjD|>XmG|1W1#!faSslv!w5M%=I9M?@p zQtOujLW+WkKO8gavb6D~K2r?%J+I+U0tZ+m|_Ht;;Uj>)X%MKyo3fEERuc zf%e7nC2qr29dc`CX{dQA2~Ad~5$7yW9n!HHR7fg$fT;XYb@TTh^)c(ID#Yd&Hci;iC?jRE^4DOk-WsfmD-<_E$&0beG!o__?K zpx26#Sv|63xm_8dz*PwwO*b~J+Q-Ithm($U>c3e4m4SuvFALeBqGQxL)^ZAQ9wxuY z^3JV2L#eOml#Siuz{GgdM5WNKP^Qx4rjlOW)pAu?xacYMOgypjY)mS}jNv4-?0yeo z!e_pOkLDA($YqLDEUDwtOaF3ZfJsn<1AM_ir3vxaU*EN#Gw-JPrb1O)2&w-48!P8H zKhLIz@(-EpHC4y;23e;p4~^3syjPo6&|Mos5#^G}bkR-uNvvC>GNLC_#F_me_F(da zo6~;ClJRHUBG2O33W~~MxB63=@4nZyl@wo2$5JVKSE#LB0FsLQnTO?zcqzQ8`}3578S{~7RoaApMvGaf@*1z4!dr2QXTB%Ix84krq`)| zNL<6t_YE%wwS=^o7Tit{D=lEw>PYe&y-!Mg!l_%GmhJk3PPhVj2;f;x38rw%sR!~F z6R6vAOkiaQ@U|yf%SwxV>v}z7Ny@!&)aSlsoE%`}up6ryfqVHv5Xso6-;9P(i;DvK%2I)vE6)#9sJ^G{MQ_@Q3Q4c6QRF^qc0>^D=fP=$D z2NKe4;z(RgOhuOFN)?exZDST#e{{5$yf6NwUw?lRV?I6|w<&YE@7CZ5p=gCBTPqXQ z>PXnB%EP7){DYt(tfTzXHO@k7$}@(0GqLy2)nw4qna0I5qd69mErSYEQLZz+^5@z1 zLRr$%-`Dc;-MyG8Z9}lDMk+1(;)QovrbQ^F_##Ki9}9oH1;BiYelIAViLDB zYwny=uEF{5Qc*e>F3RG$`td_f-x9Uc4IIa*W}ioQ?igJ@wf zV}RG*nQfshXf3r{vsx8ZodVZv?PT(0z_aQ)UG5z4%xQ>YIBGM%7hM3IG%prTHXjvL z@k6WbWVMA9ST2{}n{<*N##f|b)F44O@EO0_uD%0jd< zJH(_bgoZjX76o(CUM!oR>~0||U+`OvSfmc~O0Me3K9W&m=$o!`#Okl&0=EgNT4k9w z*S_?vfWi9cKaW?*KQL$_Scp{@P9;h{$>E+TK=X~a+kcF>1 zImFXYuY`P>ZKfBkR(h14QQm^+hGVbqIsMY7X;7pk>#*)zNon$u#8XjD6k2HwA_$)Y zmT1DB#|@rpnE39m=SAz8l)PV+KWgb0s0zmth_~mW*+!aX(Ve4Dw(V7UfVup!t;(IA zW7jUK>glK>QO0kaelQDSdV5%lYKeUjw$D~uX|s$rBlh`GzKnauomZ{R)RUdCII0pyf%y8pX|=9e3K2q z@0VaoYj7RcTm-6S!f-H;3z&lKBSruEZBh?wFBc=#GmS{Tl&*J9Yf={13@~5&R9dVJ zJaRvj8UmnskrzGHejd=&1%N6GrYSbk+dD16dPr=>z%-dB({~C#_?ee1#`rAh@N>21nxK5_?iF6~G+*WyDk@uwPwDL78+GYh?j-w}=gNzKV>r9X zA?;t$`!bAf;)Z{W#5?A})+bV#3%&HQwQmx%Dl2Td$_h_49K%t17zr)`zy8k~9+9OeeoCV^BY4Y9Z4&y@;bi zszzlr(62dw$40+`|6y)5=7GH1lKM!+H3obeJl6BERA{e~NpdY*e|0mptKb3#MV zIGwMYYF}o#bc9ltHM1_9l@?fTX?k(&y?Ub6nRn079gyc_s$3apY*o=B*nzZTLlK<2 zz5*Tm!?liT@)AP&h0-*=Y2m3r%Bc2PhQZzdc+;HFXqkB_B6$wIF1`XR>Lc8R|1q6B-^yO0v}$YW9ZSD7Fs&_8_~1{gRcc4%@KKD}@w2w221TYa zVtSLFGo-8~=Ay#kA_yg%g=gsCs@<6a1mHTm0)%Hu0Fmwa^n!B)C$?!;~-1lfMBHdk}Km4DQm?*x~09>g%Q4U>Xq*RQpWL-+bRth6&WXZ`-on0C&o?jFEZjZK$=y4gdTe(}T!qqY+N z^=e+~qi0k24n3a}1K2cwsE+v{a^Nwa|Eu)O&!2ZD^-3w{L%Of2(-&9V!u3Kjv;={^ z_|u`MoIHh~+oKuju5?3CBv(eW(i#(ozFq9x#|w^AQORHpk;Hq-R;IddsRRA9=qL zqIKy)&WSoM-A{{}Ol^W(V0s7erSV?){7`3HZq4ciXRt}C-LXs+A5|Sa+Y_bvz0F}t z;+<$ye*59FkocGu&zYZc#rIf~XQ0;PD4>o(ZmINQKBfr?how&{F2O*2>A|nP<{b1Y z1K%=hw#3=p^zL?|53Cedmu}UHDx7RQesq78*#u22p*2{-c$CP?2WtvJvy*RU!p1N2X*oCN9@V+y>u>ba!OjjgiyRXwA;8yn8;8S^dl;CJ%UzOv`0rI&Za z&|5DhXJx>(t|M|IS^0j>fX8Aow%%JYEQ!m5noGd}w8<_pdRcBUYg59ym~ywjT6FEi$u3;zr!VsFr2BgLTwx7NHjIbb z`08o`0K*>z>n2Aefeblw;t86j?Qu`#WzDBJs}a@dD?KGMHgKJ*Ez^0g9-Jjga>qkz z@9G_eq2yQJbf@Z68^Bhy1QP9b-XrEZ2+chNM1$r)MN_Ba(T}C>_ zCM*u`V;OX|tTNx>oD!qr1qts-xE?qhZQWJ+W~GU*y#;Lh@@DL&s~+|}}fT3nn@i>=V=un`HX zBG3{)(~PUfqOq}+jvZHxvZqnh zSD%jgd*|4;pnLuM63Z^T9!PxCjNE=sS+k~{1wT!J2W+Z_nmA?^$(4E3FwRHF2-{oh6HqP3{Beh^5CXi& zn7#hdxe3K@EsQr|(v}Q3+E9-5G;EDCJRb+aL;9ijU_*3iA=$|E`Vv}9{<}-+LNY(r z5Cz2PJj7%yHtz?V$GUZ5IoO?~pZ2!cpLH1e*3Raga;i*2HQMxquRFEgoM$DYs6p*}&(&!MGwyYD_Z8kGI-s!s+Rc+8h7;i~rD-ZTWH? zBvXDX>Q`B<5&y9bhqYWr$1J-eM65{}73k}_N+iqehqsHv@vXW7^o_x&t1A&Fk~NcM zqM85nCtD*ycZ583Rd8@Q22uZzpL=+d$I zBCNyEzsucq^r!(UZHuM>&M-?S`g@MtNuq2RmNY%bO(T!(YpmJQQK{iAJ-HF7cxqYv z1Z>+@C!s1#u7fmxx&P(<fkLb}mKV2%*3jaMaG4)X?jE8uO`$!(XjYCEJO`U z*@G7x7+OH~($VktQujS7XA?g*#v)~oDc1|F^KZ;R8V08XF!?2=7g$NIZg2OjlYAdH z2!onuWOTLp#^J&6Y*B77;g6P;7Y+Zbz!-Vgn3A0E z#f{^;4`dT4sUKP`K{={C5E92&$zoZcJ=i3hNHH4O*xsykJzqdFqhGzmZ*G~G5WxcM z-o0Qo{X9Si58W8Henq1ISj zq~rg#X4Bi;=O^uOSYiyAuNG1qL9U5v?H`j%Gj#Ib9F6KdRZMp$$=zK&bc-SLt?Ix~ zOM#{+7O_Xt6;jrTH>E0K*QKF4Jl@OwugX&5b8I@jU;x87HswK4RH^e8FCk(As!-JT zMDWBY^!_M%F7&c3L0GtAQMr`72nlD%RDDv}6mr8Xhu*R%TAyUx$EAP$%*au1z;SySBIkl}05*VCFZLK6m1xSkf#uykj5g8WxtsO zs5Vq~wsx=h%zn0;{`CIZJrT24ICzC;7L_}y#Tk1YQ8w&8D4+WO>p5xW{m2J_p#T8z zQ0Ne_S7@wX-(T{UnRPIL%qSiuN1TxG9-Iq(37Z`p!2cg<|9?ni@iIB$-GnTkZ19S& zQ?zUW>*mq`*)O{K!T-?V;#JA=_cYM7tEo z0QLWqN#;K$QqRGfTNubphISFWNfp;r1OK4SvQv1Mc!2nS;%)C(1cQ;ee2FJ7Zg2(- zZ6zl+ydi#;mL`3XdC?0H{NhPI{GVGui{}ZIe=~4^hNd>cP7|+A(`8nE2H<}&pzrON z1w)WoeM$Adk7aTI)c?`|Lc{}1t0+cL7#Y^0RX5k zMP9nP#M-?jz7krRJf3CL5IP0tJAs;Je4pm~^%n{3Z4P1J?3QfhgZT6wmvMlO^n-Bx zWDc8XoqK+4cHok-|6y?Vop%y>S$Go*k2aC{(lYJ+&;b@z$u8G{P%`1i-|3Nq6s z;|vAd`{35w>m<%7;_Ct&<<9NUCY(WtWq)OEcTzF=ghXi ztkjPWr$m6zvZJK{ZM^TX$Ann3_6CYx`Fa|Kgk+=79Dr@qoyH2{8A5UH3AaLy9WGuO zy^L+Y&@-;{4 zo(YB_}aLuff4=jYkrz=n2hyd7{>Nl|h zCa72fi&avixw^twkPDCfgM&!mG<8?Ackpk@9SsRXy@N#`(Jtda{8UoG)4CSBpfVHK zVe7M}LB|o-8!MaW7I{R5SG%dQ-e)7~A<4v;EAh8M=95;QzoZt)q{(rjd}3!bf2hJK zbTUp#XpZbsr@FVUhX7Kwn!4SXM5;(&&gpxOfIki8N}Mm!aF=r#3&`%HqJ_)_2gE$` zckSLem{0A`Kjl;lNB9B)t(?PyHXb14>O{jhazCbwo7DCf+k789g;ZukvoHRpNuOH4 z$Ey*N$Y*WQBpR3s*5R&i-{q;QYdP)k+yIDS!*Fl-T^=#oea+&UC2lTlBslS0UfFI4 zgQ?7M766LSv&%Nb*gNV7S0UQI_3#WTxt`8fNu>@+L-jrF_C2NV5*z0L485`MG>qyE*HDbBVpKts~p_UabT>4q&5n&4l) zTXgQ8Y)HUp`PxaXlueU1)kJrlQJ4*wXm^UZIXrZv*G=z1zhPk%4addI7A_71sxrGKrv1ffn9XPow!Q|+XH;Wa3z>jOxd=Pb<2 zhxI){hy7gm#@{2Jd8+9|R$o`vlpA$rUVgG?!rBRn5WWmndk#r!JvSaDbbUJAjq_ue z1e4-4s^xM78?_O6O&0^w78C`i-2ep>NWVZT@6WW4KexM#@?r!}-alJDCkYJst(o(t zzVSogc3vvfT{-?QhDaOXWobl}bVVZe5<0oXI`LJlN0465*e1hp0@Zyt0^d$9i2Ijt~2gAC*>#GVzqX5ULO&OJc zW~xPYgVYop9EFi_27k?1A_<0i*-$!=1#eW(cNn+mlcdsPSBdfFpG_5N9QKggEKJH@ zD8ma~bv(wr_KuA7jYN*$BIielGnraX&01l~4{vi2?sBCbPbH^_FV-78Uu1W4drXa< z{5dz&X{h5sd!uppmZ|7F?+GW7zssZ4hZV~^*$}>M{xc4ZC~Z`Oc(h)h2_ihdAy)+Y zQjP4@OE@KJ`DPT7>SRroo*q%?XPaQrt2KO-R2JPi>YHwrTaEY5Hsc>FsyPK;3!=Re zrn*~SwLz+1@4%2#gpkdGB5zW8MViCHL_{f(SE_LfnoK9X<|v^s>2Kp3#b|~Ced{Vd zFh{7p#{b}P2v37uy&;#!aY8p+To8$OJ3`V~QB@lBLQiub#XC?PxTR(e) z+k4_N3f?MX%$oxa(tanl_i08p&Nb69u7L)NqqF?bK~Cs z71Y;s78_f!kvJEEF`j~&HDmm&1&m}k{*1&3kZpYa9(+G8cqm3^uvP#di`=$M6npP~ z$y42rZyiRDu9v)dyjP)zn$`l#Ob=G6G5gjgS5*w@+J4lzV>^gUyzo~p*HQcRmy03J zhnO)$!|*q-8IuL0XMfxu@?^iD6kCz3MUDmokhT=UPUd}}UT&;~&jk(~*3SHUjzXfV z$+y4>v$J#|K5NTP2W&_CNRBD_N(m*28O>vcc)8(2stG|&8B2f6c!OkrO}0mh%n}D` z2EhxkyE>HP0V!SaG&IhUg5g5JD;72tmi>F85h)LM=<|K%j2huv+8w7~oKNp10 z5wnvdHJ+ZzB~~y!F{kx&#Y*M&CDVI3zy?~`_#ztCvDZ^tWHa@vQ|a1>Ci5aeF8m5SocS8UZ; zhQSx0eu~@rAgM88{i0DKa2}+OKu0h)X2LD`C)YUhH`dP)tI zwBX9PJ1t2G9%4ClLZ=hPVTX#H{NM{iSpudJpG$Alcc^~DKZWhXkf(C~s9;rm*Q^Vd zLg{cZs?UNp#EHB8{hL~gNCXO{#Tj>q*)dbY>ppR;0ePTQ!f&EHtXE?$EoP}J$u-4F z@*eAIs24kzJq02X6>AAQUSCpeb!XBLm<4}|j0G3a#Tk?1jacAu_ml>S2D?5asj%OE zRwr`efss=R5`u{z{2ap%;$qW#kSYd&{~Gw>OgAN}RU5gl*0TX@n>M-xoSY6PWr^de zy#s4)RDpJ1M~zzGCnc%jz__VqsuoWuDXZUldnN@J*@@eHZN6iZmlML4SD1)|N}Cy= zY{h5eT0%2Cs9S%AbS&t4XB|RwMZ`&@hny;JGX#YQ#*H}X#EL-qZHikkbXV`!&=Sg( zhk>4S@3sF7SfM)G!X<^DYBPX1=E-Xq5JluX)9k}xlMF2B+oufB>4h2mT*I&TJdxNb^Gf2e?wR0BDBiRfKTK09u1fuQ zbIDEzRA~^T#mG$G&jh#gV+Txx^=Br9%(@L;>@d0k?l1G=lUCvP01gEzt6Gi=KjO#f34-p>x z=Psh_+fSHaVi9dUlxlzq=bDGb$BXLN-y?y>?+$Fu7u|v^!v2(EF?DjxvBh|}ehni3 za1fcpmAT-Skc}9mK+k4As&n?d% z!F8ffEal#m*a1NYMdxy|V228fB&AeJB@AX>Nw~yZlhANWVqIbxcZgaK?6Sf|Sd-?R z<&?54n1}*b^JLz<7fL28?stX-!7ieds6UC3{;a_cv$pY?eb}MUEMwuZmZ>1N$*V|} z!(nnP;R(qWL*NI*55FoQwq$6*w52&3R|)7&DMDfv;5Z z;{shc2wgNOW78F?e(rD#E#qNqIlEv}+e={wEy07U8FdHxE+3-Vh}mMnH8)6+$|yJ= zj{TD3Ad-M!%Ym~%53f|5ooRypv8N99rhw=H|3RaZKzSx?x0D1;UW||S9j3$aiYZW= zzOU5iW@H?Z!8(L0?u>(U@f63qn5>V8?t74(y7ZR+;g}y@4~2u(;M@n*_11&}>9Zr? zHVVvV0$+^bo$y}a88Xf!e-G&GXkulx$wVlhvY{r`=rO05U7M1~JQ z3;4!59`{C6-WiM5Wz=lsH!)zk0FBG!4LM~&C6TDMw1jHJQj-wyr#U4^@s@~Gx}Qg& zh%bulRjvjmY#@)6ZN43*2Tuywm*UpPNm|v8He8u!md;+~I5m-JFgj6;p9>9LN1@35 zT_lzKg{R+|^Jghx5d;#7qZ0)|JkXPa)3<~O)=SlrpF>`+ysDN}(WTab^(Q}=fgL=6Jo%W}~ zof(=6JKHvt;%_|g=w)T^2q-1q00v=PPY{fd0v4S=wkRf$it|wzt<8y<4^jpt1n~g0 z1GlMG)VnY4+x&%}FA%WdcN>=CaZNz~Ji8CTWhcER=GSr#P+s<13WGt@?S#1L5E@ewt;ny~eR@gmv0t+H337XUaKJOMRAk4-vQ$4TnI z8NI@%*?pcOIsnEeV1Oy*ELVmxRN*9nW)Xf2QrX$mZEUbEHp#T5Xh>?bU3wDq_X3dkJ6dtc&ye8 zG6tkZeI&@OagMfAgZwZiV{$i{XS9upLDox_t$L&=Y}W3-Q$e~mdk$u5K)>KO4(^#b z4vVYvY*WayQZhjFJIud{ISv{Pgn-inX)HMULmz#7PsdCra9_EXayM#;ANJoiz--OR zZ+%Yej@^>-a}2a=rO^m5ym>Dn5KOhUM!i1KQzA}an$GmmXHpcA8@GK;gGZ!vIgST$ z#Rz=Z6=6h)j$!Cs*+<8CTO6ZB4+3L!Rj!cqFBOwvnf0>l$lnWaZoeJr+A0_axk~5@ z6N)doZIg1+R5S0()j)CUHU4^|YUVCC=2=@c;uS#?BOcpD0N$pMTvGLsRzu<8LFeO&XTYACH?6?ZDnEpgOJ>5-ypx*- zh4-&Z*U$)p&nE~AQ_q^)iTj)}IChQlSieEQ$2;TbxZY0_f*z7AB(K2?QV~QXz2#~N z!SwUJky*f~SBu}Z-@g9H+%n%-?(KyTSGwA^+c;~VQr{_hFfk5RJIi_zb0gFQw}hA}Hy>)ZH}@oc z@d`QuG?caFciD^>*EAYmYeA0vq#S{ph_w~pTo28raE>?;p_>94OcwKrA@yzJ`ta|4#qxWT?^ zv>OsI3KtB2>#H2k%)e8A7aUQekL%I9RT73}qvOF8fYRdL04ho!U9tqI{?!GG2X9qQ z9ip)-&$M>^Dz=D`GOO+fSDNH1-+-++K(@@8WZJ3!SKVI3^?S1i*A=XRYZ}$-H=u(iS&F_pMNn@p7)5f1S=O#O03+CN-~R5IoMnY!X9w8AN=?KzfgiW{wTTAV_ zHO!o$&WJLPI_T8Rb&`<+Ok?~Ew0Gz)ev1Z9(yFZY6=1V2cgS*YFjoPMS(WR_%Jk?U z8+j&ek`nq;%Dm7B)3SU@4cX^Phh~XRvoKaFFyyZRnMp*hz`zeohGJJ_;vAMws)O-T zkeL|@4vJ=`xI6?Y{8n{69^U7%3~oEC7*+iw-V@SRz_4iuT-3XTV&}T2m|E&IJJl9YI3?>n(s4R#pqL6i2nA1BP@au&xsnS1j67p>Cfc= zorZJ}V-2WlM%nLBH>c!b)ei#wugwvBB`KS6BgvPr*w(L7@_k%Fd) zUL&_Tx|7c(`*MFdFq&nv$l>mZ2Uq>`DP>$zfGWwhQ%qmx1MSpuT9wa|2)_F$LWAdHitYb3&gO%ewP!sOB}4@EO3><;0e+oowuM8yfRe<>&uP+ zc&U?lN$d7|OtW9(3vnHYfJM!~-!%Ot7MoJgscndp8HuL0(iZiE?lk*KT+WmssApop zV!~K!zGPg)bOmh*i@Vi>Ouvdl1jRP240fp{N?%xFyH^|T5j4G^V9xG}=~iOP-hjX4 zYU5mtc&^VwMMd=>@G|xZse|2N)AZNE(ZAOnMYtuK5#y$m8w%qP0-J?cja|or=}9q* zx#nLRN;XM^5`s~MJtlpJp_6eQ#JGovF?eB~!HE!1>6Givz!4+cJT7G#!gSD!T9(cp z?S&_rk=Y2!E9^eS*49Z|FZTIa?o@|g=5;FxC-=vRuGo@2}sjyPWYeA|es_ z4jpS(asyITMwJ$owwr}NX@Np~g^`=!#-wBcK?E};YBo>Gz>s!T0E#^?E0sT|PCr{` zgO9_h>>t74UjZyf#R;-*DebVXjM1vF8Zjuyzo02vu%!6X3o7x9OiIigSQY^TT%$qd zzuuX!Y5EuyaVD5%e`d+v4T@3;6l`di_KouDkN2Rx^k{k9{}JOyf6YMX&)nd@ z&X;H4r1(qcR`X_atnu)lcCySP*C6^Fdf=Tr4ag~4;L)WYR2fdg-(vby#?fx3+Xyot zD3k7kBdRN#zwqkPo4A^_Z7W{TN5G-TQzH5ERrW~t(F-myjlAE=ZrSK-%bn#*S zS|h92cOr#SNb$y!5#!D^9MN2z+@dZhXyfkgywPLB(D4Ol%*Pb@WktijuMfY4enkCg z73gSf=Zv@9)nJwEv1?Re63?3%CMrgroV56H-(YJZgt3YzJHoXYu4Ex-lK>V7*A59T3ZMl5O*XI&|A+JRf2 zqR`2)Z&FxUHUP=`FhvMTc$EaT5wG;=$qm5`Nl94UBQq6EE0Up%iua8u1K#X$;)0op zTP2(c`2cRYn4OY|trEsCd^)1iUN^$pmS#xGxes7!m{(+%2QKPz#jOZ}km0-!85(~7 zpiQ{k#$o)5ef+R`$GnN^&+oXTlk4Cd9KmH4c0)N9!WB`K-fN_ZE%Qj!c2Q6wN4(t+ zc2}V@v)-VN2>Bo?cl`w3_4mpDdiVKM$jqsx#C#{EmP9a22%8mN_M6wBs{G?hFZvyY z=^`5=Nl}5eM757{iR_Tgm5|fcvHk^$G&h>VjNuKEqCcLKguc(M3&h1;rFI^8TWud8 zZux;;`rM0eb<|);(=0BBTO{gbUi}PEGzw4{q^;)og_VUaA(>|X?h>XL*X#v0DWii- z+J>TaO0C1c!j1%vSp>s?H9rKc!+R=XfCf?u>!Xk!0nEA0P#5Ack}XhVR17K`j#wUEB-wy)pT?yKwwov~hAlMyU4w-A z-BLZk;LBVhq%MM7^Yv9LOAbBMW)WCx8Bp<;2PT{oVzDGl?=U%I#O>MOun?I%qA_)F z8-{$F|B}iOw{ZNovZYoj&h;z%8-`CT7N>6D+HI9BvS@&p@`VrCn1?gX4#4RPE40=e z;Kh$&jgy2HrZB0&D8{>RrGX(Ubv1lz0~k+{O?uRnu`{GRW7+UGa^sSXIq1I*T#=Au z8(EfoZFrYg%Jb4YQLtbuhRV)w+U5_IZuR=MkXY9Qqi- z8K@e!+W0EWN}4}USh0l&F=1Tzhr56K_RWy_D&Gob?97jcd*5L^`v_N80?e9lp@qA< zJzf7=Gu}eSTM}YcRutpLx0iDVqt95Z?621-?YSXW_$|+H?6A0fd$~&xG!pbp12Wc9HH>Nqh#&cpF-!D*SAZRN=qJzb9e?&4g73`L3_ma% z5+XHG10>P8dYqMluI`M}bGn4w3$h&0K{7%6J`RCxCQ#xj&l_~i39bf*q(`htozjN> z8813*wP8MmEeASWdg9>pVu5nGF%@LU1bjo_MDB!6~;Od;)sDJw#CyqeJy>I(}-B zlRvDJ>J{RBKbn2|2QK9l&{z;8YJXVeRlm>??4h7bHe)$+2$37Ui0xAyHKp+r)TR>b zyU@s8ADdebS-t=rW5Of}B?4h?;j3I8UZBWYo)Kjxdbw3$dVEYqYg;{O@Fke1&)SeA z_ne(r@3yF5tc!V&;z{+ctSP3!<=9`$Fuw--`1U{xv@w{R| zH_icE)Pkmx-;DfOxcLxQQ7GYjpjr6wd5Q?%(mgm=v5hDugk@2-YUI?i$Tw7t_eXr| zl#A)mKZx;-ZaGB=^tU)J7p=YA!CibVdXot|Ut#@jVGj+an`PnXiGMjJ%K8!S&(UJ0 zdK|k&X)7o%R4~dcBSTQY_0t&l^Tj5^7qaqh#2_ItQleFwK!NilIkiGJd>D5@#P>8gX)>Gr$si`h?q$2pai%nlJLS z31EVw`lrRJ{imfQU1$h;(TCzHm=OFXMw3L?der@Jc8q8!cpNV@3}W(+>E0(H<{%&)j2c-R-m65^Cro0`=41`+`i&634M3bCw~WuEvr28dmox zWfOdp7iF-eL~5lIr0_otV%c|t#WA=O$`E8?CB{7Uq7F5}w{Ic>8k5OT@wJI9jonvz)T+ps#b7I@JZBHf>+qP|E$Czl6iEU@cwrz9A z#{0Z=s!rAUc2-r_`UBmyYIWcD)v(6sPQgh=uT=gH6CV6_OWfo5yw#j0)SPbuIUOGM zzZ?3=i$+yl=>Gc~%~GH_RqiYz1_zxw>h9i{!jHLY-#!#VC+11WC~ONUi@yOm0_oa> z=ol+l6RJ(=F4%b7aDf42?dGWP-XK<~4kt-ba-3?k?Wb8}6e2CiC?g4uqgKD02H z{CFQb_VX4Ut(iV8#|jDg2$i{)B6*n)08^IN$uQQ>eXW8cU?h=o@aPkX7`KP6@rqq+ zrV-n{#rdaTgxs_V)}XhQ6B(FmN9vPFL<71!*vNoyf$;*5+0SvkTkH>RndA+6Xclj` zK}lwUPl4Q(|LMoDX``fqxcjhaX~GGB7^fW7>tXoDW^+x&3B)z?7==yb<8qA_(Lo;ngwQNbOGHC1B0`UW&{=i%WRvdV#eZeuI1c_j z#YUqUVk2;E7a0F7PrEy@E&{-!B8CsU7uQ<6!JSuawkt!aw)|C0>M{Q|rvE;~iI*n% z#RJ>z<^q-ZWUp@3CZ`_mX7XyGq}iCuro`*M4m0@j!j2WD$rx!nOo;z8L{#8=0 zowxPB&gVC*x{Q9Bi_^H1Fqo~yq^2xtT8Rlo4MZeIkRG?pyLGf5}b4AjDtv&_eflR^^k78)3a!{p2XD#^WGSmPWY5@g_Q!@t^?%`}%%{{{DXd zFFyZ^ufT7=?|7R32L02UDkzAH?Tu#bi>WuptqyZ54Ltq*D~}|eS16#eDhz6Wuc`a7 zB_iyc(R1>Nx#>@ZB}i0$8Z8f)Q-B}+y&rGW7NMBa7fS?wC_!F;3vd&tN+gke2EnuPmfLF?# z>;Xi?qZdkwRLQxjrya2Oy|w~$d!{?D9v`GEv@Wqny5=ohl_7o+V-hs zMMKWb2JM5l_1>FeDmR2cEosO07gJiR`gXZ&2tB_Doyy^c##wmvt=*lsTWp21#zg-B ziA1GNwML`s%njnhQmR|{ZaB(kW~Ho^V6T7VFIW@1RXv4(!0z0h#7}s$V{ncC>2 zw<-(~-#MaZOUCqV$Im4j0)c?Knbhw7mi@;%@Ht8m2nWC>a^uAblazur6F_)s%xJS@R{=0?Hv0!7EYsQbzd3I_DERk(^!EyfCqq?32YdT-6o#H;95DiWl~ zLye{R}EhUz0n#StLZlp6E92uNj#vvV6HPUk^PcC zMMAQpQ;Btg6W*R3oTl^8C;=oOmn2{32;fH{*)P|??#@~V{^2WGeI}4nT1ICwtv>o zz~;%m3lm{e6dxGaI>B)LWy0u&A7lxKhvOiMd~!0>AyAVSwVroaHe06NtcuG63+%Dl zuNK3L5I6Sd8NL74x;>h>>f`pa9D0`yklzBEq3oX%lZ!-6=U^A{e||+RUqQi(vn^Hus@3q5o>` z7o8xnVNcSdc-vd@OjpgbS4bM3mOLInJY!WI-2 zw^w_%%}20t<;Zx+QJ8rK><7NKi<_58lZ5&8j`~kNO%cQzC@o=MZ`Ac!h)O0LaH;x> zwzhL^T<9COy)!?f4e#t|UE*kPC zA`L)bVsbjO~rUq~hNGdi4l1b&L9QF-Ne(`GIgM=wa zx5oip+(p3yv>31lYgxFI7jhXf7oJf`jB(mC?vAQbXTKDohMv`XWW^*aM(?u^n@w%F z_(e*V5NHCWkGtDy}AgeK9~MA^`dg-TA|d) z_)l>*7?p=bAz2d1z9W-C^iTa|qApSnVqLqIl(WdEr-UDKT5vM=B5AnRcW|x9aEl?fsDpWwIB72H zg;z9GuTH;tUo(F{TJD%ZmADRY_VwxU?e5+BA79}B*-1mQ+gq$ii_QA6OC(nI@lg0D9q1y8Uu=SbCM_}Wjm`=j%g zpnO|OKv{X$lxF%bPHcF4EYm_0zE~V)pnaG+s~iB6_C3FU(>1EZC}khNFSp;7FKW2m zt0d3{%WRREy##l-IDcv3>NfzVO1m`LW~;nRC7la}O8p(64qTD;=B69^a}z#>1CO)Q z=YeWYG~w6e@^uatXKI|k9kfPYHhgeRA{6umSONn|CB z*tb|HrctciM+Gxm$X&U#q>0Ev<84tIPd-G(+v{6!2yj=hKF{k2@7>72@%!WrDwR*{!kVzQHFx@>7=;e3etddH-eIT|PEcYzD>4EC)m+ty$v zsP12?22`4COHIz%eWY>a_-zFQZ#O|`R{H~EoA~U=QdcsDqA5&>R=V!INak!gpW#KT z6fh6iPfw3hg~YuaH#Su8Ny*&QR--I4O5PuA!@pCLBN7&5i8Ky)R8ZYtbRGRy2@pcl z1y)1E{OT;x9BU}0QXg;=T~3{SX%n7kYK*1+EtX#dUhUsBvVJVB{n4*kz`Z)Y!_VDY z{8@Tq@JSK9k6tnsfXNq?J?++>=pg|8hhX%iu(WJyIcN7ya|haeBQ>1J9E=!8bbdH0 z92+=Bvq9km#)V7;ktFiuvx+29UARXe%?4!3+l+OVf|*5BPAq&u>Plr(NmKBC56V+Y zh!y71cmH5_#IJjOjIy2jl^hD^#z&HJf+nJ}uoC~R5hHgj(%?Y&+AUsKa#B>vY|1)N z`d3dPxS{*CxgzYK2+GR@4}4<0GLGjca8I0FC$c)av6tQ<&s3+%CulkiDZq!+-g@ir znOETqdw&UH^(fE&hDXLSNmh8LB%&W@6LaQb>}cW2C>K$K9U&7xt2-yW8s+(m=ZH7> z4c>Vf;Sa=zu2kd`C$rEb!<%pvVTZybjQR<$LIdi=BM%-DvOH{&=3Ca*q{D-5c_MbI zJjK5H#G|JowFesB>8ZtCCv{%&6%0^Ub=DY?(8EEzKNtZKbXsGikjJwaY(YxGvBF;S z74m&y>_lS6Fw6pWy2653n|Y|?u5huP-b9Clp(v*lLV-zh^*UtVgg9PW@R+PfCaD@b zI?g*W5_!nla18tv9Nsmnv_XdBRb{^)<&GmlKc{x>@(Lz1E=bkJh!3pVpb!im9qDQC7TAuL;U=nB{4X z1prbjbTG$|=2;ixu*2BiC640^wzojC7ZAD z%4@WUxashMLPOvcL+SS3R3?0V8I9e)V-D#Qy!ymzS_Uu!t}M+uzH~hEZlXJD)}hm zN44!zq5@i?|FISs2Px6a+}Wczg6o4%cGF3j3g6{dK1w|A6_CGN6SnM9U}#vmap7l* z!%m3hfbi=2K>}b2!)Aej^qQZ^>Bs7SU=J?ppD>aUJgmjLC16j(B9v5o*Z&UIlZvtMQWufnG6RqM z^WzWL;>gKR(CDwQ=S(SL>imC4SqzrbCfjY*U`3H?gI2q)Rm%<<)(ekLX@i@kAbB2# z^OTz}8?QPhGpGY7SJTP6a$O(HFqe@7?r0CW+!O|tB?_;G>$fxdmiF?9b}3NWI8?*2 zGHVEKhso2|@AyCedYP3fWtfNAdO6Y_4-J){@XM;Q7Jw2ENi^YlhFvY3wAy>fEMhn! zd6#O{sx1irWU$k9T>f78R28FTzq9|8Rg^#KMwBcGGaDOywhb(2Zaqw?ifo6W2mi~3 z3!0P03Y&{8$sbszM{jtSn4rxXR1@oVYK{X4@c!Ce{js*vhJ8XpS+ZN+seHY9`lWj- zyhZ%6EsWd@MQ)7|ORb)4JCRVFCeWA;A2l3{PNnh3x1+_;fS@Y4%EPdIi?~cw{q*CD zg-lkMi0{%KTaggE^I?0y%h^gI!qf*LGz+{cKD@f5JnuhpZxvl`GR|};WLG@b?iCVQ zu3IdLdtm2m(YE*x{iA~+l^2kR4+6n1-ed=}(;0XK@lrZ#n|B0+^pe=u>!%coZxE>o z%MQsNitN9TxoY^#J$}9+{oe~dDn3I9&kg80oY(Jx{Im{Fy{fK)zV}V6aBo>nHZ`_z= zbpE^SP1`RgFSu?&og^LuYiXi+=oF`Q*~-TP6m0fX!v%X=7QSi;oQy1`PT-y=l<04# z!&}3Tgw^8ofvV_EyvO4|R<2;fiNC9S|KcLkENre5jPBPYSIXNY_f-4ADOe;ZHl0x5 zx7_Z+XLwv_7p5Wp)=2w%ODK9H+2U>flu)uphDpdoUBV9azg2oT<#LH@rhXbN$ggX^ zSdw~8iV`q(ze1Ed;GQZMg<_h!x2>4GnXF9_LSsH`1S|YKU_+Zc;dCqie6S0-wUm`Q za(DL+btZ#Md42!Q+|R*19Y?0I(=KYdxq?kWc{P8&C)?r^AEMo?u07goo`7#oTj?~Z zzRJ0t%e1QdkoKDF<3EC4GR^kO9N}1uVn?0rt#MAXe5{^6+&S~J37FLCax0;p?H>1=F??o`K=&~Jr#qp#vW+er1p!q`%50_pp2wk?j*!DV0z zxwtEH;4?fw9B;Pb({L|?6%K86FNEGDVsVVEUv+x}(-~c(QaCpr>akKpbMlnu%8~&8 zEIoPrD=C=_q%ROO!5r!q?Cd**5t@RkohLTQiluXilRB1(gKbhvt~vhP_|labFOzq# zS!K~zX4&Amq^VB@8%n)9WWJO(_oZb%7TeC=u>;O4JTrQeTc@=>ca^TpL#-MIwaU^@ z+g1~#!7kanu3j|H9_+}s^5q2`Ht`RAb!O&CJ%CILJ9TckW4Liqwwksz!d7xYdrM6N zmruC9${Hlox_ar%F{09a#q|;_T*k#Z!?bDRsG%ETar3aJ`a^eDX{~Pl6c0Uabk>+Wl;=+6~||KGD>n z!2|Yh8A-6rh+osPHz@&p8+xJ?ryr2=kP!`V<|i>l%l`;n3qNz`3Y-2k(w%rU=NRXK zJ`2Mqxc}Qv4-b&tw3AymuC8Df-cS&7kG)WVT6f`Qr2R2)Ql^Y*$h{hsSleW@9uWtg z*jeF#v`)Hfd-|h9RKvL`=(jor{Zwjb0v_Ta7vpj>kw9jx;7dN)jHuML3?`u_7Hkdj0$VRN zp9{D%%f_6{f1C|+ux!CPBV;d@xi`PPKQ|l`p@f+gH>`vB(am60cOz_p;1INSNsTJi zy1MBn>y1@57rQ#O9=4a#gjAwtgj9S3S-()$hD3adgQ-==7PlEH>xL+lwh+(_;ysRC z@`#n8GCPW=v-(V3&uQk6`o5F)rwWk%L(=`9`~O;xJ0QWXQVLuM#Ly&h;BevKaPj{O zTtZejr*ThR7#J7`h+qhCa2R>@AhDpo1%LlL1z8|$_>w#5kl>Jrvy;3JNgJ`1>*?vo z2SrdZCj(t|WNKcxf5tpXwd>v@-xozGAm{z=^WHHR)CBS!Y6hNuC}h$;>)ZZ&n00e8|dm{q2X|7!HC~;kW%P%!yEc`TfBJR5Br6=mW9_HnHFP zTI_gZtE?H!eg|u{z%&`m$36k}`n2n}8GO#Bvcke$68Iel@+G`FVu;5&qyDU@0`Pf; zIF75EZUb)8+xr01np;e<-89nncV{5Xd&wP{1iv-6Wv6RS z^}{k{wm#l7ocn=zzo*bcpsm&a!xiiQJ>MK(5dVL!>L6UOtE@B4v=yx3ouj7ND7EMU zm?T=Z^jXPj3u}3X7<}6tar^z>s{bN*6A412?&nBta-5_IMSH=aq;hDw493)dqN~9Y z+7O~q)%WSx&rz)|o;=rJeFq2Z2drJ>JR|vfJ-I&dtuzFDy%Kups5i)KjL~aTRhp%J ze2*t>gnrXZDnBDyt5cT?eT4yQSBt;aU=5p;-VbcP`4x6*NfO=ilbTwra_*bfA(s^* z%&fQ4cPjivB(BOwnznyVHKzZSQI1&;9>dr4{&kf?^RRVZz_Nl`H%A`0B6&Zgo9v^sdx& z9K)_=IL=Ys%C~mt&f~B>1e|j5CH(3FA@fxai|~b8@5?Kasg5s05D`cOZ#`55uLKa z{pdL5qyY#xGR_`E&oykS1uZ^bt+v1E#`~(>SI=47u!zp|)ES5@>r%f@Dt>swZ1TuS z^vvOF%6v5nn|uvT*%| zon?DhNeHrpM0zVwM(zP<1|Rc%n;a0=Z1X~O17FO?h5etQcbc04biLCFyC8aR$9l(k z+UnL!P&42r*DtqGmp!427nZprB%6Idf9j+DrBT2;v0IfL%YR`kgZpMlY}gQ1MOx50 zJQkqn91Ikgd;jA1_e{`yM}9!0*-@-oMs~9eAbx)-*ztZ4kpW62%J{Q$3M=~AT!qOn z>}IDAE(}+ae6rmbtK58cJh7SfU65YFMF2d}^0i*5hNnhC!YEPB{(s#-tH=w~O6|GB(S{PeT$mH~2_d>c1XO7w!9oP<@q8kO4~ zw}!sHY$hd1kFL+_f*KP1IzTGFm0ByLP9nl>xIgAjJ4dNs_=&t}$N0$|A zJ*WUWs=&})8J67dW!v4qTdMw`XcGRaL`KzI6N}z&q6AwY7Q?HMAEuz}0ZE_fp7VJi zSj|kbOy{a4`;$IIt-xrlU9}@WdLYZb=CCk$4s7IT_Ib|t>rH`zGY|Obg;_51232GV z4dm#6?d^Oi-rJFyjai%&{)&?*NqDZpJc`NEq)!bULf;D6A~x%wo@i#nE;9TAeqWBnI$1Q8Siio z?D8Y7qxL%B$&gGN99U>M1{~N0>iJpc1=q^&w@170&^TD+yaE0?GTm@{v_5Q(@Ufbu z6!Ey*XWPlazVfE<{-$?SM}TIYD{9`iY zl@Eb+c%9VfJMWcLhN{~HhSQINPS@R{R}QPT;pIeQA6J80HL%BZflOYuSSPxHm*Z&D zlk^`us?O}5y~QHV4plGNC%a^r-fa2@_><|J6O&giRg?SLFw=$n6S1HQW@3WBZc9%$F)Z>Xf+|%TqR^ zpswBSi>)I<7Dwvo8VTzXy&)DuukWRJvI9W~g2$ z;0gKXTxl+_nKd^+h?g*+*xf$HC0h;56XUEYkx`Vz+~}N8k-8m&^RS;E8?eHR>^7bh z#Hu&FIuHDD{wUM+0jnA2lpDRM5_2pukJ-+`f?CC2>Nn$fZ(|e_*}gJ-j|QImuBooJ zV2|?L_s;{JTScX3%6baV{D%B2uI(bPB{rLtmn)EY@%kM$hhjWVhJX*aaKLzZXl*B_2wb4Z@YmiAAj8S3P^*;ZT{>~ae6v(M$`s+zTKAGR zM*aJj)4b+MDm?WLN*}0n4>8uYhvU79G%zC(R_QFi>QRaLmQYIYPTP^-K)wv)d|<}0 z7qJm_ z_({-dmQXnA_=^kY{kYFi`O;d<02#crlkd4ZYbC>Xqw?@dYv1|DJ8(DmFW1+f>Go|Z z?K`dy-_2r=pwF+UorIqm$Vb4t%&+1nKG`pki0>NJR{O}E1`Qj7bC_m^ww-B+8KJQW z)`#)1mC&-*@u(B!{fyA=MUrK%k44;f$6J(pzE{!D{{W{u%}|fEWT=4TNXSQGYvC}J zYsaK6QK8eJ_Dw9I#y)GX=ZNdL9n_g=O4NbXQb=zev`&6C*bVm~mBXC@CYCkxKNziX z)jE#A592)Jd{`fqgHNMXLjS;w`^iur+iE9E`My&BoteMWT`_q9|C$UCPK>BdwxJ)q z5L5!h{rs=qA>BVZUji}O>Mla27MUch-uF*TPg9U<#IHQL_8S1s-GeyP_(BIIXPp?I z+5IO;FFg5e?m-!~K^!8-RDBY}6%tc;t%q<=u7ft4-QnjfLp@7<`lQbQ_)VIOKGcO| zs{H;+0bY?j({H)$P{h4p@ogrX8(?O_PQZ{bQ*o)>t|0e6s!BQ%CEsOxBoYzJ0L|<2 z&$(J65G)C=AyAX}K-dM$Kq35uAdzNiH*}}HX-84cmhgGHK(_UB+Tvo&cg^8E-1=ha zdf;==uOh&<@lbzXAFPJ|u4l++|4DOsN5t^%&}3|Q9Pt7jg#7e5h|yW))EtW`Q@IE{ z>C@!98PGf43j0>*HIB}9FQlry`R@NUOhk#Qu%{0jfC!PnbbF%Xt zNDG}1UfdEIT{!b=4a0E-Bc|QmD&ju**+)HQw0)5CNondOLlphVaqH-R9viuuU)+{; zd%1z}jdI{G2q#>|=w+Sf9oYq+M}Y^(8)ou{FlPLL#t#(MMpoHYR1OHU>c`Qrua#17*AU9}e3 z;*`D z&?^D=6koI@3OctyKheSUtIzq6r66{XdY8Sc;d-2O+K%lajgk{@boDl7EX{<+=?#32yb>6$)SV&JRcZQ=9o((Kbo|mTw~J^ccwQd z6bm>cx6!0v*XL4TM!7s)+0xby-yTnh_UY_|Lt*38i4PW2I1>3Zc!c(_g#@t0|YGso7ul+CF zV;FM^DR$RdXLQ;svFYg1?m0CP;x_M#uaaY?0=;wpWApuRq63X)iU75W=bQ6+Jddqp ztX6@Zp9;wYISP-JpY;CdH|A}yz*ELn>5S~I1iF;$M_c5~A^tRve*-$|WlzDW48-uN zetxP}YQUpF7kENdOE`s3R8NJUy24J%#PgLOUzZZ);XQNS4OWD#O(FwPi5>M8F)pPkqxkCI*@J;- zB-xmGe0QnK?qc7-DhB<6YbB%T_#DTd&q8bg8_ZQ|h_>js_*cXa6N<{7^T%^{gQ9k+ z%)3`rdi+J^G$Rga!;u&>1%n973Pm>b8{KIxf>;8GLv)IBbH1;~JQ1VPhqmeU1}bse zCc2MglJwcSoBQXiZ|@f{Zs9p#{}!!=;}Nul$cWlXkPD}3ID~NBdD#k=1d101i5vh!F*Ysb+!(eB)Pj-`(1@b z4r}=I@G)Z%V)jZ;X-4r%VO693?&oq3iBNq=#whO*KE1JRDZZwwNDCX^fc6|Zvh&vv zV3m3LBh52UCqU8=4MT}*N}aa>bqU}>o5tQ|ysQy8<}bzd@*xZNOMO2M&2Ryr1cYNko>25 z;sc0u@b0U@{5!4XkE5~HzloPjB`!L-fe30=@=?FpKt|2Le^U$%Z%?6B>jHK0VAI- zMkSrAfAo0U4SdRklx7g#6nWU$(oh@~4K>k*!DN%0a%I$_R$~0Y;u3 z*~2JdypIG_Nc6DCDXvwnFS7%hkdx3zS!H%teB^fq&V>8LJu*SW5qi^8&s$kGl$Sj1 zYHn!KQbkaf%>G7hc!Nn;pTc~ouOovnF8DKPba2i_*L~(Tk<9^-6w36OBc}nU46^Y&LacZdS^4Z!<28XcBCgda$N{Ei)~k|Av4Gk=)Vy= zlAk?N0^R%OfS=&HTwg8$_G8br%v$v!yO?GrH^0_ylUd?Y9Y=lRi0x%<|2QJz4blvG zKsx0cXB&Mj%%)-VORsU)8udD(@V#FUSMBf1S)#D#EZ3fGcmMt99`PltXKb>92t26^ksbv2x#ZZVTx}i*Z*y-?jgm8APPFK1kS6wE z8R_tQrFqKw9~nsyk2<&UiVdFLA9Y`_f(sDm-Kt{rU~&S^gJ%1D9N3bR@HTv>Frs5OfCpSSGSrp*)c5u9s%UEhhE!-m6Gm{h# z0SAz(h3HX8^-4y!nWhgb2P~-D{;%H64Ir6qY4tx4KTMt1yAQSiv*c7xl?c>9D=Yu0 zLvvIfv@QWsi?|K`{H6zVAp$ph5n0f)Ip^V|d*%O)5L9#jBduLV-a(~EOt4CfQNfNM zu;as??R`#s4<_Nl+cx_&g)1KI)A@p<4Sz@p!3VnwCZcXe^KAHvuv9?;Sx zBq^-SVRsG>dsYP>_WG)PTios_y~RrcmrABIt$2;@h21m47+V!HkuCMWGmcUk)WdE{vtF-`4DV7 z(^E$w8KCdl} z=~5}BuRp>q`(LD|Moz@h6 z_)jrLoKV&aM?1gM9{gt)Wy70f4x*)#Nub@O!LXdm(x-KVwc(3&@D&C_mit5Y?g!|3}tJJfwg*I>?!m2M^w)FAqXRocCosCsM^UININswY} zQDt3ql&e6q4+FCof)ZEFz_2&Tbs zAc4Gcm`_K)-;ARbD~+7PYG75EhiEzs{RIm+m~$#K@B9>Z%>bdI>d~hl1wJJz&wgEh zZ^MSFf-_&RL@M*?qdIn1+!gz3D8*_a_J$-y3f~!*{U%y8AN(;7F5b95!|#9L#c*3^ zhIr$^9%S8KC=@4QyB2PWj!$nOx8Ffk56>JpgRGJ&_~K6H2;;RLiOxK4IH?9kGLnDx z@{8i))A~7Syl=?$HSKMOF<@!f@eNUIsg0Goho1)qEIMKu2kdr=#}xHGDsBmK@lD%v zt+Fl@`3(6R`yIMY@B9q#%cy7{&J=Z2$y|jETPmG34Wxb?^Xq;K8ot*zRZejvO&dYBUW_?Ge=?L@KO_s~v_v{3 zF)c>HLn2PP^lxtdSIx7!2+YO&f`9ms(xv0K*}{_oIKzFghbNot2*Osc`oycRrYDe_ z*j!#eHPpfnSR4@M+u}BIp9`t95aMnJpciD8QG$4ub#c0`w9;>&!dH>;!ly!M^{SaI zqyQxkL&nC%V{)(s+aGBa>jGd`RK0GL8F5Zf`UD>PdqMN2AEi6=f~bHS+n8lI5rNgb z-cn<^$o%@}zwjWeY9p{RSSpguu;ybc<5@IOb1}N)r`OizQ^g?CdddCO7X5|4_VT2t zEOAD|trH1Se=>2zkNXWdvORlJ$IWuAdTdlc(XBf{(5Zb{09w^CMh-o4_V^0YW$|HE zFmWdtKG&36(VdP0X)jlW2(Y8+Y~(ot`j?qoBt7A1=S?r^qS|~F?va+#Q9N!Q>35)I zHRTr|62rVGaYzyJ*_fN8`zZnBwCG(-w| z^R|eM66HEKW4L_s5$FzB9@R{dms4xWENW_bmLtT!7cr+O4ukp9+`p)FM7cVzWTb!- zPGPB@r}ROFLwqFb`_|HnJ7f7TTf643(+!;^qmY|>2_uE>iG|v@WWV)_`1KpmOJai} zs#B$gK7$vnI5H{1Bj;V=(|u24624UIRoFt-<(fC*<3tu^H0M6yaBROw%gfJxsdQ&u zG^kdo9;TIkIT)9$rbs5p4qrZb!q>;C{UJ|YCJ3aVf;pxclOe(JbRb3Z?ATPW>dckC z+r^}NB2|)rV#nE?9*@SqeU}2y4KllK7u{9WaO*;I65QfQ{nFSF8MX8FtQ8hg$mmxd zAWhddnWdo`{ARM%!u&v0%oca^df0WkHnacF<5MTF2l=Jv18?vxT{t*plEaeK^dIM5 z^>7H&Z4%tj4;F;TIz~c7g#$>as)7Z%&76hoS7OSSV`;20%EK_XNf=!uHQ-zz*%B?( z;BFGkVLlTTgroi*&1{<9K-WOM_9i>-4Q8Nt&S~2TFV77AUGbA%iB0l^Am1Oj$-#O} zGJ>`N`h1Rk1HlJeVv3d_? z>#5NRTIpivpk|~>Pcpn#<&wcLs5DRjQb@rsr+d>$R$qsEcJA)6&ZNTw&gzcQv*}5x zQ}OdB$1LQxR9>=Ce5Znh`$2#1IwF^WRdo4t zSM4~YzXzyAzBdvYYxm1+>cbflXPZ0ZCas}>R)&>d4{)8<=O0-_L@yV1xAvo353L~0#)Z@jrf9___B2jeQAmOGgrz~Uotz=f9 zfYP1-LF$}WdH+BMBT3Y8Xd|!Ry7)+%@4Yo33XN~7BNq7sb6>>iTWKtI>)+6dm1Ld# zxW?_De@gN%em!cNtVVK`A=pe(B(X{MG`={l0&wEu#!Lp)Vel7wBi{IdtkFF;lc4pT znvSgjW82%8d)Z1nHCW%5Ph>^ik{J?;9hyj^S*UO7;#+Ls21Gf50y$mrQu;ed^RKu5 zFe!l$0k)l_9Gz7IE%`cM^!naoBU{qpN^ZOa$hI6Cz~VZ1phYNY;-mOTrDfgHn`n^$ zutJ*ZuN7E{96B#Wv5tsmQRztrpJ+D)GVE{P-ePgQj3KRXMQ z7`iYm;5M`5dlOAm_-m(W`a3?XHzO&YeBoI=?w9GXFHw;+jPmq3l}vRum!J_@dp$S! z7$dTf(WvU6LR45;*W^6*qX4-QZs+3y0~tAF9{-H972v&1@ol7ZFxq{yKiHo*%r9zrUGxey{OFU zQx6gfZV3_>GG(}&S#b|cbigS2s0&4}69^JVg9AZdt`-_9*W)Db!W|5$+NkdYZCQd` z7`KH{>NLzCVvAGz7Zx&t7v&7KeHC8V!9-PIIS|7lgrJGalkmP0QiY?v zmd!Ld76yeQ{yC;ROsI3M1YwnCE)`kq|_SKiG z;$$t=p48r-lYhU5m$;~dlHb^-2<1_Vz=;8W-?Z?>A!!^Z==e4A2h<-9GW+|l9)9wi zWuKYkgi~H32^T?|tMKB#$B+=#ZSk`-Qg*J?J~1bAeL-C}Gu2{NEqEi}b()qFAs7}I869`t2%LA#sJM8}$5_+u|q??Lz3TE4p z|G?vBD(*Seclz-bia)+T^{R2dIkHnT647lCj)u}*j{m;vT4+gWqvIcrzPTJl*#+%q z0Khm|Z>`%84*aNrX;1Ez85G$<69gWjUREpx!&@(UAG=kjlKBY;zXOSn!=O*2fW~!3 zDI;1IMSW(2Hp^2p&EV*n)z7Ud(w-mjBVW1IUSU<)sUIvW&8oVZP}}9lQI+szBJPF| zrtp@p!mjPfdZ#}JhVt3fu@_!BAa0z-U{cD1Lmh6^xzw)@S6oqR!s`|+JENFhd}^sb!a z^WD?P!h$5%naz%|IR>3eJzcMtqOD1N!kk}z<|3@v`~4-je7=BT8B-c2g>t+E1c+^^ z>^J7MhfSA_OnEBVjAhG;SR?4U$~}=FU$jm43C8B^mj-gl$!qNyzV% zGZe=XFuYqG)?}8?HofZZ+a1f#X=Lvumna&$$oadnFOhM7mZ#%;?VLoE?OZ+d$%Lg0 zB66c@-024q$>_!OTbpUpe{QIbj zo>}lKkJ>)DXOies1wxTpfXFYfn$TI0SxFX6Aun#rd0A-BOfp+m5G$ZZh1Mk~^{iFo z==21ZB(1ILJ>oqqWQ-UWxK+JQcb1N+7Tq(rz(pRO{G1Rv`_?Q9Mu7=u?c8sPBVSKS z8o@ta0I4;iSFX||5jTFB{-2i3%ZJ+%iz6!IIG{hC?$oakaIs{zy56q@g}qX<__WsS zJ5yMGs{ecQq&hBuVMs=#PqK+3y|^7ySL0>^=ryRC=(_obq0HEwHprQm!o!{EVlglF zk>9nRGx}l-X}3al>c8|H=oeQ=fn=smYCw!Ap#=uz z;qL&y0<%A5m&HXdyWpSNBJ1UPs(t^h3%DbjSGX(KQHe7a5#Dy;(f#y{7ME9IhOJ3O ziaC^F)ibQD^!aE&6r8I4zN%CSfp2@;^`EfNxOb!>U$-4LU0R4T=?@NA{O&TOs#b}J zWr7snNk8AHV^Ho<{=6X;@C^`t*_LB82w8a~ZfiPX04V=(&o&ChX3ACm7p{tnF!G=> zXJ^o}kE5j-&HA|)@o``;;Hi=u7-&fzH<239;Gf;kW*rG46>x6*$)c$te^<#~j#*x!;#RcXra@ zDP=)j{*O?QRAai(Ba-><%E2?0=dB!IT^|tV{$6?UX(LWuls*p;~AfgZPcpmZK=F6i? zm4(m9%iSJ}FZf!d)81QV^WL}LIB!27-9+en;`0+j7uP~b&v%WT305!X zyuRFm6~tCCcBE)m42ZEe>GMdo2kv;pO-C?@Zmnc}s7P*v29+OUy-r!qW3XMwk=)q6 zXj_0g) z|2CC2kuzC*OBs{(JO8i^M0#nWy^x6kwP z@Qi@*QF8fySLMOqiXiK^4B*DbV!3VRZ7bDAM;I&q@WN2lYk|gSS3Zo-fT*JcBH?Nx zJv8yv4Yu@n{0{(GK&HRY9O{@a-=Ioeob<6W=3ij^p8n*K_`JJt$ElQq_km*zp4ucR zBR9UFj3-QD18*$+%le*uhK99#s&TS2W7uOh*bv6>Hxl7B2%x_c{qYOGTC3!`AB!P^ zD*i>#D~p8OfGQA3r z!0cww9HMPN|1CSK4_ig1E&5flcC4hTNT89wXHlUfAp&5x#=+vJ|=!$HJ+dMs?;nR$X{9Aa^0K z@YnfCbZm|_1)jeHNDTY_P7n9VYk=MzNcZc@d;IA_MJo0^1rGK5{iFvrR|7ku>FbTU zm6h*|1lcy$`OQWP1G57#!|3m#c-UJUoP|hI?(eP-KKecu@M0p-4!PPoahcyODJAg( z2PaaZ3vz?&Fy*|BZrya8LctyU`(}HDM%~MXKJj8bXh< zVAjK!YuEV}UQ?=^sBY2?jX<%vzkS4e`;$-z-)5|sgr5LU1+HI!C_}-@GE5>tj~g0U zATVV|cad`RNR(Uc!RU+sD%21Jnj7NOIh}4tUaFVd^^cg4g3cps1~Tq)LL?5Y;X~e6 z%0P6UST+B&>zj*I?0CRZ@Bq=ox-*#v&5tu1Zm$GP^9;8g64yvRPwNHiTYA6K1KuPY z@a3iX%xPSj2kUBhY`V4BBsS7qIsSUL2lCgPwP>TG@D3q(?E zfcUUCO1NWix%HLn_ypk_1K8G^)K+31!u<2#dVun0S0Zy~K5g(G^;kRxaxI;6;g~?LIyV}`xi$ph%XLmt1bQb;14C&k6|&7fqC%g(?|4m z;i}6JJ{Isc0Z!Rr;_;l;hjDG~2igk~RAW&WY_I$G>pBHP22Y^7e-x@D1O{NgzdO)( zv>NO$ALx(Hz5q0PB*rV$V#_Hhs?oeokYq~<{(Ap9n%{Bb)l5)%0;n#-`&vWs{4Cf^ zvI&D7K5s|+G?QEv{O{RL1>`&et`N2L>)x!Q&do0FPYj`n+e-o#o)R1+{KE}L%j&-j zk`*|B^hBlTPjkur+we;a;A94$=5nB?uqNMt`HfAdygaj1o@L#UAFIICd@g9_oB%J1 zZ}o?r8Jc3!JZ*5eH9el^>LRPDa9XjLB;-yFrr(LhCUyKfpQvYzSNUo7g)6+Z=U0y& zwUP@mHi?^XMSD719Kt6sA%1`y3Z>(5c85*&jZ@E|9XY4R0Un-dJpR5$Q}{+3#&n}) zm)&T6S;$e8)#kdIO6}EGe6sBjBmSsVR^G=R1XsPeSdb}wnLoy7(7*OLuRL)H5Lgd_ zYmBNXQFoO*anaq#S|MFSdA{04->dxTDXMcXNqXYc_ zM|eKx72zDWNTAi30)d`Yzu-$7OkYIk>(!jp9!+%FkeTI{9&$MboA`soPYbwS1iI?yvguKn$>-skb%H2X_VM|IH{5lf` zOQc3EwysCa!_uT8W+{3;FTYklCk6@1omt%2Rk=f&Mf!xC*R2%MU&HPfyuy1xXTC>7 zeUyZ{Vk=T6nu)Cq2+uW2#4-*7zNa>pqH8(yegVgb840(qa@g#rMlVM@EpRBn%2xfp1I|pyh zUG=YbFOX~}yVkwFPXoNqkNQvmhUqOwdwhodO-!IO7Nxnu8UK8hQ4Ste39k_K_L5Yn zwl6k?-&c$5o%CsV^&gZiV^m)7SNwD%1P?f?Y9`+^AZ6Ue$sWa-@vv=mQh;tmu*w&? zsU&N|p432Le@!a4+#^>kjM>q%Csugp6_qybAt){tF57rXzpa&is~-nEb028Te`yi8 zNM-#W+pi>dA?&eBa8F|+CgAS$a7L=p_19_U2J~6?g}nd#2FL=`9xY<&wHKEUE7Fg1 zo8lx=FNuB%Rw>Rmg8&U#QMGiHcR4o1384Gjh}@(REyX@(yo&CCR+;<2`6 zGkZ+S4LM*krbJSis%_mBUWjZn3Ar1Z-4lC-$aU1;m)<=2xv}=;T4j}Des?gHU`&Dc zc)j8Cz-WH2V0nD!*XE0RMfew4o&1FYSB-pa*!#t|BgEyiZ714;_UBC=E6AMUlSCjS zy!Fndwc$o#^x!9_(|NMT0`G6@=%5L=Ui{_OtJU}m!owEj?41(x71~(2@D&y7Nlk7Aq$SfSIJWil&yvCX zH7?>qEyTeYIP!Fq z|B=-B^MQF$W{WS+j5=R<{}cf2B|SI>^vR>Tfk3UVDUIB-e_uGpWI{mvEQ~TtE-%x? zgvLKMs1a1;LEd-1=++9u=7|`EW>7JkQ^~z&fD_Dt zos0zC3c}sd+jf2hraSBE`oS+d-9|V%yj%TieVfI^XfZ~jAS{L`x&~yeCXtA5S{Qri`#>#*U|fh4u!+Yrrcd+izY7SHoH;uZNN_+NEOm5#ZP#p8{by3&~dGi z3~cUyvHZ2Y42j#D$6vxDmzM_c-$D2avAUp~dl39Gb$Jbq+1Z)ys=aXO3VqA>;9Fr- zCMD~A$MbwYF`%&|Z%2QM_la^Uupx`uvO}4B&q2Mv$lsNQni7JIGlLG>Bi~(@4kyyN zYM_R7`U2ve(O%kJ>De2YN;gR^A5fZt+XWj_f?9C7D6-)gGEiJs7k-va%Gsh?Jb z(kA4HyS#<&6VHE#&nx5S+-l;b__b9kCmty=ECG?2aO32UyuY$zZ%nIMycCxx1&Kn2 zUKdf&4gv{%DS=n7z4QHoA*NI)J?L|=vf;Z(CR zwhCrNtyi?$hk70h!Vs|HV4njSg(n1dY#2zs;FnXUUHKc)nH=g_R7#Zq78N5=c!7SJzrf8)D< zRM4)VrrPoE_qUS`3^=l%=>5GkocvH`_TU}554@0d(!WzJ4~WD5-C#rfesxssMXynv z8ygNF3=hYF0Df-UOUG0*_8X{p@NIeWJfF4&v&$E)F;h!Ja%U<;zI*z9VsYV(V?kV< zzYTt~rCJACeK9re>#a2hMQ3_hN0ZWJV~FYn|MfJO6f><$acd*X3HHJOH*jm4EAHb_ z-#nmFK-*S{5qR`h3-9^^iSPu&}0aMLyf2gB9q|f-VOP7*gq<`v9!%R8paFHV(eZUDy z#z<1*kx6*bT|m2)Kn;}%^Yi{9EJ`AQX-1zLND>kwTYag0$t4m{J1>;f(9_u0?0`@u*5 z6eKEe#9yNPf+b&;kTAIGPn;72*`#~Xzv~LFFBeu^)^4r3xld_4<4V_MR|~&j3xel% z@M}x>@$9r+>dJd3D^zjhY4rMN8C8)tCf7Ss+!N%%SL{Y^zbI#*Sw6qXW9Apgws}h& z{PcyH(mp)_$gU6PYbcBjHU{6Ghm`uDvR&(|AqCYF^Bbdja9rh;Ui=B^Vo; zn9gfj&l8`R0d5oe`wGy@cc3s2<{x$qtvABARO7K=Mw;CNg6P(}z@yx|#>IN*{@wzn zjV-pL4tTOq4df)|m=ifn(%i&;v(rn)G#=TPX|~~j75r%mZzk7RrJ+Oq5WlYR@c!S( zsC20W=@YODmWmVAY>=(NHFx6{{E~R$$AidV$j34jwLRtzDR}y#K@Hgn<_d-J*B4Kl zSJ>+|2efeSU4V zzr2CRC2cSjA{>t*)H02HE4o{pMoc5%d&YJG1C8GxTLbid2D@Vf*?7^qO0d7Zbya4_loj^0hwLD%^k7{rLAyJf8nU8i%8=J3CneOnK_&3|VbtETroqC}P*;xMT=GARikH)dtNsUcPR@Wvo7f8YsWuj(l(=AX>Vkuphi)zqoVKSk- zb?T35%hurlhcs!=SAyNIKH8<8YI-|#6oL{yGB1+4+;KVeGbya-eAP?-=aDLC^iHU>v2lObZYlA2u`IQS>)(EdA@9(F`oxnX|W4ttmI}vbJVz_45Wk`#tRdF2~P* z#cpu=W+{Pu2=9u%9F!#c3wx6PfhBobl`LIvfFZO=uojDP-v3P5Mf@oFjZ+ygX#P+x zMvPt{NXKH6L!n9FD!?_3SH8CuZp7b1A2RmWtk|xl6tu+?=yc0@5lMSHLrrCja?^fD zzu;W_zyiZ!+vP?sdr=wAN29_}VHfU?+-(kRjtd>~Mtos$K^vEv59#g|IhjtpJ>_C+<8o7wbeZ%O@lN3ob1k6*%L!JW&`v=jq9+*UBPAc<;*w zo)agpde5G`tYH2;%^h0EWb2M^8qaz@HVowvRYbp*#N}(GRn8yWioS78?MHqiq`#7A z{>9QzlPq_IGxLw-F?yj%X0twUmixZOTa)GAXqeRIE^uGGPj=M5zbg@tD>(EfK=Zm& z)^@w04@jIsddo^(Fx+GIGA=!2vEv5|5^~g@9KHE5t3b3ova;dLTpXXb(?!shrB3g* zWKLVKrPec4uCei#ihQ61kswms5Uq5bN)=M}m!^ zSJ2*O-BY+z!Gm&0+4p@okl~8jkwV*T)A%-PtRTK1Z%IqO3l6@%1Nyh;=L0$HN-m}F z*;_y}ztH$4QW;st`PE`K#oq2STO^FK<8Mcw_W7DB=EeOE*bmylXINg_HDq&S%bCE5 zZGg?Zud21?Cs7euM4e-_6G$e8on5NX*N4Zs785p<7*E9bVqyZ!i-Z!vCVjbsKWl)T zH#g&4bjf^cKtX(~M}oJYiAR=r=>!_(^UdOLbF4Jkh^a9sRjIv zZ56i`{JHD#!(Sk!$w*NI7evNszKLG$ADo<)qeEr@44H=xppuBPr^JaszK4o_2Ex%R z{yr{rIaTSM+2GX4D(jyrA&k=U3xTr`q(I4Hw|h0pIXDdQzH7|1E$!U>Sbo2WJk$#V?z0Apn?3i` z!tYx*7r^~c;4O)iU*_A&yWfFi3x`>1#rS5kX!w3hPjqq^UUk4`UHAz1k#vR%JaFa^pO@n&3LIGpuGhlsD`Ma(yIVey5*?N;i` zq74M?(pc}mr2NI2=o^ZxTk!@YBQy2xK9vtQM5lzq~MDBHdA{O}I ze&KDF`Z)^Uu!)fhy9r8@H9l8oD`W<%#)G{&CjeU=2>O=9jqG`-{Xowc2g zR*5p`4ZaI;1Fs2OnQrr#HZfz_Yo1dX-OR4cd?oyHV_~1>oUipS1iME-sw(bhJ_Q?s2_I*L5lphLUB2A|H8 z-6>j9Yj>jGt6I_!#Q?4NM(6a8Oo5sGF7gp)IZPNWbj@X~{n14k8F_Llnn4?f8M%RU zAqOrT;q;lfJC1a?HCjT3Y`83!3Pa0)(tjU~5}xvTeJY0Rk~p~&rfW_fn=qc7?;TC& z>)+Na*I3k0$}|2;E`|X=wluVq4c|(hLdh`Y7HRT~^me{wF}5qTA!3;kLE|3x*T;6LIhpmNC*^K7;6kqEC=U4tZnf-*S}h!H~<_r#*ujxVT#d zzIg92td0C;{t_Dzs$ZL%AiAx!lT@*Iz{j3PLZn}wP_KkCy*6IggmL%btw$d-A()1{ z{R$|bpcPEc+ABE0r8To!K`$k5NaaJBPEWo2*1wm6m7@Sc?lYq;6uQIs^GOxRO5UX| z-G<|%SL1b&bAm+yt*>~J?=3f-gzCf`zVu2vm1ISOqL7PCeD6Zuwq(##7+-0u@B5H4 zK=`c&*!sB`ZsPfpEK~2Oy`4@FxKR0yG}A*ML5N2hJUx zJ2qQ6bV#Th(WFa&@4m5{IVzlAqFv^~oLx=f|In^*nQ&>zksfD0H`q#;7!*) zAkI@Z0q(R-cKA&IoJ3Ow41NPYUfUvJ-U4_Qz<30j$r}ph*`!wJw%No zhNOXMOcQ8Y7j*~L2k;R-7seEFVbQv|G3i?d=w0{1OmDTAKT;HDo$%s1Tls-q|23f1 zW}z!0^Cz7S>u{j2bf4xGO3Fq$Xg;x&BQPxC7=K&a9I_9qFE@N^dlG2(S5j~MQIdy9 zp?Y@%mt;>552;*OOq)FpzQ(3Oz%M8WU$Wbe0&U*G7s~1`wKP0J@ zQ=mofyW?eI7FvwDZwv@QQu0ClYjF+HYo~+n10I$4+l3o+!>}OKY4wc2muUzgCu->T z2(t^4CBx~Rod}S%sZu{{yX*Z;yZKJjvAM^WXQCKY$ZDQ?i>xnusZuimdHVv`-D_-I zVL*)=P~Vuh)?}>=3h~LU-;Y`{Jc+=WtrxVr{z0md$Mx2$BT8l9T5IcEjo+0HN&PZk zI!k=jRD~ZlVdSv?rh{R7J@W7IhKKp#Hn?1$(E_xQ_9^%as6>{(ob0&@V56y^lj>i~ ztr|Od9}m}gae#B9j>el97BFh-d7tRh$F2byQ>Od>Kf8z@qI14US@CS@V*D`QxJ6Vd zxbkKf$3L!I$xM;w;rNw%6-pVq1F-fQB4}piD%Kr6)TX0SsS*8TgFs=dGXQ&IsErqS z6LyfRk_$}Tq-%ntMv;m6t-5z=aCjr}H0vH^-vBR7vzwrhHTQ7ZxTPQR(TKU#nnDU{ zT*jspe9?k!d(7t-dAxya@IT+=BbugvzCye_r>Fh~)C(E=rAkYn6M7YpL3Hc(WJF_@ zVGrjwu-yEubQ8)q`2jsZYVsdvdRo_X`rb131f6uf$o!yJbV@0H7o^mwcl~RA;w{8i zEO@=Y_Fj2HP!u|mre+~9T|d@c9P~TGXh(EtnlPRSS{T+m;8bAf7F>7Xu z30YwaRYvg`gjOh)9NUh#6+#IIHR?31B15F4aiHvY1qH`86UK^5AM?>!hXoC`!d#Ne zMhE0T!IOKiO!mh3Ll9!yI|sRi;GXuiZ`058C+~u139Ve}A(1;A6OEVJJ-Ym*z6|!i zcf>96O&D&o(tF}?c0i*`{<4{jB@F1#H=EN!9VjJE86V^7E#3z%wKWL#z_NQ2gtc)I z1^%6_>}Bsz4M) zf2nfSl&nZ>f-pofNDxW>{p0nZ2lkTIs(a7bJ4M_3*ls!l3gF{(dsqAf8d(QX)tyx4 ze-#m#XSpDEcVDX8BKfsq`GilQy7YcvM_tqM0LOpxW>310d9@*?s z7iTzCEnAL6)-#mJDwPdV;_r&>Cam9`D~Se%w*lcT!iY}!V7J1vQR5-PHebbmJ_U7p zSA4-v%p2sB+$$a72w28{jrF3gLm|JHsz_8zY8psL8HAiteXrPPwVVWx!Dx6#$op($ zjI{I|`P^&1#bmsEB8gd_uOXc`6ZV(_<=mc3dpDoY=J1hp;+vQ0$@U|_R?epEQvr|t zEbpzY9OHL~u~E|iXd#|v)61PUpKiXw5^W&OPcor_HwHsz&%mFMZmbjttZvM1LD&=+ z98>va^vMc5X5#+x8_SZ=9C zxHs0GJW2fpjzvL92}%bK=o@SfL`x4b97+3aXGH<#A}uupf9GJJ)^K!Yvnj77U<+fg zZ!FAx#VGui0J|F{@haA5<$3@sFi}! z*<-brA$Q(^45-4bDLH}JJiSF3Vh}GV_zKzOL7aRt7(%{m;1|64-u(Su?gL(S5W8GLO1Kh1GSM_(tytS;;+npL-?G2!QUi57jeAjt{L7B`16T-rliVv|`#D)c zJzSZDzSep8af%}wBR8F7X?u~y|BAKv2-8anPC=sRkWJE-q9|D%=7 za|L9fgb<5BYMBF45;#~j#rnE3@3?8U+JjVp2yyz_e?*e`NU6;}_VHkbuvWXv`_o#F zsH3TcBETxYnQpXdz6UIqY)F zI-Y$?=F0uczV{K5+(rh8(e$+$ANs`j@IL?mq3^WIi_H_v=m}|T-z>i0owbV3jg>2R z-l4}=T$}ke9apLL{1K2;IC{^A&!Ax8=gt)ehNdSf3BIA=j6}}0^o%!HocgjIAiqK8 z5%NH*mLubM zUKv1d%r;vA^W}@_tfpcno;{h++5JfxXvP=llbzw5a9LH<{RaLfs6g^z?*&kff6t6> zA5v zs=|5EV0wtDNNKwx^6Un2_g;pTL-6hrMGskJFu%nCCyTVHX!6!x{Y2FQ!}JjR$75uPEkI;f2fM6ZEcyJ2EAo{$U;ps+W>$8* zC7a`HAvXZi{6bS~lfU8gY+@jBUbCGbP2g$ywAuM^cED5B!{B{NNqdhWy1s;XhfCl)mSrP%;pV3c$Y`2^$jm^#@e zq6Shkex_-L$c8KpK?gj-(C|a&EOsSDZX>>I87Co|uf8-eLgM(N*W$T?_e>A#%BIqG zeU|$D32E|KU0`a63HI)xd*a($EfArIBp(>Z=B{3V;jBEmXbEy!LqA&BcmY;`qK@lR z_Of0nD>$(#*tK}Ky0-#rQDO(L zi4ymm0v6p$c6~Hr<>H0IzD!c(1FHS+LMpV<+s1eT^=<$r3<-2PuyAz2G75w^B)u77 zxiTjlBNyues<3TFOfCPpT})Gi*Xy5r91>Z+M=-sz;Z>oQy*JH%v>|+qe_oia1PsY* zMz8HDy%KK9{>|F9F-{UAtH$35hkg#`uUm+{37S!M-5J5NFLedH@UmoW?o)^tq^}9u z7Sb}RHM>Zcijl3K1f%$raN4=0-?D6|x#?f~l z>NY9a+a_i6D-;a0fsto+?L{mqo+$SY1?zrOlU z$%0lw2@;EBJ1b>^r{6z@=1M`g{MU|h$`SRhO}{exSJc~asK}tnfHr;&)GHgiYBzik z%04t$*0mM{wTi+;{EVnp13R^1p*+TFrUd-`3T}d9#f|`J&WNGR3v8m2zSc#8YJAsM z7-{R#UX6G`IKAYDam#z&N%F=7Bzr0B2tW&)^CI5v)!oQTC2p!#C#>g3tQbyMV@`oB z)Utkd&9g!K@!t5HbDB^}!KUFg1rA>f18A}mA~sx-Z-b5h|pT(T83 zg`0^!2}R2XtH9NR$j#L!pLxAC2N2~laJkX0mSMLChBH98Uct_RDyy|9&=^gW&9^E! zX1e+z9^|>T2xYv)cbUO0m~w(J^L!eD{(5~UMsuPJ20A$)bchh(@iA!lwD>%R%LPtG z)`56zugU)NORW?ywB`b~W&8q#4l#@wPE9nyeL zuZ|UKJf(u75%-(xQYYfK;5_|5$T|3owAQ(EO1Fl~+7#Zj=b!e5PlOu?BiS6yASDbh zoQcxLCX}XkcK8_?xG*pOZJxWVBmcSuiH41HMK(3n_0qns=R{m((vGHWt~t++f-nK9 zUR4R7Y~T#`$AWzki3@o`9ukA3SG1qFdO{FX^$*m(^3^MM=g;>OzKG;R1Axh=DX3%L z);pG-p^{vrW=1wJ*BJaQH{p^@68_s7k^Tw;6xUO-3WIS z$;K$r*m_`>7V}>gi%o_jtLPV!zxNyBfCkag=L@TqH6&p@(ZvWO6)K7?zz3U{1C#@n zIH8DS=|5R8N&DNerG5za)TCOHi^8D&D85gkVH|o7JY0yszAC_UM0@sU_I1tJ@zlh^;5WLQ z=wbaKKl8A^;r9y&_BF|8rZ)bV2UCR81<{GGYUq~M!dYl#jy9!t6`Tm%b2Sw!IMU)i7ZxV=2P z`P&Rd;rBB+5QJ4o3g~5mgYIQH+qxG4R$2uF#ib2%NcA zlKI6$aWw%->;i?eUjW-gkFJ3Q@hXaD7O1((go-osvNPd8#RBvVHk(GzRRDKCOeXB> z58LC?DIA0+kFoiC{h^LTayPR(zi}7e_%A_{PVc3%F~zECxm3)*QRgQ?eoV39qg)lq zH!#*+2*D6?+z&5`aMOqM@5n5#a}WPPIv}z%6+?66X++;;As(r6zz$_pMCaE1Y-GRZ zCVM-n0k5JG;f2)3U4QR9ZUMnW>e31_`$HRk_hkKFAQVIS4 z=!M`q@)R_ZL_Pb(=!mnSzi8$6;<_-#NB6)KFSj4NMu~W$g7+lmuMOaUtXM}$yaEqj z5H6rq%w(JW{Jq{_apjO%(7N^-V9SYZgLm6q(SAc{j|?7{!aG@ZRlJ&2@>J-;w1*6Q z6o|?f7vj04-zpwj2XNuM^I!F-5rX((fSb||{7c8=IrAP3nP4`7uz#-T(*0swY($&lA|4mGp)U*Y6^YN)pxsYj(p2?S=WKSS)E?MI}Z^b z(@tT^W4ltGAMT=3raN1GaoWSq?w5!2*nJ%M?}$A1=V!iB$HA?Ofbm%fUyEd~-*W6H zci`R(bUamo<*YxDFodk(^YB>PY4CU?_^PAfx!+9%SN)`e#I6vj0coDZso*mLWDm8q z68@LMfb3&@xwo*sjOSyQN#FPSO(2-U%`|S`C(aANv191>@aNAy=LiDfGjW%Kpu8e? zakmKmrSz;m?nb~vKXYxSQ^HDm6-Xkw$33{M3!%S+ej^R&Mq#I9zx??AAwU)tJ@)T2 z^y(x!#lRsnb+GKG7W#AcIY+T$n=na@f=4Apb}24H4fdP;jg$_oP?{%xGNv-1(9<&= z4)^p0z#I90^8zFRkys=B@u2?e633_9I^hM#>rQU{{@qqr5E+meFKvXtd*TxRUTZFw z^SdrzkC6&+*XfaqLMZUljYb1`uKun`ebAt-X;B;bd$6rO0ATPbqj60i@LK7k6>a!c zT|iIWG$)Z%7WRk=bg?bnl(K`vvo(kGd4JsJ*~m@a!TACVk5Ww@eVQve{d*FE-ylLS z6o-OJx~zO;s9%IHepJ5OcC2&Mc-C^#7u?{*DZPs4c4z5+A6)!uj&kBhgi#GvrRDxy3i;b%Pz67b^NA*Wph3)Mu-qWM@p8RP>w5tFokoOt zc%r&-&Je!7G)8vX4mt2q43JQ$8IaFjew&mJjkA-dqMlwmytnryoLDvB72es8Cf);- z^n)SCkhXWG7T(`^L+`F8-xmHCWY+XjHD*^-Qo45wpZq)CD1GTYF{?OpCCqdo(IPS&D!ZPin@g?K}Fr7hagzxO>wn2){dA%O_t5`)E z>ZDY!TJCMrqgAmO{o(!~A;D6?Kk}>1rQ}MF7tem|!1HxF+Ft|lpZ+Xj9Fvc60|5V4 z)l2?*-aTT_s?RG|%h`S*Hd=>U&V&UFpYYjvic==aMmuhp!c~&mj_>({^BUrVfcI_Z z$CasYAhv%ksE}RF3LftsW?!yfvk_j-8aU@eDDD*G!f~mEg_84O$`P_FaFQHd{CoXA zI3_F9E~!g73&!t~rv$xo>BN8EJ-Bcuf>3oyQ2hxn|r3}_7LxTS#{E!VfJjsm%%FUV zhDm}8k5z`tbMYh?UhCo)i~zPuqSXb)_hP3e4G=rEFSLHzMPlG7Z82oj(dXb=F3+`Z zXANV)9m>~tP}&I0kW2*D}l z_A0gcqkkhaJ_wEu`iS8A)gB|6UI@C76eopj+#C@>3B>Cc?2KH*gb=-iR6DB%TrjX- zc3qz_re_OVCigOcyHoE>zxJ$cRGAy>`av=k;#4{vA3KW~A3(oVw-a|GvCL4!uC@+2xA%r@4>Ff+eec z2D35Nz*ex{Yq{<`VcLq|f*sakFv00yp|P9_@~sZDvCtmRi^Y7-0O8wCG%|8wVoe{mC29cj@$ryisWy-#gLgW+u(d*vI1& zJ>aId1v#Nps_{sh0<|95f4bH|1YiZ)_#&0~UX~YB0{gazJAgG=Y)kEl5CN8CYADLX)K-yd`^oHvrpZz6DlO7d`K~lR$n~O--Udgf6poD zwG>d^2`X&dp8E09X?VL1OF#96`#;b@fZ7k~cR~#+`(+EbKSLP?PZbEKA|2gsUX}jl zDj-E3^0AO^1wZ!3)${V|=4$^%RFB_Z*edXLq_q>1oE<6dUT-JcYN8Im z<&WkLz8QD&Xq3gPVNZZ3-XUYVPeEE4T{HgfG=zv{!CxeUIm;h6de4s+8g760`O9f_ z;{w6LcN@-~5daQ#`z=3@`_&z_p8xVwytpSCeglfIuT+EW6bA*igxMnXaQ7g{*SWF27x=5s^4PL8;rnC0AdAV?Dc@_qP$5m4OzL@*4@Rhh8(vHI zb!OPRKfX$HRy+IB)$mQ1PLJ2&!- z2))rG8(vWB0lY7yyzI4D(B*HhRS2@439_9Jv2959q&TjKe0{GxyZH|+Tg{6|=nod@ zg%1_`to@bJb{RvQ6YY=qt8E^qz(Yur8H7H#J?JSQ)RFxUnFs(#P?Kc!Khi018PuGcP%@Ei~z9 z$qJ=R#4$0V?KGdnll&Q_MSp_(+oDapr(J`$gs55`<4(=V0P54QK=g(22t+!$&y;qr$Eb*X%@1^L9J`8voTKEDQ{Hq z72TA5lNX-!Qs-!@6Ra4VR#Lp&iOi^Z+IX3;rQoAR*FvPk!KlLvb{029+hoc$eB_5v z(M@S7rm9@cEIAlMP};)sLKNma)4k&q9neCe^3{+%CkeKvz)Mysg{4?(UQ#OMTtvx9 zoT)kKdS-W7v8kJ7;!BLzR63Zs=$GD6H4dw+)4%`=NrG%ui5ZukFFMBaD-T<#IJKoG zS&S=ScBIR)TN~Myx*m5F$AciaH1ECe>P~c`DAXP{m87&NtgLN|weVG`hbt935OmVX zHSXu4sKgwY7IZq51eD9B4tpt;Ob=qXv|X?y7B*U!RyMRLV%J$1LTh=hZ^PG&DX5Yn zcS%T$oSZHRF<&#R>tXL_thc%zg~lj*o`cM^EaFVta+Z(nG>l>*QrNa z6*kS0HJ?{k7kZqvM)jI(0T4<5v1@yxb4wRXAy`TlRTNshw| z%c%=f*m*RqEw4H?ON^|BB8#QvI2EjsYSf(+NNJTV3EgaaPT5&8#hBB+wke;5+A5l) zb0+tztAtAwrVV;EGeaRsl{S0MqqUHX4A;ikTU^)DCz67PwCkcFdQGTo000wr0RTxS zL2AZc5~*Kx6!SDBbWvrrCdN>Lqe7t%Go?%_3Q?ruxoK7BcSh^8jYX*PJ0Mvltu>1aJxOLAibfY>5^Npzs}EQ` zZklpr?5C1TDgnc)S_$y^&0G^%uD$m1{Drv^7j^f8vuf%<+ z=HoSpLG%gb0KLk`rZZK9$1D)2GCio{E*~g%BkheAJW&|(OFX-qWsz(`c;6sk)KNUn z0GklTu>DmNk_L?B70y^28Y8+KT&3I&&Gk}DB0*_~h5_j|#GWr=5Jj3KZIY-sF0H`w zeHemmk2Z9KaSjP9KJK-&HmIf#3yB9`j`=MGL{56b+gu-vh+L-t-d$LFM{eRA_(5Tm zILG?$c|~p}x)vcKQkrroLJgES(C;B*%hnZ1joT;jz^+u%ZI1{Y3|k%ZKYH>bgAt5sZ@+TOx%(# zgy?pSCff<3!`x88q?XG8E?doO#Czo2hK5(RBmvtE3?kAFxDzEalIcHZkm$rFsaq%m z`y)55t!b!&Ftw7wL6rlDO2gpu6dR&jRA`oSqc!^q*-*l4zQt0AN6$zZ*h~a3O_(GW_2G`5{i;UMGT@btSP{V*seRlh_ENL6TRF#|XR%(4tI4 zNSa-1+!d3#T$;~HltJxzoH-P75TtC2mhl`@3v(F}q@XYlf`xNTL+s6>WXUrR@!lXP z7)DM?9wdiJmqW9jL%`8hSxHF=2(Z2kRjFK?P6($f){F&;v1$o%`QfY(n;$}CnsDzu z??4uzF;Tm34v6ehq?l<2b{GQiM?^;;128p6VIUAM zU?CD~zPJwn*3t?|W>H-(*W5Z-qJa=mz~wCW>_4W+He3bqBF907HB{+Dm}5c`!1es& z20|@osa9T-$SRpvWP)5Q$u=hA@Ptg6Od@T-gE9lugc_t0RsRI!jzmsqhOn25L6vL~ zW+q}@SXeWt9`405HQ1hyNJw=n0#H@HMSfz{K`w=o&NQ$9P~48_ia$@DDFD_YC5$nd z57OpH8+ZD))not`CyDN}7xo2FAnY*UMf&9%*g%{vi6S`Wh()rd z981PkZgoS(!z`E)1T6^&1S1MZnzh{I&1CI5Qar^RIbC45D&aaG#k80@;9IW>QW;wX z*bD|nm_bIROf4VTN)JIHkt^s`$Onh;%L<^+VN*>Xx3anc!5cqiu*pri=x%n6(>`i0 zSq96c;9HHtFob|Pbh4HzMb286nNG;~w5TXNocaDpYJAv>Ru0ijyZ&lL{qrc%5G*4+ zikL9K>6xOW=AU^YS#h}4LuGuST})kbXjw>>wDc*}EK6|8@gFzI0BF{b#T0Jf-Y=A! zCmTs2194?Jnpl)Tp~4eVQ!t1$_^O`=v7IcVbz(sl^uivjW}35*Z%I;;(1|(!H8N$U zgOj$8B}mo|2Uv5tFvdj)>lX=5xbUUsV=$vo$xI=rr65EM)x-k8PT)`yj;SoU;-cy| za*U5E_Z){poM2MK5H~7y>HC*2sv8O$z2wW3Q14DvA(%)%g(IJDCV z1PCn&I19mvsDjV}u@wjL^+CgbG^LZv5vt7gK?j{J01Ku~4lO7GJru2a5;5a^F-C~IxZ#TwxEGR0)* z0180Ippir`idd16;j9In_8gE6IAy7kANtc<9UXr|ug5V0j&5=1&54DXN$P^$-Db*lwO~0t}`!X9K%oWkJavAr<3REBv ze?>kqup%66&ddKb;2_!$iHs)*0fC3D9iSgp{B37QyZ~&~MlWEl;Nc2qPAuuipCGtG z;S;o%>ll8&l2V3BgV00d)WH`A*!EYd)oP-3)4Kx9vqk{`+W&}{C# zp+!|_)LIgzEgV^JYa;Lf7fJt_UZEiDJRsW@$x%}p;&^Of7s${uq`FY-JuGw1c#B}U zUglHgwH9VEXqn;_O6hfBvt*Wpx)zivE0wRUA2i+=G!*;dR#~As*vqae)lpMn*n7oG zt*=L67_qG*v9yI+cS+DY7Y5L#p_X+-u~TOkNh?I73MANhW_!j8q@t%%XHk=KufLR|Re$xymn%*ES+4siQVR)1lWyyy{xylU0wBlJ&)LMHpAH#VhdJ zw&8j= zvCDc(7B-G6=yR0QX=+qClVVo3qSq|(b|BZGQipPhBfgh2Uq(WeLZ)c0ST!XTp6YSRw5~?0uUfSdR4}PNs>2}2;nk_B zRH9`~Ty)_WY|#^-(1dE_EU-PW$E|L(u#1_Jovh}_?L2FENv1tQ9cerqeqQTvG+AEcq%GjHMKoO+1Rl+ONbK%9P5Q zW?2=IT+3>!4`nb_P3klhQq?M)jWoeCrCT=M_M=fpNv*umV?Ibu4u?IBgKFE-hL#|C zC6l#mi7EAo%?RR1V%J5~)I(83$wTk^lwgV_}tlDclo)%uB-=(eGtgfaY#Q0A0wwXQt*bzM|9wsOhWy=pUa3 zV7ThHgJ{YSrguVr&oVvCH^6DL+9POk3<04HX78H=+#>^&pdr;Rf0WF8(^-p(-NAsm z^hp4u%vk`9Ih~j%UJcu747mMyzLX<+(bow057zNu0m1dZ4814?S~@pC8+HiZ6dFKp zCqrG{1R43toW7hno`C(%;BE15`ulhzosxUD0q^60AFFV15d;LJ(@t>%aTHz)5wpBo92b!E$^qvdK2DGUG#SI0 zjSzDpA_%ci86=?am(3=aQYgNAjLtA%RsgE53mW02Rfdrw6bh=L1$4+*y=H`5~w%rkp1m9`!PKtUXa_`*yE)}FJun%0uM z0ZQT189neOU_Bt=q=uuNzg%|dy|q8;2yB|mQ6NYhFudK&r7ytNdIIcFwV;xZ`?|sY z51dJnoC5t^2xPr%x3PW716__DLC6or$}8iK0m2JB*d{krEpQXWf5sKB!3y#CEq2VE z{W&{ieM7vu*fSGOQ}}`STuG{LR3JLaGVsY&ePc*ghe4{BE{_4LTe0PkiB$)HW5_g} zCMoE2$dN26etK`kgBBF)4c;IGpAz-N)DEchia)8Xo{LFU+Z=w`lR_;e!eo5FhfpAc z&E3-;i99D$%;PTp3WUEV&1#Bld21V8i)Z#9w$(a$5K2+}q0kKd@2-Q4Z2A+LFUO-( ztb@J%GW3U_r9eGO-}JzE=5r3M=Wn)wwe(a7%lrJGbp%)wn5VEIe;7LPLE884{$1%2 zwIqZvz>zOtd;pAYLdyei4$%S=hnzFK3&P4jDmT8^s+yCc4{t~>u0)LR>A}6ZXJZNA zz`R0Ce-Vx8FgZNpng##!jF*j6pY|Q+aD1e;iteKpc+D`HX{6E z?LVhhf?vMOGa`S~;#G4YDxyjdQ2a>!`}&0+`%4imE#C9h`wgs6UnTqU9`6ofI{c9d znm4deqTR>HUL(04(O^AAqWwJ^{KOck>znVdo`9gfgMmEVFPEj6rM;U6!yKoY=a|Z^l2sWX7_7pAqL}$nz zKj(sqioLCu+_WZ77!BMAxC*Ku8a94?qe>~(a&6w~8`TKIusR0zx)N`!AFy1JjC0bz z{-`MOYy5Tr&}L;qsLqD!1)AuBI+Ce^zQo=srL`iac2$9X4(gH@@g0pn@f%KtCX=sJiv~2U z%6s7W&UoHgR@fgnc+GDAKp3sCX~)6ixJ+$VRO+ecXx%y4C|p zxGWClW73N)+#^*k31YHg$kXlkmA~Kil_$TA1N!4|(I6huD3&#a>D{r}z(B&w5zP3y z=Q(DeJU32UcUlQMFc+}BW!q?fcqKP-CnM$-J4EJU%)i{-!D_0OKehNVXWd%T6bolF zFlZ9}3U?4XSZNEmdvX{7nL98Mgvl2ooE`G`%L-;HSfdivg$zJQoM4 z3Y}qxOQ4bZN>X>mc|2e4=4U@}y%L3HFQb=3<9mE|5R(JqM$u7>JEP_MZ+0IE?sJFH zAE(>ZE~=v%u9AoFi}Ufyg741}0^5ZBu~enwfk%D^S|X`pjl}r6-0WK-CzmDc(^cu! zL zC&TAi^{*cRzX~(7RHdX27&*`MMi6=r{Yf1Ntmw_P)Tp7b6oS;D4bg9{0YaceheR}n z0(Q&~f)@S0;A^{?1Q+Ql{=P;w_#ai~YTUjsp|>c*Gn=RcCWd}_*)PH&bRz}VH0pt# zNKojlj_Wc8KqEU%fx`SqhPla6AQO|XcrIdrQn{E#oe*DKAAa^n+NzTgedCAh}6_VsJ8VbnU_YpvS zT8oF$5P91^MhITdB`bMNZdZgf0f3t@!2C`X_gO|-W%NXl^0i!SlCAm6A*3)VehDHP z9~K8G!#DXu%JqEgGPGE$lZq)0yPQX_sW~Fm_e8gbW`{sGkO1<0aGE#d4aX8Pq`lcf z=BnxSR&m1*S>QJo5(p74GNX;j%KzR}G$e9TiAOKfLl`^;q9Oa$Z1}!5OtE~uk*ttbgi(A@F9O8}KN+WPRsMvO1p0C%aQV|6 zl5lP)5xrD`z*6$CeNt9-_5^LqMt>a>P;wJ%4mKxPqZ}==2gh4dP>vb22TVB!k|C>v z5t2!|!8+wiIveY^tD|x;i1(q~CaC+q-IEKFSCk4yEY1dsYEdou;OEbQDk4A64;57V zDXvLPQOMGGgQp30553p(a}iHHq7IU6-<(+JLRt!S`b?!R@)2st{jaycR~!v8s2~67 z)TRM?CVeB!`u-N87_(u8mq$vY@{^1D_&O+={9A6&Bo_BIb05wL9}Ho7Becq?U+wfR zU_*@p{5=bJ7GGVZH7x|wRBjDJgn8V2f|gbqOWw^3v>{}L+MviBllnlZ@IJIvVA$ZC93GFa0e3UhzhpgIWqc&yU&yN z@;t3$`MwBabC}mQKg;-+>lg>vMo2ls|j}Vj&+%u1Mh$p+WJC{^)3B0QG`SuTWI!iL+HmYe&{*VOSjyCUCJCE zKfo`~(y+z?vsgShB%jZlDr_+dc4gmKVK+c0bJ0Y}?(7aLP$_L)v^4&3mChw+*&_|D zt<s=sWki_y`aYfns)o$!_yTK=FD?lb`4-;NT$DI<*{@DEP3b7axw2 zNP}T79TY>?H+HFZ)0<;|{raz$vxwd5#3AKx@)(l+`uzbzvWnuY3~DWxt#2|w4VYRNylyf{gM0OU^gNiZkj=ld-wX_ z4tJyuVb|VdxU${Pu$J{NxnOvhJU|EHVf_T-bFY8x^b{+QvgW-93ESZx5P#WUi@hXc zpy&(CqvPEofCO#ap^~lq2V!tzJdxy%ls5J%B&>2aeaF{11K=;=IL8R#k^##r0#v;-o=v z+>Va7q(PmYFD4pu>ze5UZwp1PS|T-4|29)(=cMKz-hD23U8M>}0`k+6vIQn{mK(tRm_JQKe5h{Hmu9{^TaOT|tsfjUF*A-+su5pd zTeP5&9jn}~{fz`w1(r+0*C=EBFL9?z=W34}{Fb%;CwqhM*x^q(^lYKrtgyxZauf4i ztQ`hD-%ZqHoIQt_?gr3eY3ht01A+_t8@~86z3eW(Jk2OxzA6ZmNjo5 zF9FWWE?@?}vK8L~KG6u>>T>>#=PZ@qaR+7cvbh@tA|FtJgb6eShVVivw17YW*0;w1 zwikiVRU|4gK0BXb4qEwe6~-c?-if1DDg-68Z0kl4&LLEXivpM{#})k(TmzmUO~*i9 zP$}_9k$vF{(wU{83_WiFDD#hJSU_GTfR`6>#*3f0)D(ksym{!9pqI z5avK;o37M^`_huW8u|U1mJphIfqu1M>dwDI2yuNJ>+&bY%+5^?E-M#GK={o(Uc((@ zZ?`j{oVdd2002*WY`8-8x~tSz6*rt%I8Zu4r&1BxiUQ%vsUVwEMBpZnNO*{@&qhg% z>CtCzq|((zy-9r^3segu20hrKU)g%m;Yri;yR5^wX5kG9PgAEj@#$U5G2#-nSm2vTMKwZgL56vdRaLsu2uz{j#jf{ILGO0!$v^*}WvQzvBvoh@HazaOmK>k#2kWP)5=udBck&;v1 zD}%#cBzUBsjtzh0FA3d483A12u*aF4#FxkxK$%8Cd8oW4t143Vac?|7P#nq}Bk000 zMtmsJsZ*5s1`&e$t@%?vf3Vlr2)#S1tcbztjn_PTH_qV&$6ekV6zQ6vq|iN(&LIA? z-$XfA#d@odH|VrVxl&}{;}sY1jt3PgTNP?q0{28kyz0NA+6)fuT%|~C{u+Lv!&KR& zB3DKD=F)OhK|Hi)D#wo63Ra7G*|l|wQ`~1nefr-{u2cK{aaO`Vb#8d4NVA(?ys1V(ggegH zm@<}rj9F+DUMU6SvxrFvpk}L)H*sk-O!7*qg}zT0_=lQ{p(Pr>41es?Tc+f$0J+&r zw3>NbIt+j{Mc@eI#HIuDEFF;D(BVOWCR>@`(4iw!RVuDK*KvjyoU2sPJ^BXm&t0gEuDZ&awV(f3AD8;ZVRO~BVTw6H$`9VE z_xzo8H*{a3;*=_vzpJ8qdMT-v5rc?(m7IoNO8ypUMf-^-50yP3WpW$!rj0{~#4BX_ zjF?3#O(6K3sUAc~)7PF+a8XbQYyD_nmX8tx08{ByT)ktJAgB-n4B`FDn%>c z0AQ#x!qF*B3brdW;%mNK!;?>0A_B%ten&=fO2Pet+{iE?BfBM|D;qE}a@x8RG79tK z%l?9Iqp48|o~m8{jG?xtjZ?TW<6Tp6but~7;nm(urW$iv6XtHf_$BZD&p27onX6>& zry%g=rm+NO6Eg;juZItbA&jl4rz{`RWL1bMJ`@H_mj-Ms4%HOR)VAG`4F76O{lQDy z%3{-q(gHe~mSU$6*!0PPA>(zsn*mG~e(liXPtW{Or)Dj5#&jw(*=!!%C&oM0PC;(w zD>YvlJ78Cv@3_o-Bv^?%z4;#6LYeu2;jjG(LWyQ>SDutTWIyqJVO zYExK54p&Xo8!EspwCIfzORGg@=~Cj1tdM-zn9gEZH^g2?@`e4t$Fq2K3Av6swtRAh zrY1?2kje^oc)h|y!GF80HIoazzta#_G=#v!U&o0Uw(?XC{!||SA20fs*>hk?EF>!EK4i)unou`&&JR9o69UoTKB_~beOme zjNVwqOuf3qN^09SoZTS-{d#|1lZ&1a+s5#=4xT>mfBH?4$M)sLo7?0s)%l-IL)Z<{ zs?7V=r7Y61c;u~Wvw?&ayY*N%)nIPNJ2$)3ifr9OgmJ71W)!v~wLt>R4L=!Lh<$+| z7~gFJH?85k?Y1DF*cKNnzS%E49B$O7?n*adUAHT2N@8fqWpNlguj69CY4u3>B;0B&&$U zxE75QePf4GR%gZh817)HvI!R8%r{3K97y;bZ_ejW$|cNa`2IMu4;jb;rV9FrA5s5u zSaYo>M-63Ng17GIwJ~)3b0yL&Dz8%V8*mIUR)KSItBCo;V-Q7I`n=)t>EyGD2xlpx z^`F=~=TU;SyX1vv^TL@}&`80me~cuU-D5*w=SzJEIJh(_pbo=wZQP%FC=>u zmoWb(k03RCe;C;_&3)CFTPz7%sK9-jQteifK8`yR$*1n7MD6#&%xu00aQWbKH`fD%3+VNRd4nCHgor3d`cVe^5QtRdfQETqUA zjJz)*-kc``K8geZM_z}fk3BUebGMvej9#|S88G*wkV$?`k#wl`svBZ;deAzWs`#34 zIh6VK>b}0Fz~MU|@`zO8$8328y<4k9^XRs~tgQT6h)45e)?;3NcYhLq_zRlC@c}a$ zN>#M{9Z4-l?AYoT*#Quk<^G=dZrJ84tg0WpkWnVA#k;D3;BkK0bl|$&p8fz*yhCyVA0dTzLu1hJdy^ic^M2o(X9x7?9Ysw z4}jeN1r`P2??v(cO1*8sCMJ3W6}5>F)b^C&EiWTf8p)$bG+?R!ei+|qTPOZIqBRT} zDn5KHH+%<3swOq~}$ z^dLrpcSwkOsbem29mb*^w5@kqeq9l&x8wS1v{*a^z`^S2Erc|d9SvQg$7VeS$@YFn zdW+)ty8Lce0C0SB4Jb#32_;O5q>)*NT}T?feqG*_vk64p0_efi<9?~cZ=PJ(F#@%B z$t7Q3XZ%NMo2c=xRFT8(!Q(243*Oh;_WBP(F902w$+~k&7=>@C`l{(hNJJ9780veX z=kAaH4Dk^8nzhy|9~pW+7_lg{5%4-dTS=6Sjh@V+5=q+|u!*VTd8i!~{Ecb5s8872 z+@_=}ms}x+N9`9-Z5{>e;o4d?3znX#kxcFA*muGz3Zfm~6^y2kE=afUe}KMh#5$k@ z-1Z0KZ;c82@m;xMKhsX|A0O4I2yNr}Vi94$ZVok9;ZW-laYrz1e?8(3 z00!Jo#&$=}X|w&ro88Y*c0_^6h@$!9+;LAG_jWO@-IftXeh| z8&UvWLh(nqw6gY0SWrAE1AhRopBcFHcY_2oDkB)*-M>4${Q|cb5EiZ=I8NXud9DgW z{F6)tn_{+)XBS6+xnZEEh0DhWtCxIog_yz5EgV6KtFb~XtJ|8-gM24iuItqS56>3R2M*igzW%_S ztd0+mOi~|sI47J!$N~d7M7rpKEtu^vln|-tA~76AmOTq9Gii9-*##y_Tuy3y~Bs_DH0#*Ykpoxr~hvL z{0#o7evz}y1#R|oO-vK>DcxIfN4Z~-ci2{@-;6rb|H|(D_ZwP5;mv*sH&6Z6DYPi2 zcsMlfkKV`C12O?V$;J^l@aK=mV{`-TUp;;ZfY^d*^t71pSMsWV)ZaqJ2Q&ugK3P&g zR|L*GyK9BBot6_YhF`Iki=O>zjnn~~TUA{Y?6vG(={X5FRiIvfxR83iwxWq_I19MC zdqA~ZvP;iLz<(@(<1^5QyOBJRBSpc7Ka;>(6mru_R0YebXLS&wKQkDDL=+y1EX2JE zZlnT%-Lol3PHJ0E-C@@Z%7~wVrF^B}I7X0v@1BSQEvsnCAMtb};HA?CfN-Y)Zyl_LZ+4i79e8FM_j0?F zuIq65HY2t!@xXs5{|w(XD><-DX?Ex{1nl`WZS$~*%Jq!lDFhi#U>d?{FW2tD4tTa9 zk9Xr+gfc9thy-d1W!EEkB^;xKy($|`XCe`k0Tc0fi{J76tIaaV&XL(ZXmJ5yF?bz^78lm5QW$eCH{kYW(;Uc0rn2NYB_BM=sdO zi%$h;1mp07N6Je=6*szE=!uzpZn^PBn8pbjog2eI&S3)XN^d}ZnOIzD(7TkUZ5J#4 zu~Us#9hc;5#RUVmZjD!K+t(rOsthsf&G4GX2ei+}8L9DcWS|8JfE|seKa^m0bnM4O z=O>ivlqOPir_GSAZHvGYB`oKs_4$)L={W+;u^?fypNOl!=yF_XshV=jz8&pwq1_Vk zb(a7WtlLc}gXAv)HAfv5QF2qYCrKR!=-O`)o18>tKw-z%p|1(9iR>B_x`}QXHGV%# zGiB1F(Xd%tnzrM_!jXXz$2YOj!?H<2)GPBhsRVvp;yL#GqvhbplQw&|0~wHnlR^=;B0oK$~cet%MUg}uH=;d4KUk{&5c;v8ba(@XU* z>b3XL1CYYKFePh9ZzM5)~fbmVynbcg5u=R4ZjBL%nA ziElV{)jv+{FS?0Hs=gkQ;mXr!IIb*Ur#Ti{xI@>j51Z(@ebYTb(&h>~c}Us1`AU2(t{6q*F*wtI(j<1iCWisW}uCk*3E%*EiL@H&i{M8J!~bWo?^hOn&DKL8 zd`t&K9fLbn~+uUTu5ju!ZtAarVh6=YOam!rhKTwhBynPXBn80ARVo$psT>+`fCxduAP z5zVVIU!Zykme&FGfXNnafS>o)$_@q9fL5GVyu)RnH_CO}HiuRzX)ZbRrF z{7Ps2XXH0`hsH5Q(i^v7SX}#oed%I=D_mt~!8iebOc@KD!e7BX@<&tng~gxE@kjq6 zaQT&6^#F-N9;LOpu4VCS*Uv)3kE8}X8;ckxj2Gp3@s6}@7n4MBAQvl}=q1?BvpBpI zmQaTmeVIjn^U)hNg#R@3O0lDtG%~=0fVTYOU>^`6W_iPuLaWrohk5MjD%dUc&;3Ih z0RYruc*)#UK7W>qOEY4ypGTk)>@eIhQI_9dc?>|W%GA`xN6pQcUqXc+We;MphJu!( z!iwLQxaOt%PCzzF%GsddbSPDbJ$bo2l(M5-*RMJAoQPYEQAv>W)lj*O+(E^=haj=a z9kyV?bh#T8%lRsMS-Bfjkr=Uq4;n8EDpbY0Qas*dH3u9dBmsQ|7-pm9IHbN&3r%z6 zA(J{4JgobZJ5kRN&erbx0mPXr##s6l!de4qa~>oY;#FO-na*!4={opSGQ1-gYSlBT zFy8-$+m#5Z&E$x~u1s?tvk8C)b;D)VlP#`Wy(f_wHytIpN_Z1BcxF)krd z!{ZXCx&|J_sD>bdl+qI+tAngY*TP>bIOMT*5}%F5Rp49wy7DomIc{EIT$9g83KYR* zS3VG*Li^(_rgKdt1}k1)Y^_C7P_rt%$hkly8P{9QJxxn20>TVul3EM8$If0I$Dr3L z*sgU!zZFJxZFs`mRBZ=u#sVSgciR2?8Ud6?0&$n2nfY}*XAGG6L>ezlA*^Fi*Q+9) zQUm&#zyD4mg8T+vLESfyP_062Qy+1Ignm3}$r2tmtY)UfnK|?$bCsbH7+uk$5rnIK zP)975&GCZ8y^*ht*S}%cXxDF>PpV+7H4l{}x6M0|2Pnu4Ux^zBGXQ%?Lp1f5yrzB> zEnDN`8UY@9#NOczwoJrN(tM$Q6uMag_$GY5viPMNtV#Dj@*s1x|- zH8m!MK0<(-w!CKj0-4PrX9=67xs)IIxheqPf21Y=R?P+>WoA9Q@=bTe4>sG!BMxtY zz_Q#zm3)kskxRtntk+sZzSG1=&J_IrcB6ex~95&k7qLoA$p2>@9 zk^gE3xy5ggo%GQX3==Z4f_0$f49L#i{+Xz~k>vHLdIS90I4%!3D~M`Vf2rMYUzaw{ z;(M*7+w0pVqQoQMJ)fwqLk7d5&fBI+CfGrz5anTVfj>{tA>V3ohokFVc+e573S=Iy zQbNy;lvk06)R9xYTUIs@A68hKIQHg_Y8nL>%XI8^e_`p#E-C>(>=4>5NakI-ux2r6fKg>pQzCfc^I9QE#2|?GPUQ`Yzl2L%Lv# zbM*?m!-oj^X+`37BW=)<51`w<>E3CkL;+G|y;9zFh! zh1>6IDr@qddzM~3y(s84Gn9zAR(i3)M;zhNg%KFredPu1`!^pr?jL>28mxLt{~<2< zOb@}>4+Y^>^ykqB!UtoY7^(lak7jnI|4XH;K9g1;sb2h0o+?=TOK*(a?wiZo#nSg_ zUs!U$#c)+yljENi$PPv0eh@(NMUe+seD>jv9+G0z+Hfm{k+M!75GT<%pm*hM-{v=H@p}oD)g3}?wi?U00O_*Z*%u{@HAXP&pHLoiVRd}HsRsMxCQ~}Pg(21E`Umu@tb0*7D&YhJ^;%e#3CwD9`c+VdCpo%L)jC%t-L zDK~xT$`20)kgFD%ENxE2c#*P}8FN{?3ERO|v|`S`Zf#j99p~Q2TfmG346_d}wCK_M z-Zfhx;p(Hw<+SCLhvWUG<^X`#_(*~xFNS9*~ zFXgT=Dqw*2MM{`iF%8^gB6x9MRAsa2^5(o5m!zHw;7fwX)!$O@nxLm~<7v*A)oT*n zJ!@&#R!nfZm%m}@V$V37Wk_-jj`EniuF9HC3g|LktBTV;cPUzCPG01uqh;gOS&~RY z%h@c1;=;NJ6|Y#5&XJKt8meM1n?r@Q$wrzFw+NQ(fUX^`Q*jwo_QKz3H+t+ zV_Ch0)v61tHJR6ZTT_OXH^^D$uc^sBKWQJsnxm~kND$8``%cx>QdfZ>e)W?i%0xz-84to z(7Rb#2oJhsbsj|9p)X~VqhsEI3fKg0@cU+QFQG1_wd0^V5b=4iK5$q#e1g@Rv+m0$3S^^0j8fKUv~P*Fy^W??KC#X zE`J98hR=K1XaM(SKod@$J`C>dCjk!Sn9!AU(gms&Tp0&Ftw0%-W zN5n!I6alsO)@0u1zW%m`i@ifv1{>A*1<<_riVH=5j}+68YkB|i-w!%%$0M`i9|j)k zwR$svG4m&iJ-egm$$?c4o|JQ7c|i~r1h+_%uLWZ(r81fv`rRka_*^GXZ{ zSW5s!?!y)Z-KNJ<5aV$#7xg2o5 z^S|@)6jpfLIH+F(dUA zc=#$>SAID0Ew2CUh(vjhW#1Y3y~r($DRf8o`f0 zn|#NYoGUupi3QD}+ zp67*&vFiQe^FwoewFD+1Zmt7|2lREy z{H3xI>C!a<{8VLL0>9|;VME}6=L4AwtDKsV^TEC{iqLi2ZNB5i5b!jrPf{M#%oSHn zlUKS*ZgJYy<8?YD!)Gcdk52;QDsP!#zlx3H2en*004jg z8UK9%!~y_SlU-_qKoG|pf&?oPBm^rGdYQiDLb+1NG4k3=1PLKXksu|A=wfGQen$H+ z-C?@*fsyppnB$Y}nrM5DA${9-S=wms5?9@A-Qi05_QfKKmhUMqIqu+q# z@pCyZQg@rVmGq@lYaDYP#T~5t@7zNAoALJ#up=wI_iJ;nw{Ph2;5k&2=S{#|npZzl z+#8ympMS`o_2Gy$2W^GS4El_CR@jM=(E-;%6(YsUD!|FG--6$mG_|y3B@VuHX5Cv@ zV{buBH8xtR1f%3$V7gRC%$go}dlKJ;h=-Y$khSg572v7EZw~jk!Y%wnsjFY3Ij67i zTTU8YIVNZ*;7Gx|+e~#V{`=8Pn`x!5XIAn%)~nbFx)bgeWZF&{sGr*`i+ZNF;%-CV zkYB-@fYa+6KOxmWeg+9S#O!oH-@r4z@+R)&x8;B1r_q2P@rIJ~(6z_!_qXx<10@3h zSno3_-+d|$(~8&hLbF{IZX;sqU{KPFn{}*Gge+WDezJ65 z-Tfaa2XTr8=_;2N)uUZ9Zd*z_nb<1p6?dWCB`g_7no?PAuD1%aEl;Y`L`-AS^0al= zxb+otQF;n4*TT~G$+Cr>sKawQ7U*iNl}@Z(A?xu~Iu+2y-&8BQZ76IFUYdYfs>1Z@ zp@nRyXN&rp-2Uv92cl+TYKLlix8EAJmTsdpQt0_}dcxgatk-F=#WX8dLCPy+cIMYg zuSJwB!EM_+)%t^Z_P;#JnsxfIM_I84@eGamTR=cjy6FJv+ zwp|)4pF!NBR0fC^oUu*QsIfqtrc@S!8UMVcD`_8UnjX-T57cs?mKSK}M2oJ`R{9nK ztMw%)Eq&YO+jfB2cdJc_?T1`?0#G8Alo_Fp!TGRm zBlHvX^*l{eZ!vv4FR=+>6GX%o5ph(~$;$VyT0P7~QAr)v@n47f`{!T(e*fpsKfZP8 z>+_JlzBc_R`?2lETR#^4DEhG**PlQ7!P#Mct^4ubk7@sZ@5haE!*|_}w*Q}ube|XY zF5D@hb?(0_Xw>kWj(4A*@ape8PyPC`pBwmWSwC>Em*shd)(#mi01o@B1d`901YkTg$VVjR}U;Z)M zt)S6mncVgV08YM7`(49cg(o?^sgeCHe(FdYkMZo$wo#N-7!myjzIG+&&R zP0-)QC%!$wcZw_v_AbmvFn1h%YGd&~_|!Ssw+*p1!;Z6H>m%|)(`_RD2Xw!J!&k6H zon>-=U_0|k5oB8@&#}rcwx`&;V2^Y5zsX$?tNV;QlkFa4m=ZHBdtLI_S3G%sFH_Eg z@7Nx`c82(}GpYmRY@<^KTjL+In_}P;u!TQNmMLejv7Hl!n(^Q913h6$xzyyD=eRYR z9T=^$zQ(Tt>ks^L3;eF=LteN4IX0N4Tpy0eAj~M|hdN}kM7-3&-=bgVST;I}r%y0- z029aL`QB(qrYhx{cpC2*-#L8czg3QT!+zkY@SWH%{ zl2-{;R8?nN+ojxfbjdT4oz*#Y0$AA2Z?KcPDn*+z*L(76X|_B?hpK?eevspKdO|wOFB!#N*yx7 z23ds87icb$S9ame`3vPu#)RAXGoYKwYj#rRq-se|_nA73GUYC`17mbnURm+0x#g8!7^XDMgp8T^|N#gm*B)LnZg{+8`->xwBlEyc1|QL2DrULJ9Bs(hWiLr^8PyqpuNCuw zULvZ_y0ZW?k01}7mUqOLeAYXK63l>OzfYHFwtS@AEf)ck`!Lh}m`y^AdoCd-nqAM> z;lMss+wKsfUKNO4=N)pcze3s&b2a>5DR-Q8kWn~N-YK(gMoalYFC$(C@{c+PuRQ7y z!IJZ#6ZcN%4&N`?A0AKYbX#m8y<<14ID91U1dM(S-^YkGt1Fk3bF<=HFO3`aIrBZu zG2yS5Huwu)c+Jsk@`_!m5bIZT+ku1iIIC+kIjZD&+&fskj>*OT!V*r7$vdZK_Y3C? z0Ko6{3IJLE#BE1SnugF51zRnu)f9B=a*DuK(ttHRuyws`TY4HkQrf4KIn4d6>@h(N z#nYms)T^OQHLf2{x*~5*g?odc4a)9|ZOX+`O1ZQ)&pi%u*<>0GQClx|{IC6DE@Ni; z+Fu^&SFx;JE-nF1D*V<4yq37O4HawSO8diGCaERJ!oBsk5b7Zi#Q& z0^?SQc3-!Z*)K0pdX!YQ+(r zHC7_7)Je3>MXMP}O8wcY_BPnQWiKwHRT_p_=CgG^z7^gsg+ods-A>~*o~B(NDffA+ z4^eX06q=(JR;rOodLmrzgZM|PbqF)7Y6?Wza*TMWp|;8`OM5yPTgrf$?5dr4Eptwa zq+Eu@G@q{(dT4=;I*DGkf?1Keg$6KBRJs7*TR_%{sT~Hh2Wb0jQf{A>&!JvrZcjlx zBjjd>IZ36x#4M9UthvrLF^htv$$g=6STT^dhQ#@rWpq9qtBQ*t_o*_O}Omb(3B=;dtd zu~{C(Rd%7iKd*62njF?H>2%bxwXH6{Wg1o1wxMCEA zJl=lkcMYW#PG}3+y6RI}rc{fd4F`3@Sz7Af4$XHNv=xtRZB|=GQ3nZmOSB!%?D=}T zsoz@B_6e#zBJYPtYa`q8iQB5GEl6lFaB7ZMI-`_bu@ydOk-7_}T27QzFRq=tYe7$I z?NfG&ETJyWZy!$T&{K-y6;+^hx2u&w+Lzq+{Fj5S%`5Av zbz5QImV>)Fugw~032kZs;KNN7TDyIHYL82b=hrH=lo_Ob$F1{NJ4V(@fW3X89xv+m zdpq1!R{B(3-i6sOV?}K`)^hk)b*P=V=#yrv5K)50x0SBNPOUdYS{+^v+;q6DeU+{o zQu;qyg2`6Nw7pMi;b7Z|*iwUe+^qGiYwK^jqT6Q6RLQx09j#$i3KY{)Co=~@bfcKNCm00etQ8XE4(>GyZ8m58Vx zl?z1NW|tMovIQEelSHgXQYPls`LE>`D37ihTv+nc^eNWOvahpod$my(g;Y$+eZlF$ zQF<-jZtye&)lD5K`Fo0qUWVp$tg`)~ZGC;av{U+k^d`Xu1UGeLY8&n*nw1-%ChzD$ zt(x9aZl;tWxcwXMV@&E8QeK$$=cVmNsZ&)Uc-=1+0Bh%cs$D!PF(uWf$^6 zWU3uvzQr`eue||voqGBgY3jnX4%P)is|8Y8Jyc%aH`~4CM5=dQ%jMBCUsjl_Ew!%^ zc`Bh?y5o1RxD8Kg_kJ1`X%At?7r>%&s&t<})D^{XhG zZdy~>s_(Q!%vGo;wA zpeJXR9St$7PGZ8)sUUo>PSq)f_XvP!D93M~6g}k@i{)9_IYSej-zg)aHbV~Fe53)Z zfoECv)~EwnhZJpz*%w z=H=#Tp+y?btIw!)g0&_*X_7=Li{D&{lG!}hc#M?A6KC^HVsM_81ThDc!6s^Y1od13 zJU`Q<>ZXZ^C+R-ib({b_XYp|`vw31{RRL3{(t{{|3v`+fOJ=- zPA~S7C{rYh(ct++6S6#K4D zQkNhKWYX_HxaS=}SR@rl&&g>3!{WG$AilHlxGT@vvyQ8XkbZ~*SEvpTpvZ=TEK$Gk zeE_Th09}ug#L0bZ3fCDmI@F~EIS$O?5tz>`vG6CG z!Xv2AqZz^iUzsH4JYrV3$*A!`P^puVgGY#mk%^gwm5rT)lZ%^2iX*usGp{l;uSAM7 zxi~j5CkMo2Nd}SZ$w`^%AO>4x>gF%u9o12GE_vjQ<25VJ!uCloV)ut7tD6*C8r z;N`gLF2uo6FHj?K{~e2!5XUulAzqHJ{}}~7{AZL{bGw_}N&=)rnB(rz{|9|zLUtqal-X(6p6_zBQU$6|Qs5FbFfcL7F-S2j z3UDzi_RaA!4^1^F6k@NeG;+^$_q9?4TBRhz#baP#Y?79p9N`=2X)MH3Sz&Az>KI~S zWbB*^^yyW1AaTvziW5j^v2Zc{(g!lHf|>pD4io$U2(<$c0T_@@A_4#mg7gCePf|k@ D00_eh literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/char_varchar_udf.txt b/sql/hive/src/test/resources/data/files/char_varchar_udf.txt new file mode 100644 index 0000000000000..570078d90e65b --- /dev/null +++ b/sql/hive/src/test/resources/data/files/char_varchar_udf.txt @@ -0,0 +1 @@ +47.3224 2923.29346 diff --git a/sql/hive/src/test/resources/data/files/datatypes.txt b/sql/hive/src/test/resources/data/files/datatypes.txt index 10daa1b11c26b..0228a27e927bb 100644 --- a/sql/hive/src/test/resources/data/files/datatypes.txt +++ b/sql/hive/src/test/resources/data/files/datatypes.txt @@ -1,3 +1,3 @@ -\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N --1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N -1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123 +\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N +-1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N\N +1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123abc123X'01FF' diff --git a/sql/hive/src/test/resources/data/files/decimal.txt b/sql/hive/src/test/resources/data/files/decimal.txt new file mode 100644 index 0000000000000..28800f5278b10 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/decimal.txt @@ -0,0 +1,18 @@ +55.33 +44.2 +435.33 +324.33 +324.33 +44.2 +55.3 +55.3 +0.0 + +66.4 +23.22 +-87.2 + +33.44 +55.3 +435.331 +-0.342 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/dept.txt b/sql/hive/src/test/resources/data/files/dept.txt new file mode 100644 index 0000000000000..292bee6ee0370 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/dept.txt @@ -0,0 +1,4 @@ +31|sales +33|engineering +34|clerical +35|marketing diff --git a/sql/hive/src/test/resources/data/files/emp.txt b/sql/hive/src/test/resources/data/files/emp.txt new file mode 100644 index 0000000000000..a0e76b90e57dc --- /dev/null +++ b/sql/hive/src/test/resources/data/files/emp.txt @@ -0,0 +1,6 @@ +Rafferty|31 +Jones|33 +Steinberg|33 +Robinson|34 +Smith|34 +John| diff --git a/sql/hive/src/test/resources/data/files/exported_table/_metadata b/sql/hive/src/test/resources/data/files/exported_table/_metadata new file mode 100644 index 0000000000000..81fbf63a54980 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/exported_table/_metadata @@ -0,0 +1 @@ +{"partitions":[],"table":"{\"1\":{\"str\":\"j1_41\"},\"2\":{\"str\":\"default\"},\"3\":{\"str\":\"johndee\"},\"4\":{\"i32\":1371900915},\"5\":{\"i32\":0},\"6\":{\"i32\":0},\"7\":{\"rec\":{\"1\":{\"lst\":[\"rec\",2,{\"1\":{\"str\":\"a\"},\"2\":{\"str\":\"string\"}},{\"1\":{\"str\":\"b\"},\"2\":{\"str\":\"int\"}}]},\"2\":{\"str\":\"hdfs://hivebase01:8020/user/hive/warehouse/j1_41\"},\"3\":{\"str\":\"org.apache.hadoop.mapred.TextInputFormat\"},\"4\":{\"str\":\"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat\"},\"5\":{\"tf\":0},\"6\":{\"i32\":-1},\"7\":{\"rec\":{\"2\":{\"str\":\"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe\"},\"3\":{\"map\":[\"str\",\"str\",2,{\"serialization.format\":\",\",\"field.delim\":\",\"}]}}},\"8\":{\"lst\":[\"str\",0]},\"9\":{\"lst\":[\"rec\",0]},\"10\":{\"map\":[\"str\",\"str\",0,{}]}}},\"8\":{\"lst\":[\"rec\",0]},\"9\":{\"map\":[\"str\",\"str\",1,{\"transient_lastDdlTime\":\"1371900931\"}]},\"12\":{\"str\":\"MANAGED_TABLE\"}}","version":"0.1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/exported_table/data/data b/sql/hive/src/test/resources/data/files/exported_table/data/data new file mode 100644 index 0000000000000..40a75acfa0016 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/exported_table/data/data @@ -0,0 +1,2 @@ +johndee,1 +burks,2 diff --git a/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt b/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt new file mode 100644 index 0000000000000..6a3906944cbd1 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/ext_test_space/folder+with space/data.txt @@ -0,0 +1,3 @@ +12 jason +13 steven +15 joe \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/futurama_episodes.avro b/sql/hive/src/test/resources/data/files/futurama_episodes.avro new file mode 100644 index 0000000000000000000000000000000000000000..c08b97b46e3f1334087b6563dc766060361e43b2 GIT binary patch literal 3044 zcmaJ@fomLD8Sjgslu}A5M<`1O@0OV6LK4#SD5Vs+-Ay;U%_doP_nNz&if?D$?!3#J z_r~|$Y`Q%wA|fIpA|fK9A}S(s93qE^$e|)4B65g`h<}T}Z)S6(J$nP2$;_MizTfxz ze&6rSz)hGvS03{?%`J$DvztCjWGddDd#f6znXKNaIvH6PSJ$gXq;x&XL+(T>aA%~X zj(zpc8*iO1Dtd7CqB5cSWc`CBFNL?}xyv$B-p2B|>p=y5X4MC$%((jGQ%~j>f|C=Y zHVNOnicc>I#d%^JPU?04)v_Ivh6_9ymd_80b7_BZx-_CEUu`zQMs z`#1X!`!73%OOWX|cY4hKpX2<~$AvS;#b=I7&pj@mJwBBFl&ufbwzB%ny|b5R|0kB8 zy{;~=%vgVFh4YpDDIX+G zB3W+ydX3lB7%(Kj;GtdNhGHyNTT*M@_WUp;(gDHH22bI%8Yh7V%X>1F=}@|*eZ=Ii z8a0v5t1w+YN~Ga6t)%h17J`V5H=AzCS2!Z^zRq3Bn<5I z@Cp%kyPF68y3Jh~$QlkJ`iaeTEZs^+dXGZEz~IF4dlRL7r6-VnjCg2-+a{u-X{2rD zc|BJ;@R4=&bhl+s5Ywe-Tu**BSD*oP$e6d3=ey1#Ij@6iLe8 z8SmRXB(qd}R|e1bMJ%^zz3c2y%aoreJkhmTror^JeG48O)UYF_+cgsstL=D7q1z@% zCqc#Pjh7ZEK+BqAa=0TX1TDpTytQ#ePE4x}H?>AZ{K%%lEOvzxz>iW$UR$+PJO)!( zLsMtm?^_#Jh!hRZ3Tp3C9N;KI-xCfv(G?rxZ=R`mliEwVwL=eR0qsP%@M2M2UAY+I zDu73FqalJ5k5@W*bWFDhE7aql;$l7;Q%DX4%sSWG_?GmNV*L&8Vko-$kn%MIrTtfW=p^N5`MEb%6 z;h)F+_Df5oIgS}d(CAo(YDY>fz(NaLo>-^jbxewrMTzx%JQf;5WktGHk`_P z)rJEh;G*QYWkcmY_G9m%hPrUa1>-#kvnaX-MrbaOb0E-1daVf?iEVmoPr*4zw0nk% zjrpq?*gUWyN)~Cdh)SfF&fXnJX<#rI)&K@e_{2)?NKeH;&@30hr&mc>{eUK|(|=9j zj0bl#2ViJuAK^ARSt~cOLt3)3x}Ew3n5%(iNIO?Z36y)~aOka;LFy?Y4RZ|@4a(u3 z`2Zm%`#`Do1mQs9Kx#R`L|b#e5IHV1!9`(qqeat z;#h3#h>fC4TcsVI^1Y`tCYjf=Y!xG(+0r&Jz9VJGNQ-=VvhhHfNbv@J(R5@;1bo@} zOh#lf1uY$@7n)JfGrzw{H7^y?-2q*EPlU%WqGxcP=os7`^FycLYejRr7%b?a@I7Ke z-0e6=Uunpw1zd+8#2BN|2tqZH^u@%Mohz&F$%)m}pI3_h%)aNKajjzbL7!lXD965Y z~51lDNq-~p<)<}zmB$x7+(2@cZVv6RVs`OZAKz3t!g|*S_6NQd;s!hgYeL*7&bGm zG^~y}T+=D+x0OTeaIk-{sJyK7B-#RkMX_g51nj%M08Sw4QhKls#md4MJ`k zD%6B6|Hk%ekvLIuM5*gyoB-~%ha$@W)r^NC6kV$40k-*OZjSjqv?2;G(|djOi0?x} zP{C|#=lj-NYvzV@i_d`ERB1l>5gwump*AfY;4@)kM+%J45e8)Gftm<#fGzxi9Ah`V Wc>^0q1LWnPsc9cwIS}PH`u_kdwz-4= literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt new file mode 100644 index 0000000000000..c242b42b6070b --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0001.txt @@ -0,0 +1,8 @@ +name message 0 +steven hive 1 +dave oozie 2 +xifa phd 3 +chuan hadoop 4 +shanyu senior 5 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt new file mode 100644 index 0000000000000..d5db38d168be7 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0002.txt @@ -0,0 +1,8 @@ +name message 0 +steven2 hive 11 +dave2 oozie 12 +xifa2 phd 13 +chuan2 hadoop 14 +shanyu2 senior 15 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt b/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt new file mode 100644 index 0000000000000..f7a763d8b9638 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_1/0003.txt @@ -0,0 +1,4 @@ +name message 0 +david3 oozie 22 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt new file mode 100644 index 0000000000000..c242b42b6070b --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/01/0001.txt @@ -0,0 +1,8 @@ +name message 0 +steven hive 1 +dave oozie 2 +xifa phd 3 +chuan hadoop 4 +shanyu senior 5 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt new file mode 100644 index 0000000000000..d5db38d168be7 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/02/0002.txt @@ -0,0 +1,8 @@ +name message 0 +steven2 hive 11 +dave2 oozie 12 +xifa2 phd 13 +chuan2 hadoop 14 +shanyu2 senior 15 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt new file mode 100644 index 0000000000000..f7a763d8b9638 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/header_footer_table_2/2012/01/03/0003.txt @@ -0,0 +1,4 @@ +name message 0 +david3 oozie 22 +footer1 footer1 0 +footer2 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt similarity index 100% rename from sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d rename to sql/hive/src/test/resources/data/files/header_footer_table_3/empty1.txt diff --git a/sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 b/sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt similarity index 100% rename from sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 rename to sql/hive/src/test/resources/data/files/header_footer_table_3/empty2.txt diff --git a/sql/hive/src/test/resources/data/files/input.txt b/sql/hive/src/test/resources/data/files/input.txt new file mode 100644 index 0000000000000..caea9919d1d7e --- /dev/null +++ b/sql/hive/src/test/resources/data/files/input.txt @@ -0,0 +1,7 @@ +a b c d e f g +a b c d e f g +a b c d e f g + d e f g +a b c d +a e f g +a d g diff --git a/sql/hive/src/test/resources/data/files/keystore.jks b/sql/hive/src/test/resources/data/files/keystore.jks new file mode 100644 index 0000000000000000000000000000000000000000..469d8a543a4d2a94535996ceee2a24891699b1cc GIT binary patch literal 2248 zcmchYc|6qn8pmhGSSK8N#uk#D-xzBsOBlPcWJ%eYvKt!9bZ>*9AskZ?hGZ*ZEJK!L zJ+ft=bdIU1kg`T0##XM*z4vv`>vjJ>e>{IY@9*n*UeD|EeO{mE$Lfz&5C{Z03h9p}+ z(#5${Qlrizjd*Qt=3Y;+F6i8J>X;dp-8O4&ppIWly63Y$Su+~i$j^H_U7hsxzU@}* z#OCn(Z zU{7sNAB~VRuL8JrNrK~q#6rAJ*ICkuE4NSp!WkpieK#@#EAOc2TU%3KH%WqHY>Iw)QG#c#aThm7Tvs zW9V+`^31@As*EJH_$HMiqL7iy`dENe#S6DgSk3P=pqMIW_vBXe3MFMW-sg}Jzi)+Xrw6Hc-|R%_>A>}c53Q2M2@-qHKlPY7a< zlJzoLawdWFprzUu>){rvT(HZuXmzU9x}>RMw&QPcdPRv0RGr+2+>V&VwTT7m5JFJj zy@C>A71mRU-lD0d+?tZzIt(5$*4CIS$dF4R##Tb z8;&bvn8cM6q7GJ}_mx9**jK)X=oJNxFy|VpsQD#Qb$7Rll4gw0hb`t;n^YetCe(6< zz@F1x%oip3Pxnfq+%K^L^Lq9oIHlutMfigC9=iArL7IeVJGDS^y8%WP>C=MA8k*pl zZO3S88Z@F4d@RoDQhZ#bWY%Zod+l?Y2M!FavmNeByx%Y6De9#eSYYN=*Ct%jKkhb7 z7!L^ALZ?H2oycfg{#3~Movg?CI)4_uFQ&(bJE^7S z!p`?Kr<~zrICUyg`2dJE6tuapVuU6n*g!|I#g4{t8=AICj!c#7QE^))msIFV8eRdN zb$&0v$VEz2j=%^NAthnEepOgNAVTum$$HBxgUMvytWkID#jx5riKo_E;xY@dvrc|f zq{ipCQf}W$I1DMdki!{zxf%1FY45A{HZ!eHqbPDwbZ(=s9~U~Za~}D;om--oLZN3J|MW(74-Fl_ z00d&Y0T9{n01@I>%mx8NAW)AkU2cGvgIk4!BC&YEU>G|H2u1vIWSMMkP!1tZxA2hg zh}iJ|60ZLvoc|;IM~#lEc!W4iv3>!87z8f>`w3t|P}Gq%BZL9LpDCLV?BDMvep29s z7%u{e0JM;Rjh7X7i@K!LlA3&*^ut`zIhhEmbe33oAAK3c%)f)!n!lvCC z)@~Z5e0Fc6pwaU1a+zb9u&ZgZdBT;tw@;L44^i;spji3A8R=Y=HgK;Oe>LL;^H`w$ zV`6;**W~093NhZ{Wky#Om+zSyecGaoZF@wf9c-fKvT|LYDKvemB%Hfe{a(G=!(r#y zC%i(;zO=}f9HT-rD@jU-*RV!kCHDhEYl`ZaDtG7hb85hWHU+E=n7pwME%H#E4 z48oiGvoGg886I})=dH$psxr{a-C@tS&DgT}(M*Jbj}hF@=A=nxGWLoTgbfS=m&pP$ zfYi}%O7THOp~Cv>`L|#e+4|*qBy#!)?VWN9d&yr#Jld)w5y2p#JskQtuCO@9R;D*< zh|m@29h@W{q_REwfrDj?r&9uKjv*9#lPxZ+pJtubxw&mJAFuY-&rSVva9edbtL^a` zVC#wxGu$k0QvPtYF=s(WS2IqRkJJ|3lm;_~Z z4D~zPz5hq}zMH7D3D)rDrvl5fhMj@$a;}$*ALPC~b%wQ+J+q%DsH`B3ntZpOqPQ~l z`HgUjJvs7SlUI2_yz)VFX2b=_v1m%>-M%Z+83QV}6oQgs&wZhaV=d1~TPwW~u?lQG z(N<|lUq}mHz1uBT^Xj@`%d(M6ZFgv>=~%lllQ15KC90j(5|wDa!{FY(SAs`3qvi|_ Js_3iE{{opy;Gh5i literal 0 HcmV?d00001 diff --git a/sql/hive/src/test/resources/data/files/kv9.txt b/sql/hive/src/test/resources/data/files/kv9.txt new file mode 100644 index 0000000000000..b72475f21bff0 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/kv9.txt @@ -0,0 +1,27 @@ +-4400 4400 +1E+99 0 +1E-99 0 +0 0 +10 10 +23232.23435 2 +2389432.23752 3 +2389432.2375 4 +10.73433 5 +0.333 0 +-0.3 0 +-0.333 0 +1.0 1 +2 2 +3.14 3 +-1.12 -1 +-1.122 -11 +1.12 1 +1.122 1 +124.00 124 +125.2 125 +-1255.49 -1255 +3.14 3 +3.140 4 +0.9999999999999999999999999 1 +-1234567890.1234567890 -1234567890 +1234567890.1234567800 1234567890 diff --git a/sql/hive/src/test/resources/data/files/loc.txt b/sql/hive/src/test/resources/data/files/loc.txt new file mode 100644 index 0000000000000..69910b7649571 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/loc.txt @@ -0,0 +1,8 @@ +OH|31|43201|2001 +IO|32|43202|2001 +CA|35|43809|2001 +FL|33|54342|2001 +UT|35||2001 +CA|35|43809|2001 +|34|40000| +FL|33|54342|2001 diff --git a/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt b/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt new file mode 100644 index 0000000000000..41586d61eda03 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/non_ascii_tbl.txt @@ -0,0 +1 @@ +1|Garçu Kôkaku kidôtai diff --git a/sql/hive/src/test/resources/data/files/orc_create_people.txt b/sql/hive/src/test/resources/data/files/orc_create_people.txt index 884598981a13c..ab93c1400769a 100644 --- a/sql/hive/src/test/resources/data/files/orc_create_people.txt +++ b/sql/hive/src/test/resources/data/files/orc_create_people.txt @@ -1,100 +1,100 @@ -1CelesteBrowning959-3763 Nec, Av.Ca -2RisaYangP.O. Box 292, 8229 Porttitor RoadOr -3VenusSuttonAp #962-8021 Egestas Rd.Ca -4GretchenHarrisonP.O. Box 636, 8734 Magna AvenueOr -5LaniIrwinAp #441-5911 Iaculis, AveCa -6VeraGeorge409-1555 Vel, AveOr -7JessicaMalone286-9779 Aliquam RoadCa -8AnnChapmanAp #504-3915 Placerat RoadOr -9NigelBartlettAp #185-385 Diam StreetCa -10AzaliaJennings5772 Diam St.Or -11PrestonCannonAp #527-8769 Nunc AvenueCa -12AllistairVasquez2562 Odio. St.Or -13ReedHayes5190 Elit StreetCa -14ElaineBarronP.O. Box 840, 8860 Sodales Av.Or -15LydiaHoodP.O. Box 698, 5666 Semper RoadCa -16VanceMaxwell298-3313 Malesuada RoadOr -17KeikoDeleonP.O. Box 732, 5921 Massa. Av.Ca -18DolanKaneAp #906-3606 Ut Rd.Or -19MerrittPerkinsP.O. Box 228, 7090 Egestas StreetCa -20CaseySalazar506-5065 Ut St.Or -21SamsonNoel1370 Ultrices, RoadCa -22ByronWalkerP.O. Box 386, 8324 Tellus AveOr -23PiperSingletonAp #500-3561 Primis St.Ca -24RiaMckinney3080 Dui Rd.Or -25RahimStanley559-9016 Nascetur StreetCa -26ChloeSteeleP.O. Box 766, 1628 Elit StreetOr -27PalomaWardAp #390-3042 Ipsum Rd.Ca -28RoaryShermanAp #409-6549 Metus St.Or -29CalvinBuckner6378 Diam AvenueCa -30CamilleGoodAp #113-8659 Suspendisse St.Or -31SteelAyala5518 Justo St.Ca -32JosiahGilbertAp #149-6651 At, Av.Or -33HamiltonCruz4620 Tellus. AveCa -34ScarletSantos586-1785 Velit. Av.Or -35LewisMcintyre629-6419 Ac Rd.Ca -36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.Or -37VelmaHaley1377 At Rd.Ca -38TatumJennings829-7432 Posuere, RoadOr -39BritanniEaton8811 Morbi StreetCa -40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.Or -41KareemAyala2706 Ridiculus StreetCa -42MaiteRush7592 Neque RoadOr -43SigneVelasquezAp #868-3039 Eget St.Ca -44ZoritaCamachoP.O. Box 651, 3340 Quis Av.Or -45GlennaCurtis953-7965 Enim AveCa -46QuinCortez4898 Ridiculus St.Or -47TalonDaltonP.O. Box 408, 7597 Integer Rd.Ca -48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.Or -49VernonReyesP.O. Box 971, 7009 Vulputate StreetCa -50TallulahHeathP.O. Box 865, 3697 Dis AveOr -51CiaranOlson2721 Et St.Ca -52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.Or -53QuinnRiceAp #647-6627 Tristique AvenueCa -54WyattPickettAp #128-3130 Vel, Rd.Or -55EmeraldCopeland857-5119 Turpis Rd.Ca -56JonasQuinnAp #441-7183 Ligula. StreetOr -57WillaBerg6672 Velit AveCa -58MalikLee998-9208 In StreetOr -59CallieMedina1620 Dui. Rd.Ca -60LukeMasonP.O. Box 143, 2070 Augue Rd.Or -61ShafiraEstrada8824 Ante StreetCa -62ElizabethRutledge315-6510 Sit St.Or -63PandoraLevine357-3596 Nibh. AveCa -64HilelPrince845-1229 Sociosqu Rd.Or -65RinahTorresAp #492-9328 At St.Ca -66YaelHobbsP.O. Box 477, 3896 In StreetOr -67NevadaNashP.O. Box 251, 1914 Tincidunt RoadCa -68MarnyHuffP.O. Box 818, 6086 Ultricies St.Or -69KimberleyMilesAp #893-3685 In RoadCa -70DuncanFullerAp #197-5216 Iaculis StreetOr -71YardleyLeblancP.O. Box 938, 1278 Sit AveCa -72HamishBrewerAp #854-781 Quisque St.Or -73PetraMoon453-6609 Curabitur StreetCa -74ReeseEstradaAp #382-3313 Malesuada St.Or -75GageHiggins7443 Eu StreetCa -76ZacheryCamachoAp #795-4143 Quam. St.Or -77KellyGarnerP.O. Box 895, 2843 Cras Rd.Ca -78HanaeCarr9440 Amet St.Or -79AnnAlston884-7948 Dictum RoadCa -80ChancellorCobbP.O. Box 889, 5978 Ac AvenueOr -81DorothyHarrell6974 Tristique AveCa -82VaughanLeon1610 Luctus Av.Or -83WynneJimenez321-9171 Felis. AvenueCa -84WillaMendoza489-182 Sed Av.Or -85CamdenGoodwin4579 Ante St.Ca -86IfeomaFrenchP.O. Box 160, 8769 Integer RoadOr -87RamonaStrong1666 Ridiculus AvenueCa -88BrettRamosAp #579-9879 Et, RoadOr -89UllaGray595-7066 Malesuada RoadCa -90KevynMccallP.O. Box 968, 1420 Aenean AvenueOr -91GenevieveWilkins908 Turpis. StreetCa -92ThaneOneil6766 Lectus St.Or -93MarikoClineP.O. Box 329, 5375 Ac St.Ca -94LaelMclean500-7010 Sit St.Or -95WinifredHopperAp #140-8982 Velit AvenueCa -96RafaelEnglandP.O. Box 405, 7857 Eget Av.Or -97DanaCarter814-601 Purus. Av.Ca -98JulietBattleAp #535-1965 Cursus St.Or -99WynterVincent626-8492 Mollis AvenueCa -100WangMitchell4023 Lacinia. AveOr +1CelesteBrowning959-3763 Nec, Av.100.002011-03-12 15:20:00Ca +2RisaYangP.O. Box 292, 8229 Porttitor Road200.002011-03-12 15:20:00Or +3VenusSuttonAp #962-8021 Egestas Rd.300.002011-03-12 15:20:00Ca +4GretchenHarrisonP.O. Box 636, 8734 Magna Avenue400.002011-03-12 15:20:00Or +5LaniIrwinAp #441-5911 Iaculis, Ave500.002011-03-12 15:20:00Ca +6VeraGeorge409-1555 Vel, Ave600.002011-03-12 15:20:00Or +7JessicaMalone286-9779 Aliquam Road700.002011-03-12 15:20:00Ca +8AnnChapmanAp #504-3915 Placerat Road800.002011-03-12 15:20:00Or +9NigelBartlettAp #185-385 Diam Street900.002011-03-12 15:20:00Ca +10AzaliaJennings5772 Diam St.100.002011-03-12 15:20:00Or +11PrestonCannonAp #527-8769 Nunc Avenue100.002011-03-12 15:20:00Ca +12AllistairVasquez2562 Odio. St.100.002011-03-12 15:20:00Or +13ReedHayes5190 Elit Street100.002011-03-12 15:20:00Ca +14ElaineBarronP.O. Box 840, 8860 Sodales Av.100.002011-03-12 15:20:00Or +15LydiaHoodP.O. Box 698, 5666 Semper Road100.002011-03-12 15:20:00Ca +16VanceMaxwell298-3313 Malesuada Road100.002011-03-12 15:20:00Or +17KeikoDeleonP.O. Box 732, 5921 Massa. Av.100.002011-03-12 15:20:00Ca +18DolanKaneAp #906-3606 Ut Rd.100.002011-03-12 15:20:00Or +19MerrittPerkinsP.O. Box 228, 7090 Egestas Street100.002011-03-12 15:20:00Ca +20CaseySalazar506-5065 Ut St.200.002011-03-12 15:20:00Or +21SamsonNoel1370 Ultrices, Road200.002012-03-12 15:20:00Ca +22ByronWalkerP.O. Box 386, 8324 Tellus Ave200.002012-03-12 15:20:00Or +23PiperSingletonAp #500-3561 Primis St.200.002012-03-12 15:20:00Ca +24RiaMckinney3080 Dui Rd.200.002012-03-12 15:20:00Or +25RahimStanley559-9016 Nascetur Street200.002012-03-12 15:20:00Ca +26ChloeSteeleP.O. Box 766, 1628 Elit Street200.002012-03-12 15:20:00Or +27PalomaWardAp #390-3042 Ipsum Rd.200.002012-03-12 15:20:00Ca +28RoaryShermanAp #409-6549 Metus St.200.002012-03-12 15:20:00Or +29CalvinBuckner6378 Diam Avenue200.002012-03-12 15:20:00Ca +30CamilleGoodAp #113-8659 Suspendisse St.300.002012-03-12 15:20:00Or +31SteelAyala5518 Justo St.300.002012-03-12 15:20:00Ca +32JosiahGilbertAp #149-6651 At, Av.300.002012-03-12 15:20:00Or +33HamiltonCruz4620 Tellus. Ave300.002012-03-12 15:20:00Ca +34ScarletSantos586-1785 Velit. Av.300.002012-03-12 15:20:00Or +35LewisMcintyre629-6419 Ac Rd.300.002012-03-12 15:20:00Ca +36ArsenioMejiaP.O. Box 767, 8625 Justo Rd.300.002012-03-12 15:20:00Or +37VelmaHaley1377 At Rd.300.002012-03-12 15:20:00Ca +38TatumJennings829-7432 Posuere, Road300.002012-03-12 15:20:00Or +39BritanniEaton8811 Morbi Street300.002012-03-12 15:20:00Ca +40AileenJacobsonP.O. Box 469, 2266 Dui, Rd.400.002012-03-12 15:20:00Or +41KareemAyala2706 Ridiculus Street400.002013-03-12 15:20:00Ca +42MaiteRush7592 Neque Road400.002013-03-12 15:20:00Or +43SigneVelasquezAp #868-3039 Eget St.400.002013-03-12 15:20:00Ca +44ZoritaCamachoP.O. Box 651, 3340 Quis Av.400.002013-03-12 15:20:00Or +45GlennaCurtis953-7965 Enim Ave400.002013-03-12 15:20:00Ca +46QuinCortez4898 Ridiculus St.400.002013-03-12 15:20:00Or +47TalonDaltonP.O. Box 408, 7597 Integer Rd.400.002013-03-12 15:20:00Ca +48DarrylBlankenshipP.O. Box 771, 1471 Non Rd.400.002013-03-12 15:20:00Or +49VernonReyesP.O. Box 971, 7009 Vulputate Street400.002013-03-12 15:20:00Ca +50TallulahHeathP.O. Box 865, 3697 Dis Ave500.002013-03-12 15:20:00Or +51CiaranOlson2721 Et St.500.002013-03-12 15:20:00Ca +52OrlandoWittP.O. Box 717, 1102 Nulla. Rd.500.002013-03-12 15:20:00Or +53QuinnRiceAp #647-6627 Tristique Avenue500.002013-03-12 15:20:00Ca +54WyattPickettAp #128-3130 Vel, Rd.500.002013-03-12 15:20:00Or +55EmeraldCopeland857-5119 Turpis Rd.500.002013-03-12 15:20:00Ca +56JonasQuinnAp #441-7183 Ligula. Street500.002013-03-12 15:20:00Or +57WillaBerg6672 Velit Ave500.002013-03-12 15:20:00Ca +58MalikLee998-9208 In Street500.002013-03-12 15:20:00Or +59CallieMedina1620 Dui. Rd.500.002013-03-12 15:20:00Ca +60LukeMasonP.O. Box 143, 2070 Augue Rd.600.002013-03-12 15:20:00Or +61ShafiraEstrada8824 Ante Street600.002014-03-12 15:20:00Ca +62ElizabethRutledge315-6510 Sit St.600.002014-03-12 15:20:00Or +63PandoraLevine357-3596 Nibh. Ave600.002014-03-12 15:20:00Ca +64HilelPrince845-1229 Sociosqu Rd.600.002014-03-12 15:20:00Or +65RinahTorresAp #492-9328 At St.600.002014-03-12 15:20:00Ca +66YaelHobbsP.O. Box 477, 3896 In Street600.002014-03-12 15:20:00Or +67NevadaNashP.O. Box 251, 1914 Tincidunt Road600.002014-03-12 15:20:00Ca +68MarnyHuffP.O. Box 818, 6086 Ultricies St.600.002014-03-12 15:20:00Or +69KimberleyMilesAp #893-3685 In Road600.002014-03-12 15:20:00Ca +70DuncanFullerAp #197-5216 Iaculis Street700.002014-03-12 15:20:00Or +71YardleyLeblancP.O. Box 938, 1278 Sit Ave700.002014-03-12 15:20:00Ca +72HamishBrewerAp #854-781 Quisque St.700.002014-03-12 15:20:00Or +73PetraMoon453-6609 Curabitur Street700.002014-03-12 15:20:00Ca +74ReeseEstradaAp #382-3313 Malesuada St.700.002014-03-12 15:20:00Or +75GageHiggins7443 Eu Street700.002014-03-12 15:20:00Ca +76ZacheryCamachoAp #795-4143 Quam. St.700.002014-03-12 15:20:00Or +77KellyGarnerP.O. Box 895, 2843 Cras Rd.700.002014-03-12 15:20:00Ca +78HanaeCarr9440 Amet St.700.002014-03-12 15:20:00Or +79AnnAlston884-7948 Dictum Road700.002014-03-12 15:20:00Ca +80ChancellorCobbP.O. Box 889, 5978 Ac Avenue800.002014-03-12 15:20:00Or +81DorothyHarrell6974 Tristique Ave800.002010-03-12 15:20:00Ca +82VaughanLeon1610 Luctus Av.800.002010-03-12 15:20:00Or +83WynneJimenez321-9171 Felis. Avenue800.002010-03-12 15:20:00Ca +84WillaMendoza489-182 Sed Av.800.002010-03-12 15:20:00Or +85CamdenGoodwin4579 Ante St.800.002010-03-12 15:20:00Ca +86IfeomaFrenchP.O. Box 160, 8769 Integer Road800.002010-03-12 15:20:00Or +87RamonaStrong1666 Ridiculus Avenue800.002010-03-12 15:20:00Ca +88BrettRamosAp #579-9879 Et, Road800.002010-03-12 15:20:00Or +89UllaGray595-7066 Malesuada Road800.002010-03-12 15:20:00Ca +90KevynMccallP.O. Box 968, 1420 Aenean Avenue900.002010-03-12 15:20:00Or +91GenevieveWilkins908 Turpis. Street900.002010-03-12 15:20:00Ca +92ThaneOneil6766 Lectus St.900.002010-03-12 15:20:00Or +93MarikoClineP.O. Box 329, 5375 Ac St.900.002010-03-12 15:20:00Ca +94LaelMclean500-7010 Sit St.900.002010-03-12 15:20:00Or +95WinifredHopperAp #140-8982 Velit Avenue900.002010-03-12 15:20:00Ca +96RafaelEnglandP.O. Box 405, 7857 Eget Av.900.002010-03-12 15:20:00Or +97DanaCarter814-601 Purus. Av.900.002010-03-12 15:20:00Ca +98JulietBattleAp #535-1965 Cursus St.900.002010-03-12 15:20:00Or +99WynterVincent626-8492 Mollis Avenue900.002010-03-12 15:20:00Ca +100WangMitchell4023 Lacinia. Ave100.002010-03-12 15:20:00Or diff --git a/sql/hive/src/test/resources/data/files/orc_split_elim.orc b/sql/hive/src/test/resources/data/files/orc_split_elim.orc new file mode 100644 index 0000000000000000000000000000000000000000..cd145d343104983b4b09603c251ee749e5f82cc7 GIT binary patch literal 246402 zcmeI)Piz%Qeh2XCKhJI#W2WfAlWOyiO-X|e%1A= zyIz00-{)XU?*~8pejIgGbMIU{8Osp5Qhsl|IQQ^D|N7?R6JdBQtu6E;3)|1u;zF$QSl8u=tMjqA*j1NS zR>yr4kHxy5K74{{sKd_QZgKQxk|$5|;X8Z%L5FOUJh!Ood`4E=YhN8-0 zcn~}fz7Kr@o&(Q;=fL-X103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h z00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70 z103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3Gc)uJ-`!`%G0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^yu$>hp$t2F;qYw| z!Zhr?c=ROZx%Z#Hl{z^%&k@$&4k6AoGi^w-LoL$jp_b{)P^)xqsC8N#>L|_UTC_#F zI@B^<$hB6q{(m8dJ^{~x=fHE|d%yt>aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<701MjZ`7xn?rNN`{L1)nCISq{#NSb;5;RY)6BFX%?`Clr-xdmGefP?xuMo+aj2s-pKH+;>FQ9+bRpNWEmNIqm1`55{C<_D zkIOildv&_kpJJ3d{nVlzr5nTkD4p-hCgiQ`>i*DIdwC>|x~jLMo*3;8T70<>ZbW8cY@#5V6EnZqXanM;9I`^Kf z#f4bqe%IxRtMjqA*j1NTE|2>r9*cE9ZTJM!a4p}N-QwuYBu}2^!*}-jgAUmyd2UhF z`4WZ;=$G~P`}g8BR$DBT$LN9lZ5 zHX(0iSNDg$+RGzx)KyJB@Kh{A=swKdE;e5m?QW62`s#XI9EMNj;pH|~pWXbt8{OXA z*ji6dCsjNfi_k6Q_r{BJzkb@kUVeOH|7I_(Ej)bi;KBE{pRL7(SmnX4%M(}UV{x&o zuB==c_f0$&>wf<55vJkmBHx?cKcJc9$nVy z!*k$!zyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h0 z0S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K0 z2ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`2 z9N@tF?!b-x0JIX2fCMBU0SQPz0uqpb1SB8<2}nQ!5|F?ogAF!2RhW_o7m*{t2BLF#@XDf)4l!_qulAI7VRkA81_f$d{;IhZ)I2ahrZg&Bk%Cu zIJE>MAOQ(TKmrnwfCMBU0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^NI(J-kbndvAOQ(T zKmrnwfCMBU0SQPz0uqpb1SB8<2}nQ!5|DrdBp?9^NI(J-kbndvAc1$4Kpb^dlYevN zWX!*!(jCv;Zmx;#ZjtWXy&e~b(NlSJxsBE4`mJtsdvjxJJw2UN@oX$Yx0K%-FU~!D z(7(R<_{9F_Tv}V`M;5l9t;K~{<-xAY6IbVBaj~mDTKQ<)H}P1k`}xBsn1=pGZFY;J zHkm3)o8-AgRp(0>F5uwv_RISzcs~X2Iq*H;00%h00S<70103K02ROh1 z4sd`29N+*4IKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4 zIKTl8aDW3G-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G z-~b0WzyS_$fCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aDW3G-~b0WzyS_$ zfCC)h00%h00S<70103K02ROh14sd`29N+*4IKTl8aNvD+;K%y`XeA&42}nQ!5|Drd zBp?9^NI(J-kbndvAc1$7z%-O$XD=MSO+uK4|9J7}Nz8NaKYuHAa&Vp_tiK&XoM~p- zkYD*B3v^dmJn$NXpi*$9UWx9}S*_Nr!waT@LO@6;h)5m3;&AmF^ z>rXMtoqlT3j?#@`f0WL5WfSsNc6EQ~tGzrDKaW+D4?LcmZmy}@-6Flbb3LBQ`^#;t zesuF@H@dyKv9+F_PO7++hsTR^4h&JSR?dI4J;WLm}?E1m@5sMm=YH- zGchtTu|(`Q<22x9f#b+b1XMYS;eE$kI*1xT_?4P^!=i2W- zyJjwX`0LZ`EpGo)7W*$232|(3Zb(~yXS2qJB_2{OndQnS9>`7C+Rb<*ihs?Glec*? zd{;DWY~Z+j`K5>9`Mpt27qumo4?RA+W4mU-?xp=3{(KI6Jb8M^F4e71SJt@{ueqnY zCc^K>uCq5P)8GGpw$W?nD%RC`6Am?n@jqO()9~5bo-?mIm@B^QjFE1(3uc;-AotQF z*nLmO%oQI6Ec3P=o02kP^Xb!}$9UJ2Fs$kce03oA@HZ#sDg1%A4OQc9q!PVU9Qs-c z(qx&K85tNCD;mff$O2G2LJL^K^Km?{LU?4Iw z2>&Vxlqi|Ipv+tTNYSaf{TVS?&61f~-_AZ_`)*r#ab~g?kD>aJ7T4&HR^P2mTfaFx ztJk@c7^?dwYxkOE-*>NgXJ8vM-Ni&~zr}z5Hn;wMd7sWXNoJ*|$~GKZ=^OpIdDFtr zO$X{MR?M2hI6bP~&x=>pA+vAqsbfJt_p0-Lhlw9L8EnSGOvrrqy3 zuGOiU*<9}OezjzQn~R*M#z|54jGYR*S6W|u-j(%s(gE?cSE{VHzp@ElcOWOn@!THU i+jZwF3Yv86jpP{Duj5kx`QB$`i> -Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:marmbrus, createTime:1413871682, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871682}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 index 2860d5c6b4cfb..1bfdd079010aa 100644 --- a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 +++ b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 @@ -1,5 +1,5 @@ -key int None +key int _bucketname string _offsets array -Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389733869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2023038695216118221/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389733869}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:marmbrus, createTime:1413871682, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871682}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d +++ b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 index c80ef36c96ad4..136628776692b 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 @@ -1 +1 @@ -754 -7678496319 \ No newline at end of file +754 -7678496319 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 index 618c1d01b726d..d32a641c0050e 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 @@ -1,12 +1,12 @@ -key int from deserializer -value string from deserializer -ds string None -ts string None +key int +value string +ds string +ts string # Partition Information # col_name data_type comment -ds string None -ts string None +ds string +ts string -Detailed Partition Information Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1389728902, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse545253106736176469/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728902}) \ No newline at end of file +Detailed Partition Information Partition(values:[2012-01-03, 2012-01-03+14:46:31], dbName:default, tableName:src_rc_merge_test_part, createTime:1413930366, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6310217467672453297/src_rc_merge_test_part/ds=2012-01-03/ts=2012-01-03+14%3A46%3A31, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413930366}) diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f b/sql/hive/src/test/resources/golden/alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f rename to sql/hive/src/test/resources/golden/alter_merge_2-3-bc9bb363f9a2026cfc70a31bb4551352 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a b/sql/hive/src/test/resources/golden/alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a rename to sql/hive/src/test/resources/golden/alter_merge_2-4-d3bf7703ba00cf7c40f2a2dbb8ca7224 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 b/sql/hive/src/test/resources/golden/alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 rename to sql/hive/src/test/resources/golden/alter_merge_2-5-6319bf26f3739260b1a77e2ea89ef147 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 index 3f10ffe7a4c47..60d3b2f4a4cd5 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 @@ -1 +1 @@ -15 \ No newline at end of file +15 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 index c80ef36c96ad4..136628776692b 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 @@ -1 +1 @@ -754 -7678496319 \ No newline at end of file +754 -7678496319 diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 index 3f10ffe7a4c47..60d3b2f4a4cd5 100644 --- a/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 +++ b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 @@ -1 +1 @@ -15 \ No newline at end of file +15 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 b/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 b/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e b/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e deleted file mode 100644 index b28ab5ccf8a1b..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e +++ /dev/null @@ -1,75 +0,0 @@ -238 val_238 10 3.0 -NULL 10 3.0 -311 val_311 10 3.0 -NULL val_27 10 3.0 -NULL val_165 10 3.0 -NULL val_409 10 3.0 -255 val_255 10 3.0 -278 val_278 10 3.0 -98 val_98 10 3.0 -NULL val_484 10 3.0 -NULL val_265 10 3.0 -NULL val_193 10 3.0 -401 val_401 10 3.0 -150 val_150 10 3.0 -273 val_273 10 3.0 -224 10 3.0 -369 10 3.0 -66 val_66 10 3.0 -128 10 3.0 -213 val_213 10 3.0 -146 val_146 10 3.0 -406 val_406 10 3.0 -NULL 10 3.0 -NULL 10 3.0 -NULL 10 3.0 -238 val_238 100x 3.0 -NULL 100x 3.0 -311 val_311 100x 3.0 -NULL val_27 100x 3.0 -NULL val_165 100x 3.0 -NULL val_409 100x 3.0 -255 val_255 100x 3.0 -278 val_278 100x 3.0 -98 val_98 100x 3.0 -NULL val_484 100x 3.0 -NULL val_265 100x 3.0 -NULL val_193 100x 3.0 -401 val_401 100x 3.0 -150 val_150 100x 3.0 -273 val_273 100x 3.0 -224 100x 3.0 -369 100x 3.0 -66 val_66 100x 3.0 -128 100x 3.0 -213 val_213 100x 3.0 -146 val_146 100x 3.0 -406 val_406 100x 3.0 -NULL 100x 3.0 -NULL 100x 3.0 -NULL 100x 3.0 -238 val_238 100x 6:30pm -NULL 100x 6:30pm -311 val_311 100x 6:30pm -NULL val_27 100x 6:30pm -NULL val_165 100x 6:30pm -NULL val_409 100x 6:30pm -255 val_255 100x 6:30pm -278 val_278 100x 6:30pm -98 val_98 100x 6:30pm -NULL val_484 100x 6:30pm -NULL val_265 100x 6:30pm -NULL val_193 100x 6:30pm -401 val_401 100x 6:30pm -150 val_150 100x 6:30pm -273 val_273 100x 6:30pm -224 100x 6:30pm -369 100x 6:30pm -66 val_66 100x 6:30pm -128 100x 6:30pm -213 val_213 100x 6:30pm -146 val_146 100x 6:30pm -406 val_406 100x 6:30pm -NULL 100x 6:30pm -NULL 100x 6:30pm -NULL 100x 6:30pm \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c b/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c deleted file mode 100644 index c5b431b6cba29..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c +++ /dev/null @@ -1 +0,0 @@ -50 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 b/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 deleted file mode 100644 index a76c74dcec6ab..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 +++ /dev/null @@ -1 +0,0 @@ -75 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a deleted file mode 100644 index 316ca7f65ba20..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts string None - -# Partition Information -# col_name data_type comment - -dt string None -ts string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 b/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a b/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 b/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 deleted file mode 100644 index a7382fabfcb49..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 +++ /dev/null @@ -1,10 +0,0 @@ -key string None -value string None -dt string None -ts double None - -# Partition Information -# col_name data_type comment - -dt string None -ts double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 b/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd b/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd deleted file mode 100644 index 410b14d2ce6f9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd +++ /dev/null @@ -1 +0,0 @@ -25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 b/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 index db182e444d31d..11487abed2b60 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,4 +1,4 @@ -key int None -value string None +key int +value string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871688}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 index 81e23f0bc1951..979969dcbfd3f 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}) \ No newline at end of file +Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 index 6dc1f3ca2c187..7e14edcdead2e 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}) \ No newline at end of file +Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/ds=2010, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 index 4b754043d63ab..77a764a814eb9 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 index f44c28ee36760..c8606b1acad0c 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 index e739ad4992ec9..59922d3b7a086 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,10 +1,10 @@ -key int from deserializer -value string from deserializer -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388805892, transient_lastDdlTime=1388805892}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871689, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413871689, transient_lastDdlTime=1413871689}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 index 092e5ed6e8a46..45ef755539479 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,4 +1,4 @@ -key int from deserializer -value string from deserializer +key int +value string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 index 197e67d09bf49..d6804307f3dc7 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 @@ -1,4 +1,4 @@ -key int from deserializer -value string from deserializer +key int +value string -Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1388805891, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805891, transient_lastDdlTime=1388805891}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:alter_partition_format_test, dbName:default, owner:marmbrus, createTime:1413871688, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/test/test/, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871688, transient_lastDdlTime=1413871688, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 index 29b5b693b8589..77ba51afd2468 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 +++ b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None +key int +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1388805891, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805891}) \ No newline at end of file +Detailed Partition Information Partition(values:[2010], dbName:default, tableName:alter_partition_format_test, createTime:1413871689, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/alter_partition_format_test/ds=2010, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871689}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-e11f1476d2f30ab7a83f95013809f9e6 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-29b4b67965f24c40bcc426043efb892d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-2100fc7853625d8c7dad5c0ee70d4690 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 rename to sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-c3fa911536c75e87febf2874e7889879 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 index 02d72f4292749..c1b66aba46e84 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 +++ b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 @@ -9,4 +9,4 @@ 3 13 1996 12 7 17 1996 12 8 18 1996 12 -8 28 1996 12 \ No newline at end of file +8 28 1996 12 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d index bca47334cedaa..12ec4459dc167 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d +++ b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d @@ -3,4 +3,4 @@ 3 13 1995 09 7 17 1995 09 8 18 1995 09 -8 28 1995 09 \ No newline at end of file +8 28 1995 09 diff --git a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 +++ b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 index e881b2a4b74eb..605b17f1d8ceb 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 @@ -3,4 +3,4 @@ 3 new_part1: new_part2: 4 new_part1: new_part2: 5 new_part1: new_part2: -6 new_part1: new_part2: \ No newline at end of file +6 new_part1: new_part2: diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f b/sql/hive/src/test/resources/golden/alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f rename to sql/hive/src/test/resources/golden/alter_rename_partition-19-d862c00e5a37ae841a6c8ec6c5d68e8c diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 b/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 index 684f1da0fd0a6..6105de433830c 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 @@ -3,4 +3,4 @@ 3 old_part1: old_part2: 4 old_part1: old_part2: 5 old_part1: old_part2: -6 old_part1: old_part2: \ No newline at end of file +6 old_part1: old_part2: diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 index fc31ec62a1280..fb196b30f4478 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 @@ -1 +1 @@ -pcol1=new_part1%3A/pcol2=new_part2%3A \ No newline at end of file +pcol1=new_part1%3A/pcol2=new_part2%3A diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b b/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b index e881b2a4b74eb..605b17f1d8ceb 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b @@ -3,4 +3,4 @@ 3 new_part1: new_part2: 4 new_part1: new_part2: 5 new_part1: new_part2: -6 new_part1: new_part2: \ No newline at end of file +6 new_part1: new_part2: diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 b/sql/hive/src/test/resources/golden/alter_rename_partition-4-833254c8f7c5b1248e1a249560652627 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 rename to sql/hive/src/test/resources/golden/alter_rename_partition-4-833254c8f7c5b1248e1a249560652627 diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 index 684f1da0fd0a6..6105de433830c 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 @@ -3,4 +3,4 @@ 3 old_part1: old_part2: 4 old_part1: old_part2: 5 old_part1: old_part2: -6 old_part1: old_part2: \ No newline at end of file +6 old_part1: old_part2: diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 index fc31ec62a1280..fb196b30f4478 100644 --- a/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 @@ -1 +1 @@ -pcol1=new_part1%3A/pcol2=new_part2%3A \ No newline at end of file +pcol1=new_part1%3A/pcol2=new_part2%3A diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 index ccd6518a50f7c..5a608a9064b84 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 @@ -1,5 +1,5 @@ -id int None -query string None -name string None +id int +query string +name string -Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413871711}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 index d135e450e6e8e..1d6d48bbe1d1c 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 index d135e450e6e8e..1d6d48bbe1d1c 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 index c3fac88f91a36..f33269bd266be 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871712, transient_lastDdlTime=1413871712, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 index 6c8f91de7cded..41e8d696d33ea 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871712, transient_lastDdlTime=1413871712, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 index 37a5b2cc47bad..7caf0af50b071 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 @@ -1,5 +1,5 @@ -id int from deserializer -query string from deserializer -name string from deserializer +id int +query string +name string -Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871711, transient_lastDdlTime=1413871711, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 index 0348dd15fd4f5..a55479e69bbea 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 @@ -1,5 +1,5 @@ -id int from deserializer -query string from deserializer -name string from deserializer +id int +query string +name string -Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1388805893, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:from deserializer), FieldSchema(name:query, type:string, comment:from deserializer), FieldSchema(name:name, type:string, comment:from deserializer)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388805893, transient_lastDdlTime=1388805893}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test_table, dbName:default, owner:marmbrus, createTime:1413871711, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe, parameters:{serialization.format=1, field.delim=,}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413871711, transient_lastDdlTime=1413871711, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 index d135e450e6e8e..1d6d48bbe1d1c 100644 --- a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 +++ b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 @@ -1,11 +1,11 @@ -id int None -query string None -name string None -dt string None +id int +query string +name string +dt string # Partition Information # col_name data_type comment -dt string None +dt string -Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1388805893, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2706017013471029005/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388805893}) \ No newline at end of file +Detailed Partition Information Partition(values:[2011], dbName:default, tableName:test_table, createTime:1413871711, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:id, type:int, comment:null), FieldSchema(name:query, type:string, comment:null), FieldSchema(name:name, type:string, comment:null), FieldSchema(name:dt, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test_table/dt=2011, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413871711}) diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 b/sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 rename to sql/hive/src/test/resources/golden/alter_varchar2-2-3a20c238eab602ad3d593b1eb3fa6dbb diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 index 40818a7de46d0..600b377716894 100644 --- a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 +++ b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 @@ -1 +1 @@ -val_238 7 \ No newline at end of file +val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c index 827220bd4996f..ad69f390bc8db 100644 --- a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c +++ b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c @@ -1 +1 @@ -1 val_238 7 \ No newline at end of file +1 val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d b/sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d rename to sql/hive/src/test/resources/golden/alter_varchar2-6-3250407f20f3766c18f44b8bfae1829d diff --git a/sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 rename to sql/hive/src/test/resources/golden/alter_varchar2-7-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c deleted file mode 100644 index 827220bd4996f..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c +++ /dev/null @@ -1 +0,0 @@ -1 val_238 7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c deleted file mode 100644 index 8a8234a35f6bb..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c +++ /dev/null @@ -1 +0,0 @@ -2 238 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c new file mode 100644 index 0000000000000..ad69f390bc8db --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-8-84e700f9dc6033c1f237fcdb95e31a0c @@ -0,0 +1 @@ +1 val_238 7 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c new file mode 100644 index 0000000000000..1f8ddaec9003a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-9-4c12c4c53d99338796be34e603dc612c @@ -0,0 +1 @@ +2 238 3 diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b index a99747531cef4..ecae907d92279 100644 --- a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b +++ b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b @@ -1,20 +1,20 @@ # col_name data_type comment -key int None -value string None -ds string None -hr string None +key int +value string +ds string +hr string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:47:52 PST 2014 +CreateTime: Mon Oct 20 23:10:25 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 Table Type: VIRTUAL_VIEW Table Parameters: - transient_lastDdlTime 1391813272 + transient_lastDdlTime 1413871825 # Storage Information SerDe Library: null diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b index 499c73127d890..10e3eca6c6791 100644 --- a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b +++ b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b @@ -1,17 +1,17 @@ # col_name data_type comment -value string None +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:47:52 PST 2014 +CreateTime: Mon Oct 20 23:10:25 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 Table Type: VIRTUAL_VIEW Table Parameters: - transient_lastDdlTime 1391813272 + transient_lastDdlTime 1413871825 # Storage Information SerDe Library: null diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b index a5fba77abdf07..c3661a1f79b57 100644 --- a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b +++ b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b @@ -1,18 +1,18 @@ # col_name data_type comment -key int None -value string None +key int +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:47:52 PST 2014 +CreateTime: Mon Oct 20 23:10:25 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 Table Type: VIRTUAL_VIEW Table Parameters: - transient_lastDdlTime 1391813272 + transient_lastDdlTime 1413871825 # Storage Information SerDe Library: null diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb b/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb deleted file mode 100644 index ee76e02af3aba..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb +++ /dev/null @@ -1,5 +0,0 @@ -foo int None -bar string None -ds string None - -Detailed Table Information Table(tableName:view1, dbName:default, owner:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 b/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 deleted file mode 100644 index 8603577477bfc..0000000000000 --- a/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 +++ /dev/null @@ -1,5 +0,0 @@ -foo int None -bar string None -ds string None - -Detailed Table Information Table(tableName:view2, dbName:default, owner:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426511, transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c rename to sql/hive/src/test/resources/golden/ambiguous_col-0-e91e3e5a22029b9b979ccbbef97add66 diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d b/sql/hive/src/test/resources/golden/ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d rename to sql/hive/src/test/resources/golden/ambiguous_col-1-b4fe82679efdf6a15e9ecff53baf8d8d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 b/sql/hive/src/test/resources/golden/ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 rename to sql/hive/src/test/resources/golden/ambiguous_col-2-dadfa3854356dead14b93c5a71a5d8ab diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 b/sql/hive/src/test/resources/golden/ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 rename to sql/hive/src/test/resources/golden/ambiguous_col-3-70509ccd2765d90b98666b6dff8afe1b diff --git a/sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to sql/hive/src/test/resources/golden/annotate_stats_join-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c b/sql/hive/src/test/resources/golden/annotate_stats_join-1-84e7846d50fc15e836c83911ce039871 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c rename to sql/hive/src/test/resources/golden/annotate_stats_join-1-84e7846d50fc15e836c83911ce039871 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 b/sql/hive/src/test/resources/golden/annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 rename to sql/hive/src/test/resources/golden/annotate_stats_join-10-a59dc1c01d48c82d46a5beab759f618d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 b/sql/hive/src/test/resources/golden/annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 rename to sql/hive/src/test/resources/golden/annotate_stats_join-11-3e0ade2476221f6396381f55a3b82e60 diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 b/sql/hive/src/test/resources/golden/annotate_stats_join-12-cef44682d339a67ba765f854da21f976 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 rename to sql/hive/src/test/resources/golden/annotate_stats_join-12-cef44682d339a67ba765f854da21f976 diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c b/sql/hive/src/test/resources/golden/annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e similarity index 100% rename from sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c rename to sql/hive/src/test/resources/golden/annotate_stats_join-13-6292c27f0c4eaeab2a5e6c317e3afa2e diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f b/sql/hive/src/test/resources/golden/annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f rename to sql/hive/src/test/resources/golden/annotate_stats_join-14-4f3042b9feebd00c540ddac40e7254d1 diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 b/sql/hive/src/test/resources/golden/annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 rename to sql/hive/src/test/resources/golden/annotate_stats_join-15-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 b/sql/hive/src/test/resources/golden/annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 rename to sql/hive/src/test/resources/golden/annotate_stats_join-16-a75699a21ea36d962a1ba45bd5a12f26 diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 b/sql/hive/src/test/resources/golden/annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 rename to sql/hive/src/test/resources/golden/annotate_stats_join-17-64380f94a431e3a1698aa2edd3d0a6b2 diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d b/sql/hive/src/test/resources/golden/annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d rename to sql/hive/src/test/resources/golden/annotate_stats_join-18-222d1fcce7420950436e00d3a1bba957 diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a b/sql/hive/src/test/resources/golden/annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589 similarity index 100% rename from sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a rename to sql/hive/src/test/resources/golden/annotate_stats_join-19-dea5f8f5c703583c7a3bdcb62cd3d589 diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d b/sql/hive/src/test/resources/golden/annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600 similarity index 100% rename from sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d rename to sql/hive/src/test/resources/golden/annotate_stats_join-2-40548ec2313af8dbdcbb8ad0477d8600 diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab b/sql/hive/src/test/resources/golden/annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8 similarity index 100% rename from sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab rename to sql/hive/src/test/resources/golden/annotate_stats_join-20-2d5e186b97166e3a625a169d0d73f9c8 diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b b/sql/hive/src/test/resources/golden/annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2 similarity index 100% rename from sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b rename to sql/hive/src/test/resources/golden/annotate_stats_join-21-ed73d4b01424287148347ccf721b37e2 diff --git a/sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 rename to sql/hive/src/test/resources/golden/annotate_stats_join-22-2cf93da6bb0efdafeaa989c9df236701 diff --git a/sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_join-23-fa90806f6137300311df032e28df3d4c similarity index 100% rename from sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_join-23-fa90806f6137300311df032e28df3d4c diff --git a/sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_join-24-4a6976344eeae35e059285ed78f9feb3 diff --git a/sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a rename to sql/hive/src/test/resources/golden/annotate_stats_join-3-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f rename to sql/hive/src/test/resources/golden/annotate_stats_join-4-a598c93d86a646cfa6ea86da54bce9b8 diff --git a/sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 b/sql/hive/src/test/resources/golden/annotate_stats_join-5-d12ba848d20d1def95379c490068f050 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 rename to sql/hive/src/test/resources/golden/annotate_stats_join-5-d12ba848d20d1def95379c490068f050 diff --git a/sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_join-6-96a4806e61c5d882affac57e115f285f similarity index 100% rename from sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_join-6-96a4806e61c5d882affac57e115f285f diff --git a/sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_join-7-38be41d6328f57350a78fb2097d1ebd2 diff --git a/sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f rename to sql/hive/src/test/resources/golden/annotate_stats_join-8-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a rename to sql/hive/src/test/resources/golden/annotate_stats_join-9-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-1-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-10-a89c94fd0de0cfc96725fea890987cc0 diff --git a/sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-2-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-3-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-4-4252006172a476fbc591eebee49bffa3 diff --git a/sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-5-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 rename to sql/hive/src/test/resources/golden/annotate_stats_limit-6-f1fd6c403a9787947877f48c772afe96 diff --git a/sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40 similarity index 100% rename from sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd rename to sql/hive/src/test/resources/golden/annotate_stats_limit-7-69b422026fce4e2d2cde9a478d6aaa40 diff --git a/sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb similarity index 100% rename from sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca rename to sql/hive/src/test/resources/golden/annotate_stats_limit-8-72f5e55c1e244084eea255c32a6f58eb diff --git a/sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f similarity index 100% rename from sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c rename to sql/hive/src/test/resources/golden/annotate_stats_limit-9-9da67c62d5e3145d450ad7953b33361f diff --git a/sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf rename to sql/hive/src/test/resources/golden/annotate_stats_part-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 rename to sql/hive/src/test/resources/golden/annotate_stats_part-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff similarity index 100% rename from sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf rename to sql/hive/src/test/resources/golden/annotate_stats_part-10-5ba0295bfe42deb678e59b3a330b14ff diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 rename to sql/hive/src/test/resources/golden/annotate_stats_part-11-dbdbe2e04c5dad6c8af78b6386b329 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_part-12-2856fb4f4af8932804cb238429d9de6f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_part-13-4fa8b0f9fb0202ac0e82fb87538d6445 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a rename to sql/hive/src/test/resources/golden/annotate_stats_part-14-62c557bfb7bf70694a32ebceea329ebd diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f rename to sql/hive/src/test/resources/golden/annotate_stats_part-15-f796cd035603726a5c4ce3e71194822b diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 b/sql/hive/src/test/resources/golden/annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 rename to sql/hive/src/test/resources/golden/annotate_stats_part-16-45eb5544124364714549f199f9c2b2ac diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 rename to sql/hive/src/test/resources/golden/annotate_stats_part-17-ad61ebd912b6bef3778c4ff38c0be5ab diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 rename to sql/hive/src/test/resources/golden/annotate_stats_part-18-b9f2dff7c7b57412cea44433ea810fa7 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f rename to sql/hive/src/test/resources/golden/annotate_stats_part-19-10cab43c2966718bb39e2f22365cd6c1 diff --git a/sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 rename to sql/hive/src/test/resources/golden/annotate_stats_part-2-a4fb8359a2179ec70777aad6366071b7 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a rename to sql/hive/src/test/resources/golden/annotate_stats_part-20-51ec5046b50543899ed54c9fc2b885af diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 rename to sql/hive/src/test/resources/golden/annotate_stats_part-21-d69cb7b7770b51bc4b99d5d0f74d4662 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 rename to sql/hive/src/test/resources/golden/annotate_stats_part-22-22f430881fbfe92736988c2a4184fe0c diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_part-23-2719ee3f0b8b07f1e5c80f8329b9f87f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 rename to sql/hive/src/test/resources/golden/annotate_stats_part-24-1f7bdd816b83f45a6d53d08b6866109f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_part-25-d1599e385e8bad6cd62d564a6fd0536f diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 rename to sql/hive/src/test/resources/golden/annotate_stats_part-26-ec26bcb2587f4638b364ba494ac373e0 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 rename to sql/hive/src/test/resources/golden/annotate_stats_part-27-7804e2e230a42a46d72546fa2381b423 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd rename to sql/hive/src/test/resources/golden/annotate_stats_part-28-40f2a1f06d181ef93edf42a591cbf15e diff --git a/sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/annotate_stats_part-3-16367c381d4b189b3640c92511244bfe similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea rename to sql/hive/src/test/resources/golden/annotate_stats_part-3-16367c381d4b189b3640c92511244bfe diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca rename to sql/hive/src/test/resources/golden/annotate_stats_part-4-397e834a25a20b68aed5e87d55ea6174 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_part-5-cef44682d339a67ba765f854da21f976 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c rename to sql/hive/src/test/resources/golden/annotate_stats_part-5-cef44682d339a67ba765f854da21f976 diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_part-6-f4263aca1612c05cb08242448191ad05 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf rename to sql/hive/src/test/resources/golden/annotate_stats_part-6-f4263aca1612c05cb08242448191ad05 diff --git a/sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 rename to sql/hive/src/test/resources/golden/annotate_stats_part-7-b222533ddda09918c4f5acc408bf1a02 diff --git a/sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 b/sql/hive/src/test/resources/golden/annotate_stats_part-8-84e3979946de67779a9704a3adc2184f similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 rename to sql/hive/src/test/resources/golden/annotate_stats_part-8-84e3979946de67779a9704a3adc2184f diff --git a/sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d b/sql/hive/src/test/resources/golden/annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d rename to sql/hive/src/test/resources/golden/annotate_stats_part-9-c6c38e93babafe56cd4f177a17d37b8 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-49-b1e2ade89ae898650f0be4f796d8947b rename to sql/hive/src/test/resources/golden/annotate_stats_table-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-50-b9d963d24994c47c3776dda6f7d3881f rename to sql/hive/src/test/resources/golden/annotate_stats_table-1-3c29684bfd2df7439ee0551eb42cfa0 diff --git a/sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 rename to sql/hive/src/test/resources/golden/annotate_stats_table-10-a7419af512a6c0b60ec51e556608cdad diff --git a/sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/annotate_stats_table-11-4be11140c2e21b3a172224d103844785 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 rename to sql/hive/src/test/resources/golden/annotate_stats_table-11-4be11140c2e21b3a172224d103844785 diff --git a/sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 rename to sql/hive/src/test/resources/golden/annotate_stats_table-12-84bc7cf67544461e9102820ed410dfb6 diff --git a/sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 rename to sql/hive/src/test/resources/golden/annotate_stats_table-13-d2acfdf279aad035b31ed61e87bff6 diff --git a/sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a rename to sql/hive/src/test/resources/golden/annotate_stats_table-14-60d1c44b8412c4983265e320ecfb25f0 diff --git a/sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4 similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca rename to sql/hive/src/test/resources/golden/annotate_stats_table-15-7b185296b6a0c6ceff2df1f89fecc6b4 diff --git a/sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c rename to sql/hive/src/test/resources/golden/annotate_stats_table-16-7cf8cb7d8d0fe3ea94d7b190a2e5ad3d diff --git a/sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c similarity index 100% rename from sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf rename to sql/hive/src/test/resources/golden/annotate_stats_table-17-cd84d629d569a09d139f38dae01e677c diff --git a/sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf b/sql/hive/src/test/resources/golden/annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf rename to sql/hive/src/test/resources/golden/annotate_stats_table-18-7012af4092c18075f4f3a00b80f491ee diff --git a/sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 b/sql/hive/src/test/resources/golden/annotate_stats_table-2-84e7846d50fc15e836c83911ce039871 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 rename to sql/hive/src/test/resources/golden/annotate_stats_table-2-84e7846d50fc15e836c83911ce039871 diff --git a/sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e b/sql/hive/src/test/resources/golden/annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e rename to sql/hive/src/test/resources/golden/annotate_stats_table-3-a598c93d86a646cfa6ea86da54bce9b8 diff --git a/sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f b/sql/hive/src/test/resources/golden/annotate_stats_table-4-d12ba848d20d1def95379c490068f050 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f rename to sql/hive/src/test/resources/golden/annotate_stats_table-4-d12ba848d20d1def95379c490068f050 diff --git a/sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 b/sql/hive/src/test/resources/golden/annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 rename to sql/hive/src/test/resources/golden/annotate_stats_table-5-4a8ad142e57c5dce2623e92f5e5ad15a diff --git a/sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 b/sql/hive/src/test/resources/golden/annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 rename to sql/hive/src/test/resources/golden/annotate_stats_table-6-a59dc1c01d48c82d46a5beab759f618d diff --git a/sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b b/sql/hive/src/test/resources/golden/annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b rename to sql/hive/src/test/resources/golden/annotate_stats_table-7-6292c27f0c4eaeab2a5e6c317e3afa2e diff --git a/sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c b/sql/hive/src/test/resources/golden/annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c rename to sql/hive/src/test/resources/golden/annotate_stats_table-8-43cb040e2fe01904bc52e198fcae9b3d diff --git a/sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 b/sql/hive/src/test/resources/golden/annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 rename to sql/hive/src/test/resources/golden/annotate_stats_table-9-eadfdc61d22bd22bbf5a69370908a82e diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-53-777edd9d575f3480ca6cebe4be57b1f6 rename to sql/hive/src/test/resources/golden/annotate_stats_union-0-2a0c41508e1e70eaedf9de99751c8fa9 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 rename to sql/hive/src/test/resources/golden/annotate_stats_union-1-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b rename to sql/hive/src/test/resources/golden/annotate_stats_union-10-3aa3eb68d092ea0157c5426a5f2ca3f9 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/annotate_stats_union-11-c1302094d7b4456550826535b529468b similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 rename to sql/hive/src/test/resources/golden/annotate_stats_union-11-c1302094d7b4456550826535b529468b diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c rename to sql/hive/src/test/resources/golden/annotate_stats_union-12-3af760f960a2e0995d1bc4afef0c5aa0 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 rename to sql/hive/src/test/resources/golden/annotate_stats_union-13-26f82fb6734d5bc6f7159b06c0949178 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 rename to sql/hive/src/test/resources/golden/annotate_stats_union-14-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 b/sql/hive/src/test/resources/golden/annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 rename to sql/hive/src/test/resources/golden/annotate_stats_union-15-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 b/sql/hive/src/test/resources/golden/annotate_stats_union-16-4252006172a476fbc591eebee49bffa3 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 rename to sql/hive/src/test/resources/golden/annotate_stats_union-16-4252006172a476fbc591eebee49bffa3 diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b b/sql/hive/src/test/resources/golden/annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b rename to sql/hive/src/test/resources/golden/annotate_stats_union-17-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/annotate_stats_union-18-58d55f8c8b2489726232a00254499144 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/annotate_stats_union-18-58d55f8c8b2489726232a00254499144 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/annotate_stats_union-19-70c851c23a7ac89f0366b9c26085cc3e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/annotate_stats_union-2-eb11e867a36e2114ef0f0f001e01984c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/annotate_stats_union-20-222d1fcce7420950436e00d3a1bba957 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/annotate_stats_union-21-a5503df8f50e057415719bcfe4c69f13 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/annotate_stats_union-22-a23089e5a76dc35df04d9936d60c4565 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/annotate_stats_union-3-e0b7cb2226234fbea8ae11ea7d9d84bd diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/annotate_stats_union-4-4252006172a476fbc591eebee49bffa3 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/annotate_stats_union-4-4252006172a476fbc591eebee49bffa3 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/annotate_stats_union-5-a2f3b8a636e46e5df514df46c452855f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/annotate_stats_union-6-222d1fcce7420950436e00d3a1bba957 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/annotate_stats_union-7-aab1a7b7e1fe4d061a580126d67dfd0a diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/annotate_stats_union-8-172e0d089bd5bcbaf54775a618d826bb diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/annotate_stats_union-9-69b422026fce4e2d2cde9a478d6aaa40 diff --git a/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 deleted file mode 100644 index 21b3b13a81191..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 +++ /dev/null @@ -1 +0,0 @@ -0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 deleted file mode 100644 index 5e5f6ff96623f..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 +++ /dev/null @@ -1,9 +0,0 @@ -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 deleted file mode 100644 index 21b3b13a81191..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 +++ /dev/null @@ -1 +0,0 @@ -0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 deleted file mode 100644 index 5e5f6ff96623f..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 +++ /dev/null @@ -1,9 +0,0 @@ -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 deleted file mode 100644 index 69ca68f501ff1..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 +++ /dev/null @@ -1,6 +0,0 @@ -0 -0 -0 -10 -20 -30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a deleted file mode 100644 index 18a1a7925ff29..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a +++ /dev/null @@ -1 +0,0 @@ -48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e b/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 deleted file mode 100644 index 21b3b13a81191..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 +++ /dev/null @@ -1 +0,0 @@ -0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 deleted file mode 100644 index 5e5f6ff96623f..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 +++ /dev/null @@ -1,9 +0,0 @@ -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 -0 val_0 2008-04-08 12 0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb deleted file mode 100644 index 5cd5fb9874d67..0000000000000 --- a/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb +++ /dev/null @@ -1 +0,0 @@ -48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba +++ b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c index 308fc0924e670..8e9be7f8c620b 100644 --- a/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c +++ b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c @@ -1 +1 @@ -34298511120 \ No newline at end of file +34298511120 diff --git a/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a +++ b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e +++ b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 +++ b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 index 69dbf8c3143e9..faeebdd1eafdf 100644 --- a/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 +++ b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 @@ -1 +1 @@ --101339664144 \ No newline at end of file +-101339664144 diff --git a/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 index eff107c7ce6bc..caf74830fbd05 100644 --- a/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 +++ b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 @@ -1 +1 @@ --136852761207 \ No newline at end of file +-136852761207 diff --git a/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 index de6c015da2059..e971081015515 100644 --- a/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 +++ b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 @@ -1 +1 @@ --97676500536 \ No newline at end of file +-97676500536 diff --git a/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e +++ b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 +++ b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 index f1871a4957ddb..0f27a9bde401c 100644 --- a/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 +++ b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 @@ -1 +1 @@ -404554174174 \ No newline at end of file +404554174174 diff --git a/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 index 006e1f82c0a47..31d87ddbd1310 100644 --- a/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 +++ b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 @@ -1 +1 @@ --793937029770 \ No newline at end of file +-793937029770 diff --git a/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 b/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c index 006e1f82c0a47..31d87ddbd1310 100644 --- a/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c +++ b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c @@ -1 +1 @@ --793937029770 \ No newline at end of file +-793937029770 diff --git a/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f index 0c9b518e65ece..42af6f2e56afc 100644 --- a/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f +++ b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f @@ -1 +1 @@ -2358131334 \ No newline at end of file +2358131334 diff --git a/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f index 795166629df40..069b64b649977 100644 --- a/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f +++ b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f @@ -1 +1 @@ -407444119660 \ No newline at end of file +407444119660 diff --git a/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 index 1c958900f5013..b8f473bf53aa3 100644 --- a/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 +++ b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 @@ -1 +1 @@ -33815990627 \ No newline at end of file +33815990627 diff --git a/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 index b1a6075f768c8..6b72a1789f57d 100644 --- a/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 +++ b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 @@ -1 +1 @@ --24276731469 \ No newline at end of file +-24276731469 diff --git a/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 index b1a6075f768c8..6b72a1789f57d 100644 --- a/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 +++ b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 @@ -1 +1 @@ --24276731469 \ No newline at end of file +-24276731469 diff --git a/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f index 9672e21fa0323..80c230cf4a09d 100644 --- a/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f +++ b/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f @@ -497,4 +497,4 @@ NULL NULL NULL NULL 496 val_496 NULL NULL NULL NULL 497 val_497 NULL NULL NULL NULL 498 val_498 NULL NULL NULL NULL 498 val_498 -NULL NULL NULL NULL 498 val_498 \ No newline at end of file +NULL NULL NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb index dba3bca53f72d..79162030a2043 100644 --- a/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb +++ b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb @@ -1 +1 @@ -344337359100 \ No newline at end of file +344337359100 diff --git a/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 b/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 index 5707ed08e7e54..13e88f30fc08c 100644 --- a/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 +++ b/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 @@ -97,4 +97,4 @@ 9 val_9 5 val_5 9 val_9 5 val_5 9 val_9 8 val_8 -9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 +++ b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa index 71094ee7360db..16b313fc58f23 100644 --- a/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 311 3 369 3 401 5 -406 4 \ No newline at end of file +406 4 diff --git a/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e index dd35c6b71fc80..a16667d785bf8 100644 --- a/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e +++ b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e @@ -1 +1 @@ -548 \ No newline at end of file +548 diff --git a/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c +++ b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f +++ b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a +++ b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d +++ b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 +++ b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 +++ b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 +++ b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 +++ b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 +++ b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 +++ b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 index 1434bb76ee93f..3afe52bf4b3ae 100644 --- a/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 +++ b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 @@ -1 +1 @@ -344360994461 \ No newline at end of file +344360994461 diff --git a/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_join32-12-4a7d51ed5c1d98c518ea74f73c6c7d6c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_join32-13-ee2dcaae78ae900ffce8d19fbadc3735 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_join32-14-7927c2ce644d1ce1de251405c8563e99 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_join32-14-7927c2ce644d1ce1de251405c8563e99 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_join32-15-4cd3b51861720ac06c6deb818c83670 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_join32-15-4cd3b51861720ac06c6deb818c83670 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_join32-20-693874ea8e06a8b155394ac27af2b1a7 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_join32-20-693874ea8e06a8b155394ac27af2b1a7 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_join32-21-bf8a1bb0baaae9fbf1c3aa656f991f42 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_join32-22-3d14d63e996851f51a98f987995d8da6 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_join32-22-3d14d63e996851f51a98f987995d8da6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_join32-23-8b183ec2c164b3b530e802ffc880a5fa diff --git a/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef +++ b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 index f7d1e92d77207..9e503cc0fbeb9 100644 --- a/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 +++ b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 @@ -1 +1 @@ -5079148035 \ No newline at end of file +5079148035 diff --git a/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 index e859b7c4ada7b..1b298efae95b9 100644 --- a/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 +++ b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 @@ -1 +1 @@ -9766083196 \ No newline at end of file +9766083196 diff --git a/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 index f15ab2fb14eaa..1e9e1b54b2074 100644 --- a/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 +++ b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 @@ -1 +1 @@ -2607643291 \ No newline at end of file +2607643291 diff --git a/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 index 5f7f06c079d24..5ac39f668df1c 100644 --- a/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 +++ b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 @@ -1 +1 @@ --2315698213 \ No newline at end of file +-2315698213 diff --git a/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f index 16f90efbe50f6..d14fbdc94256c 100644 --- a/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f +++ b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f @@ -1 +1 @@ -101861029915 \ No newline at end of file +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/auto_join_filters-2-bee6095f42de6a16708c2f9addc1b9bd diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/auto_join_filters-31-268d8fb3cb9b04eb269fe7ec40a24dfe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a b/sql/hive/src/test/resources/golden/auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a rename to sql/hive/src/test/resources/golden/auto_join_filters-32-6dc6866a65c74d69538b776b41b06c16 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 b/sql/hive/src/test/resources/golden/auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 rename to sql/hive/src/test/resources/golden/auto_join_filters-33-e884480a0f7273d3e2f2de2ba46b855c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 b/sql/hive/src/test/resources/golden/auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 rename to sql/hive/src/test/resources/golden/auto_join_filters-34-98fd86aea9cacaa82d43c7468109dd33 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 index b201b9c4d9348..acf9ebdfa9579 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 @@ -1 +1 @@ -4542003 \ No newline at end of file +4542003 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 index d365cdf04366c..4380aa676ba67 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 @@ -1 +1 @@ -4542038 \ No newline at end of file +4542038 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c index bc7bcdca25bfb..359888de9ce3c 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c @@ -1 +1 @@ -4543491 \ No newline at end of file +4543491 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 index b201b9c4d9348..acf9ebdfa9579 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 @@ -1 +1 @@ -4542003 \ No newline at end of file +4542003 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 index feea6ee0a8e0d..3b196ba0b9f87 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 @@ -1 +1 @@ -3079923 \ No newline at end of file +3079923 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 index f713b04028bbd..dba80cf2f3b4b 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 @@ -1 +1 @@ -4509891 \ No newline at end of file +4509891 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af index a94eda6b2c374..7e29fae3a0aa6 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af @@ -1 +1 @@ -3113558 \ No newline at end of file +3113558 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 index feea6ee0a8e0d..3b196ba0b9f87 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 @@ -1 +1 @@ -3079923 \ No newline at end of file +3079923 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_join_nulls-2-75b1f5331b62fedb7dbbe6ac93a3c83f diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a index 88c5f95e0d838..a4231499b4e56 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a @@ -1 +1 @@ -4543526 \ No newline at end of file +4543526 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 index f2ec932ce57f4..2fa702a938a45 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 @@ -1 +1 @@ -3112070 \ No newline at end of file +3112070 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa index a94eda6b2c374..7e29fae3a0aa6 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa @@ -1 +1 @@ -3113558 \ No newline at end of file +3113558 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 index f2ec932ce57f4..2fa702a938a45 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 @@ -1 +1 @@ -3112070 \ No newline at end of file +3112070 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 index 4125efd2dd065..27994c451682a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 @@ -1 +1 @@ -13630578 \ No newline at end of file +13630578 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 index 4125efd2dd065..27994c451682a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 @@ -1 +1 @@ -13630578 \ No newline at end of file +13630578 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc index 4125efd2dd065..27994c451682a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc @@ -1 +1 @@ -13630578 \ No newline at end of file +13630578 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 index e877d44372ecb..476d8eeee571a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 @@ -1 +1 @@ -3078400 \ No newline at end of file +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 index 18be36a9bdb54..935eec97c5601 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 @@ -1 +1 @@ -4509856 \ No newline at end of file +4509856 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 index f2ec932ce57f4..2fa702a938a45 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 @@ -1 +1 @@ -3112070 \ No newline at end of file +3112070 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e index e877d44372ecb..476d8eeee571a 100644 --- a/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e @@ -1 +1 @@ -3078400 \ No newline at end of file +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-1-2bfb628930d072124636d21d82e3b462 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-4-11af6838bb9e04152c2f9a7e2044abe0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_join_reordering_values-7-99fcaa5203ed3debb52c9086028dc8c2 diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed +++ b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 index 2eafac63a9a98..2a51623eae15c 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 @@ -3,4 +3,4 @@ 4 1 1 5 9 9 8 1 1 -9 1 1 \ No newline at end of file +9 1 1 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e index 3d2e6576f591f..f892bae472dff 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e @@ -19,4 +19,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 index 34d56da297220..74ff4beddf949 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 @@ -19,4 +19,4 @@ 5 val_5 val_5 5 val_5 val_5 8 val_8 val_8 -9 val_9 val_9 \ No newline at end of file +9 val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e index 3d2e6576f591f..f892bae472dff 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e @@ -19,4 +19,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 index 86ee83a4a2686..425151f3a411f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 @@ -1 +1 @@ -40 \ No newline at end of file +40 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e index 301160a93062d..45a4fb75db864 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e @@ -1 +1 @@ -8 \ No newline at end of file +8 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-97ba394ab6aad2547f399ebbf757a4b6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-73ee2d7b62e2aede20ca5de577cd7b7f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-5fb8b113a91fbdb15eb35fe1a1d1b4f diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc index 83be903e06482..5629a958479c7 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc @@ -1 +1 @@ -570 \ No newline at end of file +570 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 index 251ff85eda52d..1a9097317aa62 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -19,4 +19,4 @@ 5 5 5 5 8 8 -9 9 \ No newline at end of file +9 9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 index af8f457e93476..225fc24a2290f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 @@ -19,4 +19,4 @@ val_5 val_5 val_5 val_5 val_5 val_5 val_8 val_8 -val_9 val_9 \ No newline at end of file +val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 index 251ff85eda52d..1a9097317aa62 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -19,4 +19,4 @@ 5 5 5 5 8 8 -9 9 \ No newline at end of file +9 9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 index af8f457e93476..225fc24a2290f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 @@ -19,4 +19,4 @@ val_5 val_5 val_5 val_5 val_5 val_5 val_8 val_8 -val_9 val_9 \ No newline at end of file +val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 index 251ff85eda52d..1a9097317aa62 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 @@ -19,4 +19,4 @@ 5 5 5 5 8 8 -9 9 \ No newline at end of file +9 9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 index af8f457e93476..225fc24a2290f 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 @@ -19,4 +19,4 @@ val_5 val_5 val_5 val_5 val_5 val_5 val_8 val_8 -val_9 val_9 \ No newline at end of file +val_9 val_9 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a index 1758dddccea2b..f5c89552bd3e6 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a @@ -1 +1 @@ -32 \ No newline at end of file +32 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd index bea0d09c49935..c92ba56847bf4 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd @@ -1 +1 @@ -207 \ No newline at end of file +207 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-0-24ca942f094b14b92086305cc125e833 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-1-16367c381d4b189b3640c92511244bfe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-10-4706d21b17f993cc1cc94be6b7e04c28 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-11-5e81d0b41cc58d8cc151046c7a111411 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-12-e8a77ff790bfd6740489fc4374ec6c3d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-13-920818d557d2525dabb5c539a80a1bbb diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-14-455dfeeba27ecf53923db0cbf0aab908 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-15-11f98f575685beedc14a88fc47a61041 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b new file mode 100644 index 0000000000000..2da41004e8914 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-16-dff02d7b4c5242434d5e7449bdb67f8b @@ -0,0 +1,24 @@ +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +0 val_0 val_0 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +169 val_169 val_169 day1 1 +374 val_374 val_374 day1 1 +374 val_374 val_374 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +172 val_172 val_172 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 +103 val_103 val_103 day1 1 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-2-7b4ad215fc2e75c71c6614a2b6322e8e diff --git a/sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06 similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-3-365488a703b0640acda73a7d7e6efa06 diff --git a/sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-4-d0ec6d66ff349db09fd455eec149efdb diff --git a/sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-5-3b0f76816be2c1b18a2058027a19bc9f diff --git a/sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-6-86473a0498e4361e4db0b4a22f2e8571 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-7-7e87a31677022b6a1c360922ef74754e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-8-17d5c9dd1a25e8a54dc9c7444cbe98c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_16-9-ae5880516ea2f924cfbaeb919adc86e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-aa6ec7f17b48cf49f02ab7367453ab39 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-66b07c93d79ed9958b8427dad16c3ef3 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 index c24b6ae77df02..e522732c77ec9 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -38 \ No newline at end of file +38 diff --git a/sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-aa6ec7f17b48cf49f02ab7367453ab39 diff --git a/sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-66b07c93d79ed9958b8427dad16c3ef3 diff --git a/sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5 similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-ecca12a2f377c18c53563a534e9dd5a5 diff --git a/sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d similarity index 100% rename from sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-65930e1b01da720cf296ca3df668b58d diff --git a/sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e similarity index 100% rename from sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fdbb11a2de2777dfdd916b59764d5c8e diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 index dec2bf5d6199c..d6b24041cf041 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -19 \ No newline at end of file +19 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 index dec2bf5d6199c..d6b24041cf041 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -19 \ No newline at end of file +19 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed similarity index 100% rename from sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-4f56cb50ec6c5cc57974f85d54bcc8ed diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 index dec2bf5d6199c..d6b24041cf041 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -19 \ No newline at end of file +19 diff --git a/sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-9878d6ab0fe143611c88ffc0602197e3 diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-a576657b9fd1c23aed85a7409887b2fb diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-350b202868590b5edaed18007fd5cbbe diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-1155788c7c133a73c3609e8052accfa5 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 index 92c15ec11569f..f3653aba2d12e 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 @@ -1 +1 @@ -2654 \ No newline at end of file +2654 diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-aa6ec7f17b48cf49f02ab7367453ab39 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-66b07c93d79ed9958b8427dad16c3ef3 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-ecca12a2f377c18c53563a534e9dd5a5 diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-65930e1b01da720cf296ca3df668b58d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-4e24a5c0c67a137478e4a8be2a081872 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-962264967269db1d5f28a9a6c60dbf1 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-10b03ce2526bf180faaec9310cfab290 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-5cba470fbd02e730781a3b63fd9aa3e2 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06 similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-337e909605c780d00ad8895686defa06 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe similarity index 100% rename from sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-f42438f3f5c266b997686ba846420ebe diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 index aa92725341cfd..dd475631baeff 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 @@ -1 +1 @@ -76 \ No newline at end of file +76 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-47a16cf1686c81c5ba76fd92fa5e05a1 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-45b63361c1e5178b69a1531b238c8460 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-9e58f8a961723c40a5d1f742251a8fa5 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-d964114ed76536c8e3cacd231340851c diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-e44aff8a860cf3965752d3e1ce725cde diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 rename to sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-ae582a255a02d91674aab500aee79e20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 index 2eafac63a9a98..2a51623eae15c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 @@ -3,4 +3,4 @@ 4 1 1 5 9 9 8 1 1 -9 1 1 \ No newline at end of file +9 1 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 index 2eafac63a9a98..2a51623eae15c 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 @@ -3,4 +3,4 @@ 4 1 1 5 9 9 8 1 1 -9 1 1 \ No newline at end of file +9 1 1 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index d0eea8a3ca661..0000000000000 --- a/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary None -ba_val binary None - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426673, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426673}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index 0a3c9f383a6ef..0000000000000 --- a/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary from deserializer -ba_val binary from deserializer - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426674, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426674, transient_lastDdlTime=1392426674}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index 66d33b789f1a7..0000000000000 --- a/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary None -ba_val binary None - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426676, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426676}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 b/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 deleted file mode 100644 index dcd1d8643e3cb..0000000000000 --- a/sql/hive/src/test/resources/golden/between-0-df3cf89fcf2ef64199a582fae14a3321 +++ /dev/null @@ -1 +0,0 @@ -2 val_2 diff --git a/sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c rename to sql/hive/src/test/resources/golden/binary_constant-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 b/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 deleted file mode 100644 index 2e65efe2a145d..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 +++ /dev/null @@ -1 +0,0 @@ -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 b/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 new file mode 100644 index 0000000000000..78981922613b2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/binary_constant-1-604fde0397444c5c9f1d70e6287de664 @@ -0,0 +1 @@ +a diff --git a/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 b/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 deleted file mode 100644 index c5c8d29fdd13e..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 +++ /dev/null @@ -1,500 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -302 val_302 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index df14abbcc193f..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary from deserializer -ba_val binary from deserializer - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426678, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426678, transient_lastDdlTime=1392426678}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 deleted file mode 100644 index 0c246ae33e56f..0000000000000 --- a/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 +++ /dev/null @@ -1,4 +0,0 @@ -ba_key binary from deserializer -ba_val binary from deserializer - -Detailed Table Information Table(tableName:ba_test, dbName:default, owner:tnachen, createTime:1392426679, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426679}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 b/sql/hive/src/test/resources/golden/binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 rename to sql/hive/src/test/resources/golden/binarysortable_1-1-aca7ae366662c9698b5d2c01a6cb3948 diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 b/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 index 27687b47813a9..d6ee76110e8a9 100644 --- a/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 +++ b/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 @@ -7,4 +7,4 @@ a^@bc^A^B^A^@ 1.0 test^@^@^A^Atest 6.0 test^@test 4.0 -test^Atest 5.0 \ No newline at end of file +test^Atest 5.0 diff --git a/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a b/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a index 9bdd310949be8..b62097939c833 100644 --- a/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a +++ b/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a @@ -1 +1 @@ -5 5.0 5.0 5.0 5 true 1 \ No newline at end of file +5 5.0 5.0 5.0 5 true 1 diff --git a/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 b/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 deleted file mode 100644 index bbe268ea91ddf..0000000000000 --- a/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 +++ /dev/null @@ -1 +0,0 @@ -1.0 1.4 1.6 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 NULL 2147483647 -2147483648 32767 -32768 -128 127 NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 rename to sql/hive/src/test/resources/golden/char_varchar_udf-0-4de1b2fa0908a3d856474aae2bc38c08 diff --git a/sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 rename to sql/hive/src/test/resources/golden/char_varchar_udf-1-5b1e7c580ed5e756d30a4c557af8902 diff --git a/sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 rename to sql/hive/src/test/resources/golden/char_varchar_udf-2-880ce74a83bb2bfb9c9bd584b8874ac diff --git a/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 index 87d0f8dd52b68..ae8b7551d9630 100644 --- a/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 +++ b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 @@ -1 +1 @@ -10 val_10 \ No newline at end of file +10 val_10 diff --git a/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e +++ b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 +++ b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 +++ b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 index 1a018b640eb6f..e2ef71e141c46 100644 --- a/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 +++ b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 @@ -1 +1 @@ -20 val_20 20 \ No newline at end of file +20 val_20 20 diff --git a/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 index 333a4cf9af123..5b04be3cdf046 100644 --- a/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 +++ b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 @@ -1 +1 @@ -20 val_20 20 val_20 \ No newline at end of file +20 val_20 20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f index 333a4cf9af123..5b04be3cdf046 100644 --- a/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f +++ b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f @@ -1 +1 @@ -20 val_20 20 val_20 \ No newline at end of file +20 val_20 20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e index 1a018b640eb6f..e2ef71e141c46 100644 --- a/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e +++ b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e @@ -1 +1 @@ -20 val_20 20 \ No newline at end of file +20 val_20 20 diff --git a/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a b/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a index a79654385b09f..764cbe5f62b60 100644 --- a/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a +++ b/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a @@ -495,4 +495,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f +++ b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 +++ b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 +++ b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 index 54864d264245d..a9169afab64e7 100644 --- a/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 +++ b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 @@ -1 +1 @@ -20 val_20 \ No newline at end of file +20 val_20 diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f b/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f deleted file mode 100644 index 15e92afeeca27..0000000000000 --- a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 b/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 deleted file mode 100644 index 6ad4db1788424..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 +++ /dev/null @@ -1,58 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 2000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=2000.0 - Table: Employee_Part - diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 b/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 deleted file mode 100644 index 91ce2a521cde1..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 +++ /dev/null @@ -1,129 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 2000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - GatherStats: false - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Path -> Alias: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 [employee_part] - Path -> Partition: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 - Partition - base file name: employeesalary=2000.0 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - employeesalary 2000.0 - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=2000.0 - name default.employee_part - numFiles 1 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 105 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part - name default.employee_part - numFiles 2 - numPartitions 2 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 210 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.employee_part - name: default.employee_part - Truncated Path -> Alias: - /employee_part/employeesalary=2000.0 [employee_part] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-07_241_106202206012377173-1/-ext-10001 - NumFilesPerFileSink: 1 - Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-07_241_106202206012377173-1/-ext-10001/ - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - columns _col0 - columns.types struct - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=2000.0 - Table: Employee_Part - Is Table Level Stats: false - diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 b/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 deleted file mode 100644 index 777024f6946e3..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 +++ /dev/null @@ -1,58 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 4000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=4000.0 - Table: Employee_Part - diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce b/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce deleted file mode 100644 index cd72c7efbf56f..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce +++ /dev/null @@ -1,129 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME Employee_Part) (TOK_PARTSPEC (TOK_PARTVAL employeeSalary 4000.0))) (TOK_TABCOLNAME employeeID)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - employee_part - TableScan - alias: employee_part - GatherStats: false - Select Operator - expressions: - expr: employeeid - type: int - outputColumnNames: employeeid - Group By Operator - aggregations: - expr: compute_stats(employeeid, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - Path -> Alias: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 [employee_part] - Path -> Partition: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 - Partition - base file name: employeesalary=4000.0 - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - partition values: - employeesalary 4000.0 - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part/employeesalary=4000.0 - name default.employee_part - numFiles 1 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 105 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns employeeid,employeename - columns.types int:string - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/employee_part - name default.employee_part - numFiles 2 - numPartitions 2 - numRows 0 - partition_columns employeesalary - rawDataSize 0 - serialization.ddl struct employee_part { i32 employeeid, string employeename} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 210 - transient_lastDdlTime 1389728706 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.employee_part - name: default.employee_part - Truncated Path -> Alias: - /employee_part/employeesalary=4000.0 [employee_part] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0 - Select Operator - expressions: - expr: _col0 - type: struct - outputColumnNames: _col0 - File Output Operator - compressed: false - GlobalTableId: 0 - directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-24_849_6968895828655634809-1/-ext-10001 - NumFilesPerFileSink: 1 - Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-24_849_6968895828655634809-1/-ext-10001/ - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - columns _col0 - columns.types struct - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: employeeID - Column Types: int - Partition: employeesalary=4000.0 - Table: Employee_Part - Is Table Level Stats: false - diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 deleted file mode 100644 index d1e5e7375467d..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 +++ /dev/null @@ -1,73 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME UserVisits_web_text_none)) (TOK_TABCOLNAME sourceIP avgTimeOnSite adRevenue)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - uservisits_web_text_none - TableScan - alias: uservisits_web_text_none - Select Operator - expressions: - expr: sourceip - type: string - expr: avgtimeonsite - type: int - expr: adrevenue - type: float - outputColumnNames: sourceip, avgtimeonsite, adrevenue - Group By Operator - aggregations: - expr: compute_stats(sourceip, 16) - expr: compute_stats(avgtimeonsite, 16) - expr: compute_stats(adrevenue, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0, _col1, _col2 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - expr: compute_stats(VALUE._col1) - expr: compute_stats(VALUE._col2) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: sourceIP, avgTimeOnSite, adRevenue - Column Types: string, int, float - Table: UserVisits_web_text_none - diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 deleted file mode 100644 index 3f3aa581b43f9..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 +++ /dev/null @@ -1,141 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME UserVisits_web_text_none)) (TOK_TABCOLNAME sourceIP avgTimeOnSite adRevenue)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - uservisits_web_text_none - TableScan - alias: uservisits_web_text_none - GatherStats: false - Select Operator - expressions: - expr: sourceip - type: string - expr: avgtimeonsite - type: int - expr: adrevenue - type: float - outputColumnNames: sourceip, avgtimeonsite, adrevenue - Group By Operator - aggregations: - expr: compute_stats(sourceip, 16) - expr: compute_stats(avgtimeonsite, 16) - expr: compute_stats(adrevenue, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0, _col1, _col2 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - Path -> Alias: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none [uservisits_web_text_none] - Path -> Partition: - file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none - Partition - base file name: uservisits_web_text_none - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns sourceip,desturl,visitdate,adrevenue,useragent,ccode,lcode,skeyword,avgtimeonsite - columns.types string:string:string:float:string:string:string:string:int - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none - name default.uservisits_web_text_none - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct uservisits_web_text_none { string sourceip, string desturl, string visitdate, float adrevenue, string useragent, string ccode, string lcode, string skeyword, i32 avgtimeonsite} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 7060 - transient_lastDdlTime 1389728748 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns sourceip,desturl,visitdate,adrevenue,useragent,ccode,lcode,skeyword,avgtimeonsite - columns.types string:string:string:float:string:string:string:string:int - field.delim | - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7107609744565894054/uservisits_web_text_none - name default.uservisits_web_text_none - numFiles 1 - numPartitions 0 - numRows 0 - rawDataSize 0 - serialization.ddl struct uservisits_web_text_none { string sourceip, string desturl, string visitdate, float adrevenue, string useragent, string ccode, string lcode, string skeyword, i32 avgtimeonsite} - serialization.format | - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - totalSize 7060 - transient_lastDdlTime 1389728748 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.uservisits_web_text_none - name: default.uservisits_web_text_none - Truncated Path -> Alias: - /uservisits_web_text_none [uservisits_web_text_none] - Needs Tagging: false - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - expr: compute_stats(VALUE._col1) - expr: compute_stats(VALUE._col2) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0, _col1, _col2 - Select Operator - expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - outputColumnNames: _col0, _col1, _col2 - File Output Operator - compressed: false - GlobalTableId: 0 - directory: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-49_160_8862102294255849057-1/-ext-10001 - NumFilesPerFileSink: 1 - Stats Publishing Key Prefix: file:/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/marmbrus/hive_2014-01-14_11-45-49_160_8862102294255849057-1/-ext-10001/ - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - columns _col0,_col1,_col2 - columns.types struct:struct:struct - escape.delim \ - hive.serialization.extend.nesting.levels true - serialization.format 1 - TotalFiles: 1 - GatherStats: false - MultiFileSpray: false - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: sourceIP, avgTimeOnSite, adRevenue - Column Types: string, int, float - Table: UserVisits_web_text_none - Is Table Level Stats: true - diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d b/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d deleted file mode 100644 index 4ff444febde63..0000000000000 --- a/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d +++ /dev/null @@ -1,89 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME empty_tab)) (TOK_TABCOLNAME a b c d e)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - empty_tab - TableScan - alias: empty_tab - Select Operator - expressions: - expr: a - type: int - expr: b - type: double - expr: c - type: string - expr: d - type: boolean - expr: e - type: binary - outputColumnNames: a, b, c, d, e - Group By Operator - aggregations: - expr: compute_stats(a, 16) - expr: compute_stats(b, 16) - expr: compute_stats(c, 16) - expr: compute_stats(d, 16) - expr: compute_stats(e, 16) - bucketGroup: false - mode: hash - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Reduce Output Operator - sort order: - tag: -1 - value expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - expr: _col3 - type: struct - expr: _col4 - type: struct - Reduce Operator Tree: - Group By Operator - aggregations: - expr: compute_stats(VALUE._col0) - expr: compute_stats(VALUE._col1) - expr: compute_stats(VALUE._col2) - expr: compute_stats(VALUE._col3) - expr: compute_stats(VALUE._col4) - bucketGroup: false - mode: mergepartial - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - Select Operator - expressions: - expr: _col0 - type: struct - expr: _col1 - type: struct - expr: _col2 - type: struct - expr: _col3 - type: struct - expr: _col4 - type: struct - outputColumnNames: _col0, _col1, _col2, _col3, _col4 - File Output Operator - compressed: false - GlobalTableId: 0 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - - Stage: Stage-1 - Column Stats Work - Column Stats Desc: - Columns: a, b, c, d, e - Column Types: int, double, string, boolean, binary - Table: empty_tab - diff --git a/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 +++ b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf +++ b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 +++ b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 +++ b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea +++ b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 +++ b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 b/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 index 8f8e1f4b21fe3..b40eac432935c 100644 --- a/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 +++ b/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 @@ -497,4 +497,4 @@ 97 val_97 97 val_97 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 deleted file mode 100644 index 80fa68b84c17e..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 +++ /dev/null @@ -1,8 +0,0 @@ -value=2010-04-21 09%3A45%3A00 -value=val_0 -value=val_2 -value=val_4 -value=val_5 -value=val_8 -value=val_9 -value=| \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c deleted file mode 100644 index ff6141674e603..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c +++ /dev/null @@ -1,12 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -11 | -19 2010-04-21 09:45:00 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 deleted file mode 100644 index 3cacc0b93c9c9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 +++ /dev/null @@ -1 +0,0 @@ -12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 deleted file mode 100644 index 1a0aa74952afa..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 +++ /dev/null @@ -1,2 +0,0 @@ -2008-04-08 1000 -2008-04-09 1000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 deleted file mode 100644 index 80fa68b84c17e..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 +++ /dev/null @@ -1,8 +0,0 @@ -value=2010-04-21 09%3A45%3A00 -value=val_0 -value=val_2 -value=val_4 -value=val_5 -value=val_8 -value=val_9 -value=| \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c deleted file mode 100644 index ff6141674e603..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c +++ /dev/null @@ -1,12 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -11 | -19 2010-04-21 09:45:00 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 deleted file mode 100644 index 3cacc0b93c9c9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 +++ /dev/null @@ -1 +0,0 @@ -12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 deleted file mode 100644 index 1a0aa74952afa..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 +++ /dev/null @@ -1,2 +0,0 @@ -2008-04-08 1000 -2008-04-09 1000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 b/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 deleted file mode 100644 index 80665a4d4c983..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 +++ /dev/null @@ -1,12 +0,0 @@ -key int None -value string None -ds string None -hr string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -Detailed Partition Information Partition(values:[2010-08-03, 00], dbName:default, tableName:combine_3_srcpart_seq_rc, createTime:1390898644, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:hr, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/combine_3_srcpart_seq_rc/ds=2010-08-03/hr=00, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390898644, numRows=500, totalSize=15250, rawDataSize=5312}) diff --git a/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b b/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b deleted file mode 100644 index 5a87a3aec7cf5..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b +++ /dev/null @@ -1,12 +0,0 @@ -key int from deserializer -value string from deserializer -ds string None -hr string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -Detailed Partition Information Partition(values:[2010-08-03, 001], dbName:default, tableName:combine_3_srcpart_seq_rc, createTime:1390898653, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/combine_3_srcpart_seq_rc/ds=2010-08-03/hr=001, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390898653, numRows=500, totalSize=2202, rawDataSize=4551}) diff --git a/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc b/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc deleted file mode 100644 index 7b6455db7834b..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc +++ /dev/null @@ -1,30 +0,0 @@ -0 val_0 2010-08-03 00 -0 val_0 2010-08-03 00 -0 val_0 2010-08-03 00 -0 val_0 2010-08-03 001 -0 val_0 2010-08-03 001 -0 val_0 2010-08-03 001 -2 val_2 2010-08-03 00 -2 val_2 2010-08-03 001 -4 val_4 2010-08-03 00 -4 val_4 2010-08-03 001 -5 val_5 2010-08-03 00 -5 val_5 2010-08-03 00 -5 val_5 2010-08-03 00 -5 val_5 2010-08-03 001 -5 val_5 2010-08-03 001 -5 val_5 2010-08-03 001 -8 val_8 2010-08-03 00 -8 val_8 2010-08-03 001 -9 val_9 2010-08-03 00 -9 val_9 2010-08-03 001 -10 val_10 2010-08-03 00 -10 val_10 2010-08-03 001 -11 val_11 2010-08-03 00 -11 val_11 2010-08-03 001 -12 val_12 2010-08-03 00 -12 val_12 2010-08-03 00 -12 val_12 2010-08-03 001 -12 val_12 2010-08-03 001 -15 val_15 2010-08-03 00 -15 val_15 2010-08-03 00 diff --git a/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 b/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 deleted file mode 100644 index d57cb5369e219..0000000000000 --- a/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 +++ /dev/null @@ -1,30 +0,0 @@ -0 1 -0 1 -0 1 -0 11 -0 11 -0 11 -2 1 -2 11 -4 1 -4 11 -8 1 -8 11 -10 1 -10 11 -12 1 -12 1 -12 11 -12 11 -18 1 -18 1 -18 11 -18 11 -20 1 -20 11 -24 1 -24 1 -24 11 -24 11 -26 1 -26 1 diff --git a/sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af b/sql/hive/src/test/resources/golden/compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051 similarity index 100% rename from sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af rename to sql/hive/src/test/resources/golden/compute_stats_binary-1-44b15c6c9600109e064a5ea5f9c81051 diff --git a/sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 b/sql/hive/src/test/resources/golden/compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 rename to sql/hive/src/test/resources/golden/compute_stats_boolean-1-72ee4bdd5cea69136940dc40e6890e1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e index dc7b54ad01435..bb95160cb6e07 100644 --- a/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e +++ b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e @@ -1 +1 @@ -33 \ No newline at end of file +33 diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d b/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d index dd487e6fea3ff..279805d381a21 100644 --- a/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d +++ b/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d @@ -1 +1 @@ -{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1} \ No newline at end of file +{"columntype":"Boolean","counttrues":13,"countfalses":19,"countnulls":1} diff --git a/sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 b/sql/hive/src/test/resources/golden/compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 rename to sql/hive/src/test/resources/golden/compute_stats_double-1-8f634b9e334fd58e71844e6283d9794d diff --git a/sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a b/sql/hive/src/test/resources/golden/compute_stats_long-1-9313f166464633b3929707d7ef11d758 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a rename to sql/hive/src/test/resources/golden/compute_stats_long-1-9313f166464633b3929707d7ef11d758 diff --git a/sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b b/sql/hive/src/test/resources/golden/compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b rename to sql/hive/src/test/resources/golden/compute_stats_string-1-3491ef2747a8f34899108d4ae8ebc7eb diff --git a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 index c615f0148ccd1..d35bf9093ca9c 100644 --- a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 +++ b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 @@ -19,9 +19,9 @@ my_stringset struct<> from deserializer my_enumset struct<> from deserializer my_structset struct<> from deserializer optionals struct<> from deserializer -b string None +b string # Partition Information # col_name data_type comment -b string None \ No newline at end of file +b string diff --git a/sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f b/sql/hive/src/test/resources/golden/correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f rename to sql/hive/src/test/resources/golden/correlationoptimizer1-26-5522db58d123d1bec48b6e71c1b258f3 diff --git a/sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd b/sql/hive/src/test/resources/golden/correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd rename to sql/hive/src/test/resources/golden/correlationoptimizer1-32-76aad6bc7d7e4a28c33aca1f0ba30e65 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f new file mode 100644 index 0000000000000..fb15947b3d0bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-33-7722bcc896718b584f78cecdab1fdc9f @@ -0,0 +1,16 @@ +NULL NULL 10 +66 val_66 1 +98 val_98 2 +128 NULL 1 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 NULL 1 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 NULL 1 +401 val_401 5 +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 b/sql/hive/src/test/resources/golden/correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-35-e1d4857548e626bb31d70c096b8d0a95 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f new file mode 100644 index 0000000000000..fb15947b3d0bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer1-36-7722bcc896718b584f78cecdab1fdc9f @@ -0,0 +1,16 @@ +NULL NULL 10 +66 val_66 1 +98 val_98 2 +128 NULL 1 +146 val_146 2 +150 val_150 1 +213 val_213 2 +224 NULL 1 +238 val_238 2 +255 val_255 2 +273 val_273 3 +278 val_278 2 +311 val_311 3 +369 NULL 1 +401 val_401 5 +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-38-ef6502d6b282c8a6d228bba395b24724 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-33-ea87e76dba02a46cb958148333e397b7 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-39-ea87e76dba02a46cb958148333e397b7 diff --git a/sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-41-b79b220859c09354e23b533c105ccbab diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 b/sql/hive/src/test/resources/golden/correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-36-ea87e76dba02a46cb958148333e397b7 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-42-ea87e76dba02a46cb958148333e397b7 diff --git a/sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 b/sql/hive/src/test/resources/golden/correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-44-638e5300f4c892c2bf27bd91a8f81b64 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-39-66010469a9cdb66851da9a727ef9fdad rename to sql/hive/src/test/resources/golden/correlationoptimizer1-45-66010469a9cdb66851da9a727ef9fdad diff --git a/sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b b/sql/hive/src/test/resources/golden/correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b rename to sql/hive/src/test/resources/golden/correlationoptimizer1-47-3514c74c7f68f2d70cc6d51ac46c20 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad b/sql/hive/src/test/resources/golden/correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-42-66010469a9cdb66851da9a727ef9fdad rename to sql/hive/src/test/resources/golden/correlationoptimizer1-48-66010469a9cdb66851da9a727ef9fdad diff --git a/sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 b/sql/hive/src/test/resources/golden/correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-49-b9d963d24994c47c3776dda6f7d3881f diff --git a/sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 b/sql/hive/src/test/resources/golden/correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92 similarity index 100% rename from sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-50-7490df6719cd7e47aa08dbcbc3266a92 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-45-e71195e7d9f557e2abc7f03462d22dba rename to sql/hive/src/test/resources/golden/correlationoptimizer1-51-e71195e7d9f557e2abc7f03462d22dba diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/correlationoptimizer1-52-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 b/sql/hive/src/test/resources/golden/correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-53-73da9fe2b0c2ee26c021ec3f2fa27272 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba b/sql/hive/src/test/resources/golden/correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-48-e71195e7d9f557e2abc7f03462d22dba rename to sql/hive/src/test/resources/golden/correlationoptimizer1-54-e71195e7d9f557e2abc7f03462d22dba diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/correlationoptimizer1-55-b1e2ade89ae898650f0be4f796d8947b diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 b/sql/hive/src/test/resources/golden/correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-57-fcf9bcb522f542637ccdea863b408448 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-58-3070366869308907e54797927805603 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-52-3070366869308907e54797927805603 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-58-3070366869308907e54797927805603 diff --git a/sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 b/sql/hive/src/test/resources/golden/correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-60-dad56e1f06c808b29e5dc8fb0c49efb2 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 b/sql/hive/src/test/resources/golden/correlationoptimizer1-61-3070366869308907e54797927805603 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-55-3070366869308907e54797927805603 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-61-3070366869308907e54797927805603 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-62-b9d963d24994c47c3776dda6f7d3881f diff --git a/sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 b/sql/hive/src/test/resources/golden/correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-63-3cd3fbbbd8ee5c274fe3d6a45126cef4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-58-a6bba6d9b422adb386b35c62cecb548 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-64-a6bba6d9b422adb386b35c62cecb548 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/correlationoptimizer1-65-777edd9d575f3480ca6cebe4be57b1f6 diff --git a/sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f b/sql/hive/src/test/resources/golden/correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5 similarity index 100% rename from sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f rename to sql/hive/src/test/resources/golden/correlationoptimizer1-66-d6bbaf0d40010159095e4cac025c50c5 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 b/sql/hive/src/test/resources/golden/correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-61-a6bba6d9b422adb386b35c62cecb548 rename to sql/hive/src/test/resources/golden/correlationoptimizer1-67-a6bba6d9b422adb386b35c62cecb548 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 index 6c6dc691bbff2..e7c8cc75a0d6c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 @@ -1 +1 @@ -79136 500 3556 15 \ No newline at end of file +79136 500 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 index f4bb720dfd7f8..96d1ad9dd0559 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 @@ -1 +1 @@ -3556 37 3556 25 \ No newline at end of file +3556 37 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 index f4bb720dfd7f8..96d1ad9dd0559 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 @@ -1 +1 @@ -3556 37 3556 25 \ No newline at end of file +3556 37 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e index 4acbb60e81661..716f95a30304b 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e @@ -1 +1 @@ -79136 500 3556 25 \ No newline at end of file +79136 500 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e index 4acbb60e81661..716f95a30304b 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e @@ -1 +1 @@ -79136 500 3556 25 \ No newline at end of file +79136 500 3556 25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 index a95fce80fd7b4..3821ee1926f17 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 @@ -1 +1 @@ -79136 310 \ No newline at end of file +79136 310 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d index c6243d7056353..7e1b7f7408e2d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d @@ -1 +1 @@ -3556 37 3556 15 \ No newline at end of file +3556 37 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 index a95fce80fd7b4..3821ee1926f17 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 @@ -1 +1 @@ -79136 310 \ No newline at end of file +79136 310 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b index 556b77ecfc9eb..a8707661c9399 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b @@ -1 +1 @@ -9992 3531902962 9992 37 \ No newline at end of file +9992 3531902962 9992 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b index 556b77ecfc9eb..a8707661c9399 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b @@ -1 +1 @@ -9992 3531902962 9992 37 \ No newline at end of file +9992 3531902962 9992 37 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d index c6243d7056353..7e1b7f7408e2d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d @@ -1 +1 @@ -3556 37 3556 15 \ No newline at end of file +3556 37 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 index 6c6dc691bbff2..e7c8cc75a0d6c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 @@ -1 +1 @@ -79136 500 3556 15 \ No newline at end of file +79136 500 3556 15 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 index a1a6cbb91955e..9ef7747157bdd 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 @@ -1 +1 @@ -9992 107 3531902962 \ No newline at end of file +9992 107 3531902962 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 b/sql/hive/src/test/resources/golden/correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-26-8bcdcc5f01508f576d7bd6422c939225 rename to sql/hive/src/test/resources/golden/correlationoptimizer4-1-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e index 14e309fdcad89..281e236e8a80d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -1 +1 @@ -13 10 \ No newline at end of file +13 10 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e index 14e309fdcad89..281e236e8a80d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -1 +1 @@ -13 10 \ No newline at end of file +13 10 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 index 1b8c137073e37..1ed2737b0d1fb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 @@ -1 +1 @@ -22 12 \ No newline at end of file +22 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 index 1b8c137073e37..1ed2737b0d1fb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 @@ -1 +1 @@ -22 12 \ No newline at end of file +22 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b index e6c95e600a2c0..35b3da1e4da5c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b @@ -1 +1 @@ -13 12 \ No newline at end of file +13 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf index 0248cc90cb2c6..7b6dfccea7a0c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -1 +1 @@ -21 12 \ No newline at end of file +21 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 b/sql/hive/src/test/resources/golden/correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-32-ef6502d6b282c8a6d228bba395b24724 rename to sql/hive/src/test/resources/golden/correlationoptimizer4-3-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf index 0248cc90cb2c6..7b6dfccea7a0c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -1 +1 @@ -21 12 \ No newline at end of file +21 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 index 0248cc90cb2c6..7b6dfccea7a0c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 @@ -1 +1 @@ -21 12 \ No newline at end of file +21 12 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 index fd3a42ebe67e7..60ee3e8737989 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 @@ -1 +1 @@ -21 14 \ No newline at end of file +21 14 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 index fd3a42ebe67e7..60ee3e8737989 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 @@ -1 +1 @@ -21 14 \ No newline at end of file +21 14 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab b/sql/hive/src/test/resources/golden/correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-35-b79b220859c09354e23b533c105ccbab rename to sql/hive/src/test/resources/golden/correlationoptimizer4-5-b5777cff7c522c4b527f77988e7f6bf1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e index 14e309fdcad89..281e236e8a80d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -1 +1 @@ -13 10 \ No newline at end of file +13 10 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 index 19b8a2aea8f64..8f9dae31cc51c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 @@ -12,4 +12,4 @@ 311 1 311 9 369 1 369 9 401 1 401 25 -406 1 406 16 \ No newline at end of file +406 1 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db index c4a418f59625b..26151f7b6d0f1 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db @@ -12,4 +12,4 @@ 311 311 3 369 369 3 401 401 5 -406 406 4 \ No newline at end of file +406 406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db index c4a418f59625b..26151f7b6d0f1 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db @@ -12,4 +12,4 @@ 311 311 3 369 369 3 401 401 5 -406 406 4 \ No newline at end of file +406 406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de index 9c8189500649e..c7d10af90e353 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de @@ -12,4 +12,4 @@ 311 311 9 369 369 9 401 401 25 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de index 9c8189500649e..c7d10af90e353 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de @@ -12,4 +12,4 @@ 311 311 9 369 369 9 401 401 25 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 index 19b8a2aea8f64..8f9dae31cc51c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 @@ -12,4 +12,4 @@ 311 1 311 9 369 1 369 9 401 1 401 25 -406 1 406 16 \ No newline at end of file +406 1 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 index 0b1ee37d7831c..6eebe80953bf0 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 @@ -34,4 +34,4 @@ 406 1 406 406 1 406 406 1 406 -406 1 406 \ No newline at end of file +406 1 406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 index 7c591d6146fd6..4e3bbb2779bf3 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 @@ -34,4 +34,4 @@ 406 406 16 406 406 16 406 406 16 -406 406 16 \ No newline at end of file +406 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 index 8338433cd5a27..917ab36da2fbb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 @@ -34,4 +34,4 @@ 406 4 1 406 val_406 406 4 1 406 val_406 406 4 1 406 val_406 -406 4 1 406 val_406 \ No newline at end of file +406 4 1 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 index 8338433cd5a27..917ab36da2fbb 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 @@ -34,4 +34,4 @@ 406 4 1 406 val_406 406 4 1 406 val_406 406 4 1 406 val_406 -406 4 1 406 val_406 \ No newline at end of file +406 4 1 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d index 8b1d3cd388fa2..a6942feae66e5 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d @@ -12,4 +12,4 @@ 311 1 311 val_311 9 369 1 369 val_369 9 401 1 401 val_401 25 -406 1 406 val_406 16 \ No newline at end of file +406 1 406 val_406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d index 8b1d3cd388fa2..a6942feae66e5 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d @@ -12,4 +12,4 @@ 311 1 311 val_311 9 369 1 369 val_369 9 401 1 401 val_401 25 -406 1 406 val_406 16 \ No newline at end of file +406 1 406 val_406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d index 8b1d3cd388fa2..a6942feae66e5 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d @@ -12,4 +12,4 @@ 311 1 311 val_311 9 369 1 369 val_369 9 401 1 401 val_401 25 -406 1 406 val_406 16 \ No newline at end of file +406 1 406 val_406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 index 19b8a2aea8f64..8f9dae31cc51c 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 @@ -12,4 +12,4 @@ 311 1 311 9 369 1 369 9 401 1 401 25 -406 1 406 16 \ No newline at end of file +406 1 406 16 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 index 747b650237b53..1a82dfc153565 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 @@ -12,4 +12,4 @@ 311 3 311 val_311 369 3 369 401 5 401 val_401 -406 4 406 val_406 \ No newline at end of file +406 4 406 val_406 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f index 7c0af7229d62a..ecbe52c536ebe 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f @@ -21,4 +21,4 @@ val_273 1 NULL NULL val_278 1 NULL NULL val_311 1 NULL NULL val_401 1 NULL NULL -val_406 1 NULL NULL \ No newline at end of file +val_406 1 NULL NULL diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 index efdd80c9f8b89..2f62508e3342a 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 @@ -10,4 +10,4 @@ 311 val_311 3 369 3 401 val_401 5 -406 val_406 4 \ No newline at end of file +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 index efdd80c9f8b89..2f62508e3342a 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 @@ -10,4 +10,4 @@ 311 val_311 3 369 3 401 val_401 5 -406 val_406 4 \ No newline at end of file +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f index 7c0af7229d62a..ecbe52c536ebe 100644 --- a/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f @@ -21,4 +21,4 @@ val_273 1 NULL NULL val_278 1 NULL NULL val_311 1 NULL NULL val_401 1 NULL NULL -val_406 1 NULL NULL \ No newline at end of file +val_406 1 NULL NULL diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 b/sql/hive/src/test/resources/golden/count-1-c47c4abedf055b4e734232fd2e274d55 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-38-638e5300f4c892c2bf27bd91a8f81b64 rename to sql/hive/src/test/resources/golden/count-1-c47c4abedf055b4e734232fd2e274d55 diff --git a/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 index 4b020e0595d2c..966f27f6c9b9b 100644 --- a/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 +++ b/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 @@ -1,4 +1,4 @@ NULL 1 1 6 10 2 2 10 12 1 2 9 -100 1 1 3 \ No newline at end of file +100 1 1 3 diff --git a/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 index 54199fdb8166d..5eec149bfcc92 100644 --- a/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 +++ b/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 @@ -1 +1 @@ -7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 \ No newline at end of file +7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 diff --git a/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 b/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 index 2e9278da88858..162877fdb3e5d 100644 --- a/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 +++ b/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 @@ -4,4 +4,4 @@ NULL 35 23 6 12 NULL 80 2 10 100 NULL 5 10 100 45 4 -12 100 75 7 \ No newline at end of file +12 100 75 7 diff --git a/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 index 4b020e0595d2c..966f27f6c9b9b 100644 --- a/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 +++ b/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 @@ -1,4 +1,4 @@ NULL 1 1 6 10 2 2 10 12 1 2 9 -100 1 1 3 \ No newline at end of file +100 1 1 3 diff --git a/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 index 54199fdb8166d..5eec149bfcc92 100644 --- a/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 +++ b/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 @@ -1 +1 @@ -7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 \ No newline at end of file +7 7 6 6 6 7 3 3 6 7 4 5 6 6 5 6 4 5 5 5 4 diff --git a/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f b/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f deleted file mode 100644 index 7aae61e5eb82f..0000000000000 --- a/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f +++ /dev/null @@ -1,500 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -302 val_302 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 diff --git a/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 b/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 +++ b/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba b/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba deleted file mode 100644 index 66d40e52a4539..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table3, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 b/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a b/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a deleted file mode 100644 index 05b460270525d..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table4, dbName:default, owner:marmbrus, createTime:1389343869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table4, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 b/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 deleted file mode 100644 index eda3946588e3f..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 +++ /dev/null @@ -1,2 +0,0 @@ -a string from deserializer -b string from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 b/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 deleted file mode 100644 index 8fc60adf10167..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 +++ /dev/null @@ -1,4 +0,0 @@ -a string from deserializer -b string from deserializer - -Detailed Table Information Table(tableName:table5, dbName:default, owner:marmbrus, createTime:1389343869, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table5, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 deleted file mode 100644 index 5e9bc70c3a1a3..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389343868}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a deleted file mode 100644 index 6ed3515ba6876..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b int None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 b/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 deleted file mode 100644 index bec4c72e49857..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b int None - -Detailed Table Information Table(tableName:table2, dbName:default, owner:marmbrus, createTime:1389343868, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/table2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389343868}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 deleted file mode 100644 index 8e606f61a1c30..0000000000000 --- a/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 +++ /dev/null @@ -1,2 +0,0 @@ -a string None -b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 deleted file mode 100644 index 8ab6b24deab60..0000000000000 --- a/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 +++ /dev/null @@ -1,4 +0,0 @@ -a string None -b string None - -Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1388825524, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3286459698772672096/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{escape.delim=\, serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 deleted file mode 100644 index bbe37f8e2a790..0000000000000 --- a/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 +++ /dev/null @@ -1 +0,0 @@ -86 \ \ \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 b/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 deleted file mode 100644 index cbcdfbe72e8c6..0000000000000 --- a/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 +++ /dev/null @@ -1,9 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_avg 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 b/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 deleted file mode 100644 index 2f958c483a9df..0000000000000 --- a/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 +++ /dev/null @@ -1 +0,0 @@ -1.0 260.182 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 b/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 index 0e9c06c882602..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 +++ b/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_translate 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestTranslate') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc index 83f572c424926..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc +++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc @@ -1,2 +1,2 @@ -key int None -value string None \ No newline at end of file +key int +value string diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c index 83f572c424926..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c +++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c @@ -1,2 +1,2 @@ -key int None -value string None \ No newline at end of file +key int +value string diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 index 83f572c424926..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 +++ b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 @@ -1,2 +1,2 @@ -key int None -value string None \ No newline at end of file +key int +value string diff --git a/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af deleted file mode 100644 index b8ddba7f50b97..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -a string None -b string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 11:13:23 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392059603 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 deleted file mode 100644 index e87fb81a6f043..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 +++ /dev/null @@ -1 +0,0 @@ -86 val_86 diff --git a/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 b/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 deleted file mode 100644 index 375dcacb8db71..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 +++ /dev/null @@ -1,2 +0,0 @@ -100 val_100 -100 val_100 diff --git a/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 deleted file mode 100644 index 52b637c27f98d..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -a string None -b string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 11:13:23 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table2 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392059603 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 deleted file mode 100644 index d9308798dcb1c..0000000000000 --- a/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 +++ /dev/null @@ -1,28 +0,0 @@ -# col_name data_type comment - -a string None -b string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 11:13:23 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/table3 -Table Type: EXTERNAL_TABLE -Table Parameters: - EXTERNAL TRUE - transient_lastDdlTime 1392059603 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 index 3e2a912824ab3..0e52b5f82ef4a 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 +++ b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 @@ -1,19 +1,19 @@ # col_name data_type comment -a string None -b string None +a string +b string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:52:37 PST 2014 +CreateTime: Tue Oct 21 01:24:36 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table2 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table2 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1391813557 + transient_lastDdlTime 1413879876 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 index 115d12fb70c81..26b5989488752 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 +++ b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 @@ -1,2 +1,2 @@ -a string None -b string None +a string +b string diff --git a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 index 61d34badb1a2d..47808e8f20c83 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 +++ b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 @@ -1,20 +1,20 @@ # col_name data_type comment -a string None -b string None +a string +b string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:52:37 PST 2014 +CreateTime: Tue Oct 21 01:24:36 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table3 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table3 Table Type: EXTERNAL_TABLE Table Parameters: EXTERNAL TRUE - transient_lastDdlTime 1391813557 + transient_lastDdlTime 1413879876 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d index 4c2967215fe66..ad1f1f42d6b98 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d +++ b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d @@ -1,25 +1,25 @@ # col_name data_type comment -key int None -value string None +key int +value string # Partition Information # col_name data_type comment -ds string None -hr string None +ds string +hr string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:53:16 PST 2014 +CreateTime: Tue Oct 21 01:25:10 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1391813596 + transient_lastDdlTime 1413879910 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 index 115d12fb70c81..26b5989488752 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 +++ b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 @@ -1,2 +1,2 @@ -a string None -b string None +a string +b string diff --git a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af index 02dee147bca42..91e1ebbfee4de 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af +++ b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af @@ -1,19 +1,19 @@ # col_name data_type comment -a string None -b string None +a string +b string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Feb 07 14:52:37 PST 2014 +CreateTime: Tue Oct 21 01:24:36 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3766080982052101504/table1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1391813557 + transient_lastDdlTime 1413879876 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe diff --git a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a index 115d12fb70c81..26b5989488752 100644 --- a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a +++ b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a @@ -1,2 +1,2 @@ -a string None -b string None +a string +b string diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 deleted file mode 100644 index dbdf4585360ae..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:tgt_rc_merge_test -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/tgt_rc_merge_test -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:239 -maxFileSize:239 -minFileSize:239 -lastAccessTime:0 -lastUpdateTime:1389344017000 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d deleted file mode 100644 index 9a037142aa3c1..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 deleted file mode 100644 index eb6c3f6aef813..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 +++ /dev/null @@ -1 +0,0 @@ -508 -751895388 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 deleted file mode 100644 index 8a9d9d56a66d6..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:tgt_rc_merge_test -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5689195829966714752/tgt_rc_merge_test -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:2 -totalFileSize:338 -maxFileSize:169 -minFileSize:169 -lastAccessTime:0 -lastUpdateTime:1389343990000 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d deleted file mode 100644 index 9a037142aa3c1..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d +++ /dev/null @@ -1 +0,0 @@ -10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 deleted file mode 100644 index eb6c3f6aef813..0000000000000 --- a/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 +++ /dev/null @@ -1 +0,0 @@ -508 -751895388 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 index c49ad26c04d67..6485e594f54fe 100644 --- a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 +++ b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 @@ -1,4 +1,4 @@ -a string None -b array None -c array> None -d map> None +a string +b array +c array> +d map> diff --git a/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 index 41c7202c8b886..6d68db6387495 100644 --- a/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 +++ b/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 @@ -1,6 +1,6 @@ -a string None -b array None -c array> None -d map> None +a string +b array +c array> +d map> -Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1391226109, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:array, comment:null), FieldSchema(name:c, type:array>, comment:null), FieldSchema(name:d, type:map>, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1391226109}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:table1, dbName:default, owner:marmbrus, createTime:1413879912, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:array, comment:null), FieldSchema(name:c, type:array>, comment:null), FieldSchema(name:d, type:map>, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/table1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413879912}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 b/sql/hive/src/test/resources/golden/create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-41-3514c74c7f68f2d70cc6d51ac46c20 rename to sql/hive/src/test/resources/golden/create_nested_type-3-280fd0d9876e475d7dcf5383876ebc79 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 b/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 deleted file mode 100644 index 46869fc9e88eb..0000000000000 --- a/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 +++ /dev/null @@ -1,30 +0,0 @@ -# col_name data_type comment - -key int None -value string None -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 12:09:28 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Table Type: VIRTUAL_VIEW -Table Parameters: - transient_lastDdlTime 1392062968 - -# Storage Information -SerDe Library: null -InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] - -# View Information -View Original Text: select * from srcpart -View Expanded Text: select `srcpart`.`key`, `srcpart`.`value`, `srcpart`.`ds`, `srcpart`.`hr` from `default`.`srcpart` diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 b/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 index d7c386e17c096..d35fc35a810d3 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 @@ -1,19 +1,19 @@ # col_name data_type comment -key string None -value string None +key string +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Tue Jan 14 22:51:44 PST 2014 +CreateTime: Tue Oct 21 01:25:13 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single_2 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_single_2 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389768704 + transient_lastDdlTime 1413879913 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -26,4 +26,4 @@ Sort Columns: [] Skewed Columns: [key] Skewed Values: [[1], [5], [6]] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e b/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e index 0817efac83d79..e8f72845c1c7a 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e @@ -1,19 +1,19 @@ # col_name data_type comment -key string None -value string None +key string +value string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Tue Jan 14 22:51:44 PST 2014 +CreateTime: Tue Oct 21 01:25:13 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_single +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_single Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389768704 + transient_lastDdlTime 1413879913 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -26,4 +26,4 @@ Sort Columns: [] Skewed Columns: [key] Skewed Values: [[1], [5], [6]] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef b/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef index 2986dd43b0fd6..c8d58bbb1b1ce 100644 --- a/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef @@ -1,20 +1,20 @@ # col_name data_type comment -col1 string None -col2 int None -col3 string None +col1 string +col2 int +col3 string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Tue Jan 14 22:51:44 PST 2014 +CreateTime: Tue Oct 21 01:25:13 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6750581243740872392/list_bucket_multiple +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/list_bucket_multiple Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389768704 + transient_lastDdlTime 1413879913 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -27,4 +27,4 @@ Sort Columns: [] Skewed Columns: [col1, col2] Skewed Values: [[s1, 1], [s3, 3], [s13, 13], [s78, 78]] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 b/sql/hive/src/test/resources/golden/create_struct_table-1-719851d0e8b89b51bdc6be4177455a92 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-44-7490df6719cd7e47aa08dbcbc3266a92 rename to sql/hive/src/test/resources/golden/create_struct_table-1-719851d0e8b89b51bdc6be4177455a92 diff --git a/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e b/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e deleted file mode 100644 index 8af6a0338d65a..0000000000000 --- a/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_max 'org.apache.hadoop.hive.ql.udf.UDAFTestMax') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b b/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b deleted file mode 100644 index e87fb81a6f043..0000000000000 --- a/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b +++ /dev/null @@ -1 +0,0 @@ -86 val_86 diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 b/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 deleted file mode 100644 index 70d2ecdbc8f78..0000000000000 --- a/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 +++ /dev/null @@ -1,13 +0,0 @@ -key int None -value string None - -# Partition Information -# col_name data_type comment - -value string None - -Detailed Table Information Table(tableName:vp1, dbName:default, owner:marmbrus, createTime:1392062982, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:value, type:string, comment:null)], parameters:{transient_lastDdlTime=1392062982}, viewOriginalText:SELECT key, value -FROM src -WHERE key=86, viewExpandedText:SELECT `src`.`key`, `src`.`value` -FROM `default`.`src` -WHERE `src`.`key`=86, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc b/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc deleted file mode 100644 index 43bc3de44f4df..0000000000000 --- a/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None - -# Partition Information -# col_name data_type comment - -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 12:09:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Table Type: VIRTUAL_VIEW -Table Parameters: - transient_lastDdlTime 1392062982 - -# Storage Information -SerDe Library: null -InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] - -# View Information -View Original Text: SELECT key, value -FROM src -WHERE key=86 -View Expanded Text: SELECT `src`.`key`, `src`.`value` -FROM `default`.`src` -WHERE `src`.`key`=86 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 b/sql/hive/src/test/resources/golden/cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-47-73da9fe2b0c2ee26c021ec3f2fa27272 rename to sql/hive/src/test/resources/golden/cross_product_check_1-0-d782db598869f9b19e0fcf5ea2a83594 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 b/sql/hive/src/test/resources/golden/cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-51-fcf9bcb522f542637ccdea863b408448 rename to sql/hive/src/test/resources/golden/cross_product_check_1-1-bd8395c55fa2fc80b68eb043b7020b76 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/cross_product_check_1-2-4e1b3108b7e1b9d8e94e73f9dfa44617 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 b/sql/hive/src/test/resources/golden/cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-54-dad56e1f06c808b29e5dc8fb0c49efb2 rename to sql/hive/src/test/resources/golden/cross_product_check_1-3-32a3e6eb858d37f58e225d07fb323254 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 b/sql/hive/src/test/resources/golden/cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-57-3cd3fbbbd8ee5c274fe3d6a45126cef4 rename to sql/hive/src/test/resources/golden/cross_product_check_1-4-36a6b6fb71ea08ff817dd40d1ffdb970 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 b/sql/hive/src/test/resources/golden/cross_product_check_1-5-103739f072b849d212dbc40919f92b74 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer1-60-d6bbaf0d40010159095e4cac025c50c5 rename to sql/hive/src/test/resources/golden/cross_product_check_1-5-103739f072b849d212dbc40919f92b74 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/cross_product_check_1-6-9a202e3bf15be2a310504a50920e7d25 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 rename to sql/hive/src/test/resources/golden/cross_product_check_1-7-e4e93f6e0dc63e1e324fb913a26fa8ac diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 b/sql/hive/src/test/resources/golden/cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594 similarity index 100% rename from sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 rename to sql/hive/src/test/resources/golden/cross_product_check_2-0-d782db598869f9b19e0fcf5ea2a83594 diff --git a/sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 b/sql/hive/src/test/resources/golden/cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76 similarity index 100% rename from sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 rename to sql/hive/src/test/resources/golden/cross_product_check_2-1-bd8395c55fa2fc80b68eb043b7020b76 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/cross_product_check_2-2-24ca942f094b14b92086305cc125e833 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/cross_product_check_2-2-24ca942f094b14b92086305cc125e833 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/cross_product_check_2-3-44d382ce6848d3f0b900b0808747d8e9 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/cross_product_check_2-4-c14792ccac2ca64e3e9e21af4fd12d2c diff --git a/sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 b/sql/hive/src/test/resources/golden/cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254 similarity index 100% rename from sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 rename to sql/hive/src/test/resources/golden/cross_product_check_2-5-32a3e6eb858d37f58e225d07fb323254 diff --git a/sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 rename to sql/hive/src/test/resources/golden/cross_product_check_2-6-36a6b6fb71ea08ff817dd40d1ffdb970 diff --git a/sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 b/sql/hive/src/test/resources/golden/cross_product_check_2-7-103739f072b849d212dbc40919f92b74 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 rename to sql/hive/src/test/resources/golden/cross_product_check_2-7-103739f072b849d212dbc40919f92b74 diff --git a/sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 b/sql/hive/src/test/resources/golden/cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 rename to sql/hive/src/test/resources/golden/cross_product_check_2-8-9a202e3bf15be2a310504a50920e7d25 diff --git a/sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c b/sql/hive/src/test/resources/golden/cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c rename to sql/hive/src/test/resources/golden/cross_product_check_2-9-e4e93f6e0dc63e1e324fb913a26fa8ac diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec b/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec deleted file mode 100644 index 6839c16243bcd..0000000000000 --- a/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 b/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 deleted file mode 100644 index 6839c16243bcd..0000000000000 --- a/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 diff --git a/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd +++ b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e +++ b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e b/sql/hive/src/test/resources/golden/database_drop-10-8db536f925bf0f5058f97897e145a661 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e rename to sql/hive/src/test/resources/golden/database_drop-10-8db536f925bf0f5058f97897e145a661 diff --git a/sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 b/sql/hive/src/test/resources/golden/database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5 similarity index 100% rename from sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 rename to sql/hive/src/test/resources/golden/database_drop-11-1b0a6cff3151cfa0ef0a6f78587973a5 diff --git a/sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 b/sql/hive/src/test/resources/golden/database_drop-12-2ea883422b74b701711e14e61472ba06 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 rename to sql/hive/src/test/resources/golden/database_drop-12-2ea883422b74b701711e14e61472ba06 diff --git a/sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 b/sql/hive/src/test/resources/golden/database_drop-13-e02a53f7e798d2741152526516f14941 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 rename to sql/hive/src/test/resources/golden/database_drop-13-e02a53f7e798d2741152526516f14941 diff --git a/sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d b/sql/hive/src/test/resources/golden/database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35 similarity index 100% rename from sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d rename to sql/hive/src/test/resources/golden/database_drop-14-8f0fe60664d020b2a42c7f5c0c7bed35 diff --git a/sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee b/sql/hive/src/test/resources/golden/database_drop-15-7928ac876f76c990fa21d74b6c9e14f6 similarity index 100% rename from sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee rename to sql/hive/src/test/resources/golden/database_drop-15-7928ac876f76c990fa21d74b6c9e14f6 diff --git a/sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e b/sql/hive/src/test/resources/golden/database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e similarity index 100% rename from sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e rename to sql/hive/src/test/resources/golden/database_drop-16-f9036ff3f0a3101373bdbc9a52faf00e diff --git a/sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 b/sql/hive/src/test/resources/golden/database_drop-17-95b906fd73935da8746b5277170e91e8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 rename to sql/hive/src/test/resources/golden/database_drop-17-95b906fd73935da8746b5277170e91e8 diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b b/sql/hive/src/test/resources/golden/database_drop-18-f65bf675b83871af7906741a60fa1318 similarity index 100% rename from sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b rename to sql/hive/src/test/resources/golden/database_drop-18-f65bf675b83871af7906741a60fa1318 diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 b/sql/hive/src/test/resources/golden/database_drop-19-d7fefbf585dbb67491e871ef58dca752 similarity index 100% rename from sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 rename to sql/hive/src/test/resources/golden/database_drop-19-d7fefbf585dbb67491e871ef58dca752 diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 b/sql/hive/src/test/resources/golden/database_drop-20-cbb84f0db4d55930a85cff28f7400bd0 similarity index 100% rename from sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 rename to sql/hive/src/test/resources/golden/database_drop-20-cbb84f0db4d55930a85cff28f7400bd0 diff --git a/sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/database_drop-21-eea49f066631be60519ae9d6b614d7d0 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 rename to sql/hive/src/test/resources/golden/database_drop-21-eea49f066631be60519ae9d6b614d7d0 diff --git a/sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a b/sql/hive/src/test/resources/golden/database_drop-22-85833c3a68c29098827e438ff580bb94 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a rename to sql/hive/src/test/resources/golden/database_drop-22-85833c3a68c29098827e438ff580bb94 diff --git a/sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 b/sql/hive/src/test/resources/golden/database_drop-23-84a5672989118a1b5792474c1469de90 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 rename to sql/hive/src/test/resources/golden/database_drop-23-84a5672989118a1b5792474c1469de90 diff --git a/sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d b/sql/hive/src/test/resources/golden/database_drop-24-441116797e8d95554c74472fa7644440 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d rename to sql/hive/src/test/resources/golden/database_drop-24-441116797e8d95554c74472fa7644440 diff --git a/sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 b/sql/hive/src/test/resources/golden/database_drop-25-847ca97dd211889d83e5d487bfc44e23 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 rename to sql/hive/src/test/resources/golden/database_drop-25-847ca97dd211889d83e5d487bfc44e23 diff --git a/sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 b/sql/hive/src/test/resources/golden/database_drop-26-bea20178515df24fcca04c0384f1c1b7 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 rename to sql/hive/src/test/resources/golden/database_drop-26-bea20178515df24fcca04c0384f1c1b7 diff --git a/sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 b/sql/hive/src/test/resources/golden/database_drop-27-cb4b90a7f63c1646319ee7bb014a7750 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 rename to sql/hive/src/test/resources/golden/database_drop-27-cb4b90a7f63c1646319ee7bb014a7750 diff --git a/sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb b/sql/hive/src/test/resources/golden/database_drop-28-53d67cbed634cff012dac41340bf6630 similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb rename to sql/hive/src/test/resources/golden/database_drop-28-53d67cbed634cff012dac41340bf6630 diff --git a/sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb b/sql/hive/src/test/resources/golden/database_drop-29-29d3232325eda40cbb0bd1786b7d070e similarity index 100% rename from sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb rename to sql/hive/src/test/resources/golden/database_drop-29-29d3232325eda40cbb0bd1786b7d070e diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/database_drop-3-db64b724719d27c7f0db4f51f5c4edaa similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/database_drop-3-db64b724719d27c7f0db4f51f5c4edaa diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c b/sql/hive/src/test/resources/golden/database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c rename to sql/hive/src/test/resources/golden/database_drop-30-bbad0860316d8b9b1eed50d231f3ab5d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 b/sql/hive/src/test/resources/golden/database_drop-31-981f8e58caeae9cbbad3a113e3043de5 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 rename to sql/hive/src/test/resources/golden/database_drop-31-981f8e58caeae9cbbad3a113e3043de5 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f b/sql/hive/src/test/resources/golden/database_drop-32-6c8e7590238b5aca1772721f0b914ece similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f rename to sql/hive/src/test/resources/golden/database_drop-32-6c8e7590238b5aca1772721f0b914ece diff --git a/sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 b/sql/hive/src/test/resources/golden/database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 rename to sql/hive/src/test/resources/golden/database_drop-33-2bc7864932f597bdf98bdc410b1a2d9c diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/database_drop-34-491138bed44a70cb783bb2b531e1d82 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 rename to sql/hive/src/test/resources/golden/database_drop-34-491138bed44a70cb783bb2b531e1d82 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/database_drop-35-9e0285d0596607cdadf75a763a543866 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 rename to sql/hive/src/test/resources/golden/database_drop-35-9e0285d0596607cdadf75a763a543866 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 b/sql/hive/src/test/resources/golden/database_drop-36-e66471f3f1bbe2d4b3b214e47793656d similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 rename to sql/hive/src/test/resources/golden/database_drop-36-e66471f3f1bbe2d4b3b214e47793656d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 b/sql/hive/src/test/resources/golden/database_drop-37-f6410721e3125a89836817136306eac4 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 rename to sql/hive/src/test/resources/golden/database_drop-37-f6410721e3125a89836817136306eac4 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/database_drop-38-7d45d97adebe50917a94cbe232c112a8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 rename to sql/hive/src/test/resources/golden/database_drop-38-7d45d97adebe50917a94cbe232c112a8 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1 similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 rename to sql/hive/src/test/resources/golden/database_drop-39-91b4a660ae5d5d2966d6bf3b6ae751d1 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 rename to sql/hive/src/test/resources/golden/database_drop-4-f21dd011aeb417043ed58c03fd5c3bf0 diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 b/sql/hive/src/test/resources/golden/database_drop-40-10073fb74a5c792322fc52389997695b similarity index 100% rename from sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 rename to sql/hive/src/test/resources/golden/database_drop-40-10073fb74a5c792322fc52389997695b diff --git a/sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c b/sql/hive/src/test/resources/golden/database_drop-41-7164c585e1ef4d9036ed4db275811084 similarity index 100% rename from sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c rename to sql/hive/src/test/resources/golden/database_drop-41-7164c585e1ef4d9036ed4db275811084 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/database_drop-42-c55cffbfc4d950363be97ccdb028faf3 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d rename to sql/hive/src/test/resources/golden/database_drop-42-c55cffbfc4d950363be97ccdb028faf3 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/database_drop-43-cc6860c125b8b62450cb858c72716dc2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 rename to sql/hive/src/test/resources/golden/database_drop-43-cc6860c125b8b62450cb858c72716dc2 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/database_drop-44-de81fd80132350aedcd9f0d9a212fd94 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/database_drop-44-de81fd80132350aedcd9f0d9a212fd94 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea b/sql/hive/src/test/resources/golden/database_drop-45-64e22634663e57153119340583e93651 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea rename to sql/hive/src/test/resources/golden/database_drop-45-64e22634663e57153119340583e93651 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d b/sql/hive/src/test/resources/golden/database_drop-46-eb28b907b605e51026f9902287e1d90d similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d rename to sql/hive/src/test/resources/golden/database_drop-46-eb28b907b605e51026f9902287e1d90d diff --git a/sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/database_drop-47-44e4d2bdab2dfa4583da47281ed00ba3 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/database_drop-48-d419f4ff197d4291208c2028cd158909 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/database_drop-48-d419f4ff197d4291208c2028cd158909 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/database_drop-5-d419f4ff197d4291208c2028cd158909 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to sql/hive/src/test/resources/golden/database_drop-5-d419f4ff197d4291208c2028cd158909 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/database_drop-6-b7cf74929eabe781b0db79ed1043dc24 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 rename to sql/hive/src/test/resources/golden/database_drop-6-b7cf74929eabe781b0db79ed1043dc24 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de b/sql/hive/src/test/resources/golden/database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de rename to sql/hive/src/test/resources/golden/database_drop-7-a47b1b070ec7c3b9ccabc34f41aebad9 diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb b/sql/hive/src/test/resources/golden/database_drop-8-b3980119a4b2d8f531951a940cba3697 similarity index 100% rename from sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb rename to sql/hive/src/test/resources/golden/database_drop-8-b3980119a4b2d8f531951a940cba3697 diff --git a/sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 b/sql/hive/src/test/resources/golden/database_drop-9-97101266791d2b2c662bcde549422318 similarity index 100% rename from sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 rename to sql/hive/src/test/resources/golden/database_drop-9-97101266791d2b2c662bcde549422318 diff --git a/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 b/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 index f05305df0c3d3..a97bfdfeef098 100644 --- a/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 +++ b/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 @@ -1 +1 @@ -db1 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db \ No newline at end of file +db1 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db1.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 b/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 index e92c241d50092..adf00309e0fae 100644 --- a/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 +++ b/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 @@ -1,19 +1,19 @@ # col_name data_type comment -name string None -value int None +name string +value int # Detailed Table Information Database: db2 Owner: marmbrus -CreateTime: Tue Jan 14 11:40:51 PST 2014 +CreateTime: Tue Oct 21 01:25:24 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2/table_db2 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2/table_db2 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389728451 + transient_lastDdlTime 1413879924 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c index 7c77c06cda82b..2e4bf94808285 100644 --- a/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c @@ -1 +1 @@ -table_db2 \ No newline at end of file +table_db2 diff --git a/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb b/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb index cda1736e9ca6b..e7c766a71c1d2 100644 --- a/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb +++ b/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb @@ -1,19 +1,19 @@ # col_name data_type comment -name string None -value int None +name string +value int # Detailed Table Information Database: db1 Owner: marmbrus -CreateTime: Tue Jan 14 11:40:50 PST 2014 +CreateTime: Tue Oct 21 01:25:24 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/table_db1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db1.db/table_db1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1389728450 + transient_lastDdlTime 1413879924 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c index 4d8fc6a6d6fe0..6eabc06bb1362 100644 --- a/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c @@ -1 +1 @@ -table_db1 \ No newline at end of file +table_db1 diff --git a/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 b/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 index 3e23970adddcf..65f9d79a3c381 100644 --- a/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 +++ b/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 @@ -1 +1 @@ -db2 database 2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2 \ No newline at end of file +db2 database 2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2 marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 +++ b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 index 2f5fbe26f4945..4c04259aed3a7 100644 --- a/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 +++ b/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 @@ -1 +1 @@ -db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db {mapred.jobtracker.url=http://my.jobtracker.com:53000, new.property=some new props, mapred.scratch.dir=hdfs://tmp.dfs.com:50029/tmp, hive.warehouse.dir=new/warehouse/dir} \ No newline at end of file +db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 index 86362ead004d9..3cd776a0711ff 100644 --- a/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 +++ b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 @@ -1,2 +1,2 @@ db1 -default \ No newline at end of file +default diff --git a/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a b/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a index ff89c3fe899d2..4c04259aed3a7 100644 --- a/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a +++ b/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a @@ -1 +1 @@ -db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db \ No newline at end of file +db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 index 5827d2726d084..4c04259aed3a7 100644 --- a/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 +++ b/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 @@ -1 +1 @@ -db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db2.db {mapred.jobtracker.url=http://my.jobtracker.com:53000, mapred.scratch.dir=hdfs://tmp.dfs.com:50029/tmp, hive.warehouse.dir=/user/hive/warehouse} \ No newline at end of file +db2 file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/db2.db marmbrus diff --git a/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f +++ b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 +++ b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 b/sql/hive/src/test/resources/golden/date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1 similarity index 100% rename from sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 rename to sql/hive/src/test/resources/golden/date_2-2-efdf7f5d9f15edcb59a30f8ea166fbf1 diff --git a/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 b/sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120 similarity index 100% rename from sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 rename to sql/hive/src/test/resources/golden/date_3-3-c26f0641e7cec1093273b258e6bf7120 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/date_4-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/date_4-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 b/sql/hive/src/test/resources/golden/date_4-1-b84f7e931d710dcbe3c5126d998285a8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 rename to sql/hive/src/test/resources/golden/date_4-1-b84f7e931d710dcbe3c5126d998285a8 diff --git a/sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/sql/hive/src/test/resources/golden/date_4-2-6272f5e518f6a20bc96a5870ff315c4f similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 rename to sql/hive/src/test/resources/golden/date_4-2-6272f5e518f6a20bc96a5870ff315c4f diff --git a/sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 b/sql/hive/src/test/resources/golden/date_4-3-4a0e7bde447ef616b98e0f55d2886de0 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 rename to sql/hive/src/test/resources/golden/date_4-3-4a0e7bde447ef616b98e0f55d2886de0 diff --git a/sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd b/sql/hive/src/test/resources/golden/date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd rename to sql/hive/src/test/resources/golden/date_4-4-6c4c2941bae77147a4d3d8fcaa1c88c8 diff --git a/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 b/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 new file mode 100644 index 0000000000000..b61affde4ffce --- /dev/null +++ b/sql/hive/src/test/resources/golden/date_4-5-bee09a7384666043621f68297cee2e68 @@ -0,0 +1 @@ +2011-01-01 2011-01-01 diff --git a/sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea b/sql/hive/src/test/resources/golden/date_4-6-b84f7e931d710dcbe3c5126d998285a8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea rename to sql/hive/src/test/resources/golden/date_4-6-b84f7e931d710dcbe3c5126d998285a8 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/date_comparison-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 b/sql/hive/src/test/resources/golden/date_comparison-1-69eec445bd045c9dc899fafa348d8495 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 rename to sql/hive/src/test/resources/golden/date_comparison-1-69eec445bd045c9dc899fafa348d8495 diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 b/sql/hive/src/test/resources/golden/date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 rename to sql/hive/src/test/resources/golden/date_comparison-10-bcd987341fc1c38047a27d29dac6ae7c diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc b/sql/hive/src/test/resources/golden/date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc rename to sql/hive/src/test/resources/golden/date_comparison-11-a9f2560c273163e11306d4f1dd1d9d54 diff --git a/sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/date_comparison-12-4a7bac9ddcf40db6329faaec8e426543 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 rename to sql/hive/src/test/resources/golden/date_comparison-12-4a7bac9ddcf40db6329faaec8e426543 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 b/sql/hive/src/test/resources/golden/date_comparison-2-fcc400871a502009c8680509e3869ec1 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 rename to sql/hive/src/test/resources/golden/date_comparison-2-fcc400871a502009c8680509e3869ec1 diff --git a/sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 rename to sql/hive/src/test/resources/golden/date_comparison-3-b8598a4d0c948c2ddcf3eeef0abf2264 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f b/sql/hive/src/test/resources/golden/date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f rename to sql/hive/src/test/resources/golden/date_comparison-4-14d35f266be9cceb11a2ae09ec8b3835 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-13-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/date_comparison-5-c8865b14d53f2c2496fb69ee8191bf37 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-22-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/date_comparison-6-f2c907e64da8166a731ddc0ed19bad6c diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 b/sql/hive/src/test/resources/golden/date_comparison-7-5606505a92bad10023ad9a3ef77eacc9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 rename to sql/hive/src/test/resources/golden/date_comparison-7-5606505a92bad10023ad9a3ef77eacc9 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-31-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/date_comparison-8-47913d4aaf0d468ab3764cc3bfd68eb diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b b/sql/hive/src/test/resources/golden/date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b rename to sql/hive/src/test/resources/golden/date_comparison-9-1e5ce4f833b6fba45618437c8fb7643c diff --git a/sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc b/sql/hive/src/test/resources/golden/date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc rename to sql/hive/src/test/resources/golden/date_join1-2-e967e1ef6b209dfa5bdc60021dcb1964 diff --git a/sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a b/sql/hive/src/test/resources/golden/date_serde-7-580096b3b48db26bea91b80e1e1b081a similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a rename to sql/hive/src/test/resources/golden/date_serde-7-580096b3b48db26bea91b80e1e1b081a diff --git a/sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 b/sql/hive/src/test/resources/golden/date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 rename to sql/hive/src/test/resources/golden/date_udf-7-ef82dff775f4aba5d7a638b4e5fd9c5d diff --git a/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 b/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 deleted file mode 100644 index c270c7cbdfa1f..0000000000000 --- a/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 +++ /dev/null @@ -1,4 +0,0 @@ -key int None -value string None - -Detailed Table Information Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1392063041, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392063041}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 b/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 b/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 b/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 deleted file mode 100644 index c4a17c1b14c88..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 +++ /dev/null @@ -1 +0,0 @@ -1969-12-31 16:00:17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 b/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b b/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a b/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d b/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c b/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 deleted file mode 100644 index b1bd38b62a080..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 +++ /dev/null @@ -1 +0,0 @@ -13 diff --git a/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 deleted file mode 100644 index de7771ac23570..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 +++ /dev/null @@ -1 +0,0 @@ --3827 diff --git a/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 deleted file mode 100644 index 272791f402250..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 +++ /dev/null @@ -1 +0,0 @@ -3404045 diff --git a/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d deleted file mode 100644 index 272791f402250..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d +++ /dev/null @@ -1 +0,0 @@ -3404045 diff --git a/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e deleted file mode 100644 index deb8427800ee4..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e +++ /dev/null @@ -1 +0,0 @@ -3404045.5 diff --git a/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 deleted file mode 100644 index 6f31e8fe55034..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 +++ /dev/null @@ -1 +0,0 @@ -3404045.5044003 diff --git a/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 deleted file mode 100644 index 6f31e8fe55034..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 +++ /dev/null @@ -1 +0,0 @@ -3404045.5044003 diff --git a/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 b/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 deleted file mode 100644 index 6324d401a069f..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 +++ /dev/null @@ -1 +0,0 @@ -3.14 diff --git a/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 b/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 deleted file mode 100644 index 6324d401a069f..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 +++ /dev/null @@ -1 +0,0 @@ -3.14 diff --git a/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff b/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff deleted file mode 100644 index 603f18cc37bc4..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff +++ /dev/null @@ -1 +0,0 @@ -1355944339.1234567 diff --git a/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 b/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 b/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a b/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a deleted file mode 100644 index 474c8b180aea9..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a +++ /dev/null @@ -1 +0,0 @@ -0.99999999999999999999 diff --git a/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d deleted file mode 100644 index 98d9bcb75a685..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d +++ /dev/null @@ -1 +0,0 @@ -17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e deleted file mode 100644 index 53aca7545dac7..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e +++ /dev/null @@ -1 +0,0 @@ -17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 b/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 deleted file mode 100644 index 8d8753f153d7c..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 +++ /dev/null @@ -1,4 +0,0 @@ -3.14 3 -3.14 3 -3.14 3 -3.14 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 b/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 deleted file mode 100644 index 8d8753f153d7c..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 +++ /dev/null @@ -1,4 +0,0 @@ -3.14 3 -3.14 3 -3.14 3 -3.14 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b deleted file mode 100644 index 3e290231c27e2..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b +++ /dev/null @@ -1,38 +0,0 @@ -NULL 0 -NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -0.9999999999999999999999999 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 b/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 deleted file mode 100644 index 64fa7bca9a81b..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 +++ /dev/null @@ -1,38 +0,0 @@ -1234567890.12345678 1234567890 -200 200 -125.2 125 -124 124 -100 100 -20 20 -10 10 -3.14 4 -3.14 3 -3.14 3 -3.14 3 -2 2 -2 2 -1.122 1 -1.12 1 -1 1 -1 1 -0.9999999999999999999999999 1 -0.333 0 -0.33 0 -0.3 0 -0.2 0 -0.1 0 -0.02 0 -0.01 0 -0 0 -0 0 --0.3 0 --0.33 0 --0.333 0 --1.12 -1 --1.12 -1 --1.122 -11 --1255.49 -1255 --4400 4400 --1234567890.123456789 -1234567890 -NULL 0 -NULL 0 diff --git a/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b deleted file mode 100644 index 3e290231c27e2..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b +++ /dev/null @@ -1,38 +0,0 @@ -NULL 0 -NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -1 --1.12 -1 --0.333 0 --0.33 0 --0.3 0 -0 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -0.9999999999999999999999999 1 -1 1 -1 1 -1.12 1 -1.122 1 -2 2 -2 2 -3.14 3 -3.14 3 -3.14 3 -3.14 4 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 b/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 deleted file mode 100644 index 24d34ee5d8c1b..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 +++ /dev/null @@ -1,30 +0,0 @@ -NULL --1234567890.123456789 --4400 --1255.49 --1.122 --1.12 --0.333 --0.33 --0.3 -0 -0.01 -0.02 -0.1 -0.2 -0.3 -0.33 -0.333 -0.9999999999999999999999999 -1 -1.12 -1.122 -2 -3.14 -10 -20 -100 -124 -125.2 -200 -1234567890.12345678 diff --git a/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 b/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 deleted file mode 100644 index e08f588c89461..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 +++ /dev/null @@ -1,30 +0,0 @@ -NULL 0 --1234567890.123456789 -1234567890 --4400 4400 --1255.49 -1255 --1.122 -11 --1.12 -2 --0.333 0 --0.33 0 --0.3 0 -0 0 -0.01 0 -0.02 0 -0.1 0 -0.2 0 -0.3 0 -0.33 0 -0.333 0 -0.9999999999999999999999999 1 -1 2 -1.12 1 -1.122 1 -2 4 -3.14 13 -10 10 -20 20 -100 100 -124 124 -125.2 125 -200 200 -1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 b/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 deleted file mode 100644 index 796707d06b0dd..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 +++ /dev/null @@ -1,17 +0,0 @@ --1234567890 -1234567890.123456789 --1255 -1255.49 --11 -1.122 --1 -2.24 -0 0.33 -1 5.2419999999999999999999999 -2 4 -3 9.42 -4 3.14 -10 10 -20 20 -100 100 -124 124 -125 125.2 -200 200 -4400 -4400 -1234567890 1234567890.12345678 diff --git a/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 b/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 deleted file mode 100644 index 4217ad848170e..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 +++ /dev/null @@ -1,56 +0,0 @@ --1234567890.123456789 -1234567890 -1234567890.123456789 -1234567890 --4400 4400 -4400 4400 --1255.49 -1255 -1255.49 -1255 --1.122 -11 -1.122 -11 --1.12 -1 -1.12 -1 --1.12 -1 -1.12 -1 --1.12 -1 -1.12 -1 --1.12 -1 -1.12 -1 --0.333 0 -0.333 0 --0.33 0 -0.33 0 --0.3 0 -0.3 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0 0 0 0 -0.01 0 0.01 0 -0.02 0 0.02 0 -0.1 0 0.1 0 -0.2 0 0.2 0 -0.3 0 0.3 0 -0.33 0 0.33 0 -0.333 0 0.333 0 -0.9999999999999999999999999 1 0.9999999999999999999999999 1 -1 1 1 1 -1 1 1 1 -1 1 1 1 -1 1 1 1 -1.12 1 1.12 1 -1.122 1 1.122 1 -2 2 2 2 -2 2 2 2 -2 2 2 2 -2 2 2 2 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 3 -3.14 3 3.14 4 -3.14 3 3.14 4 -3.14 3 3.14 4 -3.14 4 3.14 3 -3.14 4 3.14 3 -3.14 4 3.14 3 -3.14 4 3.14 4 -10 10 10 10 -20 20 20 20 -100 100 100 100 -124 124 124 124 -125.2 125 125.2 125 -200 200 200 200 -1234567890.12345678 1234567890 1234567890.12345678 1234567890 diff --git a/sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 b/sql/hive/src/test/resources/golden/decimal_4-2-85c3185beb011f5c1e6856fc773a7484 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 rename to sql/hive/src/test/resources/golden/decimal_4-2-85c3185beb011f5c1e6856fc773a7484 diff --git a/sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 b/sql/hive/src/test/resources/golden/decimal_4-3-1451d7491441c1632fd5f751876cce6e similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 rename to sql/hive/src/test/resources/golden/decimal_4-3-1451d7491441c1632fd5f751876cce6e diff --git a/sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a b/sql/hive/src/test/resources/golden/decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a rename to sql/hive/src/test/resources/golden/decimal_4-4-1bf9ff1d72a06c33885ba695adf2511d diff --git a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e index 3e290231c27e2..f59549a6e4a46 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e +++ b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e @@ -1,5 +1,4 @@ NULL 0 -NULL 0 -1234567890.123456789 -1234567890 -4400 4400 -1255.49 -1255 @@ -11,6 +10,7 @@ NULL 0 -0.3 0 0 0 0 0 +0 0 0.01 0 0.02 0 0.1 0 diff --git a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 index 795a4b567ab7f..6bada475c6d3d 100644 --- a/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 +++ b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 @@ -1,5 +1,4 @@ NULL NULL -NULL NULL -1234567890.123456789 -3703703670.370370367 -4400 -13200 -1255.49 -3766.47 @@ -11,6 +10,7 @@ NULL NULL -0.3 -0.9 0 0 0 0 +0 0 0.01 0.03 0.02 0.06 0.1 0.3 diff --git a/sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 b/sql/hive/src/test/resources/golden/decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 rename to sql/hive/src/test/resources/golden/decimal_join-0-4668e9dee2cd7a32f2b7311d7cd35508 diff --git a/sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c b/sql/hive/src/test/resources/golden/decimal_join-1-5098974222b22a21ed847c7906df9313 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c rename to sql/hive/src/test/resources/golden/decimal_join-1-5098974222b22a21ed847c7906df9313 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b b/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b deleted file mode 100644 index 3d9e792183f3c..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b +++ /dev/null @@ -1,75 +0,0 @@ -NULL -NULL -NULL -NULL -NULL -NULL -NULL -NULL --99999999999999999999999999999999999999 --999999999999999999999999999999999999 --99999999999999999999999999999999999 --0.0000000000000000000000000000000000001 -0 -0.0000000000000000000000000000000000001 -0.123456789012345 -0.1234567890123456789012345678901234578 -1.234567890123456 -1.2345678901234567890123456789012345678 -12.34567890123456 -12.345678901234567890123456789012345678 -123.4567890123456 -123.45678901234567890123456789012345678 -1234.567890123456 -1234.5678901234567890123456789012345678 -12345.67890123456 -12345.678901234567890123456789012345678 -123456.7890123456 -123456.78901234567890123456789012345678 -1234567.890123456 -1234567.8901234567890123456789012345678 -12345678.90123456 -12345678.901234567890123456789012345678 -123456789.0123456 -123456789.01234567890123456789012345678 -1234567890.123456 -1234567890.1234567890123456789012345678 -12345678901.23456 -12345678901.234567890123456789012345678 -123456789012.3456 -123456789012.34567890123456789012345678 -1234567890123.456 -1234567890123.4567890123456789012345678 -12345678901234.56 -12345678901234.567890123456789012345678 -123456789012345.6 -123456789012345.67890123456789012345678 -1234567890123456.7890123456789012345678 -12345678901234567.890123456789012345678 -123456789012345678.90123456789012345678 -1234567890123456789.0123456789012345678 -12345678901234567890.123456789012345678 -123456789012345678901.23456789012345678 -1234567890123456789012.3456789012345678 -12345678901234567890123.456789012345678 -123456789012345678901234.56789012345678 -1234567890123456789012345.6789012345678 -12345678901234567890123456.789012345678 -123456789012345678901234567.89012345678 -1234567890123456789012345678.9012345678 -12345678901234567890123456789.012345678 -123456789012345678901234567890.12345678 -1234567890123456789012345678901.2345678 -12345678901234567890123456789012.345678 -123456789012345678901234567890123.45678 -1234567890123456789012345678901234.5678 -12345678901234567890123456789012345.678 -99999999999999999999999999999999999 -123456789012345678901234567890123456.78 -999999999999999999999999999999999999 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -12345678901234567890123456789012345678 -99999999999999999999999999999999999999 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc b/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc deleted file mode 100644 index 9853ce72ed8c3..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL --99999999999999999999999999999999999999 -99999999999999999999999999999999999998 NULL --999999999999999999999999999999999999 -999999999999999999999999999999999998 -1000000000000000000000000000000000000 --99999999999999999999999999999999999 -99999999999999999999999999999999998 -100000000000000000000000000000000000 --0.0000000000000000000000000000000000001 0.9999999999999999999999999999999999999 -1.0000000000000000000000000000000000001 -0 1 -1 -0.0000000000000000000000000000000000001 1.0000000000000000000000000000000000001 -0.9999999999999999999999999999999999999 -0.123456789012345 1.123456789012345 -0.876543210987655 -0.1234567890123456789012345678901234578 1.1234567890123456789012345678901234578 -0.8765432109876543210987654321098765422 -1.234567890123456 2.234567890123456 0.234567890123456 -1.2345678901234567890123456789012345678 2.2345678901234567890123456789012345678 0.2345678901234567890123456789012345678 -12.34567890123456 13.34567890123456 11.34567890123456 -12.345678901234567890123456789012345678 13.345678901234567890123456789012345678 11.345678901234567890123456789012345678 -123.4567890123456 124.4567890123456 122.4567890123456 -123.45678901234567890123456789012345678 124.45678901234567890123456789012345678 122.45678901234567890123456789012345678 -1234.567890123456 1235.567890123456 1233.567890123456 -1234.5678901234567890123456789012345678 1235.5678901234567890123456789012345678 1233.5678901234567890123456789012345678 -12345.67890123456 12346.67890123456 12344.67890123456 -12345.678901234567890123456789012345678 12346.678901234567890123456789012345678 12344.678901234567890123456789012345678 -123456.7890123456 123457.7890123456 123455.7890123456 -123456.78901234567890123456789012345678 123457.78901234567890123456789012345678 123455.78901234567890123456789012345678 -1234567.890123456 1234568.890123456 1234566.890123456 -1234567.8901234567890123456789012345678 1234568.8901234567890123456789012345678 1234566.8901234567890123456789012345678 -12345678.90123456 12345679.90123456 12345677.90123456 -12345678.901234567890123456789012345678 12345679.901234567890123456789012345678 12345677.901234567890123456789012345678 -123456789.0123456 123456790.0123456 123456788.0123456 -123456789.01234567890123456789012345678 123456790.01234567890123456789012345678 123456788.01234567890123456789012345678 -1234567890.123456 1234567891.123456 1234567889.123456 -1234567890.1234567890123456789012345678 1234567891.1234567890123456789012345678 1234567889.1234567890123456789012345678 -12345678901.23456 12345678902.23456 12345678900.23456 -12345678901.234567890123456789012345678 12345678902.234567890123456789012345678 12345678900.234567890123456789012345678 -123456789012.3456 123456789013.3456 123456789011.3456 -123456789012.34567890123456789012345678 123456789013.34567890123456789012345678 123456789011.34567890123456789012345678 -1234567890123.456 1234567890124.456 1234567890122.456 -1234567890123.4567890123456789012345678 1234567890124.4567890123456789012345678 1234567890122.4567890123456789012345678 -12345678901234.56 12345678901235.56 12345678901233.56 -12345678901234.567890123456789012345678 12345678901235.567890123456789012345678 12345678901233.567890123456789012345678 -123456789012345.6 123456789012346.6 123456789012344.6 -123456789012345.67890123456789012345678 123456789012346.67890123456789012345678 123456789012344.67890123456789012345678 -1234567890123456.7890123456789012345678 1234567890123457.7890123456789012345678 1234567890123455.7890123456789012345678 -12345678901234567.890123456789012345678 12345678901234568.890123456789012345678 12345678901234566.890123456789012345678 -123456789012345678.90123456789012345678 123456789012345679.90123456789012345678 123456789012345677.90123456789012345678 -1234567890123456789.0123456789012345678 1234567890123456790.0123456789012345678 1234567890123456788.0123456789012345678 -12345678901234567890.123456789012345678 12345678901234567891.123456789012345678 12345678901234567889.123456789012345678 -123456789012345678901.23456789012345678 123456789012345678902.23456789012345678 123456789012345678900.23456789012345678 -1234567890123456789012.3456789012345678 1234567890123456789013.3456789012345678 1234567890123456789011.3456789012345678 -12345678901234567890123.456789012345678 12345678901234567890124.456789012345678 12345678901234567890122.456789012345678 -123456789012345678901234.56789012345678 123456789012345678901235.56789012345678 123456789012345678901233.56789012345678 -1234567890123456789012345.6789012345678 1234567890123456789012346.6789012345678 1234567890123456789012344.6789012345678 -12345678901234567890123456.789012345678 12345678901234567890123457.789012345678 12345678901234567890123455.789012345678 -123456789012345678901234567.89012345678 123456789012345678901234568.89012345678 123456789012345678901234566.89012345678 -1234567890123456789012345678.9012345678 1234567890123456789012345679.9012345678 1234567890123456789012345677.9012345678 -12345678901234567890123456789.012345678 12345678901234567890123456790.012345678 12345678901234567890123456788.012345678 -123456789012345678901234567890.12345678 123456789012345678901234567891.12345678 123456789012345678901234567889.12345678 -1234567890123456789012345678901.2345678 1234567890123456789012345678902.2345678 1234567890123456789012345678900.2345678 -12345678901234567890123456789012.345678 12345678901234567890123456789013.345678 12345678901234567890123456789011.345678 -123456789012345678901234567890123.45678 123456789012345678901234567890124.45678 123456789012345678901234567890122.45678 -1234567890123456789012345678901234.5678 1234567890123456789012345678901235.5678 1234567890123456789012345678901233.5678 -12345678901234567890123456789012345.678 12345678901234567890123456789012346.678 12345678901234567890123456789012344.678 -99999999999999999999999999999999999 100000000000000000000000000000000000 99999999999999999999999999999999998 -123456789012345678901234567890123456.78 123456789012345678901234567890123457.78 123456789012345678901234567890123455.78 -999999999999999999999999999999999999 1000000000000000000000000000000000000 999999999999999999999999999999999998 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -12345678901234567890123456789012345678 12345678901234567890123456789012345679 12345678901234567890123456789012345677 -99999999999999999999999999999999999999 NULL 99999999999999999999999999999999999998 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef b/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef deleted file mode 100644 index 7cc75c789dee1..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL -NULL NULL NULL --99999999999999999999999999999999999999 NULL -33333333333333333333333333333333333333 --999999999999999999999999999999999999 -1999999999999999999999999999999999998 -333333333333333333333333333333333333 --99999999999999999999999999999999999 -199999999999999999999999999999999998 -33333333333333333333333333333333333 --0.0000000000000000000000000000000000001 -0.0000000000000000000000000000000000002 0 -0 0 0 -0.0000000000000000000000000000000000001 0.0000000000000000000000000000000000002 0 -0.123456789012345 0.24691357802469 0.041152263004115 -0.1234567890123456789012345678901234578 0.2469135780246913578024691357802469156 0.0411522630041152263004115226300411526 -1.234567890123456 2.469135780246912 0.411522630041152 -1.2345678901234567890123456789012345678 2.4691357802469135780246913578024691356 0.4115226300411522630041152263004115226 -12.34567890123456 24.69135780246912 4.11522630041152 -12.345678901234567890123456789012345678 24.691357802469135780246913578024691356 4.115226300411522630041152263004115226 -123.4567890123456 246.9135780246912 41.1522630041152 -123.45678901234567890123456789012345678 246.91357802469135780246913578024691356 41.15226300411522630041152263004115226 -1234.567890123456 2469.135780246912 411.522630041152 -1234.5678901234567890123456789012345678 2469.1357802469135780246913578024691356 411.5226300411522630041152263004115226 -12345.67890123456 24691.35780246912 4115.22630041152 -12345.678901234567890123456789012345678 24691.357802469135780246913578024691356 4115.226300411522630041152263004115226 -123456.7890123456 246913.5780246912 41152.2630041152 -123456.78901234567890123456789012345678 246913.57802469135780246913578024691356 41152.26300411522630041152263004115226 -1234567.890123456 2469135.780246912 411522.630041152 -1234567.8901234567890123456789012345678 2469135.7802469135780246913578024691356 411522.6300411522630041152263004115226 -12345678.90123456 24691357.80246912 4115226.30041152 -12345678.901234567890123456789012345678 24691357.802469135780246913578024691356 4115226.300411522630041152263004115226 -123456789.0123456 246913578.0246912 41152263.0041152 -123456789.01234567890123456789012345678 246913578.02469135780246913578024691356 41152263.00411522630041152263004115226 -1234567890.123456 2469135780.246912 411522630.041152 -1234567890.1234567890123456789012345678 2469135780.2469135780246913578024691356 411522630.0411522630041152263004115226 -12345678901.23456 24691357802.46912 4115226300.41152 -12345678901.234567890123456789012345678 24691357802.469135780246913578024691356 4115226300.411522630041152263004115226 -123456789012.3456 246913578024.6912 41152263004.1152 -123456789012.34567890123456789012345678 246913578024.69135780246913578024691356 41152263004.11522630041152263004115226 -1234567890123.456 2469135780246.912 411522630041.152 -1234567890123.4567890123456789012345678 2469135780246.9135780246913578024691356 411522630041.1522630041152263004115226 -12345678901234.56 24691357802469.12 4115226300411.52 -12345678901234.567890123456789012345678 24691357802469.135780246913578024691356 4115226300411.522630041152263004115226 -123456789012345.6 246913578024691.2 41152263004115.2 -123456789012345.67890123456789012345678 246913578024691.35780246913578024691356 41152263004115.22630041152263004115226 -1234567890123456.7890123456789012345678 2469135780246913.5780246913578024691356 411522630041152.2630041152263004115226 -12345678901234567.890123456789012345678 24691357802469135.780246913578024691356 4115226300411522.630041152263004115226 -123456789012345678.90123456789012345678 246913578024691357.80246913578024691356 41152263004115226.30041152263004115226 -1234567890123456789.0123456789012345678 2469135780246913578.0246913578024691356 411522630041152263.0041152263004115226 -12345678901234567890.123456789012345678 24691357802469135780.246913578024691356 4115226300411522630.041152263004115226 -123456789012345678901.23456789012345678 246913578024691357802.46913578024691356 41152263004115226300.41152263004115226 -1234567890123456789012.3456789012345678 2469135780246913578024.6913578024691356 411522630041152263004.1152263004115226 -12345678901234567890123.456789012345678 24691357802469135780246.913578024691356 4115226300411522630041.152263004115226 -123456789012345678901234.56789012345678 246913578024691357802469.13578024691356 41152263004115226300411.52263004115226 -1234567890123456789012345.6789012345678 2469135780246913578024691.3578024691356 411522630041152263004115.2263004115226 -12345678901234567890123456.789012345678 24691357802469135780246913.578024691356 4115226300411522630041152.263004115226 -123456789012345678901234567.89012345678 246913578024691357802469135.78024691356 41152263004115226300411522.63004115226 -1234567890123456789012345678.9012345678 2469135780246913578024691357.8024691356 411522630041152263004115226.3004115226 -12345678901234567890123456789.012345678 24691357802469135780246913578.024691356 4115226300411522630041152263.004115226 -123456789012345678901234567890.12345678 246913578024691357802469135780.24691356 41152263004115226300411522630.04115226 -1234567890123456789012345678901.2345678 2469135780246913578024691357802.4691356 411522630041152263004115226300.4115226 -12345678901234567890123456789012.345678 24691357802469135780246913578024.691356 4115226300411522630041152263004.115226 -123456789012345678901234567890123.45678 246913578024691357802469135780246.91356 41152263004115226300411522630041.15226 -1234567890123456789012345678901234.5678 2469135780246913578024691357802469.1356 411522630041152263004115226300411.5226 -12345678901234567890123456789012345.678 24691357802469135780246913578024691.356 4115226300411522630041152263004115.226 -99999999999999999999999999999999999 199999999999999999999999999999999998 33333333333333333333333333333333333 -123456789012345678901234567890123456.78 246913578024691357802469135780246913.56 41152263004115226300411522630041152.26 -999999999999999999999999999999999999 1999999999999999999999999999999999998 333333333333333333333333333333333333 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -12345678901234567890123456789012345678 24691357802469135780246913578024691356 4115226300411522630041152263004115226 -99999999999999999999999999999999999999 NULL 33333333333333333333333333333333333333 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 b/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 deleted file mode 100644 index c40875630d1b2..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL --99999999999999999999999999999999999999 -11111111111111111111111111111111111111 --999999999999999999999999999999999999 -111111111111111111111111111111111111 --99999999999999999999999999999999999 -11111111111111111111111111111111111 --0.0000000000000000000000000000000000001 0 -0 0 -0.0000000000000000000000000000000000001 0 -0.123456789012345 0.0137174210013716666666666666666666667 -0.1234567890123456789012345678901234578 0.0137174210013717421001371742100137175 -1.234567890123456 0.1371742100137173333333333333333333333 -1.2345678901234567890123456789012345678 0.1371742100137174210013717421001371742 -12.34567890123456 1.3717421001371733333333333333333333333 -12.345678901234567890123456789012345678 1.371742100137174210013717421001371742 -123.4567890123456 13.717421001371733333333333333333333333 -123.45678901234567890123456789012345678 13.71742100137174210013717421001371742 -1234.567890123456 137.17421001371733333333333333333333333 -1234.5678901234567890123456789012345678 137.1742100137174210013717421001371742 -12345.67890123456 1371.7421001371733333333333333333333333 -12345.678901234567890123456789012345678 1371.742100137174210013717421001371742 -123456.7890123456 13717.421001371733333333333333333333333 -123456.78901234567890123456789012345678 13717.42100137174210013717421001371742 -1234567.890123456 137174.21001371733333333333333333333333 -1234567.8901234567890123456789012345678 137174.2100137174210013717421001371742 -12345678.90123456 1371742.1001371733333333333333333333333 -12345678.901234567890123456789012345678 1371742.100137174210013717421001371742 -123456789.0123456 13717421.001371733333333333333333333333 -123456789.01234567890123456789012345678 13717421.00137174210013717421001371742 -1234567890.123456 137174210.01371733333333333333333333333 -1234567890.1234567890123456789012345678 137174210.0137174210013717421001371742 -12345678901.23456 1371742100.1371733333333333333333333333 -12345678901.234567890123456789012345678 1371742100.137174210013717421001371742 -123456789012.3456 13717421001.371733333333333333333333333 -123456789012.34567890123456789012345678 13717421001.37174210013717421001371742 -1234567890123.456 137174210013.71733333333333333333333333 -1234567890123.4567890123456789012345678 137174210013.7174210013717421001371742 -12345678901234.56 1371742100137.1733333333333333333333333 -12345678901234.567890123456789012345678 1371742100137.174210013717421001371742 -123456789012345.6 13717421001371.733333333333333333333333 -123456789012345.67890123456789012345678 13717421001371.74210013717421001371742 -1234567890123456.7890123456789012345678 137174210013717.4210013717421001371742 -12345678901234567.890123456789012345678 1371742100137174.210013717421001371742 -123456789012345678.90123456789012345678 13717421001371742.10013717421001371742 -1234567890123456789.0123456789012345678 137174210013717421.0013717421001371742 -12345678901234567890.123456789012345678 1371742100137174210.013717421001371742 -123456789012345678901.23456789012345678 13717421001371742100.13717421001371742 -1234567890123456789012.3456789012345678 137174210013717421001.3717421001371742 -12345678901234567890123.456789012345678 1371742100137174210013.717421001371742 -123456789012345678901234.56789012345678 13717421001371742100137.17421001371742 -1234567890123456789012345.6789012345678 137174210013717421001371.7421001371742 -12345678901234567890123456.789012345678 1371742100137174210013717.421001371742 -123456789012345678901234567.89012345678 13717421001371742100137174.21001371742 -1234567890123456789012345678.9012345678 137174210013717421001371742.1001371742 -12345678901234567890123456789.012345678 1371742100137174210013717421.001371742 -123456789012345678901234567890.12345678 13717421001371742100137174210.01371742 -1234567890123456789012345678901.2345678 137174210013717421001371742100.1371742 -12345678901234567890123456789012.345678 1371742100137174210013717421001.371742 -123456789012345678901234567890123.45678 13717421001371742100137174210013.71742 -1234567890123456789012345678901234.5678 137174210013717421001371742100137.1742 -12345678901234567890123456789012345.678 1371742100137174210013717421001371.742 -99999999999999999999999999999999999 11111111111111111111111111111111111 -123456789012345678901234567890123456.78 13717421001371742100137174210013717.42 -999999999999999999999999999999999999 111111111111111111111111111111111111 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -12345678901234567890123456789012345678 1371742100137174210013717421001371742 -99999999999999999999999999999999999999 11111111111111111111111111111111111111 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 b/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 deleted file mode 100644 index bd23d17293f79..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL --99999999999999999999999999999999999999 -3703703703703703703703703703703703703.7 --999999999999999999999999999999999999 -37037037037037037037037037037037037 --99999999999999999999999999999999999 -3703703703703703703703703703703703.6667 --0.0000000000000000000000000000000000001 0 -0 0 -0.0000000000000000000000000000000000001 0 -0.123456789012345 0.0045724736671238888888888888888888889 -0.1234567890123456789012345678901234578 0.0045724736671239140333790580700045725 -1.234567890123456 0.0457247366712391111111111111111111111 -1.2345678901234567890123456789012345678 0.0457247366712391403337905807000457247 -12.34567890123456 0.4572473667123911111111111111111111111 -12.345678901234567890123456789012345678 0.4572473667123914033379058070004572473 -123.4567890123456 4.5724736671239111111111111111111111111 -123.45678901234567890123456789012345678 4.5724736671239140333790580700045724733 -1234.567890123456 45.724736671239111111111111111111111111 -1234.5678901234567890123456789012345678 45.724736671239140333790580700045724733 -12345.67890123456 457.24736671239111111111111111111111111 -12345.678901234567890123456789012345678 457.24736671239140333790580700045724733 -123456.7890123456 4572.4736671239111111111111111111111111 -123456.78901234567890123456789012345678 4572.4736671239140333790580700045724733 -1234567.890123456 45724.736671239111111111111111111111111 -1234567.8901234567890123456789012345678 45724.736671239140333790580700045724733 -12345678.90123456 457247.36671239111111111111111111111111 -12345678.901234567890123456789012345678 457247.36671239140333790580700045724733 -123456789.0123456 4572473.6671239111111111111111111111111 -123456789.01234567890123456789012345678 4572473.6671239140333790580700045724733 -1234567890.123456 45724736.671239111111111111111111111111 -1234567890.1234567890123456789012345678 45724736.671239140333790580700045724733 -12345678901.23456 457247366.71239111111111111111111111111 -12345678901.234567890123456789012345678 457247366.71239140333790580700045724733 -123456789012.3456 4572473667.1239111111111111111111111111 -123456789012.34567890123456789012345678 4572473667.1239140333790580700045724733 -1234567890123.456 45724736671.239111111111111111111111111 -1234567890123.4567890123456789012345678 45724736671.239140333790580700045724733 -12345678901234.56 457247366712.39111111111111111111111111 -12345678901234.567890123456789012345678 457247366712.39140333790580700045724733 -123456789012345.6 4572473667123.9111111111111111111111111 -123456789012345.67890123456789012345678 4572473667123.9140333790580700045724733 -1234567890123456.7890123456789012345678 45724736671239.140333790580700045724733 -12345678901234567.890123456789012345678 457247366712391.40333790580700045724733 -123456789012345678.90123456789012345678 4572473667123914.0333790580700045724733 -1234567890123456789.0123456789012345678 45724736671239140.333790580700045724733 -12345678901234567890.123456789012345678 457247366712391403.33790580700045724733 -123456789012345678901.23456789012345678 4572473667123914033.3790580700045724733 -1234567890123456789012.3456789012345678 45724736671239140333.790580700045724733 -12345678901234567890123.456789012345678 457247366712391403337.90580700045724733 -123456789012345678901234.56789012345678 4572473667123914033379.0580700045724733 -1234567890123456789012345.6789012345678 45724736671239140333790.580700045724733 -12345678901234567890123456.789012345678 457247366712391403337905.80700045724733 -123456789012345678901234567.89012345678 4572473667123914033379058.0700045724733 -1234567890123456789012345678.9012345678 45724736671239140333790580.700045724733 -12345678901234567890123456789.012345678 457247366712391403337905807.00045724733 -123456789012345678901234567890.12345678 4572473667123914033379058070.0045724733 -1234567890123456789012345678901.2345678 45724736671239140333790580700.045724733 -12345678901234567890123456789012.345678 457247366712391403337905807000.45724733 -123456789012345678901234567890123.45678 4572473667123914033379058070004.5724733 -1234567890123456789012345678901234.5678 45724736671239140333790580700045.724733 -12345678901234567890123456789012345.678 457247366712391403337905807000457.24733 -99999999999999999999999999999999999 3703703703703703703703703703703703.6667 -123456789012345678901234567890123456.78 4572473667123914033379058070004572.4733 -999999999999999999999999999999999999 37037037037037037037037037037037037 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -12345678901234567890123456789012345678 457247366712391403337905807000457247.33 -99999999999999999999999999999999999999 3703703703703703703703703703703703703.7 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e b/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e deleted file mode 100644 index c1e0db0174c63..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e +++ /dev/null @@ -1,75 +0,0 @@ -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL -NULL NULL --99999999999999999999999999999999999999 NULL --999999999999999999999999999999999999 NULL --99999999999999999999999999999999999 NULL --0.0000000000000000000000000000000000001 NULL -0 0 -0.0000000000000000000000000000000000001 NULL -0.123456789012345 0.015241578753238669120562399025 -0.1234567890123456789012345678901234578 NULL -1.234567890123456 1.524157875323881726870921383936 -1.2345678901234567890123456789012345678 NULL -12.34567890123456 152.4157875323881726870921383936 -12.345678901234567890123456789012345678 NULL -123.4567890123456 15241.57875323881726870921383936 -123.45678901234567890123456789012345678 NULL -1234.567890123456 1524157.875323881726870921383936 -1234.5678901234567890123456789012345678 NULL -12345.67890123456 152415787.5323881726870921383936 -12345.678901234567890123456789012345678 NULL -123456.7890123456 15241578753.23881726870921383936 -123456.78901234567890123456789012345678 NULL -1234567.890123456 1524157875323.881726870921383936 -1234567.8901234567890123456789012345678 NULL -12345678.90123456 152415787532388.1726870921383936 -12345678.901234567890123456789012345678 NULL -123456789.0123456 15241578753238817.26870921383936 -123456789.01234567890123456789012345678 NULL -1234567890.123456 1524157875323881726.870921383936 -1234567890.1234567890123456789012345678 NULL -12345678901.23456 152415787532388172687.0921383936 -12345678901.234567890123456789012345678 NULL -123456789012.3456 15241578753238817268709.21383936 -123456789012.34567890123456789012345678 NULL -1234567890123.456 1524157875323881726870921.383936 -1234567890123.4567890123456789012345678 NULL -12345678901234.56 152415787532388172687092138.3936 -12345678901234.567890123456789012345678 NULL -123456789012345.6 15241578753238817268709213839.36 -123456789012345.67890123456789012345678 NULL -1234567890123456.7890123456789012345678 NULL -12345678901234567.890123456789012345678 NULL -123456789012345678.90123456789012345678 NULL -1234567890123456789.0123456789012345678 NULL -12345678901234567890.123456789012345678 NULL -123456789012345678901.23456789012345678 NULL -1234567890123456789012.3456789012345678 NULL -12345678901234567890123.456789012345678 NULL -123456789012345678901234.56789012345678 NULL -1234567890123456789012345.6789012345678 NULL -12345678901234567890123456.789012345678 NULL -123456789012345678901234567.89012345678 NULL -1234567890123456789012345678.9012345678 NULL -12345678901234567890123456789.012345678 NULL -123456789012345678901234567890.12345678 NULL -1234567890123456789012345678901.2345678 NULL -12345678901234567890123456789012.345678 NULL -123456789012345678901234567890123.45678 NULL -1234567890123456789012345678901234.5678 NULL -12345678901234567890123456789012345.678 NULL -99999999999999999999999999999999999 NULL -123456789012345678901234567890123456.78 NULL -999999999999999999999999999999999999 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -12345678901234567890123456789012345678 NULL -99999999999999999999999999999999999999 NULL diff --git a/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab b/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab deleted file mode 100644 index 81af0e4cd3ab8..0000000000000 --- a/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab +++ /dev/null @@ -1 +0,0 @@ -NULL NULL diff --git a/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c +++ b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be index ded23df148827..3aa0e30600f3c 100644 --- a/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be +++ b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be @@ -1 +1 @@ -ds=__HIVE_DEFAULT_PARTITION__ \ No newline at end of file +ds=__HIVE_DEFAULT_PARTITION__ diff --git a/sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 b/sql/hive/src/test/resources/golden/delimiter-1-121ff21e6931a82235de8301118cbed8 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 rename to sql/hive/src/test/resources/golden/delimiter-1-121ff21e6931a82235de8301118cbed8 diff --git a/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 index f438072c76b5f..424a2fee06987 100644 --- a/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 +++ b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 @@ -1,3 +1,3 @@ 35 40 48 32 -100100 40 \ No newline at end of file +100100 40 diff --git a/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 index f438072c76b5f..424a2fee06987 100644 --- a/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 +++ b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 @@ -1,3 +1,3 @@ 35 40 48 32 -100100 40 \ No newline at end of file +100100 40 diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 b/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 deleted file mode 100644 index d980efc81b947..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 +++ /dev/null @@ -1,6 +0,0 @@ -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 b/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 deleted file mode 100644 index 01b9151074b22..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 11:42:35 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/test_table -Table Type: MANAGED_TABLE -Table Parameters: - comment table comment\ntwo lines - transient_lastDdlTime 1389728555 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a deleted file mode 100644 index c56a79e4f322e..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1","comment":"Test database","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 deleted file mode 100644 index aa08c38c68d1d..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1","params":{"id":"jsondb1"},"comment":"Test database","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 deleted file mode 100644 index 513aeaab1dc66..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 +++ /dev/null @@ -1 +0,0 @@ -{"databases":["default","jsondb1"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c b/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c deleted file mode 100644 index 606069d6291b4..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c +++ /dev/null @@ -1 +0,0 @@ -{"databases":["jsondb1"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a deleted file mode 100644 index bea7c01440c46..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 deleted file mode 100644 index bea7c01440c46..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 +++ /dev/null @@ -1 +0,0 @@ -{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsondb1.db","database":"jsondb1"} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 index 381821184d693..1ed0de6860c08 100644 --- a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 +++ b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 @@ -1,19 +1,19 @@ # col_name data_type comment -key int None +key int # Partition Information # col_name data_type comment -value string None +value string # Detailed Partition Information Partition Value: [val_86] Database: default Table: view_partitioned -CreateTime: Fri Feb 07 15:09:16 PST 2014 +CreateTime: Tue Oct 21 01:26:15 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Location: null Partition Parameters: - transient_lastDdlTime 1391814556 + transient_lastDdlTime 1413879975 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 deleted file mode 100644 index 7b51873776ad8..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 +++ /dev/null @@ -1 +0,0 @@ -{"columns":[{"name":"key","type":"int"}]} diff --git a/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 deleted file mode 100644 index 35fac1b6f2579..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 +++ /dev/null @@ -1,12 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2_abcdefghiklmnopqrstuvxyz string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that is - greater than 80 chars and is - likely to spill into multiple - lines \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 b/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 deleted file mode 100644 index b57f8955ca397..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 +++ /dev/null @@ -1,24 +0,0 @@ -col_name data_type comment - -col1 int col1 one - line - comment -col2_abcdefghiklmnopqrstuvxyz string col2 - two lines - comment -col3 string col3 - three - lines - comment -col4 string col4 very - long - comment - that is - greater - than 80 - chars and - is likely - to spill - into - multiple - lines \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 b/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 deleted file mode 100644 index 3b7fe3c133089..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 +++ /dev/null @@ -1,37 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that - is greater than 80 chars - and is likely to spill into - multiple lines -col5 string col5 very long multi-line - comment where each line is - very long by itself and is - likely to spill - into multiple lines. Lorem - ipsum dolor sit amet, - consectetur adipiscing - elit. Proin in dolor nisl, - sodales - adipiscing tortor. Integer - venenatis -col6 string This comment has a very - long single word ABCDEFGHIJ - KLMNOPQRSTUVXYZabcdefghijkl - mnopqrstuvzxyz123 which - will not fit in a line by - itself for small column - widths. -col7_nocomment string None -ds string None - -# Partition Information -col_name data_type comment - -ds string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 b/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 deleted file mode 100644 index 49175da27357f..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 +++ /dev/null @@ -1,23 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that is greater than 80 - chars and is likely to spill into multiple - lines -col5 string col5 very long multi-line comment where each - line is very long by itself and is likely to - spill - into multiple lines. Lorem ipsum dolor sit - amet, consectetur adipiscing elit. Proin in - dolor nisl, sodales - adipiscing tortor. Integer venenatis -col6 string This comment has a very long single word ABCDEF - GHIJKLMNOPQRSTUVXYZabcdefghijklmnopqrstuvzxyz12 - 3 which will not fit in a line by itself for - small column widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 b/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 deleted file mode 100644 index c3e77e079a9d4..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 +++ /dev/null @@ -1,14 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that is greater than 80 chars and is likely to spill into multiple lines -col5 string col5 very long multi-line comment where each line is very long by itself and is likely to spill - into multiple lines. Lorem ipsum dolor sit amet, consectetur adipiscing elit. Proin in dolor nisl, sodales - adipiscing tortor. Integer venenatis -col6 string This comment has a very long single word ABCDEFGHIJKLMNOPQRSTUVXYZabcdefghijklmnopqrstuvzxyz123 which will not fit in a line by itself for small column widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 b/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 deleted file mode 100644 index 452f75a11fdd0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 +++ /dev/null @@ -1,44 +0,0 @@ -col_name data_type comment - -col1 int col1 one line - comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long - comment that is - greater than 80 - chars and is - likely to spill - into multiple - lines -col5 string col5 very long - multi-line - comment where - each line is very - long by itself - and is likely to - spill - into multiple - lines. Lorem - ipsum dolor sit - amet, consectetur - adipiscing elit. - Proin in dolor - nisl, sodales - adipiscing - tortor. Integer - venenatis -col6 string This comment has - a very long - single word ABCDE - FGHIJKLMNOPQRSTUV - XYZabcdefghijklmn - opqrstuvzxyz123 - which will not - fit in a line by - itself for small - column widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 deleted file mode 100644 index ee5a10c85057a..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 +++ /dev/null @@ -1,31 +0,0 @@ -col_name data_type comment - -col1 int col1 one line comment -col2 string col2 - two lines comment -col3 string col3 - three lines - comment -col4 string col4 very long comment that - is greater than 80 chars - and is likely to spill into - multiple lines -col5 string col5 very long multi-line - comment where each line is - very long by itself and is - likely to spill - into multiple lines. Lorem - ipsum dolor sit amet, - consectetur adipiscing - elit. Proin in dolor nisl, - sodales - adipiscing tortor. Integer - venenatis -col6 string This comment has a very - long single word ABCDEFGHIJ - KLMNOPQRSTUVXYZabcdefghijkl - mnopqrstuvzxyz123 which - will not fit in a line by - itself for small column - widths. -col7_nocomment string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 b/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 deleted file mode 100644 index 4184ce21dc079..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Table Information Table(tableName:t1, dbName:db1, owner:marmbrus, createTime:1389728588, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728588}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f b/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f deleted file mode 100644 index c94d6dcb90042..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f +++ /dev/null @@ -1,33 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Table Information -Database: db1 -Owner: marmbrus -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 b/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 b/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 b/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 deleted file mode 100644 index f3d242157dd98..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 +++ /dev/null @@ -1,3 +0,0 @@ -# col_name data_type comment - -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 b/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 b/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 b/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 deleted file mode 100644 index f3d242157dd98..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 +++ /dev/null @@ -1,3 +0,0 @@ -# col_name data_type comment - -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 b/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa b/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa deleted file mode 100644 index 0dea48c260ab2..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa +++ /dev/null @@ -1 +0,0 @@ -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a b/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a deleted file mode 100644 index f3d242157dd98..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a +++ /dev/null @@ -1,3 +0,0 @@ -# col_name data_type comment - -key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 b/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f b/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f deleted file mode 100644 index aa25ca5a29dd3..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Partition Information Partition(values:[4, 5], dbName:db1, tableName:t1, createTime:1389728588, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728588}) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 b/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 deleted file mode 100644 index 311870f6ad6b0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Partition Information -Partition Value: [4, 5] -Database: db1 -Table: t1 -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5 -Partition Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 b/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e b/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e deleted file mode 100644 index aa25ca5a29dd3..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Partition Information Partition(values:[4, 5], dbName:db1, tableName:t1, createTime:1389728588, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389728588}) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec b/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec deleted file mode 100644 index 311870f6ad6b0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Partition Information -Partition Value: [4, 5] -Database: db1 -Table: t1 -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1/ds=4/part=5 -Partition Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 b/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d b/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d deleted file mode 100644 index 4184ce21dc079..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d +++ /dev/null @@ -1,12 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -Detailed Table Information Table(tableName:t1, dbName:db1, owner:marmbrus, createTime:1389728588, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key1, type:int, comment:null), FieldSchema(name:value1, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:part, type:string, comment:null)], parameters:{transient_lastDdlTime=1389728588}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 b/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 deleted file mode 100644 index c94d6dcb90042..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 +++ /dev/null @@ -1,33 +0,0 @@ -# col_name data_type comment - -key1 int None -value1 string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None - -# Detailed Table Information -Database: db1 -Owner: marmbrus -CreateTime: Tue Jan 14 11:43:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/db1.db/t1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1389728588 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b b/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b deleted file mode 100644 index 4f76eaca6cd8b..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b +++ /dev/null @@ -1,10 +0,0 @@ -key1 int None -value1 string None -ds string None -part string None - -# Partition Information -# col_name data_type comment - -ds string None -part string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c deleted file mode 100644 index f8bc404bf7308..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c +++ /dev/null @@ -1 +0,0 @@ -{"tables":["jsontable","src","srcpart"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a b/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a deleted file mode 100644 index 5895645dbbb50..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a +++ /dev/null @@ -1 +0,0 @@ -{"tables":["jsontable"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 b/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 deleted file mode 100644 index 353bf2df92f18..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 +++ /dev/null @@ -1 +0,0 @@ -{"tables":[]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 b/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 deleted file mode 100644 index 96c1178ae6eab..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 +++ /dev/null @@ -1 +0,0 @@ -{"columns":[{"name":"key","type":"int"},{"name":"value","type":"string"}]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 b/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 deleted file mode 100644 index 4cf10d1d762b0..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 +++ /dev/null @@ -1 +0,0 @@ -{"columns":[{"name":"key","type":"int"},{"name":"value","type":"string"}],"tableInfo":{"owner":"marmbrus","parameters":{"id":"jsontable","last_modified_by":"marmbrus","last_modified_time":"1389728616","transient_lastDdlTime":"1389728616","comment":"json table"},"createTime":1389728615,"dbName":"default","tableName":"jsontable","privileges":null,"tableType":"MANAGED_TABLE","sd":{"location":"file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/jsontable","parameters":{},"inputFormat":"org.apache.hadoop.mapred.TextInputFormat","outputFormat":"org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat","compressed":false,"cols":[{"name":"key","type":"int","comment":null,"setName":true,"setType":true,"setComment":false},{"name":"value","type":"string","comment":null,"setName":true,"setType":true,"setComment":false}],"serdeInfo":{"name":null,"parameters":{"serialization.format":"1"},"serializationLib":"org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe","setName":false,"parametersSize":1,"setParameters":true,"setSerializationLib":true},"colsSize":2,"skewedInfo":{"skewedColNames":[],"skewedColValues":[],"skewedColValueLocationMaps":{},"skewedColNamesSize":0,"skewedColNamesIterator":[],"setSkewedColNames":true,"skewedColValuesSize":0,"skewedColValuesIterator":[],"setSkewedColValues":true,"skewedColValueLocationMapsSize":0,"setSkewedColValueLocationMaps":true},"bucketCols":[],"numBuckets":-1,"sortCols":[],"storedAsSubDirectories":false,"setSkewedInfo":true,"parametersSize":0,"setParameters":true,"colsIterator":[{"name":"key","type":"int","comment":null,"setName":true,"setType":true,"setComment":false},{"name":"value","type":"string","comment":null,"setName":true,"setType":true,"setComment":false}],"setCols":true,"setLocation":true,"setInputFormat":true,"setOutputFormat":true,"setCompressed":true,"setNumBuckets":true,"setSerdeInfo":true,"bucketColsSize":0,"bucketColsIterator":[],"setBucketCols":true,"sortColsSize":0,"sortColsIterator":[],"setSortCols":true,"setStoredAsSubDirectories":true},"partitionKeys":[],"viewOriginalText":null,"lastAccessTime":0,"retention":0,"viewExpandedText":null,"partitionKeysSize":0,"setTableType":true,"setTableName":true,"setDbName":true,"setOwner":true,"setCreateTime":true,"setLastAccessTime":true,"setRetention":true,"setSd":true,"partitionKeysIterator":[],"setPartitionKeys":true,"parametersSize":5,"setParameters":true,"setViewOriginalText":false,"setViewExpandedText":false,"setPrivileges":false}} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b +++ b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c +++ b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 b/sql/hive/src/test/resources/golden/disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 rename to sql/hive/src/test/resources/golden/disable_file_format_check-2-2d27f92dfced693fa3a68ecce5e2e838 diff --git a/sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 b/sql/hive/src/test/resources/golden/disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 rename to sql/hive/src/test/resources/golden/disable_file_format_check-4-a2150709a6ff73326bdf4865dd124a23 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-a071dedef216e84d1cb2f0de6d34fd1a diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b deleted file mode 100644 index 7ae602958428e..0000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b +++ /dev/null @@ -1 +0,0 @@ -238 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b similarity index 100% rename from sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-ce3797dc14a603cba2a5e58c8612de5b diff --git a/sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-f5340880d2be7b0643eb995673e89d11 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c deleted file mode 100644 index ca21e093aa698..0000000000000 --- a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c +++ /dev/null @@ -1 +0,0 @@ -1 {"a1":"b1"} foo1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-714ab8c97f4d8993680b91e1ed8f3782 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c new file mode 100644 index 0000000000000..573c4b56de599 --- /dev/null +++ b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-34064fd15c28dba55865cb8f3c5ba68c @@ -0,0 +1 @@ +1 {"a1":"b1"} foo1 diff --git a/sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 rename to sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-6-f40a07d7654573e1a8517770eb8529e7 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/distinct_stats-0-418ec894d08c33fd712eb358f579b7a0 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/distinct_stats-0-418ec894d08c33fd712eb358f579b7a0 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/distinct_stats-1-10987e425ba8ba8d9c01538f16eab970 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/distinct_stats-1-10987e425ba8ba8d9c01538f16eab970 diff --git a/sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 b/sql/hive/src/test/resources/golden/distinct_stats-2-a2d8f812612283b20ec3f1e92a263440 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 rename to sql/hive/src/test/resources/golden/distinct_stats-2-a2d8f812612283b20ec3f1e92a263440 diff --git a/sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 b/sql/hive/src/test/resources/golden/distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 rename to sql/hive/src/test/resources/golden/distinct_stats-3-a4397664f1f109ef0aa5ff36961b25b diff --git a/sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 b/sql/hive/src/test/resources/golden/distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 rename to sql/hive/src/test/resources/golden/distinct_stats-4-e540680af4a857404a0cb7cabc1bcf31 diff --git a/sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 b/sql/hive/src/test/resources/golden/distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc similarity index 100% rename from sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 rename to sql/hive/src/test/resources/golden/distinct_stats-5-32e9736bf27c1d2e4399a8125e14befc diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc b/sql/hive/src/test/resources/golden/distinct_stats-6-297a53801744e6c4786e315e32c6189a similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc rename to sql/hive/src/test/resources/golden/distinct_stats-6-297a53801744e6c4786e315e32c6189a diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 b/sql/hive/src/test/resources/golden/distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 rename to sql/hive/src/test/resources/golden/distinct_stats-7-92b9ef922e6b63a9de3ebcc23ee2d02d diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 b/sql/hive/src/test/resources/golden/distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163 similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 rename to sql/hive/src/test/resources/golden/distinct_stats-8-ca16024e6f5399b1d035f5b9fd665163 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/distinct_stats-9-4b2407991ccf180e0eb38bf3d2ef2ec8 diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee similarity index 100% rename from sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-2-fb7b53f61989f4f645dac4a8f017d6ee diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-3-46fe5bb027667f528d7179b239e3427f diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-4-26dcd2b2f263b5b417430efcf354663a diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-5-2a1bd5ed3955825a9dbb76769f7fe4ea diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-6-7a9e67189d3d4151f23b12c22bde06b5 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-7-16c31455a193e1cb06a2ede4e9f5d5dd diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-8-2a1bd5ed3955825a9dbb76769f7fe4ea diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-9-40110efef10f6f7b873dcd1d53463101 diff --git a/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 +++ b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 +++ b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-0-a99af48bbcbaba062e9bc387ae2b4975 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-1-e67a0f7ff61a97b2c49386890ea88c54 diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25 similarity index 100% rename from sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-2-19915227905aab376d918b3cada85c25 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-3-e32b952789a115ff02201dfa618d92b2 diff --git a/sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-4-58aff7fa05fba3a7549629a17e285036 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-5-e32b952789a115ff02201dfa618d92b2 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/drop_index_removes_partition_dirs-6-4642f8a18bf7409849f2e91d7a05f352 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 index c9d701778f9ab..2895d472ca5d9 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 @@ -1 +1 @@ -b=2/c=2 \ No newline at end of file +b=2/c=2 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 index 53e09b6e34202..0f9c2f1d90639 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 @@ -1,3 +1,3 @@ b=1/c=1 b=1/c=2 -b=2/c=2 \ No newline at end of file +b=2/c=2 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 b/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 index 31b543e8b4122..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 @@ -1,12 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_DROPPARTS mp (TOK_PARTSPEC (TOK_PARTVAL b = '1'))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Drop Table Operator: - Drop Table - table: mp - diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 index c9d701778f9ab..2895d472ca5d9 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 @@ -1 +1 @@ -b=2/c=2 \ No newline at end of file +b=2/c=2 diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 index d1e9fd1cd0a21..30eb53bdc8e70 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -1,10 +1,10 @@ -a string None -b int None -c string None -d string None +a string +b int +c string +d string # Partition Information # col_name data_type comment -c string None -d string None \ No newline at end of file +c string +d string diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 index 3377ef5cf498f..5e6d96c4e60aa 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 @@ -6,4 +6,4 @@ c=India/d=3 c=Russia/d=3 c=US/d=1 c=US/d=2 -c=Uganda/d=2 \ No newline at end of file +c=Uganda/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 index 40e71fb79ad0f..e91541d1527d7 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 @@ -5,4 +5,4 @@ c=Greece/d=2 c=India/d=3 c=Russia/d=3 c=US/d=2 -c=Uganda/d=2 \ No newline at end of file +c=Uganda/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 index c03d86a551c29..316e63e21e7b1 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 @@ -3,4 +3,4 @@ c=France/d=4 c=Germany/d=2 c=Greece/d=2 c=India/d=3 -c=Russia/d=3 \ No newline at end of file +c=Russia/d=3 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 index 133c0256f898f..231c59f365307 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 @@ -2,4 +2,4 @@ c=Canada/d=3 c=France/d=4 c=Germany/d=2 c=Greece/d=2 -c=India/d=3 \ No newline at end of file +c=India/d=3 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 index 98e52eedc2b03..02a7003ca12de 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 @@ -1,3 +1,3 @@ c=Canada/d=3 c=France/d=4 -c=Germany/d=2 \ No newline at end of file +c=Germany/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 index 687f7dd7e13a0..2ffea3b50a9b4 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 @@ -1 +1 @@ -c=France/d=4 \ No newline at end of file +c=France/d=4 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 index 687f7dd7e13a0..2ffea3b50a9b4 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 @@ -1 +1 @@ -c=France/d=4 \ No newline at end of file +c=France/d=4 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 index 1329d173d6a21..ca4194fbcf3ff 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -1,10 +1,10 @@ -a string None -b int None -c int None -d int None +a string +b int +c int +d int # Partition Information # col_name data_type comment -c int None -d int None \ No newline at end of file +c int +d int diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 index 08051a26d24cc..7ace4dc662306 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 @@ -2,4 +2,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 index 7f6e4ae8abf83..1ca1833c09245 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 @@ -1,3 +1,3 @@ c=1/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-13-8117981303487dc4c4873356931ef26a diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..fee1b1ad01412 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-14-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1 @@ +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba b/sql/hive/src/test/resources/golden/drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-15-67d75c72ea2d3982c55f3a850d93f83c diff --git a/sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 rename to sql/hive/src/test/resources/golden/drop_partitions_filter2-7-f34625fd49a5e655cba3abb5cb8c5417 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 index 226ef460b53a6..b77f18cd02020 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 @@ -3,4 +3,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=30/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 index 01562f65d807f..77bc36b96870b 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 @@ -1,10 +1,10 @@ -a string None -b int None -c string None -d int None +a string +b int +c string +d int # Partition Information # col_name data_type comment -c string None -d int None \ No newline at end of file +c string +d int diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 index 08051a26d24cc..50e8df00f1597 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 @@ -2,4 +2,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=3/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 index 7f6e4ae8abf83..20bc2b0c74d32 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 @@ -1,3 +1,3 @@ c=1/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=3/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 index 226ef460b53a6..6200b3ad2dd94 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 @@ -3,4 +3,4 @@ c=1/d=2 c=2/d=1 c=2/d=2 c=3/d=1 -c=3/d=2 \ No newline at end of file +c=3/d=2 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 index 8caab1c99b27d..73f873dbcf00b 100644 --- a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 +++ b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 @@ -1,10 +1,10 @@ -c1 string None -c2 string None -p string None +c1 string +c2 string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1389728724, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5682582801957547950/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1389728724, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1389728724}) \ No newline at end of file +Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl_protectmode_no_drop, createTime:1413879999, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:c1, type:string, comment:null), FieldSchema(name:c2, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl_protectmode_no_drop/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413879999, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1413879999, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 +++ b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 +++ b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 index fbe12dca4efc8..a4b5a45443235 100644 --- a/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 +++ b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 @@ -1,3 +1,3 @@ p=p1 p=p2 -p=p3 \ No newline at end of file +p=p3 diff --git a/sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3 similarity index 100% rename from sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-0-97b52abf021c81b8364041c1a0bbccf3 diff --git a/sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff similarity index 100% rename from sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-1-f11a45c42752d06821ccd26d948d51ff diff --git a/sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7 similarity index 100% rename from sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-2-c0b85445b616f93c5e6d090fa35072e7 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-3-10a71bca930d911cc4c2022575b17299 diff --git a/sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208 similarity index 100% rename from sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-4-b2ca31dd6cc5c32e33df700786f5b208 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-5-10a71bca930d911cc4c2022575b17299 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/drop_table_removes_partition_dirs-6-d1c175a9d042ecd389f2f93fc867591d diff --git a/sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260 similarity index 100% rename from sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-3cd14bc5b126ff8b337c4abc09134260 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe +++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-ce929ee6a92b81d8080ca322c1c38a4b diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-f3a5a998099b756a21cf9122a15b09d5 diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-c8aa1ebce4b0b2b7f46bb3c2502f8b49 diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-fed732d2e7d94a4fc02e7694f9f9a39c diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-adab6a0187003ab7ee6f217c9e409d91 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 +++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-adab6a0187003ab7ee6f217c9e409d91 diff --git a/sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b rename to sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-4b099f87e221b8fd5c0d0d4a97c0d146 diff --git a/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf b/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf deleted file mode 100644 index 8273b7ed19da6..0000000000000 --- a/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf +++ /dev/null @@ -1,10 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -100 val_100 -100 val_100 -103 val_103 -103 val_103 -104 val_104 -104 val_104 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 b/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 deleted file mode 100644 index 045906a29a1da..0000000000000 --- a/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 +++ /dev/null @@ -1,10 +0,0 @@ -98 val_98 -98 val_98 -97 val_97 -97 val_97 -96 val_96 -95 val_95 -95 val_95 -92 val_92 -90 val_90 -90 val_90 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 deleted file mode 100644 index a949a93dfcca6..0000000000000 --- a/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 +++ /dev/null @@ -1 +0,0 @@ -128 diff --git a/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 deleted file mode 100644 index e2799aa7e14b5885aad48117308f489740683449..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 252 zcmV~$15yM4006)yw>8yeIstVOFf?K*Vo(yd3YKK%v^3K=qN#Hcaj zCQO<#ZN{uQ^A;>xvTVhwHS0EP+OloOu08t>96ECB#HlmqE?l~D?Z&M;_Z~cY^6bT{ OH}5`t`tt3^uRs6QQ6wq= diff --git a/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 deleted file mode 100644 index a949a93dfcca6..0000000000000 --- a/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 +++ /dev/null @@ -1 +0,0 @@ -128 diff --git a/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 deleted file mode 100644 index e2799aa7e14b5885aad48117308f489740683449..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 252 zcmV~$15yM4006)yw>8yeIstVOFf?K*Vo(yd3YKK%v^3K=qN#Hcaj zCQO<#ZN{uQ^A;>xvTVhwHS0EP+OloOu08t>96ECB#HlmqE?l~D?Z&M;_Z~cY^6bT{ OH}5`t`tt3^uRs6QQ6wq= diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d b/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 b/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 b/sql/hive/src/test/resources/golden/explain_rearrange-0-6f7c8515c354fb050829ebd66413425 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 rename to sql/hive/src/test/resources/golden/explain_rearrange-0-6f7c8515c354fb050829ebd66413425 diff --git a/sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 b/sql/hive/src/test/resources/golden/explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 rename to sql/hive/src/test/resources/golden/explain_rearrange-1-378d42317b39c6519f15bd2f99c5ddc4 diff --git a/sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e b/sql/hive/src/test/resources/golden/explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e rename to sql/hive/src/test/resources/golden/explain_rearrange-10-3f2680208772a0e51aefc4ef5604dddf diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/explain_rearrange-11-f2ca12a948fd9b5b842168e7c7d7b768 diff --git a/sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee rename to sql/hive/src/test/resources/golden/explain_rearrange-12-3d63f0bb8fbacbcff9e5989ddf1bcc8e diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/explain_rearrange-13-5baad22ed7efa18d73eb8349e57cf331 diff --git a/sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef b/sql/hive/src/test/resources/golden/explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef rename to sql/hive/src/test/resources/golden/explain_rearrange-14-490d6253b73064ce403e4d04a8bc18f3 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-2-24ca942f094b14b92086305cc125e833 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/explain_rearrange-2-24ca942f094b14b92086305cc125e833 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f similarity index 100% rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/explain_rearrange-3-3b0f76816be2c1b18a2058027a19bc9f diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 b/sql/hive/src/test/resources/golden/explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 rename to sql/hive/src/test/resources/golden/explain_rearrange-4-86473a0498e4361e4db0b4a22f2e8571 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/explain_rearrange-5-d0ec6d66ff349db09fd455eec149efdb diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/explain_rearrange-6-cda81d86d127fca0e2fbc2161e91400d diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/explain_rearrange-7-5b13cfa4b730e38ef2794c1532968e04 diff --git a/sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 b/sql/hive/src/test/resources/golden/explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 rename to sql/hive/src/test/resources/golden/explain_rearrange-8-1fd9c02fc67c3a403cb73eb10ed9fc12 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac similarity index 100% rename from sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/explain_rearrange-9-73b9ac83dbc9874dc9379ad4364d40ac diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 +++ b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 b/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 index fb12634ea81c1..8059361d2485f 100644 --- a/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 +++ b/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 @@ -1 +1 @@ -500 130091 260.182 0 498 142.92680950752384 20428.072876000006 \ No newline at end of file +500 130091 260.182 0 498 142.92680950752384 20428.072876000006 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 +++ b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 index d23e05acf7ba5..e34118512c1d7 100644 --- a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 +++ b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 @@ -497,4 +497,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 index 6280b32facd66..1cf9f21c9ed88 100644 --- a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 +++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 @@ -1,4 +1,4 @@ -key int None -value string None +key int +value string -Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823397, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823397}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1413880056, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413880056}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 index e793ec2f946e5..62d6734063fdd 100644 --- a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 +++ b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 @@ -1,4 +1,4 @@ -key int None -value string None +key int +value string -Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823407}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1413880064, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413880064}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e b/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e index cc545367b951b..31956a614026d 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e @@ -9,4 +9,4 @@ 66 val_66 213 val_213 146 val_146 -406 val_406 \ No newline at end of file +406 val_406 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd index 140c6590a27fc..51eb2d30c97eb 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd @@ -1 +1 @@ -4 5 0 2010-04-17 \ No newline at end of file +4 5 0 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 index 51f5701ceae46..7dc9c13cfee04 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 @@ -1 +1 @@ -4 1 1 8 4 5 1 0 9 U 2 2 0 2 1 1 J C A U 2 s 2 NULL NULL NULL NULL NULL NULL 1 j S 6 NULL 1 2 J g 1 e 2 1 2 U P p 3 0 0 0 1 1 1 0 0 0 6 2 j NULL NULL NULL NULL NULL NULL 5 NULL NULL j 2 2 1 2 2 1 1 1 1 1 1 1 1 32 NULL 2010-04-17 \ No newline at end of file +4 1 1 8 4 5 1 0 9 U 2 2 0 2 1 1 J C A U 2 s 2 NULL NULL NULL NULL NULL NULL 1 j S 6 NULL 1 2 J g 1 e 2 1 2 U P p 3 0 0 0 1 1 1 0 0 0 6 2 j NULL NULL NULL NULL NULL NULL 5 NULL NULL j 2 2 1 2 2 1 1 1 1 1 1 1 1 32 NULL 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 index 69dcc336a73f2..19a6df9f78cd7 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 @@ -1 +1 @@ -5 5 4 \ No newline at end of file +5 5 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-4-fe8b55e4e4098d7a2662338783a50306 diff --git a/sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-5-cf724251613216ec54f8ac2e6b9b92fd diff --git a/sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d b/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d rename to sql/hive/src/test/resources/golden/filter_join_breaktask2-6-8c782ae8f8245bdbe90d068a6b577d1e diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e b/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e index c0f577c3cd2fd..d6961f6e46e93 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e @@ -1 +1 @@ -5 name NULL 2 kavin NULL 9 c 8 0 0 7 1 2 0 3 2 NULL 1 NULL 3 2 0 0 5 10 2010-04-17 \ No newline at end of file +5 name NULL 2 kavin NULL 9 c 8 0 0 7 1 2 0 3 2 NULL 1 NULL 3 2 0 0 5 10 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 index e426b4879bcb6..be23778bd2cc4 100644 --- a/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 @@ -1 +1 @@ -5 1 1 1 0 0 4 2010-04-17 \ No newline at end of file +5 1 1 1 0 0 4 2010-04-17 diff --git a/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d +++ b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 +++ b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 +++ b/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd b/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd index 8939d9f8af186..07df09912b06e 100644 --- a/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd +++ b/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd @@ -306,4 +306,4 @@ val_92 1 1 111 val_95 2 1 111 val_96 1 1 111 val_97 2 1 111 -val_98 2 1 111 \ No newline at end of file +val_98 2 1 111 diff --git a/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d b/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d index 82a0329b4d459..efbfb713807a0 100644 --- a/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d +++ b/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d @@ -306,4 +306,4 @@ 95 2 1 111 96 1 1 111 97 2 1 111 -98 2 1 111 \ No newline at end of file +98 2 1 111 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/groupby12-0-67e864faaff4c6b2a8e1c9fbd188bb66 diff --git a/sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e b/sql/hive/src/test/resources/golden/groupby12-1-13ab74a58da514fe01dbeda0c3e79883 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e rename to sql/hive/src/test/resources/golden/groupby12-1-13ab74a58da514fe01dbeda0c3e79883 diff --git a/sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c b/sql/hive/src/test/resources/golden/groupby12-2-fd150794945892f3c926a1881cd819f4 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c rename to sql/hive/src/test/resources/golden/groupby12-2-fd150794945892f3c926a1881cd819f4 diff --git a/sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 b/sql/hive/src/test/resources/golden/groupby12-3-8018bb917a0706925c14421ec2761663 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 rename to sql/hive/src/test/resources/golden/groupby12-3-8018bb917a0706925c14421ec2761663 diff --git a/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..98f8836673e8a --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby12-4-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,309 @@ +3 1 +1 1 +1 1 +3 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +2 1 +1 1 +1 1 +2 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +3 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +2 1 +1 1 +3 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +1 1 +1 1 +3 1 +1 1 +2 1 +1 1 +2 1 +2 1 +2 1 +2 1 +2 1 +1 1 +1 1 +2 1 +1 1 +1 1 +2 1 +3 1 +2 1 +2 1 +1 1 +3 1 +2 1 +1 1 +1 1 +2 1 +1 1 +2 1 +4 1 +1 1 +1 1 +2 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +3 1 +1 1 +4 1 +1 1 +2 1 +2 1 +2 1 +2 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +3 1 +1 1 +1 1 +2 1 +1 1 +3 1 +1 1 +2 1 +1 1 +2 1 +3 1 +2 1 +1 1 +1 1 +2 1 +2 1 +2 1 +3 1 +2 1 +2 1 +1 1 +2 1 +2 1 +1 1 +2 1 +2 1 +1 1 +2 1 +2 1 +1 1 +1 1 +2 1 +5 1 +2 1 +1 1 +2 1 +2 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +2 1 +3 1 +1 1 +1 1 +4 1 +2 1 +2 1 +2 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +3 1 +1 1 +1 1 +1 1 +2 1 +1 1 +2 1 +1 1 +3 1 +1 1 +3 1 +2 1 +3 1 +2 1 +2 1 +1 1 +2 1 +3 1 +2 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +2 1 +1 1 +5 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +3 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +3 1 +1 1 +1 1 +1 1 +1 1 +1 1 +2 1 +3 1 +2 1 +2 1 +1 1 +5 1 +1 1 +3 1 +2 1 +4 1 +1 1 +3 1 +1 1 +2 1 +2 1 +3 1 +1 1 +1 1 +1 1 +2 1 +1 1 +2 1 +3 1 +3 1 +1 1 +1 1 +1 1 +1 1 +3 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +3 1 +1 1 +1 1 +2 1 +2 1 +1 1 +2 1 +2 1 +3 1 +1 1 +4 1 +5 1 +1 1 +1 1 +1 1 +1 1 +2 1 +1 1 +3 1 +1 1 +1 1 +1 1 +1 1 +1 1 +1 1 +4 1 +1 1 +1 1 +2 1 +1 1 +1 1 +1 1 +1 1 +1 1 +3 1 diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e b/sql/hive/src/test/resources/golden/groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e rename to sql/hive/src/test/resources/golden/groupby1_limit-3-607512a8dd16cd9ddde561eeabfa51db diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 b/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 index 9cf9606d751ac..f92f60d11d4ef 100644 --- a/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 +++ b/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 @@ -2,4 +2,4 @@ 2 2.0 4 4.0 5 15.0 -8 8.0 \ No newline at end of file +8 8.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb b/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-6-f7eda18efd187ec4bf4cb135833746cb @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c index 3210a3048ff48..ba568b8fd6cf2 100644 --- a/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c +++ b/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c @@ -7,4 +7,4 @@ 6 5 6398.0 7 6 7735.0 8 8 8762.0 -9 7 91047.0 \ No newline at end of file +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e b/sql/hive/src/test/resources/golden/groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e rename to sql/hive/src/test/resources/golden/groupby2_limit-1-a56d6499aef913e11ef599ac8b4f2a25 diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 b/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 deleted file mode 100644 index 9cf9606d751ac..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 +++ /dev/null @@ -1,5 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a b/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a new file mode 100644 index 0000000000000..f92f60d11d4ef --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_limit-2-e7a95dc27fbfcb10bf92a6db61522b6a @@ -0,0 +1,5 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 3210a3048ff48..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 -1 71 116414.0 -2 69 225571.0 -3 62 332004.0 -4 74 452763.0 -5 6 5397.0 -6 5 6398.0 -7 6 7735.0 -8 8 8762.0 -9 7 91047.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..ba568b8fd6cf2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,10 @@ +0 1 00.0 +1 71 116414.0 +2 69 225571.0 +3 62 332004.0 +4 74 452763.0 +5 6 5397.0 +6 5 6398.0 +7 6 7735.0 +8 8 8762.0 +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f index 3210a3048ff48..ba568b8fd6cf2 100644 --- a/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 5 6398.0 7 6 7735.0 8 8 8762.0 -9 7 91047.0 \ No newline at end of file +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 b/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 new file mode 100644 index 0000000000000..ba568b8fd6cf2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-6-232957c90d04497da7fe4e599e0dbb94 @@ -0,0 +1,10 @@ +0 1 00.0 +1 71 116414.0 +2 69 225571.0 +3 62 332004.0 +4 74 452763.0 +5 6 5397.0 +6 5 6398.0 +7 6 7735.0 +8 8 8762.0 +9 7 91047.0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c deleted file mode 100644 index 3210a3048ff48..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 -1 71 116414.0 -2 69 225571.0 -3 62 332004.0 -4 74 452763.0 -5 6 5397.0 -6 5 6398.0 -7 6 7735.0 -8 8 8762.0 -9 7 91047.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f index f55b5c9eef39f..8b1acc12b635c 100644 --- a/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 7 8 -9 \ No newline at end of file +9 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f index 349d8b75d942b..3975bfc1af512 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -130091 \ No newline at end of file +130091 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f index 349d8b75d942b..3975bfc1af512 100644 --- a/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -1 +1 @@ -130091 \ No newline at end of file +130091 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f index f55b5c9eef39f..8b1acc12b635c 100644 --- a/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 7 8 -9 \ No newline at end of file +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f b/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f new file mode 100644 index 0000000000000..8b1acc12b635c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-6-2d1fb04c7060fafe470e14061b2a5b6f @@ -0,0 +1,10 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f55b5c9eef39f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 -1 -2 -3 -4 -5 -6 -7 -8 -9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 index 339756be98e73..4953d093489ee 100644 --- a/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 495.0 496 496.0 497 497.0 -498 1494.0 \ No newline at end of file +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-10-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-11-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-10-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 339756be98e73..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 -17 17.0 -18 36.0 -19 19.0 -20 20.0 -24 48.0 -26 52.0 -27 27.0 -28 28.0 -30 30.0 -33 33.0 -34 34.0 -35 105.0 -37 74.0 -41 41.0 -42 84.0 -43 43.0 -44 44.0 -47 47.0 -51 102.0 -53 53.0 -54 54.0 -57 57.0 -58 116.0 -64 64.0 -65 65.0 -66 66.0 -67 134.0 -69 69.0 -70 210.0 -72 144.0 -74 74.0 -76 152.0 -77 77.0 -78 78.0 -80 80.0 -82 82.0 -83 166.0 -84 168.0 -85 85.0 -86 86.0 -87 87.0 -90 270.0 -92 92.0 -95 190.0 -96 96.0 -97 194.0 -98 196.0 -100 200.0 -103 206.0 -104 208.0 -105 105.0 -111 111.0 -113 226.0 -114 114.0 -116 116.0 -118 236.0 -119 357.0 -120 240.0 -125 250.0 -126 126.0 -128 384.0 -129 258.0 -131 131.0 -133 133.0 -134 268.0 -136 136.0 -137 274.0 -138 552.0 -143 143.0 -145 145.0 -146 292.0 -149 298.0 -150 150.0 -152 304.0 -153 153.0 -155 155.0 -156 156.0 -157 157.0 -158 158.0 -160 160.0 -162 162.0 -163 163.0 -164 328.0 -165 330.0 -166 166.0 -167 501.0 -168 168.0 -169 676.0 -170 170.0 -172 344.0 -174 348.0 -175 350.0 -176 352.0 -177 177.0 -178 178.0 -179 358.0 -180 180.0 -181 181.0 -183 183.0 -186 186.0 -187 561.0 -189 189.0 -190 190.0 -191 382.0 -192 192.0 -193 579.0 -194 194.0 -195 390.0 -196 196.0 -197 394.0 -199 597.0 -200 400.0 -201 201.0 -202 202.0 -203 406.0 -205 410.0 -207 414.0 -208 624.0 -209 418.0 -213 426.0 -214 214.0 -216 432.0 -217 434.0 -218 218.0 -219 438.0 -221 442.0 -222 222.0 -223 446.0 -224 448.0 -226 226.0 -228 228.0 -229 458.0 -230 1150.0 -233 466.0 -235 235.0 -237 474.0 -238 476.0 -239 478.0 -241 241.0 -242 484.0 -244 244.0 -247 247.0 -248 248.0 -249 249.0 -252 252.0 -255 510.0 -256 512.0 -257 257.0 -258 258.0 -260 260.0 -262 262.0 -263 263.0 -265 530.0 -266 266.0 -272 544.0 -273 819.0 -274 274.0 -275 275.0 -277 1108.0 -278 556.0 -280 560.0 -281 562.0 -282 564.0 -283 283.0 -284 284.0 -285 285.0 -286 286.0 -287 287.0 -288 576.0 -289 289.0 -291 291.0 -292 292.0 -296 296.0 -298 894.0 -302 302.0 -305 305.0 -306 306.0 -307 614.0 -308 308.0 -309 618.0 -310 310.0 -311 933.0 -315 315.0 -316 948.0 -317 634.0 -318 954.0 -321 642.0 -322 644.0 -323 323.0 -325 650.0 -327 981.0 -331 662.0 -332 332.0 -333 666.0 -335 335.0 -336 336.0 -338 338.0 -339 339.0 -341 341.0 -342 684.0 -344 688.0 -345 345.0 -348 1740.0 -351 351.0 -353 706.0 -356 356.0 -360 360.0 -362 362.0 -364 364.0 -365 365.0 -366 366.0 -367 734.0 -368 368.0 -369 1107.0 -373 373.0 -374 374.0 -375 375.0 -377 377.0 -378 378.0 -379 379.0 -382 764.0 -384 1152.0 -386 386.0 -389 389.0 -392 392.0 -393 393.0 -394 394.0 -395 790.0 -396 1188.0 -397 794.0 -399 798.0 -400 400.0 -401 2005.0 -402 402.0 -403 1209.0 -404 808.0 -406 1624.0 -407 407.0 -409 1227.0 -411 411.0 -413 826.0 -414 828.0 -417 1251.0 -418 418.0 -419 419.0 -421 421.0 -424 848.0 -427 427.0 -429 858.0 -430 1290.0 -431 1293.0 -432 432.0 -435 435.0 -436 436.0 -437 437.0 -438 1314.0 -439 878.0 -443 443.0 -444 444.0 -446 446.0 -448 448.0 -449 449.0 -452 452.0 -453 453.0 -454 1362.0 -455 455.0 -457 457.0 -458 916.0 -459 918.0 -460 460.0 -462 924.0 -463 926.0 -466 1398.0 -467 467.0 -468 1872.0 -469 2345.0 -470 470.0 -472 472.0 -475 475.0 -477 477.0 -478 956.0 -479 479.0 -480 1440.0 -481 481.0 -482 482.0 -483 483.0 -484 484.0 -485 485.0 -487 487.0 -489 1956.0 -490 490.0 -491 491.0 -492 984.0 -493 493.0 -494 494.0 -495 495.0 -496 496.0 -497 497.0 -498 1494.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..4953d093489ee --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-11-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 0.0 +2 2.0 +4 4.0 +5 15.0 +8 8.0 +9 9.0 +10 10.0 +11 11.0 +12 24.0 +15 30.0 +17 17.0 +18 36.0 +19 19.0 +20 20.0 +24 48.0 +26 52.0 +27 27.0 +28 28.0 +30 30.0 +33 33.0 +34 34.0 +35 105.0 +37 74.0 +41 41.0 +42 84.0 +43 43.0 +44 44.0 +47 47.0 +51 102.0 +53 53.0 +54 54.0 +57 57.0 +58 116.0 +64 64.0 +65 65.0 +66 66.0 +67 134.0 +69 69.0 +70 210.0 +72 144.0 +74 74.0 +76 152.0 +77 77.0 +78 78.0 +80 80.0 +82 82.0 +83 166.0 +84 168.0 +85 85.0 +86 86.0 +87 87.0 +90 270.0 +92 92.0 +95 190.0 +96 96.0 +97 194.0 +98 196.0 +100 200.0 +103 206.0 +104 208.0 +105 105.0 +111 111.0 +113 226.0 +114 114.0 +116 116.0 +118 236.0 +119 357.0 +120 240.0 +125 250.0 +126 126.0 +128 384.0 +129 258.0 +131 131.0 +133 133.0 +134 268.0 +136 136.0 +137 274.0 +138 552.0 +143 143.0 +145 145.0 +146 292.0 +149 298.0 +150 150.0 +152 304.0 +153 153.0 +155 155.0 +156 156.0 +157 157.0 +158 158.0 +160 160.0 +162 162.0 +163 163.0 +164 328.0 +165 330.0 +166 166.0 +167 501.0 +168 168.0 +169 676.0 +170 170.0 +172 344.0 +174 348.0 +175 350.0 +176 352.0 +177 177.0 +178 178.0 +179 358.0 +180 180.0 +181 181.0 +183 183.0 +186 186.0 +187 561.0 +189 189.0 +190 190.0 +191 382.0 +192 192.0 +193 579.0 +194 194.0 +195 390.0 +196 196.0 +197 394.0 +199 597.0 +200 400.0 +201 201.0 +202 202.0 +203 406.0 +205 410.0 +207 414.0 +208 624.0 +209 418.0 +213 426.0 +214 214.0 +216 432.0 +217 434.0 +218 218.0 +219 438.0 +221 442.0 +222 222.0 +223 446.0 +224 448.0 +226 226.0 +228 228.0 +229 458.0 +230 1150.0 +233 466.0 +235 235.0 +237 474.0 +238 476.0 +239 478.0 +241 241.0 +242 484.0 +244 244.0 +247 247.0 +248 248.0 +249 249.0 +252 252.0 +255 510.0 +256 512.0 +257 257.0 +258 258.0 +260 260.0 +262 262.0 +263 263.0 +265 530.0 +266 266.0 +272 544.0 +273 819.0 +274 274.0 +275 275.0 +277 1108.0 +278 556.0 +280 560.0 +281 562.0 +282 564.0 +283 283.0 +284 284.0 +285 285.0 +286 286.0 +287 287.0 +288 576.0 +289 289.0 +291 291.0 +292 292.0 +296 296.0 +298 894.0 +302 302.0 +305 305.0 +306 306.0 +307 614.0 +308 308.0 +309 618.0 +310 310.0 +311 933.0 +315 315.0 +316 948.0 +317 634.0 +318 954.0 +321 642.0 +322 644.0 +323 323.0 +325 650.0 +327 981.0 +331 662.0 +332 332.0 +333 666.0 +335 335.0 +336 336.0 +338 338.0 +339 339.0 +341 341.0 +342 684.0 +344 688.0 +345 345.0 +348 1740.0 +351 351.0 +353 706.0 +356 356.0 +360 360.0 +362 362.0 +364 364.0 +365 365.0 +366 366.0 +367 734.0 +368 368.0 +369 1107.0 +373 373.0 +374 374.0 +375 375.0 +377 377.0 +378 378.0 +379 379.0 +382 764.0 +384 1152.0 +386 386.0 +389 389.0 +392 392.0 +393 393.0 +394 394.0 +395 790.0 +396 1188.0 +397 794.0 +399 798.0 +400 400.0 +401 2005.0 +402 402.0 +403 1209.0 +404 808.0 +406 1624.0 +407 407.0 +409 1227.0 +411 411.0 +413 826.0 +414 828.0 +417 1251.0 +418 418.0 +419 419.0 +421 421.0 +424 848.0 +427 427.0 +429 858.0 +430 1290.0 +431 1293.0 +432 432.0 +435 435.0 +436 436.0 +437 437.0 +438 1314.0 +439 878.0 +443 443.0 +444 444.0 +446 446.0 +448 448.0 +449 449.0 +452 452.0 +453 453.0 +454 1362.0 +455 455.0 +457 457.0 +458 916.0 +459 918.0 +460 460.0 +462 924.0 +463 926.0 +466 1398.0 +467 467.0 +468 1872.0 +469 2345.0 +470 470.0 +472 472.0 +475 475.0 +477 477.0 +478 956.0 +479 479.0 +480 1440.0 +481 481.0 +482 482.0 +483 483.0 +484 484.0 +485 485.0 +487 487.0 +489 1956.0 +490 490.0 +491 491.0 +492 984.0 +493 493.0 +494 494.0 +495 495.0 +496 496.0 +497 497.0 +498 1494.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 index dfca4e169cbe8..951e74db0fe23 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 @@ -7,4 +7,4 @@ 10 10.0 11 11.0 12 24.0 -15 30.0 \ No newline at end of file +15 30.0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 rename to sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-ab2390a3b0ba76907a7bee390b2924a1 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 index dfca4e169cbe8..951e74db0fe23 100644 --- a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 @@ -7,4 +7,4 @@ 10 10.0 11 11.0 12 24.0 -15 30.0 \ No newline at end of file +15 30.0 diff --git a/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map-7-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map-8-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 b/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-7-c1a8cf4ef8060a4703b0affe40496169 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 deleted file mode 100644 index 326493a1d8cb6..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 +++ /dev/null @@ -1,309 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 -34 1 -35 1 -37 1 -41 1 -42 1 -43 1 -44 1 -47 1 -51 1 -53 1 -54 1 -57 1 -58 1 -64 1 -65 1 -66 1 -67 1 -69 1 -70 1 -72 1 -74 1 -76 1 -77 1 -78 1 -80 1 -82 1 -83 1 -84 1 -85 1 -86 1 -87 1 -90 1 -92 1 -95 1 -96 1 -97 1 -98 1 -100 1 -103 1 -104 1 -105 1 -111 1 -113 1 -114 1 -116 1 -118 1 -119 1 -120 1 -125 1 -126 1 -128 1 -129 1 -131 1 -133 1 -134 1 -136 1 -137 1 -138 1 -143 1 -145 1 -146 1 -149 1 -150 1 -152 1 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 1 -165 1 -166 1 -167 1 -168 1 -169 1 -170 1 -172 1 -174 1 -175 1 -176 1 -177 1 -178 1 -179 1 -180 1 -181 1 -183 1 -186 1 -187 1 -189 1 -190 1 -191 1 -192 1 -193 1 -194 1 -195 1 -196 1 -197 1 -199 1 -200 1 -201 1 -202 1 -203 1 -205 1 -207 1 -208 1 -209 1 -213 1 -214 1 -216 1 -217 1 -218 1 -219 1 -221 1 -222 1 -223 1 -224 1 -226 1 -228 1 -229 1 -230 1 -233 1 -235 1 -237 1 -238 1 -239 1 -241 1 -242 1 -244 1 -247 1 -248 1 -249 1 -252 1 -255 1 -256 1 -257 1 -258 1 -260 1 -262 1 -263 1 -265 1 -266 1 -272 1 -273 1 -274 1 -275 1 -277 1 -278 1 -280 1 -281 1 -282 1 -283 1 -284 1 -285 1 -286 1 -287 1 -288 1 -289 1 -291 1 -292 1 -296 1 -298 1 -302 1 -305 1 -306 1 -307 1 -308 1 -309 1 -310 1 -311 1 -315 1 -316 1 -317 1 -318 1 -321 1 -322 1 -323 1 -325 1 -327 1 -331 1 -332 1 -333 1 -335 1 -336 1 -338 1 -339 1 -341 1 -342 1 -344 1 -345 1 -348 1 -351 1 -353 1 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 1 -368 1 -369 1 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 1 -384 1 -386 1 -389 1 -392 1 -393 1 -394 1 -395 1 -396 1 -397 1 -399 1 -400 1 -401 1 -402 1 -403 1 -404 1 -406 1 -407 1 -409 1 -411 1 -413 1 -414 1 -417 1 -418 1 -419 1 -421 1 -424 1 -427 1 -429 1 -430 1 -431 1 -432 1 -435 1 -436 1 -437 1 -438 1 -439 1 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 1 -455 1 -457 1 -458 1 -459 1 -460 1 -462 1 -463 1 -466 1 -467 1 -468 1 -469 1 -470 1 -472 1 -475 1 -477 1 -478 1 -479 1 -480 1 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 1 -490 1 -491 1 -492 1 -493 1 -494 1 -495 1 -496 1 -497 1 -498 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 b/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-8-6c26c5d39c4bdb61728defa9b44bfb52 @@ -0,0 +1,309 @@ +0 1 +2 1 +4 1 +5 1 +8 1 +9 1 +10 1 +11 1 +12 1 +15 1 +17 1 +18 1 +19 1 +20 1 +24 1 +26 1 +27 1 +28 1 +30 1 +33 1 +34 1 +35 1 +37 1 +41 1 +42 1 +43 1 +44 1 +47 1 +51 1 +53 1 +54 1 +57 1 +58 1 +64 1 +65 1 +66 1 +67 1 +69 1 +70 1 +72 1 +74 1 +76 1 +77 1 +78 1 +80 1 +82 1 +83 1 +84 1 +85 1 +86 1 +87 1 +90 1 +92 1 +95 1 +96 1 +97 1 +98 1 +100 1 +103 1 +104 1 +105 1 +111 1 +113 1 +114 1 +116 1 +118 1 +119 1 +120 1 +125 1 +126 1 +128 1 +129 1 +131 1 +133 1 +134 1 +136 1 +137 1 +138 1 +143 1 +145 1 +146 1 +149 1 +150 1 +152 1 +153 1 +155 1 +156 1 +157 1 +158 1 +160 1 +162 1 +163 1 +164 1 +165 1 +166 1 +167 1 +168 1 +169 1 +170 1 +172 1 +174 1 +175 1 +176 1 +177 1 +178 1 +179 1 +180 1 +181 1 +183 1 +186 1 +187 1 +189 1 +190 1 +191 1 +192 1 +193 1 +194 1 +195 1 +196 1 +197 1 +199 1 +200 1 +201 1 +202 1 +203 1 +205 1 +207 1 +208 1 +209 1 +213 1 +214 1 +216 1 +217 1 +218 1 +219 1 +221 1 +222 1 +223 1 +224 1 +226 1 +228 1 +229 1 +230 1 +233 1 +235 1 +237 1 +238 1 +239 1 +241 1 +242 1 +244 1 +247 1 +248 1 +249 1 +252 1 +255 1 +256 1 +257 1 +258 1 +260 1 +262 1 +263 1 +265 1 +266 1 +272 1 +273 1 +274 1 +275 1 +277 1 +278 1 +280 1 +281 1 +282 1 +283 1 +284 1 +285 1 +286 1 +287 1 +288 1 +289 1 +291 1 +292 1 +296 1 +298 1 +302 1 +305 1 +306 1 +307 1 +308 1 +309 1 +310 1 +311 1 +315 1 +316 1 +317 1 +318 1 +321 1 +322 1 +323 1 +325 1 +327 1 +331 1 +332 1 +333 1 +335 1 +336 1 +338 1 +339 1 +341 1 +342 1 +344 1 +345 1 +348 1 +351 1 +353 1 +356 1 +360 1 +362 1 +364 1 +365 1 +366 1 +367 1 +368 1 +369 1 +373 1 +374 1 +375 1 +377 1 +378 1 +379 1 +382 1 +384 1 +386 1 +389 1 +392 1 +393 1 +394 1 +395 1 +396 1 +397 1 +399 1 +400 1 +401 1 +402 1 +403 1 +404 1 +406 1 +407 1 +409 1 +411 1 +413 1 +414 1 +417 1 +418 1 +419 1 +421 1 +424 1 +427 1 +429 1 +430 1 +431 1 +432 1 +435 1 +436 1 +437 1 +438 1 +439 1 +443 1 +444 1 +446 1 +448 1 +449 1 +452 1 +453 1 +454 1 +455 1 +457 1 +458 1 +459 1 +460 1 +462 1 +463 1 +466 1 +467 1 +468 1 +469 1 +470 1 +472 1 +475 1 +477 1 +478 1 +479 1 +480 1 +481 1 +482 1 +483 1 +484 1 +485 1 +487 1 +489 1 +490 1 +491 1 +492 1 +493 1 +494 1 +495 1 +496 1 +497 1 +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e +++ b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 index 0e40f3f1a49a9..c156bd5f5d8c6 100644 --- a/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 val_495 1 496 val_496 1 497 val_497 1 -498 val_498 1 \ No newline at end of file +498 val_498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 index df07a9da29f01..93e965c771403 100644 --- a/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 index cf0c065dc6532..742479ac713c5 100644 --- a/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 val_495 1 496 val_496 1 497 val_497 1 -498 val_498 3 \ No newline at end of file +498 val_498 3 diff --git a/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 index 4ef88d57aa72b..deaf15e462910 100644 --- a/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 95 val_95 1 96 val_96 1 97 val_97 1 -98 val_98 1 \ No newline at end of file +98 val_98 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 index 0e40f3f1a49a9..c156bd5f5d8c6 100644 --- a/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 495 val_495 1 496 val_496 1 497 val_497 1 -498 val_498 1 \ No newline at end of file +498 val_498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 index 326493a1d8cb6..8bfbbf1e69f57 100644 --- a/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -306,4 +306,4 @@ 495 1 496 1 497 1 -498 1 \ No newline at end of file +498 1 diff --git a/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 index 4ef88d57aa72b..deaf15e462910 100644 --- a/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 @@ -306,4 +306,4 @@ 95 val_95 1 96 val_96 1 97 val_97 1 -98 val_98 1 \ No newline at end of file +98 val_98 1 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index ae9bbc3e2c2f7..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 -1 71 132828.0 -2 69 251142.0 -3 62 364008.0 -4 74 4105526.0 -5 6 5794.0 -6 5 6796.0 -7 6 71470.0 -8 8 81524.0 -9 7 92094.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b new file mode 100644 index 0000000000000..1b9d97300aa08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-c83b156356c269b6aef263640a4f6b7b @@ -0,0 +1,10 @@ +0 1 00.0 +1 71 132828.0 +2 69 251142.0 +3 62 364008.0 +4 74 4105526.0 +5 6 5794.0 +6 5 6796.0 +7 6 71470.0 +8 8 81524.0 +9 7 92094.0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 64bb7c62c1885..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 1 -1 71 132828.0 10044 71 -2 69 251142.0 15780 69 -3 62 364008.0 20119 62 -4 74 4105526.0 30965 74 -5 6 5794.0 278 6 -6 5 6796.0 331 5 -7 6 71470.0 447 6 -8 8 81524.0 595 8 -9 7 92094.0 577 7 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 index 6b6a788e382db..0b7e79a79bd11 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 @@ -3,4 +3,4 @@ 4 1 5 1 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 index c33eed60c8d5b..d63ed5dbe78bb 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 @@ -3,4 +3,4 @@ 8 1 10 1 16 1 -18 1 \ No newline at end of file +18 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 index 6b6a788e382db..0b7e79a79bd11 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 @@ -3,4 +3,4 @@ 4 1 5 1 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 index c33eed60c8d5b..d63ed5dbe78bb 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 @@ -3,4 +3,4 @@ 8 1 10 1 16 1 -18 1 \ No newline at end of file +18 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e deleted file mode 100644 index 106132fc993d3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e +++ /dev/null @@ -1,5 +0,0 @@ -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 deleted file mode 100644 index 65235356ea425..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 +++ /dev/null @@ -1,5 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd deleted file mode 100644 index 922f4bfc83e44..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 deleted file mode 100644 index 017878bc9bee5..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 3 -1 4 1878.0 878 6 -1 5 1729.0 729 8 -1 6 11282.0 1282 12 -1 6 11494.0 1494 11 -1 7 11171.0 1171 11 -1 7 11516.0 1516 10 -1 8 11263.0 1263 10 -1 9 12294.0 2294 14 -1 9 12654.0 2654 16 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae deleted file mode 100644 index f21a658e3c68f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae +++ /dev/null @@ -1,32 +0,0 @@ -5 1 5102.0 102 2 -5 1 5116.0 116 2 -5 1 515.0 15 3 -5 1 553.0 53 1 -5 1 554.0 54 1 -5 1 557.0 57 1 -6 1 6134.0 134 2 -6 1 664.0 64 1 -6 1 665.0 65 1 -6 1 666.0 66 1 -6 1 669.0 69 1 -7 1 7144.0 144 2 -7 1 7152.0 152 2 -7 1 7210.0 210 3 -7 1 774.0 74 1 -7 1 777.0 77 1 -7 1 778.0 78 1 -8 1 8166.0 166 2 -8 1 8168.0 168 2 -8 1 88.0 8 1 -8 1 880.0 80 1 -8 1 882.0 82 1 -8 1 885.0 85 1 -8 1 886.0 86 1 -8 1 887.0 87 1 -9 1 9190.0 190 2 -9 1 9194.0 194 2 -9 1 9196.0 196 2 -9 1 9270.0 270 3 -9 1 99.0 9 1 -9 1 992.0 92 1 -9 1 996.0 96 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e deleted file mode 100644 index 106132fc993d3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e +++ /dev/null @@ -1,5 +0,0 @@ -5 6 5397.0 278 10 -6 5 6398.0 331 6 -7 6 7735.0 447 10 -8 8 8762.0 595 10 -9 7 91047.0 577 12 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 deleted file mode 100644 index 65235356ea425..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 +++ /dev/null @@ -1,5 +0,0 @@ -0 1 00.0 0 3 -1 71 116414.0 10044 115 -2 69 225571.0 15780 111 -3 62 332004.0 20119 99 -4 74 452763.0 30965 124 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 index dbc7aebdbbae5..dcb604016a969 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 @@ -2,4 +2,4 @@ 6 5 7 6 8 8 -9 7 \ No newline at end of file +9 7 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 index ebf9d6978dec6..cecfbbd281537 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 @@ -2,4 +2,4 @@ 1 71 115 2 69 111 3 62 99 -4 74 124 \ No newline at end of file +4 74 124 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 index a5ae9e2a62227..83d4ac2489823 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ 100 2 -200 2 \ No newline at end of file +200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 index 16c7a647a2344..ab65c1a003b5b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -400 1 \ No newline at end of file +400 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 index 67f077e51a647..11c303a71007c 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ val_100 2 -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 index b06ad20135fbe..1e4c20551b68b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 index a5ae9e2a62227..83d4ac2489823 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ 100 2 -200 2 \ No newline at end of file +200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 index 16c7a647a2344..ab65c1a003b5b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -400 1 \ No newline at end of file +400 1 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 index 67f077e51a647..11c303a71007c 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 @@ -1,2 +1,2 @@ val_100 2 -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 index b06ad20135fbe..1e4c20551b68b 100644 --- a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 @@ -1 +1 @@ -val_200 2 \ No newline at end of file +val_200 2 diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb index efe5cc7795c65..26dbfc34ad4fb 100644 --- a/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb +++ b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb @@ -1 +1 @@ --30.33 \ No newline at end of file +-30.33 diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 index efe5cc7795c65..26dbfc34ad4fb 100644 --- a/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 +++ b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 @@ -1 +1 @@ --30.33 \ No newline at end of file +-30.33 diff --git a/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 b/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 deleted file mode 100644 index f2a91fe3bfab0..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 +++ /dev/null @@ -1,13 +0,0 @@ -0 val_0 1 -2 val_2 1 -4 val_4 1 -5 val_5 1 -8 val_8 1 -9 val_9 1 -10 val_10 1 -11 val_11 1 -12 val_12 1 -15 val_15 1 -17 val_17 1 -18 val_18 1 -19 val_19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f b/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f deleted file mode 100644 index c8e666cb01e8e..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f +++ /dev/null @@ -1,14 +0,0 @@ -0 3 -5 3 -12 2 -15 2 -18 2 -2 1 -4 1 -8 1 -9 1 -10 1 -11 1 -17 1 -19 1 -20 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb b/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb deleted file mode 100644 index 96a824a81c589..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb +++ /dev/null @@ -1,3 +0,0 @@ -19 val_19 19 val_19 -18 val_18 18 val_18 -17 val_17 17 val_17 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d deleted file mode 100644 index a79396dac079a..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d +++ /dev/null @@ -1,13 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 deleted file mode 100644 index f2a91fe3bfab0..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 +++ /dev/null @@ -1,13 +0,0 @@ -0 val_0 1 -2 val_2 1 -4 val_4 1 -5 val_5 1 -8 val_8 1 -9 val_9 1 -10 val_10 1 -11 val_11 1 -12 val_12 1 -15 val_15 1 -17 val_17 1 -18 val_18 1 -19 val_19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d deleted file mode 100644 index a79396dac079a..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d +++ /dev/null @@ -1,13 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f index ae9bbc3e2c2f7..1b9d97300aa08 100644 --- a/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ 6 5 6796.0 7 6 71470.0 8 8 81524.0 -9 7 92094.0 \ No newline at end of file +9 7 92094.0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 64bb7c62c1885..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -0 1 00.0 0 1 -1 71 132828.0 10044 71 -2 69 251142.0 15780 69 -3 62 364008.0 20119 62 -4 74 4105526.0 30965 74 -5 6 5794.0 278 6 -6 5 6796.0 331 5 -7 6 71470.0 447 6 -8 8 81524.0 595 8 -9 7 92094.0 577 7 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index c1cc4ee204773..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 1 -1 2 1 -1 3 1 -1 7 1 -1 8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index f0192040e147b..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 2 1 -2 3 1 -3 4 1 -7 8 1 -8 9 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index b6c2eb98e5e49..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -2 1 -4 1 -6 1 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 8e7ee8a2b47bb..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -1 1 -2 1 -2 1 -3 1 -3 1 -7 1 -7 1 -8 2 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 0b2b54cd94e4f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -2 1 -2 1 -3 1 -4 1 -6 1 -7 1 -8 2 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 3d4708b7c9d64..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 2 -2 2 -3 2 -7 2 -8 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 deleted file mode 100644 index 7ca6b0b28a960..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 2 1 -2 1 12 2 1 -3 1 13 2 1 -7 1 17 2 1 -8 1 18 2 1 -8 1 28 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 58e16ef3c0ef3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 2 11 1 -2 2 12 1 -3 2 13 1 -7 2 17 1 -8 2 18 1 -8 2 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index d15db8c5d079f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1 +0,0 @@ -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 deleted file mode 100644 index 3d1609d961673..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 +++ /dev/null @@ -1,2 +0,0 @@ -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 index c6bb9dbfd6497..bfca78293c988 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 @@ -1,2 +1,2 @@ 0 -11 \ No newline at end of file +11 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 index c6bb9dbfd6497..bfca78293c988 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 @@ -1,2 +1,2 @@ 0 -11 \ No newline at end of file +11 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 b/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 deleted file mode 100644 index ded2854cdf564..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 +++ /dev/null @@ -1,6 +0,0 @@ -1 3 3 0.0 -1 1 1 2.0 -1 1 1 4.0 -1 3 3 5.0 -1 1 1 8.0 -1 1 1 9.0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 b/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 deleted file mode 100644 index 487b4c4a5cc6f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 +++ /dev/null @@ -1,6 +0,0 @@ -0 1 3 3 0.0 -2 1 1 1 2.0 -4 1 1 1 4.0 -5 1 3 3 5.0 -8 1 1 1 8.0 -9 1 1 1 9.0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 b/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 b/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 deleted file mode 100644 index 6a5fe2835fc56..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 +++ /dev/null @@ -1 +0,0 @@ -6 10 10 28.0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 b/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 index c5b99ed941efc..e6a233467dcf5 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 @@ -3,4 +3,4 @@ 13 1 17 1 18 1 -28 1 \ No newline at end of file +28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 b/sql/hive/src/test/resources/golden/groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 rename to sql/hive/src/test/resources/golden/groupby_sort_2-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce b/sql/hive/src/test/resources/golden/groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce rename to sql/hive/src/test/resources/golden/groupby_sort_3-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 b/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c b/sql/hive/src/test/resources/golden/groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c rename to sql/hive/src/test/resources/golden/groupby_sort_4-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 b/sql/hive/src/test/resources/golden/groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 rename to sql/hive/src/test/resources/golden/groupby_sort_5-13-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb b/sql/hive/src/test/resources/golden/groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb rename to sql/hive/src/test/resources/golden/groupby_sort_5-20-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 b/sql/hive/src/test/resources/golden/groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 rename to sql/hive/src/test/resources/golden/groupby_sort_5-5-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd index 4e31460a412ba..e7273779ac1b8 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd @@ -2,4 +2,4 @@ 2 1 3 1 7 1 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 rename to sql/hive/src/test/resources/golden/groupby_sort_6-9-591e03d1cfc10821a601498df1ed6675 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 index 10f4a1f5ff34c..0f333f42821a0 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 @@ -3,4 +3,4 @@ 3 13 1 7 17 1 8 18 1 -8 28 1 \ No newline at end of file +8 28 1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 rename to sql/hive/src/test/resources/golden/groupby_sort_7-5-43e94a517107a5bcf6fee78e6c88a1cc diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d index 7813681f5b41c..7ed6ff82de6bc 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d @@ -1 +1 @@ -5 \ No newline at end of file +5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 b/sql/hive/src/test/resources/golden/groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc similarity index 100% rename from sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 rename to sql/hive/src/test/resources/golden/groupby_sort_8-5-43e94a517107a5bcf6fee78e6c88a1cc diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d index 7813681f5b41c..7ed6ff82de6bc 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d @@ -1 +1 @@ -5 \ No newline at end of file +5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 rename to sql/hive/src/test/resources/golden/groupby_sort_9-5-43e94a517107a5bcf6fee78e6c88a1cc diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c b/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c index 3d4708b7c9d64..612dcbb640d46 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c @@ -2,4 +2,4 @@ 2 2 3 2 7 2 -8 4 \ No newline at end of file +8 4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index c1cc4ee204773..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 1 -1 2 1 -1 3 1 -1 7 1 -1 8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 deleted file mode 100644 index f0192040e147b..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 +++ /dev/null @@ -1,5 +0,0 @@ -1 2 1 -2 3 1 -3 4 1 -7 8 1 -8 9 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index b6c2eb98e5e49..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -2 1 -4 1 -6 1 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 8e7ee8a2b47bb..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -1 1 -2 1 -2 1 -3 1 -3 1 -7 1 -7 1 -8 2 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 0b2b54cd94e4f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -2 1 -2 1 -3 1 -4 1 -6 1 -7 1 -8 2 -14 1 -16 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 3d4708b7c9d64..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 2 -2 2 -3 2 -7 2 -8 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 deleted file mode 100644 index 7ca6b0b28a960..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 2 1 -2 1 12 2 1 -3 1 13 2 1 -7 1 17 2 1 -8 1 18 2 1 -8 1 28 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 97a3b8c2f5977..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 1 11 1 -2 1 12 1 -3 1 13 1 -7 1 17 1 -8 1 18 1 -8 1 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca deleted file mode 100644 index 58e16ef3c0ef3..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca +++ /dev/null @@ -1,6 +0,0 @@ -1 2 11 1 -2 2 12 1 -3 2 13 1 -7 2 17 1 -8 2 18 1 -8 2 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index 4e31460a412ba..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1,5 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd deleted file mode 100644 index 10f4a1f5ff34c..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd +++ /dev/null @@ -1,6 +0,0 @@ -1 11 1 -2 12 1 -3 13 1 -7 17 1 -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 deleted file mode 100644 index d15db8c5d079f..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 +++ /dev/null @@ -1 +0,0 @@ -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 deleted file mode 100644 index 3d1609d961673..0000000000000 --- a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 +++ /dev/null @@ -1,2 +0,0 @@ -8 18 1 -8 28 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa b/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa rename to sql/hive/src/test/resources/golden/groupby_sort_test_1-6-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f index a99cee758fe57..3cb614bdd84e8 100644 --- a/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f +++ b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f @@ -1 +1 @@ -3556498 \ No newline at end of file +3556498 diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf b/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 b/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 b/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 deleted file mode 100644 index d7c6f236687d8..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_2__, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_2__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344545}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 b/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 b/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 deleted file mode 100644 index 4c6ec0ba34bb8..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:src_idx_src_index_3, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/src_idx_src_index_3, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344545}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 b/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 b/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 deleted file mode 100644 index 0c6af94247b85..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_4__, dbName:default, owner:null, createTime:1389344545, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_4__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 b/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d b/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d deleted file mode 100644 index e4a5816e6f1cc..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_5__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_5__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{escape.delim=\, serialization.format= , field.delim= \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad b/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 b/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 deleted file mode 100644 index 3d8751ca47049..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 +++ /dev/null @@ -1,5 +0,0 @@ -key int from deserializer -_bucketname string from deserializer -_offsets array from deserializer - -Detailed Table Information Table(tableName:default__src_src_index_6__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_6__, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f b/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 b/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 deleted file mode 100644 index 0e4852b319052..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 +++ /dev/null @@ -1,5 +0,0 @@ -key int from deserializer -_bucketname string from deserializer -_offsets array from deserializer - -Detailed Table Information Table(tableName:src_idx_src_index_7, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/src_idx_src_index_7, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e b/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 deleted file mode 100644 index 41a5492fc5331..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_8__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_8__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 b/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf b/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf deleted file mode 100644 index 8212bf8b7d2fb..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf +++ /dev/null @@ -1,5 +0,0 @@ -key int None -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__src_src_index_9__, dbName:default, owner:null, createTime:1389344546, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4441354405523276795/default__src_src_index_9__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:key, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{prop2=val2, prop1=val1, transient_lastDdlTime=1389344546}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca b/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 b/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 b/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe b/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f b/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d deleted file mode 100644 index e8310385c56dc..0000000000000 --- a/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d +++ /dev/null @@ -1 +0,0 @@ -src \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c b/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 b/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/index_serde-0-db64b724719d27c7f0db4f51f5c4edaa similarity index 100% rename from sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/index_serde-0-db64b724719d27c7f0db4f51f5c4edaa diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 b/sql/hive/src/test/resources/golden/index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 rename to sql/hive/src/test/resources/golden/index_serde-1-6560d12b69d55e5297a145ebc4bb0cb3 diff --git a/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 b/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 deleted file mode 100644 index 48522980f81a8..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 +++ /dev/null @@ -1,3 +0,0 @@ -number int from deserializer -first_name string from deserializer -last_name string from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b b/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b deleted file mode 100644 index 4ed570f9070eb..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b +++ /dev/null @@ -1,5 +0,0 @@ -7 Sylvester McCoy -8 Paul McGann -9 Christopher Eccleston -10 David Tennant -11 Matt Smith \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/index_serde-10-c85e061ea9c5b90ca69b7450faad14b6 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 rename to sql/hive/src/test/resources/golden/index_serde-10-c85e061ea9c5b90ca69b7450faad14b6 diff --git a/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b b/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b new file mode 100644 index 0000000000000..63d56733b58b0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-11-123301a057d4a46072d0431e00e20c4b @@ -0,0 +1,5 @@ +7 Sylvester McCoy +8 Paul McGann +9 Christopher Eccleston +10 David Tennant +11 Matt Smith diff --git a/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 b/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad b/sql/hive/src/test/resources/golden/index_serde-12-309e916d683a1a12ab62565697cb0046 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad rename to sql/hive/src/test/resources/golden/index_serde-12-309e916d683a1a12ab62565697cb0046 diff --git a/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 b/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 b/sql/hive/src/test/resources/golden/index_serde-13-d590fd7cb9d433143de490d75686dd4 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 rename to sql/hive/src/test/resources/golden/index_serde-13-d590fd7cb9d433143de490d75686dd4 diff --git a/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 b/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 new file mode 100644 index 0000000000000..e716294e919d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-2-f92d6c66d21791c11d2a822df04c1b63 @@ -0,0 +1,3 @@ +number int from deserializer +first_name string from deserializer +last_name string from deserializer diff --git a/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c b/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 b/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 b/sql/hive/src/test/resources/golden/index_serde-3-ebab588c84a7a29f03b41dcd98132229 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 rename to sql/hive/src/test/resources/golden/index_serde-3-ebab588c84a7a29f03b41dcd98132229 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/index_serde-4-afcf2a156ccd4f79a0489b4593908d79 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a rename to sql/hive/src/test/resources/golden/index_serde-4-afcf2a156ccd4f79a0489b4593908d79 diff --git a/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 b/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 deleted file mode 100644 index d2ca633d0ae01..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 +++ /dev/null @@ -1,5 +0,0 @@ -number int from deserializer -_bucketname string -_offsets array - -Detailed Table Information Table(tableName:default__doctors_doctors_index__, dbName:default, owner:null, createTime:1389729651, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:number, type:int, comment:from deserializer), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/default__doctors_doctors_index__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:number, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389729651}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 b/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 new file mode 100644 index 0000000000000..c344129fb8f69 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-5-d7547751c37375a9238043bbe250e716 @@ -0,0 +1,5 @@ +number int from deserializer +_bucketname string +_offsets array + +Detailed Table Information Table(tableName:default__doctors_doctors_index__, dbName:default, owner:marmbrus, createTime:1414101838, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:number, type:int, comment:from deserializer), FieldSchema(name:_bucketname, type:string, comment:), FieldSchema(name:_offsets, type:array, comment:)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1438070583820061187/default__doctors_doctors_index__, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[Order(col:number, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1414101838}, viewOriginalText:null, viewExpandedText:null, tableType:INDEX_TABLE) diff --git a/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 b/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 b/sql/hive/src/test/resources/golden/index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 rename to sql/hive/src/test/resources/golden/index_serde-6-e6ff4b23b7f102e359afb4d53a1dedc3 diff --git a/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/index_serde-7-c9d7dcde469d3b9a66965a64dd15e4ae diff --git a/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/index_serde-8-3b03210f94ec40db9ab02620645014d1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/index_serde-8-3b03210f94ec40db9ab02620645014d1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/index_serde-9-35f48c7d6fa164bb84643657bc9280a8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/index_serde-9-35f48c7d6fa164bb84643657bc9280a8 diff --git a/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 b/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e b/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 b/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd b/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 index 3b7cf42f96358..821c3c8c89252 100644 --- a/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 +++ b/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 b/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 index 010e999c36749..e8a910f80f457 100644 --- a/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 +++ b/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 @@ -1,3 +1,3 @@ -a int None +a int -Detailed Table Information Table(tableName:test, dbName:default, owner:marmbrus, createTime:1389729862, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1389729862}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test, dbName:default, owner:marmbrus, createTime:1413881850, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:int, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1413881850}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 b/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 +++ b/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f b/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f +++ b/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 b/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 index e00bf4ec2c7e5..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 +++ b/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_DESCTABLE (TOK_TABTYPE TEST1)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Describe Table Operator: - Describe Table - table: TEST1 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 +++ b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 b/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 index 175d371fd09c9..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 +++ b/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_DESCTABLE (TOK_TABTYPE TEST10)) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Describe Table Operator: - Describe Table - table: TEST10 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 b/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 index 8dcdf43e31be3..4cb356c235573 100644 --- a/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 +++ b/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None -hr string None +key int +value string +ds string +hr string # Partition Information # col_name data_type comment -ds string None -hr string None \ No newline at end of file +ds string +hr string diff --git a/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f index 37dd922fd0787..46057aa0a8fca 100644 --- a/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 28 val_28 37 val_37 90 val_90 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 index fdf93911ee847..d8bd3b13b83f1 100644 --- a/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 +++ b/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 @@ -7,4 +7,4 @@ 66 val_66 82 val_82 86 val_86 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e +++ b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 +++ b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c b/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c index 4220cf5f30392..a66a07386eef8 100644 --- a/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c +++ b/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c @@ -308,4 +308,4 @@ 222 2008-04-08 12 403 2008-04-08 12 400 2008-04-08 12 -200 2008-04-08 12 \ No newline at end of file +200 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f index 37dd922fd0787..46057aa0a8fca 100644 --- a/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 28 val_28 37 val_37 90 val_90 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd b/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd index d6e0c29932b9b..9ee31317478d5 100644 --- a/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd +++ b/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd @@ -102,4 +102,4 @@ 152 val_152 194 val_194 126 val_126 -169 val_169 \ No newline at end of file +169 val_169 diff --git a/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f index 703a4eef24f3f..0190981db84ed 100644 --- a/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 97 val_97 97 val_97 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf b/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 b/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 4335dce6a9929..0000000000000 --- a/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,5 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -11 val_11 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 index ded361eb294f0..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 +++ b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 @@ -1,2 +1,2 @@ -key int None -value string None +key int +value string diff --git a/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 b/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 b/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 b/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 1c48b3680a3ac..0000000000000 --- a/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -NULL NULL --1461153966 {"myint":49,"mystring":"343","underscore_int":7} --1952710705 {"myint":25,"mystring":"125","underscore_int":5} --734328905 {"myint":16,"mystring":"64","underscore_int":4} --751827636 {"myint":4,"mystring":"8","underscore_int":2} -1244525196 {"myint":36,"mystring":"216","underscore_int":6} -1638581586 {"myint":64,"mystring":"512","underscore_int":8} -1712634731 {"myint":0,"mystring":"0","underscore_int":0} -336964422 {"myint":81,"mystring":"729","underscore_int":9} -465985201 {"myint":1,"mystring":"1","underscore_int":1} -477111225 {"myint":9,"mystring":"27","underscore_int":3} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc b/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a b/sql/hive/src/test/resources/golden/input19-1-f32df514de8156b5f5b435eea2c9be40 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a rename to sql/hive/src/test/resources/golden/input19-1-f32df514de8156b5f5b435eea2c9be40 diff --git a/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 b/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 index 795dbe3a976eb..db9438946fc84 100644 --- a/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 +++ b/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 @@ -1 +1 @@ -127.0.0.1 NULL frank 10/Oct/2000:13:55:36 -0700 GET /apache_pb.gif HTTP/1.0 200 2326 \ No newline at end of file +127.0.0.1 NULL frank 10/Oct/2000:13:55:36 -0700 GET /apache_pb.gif HTTP/1.0 200 2326 diff --git a/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 index fdf93911ee847..d8bd3b13b83f1 100644 --- a/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 +++ b/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 @@ -7,4 +7,4 @@ 66 val_66 82 val_82 86 val_86 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 b/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 index 8a8f1a1b8bbe1..f2f1112224cd7 100644 --- a/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 +++ b/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 @@ -2,4 +2,4 @@ 37 val_37 66 val_66 86 val_86 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 +++ b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd +++ b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b index ca0726f517eeb..77eaef91c9c3f 100644 --- a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b +++ b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b @@ -1,3 +1,3 @@ -a array None -b double None -c map None \ No newline at end of file +a array +b double +c map diff --git a/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c index 2c861553f9aa2..2dd749277aa48 100644 --- a/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c @@ -1,4 +1,4 @@ src srcpart test2a -test2b \ No newline at end of file +test2b diff --git a/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c index 3e40a0c866d4d..d2cb69524ba34 100644 --- a/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c @@ -1,3 +1,3 @@ src srcpart -test2b \ No newline at end of file +test2b diff --git a/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 b/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 index 5337f342fedd8..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 +++ b/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - TOK_SHOWTABLES - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Table Operator: - Show Tables - database name: default - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 b/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 b/sql/hive/src/test/resources/golden/input21-1-c45ad493e95150b580be778da6065f36 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 rename to sql/hive/src/test/resources/golden/input21-1-c45ad493e95150b580be778da6065f36 diff --git a/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe b/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe index c6c298df200ca..8b39955512bc7 100644 --- a/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe +++ b/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe @@ -7,4 +7,4 @@ NULL 1 same 5 NULL NULL same 6 1.0 NULL same 7 1.0 1 same 8 -1.0 1 same 9 \ No newline at end of file +1.0 1 same 9 diff --git a/sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 b/sql/hive/src/test/resources/golden/input22-1-b663ec84da3f9d9b9594ea2da81b1442 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 rename to sql/hive/src/test/resources/golden/input22-1-b663ec84da3f9d9b9594ea2da81b1442 diff --git a/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 b/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 index 336ba0545635d..891dedb34c1dd 100644 --- a/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 +++ b/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 @@ -7,4 +7,4 @@ 103 103 104 -104 \ No newline at end of file +104 diff --git a/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 +++ b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 b/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 index b8fd0ab6545d6..833b21e4d6e4d 100644 --- a/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 +++ b/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 @@ -2,4 +2,4 @@ 0 val_0 2008-04-08 11 0 val_0 2008-04-08 11 2 val_2 2008-04-08 11 -4 val_4 2008-04-08 11 \ No newline at end of file +4 val_4 2008-04-08 11 diff --git a/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 b/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 index 899417ee77ad8..badda49814562 100644 --- a/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 +++ b/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 @@ -2,4 +2,4 @@ 86 val_86 27 val_27 165 val_165 -255 val_255 \ No newline at end of file +255 val_255 diff --git a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d index 743be67e8d1c3..d3ffb995aff4b 100644 --- a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d +++ b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d @@ -1,2 +1,2 @@ -a int None -b double None \ No newline at end of file +a int +b double diff --git a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 index 594b29ca1410f..bd673a6c1f1d4 100644 --- a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 +++ b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 @@ -1,4 +1,4 @@ -a array None -b double None -c map None -x double None \ No newline at end of file +a array +b double +c map +x double diff --git a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c index ac382c7369264..f5b9883df09c0 100644 --- a/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c @@ -1,4 +1,4 @@ src srcpart test3a -test3c \ No newline at end of file +test3c diff --git a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 index cb17be511e875..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 +++ b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_REPLACECOLS TEST3c (TOK_TABCOLLIST (TOK_TABCOL R1 TOK_INT) (TOK_TABCOL R2 TOK_DOUBLE))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Alter Table Operator: - Alter Table - type: replace columns - new columns: r1 int, r2 double - old name: TEST3c - diff --git a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 index b906fd3c2e775..ea55abd792314 100644 --- a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 +++ b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 @@ -1,4 +1,4 @@ -r1 int None -r2 double None +r1 int +r2 double -Detailed Table Information Table(tableName:test3c, dbName:default, owner:marmbrus, createTime:1389730377, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:r1, type:int, comment:null), FieldSchema(name:r2, type:double, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/test3c, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1389730378, transient_lastDdlTime=1389730378}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:test3c, dbName:default, owner:marmbrus, createTime:1413882084, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:r1, type:int, comment:null), FieldSchema(name:r2, type:double, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/test3c, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413882084, transient_lastDdlTime=1413882084, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 index ca0726f517eeb..77eaef91c9c3f 100644 --- a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 +++ b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 @@ -1,3 +1,3 @@ -a array None -b double None -c map None \ No newline at end of file +a array +b double +c map diff --git a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c index 6b1ce270d97e9..b584fd7c6fd36 100644 --- a/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c @@ -1,4 +1,4 @@ src srcpart test3a -test3b \ No newline at end of file +test3b diff --git a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 index 92c0ed68f8a7b..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 +++ b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_ADDCOLS TEST3b (TOK_TABCOLLIST (TOK_TABCOL X TOK_DOUBLE))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Alter Table Operator: - Alter Table - type: add columns - new columns: x double - old name: TEST3b - diff --git a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 index 594b29ca1410f..bd673a6c1f1d4 100644 --- a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 +++ b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 @@ -1,4 +1,4 @@ -a array None -b double None -c map None -x double None \ No newline at end of file +a array +b double +c map +x double diff --git a/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be index 09bbc29377720..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be +++ b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ALTERTABLE_RENAME TEST3b TEST3c) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Alter Table Operator: - Alter Table - type: rename - new name: TEST3c - old name: TEST3b - diff --git a/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c b/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab b/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 b/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 b/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f b/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f deleted file mode 100644 index 25bf17fc5aaab..0000000000000 --- a/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f +++ /dev/null @@ -1 +0,0 @@ -18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe b/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f b/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 b/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d b/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 b/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e b/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 b/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 b/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 b/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 deleted file mode 100644 index 25bf17fc5aaab..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 +++ /dev/null @@ -1 +0,0 @@ -18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 b/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d b/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe b/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 deleted file mode 100644 index 25bf17fc5aaab..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 +++ /dev/null @@ -1 +0,0 @@ -18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 b/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d b/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece b/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d b/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 b/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 b/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 deleted file mode 100644 index 6731b0cb0baaa..0000000000000 --- a/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 +++ /dev/null @@ -1,20 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -0 val_1 -0 val_1 -1 val_2 -10 val_10 -10 val_11 -100 val_100 -100 val_100 -100 val_101 -100 val_101 -101 val_102 -102 val_103 -103 val_103 -103 val_103 -104 val_104 -104 val_104 -104 val_105 -104 val_105 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 b/sql/hive/src/test/resources/golden/input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e similarity index 100% rename from sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 rename to sql/hive/src/test/resources/golden/input4-1-7ce0bc5e5feeb09bf3fc139e102fb00e diff --git a/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 b/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 deleted file mode 100644 index 65a457b52b0a6..0000000000000 --- a/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 +++ /dev/null @@ -1,27 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_LOAD '/Users/marmbrus/workspace/hive/data/files/kv1.txt' (TOK_TAB (TOK_TABNAME INPUT4)) LOCAL) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - Stage-2 depends on stages: Stage-1 - -STAGE PLANS: - Stage: Stage-0 - Copy - source: file:/Users/marmbrus/workspace/hive/data/files/kv1.txt - destination: file:/tmp/hive-marmbrus/hive_2014-01-14_12-16-46_262_527870677085258278-1/-ext-10000 - - Stage: Stage-1 - Move Operator - tables: - replace: false - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.input4 - - Stage: Stage-2 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 b/sql/hive/src/test/resources/golden/input4-2-b663ec84da3f9d9b9594ea2da81b1442 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 rename to sql/hive/src/test/resources/golden/input4-2-b663ec84da3f9d9b9594ea2da81b1442 diff --git a/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 b/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 index 218c223b050b7..103b537db90c6 100644 --- a/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 +++ b/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 @@ -497,4 +497,4 @@ val_169 169 val_403 403 val_400 400 val_200 200 -val_97 97 \ No newline at end of file +val_97 97 diff --git a/sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb b/sql/hive/src/test/resources/golden/input40-1-a8adb8ae1d13607851431a1baf7578ba similarity index 100% rename from sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb rename to sql/hive/src/test/resources/golden/input40-1-a8adb8ae1d13607851431a1baf7578ba diff --git a/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 b/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 b/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 +++ b/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e b/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 b/sql/hive/src/test/resources/golden/input40-4-fdeea6b676c670b17c8d91e24a97a127 similarity index 100% rename from sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 rename to sql/hive/src/test/resources/golden/input40-4-fdeea6b676c670b17c8d91e24a97a127 diff --git a/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 index 6bc66cd8fe19b..e4b818f03539d 100644 --- a/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 +++ b/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 @@ -497,4 +497,4 @@ 97 val_97 2009-08-01 97 val_97 2009-08-01 98 val_98 2009-08-01 -98 val_98 2009-08-01 \ No newline at end of file +98 val_98 2009-08-01 diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 b/sql/hive/src/test/resources/golden/input40-6-6651f53efc5d03ed2d43b9d7aecc0002 similarity index 100% rename from sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 rename to sql/hive/src/test/resources/golden/input40-6-6651f53efc5d03ed2d43b9d7aecc0002 diff --git a/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 b/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 index ee3dddd8cff71..4467e7af00c0e 100644 --- a/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 +++ b/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 @@ -997,4 +997,4 @@ 97 val_98 2009-08-01 98 val_98 2009-08-01 98 val_98 2009-08-01 -99 val_100 2009-08-01 \ No newline at end of file +99 val_100 2009-08-01 diff --git a/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 +++ b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee index 579784a58a66c..61191cde2953b 100644 --- a/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee +++ b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee @@ -1,2 +1,2 @@ 0 -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a b/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 b/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f b/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca b/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 b/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 b/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab b/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb b/sql/hive/src/test/resources/golden/input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537 similarity index 100% rename from sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb rename to sql/hive/src/test/resources/golden/input4_cb_delim-1-353d2238b781a117888a67bb7b2b2537 diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 b/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 b/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 index 218c223b050b7..103b537db90c6 100644 --- a/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 +++ b/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 @@ -497,4 +497,4 @@ val_169 169 val_403 403 val_400 400 val_200 200 -val_97 97 \ No newline at end of file +val_97 97 diff --git a/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d b/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af b/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af deleted file mode 100644 index 217a1915f8826..0000000000000 --- a/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af +++ /dev/null @@ -1,10 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea b/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 b/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad b/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index f8aa003a65bb4..0000000000000 --- a/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -NULL NULL -[0,0,0] [{"myint":0,"mystring":"0","underscore_int":0}] -[1,2,3] [{"myint":1,"mystring":"1","underscore_int":1}] -[2,4,6] [{"myint":4,"mystring":"8","underscore_int":2}] -[3,6,9] [{"myint":9,"mystring":"27","underscore_int":3}] -[4,8,12] [{"myint":16,"mystring":"64","underscore_int":4}] -[5,10,15] [{"myint":25,"mystring":"125","underscore_int":5}] -[6,12,18] [{"myint":36,"mystring":"216","underscore_int":6}] -[7,14,21] [{"myint":49,"mystring":"343","underscore_int":7}] -[8,16,24] [{"myint":64,"mystring":"512","underscore_int":8}] -[9,18,27] [{"myint":81,"mystring":"729","underscore_int":9}] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f index b4dc488518f24..f3f63f08fcf70 100644 --- a/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f @@ -7,4 +7,4 @@ NULL val_265 NULL val_193 NULL NULL -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f index 0b8a8960a992b..65cada3d45b2b 100644 --- a/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f @@ -22,4 +22,4 @@ NULL 146 NULL 406 NULL NULL NULL NULL -NULL NULL \ No newline at end of file +NULL NULL diff --git a/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f index c87107a2f1168..416fbfb9e5228 100644 --- a/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f @@ -22,4 +22,4 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL -NULL NULL NULL \ No newline at end of file +NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e b/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 b/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be b/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 b/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 deleted file mode 100644 index e782acd4d1e7f..0000000000000 --- a/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 b/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 deleted file mode 100644 index 6038b8aa32884..0000000000000 --- a/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL 1712634731 record_0 -1 10 NULL 465985200 record_1 -2 20 NULL -751827638 record_2 -3 30 NULL 477111222 record_3 -4 40 NULL -734328909 record_4 -5 50 NULL -1952710710 record_5 -6 60 NULL 1244525190 record_6 -7 70 NULL -1461153973 record_7 -8 80 NULL 1638581578 record_8 -9 90 NULL 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc b/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 b/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index e782acd4d1e7f..0000000000000 --- a/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 b/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 deleted file mode 100644 index 6038b8aa32884..0000000000000 --- a/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL 1712634731 record_0 -1 10 NULL 465985200 record_1 -2 20 NULL -751827638 record_2 -3 30 NULL 477111222 record_3 -4 40 NULL -734328909 record_4 -5 50 NULL -1952710710 record_5 -6 60 NULL 1244525190 record_6 -7 70 NULL -1461153973 record_7 -8 80 NULL 1638581578 record_8 -9 90 NULL 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 b/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 b/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c b/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c deleted file mode 100644 index 6af528eab23a8..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c +++ /dev/null @@ -1,11 +0,0 @@ -{"key_0":"value_0"} -{"key_1":"value_1"} -{"key_2":"value_2"} -{"key_3":"value_3"} -{"key_4":"value_4"} -{"key_5":"value_5"} -{"key_6":"value_6"} -{"key_7":"value_7"} -{"key_8":"value_8"} -{"key_9":"value_9"} -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc b/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 b/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 deleted file mode 100644 index e782acd4d1e7f..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] ["0","0","0"] {"key_0":"value_0"} 1712634731 record_0 -[1,2,3] ["10","100","1000"] {"key_1":"value_1"} 465985200 record_1 -[2,4,6] ["20","200","2000"] {"key_2":"value_2"} -751827638 record_2 -[3,6,9] ["30","300","3000"] {"key_3":"value_3"} 477111222 record_3 -[4,8,12] ["40","400","4000"] {"key_4":"value_4"} -734328909 record_4 -[5,10,15] ["50","500","5000"] {"key_5":"value_5"} -1952710710 record_5 -[6,12,18] ["60","600","6000"] {"key_6":"value_6"} 1244525190 record_6 -[7,14,21] ["70","700","7000"] {"key_7":"value_7"} -1461153973 record_7 -[8,16,24] ["80","800","8000"] {"key_8":"value_8"} 1638581578 record_8 -[9,18,27] ["90","900","9000"] {"key_9":"value_9"} 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db b/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db deleted file mode 100644 index 6038b8aa32884..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL 1712634731 record_0 -1 10 NULL 465985200 record_1 -2 20 NULL -751827638 record_2 -3 30 NULL 477111222 record_3 -4 40 NULL -734328909 record_4 -5 50 NULL -1952710710 record_5 -6 60 NULL 1244525190 record_6 -7 70 NULL -1461153973 record_7 -8 80 NULL 1638581578 record_8 -9 90 NULL 336964413 record_9 -NULL NULL NULL 0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 b/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 b/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c deleted file mode 100644 index 1bb008b44d6ee..0000000000000 --- a/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c +++ /dev/null @@ -1,11 +0,0 @@ -[0,0,0] -[1,2,3] -[2,4,6] -[3,6,9] -[4,8,12] -[5,10,15] -[6,12,18] -[7,14,21] -[8,16,24] -[9,18,27] -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 b/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 index 17f24d2991d14..03887aed65852 100644 --- a/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 +++ b/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 @@ -17,4 +17,4 @@ 369 val_369 66 val_66 128 val_128 -213 val_213 \ No newline at end of file +213 val_213 diff --git a/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb b/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb index 499e6b8ab6b7e..d7a8f25b41301 100644 --- a/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb +++ b/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb @@ -997,4 +997,4 @@ 403 val_403 2008-04-08 12 400 val_400 2008-04-08 12 200 val_200 2008-04-08 12 -97 val_97 2008-04-08 12 \ No newline at end of file +97 val_97 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f index 3e5ae10e4670a..7ae7ecbe28de6 100644 --- a/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 28 val_28 12 2008-04-08 37 val_37 12 2008-04-08 90 val_90 12 2008-04-08 -97 val_97 12 2008-04-08 \ No newline at end of file +97 val_97 12 2008-04-08 diff --git a/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 index 82116102c1f54..89c49ce857f5a 100644 --- a/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 +++ b/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 @@ -1,12 +1,12 @@ -a string None -b string None -ds string None -ts string None +a string +b string +ds string +ts string # Partition Information # col_name data_type comment -ds string None -ts string None +ds string +ts string -Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1388798899, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6540137288252557391/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1388798899}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1413882241, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882241, COLUMN_STATS_ACCURATE=true, totalSize=4, numRows=1, rawDataSize=3}) diff --git a/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 index c8d0d55930069..6de1c02821c77 100644 --- a/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 +++ b/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 @@ -1 +1 @@ -1 2 2008 04 08 10:11:12=455 \ No newline at end of file +1 2 2008 04 08 10:11:12=455 diff --git a/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 index 869eb58e70d82..f5c60fae1925e 100644 --- a/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 +++ b/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 @@ -1,12 +1,12 @@ -a string None -b string None -ds string None -ts string None +a string +b string +ds string +ts string # Partition Information # col_name data_type comment -ds string None -ts string None +ds string +ts string -Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1388798920, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6540137288252557391/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1388798920, numRows=1, totalSize=4, rawDataSize=3}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008 04 08, 10:11:12=455], dbName:default, tableName:part_special, createTime:1413882252, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:a, type:string, comment:null), FieldSchema(name:b, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:ts, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/part_special/ds=2008 04 08/ts=10%3A11%3A12%3D455, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882252, COLUMN_STATS_ACCURATE=true, totalSize=4, numRows=1, rawDataSize=3}) diff --git a/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 index c8d0d55930069..6de1c02821c77 100644 --- a/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 +++ b/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 @@ -1 +1 @@ -1 2 2008 04 08 10:11:12=455 \ No newline at end of file +1 2 2008 04 08 10:11:12=455 diff --git a/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 b/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 index fd945cc15d9ca..883de3e945c46 100644 --- a/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 +++ b/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 @@ -81,4 +81,4 @@ 97 val_97 12 2008-04-08 97 val_97 12 2008-04-08 98 val_98 12 2008-04-08 -98 val_98 12 2008-04-08 \ No newline at end of file +98 val_98 12 2008-04-08 diff --git a/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 b/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 index f23877361f33b..b23aa27263654 100644 --- a/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 +++ b/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 @@ -81,4 +81,4 @@ 97 val_97 12 2008-04-09 97 val_97 12 2008-04-09 98 val_98 12 2008-04-09 -98 val_98 12 2008-04-09 \ No newline at end of file +98 val_98 12 2008-04-09 diff --git a/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 b/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 index f4026a591a958..31b575a403f81 100644 --- a/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 +++ b/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 @@ -497,4 +497,4 @@ 403 val_403 2008-04-08 11 400 val_400 2008-04-08 11 200 val_200 2008-04-08 11 -97 val_97 2008-04-08 11 \ No newline at end of file +97 val_97 2008-04-08 11 diff --git a/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 index 853c3bc8df7f0..dfabe7827837c 100644 --- a/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 +++ b/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 @@ -165,4 +165,4 @@ 98 val_98 2008-04-08 11 98 val_98 2008-04-08 11 98 val_98 2008-04-08 12 -98 val_98 2008-04-08 12 \ No newline at end of file +98 val_98 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 b/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 index 540ca86636f3c..ed8993a9cbd0c 100644 --- a/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 +++ b/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 @@ -333,4 +333,4 @@ 98 val_98 2008-04-08 12 98 val_98 2008-04-08 12 98 val_98 2008-04-08 12 -98 val_98 2008-04-08 12 \ No newline at end of file +98 val_98 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 b/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 index a1728c82f0b35..5ee171a64f7ab 100644 --- a/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 +++ b/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 @@ -7,4 +7,4 @@ 255 val_255 2008-04-08 11 278 val_278 2008-04-08 11 98 val_98 2008-04-08 11 -484 val_484 2008-04-08 11 \ No newline at end of file +484 val_484 2008-04-08 11 diff --git a/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 b/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 index 438355d7b06f5..b7704cd0a2f0f 100644 --- a/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 +++ b/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 @@ -997,4 +997,4 @@ 498 val_498 2008-04-08 11 498 val_498 2008-04-08 12 498 val_498 2008-04-08 12 -498 val_498 2008-04-08 12 \ No newline at end of file +498 val_498 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 b/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 b/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f b/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 b/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index c1a6abba038e7..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,11 +0,0 @@ -0 0 NULL -2 1 NULL -4 8 value_2 -6 27 NULL -8 64 NULL -10 125 NULL -12 216 NULL -14 343 NULL -16 512 NULL -18 729 NULL -NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 b/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 b/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd b/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 51645b2a07a39..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,10 +0,0 @@ -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 -3 1 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac b/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 b/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 deleted file mode 100644 index 373a573714f4e..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 +++ /dev/null @@ -1,11 +0,0 @@ -NULL [0] -NULL [1] -NULL [4] -NULL [9] -NULL [16] -NULL [25] -NULL [36] -NULL [49] -NULL [64] -value_9 [81] -NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 deleted file mode 100644 index 7490d2d44d71a..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 +++ /dev/null @@ -1,10 +0,0 @@ -NULL [0] -NULL [1] -NULL [4] -NULL [9] -NULL [16] -NULL [25] -NULL [36] -NULL [49] -NULL [64] -value_9 [81] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 deleted file mode 100644 index 7490d2d44d71a..0000000000000 --- a/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 +++ /dev/null @@ -1,10 +0,0 @@ -NULL [0] -NULL [1] -NULL [4] -NULL [9] -NULL [16] -NULL [25] -NULL [36] -NULL [49] -NULL [64] -value_9 [81] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 index 679d54cb5cb5e..3577c8a431869 100644 --- a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 +++ b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 @@ -1,10 +1,10 @@ -key int None -value string None -ds string None -country string None +key int +value string +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None +ds string +country string diff --git a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c index ded361eb294f0..90f9bd0430a4c 100644 --- a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c +++ b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c @@ -1,2 +1,2 @@ -key int None -value string None +key int +value string diff --git a/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 b/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 index d5a489a9a5ed5..454e27ff0c28d 100644 --- a/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 +++ b/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 @@ -1,15 +1,15 @@ -viewtime string None -userid int None -page_url string None -referrer_url string None -friends array None -properties map None +viewtime string +userid int +page_url string +referrer_url string +friends array +properties map ip string IP Address of the User -ds string None -country string None +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None \ No newline at end of file +ds string +country string diff --git a/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 b/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 index e0bedb0512cfc..0ea7cee2a9cf9 100644 --- a/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 +++ b/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 @@ -1,17 +1,17 @@ -viewtime string None -userid int None -page_url string None -referrer_url string None -friends array None -properties map None +viewtime string +userid int +page_url string +referrer_url string +friends array +properties map ip string IP Address of the User -ds string None -country string None +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None +ds string +country string -Detailed Table Information Table(tableName:inputddl4, dbName:default, owner:marmbrus, createTime:1389731336, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:viewtime, type:string, comment:null), FieldSchema(name:userid, type:int, comment:null), FieldSchema(name:page_url, type:string, comment:null), FieldSchema(name:referrer_url, type:string, comment:null), FieldSchema(name:friends, type:array, comment:null), FieldSchema(name:properties, type:map, comment:null), FieldSchema(name:ip, type:string, comment:IP Address of the User), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[userid], sortCols:[Order(col:viewtime, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1389731336, comment=This is the page view table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:inputddl4, dbName:default, owner:marmbrus, createTime:1413882343, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:viewtime, type:string, comment:null), FieldSchema(name:userid, type:int, comment:null), FieldSchema(name:page_url, type:string, comment:null), FieldSchema(name:referrer_url, type:string, comment:null), FieldSchema(name:friends, type:array, comment:null), FieldSchema(name:properties, type:map, comment:null), FieldSchema(name:ip, type:string, comment:IP Address of the User), FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl4, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[userid], sortCols:[Order(col:viewtime, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882343, comment=This is the page view table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 b/sql/hive/src/test/resources/golden/inputddl6-1-572c1abb70f09726d1ba77bdc884597b similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 rename to sql/hive/src/test/resources/golden/inputddl6-1-572c1abb70f09726d1ba77bdc884597b diff --git a/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf b/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 b/sql/hive/src/test/resources/golden/inputddl6-2-3a4def4b370f75c5fcc1174626490363 similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 rename to sql/hive/src/test/resources/golden/inputddl6-2-3a4def4b370f75c5fcc1174626490363 diff --git a/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c b/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c b/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c index f1427896e8330..a8a418d9736d9 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c +++ b/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:inputddl6, dbName:default, owner:marmbrus, createTime:1389731342, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=2, numFiles=2, transient_lastDdlTime=1389731342, numRows=0, totalSize=11624, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:inputddl6, dbName:default, owner:marmbrus, createTime:1413882344, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl6, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882344}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 b/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 index 822897217e867..05507162a9244 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 +++ b/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2008-04-08], dbName:default, tableName:inputddl6, createTime:1389731342, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl6/ds=2008-04-08, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731342, numRows=0, totalSize=5812, rawDataSize=0}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008-04-08], dbName:default, tableName:inputddl6, createTime:1413882344, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl6/ds=2008-04-08, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882344, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0}) diff --git a/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 index 94bcaaee2408c..6e2459ea22ee1 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 +++ b/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 @@ -1,2 +1,2 @@ ds=2008-04-08 -ds=2008-04-09 \ No newline at end of file +ds=2008-04-09 diff --git a/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 index b12a9f82cd90a..017a142ab30b7 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 +++ b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 @@ -1 +1 @@ -ds=2008-04-09 \ No newline at end of file +ds=2008-04-09 diff --git a/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 b/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 index a6c282ab6f573..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 +++ b/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_DESCTABLE (TOK_TABTYPE INPUTDDL6 (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09'))) EXTENDED) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Describe Table Operator: - Describe Table - partition: - ds 2008-04-09 - table: INPUTDDL6 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 b/sql/hive/src/test/resources/golden/inputddl7-1-7195712efb4910294f63303ebce24453 similarity index 100% rename from sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 rename to sql/hive/src/test/resources/golden/inputddl7-1-7195712efb4910294f63303ebce24453 diff --git a/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d b/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c b/sql/hive/src/test/resources/golden/inputddl7-10-7c9248b56948716913d332bd712d69bd similarity index 100% rename from sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c rename to sql/hive/src/test/resources/golden/inputddl7-10-7c9248b56948716913d332bd712d69bd diff --git a/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 +++ b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 b/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 index ef633a4aa09e0..edfcdbb1211bb 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 +++ b/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 @@ -1,3 +1,3 @@ -name string None +name string -Detailed Table Information Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1389731349, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1389731349, numRows=0, totalSize=5812, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:t1, dbName:default, owner:marmbrus, createTime:1413882345, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413882345, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a b/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a index 86d5d8a125fbe..86c9b459e36c5 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a +++ b/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a @@ -1,3 +1,3 @@ -name string None +name string -Detailed Table Information Table(tableName:t2, dbName:default, owner:marmbrus, createTime:1389731362, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t2, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, transient_lastDdlTime=1389731362, numRows=0, totalSize=10508, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:t2, dbName:default, owner:marmbrus, createTime:1413882355, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t2, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413882355, COLUMN_STATS_ACCURATE=true, totalSize=10508, numRows=0, rawDataSize=0}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 b/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 index b14d78536150c..04956c2c1c447 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 +++ b/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 @@ -1,9 +1,9 @@ -name string None -ds string None +name string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t3, createTime:1389731375, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t3/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731375, numRows=0, totalSize=5812, rawDataSize=0}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t3, createTime:1413882365, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t3/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882365, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=0, rawDataSize=0}) diff --git a/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe b/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe index e476a7fa3959b..76ecadd3851a6 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe +++ b/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe @@ -1,9 +1,9 @@ -name string None -ds string None +name string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t4, createTime:1389731388, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/t4/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1389731388, numRows=0, totalSize=10508, rawDataSize=0}) \ No newline at end of file +Detailed Partition Information Partition(values:[2008-04-09], dbName:default, tableName:t4, createTime:1413882375, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:name, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/t4/ds=2008-04-09, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413882375, COLUMN_STATS_ACCURATE=true, totalSize=10508, numRows=0, rawDataSize=0}) diff --git a/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 +++ b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 b/sql/hive/src/test/resources/golden/inputddl7-4-68715ba2c11220be62394c86453e6d54 similarity index 100% rename from sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 rename to sql/hive/src/test/resources/golden/inputddl7-4-68715ba2c11220be62394c86453e6d54 diff --git a/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 b/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 +++ b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 b/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 b/sql/hive/src/test/resources/golden/inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71 similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 rename to sql/hive/src/test/resources/golden/inputddl7-7-59dd2d2556769e19bdc0a444f40f8a71 diff --git a/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 +++ b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 b/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 index 66e6efceed3dc..5166f3678f405 100644 --- a/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 +++ b/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 @@ -4,13 +4,13 @@ lint array from deserializer lstring array from deserializer lintstring array> from deserializer mstringstring map from deserializer -ds string None -country string None +ds string +country string # Partition Information # col_name data_type comment -ds string None -country string None +ds string +country string -Detailed Table Information Table(tableName:inputddl8, dbName:default, owner:marmbrus, createTime:1389731407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7216708901107607121/inputddl8, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.Complex, serialization.format=com.facebook.thrift.protocol.TBinaryProtocol}), bucketCols:[aint], sortCols:[Order(col:lint, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1389731407, comment=This is a thrift based table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:inputddl8, dbName:default, owner:marmbrus, createTime:1413882387, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/inputddl8, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:32, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.Complex, serialization.format=com.facebook.thrift.protocol.TBinaryProtocol}), bucketCols:[aint], sortCols:[Order(col:lint, order:1)], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null), FieldSchema(name:country, type:string, comment:null)], parameters:{transient_lastDdlTime=1413882387, comment=This is a thrift based table}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 rename to sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-86653b3af59df59f225ee00ff5fc119f diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 rename to sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-e81d45a5bec5642ec4b762f1c1a482af diff --git a/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d +++ b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef +++ b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef index e37d32abba426..83b33d238dab9 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef +++ b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef @@ -1 +1 @@ -1000 \ No newline at end of file +1000 diff --git a/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef index 37021f4a27201..3d86ec6498f3f 100644 --- a/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef +++ b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef @@ -1 +1 @@ -1500 \ No newline at end of file +1500 diff --git a/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b b/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 b/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb deleted file mode 100644 index 5e96d815b6b78..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb +++ /dev/null @@ -1 +0,0 @@ --826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 b/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 b/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb deleted file mode 100644 index eb9dc5833c2f9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb +++ /dev/null @@ -1 +0,0 @@ -10226524244 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 b/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 b/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb deleted file mode 100644 index 28ced898ab537..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb +++ /dev/null @@ -1 +0,0 @@ -20453048488 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d b/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d deleted file mode 100644 index ae4ee13c08e76..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d +++ /dev/null @@ -1 +0,0 @@ -200 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b b/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 b/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 b/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa b/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 deleted file mode 100644 index 84d7f3929d86d..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 +++ /dev/null @@ -1 +0,0 @@ --27100860056 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 b/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 b/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 deleted file mode 100644 index ae4ee13c08e76..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 +++ /dev/null @@ -1 +0,0 @@ -200 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 deleted file mode 100644 index 3395f3bcc7b51..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 +++ /dev/null @@ -1 +0,0 @@ --24159954504 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 b/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 b/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 deleted file mode 100644 index ee0a47c9f6e00..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 +++ /dev/null @@ -1 +0,0 @@ --36239931656 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 deleted file mode 100644 index 5e96d815b6b78..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 +++ /dev/null @@ -1 +0,0 @@ --826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 deleted file mode 100644 index 2ed5a7da11dcd..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 +++ /dev/null @@ -1 +0,0 @@ -9399898328 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 b/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d b/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd b/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 b/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 deleted file mode 100644 index 00ffdd24b0cff..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 +++ /dev/null @@ -1 +0,0 @@ -7813690682 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 deleted file mode 100644 index eb9dc5833c2f9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 +++ /dev/null @@ -1 +0,0 @@ -10226524244 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 b/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 b/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 b/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 deleted file mode 100644 index e2954bd63682e..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 +++ /dev/null @@ -1 +0,0 @@ --1653251832 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa b/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 b/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b b/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b deleted file mode 100644 index e2954bd63682e..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b +++ /dev/null @@ -1 +0,0 @@ --1653251832 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b b/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a b/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 deleted file mode 100644 index 5e96d815b6b78..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 +++ /dev/null @@ -1 +0,0 @@ --826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c b/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 b/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d b/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 deleted file mode 100644 index 3b6ef434b37e7..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 +++ /dev/null @@ -1 +0,0 @@ --18626052920 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 b/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 b/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 deleted file mode 100644 index 0744f3dae0e0a..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 +++ /dev/null @@ -1 +0,0 @@ --37252105840 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c b/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 b/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 b/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 b/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 deleted file mode 100644 index c0066b75af40e..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 +++ /dev/null @@ -1 +0,0 @@ -481928560 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f b/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 b/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 deleted file mode 100644 index 0b4a44e064f85..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 +++ /dev/null @@ -1 +0,0 @@ -963857120 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 b/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba b/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 b/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 deleted file mode 100644 index d541b5d207233..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 +++ /dev/null @@ -1 +0,0 @@ --35226404960 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 b/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 deleted file mode 100644 index c21f4017362c1..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f b/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 b/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 b/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 b/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 b/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab b/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab deleted file mode 100644 index d541b5d207233..0000000000000 --- a/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab +++ /dev/null @@ -1 +0,0 @@ --35226404960 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 b/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a b/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b b/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 b/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 b/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 index 5707ed08e7e54..13e88f30fc08c 100644 --- a/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 +++ b/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 @@ -97,4 +97,4 @@ 9 val_9 5 val_5 9 val_9 5 val_5 9 val_9 8 val_8 -9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 +++ b/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d b/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d +++ b/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 b/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 index c888cfd6a0479..e9016953d4e48 100644 --- a/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 +++ b/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 @@ -145,4 +145,4 @@ 90 val_90 90 val_90 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f b/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f index 1780076ae513f..598608f124d38 100644 --- a/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f +++ b/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f @@ -229,4 +229,4 @@ 37 val_37 37 val_37 37 val_37 -37 val_37 \ No newline at end of file +37 val_37 diff --git a/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 b/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 index a82a9a22c6b87..f9890a4c2d194 100644 --- a/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 +++ b/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 @@ -161,4 +161,4 @@ 90 val_90 90 val_90 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e +++ b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 +++ b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 +++ b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 index 941e6bb7c887e..bd6afa8dada84 100644 --- a/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 +++ b/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 @@ -1749,4 +1749,4 @@ 403 val_403 400 val_400 200 val_200 -200 val_200 \ No newline at end of file +200 val_200 diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 index 941e6bb7c887e..bd6afa8dada84 100644 --- a/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 @@ -1749,4 +1749,4 @@ 403 val_403 400 val_400 200 val_200 -200 val_200 \ No newline at end of file +200 val_200 diff --git a/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 b/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 index c2e3ea8b0c8e2..b212e93a0a8c2 100644 --- a/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 +++ b/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 -498 val_498 498 val_498 \ No newline at end of file +498 val_498 498 val_498 diff --git a/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f index 66fafbdf72b63..3df4716f0b05f 100644 --- a/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 200 val_200 200 val_200 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 b/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 index 10628f6c64bd4..947ed199b0d4d 100644 --- a/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 +++ b/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 @@ -307,4 +307,4 @@ NULL NULL NULL 7 495 1 NULL NULL 496 1 NULL NULL 497 1 NULL NULL -498 3 NULL NULL \ No newline at end of file +498 3 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 b/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 index 2945c31ed4f00..12a176c7bc7cb 100644 --- a/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 +++ b/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 @@ -469,4 +469,4 @@ 200 val_400 200 val_400 97 val_194 -97 val_194 \ No newline at end of file +97 val_194 diff --git a/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 +++ b/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 +++ b/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f index 9672e21fa0323..80c230cf4a09d 100644 --- a/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f +++ b/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f @@ -497,4 +497,4 @@ NULL NULL NULL NULL 496 val_496 NULL NULL NULL NULL 497 val_497 NULL NULL NULL NULL 498 val_498 NULL NULL NULL NULL 498 val_498 -NULL NULL NULL NULL 498 val_498 \ No newline at end of file +NULL NULL NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 b/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 index 5707ed08e7e54..13e88f30fc08c 100644 --- a/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 +++ b/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 @@ -97,4 +97,4 @@ 9 val_9 5 val_5 9 val_9 5 val_5 9 val_9 8 val_8 -9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 +++ b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 +++ b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa index 183353e5c705b..d3d377e8ae74d 100644 --- a/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -34,4 +34,4 @@ 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa index 4c88927e56579..120056ea10c60 100644 --- a/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 b/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 index 2b75023ea4a9f..2adcbc0f14b62 100644 --- a/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 +++ b/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 @@ -38,4 +38,4 @@ NULL val_484 val_484 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa index da3c427cab73f..c998494e1c524 100644 --- a/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa index f553ce0ca41f3..c8445b6e2c784 100644 --- a/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 401 1 5 406 1 4 66 1 1 -98 1 2 \ No newline at end of file +98 1 2 diff --git a/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f index 8886dc7e8f229..9c33812fa7ea4 100644 --- a/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f @@ -2651,4 +2651,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa index 71094ee7360db..16b313fc58f23 100644 --- a/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 311 3 369 3 401 5 -406 4 \ No newline at end of file +406 4 diff --git a/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa index 5d28208ab255c..7c33b34887d6b 100644 --- a/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -12,4 +12,4 @@ 401 1 406 1 66 1 -98 1 \ No newline at end of file +98 1 diff --git a/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a +++ b/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 index b1251794645ce..13c35f8a6c6ba 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 @@ -94,4 +94,4 @@ NULL val_484 val_484 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 +++ b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a +++ b/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa index a6538b605a817..a4dde6240cac7 100644 --- a/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -82,4 +82,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa index e18d1ff802c93..6fbe456119b70 100644 --- a/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -31,4 +31,4 @@ 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 +++ b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa index efdd80c9f8b89..2f62508e3342a 100644 --- a/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa @@ -10,4 +10,4 @@ 311 val_311 3 369 3 401 val_401 5 -406 val_406 4 \ No newline at end of file +406 val_406 4 diff --git a/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 +++ b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa index a250f202c4df0..4cb7ec20b1a33 100644 --- a/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa @@ -306,4 +306,4 @@ 495 1 1 496 1 1 497 1 1 -498 3 3 \ No newline at end of file +498 3 3 diff --git a/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 +++ b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa index 183353e5c705b..d3d377e8ae74d 100644 --- a/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa @@ -34,4 +34,4 @@ 406 val_406 val_406 406 val_406 val_406 406 val_406 val_406 -406 val_406 val_406 \ No newline at end of file +406 val_406 val_406 diff --git a/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 b/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 index 5e020e3aff653..3dcf0e17586a9 100644 --- a/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 +++ b/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 @@ -1,2 +1,2 @@ 100 101 102 103 104 105 106 107 108 109 110 111 -100 101 102 103 104 105 106 107 108 109 110 111 \ No newline at end of file +100 101 102 103 104 105 106 107 108 109 110 111 diff --git a/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef index fce97c9211916..edbff1207ded9 100644 --- a/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef +++ b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef @@ -1 +1 @@ -val_111 105 2 \ No newline at end of file +val_111 105 2 diff --git a/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa index 5b45955ceb274..56d4dbe5b4d25 100644 --- a/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -563,4 +563,4 @@ 98 val_98 98 val_98 98 val_98 98 val_98 98 val_98 98 val_98 -98 val_98 98 val_98 \ No newline at end of file +98 val_98 98 val_98 diff --git a/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f index e45f3234d5a88..dc3a273d7a367 100644 --- a/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f @@ -8,4 +8,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac +++ b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa b/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa index 8543fe173f87f..50237fbde9e34 100644 --- a/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa +++ b/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa @@ -563,4 +563,4 @@ 400 val_400 NULL NULL 200 val_200 NULL NULL 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda +++ b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 b/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 index 8543fe173f87f..50237fbde9e34 100644 --- a/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 +++ b/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 @@ -563,4 +563,4 @@ 400 val_400 NULL NULL 200 val_200 NULL NULL 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 b/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 +++ b/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 +++ b/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 index bd6e20b8de1e4..7568f8cde28f5 100644 --- a/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 +++ b/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 @@ -545,4 +545,4 @@ NULL NULL NULL NULL 498 val_498 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 5 val_5 8 val_8 8 val_8 8 val_8 -9 val_9 9 val_9 9 val_9 \ No newline at end of file +9 val_9 9 val_9 9 val_9 diff --git a/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 +++ b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 index 7265626e5dbde..c20eb16d47f91 100644 --- a/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 +++ b/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 @@ -1,3 +1,3 @@ 0 val_0 NULL NULL 0 val_0 NULL NULL -0 val_0 NULL NULL \ No newline at end of file +0 val_0 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 index 7265626e5dbde..c20eb16d47f91 100644 --- a/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 +++ b/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 @@ -1,3 +1,3 @@ 0 val_0 NULL NULL 0 val_0 NULL NULL -0 val_0 NULL NULL \ No newline at end of file +0 val_0 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f index 06b2b4d7e6d47..b52cff5c472e4 100644 --- a/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f @@ -6,4 +6,4 @@ NULL NULL 24 val_24 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f index 36ca4a0805f4b..fb58885263569 100644 --- a/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f @@ -11,4 +11,4 @@ 19 val_19 19 val_19 NULL NULL 20 val_20 NULL NULL 24 val_24 -NULL NULL 24 val_24 \ No newline at end of file +NULL NULL 24 val_24 diff --git a/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f index eb0a6246c1988..0e75c1b63abb5 100644 --- a/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f @@ -11,4 +11,4 @@ 19 val_19 19 val_19 NULL NULL NULL NULL 20 val_20 NULL NULL NULL NULL 24 val_24 NULL NULL -NULL NULL 24 val_24 NULL NULL \ No newline at end of file +NULL NULL 24 val_24 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f index f2e6e07d8fa1f..c3f5359beb06f 100644 --- a/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f @@ -2,4 +2,4 @@ 12 val_12 NULL NULL 15 val_15 NULL NULL 11 val_11 NULL NULL -12 val_12 NULL NULL \ No newline at end of file +12 val_12 NULL NULL diff --git a/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a b/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 b/sql/hive/src/test/resources/golden/join_1to1-1-789b2636cfb6a08965e0bd190e419762 similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 rename to sql/hive/src/test/resources/golden/join_1to1-1-789b2636cfb6a08965e0bd190e419762 diff --git a/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e +++ b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 +++ b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 +++ b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e +++ b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 b/sql/hive/src/test/resources/golden/join_1to1-3-6228e662e573a00ed04550d049d97a3b similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 rename to sql/hive/src/test/resources/golden/join_1to1-3-6228e662e573a00ed04550d049d97a3b diff --git a/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 b/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 +++ b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 +++ b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e index 52a4d2c18e701..b57efb6ce27a2 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e +++ b/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e @@ -27,4 +27,4 @@ 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b +++ b/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c +++ b/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 index fb5560e2d006c..72e8fcf3ec4d0 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 +++ b/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 @@ -33,4 +33,4 @@ NULL 10050 66 NULL NULL NULL 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 index 0ae4eb2d33595..0f21800b43be9 100644 --- a/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 +++ b/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 @@ -31,4 +31,4 @@ NULL 10050 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 70 10040 66 NULL NULL NULL 80 10040 88 NULL NULL NULL -80 10040 88 NULL NULL NULL \ No newline at end of file +80 10040 88 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/join_array-2-16840a0266cad03a1a0b134d105b854f similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 rename to sql/hive/src/test/resources/golden/join_array-2-16840a0266cad03a1a0b134d105b854f diff --git a/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de b/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/join_array-3-a6ca6b64324596831033fdfe5b63a942 similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 rename to sql/hive/src/test/resources/golden/join_array-3-a6ca6b64324596831033fdfe5b63a942 diff --git a/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 b/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 index ce9688a065b34..84660b3eb4327 100644 --- a/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 +++ b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 @@ -1 +1 @@ -10320092026892491 3312 \ No newline at end of file +10320092026892491 3312 diff --git a/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a b/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a index f937af554adef..1d1c48ae5a48f 100644 --- a/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a +++ b/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a @@ -1,2 +1,2 @@ 10320092002467760 [0,23,37,48,53,55,55,56,60,66,72,76,77,78,80,81,87,88,90,90,91,90,92,97,100,103,104,107,108,108,109,110,113,113,113,113,113,113,114,116,116,116,117,116,117,117,117,115,115,117,117,117,121,120,131,131,131,125,125,124,124,128,128,131,131,132,133,134,134,134,134,26,26,null,null,null,null,116] -10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] \ No newline at end of file +10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] diff --git a/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c b/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c index f937af554adef..1d1c48ae5a48f 100644 --- a/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c +++ b/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c @@ -1,2 +1,2 @@ 10320092002467760 [0,23,37,48,53,55,55,56,60,66,72,76,77,78,80,81,87,88,90,90,91,90,92,97,100,103,104,107,108,108,109,110,113,113,113,113,113,113,114,116,116,116,117,116,117,117,117,115,115,117,117,117,121,120,131,131,131,125,125,124,124,128,128,131,131,132,133,134,134,134,134,26,26,null,null,null,null,116] -10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] \ No newline at end of file +10320092026892491 [0,2,59,106,131,142,159,244,320,398,417,433,553,616,710,826,917,971,1046,1051,1093,1112,1142,1215,1220,1226,1232,1267,1364,1549,1646,1948,2170,2272,2325,2433,2534,2852,2925,2992,3119,3207,3279,3323,3412,3637,3645,3634,3450,3473,3638,3688,3736,3758,3812,3862,3873,3868,3883,4118,4134,4127,4170,4216,null,null,null,null,3139] diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 b/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 b/sql/hive/src/test/resources/golden/join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf similarity index 100% rename from sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 rename to sql/hive/src/test/resources/golden/join_casesensitive-1-d1e9ae71a3ed691c39bb8f77ab28edbf diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc b/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 b/sql/hive/src/test/resources/golden/join_casesensitive-3-93300f4a9242fa2804a5b368538d83f similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 rename to sql/hive/src/test/resources/golden/join_casesensitive-3-93300f4a9242fa2804a5b368538d83f diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd b/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd index 810f01f682c37..f7d190a11d65c 100644 --- a/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd +++ b/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd @@ -82,4 +82,4 @@ NULL 10050 66 50 10050 88 80 10040 88 80 10040 66 80 10040 88 80 10040 66 80 10040 88 80 10040 66 -80 10040 88 80 10040 66 \ No newline at end of file +80 10040 88 80 10040 66 diff --git a/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a +++ b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a +++ b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d b/sql/hive/src/test/resources/golden/join_filters-1-bee6095f42de6a16708c2f9addc1b9bd similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d rename to sql/hive/src/test/resources/golden/join_filters-1-bee6095f42de6a16708c2f9addc1b9bd diff --git a/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 +++ b/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe +++ b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b +++ b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 +++ b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b +++ b/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 +++ b/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb +++ b/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 +++ b/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 +++ b/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e +++ b/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 +++ b/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 +++ b/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba +++ b/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 +++ b/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 +++ b/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba +++ b/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 +++ b/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 +++ b/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 +++ b/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 +++ b/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e +++ b/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a +++ b/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e +++ b/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 +++ b/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada +++ b/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 +++ b/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 +++ b/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf +++ b/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 +++ b/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 +++ b/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 +++ b/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 +++ b/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f +++ b/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 +++ b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf +++ b/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e +++ b/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e +++ b/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f +++ b/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 +++ b/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 +++ b/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca +++ b/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 +++ b/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 +++ b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 +++ b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 +++ b/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf +++ b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f +++ b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab +++ b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 +++ b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 +++ b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe +++ b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b +++ b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 +++ b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b +++ b/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 +++ b/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 +++ b/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb +++ b/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 +++ b/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 +++ b/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e +++ b/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f b/sql/hive/src/test/resources/golden/join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f rename to sql/hive/src/test/resources/golden/join_filters-46-268d8fb3cb9b04eb269fe7ec40a24dfe diff --git a/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 b/sql/hive/src/test/resources/golden/join_filters-47-6dc6866a65c74d69538b776b41b06c16 similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 rename to sql/hive/src/test/resources/golden/join_filters-47-6dc6866a65c74d69538b776b41b06c16 diff --git a/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 b/sql/hive/src/test/resources/golden/join_filters-48-e884480a0f7273d3e2f2de2ba46b855c similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 rename to sql/hive/src/test/resources/golden/join_filters-48-e884480a0f7273d3e2f2de2ba46b855c diff --git a/sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 b/sql/hive/src/test/resources/golden/join_filters-49-98fd86aea9cacaa82d43c7468109dd33 similarity index 100% rename from sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 rename to sql/hive/src/test/resources/golden/join_filters-49-98fd86aea9cacaa82d43c7468109dd33 diff --git a/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e +++ b/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 +++ b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac +++ b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 +++ b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 +++ b/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba +++ b/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 +++ b/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 +++ b/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba +++ b/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 +++ b/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 +++ b/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 +++ b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 +++ b/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 +++ b/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e +++ b/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a +++ b/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 +++ b/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada +++ b/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 +++ b/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 +++ b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 +++ b/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 +++ b/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d +++ b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e +++ b/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 +++ b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d +++ b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 +++ b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa +++ b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 +++ b/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 +++ b/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e +++ b/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 index 51a29d71030ee..a8dc3bebed62c 100644 --- a/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 +++ b/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 @@ -1,4 +1,4 @@ NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf +++ b/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 +++ b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 +++ b/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 +++ b/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 index 759113b3f1c96..0fd19b648c185 100644 --- a/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 +++ b/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 +++ b/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f +++ b/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf +++ b/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e index 24a0c79d42923..3414d86363093 100644 --- a/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e +++ b/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e @@ -4,4 +4,4 @@ NULL NULL 48 NULL NULL 40 NULL NULL 12 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e +++ b/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f +++ b/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 +++ b/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa +++ b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 +++ b/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca index ea646edf6d143..04ecda59b21bd 100644 --- a/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca +++ b/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca @@ -1,4 +1,4 @@ NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL NULL NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 index d735f349a9542..db3f1d99ec261 100644 --- a/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 +++ b/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 @@ -1,4 +1,4 @@ NULL NULL NULL NULL NULL 40 NULL NULL NULL NULL 12 35 NULL NULL NULL NULL 48 NULL -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 +++ b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 +++ b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf +++ b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f +++ b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab +++ b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 +++ b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 +++ b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce b/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 b/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 b/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 b/sql/hive/src/test/resources/golden/join_hive_626-3-4a2f2f2858540afea9a195b5322941ee similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 rename to sql/hive/src/test/resources/golden/join_hive_626-3-4a2f2f2858540afea9a195b5322941ee diff --git a/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 b/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 b/sql/hive/src/test/resources/golden/join_hive_626-4-4bb73b33747da4ed852df381b7b45a71 similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 rename to sql/hive/src/test/resources/golden/join_hive_626-4-4bb73b33747da4ed852df381b7b45a71 diff --git a/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 b/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 rename to sql/hive/src/test/resources/golden/join_hive_626-5-c6a3ae6f3539ab48b996060fb51d8ebe diff --git a/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 index e3143e49c2bad..8b00ef084005b 100644 --- a/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 +++ b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 @@ -1 +1 @@ -foo1 bar10 2 \ No newline at end of file +foo1 bar10 2 diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa index 4c88927e56579..120056ea10c60 100644 --- a/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa index 4c88927e56579..120056ea10c60 100644 --- a/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa +++ b/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa @@ -104,4 +104,4 @@ 98 val_98 val_98 98 val_98 val_98 98 val_98 val_98 -98 val_98 val_98 \ No newline at end of file +98 val_98 val_98 diff --git a/sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 rename to sql/hive/src/test/resources/golden/join_nulls-1-75b1f5331b62fedb7dbbe6ac93a3c83f diff --git a/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b b/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b index 4bc2d3969f17f..610f43b2ac6dc 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b +++ b/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b @@ -1,3 +1,3 @@ NULL 35 NULL 35 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 b/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 index 8414cfbede40a..19621e45f3758 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 +++ b/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 b/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 index c11ae6cce7abb..b512581d48dfe 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 +++ b/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d b/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d index e28e1b1e5e8b1..ee5b5e1ec3096 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d +++ b/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d @@ -1,3 +1,3 @@ NULL NULL NULL 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 b/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 index 85192b86801e2..9a862c2d08e7c 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 +++ b/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 @@ -1,3 +1,3 @@ NULL NULL NULL 35 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 b/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 index c19237d5d4309..5d0c8eb89e421 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 +++ b/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 @@ -1,3 +1,3 @@ NULL NULL 48 NULL NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 b/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 index e28e1b1e5e8b1..ee5b5e1ec3096 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 +++ b/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 @@ -1,3 +1,3 @@ NULL NULL NULL 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f b/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f index 56ca29ffe263c..25db723b06378 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f +++ b/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f @@ -2,4 +2,4 @@ NULL NULL NULL 35 NULL NULL 48 NULL NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 b/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 index 92e228d381628..810287110dc40 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 +++ b/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 @@ -1,4 +1,4 @@ NULL NULL NULL 35 NULL 35 NULL NULL 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c b/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c index f57bbd5152852..115f8d9531800 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c +++ b/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c @@ -1,4 +1,4 @@ NULL NULL 48 NULL NULL 35 NULL 35 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 b/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 +++ b/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 b/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 index 56ca29ffe263c..25db723b06378 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 +++ b/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 @@ -2,4 +2,4 @@ NULL NULL NULL 35 NULL NULL 48 NULL NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e b/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e index 0db9ffd61a3ad..9bca4244a2eb4 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e +++ b/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e @@ -1,3 +1,3 @@ NULL NULL NULL NULL 48 NULL NULL 35 NULL 35 NULL 35 -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 b/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 index ed1b35fbe80ae..f7ff69b919598 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 +++ b/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 @@ -1,3 +1,3 @@ NULL NULL 48 NULL NULL NULL NULL 35 NULL 35 NULL 35 -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 b/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 index 0db9ffd61a3ad..9bca4244a2eb4 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 +++ b/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 @@ -1,3 +1,3 @@ NULL NULL NULL NULL 48 NULL NULL 35 NULL 35 NULL 35 -100 100 100 100 100 100 \ No newline at end of file +100 100 100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 b/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 +++ b/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 +++ b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 b/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 index bdd09da460bbe..309eec4c7d309 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 +++ b/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 @@ -1,2 +1,2 @@ 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 b/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 index dfe4f4318fc66..96e8fb0241578 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 +++ b/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 @@ -1,2 +1,2 @@ NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb +++ b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb b/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb index bdd09da460bbe..309eec4c7d309 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb +++ b/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb @@ -1,2 +1,2 @@ 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 b/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 +++ b/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 b/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 index dfe4f4318fc66..96e8fb0241578 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 +++ b/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 @@ -1,2 +1,2 @@ NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 +++ b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 b/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 index c11ae6cce7abb..b512581d48dfe 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 +++ b/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 b/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 index 8414cfbede40a..19621e45f3758 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 +++ b/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 b/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 index 4bc2d3969f17f..610f43b2ac6dc 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 +++ b/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 @@ -1,3 +1,3 @@ NULL 35 NULL 35 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 b/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 index e28e1b1e5e8b1..ee5b5e1ec3096 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 +++ b/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 @@ -1,3 +1,3 @@ NULL NULL NULL 35 NULL NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 b/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 index 85192b86801e2..9a862c2d08e7c 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 +++ b/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 @@ -1,3 +1,3 @@ NULL NULL NULL 35 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc b/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc index c19237d5d4309..5d0c8eb89e421 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc +++ b/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc @@ -1,3 +1,3 @@ NULL NULL 48 NULL NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 b/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 index 0d14bf4e38509..bb1fee6e62e99 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 +++ b/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 @@ -6,4 +6,4 @@ NULL 35 100 100 48 NULL 100 100 100 100 NULL 35 100 100 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 b/sql/hive/src/test/resources/golden/join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe similarity index 100% rename from sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 rename to sql/hive/src/test/resources/golden/join_nulls-40-268d8fb3cb9b04eb269fe7ec40a24dfe diff --git a/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e b/sql/hive/src/test/resources/golden/join_nulls-41-6dc6866a65c74d69538b776b41b06c16 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e rename to sql/hive/src/test/resources/golden/join_nulls-41-6dc6866a65c74d69538b776b41b06c16 diff --git a/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe b/sql/hive/src/test/resources/golden/join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe rename to sql/hive/src/test/resources/golden/join_nulls-42-e884480a0f7273d3e2f2de2ba46b855c diff --git a/sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 b/sql/hive/src/test/resources/golden/join_nulls-43-98fd86aea9cacaa82d43c7468109dd33 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 rename to sql/hive/src/test/resources/golden/join_nulls-43-98fd86aea9cacaa82d43c7468109dd33 diff --git a/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 +++ b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 +++ b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 +++ b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 b/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 index 7d9efd15fa287..c823f539ee328 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 +++ b/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 @@ -1,4 +1,4 @@ 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 b/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 +++ b/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 b/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 index 4d2ff6a237416..2b06d1c29d984 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 +++ b/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 @@ -3,4 +3,4 @@ NULL NULL NULL 135 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c +++ b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 b/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 index 7d9efd15fa287..c823f539ee328 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 +++ b/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 @@ -1,4 +1,4 @@ 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f b/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f index 695bb1544ec89..06f37e109247c 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f +++ b/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL 48 NULL 100 100 100 100 148 NULL 148 NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 b/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 +++ b/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 b/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 index b89f5e6d42a0f..340baa63de2f7 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 +++ b/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 @@ -1,2 +1,2 @@ 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 b/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 index b81820f886894..1ef56d9d1d051 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 +++ b/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 @@ -3,4 +3,4 @@ NULL 135 NULL NULL 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 b/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 index 71cd5ce0041b8..768ebfc0854e4 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 +++ b/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 @@ -3,4 +3,4 @@ NULL NULL NULL 135 NULL NULL 48 NULL NULL NULL 148 NULL 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 b/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 index 86b3dab6bffc8..88b1f5331b84e 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 +++ b/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 @@ -1,4 +1,4 @@ NULL 35 NULL 35 NULL 135 NULL 135 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 b/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 index 169df31887b96..b060f40ffdf48 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 +++ b/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 @@ -3,4 +3,4 @@ NULL NULL 148 NULL NULL 35 NULL 35 NULL 135 NULL 135 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a b/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a index 86b3dab6bffc8..88b1f5331b84e 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a +++ b/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a @@ -1,4 +1,4 @@ NULL 35 NULL 35 NULL 135 NULL 135 100 100 100 100 -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 b/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 index bdd5dcf63fd00..1256e9451914b 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 +++ b/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 @@ -3,4 +3,4 @@ NULL 135 NULL 135 48 NULL NULL NULL 100 100 100 100 148 NULL NULL NULL -200 200 200 200 \ No newline at end of file +200 200 200 200 diff --git a/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 b/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 index bdd09da460bbe..309eec4c7d309 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 +++ b/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 @@ -1,2 +1,2 @@ 48 NULL 48 NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 b/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 index dfe4f4318fc66..96e8fb0241578 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 +++ b/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 @@ -1,2 +1,2 @@ NULL 35 NULL 35 -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 index 6dc85dd550540..46a059f07aca6 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 +++ b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 @@ -1 +1 @@ -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b b/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b index c11ae6cce7abb..b512581d48dfe 100644 --- a/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b +++ b/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b @@ -1,3 +1,3 @@ NULL 35 NULL NULL 48 NULL NULL NULL -100 100 100 100 \ No newline at end of file +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 rename to sql/hive/src/test/resources/golden/join_nullsafe-2-5bb63fafa390b1d4c20e225a8a648dcf diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 b/sql/hive/src/test/resources/golden/join_nullsafe-2-793e288c9e0971f0bf3f37493f76dc7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0 similarity index 100% rename from sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 rename to sql/hive/src/test/resources/golden/join_nullsafe-20-88faf8a93ba6759bd6f2bbcbdcfecda0 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 b/sql/hive/src/test/resources/golden/join_nullsafe-20-d6fc260320c577eec9a5db0d4135d224 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_nullsafe-21-24332b9390108fb3379e1acc599293a1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/join_nullsafe-21-24332b9390108fb3379e1acc599293a1 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e b/sql/hive/src/test/resources/golden/join_nullsafe-21-a60dae725ffc543f805242611d99de4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 b/sql/hive/src/test/resources/golden/join_nullsafe-22-24c80d0f9e3d72c48d947770fa184985 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/join_nullsafe-22-4be80634a6bd916e3ebd60a124f0a48e diff --git a/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 b/sql/hive/src/test/resources/golden/join_nullsafe-23-3fe6ae20cab3417759dcc654a3a26746 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/join_nullsafe-23-e4425d56be43c21124d95160653ce0ac similarity index 100% rename from sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/join_nullsafe-23-e4425d56be43c21124d95160653ce0ac diff --git a/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 b/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 index 59821aeea3008..c9c79b85dfe24 100644 --- a/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 +++ b/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 b/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 +++ b/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder2-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_reorder2-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_reorder2-5-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/join_reorder2-6-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/join_reorder2-7-5f4cfbbe53c5e808ee08b26514272034 diff --git a/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 b/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 +++ b/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 b/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 +++ b/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/join_reorder3-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/join_reorder3-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_reorder3-5-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/join_reorder3-6-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/join_reorder3-7-5f4cfbbe53c5e808ee08b26514272034 diff --git a/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 b/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 index 06a83e95d4f79..33e6a1546ca65 100644 --- a/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 +++ b/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 @@ -1 +1 @@ -2 12 2 22 2 12 2 12 \ No newline at end of file +2 12 2 22 2 12 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f +++ b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 +++ b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/join_reorder4-3-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/join_reorder4-3-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/join_reorder4-4-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 b/sql/hive/src/test/resources/golden/join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 rename to sql/hive/src/test/resources/golden/join_reorder4-5-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 +++ b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e b/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 b/sql/hive/src/test/resources/golden/join_star-10-a9e579038e3d4826fdae475d7058ab82 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 rename to sql/hive/src/test/resources/golden/join_star-10-a9e579038e3d4826fdae475d7058ab82 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 b/sql/hive/src/test/resources/golden/join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 rename to sql/hive/src/test/resources/golden/join_star-11-72730ecdad9c0fd4c6ce64a0cb89fb74 diff --git a/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae b/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 b/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 b/sql/hive/src/test/resources/golden/join_star-12-f581d6d305d652cd0f4e4fa912eb578d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 rename to sql/hive/src/test/resources/golden/join_star-12-f581d6d305d652cd0f4e4fa912eb578d diff --git a/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb b/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 b/sql/hive/src/test/resources/golden/join_star-13-7268564732cbb7489248f9d818f80c14 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 rename to sql/hive/src/test/resources/golden/join_star-13-7268564732cbb7489248f9d818f80c14 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 b/sql/hive/src/test/resources/golden/join_star-14-2ee0fcf000f8687fc8941bf212477e57 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 rename to sql/hive/src/test/resources/golden/join_star-14-2ee0fcf000f8687fc8941bf212477e57 diff --git a/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 b/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 b/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 b/sql/hive/src/test/resources/golden/join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 rename to sql/hive/src/test/resources/golden/join_star-15-43b0b3b5e40044f8dbaeef2c7fc9e3e9 diff --git a/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 +++ b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 b/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 index e60bca6269264..f97ee71bc2333 100644 --- a/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 +++ b/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 @@ -7,4 +7,4 @@ 61 62 3 71 72 3 81 82 3 -91 92 3 \ No newline at end of file +91 92 3 diff --git a/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b b/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b index e08d04997c56b..e56cb5c03eaa4 100644 --- a/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b +++ b/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b @@ -7,4 +7,4 @@ 61 62 3 4 71 72 3 4 81 82 3 4 -91 92 3 4 \ No newline at end of file +91 92 3 4 diff --git a/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 b/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 index b89911131bd13..80924380b7d76 100644 --- a/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 +++ b/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 @@ -7,4 +7,4 @@ 61 62 3 3 71 72 3 3 81 82 3 3 -91 92 3 3 \ No newline at end of file +91 92 3 3 diff --git a/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 b/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 index b89911131bd13..80924380b7d76 100644 --- a/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 +++ b/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 @@ -7,4 +7,4 @@ 61 62 3 3 71 72 3 3 81 82 3 3 -91 92 3 3 \ No newline at end of file +91 92 3 3 diff --git a/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 b/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 index 19611e75e33c3..ebfbb5cdae331 100644 --- a/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 +++ b/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 @@ -7,4 +7,4 @@ 61 62 3 3 4 4 4 4 4 71 72 3 3 4 4 4 4 4 81 82 3 3 4 4 4 4 4 -91 92 3 3 4 4 4 4 4 \ No newline at end of file +91 92 3 3 4 4 4 4 4 diff --git a/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 b/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/join_star-8-c3d53a4daab9614a09870dc8e9571f74 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/join_star-8-c3d53a4daab9614a09870dc8e9571f74 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/join_star-9-3f7ccccc2488de5f33a38cb3cc3eb628 diff --git a/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 b/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 b/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 deleted file mode 100644 index 07433297e6de1..0000000000000 --- a/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 +++ /dev/null @@ -1,6 +0,0 @@ -aint int from deserializer -astring string from deserializer -lint array from deserializer -lstring array from deserializer -lintstring array> from deserializer -mstringstring map from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db b/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b b/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b deleted file mode 100644 index 4aeaf4ff1f1b0..0000000000000 --- a/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b +++ /dev/null @@ -1,11 +0,0 @@ -1712634731 [{"myint":0,"mystring":"0","underscore_int":0}] -465985200 [{"myint":1,"mystring":"1","underscore_int":1}] --751827638 [{"myint":4,"mystring":"8","underscore_int":2}] -477111222 [{"myint":9,"mystring":"27","underscore_int":3}] --734328909 [{"myint":16,"mystring":"64","underscore_int":4}] --1952710710 [{"myint":25,"mystring":"125","underscore_int":5}] -1244525190 [{"myint":36,"mystring":"216","underscore_int":6}] --1461153973 [{"myint":49,"mystring":"343","underscore_int":7}] -1638581578 [{"myint":64,"mystring":"512","underscore_int":8}] -336964413 [{"myint":81,"mystring":"729","underscore_int":9}] -0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 b/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 b/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b b/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 b/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef b/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 b/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 b/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 deleted file mode 100644 index 0da0d93886e01..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 +++ /dev/null @@ -1,2 +0,0 @@ -key1 100 -key2 200 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a b/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 b/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 deleted file mode 100644 index 0da0d93886e01..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 +++ /dev/null @@ -1,2 +0,0 @@ -key1 100 -key2 200 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe b/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 b/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 b/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa b/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa deleted file mode 100644 index a2a7fdd233a2a..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa +++ /dev/null @@ -1,10 +0,0 @@ -238 val_238 NULL -86 val_86 NULL -311 val_311 NULL -27 val_27 NULL -165 val_165 NULL -409 val_409 NULL -255 val_255 NULL -278 val_278 NULL -98 val_98 NULL -484 val_484 NULL diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 b/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d b/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d deleted file mode 100644 index 37d918a386d7d..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d +++ /dev/null @@ -1,10 +0,0 @@ -238 val_238 4 -238 val_238 5 -86 val_86 4 -86 val_86 5 -311 val_311 4 -311 val_311 5 -27 val_27 4 -27 val_27 5 -165 val_165 4 -165 val_165 5 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e b/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 b/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 b/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 deleted file mode 100644 index 3ad56f646ed85..0000000000000 --- a/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 +++ /dev/null @@ -1,10 +0,0 @@ -238 NULL NULL -86 NULL NULL -311 ["val_311","val_311"] val_311 -311 ["val_311","val_311"] val_311 -27 NULL NULL -165 NULL NULL -409 ["val_409","val_409"] val_409 -409 ["val_409","val_409"] val_409 -255 NULL NULL -278 NULL NULL diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b b/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b new file mode 100644 index 0000000000000..cbeb8081f06c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view_ppd-10-a537ad7282d1c9957cdae74ad87c790b @@ -0,0 +1,6 @@ +val_0 2 +val_0 3 +val_0 2 +val_0 3 +val_0 2 +val_0 3 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/lateral_view_ppd-9-dc6fea663d875b082d38bd326d21cd95 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b b/sql/hive/src/test/resources/golden/leftsemijoin-5-9c307c0559d735960ce77efa95b2b17b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/leftsemijoin-5-aba449db0d4fe6dc9771426e102bb543 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c b/sql/hive/src/test/resources/golden/leftsemijoin-6-82921fc96eef547ec0f71027ee88298c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/leftsemijoin-6-9f50dce576b019c0be997055b8876621 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/leftsemijoin-6-9f50dce576b019c0be997055b8876621 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 b/sql/hive/src/test/resources/golden/leftsemijoin-7-b30aa3b4a45db6b64bb46b4d9bd32ff0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/leftsemijoin-7-fff6ca40e6048d52dc2d3afc68e8353e diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-5b2e555868faa404ea09928936178181 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-1-5b2e555868faa404ea09928936178181 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593 b/sql/hive/src/test/resources/golden/leftsemijoin_mr-1-aa3f07f028027ffd13ab5535dc821593 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 b/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-645cf8b871c9b27418d6fa1d1bda9a52 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/leftsemijoin_mr-3-c019cb2a855138da0d0b1e5c67cd6354 diff --git a/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 index bf0d87ab1b2b0..b8626c4cff284 100644 --- a/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 +++ b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 @@ -1 +1 @@ -4 \ No newline at end of file +4 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 b/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f b/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f deleted file mode 100644 index 9e3b31ad52c13..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f +++ /dev/null @@ -1,20 +0,0 @@ -0 -2 -4 -5 -8 -9 -10 -11 -12 -15 -17 -18 -19 -20 -24 -26 -27 -28 -30 -33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 b/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 b/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 deleted file mode 100644 index ebf6c0424c26d..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 +++ /dev/null @@ -1,20 +0,0 @@ -0 1 -2 1 -4 1 -5 1 -8 1 -9 1 -10 1 -11 1 -12 1 -15 1 -17 1 -18 1 -19 1 -20 1 -24 1 -26 1 -27 1 -28 1 -30 1 -33 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 b/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 b/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 b/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 b/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 deleted file mode 100644 index 153dcec21bc5b..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 +++ /dev/null @@ -1,20 +0,0 @@ -val_0 0 -val_2 2 -val_4 4 -val_8 8 -val_9 9 -val_10 10 -val_11 11 -val_5 15 -val_17 17 -val_19 19 -val_20 20 -val_12 24 -val_27 27 -val_28 28 -val_30 30 -val_15 30 -val_33 33 -val_34 34 -val_18 36 -val_41 41 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 b/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 b/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb b/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 b/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 deleted file mode 100644 index ae8f0265b71ca..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 +++ /dev/null @@ -1,20 +0,0 @@ -val_0 0 -val_10 10 -val_100 200 -val_103 206 -val_104 208 -val_105 105 -val_11 11 -val_111 111 -val_113 226 -val_114 114 -val_116 116 -val_118 236 -val_119 357 -val_12 24 -val_120 240 -val_125 250 -val_126 126 -val_128 384 -val_129 258 -val_131 131 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e b/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a b/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 b/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 deleted file mode 100644 index e604892422d59..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 +++ /dev/null @@ -1,100 +0,0 @@ -0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 -0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 -0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 val_0 -2 val_2 val_2 val_2 val_2 val_2 val_2 val_2 val_2 -4 val_4 val_4 val_4 val_4 val_4 val_4 val_4 val_4 -5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 -5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 -5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 val_5 -8 val_8 val_8 val_8 val_8 val_8 val_8 val_8 val_8 -9 val_9 val_9 val_9 val_9 val_9 val_9 val_9 val_9 -10 val_10 val_10 val_10 val_10 val_10 val_10 val_10 val_10 -11 val_11 val_11 val_11 val_11 val_11 val_11 val_11 val_11 -12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 -12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 val_12 -15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 -15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 val_15 -17 val_17 val_17 val_17 val_17 val_17 val_17 val_17 val_17 -18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 -18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 val_18 -19 val_19 val_19 val_19 val_19 val_19 val_19 val_19 val_19 -20 val_20 val_20 val_20 val_20 val_20 val_20 val_20 val_20 -24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 -24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 val_24 -26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 -26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 val_26 -27 val_27 val_27 val_27 val_27 val_27 val_27 val_27 val_27 -28 val_28 val_28 val_28 val_28 val_28 val_28 val_28 val_28 -30 val_30 val_30 val_30 val_30 val_30 val_30 val_30 val_30 -33 val_33 val_33 val_33 val_33 val_33 val_33 val_33 val_33 -34 val_34 val_34 val_34 val_34 val_34 val_34 val_34 val_34 -35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 -35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 -35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 val_35 -37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 -37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 val_37 -41 val_41 val_41 val_41 val_41 val_41 val_41 val_41 val_41 -42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 -42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 val_42 -43 val_43 val_43 val_43 val_43 val_43 val_43 val_43 val_43 -44 val_44 val_44 val_44 val_44 val_44 val_44 val_44 val_44 -47 val_47 val_47 val_47 val_47 val_47 val_47 val_47 val_47 -51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 -51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 val_51 -53 val_53 val_53 val_53 val_53 val_53 val_53 val_53 val_53 -54 val_54 val_54 val_54 val_54 val_54 val_54 val_54 val_54 -57 val_57 val_57 val_57 val_57 val_57 val_57 val_57 val_57 -58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 -58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 val_58 -64 val_64 val_64 val_64 val_64 val_64 val_64 val_64 val_64 -65 val_65 val_65 val_65 val_65 val_65 val_65 val_65 val_65 -66 val_66 val_66 val_66 val_66 val_66 val_66 val_66 val_66 -67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 -67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 val_67 -69 val_69 val_69 val_69 val_69 val_69 val_69 val_69 val_69 -70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 -70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 -70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 val_70 -72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 -72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 val_72 -74 val_74 val_74 val_74 val_74 val_74 val_74 val_74 val_74 -76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 -76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 val_76 -77 val_77 val_77 val_77 val_77 val_77 val_77 val_77 val_77 -78 val_78 val_78 val_78 val_78 val_78 val_78 val_78 val_78 -80 val_80 val_80 val_80 val_80 val_80 val_80 val_80 val_80 -82 val_82 val_82 val_82 val_82 val_82 val_82 val_82 val_82 -83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 -83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 val_83 -84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 -84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 val_84 -85 val_85 val_85 val_85 val_85 val_85 val_85 val_85 val_85 -86 val_86 val_86 val_86 val_86 val_86 val_86 val_86 val_86 -87 val_87 val_87 val_87 val_87 val_87 val_87 val_87 val_87 -90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 -90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 -90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 val_90 -92 val_92 val_92 val_92 val_92 val_92 val_92 val_92 val_92 -95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 -95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 val_95 -96 val_96 val_96 val_96 val_96 val_96 val_96 val_96 val_96 -97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 -97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 val_97 -98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 -98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 val_98 -100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 -100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 val_100 -103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 -103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 val_103 -104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 -104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 val_104 -105 val_105 val_105 val_105 val_105 val_105 val_105 val_105 val_105 -111 val_111 val_111 val_111 val_111 val_111 val_111 val_111 val_111 -113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 -113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 val_113 -114 val_114 val_114 val_114 val_114 val_114 val_114 val_114 val_114 -116 val_116 val_116 val_116 val_116 val_116 val_116 val_116 val_116 -118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 -118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 val_118 -119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 -119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 val_119 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a b/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c b/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c deleted file mode 100644 index 92dc6ce9dbf9e..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c +++ /dev/null @@ -1,100 +0,0 @@ -0 -200 -206 -208 -105 -10 -111 -226 -114 -116 -236 -357 -11 -240 -250 -126 -384 -258 -24 -131 -133 -268 -136 -274 -552 -143 -145 -292 -298 -150 -304 -153 -155 -156 -157 -158 -30 -160 -162 -163 -328 -330 -166 -501 -168 -676 -170 -344 -348 -350 -352 -177 -178 -358 -17 -180 -181 -183 -186 -561 -189 -36 -190 -382 -192 -579 -194 -390 -196 -394 -597 -19 -400 -201 -202 -406 -410 -414 -624 -418 -20 -426 -214 -432 -434 -218 -438 -442 -222 -446 -448 -226 -228 -458 -1150 -466 -235 -474 -476 -478 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 b/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 deleted file mode 100644 index 95f5492558a9b..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 +++ /dev/null @@ -1,20 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -2 val_2 -4 val_4 -5 val_5 -5 val_5 -5 val_5 -8 val_8 -9 val_9 -10 val_10 -11 val_11 -12 val_12 -12 val_12 -15 val_15 -15 val_15 -17 val_17 -18 val_18 -18 val_18 -19 val_19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f b/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 b/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 deleted file mode 100644 index 0bac402cfa497..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 +++ /dev/null @@ -1,20 +0,0 @@ -498 val_498 -498 val_498 -498 val_498 -497 val_497 -496 val_496 -495 val_495 -494 val_494 -493 val_493 -492 val_492 -492 val_492 -491 val_491 -490 val_490 -489 val_489 -489 val_489 -489 val_489 -489 val_489 -487 val_487 -485 val_485 -484 val_484 -483 val_483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 b/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 b/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 deleted file mode 100644 index b57d0cc951566..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 +++ /dev/null @@ -1,20 +0,0 @@ -val_0 3 -val_10 11 -val_100 202 -val_103 208 -val_104 210 -val_105 106 -val_11 12 -val_111 112 -val_113 228 -val_114 115 -val_116 117 -val_118 238 -val_119 360 -val_12 26 -val_120 242 -val_125 252 -val_126 127 -val_128 387 -val_129 260 -val_131 132 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 b/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a b/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a deleted file mode 100644 index 0e95c446f3b0d..0000000000000 --- a/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a +++ /dev/null @@ -1,20 +0,0 @@ -val_0 1.0 -val_10 11.0 -val_100 101.0 -val_103 104.0 -val_104 105.0 -val_105 106.0 -val_11 12.0 -val_111 112.0 -val_113 114.0 -val_114 115.0 -val_116 117.0 -val_118 119.0 -val_119 120.0 -val_12 13.0 -val_120 121.0 -val_125 126.0 -val_126 127.0 -val_128 129.0 -val_129 130.0 -val_131 132.0 \ No newline at end of file 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 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f b/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/literal_double-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/literal_double-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/literal_double-1-10ef1098e35d900983be3814de8f974f similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/literal_double-1-10ef1098e35d900983be3814de8f974f diff --git a/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 b/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 deleted file mode 100644 index 69505e73feb1d..0000000000000 --- a/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 +++ /dev/null @@ -1 +0,0 @@ -3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 b/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 new file mode 100644 index 0000000000000..24ca45210038a --- /dev/null +++ b/sql/hive/src/test/resources/golden/literal_double-2-3863c17e03c9c1cd68452106a8721d13 @@ -0,0 +1 @@ +3.14 -3.14 3.14E8 3.14E-8 -3.14E8 -3.14E-8 3.14E8 3.14E8 3.14E-8 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/literal_ints-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 b/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/literal_ints-1-5ffd1b49cdda4149aef2c61c53a56890 diff --git a/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 b/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 deleted file mode 100644 index 6dc85dd550540..0000000000000 --- a/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 +++ /dev/null @@ -1 +0,0 @@ -100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 b/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 new file mode 100644 index 0000000000000..46a059f07aca6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/literal_ints-2-b41c42ce2f6ba483b68bb08752b95ec4 @@ -0,0 +1 @@ +100 100 100 100 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/literal_string-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/literal_string-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 b/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d b/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d deleted file mode 100644 index 6323f4efa99c7..0000000000000 --- a/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d +++ /dev/null @@ -1 +0,0 @@ -facebook facebook facebook facebook facebook facebook facebook facebook facebook facebook \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/literal_string-1-9b48d41e5c9e41ddc070e2fd31ace15 diff --git a/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d b/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d new file mode 100644 index 0000000000000..1d05317d62547 --- /dev/null +++ b/sql/hive/src/test/resources/golden/literal_string-2-2cf4b7268b47246afdf6c792acca379d @@ -0,0 +1 @@ +facebook facebook facebook facebook facebook facebook facebook facebook facebook facebook diff --git a/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 b/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc b/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 b/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 b/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca deleted file mode 100644 index de5212a3c320f772b6a3a2b246202cd99d07c012..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 114 zcmXYpK@xx<31zZO;W_B|0{Q<~&`YzM%(UUWxDW~|hJHUCips`;RC1i>>>>>>>>>>>>>>>>>>>>>> None -max_nested_map array>>>>>>>>>>>>>>>>>>>>> None -max_nested_struct array>>>>>>>>>>>>>>>>>>>>>> None -simple_string string None diff --git a/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 b/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 deleted file mode 100644 index 8ce70d8c71782..0000000000000 --- a/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 +++ /dev/null @@ -1,8 +0,0 @@ -simple_int int None -max_nested_array array>>>>>>>>>>>>>>>>>>>>>> None -max_nested_map array>>>>>>>>>>>>>>>>>>>>> None -max_nested_struct array>>>>>>>>>>>>>>>>>>>>>> None -simple_string string None - -Detailed Table Information Table(tableName:nestedcomplex, dbName:default, owner:marmbrus, createTime:1391226936, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:simple_int, type:int, comment:null), FieldSchema(name:max_nested_array, type:array>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_map, type:array>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:max_nested_struct, type:array>>>>>>>>>>>>>>>>>>>>>>, comment:null), FieldSchema(name:simple_string, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/nestedcomplex, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1, line.delim= -, hive.serialization.extend.nesting.levels=true}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1391226936}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 b/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d b/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d deleted file mode 100644 index 228853cffe527..0000000000000 --- a/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d +++ /dev/null @@ -1,2 +0,0 @@ -2 [[[[[[[[[[[[[[[[[[[[[[[0,3,2]]]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[{"k1":"v1","k3":"v3"}]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[[{"s":"b","i":10}]]]]]]]]]]]]]]]]]]]]]] 2 -3 [[[[[[[[[[[[[[[[[[[[[[[0,1,2]]]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[{"k1":"v1","k2":"v2"}]]]]]]]]]]]]]]]]]]]]] [[[[[[[[[[[[[[[[[[[[[[{"s":"a","i":10}]]]]]]]]]]]]]]]]]]]]]] 2 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 b/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 index cdcd12dca3ca2..9c4a8fd131407 100644 --- a/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 +++ b/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 @@ -81,4 +81,4 @@ val_84 val_28 val_37 val_90 -val_97 \ No newline at end of file +val_97 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 index a0cfcf2621f2e..34e8c1127f016 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 @@ -22,4 +22,4 @@ NULL val_484 25 311 val_311 25 369 25 401 val_401 25 -406 val_406 25 \ No newline at end of file +406 val_406 25 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 index a0cfcf2621f2e..34e8c1127f016 100644 --- a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 @@ -22,4 +22,4 @@ NULL val_484 25 311 val_311 25 369 25 401 val_401 25 -406 val_406 25 \ No newline at end of file +406 val_406 25 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 deleted file mode 100644 index 24e19ec6caa8f..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 +++ /dev/null @@ -1 +0,0 @@ --826625916 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 deleted file mode 100644 index 05b0c43f926bd..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 +++ /dev/null @@ -1 +0,0 @@ -10226524244 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 deleted file mode 100644 index bc15b9ca54985..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 +++ /dev/null @@ -1 +0,0 @@ -20453048488 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad deleted file mode 100644 index 08839f6bb296e..0000000000000 --- a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad +++ /dev/null @@ -1 +0,0 @@ -200 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f index fce122031e88f..138522d542895 100644 --- a/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f @@ -54,4 +54,4 @@ 1234 95 2.0 1234 96 1.0 1234 97 2.0 -1234 98 2.0 \ No newline at end of file +1234 98 2.0 diff --git a/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f index fce122031e88f..138522d542895 100644 --- a/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f @@ -54,4 +54,4 @@ 1234 95 2.0 1234 96 1.0 1234 97 2.0 -1234 98 2.0 \ No newline at end of file +1234 98 2.0 diff --git a/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 b/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c b/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c deleted file mode 100644 index d111428eaabb0..0000000000000 --- a/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c +++ /dev/null @@ -1 +0,0 @@ -[null,0] [null,[]] [null,{}] [null,{"col1":0}] diff --git a/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 +++ b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 +++ b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 +++ b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f +++ b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/nullgroup3-1-61ead7f73d525e0d9e21beba91a3d39e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/nullgroup3-12-61ead7f73d525e0d9e21beba91a3d39e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup3-13-a5e12cfbc1799ce9fa9628d81b8c0b06 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 +++ b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-18-113e2b587784d54d2a5b5f091389397e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup3-18-113e2b587784d54d2a5b5f091389397e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-19-77de4b2d65eee228848625cdd422178d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup3-19-77de4b2d65eee228848625cdd422178d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup3-2-a5e12cfbc1799ce9fa9628d81b8c0b06 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 +++ b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 +++ b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup3-6-113e2b587784d54d2a5b5f091389397e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup3-6-113e2b587784d54d2a5b5f091389397e diff --git a/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup3-7-77de4b2d65eee228848625cdd422178d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup3-7-77de4b2d65eee228848625cdd422178d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 +++ b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/nullgroup5-1-642e12a05bf01a6716bfa15ed0012629 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 rename to sql/hive/src/test/resources/golden/nullgroup5-1-642e12a05bf01a6716bfa15ed0012629 diff --git a/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b b/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/nullgroup5-3-35517978e09aa1bd1d15f51d11e08fd5 diff --git a/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c b/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c b/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c index c5c8d29fdd13e..7aae61e5eb82f 100644 --- a/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c +++ b/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c @@ -497,4 +497,4 @@ 403 val_403 400 val_400 200 val_200 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 +++ b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 b/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/nullscript-1-f07dfd6ef687e038083deca5941d8174 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/nullscript-1-f07dfd6ef687e038083deca5941d8174 diff --git a/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 b/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/nullscript-2-bb1abcf2f4a2a5cd5c058104901627bb diff --git a/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 b/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 index d23e05acf7ba5..e34118512c1d7 100644 --- a/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 +++ b/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 @@ -497,4 +497,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc b/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 b/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 deleted file mode 100644 index da0ddb96957e1..0000000000000 --- a/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL 1 0 0.0 diff --git a/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 7541739b48608..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 b/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 78c0010bebbc6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,34 +0,0 @@ -# col_name data_type comment - -key int from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - last_modified_by marmbrus - last_modified_time 1391226987 - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 6881640dd3cd4..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 b/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 b/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 deleted file mode 100644 index dd4fa77f326ad..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 +++ /dev/null @@ -1,29 +0,0 @@ -# col_name data_type comment - -str string from deserializer -mp map from deserializer -lst array from deserializer -strct struct from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create_complex -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 b/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 b/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 deleted file mode 100644 index 88c8812029d82..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 +++ /dev/null @@ -1,3 +0,0 @@ -line1 {"key11":"value11","key12":"value12","key13":"value13"} ["a","b","c"] {"a":"one","b":"two"} -line2 {"key21":"value21","key22":"value22","key23":"value23"} ["d","e","f"] {"a":"three","b":"four"} -line3 {"key31":"value31","key32":"value32","key33":"value33"} ["g","h","i"] {"a":"five","b":"six"} diff --git a/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 b/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 b/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 deleted file mode 100644 index 67946888f1baf..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 +++ /dev/null @@ -1,3 +0,0 @@ -line1 {"key12":"value12","key11":"value11","key13":"value13"} ["a","b","c"] {"A":"one","B":"two"} -line2 {"key21":"value21","key23":"value23","key22":"value22"} ["d","e","f"] {"A":"three","B":"four"} -line3 {"key33":"value33","key31":"value31","key32":"value32"} ["g","h","i"] {"A":"five","B":"six"} diff --git a/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead b/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead deleted file mode 100644 index 83db48f84ec87..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead +++ /dev/null @@ -1,3 +0,0 @@ -line1 -line2 -line3 diff --git a/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c b/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c deleted file mode 100644 index db4876dd3c809..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c +++ /dev/null @@ -1,3 +0,0 @@ -{"key12":"value12","key11":"value11","key13":"value13"} -{"key21":"value21","key23":"value23","key22":"value22"} -{"key33":"value33","key31":"value31","key32":"value32"} diff --git a/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 b/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 deleted file mode 100644 index 3b2e1cf7e0098..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 +++ /dev/null @@ -1,3 +0,0 @@ -["a","b","c"] -["d","e","f"] -["g","h","i"] diff --git a/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 b/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 deleted file mode 100644 index 0f890dfd226d6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 +++ /dev/null @@ -1,3 +0,0 @@ -{"a":"one","b":"two"} -{"a":"three","b":"four"} -{"a":"five","b":"six"} diff --git a/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 b/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d b/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 b/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 b/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a b/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a deleted file mode 100644 index 0c6f532488607..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a +++ /dev/null @@ -1,10 +0,0 @@ -91 Genevieve Wilkins 908 Turpis. Street -93 Mariko Cline P.O. Box 329, 5375 Ac St. -95 Winifred Hopper Ap #140-8982 Velit Avenue -97 Dana Carter 814-601 Purus. Av. -99 Wynter Vincent 626-8492 Mollis Avenue -92 Thane Oneil 6766 Lectus St. -94 Lael Mclean 500-7010 Sit St. -96 Rafael England P.O. Box 405, 7857 Eget Av. -98 Juliet Battle Ap #535-1965 Cursus St. -100 Wang Mitchell 4023 Lacinia. Ave diff --git a/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 b/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 b/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 deleted file mode 100644 index ecd0704ce3386..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -str string None -mp map None -lst array None -strct struct None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create_staging -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226986 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - colelction.delim , - field.delim | - mapkey.delim : - serialization.format | diff --git a/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 b/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 deleted file mode 100644 index 6881640dd3cd4..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Fri Jan 31 19:56:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse7445586986532881162/orc_create -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1391226987 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde -InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d index f799fca27c353..5bba4beaf3355 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d @@ -7,4 +7,4 @@ 255 278 98 -484 \ No newline at end of file +484 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-6ced6a6f5189c7a315d92ebf3dcc68d3 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 index f953881c894c1..0b6d77d2e825b 100644 --- a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 @@ -1 +1 @@ -1082202951192 \ No newline at end of file +1082202951192 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e b/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 deleted file mode 100644 index f172be3e72712..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 +++ /dev/null @@ -1,10 +0,0 @@ -0 3 2 -165 NULL 1 -2 1 2 -238 NULL 1 -27 NULL 1 -311 NULL 1 -4 1 2 -5 3 2 -8 1 2 -86 NULL 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf b/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 b/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 deleted file mode 100644 index e3c0074c9d4f6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 +++ /dev/null @@ -1,9 +0,0 @@ - - - - - - - - - diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 b/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e b/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e deleted file mode 100644 index 4cd1242d9fe61..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e +++ /dev/null @@ -1,10 +0,0 @@ - - - - - - - -165 -255 -27 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f deleted file mode 100644 index e7700b8aee5c6..0000000000000 --- a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f +++ /dev/null @@ -1,5 +0,0 @@ -1 -NULL -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 b/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 index 217a1915f8826..4dba25230ed1b 100644 --- a/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 +++ b/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 @@ -7,4 +7,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 b/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 index ab34c87e42364..fde2ef3e83ce5 100644 --- a/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 +++ b/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 @@ -7,4 +7,4 @@ 494 val_494 493 val_493 492 val_492 -492 val_492 \ No newline at end of file +492 val_492 diff --git a/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f b/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f index 217a1915f8826..4dba25230ed1b 100644 --- a/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f +++ b/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f @@ -7,4 +7,4 @@ 5 val_5 5 val_5 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 b/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 index 549da558b18da..83a26e92a694b 100644 --- a/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 +++ b/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 19 val_19 19 val_19 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 b/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 index 549da558b18da..83a26e92a694b 100644 --- a/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 +++ b/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 19 val_19 19 val_19 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 +++ b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 +++ b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a +++ b/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 +++ b/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a +++ b/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 index c912af4a5c676..5710fb29e7e02 100644 --- a/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 +++ b/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 @@ -306,4 +306,4 @@ 495 val_495 496 val_496 497 val_497 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c +++ b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d b/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 b/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 deleted file mode 100644 index b431d3fc6dcf6..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Apr 29 20:55:07 PDT 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 48 - rawDataSize 512 - totalSize 560 - transient_lastDdlTime 1398830107 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 deleted file mode 100644 index 3b733e2d6c451..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 +++ /dev/null @@ -1,48 +0,0 @@ -128 val_128 -128 val_128 -150 val_150 -150 val_150 -165 val_165 -165 val_165 -193 val_193 -193 val_193 -213 val_213 -213 val_213 -213 val_213 -213 val_213 -213 val_214 -213 val_214 -224 val_224 -224 val_224 -238 val_238 -238 val_238 -238 val_239 -238 val_239 -238 val_240 -238 val_240 -255 val_255 -255 val_255 -265 val_265 -265 val_265 -27 val_27 -27 val_27 -273 val_273 -273 val_273 -278 val_278 -278 val_278 -311 val_311 -311 val_311 -369 val_369 -369 val_369 -401 val_401 -401 val_401 -409 val_409 -409 val_409 -484 val_484 -484 val_484 -66 val_66 -66 val_66 -86 val_86 -86 val_86 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f b/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 deleted file mode 100644 index 7f67251e61787..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Apr 29 20:54:55 PDT 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 48 - rawDataSize 512 - totalSize 560 - transient_lastDdlTime 1398830095 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 deleted file mode 100644 index 3b733e2d6c451..0000000000000 --- a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 +++ /dev/null @@ -1,48 +0,0 @@ -128 val_128 -128 val_128 -150 val_150 -150 val_150 -165 val_165 -165 val_165 -193 val_193 -193 val_193 -213 val_213 -213 val_213 -213 val_213 -213 val_213 -213 val_214 -213 val_214 -224 val_224 -224 val_224 -238 val_238 -238 val_238 -238 val_239 -238 val_239 -238 val_240 -238 val_240 -255 val_255 -255 val_255 -265 val_265 -265 val_265 -27 val_27 -27 val_27 -273 val_273 -273 val_273 -278 val_278 -278 val_278 -311 val_311 -311 val_311 -369 val_369 -369 val_369 -401 val_401 -401 val_401 -409 val_409 -409 val_409 -484 val_484 -484 val_484 -66 val_66 -66 val_66 -86 val_86 -86 val_86 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 b/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 b/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc b/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd b/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec b/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 b/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 b/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea b/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea +++ b/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 b/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b b/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 b/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 index ffa0de8a0c322..a98dc8e8e1b86 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -1,24 +1,24 @@ # col_name data_type comment -c1 tinyint None +c1 tinyint # Partition Information # col_name data_type comment -c2 string None +c2 string # Detailed Partition Information Partition Value: [v1] Database: default Table: mytbl -CreateTime: Fri Jan 03 17:18:52 PST 2014 +CreateTime: Tue Oct 21 04:00:26 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1 Partition Parameters: a myval b yourval - transient_lastDdlTime 1388798332 + transient_lastDdlTime 1413889226 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -29,4 +29,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 index 0909bab11a6ec..2bef99dafeb29 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -1,22 +1,22 @@ # col_name data_type comment -c1 tinyint None +c1 tinyint # Partition Information # col_name data_type comment -c2 string None +c2 string # Detailed Partition Information Partition Value: [v1] Database: default Table: mytbl -CreateTime: Fri Jan 03 17:18:59 PST 2014 +CreateTime: Tue Oct 21 04:00:27 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1 Partition Parameters: - transient_lastDdlTime 1388798339 + transient_lastDdlTime 1413889227 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -27,4 +27,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 index 55fb6df62f0a7..7b6b90f7f4200 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 @@ -1,25 +1,25 @@ # col_name data_type comment -c1 tinyint None +c1 tinyint # Partition Information # col_name data_type comment -c2 string None +c2 string # Detailed Partition Information Partition Value: [v1] Database: default Table: mytbl -CreateTime: Fri Jan 03 17:19:02 PST 2014 +CreateTime: Tue Oct 21 04:00:28 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/mytbl/c2=v1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/mytbl/c2=v1 Partition Parameters: a myval b yourval c noval - transient_lastDdlTime 1388798342 + transient_lastDdlTime 1413889228 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -30,4 +30,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 b/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partcols1-1-b562ff3e36de23a686d122967a1f91c8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/partcols1-1-b562ff3e36de23a686d122967a1f91c8 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/partition_date-1-6b0952309c3ebdd6dcb7066891d1bd74 diff --git a/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f b/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-10-aad6078a09b7bd8f5141437e86bb229f @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 b/sql/hive/src/test/resources/golden/partition_date-11-a01c3791f59e819c750e213b6c65d084 similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 rename to sql/hive/src/test/resources/golden/partition_date-11-a01c3791f59e819c750e213b6c65d084 diff --git a/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 b/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 new file mode 100644 index 0000000000000..b4de394767536 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-12-2662a237c86cf9e4a150a4f1856b8239 @@ -0,0 +1 @@ +11 diff --git a/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 b/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 new file mode 100644 index 0000000000000..64bb6b746dcea --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-13-aa513c8ee1cbfd1c94f5772c110d4dc9 @@ -0,0 +1 @@ +30 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/partition_date-14-3f187dad9a2fdfc6f7a3566309016f9c diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/partition_date-15-e4366325f3a0c4a8e92be59f4de73fce similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/partition_date-15-e4366325f3a0c4a8e92be59f4de73fce diff --git a/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac b/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-16-32cf81c1a36451eccb07b20dffd930ac @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 b/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-17-8654528691598a5eef8e3c1059d24117 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 b/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-18-a1e769630ac3caed7325a3a256713b24 @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d b/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-19-95f1399e12124840caee7492c0f3036d @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_date-2-220048240a7050a98411ddbc6b1f82cf similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/partition_date-2-220048240a7050a98411ddbc6b1f82cf diff --git a/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba b/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-20-e734eb4fc8894c9a2b2b9cdac4270fba @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 b/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-21-b60c0a3677591991152a0aa500bdce68 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/partition_date-22-fe277bd0a30e016826d242fd5bd61714 similarity index 100% rename from sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 rename to sql/hive/src/test/resources/golden/partition_date-22-fe277bd0a30e016826d242fd5bd61714 diff --git a/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 b/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-23-7f014b494185e4c2a6048efb89131344 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_date-24-7ec1f3a845e2c49191460e15af30aa30 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/partition_date-24-7ec1f3a845e2c49191460e15af30aa30 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_date-3-f8887dd18c21bf0306e293d463b3e1d7 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/partition_date-3-f8887dd18c21bf0306e293d463b3e1d7 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partition_date-4-f7c20c208e052305c1710a5e1e80c5c8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/partition_date-4-f7c20c208e052305c1710a5e1e80c5c8 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_date-5-59355f4e222dcd4f77f51d15bd896f11 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/partition_date-5-59355f4e222dcd4f77f51d15bd896f11 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_date-6-416ab10ac818d432cd3a81870d6e5164 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/partition_date-6-416ab10ac818d432cd3a81870d6e5164 diff --git a/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc b/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc new file mode 100644 index 0000000000000..758a01e987473 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-7-1405c311915f27b0cc616c83d39eaacc @@ -0,0 +1,3 @@ +2000-01-01 +2013-08-08 +2013-12-10 diff --git a/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 b/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 new file mode 100644 index 0000000000000..24192eefd2caf --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-8-7703adfcfdd44c9250f9eba478004714 @@ -0,0 +1,5 @@ +165 val_165 2000-01-01 2 +238 val_238 2000-01-01 2 +27 val_27 2000-01-01 2 +311 val_311 2000-01-01 2 +86 val_86 2000-01-01 2 diff --git a/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 b/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_date-9-a425c11c12c9ce4c9c43d4fbccee5347 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index 81df179618406..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14:18:26 -ts=2011-01-11+15:18:26 -ts=2011-01-11+16:18:26 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index e16053e3110ae..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14%3A18%3A26 -ts=2011-01-11+15%3A18%3A26 -ts=2011-01-11+16%3A18%3A26 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 index 8af58e2b29f4a..c97e50a8a58cd 100644 --- a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 +++ b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 @@ -1,8 +1,8 @@ -key string None -value string None -dt string None +key string +value string +dt string # Partition Information # col_name data_type comment -dt string None \ No newline at end of file +dt string diff --git a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 index 82a07522b0af0..39db984c884ad 100644 --- a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 +++ b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 @@ -1,9 +1,9 @@ -key string None -value string None -x string None -dt string None +key string +value string +x string +dt string # Partition Information # col_name data_type comment -dt string None \ No newline at end of file +dt string diff --git a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 index 8af58e2b29f4a..c97e50a8a58cd 100644 --- a/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 +++ b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 @@ -1,8 +1,8 @@ -key string None -value string None -dt string None +key string +value string +dt string # Partition Information # col_name data_type comment -dt string None \ No newline at end of file +dt string diff --git a/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 b/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 index 95f5492558a9b..7e53c64af58aa 100644 --- a/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 +++ b/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 @@ -17,4 +17,4 @@ 17 val_17 18 val_18 18 val_18 -19 val_19 \ No newline at end of file +19 val_19 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index e16053e3110ae..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14%3A18%3A26 -ts=2011-01-11+15%3A18%3A26 -ts=2011-01-11+16%3A18%3A26 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 deleted file mode 100644 index e16053e3110ae..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 +++ /dev/null @@ -1,3 +0,0 @@ -ts=2011-01-11+14%3A18%3A26 -ts=2011-01-11+15%3A18%3A26 -ts=2011-01-11+16%3A18%3A26 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 deleted file mode 100644 index 00750edc07d64..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 +++ /dev/null @@ -1 +0,0 @@ -3 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_type_check-11-60aad884be613d18d3b89fca3b90dc94 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/partition_type_check-11-60aad884be613d18d3b89fca3b90dc94 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a b/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/partition_type_check-2-3a6bb204d9524299f28adf1cc35d6f4d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f b/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 b/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/partition_type_check-6-663051c7106b8ee6913ca1b007157941 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/partition_type_check-6-663051c7106b8ee6913ca1b007157941 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/partition_varchar1-2-bca5c9edccc3a84e0a9ef92ebcbe746 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 b/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 b/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/partition_varchar1-3-9cec4b1d156f5a9cb587470b98de15 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef b/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/partition_varchar1-4-e9c4530e270db6d44cc54292e4eff680 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/partition_varchar1-5-661aefd18c44c1eb02c2aaf8fab59f73 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce b/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde deleted file mode 100644 index 39c80f1b77eab..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde +++ /dev/null @@ -1,1000 +0,0 @@ -0 val_0 NULL -0 val_0 NULL -0 val_0 NULL -0 val_0 0 -0 val_0 0 -0 val_0 0 -10 val_10 NULL -10 val_10 10 -100 val_100 NULL -100 val_100 NULL -100 val_100 100 -100 val_100 100 -103 val_103 NULL -103 val_103 NULL -103 val_103 103 -103 val_103 103 -104 val_104 NULL -104 val_104 NULL -104 val_104 104 -104 val_104 104 -105 val_105 NULL -105 val_105 105 -11 val_11 NULL -11 val_11 11 -111 val_111 NULL -111 val_111 111 -113 val_113 NULL -113 val_113 NULL -113 val_113 113 -113 val_113 113 -114 val_114 NULL -114 val_114 114 -116 val_116 NULL -116 val_116 116 -118 val_118 NULL -118 val_118 NULL -118 val_118 118 -118 val_118 118 -119 val_119 NULL -119 val_119 NULL -119 val_119 NULL -119 val_119 119 -119 val_119 119 -119 val_119 119 -12 val_12 NULL -12 val_12 NULL -12 val_12 12 -12 val_12 12 -120 val_120 NULL -120 val_120 NULL -120 val_120 120 -120 val_120 120 -125 val_125 NULL -125 val_125 NULL -125 val_125 125 -125 val_125 125 -126 val_126 NULL -126 val_126 126 -128 val_128 NULL -128 val_128 NULL -128 val_128 NULL -128 val_128 128 -128 val_128 128 -128 val_128 128 -129 val_129 NULL -129 val_129 NULL -129 val_129 129 -129 val_129 129 -131 val_131 NULL -131 val_131 131 -133 val_133 NULL -133 val_133 133 -134 val_134 NULL -134 val_134 NULL -134 val_134 134 -134 val_134 134 -136 val_136 NULL -136 val_136 136 -137 val_137 NULL -137 val_137 NULL -137 val_137 137 -137 val_137 137 -138 val_138 NULL -138 val_138 NULL -138 val_138 NULL -138 val_138 NULL -138 val_138 138 -138 val_138 138 -138 val_138 138 -138 val_138 138 -143 val_143 NULL -143 val_143 143 -145 val_145 NULL -145 val_145 145 -146 val_146 NULL -146 val_146 NULL -146 val_146 146 -146 val_146 146 -149 val_149 NULL -149 val_149 NULL -149 val_149 149 -149 val_149 149 -15 val_15 NULL -15 val_15 NULL -15 val_15 15 -15 val_15 15 -150 val_150 NULL -150 val_150 150 -152 val_152 NULL -152 val_152 NULL -152 val_152 152 -152 val_152 152 -153 val_153 NULL -153 val_153 153 -155 val_155 NULL -155 val_155 155 -156 val_156 NULL -156 val_156 156 -157 val_157 NULL -157 val_157 157 -158 val_158 NULL -158 val_158 158 -160 val_160 NULL -160 val_160 160 -162 val_162 NULL -162 val_162 162 -163 val_163 NULL -163 val_163 163 -164 val_164 NULL -164 val_164 NULL -164 val_164 164 -164 val_164 164 -165 val_165 NULL -165 val_165 NULL -165 val_165 165 -165 val_165 165 -166 val_166 NULL -166 val_166 166 -167 val_167 NULL -167 val_167 NULL -167 val_167 NULL -167 val_167 167 -167 val_167 167 -167 val_167 167 -168 val_168 NULL -168 val_168 168 -169 val_169 NULL -169 val_169 NULL -169 val_169 NULL -169 val_169 NULL -169 val_169 169 -169 val_169 169 -169 val_169 169 -169 val_169 169 -17 val_17 NULL -17 val_17 17 -170 val_170 NULL -170 val_170 170 -172 val_172 NULL -172 val_172 NULL -172 val_172 172 -172 val_172 172 -174 val_174 NULL -174 val_174 NULL -174 val_174 174 -174 val_174 174 -175 val_175 NULL -175 val_175 NULL -175 val_175 175 -175 val_175 175 -176 val_176 NULL -176 val_176 NULL -176 val_176 176 -176 val_176 176 -177 val_177 NULL -177 val_177 177 -178 val_178 NULL -178 val_178 178 -179 val_179 NULL -179 val_179 NULL -179 val_179 179 -179 val_179 179 -18 val_18 NULL -18 val_18 NULL -18 val_18 18 -18 val_18 18 -180 val_180 NULL -180 val_180 180 -181 val_181 NULL -181 val_181 181 -183 val_183 NULL -183 val_183 183 -186 val_186 NULL -186 val_186 186 -187 val_187 NULL -187 val_187 NULL -187 val_187 NULL -187 val_187 187 -187 val_187 187 -187 val_187 187 -189 val_189 NULL -189 val_189 189 -19 val_19 NULL -19 val_19 19 -190 val_190 NULL -190 val_190 190 -191 val_191 NULL -191 val_191 NULL -191 val_191 191 -191 val_191 191 -192 val_192 NULL -192 val_192 192 -193 val_193 NULL -193 val_193 NULL -193 val_193 NULL -193 val_193 193 -193 val_193 193 -193 val_193 193 -194 val_194 NULL -194 val_194 194 -195 val_195 NULL -195 val_195 NULL -195 val_195 195 -195 val_195 195 -196 val_196 NULL -196 val_196 196 -197 val_197 NULL -197 val_197 NULL -197 val_197 197 -197 val_197 197 -199 val_199 NULL -199 val_199 NULL -199 val_199 NULL -199 val_199 199 -199 val_199 199 -199 val_199 199 -2 val_2 NULL -2 val_2 2 -20 val_20 NULL -20 val_20 20 -200 val_200 NULL -200 val_200 NULL -200 val_200 200 -200 val_200 200 -201 val_201 NULL -201 val_201 201 -202 val_202 NULL -202 val_202 202 -203 val_203 NULL -203 val_203 NULL -203 val_203 203 -203 val_203 203 -205 val_205 NULL -205 val_205 NULL -205 val_205 205 -205 val_205 205 -207 val_207 NULL -207 val_207 NULL -207 val_207 207 -207 val_207 207 -208 val_208 NULL -208 val_208 NULL -208 val_208 NULL -208 val_208 208 -208 val_208 208 -208 val_208 208 -209 val_209 NULL -209 val_209 NULL -209 val_209 209 -209 val_209 209 -213 val_213 NULL -213 val_213 NULL -213 val_213 213 -213 val_213 213 -214 val_214 NULL -214 val_214 214 -216 val_216 NULL -216 val_216 NULL -216 val_216 216 -216 val_216 216 -217 val_217 NULL -217 val_217 NULL -217 val_217 217 -217 val_217 217 -218 val_218 NULL -218 val_218 218 -219 val_219 NULL -219 val_219 NULL -219 val_219 219 -219 val_219 219 -221 val_221 NULL -221 val_221 NULL -221 val_221 221 -221 val_221 221 -222 val_222 NULL -222 val_222 222 -223 val_223 NULL -223 val_223 NULL -223 val_223 223 -223 val_223 223 -224 val_224 NULL -224 val_224 NULL -224 val_224 224 -224 val_224 224 -226 val_226 NULL -226 val_226 226 -228 val_228 NULL -228 val_228 228 -229 val_229 NULL -229 val_229 NULL -229 val_229 229 -229 val_229 229 -230 val_230 NULL -230 val_230 NULL -230 val_230 NULL -230 val_230 NULL -230 val_230 NULL -230 val_230 230 -230 val_230 230 -230 val_230 230 -230 val_230 230 -230 val_230 230 -233 val_233 NULL -233 val_233 NULL -233 val_233 233 -233 val_233 233 -235 val_235 NULL -235 val_235 235 -237 val_237 NULL -237 val_237 NULL -237 val_237 237 -237 val_237 237 -238 val_238 NULL -238 val_238 NULL -238 val_238 238 -238 val_238 238 -239 val_239 NULL -239 val_239 NULL -239 val_239 239 -239 val_239 239 -24 val_24 NULL -24 val_24 NULL -24 val_24 24 -24 val_24 24 -241 val_241 NULL -241 val_241 241 -242 val_242 NULL -242 val_242 NULL -242 val_242 242 -242 val_242 242 -244 val_244 NULL -244 val_244 244 -247 val_247 NULL -247 val_247 247 -248 val_248 NULL -248 val_248 248 -249 val_249 NULL -249 val_249 249 -252 val_252 NULL -252 val_252 252 -255 val_255 NULL -255 val_255 NULL -255 val_255 255 -255 val_255 255 -256 val_256 NULL -256 val_256 NULL -256 val_256 256 -256 val_256 256 -257 val_257 NULL -257 val_257 257 -258 val_258 NULL -258 val_258 258 -26 val_26 NULL -26 val_26 NULL -26 val_26 26 -26 val_26 26 -260 val_260 NULL -260 val_260 260 -262 val_262 NULL -262 val_262 262 -263 val_263 NULL -263 val_263 263 -265 val_265 NULL -265 val_265 NULL -265 val_265 265 -265 val_265 265 -266 val_266 NULL -266 val_266 266 -27 val_27 NULL -27 val_27 27 -272 val_272 NULL -272 val_272 NULL -272 val_272 272 -272 val_272 272 -273 val_273 NULL -273 val_273 NULL -273 val_273 NULL -273 val_273 273 -273 val_273 273 -273 val_273 273 -274 val_274 NULL -274 val_274 274 -275 val_275 NULL -275 val_275 275 -277 val_277 NULL -277 val_277 NULL -277 val_277 NULL -277 val_277 NULL -277 val_277 277 -277 val_277 277 -277 val_277 277 -277 val_277 277 -278 val_278 NULL -278 val_278 NULL -278 val_278 278 -278 val_278 278 -28 val_28 NULL -28 val_28 28 -280 val_280 NULL -280 val_280 NULL -280 val_280 280 -280 val_280 280 -281 val_281 NULL -281 val_281 NULL -281 val_281 281 -281 val_281 281 -282 val_282 NULL -282 val_282 NULL -282 val_282 282 -282 val_282 282 -283 val_283 NULL -283 val_283 283 -284 val_284 NULL -284 val_284 284 -285 val_285 NULL -285 val_285 285 -286 val_286 NULL -286 val_286 286 -287 val_287 NULL -287 val_287 287 -288 val_288 NULL -288 val_288 NULL -288 val_288 288 -288 val_288 288 -289 val_289 NULL -289 val_289 289 -291 val_291 NULL -291 val_291 291 -292 val_292 NULL -292 val_292 292 -296 val_296 NULL -296 val_296 296 -298 val_298 NULL -298 val_298 NULL -298 val_298 NULL -298 val_298 298 -298 val_298 298 -298 val_298 298 -30 val_30 NULL -30 val_30 30 -302 val_302 NULL -302 val_302 302 -305 val_305 NULL -305 val_305 305 -306 val_306 NULL -306 val_306 306 -307 val_307 NULL -307 val_307 NULL -307 val_307 307 -307 val_307 307 -308 val_308 NULL -308 val_308 308 -309 val_309 NULL -309 val_309 NULL -309 val_309 309 -309 val_309 309 -310 val_310 NULL -310 val_310 310 -311 val_311 NULL -311 val_311 NULL -311 val_311 NULL -311 val_311 311 -311 val_311 311 -311 val_311 311 -315 val_315 NULL -315 val_315 315 -316 val_316 NULL -316 val_316 NULL -316 val_316 NULL -316 val_316 316 -316 val_316 316 -316 val_316 316 -317 val_317 NULL -317 val_317 NULL -317 val_317 317 -317 val_317 317 -318 val_318 NULL -318 val_318 NULL -318 val_318 NULL -318 val_318 318 -318 val_318 318 -318 val_318 318 -321 val_321 NULL -321 val_321 NULL -321 val_321 321 -321 val_321 321 -322 val_322 NULL -322 val_322 NULL -322 val_322 322 -322 val_322 322 -323 val_323 NULL -323 val_323 323 -325 val_325 NULL -325 val_325 NULL -325 val_325 325 -325 val_325 325 -327 val_327 NULL -327 val_327 NULL -327 val_327 NULL -327 val_327 327 -327 val_327 327 -327 val_327 327 -33 val_33 NULL -33 val_33 33 -331 val_331 NULL -331 val_331 NULL -331 val_331 331 -331 val_331 331 -332 val_332 NULL -332 val_332 332 -333 val_333 NULL -333 val_333 NULL -333 val_333 333 -333 val_333 333 -335 val_335 NULL -335 val_335 335 -336 val_336 NULL -336 val_336 336 -338 val_338 NULL -338 val_338 338 -339 val_339 NULL -339 val_339 339 -34 val_34 NULL -34 val_34 34 -341 val_341 NULL -341 val_341 341 -342 val_342 NULL -342 val_342 NULL -342 val_342 342 -342 val_342 342 -344 val_344 NULL -344 val_344 NULL -344 val_344 344 -344 val_344 344 -345 val_345 NULL -345 val_345 345 -348 val_348 NULL -348 val_348 NULL -348 val_348 NULL -348 val_348 NULL -348 val_348 NULL -348 val_348 348 -348 val_348 348 -348 val_348 348 -348 val_348 348 -348 val_348 348 -35 val_35 NULL -35 val_35 NULL -35 val_35 NULL -35 val_35 35 -35 val_35 35 -35 val_35 35 -351 val_351 NULL -351 val_351 351 -353 val_353 NULL -353 val_353 NULL -353 val_353 353 -353 val_353 353 -356 val_356 NULL -356 val_356 356 -360 val_360 NULL -360 val_360 360 -362 val_362 NULL -362 val_362 362 -364 val_364 NULL -364 val_364 364 -365 val_365 NULL -365 val_365 365 -366 val_366 NULL -366 val_366 366 -367 val_367 NULL -367 val_367 NULL -367 val_367 367 -367 val_367 367 -368 val_368 NULL -368 val_368 368 -369 val_369 NULL -369 val_369 NULL -369 val_369 NULL -369 val_369 369 -369 val_369 369 -369 val_369 369 -37 val_37 NULL -37 val_37 NULL -37 val_37 37 -37 val_37 37 -373 val_373 NULL -373 val_373 373 -374 val_374 NULL -374 val_374 374 -375 val_375 NULL -375 val_375 375 -377 val_377 NULL -377 val_377 377 -378 val_378 NULL -378 val_378 378 -379 val_379 NULL -379 val_379 379 -382 val_382 NULL -382 val_382 NULL -382 val_382 382 -382 val_382 382 -384 val_384 NULL -384 val_384 NULL -384 val_384 NULL -384 val_384 384 -384 val_384 384 -384 val_384 384 -386 val_386 NULL -386 val_386 386 -389 val_389 NULL -389 val_389 389 -392 val_392 NULL -392 val_392 392 -393 val_393 NULL -393 val_393 393 -394 val_394 NULL -394 val_394 394 -395 val_395 NULL -395 val_395 NULL -395 val_395 395 -395 val_395 395 -396 val_396 NULL -396 val_396 NULL -396 val_396 NULL -396 val_396 396 -396 val_396 396 -396 val_396 396 -397 val_397 NULL -397 val_397 NULL -397 val_397 397 -397 val_397 397 -399 val_399 NULL -399 val_399 NULL -399 val_399 399 -399 val_399 399 -4 val_4 NULL -4 val_4 4 -400 val_400 NULL -400 val_400 400 -401 val_401 NULL -401 val_401 NULL -401 val_401 NULL -401 val_401 NULL -401 val_401 NULL -401 val_401 401 -401 val_401 401 -401 val_401 401 -401 val_401 401 -401 val_401 401 -402 val_402 NULL -402 val_402 402 -403 val_403 NULL -403 val_403 NULL -403 val_403 NULL -403 val_403 403 -403 val_403 403 -403 val_403 403 -404 val_404 NULL -404 val_404 NULL -404 val_404 404 -404 val_404 404 -406 val_406 NULL -406 val_406 NULL -406 val_406 NULL -406 val_406 NULL -406 val_406 406 -406 val_406 406 -406 val_406 406 -406 val_406 406 -407 val_407 NULL -407 val_407 407 -409 val_409 NULL -409 val_409 NULL -409 val_409 NULL -409 val_409 409 -409 val_409 409 -409 val_409 409 -41 val_41 NULL -41 val_41 41 -411 val_411 NULL -411 val_411 411 -413 val_413 NULL -413 val_413 NULL -413 val_413 413 -413 val_413 413 -414 val_414 NULL -414 val_414 NULL -414 val_414 414 -414 val_414 414 -417 val_417 NULL -417 val_417 NULL -417 val_417 NULL -417 val_417 417 -417 val_417 417 -417 val_417 417 -418 val_418 NULL -418 val_418 418 -419 val_419 NULL -419 val_419 419 -42 val_42 NULL -42 val_42 NULL -42 val_42 42 -42 val_42 42 -421 val_421 NULL -421 val_421 421 -424 val_424 NULL -424 val_424 NULL -424 val_424 424 -424 val_424 424 -427 val_427 NULL -427 val_427 427 -429 val_429 NULL -429 val_429 NULL -429 val_429 429 -429 val_429 429 -43 val_43 NULL -43 val_43 43 -430 val_430 NULL -430 val_430 NULL -430 val_430 NULL -430 val_430 430 -430 val_430 430 -430 val_430 430 -431 val_431 NULL -431 val_431 NULL -431 val_431 NULL -431 val_431 431 -431 val_431 431 -431 val_431 431 -432 val_432 NULL -432 val_432 432 -435 val_435 NULL -435 val_435 435 -436 val_436 NULL -436 val_436 436 -437 val_437 NULL -437 val_437 437 -438 val_438 NULL -438 val_438 NULL -438 val_438 NULL -438 val_438 438 -438 val_438 438 -438 val_438 438 -439 val_439 NULL -439 val_439 NULL -439 val_439 439 -439 val_439 439 -44 val_44 NULL -44 val_44 44 -443 val_443 NULL -443 val_443 443 -444 val_444 NULL -444 val_444 444 -446 val_446 NULL -446 val_446 446 -448 val_448 NULL -448 val_448 448 -449 val_449 NULL -449 val_449 449 -452 val_452 NULL -452 val_452 452 -453 val_453 NULL -453 val_453 453 -454 val_454 NULL -454 val_454 NULL -454 val_454 NULL -454 val_454 454 -454 val_454 454 -454 val_454 454 -455 val_455 NULL -455 val_455 455 -457 val_457 NULL -457 val_457 457 -458 val_458 NULL -458 val_458 NULL -458 val_458 458 -458 val_458 458 -459 val_459 NULL -459 val_459 NULL -459 val_459 459 -459 val_459 459 -460 val_460 NULL -460 val_460 460 -462 val_462 NULL -462 val_462 NULL -462 val_462 462 -462 val_462 462 -463 val_463 NULL -463 val_463 NULL -463 val_463 463 -463 val_463 463 -466 val_466 NULL -466 val_466 NULL -466 val_466 NULL -466 val_466 466 -466 val_466 466 -466 val_466 466 -467 val_467 NULL -467 val_467 467 -468 val_468 NULL -468 val_468 NULL -468 val_468 NULL -468 val_468 NULL -468 val_468 468 -468 val_468 468 -468 val_468 468 -468 val_468 468 -469 val_469 NULL -469 val_469 NULL -469 val_469 NULL -469 val_469 NULL -469 val_469 NULL -469 val_469 469 -469 val_469 469 -469 val_469 469 -469 val_469 469 -469 val_469 469 -47 val_47 NULL -47 val_47 47 -470 val_470 NULL -470 val_470 470 -472 val_472 NULL -472 val_472 472 -475 val_475 NULL -475 val_475 475 -477 val_477 NULL -477 val_477 477 -478 val_478 NULL -478 val_478 NULL -478 val_478 478 -478 val_478 478 -479 val_479 NULL -479 val_479 479 -480 val_480 NULL -480 val_480 NULL -480 val_480 NULL -480 val_480 480 -480 val_480 480 -480 val_480 480 -481 val_481 NULL -481 val_481 481 -482 val_482 NULL -482 val_482 482 -483 val_483 NULL -483 val_483 483 -484 val_484 NULL -484 val_484 484 -485 val_485 NULL -485 val_485 485 -487 val_487 NULL -487 val_487 487 -489 val_489 NULL -489 val_489 NULL -489 val_489 NULL -489 val_489 NULL -489 val_489 489 -489 val_489 489 -489 val_489 489 -489 val_489 489 -490 val_490 NULL -490 val_490 490 -491 val_491 NULL -491 val_491 491 -492 val_492 NULL -492 val_492 NULL -492 val_492 492 -492 val_492 492 -493 val_493 NULL -493 val_493 493 -494 val_494 NULL -494 val_494 494 -495 val_495 NULL -495 val_495 495 -496 val_496 NULL -496 val_496 496 -497 val_497 NULL -497 val_497 497 -498 val_498 NULL -498 val_498 NULL -498 val_498 NULL -498 val_498 498 -498 val_498 498 -498 val_498 498 -5 val_5 NULL -5 val_5 NULL -5 val_5 NULL -5 val_5 5 -5 val_5 5 -5 val_5 5 -51 val_51 NULL -51 val_51 NULL -51 val_51 51 -51 val_51 51 -53 val_53 NULL -53 val_53 53 -54 val_54 NULL -54 val_54 54 -57 val_57 NULL -57 val_57 57 -58 val_58 NULL -58 val_58 NULL -58 val_58 58 -58 val_58 58 -64 val_64 NULL -64 val_64 64 -65 val_65 NULL -65 val_65 65 -66 val_66 NULL -66 val_66 66 -67 val_67 NULL -67 val_67 NULL -67 val_67 67 -67 val_67 67 -69 val_69 NULL -69 val_69 69 -70 val_70 NULL -70 val_70 NULL -70 val_70 NULL -70 val_70 70 -70 val_70 70 -70 val_70 70 -72 val_72 NULL -72 val_72 NULL -72 val_72 72 -72 val_72 72 -74 val_74 NULL -74 val_74 74 -76 val_76 NULL -76 val_76 NULL -76 val_76 76 -76 val_76 76 -77 val_77 NULL -77 val_77 77 -78 val_78 NULL -78 val_78 78 -8 val_8 NULL -8 val_8 8 -80 val_80 NULL -80 val_80 80 -82 val_82 NULL -82 val_82 82 -83 val_83 NULL -83 val_83 NULL -83 val_83 83 -83 val_83 83 -84 val_84 NULL -84 val_84 NULL -84 val_84 84 -84 val_84 84 -85 val_85 NULL -85 val_85 85 -86 val_86 NULL -86 val_86 86 -87 val_87 NULL -87 val_87 87 -9 val_9 NULL -9 val_9 9 -90 val_90 NULL -90 val_90 NULL -90 val_90 NULL -90 val_90 90 -90 val_90 90 -90 val_90 90 -92 val_92 NULL -92 val_92 92 -95 val_95 NULL -95 val_95 NULL -95 val_95 95 -95 val_95 95 -96 val_96 NULL -96 val_96 96 -97 val_97 NULL -97 val_97 NULL -97 val_97 97 -97 val_97 97 -98 val_98 NULL -98 val_98 NULL -98 val_98 98 -98 val_98 98 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index dcf40c875b9ad..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:284 -maxFileSize:284 -minFileSize:284 -lastAccessTime:0 -lastUpdateTime:1390900729000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 deleted file mode 100644 index 9639a1e84c615..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 +++ /dev/null @@ -1,50 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 deleted file mode 100644 index 707036ebbc76c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:3 -totalFileSize:1415 -maxFileSize:895 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 deleted file mode 100644 index 06316a924c38b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index 735d41e54e958..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:284 -maxFileSize:284 -minFileSize:284 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 deleted file mode 100644 index 27dfc622c9bb6..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=102 -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:895 -maxFileSize:895 -minFileSize:895 -lastAccessTime:0 -lastUpdateTime:1390900762000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 deleted file mode 100644 index 1812e0d53e443..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900706000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 deleted file mode 100644 index fca99e91bbd8f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 +++ /dev/null @@ -1,75 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f deleted file mode 100644 index fca99e91bbd8f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f +++ /dev/null @@ -1,75 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 deleted file mode 100644 index 3532257511613..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900706000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 deleted file mode 100644 index 9b9389353dd5f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 deleted file mode 100644 index a02458b88bf52..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:2 -totalFileSize:520 -maxFileSize:284 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900729000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 deleted file mode 100644 index 301bff44316ff..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 +++ /dev/null @@ -1,15 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/partition_test_partitioned/dt=100 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:236 -maxFileSize:236 -minFileSize:236 -lastAccessTime:0 -lastUpdateTime:1390900729000 - diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e deleted file mode 100644 index 001841c8a1cd4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf deleted file mode 100644 index 63f35fd827de3..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf deleted file mode 100644 index 63f35fd827de3..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e deleted file mode 100644 index f75126345f351..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 11542e84a3768..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 1b97612ce3f5a..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,4 +0,0 @@ -476 val_238 -476 val_238 -194 val_97 -194 val_97 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index f8e13d5235028..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,4 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -97 val_97 NULL 2 -97 val_97 NULL 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 deleted file mode 100644 index 919a66a94c5cb..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 +++ /dev/null @@ -1,6 +0,0 @@ -476 val_238 NULL -476 val_238 NULL -194 val_97 NULL -194 val_97 NULL -400 val_200 val_200 -400 val_200 val_200 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 45c169f0d8330..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,6 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -97 val_97 NULL 2 -97 val_97 NULL 2 -200 val_200 val_200 3 -200 val_200 val_200 3 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 deleted file mode 100644 index b6a7d89c68e0c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 +++ /dev/null @@ -1 +0,0 @@ -16 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d deleted file mode 100644 index b6a7d89c68e0c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d +++ /dev/null @@ -1 +0,0 @@ -16 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 deleted file mode 100644 index 425151f3a411f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 +++ /dev/null @@ -1 +0,0 @@ -40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 deleted file mode 100644 index 425151f3a411f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 +++ /dev/null @@ -1 +0,0 @@ -40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 deleted file mode 100644 index 425151f3a411f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 +++ /dev/null @@ -1 +0,0 @@ -40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 deleted file mode 100644 index c739b42c4d2ce..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 +++ /dev/null @@ -1 +0,0 @@ -44 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 11542e84a3768..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 deleted file mode 100644 index 025abe4ec330b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 +++ /dev/null @@ -1,3 +0,0 @@ -476 val_238 NULL 1 -476 val_238 NULL 1 -172 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index f067c1ed602dc..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,3 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -86 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 11542e84a3768..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 deleted file mode 100644 index 025abe4ec330b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 +++ /dev/null @@ -1,3 +0,0 @@ -476 val_238 NULL 1 -476 val_238 NULL 1 -172 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index f067c1ed602dc..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,3 +0,0 @@ -238 val_238 NULL 1 -238 val_238 NULL 1 -86 val_86 val_86 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index 53aab40f88b50..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476.0 val_238 -476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd deleted file mode 100644 index 6d512a4f0bdc4..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd +++ /dev/null @@ -1,2 +0,0 @@ -238 val_238 1 -238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b deleted file mode 100644 index de31196d97c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b +++ /dev/null @@ -1,2 +0,0 @@ -476 val_238 -476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 deleted file mode 100644 index 325e26b3d9737..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 +++ /dev/null @@ -1,75 +0,0 @@ -238 val_238 100 -NULL 100 -311 val_311 100 -NULL val_27 100 -NULL val_165 100 -NULL val_409 100 -255 val_255 100 -278 val_278 100 -98 val_98 100 -NULL val_484 100 -NULL val_265 100 -NULL val_193 100 -401 val_401 100 -150 val_150 100 -273 val_273 100 -224 100 -369 100 -66 val_66 100 -128 100 -213 val_213 100 -146 val_146 100 -406 val_406 100 -NULL 100 -NULL 100 -NULL 100 -238 val_238 101 -NULL 101 -311 val_311 101 -NULL val_27 101 -NULL val_165 101 -NULL val_409 101 -255 val_255 101 -278 val_278 101 -98 val_98 101 -NULL val_484 101 -NULL val_265 101 -NULL val_193 101 -401 val_401 101 -150 val_150 101 -273 val_273 101 -224 101 -369 101 -66 val_66 101 -128 101 -213 val_213 101 -146 val_146 101 -406 val_406 101 -NULL 101 -NULL 101 -NULL 101 -238 val_238 102 -NULL 102 -311 val_311 102 -NULL val_27 102 -NULL val_165 102 -NULL val_409 102 -255 val_255 102 -278 val_278 102 -98 val_98 102 -NULL val_484 102 -NULL val_265 102 -NULL val_193 102 -401 val_401 102 -150 val_150 102 -273 val_273 102 -224 102 -369 102 -66 val_66 102 -128 102 -213 val_213 102 -146 val_146 102 -406 val_406 102 -NULL 102 -NULL 102 -NULL 102 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 deleted file mode 100644 index 325e26b3d9737..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 +++ /dev/null @@ -1,75 +0,0 @@ -238 val_238 100 -NULL 100 -311 val_311 100 -NULL val_27 100 -NULL val_165 100 -NULL val_409 100 -255 val_255 100 -278 val_278 100 -98 val_98 100 -NULL val_484 100 -NULL val_265 100 -NULL val_193 100 -401 val_401 100 -150 val_150 100 -273 val_273 100 -224 100 -369 100 -66 val_66 100 -128 100 -213 val_213 100 -146 val_146 100 -406 val_406 100 -NULL 100 -NULL 100 -NULL 100 -238 val_238 101 -NULL 101 -311 val_311 101 -NULL val_27 101 -NULL val_165 101 -NULL val_409 101 -255 val_255 101 -278 val_278 101 -98 val_98 101 -NULL val_484 101 -NULL val_265 101 -NULL val_193 101 -401 val_401 101 -150 val_150 101 -273 val_273 101 -224 101 -369 101 -66 val_66 101 -128 101 -213 val_213 101 -146 val_146 101 -406 val_406 101 -NULL 101 -NULL 101 -NULL 101 -238 val_238 102 -NULL 102 -311 val_311 102 -NULL val_27 102 -NULL val_165 102 -NULL val_409 102 -255 val_255 102 -278 val_278 102 -98 val_98 102 -NULL val_484 102 -NULL val_265 102 -NULL val_193 102 -401 val_401 102 -150 val_150 102 -273 val_273 102 -224 102 -369 102 -66 val_66 102 -128 102 -213 val_213 102 -146 val_146 102 -406 val_406 102 -NULL 102 -NULL 102 -NULL 102 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 deleted file mode 100644 index 79c8f8e614a1f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index f487b340cd55b..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,14 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.hive.ql.io.RCFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:284 -maxFileSize:284 -minFileSize:284 -lastAccessTime:0 -lastUpdateTime:1388799388000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 deleted file mode 100644 index 0c7739c2b9fbb..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=102 -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:895 -maxFileSize:895 -minFileSize:895 -lastAccessTime:0 -lastUpdateTime:1388799405000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a deleted file mode 100644 index 79c8f8e614a1f..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a +++ /dev/null @@ -1,25 +0,0 @@ -238 -NULL -311 -NULL -NULL -NULL -255 -278 -98 -NULL -NULL -NULL -401 -150 -273 -224 -369 -66 -128 -213 -146 -406 -NULL -NULL -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a deleted file mode 100644 index 0c6fbc997de7a..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a +++ /dev/null @@ -1,14 +0,0 @@ -tableName:partition_test_partitioned -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/partition_test_partitioned/dt=101 -inputformat:org.apache.hadoop.mapred.SequenceFileInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -columns:struct columns { string key, string value} -partitioned:true -partitionColumns:struct partition_columns { string dt} -totalNumberFiles:1 -totalFileSize:895 -maxFileSize:895 -minFileSize:895 -lastAccessTime:0 -lastUpdateTime:1388799434000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 index f0d140e18c053..eb4c6a843cb5d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 @@ -1,2 +1,2 @@ 101 25 -102 25 \ No newline at end of file +102 25 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 index 6425bae2a6a8f..95846abf28b2f 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 @@ -1,3 +1,3 @@ 101 25 102 25 -103 25 \ No newline at end of file +103 25 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f index d8263ee986059..0cfbf08886fca 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f @@ -1 +1 @@ -2 \ No newline at end of file +2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 index d8263ee986059..0cfbf08886fca 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 @@ -1 +1 @@ -2 \ No newline at end of file +2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 index 3f10ffe7a4c47..60d3b2f4a4cd5 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 @@ -1 +1 @@ -15 \ No newline at end of file +15 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 index d8263ee986059..0cfbf08886fca 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 @@ -1 +1 @@ -2 \ No newline at end of file +2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b deleted file mode 100644 index 5e06930239fea..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b +++ /dev/null @@ -1,20 +0,0 @@ -0 val_0 1 -0 val_0 1 -0 val_0 1 -0 val_0 2 -0 val_0 2 -0 val_0 2 -0 val_0 3 -0 val_0 3 -0 val_0 3 -10 val_10 1 -10 val_10 2 -10 val_10 3 -100 val_100 1 -100 val_100 1 -100 val_100 2 -100 val_100 2 -100 val_100 3 -100 val_100 3 -103 val_103 1 -103 val_103 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd deleted file mode 100644 index 1bd9063a9c63c..0000000000000 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd +++ /dev/null @@ -1,20 +0,0 @@ -0.0 val_0 1 -0.0 val_0 1 -0.0 val_0 1 -0.0 val_0 2 -0.0 val_0 2 -0.0 val_0 2 -0.0 val_0 3 -0.0 val_0 3 -0.0 val_0 3 -4.0 val_2 1 -4.0 val_2 2 -4.0 val_2 3 -8.0 val_4 1 -8.0 val_4 2 -8.0 val_4 3 -10.0 val_5 1 -10.0 val_5 1 -10.0 val_5 1 -10.0 val_5 2 -10.0 val_5 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b index f259b4eefc608..967305a18236e 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b @@ -17,4 +17,4 @@ 104 val_104 1 104 val_104 1 104 val_104 2 -104 val_104 2 \ No newline at end of file +104 val_104 2 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd index 89afce150e7a9..f26a7375a6819 100644 --- a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd @@ -17,4 +17,4 @@ 16.0 val_8 1 16.0 val_8 2 18.0 val_9 1 -18.0 val_9 2 \ No newline at end of file +18.0 val_9 2 diff --git a/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 index 55b2a1b47afa2..ef98fd7908a01 100644 --- a/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 +++ b/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 @@ -493,4 +493,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 index 55b2a1b47afa2..ef98fd7908a01 100644 --- a/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 +++ b/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 @@ -493,4 +493,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b index 878c86cb46b8a..d2aff0f701be9 100644 --- a/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b +++ b/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b @@ -133,4 +133,4 @@ 480 3 489 4 492 2 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b index 878c86cb46b8a..d2aff0f701be9 100644 --- a/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b +++ b/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b @@ -133,4 +133,4 @@ 480 3 489 4 492 2 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c index c87107a2f1168..416fbfb9e5228 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c @@ -22,4 +22,4 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL -NULL NULL NULL \ No newline at end of file +NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c index c87107a2f1168..416fbfb9e5228 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c @@ -22,4 +22,4 @@ NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL -NULL NULL NULL \ No newline at end of file +NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 index e37d32abba426..83b33d238dab9 100644 --- a/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 +++ b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 @@ -1 +1 @@ -1000 \ No newline at end of file +1000 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d index e1659e6072577..f005a4fe5bf86 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d +++ b/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d @@ -126,4 +126,4 @@ val_395 val_396 val_397 val_399 -val_4 \ No newline at end of file +val_4 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d index e1659e6072577..f005a4fe5bf86 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d +++ b/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d @@ -126,4 +126,4 @@ val_395 val_396 val_397 val_399 -val_4 \ No newline at end of file +val_4 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f index 7e66866e2dd60..ab02a73437ed7 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f +++ b/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f @@ -2,4 +2,4 @@ val_4 1 val_399 2 val_396 3 val_277 4 -val_348 5 \ No newline at end of file +val_348 5 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f index 7e66866e2dd60..ab02a73437ed7 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f +++ b/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f @@ -2,4 +2,4 @@ val_4 1 val_399 2 val_396 3 val_277 4 -val_348 5 \ No newline at end of file +val_348 5 diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 index fbce9efa766d7..a29747aef8046 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 +++ b/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 @@ -709,4 +709,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 index fbce9efa766d7..a29747aef8046 100644 --- a/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 +++ b/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 @@ -709,4 +709,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 index 9e6c4359e78a8..f2748faa4ea7e 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 +++ b/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 @@ -1707,4 +1707,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 index 9e6c4359e78a8..f2748faa4ea7e 100644 --- a/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 +++ b/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 @@ -1707,4 +1707,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b index 0e11aea14d61d..91f2ce2b61787 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b +++ b/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b @@ -1764,4 +1764,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b index 0e11aea14d61d..91f2ce2b61787 100644 --- a/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b +++ b/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b @@ -1764,4 +1764,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 index 997f37b76bedb..c0ffb7aeca9b6 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 index b3bf95dd32bac..9365b77dec065 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 index b3bf95dd32bac..9365b77dec065 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 @@ -3,4 +3,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -19 val_19 19 val_19 \ No newline at end of file +19 val_19 19 val_19 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe index d4a363c49aeaf..daa4ea309e2c1 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe @@ -7,4 +7,4 @@ 18 val_18 18 val_18 18 18 val_18 18 val_18 18 18 val_18 18 val_18 18 -18 val_18 18 val_18 18 \ No newline at end of file +18 val_18 18 val_18 18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe index d4a363c49aeaf..daa4ea309e2c1 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe @@ -7,4 +7,4 @@ 18 val_18 18 val_18 18 18 val_18 18 val_18 18 18 val_18 18 val_18 18 -18 val_18 18 val_18 18 \ No newline at end of file +18 val_18 18 val_18 18 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 deleted file mode 100644 index 0190981db84ed..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 +++ /dev/null @@ -1,84 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -11 val_11 -12 val_12 -12 val_12 -15 val_15 -15 val_15 -17 val_17 -18 val_18 -18 val_18 -19 val_19 -2 val_2 -20 val_20 -24 val_24 -24 val_24 -26 val_26 -26 val_26 -27 val_27 -28 val_28 -30 val_30 -33 val_33 -34 val_34 -35 val_35 -35 val_35 -35 val_35 -37 val_37 -37 val_37 -4 val_4 -41 val_41 -42 val_42 -42 val_42 -43 val_43 -44 val_44 -47 val_47 -5 val_5 -5 val_5 -5 val_5 -51 val_51 -51 val_51 -53 val_53 -54 val_54 -57 val_57 -58 val_58 -58 val_58 -64 val_64 -65 val_65 -66 val_66 -67 val_67 -67 val_67 -69 val_69 -70 val_70 -70 val_70 -70 val_70 -72 val_72 -72 val_72 -74 val_74 -76 val_76 -76 val_76 -77 val_77 -78 val_78 -8 val_8 -80 val_80 -82 val_82 -83 val_83 -83 val_83 -84 val_84 -84 val_84 -85 val_85 -86 val_86 -87 val_87 -9 val_9 -90 val_90 -90 val_90 -90 val_90 -92 val_92 -95 val_95 -95 val_95 -96 val_96 -97 val_97 -97 val_97 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 deleted file mode 100644 index 0190981db84ed..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 +++ /dev/null @@ -1,84 +0,0 @@ -0 val_0 -0 val_0 -0 val_0 -10 val_10 -11 val_11 -12 val_12 -12 val_12 -15 val_15 -15 val_15 -17 val_17 -18 val_18 -18 val_18 -19 val_19 -2 val_2 -20 val_20 -24 val_24 -24 val_24 -26 val_26 -26 val_26 -27 val_27 -28 val_28 -30 val_30 -33 val_33 -34 val_34 -35 val_35 -35 val_35 -35 val_35 -37 val_37 -37 val_37 -4 val_4 -41 val_41 -42 val_42 -42 val_42 -43 val_43 -44 val_44 -47 val_47 -5 val_5 -5 val_5 -5 val_5 -51 val_51 -51 val_51 -53 val_53 -54 val_54 -57 val_57 -58 val_58 -58 val_58 -64 val_64 -65 val_65 -66 val_66 -67 val_67 -67 val_67 -69 val_69 -70 val_70 -70 val_70 -70 val_70 -72 val_72 -72 val_72 -74 val_74 -76 val_76 -76 val_76 -77 val_77 -78 val_78 -8 val_8 -80 val_80 -82 val_82 -83 val_83 -83 val_83 -84 val_84 -84 val_84 -85 val_85 -86 val_86 -87 val_87 -9 val_9 -90 val_90 -90 val_90 -90 val_90 -92 val_92 -95 val_95 -95 val_95 -96 val_96 -97 val_97 -97 val_97 -98 val_98 -98 val_98 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 +++ b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e index 4c3919232c73e..c30d4a581ba6c 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e +++ b/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e @@ -168,4 +168,4 @@ 90 val_90 403 val_403 400 val_400 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 +++ b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e index 4c3919232c73e..c30d4a581ba6c 100644 --- a/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e +++ b/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e @@ -168,4 +168,4 @@ 90 val_90 403 val_403 400 val_400 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 b/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 b/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f b/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c b/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 b/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 b/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 b/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 b/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 b/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c b/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 b/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c b/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 b/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 b/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 deleted file mode 100644 index d183a30ddf3b6..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 +++ /dev/null @@ -1,36 +0,0 @@ -238 val_238 2008-04-08 11 -86 val_86 2008-04-08 11 -311 val_311 2008-04-08 11 -27 val_27 2008-04-08 11 -165 val_165 2008-04-08 11 -409 val_409 2008-04-08 11 -255 val_255 2008-04-08 11 -278 val_278 2008-04-08 11 -98 val_98 2008-04-08 11 -238 val_238 2008-04-08 12 -86 val_86 2008-04-08 12 -311 val_311 2008-04-08 12 -27 val_27 2008-04-08 12 -165 val_165 2008-04-08 12 -409 val_409 2008-04-08 12 -255 val_255 2008-04-08 12 -278 val_278 2008-04-08 12 -98 val_98 2008-04-08 12 -238 val_238 2008-04-09 11 -86 val_86 2008-04-09 11 -311 val_311 2008-04-09 11 -27 val_27 2008-04-09 11 -165 val_165 2008-04-09 11 -409 val_409 2008-04-09 11 -255 val_255 2008-04-09 11 -278 val_278 2008-04-09 11 -98 val_98 2008-04-09 11 -238 val_238 2008-04-09 12 -86 val_86 2008-04-09 12 -311 val_311 2008-04-09 12 -27 val_27 2008-04-09 12 -165 val_165 2008-04-09 12 -409 val_409 2008-04-09 12 -255 val_255 2008-04-09 12 -278 val_278 2008-04-09 12 -98 val_98 2008-04-09 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 b/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 b/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 deleted file mode 100644 index 5b2461e35b5f4..0000000000000 --- a/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 +++ /dev/null @@ -1,36 +0,0 @@ -238 val_238 2008-04-08 11 0 -238 val_238 2008-04-08 11 0 -86 val_86 2008-04-08 11 12 -311 val_311 2008-04-08 11 22 -311 val_311 2008-04-08 11 22 -311 val_311 2008-04-08 11 22 -27 val_27 2008-04-08 11 34 -165 val_165 2008-04-08 11 44 -165 val_165 2008-04-08 11 44 -238 val_238 2008-04-08 12 0 -238 val_238 2008-04-08 12 0 -86 val_86 2008-04-08 12 12 -311 val_311 2008-04-08 12 22 -311 val_311 2008-04-08 12 22 -311 val_311 2008-04-08 12 22 -27 val_27 2008-04-08 12 34 -165 val_165 2008-04-08 12 44 -165 val_165 2008-04-08 12 44 -238 val_238 2008-04-09 11 0 -238 val_238 2008-04-09 11 0 -86 val_86 2008-04-09 11 12 -311 val_311 2008-04-09 11 22 -311 val_311 2008-04-09 11 22 -311 val_311 2008-04-09 11 22 -27 val_27 2008-04-09 11 34 -165 val_165 2008-04-09 11 44 -165 val_165 2008-04-09 11 44 -238 val_238 2008-04-09 12 0 -238 val_238 2008-04-09 12 0 -86 val_86 2008-04-09 12 12 -311 val_311 2008-04-09 12 22 -311 val_311 2008-04-09 12 22 -311 val_311 2008-04-09 12 22 -27 val_27 2008-04-09 12 34 -165 val_165 2008-04-09 12 44 -165 val_165 2008-04-09 12 44 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/ppr_pushdown-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/ppr_pushdown-1-855b235f9c760ba9d6d0200bfd1ad08b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 b/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 b/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/ppr_pushdown-10-2957fd9b211cee5f0372525a1de55c19 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 b/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown-11-b8d6f0ffc8294497c792b26958adee45 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 b/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown-12-ab1b9a0cdd9586c96d8856a9d632b89c diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 b/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 rename to sql/hive/src/test/resources/golden/ppr_pushdown-13-2c316c67fd494d878fbbea107d283c3b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-14-53b4be82a1538844d03b200429efa02b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown-14-53b4be82a1538844d03b200429efa02b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c b/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown-15-71ab93d38ed2908069091c7c8cc0aba1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb b/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae b/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/ppr_pushdown-16-855cb54d28034fdb20a3615ee0918d63 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/ppr_pushdown-17-d3ccf2722a8b7281fcee61b2544772c8 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 b/sql/hive/src/test/resources/golden/ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 rename to sql/hive/src/test/resources/golden/ppr_pushdown-18-fcd10fbfc55a2c0aa843fe618f9613c6 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 b/sql/hive/src/test/resources/golden/ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 rename to sql/hive/src/test/resources/golden/ppr_pushdown-19-ff36e3978571ac05f11e8322c024e4b6 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 b/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/ppr_pushdown-2-f689aaf15b08c433d1e93ce977a9b6d5 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a b/sql/hive/src/test/resources/golden/ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a rename to sql/hive/src/test/resources/golden/ppr_pushdown-20-481005cf034ef3d7b998da32eb82aa9a diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 b/sql/hive/src/test/resources/golden/ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 rename to sql/hive/src/test/resources/golden/ppr_pushdown-21-9073825e8b9804331f780980d1f9fa92 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d b/sql/hive/src/test/resources/golden/ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d rename to sql/hive/src/test/resources/golden/ppr_pushdown-22-9cb28f0d3a434c9d1ab039192906ec9d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d b/sql/hive/src/test/resources/golden/ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d rename to sql/hive/src/test/resources/golden/ppr_pushdown-23-678f50025924fee7b59f66e2abdb472d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 b/sql/hive/src/test/resources/golden/ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 rename to sql/hive/src/test/resources/golden/ppr_pushdown-24-4d5bfa800ba434f464b07bf9b5d39f20 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 b/sql/hive/src/test/resources/golden/ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 rename to sql/hive/src/test/resources/golden/ppr_pushdown-25-22663f09ea1c1bc303756067e84df5a7 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b b/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown-26-e789d9b469aa1fffe4ce0a15a8c1fb9b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 b/sql/hive/src/test/resources/golden/ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 rename to sql/hive/src/test/resources/golden/ppr_pushdown-27-8065c18e387fd8bffae19a80af8dc1d4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 b/sql/hive/src/test/resources/golden/ppr_pushdown-28-b72de558c88ae91460989938000e0d27 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 rename to sql/hive/src/test/resources/golden/ppr_pushdown-28-b72de558c88ae91460989938000e0d27 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 b/sql/hive/src/test/resources/golden/ppr_pushdown-29-7217735d357770af4ffee730e4e9add4 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 rename to sql/hive/src/test/resources/golden/ppr_pushdown-29-7217735d357770af4ffee730e4e9add4 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown-3-5eede4874e678021938683fc2f5dc900 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown-3-5eede4874e678021938683fc2f5dc900 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 b/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea b/sql/hive/src/test/resources/golden/ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea rename to sql/hive/src/test/resources/golden/ppr_pushdown-30-630e2f7918b7727fc4ca057fa21e2eea diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 b/sql/hive/src/test/resources/golden/ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 rename to sql/hive/src/test/resources/golden/ppr_pushdown-31-2e062414293b643ea4d7f6df92f939e4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf b/sql/hive/src/test/resources/golden/ppr_pushdown-32-5eba4bf10315099129eae319d73636cf similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf rename to sql/hive/src/test/resources/golden/ppr_pushdown-32-5eba4bf10315099129eae319d73636cf diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b b/sql/hive/src/test/resources/golden/ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b rename to sql/hive/src/test/resources/golden/ppr_pushdown-33-35af0585a4f98bc222c786688cb6de6b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 b/sql/hive/src/test/resources/golden/ppr_pushdown-34-a5594625510703427ab8bae5d0563f73 similarity index 100% rename from sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 rename to sql/hive/src/test/resources/golden/ppr_pushdown-34-a5594625510703427ab8bae5d0563f73 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a rename to sql/hive/src/test/resources/golden/ppr_pushdown-4-c5c542f8ee81cd0afd44e67fc7b4d306 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 b/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de b/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 rename to sql/hive/src/test/resources/golden/ppr_pushdown-5-f54bebec398f0fdfdbc0393123dba234 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d b/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 rename to sql/hive/src/test/resources/golden/ppr_pushdown-6-78af12432bcbf99d4a0d41c25f964de diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 rename to sql/hive/src/test/resources/golden/ppr_pushdown-7-58724fbe96a0b3deceef20c8cc5e318d diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown-8-14570f946e75924d7926c809485951d1 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/ppr_pushdown-8-14570f946e75924d7926c809485951d1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c b/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 b/sql/hive/src/test/resources/golden/ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 rename to sql/hive/src/test/resources/golden/ppr_pushdown-9-3ba325662296fc455f07f1c835495e4c diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/ppr_pushdown2-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 b/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-1-855b235f9c760ba9d6d0200bfd1ad08b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 new file mode 100644 index 0000000000000..f50a5fea8dd5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-4fa4ba4c739b6f44975d41e4705d4389 @@ -0,0 +1 @@ +1 1 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b b/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/ppr_pushdown2-11-ab3e46183487096745d7d90e3020e94c diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a b/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-12-b10b9e14f5a7a7a92c0c68df6dbc656a diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc rename to sql/hive/src/test/resources/golden/ppr_pushdown2-13-154f9859bd0822e287fbfdff12fd45ff diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 b/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 deleted file mode 100644 index e3e04ee48543d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 +++ /dev/null @@ -1 +0,0 @@ -3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-14-cf4a30b5c8329d8d79ddf762f318fbb3 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 deleted file mode 100644 index c458b0f57aba9..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 +++ /dev/null @@ -1 +0,0 @@ -2 1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 new file mode 100644 index 0000000000000..3def25c3c65af --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-8d01597374157f2d3d066840983ba1f8 @@ -0,0 +1 @@ +3 1 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb deleted file mode 100644 index c458b0f57aba9..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb +++ /dev/null @@ -1 +0,0 @@ -2 1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 new file mode 100644 index 0000000000000..55c794b56ec9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-5614065e1b8e709f68be4fa67666f41 @@ -0,0 +1 @@ +2 1 1 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb new file mode 100644 index 0000000000000..55c794b56ec9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-3a67618e47c977f58c9dd8f4b9a576eb @@ -0,0 +1 @@ +2 1 1 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe deleted file mode 100644 index 63511415ddf55..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe +++ /dev/null @@ -1,2 +0,0 @@ -3 1 2 1 -1 1 2 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe new file mode 100644 index 0000000000000..8d13286371dab --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-44e5f3ac566f60d8b17ef19c18a11ebe @@ -0,0 +1,2 @@ +3 1 2 1 +1 1 2 3 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 deleted file mode 100644 index c592b5d505b0e..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 +++ /dev/null @@ -1,3 +0,0 @@ -2 1 1 2 -3 1 2 1 -1 1 2 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 deleted file mode 100644 index e3e04ee48543d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 +++ /dev/null @@ -1 +0,0 @@ -3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 new file mode 100644 index 0000000000000..2bc7fedb12a50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-e2c7e9b01ec95dfcc685827e24d66775 @@ -0,0 +1,3 @@ +2 1 1 2 +3 1 2 1 +1 1 2 3 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e b/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd rename to sql/hive/src/test/resources/golden/ppr_pushdown2-2-ffa167b63d612a4986d02f5c0623ea7b diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 new file mode 100644 index 0000000000000..3def25c3c65af --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-bd7e1917f8d2cf50c062a22ef3fa15b5 @@ -0,0 +1 @@ +3 1 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 deleted file mode 100644 index e3e04ee48543d..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 +++ /dev/null @@ -1 +0,0 @@ -3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 b/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 new file mode 100644 index 0000000000000..3def25c3c65af --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-21-ece6fe0efc1e658b36ddc10f0653d229 @@ -0,0 +1 @@ +3 1 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c deleted file mode 100644 index 679926f7d738c..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c +++ /dev/null @@ -1 +0,0 @@ -2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-c7753746c190414723d66a8f876499c7 similarity index 100% rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-3-c7753746c190414723d66a8f876499c7 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c new file mode 100644 index 0000000000000..bfde072a7963c --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-1886675984606b6c972c4a26dca6fd2c @@ -0,0 +1 @@ +2 2 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 deleted file mode 100644 index fcc7be2cb12aa..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 +++ /dev/null @@ -1 +0,0 @@ -22 22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 new file mode 100644 index 0000000000000..38212d1943095 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-33b137b28e7246ec3c2acb937c638910 @@ -0,0 +1 @@ +22 22 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 b/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba b/sql/hive/src/test/resources/golden/ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867 similarity index 100% rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba rename to sql/hive/src/test/resources/golden/ppr_pushdown2-6-b5a2518af801f95fe52a75dfc1d3e867 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 b/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 b/sql/hive/src/test/resources/golden/ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b similarity index 100% rename from sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-7-e89a8d1f66fdf9ce68f345de1f728c5b diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 rename to sql/hive/src/test/resources/golden/ppr_pushdown2-8-4507a3f200b3ce384191c91acd324dc7 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 deleted file mode 100644 index 3ebc6d7fa2375..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 +++ /dev/null @@ -1 +0,0 @@ -2 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 deleted file mode 100644 index a4544ab84afa3..0000000000000 --- a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 +++ /dev/null @@ -1 +0,0 @@ -1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 new file mode 100644 index 0000000000000..699fa0cd95c4f --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-c86940e70f705f50e9091c257ee2bb40 @@ -0,0 +1 @@ +2 2 1 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 b/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 index e22f840876d2e..546fd0b0e1051 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 @@ -37,4 +37,4 @@ 5 val_5 2008-04-09 12 2 val_2 2008-04-09 12 5 val_5 2008-04-09 12 -9 val_9 2008-04-09 12 \ No newline at end of file +9 val_9 2008-04-09 12 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb b/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb index 355ed1617e200..2857cdf0aba86 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb @@ -1997,4 +1997,4 @@ 403 val_403 2008-04-09 12 400 val_400 2008-04-09 12 200 val_200 2008-04-09 12 -97 val_97 2008-04-09 12 \ No newline at end of file +97 val_97 2008-04-09 12 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f b/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f index 57005044dde38..7f3ca6e01ea06 100644 --- a/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f @@ -1997,4 +1997,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 b/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a b/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a deleted file mode 100644 index 943e1be13b615..0000000000000 --- a/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a +++ /dev/null @@ -1,10 +0,0 @@ -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 0 val_0 -0 val_0 2 val_2 diff --git a/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 b/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 deleted file mode 100644 index 951e74db0fe23..0000000000000 --- a/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 +++ /dev/null @@ -1,10 +0,0 @@ -0 0.0 -2 2.0 -4 4.0 -5 15.0 -8 8.0 -9 9.0 -10 10.0 -11 11.0 -12 24.0 -15 30.0 diff --git a/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 +++ b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f b/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 b/sql/hive/src/test/resources/golden/progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 rename to sql/hive/src/test/resources/golden/progress_1-2-b6c8c8fc9df98af4dead5efabf5f162c diff --git a/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 index 0b3e0a69a8c79..e9c02dad1826a 100644 --- a/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 +++ b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 @@ -1 +1 @@ -5000 \ No newline at end of file +5000 diff --git a/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 index eb63636d0bfd4..41b92dbf0214a 100644 --- a/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 +++ b/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1388801768, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388801769, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801769}) \ No newline at end of file +Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1413890256, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890256, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d index 964ede006ad21..27254dd97006d 100644 --- a/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d index 68b03670f5f80..4d09f8c357ea7 100644 --- a/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=OFFLINE,NO_DROP, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=OFFLINE,NO_DROP, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d index ec18ce7411535..dbc128d2a1d8c 100644 --- a/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801770, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1388801770}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, PROTECT_MODE=NO_DROP, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d index a6202ac5fc0eb..7774c774cc8a1 100644 --- a/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d +++ b/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1388801768, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1388801771, transient_lastDdlTime=1388801771}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl2, dbName:default, owner:marmbrus, createTime:1413890256, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:p, type:string, comment:null)], parameters:{last_modified_by=marmbrus, last_modified_time=1413890256, transient_lastDdlTime=1413890256}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 index 0a6cebbbd0b82..567b9b3a5d228 100644 --- a/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 +++ b/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 @@ -1,9 +1,9 @@ -col string None -p string None +col string +p string # Partition Information # col_name data_type comment -p string None +p string -Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1388801768, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{last_modified_by=marmbrus, last_modified_time=1388801783, transient_lastDdlTime=1388801783}) \ No newline at end of file +Detailed Partition Information Partition(values:[p1], dbName:default, tableName:tbl2, createTime:1413890256, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null), FieldSchema(name:p, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl2/p=p1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890263, transient_lastDdlTime=1413890263, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}) diff --git a/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 index a1ff1f8341f30..be56722166fe4 100644 --- a/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 +++ b/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 @@ -1,3 +1,3 @@ -col string None +col string -Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1388801746, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388801756, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1388801756}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1413890242, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890249, PROTECT_MODE=OFFLINE, transient_lastDdlTime=1413890249, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 index 9098c9419556b..a3c3c67860fdf 100644 --- a/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 +++ b/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 @@ -1,3 +1,3 @@ -col string None +col string -Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1388801746, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse3973694235577030193/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=marmbrus, last_modified_time=1388801756, transient_lastDdlTime=1388801756}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:tbl1, dbName:default, owner:marmbrus, createTime:1413890242, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:col, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tbl1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=0, last_modified_by=marmbrus, last_modified_time=1413890249, transient_lastDdlTime=1413890249, COLUMN_STATS_ACCURATE=false, totalSize=0, numRows=-1, rawDataSize=-1}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a b/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a index 3a2f20c637883..e3c6c66098c10 100644 --- a/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a +++ b/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a @@ -20,4 +20,4 @@ 17 val_17 2000-04-09 18 val_18 2000-04-09 18 val_18 2000-04-09 -19 val_19 2000-04-09 \ No newline at end of file +19 val_19 2000-04-09 diff --git a/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 index 2f3fe0189d2c6..10db2ad303244 100644 --- a/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 +++ b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 @@ -1 +1 @@ -aa; \ No newline at end of file +aa; diff --git a/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb index b5b5773c405b4..e0b3f1b09bd18 100644 --- a/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb +++ b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb @@ -1 +1 @@ -bb \ No newline at end of file +bb diff --git a/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 index 2652f5f42c003..46c1d6125b7b4 100644 --- a/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 +++ b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 @@ -1 +1 @@ -cc \ No newline at end of file +cc diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/quote2-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/quote2-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 b/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 b/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 deleted file mode 100644 index ed1f53a6588d0..0000000000000 --- a/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 +++ /dev/null @@ -1 +0,0 @@ -abc abc abc' abc" abc\ abc\ abc\' abc\" abc\\ abc\\ abc\\' abc\\" abc\\\ abc\\\ abc""""\ abc''''\ awk '{print NR"\t"$0}' tab tab tab tab \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 rename to sql/hive/src/test/resources/golden/quote2-1-ea1a1d0c5f9a3248afbb65e6632c5118 diff --git a/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 b/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 new file mode 100644 index 0000000000000..4c8564d085999 --- /dev/null +++ b/sql/hive/src/test/resources/golden/quote2-2-34f3c423b2fb1f0b11457f45a60042b9 @@ -0,0 +1 @@ +abc abc abc' abc" abc\ abc\ abc\' abc\" abc\\ abc\\ abc\\' abc\\" abc\\\ abc\\\ abc""""\ abc''''\ awk '{print NR"\t"$0}' tab tab tab tab diff --git a/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e +++ b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d b/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 b/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 deleted file mode 100644 index 73b8edab36833..0000000000000 --- a/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 +++ /dev/null @@ -1,56 +0,0 @@ -409 val_409 -429 val_429 -209 val_209 -153 val_153 -203 val_203 -170 val_170 -489 val_489 -378 val_378 -221 val_221 -498 val_498 -469 val_469 -176 val_176 -176 val_176 -384 val_384 -217 val_217 -431 val_431 -51 val_51 -288 val_288 -457 val_457 -197 val_197 -77 val_77 -138 val_138 -277 val_277 -224 val_224 -309 val_309 -389 val_389 -331 val_331 -317 val_317 -336 val_336 -42 val_42 -458 val_458 -78 val_78 -453 val_453 -74 val_74 -103 val_103 -467 val_467 -202 val_202 -469 val_469 -44 val_44 -454 val_454 -70 val_70 -491 val_491 -199 val_199 -169 val_169 -310 val_310 -233 val_233 -133 val_133 -26 val_26 -134 val_134 -18 val_18 -298 val_298 -348 val_348 -469 val_469 -37 val_37 -152 val_152 -400 val_400 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 deleted file mode 100644 index e8738e9c837ee..0000000000000 --- a/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 +++ /dev/null @@ -1,116 +0,0 @@ -103 val_103 2008-04-08 11 -118 val_118 2008-04-08 12 -119 val_119 2008-04-08 12 -119 val_119 2008-04-08 12 -126 val_126 2008-04-08 12 -131 val_131 2008-04-08 12 -133 val_133 2008-04-08 11 -134 val_134 2008-04-08 11 -138 val_138 2008-04-08 11 -143 val_143 2008-04-08 12 -152 val_152 2008-04-08 11 -153 val_153 2008-04-08 11 -162 val_162 2008-04-08 12 -169 val_169 2008-04-08 11 -170 val_170 2008-04-08 11 -175 val_175 2008-04-08 12 -176 val_176 2008-04-08 11 -176 val_176 2008-04-08 11 -18 val_18 2008-04-08 11 -18 val_18 2008-04-08 12 -191 val_191 2008-04-08 12 -197 val_197 2008-04-08 11 -199 val_199 2008-04-08 11 -200 val_200 2008-04-08 12 -201 val_201 2008-04-08 12 -202 val_202 2008-04-08 11 -203 val_203 2008-04-08 11 -209 val_209 2008-04-08 11 -214 val_214 2008-04-08 12 -217 val_217 2008-04-08 11 -218 val_218 2008-04-08 12 -221 val_221 2008-04-08 11 -223 val_223 2008-04-08 12 -224 val_224 2008-04-08 11 -229 val_229 2008-04-08 12 -230 val_230 2008-04-08 12 -233 val_233 2008-04-08 11 -233 val_233 2008-04-08 12 -237 val_237 2008-04-08 12 -238 val_238 2008-04-08 12 -256 val_256 2008-04-08 12 -26 val_26 2008-04-08 11 -265 val_265 2008-04-08 12 -273 val_273 2008-04-08 12 -277 val_277 2008-04-08 11 -277 val_277 2008-04-08 12 -280 val_280 2008-04-08 12 -286 val_286 2008-04-08 12 -288 val_288 2008-04-08 11 -298 val_298 2008-04-08 11 -309 val_309 2008-04-08 11 -309 val_309 2008-04-08 12 -310 val_310 2008-04-08 11 -317 val_317 2008-04-08 11 -322 val_322 2008-04-08 12 -323 val_323 2008-04-08 12 -325 val_325 2008-04-08 12 -331 val_331 2008-04-08 11 -332 val_332 2008-04-08 12 -336 val_336 2008-04-08 11 -336 val_336 2008-04-08 12 -339 val_339 2008-04-08 12 -341 val_341 2008-04-08 12 -342 val_342 2008-04-08 12 -348 val_348 2008-04-08 11 -348 val_348 2008-04-08 12 -35 val_35 2008-04-08 12 -364 val_364 2008-04-08 12 -37 val_37 2008-04-08 11 -378 val_378 2008-04-08 11 -384 val_384 2008-04-08 11 -389 val_389 2008-04-08 11 -400 val_400 2008-04-08 11 -403 val_403 2008-04-08 12 -407 val_407 2008-04-08 12 -409 val_409 2008-04-08 11 -417 val_417 2008-04-08 12 -42 val_42 2008-04-08 11 -424 val_424 2008-04-08 12 -429 val_429 2008-04-08 11 -429 val_429 2008-04-08 12 -430 val_430 2008-04-08 12 -431 val_431 2008-04-08 11 -432 val_432 2008-04-08 12 -44 val_44 2008-04-08 11 -453 val_453 2008-04-08 11 -454 val_454 2008-04-08 11 -457 val_457 2008-04-08 11 -457 val_457 2008-04-08 12 -458 val_458 2008-04-08 11 -466 val_466 2008-04-08 12 -467 val_467 2008-04-08 11 -469 val_469 2008-04-08 11 -469 val_469 2008-04-08 11 -469 val_469 2008-04-08 11 -47 val_47 2008-04-08 12 -470 val_470 2008-04-08 12 -489 val_489 2008-04-08 11 -491 val_491 2008-04-08 11 -496 val_496 2008-04-08 12 -498 val_498 2008-04-08 11 -498 val_498 2008-04-08 12 -51 val_51 2008-04-08 11 -58 val_58 2008-04-08 12 -70 val_70 2008-04-08 11 -72 val_72 2008-04-08 12 -74 val_74 2008-04-08 11 -77 val_77 2008-04-08 11 -77 val_77 2008-04-08 12 -78 val_78 2008-04-08 11 -82 val_82 2008-04-08 12 -87 val_87 2008-04-08 12 -90 val_90 2008-04-08 12 -97 val_97 2008-04-08 12 -97 val_97 2008-04-08 12 -98 val_98 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 b/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a b/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b index d1b3011bffd91..2d2bcebee4a8d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b +++ b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b @@ -1,2 +1,2 @@ -key string None -value string None +key string +value string diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 b/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 b/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c b/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 b/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 deleted file mode 100644 index b141fed81f206..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 +++ /dev/null @@ -1 +0,0 @@ -53278638794 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 b/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 deleted file mode 100644 index b141fed81f206..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 +++ /dev/null @@ -1 +0,0 @@ -53278638794 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee b/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 b/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 b/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f b/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 b/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 b/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a b/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 b/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 b/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 deleted file mode 100644 index 8d25d618795b9..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 +++ /dev/null @@ -1 +0,0 @@ --4208881187 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 b/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 deleted file mode 100644 index 8d25d618795b9..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 +++ /dev/null @@ -1 +0,0 @@ --4208881187 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 b/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b b/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f deleted file mode 100644 index 11ff946b46f0f..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f +++ /dev/null @@ -1 +0,0 @@ -14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 b/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 b/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee b/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee index b0135533064c9..2918f17e964c0 100644 --- a/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee +++ b/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee @@ -22,4 +22,4 @@ NULL val_193 406 val_406 NULL NULL -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 b/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 index 06b2b4d7e6d47..b52cff5c472e4 100644 --- a/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 +++ b/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 @@ -6,4 +6,4 @@ NULL NULL 24 val_24 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b deleted file mode 100644 index 60878ffb77064..0000000000000 --- a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b +++ /dev/null @@ -1 +0,0 @@ -238 val_238 diff --git a/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 b/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 b/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db b/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 b/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 b/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 b/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 b/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa b/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b index 726e0947d8302..017e14d2ebed4 100644 --- a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a int None -b int None -c int None \ No newline at end of file +a int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b index 500201be8d922..a92663b0674bf 100644 --- a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a1 int test comment1 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b index 71af16a5fa4d7..899341a881857 100644 --- a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ a2 int test comment2 -b int None -c int None \ No newline at end of file +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b index f5b2e72aeccab..26b38dcc6d855 100644 --- a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a int test comment2 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c index ecafeaea5f61a..85c1918f46567 100644 --- a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c +++ b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c @@ -1,2 +1,2 @@ src -srcpart \ No newline at end of file +srcpart diff --git a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b index 726e0947d8302..017e14d2ebed4 100644 --- a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a int None -b int None -c int None \ No newline at end of file +a int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b index 17127eaec9755..2fbb615dd5994 100644 --- a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a string None -b int None -c int None \ No newline at end of file +a string +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b index bfcefe4d18046..173fbad7b1eb3 100644 --- a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a1 int None -b int None -c int None \ No newline at end of file +a1 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b index c436c39a16b8a..bad9feb96a886 100644 --- a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a2 int None -b int None -c int None \ No newline at end of file +a2 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b index bb1507e7488f5..4f23db53afff2 100644 --- a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None -a int None -c int None \ No newline at end of file +b int +a int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b index 17127eaec9755..2fbb615dd5994 100644 --- a/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a string None -b int None -c int None \ No newline at end of file +a string +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b index 500201be8d922..a92663b0674bf 100644 --- a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a1 int test comment1 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b index 71af16a5fa4d7..899341a881857 100644 --- a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ a2 int test comment2 -b int None -c int None \ No newline at end of file +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b index f5b2e72aeccab..26b38dcc6d855 100644 --- a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None +b int a int test comment2 -c int None \ No newline at end of file +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b index bfcefe4d18046..173fbad7b1eb3 100644 --- a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a1 int None -b int None -c int None \ No newline at end of file +a1 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b index c436c39a16b8a..bad9feb96a886 100644 --- a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -a2 int None -b int None -c int None \ No newline at end of file +a2 int +b int +c int diff --git a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b index bb1507e7488f5..4f23db53afff2 100644 --- a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b +++ b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b @@ -1,3 +1,3 @@ -b int None -a int None -c int None \ No newline at end of file +b int +a int +c int diff --git a/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 b/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 b/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 index 3a57720041fb3..d091388cd5e19 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 +++ b/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 b/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 index 19492fd335bcb..a86e9c5af723e 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 +++ b/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f b/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f index 3a57720041fb3..d091388cd5e19 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f +++ b/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e b/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e index 19492fd335bcb..a86e9c5af723e 100644 --- a/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e +++ b/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e @@ -9,4 +9,4 @@ 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 18 val_18 -18 val_18 18 val_18 \ No newline at end of file +18 val_18 18 val_18 diff --git a/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 b/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 b/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 deleted file mode 100644 index 0bc999a3e6aa2..0000000000000 --- a/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef b/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 b/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 deleted file mode 100644 index 0bc999a3e6aa2..0000000000000 --- a/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 +++ /dev/null @@ -1,2 +0,0 @@ -1 -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 b/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 b/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 b/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 b/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f b/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f deleted file mode 100644 index 9cf5170e82d7d..0000000000000 --- a/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f +++ /dev/null @@ -1 +0,0 @@ -238 val_238 238 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 b/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 b/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 b/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 new file mode 100644 index 0000000000000..a81afd7b04319 --- /dev/null +++ b/sql/hive/src/test/resources/golden/select from thrift based table-0-304c4992f5517febd10f43c57df4da49 @@ -0,0 +1,11 @@ +1712634731 record_0 [0,0,0] ["0","0","0"] [{"myint":0,"mystring":"0","underscore_int":0}] {"key_0":"value_0"} +465985200 record_1 [1,2,3] ["10","100","1000"] [{"myint":1,"mystring":"1","underscore_int":1}] {"key_1":"value_1"} +-751827638 record_2 [2,4,6] ["20","200","2000"] [{"myint":4,"mystring":"8","underscore_int":2}] {"key_2":"value_2"} +477111222 record_3 [3,6,9] ["30","300","3000"] [{"myint":9,"mystring":"27","underscore_int":3}] {"key_3":"value_3"} +-734328909 record_4 [4,8,12] ["40","400","4000"] [{"myint":16,"mystring":"64","underscore_int":4}] {"key_4":"value_4"} +-1952710710 record_5 [5,10,15] ["50","500","5000"] [{"myint":25,"mystring":"125","underscore_int":5}] {"key_5":"value_5"} +1244525190 record_6 [6,12,18] ["60","600","6000"] [{"myint":36,"mystring":"216","underscore_int":6}] {"key_6":"value_6"} +-1461153973 record_7 [7,14,21] ["70","700","7000"] [{"myint":49,"mystring":"343","underscore_int":7}] {"key_7":"value_7"} +1638581578 record_8 [8,16,24] ["80","800","8000"] [{"myint":64,"mystring":"512","underscore_int":8}] {"key_8":"value_8"} +336964413 record_9 [9,18,27] ["90","900","9000"] [{"myint":81,"mystring":"729","underscore_int":9}] {"key_9":"value_9"} +0 NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 index 5f4de85940513..016f64cc26f2a 100644 --- a/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 +++ b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 @@ -1 +1 @@ -0 val_0 \ No newline at end of file +0 val_0 diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e index f35d9602e1a7b..f9aaa4d565f4a 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e +++ b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e @@ -1 +1 @@ -498 \ No newline at end of file +498 diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 +++ b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e index f35d9602e1a7b..f9aaa4d565f4a 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e +++ b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e @@ -1 +1 @@ -498 \ No newline at end of file +498 diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde +++ b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e index f35d9602e1a7b..f9aaa4d565f4a 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e +++ b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e @@ -1 +1 @@ -498 \ No newline at end of file +498 diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f +++ b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e index d00ee7786a57c..e69de29bb2d1d 100644 --- a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e +++ b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e @@ -1,22 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATETABLE (TOK_TABNAME serde_regex) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL host TOK_STRING) (TOK_TABCOL identity TOK_STRING) (TOK_TABCOL user TOK_STRING) (TOK_TABCOL time TOK_STRING) (TOK_TABCOL request TOK_STRING) (TOK_TABCOL status TOK_STRING) (TOK_TABCOL size TOK_INT) (TOK_TABCOL referer TOK_STRING) (TOK_TABCOL agent TOK_STRING)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?"))))) TOK_TBLTEXTFILE) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Create Table Operator: - Create Table - columns: host string, identity string, user string, time string, request string, status string, size int, referer string, agent string - if not exists: false - input format: org.apache.hadoop.mapred.TextInputFormat - # buckets: -1 - output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat - serde name: org.apache.hadoop.hive.serde2.RegexSerDe - serde properties: - input.regex ([^ ]*) ([^ ]*) ([^ ]*) (-|\[[^\]]*\]) ([^ "]*|"[^"]*") (-|[0-9]*) (-|[0-9]*)(?: ([^ "]*|"[^"]*") ([^ "]*|"[^"]*"))? - name: serde_regex - isExternal: false - - diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 index 3e290231c27e2..93cdc5c85645c 100644 --- a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 +++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 @@ -1,5 +1,4 @@ NULL 0 -NULL 0 -1234567890.123456789 -1234567890 -4400 4400 -1255.49 -1255 @@ -11,6 +10,7 @@ NULL 0 -0.3 0 0 0 0 0 +0 0 0.01 0 0.02 0 0.1 0 @@ -18,7 +18,7 @@ NULL 0 0.3 0 0.33 0 0.333 0 -0.9999999999999999999999999 1 +1 1 1 1 1 1 1.12 1 diff --git a/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 b/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 b/sql/hive/src/test/resources/golden/serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 rename to sql/hive/src/test/resources/golden/serde_regex-2-e84d30fcc6cf11e82b54ea63e7d1d611 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 b/sql/hive/src/test/resources/golden/serde_regex-3-3ee9e78ff563d6b48741a41885f92c81 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 rename to sql/hive/src/test/resources/golden/serde_regex-3-3ee9e78ff563d6b48741a41885f92c81 diff --git a/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc b/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 deleted file mode 100644 index da61769c6599d..0000000000000 --- a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 +++ /dev/null @@ -1,22 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATETABLE (TOK_TABNAME serde_regex1) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL key TOK_DECIMAL) (TOK_TABCOL value TOK_INT)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*)"))))) TOK_TBLTEXTFILE) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Create Table Operator: - Create Table - columns: key decimal, value int - if not exists: false - input format: org.apache.hadoop.mapred.TextInputFormat - # buckets: -1 - output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat - serde name: org.apache.hadoop.hive.serde2.RegexSerDe - serde properties: - input.regex ([^ ]*) ([^ ]*) - name: serde_regex1 - isExternal: false - - diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/serde_regex-7-bf456bcf6be7334488424dfeadf27d75 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 rename to sql/hive/src/test/resources/golden/serde_regex-7-bf456bcf6be7334488424dfeadf27d75 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 b/sql/hive/src/test/resources/golden/serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 rename to sql/hive/src/test/resources/golden/serde_regex-8-a4cf34af32b83e40e5c8b6d083938b54 diff --git a/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 b/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b b/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 b/sql/hive/src/test/resources/golden/serde_regex-9-f1175f3322abec6f258dd49a5905bce0 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 rename to sql/hive/src/test/resources/golden/serde_regex-9-f1175f3322abec6f258dd49a5905bce0 diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 b/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 index d1168556e09d4..b1663e9a8c00d 100644 --- a/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 +++ b/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 @@ -1,11 +1,11 @@ myint int from deserializer mystring string from deserializer underscore_int int from deserializer -b string None +b string # Partition Information # col_name data_type comment -b string None +b string -Detailed Table Information Table(tableName:int_string, dbName:default, owner:marmbrus, createTime:1389733035, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/int_string, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:b, type:string, comment:null)], parameters:{transient_lastDdlTime=1389733035}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) \ No newline at end of file +Detailed Table Information Table(tableName:int_string, dbName:default, owner:marmbrus, createTime:1413891326, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/int_string, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:b, type:string, comment:null)], parameters:{transient_lastDdlTime=1413891326}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa b/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa index 0b8f428b24193..f5ec7a9aca8f3 100644 --- a/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa +++ b/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa @@ -1,11 +1,11 @@ myint int from deserializer mystring string from deserializer underscore_int int from deserializer -b string None +b string # Partition Information # col_name data_type comment -b string None +b string -Detailed Partition Information Partition(values:[part1], dbName:default, tableName:int_string, createTime:1389733036, lastAccessTime:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/int_string/b=part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1389733036}) \ No newline at end of file +Detailed Partition Information Partition(values:[part1], dbName:default, tableName:int_string, createTime:1413891326, lastAccessTime:0, sd:StorageDescriptor(cols:[], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/int_string/b=part1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer, parameters:{serialization.class=org.apache.hadoop.hive.serde2.thrift.test.IntString, serialization.format=org.apache.thrift.protocol.TBinaryProtocol}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{transient_lastDdlTime=1413891326}) diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d +++ b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd +++ b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b +++ b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d +++ b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 +++ b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 index 3c1fc128bedce..69c7709aa90cb 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-1-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) CLUSTERED BY ( key) SORTED BY ( @@ -13,6 +13,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132100') + 'transient_lastDdlTime'='1413891329') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 index 2ece813dd7d56..501bb6ab32f25 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-3-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE TABLE tmp_showcrt1( - key smallint, - value float) +CREATE TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'temporary table' CLUSTERED BY ( key) @@ -14,9 +14,14 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( + 'numFiles'='0', 'EXTERNAL'='FALSE', - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132100', - 'transient_lastDdlTime'='1407132100') + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 index 2af657bd29506..6e353675b5ed8 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-5-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'changed comment' CLUSTERED BY ( key) @@ -14,8 +14,13 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132100', - 'transient_lastDdlTime'='1407132100') + 'numFiles'='0', + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 index f793ffb7a0bfd..6e353675b5ed8 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-7-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'changed comment' CLUSTERED BY ( key) @@ -14,8 +14,13 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132101', - 'transient_lastDdlTime'='1407132101') + 'numFiles'='0', + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 index c65aff26a7fc1..da849512f4d8f 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_alter-9-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key smallint, - value float) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` smallint, + `value` float) COMMENT 'changed comment' CLUSTERED BY ( key) @@ -14,8 +14,13 @@ STORED BY WITH SERDEPROPERTIES ( 'serialization.format'='1') LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'last_modified_by'='tianyi', - 'last_modified_time'='1407132101', - 'transient_lastDdlTime'='1407132101') + 'numFiles'='0', + 'last_modified_by'='marmbrus', + 'last_modified_time'='1413891329', + 'transient_lastDdlTime'='1413891329', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 index b5a18368ed85e..90f8415a1c6be 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 +++ b/sql/hive/src/test/resources/golden/show_create_table_db_table-4-b585371b624cbab2616a49f553a870a0 @@ -1,6 +1,6 @@ -CREATE TABLE tmp_feng.tmp_showcrt( - key string, - value int) +CREATE TABLE `tmp_feng.tmp_showcrt`( + `key` string, + `value` int) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' STORED AS INPUTFORMAT @@ -8,6 +8,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_feng.db/tmp_showcrt' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_feng.db/tmp_showcrt' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132107') + 'transient_lastDdlTime'='1413891330') diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 b/sql/hive/src/test/resources/golden/show_create_table_delimited-0-97228478b9925f06726ceebb6571bf34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 index d36ad25dc8273..4ee22e5230316 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_delimited-1-2a91d52719cf4552ebeb867204552a26 @@ -1,7 +1,7 @@ -CREATE TABLE tmp_showcrt1( - key int, - value string, - newvalue bigint) +CREATE TABLE `tmp_showcrt1`( + `key` int, + `value` string, + `newvalue` bigint) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' COLLECTION ITEMS TERMINATED BY '|' @@ -12,6 +12,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/testTempFiles4427612185729633290spark.hive.tmp/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132730') + 'transient_lastDdlTime'='1413891331') diff --git a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 index 9e572c0d7df6a..2a1acca6efb8d 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_partitioned-1-2a91d52719cf4552ebeb867204552a26 @@ -1,9 +1,9 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key string, - newvalue boolean COMMENT 'a new value') +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` string, + `newvalue` boolean COMMENT 'a new value') COMMENT 'temporary table' PARTITIONED BY ( - value bigint COMMENT 'some value') + `value` bigint COMMENT 'some value') ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe' STORED AS INPUTFORMAT @@ -11,6 +11,6 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132112') + 'transient_lastDdlTime'='1413891331') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 index 69a38e1a7b20a..6fda2570b53f1 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-1-2a91d52719cf4552ebeb867204552a26 @@ -1,7 +1,7 @@ -CREATE TABLE tmp_showcrt1( - key int, - value string, - newvalue bigint) +CREATE TABLE `tmp_showcrt1`( + `key` int, + `value` string, + `newvalue` bigint) COMMENT 'temporary table' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' @@ -10,6 +10,11 @@ STORED AS INPUTFORMAT OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132115') + 'numFiles'='0', + 'transient_lastDdlTime'='1413891332', + 'COLUMN_STATS_ACCURATE'='false', + 'totalSize'='0', + 'numRows'='-1', + 'rawDataSize'='-1') diff --git a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 index b4e693dc622fb..cbbbb7b3ce3c7 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 +++ b/sql/hive/src/test/resources/golden/show_create_table_serde-4-2a91d52719cf4552ebeb867204552a26 @@ -1,6 +1,6 @@ -CREATE EXTERNAL TABLE tmp_showcrt1( - key string, - value boolean) +CREATE EXTERNAL TABLE `tmp_showcrt1`( + `key` string, + `value` boolean) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' STORED BY @@ -9,6 +9,6 @@ WITH SERDEPROPERTIES ( 'serialization.format'='$', 'field.delim'=',') LOCATION - 'file:/tmp/sparkHiveWarehouse1280221975983654134/tmp_showcrt1' + 'file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmp_showcrt1' TBLPROPERTIES ( - 'transient_lastDdlTime'='1407132115') + 'transient_lastDdlTime'='1413891332') diff --git a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 index be3fb3ce30960..a721f07bb90b7 100644 --- a/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 +++ b/sql/hive/src/test/resources/golden/show_create_table_view-1-1e931ea3fa6065107859ffbb29bb0ed7 @@ -1 +1 @@ -CREATE VIEW tmp_copy_src AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src` +CREATE VIEW `tmp_copy_src` AS SELECT `src`.`key`, `src`.`value` FROM `default`.`src` diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 index fd33cfcc9ab20..442b05b4cf4ee 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 @@ -1 +1 @@ -concat \ No newline at end of file +concat diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d index fd33cfcc9ab20..442b05b4cf4ee 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d @@ -1 +1 @@ -concat \ No newline at end of file +concat diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 index a3998eeeee623..b576089faa484 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 @@ -1 +1 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN \ No newline at end of file +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 index a3998eeeee623..b576089faa484 100644 --- a/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 @@ -1 +1 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN \ No newline at end of file +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN diff --git a/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 index 9d40ffaef5862..3049cd6243ad8 100644 --- a/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 +++ b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 @@ -32,6 +32,7 @@ case ceil ceiling coalesce +collect_list collect_set compute_stats concat @@ -45,6 +46,7 @@ covar_pop covar_samp create_union cume_dist +current_database date_add date_sub datediff @@ -123,6 +125,7 @@ percentile percentile_approx pi pmod +posexplode positive pow power @@ -189,4 +192,4 @@ xpath_short xpath_string year | -~ \ No newline at end of file +~ diff --git a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 index c62b965cb1559..175795534fff5 100644 --- a/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 +++ b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 @@ -2,6 +2,7 @@ case ceil ceiling coalesce +collect_list collect_set compute_stats concat @@ -14,4 +15,5 @@ count covar_pop covar_samp create_union -cume_dist \ No newline at end of file +cume_dist +current_database diff --git a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c index a56b5a3766c5c..3c25d656bda1c 100644 --- a/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c +++ b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c @@ -1,6 +1,7 @@ assert_true case coalesce +current_database decode e encode @@ -19,6 +20,7 @@ negative ntile parse_url_tuple percentile +posexplode positive regexp_replace reverse @@ -29,4 +31,4 @@ to_date translate ucase variance -xpath_double \ No newline at end of file +xpath_double diff --git a/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 index 86605075c3d25..b5a372a5ff50a 100644 --- a/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 +++ b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 @@ -1,3 +1,3 @@ log log10 -log2 \ No newline at end of file +log2 diff --git a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 index 312f6cdbf68c7..cd2e58d04a4ef 100644 --- a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 +++ b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 @@ -1,4 +1,4 @@ date_add date_sub datediff -to_date \ No newline at end of file +to_date diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b deleted file mode 100644 index e772f4a83fecd..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_full) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 deleted file mode 100644 index d68fbfc9c1e0f..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 +++ /dev/null @@ -1,4 +0,0 @@ -idx_1 show_idx_full key default__show_idx_full_idx_1__ compact -idx_2 show_idx_full value1 default__show_idx_full_idx_2__ compact -idx_comment show_idx_full value2 default__show_idx_full_idx_comment__ compact index comment -idx_compound show_idx_full key, value1 default__show_idx_full_idx_compound__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e deleted file mode 100644 index 94ee57be9dcaf..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_empty) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 deleted file mode 100644 index 7e68a8acb1a87..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_t1) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 b/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 deleted file mode 100644 index 36d22451eba3e..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 +++ /dev/null @@ -1 +0,0 @@ -idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 b/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 deleted file mode 100644 index 7e68a8acb1a87..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_t1) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 b/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 deleted file mode 100644 index 36d22451eba3e..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 +++ /dev/null @@ -1 +0,0 @@ -idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e b/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e deleted file mode 100644 index 4dddeee9a233f..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e +++ /dev/null @@ -1,16 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWINDEXES show_idx_t1 FORMATTED) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Index Operator: - Show Indexes - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 b/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 deleted file mode 100644 index 76e0434294b4f..0000000000000 --- a/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 +++ /dev/null @@ -1,4 +0,0 @@ -idx_name tab_name col_names idx_tab_name idx_type comment - - -idx_t1 show_idx_t1 key default__show_idx_t1_idx_t1__ compact \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a index e9c723bbd136e..8c43153cf66f9 100644 --- a/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a +++ b/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a @@ -1,4 +1,4 @@ ds=2008-04-08/hr=11 ds=2008-04-08/hr=12 ds=2008-04-09/hr=11 -ds=2008-04-09/hr=12 \ No newline at end of file +ds=2008-04-09/hr=12 diff --git a/sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9 similarity index 100% rename from sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 rename to sql/hive/src/test/resources/golden/show_partitions-1-e69b801a3c6c5f6692050bcdb0e31db9 diff --git a/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 b/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 deleted file mode 100644 index 19b4a62499762..0000000000000 --- a/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 +++ /dev/null @@ -1,2 +0,0 @@ -ds=2008-04-08/hr=11 -ds=2008-04-09/hr=11 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca b/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca deleted file mode 100644 index f3614273fa8fa..0000000000000 --- a/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca +++ /dev/null @@ -1,2 +0,0 @@ -ds=2008-04-08/hr=11 -ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 b/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 new file mode 100644 index 0000000000000..8b3fd053b6fb6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_partitions-2-e94d4100cb64c67f1127b4e255d28ae0 @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-09/hr=11 diff --git a/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 b/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 deleted file mode 100644 index 0cdd3e8594c59..0000000000000 --- a/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 +++ /dev/null @@ -1 +0,0 @@ -ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 b/sql/hive/src/test/resources/golden/show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca similarity index 100% rename from sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 rename to sql/hive/src/test/resources/golden/show_partitions-3-a1bde7c2c040b4d45ddceac9983c2ca diff --git a/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 b/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 new file mode 100644 index 0000000000000..dbd11ad78405b --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_partitions-4-9e3f80cb6ed9883c715ea8aa3f391d70 @@ -0,0 +1 @@ +ds=2008-04-08/hr=12 diff --git a/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 b/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce b/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 b/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 b/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c b/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c deleted file mode 100644 index 60f7943eda4a9..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c +++ /dev/null @@ -1,3 +0,0 @@ -bar -baz -foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 b/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 deleted file mode 100644 index 4ffc580e2b8f3..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 +++ /dev/null @@ -1,4 +0,0 @@ -shtb_test1 -shtb_test2 -src -srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c b/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c deleted file mode 100644 index 60f7943eda4a9..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c +++ /dev/null @@ -1,3 +0,0 @@ -bar -baz -foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe b/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe deleted file mode 100644 index 4ffc580e2b8f3..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe +++ /dev/null @@ -1,4 +0,0 @@ -shtb_test1 -shtb_test2 -src -srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 b/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab b/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 b/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 b/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 b/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 deleted file mode 100644 index 916ac1482c061..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 +++ /dev/null @@ -1,18 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWTABLES 'shtb_*') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Table Operator: - Show Tables - database name: default - pattern: shtb_* - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 b/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 deleted file mode 100644 index 19102815663d2..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 +++ /dev/null @@ -1 +0,0 @@ -foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c b/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c deleted file mode 100644 index b67b816ee4b45..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c +++ /dev/null @@ -1,2 +0,0 @@ -shtb_test1 -shtb_test2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 b/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 deleted file mode 100644 index 8f06e234b2a6e..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 +++ /dev/null @@ -1,18 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWTABLES 'shtb_test1|shtb_test2') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Table Operator: - Show Tables - database name: default - pattern: shtb_test1|shtb_test2 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d b/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d deleted file mode 100644 index b67b816ee4b45..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d +++ /dev/null @@ -1,2 +0,0 @@ -shtb_test1 -shtb_test2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 b/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c b/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 b/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa b/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa deleted file mode 100644 index bec424bb026e9..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa +++ /dev/null @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOW_TABLESTATUS `src` default) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 b/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 b/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d b/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 b/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 b/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 b/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c b/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 b/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 deleted file mode 100644 index f8b64f6056ea7..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:srcpart -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/srcpart/ds=2008-04-08/hr=11 -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:true -partitionColumns:struct partition_columns { string ds, string hr} -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733249000 diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 b/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 deleted file mode 100644 index 9392b7dc686f6..0000000000000 --- a/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 +++ /dev/null @@ -1,14 +0,0 @@ -tableName:src -owner:marmbrus -location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4281266679489657486/src -inputformat:org.apache.hadoop.mapred.TextInputFormat -outputformat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -columns:struct columns { i32 key, string value} -partitioned:false -partitionColumns: -totalNumberFiles:1 -totalFileSize:5812 -maxFileSize:5812 -minFileSize:5812 -lastAccessTime:0 -lastUpdateTime:1389733248000 diff --git a/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 b/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 index ce1a3441a1bc0..d882eea53ca3c 100644 --- a/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 +++ b/sql/hive/src/test/resources/golden/show_tblproperties-4-6c63215ea599f6533666c4d70606b139 @@ -1,6 +1,10 @@ - -last_modified_by ocquery -last_modified_time 1408598216 +numFiles 0 +last_modified_by marmbrus +last_modified_time 1413891337 tmp true -transient_lastDdlTime 1408598216 +transient_lastDdlTime 1413891337 +COLUMN_STATS_ACCURATE false +totalSize 0 +numRows -1 bar bar value +rawDataSize -1 diff --git a/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 b/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 deleted file mode 100644 index b590724bca78d..0000000000000 --- a/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 +++ /dev/null @@ -1,17 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_SHOWPARTITIONS srcpart) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-0 - Show Partitions Operator: - Show Partitions - table: srcpart - - Stage: Stage-1 - Fetch Operator - limit: -1 - diff --git a/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a deleted file mode 100644 index e9c723bbd136e..0000000000000 --- a/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a +++ /dev/null @@ -1,4 +0,0 @@ -ds=2008-04-08/hr=11 -ds=2008-04-08/hr=12 -ds=2008-04-09/hr=11 -ds=2008-04-09/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 b/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a b/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 b/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 b/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 b/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 b/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 b/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 b/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 index 872146532307a..19304c010452e 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 @@ -1 +1 @@ -2 12 2 22 2 12 \ No newline at end of file +2 12 2 22 2 12 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a b/sql/hive/src/test/resources/golden/skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a rename to sql/hive/src/test/resources/golden/skewjoinopt13-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 rename to sql/hive/src/test/resources/golden/skewjoinopt13-6-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 rename to sql/hive/src/test/resources/golden/skewjoinopt13-8-8eb53fb8f05a43ee377aa1c927857e7c diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 b/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 b/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 b/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 b/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 b/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 b/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d b/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 b/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 b/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e b/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 b/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 b/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 b/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa b/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa index ee1bb6b112381..6ca70c5267e65 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa @@ -3,4 +3,4 @@ 8 18 8 18 8 18 8 18 8 28 8 18 -8 28 8 18 \ No newline at end of file +8 28 8 18 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a b/sql/hive/src/test/resources/golden/skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a rename to sql/hive/src/test/resources/golden/skewjoinopt18-4-abf4b7f158999af331d5dbfddf32fa68 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 b/sql/hive/src/test/resources/golden/skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 rename to sql/hive/src/test/resources/golden/skewjoinopt18-8-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e b/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 b/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d b/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 b/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 b/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb b/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f b/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 b/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d b/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 b/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c b/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd b/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 b/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 index acd4039d35669..d3938a35d72f5 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 @@ -1,4 +1,4 @@ 2 1 2 22 3 1 3 13 8 2 8 18 -8 2 8 18 \ No newline at end of file +8 2 8 18 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 rename to sql/hive/src/test/resources/golden/skewjoinopt9-4-c0f14def6a135cc50cba364e810ce28e diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 rename to sql/hive/src/test/resources/golden/skewjoinopt9-6-ade68a23d7b1a4f328623bb5a0f07488 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d b/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d index f608d53f51ebe..826338ea56810 100644 --- a/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d @@ -9,4 +9,4 @@ 8 28 8 18 8 28 8 18 8 28 8 18 -8 28 8 18 \ No newline at end of file +8 28 8 18 diff --git a/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 index 7a442f02e8d7a..1179e20c2847c 100644 --- a/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 +++ b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 @@ -1,3 +1,3 @@ 0 2 0 2 -0 2 \ No newline at end of file +0 2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 index 836ee718649ad..da83658b68646 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 @@ -1 +1 @@ -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa b/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa index 8f358bffec51d..1ab49661a01f4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa @@ -2,4 +2,4 @@ 51 val_3 51 val_30 52 val_4 NULL NULL 53 val_5 NULL NULL -49 val_10 NULL NULL \ No newline at end of file +49 val_10 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 index 6197cf72c3454..ba7969b99d79f 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 @@ -1,4 +1,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 NULL NULL 50 val_25 -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 index d650d44f42404..ad8b511265e20 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 @@ -5,4 +5,4 @@ NULL NULL 50 val_23 NULL NULL 50 val_25 51 val_3 51 val_30 52 val_4 NULL NULL -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 index 836ee718649ad..da83658b68646 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 @@ -1 +1 @@ -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 index 8f358bffec51d..1ab49661a01f4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 @@ -2,4 +2,4 @@ 51 val_3 51 val_30 52 val_4 NULL NULL 53 val_5 NULL NULL -49 val_10 NULL NULL \ No newline at end of file +49 val_10 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b b/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b index 6197cf72c3454..ba7969b99d79f 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b @@ -1,4 +1,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 NULL NULL 50 val_25 -51 val_3 51 val_30 \ No newline at end of file +51 val_3 51 val_30 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae b/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae index d650d44f42404..ad8b511265e20 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae @@ -5,4 +5,4 @@ NULL NULL 50 val_23 NULL NULL 50 val_25 51 val_3 51 val_30 52 val_4 NULL NULL -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 rename to sql/hive/src/test/resources/golden/smb_mapjoin_1-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-3-68d65d622e45f86d4a6c7d1d09ef823b diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d31ad2289181131982ef3e9cd8c6386e diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b b/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f7fb003fa65cadcd0b13cbdd7b355988 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 rename to sql/hive/src/test/resources/golden/smb_mapjoin_10-6-14b8b2e10032ab2d4a0e7a18979cdb59 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa b/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb b/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af deleted file mode 100644 index 8975db9a05036..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af +++ /dev/null @@ -1 +0,0 @@ -293 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b b/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e deleted file mode 100644 index 8975db9a05036..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e +++ /dev/null @@ -1 +0,0 @@ -293 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 index b828077157966..9f4c46e548d04 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 @@ -7,4 +7,4 @@ 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 -2 val_2 2 val_2 \ No newline at end of file +2 val_2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a b/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a index 4a9735f855f96..ec7496a567609 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a @@ -3,4 +3,4 @@ 4 1 5 9 8 1 -9 1 \ No newline at end of file +9 1 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 index 62f9457511f87..1e8b314962144 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 @@ -1 +1 @@ -6 \ No newline at end of file +6 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 index 2ebc6516c7df1..f6b91e0e1f8dd 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 @@ -1 +1 @@ -56 \ No newline at end of file +56 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 index 2edeafb09db00..209e3ef4b6247 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 @@ -1 +1 @@ -20 \ No newline at end of file +20 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a b/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a index b828077157966..9f4c46e548d04 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a @@ -7,4 +7,4 @@ 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 0 val_0 -2 val_2 2 val_2 \ No newline at end of file +2 val_2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f index 471d725e7bfa3..46d384b9dfb9b 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f @@ -7,4 +7,4 @@ 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 -2 2 val_2 2 2 val_2 \ No newline at end of file +2 2 val_2 2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e index 471d725e7bfa3..46d384b9dfb9b 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e @@ -7,4 +7,4 @@ 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 -2 2 val_2 2 2 val_2 \ No newline at end of file +2 2 val_2 2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 index 471d725e7bfa3..46d384b9dfb9b 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 @@ -7,4 +7,4 @@ 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 0 0 val_0 -2 2 val_2 2 2 val_2 \ No newline at end of file +2 2 val_2 2 2 val_2 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a index 25398d9017c7f..a84e60c846ab2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a @@ -1 +1 @@ -4378 \ No newline at end of file +4378 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 index 25398d9017c7f..a84e60c846ab2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 @@ -1 +1 @@ -4378 \ No newline at end of file +4378 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc index 09b5b315bcf45..e0fa4e2d8601d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc @@ -1 +1 @@ -13126 \ No newline at end of file +13126 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a b/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc b/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b b/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb b/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb deleted file mode 100644 index dce6588ca1420..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb +++ /dev/null @@ -1 +0,0 @@ -36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 deleted file mode 100644 index 86ee83a4a2686..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 +++ /dev/null @@ -1 +0,0 @@ -40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f deleted file mode 100644 index d99e90eb9675f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f +++ /dev/null @@ -1 +0,0 @@ -29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d b/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac b/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac index 7b5974818c085..4482f7ff91c62 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac @@ -4,4 +4,4 @@ 49 val_1 49 val_17 49 val_10 49 val_17 49 val_1 49 val_19 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 index 0e1d132524064..cdacc0434caa7 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 @@ -6,4 +6,4 @@ 53 val_5 NULL NULL 49 val_10 49 val_10 49 val_10 49 val_17 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 index 32be455fba8c4..37d71f5b522c6 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 @@ -6,4 +6,4 @@ 49 val_1 49 val_19 49 val_10 49 val_19 NULL NULL 50 val_20 -NULL NULL 50 val_23 \ No newline at end of file +NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca b/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca index 2496c4400b0b9..364a70c242916 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca @@ -8,4 +8,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 51 val_3 NULL NULL 52 val_4 52 val_4 -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 index 7b5974818c085..4482f7ff91c62 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 @@ -4,4 +4,4 @@ 49 val_1 49 val_17 49 val_10 49 val_17 49 val_1 49 val_19 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b b/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b index 0e1d132524064..cdacc0434caa7 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b @@ -6,4 +6,4 @@ 53 val_5 NULL NULL 49 val_10 49 val_10 49 val_10 49 val_17 -49 val_10 49 val_19 \ No newline at end of file +49 val_10 49 val_19 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 index 32be455fba8c4..37d71f5b522c6 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 @@ -6,4 +6,4 @@ 49 val_1 49 val_19 49 val_10 49 val_19 NULL NULL 50 val_20 -NULL NULL 50 val_23 \ No newline at end of file +NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 index 2496c4400b0b9..364a70c242916 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 @@ -8,4 +8,4 @@ NULL NULL 50 val_20 NULL NULL 50 val_23 51 val_3 NULL NULL 52 val_4 52 val_4 -53 val_5 NULL NULL \ No newline at end of file +53 val_5 NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab b/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 rename to sql/hive/src/test/resources/golden/smb_mapjoin_2-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d deleted file mode 100644 index b6e27607fb529..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d +++ /dev/null @@ -1 +0,0 @@ -242 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 deleted file mode 100644 index ce83bd94b3310..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 +++ /dev/null @@ -1 +0,0 @@ -258 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 deleted file mode 100644 index 34251f6b242e7..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 +++ /dev/null @@ -1 +0,0 @@ -247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff b/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 deleted file mode 100644 index eb1f49486af7c..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 +++ /dev/null @@ -1 +0,0 @@ -500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 deleted file mode 100644 index 99bc3d518639f..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 +++ /dev/null @@ -1 +0,0 @@ -253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb b/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc b/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-10-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-11-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f rename to sql/hive/src/test/resources/golden/smb_mapjoin_25-12-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 index 2c05a72679b22..8aa583680ba51 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 @@ -3,4 +3,4 @@ 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 index 0d9bce83ea87c..8f8addd11eabf 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 @@ -4,4 +4,4 @@ 50 val_23 50 val_23 50 val_25 50 val_20 50 val_25 50 val_23 -51 val_30 NULL NULL \ No newline at end of file +51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e b/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e index d4c1adc92802d..c94cb185c6199 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e @@ -7,4 +7,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a b/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a index b62eec8583c92..fb499272e90c5 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a @@ -8,4 +8,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_25 50 val_23 51 val_30 NULL NULL -NULL NULL 52 val_4 \ No newline at end of file +NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 index 2c05a72679b22..8aa583680ba51 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 @@ -3,4 +3,4 @@ 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd b/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd index 0d9bce83ea87c..8f8addd11eabf 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd @@ -4,4 +4,4 @@ 50 val_23 50 val_23 50 val_25 50 val_20 50 val_25 50 val_23 -51 val_30 NULL NULL \ No newline at end of file +51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 index d4c1adc92802d..c94cb185c6199 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 @@ -7,4 +7,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_20 50 val_23 50 val_23 50 val_23 -50 val_25 50 val_23 \ No newline at end of file +50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 index b62eec8583c92..fb499272e90c5 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 @@ -8,4 +8,4 @@ NULL NULL 49 val_19 50 val_25 50 val_20 50 val_25 50 val_23 51 val_30 NULL NULL -NULL NULL 52 val_4 \ No newline at end of file +NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea b/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea rename to sql/hive/src/test/resources/golden/smb_mapjoin_3-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d b/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d index 46af2ffd2b620..31c2549443bb1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d @@ -2,4 +2,4 @@ 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL 53 val_5 NULL NULL NULL NULL -49 val_10 NULL NULL NULL NULL \ No newline at end of file +49 val_10 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 index 9047baeb236e5..fa5860b62f611 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 @@ -3,4 +3,4 @@ NULL NULL NULL NULL 49 val_10 NULL NULL NULL NULL 49 val_17 NULL NULL NULL NULL 49 val_19 NULL NULL NULL NULL 50 val_20 -NULL NULL NULL NULL 50 val_23 \ No newline at end of file +NULL NULL NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 index 47626758b4182..00e10b37e2fa1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 @@ -8,4 +8,4 @@ NULL NULL NULL NULL 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 index 29616d6368661..84486137a37e4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 @@ -3,4 +3,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 index 6f90cbf839656..e2c6bab43530d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 @@ -4,4 +4,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -51 val_3 51 val_30 NULL NULL \ No newline at end of file +51 val_3 51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 index 3b195f9dff935..8ea9abf42f678 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e b/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e index e18ae75d14a37..f0bda77df9202 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e @@ -8,4 +8,4 @@ NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 index 8e51f0864314a..0c9e2268914bd 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 @@ -3,4 +3,4 @@ NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b b/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f index 51676ca2c94e4..c9ca4763cbabe 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f @@ -8,4 +8,4 @@ NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 index 4a125291de271..812d839a1c642 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 index 660e9044e889a..c019550d25827 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 @@ -12,4 +12,4 @@ NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 rename to sql/hive/src/test/resources/golden/smb_mapjoin_4-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 index 46af2ffd2b620..31c2549443bb1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 @@ -2,4 +2,4 @@ 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL 53 val_5 NULL NULL NULL NULL -49 val_10 NULL NULL NULL NULL \ No newline at end of file +49 val_10 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 index 9047baeb236e5..fa5860b62f611 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 @@ -3,4 +3,4 @@ NULL NULL NULL NULL 49 val_10 NULL NULL NULL NULL 49 val_17 NULL NULL NULL NULL 49 val_19 NULL NULL NULL NULL 50 val_20 -NULL NULL NULL NULL 50 val_23 \ No newline at end of file +NULL NULL NULL NULL 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc index 47626758b4182..00e10b37e2fa1 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc @@ -8,4 +8,4 @@ NULL NULL NULL NULL 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d b/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d index 29616d6368661..84486137a37e4 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d @@ -3,4 +3,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a index 6f90cbf839656..e2c6bab43530d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a @@ -4,4 +4,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -51 val_3 51 val_30 NULL NULL \ No newline at end of file +51 val_3 51 val_30 NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b b/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b index 3b195f9dff935..8ea9abf42f678 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_23 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 index e18ae75d14a37..f0bda77df9202 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 @@ -8,4 +8,4 @@ NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 index 8e51f0864314a..0c9e2268914bd 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 @@ -3,4 +3,4 @@ NULL NULL 50 val_20 50 val_23 NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 -NULL NULL 50 val_25 50 val_23 \ No newline at end of file +NULL NULL 50 val_25 50 val_23 diff --git a/sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-3-bd7036a4c0b57349a588b974ffaa502 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 index 51676ca2c94e4..c9ca4763cbabe 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 @@ -8,4 +8,4 @@ NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc index 4a125291de271..812d839a1c642 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc @@ -7,4 +7,4 @@ NULL NULL 50 val_23 50 val_20 NULL NULL 50 val_23 50 val_23 NULL NULL 50 val_25 50 val_20 NULL NULL 50 val_25 50 val_23 -NULL NULL NULL NULL 52 val_4 \ No newline at end of file +NULL NULL NULL NULL 52 val_4 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 index 660e9044e889a..c019550d25827 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 @@ -12,4 +12,4 @@ NULL NULL 50 val_25 50 val_23 51 val_3 51 val_30 NULL NULL 52 val_4 NULL NULL NULL NULL NULL NULL NULL NULL 52 val_4 -53 val_5 NULL NULL NULL NULL \ No newline at end of file +53 val_5 NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-4-22ace1b9a0302d2b8a4aa57a2c2f6423 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-6d835f651b099615df163be284e833de similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 rename to sql/hive/src/test/resources/golden/smb_mapjoin_5-5-6d835f651b099615df163be284e833de diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca index c2e3ea8b0c8e2..b212e93a0a8c2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 -498 val_498 498 val_498 \ No newline at end of file +498 val_498 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca index c2e3ea8b0c8e2..b212e93a0a8c2 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca @@ -1025,4 +1025,4 @@ 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 498 val_498 -498 val_498 498 val_498 \ No newline at end of file +498 val_498 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a index 0e17d179f4167..11bd621866ba8 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a @@ -1 +1 @@ -278697 278697 101852390308 101852390308 \ No newline at end of file +278697 278697 101852390308 101852390308 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 index 4cd5eefea2a45..dfdc3444cc072 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 @@ -497,4 +497,4 @@ NULL NULL 496 val_496 NULL NULL 497 val_497 NULL NULL 498 val_498 NULL NULL 498 val_498 -NULL NULL 498 val_498 \ No newline at end of file +NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a index 4cd5eefea2a45..dfdc3444cc072 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a @@ -497,4 +497,4 @@ NULL NULL 496 val_496 NULL NULL 497 val_497 NULL NULL 498 val_498 NULL NULL 498 val_498 -NULL NULL 498 val_498 \ No newline at end of file +NULL NULL 498 val_498 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e index ca97f45c90026..2e711200bae28 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e @@ -1 +1 @@ -0 130091 0 36210398070 \ No newline at end of file +0 130091 0 36210398070 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a index ca97f45c90026..2e711200bae28 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a @@ -1 +1 @@ -0 130091 0 36210398070 \ No newline at end of file +0 130091 0 36210398070 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff index ca97f45c90026..2e711200bae28 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff @@ -1 +1 @@ -0 130091 0 36210398070 \ No newline at end of file +0 130091 0 36210398070 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb b/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac b/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac rename to sql/hive/src/test/resources/golden/smb_mapjoin_7-8-f983875c44b290b0884a22b6be6adc8 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c b/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c rename to sql/hive/src/test/resources/golden/smb_mapjoin_7-9-84a394d962965e38593883742cc32c0d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 rename to sql/hive/src/test/resources/golden/smb_mapjoin_8-5-eee18fc4192a4aa92a066eb66513be93 diff --git a/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 b/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 +++ b/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd index 8fdd954df9831..2bd5a0a98a36c 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd @@ -1 +1 @@ -22 \ No newline at end of file +22 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a index 67d892c80f493..1f3d8a7a1fc08 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a @@ -1 +1 @@ -1028 \ No newline at end of file +1028 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 index 688eea009d292..d3a7b34f283b1 100644 --- a/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 +++ b/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1390903702, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390903702, numRows=500, totalSize=5812, rawDataSize=5312}) +Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1413893941, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413893941, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}) diff --git a/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c index 4d3a3d8f237b3..965913df89c64 100644 --- a/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c +++ b/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1390903694, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, p3=v3, transient_lastDdlTime=1390903702, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1413893934, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413893934}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c index a18ab93992cc5..b34342d7815e1 100644 --- a/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c +++ b/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c @@ -1,4 +1,4 @@ -key string None -value string None +key string +value string -Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1390903702, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, p3=v3, transient_lastDdlTime=1390903710, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1413893941, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413893948, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 index d861060f13cae..c1e56cd3180cb 100644 --- a/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 +++ b/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1390903719, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1390903719, numRows=500, totalSize=5812, rawDataSize=5312}) +Detailed Partition Information Partition(values:[1], dbName:default, tableName:stats_partitioned, createTime:1413893955, lastAccessTime:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned/ds=1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), parameters:{numFiles=1, transient_lastDdlTime=1413893955, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}) diff --git a/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c index aa4dfe1e5ba39..25d227999c390 100644 --- a/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c +++ b/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c @@ -1,10 +1,10 @@ -key string None -value string None -ds string None +key string +value string +ds string # Partition Information # col_name data_type comment -ds string None +ds string -Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1390903710, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{numPartitions=1, numFiles=1, p3=v3, transient_lastDdlTime=1390903719, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_partitioned, dbName:default, owner:marmbrus, createTime:1413893948, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[FieldSchema(name:ds, type:string, comment:null)], parameters:{transient_lastDdlTime=1413893948}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c index 8a5ae7cda1c57..8b503b045f6ec 100644 --- a/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c +++ b/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c @@ -1,4 +1,4 @@ -key string None -value string None +key string +value string -Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1390903686, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numPartitions=0, numFiles=1, p3=v3, transient_lastDdlTime=1390903694, numRows=500, totalSize=5812, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) +Detailed Table Information Table(tableName:stats_non_partitioned, dbName:default, owner:marmbrus, createTime:1413893927, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:string, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/stats_non_partitioned, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{numFiles=1, transient_lastDdlTime=1413893934, COLUMN_STATS_ACCURATE=true, totalSize=5812, numRows=500, rawDataSize=5312}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e b/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b b/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b deleted file mode 100644 index f87b67cb2006f..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:02 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_t1 -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 4 - numPartitions 4 - numRows 2000 - p3 v3 - rawDataSize 21248 - totalSize 23248 - transient_lastDdlTime 1390904237 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 b/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 b/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 b/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 b/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 deleted file mode 100644 index 19a5e1e042ae7..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 +++ /dev/null @@ -1,34 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:02 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_t1 -Table Type: MANAGED_TABLE -Table Parameters: - p3 v3 - transient_lastDdlTime 1390904222 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e b/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e deleted file mode 100644 index 7ddea146475d1..0000000000000 --- a/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_t1) (TOK_PARTSPEC (TOK_PARTVAL ds) (TOK_PARTVAL hr)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_t1 - TableScan - alias: analyze_t1 - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 b/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b b/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a b/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a deleted file mode 100644 index 5c8878cba5e71..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a +++ /dev/null @@ -1,37 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 15:57:53 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2617911713640090101/stats_partitioned -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390953481 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 b/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa b/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa deleted file mode 100644 index d9ba3814de367..0000000000000 --- a/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa +++ /dev/null @@ -1,37 +0,0 @@ -# col_name data_type comment - -key string None -value string None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 15:57:53 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2617911713640090101/stats_partitioned -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 500 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1390953492 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 b/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 deleted file mode 100644 index bf2a7f452edc3..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 +++ /dev/null @@ -1,6 +0,0 @@ -1 test_part test_Part -2 test_part test_Part -3 test_part test_Part -4 test_part test_Part -5 test_part test_Part -6 test_part test_Part diff --git a/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc b/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa deleted file mode 100644 index 1e8b314962144..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa +++ /dev/null @@ -1 +0,0 @@ -6 diff --git a/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 b/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 deleted file mode 100644 index bf2a7f452edc3..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 +++ /dev/null @@ -1,6 +0,0 @@ -1 test_part test_Part -2 test_part test_Part -3 test_part test_Part -4 test_part test_Part -5 test_part test_Part -6 test_part test_Part diff --git a/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 b/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b b/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b deleted file mode 100644 index cb1c9def371df..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b +++ /dev/null @@ -1,38 +0,0 @@ -# col_name data_type comment - -col1 string None - -# Partition Information -# col_name data_type comment - -pcol1 string None -pcol2 string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:38 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_dst -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 6 - p3 v3 - rawDataSize 6 - totalSize 171 - transient_lastDdlTime 1390904285 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 b/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 b/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 deleted file mode 100644 index 8bc7c14574497..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 +++ /dev/null @@ -1,43 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_LOAD '/Users/marmbrus/workspace/hive/data/files/test.dat' (TOK_TAB (TOK_TABNAME hive_test_src)) local overwrite) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - Stage-2 depends on stages: Stage-1 - -STAGE PLANS: - Stage: Stage-0 - Copy - source: file:/Users/marmbrus/workspace/hive/data/files/test.dat - destination: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10000 - - Stage: Stage-1 - Move Operator - tables: - replace: true - source: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10000 - table: - input format: org.apache.hadoop.mapred.TextInputFormat - output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - properties: - bucket_count -1 - columns col1 - columns.types string - file.inputformat org.apache.hadoop.mapred.TextInputFormat - file.outputformat org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat - location file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_src - name default.hive_test_src - p3 v3 - serialization.ddl struct hive_test_src { string col1} - serialization.format 1 - serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - transient_lastDdlTime 1390904258 - serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - name: default.hive_test_src - tmp directory: file:/tmp/hive-marmbrus/hive_2014-01-28_02-17-38_072_1020656193356018580-1/-ext-10001 - - Stage: Stage-2 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c b/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d b/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d deleted file mode 100644 index 4916ef91b0866..0000000000000 --- a/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -col1 string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:17:38 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/hive_test_src -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 0 - p3 v3 - rawDataSize 0 - totalSize 11 - transient_lastDdlTime 1390904258 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 b/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 b/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 b/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 b/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 deleted file mode 100644 index 1634adfc4d70d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 +++ /dev/null @@ -1,2 +0,0 @@ -ds=2008-12-31/hr=11 -ds=2008-12-31/hr=12 diff --git a/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 b/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 deleted file mode 100644 index d7a8f25b41301..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 +++ /dev/null @@ -1,1000 +0,0 @@ -238 val_238 2008-04-08 11 -86 val_86 2008-04-08 11 -311 val_311 2008-04-08 11 -27 val_27 2008-04-08 11 -165 val_165 2008-04-08 11 -409 val_409 2008-04-08 11 -255 val_255 2008-04-08 11 -278 val_278 2008-04-08 11 -98 val_98 2008-04-08 11 -484 val_484 2008-04-08 11 -265 val_265 2008-04-08 11 -193 val_193 2008-04-08 11 -401 val_401 2008-04-08 11 -150 val_150 2008-04-08 11 -273 val_273 2008-04-08 11 -224 val_224 2008-04-08 11 -369 val_369 2008-04-08 11 -66 val_66 2008-04-08 11 -128 val_128 2008-04-08 11 -213 val_213 2008-04-08 11 -146 val_146 2008-04-08 11 -406 val_406 2008-04-08 11 -429 val_429 2008-04-08 11 -374 val_374 2008-04-08 11 -152 val_152 2008-04-08 11 -469 val_469 2008-04-08 11 -145 val_145 2008-04-08 11 -495 val_495 2008-04-08 11 -37 val_37 2008-04-08 11 -327 val_327 2008-04-08 11 -281 val_281 2008-04-08 11 -277 val_277 2008-04-08 11 -209 val_209 2008-04-08 11 -15 val_15 2008-04-08 11 -82 val_82 2008-04-08 11 -403 val_403 2008-04-08 11 -166 val_166 2008-04-08 11 -417 val_417 2008-04-08 11 -430 val_430 2008-04-08 11 -252 val_252 2008-04-08 11 -292 val_292 2008-04-08 11 -219 val_219 2008-04-08 11 -287 val_287 2008-04-08 11 -153 val_153 2008-04-08 11 -193 val_193 2008-04-08 11 -338 val_338 2008-04-08 11 -446 val_446 2008-04-08 11 -459 val_459 2008-04-08 11 -394 val_394 2008-04-08 11 -237 val_237 2008-04-08 11 -482 val_482 2008-04-08 11 -174 val_174 2008-04-08 11 -413 val_413 2008-04-08 11 -494 val_494 2008-04-08 11 -207 val_207 2008-04-08 11 -199 val_199 2008-04-08 11 -466 val_466 2008-04-08 11 -208 val_208 2008-04-08 11 -174 val_174 2008-04-08 11 -399 val_399 2008-04-08 11 -396 val_396 2008-04-08 11 -247 val_247 2008-04-08 11 -417 val_417 2008-04-08 11 -489 val_489 2008-04-08 11 -162 val_162 2008-04-08 11 -377 val_377 2008-04-08 11 -397 val_397 2008-04-08 11 -309 val_309 2008-04-08 11 -365 val_365 2008-04-08 11 -266 val_266 2008-04-08 11 -439 val_439 2008-04-08 11 -342 val_342 2008-04-08 11 -367 val_367 2008-04-08 11 -325 val_325 2008-04-08 11 -167 val_167 2008-04-08 11 -195 val_195 2008-04-08 11 -475 val_475 2008-04-08 11 -17 val_17 2008-04-08 11 -113 val_113 2008-04-08 11 -155 val_155 2008-04-08 11 -203 val_203 2008-04-08 11 -339 val_339 2008-04-08 11 -0 val_0 2008-04-08 11 -455 val_455 2008-04-08 11 -128 val_128 2008-04-08 11 -311 val_311 2008-04-08 11 -316 val_316 2008-04-08 11 -57 val_57 2008-04-08 11 -302 val_302 2008-04-08 11 -205 val_205 2008-04-08 11 -149 val_149 2008-04-08 11 -438 val_438 2008-04-08 11 -345 val_345 2008-04-08 11 -129 val_129 2008-04-08 11 -170 val_170 2008-04-08 11 -20 val_20 2008-04-08 11 -489 val_489 2008-04-08 11 -157 val_157 2008-04-08 11 -378 val_378 2008-04-08 11 -221 val_221 2008-04-08 11 -92 val_92 2008-04-08 11 -111 val_111 2008-04-08 11 -47 val_47 2008-04-08 11 -72 val_72 2008-04-08 11 -4 val_4 2008-04-08 11 -280 val_280 2008-04-08 11 -35 val_35 2008-04-08 11 -427 val_427 2008-04-08 11 -277 val_277 2008-04-08 11 -208 val_208 2008-04-08 11 -356 val_356 2008-04-08 11 -399 val_399 2008-04-08 11 -169 val_169 2008-04-08 11 -382 val_382 2008-04-08 11 -498 val_498 2008-04-08 11 -125 val_125 2008-04-08 11 -386 val_386 2008-04-08 11 -437 val_437 2008-04-08 11 -469 val_469 2008-04-08 11 -192 val_192 2008-04-08 11 -286 val_286 2008-04-08 11 -187 val_187 2008-04-08 11 -176 val_176 2008-04-08 11 -54 val_54 2008-04-08 11 -459 val_459 2008-04-08 11 -51 val_51 2008-04-08 11 -138 val_138 2008-04-08 11 -103 val_103 2008-04-08 11 -239 val_239 2008-04-08 11 -213 val_213 2008-04-08 11 -216 val_216 2008-04-08 11 -430 val_430 2008-04-08 11 -278 val_278 2008-04-08 11 -176 val_176 2008-04-08 11 -289 val_289 2008-04-08 11 -221 val_221 2008-04-08 11 -65 val_65 2008-04-08 11 -318 val_318 2008-04-08 11 -332 val_332 2008-04-08 11 -311 val_311 2008-04-08 11 -275 val_275 2008-04-08 11 -137 val_137 2008-04-08 11 -241 val_241 2008-04-08 11 -83 val_83 2008-04-08 11 -333 val_333 2008-04-08 11 -180 val_180 2008-04-08 11 -284 val_284 2008-04-08 11 -12 val_12 2008-04-08 11 -230 val_230 2008-04-08 11 -181 val_181 2008-04-08 11 -67 val_67 2008-04-08 11 -260 val_260 2008-04-08 11 -404 val_404 2008-04-08 11 -384 val_384 2008-04-08 11 -489 val_489 2008-04-08 11 -353 val_353 2008-04-08 11 -373 val_373 2008-04-08 11 -272 val_272 2008-04-08 11 -138 val_138 2008-04-08 11 -217 val_217 2008-04-08 11 -84 val_84 2008-04-08 11 -348 val_348 2008-04-08 11 -466 val_466 2008-04-08 11 -58 val_58 2008-04-08 11 -8 val_8 2008-04-08 11 -411 val_411 2008-04-08 11 -230 val_230 2008-04-08 11 -208 val_208 2008-04-08 11 -348 val_348 2008-04-08 11 -24 val_24 2008-04-08 11 -463 val_463 2008-04-08 11 -431 val_431 2008-04-08 11 -179 val_179 2008-04-08 11 -172 val_172 2008-04-08 11 -42 val_42 2008-04-08 11 -129 val_129 2008-04-08 11 -158 val_158 2008-04-08 11 -119 val_119 2008-04-08 11 -496 val_496 2008-04-08 11 -0 val_0 2008-04-08 11 -322 val_322 2008-04-08 11 -197 val_197 2008-04-08 11 -468 val_468 2008-04-08 11 -393 val_393 2008-04-08 11 -454 val_454 2008-04-08 11 -100 val_100 2008-04-08 11 -298 val_298 2008-04-08 11 -199 val_199 2008-04-08 11 -191 val_191 2008-04-08 11 -418 val_418 2008-04-08 11 -96 val_96 2008-04-08 11 -26 val_26 2008-04-08 11 -165 val_165 2008-04-08 11 -327 val_327 2008-04-08 11 -230 val_230 2008-04-08 11 -205 val_205 2008-04-08 11 -120 val_120 2008-04-08 11 -131 val_131 2008-04-08 11 -51 val_51 2008-04-08 11 -404 val_404 2008-04-08 11 -43 val_43 2008-04-08 11 -436 val_436 2008-04-08 11 -156 val_156 2008-04-08 11 -469 val_469 2008-04-08 11 -468 val_468 2008-04-08 11 -308 val_308 2008-04-08 11 -95 val_95 2008-04-08 11 -196 val_196 2008-04-08 11 -288 val_288 2008-04-08 11 -481 val_481 2008-04-08 11 -457 val_457 2008-04-08 11 -98 val_98 2008-04-08 11 -282 val_282 2008-04-08 11 -197 val_197 2008-04-08 11 -187 val_187 2008-04-08 11 -318 val_318 2008-04-08 11 -318 val_318 2008-04-08 11 -409 val_409 2008-04-08 11 -470 val_470 2008-04-08 11 -137 val_137 2008-04-08 11 -369 val_369 2008-04-08 11 -316 val_316 2008-04-08 11 -169 val_169 2008-04-08 11 -413 val_413 2008-04-08 11 -85 val_85 2008-04-08 11 -77 val_77 2008-04-08 11 -0 val_0 2008-04-08 11 -490 val_490 2008-04-08 11 -87 val_87 2008-04-08 11 -364 val_364 2008-04-08 11 -179 val_179 2008-04-08 11 -118 val_118 2008-04-08 11 -134 val_134 2008-04-08 11 -395 val_395 2008-04-08 11 -282 val_282 2008-04-08 11 -138 val_138 2008-04-08 11 -238 val_238 2008-04-08 11 -419 val_419 2008-04-08 11 -15 val_15 2008-04-08 11 -118 val_118 2008-04-08 11 -72 val_72 2008-04-08 11 -90 val_90 2008-04-08 11 -307 val_307 2008-04-08 11 -19 val_19 2008-04-08 11 -435 val_435 2008-04-08 11 -10 val_10 2008-04-08 11 -277 val_277 2008-04-08 11 -273 val_273 2008-04-08 11 -306 val_306 2008-04-08 11 -224 val_224 2008-04-08 11 -309 val_309 2008-04-08 11 -389 val_389 2008-04-08 11 -327 val_327 2008-04-08 11 -242 val_242 2008-04-08 11 -369 val_369 2008-04-08 11 -392 val_392 2008-04-08 11 -272 val_272 2008-04-08 11 -331 val_331 2008-04-08 11 -401 val_401 2008-04-08 11 -242 val_242 2008-04-08 11 -452 val_452 2008-04-08 11 -177 val_177 2008-04-08 11 -226 val_226 2008-04-08 11 -5 val_5 2008-04-08 11 -497 val_497 2008-04-08 11 -402 val_402 2008-04-08 11 -396 val_396 2008-04-08 11 -317 val_317 2008-04-08 11 -395 val_395 2008-04-08 11 -58 val_58 2008-04-08 11 -35 val_35 2008-04-08 11 -336 val_336 2008-04-08 11 -95 val_95 2008-04-08 11 -11 val_11 2008-04-08 11 -168 val_168 2008-04-08 11 -34 val_34 2008-04-08 11 -229 val_229 2008-04-08 11 -233 val_233 2008-04-08 11 -143 val_143 2008-04-08 11 -472 val_472 2008-04-08 11 -322 val_322 2008-04-08 11 -498 val_498 2008-04-08 11 -160 val_160 2008-04-08 11 -195 val_195 2008-04-08 11 -42 val_42 2008-04-08 11 -321 val_321 2008-04-08 11 -430 val_430 2008-04-08 11 -119 val_119 2008-04-08 11 -489 val_489 2008-04-08 11 -458 val_458 2008-04-08 11 -78 val_78 2008-04-08 11 -76 val_76 2008-04-08 11 -41 val_41 2008-04-08 11 -223 val_223 2008-04-08 11 -492 val_492 2008-04-08 11 -149 val_149 2008-04-08 11 -449 val_449 2008-04-08 11 -218 val_218 2008-04-08 11 -228 val_228 2008-04-08 11 -138 val_138 2008-04-08 11 -453 val_453 2008-04-08 11 -30 val_30 2008-04-08 11 -209 val_209 2008-04-08 11 -64 val_64 2008-04-08 11 -468 val_468 2008-04-08 11 -76 val_76 2008-04-08 11 -74 val_74 2008-04-08 11 -342 val_342 2008-04-08 11 -69 val_69 2008-04-08 11 -230 val_230 2008-04-08 11 -33 val_33 2008-04-08 11 -368 val_368 2008-04-08 11 -103 val_103 2008-04-08 11 -296 val_296 2008-04-08 11 -113 val_113 2008-04-08 11 -216 val_216 2008-04-08 11 -367 val_367 2008-04-08 11 -344 val_344 2008-04-08 11 -167 val_167 2008-04-08 11 -274 val_274 2008-04-08 11 -219 val_219 2008-04-08 11 -239 val_239 2008-04-08 11 -485 val_485 2008-04-08 11 -116 val_116 2008-04-08 11 -223 val_223 2008-04-08 11 -256 val_256 2008-04-08 11 -263 val_263 2008-04-08 11 -70 val_70 2008-04-08 11 -487 val_487 2008-04-08 11 -480 val_480 2008-04-08 11 -401 val_401 2008-04-08 11 -288 val_288 2008-04-08 11 -191 val_191 2008-04-08 11 -5 val_5 2008-04-08 11 -244 val_244 2008-04-08 11 -438 val_438 2008-04-08 11 -128 val_128 2008-04-08 11 -467 val_467 2008-04-08 11 -432 val_432 2008-04-08 11 -202 val_202 2008-04-08 11 -316 val_316 2008-04-08 11 -229 val_229 2008-04-08 11 -469 val_469 2008-04-08 11 -463 val_463 2008-04-08 11 -280 val_280 2008-04-08 11 -2 val_2 2008-04-08 11 -35 val_35 2008-04-08 11 -283 val_283 2008-04-08 11 -331 val_331 2008-04-08 11 -235 val_235 2008-04-08 11 -80 val_80 2008-04-08 11 -44 val_44 2008-04-08 11 -193 val_193 2008-04-08 11 -321 val_321 2008-04-08 11 -335 val_335 2008-04-08 11 -104 val_104 2008-04-08 11 -466 val_466 2008-04-08 11 -366 val_366 2008-04-08 11 -175 val_175 2008-04-08 11 -403 val_403 2008-04-08 11 -483 val_483 2008-04-08 11 -53 val_53 2008-04-08 11 -105 val_105 2008-04-08 11 -257 val_257 2008-04-08 11 -406 val_406 2008-04-08 11 -409 val_409 2008-04-08 11 -190 val_190 2008-04-08 11 -406 val_406 2008-04-08 11 -401 val_401 2008-04-08 11 -114 val_114 2008-04-08 11 -258 val_258 2008-04-08 11 -90 val_90 2008-04-08 11 -203 val_203 2008-04-08 11 -262 val_262 2008-04-08 11 -348 val_348 2008-04-08 11 -424 val_424 2008-04-08 11 -12 val_12 2008-04-08 11 -396 val_396 2008-04-08 11 -201 val_201 2008-04-08 11 -217 val_217 2008-04-08 11 -164 val_164 2008-04-08 11 -431 val_431 2008-04-08 11 -454 val_454 2008-04-08 11 -478 val_478 2008-04-08 11 -298 val_298 2008-04-08 11 -125 val_125 2008-04-08 11 -431 val_431 2008-04-08 11 -164 val_164 2008-04-08 11 -424 val_424 2008-04-08 11 -187 val_187 2008-04-08 11 -382 val_382 2008-04-08 11 -5 val_5 2008-04-08 11 -70 val_70 2008-04-08 11 -397 val_397 2008-04-08 11 -480 val_480 2008-04-08 11 -291 val_291 2008-04-08 11 -24 val_24 2008-04-08 11 -351 val_351 2008-04-08 11 -255 val_255 2008-04-08 11 -104 val_104 2008-04-08 11 -70 val_70 2008-04-08 11 -163 val_163 2008-04-08 11 -438 val_438 2008-04-08 11 -119 val_119 2008-04-08 11 -414 val_414 2008-04-08 11 -200 val_200 2008-04-08 11 -491 val_491 2008-04-08 11 -237 val_237 2008-04-08 11 -439 val_439 2008-04-08 11 -360 val_360 2008-04-08 11 -248 val_248 2008-04-08 11 -479 val_479 2008-04-08 11 -305 val_305 2008-04-08 11 -417 val_417 2008-04-08 11 -199 val_199 2008-04-08 11 -444 val_444 2008-04-08 11 -120 val_120 2008-04-08 11 -429 val_429 2008-04-08 11 -169 val_169 2008-04-08 11 -443 val_443 2008-04-08 11 -323 val_323 2008-04-08 11 -325 val_325 2008-04-08 11 -277 val_277 2008-04-08 11 -230 val_230 2008-04-08 11 -478 val_478 2008-04-08 11 -178 val_178 2008-04-08 11 -468 val_468 2008-04-08 11 -310 val_310 2008-04-08 11 -317 val_317 2008-04-08 11 -333 val_333 2008-04-08 11 -493 val_493 2008-04-08 11 -460 val_460 2008-04-08 11 -207 val_207 2008-04-08 11 -249 val_249 2008-04-08 11 -265 val_265 2008-04-08 11 -480 val_480 2008-04-08 11 -83 val_83 2008-04-08 11 -136 val_136 2008-04-08 11 -353 val_353 2008-04-08 11 -172 val_172 2008-04-08 11 -214 val_214 2008-04-08 11 -462 val_462 2008-04-08 11 -233 val_233 2008-04-08 11 -406 val_406 2008-04-08 11 -133 val_133 2008-04-08 11 -175 val_175 2008-04-08 11 -189 val_189 2008-04-08 11 -454 val_454 2008-04-08 11 -375 val_375 2008-04-08 11 -401 val_401 2008-04-08 11 -421 val_421 2008-04-08 11 -407 val_407 2008-04-08 11 -384 val_384 2008-04-08 11 -256 val_256 2008-04-08 11 -26 val_26 2008-04-08 11 -134 val_134 2008-04-08 11 -67 val_67 2008-04-08 11 -384 val_384 2008-04-08 11 -379 val_379 2008-04-08 11 -18 val_18 2008-04-08 11 -462 val_462 2008-04-08 11 -492 val_492 2008-04-08 11 -100 val_100 2008-04-08 11 -298 val_298 2008-04-08 11 -9 val_9 2008-04-08 11 -341 val_341 2008-04-08 11 -498 val_498 2008-04-08 11 -146 val_146 2008-04-08 11 -458 val_458 2008-04-08 11 -362 val_362 2008-04-08 11 -186 val_186 2008-04-08 11 -285 val_285 2008-04-08 11 -348 val_348 2008-04-08 11 -167 val_167 2008-04-08 11 -18 val_18 2008-04-08 11 -273 val_273 2008-04-08 11 -183 val_183 2008-04-08 11 -281 val_281 2008-04-08 11 -344 val_344 2008-04-08 11 -97 val_97 2008-04-08 11 -469 val_469 2008-04-08 11 -315 val_315 2008-04-08 11 -84 val_84 2008-04-08 11 -28 val_28 2008-04-08 11 -37 val_37 2008-04-08 11 -448 val_448 2008-04-08 11 -152 val_152 2008-04-08 11 -348 val_348 2008-04-08 11 -307 val_307 2008-04-08 11 -194 val_194 2008-04-08 11 -414 val_414 2008-04-08 11 -477 val_477 2008-04-08 11 -222 val_222 2008-04-08 11 -126 val_126 2008-04-08 11 -90 val_90 2008-04-08 11 -169 val_169 2008-04-08 11 -403 val_403 2008-04-08 11 -400 val_400 2008-04-08 11 -200 val_200 2008-04-08 11 -97 val_97 2008-04-08 11 -238 val_238 2008-04-08 12 -86 val_86 2008-04-08 12 -311 val_311 2008-04-08 12 -27 val_27 2008-04-08 12 -165 val_165 2008-04-08 12 -409 val_409 2008-04-08 12 -255 val_255 2008-04-08 12 -278 val_278 2008-04-08 12 -98 val_98 2008-04-08 12 -484 val_484 2008-04-08 12 -265 val_265 2008-04-08 12 -193 val_193 2008-04-08 12 -401 val_401 2008-04-08 12 -150 val_150 2008-04-08 12 -273 val_273 2008-04-08 12 -224 val_224 2008-04-08 12 -369 val_369 2008-04-08 12 -66 val_66 2008-04-08 12 -128 val_128 2008-04-08 12 -213 val_213 2008-04-08 12 -146 val_146 2008-04-08 12 -406 val_406 2008-04-08 12 -429 val_429 2008-04-08 12 -374 val_374 2008-04-08 12 -152 val_152 2008-04-08 12 -469 val_469 2008-04-08 12 -145 val_145 2008-04-08 12 -495 val_495 2008-04-08 12 -37 val_37 2008-04-08 12 -327 val_327 2008-04-08 12 -281 val_281 2008-04-08 12 -277 val_277 2008-04-08 12 -209 val_209 2008-04-08 12 -15 val_15 2008-04-08 12 -82 val_82 2008-04-08 12 -403 val_403 2008-04-08 12 -166 val_166 2008-04-08 12 -417 val_417 2008-04-08 12 -430 val_430 2008-04-08 12 -252 val_252 2008-04-08 12 -292 val_292 2008-04-08 12 -219 val_219 2008-04-08 12 -287 val_287 2008-04-08 12 -153 val_153 2008-04-08 12 -193 val_193 2008-04-08 12 -338 val_338 2008-04-08 12 -446 val_446 2008-04-08 12 -459 val_459 2008-04-08 12 -394 val_394 2008-04-08 12 -237 val_237 2008-04-08 12 -482 val_482 2008-04-08 12 -174 val_174 2008-04-08 12 -413 val_413 2008-04-08 12 -494 val_494 2008-04-08 12 -207 val_207 2008-04-08 12 -199 val_199 2008-04-08 12 -466 val_466 2008-04-08 12 -208 val_208 2008-04-08 12 -174 val_174 2008-04-08 12 -399 val_399 2008-04-08 12 -396 val_396 2008-04-08 12 -247 val_247 2008-04-08 12 -417 val_417 2008-04-08 12 -489 val_489 2008-04-08 12 -162 val_162 2008-04-08 12 -377 val_377 2008-04-08 12 -397 val_397 2008-04-08 12 -309 val_309 2008-04-08 12 -365 val_365 2008-04-08 12 -266 val_266 2008-04-08 12 -439 val_439 2008-04-08 12 -342 val_342 2008-04-08 12 -367 val_367 2008-04-08 12 -325 val_325 2008-04-08 12 -167 val_167 2008-04-08 12 -195 val_195 2008-04-08 12 -475 val_475 2008-04-08 12 -17 val_17 2008-04-08 12 -113 val_113 2008-04-08 12 -155 val_155 2008-04-08 12 -203 val_203 2008-04-08 12 -339 val_339 2008-04-08 12 -0 val_0 2008-04-08 12 -455 val_455 2008-04-08 12 -128 val_128 2008-04-08 12 -311 val_311 2008-04-08 12 -316 val_316 2008-04-08 12 -57 val_57 2008-04-08 12 -302 val_302 2008-04-08 12 -205 val_205 2008-04-08 12 -149 val_149 2008-04-08 12 -438 val_438 2008-04-08 12 -345 val_345 2008-04-08 12 -129 val_129 2008-04-08 12 -170 val_170 2008-04-08 12 -20 val_20 2008-04-08 12 -489 val_489 2008-04-08 12 -157 val_157 2008-04-08 12 -378 val_378 2008-04-08 12 -221 val_221 2008-04-08 12 -92 val_92 2008-04-08 12 -111 val_111 2008-04-08 12 -47 val_47 2008-04-08 12 -72 val_72 2008-04-08 12 -4 val_4 2008-04-08 12 -280 val_280 2008-04-08 12 -35 val_35 2008-04-08 12 -427 val_427 2008-04-08 12 -277 val_277 2008-04-08 12 -208 val_208 2008-04-08 12 -356 val_356 2008-04-08 12 -399 val_399 2008-04-08 12 -169 val_169 2008-04-08 12 -382 val_382 2008-04-08 12 -498 val_498 2008-04-08 12 -125 val_125 2008-04-08 12 -386 val_386 2008-04-08 12 -437 val_437 2008-04-08 12 -469 val_469 2008-04-08 12 -192 val_192 2008-04-08 12 -286 val_286 2008-04-08 12 -187 val_187 2008-04-08 12 -176 val_176 2008-04-08 12 -54 val_54 2008-04-08 12 -459 val_459 2008-04-08 12 -51 val_51 2008-04-08 12 -138 val_138 2008-04-08 12 -103 val_103 2008-04-08 12 -239 val_239 2008-04-08 12 -213 val_213 2008-04-08 12 -216 val_216 2008-04-08 12 -430 val_430 2008-04-08 12 -278 val_278 2008-04-08 12 -176 val_176 2008-04-08 12 -289 val_289 2008-04-08 12 -221 val_221 2008-04-08 12 -65 val_65 2008-04-08 12 -318 val_318 2008-04-08 12 -332 val_332 2008-04-08 12 -311 val_311 2008-04-08 12 -275 val_275 2008-04-08 12 -137 val_137 2008-04-08 12 -241 val_241 2008-04-08 12 -83 val_83 2008-04-08 12 -333 val_333 2008-04-08 12 -180 val_180 2008-04-08 12 -284 val_284 2008-04-08 12 -12 val_12 2008-04-08 12 -230 val_230 2008-04-08 12 -181 val_181 2008-04-08 12 -67 val_67 2008-04-08 12 -260 val_260 2008-04-08 12 -404 val_404 2008-04-08 12 -384 val_384 2008-04-08 12 -489 val_489 2008-04-08 12 -353 val_353 2008-04-08 12 -373 val_373 2008-04-08 12 -272 val_272 2008-04-08 12 -138 val_138 2008-04-08 12 -217 val_217 2008-04-08 12 -84 val_84 2008-04-08 12 -348 val_348 2008-04-08 12 -466 val_466 2008-04-08 12 -58 val_58 2008-04-08 12 -8 val_8 2008-04-08 12 -411 val_411 2008-04-08 12 -230 val_230 2008-04-08 12 -208 val_208 2008-04-08 12 -348 val_348 2008-04-08 12 -24 val_24 2008-04-08 12 -463 val_463 2008-04-08 12 -431 val_431 2008-04-08 12 -179 val_179 2008-04-08 12 -172 val_172 2008-04-08 12 -42 val_42 2008-04-08 12 -129 val_129 2008-04-08 12 -158 val_158 2008-04-08 12 -119 val_119 2008-04-08 12 -496 val_496 2008-04-08 12 -0 val_0 2008-04-08 12 -322 val_322 2008-04-08 12 -197 val_197 2008-04-08 12 -468 val_468 2008-04-08 12 -393 val_393 2008-04-08 12 -454 val_454 2008-04-08 12 -100 val_100 2008-04-08 12 -298 val_298 2008-04-08 12 -199 val_199 2008-04-08 12 -191 val_191 2008-04-08 12 -418 val_418 2008-04-08 12 -96 val_96 2008-04-08 12 -26 val_26 2008-04-08 12 -165 val_165 2008-04-08 12 -327 val_327 2008-04-08 12 -230 val_230 2008-04-08 12 -205 val_205 2008-04-08 12 -120 val_120 2008-04-08 12 -131 val_131 2008-04-08 12 -51 val_51 2008-04-08 12 -404 val_404 2008-04-08 12 -43 val_43 2008-04-08 12 -436 val_436 2008-04-08 12 -156 val_156 2008-04-08 12 -469 val_469 2008-04-08 12 -468 val_468 2008-04-08 12 -308 val_308 2008-04-08 12 -95 val_95 2008-04-08 12 -196 val_196 2008-04-08 12 -288 val_288 2008-04-08 12 -481 val_481 2008-04-08 12 -457 val_457 2008-04-08 12 -98 val_98 2008-04-08 12 -282 val_282 2008-04-08 12 -197 val_197 2008-04-08 12 -187 val_187 2008-04-08 12 -318 val_318 2008-04-08 12 -318 val_318 2008-04-08 12 -409 val_409 2008-04-08 12 -470 val_470 2008-04-08 12 -137 val_137 2008-04-08 12 -369 val_369 2008-04-08 12 -316 val_316 2008-04-08 12 -169 val_169 2008-04-08 12 -413 val_413 2008-04-08 12 -85 val_85 2008-04-08 12 -77 val_77 2008-04-08 12 -0 val_0 2008-04-08 12 -490 val_490 2008-04-08 12 -87 val_87 2008-04-08 12 -364 val_364 2008-04-08 12 -179 val_179 2008-04-08 12 -118 val_118 2008-04-08 12 -134 val_134 2008-04-08 12 -395 val_395 2008-04-08 12 -282 val_282 2008-04-08 12 -138 val_138 2008-04-08 12 -238 val_238 2008-04-08 12 -419 val_419 2008-04-08 12 -15 val_15 2008-04-08 12 -118 val_118 2008-04-08 12 -72 val_72 2008-04-08 12 -90 val_90 2008-04-08 12 -307 val_307 2008-04-08 12 -19 val_19 2008-04-08 12 -435 val_435 2008-04-08 12 -10 val_10 2008-04-08 12 -277 val_277 2008-04-08 12 -273 val_273 2008-04-08 12 -306 val_306 2008-04-08 12 -224 val_224 2008-04-08 12 -309 val_309 2008-04-08 12 -389 val_389 2008-04-08 12 -327 val_327 2008-04-08 12 -242 val_242 2008-04-08 12 -369 val_369 2008-04-08 12 -392 val_392 2008-04-08 12 -272 val_272 2008-04-08 12 -331 val_331 2008-04-08 12 -401 val_401 2008-04-08 12 -242 val_242 2008-04-08 12 -452 val_452 2008-04-08 12 -177 val_177 2008-04-08 12 -226 val_226 2008-04-08 12 -5 val_5 2008-04-08 12 -497 val_497 2008-04-08 12 -402 val_402 2008-04-08 12 -396 val_396 2008-04-08 12 -317 val_317 2008-04-08 12 -395 val_395 2008-04-08 12 -58 val_58 2008-04-08 12 -35 val_35 2008-04-08 12 -336 val_336 2008-04-08 12 -95 val_95 2008-04-08 12 -11 val_11 2008-04-08 12 -168 val_168 2008-04-08 12 -34 val_34 2008-04-08 12 -229 val_229 2008-04-08 12 -233 val_233 2008-04-08 12 -143 val_143 2008-04-08 12 -472 val_472 2008-04-08 12 -322 val_322 2008-04-08 12 -498 val_498 2008-04-08 12 -160 val_160 2008-04-08 12 -195 val_195 2008-04-08 12 -42 val_42 2008-04-08 12 -321 val_321 2008-04-08 12 -430 val_430 2008-04-08 12 -119 val_119 2008-04-08 12 -489 val_489 2008-04-08 12 -458 val_458 2008-04-08 12 -78 val_78 2008-04-08 12 -76 val_76 2008-04-08 12 -41 val_41 2008-04-08 12 -223 val_223 2008-04-08 12 -492 val_492 2008-04-08 12 -149 val_149 2008-04-08 12 -449 val_449 2008-04-08 12 -218 val_218 2008-04-08 12 -228 val_228 2008-04-08 12 -138 val_138 2008-04-08 12 -453 val_453 2008-04-08 12 -30 val_30 2008-04-08 12 -209 val_209 2008-04-08 12 -64 val_64 2008-04-08 12 -468 val_468 2008-04-08 12 -76 val_76 2008-04-08 12 -74 val_74 2008-04-08 12 -342 val_342 2008-04-08 12 -69 val_69 2008-04-08 12 -230 val_230 2008-04-08 12 -33 val_33 2008-04-08 12 -368 val_368 2008-04-08 12 -103 val_103 2008-04-08 12 -296 val_296 2008-04-08 12 -113 val_113 2008-04-08 12 -216 val_216 2008-04-08 12 -367 val_367 2008-04-08 12 -344 val_344 2008-04-08 12 -167 val_167 2008-04-08 12 -274 val_274 2008-04-08 12 -219 val_219 2008-04-08 12 -239 val_239 2008-04-08 12 -485 val_485 2008-04-08 12 -116 val_116 2008-04-08 12 -223 val_223 2008-04-08 12 -256 val_256 2008-04-08 12 -263 val_263 2008-04-08 12 -70 val_70 2008-04-08 12 -487 val_487 2008-04-08 12 -480 val_480 2008-04-08 12 -401 val_401 2008-04-08 12 -288 val_288 2008-04-08 12 -191 val_191 2008-04-08 12 -5 val_5 2008-04-08 12 -244 val_244 2008-04-08 12 -438 val_438 2008-04-08 12 -128 val_128 2008-04-08 12 -467 val_467 2008-04-08 12 -432 val_432 2008-04-08 12 -202 val_202 2008-04-08 12 -316 val_316 2008-04-08 12 -229 val_229 2008-04-08 12 -469 val_469 2008-04-08 12 -463 val_463 2008-04-08 12 -280 val_280 2008-04-08 12 -2 val_2 2008-04-08 12 -35 val_35 2008-04-08 12 -283 val_283 2008-04-08 12 -331 val_331 2008-04-08 12 -235 val_235 2008-04-08 12 -80 val_80 2008-04-08 12 -44 val_44 2008-04-08 12 -193 val_193 2008-04-08 12 -321 val_321 2008-04-08 12 -335 val_335 2008-04-08 12 -104 val_104 2008-04-08 12 -466 val_466 2008-04-08 12 -366 val_366 2008-04-08 12 -175 val_175 2008-04-08 12 -403 val_403 2008-04-08 12 -483 val_483 2008-04-08 12 -53 val_53 2008-04-08 12 -105 val_105 2008-04-08 12 -257 val_257 2008-04-08 12 -406 val_406 2008-04-08 12 -409 val_409 2008-04-08 12 -190 val_190 2008-04-08 12 -406 val_406 2008-04-08 12 -401 val_401 2008-04-08 12 -114 val_114 2008-04-08 12 -258 val_258 2008-04-08 12 -90 val_90 2008-04-08 12 -203 val_203 2008-04-08 12 -262 val_262 2008-04-08 12 -348 val_348 2008-04-08 12 -424 val_424 2008-04-08 12 -12 val_12 2008-04-08 12 -396 val_396 2008-04-08 12 -201 val_201 2008-04-08 12 -217 val_217 2008-04-08 12 -164 val_164 2008-04-08 12 -431 val_431 2008-04-08 12 -454 val_454 2008-04-08 12 -478 val_478 2008-04-08 12 -298 val_298 2008-04-08 12 -125 val_125 2008-04-08 12 -431 val_431 2008-04-08 12 -164 val_164 2008-04-08 12 -424 val_424 2008-04-08 12 -187 val_187 2008-04-08 12 -382 val_382 2008-04-08 12 -5 val_5 2008-04-08 12 -70 val_70 2008-04-08 12 -397 val_397 2008-04-08 12 -480 val_480 2008-04-08 12 -291 val_291 2008-04-08 12 -24 val_24 2008-04-08 12 -351 val_351 2008-04-08 12 -255 val_255 2008-04-08 12 -104 val_104 2008-04-08 12 -70 val_70 2008-04-08 12 -163 val_163 2008-04-08 12 -438 val_438 2008-04-08 12 -119 val_119 2008-04-08 12 -414 val_414 2008-04-08 12 -200 val_200 2008-04-08 12 -491 val_491 2008-04-08 12 -237 val_237 2008-04-08 12 -439 val_439 2008-04-08 12 -360 val_360 2008-04-08 12 -248 val_248 2008-04-08 12 -479 val_479 2008-04-08 12 -305 val_305 2008-04-08 12 -417 val_417 2008-04-08 12 -199 val_199 2008-04-08 12 -444 val_444 2008-04-08 12 -120 val_120 2008-04-08 12 -429 val_429 2008-04-08 12 -169 val_169 2008-04-08 12 -443 val_443 2008-04-08 12 -323 val_323 2008-04-08 12 -325 val_325 2008-04-08 12 -277 val_277 2008-04-08 12 -230 val_230 2008-04-08 12 -478 val_478 2008-04-08 12 -178 val_178 2008-04-08 12 -468 val_468 2008-04-08 12 -310 val_310 2008-04-08 12 -317 val_317 2008-04-08 12 -333 val_333 2008-04-08 12 -493 val_493 2008-04-08 12 -460 val_460 2008-04-08 12 -207 val_207 2008-04-08 12 -249 val_249 2008-04-08 12 -265 val_265 2008-04-08 12 -480 val_480 2008-04-08 12 -83 val_83 2008-04-08 12 -136 val_136 2008-04-08 12 -353 val_353 2008-04-08 12 -172 val_172 2008-04-08 12 -214 val_214 2008-04-08 12 -462 val_462 2008-04-08 12 -233 val_233 2008-04-08 12 -406 val_406 2008-04-08 12 -133 val_133 2008-04-08 12 -175 val_175 2008-04-08 12 -189 val_189 2008-04-08 12 -454 val_454 2008-04-08 12 -375 val_375 2008-04-08 12 -401 val_401 2008-04-08 12 -421 val_421 2008-04-08 12 -407 val_407 2008-04-08 12 -384 val_384 2008-04-08 12 -256 val_256 2008-04-08 12 -26 val_26 2008-04-08 12 -134 val_134 2008-04-08 12 -67 val_67 2008-04-08 12 -384 val_384 2008-04-08 12 -379 val_379 2008-04-08 12 -18 val_18 2008-04-08 12 -462 val_462 2008-04-08 12 -492 val_492 2008-04-08 12 -100 val_100 2008-04-08 12 -298 val_298 2008-04-08 12 -9 val_9 2008-04-08 12 -341 val_341 2008-04-08 12 -498 val_498 2008-04-08 12 -146 val_146 2008-04-08 12 -458 val_458 2008-04-08 12 -362 val_362 2008-04-08 12 -186 val_186 2008-04-08 12 -285 val_285 2008-04-08 12 -348 val_348 2008-04-08 12 -167 val_167 2008-04-08 12 -18 val_18 2008-04-08 12 -273 val_273 2008-04-08 12 -183 val_183 2008-04-08 12 -281 val_281 2008-04-08 12 -344 val_344 2008-04-08 12 -97 val_97 2008-04-08 12 -469 val_469 2008-04-08 12 -315 val_315 2008-04-08 12 -84 val_84 2008-04-08 12 -28 val_28 2008-04-08 12 -37 val_37 2008-04-08 12 -448 val_448 2008-04-08 12 -152 val_152 2008-04-08 12 -348 val_348 2008-04-08 12 -307 val_307 2008-04-08 12 -194 val_194 2008-04-08 12 -414 val_414 2008-04-08 12 -477 val_477 2008-04-08 12 -222 val_222 2008-04-08 12 -126 val_126 2008-04-08 12 -90 val_90 2008-04-08 12 -169 val_169 2008-04-08 12 -403 val_403 2008-04-08 12 -400 val_400 2008-04-08 12 -200 val_200 2008-04-08 12 -97 val_97 2008-04-08 12 diff --git a/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 b/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 deleted file mode 100644 index 653516475da22..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 +++ /dev/null @@ -1,1000 +0,0 @@ -238 val_238 2008-12-31 11 -86 val_86 2008-12-31 11 -311 val_311 2008-12-31 11 -27 val_27 2008-12-31 11 -165 val_165 2008-12-31 11 -409 val_409 2008-12-31 11 -255 val_255 2008-12-31 11 -278 val_278 2008-12-31 11 -98 val_98 2008-12-31 11 -484 val_484 2008-12-31 11 -265 val_265 2008-12-31 11 -193 val_193 2008-12-31 11 -401 val_401 2008-12-31 11 -150 val_150 2008-12-31 11 -273 val_273 2008-12-31 11 -224 val_224 2008-12-31 11 -369 val_369 2008-12-31 11 -66 val_66 2008-12-31 11 -128 val_128 2008-12-31 11 -213 val_213 2008-12-31 11 -146 val_146 2008-12-31 11 -406 val_406 2008-12-31 11 -429 val_429 2008-12-31 11 -374 val_374 2008-12-31 11 -152 val_152 2008-12-31 11 -469 val_469 2008-12-31 11 -145 val_145 2008-12-31 11 -495 val_495 2008-12-31 11 -37 val_37 2008-12-31 11 -327 val_327 2008-12-31 11 -281 val_281 2008-12-31 11 -277 val_277 2008-12-31 11 -209 val_209 2008-12-31 11 -15 val_15 2008-12-31 11 -82 val_82 2008-12-31 11 -403 val_403 2008-12-31 11 -166 val_166 2008-12-31 11 -417 val_417 2008-12-31 11 -430 val_430 2008-12-31 11 -252 val_252 2008-12-31 11 -292 val_292 2008-12-31 11 -219 val_219 2008-12-31 11 -287 val_287 2008-12-31 11 -153 val_153 2008-12-31 11 -193 val_193 2008-12-31 11 -338 val_338 2008-12-31 11 -446 val_446 2008-12-31 11 -459 val_459 2008-12-31 11 -394 val_394 2008-12-31 11 -237 val_237 2008-12-31 11 -482 val_482 2008-12-31 11 -174 val_174 2008-12-31 11 -413 val_413 2008-12-31 11 -494 val_494 2008-12-31 11 -207 val_207 2008-12-31 11 -199 val_199 2008-12-31 11 -466 val_466 2008-12-31 11 -208 val_208 2008-12-31 11 -174 val_174 2008-12-31 11 -399 val_399 2008-12-31 11 -396 val_396 2008-12-31 11 -247 val_247 2008-12-31 11 -417 val_417 2008-12-31 11 -489 val_489 2008-12-31 11 -162 val_162 2008-12-31 11 -377 val_377 2008-12-31 11 -397 val_397 2008-12-31 11 -309 val_309 2008-12-31 11 -365 val_365 2008-12-31 11 -266 val_266 2008-12-31 11 -439 val_439 2008-12-31 11 -342 val_342 2008-12-31 11 -367 val_367 2008-12-31 11 -325 val_325 2008-12-31 11 -167 val_167 2008-12-31 11 -195 val_195 2008-12-31 11 -475 val_475 2008-12-31 11 -17 val_17 2008-12-31 11 -113 val_113 2008-12-31 11 -155 val_155 2008-12-31 11 -203 val_203 2008-12-31 11 -339 val_339 2008-12-31 11 -0 val_0 2008-12-31 11 -455 val_455 2008-12-31 11 -128 val_128 2008-12-31 11 -311 val_311 2008-12-31 11 -316 val_316 2008-12-31 11 -57 val_57 2008-12-31 11 -302 val_302 2008-12-31 11 -205 val_205 2008-12-31 11 -149 val_149 2008-12-31 11 -438 val_438 2008-12-31 11 -345 val_345 2008-12-31 11 -129 val_129 2008-12-31 11 -170 val_170 2008-12-31 11 -20 val_20 2008-12-31 11 -489 val_489 2008-12-31 11 -157 val_157 2008-12-31 11 -378 val_378 2008-12-31 11 -221 val_221 2008-12-31 11 -92 val_92 2008-12-31 11 -111 val_111 2008-12-31 11 -47 val_47 2008-12-31 11 -72 val_72 2008-12-31 11 -4 val_4 2008-12-31 11 -280 val_280 2008-12-31 11 -35 val_35 2008-12-31 11 -427 val_427 2008-12-31 11 -277 val_277 2008-12-31 11 -208 val_208 2008-12-31 11 -356 val_356 2008-12-31 11 -399 val_399 2008-12-31 11 -169 val_169 2008-12-31 11 -382 val_382 2008-12-31 11 -498 val_498 2008-12-31 11 -125 val_125 2008-12-31 11 -386 val_386 2008-12-31 11 -437 val_437 2008-12-31 11 -469 val_469 2008-12-31 11 -192 val_192 2008-12-31 11 -286 val_286 2008-12-31 11 -187 val_187 2008-12-31 11 -176 val_176 2008-12-31 11 -54 val_54 2008-12-31 11 -459 val_459 2008-12-31 11 -51 val_51 2008-12-31 11 -138 val_138 2008-12-31 11 -103 val_103 2008-12-31 11 -239 val_239 2008-12-31 11 -213 val_213 2008-12-31 11 -216 val_216 2008-12-31 11 -430 val_430 2008-12-31 11 -278 val_278 2008-12-31 11 -176 val_176 2008-12-31 11 -289 val_289 2008-12-31 11 -221 val_221 2008-12-31 11 -65 val_65 2008-12-31 11 -318 val_318 2008-12-31 11 -332 val_332 2008-12-31 11 -311 val_311 2008-12-31 11 -275 val_275 2008-12-31 11 -137 val_137 2008-12-31 11 -241 val_241 2008-12-31 11 -83 val_83 2008-12-31 11 -333 val_333 2008-12-31 11 -180 val_180 2008-12-31 11 -284 val_284 2008-12-31 11 -12 val_12 2008-12-31 11 -230 val_230 2008-12-31 11 -181 val_181 2008-12-31 11 -67 val_67 2008-12-31 11 -260 val_260 2008-12-31 11 -404 val_404 2008-12-31 11 -384 val_384 2008-12-31 11 -489 val_489 2008-12-31 11 -353 val_353 2008-12-31 11 -373 val_373 2008-12-31 11 -272 val_272 2008-12-31 11 -138 val_138 2008-12-31 11 -217 val_217 2008-12-31 11 -84 val_84 2008-12-31 11 -348 val_348 2008-12-31 11 -466 val_466 2008-12-31 11 -58 val_58 2008-12-31 11 -8 val_8 2008-12-31 11 -411 val_411 2008-12-31 11 -230 val_230 2008-12-31 11 -208 val_208 2008-12-31 11 -348 val_348 2008-12-31 11 -24 val_24 2008-12-31 11 -463 val_463 2008-12-31 11 -431 val_431 2008-12-31 11 -179 val_179 2008-12-31 11 -172 val_172 2008-12-31 11 -42 val_42 2008-12-31 11 -129 val_129 2008-12-31 11 -158 val_158 2008-12-31 11 -119 val_119 2008-12-31 11 -496 val_496 2008-12-31 11 -0 val_0 2008-12-31 11 -322 val_322 2008-12-31 11 -197 val_197 2008-12-31 11 -468 val_468 2008-12-31 11 -393 val_393 2008-12-31 11 -454 val_454 2008-12-31 11 -100 val_100 2008-12-31 11 -298 val_298 2008-12-31 11 -199 val_199 2008-12-31 11 -191 val_191 2008-12-31 11 -418 val_418 2008-12-31 11 -96 val_96 2008-12-31 11 -26 val_26 2008-12-31 11 -165 val_165 2008-12-31 11 -327 val_327 2008-12-31 11 -230 val_230 2008-12-31 11 -205 val_205 2008-12-31 11 -120 val_120 2008-12-31 11 -131 val_131 2008-12-31 11 -51 val_51 2008-12-31 11 -404 val_404 2008-12-31 11 -43 val_43 2008-12-31 11 -436 val_436 2008-12-31 11 -156 val_156 2008-12-31 11 -469 val_469 2008-12-31 11 -468 val_468 2008-12-31 11 -308 val_308 2008-12-31 11 -95 val_95 2008-12-31 11 -196 val_196 2008-12-31 11 -288 val_288 2008-12-31 11 -481 val_481 2008-12-31 11 -457 val_457 2008-12-31 11 -98 val_98 2008-12-31 11 -282 val_282 2008-12-31 11 -197 val_197 2008-12-31 11 -187 val_187 2008-12-31 11 -318 val_318 2008-12-31 11 -318 val_318 2008-12-31 11 -409 val_409 2008-12-31 11 -470 val_470 2008-12-31 11 -137 val_137 2008-12-31 11 -369 val_369 2008-12-31 11 -316 val_316 2008-12-31 11 -169 val_169 2008-12-31 11 -413 val_413 2008-12-31 11 -85 val_85 2008-12-31 11 -77 val_77 2008-12-31 11 -0 val_0 2008-12-31 11 -490 val_490 2008-12-31 11 -87 val_87 2008-12-31 11 -364 val_364 2008-12-31 11 -179 val_179 2008-12-31 11 -118 val_118 2008-12-31 11 -134 val_134 2008-12-31 11 -395 val_395 2008-12-31 11 -282 val_282 2008-12-31 11 -138 val_138 2008-12-31 11 -238 val_238 2008-12-31 11 -419 val_419 2008-12-31 11 -15 val_15 2008-12-31 11 -118 val_118 2008-12-31 11 -72 val_72 2008-12-31 11 -90 val_90 2008-12-31 11 -307 val_307 2008-12-31 11 -19 val_19 2008-12-31 11 -435 val_435 2008-12-31 11 -10 val_10 2008-12-31 11 -277 val_277 2008-12-31 11 -273 val_273 2008-12-31 11 -306 val_306 2008-12-31 11 -224 val_224 2008-12-31 11 -309 val_309 2008-12-31 11 -389 val_389 2008-12-31 11 -327 val_327 2008-12-31 11 -242 val_242 2008-12-31 11 -369 val_369 2008-12-31 11 -392 val_392 2008-12-31 11 -272 val_272 2008-12-31 11 -331 val_331 2008-12-31 11 -401 val_401 2008-12-31 11 -242 val_242 2008-12-31 11 -452 val_452 2008-12-31 11 -177 val_177 2008-12-31 11 -226 val_226 2008-12-31 11 -5 val_5 2008-12-31 11 -497 val_497 2008-12-31 11 -402 val_402 2008-12-31 11 -396 val_396 2008-12-31 11 -317 val_317 2008-12-31 11 -395 val_395 2008-12-31 11 -58 val_58 2008-12-31 11 -35 val_35 2008-12-31 11 -336 val_336 2008-12-31 11 -95 val_95 2008-12-31 11 -11 val_11 2008-12-31 11 -168 val_168 2008-12-31 11 -34 val_34 2008-12-31 11 -229 val_229 2008-12-31 11 -233 val_233 2008-12-31 11 -143 val_143 2008-12-31 11 -472 val_472 2008-12-31 11 -322 val_322 2008-12-31 11 -498 val_498 2008-12-31 11 -160 val_160 2008-12-31 11 -195 val_195 2008-12-31 11 -42 val_42 2008-12-31 11 -321 val_321 2008-12-31 11 -430 val_430 2008-12-31 11 -119 val_119 2008-12-31 11 -489 val_489 2008-12-31 11 -458 val_458 2008-12-31 11 -78 val_78 2008-12-31 11 -76 val_76 2008-12-31 11 -41 val_41 2008-12-31 11 -223 val_223 2008-12-31 11 -492 val_492 2008-12-31 11 -149 val_149 2008-12-31 11 -449 val_449 2008-12-31 11 -218 val_218 2008-12-31 11 -228 val_228 2008-12-31 11 -138 val_138 2008-12-31 11 -453 val_453 2008-12-31 11 -30 val_30 2008-12-31 11 -209 val_209 2008-12-31 11 -64 val_64 2008-12-31 11 -468 val_468 2008-12-31 11 -76 val_76 2008-12-31 11 -74 val_74 2008-12-31 11 -342 val_342 2008-12-31 11 -69 val_69 2008-12-31 11 -230 val_230 2008-12-31 11 -33 val_33 2008-12-31 11 -368 val_368 2008-12-31 11 -103 val_103 2008-12-31 11 -296 val_296 2008-12-31 11 -113 val_113 2008-12-31 11 -216 val_216 2008-12-31 11 -367 val_367 2008-12-31 11 -344 val_344 2008-12-31 11 -167 val_167 2008-12-31 11 -274 val_274 2008-12-31 11 -219 val_219 2008-12-31 11 -239 val_239 2008-12-31 11 -485 val_485 2008-12-31 11 -116 val_116 2008-12-31 11 -223 val_223 2008-12-31 11 -256 val_256 2008-12-31 11 -263 val_263 2008-12-31 11 -70 val_70 2008-12-31 11 -487 val_487 2008-12-31 11 -480 val_480 2008-12-31 11 -401 val_401 2008-12-31 11 -288 val_288 2008-12-31 11 -191 val_191 2008-12-31 11 -5 val_5 2008-12-31 11 -244 val_244 2008-12-31 11 -438 val_438 2008-12-31 11 -128 val_128 2008-12-31 11 -467 val_467 2008-12-31 11 -432 val_432 2008-12-31 11 -202 val_202 2008-12-31 11 -316 val_316 2008-12-31 11 -229 val_229 2008-12-31 11 -469 val_469 2008-12-31 11 -463 val_463 2008-12-31 11 -280 val_280 2008-12-31 11 -2 val_2 2008-12-31 11 -35 val_35 2008-12-31 11 -283 val_283 2008-12-31 11 -331 val_331 2008-12-31 11 -235 val_235 2008-12-31 11 -80 val_80 2008-12-31 11 -44 val_44 2008-12-31 11 -193 val_193 2008-12-31 11 -321 val_321 2008-12-31 11 -335 val_335 2008-12-31 11 -104 val_104 2008-12-31 11 -466 val_466 2008-12-31 11 -366 val_366 2008-12-31 11 -175 val_175 2008-12-31 11 -403 val_403 2008-12-31 11 -483 val_483 2008-12-31 11 -53 val_53 2008-12-31 11 -105 val_105 2008-12-31 11 -257 val_257 2008-12-31 11 -406 val_406 2008-12-31 11 -409 val_409 2008-12-31 11 -190 val_190 2008-12-31 11 -406 val_406 2008-12-31 11 -401 val_401 2008-12-31 11 -114 val_114 2008-12-31 11 -258 val_258 2008-12-31 11 -90 val_90 2008-12-31 11 -203 val_203 2008-12-31 11 -262 val_262 2008-12-31 11 -348 val_348 2008-12-31 11 -424 val_424 2008-12-31 11 -12 val_12 2008-12-31 11 -396 val_396 2008-12-31 11 -201 val_201 2008-12-31 11 -217 val_217 2008-12-31 11 -164 val_164 2008-12-31 11 -431 val_431 2008-12-31 11 -454 val_454 2008-12-31 11 -478 val_478 2008-12-31 11 -298 val_298 2008-12-31 11 -125 val_125 2008-12-31 11 -431 val_431 2008-12-31 11 -164 val_164 2008-12-31 11 -424 val_424 2008-12-31 11 -187 val_187 2008-12-31 11 -382 val_382 2008-12-31 11 -5 val_5 2008-12-31 11 -70 val_70 2008-12-31 11 -397 val_397 2008-12-31 11 -480 val_480 2008-12-31 11 -291 val_291 2008-12-31 11 -24 val_24 2008-12-31 11 -351 val_351 2008-12-31 11 -255 val_255 2008-12-31 11 -104 val_104 2008-12-31 11 -70 val_70 2008-12-31 11 -163 val_163 2008-12-31 11 -438 val_438 2008-12-31 11 -119 val_119 2008-12-31 11 -414 val_414 2008-12-31 11 -200 val_200 2008-12-31 11 -491 val_491 2008-12-31 11 -237 val_237 2008-12-31 11 -439 val_439 2008-12-31 11 -360 val_360 2008-12-31 11 -248 val_248 2008-12-31 11 -479 val_479 2008-12-31 11 -305 val_305 2008-12-31 11 -417 val_417 2008-12-31 11 -199 val_199 2008-12-31 11 -444 val_444 2008-12-31 11 -120 val_120 2008-12-31 11 -429 val_429 2008-12-31 11 -169 val_169 2008-12-31 11 -443 val_443 2008-12-31 11 -323 val_323 2008-12-31 11 -325 val_325 2008-12-31 11 -277 val_277 2008-12-31 11 -230 val_230 2008-12-31 11 -478 val_478 2008-12-31 11 -178 val_178 2008-12-31 11 -468 val_468 2008-12-31 11 -310 val_310 2008-12-31 11 -317 val_317 2008-12-31 11 -333 val_333 2008-12-31 11 -493 val_493 2008-12-31 11 -460 val_460 2008-12-31 11 -207 val_207 2008-12-31 11 -249 val_249 2008-12-31 11 -265 val_265 2008-12-31 11 -480 val_480 2008-12-31 11 -83 val_83 2008-12-31 11 -136 val_136 2008-12-31 11 -353 val_353 2008-12-31 11 -172 val_172 2008-12-31 11 -214 val_214 2008-12-31 11 -462 val_462 2008-12-31 11 -233 val_233 2008-12-31 11 -406 val_406 2008-12-31 11 -133 val_133 2008-12-31 11 -175 val_175 2008-12-31 11 -189 val_189 2008-12-31 11 -454 val_454 2008-12-31 11 -375 val_375 2008-12-31 11 -401 val_401 2008-12-31 11 -421 val_421 2008-12-31 11 -407 val_407 2008-12-31 11 -384 val_384 2008-12-31 11 -256 val_256 2008-12-31 11 -26 val_26 2008-12-31 11 -134 val_134 2008-12-31 11 -67 val_67 2008-12-31 11 -384 val_384 2008-12-31 11 -379 val_379 2008-12-31 11 -18 val_18 2008-12-31 11 -462 val_462 2008-12-31 11 -492 val_492 2008-12-31 11 -100 val_100 2008-12-31 11 -298 val_298 2008-12-31 11 -9 val_9 2008-12-31 11 -341 val_341 2008-12-31 11 -498 val_498 2008-12-31 11 -146 val_146 2008-12-31 11 -458 val_458 2008-12-31 11 -362 val_362 2008-12-31 11 -186 val_186 2008-12-31 11 -285 val_285 2008-12-31 11 -348 val_348 2008-12-31 11 -167 val_167 2008-12-31 11 -18 val_18 2008-12-31 11 -273 val_273 2008-12-31 11 -183 val_183 2008-12-31 11 -281 val_281 2008-12-31 11 -344 val_344 2008-12-31 11 -97 val_97 2008-12-31 11 -469 val_469 2008-12-31 11 -315 val_315 2008-12-31 11 -84 val_84 2008-12-31 11 -28 val_28 2008-12-31 11 -37 val_37 2008-12-31 11 -448 val_448 2008-12-31 11 -152 val_152 2008-12-31 11 -348 val_348 2008-12-31 11 -307 val_307 2008-12-31 11 -194 val_194 2008-12-31 11 -414 val_414 2008-12-31 11 -477 val_477 2008-12-31 11 -222 val_222 2008-12-31 11 -126 val_126 2008-12-31 11 -90 val_90 2008-12-31 11 -169 val_169 2008-12-31 11 -403 val_403 2008-12-31 11 -400 val_400 2008-12-31 11 -200 val_200 2008-12-31 11 -97 val_97 2008-12-31 11 -238 val_238 2008-12-31 12 -86 val_86 2008-12-31 12 -311 val_311 2008-12-31 12 -27 val_27 2008-12-31 12 -165 val_165 2008-12-31 12 -409 val_409 2008-12-31 12 -255 val_255 2008-12-31 12 -278 val_278 2008-12-31 12 -98 val_98 2008-12-31 12 -484 val_484 2008-12-31 12 -265 val_265 2008-12-31 12 -193 val_193 2008-12-31 12 -401 val_401 2008-12-31 12 -150 val_150 2008-12-31 12 -273 val_273 2008-12-31 12 -224 val_224 2008-12-31 12 -369 val_369 2008-12-31 12 -66 val_66 2008-12-31 12 -128 val_128 2008-12-31 12 -213 val_213 2008-12-31 12 -146 val_146 2008-12-31 12 -406 val_406 2008-12-31 12 -429 val_429 2008-12-31 12 -374 val_374 2008-12-31 12 -152 val_152 2008-12-31 12 -469 val_469 2008-12-31 12 -145 val_145 2008-12-31 12 -495 val_495 2008-12-31 12 -37 val_37 2008-12-31 12 -327 val_327 2008-12-31 12 -281 val_281 2008-12-31 12 -277 val_277 2008-12-31 12 -209 val_209 2008-12-31 12 -15 val_15 2008-12-31 12 -82 val_82 2008-12-31 12 -403 val_403 2008-12-31 12 -166 val_166 2008-12-31 12 -417 val_417 2008-12-31 12 -430 val_430 2008-12-31 12 -252 val_252 2008-12-31 12 -292 val_292 2008-12-31 12 -219 val_219 2008-12-31 12 -287 val_287 2008-12-31 12 -153 val_153 2008-12-31 12 -193 val_193 2008-12-31 12 -338 val_338 2008-12-31 12 -446 val_446 2008-12-31 12 -459 val_459 2008-12-31 12 -394 val_394 2008-12-31 12 -237 val_237 2008-12-31 12 -482 val_482 2008-12-31 12 -174 val_174 2008-12-31 12 -413 val_413 2008-12-31 12 -494 val_494 2008-12-31 12 -207 val_207 2008-12-31 12 -199 val_199 2008-12-31 12 -466 val_466 2008-12-31 12 -208 val_208 2008-12-31 12 -174 val_174 2008-12-31 12 -399 val_399 2008-12-31 12 -396 val_396 2008-12-31 12 -247 val_247 2008-12-31 12 -417 val_417 2008-12-31 12 -489 val_489 2008-12-31 12 -162 val_162 2008-12-31 12 -377 val_377 2008-12-31 12 -397 val_397 2008-12-31 12 -309 val_309 2008-12-31 12 -365 val_365 2008-12-31 12 -266 val_266 2008-12-31 12 -439 val_439 2008-12-31 12 -342 val_342 2008-12-31 12 -367 val_367 2008-12-31 12 -325 val_325 2008-12-31 12 -167 val_167 2008-12-31 12 -195 val_195 2008-12-31 12 -475 val_475 2008-12-31 12 -17 val_17 2008-12-31 12 -113 val_113 2008-12-31 12 -155 val_155 2008-12-31 12 -203 val_203 2008-12-31 12 -339 val_339 2008-12-31 12 -0 val_0 2008-12-31 12 -455 val_455 2008-12-31 12 -128 val_128 2008-12-31 12 -311 val_311 2008-12-31 12 -316 val_316 2008-12-31 12 -57 val_57 2008-12-31 12 -302 val_302 2008-12-31 12 -205 val_205 2008-12-31 12 -149 val_149 2008-12-31 12 -438 val_438 2008-12-31 12 -345 val_345 2008-12-31 12 -129 val_129 2008-12-31 12 -170 val_170 2008-12-31 12 -20 val_20 2008-12-31 12 -489 val_489 2008-12-31 12 -157 val_157 2008-12-31 12 -378 val_378 2008-12-31 12 -221 val_221 2008-12-31 12 -92 val_92 2008-12-31 12 -111 val_111 2008-12-31 12 -47 val_47 2008-12-31 12 -72 val_72 2008-12-31 12 -4 val_4 2008-12-31 12 -280 val_280 2008-12-31 12 -35 val_35 2008-12-31 12 -427 val_427 2008-12-31 12 -277 val_277 2008-12-31 12 -208 val_208 2008-12-31 12 -356 val_356 2008-12-31 12 -399 val_399 2008-12-31 12 -169 val_169 2008-12-31 12 -382 val_382 2008-12-31 12 -498 val_498 2008-12-31 12 -125 val_125 2008-12-31 12 -386 val_386 2008-12-31 12 -437 val_437 2008-12-31 12 -469 val_469 2008-12-31 12 -192 val_192 2008-12-31 12 -286 val_286 2008-12-31 12 -187 val_187 2008-12-31 12 -176 val_176 2008-12-31 12 -54 val_54 2008-12-31 12 -459 val_459 2008-12-31 12 -51 val_51 2008-12-31 12 -138 val_138 2008-12-31 12 -103 val_103 2008-12-31 12 -239 val_239 2008-12-31 12 -213 val_213 2008-12-31 12 -216 val_216 2008-12-31 12 -430 val_430 2008-12-31 12 -278 val_278 2008-12-31 12 -176 val_176 2008-12-31 12 -289 val_289 2008-12-31 12 -221 val_221 2008-12-31 12 -65 val_65 2008-12-31 12 -318 val_318 2008-12-31 12 -332 val_332 2008-12-31 12 -311 val_311 2008-12-31 12 -275 val_275 2008-12-31 12 -137 val_137 2008-12-31 12 -241 val_241 2008-12-31 12 -83 val_83 2008-12-31 12 -333 val_333 2008-12-31 12 -180 val_180 2008-12-31 12 -284 val_284 2008-12-31 12 -12 val_12 2008-12-31 12 -230 val_230 2008-12-31 12 -181 val_181 2008-12-31 12 -67 val_67 2008-12-31 12 -260 val_260 2008-12-31 12 -404 val_404 2008-12-31 12 -384 val_384 2008-12-31 12 -489 val_489 2008-12-31 12 -353 val_353 2008-12-31 12 -373 val_373 2008-12-31 12 -272 val_272 2008-12-31 12 -138 val_138 2008-12-31 12 -217 val_217 2008-12-31 12 -84 val_84 2008-12-31 12 -348 val_348 2008-12-31 12 -466 val_466 2008-12-31 12 -58 val_58 2008-12-31 12 -8 val_8 2008-12-31 12 -411 val_411 2008-12-31 12 -230 val_230 2008-12-31 12 -208 val_208 2008-12-31 12 -348 val_348 2008-12-31 12 -24 val_24 2008-12-31 12 -463 val_463 2008-12-31 12 -431 val_431 2008-12-31 12 -179 val_179 2008-12-31 12 -172 val_172 2008-12-31 12 -42 val_42 2008-12-31 12 -129 val_129 2008-12-31 12 -158 val_158 2008-12-31 12 -119 val_119 2008-12-31 12 -496 val_496 2008-12-31 12 -0 val_0 2008-12-31 12 -322 val_322 2008-12-31 12 -197 val_197 2008-12-31 12 -468 val_468 2008-12-31 12 -393 val_393 2008-12-31 12 -454 val_454 2008-12-31 12 -100 val_100 2008-12-31 12 -298 val_298 2008-12-31 12 -199 val_199 2008-12-31 12 -191 val_191 2008-12-31 12 -418 val_418 2008-12-31 12 -96 val_96 2008-12-31 12 -26 val_26 2008-12-31 12 -165 val_165 2008-12-31 12 -327 val_327 2008-12-31 12 -230 val_230 2008-12-31 12 -205 val_205 2008-12-31 12 -120 val_120 2008-12-31 12 -131 val_131 2008-12-31 12 -51 val_51 2008-12-31 12 -404 val_404 2008-12-31 12 -43 val_43 2008-12-31 12 -436 val_436 2008-12-31 12 -156 val_156 2008-12-31 12 -469 val_469 2008-12-31 12 -468 val_468 2008-12-31 12 -308 val_308 2008-12-31 12 -95 val_95 2008-12-31 12 -196 val_196 2008-12-31 12 -288 val_288 2008-12-31 12 -481 val_481 2008-12-31 12 -457 val_457 2008-12-31 12 -98 val_98 2008-12-31 12 -282 val_282 2008-12-31 12 -197 val_197 2008-12-31 12 -187 val_187 2008-12-31 12 -318 val_318 2008-12-31 12 -318 val_318 2008-12-31 12 -409 val_409 2008-12-31 12 -470 val_470 2008-12-31 12 -137 val_137 2008-12-31 12 -369 val_369 2008-12-31 12 -316 val_316 2008-12-31 12 -169 val_169 2008-12-31 12 -413 val_413 2008-12-31 12 -85 val_85 2008-12-31 12 -77 val_77 2008-12-31 12 -0 val_0 2008-12-31 12 -490 val_490 2008-12-31 12 -87 val_87 2008-12-31 12 -364 val_364 2008-12-31 12 -179 val_179 2008-12-31 12 -118 val_118 2008-12-31 12 -134 val_134 2008-12-31 12 -395 val_395 2008-12-31 12 -282 val_282 2008-12-31 12 -138 val_138 2008-12-31 12 -238 val_238 2008-12-31 12 -419 val_419 2008-12-31 12 -15 val_15 2008-12-31 12 -118 val_118 2008-12-31 12 -72 val_72 2008-12-31 12 -90 val_90 2008-12-31 12 -307 val_307 2008-12-31 12 -19 val_19 2008-12-31 12 -435 val_435 2008-12-31 12 -10 val_10 2008-12-31 12 -277 val_277 2008-12-31 12 -273 val_273 2008-12-31 12 -306 val_306 2008-12-31 12 -224 val_224 2008-12-31 12 -309 val_309 2008-12-31 12 -389 val_389 2008-12-31 12 -327 val_327 2008-12-31 12 -242 val_242 2008-12-31 12 -369 val_369 2008-12-31 12 -392 val_392 2008-12-31 12 -272 val_272 2008-12-31 12 -331 val_331 2008-12-31 12 -401 val_401 2008-12-31 12 -242 val_242 2008-12-31 12 -452 val_452 2008-12-31 12 -177 val_177 2008-12-31 12 -226 val_226 2008-12-31 12 -5 val_5 2008-12-31 12 -497 val_497 2008-12-31 12 -402 val_402 2008-12-31 12 -396 val_396 2008-12-31 12 -317 val_317 2008-12-31 12 -395 val_395 2008-12-31 12 -58 val_58 2008-12-31 12 -35 val_35 2008-12-31 12 -336 val_336 2008-12-31 12 -95 val_95 2008-12-31 12 -11 val_11 2008-12-31 12 -168 val_168 2008-12-31 12 -34 val_34 2008-12-31 12 -229 val_229 2008-12-31 12 -233 val_233 2008-12-31 12 -143 val_143 2008-12-31 12 -472 val_472 2008-12-31 12 -322 val_322 2008-12-31 12 -498 val_498 2008-12-31 12 -160 val_160 2008-12-31 12 -195 val_195 2008-12-31 12 -42 val_42 2008-12-31 12 -321 val_321 2008-12-31 12 -430 val_430 2008-12-31 12 -119 val_119 2008-12-31 12 -489 val_489 2008-12-31 12 -458 val_458 2008-12-31 12 -78 val_78 2008-12-31 12 -76 val_76 2008-12-31 12 -41 val_41 2008-12-31 12 -223 val_223 2008-12-31 12 -492 val_492 2008-12-31 12 -149 val_149 2008-12-31 12 -449 val_449 2008-12-31 12 -218 val_218 2008-12-31 12 -228 val_228 2008-12-31 12 -138 val_138 2008-12-31 12 -453 val_453 2008-12-31 12 -30 val_30 2008-12-31 12 -209 val_209 2008-12-31 12 -64 val_64 2008-12-31 12 -468 val_468 2008-12-31 12 -76 val_76 2008-12-31 12 -74 val_74 2008-12-31 12 -342 val_342 2008-12-31 12 -69 val_69 2008-12-31 12 -230 val_230 2008-12-31 12 -33 val_33 2008-12-31 12 -368 val_368 2008-12-31 12 -103 val_103 2008-12-31 12 -296 val_296 2008-12-31 12 -113 val_113 2008-12-31 12 -216 val_216 2008-12-31 12 -367 val_367 2008-12-31 12 -344 val_344 2008-12-31 12 -167 val_167 2008-12-31 12 -274 val_274 2008-12-31 12 -219 val_219 2008-12-31 12 -239 val_239 2008-12-31 12 -485 val_485 2008-12-31 12 -116 val_116 2008-12-31 12 -223 val_223 2008-12-31 12 -256 val_256 2008-12-31 12 -263 val_263 2008-12-31 12 -70 val_70 2008-12-31 12 -487 val_487 2008-12-31 12 -480 val_480 2008-12-31 12 -401 val_401 2008-12-31 12 -288 val_288 2008-12-31 12 -191 val_191 2008-12-31 12 -5 val_5 2008-12-31 12 -244 val_244 2008-12-31 12 -438 val_438 2008-12-31 12 -128 val_128 2008-12-31 12 -467 val_467 2008-12-31 12 -432 val_432 2008-12-31 12 -202 val_202 2008-12-31 12 -316 val_316 2008-12-31 12 -229 val_229 2008-12-31 12 -469 val_469 2008-12-31 12 -463 val_463 2008-12-31 12 -280 val_280 2008-12-31 12 -2 val_2 2008-12-31 12 -35 val_35 2008-12-31 12 -283 val_283 2008-12-31 12 -331 val_331 2008-12-31 12 -235 val_235 2008-12-31 12 -80 val_80 2008-12-31 12 -44 val_44 2008-12-31 12 -193 val_193 2008-12-31 12 -321 val_321 2008-12-31 12 -335 val_335 2008-12-31 12 -104 val_104 2008-12-31 12 -466 val_466 2008-12-31 12 -366 val_366 2008-12-31 12 -175 val_175 2008-12-31 12 -403 val_403 2008-12-31 12 -483 val_483 2008-12-31 12 -53 val_53 2008-12-31 12 -105 val_105 2008-12-31 12 -257 val_257 2008-12-31 12 -406 val_406 2008-12-31 12 -409 val_409 2008-12-31 12 -190 val_190 2008-12-31 12 -406 val_406 2008-12-31 12 -401 val_401 2008-12-31 12 -114 val_114 2008-12-31 12 -258 val_258 2008-12-31 12 -90 val_90 2008-12-31 12 -203 val_203 2008-12-31 12 -262 val_262 2008-12-31 12 -348 val_348 2008-12-31 12 -424 val_424 2008-12-31 12 -12 val_12 2008-12-31 12 -396 val_396 2008-12-31 12 -201 val_201 2008-12-31 12 -217 val_217 2008-12-31 12 -164 val_164 2008-12-31 12 -431 val_431 2008-12-31 12 -454 val_454 2008-12-31 12 -478 val_478 2008-12-31 12 -298 val_298 2008-12-31 12 -125 val_125 2008-12-31 12 -431 val_431 2008-12-31 12 -164 val_164 2008-12-31 12 -424 val_424 2008-12-31 12 -187 val_187 2008-12-31 12 -382 val_382 2008-12-31 12 -5 val_5 2008-12-31 12 -70 val_70 2008-12-31 12 -397 val_397 2008-12-31 12 -480 val_480 2008-12-31 12 -291 val_291 2008-12-31 12 -24 val_24 2008-12-31 12 -351 val_351 2008-12-31 12 -255 val_255 2008-12-31 12 -104 val_104 2008-12-31 12 -70 val_70 2008-12-31 12 -163 val_163 2008-12-31 12 -438 val_438 2008-12-31 12 -119 val_119 2008-12-31 12 -414 val_414 2008-12-31 12 -200 val_200 2008-12-31 12 -491 val_491 2008-12-31 12 -237 val_237 2008-12-31 12 -439 val_439 2008-12-31 12 -360 val_360 2008-12-31 12 -248 val_248 2008-12-31 12 -479 val_479 2008-12-31 12 -305 val_305 2008-12-31 12 -417 val_417 2008-12-31 12 -199 val_199 2008-12-31 12 -444 val_444 2008-12-31 12 -120 val_120 2008-12-31 12 -429 val_429 2008-12-31 12 -169 val_169 2008-12-31 12 -443 val_443 2008-12-31 12 -323 val_323 2008-12-31 12 -325 val_325 2008-12-31 12 -277 val_277 2008-12-31 12 -230 val_230 2008-12-31 12 -478 val_478 2008-12-31 12 -178 val_178 2008-12-31 12 -468 val_468 2008-12-31 12 -310 val_310 2008-12-31 12 -317 val_317 2008-12-31 12 -333 val_333 2008-12-31 12 -493 val_493 2008-12-31 12 -460 val_460 2008-12-31 12 -207 val_207 2008-12-31 12 -249 val_249 2008-12-31 12 -265 val_265 2008-12-31 12 -480 val_480 2008-12-31 12 -83 val_83 2008-12-31 12 -136 val_136 2008-12-31 12 -353 val_353 2008-12-31 12 -172 val_172 2008-12-31 12 -214 val_214 2008-12-31 12 -462 val_462 2008-12-31 12 -233 val_233 2008-12-31 12 -406 val_406 2008-12-31 12 -133 val_133 2008-12-31 12 -175 val_175 2008-12-31 12 -189 val_189 2008-12-31 12 -454 val_454 2008-12-31 12 -375 val_375 2008-12-31 12 -401 val_401 2008-12-31 12 -421 val_421 2008-12-31 12 -407 val_407 2008-12-31 12 -384 val_384 2008-12-31 12 -256 val_256 2008-12-31 12 -26 val_26 2008-12-31 12 -134 val_134 2008-12-31 12 -67 val_67 2008-12-31 12 -384 val_384 2008-12-31 12 -379 val_379 2008-12-31 12 -18 val_18 2008-12-31 12 -462 val_462 2008-12-31 12 -492 val_492 2008-12-31 12 -100 val_100 2008-12-31 12 -298 val_298 2008-12-31 12 -9 val_9 2008-12-31 12 -341 val_341 2008-12-31 12 -498 val_498 2008-12-31 12 -146 val_146 2008-12-31 12 -458 val_458 2008-12-31 12 -362 val_362 2008-12-31 12 -186 val_186 2008-12-31 12 -285 val_285 2008-12-31 12 -348 val_348 2008-12-31 12 -167 val_167 2008-12-31 12 -18 val_18 2008-12-31 12 -273 val_273 2008-12-31 12 -183 val_183 2008-12-31 12 -281 val_281 2008-12-31 12 -344 val_344 2008-12-31 12 -97 val_97 2008-12-31 12 -469 val_469 2008-12-31 12 -315 val_315 2008-12-31 12 -84 val_84 2008-12-31 12 -28 val_28 2008-12-31 12 -37 val_37 2008-12-31 12 -448 val_448 2008-12-31 12 -152 val_152 2008-12-31 12 -348 val_348 2008-12-31 12 -307 val_307 2008-12-31 12 -194 val_194 2008-12-31 12 -414 val_414 2008-12-31 12 -477 val_477 2008-12-31 12 -222 val_222 2008-12-31 12 -126 val_126 2008-12-31 12 -90 val_90 2008-12-31 12 -169 val_169 2008-12-31 12 -403 val_403 2008-12-31 12 -400 val_400 2008-12-31 12 -200 val_200 2008-12-31 12 -97 val_97 2008-12-31 12 diff --git a/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c b/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c deleted file mode 100644 index 2ac44ee23f45c..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: nzhang_part1 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 b/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 deleted file mode 100644 index f0b07ca89fcb3..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: nzhang_part1 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 b/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 deleted file mode 100644 index f94614124bcf6..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-12-31, 11] -Database: default -Table: nzhang_part2 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2/ds=2008-12-31/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 b/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 deleted file mode 100644 index f00fb5b3bd491..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-12-31, 12] -Database: default -Table: nzhang_part2 -CreateTime: Tue Jan 28 02:18:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2/ds=2008-12-31/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 b/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 deleted file mode 100644 index 7335c8d32fa68..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:17 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part1 -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - p3 v3 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 b/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 deleted file mode 100644 index 674cd9b0b64e5..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:17 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/nzhang_part2 -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - p3 v3 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1390904306 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d b/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 b/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f b/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 b/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df b/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df deleted file mode 100644 index c9a75eb6639d4..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_src))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_src - TableScan - alias: analyze_src - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 b/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 b/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 deleted file mode 100644 index e622a0d3704a1..0000000000000 --- a/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 +++ /dev/null @@ -1,33 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:36 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_src -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 0 - numRows 500 - p3 v3 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904324 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index bf589ab894312..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1390904332 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index e6170a33d7de7..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - transient_lastDdlTime 1390904332 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index 45f6b344b04ca..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:18:45 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - p3 v3 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1390904348 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 9ff40db416cb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904340 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 5c13e8793cc76..0000000000000 --- a/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:18:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904348 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index bafa551db56e9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,38 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 14:57:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 1000 - rawDataSize 10624 - totalSize 11624 - transient_lastDdlTime 1389740280 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 b/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 deleted file mode 100644 index d5341825cff70..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 +++ /dev/null @@ -1,18 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 b/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 7085e9397e59b..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:57:50 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1389740280 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index a3eb8c6466032..0000000000000 --- a/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:57:50 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1389740280 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 b/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 deleted file mode 100644 index d1e0c6243244d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 12)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 27ff9a687f5c6..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904374 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d b/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d deleted file mode 100644 index ce340cced8844..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09') (TOK_PARTVAL hr 11)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 b/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index 649828a63174b..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904382 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 b/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 deleted file mode 100644 index e4c8bf3e0005f..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-09') (TOK_PARTVAL hr 12)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 b/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index 74aeceec37398..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904390 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 b/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 deleted file mode 100644 index 25d9c86f74634..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds) (TOK_PARTVAL hr)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 b/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 5939c257f2f67..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 8dc22de8e0ee6..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index a19f38cc6a364..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index aead24acd518c..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index 5f8bf1774f144..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:19:10 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 4 - numPartitions 4 - numRows 2000 - p3 v3 - rawDataSize 21248 - totalSize 23248 - transient_lastDdlTime 1390904398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 b/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 deleted file mode 100644 index d30acbf86a295..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 +++ /dev/null @@ -1,19 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11)))) - -STAGE DEPENDENCIES: - Stage-0 is a root stage - Stage-1 depends on stages: Stage-0 - -STAGE PLANS: - Stage: Stage-0 - Map Reduce - Alias -> Map Operator Tree: - analyze_srcpart - TableScan - alias: analyze_srcpart - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 deleted file mode 100644 index 07a61fc1a8bd1..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 28 02:19:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904366 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index bd16e76d28d83..0000000000000 --- a/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,39 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 28 02:19:10 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 1 - numPartitions 1 - numRows 500 - p3 v3 - rawDataSize 5312 - totalSize 5812 - transient_lastDdlTime 1390904366 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c rename to sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6f3df708fa339df236ec9375d2ad37fa diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-36eee5cbac5c0c3228e499805b32f6 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 b/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 index 4475cb837eaa4..e01e7187693ed 100644 --- a/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 @@ -1,27 +1,28 @@ # col_name data_type comment -key string None -value string None +key string +value string # Partition Information # col_name data_type comment -part string None +part string # Detailed Partition Information Partition Value: [1] Database: default Table: tmptable -CreateTime: Sun Jan 05 00:32:00 PST 2014 +CreateTime: Tue Oct 21 05:20:33 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2020775722466758355/tmptable/part=1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/tmptable/part=1 Partition Parameters: + COLUMN_STATS_ACCURATE true numFiles 1 numRows 0 rawDataSize 0 totalSize 0 - transient_lastDdlTime 1388910720 + transient_lastDdlTime 1413894033 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -32,4 +33,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d deleted file mode 100644 index 5949a76d84a11..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740295 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 deleted file mode 100644 index 97f5929fd816a..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1389740294 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c deleted file mode 100644 index 6d08ff47abc2c..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-09/hr=12 -Partition Parameters: - transient_lastDdlTime 1389740294 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 deleted file mode 100644 index f441c8b483868..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 +++ /dev/null @@ -1,38 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 14:58:04 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart -Table Type: MANAGED_TABLE -Table Parameters: - numFiles 2 - numPartitions 2 - numRows 0 - rawDataSize 0 - totalSize 11624 - transient_lastDdlTime 1389740295 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab b/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 b/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 b/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 b/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 deleted file mode 100644 index f263eed117b10..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08'))) noscan) - -STAGE DEPENDENCIES: - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 b/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e b/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e deleted file mode 100644 index 6a115f7e6e61a..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f b/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f deleted file mode 100644 index 1d96413b915da..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 12] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:26 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-08/hr=12 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 b/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 deleted file mode 100644 index 4c4380e63b7f9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a b/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a deleted file mode 100644 index ba90d8d3d2477..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 12] -Database: default -Table: analyze_srcpart_partial -CreateTime: Tue Jan 14 14:58:27 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart_partial/ds=2008-04-09/hr=12 -Partition Parameters: - transient_lastDdlTime 1389740307 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 b/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c b/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 b/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 deleted file mode 100644 index b671e68cc4c3b..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) noscan) - -STAGE DEPENDENCIES: - Stage-1 is a root stage - -STAGE PLANS: - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e b/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae b/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 b/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 deleted file mode 100644 index 96499c0fb0f7e..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key int None -value string None - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart -CreateTime: Tue Jan 14 14:58:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse488810659186490763/analyze_srcpart/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 0 - rawDataSize 0 - totalSize 5812 - transient_lastDdlTime 1389740294 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 deleted file mode 100644 index b5fc469438c83..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 14 14:36:56 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - transient_lastDdlTime 1389739016 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef deleted file mode 100644 index cc58ef9026786..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef +++ /dev/null @@ -1,14 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial_scan) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) partialscan) - -STAGE DEPENDENCIES: - Stage-2 is a root stage - Stage-1 depends on stages: Stage-2 - -STAGE PLANS: - Stage: Stage-2 - Partial Scan Statistics - - Stage: Stage-1 - Stats-Aggr Operator - diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c deleted file mode 100644 index 3243fe9fb497d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 14 14:36:56 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 4812 - totalSize 7456 - transient_lastDdlTime 1389739019 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 deleted file mode 100644 index 6accd64d06d69..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 14 14:36:56 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse2847673635801001933/analyze_srcpart_partial_scan/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1389739016 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 b/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 deleted file mode 100644 index 942b7cfe07235..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 28 02:20:21 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - transient_lastDdlTime 1390904421 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef deleted file mode 100644 index 9c704a6ef4126..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef +++ /dev/null @@ -1,15 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_ANALYZE (TOK_TAB (TOK_TABNAME analyze_srcpart_partial_scan) (TOK_PARTSPEC (TOK_PARTVAL ds '2008-04-08') (TOK_PARTVAL hr 11))) partialscan) - -STAGE DEPENDENCIES: - Stage-2 is a root stage - Stage-1 depends on stages: Stage-2 - -STAGE PLANS: - Stage: Stage-2 - Partial Scan Statistics - - Stage: Stage-1 - Stats-Aggr Operator - - diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c deleted file mode 100644 index feee75f095d0d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c +++ /dev/null @@ -1,36 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-08, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 28 02:20:21 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-08/hr=11 -Partition Parameters: - numFiles 1 - numRows 500 - rawDataSize 4812 - totalSize 7456 - transient_lastDdlTime 1390904425 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 deleted file mode 100644 index f39d366764c95..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -value string from deserializer - -# Partition Information -# col_name data_type comment - -ds string None -hr string None - -# Detailed Partition Information -Partition Value: [2008-04-09, 11] -Database: default -Table: analyze_srcpart_partial_scan -CreateTime: Tue Jan 28 02:20:21 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse6423793619827660847/analyze_srcpart_partial_scan/ds=2008-04-09/hr=11 -Partition Parameters: - transient_lastDdlTime 1390904421 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca b/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/stats_publisher_error_1-1-36eee5cbac5c0c3228e499805b32f6 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 b/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 deleted file mode 100644 index 9bed96d855476..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 +++ /dev/null @@ -1 +0,0 @@ -str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text diff --git a/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c b/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c deleted file mode 100644 index 1da11fbfa31d5..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c +++ /dev/null @@ -1,2 +0,0 @@ -str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text -Split text into key-value pairs using two delimiters. The first delimiter seperates pairs, and the second delimiter sperates key and value. If only one parameter is given, default delimiters are used: ',' as delimiter1 and '=' as delimiter2. diff --git a/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a b/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e b/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e deleted file mode 100644 index cd4d4a384163b..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e +++ /dev/null @@ -1,3 +0,0 @@ -444 -444 -444 diff --git a/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec b/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 b/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 deleted file mode 100644 index e8183f05f5db6..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 +++ /dev/null @@ -1,3 +0,0 @@ -1 -1 -1 diff --git a/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 b/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda b/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda deleted file mode 100644 index 62813f9d10491..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda +++ /dev/null @@ -1,3 +0,0 @@ -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} diff --git a/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf b/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad b/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad deleted file mode 100644 index 62813f9d10491..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad +++ /dev/null @@ -1,3 +0,0 @@ -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} -{"b":"2","a":"1","c":"3"} diff --git a/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 b/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd b/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd deleted file mode 100644 index e8183f05f5db6..0000000000000 --- a/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd +++ /dev/null @@ -1,3 +0,0 @@ -1 -1 -1 diff --git a/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 b/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 index 8938b37682023..cfb0572d8663d 100644 --- a/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 +++ b/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 @@ -497,4 +497,4 @@ test test test test -test \ No newline at end of file +test diff --git a/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca b/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca index 5b3f76a5f7797..0bc48337e2dc0 100644 --- a/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca +++ b/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca @@ -255,4 +255,4 @@ 495 1 496 1 497 1 -498 3 \ No newline at end of file +498 3 diff --git a/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 +++ b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 b/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a b/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 b/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 b/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b b/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 b/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 b/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 b/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e +++ b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 b/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 index 92cca3b6f1e24..95e2ae6a0fd50 100644 --- a/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 +++ b/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 @@ -1,2 +1,2 @@ -a int None -b string None \ No newline at end of file +a int +b string diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 b/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 index 55d9485999072..b70e127e82d05 100644 --- a/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 +++ b/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 @@ -497,4 +497,4 @@ 497 val_497 498 val_498 498 val_498 -498 val_498 \ No newline at end of file +498 val_498 diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c b/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/timestamp_1-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 b/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/timestamp_1-1-d362501d0176855077e65f8faf067fa8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 rename to sql/hive/src/test/resources/golden/timestamp_1-1-d362501d0176855077e65f8faf067fa8 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-10-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-18-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-10-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-11-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-10-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-11-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 b/sql/hive/src/test/resources/golden/timestamp_1-12-6328d3b3dfd295dd5ec453ffb47ff4d0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-11-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-12-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d b/sql/hive/src/test/resources/golden/timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d rename to sql/hive/src/test/resources/golden/timestamp_1-13-d242038c04dd4ee6075c7eebc0f75f17 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-4-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/timestamp_1-14-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-14-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-15-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-15-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-16-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-17-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-16-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-17-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-17-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-18-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-19-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-27-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-19-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d b/sql/hive/src/test/resources/golden/timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d rename to sql/hive/src/test/resources/golden/timestamp_1-2-1d7cf3a2512fa1876b422b79bbe05426 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 b/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-20-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-19-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-20-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-20-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-21-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e b/sql/hive/src/test/resources/golden/timestamp_1-21-d8fff1a6c464e50eb955babfafb0b98e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 rename to sql/hive/src/test/resources/golden/timestamp_1-22-cdb04b49b836e0244f6f0857aea7da8a diff --git a/sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-40-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/timestamp_1-23-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-23-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-24-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-24-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-25-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-26-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-25-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-26-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-26-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-27-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-28-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-36-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-28-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-29-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-28-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-29-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/timestamp_1-3-74f477a8b726f5193dd42ef378a793c4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c rename to sql/hive/src/test/resources/golden/timestamp_1-3-74f477a8b726f5193dd42ef378a793c4 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 b/sql/hive/src/test/resources/golden/timestamp_1-3-819633b45e3e1779bca6bcb7b77fe5a1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 b/sql/hive/src/test/resources/golden/timestamp_1-30-273256141c33eb88194cad22eb940d21 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-29-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-30-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 rename to sql/hive/src/test/resources/golden/timestamp_1-31-9587976bd7e6caa5b667975c14e8dd53 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 b/sql/hive/src/test/resources/golden/timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-49-90269c1e50c7ae8e75ca9cc297982135 rename to sql/hive/src/test/resources/golden/timestamp_1-32-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-32-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-33-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-33-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-34-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-35-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-34-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-35-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-35-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-36-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-37-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-45-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-37-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-38-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-37-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-38-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d b/sql/hive/src/test/resources/golden/timestamp_1-39-b2fe5cc7c8ee62d3bb0c120c9a6c305d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-38-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-39-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/timestamp_1-4-d833b177fac3162215468dde991f71d1 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea rename to sql/hive/src/test/resources/golden/timestamp_1-4-d833b177fac3162215468dde991f71d1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 rename to sql/hive/src/test/resources/golden/timestamp_1-40-4ebcf4bcc059feba0fd9f76f26193f3b diff --git a/sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-13-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_1-41-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-41-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-42-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-42-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-43-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-44-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-43-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-44-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-44-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-45-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-46-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-54-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-46-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-47-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-46-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-47-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 b/sql/hive/src/test/resources/golden/timestamp_1-48-7029255241de8e8b9710801319990044 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-47-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-48-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 rename to sql/hive/src/test/resources/golden/timestamp_1-49-7a59f9f939efc4b96f8159d00b39ed3 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-22-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_1-5-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-31-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_1-50-90269c1e50c7ae8e75ca9cc297982135 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-5-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-51-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-51-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-52-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-53-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-52-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-53-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-53-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-54-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac b/sql/hive/src/test/resources/golden/timestamp_1-55-67f274bf16de625cf4e85af0c6185cac similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-9-67f274bf16de625cf4e85af0c6185cac rename to sql/hive/src/test/resources/golden/timestamp_1-55-67f274bf16de625cf4e85af0c6185cac diff --git a/sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e b/sql/hive/src/test/resources/golden/timestamp_1-56-343c75daac6695917608c17db8bf473e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-55-343c75daac6695917608c17db8bf473e rename to sql/hive/src/test/resources/golden/timestamp_1-56-343c75daac6695917608c17db8bf473e diff --git a/sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a b/sql/hive/src/test/resources/golden/timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-56-cf19f7359a6d3456c4526b2c69f92d6a rename to sql/hive/src/test/resources/golden/timestamp_1-57-cf19f7359a6d3456c4526b2c69f92d6a diff --git a/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-57-d362501d0176855077e65f8faf067fa8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/timestamp_1-58-d362501d0176855077e65f8faf067fa8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 rename to sql/hive/src/test/resources/golden/timestamp_1-58-d362501d0176855077e65f8faf067fa8 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 b/sql/hive/src/test/resources/golden/timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-50-e6bfca320c4ee3aff39cf2f179d57da6 rename to sql/hive/src/test/resources/golden/timestamp_1-6-e6bfca320c4ee3aff39cf2f179d57da6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 b/sql/hive/src/test/resources/golden/timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-6-d0291a9bd42054b2732cb4f54cf39ae7 rename to sql/hive/src/test/resources/golden/timestamp_1-7-d0291a9bd42054b2732cb4f54cf39ae7 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 b/sql/hive/src/test/resources/golden/timestamp_1-8-e7b398d2a8107a42419c83771bda41e6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-7-e7b398d2a8107a42419c83771bda41e6 rename to sql/hive/src/test/resources/golden/timestamp_1-8-e7b398d2a8107a42419c83771bda41e6 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf b/sql/hive/src/test/resources/golden/timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_1-8-a3eeec08bccae78d0d94ad2cb923e1cf rename to sql/hive/src/test/resources/golden/timestamp_1-9-a3eeec08bccae78d0d94ad2cb923e1cf diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/timestamp_2-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 b/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 rename to sql/hive/src/test/resources/golden/timestamp_2-1-ea7192a4a5a985bcc8aab9aa79d9f028 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-10-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-18-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-10-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-11-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-10-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-11-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-12-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-11-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-12-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd b/sql/hive/src/test/resources/golden/timestamp_2-12-7350308cbf49d6ebd6599d3802750acd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/timestamp_2-13-5f450162886ccc79af149541527f5643 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 rename to sql/hive/src/test/resources/golden/timestamp_2-13-5f450162886ccc79af149541527f5643 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-14-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-4-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_2-14-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-14-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-15-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-15-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-16-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-16-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-17-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-17-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-18-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-19-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-27-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-19-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 rename to sql/hive/src/test/resources/golden/timestamp_2-2-61dbdf6d26c2a3f1143f6fdae999b1b4 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 b/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-20-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-19-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-20-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-21-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-20-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-21-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b b/sql/hive/src/test/resources/golden/timestamp_2-21-5eb58e5d3c5b9f766f0b497bf59c47b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/timestamp_2-22-469fe94fb60f4b00809190c303434641 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 rename to sql/hive/src/test/resources/golden/timestamp_2-22-469fe94fb60f4b00809190c303434641 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-23-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-40-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_2-23-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-23-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-24-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-24-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-25-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-25-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-26-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-26-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-27-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-28-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-36-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-28-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-29-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-28-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-29-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 rename to sql/hive/src/test/resources/golden/timestamp_2-3-a5092ff0f5a3d3b8f4171994932d4d19 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 b/sql/hive/src/test/resources/golden/timestamp_2-3-a95a52c3a66e1f211ea04a0a10bd3b74 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-30-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-29-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-30-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 b/sql/hive/src/test/resources/golden/timestamp_2-30-ffe6b6ddaaba84152074f7781fba2243 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/timestamp_2-31-8f506498acf0c99c30960a00981ef460 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f rename to sql/hive/src/test/resources/golden/timestamp_2-31-8f506498acf0c99c30960a00981ef460 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e b/sql/hive/src/test/resources/golden/timestamp_2-32-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-49-25f6ec69328af6cba76899194e0dd84e rename to sql/hive/src/test/resources/golden/timestamp_2-32-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-32-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-33-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-33-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-34-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-34-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-35-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-35-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-36-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-37-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-45-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-37-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-38-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-37-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-38-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-39-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-38-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-39-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 b/sql/hive/src/test/resources/golden/timestamp_2-39-8236608f28681eac5503195096a34181 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 rename to sql/hive/src/test/resources/golden/timestamp_2-4-81d6d29dcb3fd12a519426dff64411d2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 rename to sql/hive/src/test/resources/golden/timestamp_2-40-972a007e54d1c09e9ac9549c19a32dbb diff --git a/sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 b/sql/hive/src/test/resources/golden/timestamp_2-41-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-4-935d0d2492beab99bbbba26ba62a1db4 rename to sql/hive/src/test/resources/golden/timestamp_2-41-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-41-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-42-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-42-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-43-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-43-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-44-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-44-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-45-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-46-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-54-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-46-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-47-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-46-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-47-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-48-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-47-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-48-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb b/sql/hive/src/test/resources/golden/timestamp_2-48-654e5533ec6dc911996abc7e47af8ccb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/timestamp_2-49-650d2727b007638e0ed39b37c9498d66 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f rename to sql/hive/src/test/resources/golden/timestamp_2-49-650d2727b007638e0ed39b37c9498d66 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 b/sql/hive/src/test/resources/golden/timestamp_2-5-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 rename to sql/hive/src/test/resources/golden/timestamp_2-5-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 b/sql/hive/src/test/resources/golden/timestamp_2-50-25f6ec69328af6cba76899194e0dd84e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 rename to sql/hive/src/test/resources/golden/timestamp_2-50-25f6ec69328af6cba76899194e0dd84e diff --git a/sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-5-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-51-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-51-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-52-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-52-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-53-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-53-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-54-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a b/sql/hive/src/test/resources/golden/timestamp_2-55-252aebfe7882335d31bfc53a8705b7a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-9-252aebfe7882335d31bfc53a8705b7a rename to sql/hive/src/test/resources/golden/timestamp_2-55-252aebfe7882335d31bfc53a8705b7a diff --git a/sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 b/sql/hive/src/test/resources/golden/timestamp_2-56-5181279a0bf8939fe46ddacae015dad8 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-55-5181279a0bf8939fe46ddacae015dad8 rename to sql/hive/src/test/resources/golden/timestamp_2-56-5181279a0bf8939fe46ddacae015dad8 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 b/sql/hive/src/test/resources/golden/timestamp_2-57-240fce5f58794fa051824e8732c00c03 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-56-240fce5f58794fa051824e8732c00c03 rename to sql/hive/src/test/resources/golden/timestamp_2-57-240fce5f58794fa051824e8732c00c03 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-57-ea7192a4a5a985bcc8aab9aa79d9f028 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef rename to sql/hive/src/test/resources/golden/timestamp_2-58-ea7192a4a5a985bcc8aab9aa79d9f028 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 b/sql/hive/src/test/resources/golden/timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-50-93c769be4cff93bea6e62bfe4e2a8742 rename to sql/hive/src/test/resources/golden/timestamp_2-6-93c769be4cff93bea6e62bfe4e2a8742 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 b/sql/hive/src/test/resources/golden/timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-6-5bdbf67419cc060b82d091d80ce59bf9 rename to sql/hive/src/test/resources/golden/timestamp_2-7-5bdbf67419cc060b82d091d80ce59bf9 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 b/sql/hive/src/test/resources/golden/timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-7-de3c42ab06c17ae895fd7deaf7bd9571 rename to sql/hive/src/test/resources/golden/timestamp_2-8-de3c42ab06c17ae895fd7deaf7bd9571 diff --git a/sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef b/sql/hive/src/test/resources/golden/timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_2-8-da3937d21b7c2cfe1e624e812ae1d3ef rename to sql/hive/src/test/resources/golden/timestamp_2-9-da3937d21b7c2cfe1e624e812ae1d3ef diff --git a/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/timestamp_3-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 rename to sql/hive/src/test/resources/golden/timestamp_3-1-165256158e3db1ce19c3c9db3c8011d2 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b b/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b b/sql/hive/src/test/resources/golden/timestamp_3-10-ffc79abb874323e165963aa39f460a9b similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-9-ffc79abb874323e165963aa39f460a9b rename to sql/hive/src/test/resources/golden/timestamp_3-10-ffc79abb874323e165963aa39f460a9b diff --git a/sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 b/sql/hive/src/test/resources/golden/timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-10-7b1ec929239ee305ea9da46ebb990c67 rename to sql/hive/src/test/resources/golden/timestamp_3-11-7b1ec929239ee305ea9da46ebb990c67 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-12-165256158e3db1ce19c3c9db3c8011d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a b/sql/hive/src/test/resources/golden/timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-11-a63f40f6c4a022c16f8cf810e3b7ed2a rename to sql/hive/src/test/resources/golden/timestamp_3-12-a63f40f6c4a022c16f8cf810e3b7ed2a diff --git a/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 b/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 new file mode 100644 index 0000000000000..4cfaa1b324da3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_3-13-7d225bcfa35f20da7dd63e7f8a413a77 @@ -0,0 +1 @@ +2011-04-29 20:46:56.4485 1.3041352164485E9 1 1.3041352164485E9 1.3041352164485E9 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 rename to sql/hive/src/test/resources/golden/timestamp_3-14-165256158e3db1ce19c3c9db3c8011d2 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 b/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/timestamp_3-2-81edf5107270547641586aa02b4e7d9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d rename to sql/hive/src/test/resources/golden/timestamp_3-2-81edf5107270547641586aa02b4e7d9b diff --git a/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a b/sql/hive/src/test/resources/golden/timestamp_3-3-6143888a940bfcac1133330764f5a31a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/timestamp_3-3-7a012a0d98729da25b5ac374855dcee4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b rename to sql/hive/src/test/resources/golden/timestamp_3-3-7a012a0d98729da25b5ac374855dcee4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/timestamp_3-4-86514381187b246a5685577c1968c559 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad rename to sql/hive/src/test/resources/golden/timestamp_3-4-86514381187b246a5685577c1968c559 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 b/sql/hive/src/test/resources/golden/timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 rename to sql/hive/src/test/resources/golden/timestamp_3-5-935d0d2492beab99bbbba26ba62a1db4 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 b/sql/hive/src/test/resources/golden/timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-5-8fe348d5d9b9903a26eda32d308b8e41 rename to sql/hive/src/test/resources/golden/timestamp_3-6-8fe348d5d9b9903a26eda32d308b8e41 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 b/sql/hive/src/test/resources/golden/timestamp_3-7-6be5fe01c502cd24db32a3781c97a703 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-6-6be5fe01c502cd24db32a3781c97a703 rename to sql/hive/src/test/resources/golden/timestamp_3-7-6be5fe01c502cd24db32a3781c97a703 diff --git a/sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add b/sql/hive/src/test/resources/golden/timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-7-6066ba0451cd0fcfac4bea6376e72add rename to sql/hive/src/test/resources/golden/timestamp_3-8-6066ba0451cd0fcfac4bea6376e72add diff --git a/sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e b/sql/hive/src/test/resources/golden/timestamp_3-9-22e03daa775eab145d39ec0730953f7e similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_3-8-22e03daa775eab145d39ec0730953f7e rename to sql/hive/src/test/resources/golden/timestamp_3-9-22e03daa775eab145d39ec0730953f7e diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/timestamp_comparison-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 b/sql/hive/src/test/resources/golden/timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 rename to sql/hive/src/test/resources/golden/timestamp_comparison-1-4b68f7ad0f8cf337e42bf16a45e15818 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d b/sql/hive/src/test/resources/golden/timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d rename to sql/hive/src/test/resources/golden/timestamp_comparison-2-60557e7bd2822c89fa8b076a9d0520fc diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc b/sql/hive/src/test/resources/golden/timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc rename to sql/hive/src/test/resources/golden/timestamp_comparison-3-f96a9d88327951bd93f672dc2463ecd4 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 b/sql/hive/src/test/resources/golden/timestamp_comparison-4-13e17ed811165196416f777cbc162592 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 rename to sql/hive/src/test/resources/golden/timestamp_comparison-4-13e17ed811165196416f777cbc162592 diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 b/sql/hive/src/test/resources/golden/timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 rename to sql/hive/src/test/resources/golden/timestamp_comparison-5-4fa8a36edbefde4427c2ab2cf30e6399 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 b/sql/hive/src/test/resources/golden/timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 rename to sql/hive/src/test/resources/golden/timestamp_comparison-6-7e4fb6e8ba01df422e4c67e06a0c8453 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 b/sql/hive/src/test/resources/golden/timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 rename to sql/hive/src/test/resources/golden/timestamp_comparison-7-8c8e73673a950f6b3d960b08fcea076f diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 b/sql/hive/src/test/resources/golden/timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 rename to sql/hive/src/test/resources/golden/timestamp_comparison-8-510c0a2a57dc5df8588bd13c4152f8bc diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 b/sql/hive/src/test/resources/golden/timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 rename to sql/hive/src/test/resources/golden/timestamp_comparison-9-659d5b1ae8200f13f265270e52a3dd65 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 rename to sql/hive/src/test/resources/golden/timestamp_lazy-2-bb5a4a13274290029bd07d95c2f92563 diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 b/sql/hive/src/test/resources/golden/timestamp_lazy-2-cdb72e0c24fd9277a41fe0c7b1392e34 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/timestamp_null-2-51762cf5079877abf7d81127738f4e5 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c rename to sql/hive/src/test/resources/golden/timestamp_null-2-51762cf5079877abf7d81127738f4e5 diff --git a/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 b/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/timestamp_udf-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af b/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d b/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a rename to sql/hive/src/test/resources/golden/timestamp_udf-1-79914c5347620c6e62a8e0b9a95984af diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a b/sql/hive/src/test/resources/golden/timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-18-cb033ecad964a2623bc633ac1d3f752a rename to sql/hive/src/test/resources/golden/timestamp_udf-10-287614364eaa3fb82aad08c6b62cc938 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09 b/sql/hive/src/test/resources/golden/timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-10-dbc23736a61d9482d13cacada02a7a09 rename to sql/hive/src/test/resources/golden/timestamp_udf-11-dbc23736a61d9482d13cacada02a7a09 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35 b/sql/hive/src/test/resources/golden/timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-11-442cf850a0cc1f1dcfdeaeffbffb2c35 rename to sql/hive/src/test/resources/golden/timestamp_udf-12-442cf850a0cc1f1dcfdeaeffbffb2c35 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6 b/sql/hive/src/test/resources/golden/timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-12-51959036fd4ac4f1e24f4e06eb9b0b6 rename to sql/hive/src/test/resources/golden/timestamp_udf-13-51959036fd4ac4f1e24f4e06eb9b0b6 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a b/sql/hive/src/test/resources/golden/timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-13-6ab3f356deaf807e8accc37e1f4849a rename to sql/hive/src/test/resources/golden/timestamp_udf-14-6ab3f356deaf807e8accc37e1f4849a diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-14-c745a1016461403526d44928a269c1de b/sql/hive/src/test/resources/golden/timestamp_udf-15-c745a1016461403526d44928a269c1de similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-14-c745a1016461403526d44928a269c1de rename to sql/hive/src/test/resources/golden/timestamp_udf-15-c745a1016461403526d44928a269c1de diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe b/sql/hive/src/test/resources/golden/timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-15-7ab76c4458c7f78038c8b1df0fdeafbe rename to sql/hive/src/test/resources/golden/timestamp_udf-16-7ab76c4458c7f78038c8b1df0fdeafbe diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-16-b36e87e17ca24d82072220bff559c718 b/sql/hive/src/test/resources/golden/timestamp_udf-17-b36e87e17ca24d82072220bff559c718 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-16-b36e87e17ca24d82072220bff559c718 rename to sql/hive/src/test/resources/golden/timestamp_udf-17-b36e87e17ca24d82072220bff559c718 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-17-dad44d2d4a421286e9da080271bd2639 b/sql/hive/src/test/resources/golden/timestamp_udf-18-dad44d2d4a421286e9da080271bd2639 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-17-dad44d2d4a421286e9da080271bd2639 rename to sql/hive/src/test/resources/golden/timestamp_udf-18-dad44d2d4a421286e9da080271bd2639 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af b/sql/hive/src/test/resources/golden/timestamp_udf-19-79914c5347620c6e62a8e0b9a95984af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938 b/sql/hive/src/test/resources/golden/timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-9-287614364eaa3fb82aad08c6b62cc938 rename to sql/hive/src/test/resources/golden/timestamp_udf-19-cb033ecad964a2623bc633ac1d3f752a diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 rename to sql/hive/src/test/resources/golden/timestamp_udf-2-59fc1842a23369235d42ed040d45fb3d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 b/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d b/sql/hive/src/test/resources/golden/timestamp_udf-20-59fc1842a23369235d42ed040d45fb3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 rename to sql/hive/src/test/resources/golden/timestamp_udf-20-79914c5347620c6e62a8e0b9a95984af diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa rename to sql/hive/src/test/resources/golden/timestamp_udf-21-59fc1842a23369235d42ed040d45fb3d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/timestamp_udf-3-9039f474f9a96e9f15ace528faeed923 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 rename to sql/hive/src/test/resources/golden/timestamp_udf-3-9039f474f9a96e9f15ace528faeed923 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f b/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc b/sql/hive/src/test/resources/golden/timestamp_udf-4-80ce02ec84ee8abcb046367ca37279cc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/timestamp_udf-4-b0fd4ca3b22eb732a32772399331352f diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/timestamp_udf-5-66868a2b075de978784011e9955483d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 rename to sql/hive/src/test/resources/golden/timestamp_udf-5-66868a2b075de978784011e9955483d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-5-1124399033bcadf3874fb48f593392d b/sql/hive/src/test/resources/golden/timestamp_udf-6-1124399033bcadf3874fb48f593392d similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-5-1124399033bcadf3874fb48f593392d rename to sql/hive/src/test/resources/golden/timestamp_udf-6-1124399033bcadf3874fb48f593392d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4 b/sql/hive/src/test/resources/golden/timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-6-5810193ce35d38c23f4fc4b4979d60a4 rename to sql/hive/src/test/resources/golden/timestamp_udf-7-5810193ce35d38c23f4fc4b4979d60a4 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9 b/sql/hive/src/test/resources/golden/timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-7-250e640a6a818f989f3f3280b00f64f9 rename to sql/hive/src/test/resources/golden/timestamp_udf-8-250e640a6a818f989f3f3280b00f64f9 diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-8-975df43df015d86422965af456f87a94 b/sql/hive/src/test/resources/golden/timestamp_udf-9-975df43df015d86422965af456f87a94 similarity index 100% rename from sql/hive/src/test/resources/golden/timestamp_udf-8-975df43df015d86422965af456f87a94 rename to sql/hive/src/test/resources/golden/timestamp_udf-9-975df43df015d86422965af456f87a94 diff --git a/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe b/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe index d23e05acf7ba5..e34118512c1d7 100644 --- a/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe +++ b/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe @@ -497,4 +497,4 @@ 403 400 200 -97 \ No newline at end of file +97 diff --git a/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 b/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 b/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 b/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 b/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 b/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 b/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f b/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f deleted file mode 100644 index c6f628b1a3eef..0000000000000 --- a/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f +++ /dev/null @@ -1 +0,0 @@ -[0,1,2] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 b/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 index 839efe2d57956..9370a501a5179 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 +++ b/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 @@ -165,4 +165,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 +++ b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 b/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 index 839efe2d57956..9370a501a5179 100644 --- a/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 +++ b/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 @@ -165,4 +165,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d b/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d index 66fafbdf72b63..3df4716f0b05f 100644 --- a/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d +++ b/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 200 val_200 200 val_200 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 b/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 index 66fafbdf72b63..3df4716f0b05f 100644 --- a/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 +++ b/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 @@ -1025,4 +1025,4 @@ 200 val_200 200 val_200 200 val_200 200 val_200 97 val_97 97 val_97 -97 val_97 97 val_97 \ No newline at end of file +97 val_97 97 val_97 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/type_cast_1-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d b/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 rename to sql/hive/src/test/resources/golden/type_cast_1-1-60ea21e6e7d054a65f959fc89acf1b3d diff --git a/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/type_cast_1-2-53a667981ad567b2ab977f67d65c5825 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/type_widening-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/type_widening-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 b/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/type_widening-1-630ac2c7e7dea4837384ccd572209229 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 rename to sql/hive/src/test/resources/golden/type_widening-1-630ac2c7e7dea4837384ccd572209229 diff --git a/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 b/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 b/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/type_widening-2-cfbdf2b6fca84c6e23d4e691d2221bd6 diff --git a/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 b/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 deleted file mode 100644 index cf940f4c5faa8..0000000000000 --- a/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 +++ /dev/null @@ -1,1000 +0,0 @@ -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -0 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 -9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c rename to sql/hive/src/test/resources/golden/type_widening-3-a0ba6952d9bf830d1d1ea7aebd3784a2 diff --git a/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 b/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 new file mode 100644 index 0000000000000..00841d23b3f94 --- /dev/null +++ b/sql/hive/src/test/resources/golden/type_widening-4-65da8c67f6903286168acb39ac67fc04 @@ -0,0 +1,1000 @@ +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +0 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 +9223372036854775807 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e b/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e index fed3d6802023a..bb3393324db7a 100644 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e @@ -1 +1 @@ -collect_set(x) - Returns a set of objects with duplicate elements eliminated \ No newline at end of file +collect_set(x) - Returns a set of objects with duplicate elements eliminated diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 b/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 index fed3d6802023a..bb3393324db7a 100644 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 @@ -1 +1 @@ -collect_set(x) - Returns a set of objects with duplicate elements eliminated \ No newline at end of file +collect_set(x) - Returns a set of objects with duplicate elements eliminated diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db index cedc3068ee4a1..c87ba74c9000a 100644 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db @@ -17,4 +17,4 @@ 27 ["val_27"] 28 ["val_28"] 30 ["val_30"] -33 ["val_33"] \ No newline at end of file +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 b/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 new file mode 100644 index 0000000000000..337e96635cc70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-11-5c3768074977ef68a1b9bb72eb9ef02 @@ -0,0 +1,20 @@ +0 ["val_0","val_0","val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5","val_5","val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12","val_12"] +15 ["val_15","val_15"] +17 ["val_17"] +18 ["val_18","val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24","val_24"] +26 ["val_26","val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udaf_collect_set-12-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udaf_collect_set-13-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db deleted file mode 100644 index cedc3068ee4a1..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db +++ /dev/null @@ -1,20 +0,0 @@ -0 ["val_0"] -2 ["val_2"] -4 ["val_4"] -5 ["val_5"] -8 ["val_8"] -9 ["val_9"] -10 ["val_10"] -11 ["val_11"] -12 ["val_12"] -15 ["val_15"] -17 ["val_17"] -18 ["val_18"] -19 ["val_19"] -20 ["val_20"] -24 ["val_24"] -26 ["val_26"] -27 ["val_27"] -28 ["val_28"] -30 ["val_30"] -33 ["val_33"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..c87ba74c9000a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-14-c8bc33095e1a195bb7b5e579d8d78db @@ -0,0 +1,20 @@ +0 ["val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12"] +15 ["val_15"] +17 ["val_17"] +18 ["val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24"] +26 ["val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udaf_collect_set-15-863233ccd616401efb4bf83c4b9e3a52 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udaf_collect_set-16-a00d1791b7fa7ac5a0505d95c3d12257 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..c87ba74c9000a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-17-c8bc33095e1a195bb7b5e579d8d78db @@ -0,0 +1,20 @@ +0 ["val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12"] +15 ["val_15"] +17 ["val_17"] +18 ["val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24"] +26 ["val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb b/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb new file mode 100644 index 0000000000000..28abc06ee9140 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-2-4747c35670a011344982573ba31a9bb @@ -0,0 +1 @@ +collect_list(x) - Returns a list of objects with duplicates diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 b/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 new file mode 100644 index 0000000000000..28abc06ee9140 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-3-9aa348a25ca17ab5b636d3ea2d6df986 @@ -0,0 +1 @@ +collect_list(x) - Returns a list of objects with duplicates diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udaf_collect_set-4-1d351f7e821fcaf66c6f7503e42fb291 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db deleted file mode 100644 index cedc3068ee4a1..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db +++ /dev/null @@ -1,20 +0,0 @@ -0 ["val_0"] -2 ["val_2"] -4 ["val_4"] -5 ["val_5"] -8 ["val_8"] -9 ["val_9"] -10 ["val_10"] -11 ["val_11"] -12 ["val_12"] -15 ["val_15"] -17 ["val_17"] -18 ["val_18"] -19 ["val_19"] -20 ["val_20"] -24 ["val_24"] -26 ["val_26"] -27 ["val_27"] -28 ["val_28"] -30 ["val_30"] -33 ["val_33"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udaf_collect_set-5-a7dc16cb82c595b18d4258a38a304b1e diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..c87ba74c9000a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-6-c8bc33095e1a195bb7b5e579d8d78db @@ -0,0 +1,20 @@ +0 ["val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12"] +15 ["val_15"] +17 ["val_17"] +18 ["val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24"] +26 ["val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 b/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 new file mode 100644 index 0000000000000..337e96635cc70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-7-1fd4f3dcdac818ccc95c5033c6d01b56 @@ -0,0 +1,20 @@ +0 ["val_0","val_0","val_0"] +2 ["val_2"] +4 ["val_4"] +5 ["val_5","val_5","val_5"] +8 ["val_8"] +9 ["val_9"] +10 ["val_10"] +11 ["val_11"] +12 ["val_12","val_12"] +15 ["val_15","val_15"] +17 ["val_17"] +18 ["val_18","val_18"] +19 ["val_19"] +20 ["val_20"] +24 ["val_24","val_24"] +26 ["val_26","val_26"] +27 ["val_27"] +28 ["val_28"] +30 ["val_30"] +33 ["val_33"] diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db deleted file mode 100644 index cedc3068ee4a1..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db +++ /dev/null @@ -1,20 +0,0 @@ -0 ["val_0"] -2 ["val_2"] -4 ["val_4"] -5 ["val_5"] -8 ["val_8"] -9 ["val_9"] -10 ["val_10"] -11 ["val_11"] -12 ["val_12"] -15 ["val_15"] -17 ["val_17"] -18 ["val_18"] -19 ["val_19"] -20 ["val_20"] -24 ["val_24"] -26 ["val_26"] -27 ["val_27"] -28 ["val_28"] -30 ["val_30"] -33 ["val_33"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udaf_collect_set-8-863233ccd616401efb4bf83c4b9e3a52 diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udaf_collect_set-9-a7dc16cb82c595b18d4258a38a304b1e diff --git a/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/udaf_corr-2-e886f45c8f085596ffd420f89cdc2909 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 rename to sql/hive/src/test/resources/golden/udaf_corr-2-e886f45c8f085596ffd420f89cdc2909 diff --git a/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e b/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e index 6e5d422b3132e..06b8ee9e72496 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e +++ b/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e @@ -1,2 +1,2 @@ corr(x,y) - Returns the Pearson coefficient of correlation -between a set of number pairs \ No newline at end of file +between a set of number pairs diff --git a/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a b/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a index fcb49ae69f74f..f516ef3a30fdf 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a +++ b/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a @@ -6,4 +6,4 @@ a singleton set, NULL will be returned. Otherwise, it computes the following: COVAR_POP(x,y)/(STDDEV_POP(x)*STDDEV_POP(y)) where neither x nor y is null, COVAR_POP is the population covariance, -and STDDEV_POP is the population standard deviation. \ No newline at end of file +and STDDEV_POP is the population standard deviation. diff --git a/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb +++ b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 +++ b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 +++ b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 b/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 index 3f730875aef8c..946c9d58047e5 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 +++ b/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 @@ -3,4 +3,4 @@ 3 NULL 4 NULL 5 NULL -6 NULL \ No newline at end of file +6 NULL diff --git a/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 index 5d97236e8b03f..011d78d68766d 100644 --- a/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 +++ b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 @@ -1 +1 @@ -0.6633880657639323 \ No newline at end of file +0.6633880657639323 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 rename to sql/hive/src/test/resources/golden/udaf_covar_pop-2-e886f45c8f085596ffd420f89cdc2909 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 b/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 index 0fed030c22af3..eadc2e1aa99b7 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 @@ -1 +1 @@ -covar_pop(x,y) - Returns the population covariance of a set of number pairs \ No newline at end of file +covar_pop(x,y) - Returns the population covariance of a set of number pairs diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad b/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad index 4037062d2da93..7323e72fccc7c 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad @@ -3,4 +3,4 @@ The function takes as arguments any pair of numeric types and returns a double. Any pair with a NULL is ignored. If the function is applied to an empty set, NULL will be returned. Otherwise, it computes the following: (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/COUNT(x,y) -where neither x nor y is null. \ No newline at end of file +where neither x nor y is null. diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae index 171538eb0b00f..ba66466c2a0d0 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae @@ -1 +1 @@ -0.0 \ No newline at end of file +0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f b/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f index 848e15bc61476..104018ecd43d6 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f @@ -3,4 +3,4 @@ 3 0.0 4 0.0 5 0.0 -6 0.0 \ No newline at end of file +6 0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 index 1a49bf590b346..16f4e6bd601b6 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 @@ -1 +1 @@ -3.624999999999999 \ No newline at end of file +3.624999999999999 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 rename to sql/hive/src/test/resources/golden/udaf_covar_samp-2-e886f45c8f085596ffd420f89cdc2909 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf b/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf index 97a97c8b5965a..b301d988192fd 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf @@ -1 +1 @@ -covar_samp(x,y) - Returns the sample covariance of a set of number pairs \ No newline at end of file +covar_samp(x,y) - Returns the sample covariance of a set of number pairs diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c b/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c index e666adba8df68..de059e0e79a27 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c @@ -3,4 +3,4 @@ The function takes as arguments any pair of numeric types and returns a double. Any pair with a NULL is ignored. If the function is applied to an empty set, NULL will be returned. Otherwise, it computes the following: (SUM(x*y)-SUM(x)*SUM(y)/COUNT(x,y))/(COUNT(x,y)-1) -where neither x nor y is null. \ No newline at end of file +where neither x nor y is null. diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 index fe3a0735d98b8..7951defec192a 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 @@ -1 +1 @@ -NULL \ No newline at end of file +NULL diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 index 171538eb0b00f..ba66466c2a0d0 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 @@ -1 +1 @@ -0.0 \ No newline at end of file +0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a b/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a index 848e15bc61476..104018ecd43d6 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a @@ -3,4 +3,4 @@ 3 0.0 4 0.0 5 0.0 -6 0.0 \ No newline at end of file +6 0.0 diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 index 30d98234a23e9..b8adc8f23da34 100644 --- a/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 @@ -1 +1 @@ -4.833333333333332 \ No newline at end of file +4.833333333333332 diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 index 3e8bb17f24d6f..df3189a887974 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 @@ -1 +1 @@ -[{"x":135.0284552845532,"y":246.0},{"x":381.39370078740143,"y":254.0}] \ No newline at end of file +[{"x":135.0284552845532,"y":246.0},{"x":381.39370078740143,"y":254.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 index 048770d4c8402..b1f27ead33687 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 @@ -1 +1 @@ -[{"x":96.7349397590361,"y":166.0},{"x":257.14970059880255,"y":167.0},{"x":425.6826347305388,"y":167.0}] \ No newline at end of file +[{"x":96.7349397590361,"y":166.0},{"x":257.14970059880255,"y":167.0},{"x":425.6826347305388,"y":167.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 index 8b54db8da7167..78a46aac82c11 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 @@ -1 +1 @@ -[{"x":9.761904761904763,"y":21.0},{"x":33.84210526315789,"y":19.0},{"x":62.75000000000001,"y":20.0},{"x":90.90322580645162,"y":31.0},{"x":122.91666666666667,"y":24.0},{"x":146.33333333333334,"y":21.0},{"x":170.70967741935485,"y":31.0},{"x":194.3571428571428,"y":28.0},{"x":214.84615384615384,"y":26.0},{"x":235.08695652173907,"y":23.0},{"x":257.80000000000007,"y":15.0},{"x":281.0333333333333,"y":30.0},{"x":298.0,"y":1.0},{"x":313.0000000000001,"y":29.0},{"x":339.5925925925926,"y":27.0},{"x":372.49999999999983,"y":24.0},{"x":402.23684210526324,"y":38.0},{"x":430.6896551724138,"y":29.0},{"x":462.32352941176464,"y":34.0},{"x":487.72413793103453,"y":29.0}] \ No newline at end of file +[{"x":9.761904761904763,"y":21.0},{"x":33.84210526315789,"y":19.0},{"x":62.75000000000001,"y":20.0},{"x":90.90322580645162,"y":31.0},{"x":122.91666666666667,"y":24.0},{"x":146.33333333333334,"y":21.0},{"x":170.70967741935485,"y":31.0},{"x":194.3571428571428,"y":28.0},{"x":214.84615384615384,"y":26.0},{"x":235.08695652173907,"y":23.0},{"x":257.80000000000007,"y":15.0},{"x":281.0333333333333,"y":30.0},{"x":298.0,"y":1.0},{"x":313.0000000000001,"y":29.0},{"x":339.5925925925926,"y":27.0},{"x":372.49999999999983,"y":24.0},{"x":402.23684210526324,"y":38.0},{"x":430.6896551724138,"y":29.0},{"x":462.32352941176464,"y":34.0},{"x":487.72413793103453,"y":29.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 index aed3f1e704750..4f7995f874388 100644 --- a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 +++ b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 @@ -1 +1 @@ -[{"x":0.0,"y":3.0},{"x":2.0,"y":1.0},{"x":4.75,"y":4.0},{"x":8.0,"y":1.0},{"x":9.5,"y":2.0},{"x":11.666666666666666,"y":3.0},{"x":15.0,"y":2.0},{"x":17.666666666666664,"y":3.0},{"x":19.5,"y":2.0},{"x":24.0,"y":2.0},{"x":26.333333333333336,"y":3.0},{"x":28.0,"y":1.0},{"x":30.0,"y":1.0},{"x":33.0,"y":1.0},{"x":34.75,"y":4.0},{"x":37.0,"y":2.0},{"x":41.666666666666664,"y":3.0},{"x":43.5,"y":2.0},{"x":47.0,"y":1.0},{"x":51.0,"y":2.0},{"x":53.5,"y":2.0},{"x":57.666666666666664,"y":3.0},{"x":64.5,"y":2.0},{"x":66.66666666666666,"y":3.0},{"x":69.75,"y":4.0},{"x":72.0,"y":2.0},{"x":74.0,"y":1.0},{"x":76.33333333333333,"y":3.0},{"x":78.0,"y":1.0},{"x":80.0,"y":1.0},{"x":82.0,"y":1.0},{"x":83.5,"y":4.0},{"x":85.5,"y":2.0},{"x":87.0,"y":1.0},{"x":90.0,"y":3.0},{"x":92.0,"y":1.0},{"x":95.33333333333333,"y":3.0},{"x":97.5,"y":4.0},{"x":100.0,"y":2.0},{"x":103.5,"y":4.0},{"x":105.0,"y":1.0},{"x":111.0,"y":1.0},{"x":113.33333333333333,"y":3.0},{"x":116.0,"y":1.0},{"x":118.0,"y":2.0},{"x":119.4,"y":5.0},{"x":125.33333333333333,"y":3.0},{"x":128.4,"y":5.0},{"x":131.0,"y":1.0},{"x":133.66666666666666,"y":3.0},{"x":136.66666666666666,"y":3.0},{"x":138.0,"y":4.0},{"x":143.0,"y":1.0},{"x":145.66666666666666,"y":3.0},{"x":149.33333333333331,"y":3.0},{"x":152.33333333333334,"y":3.0},{"x":155.5,"y":2.0},{"x":157.5,"y":2.0},{"x":160.0,"y":1.0},{"x":162.5,"y":2.0},{"x":164.5,"y":4.0},{"x":166.75,"y":4.0},{"x":168.8,"y":5.0},{"x":170.0,"y":1.0},{"x":172.0,"y":2.0},{"x":174.5,"y":4.0},{"x":176.33333333333331,"y":3.0},{"x":178.0,"y":1.0},{"x":179.33333333333331,"y":3.0},{"x":181.0,"y":1.0},{"x":183.0,"y":1.0},{"x":186.75,"y":4.0},{"x":189.0,"y":1.0},{"x":190.66666666666666,"y":3.0},{"x":192.75,"y":4.0},{"x":194.0,"y":1.0},{"x":195.33333333333331,"y":3.0},{"x":197.0,"y":2.0},{"x":199.4,"y":5.0},{"x":201.0,"y":1.0},{"x":202.66666666666669,"y":3.0},{"x":205.0,"y":2.0},{"x":207.0,"y":2.0},{"x":208.40000000000003,"y":5.0},{"x":213.33333333333331,"y":3.0},{"x":216.0,"y":2.0},{"x":217.33333333333331,"y":3.0},{"x":219.0,"y":2.0},{"x":221.33333333333331,"y":3.0},{"x":223.5,"y":4.0},{"x":226.0,"y":1.0},{"x":228.66666666666663,"y":3.0},{"x":230.0,"y":5.0},{"x":233.0,"y":2.0},{"x":235.0,"y":1.0},{"x":237.5,"y":4.0},{"x":239.0,"y":2.0},{"x":241.66666666666669,"y":3.0},{"x":244.0,"y":1.0},{"x":247.5,"y":2.0},{"x":249.0,"y":1.0},{"x":252.0,"y":1.0},{"x":255.5,"y":4.0},{"x":257.5,"y":2.0},{"x":260.0,"y":1.0},{"x":262.5,"y":2.0},{"x":265.3333333333333,"y":3.0},{"x":272.6,"y":5.0},{"x":274.5,"y":2.0},{"x":277.3333333333333,"y":6.0},{"x":280.0,"y":2.0},{"x":281.5,"y":4.0},{"x":283.5,"y":2.0},{"x":285.0,"y":1.0},{"x":286.5,"y":2.0},{"x":288.3333333333333,"y":3.0},{"x":291.5,"y":2.0},{"x":296.0,"y":1.0},{"x":298.0,"y":3.0},{"x":302.0,"y":1.0},{"x":305.5,"y":2.0},{"x":307.3333333333333,"y":3.0},{"x":309.0,"y":2.0},{"x":310.75,"y":4.0},{"x":315.75,"y":4.0},{"x":317.6,"y":5.0},{"x":321.5,"y":4.0},{"x":323.0,"y":1.0},{"x":325.0,"y":2.0},{"x":327.0,"y":3.0},{"x":331.3333333333333,"y":3.0},{"x":333.0,"y":2.0},{"x":335.5,"y":2.0},{"x":338.5,"y":2.0},{"x":341.66666666666663,"y":3.0},{"x":344.3333333333333,"y":3.0},{"x":348.0,"y":5.0},{"x":351.0,"y":1.0},{"x":353.0,"y":2.0},{"x":356.0,"y":1.0},{"x":360.0,"y":1.0},{"x":362.0,"y":1.0},{"x":364.5,"y":2.0},{"x":366.66666666666663,"y":3.0},{"x":368.75,"y":4.0},{"x":373.5,"y":2.0},{"x":375.0,"y":1.0},{"x":377.5,"y":2.0},{"x":379.0,"y":1.0},{"x":382.0,"y":2.0},{"x":384.0,"y":3.0},{"x":386.0,"y":1.0},{"x":389.0,"y":1.0},{"x":392.0,"y":1.0},{"x":393.5,"y":2.0},{"x":395.6,"y":5.0},{"x":397.0,"y":2.0},{"x":399.0,"y":2.0},{"x":400.0,"y":1.0},{"x":401.16666666666663,"y":6.0},{"x":403.40000000000003,"y":5.0},{"x":406.20000000000005,"y":5.0},{"x":409.0,"y":3.0},{"x":411.0,"y":1.0},{"x":413.5,"y":4.0},{"x":417.0,"y":3.0},{"x":418.5,"y":2.0},{"x":421.0,"y":1.0},{"x":424.0,"y":2.0},{"x":427.0,"y":1.0},{"x":429.6,"y":5.0},{"x":431.25,"y":4.0},{"x":435.5,"y":2.0},{"x":437.75,"y":4.0},{"x":439.0,"y":2.0},{"x":443.5,"y":2.0},{"x":446.0,"y":1.0},{"x":448.5,"y":2.0},{"x":452.5,"y":2.0},{"x":454.24999999999994,"y":4.0},{"x":457.66666666666663,"y":3.0},{"x":459.33333333333337,"y":3.0},{"x":462.5,"y":4.0},{"x":466.0,"y":3.0},{"x":467.80000000000007,"y":5.0},{"x":469.16666666666663,"y":6.0},{"x":472.0,"y":1.0},{"x":475.0,"y":1.0},{"x":477.0,"y":1.0},{"x":478.33333333333326,"y":3.0},{"x":480.25,"y":4.0},{"x":482.5,"y":2.0},{"x":484.5,"y":2.0},{"x":487.0,"y":1.0},{"x":489.2,"y":5.0},{"x":491.66666666666663,"y":3.0},{"x":493.0,"y":1.0},{"x":494.5,"y":2.0},{"x":496.0,"y":1.0},{"x":497.75,"y":4.0}] \ No newline at end of file +[{"x":0.0,"y":3.0},{"x":2.0,"y":1.0},{"x":4.75,"y":4.0},{"x":8.0,"y":1.0},{"x":9.5,"y":2.0},{"x":11.666666666666666,"y":3.0},{"x":15.0,"y":2.0},{"x":17.666666666666664,"y":3.0},{"x":19.5,"y":2.0},{"x":24.0,"y":2.0},{"x":26.333333333333336,"y":3.0},{"x":28.0,"y":1.0},{"x":30.0,"y":1.0},{"x":33.0,"y":1.0},{"x":34.75,"y":4.0},{"x":37.0,"y":2.0},{"x":41.666666666666664,"y":3.0},{"x":43.5,"y":2.0},{"x":47.0,"y":1.0},{"x":51.0,"y":2.0},{"x":53.5,"y":2.0},{"x":57.666666666666664,"y":3.0},{"x":64.5,"y":2.0},{"x":66.66666666666666,"y":3.0},{"x":69.75,"y":4.0},{"x":72.0,"y":2.0},{"x":74.0,"y":1.0},{"x":76.33333333333333,"y":3.0},{"x":78.0,"y":1.0},{"x":80.0,"y":1.0},{"x":82.0,"y":1.0},{"x":83.5,"y":4.0},{"x":85.5,"y":2.0},{"x":87.0,"y":1.0},{"x":90.0,"y":3.0},{"x":92.0,"y":1.0},{"x":95.33333333333333,"y":3.0},{"x":97.5,"y":4.0},{"x":100.0,"y":2.0},{"x":103.5,"y":4.0},{"x":105.0,"y":1.0},{"x":111.0,"y":1.0},{"x":113.33333333333333,"y":3.0},{"x":116.0,"y":1.0},{"x":118.0,"y":2.0},{"x":119.4,"y":5.0},{"x":125.33333333333333,"y":3.0},{"x":128.4,"y":5.0},{"x":131.0,"y":1.0},{"x":133.66666666666666,"y":3.0},{"x":136.66666666666666,"y":3.0},{"x":138.0,"y":4.0},{"x":143.0,"y":1.0},{"x":145.66666666666666,"y":3.0},{"x":149.33333333333331,"y":3.0},{"x":152.33333333333334,"y":3.0},{"x":155.5,"y":2.0},{"x":157.5,"y":2.0},{"x":160.0,"y":1.0},{"x":162.5,"y":2.0},{"x":164.5,"y":4.0},{"x":166.75,"y":4.0},{"x":168.8,"y":5.0},{"x":170.0,"y":1.0},{"x":172.0,"y":2.0},{"x":174.5,"y":4.0},{"x":176.33333333333331,"y":3.0},{"x":178.0,"y":1.0},{"x":179.33333333333331,"y":3.0},{"x":181.0,"y":1.0},{"x":183.0,"y":1.0},{"x":186.75,"y":4.0},{"x":189.0,"y":1.0},{"x":190.66666666666666,"y":3.0},{"x":192.75,"y":4.0},{"x":194.0,"y":1.0},{"x":195.33333333333331,"y":3.0},{"x":197.0,"y":2.0},{"x":199.4,"y":5.0},{"x":201.0,"y":1.0},{"x":202.66666666666669,"y":3.0},{"x":205.0,"y":2.0},{"x":207.0,"y":2.0},{"x":208.40000000000003,"y":5.0},{"x":213.33333333333331,"y":3.0},{"x":216.0,"y":2.0},{"x":217.33333333333331,"y":3.0},{"x":219.0,"y":2.0},{"x":221.33333333333331,"y":3.0},{"x":223.5,"y":4.0},{"x":226.0,"y":1.0},{"x":228.66666666666663,"y":3.0},{"x":230.0,"y":5.0},{"x":233.0,"y":2.0},{"x":235.0,"y":1.0},{"x":237.5,"y":4.0},{"x":239.0,"y":2.0},{"x":241.66666666666669,"y":3.0},{"x":244.0,"y":1.0},{"x":247.5,"y":2.0},{"x":249.0,"y":1.0},{"x":252.0,"y":1.0},{"x":255.5,"y":4.0},{"x":257.5,"y":2.0},{"x":260.0,"y":1.0},{"x":262.5,"y":2.0},{"x":265.3333333333333,"y":3.0},{"x":272.6,"y":5.0},{"x":274.5,"y":2.0},{"x":277.3333333333333,"y":6.0},{"x":280.0,"y":2.0},{"x":281.5,"y":4.0},{"x":283.5,"y":2.0},{"x":285.0,"y":1.0},{"x":286.5,"y":2.0},{"x":288.3333333333333,"y":3.0},{"x":291.5,"y":2.0},{"x":296.0,"y":1.0},{"x":298.0,"y":3.0},{"x":302.0,"y":1.0},{"x":305.5,"y":2.0},{"x":307.3333333333333,"y":3.0},{"x":309.0,"y":2.0},{"x":310.75,"y":4.0},{"x":315.75,"y":4.0},{"x":317.6,"y":5.0},{"x":321.5,"y":4.0},{"x":323.0,"y":1.0},{"x":325.0,"y":2.0},{"x":327.0,"y":3.0},{"x":331.3333333333333,"y":3.0},{"x":333.0,"y":2.0},{"x":335.5,"y":2.0},{"x":338.5,"y":2.0},{"x":341.66666666666663,"y":3.0},{"x":344.3333333333333,"y":3.0},{"x":348.0,"y":5.0},{"x":351.0,"y":1.0},{"x":353.0,"y":2.0},{"x":356.0,"y":1.0},{"x":360.0,"y":1.0},{"x":362.0,"y":1.0},{"x":364.5,"y":2.0},{"x":366.66666666666663,"y":3.0},{"x":368.75,"y":4.0},{"x":373.5,"y":2.0},{"x":375.0,"y":1.0},{"x":377.5,"y":2.0},{"x":379.0,"y":1.0},{"x":382.0,"y":2.0},{"x":384.0,"y":3.0},{"x":386.0,"y":1.0},{"x":389.0,"y":1.0},{"x":392.0,"y":1.0},{"x":393.5,"y":2.0},{"x":395.6,"y":5.0},{"x":397.0,"y":2.0},{"x":399.0,"y":2.0},{"x":400.0,"y":1.0},{"x":401.16666666666663,"y":6.0},{"x":403.40000000000003,"y":5.0},{"x":406.20000000000005,"y":5.0},{"x":409.0,"y":3.0},{"x":411.0,"y":1.0},{"x":413.5,"y":4.0},{"x":417.0,"y":3.0},{"x":418.5,"y":2.0},{"x":421.0,"y":1.0},{"x":424.0,"y":2.0},{"x":427.0,"y":1.0},{"x":429.6,"y":5.0},{"x":431.25,"y":4.0},{"x":435.5,"y":2.0},{"x":437.75,"y":4.0},{"x":439.0,"y":2.0},{"x":443.5,"y":2.0},{"x":446.0,"y":1.0},{"x":448.5,"y":2.0},{"x":452.5,"y":2.0},{"x":454.24999999999994,"y":4.0},{"x":457.66666666666663,"y":3.0},{"x":459.33333333333337,"y":3.0},{"x":462.5,"y":4.0},{"x":466.0,"y":3.0},{"x":467.80000000000007,"y":5.0},{"x":469.16666666666663,"y":6.0},{"x":472.0,"y":1.0},{"x":475.0,"y":1.0},{"x":477.0,"y":1.0},{"x":478.33333333333326,"y":3.0},{"x":480.25,"y":4.0},{"x":482.5,"y":2.0},{"x":484.5,"y":2.0},{"x":487.0,"y":1.0},{"x":489.2,"y":5.0},{"x":491.66666666666663,"y":3.0},{"x":493.0,"y":1.0},{"x":494.5,"y":2.0},{"x":496.0,"y":1.0},{"x":497.75,"y":4.0}] diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 deleted file mode 100644 index 2953abcf1e644..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 +++ /dev/null @@ -1 +0,0 @@ -0.0 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 deleted file mode 100644 index 17c47d308f029..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 +++ /dev/null @@ -1 +0,0 @@ -254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 deleted file mode 100644 index 60cbe79310729..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 +++ /dev/null @@ -1 +0,0 @@ -255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c deleted file mode 100644 index 014c315649096..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c +++ /dev/null @@ -1 +0,0 @@ -[23.355555555555558,254.08333333333334,476.5612244897959,489.50000000000006] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 deleted file mode 100644 index a8986e32ff75a..0000000000000 --- a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 +++ /dev/null @@ -1 +0,0 @@ -[26.0,255.5,479.0,491.0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 b/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da b/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f b/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 493daf5d79c54..0000000000000 --- a/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1 +0,0 @@ -true false true true true false false false true true false true true acc abc abb hive hadoop AaAbAcA false diff --git a/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 index 398f517b25b5f..2d3e77d99bca3 100644 --- a/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 +++ b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 @@ -1 +1 @@ -| abc | abc | abc | \ No newline at end of file +| abc | abc | abc | diff --git a/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c b/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 b/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e b/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index b2890dc4cdb3a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1 +0,0 @@ -0 NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 index 56a6051ca2b02..d00491fd7e5bb 100644 --- a/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 +++ b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 @@ -1 +1 @@ -1 \ No newline at end of file +1 diff --git a/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e b/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e index 132dbea91f8a0..5657917e40f7d 100644 --- a/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e +++ b/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e @@ -1 +1 @@ -1 2 2 a 0.1 2 126 128 128 1.0 128 \ No newline at end of file +1 2 2 a 0.1 2 126 128 128 1.0 128 diff --git a/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 b/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 index 51958ff45ab8d..2fcaff3dad9fe 100644 --- a/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 +++ b/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 @@ -1 +1 @@ -1.098612288668 NULL NULL 1.098612288668 NULL NULL 1.584962500721 NULL NULL 0.47712125472 NULL NULL 1.584962500721 NULL NULL NULL -1.0 7.389056098931 8.0 8.0 0.125 8.0 2.0 NaN 1.0 1 8 8 +1.098612288668 NULL NULL 1.098612288668 NULL NULL 1.584962500721 NULL NULL 0.47712125472 NULL NULL 1.584962500721 NULL NULL NULL -1.0 7.389056098931 8.0 8.0 0.125 8.0 2.0 NaN 1.0 1.0 8.0 8.0 diff --git a/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b index 8a89b039b7151..60b09adaf8029 100644 --- a/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b +++ b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b @@ -1 +1 @@ -1.0 1.0 1 \ No newline at end of file +1.0 1.0 1 diff --git a/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c b/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c index 83e2e121222a2..cb93f99495494 100644 --- a/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c +++ b/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c @@ -1 +1 @@ --1 2 32 -1 2009-01-01 2009-12-31 2008-03-01 2009-03-02 2008-02-28 2009-02-27 2008-12-31 2008-01-02 2008-02-28 2009-02-27 2006-02-28 2005-02-28 \ No newline at end of file +-1 2 32 -1 2009-01-01 2009-12-31 2008-03-01 2009-03-02 2008-02-28 2009-02-27 2008-12-31 2008-01-02 2008-02-28 2009-02-27 2006-02-28 2005-02-28 diff --git a/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_E-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_E-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/udf_E-1-cad0779d18f326c8e453bf2b5fe43596 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b rename to sql/hive/src/test/resources/golden/udf_E-1-cad0779d18f326c8e453bf2b5fe43596 diff --git a/sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c rename to sql/hive/src/test/resources/golden/udf_E-2-d0fd9aa04fdeb948bdcf8559f7095c02 diff --git a/sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a rename to sql/hive/src/test/resources/golden/udf_E-3-72bb4231ea2a877b4d93a53cd7b6b82a diff --git a/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-4-e8924af3bf99d2e01546a965303ffd09 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 rename to sql/hive/src/test/resources/golden/udf_E-4-e8924af3bf99d2e01546a965303ffd09 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 rename to sql/hive/src/test/resources/golden/udf_E-5-9d54c12bf727e05e9f9d67c61402a1d4 diff --git a/sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c rename to sql/hive/src/test/resources/golden/udf_E-6-d0fd9aa04fdeb948bdcf8559f7095c02 diff --git a/sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a rename to sql/hive/src/test/resources/golden/udf_E-7-72bb4231ea2a877b4d93a53cd7b6b82a diff --git a/sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-8-e8924af3bf99d2e01546a965303ffd09 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 rename to sql/hive/src/test/resources/golden/udf_E-8-e8924af3bf99d2e01546a965303ffd09 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_PI-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/udf_PI-1-13fd9345fd15b654d18b2707e5274b2b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 rename to sql/hive/src/test/resources/golden/udf_PI-1-13fd9345fd15b654d18b2707e5274b2b diff --git a/sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-2-97a12f6967726e425469ecfa70177ff0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 rename to sql/hive/src/test/resources/golden/udf_PI-2-97a12f6967726e425469ecfa70177ff0 diff --git a/sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-3-9c1476a2eab7455594e97b338ee3c188 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 rename to sql/hive/src/test/resources/golden/udf_PI-3-9c1476a2eab7455594e97b338ee3c188 diff --git a/sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-4-890f3c276eff2c459d8dc79d5a71c866 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 rename to sql/hive/src/test/resources/golden/udf_PI-4-890f3c276eff2c459d8dc79d5a71c866 diff --git a/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 rename to sql/hive/src/test/resources/golden/udf_PI-5-cd1c31c39277a02bab8e44f8c29a6c2d diff --git a/sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-6-97a12f6967726e425469ecfa70177ff0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 rename to sql/hive/src/test/resources/golden/udf_PI-6-97a12f6967726e425469ecfa70177ff0 diff --git a/sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-7-9c1476a2eab7455594e97b338ee3c188 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 rename to sql/hive/src/test/resources/golden/udf_PI-7-9c1476a2eab7455594e97b338ee3c188 diff --git a/sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-8-890f3c276eff2c459d8dc79d5a71c866 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 rename to sql/hive/src/test/resources/golden/udf_PI-8-890f3c276eff2c459d8dc79d5a71c866 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_abs-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 b/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 deleted file mode 100644 index b613b3b9fe967..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 +++ /dev/null @@ -1 +0,0 @@ -abs(x) - returns the absolute value of x \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 b/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 new file mode 100644 index 0000000000000..f1e01bf0d2fc8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-1-6fe2e69c338fc823d3f61c9236eb2234 @@ -0,0 +1 @@ +abs(x) - returns the absolute value of x diff --git a/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 b/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 deleted file mode 100644 index c2fb6dc49322d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 +++ /dev/null @@ -1,6 +0,0 @@ -abs(x) - returns the absolute value of x -Example: - > SELECT abs(0) FROM src LIMIT 1; - 0 - > SELECT abs(-5) FROM src LIMIT 1; - 5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 b/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 b/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 new file mode 100644 index 0000000000000..7dea2445fa616 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-2-eeb77ae8a0dcebbc0991923ca0932072 @@ -0,0 +1,6 @@ +abs(x) - returns the absolute value of x +Example: + > SELECT abs(0) FROM src LIMIT 1; + 0 + > SELECT abs(-5) FROM src LIMIT 1; + 5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/udf_abs-3-50cb3c23902cd29e0dbff188c71062e5 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 rename to sql/hive/src/test/resources/golden/udf_abs-3-50cb3c23902cd29e0dbff188c71062e5 diff --git a/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 b/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 deleted file mode 100644 index f5a78bc6e59b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 +++ /dev/null @@ -1 +0,0 @@ -0 1 123 9223372036854775807 9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c b/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 b/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 new file mode 100644 index 0000000000000..c7b1f75a06cb3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-4-4ae7f62f8d996f0066037cecbf2e01c4 @@ -0,0 +1 @@ +0 1 123 9223372036854775807 9223372036854775807 diff --git a/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 b/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 deleted file mode 100644 index 9e57fa65e8014..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 +++ /dev/null @@ -1 +0,0 @@ -0.0 3.14159265 3.14159265 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/udf_abs-5-5cd4c198e0de884ad436864b95fece6c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 rename to sql/hive/src/test/resources/golden/udf_abs-5-5cd4c198e0de884ad436864b95fece6c diff --git a/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b b/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b new file mode 100644 index 0000000000000..a07dd4e9d970d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-6-7aa32a019499c6464aded2e357c6843b @@ -0,0 +1 @@ +0.0 3.14159265 3.14159265 diff --git a/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 b/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 deleted file mode 100644 index 204db415144a7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 +++ /dev/null @@ -1 +0,0 @@ -acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_acos-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 b/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 new file mode 100644 index 0000000000000..b32b4244d6aeb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-1-4f49cb5a5c87efea534d63ed76435f06 @@ -0,0 +1 @@ +acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise diff --git a/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f b/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f deleted file mode 100644 index d49af1cbf4800..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f +++ /dev/null @@ -1,6 +0,0 @@ -acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise -Example: - > SELECT acos(1) FROM src LIMIT 1; - 0 - > SELECT acos(2) FROM src LIMIT 1; - NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b b/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f b/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f new file mode 100644 index 0000000000000..93925317cd798 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-2-d330511cf4f626cd844b18f57f99a85f @@ -0,0 +1,6 @@ +acos(x) - returns the arc cosine of x if -1<=x<=1 or NULL otherwise +Example: + > SELECT acos(1) FROM src LIMIT 1; + 0 + > SELECT acos(2) FROM src LIMIT 1; + NULL diff --git a/sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 b/sql/hive/src/test/resources/golden/udf_acos-3-661a0a85283df2a5c1567d60850e362b similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 rename to sql/hive/src/test/resources/golden/udf_acos-3-661a0a85283df2a5c1567d60850e362b diff --git a/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 b/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 deleted file mode 100644 index 5548bdb7cf26a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 +++ /dev/null @@ -1 +0,0 @@ -1.5707963267948966 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf b/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf new file mode 100644 index 0000000000000..53a6e14702ed1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-4-23d588eece08fbea7431044524f1cecf @@ -0,0 +1 @@ +1.5707963267948966 diff --git a/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a b/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a deleted file mode 100644 index 1f2d6faad9a2c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a +++ /dev/null @@ -1 +0,0 @@ -2.0943951023931957 0.7208187608700897 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e b/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e new file mode 100644 index 0000000000000..edee342d1d902 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-5-578612589fdb1ae21ee488924848fb4e @@ -0,0 +1 @@ +2.0943951023931957 0.7208187608700897 diff --git a/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e b/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 b/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-6-4d2bd33cee047e9a8bb740760c7cc3b4 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 +++ b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 +++ b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_array-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_array-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee b/sql/hive/src/test/resources/golden/udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee similarity index 100% rename from sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee rename to sql/hive/src/test/resources/golden/udf_array-1-2e9c2a7d9325bd9a53cf9f181b6333ee diff --git a/sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a b/sql/hive/src/test/resources/golden/udf_array-2-570741914bb78300b0233e5f38d7f08a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a rename to sql/hive/src/test/resources/golden/udf_array-2-570741914bb78300b0233e5f38d7f08a diff --git a/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 b/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/udf_array-3-47818d42e5e7667d8754c3f9a4b8053a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc rename to sql/hive/src/test/resources/golden/udf_array-3-47818d42e5e7667d8754c3f9a4b8053a diff --git a/sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 b/sql/hive/src/test/resources/golden/udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 rename to sql/hive/src/test/resources/golden/udf_array-4-51410e4d4d679fe5a8dd7a860f4efc47 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_array_contains-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a b/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a deleted file mode 100644 index 783e7d086a5c6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a +++ /dev/null @@ -1 +0,0 @@ -array_contains(array, value) - Returns TRUE if the array contains value. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a b/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a new file mode 100644 index 0000000000000..f17ecea1ab5ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array_contains-1-d9a90108b052b111e8de4433e008b25a @@ -0,0 +1 @@ +array_contains(array, value) - Returns TRUE if the array contains value. diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c b/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c deleted file mode 100644 index 0ccae5649af4a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c +++ /dev/null @@ -1,4 +0,0 @@ -array_contains(array, value) - Returns TRUE if the array contains value. -Example: - > SELECT array_contains(array(1, 2, 3), 2) FROM src LIMIT 1; - true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 b/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c b/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c new file mode 100644 index 0000000000000..7f4bbfc04c1c9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array_contains-2-eff16c7836252e01f3d8190cd833f79c @@ -0,0 +1,4 @@ +array_contains(array, value) - Returns TRUE if the array contains value. +Example: + > SELECT array_contains(array(1, 2, 3), 2) FROM src LIMIT 1; + true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c b/sql/hive/src/test/resources/golden/udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c rename to sql/hive/src/test/resources/golden/udf_array_contains-3-6b700cb69af298aef45b96bf5ac862d diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 b/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d b/sql/hive/src/test/resources/golden/udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d rename to sql/hive/src/test/resources/golden/udf_array_contains-4-bdb5a7e0ab81172a438145a1c406b1e8 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_ascii-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c b/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c deleted file mode 100644 index e121c27212c66..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c +++ /dev/null @@ -1 +0,0 @@ -ascii(str) - returns the numeric value of the first character of str \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c b/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c new file mode 100644 index 0000000000000..5e2c4b7209d9b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_ascii-1-72924c23459330cca6a54c70b12a542c @@ -0,0 +1 @@ +ascii(str) - returns the numeric value of the first character of str diff --git a/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 b/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 deleted file mode 100644 index 7c9fb835dfe57..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 +++ /dev/null @@ -1,6 +0,0 @@ -ascii(str) - returns the numeric value of the first character of str -Returns 0 if str is empty or NULL if str is NULL -Example: - > SELECT ascii('222') FROM src LIMIT 1; 50 - > SELECT ascii(2) FROM src LIMIT 1; - 50 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa b/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 b/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 new file mode 100644 index 0000000000000..87ae6b33499c4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_ascii-2-dab8656d7f001e85615442d60df4b6b3 @@ -0,0 +1,6 @@ +ascii(str) - returns the numeric value of the first character of str +Returns 0 if str is empty or NULL if str is NULL +Example: + > SELECT ascii('222') FROM src LIMIT 1; 50 + > SELECT ascii(2) FROM src LIMIT 1; + 50 diff --git a/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 b/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 deleted file mode 100644 index 726fda4c15ed3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 +++ /dev/null @@ -1 +0,0 @@ -70 0 33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e rename to sql/hive/src/test/resources/golden/udf_ascii-3-fc25cec86e0dafaf1633c2e3a6d2fc34 diff --git a/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf b/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf new file mode 100644 index 0000000000000..ba9d9a3b79bab --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_ascii-4-db9a06881a216f0252fa786d98c9bf @@ -0,0 +1 @@ +70 0 33 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_asin-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd b/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd deleted file mode 100644 index 3a56bffc8da34..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd +++ /dev/null @@ -1 +0,0 @@ -asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 b/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 deleted file mode 100644 index ef207d3f5da38..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 +++ /dev/null @@ -1,6 +0,0 @@ -asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise -Example: - > SELECT asin(0) FROM src LIMIT 1; - 0 - > SELECT asin(2) FROM src LIMIT 1; - NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd b/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd new file mode 100644 index 0000000000000..f5ffb5d0b3939 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-1-99da197a53767060e3fa0250254d59cd @@ -0,0 +1 @@ +asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise diff --git a/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 b/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 new file mode 100644 index 0000000000000..80c16be805ee8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-2-3d0c3fa6121f8f5158d221074f1d4129 @@ -0,0 +1,6 @@ +asin(x) - returns the arc sine of x if -1<=x<=1 or NULL otherwise +Example: + > SELECT asin(0) FROM src LIMIT 1; + 0 + > SELECT asin(2) FROM src LIMIT 1; + NULL diff --git a/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f b/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e b/sql/hive/src/test/resources/golden/udf_asin-3-4b7ee6310a49ebf784a4a712748348ac similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e rename to sql/hive/src/test/resources/golden/udf_asin-3-4b7ee6310a49ebf784a4a712748348ac diff --git a/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 b/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 b/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 deleted file mode 100644 index a9d9b65be4189..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 +++ /dev/null @@ -1 +0,0 @@ --0.5235987755982989 0.7208187608700897 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce b/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-4-a3edf78ff8ccc629ee7d7518707b69ce @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 b/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 b/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 new file mode 100644 index 0000000000000..46381bfe173d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-5-8dcbcf784496053e3b57c579aca809a6 @@ -0,0 +1 @@ +-0.5235987755982989 0.7208187608700897 diff --git a/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f b/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-6-114c8141f1e831c70d70c570f0ae778f @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_atan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b deleted file mode 100644 index 019676ad50bc3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b +++ /dev/null @@ -1 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 deleted file mode 100644 index ce9e52716ff44..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 +++ /dev/null @@ -1,4 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) -Example: - > SELECT atan(0) FROM src LIMIT 1; - 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..1d34577c5c6b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-1-c79ed30c2444c8493d0db98c33c9132b @@ -0,0 +1 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..6d7f1d49d9e72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-2-77e7ac1b89a4eac9102176cd73f67a62 @@ -0,0 +1,4 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) +Example: + > SELECT atan(0) FROM src LIMIT 1; + 0 diff --git a/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 deleted file mode 100644 index 37b3e6018d62c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 +++ /dev/null @@ -1 +0,0 @@ -0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e b/sql/hive/src/test/resources/golden/udf_atan-3-e6f97a834028a67e6c3033949f98fbf8 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e rename to sql/hive/src/test/resources/golden/udf_atan-3-e6f97a834028a67e6c3033949f98fbf8 diff --git a/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b deleted file mode 100644 index 019676ad50bc3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b +++ /dev/null @@ -1 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 b/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 new file mode 100644 index 0000000000000..fe43015585aa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-4-eacd47571ba5c67f11e025d8d4de5811 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 diff --git a/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 deleted file mode 100644 index ce9e52716ff44..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 +++ /dev/null @@ -1,4 +0,0 @@ -atan(x) - returns the atan (arctan) of x (x is in radians) -Example: - > SELECT atan(0) FROM src LIMIT 1; - 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..1d34577c5c6b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-5-c79ed30c2444c8493d0db98c33c9132b @@ -0,0 +1 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..6d7f1d49d9e72 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-6-77e7ac1b89a4eac9102176cd73f67a62 @@ -0,0 +1,4 @@ +atan(x) - returns the atan (arctan) of x (x is in radians) +Example: + > SELECT atan(0) FROM src LIMIT 1; + 0 diff --git a/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 deleted file mode 100644 index 37b3e6018d62c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 +++ /dev/null @@ -1 +0,0 @@ -0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/udf_atan-7-e6f97a834028a67e6c3033949f98fbf8 similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d rename to sql/hive/src/test/resources/golden/udf_atan-7-e6f97a834028a67e6c3033949f98fbf8 diff --git a/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 b/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 new file mode 100644 index 0000000000000..fe43015585aa4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-8-eacd47571ba5c67f11e025d8d4de5811 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 diff --git a/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 b/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 index bd171b0e9a29c..13ba346a52c93 100644 --- a/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 +++ b/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 @@ -1 +1 @@ -avg(x) - Returns the mean of a set of numbers \ No newline at end of file +avg(x) - Returns the mean of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f b/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f index bd171b0e9a29c..13ba346a52c93 100644 --- a/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f +++ b/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f @@ -1 +1 @@ -avg(x) - Returns the mean of a set of numbers \ No newline at end of file +avg(x) - Returns the mean of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 index 27b6cd42b68cf..c538e81220b1f 100644 --- a/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 +++ b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 @@ -1 +1 @@ -There is no documentation for function 'bigint' \ No newline at end of file +There is no documentation for function 'bigint' diff --git a/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 index 27b6cd42b68cf..c538e81220b1f 100644 --- a/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 +++ b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 @@ -1 +1 @@ -There is no documentation for function 'bigint' \ No newline at end of file +There is no documentation for function 'bigint' diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_bin-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b b/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b deleted file mode 100644 index 4790a2c1f9292..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b +++ /dev/null @@ -1 +0,0 @@ -bin(n) - returns n in binary \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 b/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 deleted file mode 100644 index 4d82712e858e4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 +++ /dev/null @@ -1,5 +0,0 @@ -bin(n) - returns n in binary -n is a BIGINT. Returns NULL if n is NULL. -Example: - > SELECT bin(13) FROM src LIMIT 1 - '1101' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b b/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b new file mode 100644 index 0000000000000..ef60184c54fb5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-1-ebbb090f6fa6b322a52bec3ba19dfe5b @@ -0,0 +1 @@ +bin(n) - returns n in binary diff --git a/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b b/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b deleted file mode 100644 index 20fae133f0b4d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b +++ /dev/null @@ -1 +0,0 @@ -1 0 101111101011100001101100101 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 b/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 new file mode 100644 index 0000000000000..115f93fe0fd3e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-2-843803a1b6ada107c11397af1a2f55d6 @@ -0,0 +1,5 @@ +bin(n) - returns n in binary +n is a BIGINT. Returns NULL if n is NULL. +Example: + > SELECT bin(13) FROM src LIMIT 1 + '1101' diff --git a/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab b/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab new file mode 100644 index 0000000000000..83b020539cb9a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-3-6fda27c8567ac896538cba3f2b230ab @@ -0,0 +1 @@ +1 0 101111101011100001101100101 diff --git a/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 b/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 deleted file mode 100644 index 76ea1be7f9214..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 +++ /dev/null @@ -1 +0,0 @@ -1111111111111111111111111111111111111111111111111111111111111011 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 b/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 new file mode 100644 index 0000000000000..1b12d2f6de475 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-4-b9bac215e81c8d5c8324b1287542ced3 @@ -0,0 +1 @@ +1111111111111111111111111111111111111111111111111111111111111011 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_bitmap_and-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d deleted file mode 100644 index 652a71b48011c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d +++ /dev/null @@ -1 +0,0 @@ -[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 deleted file mode 100644 index 6d061bf1198c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 +++ /dev/null @@ -1 +0,0 @@ -[13,1,4,2,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 new file mode 100644 index 0000000000000..1211192ad4be3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-c9e0d8424ec5f433565397b113ae4f57 @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 new file mode 100644 index 0000000000000..33b9ad33d4f01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-5a60dfc80bef392fa390adddab0c0f87 @@ -0,0 +1 @@ +[13,1,4,2,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-3-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-4-a486db1f5a06f9623a0e5abd7737b0c6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-5-549dbeb1293c4c49ae08bf08acdbdf23 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 b/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 deleted file mode 100644 index 43be09952b09c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 +++ /dev/null @@ -1,10 +0,0 @@ -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] -[13,1,4,2,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 new file mode 100644 index 0000000000000..0db773930c274 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-ff2860a163cbe78d5affac8047199296 @@ -0,0 +1,10 @@ +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] +[13,1,4,2,0] diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 rename to sql/hive/src/test/resources/golden/udf_bitmap_and-7-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a174269b5d1757398ab7f89cf1c97bfa diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 b/sql/hive/src/test/resources/golden/udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 rename to sql/hive/src/test/resources/golden/udf_bitmap_empty-2-f5d9880a3278b5632b356bbe6ecd90d3 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c deleted file mode 100644 index 652a71b48011c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c +++ /dev/null @@ -1 +0,0 @@ -[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef deleted file mode 100644 index 163f4734f4c21..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef +++ /dev/null @@ -1 +0,0 @@ -[13,2,4,8589934592,4224,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef new file mode 100644 index 0000000000000..1211192ad4be3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-cd510a5926df24d1ddbf8d0cce9d76ef @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 new file mode 100644 index 0000000000000..c1bd6bef82398 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-ab062e08acfd7e552a64ea967a0360c8 @@ -0,0 +1 @@ +[13,2,4,8589934592,4224,0] diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d b/sql/hive/src/test/resources/golden/udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d rename to sql/hive/src/test/resources/golden/udf_bitmap_or-3-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 b/sql/hive/src/test/resources/golden/udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-4-a486db1f5a06f9623a0e5abd7737b0c6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 b/sql/hive/src/test/resources/golden/udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-5-549dbeb1293c4c49ae08bf08acdbdf23 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 deleted file mode 100644 index c977a4ce25c3f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 +++ /dev/null @@ -1,10 +0,0 @@ -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] -[13,2,4,8589934592,4224,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 new file mode 100644 index 0000000000000..62f4e245e2cdd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-ea92fff4d814552b57535ed342a5dde0 @@ -0,0 +1,10 @@ +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] +[13,2,4,8589934592,4224,0] diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 b/sql/hive/src/test/resources/golden/udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 rename to sql/hive/src/test/resources/golden/udf_bitmap_or-7-d550d017f9eb9176593719ea34eaae9b diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a index 71cadba7649e8..bb00e0889dc07 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a +++ b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a @@ -1 +1 @@ -a & b - Bitwise and \ No newline at end of file +a & b - Bitwise and diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b b/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b index ead02cba53aa4..dc67d624d14be 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b +++ b/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b @@ -1,4 +1,4 @@ a & b - Bitwise and Example: > SELECT 3 & 5 FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f index 8b06e374c708d..d417f76be9940 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f +++ b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f @@ -1 +1 @@ -~ n - Bitwise not \ No newline at end of file +~ n - Bitwise not diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d b/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d index 413ca8a653e1f..db249e9032680 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d +++ b/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d @@ -1,4 +1,4 @@ ~ n - Bitwise not Example: > SELECT ~ 0 FROM src LIMIT 1; - -1 \ No newline at end of file + -1 diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 index bdbd28cad97c4..32343b4afe8bb 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 +++ b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 @@ -1 +1 @@ -a | b - Bitwise or \ No newline at end of file +a | b - Bitwise or diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb b/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb index 4c116be2acae3..8ced93b4f8e88 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb +++ b/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb @@ -1,4 +1,4 @@ a | b - Bitwise or Example: > SELECT 3 | 5 FROM src LIMIT 1; - 7 \ No newline at end of file + 7 diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 index b483f75f1b956..778fda3969951 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 +++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 @@ -1 +1 @@ -a ^ b - Bitwise exclusive or \ No newline at end of file +a ^ b - Bitwise exclusive or diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f index 03bf261d7cf51..6f1ab55c41ce7 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f +++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f @@ -1,4 +1,4 @@ a ^ b - Bitwise exclusive or Example: > SELECT 3 ^ 5 FROM src LIMIT 1; - 2 \ No newline at end of file + 2 diff --git a/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 index 8134698dfbdcd..495b36e68ecde 100644 --- a/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 +++ b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 @@ -1 +1 @@ -There is no documentation for function 'boolean' \ No newline at end of file +There is no documentation for function 'boolean' diff --git a/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e index 8134698dfbdcd..495b36e68ecde 100644 --- a/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e +++ b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e @@ -1 +1 @@ -There is no documentation for function 'boolean' \ No newline at end of file +There is no documentation for function 'boolean' diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udf_case-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 rename to sql/hive/src/test/resources/golden/udf_case-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 b/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 deleted file mode 100644 index 645bd7302810f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'case' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 b/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 deleted file mode 100644 index 645bd7302810f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'case' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 b/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 new file mode 100644 index 0000000000000..e5bc5948a0894 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-1-5bcbe4c0e0a75462160814a10b9449f4 @@ -0,0 +1 @@ +There is no documentation for function 'case' diff --git a/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 b/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 new file mode 100644 index 0000000000000..e5bc5948a0894 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-2-54acf006155d8822a50e47729be24004 @@ -0,0 +1 @@ +There is no documentation for function 'case' diff --git a/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f b/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 b/sql/hive/src/test/resources/golden/udf_case-3-48789112d79aeb450d9f49184fc20e1c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 rename to sql/hive/src/test/resources/golden/udf_case-3-48789112d79aeb450d9f49184fc20e1c diff --git a/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c b/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c deleted file mode 100644 index 1ca086d1dca4d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c +++ /dev/null @@ -1 +0,0 @@ -2 5 15 NULL 20 24 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 b/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 deleted file mode 100644 index b920295f69a53..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 +++ /dev/null @@ -1 +0,0 @@ -yo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 b/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 new file mode 100644 index 0000000000000..a7f5e19894027 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-4-d39ed6ecd256fa99657f13709cb1c6e3 @@ -0,0 +1 @@ +2 5 15 NULL 20 24 diff --git a/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb b/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb new file mode 100644 index 0000000000000..092bfb9bdf74d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-5-f53c9bb8a2d20ef7ff1fc7b3403270eb @@ -0,0 +1 @@ +yo diff --git a/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc b/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc new file mode 100644 index 0000000000000..db46b5a72aa80 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-6-ff583116ba2edd78202349faf1e757dc @@ -0,0 +1 @@ +123 123.0 abcd diff --git a/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c b/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 b/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 b/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 deleted file mode 100644 index d97a0ef52b2e8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 +++ /dev/null @@ -1,3 +0,0 @@ -1 zero 0 -3 10 is ten NULL -100 default NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 b/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 index 4854a3c7773ac..badaea42f1d2c 100644 --- a/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 +++ b/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 @@ -1 +1 @@ -ceil(x) - Find the smallest integer not smaller than x \ No newline at end of file +ceil(x) - Find the smallest integer not smaller than x diff --git a/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb b/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb index 23685df97892d..7d608c5ebd2a3 100644 --- a/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb +++ b/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb @@ -4,4 +4,4 @@ Example: > SELECT ceil(-0.1) FROM src LIMIT 1; 0 > SELECT ceil(5) FROM src LIMIT 1; - 5 \ No newline at end of file + 5 diff --git a/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd b/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd index bf61b7e89d8b3..31d18aaefac89 100644 --- a/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd +++ b/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd @@ -1 +1 @@ -ceiling(x) - Find the smallest integer not smaller than x \ No newline at end of file +ceiling(x) - Find the smallest integer not smaller than x diff --git a/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df b/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df index 79a06f0ee2a5d..564d0566997d4 100644 --- a/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df +++ b/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df @@ -4,4 +4,4 @@ Example: > SELECT ceiling(-0.1) FROM src LIMIT 1; 0 > SELECT ceiling(5) FROM src LIMIT 1; - 5 \ No newline at end of file + 5 diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 b/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 deleted file mode 100644 index 66313c54cc684..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 +++ /dev/null @@ -1 +0,0 @@ -coalesce(a1, a2, ...) - Returns the first non-null argument \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 b/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 deleted file mode 100644 index 04d69977c3df4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 +++ /dev/null @@ -1,4 +0,0 @@ -coalesce(a1, a2, ...) - Returns the first non-null argument -Example: - > SELECT coalesce(NULL, 1, NULL) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 b/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 b/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 deleted file mode 100644 index cd5686d89eba4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 +++ /dev/null @@ -1 +0,0 @@ -1 1 2 1 3 4 1 1 2 1 3 4 1.0 1.0 2.0 2.0 2.0 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 b/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 b/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 deleted file mode 100644 index 98339f97ef2f8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 +++ /dev/null @@ -1,11 +0,0 @@ -0 0 999 -2 1 999 -4 8 value_2 -6 27 999 -8 64 999 -10 125 999 -12 216 999 -14 343 999 -16 512 999 -18 729 999 -999 999 999 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 b/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 deleted file mode 100644 index 59d3f01051903..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_udf_get_java_string 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_concat-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 deleted file mode 100644 index a3998eeeee623..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 +++ /dev/null @@ -1 +0,0 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 b/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 deleted file mode 100644 index 0afd542a05033..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 +++ /dev/null @@ -1,5 +0,0 @@ -concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN -Returns NULL if any argument is NULL. -Example: - > SELECT concat('abc', 'def') FROM src LIMIT 1; - 'abcdef' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 new file mode 100644 index 0000000000000..b576089faa484 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-1-7bc53505a4e6587132870d8d0a704d2 @@ -0,0 +1 @@ +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN diff --git a/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 b/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 deleted file mode 100644 index 69af4e63792dc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 +++ /dev/null @@ -1 +0,0 @@ -ab abc NULL NULL a NULL 123a 12 1 1234abcextra argument \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 b/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 new file mode 100644 index 0000000000000..e303d18a08100 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-2-765c520f239cdff1ea3f8d22ba83e031 @@ -0,0 +1,5 @@ +concat(str1, str2, ... strN) - returns the concatenation of str1, str2, ... strN or concat(bin1, bin2, ... binN) - returns the concatenation of bytes in binary data bin1, bin2, ... binN +Returns NULL if any argument is NULL. +Example: + > SELECT concat('abc', 'def') FROM src LIMIT 1; + 'abcdef' diff --git a/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 b/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 new file mode 100644 index 0000000000000..21f03c7abb9b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-3-a38183c2685e912befe6246f1b6f93b8 @@ -0,0 +1 @@ +ab abc NULL NULL a NULL 123a 12 1 1234abcextra argument diff --git a/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde b/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde new file mode 100644 index 0000000000000..09b34f2aa1b69 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat-4-a20ebbc181e5ee4a1c22ddafd212ddde @@ -0,0 +1 @@ +abcd abcd diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f index aecd5d9f2a5c2..f4cd225e4b2b7 100644 --- a/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f @@ -54,4 +54,4 @@ 1234 95 1234 96 1234 97 -1234 98 \ No newline at end of file +1234 98 diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f index 455cef3f57162..fcdbdaa75fd52 100644 --- a/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f +++ b/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f @@ -81,4 +81,4 @@ 1234abcextra argument val_28 1234abcextra argument val_37 1234abcextra argument val_90 -1234abcextra argument val_97 \ No newline at end of file +1234abcextra argument val_97 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 b/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 deleted file mode 100644 index c901f70500cba..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 +++ /dev/null @@ -1 +0,0 @@ -concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/udf_concat_ws-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 b/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 new file mode 100644 index 0000000000000..14d190e03b755 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-1-4c8bd7794a527e544c838408313eeaa8 @@ -0,0 +1 @@ +concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab b/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab deleted file mode 100644 index 3e751463e055d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab +++ /dev/null @@ -1,4 +0,0 @@ -concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. -Example: - > SELECT concat_ws('.', 'www', array('facebook', 'com')) FROM src LIMIT 1; - 'www.facebook.com' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab b/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab new file mode 100644 index 0000000000000..f3be71e54421c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-2-b8b80f7e9bf4348367444c73020b3cab @@ -0,0 +1,4 @@ +concat_ws(separator, [string | array(string)]+) - returns the concatenation of the strings separated by the separator. +Example: + > SELECT concat_ws('.', 'www', array('facebook', 'com')) FROM src LIMIT 1; + 'www.facebook.com' diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c b/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 b/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e b/sql/hive/src/test/resources/golden/udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e rename to sql/hive/src/test/resources/golden/udf_concat_ws-3-ce7c8205cdc107e1fb865d7d48b84a3c diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f b/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 b/sql/hive/src/test/resources/golden/udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 rename to sql/hive/src/test/resources/golden/udf_concat_ws-4-b13a1f7f63d2a54efa331c82bd635d63 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 b/sql/hive/src/test/resources/golden/udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 rename to sql/hive/src/test/resources/golden/udf_concat_ws-5-a507af4996b13433b0ae100fcb32358f diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 b/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 deleted file mode 100644 index 7896fd787f3a2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 +++ /dev/null @@ -1 +0,0 @@ -xyzabc8675309 abc,xyz,8675309 NULL abc**8675309 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 b/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 b/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 new file mode 100644 index 0000000000000..720cafd9370a7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-6-ca4f051369183cae36fc9a7bec6a9341 @@ -0,0 +1 @@ +xyzabc8675309 abc,xyz,8675309 NULL abc**8675309 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c b/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c deleted file mode 100644 index a0ec688a3b084..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c +++ /dev/null @@ -1 +0,0 @@ -www.face.book.com.1234 www-face-book-com-1234 wwwFfaceFbookFcomF1234 www_face_book_com_1234 www**face**book**com**1234 www[]face[]book[]com[]1234 wwwAAAfaceAAAbookAAAcomAAA1234 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c b/sql/hive/src/test/resources/golden/udf_concat_ws-7-97071809ba7701b78e3729996f14b591 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c rename to sql/hive/src/test/resources/golden/udf_concat_ws-7-97071809ba7701b78e3729996f14b591 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 b/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 new file mode 100644 index 0000000000000..93b36d28322c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-8-3bfc563ebf7e2cdb811766a54d84f224 @@ -0,0 +1 @@ +www.face.book.com.1234 www-face-book-com-1234 wwwFfaceFbookFcomF1234 www_face_book_com_1234 www**face**book**com**1234 www[]face[]book[]com[]1234 wwwAAAfaceAAAbookAAAcomAAA1234 diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 b/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 deleted file mode 100644 index 0c9f2d12ba117..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 b/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 new file mode 100644 index 0000000000000..edb4b1f84001b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_concat_ws-9-8f60d81b410f4825809aa510806f2df2 @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to sql/hive/src/test/resources/golden/udf_conv-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a b/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a deleted file mode 100644 index b8fbe88a19971..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a +++ /dev/null @@ -1 +0,0 @@ -conv(num, from_base, to_base) - convert num from from_base to to_base \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab b/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab deleted file mode 100644 index 4f4b0c594c459..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab +++ /dev/null @@ -1,7 +0,0 @@ -conv(num, from_base, to_base) - convert num from from_base to to_base -If to_base is negative, treat num as a signed integer,otherwise, treat it as an unsigned integer. -Example: - > SELECT conv('100', 2, 10) FROM src LIMIT 1; - '4' - > SELECT conv(-10, 16, -10) FROM src LIMIT 1; - '16' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a b/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a new file mode 100644 index 0000000000000..0753228c31bd6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-1-d552befca345f3396464529cfde9f75a @@ -0,0 +1 @@ +conv(num, from_base, to_base) - convert num from from_base to to_base diff --git a/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 b/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 deleted file mode 100644 index 9512cc4241554..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 +++ /dev/null @@ -1,3 +0,0 @@ -EE 568 -56 134 -137 785 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 b/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 new file mode 100644 index 0000000000000..191900972dd95 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-10-f9ea15984e84250494e81e25d6a401c0 @@ -0,0 +1 @@ +5 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f b/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f new file mode 100644 index 0000000000000..ad3fa0267fa6b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-11-2ce9111b47ed100bebc8d08de54efd1f @@ -0,0 +1,3 @@ +EE 568 +56 134 +137 785 diff --git a/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab b/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab new file mode 100644 index 0000000000000..2d66703d6f3f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-2-5e5904af10b5d23f20ae28dc4b9a49ab @@ -0,0 +1,7 @@ +conv(num, from_base, to_base) - convert num from from_base to to_base +If to_base is negative, treat num as a signed integer,otherwise, treat it as an unsigned integer. +Example: + > SELECT conv('100', 2, 10) FROM src LIMIT 1; + '4' + > SELECT conv(-10, 16, -10) FROM src LIMIT 1; + '16' diff --git a/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 b/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 deleted file mode 100644 index 91e56a11fa628..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 +++ /dev/null @@ -1 +0,0 @@ -3HL 22 33 116ED2B2FB4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a b/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a new file mode 100644 index 0000000000000..4563fcc478648 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-3-5f43d0bec0421c86c49e2546c5ee923a @@ -0,0 +1 @@ +3HL 22 33 116ED2B2FB4 diff --git a/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d b/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d deleted file mode 100644 index 08f70d742b8e4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d +++ /dev/null @@ -1 +0,0 @@ --641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 b/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 deleted file mode 100644 index a2a44daa718cc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 +++ /dev/null @@ -1 +0,0 @@ -FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a b/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a new file mode 100644 index 0000000000000..632c3705a0493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-4-5df8d45902a3537e67545e629a96328a @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 b/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 deleted file mode 100644 index 6948fe3783119..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 +++ /dev/null @@ -1 +0,0 @@ -5 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 b/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 new file mode 100644 index 0000000000000..3a7a2ec34f909 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-5-8cdbb45b8c44fa97456da5bc4a43f459 @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a b/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a deleted file mode 100644 index e1021e50fdcbb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a +++ /dev/null @@ -1 +0,0 @@ -3HL 22 33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f b/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f new file mode 100644 index 0000000000000..191900972dd95 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-6-e5430adfa782ea9094e570d339626c0f @@ -0,0 +1 @@ +5 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b b/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b new file mode 100644 index 0000000000000..c5348e173c243 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-7-687f9b8a09f458d771d5641eec40031b @@ -0,0 +1 @@ +3HL 22 33 diff --git a/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e b/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e deleted file mode 100644 index 08f70d742b8e4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e +++ /dev/null @@ -1 +0,0 @@ --641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb b/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb deleted file mode 100644 index a2a44daa718cc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb +++ /dev/null @@ -1 +0,0 @@ -FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 b/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 new file mode 100644 index 0000000000000..632c3705a0493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-8-384902bf8d45b6b56b2bdc5db550c10 @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 b/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 new file mode 100644 index 0000000000000..3a7a2ec34f909 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-9-28b833d0cd96f74c23aa7cf8c4f5a167 @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 diff --git a/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 b/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 deleted file mode 100644 index 6948fe3783119..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 +++ /dev/null @@ -1 +0,0 @@ -5 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 b/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 deleted file mode 100644 index 99a7d42bb5b57..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 +++ /dev/null @@ -1 +0,0 @@ -cos(x) - returns the cosine of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_cos-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 b/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 deleted file mode 100644 index 196294de1f19f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 +++ /dev/null @@ -1,4 +0,0 @@ -cos(x) - returns the cosine of x (x is in radians) -Example: - > SELECT cos(0) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 b/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 new file mode 100644 index 0000000000000..fa0e6975503cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-1-44f411146a7190b89f2bc8b4aa61cae3 @@ -0,0 +1 @@ +cos(x) - returns the cosine of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 b/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 new file mode 100644 index 0000000000000..09d6150733802 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-2-176030bdf43ff83ed8b3112d0c79f2f5 @@ -0,0 +1,4 @@ +cos(x) - returns the cosine of x (x is in radians) +Example: + > SELECT cos(0) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 b/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/sql/hive/src/test/resources/golden/udf_cos-3-166acc86afd6ececfe43800e38f106c9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 rename to sql/hive/src/test/resources/golden/udf_cos-3-166acc86afd6ececfe43800e38f106c9 diff --git a/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 b/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 deleted file mode 100644 index c0690acff887b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 +++ /dev/null @@ -1 +0,0 @@ -0.5570225467662173 7.963267107332633E-4 0.8775825618903728 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 b/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 new file mode 100644 index 0000000000000..14417ab71b694 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-4-efda2f85872c6144280970eab84ef4d4 @@ -0,0 +1 @@ +0.5570225467662173 7.963267107332633E-4 0.8775825618903728 diff --git a/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 b/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 index e01715295adc8..56cf7bdf7b040 100644 --- a/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 +++ b/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 @@ -1,3 +1,3 @@ count(*) - Returns the total number of retrieved rows, including rows containing NULL values. count(expr) - Returns the number of rows for which the supplied expression is non-NULL. -count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. \ No newline at end of file +count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. diff --git a/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f b/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f index e01715295adc8..56cf7bdf7b040 100644 --- a/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f +++ b/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f @@ -1,3 +1,3 @@ count(*) - Returns the total number of retrieved rows, including rows containing NULL values. count(expr) - Returns the number of rows for which the supplied expression is non-NULL. -count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. \ No newline at end of file +count(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL. diff --git a/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e +++ b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e +++ b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 +++ b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 +++ b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 index e0da8ae09ae60..7536e3d326192 100644 --- a/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 +++ b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 @@ -1 +1 @@ -309 \ No newline at end of file +309 diff --git a/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 index e0da8ae09ae60..7536e3d326192 100644 --- a/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 +++ b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 @@ -1 +1 @@ -309 \ No newline at end of file +309 diff --git a/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 index eb1f49486af7c..1b79f38e25b24 100644 --- a/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 +++ b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 @@ -1 +1 @@ -500 \ No newline at end of file +500 diff --git a/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 b/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 index 83e0fc4e63a25..2e77bafd12f7d 100644 --- a/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 +++ b/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 @@ -1 +1 @@ -date_add(start_date, num_days) - Returns the date that is num_days after start_date. \ No newline at end of file +date_add(start_date, num_days) - Returns the date that is num_days after start_date. diff --git a/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 b/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 index 83b9851499f71..3c91e138d7bd5 100644 --- a/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 +++ b/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 @@ -2,4 +2,4 @@ date_add(start_date, num_days) - Returns the date that is num_days after start_d start_date is a string in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. num_days is a number. The time part of start_date is ignored. Example: > SELECT date_add('2009-30-07', 1) FROM src LIMIT 1; - '2009-31-07' \ No newline at end of file + '2009-31-07' diff --git a/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 b/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 index 7dec81f2b6d4e..3ee6ac4820852 100644 --- a/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 +++ b/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 @@ -1 +1 @@ -date_sub(start_date, num_days) - Returns the date that is num_days before start_date. \ No newline at end of file +date_sub(start_date, num_days) - Returns the date that is num_days before start_date. diff --git a/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 b/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 index 105b63424062b..29d663f35c586 100644 --- a/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 +++ b/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 @@ -2,4 +2,4 @@ date_sub(start_date, num_days) - Returns the date that is num_days before start_ start_date is a string in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. num_days is a number. The time part of start_date is ignored. Example: > SELECT date_sub('2009-30-07', 1) FROM src LIMIT 1; - '2009-29-07' \ No newline at end of file + '2009-29-07' diff --git a/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 b/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 index b59d456397d53..64cae647c8005 100644 --- a/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 +++ b/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 @@ -1 +1 @@ -datediff(date1, date2) - Returns the number of days between date1 and date2 \ No newline at end of file +datediff(date1, date2) - Returns the number of days between date1 and date2 diff --git a/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 index c240df94594d4..7ccaee7ad3bd4 100644 --- a/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 +++ b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 @@ -2,4 +2,4 @@ datediff(date1, date2) - Returns the number of days between date1 and date2 date1 and date2 are strings in the format 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. The time parts are ignored.If date1 is earlier than date2, the result is negative. Example: > SELECT datediff('2009-30-07', '2009-31-07') FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 b/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 index 11e32a4509b3f..d4017178b4e6b 100644 --- a/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 +++ b/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 @@ -1 +1 @@ -day(date) - Returns the date of the month of date \ No newline at end of file +day(date) - Returns the date of the month of date diff --git a/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 index 9da0858ba92a5..6135aafa50860 100644 --- a/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 +++ b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 @@ -3,4 +3,4 @@ Synonyms: dayofmonth date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. Example: > SELECT day('2009-30-07', 1) FROM src LIMIT 1; - 30 \ No newline at end of file + 30 diff --git a/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 b/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 index 33e90a2af295f..47a7018d9d5ac 100644 --- a/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 +++ b/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 @@ -1 +1 @@ -dayofmonth(date) - Returns the date of the month of date \ No newline at end of file +dayofmonth(date) - Returns the date of the month of date diff --git a/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 index ee9911af3248c..d9490e20a3b6d 100644 --- a/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 +++ b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 @@ -3,4 +3,4 @@ Synonyms: day date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'yyyy-MM-dd'. Example: > SELECT dayofmonth('2009-30-07', 1) FROM src LIMIT 1; - 30 \ No newline at end of file + 30 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/udf_degrees-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_degrees-1-f24ce67606944e23a4adc79f91cf0c17 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 rename to sql/hive/src/test/resources/golden/udf_degrees-2-aaee5dd4e87eaae3e65a585e07f1a3e4 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b rename to sql/hive/src/test/resources/golden/udf_degrees-3-2fd3a55901fe765f8f154531a7f5fd6b diff --git a/sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 rename to sql/hive/src/test/resources/golden/udf_degrees-4-42f653c3c3cc3c94bb9ab9c5a4d1ca96 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_degrees-5-3a6468b02be2605c91b31987e76fb9a8 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 rename to sql/hive/src/test/resources/golden/udf_degrees-6-aaee5dd4e87eaae3e65a585e07f1a3e4 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b rename to sql/hive/src/test/resources/golden/udf_degrees-7-2fd3a55901fe765f8f154531a7f5fd6b diff --git a/sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 rename to sql/hive/src/test/resources/golden/udf_degrees-8-42f653c3c3cc3c94bb9ab9c5a4d1ca96 diff --git a/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db b/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db deleted file mode 100644 index d98718752a36f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db +++ /dev/null @@ -1 +0,0 @@ -a div b - Divide a by b rounded to the long integer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/udf_div-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e rename to sql/hive/src/test/resources/golden/udf_div-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db b/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db new file mode 100644 index 0000000000000..b02a7003ce768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_div-1-31d31c7d5c544327dabfd874c88314db @@ -0,0 +1 @@ +a div b - Divide a by b rounded to the long integer diff --git a/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 b/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 deleted file mode 100644 index 59265a74f9b3f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 +++ /dev/null @@ -1,4 +0,0 @@ -a div b - Divide a by b rounded to the long integer -Example: - > SELECT 3 div 2 FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b b/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 b/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 new file mode 100644 index 0000000000000..6b79eff6e6092 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_div-2-f23a07b1c6b1a98b303863188c10a8d8 @@ -0,0 +1,4 @@ +a div b - Divide a by b rounded to the long integer +Example: + > SELECT 3 div 2 FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 b/sql/hive/src/test/resources/golden/udf_div-3-5111340caad64e36370d9d4bc4db5f27 similarity index 100% rename from sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 rename to sql/hive/src/test/resources/golden/udf_div-3-5111340caad64e36370d9d4bc4db5f27 diff --git a/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 b/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 deleted file mode 100644 index 8b623e47785f6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 +++ /dev/null @@ -1 +0,0 @@ -a / b - Divide a by b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 b/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 deleted file mode 100644 index 2acf2b6a64648..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 +++ /dev/null @@ -1,4 +0,0 @@ -a / b - Divide a by b -Example: - > SELECT 3 / 2 FROM src LIMIT 1; - 1.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 b/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 deleted file mode 100644 index 400122e60f599..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 +++ /dev/null @@ -1 +0,0 @@ -1.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 index 54de9e9f8edec..1d0a61f1b635f 100644 --- a/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 +++ b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 @@ -1 +1 @@ -There is no documentation for function 'double' \ No newline at end of file +There is no documentation for function 'double' diff --git a/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 index 54de9e9f8edec..1d0a61f1b635f 100644 --- a/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 +++ b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 @@ -1 +1 @@ -There is no documentation for function 'double' \ No newline at end of file +There is no documentation for function 'double' diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 b/sql/hive/src/test/resources/golden/udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 rename to sql/hive/src/test/resources/golden/udf_elt-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f b/sql/hive/src/test/resources/golden/udf_elt-1-b46b060da76d1772db998c26a62a608f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f rename to sql/hive/src/test/resources/golden/udf_elt-1-b46b060da76d1772db998c26a62a608f diff --git a/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a b/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 b/sql/hive/src/test/resources/golden/udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 rename to sql/hive/src/test/resources/golden/udf_elt-2-e9f1bf17ad2a6f7bf3e40798ceebdbf4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_elt-3-c2554fac72a2a51bb33faae16aec3507 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_elt-3-c2554fac72a2a51bb33faae16aec3507 diff --git a/sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 b/sql/hive/src/test/resources/golden/udf_elt-4-533ad9c703c320a6556c09dd5f9ac351 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 rename to sql/hive/src/test/resources/golden/udf_elt-4-533ad9c703c320a6556c09dd5f9ac351 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 rename to sql/hive/src/test/resources/golden/udf_equal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 b/sql/hive/src/test/resources/golden/udf_equal-1-36b6cdf7c5f68c91155569b1622f5876 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-0-36b6cdf7c5f68c91155569b1622f5876 rename to sql/hive/src/test/resources/golden/udf_equal-1-36b6cdf7c5f68c91155569b1622f5876 diff --git a/sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 b/sql/hive/src/test/resources/golden/udf_equal-2-2422b50b96502dde8b661acdfebd8892 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-1-2422b50b96502dde8b661acdfebd8892 rename to sql/hive/src/test/resources/golden/udf_equal-2-2422b50b96502dde8b661acdfebd8892 diff --git a/sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 b/sql/hive/src/test/resources/golden/udf_equal-3-e0faab0f5e736c24bcc5503aeac55053 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-2-e0faab0f5e736c24bcc5503aeac55053 rename to sql/hive/src/test/resources/golden/udf_equal-3-e0faab0f5e736c24bcc5503aeac55053 diff --git a/sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 b/sql/hive/src/test/resources/golden/udf_equal-4-39d8d6f197803de927f0af5409ec2f33 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-3-39d8d6f197803de927f0af5409ec2f33 rename to sql/hive/src/test/resources/golden/udf_equal-4-39d8d6f197803de927f0af5409ec2f33 diff --git a/sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 b/sql/hive/src/test/resources/golden/udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-4-94ac2476006425e1b3bcddf29ad07b16 rename to sql/hive/src/test/resources/golden/udf_equal-5-ee018fc3267dbdd55b60ed4e6f56c9ca diff --git a/sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 b/sql/hive/src/test/resources/golden/udf_equal-6-878650cf21e9360a07d204c8ffb0cde7 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-5-878650cf21e9360a07d204c8ffb0cde7 rename to sql/hive/src/test/resources/golden/udf_equal-6-878650cf21e9360a07d204c8ffb0cde7 diff --git a/sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e b/sql/hive/src/test/resources/golden/udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-6-1635ef051fecdfc7891d9f5a9a3a545e rename to sql/hive/src/test/resources/golden/udf_equal-7-1635ef051fecdfc7891d9f5a9a3a545e diff --git a/sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 b/sql/hive/src/test/resources/golden/udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_equal-7-78f1b96c199e307714fa1b804e5bae27 rename to sql/hive/src/test/resources/golden/udf_equal-8-276101b04b10b7cd6d59061a8cbf42d2 diff --git a/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a b/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a index 9b3dbf316d1fb..2a5080be93ac4 100644 --- a/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a +++ b/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a @@ -1 +1 @@ -exp(x) - Returns e to the power of x \ No newline at end of file +exp(x) - Returns e to the power of x diff --git a/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 b/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 index a42c95bb8d213..b5a4d037f4014 100644 --- a/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 +++ b/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 @@ -1,4 +1,4 @@ exp(x) - Returns e to the power of x Example: > SELECT exp(0) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 b/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 deleted file mode 100644 index a30bc26f5ba58..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 +++ /dev/null @@ -1 +0,0 @@ -field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udf_field-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 rename to sql/hive/src/test/resources/golden/udf_field-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 b/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 new file mode 100644 index 0000000000000..2e6133785ac7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-1-277b4a4dcb38cabb6df50147c77e0a33 @@ -0,0 +1 @@ +field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found diff --git a/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f b/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f deleted file mode 100644 index bb55c0a1db1f5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f +++ /dev/null @@ -1,2 +0,0 @@ -field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found -All primitive types are supported, arguments are compared using str.equals(x). If str is NULL, the return value is 0. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 b/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 deleted file mode 100644 index 275f46482425d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 +++ /dev/null @@ -1,2 +0,0 @@ -86 val_86 0 0 2 1 1 0 0 -66 val_66 1 1 0 0 0 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-10-ca9db7e6bb687606bc273d1f6c191035 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_field-10-ca9db7e6bb687606bc273d1f6c191035 diff --git a/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 b/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 new file mode 100644 index 0000000000000..a13456f1bfdda --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-11-7982ea72163dbc4cd45f53454edf66c8 @@ -0,0 +1,2 @@ +86 val_86 0 0 2 1 1 0 0 +66 val_66 1 1 0 0 0 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f b/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f new file mode 100644 index 0000000000000..f9d418fee7b53 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-2-379d8580693493f620a8f4084709324f @@ -0,0 +1,2 @@ +field(str, str1, str2, ...) - returns the index of str in the str1,str2,... list or 0 if not found +All primitive types are supported, arguments are compared using str.equals(x). If str is NULL, the return value is 0. diff --git a/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b b/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b deleted file mode 100644 index c42823854fb0f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b +++ /dev/null @@ -1 +0,0 @@ -0 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 b/sql/hive/src/test/resources/golden/udf_field-3-5960d42257b272f6ba043406229bbf26 similarity index 100% rename from sql/hive/src/test/resources/golden/div-0-5e7fc5719c7265bc1d6af52005ebad03 rename to sql/hive/src/test/resources/golden/udf_field-3-5960d42257b272f6ba043406229bbf26 diff --git a/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 b/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 deleted file mode 100644 index 5869234249808..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 +++ /dev/null @@ -1 +0,0 @@ -1 2 3 4 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 b/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 new file mode 100644 index 0000000000000..e5449f0bfa473 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-4-212d8b2297bf6a3311d24d68de67b5c6 @@ -0,0 +1 @@ +1 2 3 4 4 diff --git a/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 b/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 deleted file mode 100644 index 5869234249808..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 +++ /dev/null @@ -1 +0,0 @@ -1 2 3 4 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 b/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b b/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b new file mode 100644 index 0000000000000..e5449f0bfa473 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-5-949c2de113b214d83734c0c177f04b6b @@ -0,0 +1 @@ +1 2 3 4 4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_field-6-2d7c5cbe891c4a9dda34f9842f8e0828 diff --git a/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 b/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/udf_field-7-3fd8b0c333acdf28c676315b03e2e10 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/udf_field-7-3fd8b0c333acdf28c676315b03e2e10 diff --git a/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef b/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef deleted file mode 100644 index a76563207da24..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef +++ /dev/null @@ -1,2 +0,0 @@ -86 val_86 0 0 2 0 0 0 0 0 2 0 -66 val_66 1 1 0 0 0 1 0 0 2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 b/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef b/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef new file mode 100644 index 0000000000000..9af9d61b8b135 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-8-6aa3518e9f55299754521e959e9376ef @@ -0,0 +1,2 @@ +86 val_86 0 0 2 0 0 0 0 0 2 0 +66 val_66 1 1 0 0 0 1 0 0 2 2 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 b/sql/hive/src/test/resources/golden/udf_field-9-66dc6c81db0ac9b2075783b0d8976083 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 rename to sql/hive/src/test/resources/golden/udf_field-9-66dc6c81db0ac9b2075783b0d8976083 diff --git a/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 b/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 b/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 deleted file mode 100644 index f14679978b79e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 +++ /dev/null @@ -1 +0,0 @@ -find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_find_in_set-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 b/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 new file mode 100644 index 0000000000000..342fb7fad55b0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-1-18d3e88b18c18a00598146a3307276f2 @@ -0,0 +1 @@ +find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 b/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 deleted file mode 100644 index 1bebc68416bff..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 +++ /dev/null @@ -1,7 +0,0 @@ -find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. -Example: - > SELECT find_in_set('ab','abc,b,ab,c,def') FROM src LIMIT 1; - 3 - > SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,956')=0; - 311 val_311 - 128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c b/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 b/sql/hive/src/test/resources/golden/udf_find_in_set-10-df21f44247d7275a292520c1605c4aab similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 rename to sql/hive/src/test/resources/golden/udf_find_in_set-10-df21f44247d7275a292520c1605c4aab diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a b/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e b/sql/hive/src/test/resources/golden/udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e rename to sql/hive/src/test/resources/golden/udf_find_in_set-11-91f8c37820f31d0d1b16029a59a185ad diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 b/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 b/sql/hive/src/test/resources/golden/udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 rename to sql/hive/src/test/resources/golden/udf_find_in_set-12-692f41c998bbc6bec0f374eae4356739 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a b/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a deleted file mode 100644 index e440e5c842586..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a +++ /dev/null @@ -1 +0,0 @@ -3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 b/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-13-45e5ae8f60de2c41f189db7922a04917 @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 b/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 b/sql/hive/src/test/resources/golden/udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 rename to sql/hive/src/test/resources/golden/udf_find_in_set-14-8e410ecfad2d408ad7d2554ccd3a6621 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb b/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 rename to sql/hive/src/test/resources/golden/udf_find_in_set-15-c50e6ff95c05bfa854b33b03db858cd9 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 rename to sql/hive/src/test/resources/golden/udf_find_in_set-16-8e17f41ae6e8b1075af4790a8fd88e13 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 b/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 b/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 deleted file mode 100644 index 99f516951ae7d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 +++ /dev/null @@ -1,2 +0,0 @@ -311 val_311 -128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_find_in_set-17-fe61f992f2d971d006155bdec3143803 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 rename to sql/hive/src/test/resources/golden/udf_find_in_set-17-fe61f992f2d971d006155bdec3143803 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 b/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 new file mode 100644 index 0000000000000..01228944b05a5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-18-5b556a29e1685605bcc47bce60cf66c8 @@ -0,0 +1,2 @@ +311 val_311 +128 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 b/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 new file mode 100644 index 0000000000000..d856144af1f86 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-2-5fb7a7a1725749dc3853eb80fba19246 @@ -0,0 +1,7 @@ +find_in_set(str,str_array) - Returns the first occurrence of str in str_array where str_array is a comma-delimited string. Returns null if either argument is null. Returns 0 if the first argument has any commas. +Example: + > SELECT find_in_set('ab','abc,b,ab,c,def') FROM src LIMIT 1; + 3 + > SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,956')=0; + 311 val_311 + 128 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 b/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e b/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e deleted file mode 100644 index 5817cbcff62ac..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e +++ /dev/null @@ -1,25 +0,0 @@ -1 -NULL -1 -NULL -NULL -NULL -1 -1 -1 -NULL -NULL -NULL -1 -1 -1 -1 -1 -1 -1 -1 -1 -1 -NULL -NULL -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb rename to sql/hive/src/test/resources/golden/udf_find_in_set-3-b3823bdc04a7f98951b55c3e30d2a772 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e b/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e new file mode 100644 index 0000000000000..df725cb2c6aa7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-4-132b7bc7812db7683eb3bff607275d0e @@ -0,0 +1,25 @@ +1 +NULL +1 +NULL +NULL +NULL +1 +1 +1 +NULL +NULL +NULL +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +NULL +NULL +NULL diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b b/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 rename to sql/hive/src/test/resources/golden/udf_find_in_set-5-6f25b5bba89e1fcae171f5d595acc4ee diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f b/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 b/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 deleted file mode 100644 index e440e5c842586..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 +++ /dev/null @@ -1 +0,0 @@ -3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba b/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-6-7bf387b94afb755faca4ad73bb7c42ba @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 b/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 b/sql/hive/src/test/resources/golden/udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826 similarity index 100% rename from sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 rename to sql/hive/src/test/resources/golden/udf_find_in_set-7-730d5e95ef748ad946eceefbcd633826 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 b/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/udf_find_in_set-8-ea11724531f191940e455d13878a0e69 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 rename to sql/hive/src/test/resources/golden/udf_find_in_set-8-ea11724531f191940e455d13878a0e69 diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 b/sql/hive/src/test/resources/golden/udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 rename to sql/hive/src/test/resources/golden/udf_find_in_set-9-81f9999ed1b063ce7f17d89bd0162777 diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 b/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 index 9d15b5f5956d7..467e25bc261c2 100644 --- a/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 +++ b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 @@ -1 +1 @@ -There is no documentation for function 'float' \ No newline at end of file +There is no documentation for function 'float' diff --git a/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b index 9d15b5f5956d7..467e25bc261c2 100644 --- a/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b +++ b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b @@ -1 +1 @@ -There is no documentation for function 'float' \ No newline at end of file +There is no documentation for function 'float' diff --git a/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 b/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 index c76710bfcc1b5..de1563b40b836 100644 --- a/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 +++ b/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 @@ -1 +1 @@ -floor(x) - Find the largest integer not greater than x \ No newline at end of file +floor(x) - Find the largest integer not greater than x diff --git a/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 b/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 index 3f73eea16e183..ab6951202eb2a 100644 --- a/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 +++ b/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 @@ -3,4 +3,4 @@ Example: > SELECT floor(-0.1) FROM src LIMIT 1; -1 > SELECT floor(5) FROM src LIMIT 1; - 5 \ No newline at end of file + 5 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to sql/hive/src/test/resources/golden/udf_format_number-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 b/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 deleted file mode 100644 index c981e1726c070..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 +++ /dev/null @@ -1 +0,0 @@ -format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb rename to sql/hive/src/test/resources/golden/udf_format_number-1-e86d559aeb84a4cc017a103182c22bfb diff --git a/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 b/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 new file mode 100644 index 0000000000000..8ee27a864b6dd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-10-3bddca6913ea7e281e223b0603010b77 @@ -0,0 +1 @@ +-9,223,372,036,854,775,807.0000000000 9,223,372,036,854,775,807.00000000000000000000 0.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005 179,769,313,486,231,570,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 b/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 new file mode 100644 index 0000000000000..14a40602519b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-2-525f133cfff86d44afdeeda667c365a5 @@ -0,0 +1 @@ +format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT diff --git a/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd b/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd deleted file mode 100644 index b2aa527ca0ddb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd +++ /dev/null @@ -1,4 +0,0 @@ -format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT -Example: - > SELECT format_number(12332.123456, 4) FROM src LIMIT 1; - '12,332.1235' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd b/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd new file mode 100644 index 0000000000000..def95a79e7375 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-3-591f302d5c1cd24e153a598fa0b352fd @@ -0,0 +1,4 @@ +format_number(X, D) - Formats the number X to a format like '#,###,###.##', rounded to D decimal places, and returns the result as a string. If D is 0, the result has no decimal point or fractional part. This is supposed to function like MySQL's FORMAT +Example: + > SELECT format_number(12332.123456, 4) FROM src LIMIT 1; + '12,332.1235' diff --git a/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c b/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 b/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 deleted file mode 100644 index 89e118cc62bf3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 +++ /dev/null @@ -1 +0,0 @@ -12,332.1235 12,332.1000 12,332 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb rename to sql/hive/src/test/resources/golden/udf_format_number-4-7969ffc4e80f7214a8eead8e1084368a diff --git a/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 b/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 deleted file mode 100644 index 33e21fa7dbfc4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 +++ /dev/null @@ -1 +0,0 @@ -0.123456789000 12,345,678.12346 1,234,567.1234568 123,456 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc b/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc new file mode 100644 index 0000000000000..055b84b4b35d8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-5-da5bf00d45d2bd758489f661caffd4dc @@ -0,0 +1 @@ +12,332.1235 12,332.1000 12,332 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 b/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 new file mode 100644 index 0000000000000..9f12f9921318d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-6-6dfca21d142652fec9017ba828a226c8 @@ -0,0 +1 @@ +0.123456789000 12,345,678.12346 1,234,567.1234568 123,456 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 b/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 deleted file mode 100644 index 07b05cd16f5a3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 +++ /dev/null @@ -1 +0,0 @@ --123,456 -1,234,567.12 -0.123456789000000 -12,345.1235 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 b/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 new file mode 100644 index 0000000000000..032768d688943 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-7-31eda4b0f31406add3a61e2503590113 @@ -0,0 +1 @@ +-123,456 -1,234,567.12 -0.123456789000000 -12,345.1235 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 b/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 deleted file mode 100644 index 2b600a6a83aa8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 +++ /dev/null @@ -1 +0,0 @@ -0.0000 0.0 0.0 0.0 -0.0000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea b/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea new file mode 100644 index 0000000000000..8077e5a60e4ef --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-8-b297476c6348209933132202030eb8ea @@ -0,0 +1 @@ +0.0000 0.0 0.0 0.0 -0.0000 diff --git a/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e b/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e deleted file mode 100644 index d027b2cb0b94a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e +++ /dev/null @@ -1 +0,0 @@ -0 1.0000 12.00 123.00000 1,234.0000000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 b/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 deleted file mode 100644 index afdec63c8dfca..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 +++ /dev/null @@ -1 +0,0 @@ --9,223,372,036,854,775,807.0000000000 9,223,372,036,854,775,807.00000000000000000000 0.000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000005 179,769,313,486,231,570,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 b/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 new file mode 100644 index 0000000000000..f9f98b94234f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_format_number-9-a21fbe58ff475634c8ed9829c6b8c187 @@ -0,0 +1 @@ +0 1.0000 12.00 123.00000 1,234.0000000 diff --git a/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f b/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f index 10ce106f121ba..38550ea419625 100644 --- a/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f +++ b/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f @@ -1 +1 @@ -from_unixtime(unix_time, format) - returns unix_time in the specified format \ No newline at end of file +from_unixtime(unix_time, format) - returns unix_time in the specified format diff --git a/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 b/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 index 20f4d2b35c49a..ef15f822d80f5 100644 --- a/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 +++ b/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 @@ -1,4 +1,4 @@ from_unixtime(unix_time, format) - returns unix_time in the specified format Example: > SELECT from_unixtime(0, 'yyyy-MM-dd HH:mm:ss') FROM src LIMIT 1; - '1970-01-01 00:00:00' \ No newline at end of file + '1970-01-01 00:00:00' diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f b/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f deleted file mode 100644 index 4e4f3f7255fd3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f +++ /dev/null @@ -1 +0,0 @@ -get_json_object(json_txt, path) - Extract a json object from path diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 b/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 deleted file mode 100644 index 0e1dcf934a023..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 +++ /dev/null @@ -1,16 +0,0 @@ -get_json_object(json_txt, path) - Extract a json object from path -Extract json object from a json string based on json path specified, and return json string of the extracted json object. It will return null if the input json string is invalid. -A limited version of JSONPath supported: - $ : Root object - . : Child operator - [] : Subscript operator for array - * : Wildcard for [] -Syntax not supported that's worth noticing: - '' : Zero length string as key - .. : Recursive descent - &#064; : Current object/element - () : Script expression - ?() : Filter (script) expression. - [,] : Union operator - [start:end:step] : array slice operator - diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 b/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 rename to sql/hive/src/test/resources/golden/udf_greaterthan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 b/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 deleted file mode 100644 index 54d6096d07bbe..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 +++ /dev/null @@ -1 +0,0 @@ -a > b - Returns TRUE if a is greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c b/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c deleted file mode 100644 index 54d6096d07bbe..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c +++ /dev/null @@ -1 +0,0 @@ -a > b - Returns TRUE if a is greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 b/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 new file mode 100644 index 0000000000000..1eec522da2a1e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthan-1-99d268829a124103cb3429c53fdc4de4 @@ -0,0 +1 @@ +a > b - Returns TRUE if a is greater than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f b/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f deleted file mode 100644 index 679b0376125f0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f +++ /dev/null @@ -1 +0,0 @@ -true false false false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c b/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c new file mode 100644 index 0000000000000..1eec522da2a1e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthan-2-8aab8e39726a986e10e1e572939fd63c @@ -0,0 +1 @@ +a > b - Returns TRUE if a is greater than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 b/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 new file mode 100644 index 0000000000000..096c64e2afd93 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthan-3-75fcadcdc6c050f1c7e70c71dc89c800 @@ -0,0 +1 @@ +true false false false diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 rename to sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a deleted file mode 100644 index abf7dfdab730c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a +++ /dev/null @@ -1 +0,0 @@ -a >= b - Returns TRUE if a is not smaller than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 deleted file mode 100644 index abf7dfdab730c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 +++ /dev/null @@ -1 +0,0 @@ -a >= b - Returns TRUE if a is not smaller than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a new file mode 100644 index 0000000000000..d5422146acd0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-a7214027a91abf6501881e2af313347a @@ -0,0 +1 @@ +a >= b - Returns TRUE if a is not smaller than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 new file mode 100644 index 0000000000000..d5422146acd0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-3669f2008e7f428f365efadbcb5ae451 @@ -0,0 +1 @@ +a >= b - Returns TRUE if a is not smaller than b diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 deleted file mode 100644 index 1fb1894fc21f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 +++ /dev/null @@ -1 +0,0 @@ -true false true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 new file mode 100644 index 0000000000000..435a5f7b0efb3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-3-631662997e0c8d24d80b5d64a17446d2 @@ -0,0 +1 @@ +true false true true diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 rename to sql/hive/src/test/resources/golden/udf_hash-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 b/sql/hive/src/test/resources/golden/udf_hash-1-b9e3a3986320d275982797140edfccf4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 rename to sql/hive/src/test/resources/golden/udf_hash-1-b9e3a3986320d275982797140edfccf4 diff --git a/sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a b/sql/hive/src/test/resources/golden/udf_hash-2-a18646b51501d0b1beb967dc79afbd1a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a rename to sql/hive/src/test/resources/golden/udf_hash-2-a18646b51501d0b1beb967dc79afbd1a diff --git a/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 b/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/udf_hash-3-2646a87ce26c383a9dafea9b56281ab7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 rename to sql/hive/src/test/resources/golden/udf_hash-3-2646a87ce26c383a9dafea9b56281ab7 diff --git a/sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae b/sql/hive/src/test/resources/golden/udf_hash-4-d1368c2e3cd113e46202156b44811987 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae rename to sql/hive/src/test/resources/golden/udf_hash-4-d1368c2e3cd113e46202156b44811987 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 rename to sql/hive/src/test/resources/golden/udf_hex-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 b/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 deleted file mode 100644 index f87a6117eacf7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 +++ /dev/null @@ -1 +0,0 @@ -hex(n, bin, or str) - Convert the argument to hexadecimal \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 b/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 new file mode 100644 index 0000000000000..c45cea8bc13b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-1-c8b923c23d5eb31446780f28acbd4e16 @@ -0,0 +1 @@ +hex(n, bin, or str) - Convert the argument to hexadecimal diff --git a/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c b/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c deleted file mode 100644 index e8cee4afc3a80..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c +++ /dev/null @@ -1,8 +0,0 @@ -hex(n, bin, or str) - Convert the argument to hexadecimal -If the argument is a string, returns two hex digits for each character in the string. -If the argument is a number or binary, returns the hexadecimal representation. -Example: - > SELECT hex(17) FROM src LIMIT 1; - 'H1' - > SELECT hex('Facebook') FROM src LIMIT 1; - '46616365626F6F6B' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 b/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 deleted file mode 100644 index 34eb75a6c784a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 +++ /dev/null @@ -1 +0,0 @@ -46616365626F6F6B 00 71776572747975696F706173646667686A6B6C \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c b/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c new file mode 100644 index 0000000000000..bcc46336d0fd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-2-d55348c0ccd133b7abb690f6949b520c @@ -0,0 +1,8 @@ +hex(n, bin, or str) - Convert the argument to hexadecimal +If the argument is a string, returns two hex digits for each character in the string. +If the argument is a number or binary, returns the hexadecimal representation. +Example: + > SELECT hex(17) FROM src LIMIT 1; + 'H1' + > SELECT hex('Facebook') FROM src LIMIT 1; + '46616365626F6F6B' diff --git a/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 b/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 new file mode 100644 index 0000000000000..b0ffe57c8e161 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-3-3a1de5be8ce350612ee6a93303107470 @@ -0,0 +1 @@ +46616365626F6F6B 00 71776572747975696F706173646667686A6B6C diff --git a/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf b/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf deleted file mode 100644 index 2a45250c67a63..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf +++ /dev/null @@ -1 +0,0 @@ -1 0 FACEB005 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 b/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 new file mode 100644 index 0000000000000..8c56faa0f9db9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-4-a7f99c9ad67c837658b924c0a979cf01 @@ -0,0 +1 @@ +1 0 FACEB005 diff --git a/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea b/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea deleted file mode 100644 index 50d9557967ac4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea +++ /dev/null @@ -1 +0,0 @@ -FFFFFFFFFFFFFFFB \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e b/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e new file mode 100644 index 0000000000000..b766160c67704 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-5-1a9b53bd38a3693e66f7c03092e15c8e @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFB diff --git a/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce b/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce deleted file mode 100644 index 48911456dc339..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce +++ /dev/null @@ -1 +0,0 @@ -hour(date) - Returns the hour of date diff --git a/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 b/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 deleted file mode 100644 index be1a966635202..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 +++ /dev/null @@ -1,7 +0,0 @@ -hour(date) - Returns the hour of date -date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. -Example: - > SELECT hour('2009-07-30 12:58:59') FROM src LIMIT 1; - 12 - > SELECT hour('12:58:59') FROM src LIMIT 1; - 12 diff --git a/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a b/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 b/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 deleted file mode 100644 index 1a35f9f158133..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 +++ /dev/null @@ -1 +0,0 @@ -13 13 NULL diff --git a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a b/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a deleted file mode 100644 index b1bd38b62a080..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a +++ /dev/null @@ -1 +0,0 @@ -13 diff --git a/sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/udf_if-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a rename to sql/hive/src/test/resources/golden/udf_if-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c rename to sql/hive/src/test/resources/golden/udf_if-1-b7ffa85b5785cccef2af1b285348cc2c diff --git a/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a rename to sql/hive/src/test/resources/golden/udf_if-2-30cf7f51f92b5684e556deff3032d49a diff --git a/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 b/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/udf_if-3-59e90bb74481aaf35480076806daf365 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 rename to sql/hive/src/test/resources/golden/udf_if-3-59e90bb74481aaf35480076806daf365 diff --git a/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca b/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f b/sql/hive/src/test/resources/golden/udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f rename to sql/hive/src/test/resources/golden/udf_if-4-c4f4d2c83281f9c2380b5efac55fe6eb diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/udf_if-5-841a8068d35a42179d3654e1a2848c43 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c rename to sql/hive/src/test/resources/golden/udf_if-5-841a8068d35a42179d3654e1a2848c43 diff --git a/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc b/sql/hive/src/test/resources/golden/udf_if-6-508f9140dd33931c7b9ad336dceb32cf similarity index 100% rename from sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc rename to sql/hive/src/test/resources/golden/udf_if-6-508f9140dd33931c7b9ad336dceb32cf diff --git a/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 b/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 deleted file mode 100644 index ee6e42ce6a83d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 +++ /dev/null @@ -1 +0,0 @@ -true false true true false true true true NULL NULL true diff --git a/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 b/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 deleted file mode 100644 index 993d93304f95e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 +++ /dev/null @@ -1,3 +0,0 @@ -238 -86 -238 diff --git a/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 b/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 deleted file mode 100644 index cd15a08a539ab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 +++ /dev/null @@ -1 +0,0 @@ -in_file(str, filename) - Returns true if str appears in the file diff --git a/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 b/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 b/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 deleted file mode 100644 index df5a0561fb8f5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 +++ /dev/null @@ -1 +0,0 @@ -Function '`index`' does not exist. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_index-0-e91e3e5a22029b9b979ccbbef97add66 similarity index 100% rename from sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_index-0-e91e3e5a22029b9b979ccbbef97add66 diff --git a/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 b/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 deleted file mode 100644 index df5a0561fb8f5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 +++ /dev/null @@ -1 +0,0 @@ -Function '`index`' does not exist. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 b/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 new file mode 100644 index 0000000000000..bb0a912824bbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_index-1-a277ac394cae40cb55d1ef3aa5add260 @@ -0,0 +1 @@ +Function '`index`' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 b/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 new file mode 100644 index 0000000000000..bb0a912824bbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_index-2-1f5e109131b0c67ebea521fa4902a8f6 @@ -0,0 +1 @@ +Function '`index`' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 b/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 deleted file mode 100644 index ae27b5efea045..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 +++ /dev/null @@ -1 +0,0 @@ -instr(str, substr) - Returns the index of the first occurance of substr in str \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 rename to sql/hive/src/test/resources/golden/udf_instr-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 b/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 new file mode 100644 index 0000000000000..06461b525b058 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922 @@ -0,0 +1 @@ +instr(str, substr) - Returns the index of the first occurance of substr in str diff --git a/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 b/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 deleted file mode 100644 index 35de2f0d8653f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 +++ /dev/null @@ -1,4 +0,0 @@ -instr(str, substr) - Returns the index of the first occurance of substr in str -Example: - > SELECT instr('Facebook', 'boo') FROM src LIMIT 1; - 5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 b/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 b/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 new file mode 100644 index 0000000000000..5a8c34271f443 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182 @@ -0,0 +1,4 @@ +instr(str, substr) - Returns the index of the first occurance of substr in str +Example: + > SELECT instr('Facebook', 'boo') FROM src LIMIT 1; + 5 diff --git a/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 b/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 deleted file mode 100644 index 4d34e6df7039b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 +++ /dev/null @@ -1 +0,0 @@ -1 0 2 2 0 0 2 3 4 2 3 NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 rename to sql/hive/src/test/resources/golden/udf_instr-3-c40fbd09410b11388ce7a6e9bea5846f diff --git a/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 b/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 new file mode 100644 index 0000000000000..8883b1a631ab7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_instr-4-7017a441a31abc235d9359440cefda49 @@ -0,0 +1 @@ +1 0 2 2 0 0 2 3 4 2 3 NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba index cda33a8e2e555..5549ad557f6b2 100644 --- a/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba +++ b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba @@ -1 +1 @@ -There is no documentation for function 'int' \ No newline at end of file +There is no documentation for function 'int' diff --git a/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 index cda33a8e2e555..5549ad557f6b2 100644 --- a/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 +++ b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 @@ -1 +1 @@ -There is no documentation for function 'int' \ No newline at end of file +There is no documentation for function 'int' diff --git a/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 index 723b5aaf2af6b..4ae9ea2a16e87 100644 --- a/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 +++ b/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 @@ -1 +1 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file +isnotnull a - Returns true if a is not NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 index 723b5aaf2af6b..4ae9ea2a16e87 100644 --- a/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 +++ b/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 @@ -1 +1 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file +isnotnull a - Returns true if a is not NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 index 99510c6f1db15..237cdf99168d0 100644 --- a/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 +++ b/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 @@ -1 +1 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file +isnull a - Returns true if a is NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 index 99510c6f1db15..237cdf99168d0 100644 --- a/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 +++ b/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 @@ -1 +1 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file +isnull a - Returns true if a is NULL and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 deleted file mode 100644 index 99510c6f1db15..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 +++ /dev/null @@ -1 +0,0 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 deleted file mode 100644 index 99510c6f1db15..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 +++ /dev/null @@ -1 +0,0 @@ -isnull a - Returns true if a is NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 deleted file mode 100644 index 723b5aaf2af6b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 +++ /dev/null @@ -1 +0,0 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 deleted file mode 100644 index 723b5aaf2af6b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 +++ /dev/null @@ -1 +0,0 @@ -isnotnull a - Returns true if a is not NULL and false otherwise \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 deleted file mode 100644 index eedfbc67d50c7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 +++ /dev/null @@ -1 +0,0 @@ -true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b deleted file mode 100644 index eedfbc67d50c7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b +++ /dev/null @@ -1 +0,0 @@ -true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 deleted file mode 100644 index b703a30abffba..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 +++ /dev/null @@ -1 +0,0 @@ -java_method(class,method[,arg1[,arg2..]]) calls method with reflection \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a deleted file mode 100644 index 07375dc92d625..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a +++ /dev/null @@ -1,3 +0,0 @@ -java_method(class,method[,arg1[,arg2..]]) calls method with reflection -Synonyms: reflect -Use this UDF to call Java methods by matching the argument signature diff --git a/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd deleted file mode 100644 index 9b93703dae806..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd +++ /dev/null @@ -1 +0,0 @@ -1 true 3 2 3 2.718281828459045 1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad b/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad index ec6779df2818d..afe1bf6cd351a 100644 --- a/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad +++ b/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad @@ -1 +1 @@ -lcase(str) - Returns str with all characters changed to lowercase \ No newline at end of file +lcase(str) - Returns str with all characters changed to lowercase diff --git a/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e b/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e index 4da30f6c23a1b..191001deca9f7 100644 --- a/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e +++ b/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e @@ -2,4 +2,4 @@ lcase(str) - Returns str with all characters changed to lowercase Synonyms: lower Example: > SELECT lcase('Facebook') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b b/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b deleted file mode 100644 index d4d2bd508b44f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b +++ /dev/null @@ -1 +0,0 @@ -length(str | binary) - Returns the length of str or number of bytes in binary data \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/udf_length-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee rename to sql/hive/src/test/resources/golden/udf_length-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b b/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b new file mode 100644 index 0000000000000..3e2bae7430e67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-1-38364b60c3a2409f53c9aa2dae19903b @@ -0,0 +1 @@ +length(str | binary) - Returns the length of str or number of bytes in binary data diff --git a/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 b/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 deleted file mode 100644 index 79c1c54639266..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 +++ /dev/null @@ -1,4 +0,0 @@ -length(str | binary) - Returns the length of str or number of bytes in binary data -Example: - > SELECT length('Facebook') FROM src LIMIT 1; - 8 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/udf_length-10-de456a5765db4a06110d9483985aa4a6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea rename to sql/hive/src/test/resources/golden/udf_length-10-de456a5765db4a06110d9483985aa4a6 diff --git a/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 b/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 deleted file mode 100644 index d8263ee986059..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 +++ /dev/null @@ -1 +0,0 @@ -2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 b/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 new file mode 100644 index 0000000000000..0cfbf08886fca --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-11-f3a9bd30540345db0f69b6847014b333 @@ -0,0 +1 @@ +2 diff --git a/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 b/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 b/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 new file mode 100644 index 0000000000000..f1d3b1648718d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-2-f183e1f8ae516bb483132ed106289b67 @@ -0,0 +1,4 @@ +length(str | binary) - Returns the length of str or number of bytes in binary data +Example: + > SELECT length('Facebook') FROM src LIMIT 1; + 8 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/udf_length-3-af46cb6887618240836eaf5be8afbba6 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 rename to sql/hive/src/test/resources/golden/udf_length-3-af46cb6887618240836eaf5be8afbba6 diff --git a/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d b/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 b/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/udf_length-4-dcd6404afce1103d5054527e6c216d6d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 rename to sql/hive/src/test/resources/golden/udf_length-4-dcd6404afce1103d5054527e6c216d6d diff --git a/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f deleted file mode 100644 index 6ecb2baabb297..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f +++ /dev/null @@ -1,25 +0,0 @@ -7 -0 -7 -6 -7 -7 -7 -7 -6 -7 -7 -7 -7 -7 -7 -0 -0 -6 -0 -7 -7 -7 -0 -0 -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/udf_length-5-ba49ba4e6505c74bc33b5003f3930c43 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 rename to sql/hive/src/test/resources/golden/udf_length-5-ba49ba4e6505c74bc33b5003f3930c43 diff --git a/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..c77515b2f479b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1,25 @@ +7 +0 +7 +6 +7 +7 +7 +7 +6 +7 +7 +7 +7 +7 +7 +0 +0 +6 +0 +7 +7 +7 +0 +0 +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 rename to sql/hive/src/test/resources/golden/udf_length-7-460dffb0f1ab0ac0ebc4fd545809aa9a diff --git a/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f b/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf b/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/udf_length-8-8f28e6c488df47d89dca670f04a7563f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 rename to sql/hive/src/test/resources/golden/udf_length-8-8f28e6c488df47d89dca670f04a7563f diff --git a/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 b/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/udf_length-9-e41b220da98996f997b26ba7ef457a84 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 rename to sql/hive/src/test/resources/golden/udf_length-9-e41b220da98996f997b26ba7ef457a84 diff --git a/sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 rename to sql/hive/src/test/resources/golden/udf_lessthan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 b/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 deleted file mode 100644 index b43707d550c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 +++ /dev/null @@ -1 +0,0 @@ -a < b - Returns TRUE if a is less than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 b/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 deleted file mode 100644 index b43707d550c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 +++ /dev/null @@ -1 +0,0 @@ -a < b - Returns TRUE if a is less than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 b/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 new file mode 100644 index 0000000000000..d7621677e2fce --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthan-1-a0d9e8b51e3d13685b3889db38f22427 @@ -0,0 +1 @@ +a < b - Returns TRUE if a is less than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 b/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 deleted file mode 100644 index 4364012093724..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 +++ /dev/null @@ -1 +0,0 @@ -false true false false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 b/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 new file mode 100644 index 0000000000000..d7621677e2fce --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthan-2-952c655a1092a410e5346f1205cb8142 @@ -0,0 +1 @@ +a < b - Returns TRUE if a is less than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 b/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 new file mode 100644 index 0000000000000..5926a6ac7746d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthan-3-677a1383983c94ba8008535b5a193153 @@ -0,0 +1 @@ +false true false false diff --git a/sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 rename to sql/hive/src/test/resources/golden/udf_lessthanorequal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b b/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b deleted file mode 100644 index c36acd7ce80bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b +++ /dev/null @@ -1 +0,0 @@ -a <= b - Returns TRUE if a is not greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 deleted file mode 100644 index c36acd7ce80bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 +++ /dev/null @@ -1 +0,0 @@ -a <= b - Returns TRUE if a is not greater than b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b new file mode 100644 index 0000000000000..7f18733b6e47e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-b3b021456c85da76d1879191886a425b @@ -0,0 +1 @@ +a <= b - Returns TRUE if a is not greater than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c deleted file mode 100644 index 38e013b6914c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c +++ /dev/null @@ -1 +0,0 @@ -false true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 new file mode 100644 index 0000000000000..7f18733b6e47e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-869d983466744ad73b109211e1638869 @@ -0,0 +1 @@ +a <= b - Returns TRUE if a is not greater than b diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 new file mode 100644 index 0000000000000..fc2c7b1da1920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-3-947dd56091ae1ef399ab32ce58317667 @@ -0,0 +1 @@ +false true true true diff --git a/sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/udf_like-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 rename to sql/hive/src/test/resources/golden/udf_like-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 b/sql/hive/src/test/resources/golden/udf_like-1-e0ba9a953e50554bdcbc55585cffde09 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 rename to sql/hive/src/test/resources/golden/udf_like-1-e0ba9a953e50554bdcbc55585cffde09 diff --git a/sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a b/sql/hive/src/test/resources/golden/udf_like-2-9781f89d352c506e972ad2a1d58ec03a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a rename to sql/hive/src/test/resources/golden/udf_like-2-9781f89d352c506e972ad2a1d58ec03a diff --git a/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 b/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/udf_like-3-dbc46cb33f0dd356af03006d9492f8b7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 rename to sql/hive/src/test/resources/golden/udf_like-3-dbc46cb33f0dd356af03006d9492f8b7 diff --git a/sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc b/sql/hive/src/test/resources/golden/udf_like-4-bef03784eab9d5e8404fd24960dea4fc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc rename to sql/hive/src/test/resources/golden/udf_like-4-bef03784eab9d5e8404fd24960dea4fc diff --git a/sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 b/sql/hive/src/test/resources/golden/udf_like-5-47bfd4d65090dab890b467ae06cf3bd5 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 rename to sql/hive/src/test/resources/golden/udf_like-5-47bfd4d65090dab890b467ae06cf3bd5 diff --git a/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 b/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 index 0b7372cb446d9..134096b19d8c3 100644 --- a/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 +++ b/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 @@ -1 +1 @@ -ln(x) - Returns the natural logarithm of x \ No newline at end of file +ln(x) - Returns the natural logarithm of x diff --git a/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 b/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 index bdf2f49ab17ca..c4a1a46f93370 100644 --- a/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 +++ b/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 @@ -1,4 +1,4 @@ ln(x) - Returns the natural logarithm of x Example: > SELECT ln(1) FROM src LIMIT 1; - 0 \ No newline at end of file + 0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_locate-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e b/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e deleted file mode 100644 index 63b152162407b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e +++ /dev/null @@ -1 +0,0 @@ -locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e b/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e new file mode 100644 index 0000000000000..84bea329540d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e @@ -0,0 +1 @@ +locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos diff --git a/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 b/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 deleted file mode 100644 index ea5465b67683e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 +++ /dev/null @@ -1,4 +0,0 @@ -locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos -Example: - > SELECT locate('bar', 'foobarbar', 5) FROM src LIMIT 1; - 7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c b/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 b/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 new file mode 100644 index 0000000000000..092e12586b9e8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478 @@ -0,0 +1,4 @@ +locate(substr, str[, pos]) - Returns the position of the first occurance of substr in str after position pos +Example: + > SELECT locate('bar', 'foobarbar', 5) FROM src LIMIT 1; + 7 diff --git a/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 b/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 deleted file mode 100644 index cd97bbb17fa81..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 +++ /dev/null @@ -1 +0,0 @@ -1 0 2 2 4 4 0 0 2 3 4 2 3 NULL NULL 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/udf_locate-3-ce4a131f99dc9befa926027380b38dbb similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 rename to sql/hive/src/test/resources/golden/udf_locate-3-ce4a131f99dc9befa926027380b38dbb diff --git a/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b b/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b new file mode 100644 index 0000000000000..f98ccc3556b42 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_locate-4-104cbfb3b59ad563810ddd7304a58b1b @@ -0,0 +1 @@ +1 0 2 2 4 4 0 0 2 3 4 2 3 NULL NULL 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a b/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a index e0dba6b06d9a4..c2c4ca2cd4385 100644 --- a/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a +++ b/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a @@ -1 +1 @@ -log([b], x) - Returns the logarithm of x with base b \ No newline at end of file +log([b], x) - Returns the logarithm of x with base b diff --git a/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 b/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 index 3dedfb4ddfd88..d6bc6ebbfd2d1 100644 --- a/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 +++ b/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 @@ -1,4 +1,4 @@ log([b], x) - Returns the logarithm of x with base b Example: > SELECT log(13, 13) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f b/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f index 61749c48f158e..b9540eb85a360 100644 --- a/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f +++ b/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f @@ -1 +1 @@ -log10(x) - Returns the logarithm of x with base 10 \ No newline at end of file +log10(x) - Returns the logarithm of x with base 10 diff --git a/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 b/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 index 68ce39dd2b084..b71baf542f924 100644 --- a/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 +++ b/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 @@ -1,4 +1,4 @@ log10(x) - Returns the logarithm of x with base 10 Example: > SELECT log10(10) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 b/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 index 177008a543627..8214317d87b0b 100644 --- a/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 +++ b/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 @@ -1 +1 @@ -log2(x) - Returns the logarithm of x with base 2 \ No newline at end of file +log2(x) - Returns the logarithm of x with base 2 diff --git a/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 b/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 index 4b856f91a7a14..579ad959cd884 100644 --- a/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 +++ b/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 @@ -1,4 +1,4 @@ log2(x) - Returns the logarithm of x with base 2 Example: > SELECT log2(2) FROM src LIMIT 1; - 1 \ No newline at end of file + 1 diff --git a/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 b/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 deleted file mode 100644 index d9d0dc035c9ab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION test_udf_get_java_boolean 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaBoolean') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 b/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 index 9bfc44922ea38..de8e1518c93d7 100644 --- a/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 +++ b/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 @@ -1 +1 @@ -lower(str) - Returns str with all characters changed to lowercase \ No newline at end of file +lower(str) - Returns str with all characters changed to lowercase diff --git a/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c b/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c index da912776ab009..532f348d7da6f 100644 --- a/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c +++ b/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c @@ -2,4 +2,4 @@ lower(str) - Returns str with all characters changed to lowercase Synonyms: lcase Example: > SELECT lower('Facebook') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 index 9640d1794dcbc..8fbb5d4429d96 100644 --- a/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 +++ b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 @@ -1 +1 @@ -abc 123 ABC 123 \ No newline at end of file +abc 123 ABC 123 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_lpad-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef b/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef deleted file mode 100644 index 672d2d07f68b5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef +++ /dev/null @@ -1 +0,0 @@ -lpad(str, len, pad) - Returns str, left-padded with pad to a length of len \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef b/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef new file mode 100644 index 0000000000000..e756eab5119d2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lpad-1-995646acf1e23cea7825412915921bef @@ -0,0 +1 @@ +lpad(str, len, pad) - Returns str, left-padded with pad to a length of len diff --git a/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 b/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 deleted file mode 100644 index 66db6624a376a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 +++ /dev/null @@ -1,6 +0,0 @@ -lpad(str, len, pad) - Returns str, left-padded with pad to a length of len -If str is longer than len, the return value is shortened to len characters. -Example: - > SELECT lpad('hi', 5, '??') FROM src LIMIT 1; - '???hi' > SELECT lpad('hi', 1, '??') FROM src LIMIT 1; - 'h' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 b/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 b/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 new file mode 100644 index 0000000000000..122971dbc2df9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lpad-2-f58bb0fd11cb70cf197c01555ac924a8 @@ -0,0 +1,6 @@ +lpad(str, len, pad) - Returns str, left-padded with pad to a length of len +If str is longer than len, the return value is shortened to len characters. +Example: + > SELECT lpad('hi', 5, '??') FROM src LIMIT 1; + '???hi' > SELECT lpad('hi', 1, '??') FROM src LIMIT 1; + 'h' diff --git a/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae b/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae deleted file mode 100644 index 63a7235050df3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae +++ /dev/null @@ -1 +0,0 @@ -h ...hi 1231hi \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f rename to sql/hive/src/test/resources/golden/udf_lpad-3-ea9a05f035dedfe15d3a7f3d7756a2d7 diff --git a/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 b/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 new file mode 100644 index 0000000000000..f75d0d891636a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lpad-4-48234ef55a8ec06cd4b570b9b9edab73 @@ -0,0 +1 @@ +h ...hi 1231hi diff --git a/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f b/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f index 0c9f6fd1c269e..419d80fe502a3 100644 --- a/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f +++ b/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f @@ -1 +1 @@ -ltrim(str) - Removes the leading space characters from str \ No newline at end of file +ltrim(str) - Removes the leading space characters from str diff --git a/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 b/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 index 3ee2e690d28c1..fc994e90aaa05 100644 --- a/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 +++ b/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 @@ -1,4 +1,4 @@ ltrim(str) - Removes the leading space characters from str Example: > SELECT ltrim(' facebook') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_map-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_map-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e b/sql/hive/src/test/resources/golden/udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e similarity index 100% rename from sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e rename to sql/hive/src/test/resources/golden/udf_map-1-9feb9f29115f94b3bb4f6a36160bd17e diff --git a/sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf b/sql/hive/src/test/resources/golden/udf_map-2-1f8cd98df9bf7b2528506551fef87dcf similarity index 100% rename from sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf rename to sql/hive/src/test/resources/golden/udf_map-2-1f8cd98df9bf7b2528506551fef87dcf diff --git a/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 b/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/udf_map-3-be7b52baa973b8b59b7ca63fea19aa99 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 rename to sql/hive/src/test/resources/golden/udf_map-3-be7b52baa973b8b59b7ca63fea19aa99 diff --git a/sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 b/sql/hive/src/test/resources/golden/udf_map-4-60cb9c30285f7a9f99377ccbd143eb06 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 rename to sql/hive/src/test/resources/golden/udf_map-4-60cb9c30285f7a9f99377ccbd143eb06 diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa b/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa deleted file mode 100644 index b88b74b978b8f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa +++ /dev/null @@ -1 +0,0 @@ -map_keys(map) - Returns an unordered array containing the keys of the input map. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f b/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f deleted file mode 100644 index b88b74b978b8f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f +++ /dev/null @@ -1 +0,0 @@ -map_keys(map) - Returns an unordered array containing the keys of the input map. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e b/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e deleted file mode 100644 index 3a26a2e5e94d5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e +++ /dev/null @@ -1 +0,0 @@ -[1,2,3] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a b/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a deleted file mode 100644 index 9b4c194f58a8e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a +++ /dev/null @@ -1 +0,0 @@ -["b","a","c"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 b/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 deleted file mode 100644 index e811d1976a24c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 +++ /dev/null @@ -1 +0,0 @@ -map_values(map) - Returns an unordered array containing the values of the input map. diff --git a/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 b/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 deleted file mode 100644 index e811d1976a24c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 +++ /dev/null @@ -1 +0,0 @@ -map_values(map) - Returns an unordered array containing the values of the input map. diff --git a/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 b/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 deleted file mode 100644 index 9d804e48987c6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 +++ /dev/null @@ -1 +0,0 @@ -["a","b","c"] diff --git a/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e b/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e deleted file mode 100644 index be56bd1673eaa..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e +++ /dev/null @@ -1 +0,0 @@ -[2,1,3] diff --git a/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f b/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f deleted file mode 100644 index c6cd9573ed8ed..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f +++ /dev/null @@ -1 +0,0 @@ -max(expr) - Returns the maximum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 b/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 deleted file mode 100644 index c6cd9573ed8ed..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 +++ /dev/null @@ -1 +0,0 @@ -max(expr) - Returns the maximum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 deleted file mode 100644 index 21096f0f69749..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 +++ /dev/null @@ -1 +0,0 @@ -{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 b/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 deleted file mode 100644 index 0d59cabcde8fb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 +++ /dev/null @@ -1 +0,0 @@ -min(expr) - Returns the minimum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 b/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 deleted file mode 100644 index 0d59cabcde8fb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 +++ /dev/null @@ -1 +0,0 @@ -min(expr) - Returns the minimum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 deleted file mode 100644 index b26a7e93c5d9f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 +++ /dev/null @@ -1 +0,0 @@ -{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee index b6bd6049a90b9..231e4f382566d 100644 --- a/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee +++ b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee @@ -1 +1 @@ -minute(date) - Returns the minute of date \ No newline at end of file +minute(date) - Returns the minute of date diff --git a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 index 64c3cda697e1e..ea842ea174ae4 100644 --- a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 +++ b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 @@ -4,4 +4,4 @@ Example: > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1; 58 > SELECT minute('12:58:59') FROM src LIMIT 1; - 58 \ No newline at end of file + 58 diff --git a/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c index 5bd5433375d5c..e9ca4578409c8 100644 --- a/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c +++ b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c @@ -1 +1 @@ -14 14 NULL \ No newline at end of file +14 14 NULL diff --git a/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 b/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 index 1bcf0c71008b0..6df0bfb4fecf8 100644 --- a/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 +++ b/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 @@ -1 +1 @@ -a % b - Returns the remainder when dividing a by b \ No newline at end of file +a % b - Returns the remainder when dividing a by b diff --git a/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 b/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 index 1bcf0c71008b0..6df0bfb4fecf8 100644 --- a/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 +++ b/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 @@ -1 +1 @@ -a % b - Returns the remainder when dividing a by b \ No newline at end of file +a % b - Returns the remainder when dividing a by b diff --git a/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee index b6bd6049a90b9..231e4f382566d 100644 --- a/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee +++ b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee @@ -1 +1 @@ -minute(date) - Returns the minute of date \ No newline at end of file +minute(date) - Returns the minute of date diff --git a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 index 64c3cda697e1e..ea842ea174ae4 100644 --- a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 +++ b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 @@ -4,4 +4,4 @@ Example: > SELECT minute('2009-07-30 12:58:59') FROM src LIMIT 1; 58 > SELECT minute('12:58:59') FROM src LIMIT 1; - 58 \ No newline at end of file + 58 diff --git a/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 b/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 deleted file mode 100644 index 3142f7f94be9e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 +++ /dev/null @@ -1 +0,0 @@ -negative a - Returns -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_negative-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 b/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 new file mode 100644 index 0000000000000..7ba1b77782f07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-1-1b770ec6fb07bb771af2231a9723ec8 @@ -0,0 +1 @@ +negative a - Returns -a diff --git a/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf b/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf deleted file mode 100644 index 3142f7f94be9e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf +++ /dev/null @@ -1 +0,0 @@ -negative a - Returns -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 b/sql/hive/src/test/resources/golden/udf_negative-10-1cd28efecc0d26f463221195f5e39956 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 rename to sql/hive/src/test/resources/golden/udf_negative-10-1cd28efecc0d26f463221195f5e39956 diff --git a/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf b/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf new file mode 100644 index 0000000000000..7ba1b77782f07 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-2-5f64266721b1ed31cfe84ee2f2377bdf @@ -0,0 +1 @@ +negative a - Returns -a diff --git a/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c b/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c deleted file mode 100644 index 0f4a4dce9dd3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c +++ /dev/null @@ -1 +0,0 @@ -a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c b/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c new file mode 100644 index 0000000000000..1ad8be1242f00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-3-a6863d2c5fc8c3131fe70080a011392c @@ -0,0 +1 @@ +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 deleted file mode 100644 index 0f4a4dce9dd3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 +++ /dev/null @@ -1 +0,0 @@ -a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 new file mode 100644 index 0000000000000..1ad8be1242f00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-4-b90eec030fee9cbd177f9615b782d722 @@ -0,0 +1 @@ +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c b/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a b/sql/hive/src/test/resources/golden/udf_negative-5-771e76b0acd8ddb128781da7819d0e47 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a rename to sql/hive/src/test/resources/golden/udf_negative-5-771e76b0acd8ddb128781da7819d0e47 diff --git a/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e b/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 b/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d b/sql/hive/src/test/resources/golden/udf_negative-6-f62c4a097c592871d896a7dc47c42f61 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d rename to sql/hive/src/test/resources/golden/udf_negative-6-f62c4a097c592871d896a7dc47c42f61 diff --git a/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b b/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 b/sql/hive/src/test/resources/golden/udf_negative-7-f838053f5ca5c8746dc299473dff0490 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 rename to sql/hive/src/test/resources/golden/udf_negative-7-f838053f5ca5c8746dc299473dff0490 diff --git a/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 b/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da b/sql/hive/src/test/resources/golden/udf_negative-8-f4f23aa6f634913d194a69261af8f3f6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da rename to sql/hive/src/test/resources/golden/udf_negative-8-f4f23aa6f634913d194a69261af8f3f6 diff --git a/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa b/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 b/sql/hive/src/test/resources/golden/udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 rename to sql/hive/src/test/resources/golden/udf_negative-9-f6a78fa3ea0f519d0e4abc5be7a960e5 diff --git a/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc index 1ad92784d47ac..85b491240e711 100644 --- a/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc +++ b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc @@ -1 +1 @@ -not a - Logical not \ No newline at end of file +not a - Logical not diff --git a/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 index 6520d74f02566..0956574686405 100644 --- a/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 +++ b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 @@ -1,2 +1,2 @@ not a - Logical not -Synonyms: ! \ No newline at end of file +Synonyms: ! diff --git a/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 index c0e84e95d01c2..e6b0fa3967745 100644 --- a/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 +++ b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 @@ -1 +1 @@ -! a - Logical not \ No newline at end of file +! a - Logical not diff --git a/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a index 05cb1f6ab03d7..00f749069b4a3 100644 --- a/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a +++ b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a @@ -1,2 +1,2 @@ ! a - Logical not -Synonyms: not \ No newline at end of file +Synonyms: not diff --git a/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e b/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e deleted file mode 100644 index 9e2108ff72b18..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e +++ /dev/null @@ -1 +0,0 @@ -a <> b - Returns TRUE if a is not equal to b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e rename to sql/hive/src/test/resources/golden/udf_notequal-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e b/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e new file mode 100644 index 0000000000000..5bcfd313a8ff6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-1-27c0d57f7c7c48ceb7bc671f7160254e @@ -0,0 +1 @@ +a <> b - Returns TRUE if a is not equal to b diff --git a/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a b/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a deleted file mode 100644 index 162f808ed5b67..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a +++ /dev/null @@ -1,2 +0,0 @@ -a <> b - Returns TRUE if a is not equal to b -Synonyms: != \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 b/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 deleted file mode 100644 index 2dab01a3b2765..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 +++ /dev/null @@ -1 +0,0 @@ -a != b - Returns TRUE if a is not equal to b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a b/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a new file mode 100644 index 0000000000000..d8be357a858c4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-2-a7f0d1064f8f398ef504008015fddf9a @@ -0,0 +1,2 @@ +a <> b - Returns TRUE if a is not equal to b +Synonyms: != diff --git a/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 b/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 new file mode 100644 index 0000000000000..aeffc394679cf --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-3-7d815b1218c85e4cf69d6780cab17520 @@ -0,0 +1 @@ +a != b - Returns TRUE if a is not equal to b diff --git a/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 b/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 deleted file mode 100644 index 1f73486ba2896..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 +++ /dev/null @@ -1,2 +0,0 @@ -a != b - Returns TRUE if a is not equal to b -Synonyms: <> \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 b/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 new file mode 100644 index 0000000000000..32e73741d4cbc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-4-b72baeb22fad61bb31ce2d2e69375f57 @@ -0,0 +1,2 @@ +a != b - Returns TRUE if a is not equal to b +Synonyms: <> diff --git a/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 b/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 b/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 deleted file mode 100644 index f82286d8bb069..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 +++ /dev/null @@ -1,499 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/udf_notequal-5-eb04e5ee00967799c913e8a5b424a332 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 rename to sql/hive/src/test/resources/golden/udf_notequal-5-eb04e5ee00967799c913e8a5b424a332 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 b/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 b/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 new file mode 100644 index 0000000000000..36244b327dc2f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-6-e361b9cf294c4aa25018b081a2c05e07 @@ -0,0 +1,499 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/udf_notequal-7-46a6514f2d7e6a097035ec1559df0096 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f rename to sql/hive/src/test/resources/golden/udf_notequal-7-46a6514f2d7e6a097035ec1559df0096 diff --git a/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd b/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd deleted file mode 100644 index f82286d8bb069..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd +++ /dev/null @@ -1,499 +0,0 @@ -238 val_238 -86 val_86 -311 val_311 -27 val_27 -165 val_165 -409 val_409 -255 val_255 -278 val_278 -98 val_98 -484 val_484 -265 val_265 -193 val_193 -401 val_401 -150 val_150 -273 val_273 -224 val_224 -369 val_369 -66 val_66 -128 val_128 -213 val_213 -146 val_146 -406 val_406 -429 val_429 -374 val_374 -152 val_152 -469 val_469 -145 val_145 -495 val_495 -37 val_37 -327 val_327 -281 val_281 -277 val_277 -209 val_209 -15 val_15 -82 val_82 -403 val_403 -166 val_166 -417 val_417 -430 val_430 -252 val_252 -292 val_292 -219 val_219 -287 val_287 -153 val_153 -193 val_193 -338 val_338 -446 val_446 -459 val_459 -394 val_394 -237 val_237 -482 val_482 -174 val_174 -413 val_413 -494 val_494 -207 val_207 -199 val_199 -466 val_466 -208 val_208 -174 val_174 -399 val_399 -396 val_396 -247 val_247 -417 val_417 -489 val_489 -162 val_162 -377 val_377 -397 val_397 -309 val_309 -365 val_365 -266 val_266 -439 val_439 -342 val_342 -367 val_367 -325 val_325 -167 val_167 -195 val_195 -475 val_475 -17 val_17 -113 val_113 -155 val_155 -203 val_203 -339 val_339 -0 val_0 -455 val_455 -128 val_128 -311 val_311 -316 val_316 -57 val_57 -205 val_205 -149 val_149 -438 val_438 -345 val_345 -129 val_129 -170 val_170 -20 val_20 -489 val_489 -157 val_157 -378 val_378 -221 val_221 -92 val_92 -111 val_111 -47 val_47 -72 val_72 -4 val_4 -280 val_280 -35 val_35 -427 val_427 -277 val_277 -208 val_208 -356 val_356 -399 val_399 -169 val_169 -382 val_382 -498 val_498 -125 val_125 -386 val_386 -437 val_437 -469 val_469 -192 val_192 -286 val_286 -187 val_187 -176 val_176 -54 val_54 -459 val_459 -51 val_51 -138 val_138 -103 val_103 -239 val_239 -213 val_213 -216 val_216 -430 val_430 -278 val_278 -176 val_176 -289 val_289 -221 val_221 -65 val_65 -318 val_318 -332 val_332 -311 val_311 -275 val_275 -137 val_137 -241 val_241 -83 val_83 -333 val_333 -180 val_180 -284 val_284 -12 val_12 -230 val_230 -181 val_181 -67 val_67 -260 val_260 -404 val_404 -384 val_384 -489 val_489 -353 val_353 -373 val_373 -272 val_272 -138 val_138 -217 val_217 -84 val_84 -348 val_348 -466 val_466 -58 val_58 -8 val_8 -411 val_411 -230 val_230 -208 val_208 -348 val_348 -24 val_24 -463 val_463 -431 val_431 -179 val_179 -172 val_172 -42 val_42 -129 val_129 -158 val_158 -119 val_119 -496 val_496 -0 val_0 -322 val_322 -197 val_197 -468 val_468 -393 val_393 -454 val_454 -100 val_100 -298 val_298 -199 val_199 -191 val_191 -418 val_418 -96 val_96 -26 val_26 -165 val_165 -327 val_327 -230 val_230 -205 val_205 -120 val_120 -131 val_131 -51 val_51 -404 val_404 -43 val_43 -436 val_436 -156 val_156 -469 val_469 -468 val_468 -308 val_308 -95 val_95 -196 val_196 -288 val_288 -481 val_481 -457 val_457 -98 val_98 -282 val_282 -197 val_197 -187 val_187 -318 val_318 -318 val_318 -409 val_409 -470 val_470 -137 val_137 -369 val_369 -316 val_316 -169 val_169 -413 val_413 -85 val_85 -77 val_77 -0 val_0 -490 val_490 -87 val_87 -364 val_364 -179 val_179 -118 val_118 -134 val_134 -395 val_395 -282 val_282 -138 val_138 -238 val_238 -419 val_419 -15 val_15 -118 val_118 -72 val_72 -90 val_90 -307 val_307 -19 val_19 -435 val_435 -10 val_10 -277 val_277 -273 val_273 -306 val_306 -224 val_224 -309 val_309 -389 val_389 -327 val_327 -242 val_242 -369 val_369 -392 val_392 -272 val_272 -331 val_331 -401 val_401 -242 val_242 -452 val_452 -177 val_177 -226 val_226 -5 val_5 -497 val_497 -402 val_402 -396 val_396 -317 val_317 -395 val_395 -58 val_58 -35 val_35 -336 val_336 -95 val_95 -11 val_11 -168 val_168 -34 val_34 -229 val_229 -233 val_233 -143 val_143 -472 val_472 -322 val_322 -498 val_498 -160 val_160 -195 val_195 -42 val_42 -321 val_321 -430 val_430 -119 val_119 -489 val_489 -458 val_458 -78 val_78 -76 val_76 -41 val_41 -223 val_223 -492 val_492 -149 val_149 -449 val_449 -218 val_218 -228 val_228 -138 val_138 -453 val_453 -30 val_30 -209 val_209 -64 val_64 -468 val_468 -76 val_76 -74 val_74 -342 val_342 -69 val_69 -230 val_230 -33 val_33 -368 val_368 -103 val_103 -296 val_296 -113 val_113 -216 val_216 -367 val_367 -344 val_344 -167 val_167 -274 val_274 -219 val_219 -239 val_239 -485 val_485 -116 val_116 -223 val_223 -256 val_256 -263 val_263 -70 val_70 -487 val_487 -480 val_480 -401 val_401 -288 val_288 -191 val_191 -5 val_5 -244 val_244 -438 val_438 -128 val_128 -467 val_467 -432 val_432 -202 val_202 -316 val_316 -229 val_229 -469 val_469 -463 val_463 -280 val_280 -2 val_2 -35 val_35 -283 val_283 -331 val_331 -235 val_235 -80 val_80 -44 val_44 -193 val_193 -321 val_321 -335 val_335 -104 val_104 -466 val_466 -366 val_366 -175 val_175 -403 val_403 -483 val_483 -53 val_53 -105 val_105 -257 val_257 -406 val_406 -409 val_409 -190 val_190 -406 val_406 -401 val_401 -114 val_114 -258 val_258 -90 val_90 -203 val_203 -262 val_262 -348 val_348 -424 val_424 -12 val_12 -396 val_396 -201 val_201 -217 val_217 -164 val_164 -431 val_431 -454 val_454 -478 val_478 -298 val_298 -125 val_125 -431 val_431 -164 val_164 -424 val_424 -187 val_187 -382 val_382 -5 val_5 -70 val_70 -397 val_397 -480 val_480 -291 val_291 -24 val_24 -351 val_351 -255 val_255 -104 val_104 -70 val_70 -163 val_163 -438 val_438 -119 val_119 -414 val_414 -200 val_200 -491 val_491 -237 val_237 -439 val_439 -360 val_360 -248 val_248 -479 val_479 -305 val_305 -417 val_417 -199 val_199 -444 val_444 -120 val_120 -429 val_429 -169 val_169 -443 val_443 -323 val_323 -325 val_325 -277 val_277 -230 val_230 -478 val_478 -178 val_178 -468 val_468 -310 val_310 -317 val_317 -333 val_333 -493 val_493 -460 val_460 -207 val_207 -249 val_249 -265 val_265 -480 val_480 -83 val_83 -136 val_136 -353 val_353 -172 val_172 -214 val_214 -462 val_462 -233 val_233 -406 val_406 -133 val_133 -175 val_175 -189 val_189 -454 val_454 -375 val_375 -401 val_401 -421 val_421 -407 val_407 -384 val_384 -256 val_256 -26 val_26 -134 val_134 -67 val_67 -384 val_384 -379 val_379 -18 val_18 -462 val_462 -492 val_492 -100 val_100 -298 val_298 -9 val_9 -341 val_341 -498 val_498 -146 val_146 -458 val_458 -362 val_362 -186 val_186 -285 val_285 -348 val_348 -167 val_167 -18 val_18 -273 val_273 -183 val_183 -281 val_281 -344 val_344 -97 val_97 -469 val_469 -315 val_315 -84 val_84 -28 val_28 -37 val_37 -448 val_448 -152 val_152 -348 val_348 -307 val_307 -194 val_194 -414 val_414 -477 val_477 -222 val_222 -126 val_126 -90 val_90 -169 val_169 -403 val_403 -400 val_400 -200 val_200 -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd b/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd new file mode 100644 index 0000000000000..36244b327dc2f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notequal-8-a71fea4e9514cda0da9542a7701613dd @@ -0,0 +1,499 @@ +238 val_238 +86 val_86 +311 val_311 +27 val_27 +165 val_165 +409 val_409 +255 val_255 +278 val_278 +98 val_98 +484 val_484 +265 val_265 +193 val_193 +401 val_401 +150 val_150 +273 val_273 +224 val_224 +369 val_369 +66 val_66 +128 val_128 +213 val_213 +146 val_146 +406 val_406 +429 val_429 +374 val_374 +152 val_152 +469 val_469 +145 val_145 +495 val_495 +37 val_37 +327 val_327 +281 val_281 +277 val_277 +209 val_209 +15 val_15 +82 val_82 +403 val_403 +166 val_166 +417 val_417 +430 val_430 +252 val_252 +292 val_292 +219 val_219 +287 val_287 +153 val_153 +193 val_193 +338 val_338 +446 val_446 +459 val_459 +394 val_394 +237 val_237 +482 val_482 +174 val_174 +413 val_413 +494 val_494 +207 val_207 +199 val_199 +466 val_466 +208 val_208 +174 val_174 +399 val_399 +396 val_396 +247 val_247 +417 val_417 +489 val_489 +162 val_162 +377 val_377 +397 val_397 +309 val_309 +365 val_365 +266 val_266 +439 val_439 +342 val_342 +367 val_367 +325 val_325 +167 val_167 +195 val_195 +475 val_475 +17 val_17 +113 val_113 +155 val_155 +203 val_203 +339 val_339 +0 val_0 +455 val_455 +128 val_128 +311 val_311 +316 val_316 +57 val_57 +205 val_205 +149 val_149 +438 val_438 +345 val_345 +129 val_129 +170 val_170 +20 val_20 +489 val_489 +157 val_157 +378 val_378 +221 val_221 +92 val_92 +111 val_111 +47 val_47 +72 val_72 +4 val_4 +280 val_280 +35 val_35 +427 val_427 +277 val_277 +208 val_208 +356 val_356 +399 val_399 +169 val_169 +382 val_382 +498 val_498 +125 val_125 +386 val_386 +437 val_437 +469 val_469 +192 val_192 +286 val_286 +187 val_187 +176 val_176 +54 val_54 +459 val_459 +51 val_51 +138 val_138 +103 val_103 +239 val_239 +213 val_213 +216 val_216 +430 val_430 +278 val_278 +176 val_176 +289 val_289 +221 val_221 +65 val_65 +318 val_318 +332 val_332 +311 val_311 +275 val_275 +137 val_137 +241 val_241 +83 val_83 +333 val_333 +180 val_180 +284 val_284 +12 val_12 +230 val_230 +181 val_181 +67 val_67 +260 val_260 +404 val_404 +384 val_384 +489 val_489 +353 val_353 +373 val_373 +272 val_272 +138 val_138 +217 val_217 +84 val_84 +348 val_348 +466 val_466 +58 val_58 +8 val_8 +411 val_411 +230 val_230 +208 val_208 +348 val_348 +24 val_24 +463 val_463 +431 val_431 +179 val_179 +172 val_172 +42 val_42 +129 val_129 +158 val_158 +119 val_119 +496 val_496 +0 val_0 +322 val_322 +197 val_197 +468 val_468 +393 val_393 +454 val_454 +100 val_100 +298 val_298 +199 val_199 +191 val_191 +418 val_418 +96 val_96 +26 val_26 +165 val_165 +327 val_327 +230 val_230 +205 val_205 +120 val_120 +131 val_131 +51 val_51 +404 val_404 +43 val_43 +436 val_436 +156 val_156 +469 val_469 +468 val_468 +308 val_308 +95 val_95 +196 val_196 +288 val_288 +481 val_481 +457 val_457 +98 val_98 +282 val_282 +197 val_197 +187 val_187 +318 val_318 +318 val_318 +409 val_409 +470 val_470 +137 val_137 +369 val_369 +316 val_316 +169 val_169 +413 val_413 +85 val_85 +77 val_77 +0 val_0 +490 val_490 +87 val_87 +364 val_364 +179 val_179 +118 val_118 +134 val_134 +395 val_395 +282 val_282 +138 val_138 +238 val_238 +419 val_419 +15 val_15 +118 val_118 +72 val_72 +90 val_90 +307 val_307 +19 val_19 +435 val_435 +10 val_10 +277 val_277 +273 val_273 +306 val_306 +224 val_224 +309 val_309 +389 val_389 +327 val_327 +242 val_242 +369 val_369 +392 val_392 +272 val_272 +331 val_331 +401 val_401 +242 val_242 +452 val_452 +177 val_177 +226 val_226 +5 val_5 +497 val_497 +402 val_402 +396 val_396 +317 val_317 +395 val_395 +58 val_58 +35 val_35 +336 val_336 +95 val_95 +11 val_11 +168 val_168 +34 val_34 +229 val_229 +233 val_233 +143 val_143 +472 val_472 +322 val_322 +498 val_498 +160 val_160 +195 val_195 +42 val_42 +321 val_321 +430 val_430 +119 val_119 +489 val_489 +458 val_458 +78 val_78 +76 val_76 +41 val_41 +223 val_223 +492 val_492 +149 val_149 +449 val_449 +218 val_218 +228 val_228 +138 val_138 +453 val_453 +30 val_30 +209 val_209 +64 val_64 +468 val_468 +76 val_76 +74 val_74 +342 val_342 +69 val_69 +230 val_230 +33 val_33 +368 val_368 +103 val_103 +296 val_296 +113 val_113 +216 val_216 +367 val_367 +344 val_344 +167 val_167 +274 val_274 +219 val_219 +239 val_239 +485 val_485 +116 val_116 +223 val_223 +256 val_256 +263 val_263 +70 val_70 +487 val_487 +480 val_480 +401 val_401 +288 val_288 +191 val_191 +5 val_5 +244 val_244 +438 val_438 +128 val_128 +467 val_467 +432 val_432 +202 val_202 +316 val_316 +229 val_229 +469 val_469 +463 val_463 +280 val_280 +2 val_2 +35 val_35 +283 val_283 +331 val_331 +235 val_235 +80 val_80 +44 val_44 +193 val_193 +321 val_321 +335 val_335 +104 val_104 +466 val_466 +366 val_366 +175 val_175 +403 val_403 +483 val_483 +53 val_53 +105 val_105 +257 val_257 +406 val_406 +409 val_409 +190 val_190 +406 val_406 +401 val_401 +114 val_114 +258 val_258 +90 val_90 +203 val_203 +262 val_262 +348 val_348 +424 val_424 +12 val_12 +396 val_396 +201 val_201 +217 val_217 +164 val_164 +431 val_431 +454 val_454 +478 val_478 +298 val_298 +125 val_125 +431 val_431 +164 val_164 +424 val_424 +187 val_187 +382 val_382 +5 val_5 +70 val_70 +397 val_397 +480 val_480 +291 val_291 +24 val_24 +351 val_351 +255 val_255 +104 val_104 +70 val_70 +163 val_163 +438 val_438 +119 val_119 +414 val_414 +200 val_200 +491 val_491 +237 val_237 +439 val_439 +360 val_360 +248 val_248 +479 val_479 +305 val_305 +417 val_417 +199 val_199 +444 val_444 +120 val_120 +429 val_429 +169 val_169 +443 val_443 +323 val_323 +325 val_325 +277 val_277 +230 val_230 +478 val_478 +178 val_178 +468 val_468 +310 val_310 +317 val_317 +333 val_333 +493 val_493 +460 val_460 +207 val_207 +249 val_249 +265 val_265 +480 val_480 +83 val_83 +136 val_136 +353 val_353 +172 val_172 +214 val_214 +462 val_462 +233 val_233 +406 val_406 +133 val_133 +175 val_175 +189 val_189 +454 val_454 +375 val_375 +401 val_401 +421 val_421 +407 val_407 +384 val_384 +256 val_256 +26 val_26 +134 val_134 +67 val_67 +384 val_384 +379 val_379 +18 val_18 +462 val_462 +492 val_492 +100 val_100 +298 val_298 +9 val_9 +341 val_341 +498 val_498 +146 val_146 +458 val_458 +362 val_362 +186 val_186 +285 val_285 +348 val_348 +167 val_167 +18 val_18 +273 val_273 +183 val_183 +281 val_281 +344 val_344 +97 val_97 +469 val_469 +315 val_315 +84 val_84 +28 val_28 +37 val_37 +448 val_448 +152 val_152 +348 val_348 +307 val_307 +194 val_194 +414 val_414 +477 val_477 +222 val_222 +126 val_126 +90 val_90 +169 val_169 +403 val_403 +400 val_400 +200 val_200 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 b/sql/hive/src/test/resources/golden/udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 rename to sql/hive/src/test/resources/golden/udf_notop-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 b/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 deleted file mode 100644 index a55e3339049e8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 +++ /dev/null @@ -1 +0,0 @@ -false true false false true false true false true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 b/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 new file mode 100644 index 0000000000000..5ffd61b380318 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_notop-1-1ce21a9b4492969c1a97612b0ccc19f2 @@ -0,0 +1 @@ +false true false false true false true false true true diff --git a/sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_nvl-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 b/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 deleted file mode 100644 index 5ffcb851d56f9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 +++ /dev/null @@ -1 +0,0 @@ -nvl(value,default_value) - Returns default value if value is null else returns value \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 b/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 deleted file mode 100644 index 5afff220da695..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 +++ /dev/null @@ -1,4 +0,0 @@ -nvl(value,default_value) - Returns default value if value is null else returns value -Example: - > SELECT nvl(null,'bla') FROM src LIMIT 1; - bla \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 b/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 new file mode 100644 index 0000000000000..b043150b9c901 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_nvl-1-df7383141df0bb71ccb57f3eef9775b4 @@ -0,0 +1 @@ +nvl(value,default_value) - Returns default value if value is null else returns value diff --git a/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def b/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 b/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 new file mode 100644 index 0000000000000..4daa1bbffa621 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_nvl-2-6ec6c4b23c742fc604c9937a25b0b092 @@ -0,0 +1,4 @@ +nvl(value,default_value) - Returns default value if value is null else returns value +Example: + > SELECT nvl(null,'bla') FROM src LIMIT 1; + bla diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef rename to sql/hive/src/test/resources/golden/udf_nvl-3-47199a1c23cb1cc6827c601bb66513d3 diff --git a/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 b/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 deleted file mode 100644 index 2087e17494459..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 +++ /dev/null @@ -1 +0,0 @@ -1 5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 b/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 new file mode 100644 index 0000000000000..273bc7331072a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_nvl-4-656661e80deb75729fef313d5e2bd330 @@ -0,0 +1 @@ +1 5 diff --git a/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 index 9138f44ad2a41..f0e20ea40509c 100644 --- a/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 +++ b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 @@ -1 +1 @@ -a or b - Logical or \ No newline at end of file +a or b - Logical or diff --git a/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 index 9138f44ad2a41..f0e20ea40509c 100644 --- a/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 +++ b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 @@ -1 +1 @@ -a or b - Logical or \ No newline at end of file +a or b - Logical or diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 b/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 index 0535b085e50c2..1ce92fd715822 100644 --- a/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 +++ b/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 @@ -1 +1 @@ -parse_url(url, partToExtract[, key]) - extracts a part from a URL \ No newline at end of file +parse_url(url, partToExtract[, key]) - extracts a part from a URL diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab b/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab index 7178e07a07e48..bd448a4e4cade 100644 --- a/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab +++ b/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab @@ -7,4 +7,4 @@ Example: > SELECT parse_url('http://facebook.com/path/p1.php?query=1', 'QUERY') FROM src LIMIT 1; 'query=1' > SELECT parse_url('http://facebook.com/path/p1.php?query=1', 'QUERY', 'query') FROM src LIMIT 1; - '1' \ No newline at end of file + '1' diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 b/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 index f5483d4e3dafe..e3ece483b53fd 100644 --- a/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 +++ b/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 @@ -1 +1 @@ -facebook.com /path1/p.php k1=v1&k2=v2 Ref1 v2 v1 NULL /path1/p.php?k1=v1&k2=v2 http NULL facebook.com \ No newline at end of file +facebook.com /path1/p.php k1=v1&k2=v2 Ref1 v2 v1 NULL /path1/p.php?k1=v1&k2=v2 http NULL facebook.com diff --git a/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 b/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 deleted file mode 100644 index 2025042f5d493..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 +++ /dev/null @@ -1 +0,0 @@ -percentile(expr, pc) - Returns the percentile(s) of expr at pc (range: [0,1]).pc can be a double or double array diff --git a/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a b/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a deleted file mode 100644 index 2025042f5d493..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a +++ /dev/null @@ -1 +0,0 @@ -percentile(expr, pc) - Returns the percentile(s) of expr at pc (range: [0,1]).pc can be a double or double array diff --git a/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 deleted file mode 100644 index 5d2fc352ee060..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 +++ /dev/null @@ -1 +0,0 @@ -a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae deleted file mode 100644 index 5d2fc352ee060..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae +++ /dev/null @@ -1 +0,0 @@ -a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 deleted file mode 100644 index 0b46af11c4516..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 +++ /dev/null @@ -1 +0,0 @@ -6.89 51.7 18.09 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 deleted file mode 100644 index 5eb0813b60eb6..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 +++ /dev/null @@ -1 +0,0 @@ -8 51 15 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 deleted file mode 100644 index e21e4b08e7a62..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 +++ /dev/null @@ -1 +0,0 @@ -5 50 0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 deleted file mode 100644 index e0bc2a844fb46..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 +++ /dev/null @@ -1 +0,0 @@ -8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 deleted file mode 100644 index e0bc2a844fb46..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 +++ /dev/null @@ -1 +0,0 @@ -8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 deleted file mode 100644 index e0bc2a844fb46..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 +++ /dev/null @@ -1 +0,0 @@ -8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd deleted file mode 100644 index 48371142e9b5d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd +++ /dev/null @@ -1 +0,0 @@ -6.8899984 51.700005 18.089996 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 deleted file mode 100644 index ab842acd48b3c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 +++ /dev/null @@ -1 +0,0 @@ -6.890000000000011 51.699999999999996 18.090000000000003 diff --git a/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 index 7c98729cc195c..6374b08a607ab 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 +++ b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 @@ -1 +1 @@ -positive a - Returns a \ No newline at end of file +positive a - Returns a diff --git a/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c index 7c98729cc195c..6374b08a607ab 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c +++ b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c @@ -1 +1 @@ -positive a - Returns a \ No newline at end of file +positive a - Returns a diff --git a/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 +++ b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 index e8f11c444a808..9d794539b4ae0 100644 --- a/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 +++ b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 @@ -1 +1 @@ -a + b - Returns a+b \ No newline at end of file +a + b - Returns a+b diff --git a/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 b/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 index 43f197e360c82..67377963d2aa1 100644 --- a/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 +++ b/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 @@ -1 +1 @@ -pow(x1, x2) - raise x1 to the power of x2 \ No newline at end of file +pow(x1, x2) - raise x1 to the power of x2 diff --git a/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 b/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 index afc2e4462a1a8..ded9d10a595ff 100644 --- a/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 +++ b/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 @@ -2,4 +2,4 @@ pow(x1, x2) - raise x1 to the power of x2 Synonyms: power Example: > SELECT pow(2, 3) FROM src LIMIT 1; - 8 \ No newline at end of file + 8 diff --git a/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 b/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 index 5e3a6a8f31fd3..90e23c3255b77 100644 --- a/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 +++ b/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 @@ -1 +1 @@ -power(x1, x2) - raise x1 to the power of x2 \ No newline at end of file +power(x1, x2) - raise x1 to the power of x2 diff --git a/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 b/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 index c3414e29768e3..4890e2e989d34 100644 --- a/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 +++ b/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 @@ -2,4 +2,4 @@ power(x1, x2) - raise x1 to the power of x2 Synonyms: pow Example: > SELECT power(2, 3) FROM src LIMIT 1; - 8 \ No newline at end of file + 8 diff --git a/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e b/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e deleted file mode 100644 index 1635ff88dd768..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e +++ /dev/null @@ -1 +0,0 @@ -printf(String format, Obj... args) - function that can format strings according to printf-style format strings diff --git a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee b/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee deleted file mode 100644 index 62440ee68e145..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee +++ /dev/null @@ -1,4 +0,0 @@ -printf(String format, Obj... args) - function that can format strings according to printf-style format strings -Example: - > SELECT printf("Hello World %d %s", 100, "days")FROM src LIMIT 1; - "Hello World 100 days" diff --git a/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 b/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 b/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 deleted file mode 100644 index 39cb945991403..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 +++ /dev/null @@ -1 +0,0 @@ -Hello World 100 days diff --git a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb b/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb deleted file mode 100644 index 04bf5e552a576..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb +++ /dev/null @@ -1 +0,0 @@ -All Type Test: false, A, 15000, 1.234000e+01, +27183.2401, 2300.41, 32, corret, 0x1.002p8 diff --git a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 b/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 deleted file mode 100644 index 2e9f7509968a3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 +++ /dev/null @@ -1 +0,0 @@ -Color red, String Null: null, number1 123456, number2 00089, Integer Null: null, hex 0xff, float 3.14 Double Null: null diff --git a/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 b/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_radians-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/udf_radians-1-58b73fc96927d447d1225f021eaa378 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 rename to sql/hive/src/test/resources/golden/udf_radians-1-58b73fc96927d447d1225f021eaa378 diff --git a/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 deleted file mode 100644 index 116b6bc461ed0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 +++ /dev/null @@ -1 +0,0 @@ -1.000000357564167 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..b00bf83e61d82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-10-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 @@ -0,0 +1,5 @@ +radians(x) - Converts degrees to radians +Example: + > SELECT radians(90) FROM src LIMIT 1; + 1.5707963267949mo + diff --git a/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 deleted file mode 100644 index aaf3b31fef488..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 +++ /dev/null @@ -1 +0,0 @@ -2.4999991485811655 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 b/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 new file mode 100644 index 0000000000000..6b0996864478b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-2-cb8462f25c18b7405c41a50e52476d04 @@ -0,0 +1 @@ +1.000000357564167 diff --git a/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 deleted file mode 100644 index 73ad88be4ef3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 +++ /dev/null @@ -1 +0,0 @@ -radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 b/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 new file mode 100644 index 0000000000000..1f204866982ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-3-bd00297cb26f599913b14a635e768be3 @@ -0,0 +1 @@ +2.4999991485811655 diff --git a/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..99e71b16f18ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-4-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians diff --git a/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 deleted file mode 100644 index e0237c0058f55..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 +++ /dev/null @@ -1,4 +0,0 @@ -radians(x) - Converts degrees to radians -Example: - > SELECT radians(90) FROM src LIMIT 1; - 1.5707963267949mo diff --git a/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..b00bf83e61d82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-5-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 @@ -0,0 +1,5 @@ +radians(x) - Converts degrees to radians +Example: + > SELECT radians(90) FROM src LIMIT 1; + 1.5707963267949mo + diff --git a/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/udf_radians-6-70c9e7199b5898e2c3a4943ec58da113 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 rename to sql/hive/src/test/resources/golden/udf_radians-6-70c9e7199b5898e2c3a4943ec58da113 diff --git a/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 deleted file mode 100644 index 116b6bc461ed0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 +++ /dev/null @@ -1 +0,0 @@ -1.000000357564167 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 deleted file mode 100644 index aaf3b31fef488..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 +++ /dev/null @@ -1 +0,0 @@ -2.4999991485811655 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 b/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 new file mode 100644 index 0000000000000..6b0996864478b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-7-cb8462f25c18b7405c41a50e52476d04 @@ -0,0 +1 @@ +1.000000357564167 diff --git a/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 deleted file mode 100644 index 73ad88be4ef3d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 +++ /dev/null @@ -1 +0,0 @@ -radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 b/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 new file mode 100644 index 0000000000000..1f204866982ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-8-bd00297cb26f599913b14a635e768be3 @@ -0,0 +1 @@ +2.4999991485811655 diff --git a/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..99e71b16f18ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-9-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians diff --git a/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 deleted file mode 100644 index e0237c0058f55..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 +++ /dev/null @@ -1,4 +0,0 @@ -radians(x) - Converts degrees to radians -Example: - > SELECT radians(90) FROM src LIMIT 1; - 1.5707963267949mo diff --git a/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 b/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 index d19c5b50fe5c4..a4de35428cc8f 100644 --- a/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 +++ b/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 @@ -1 +1 @@ -rand([seed]) - Returns a pseudorandom number between 0 and 1 \ No newline at end of file +rand([seed]) - Returns a pseudorandom number between 0 and 1 diff --git a/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e b/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e index d19c5b50fe5c4..a4de35428cc8f 100644 --- a/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e +++ b/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e @@ -1 +1 @@ -rand([seed]) - Returns a pseudorandom number between 0 and 1 \ No newline at end of file +rand([seed]) - Returns a pseudorandom number between 0 and 1 diff --git a/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 b/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 deleted file mode 100644 index 1d7658151cd62..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 +++ /dev/null @@ -1 +0,0 @@ -reflect(class,method[,arg1[,arg2..]]) calls method with reflection diff --git a/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee b/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee deleted file mode 100644 index ddf986c01e5b1..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee +++ /dev/null @@ -1,4 +0,0 @@ -reflect(class,method[,arg1[,arg2..]]) calls method with reflection -Synonyms: java_method -Use this UDF to call Java methods by matching the argument signature - diff --git a/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 b/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 b/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 deleted file mode 100644 index cd35e5b290db5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 +++ /dev/null @@ -1 +0,0 @@ -reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 b/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 deleted file mode 100644 index 48ef97292ab62..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 +++ /dev/null @@ -1,3 +0,0 @@ -reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection -Use this UDF to call Java methods by matching the argument signature - diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca b/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 b/sql/hive/src/test/resources/golden/udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 rename to sql/hive/src/test/resources/golden/udf_regexp-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 b/sql/hive/src/test/resources/golden/udf_regexp-1-19917611f74aedc0922560f7f2595948 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 rename to sql/hive/src/test/resources/golden/udf_regexp-1-19917611f74aedc0922560f7f2595948 diff --git a/sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 b/sql/hive/src/test/resources/golden/udf_regexp-2-f7f0527cd47612d7f256edd5f8963800 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 rename to sql/hive/src/test/resources/golden/udf_regexp-2-f7f0527cd47612d7f256edd5f8963800 diff --git a/sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 b/sql/hive/src/test/resources/golden/udf_regexp-3-59aff54bae544ee620141e4e629f167a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 rename to sql/hive/src/test/resources/golden/udf_regexp-3-59aff54bae544ee620141e4e629f167a diff --git a/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 b/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 index 429057caf71f0..6f4b3cea94c03 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 +++ b/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 @@ -1 +1 @@ -regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp \ No newline at end of file +regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp diff --git a/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb b/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb index 30e237490811f..fc9d4aa686db9 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb +++ b/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb @@ -1,4 +1,4 @@ regexp_extract(str, regexp[, idx]) - extracts a group that matches regexp Example: > SELECT regexp_extract('100-200', '(\d+)-(\d+)', 1) FROM src LIMIT 1; - '100' \ No newline at end of file + '100' diff --git a/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e b/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e index 1a38701d68875..193093b0edcaa 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e +++ b/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e @@ -1 +1 @@ -regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep \ No newline at end of file +regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep diff --git a/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc b/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc index bebfabaf637fb..6e3577aba5da2 100644 --- a/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc +++ b/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc @@ -1,4 +1,4 @@ regexp_replace(str, regexp, rep) - replace all substrings of str that match regexp with rep Example: > SELECT regexp_replace('100-200', '(\d+)', 'num') FROM src LIMIT 1; - 'num-num' \ No newline at end of file + 'num-num' diff --git a/sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_repeat-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b b/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b deleted file mode 100644 index 694c367436f3c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b +++ /dev/null @@ -1 +0,0 @@ -repeat(str, n) - repeat str n times \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b b/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b deleted file mode 100644 index 5df19ba1c2cbf..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b +++ /dev/null @@ -1,4 +0,0 @@ -repeat(str, n) - repeat str n times -Example: - > SELECT repeat('123', 2) FROM src LIMIT 1; - '123123' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b b/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b new file mode 100644 index 0000000000000..23a6a30503468 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-1-fdeae3e88f27ef148281d69ac8c4b23b @@ -0,0 +1 @@ +repeat(str, n) - repeat str n times diff --git a/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b b/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b new file mode 100644 index 0000000000000..4f4b491b2807c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-2-836be47190989d8975a09a545ecbfe0b @@ -0,0 +1,4 @@ +repeat(str, n) - repeat str n times +Example: + > SELECT repeat('123', 2) FROM src LIMIT 1; + '123123' diff --git a/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a b/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d rename to sql/hive/src/test/resources/golden/udf_repeat-3-3a3180b4d7c59ee477ce4bebf8e6adec diff --git a/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 b/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 deleted file mode 100644 index 45425cf087c09..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 +++ /dev/null @@ -1 +0,0 @@ -FacebookFacebookFacebook \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 b/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 new file mode 100644 index 0000000000000..5a355c1c58fc7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-4-64c5fce0c5ad4c26680a842aa740dc57 @@ -0,0 +1 @@ +FacebookFacebookFacebook diff --git a/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 b/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 index fd9ac3081a1cc..068aeb36f6477 100644 --- a/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 +++ b/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 @@ -1 +1 @@ -str rlike regexp - Returns true if str matches regexp and false otherwise \ No newline at end of file +str rlike regexp - Returns true if str matches regexp and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca b/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca index 43b06945caa54..ab346793b9d3d 100644 --- a/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca +++ b/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca @@ -2,4 +2,4 @@ str rlike regexp - Returns true if str matches regexp and false otherwise Synonyms: regexp Example: > SELECT 'fb' rlike '.*' FROM src LIMIT 1; - true \ No newline at end of file + true diff --git a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 deleted file mode 100644 index e4586b2e73a93..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 +++ /dev/null @@ -1 +0,0 @@ -round(x[, d]) - round x to d decimal places \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_round-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_round-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 new file mode 100644 index 0000000000000..49fdc0a774e70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-1-10b53ca1f15fd7879365926f86512d15 @@ -0,0 +1 @@ +round(x[, d]) - round x to d decimal places diff --git a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 deleted file mode 100644 index c0d5b480e9751..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 +++ /dev/null @@ -1,4 +0,0 @@ -round(x[, d]) - round x to d decimal places -Example: - > SELECT round(12.3456, 1) FROM src LIMIT 1; - 12.3' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 new file mode 100644 index 0000000000000..862adeae821ff --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-2-2367bcc43510dedc80bdb6707e434da8 @@ -0,0 +1,4 @@ +round(x[, d]) - round x to d decimal places +Example: + > SELECT round(12.3456, 1) FROM src LIMIT 1; + 12.3' diff --git a/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 b/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 deleted file mode 100644 index 0924e3869076d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL Infinity NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c b/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c deleted file mode 100644 index 3b9c30929a240..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c +++ /dev/null @@ -1 +0,0 @@ -55555 55555.0 55555.0 55555.0 55555.0 55560.0 55600.0 56000.0 60000.0 100000.0 0.0 0.0 0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de b/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de new file mode 100644 index 0000000000000..fc6e4224259d8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-3-fa4d11da8e1eba258ed191ed5f1447de @@ -0,0 +1 @@ +NULL NULL NULL NULL NaN diff --git a/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 b/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 new file mode 100644 index 0000000000000..f8d833cc0880e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-4-b87ccaa1e0a87c558b56d59a8a074396 @@ -0,0 +1 @@ +55555 55555 55555 55555 55555 55560 55600 56000 60000 100000 0 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 b/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 deleted file mode 100644 index 1f243f6cbc6db..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 +++ /dev/null @@ -1 +0,0 @@ -125.0 125.0 125.3 125.32 125.315 125.315 130.0 100.0 0.0 0.0 -125.0 -125.0 -125.3 -125.32 -125.315 -125.315 -130.0 -100.0 0.0 0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d b/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d new file mode 100644 index 0000000000000..389ab6417f19d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-5-441d0075081ae87579c959d714c4922d @@ -0,0 +1 @@ +125.0 125.0 125.3 125.32 125.315 125.315 130.0 100.0 0.0 0.0 -125.0 -125.0 -125.3 -125.32 -125.315 -125.315 -130.0 -100.0 0.0 0.0 diff --git a/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a b/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a deleted file mode 100644 index 918404cec8047..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a +++ /dev/null @@ -1 +0,0 @@ -0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 3.0 3.1 3.14 3.142 3.1416 3.14159 3.141593 3.1415927 3.14159265 3.141592654 3.1415926536 3.14159265359 3.14159265359 3.1415926535898 3.1415926535898 3.14159265358979 3.141592653589793 3.141592653589793 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 b/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 new file mode 100644 index 0000000000000..3b083b3e26c4a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-6-4658ec3bc034b43e0477bf2474939449 @@ -0,0 +1 @@ +0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 0.0 3.0 3.1 3.14 3.142 3.1416 3.14159 3.141593 3.1415927 3.14159265 3.141592654 3.1415926536 3.14159265359 3.14159265359 3.1415926535898 3.1415926535898 3.14159265358979 3.141592653589793 3.141592653589793 diff --git a/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b b/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b deleted file mode 100644 index af105563af144..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b +++ /dev/null @@ -1 +0,0 @@ -1809242.315111134 -1809242.315111134 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d b/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d new file mode 100644 index 0000000000000..c3496bf5b6d1b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-7-74ff5a4862c80bd8fd84bede1a0320d @@ -0,0 +1 @@ +1809242.315111134 -1809242.315111134 1809242.315111134 -1809242.315111134 diff --git a/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 b/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce b/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 b/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 b/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 b/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b b/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b deleted file mode 100644 index f40e633f703c9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b +++ /dev/null @@ -1 +0,0 @@ -Infinity Infinity Infinity Infinity \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a rename to sql/hive/src/test/resources/golden/udf_round_3-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c b/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c deleted file mode 100644 index 3714de0db18dc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c +++ /dev/null @@ -1 +0,0 @@ --128 127 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d b/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d new file mode 100644 index 0000000000000..0a104d81ef51e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-1-15a7f123f596e28e6f238063ba4e3d6d @@ -0,0 +1 @@ +-128 127 0 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 b/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 deleted file mode 100644 index a9265d7b8a1b7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 +++ /dev/null @@ -1 +0,0 @@ --32768 32767 -129 128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 b/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 deleted file mode 100644 index 2d25f54073df7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 +++ /dev/null @@ -1 +0,0 @@ --2147483648 2147483647 -32769 32768 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca b/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca new file mode 100644 index 0000000000000..972dee75fae8a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-2-a03bf4e99027d4814a32c84d89d42cca @@ -0,0 +1 @@ +-32768 32767 -129 128 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 b/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 new file mode 100644 index 0000000000000..50928a309cf3a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-3-e3c5b35d67ef3de2800a1836718e8ac9 @@ -0,0 +1 @@ +-2147483648 2147483647 -32769 32768 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 b/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 deleted file mode 100644 index 41a2624c6cfeb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 +++ /dev/null @@ -1 +0,0 @@ --9223372036854775808 9223372036854775807 -2147483649 2147483648 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 b/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 new file mode 100644 index 0000000000000..c9e55e0ec0b95 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-4-8449fbdabbb4b1e6beab89be0af498f2 @@ -0,0 +1 @@ +-9223372036854775808 9223372036854775807 -2147483649 2147483648 diff --git a/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 b/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 deleted file mode 100644 index 98d3f53dfc442..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 +++ /dev/null @@ -1 +0,0 @@ -126.0 127.0 32766.0 32767.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad b/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad new file mode 100644 index 0000000000000..3898a62cae1b5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-5-3844531c8cff115d6a33636db0a26ad @@ -0,0 +1 @@ +126.0 127.0 32766.0 32767.0 diff --git a/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 rename to sql/hive/src/test/resources/golden/udf_rpad-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c b/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c deleted file mode 100644 index 7cb2d71d4b80d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c +++ /dev/null @@ -1 +0,0 @@ -rpad(str, len, pad) - Returns str, right-padded with pad to a length of len \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 b/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 deleted file mode 100644 index 2b198b8dc96c4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 +++ /dev/null @@ -1,6 +0,0 @@ -rpad(str, len, pad) - Returns str, right-padded with pad to a length of len -If str is longer than len, the return value is shortened to len characters. -Example: - > SELECT rpad('hi', 5, '??') FROM src LIMIT 1; - 'hi???' > SELECT rpad('hi', 1, '??') FROM src LIMIT 1; - 'h' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c b/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c new file mode 100644 index 0000000000000..f451030a3a142 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_rpad-1-56de896c5fe8b40b22a9ed55ed79889c @@ -0,0 +1 @@ +rpad(str, len, pad) - Returns str, right-padded with pad to a length of len diff --git a/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 b/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 b/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 new file mode 100644 index 0000000000000..67dcf2427362d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_rpad-2-48d17e5d5d6188300d048f987fab2ca0 @@ -0,0 +1,6 @@ +rpad(str, len, pad) - Returns str, right-padded with pad to a length of len +If str is longer than len, the return value is shortened to len characters. +Example: + > SELECT rpad('hi', 5, '??') FROM src LIMIT 1; + 'hi???' > SELECT rpad('hi', 1, '??') FROM src LIMIT 1; + 'h' diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/udf_rpad-3-66acb969c28a8e376782ccd0d442b450 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b rename to sql/hive/src/test/resources/golden/udf_rpad-3-66acb969c28a8e376782ccd0d442b450 diff --git a/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 b/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 deleted file mode 100644 index b2645e365c8a9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 +++ /dev/null @@ -1 +0,0 @@ -h hi... hi1231 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b b/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b new file mode 100644 index 0000000000000..0d73ca82abf89 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_rpad-4-299dee5a72aad2a2738d7841a89bb71b @@ -0,0 +1 @@ +h hi... hi1231 diff --git a/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a b/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a index 94d6aeed8f533..d0d2416d7ee90 100644 --- a/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a +++ b/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a @@ -1 +1 @@ -rtrim(str) - Removes the trailing space characters from str \ No newline at end of file +rtrim(str) - Removes the trailing space characters from str diff --git a/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd b/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd index 957e608a7c732..697cdcc8198b3 100644 --- a/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd +++ b/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd @@ -1,4 +1,4 @@ rtrim(str) - Removes the trailing space characters from str Example: > SELECT rtrim('facebook ') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/udf_second-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a rename to sql/hive/src/test/resources/golden/udf_second-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 b/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 deleted file mode 100644 index 577c90254cb5a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 +++ /dev/null @@ -1 +0,0 @@ -second(date) - Returns the second of date \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 b/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 deleted file mode 100644 index 4b48294e5b9ad..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 +++ /dev/null @@ -1,7 +0,0 @@ -second(date) - Returns the second of date -date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. -Example: - > SELECT second('2009-07-30 12:58:59') FROM src LIMIT 1; - 59 - > SELECT second('12:58:59') FROM src LIMIT 1; - 59 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 b/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 new file mode 100644 index 0000000000000..ad2b24b8eb11c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-1-e004a6f20da3fa6db868ec847b217ff2 @@ -0,0 +1 @@ +second(date) - Returns the second of date diff --git a/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 b/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 new file mode 100644 index 0000000000000..9dc38c3e79129 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-2-3525f55f4f13253c42b3abaa53d77888 @@ -0,0 +1,7 @@ +second(date) - Returns the second of date +date is a string in the format of 'yyyy-MM-dd HH:mm:ss' or 'HH:mm:ss'. +Example: + > SELECT second('2009-07-30 12:58:59') FROM src LIMIT 1; + 59 + > SELECT second('12:58:59') FROM src LIMIT 1; + 59 diff --git a/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f b/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf b/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf deleted file mode 100644 index 4d5ef5cf4a699..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf +++ /dev/null @@ -1 +0,0 @@ -15 15 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/udf_second-3-d678372e3837a16be245d2e33482f17f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 rename to sql/hive/src/test/resources/golden/udf_second-3-d678372e3837a16be245d2e33482f17f diff --git a/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf b/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf new file mode 100644 index 0000000000000..3ddfab9c754f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-4-2496e4d3c64ca028184431c2930d82cf @@ -0,0 +1 @@ +15 15 NULL diff --git a/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 deleted file mode 100644 index fea4f860c8465..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 +++ /dev/null @@ -1,31 +0,0 @@ -41626672616765 -446174656E -48697665 -496E74657270756E6B74696F6E -4D756C7469706C65 -53C3A4747A65 -554446 -5665727765636873656C756E67 -5765726B7A657567 -616C73 -61757367657A656963686E65746573 -646965 -646965 -6469657365 -646F6368 -65696E -66756E6B74696F6E69657274 -66C3BC72 -676562696C646574656E -696D6D6572 -697374 -697374 -6D61736368696E656C6C65 -6E6F6368 -7363686C65636874 -756E64 -756E64 -7669656C6C6569636874 -7669656C7365697469676572 -766F6E -C39C6265727365747A756E67 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 b/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 deleted file mode 100644 index c49ca6bb5a1b0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 +++ /dev/null @@ -1 +0,0 @@ -[["Hive","is","an","excellent","tool","for","data","querying","and","perhaps","more","versatile","than","machine","translation"],["Multiple","ill-formed","sentences","confounding","punctuation","and","yet","this","UDF","still","works"]] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 b/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 deleted file mode 100644 index b798628e56686..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 +++ /dev/null @@ -1,50 +0,0 @@ -48697665 -48697665 -554446 -6175746F6D617469717565 -6175746F6D617469717565 -6365 -636F6E667573696F6E -6465 -6465 -646573 -646F6E6EC3A96573 -646F6E6EC3A96573 -656E636F7265 -657374 -657374 -6574 -6574 -6574 -657863656C6C656E74 -657863656C6C656E74 -666F6E6374696F6E6E65 -666F726DC3A96573 -6C61 -6C61 -6C61 -6C6573 -6C6573 -6D616C -6D756C7469706C6573 -6F7574696C -6F7574696C -706575742DC3AA747265 -706575742DC3AA747265 -70687261736573 -706C7573 -706C7573 -706F6C7976616C656E74 -706F6C7976616C656E74 -706F6E6374756174696F6E -706F7572 -706F7572 -706F757274616E74 -717565 -717565 -72657175C3AA746573 -72657175C3AA746573 -74726164756374696F6E -74726164756374696F6E -756E -756E \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 b/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de b/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 b/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_sign-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c rename to sql/hive/src/test/resources/golden/udf_sign-1-cddd6ec2a7dfc2f8f7e35bc39df541f9 diff --git a/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/sql/hive/src/test/resources/golden/udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a rename to sql/hive/src/test/resources/golden/udf_sign-10-9a5326b0bf612fed4ce0b04770bebc16 diff --git a/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa deleted file mode 100644 index 3345fbb8f2c35..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa +++ /dev/null @@ -1 +0,0 @@ -sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a deleted file mode 100644 index bdacec8810e77..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a +++ /dev/null @@ -1,4 +0,0 @@ -sign(x) - returns the sign of x ) -Example: - > SELECT sign(40) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..60533947bcfb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-11-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) diff --git a/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..d888f9e9cda19 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-12-74237f5ecc497813cf9738b21647847a @@ -0,0 +1,4 @@ +sign(x) - returns the sign of x ) +Example: + > SELECT sign(40) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef deleted file mode 100644 index 31a5b0b81dc51..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef +++ /dev/null @@ -1 +0,0 @@ --1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 b/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-2-fba3eb5c16eca01b0c0f5918dbbffbc7 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 b/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 new file mode 100644 index 0000000000000..18e16e38c5de6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-3-d3e4489fc6873b5dbc4fe3e99ef13900 @@ -0,0 +1 @@ +-1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/sql/hive/src/test/resources/golden/udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c rename to sql/hive/src/test/resources/golden/udf_sign-4-9a5326b0bf612fed4ce0b04770bebc16 diff --git a/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa deleted file mode 100644 index 3345fbb8f2c35..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa +++ /dev/null @@ -1 +0,0 @@ -sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a deleted file mode 100644 index bdacec8810e77..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a +++ /dev/null @@ -1,4 +0,0 @@ -sign(x) - returns the sign of x ) -Example: - > SELECT sign(40) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..60533947bcfb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-5-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) diff --git a/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..d888f9e9cda19 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-6-74237f5ecc497813cf9738b21647847a @@ -0,0 +1,4 @@ +sign(x) - returns the sign of x ) +Example: + > SELECT sign(40) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/udf_sign-7-ed2aaa1a416c0cccc04de970424e1860 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a rename to sql/hive/src/test/resources/golden/udf_sign-7-ed2aaa1a416c0cccc04de970424e1860 diff --git a/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef deleted file mode 100644 index 31a5b0b81dc51..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef +++ /dev/null @@ -1 +0,0 @@ --1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 b/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-8-fba3eb5c16eca01b0c0f5918dbbffbc7 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 b/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 new file mode 100644 index 0000000000000..18e16e38c5de6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-9-d3e4489fc6873b5dbc4fe3e99ef13900 @@ -0,0 +1 @@ +-1.0 diff --git a/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee b/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee deleted file mode 100644 index 6155d9a2eae55..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee +++ /dev/null @@ -1 +0,0 @@ -sin(x) - returns the sine of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_sin-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 b/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 deleted file mode 100644 index a39ed8840b916..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 +++ /dev/null @@ -1,4 +0,0 @@ -sin(x) - returns the sine of x (x is in radians) -Example: - > SELECT sin(0) FROM src LIMIT 1; - 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee b/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee new file mode 100644 index 0000000000000..86bbf99dbf067 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-1-40b50393869eb0bcde66e36fe41078ee @@ -0,0 +1 @@ +sin(x) - returns the sine of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 b/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 new file mode 100644 index 0000000000000..0efbed397abf9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-2-2f867f432fb322e21dce353d7eb50c63 @@ -0,0 +1,4 @@ +sin(x) - returns the sine of x (x is in radians) +Example: + > SELECT sin(0) FROM src LIMIT 1; + 0 diff --git a/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab b/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 b/sql/hive/src/test/resources/golden/udf_sin-3-1d3a615e3aa252a317daa601811820b1 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 rename to sql/hive/src/test/resources/golden/udf_sin-3-1d3a615e3aa252a317daa601811820b1 diff --git a/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 b/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 deleted file mode 100644 index ee21925b0ccc5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 +++ /dev/null @@ -1 +0,0 @@ -0.8304973704919705 0.9999996829318346 -0.479425538604203 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a b/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a new file mode 100644 index 0000000000000..4ca4af756468c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-4-951fb8d311f52ab59d3bacd37d3e611a @@ -0,0 +1 @@ +0.8304973704919705 0.9999996829318346 -0.479425538604203 diff --git a/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 b/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 deleted file mode 100644 index d8952629349a9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 +++ /dev/null @@ -1 +0,0 @@ -size(a) - Returns the size of a diff --git a/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 b/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 deleted file mode 100644 index d8952629349a9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 +++ /dev/null @@ -1 +0,0 @@ -size(a) - Returns the size of a diff --git a/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 b/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd b/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd deleted file mode 100644 index 08a708b090282..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd +++ /dev/null @@ -1 +0,0 @@ -3 1 1 -1 diff --git a/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 index 572ecb0462eb7..132d6138e4983 100644 --- a/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 +++ b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 @@ -1 +1 @@ -There is no documentation for function 'smallint' \ No newline at end of file +There is no documentation for function 'smallint' diff --git a/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 index 572ecb0462eb7..132d6138e4983 100644 --- a/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 +++ b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 @@ -1 +1 @@ -There is no documentation for function 'smallint' \ No newline at end of file +There is no documentation for function 'smallint' diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 deleted file mode 100644 index d514df4191b89..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 +++ /dev/null @@ -1 +0,0 @@ -sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 deleted file mode 100644 index 43e36513de881..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 +++ /dev/null @@ -1,4 +0,0 @@ -sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. -Example: - > SELECT sort_array(array('b', 'd', 'c', 'a')) FROM src LIMIT 1; - 'a', 'b', 'c', 'd' diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 b/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 deleted file mode 100644 index 2e9458debfd0f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 +++ /dev/null @@ -1 +0,0 @@ -["a","b","c","d","e","f","g"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e b/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e deleted file mode 100644 index 94f18d09863a7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e +++ /dev/null @@ -1 +0,0 @@ -["enterprise databases","hadoop distributed file system","hadoop map-reduce"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 b/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 deleted file mode 100644 index e1968ef44a2eb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 +++ /dev/null @@ -1 +0,0 @@ -[1,2,3,4,5,6,7,8,9] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 b/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 deleted file mode 100644 index b6c12ace4162b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 +++ /dev/null @@ -1 +0,0 @@ -[-3.445,0.777,1.0,1.325,2.003,2.333,9.0] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 b/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a b/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 b/sql/hive/src/test/resources/golden/udf_space-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 rename to sql/hive/src/test/resources/golden/udf_space-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 b/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 deleted file mode 100644 index ed5bda787df23..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 +++ /dev/null @@ -1 +0,0 @@ -space(n) - returns n spaces \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 b/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 new file mode 100644 index 0000000000000..a443bc3cbf0bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-1-91e879c9f01d90eab7bf12fcef256010 @@ -0,0 +1 @@ +space(n) - returns n spaces diff --git a/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 b/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 deleted file mode 100644 index 5713d4b0464be..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 +++ /dev/null @@ -1,4 +0,0 @@ -space(n) - returns n spaces -Example: - > SELECT space(2) FROM src LIMIT 1; - ' ' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 b/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 b/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 new file mode 100644 index 0000000000000..25a7583ef01e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-2-e4eaf5e96807e122548cb43be9a26754 @@ -0,0 +1,4 @@ +space(n) - returns n spaces +Example: + > SELECT space(2) FROM src LIMIT 1; + ' ' diff --git a/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 b/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 deleted file mode 100644 index 85a16b2abe5ef..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 +++ /dev/null @@ -1 +0,0 @@ -10 0 1 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/udf_space-3-a1b9dad63547f7ba73a5230d650983b0 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 rename to sql/hive/src/test/resources/golden/udf_space-3-a1b9dad63547f7ba73a5230d650983b0 diff --git a/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 b/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 deleted file mode 100644 index 8dfaf2745f666..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 b/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 new file mode 100644 index 0000000000000..6f07be9b1d043 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-4-d9de5746edd753507c3f054e1bee7702 @@ -0,0 +1 @@ +10 0 1 0 0 diff --git a/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 b/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 new file mode 100644 index 0000000000000..8f243e851f12f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-5-ce5288dcc60f9412109930bd56752a65 @@ -0,0 +1 @@ + diff --git a/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 b/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 deleted file mode 100644 index 6b183ccfb17b4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 +++ /dev/null @@ -1 +0,0 @@ -split(str, regex) - Splits str around occurances that match regex \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 b/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 deleted file mode 100644 index 9e5c522da155b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 +++ /dev/null @@ -1,4 +0,0 @@ -split(str, regex) - Splits str around occurances that match regex -Example: - > SELECT split('oneAtwoBthreeC', '[ABC]') FROM src LIMIT 1; - ["one", "two", "three"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 b/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 b/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 deleted file mode 100644 index 9174f1a92557e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 +++ /dev/null @@ -1 +0,0 @@ -["a","b","c"] ["one","two","three"] [] ["5","4","1","2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 b/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 index eadd6eeb8b149..01efc31b30ede 100644 --- a/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 +++ b/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 @@ -1 +1 @@ -sqrt(x) - returns the square root of x \ No newline at end of file +sqrt(x) - returns the square root of x diff --git a/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 b/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 index 74df32f221278..16b77c4130fb6 100644 --- a/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 +++ b/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 @@ -1,4 +1,4 @@ sqrt(x) - returns the square root of x Example: > SELECT sqrt(4) FROM src LIMIT 1; - 2 \ No newline at end of file + 2 diff --git a/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b b/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b index 2966c01c1b7b2..5cedcfd415c88 100644 --- a/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b +++ b/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b @@ -1 +1 @@ -std(x) - Returns the standard deviation of a set of numbers \ No newline at end of file +std(x) - Returns the standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 b/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 index 388eaa1c7439a..d54ebfbd6fb1a 100644 --- a/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 +++ b/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 @@ -1,2 +1,2 @@ std(x) - Returns the standard deviation of a set of numbers -Synonyms: stddev_pop, stddev \ No newline at end of file +Synonyms: stddev_pop, stddev diff --git a/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 b/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 index 74c6e1eacc379..c8b11307792b3 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 +++ b/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 @@ -1 +1 @@ -stddev(x) - Returns the standard deviation of a set of numbers \ No newline at end of file +stddev(x) - Returns the standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d b/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d index 1c461b68c1440..5f674788180e8 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d +++ b/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d @@ -1,2 +1,2 @@ stddev(x) - Returns the standard deviation of a set of numbers -Synonyms: stddev_pop, std \ No newline at end of file +Synonyms: stddev_pop, std diff --git a/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 b/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 index 741771707f280..98b461bd09708 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 +++ b/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 @@ -1 +1 @@ -Function 'udf_stddev_pop' does not exist. \ No newline at end of file +Function 'udf_stddev_pop' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 b/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 index 741771707f280..98b461bd09708 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 +++ b/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 @@ -1 +1 @@ -Function 'udf_stddev_pop' does not exist. \ No newline at end of file +Function 'udf_stddev_pop' does not exist. diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 index c1fb091d2c425..cc3c37142270e 100644 --- a/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 +++ b/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 @@ -1 +1 @@ -stddev_samp(x) - Returns the sample standard deviation of a set of numbers \ No newline at end of file +stddev_samp(x) - Returns the sample standard deviation of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 index c61c4e1135410..4568e35cb1a21 100644 --- a/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 +++ b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 @@ -1 +1 @@ -There is no documentation for function 'string' \ No newline at end of file +There is no documentation for function 'string' diff --git a/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d index c61c4e1135410..4568e35cb1a21 100644 --- a/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d +++ b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d @@ -1 +1 @@ -There is no documentation for function 'string' \ No newline at end of file +There is no documentation for function 'string' diff --git a/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd b/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd deleted file mode 100644 index c4efdaebc8aab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd +++ /dev/null @@ -1 +0,0 @@ -substr(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstr(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len diff --git a/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 b/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 deleted file mode 100644 index 44958996269c9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 +++ /dev/null @@ -1,10 +0,0 @@ -substr(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstr(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len -Synonyms: substring -pos is a 1-based index. If pos<0 the starting position is determined by counting backwards from the end of str. -Example: - > SELECT substr('Facebook', 5) FROM src LIMIT 1; - 'book' - > SELECT substr('Facebook', -5) FROM src LIMIT 1; - 'ebook' - > SELECT substr('Facebook', 5, 1) FROM src LIMIT 1; - 'b' diff --git a/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 b/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 deleted file mode 100644 index 9516b839a3956..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 b/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 deleted file mode 100644 index 7f6c057b5eebd..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 +++ /dev/null @@ -1 +0,0 @@ - \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a b/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a deleted file mode 100644 index 042ca15005869..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a +++ /dev/null @@ -1 +0,0 @@ -CDEF CDEF CDEFG CDEFG ABC ABC BC C ABC BC A A A \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 b/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 deleted file mode 100644 index 85346dc923b4a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 +++ /dev/null @@ -1 +0,0 @@ -A AB ABC ABC A AB ABC ABC B BC BC BC C C C C \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 b/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 deleted file mode 100644 index 9d607f4d84214..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 +++ /dev/null @@ -1 +0,0 @@ -C C C C B BC BC BC A AB ABC ABC \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 b/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 deleted file mode 100644 index 042ca15005869..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 +++ /dev/null @@ -1 +0,0 @@ -CDEF CDEF CDEFG CDEFG ABC ABC BC C ABC BC A A A \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade b/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade deleted file mode 100644 index c14d8d4279c5f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL NULL A AB ABC ABC A AB ABC ABC B BC BC BC C C C C C C C C B BC BC BC A AB ABC ABC diff --git a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d b/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d deleted file mode 100644 index 94a57d86c88fd..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d +++ /dev/null @@ -1 +0,0 @@ -玩 玩 玩玩玩 abc 玩玩玩 diff --git a/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece b/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece index b0ade52e69a9d..6a2376d85e7bc 100644 --- a/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece +++ b/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece @@ -1 +1 @@ -substring(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstring(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len \ No newline at end of file +substring(str, pos[, len]) - returns the substring of str that starts at pos and is of length len orsubstring(bin, pos[, len]) - returns the slice of byte array that starts at pos and is of length len diff --git a/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 b/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 index 1d61dc4518087..32941e727ce38 100644 --- a/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 +++ b/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 @@ -7,4 +7,4 @@ Example: > SELECT substring('Facebook', -5) FROM src LIMIT 1; 'ebook' > SELECT substring('Facebook', 5, 1) FROM src LIMIT 1; - 'b' \ No newline at end of file + 'b' diff --git a/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 index 0f4a4dce9dd3d..1ad8be1242f00 100644 --- a/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 +++ b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 @@ -1 +1 @@ -a - b - Returns the difference a-b \ No newline at end of file +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 index 0f4a4dce9dd3d..1ad8be1242f00 100644 --- a/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 +++ b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 @@ -1 +1 @@ -a - b - Returns the difference a-b \ No newline at end of file +a - b - Returns the difference a-b diff --git a/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 +++ b/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b +++ b/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 +++ b/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b index 744935026110c..3c8fbd8c4d900 100644 --- a/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b +++ b/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b @@ -1 +1 @@ -sum(x) - Returns the sum of a set of numbers \ No newline at end of file +sum(x) - Returns the sum of a set of numbers diff --git a/sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 rename to sql/hive/src/test/resources/golden/udf_tan-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 deleted file mode 100644 index c91072e69740b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 +++ /dev/null @@ -1 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 deleted file mode 100644 index 4cc0d466f00f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 +++ /dev/null @@ -1,4 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) -Example: - > SELECT tan(0) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..3ac5f19b2f2f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-1-c21aa640b4edabf6613dd705d029c878 @@ -0,0 +1 @@ +tan(x) - returns the tangent of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..f26ecfe4a29d3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-2-a8c8eaa832aa9a4345b2fb9cd5e1d505 @@ -0,0 +1,4 @@ +tan(x) - returns the tangent of x (x is in radians) +Example: + > SELECT tan(0) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 deleted file mode 100644 index 04ec33c11c3b4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 +++ /dev/null @@ -1 +0,0 @@ -1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 b/sql/hive/src/test/resources/golden/udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 rename to sql/hive/src/test/resources/golden/udf_tan-3-8b46b68ff11c5fb05fb4fd7605895f0b diff --git a/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c b/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c new file mode 100644 index 0000000000000..e9f2fa411fa33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-4-769fde617744ccfaa29cefec81b8704c @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 diff --git a/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 deleted file mode 100644 index c91072e69740b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 +++ /dev/null @@ -1 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 deleted file mode 100644 index 4cc0d466f00f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 +++ /dev/null @@ -1,4 +0,0 @@ -tan(x) - returns the tangent of x (x is in radians) -Example: - > SELECT tan(0) FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..3ac5f19b2f2f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-5-c21aa640b4edabf6613dd705d029c878 @@ -0,0 +1 @@ +tan(x) - returns the tangent of x (x is in radians) diff --git a/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 deleted file mode 100644 index fe3a0735d98b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 +++ /dev/null @@ -1 +0,0 @@ -NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..f26ecfe4a29d3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-6-a8c8eaa832aa9a4345b2fb9cd5e1d505 @@ -0,0 +1,4 @@ +tan(x) - returns the tangent of x (x is in radians) +Example: + > SELECT tan(0) FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 deleted file mode 100644 index 04ec33c11c3b4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 +++ /dev/null @@ -1 +0,0 @@ -1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 b/sql/hive/src/test/resources/golden/udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 rename to sql/hive/src/test/resources/golden/udf_tan-7-8b46b68ff11c5fb05fb4fd7605895f0b diff --git a/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c b/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c new file mode 100644 index 0000000000000..e9f2fa411fa33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-8-769fde617744ccfaa29cefec81b8704c @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 diff --git a/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad b/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad deleted file mode 100644 index 89617d8af0085..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION testlength 'org.apache.hadoop.hive.ql.udf.UDFTestLength') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 b/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 deleted file mode 100644 index 6d967660b3d41..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 +++ /dev/null @@ -1,10 +0,0 @@ -ABSTRACT SYNTAX TREE: - (TOK_CREATEFUNCTION testlength2 'org.apache.hadoop.hive.ql.udf.UDFTestLength2') - -STAGE DEPENDENCIES: - Stage-0 is a root stage - -STAGE PLANS: - Stage: Stage-0 - - diff --git a/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d index 1b13d42908539..898364b6d3e0b 100644 --- a/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d +++ b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d @@ -1 +1 @@ -There is no documentation for function 'tinyint' \ No newline at end of file +There is no documentation for function 'tinyint' diff --git a/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 index 1b13d42908539..898364b6d3e0b 100644 --- a/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 +++ b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 @@ -1 +1 @@ -There is no documentation for function 'tinyint' \ No newline at end of file +There is no documentation for function 'tinyint' diff --git a/sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 b/sql/hive/src/test/resources/golden/udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 rename to sql/hive/src/test/resources/golden/udf_to_byte-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 b/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/sql/hive/src/test/resources/golden/udf_to_byte-1-94eb069fb446b7758f7e06386486bec9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 rename to sql/hive/src/test/resources/golden/udf_to_byte-1-94eb069fb446b7758f7e06386486bec9 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 b/sql/hive/src/test/resources/golden/udf_to_byte-2-233102b562824cf38010868478e91e1 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 rename to sql/hive/src/test/resources/golden/udf_to_byte-2-233102b562824cf38010868478e91e1 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc b/sql/hive/src/test/resources/golden/udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc rename to sql/hive/src/test/resources/golden/udf_to_byte-3-5dc0e4c21764683d98700860d2c8ab31 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 b/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 deleted file mode 100644 index 3a2e3f4984a0e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 +++ /dev/null @@ -1 +0,0 @@ --1 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b b/sql/hive/src/test/resources/golden/udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b rename to sql/hive/src/test/resources/golden/udf_to_byte-4-dafb27507b4d30fd2231680f9ea80c82 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 rename to sql/hive/src/test/resources/golden/udf_to_byte-5-eb6600cd2260e8e75253e7844c0d7dc2 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 b/sql/hive/src/test/resources/golden/udf_to_byte-6-489cd2d26b9efde2cdbff19254289371 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 rename to sql/hive/src/test/resources/golden/udf_to_byte-6-489cd2d26b9efde2cdbff19254289371 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 b/sql/hive/src/test/resources/golden/udf_to_byte-7-1eaba393f93af1763dd761172fb78d52 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 rename to sql/hive/src/test/resources/golden/udf_to_byte-7-1eaba393f93af1763dd761172fb78d52 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 b/sql/hive/src/test/resources/golden/udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 rename to sql/hive/src/test/resources/golden/udf_to_byte-8-8fe36cf8fba87514744a89fe50414f79 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 b/sql/hive/src/test/resources/golden/udf_to_byte-9-322163c32973ccc3a5168463db7a8589 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 rename to sql/hive/src/test/resources/golden/udf_to_byte-9-322163c32973ccc3a5168463db7a8589 diff --git a/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 b/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 index 54a99d2a94ef4..caf10b953f895 100644 --- a/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 +++ b/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 @@ -1 +1 @@ -to_date(expr) - Extracts the date part of the date or datetime expression expr \ No newline at end of file +to_date(expr) - Extracts the date part of the date or datetime expression expr diff --git a/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e b/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e index 81ddc59fcb531..d9cc4956e5f84 100644 --- a/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e +++ b/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e @@ -1,4 +1,4 @@ to_date(expr) - Extracts the date part of the date or datetime expression expr Example: - > SELECT to_date('2009-30-07 04:17:52') FROM src LIMIT 1; - '2009-30-07' \ No newline at end of file + > SELECT to_date('2009-07-30 04:17:52') FROM src LIMIT 1; + '2009-07-30' diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe deleted file mode 100644 index 319fde05380bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe +++ /dev/null @@ -1 +0,0 @@ --7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f deleted file mode 100644 index 8c1c4fe62b6c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f +++ /dev/null @@ -1 +0,0 @@ --18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 deleted file mode 100644 index 1b650de78904f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 +++ /dev/null @@ -1 +0,0 @@ --129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 deleted file mode 100644 index 3a3bd0df03b5b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 +++ /dev/null @@ -1 +0,0 @@ --1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 deleted file mode 100644 index 38f7ad5afa0ab..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 +++ /dev/null @@ -1 +0,0 @@ --3.140000104904175 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 deleted file mode 100644 index f45d1f04dc920..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 +++ /dev/null @@ -1 +0,0 @@ --38.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 deleted file mode 100644 index 319fde05380bc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 +++ /dev/null @@ -1 +0,0 @@ --7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce deleted file mode 100644 index 8c1c4fe62b6c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce +++ /dev/null @@ -1 +0,0 @@ --18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 deleted file mode 100644 index 1b650de78904f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 +++ /dev/null @@ -1 +0,0 @@ --129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 deleted file mode 100644 index 3a3bd0df03b5b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 +++ /dev/null @@ -1 +0,0 @@ --1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c deleted file mode 100644 index f45d1f04dc920..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c +++ /dev/null @@ -1 +0,0 @@ --38.14 diff --git a/sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb b/sql/hive/src/test/resources/golden/udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb rename to sql/hive/src/test/resources/golden/udf_to_long-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 b/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/sql/hive/src/test/resources/golden/udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 rename to sql/hive/src/test/resources/golden/udf_to_long-1-8aeb3ba62d4a0cecfff363741e8042f6 diff --git a/sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 b/sql/hive/src/test/resources/golden/udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947 similarity index 100% rename from sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 rename to sql/hive/src/test/resources/golden/udf_to_long-2-7e8fa1ae8d00a121ec14941a48d24947 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 b/sql/hive/src/test/resources/golden/udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 rename to sql/hive/src/test/resources/golden/udf_to_long-3-6e5936fba8e7486beb9ab998548bbe9b diff --git a/sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 b/sql/hive/src/test/resources/golden/udf_to_long-4-8c284b082a256abf0426d4f6f1971703 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 rename to sql/hive/src/test/resources/golden/udf_to_long-4-8c284b082a256abf0426d4f6f1971703 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 b/sql/hive/src/test/resources/golden/udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 rename to sql/hive/src/test/resources/golden/udf_to_long-5-6bb29b93f6b0f1427ba93efb4e78810a diff --git a/sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d b/sql/hive/src/test/resources/golden/udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d rename to sql/hive/src/test/resources/golden/udf_to_long-6-290b5a4ce01563482e81b3b532ebf9db diff --git a/sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 b/sql/hive/src/test/resources/golden/udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 rename to sql/hive/src/test/resources/golden/udf_to_long-7-da20f84586dac3e50ee9d5b9078f44db diff --git a/sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e b/sql/hive/src/test/resources/golden/udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e rename to sql/hive/src/test/resources/golden/udf_to_long-8-90f068b4b6275bdd1c4c431fb7fa90e2 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 b/sql/hive/src/test/resources/golden/udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 rename to sql/hive/src/test/resources/golden/udf_to_long-9-cc8b79539085fe0e00f672b562c51cd0 diff --git a/sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_to_short-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a b/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a deleted file mode 100644 index 7951defec192a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a +++ /dev/null @@ -1 +0,0 @@ -NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b b/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a b/sql/hive/src/test/resources/golden/udf_to_short-1-94f15fe043839493107058a06a210cf7 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a rename to sql/hive/src/test/resources/golden/udf_to_short-1-94f15fe043839493107058a06a210cf7 diff --git a/sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 b/sql/hive/src/test/resources/golden/udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570 similarity index 100% rename from sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 rename to sql/hive/src/test/resources/golden/udf_to_short-2-981e4cb6654fde7eb4634c7ad72f8570 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 b/sql/hive/src/test/resources/golden/udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 rename to sql/hive/src/test/resources/golden/udf_to_short-3-b259ee30ecf279bb4ad12d1515ca2767 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce b/sql/hive/src/test/resources/golden/udf_to_short-4-40ffb132d5641645e2b8043dc056fb0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce rename to sql/hive/src/test/resources/golden/udf_to_short-4-40ffb132d5641645e2b8043dc056fb0 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 b/sql/hive/src/test/resources/golden/udf_to_short-5-5d6c46b0154d1073c035a79dbf612479 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 rename to sql/hive/src/test/resources/golden/udf_to_short-5-5d6c46b0154d1073c035a79dbf612479 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 b/sql/hive/src/test/resources/golden/udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 rename to sql/hive/src/test/resources/golden/udf_to_short-6-6561b41835a21f973cbbc2dd80eef87f diff --git a/sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 b/sql/hive/src/test/resources/golden/udf_to_short-7-9f83813005b639a23901ca6ff87ff473 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 rename to sql/hive/src/test/resources/golden/udf_to_short-7-9f83813005b639a23901ca6ff87ff473 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d b/sql/hive/src/test/resources/golden/udf_to_short-8-885656e165feb3a674cf636dbf08716c similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d rename to sql/hive/src/test/resources/golden/udf_to_short-8-885656e165feb3a674cf636dbf08716c diff --git a/sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 b/sql/hive/src/test/resources/golden/udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 rename to sql/hive/src/test/resources/golden/udf_to_short-9-750382fa1a1b3ed5dca0d549d3a68996 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc b/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc deleted file mode 100644 index 7951defec192a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc +++ /dev/null @@ -1 +0,0 @@ -NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 b/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 deleted file mode 100644 index ef2f5130b8575..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 +++ /dev/null @@ -1 +0,0 @@ -TRUE diff --git a/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 b/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 deleted file mode 100644 index d00491fd7e5bb..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 b/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 deleted file mode 100644 index 6cc8a61f8f6ad..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 +++ /dev/null @@ -1 +0,0 @@ --18 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 b/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 deleted file mode 100644 index 9828ff22b667b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 +++ /dev/null @@ -1 +0,0 @@ --129 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 b/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 deleted file mode 100644 index 450a6125550e5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 +++ /dev/null @@ -1 +0,0 @@ --1025 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 b/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 b/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea b/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea deleted file mode 100644 index 01e913dbfe725..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea +++ /dev/null @@ -1 +0,0 @@ --3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b b/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b deleted file mode 100644 index bc56c4d89448a..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b +++ /dev/null @@ -1 +0,0 @@ -Foo diff --git a/sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 rename to sql/hive/src/test/resources/golden/udf_translate-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 b/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 deleted file mode 100644 index 4255dc76f501e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 +++ /dev/null @@ -1 +0,0 @@ -translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 b/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 new file mode 100644 index 0000000000000..e92c71fd3ccf9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-1-7fe940a2b26fa19a3cfee39e56fb1241 @@ -0,0 +1 @@ +translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string diff --git a/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 deleted file mode 100644 index e7beead53b399..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 +++ /dev/null @@ -1,13 +0,0 @@ -translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string -translate(string input, string from, string to) is an equivalent function to translate in PostGreSQL. It works on a character by character basis on the input string (first parameter). A character in the input is checked for presence in the from string (second parameter). If a match happens, the character from to string (third parameter) which appears at the same index as the character in from string is obtained. This character is emitted in the output string instead of the original character from the input string. If the to string is shorter than the from string, there may not be a character present at the same index in the to string. In such a case, nothing is emitted for the original character and it's deleted from the output string. -For example, - -translate('abcdef', 'adc', '19') returns '1b9ef' replacing 'a' with '1', 'd' with '9' and removing 'c' from the input string - -translate('a b c d', ' ', '') return 'abcd' removing all spaces from the input string - -If the same character is present multiple times in the input string, the first occurence of the character is the one that's considered for matching. However, it is not recommended to have the same character more than once in the from string since it's not required and adds to confusion. - -For example, - -translate('abcdef', 'ada', '192') returns '1bc9ef' replaces 'a' with '1' and 'd' with '9' ignoring the second occurence of 'a' in the from string mapping it to '2' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 b/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 deleted file mode 100644 index 7d44692e4f7c0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 +++ /dev/null @@ -1 +0,0 @@ -12cd 12cd \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae b/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae new file mode 100644 index 0000000000000..ae8343d33bc11 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-10-ca7c17e78c6a3d4e19dbd66622a87eae @@ -0,0 +1 @@ +NULL NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a b/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a new file mode 100644 index 0000000000000..2e2f35d06c042 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-11-40c4e7adff4dde739d7797d212892c5a @@ -0,0 +1 @@ +12cd 12cd diff --git a/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b b/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b deleted file mode 100644 index 90e0d872f8bfc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b +++ /dev/null @@ -1 +0,0 @@ -123d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 b/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 deleted file mode 100644 index 0770d02cb2303..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 +++ /dev/null @@ -1 +0,0 @@ -Ãbcd \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf b/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf new file mode 100644 index 0000000000000..36284d8236d0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-12-d81fd3267ec96cff31079142bf5d49bf @@ -0,0 +1 @@ +123d diff --git a/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 b/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 new file mode 100644 index 0000000000000..0208f1f4980c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-13-26085a3eba1a1b34684ec4e6c1723527 @@ -0,0 +1 @@ +Ãbcd diff --git a/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 b/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 new file mode 100644 index 0000000000000..9ced4ee32cf0b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8 @@ -0,0 +1,13 @@ +translate(input, from, to) - translates the input string by replacing the characters present in the from string with the corresponding characters in the to string +translate(string input, string from, string to) is an equivalent function to translate in PostGreSQL. It works on a character by character basis on the input string (first parameter). A character in the input is checked for presence in the from string (second parameter). If a match happens, the character from to string (third parameter) which appears at the same index as the character in from string is obtained. This character is emitted in the output string instead of the original character from the input string. If the to string is shorter than the from string, there may not be a character present at the same index in the to string. In such a case, nothing is emitted for the original character and it's deleted from the output string. +For example, + +translate('abcdef', 'adc', '19') returns '1b9ef' replacing 'a' with '1', 'd' with '9' and removing 'c' from the input string + +translate('a b c d', ' ', '') return 'abcd' removing all spaces from the input string + +If the same character is present multiple times in the input string, the first occurence of the character is the one that's considered for matching. However, it is not recommended to have the same character more than once in the from string since it's not required and adds to confusion. + +For example, + +translate('abcdef', 'ada', '192') returns '1bc9ef' replaces 'a' with '1' and 'd' with '9' ignoring the second occurence of 'a' in the from string mapping it to '2' diff --git a/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a b/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/udf_translate-3-42aba80bf1913dd7c64545831f476c58 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 rename to sql/hive/src/test/resources/golden/udf_translate-3-42aba80bf1913dd7c64545831f476c58 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa rename to sql/hive/src/test/resources/golden/udf_translate-4-20904c8be8fed5cbd2d66ead6248a60a diff --git a/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba b/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 rename to sql/hive/src/test/resources/golden/udf_translate-5-5d4abaf86254bacaa545c769bd7e50ba diff --git a/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 b/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b b/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b deleted file mode 100644 index 0a54ee40ecb13..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b +++ /dev/null @@ -1 +0,0 @@ -12cd 12d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/udf_translate-6-f2637240d227f9732d3db76f2e9d3a59 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad rename to sql/hive/src/test/resources/golden/udf_translate-6-f2637240d227f9732d3db76f2e9d3a59 diff --git a/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 b/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 deleted file mode 100644 index 0a54ee40ecb13..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 +++ /dev/null @@ -1 +0,0 @@ -12cd 12d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 b/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 new file mode 100644 index 0000000000000..fbf71ff5ad9eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-7-f8de3ab54db5d6a44fddb542b3d99704 @@ -0,0 +1 @@ +12cd 12d diff --git a/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 b/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 new file mode 100644 index 0000000000000..fbf71ff5ad9eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-8-1747ed8fbb4ef889df3db937ee51e2b0 @@ -0,0 +1 @@ +12cd 12d diff --git a/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 b/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 deleted file mode 100644 index 50c123df9d1d3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 +++ /dev/null @@ -1 +0,0 @@ -1bc \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 b/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 deleted file mode 100644 index b98d7e1c34024..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 b/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 new file mode 100644 index 0000000000000..bb456baa0a608 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-9-f1bd0dd5226ee632db3c72c5fc2aaeb0 @@ -0,0 +1 @@ +1bc diff --git a/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f b/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f index f0620a190ae4e..598aeb205875b 100644 --- a/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f +++ b/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f @@ -1 +1 @@ -trim(str) - Removes the leading and trailing space characters from str \ No newline at end of file +trim(str) - Removes the leading and trailing space characters from str diff --git a/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 b/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 index 49a13dd64b286..b3ec336a877ee 100644 --- a/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 +++ b/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 @@ -1,4 +1,4 @@ trim(str) - Removes the leading and trailing space characters from str Example: > SELECT trim(' facebook ') FROM src LIMIT 1; - 'facebook' \ No newline at end of file + 'facebook' diff --git a/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d b/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d index 057c231dccf0e..66412c3ba77a4 100644 --- a/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d +++ b/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d @@ -1 +1 @@ -ucase(str) - Returns str with all characters changed to uppercase \ No newline at end of file +ucase(str) - Returns str with all characters changed to uppercase diff --git a/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f b/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f index 128d1e21fae83..814e4f0d39dcf 100644 --- a/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f +++ b/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f @@ -2,4 +2,4 @@ ucase(str) - Returns str with all characters changed to uppercase Synonyms: upper Example: > SELECT ucase('Facebook') FROM src LIMIT 1; - 'FACEBOOK' \ No newline at end of file + 'FACEBOOK' diff --git a/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 b/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 deleted file mode 100644 index 44b2a42cc26c5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 +++ /dev/null @@ -1 +0,0 @@ -unhex(str) - Converts hexadecimal argument to binary diff --git a/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 b/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 deleted file mode 100644 index 97af3b812a429..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 +++ /dev/null @@ -1,14 +0,0 @@ -unhex(str) - Converts hexadecimal argument to binary -Performs the inverse operation of HEX(str). That is, it interprets -each pair of hexadecimal digits in the argument as a number and -converts it to the byte representation of the number. The -resulting characters are returned as a binary string. - -Example: -> SELECT DECODE(UNHEX('4D7953514C'), 'UTF-8') from src limit 1; -'MySQL' - -The characters in the argument string must be legal hexadecimal -digits: '0' .. '9', 'A' .. 'F', 'a' .. 'f'. If UNHEX() encounters -any nonhexadecimal digits in the argument, it returns NULL. Also, -if there are an odd number of characters a leading 0 is appended. diff --git a/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 b/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 deleted file mode 100644 index b4a6f2b692227..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 +++ /dev/null @@ -1 +0,0 @@ -MySQL 1267 a -4 diff --git a/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 b/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 deleted file mode 100644 index 3a67adaf0a9a8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 +++ /dev/null @@ -1 +0,0 @@ -NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c b/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c deleted file mode 100644 index 3f3bb2b7ce02e..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c +++ /dev/null @@ -1 +0,0 @@ -create_union(tag, obj1, obj2, obj3, ...) - Creates a union with the object for given tag \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 b/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 deleted file mode 100644 index 748ed97055401..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 +++ /dev/null @@ -1,4 +0,0 @@ -create_union(tag, obj1, obj2, obj3, ...) - Creates a union with the object for given tag -Example: - > SELECT create_union(1, 1, "one") FROM src LIMIT 1; - one \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 b/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 b/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 deleted file mode 100644 index 4b9ae5402aa53..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 +++ /dev/null @@ -1,2 +0,0 @@ -{0:238} {1:"val_238"} {1:{"col1":2,"col2":"b"}} -{0:86} {0:2.0} {1:{"col1":2,"col2":"b"}} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb b/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-0-d555c8cd733572bfa8cd3362da9480cb rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-1-d555c8cd733572bfa8cd3362da9480cb diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-28c40e51e55bed62693e626efda5d9c5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-1-8a9dbadae706047715cf5f903ff4a724 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-2-8a9dbadae706047715cf5f903ff4a724 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-3-28c40e51e55bed62693e626efda5d9c5 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed b/sql/hive/src/test/resources/golden/udf_unix_timestamp-3-732b21d386f2002b87eaf02d0b9951ed deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-4-6059ff48788d0fb8317fd331172ecea9 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-4-b2e42ebb75cecf09961d36587797f6d0 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-5-b2e42ebb75cecf09961d36587797f6d0 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-5-31243f5cb64356425b9f95ba011ac9d6 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-6-31243f5cb64356425b9f95ba011ac9d6 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 b/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-6-9b0f20bde1aaf9102b67a5498b167f31 rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-7-9b0f20bde1aaf9102b67a5498b167f31 diff --git a/sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d b/sql/hive/src/test/resources/golden/udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d similarity index 100% rename from sql/hive/src/test/resources/golden/udf_unix_timestamp-7-47f433ff6ccce4c666440cc1a228a96d rename to sql/hive/src/test/resources/golden/udf_unix_timestamp-8-47f433ff6ccce4c666440cc1a228a96d diff --git a/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead b/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead index 2e239cca5b860..40014886d9646 100644 --- a/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead +++ b/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead @@ -1 +1 @@ -upper(str) - Returns str with all characters changed to uppercase \ No newline at end of file +upper(str) - Returns str with all characters changed to uppercase diff --git a/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d b/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d index 673307e284b2e..0fb1bf1823d84 100644 --- a/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d +++ b/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d @@ -2,4 +2,4 @@ upper(str) - Returns str with all characters changed to uppercase Synonyms: ucase Example: > SELECT upper('Facebook') FROM src LIMIT 1; - 'FACEBOOK' \ No newline at end of file + 'FACEBOOK' diff --git a/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de index 9dbf241e28107..79e293fd4ed52 100644 --- a/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de +++ b/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de @@ -1 +1 @@ -var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file +var_pop(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 index 234a65bd8255d..4b4de5db812e9 100644 --- a/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 +++ b/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 @@ -1,2 +1,2 @@ var_pop(x) - Returns the variance of a set of numbers -Synonyms: variance \ No newline at end of file +Synonyms: variance diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 +++ b/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b +++ b/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 +++ b/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b index 673deeb031b44..42a442f7a2ded 100644 --- a/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b +++ b/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b @@ -1 +1 @@ -var_samp(x) - Returns the sample variance of a set of numbers \ No newline at end of file +var_samp(x) - Returns the sample variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 index f58ae0769d5a9..260300b7d3bd9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 +++ b/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 @@ -1 +1 @@ -variance(x) - Returns the variance of a set of numbers \ No newline at end of file +variance(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a index e17528e607aba..b14cf43bc0a95 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a +++ b/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a @@ -1,2 +1,2 @@ variance(x) - Returns the variance of a set of numbers -Synonyms: var_pop \ No newline at end of file +Synonyms: var_pop diff --git a/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de index 9dbf241e28107..79e293fd4ed52 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de +++ b/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de @@ -1 +1 @@ -var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file +var_pop(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 index 234a65bd8255d..4b4de5db812e9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 +++ b/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 @@ -1,2 +1,2 @@ var_pop(x) - Returns the variance of a set of numbers -Synonyms: variance \ No newline at end of file +Synonyms: variance diff --git a/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 index f58ae0769d5a9..260300b7d3bd9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 +++ b/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 @@ -1 +1 @@ -variance(x) - Returns the variance of a set of numbers \ No newline at end of file +variance(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a index e17528e607aba..b14cf43bc0a95 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a +++ b/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a @@ -1,2 +1,2 @@ variance(x) - Returns the variance of a set of numbers -Synonyms: var_pop \ No newline at end of file +Synonyms: var_pop diff --git a/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de index 9dbf241e28107..79e293fd4ed52 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de +++ b/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de @@ -1 +1 @@ -var_pop(x) - Returns the variance of a set of numbers \ No newline at end of file +var_pop(x) - Returns the variance of a set of numbers diff --git a/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 index 234a65bd8255d..4b4de5db812e9 100644 --- a/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 +++ b/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 @@ -1,2 +1,2 @@ var_pop(x) - Returns the variance of a set of numbers -Synonyms: variance \ No newline at end of file +Synonyms: variance diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 b/sql/hive/src/test/resources/golden/udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 rename to sql/hive/src/test/resources/golden/udf_weekofyear-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 b/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 deleted file mode 100644 index 29e3b370b03b8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 +++ /dev/null @@ -1 +0,0 @@ -weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd b/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd deleted file mode 100644 index c7939a11937c5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd +++ /dev/null @@ -1,6 +0,0 @@ -weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. -Examples: - > SELECT weekofyear('2008-02-20') FROM src LIMIT 1; - 8 - > SELECT weekofyear('1980-12-31 12:59:59') FROM src LIMIT 1; - 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 b/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 new file mode 100644 index 0000000000000..dca04d210339a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_weekofyear-1-d6b4490b549a358be375511e39627dc2 @@ -0,0 +1 @@ +weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 b/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 deleted file mode 100644 index 20d1dc50ce1f7..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 +++ /dev/null @@ -1 +0,0 @@ -1 1 2 1 52 8 52 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd b/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd new file mode 100644 index 0000000000000..337257dabd8d1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_weekofyear-2-b7bbdfabe6054a66701250fd70065ddd @@ -0,0 +1,6 @@ +weekofyear(date) - Returns the week of the year of the given date. A week is considered to start on a Monday and week 1 is the first week with >3 days. +Examples: + > SELECT weekofyear('2008-02-20') FROM src LIMIT 1; + 8 + > SELECT weekofyear('1980-12-31 12:59:59') FROM src LIMIT 1; + 1 diff --git a/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 b/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 new file mode 100644 index 0000000000000..9d0163569f7b3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_weekofyear-3-d5dd3abb6c8c7046a85dd05f51126285 @@ -0,0 +1 @@ +1 1 2 1 52 8 52 1 diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 b/sql/hive/src/test/resources/golden/udf_when-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 rename to sql/hive/src/test/resources/golden/udf_when-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 b/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 deleted file mode 100644 index 2e36162dbaa33..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'when' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 b/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 deleted file mode 100644 index 2e36162dbaa33..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 +++ /dev/null @@ -1 +0,0 @@ -There is no documentation for function 'when' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 b/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 new file mode 100644 index 0000000000000..3626da8447ca0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-1-88b97c6722176393e9b3d089559d2d11 @@ -0,0 +1 @@ +There is no documentation for function 'when' diff --git a/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 b/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 new file mode 100644 index 0000000000000..3626da8447ca0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-2-7365d5fe39dc7a025c942dad8fb9f0d4 @@ -0,0 +1 @@ +There is no documentation for function 'when' diff --git a/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 b/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/udf_when-3-734890c41528b9d918db66b0582228a4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 rename to sql/hive/src/test/resources/golden/udf_when-3-734890c41528b9d918db66b0582228a4 diff --git a/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f b/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f deleted file mode 100644 index 6929fdca28eb8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f +++ /dev/null @@ -1 +0,0 @@ -2 9 14 NULL 24 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 b/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 new file mode 100644 index 0000000000000..b28b0050514f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-4-c57d6eb11efc29ce3a9c450488f3d750 @@ -0,0 +1 @@ +2 9 14 NULL 24 NULL diff --git a/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 b/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 new file mode 100644 index 0000000000000..db46b5a72aa80 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-5-6ed21e998c0fc32c39f6375136f55de6 @@ -0,0 +1 @@ +123 123.0 abcd diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 b/sql/hive/src/test/resources/golden/udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 rename to sql/hive/src/test/resources/golden/udf_xpath-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 b/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 deleted file mode 100644 index 18d9720653a5b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 +++ /dev/null @@ -1 +0,0 @@ -xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 b/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 deleted file mode 100644 index 3b092dafd32c2..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 +++ /dev/null @@ -1,8 +0,0 @@ -xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression -Example: - > SELECT xpath('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 - [] - > SELECT xpath('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 - ["b1","b2","b3"] - > SELECT xpath('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 - ["c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 b/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 new file mode 100644 index 0000000000000..d53b034f8fc6d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-1-622670bd1cbf4bf0cf04a553006e3c8 @@ -0,0 +1 @@ +xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 b/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 new file mode 100644 index 0000000000000..bff795649a7ea --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-2-5a6f10392053a4eabe62f5cabb073a71 @@ -0,0 +1,8 @@ +xpath(xml, xpath) - Returns a string array of values within xml nodes that match the xpath expression +Example: + > SELECT xpath('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 + [] + > SELECT xpath('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 + ["b1","b2","b3"] + > SELECT xpath('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 + ["c1","c2"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d b/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d deleted file mode 100644 index 0637a088a01e8..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d +++ /dev/null @@ -1 +0,0 @@ -[] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea b/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea deleted file mode 100644 index 8e8aff2387621..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea +++ /dev/null @@ -1 +0,0 @@ -["b1","b2","b3","c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 b/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 new file mode 100644 index 0000000000000..fe51488c7066f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-3-f0b9adf99c68290c86d0f40f45111e18 @@ -0,0 +1 @@ +[] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a b/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a deleted file mode 100644 index e518b1f37e124..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a +++ /dev/null @@ -1 +0,0 @@ -["b1","b2","b3"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e b/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e new file mode 100644 index 0000000000000..96e245a7eb37c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-4-6d5e7ed902ac051f0cdba43d7a30434e @@ -0,0 +1 @@ +["b1","b2","b3","c1","c2"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 b/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 deleted file mode 100644 index 45615b772ba35..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 +++ /dev/null @@ -1 +0,0 @@ -["c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 b/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 new file mode 100644 index 0000000000000..44def14ea2c68 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-5-b66a64f91dd812fda2859863855988b5 @@ -0,0 +1 @@ +["b1","b2","b3"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f b/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f deleted file mode 100644 index 327bf3a1e76cc..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f +++ /dev/null @@ -1 +0,0 @@ -["b1","c1"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a b/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a new file mode 100644 index 0000000000000..6a97cc328aa70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-6-d4f95ebc0057639138900722c74ee17a @@ -0,0 +1 @@ +["c1","c2"] diff --git a/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 b/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 new file mode 100644 index 0000000000000..8b4abae5acff8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-7-6b8fceac3bd654662f067da60670e1d9 @@ -0,0 +1 @@ +["b1","c1"] diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 deleted file mode 100644 index 134e9e99b7423..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 +++ /dev/null @@ -1 +0,0 @@ -xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 new file mode 100644 index 0000000000000..d47e3fe7d34a1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-8b2f3f8132bfc8344e243cdaf45eb371 @@ -0,0 +1 @@ +xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 deleted file mode 100644 index 624889cc0ac67..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 +++ /dev/null @@ -1,6 +0,0 @@ -xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression -Example: - > SELECT xpath_boolean('1','a/b') FROM src LIMIT 1; - true - > SELECT xpath_boolean('1','a/b = 2') FROM src LIMIT 1; - false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 new file mode 100644 index 0000000000000..4f3c2652aad11 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-9e935539c9a3f3a118145096e7f978c4 @@ -0,0 +1,6 @@ +xpath_boolean(xml, xpath) - Evaluates a boolean xpath expression +Example: + > SELECT xpath_boolean('1','a/b') FROM src LIMIT 1; + true + > SELECT xpath_boolean('1','a/b = 2') FROM src LIMIT 1; + false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-3-9b97a08303a7a89e7575687f6d7ba435 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-4-284ce7d6fc6850ca853111145784286b diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-5-95a0dfa897ba9050ad751a78aeb72f3d diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f deleted file mode 100644 index 02e4a84d62c4b..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f +++ /dev/null @@ -1 +0,0 @@ -false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-6-796c717c9d52d3efa374d12fe65259e6 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 deleted file mode 100644 index f32a5804e292d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 +++ /dev/null @@ -1 +0,0 @@ -true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-cc3ae9a7505e04a2e9b950442a81a559 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f rename to sql/hive/src/test/resources/golden/udf_xpath_boolean-8-62a1af222d4e12c59cef71b979c6e58 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 b/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 deleted file mode 100644 index b56f4d00a8a74..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 +++ /dev/null @@ -1 +0,0 @@ -xpath_number(xml, xpath) - Returns a double value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_xpath_double-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 b/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 deleted file mode 100644 index b6b6227174647..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 +++ /dev/null @@ -1,5 +0,0 @@ -xpath_number(xml, xpath) - Returns a double value that matches the xpath expression -Synonyms: xpath_double -Example: - > SELECT xpath_number('12','sum(a/b)') FROM src LIMIT 1; - 3.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 b/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 new file mode 100644 index 0000000000000..a9cb2ecb60e28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-1-39199612969071d58b24034a2d17ca67 @@ -0,0 +1 @@ +xpath_number(xml, xpath) - Returns a double value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 b/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 new file mode 100644 index 0000000000000..fe6b09a7d14cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-10-ad0f9117f6f52590d38e619e998a6648 @@ -0,0 +1 @@ +15.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 b/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 deleted file mode 100644 index 7104585f9a439..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 +++ /dev/null @@ -1 +0,0 @@ -7.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 b/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 deleted file mode 100644 index 6e6366051638f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 +++ /dev/null @@ -1 +0,0 @@ -5.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 b/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 new file mode 100644 index 0000000000000..4fedf1d20e157 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-11-bf1f96ce71879c17d91ac9df44c36d29 @@ -0,0 +1 @@ +7.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb b/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb new file mode 100644 index 0000000000000..819e07a22435f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-12-9621353ce5c583ca34216d357c5bb0eb @@ -0,0 +1 @@ +5.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 b/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 new file mode 100644 index 0000000000000..0f303917eb1bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-2-2d5ceab0a14d6e837ed153e1a2235bb2 @@ -0,0 +1,5 @@ +xpath_number(xml, xpath) - Returns a double value that matches the xpath expression +Synonyms: xpath_double +Example: + > SELECT xpath_number('12','sum(a/b)') FROM src LIMIT 1; + 3.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 b/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 deleted file mode 100644 index 5b6de7db2f9f3..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 +++ /dev/null @@ -1 +0,0 @@ -xpath_double(xml, xpath) - Returns a double value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 b/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 new file mode 100644 index 0000000000000..20ff7713a919d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-3-a4d22cea9dffaf26b485b3eb08963789 @@ -0,0 +1 @@ +xpath_double(xml, xpath) - Returns a double value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 b/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 deleted file mode 100644 index b8accbcae1e17..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 +++ /dev/null @@ -1,5 +0,0 @@ -xpath_double(xml, xpath) - Returns a double value that matches the xpath expression -Synonyms: xpath_number -Example: - > SELECT xpath_double('12','sum(a/b)') FROM src LIMIT 1; - 3.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c b/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 b/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 new file mode 100644 index 0000000000000..89c174f371b88 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-4-ea050c5b9e74d4b87b037236ef9e4fc2 @@ -0,0 +1,5 @@ +xpath_double(xml, xpath) - Returns a double value that matches the xpath expression +Synonyms: xpath_number +Example: + > SELECT xpath_double('12','sum(a/b)') FROM src LIMIT 1; + 3.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 b/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-5-1e0514d71f99da09e01a414a4e01d046 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 b/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 b/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 deleted file mode 100644 index 0a8ac8c629540..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 +++ /dev/null @@ -1 +0,0 @@ -8.0E19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 b/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-6-6a3985167fedd579f7bd5745133a3524 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c b/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 b/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 new file mode 100644 index 0000000000000..f65000a36a478 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-7-d97e93fb4b69522300f505e04b6674c8 @@ -0,0 +1 @@ +8.0E19 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d b/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 b/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-8-fce6cafa07b75c9843c1c1964e84fc10 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e b/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e deleted file mode 100644 index 3d3be3c32ed85..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e +++ /dev/null @@ -1 +0,0 @@ -15.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 b/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-9-f27361521c35bf4f6581dba4c185d550 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 b/sql/hive/src/test/resources/golden/udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 rename to sql/hive/src/test/resources/golden/udf_xpath_float-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 b/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 deleted file mode 100644 index ea4df0ae58e12..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 +++ /dev/null @@ -1 +0,0 @@ -xpath_float(xml, xpath) - Returns a float value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 b/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 new file mode 100644 index 0000000000000..b7dea42c78212 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-1-7483dafee0dc7334eecabba31977e791 @@ -0,0 +1 @@ +xpath_float(xml, xpath) - Returns a float value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a b/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a deleted file mode 100644 index 6bc4d4a46de72..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a +++ /dev/null @@ -1,4 +0,0 @@ -xpath_float(xml, xpath) - Returns a float value that matches the xpath expression -Example: - > SELECT xpath_float('12','sum(a/b)') FROM src LIMIT 1; - 3.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab b/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab new file mode 100644 index 0000000000000..819e07a22435f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-10-89ae28cf8e6b6f422d307a0085774cab @@ -0,0 +1 @@ +5.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 b/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a b/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a new file mode 100644 index 0000000000000..02b139e7a3610 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-2-f6ddff2daba006787aeb861ca6f9d37a @@ -0,0 +1,4 @@ +xpath_float(xml, xpath) - Returns a float value that matches the xpath expression +Example: + > SELECT xpath_float('12','sum(a/b)') FROM src LIMIT 1; + 3.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 b/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 deleted file mode 100644 index b955cad2a39a0..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 +++ /dev/null @@ -1 +0,0 @@ -NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce b/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-3-b743a9cb9f8688220e0a6346856f18ce @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 new file mode 100644 index 0000000000000..736991a138745 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6720ee0163b0f0ddd2ab72fa9ab935e3 @@ -0,0 +1 @@ +NaN diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 deleted file mode 100644 index 0a8ac8c629540..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 +++ /dev/null @@ -1 +0,0 @@ -8.0E19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f b/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f deleted file mode 100644 index 171538eb0b00f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f +++ /dev/null @@ -1 +0,0 @@ -0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 b/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 new file mode 100644 index 0000000000000..f65000a36a478 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-5-f5e1100f6e8de31081042413b4039fb2 @@ -0,0 +1 @@ +8.0E19 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 b/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 new file mode 100644 index 0000000000000..ba66466c2a0d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-6-5e8457209d15467af7f14c09dfadb894 @@ -0,0 +1 @@ +0.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf b/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf deleted file mode 100644 index 9f8e9b69a33f4..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf +++ /dev/null @@ -1 +0,0 @@ -1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 b/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-7-580ffe4dabef758c4fcb52050b315615 @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa b/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa deleted file mode 100644 index 3d3be3c32ed85..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa +++ /dev/null @@ -1 +0,0 @@ -15.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 b/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 deleted file mode 100644 index 7104585f9a439..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 +++ /dev/null @@ -1 +0,0 @@ -7.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 b/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 new file mode 100644 index 0000000000000..fe6b09a7d14cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-8-73cfa6fe399ca3e240b84b47ee4f8bc8 @@ -0,0 +1 @@ +15.0 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 b/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 deleted file mode 100644 index 6e6366051638f..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 +++ /dev/null @@ -1 +0,0 @@ -5.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a b/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a new file mode 100644 index 0000000000000..4fedf1d20e157 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-9-bf66b884da717d46f0b12c3b8cf8313a @@ -0,0 +1 @@ +7.0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 b/sql/hive/src/test/resources/golden/udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 rename to sql/hive/src/test/resources/golden/udf_xpath_int-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 b/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 deleted file mode 100644 index 67ab19492e405..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 +++ /dev/null @@ -1 +0,0 @@ -xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 b/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 deleted file mode 100644 index 33349b0b22ad9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 +++ /dev/null @@ -1,4 +0,0 @@ -xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression -Example: - > SELECT xpath_int('12','sum(a/b)') FROM src LIMIT 1; - 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 b/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 new file mode 100644 index 0000000000000..000f90d69e7f7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-1-a9ed102a70e7e9d505be1555f0c7eb12 @@ -0,0 +1 @@ +xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a b/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-10-ea25feb474c40c6985152093d4dbb13a @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 b/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 new file mode 100644 index 0000000000000..f969b9a48d10f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-2-26ddf7e46a66065909e9e547f0459910 @@ -0,0 +1,4 @@ +xpath_int(xml, xpath) - Returns an integer value that matches the xpath expression +Example: + > SELECT xpath_int('12','sum(a/b)') FROM src LIMIT 1; + 3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba b/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 rename to sql/hive/src/test/resources/golden/udf_xpath_int-3-a04ed6832ab542d6ee5903039511a826 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba b/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 b/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 deleted file mode 100644 index 55a89f5564d21..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 +++ /dev/null @@ -1 +0,0 @@ -2147483647 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f similarity index 100% rename from sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe rename to sql/hive/src/test/resources/golden/udf_xpath_int-4-bf5a4dbb7a98abc91111a3798b56809f diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 b/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 b/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 new file mode 100644 index 0000000000000..a51fa7d1efef6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-5-f49db0ecb889722ec68d1753c084b0e1 @@ -0,0 +1 @@ +2147483647 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a b/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_int-6-ac509f06f01c02924adef220404fc515 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_int-6-ac509f06f01c02924adef220404fc515 diff --git a/sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/udf_xpath_int-7-87ff12d650afb8f511d906778450fae7 similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d rename to sql/hive/src/test/resources/golden/udf_xpath_int-7-87ff12d650afb8f511d906778450fae7 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 b/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 deleted file mode 100644 index 3f10ffe7a4c47..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 +++ /dev/null @@ -1 +0,0 @@ -15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 b/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-8-a175811eca252aa77c16a44fbb0ee7b2 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 b/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 deleted file mode 100644 index c7930257dfef5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 +++ /dev/null @@ -1 +0,0 @@ -7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 b/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 new file mode 100644 index 0000000000000..7f8f011eb73d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-9-7da88f589199f5ca873780fb22614573 @@ -0,0 +1 @@ +7 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 b/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 deleted file mode 100644 index 7813681f5b41c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 +++ /dev/null @@ -1 +0,0 @@ -5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_long-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b b/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b deleted file mode 100644 index 748fd827e8545..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b +++ /dev/null @@ -1 +0,0 @@ -xpath_long(xml, xpath) - Returns a long value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 b/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 deleted file mode 100644 index 65241b6ce9b64..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 +++ /dev/null @@ -1,4 +0,0 @@ -xpath_long(xml, xpath) - Returns a long value that matches the xpath expression -Example: - > SELECT xpath_long('12','sum(a/b)') FROM src LIMIT 1; - 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b b/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b new file mode 100644 index 0000000000000..ee1a589d6652b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-1-d274b272251e95ec2e8139bfa471bd0b @@ -0,0 +1 @@ +xpath_long(xml, xpath) - Returns a long value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 b/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-10-caeac94758a40493a5227fcdb8ec2f87 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 b/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 new file mode 100644 index 0000000000000..b9eb5a4a99c02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-2-43fbf4d944cf7eaf57b4d6166b803fb6 @@ -0,0 +1,4 @@ +xpath_long(xml, xpath) - Returns a long value that matches the xpath expression +Example: + > SELECT xpath_long('12','sum(a/b)') FROM src LIMIT 1; + 3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b b/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_long-3-9df8d27e31d96e0b35b9b40910d4bd98 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d b/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 rename to sql/hive/src/test/resources/golden/udf_xpath_long-4-3211913c56521887d30e3d1a50762b3f diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 b/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 deleted file mode 100644 index 996d127e59365..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 +++ /dev/null @@ -1 +0,0 @@ -9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d b/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 b/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 new file mode 100644 index 0000000000000..2045006edaf5e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-5-d580a8f01a546baddd939b95722e6354 @@ -0,0 +1 @@ +9223372036854775807 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 rename to sql/hive/src/test/resources/golden/udf_xpath_long-6-b695348ed3faec63be2c07d0d4afaaf3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 b/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 b/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 deleted file mode 100644 index 3f10ffe7a4c47..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 +++ /dev/null @@ -1 +0,0 @@ -15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc b/sql/hive/src/test/resources/golden/udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b similarity index 100% rename from sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc rename to sql/hive/src/test/resources/golden/udf_xpath_long-7-ed5af6d7451107a753b2c7ff130ac73b diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 b/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-8-2f952b56682969bb203fa3d9102f7015 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 b/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 deleted file mode 100644 index c7930257dfef5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 +++ /dev/null @@ -1 +0,0 @@ -7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 b/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 deleted file mode 100644 index 7813681f5b41c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 +++ /dev/null @@ -1 +0,0 @@ -5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd b/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd new file mode 100644 index 0000000000000..7f8f011eb73d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-9-947b6e08ba9c7defd75d00412f9bc4fd @@ -0,0 +1 @@ +7 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f rename to sql/hive/src/test/resources/golden/udf_xpath_short-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 b/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 deleted file mode 100644 index 22ccb9ca9defa..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 +++ /dev/null @@ -1 +0,0 @@ -xpath_short(xml, xpath) - Returns a short value that matches the xpath expression \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 b/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 deleted file mode 100644 index 931ce2abb3d4d..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 +++ /dev/null @@ -1,4 +0,0 @@ -xpath_short(xml, xpath) - Returns a short value that matches the xpath expression -Example: - > SELECT xpath_short('12','sum(a/b)') FROM src LIMIT 1; - 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 b/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 new file mode 100644 index 0000000000000..63ab4f15e8c84 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-1-7d5231aed9cbbf68cd740791f9e5be17 @@ -0,0 +1 @@ +xpath_short(xml, xpath) - Returns a short value that matches the xpath expression diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 b/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-10-b537709676634250e13914e76cd9a530 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 b/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 b/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 new file mode 100644 index 0000000000000..ecb5e9a83109c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-2-5d45932563b78e0b435b8cfebfe3cc2 @@ -0,0 +1,4 @@ +xpath_short(xml, xpath) - Returns a short value that matches the xpath expression +Example: + > SELECT xpath_short('12','sum(a/b)') FROM src LIMIT 1; + 3 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 b/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f rename to sql/hive/src/test/resources/golden/udf_xpath_short-3-c09b4ae6886fa58dcdd728bef45e7efa diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 rename to sql/hive/src/test/resources/golden/udf_xpath_short-4-84f5f6bb2fdc0987d281d52a53a4b24e diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c b/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c deleted file mode 100644 index d7d17fcbef95c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c +++ /dev/null @@ -1 +0,0 @@ --1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e b/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861 similarity index 100% rename from sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 rename to sql/hive/src/test/resources/golden/udf_xpath_short-5-c09fd0565ed041c773fee9bd0436e861 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758 similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 rename to sql/hive/src/test/resources/golden/udf_xpath_short-6-16ced3de15d4ec87a4e7001376551758 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 b/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 deleted file mode 100644 index 56a6051ca2b02..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 +++ /dev/null @@ -1 +0,0 @@ -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b b/sql/hive/src/test/resources/golden/udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2 similarity index 100% rename from sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b rename to sql/hive/src/test/resources/golden/udf_xpath_short-7-8ffdf20c15f3ed81bb5a92c61d200ae2 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d b/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d deleted file mode 100644 index 3f10ffe7a4c47..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d +++ /dev/null @@ -1 +0,0 @@ -15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 b/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-8-102ad2dea8d94528b402d980a45d53d4 @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 b/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 deleted file mode 100644 index c7930257dfef5..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 +++ /dev/null @@ -1 +0,0 @@ -7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e b/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e deleted file mode 100644 index 7813681f5b41c..0000000000000 --- a/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e +++ /dev/null @@ -1 +0,0 @@ -5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d b/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d new file mode 100644 index 0000000000000..7f8f011eb73d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-9-d571e18b7d8ad44fef2e0b2424f34a0d @@ -0,0 +1 @@ +7 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada similarity index 100% rename from sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf rename to sql/hive/src/test/resources/golden/udf_xpath_string-0-50131c0ba7b7a6b65c789a5a8497bada diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 b/sql/hive/src/test/resources/golden/udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 rename to sql/hive/src/test/resources/golden/udf_xpath_string-1-e315d11e9feb29177b5cb1e221c7cfa4 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc b/sql/hive/src/test/resources/golden/udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc rename to sql/hive/src/test/resources/golden/udf_xpath_string-10-d87fb71039c9d2419d750a0721c5696f diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 b/sql/hive/src/test/resources/golden/udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 rename to sql/hive/src/test/resources/golden/udf_xpath_string-2-110b583cde6cd23c486d8223c444cbe9 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f b/sql/hive/src/test/resources/golden/udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f rename to sql/hive/src/test/resources/golden/udf_xpath_string-3-17e4d8122b93a1ebdba6c1d2cf9ce0c4 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a b/sql/hive/src/test/resources/golden/udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a rename to sql/hive/src/test/resources/golden/udf_xpath_string-4-302630fe7dac2cc61fe7d36ead0f41ab diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a b/sql/hive/src/test/resources/golden/udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a rename to sql/hive/src/test/resources/golden/udf_xpath_string-5-19357ba9cb87d3a5717543d2afdc96e2 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 b/sql/hive/src/test/resources/golden/udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 rename to sql/hive/src/test/resources/golden/udf_xpath_string-6-4837e470f745487fae4f498b3a2946bc diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 b/sql/hive/src/test/resources/golden/udf_xpath_string-7-8e42951d002e3c4034b4a51928442706 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 rename to sql/hive/src/test/resources/golden/udf_xpath_string-7-8e42951d002e3c4034b4a51928442706 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f b/sql/hive/src/test/resources/golden/udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f rename to sql/hive/src/test/resources/golden/udf_xpath_string-8-fdfa4e17d70608dcc634c9e1e8a8f288 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 b/sql/hive/src/test/resources/golden/udf_xpath_string-9-84109613320bd05abccd1058044d62c3 similarity index 100% rename from sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 rename to sql/hive/src/test/resources/golden/udf_xpath_string-9-84109613320bd05abccd1058044d62c3 diff --git a/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a b/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a deleted file mode 100644 index 599bf1880a83a..0000000000000 --- a/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a +++ /dev/null @@ -1 +0,0 @@ -stack(n, cols...) - turns k columns into n rows of size k/n each diff --git a/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 b/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 b/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 index e82f93fe65c5e..f00ecaf75c084 100644 --- a/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 +++ b/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 @@ -1,18 +1,18 @@ # col_name data_type comment -a string None +a string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 18:39:52 PST 2014 +CreateTime: Tue Oct 21 05:37:51 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388803192 + transient_lastDdlTime 1413895071 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Bucket Columns: [] Sort Columns: [] Storage Desc Params: field.delim \u0001 - serialization.format \u0001 \ No newline at end of file + serialization.format \u0001 diff --git a/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 index e82f93fe65c5e..f00ecaf75c084 100644 --- a/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 +++ b/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 @@ -1,18 +1,18 @@ # col_name data_type comment -a string None +a string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 18:39:52 PST 2014 +CreateTime: Tue Oct 21 05:37:51 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388803192 + transient_lastDdlTime 1413895071 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Bucket Columns: [] Sort Columns: [] Storage Desc Params: field.delim \u0001 - serialization.format \u0001 \ No newline at end of file + serialization.format \u0001 diff --git a/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 index 2be72c6e4e8c7..79e197569a51c 100644 --- a/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 +++ b/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 @@ -1,18 +1,18 @@ # col_name data_type comment -a string None +a string # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 18:39:52 PST 2014 +CreateTime: Tue Oct 21 05:37:51 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/k1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/k1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388803192 + transient_lastDdlTime 1413895071 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +24,4 @@ Bucket Columns: [] Sort Columns: [] Storage Desc Params: field.delim | - serialization.format | \ No newline at end of file + serialization.format | diff --git a/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 index 1d5891034ddec..6489eb562dbf7 100644 --- a/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 +++ b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 @@ -1,3 +1,3 @@ tst1 500 tst2 500 -tst3 500 \ No newline at end of file +tst3 500 diff --git a/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f index 1ec0096b865a7..c86307863b92d 100644 --- a/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f +++ b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f @@ -1,3 +1,3 @@ tst1 1 tst2 1 -tst3 1 \ No newline at end of file +tst3 1 diff --git a/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 b/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 b/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 b/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 index 2209b11d74282..f085b06e588d4 100644 --- a/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 +++ b/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 @@ -997,4 +997,4 @@ 200 val_200 200 val_200 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 b/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 index 07f67df79dcbd..5ba4e492aa0f4 100644 --- a/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 +++ b/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 @@ -14,4 +14,4 @@ NULL 10 406 1 66 1 98 1 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 b/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 index b00b55f6fa92d..20cb88d87f6bf 100644 --- a/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 +++ b/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 @@ -14,4 +14,4 @@ NULL 20 406 2 66 2 98 2 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 index 6f680072350ab..eeb33ca418288 100644 --- a/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 +++ b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 @@ -1 +1 @@ -12500 \ No newline at end of file +12500 diff --git a/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 index a5a9e42fff209..4d199c29dd4db 100644 --- a/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 +++ b/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 @@ -307,4 +307,4 @@ 96 1 97 1 98 1 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 index be13b26eadc86..5b6f2e2d4f773 100644 --- a/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 +++ b/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 @@ -307,4 +307,4 @@ 96 val_96 1 97 val_97 1 98 val_98 1 -tst1 500 1 \ No newline at end of file +tst1 500 1 diff --git a/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 index 3dee790d7c2ca..11a5a2c531a3d 100644 --- a/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 +++ b/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 @@ -498,4 +498,4 @@ 97 val_97 98 val_98 98 val_98 -tst1 500 \ No newline at end of file +tst1 500 diff --git a/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 index e438a64050723..b95429f0c39e8 100644 --- a/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 +++ b/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 @@ -498,4 +498,4 @@ 97 val_97 val_97 98 val_98 val_98 98 val_98 val_98 -tst1 500 500 \ No newline at end of file +tst1 500 500 diff --git a/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 index e4dfefc43e26a..f21cd3313298c 100644 --- a/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 +++ b/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 @@ -307,4 +307,4 @@ 96 1 97 2 98 2 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 index e438a64050723..b95429f0c39e8 100644 --- a/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 +++ b/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 @@ -498,4 +498,4 @@ 97 val_97 val_97 98 val_98 val_98 98 val_98 val_98 -tst1 500 500 \ No newline at end of file +tst1 500 500 diff --git a/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 index e37d32abba426..83b33d238dab9 100644 --- a/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 +++ b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 @@ -1 +1 @@ -1000 \ No newline at end of file +1000 diff --git a/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c b/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c index 385b8df6703a0..f8539c4817e21 100644 --- a/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c +++ b/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c @@ -20,4 +20,4 @@ 5 val_5 5 val_5 5 val_5 5 val_5 9 val_9 9 val_9 -tst1 500 tst1 500 \ No newline at end of file +tst1 500 tst1 500 diff --git a/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c b/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 b/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 deleted file mode 100644 index 26a2aff52e9a9..0000000000000 --- a/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 +++ /dev/null @@ -1,536 +0,0 @@ -NULL 2 -0 7 -001 2 -002 2 -004 1 -01 1 -013 1 -02 1 -021 2 -03 1 -032 5 -034 3 -051 1 -061 1 -062 1 -063 1 -064 1 -07 3 -071 1 -074 1 -08 1 -081 1 -082 2 -084 3 -09 3 -091 1 -094 1 -1 500 -10 2 -100 2 -102 1 -103 2 -104 7 -105 1 -11 2 -111 2 -113 5 -114 2 -116 1 -118 2 -119 3 -12 2 -120 2 -122 2 -123 2 -124 1 -125 2 -126 1 -128 3 -129 2 -131 2 -133 3 -134 5 -136 1 -137 2 -138 4 -14 1 -142 1 -143 2 -145 1 -146 2 -149 2 -15 4 -150 1 -152 2 -153 2 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 2 -165 2 -166 1 -167 3 -168 1 -169 4 -17 1 -170 1 -172 2 -174 2 -175 2 -176 2 -177 1 -178 1 -179 2 -18 2 -180 1 -181 2 -182 2 -183 1 -184 1 -186 1 -187 3 -189 1 -19 1 -190 1 -191 4 -192 2 -193 3 -194 2 -195 2 -196 1 -197 2 -199 3 -2 2 -20 2 -200 2 -201 1 -202 2 -203 3 -204 1 -205 2 -207 2 -208 3 -209 2 -21 2 -213 2 -214 1 -216 2 -217 2 -218 1 -219 2 -221 2 -222 2 -223 4 -224 2 -226 1 -228 1 -229 2 -230 5 -233 3 -234 1 -235 1 -237 2 -238 2 -239 2 -24 4 -241 1 -242 4 -243 2 -244 1 -247 1 -248 1 -249 1 -251 2 -252 2 -254 1 -255 2 -256 2 -257 1 -258 1 -26 2 -260 1 -261 1 -262 2 -263 2 -264 2 -265 2 -266 1 -27 3 -271 2 -272 4 -273 3 -274 2 -275 1 -277 4 -278 2 -28 2 -280 2 -281 2 -282 4 -283 3 -284 2 -285 1 -286 1 -287 1 -288 2 -289 1 -29 1 -291 2 -292 2 -293 1 -294 2 -296 1 -298 3 -30 2 -301 2 -302 3 -304 3 -305 1 -306 1 -307 2 -308 1 -309 2 -310 1 -311 5 -312 2 -314 2 -315 1 -316 3 -317 2 -318 3 -321 2 -322 4 -323 2 -325 2 -327 3 -33 2 -331 3 -332 3 -333 4 -335 1 -336 1 -338 1 -339 1 -34 2 -341 2 -342 2 -344 3 -345 1 -348 5 -35 4 -351 2 -353 4 -354 1 -356 1 -360 1 -361 1 -362 2 -364 3 -365 1 -366 1 -367 2 -368 1 -369 3 -37 2 -372 3 -373 2 -374 1 -375 1 -377 1 -378 1 -379 1 -38 2 -381 1 -382 3 -384 4 -386 1 -389 1 -391 3 -392 1 -393 2 -394 2 -395 2 -396 3 -397 2 -399 2 -4 2 -40 1 -400 1 -401 7 -402 1 -403 3 -404 4 -406 4 -407 1 -409 3 -41 1 -411 2 -412 1 -413 2 -414 4 -417 3 -418 1 -419 1 -42 4 -421 1 -422 2 -424 4 -427 1 -429 2 -43 2 -430 3 -431 5 -432 1 -435 1 -436 1 -437 1 -438 3 -439 2 -44 2 -442 1 -443 3 -444 2 -446 1 -448 1 -449 1 -45 1 -452 1 -453 1 -454 6 -455 1 -457 1 -458 2 -459 2 -46 1 -460 1 -461 2 -462 2 -463 3 -466 3 -467 1 -468 4 -469 5 -47 2 -470 1 -471 2 -472 2 -473 1 -475 1 -477 1 -478 2 -479 1 -48 2 -480 3 -481 1 -482 2 -483 4 -484 2 -485 1 -487 1 -489 4 -490 1 -491 2 -492 2 -493 2 -494 2 -495 1 -496 1 -497 1 -498 3 -5 6 -50 1 -501 1 -502 2 -503 1 -51 4 -513 1 -521 2 -523 2 -53 4 -532 1 -533 1 -534 1 -54 1 -541 1 -543 1 -551 1 -552 2 -554 1 -56 1 -561 2 -562 2 -563 1 -57 1 -571 2 -572 1 -573 1 -574 1 -58 3 -582 1 -584 1 -59 2 -591 2 -593 2 -594 1 -60 1 -603 1 -604 4 -611 1 -612 2 -613 3 -62 2 -621 1 -622 1 -631 1 -633 1 -634 1 -64 1 -641 2 -644 1 -65 1 -651 1 -652 2 -653 1 -66 2 -661 1 -662 1 -663 1 -664 3 -67 4 -671 2 -68 1 -681 1 -682 1 -683 1 -69 2 -691 1 -692 1 -693 3 -694 1 -70 4 -702 2 -703 2 -704 1 -71 1 -712 2 -713 2 -714 3 -72 3 -723 3 -724 1 -73 2 -731 2 -732 2 -734 1 -74 2 -742 1 -75 1 -751 1 -752 1 -754 1 -76 4 -761 3 -763 2 -764 1 -77 2 -771 1 -772 4 -773 1 -774 1 -78 2 -781 3 -782 1 -784 1 -79 2 -791 2 -793 2 -794 1 -8 2 -80 2 -802 3 -803 1 -81 2 -811 2 -812 1 -813 3 -814 1 -82 2 -821 3 -822 1 -83 2 -831 4 -832 2 -833 1 -834 3 -84 2 -842 1 -843 5 -844 1 -85 3 -851 1 -852 1 -854 2 -86 1 -861 1 -863 1 -864 4 -87 2 -871 1 -872 2 -873 1 -874 2 -882 2 -89 2 -892 3 -894 3 -9 2 -90 4 -902 2 -903 2 -904 3 -91 1 -911 3 -912 2 -914 1 -92 1 -921 2 -922 2 -924 2 -932 2 -933 1 -934 2 -941 2 -942 1 -944 1 -95 2 -954 2 -96 2 -961 4 -963 3 -964 5 -97 2 -971 2 -973 1 -974 1 -98 2 -981 1 -982 1 -983 1 -984 4 -991 3 -993 2 -record_0 1 -record_1 1 -record_2 1 -record_3 1 -record_4 1 -record_5 1 -record_6 1 -record_7 1 -record_8 1 -record_9 1 diff --git a/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd b/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd index bf0f76662bd2c..14ac2a0543eb7 100644 --- a/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd +++ b/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd @@ -997,4 +997,4 @@ 98 val_98 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 b/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 index 199095f1f9848..61d7280758de9 100644 --- a/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 +++ b/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 @@ -5,4 +5,4 @@ 97 val_97 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 index 293f324297419..0e14af56f8b08 100644 --- a/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 +++ b/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 @@ -17,4 +17,4 @@ 8 val_8 8 val_8 8 val_8 -9 val_9 \ No newline at end of file +9 val_9 diff --git a/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 index 87a971baed428..ba254c7ab023b 100644 --- a/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 +++ b/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 @@ -17,4 +17,4 @@ 5 val_5 5 val_5 5 val_5 -5 val_5 \ No newline at end of file +5 val_5 diff --git a/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 index a6f502e6c3e3a..996f02bc6496c 100644 --- a/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 +++ b/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 @@ -17,4 +17,4 @@ 5 val_5 5 val_5 5 val_5 -5 val_5 \ No newline at end of file +5 val_5 diff --git a/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 b/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 index dadf53962e7a1..b1fb9cce36fa2 100644 --- a/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 +++ b/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 @@ -3,4 +3,4 @@ 4 2 5 6 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e b/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e index dadf53962e7a1..b1fb9cce36fa2 100644 --- a/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e +++ b/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e @@ -3,4 +3,4 @@ 4 2 5 6 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 b/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 index ede645acaf95b..48255af904349 100644 --- a/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 +++ b/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 @@ -3,4 +3,4 @@ 4 2 5 2 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 b/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 index ede645acaf95b..48255af904349 100644 --- a/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 +++ b/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 @@ -3,4 +3,4 @@ 4 2 5 2 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb b/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb index dadf53962e7a1..b1fb9cce36fa2 100644 --- a/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb +++ b/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb @@ -3,4 +3,4 @@ 4 2 5 6 8 2 -9 2 \ No newline at end of file +9 2 diff --git a/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 b/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 index a71793e8c52a3..c7f78ecade3e8 100644 --- a/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 +++ b/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 @@ -3,4 +3,4 @@ val_2 2 val_4 2 val_5 6 val_8 2 -val_9 2 \ No newline at end of file +val_9 2 diff --git a/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e b/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 b/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b b/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 b/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 deleted file mode 100644 index 9bf0de2f06c9a..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 +++ /dev/null @@ -1,32 +0,0 @@ -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -2.0 2.0 -2.0 2.0 -4.0 4.0 -4.0 4.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5.0 -8.0 8.0 -8.0 8.0 -9.0 9.0 -9.0 9.0 diff --git a/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 b/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 b/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 deleted file mode 100644 index 462dca3124c41..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 +++ /dev/null @@ -1,20 +0,0 @@ -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -2.0 -2.0 -4.0 -4.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -8.0 -8.0 -9.0 -9.0 diff --git a/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e b/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 b/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 deleted file mode 100644 index 2e662f39d1572..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 +++ /dev/null @@ -1,32 +0,0 @@ -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -2.0 -2.0 -4.0 -4.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -8.0 -8.0 -9.0 -9.0 diff --git a/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b b/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 b/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 deleted file mode 100644 index 2e662f39d1572..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 +++ /dev/null @@ -1,32 +0,0 @@ -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -0.0 -2.0 -2.0 -4.0 -4.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -5.0 -8.0 -8.0 -9.0 -9.0 diff --git a/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb b/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db b/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db deleted file mode 100644 index 106f1874451ff..0000000000000 --- a/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db +++ /dev/null @@ -1,32 +0,0 @@ -0.0 0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -0.0 0 -0.0 0.0 -0.0 0.0 -0.0 0.0 -2.0 2.0 -2.0 2 -4.0 4 -4.0 4.0 -5.0 5 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5 -5.0 5.0 -5.0 5.0 -5.0 5.0 -5.0 5 -5.0 5.0 -5.0 5.0 -5.0 5.0 -8.0 8.0 -8.0 8 -9.0 9 -9.0 9.0 diff --git a/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 index d572335ff5185..7369ee2317540 100644 --- a/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 +++ b/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 @@ -27,4 +27,4 @@ 86 val_86 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 b/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/union34-4-70479e10c016e5ac448394dbadb32794 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c rename to sql/hive/src/test/resources/golden/union34-4-70479e10c016e5ac448394dbadb32794 diff --git a/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 +++ b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 index d572335ff5185..7369ee2317540 100644 --- a/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 +++ b/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 @@ -27,4 +27,4 @@ 86 val_86 98 val_98 98 val_98 -98 val_98 \ No newline at end of file +98 val_98 diff --git a/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b +++ b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 index 948aca9180ba9..abc382d9b3571 100644 --- a/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 +++ b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 @@ -1,2 +1,2 @@ tst1 500 -tst2 500 \ No newline at end of file +tst2 500 diff --git a/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 index c0c7d9f5e8ef5..9daa3f92b46e0 100644 --- a/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 +++ b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 @@ -1,2 +1,2 @@ tst1 1 -tst2 1 \ No newline at end of file +tst2 1 diff --git a/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 b/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 index b5e91e032212a..6f79451ce69b5 100644 --- a/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 +++ b/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 @@ -23,4 +23,4 @@ NULL val_484 406 val_406 66 val_66 98 val_98 -tst1 500 \ No newline at end of file +tst1 500 diff --git a/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 +++ b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 b/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 index 07f67df79dcbd..5ba4e492aa0f4 100644 --- a/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 +++ b/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 @@ -14,4 +14,4 @@ NULL 10 406 1 66 1 98 1 -tst1 1 \ No newline at end of file +tst1 1 diff --git a/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 b/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 index 3617909902993..52982787dbb4a 100644 --- a/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 +++ b/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 @@ -1497,4 +1497,4 @@ 200 val_200 97 val_97 97 val_97 -97 val_97 \ No newline at end of file +97 val_97 diff --git a/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 index 37021f4a27201..3d86ec6498f3f 100644 --- a/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 +++ b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 @@ -1 +1 @@ -1500 \ No newline at end of file +1500 diff --git a/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c b/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/union_date-4-d85fe746334b430941c5db3665e744d4 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 rename to sql/hive/src/test/resources/golden/union_date-4-d85fe746334b430941c5db3665e744d4 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/union_date-5-82eebfded24cef08e0a881d1bcca02b1 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 rename to sql/hive/src/test/resources/golden/union_date-5-82eebfded24cef08e0a881d1bcca02b1 diff --git a/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d b/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a b/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a deleted file mode 100644 index 468f774ef5146..0000000000000 --- a/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a +++ /dev/null @@ -1,10 +0,0 @@ -val_238 -NULL -val_86 -NULL -val_311 -NULL -val_27 -NULL -val_165 -NULL diff --git a/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index c203753d8e42d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:49:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069004 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 b/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 b/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 94a711cdab117..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:50:08 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069036 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 b/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 +++ b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c +++ b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 index d9cfa9e952c3a..4c47424eb1160 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ b/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 @@ -1,19 +1,24 @@ # col_name data_type comment -key string from deserializer -values bigint from deserializer +key string +values bigint # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 19:05:51 PST 2014 +CreateTime: Tue Oct 21 05:59:44 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/outputtbl1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388804758 + COLUMN_STATS_ACCURATE false + numFiles 1 + numRows -1 + rawDataSize -1 + totalSize 184 + transient_lastDdlTime 1413896391 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe @@ -24,4 +29,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e index 2817d74afee13..8930b40bed84f 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e +++ b/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e @@ -15,4 +15,4 @@ 8 2 8 2 8 3 -8 3 \ No newline at end of file +8 3 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a +++ b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 +++ b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc +++ b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 +++ b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/union_remove_11-9-94da21f150ed2c56046b80e46da8884d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 rename to sql/hive/src/test/resources/golden/union_remove_11-9-94da21f150ed2c56046b80e46da8884d diff --git a/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 57d5146698ee7..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:50:41 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069061 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 b/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 b/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af b/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index c914024c14c8e..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:51:05 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069086 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb b/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 5a55428afbe1a..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:51:30 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069108 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d b/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index a2441e330ffc6..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:51:52 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069112 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index a58273a1b054e..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d b/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index bec8202dcbcad..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:52:19 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069139 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index a58273a1b054e..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a b/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 b/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d b/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index d0b37e7625a64..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Tue Jan 14 14:38:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse4075462935071533647/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1389739098 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index c21f4017362c1..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,2 +0,0 @@ -ds=1 -ds=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 b/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 deleted file mode 100644 index 1ab5c482feca9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 +++ /dev/null @@ -1,6 +0,0 @@ -1 1 1 -2 1 1 -3 1 1 -7 1 1 -8 1 1 -8 1 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a b/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a deleted file mode 100644 index 94ec062e2f4f3..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a +++ /dev/null @@ -1,6 +0,0 @@ -1 2 2 -2 2 2 -3 2 2 -7 2 2 -8 2 2 -8 2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b b/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad b/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 b/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 971e095df7be8..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,32 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Partition Information -# col_name data_type comment - -ds string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:52:47 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069167 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 deleted file mode 100644 index 5690d94a65857..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 +++ /dev/null @@ -1,6 +0,0 @@ -ds=11 -ds=12 -ds=13 -ds=17 -ds=18 -ds=28 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 b/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 b/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b b/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 0d0d3a30af36c..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:53:14 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069215 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 deleted file mode 100644 index 719e4a7bd4dbb..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 +++ /dev/null @@ -1,10 +0,0 @@ -1 1 -2 1 -3 1 -7 1 -8 2 -1 1 -2 1 -3 1 -7 1 -8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 b/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 b/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 deleted file mode 100644 index 361f2ab78eae3..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 +++ /dev/null @@ -1,2 +0,0 @@ -7 1 -7 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e b/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 b/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 b/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index d3c821fa22182..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:53:36 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069242 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e b/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 b/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 978049640929c..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -values bigint None -key string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:54:06 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069268 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e b/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f b/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa b/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 b/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index ce7e0438f9eff..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,26 +0,0 @@ -# col_name data_type comment - -key string None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:54:32 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069293 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f b/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 b/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 b/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 b/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index c261ce48bdfb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,28 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None -values2 bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:54:58 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069319 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 deleted file mode 100644 index d36ac6b834792..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 +++ /dev/null @@ -1,10 +0,0 @@ -1 1 1 -2 1 1 -3 1 1 -7 1 1 -8 2 2 -1 1 1 -2 1 1 -3 1 1 -7 1 1 -8 2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd b/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af b/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 b/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 b/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb b/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 b/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 50297abd4e9a6..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:55:20 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069345 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 b/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 b/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 b/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index f3c452ebe9284..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key double None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:55:49 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069371 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 b/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e b/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f b/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 +++ b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c +++ b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 index d46af80f0d1ae..8cd7d42b20df5 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ b/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 @@ -1,19 +1,24 @@ # col_name data_type comment -key string None -values bigint None +key string +values bigint # Detailed Table Information Database: default Owner: marmbrus -CreateTime: Fri Jan 03 19:09:55 PST 2014 +CreateTime: Tue Oct 21 06:00:02 PDT 2014 LastAccessTime: UNKNOWN Protect Mode: None Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse5393816597631538368/outputtbl1 +Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse1201055597819413730/outputtbl1 Table Type: MANAGED_TABLE Table Parameters: - transient_lastDdlTime 1388805002 + COLUMN_STATS_ACCURATE false + numFiles 1 + numRows -1 + rawDataSize -1 + totalSize 72 + transient_lastDdlTime 1413896409 # Storage Information SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe @@ -24,4 +29,4 @@ Num Buckets: -1 Bucket Columns: [] Sort Columns: [] Storage Desc Params: - serialization.format 1 \ No newline at end of file + serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e index 2817d74afee13..8930b40bed84f 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e +++ b/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e @@ -15,4 +15,4 @@ 8 2 8 2 8 3 -8 3 \ No newline at end of file +8 3 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 +++ b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/union_remove_3-8-94da21f150ed2c56046b80e46da8884d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b rename to sql/hive/src/test/resources/golden/union_remove_3-8-94da21f150ed2c56046b80e46da8884d diff --git a/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 67faa5b13d984..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:56:16 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069398 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead b/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 19aeb0b602859..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string None -values bigint None - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:56:42 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069430 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -InputFormat: org.apache.hadoop.mapred.TextInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 b/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 +++ b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c +++ b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a +++ b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e index 8e7ee8a2b47bb..b4ea93dc7dd89 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e +++ b/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e @@ -7,4 +7,4 @@ 7 1 7 1 8 2 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 b/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 index 8e7ee8a2b47bb..b4ea93dc7dd89 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 +++ b/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 @@ -7,4 +7,4 @@ 7 1 7 1 8 2 -8 2 \ No newline at end of file +8 2 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 +++ b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 +++ b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab +++ b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 index c227083464fb9..573541ac9702d 100644 --- a/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 +++ b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 @@ -1 +1 @@ -0 \ No newline at end of file +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/union_remove_6-9-94da21f150ed2c56046b80e46da8884d similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 rename to sql/hive/src/test/resources/golden/union_remove_6-9-94da21f150ed2c56046b80e46da8884d diff --git a/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 1c63b3851aad2..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:57:18 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069463 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca b/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index f718ab2990cdd..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:57:48 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069495 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee b/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 b/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c b/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 deleted file mode 100644 index 65a91a6027545..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 +++ /dev/null @@ -1,27 +0,0 @@ -# col_name data_type comment - -key string from deserializer -values bigint from deserializer - -# Detailed Table Information -Database: default -Owner: marmbrus -CreateTime: Mon Feb 10 13:58:19 PST 2014 -LastAccessTime: UNKNOWN -Protect Mode: None -Retention: 0 -Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse561133087079823206/outputtbl1 -Table Type: MANAGED_TABLE -Table Parameters: - transient_lastDdlTime 1392069518 - -# Storage Information -SerDe Library: org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe -InputFormat: org.apache.hadoop.hive.ql.io.RCFileInputFormat -OutputFormat: org.apache.hadoop.hive.ql.io.RCFileOutputFormat -Compressed: No -Num Buckets: -1 -Bucket Columns: [] -Sort Columns: [] -Storage Desc Params: - serialization.format 1 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 b/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 b/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 index 42d07ed10bad4..760e62cce0bba 100644 --- a/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 +++ b/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 @@ -497,4 +497,4 @@ 97 97 98 -98 \ No newline at end of file +98 diff --git a/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 b/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 index 2f23db6a2ff4a..3bafdf359f6fa 100644 --- a/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 +++ b/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 @@ -997,4 +997,4 @@ 98 98 98 -98 \ No newline at end of file +98 diff --git a/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d b/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 b/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 b/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 b/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e b/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 b/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 deleted file mode 100644 index 573541ac9702d..0000000000000 --- a/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 +++ /dev/null @@ -1 +0,0 @@ -0 diff --git a/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da b/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da index 08661a3d7f794..6fc243e718c1b 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da +++ b/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da @@ -2,4 +2,4 @@ val_0 0 3 val_10 10 1 val_100 200 2 val_103 206 2 -val_104 208 2 \ No newline at end of file +val_104 208 2 diff --git a/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f b/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f index 08661a3d7f794..6fc243e718c1b 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f +++ b/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f @@ -2,4 +2,4 @@ val_0 0 3 val_10 10 1 val_100 200 2 val_103 206 2 -val_104 208 2 \ No newline at end of file +val_104 208 2 diff --git a/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 b/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 index f0c744ca37d95..0052fe0c959c5 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 +++ b/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 @@ -2,4 +2,4 @@ val_98 196 2 val_97 194 2 val_96 96 1 val_95 190 2 -val_92 92 1 \ No newline at end of file +val_92 92 1 diff --git a/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f b/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f index f0c744ca37d95..0052fe0c959c5 100644 --- a/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f +++ b/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f @@ -2,4 +2,4 @@ val_98 196 2 val_97 194 2 val_96 96 1 val_95 190 2 -val_92 92 1 \ No newline at end of file +val_92 92 1 diff --git a/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce b/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce index 35e7795b9c2a7..fa55ed96ee0b9 100644 --- a/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce +++ b/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce @@ -1,3 +1,3 @@ 1 abc 1 abc 2 abc 2 abc -3 abc 3 abc \ No newline at end of file +3 abc 3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c b/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c index 35e7795b9c2a7..fa55ed96ee0b9 100644 --- a/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c +++ b/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c @@ -1,3 +1,3 @@ 1 abc 1 abc 2 abc 2 abc -3 abc 3 abc \ No newline at end of file +3 abc 3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef b/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 rename to sql/hive/src/test/resources/golden/varchar_join1-6-ab4392aa5ff499ec43229425ff23e22f diff --git a/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 b/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/varchar_join1-7-a01639290aaceb4b85aa6e44319f6386 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 rename to sql/hive/src/test/resources/golden/varchar_join1-7-a01639290aaceb4b85aa6e44319f6386 diff --git a/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 b/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 rename to sql/hive/src/test/resources/golden/varchar_join1-8-c05d7b534b51cecdc2ba2de4ce57ba37 diff --git a/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 b/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 index 35e7795b9c2a7..fa55ed96ee0b9 100644 --- a/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 +++ b/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 @@ -1,3 +1,3 @@ 1 abc 1 abc 2 abc 2 abc -3 abc 3 abc \ No newline at end of file +3 abc 3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 b/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 deleted file mode 100644 index 05066db296756..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 +++ /dev/null @@ -1 +0,0 @@ -c1 array None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 b/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 deleted file mode 100644 index 05cd8fccf276c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 +++ /dev/null @@ -1 +0,0 @@ -["val_0","val_0"] diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f b/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 b/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 b/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 deleted file mode 100644 index 1ec6828af9fd2..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 +++ /dev/null @@ -1 +0,0 @@ -c1 map None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 b/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 deleted file mode 100644 index 8f12062de4e27..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 +++ /dev/null @@ -1 +0,0 @@ -{0:"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 b/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc b/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 b/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 deleted file mode 100644 index 030d9d6ec0429..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 +++ /dev/null @@ -1 +0,0 @@ -c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 b/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 deleted file mode 100644 index f350c1112ac2c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 +++ /dev/null @@ -1 +0,0 @@ -{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 b/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad b/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad deleted file mode 100644 index 030d9d6ec0429..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad +++ /dev/null @@ -1 +0,0 @@ -c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 b/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 deleted file mode 100644 index f350c1112ac2c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 +++ /dev/null @@ -1 +0,0 @@ -{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 b/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 b/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 deleted file mode 100644 index 030d9d6ec0429..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 +++ /dev/null @@ -1 +0,0 @@ -c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb b/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb deleted file mode 100644 index f350c1112ac2c..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb +++ /dev/null @@ -1 +0,0 @@ -{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b b/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 b/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 b/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 b/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 b/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e b/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 b/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e b/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 b/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 b/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 b/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a b/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d b/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad b/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 b/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b b/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 b/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 b/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 b/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc b/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f b/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 b/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e b/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c b/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a b/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 b/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb b/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 b/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 b/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 b/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 b/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 b/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 deleted file mode 100644 index 3f02a574d3dd5..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 +++ /dev/null @@ -1,5 +0,0 @@ -474 val_475 -62 val_63 -468 val_469 -272 val_273 -448 val_449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 b/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 deleted file mode 100644 index 1e2a6a6562290..0000000000000 --- a/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 +++ /dev/null @@ -1,5 +0,0 @@ -val_0 3 -val_1 2 -val_10 1 -val_100 2 -val_101 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 b/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 index 173eb955a485a..52bb2199e16a3 100644 --- a/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 +++ b/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 @@ -1,4 +1,4 @@ 1 abc 1 abc 2 abc -3 abc \ No newline at end of file +3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 b/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 index 173eb955a485a..52bb2199e16a3 100644 --- a/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 +++ b/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 @@ -1,4 +1,4 @@ 1 abc 1 abc 2 abc -3 abc \ No newline at end of file +3 abc diff --git a/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 b/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b rename to sql/hive/src/test/resources/golden/varchar_union1-6-f338f341c5f86d0a44cabfb4f7bddc3b diff --git a/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 b/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29 similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc rename to sql/hive/src/test/resources/golden/varchar_union1-7-ea0d1fbae997b50dc34f7610480bbe29 diff --git a/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 b/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/varchar_union1-8-f3be9a2498927d692356c2cf871d25bf similarity index 100% rename from sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e rename to sql/hive/src/test/resources/golden/varchar_union1-8-f3be9a2498927d692356c2cf871d25bf diff --git a/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 b/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 index 173eb955a485a..52bb2199e16a3 100644 --- a/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 +++ b/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 @@ -1,4 +1,4 @@ 1 abc 1 abc 2 abc -3 abc \ No newline at end of file +3 abc diff --git a/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 b/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 b/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 b/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 b/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 b/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee b/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 b/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a b/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 b/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b b/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 b/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 b/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 b/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 b/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf b/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb b/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 b/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de b/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 b/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 b/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a b/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 b/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb b/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 b/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 b/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 b/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe b/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d b/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d deleted file mode 100644 index ea00577174e43..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d +++ /dev/null @@ -1,500 +0,0 @@ -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 238 0 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 86 12 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 22 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 27 34 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 165 44 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 255 68 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 278 80 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 98 92 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 484 102 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 265 114 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 126 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 138 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 150 150 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 162 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 224 174 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 186 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 66 198 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 208 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 213 220 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 146 232 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 244 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 429 256 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 374 268 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 152 280 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 292 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 145 304 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 495 316 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 37 328 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 338 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 281 350 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 362 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 209 374 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 15 386 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 82 396 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 406 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 166 418 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 430 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 442 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 252 454 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 292 466 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 219 478 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 287 490 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 153 502 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 514 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 338 526 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 446 538 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 459 550 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 394 562 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 237 574 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 482 586 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 174 598 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 413 610 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 494 622 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 207 634 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 646 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 658 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 670 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 174 682 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 399 694 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 706 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 247 718 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 730 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 742 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 162 754 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 377 766 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 397 778 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 309 790 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 365 802 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 266 814 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 439 826 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 342 838 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 367 850 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 325 862 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 874 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 195 886 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 475 898 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 17 910 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 113 920 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 155 932 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 203 944 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 339 956 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 968 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 455 976 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 988 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 1000 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 1012 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 57 1024 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 302 1034 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 205 1046 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 149 1058 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 1070 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 345 1082 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 129 1094 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 170 1106 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 20 1118 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 1128 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 157 1140 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 378 1152 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 221 1164 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 92 1176 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 111 1186 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 47 1198 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 72 1208 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 4 1218 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 280 1226 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 1238 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 427 1248 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 1260 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 1272 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 356 1284 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 399 1296 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 1308 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 382 1320 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 1332 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 125 1344 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 386 1356 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 437 1368 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 1380 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 192 1392 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 286 1404 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 1416 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 176 1428 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 54 1440 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 459 1450 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 51 1462 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 1472 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 103 1484 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 239 1496 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 213 1508 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 216 1520 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 1532 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 278 1544 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 176 1556 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 289 1568 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 221 1580 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 65 1592 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 1602 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 332 1614 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 1626 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 275 1638 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 137 1650 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 241 1662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 83 1674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 333 1684 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 180 1696 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 284 1708 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 12 1720 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 1730 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 181 1742 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 67 1754 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 260 1764 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 404 1776 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 1788 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 1800 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 353 1812 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 373 1824 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 272 1836 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 1848 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 217 1860 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 84 1872 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 1882 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 1894 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 58 1906 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 8 1916 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 411 1924 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 1936 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 1948 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 1960 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 24 1972 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 463 1982 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 1994 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 179 2006 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 172 2018 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 42 2030 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 129 2040 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 158 2052 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 2064 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 496 2076 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 2088 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 322 2096 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 197 2108 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 2120 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 393 2132 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 2144 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 100 2156 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 2168 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 2180 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 191 2192 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 418 2204 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 96 2216 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 26 2226 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 165 2236 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 2248 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 2260 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 205 2272 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 120 2284 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 131 2296 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 51 2308 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 404 2318 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 43 2330 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 436 2340 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 156 2352 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 2364 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 2376 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 308 2388 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 95 2400 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 196 2410 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 288 2422 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 481 2434 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 457 2446 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 98 2458 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 282 2468 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 197 2480 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 2492 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 2504 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 2516 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 2528 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 470 2540 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 137 2552 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 2564 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 2576 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 2588 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 413 2600 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 85 2612 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 77 2622 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 2632 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 490 2640 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 87 2652 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 364 2662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 179 2674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 118 2686 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 134 2698 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 395 2710 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 282 2722 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 2734 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 238 2746 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 419 2758 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 15 2770 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 118 2780 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 72 2792 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 2802 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 307 2812 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 19 2824 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 435 2834 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 10 2846 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 2856 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 2868 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 306 2880 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 224 2892 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 309 2904 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 389 2916 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 2928 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 242 2940 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 2952 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 392 2964 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 272 2976 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 331 2988 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 3000 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 242 3012 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 452 3024 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 177 3036 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 226 3048 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 3060 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 497 3068 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 402 3080 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 3092 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 317 3104 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 395 3116 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 58 3128 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 3138 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 336 3148 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 95 3160 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 11 3170 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 168 3180 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 34 3192 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 229 3202 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 233 3214 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 143 3226 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 472 3238 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 322 3250 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 3262 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 160 3274 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 195 3286 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 42 3298 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 321 3308 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 3320 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 3332 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 3344 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 458 3356 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 78 3368 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 76 3378 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 41 3388 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 223 3398 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 492 3410 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 149 3422 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 449 3434 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 218 3446 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 228 3458 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 3470 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 453 3482 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 30 3494 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 209 3504 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 64 3516 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 3526 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 76 3538 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 74 3548 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 342 3558 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 69 3570 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 3580 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 33 3592 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 368 3602 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 103 3614 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 296 3626 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 113 3638 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 216 3650 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 367 3662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 344 3674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 3686 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 274 3698 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 219 3710 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 239 3722 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 485 3734 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 116 3746 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 223 3758 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 256 3770 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 263 3782 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 3794 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 487 3804 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 3816 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 3828 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 288 3840 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 191 3852 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 3864 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 244 3872 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 3884 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 3896 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 467 3908 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 432 3920 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 202 3932 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 3944 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 229 3956 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 3968 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 463 3980 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 280 3992 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 2 4004 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 4012 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 283 4022 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 331 4034 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 235 4046 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 80 4058 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 44 4068 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 4078 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 321 4090 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 335 4102 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 104 4114 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 4126 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 366 4138 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 175 4150 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 4162 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 483 4174 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 53 4186 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 105 4196 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 257 4208 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 4220 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 4232 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 190 4244 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 4256 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 4268 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 114 4280 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 258 4292 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 4304 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 203 4314 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 262 4326 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 4338 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 424 4350 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 12 4362 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 4372 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 201 4384 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 217 4396 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 164 4408 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 4420 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 4432 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 478 4444 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 4456 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 125 4468 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 4480 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 164 4492 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 424 4504 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 4516 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 382 4528 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 4540 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 4548 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 397 4558 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 4570 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 291 4582 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 24 4594 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 351 4604 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 255 4616 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 104 4628 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 4640 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 163 4650 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 4662 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 4674 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 414 4686 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 200 4698 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 491 4710 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 237 4722 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 439 4734 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 360 4746 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 248 4758 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 479 4770 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 305 4782 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 4794 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 4806 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 444 4818 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 120 4830 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 429 4842 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 443 4866 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 323 4878 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 325 4890 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 4902 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 4914 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 478 4926 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 178 4938 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 4950 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 310 4962 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 317 4974 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 333 4986 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 493 4998 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 460 5010 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 207 5022 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 249 5034 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 265 5046 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 5058 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 83 5070 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 136 5080 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 353 5092 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 172 5104 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 214 5116 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 462 5128 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 233 5140 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 5152 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 133 5164 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 175 5176 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 189 5188 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 5200 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 375 5212 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 5224 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 421 5236 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 407 5248 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 5260 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 256 5272 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 26 5284 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 134 5294 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 67 5306 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 5316 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 379 5328 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 18 5340 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 462 5350 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 492 5362 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 100 5374 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 5386 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 9 5398 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 341 5406 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 5418 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 146 5430 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 458 5442 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 362 5454 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 186 5466 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 285 5478 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 5490 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 5502 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 18 5514 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 5524 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 183 5536 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 281 5548 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 344 5560 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 97 5572 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 5582 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 315 5594 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 84 5606 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 28 5616 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 37 5626 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 448 5636 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 152 5648 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 5660 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 307 5672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 194 5684 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 414 5696 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 477 5708 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 222 5720 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 126 5732 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 5744 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 5754 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 5766 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 400 5778 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 200 5790 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 97 5802 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 b/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 deleted file mode 100644 index df07a9da29f01..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 +++ /dev/null @@ -1,309 +0,0 @@ -0 3 -2 1 -4 1 -5 3 -8 1 -9 1 -10 1 -11 1 -12 2 -15 2 -17 1 -18 2 -19 1 -20 1 -24 2 -26 2 -27 1 -28 1 -30 1 -33 1 -34 1 -35 3 -37 2 -41 1 -42 2 -43 1 -44 1 -47 1 -51 2 -53 1 -54 1 -57 1 -58 2 -64 1 -65 1 -66 1 -67 2 -69 1 -70 3 -72 2 -74 1 -76 2 -77 1 -78 1 -80 1 -82 1 -83 2 -84 2 -85 1 -86 1 -87 1 -90 3 -92 1 -95 2 -96 1 -97 2 -98 2 -100 2 -103 2 -104 2 -105 1 -111 1 -113 2 -114 1 -116 1 -118 2 -119 3 -120 2 -125 2 -126 1 -128 3 -129 2 -131 1 -133 1 -134 2 -136 1 -137 2 -138 4 -143 1 -145 1 -146 2 -149 2 -150 1 -152 2 -153 1 -155 1 -156 1 -157 1 -158 1 -160 1 -162 1 -163 1 -164 2 -165 2 -166 1 -167 3 -168 1 -169 4 -170 1 -172 2 -174 2 -175 2 -176 2 -177 1 -178 1 -179 2 -180 1 -181 1 -183 1 -186 1 -187 3 -189 1 -190 1 -191 2 -192 1 -193 3 -194 1 -195 2 -196 1 -197 2 -199 3 -200 2 -201 1 -202 1 -203 2 -205 2 -207 2 -208 3 -209 2 -213 2 -214 1 -216 2 -217 2 -218 1 -219 2 -221 2 -222 1 -223 2 -224 2 -226 1 -228 1 -229 2 -230 5 -233 2 -235 1 -237 2 -238 2 -239 2 -241 1 -242 2 -244 1 -247 1 -248 1 -249 1 -252 1 -255 2 -256 2 -257 1 -258 1 -260 1 -262 1 -263 1 -265 2 -266 1 -272 2 -273 3 -274 1 -275 1 -277 4 -278 2 -280 2 -281 2 -282 2 -283 1 -284 1 -285 1 -286 1 -287 1 -288 2 -289 1 -291 1 -292 1 -296 1 -298 3 -302 1 -305 1 -306 1 -307 2 -308 1 -309 2 -310 1 -311 3 -315 1 -316 3 -317 2 -318 3 -321 2 -322 2 -323 1 -325 2 -327 3 -331 2 -332 1 -333 2 -335 1 -336 1 -338 1 -339 1 -341 1 -342 2 -344 2 -345 1 -348 5 -351 1 -353 2 -356 1 -360 1 -362 1 -364 1 -365 1 -366 1 -367 2 -368 1 -369 3 -373 1 -374 1 -375 1 -377 1 -378 1 -379 1 -382 2 -384 3 -386 1 -389 1 -392 1 -393 1 -394 1 -395 2 -396 3 -397 2 -399 2 -400 1 -401 5 -402 1 -403 3 -404 2 -406 4 -407 1 -409 3 -411 1 -413 2 -414 2 -417 3 -418 1 -419 1 -421 1 -424 2 -427 1 -429 2 -430 3 -431 3 -432 1 -435 1 -436 1 -437 1 -438 3 -439 2 -443 1 -444 1 -446 1 -448 1 -449 1 -452 1 -453 1 -454 3 -455 1 -457 1 -458 2 -459 2 -460 1 -462 2 -463 2 -466 3 -467 1 -468 4 -469 5 -470 1 -472 1 -475 1 -477 1 -478 2 -479 1 -480 3 -481 1 -482 1 -483 1 -484 1 -485 1 -487 1 -489 4 -490 1 -491 1 -492 2 -493 1 -494 1 -495 1 -496 1 -497 1 -498 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 b/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 b/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 deleted file mode 100644 index 2a0cdc268347a..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 +++ /dev/null @@ -1,309 +0,0 @@ -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 0 [968,2632,2088] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 2 [4004] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 4 [1218] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 5 [4540,3864,3060] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 8 [1916] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 9 [5398] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 10 [2846] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 11 [3170] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 12 [4362,1720] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 15 [386,2770] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 17 [910] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 18 [5514,5340] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 19 [2824] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 20 [1118] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 24 [1972,4594] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 26 [5284,2226] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 27 [34] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 28 [5616] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 30 [3494] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 33 [3592] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 34 [3192] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 35 [1238,4012,3138] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 37 [328,5626] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 41 [3388] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 42 [3298,2030] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 43 [2330] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 44 [4068] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 47 [1198] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 51 [1462,2308] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 53 [4186] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 54 [1440] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 57 [1024] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 58 [3128,1906] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 64 [3516] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 65 [1592] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 66 [198] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 67 [5306,1754] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 69 [3570] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 70 [3794,4640,4548] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 72 [2792,1208] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 74 [3548] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 76 [3538,3378] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 77 [2622] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 78 [3368] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 80 [4058] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 82 [396] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 83 [5070,1674] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 84 [5606,1872] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 85 [2612] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 86 [12] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 87 [2652] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 90 [2802,5744,4304] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 92 [1176] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 95 [2400,3160] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 96 [2216] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 97 [5572,5802] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 98 [2458,92] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 100 [5374,2156] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 103 [3614,1484] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 104 [4628,4114] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 105 [4196] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 111 [1186] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 113 [920,3638] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 114 [4280] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 116 [3746] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 118 [2780,2686] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 119 [2064,4674,3332] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 120 [2284,4830] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 125 [4468,1344] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 126 [5732] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 128 [3896,988,208] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 129 [2040,1094] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 131 [2296] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 133 [5164] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 134 [2698,5294] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 136 [5080] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 137 [2552,1650] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 138 [2734,1848,1472,3470] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 143 [3226] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 145 [304] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 146 [232,5430] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 149 [3422,1058] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 150 [150] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 152 [280,5648] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 153 [502] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 155 [932] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 156 [2352] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 157 [1140] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 158 [2052] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 160 [3274] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 162 [754] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 163 [4650] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 164 [4492,4408] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 165 [2236,44] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 166 [418] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 167 [5502,874,3686] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 168 [3180] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 169 [5754,1308,2588,4854] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 170 [1106] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 172 [2018,5104] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 174 [682,598] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 175 [5176,4150] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 176 [1428,1556] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 177 [3036] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 178 [4938] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 179 [2674,2006] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 180 [1696] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 181 [1742] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 183 [5536] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 186 [5466] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 187 [2492,1416,4516] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 189 [5188] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 190 [4244] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 191 [3852,2192] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 192 [1392] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 193 [514,126,4078] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 194 [5684] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 195 [3286,886] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 196 [2410] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 197 [2480,2108] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 199 [646,2180,4806] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 200 [4698,5790] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 201 [4384] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 202 [3932] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 203 [4314,944] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 205 [1046,2272] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 207 [5022,634] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 208 [670,1948,1272] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 209 [3504,374] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 213 [220,1508] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 214 [5116] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 216 [1520,3650] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 217 [4396,1860] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 218 [3446] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 219 [478,3710] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 221 [1580,1164] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 222 [5720] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 223 [3398,3758] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 224 [174,2892] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 226 [3048] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 228 [3458] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 229 [3202,3956] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 230 [1730,1936,4914,2260,3580] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 233 [5140,3214] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 235 [4046] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 237 [4722,574] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 238 [0,2746] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 239 [1496,3722] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 241 [1662] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 242 [2940,3012] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 244 [3872] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 247 [718] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 248 [4758] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 249 [5034] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 252 [454] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 255 [68,4616] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 256 [5272,3770] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 257 [4208] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 258 [4292] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 260 [1764] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 262 [4326] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 263 [3782] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 265 [114,5046] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 266 [814] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 272 [1836,2976] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 273 [2868,5524,162] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 274 [3698] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 275 [1638] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 277 [4902,1260,2856,362] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 278 [1544,80] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 280 [3992,1226] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 281 [350,5548] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 282 [2468,2722] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 283 [4022] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 284 [1708] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 285 [5478] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 286 [1404] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 287 [490] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 288 [2422,3840] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 289 [1568] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 291 [4582] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 292 [466] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 296 [3626] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 298 [5386,4456,2168] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 302 [1034] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 305 [4782] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 306 [2880] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 307 [2812,5672] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 308 [2388] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 309 [790,2904] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 310 [4962] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 311 [1000,1626,22] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 315 [5594] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 316 [3944,2576,1012] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 317 [3104,4974] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 318 [2504,1602,2516] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 321 [4090,3308] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 322 [3250,2096] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 323 [4878] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 325 [4890,862] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 327 [2928,338,2248] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 331 [2988,4034] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 332 [1614] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 333 [1684,4986] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 335 [4102] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 336 [3148] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 338 [526] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 339 [956] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 341 [5406] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 342 [3558,838] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 344 [3674,5560] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 345 [1082] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 348 [5660,1882,1960,4338,5490] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 351 [4604] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 353 [1812,5092] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 356 [1284] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 360 [4746] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 362 [5454] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 364 [2662] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 365 [802] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 366 [4138] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 367 [850,3662] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 368 [3602] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 369 [186,2564,2952] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 373 [1824] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 374 [268] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 375 [5212] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 377 [766] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 378 [1152] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 379 [5328] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 382 [1320,4528] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 384 [5316,5260,1788] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 386 [1356] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 389 [2916] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 392 [2964] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 393 [2132] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 394 [562] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 395 [3116,2710] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 396 [4372,706,3092] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 397 [4558,778] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 399 [694,1296] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 400 [5778] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 401 [138,3000,3828,4268,5224] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 402 [3080] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 403 [5766,4162,406] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 404 [1776,2318] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 406 [244,4220,4256,5152] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 407 [5248] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 409 [4232,56,2528] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 411 [1924] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 413 [610,2600] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 414 [5696,4686] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 417 [730,4794,430] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 418 [2204] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 419 [2758] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 421 [5236] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 424 [4350,4504] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 427 [1248] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 429 [4842,256] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 430 [442,1532,3320] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 431 [4420,1994,4480] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 432 [3920] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 435 [2834] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 436 [2340] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 437 [1368] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 438 [3884,4662,1070] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 439 [4734,826] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 443 [4866] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 444 [4818] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 446 [538] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 448 [5636] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 449 [3434] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 452 [3024] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 453 [3482] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 454 [2144,4432,5200] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 455 [976] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 457 [2446] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 458 [3356,5442] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 459 [550,1450] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 460 [5010] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 462 [5350,5128] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 463 [1982,3980] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 466 [658,1894,4126] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 467 [3908] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 468 [3526,4950,2120,2376] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 469 [292,3968,1380,5582,2364] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 470 [2540] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 472 [3238] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 475 [898] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 477 [5708] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 478 [4444,4926] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 479 [4770] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 480 [4570,5058,3816] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 481 [2434] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 482 [586] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 483 [4174] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 484 [102] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 485 [3734] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 487 [3804] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 489 [1128,1800,3344,742] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 490 [2640] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 491 [4710] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 492 [5362,3410] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 493 [4998] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 494 [622] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 495 [316] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 496 [2076] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 497 [3068] -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src/kv1.txt 498 [5418,3262,1332] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 b/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 b/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 deleted file mode 100644 index 94754d8bd4c82..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 +++ /dev/null @@ -1 +0,0 @@ -97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 b/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 b/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 deleted file mode 100644 index c227083464fb9..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 +++ /dev/null @@ -1 +0,0 @@ -0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 b/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 b/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 deleted file mode 100644 index e6902f60ff486..0000000000000 --- a/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 +++ /dev/null @@ -1,500 +0,0 @@ -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 0 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 0 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 0 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 2 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 4 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 5 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 5 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 5 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 8 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 9 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 10 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 11 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 12 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 12 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 15 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 15 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 17 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 18 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 18 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 19 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 20 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 24 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 24 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 26 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 26 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 27 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 28 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 30 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 33 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 34 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 35 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 35 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 35 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 37 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 37 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 41 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 42 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 42 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 43 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 44 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 47 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 51 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 51 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 53 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 54 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 57 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 58 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 58 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 64 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 65 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 66 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 67 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 67 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 69 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 70 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 70 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 70 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 72 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 72 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 74 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 76 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 76 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 77 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 78 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 80 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 82 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 83 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 83 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 84 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 84 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 85 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 86 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 87 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 90 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 90 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 90 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 92 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 95 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 95 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 96 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 97 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 97 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 98 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 98 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 100 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 100 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 103 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 103 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 104 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 104 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 105 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 111 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 113 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 113 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 114 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 116 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 118 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 118 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 119 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 119 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 119 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 120 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 120 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 125 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 125 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 126 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 128 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 128 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 128 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 129 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 129 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 131 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 133 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 134 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 134 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 136 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 137 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 137 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 138 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 143 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 145 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 146 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 146 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 149 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 149 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 150 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 152 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 152 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 153 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 155 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 156 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 157 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 158 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 160 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 162 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 163 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 164 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 164 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 165 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 165 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 166 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 167 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 167 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 167 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 168 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 169 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 170 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 172 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 172 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 174 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 174 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 175 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 175 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 176 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 176 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 177 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 178 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 179 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 179 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 180 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 181 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 183 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 186 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 187 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 187 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 187 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 189 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 190 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 191 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 191 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 192 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 193 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 193 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 193 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 194 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 195 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 195 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 196 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 197 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 197 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 199 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 199 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 199 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 200 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 200 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 201 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 202 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 203 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 203 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 205 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 205 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 207 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 207 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 208 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 208 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 208 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 209 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 209 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 213 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 213 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 214 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 216 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 216 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 217 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 217 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 218 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 219 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 219 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 221 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 221 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 222 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 223 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 223 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 224 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 224 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 226 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 228 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 229 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 229 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 230 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 233 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 233 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 235 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 237 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 237 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 238 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 238 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 239 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 239 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 241 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 242 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 242 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 244 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 247 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 248 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 249 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 252 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 255 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 255 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 256 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 256 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 257 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 258 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 260 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 262 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 263 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 265 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 265 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 266 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 272 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 272 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 273 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 273 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 273 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 274 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 275 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 277 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 278 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 278 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 280 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 280 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 281 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 281 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 282 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 282 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 283 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 284 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 285 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 286 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 287 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 288 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 288 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 289 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 291 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 292 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 296 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 298 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 298 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 298 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 302 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 305 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 306 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 307 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 307 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 308 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 309 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 309 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 310 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 311 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 311 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 311 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 315 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 316 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 316 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 316 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 317 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 317 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 318 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 318 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 318 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 321 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 321 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 322 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 322 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 323 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 325 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 325 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 327 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 327 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 327 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 331 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 331 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 332 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 333 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 333 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 335 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 336 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 338 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 339 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 341 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 342 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 342 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 344 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 344 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 345 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 348 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 351 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 353 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 353 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 356 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 360 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 362 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 364 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 365 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 366 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 367 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 367 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 368 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 369 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 369 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 369 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 373 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 374 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 375 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 377 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 378 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 379 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 382 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 382 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 384 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 384 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 384 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 386 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 389 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 392 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 393 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 394 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 395 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 395 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 396 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 396 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 396 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 397 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 397 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 399 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 399 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 400 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 401 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 402 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 403 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 403 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 403 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 404 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 404 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 406 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 407 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 409 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 409 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 409 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 411 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 413 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 413 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 414 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 414 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 417 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 417 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 417 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 418 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 419 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 421 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 424 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 424 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 427 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 429 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 429 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 430 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 430 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 430 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 431 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 431 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 431 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 432 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 435 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 436 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 437 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 438 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 438 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 438 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 439 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 439 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 443 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 444 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 446 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 448 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 449 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 452 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 453 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 454 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 454 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 454 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 455 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 457 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 458 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 458 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 459 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 459 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 460 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 462 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 462 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 463 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 463 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 466 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 466 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 466 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 467 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 468 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 469 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 470 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 472 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 475 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 477 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 478 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 478 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 479 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 480 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 480 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 480 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 481 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 482 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 483 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 484 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 485 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 487 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 489 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 490 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 491 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 492 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 492 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 493 3672 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 494 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 495 56 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 496 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 497 2449 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 498 1249 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 498 4854 -file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sharkWarehouse1091803796917701818/src_index_test_rc/000000_0 498 2449 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 b/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 deleted file mode 100644 index e69de29bb2d1d..0000000000000 diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties index 9fdb526d945e0..5bc08062d30eb 100644 --- a/sql/hive/src/test/resources/log4j.properties +++ b/sql/hive/src/test/resources/log4j.properties @@ -42,6 +42,12 @@ log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF +log4j.additivity.hive.log=false +log4j.logger.hive.log=OFF + +log4j.additivity.parquet.hadoop.ParquetRecordReader=false +log4j.logger.parquet.hadoop.ParquetRecordReader=OFF + log4j.additivity.hive.ql.metadata.Hive=false log4j.logger.hive.ql.metadata.Hive=OFF diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q new file mode 100644 index 0000000000000..c640ca148b70b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs.q @@ -0,0 +1,43 @@ + +select + cdouble + ,Round(cdouble, 2) + ,Floor(cdouble) + ,Ceil(cdouble) + ,Rand(98007) as rnd + ,Exp(ln(cdouble)) + ,Ln(cdouble) + ,Ln(cfloat) + ,Log10(cdouble) + -- Use log2 as a representative function to test all input types. + ,Log2(cdouble) + ,Log2(cfloat) + ,Log2(cbigint) + ,Log2(cint) + ,Log2(csmallint) + ,Log2(ctinyint) + ,Log(2.0, cdouble) + ,Pow(log2(cdouble), 2.0) + ,Power(log2(cdouble), 2.0) + ,Sqrt(cdouble) + ,Sqrt(cbigint) + ,Bin(cbigint) + ,Hex(cdouble) + ,Conv(cbigint, 10, 16) + ,Abs(cdouble) + ,Abs(ctinyint) + ,Pmod(cint, 3) + ,Sin(cdouble) + ,Asin(cdouble) + ,Cos(cdouble) + ,ACos(cdouble) + ,Atan(cdouble) + ,Degrees(cdouble) + ,Radians(cdouble) + ,Positive(cdouble) + ,Positive(cbigint) + ,Negative(cdouble) + ,Sign(cdouble) + ,Sign(cbigint) +from alltypesorc order by rnd limit 400; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv new file mode 100644 index 0000000000000..51f231008f6d2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_00.qv @@ -0,0 +1 @@ +SET hive.vectorized.execution.enabled = false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv new file mode 100644 index 0000000000000..18e02dc854baf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientcompare/vectorized_math_funcs_01.qv @@ -0,0 +1 @@ +SET hive.vectorized.execution.enabled = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q index 4881757a4613a..4193315d30043 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q @@ -1,9 +1,9 @@ set hive.exec.concatenate.check.index=true; create table src_rc_concatenate_test(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_concatenate_test; -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_concatenate_test; -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_concatenate_test; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test; +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test; +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test; show table extended like `src_rc_concatenate_test`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q index 5f9d5ef9ca09e..8cbb25cfa9725 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q @@ -2,7 +2,7 @@ create table if not exists alter_part_invalidspec(key string, value string ) partitioned by (year string, month string) stored as textfile ; -- Load data -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='10'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='12'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='10'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_invalidspec partition (year='1996', month='12'); alter table alter_part_invalidspec partition (year='1997') enable no_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q index 92af30b6b5a15..3c0ff02b1ac1d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q @@ -2,8 +2,8 @@ create table if not exists alter_part_nodrop_part(key string, value string ) partitioned by (year string, month string) stored as textfile ; -- Load data -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='10'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='12'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='10'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_part partition (year='1996', month='12'); alter table alter_part_nodrop_part partition (year='1996') enable no_drop; alter table alter_part_nodrop_part drop partition (year='1996'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q index 135411fd320ef..f2135b1aa02e4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q @@ -2,8 +2,8 @@ create table if not exists alter_part_nodrop_table(key string, value string ) partitioned by (year string, month string) stored as textfile ; -- Load data -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='10'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='12'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='10'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_nodrop_table partition (year='1996', month='12'); alter table alter_part_nodrop_table partition (year='1996') enable no_drop; drop table alter_part_nodrop_table; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q index 899145deaf8c5..7376d8bfe4a74 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q @@ -2,8 +2,8 @@ create table if not exists alter_part_offline (key string, value string ) partitioned by (year string, month string) stored as textfile ; -- Load data -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='10'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='12'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='10'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_offline partition (year='1996', month='12'); alter table alter_part_offline partition (year='1996') disable offline; select * from alter_part_offline where year = '1996'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q index 26ba2878903fe..be971f1849869 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q @@ -1,5 +1,5 @@ create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q index 6e51c2f7629c9..4babdda2dbe2a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q @@ -1,5 +1,5 @@ create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q index 2d4ce0b9f6051..3af807ef61217 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q @@ -1,5 +1,5 @@ create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; insert overwrite table alter_rename_partition partition (pCol1='old_part1:', pcol2='old_part2:') select col1 from alter_rename_partition_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q index fdf20f850e8f8..9e8bcbd1bbf78 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q @@ -1,2 +1,3 @@ +set hive.support.quoted.identifiers=none; -- TOK_TABLE_OR_COL explain select * from (select `.*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q index de59bc579a0fe..33d4aed3cd9ad 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q @@ -1,2 +1,3 @@ +set hive.support.quoted.identifiers=none; -- DOT explain select * from (select a.`[kv].*`, b.`[kv].*` from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q new file mode 100644 index 0000000000000..78a97019f192e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_non_existent_tbl.q @@ -0,0 +1 @@ +analyze table nonexistent compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q index bea25391628a6..130b37b5c9d52 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q @@ -14,5 +14,5 @@ create table tstsrcpart like srcpart; -- to be thrown during the LOAD step. This former behavior is tested -- in clientpositive/archive_corrupt.q -load data local inpath '../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11'); +load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q new file mode 100644 index 0000000000000..a1709dae5f5b8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addjar.q @@ -0,0 +1,7 @@ +set hive.security.authorization.enabled=true; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; + +-- running a sql query to initialize the authorization - not needed in real HS2 mode +show tables; + +add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q new file mode 100644 index 0000000000000..8abdd2b3cde8b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_addpartition.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +set user.name=user1; +-- check add partition without insert privilege +create table tpart(i int, j int) partitioned by (k string); + +set user.name=user2; +alter table tpart add partition (k = 'abc'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q new file mode 100644 index 0000000000000..f716262e23bbb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if alter table owner fails +-- for now, alter db owner is allowed only for admin + +create database dbao; +alter database dbao set owner user user2; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q new file mode 100644 index 0000000000000..f9049350180ee --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_alter_db_owner_default.q @@ -0,0 +1,8 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if alter table owner fails +alter database default set owner user user1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q new file mode 100644 index 0000000000000..de91e91923308 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_all_role.q @@ -0,0 +1,6 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; +create role all; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q new file mode 100644 index 0000000000000..42a42f65b28a9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_default_role.q @@ -0,0 +1,6 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; +create role default; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q new file mode 100644 index 0000000000000..0d14cde6d5460 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_cannot_create_none_role.q @@ -0,0 +1,6 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; +create role None; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q new file mode 100644 index 0000000000000..d5ea284f14749 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_caseinsensitivity.q @@ -0,0 +1,17 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; + +create role testrole; +show roles; +drop role TESTROLE; +show roles; +create role TESTROLE; +show roles; +grant role testROLE to user hive_admin_user; +set role testrolE; +set role adMin; +show roles; +create role TESTRoLE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q new file mode 100644 index 0000000000000..02bbe090cfba7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func1.q @@ -0,0 +1,7 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=hive_test_user; + +-- permanent function creation should fail for non-admin roles +create function perm_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q new file mode 100644 index 0000000000000..8760fa8d82259 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_func2.q @@ -0,0 +1,8 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=hive_test_user; + +-- temp function creation should fail for non-admin roles +create temporary function temp_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q new file mode 100644 index 0000000000000..c904a100c515c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_macro1.q @@ -0,0 +1,8 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=hive_test_user; + +-- temp macro creation should fail for non-admin roles +create temporary macro mymacro1(x double) x * x; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q new file mode 100644 index 0000000000000..a84fe64bd618e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_create_role_no_admin.q @@ -0,0 +1,3 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +-- this test will fail because hive_test_user is not in admin role. +create role r1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q new file mode 100644 index 0000000000000..9b1f2ea6c6acb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_createview.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +-- check create view without select privileges +create table t1(i int); +set user.name=user1; +create view v1 as select * from t1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q new file mode 100644 index 0000000000000..1cf74a365d79e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_ctas.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +-- check query without select privilege fails +create table t1(i int); + +set user.name=user1; +create table t2 as select * from t1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q new file mode 100644 index 0000000000000..47663c9bb93e3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_desc_table_nosel.q @@ -0,0 +1,14 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if alter table fails as different user +create table t1(i int); +desc t1; + +grant all on table t1 to user user2; +revoke select on table t1 from user user2; + +set user.name=user2; +desc t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q new file mode 100644 index 0000000000000..7d47a7b64967b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_dfs.q @@ -0,0 +1,7 @@ +set hive.security.authorization.enabled=true; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory; + +-- running a sql query to initialize the authorization - not needed in real HS2 mode +show tables; +dfs -ls ${system:test.tmp.dir}/ + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q new file mode 100644 index 0000000000000..64b300c8d9b2f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_disallow_transform.q @@ -0,0 +1,3 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set role ALL; +SELECT TRANSFORM (*) USING 'cat' AS (key, value) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q new file mode 100644 index 0000000000000..edeae9b71d7ac --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_cascade.q @@ -0,0 +1,22 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- ensure that drop database cascade works +create database dba1; +create table dba1.tab1(i int); +drop database dba1 cascade; + +-- check if drop database fails if the db has a table for which user does not have permission +create database dba2; +create table dba2.tab2(i int); + +set user.name=hive_admin_user; +set role ADMIN; +alter database dba2 set owner user user2; + +set user.name=user2; +show current roles; +drop database dba2 cascade ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q new file mode 100644 index 0000000000000..46d4d0f92c8e3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_db_empty.q @@ -0,0 +1,27 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if changing owner and dropping as other user works +create database dba1; + +set user.name=hive_admin_user; +set role ADMIN; +alter database dba1 set owner user user2; + +set user.name=user2; +show current roles; +drop database dba1; + + +set user.name=user1; +-- check if dropping db as another user fails +show current roles; +create database dba2; + +set user.name=user2; +show current roles; + +drop database dba2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q new file mode 100644 index 0000000000000..a7aa17f5abfcf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_drop_role_no_admin.q @@ -0,0 +1,10 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; +show current roles; +create role r1; +set role ALL; +show current roles; +drop role r1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q new file mode 100644 index 0000000000000..f05e9458fa804 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_droppartition.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_drop_part_1; + +-- check drop partition without delete privilege +create table tpart(i int, j int) partitioned by (k string); +alter table tpart add partition (k = 'abc') location 'file:${system:test.tmp.dir}/authz_drop_part_1' ; +set user.name=user1; +alter table tpart drop partition (k = 'abc'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q new file mode 100644 index 0000000000000..d969e39027e99 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_create_db.q @@ -0,0 +1,5 @@ +set hive.security.authorization.enabled=true; + +create database db_to_fail; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q new file mode 100644 index 0000000000000..87719b0043e2e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_drop_db.q @@ -0,0 +1,5 @@ +set hive.security.authorization.enabled=false; +create database db_fail_to_drop; +set hive.security.authorization.enabled=true; + +drop database db_fail_to_drop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q new file mode 100644 index 0000000000000..f3c86b97ce76f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_allpriv.q @@ -0,0 +1,14 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE table_priv_allf(i int); + +-- grant insert to user2 WITH grant option +GRANT INSERT ON table_priv_allf TO USER user2 with grant option; + +set user.name=user2; +-- try grant all to user3, without having all privileges +GRANT ALL ON table_priv_allf TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q new file mode 100644 index 0000000000000..7808cb3ec7b39 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_dup.q @@ -0,0 +1,16 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE tauth_gdup(i int); + +-- It should be possible to revert owners privileges +revoke SELECT ON tauth_gdup from user user1; + +show grant user user1 on table tauth_gdup; + +-- Owner already has all privileges granted, another grant would become duplicate +-- and result in error +GRANT INSERT ON tauth_gdup TO USER user1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q new file mode 100644 index 0000000000000..8dc8e45a79075 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail1.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE table_priv_gfail1(i int); + +set user.name=user2; +-- try grant insert to user3 as user2 +GRANT INSERT ON table_priv_gfail1 TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q new file mode 100644 index 0000000000000..d51c1c3507eef --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_grant_table_fail_nogrant.q @@ -0,0 +1,14 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE table_priv_gfail1(i int); + +-- grant insert to user2 WITHOUT grant option +GRANT INSERT ON table_priv_gfail1 TO USER user2; + +set user.name=user2; +-- try grant insert to user3 +GRANT INSERT ON table_priv_gfail1 TO USER user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q new file mode 100644 index 0000000000000..2fa3cb260b07e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noinspriv.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +-- check insert without select priv +create table t1(i int); + +set user.name=user1; +create table user2tab(i int); +insert into table t1 select * from user2tab; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q new file mode 100644 index 0000000000000..b9bee4ea40d40 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_insert_noselectpriv.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +-- check insert without select priv +create table t1(i int); + +set user.name=user1; +create table t2(i int); +insert into table t2 select * from t1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q new file mode 100644 index 0000000000000..2a1da23daeb18 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v1.q @@ -0,0 +1,6 @@ +create table if not exists authorization_invalid_v1 (key int, value string); +grant delete on table authorization_invalid_v1 to user hive_test_user; +drop table authorization_invalid_v1; + + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q new file mode 100644 index 0000000000000..9c724085d9901 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_invalid_priv_v2.q @@ -0,0 +1,5 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; + +create table if not exists authorization_invalid_v2 (key int, value string); +grant index on table authorization_invalid_v2 to user hive_test_user; +drop table authorization_invalid_v2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q new file mode 100644 index 0000000000000..8a3300cb2e378 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_rename.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if alter table fails as different user +create table t1(i int); + +set user.name=user2; +alter table t1 rename to tnew1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q new file mode 100644 index 0000000000000..0172c4c74c82d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_alter_tab_serdeprop.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if alter table fails as different user +create table t1(i int); + +set user.name=user2; +ALTER TABLE t1 SET SERDEPROPERTIES ('field.delim' = ','); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q new file mode 100644 index 0000000000000..2d0e52da008d8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_tab.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if create table fails as different user +create table t1(i int); + +set user.name=user2; +drop table t1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q new file mode 100644 index 0000000000000..76bbab42b3750 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_not_owner_drop_view.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if create table fails as different user +create table t1(i int); +create view vt1 as select * from t1; + +set user.name=user2; +drop view vt1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q new file mode 100644 index 0000000000000..bbf3b66970b6a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_priv_current_role_neg.q @@ -0,0 +1,29 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; + +-- the test verifies that authorization is happening with privileges of the current roles + +-- grant privileges with grant option for table to role2 +create role role2; +grant role role2 to user user2; +create table tpriv_current_role(i int); +grant all on table tpriv_current_role to role role2 with grant option; + +set user.name=user2; +-- switch to user2 + +-- by default all roles should be in current roles, and grant to new user should work +show current roles; +grant all on table tpriv_current_role to user user3; + +set role role2; +-- switch to role2, grant should work +grant all on table tpriv_current_role to user user4; +show grant user user4 on table tpriv_current_role; + +set role PUBLIC; +-- set role to public, should fail as role2 is not one of the current roles +grant all on table tpriv_current_role to user user5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q new file mode 100644 index 0000000000000..002389f203e25 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_create.q @@ -0,0 +1 @@ +create role PUBLIC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q new file mode 100644 index 0000000000000..69c5a8de8b05f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_public_drop.q @@ -0,0 +1 @@ +drop role PUBLIC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q new file mode 100644 index 0000000000000..e19bf370fa077 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail1.q @@ -0,0 +1,14 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE table_priv_rfail1(i int); + +-- grant insert to user2 +GRANT INSERT ON table_priv_rfail1 TO USER user2; + +set user.name=user3; +-- try dropping the privilege as user3 +REVOKE INSERT ON TABLE table_priv_rfail1 FROM USER user2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q new file mode 100644 index 0000000000000..4b0cf3286ae71 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_revoke_table_fail2.q @@ -0,0 +1,18 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE table_priv_rfai2(i int); + +-- grant insert to user2 +GRANT INSERT ON table_priv_rfai2 TO USER user2; +GRANT SELECT ON table_priv_rfai2 TO USER user3 WITH GRANT OPTION; + +set user.name=user3; +-- grant select as user3 to user 2 +GRANT SELECT ON table_priv_rfai2 TO USER user2; + +-- try dropping the privilege as user3 +REVOKE INSERT ON TABLE table_priv_rfai2 FROM USER user2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q new file mode 100644 index 0000000000000..a819d204f56b6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles1.q @@ -0,0 +1,12 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; +-- this is applicable to any security mode as check is in metastore +create role role1; +create role role2; +grant role role1 to role role2; + +-- this will create a cycle +grant role role2 to role role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q new file mode 100644 index 0000000000000..423f030630b6c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_cycles2.q @@ -0,0 +1,24 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=hive_admin_user; +set role ADMIN; +-- this is applicable to any security mode as check is in metastore + +create role role1; + +create role role2; +grant role role2 to role role1; + +create role role3; +grant role role3 to role role2; + +create role role4; +grant role role4 to role role3; + +create role role5; +grant role role5 to role role4; + +-- this will create a cycle in middle of the hierarchy +grant role role2 to role role4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q new file mode 100644 index 0000000000000..c5c500a71251f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_role_grant.q @@ -0,0 +1,22 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; + +set role ADMIN; + +---------------------------------------- +-- role granting with admin option +-- since user2 doesn't have admin option for role_noadmin, last grant should fail +---------------------------------------- + +create role role_noadmin; +create role src_role_wadmin; +grant src_role_wadmin to user user2 with admin option; +grant role_noadmin to user user2; +show role grant user user2; + + +set user.name=user2; +set role role_noadmin; +grant src_role_wadmin to user user3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q new file mode 100644 index 0000000000000..d9f4c7cdb850b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_rolehierarchy_privs.q @@ -0,0 +1,74 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +set user.name=hive_admin_user; +show current roles; +set role ADMIN; + +---------- +-- create the following user, role mapping +-- user1 -> role1 -> role2 -> role3 +---------- + +create role role1; +grant role1 to user user1; + +create role role2; +grant role2 to role role1; + +create role role3; +grant role3 to role role2; + + +create table t1(i int); +grant select on t1 to role role3; + +set user.name=user1; +show current roles; +select * from t1; + +set user.name=hive_admin_user; +show current roles; +grant select on t1 to role role2; + + +set user.name=user1; +show current roles; +select * from t1; + +set user.name=hive_admin_user; +set role ADMIN; +show current roles; +revoke select on table t1 from role role2; + + +create role role4; +grant role4 to user user1; +grant role3 to role role4;; + +set user.name=user1; +show current roles; +select * from t1; + +set user.name=hive_admin_user; +show current roles; +set role ADMIN; + +-- Revoke role3 from hierarchy one at a time and check permissions +-- after revoking from both, select should fail +revoke role3 from role role2; + +set user.name=user1; +show current roles; +select * from t1; + +set user.name=hive_admin_user; +show current roles; +set role ADMIN; +revoke role3 from role role4; + +set user.name=user1; +show current roles; +select * from t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q new file mode 100644 index 0000000000000..39871793af398 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select.q @@ -0,0 +1,9 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +-- check query without select privilege fails +create table t1(i int); + +set user.name=user1; +select * from t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q new file mode 100644 index 0000000000000..a4071cd0d4d87 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_select_view.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +-- check create view without select privileges +create table t1(i int); +create view v1 as select * from t1; +set user.name=user1; +select * from v1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q new file mode 100644 index 0000000000000..9ba3a82a5608e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg1.q @@ -0,0 +1,6 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; + +-- an error should be thrown if 'set role ' is done for role that does not exist + +set role nosuchroleexists; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q new file mode 100644 index 0000000000000..03f748fcc9b7a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_set_role_neg2.q @@ -0,0 +1,16 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; + +-- an error should be thrown if 'set role ' is done for role that does not exist + +create role rset_role_neg; +grant role rset_role_neg to user user2; + +set user.name=user2; +set role rset_role_neg; +set role public; +set role nosuchroleexists;; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q new file mode 100644 index 0000000000000..d8190de950de7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_parts_nosel.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- check if alter table fails as different user +create table t_show_parts(i int) partitioned by (j string); + +set user.name=user2; +show partitions t_show_parts; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q new file mode 100644 index 0000000000000..2afe87fc30c9e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_no_admin.q @@ -0,0 +1,3 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +-- This test will fail because hive_test_user is not in admin role +show principals role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q new file mode 100644 index 0000000000000..69cea2f2673f0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_role_principals_v1.q @@ -0,0 +1,2 @@ +-- This test will fail because the command is not currently supported in auth mode v1 +show principals role1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q new file mode 100644 index 0000000000000..0fc9fca940c39 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_show_roles_no_admin.q @@ -0,0 +1,3 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +-- This test will fail because hive_test_user is not in admin role +show roles; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q new file mode 100644 index 0000000000000..285600b23a149 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_truncate.q @@ -0,0 +1,9 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +-- check add partition without insert privilege +create table t1(i int, j int); +set user.name=user1; +truncate table t1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q new file mode 100644 index 0000000000000..d82ac710cc3ba --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_add_partition.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part; +dfs -touchz ${system:test.tmp.dir}/a_uri_add_part/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/a_uri_add_part/1.txt; + +create table tpart(i int, j int) partitioned by (k string); +alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/a_uri_add_part/'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q new file mode 100644 index 0000000000000..d38ba74d9006a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_alterpart_loc.q @@ -0,0 +1,16 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc_perm; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_alterpart_loc; +dfs -touchz ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/az_uri_alterpart_loc/1.txt; + +create table tpart(i int, j int) partitioned by (k string); +alter table tpart add partition (k = 'abc') location '${system:test.tmp.dir}/az_uri_alterpart_loc_perm/'; + +alter table tpart partition (k = 'abc') set location '${system:test.tmp.dir}/az_uri_alterpart_loc/'; + + +-- Attempt to set partition to location without permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q new file mode 100644 index 0000000000000..c446b8636fb32 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_altertab_setloc.q @@ -0,0 +1,13 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_altertab_setloc; +dfs -touchz ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt; + +create table t1(i int); + +alter table t1 set location '${system:test.tmp.dir}/az_uri_altertab_setloc/1.txt' + +-- Attempt to set location of table to a location without permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q new file mode 100644 index 0000000000000..c8e1fb43ee317 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table1.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab1; +dfs -touchz ${system:test.tmp.dir}/a_uri_crtab1/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab1/1.txt; + +create table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext'; + +-- Attempt to create table with dir that does not have write permission should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q new file mode 100644 index 0000000000000..c8549b4563b2f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_create_table_ext.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_crtab_ext; +dfs -touchz ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/a_uri_crtab_ext/1.txt; + +create external table t1(i int) location '${system:test.tmp.dir}/a_uri_crtab_ext'; + +-- Attempt to create table with dir that does not have write permission should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q new file mode 100644 index 0000000000000..edfdf5a8fc407 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_createdb.q @@ -0,0 +1,12 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_createdb; +dfs -touchz ${system:test.tmp.dir}/az_uri_createdb/1.txt; +dfs -chmod 300 ${system:test.tmp.dir}/az_uri_createdb/1.txt; + +create database az_test_db location '${system:test.tmp.dir}/az_uri_createdb/'; + +-- Attempt to create db for dir without sufficient permissions should fail + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q new file mode 100644 index 0000000000000..81763916a0b81 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_export.q @@ -0,0 +1,22 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +set hive.test.mode=true; +set hive.test.mode.prefix=; +set hive.test.mode.nosamplelist=export_auth_uri; + + +create table export_auth_uri ( dep_id int comment "department id") + stored as textfile; + +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/temp; +dfs -rmr target/tmp/ql/test/data/exports/export_auth_uri; + + +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/export_auth_uri/; +dfs -chmod 555 target/tmp/ql/test/data/exports/export_auth_uri; + +export table export_auth_uri to 'ql/test/data/exports/export_auth_uri'; + +-- Attempt to export to location without sufficient permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q new file mode 100644 index 0000000000000..4ea4dc0a4747a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_import.q @@ -0,0 +1,25 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +set hive.test.mode=true; +set hive.test.mode.prefix=; +set hive.test.mode.nosamplelist=import_auth_uri; + + +create table import_auth_uri ( dep_id int comment "department id") + stored as textfile; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/import_auth_uri/temp; +dfs -rmr target/tmp/ql/test/data/exports/import_auth_uri; +export table import_auth_uri to 'ql/test/data/exports/import_auth_uri'; +drop table import_auth_uri; + +dfs -touchz target/tmp/ql/test/data/exports/import_auth_uri/1.txt; +dfs -chmod 555 target/tmp/ql/test/data/exports/import_auth_uri/1.txt; + +create database importer; +use importer; + +import from 'ql/test/data/exports/import_auth_uri'; + +-- Attempt to import from location without sufficient permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q new file mode 100644 index 0000000000000..1a8f9cb2ad197 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_index.q @@ -0,0 +1,13 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_index; +dfs -touchz ${system:test.tmp.dir}/az_uri_index/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/az_uri_index/1.txt; + + +create table t1(i int); +create index idt1 on table t1 (i) as 'COMPACT' WITH DEFERRED REBUILD LOCATION '${system:test.tmp.dir}/az_uri_index/'; + +-- Attempt to use location for index that does not have permissions should fail diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q new file mode 100644 index 0000000000000..81b6e522c1abb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert.q @@ -0,0 +1,14 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert; +dfs -touchz ${system:test.tmp.dir}/az_uri_insert/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert/1.txt; + +create table t1(i int, j int); + +insert overwrite directory '${system:test.tmp.dir}/az_uri_insert/' select * from t1; + +-- Attempt to insert into uri without permissions should fail + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q new file mode 100644 index 0000000000000..0a2fd8919f455 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_insert_local.q @@ -0,0 +1,14 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/az_uri_insert_local; +dfs -touchz ${system:test.tmp.dir}/az_uri_insert_local/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/az_uri_insert_local/1.txt; + +create table t1(i int, j int); + +insert overwrite local directory '${system:test.tmp.dir}/az_uri_insert_local/' select * from t1; + +-- Attempt to insert into uri without permissions should fail + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q new file mode 100644 index 0000000000000..6af41f0cdaa23 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_uri_load_data.q @@ -0,0 +1,11 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/authz_uri_load_data; +dfs -touchz ${system:test.tmp.dir}/authz_uri_load_data/1.txt; +dfs -chmod 555 ${system:test.tmp.dir}/authz_uri_load_data/1.txt; + +create table t1(i int); +load data inpath 'pfile:${system:test.tmp.dir}/authz_uri_load_data/' overwrite into table t1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q new file mode 100644 index 0000000000000..d8beac370d4b6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_create_tbl.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set hive.security.authorization.enabled=true; +set user.name=user33; +create database db23221; +use db23221; + +set user.name=user44; +create table twew221(a string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q new file mode 100644 index 0000000000000..bfd3165237774 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_grant_public.q @@ -0,0 +1 @@ +grant role PUBLIC to user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q new file mode 100644 index 0000000000000..2b29822371b19 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorize_revoke_public.q @@ -0,0 +1 @@ +revoke role PUBLIC from user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q index 048a8fd5cfb8e..6bebb8942d613 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q @@ -1,19 +1,19 @@ CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket22.txt' +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -- The number of buckets in the 2 tables above (being joined later) dont match. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q index 9478a2f1b989c..802fcd903c0ac 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q @@ -8,10 +8,10 @@ into 2 BUCKETS stored as textfile; create table table2(key string, value string) clustered by (value, key) into 2 BUCKETS stored as textfile; -load data local inpath '../data/files/T1.txt' overwrite into table table1; +load data local inpath '../../data/files/T1.txt' overwrite into table table1; -load data local inpath '../data/files/T1.txt' overwrite into table table2; -load data local inpath '../data/files/T2.txt' overwrite into table table2; +load data local inpath '../../data/files/T1.txt' overwrite into table table2; +load data local inpath '../../data/files/T2.txt' overwrite into table table2; set hive.optimize.bucketmapjoin = true; set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q index 69afe0ae700ed..ac5abebb0b4b6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q @@ -8,13 +8,13 @@ into 2 BUCKETS stored as textfile; create table table2(key string, value string) clustered by (value, key) into 2 BUCKETS stored as textfile; -load data local inpath '../data/files/T1.txt' overwrite into table table1 partition (ds='1'); -load data local inpath '../data/files/T2.txt' overwrite into table table1 partition (ds='1'); +load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='1'); +load data local inpath '../../data/files/T2.txt' overwrite into table table1 partition (ds='1'); -load data local inpath '../data/files/T1.txt' overwrite into table table1 partition (ds='2'); +load data local inpath '../../data/files/T1.txt' overwrite into table table1 partition (ds='2'); -load data local inpath '../data/files/T1.txt' overwrite into table table2; -load data local inpath '../data/files/T2.txt' overwrite into table table2; +load data local inpath '../../data/files/T1.txt' overwrite into table table2; +load data local inpath '../../data/files/T2.txt' overwrite into table table2; set hive.optimize.bucketmapjoin = true; set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q deleted file mode 100644 index 0ff44779657ee..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q +++ /dev/null @@ -1,2 +0,0 @@ -EXPLAIN -SELECT x.key, x.value as key FROM SRC x CLUSTER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q index af923504c8c4f..b4887c4115854 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q @@ -3,12 +3,12 @@ DROP TABLE Employee_Part; CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); -- dynamic partitioning syntax explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q index d9725ddc7fdf6..2f8e9271ddd34 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q @@ -3,12 +3,12 @@ DROP TABLE Employee_Part; CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); -- don't specify all partitioning keys explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q index eb73962ba133c..34f91fc8d1de8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q @@ -3,12 +3,12 @@ DROP TABLE Employee_Part; CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); -- specify invalid values for the partitioning keys explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q index dbfaaecbdf614..49d89dd12132b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q @@ -3,12 +3,12 @@ DROP TABLE Employee_Part; CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double, country string) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); -LOAD DATA LOCAL INPATH "../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='2000.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='4000.0', country='USA'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3500.0', country='UK'); +LOAD DATA LOCAL INPATH "../../data/files/employee2.dat" INTO TABLE Employee_Part partition(employeeSalary='3000.0', country='UK'); -- specify partitioning clause multiple times explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q index ca8548958fcd6..a4e0056bff370 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q @@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none ( avgTimeOnSite int) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; +LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; explain analyze table UserVisits_web_text_none compute statistics for columns destIP; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q index 5bbd70d86b297..85a5f0a021940 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q @@ -8,7 +8,7 @@ CREATE TABLE table_complex_type ( d MAP> ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type; +LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table_complex_type; explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q index ca8548958fcd6..a4e0056bff370 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q @@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none ( avgTimeOnSite int) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; +LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; explain analyze table UserVisits_web_text_none compute statistics for columns destIP; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q new file mode 100644 index 0000000000000..c314a940f95c2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compile_processor.q @@ -0,0 +1,8 @@ + +compile `import org.apache.hadoop.hive.ql.exec.UDF \; +public class Pyth extsfgsfgfsends UDF { + public double evaluate(double a, double b){ + return Math.sqrt((a*a) + (b*b)) \; + } +} ` AS GROOVY NAMED Pyth.groovy; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q new file mode 100644 index 0000000000000..5974811280350 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compute_stats_long.q @@ -0,0 +1,7 @@ +create table tab_int(a int); + +-- insert some data +LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int; + +-- compute stats should raise an error since the number of bit vectors > 1024 +select compute_stats(a, 10000) from tab_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q new file mode 100644 index 0000000000000..3b71e00b2eaa2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_class.q @@ -0,0 +1 @@ +create function default.badfunc as 'my.nonexistent.class'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q new file mode 100644 index 0000000000000..ae95391edd3e5 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonexistent_db.q @@ -0,0 +1 @@ +create function nonexistentdb.badfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q new file mode 100644 index 0000000000000..2083064593299 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_function_nonudf_class.q @@ -0,0 +1 @@ +create function default.badfunc as 'java.lang.String'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q new file mode 100644 index 0000000000000..2160b4719662b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_recursion.q @@ -0,0 +1,4 @@ +explain +with q1 as ( select key from q2 where key = '5'), +q2 as ( select key from q1 where key = '5') +select * from (select key from q1) a; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q new file mode 100644 index 0000000000000..e52a1d97db801 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cte_with_in_subquery.q @@ -0,0 +1 @@ +select * from (with q1 as ( select key from q2 where key = '5') select * from q1) a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q deleted file mode 100644 index b7fac0d3df346..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q +++ /dev/null @@ -1,2 +0,0 @@ --- Not in YYYY-MM-DD format -SELECT DATE '2001-1-1' FROM src LIMIT 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q new file mode 100644 index 0000000000000..1c658c79b99ea --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodblock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop database if exists drop_nodblock; +create database drop_nodblock; +lock database drop_nodblock shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q new file mode 100644 index 0000000000000..ef4b323f063b6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_nodbunlock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop database if exists drop_nodbunlock; +create database drop_nodbunlock; +unlock database drop_nodbunlock; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q new file mode 100644 index 0000000000000..4a0c6c25c67c7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notablelock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop table if exists drop_notablelock; +create table drop_notablelock (c int); +lock table drop_notablelock shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q new file mode 100644 index 0000000000000..0b00046579f43 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dbtxnmgr_notableunlock.q @@ -0,0 +1,6 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +drop table if exists drop_notableunlock; +create table drop_notableunlock (c int); +unlock table drop_notableunlock; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q index 7b0c92311a11a..0bd6985e031b6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q @@ -1,4 +1,4 @@ -ADD JAR ../data/files/TestSerDe.jar; -DELETE JAR ../data/files/TestSerDe.jar; +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; +DELETE JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q new file mode 100644 index 0000000000000..892ef00e3f86c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_func_nonexistent.q @@ -0,0 +1,3 @@ +set hive.exec.drop.ignorenonexistent=false; +-- Can't use DROP FUNCTION if the function doesn't exist and IF EXISTS isn't specified +drop function nonexistent_function; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q deleted file mode 100644 index 4d238d73a9116..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q +++ /dev/null @@ -1,11 +0,0 @@ -create table ptestfilter (a string, b int) partitioned by (c string, d int); -describe ptestfilter; - -alter table ptestfilter add partition (c='US', d=1); -alter table ptestfilter add partition (c='US', d=2); -show partitions ptestfilter; - -alter table ptestfilter drop partition (c='US', d<'2'); - - - diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q index 0be2e71c94dff..0ad99d100dc07 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q @@ -8,7 +8,7 @@ create table source_table like srcpart; create table dest_table like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE source_table partition(ds='2008-04-08', hr=11); -- Tests creating dynamic partitions with characters not in the whitelist (i.e. 9) -- If the directory is not empty the hook will throw an error, instead the error should come from the metastore diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q index e4f0daca92bd1..ca60d047efdd5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q @@ -3,8 +3,8 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='h1'); -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='h2'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h1'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='h2'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q index 4d1e0a62a431f..7083edc32b98d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q @@ -8,5 +8,5 @@ ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; --- exchange_part_test2 table partition (ds='2013-04-05') already exists thus this query will fail +-- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q index 23777db3ea93d..6dfe81a8b0568 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q @@ -9,5 +9,5 @@ ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='3'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; --- exchange_part_test2 table partition (ds='2013-04-05', hr='3') already exists thus this query will fail +-- exchange_part_test1 table partition (ds='2013-04-05') already exists thus this query will fail alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q index 350bf248acc9c..60671e52e05d5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q @@ -9,5 +9,5 @@ ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; --- exchange_part_test2 table partition (ds='2013-04-05', hr='1') already exists thus this query will fail +-- exchange_part_test2 table partition (ds='2013-04-05') already exists thus this query will fail alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q index 81944b3330853..38c0eda2368bd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q @@ -2,5 +2,5 @@ CREATE TABLE exchange_part_test1 (f1 string) PARTITIONED BY (ds STRING); CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING); SHOW PARTITIONS exchange_part_test1; --- exchange_part_test1 partition (ds='2013-04-05') does not exist thus this query will fail +-- exchange_part_test2 partition (ds='2013-04-05') does not exist thus this query will fail alter table exchange_part_test1 exchange partition (ds='2013-04-05') with table exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q index d86ecd5785d02..6ffc33acb92ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'nosuchschema://nosuchauthority/ql/test/data/exports/exim_department'; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q index 5f3223152f766..970e6463e24a5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -16,9 +16,9 @@ use importer; create table exim_department ( dep_id int comment "department identifier") stored as textfile tblproperties("maker"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q index d7204dc478d25..358918363d830 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -27,12 +27,12 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "iso code", emp_state string comment "free-form text") stored as textfile tblproperties("maker"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q index 6cd7eda455ee6..45268c21c00e3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -18,6 +18,6 @@ create table exim_department ( dep_key int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q index 7f3f577c433b3..cad6c90fd316e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -18,6 +18,6 @@ create table exim_department ( dep_id int comment "department id", dep_name stri tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q index d3ec9fff82bf9..f5f904f42af5a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -18,6 +18,6 @@ create table exim_department ( dep_id bigint comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q index 1cc691fc2912d..c56329c03f89f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -18,6 +18,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q index 27830ad5f93dd..afaedcd37bf72 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -21,6 +21,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q index d85048a97a876..230b28c402cc5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -19,6 +19,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q index 84b3786a161c6..c2e00a9663468 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -23,6 +23,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q index eaf9c579d51d4..a6586ead0c23f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -19,6 +19,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q index 092fd779541c3..990a686ebeea6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q @@ -5,9 +5,9 @@ create table exim_department ( dep_id int comment "department id") clustered by (dep_id) sorted by (dep_id desc) into 10 buckets stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -20,6 +20,6 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q index 05de3d77b07b8..02537ef022d82 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -18,7 +18,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q index dc194ca814ee4..897c6747354b7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -19,7 +19,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q index e233707cc4db2..12013e5ccfc49 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q @@ -5,9 +5,9 @@ create table exim_department ( dep_id int comment "department id") partitioned by (dep_org string) stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department partition (dep_org="hr"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr"); +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -19,7 +19,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q index a10788e3e3b6b..d8d2b8008c9ee 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q @@ -5,9 +5,9 @@ create table exim_department ( dep_id int comment "department id") partitioned by (dep_org string) stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department partition (dep_org="hr"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department partition (dep_org="hr"); +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -20,7 +20,7 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import from 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q index cc4a56ca34b88..82dcce9455958 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -25,6 +25,6 @@ import table exim_employee partition (emp_country="us") from 'ql/test/data/expor describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q index 140e3bb3b1bbc..d92efeb9a70ef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -25,6 +25,6 @@ import table exim_employee partition (emp_country="us", emp_state="kl") from 'ql describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q index 048befe4d3d43..12d827b9c838c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q @@ -4,9 +4,9 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -17,7 +17,7 @@ create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); import external table exim_department from 'ql/test/data/exports/exim_department'; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop table exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q index 89cbb9ecd8086..726dee53955af 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q @@ -4,17 +4,17 @@ set hive.test.mode.prefix=; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; create table exim_department ( dep_id int comment "department id") stored as textfile @@ -22,9 +22,9 @@ create table exim_department ( dep_id int comment "department id") tblproperties("creator"="krishna"); import table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore2/exim_department'; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop table exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q index 0cbfc85258d2c..d187c78202034 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q @@ -6,16 +6,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -29,7 +29,7 @@ create table exim_employee ( emp_id int comment "employee id") tblproperties("creator"="krishna"); import external table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee'; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop table exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q index d9ab0cf0e4e50..b818686f773df 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q @@ -5,8 +5,8 @@ create table exim_department ( dep_id int) stored as textfile; set hive.security.authorization.enabled=true; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; set hive.security.authorization.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q index 2dbd534074fa8..4acefb9f0ae12 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q @@ -2,9 +2,9 @@ set hive.test.mode=true; set hive.test.mode.prefix=; create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -18,5 +18,5 @@ import from 'ql/test/data/exports/exim_department'; set hive.security.authorization.enabled=false; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q index ccbcee3698dae..467014e4679f6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q @@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -26,6 +26,6 @@ set hive.security.authorization.enabled=true; import from 'ql/test/data/exports/exim_employee'; set hive.security.authorization.enabled=false; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop table exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q index 50bfe005c4278..595fa7e764952 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q @@ -3,9 +3,9 @@ set hive.test.mode.prefix=; set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -19,5 +19,5 @@ set hive.security.authorization.enabled=false; select * from exim_department; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q index 9f44f225e955a..82230f782eac3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q @@ -2,6 +2,6 @@ CREATE TABLE fetchtask_ioexception ( KEY STRING, VALUE STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1_broken.seq' OVERWRITE INTO TABLE fetchtask_ioexception; +LOAD DATA LOCAL INPATH '../../data/files/kv1_broken.seq' OVERWRITE INTO TABLE fetchtask_ioexception; SELECT * FROM fetchtask_ioexception; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q new file mode 100644 index 0000000000000..286cf1afb491b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/file_with_header_footer_negative.q @@ -0,0 +1,13 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_file_with_header_footer_negative/; + +dfs -copyFromLocal ../data/files/header_footer_table_1 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1; + +dfs -copyFromLocal ../data/files/header_footer_table_2 hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_2; + +CREATE EXTERNAL TABLE header_footer_table_1 (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test_file_with_header_footer_negative/header_footer_table_1' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="200"); + +SELECT * FROM header_footer_table_1; + +DROP TABLE header_footer_table_1; + +dfs -rmr hdfs:///tmp/test_file_with_header_footer_negative; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q index 1cdaffd1f31a3..1ab828c8beae4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q @@ -1,6 +1,6 @@ -- begin part(string, int) pass(string, string) CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' ; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='second'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='second'); select * from tab1; drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q new file mode 100644 index 0000000000000..49e6a092fc127 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type3.q @@ -0,0 +1,4 @@ +create table tab1(c int) partitioned by (i int); +alter table tab1 add partition(i = "some name"); + +drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q new file mode 100644 index 0000000000000..50f486e6245cf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type4.q @@ -0,0 +1,3 @@ +create table tab1(s string) PARTITIONED BY(dt date, st string); +alter table tab1 add partition (dt=date 'foo', st='foo'); +drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q index 7d003e3e4b440..5bb889c027743 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; drop index src_index on src; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q index d79674539a1b2..c6600e69b6a7c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; drop index src_index on src; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q new file mode 100644 index 0000000000000..c20c168a887c2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into5.q @@ -0,0 +1,9 @@ +DROP TABLE if exists insert_into5_neg; + +CREATE TABLE insert_into5_neg (key int, value string) TBLPROPERTIES ("immutable"="true"); + +INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100; + +INSERT INTO TABLE insert_into5_neg SELECT * FROM src LIMIT 100; + +DROP TABLE insert_into5_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q new file mode 100644 index 0000000000000..a92ee5ca94a33 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into6.q @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS insert_into6_neg; + +CREATE TABLE insert_into6_neg (key int, value string) + PARTITIONED BY (ds string) TBLPROPERTIES("immutable"="true") ; + +INSERT INTO TABLE insert_into6_neg PARTITION (ds='1') + SELECT * FROM src LIMIT 100; + +INSERT INTO TABLE insert_into6_neg PARTITION (ds='2') + SELECT * FROM src LIMIT 100; + +SELECT COUNT(*) from insert_into6_neg; + +INSERT INTO TABLE insert_into6_neg PARTITION (ds='1') + SELECT * FROM src LIMIT 100; + +DROP TABLE insert_into6_neg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q index cbf65c4ac69fe..a8f77c28a8251 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q @@ -4,6 +4,6 @@ create table srcpart_dp like srcpart; create table destpart_dp like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_dp partition(ds='2008-04-08', hr=11); insert overwrite table destpart_dp partition (ds='2008-04-08', hr) if not exists select key, value, hr from srcpart_dp where ds='2008-04-08'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q new file mode 100644 index 0000000000000..ba7d164c77155 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_1.q @@ -0,0 +1,2 @@ +drop table invalid_char_length_1; +create table invalid_char_length_1 (c1 char(1000000)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q new file mode 100644 index 0000000000000..866b43d31273d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_2.q @@ -0,0 +1 @@ +select cast(value as char(100000)) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q new file mode 100644 index 0000000000000..481b630d20489 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_char_length_3.q @@ -0,0 +1,3 @@ +drop table invalid_char_length_3; +create table invalid_char_length_3 (c1 char(0)); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q deleted file mode 100644 index 14b3409cb4cc4..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q +++ /dev/null @@ -1,4 +0,0 @@ -ADD JAR ../data/files/TestSerDe.jar; -CREATE TABLE DELETEJAR(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' -STORED AS TEXTFILE -TBLPROPERTIES('columns'='valid_colname,invalid.colname'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q new file mode 100644 index 0000000000000..e39a38e2fcd47 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_alt_syntax_comma_on.q @@ -0,0 +1,3 @@ +explain select * +from src s1 , +src s2 on s1.key = s2.key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q new file mode 100644 index 0000000000000..c0da913c28812 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous.q @@ -0,0 +1,6 @@ + + +explain select s1.key, s2.key +from src s1, src s2 +where key = s2.key +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q new file mode 100644 index 0000000000000..8e219637eb0c6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_cond_unqual_ambiguous_vc.q @@ -0,0 +1,5 @@ + +explain select s1.key, s2.key +from src s1, src s2 +where INPUT__FILE__NAME = s2.INPUT__FILE__NAME +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q new file mode 100644 index 0000000000000..d59394544ccf0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition.q @@ -0,0 +1,7 @@ +set hive.limit.query.max.table.partition=1; + +explain select * from srcpart limit 1; +select * from srcpart limit 1; + +explain select * from srcpart; +select * from srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q new file mode 100644 index 0000000000000..0afd4a965ab94 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/limit_partition_stats.q @@ -0,0 +1,18 @@ +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.stats.autogather=true; +set hive.compute.query.using.stats=true; + +create table part (c int) partitioned by (d string); +insert into table part partition (d) +select hr,ds from srcpart; + +set hive.limit.query.max.table.partition=1; + +explain select count(*) from part; +select count(*) from part; + +set hive.compute.query.using.stats=false; + +explain select count(*) from part; +select count(*) from part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q index f86cd92d9d4c7..eb72d940a5392 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q @@ -1,4 +1,4 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; alter table hive_test_src add partition (pcol1 = 'test_part'); set hive.security.authorization.enabled=true; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q index 387aaed9a1e50..75a5216e00d82 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q @@ -2,4 +2,4 @@ CREATE TABLE non_native2(key int, value string) STORED BY 'org.apache.hadoop.hive.ql.metadata.DefaultStorageHandler'; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE non_native2; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE non_native2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q index d807c698777f1..32653631ad6a4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q @@ -1,3 +1,3 @@ create table hive_test_src ( col1 string ) stored as textfile; set hive.security.authorization.enabled=true; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q index c409d5a94a9ce..315988dc0a959 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q @@ -1,3 +1,3 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; set hive.security.authorization.enabled=true; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q index 356c16a664386..81517991b26fa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q @@ -1,2 +1,2 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; -load data local inpath '../data/files/test.dat' into table hive_test_src; +load data local inpath '../../data/files/test.dat' into table hive_test_src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q index eed5651cbf20b..c56f0d408d4ad 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q @@ -4,4 +4,4 @@ set hive.mapred.supports.subdirectories=true; CREATE TABLE if not exists stored_as_dirs_multiple (col1 STRING, col2 int, col3 STRING) SKEWED BY (col1, col2) ON (('s1',1), ('s3',3), ('s13',13), ('s78',78)) stored as DIRECTORIES; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE stored_as_dirs_multiple; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE stored_as_dirs_multiple; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q index 927f02e82bf91..64182eac8362d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q @@ -1,3 +1,3 @@ DROP VIEW xxx11; CREATE VIEW xxx11 AS SELECT * FROM src; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE xxx11; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE xxx11; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q index 16feeca22649a..f0c3b59d30ddf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q @@ -3,4 +3,4 @@ CREATE TABLE load_wrong_fileformat_T1(name STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE load_wrong_fileformat_T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q index 7e589fbfde620..4d79bbeb102c7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q @@ -3,4 +3,4 @@ CREATE TABLE T1(name STRING) STORED AS RCFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file +LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q index ff5ed4e2e3107..050c819a2f04b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q @@ -3,4 +3,4 @@ CREATE TABLE T1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file +LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q index ffb64ed643b14..7f5ad754142ab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q @@ -1,3 +1,3 @@ CREATE TABLE loadpart1(a STRING, b STRING) PARTITIONED BY (ds STRING,ds1 STRING); -LOAD DATA LOCAL INPATH '../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); +LOAD DATA LOCAL INPATH '../../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q index 8f4b37a9d49f6..ed9e21dd8a1fb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q @@ -1,4 +1,4 @@ set hive.exec.mode.local.auto=true; set hive.exec.failure.hooks=org.apache.hadoop.hive.ql.hooks.VerifySessionStateLocalErrorsHook; -FROM src SELECT TRANSFORM(key, value) USING 'python ../data/scripts/cat_error.py' AS (key, value); +FROM src SELECT TRANSFORM(key, value) USING 'python ../../data/scripts/cat_error.py' AS (key, value); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q new file mode 100644 index 0000000000000..4966f2b9b2825 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_query_tbl_in_locked_db.q @@ -0,0 +1,17 @@ +create database lockneg1; +use lockneg1; + +create table tstsrcpart like default.srcpart; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from default.srcpart where ds='2008-04-08' and hr='11'; + +lock database lockneg1 shared; +show locks database lockneg1; +select count(1) from tstsrcpart where ds='2008-04-08' and hr='11'; + +unlock database lockneg1; +show locks database lockneg1; +lock database lockneg1 exclusive; +show locks database lockneg1; +select count(1) from tstsrcpart where ds='2008-04-08' and hr='11'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q new file mode 100644 index 0000000000000..1f9ad90898dce --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_db_lock_conflict.q @@ -0,0 +1,6 @@ +set hive.lock.numretries=0; + +create database lockneg4; + +lock database lockneg4 exclusive; +lock database lockneg4 shared; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q new file mode 100644 index 0000000000000..8cbe31083b400 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_drop_locked_db.q @@ -0,0 +1,8 @@ +set hive.lock.numretries=0; + +create database lockneg9; + +lock database lockneg9 shared; +show locks database lockneg9; + +drop database lockneg9; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q new file mode 100644 index 0000000000000..4127a6f150a13 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg_try_lock_db_in_use.q @@ -0,0 +1,15 @@ +set hive.lock.numretries=0; + +create database lockneg2; +use lockneg2; + +create table tstsrcpart like default.srcpart; + +insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='11') +select key, value from default.srcpart where ds='2008-04-08' and hr='11'; + +lock table tstsrcpart shared; +show locks; + +lock database lockneg2 exclusive; +show locks; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q index ac6c4ee549d85..09f13f52aeadf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q @@ -10,6 +10,6 @@ simple_string string) -- This should fail in as extended nesting levels are not enabled using the serdeproperty hive.serialization.extend.nesting.levels -load data local inpath '../data/files/nested_complex.txt' overwrite into table nestedcomplex; +load data local inpath '../../data/files/nested_complex.txt' overwrite into table nestedcomplex; select * from nestedcomplex sort by simple_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q index 4841f9e11c84e..6669bf62d8822 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q @@ -2,6 +2,6 @@ CREATE TABLE nopart_insert(a STRING, b STRING) PARTITIONED BY (ds STRING); INSERT OVERWRITE TABLE nopart_insert -SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/error_script' AS (tkey, tvalue) +SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q index 6e5ad6eb41a8b..966982fd5ce52 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q @@ -1,5 +1,5 @@ CREATE TABLE nopart_load(a STRING, b STRING) PARTITIONED BY (ds STRING); -load data local inpath '../data/files/kv1.txt' overwrite into table nopart_load ; +load data local inpath '../../data/files/kv1.txt' overwrite into table nopart_load ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q deleted file mode 100644 index 6cc3e87288a89..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q +++ /dev/null @@ -1,4 +0,0 @@ -CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE; - -FROM src -INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, sum(src.value) WHERE src.key < 100 group by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q new file mode 100644 index 0000000000000..745a7867264e3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_char.q @@ -0,0 +1,3 @@ +drop table if exists parquet_char; + +create table parquet_char (t char(10)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q new file mode 100644 index 0000000000000..89d3602fd3e97 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_date.q @@ -0,0 +1,3 @@ +drop table if exists parquet_date; + +create table parquet_date (t date) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q new file mode 100644 index 0000000000000..8a4973110a51f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_decimal.q @@ -0,0 +1,3 @@ +drop table if exists parquet_decimal; + +create table parquet_decimal (t decimal(4,2)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q new file mode 100644 index 0000000000000..4ef36fa0efc49 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_timestamp.q @@ -0,0 +1,3 @@ +drop table if exists parquet_timestamp; + +create table parquet_timestamp (t timestamp) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q new file mode 100644 index 0000000000000..55825f76dc240 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/parquet_varchar.q @@ -0,0 +1,3 @@ +drop table if exists parquet_varchar; + +create table parquet_varchar (t varchar(10)) stored as parquet; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q index 72b55ea25d8e9..3fdc036996563 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q @@ -4,6 +4,6 @@ drop table tbl_protectmode6; create table tbl_protectmode6 (c1 string,c2 string) partitioned by (p string); alter table tbl_protectmode6 add partition (p='p1'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); alter table tbl_protectmode6 partition (p='p1') enable offline; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE tbl_protectmode6 partition (p='p1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q index 542367ace22e0..ef372259ed3e3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q @@ -12,7 +12,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- testAggrFuncsWithNoGBYNoPartDef select p_mfgr, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q index 95b35113e3cf0..58430423436b7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q @@ -12,7 +12,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- testAmbiguousWindowDefn select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q index 8333ddc948419..a171961a683ee 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q @@ -1,2 +1,3 @@ +set hive.support.quoted.identifiers=none; EXPLAIN SELECT `+++` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q index d1aa1f1a9542d..7bac1c775522b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q @@ -1,2 +1,3 @@ +set hive.support.quoted.identifiers=none; EXPLAIN SELECT `.a.` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q index 53971916e6c96..300d145508887 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q @@ -1,2 +1,3 @@ +set hive.support.quoted.identifiers=none; EXPLAIN SELECT `..`, count(1) FROM srcpart GROUP BY `..`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q deleted file mode 100644 index 6b1c09decfb35..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q +++ /dev/null @@ -1,3 +0,0 @@ -set hive.exec.script.allow.partial.consumption = false; --- Tests exception in ScriptOperator.close() by passing to the operator a small amount of data -SELECT TRANSFORM(*) USING 'true' AS a, b FROM (SELECT TRANSFORM(*) USING 'echo' AS a, b FROM src LIMIT 1) tmp; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q index e46aed03b147f..8ca849b82d8ad 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q @@ -1,7 +1,7 @@ EXPLAIN -SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/error_script' AS (tkey, tvalue) +SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) FROM src; -SELECT TRANSFORM(src.key, src.value) USING '../data/scripts/error_script' AS (tkey, tvalue) +SELECT TRANSFORM(src.key, src.value) USING '../../data/scripts/error_script' AS (tkey, tvalue) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q index a3955744221a4..d523d03e906c1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q @@ -16,8 +16,8 @@ WITH SERDEPROPERTIES ( ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH "../data/files/apache.access.log" INTO TABLE serde_regex; -LOAD DATA LOCAL INPATH "../data/files/apache.access.2.log" INTO TABLE serde_regex; +LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex; +LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex; -- raise an exception SELECT * FROM serde_regex ORDER BY time; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q new file mode 100644 index 0000000000000..579e9408b6c35 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation2.q @@ -0,0 +1,5 @@ +-- should fail: hive.fetch.task.conversion accepts minimal or more +desc src; + +set hive.conf.validation=true; +set hive.fetch.task.conversion=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q index 401cc37f67dd0..1b2872d3d7ed8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q @@ -6,7 +6,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; set hive.test.dummystats.aggregator=connect; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q index c7e63591adde0..0fa9ff6820371 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q @@ -5,7 +5,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=""; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q index 7fa0f55f2a450..be7c4f72feb9b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q @@ -6,7 +6,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; set hive.test.dummystats.publisher=connect; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q index f82d4b54b697b..652afe7c5bfba 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q @@ -5,7 +5,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=""; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q new file mode 100644 index 0000000000000..9013df6f938dc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_exists_implicit_gby.q @@ -0,0 +1,10 @@ + + +select * +from src b +where exists + (select count(*) + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9' + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q new file mode 100644 index 0000000000000..a9bc6ee6a38cb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_groupby.q @@ -0,0 +1,5 @@ + + +select count(*) +from src +group by src.key in (select key from src s1 where s1.key > '9') \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q new file mode 100644 index 0000000000000..1365389cb269a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_in_select.q @@ -0,0 +1,6 @@ + + + +select src.key in (select key from src s1 where s1.key > '9') +from src +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q new file mode 100644 index 0000000000000..6805c5b16b0f4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_multiple_cols_in_select.q @@ -0,0 +1,7 @@ + + +explain + select * +from src +where src.key in (select * from src s1 where s1.key > '9') +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q new file mode 100644 index 0000000000000..e8c41e6b17ae7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_nested_subquery.q @@ -0,0 +1,18 @@ + + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +select * +from part x +where x.p_name in (select y.p_name from part y where exists (select z.p_name from part z where y.p_name = z.p_name)) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q new file mode 100644 index 0000000000000..852b2953ff463 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_notexists_implicit_gby.q @@ -0,0 +1,10 @@ + + +select * +from src b +where not exists + (select sum(1) + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9' + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q new file mode 100644 index 0000000000000..d442f077c0707 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_shared_alias.q @@ -0,0 +1,6 @@ + + +select * +from src +where src.key in (select key from src where key > '9') +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q new file mode 100644 index 0000000000000..8ea94c5fc6d76 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_subquery_chain.q @@ -0,0 +1,6 @@ + +explain +select * +from src +where src.key in (select key from src) in (select key from src) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q new file mode 100644 index 0000000000000..99ff9ca703835 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_unqual_corr_expr.q @@ -0,0 +1,6 @@ + + +select * +from src +where key in (select key from src) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q new file mode 100644 index 0000000000000..105d3d22d9d2b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_windowing_corr.q @@ -0,0 +1,26 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + + +-- corr and windowing +select p_mfgr, p_name, p_size +from part a +where a.p_size in + (select first_value(p_size) over(partition by p_mfgr order by p_size) + from part b + where a.p_brand = b.p_brand) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q new file mode 100644 index 0000000000000..c2c322178f386 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subquery_with_or_cond.q @@ -0,0 +1,5 @@ + +select count(*) +from src +where src.key in (select key from src s1 where s1.key > '9') or src.value is not null +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q deleted file mode 100644 index 2fb5ff74cc3ce..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE '1' - WHEN 1 THEN 2 - WHEN 3 THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q deleted file mode 100644 index 5772dc1a95c9d..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE 1 - WHEN 1 THEN '2' - WHEN 3 THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q deleted file mode 100644 index 5aaf0188eb9c6..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE 1 - WHEN 1 THEN NULL - WHEN 3 THEN '2' - ELSE 7 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q new file mode 100644 index 0000000000000..68050fd95cd22 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_invalid.q @@ -0,0 +1 @@ +select default.nonexistfunc() from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q new file mode 100644 index 0000000000000..bcfa217737e33 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_local_resource.q @@ -0,0 +1 @@ +create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file '../../data/files/sales.txt'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q new file mode 100644 index 0000000000000..d37665dde69bc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_nonexistent_resource.q @@ -0,0 +1 @@ +create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'nonexistent_file.txt'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q new file mode 100644 index 0000000000000..476dfa21a237f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_qualified_name.q @@ -0,0 +1 @@ +create temporary function default.myfunc as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q deleted file mode 100644 index 79fa65f63da56..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE - WHEN 1=2 THEN '2' - WHEN 3=4 THEN 4 - ELSE 5 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q deleted file mode 100644 index 8bb5fdd7ea377..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q +++ /dev/null @@ -1,6 +0,0 @@ -SELECT CASE - WHEN 1=2 THEN '2' - WHEN 3=4 THEN '5' - ELSE 5.3 - END -FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q deleted file mode 100644 index 3c4204f780004..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q +++ /dev/null @@ -1,6 +0,0 @@ - -CREATE TEMPORARY FUNCTION example_arraysum AS 'org.apache.hadoop.hive.contrib.udf.example.UDFExampleArraySum'; - -SELECT example_arraysum(lint)FROM src_thrift; - -DROP TEMPORARY FUNCTION example_arraysum; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q deleted file mode 100644 index e3c5c830897ea..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q +++ /dev/null @@ -1,4 +0,0 @@ -explain -select s1.key as key, s1.value as value from src s1 - UNION ALL -select s2.key as key, s2.value as value from src s2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q new file mode 100644 index 0000000000000..c5b593e4bb556 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_invalid_udaf.q @@ -0,0 +1 @@ +select nonexistfunc(key) over () from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q new file mode 100644 index 0000000000000..15f8fae292bbb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_neg.q @@ -0,0 +1,26 @@ +DROP TABLE IF EXISTS part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + + +select p_mfgr, p_name, p_size, +min(p_retailprice), +rank() over(distribute by p_mfgr sort by p_name)as r, +dense_rank() over(distribute by p_mfgr sort by p_name) as dr, +p_size, p_size - lag(p_size,-1,p_size) over(distribute by p_mfgr sort by p_name) as deltaSz +from part +group by p_mfgr, p_name, p_size +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q index 5fd19455f93cd..312a0177fafff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q @@ -15,7 +15,7 @@ describe extended alter1; alter table alter1 set serdeproperties('s1'='10', 's2' ='20'); describe extended alter1; -add jar ../data/files/TestSerDe.jar; +add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; alter table alter1 set serde 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties('s1'='9'); describe extended alter1; @@ -56,7 +56,7 @@ DESCRIBE EXTENDED alter1; ALTER TABLE alter1 SET SERDEPROPERTIES('s1'='10', 's2' ='20'); DESCRIBE EXTENDED alter1; -add jar ../data/files/TestSerDe.jar; +add jar ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' WITH SERDEPROPERTIES ('s1'='9'); DESCRIBE EXTENDED alter1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q index 3cde00748b98e..91e4e9bad0eda 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q @@ -1,5 +1,5 @@ create table alter3_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter3_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter3_src ; create table alter3 ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; @@ -34,7 +34,7 @@ USE alter3_db; SHOW TABLES; CREATE TABLE alter3_src (col1 STRING) STORED AS TEXTFILE ; -LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE alter3_src ; +LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE alter3_src ; CREATE TABLE alter3 (col1 STRING) PARTITIONED BY (pcol1 STRING, pcol2 STRING) STORED AS SEQUENCEFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q index 0d14f228d1c30..66c9f8dc54b91 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q @@ -3,7 +3,7 @@ -- create table alter5_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter5_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter5_src ; create table alter5 ( col1 string ) partitioned by (dt string); @@ -32,7 +32,7 @@ USE alter5_db; SHOW TABLES; create table alter5_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter5_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter5_src ; create table alter5 ( col1 string ) partitioned by (dt string); alter table alter5 add partition (dt='a') location 'parta'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q new file mode 100644 index 0000000000000..4ecb7e7389f6b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char1.q @@ -0,0 +1,32 @@ +drop table alter_char_1; + +create table alter_char_1 (key string, value string); +insert overwrite table alter_char_1 + select key, value from src order by key limit 5; + +select * from alter_char_1 order by key; + +-- change column to char +alter table alter_char_1 change column value value char(20); +-- contents should still look the same +select * from alter_char_1 order by key; + +-- change column to smaller char +alter table alter_char_1 change column value value char(3); +-- value column should be truncated now +select * from alter_char_1 order by key; + +-- change back to bigger char +alter table alter_char_1 change column value value char(20); +-- column values should be full size again +select * from alter_char_1 order by key; + +-- add char column +alter table alter_char_1 add columns (key2 int, value2 char(10)); +select * from alter_char_1 order by key; + +insert overwrite table alter_char_1 + select key, value, key, value from src order by key limit 5; +select * from alter_char_1 order by key; + +drop table alter_char_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q new file mode 100644 index 0000000000000..7fa9fcef11a14 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_char2.q @@ -0,0 +1,22 @@ + +-- alter column type, with partitioned table +drop table if exists alter_char2; + +create table alter_char2 ( + c1 char(255) +) partitioned by (hr int); + +insert overwrite table alter_char2 partition (hr=1) + select value from src limit 1; + +select c1, length(c1) from alter_char2; + +alter table alter_char2 change column c1 c1 char(10); + +select hr, c1, length(c1) from alter_char2 where hr = 1; + +insert overwrite table alter_char2 partition (hr=2) + select key from src limit 1; + +select hr, c1, length(c1) from alter_char2 where hr = 1; +select hr, c1, length(c1) from alter_char2 where hr = 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q index 807ef539c8643..e1c3780a9b0ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q @@ -1,9 +1,9 @@ set hive.exec.concatenate.check.index =false; create table src_rc_concatenate_test(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_concatenate_test; -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_concatenate_test; -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_concatenate_test; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test; +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test; +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test; show table extended like `src_rc_concatenate_test`; @@ -26,9 +26,9 @@ create table src_rc_concatenate_test_part(key int, value string) partitioned by alter table src_rc_concatenate_test_part add partition (ds='2011'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_concatenate_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_concatenate_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_concatenate_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_concatenate_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_concatenate_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_concatenate_test_part partition (ds='2011'); show table extended like `src_rc_concatenate_test_part` partition (ds='2011'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q new file mode 100644 index 0000000000000..b224f3339e4ab --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_db_owner.q @@ -0,0 +1,9 @@ + +create database db_alter_onr; +describe database db_alter_onr; + +alter database db_alter_onr set owner user user1; +describe database db_alter_onr; + +alter database db_alter_onr set owner role role1; +describe database db_alter_onr; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q index c3502739d54a9..ceabd0830a03d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q @@ -1,8 +1,8 @@ create table src_rc_merge_test(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test; -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test; -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test; show table extended like `src_rc_merge_test`; @@ -21,9 +21,9 @@ create table src_rc_merge_test_part(key int, value string) partitioned by (ds st alter table src_rc_merge_test_part add partition (ds='2011'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2011'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2011'); show table extended like `src_rc_merge_test_part` partition (ds='2011'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q index 65ddfed13dac4..e09703d1c6c85 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q @@ -3,9 +3,9 @@ create table src_rc_merge_test_part(key int, value string) partitioned by (ds st alter table src_rc_merge_test_part add partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); desc extended src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part partition (ds='2012-01-03', ts='2012-01-03+14:46:31'); select count(1) from src_rc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31'; select sum(hash(key)), sum(hash(value)) from src_rc_merge_test_part where ds='2012-01-03' and ts='2012-01-03+14:46:31'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q index 23bae55ab2add..0af87e2c076e6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q @@ -1,8 +1,8 @@ create table src_rc_merge_test_stat(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat; -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat; -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_stat; +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_stat; +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_stat; show table extended like `src_rc_merge_test_stat`; desc extended src_rc_merge_test_stat; @@ -21,9 +21,9 @@ create table src_rc_merge_test_part_stat(key int, value string) partitioned by ( alter table src_rc_merge_test_part_stat add partition (ds='2011'); -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); -load data local inpath '../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); -load data local inpath '../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_2.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); +load data local inpath '../../data/files/smbbucket_3.rc' into table src_rc_merge_test_part_stat partition (ds='2011'); show table extended like `src_rc_merge_test_part_stat` partition (ds='2011'); desc extended src_rc_merge_test_part_stat; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q index b6d1eb8f2d4b8..5dda4c08fd5b3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q @@ -1,4 +1,4 @@ - +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) create table tst1(key string, value string) partitioned by (ds string) clustered by (key) into 10 buckets; alter table tst1 clustered by (key) into 8 buckets; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q index c6a4ad24fcdde..acc028bdd81eb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q @@ -1,3 +1,4 @@ +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) -- Tests that when overwriting a partition in a table after altering the bucketing/sorting metadata -- the partition metadata is updated as well. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q new file mode 100644 index 0000000000000..d81430441c720 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2_h23.q @@ -0,0 +1,85 @@ +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) +-- Tests that when overwriting a partition in a table after altering the bucketing/sorting metadata +-- the partition metadata is updated as well. + +CREATE TABLE tst1(key STRING, value STRING) PARTITIONED BY (ds STRING); + +DESCRIBE FORMATTED tst1; + +SET hive.enforce.bucketing=true; +SET hive.enforce.sorting=true; +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test an unbucketed partition gets converted to bucketed +ALTER TABLE tst1 CLUSTERED BY (key) INTO 8 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test an unsorted partition gets converted to sorted +ALTER TABLE tst1 CLUSTERED BY (key) SORTED BY (key DESC) INTO 8 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the bucket columns +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (key DESC) INTO 8 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the number of buckets +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (key DESC) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the sort columns +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (value DESC) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test changing the sort order +ALTER TABLE tst1 CLUSTERED BY (value) SORTED BY (value ASC) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test a sorted partition gets converted to unsorted +ALTER TABLE tst1 CLUSTERED BY (value) INTO 4 BUCKETS; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); + +-- Test a bucketed partition gets converted to unbucketed +ALTER TABLE tst1 NOT CLUSTERED; + +DESCRIBE FORMATTED tst1; + +INSERT OVERWRITE TABLE tst1 PARTITION (ds = '1') SELECT key, value FROM src; + +DESCRIBE FORMATTED tst1 PARTITION (ds = '1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q new file mode 100644 index 0000000000000..a03992510b072 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table_h23.q @@ -0,0 +1,59 @@ +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20,0.20S) +create table tst1(key string, value string) partitioned by (ds string) clustered by (key) into 10 buckets; + +alter table tst1 clustered by (key) into 8 buckets; + +describe formatted tst1; + +set hive.enforce.bucketing=true; +insert overwrite table tst1 partition (ds='1') select key, value from src; + +describe formatted tst1 partition (ds = '1'); + +-- Test changing bucket number + +alter table tst1 clustered by (key) into 12 buckets; + +insert overwrite table tst1 partition (ds='1') select key, value from src; + +describe formatted tst1 partition (ds = '1'); + +describe formatted tst1; + +-- Test changing bucket number of (table/partition) + +alter table tst1 into 4 buckets; + +describe formatted tst1; + +describe formatted tst1 partition (ds = '1'); + +alter table tst1 partition (ds = '1') into 6 buckets; + +describe formatted tst1; + +describe formatted tst1 partition (ds = '1'); + +-- Test adding sort order + +alter table tst1 clustered by (key) sorted by (key asc) into 12 buckets; + +describe formatted tst1; + +-- Test changing sort order + +alter table tst1 clustered by (key) sorted by (value desc) into 12 buckets; + +describe formatted tst1; + +-- Test removing test order + +alter table tst1 clustered by (value) into 12 buckets; + +describe formatted tst1; + +-- Test removing buckets + +alter table tst1 not clustered; + +describe formatted tst1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q index 5479afbbd5ab0..19c0f9d1d88ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q @@ -10,48 +10,48 @@ desc alter_coltype; select count(*) from alter_coltype where dt = '100x'; -- alter partition key column data type for dt column. -alter table alter_coltype partition column (dt int); +-- alter table alter_coltype partition column (dt int); -- load a new partition using new data type. -insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1; +-- insert overwrite table alter_coltype partition(dt=10, ts='3.0') select * from src1; -- make sure the partition predicate still works. -select count(*) from alter_coltype where dt = '100x'; -explain extended select count(*) from alter_coltype where dt = '100x'; +-- select count(*) from alter_coltype where dt = '100x'; +-- explain extended select count(*) from alter_coltype where dt = '100x'; -select count(*) from alter_coltype where dt = 100; +-- select count(*) from alter_coltype where dt = '100'; -- alter partition key column data type for ts column. -alter table alter_coltype partition column (ts double); +-- alter table alter_coltype partition column (ts double); -alter table alter_coltype partition column (dt string); +-- alter table alter_coltype partition column (dt string); -- load a new partition using new data type. -insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1; +-- insert overwrite table alter_coltype partition(dt='100x', ts=3.0) select * from src1; -- validate partition key column predicate can still work. -select count(*) from alter_coltype where ts = '6:30pm'; -explain extended select count(*) from alter_coltype where ts = '6:30pm'; +-- select count(*) from alter_coltype where ts = '6:30pm'; +-- explain extended select count(*) from alter_coltype where ts = '6:30pm'; -- validate partition key column predicate on two different partition column data type -- can still work. -select count(*) from alter_coltype where ts = 3.0 and dt=10; -explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10; +-- select count(*) from alter_coltype where ts = 3.0 and dt=10; +-- explain extended select count(*) from alter_coltype where ts = 3.0 and dt=10; -- query where multiple partition values (of different datatypes) are being selected -select key, value, dt, ts from alter_coltype where dt is not null; -explain extended select key, value, dt, ts from alter_coltype where dt is not null; +-- select key, value, dt, ts from alter_coltype where dt is not null; +-- explain extended select key, value, dt, ts from alter_coltype where dt is not null; -select count(*) from alter_coltype where ts = 3.0; +-- select count(*) from alter_coltype where ts = 3.0; -- make sure the partition predicate still works. -select count(*) from alter_coltype where dt = '100x' or dt = '10'; -explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10'; +-- select count(*) from alter_coltype where dt = '100x' or dt = '10'; +-- explain extended select count(*) from alter_coltype where dt = '100x' or dt = '10'; -desc alter_coltype; -desc alter_coltype partition (dt='100x', ts='6:30pm'); -desc alter_coltype partition (dt='100x', ts=3.0); -desc alter_coltype partition (dt=10, ts=3.0); +-- desc alter_coltype; +-- desc alter_coltype partition (dt='100x', ts='6:30pm'); +-- desc alter_coltype partition (dt='100x', ts=3.0); +-- desc alter_coltype partition (dt=10, ts=3.0); drop table alter_coltype; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q index 7bcb9f071c415..7a1f3dd51d617 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q @@ -2,10 +2,10 @@ create table if not exists alter_part_protect_mode(key string, value string ) partitioned by (year string, month string) stored as textfile ; -- Load data -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='10'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='12'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1995', month='09'); -load data local inpath '../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1994', month='07'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='10'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1996', month='12'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1995', month='09'); +load data local inpath '../../data/files/T1.txt' overwrite into table alter_part_protect_mode partition (year='1994', month='07'); -- offline alter table alter_part_protect_mode partition (year='1996') disable offline; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q index d498cd52a5ffb..8ebbe98824dec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q @@ -4,7 +4,7 @@ DROP TABLE alter_rename_partition; SHOW TABLES; create table alter_rename_partition_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table alter_rename_partition_src ; +load data local inpath '../../data/files/test.dat' overwrite into table alter_rename_partition_src ; create table alter_rename_partition ( col1 string ) partitioned by (pcol1 string , pcol2 string) stored as sequencefile; @@ -28,7 +28,7 @@ USE alter_rename_partition_db; SHOW TABLES; CREATE TABLE alter_rename_partition_src (col1 STRING) STORED AS TEXTFILE ; -LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE alter_rename_partition_src ; +LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE alter_rename_partition_src ; CREATE TABLE alter_rename_partition (col1 STRING) PARTITIONED BY (pcol1 STRING, pcol2 STRING) STORED AS SEQUENCEFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q index 5a481e7f8ebfe..b870108bddd2b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q @@ -7,7 +7,7 @@ create table alter_varchar2 ( ) partitioned by (hr int); insert overwrite table alter_varchar2 partition (hr=1) - select value from src limit 1; + select value from src tablesample (1 rows); select c1, length(c1) from alter_varchar2; @@ -16,7 +16,9 @@ alter table alter_varchar2 change column c1 c1 varchar(10); select hr, c1, length(c1) from alter_varchar2 where hr = 1; insert overwrite table alter_varchar2 partition (hr=2) - select key from src limit 1; + select key from src tablesample (1 rows); + +set hive.fetch.task.conversion=more; select hr, c1, length(c1) from alter_varchar2 where hr = 1; select hr, c1, length(c1) from alter_varchar2 where hr = 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q index e7053c1c8eb55..5ccd2c8c62dcf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q @@ -1,3 +1,4 @@ +set hive.support.quoted.identifiers=none; -- TOK_ALLCOLREF explain select * from (select a.key, a.* from (select * from src) a join (select * from src1) b on (a.key = b.key)) t; -- DOT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q new file mode 100644 index 0000000000000..ec973e15969d4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_filter.q @@ -0,0 +1,76 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- column stats are not COMPLETE, so stats are not updated +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc where state='OH'; + +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- state column has 5 distincts. numRows/countDistincts +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where state='OH'; + +-- not equals comparison shouldn't affect number of rows +-- numRows: 8 rawDataSize: 804 +explain extended select * from loc_orc where state!='OH'; +explain extended select * from loc_orc where state<>'OH'; + +-- nulls are treated as constant equality comparison +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where zip is null; +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where !(zip is not null); + +-- not nulls are treated as inverse of nulls +-- numRows: 7 rawDataSize: 702 +explain extended select * from loc_orc where zip is not null; +-- numRows: 7 rawDataSize: 702 +explain extended select * from loc_orc where !(zip is null); + +-- NOT evaluation. true will pass all rows, false will not pass any rows +-- numRows: 8 rawDataSize: 804 +explain extended select * from loc_orc where !false; +-- numRows: 0 rawDataSize: 0 +explain extended select * from loc_orc where !true; + +-- OR evaluation. 1 row for OH and 1 row for CA +-- numRows: 2 rawDataSize: 204 +explain extended select * from loc_orc where state='OH' or state='CA'; + +-- AND evaluation. cascadingly apply rules. 8/2 = 4/2 = 2 +-- numRows: 2 rawDataSize: 204 +explain extended select * from loc_orc where year=2001 and year is null; +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where year=2001 and state='OH' and state='FL'; + +-- AND and OR together. left expr will yield 1 row and right will yield 1 row +-- numRows: 3 rawDataSize: 306 +explain extended select * from loc_orc where (year=2001 and year is null) or (state='CA'); + +-- AND and OR together. left expr will yield 8 rows and right will yield 1 row +-- numRows: 1 rawDataSize: 102 +explain extended select * from loc_orc where (year=2001 or year is null) and (state='CA'); + +-- all inequality conditions rows/3 is the rules +-- numRows: 2 rawDataSize: 204 +explain extended select * from loc_orc where locid < 30; +explain extended select * from loc_orc where locid > 30; +explain extended select * from loc_orc where locid <= 30; +explain extended select * from loc_orc where locid >= 30; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q new file mode 100644 index 0000000000000..05cb036b466a2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_groupby.q @@ -0,0 +1,69 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- partial column stats +analyze table loc_orc compute statistics for columns state; + +-- inner group by: map - numRows: 8 reduce - numRows: 4 +-- outer group by: map - numRows: 4 reduce numRows: 2 +explain extended select a, c, min(b) +from ( select state as a, locid as b, count(*) as c + from loc_orc + group by state,locid + ) sq1 +group by a,c; + +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- only one distinct value in year column + 1 NULL value +-- map-side GBY: numRows: 8 (map-side will not do any reduction) +-- reduce-side GBY: numRows: 2 +explain extended select year from loc_orc group by year; + +-- map-side GBY: numRows: 8 +-- reduce-side GBY: numRows: 4 +explain extended select state,locid from loc_orc group by state,locid; + +-- map-side GBY numRows: 32 reduce-side GBY numRows: 16 +explain extended select state,locid from loc_orc group by state,locid with cube; + +-- map-side GBY numRows: 24 reduce-side GBY numRows: 12 +explain extended select state,locid from loc_orc group by state,locid with rollup; + +-- map-side GBY numRows: 8 reduce-side GBY numRows: 4 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state)); + +-- map-side GBY numRows: 16 reduce-side GBY numRows: 8 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state),(locid)); + +-- map-side GBY numRows: 24 reduce-side GBY numRows: 12 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state),(locid),()); + +-- map-side GBY numRows: 32 reduce-side GBY numRows: 16 +explain extended select state,locid from loc_orc group by state,locid grouping sets((state,locid),(state),(locid),()); + +set hive.stats.map.parallelism=10; + +-- map-side GBY: numRows: 80 (map-side will not do any reduction) +-- reduce-side GBY: numRows: 2 Reason: numDistinct of year is 2. numRows = min(80/2, 2) +explain extended select year from loc_orc group by year; + +-- map-side GBY numRows: 320 reduce-side GBY numRows: 42 Reason: numDistinct of state and locid are 6,7 resp. numRows = min(320/2, 6*7) +explain extended select state,locid from loc_orc group by state,locid with cube; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q new file mode 100644 index 0000000000000..965b0b7ed0a3e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q @@ -0,0 +1,81 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists emp_staging ( + lastname string, + deptid int +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists dept_staging ( + deptid int, + deptname string +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists emp_orc like emp_staging; +alter table emp_orc set fileformat orc; + +create table if not exists dept_orc like dept_staging; +alter table dept_orc set fileformat orc; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +LOAD DATA LOCAL INPATH '../../data/files/emp.txt' OVERWRITE INTO TABLE emp_staging; +LOAD DATA LOCAL INPATH '../../data/files/dept.txt' OVERWRITE INTO TABLE dept_staging; +LOAD DATA LOCAL INPATH '../../data/files/loc.txt' OVERWRITE INTO TABLE loc_staging; + +insert overwrite table emp_orc select * from emp_staging; +insert overwrite table dept_orc select * from dept_staging; +insert overwrite table loc_orc select * from loc_staging; + +analyze table emp_orc compute statistics for columns lastname,deptid; +analyze table dept_orc compute statistics for columns deptname,deptid; +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- number of rows +-- emp_orc - 6 +-- dept_orc - 4 +-- loc_orc - 8 + +-- count distincts for relevant columns (since count distinct values are approximate in some cases count distint values will be greater than number of rows) +-- emp_orc.deptid - 3 +-- emp_orc.lastname - 7 +-- dept_orc.deptid - 6 +-- dept_orc.deptname - 5 +-- loc_orc.locid - 6 +-- loc_orc.state - 7 + +-- Expected output rows: 4 +-- Reason: #rows = (6*4)/max(3,6) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid); + +-- 3 way join +-- Expected output rows: 4 +-- Reason: #rows = (6*4*6)/max(3,6)*max(6,3) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid) join emp_orc e1 on (e.deptid = e1.deptid); + +-- Expected output rows: 5 +-- Reason: #rows = (6*4*8)/max(3,6)*max(6,6) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid) join loc_orc l on (e.deptid = l.locid); + +-- join keys of different types +-- Expected output rows: 4 +-- Reason: #rows = (6*4*8)/max(3,6)*max(6,7) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid) join loc_orc l on (e.deptid = l.state); + +-- multi-attribute join +-- Expected output rows: 0 +-- Reason: #rows = (6*4)/max(3,6)*max(7,5) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid and e.lastname = d.deptname); + +-- 3 way and multi-attribute join +-- Expected output rows: 0 +-- Reason: #rows = (6*4*8)/max(3,6)*max(7,5)*max(3,6)*max(7,7) +explain extended select * from emp_orc e join dept_orc d on (e.deptid = d.deptid and e.lastname = d.deptname) join loc_orc l on (e.deptid = l.locid and e.lastname = l.state); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q new file mode 100644 index 0000000000000..0a9f880b5f31e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_limit.q @@ -0,0 +1,30 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +analyze table loc_orc compute statistics for columns state, locid, zip, year; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- numRows: 4 rawDataSize: 396 +explain extended select * from loc_orc limit 4; + +-- greater than the available number of rows +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc limit 16; + +-- numRows: 0 rawDataSize: 0 +explain extended select * from loc_orc limit 0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q new file mode 100644 index 0000000000000..839c7d84962fc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_part.q @@ -0,0 +1,85 @@ +set hive.stats.fetch.column.stats=true; +set hive.stats.autogather=false; +set hive.exec.dynamic.partition=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year string +) row format delimited fields terminated by '|' stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/loc.txt' OVERWRITE INTO TABLE loc_staging; + +create table if not exists loc_orc ( + state string, + locid int, + zip bigint +) partitioned by(year string) stored as orc; + +-- basicStatState: NONE colStatState: NONE +explain extended select * from loc_orc; + +insert overwrite table loc_orc partition(year) select * from loc_staging; + +-- stats are disabled. basic stats will report the file size but not raw data size. so initial statistics will be PARTIAL + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from loc_orc; + +-- partition level analyze statistics for specific parition +analyze table loc_orc partition(year='2001') compute statistics; + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from loc_orc where year='__HIVE_DEFAULT_PARTITION__'; + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from loc_orc; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc where year='2001'; + +-- partition level analyze statistics for all partitions +analyze table loc_orc partition(year) compute statistics; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc where year='__HIVE_DEFAULT_PARTITION__'; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from loc_orc where year='2001' or year='__HIVE_DEFAULT_PARTITION__'; + +-- both partitions will be pruned +-- basicStatState: NONE colStatState: NONE +explain extended select * from loc_orc where year='2001' and year='__HIVE_DEFAULT_PARTITION__'; + +-- partition level partial column statistics +analyze table loc_orc partition(year='2001') compute statistics for columns state,locid; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select zip from loc_orc; + +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select state from loc_orc; + +-- column statistics for __HIVE_DEFAULT_PARTITION__ is not supported yet. Hence colStatState reports PARTIAL +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select state,locid from loc_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select state,locid from loc_orc where year='2001'; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select state,locid from loc_orc where year!='2001'; + +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select * from loc_orc; + +-- This is to test filter expression evaluation on partition column +-- numRows: 2 dataSize: 8 basicStatState: COMPLETE colStatState: COMPLETE +explain extended select locid from loc_orc where locid>0 and year='2001'; +explain extended select locid,year from loc_orc where locid>0 and year='2001'; +explain extended select * from (select locid,year from loc_orc) test where locid>0 and year='2001'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q new file mode 100644 index 0000000000000..5fc3f64b90180 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_select.q @@ -0,0 +1,143 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists alltypes ( + bo1 boolean, + ti1 tinyint, + si1 smallint, + i1 int, + bi1 bigint, + f1 float, + d1 double, + de1 decimal, + ts1 timestamp, + da1 timestamp, + s1 string, + vc1 varchar(5), + m1 map, + l1 array, + st1 struct +) row format delimited fields terminated by '|' +collection items terminated by ',' +map keys terminated by ':' stored as textfile; + +create table alltypes_orc like alltypes; +alter table alltypes_orc set fileformat orc; + +load data local inpath '../../data/files/alltypes.txt' overwrite into table alltypes; + +insert overwrite table alltypes_orc select * from alltypes; + +-- basicStatState: COMPLETE colStatState: NONE numRows: 2 rawDataSize: 1514 +explain extended select * from alltypes_orc; + +-- statistics for complex types are not supported yet +analyze table alltypes_orc compute statistics for columns bo1, ti1, si1, i1, bi1, f1, d1, s1, vc1; + +-- numRows: 2 rawDataSize: 1514 +explain extended select * from alltypes_orc; + +-- numRows: 2 rawDataSize: 8 +explain extended select bo1 from alltypes_orc; + +-- col alias renaming +-- numRows: 2 rawDataSize: 8 +explain extended select i1 as int1 from alltypes_orc; + +-- numRows: 2 rawDataSize: 174 +explain extended select s1 from alltypes_orc; + +-- column statistics for complex types unsupported and so statistics will not be updated +-- numRows: 2 rawDataSize: 1514 +explain extended select m1 from alltypes_orc; + +-- numRows: 2 rawDataSize: 246 +explain extended select bo1, ti1, si1, i1, bi1, f1, d1,s1 from alltypes_orc; + +-- numRows: 2 rawDataSize: 0 +explain extended select null from alltypes_orc; + +-- numRows: 2 rawDataSize: 8 +explain extended select 11 from alltypes_orc; + +-- numRows: 2 rawDataSize: 16 +explain extended select 11L from alltypes_orc; + +-- numRows: 2 rawDataSize: 16 +explain extended select 11.0 from alltypes_orc; + +-- numRows: 2 rawDataSize: 178 +explain extended select "hello" from alltypes_orc; +explain extended select cast("hello" as char(5)) from alltypes_orc; +explain extended select cast("hello" as varchar(5)) from alltypes_orc; + +-- numRows: 2 rawDataSize: 96 +explain extended select unbase64("0xe23") from alltypes_orc; + +-- numRows: 2 rawDataSize: 16 +explain extended select cast("1" as TINYINT), cast("20" as SMALLINT) from alltypes_orc; + +-- numRows: 2 rawDataSize: 80 +explain extended select cast("1970-12-31 15:59:58.174" as TIMESTAMP) from alltypes_orc; + +-- numRows: 2 rawDataSize: 112 +explain extended select cast("1970-12-31 15:59:58.174" as DATE) from alltypes_orc; + +-- numRows: 2 rawDataSize: 224 +explain extended select cast("58.174" as DECIMAL) from alltypes_orc; + +-- numRows: 2 rawDataSize: 112 +explain extended select array(1,2,3) from alltypes_orc; + +-- numRows: 2 rawDataSize: 1508 +explain extended select str_to_map("a=1 b=2 c=3", " ", "=") from alltypes_orc; + +-- numRows: 2 rawDataSize: 112 +explain extended select NAMED_STRUCT("a", 11, "b", 11) from alltypes_orc; + +-- numRows: 2 rawDataSize: 250 +explain extended select CREATE_UNION(0, "hello") from alltypes_orc; + +-- COUNT(*) is projected as new column. It is not projected as GenericUDF and so datasize estimate will be based on number of rows +-- numRows: 1 rawDataSize: 8 +explain extended select count(*) from alltypes_orc; + +-- COUNT(1) is projected as new column. It is not projected as GenericUDF and so datasize estimate will be based on number of rows +-- numRows: 1 rawDataSize: 8 +explain extended select count(1) from alltypes_orc; + +-- column statistics for complex column types will be missing. data size will be calculated from available column statistics +-- numRows: 2 rawDataSize: 254 +explain extended select *,11 from alltypes_orc; + +-- subquery selects +-- inner select - numRows: 2 rawDataSize: 8 +-- outer select - numRows: 2 rawDataSize: 8 +explain extended select i1 from (select i1 from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 16 +-- outer select - numRows: 2 rawDataSize: 8 +explain extended select i1 from (select i1,11 from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 16 +-- outer select - numRows: 2 rawDataSize: 186 +explain extended select i1,"hello" from (select i1,11 from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 24 +-- outer select - numRows: 2 rawDataSize: 16 +explain extended select x from (select i1,11.0 as x from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 104 +-- outer select - numRows: 2 rawDataSize: 186 +explain extended select x,"hello" from (select i1 as x, unbase64("0xe23") as ub from alltypes_orc limit 10) temp; + +-- inner select - numRows: 2 rawDataSize: 186 +-- middle select - numRows: 2 rawDataSize: 178 +-- outer select - numRows: 2 rawDataSize: 194 +explain extended select h, 11.0 from (select hell as h from (select i1, "hello" as hell from alltypes_orc limit 10) in1 limit 10) in2; + +-- This test is for FILTER operator where filter expression is a boolean column +-- numRows: 2 rawDataSize: 8 +explain extended select bo1 from alltypes_orc where bo1; + +-- numRows: 0 rawDataSize: 0 +explain extended select bo1 from alltypes_orc where !bo1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q new file mode 100644 index 0000000000000..4140fe610d7cd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_table.q @@ -0,0 +1,53 @@ +set hive.stats.fetch.column.stats=true; +set hive.stats.autogather=false; + +create table if not exists emp_staging ( + lastname string, + deptid int +) row format delimited fields terminated by '|' stored as textfile; + +create table if not exists emp_orc like emp_staging; +alter table emp_orc set fileformat orc; + +-- basicStatState: NONE colStatState: NONE +explain extended select * from emp_orc; + +LOAD DATA LOCAL INPATH '../../data/files/emp.txt' OVERWRITE INTO TABLE emp_staging; + +insert overwrite table emp_orc select * from emp_staging; + +-- stats are disabled. basic stats will report the file size but not raw data size. so initial statistics will be PARTIAL + +-- basicStatState: PARTIAL colStatState: NONE +explain extended select * from emp_orc; + +-- table level analyze statistics +analyze table emp_orc compute statistics; + +-- basicStatState: COMPLETE colStatState: NONE +explain extended select * from emp_orc; + +-- column level partial statistics +analyze table emp_orc compute statistics for columns deptid; + +-- basicStatState: COMPLETE colStatState: PARTIAL +explain extended select * from emp_orc; + +-- all selected columns have statistics +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select deptid from emp_orc; + +-- column level complete statistics +analyze table emp_orc compute statistics for columns lastname,deptid; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select * from emp_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select lastname from emp_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select deptid from emp_orc; + +-- basicStatState: COMPLETE colStatState: COMPLETE +explain extended select lastname,deptid from emp_orc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q new file mode 100644 index 0000000000000..586d9e1e2c023 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_union.q @@ -0,0 +1,55 @@ +set hive.stats.fetch.column.stats=true; + +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- numRows: 8 rawDataSize: 688 +explain extended select state from loc_orc; + +-- numRows: 16 rawDataSize: 1376 +explain extended select * from (select state from loc_orc union all select state from loc_orc) tmp; + +-- numRows: 8 rawDataSize: 796 +explain extended select * from loc_orc; + +-- numRows: 16 rawDataSize: 1592 +explain extended select * from (select * from loc_orc union all select * from loc_orc) tmp; + +create database test; +use test; +create table if not exists loc_staging ( + state string, + locid int, + zip bigint, + year int +) row format delimited fields terminated by '|' stored as textfile; + +create table loc_orc like loc_staging; +alter table loc_orc set fileformat orc; + +load data local inpath '../../data/files/loc.txt' overwrite into table loc_staging; + +insert overwrite table loc_orc select * from loc_staging; + +analyze table loc_staging compute statistics; +analyze table loc_staging compute statistics for columns state,locid,zip,year; +analyze table loc_orc compute statistics for columns state,locid,zip,year; + +-- numRows: 16 rawDataSize: 1376 +explain extended select * from (select state from default.loc_orc union all select state from test.loc_orc) temp; + +-- numRows: 16 rawDataSize: 1376 +explain extended select * from (select state from test.loc_staging union all select state from test.loc_orc) temp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q new file mode 100644 index 0000000000000..3788301ebb3d2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ansi_sql_arithmetic.q @@ -0,0 +1,13 @@ + +set hive.compat=latest; + +-- With ansi sql arithmetic enabled, int / int => exact numeric type +explain select cast(key as int) / cast(key as int) from src limit 1; +select cast(key as int) / cast(key as int) from src limit 1; + + +set hive.compat=0.12; + +-- With ansi sql arithmetic disabled, int / int => double +explain select cast(key as int) / cast(key as int) from src limit 1; +select cast(key as int) / cast(key as int) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q index b83eab5d38137..cc9801d8871b8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q @@ -14,7 +14,7 @@ create table tstsrcpart like srcpart; -- to be thrown during the LOAD step. This behavior is now tested in -- clientnegative/archive_corrupt.q -load data local inpath '../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11'); +load data local inpath '../../data/files/archive_corrupt.rc' overwrite into table tstsrcpart partition (ds='2008-04-08', hr='11'); insert overwrite table tstsrcpart partition (ds='2008-04-08', hr='12') select key, value from srcpart where ds='2008-04-08' and hr='12'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q index ddc06a99bb5bc..50c0faa5e4108 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q @@ -1,7 +1,7 @@ set hive.archive.enabled = true; set hive.enforce.bucketing = true; --- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) +-- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.20) drop table tstsrc; drop table tstsrcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q new file mode 100644 index 0000000000000..79ae17ad5da2c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1_sql_std.q @@ -0,0 +1,36 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; + +create table src_autho_test (key STRING, value STRING) ; + +set hive.security.authorization.enabled=true; +set role ADMIN; +--table grant to user + +grant select on table src_autho_test to user user_sauth; + +show grant user user_sauth on table src_autho_test; + + +revoke select on table src_autho_test from user user_sauth; +show grant user user_sauth on table src_autho_test; + +--role +create role src_role; +grant role src_role to user user_sauth; +show role grant user user_sauth; + +--table grant to role + +grant select on table src_autho_test to role src_role; + +show grant role src_role on table src_autho_test; +revoke select on table src_autho_test from role src_role; + +-- drop role +drop role src_role; + +set hive.security.authorization.enabled=false; +drop table src_autho_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q index 4fc79b9ed294a..3353c534e1793 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q @@ -6,7 +6,7 @@ ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="T set hive.security.authorization.enabled=true; -- column grant to user -grant Create on table authorization_part to user hive_test_user; +grant Create on authorization_part to user hive_test_user; grant Update on table authorization_part to user hive_test_user; grant Drop on table authorization_part to user hive_test_user; grant select on table src_auth_tmp to user hive_test_user; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q new file mode 100644 index 0000000000000..1abe659fa4476 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_9.q @@ -0,0 +1,17 @@ +-- SORT_BEFORE_DIFF + +create table dummy (key string, value string); + +grant select on database default to user hive_test_user; +grant select on table dummy to user hive_test_user; +grant select (key, value) on table dummy to user hive_test_user; + +show grant user hive_test_user on database default; +show grant user hive_test_user on table dummy; +show grant user hive_test_user on all; + +grant select on database default to user hive_test_user2; +grant select on table dummy to user hive_test_user2; +grant select (key, value) on table dummy to user hive_test_user2; + +show grant on all; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q new file mode 100644 index 0000000000000..45c4a7dc85b72 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_admin_almighty1.q @@ -0,0 +1,17 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_test_user; + +-- actions from admin should work as if admin has all privileges + +create table t1(i int); +set user.name=hive_admin_user; + +show current roles; +set role ADMIN; +show current roles; +select * from t1; +grant all on table t1 to user user1; +show grant user user1 on table t1; +drop table t1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q new file mode 100644 index 0000000000000..65a7b339c2b76 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_func1.q @@ -0,0 +1,14 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=hive_admin_user; + +-- admin required for create function +set role ADMIN; + +create temporary function temp_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; +create function perm_fn as 'org.apache.hadoop.hive.ql.udf.UDFAscii'; + +drop temporary function temp_fn; +drop function perm_fn; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q new file mode 100644 index 0000000000000..fb60500b899ec --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_macro1.q @@ -0,0 +1,12 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=hive_admin_user; + +-- admin required for create macro +set role ADMIN; + +create temporary macro mymacro1(x double) x * x; + +drop temporary macro mymacro1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q new file mode 100644 index 0000000000000..17f4861cd20f8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_create_table_owner_privs.q @@ -0,0 +1,10 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; + +create table create_table_creator_priv_test(i int); + +-- all privileges should have been set for user + +show grant user user1 on table create_table_creator_priv_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q new file mode 100644 index 0000000000000..8473178cd6607 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_public_role.q @@ -0,0 +1,18 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE t_gpr1(i int); + +-- all privileges should have been set for user + +GRANT ALL ON t_gpr1 TO ROLE public; + +SHOW GRANT ON TABLE t_gpr1; + +set user.name=user2; +SHOW CURRENT ROLES; +-- user2 should be able to do a describe table, as pubic is in the current roles +DESC t_gpr1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q new file mode 100644 index 0000000000000..02d364edb488d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_grant_table_priv.q @@ -0,0 +1,43 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE table_priv1(i int); + +-- all privileges should have been set for user + +-- grant insert privilege to another user +GRANT INSERT ON table_priv1 TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv1; + +-- grant select privilege to another user with grant +GRANT SELECT ON table_priv1 TO USER user2 with grant option; +SHOW GRANT USER user2 ON TABLE table_priv1; + +set user.name=user2; +-- change to other user - user2 +-- grant permissions to another user as user2 +GRANT SELECT ON table_priv1 TO USER user3 with grant option; +SHOW GRANT USER user3 ON TABLE table_priv1; + +set user.name=user3; +-- change to other user - user3 +-- grant permissions to another user as user3 +GRANT SELECT ON table_priv1 TO USER user4 with grant option; +SHOW GRANT USER user4 ON TABLE table_priv1; + +set user.name=user1; +-- switched back to table owner + +-- grant all with grant to user22 +GRANT ALL ON table_priv1 TO USER user22 with grant option; +SHOW GRANT USER user22 ON TABLE table_priv1; + +set user.name=user22; + +-- grant all without grant to user33 +GRANT ALL ON table_priv1 TO USER user33 with grant option; +SHOW GRANT USER user33 ON TABLE table_priv1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q new file mode 100644 index 0000000000000..1f177ffd1fadb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_index.q @@ -0,0 +1,12 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.stats.dbclass=fs; +set hive.security.authorization.enabled=true; +create table t1 (a int); +create index t1_index on table t1(a) as 'COMPACT' WITH DEFERRED REBUILD; +desc formatted default__t1_t1_index__; +alter index t1_index on t1 rebuild; + +drop table t1; + +set hive.security.authorization.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q new file mode 100644 index 0000000000000..85d8b1114b010 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions.q @@ -0,0 +1,16 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- actions that require user to be table owner +create table t1(i int); + +ALTER TABLE t1 SET SERDEPROPERTIES ('field.delim' = ','); +drop table t1; + +create table t1(i int); +create view vt1 as select * from t1; + +drop view vt1; +alter table t1 rename to tnew1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q new file mode 100644 index 0000000000000..36ab2600dcd2d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_owner_actions_db.q @@ -0,0 +1,21 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=hive_admin_user; + +set role admin; +-- create role, db, make role the owner of db +create role testrole; +grant role testrole to user hrt_1; +create database testdb; +alter database testdb set owner role testrole; +desc database testdb; + +-- actions that require user to be db owner +-- create table +use testdb; +create table foobar (foo string, bar string); + +-- drop db +drop database testdb cascade; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q new file mode 100644 index 0000000000000..bee091b1d349c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_parts.q @@ -0,0 +1,19 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part1; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/a_uri_add_part2; + + + + +-- check add partition without insert privilege +create table tpart(i int, j int) partitioned by (k string); + +alter table tpart add partition (k = '1') location '${system:test.tmp.dir}/a_uri_add_part1/'; +alter table tpart add partition (k = '2') location '${system:test.tmp.dir}/a_uri_add_part2/'; + +select count(*) from tpart; + +analyze table tpart partition (k) compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q new file mode 100644 index 0000000000000..ccda3b5157979 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_revoke_table_priv.q @@ -0,0 +1,61 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set user.name=user1; +-- current user has been set (comment line before the set cmd is resulting in parse error!!) + +CREATE TABLE table_priv_rev(i int); + +-- grant insert privilege to user2 +GRANT INSERT ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- revoke insert privilege from user2 +REVOKE INSERT ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- grant all privileges one at a time -- +-- grant insert privilege to user2 +GRANT INSERT ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- grant select privilege to user2, with grant option +GRANT SELECT ON table_priv_rev TO USER user2 WITH GRANT OPTION; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- grant update privilege to user2 +GRANT UPDATE ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- grant delete privilege to user2 +GRANT DELETE ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + + +-- start revoking -- +-- revoke update privilege from user2 +REVOKE UPDATE ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- revoke DELETE privilege from user2 +REVOKE DELETE ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- revoke insert privilege from user2 +REVOKE INSERT ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +-- revoke select privilege from user2 +REVOKE SELECT ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; +SHOW GRANT USER user2 ON ALL; + +-- grant all followed by revoke all +GRANT ALL ON table_priv_rev TO USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; + +REVOKE ALL ON TABLE table_priv_rev FROM USER user2; +SHOW GRANT USER user2 ON TABLE table_priv_rev; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q new file mode 100644 index 0000000000000..f89d0dc985688 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant1.q @@ -0,0 +1,38 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; + +-- enable sql standard authorization +-- role granting without role keyword +set role ADMIN; +create role src_role2; +grant src_role2 to user user2 ; +show role grant user user2; +show roles; + +-- revoke role without role keyword +revoke src_role2 from user user2; +show role grant user user2; +show roles; + +---------------------------------------- +-- role granting without role keyword, with admin option (syntax check) +---------------------------------------- + +create role src_role_wadmin; +grant src_role_wadmin to user user2 with admin option; +show role grant user user2; + +-- revoke role without role keyword +revoke src_role_wadmin from user user2; +show role grant user user2; + + + +-- drop roles +show roles; +drop role src_role2; +show roles; +drop role src_role_wadmin; +show roles; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q new file mode 100644 index 0000000000000..984d7ed1d091a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_role_grant2.q @@ -0,0 +1,34 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + +set hive.cli.print.header=true; +set user.name=hive_admin_user; +set role ADMIN; + +---------------------------------------- +-- role granting with admin option +---------------------------------------- + +create role src_role_wadmin; +grant src_role_wadmin to user user2 with admin option; +show role grant user user2; +show principals src_role_wadmin; + +set user.name=user2; +set role src_role_wadmin; +grant src_role_wadmin to user user3; +show role grant user user3; + +set user.name=hive_admin_user; +set role ADMIN; +show principals src_role_wadmin; + +set user.name=user2; +set role src_role_wadmin; +revoke src_role_wadmin from user user3; +show role grant user user3; + +set user.name=hive_admin_user; +set role ADMIN; +show principals src_role_wadmin; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q new file mode 100644 index 0000000000000..6b5af6e94e16a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_set_show_current_role.q @@ -0,0 +1,21 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set user.name=hive_admin_user; +set role ADMIN; +show current roles; + +create role r1; +grant role r1 to user hive_admin_user; +set role r1; +show current roles; + +set role PUBLIC; +show current roles; + +set role ALL; +show current roles; + +set role ADMIN; +drop role r1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q new file mode 100644 index 0000000000000..bd7bbfedf83e8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_view_sqlstd.q @@ -0,0 +1,66 @@ +set hive.users.in.admin.role=hive_admin_user; +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; +set hive.security.authorization.enabled=true; +set user.name=user1; + +-- Test view authorization , and 'show grant' variants + +create table t1(i int, j int, k int); +show grant on table t1; + +-- protecting certain columns +create view vt1 as select i,k from t1; + +-- protecting certain rows +create view vt2 as select * from t1 where i > 1; + +show grant user user1 on all; + +--view grant to user +-- try with and without table keyword + +grant select on vt1 to user user2; +grant insert on table vt1 to user user3; + +show grant user user2 on table vt1; +show grant user user3 on table vt1; + + +set user.name=user2; +select * from vt1; + +set user.name=user1; + +grant all on table vt2 to user user2; +show grant user user2 on table vt2; +show grant user user2 on all; + +revoke all on vt2 from user user2; +show grant user user2 on table vt2; + +show grant on table vt2; + + +revoke select on table vt1 from user user2; +show grant user user2 on table vt1; + +show grant user user2 on all; + +-- grant privileges on roles for view, after next statement +show grant user user3 on table vt1; + +set user.name=hive_admin_user; +show current roles; +set role ADMIN; +create role role_v; +grant role_v to user user4 ; +show role grant user user4; +show roles; + +grant all on table vt2 to role role_v; +show grant role role_v on table vt2; + +revoke delete on table vt2 from role role_v; +show grant role role_v on table vt2; +show grant on table vt2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q index eaf7489a17025..b8734abfd12ac 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q @@ -1,3 +1,5 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + set hive.auto.convert.join = true; set hive.mapjoin.localtask.max.memory.usage = 0.0001; set hive.mapjoin.check.memory.rows = 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q index 289bfbc6da917..e7846eeecf5c3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q @@ -32,10 +32,10 @@ from studenttab10k_smb s join votertab10k_smb v on (s.name = v.name) group by s.name; -load data local inpath '../data/files/empty1.txt' into table studenttab10k_smb; -load data local inpath '../data/files/empty2.txt' into table studenttab10k_smb; -load data local inpath '../data/files/empty1.txt' into table votertab10k_smb; -load data local inpath '../data/files/empty2.txt' into table votertab10k_smb; +load data local inpath '../../data/files/empty1.txt' into table studenttab10k_smb; +load data local inpath '../../data/files/empty2.txt' into table studenttab10k_smb; +load data local inpath '../../data/files/empty1.txt' into table votertab10k_smb; +load data local inpath '../../data/files/empty2.txt' into table votertab10k_smb; explain select s.name, count(distinct registration) from studenttab10k_smb s join votertab10k_smb v @@ -51,10 +51,10 @@ group by s.name; create table studenttab10k_part (name string, age int, gpa double) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets; create table votertab10k_part (name string, age int, registration string, contributions float) partitioned by (p string) clustered by (name) sorted by (name) into 2 buckets; -load data local inpath '../data/files/empty1.txt' into table studenttab10k_part partition (p='foo'); -load data local inpath '../data/files/empty2.txt' into table studenttab10k_part partition (p='foo'); -load data local inpath '../data/files/empty1.txt' into table votertab10k_part partition (p='foo'); -load data local inpath '../data/files/empty2.txt' into table votertab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty1.txt' into table studenttab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty2.txt' into table studenttab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty1.txt' into table votertab10k_part partition (p='foo'); +load data local inpath '../../data/files/empty2.txt' into table votertab10k_part partition (p='foo'); explain select s.name, count(distinct registration) from studenttab10k_part s join votertab10k_part v diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q index 458504cdc3d04..eefd2111c97b4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q @@ -1,7 +1,7 @@ set hive.auto.convert.join = true; CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in3.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE myinput1; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value; @@ -38,10 +38,10 @@ SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN my CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2; SET hive.optimize.bucketmapjoin = true; SET hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q index 766348d7c001e..d1b7bb40189a4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q @@ -1,7 +1,7 @@ set hive.auto.convert.join = true; CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in1.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE myinput1; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a JOIN myinput1 b; SELECT sum(hash(a.key,a.value,b.key,b.value)) FROM myinput1 a LEFT OUTER JOIN myinput1 b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q index 46a4a0d5339a7..55bd975803d53 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q @@ -1,13 +1,13 @@ -- HIVE-5056 RS has expression list for values, but it's ignored in MapJoinProcessor create table testsrc ( `key` int,`val` string); -load data local inpath '../data/files/kv1.txt' overwrite into table testsrc; +load data local inpath '../../data/files/kv1.txt' overwrite into table testsrc; drop table if exists orderpayment_small; create table orderpayment_small (`dealid` int,`date` string,`time` string, `cityid` int, `userid` int); -insert overwrite table orderpayment_small select 748, '2011-03-24', '2011-03-24', 55 ,5372613 from testsrc limit 1; +insert overwrite table orderpayment_small select 748, '2011-03-24', '2011-03-24', 55 ,5372613 from testsrc tablesample (1 rows); drop table if exists user_small; create table user_small( userid int); -insert overwrite table user_small select key from testsrc limit 100; +insert overwrite table user_small select key from testsrc tablesample (100 rows); set hive.auto.convert.join.noconditionaltask.size = 200; explain extended SELECT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q new file mode 100644 index 0000000000000..f23e227f1ec90 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_without_localtask.q @@ -0,0 +1,29 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; +set hive.auto.convert.join=true; +set hive.auto.convert.join.use.nonstaged=true; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain +select a.* from src a join src b on a.key=b.key limit 40; + +select a.* from src a join src b on a.key=b.key limit 40; + +explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40; + +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value limit 40; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + +set hive.mapjoin.localtask.max.memory.usage = 0.0001; +set hive.mapjoin.check.memory.rows = 2; + +-- fallback to common join +select a.* from src a join src b on a.key=b.key join src c on a.value=c.value where a.key>100 limit 40; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q index e76b560f41899..ddd2c1882ec4d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q @@ -2,19 +2,19 @@ CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); - -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q index f9fa1e4d4e1db..da2e26fde7069 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q @@ -1,19 +1,19 @@ -- small 1 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); - -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q index db53a656a850a..f434b33603603 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q @@ -2,19 +2,19 @@ CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; @@ -23,9 +23,9 @@ set hive.optimize.bucketmapjoin.sortedmerge = true; CREATE TABLE bucket_medium (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_medium partition(ds='2008-04-08'); explain extended select count(*) FROM bucket_small a JOIN bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN bucket_medium d ON c.key = b.key; select count(*) FROM bucket_small a JOIN bucket_medium b ON a.key = b.key JOIN bucket_big c ON c.key = b.key JOIN bucket_medium d ON c.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q new file mode 100644 index 0000000000000..7e9555d7917df --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_16.q @@ -0,0 +1,92 @@ +set hive.auto.convert.join=true; + +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +CREATE TABLE stage_bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (file_tag STRING); + +CREATE TABLE bucket_big +( +key BIGINT, +value STRING +) +PARTITIONED BY (day STRING, pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile; + +CREATE TABLE stage_bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (file_tag STRING); + +CREATE TABLE bucket_small +( +key BIGINT, +value string +) +PARTITIONED BY (pri bigint) +clustered by (key) sorted by (key) into 12 buckets +stored as RCFile; + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='1'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' overwrite into table stage_bucket_small partition (file_tag='2'); + +insert overwrite table bucket_small partition(pri) +select +key, +value, +file_tag as pri +from +stage_bucket_small +where file_tag between 1 and 2; + +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' overwrite into table stage_bucket_big partition (file_tag='1'); + +insert overwrite table bucket_big partition(day,pri) +select +key, +value, +'day1' as day, +1 as pri +from +stage_bucket_big +where +file_tag='1'; + +select +a.key , +a.value , +b.value , +'day1' as day, +1 as pri +from +( +select +key, +value +from bucket_big where day='day1' +) a +left outer join +( +select +key, +value +from bucket_small +where pri between 1 and 2 +) b +on +(a.key = b.key) +; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q index 75339778af802..eef5483b5347d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q @@ -1,16 +1,16 @@ -- small 1 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q index 39a695fdf039d..c094ecdb6be5e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q @@ -1,16 +1,16 @@ -- small 2 part, 2 bucket & big 1 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q index 6072272c478a3..18acfbfb76543 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q @@ -1,18 +1,18 @@ -- small 2 part, 4 bucket & big 1 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q index a28ce3de5fc63..98d6df9b19591 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q @@ -1,13 +1,13 @@ -- small no part, 4 bucket & big no part, 2 bucket CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small; CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; set hive.auto.convert.sortmerge.join=true; set hive.optimize.bucketmapjoin = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q index d62f6377218c6..e19cc317f36e3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q @@ -1,21 +1,21 @@ -- small 2 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q index 6302a1be1477c..a66806f21a8a3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q @@ -1,21 +1,21 @@ -- small 2 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/smallsrcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); - -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.auto.convert.join=true; set hive.auto.convert.sortmerge.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q index 8367206231974..cb6f173ccfa76 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q @@ -35,7 +35,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ ] }'); -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors4; set hive.exec.compress.output=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q index 3fe8ff12b4506..f723cbcc608fb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q @@ -36,7 +36,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors_with_new_field; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors_with_new_field; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors_with_new_field; SELECT * FROM doctors_with_new_field ORDER BY first_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q index 25b77c06270d0..4c33a834668b0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q @@ -37,7 +37,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors4; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors4; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors4; CREATE TABLE episodes ROW FORMAT @@ -70,7 +70,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE episodes; -LOAD DATA LOCAL INPATH '../data/files/episodes.avro' INTO TABLE episodes; +LOAD DATA LOCAL INPATH '../../data/files/episodes.avro' INTO TABLE episodes; SELECT e.title, e.air_date, d.first_name, d.last_name, d.extra_field, e.air_date FROM doctors4 d JOIN episodes e ON (d.number=e.doctor) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q index 584c6f740ba91..f90ceb96f5db9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q @@ -17,7 +17,7 @@ CREATE TABLE test_serializer(string1 STRING, ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' COLLECTION ITEMS TERMINATED BY ':' MAP KEYS TERMINATED BY '#' LINES TERMINATED BY '\n' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/csv.txt' INTO TABLE test_serializer; +LOAD DATA LOCAL INPATH '../../data/files/csv.txt' INTO TABLE test_serializer; CREATE TABLE as_avro ROW FORMAT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q index 8e4d40f2bd59b..6fe5117026ce8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q @@ -28,7 +28,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ ] }'); -LOAD DATA LOCAL INPATH '../data/files/episodes.avro' INTO TABLE episodes; +LOAD DATA LOCAL INPATH '../../data/files/episodes.avro' INTO TABLE episodes; CREATE TABLE episodes_partitioned PARTITIONED BY (doctor_pt INT) @@ -66,7 +66,7 @@ INSERT OVERWRITE TABLE episodes_partitioned PARTITION (doctor_pt) SELECT title, SELECT * FROM episodes_partitioned WHERE doctor_pt > 6 ORDER BY air_date; -- Verify that Fetch works in addition to Map -SELECT * FROM episodes_partitioned LIMIT 5; +SELECT * FROM episodes_partitioned ORDER BY air_date LIMIT 5; -- Fetch w/filter to specific partition SELECT * FROM episodes_partitioned WHERE doctor_pt = 6; -- Fetch w/non-existant partition diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q index e3f8b07b30c59..dbb999503b60f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q @@ -30,7 +30,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors; SELECT * FROM doctors ORDER BY number; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q index e0a8b95401d0b..4f80dc33c9666 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q @@ -1 +1,3 @@ -select cast(cast('a' as binary) as string) from src limit 1; +set hive.fetch.task.conversion=more; + +select cast(cast('a' as binary) as string) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q index eadf07d14d756..1f5c98a23918f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q @@ -3,6 +3,7 @@ drop table ba_test; -- Everything in ba_table1.q + columnar serde in RCFILE. create table ba_test (ba_key binary, ba_val binary) stored as rcfile; +alter table ba_test set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; describe extended ba_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q index a98a2305cfcaf..39c1d25e73397 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q @@ -3,7 +3,7 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '9' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE mytable; +LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE mytable; EXPLAIN SELECT REGEXP_REPLACE(REGEXP_REPLACE(REGEXP_REPLACE(key, '\001', '^A'), '\0', '^@'), '\002', '^B'), value diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q new file mode 100644 index 0000000000000..956a61f7bd3c1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_if_with_path_filter.q @@ -0,0 +1,15 @@ +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/bmjpathfilter; + +create table t1 (dt string) location '${system:test.tmp.dir}/bmjpathfilter/t1'; +Create table t2 (dt string) stored as orc; +dfs -touchz ${system:test.tmp.dir}/bmjpathfilter/t1/_SUCCESS; + +SET hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; +SET hive.optimize.bucketmapjoin=true; + +SELECT /*+ MAPJOIN(b) */ a.dt FROM t1 a JOIN t2 b ON (a.dt = b.dt); + +SET hive.optimize.bucketmapjoin=false; +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; + +dfs -rmr ${system:test.tmp.dir}/bmjpathfilter; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q index 33dd5d5cd2e0a..6bdb09ed64b07 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q @@ -9,8 +9,8 @@ sorted by (key, value) into 1 BUCKETS stored as textfile; create table table2(key string, value string) clustered by (value, key) sorted by (value, key) into 1 BUCKETS stored as textfile; -load data local inpath '../data/files/SortCol1Col2.txt' overwrite into table table1; -load data local inpath '../data/files/SortCol2Col1.txt' overwrite into table table2; +load data local inpath '../../data/files/SortCol1Col2.txt' overwrite into table table1; +load data local inpath '../../data/files/SortCol2Col1.txt' overwrite into table table2; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q index d1097e70a9880..07f6d150ea053 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q @@ -9,8 +9,8 @@ sorted by (key desc, value desc) into 1 BUCKETS stored as textfile; create table table2(key string, value string) clustered by (value, key) sorted by (value desc, key desc) into 1 BUCKETS stored as textfile; -load data local inpath '../data/files/SortCol1Col2.txt' overwrite into table table1; -load data local inpath '../data/files/SortCol2Col1.txt' overwrite into table table2; +load data local inpath '../../data/files/SortCol1Col2.txt' overwrite into table table1; +load data local inpath '../../data/files/SortCol2Col1.txt' overwrite into table table2; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q new file mode 100644 index 0000000000000..c9266a59c342a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez1.q @@ -0,0 +1,85 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); + +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part; + +CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin; + +set hive.convert.join.bucket.mapjoin.tez = true; +explain +select a.key, a.value, b.value +from tab a join tab_part b on a.key = b.key; + +-- one side is really bucketed. srcbucket_mapjoin is not really a bucketed table. +-- In this case the sub-query is chosen as the big table. +explain +select a.k1, a.v1, b.value +from (select sum(substr(srcbucket_mapjoin.value,5)) as v1, key as k1 from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a +join tab b on a.k1 = b.key; + +explain +select a.k1, a.v1, b.value +from (select sum(substr(tab.value,5)) as v1, key as k1 from tab_part join tab on tab_part.key = tab.key GROUP BY tab.key) a +join tab b on a.k1 = b.key; + +explain +select a.k1, a.v1, b.value +from (select sum(substr(x.value,5)) as v1, x.key as k1 from tab x join tab y on x.key = y.key GROUP BY x.key) a +join tab_part b on a.k1 = b.key; + +-- multi-way join +explain +select a.key, a.value, b.value +from tab_part a join tab b on a.key = b.key join tab c on a.key = c.key; + +explain +select a.key, a.value, c.value +from (select x.key, x.value from tab_part x join tab y on x.key = y.key) a join tab c on a.key = c.key; + +-- in this case sub-query is the small table +explain +select a.key, a.value, b.value +from (select key, sum(substr(srcbucket_mapjoin.value,5)) as value from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a +join tab_part b on a.key = b.key; + +set hive.map.aggr=false; +explain +select a.key, a.value, b.value +from (select key, sum(substr(srcbucket_mapjoin.value,5)) as value from srcbucket_mapjoin GROUP BY srcbucket_mapjoin.key) a +join tab_part b on a.key = b.key; + +-- join on non-bucketed column results in broadcast join. +explain +select a.key, a.value, b.value +from tab a join tab_part b on a.value = b.value; + +CREATE TABLE tab1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab1 +select key,value from srcbucket_mapjoin; + +explain +select a.key, a.value, b.value +from tab1 a join tab_part b on a.key = b.key; + +explain select a.key, b.key from tab_part a join tab_part c on a.key = c.key join tab_part b on a.value = b.value; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q new file mode 100644 index 0000000000000..a3588ec94ccee --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_tez2.q @@ -0,0 +1,50 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000; + +CREATE TABLE srcbucket_mapjoin(key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin partition(ds='2008-04-08'); + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); + +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin_part; + +CREATE TABLE tab(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab partition (ds='2008-04-08') +select key,value from srcbucket_mapjoin; + +set hive.convert.join.bucket.mapjoin.tez = true; + +explain select a.key, b.key from tab_part a join tab_part c on a.key = c.key join tab_part b on a.value = b.value; + +CREATE TABLE tab1(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; +insert overwrite table tab1 +select key,value from srcbucket_mapjoin; + +explain +select a.key, a.value, b.value +from tab1 a join src b on a.key = b.key; + +explain +select a.key, b.key from (select key from tab_part where key > 1) a join (select key from tab_part where key > 2) b on a.key = b.key; + +explain +select a.key, b.key from (select key from tab_part where key > 1) a left outer join (select key from tab_part where key > 2) b on a.key = b.key; + +explain +select a.key, b.key from (select key from tab_part where key > 1) a right outer join (select key from tab_part where key > 2) b on a.key = b.key; + +explain select a.key, b.key from (select distinct key from tab) a join tab b on b.key = a.key; + +explain select a.value, b.value from (select distinct value from tab) a join tab b on b.key = a.value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q index 5b013995d5ba4..047a2a5230ae0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q @@ -1,18 +1,18 @@ -- small 1 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q index f952f2ee6d445..d58e8448a09a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q @@ -1,16 +1,16 @@ -- small 1 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q index 461fbb196e758..fd80174f23119 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q @@ -1,16 +1,16 @@ -- small 2 part, 2 bucket & big 1 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q index 366da4473b36d..5d21ea5d63218 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q @@ -1,18 +1,18 @@ -- small 2 part, 4 bucket & big 1 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q index 411fed392a45e..5078072677be6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q @@ -1,13 +1,13 @@ -- small no part, 4 bucket & big no part, 2 bucket CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; CREATE TABLE bucket_big (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big; set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q index 204d1e9010d5d..0f7c72f4eaf43 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q @@ -1,16 +1,16 @@ -- small no part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small; CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q index b0bca460f303f..c528f61302743 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q @@ -1,21 +1,21 @@ -- small 2 part, 4 bucket & big 2 part, 2 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q index 9533c5512f3dd..27c55a36f445b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q @@ -1,21 +1,21 @@ -- small 2 part, 2 bucket & big 2 part, 4 bucket CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-09'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; explain extended select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q index 4c7f0c9595488..d2e12e82d4a26 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q @@ -3,7 +3,7 @@ set mapred.min.split.size = 64; CREATE TABLE T1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE; @@ -26,8 +26,8 @@ EXPLAIN SELECT COUNT(1) FROM T2; SELECT COUNT(1) FROM T2; CREATE TABLE T3(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T3; EXPLAIN SELECT COUNT(1) FROM T3; SELECT COUNT(1) FROM T3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q index 9cdfe8e6e9f6e..8b7535dce1ee2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q @@ -1,17 +1,17 @@ CREATE TABLE bucket_small (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_small partition(ds='2008-04-08'); CREATE TABLE bucket_big (key string, value string) partitioned by (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-08'); -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket_big partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; select /* + MAPJOIN(a) */ count(*) FROM bucket_small a JOIN bucket_big b ON a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q index 2bd8e1e2e5d98..204e75988fb54 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q @@ -25,16 +25,16 @@ select /*+mapjoin(a)*/ a.key, a.value, b.value from srcbucket_mapjoin_part a join srcbucket_mapjoin_part_2 b on a.key=b.key where b.ds="2008-04-08"; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q index cf4222bdd781e..09c0ae2bfcbc1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q @@ -2,23 +2,23 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key) INTO 3 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q index e10ab522a296d..d330b7718c5f2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q @@ -2,25 +2,25 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_1 CLUSTERED BY (key) INTO 4 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='2'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='2'); set hive.optimize.bucketmapjoin=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q index 8139000f4a733..43a9de4e8edef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q @@ -2,20 +2,20 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 NOT CLUSTERED; CREATE TABLE srcbucket_mapjoin_part_3 (key INT, value STRING) PARTITIONED BY (part STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_3 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_3 CLUSTERED BY (key) INTO 2 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q index fdbadfcb04c9c..108b67a9f7644 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q @@ -1,12 +1,12 @@ CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); @@ -76,8 +76,8 @@ from bucketmapjoin_hash_result_1 a left outer join bucketmapjoin_hash_result_2 b on a.key = b.key; -- HIVE-3210 -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q index 8fda80231522e..78c23d5132503 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q @@ -1,16 +1,16 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q index c1a8f2ab7bfa0..54626e7c48ded 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q @@ -1,17 +1,17 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q index 2df49b64f3227..72cffc2da2874 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q @@ -1,22 +1,22 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-09'); CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q index 3a96c642b3e4c..a15570b77687f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q @@ -2,13 +2,13 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (ds='2008-04-08', hr='0'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (ds STRING, hr STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (ds='2008-04-08', hr='0'); set hive.optimize.bucketmapjoin=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q index 5e8daa5f8f125..f467ea6cb3ea1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q @@ -2,13 +2,13 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 3 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q index 86344a53f5a95..f1d5f581848d0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q @@ -2,14 +2,14 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; CREATE TABLE srcbucket_mapjoin_part_1 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_1 PARTITION (part='1'); CREATE TABLE srcbucket_mapjoin_part_2 (key INT, value STRING) PARTITIONED BY (part STRING) CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; @@ -28,8 +28,8 @@ ON a.key = b.key AND a.part = '1' and b.part = '1'; ALTER TABLE srcbucket_mapjoin_part_2 DROP PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part_2 PARTITION (part='1'); ALTER TABLE srcbucket_mapjoin_part_2 CLUSTERED BY (key) INTO 2 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q index d7634333e2fee..ea140ddda91fe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q @@ -3,13 +3,13 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 3 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q index 901f0567590ec..e2c0d8c591719 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q @@ -1,12 +1,12 @@ CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-09'); set hive.optimize.bucketmapjoin = true; create table bucketmapjoin_tmp_result (key string , value1 string, value2 string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q index d66e1238bb79b..6398fff909eab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q @@ -8,21 +8,21 @@ create table test2 (key string, value string) clustered by (value) sorted by (va create table test3 (key string, value string) clustered by (key, value) sorted by (key, value) into 3 buckets; create table test4 (key string, value string) clustered by (value, key) sorted by (value, key) into 3 buckets; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test1; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test1; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test1; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test1; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test1; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test2; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test2; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test2; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test2; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test2; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test2; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test3; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test3; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test3; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test3; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test3; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test3; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE test4; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE test4; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE test4; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE test4; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE test4; +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE test4; set hive.optimize.bucketmapjoin = true; -- should be allowed diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q index 729ffdc8687b5..b1551f2feb900 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + -- cast string floats to integer types select cast('1' as float), @@ -27,4 +29,4 @@ select cast('127' as tinyint), cast('1.0a' as int), cast('-1.-1' as int) -from src limit 1; +from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q new file mode 100644 index 0000000000000..840a84a7b3e30 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_1.q @@ -0,0 +1,32 @@ +drop table char1; +drop table char1_1; + +create table char1 (key char(10), value char(20)); +create table char1_1 (key string, value string); + +-- load from file +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table char1; +select * from char1 order by key, value limit 2; + +-- insert overwrite, from same/different length char +insert overwrite table char1 + select cast(key as char(10)), cast(value as char(15)) from src order by key, value limit 2; +select key, value from char1 order by key, value; + +-- insert overwrite, from string +insert overwrite table char1 + select key, value from src order by key, value limit 2; +select key, value from char1 order by key, value; + +-- insert string from char +insert overwrite table char1_1 + select key, value from char1 order by key, value limit 2; +select key, value from char1_1 order by key, value; + +-- respect string length +insert overwrite table char1 + select key, cast(value as char(3)) from src order by key, value limit 2; +select key, value from char1 order by key, value; + +drop table char1; +drop table char1_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q new file mode 100644 index 0000000000000..3e4900cb7c949 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_2.q @@ -0,0 +1,36 @@ +drop table char_2; + +create table char_2 ( + key char(10), + value char(20) +); + +insert overwrite table char_2 select * from src; + +select value, sum(cast(key as int)), count(*) numrows +from src +group by value +order by value asc +limit 5; + +-- should match the query from src +select value, sum(cast(key as int)), count(*) numrows +from char_2 +group by value +order by value asc +limit 5; + +select value, sum(cast(key as int)), count(*) numrows +from src +group by value +order by value desc +limit 5; + +-- should match the query from src +select value, sum(cast(key as int)), count(*) numrows +from char_2 +group by value +order by value desc +limit 5; + +drop table char_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q new file mode 100644 index 0000000000000..7f44d4d508537 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_cast.q @@ -0,0 +1,92 @@ + +-- Cast from char to other data types +select + cast(cast('11' as string) as tinyint), + cast(cast('11' as string) as smallint), + cast(cast('11' as string) as int), + cast(cast('11' as string) as bigint), + cast(cast('11.00' as string) as float), + cast(cast('11.00' as string) as double), + cast(cast('11.00' as string) as decimal) +from src limit 1; + +select + cast(cast('11' as char(10)) as tinyint), + cast(cast('11' as char(10)) as smallint), + cast(cast('11' as char(10)) as int), + cast(cast('11' as char(10)) as bigint), + cast(cast('11.00' as char(10)) as float), + cast(cast('11.00' as char(10)) as double), + cast(cast('11.00' as char(10)) as decimal) +from src limit 1; + +select + cast(cast('2011-01-01' as string) as date), + cast(cast('2011-01-01 01:02:03' as string) as timestamp) +from src limit 1; + +select + cast(cast('2011-01-01' as char(10)) as date), + cast(cast('2011-01-01 01:02:03' as char(30)) as timestamp) +from src limit 1; + +-- no tests from string/char to boolean, that conversion doesn't look useful +select + cast(cast('abc123' as string) as string), + cast(cast('abc123' as string) as varchar(10)), + cast(cast('abc123' as string) as char(10)) +from src limit 1; + +select + cast(cast('abc123' as char(10)) as string), + cast(cast('abc123' as char(10)) as varchar(10)), + cast(cast('abc123' as char(10)) as char(10)) +from src limit 1; + +select + cast(cast('abc123' as varchar(10)) as string), + cast(cast('abc123' as varchar(10)) as varchar(10)), + cast(cast('abc123' as varchar(10)) as char(10)) +from src limit 1; + +-- cast from other types to char +select + cast(cast(11 as tinyint) as string), + cast(cast(11 as smallint) as string), + cast(cast(11 as int) as string), + cast(cast(11 as bigint) as string), + cast(cast(11.00 as float) as string), + cast(cast(11.00 as double) as string), + cast(cast(11.00 as decimal) as string) +from src limit 1; + +select + cast(cast(11 as tinyint) as char(10)), + cast(cast(11 as smallint) as char(10)), + cast(cast(11 as int) as char(10)), + cast(cast(11 as bigint) as char(10)), + cast(cast(11.00 as float) as char(10)), + cast(cast(11.00 as double) as char(10)), + cast(cast(11.00 as decimal) as char(10)) +from src limit 1; + +select + cast(date '2011-01-01' as string), + cast(timestamp('2011-01-01 01:02:03') as string) +from src limit 1; + +select + cast(date '2011-01-01' as char(10)), + cast(timestamp('2011-01-01 01:02:03') as char(30)) +from src limit 1; + +select + cast(true as string), + cast(false as string) +from src limit 1; + +select + cast(true as char(10)), + cast(false as char(10)) +from src limit 1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q new file mode 100644 index 0000000000000..e1cfdb2bdb07a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_comparison.q @@ -0,0 +1,40 @@ + +-- Should all be true +select + cast('abc' as char(10)) = cast('abc' as char(10)), + cast('abc' as char(10)) <= cast('abc' as char(10)), + cast('abc' as char(10)) >= cast('abc' as char(10)), + cast('abc' as char(10)) < cast('abd' as char(10)), + cast('abc' as char(10)) > cast('abb' as char(10)), + cast('abc' as char(10)) <> cast('abb' as char(10)) +from src limit 1; + +-- Different char lengths should still compare the same +select + cast('abc' as char(10)) = cast('abc' as char(3)), + cast('abc' as char(10)) <= cast('abc' as char(3)), + cast('abc' as char(10)) >= cast('abc' as char(3)), + cast('abc' as char(10)) < cast('abd' as char(3)), + cast('abc' as char(10)) > cast('abb' as char(3)), + cast('abc' as char(10)) <> cast('abb' as char(3)) +from src limit 1; + +-- Should work with string types as well +select + cast('abc' as char(10)) = 'abc', + cast('abc' as char(10)) <= 'abc', + cast('abc' as char(10)) >= 'abc', + cast('abc' as char(10)) < 'abd', + cast('abc' as char(10)) > 'abb', + cast('abc' as char(10)) <> 'abb' +from src limit 1; + +-- leading space is significant for char +select + cast(' abc' as char(10)) <> cast('abc' as char(10)) +from src limit 1; + +-- trailing space is not significant for char +select + cast('abc ' as char(10)) = cast('abc' as char(10)) +from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q new file mode 100644 index 0000000000000..373352dee0078 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_join1.q @@ -0,0 +1,35 @@ +drop table char_join1_ch1; +drop table char_join1_ch2; +drop table char_join1_str; + +create table char_join1_ch1 ( + c1 int, + c2 char(10) +); + +create table char_join1_ch2 ( + c1 int, + c2 char(20) +); + +create table char_join1_str ( + c1 int, + c2 string +); + +load data local inpath '../../data/files/vc1.txt' into table char_join1_ch1; +load data local inpath '../../data/files/vc1.txt' into table char_join1_ch2; +load data local inpath '../../data/files/vc1.txt' into table char_join1_str; + +-- Join char with same length char +select * from char_join1_ch1 a join char_join1_ch1 b on (a.c2 = b.c2) order by a.c1; + +-- Join char with different length char +select * from char_join1_ch1 a join char_join1_ch2 b on (a.c2 = b.c2) order by a.c1; + +-- Join char with string +select * from char_join1_ch1 a join char_join1_str b on (a.c2 = b.c2) order by a.c1; + +drop table char_join1_ch1; +drop table char_join1_ch2; +drop table char_join1_str; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q new file mode 100644 index 0000000000000..c710b6c73174b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_nested_types.q @@ -0,0 +1,53 @@ +drop table char_nested_1; +drop table char_nested_array; +drop table char_nested_map; +drop table char_nested_struct; +drop table char_nested_cta; +drop table char_nested_view; + +create table char_nested_1 (key int, value char(20)); +insert overwrite table char_nested_1 + select key, value from src order by key limit 1; + +-- arrays +create table char_nested_array (c1 array); +insert overwrite table char_nested_array + select array(value, value) from char_nested_1; +describe char_nested_array; +select * from char_nested_array; + +-- maps +create table char_nested_map (c1 map); +insert overwrite table char_nested_map + select map(key, value) from char_nested_1; +describe char_nested_map; +select * from char_nested_map; + +-- structs +create table char_nested_struct (c1 struct); +insert overwrite table char_nested_struct + select named_struct('a', key, + 'b', value, + 'c', cast(value as string)) + from char_nested_1; +describe char_nested_struct; +select * from char_nested_struct; + +-- nested type with create table as +create table char_nested_cta as + select * from char_nested_struct; +describe char_nested_cta; +select * from char_nested_cta; + +-- nested type with view +create table char_nested_view as + select * from char_nested_struct; +describe char_nested_view; +select * from char_nested_view; + +drop table char_nested_1; +drop table char_nested_array; +drop table char_nested_map; +drop table char_nested_struct; +drop table char_nested_cta; +drop table char_nested_view; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q new file mode 100644 index 0000000000000..4340b4de6d597 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_serde.q @@ -0,0 +1,102 @@ +drop table if exists char_serde_regex; +drop table if exists char_serde_lb; +drop table if exists char_serde_ls; +drop table if exists char_serde_c; +drop table if exists char_serde_lbc; +drop table if exists char_serde_orc; + +-- +-- RegexSerDe +-- +create table char_serde_regex ( + key char(15), + value char(20) +) +row format serde 'org.apache.hadoop.hive.serde2.RegexSerDe' +with serdeproperties ( + "input.regex" = "([^]*)([^]*)" +) +stored as textfile; + +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table char_serde_regex; + +select * from char_serde_regex limit 5; +select value, count(*) from char_serde_regex group by value limit 5; + +-- +-- LazyBinary +-- +create table char_serde_lb ( + key char(15), + value char(20) +); +alter table char_serde_lb set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; + +insert overwrite table char_serde_lb + select key, value from char_serde_regex; +select * from char_serde_lb limit 5; +select value, count(*) from char_serde_lb group by value limit 5; + +-- +-- LazySimple +-- +create table char_serde_ls ( + key char(15), + value char(20) +); +alter table char_serde_ls set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; + +insert overwrite table char_serde_ls + select key, value from char_serde_lb; +select * from char_serde_ls limit 5; +select value, count(*) from char_serde_ls group by value limit 5; + +-- +-- Columnar +-- +create table char_serde_c ( + key char(15), + value char(20) +) stored as rcfile; +alter table char_serde_c set serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'; + +insert overwrite table char_serde_c + select key, value from char_serde_ls; +select * from char_serde_c limit 5; +select value, count(*) from char_serde_c group by value limit 5; + +-- +-- LazyBinaryColumnar +-- +create table char_serde_lbc ( + key char(15), + value char(20) +) stored as rcfile; +alter table char_serde_lbc set serde 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; + +insert overwrite table char_serde_lbc + select key, value from char_serde_c; +select * from char_serde_lbc limit 5; +select value, count(*) from char_serde_lbc group by value limit 5; + +-- +-- ORC +-- +create table char_serde_orc ( + key char(15), + value char(20) +) stored as orc; +alter table char_serde_orc set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; + + +insert overwrite table char_serde_orc + select key, value from char_serde_lbc; +select * from char_serde_orc limit 5; +select value, count(*) from char_serde_orc group by value limit 5; + +drop table if exists char_serde_regex; +drop table if exists char_serde_lb; +drop table if exists char_serde_ls; +drop table if exists char_serde_c; +drop table if exists char_serde_lbc; +drop table if exists char_serde_orc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q new file mode 100644 index 0000000000000..629d41dca9870 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_udf1.q @@ -0,0 +1,156 @@ +drop table char_udf_1; + +create table char_udf_1 (c1 string, c2 string, c3 char(10), c4 char(20)); +insert overwrite table char_udf_1 + select key, value, key, value from src where key = '238' limit 1; + +-- UDFs with char support +select + concat(c1, c2), + concat(c3, c4), + concat(c1, c2) = concat(c3, c4) +from char_udf_1 limit 1; + +select + upper(c2), + upper(c4), + upper(c2) = upper(c4) +from char_udf_1 limit 1; + +select + lower(c2), + lower(c4), + lower(c2) = lower(c4) +from char_udf_1 limit 1; + +-- Scalar UDFs +select + ascii(c2), + ascii(c4), + ascii(c2) = ascii(c4) +from char_udf_1 limit 1; + +select + concat_ws('|', c1, c2), + concat_ws('|', c3, c4), + concat_ws('|', c1, c2) = concat_ws('|', c3, c4) +from char_udf_1 limit 1; + +select + decode(encode(c2, 'US-ASCII'), 'US-ASCII'), + decode(encode(c4, 'US-ASCII'), 'US-ASCII'), + decode(encode(c2, 'US-ASCII'), 'US-ASCII') = decode(encode(c4, 'US-ASCII'), 'US-ASCII') +from char_udf_1 limit 1; + +select + instr(c2, '_'), + instr(c4, '_'), + instr(c2, '_') = instr(c4, '_') +from char_udf_1 limit 1; + +select + length(c2), + length(c4), + length(c2) = length(c4) +from char_udf_1 limit 1; + +select + locate('a', 'abcdabcd', 3), + locate(cast('a' as char(1)), cast('abcdabcd' as char(10)), 3), + locate('a', 'abcdabcd', 3) = locate(cast('a' as char(1)), cast('abcdabcd' as char(10)), 3) +from char_udf_1 limit 1; + +select + lpad(c2, 15, ' '), + lpad(c4, 15, ' '), + lpad(c2, 15, ' ') = lpad(c4, 15, ' ') +from char_udf_1 limit 1; + +select + ltrim(c2), + ltrim(c4), + ltrim(c2) = ltrim(c4) +from char_udf_1 limit 1; + +select + regexp(c2, 'val'), + regexp(c4, 'val'), + regexp(c2, 'val') = regexp(c4, 'val') +from char_udf_1 limit 1; + +select + regexp_extract(c2, 'val_([0-9]+)', 1), + regexp_extract(c4, 'val_([0-9]+)', 1), + regexp_extract(c2, 'val_([0-9]+)', 1) = regexp_extract(c4, 'val_([0-9]+)', 1) +from char_udf_1 limit 1; + +select + regexp_replace(c2, 'val', 'replaced'), + regexp_replace(c4, 'val', 'replaced'), + regexp_replace(c2, 'val', 'replaced') = regexp_replace(c4, 'val', 'replaced') +from char_udf_1 limit 1; + +select + reverse(c2), + reverse(c4), + reverse(c2) = reverse(c4) +from char_udf_1 limit 1; + +select + rpad(c2, 15, ' '), + rpad(c4, 15, ' '), + rpad(c2, 15, ' ') = rpad(c4, 15, ' ') +from char_udf_1 limit 1; + +select + rtrim(c2), + rtrim(c4), + rtrim(c2) = rtrim(c4) +from char_udf_1 limit 1; + +select + sentences('See spot run. See jane run.'), + sentences(cast('See spot run. See jane run.' as char(50))) +from char_udf_1 limit 1; + +select + split(c2, '_'), + split(c4, '_') +from char_udf_1 limit 1; + +select + str_to_map('a:1,b:2,c:3',',',':'), + str_to_map(cast('a:1,b:2,c:3' as char(20)),',',':') +from char_udf_1 limit 1; + +select + substr(c2, 1, 3), + substr(c4, 1, 3), + substr(c2, 1, 3) = substr(c4, 1, 3) +from char_udf_1 limit 1; + +select + trim(c2), + trim(c4), + trim(c2) = trim(c4) +from char_udf_1 limit 1; + + +-- Aggregate Functions +select + compute_stats(c2, 16), + compute_stats(c4, 16) +from char_udf_1; + +select + min(c2), + min(c4) +from char_udf_1; + +select + max(c2), + max(c4) +from char_udf_1; + + +drop table char_udf_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q new file mode 100644 index 0000000000000..2ce5e89c2aa9e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_union1.q @@ -0,0 +1,47 @@ +drop table char_union1_ch1; +drop table char_union1_ch2; +drop table char_union1_str; + +create table char_union1_ch1 ( + c1 int, + c2 char(10) +); + +create table char_union1_ch2 ( + c1 int, + c2 char(20) +); + +create table char_union1_str ( + c1 int, + c2 string +); + +load data local inpath '../../data/files/vc1.txt' into table char_union1_ch1; +load data local inpath '../../data/files/vc1.txt' into table char_union1_ch2; +load data local inpath '../../data/files/vc1.txt' into table char_union1_str; + +-- union char with same length char +select * from ( + select * from char_union1_ch1 + union all + select * from char_union1_ch1 limit 1 +) q1 sort by c1; + +-- union char with different length char +select * from ( + select * from char_union1_ch1 + union all + select * from char_union1_ch2 limit 1 +) q1 sort by c1; + +-- union char with string +select * from ( + select * from char_union1_ch1 + union all + select * from char_union1_str limit 1 +) q1 sort by c1; + +drop table char_union1_ch1; +drop table char_union1_ch2; +drop table char_union1_str; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q new file mode 100644 index 0000000000000..332b84087e8ef --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/char_varchar_udf.q @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS char_varchar_udf; + +CREATE TABLE char_varchar_udf (c char(8), vc varchar(10)) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; +LOAD DATA LOCAL INPATH '../../data/files/char_varchar_udf.txt' INTO TABLE char_varchar_udf; + +SELECT ROUND(c, 2), ROUND(vc, 3) FROM char_varchar_udf; +SELECT AVG(c), AVG(vc), SUM(c), SUM(vc) FROM char_varchar_udf; + +DROP TABLE char_varchar_udf; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q index 3c8a309991d75..fbf8bba56e52c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q @@ -4,7 +4,7 @@ SET hive.stats.collect.scancols=true; -- This test is used for testing the ColumnAccessAnalyzer CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q index 98627776d3163..9dfe8ffbc3058 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q @@ -4,8 +4,8 @@ DROP TABLE Employee_Part; CREATE TABLE Employee_Part(employeeID int, employeeName String) partitioned by (employeeSalary double) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=2000.0); -LOAD DATA LOCAL INPATH "../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=4000.0); +LOAD DATA LOCAL INPATH "../../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=2000.0); +LOAD DATA LOCAL INPATH "../../data/files/employee.dat" INTO TABLE Employee_Part partition(employeeSalary=4000.0); explain analyze table Employee_Part partition (employeeSalary=2000.0) compute statistics for columns employeeID; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q index 72d88a67b5212..170fbc51916a0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q @@ -13,7 +13,7 @@ CREATE TABLE UserVisits_web_text_none ( avgTimeOnSite int) row format delimited fields terminated by '|' stored as textfile; -LOAD DATA LOCAL INPATH "../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; +LOAD DATA LOCAL INPATH "../../data/files/UserVisits.dat" INTO TABLE UserVisits_web_text_none; explain analyze table UserVisits_web_text_none compute statistics for columns sourceIP, avgTimeOnSite, adRevenue; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q new file mode 100644 index 0000000000000..6be02ec8a7bbb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compile_processor.q @@ -0,0 +1,12 @@ + +compile `import org.apache.hadoop.hive.ql.exec.UDF \; +public class Pyth extends UDF { + public double evaluate(double a, double b){ + return Math.sqrt((a*a) + (b*b)) \; + } +} ` AS GROOVY NAMED Pyth.groovy; +CREATE TEMPORARY FUNCTION Pyth as 'Pyth'; + +SELECT Pyth(3,4) FROM src tablesample (1 rows); + +DROP TEMPORARY FUNCTION Pyth; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q index c19813602782f..fd15634f202f7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q @@ -1,7 +1,7 @@ create table tab_binary(a binary); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/binary.txt" INTO TABLE tab_binary; +LOAD DATA LOCAL INPATH "../../data/files/binary.txt" INTO TABLE tab_binary; select count(*) from tab_binary; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q index dc76f7c7d57e1..cddb53f8f64f5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q @@ -1,7 +1,7 @@ create table tab_bool(a boolean); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/bool.txt" INTO TABLE tab_bool; +LOAD DATA LOCAL INPATH "../../data/files/bool.txt" INTO TABLE tab_bool; select count(*) from tab_bool; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q new file mode 100644 index 0000000000000..77ec066dad985 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_decimal.q @@ -0,0 +1,11 @@ +set hive.stats.autogather=true; + +create table tab_decimal(a decimal(10,3)); + +-- insert some data +LOAD DATA LOCAL INPATH "../../data/files/decimal.txt" INTO TABLE tab_decimal; + +select count(*) from tab_decimal; + +-- compute statistical summary of data +select compute_stats(a, 18) from tab_decimal; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q index 6c6dc47e67701..7a1e0f6295dc4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q @@ -1,7 +1,7 @@ create table tab_double(a double); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/double.txt" INTO TABLE tab_double; +LOAD DATA LOCAL INPATH "../../data/files/double.txt" INTO TABLE tab_double; select count(*) from tab_double; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q index 7d0a1584a28cc..6a2070f7808f5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q @@ -1,7 +1,7 @@ create table tab_int(a int); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/int.txt" INTO TABLE tab_int; +LOAD DATA LOCAL INPATH "../../data/files/int.txt" INTO TABLE tab_int; select count(*) from tab_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q index f146f6b95b544..0023e7f6bd818 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q @@ -1,7 +1,7 @@ create table tab_string(a string); -- insert some data -LOAD DATA LOCAL INPATH "../data/files/string.txt" INTO TABLE tab_string; +LOAD DATA LOCAL INPATH "../../data/files/string.txt" INTO TABLE tab_string; select count(*) from tab_string; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q index ced72d60445c2..d51b80194e90a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, @@ -7,7 +9,7 @@ SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, IF(ARRAY_CONTAINS(MAP_KEYS(MAP("b", "x")), "b"), "F2", "B2"), 2 ).F2 - FROM src LIMIT 1; + FROM src tablesample (1 rows); SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, @@ -17,4 +19,4 @@ SELECT NAMED_STRUCT( IF(ARRAY_CONTAINS(ARRAY(1, 2), 3), "F1", "B1"), 1, IF(ARRAY_CONTAINS(MAP_KEYS(MAP("b", "x")), "b"), "F2", "B2"), 2 ).F2 - FROM src LIMIT 1; + FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q index b3fd3f760eaf0..0596f965ed89c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q @@ -104,7 +104,7 @@ FROM (SELECT x.key AS key, count(1) AS cnt set hive.optimize.correlation=false; -- If the key of a GroupByOperator is the right table's key in --- a Left Outer Join, we cannot use a single MR to execute these two +-- a Left Outer Join, we cannot use a single MR to execute these two -- operators because those keys with a null value are not grouped. EXPLAIN SELECT SUM(HASH(tmp.key)), SUM(HASH(tmp.cnt)) @@ -129,6 +129,29 @@ FROM (SELECT y.key AS key, count(1) AS cnt FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key) GROUP BY y.key) tmp; +set hive.optimize.correlation=false; +-- If a column of the key of a GroupByOperator is the right table's key in +-- a Left Outer Join, we cannot use a single MR to execute these two +-- operators because those keys with a null value are not grouped. +EXPLAIN +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + +set hive.optimize.correlation=true; +EXPLAIN +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + +SELECT x.key, y.value, count(1) AS cnt +FROM src1 x LEFT OUTER JOIN src y ON (x.key = y.key AND x.value = y.value) +GROUP BY x.key, y.value; + set hive.optimize.correlation=false; -- If the key of a GroupByOperator is the right table's key in -- a Right Outer Join, these two operators will be executed in diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q index 70fcdfc0b4288..953d191fc6087 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q @@ -1,9 +1,9 @@ CREATE TABLE T1(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T3; set hive.auto.convert.join=false; set hive.optimize.correlation=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q index ac836c0bfdc91..287c7a389d1a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q @@ -1,11 +1,11 @@ CREATE TABLE T1(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; CREATE TABLE T2(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T2; CREATE TABLE T3(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/kv3.txt' INTO TABLE T3; CREATE TABLE T4(key INT, val STRING); -LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' INTO TABLE T4; CREATE TABLE dest_co1(key INT, val STRING); CREATE TABLE dest_co2(key INT, val STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q index 0d66a5ec345e6..74ae9e428befe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q @@ -1,5 +1,5 @@ create table abcd (a int, b int, c int, d int); -LOAD DATA LOCAL INPATH '../data/files/in4.txt' INTO TABLE abcd; +LOAD DATA LOCAL INPATH '../../data/files/in4.txt' INTO TABLE abcd; select * from abcd; set hive.map.aggr=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q new file mode 100644 index 0000000000000..ad924d345304c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_func1.q @@ -0,0 +1,30 @@ + +-- qtest_get_java_boolean should already be created during test initialization +select qtest_get_java_boolean('true'), qtest_get_java_boolean('false') from src limit 1; + +create database mydb; +create function mydb.func1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'; + +show functions mydb.func1; + +select mydb.func1('abc') from src limit 1; + +drop function mydb.func1; + +-- function should now be gone +show functions mydb.func1; + +-- To test function name resolution +create function mydb.qtest_get_java_boolean as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFUpper'; + +use default; +-- unqualified function should resolve to one in default db +select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1; + +use mydb; +-- unqualified function should resolve to one in mydb db +select qtest_get_java_boolean('abc'), default.qtest_get_java_boolean('abc'), mydb.qtest_get_java_boolean('abc') from default.src limit 1; + +drop function mydb.qtest_get_java_boolean; + +drop database mydb cascade; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q index cb4d6578af476..13539a65f512f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q @@ -21,8 +21,8 @@ INSERT OVERWRITE TABLE table2 SELECT key, value FROM src WHERE key = 100; SELECT * FROM table1; SELECT * FROM table2; -CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test'; -CREATE EXTERNAL TABLE table5 LIKE table4 LOCATION '${system:test.src.data.dir}/files/ext_test'; +CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:hive.root}/data/files/ext_test'; +CREATE EXTERNAL TABLE table5 LIKE table4 LOCATION '${system:hive.root}/data/files/ext_test'; SELECT * FROM table4; SELECT * FROM table5; @@ -31,5 +31,5 @@ DROP TABLE table5; SELECT * FROM table4; DROP TABLE table4; -CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test'; +CREATE EXTERNAL TABLE table4 (a INT) LOCATION '${system:hive.root}/data/files/ext_test'; SELECT * FROM table4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q index 4418b3430affc..483931b6fff4b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q @@ -1,6 +1,6 @@ create table src_rc_merge_test(key int, value string) stored as rcfile; -load data local inpath '../data/files/smbbucket_1.rc' into table src_rc_merge_test; +load data local inpath '../../data/files/smbbucket_1.rc' into table src_rc_merge_test; set hive.exec.compress.output = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q index 2debd0d71d224..735b139719700 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q @@ -9,7 +9,7 @@ CREATE TABLE table1 ( DESCRIBE table1; DESCRIBE EXTENDED table1; -LOAD DATA LOCAL INPATH '../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table1; +LOAD DATA LOCAL INPATH '../../data/files/create_nested_type.txt' OVERWRITE INTO TABLE table1; SELECT * from table1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q index dd5aa63e4578a..1e5d151f286ba 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q @@ -4,7 +4,7 @@ row format delimited fields terminated by '\t' collection items terminated by '\001'; -load data local inpath '../data/files/kv1.txt' +load data local inpath '../../data/files/kv1.txt' overwrite into table abc; SELECT strct, strct.a, strct.b FROM abc LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q index bb0e5b989d54d..6bc4d29358c11 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q @@ -4,7 +4,7 @@ strct struct); create table abc(mydata uniontype,struct>, strct struct); -load data local inpath '../data/files/union_input.txt' +load data local inpath '../../data/files/union_input.txt' overwrite into table abc; SELECT * FROM abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q index 2199750f42f7e..11ba9c8afc61d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q @@ -11,3 +11,14 @@ describe formatted w; drop view v; drop view w; + + +-- HIVE-4116 Can't use views using map datatype. + +CREATE TABLE items (id INT, name STRING, info MAP); + +explain +CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items; +CREATE VIEW priceview AS SELECT items.id, items.info['price'] FROM items; + +select * from priceview; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q new file mode 100644 index 0000000000000..fb38c947b32d4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_1.q @@ -0,0 +1,26 @@ + +create table A as +select * from src; + +create table B as +select * from src +limit 10; + +set hive.auto.convert.join.noconditionaltask.size=100; + +explain select * from A join B; + +explain select * from B d1 join B d2 on d1.key = d2.key join A; + +explain select * from A join + (select d1.key + from B d1 join B d2 on d1.key = d2.key + where 1 = 1 group by d1.key) od1; + +explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1; + +explain select * from +(select A.key from A group by key) ss join +(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q new file mode 100644 index 0000000000000..479d57137e946 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_product_check_2.q @@ -0,0 +1,27 @@ +create table A as +select * from src; + +create table B as +select * from src +limit 10; + +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000000; + +explain select * from A join B; + +explain select * from B d1 join B d2 on d1.key = d2.key join A; + +explain select * from A join + (select d1.key + from B d1 join B d2 on d1.key = d2.key + where 1 = 1 group by d1.key) od1; + +explain select * from A join (select d1.key from B d1 join B d2 where 1 = 1 group by d1.key) od1; + +explain select * from +(select A.key from A group by key) ss join +(select d1.key from B d1 join B d2 on d1.key = d2.key where 1 = 1 group by d1.key) od1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q index e595904b412fa..71af40e7e4140 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q @@ -56,7 +56,7 @@ set hive.exec.mode.local.auto=true; create table nzhang_ctas5 row format delimited fields terminated by ',' lines terminated by '\012' stored as textfile as select key, value from src sort by key, value limit 10; create table nzhang_ctas6 (key string, `to` string); -insert overwrite table nzhang_ctas6 select key, value from src limit 10; +insert overwrite table nzhang_ctas6 select key, value from src tablesample (10 rows); create table nzhang_ctas7 as select key, `to` from nzhang_ctas6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q new file mode 100644 index 0000000000000..ecfe74afd0273 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_char.q @@ -0,0 +1,22 @@ +drop table ctas_char_1; +drop table ctas_char_2; +drop view ctas_char_3; + +create table ctas_char_1 (key char(10), value string); +insert overwrite table ctas_char_1 + select key, value from src sort by key, value limit 5; + +-- create table as with char column +create table ctas_char_2 as select key, value from ctas_char_1; + +-- view with char column +create view ctas_char_3 as select key, value from ctas_char_2; + +select key, value from ctas_char_1; +select * from ctas_char_2; +select * from ctas_char_3; + + +drop table ctas_char_1; +drop table ctas_char_2; +drop view ctas_char_3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q index 4961b971dbdec..f39689de03a55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q @@ -58,11 +58,6 @@ create table nzhang_ctas6 (key string, `to` string); insert overwrite table nzhang_ctas6 select key, value from src limit 10; create table nzhang_ctas7 as select key, `to` from nzhang_ctas6; - - - - - - - - +create table nzhang_ctas8 as select 3.14BD from nzhang_ctas6 limit 1; +desc nzhang_ctas8; +drop table nzhang_ctas8; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q new file mode 100644 index 0000000000000..ca4132f7d3fb9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_1.q @@ -0,0 +1,28 @@ +explain +with q1 as ( select key from src where key = '5') +select * +from q1 +; + +with q1 as ( select key from src where key = '5') +select * +from q1 +; + +-- in subquery +explain +with q1 as ( select key from src where key = '5') +select * from (select key from q1) a; + +with q1 as ( select key from src where key = '5') +select * from (select key from q1) a; + +-- chaining +explain +with q1 as ( select key from q2 where key = '5'), +q2 as ( select key from src where key = '5') +select * from (select key from q1) a; + +with q1 as ( select key from q2 where key = '5'), +q2 as ( select key from src where key = '5') +select * from (select key from q1) a; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q new file mode 100644 index 0000000000000..b49620aca144d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cte_2.q @@ -0,0 +1,56 @@ + +-- union test +with q1 as (select * from src where key= '5'), +q2 as (select * from src s2 where key = '4') +select * from q1 union all select * from q2 +; + +-- insert test +create table s1 like src; +with q1 as ( select key, value from src where key = '5') +from q1 +insert overwrite table s1 +select * +; +select * from s1; +drop table s1; + +-- from style +with q1 as (select * from src where key= '5') +from q1 +select * +; + +-- ctas +create table s2 as +with q1 as ( select key from src where key = '4') +select * from q1 +; + +select * from s2; +drop table s2; + +-- view test +create view v1 as +with q1 as ( select key from src where key = '5') +select * from q1 +; + +select * from v1; + +drop view v1; + + +-- view test, name collision +create view v1 as +with q1 as ( select key from src where key = '5') +select * from q1 +; + +with q1 as ( select key from src where key = '4') +select * from v1 +; + +drop view v1; + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q index ff5e86dc5a6b4..d769d05602a17 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q @@ -1,6 +1,7 @@ -ADD JAR ../build/ql/test/test-udfs.jar; CREATE TABLE src1_rot13_iof(key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.udf.Rot13InputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.udf.Rot13OutputFormat'; +DESCRIBE EXTENDED src1_rot13_iof; +SELECT * FROM src1 ORDER BY key, value; INSERT OVERWRITE TABLE src1_rot13_iof SELECT * FROM src1; -SELECT * FROM src1_rot13_iof; +SELECT * FROM src1_rot13_iof ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q index 9140a42b650e8..e3ceaccfa5ef6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q @@ -58,7 +58,7 @@ SHOW TABLES; DESCRIBE EXTENDED test_table_like; -- LOAD and SELECT -LOAD DATA LOCAL INPATH '../data/files/test.dat' +LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE test_table; SELECT * FROM test_table; @@ -146,7 +146,7 @@ CREATE TABLE db1.src(key STRING, value STRING) STORED AS TEXTFILE; -- LOAD into foreign table -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE db1.src; -- SELECT from foreign table @@ -158,7 +158,7 @@ PARTITIONED BY (ds STRING, hr STRING) STORED AS TEXTFILE; -- LOAD data into Partitioned foreign table -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE db1.srcpart PARTITION (ds='2008-04-08', hr='11'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q index 4e17c7ad5ac10..1371273245215 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q @@ -8,13 +8,14 @@ CREATE DATABASE db5; SHOW DATABASES; USE db5; +set hive.stats.dbclass=fs; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp; dfs -rmr ${system:test.tmp.dir}/dbcascade; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade; -- add a table, index and view CREATE TABLE temp_tbl (id INT, name STRING); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE temp_tbl; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE temp_tbl; CREATE VIEW temp_tbl_view AS SELECT * FROM temp_tbl; CREATE INDEX idx1 ON TABLE temp_tbl(id) AS 'COMPACT' with DEFERRED REBUILD; ALTER INDEX idx1 ON temp_tbl REBUILD; @@ -23,15 +24,15 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp_tbl2; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/temp_tbl2_idx2; -- add a table, index and view with a different storage location CREATE TABLE temp_tbl2 (id INT, name STRING) LOCATION 'file:${system:test.tmp.dir}/dbcascade/temp_tbl2'; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' into table temp_tbl2; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' into table temp_tbl2; CREATE VIEW temp_tbl2_view AS SELECT * FROM temp_tbl2; CREATE INDEX idx2 ON TABLE temp_tbl2(id) AS 'COMPACT' with DEFERRED REBUILD LOCATION 'file:${system:test.tmp.dir}/dbcascade/temp_tbl2_idx2'; ALTER INDEX idx2 ON temp_tbl2 REBUILD; -- add a partitioned table, index and view CREATE TABLE part_tab (id INT, name STRING) PARTITIONED BY (ds string); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab PARTITION (ds='2009-04-09'); CREATE INDEX idx3 ON TABLE part_tab(id) AS 'COMPACT' with DEFERRED REBUILD; ALTER INDEX idx3 ON part_tab PARTITION (ds='2008-04-09') REBUILD; ALTER INDEX idx3 ON part_tab PARTITION (ds='2009-04-09') REBUILD; @@ -41,8 +42,8 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab2_idx4; -- add a partitioned table, index and view with a different storage location CREATE TABLE part_tab2 (id INT, name STRING) PARTITIONED BY (ds string) LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab2'; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab2 PARTITION (ds='2009-04-09'); CREATE INDEX idx4 ON TABLE part_tab2(id) AS 'COMPACT' with DEFERRED REBUILD LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab2_idx4'; ALTER INDEX idx4 ON part_tab2 PARTITION (ds='2008-04-09') REBUILD; @@ -56,8 +57,8 @@ dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/dbcascade/part_tab3_idx5; CREATE TABLE part_tab3 (id INT, name STRING) PARTITIONED BY (ds string) LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3'; ALTER TABLE part_tab3 ADD PARTITION (ds='2007-04-09') LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3_p1'; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE part_tab3 PARTITION (ds='2009-04-09'); CREATE INDEX idx5 ON TABLE part_tab3(id) AS 'COMPACT' with DEFERRED REBUILD LOCATION 'file:${system:test.tmp.dir}/dbcascade/part_tab3_idx5'; ALTER INDEX idx5 ON part_tab3 PARTITION (ds='2008-04-09') REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q index a2322fc1e45a0..7d89ac9268f9b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q @@ -1,21 +1,23 @@ +set hive.fetch.task.conversion=more; + drop table date_1; create table date_1 (d date); insert overwrite table date_1 - select cast('2011-01-01' as date) from src limit 1; + select cast('2011-01-01' as date) from src tablesample (1 rows); select * from date_1 limit 1; select d, count(d) from date_1 group by d; insert overwrite table date_1 - select date '2011-01-01' from src limit 1; + select date '2011-01-01' from src tablesample (1 rows); select * from date_1 limit 1; select d, count(d) from date_1 group by d; insert overwrite table date_1 - select cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src limit 1; + select cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src tablesample (1 rows); select * from date_1 limit 1; select d, count(d) from date_1 group by d; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q index 0821e012ba26b..c5346c87dd25c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q @@ -8,7 +8,7 @@ create table date_2 ( FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_2; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_2; select fl_date, fl_num from date_2 order by fl_date asc, fl_num desc; select fl_date, fl_num from date_2 order by fl_date desc, fl_num asc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q index be251484459b6..383fb4e9904da 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q @@ -7,7 +7,7 @@ create table date_3 ( alter table date_3 add columns (c2 date); insert overwrite table date_3 - select 1, cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src limit 1; + select 1, cast(cast('2011-01-01 00:00:00' as timestamp) as date) from src tablesample (1 rows); select * from date_3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q index 4801a79b8795b..c840089f2efeb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + drop table date_4; create table date_4 (d date); @@ -5,7 +7,7 @@ alter table date_4 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe -- Test date literal syntax insert overwrite table date_4 - select date '2011-01-01' from src limit 1; + select date '2011-01-01' from src tablesample (1 rows); select d, date '2011-01-01' from date_4 limit 1; drop table date_4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q index bdcb6c1b6e447..86c7362e295f2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + -- Comparisons against same value select cast('2011-05-06' as date) > cast('2011-05-06' as date) from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q index a5844b76e9ef8..34bb8c8990a6b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q @@ -8,7 +8,7 @@ create table date_join1 ( FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE date_join1; +LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE date_join1; -- Note that there are 2 rows with date 2000-11-28, so we should expect 4 rows with that date in the join results select t1.fl_num, t1.fl_date, t2.fl_num, t2.fl_date diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q index ffc06d270d53a..24b48206802ae 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q @@ -22,7 +22,7 @@ with serdeproperties ( ) stored as textfile; -load data local inpath '../data/files/flights_tiny.txt.1' overwrite into table date_serde_regex; +load data local inpath '../../data/files/flights_tiny.txt.1' overwrite into table date_serde_regex; select * from date_serde_regex; select fl_date, count(*) from date_serde_regex group by fl_date; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q index 9696320a85fcd..c55b9f9147ada 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q @@ -17,7 +17,7 @@ create table date_udf_flight ( ARR_DELAY float, FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight; -- Test UDFs with date input select unix_timestamp(d), year(d), month(d), day(d), dayofmonth(d), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q new file mode 100644 index 0000000000000..6612fe8babdf8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q @@ -0,0 +1,12 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +create table T1(key string, val string) stored as textfile; + +set hive.txn.testing=true; +alter table T1 compact 'major'; + +alter table T1 compact 'minor'; + +drop table T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q new file mode 100644 index 0000000000000..599cad9afc611 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q @@ -0,0 +1,14 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +create table T1(key string, val string) partitioned by (ds string) stored as textfile; + +alter table T1 add partition (ds = 'today'); +alter table T1 add partition (ds = 'yesterday'); + +alter table T1 partition (ds = 'today') compact 'major'; + +alter table T1 partition (ds = 'yesterday') compact 'minor'; + +drop table T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q new file mode 100644 index 0000000000000..871d292a59ce3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q @@ -0,0 +1,15 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +create database D1; + +use D1; + +create table T1(key string, val string) stored as textfile; + +alter table T1 compact 'major'; + +alter table T1 compact 'minor'; + +drop table T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q new file mode 100644 index 0000000000000..3126bd6e5460e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_ddl1.q @@ -0,0 +1,59 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create database D1; + +alter database D1 set dbproperties('test'='yesthisis'); + +drop database D1; + +create table T1(key string, val string) stored as textfile; + +create table T2 like T1; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T3 as select * from T1; + +create table T4 (key char(10), val decimal(5,2), b int) + partitioned by (ds string) + clustered by (b) into 10 buckets + stored as orc; + +alter table T3 rename to newT3; + +alter table T2 set tblproperties ('test'='thisisatest'); + +alter table T2 set serde 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; +alter table T2 set serdeproperties ('test'='thisisatest'); + +alter table T2 clustered by (key) into 32 buckets; + +alter table T4 add partition (ds='today'); + +alter table T4 partition (ds='today') rename to partition(ds='yesterday'); + +alter table T4 drop partition (ds='yesterday'); + +alter table T4 add partition (ds='tomorrow'); + +create table T5 (a string, b int); +alter table T5 set fileformat orc; + +create table T7 (a string, b int); +alter table T7 set location 'file:///tmp'; + +alter table T2 touch; +alter table T4 touch partition (ds='tomorrow'); + +create view V1 as select key from T1; +alter view V1 set tblproperties ('test'='thisisatest'); +drop view V1; + + + +drop table T1; +drop table T2; +drop table newT3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q new file mode 100644 index 0000000000000..970069aca6766 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query1.q @@ -0,0 +1,17 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) stored as textfile; + +insert into table T2 select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q new file mode 100644 index 0000000000000..00942e5357892 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query2.q @@ -0,0 +1,17 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) stored as textfile; + +insert overwrite table T2 select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q new file mode 100644 index 0000000000000..75b642b5492cc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query3.q @@ -0,0 +1,21 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string, val string) partitioned by (pval string) stored as textfile; + +insert into table T2 partition (pval = '1') select * from T1; + +select * from T2; + +insert overwrite table T2 partition (pval = '1') select * from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q new file mode 100644 index 0000000000000..57eb4424ea5cc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query4.q @@ -0,0 +1,19 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +create table T1(key string, val string) stored as textfile; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +select * from T1; + +create table T2(key string) partitioned by (val string) stored as textfile; + +insert overwrite table T2 partition (val) select key, val from T1; + +select * from T2; + +drop table T1; +drop table T2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q new file mode 100644 index 0000000000000..d22b98fd7df2d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_query5.q @@ -0,0 +1,24 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; + +create database foo; + +use foo; + +create table T1(key string, val string) partitioned by (ds string) stored as textfile; + +alter table T1 add partition (ds='today'); + +create view V1 as select key from T1; + +show tables; + +describe T1; + +drop view V1; + +drop table T1; + +show databases; + +drop database foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q new file mode 100644 index 0000000000000..7c71fdd9d230f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q @@ -0,0 +1,11 @@ +set hive.support.concurrency=true; +set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager; +set hive.txn.testing=true; + +show locks; + +show locks extended; + +show locks default; + +show transactions; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q index 6c689e188a908..f52b1923eb067 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q @@ -1,18 +1,22 @@ -drop table decimal_1; +set hive.fetch.task.conversion=more; + +drop table if exists decimal_1; -create table decimal_1 (t decimal); +create table decimal_1 (t decimal(4,2), u decimal(5), v decimal); alter table decimal_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; +desc decimal_1; + insert overwrite table decimal_1 - select cast('17.29' as decimal) from src limit 1; -select cast(t as boolean) from decimal_1 limit 1; -select cast(t as tinyint) from decimal_1 limit 1; -select cast(t as smallint) from decimal_1 limit 1; -select cast(t as int) from decimal_1 limit 1; -select cast(t as bigint) from decimal_1 limit 1; -select cast(t as float) from decimal_1 limit 1; -select cast(t as double) from decimal_1 limit 1; -select cast(t as string) from decimal_1 limit 1; -select cast(t as timestamp) from decimal_1 limit 1; + select cast('17.29' as decimal(4,2)), 3.1415926BD, 3115926.54321BD from src tablesample (1 rows); +select cast(t as boolean) from decimal_1; +select cast(t as tinyint) from decimal_1; +select cast(t as smallint) from decimal_1; +select cast(t as int) from decimal_1; +select cast(t as bigint) from decimal_1; +select cast(t as float) from decimal_1; +select cast(t as double) from decimal_1; +select cast(t as string) from decimal_1; +select cast(t as timestamp) from decimal_1; drop table decimal_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q index 4890618a0dc32..2c4d919079423 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q @@ -1,40 +1,42 @@ +set hive.fetch.task.conversion=more; + drop table decimal_2; -create table decimal_2 (t decimal); +create table decimal_2 (t decimal(18,9)); alter table decimal_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; insert overwrite table decimal_2 - select cast('17.29' as decimal) from src limit 1; + select cast('17.29' as decimal(4,2)) from src tablesample (1 rows); -select cast(t as boolean) from decimal_2 limit 1; -select cast(t as tinyint) from decimal_2 limit 1; -select cast(t as smallint) from decimal_2 limit 1; -select cast(t as int) from decimal_2 limit 1; -select cast(t as bigint) from decimal_2 limit 1; -select cast(t as float) from decimal_2 limit 1; -select cast(t as double) from decimal_2 limit 1; -select cast(t as string) from decimal_2 limit 1; +select cast(t as boolean) from decimal_2; +select cast(t as tinyint) from decimal_2; +select cast(t as smallint) from decimal_2; +select cast(t as int) from decimal_2; +select cast(t as bigint) from decimal_2; +select cast(t as float) from decimal_2; +select cast(t as double) from decimal_2; +select cast(t as string) from decimal_2; insert overwrite table decimal_2 - select cast('3404045.5044003' as decimal) from src limit 1; + select cast('3404045.5044003' as decimal(18,9)) from src tablesample (1 rows); -select cast(t as boolean) from decimal_2 limit 1; -select cast(t as tinyint) from decimal_2 limit 1; -select cast(t as smallint) from decimal_2 limit 1; -select cast(t as int) from decimal_2 limit 1; -select cast(t as bigint) from decimal_2 limit 1; -select cast(t as float) from decimal_2 limit 1; -select cast(t as double) from decimal_2 limit 1; -select cast(t as string) from decimal_2 limit 1; +select cast(t as boolean) from decimal_2; +select cast(t as tinyint) from decimal_2; +select cast(t as smallint) from decimal_2; +select cast(t as int) from decimal_2; +select cast(t as bigint) from decimal_2; +select cast(t as float) from decimal_2; +select cast(t as double) from decimal_2; +select cast(t as string) from decimal_2; -select cast(3.14 as decimal) from decimal_2 limit 1; -select cast(cast(3.14 as float) as decimal) from decimal_2 limit 1; -select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal) from decimal_2 limit 1; -select cast(true as decimal) from decimal_2 limit 1; -select cast(3Y as decimal) from decimal_2 limit 1; -select cast(3S as decimal) from decimal_2 limit 1; -select cast(cast(3 as int) as decimal) from decimal_2 limit 1; -select cast(3L as decimal) from decimal_2 limit 1; -select cast(0.99999999999999999999 as decimal) from decimal_2 limit 1; -select cast('0.99999999999999999999' as decimal) from decimal_2 limit 1; +select cast(3.14 as decimal(4,2)) from decimal_2; +select cast(cast(3.14 as float) as decimal(4,2)) from decimal_2; +select cast(cast('2012-12-19 11:12:19.1234567' as timestamp) as decimal(30,8)) from decimal_2; +select cast(true as decimal) from decimal_2; +select cast(3Y as decimal) from decimal_2; +select cast(3S as decimal) from decimal_2; +select cast(cast(3 as int) as decimal) from decimal_2; +select cast(3L as decimal) from decimal_2; +select cast(0.99999999999999999999 as decimal(20,19)) from decimal_2; +select cast('0.99999999999999999999' as decimal(20,20)) from decimal_2; drop table decimal_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q index 28211e3f14cc1..e4fba06fea1a9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS DECIMAL_3; -CREATE TABLE DECIMAL_3(key decimal, value int) +CREATE TABLE DECIMAL_3(key decimal(38,18), value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_3; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_3; SELECT * FROM DECIMAL_3 ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q index e8a89c131cd85..699ba3cb4f558 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q @@ -1,15 +1,15 @@ DROP TABLE IF EXISTS DECIMAL_4_1; DROP TABLE IF EXISTS DECIMAL_4_2; -CREATE TABLE DECIMAL_4_1(key decimal, value int) +CREATE TABLE DECIMAL_4_1(key decimal(35,25), value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -CREATE TABLE DECIMAL_4_2(key decimal, value decimal) +CREATE TABLE DECIMAL_4_2(key decimal(35,25), value decimal(35,25)) STORED AS ORC; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_4_1; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_4_1; INSERT OVERWRITE TABLE DECIMAL_4_2 SELECT key, key * 3 FROM DECIMAL_4_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q new file mode 100644 index 0000000000000..70e5db0f70182 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_5.q @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS DECIMAL_5; + +CREATE TABLE DECIMAL_5(key decimal(10,5), value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_5; + +SELECT key FROM DECIMAL_5 ORDER BY key; + +SELECT DISTINCT key FROM DECIMAL_5 ORDER BY key; + +SELECT cast(key as decimal) FROM DECIMAL_5; + +SELECT cast(key as decimal(6,3)) FROM DECIMAL_5; + +DROP TABLE DECIMAL_5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q new file mode 100644 index 0000000000000..b58e224256122 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_6.q @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS DECIMAL_6_1; +DROP TABLE IF EXISTS DECIMAL_6_2; +DROP TABLE IF EXISTS DECIMAL_6_3; + +CREATE TABLE DECIMAL_6_1(key decimal(10,5), value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +CREATE TABLE DECIMAL_6_2(key decimal(17,4), value int) +ROW FORMAT DELIMITED + FIELDS TERMINATED BY ' ' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/kv9.txt' INTO TABLE DECIMAL_6_1; +LOAD DATA LOCAL INPATH '../../data/files/kv9.txt' INTO TABLE DECIMAL_6_2; + +SELECT T.key from ( + SELECT key, value from DECIMAL_6_1 + UNION ALL + SELECT key, value from DECIMAL_6_2 +) T order by T.key; + +CREATE TABLE DECIMAL_6_3 AS SELECT key + 5.5 AS k, value * 11 AS v from DECIMAL_6_1 ORDER BY v; + +desc DECIMAL_6_3; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q index 589fc6597dfa4..86c14d9351ac1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q @@ -1,6 +1,6 @@ -- HIVE-5292 Join on decimal columns fails -create table src_dec (key decimal, value string); -load data local inpath '../data/files/kv1.txt' into table src_dec; +create table src_dec (key decimal(3,0), value string); +load data local inpath '../../data/files/kv1.txt' into table src_dec; select * from src_dec a join src_dec b on a.key=b.key+450; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q index 403c2be3fbc10..739352f9ef1e5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS DECIMAL_PRECISION; -CREATE TABLE DECIMAL_PRECISION(dec decimal) +CREATE TABLE DECIMAL_PRECISION(dec decimal(20,10)) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv8.txt' INTO TABLE DECIMAL_PRECISION; +LOAD DATA LOCAL INPATH '../../data/files/kv8.txt' INTO TABLE DECIMAL_PRECISION; SELECT * FROM DECIMAL_PRECISION ORDER BY dec; @@ -15,13 +15,14 @@ SELECT dec, dec / 9 FROM DECIMAL_PRECISION ORDER BY dec; SELECT dec, dec / 27 FROM DECIMAL_PRECISION ORDER BY dec; SELECT dec, dec * dec FROM DECIMAL_PRECISION ORDER BY dec; +EXPLAIN SELECT avg(dec), sum(dec) FROM DECIMAL_PRECISION; SELECT avg(dec), sum(dec) FROM DECIMAL_PRECISION; -SELECT dec * cast('123456789012345678901234567890.123456789' as decimal) FROM DECIMAL_PRECISION LIMIT 1; -SELECT * from DECIMAL_PRECISION WHERE dec > cast('123456789012345678901234567890.123456789' as decimal) LIMIT 1; -SELECT dec * 123456789012345678901234567890.123456789 FROM DECIMAL_PRECISION LIMIT 1; +SELECT dec * cast('12345678901234567890.12345678' as decimal(38,18)) FROM DECIMAL_PRECISION LIMIT 1; +SELECT * from DECIMAL_PRECISION WHERE dec > cast('1234567890123456789012345678.12345678' as decimal(38,18)) LIMIT 1; +SELECT dec * 12345678901234567890.12345678 FROM DECIMAL_PRECISION LIMIT 1; -SELECT MIN(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION; -SELECT COUNT(cast('123456789012345678901234567890.123456789' as decimal)) FROM DECIMAL_PRECISION; +SELECT MIN(cast('12345678901234567890.12345678' as decimal(38,18))) FROM DECIMAL_PRECISION; +SELECT COUNT(cast('12345678901234567890.12345678' as decimal(38,18))) FROM DECIMAL_PRECISION; DROP TABLE DECIMAL_PRECISION; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q index 3556807705ae5..cf3a86cd4d78f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q @@ -8,7 +8,7 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_TEXT; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_TEXT; SELECT * FROM DECIMAL_TEXT ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q index b5ff088d1613a..0c9f1b86a9e97 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DROP TABLE IF EXISTS DECIMAL_UDF; -CREATE TABLE DECIMAL_UDF (key decimal, value int) +CREATE TABLE DECIMAL_UDF (key decimal(20,10), value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv7.txt' INTO TABLE DECIMAL_UDF; +LOAD DATA LOCAL INPATH '../../data/files/kv7.txt' INTO TABLE DECIMAL_UDF; -- addition EXPLAIN SELECT key + key FROM DECIMAL_UDF; @@ -70,8 +72,8 @@ EXPLAIN SELECT abs(key) FROM DECIMAL_UDF; SELECT abs(key) FROM DECIMAL_UDF; -- avg -EXPLAIN SELECT value, sum(key) / count(key), avg(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; -SELECT value, sum(key) / count(key), avg(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; +EXPLAIN SELECT value, sum(key) / count(key), avg(key), sum(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; +SELECT value, sum(key) / count(key), avg(key), sum(key) FROM DECIMAL_UDF GROUP BY value ORDER BY value; -- negative EXPLAIN SELECT -key FROM DECIMAL_UDF; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q index 112ac57c3be5b..14d508c07dd3d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q @@ -3,7 +3,7 @@ row format delimited fields terminated by '\t' lines terminated by '\n' stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/in7.txt' INTO TABLE impressions; +LOAD DATA LOCAL INPATH '../../data/files/in7.txt' INTO TABLE impressions; select * from impressions; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q new file mode 100644 index 0000000000000..89e49311fa48d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_tbl_part_cols.q @@ -0,0 +1,7 @@ +create table t1 (a int, b string) partitioned by (c int, d string); +describe t1; + +set hive.display.partition.cols.separately=false; +describe t1; + +set hive.display.partition.cols.separately=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q index 6ea4156b3493e..81a5b3a6e6c82 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q @@ -1,9 +1,9 @@ set hive.fileformat.check = false; create table kv_fileformat_check_txt (key string, value string) stored as textfile; -load data local inpath '../data/files/kv1.seq' overwrite into table kv_fileformat_check_txt; +load data local inpath '../../data/files/kv1.seq' overwrite into table kv_fileformat_check_txt; create table kv_fileformat_check_seq (key string, value string) stored as sequencefile; -load data local inpath '../data/files/kv1.txt' overwrite into table kv_fileformat_check_seq; +load data local inpath '../../data/files/kv1.txt' overwrite into table kv_fileformat_check_seq; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q index 2a1e7276cacfe..991b930d54ca4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q @@ -1,7 +1,9 @@ +set hive.fetch.task.conversion=more; + SET hive.metastore.disallow.incompatible.col.type.changes=false; SELECT * FROM src LIMIT 1; CREATE TABLE test_table123 (a INT, b MAP) PARTITIONED BY (ds STRING) STORED AS SEQUENCEFILE; -INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src LIMIT 1; +INSERT OVERWRITE TABLE test_table123 PARTITION(ds="foo1") SELECT 1, MAP("a1", "b1") FROM src tablesample (1 rows); SELECT * from test_table123 WHERE ds="foo1"; -- This should now work as hive.metastore.disallow.incompatible.col.type.changes is false ALTER TABLE test_table123 REPLACE COLUMNS (a INT, b STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q new file mode 100644 index 0000000000000..725183380b9ad --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/distinct_stats.q @@ -0,0 +1,20 @@ +set hive.stats.autogather=true; + +set hive.compute.query.using.stats=true; +create table t1 (a string, b string); + +insert into table t1 select * from src; + +analyze table t1 compute statistics for columns a,b; + +explain +select count(distinct b) from t1 group by a; + +explain +select distinct(b) from t1; + +explain +select a, count(*) from t1 group by a; + +drop table t1; +set hive.compute.query.using.stats = false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q index 798aa6d51a185..54e6a35b5adfe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q @@ -6,7 +6,7 @@ alter table ptestfilter add partition (c=1, d=2); alter table ptestFilter add partition (c=2, d=1); alter table ptestfilter add partition (c=2, d=2); alter table ptestfilter add partition (c=3, d=1); -alter table ptestfilter add partition (c=3, d=2); +alter table ptestfilter add partition (c=30, d=2); show partitions ptestfilter; alter table ptestfilter drop partition (c=1, d=1); @@ -15,6 +15,9 @@ show partitions ptestfilter; alter table ptestfilter drop partition (c=2); show partitions ptestfilter; +alter table ptestfilter drop partition (c<4); +show partitions ptestfilter; + drop table ptestfilter; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q new file mode 100644 index 0000000000000..797a27c23b01e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_with_concurrency.q @@ -0,0 +1,8 @@ +set hive.lock.numretries=1; +set hive.lock.sleep.between.retries=1; +set hive.support.concurrency=true; +set hive.lock.manager=org.apache.hadoop.hive.ql.lockmgr.EmbeddedLockManager; + +drop table if exists drop_with_concurrency_1; +create table drop_with_concurrency_1 (c1 int); +drop table drop_with_concurrency_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q index 397a2200a8972..699e58effcac0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q @@ -1,19 +1,19 @@ -create table dynamic_part_table(intcol int) partitioned by (partcol1 int, partcol2 int); +create table dynamic_part_table(intcol string) partitioned by (partcol1 string, partcol2 string); set hive.exec.dynamic.partition.mode=nonstrict; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, 1, 1 from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', '1', '1' from src where key=150; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, NULL, 1 from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', NULL, '1' from src where key=150; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, 1, NULL from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', '1', NULL from src where key=150; -insert into table dynamic_part_table partition(partcol1, partcol2) select 1, NULL, NULL from src where key=150; +insert into table dynamic_part_table partition(partcol1, partcol2) select '1', NULL, NULL from src where key=150; -explain extended select intcol from dynamic_part_table where partcol1=1 and partcol2=1; +explain extended select intcol from dynamic_part_table where partcol1='1' and partcol2='1'; set hive.exec.dynamic.partition.mode=strict; -explain extended select intcol from dynamic_part_table where partcol1=1 and partcol2=1; +explain extended select intcol from dynamic_part_table where partcol1='1' and partcol2='1'; -explain extended select intcol from dynamic_part_table where (partcol1=1 and partcol2=1)or (partcol1=1 and partcol2='__HIVE_DEFAULT_PARTITION__'); +explain extended select intcol from dynamic_part_table where (partcol1='1' and partcol2='1')or (partcol1='1' and partcol2='__HIVE_DEFAULT_PARTITION__'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q new file mode 100644 index 0000000000000..5f1a5ce809e0c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q @@ -0,0 +1,161 @@ +set hive.optimize.sort.dynamic.partition=true; +set hive.exec.dynamic.partition=true; +set hive.exec.max.dynamic.partitions=1000; +set hive.exec.max.dynamic.partitions.pernode=1000; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.vectorized.execution.enabled=true; +set hive.enforce.bucketing=false; +set hive.enforce.sorting=false; + +create table over1k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal(4,2), + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over1k' into table over1k; + +create table over1k_orc like over1k; +alter table over1k_orc set fileformat orc; +insert overwrite table over1k_orc select * from over1k; + +create table over1k_part_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint) stored as orc; + +create table over1k_part_limit_orc like over1k_part_orc; +alter table over1k_part_limit_orc set fileformat orc; + +create table over1k_part_buck_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets stored as orc; + +create table over1k_part_buck_sort_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets stored as orc; + +-- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization +explain insert overwrite table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +explain insert overwrite table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +explain insert overwrite table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +explain insert overwrite table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +insert overwrite table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +insert overwrite table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +insert overwrite table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +insert overwrite table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +-- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization +explain insert into table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +explain insert into table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +explain insert into table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +explain insert into table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +insert into table over1k_part_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by si; +insert into table over1k_part_limit_orc partition(ds="foo", t) select si,i,b,f,t from over1k_orc where t is null or t=27 limit 10; +insert into table over1k_part_buck_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +insert into table over1k_part_buck_sort_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +desc formatted over1k_part_orc partition(ds="foo",t=27); +desc formatted over1k_part_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_limit_orc partition(ds="foo",t=27); +desc formatted over1k_part_limit_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck_orc partition(t=27); +desc formatted over1k_part_buck_orc partition(t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck_sort_orc partition(t=27); +desc formatted over1k_part_buck_sort_orc partition(t="__HIVE_DEFAULT_PARTITION__"); + +select count(*) from over1k_part_orc; +select count(*) from over1k_part_limit_orc; +select count(*) from over1k_part_buck_orc; +select count(*) from over1k_part_buck_sort_orc; + +-- tests for HIVE-6883 +create table over1k_part2_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint); + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; + +desc formatted over1k_part2_orc partition(ds="foo",t=27); +desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2_orc; +select count(*) from over1k_part2_orc; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part2_orc partition(ds="foo",t) select si,i,b,f,t from over1k_orc where t is null or t=27 order by i; + +desc formatted over1k_part2_orc partition(ds="foo",t=27); +desc formatted over1k_part2_orc partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2_orc; +select count(*) from over1k_part2_orc; + +-- hadoop-1 does not honor number of reducers in local mode. There is always only 1 reducer irrespective of the number of buckets. +-- Hence all records go to one bucket and all other buckets will be empty. Similar to HIVE-6867. However, hadoop-2 honors number +-- of reducers and records are spread across all reducers. To avoid this inconsistency we will make number of buckets to 1 for this test. +create table over1k_part_buck_sort2_orc( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 1 buckets; + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +desc formatted over1k_part_buck_sort2_orc partition(t=27); +desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2_orc; +select count(*) from over1k_part_buck_sort2_orc; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part_buck_sort2_orc partition(t) select si,i,b,f,t from over1k_orc where t is null or t=27; + +desc formatted over1k_part_buck_sort2_orc partition(t=27); +desc formatted over1k_part_buck_sort2_orc partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2_orc; +select count(*) from over1k_part_buck_sort2_orc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q new file mode 100644 index 0000000000000..52b5d1e0c1d8a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q @@ -0,0 +1,155 @@ +set hive.optimize.sort.dynamic.partition=true; +set hive.exec.dynamic.partition=true; +set hive.exec.max.dynamic.partitions=1000; +set hive.exec.max.dynamic.partitions.pernode=1000; +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.enforce.bucketing=false; +set hive.enforce.sorting=false; + +create table over1k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal(4,2), + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over1k' into table over1k; + +create table over1k_part( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint); + +create table over1k_part_limit like over1k_part; + +create table over1k_part_buck( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) into 4 buckets; + +create table over1k_part_buck_sort( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 4 buckets; + +-- map-only jobs converted to map-reduce job by hive.optimize.sort.dynamic.partition optimization +explain insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert overwrite table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +explain insert overwrite table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +insert overwrite table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +insert overwrite table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +insert overwrite table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +insert overwrite table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +set hive.enforce.bucketing=true; +set hive.enforce.sorting=true; + +-- map-reduce jobs modified by hive.optimize.sort.dynamic.partition optimization +explain insert into table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert into table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +explain insert into table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +explain insert into table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +insert into table over1k_part partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27; +insert into table over1k_part_limit partition(ds="foo", t) select si,i,b,f,t from over1k where t is null or t=27 limit 10; +insert into table over1k_part_buck partition(t) select si,i,b,f,t from over1k where t is null or t=27; +insert into table over1k_part_buck_sort partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +desc formatted over1k_part partition(ds="foo",t=27); +desc formatted over1k_part partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_limit partition(ds="foo",t=27); +desc formatted over1k_part_limit partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck partition(t=27); +desc formatted over1k_part_buck partition(t="__HIVE_DEFAULT_PARTITION__"); +desc formatted over1k_part_buck_sort partition(t=27); +desc formatted over1k_part_buck_sort partition(t="__HIVE_DEFAULT_PARTITION__"); + +select count(*) from over1k_part; +select count(*) from over1k_part_limit; +select count(*) from over1k_part_buck; +select count(*) from over1k_part_buck_sort; + +-- tests for HIVE-6883 +create table over1k_part2( + si smallint, + i int, + b bigint, + f float) + partitioned by (ds string, t tinyint); + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; + +desc formatted over1k_part2 partition(ds="foo",t=27); +desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2; +select count(*) from over1k_part2; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part2 partition(ds="foo",t) select si,i,b,f,t from over1k where t is null or t=27 order by i; + +desc formatted over1k_part2 partition(ds="foo",t=27); +desc formatted over1k_part2 partition(ds="foo",t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part2; +select count(*) from over1k_part2; + +-- hadoop-1 does not honor number of reducers in local mode. There is always only 1 reducer irrespective of the number of buckets. +-- Hence all records go to one bucket and all other buckets will be empty. Similar to HIVE-6867. However, hadoop-2 honors number +-- of reducers and records are spread across all reducers. To avoid this inconsistency we will make number of buckets to 1 for this test. +create table over1k_part_buck_sort2( + si smallint, + i int, + b bigint, + f float) + partitioned by (t tinyint) + clustered by (si) + sorted by (f) into 1 buckets; + +set hive.optimize.sort.dynamic.partition=false; +explain insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; +set hive.optimize.sort.dynamic.partition=true; +explain insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +set hive.optimize.sort.dynamic.partition=false; +insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +desc formatted over1k_part_buck_sort2 partition(t=27); +desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2; +select count(*) from over1k_part_buck_sort2; + +set hive.optimize.sort.dynamic.partition=true; +insert overwrite table over1k_part_buck_sort2 partition(t) select si,i,b,f,t from over1k where t is null or t=27; + +desc formatted over1k_part_buck_sort2 partition(t=27); +desc formatted over1k_part_buck_sort2 partition(t="__HIVE_DEFAULT_PARTITION__"); + +select * from over1k_part_buck_sort2; +select count(*) from over1k_part_buck_sort2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q index a7f4cf79c980f..d29a7a8f105d6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q @@ -8,7 +8,7 @@ DROP TABLE escape1; DROP TABLE escape_raw; CREATE TABLE escape_raw (s STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/escapetest.txt' INTO TABLE escape_raw; +LOAD DATA LOCAL INPATH '../../data/files/escapetest.txt' INTO TABLE escape_raw; SELECT count(*) from escape_raw; SELECT * from escape_raw; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q index 473cbf8c94e4e..24601343b1474 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q @@ -10,7 +10,7 @@ DROP TABLE IF EXISTS escape2; DROP TABLE IF EXISTS escape_raw; CREATE TABLE escape_raw (s STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/escapetest.txt' INTO TABLE escape_raw; +LOAD DATA LOCAL INPATH '../../data/files/escapetest.txt' INTO TABLE escape_raw; SELECT count(*) from escape_raw; SELECT * from escape_raw; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q index 6e8bf8ebc1fa2..4be6e3f6d876b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q @@ -3,7 +3,7 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q index 27b335a3d7844..f346ddeec4963 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q @@ -3,7 +3,7 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q index 7b9060d420a1e..7c076cebe8ad4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q @@ -3,8 +3,9 @@ CREATE TABLE exchange_part_test2 (f1 string) PARTITIONED BY (ds STRING, hr STRIN SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='1'); -ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2013-04-05', hr='2'); +ALTER TABLE exchange_part_test1 ADD PARTITION (ds='2014-01-03', hr='1'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='1'); +ALTER TABLE exchange_part_test2 ADD PARTITION (ds='2013-04-05', hr='2'); SHOW PARTITIONS exchange_part_test1; SHOW PARTITIONS exchange_part_test2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q index 7fa96b629a0fa..8288bbfd86acc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q @@ -1,3 +1,6 @@ +set hive.security.authorization.manager=org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactoryForTest; +set hive.security.authenticator.manager=org.apache.hadoop.hive.ql.security.SessionStateConfigUserAuthenticator; + set hive.test.mode=true; set hive.test.mode.prefix=; set hive.test.mode.nosamplelist=exim_department,exim_employee; @@ -5,8 +8,8 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -16,7 +19,7 @@ use importer; import from 'ql/test/data/exports/exim_department'; describe extended exim_department; show table extended like exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q index 9920e778d18a6..1e2eed803a01d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -17,7 +17,7 @@ use importer; import from 'ql/test/data/exports/exim_department'; describe extended exim_department; show table extended like exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q index 4017c83aa3dd4..474a5a4a5bb5f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q @@ -7,8 +7,8 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -18,7 +18,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q index 21138f0263418..dbd2c6bf5c4c5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q @@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -20,7 +20,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q index 5f6bdee83cc90..47d949aa36d9b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -21,6 +21,6 @@ import from 'ql/test/data/exports/exim_department'; describe extended exim_department; select * from exim_department; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q index 69c6faa30a07c..b2567fb270326 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -26,7 +26,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q index cdc02fa25c169..82df69874b47a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q @@ -19,8 +19,8 @@ alter table exim_employee set fileformat outputformat "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"; alter table exim_employee add partition (emp_country='in', emp_state='ka'); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -32,7 +32,7 @@ describe extended exim_employee; describe extended exim_employee partition (emp_country='in', emp_state='tn'); describe extended exim_employee partition (emp_country='in', emp_state='ka'); show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q index 50a59463b1870..a2c977356c84c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee partition (emp_state="ka") to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -26,7 +26,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q index 5136090929fc5..3a612964222eb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee partition (emp_country="in",emp_state="ka") to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -26,7 +26,7 @@ use importer; import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; show table extended like exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q index 5b9d4ddc03c59..8c774d5a8ba84 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -28,12 +28,12 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "iso code", emp_state string comment "free-form text") stored as textfile tblproperties("maker"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="al"); import from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q index 173f1569c501a..8a1d9454763c7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee,exim_imported_dept create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -17,12 +17,12 @@ create table exim_department ( dep_id int comment "department id") partitioned by (emp_org string) stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department partition (emp_org="hr"); +load data local inpath "../../data/files/test.dat" into table exim_department partition (emp_org="hr"); import table exim_imported_dept from 'ql/test/data/exports/exim_department'; describe extended exim_imported_dept; select * from exim_imported_dept; drop table exim_imported_dept; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q index 178b76674d862..53fc2936bb2f3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -27,14 +27,14 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); import table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q index 413f2aa4762f5..54859eed196cd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q @@ -2,18 +2,18 @@ set hive.test.mode=true; set hive.test.mode.prefix=; set hive.test.mode.nosamplelist=exim_department,exim_employee; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; create external table exim_department ( dep_id int comment "department id") stored as textfile location 'ql/test/data/tablestore/exim_department' tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; create database importer; use importer; @@ -22,6 +22,6 @@ import from 'ql/test/data/exports/exim_department'; describe extended exim_department; select * from exim_department; drop table exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q index f3b2896a322a5..4fc39dcff0620 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q @@ -5,9 +5,9 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -17,7 +17,7 @@ use importer; import external table exim_department from 'ql/test/data/exports/exim_department'; describe extended exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q index 37d063432e16a..e4d50ffe5b8f0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q @@ -5,24 +5,24 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/ql/test/data/exports/exim_department/temp; +dfs -rmr ${system:test.tmp.dir}/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department/temp; +dfs -rmr ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department; import external table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore/exim_department'; describe extended exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr ${system:test.tmp.dir}/ql/test/data/exports/exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr ${system:test.tmp.dir}/ql/test/data/tablestore/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q index fb5058b840bae..909d23794be9d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q @@ -5,24 +5,24 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; import table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore/exim_department'; describe extended exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q index 031b6bda6cece..dbb5fd93435a0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q @@ -5,17 +5,17 @@ set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int comment "department id") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_department/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; create table exim_department ( dep_id int comment "department id") stored as textfile @@ -24,9 +24,9 @@ create table exim_department ( dep_id int comment "department id") import table exim_department from 'ql/test/data/exports/exim_department' location 'ql/test/data/tablestore/exim_department'; describe extended exim_department; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; select * from exim_department; -dfs -rmr ../build/ql/test/data/tablestore/exim_department; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_department; select * from exim_department; drop table exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q index ff088c70d7ffb..989dd6cf56547 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; create external table exim_employee ( emp_id int comment "employee id") comment "employee table" @@ -32,17 +32,17 @@ create external table exim_employee ( emp_id int comment "employee id") stored as textfile location 'ql/test/data/tablestore/exim_employee' tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); import external table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee'; describe extended exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q index 6f4ee7a01c256..7eec358850e29 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q @@ -7,26 +7,26 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore2/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore2/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore2/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore2/exim_employee; create external table exim_employee ( emp_id int comment "employee id") comment "employee table" @@ -39,11 +39,11 @@ import table exim_employee partition (emp_country="us", emp_state="tn") location 'ql/test/data/tablestore/exim_employee'; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; -dfs -rmr ../build/ql/test/data/tablestore2/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore2/exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q index 56ec152948aed..20cd7e0513c7b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; create table exim_employee ( emp_id int comment "employee id") comment "employee table" @@ -39,9 +39,9 @@ alter table exim_employee add partition (emp_country="us", emp_state="ap") show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); show table extended like exim_employee partition (emp_country="us", emp_state="ap"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q index 7aa1297dc7d02..a300b1dbf1285 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q @@ -7,16 +7,16 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -29,7 +29,7 @@ describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); select * from exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q index cb9f8efc087da..a821c75d70cca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q @@ -7,20 +7,20 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test2.dat" +load data local inpath "../../data/files/test2.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; import external table exim_employee from 'ql/test/data/exports/exim_employee' @@ -29,9 +29,9 @@ describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="in", emp_state="tn"); show table extended like exim_employee partition (emp_country="in", emp_state="ka"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q index bdbd19df70a21..be1216453bc18 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; import external table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee' @@ -32,9 +32,9 @@ import external table exim_employee partition (emp_country="us", emp_state="tn") describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q index eb44961a9b7ca..000904aa6634f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q @@ -7,24 +7,24 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="ka"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="tn"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="us", emp_state="ka"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; create database importer; use importer; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/tablestore/exim_employee/temp; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/tablestore/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; import table exim_employee partition (emp_country="us", emp_state="tn") from 'ql/test/data/exports/exim_employee' @@ -32,9 +32,9 @@ import table exim_employee partition (emp_country="us", emp_state="tn") describe extended exim_employee; show table extended like exim_employee; show table extended like exim_employee partition (emp_country="us", emp_state="tn"); -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; select * from exim_employee; -dfs -rmr ../build/ql/test/data/tablestore/exim_employee; +dfs -rmr target/tmp/ql/test/data/tablestore/exim_employee; select * from exim_employee; drop table exim_employee; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q index 822ed70a38c0c..293a011cb20f6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q @@ -2,13 +2,13 @@ set hive.test.mode=true; set hive.test.mode.prefix=; create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../data/files/test.dat" into table exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; set hive.security.authorization.enabled=true; grant Select on table exim_department to user hive_test_user; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; set hive.security.authorization.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q index 440d08d2dc7b5..03714ab17dcbf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q @@ -3,9 +3,9 @@ set hive.test.mode.prefix=; set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/temp; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -22,5 +22,5 @@ set hive.security.authorization.enabled=false; select * from exim_department; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q index 30fc343dd8f9c..cb6af0efbca62 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q @@ -7,10 +7,10 @@ create table exim_employee ( emp_id int comment "employee id") partitioned by (emp_country string comment "two char iso code", emp_state string comment "free text") stored as textfile tblproperties("creator"="krishna"); -load data local inpath "../data/files/test.dat" +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in", emp_state="tn"); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_employee/temp; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_employee/temp; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; export table exim_employee to 'ql/test/data/exports/exim_employee'; drop table exim_employee; @@ -29,6 +29,6 @@ import from 'ql/test/data/exports/exim_employee'; set hive.security.authorization.enabled=false; select * from exim_employee; -dfs -rmr ../build/ql/test/data/exports/exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; drop table exim_employee; drop database importer; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q index 2dc5af6ce4774..8934c47372384 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q @@ -3,9 +3,9 @@ set hive.test.mode.prefix=; set hive.test.mode.nosamplelist=exim_department,exim_employee; create table exim_department ( dep_id int) stored as textfile; -load data local inpath "../data/files/test.dat" into table exim_department; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/exim_department/test; -dfs -rmr ../build/ql/test/data/exports/exim_department; +load data local inpath "../../data/files/test.dat" into table exim_department; +dfs ${system:test.dfs.mkdir} target/tmp/ql/test/data/exports/exim_department/test; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; export table exim_department to 'ql/test/data/exports/exim_department'; drop table exim_department; @@ -20,5 +20,5 @@ set hive.security.authorization.enabled=false; select * from exim_department; drop table exim_department; drop database importer; -dfs -rmr ../build/ql/test/data/exports/exim_department; +dfs -rmr target/tmp/ql/test/data/exports/exim_department; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q new file mode 100644 index 0000000000000..f58c9f948d62b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_hidden_files.q @@ -0,0 +1,22 @@ +set hive.test.mode=true; +set hive.test.mode.prefix=; +set hive.test.mode.nosamplelist=exim_department,exim_employee; + +create table exim_employee ( emp_id int) partitioned by (emp_country string); +load data local inpath "../../data/files/test.dat" into table exim_employee partition (emp_country="in"); + +dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/exim_employee/emp_country=in/_logs; +dfs -touchz ${system:test.warehouse.dir}/exim_employee/emp_country=in/_logs/job.xml; +export table exim_employee to 'ql/test/data/exports/exim_employee'; +drop table exim_employee; + +create database importer; +use importer; + +import from 'ql/test/data/exports/exim_employee'; +describe formatted exim_employee; +select * from exim_employee; +dfs -rmr target/tmp/ql/test/data/exports/exim_employee; +drop table exim_employee; +drop database importer; +use default; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q new file mode 100644 index 0000000000000..ca2da354bd367 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_rearrange.q @@ -0,0 +1,98 @@ +-- query from auto_sortmerge_join_9.q + +CREATE TABLE tbl1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; +CREATE TABLE tbl2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; + +set hive.auto.convert.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.sortmerge.join=true; +set hive.auto.convert.sortmerge.join.to.mapjoin=false; + +set hive.explain.dependency.append.tasktype=true; + +-- default behavior + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +order by src1.key, src1.cnt1, src2.cnt1; + +set hive.stageid.rearrange=IDONLY; + +-- changes id only + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +order by src1.key, src1.cnt1, src2.cnt1; + +set hive.stageid.rearrange=TRAVERSE; + +-- assign ids in traverse order + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +order by src1.key, src1.cnt1, src2.cnt1; + +set hive.stageid.rearrange=EXECUTION; + +-- assign ids in execution order + +explain +select src1.key, src1.cnt1, src2.cnt1 from +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq1 group by key +) src1 +join +( + select key, count(*) as cnt1 from + ( + select a.key as key, a.value as val1, b.value as val2 from tbl1 a join tbl2 b on a.key = b.key + ) subq2 group by key +) src2 +on src1.key = src2.key +order by src1.key, src1.cnt1, src2.cnt1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q new file mode 100644 index 0000000000000..ad070464f9c4e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/external_table_with_space_in_location_path.q @@ -0,0 +1,23 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test/; + +dfs -copyFromLocal ../../data/files/ext_test_space hdfs:///tmp/test/ext_test_space; + +CREATE EXTERNAL TABLE spacetest (id int, message string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/ext_test_space/folder+with space'; + +SELECT * FROM spacetest; + +SELECT count(*) FROM spacetest; + +DROP TABLE spacetest; + +CREATE EXTERNAL TABLE spacetestpartition (id int, message string) PARTITIONED BY (day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; + +ALTER TABLE spacetestpartition ADD PARTITION (day=10) LOCATION 'hdfs:///tmp/test/ext_test_space/folder+with space'; + +SELECT * FROM spacetestpartition; + +SELECT count(*) FROM spacetestpartition; + +DROP TABLE spacetestpartition; + +dfs -rmr hdfs:///tmp/test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q new file mode 100644 index 0000000000000..8b65c7896d590 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/file_with_header_footer.q @@ -0,0 +1,39 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test/; + +dfs -copyFromLocal ../../data/files/header_footer_table_1 hdfs:///tmp/test/header_footer_table_1; + +dfs -copyFromLocal ../../data/files/header_footer_table_2 hdfs:///tmp/test/header_footer_table_2; + +dfs -copyFromLocal ../../data/files/header_footer_table_3 hdfs:///tmp/test/header_footer_table_3; + +CREATE EXTERNAL TABLE header_footer_table_1 (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/header_footer_table_1' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2"); + +SELECT * FROM header_footer_table_1; + +SELECT * FROM header_footer_table_1 WHERE id < 50; + +CREATE EXTERNAL TABLE header_footer_table_2 (name string, message string, id int) PARTITIONED BY (year int, month int, day int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2"); + +ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=1) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/01'; + +ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=2) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/02'; + +ALTER TABLE header_footer_table_2 ADD PARTITION (year=2012, month=1, day=3) location 'hdfs:///tmp/test/header_footer_table_2/2012/01/03'; + +SELECT * FROM header_footer_table_2; + +SELECT * FROM header_footer_table_2 WHERE id < 50; + +CREATE EXTERNAL TABLE emptytable (name string, message string, id int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' LOCATION 'hdfs:///tmp/test/header_footer_table_3' tblproperties ("skip.header.line.count"="1", "skip.footer.line.count"="2"); + +SELECT * FROM emptytable; + +SELECT * FROM emptytable WHERE id < 50; + +DROP TABLE header_footer_table_1; + +DROP TABLE header_footer_table_2; + +DROP TABLE emptytable; + +dfs -rmr hdfs:///tmp/test; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q index f8d855b25948a..7f4258f7bc15b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q @@ -12,11 +12,11 @@ create table T3 (c0 bigint, c1 bigint, c2 int) partitioned by (ds string); create table T4 (c0 bigint, c1 string, c2 string, c3 string, c4 string, c5 string, c6 string, c7 string, c8 string, c9 string, c10 string, c11 string, c12 string, c13 string, c14 string, c15 string, c16 string, c17 string, c18 string, c19 string, c20 string, c21 string, c22 string, c23 string, c24 string, c25 string, c26 string, c27 string, c28 string, c29 string, c30 string, c31 string, c32 string, c33 string, c34 string, c35 string, c36 string, c37 string, c38 string, c39 string, c40 string, c41 string, c42 string, c43 string, c44 string, c45 string, c46 string, c47 string, c48 string, c49 string, c50 string, c51 string, c52 string, c53 string, c54 string, c55 string, c56 string, c57 string, c58 string, c59 string, c60 string, c61 string, c62 string, c63 string, c64 string, c65 string, c66 string, c67 bigint, c68 string, c69 string, c70 bigint, c71 bigint, c72 bigint, c73 string, c74 string, c75 string, c76 string, c77 string, c78 string, c79 string, c80 string, c81 bigint, c82 bigint, c83 bigint) partitioned by (ds string); -insert overwrite table T1 partition (ds='2010-04-17') select '5', '1', '1', '1', 0, 0,4 from src limit 1; +insert overwrite table T1 partition (ds='2010-04-17') select '5', '1', '1', '1', 0, 0,4 from src tablesample (1 rows); -insert overwrite table T2 partition(ds='2010-04-17') select '5','name', NULL, '2', 'kavin',NULL, '9', 'c', '8', '0', '0', '7', '1','2', '0', '3','2', NULL, '1', NULL, '3','2','0','0','5','10' from src limit 1; +insert overwrite table T2 partition(ds='2010-04-17') select '5','name', NULL, '2', 'kavin',NULL, '9', 'c', '8', '0', '0', '7', '1','2', '0', '3','2', NULL, '1', NULL, '3','2','0','0','5','10' from src tablesample (1 rows); -insert overwrite table T3 partition (ds='2010-04-17') select 4,5,0 from src limit 1; +insert overwrite table T3 partition (ds='2010-04-17') select 4,5,0 from src tablesample (1 rows); insert overwrite table T4 partition(ds='2010-04-17') select 4,'1','1','8','4','5','1','0','9','U','2','2', '0','2','1','1','J','C','A','U', '2','s', '2',NULL, NULL, NULL,NULL, NULL, NULL,'1','j', 'S', '6',NULL,'1', '2', 'J', 'g', '1', 'e', '2', '1', '2', 'U', 'P', 'p', '3', '0', '0', '0', '1', '1', '1', '0', '0', '0', '6', '2', 'j',NULL, NULL, NULL,NULL,NULL, NULL, '5',NULL, 'j', 'j', 2, 2, 1, '2', '2', '1', '1', '1', '1', '1', '1', 1, 1, 32,NULL from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q new file mode 100644 index 0000000000000..69d543f472347 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_numeric.q @@ -0,0 +1,21 @@ +set hive.exec.dynamic.partition.mode=nonstrict; + +create table partint(key string, value string) partitioned by (ds string, hr int); +insert overwrite table partint partition(ds, hr) select * from srcpart where ds = '2008-04-08'; + +explain select key, value, hr from partint where hr < 11; +select key, value, hr from partint where hr < 11; + +explain select key, value, hr from partint where hr <= 12 and hr > 11; +select key, value, hr from partint where hr <= 12 and hr > 11; + +explain select key, value, hr from partint where hr between 11 and 12; +select key, value, hr from partint where hr between 11 and 12; + +explain select key, value, hr from partint where hr not between 12 and 14; +select key, value, hr from partint where hr not between 12 and 14; + +explain select key, value, hr from partint where hr < 13; +select key, value, hr from partint where hr < 13; + +drop table partint; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q index b76cf34120701..c8a08af054579 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q @@ -8,9 +8,9 @@ drop table gl_src_part1; create table gl_src1 (key int, value string) stored as textfile; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src1; @@ -49,10 +49,10 @@ select key from gl_src2 ORDER BY key ASC limit 10; -- partition create table gl_src_part1 (key int, value string) partitioned by (p string) stored as textfile; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE gl_src_part1 partition(p='11'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE gl_src_part1 partition(p='12'); select key from gl_src_part1 where p like '1%' ORDER BY key ASC limit 10; select key from gl_src_part1 where p='11' ORDER BY key ASC limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q index db38d43fe4354..7750cb90b5ca5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q @@ -6,7 +6,7 @@ CREATE TABLE dest1(key INT, val1 INT, val2 INT); CREATE TABLE dest2(key INT, val1 INT, val2 INT); CREATE TABLE INPUT(key INT, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv5.txt' INTO TABLE INPUT; +LOAD DATA LOCAL INPATH '../../data/files/kv5.txt' INTO TABLE INPUT; EXPLAIN FROM INPUT diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q new file mode 100644 index 0000000000000..6e9aee1d11d58 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby12.q @@ -0,0 +1,13 @@ +set hive.map.aggr=false; + +CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT COUNT(src.key), COUNT(DISTINCT value) GROUP BY src.key; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT COUNT(src.key), COUNT(DISTINCT value) GROUP BY src.key; + +SELECT dest1.* FROM dest1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q index 1b6891e33a37e..55133332a8662 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q @@ -5,6 +5,6 @@ CREATE TABLE dest1(key INT, value DOUBLE) STORED AS TEXTFILE; EXPLAIN FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key LIMIT 5; -FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key LIMIT 5; +FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key ORDER BY src.key LIMIT 5; SELECT dest1.* FROM dest1 ORDER BY dest1.key ASC , dest1.value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q index 82cff36422e62..dde37dfd47145 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q @@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) G FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q index 874995888b447..f346cb7e90147 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q @@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) G FROM src INSERT OVERWRITE TABLE dest1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q index 1b10f1e024819..c587b5f658f68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q @@ -9,4 +9,4 @@ FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) FROM src INSERT OVERWRITE TABLE dest_g1 SELECT src.key, sum(substr(src.value,5)) GROUP BY src.key; -SELECT dest_g1.* FROM dest_g1; +SELECT dest_g1.* FROM dest_g1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q index 4bc263c77f1b4..30499248cac15 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q @@ -1,7 +1,7 @@ set mapred.reduce.tasks=31; EXPLAIN -SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 5; +SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5; -SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key LIMIT 5; +SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key ORDER BY src.key LIMIT 5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q index c3cf598fb1c90..794ec758e9edb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q @@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(s FROM src INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q index 25e6789b63ef2..55d1a34b3c921 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q @@ -11,4 +11,15 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(s FROM src INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; + +-- HIVE-5560 when group by key is used in distinct funtion, invalid result are returned + +EXPLAIN +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.key,1,1)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1), count(DISTINCT substr(src.key,1,1)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); + +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q index c3c82d51749fe..6d7cb61e2d44a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q @@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr FROM src INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))) GROUP BY substr(src.key,1,1); -SELECT dest_g2.* FROM dest_g2; +SELECT dest_g2.* FROM dest_g2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q index b80c271afe5dc..b2450c9ea04e1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q @@ -11,4 +11,4 @@ INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr FROM src INSERT OVERWRITE TABLE dest_g2 SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(src.key,1,1),sum(substr(src.value,5))), sum(DISTINCT substr(src.value, 5)), count(src.value) GROUP BY substr(src.key,1,1); -SELECT dest_g2.* FROM dest_g2; +SELECT dest_g2.* FROM dest_g2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q index 99c2d2d5a1784..a1ebf90aadfea 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q @@ -12,5 +12,5 @@ INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key, FROM src INSERT OVERWRITE TABLE dest1 SELECT substr(src.key,1,1) GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q index be60785d87ea5..e96568b398d87 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q @@ -16,5 +16,5 @@ SELECT src.key, sum(substr(src.value,5)) FROM src GROUP BY src.key; -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q index fbf761c3aea55..ced122fae3f50 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q @@ -11,6 +11,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); FROM src INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q index ac79a286055d5..0d3727b052858 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q @@ -11,6 +11,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); FROM src INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q index 2c99d362ffff7..466c13222f29f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q @@ -12,6 +12,6 @@ INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); FROM src INSERT OVERWRITE TABLE dest1 SELECT DISTINCT substr(src.value,5,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q index b1457d9349aee..2b8c5db41ea92 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q @@ -18,5 +18,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q index 481b3cd084f16..5895ed4599849 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q @@ -17,5 +17,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q index a34ac8f64b117..ee6d7bf83084e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q @@ -17,5 +17,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q index 94a3dcf7ec3f2..8c2308e5d75c3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q @@ -18,5 +18,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q index 802aea244da9e..e673cc61622c8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q @@ -14,8 +14,8 @@ INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY S INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10; FROM SRC -INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10 -INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key limit 10; +INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10 +INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, sum(SUBSTR(SRC.value,5)) GROUP BY SRC.key ORDER BY SRC.key limit 10; SELECT DEST1.* FROM DEST1 ORDER BY key ASC, value ASC; SELECT DEST2.* FROM DEST2 ORDER BY key ASC, value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q index 62b6ff5ddebb6..0252e993363aa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q @@ -14,6 +14,6 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q index 846fd01017172..b5e1f63a45257 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q @@ -14,6 +14,6 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q index 20c8bef34223a..da85504ca18c6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q @@ -15,5 +15,5 @@ FROM SRC INSERT OVERWRITE TABLE DEST1 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key INSERT OVERWRITE TABLE DEST2 SELECT SRC.key, COUNT(DISTINCT SUBSTR(SRC.value,5)) GROUP BY SRC.key; -SELECT DEST1.* FROM DEST1; -SELECT DEST2.* FROM DEST2; +SELECT DEST1.* FROM DEST1 ORDER BY key; +SELECT DEST2.* FROM DEST2 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q index 7e97f75cecc92..2e3eddcb1f866 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q @@ -1,7 +1,7 @@ set hive.map.aggr.hash.percentmemory = 0.3; set hive.mapred.local.mem = 384; -add file ../data/scripts/dumpdata_script.py; +add file ../../data/scripts/dumpdata_script.py; select count(distinct subq.key) from (FROM src MAP src.key USING 'python dumpdata_script.py' AS key WHERE src.key = 10) subq; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q index 46e1f00d0f224..099beb4319e09 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q @@ -3,7 +3,7 @@ set hive.groupby.skewindata=false; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; EXPLAIN SELECT key, val, count(1) FROM T1 GROUP BY key, val with cube; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q index bced21f9e494a..de4a7c3cb5e54 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q @@ -1,6 +1,6 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; SELECT key, val, GROUPING__ID from T1 group by key, val with cube; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q index ffc627c82eaff..f451f17834502 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q @@ -1,6 +1,6 @@ CREATE TABLE T1(key INT, value INT) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/groupby_groupingid.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/groupby_groupingid.txt' INTO TABLE T1; set hive.groupby.skewindata = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q index 4fba7338f1d0a..804dfb36cf2c6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q @@ -1,6 +1,6 @@ CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; SELECT * FROM T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q index 9f2286cc9be31..30f1b420cc7c2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q @@ -2,7 +2,7 @@ set hive.new.job.grouping.set.cardinality=2; CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; -- Since 4 grouping sets would be generated for the query below, an additional MR job should be created EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q index 9a00d0a7aa77f..707737798dd6a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q @@ -4,8 +4,8 @@ -- additional MR job is created for processing the grouping sets. CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets2.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets2.txt' INTO TABLE T1; set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; set hive.new.job.grouping.set.cardinality = 30; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q index 25f1fcd793a74..ff83185d819c5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q @@ -4,7 +4,7 @@ set hive.merge.mapredfiles = false; CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; -- This tests that cubes and rollups work fine inside sub-queries. EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q index fb0c5913fa07f..d94bd81f84f3b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q @@ -4,7 +4,7 @@ set hive.merge.mapredfiles = false; CREATE TABLE T1(a STRING, b STRING, c STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY ' ' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/grouping_sets.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/grouping_sets.txt' INTO TABLE T1; -- This tests that cubes and rollups work fine where the source is a sub-query EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q index f0a8b72b0c83e..4a199365cf968 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q @@ -17,4 +17,4 @@ SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(s WHERE src.ds = '2008-04-08' GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q index b863344485d9f..cb3ee82918611 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q @@ -17,4 +17,4 @@ SELECT substr(src.key,1,1), count(DISTINCT substr(src.value,5)), concat(substr(s WHERE src.ds = '2008-04-08' GROUP BY substr(src.key,1,1); -SELECT dest1.* FROM dest1; +SELECT dest1.* FROM dest1 ORDER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q new file mode 100644 index 0000000000000..663e33b4c7ad8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_resolution.q @@ -0,0 +1,61 @@ + + +set hive.map.aggr=false; +set hive.groupby.skewindata=false; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +set hive.map.aggr=false; +set hive.groupby.skewindata=true; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +set hive.map.aggr=true; +set hive.groupby.skewindata=false; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +set hive.map.aggr=true; +set hive.groupby.skewindata=true; +explain select key, count(*) from src b group by b.key; +explain select b.key, count(*) from src b group by key; + +-- windowing after group by +select key, count(*), rank() over(order by count(*)) +from src b +where key < '12' +group by b.key +order by b.key; + +-- having after group by +select key, count(*) +from src b +group by b.key +having key < '12' +order by b.key; + +-- having and windowing +select key, count(*), rank() over(order by count(*)) +from src b +group by b.key +having key < '12' +order by b.key +; + +explain +select key, count(*), rank() over(order by count(*)) +from src b +group by b.key +having key < '12' +; + +-- order by +select key +from src t +where key < '12' +group by t.key +order by t.key; + +-- cluster by +EXPLAIN +SELECT x.key, x.value as key FROM SRC x CLUSTER BY key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q index f79b0c472ebb1..ee8038c7d9f5a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q @@ -3,7 +3,7 @@ set hive.groupby.skewindata=false; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; EXPLAIN SELECT key, val, count(1) FROM T1 GROUP BY key, val with rollup; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q index 911a11ae8990f..7401a9ca1d9bd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q index 31b4ec5c74dc2..700a8af91548e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q index 103c57a123576..2ef8447935a66 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q index e43da3c93225f..3c959e381f220 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key, val) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q index bef5e5d2d547a..dd05238f1ccc4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; @@ -30,7 +30,7 @@ DROP TABLE T1; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (val, key) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; @@ -52,7 +52,7 @@ DROP TABLE T1; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (val) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q index cf076e8125612..aa09aec34b233 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q @@ -17,7 +17,7 @@ SELECT key, count(1) FROM T1 where ds = '1' GROUP BY key; SELECT * FROM outputTbl1 ORDER BY key; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='2'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='2'); -- The plan should not be converted to a map-side group since no partition is being accessed EXPLAIN EXTENDED diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q index c2d42154e516c..99337859fb00b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string) CLUSTERED BY (val) SORTED BY (key, val) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q index 121804e60a9da..f53295e4b2435 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q index 1c3d1cdcc265d..296336d0f9f1a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q @@ -6,7 +6,7 @@ set hive.map.groupby.sorted=true; CREATE TABLE T1(key STRING, val STRING) PARTITIONED BY (ds string) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1 PARTITION (ds='1'); -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 PARTITION (ds='1') select key, val from T1 where ds = '1'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q index 068c26a9c2b4b..db0faa04da0ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q @@ -7,7 +7,7 @@ set hive.groupby.skewindata=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q index 8efa05e2544d4..4ec138e51a806 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q @@ -7,7 +7,7 @@ set hive.map.groupby.sorted.testmode=true; CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; -- perform an insert to make sure there are 2 files INSERT OVERWRITE TABLE T1 select key, val from T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q new file mode 100644 index 0000000000000..cb147c5feab24 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/import_exported_table.q @@ -0,0 +1,13 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/exported_table/; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_import_exported_table/exported_table/data/; + +dfs -copyFromLocal ../../data/files/exported_table/_metadata hdfs:///tmp/test_import_exported_table/exported_table; +dfs -copyFromLocal ../../data/files/exported_table/data/data hdfs:///tmp/test_import_exported_table/exported_table/data; + +IMPORT FROM '/tmp/test_import_exported_table/exported_table'; +DESCRIBE j1_41; +SELECT * from j1_41; + +dfs -rmr hdfs:///tmp/test_import_exported_table; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q index 33a1fc581ed13..03d77f1f19b01 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q @@ -1,15 +1,18 @@ +set hive.stats.dbclass=fs; SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; create table foobar(key int, value string) PARTITIONED BY (ds string, hr string); alter table foobar add partition (ds='2008-04-08',hr='12'); -CREATE INDEX srcpart_auth_index ON TABLE foobar(key) as 'BITMAP' WITH DEFERRED REBUILD; +CREATE INDEX srcpart_AUTH_index ON TABLE foobar(key) as 'BITMAP' WITH DEFERRED REBUILD; +SHOW INDEXES ON foobar; + grant select on table foobar to user hive_test_user; -grant select on table default__foobar_srcpart_auth_index__ to user hive_test_user; -grant update on table default__foobar_srcpart_auth_index__ to user hive_test_user; -grant create on table default__foobar_srcpart_auth_index__ to user hive_test_user; +grant select on table default__foobar_srcpart_auth_indeX__ to user hive_test_user; +grant update on table default__foobar_srcpart_auth_indEx__ to user hive_test_user; +grant create on table default__foobar_srcpart_auth_inDex__ to user hive_test_user; set hive.security.authorization.enabled=true; -ALTER INDEX srcpart_auth_index ON foobar PARTITION (ds='2008-04-08',hr='12') REBUILD; +ALTER INDEX srcpart_auth_INDEX ON foobar PARTITION (ds='2008-04-08',hr='12') REBUILD; set hive.security.authorization.enabled=false; DROP INDEX srcpart_auth_index on foobar; DROP TABLE foobar; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q index cb8a1d6293f8d..77733aac02686 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q @@ -3,6 +3,7 @@ -- without indexing SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; @@ -25,4 +26,4 @@ SET hive.optimize.index.filter.compact.minsize=0; EXPLAIN SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key; SELECT key, value FROM src WHERE key > 80 AND key < 100 ORDER BY key; -DROP INDEX src_index on src; \ No newline at end of file +DROP INDEX src_index on src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q index cb32162d40a06..41f4a40823e4d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q @@ -3,6 +3,7 @@ -- Create temp, and populate it with some values in src. CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE; +set hive.stats.dbclass=fs; -- Build an index on temp. CREATE INDEX temp_index ON TABLE temp(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX temp_index ON temp REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q index 790e6c223f746..2967bd60d8bc6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test automatic use of index on different file formats CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; @@ -16,4 +17,4 @@ SET hive.optimize.index.filter.compact.minsize=0; EXPLAIN SELECT key, value FROM src WHERE key=86 ORDER BY key; SELECT key, value FROM src WHERE key=86 ORDER BY key; -DROP INDEX src_index on src; \ No newline at end of file +DROP INDEX src_index on src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q index 2bf8481f1d6e2..a672e06e79332 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q @@ -4,6 +4,7 @@ EXPLAIN SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q index 808a04cc360a6..d78e0fd58a074 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q @@ -4,6 +4,7 @@ EXPLAIN SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; SELECT a.key, a.value FROM src a JOIN srcpart b ON (a.key = b.key) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q index 06e97fa76bc97..f0a91b4b8a592 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- With multiple indexes, make sure we choose which to use in a consistent order CREATE INDEX src_key_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; @@ -13,4 +14,4 @@ EXPLAIN SELECT key, value FROM src WHERE key=86 ORDER BY key; SELECT key, value FROM src WHERE key=86 ORDER BY key; DROP INDEX src_key_index ON src; -DROP INDEX src_val_index ON src; \ No newline at end of file +DROP INDEX src_val_index ON src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q index 5013d29e732c4..70166b36c5f3c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test automatic use of index on table with partitions CREATE INDEX src_part_index ON TABLE srcpart(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_part_index ON srcpart REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q index 0984a4a21ba17..1d9efbbc6d738 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q @@ -3,6 +3,7 @@ EXPLAIN SELECT a.key, b.key FROM src a JOIN src b ON (a.value = b.value) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; SELECT a.key, b.key FROM src a JOIN src b ON (a.value = b.value) WHERE a.key > 80 AND a.key < 100 AND b.key > 70 AND b.key < 90 ORDER BY a.key; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q index d8f3eda1813c8..acd4194b0e7a4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test cases where the index should not be used automatically CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q index f9deb2883960c..673c835fb9084 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP INDEX srcpart_index_proj on srcpart; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q index 55633d9225fa7..adec8f1b3bfb7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; EXPLAIN CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q index bd15a21fa4200..1ffa6eeebbfb3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; EXPLAIN CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q index 0d2c811459dcb..e7a093c118ba0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q @@ -1,3 +1,6 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; + EXPLAIN CREATE INDEX src1_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; EXPLAIN @@ -17,10 +20,10 @@ SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ WHERE key = 0) a JOIN - (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ - WHERE value = "val_0") b - ON - a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q index 672ce29f1bb47..56cd44dd5b323 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q @@ -1,3 +1,6 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; + -- try the query without indexing, with manual indexing, and with automatic indexing -- without indexing SELECT key, value FROM src WHERE key=0 AND value = "val_0" ORDER BY key; @@ -21,12 +24,12 @@ SELECT * FROM default__src_src2_index__ ORDER BY value; EXPLAIN SELECT a.bucketname AS `_bucketname`, COLLECT_SET(a.offset) as `_offsets` FROM (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src1_index__ - WHERE key = 0) a - JOIN - (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ - WHERE value = "val_0") b - ON - a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT + WHERE key = 0) a + JOIN + (SELECT `_bucketname` AS bucketname, `_offset` AS offset, `_bitmaps` AS bitmaps FROM default__src_src2_index__ + WHERE value = "val_0") b + ON + a.bucketname = b.bucketname AND a.offset = b.offset WHERE NOT EWAH_BITMAP_EMPTY(EWAH_BITMAP_AND(a.bitmaps, b.bitmaps)) GROUP BY a.bucketname; INSERT OVERWRITE DIRECTORY "${system:test.tmp.dir}/index_result" diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q index 90d7987594bac..3b310cee4cb29 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test automatic use of index on table with partitions CREATE INDEX src_part_index ON TABLE srcpart(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_part_index ON srcpart REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q index 2f5e5d4fb231b..32ecfb9db8a0e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; SET hive.exec.compress.result=true; CREATE INDEX src_index ON TABLE src(key) as 'BITMAP' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q index 054df51c32180..26a351ea31858 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE srcpart_rc (key int, value string) PARTITIONED BY (ds string, hr int) STORED AS RCFILE; INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q index a936f1127f690..98cbec147db51 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP INDEX srcpart_index_proj on srcpart; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q index 837033be403a9..97276f488e742 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; EXPLAIN CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q index 56119ac11f32f..1eb3f5c3dbcbf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE srcpart_rc (key int, value string) PARTITIONED BY (ds string, hr int) STORED AS RCFILE; INSERT OVERWRITE TABLE srcpart_rc PARTITION (ds='2008-04-08', hr=11) SELECT key, value FROM srcpart WHERE ds = '2008-04-08' AND hr = 11; @@ -42,4 +43,4 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; SELECT key, value FROM srcpart_rc WHERE key=100 ORDER BY key; DROP INDEX srcpart_rc_index on srcpart_rc; -DROP TABLE srcpart_rc; \ No newline at end of file +DROP TABLE srcpart_rc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q index f3fcb4af3d5e6..599b4ac1149ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE src_index_test_rc (key int, value string) STORED AS RCFILE; INSERT OVERWRITE TABLE src_index_test_rc SELECT * FROM src; @@ -16,4 +17,4 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; SELECT key, value FROM src_index_test_rc WHERE key=100 ORDER BY key; DROP INDEX src_index on src_index_test_rc; -DROP TABLE src_index_test_rc; \ No newline at end of file +DROP TABLE src_index_test_rc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q index d0d9a32adcba1..e72b27c781a80 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q @@ -1,6 +1,6 @@ SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; SET hive.default.fileformat=TextFile; - +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; @@ -129,4 +129,4 @@ SELECT * FROM src WHERE key >= '9'; SET hive.exec.post.hooks=; -DROP INDEX src_index ON src; \ No newline at end of file +DROP INDEX src_index ON src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q index 84ed3cc932e8c..963b8f74e5f07 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q @@ -1,4 +1,5 @@ SET hive.exec.compress.result=true; +set hive.stats.dbclass=fs; CREATE INDEX src_index ON TABLE src(key) as 'COMPACT' WITH DEFERRED REBUILD; ALTER INDEX src_index ON src REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q index 062821e870dce..ef020b63d5a72 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; drop index src_index_2 on src; drop index src_index_3 on src; drop index src_index_4 on src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q index a6fe16ba3b809..20186a7400494 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- Want to ensure we can build and use indices on tables stored with SerDes -- Build the (Avro backed) table CREATE TABLE doctors @@ -31,7 +32,7 @@ TBLPROPERTIES ('avro.schema.literal'='{ DESCRIBE doctors; -LOAD DATA LOCAL INPATH '../data/files/doctors.avro' INTO TABLE doctors; +LOAD DATA LOCAL INPATH '../../data/files/doctors.avro' INTO TABLE doctors; -- Create and build an index CREATE INDEX doctors_index ON TABLE doctors(number) AS 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q index 82e15b97b6ac4..ecab2b7f6b332 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- test that stale indexes are not used CREATE TABLE temp(key STRING, val STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q index e7cfeff31f6dc..a93ccf7f95fc3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; -- Test if index is actually being used. -- Create temp, and populate it with some values in src. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q index 119994e91b056..728b8cc4a9497 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q @@ -47,12 +47,12 @@ CREATE TABLE srcpart_merge_dp LIKE srcpart; CREATE TABLE srcpart_merge_dp_rc LIKE srcpart; ALTER TABLE srcpart_merge_dp_rc SET FILEFORMAT RCFILE; -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=11); -LOAD DATA LOCAL INPATH '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12); +LOAD DATA LOCAL INPATH '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp PARTITION(ds='2008-04-08', hr=12); INSERT OVERWRITE TABLE srcpart_merge_dp_rc PARTITION (ds = '2008-04-08', hr) SELECT key, value, hr FROM srcpart_merge_dp WHERE ds = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q index a039dc5a2a040..ce5ed8419dd44 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q @@ -1,7 +1,7 @@ DROP TABLE infertypes; CREATE TABLE infertypes(ti TINYINT, si SMALLINT, i INT, bi BIGINT, fl FLOAT, db DOUBLE, str STRING); -LOAD DATA LOCAL INPATH '../data/files/infer_const_type.txt' OVERWRITE INTO TABLE infertypes; +LOAD DATA LOCAL INPATH '../../data/files/infer_const_type.txt' OVERWRITE INTO TABLE infertypes; SELECT * FROM infertypes; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q index 40fbc84a981f1..620e9dcfae6cf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q @@ -7,15 +7,15 @@ FROM src INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200 INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key WHERE src.key >= 200 and src.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; FROM src INSERT OVERWRITE TABLE dest1 SELECT src.* WHERE src.key < 100 INSERT OVERWRITE TABLE dest2 SELECT src.key, src.value WHERE src.key >= 100 and src.key < 200 INSERT OVERWRITE TABLE dest3 PARTITION(ds='2008-04-08', hr='12') SELECT src.key WHERE src.key >= 200 and src.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/dest4.out' SELECT src.value WHERE src.key >= 300; SELECT dest1.* FROM dest1; SELECT dest2.* FROM dest2; SELECT dest3.* FROM dest3; -dfs -cat ../build/ql/test/data/warehouse/dest4.out/*; +dfs -cat ${system:test.warehouse.dir}/dest4.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q index 82e6d81426efc..4990d0ba1cd54 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q @@ -1,6 +1,6 @@ -- TestSerDe is a user defined serde where the default delimiter is Ctrl-B DROP TABLE INPUT16; -ADD JAR ../data/files/TestSerDe.jar; +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; CREATE TABLE INPUT16(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1_cb.txt' INTO TABLE INPUT16; +LOAD DATA LOCAL INPATH '../../data/files/kv1_cb.txt' INTO TABLE INPUT16; SELECT INPUT16.VALUE, INPUT16.KEY FROM INPUT16; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q index 5dab4103d8dda..9272a92c8102e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q @@ -4,8 +4,8 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; -- the user is overwriting it with ctrlC DROP TABLE INPUT16_CC; -ADD JAR ../data/files/TestSerDe.jar; +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; CREATE TABLE INPUT16_CC(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' with serdeproperties ('testserde.default.serialization.format'='\003', 'dummy.prop.not.used'='dummyy.val') STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC; +LOAD DATA LOCAL INPATH '../../data/files/kv1_cc.txt' INTO TABLE INPUT16_CC; SELECT INPUT16_CC.VALUE, INPUT16_CC.KEY FROM INPUT16_CC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q index fec44e97669dd..3dc7fec9f6669 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q @@ -1,5 +1,5 @@ create table apachelog(ipaddress STRING,identd STRING,user_name STRING,finishtime STRING,requestline string,returncode INT,size INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe' WITH SERDEPROPERTIES ( 'serialization.format'= 'org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol', 'quote.delim'= '("|\\[|\\])', 'field.delim'=' ', 'serialization.null.format'='-' ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/apache.access.log' INTO TABLE apachelog; +LOAD DATA LOCAL INPATH '../../data/files/apache.access.log' INTO TABLE apachelog; SELECT a.* FROM apachelog a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q index 0566ab18c2537..ff430abb8e8aa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q @@ -1,6 +1,6 @@ CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; -ADD FILE ../data/scripts/input20_script; +ADD FILE ../../data/scripts/input20_script.py; EXPLAIN FROM ( @@ -12,7 +12,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS key, value; FROM ( @@ -24,7 +24,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS key, value; SELECT * FROM dest1 SORT BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q index d7c814e58061c..43cd01e684b37 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q @@ -1,7 +1,7 @@ CREATE TABLE src_null(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/null.txt' INTO TABLE src_null; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE src_null; EXPLAIN SELECT * FROM src_null DISTRIBUTE BY c SORT BY d; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q index 853947be57a1f..8803e4dbeb8cf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q @@ -1,5 +1,5 @@ CREATE TABLE INPUT4(KEY STRING, VALUE STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4; EXPLAIN SELECT a.KEY2 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q index 7ab17515af5c7..8b6b21502001e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q @@ -1,6 +1,6 @@ CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; -ADD FILE ../data/scripts/input20_script; +ADD FILE ../../data/scripts/input20_script.py; EXPLAIN FROM ( @@ -12,7 +12,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS (key STRING, value STRING); FROM ( @@ -24,7 +24,7 @@ FROM ( ) tmap INSERT OVERWRITE TABLE dest1 REDUCE tmap.key, tmap.value -USING 'input20_script' +USING 'python input20_script.py' AS (key STRING, value STRING); SELECT * FROM dest1 SORT BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q index 6fd136afec444..6ded61aa23990 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q @@ -1,6 +1,6 @@ create table documents(contents string) stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/docurl.txt' INTO TABLE documents; +LOAD DATA LOCAL INPATH '../../data/files/docurl.txt' INTO TABLE documents; select url, count(1) @@ -8,7 +8,7 @@ FROM ( FROM documents MAP documents.contents - USING 'java -cp ../build/ql/test/classes org.apache.hadoop.hive.scripts.extracturl' AS (url, count) + USING 'java -cp ../util/target/classes/ org.apache.hadoop.hive.scripts.extracturl' AS (url, count) ) subq group by url; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q index 3584820acaf4a..f983aca847d95 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q @@ -1,7 +1,7 @@ CREATE TABLE T1(key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/kv2.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv2.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, value STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q index 08d6d97603045..1186bbbbe6bbd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q @@ -1,7 +1,7 @@ CREATE TABLE INPUT4(KEY STRING, VALUE STRING) STORED AS TEXTFILE; EXPLAIN -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUT4; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUT4; EXPLAIN FORMATTED SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias; SELECT Input4Alias.VALUE, Input4Alias.KEY FROM INPUT4 AS Input4Alias diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q index 4166cb5f94939..ab187b5d7e7d1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q @@ -2,15 +2,15 @@ create table tmp_insert_test (key string, value string) stored as textfile; -load data local inpath '../data/files/kv1.txt' into table tmp_insert_test; +load data local inpath '../../data/files/kv1.txt' into table tmp_insert_test; select * from tmp_insert_test; create table tmp_insert_test_p (key string, value string) partitioned by (ds string) stored as textfile; -load data local inpath '../data/files/kv1.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); +load data local inpath '../../data/files/kv1.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); select * from tmp_insert_test_p where ds= '2009-08-01' order by key, value; -load data local inpath '../data/files/kv2.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); +load data local inpath '../../data/files/kv2.txt' into table tmp_insert_test_p partition (ds = '2009-08-01'); select * from tmp_insert_test_p where ds= '2009-08-01' order by key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q index 5512dc328065d..3182bbef39812 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q @@ -1,7 +1,7 @@ drop table tst_src1; create table tst_src1 like src1; -load data local inpath '../data/files/kv1.txt' into table tst_src1 ; +load data local inpath '../../data/files/kv1.txt' into table tst_src1 ; select count(1) from tst_src1; -load data local inpath '../data/files/kv1.txt' into table tst_src1 ; +load data local inpath '../../data/files/kv1.txt' into table tst_src1 ; select count(1) from tst_src1; drop table tst_src1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q index 4557edc178b7f..2e975e58c1e93 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q @@ -3,4 +3,4 @@ CREATE TABLE dest(key INT, value STRING) STORED AS TEXTFILE; SET hive.output.file.extension=.txt; INSERT OVERWRITE TABLE dest SELECT src.* FROM src; -dfs -cat ../build/ql/test/data/warehouse/dest/*.txt \ No newline at end of file +dfs -cat ${system:test.warehouse.dir}/dest/*.txt diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q index 633a8c6edb5cc..334da264d6e55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q @@ -2,8 +2,8 @@ SET hive.insert.into.multilevel.dirs=true; SET hive.output.file.extension=.txt; -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/x/y/z/' SELECT src.* FROM src; +INSERT OVERWRITE DIRECTORY 'target/data/x/y/z/' SELECT src.* FROM src; -dfs -cat ../build/ql/test/data/x/y/z/*.txt; +dfs -cat ${system:build.dir}/data/x/y/z/*.txt; -dfs -rmr ../build/ql/test/data/x; \ No newline at end of file +dfs -rmr ${system:build.dir}/data/x; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q index 8c57dd3f25aa6..b18d60aa74e73 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q @@ -1,4 +1,4 @@ CREATE TABLE INPUT4_CB(KEY STRING, VALUE STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\002' LINES TERMINATED BY '\012' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1_cb.txt' INTO TABLE INPUT4_CB; +LOAD DATA LOCAL INPATH '../../data/files/kv1_cb.txt' INTO TABLE INPUT4_CB; SELECT INPUT4_CB.VALUE, INPUT4_CB.KEY FROM INPUT4_CB; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q index 4f5824df5c9d5..b108cbd6b23e5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q @@ -1,2 +1,2 @@ -dfs -cat ../data/files/kv1.txt; +dfs -cat ../../data/files/kv1.txt; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q index 9a7ca5da1277f..87c55a26d7b83 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q @@ -1,7 +1,7 @@ -- test for internationalization -- kv4.txt contains the utf-8 character 0xE982B5E993AE which we are verifying later on CREATE TABLE INPUTDDL5(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE INPUTDDL5; +LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE INPUTDDL5; DESCRIBE INPUTDDL5; SELECT INPUTDDL5.name from INPUTDDL5; SELECT count(1) FROM INPUTDDL5 WHERE INPUTDDL5.name = _UTF-8 0xE982B5E993AE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q index d33ab8d9a78f4..6c709399a3c0d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q @@ -2,8 +2,8 @@ -- test for describe extended table partition -- test for alter table drop partition CREATE TABLE INPUTDDL6(KEY STRING, VALUE STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE INPUTDDL6 PARTITION (ds='2008-04-08'); DESCRIBE EXTENDED INPUTDDL6; DESCRIBE EXTENDED INPUTDDL6 PARTITION (ds='2008-04-08'); SHOW PARTITIONS INPUTDDL6; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q index 8a73935feec51..27e587a283cd3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q @@ -3,22 +3,22 @@ CREATE TABLE T1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T1; SELECT COUNT(1) FROM T1; CREATE TABLE T2(name STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T2; SELECT COUNT(1) FROM T2; CREATE TABLE T3(name STRING) PARTITIONED BY(ds STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE T3 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE T3 PARTITION (ds='2008-04-09'); SELECT COUNT(1) FROM T3 where T3.ds='2008-04-09'; CREATE TABLE T4(name STRING) PARTITIONED BY(ds STRING) STORED AS SEQUENCEFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.seq' INTO TABLE T4 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.seq' INTO TABLE T4 PARTITION (ds='2008-04-09'); SELECT COUNT(1) FROM T4 where T4.ds='2008-04-09'; DESCRIBE EXTENDED T1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q index 6ad70b5673f30..6b00f977c4c68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q @@ -1,8 +1,8 @@ CREATE TABLE sourceTable (one string,two string) PARTITIONED BY (ds string,hr string); -load data local inpath '../data/files/kv1.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='11'); +load data local inpath '../../data/files/kv1.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='11'); -load data local inpath '../data/files/kv3.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='12'); +load data local inpath '../../data/files/kv3.txt' INTO TABLE sourceTable partition(ds='2011-11-11', hr='12'); CREATE TABLE destinTable (one string,two string) PARTITIONED BY (ds string,hr string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q index 598d30eaebba2..bd1eb752879e6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q @@ -4,9 +4,9 @@ CREATE DATABASE db2; CREATE TABLE db1.sourceTable (one string,two string) PARTITIONED BY (ds string); -load data local inpath '../data/files/kv1.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); +load data local inpath '../../data/files/kv1.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); -load data local inpath '../data/files/kv3.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); +load data local inpath '../../data/files/kv3.txt' INTO TABLE db1.sourceTable partition(ds='2011-11-11'); CREATE TABLE db2.destinTable (one string,two string) PARTITIONED BY (ds string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q index e58b212e190bb..4ff0edcd61bca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q @@ -4,10 +4,10 @@ DROP TABLE insert_into3b; CREATE TABLE insert_into3a (key int, value string); CREATE TABLE insert_into3b (key int, value string); -EXPLAIN FROM src INSERT INTO TABLE insert_into3a SELECT * LIMIT 50 - INSERT INTO TABLE insert_into3b SELECT * LIMIT 100; -FROM src INSERT INTO TABLE insert_into3a SELECT * LIMIT 50 - INSERT INTO TABLE insert_into3b SELECT * LIMIT 100; +EXPLAIN FROM src INSERT INTO TABLE insert_into3a SELECT * ORDER BY key, value LIMIT 50 + INSERT INTO TABLE insert_into3b SELECT * ORDER BY key, value LIMIT 100; +FROM src INSERT INTO TABLE insert_into3a SELECT * ORDER BY key, value LIMIT 50 + INSERT INTO TABLE insert_into3b SELECT * ORDER BY key, value LIMIT 100; SELECT SUM(HASH(c)) FROM ( SELECT TRANSFORM(*) USING 'tr \t _' AS (c) FROM insert_into3a ) t; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q index 25c127f67f980..6d069f5411d45 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q @@ -1,40 +1,40 @@ -insert overwrite local directory '../data/files/local_src_table_1' +insert overwrite local directory '../../data/files/local_src_table_1' select * from src ; -dfs -cat ../data/files/local_src_table_1/000000_0; +dfs -cat ../../data/files/local_src_table_1/000000_0; -insert overwrite local directory '../data/files/local_src_table_2' +insert overwrite local directory '../../data/files/local_src_table_2' row format delimited FIELDS TERMINATED BY ':' select * from src ; -dfs -cat ../data/files/local_src_table_2/000000_0; +dfs -cat ../../data/files/local_src_table_2/000000_0; create table array_table (a array, b array) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' COLLECTION ITEMS TERMINATED BY ','; -load data local inpath "../data/files/array_table.txt" overwrite into table array_table; +load data local inpath "../../data/files/array_table.txt" overwrite into table array_table; -insert overwrite local directory '../data/files/local_array_table_1' +insert overwrite local directory '../../data/files/local_array_table_1' select * from array_table; -dfs -cat ../data/files/local_array_table_1/000000_0; +dfs -cat ../../data/files/local_array_table_1/000000_0; -insert overwrite local directory '../data/files/local_array_table_2' +insert overwrite local directory '../../data/files/local_array_table_2' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' select * from array_table; -dfs -cat ../data/files/local_array_table_2/000000_0; +dfs -cat ../../data/files/local_array_table_2/000000_0; -insert overwrite local directory '../data/files/local_array_table_2_withfields' +insert overwrite local directory '../../data/files/local_array_table_2_withfields' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' select b,a from array_table; -dfs -cat ../data/files/local_array_table_2_withfields/000000_0; +dfs -cat ../../data/files/local_array_table_2_withfields/000000_0; create table map_table (foo STRING , bar MAP) @@ -44,63 +44,63 @@ COLLECTION ITEMS TERMINATED BY ',' MAP KEYS TERMINATED BY ':' STORED AS TEXTFILE; -load data local inpath "../data/files/map_table.txt" overwrite into table map_table; +load data local inpath "../../data/files/map_table.txt" overwrite into table map_table; -insert overwrite local directory '../data/files/local_map_table_1' +insert overwrite local directory '../../data/files/local_map_table_1' select * from map_table; -dfs -cat ../data/files/local_map_table_1/000000_0; +dfs -cat ../../data/files/local_map_table_1/000000_0; -insert overwrite local directory '../data/files/local_map_table_2' +insert overwrite local directory '../../data/files/local_map_table_2' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' MAP KEYS TERMINATED BY '=' select * from map_table; -dfs -cat ../data/files/local_map_table_2/000000_0; +dfs -cat ../../data/files/local_map_table_2/000000_0; -insert overwrite local directory '../data/files/local_map_table_2_withfields' +insert overwrite local directory '../../data/files/local_map_table_2_withfields' ROW FORMAT DELIMITED FIELDS TERMINATED BY ':' COLLECTION ITEMS TERMINATED BY '#' MAP KEYS TERMINATED BY '=' select bar,foo from map_table; -dfs -cat ../data/files/local_map_table_2_withfields/000000_0; +dfs -cat ../../data/files/local_map_table_2_withfields/000000_0; -insert overwrite local directory '../data/files/local_array_table_3' +insert overwrite local directory '../../data/files/local_array_table_3' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.DelimitedJSONSerDe' STORED AS TEXTFILE select * from array_table; -dfs -cat ../data/files/local_array_table_3/000000_0; +dfs -cat ../../data/files/local_array_table_3/000000_0; -insert overwrite local directory '../data/files/local_map_table_3' +insert overwrite local directory '../../data/files/local_map_table_3' ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.DelimitedJSONSerDe' STORED AS TEXTFILE select * from map_table; -dfs -cat ../data/files/local_map_table_3/000000_0; +dfs -cat ../../data/files/local_map_table_3/000000_0; -insert overwrite local directory '../data/files/local_rctable' +insert overwrite local directory '../../data/files/local_rctable' STORED AS RCFILE select value,key from src; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/local_rctable/temp; dfs -rmr ${system:test.tmp.dir}/local_rctable; dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/local_rctable; -dfs -put ../data/files/local_rctable/000000_0 ${system:test.tmp.dir}/local_rctable/000000_0; +dfs -put ../../data/files/local_rctable/000000_0 ${system:test.tmp.dir}/local_rctable/000000_0; create external table local_rctable(value string, key string) STORED AS RCFILE LOCATION '${system:test.tmp.dir}/local_rctable'; -insert overwrite local directory '../data/files/local_rctable_out' +insert overwrite local directory '../../data/files/local_rctable_out' ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' select key,value from local_rctable; -dfs -cat ../data/files/local_rctable_out/000000_0; +dfs -cat ../../data/files/local_rctable_out/000000_0; drop table local_rctable; drop table array_table; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q index b403814adb4c9..4d1ae2186e796 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q @@ -1,9 +1,9 @@ CREATE TABLE join_1to1_1(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in5.txt' INTO TABLE join_1to1_1; +LOAD DATA LOCAL INPATH '../../data/files/in5.txt' INTO TABLE join_1to1_1; CREATE TABLE join_1to1_2(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in6.txt' INTO TABLE join_1to1_2; +LOAD DATA LOCAL INPATH '../../data/files/in6.txt' INTO TABLE join_1to1_2; set hive.outerjoin.supports.filters=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q new file mode 100644 index 0000000000000..0b0c53803d01e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_alt_syntax.q @@ -0,0 +1,41 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +explain select p1.p_name, p2.p_name +from part p1 , part p2; + +explain select p1.p_name, p2.p_name, p3.p_name +from part p1 ,part p2 ,part p3 +where p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select p1.p_name, p2.p_name, p3.p_name +from part p1 , (select p_name from part) p2 ,part p3 +where p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select p1.p_name, p2.p_name, p3.p_name +from part p1 , part p2 , part p3 +where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name; + +explain select p1.p_name, p2.p_name, p3.p_name, p4.p_name +from part p1 , part p2 join part p3 on p2.p_name = p1.p_name join part p4 +where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; + +explain select p1.p_name, p2.p_name, p3.p_name, p4.p_name +from part p1 join part p2 on p2.p_name = p1.p_name , part p3 , part p4 +where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q index e4d95a51fa927..81e984e7eb018 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q @@ -1,8 +1,8 @@ create table tinyA(a bigint, b bigint) stored as textfile; create table tinyB(a bigint, bList array) stored as textfile; -load data local inpath '../data/files/tiny_a.txt' into table tinyA; -load data local inpath '../data/files/tiny_b.txt' into table tinyB; +load data local inpath '../../data/files/tiny_a.txt' into table tinyA; +load data local inpath '../../data/files/tiny_b.txt' into table tinyB; select * from tinyA; select * from tinyB; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q index a9b69c5b1c448..0c0962ceceebc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q @@ -1,8 +1,8 @@ CREATE TABLE joinone(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in5.txt' INTO TABLE joinone; +LOAD DATA LOCAL INPATH '../../data/files/in5.txt' INTO TABLE joinone; CREATE TABLE joinTwo(key1 int, key2 int, value int); -LOAD DATA LOCAL INPATH '../data/files/in6.txt' INTO TABLE joinTwo; +LOAD DATA LOCAL INPATH '../../data/files/in6.txt' INTO TABLE joinTwo; SELECT * FROM joinone JOIN joinTwo ON(joinone.key2=joinTwo.key2) ORDER BY joinone.key1 ASC, joinone.key2 ASC, joinone.value ASC, joinTwo.key1 ASC, joinTwo.key2 ASC, joinTwo.value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q new file mode 100644 index 0000000000000..7f493671b80c1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_1.q @@ -0,0 +1,30 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + + + +explain select * +from part p1 join part p2 join part p3 on p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_name = p1.p_name and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_partkey = 1 and p3.p_name = p2.p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q new file mode 100644 index 0000000000000..ca280104d9a75 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_2.q @@ -0,0 +1,24 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + + +explain select * +from part p1 join part p2 join part p3 on p1.p_name = p2.p_name join part p4 on p2.p_name = p3.p_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_name = p1.p_name join part p4 on p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q new file mode 100644 index 0000000000000..b308838d6243b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_3.q @@ -0,0 +1,34 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + + + +explain select * +from part p1 join part p2 join part p3 +where p1.p_name = p2.p_name and p2.p_name = p3.p_name; + +explain select * +from part p1 join part p2 join part p3 +where p2.p_name = p1.p_name and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 +where p2.p_partkey + p1.p_partkey = p1.p_partkey and p3.p_name = p2.p_name; + +explain select * +from part p1 join part p2 join part p3 +where p2.p_partkey = 1 and p3.p_name = p2.p_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q new file mode 100644 index 0000000000000..477682e4e8ac7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_4.q @@ -0,0 +1,26 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + + +explain select * +from part p1 join part p2 join part p3 on p1.p_name = p2.p_name join part p4 +where p2.p_name = p3.p_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part p2 join part p3 on p2.p_name = p1.p_name join part p4 +where p2.p_name = p3.p_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2.p_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q new file mode 100644 index 0000000000000..1013f51a0d806 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual1.q @@ -0,0 +1,52 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name and p2_name = p3_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_partkey + p_partkey = p1.p_partkey and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_partkey = 1 and p3_name = p2_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q new file mode 100644 index 0000000000000..6232357752851 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual2.q @@ -0,0 +1,47 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name join part p4 on p2_name = p3_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name join part p4 on p2_name = p3_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q new file mode 100644 index 0000000000000..6ac86042c0452 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual3.q @@ -0,0 +1,56 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 +where p1.p_name = p2_name and p2_name = p3_name; + +explain select * +from part p1 join part2 p2 join part3 p3 +where p2_name = p1.p_name and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 +where p2_partkey + p1.p_partkey = p1.p_partkey and p3_name = p2_name; + +explain select * +from part p1 join part2 p2 join part3 p3 +where p2_partkey = 1 and p3_name = p2_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q new file mode 100644 index 0000000000000..0db4d5e3cff1a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_cond_pushdown_unqual4.q @@ -0,0 +1,49 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +create table part3( + p3_partkey INT, + p3_name STRING, + p3_mfgr STRING, + p3_brand STRING, + p3_type STRING, + p3_size INT, + p3_container STRING, + p3_retailprice DOUBLE, + p3_comment STRING +); + +explain select * +from part p1 join part2 p2 join part3 p3 on p1.p_name = p2_name join part p4 +where p2_name = p3_name and p1.p_name = p4.p_name; + +explain select * +from part p1 join part2 p2 join part3 p3 on p2_name = p1.p_name join part p4 +where p2_name = p3_name and p1.p_partkey = p4.p_partkey + and p1.p_partkey = p2_partkey; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q index d54aa950a1ec2..49b6c6f920209 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q @@ -1,5 +1,5 @@ CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in3.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in3.txt' INTO TABLE myinput1; SELECT * FROM myinput1 a JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b on a.key > 40 AND a.value > 50 AND a.key = a.value AND b.key > 40 AND b.value > 50 AND b.key = b.value ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; @@ -55,10 +55,10 @@ SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.valu CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2; SET hive.optimize.bucketmapjoin = true; SET hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q index 31b0c8c91c10b..c4c239cae2d8a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q @@ -13,9 +13,9 @@ delimited fields terminated by ',' stored as textfile; create table hive_count (bar_id int, n int) row format delimited fields terminated by ',' stored as textfile; -load data local inpath '../data/files/hive_626_foo.txt' overwrite into table hive_foo; -load data local inpath '../data/files/hive_626_bar.txt' overwrite into table hive_bar; -load data local inpath '../data/files/hive_626_count.txt' overwrite into table hive_count; +load data local inpath '../../data/files/hive_626_foo.txt' overwrite into table hive_foo; +load data local inpath '../../data/files/hive_626_bar.txt' overwrite into table hive_bar; +load data local inpath '../../data/files/hive_626_count.txt' overwrite into table hive_count; explain select hive_foo.foo_name, hive_bar.bar_name, n from hive_foo join hive_bar on hive_foo.foo_id = diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q new file mode 100644 index 0000000000000..a0046dbc41332 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_merging.q @@ -0,0 +1,25 @@ + + +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +explain select p1.p_size, p2.p_size +from part p1 left outer join part p2 on p1.p_partkey = p2.p_partkey + right outer join part p3 on p2.p_partkey = p3.p_partkey and + p1.p_size > 10 +; + +explain select p1.p_size, p2.p_size +from part p1 left outer join part p2 on p1.p_partkey = p2.p_partkey + right outer join part p3 on p2.p_partkey = p3.p_partkey and + p1.p_size > 10 and p1.p_size > p2.p_size + 10 +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q index 4ff60713d6b44..047a769eb8e95 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q @@ -1,5 +1,5 @@ CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in1.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' INTO TABLE myinput1; SELECT * FROM myinput1 a JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; SELECT * FROM myinput1 a LEFT OUTER JOIN myinput1 b ORDER BY a.key ASC, a.value ASC, b.key ASC, b.value ASC; @@ -42,10 +42,10 @@ SELECT /*+ MAPJOIN(a) */ * FROM myinput1 a RIGHT OUTER JOIN myinput1 b ON a.valu CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in1.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in2.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in1.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in2.txt' into table smb_input2; SET hive.optimize.bucketmapJOIN = true; SET hive.optimize.bucketmapJOIN.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q index 05b57bce202d2..5e22517edbd72 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q @@ -1,7 +1,7 @@ set hive.nullsafe.equijoin=true; CREATE TABLE myinput1(key int, value int); -LOAD DATA LOCAL INPATH '../data/files/in8.txt' INTO TABLE myinput1; +LOAD DATA LOCAL INPATH '../../data/files/in8.txt' INTO TABLE myinput1; -- merging explain select * from myinput1 a join myinput1 b on a.key<=>b.value ORDER BY a.key, a.value, b.key, b.value; @@ -31,10 +31,10 @@ SELECT /*+ MAPJOIN(b) */ * FROM myinput1 a JOIN myinput1 b ON a.key<=>b.value OR -- smbs CREATE TABLE smb_input1(key int, value int) CLUSTERED BY (key) SORTED BY (key) INTO 2 BUCKETS; CREATE TABLE smb_input2(key int, value int) CLUSTERED BY (value) SORTED BY (value) INTO 2 BUCKETS; -LOAD DATA LOCAL INPATH '../data/files/in8.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in9.txt' into table smb_input1; -LOAD DATA LOCAL INPATH '../data/files/in8.txt' into table smb_input2; -LOAD DATA LOCAL INPATH '../data/files/in9.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in8.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in9.txt' into table smb_input1; +LOAD DATA LOCAL INPATH '../../data/files/in8.txt' into table smb_input2; +LOAD DATA LOCAL INPATH '../../data/files/in9.txt' into table smb_input2; SET hive.optimize.bucketmapJOIN = true; SET hive.optimize.bucketmapJOIN.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q index b92a79ba07ab2..b209c50b66194 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q @@ -6,9 +6,9 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; EXPLAIN FROM T1 a JOIN src c ON c.key+1=a.key SELECT a.key, a.val, c.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q index 238c0adad3128..ca1e65ebef6f0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q @@ -8,10 +8,10 @@ CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4; EXPLAIN SELECT /*+ STREAMTABLE(a) */ * diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q index 1bda28fbc3d5f..994be164aa62c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q @@ -8,10 +8,10 @@ CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4; EXPLAIN SELECT /*+ STREAMTABLE(a,c) */ * diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q index 126f356ef785a..16ef2046be35a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q @@ -2,9 +2,9 @@ CREATE TABLE T1(key1 STRING, val1 STRING) STORED AS TEXTFILE; CREATE TABLE T2(key2 STRING, val2 STRING) STORED AS TEXTFILE; CREATE TABLE T3(key3 STRING, val3 STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; set hive.auto.convert.join=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q index 8314161975761..c95a13b9cac5a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q @@ -7,14 +7,14 @@ create table dim5(f9 int, f10 int); create table dim6(f11 int, f12 int); create table dim7(f13 int, f14 int); -LOAD DATA LOCAL INPATH '../data/files/fact-data.txt' INTO TABLE fact; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim1; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim2; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim3; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim4; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim5; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim6; -LOAD DATA LOCAL INPATH '../data/files/dim-data.txt' INTO TABLE dim7; +LOAD DATA LOCAL INPATH '../../data/files/fact-data.txt' INTO TABLE fact; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim1; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim2; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim3; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim4; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim5; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim6; +LOAD DATA LOCAL INPATH '../../data/files/dim-data.txt' INTO TABLE dim7; set hive.auto.convert.join=true; set hive.auto.convert.join.noconditionaltask=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q index 11e54a5b04c07..df7343259ae75 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + --HIVE-2608 Do not require AS a,b,c part in LATERAL VIEW EXPLAIN SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2; SELECT myTab.* from src LATERAL VIEW explode(map('key1', 100, 'key2', 200)) myTab limit 2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q index 7be86a6f10ea2..65ae518cd5be2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q @@ -11,3 +11,7 @@ SELECT value, myCol FROM (SELECT * FROM srcpart LATERAL VIEW explode(array(1,2,3 EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE key='0'; SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol LATERAL VIEW explode(array(1,2,3)) myTable2 AS myCol2) a WHERE key='0'; + +-- HIVE-4293 Predicates following UDTF operator are removed by PPD +EXPLAIN SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0'; +SELECT value, myCol FROM (SELECT * FROM src LATERAL VIEW explode(array(1,2,3)) myTable AS myCol WHERE myCol > 1) a WHERE key='0'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q new file mode 100644 index 0000000000000..7f31797f314b3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lb_fs_stats.q @@ -0,0 +1,19 @@ +set hive.mapred.supports.subdirectories=true; +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +set hive.merge.mapfiles=false; +set hive.merge.mapredfiles=false; +set mapred.input.dir.recursive=true; +set hive.stats.dbclass=fs; +-- Tests truncating a column from a list bucketing table + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +CREATE TABLE test_tab (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; + +ALTER TABLE test_tab SKEWED BY (key) ON ("484") STORED AS DIRECTORIES; + +INSERT OVERWRITE TABLE test_tab PARTITION (part = '1') SELECT * FROM src; + +describe formatted test_tab partition (part='1'); + +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q index f49766771067b..5623cbfac51a7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; --1. testLagWithPTFWindowing select p_mfgr, p_name, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q index 6ef3bdb107a12..e53abce763865 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q @@ -11,7 +11,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testLeadUDAF select p_mfgr, p_retailprice, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q index abe3d3317157c..0c16fb8dfd988 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q @@ -9,9 +9,9 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; CREATE TABLE things (id INT, name STRING) partitioned by (ds string) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; -load data local inpath '../data/files/sales.txt' INTO TABLE sales; -load data local inpath '../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23'); -load data local inpath '../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24'); +load data local inpath '../../data/files/sales.txt' INTO TABLE sales; +load data local inpath '../../data/files/things.txt' INTO TABLE things partition(ds='2011-10-23'); +load data local inpath '../../data/files/things2.txt' INTO TABLE things partition(ds='2011-10-24'); SELECT name,id FROM sales ORDER BY name ASC, id ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q index 5813ca3c619cc..c9ebe0e8fad12 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q @@ -1,7 +1,7 @@ CREATE TABLE T1(key INT); -LOAD DATA LOCAL INPATH '../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t1.txt' INTO TABLE T1; CREATE TABLE T2(key INT); -LOAD DATA LOCAL INPATH '../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/leftsemijoin_mr_t2.txt' INTO TABLE T2; -- Run this query using TestMinimrCliDriver diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q new file mode 100644 index 0000000000000..e91adab59ddfa --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_partition_metadataonly.q @@ -0,0 +1,7 @@ +set hive.limit.query.max.table.partition=1; + +explain select ds from srcpart where hr=11 and ds='2008-04-08'; +select ds from srcpart where hr=11 and ds='2008-04-08'; + +explain select distinct hr from srcpart; +select distinct hr from srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q index e4d0aa06bde87..adfe1e63e889f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q @@ -22,12 +22,17 @@ select value,avg(key + 1) from src group by value order by value limit 20; -- distincts explain -select distinct(key) from src limit 20; -select distinct(key) from src limit 20; +select distinct(cdouble) from alltypesorc limit 20; +select distinct(cdouble) from alltypesorc limit 20; explain -select key, count(distinct(key)) from src group by key limit 20; -select key, count(distinct(key)) from src group by key limit 20; +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; + +-- multi distinct +explain +select ctinyint, count(distinct(cstring1)), count(distinct(cstring2)) from alltypesorc group by ctinyint limit 20; +select ctinyint, count(distinct(cstring1)), count(distinct(cstring2)) from alltypesorc group by ctinyint limit 20; -- limit zero explain diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q index a86ddf14047ef..e17ded1ee1b9b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q @@ -16,7 +16,3 @@ CREATE TABLE dest_3(key STRING, c1 INT); EXPLAIN FROM src INSERT OVERWRITE TABLE dest_2 SELECT value, sum(key) GROUP BY value INSERT OVERWRITE TABLE dest_3 SELECT value, sum(key) GROUP BY value limit 20; - --- nagative, multi distinct -explain -select count(distinct key)+count(distinct value) from src limit 20; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q index 3a39f42b2a77d..80aba5d4a526f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q @@ -48,13 +48,13 @@ set hive.optimize.listbucketing=true; explain extended select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -- 51 and val_51 in the table so skewed data for 51 and val_14 should be none -- but query should succeed for 51 or 51 and val_14 -select * from srcpart where ds = '2008-04-08' and key = '51' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '51' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where key = '51' ORDER BY key, value, ds, hr; -select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '51' and value = 'val_14' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where key = '51' and value = 'val_14' ORDER BY key, value, ds, hr; -- queries with < <= > >= should work for skewed test although we don't benefit from pruning diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q index 918c817e49eff..380d148ac9f43 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q @@ -65,7 +65,7 @@ set hive.optimize.listbucketing=true; explain extended select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; select * from list_bucketing_static_part where ds = '2008-04-08' and hr = '11' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value; +select * from srcpart where ds = '2008-04-08' and key = '484' and value = 'val_484' ORDER BY key, value, ds, hr; -- clean up drop table list_bucketing_static_part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q index a6ad4b8485a26..08b21dc689424 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT -1BD, 0BD, 1BD, 3.14BD, -3.14BD, 99999999999999999BD, 99999999999999999.9999999999999BD, 1E-99BD, 1E99BD FROM src LIMIT 1; SELECT -1BD, 0BD, 1BD, 3.14BD, -3.14BD, 99999999999999999BD, 99999999999999999.9999999999999BD, 1E-99BD, 1E99BD FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q index 08836127b9958..766da699ea250 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 FROM src LIMIT 1; SELECT 3.14, -3.14, 3.14e8, 3.14e-8, -3.14e8, -3.14e-8, 3.14e+8, 3.14E8, 3.14E-8 FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q index 9da622e246793..5fd0cfabf3076 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1; SELECT 100, 100Y, 100S, 100L FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q index 21f0890ada168..c57dc572529e1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 'face''book', 'face' 'book', 'face' 'book', "face""book", "face" "book", "face" diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q index 7da363183ccf6..653918afc0fa3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q @@ -4,7 +4,7 @@ FIELDS TERMINATED BY '9' STORED AS TEXTFILE; -- this query loads native binary data, stores in a table and then queries it. Note that string.txt contains binary data. Also uses transform clause and then length udf. -LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE mytable; +LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE mytable; create table dest1 (key binary, value int); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q index 52b4937d4a9dd..5f0a015693d9f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q @@ -23,8 +23,8 @@ insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, v show partitions nzhang_part1; show partitions nzhang_part2; -select * from nzhang_part1 where ds is not null and hr is not null; -select * from nzhang_part2 where ds is not null and hr is not null; +select * from nzhang_part1 where ds is not null and hr is not null order by ds, hr, key; +select * from nzhang_part2 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q index 9517664675d69..dd84599e69e6d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q @@ -19,6 +19,6 @@ insert overwrite table nzhang_part10 partition(ds='2008-12-31', hr) select key, show partitions nzhang_part10; -select * from nzhang_part10 where ds is not null and hr is not null; +select * from nzhang_part10 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q index e4c8c17f63048..29f951aa69f67 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q @@ -14,6 +14,6 @@ insert overwrite table nzhang_part3 partition (ds, hr) select key, value, ds, hr insert overwrite table nzhang_part3 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null; -select * from nzhang_part3 where ds is not null and hr is not null; +select * from nzhang_part3 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q index 3f3a0c8d51b41..942c245db8b64 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q @@ -17,8 +17,8 @@ insert overwrite table nzhang_part4 partition (ds, hr) select key, value, ds, hr insert overwrite table nzhang_part4 partition (ds, hr) select key, value, ds, hr from srcpart where ds is not null and hr is not null; show partitions nzhang_part4; -select * from nzhang_part4 where ds='2008-04-08' and hr is not null; +select * from nzhang_part4 where ds='2008-04-08' and hr is not null order by ds, hr, key; -select * from nzhang_part4 where ds is not null and hr is not null; +select * from nzhang_part4 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q index 8073500c0bf07..6768e4373a056 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q @@ -20,5 +20,5 @@ insert overwrite table nzhang_part8 partition(ds='2008-12-31', hr) select key, v show partitions nzhang_part8; -select * from nzhang_part8 where ds is not null and hr is not null; +select * from nzhang_part8 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q index 01fa596cdf04a..4680033cbd541 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q @@ -19,5 +19,5 @@ insert overwrite table nzhang_part9 partition (ds, hr) select key, value, ds, hr show partitions nzhang_part9; -select * from nzhang_part9 where ds is not null and hr is not null; +select * from nzhang_part9 where ds is not null and hr is not null order by ds, hr, key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q index 6d2a8b82d33c1..35eb2198081f0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q @@ -2,4 +2,4 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored alter table hive_test_src add partition (pcol1 = 'test_part'); set hive.security.authorization.enabled=true; grant Update on table hive_test_src to user hive_test_user; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q index 3b8951a1a782f..6bac47fb9052a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q @@ -2,4 +2,5 @@ CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT); -LOAD DATA LOCAL INPATH '../data/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA LOCAL INPATH '../../data/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA LOCAL INPATH '../../data/files/person+age.txt' INTO TABLE load_file_with_space_in_the_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q index c1ac29c172f60..2f06ca464ff99 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q @@ -2,9 +2,9 @@ create table load_overwrite (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/load_overwrite'; create table load_overwrite2 (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/load2_overwrite2'; -load data local inpath '../data/files/kv1.txt' into table load_overwrite; -load data local inpath '../data/files/kv2.txt' into table load_overwrite; -load data local inpath '../data/files/kv3.txt' into table load_overwrite; +load data local inpath '../../data/files/kv1.txt' into table load_overwrite; +load data local inpath '../../data/files/kv2.txt' into table load_overwrite; +load data local inpath '../../data/files/kv3.txt' into table load_overwrite; show table extended like load_overwrite; desc extended load_overwrite; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q index 7255324d1653d..a75758a0728d5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q @@ -4,17 +4,17 @@ create table result (key string, value string); create table loader (key string, value string); -load data local inpath '../data/files/kv1.txt' into table loader; +load data local inpath '../../data/files/kv1.txt' into table loader; load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result; show table extended like result; -load data local inpath '../data/files/kv1.txt' into table loader; +load data local inpath '../../data/files/kv1.txt' into table loader; load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result; show table extended like result; -load data local inpath '../data/files/kv1.txt' into table loader; +load data local inpath '../../data/files/kv1.txt' into table loader; load data inpath '/build/ql/test/data/warehouse/loader/kv1.txt' into table result; show table extended like result; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q new file mode 100644 index 0000000000000..51a803130a660 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs_overwrite.q @@ -0,0 +1,20 @@ +--HIVE 6209 + +drop table target; +drop table temp; + +create table target (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/target'; +create table temp (key string, value string) stored as textfile location 'file:${system:test.tmp.dir}/temp'; + +set fs.pfile.impl.disable.cache=false; + +load data local inpath '../../data/files/kv1.txt' into table temp; +load data inpath '${system:test.tmp.dir}/temp/kv1.txt' overwrite into table target; +select count(*) from target; + +load data local inpath '../../data/files/kv2.txt' into table temp; +load data inpath '${system:test.tmp.dir}/temp/kv2.txt' overwrite into table target; +select count(*) from target; + +drop table target; +drop table temp; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q index cce297cca46db..55ac1a8185a46 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q @@ -1,9 +1,10 @@ -dfs -mkdir hdfs:///tmp/test/; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/; -dfs -copyFromLocal ../data/files hdfs:///tmp/test/.; +dfs -copyFromLocal ../../data/files hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/.; CREATE TABLE load_file_with_space_in_the_name(name STRING, age INT); -LOAD DATA INPATH 'hdfs:///tmp/test/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA INPATH 'hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/files/person age.txt' INTO TABLE load_file_with_space_in_the_name; +LOAD DATA INPATH 'hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name/files/person+age.txt' INTO TABLE load_file_with_space_in_the_name; -dfs -rmr hdfs:///tmp/test; +dfs -rmr hdfs:///tmp/test_load_hdfs_file_with_space_in_the_name; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q index 40d8210e57d2a..fdee45114bb15 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q @@ -1,4 +1,4 @@ create table hive_test_src ( col1 string ) stored as textfile; set hive.security.authorization.enabled=true; grant Update on table hive_test_src to user hive_test_user; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q index 73853f15a94bb..080c78496a653 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q @@ -5,11 +5,11 @@ show table extended like load_overwrite; select count(*) from load_overwrite; -load data local inpath '../data/files/kv1.txt' into table load_overwrite; +load data local inpath '../../data/files/kv1.txt' into table load_overwrite; show table extended like load_overwrite; select count(*) from load_overwrite; -load data local inpath '../data/files/kv1.txt' overwrite into table load_overwrite; +load data local inpath '../../data/files/kv1.txt' overwrite into table load_overwrite; show table extended like load_overwrite; select count(*) from load_overwrite; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q index ff54324a5a50c..cee5873ca5b9f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q @@ -1,4 +1,4 @@ create table hive_test_src ( col1 string ) partitioned by (pcol1 string) stored as textfile; set hive.security.authorization.enabled=true; grant Update on table hive_test_src to user hive_test_user; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src partition (pcol1 = 'test_part'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q index 0813bb23c3746..735befef6f9c9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q @@ -2,7 +2,7 @@ create table hive_test_src ( col1 string ) stored as textfile ; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; create table hive_test_dst ( col1 string ) partitioned by ( pcol1 string , pcol2 string) stored as sequencefile; insert overwrite table hive_test_dst partition ( pcol1='test_part', pCol2='test_Part') select col1 from hive_test_src ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q new file mode 100644 index 0000000000000..a252eaa00d77f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart2.q @@ -0,0 +1,9 @@ + +create table hive_test ( col1 string ) partitioned by ( pcol1 string , pcol2 string) stored as textfile; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test partition (pcol1='part1',pcol2='part1') ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test partition (pcol2='part2',pcol1='part2') ; +select * from hive_test where pcol1='part1' and pcol2='part1'; +select * from hive_test where pcol1='part2' and pcol2='part2'; + + + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q index 6e4df215479bb..cc9c1fec3bc42 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q @@ -1,6 +1,6 @@ set hive.cli.errors.ignore=true; -ADD FILE ../data/scripts/error_script; +ADD FILE ../../data/scripts/error_script; -- EXCLUDE_HADOOP_MAJOR_VERSIONS(0.17, 0.18, 0.19, 0.20, 0.20S, 0.23) -- (this test is flaky so it is currently disabled for all Hadoop versions) @@ -14,7 +14,7 @@ FROM src; DESCRIBE loadpart1; SHOW PARTITIONS loadpart1; -LOAD DATA LOCAL INPATH '../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); +LOAD DATA LOCAL INPATH '../../data1/files/kv1.txt' INTO TABLE loadpart1 PARTITION(ds='2009-05-05'); SHOW PARTITIONS loadpart1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q index fd0f7f2b0cdd4..47b05ff4490fb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + CREATE TEMPORARY MACRO SIGMOID (x DOUBLE) 1.0 / (1.0 + EXP(-x)); SELECT SIGMOID(2) FROM src LIMIT 1; EXPLAIN SELECT SIGMOID(2) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q new file mode 100644 index 0000000000000..f56f074616678 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_addjar.q @@ -0,0 +1,14 @@ + +set hive.auto.convert.join=true; +set hive.auto.convert.join.use.nonstaged=false; + +add jar ${system:maven.local.repository}/org/apache/hive/hcatalog/hive-hcatalog-core/${system:hive.version}/hive-hcatalog-core-${system:hive.version}.jar; + +CREATE TABLE t1 (a string, b string) +ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' +; +LOAD DATA LOCAL INPATH "../../data/files/sample.json" INTO TABLE t1; +select * from src join t1 on src.key =t1.a; +drop table t1; +set hive.auto.convert.join=false; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q new file mode 100644 index 0000000000000..b65a7be2d25cd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_decimal.q @@ -0,0 +1,35 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask=true; +set hive.auto.convert.join.noconditionaltask.size=10000000; + +CREATE TABLE over1k(t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal(4,2), + bin binary) +ROW FORMAT DELIMITED FIELDS TERMINATED BY '|' +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE over1k; + +CREATE TABLE t1(dec decimal(4,2)) STORED AS ORC; +INSERT INTO TABLE t1 select dec from over1k; +CREATE TABLE t2(dec decimal(4,0)) STORED AS ORC; +INSERT INTO TABLE t2 select dec from over1k; + +explain +select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec); + +set hive.mapjoin.optimized.keys=false; + +select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec); + +set hive.mapjoin.optimized.keys=true; + +select t1.dec, t2.dec from t1 join t2 on (t1.dec=t2.dec); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q index d6811d493263f..a9e1960a5bb05 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q @@ -1,4 +1,5 @@ -set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.MapJoinCounterHook ; +set hive.exec.post.hooks = org.apache.hadoop.hive.ql.hooks.MapJoinCounterHook,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + drop table dest1; CREATE TABLE dest1(key INT, value STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q index 3f87db28ed2e2..1eb95f6378669 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q @@ -6,6 +6,14 @@ set hive.auto.convert.join.noconditionaltask.size=10000; explain select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key); -explain select count(*) from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) group by ds; +explain +select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'; +select srcpart.key from srcpart join src on (srcpart.value=src.value) join src1 on (srcpart.key=src1.key) where srcpart.value > 'val_450'; + +explain +select count(*) from srcpart join src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds; +select count(*) from srcpart join src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds; + +set hive.mapjoin.lazy.hashtable=false; select count(*) from srcpart join src src on (srcpart.value=src.value) join src src1 on (srcpart.key=src1.key) group by ds; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q new file mode 100644 index 0000000000000..b23361724e669 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_memcheck.q @@ -0,0 +1,16 @@ + +set hive.auto.convert.join = true; + +create table src0 like src; +insert into table src0 select * from src where src.key < 10; + +set hive.mapjoin.check.memory.rows=1; + +explain +select src1.key as k1, src1.value as v1, src2.key, src2.value +from src0 src1 inner join src0 src2 on src1.key = src2.key order by k1, v1; + +select src1.key as k1, src1.value as v1, src2.key, src2.value +from src0 src1 inner join src0 src2 on src1.key = src2.key order by k1, v1; + +drop table src0; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q index 9980946057fe1..aed89905238b2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q @@ -11,9 +11,9 @@ ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; CREATE TABLE z (id INT, name STRING) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t'; -load data local inpath '../data/files/x.txt' INTO TABLE x; -load data local inpath '../data/files/y.txt' INTO TABLE y; -load data local inpath '../data/files/z.txt' INTO TABLE z; +load data local inpath '../../data/files/x.txt' INTO TABLE x; +load data local inpath '../../data/files/y.txt' INTO TABLE y; +load data local inpath '../../data/files/z.txt' INTO TABLE z; set hive.auto.convert.join=true; set hive.auto.convert.join.noconditionaltask=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q index aacd0cd68fc50..c5c7ea202fa76 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q @@ -37,7 +37,7 @@ where ds is not null; show partitions merge_src_part2; -select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC; +select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC, ds ASC; drop table merge_src_part2; @@ -54,4 +54,4 @@ select key, value, ds; show partitions merge_src_part2; -select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC; +select * from merge_src_part2 where ds is not null ORDER BY key ASC, value ASC, ds ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q index 744783bd62f8d..5a167aa0f08b3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q @@ -11,14 +11,14 @@ insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, v insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08'; -select * from nzhang_part; +select * from nzhang_part ORDER BY key, value, ds, hr; explain insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08'; insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08'; -select * from nzhang_part; +select * from nzhang_part ORDER BY key, value, ds, hr; explain insert overwrite table nzhang_part partition (ds='2010-08-15', hr) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q index 1379426b601f0..ae319865f6e7b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q @@ -5,15 +5,16 @@ create table srcpart_merge_dp like srcpart; create table merge_dynamic_part like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); set hive.input.format=org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; set hive.merge.mapfiles=false; set hive.merge.mapredfiles=false; set hive.merge.smallfiles.avgsize=1000000000; +set hive.optimize.sort.dynamic.partition=false; explain insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08'; insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q index b51c70ed03a46..73a71e6265d50 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q @@ -5,12 +5,12 @@ create table srcpart_merge_dp like srcpart; create table merge_dynamic_part like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket0.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket0.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; @@ -18,6 +18,7 @@ set hive.merge.mapfiles=true; set hive.merge.mapredfiles=true; set hive.merge.smallfiles.avgsize=3000; set hive.exec.compress.output=false; +set hive.optimize.sort.dynamic.partition=false; explain insert overwrite table merge_dynamic_part partition (ds='2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds='2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q index b3bcf01ea043d..43be59e5d5f40 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q @@ -5,20 +5,20 @@ create table srcpart_merge_dp like srcpart; create table merge_dynamic_part like srcpart; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); - -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); - -load data local inpath '../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); -load data local inpath '../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); -load data local inpath '../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); -load data local inpath '../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); + +load data local inpath '../../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); +load data local inpath '../../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=11); +load data local inpath '../../data/files/kv1.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); +load data local inpath '../../data/files/kv2.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-09', hr=12); show partitions srcpart_merge_dp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q index ef769a042d7ce..589717096a9d9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q @@ -9,15 +9,15 @@ alter table srcpart_merge_dp_rc set fileformat RCFILE; create table merge_dynamic_part like srcpart; alter table merge_dynamic_part set fileformat RCFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); - -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); insert overwrite table srcpart_merge_dp_rc partition (ds = '2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q index a196fa05288b7..9f64724563865 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q @@ -8,12 +8,12 @@ alter table srcpart_merge_dp_rc set fileformat RCFILE; create table merge_dynamic_part like srcpart; alter table merge_dynamic_part set fileformat RCFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=11); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcpart_merge_dp partition(ds='2008-04-08', hr=12); insert overwrite table srcpart_merge_dp_rc partition (ds = '2008-04-08', hr) select key, value, hr from srcpart_merge_dp where ds = '2008-04-08'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q index 41be152e7871d..e2da61a783950 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q @@ -1,8 +1,8 @@ create table tmp_meta_export_listener_drop_test (foo string); -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/exports/HIVE-3427; +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/data/exports/HIVE-3427; set hive.metastore.pre.event.listeners=org.apache.hadoop.hive.ql.parse.MetaDataExportListener; -set hive.metadata.export.location=../build/ql/test/data/exports/HIVE-3427; +set hive.metadata.export.location=${system:test.tmp.dir}/data/exports/HIVE-3427; set hive.move.exported.metadata.to.trash=false; drop table tmp_meta_export_listener_drop_test; -dfs -rmr ../build/ql/test/data/exports/HIVE-3427; +dfs -rmr ${system:test.tmp.dir}/data/exports/HIVE-3427; set hive.metastore.pre.event.listeners=; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q new file mode 100644 index 0000000000000..b549a56232108 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries.q @@ -0,0 +1,77 @@ +set hive.stats.dbclass=fs; +set hive.compute.query.using.stats=true; +set hive.stats.autogather=true; +create table over10k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over10k' into table over10k; + +create table stats_tbl( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary); + +create table stats_tbl_part( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) partitioned by (dt string); + + +insert overwrite table stats_tbl select * from over10k; + +insert into table stats_tbl_part partition (dt='2010') select * from over10k where t>0 and t<30; +insert into table stats_tbl_part partition (dt='2011') select * from over10k where t>30 and t<60; +insert into table stats_tbl_part partition (dt='2012') select * from over10k where t>60; + +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b) from stats_tbl; +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b) from stats_tbl_part; + +analyze table stats_tbl compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt='2010') compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt='2011') compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt='2012') compute statistics for columns t,si,i,b,f,d,bo,s,bin; + +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl; +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl; +explain +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part; +select count(*), sum(1), sum(0.2), count(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part; + +explain select count(ts) from stats_tbl_part; + +drop table stats_tbl; +drop table stats_tbl_part; + +set hive.compute.query.using.stats=false; +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q new file mode 100644 index 0000000000000..09f4bff616a50 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_only_queries_with_filters.q @@ -0,0 +1,51 @@ +set hive.stats.dbclass=fs; +set hive.compute.query.using.stats=true; +create table over10k( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) + row format delimited + fields terminated by '|'; + +load data local inpath '../../data/files/over10k' into table over10k; + +create table stats_tbl_part( + t tinyint, + si smallint, + i int, + b bigint, + f float, + d double, + bo boolean, + s string, + ts timestamp, + dec decimal, + bin binary) partitioned by (dt int); + + +from over10k +insert overwrite table stats_tbl_part partition (dt=2010) select t,si,i,b,f,d,bo,s,ts,dec,bin where t>0 and t<30 +insert overwrite table stats_tbl_part partition (dt=2014) select t,si,i,b,f,d,bo,s,ts,dec,bin where t > 30 and t<60; + +analyze table stats_tbl_part partition(dt) compute statistics; +analyze table stats_tbl_part partition(dt=2010) compute statistics for columns t,si,i,b,f,d,bo,s,bin; +analyze table stats_tbl_part partition(dt=2014) compute statistics for columns t,si,i,b,f,d,bo,s,bin; + +explain +select count(*), count(1), sum(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt = 2010; +select count(*), count(1), sum(1), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt = 2010; +explain +select count(*), count(1), sum(1), sum(2), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt > 2010; +select count(*), count(1), sum(1), sum(2), count(s), count(bo), count(bin), count(si), max(i), min(b), max(f), min(d) from stats_tbl_part where dt > 2010; + +drop table stats_tbl_part; +set hive.compute.query.using.stats=false; +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q index 067c143c0e684..2a6059b3a37ff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q @@ -15,7 +15,7 @@ GROUP BY key, value, ds, hr; show partitions nzhang_t1; show partitions nzhang_t2; -select * from nzhang_t1; -select * from nzhang_t2; +select * from nzhang_t1 order by key, value; +select * from nzhang_t2 order by key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q new file mode 100644 index 0000000000000..9f068cc71394e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mrr.q @@ -0,0 +1,59 @@ +-- simple query with multiple reduce stages +EXPLAIN SELECT key, count(value) as cnt FROM src GROUP BY key ORDER BY cnt; +SELECT key, count(value) as cnt FROM src GROUP BY key ORDER BY cnt; + +set hive.auto.convert.join=false; +-- join query with multiple reduce stages; +EXPLAIN SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; +SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; + +set hive.auto.convert.join=true; +-- same query with broadcast join +EXPLAIN SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; +SELECT s2.key, count(distinct s2.value) as cnt FROM src s1 join src s2 on (s1.key = s2.key) GROUP BY s2.key ORDER BY cnt; + +set hive.auto.convert.join=false; +-- query with multiple branches in the task dag +EXPLAIN +SELECT * +FROM + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s1 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s2 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s3 + ON (s1.key = s2.key and s1.key = s3.key) +WHERE + s1.cnt > 1 +ORDER BY s1.key; + +SELECT * +FROM + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s1 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s2 + JOIN + (SELECT key, count(value) as cnt + FROM src GROUP BY key ORDER BY cnt) s3 + ON (s1.key = s2.key and s1.key = s3.key) +WHERE + s1.cnt > 1 +ORDER BY s1.key; + +set hive.auto.convert.join=true; +-- query with broadcast join in the reduce stage +EXPLAIN +SELECT * +FROM + (SELECT key, count(value) as cnt FROM src GROUP BY key) s1 + JOIN src ON (s1.key = src.key); + +SELECT * +FROM + (SELECT key, count(value) as cnt FROM src GROUP BY key) s1 + JOIN src ON (s1.key = src.key); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q index 9a0a792a91897..455f550ae3ac5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q @@ -1,3 +1,5 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + create table smallTbl1(key string, value string); insert overwrite table smallTbl1 select * from src where key < 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q index ce6cf6d8d6c0d..141db4db0a3f3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q @@ -1,3 +1,4 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; set hive.auto.convert.join=true; set hive.auto.convert.join.noconditionaltask=true; set hive.auto.convert.join.noconditionaltask.size=6000; @@ -187,3 +188,28 @@ FROM part_table x JOIN src1 y ON (x.key = y.key); SELECT count(*) FROM part_table x JOIN src1 y ON (x.key = y.key); +set hive.auto.convert.join.noconditionaltask.size=10000000; +set hive.optimize.correlation=false; +-- HIVE-5891 Alias conflict when merging multiple mapjoin tasks into their common +-- child mapred task +EXPLAIN +SELECT * FROM ( + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key + UNION ALL + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key +) x; + +SELECT * FROM ( + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key + UNION ALL + SELECT c.key FROM + (SELECT a.key FROM src a JOIN src b ON a.key=b.key GROUP BY a.key) tmp + JOIN src c ON tmp.key=c.key +) x; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q index b94fbb7b8be43..6fd76b859e4e4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q @@ -17,6 +17,6 @@ describe nestedcomplex; describe extended nestedcomplex; -load data local inpath '../data/files/nested_complex.txt' overwrite into table nestedcomplex; +load data local inpath '../../data/files/nested_complex.txt' overwrite into table nestedcomplex; select * from nestedcomplex sort by simple_int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q index 722ecf6d972ca..11168fcd3b05c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q @@ -1,4 +1,4 @@ -add file ../data/scripts/newline.py; +add file ../../data/scripts/newline.py; set hive.transform.escape.input=true; create table tmp_tmp(key string, value string) stored as rcfile; @@ -10,10 +10,10 @@ select * from tmp_tmp ORDER BY key ASC, value ASC; drop table tmp_tmp; -add file ../data/scripts/escapednewline.py; -add file ../data/scripts/escapedtab.py; -add file ../data/scripts/doubleescapedtab.py; -add file ../data/scripts/escapedcarriagereturn.py; +add file ../../data/scripts/escapednewline.py; +add file ../../data/scripts/escapedtab.py; +add file ../../data/scripts/doubleescapedtab.py; +add file ../../data/scripts/escapedcarriagereturn.py; create table tmp_tmp(key string, value string) stored as rcfile; insert overwrite table tmp_tmp diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q new file mode 100644 index 0000000000000..e6343e2f535c0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch_threshold.q @@ -0,0 +1,9 @@ +set hive.fetch.task.conversion=more; + +explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; +explain select cast(key as int) * 10, upper(value) from src limit 10; + +set hive.fetch.task.conversion.threshold=100; + +explain select * from srcpart where ds='2008-04-08' AND hr='11' limit 10; +explain select cast(key as int) * 10, upper(value) from src limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q index 9cb89da373d29..e33b4bfcab7ba 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q @@ -1,12 +1,12 @@ CREATE TABLE table(string string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/docurl.txt' INTO TABLE table; +LOAD DATA LOCAL INPATH '../../data/files/docurl.txt' INTO TABLE table; SELECT table, count(1) FROM ( FROM table SELECT TRANSFORM (table.string) - USING 'java -cp ../build/ql/test/classes org.apache.hadoop.hive.scripts.extracturl' AS (table, count) + USING 'java -cp ../util/target/classes/ org.apache.hadoop.hive.scripts.extracturl' AS (table, count) ) subq GROUP BY table; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q new file mode 100644 index 0000000000000..aa79674409d2d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias3.q @@ -0,0 +1,4 @@ +CREATE TABLE dest1(c string, key INT, value DOUBLE) STORED AS TEXTFILE; + +FROM src +INSERT OVERWRITE TABLE dest1 SELECT '1234', src.key, sum(src.value) WHERE src.key < 100 group by key; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q index 48c39b81fdd8f..bd0cb8d12d88f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q @@ -2,10 +2,10 @@ EXPLAIN SELECT ARRAY(NULL, 0), ARRAY(NULL, ARRAY()), ARRAY(NULL, MAP()), ARRAY(NULL, STRUCT(0)) - FROM src LIMIT 1; + FROM src tablesample (1 rows); SELECT ARRAY(NULL, 0), ARRAY(NULL, ARRAY()), ARRAY(NULL, MAP()), ARRAY(NULL, STRUCT(0)) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q index fa4a8639446e5..4b43d608e86df 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q @@ -3,7 +3,7 @@ create table temp_null(a int) stored as textfile; -load data local inpath '../data/files/test.dat' overwrite into table temp_null; +load data local inpath '../../data/files/test.dat' overwrite into table temp_null; select null, null from temp_null; @@ -15,8 +15,8 @@ create table tt_b(a int, b string) row format serde "org.apache.hadoop.hive.serd insert overwrite table tt_b select null, null from temp_null; select * from tt_b; -insert overwrite directory "../build/ql/test/data/warehouse/null_columns.out" select null, null from temp_null; -dfs -cat ../build/ql/test/data/warehouse/null_columns.out/*; +insert overwrite directory "target/warehouse/null_columns.out" select null, null from temp_null; +dfs -cat ${system:test.warehouse.dir}/null_columns.out/*; create table temp_null2 (key string, value string) partitioned by (ds string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q new file mode 100644 index 0000000000000..c9a7dab5eb9e7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformat.q @@ -0,0 +1,24 @@ +-- base table with null data +DROP TABLE IF EXISTS base_tab; +CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab; +DESCRIBE EXTENDED base_tab; + +-- table with non-default null format +DROP TABLE IF EXISTS null_tab1; +EXPLAIN CREATE TABLE null_tab1(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'; +CREATE TABLE null_tab1(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'; +DESCRIBE EXTENDED null_tab1; +SHOW CREATE TABLE null_tab1; + +-- load null data from another table and verify that the null is stored in the expected format +INSERT OVERWRITE TABLE null_tab1 SELECT a,b FROM base_tab; +dfs -cat ${system:test.warehouse.dir}/null_tab1/*; +SELECT * FROM null_tab1; +-- alter the null format and verify that the old null format is no longer in effect +ALTER TABLE null_tab1 SET SERDEPROPERTIES ( 'serialization.null.format'='foo'); +SELECT * FROM null_tab1; + + +DROP TABLE null_tab1; +DROP TABLE base_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q new file mode 100644 index 0000000000000..d077981d02ad0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatCTAS.q @@ -0,0 +1,24 @@ +-- base table with null data +DROP TABLE IF EXISTS base_tab; +CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab; +DESCRIBE EXTENDED base_tab; + +-- table with non-default null format +DROP TABLE IF EXISTS null_tab3; +EXPLAIN CREATE TABLE null_tab3 ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull' + AS SELECT a, b FROM base_tab; +CREATE TABLE null_tab3 ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull' + AS SELECT a, b FROM base_tab; +DESCRIBE EXTENDED null_tab3; +SHOW CREATE TABLE null_tab3; + +dfs -cat ${system:test.warehouse.dir}/null_tab3/*; +SELECT * FROM null_tab3; +-- alter the null format and verify that the old null format is no longer in effect +ALTER TABLE null_tab3 SET SERDEPROPERTIES ( 'serialization.null.format'='foo'); +SELECT * FROM null_tab3; + + +DROP TABLE null_tab3; +DROP TABLE base_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q new file mode 100644 index 0000000000000..d29863839f728 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullformatdir.q @@ -0,0 +1,21 @@ +-- base table with null data +DROP TABLE IF EXISTS base_tab; +CREATE TABLE base_tab(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE base_tab; +DESCRIBE EXTENDED base_tab; + +dfs ${system:test.dfs.mkdir} ${system:test.tmp.dir}/hive_test/nullformat/tmp; +dfs -rmr ${system:test.tmp.dir}/hive_test/nullformat/*; +INSERT OVERWRITE LOCAL DIRECTORY '${system:test.tmp.dir}/hive_test/nullformat' + ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull' SELECT a,b FROM base_tab; +dfs -cat ${system:test.tmp.dir}/hive_test/nullformat/000000_0; + +-- load the exported data back into a table with same null format and verify null values +DROP TABLE IF EXISTS null_tab2; +CREATE TABLE null_tab2(a STRING, b STRING) ROW FORMAT DELIMITED NULL DEFINED AS 'fooNull'; +LOAD DATA LOCAL INPATH '${system:test.tmp.dir}/hive_test/nullformat/000000_0' INTO TABLE null_tab2; +SELECT * FROM null_tab2; + + +dfs -rmr ${system:test.tmp.dir}/hive_test/nullformat; +DROP TABLE base_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q index a5bc9ff1f36ff..19e5b10786fc0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q @@ -1,28 +1,28 @@ CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl; select count(1) from tstparttbl; CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl2; select count(1) from tstparttbl2; DROP TABLE tstparttbl; CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl; select count(1) from tstparttbl; DROP TABLE tstparttbl2; CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE tstparttbl2 PARTITION (ds='2008-04-08'); explain select count(1) from tstparttbl2; select count(1) from tstparttbl2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q index 12773b6159a5d..b4b68fb8b693a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q @@ -1,10 +1,10 @@ CREATE TABLE tstparttbl(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl PARTITION (ds='2009-04-09'); CREATE TABLE tstparttbl2(KEY STRING, VALUE STRING) PARTITIONED BY(ds string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE tstparttbl2 PARTITION (ds='2009-04-09'); +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE tstparttbl2 PARTITION (ds='2009-04-09'); explain select u.* from diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q index 95c9e1df37679..11f4a7a78f813 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q @@ -1,7 +1,7 @@ CREATE TABLE nullscript(KEY STRING, VALUE STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE nullscript; -LOAD DATA LOCAL INPATH '../data/files/nullfile.txt' INTO TABLE nullscript; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE nullscript; +LOAD DATA LOCAL INPATH '../../data/files/nullfile.txt' INTO TABLE nullscript; explain select transform(key) using 'cat' as key1 from nullscript; select transform(key) using 'cat' as key1 from nullscript; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q index 7f858d3e6f136..d51c2107e1586 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT null + 7, 1.0 - null, null + null, CAST(21 AS BIGINT) % CAST(5 AS TINYINT), CAST(21 AS BIGINT) % CAST(21 AS BIGINT), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q index b685ae6c4e53f..ec9e8076fd533 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; select 1.0 < 2.0 from src limit 1; select 2.0 < 2.0 from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q deleted file mode 100644 index 5993041405ed8..0000000000000 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q +++ /dev/null @@ -1,6 +0,0 @@ -SET hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.OptrStatGroupByHook; -SET hive.exec.mode.local.auto=false; -SET hive.task.progress=true; --- This test executes the OptrStatGroupBy hook which prints the optr level --- stats of GROUPBY optr present is the plan of below query -SELECT count(1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q new file mode 100644 index 0000000000000..915f4f0d71e63 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_analyze.q @@ -0,0 +1,179 @@ +CREATE TABLE orc_create_people_staging ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp, + state string); + +LOAD DATA LOCAL INPATH '../../data/files/orc_create_people.txt' OVERWRITE INTO TABLE orc_create_people_staging; + +set hive.exec.dynamic.partition.mode=nonstrict; + +set hive.stats.autogather=false; +-- non-partitioned table +-- partial scan gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp, + state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people SELECT * FROM orc_create_people_staging ORDER BY id; + +set hive.stats.autogather = true; +analyze table orc_create_people compute statistics partialscan; + +desc formatted orc_create_people; + +drop table orc_create_people; + +-- auto stats gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp, + state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people SELECT * FROM orc_create_people_staging ORDER BY id; + +desc formatted orc_create_people; + +drop table orc_create_people; + +set hive.stats.autogather=false; +-- partitioned table +-- partial scan gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +set hive.stats.autogather = true; +analyze table orc_create_people partition(state) compute statistics partialscan; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +-- auto stats gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +set hive.stats.autogather=false; +-- partitioned and bucketed table +-- partial scan gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +clustered by (first_name) +sorted by (last_name) +into 4 buckets +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +set hive.stats.autogather = true; +analyze table orc_create_people partition(state) compute statistics partialscan; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +-- auto stats gather +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +clustered by (first_name) +sorted by (last_name) +into 4 buckets +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="Or"); + +drop table orc_create_people; + +set hive.stats.autogather=false; +-- create table with partitions containing text and ORC files. +-- ORC files implements StatsProvidingRecordReader but text files does not. +-- So the partition containing text file should not have statistics. +CREATE TABLE orc_create_people ( + id int, + first_name string, + last_name string, + address string, + salary decimal, + start_date timestamp) +PARTITIONED BY (state string) +STORED AS orc; + +INSERT OVERWRITE TABLE orc_create_people PARTITION (state) + SELECT * FROM orc_create_people_staging ORDER BY id; + +-- set the table to text format +ALTER TABLE orc_create_people SET SERDE 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; +ALTER TABLE orc_create_people SET FILEFORMAT TEXTFILE; + +-- load the text data into a new partition +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE orc_create_people PARTITION(state="OH"); + +-- set the table back to orc +ALTER TABLE orc_create_people SET SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde'; +ALTER TABLE orc_create_people SET FILEFORMAT ORC; + +set hive.stats.autogather = true; +analyze table orc_create_people partition(state) compute statistics noscan; + +desc formatted orc_create_people partition(state="Ca"); +desc formatted orc_create_people partition(state="OH"); + +drop table orc_create_people; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q index 6aca5486445c8..a82c1a55d5f6c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q @@ -40,7 +40,7 @@ set hive.default.fileformat=orc; CREATE TABLE orc_create (key INT, value STRING) PARTITIONED BY (ds string); -set hive.default.fileformat=text; +set hive.default.fileformat=TextFile; DESCRIBE FORMATTED orc_create; @@ -53,7 +53,7 @@ CREATE TABLE orc_create_complex ( DESCRIBE FORMATTED orc_create_complex; -LOAD DATA LOCAL INPATH '../data/files/orc_create.txt' OVERWRITE INTO TABLE orc_create_staging; +LOAD DATA LOCAL INPATH '../../data/files/orc_create.txt' OVERWRITE INTO TABLE orc_create_staging; SELECT * from orc_create_staging; @@ -70,16 +70,20 @@ CREATE TABLE orc_create_people_staging ( first_name string, last_name string, address string, + salary decimal, + start_date timestamp, state string); -LOAD DATA LOCAL INPATH '../data/files/orc_create_people.txt' +LOAD DATA LOCAL INPATH '../../data/files/orc_create_people.txt' OVERWRITE INTO TABLE orc_create_people_staging; CREATE TABLE orc_create_people ( id int, first_name string, last_name string, - address string) + address string, + salary decimal, + start_date timestamp) PARTITIONED BY (state string) STORED AS orc; @@ -92,9 +96,26 @@ SET hive.optimize.index.filter=true; -- test predicate push down with partition pruning SELECT COUNT(*) FROM orc_create_people where id < 10 and state = 'Ca'; +-- test predicate push down +SELECT COUNT(*) FROM orc_create_people where id = 50; +SELECT COUNT(*) FROM orc_create_people where id between 10 and 20; +SELECT COUNT(*) FROM orc_create_people where id > 10 and id < 100; +SELECT COUNT(*) FROM orc_create_people where (id + 1) = 20; +SELECT COUNT(*) FROM orc_create_people where (id + 10) < 200; +SELECT COUNT(*) FROM orc_create_people where id < 30 or first_name = "Rafael"; +SELECT COUNT(*) FROM orc_create_people + where length(substr(first_name, 1, 2)) <= 2 and last_name like '%'; +SELECT COUNT(*) FROM orc_create_people where salary = 200.00; +SELECT COUNT(*) FROM orc_create_people WHERE start_date IS NULL; +SELECT COUNT(*) FROM orc_create_people WHERE YEAR(start_date) = 2014; + +-- test predicate push down with partition pruning +SELECT COUNT(*) FROM orc_create_people where salary = 200.00 and state = 'Ca'; + -- test predicate push down with no column projection SELECT id, first_name, last_name, address - FROM orc_create_people WHERE id > 90; + FROM orc_create_people WHERE id > 90 + ORDER BY id, first_name, last_name; DROP TABLE orc_create; DROP TABLE orc_create_complex; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q index f916012b8365e..a0eaab75fac3e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q @@ -19,7 +19,7 @@ SELECT * FROM test_orc; ALTER TABLE test_orc SET SERDEPROPERTIES ('orc.stripe.size' = '1'); CREATE TABLE src_thousand(key STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1kv2.cogroup.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1kv2.cogroup.txt' INTO TABLE src_thousand; set hive.exec.orc.dictionary.key.size.threshold=0.5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q index cbfd7b359f878..0c8861e41c4bb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q @@ -10,7 +10,7 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; -- to another partition -- This can produce unexpected results with CombineHiveInputFormat -INSERT OVERWRITE TABLE test_orc PARTITION (part = '1') SELECT key FROM src LIMIT 5; +INSERT OVERWRITE TABLE test_orc PARTITION (part = '1') SELECT key FROM src tablesample (5 rows); ALTER TABLE test_orc ADD COLUMNS (cnt INT); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q new file mode 100644 index 0000000000000..f7e80a75b3ba6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols2.q @@ -0,0 +1,11 @@ +-- Create a table with one column, write to it, then add an additional column +-- This can break reads + +CREATE TABLE test_orc (key STRING) +STORED AS ORC; + +INSERT OVERWRITE TABLE test_orc SELECT key FROM src LIMIT 5; + +ALTER TABLE test_orc ADD COLUMNS (value STRING); + +SELECT * FROM test_orc order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q index 0ef57d18ccaf7..34cd6d47dfcd9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q @@ -3,13 +3,13 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.orc.OrcSerde' STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat'; -INSERT OVERWRITE TABLE test_orc SELECT '' FROM src limit 10; +INSERT OVERWRITE TABLE test_orc SELECT '' FROM src tablesample (10 rows); -- Test reading a column which is just empty strings SELECT * FROM test_orc; -INSERT OVERWRITE TABLE test_orc SELECT IF (key % 3 = 0, key, '') FROM src limit 10; +INSERT OVERWRITE TABLE test_orc SELECT IF (key % 3 = 0, key, '') FROM src tablesample (10 rows); -- Test reading a column which has some empty strings diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q index 6685da7a82245..83c5a0505e57e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q @@ -10,7 +10,7 @@ ALTER TABLE test_orc SET SERDEPROPERTIES ('orc.row.index.stride' = '1000'); -- to last index stride are the same (there's only two index strides) CREATE TABLE src_null(a STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/nulls.txt' INTO TABLE src_null; +LOAD DATA LOCAL INPATH '../../data/files/nulls.txt' INTO TABLE src_null; INSERT OVERWRITE TABLE test_orc SELECT a FROM src_null; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q new file mode 100644 index 0000000000000..b81adf2af3e53 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_min_max.q @@ -0,0 +1,32 @@ +create table if not exists alltypes ( + bo boolean, + ti tinyint, + si smallint, + i int, + bi bigint, + f float, + d double, + de decimal(10,3), + ts timestamp, + da date, + s string, + c char(5), + vc varchar(5), + m map, + l array, + st struct +) row format delimited fields terminated by '|' +collection items terminated by ',' +map keys terminated by ':' stored as textfile; + +create table alltypes_orc like alltypes; +alter table alltypes_orc set fileformat orc; + +load data local inpath '../../data/files/alltypes2.txt' overwrite into table alltypes; + +insert overwrite table alltypes_orc select * from alltypes; + +select min(bo), max(bo), min(ti), max(ti), min(si), max(si), min(i), max(i), min(bi), max(bi), min(f), max(f), min(d), max(d), min(de), max(de), min(ts), max(ts), min(da), max(da), min(s), max(s), min(c), max(c), min(vc), max(vc) from alltypes; + +select min(bo), max(bo), min(ti), max(ti), min(si), max(si), min(i), max(i), min(bi), max(bi), min(f), max(f), min(d), max(d), min(de), max(de), min(ts), max(ts), min(da), max(da), min(s), max(s), min(c), max(c), min(vc), max(vc) from alltypes_orc; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q new file mode 100644 index 0000000000000..1f5f54ae19ee8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_char.q @@ -0,0 +1,76 @@ +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET mapred.min.split.size=1000; +SET mapred.max.split.size=5000; + +create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); + +insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl; + +set hive.optimize.index.filter=false; + +-- char data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests) +select sum(hash(*)) from newtypesorc where c="apple"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c="apple"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c!="apple"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c!="apple"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c<"hello"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c<"hello"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c<="hello"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c<="hello"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c="apple "; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c="apple "; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c in ("apple", "carrot"); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c in ("apple", "carrot"); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c in ("apple", "hello"); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c in ("apple", "hello"); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c in ("carrot"); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c in ("carrot"); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c between "apple" and "carrot"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c between "apple" and "carrot"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c between "apple" and "zombie"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c between "apple" and "zombie"; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where c between "carrot" and "carrot1"; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where c between "carrot" and "carrot1"; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q new file mode 100644 index 0000000000000..c34be867e484f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_date.q @@ -0,0 +1,97 @@ +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET mapred.min.split.size=1000; +SET mapred.max.split.size=5000; + +create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); + +insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl; + +-- date data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests) +select sum(hash(*)) from newtypesorc where da='1970-02-20'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da='1970-02-20'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as date); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as date); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as varchar(20)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da=cast('1970-02-20' as varchar(20)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da!='1970-02-20'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da!='1970-02-20'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<'1970-02-27'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<'1970-02-27'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<'1970-02-29'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<'1970-02-29'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<'1970-02-15'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<'1970-02-15'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<='1970-02-20'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<='1970-02-20'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da<='1970-02-27'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da<='1970-02-27'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-20' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-20' as date), cast('1970-02-27' as date)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-22' as date)); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da in (cast('1970-02-21' as date), cast('1970-02-22' as date)); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-28'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da between '1970-02-19' and '1970-02-28'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where da between '1970-02-18' and '1970-02-19'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where da between '1970-02-18' and '1970-02-19'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q new file mode 100644 index 0000000000000..a93590eacca01 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ppd_decimal.q @@ -0,0 +1,151 @@ +SET hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SET mapred.min.split.size=1000; +SET mapred.max.split.size=5000; + +create table newtypesorc(c char(10), v varchar(10), d decimal(5,3), da date) stored as orc tblproperties("orc.stripe.size"="16777216"); + +insert overwrite table newtypesorc select * from (select cast("apple" as char(10)), cast("bee" as varchar(10)), 0.22, cast("1970-02-20" as date) from src src1 union all select cast("hello" as char(10)), cast("world" as varchar(10)), 11.22, cast("1970-02-27" as date) from src src2) uniontbl; + +-- decimal data types (EQUAL, NOT_EQUAL, LESS_THAN, LESS_THAN_EQUALS, IN, BETWEEN tests) +select sum(hash(*)) from newtypesorc where d=0.22; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d=0.22; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d='0.22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d='0.22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d=cast('0.22' as float); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d=cast('0.22' as float); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d!=0.22; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d!=0.22; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d!='0.22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d!='0.22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d!=cast('0.22' as float); + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d!=cast('0.22' as float); + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d<11.22; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d<11.22; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d<'11.22'; + +set hive.optimize.index.filter=true; +select sum(hash(*)) from newtypesorc where d<'11.22'; + +set hive.optimize.index.filter=false; +select sum(hash(*)) from newtypesorc where d, + lst ARRAY, + strct STRUCT +) ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|' +COLLECTION ITEMS TERMINATED BY ',' +MAP KEYS TERMINATED BY ':'; + +CREATE TABLE parquet_create ( + id int, + str string, + mp MAP, + lst ARRAY, + strct STRUCT +) STORED AS PARQUET; + +DESCRIBE FORMATTED parquet_create; + +LOAD DATA LOCAL INPATH '../../data/files/parquet_create.txt' OVERWRITE INTO TABLE parquet_create_staging; + +SELECT * FROM parquet_create_staging; + +INSERT OVERWRITE TABLE parquet_create SELECT * FROM parquet_create_staging; + +SELECT * FROM parquet_create group by id; +SELECT id, count(0) FROM parquet_create group by id; +SELECT str from parquet_create; +SELECT mp from parquet_create; +SELECT lst from parquet_create; +SELECT strct from parquet_create; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q new file mode 100644 index 0000000000000..652aef1b2ba53 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_ctas.q @@ -0,0 +1,24 @@ +drop table staging; +drop table parquet_ctas; +drop table parquet_ctas_advanced; +drop table parquet_ctas_alias; +drop table parquet_ctas_mixed; + +create table staging (key int, value string) stored as textfile; +insert into table staging select * from src order by key limit 10; + +create table parquet_ctas stored as parquet as select * from staging; +describe parquet_ctas; +select * from parquet_ctas; + +create table parquet_ctas_advanced stored as parquet as select key+1,concat(value,"value") from staging; +describe parquet_ctas_advanced; +select * from parquet_ctas_advanced; + +create table parquet_ctas_alias stored as parquet as select key+1 as mykey,concat(value,"value") as myvalue from staging; +describe parquet_ctas_alias; +select * from parquet_ctas_alias; + +create table parquet_ctas_mixed stored as parquet as select key,key+1,concat(value,"value") as myvalue from staging; +describe parquet_ctas_mixed; +select * from parquet_ctas_mixed; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q new file mode 100644 index 0000000000000..5d4f68ea43723 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_partitioned.q @@ -0,0 +1,34 @@ +set hive.exec.dynamic.partition.mode=nonstrict; +set hive.exec.dynamic.partition=true; + +DROP TABLE parquet_partitioned_staging; +DROP TABLE parquet_partitioned; + +CREATE TABLE parquet_partitioned_staging ( + id int, + str string, + part string +) ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +CREATE TABLE parquet_partitioned ( + id int, + str string +) PARTITIONED BY (part string) +STORED AS PARQUET; + +DESCRIBE FORMATTED parquet_partitioned; + +LOAD DATA LOCAL INPATH '../../data/files/parquet_partitioned.txt' OVERWRITE INTO TABLE parquet_partitioned_staging; + +SELECT * FROM parquet_partitioned_staging; + +INSERT OVERWRITE TABLE parquet_partitioned PARTITION (part) SELECT * FROM parquet_partitioned_staging; + +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +SELECT * FROM parquet_partitioned ORDER BY id, str; +SELECT part, COUNT(0) FROM parquet_partitioned GROUP BY part; + +set hive.input.format=org.apache.hadoop.hive.ql.io.CombineHiveInputFormat; +SELECT * FROM parquet_partitioned ORDER BY id, str; +SELECT part, COUNT(0) FROM parquet_partitioned GROUP BY part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q new file mode 100644 index 0000000000000..5d6333c934b74 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parquet_types.q @@ -0,0 +1,38 @@ +DROP TABLE parquet_types_staging; +DROP TABLE parquet_types; + +CREATE TABLE parquet_types_staging ( + cint int, + ctinyint tinyint, + csmallint smallint, + cfloat float, + cdouble double, + cstring1 string +) ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +CREATE TABLE parquet_types ( + cint int, + ctinyint tinyint, + csmallint smallint, + cfloat float, + cdouble double, + cstring1 string +) STORED AS PARQUET; + +LOAD DATA LOCAL INPATH '../../data/files/parquet_types.txt' OVERWRITE INTO TABLE parquet_types_staging; + +INSERT OVERWRITE TABLE parquet_types SELECT * FROM parquet_types_staging; + +SELECT * FROM parquet_types; + +SELECT ctinyint, + MAX(cint), + MIN(csmallint), + COUNT(cstring1), + AVG(cfloat), + STDDEV_POP(cdouble) +FROM parquet_types +GROUP BY ctinyint +ORDER BY ctinyint +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q index b7f8c64d4261f..03a5760e690e9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q @@ -1,7 +1,7 @@ create table test1(col1 string) partitioned by (partitionId int); insert overwrite table test1 partition (partitionId=1) - select key from src limit 10; + select key from src tablesample (10 rows); FROM ( FROM test1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q index 8738afdfa099a..70a7b252154d1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q @@ -1,45 +1,58 @@ drop table partition_date_1; -create table partition_date_1 (key string, value string) partitioned by (dt date, region int); +create table partition_date_1 (key string, value string) partitioned by (dt date, region string); + +insert overwrite table partition_date_1 partition(dt='2000-01-01', region= '1') + select * from src tablesample (10 rows); +insert overwrite table partition_date_1 partition(dt='2000-01-01', region= '2') + select * from src tablesample (5 rows); +insert overwrite table partition_date_1 partition(dt='2013-12-10', region= '2020-20-20') + select * from src tablesample (5 rows); +insert overwrite table partition_date_1 partition(dt='2013-08-08', region= '1') + select * from src tablesample (20 rows); +insert overwrite table partition_date_1 partition(dt='2013-08-08', region= '10') + select * from src tablesample (11 rows); -insert overwrite table partition_date_1 partition(dt='2000-01-01', region=1) - select * from src limit 10; -insert overwrite table partition_date_1 partition(dt='2000-01-01', region=2) - select * from src limit 5; -insert overwrite table partition_date_1 partition(dt='2013-08-08', region=1) - select * from src limit 20; -insert overwrite table partition_date_1 partition(dt='2013-08-08', region=10) - select * from src limit 11; select distinct dt from partition_date_1; -select * from partition_date_1 where dt = '2000-01-01' and region = 2 order by key,value; +select * from partition_date_1 where dt = '2000-01-01' and region = '2' order by key,value; -- 15 select count(*) from partition_date_1 where dt = date '2000-01-01'; -- 15. Also try with string value in predicate select count(*) from partition_date_1 where dt = '2000-01-01'; -- 5 -select count(*) from partition_date_1 where dt = date '2000-01-01' and region = 2; +select count(*) from partition_date_1 where dt = date '2000-01-01' and region = '2'; -- 11 -select count(*) from partition_date_1 where dt = date '2013-08-08' and region = 10; +select count(*) from partition_date_1 where dt = date '2013-08-08' and region = '10'; -- 30 -select count(*) from partition_date_1 where region = 1; +select count(*) from partition_date_1 where region = '1'; -- 0 -select count(*) from partition_date_1 where dt = date '2000-01-01' and region = 3; +select count(*) from partition_date_1 where dt = date '2000-01-01' and region = '3'; -- 0 select count(*) from partition_date_1 where dt = date '1999-01-01'; -- Try other comparison operations -- 20 -select count(*) from partition_date_1 where dt > date '2000-01-01' and region = 1; +select count(*) from partition_date_1 where dt > date '2000-01-01' and region = '1'; -- 10 -select count(*) from partition_date_1 where dt < date '2000-01-02' and region = 1; +select count(*) from partition_date_1 where dt < date '2000-01-02' and region = '1'; -- 20 -select count(*) from partition_date_1 where dt >= date '2000-01-02' and region = 1; +select count(*) from partition_date_1 where dt >= date '2000-01-02' and region = '1'; -- 10 -select count(*) from partition_date_1 where dt <= date '2000-01-01' and region = 1; +select count(*) from partition_date_1 where dt <= date '2000-01-01' and region = '1'; -- 20 -select count(*) from partition_date_1 where dt <> date '2000-01-01' and region = 1; +select count(*) from partition_date_1 where dt <> date '2000-01-01' and region = '1'; +-- 10 +select count(*) from partition_date_1 where dt between date '1999-12-30' and date '2000-01-03' and region = '1'; + + +-- Try a string key with date-like strings + +-- 5 +select count(*) from partition_date_1 where region = '2020-20-20'; +-- 5 +select count(*) from partition_date_1 where region > '2010-01-01'; drop table partition_date_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q index 9b84b59608503..c932ed1023637 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q @@ -3,7 +3,7 @@ drop table partition_date2_1; create table partition_date2_1 (key string, value string) partitioned by (dt date, region int); -- test date literal syntax -from (select * from src limit 1) x +from (select * from src tablesample (1 rows)) x insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=1) select * insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=2) select * insert overwrite table partition_date2_1 partition(dt=date '1999-01-01', region=2) select *; @@ -13,7 +13,7 @@ select * from partition_date2_1; -- insert overwrite insert overwrite table partition_date2_1 partition(dt=date '2000-01-01', region=2) - select 'changed_key', 'changed_value' from src limit 2; + select 'changed_key', 'changed_value' from src tablesample (2 rows); select * from partition_date2_1; -- truncate @@ -41,7 +41,7 @@ alter table partition_date2_1 partition(dt=date '1980-01-02', region=3) describe extended partition_date2_1 partition(dt=date '1980-01-02', region=3); insert overwrite table partition_date2_1 partition(dt=date '1980-01-02', region=3) - select * from src limit 2; + select * from src tablesample (2 rows); select * from partition_date2_1 order by key,value,dt,region; -- alter table set location diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q index ba193cd51a26d..a8381a4200f0a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q @@ -1,9 +1,9 @@ create table sc as select * -from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 +from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s; + select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s; create table sc_part (key string) partitioned by (ts string) stored as rcfile; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q index 81344334dfe01..b0b1ff4db6074 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q @@ -1,9 +1,9 @@ create table sc as select * -from (select '2011-01-11', '2011-01-11+14:18:26' from src limit 1 +from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+15:18:26' from src limit 1 + select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows) union all - select '2011-01-11', '2011-01-11+16:18:26' from src limit 1 ) s; + select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s; create table sc_part (key string) partitioned by (ts string) stored as rcfile; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q index 7f1accadac6ea..c9bca99b9cdf6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q @@ -2,14 +2,14 @@ set hive.typecheck.on.insert = true; -- begin part(string, string) pass(string, int) CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day string) stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day=2); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day=2); select * from tab1; drop table tab1; -- begin part(string, int) pass(string, string) CREATE TABLE tab1 (id1 int,id2 string) PARTITIONED BY(month string,day int) stored as textfile; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2'); select * from tab1; drop table tab1; @@ -17,7 +17,7 @@ drop table tab1; -- begin part(string, date) pass(string, date) create table tab1 (id1 int, id2 string) PARTITIONED BY(month string,day date) stored as textfile; alter table tab1 add partition (month='June', day='2008-01-01'); -LOAD DATA LOCAL INPATH '../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2008-01-01'); +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' overwrite into table tab1 PARTITION(month='June', day='2008-01-01'); select id1, id2, day from tab1 where day='2008-01-01'; drop table tab1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q index d700b1cbf8566..22aadd3b5359f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q @@ -3,13 +3,13 @@ drop table partition_varchar_1; create table partition_varchar_1 (key string, value varchar(20)) partitioned by (dt varchar(10), region int); insert overwrite table partition_varchar_1 partition(dt='2000-01-01', region=1) - select * from src limit 10; + select * from src tablesample (10 rows); insert overwrite table partition_varchar_1 partition(dt='2000-01-01', region=2) - select * from src limit 5; + select * from src tablesample (5 rows); insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=1) - select * from src limit 20; + select * from src tablesample (20 rows); insert overwrite table partition_varchar_1 partition(dt='2013-08-08', region=10) - select * from src limit 11; + select * from src tablesample (11 rows); select distinct dt from partition_varchar_1; select * from partition_varchar_1 where dt = '2000-01-01' and region = 2 order by key,value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q new file mode 100644 index 0000000000000..92cb742f15011 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar2.q @@ -0,0 +1,10 @@ +drop table partition_varchar_2; + +create table partition_varchar_2 (key string, value varchar(20)) partitioned by (dt varchar(15), region int); + +insert overwrite table partition_varchar_2 partition(dt='2000-01-01', region=1) + select * from src order by key limit 1; + +select * from partition_varchar_2 where cast(dt as varchar(10)) = '2000-01-01'; + +drop table partition_varchar_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q index e9b574c1ca380..3cf488fb0337f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q @@ -3,9 +3,9 @@ -- CustomSerDe(1, 2, 3) irrespective of the inserted values DROP TABLE PW17; -ADD JAR ../build/ql/test/test-serdes.jar; +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-custom-serde/${system:hive.version}/hive-it-custom-serde-${system:hive.version}.jar; CREATE TABLE PW17(USER STRING, COMPLEXDT ARRAY) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17 PARTITION (YEAR='1'); +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17 PARTITION (YEAR='1'); ALTER TABLE PW17 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe2'; ALTER TABLE PW17 SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1'; -- Without the fix HIVE-5199, will throw cast exception via FetchOperator @@ -14,13 +14,13 @@ SELECT * FROM PW17; -- Test for non-parititioned table. DROP TABLE PW17_2; CREATE TABLE PW17_2(USER STRING, COMPLEXDT ARRAY) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe1'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_2; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_2; -- Without the fix HIVE-5199, will throw cast exception via MapOperator SELECT COUNT(*) FROM PW17_2; DROP TABLE PW17_3; CREATE TABLE PW17_3(USER STRING, COMPLEXDT ARRAY >) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_3 PARTITION (YEAR='1'); +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_3 PARTITION (YEAR='1'); ALTER TABLE PW17_3 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe2'; ALTER TABLE PW17_3 SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3'; -- Without the fix HIVE-5285, will throw cast exception via FetchOperator @@ -28,7 +28,7 @@ SELECT * FROM PW17; DROP TABLE PW17_4; CREATE TABLE PW17_4(USER STRING, COMPLEXDT ARRAY >) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe3'; -LOAD DATA LOCAL INPATH '../data/files/pw17.txt' INTO TABLE PW17_4; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW17_4; -- Without the fix HIVE-5285, will throw cast exception via MapOperator SELECT COUNT(*) FROM PW17_4; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q new file mode 100644 index 0000000000000..40ed2585f5122 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat18.q @@ -0,0 +1,19 @@ +-- HIVE-5202 : Tests for SettableUnionObjectInspectors +-- CustomSerDe(4,5) are used here. +-- The final results should be all NULL columns deserialized using +-- CustomSerDe(4, 5) irrespective of the inserted values + +DROP TABLE PW18; +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-custom-serde/${system:hive.version}/hive-it-custom-serde-${system:hive.version}.jar; +CREATE TABLE PW18(USER STRING, COMPLEXDT UNIONTYPE) PARTITIONED BY (YEAR STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe5'; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW18 PARTITION (YEAR='1'); +ALTER TABLE PW18 PARTITION(YEAR='1') SET SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe4'; +-- Without the fix HIVE-5202, will throw unsupported data type exception. +SELECT * FROM PW18; + +-- Test for non-parititioned table. +DROP TABLE PW18_2; +CREATE TABLE PW18_2(USER STRING, COMPLEXDT UNIONTYPE) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.CustomSerDe5'; +LOAD DATA LOCAL INPATH '../../data/files/pw17.txt' INTO TABLE PW18_2; +-- Without the fix HIVE-5202, will throw unsupported data type exception +SELECT COUNT(*) FROM PW18_2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q index 09a39ae4e4476..3be0ff23b8124 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q @@ -127,7 +127,7 @@ create table ab(strct struct) row format delimited fields terminated by '\t' collection items terminated by '\001'; -load data local inpath '../data/files/kv1.txt' +load data local inpath '../../data/files/kv1.txt' overwrite into table ab; -- Create partitioned table with struct data: diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q new file mode 100644 index 0000000000000..475d45c19ff0c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join4.q @@ -0,0 +1,22 @@ +create table dual(a string); + +set hive.optimize.ppd=true; +drop table if exists test_tbl ; + +create table test_tbl (id string,name string); + +insert into table test_tbl +select 'a','b' from dual; + +explain +select t2.* +from +(select id,name from (select id,name from test_tbl) t1 sort by id) t2 +join test_tbl t3 on (t2.id=t3.id ) +where t2.name='c' and t3.id='a'; + +select t2.* +from +(select id,name from (select id,name from test_tbl) t1 sort by id) t2 +join test_tbl t3 on (t2.id=t3.id ) +where t2.name='c' and t3.id='a'; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q index a802df1b98819..06fe7ce580c84 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q @@ -10,18 +10,18 @@ FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; SELECT mi1.* FROM mi1; SELECT mi2.* FROM mi2; SELECT mi3.* FROM mi3; -dfs -cat ../build/ql/test/data/warehouse/mi4.out/*; +dfs -cat ${system:test.warehouse.dir}/mi4.out/*; set hive.ppd.remove.duplicatefilters=true; @@ -31,15 +31,15 @@ FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; FROM src a JOIN src b ON (a.key = b.key) INSERT OVERWRITE TABLE mi1 SELECT a.* WHERE a.key < 100 INSERT OVERWRITE TABLE mi2 SELECT a.key, a.value WHERE a.key >= 100 and a.key < 200 INSERT OVERWRITE TABLE mi3 PARTITION(ds='2008-04-08', hr='12') SELECT a.key WHERE a.key >= 200 and a.key < 300 -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; +INSERT OVERWRITE DIRECTORY 'target/warehouse/mi4.out' SELECT a.value WHERE a.key >= 300; SELECT mi1.* FROM mi1; SELECT mi2.* FROM mi2; SELECT mi3.* FROM mi3; -dfs -cat ../build/ql/test/data/warehouse/mi4.out/*; +dfs -cat ${system:test.warehouse.dir}/mi4.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q index 65a498d021f77..530ef9c4d849a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q @@ -36,3 +36,12 @@ FROM ( CLUSTER BY tkey ) tmap SELECT tmap.tkey, tmap.tvalue WHERE tmap.tkey < 100; + +-- test described in HIVE-4598 + +EXPLAIN +FROM ( + FROM ( SELECT * FROM src ) mapout REDUCE * USING 'cat' AS x,y +) reduced +insert overwrite local directory '/tmp/a' select * where x='a' or x='b' +insert overwrite local directory '/tmp/b' select * where x='c' or x='d'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q new file mode 100644 index 0000000000000..d90532cfa4bcd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udtf.q @@ -0,0 +1,12 @@ +explain +SELECT value from ( + select explode(array(key, value)) as (value) from ( + select * FROM src WHERE key > 400 + ) A +) B WHERE value < 450; + +SELECT value from ( + select explode(array(key, value)) as (value) from ( + select * FROM src WHERE key > 400 + ) A +) B WHERE value < 450; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q index d635e2d171469..a7606c5a5f2a5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q @@ -5,26 +5,26 @@ drop view v; create table t1_new (key string, value string) partitioned by (ds string); insert overwrite table t1_new partition (ds = '2011-10-15') -select 'key1', 'value1' from src limit 1; +select 'key1', 'value1' from src tablesample (1 rows); insert overwrite table t1_new partition (ds = '2011-10-16') -select 'key2', 'value2' from src limit 1; +select 'key2', 'value2' from src tablesample (1 rows); create table t1_old (keymap string, value string) partitioned by (ds string); insert overwrite table t1_old partition (ds = '2011-10-13') -select 'keymap3', 'value3' from src limit 1; +select 'keymap3', 'value3' from src tablesample (1 rows); insert overwrite table t1_old partition (ds = '2011-10-14') -select 'keymap4', 'value4' from src limit 1; +select 'keymap4', 'value4' from src tablesample (1 rows); create table t1_mapping (key string, keymap string) partitioned by (ds string); insert overwrite table t1_mapping partition (ds = '2011-10-13') -select 'key3', 'keymap3' from src limit 1; +select 'key3', 'keymap3' from src tablesample (1 rows); insert overwrite table t1_mapping partition (ds = '2011-10-14') -select 'key4', 'keymap4' from src limit 1; +select 'key4', 'keymap4' from src tablesample (1 rows); create view t1 partitioned on (ds) as diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q index 860dd631ce10a..440005fdee951 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + create table ppr_test (key string) partitioned by (ds string); alter table ppr_test add partition (ds = '1234'); @@ -9,14 +11,14 @@ alter table ppr_test add partition (ds = '12:4'); alter table ppr_test add partition (ds = '12%4'); alter table ppr_test add partition (ds = '12*4'); -insert overwrite table ppr_test partition(ds = '1234') select * from (select '1234' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '1224') select * from (select '1224' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '1214') select * from (select '1214' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12+4') select * from (select '12+4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12.4') select * from (select '12.4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12:4') select * from (select '12:4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12%4') select * from (select '12%4' from src limit 1 union all select 'abcd' from src limit 1) s; -insert overwrite table ppr_test partition(ds = '12*4') select * from (select '12*4' from src limit 1 union all select 'abcd' from src limit 1) s; +insert overwrite table ppr_test partition(ds = '1234') select * from (select '1234' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '1224') select * from (select '1224' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '1214') select * from (select '1214' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12+4') select * from (select '12+4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12.4') select * from (select '12.4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12:4') select * from (select '12:4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12%4') select * from (select '12%4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; +insert overwrite table ppr_test partition(ds = '12*4') select * from (select '12*4' from src tablesample (1 rows) union all select 'abcd' from src tablesample (1 rows)) s; select * from ppr_test where ds = '1234' order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q index 67c0da0dfc591..8c6090653811b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q @@ -1,24 +1,26 @@ +set hive.fetch.task.conversion=more; + create table ppr_test (key string) partitioned by (ds string); -insert overwrite table ppr_test partition(ds='2') select '2' from src limit 1; -insert overwrite table ppr_test partition(ds='22') select '22' from src limit 1; +insert overwrite table ppr_test partition(ds='2') select '2' from src tablesample (1 rows); +insert overwrite table ppr_test partition(ds='22') select '22' from src tablesample (1 rows); select * from ppr_test where ds = '2'; select * from ppr_test where ds = '22'; create table ppr_test2 (key string) partitioned by (ds string, s string); -insert overwrite table ppr_test2 partition(ds='1', s='2') select '1' from src limit 1; -insert overwrite table ppr_test2 partition(ds='2', s='1') select '2' from src limit 1; +insert overwrite table ppr_test2 partition(ds='1', s='2') select '1' from src tablesample (1 rows); +insert overwrite table ppr_test2 partition(ds='2', s='1') select '2' from src tablesample (1 rows); select * from ppr_test2 where s = '1'; select * from ppr_test2 where ds = '1'; create table ppr_test3 (key string) partitioned by (col string, ol string, l string); -insert overwrite table ppr_test3 partition(col='1', ol='2', l = '3') select '1' from src limit 1; -insert overwrite table ppr_test3 partition(col='1', ol='1', l = '2') select '2' from src limit 1; -insert overwrite table ppr_test3 partition(col='1', ol='2', l = '1') select '3' from src limit 1; +insert overwrite table ppr_test3 partition(col='1', ol='2', l = '3') select '1' from src tablesample (1 rows); +insert overwrite table ppr_test3 partition(col='1', ol='1', l = '2') select '2' from src tablesample (1 rows); +insert overwrite table ppr_test3 partition(col='1', ol='2', l = '1') select '3' from src tablesample (1 rows); select * from ppr_test3 where l = '1'; select * from ppr_test3 where l = '2'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q index ad908a02ad181..22ee92634d123 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q @@ -2,7 +2,7 @@ set hive.heartbeat.interval=5; CREATE TABLE PROGRESS_1(key int, value string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv6.txt' INTO TABLE PROGRESS_1; +LOAD DATA LOCAL INPATH '../../data/files/kv6.txt' INTO TABLE PROGRESS_1; select count(1) from PROGRESS_1 t1 join PROGRESS_1 t2 on t1.key=t2.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q index eea5415d682a1..d56b4123554c4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; --1. test1 select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q index 03f435e4539e2..9799534ff4a55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q @@ -9,11 +9,11 @@ CREATE TABLE part( p_type STRING, p_size INT, p_container STRING, - p_retailprice DECIMAL, + p_retailprice DECIMAL(6,2), p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. aggregate functions with decimal type diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q index 885c3b3d43e17..4fe9710d0d247 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testNoPTFNoWindowing select p_mfgr, p_name, p_size diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q index 72eeb104d5298..0cde350f73693 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q @@ -10,7 +10,7 @@ ARR_DELAY float, FL_NUM string ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; -- 1. basic Matchpath test select origin_city_name, fl_num, year, month, day_of_month, sz, tpath diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q index 535a233a9e3c4..a68c578848dcb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q @@ -12,7 +12,7 @@ CREATE TABLE part_rc( p_comment STRING ) STORED AS RCFILE ; -LOAD DATA LOCAL INPATH '../data/files/part.rc' overwrite into table part_rc; +LOAD DATA LOCAL INPATH '../../data/files/part.rc' overwrite into table part_rc; -- testWindowingPTFWithPartRC select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q index a2140cd049f2d..4b508e9df0449 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q @@ -10,7 +10,7 @@ ARR_DELAY float, FL_NUM string ); -LOAD DATA LOCAL INPATH '../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt' OVERWRITE INTO TABLE flights_tiny; create temporary function matchpathtest as 'org.apache.hadoop.hive.ql.udf.ptf.MatchPath$MatchPathResolver'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q index 4aa8ce11bec9c..c5d65f0efa211 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q @@ -12,7 +12,7 @@ CREATE TABLE part_seq( p_comment STRING ) STORED AS SEQUENCEFILE ; -LOAD DATA LOCAL INPATH '../data/files/part.seq' overwrite into table part_seq; +LOAD DATA LOCAL INPATH '../../data/files/part.seq' overwrite into table part_seq; -- testWindowingPTFWithPartSeqFile select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q index f198baa6e4c5b..57e8cc673cace 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q @@ -1,3 +1,5 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; DROP TABLE lineitem; CREATE TABLE lineitem (L_ORDERKEY INT, @@ -19,7 +21,7 @@ CREATE TABLE lineitem (L_ORDERKEY INT, ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; -LOAD DATA LOCAL INPATH '../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; +LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; CREATE INDEX lineitem_lshipdate_idx ON TABLE lineitem(l_shipdate) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(l_shipdate)"); ALTER INDEX lineitem_lshipdate_idx ON lineitem REBUILD; @@ -156,7 +158,7 @@ DROP INDEX tbl_part_index on tblpart; DROP TABLE tblpart; CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; -LOAD DATA LOCAL INPATH '../data/files/tbl.txt' OVERWRITE INTO TABLE tbl; +LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl; CREATE INDEX tbl_key_idx ON TABLE tbl(key) AS 'org.apache.hadoop.hive.ql.index.AggregateIndexHandler' WITH DEFERRED REBUILD IDXPROPERTIES("AGGREGATES"="count(key)"); ALTER INDEX tbl_key_idx ON tbl REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q index 65b9f8776d192..c93902ab3e39c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT 'abc', "abc", diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q new file mode 100644 index 0000000000000..a34a25af4bb96 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_alter.q @@ -0,0 +1,21 @@ + +set hive.support.quoted.identifiers=column; + +create table src_b3(`x+1` string, `!@#$%^&*()_q` string) ; + +alter table src_b3 +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +; + + +-- alter partition +create table src_p3(`x+1` string, `y&y` string) partitioned by (`!@#$%^&*()_q` string); + +insert overwrite table src_p3 partition(`!@#$%^&*()_q`='a') select * from src; +show partitions src_p3; + +alter table src_p3 add if not exists partition(`!@#$%^&*()_q`='b'); +show partitions src_p3; + +alter table src_p3 partition(`!@#$%^&*()_q`='b') rename to partition(`!@#$%^&*()_q`='c'); +show partitions src_p3; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q new file mode 100644 index 0000000000000..680868e549cee --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_basic.q @@ -0,0 +1,34 @@ + +set hive.support.quoted.identifiers=column; + +-- basic +create table t1(`x+1` string, `y&y` string, `!@#$%^&*()_q` string); +describe t1; +select `x+1`, `y&y`, `!@#$%^&*()_q` from t1; +explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1; +explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1 where `!@#$%^&*()_q` = '1'; +explain select `x+1`, `y&y`, `!@#$%^&*()_q` from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = '1'; +explain select `x+1`, `y&y`, `!@#$%^&*()_q`, rank() over(partition by `!@#$%^&*()_q` order by `y&y`) +from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = '1'; + +-- case insensitive +explain select `X+1`, `Y&y`, `!@#$%^&*()_Q`, rank() over(partition by `!@#$%^&*()_q` order by `y&y`) +from t1 where `!@#$%^&*()_q` = '1' group by `x+1`, `y&Y`, `!@#$%^&*()_q` having `!@#$%^&*()_Q` = '1'; + + +-- escaped back ticks +create table t4(`x+1``` string, `y&y` string); +describe t4; +insert into table t4 select * from src; +select `x+1```, `y&y`, rank() over(partition by `x+1``` order by `y&y`) +from t4 where `x+1``` = '10' group by `x+1```, `y&y` having `x+1``` = '10'; + +-- view +create view v1 as +select `x+1```, `y&y` +from t4 where `x+1``` < '200'; + +select `x+1```, `y&y`, rank() over(partition by `x+1``` order by `y&y`) +from v1 +group by `x+1```, `y&y` +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q new file mode 100644 index 0000000000000..e9416ae282228 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_partition.q @@ -0,0 +1,24 @@ + +set hive.support.quoted.identifiers=column; + + +create table src_p(`x+1` string, `y&y` string) partitioned by (`!@#$%^&*()_q` string); +insert overwrite table src_p partition(`!@#$%^&*()_q`='a') select * from src; + +show partitions src_p; + +explain select `x+1`, `y&y`, `!@#$%^&*()_q` +from src_p where `!@#$%^&*()_q` = 'a' and `x+1`='10' +group by `x+1`, `y&y`, `!@#$%^&*()_q` having `!@#$%^&*()_q` = 'a' +; + +set hive.exec.dynamic.partition.mode=nonstrict +; + +create table src_p2(`x+1` string) partitioned by (`!@#$%^&*()_q` string); + +insert overwrite table src_p2 partition(`!@#$%^&*()_q`) +select key, value as `!@#$%^&*()_q` from src where key < '200' +; + +show partitions src_p2; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q new file mode 100644 index 0000000000000..5c959674117f1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_skew.q @@ -0,0 +1,26 @@ + +set hive.support.quoted.identifiers=column; + +set hive.mapred.supports.subdirectories=true; +set hive.internal.ddl.list.bucketing.enable=true; +set hive.optimize.skewjoin.compiletime = true; + +CREATE TABLE T1(`!@#$%^&*()_q` string, `y&y` string) +SKEWED BY (`!@#$%^&*()_q`) ON ((2)) STORED AS TEXTFILE +; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; + +CREATE TABLE T2(`!@#$%^&*()_q` string, `y&y` string) +SKEWED BY (`!@#$%^&*()_q`) ON ((2)) STORED AS TEXTFILE +; + +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T2; + +-- a simple join query with skew on both the tables on the join key +-- adding a order by at the end to make the results deterministic + +EXPLAIN +SELECT a.*, b.* FROM T1 a JOIN T2 b ON a. `!@#$%^&*()_q` = b. `!@#$%^&*()_q` +; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q new file mode 100644 index 0000000000000..38d1b99c4b8e6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_smb.q @@ -0,0 +1,34 @@ + +set hive.support.quoted.identifiers=column; + + +set hive.enforce.bucketing = true; +set hive.enforce.sorting = true; +create table src_b(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +; + +insert overwrite table src_b +select * from src +; + +create table src_b2(`x+1` string, `!@#$%^&*()_q` string) +clustered by (`!@#$%^&*()_q`) sorted by (`!@#$%^&*()_q`) into 2 buckets +; + +insert overwrite table src_b2 +select * from src +; + +set hive.auto.convert.join=true; +set hive.auto.convert.sortmerge.join=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; + +set hive.auto.convert.sortmerge.join.to.mapjoin=false; +set hive.auto.convert.sortmerge.join.bigtable.selection.policy = org.apache.hadoop.hive.ql.optimizer.TableSizeBasedBigTableSelectorForAutoSMJ; + +select a.`x+1`, a.`!@#$%^&*()_q`, b.`x+1`, b.`!@#$%^&*()_q` +from src_b a join src_b2 b on a.`!@#$%^&*()_q` = b.`!@#$%^&*()_q` +where a.`x+1` < '11' +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q new file mode 100644 index 0000000000000..d64e9cb9d524d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quotedid_tblproperty.q @@ -0,0 +1,8 @@ +ADD JAR ${system:maven.local.repository}/org/apache/hive/hive-it-test-serde/${system:hive.version}/hive-it-test-serde-${system:hive.version}.jar; + +CREATE TABLE xyz(KEY STRING, VALUE STRING) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' +STORED AS TEXTFILE +TBLPROPERTIES('columns'='valid_colname,invalid.colname') +; + +describe xyz; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q index 3e83e6693b276..df460c89aa896 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q @@ -1,7 +1,7 @@ set hive.map.aggr.hash.percentmemory = 0.3; set hive.mapred.local.mem = 256; -add file ../data/scripts/dumpdata_script.py; +add file ../../data/scripts/dumpdata_script.py; CREATE table columnTable_Bigdata (key STRING, value STRING) ROW FORMAT SERDE diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q index 9cfcee5a599c6..1c311fc478203 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q @@ -1,3 +1,5 @@ +set hive.support.quoted.identifiers=none; + EXPLAIN SELECT * FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q index 926601c647bc3..c4fcaaf95b6a1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q @@ -1,4 +1,4 @@ -dfs -put ../data/scripts/newline.py /newline.py; +dfs -put ../../data/scripts/newline.py /newline.py; add file hdfs:///newline.py; set hive.transform.escape.input=true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q index 8d04d3e991c59..df199b0d7765b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q @@ -1,10 +1,12 @@ +DROP TABLE IF EXISTS repairtable; + CREATE TABLE repairtable(col STRING) PARTITIONED BY (p1 STRING, p2 STRING); MSCK TABLE repairtable; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/warehouse/repairtable/p1=a/p2=a; -dfs ${system:test.dfs.mkdir} ../build/ql/test/data/warehouse/repairtable/p1=b/p2=a; -dfs -touchz ../build/ql/test/data/warehouse/repairtable/p1=b/p2=a/datafile; +dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=a/p2=a; +dfs ${system:test.dfs.mkdir} ${system:test.warehouse.dir}/repairtable/p1=b/p2=a; +dfs -touchz ${system:test.warehouse.dir}/repairtable/p1=b/p2=a/datafile; MSCK TABLE repairtable; @@ -12,4 +14,4 @@ MSCK REPAIR TABLE repairtable; MSCK TABLE repairtable; - +DROP TABLE repairtable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q new file mode 100644 index 0000000000000..a0514c86ff211 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/root_dir_external_table.q @@ -0,0 +1,11 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test_root_dir_external_table; + +insert overwrite directory "hdfs:///tmp/test_root_dir_external_table" select key from src where (key < 20) order by key; + +dfs -cp /tmp/test_root_dir_external_table/000000_0 /000000_0; +dfs -rmr hdfs:///tmp/test_root_dir_external_table; + +create external table roottable (key string) row format delimited fields terminated by '\\t' stored as textfile location 'hdfs:///'; +select count(*) from roottable; + +dfs -rmr /000000_0; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q index ecd4d13d0e23b..b3c38bf577199 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q @@ -1,5 +1,5 @@ -dfs -mkdir file:///tmp/test; -dfs -mkdir hdfs:///tmp/test; +dfs ${system:test.dfs.mkdir} file:///tmp/test; +dfs ${system:test.dfs.mkdir} hdfs:///tmp/test; create external table dynPart (key string) partitioned by (value string, value2 string) row format delimited fields terminated by '\\t' stored as textfile; insert overwrite local directory "/tmp/test" select key from src where (key = 10) order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q index 4f65016f3081c..2dfb12951f9d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q @@ -1,7 +1,9 @@ set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; + +-- EXCLUDE_OS_WINDOWS CREATE TABLE dest1(key INT, value STRING); -ADD FILE src/test/scripts/testgrep; +ADD FILE ../../ql/src/test/scripts/testgrep; FROM ( FROM src diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q new file mode 100644 index 0000000000000..0008ae51c4365 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1_win.q @@ -0,0 +1,16 @@ +set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat; +-- INCLUDE_OS_WINDOWS + +CREATE TABLE dest1(key INT, value STRING); + +ADD FILE src/test/scripts/testgrep_win.bat; + +FROM ( + FROM src + SELECT TRANSFORM(src.key, src.value) + USING 'testgrep_win.bat' AS (tkey, tvalue) + CLUSTER BY tkey +) tmap +INSERT OVERWRITE TABLE dest1 SELECT tmap.tkey, tmap.tvalue; + +SELECT dest1.* FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q new file mode 100644 index 0000000000000..25a1a81283221 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_dummy_source.q @@ -0,0 +1,33 @@ +explain +select 'a', 100; +select 'a', 100; + +--evaluation +explain +select 1 + 1; +select 1 + 1; + +-- explode (not possible for lateral view) +explain +select explode(array('a', 'b')); +select explode(array('a', 'b')); + +set hive.fetch.task.conversion=more; + +explain +select 'a', 100; +select 'a', 100; + +explain +select 1 + 1; +select 1 + 1; + +explain +select explode(array('a', 'b')); +select explode(array('a', 'b')); + +-- subquery +explain +select 2 + 3,x from (select 1 + 2 x) X; +select 2 + 3,x from (select 1 + 2 x) X; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q index 2a287bd877759..accdb54744cc1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q @@ -31,8 +31,8 @@ WITH SERDEPROPERTIES ( ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH "../data/files/apache.access.log" INTO TABLE serde_regex; -LOAD DATA LOCAL INPATH "../data/files/apache.access.2.log" INTO TABLE serde_regex; +LOAD DATA LOCAL INPATH "../../data/files/apache.access.log" INTO TABLE serde_regex; +LOAD DATA LOCAL INPATH "../../data/files/apache.access.2.log" INTO TABLE serde_regex; SELECT * FROM serde_regex ORDER BY time; @@ -42,7 +42,7 @@ DROP TABLE serde_regex; EXPLAIN CREATE TABLE serde_regex1( - key decimal, + key decimal(38,18), value int) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' WITH SERDEPROPERTIES ( @@ -51,7 +51,7 @@ WITH SERDEPROPERTIES ( STORED AS TEXTFILE; CREATE TABLE serde_regex1( - key decimal, + key decimal(38,18), value int) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.RegexSerDe' WITH SERDEPROPERTIES ( @@ -59,7 +59,7 @@ WITH SERDEPROPERTIES ( ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH "../data/files/kv7.txt" INTO TABLE serde_regex1; +LOAD DATA LOCAL INPATH "../../data/files/kv7.txt" INTO TABLE serde_regex1; SELECT key, value FROM serde_regex1 ORDER BY key, value; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q index 7e3d1f4d8aa41..d10239c31af62 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q @@ -24,7 +24,7 @@ set b=a; set c=${hiveconf:${hiveconf:b}}; set c; -set jar=${system:build.ivy.lib.dir}/default/derby-${system:derby.version}.jar; +set jar=${system:maven.local.repository}/org/apache/derby/derby/${system:derby.version}/derby-${system:derby.version}.jar; add file ${hiveconf:jar}; delete file ${hiveconf:jar}; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q index 5fcdf97e2db47..9758c16caa5ad 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP TABLE show_idx_empty; DROP TABLE show_idx_full; @@ -24,4 +25,4 @@ SHOW INDEXES ON show_idx_empty; DROP INDEX idx_1 on show_idx_full; DROP INDEX idx_2 on show_idx_full; DROP TABLE show_idx_empty; -DROP TABLE show_idx_full; \ No newline at end of file +DROP TABLE show_idx_full; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q index ab588937e179b..bb43c5e1387a9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; DROP TABLE show_idx_t1; CREATE TABLE show_idx_t1(KEY STRING, VALUE STRING); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q index 7fa7b828bd72e..1fc1d8e1f2a8b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q @@ -1,4 +1,5 @@ SHOW PARTITIONS srcpart; +SHOW PARTITIONS default.srcpart; SHOW PARTITIONS srcpart PARTITION(hr='11'); SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08'); -SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08', hr='12'); \ No newline at end of file +SHOW PARTITIONS srcpart PARTITION(ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q new file mode 100644 index 0000000000000..d8ce96a37d7a8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_roles.q @@ -0,0 +1,4 @@ +create role role1; +create role role2; + +show roles; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q index 9184d6da897c0..55fb7b67ffc83 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q @@ -1,3 +1,4 @@ +set hive.support.quoted.identifiers=none; EXPLAIN SHOW TABLE EXTENDED IN default LIKE `src`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q index ad917beeef9ce..47535eab638d9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q @@ -13,10 +13,10 @@ CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T4(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE dest_j1(key INT, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T4; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T4; EXPLAIN diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q new file mode 100644 index 0000000000000..b8ca592ab70a1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_noskew.q @@ -0,0 +1,9 @@ +set hive.auto.convert.join=false; +set hive.optimize.skewjoin=true; + +explain +create table noskew as select a.* from src a join src b on a.key=b.key order by a.key limit 30; + +create table noskew as select a.* from src a join src b on a.key=b.key order by a.key limit 30; + +select * from noskew; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q index 03eab4cd6d54e..fc07742cd7422 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q @@ -20,12 +20,12 @@ set mapred.input.dir.recursive=true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple join query with skew on both the tables on the join key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q index 9cb919531f7c5..50cfc61962af9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q @@ -12,16 +12,16 @@ set mapred.input.dir.recursive=true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This is to test the union->selectstar->filesink and skewjoin optimization -- Union of 3 map-reduce subqueries is performed for the skew join diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q index af446bb65c08e..504ba8be2a29e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple join query with skew on both the tables on the join key -- adding a order by at the end to make the results deterministic diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q index 199f3201afb77..f35af901704ef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q @@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; drop table array_valued_T1; create table array_valued_T1 (key string, value array) SKEWED BY (key) ON ((8)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q index ef61fb22f17a1..9e00bdcd76080 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q @@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- This test is to verify the skew join compile optimization when the join is followed -- by a union. Both sides of a union consist of a join, which should have used diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q index b5d9d9bc46874..171995069b77b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key, val) ON ((3, 13), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- Both the join tables are skewed by 2 keys, and one of the skewed values -- is common to both the tables. The join key matches the skewed key set. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q index 0634c4f4ff6db..5ef217c90064a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q @@ -4,16 +4,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) SKEWED BY (val) ON ((12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for skewed join compile time optimization for more than 2 tables. -- The join key for table 3 is different from the join key used for joining diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q index 0f031dd4fc0ae..df1a26bcc7d9d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q @@ -5,16 +5,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) SKEWED BY (val) ON ((12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for skewed join compile time optimization for more than 2 tables. -- The join key for table 3 is different from the join key used for joining diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q index d5474a455e9dd..1db5472396db1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q @@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE tmpT1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE tmpT1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE tmpT1; -- testing skew on other data types - int CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2)); @@ -12,7 +12,7 @@ INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1; CREATE TABLE tmpT2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE tmpT2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE tmpT2; CREATE TABLE T2(key INT, val STRING) SKEWED BY (key) ON ((3)); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q index 46b4f6d6e600d..915de612ded53 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the both the columns diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q index 0592ca8c3e498..2ee79cc758531 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the first skewed column @@ -31,12 +31,12 @@ DROP TABLE T2; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the both the columns diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q index 433fea336dfd4..9d06cc030699a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q @@ -4,7 +4,7 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE tmpT1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE tmpT1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE tmpT1; -- testing skew on other data types - int CREATE TABLE T1(key INT, val STRING) SKEWED BY (key) ON ((2)); @@ -16,7 +16,7 @@ INSERT OVERWRITE TABLE T1 SELECT key, val FROM tmpT1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- Once HIVE-3445 is fixed, the compile time skew join optimization would be -- applicable here. Till the above jira is fixed, it would be performed as a diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q index 0b11ebe4cb696..075645f89d452 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q @@ -6,11 +6,11 @@ CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) INTO 4 BUCKETS SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- add a test where the skewed key is also the bucketized key -- it should not matter, and the compile time skewed join diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q index 34fcdbfac4cb7..f7acaad18e1ea 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (7)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple query with skew on both the tables on the join key -- multiple skew values are present for the skewed keys diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q index f217052881e24..9b908ce21b1d2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q @@ -6,11 +6,11 @@ CREATE TABLE T1(key STRING, val STRING) CLUSTERED BY (key) SORTED BY (key) INTO 4 BUCKETS SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- add a test where the skewed key is also the bucketized/sorted key -- it should not matter, and the compile time skewed join diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q index f6002ad49802a..22ea4f06218ac 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- a simple query with skew on both the tables. One of the skewed -- value is common to both the tables. The skewed value should not be diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q index ca83c446085fa..8496b1aa79c0a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q @@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- only of the tables of the join (the left table of the join) is skewed -- the skewed filter would still be applied to both the tables diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q index 3d7884c5e3dcb..152de5bde72c5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- One of the tables is skewed by 2 columns, and the other table is -- skewed by one column. Ths join is performed on the first skewed column diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q index 36cf8ceeaebb4..2e261bde66bbf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q @@ -5,12 +5,12 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key, val) ON ((2, 12), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key, val) ON ((3, 13), (8, 18)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- Both the join tables are skewed by 2 keys, and one of the skewed values -- is common to both the tables. The join key is a subset of the skewed key set: diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q index cf84f67b6a0fc..e4d9605f6f7af 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q @@ -5,16 +5,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for validating skewed join compile time optimization for more than -- 2 tables. The join key is the same, and so a 3-way join would be performed. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q index d0ac845f86581..85746d9611dab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q @@ -4,16 +4,16 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) SKEWED BY (key) ON ((3), (8)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; -- This test is for validating skewed join compile time optimization for more than -- 2 tables. The join key is the same, and so a 3-way join would be performed. diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q index 04834033a11e6..889ab6c3f5534 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q @@ -5,11 +5,11 @@ set hive.optimize.skewjoin.compiletime = true; CREATE TABLE T1(key STRING, val STRING) SKEWED BY (key) ON ((2)) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; -- no skew join compile time optimization would be performed if one of the -- join sources is a sub-query consisting of a union all diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q index 359513e424dba..9dee4110f5991 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q index a79ebf62d0693..1fbe2090eaf2a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q @@ -6,11 +6,11 @@ alter table tmp_smb_bucket_10 add partition (ds = '2'); -- add dummy files to make sure that the number of files in each partition is same as number of buckets -load data local inpath '../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); -load data local inpath '../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); +load data local inpath '../../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); +load data local inpath '../../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='1'); -load data local inpath '../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); -load data local inpath '../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); +load data local inpath '../../data/files/smbbucket_1.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); +load data local inpath '../../data/files/smbbucket_2.rc' INTO TABLE tmp_smb_bucket_10 partition(ds='2'); set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q index 9d86314879d68..e2b24333ad416 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q index 8b534e85aee1d..e43174bc0768b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q @@ -10,9 +10,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; explain select * from (select a.key from smb_bucket_1 a join smb_bucket_2 b on (a.key = b.key) where a.key = 5) t1 left outer join (select c.key from smb_bucket_2 c join smb_bucket_3 d on (c.key = d.key) where c.key=5) t2 on (t1.key=t2.key) where t2.key=5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q index 73b21fae250e0..b379706cc8ac5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q index 83143b170ed5c..2b3f67ea4eea1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q index 61ec084f64ffa..406604e621ad0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q @@ -6,9 +6,9 @@ create table smb_bucket_1(key int, value string) CLUSTERED BY (key) SORTED BY (k create table smb_bucket_2(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; create table smb_bucket_3(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS STORED AS RCFILE; -load data local inpath '../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; -load data local inpath '../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; -load data local inpath '../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; +load data local inpath '../../data/files/smbbucket_1.rc' overwrite into table smb_bucket_1; +load data local inpath '../../data/files/smbbucket_2.rc' overwrite into table smb_bucket_2; +load data local inpath '../../data/files/smbbucket_3.rc' overwrite into table smb_bucket_3; set hive.optimize.bucketmapjoin = true; set hive.optimize.bucketmapjoin.sortedmerge = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q index 1488b1f949527..ca1c7491b7298 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q @@ -15,8 +15,8 @@ create table smb_join_results(k1 int, v1 string, k2 int, v2 string); create table smb_join_results_empty_bigtable(k1 int, v1 string, k2 int, v2 string); create table normal_join_results(k1 int, v1 string, k2 int, v2 string); -load data local inpath '../data/files/empty1.txt' into table smb_bucket4_1; -load data local inpath '../data/files/empty2.txt' into table smb_bucket4_1; +load data local inpath '../../data/files/empty1.txt' into table smb_bucket4_1; +load data local inpath '../../data/files/empty2.txt' into table smb_bucket4_1; insert overwrite table smb_bucket4_2 select * from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q index 6f282ed441bfa..f296057d43e38 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q @@ -5,7 +5,7 @@ set hive.exec.reducers.max = 1; create table smb_bucket_input (key int, value string) stored as rcfile; -load data local inpath '../data/files/smb_bucket_input.rc' into table smb_bucket_input; +load data local inpath '../../data/files/smb_bucket_input.rc' into table smb_bucket_input; CREATE TABLE smb_bucket4_1(key int, value string) CLUSTERED BY (key) SORTED BY (key) INTO 1 BUCKETS; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q index 6fe3d211a00b0..76ca152ef55f4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q @@ -1 +1 @@ -source ../data/files/source.txt; +source ../../data/files/source.txt; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q new file mode 100644 index 0000000000000..f5d7ff8fdd9bf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split.q @@ -0,0 +1,8 @@ +DROP TABLE tmp_jo_tab_test; +CREATE table tmp_jo_tab_test (message_line STRING) +STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/input.txt' +OVERWRITE INTO TABLE tmp_jo_tab_test; + +select size(split(message_line, '\t')) from tmp_jo_tab_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q index 0b783de153b29..359d27b31523a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q @@ -26,5 +26,5 @@ DESCRIBE FORMATTED tmptable; -- Load a file into a existing table -- Some stats (numFiles, totalSize) should be updated correctly -- Some other stats (numRows, rawDataSize) should be cleared -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE tmptable; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE tmptable; DESCRIBE FORMATTED tmptable; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q index 6618c913ea700..d037c003b7582 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q @@ -2,25 +2,25 @@ set datanucleus.cache.collections=false; set hive.stats.autogather=true; CREATE TABLE srcbucket_mapjoin(key int, value string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin; +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin; CREATE TABLE srcbucket_mapjoin_part (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; explain -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part partition(ds='2008-04-08'); desc formatted srcbucket_mapjoin_part partition(ds='2008-04-08'); CREATE TABLE srcbucket_mapjoin_part_2 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); -load data local inpath '../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE srcbucket_mapjoin_part_2 partition(ds='2008-04-08'); create table bucketmapjoin_hash_result_1 (key bigint , value1 bigint, value2 bigint); create table bucketmapjoin_hash_result_2 (key bigint , value1 bigint, value2 bigint); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q index 425de64c26e83..e773cd749403f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q @@ -13,7 +13,7 @@ insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select -- Some other stats (numRows, rawDataSize) should be cleared desc formatted stats_part partition (ds='2010-04-08', hr='13'); -load data local inpath '../data/files/srcbucket20.txt' INTO TABLE stats_part partition (ds='2010-04-08', hr='13'); +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE stats_part partition (ds='2010-04-08', hr='13'); desc formatted stats_part partition (ds='2010-04-08', hr='13'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q index da4af9655d16a..51514bd7738ff 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q @@ -1,7 +1,7 @@ set datanucleus.cache.collections=false; set hive.stats.autogather=true; set hive.stats.reliable=true; -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.KeyVerifyingStatsAggregator; @@ -56,7 +56,7 @@ insert overwrite table stats_part partition (ds='2010-04-08', hr = '13') select desc formatted stats_part partition (ds='2010-04-08', hr = '13'); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.KeyVerifyingStatsAggregator; set hive.stats.key.prefix.max.length=0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q index 5962348d9c317..fd7e0eaca8c39 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q @@ -5,9 +5,9 @@ drop table hive_test_dst; create table hive_test_src ( col1 string ) stored as textfile ; explain extended -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; -load data local inpath '../data/files/test.dat' overwrite into table hive_test_src ; +load data local inpath '../../data/files/test.dat' overwrite into table hive_test_src ; desc formatted hive_test_src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q index 62580042d4ded..80a67f405cb0b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q @@ -25,8 +25,8 @@ insert overwrite table nzhang_part2 partition(ds='2008-12-31', hr) select key, v show partitions nzhang_part1; show partitions nzhang_part2; -select * from nzhang_part1 where ds is not null and hr is not null; -select * from nzhang_part2 where ds is not null and hr is not null; +select * from nzhang_part1 where ds is not null and hr is not null order by ds, hr, key; +select * from nzhang_part2 where ds is not null and hr is not null order by ds, hr, key; describe formatted nzhang_part1 partition(ds='2008-04-08',hr=11); describe formatted nzhang_part1 partition(ds='2008-04-08',hr=12); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q index 4e7d3dc547a30..5e6b0aaa1253b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q @@ -1,12 +1,12 @@ -- In this test, there is a dummy stats aggregator which throws an error when various --- methods are called (as indicated by the parameter hive.test.dummystats.agregator) +-- methods are called (as indicated by the parameter hive.test.dummystats.aggregator) -- Since stats need not be reliable (by setting hive.stats.reliable to false), the -- insert statements succeed. The insert statement succeeds even if the stats aggregator -- is set to null, since stats need not be reliable. create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; set hive.stats.reliable=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q new file mode 100644 index 0000000000000..3c1f132a68f2f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter.q @@ -0,0 +1,16 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=false; + +-- by analyze +create table dummy1 as select * from src; + +analyze table dummy1 compute statistics; +desc formatted dummy1; + +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; + +-- by autogather +create table dummy2 as select * from src; + +desc formatted dummy2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q new file mode 100644 index 0000000000000..e1274c0cb5197 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_counter_partitioned.q @@ -0,0 +1,45 @@ +set hive.stats.dbclass=counter; +set hive.stats.autogather=true; +set hive.exec.dynamic.partition.mode=nonstrict; + +-- partitioned table analyze + +create table dummy (key string, value string) partitioned by (ds string, hr string); + +load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='12'); +load data local inpath '../../data/files/kv1.txt' into table dummy partition (ds='2008',hr='11'); + +analyze table dummy partition (ds,hr) compute statistics; +describe formatted dummy partition (ds='2008', hr='11'); +describe formatted dummy partition (ds='2008', hr='12'); + +drop table dummy; + +-- static partitioned table on insert + +create table dummy (key string, value string) partitioned by (ds string, hr string); + +insert overwrite table dummy partition (ds='10',hr='11') select * from src; +insert overwrite table dummy partition (ds='10',hr='12') select * from src; + +describe formatted dummy partition (ds='10', hr='11'); +describe formatted dummy partition (ds='10', hr='12'); + +drop table dummy; + +-- dynamic partitioned table on insert + +create table dummy (key int) partitioned by (hr int); + +CREATE TABLE tbl(key int, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'; +LOAD DATA LOCAL INPATH '../../data/files/tbl.txt' OVERWRITE INTO TABLE tbl; + +insert overwrite table dummy partition (hr) select * from tbl; + +describe formatted dummy partition (hr=1997); +describe formatted dummy partition (hr=1994); +describe formatted dummy partition (hr=1998); +describe formatted dummy partition (hr=1996); + +drop table tbl; +drop table dummy; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q new file mode 100644 index 0000000000000..a7fce6e3e503f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_invalidation.q @@ -0,0 +1,15 @@ +set hive.stats.autogather=true; + +CREATE TABLE stats_invalid (key string, value string); + +insert overwrite table stats_invalid +select * from src; + +analyze table stats_invalid compute statistics for columns key,value; + +desc formatted stats_invalid; +alter table stats_invalid add columns (new_col string); + +desc formatted stats_invalid; +drop table stats_invalid; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q new file mode 100644 index 0000000000000..5982643741548 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_list_bucket.q @@ -0,0 +1,45 @@ + +set hive.mapred.supports.subdirectories=true; + +-- INCLUDE_HADOOP_MAJOR_VERSIONS(0.23) + +drop table stats_list_bucket; +drop table stats_list_bucket_1; + +create table stats_list_bucket ( + c1 string, + c2 string +) partitioned by (ds string, hr string) +skewed by (c1, c2) on (('466','val_466'),('287','val_287'),('82','val_82')) +stored as directories +stored as rcfile; + +set hive.stats.key.prefix.max.length=1; + +-- Make sure we use hashed IDs during stats publishing. +-- Try partitioned table with list bucketing. +-- The stats should show 500 rows loaded, as many rows as the src table has. + +insert overwrite table stats_list_bucket partition (ds = '2008-04-08', hr = '11') + select key, value from src; + +desc formatted stats_list_bucket partition (ds = '2008-04-08', hr = '11'); + +-- Also try non-partitioned table with list bucketing. +-- Stats should show the same number of rows. + +create table stats_list_bucket_1 ( + c1 string, + c2 string +) +skewed by (c1, c2) on (('466','val_466'),('287','val_287'),('82','val_82')) +stored as directories +stored as rcfile; + +insert overwrite table stats_list_bucket_1 + select key, value from src; + +desc formatted stats_list_bucket_1; + +drop table stats_list_bucket; +drop table stats_list_bucket_1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q index c934fb2051258..b106b30476c00 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q @@ -1,12 +1,12 @@ -- test analyze table compute statistiscs [noscan] on external table -- 1 test table -CREATE EXTERNAL TABLE anaylyze_external (a INT) LOCATION '${system:test.src.data.dir}/files/ext_test'; +CREATE EXTERNAL TABLE anaylyze_external (a INT) LOCATION '${system:hive.root}/data/files/ext_test'; SELECT * FROM anaylyze_external; -analyze table anaylyze_external compute statistics; -describe formatted anaylyze_external; analyze table anaylyze_external compute statistics noscan; describe formatted anaylyze_external; +analyze table anaylyze_external compute statistics; +describe formatted anaylyze_external; drop table anaylyze_external; -- 2 test partition @@ -21,10 +21,10 @@ CREATE EXTERNAL TABLE anaylyze_external (key string, val string) partitioned by ALTER TABLE anaylyze_external ADD PARTITION (insertdate='2008-01-01') location 'pfile://${system:test.tmp.dir}/texternal/2008-01-01'; select count(*) from anaylyze_external where insertdate='2008-01-01'; -- analyze -analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics; -describe formatted anaylyze_external PARTITION (insertdate='2008-01-01'); analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics noscan; describe formatted anaylyze_external PARTITION (insertdate='2008-01-01'); +analyze table anaylyze_external PARTITION (insertdate='2008-01-01') compute statistics; +describe formatted anaylyze_external PARTITION (insertdate='2008-01-01'); dfs -rmr ${system:test.tmp.dir}/texternal; drop table anaylyze_external; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q new file mode 100644 index 0000000000000..b47bc48958c8d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_only_null.q @@ -0,0 +1,41 @@ +set hive.stats.dbclass=fs; +set hive.compute.query.using.stats=true; +set hive.stats.autogather=true; +CREATE TABLE temps_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; + +CREATE TABLE stats_null(a double, b int, c STRING, d smallint) STORED AS TEXTFILE; + +CREATE TABLE stats_null_part(a double, b int, c STRING, d smallint) partitioned by (dt string) STORED AS TEXTFILE; + +LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE temps_null; + +insert overwrite table stats_null select * from temps_null; +insert into table stats_null_part partition(dt='2010') select * from temps_null where d <=5; + +insert into table stats_null_part partition(dt='2011') select * from temps_null where d > 5; +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null; +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null_part; + + +analyze table stats_null compute statistics for columns a,b,c,d; +analyze table stats_null_part partition(dt='2010') compute statistics for columns a,b,c,d; +analyze table stats_null_part partition(dt='2011') compute statistics for columns a,b,c,d; + +describe formatted stats_null_part partition (dt='2010'); +describe formatted stats_null_part partition (dt='2011'); + +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null; +explain +select count(*), count(a), count(b), count(c), count(d) from stats_null_part; + + +select count(*), count(a), count(b), count(c), count(d) from stats_null; +select count(*), count(a), count(b), count(c), count(d) from stats_null_part; +drop table stats_null; +drop table stats_null_part; +drop table temps_null; +set hive.compute.query.using.stats=false; +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q index 6d383f213d1be..513b8e75a0c5e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q @@ -6,7 +6,7 @@ create table tmptable(key string, value string); -set hive.stats.dbclass=dummy; +set hive.stats.dbclass=custom; set hive.stats.default.publisher=org.apache.hadoop.hive.ql.stats.DummyStatsPublisher; set hive.stats.default.aggregator=org.apache.hadoop.hive.ql.stats.DummyStatsAggregator; set hive.stats.reliable=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q new file mode 100644 index 0000000000000..82a2295ac27b4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/statsfs.q @@ -0,0 +1,63 @@ +set hive.stats.dbclass=fs; + +-- stats computation on partitioned table with analyze command + +create table t1 (key string, value string) partitioned by (ds string); +load data local inpath '../../data/files/kv1.txt' into table t1 partition (ds = '2010'); +load data local inpath '../../data/files/kv1.txt' into table t1 partition (ds = '2011'); + +analyze table t1 partition (ds) compute statistics; + +describe formatted t1 partition (ds='2010'); +describe formatted t1 partition (ds='2011'); + +drop table t1; + +-- stats computation on partitioned table with autogather on insert query + +create table t1 (key string, value string) partitioned by (ds string); + +insert into table t1 partition (ds='2010') select * from src; +insert into table t1 partition (ds='2011') select * from src; + +describe formatted t1 partition (ds='2010'); +describe formatted t1 partition (ds='2011'); + +drop table t1; + +-- analyze stmt on unpartitioned table + +create table t1 (key string, value string); +load data local inpath '../../data/files/kv1.txt' into table t1; + +analyze table t1 compute statistics; + +describe formatted t1 ; + +drop table t1; + +-- stats computation on unpartitioned table with autogather on insert query + +create table t1 (key string, value string); + +insert into table t1 select * from src; + +describe formatted t1 ; + +drop table t1; + +-- stats computation on partitioned table with autogather on insert query with dynamic partitioning + + +create table t1 (key string, value string) partitioned by (ds string, hr string); + +set hive.exec.dynamic.partition.mode=nonstrict; +insert into table t1 partition (ds,hr) select * from srcpart; + +describe formatted t1 partition (ds='2008-04-08',hr='11'); +describe formatted t1 partition (ds='2008-04-09',hr='12'); + +drop table t1; +set hive.exec.dynamic.partition.mode=strict; + +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q index c3b206bba6325..ae83407f84333 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + desc function str_to_map; desc function extended str_to_map; @@ -19,7 +21,7 @@ limit 3; drop table tbl_s2m; -create table tbl_s2m as select 'ABC=CC_333=444' as t from src limit 3; +create table tbl_s2m as select 'ABC=CC_333=444' as t from src tablesample (3 rows); select str_to_map(t,'_','=')['333'] from tbl_s2m; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q index 3fb1558a90f0b..14fa321c11c2f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q @@ -2,12 +2,12 @@ EXPLAIN FROM ( FROM src select src.* WHERE src.key < 100 ) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; FROM ( FROM src select src.* WHERE src.key < 100 ) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; -dfs -cat ../build/ql/test/data/warehouse/union.out/*; +dfs -cat ${system:test.warehouse.dir}/union.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q new file mode 100644 index 0000000000000..1d539825424f0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq_where_serialization.q @@ -0,0 +1,5 @@ +set hive.auto.convert.join=true; +set hive.auto.convert.join.noconditionaltask.size=10000000; +explain select src.key from src where src.key in ( select distinct key from src); + +set hive.auto.convert.join=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q new file mode 100644 index 0000000000000..ffc33dc2cf852 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_alias.q @@ -0,0 +1,16 @@ +EXPLAIN +FROM ( + FROM src select src.* WHERE src.key < 100 +) as unioninput +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; + +EXPLAIN +SELECT * FROM +( SELECT * FROM + ( SELECT * FROM src as s ) as src1 +) as src2; + +SELECT * FROM +( SELECT * FROM + ( SELECT * FROM src as s ) as src1 +) as src2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q new file mode 100644 index 0000000000000..f812e36070023 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists.q @@ -0,0 +1,45 @@ + + +-- no agg, corr +explain +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9' + ) +; + +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9' + ) +; + +-- view test +create view cv1 as +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +; + +select * from cv1 +; + +-- sq in from +select * +from (select * + from src b + where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') + ) a +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q new file mode 100644 index 0000000000000..690aa10527a89 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_exists_having.q @@ -0,0 +1,60 @@ + + +-- no agg, corr +explain +select b.key, count(*) +from src b +group by b.key +having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +; + +select b.key, count(*) +from src b +group by b.key +having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +; + +-- view test +create view cv1 as +select b.key, count(*) as c +from src b +group by b.key +having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +; + +select * from cv1; + +-- sq in from +select * +from (select b.key, count(*) + from src b + group by b.key + having exists + (select a.key + from src a + where a.key = b.key and a.value > 'val_9' + ) +) a +; + +-- join on agg +select b.key, min(b.value) +from src b +group by b.key +having exists ( select a.key + from src a + where a.value > 'val_9' and a.value = min(b.value) + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q new file mode 100644 index 0000000000000..69f40f9b8ca9a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in.q @@ -0,0 +1,163 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +DROP TABLE lineitem; +CREATE TABLE lineitem (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; + +-- non agg, non corr +explain + select * +from src +where src.key in (select key from src s1 where s1.key > '9') +; + +select * +from src +where src.key in (select key from src s1 where s1.key > '9') +order by key +; + +-- non agg, corr +explain +select * +from src b +where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +; + +select * +from src b +where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +order by b.key +; + +-- agg, non corr +explain +select p_name, p_size +from +part where part.p_size in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +; +select p_name, p_size +from +part where part.p_size in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +order by p_name +; + +-- agg, corr +explain +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +; + +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, p_name, p_size +; + +-- distinct, corr +explain +select * +from src b +where b.key in + (select distinct a.key + from src a + where b.value = a.value and a.key > '9' + ) +; + +select * +from src b +where b.key in + (select distinct a.key + from src a + where b.value = a.value and a.key > '9' + ) +order by b.key +; + +-- non agg, non corr, windowing +select p_mfgr, p_name, p_size +from part +where part.p_size in + (select first_value(p_size) over(partition by p_mfgr order by p_size) from part) +order by p_mfgr, p_name, p_size +; + +-- non agg, non corr, with join in Parent Query +explain +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +where li.l_linenumber = 1 and + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR') +; + +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +where li.l_linenumber = 1 and + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR') +order by p.p_partkey, li.l_suppkey +; + +-- non agg, corr, with join in Parent Query +select p.p_partkey, li.l_suppkey +from (select distinct l_partkey as p_partkey from lineitem) p join lineitem li on p.p_partkey = li.l_partkey +where li.l_linenumber = 1 and + li.l_orderkey in (select l_orderkey from lineitem where l_shipmode = 'AIR' and l_linenumber = li.l_linenumber) +order by p.p_partkey, li.l_suppkey +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q new file mode 100644 index 0000000000000..84045568f4501 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_in_having.q @@ -0,0 +1,104 @@ + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +-- non agg, non corr +explain + select key, count(*) +from src +group by key +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + + +select s1.key, count(*) from src s1 where s1.key > '9' group by s1.key order by s1.key; + +select key, count(*) +from src +group by key +having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key ) +order by key +; + +-- non agg, corr +explain + select key, value, count(*) +from src b +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' and s1.value = b.value group by s1.key ) +; + +-- agg, non corr +explain +select p_mfgr, avg(p_size) +from part b +group by b.p_mfgr +having b.p_mfgr in + (select p_mfgr + from part + group by p_mfgr + having max(p_size) - min(p_size) < 20 + ) +; + +-- join on agg +select b.key, min(b.value) +from src b +group by b.key +having b.key in ( select a.key + from src a + where a.value > 'val_9' and a.value = min(b.value) + ) +order by b.key +; + +-- where and having +-- Plan is: +-- Stage 1: b semijoin sq1:src (subquery in where) +-- Stage 2: group by Stage 1 o/p +-- Stage 5: group by on sq2:src (subquery in having) +-- Stage 6: Stage 2 o/p semijoin Stage 5 +explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + +set hive.auto.convert.join=true; +-- Plan is: +-- Stage 5: group by on sq2:src (subquery in having) +-- Stage 10: hashtable for sq1:src (subquery in where) +-- Stage 2: b map-side semijoin Stage 10 o/p +-- Stage 3: Stage 2 semijoin Stage 5 +-- Stage 9: construct hastable for Stage 5 o/p +-- Stage 6: Stage 2 map-side semijoin Stage 9 +explain +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + +-- non agg, non corr, windowing +explain +select p_mfgr, p_name, avg(p_size) +from part +group by p_mfgr, p_name +having p_name in + (select first_value(p_name) over(partition by p_mfgr order by p_size) from part) +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q new file mode 100644 index 0000000000000..ed36d9ef6e961 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_multiinsert.q @@ -0,0 +1,82 @@ +set hive.exec.post.hooks=org.apache.hadoop.hive.ql.hooks.PostExecutePrinter,org.apache.hadoop.hive.ql.hooks.PrintCompletedTasksHook; + +CREATE TABLE src_4( + key STRING, + value STRING +) +; + +CREATE TABLE src_5( + key STRING, + value STRING +) +; + +explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +select * from src_4 +; +select * from src_5 +; +set hive.auto.convert.join=true; + +explain +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +from src b +INSERT OVERWRITE TABLE src_4 + select * + where b.key in + (select a.key + from src a + where b.value = a.value and a.key > '9' + ) +INSERT OVERWRITE TABLE src_5 + select * + where b.key not in ( select key from src s1 where s1.key > '2') + order by key +; + +select * from src_4 +; +select * from src_5 +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q new file mode 100644 index 0000000000000..43a801fa9683e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists.q @@ -0,0 +1,41 @@ + + +-- no agg, corr +explain +select * +from src b +where not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_2' + ) +; + +select * +from src b +where not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_2' + ) +; + +-- distinct, corr +explain +select * +from src b +where not exists + (select distinct a.key + from src a + where b.value = a.value and a.value > 'val_2' + ) +; + +select * +from src b +where not exists + (select a.key + from src a + where b.value = a.value and a.value > 'val_2' + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q new file mode 100644 index 0000000000000..7205d17bc4861 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notexists_having.q @@ -0,0 +1,46 @@ + + +-- no agg, corr +explain +select * +from src b +group by key, value +having not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_12' + ) +; + +select * +from src b +group by key, value +having not exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_12' + ) +; + + +-- distinct, corr +explain +select * +from src b +group by key, value +having not exists + (select distinct a.key + from src a + where b.value = a.value and a.value > 'val_12' + ) +; + +select * +from src b +group by key, value +having not exists + (select distinct a.key + from src a + where b.value = a.value and a.value > 'val_12' + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q new file mode 100644 index 0000000000000..d5f60860313da --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin.q @@ -0,0 +1,143 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + +DROP TABLE lineitem; +CREATE TABLE lineitem (L_ORDERKEY INT, + L_PARTKEY INT, + L_SUPPKEY INT, + L_LINENUMBER INT, + L_QUANTITY DOUBLE, + L_EXTENDEDPRICE DOUBLE, + L_DISCOUNT DOUBLE, + L_TAX DOUBLE, + L_RETURNFLAG STRING, + L_LINESTATUS STRING, + l_shipdate STRING, + L_COMMITDATE STRING, + L_RECEIPTDATE STRING, + L_SHIPINSTRUCT STRING, + L_SHIPMODE STRING, + L_COMMENT STRING) +ROW FORMAT DELIMITED +FIELDS TERMINATED BY '|'; + +LOAD DATA LOCAL INPATH '../../data/files/lineitem.txt' OVERWRITE INTO TABLE lineitem; + +-- non agg, non corr +explain +select * +from src +where src.key not in + ( select key from src s1 + where s1.key > '2' + ) +; + +select * +from src +where src.key not in ( select key from src s1 where s1.key > '2') +order by key +; + +-- non agg, corr +explain +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +; + +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, b.p_name +; + +-- agg, non corr +explain +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +; +select p_name, p_size +from +part where part.p_size not in + (select avg(p_size) + from (select p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 + ) +order by p_name, p_size +; + +-- agg, corr +explain +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +; + +select p_mfgr, p_name, p_size +from part b where b.p_size not in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = a.p_mfgr + ) +order by p_mfgr, p_size +; + +-- non agg, non corr, Group By in Parent Query +select li.l_partkey, count(*) +from lineitem li +where li.l_linenumber = 1 and + li.l_orderkey not in (select l_orderkey from lineitem where l_shipmode = 'AIR') +group by li.l_partkey +; + +-- alternate not in syntax +select * +from src +where not src.key in ( select key from src s1 where s1.key > '2') +order by key +; + +-- null check +create view T1_v as +select key from src where key <'11'; + +create view T2_v as +select case when key > '104' then null else key end as key from T1_v; + +explain +select * +from T1_v where T1_v.key not in (select T2_v.key from T2_v); + +select * +from T1_v where T1_v.key not in (select T2_v.key from T2_v); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q new file mode 100644 index 0000000000000..a586f02272a73 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_notin_having.q @@ -0,0 +1,74 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; + + +-- non agg, non corr +explain +select key, count(*) +from src +group by key +having key not in + ( select key from src s1 + where s1.key > '12' + ) +; + +-- non agg, corr +explain +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) +; + +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from (select p_mfgr, min(p_retailprice) l, max(p_retailprice) r, avg(p_retailprice) a from part group by p_mfgr) a + where min(p_retailprice) = l and r - l > 600 + ) +; + +-- agg, non corr +explain +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from part a + group by p_mfgr + having max(p_retailprice) - min(p_retailprice) > 600 + ) +; + +select b.p_mfgr, min(p_retailprice) +from part b +group by b.p_mfgr +having b.p_mfgr not in + (select p_mfgr + from part a + group by p_mfgr + having max(p_retailprice) - min(p_retailprice) > 600 + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q new file mode 100644 index 0000000000000..749435c005134 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_unqualcolumnrefs.q @@ -0,0 +1,83 @@ +DROP TABLE part; + +-- data setup +CREATE TABLE part( + p_partkey INT, + p_name STRING, + p_mfgr STRING, + p_brand STRING, + p_type STRING, + p_size INT, + p_container STRING, + p_retailprice DOUBLE, + p_comment STRING +); + +create table src11 (key1 string, value1 string); + +create table part2( + p2_partkey INT, + p2_name STRING, + p2_mfgr STRING, + p2_brand STRING, + p2_type STRING, + p2_size INT, + p2_container STRING, + p2_retailprice DOUBLE, + p2_comment STRING +); + +-- non agg, corr +explain select * from src11 where src11.key1 in (select key from src where src11.value1 = value and key > '9'); + +explain select * from src a where a.key in (select key from src where a.value = value and key > '9'); + +-- agg, corr +explain +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p2_size) + from (select p2_mfgr, p2_size, rank() over(partition by p2_mfgr order by p2_size) as r from part2) a + where r <= 2 and b.p_mfgr = p2_mfgr + ) +; + + +explain +select p_mfgr, p_name, p_size +from part b where b.p_size in + (select min(p_size) + from (select p_mfgr, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = p_mfgr + ) +; + +-- distinct, corr +explain +select * +from src b +where b.key in + (select distinct key + from src + where b.value = value and key > '9' + ) +; + +-- non agg, corr, having +explain + select key, value, count(*) +from src b +group by key, value +having count(*) in (select count(*) from src where src.key > '9' and src.value = b.value group by key ) +; + +-- non agg, corr +explain +select p_mfgr, b.p_name, p_size +from part b +where b.p_name not in + (select p_name + from (select p_mfgr, p_name, p_size, rank() over(partition by p_mfgr order by p_size) as r from part) a + where r <= 2 and b.p_mfgr = p_mfgr + ) +; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q new file mode 100644 index 0000000000000..9f6712fc181ff --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subquery_views.q @@ -0,0 +1,48 @@ + + +-- exists test +create view cv1 as +select * +from src b +where exists + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_9') +; + +select * +from cv1 where cv1.key in (select key from cv1 c where c.key > '95') order by key; +; + + +-- not in test +create view cv2 as +select * +from src b +where b.key not in + (select a.key + from src a + where b.value = a.value and a.key = b.key and a.value > 'val_11' + ) +; + +select * +from cv2 where cv2.key in (select key from cv2 c where c.key < '11') order by key; +; + +-- in where + having +create view cv3 as +select key, value, count(*) +from src b +where b.key in (select key from src where src.key > '8') +group by key, value +having count(*) in (select count(*) from src s1 where s1.key > '9' group by s1.key ) +; + +select * from cv3 order by key; + + +-- join of subquery views +select * +from cv3 +where cv3.key in (select key from cv1) order by key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q index bb9d6f34ed8ec..d633b97f4c931 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q @@ -1,12 +1,12 @@ - +DROP TABLE IF EXISTS symlink_text_input_format; EXPLAIN CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; CREATE TABLE symlink_text_input_format (key STRING, value STRING) STORED AS INPUTFORMAT 'org.apache.hadoop.hive.ql.io.SymlinkTextInputFormat' OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat'; -dfs -cp ../data/files/symlink1.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink1.txt; -dfs -cp ../data/files/symlink2.txt ../build/ql/test/data/warehouse/symlink_text_input_format/symlink2.txt; +dfs -cp ../../data/files/symlink1.txt ${system:test.warehouse.dir}/symlink_text_input_format/symlink1.txt; +dfs -cp ../../data/files/symlink2.txt ${system:test.warehouse.dir}/symlink_text_input_format/symlink2.txt; EXPLAIN SELECT * FROM symlink_text_input_format order by key, value; @@ -20,5 +20,4 @@ EXPLAIN SELECT count(1) FROM symlink_text_input_format; SELECT count(1) FROM symlink_text_input_format; - - +DROP TABLE symlink_text_input_format; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q index 8b1a390149865..23209d85e4f15 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q @@ -4,7 +4,7 @@ SET hive.stats.collect.tablekeys=true; -- This test is used for testing the TableAccessAnalyzer CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q index d2da5ac174a15..a4f0fdb1c1f26 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q @@ -1,5 +1,5 @@ create table if not exists test_boolean(dummy tinyint); -insert overwrite table test_boolean select 1 from src limit 1; +insert overwrite table test_boolean select 1 from src tablesample (1 rows); SELECT 1 FROM ( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q new file mode 100644 index 0000000000000..87d251f40fd7a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_dml.q @@ -0,0 +1,40 @@ +set hive.exec.dynamic.partition.mode=nonstrict; + +-- CTAS +EXPLAIN CREATE TABLE tmp_src AS SELECT * FROM (SELECT value, count(value) AS cnt FROM src GROUP BY value) f1 ORDER BY cnt; +CREATE TABLE tmp_src AS SELECT * FROM (SELECT value, count(value) AS cnt FROM src GROUP BY value) f1 ORDER BY cnt; + +SELECT * FROM tmp_src; + +-- dyn partitions +CREATE TABLE tmp_src_part (c string) PARTITIONED BY (d int); +EXPLAIN INSERT INTO TABLE tmp_src_part PARTITION (d) SELECT * FROM tmp_src; +INSERT INTO TABLE tmp_src_part PARTITION (d) SELECT * FROM tmp_src; + +SELECT * FROM tmp_src_part; + +-- multi insert +CREATE TABLE even (c int, d string); +CREATE TABLE odd (c int, d string); + +EXPLAIN +FROM src +INSERT INTO TABLE even SELECT key, value WHERE key % 2 = 0 +INSERT INTO TABLE odd SELECT key, value WHERE key % 2 = 1; + +FROM src +INSERT INTO TABLE even SELECT key, value WHERE key % 2 = 0 +INSERT INTO TABLE odd SELECT key, value WHERE key % 2 = 1; + +SELECT * FROM even; +SELECT * FROM odd; + +-- create empty table +CREATE TABLE empty STORED AS orc AS SELECT * FROM tmp_src_part WHERE d = -1000; +SELECT * FROM empty; + +-- drop the tables +DROP TABLE even; +DROP TABLE odd; +DROP TABLE tmp_src; +DROP TABLE tmp_src_part; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q new file mode 100644 index 0000000000000..7f2e28fbafa8d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_fsstat.q @@ -0,0 +1,19 @@ +set hive.execution.engine=tez; +CREATE TABLE tab_part (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; +CREATE TABLE t1 (key int, value string) partitioned by (ds string) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE; + +load data local inpath '../../data/files/srcbucket20.txt' INTO TABLE t1 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket21.txt' INTO TABLE t1 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket22.txt' INTO TABLE t1 partition(ds='2008-04-08'); +load data local inpath '../../data/files/srcbucket23.txt' INTO TABLE t1 partition(ds='2008-04-08'); + +set hive.enforce.bucketing=true; +set hive.enforce.sorting = true; +set hive.optimize.bucketingsorting=false; +set hive.stats.dbclass=fs; + +insert overwrite table tab_part partition (ds='2008-04-08') +select key,value from t1; +describe formatted tab_part partition(ds='2008-04-08'); + +set hive.stats.dbclass=jdbc:derby; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q new file mode 100644 index 0000000000000..d7a652fb8c8bd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_insert_overwrite_local_directory_1.q @@ -0,0 +1,5 @@ +insert overwrite local directory '${system:test.tmp.dir}/tez_local_src_table_1' +select * from src order by key limit 10 ; +dfs -cat file:${system:test.tmp.dir}/tez_local_src_table_1/000000_0 ; + +dfs -rmr file:${system:test.tmp.dir}/tez_local_src_table_1/ ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q new file mode 100644 index 0000000000000..f309e3fe0eed6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_join_tests.q @@ -0,0 +1,12 @@ +explain +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; + +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key)) x right outer join src c on (x.value = c.value) order by x.key; +select * from src1 a left outer join src b on (a.key = b.key) right outer join src c on (a.value = c.value) order by a.key; +select * from src1 a left outer join src b on (a.key = b.key) left outer join src c on (a.value = c.value) order by a.key; +select * from src1 a left outer join src b on (a.key = b.key) join src c on (a.key = c.key); +select * from src1 a join src b on (a.key = b.key) join src c on (a.key = c.key); + +select count(*) from src1 a join src b on (a.key = b.key) join src c on (a.key = c.key); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q new file mode 100644 index 0000000000000..9193843824f6d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_joins_explain.q @@ -0,0 +1,5 @@ +explain +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; + +select * from (select b.key, b.value from src1 a left outer join src b on (a.key = b.key) order by b.key) x right outer join src c on (x.value = c.value) order by x.key; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q new file mode 100644 index 0000000000000..2f1c73f8e528a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_schema_evolution.q @@ -0,0 +1,14 @@ +create table test (key int, value string) partitioned by (p int) stored as textfile; + +insert into table test partition (p=1) select * from src limit 10; + +alter table test set fileformat orc; + +insert into table test partition (p=2) select * from src limit 10; + +describe test; + +select * from test where p=1 and key > 0; +select * from test where p=2 and key > 0; +select * from test where key > 0; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q new file mode 100644 index 0000000000000..f80d94c4a15fd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tez_union.q @@ -0,0 +1,94 @@ +set hive.auto.convert.join=true; + +explain +select s1.key as key, s1.value as value from src s1 join src s3 on s1.key=s3.key +UNION ALL +select s2.key as key, s2.value as value from src s2; + +create table ut as +select s1.key as key, s1.value as value from src s1 join src s3 on s1.key=s3.key +UNION ALL +select s2.key as key, s2.value as value from src s2; + +select * from ut order by key, value limit 20; +drop table ut; + +set hive.auto.convert.join=false; + +explain +with u as (select * from src union all select * from src) +select count(*) from (select u1.key as k1, u2.key as k2 from +u as u1 join u as u2 on (u1.key = u2.key)) a; + +create table ut as +with u as (select * from src union all select * from src) +select count(*) as cnt from (select u1.key as k1, u2.key as k2 from +u as u1 join u as u2 on (u1.key = u2.key)) a; + +select * from ut order by cnt limit 20; +drop table ut; + +set hive.auto.convert.join=true; + +explain select s1.key as skey, u1.key as ukey from +src s1 +join (select * from src union all select * from src) u1 on s1.key = u1.key; + +create table ut as +select s1.key as skey, u1.key as ukey from +src s1 +join (select * from src union all select * from src) u1 on s1.key = u1.key; + +select * from ut order by skey, ukey limit 20; +drop table ut; + +explain select s1.key as skey, u1.key as ukey, s8.key as lkey from +src s1 +join (select s2.key as key from src s2 join src s3 on s2.key = s3.key + union all select s4.key from src s4 join src s5 on s4.key = s5.key + union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1 on (s1.key = u1.key) +join src s8 on (u1.key = s8.key) +order by lkey; + +create table ut as +select s1.key as skey, u1.key as ukey, s8.key as lkey from +src s1 +join (select s2.key as key from src s2 join src s3 on s2.key = s3.key + union all select s4.key from src s4 join src s5 on s4.key = s5.key + union all select s6.key from src s6 join src s7 on s6.key = s7.key) u1 on (s1.key = u1.key) +join src s8 on (u1.key = s8.key) +order by lkey; + +select * from ut order by skey, ukey, lkey limit 100; + +drop table ut; + +explain +select s2.key as key from src s2 join src s3 on s2.key = s3.key +union all select s4.key from src s4 join src s5 on s4.key = s5.key; + +create table ut as +select s2.key as key from src s2 join src s3 on s2.key = s3.key +union all select s4.key from src s4 join src s5 on s4.key = s5.key; + +select * from ut order by key limit 30; + +drop table ut; + +explain +select * from +(select * from src union all select * from src) u +left outer join src s on u.key = s.key; + +explain +select u.key as ukey, s.key as skey from +(select * from src union all select * from src) u +right outer join src s on u.key = s.key; + +create table ut as +select u.key as ukey, s.key as skey from +(select * from src union all select * from src) u +right outer join src s on u.key = s.key; + +select * from ut order by ukey, skey limit 20; +drop table ut; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q index f2c3b596af44a..ce79eefaae4d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q @@ -1,10 +1,12 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_1; create table timestamp_1 (t timestamp); alter table timestamp_1 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; -insert overwrite table timestamp_1 - select cast('2011-01-01 01:01:01' as timestamp) from src limit 1; +insert overwrite table timestamp_1 + select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -15,7 +17,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01' from src limit 1; + select '2011-01-01 01:01:01' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -26,7 +28,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.1' from src limit 1; + select '2011-01-01 01:01:01.1' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -37,7 +39,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.0001' from src limit 1; + select '2011-01-01 01:01:01.0001' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -48,7 +50,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.000100000' from src limit 1; + select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; @@ -59,7 +61,7 @@ select cast(t as double) from timestamp_1 limit 1; select cast(t as string) from timestamp_1 limit 1; insert overwrite table timestamp_1 - select '2011-01-01 01:01:01.001000011' from src limit 1; + select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows); select cast(t as boolean) from timestamp_1 limit 1; select cast(t as tinyint) from timestamp_1 limit 1; select cast(t as smallint) from timestamp_1 limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q index b93208f48c453..351f5ca519499 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q @@ -1,10 +1,12 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_2; create table timestamp_2 (t timestamp); alter table timestamp_2 set serde 'org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe'; -insert overwrite table timestamp_2 - select cast('2011-01-01 01:01:01' as timestamp) from src limit 1; +insert overwrite table timestamp_2 + select cast('2011-01-01 01:01:01' as timestamp) from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -15,7 +17,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01' from src limit 1; + select '2011-01-01 01:01:01' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -26,7 +28,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.1' from src limit 1; + select '2011-01-01 01:01:01.1' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -37,7 +39,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.0001' from src limit 1; + select '2011-01-01 01:01:01.0001' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -48,7 +50,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.000100000' from src limit 1; + select '2011-01-01 01:01:01.000100000' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; @@ -59,7 +61,7 @@ select cast(t as double) from timestamp_2 limit 1; select cast(t as string) from timestamp_2 limit 1; insert overwrite table timestamp_2 - select '2011-01-01 01:01:01.001000011' from src limit 1; + select '2011-01-01 01:01:01.001000011' from src tablesample (1 rows); select cast(t as boolean) from timestamp_2 limit 1; select cast(t as tinyint) from timestamp_2 limit 1; select cast(t as smallint) from timestamp_2 limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q index cda724f9e8f43..0e1a8d5526783 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q @@ -1,10 +1,12 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_3; create table timestamp_3 (t timestamp); alter table timestamp_3 set serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'; -insert overwrite table timestamp_3 - select cast(cast('1.3041352164485E9' as double) as timestamp) from src limit 1; +insert overwrite table timestamp_3 + select cast(cast('1.3041352164485E9' as double) as timestamp) from src tablesample (1 rows); select cast(t as boolean) from timestamp_3 limit 1; select cast(t as tinyint) from timestamp_3 limit 1; select cast(t as smallint) from timestamp_3 limit 1; @@ -14,4 +16,6 @@ select cast(t as float) from timestamp_3 limit 1; select cast(t as double) from timestamp_3 limit 1; select cast(t as string) from timestamp_3 limit 1; +select t, sum(t), count(*), sum(t)/count(*), avg(t) from timestamp_3 group by t; + drop table timestamp_3; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q index f64ae48b85db7..30fee3cbf6013 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q @@ -1,5 +1,6 @@ +set hive.fetch.task.conversion=more; -select cast('2011-05-06 07:08:09' as timestamp) > +select cast('2011-05-06 07:08:09' as timestamp) > cast('2011-05-06 07:08:09' as timestamp) from src limit 1; select cast('2011-05-06 07:08:09' as timestamp) < diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q index 7a1005295eb28..e9a0cfae7cc9a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q @@ -1,6 +1,6 @@ drop table timestamp_lazy; create table timestamp_lazy (t timestamp, key string, value string); -insert overwrite table timestamp_lazy select cast('2011-01-01 01:01:01' as timestamp), key, value from src limit 5; +insert overwrite table timestamp_lazy select cast('2011-01-01 01:01:01' as timestamp), key, value from src tablesample (5 rows); select t,key,value from timestamp_lazy ORDER BY key ASC, value ASC; select t,key,value from timestamp_lazy distribute by t sort by key ASC, value ASC; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q index efd5bc4b78bf4..36f35413e9b48 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS timestamp_null; CREATE TABLE timestamp_null (t1 TIMESTAMP); -LOAD DATA LOCAL INPATH '../data/files/test.dat' OVERWRITE INTO TABLE timestamp_null; +LOAD DATA LOCAL INPATH '../../data/files/test.dat' OVERWRITE INTO TABLE timestamp_null; SELECT * FROM timestamp_null LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q index 2620acefee7f7..ade9fb408c092 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q @@ -1,13 +1,15 @@ +set hive.fetch.task.conversion=more; + drop table timestamp_udf; drop table timestamp_udf_string; create table timestamp_udf (t timestamp); create table timestamp_udf_string (t string); -from src +from (select * from src tablesample (1 rows)) s insert overwrite table timestamp_udf - select '2011-05-06 07:08:09.1234567' limit 1 + select '2011-05-06 07:08:09.1234567' insert overwrite table timestamp_udf_string - select '2011-05-06 07:08:09.1234567' limit 1; + select '2011-05-06 07:08:09.1234567'; -- Test UDFs with Timestamp input select unix_timestamp(t), year(t), month(t), day(t), dayofmonth(t), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q index 962077c2ca565..3bed2b6727e7b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q @@ -12,7 +12,7 @@ SELECT transform(*) USING 'cat' AS (col array) FROM transform1_t1; create table transform1_t2(col array); insert overwrite table transform1_t2 -select array(1,2,3) from src limit 1; +select array(1,2,3) from src tablesample (1 rows); EXPLAIN SELECT transform('0\0021\0022') USING 'cat' AS (col array) FROM transform1_t2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q index d756b47e464d0..0bfb23ead6869 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q @@ -5,7 +5,7 @@ ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' STORED A set hive.stats.autogather=true; -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows); DESC FORMATTED test_tab; @@ -20,7 +20,7 @@ DESC FORMATTED test_tab; SELECT * FROM test_tab ORDER BY value; -- Truncate multiple columns -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows); TRUNCATE TABLE test_tab COLUMNS (key, value); @@ -40,7 +40,7 @@ SELECT * FROM test_tab ORDER BY value; -- Test truncating with a binary serde ALTER TABLE test_tab SET SERDE 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'; -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (10 rows); DESC FORMATTED test_tab; @@ -65,7 +65,7 @@ SELECT * FROM test_tab ORDER BY value; -- Test truncating a partition CREATE TABLE test_tab_part (key STRING, value STRING) PARTITIONED BY (part STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE test_tab_part PARTITION (part = '1') SELECT * FROM src LIMIT 10; +INSERT OVERWRITE TABLE test_tab_part PARTITION (part = '1') SELECT * FROM src tablesample (10 rows); DESC FORMATTED test_tab_part PARTITION (part = '1'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q index a7aab357eaf2f..7a59efc4d1b01 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q @@ -2,9 +2,9 @@ CREATE TABLE test_tab (key STRING, value STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE test_tab SELECT * FROM src LIMIT 5; +INSERT OVERWRITE TABLE test_tab SELECT * FROM src tablesample (5 rows); -INSERT INTO TABLE test_tab SELECT * FROM src LIMIT 5; +INSERT INTO TABLE test_tab SELECT * FROM src tablesample (5 rows); -- The value should be 2 indicating the table has 2 files SELECT COUNT(DISTINCT INPUT__FILE__NAME) FROM test_tab; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q index c0e81e9ac051f..975c0f1ae8426 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q @@ -1,5 +1,5 @@ create table src_truncate (key string, value string); -load data local inpath '../data/files/kv1.txt' into table src_truncate;; +load data local inpath '../../data/files/kv1.txt' into table src_truncate;; create table srcpart_truncate (key string, value string) partitioned by (ds string, hr string); alter table srcpart_truncate add partition (ds='2008-04-08', hr='11'); @@ -7,10 +7,10 @@ alter table srcpart_truncate add partition (ds='2008-04-08', hr='12'); alter table srcpart_truncate add partition (ds='2008-04-09', hr='11'); alter table srcpart_truncate add partition (ds='2008-04-09', hr='12'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='11'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='12'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='11'); -load data local inpath '../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='12'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='11'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-08', hr='12'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='11'); +load data local inpath '../../data/files/kv1.txt' into table srcpart_truncate partition (ds='2008-04-09', hr='12'); set hive.fetch.task.convertion=more; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q index a1607320c7a2d..4d1d978f829ee 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT IF(false, 1, cast(2 as smallint)) + 3 FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q index 63dd66ebedd0c..4c4a828fe0ee2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- casting from null should yield null select diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q index 0d36bc44fe08b..b18c01425c8c6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- Check for int, bigint automatic type widening conversions in UDFs, UNIONS EXPLAIN SELECT COALESCE(0, 9223372036854775807) FROM src LIMIT 1; SELECT COALESCE(0, 9223372036854775807) FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q index 45aaa022dc8a4..04bea32101bdf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q @@ -1,6 +1,9 @@ DESCRIBE FUNCTION collect_set; DESCRIBE FUNCTION EXTENDED collect_set; +DESCRIBE FUNCTION collect_list; +DESCRIBE FUNCTION EXTENDED collect_list; + set hive.map.aggr = false; set hive.groupby.skewindata = false; @@ -8,6 +11,10 @@ SELECT key, collect_set(value) FROM src GROUP BY key ORDER BY key limit 20; +SELECT key, collect_list(value) +FROM src +GROUP BY key ORDER by key limit 20; + set hive.map.aggr = true; set hive.groupby.skewindata = false; @@ -15,6 +22,10 @@ SELECT key, collect_set(value) FROM src GROUP BY key ORDER BY key limit 20; +SELECT key, collect_list(value) +FROM src +GROUP BY key ORDER BY key limit 20; + set hive.map.aggr = false; set hive.groupby.skewindata = true; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q index dda7aaa5f2041..f065385688a1d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q @@ -1,5 +1,5 @@ CREATE TABLE kafka (contents STRING); -LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka; +LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka; set mapred.reduce.tasks=1; set hive.exec.reducers.max=1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q index 6cc9ce2630dd1..a2edec4d64e4c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q @@ -2,7 +2,7 @@ DROP TABLE covar_tab; CREATE TABLE covar_tab (a INT, b INT, c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE +LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE INTO TABLE covar_tab; DESCRIBE FUNCTION corr; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q index 0f5d5f35bf02a..a9937bae3c21c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q @@ -2,7 +2,7 @@ DROP TABLE covar_tab; CREATE TABLE covar_tab (a INT, b INT, c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE +LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE INTO TABLE covar_tab; DESCRIBE FUNCTION covar_pop; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q index 72b9c4bd40049..2b50d8f238140 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q @@ -2,7 +2,7 @@ DROP TABLE covar_tab; CREATE TABLE covar_tab (a INT, b INT, c INT) ROW FORMAT DELIMITED FIELDS TERMINATED BY '\t' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/covar_tab.txt' OVERWRITE +LOAD DATA LOCAL INPATH '../../data/files/covar_tab.txt' OVERWRITE INTO TABLE covar_tab; DESCRIBE FUNCTION covar_samp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q index 31ffd29a88acd..6a2fde52e42f6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q @@ -1,5 +1,5 @@ CREATE TABLE kafka (contents STRING); -LOAD DATA LOCAL INPATH '../data/files/text-en.txt' INTO TABLE kafka; +LOAD DATA LOCAL INPATH '../../data/files/text-en.txt' INTO TABLE kafka; set mapred.reduce.tasks=1; set hive.exec.reducers.max=1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q new file mode 100644 index 0000000000000..8ebf01dcecb8c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile.q @@ -0,0 +1 @@ +select percentile(cast(key as bigint), 0.3) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q index 66c408d71bc1a..5b8ad7a08f5d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q @@ -1,10 +1,10 @@ -- INCLUDE_HADOOP_MAJOR_VERSIONS(0.20, 0.20S) CREATE TABLE bucket (key double, value string) CLUSTERED BY (key) SORTED BY (key DESC) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; create table t1 (result double); create table t2 (result double); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q index 07bfb6e1fb2ab..1efa2951efd27 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q @@ -2,10 +2,10 @@ -- 0.23 changed input order of data in reducer task, which affects result of percentile_approx CREATE TABLE bucket (key double, value string) CLUSTERED BY (key) SORTED BY (key DESC) INTO 4 BUCKETS STORED AS TEXTFILE; -load data local inpath '../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; -load data local inpath '../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket1outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket2outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket3outof4.txt' INTO TABLE bucket; +load data local inpath '../../data/files/srcsortbucket4outof4.txt' INTO TABLE bucket; create table t1 (result double); create table t2 (result double); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q new file mode 100644 index 0000000000000..0d86a42128d25 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_sum_list.q @@ -0,0 +1,6 @@ +-- HIVE-5279 +-- GenericUDAFSumList has Converter which does not have default constructor +-- After +create temporary function sum_list as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSumList'; + +select sum_list(array(key, key)) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q index 113af61062b0d..41bdec08278a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q @@ -1,14 +1,16 @@ -explain -select E() FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select E() FROM src tablesample (1 rows); -select E() FROM src LIMIT 1; +select E() FROM src tablesample (1 rows); DESCRIBE FUNCTION E; DESCRIBE FUNCTION EXTENDED E; explain -select E() FROM src LIMIT 1; +select E() FROM src tablesample (1 rows); -select E() FROM src LIMIT 1; +select E() FROM src tablesample (1 rows); DESCRIBE FUNCTION E; DESCRIBE FUNCTION EXTENDED E; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q index 1fde7df5d251e..945483ecbfea8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q @@ -1,14 +1,16 @@ -explain -select PI() FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select PI() FROM src tablesample (1 rows); -select PI() FROM src LIMIT 1; +select PI() FROM src tablesample (1 rows); DESCRIBE FUNCTION PI; DESCRIBE FUNCTION EXTENDED PI; explain -select PI() FROM src LIMIT 1; +select PI() FROM src tablesample (1 rows); -select PI() FROM src LIMIT 1; +select PI() FROM src tablesample (1 rows); DESCRIBE FUNCTION PI; DESCRIBE FUNCTION EXTENDED PI; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q index f4f227d0dc168..0c06a5b6cd7f9 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION abs; DESCRIBE FUNCTION EXTENDED abs; @@ -7,7 +9,7 @@ EXPLAIN SELECT abs(123), abs(-9223372036854775807), abs(9223372036854775807) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT abs(0), @@ -15,16 +17,16 @@ SELECT abs(123), abs(-9223372036854775807), abs(9223372036854775807) -FROM src LIMIT 1; +FROM src tablesample (1 rows); EXPLAIN SELECT abs(0.0), abs(-3.14159265), abs(3.14159265) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT abs(0.0), abs(-3.14159265), abs(3.14159265) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q index 625a2aa5c6aa6..f9adc16931dd6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION acos; DESCRIBE FUNCTION EXTENDED acos; SELECT acos(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT acos(0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT acos(-0.5), asin(0.66) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT acos(2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q index fca8fe8d1c00c..5a6a1830b1a51 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION array; DESCRIBE FUNCTION EXTENDED array; EXPLAIN SELECT array(), array()[1], array(1, 2, 3), array(1, 2, 3)[2], array(1,"a", 2, 3), array(1,"a", 2, 3)[2], -array(array(1), array(2), array(3), array(4))[1][0] FROM src LIMIT 1; +array(array(1), array(2), array(3), array(4))[1][0] FROM src tablesample (1 rows); SELECT array(), array()[1], array(1, 2, 3), array(1, 2, 3)[2], array(1,"a", 2, 3), array(1,"a", 2, 3)[2], -array(array(1), array(2), array(3), array(4))[1][0] FROM src LIMIT 1; +array(array(1), array(2), array(3), array(4))[1][0] FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q index 937bb0be03bd3..d2dad644065a2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q @@ -1,9 +1,11 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION array_contains; DESCRIBE FUNCTION EXTENDED array_contains; -- evalutes function for array of primitives -SELECT array_contains(array(1, 2, 3), 1) FROM src LIMIT 1; +SELECT array_contains(array(1, 2, 3), 1) FROM src tablesample (1 rows); -- evaluates function for nested arrays SELECT array_contains(array(array(1,2), array(2,3), array(3,4)), array(1,2)) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q index 53b389fd38fd8..3d885a2563520 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION ascii; DESCRIBE FUNCTION EXTENDED ascii; @@ -5,10 +7,10 @@ EXPLAIN SELECT ascii('Facebook'), ascii(''), ascii('!') -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT ascii('Facebook'), ascii(''), ascii('!') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q index f95a5f57df8f8..73b77d10f0b69 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION asin; DESCRIBE FUNCTION EXTENDED asin; SELECT asin(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT asin(0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT asin(-0.5), asin(0.66) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT asin(2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q index d4ef03deb81b0..090438cb0f0cb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q @@ -1,16 +1,18 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION atan; DESCRIBE FUNCTION EXTENDED atan; SELECT atan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT atan(1), atan(6), atan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); DESCRIBE FUNCTION atan; DESCRIBE FUNCTION EXTENDED atan; SELECT atan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT atan(1), atan(6), atan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q index eb3ccea82e631..b22ee9c3cecf1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + describe function between; describe function extended between; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q index 2b9ad62a39dbe..c5a7ac1a60bcd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION bin; DESCRIBE FUNCTION EXTENDED bin; @@ -5,7 +7,7 @@ SELECT bin(1), bin(0), bin(99992421) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Negative numbers should be treated as two's complement (64 bit). -SELECT bin(-5) FROM src LIMIT 1; +SELECT bin(-5) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q index 7ea50dac31d07..ed7711cd6d5de 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q @@ -1,11 +1,13 @@ -select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src limit 1; -select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src limit 1; +set hive.fetch.task.conversion=more; + +select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows); +select ewah_bitmap_and(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src tablesample (1 rows); drop table bitmap_test; create table bitmap_test (a array, b array); insert overwrite table bitmap_test -select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src limit 10; +select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src tablesample (10 rows); select ewah_bitmap_and(a,b) from bitmap_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q index 88e961683e289..142b248cdd250 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q @@ -1,3 +1,5 @@ -select ewah_bitmap_empty(array(13,2,4,8589934592,0,0)) from src limit 1; +set hive.fetch.task.conversion=more; -select ewah_bitmap_empty(array(13,2,4,8589934592,4096,0)) from src limit 1; +select ewah_bitmap_empty(array(13,2,4,8589934592,0,0)) from src tablesample (1 rows); + +select ewah_bitmap_empty(array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q index 0b71e681a53d5..00785b73faa05 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q @@ -1,11 +1,13 @@ -select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src limit 1; -select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src limit 1; +set hive.fetch.task.conversion=more; + +select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(13,2,4,8589934592,4096,0)) from src tablesample (1 rows); +select ewah_bitmap_or(array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0)) from src tablesample (1 rows); drop table bitmap_test; create table bitmap_test (a array, b array); insert overwrite table bitmap_test -select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src limit 10; +select array(13,2,4,8589934592,4096,0), array(8,2,4,8589934592,128,0) from src tablesample (10 rows); select ewah_bitmap_or(a,b) from bitmap_test; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q index 4f71e70e1f3bf..43573bfb12fd8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION case; DESCRIBE FUNCTION EXTENDED case; @@ -27,7 +29,7 @@ SELECT CASE 1 WHEN 22 THEN 23 WHEN 21 THEN 24 END -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT CASE 1 WHEN 1 THEN 2 @@ -54,10 +56,27 @@ SELECT CASE 1 WHEN 22 THEN 23 WHEN 21 THEN 24 END -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- verify that short-circuiting is working correctly for CASE -- we should never get to the ELSE branch, which would raise an exception SELECT CASE 1 WHEN 1 THEN 'yo' ELSE reflect('java.lang.String', 'bogus', 1) END -FROM src LIMIT 1; +FROM src tablesample (1 rows); + +-- Allow compatible types in when/return type +SELECT CASE 1 + WHEN 1 THEN 123.0BD + ELSE 0.0BD + END, + CASE 1 + WHEN 1.0 THEN 123 + WHEN 2 THEN 1.0 + ELSE 222.02BD + END, + CASE 'abc' + WHEN cast('abc' as varchar(3)) THEN 'abcd' + WHEN 'efg' THEN cast('efgh' as varchar(10)) + ELSE cast('ijkl' as char(4)) + END +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q index 736bb053cddc7..2aa76f1f1d82e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN SELECT CASE src_thrift.lint[0] WHEN 0 THEN src_thrift.lint[0] + 1 @@ -14,7 +16,7 @@ SELECT CASE src_thrift.lint[0] WHEN '0' THEN src_thrift.lstring ELSE NULL END)[0] -FROM src_thrift LIMIT 3; +FROM src_thrift tablesample (3 rows); SELECT CASE src_thrift.lint[0] WHEN 0 THEN src_thrift.lint[0] + 1 @@ -31,4 +33,4 @@ SELECT CASE src_thrift.lint[0] WHEN '0' THEN src_thrift.lstring ELSE NULL END)[0] -FROM src_thrift LIMIT 3; +FROM src_thrift tablesample (3 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q index 48ca29cbc3ba6..d3c417babd466 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION coalesce; DESCRIBE FUNCTION EXTENDED coalesce; @@ -20,7 +22,7 @@ SELECT COALESCE(1), COALESCE(NULL, 2.0, 3.0), COALESCE(2.0, NULL, 3.0), COALESCE(IF(TRUE, NULL, 0), NULL) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT COALESCE(1), COALESCE(1, 2), @@ -40,7 +42,7 @@ SELECT COALESCE(1), COALESCE(NULL, 2.0, 3.0), COALESCE(2.0, NULL, 3.0), COALESCE(IF(TRUE, NULL, 0), NULL) -FROM src LIMIT 1; +FROM src tablesample (1 rows); EXPLAIN SELECT COALESCE(src_thrift.lint[1], 999), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q index 6c12f81304e46..c7983b8eb2050 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + EXPLAIN CREATE TEMPORARY FUNCTION test_udf_get_java_string AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFTestGetJavaString'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q index f642f6a2d00df..e35a1cfa170ca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION concat; DESCRIBE FUNCTION EXTENDED concat; @@ -12,4 +14,10 @@ SELECT concat(1, 2), concat(1), concat('1234', 'abc', 'extra argument') -FROM src LIMIT 1; +FROM src tablesample (1 rows); + +-- binary/mixed +SELECT + concat(cast('ab' as binary), cast('cd' as binary)), + concat('ab', cast('cd' as binary)) +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q index 6a0ce20dc37b7..538dfae06f20d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION concat_ws; DESCRIBE FUNCTION EXTENDED concat_ws; @@ -24,7 +26,7 @@ SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), concat_ws('_', array('www', 'face'), array('book', 'com', '1234')), concat_ws('**', 'www', array('face'), array('book', 'com', '1234')), concat_ws('[]', array('www'), 'face', array('book', 'com', '1234')), - concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1; + concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows); SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), concat_ws('-', 'www', array('face', 'book', 'com'), '1234'), @@ -32,7 +34,7 @@ SELECT concat_ws('.', array('www', 'face', 'book', 'com'), '1234'), concat_ws('_', array('www', 'face'), array('book', 'com', '1234')), concat_ws('**', 'www', array('face'), array('book', 'com', '1234')), concat_ws('[]', array('www'), 'face', array('book', 'com', '1234')), - concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1; + concat_ws('AAA', array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows); SELECT concat_ws(NULL, array('www', 'face', 'book', 'com'), '1234'), concat_ws(NULL, 'www', array('face', 'book', 'com'), '1234'), @@ -40,4 +42,4 @@ SELECT concat_ws(NULL, array('www', 'face', 'book', 'com'), '1234'), concat_ws(NULL, array('www', 'face'), array('book', 'com', '1234')), concat_ws(NULL, 'www', array('face'), array('book', 'com', '1234')), concat_ws(NULL, array('www'), 'face', array('book', 'com', '1234')), - concat_ws(NULL, array('www'), array('face', 'book', 'com'), '1234') FROM dest1 LIMIT 1; + concat_ws(NULL, array('www'), array('face', 'book', 'com'), '1234') FROM dest1 tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q index 212bcfb57938f..c6d6cf8600c8c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION conv; DESCRIBE FUNCTION EXTENDED conv; @@ -9,7 +11,7 @@ SELECT conv('22', 10, 10), conv('110011', 2, 16), conv('facebook', 36, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Test negative numbers. If to_base is positive, the number should be handled -- as a two's complement (64-bit) @@ -18,7 +20,7 @@ SELECT conv('1011', 2, -16), conv('-1', 10, 16), conv('-15', 10, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Test overflow. If a number is two large, the result should be -1 (if signed) -- or MAX_LONG (if unsigned) @@ -27,7 +29,7 @@ SELECT conv('9223372036854775807', 36, -16), conv('-9223372036854775807', 36, 16), conv('-9223372036854775807', 36, -16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Test with invalid input. If one of the bases is invalid, the result should -- be NULL. If there is an invalid digit in the number, the longest valid @@ -37,7 +39,7 @@ SELECT conv('131', 1, 5), conv('515', 5, 100), conv('10', -2, 2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Perform the same tests with number arguments. @@ -45,31 +47,31 @@ SELECT conv(4521, 10, 36), conv(22, 10, 10), conv(110011, 2, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT conv(-641, 10, -10), conv(1011, 2, -16), conv(-1, 10, 16), conv(-15, 10, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT conv(9223372036854775807, 36, 16), conv(9223372036854775807, 36, -16), conv(-9223372036854775807, 36, 16), conv(-9223372036854775807, 36, -16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT conv(123455, 3, 10), conv(131, 1, 5), conv(515, 5, 100), conv('10', -2, 2) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Make sure that state is properly reset. SELECT conv(key, 10, 16), conv(key, 16, 10) -FROM src LIMIT 3; +FROM src tablesample (3 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q index 7887c4c2603f1..11ef8d7d87b28 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION cos; DESCRIBE FUNCTION EXTENDED cos; SELECT cos(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT cos(0.98), cos(1.57), cos(-0.5) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q new file mode 100644 index 0000000000000..4ada035d87726 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_current_database.q @@ -0,0 +1,26 @@ +DESCRIBE FUNCTION current_database; + +explain +select current_database(); +select current_database(); + +create database xxx; +use xxx; + +explain +select current_database(); +select current_database(); + +set hive.fetch.task.conversion=more; + +use default; + +explain +select current_database(); +select current_database(); + +use xxx; + +explain +select current_database(); +select current_database(); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q index 014ca1c6e6afb..d5360fe3b22f7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q @@ -1,14 +1,16 @@ -explain -select degrees(PI()) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select degrees(PI()) FROM src tablesample (1 rows); -select degrees(PI()) FROM src LIMIT 1; +select degrees(PI()) FROM src tablesample (1 rows); DESCRIBE FUNCTION degrees; DESCRIBE FUNCTION EXTENDED degrees; explain -select degrees(PI()) FROM src LIMIT 1; +select degrees(PI()) FROM src tablesample (1 rows); -select degrees(PI()) FROM src LIMIT 1; +select degrees(PI()) FROM src tablesample (1 rows); DESCRIBE FUNCTION degrees; DESCRIBE FUNCTION EXTENDED degrees; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q index 4229e625a042a..b0d2844d42346 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION div; DESCRIBE FUNCTION EXTENDED div; -SELECT 3 DIV 2 FROM SRC LIMIT 1; +SELECT 3 DIV 2 FROM SRC tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q index dc4b2e7884d68..d36ba99118f51 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION /; DESCRIBE FUNCTION EXTENDED /; -SELECT 3 / 2 FROM SRC LIMIT 1; +SELECT 3 / 2 FROM SRC tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q index c32340ac89feb..fae764965a9a6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION elt; DESCRIBE FUNCTION EXTENDED elt; @@ -13,7 +15,7 @@ SELECT elt(2, 'abc', 'defg'), elt(null, 'abc', 'defg'), elt(0, 'abc', 'defg'), elt(3, 'abc', 'defg') -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT elt(2, 'abc', 'defg'), elt(3, 'aa', 'bb', 'cc', 'dd', 'ee', 'ff', 'gg'), @@ -26,4 +28,4 @@ SELECT elt(2, 'abc', 'defg'), elt(null, 'abc', 'defg'), elt(0, 'abc', 'defg'), elt(3, 'abc', 'defg') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q index f5e9a7df4be00..ea9b18bf1fad2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q @@ -1,12 +1,14 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION =; DESCRIBE FUNCTION EXTENDED =; DESCRIBE FUNCTION ==; DESCRIBE FUNCTION EXTENDED ==; -SELECT true=false, false=true, false=false, true=true, NULL=NULL, true=NULL, NULL=true, false=NULL, NULL=false FROM src LIMIT 1; +SELECT true=false, false=true, false=false, true=true, NULL=NULL, true=NULL, NULL=true, false=NULL, NULL=false FROM src tablesample (1 rows); DESCRIBE FUNCTION <=>; DESCRIBE FUNCTION EXTENDED <=>; -SELECT true<=>false, false<=>true, false<=>false, true<=>true, NULL<=>NULL, true<=>NULL, NULL<=>true, false<=>NULL, NULL<=>false FROM src LIMIT 1; +SELECT true<=>false, false<=>true, false<=>false, true<=>true, NULL<=>NULL, true<=>NULL, NULL<=>true, false<=>NULL, NULL<=>false FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q index 19af288ff8404..ae651644a778a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q @@ -1,22 +1,24 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION explode; DESCRIBE FUNCTION EXTENDED explode; -EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3; -EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol; +EXPLAIN EXTENDED SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows); +EXPLAIN EXTENDED SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)) a GROUP BY a.myCol; -SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3; -SELECT explode(array(1,2,3)) AS (myCol) FROM src LIMIT 3; -SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src LIMIT 3) a GROUP BY a.myCol; +SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows); +SELECT explode(array(1,2,3)) AS (myCol) FROM src tablesample (1 rows); +SELECT a.myCol, count(1) FROM (SELECT explode(array(1,2,3)) AS myCol FROM src tablesample (1 rows)) a GROUP BY a.myCol; -EXPLAIN EXTENDED SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3; -EXPLAIN EXTENDED SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3) a GROUP BY a.key, a.val; +EXPLAIN EXTENDED SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows); +EXPLAIN EXTENDED SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)) a GROUP BY a.key, a.val; -SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3; -SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src LIMIT 3) a GROUP BY a.key, a.val; +SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows); +SELECT a.key, a.val, count(1) FROM (SELECT explode(map(1,'one',2,'two',3,'three')) AS (key,val) FROM src tablesample (1 rows)) a GROUP BY a.key, a.val; drop table lazy_array_map; create table lazy_array_map (map_col map, array_col array); -INSERT OVERWRITE TABLE lazy_array_map select map(1,'one',2,'two',3,'three'), array('100','200','300') FROM src LIMIT 1; +INSERT OVERWRITE TABLE lazy_array_map select map(1,'one',2,'two',3,'three'), array('100','200','300') FROM src tablesample (1 rows); SELECT array_col, myCol from lazy_array_map lateral view explode(array_col) X AS myCol; SELECT map_col, myKey, myValue from lazy_array_map lateral view explode(map_col) X AS myKey, myValue; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q index e995f5cf3c084..be92c024d3ed2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION field; DESCRIBE FUNCTION EXTENDED field; @@ -5,7 +7,7 @@ SELECT field("x", "a", "b", "c", "d"), field(NULL, "a", "b", "c", "d"), field(0, 1, 2, 3, 4) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT field("a", "a", "b", "c", "d"), @@ -13,7 +15,7 @@ SELECT field("c", "a", "b", "c", "d"), field("d", "a", "b", "c", "d"), field("d", "a", "b", NULL, "d") -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT field(1, 1, 2, 3, 4), @@ -21,11 +23,11 @@ SELECT field(3, 1, 2, 3, 4), field(4, 1, 2, 3, 4), field(4, 1, 2, NULL, 4) -FROM src LIMIT 1; +FROM src tablesample (1 rows); CREATE TABLE test_table(col1 STRING, col2 STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE test_table; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE test_table; select col1,col2, field("66",col1), @@ -42,7 +44,7 @@ from test_table where col1="86" or col1="66"; CREATE TABLE test_table1(col1 int, col2 string) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' INTO TABLE test_table1; +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' INTO TABLE test_table1; select col1,col2, field(66,col1), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q index eac2c6e91c3d1..72c65b419808e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION find_in_set; DESCRIBE FUNCTION EXTENDED find_in_set; @@ -6,18 +8,18 @@ FROM src1 SELECT find_in_set(src1.key,concat(src1.key,',',src1.value)); FROM src1 SELECT find_in_set(src1.key,concat(src1.key,',',src1.value)); -SELECT find_in_set('ab','ab,abc,abcde') FROM src1 LIMIT 1; -SELECT find_in_set('ab','abc,ab,bbb') FROM src1 LIMIT 1; -SELECT find_in_set('ab','def,abc,ab') FROM src1 LIMIT 1; -SELECT find_in_set('ab','abc,abd,abf') FROM src1 LIMIT 1; -SELECT find_in_set(null,'a,b,c') FROM src1 LIMIT 1; -SELECT find_in_set('a',null) FROM src1 LIMIT 1; -SELECT find_in_set('', '') FROM src1 LIMIT 1; -SELECT find_in_set('',',') FROM src1 LIMIT 1; -SELECT find_in_set('','a,,b') FROM src1 LIMIT 1; -SELECT find_in_set('','a,b,') FROM src1 LIMIT 1; -SELECT find_in_set(',','a,b,d,') FROM src1 LIMIT 1; -SELECT find_in_set('a','') FROM src1 LIMIT 1; -SELECT find_in_set('a,','a,b,c,d') FROM src1 LIMIT 1; +SELECT find_in_set('ab','ab,abc,abcde') FROM src1 tablesample (1 rows); +SELECT find_in_set('ab','abc,ab,bbb') FROM src1 tablesample (1 rows); +SELECT find_in_set('ab','def,abc,ab') FROM src1 tablesample (1 rows); +SELECT find_in_set('ab','abc,abd,abf') FROM src1 tablesample (1 rows); +SELECT find_in_set(null,'a,b,c') FROM src1 tablesample (1 rows); +SELECT find_in_set('a',null) FROM src1 tablesample (1 rows); +SELECT find_in_set('', '') FROM src1 tablesample (1 rows); +SELECT find_in_set('',',') FROM src1 tablesample (1 rows); +SELECT find_in_set('','a,,b') FROM src1 tablesample (1 rows); +SELECT find_in_set('','a,b,') FROM src1 tablesample (1 rows); +SELECT find_in_set(',','a,b,d,') FROM src1 tablesample (1 rows); +SELECT find_in_set('a','') FROM src1 tablesample (1 rows); +SELECT find_in_set('a,','a,b,c,d') FROM src1 tablesample (1 rows); SELECT * FROM src1 WHERE NOT find_in_set(key,'311,128,345,2,956')=0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q index e2084cddf0ec7..2504bd0b68306 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + use default; -- Test format_number() UDF @@ -7,26 +9,26 @@ DESCRIBE FUNCTION EXTENDED format_number; EXPLAIN SELECT format_number(12332.123456, 4), format_number(12332.1,4), - format_number(12332.2,0) FROM src limit 1; + format_number(12332.2,0) FROM src tablesample (1 rows); SELECT format_number(12332.123456, 4), format_number(12332.1,4), format_number(12332.2,0) -FROM src limit 1; +FROM src tablesample (1 rows); -- positive numbers SELECT format_number(0.123456789, 12), format_number(12345678.123456789, 5), format_number(1234567.123456789, 7), format_number(123456.123456789, 0) -FROM src limit 1; +FROM src tablesample (1 rows); -- negative numbers SELECT format_number(-123456.123456789, 0), format_number(-1234567.123456789, 2), format_number(-0.123456789, 15), format_number(-12345.123456789, 4) -FROM src limit 1; +FROM src tablesample (1 rows); -- zeros SELECT format_number(0.0, 4), @@ -34,7 +36,7 @@ SELECT format_number(0.0, 4), format_number(000.0000, 1), format_number(00000.0000, 1), format_number(-00.0, 4) -FROM src limit 1; +FROM src tablesample (1 rows); -- integers SELECT format_number(0, 0), @@ -42,7 +44,7 @@ SELECT format_number(0, 0), format_number(12, 2), format_number(123, 5), format_number(1234, 7) -FROM src limit 1; +FROM src tablesample (1 rows); -- long and double boundary -- 9223372036854775807 is LONG_MAX @@ -54,4 +56,4 @@ SELECT format_number(-9223372036854775807, 10), format_number(9223372036854775807, 20), format_number(4.9E-324, 324), format_number(1.7976931348623157E308, 308) -FROM src limit 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q index 464f2df3dcd7a..05f7f5a9811bc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION get_json_object; DESCRIBE FUNCTION EXTENDED get_json_object; @@ -5,6 +7,8 @@ CREATE TABLE dest1(c1 STRING) STORED AS TEXTFILE; FROM src INSERT OVERWRITE TABLE dest1 SELECT ' abc ' WHERE src.key = 86; +set hive.fetch.task.conversion=more; + EXPLAIN SELECT get_json_object(src_json.json, '$.owner') FROM src_json; @@ -33,8 +37,8 @@ SELECT get_json_object(src_json.json, '$.fb:testid') FROM src_json; CREATE TABLE dest2(c1 STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE dest2 SELECT '{"a":"b\nc"}' FROM src LIMIT 1; +INSERT OVERWRITE TABLE dest2 SELECT '{"a":"b\nc"}' FROM src tablesample (1 rows); SELECT * FROM dest2; -SELECT get_json_object(c1, '$.a') FROM dest2; \ No newline at end of file +SELECT get_json_object(c1, '$.a') FROM dest2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q index aea110a9431cf..230bd244ffbc7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION >; DESCRIBE FUNCTION EXTENDED >; -SELECT true>false, false>true, false>false, true>true FROM src LIMIT 1; \ No newline at end of file +SELECT true>false, false>true, false>false, true>true FROM src tablesample (1 rows); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q index 8de165b9ea9ed..025eed7dd5582 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION >=; DESCRIBE FUNCTION EXTENDED >=; -SELECT true>=false, false>=true, false>=false, true>=true FROM src LIMIT 1; \ No newline at end of file +SELECT true>=false, false>=true, false>=false, true>=true FROM src tablesample (1 rows); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q index faf372218a107..5814a1779bd16 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION hash; DESCRIBE FUNCTION EXTENDED hash; @@ -7,11 +9,11 @@ SELECT hash(CAST(1 AS TINYINT)), hash(CAST(2 AS SMALLINT)), hash(CAST(1.25 AS FLOAT)), hash(CAST(16.0 AS DOUBLE)), hash('400'), hash('abc'), hash(TRUE), hash(FALSE), hash(1, 2, 3) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT hash(CAST(1 AS TINYINT)), hash(CAST(2 AS SMALLINT)), hash(3), hash(CAST('123456789012' AS BIGINT)), hash(CAST(1.25 AS FLOAT)), hash(CAST(16.0 AS DOUBLE)), hash('400'), hash('abc'), hash(TRUE), hash(FALSE), hash(1, 2, 3) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q index 37e035ad42b00..0e5457965a71e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION hex; DESCRIBE FUNCTION EXTENDED hex; @@ -7,14 +9,14 @@ SELECT hex('Facebook'), hex('\0'), hex('qwertyuiopasdfghjkl') -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- If the argument is a number, hex should convert it to hexadecimal. SELECT hex(1), hex(0), hex(4207849477) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Negative numbers should be treated as two's complement (64 bit). -SELECT hex(-5) FROM src LIMIT 1; +SELECT hex(-5) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q index c5c366daa8c83..b9811e6c6f652 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION hour; DESCRIBE FUNCTION EXTENDED hour; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q index 4f7c8b4a36bad..d9285ff7ce19d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION if; DESCRIBE FUNCTION EXTENDED if; @@ -8,7 +10,7 @@ SELECT IF(TRUE, 1, 2) AS COL1, IF(2=2, 1, NULL) AS COL4, IF(2=2, NULL, 1) AS COL5, IF(IF(TRUE, NULL, FALSE), 1, 2) AS COL6 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT IF(TRUE, 1, 2) AS COL1, @@ -17,7 +19,7 @@ SELECT IF(TRUE, 1, 2) AS COL1, IF(2=2, 1, NULL) AS COL4, IF(2=2, NULL, 1) AS COL5, IF(IF(TRUE, NULL, FALSE), 1, 2) AS COL6 -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- Type conversions EXPLAIN @@ -25,10 +27,10 @@ SELECT IF(TRUE, CAST(128 AS SMALLINT), CAST(1 AS TINYINT)) AS COL1, IF(FALSE, 1, 1.1) AS COL2, IF(FALSE, 1, 'ABC') AS COL3, IF(FALSE, 'ABC', 12.3) AS COL4 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT IF(TRUE, CAST(128 AS SMALLINT), CAST(1 AS TINYINT)) AS COL1, IF(FALSE, 1, 1.1) AS COL2, IF(FALSE, 1, 'ABC') AS COL3, IF(FALSE, 'ABC', 12.3) AS COL4 -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q index 75778138742d5..a7ce3c6f0bdbb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + SELECT 1 IN (1, 2, 3), 4 IN (1, 2, 3), array(1,2,3) IN (array(1,2,3)), @@ -8,6 +10,6 @@ SELECT 1 IN (1, 2, 3), 1 IN (1, 2, 3) OR false IN(false), NULL IN (1, 2, 3), 4 IN (1, 2, 3, NULL), - (1+3) IN (5, 6, (1+2) + 1) FROM src LIMIT 1; + (1+3) IN (5, 6, (1+2) + 1) FROM src tablesample (1 rows); SELECT key FROM src WHERE key IN ("238", 86); \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q index 4da478908ddbd..9d9efe8e23d6e 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q @@ -1,12 +1,12 @@ DESCRIBE FUNCTION in_file; EXPLAIN -SELECT in_file("303", "../data/files/test2.dat"), - in_file("304", "../data/files/test2.dat"), - in_file(CAST(NULL AS STRING), "../data/files/test2.dat") +SELECT in_file("303", "../../data/files/test2.dat"), + in_file("304", "../../data/files/test2.dat"), + in_file(CAST(NULL AS STRING), "../../data/files/test2.dat") FROM src LIMIT 1; -SELECT in_file("303", "../data/files/test2.dat"), - in_file("304", "../data/files/test2.dat"), - in_file(CAST(NULL AS STRING), "../data/files/test2.dat") +SELECT in_file("303", "../../data/files/test2.dat"), + in_file("304", "../../data/files/test2.dat"), + in_file(CAST(NULL AS STRING), "../../data/files/test2.dat") FROM src LIMIT 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q index 9079d0e5743f6..6844f9d0bfc6c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q @@ -1,2 +1,3 @@ +set hive.support.quoted.identifiers=none; DESCRIBE FUNCTION `index`; DESCRIBE FUNCTION EXTENDED `index`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q index 39006f4b9b117..95d55f71c11cd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + describe function inline; explain SELECT inline( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q index 20ed8e4ea0176..790a1049d1777 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION instr; DESCRIBE FUNCTION EXTENDED instr; @@ -15,7 +17,7 @@ SELECT instr('abcd', 'abc'), instr(CAST(16.0 AS DOUBLE), '.0'), instr(null, 'abc'), instr('abcd', null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT instr('abcd', 'abc'), instr('abcabc', 'ccc'), @@ -30,4 +32,4 @@ SELECT instr('abcd', 'abc'), instr(CAST(16.0 AS DOUBLE), '.0'), instr(null, 'abc'), instr('abcd', null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q index d1569cc7f6dbb..efb834efdc64f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION isnull; DESCRIBE FUNCTION EXTENDED isnull; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q index 2f28be1e9d0e1..51280b2567cca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION java_method; DESCRIBE FUNCTION EXTENDED java_method; @@ -11,7 +13,7 @@ SELECT java_method("java.lang.String", "valueOf", 1), java_method("java.lang.Math", "round", 2.5), java_method("java.lang.Math", "exp", 1.0), java_method("java.lang.Math", "floor", 1.9) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT java_method("java.lang.String", "valueOf", 1), @@ -21,5 +23,5 @@ SELECT java_method("java.lang.String", "valueOf", 1), java_method("java.lang.Math", "round", 2.5), java_method("java.lang.Math", "exp", 1.0), java_method("java.lang.Math", "floor", 1.9) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q index b84307970d630..4413751ae647b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION length; DESCRIBE FUNCTION EXTENDED length; @@ -9,6 +11,6 @@ DROP TABLE dest1; -- Test with non-ascii characters. CREATE TABLE dest1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE dest1; +LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE dest1; EXPLAIN SELECT length(dest1.name) FROM dest1; SELECT length(dest1.name) FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q index a2577006a885d..03326777bfa9f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q @@ -1,4 +1,6 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION <; DESCRIBE FUNCTION EXTENDED <; -SELECT true; DESCRIBE FUNCTION EXTENDED <>; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q index 88396545de933..dceab7edaa1fa 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + SELECT 1 NOT IN (1, 2, 3), 4 NOT IN (1, 2, 3), 1 = 2 NOT IN (true, false), @@ -7,4 +9,4 @@ SELECT 1 NOT IN (1, 2, 3), "abc" NOT RLIKE "^bc", "abc" NOT REGEXP "^ab", "abc" NOT REGEXP "^bc", - 1 IN (1, 2) AND "abc" NOT LIKE "bc%" FROM src LIMIT 1; + 1 IN (1, 2) AND "abc" NOT LIKE "bc%" FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q index 0133b4b90bcbc..97162576df167 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; DESCRIBE FUNCTION nvl; DESCRIBE FUNCTION EXTENDED nvl; @@ -5,9 +6,9 @@ DESCRIBE FUNCTION EXTENDED nvl; EXPLAIN SELECT NVL( 1 , 2 ) AS COL1, NVL( NULL, 5 ) AS COL2 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT NVL( 1 , 2 ) AS COL1, NVL( NULL, 5 ) AS COL2 -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q index 9ff73d42b41e6..d42a2f337ebca 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q @@ -1,20 +1,22 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION pmod; DESCRIBE FUNCTION EXTENDED pmod; SELECT pmod(null, null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT pmod(-100,9), pmod(-50,101), pmod(-1000,29) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT pmod(100,19), pmod(50,125), pmod(300,15) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -SELECT pmod(CAST(-100 AS TINYINT),CAST(9 AS TINYINT)), pmod(CAST(-50 AS TINYINT),CAST(101 AS TINYINT)), pmod(CAST(-100 AS TINYINT),CAST(29 AS TINYINT)) FROM src LIMIT 1; -SELECT pmod(CAST(-100 AS SMALLINT),CAST(9 AS SMALLINT)), pmod(CAST(-50 AS SMALLINT),CAST(101 AS SMALLINT)), pmod(CAST(-100 AS SMALLINT),CAST(29 AS SMALLINT)) FROM src LIMIT 1; -SELECT pmod(CAST(-100 AS BIGINT),CAST(9 AS BIGINT)), pmod(CAST(-50 AS BIGINT),CAST(101 AS BIGINT)), pmod(CAST(-100 AS BIGINT),CAST(29 AS BIGINT)) FROM src LIMIT 1; +SELECT pmod(CAST(-100 AS TINYINT),CAST(9 AS TINYINT)), pmod(CAST(-50 AS TINYINT),CAST(101 AS TINYINT)), pmod(CAST(-100 AS TINYINT),CAST(29 AS TINYINT)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100 AS SMALLINT),CAST(9 AS SMALLINT)), pmod(CAST(-50 AS SMALLINT),CAST(101 AS SMALLINT)), pmod(CAST(-100 AS SMALLINT),CAST(29 AS SMALLINT)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100 AS BIGINT),CAST(9 AS BIGINT)), pmod(CAST(-50 AS BIGINT),CAST(101 AS BIGINT)), pmod(CAST(-100 AS BIGINT),CAST(29 AS BIGINT)) FROM src tablesample (1 rows); -SELECT pmod(CAST(-100.91 AS FLOAT),CAST(9.8 AS FLOAT)), pmod(CAST(-50.1 AS FLOAT),CAST(101.8 AS FLOAT)), pmod(CAST(-100.91 AS FLOAT),CAST(29.75 AS FLOAT)) FROM src LIMIT 1; -SELECT pmod(CAST(-100.91 AS DOUBLE),CAST(9.8 AS DOUBLE)), pmod(CAST(-50.1 AS DOUBLE),CAST(101.8 AS DOUBLE)), pmod(CAST(-100.91 AS DOUBLE),CAST(29.75 AS DOUBLE)) FROM src LIMIT 1; -SELECT pmod(CAST(-100.91 AS DECIMAL),CAST(9.8 AS DECIMAL)), pmod(CAST(-50.1 AS DECIMAL),CAST(101.8 AS DECIMAL)), pmod(CAST(-100.91 AS DECIMAL),CAST(29.75 AS DECIMAL)) FROM src LIMIT 1; +SELECT pmod(CAST(-100.91 AS FLOAT),CAST(9.8 AS FLOAT)), pmod(CAST(-50.1 AS FLOAT),CAST(101.8 AS FLOAT)), pmod(CAST(-100.91 AS FLOAT),CAST(29.75 AS FLOAT)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100.91 AS DOUBLE),CAST(9.8 AS DOUBLE)), pmod(CAST(-50.1 AS DOUBLE),CAST(101.8 AS DOUBLE)), pmod(CAST(-100.91 AS DOUBLE),CAST(29.75 AS DOUBLE)) FROM src tablesample (1 rows); +SELECT pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(9.8 AS DECIMAL(2,1))), pmod(CAST(-50.1 AS DECIMAL(3,1)),CAST(101.8 AS DECIMAL(4,1))), pmod(CAST(-100.91 AS DECIMAL(5,2)),CAST(29.75 AS DECIMAL(4,2))) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q index 99e89ccbf773a..115e4e56f1806 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q @@ -4,21 +4,23 @@ use default; DESCRIBE FUNCTION printf; DESCRIBE FUNCTION EXTENDED printf; +set hive.fetch.task.conversion=more; + EXPLAIN -SELECT printf("Hello World %d %s", 100, "days") FROM src LIMIT 1; +SELECT printf("Hello World %d %s", 100, "days") FROM src tablesample (1 rows); -- Test Primitive Types -SELECT printf("Hello World %d %s", 100, "days") FROM src LIMIT 1; -SELECT printf("All Type Test: %b, %c, %d, %e, %+10.4f, %g, %h, %s, %a", false, 65, 15000, 12.3400, 27183.240051, 2300.41, 50, "corret", 256.125) FROM src LIMIT 1; +SELECT printf("Hello World %d %s", 100, "days") FROM src tablesample (1 rows); +SELECT printf("All Type Test: %b, %c, %d, %e, %+10.4f, %g, %h, %s, %a", false, 65, 15000, 12.3400, 27183.240051, 2300.41, 50, "corret", 256.125) FROM src tablesample (1 rows); -- Test NULL Values -SELECT printf("Color %s, String Null: %s, number1 %d, number2 %05d, Integer Null: %d, hex %#x, float %5.2f Double Null: %f\n", "red", NULL, 123456, 89, NULL, 255, 3.14159, NULL) FROM src LIMIT 1; +SELECT printf("Color %s, String Null: %s, number1 %d, number2 %05d, Integer Null: %d, hex %#x, float %5.2f Double Null: %f\n", "red", NULL, 123456, 89, NULL, 255, 3.14159, NULL) FROM src tablesample (1 rows); -- Test Timestamp create table timestamp_udf (t timestamp); -from src +from (select * from src tablesample (1 rows)) s insert overwrite table timestamp_udf - select '2011-05-06 07:08:09.1234567' limit 1; + select '2011-05-06 07:08:09.1234567'; select printf("timestamp: %s", t) from timestamp_udf; drop table timestamp_udf; @@ -27,7 +29,7 @@ CREATE TABLE binay_udf(key binary, value int) ROW FORMAT DELIMITED FIELDS TERMINATED BY '9' STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/string.txt' INTO TABLE binay_udf; +LOAD DATA LOCAL INPATH '../../data/files/string.txt' INTO TABLE binay_udf; create table dest1 (key binary, value int); insert overwrite table dest1 select transform(*) using 'cat' as key binary, value int from binay_udf; select value, printf("format key: %s", key) from dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q index 001d1cf8b6464..19242bd757b41 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q @@ -1,16 +1,18 @@ -explain -select radians(57.2958) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select radians(57.2958) FROM src tablesample (1 rows); -select radians(57.2958) FROM src LIMIT 1; -select radians(143.2394) FROM src LIMIT 1; +select radians(57.2958) FROM src tablesample (1 rows); +select radians(143.2394) FROM src tablesample (1 rows); DESCRIBE FUNCTION radians; DESCRIBE FUNCTION EXTENDED radians; explain -select radians(57.2958) FROM src LIMIT 1; +select radians(57.2958) FROM src tablesample (1 rows); -select radians(57.2958) FROM src LIMIT 1; -select radians(143.2394) FROM src LIMIT 1; +select radians(57.2958) FROM src tablesample (1 rows); +select radians(143.2394) FROM src tablesample (1 rows); DESCRIBE FUNCTION radians; DESCRIBE FUNCTION EXTENDED radians; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q index f357ff5049564..cef1e4a5d9195 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION reflect; DESCRIBE FUNCTION EXTENDED reflect; @@ -10,7 +12,7 @@ SELECT reflect("java.lang.String", "valueOf", 1), reflect("java.lang.Math", "exp", 1.0), reflect("java.lang.Math", "floor", 1.9), reflect("java.lang.Integer", "valueOf", key, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT reflect("java.lang.String", "valueOf", 1), @@ -21,4 +23,4 @@ SELECT reflect("java.lang.String", "valueOf", 1), reflect("java.lang.Math", "exp", 1.0), reflect("java.lang.Math", "floor", 1.9), reflect("java.lang.Integer", "valueOf", key, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q index 9ffd755292f66..a65294b335844 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION reflect2; DESCRIBE FUNCTION EXTENDED reflect2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q index 3aee10963d747..12b685b32c69c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q @@ -1,6 +1,8 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION regexp; DESCRIBE FUNCTION EXTENDED regexp; SELECT 'fofo' REGEXP '^fo', 'fo\no' REGEXP '^fo\no$', 'Bn' REGEXP '^Ba*n', 'afofo' REGEXP 'fo', 'afofo' REGEXP '^fo', 'Baan' REGEXP '^Ba?n', 'axe' REGEXP 'pi|apa', 'pip' REGEXP '^(pi)*$' -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q index 162085f4c71c8..91474bac2a16b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION repeat; DESCRIBE FUNCTION EXTENDED repeat; @@ -6,11 +8,11 @@ EXPLAIN SELECT repeat("", 4), repeat("asd", 0), repeat("asdf", -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT repeat("Facebook", 3), repeat("", 4), repeat("asd", 0), repeat("asdf", -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q index 81f765ec5937c..89aafe3443eeb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q @@ -11,5 +11,5 @@ DROP TABLE dest1; -- kv4.txt contains the text 0xE982B5E993AE, which should be reversed to -- 0xE993AEE982B5 CREATE TABLE dest1(name STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv4.txt' INTO TABLE dest1; +LOAD DATA LOCAL INPATH '../../data/files/kv4.txt' INTO TABLE dest1; SELECT count(1) FROM dest1 WHERE reverse(dest1.name) = _UTF-8 0xE993AEE982B5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q index 18ebba8708493..88b22749a3095 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q @@ -1,15 +1,17 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION round; DESCRIBE FUNCTION EXTENDED round; SELECT round(null), round(null, 0), round(125, null), round(1.0/0.0, 0), round(power(-1.0,0.5), 0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT round(55555), round(55555, 0), round(55555, 1), round(55555, 2), round(55555, 3), round(55555, -1), round(55555, -2), round(55555, -3), round(55555, -4), round(55555, -5), round(55555, -6), round(55555, -7), round(55555, -8) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT round(125.315), round(125.315, 0), @@ -18,7 +20,7 @@ SELECT round(-125.315), round(-125.315, 0), round(-125.315, 1), round(-125.315, 2), round(-125.315, 3), round(-125.315, 4), round(-125.315, -1), round(-125.315, -2), round(-125.315, -3), round(-125.315, -4) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT round(3.141592653589793, -15), round(3.141592653589793, -16), @@ -38,7 +40,7 @@ SELECT round(3.141592653589793, 12), round(3.141592653589793, 13), round(3.141592653589793, 13), round(3.141592653589793, 14), round(3.141592653589793, 15), round(3.141592653589793, 16) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -SELECT round(1809242.3151111344, 9), round(-1809242.3151111344, 9) -FROM src LIMIT 1; +SELECT round(1809242.3151111344, 9), round(-1809242.3151111344, 9), round(1809242.3151111344BD, 9), round(-1809242.3151111344BD, 9) +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q index 6be30855aea1b..43988c1225cd8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q @@ -1,8 +1,10 @@ --- test for NaN (not-a-number) +set hive.fetch.task.conversion=more; + +-- test for NaN (not-a-number) create table tstTbl1(n double); insert overwrite table tstTbl1 -select 'NaN' from src limit 1; +select 'NaN' from src tablesample (1 rows); select * from tstTbl1; @@ -10,4 +12,4 @@ select round(n, 1) from tstTbl1; select round(n) from tstTbl1; -- test for Infinity -select round(1/0), round(1/0, 2), round(1.0/0.0), round(1.0/0.0, 2) from src limit 1; +select round(1/0), round(1/0, 2), round(1.0/0.0), round(1.0/0.0, 2) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q index 50a1f44419047..f042b6f3fa9c7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + -- test for TINYINT -select round(-128), round(127), round(0) from src limit 1; +select round(-128), round(127), round(0) from src tablesample (1 rows); -- test for SMALLINT -select round(-32768), round(32767), round(-129), round(128) from src limit 1; +select round(-32768), round(32767), round(-129), round(128) from src tablesample (1 rows); -- test for INT -select round(cast(negative(pow(2, 31)) as INT)), round(cast((pow(2, 31) - 1) as INT)), round(-32769), round(32768) from src limit 1; +select round(cast(negative(pow(2, 31)) as INT)), round(cast((pow(2, 31) - 1) as INT)), round(-32769), round(32768) from src tablesample (1 rows); -- test for BIGINT -select round(cast(negative(pow(2, 63)) as BIGINT)), round(cast((pow(2, 63) - 1) as BIGINT)), round(cast(negative(pow(2, 31) + 1) as BIGINT)), round(cast(pow(2, 31) as BIGINT)) from src limit 1; +select round(cast(negative(pow(2, 63)) as BIGINT)), round(cast((pow(2, 63) - 1) as BIGINT)), round(cast(negative(pow(2, 31) + 1) as BIGINT)), round(cast(pow(2, 31) as BIGINT)) from src tablesample (1 rows); -- test for DOUBLE -select round(126.1), round(126.7), round(32766.1), round(32766.7) from src limit 1; +select round(126.1), round(126.7), round(32766.1), round(32766.7) from src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q index 01e5fbd429b17..4ee69e8985075 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION rpad; DESCRIBE FUNCTION EXTENDED rpad; @@ -5,10 +7,10 @@ EXPLAIN SELECT rpad('hi', 1, '?'), rpad('hi', 5, '.'), rpad('hi', 6, '123') -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT rpad('hi', 1, '?'), rpad('hi', 5, '.'), rpad('hi', 6, '123') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q index 1943188086ea5..f63426d7e7257 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION second; DESCRIBE FUNCTION EXTENDED second; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q index abceb3439ec1e..b1602e8a257ef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q @@ -1,20 +1,22 @@ -explain -select sign(0) FROM src LIMIT 1; -select sign(0) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; + +explain +select sign(0) FROM src tablesample (1 rows); +select sign(0) FROM src tablesample (1 rows); -select sign(-45) FROM src LIMIT 1; +select sign(-45) FROM src tablesample (1 rows); -select sign(46) FROM src LIMIT 1; +select sign(46) FROM src tablesample (1 rows); DESCRIBE FUNCTION sign; DESCRIBE FUNCTION EXTENDED sign; explain -select sign(0) FROM src LIMIT 1; -select sign(0) FROM src LIMIT 1; +select sign(0) FROM src tablesample (1 rows); +select sign(0) FROM src tablesample (1 rows); -select sign(-45) FROM src LIMIT 1; +select sign(-45) FROM src tablesample (1 rows); -select sign(46) FROM src LIMIT 1; +select sign(46) FROM src tablesample (1 rows); DESCRIBE FUNCTION sign; DESCRIBE FUNCTION EXTENDED sign; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q index abb7cac8dac9d..79745be772f6f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION sin; DESCRIBE FUNCTION EXTENDED sin; SELECT sin(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT sin(0.98), sin(1.57), sin(-0.5) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q index 8aaa68a6e318a..f6f76a30e7258 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION size; DESCRIBE FUNCTION EXTENDED size; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q index ef0973212a27c..313bcf8a1ea52 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + use default; -- Test sort_array() UDF @@ -6,16 +8,16 @@ DESCRIBE FUNCTION EXTENDED sort_array; -- Evaluate function against STRING valued keys EXPLAIN -SELECT sort_array(array("b", "d", "c", "a")) FROM src LIMIT 1; +SELECT sort_array(array("b", "d", "c", "a")) FROM src tablesample (1 rows); -SELECT sort_array(array("f", "a", "g", "c", "b", "d", "e")) FROM src LIMIT 1; -SELECT sort_array(sort_array(array("hadoop distributed file system", "enterprise databases", "hadoop map-reduce"))) FROM src LIMIT 1; +SELECT sort_array(array("f", "a", "g", "c", "b", "d", "e")) FROM src tablesample (1 rows); +SELECT sort_array(sort_array(array("hadoop distributed file system", "enterprise databases", "hadoop map-reduce"))) FROM src tablesample (1 rows); -- Evaluate function against INT valued keys -SELECT sort_array(array(2, 9, 7, 3, 5, 4, 1, 6, 8)) FROM src LIMIT 1; +SELECT sort_array(array(2, 9, 7, 3, 5, 4, 1, 6, 8)) FROM src tablesample (1 rows); -- Evaluate function against FLOAT valued keys -SELECT sort_array(sort_array(array(2.333, 9, 1.325, 2.003, 0.777, -3.445, 1))) FROM src LIMIT 1; +SELECT sort_array(sort_array(array(2.333, 9, 1.325, 2.003, 0.777, -3.445, 1))) FROM src tablesample (1 rows); -- Test it against data in a table. CREATE TABLE dest1 ( @@ -30,7 +32,7 @@ CREATE TABLE dest1 ( timestamps ARRAY ) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/primitive_type_arrays.txt' OVERWRITE INTO TABLE dest1; +LOAD DATA LOCAL INPATH '../../data/files/primitive_type_arrays.txt' OVERWRITE INTO TABLE dest1; SELECT sort_array(tinyints), sort_array(smallints), sort_array(ints), sort_array(bigints), sort_array(booleans), sort_array(floats), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q index cf6466fb63bad..cc616f784fcea 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION space; DESCRIBE FUNCTION EXTENDED space; @@ -7,7 +9,7 @@ EXPLAIN SELECT space(1), space(-1), space(-100) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT length(space(10)), @@ -15,7 +17,7 @@ SELECT length(space(1)), length(space(-1)), length(space(-100)) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT space(10), @@ -23,5 +25,5 @@ SELECT space(1), space(-1), space(-100) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q index f79901736cf7c..55919eac743b7 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION split; DESCRIBE FUNCTION EXTENDED split; @@ -6,11 +8,11 @@ EXPLAIN SELECT split('oneAtwoBthreeC', '[ABC]'), split('', '.'), split(50401020, 0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT split('a b c', ' '), split('oneAtwoBthreeC', '[ABC]'), split('', '.'), split(50401020, 0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q index 3ee19c83699bb..ee2135b509ae2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q @@ -1,9 +1,11 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION struct; DESCRIBE FUNCTION EXTENDED struct; EXPLAIN SELECT struct(1), struct(1, "a"), struct(1, "b", 1.5).col1, struct(1, struct("a", 1.5)).col2.col1 -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT struct(1), struct(1, "a"), struct(1, "b", 1.5).col1, struct(1, struct("a", 1.5)).col2.col1 -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q index 32757bef51c1c..2d04f904bbc6f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION substr; DESCRIBE FUNCTION EXTENDED substr; @@ -5,7 +7,7 @@ SELECT substr(null, 1), substr(null, 1, 1), substr('ABC', null), substr('ABC', null, 1), substr('ABC', 1, null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABC', 1, 0), substr('ABC', 1, -1), substr('ABC', 2, -100), @@ -14,7 +16,7 @@ SELECT substr('ABC', 100), substr('ABC', 100, 100), substr('ABC', -100), substr('ABC', -100, 100), substr('ABC', 2147483647), substr('ABC', 2147483647, 2147483647) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABCDEFG', 3, 4), substr('ABCDEFG', -5, 4), @@ -22,7 +24,7 @@ SELECT substr('ABC', 0), substr('ABC', 1), substr('ABC', 2), substr('ABC', 3), substr('ABC', 1, 2147483647), substr('ABC', 2, 2147483647), substr('A', 0), substr('A', 1), substr('A', -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABC', 0, 1), substr('ABC', 0, 2), substr('ABC', 0, 3), substr('ABC', 0, 4), @@ -30,14 +32,14 @@ SELECT substr('ABC', 2, 1), substr('ABC', 2, 2), substr('ABC', 2, 3), substr('ABC', 2, 4), substr('ABC', 3, 1), substr('ABC', 3, 2), substr('ABC', 3, 3), substr('ABC', 3, 4), substr('ABC', 4, 1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT substr('ABC', -1, 1), substr('ABC', -1, 2), substr('ABC', -1, 3), substr('ABC', -1, 4), substr('ABC', -2, 1), substr('ABC', -2, 2), substr('ABC', -2, 3), substr('ABC', -2, 4), substr('ABC', -3, 1), substr('ABC', -3, 2), substr('ABC', -3, 3), substr('ABC', -3, 4), substr('ABC', -4, 1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- substring() is a synonim of substr(), so just perform some basic tests SELECT @@ -46,7 +48,7 @@ SELECT substring('ABC', 0), substring('ABC', 1), substring('ABC', 2), substring('ABC', 3), substring('ABC', 1, 2147483647), substring('ABC', 2, 2147483647), substring('A', 0), substring('A', 1), substring('A', -1) -FROM src LIMIT 1; +FROM src tablesample (1 rows); -- test for binary substr SELECT @@ -63,7 +65,7 @@ SELECT substr(ABC, -3, 1), substr(ABC, -3, 2), substr(ABC, -3, 3), substr(ABC, -3, 4), substr(ABC, -4, 1) FROM ( - select CAST(concat(substr(value, 1, 0), 'ABC') as BINARY) as ABC from src LIMIT 1 + select CAST(concat(substr(value, 1, 0), 'ABC') as BINARY) as ABC from src tablesample (1 rows) ) X; -- test UTF-8 substr @@ -72,4 +74,4 @@ SELECT substr("abc 玩", 5), substr("abc 玩玩玩 abc", 5), substr("abc 玩玩玩 abc", 5, 3) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q index f103da9ecb2de..3980fe83fbcb2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q @@ -1,16 +1,18 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION tan; DESCRIBE FUNCTION EXTENDED tan; SELECT tan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT tan(1), tan(6), tan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); DESCRIBE FUNCTION tan; DESCRIBE FUNCTION EXTENDED tan; SELECT tan(null) -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT tan(1), tan(6), tan(-1.0) -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q index 322a061d645d7..c94a52133d324 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q @@ -1,12 +1,10 @@ +set hive.fetch.task.conversion=more; + EXPLAIN CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength'; CREATE TEMPORARY FUNCTION testlength AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength'; -CREATE TABLE dest1(len INT); - -FROM src INSERT OVERWRITE TABLE dest1 SELECT testlength(src.value); - -SELECT dest1.* FROM dest1; +SELECT testlength(src.value) FROM src; DROP TEMPORARY FUNCTION testlength; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q index 6de270902f7fb..27e46c24a83d5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q @@ -1,12 +1,10 @@ +set hive.fetch.task.conversion=more; + EXPLAIN CREATE TEMPORARY FUNCTION testlength2 AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength2'; CREATE TEMPORARY FUNCTION testlength2 AS 'org.apache.hadoop.hive.ql.udf.UDFTestLength2'; -CREATE TABLE dest1(len INT); - -FROM src INSERT OVERWRITE TABLE dest1 SELECT testlength2(src.value); - -SELECT dest1.* FROM dest1; +SELECT testlength2(src.value) FROM src; DROP TEMPORARY FUNCTION testlength2; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q index ca23f719f93e3..8bea7abcbc4d3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q @@ -1,44 +1,46 @@ --- 'true' cases: +set hive.fetch.task.conversion=more; -SELECT CAST(CAST(1 AS TINYINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(2 AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-4 AS INT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-444 AS BIGINT) AS BOOLEAN) FROM src LIMIT 1; +-- 'true' cases: -SELECT CAST(CAST(7.0 AS FLOAT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-8.0 AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(-99.0 AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(1 AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(2 AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-4 AS INT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-444 AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST('Foo' AS STRING) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(7.0 AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-8.0 AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(-99.0 AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST('2011-05-06 07:08:09' as timestamp) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST('Foo' AS STRING) AS BOOLEAN) FROM src tablesample (1 rows); + +SELECT CAST(CAST('2011-05-06 07:08:09' as timestamp) AS BOOLEAN) FROM src tablesample (1 rows); -- 'false' cases: -SELECT CAST(CAST(0 AS TINYINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0 AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0 AS INT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0 AS BIGINT) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(0 AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0 AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0 AS INT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0 AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(0.0 AS FLOAT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0.0 AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(0.0 AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(0.0 AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0.0 AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(0.0 AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST('' AS STRING) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST('' AS STRING) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(0 as timestamp) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(0 as timestamp) AS BOOLEAN) FROM src tablesample (1 rows); -- 'NULL' cases: -SELECT CAST(NULL AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(NULL AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(NULL AS TINYINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS SMALLINT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS INT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS BIGINT) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(NULL AS TINYINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS SMALLINT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS INT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS BIGINT) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(NULL AS FLOAT) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS DOUBLE) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL AS DECIMAL) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(NULL AS FLOAT) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS DOUBLE) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL AS DECIMAL) AS BOOLEAN) FROM src tablesample (1 rows); -SELECT CAST(CAST(NULL AS STRING) AS BOOLEAN) FROM src LIMIT 1; -SELECT CAST(CAST(NULL as timestamp) AS BOOLEAN) FROM src LIMIT 1; +SELECT CAST(CAST(NULL AS STRING) AS BOOLEAN) FROM src tablesample (1 rows); +SELECT CAST(CAST(NULL as timestamp) AS BOOLEAN) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q index ded930d1115a4..aa0a250e4678d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Byte type: -SELECT CAST(NULL AS TINYINT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS TINYINT) FROM src LIMIT 1; +-- Conversion of main primitive types to Byte type: +SELECT CAST(NULL AS TINYINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-18 AS SMALLINT) AS TINYINT) FROM src LIMIT 1; -SELECT CAST(-129 AS TINYINT) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS TINYINT) FROM src LIMIT 1; +SELECT CAST(TRUE AS TINYINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS TINYINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS TINYINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS TINYINT) FROM src LIMIT 1; +SELECT CAST(CAST(-18 AS SMALLINT) AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS TINYINT) FROM src tablesample (1 rows); -SELECT CAST('-38' AS TINYINT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS TINYINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL) AS TINYINT) FROM src tablesample (1 rows); + +SELECT CAST('-38' AS TINYINT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q index b0a248ad70bba..005ec9d24e1ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Double type: -SELECT CAST(NULL AS DOUBLE) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS DOUBLE) FROM src LIMIT 1; +-- Conversion of main primitive types to Double type: +SELECT CAST(NULL AS DOUBLE) FROM src tablesample (1 rows); -SELECT CAST(CAST(-7 AS TINYINT) AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(-129 AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS DOUBLE) FROM src LIMIT 1; +SELECT CAST(TRUE AS DOUBLE) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS FLOAT) AS DOUBLE) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS DOUBLE) FROM src LIMIT 1; +SELECT CAST(CAST(-7 AS TINYINT) AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(-129 AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS DOUBLE) FROM src tablesample (1 rows); -SELECT CAST('-38.14' AS DOUBLE) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS FLOAT) AS DOUBLE) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS DOUBLE) FROM src tablesample (1 rows); + +SELECT CAST('-38.14' AS DOUBLE) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q index c91d18cc2f5c7..95671f15fe0ec 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Float type: -SELECT CAST(NULL AS FLOAT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS FLOAT) FROM src LIMIT 1; +-- Conversion of main primitive types to Float type: +SELECT CAST(NULL AS FLOAT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-7 AS TINYINT) AS FLOAT) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS FLOAT) FROM src LIMIT 1; -SELECT CAST(-129 AS FLOAT) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS FLOAT) FROM src LIMIT 1; +SELECT CAST(TRUE AS FLOAT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS FLOAT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS FLOAT) FROM src LIMIT 1; +SELECT CAST(CAST(-7 AS TINYINT) AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS FLOAT) FROM src tablesample (1 rows); -SELECT CAST('-38.14' AS FLOAT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS FLOAT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS FLOAT) FROM src tablesample (1 rows); + +SELECT CAST('-38.14' AS FLOAT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q index 45dc6f8bd32fa..706411a398463 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Long type: -SELECT CAST(NULL AS BIGINT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS BIGINT) FROM src LIMIT 1; +-- Conversion of main primitive types to Long type: +SELECT CAST(NULL AS BIGINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-7 AS TINYINT) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(-129 AS BIGINT) FROM src LIMIT 1; +SELECT CAST(TRUE AS BIGINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS BIGINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS BIGINT) FROM src LIMIT 1; +SELECT CAST(CAST(-7 AS TINYINT) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS BIGINT) FROM src tablesample (1 rows); -SELECT CAST('-38' AS BIGINT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS BIGINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL) AS BIGINT) FROM src tablesample (1 rows); + +SELECT CAST('-38' AS BIGINT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q index 7d843c1ea12ea..5cc4e57c8c9b5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q @@ -1,15 +1,17 @@ --- Conversion of main primitive types to Short type: -SELECT CAST(NULL AS SMALLINT) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS SMALLINT) FROM src LIMIT 1; +-- Conversion of main primitive types to Short type: +SELECT CAST(NULL AS SMALLINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-18 AS TINYINT) AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(-129 AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS SMALLINT) FROM src LIMIT 1; +SELECT CAST(TRUE AS SMALLINT) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS SMALLINT) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS SMALLINT) FROM src LIMIT 1; +SELECT CAST(CAST(-18 AS TINYINT) AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(-129 AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS SMALLINT) FROM src tablesample (1 rows); -SELECT CAST('-38' AS SMALLINT) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS SMALLINT) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL) AS SMALLINT) FROM src tablesample (1 rows); + +SELECT CAST('-38' AS SMALLINT) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q index 3b585e7170c60..ac4b5242e1fc4 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q @@ -1,16 +1,18 @@ --- Conversion of main primitive types to String type: -SELECT CAST(NULL AS STRING) FROM src LIMIT 1; +set hive.fetch.task.conversion=more; -SELECT CAST(TRUE AS STRING) FROM src LIMIT 1; +-- Conversion of main primitive types to String type: +SELECT CAST(NULL AS STRING) FROM src tablesample (1 rows); -SELECT CAST(CAST(1 AS TINYINT) AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-18 AS SMALLINT) AS STRING) FROM src LIMIT 1; -SELECT CAST(-129 AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-1025 AS BIGINT) AS STRING) FROM src LIMIT 1; +SELECT CAST(TRUE AS STRING) FROM src tablesample (1 rows); -SELECT CAST(CAST(-3.14 AS DOUBLE) AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS FLOAT) AS STRING) FROM src LIMIT 1; -SELECT CAST(CAST(-3.14 AS DECIMAL) AS STRING) FROM src LIMIT 1; +SELECT CAST(CAST(1 AS TINYINT) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-18 AS SMALLINT) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(-129 AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-1025 AS BIGINT) AS STRING) FROM src tablesample (1 rows); -SELECT CAST('Foo' AS STRING) FROM src LIMIT 1; +SELECT CAST(CAST(-3.14 AS DOUBLE) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS FLOAT) AS STRING) FROM src tablesample (1 rows); +SELECT CAST(CAST(-3.14 AS DECIMAL(3,2)) AS STRING) FROM src tablesample (1 rows); + +SELECT CAST('Foo' AS STRING) FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q index 3024074bba4b3..0a2758edfcd4f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION to_unix_timestamp; DESCRIBE FUNCTION EXTENDED to_unix_timestamp; create table oneline(key int, value string); -load data local inpath '../data/files/things.txt' into table oneline; +load data local inpath '../../data/files/things.txt' into table oneline; SELECT '2009-03-20 11:30:01', diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q index cba6ff90f643f..21d799882f59b 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION translate; DESCRIBE FUNCTION EXTENDED translate; @@ -10,28 +12,28 @@ FROM src INSERT OVERWRITE TABLE table_translate SELECT 'abcd', 'ahd', '12' WHERE -- Run some queries on constant input parameters SELECT translate('abcd', 'ab', '12'), - translate('abcd', 'abc', '12') FROM src LIMIT 1; + translate('abcd', 'abc', '12') FROM src tablesample (1 rows); -- Run some queries where first parameter being a table column while the other two being constants SELECT translate(table_input.input, 'ab', '12'), - translate(table_input.input, 'abc', '12') FROM table_input LIMIT 1; + translate(table_input.input, 'abc', '12') FROM table_input tablesample (1 rows); -- Run some queries where all parameters are coming from table columns -SELECT translate(input_string, from_string, to_string) FROM table_translate LIMIT 1; +SELECT translate(input_string, from_string, to_string) FROM table_translate tablesample (1 rows); -- Run some queries where some parameters are NULL SELECT translate(NULL, 'ab', '12'), translate('abcd', NULL, '12'), translate('abcd', 'ab', NULL), - translate(NULL, NULL, NULL) FROM src LIMIT 1; + translate(NULL, NULL, NULL) FROM src tablesample (1 rows); -- Run some queries where the same character appears several times in the from string (2nd argument) of the UDF SELECT translate('abcd', 'aba', '123'), - translate('abcd', 'aba', '12') FROM src LIMIT 1; + translate('abcd', 'aba', '12') FROM src tablesample (1 rows); -- Run some queries for the ignorant case when the 3rd parameter has more characters than the second one -SELECT translate('abcd', 'abc', '1234') FROM src LIMIT 1; +SELECT translate('abcd', 'abc', '1234') FROM src tablesample (1 rows); -- Test proper function over UTF-8 characters -SELECT translate('Àbcd', 'À', 'Ã') FROM src LIMIT 1; +SELECT translate('Àbcd', 'À', 'Ã') FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q index e80021c8b2c51..257e469ffb16c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION unhex; DESCRIBE FUNCTION EXTENDED unhex; @@ -9,11 +11,11 @@ SELECT unhex('61'), unhex('2D34'), unhex('') -FROM src limit 1; +FROM src tablesample (1 rows); -- Bad inputs SELECT unhex('MySQL'), unhex('G123'), unhex('\0') -FROM src limit 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q index 9140d22fb58c7..3876beb17d7ce 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION create_union; DESCRIBE FUNCTION EXTENDED create_union; EXPLAIN SELECT create_union(0, key), create_union(if(key<100, 0, 1), 2.0, value), create_union(1, "a", struct(2, "b")) -FROM src LIMIT 2; +FROM src tablesample (2 rows); SELECT create_union(0, key), create_union(if(key<100, 0, 1), 2.0, value), create_union(1, "a", struct(2, "b")) -FROM src LIMIT 2; +FROM src tablesample (2 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q index 89288a1193c87..1664329c33d98 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION unix_timestamp; DESCRIBE FUNCTION EXTENDED unix_timestamp; create table oneline(key int, value string); -load data local inpath '../data/files/things.txt' into table oneline; +load data local inpath '../../data/files/things.txt' into table oneline; SELECT '2009-03-20 11:30:01', diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q new file mode 100644 index 0000000000000..093187ddc30cb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_using.q @@ -0,0 +1,15 @@ +dfs ${system:test.dfs.mkdir} hdfs:///tmp/udf_using; + +dfs -copyFromLocal ../../data/files/sales.txt hdfs:///tmp/udf_using/sales.txt; + +create function lookup as 'org.apache.hadoop.hive.ql.udf.UDFFileLookup' using file 'hdfs:///tmp/udf_using/sales.txt'; + +create table udf_using (c1 string); +insert overwrite table udf_using select 'Joe' from src limit 2; + +select c1, lookup(c1) from udf_using; + +drop table udf_using; +drop function lookup; + +dfs -rmr hdfs:///tmp/udf_using; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q index 4b7b4ea55aa81..abb0a2d7d2c08 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q @@ -1,6 +1,8 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION weekofyear; DESCRIBE FUNCTION EXTENDED weekofyear; SELECT weekofyear('1980-01-01'), weekofyear('1980-01-06'), weekofyear('1980-01-07'), weekofyear('1980-12-31'), weekofyear('1984-1-1'), weekofyear('2008-02-20 00:00:00'), weekofyear('1980-12-28 23:59:59'), weekofyear('1980-12-29 23:59:59') -FROM src LIMIT 1; +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q index d94a594f026c5..4eb7f6918a0dc 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION when; DESCRIBE FUNCTION EXTENDED when; @@ -27,7 +29,7 @@ SELECT CASE WHEN 25=26 THEN 27 WHEN 28=28 THEN NULL END -FROM src LIMIT 1; +FROM src tablesample (1 rows); SELECT CASE WHEN 1=1 THEN 2 @@ -54,4 +56,21 @@ SELECT CASE WHEN 25=26 THEN 27 WHEN 28=28 THEN NULL END -FROM src LIMIT 1; +FROM src tablesample (1 rows); + +-- Allow compatible types to be used in return value +SELECT CASE + WHEN 1=1 THEN 123.0BD + ELSE 0.0BD + END, + CASE + WHEN 1=1 THEN 123 + WHEN 1=2 THEN 1.0 + ELSE 222.02BD + END, + CASE + WHEN 1=1 THEN 'abcd' + WHEN 1=2 THEN cast('efgh' as varchar(10)) + ELSE cast('ijkl' as char(4)) + END +FROM src tablesample (1 rows); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q index fca1ba11d85b2..1ad38abcf6ec5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q @@ -1,8 +1,10 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath ; DESCRIBE FUNCTION EXTENDED xpath ; -SELECT xpath ('b1b2b3c1c2', 'a/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/*/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/b/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/c/text()') FROM src LIMIT 1 ; -SELECT xpath ('b1b2b3c1c2', 'a/*[@class="bb"]/text()') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath ('b1b2b3c1c2', 'a/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/*/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/b/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/c/text()') FROM src tablesample (1 rows) ; +SELECT xpath ('b1b2b3c1c2', 'a/*[@class="bb"]/text()') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q index 3a6e613eb6a0e..6e3ff244b0434 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q @@ -1,9 +1,11 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_boolean ; DESCRIBE FUNCTION EXTENDED xpath_boolean ; -SELECT xpath_boolean ('b', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_boolean ('b', 'a/c') FROM src LIMIT 1 ; -SELECT xpath_boolean ('b', 'a/b = "b"') FROM src LIMIT 1 ; -SELECT xpath_boolean ('b', 'a/b = "c"') FROM src LIMIT 1 ; -SELECT xpath_boolean ('10', 'a/b < 10') FROM src LIMIT 1 ; -SELECT xpath_boolean ('10', 'a/b = 10') FROM src LIMIT 1 ; +SELECT xpath_boolean ('b', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('b', 'a/c') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('b', 'a/b = "b"') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('b', 'a/b = "c"') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('10', 'a/b < 10') FROM src tablesample (1 rows) ; +SELECT xpath_boolean ('10', 'a/b = 10') FROM src tablesample (1 rows) ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q index 4328747f44497..68441762e5a66 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q @@ -1,14 +1,16 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_number ; DESCRIBE FUNCTION EXTENDED xpath_number ; DESCRIBE FUNCTION xpath_double ; DESCRIBE FUNCTION EXTENDED xpath_double ; -SELECT xpath_double ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_double ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_double ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_double ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_double ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_double ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_double ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_double ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_double ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_double ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q index 1f1482833c858..4596a322604a5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_float ; DESCRIBE FUNCTION EXTENDED xpath_float ; -SELECT xpath_float ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_float ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_float ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_float ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_float ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_float ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_float ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_float ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_float ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_float ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q index 9b50bd9eeb365..9f3898f1147ac 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_int ; DESCRIBE FUNCTION EXTENDED xpath_int ; -SELECT xpath_int ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_int ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_int ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_int ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_int ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_int ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_int ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_int ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_int ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_int ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q index 04ee61b1f0e9d..3a335937c54fb 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_long ; DESCRIBE FUNCTION EXTENDED xpath_long ; -SELECT xpath_long ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_long ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_long ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_long ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_long ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; +SELECT xpath_long ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_long ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_long ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_long ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_long ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q index 6a1abdc0bdb21..073056e72e7a1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_short ; DESCRIBE FUNCTION EXTENDED xpath_short ; -SELECT xpath_short ('this is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_short ('this 2 is not a number', 'a') FROM src LIMIT 1 ; -SELECT xpath_short ('200000000040000000000', 'a/b * a/c') FROM src LIMIT 1 ; -SELECT xpath_short ('try a boolean', 'a = 10') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'sum(a/*)') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'sum(a/b)') FROM src LIMIT 1 ; -SELECT xpath_short ('1248', 'sum(a/b[@class="odd"])') FROM src LIMIT 1 ; \ No newline at end of file +SELECT xpath_short ('this is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_short ('this 2 is not a number', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_short ('200000000040000000000', 'a/b * a/c') FROM src tablesample (1 rows) ; +SELECT xpath_short ('try a boolean', 'a = 10') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'sum(a/*)') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'sum(a/b)') FROM src tablesample (1 rows) ; +SELECT xpath_short ('1248', 'sum(a/b[@class="odd"])') FROM src tablesample (1 rows) ; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q index ebbc913b71bfb..1f1731c67d114 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q @@ -1,11 +1,13 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION xpath_string ; DESCRIBE FUNCTION EXTENDED xpath_string ; -SELECT xpath_string ('bbcc', 'a') FROM src LIMIT 1 ; -SELECT xpath_string ('bbcc', 'a/b') FROM src LIMIT 1 ; -SELECT xpath_string ('bbcc', 'a/c') FROM src LIMIT 1 ; -SELECT xpath_string ('bbcc', 'a/d') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', '//b') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', 'a/b[1]') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', 'a/b[2]') FROM src LIMIT 1 ; -SELECT xpath_string ('b1b2', 'a/b[@id="b_2"]') FROM src LIMIT 1 ; +SELECT xpath_string ('bbcc', 'a') FROM src tablesample (1 rows) ; +SELECT xpath_string ('bbcc', 'a/b') FROM src tablesample (1 rows) ; +SELECT xpath_string ('bbcc', 'a/c') FROM src tablesample (1 rows) ; +SELECT xpath_string ('bbcc', 'a/d') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', '//b') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', 'a/b[1]') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', 'a/b[2]') FROM src tablesample (1 rows) ; +SELECT xpath_string ('b1b2', 'a/b[@id="b_2"]') FROM src tablesample (1 rows) ; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q index 638a4e9ca57e7..1d405b3560c5c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q @@ -1,3 +1,5 @@ +set hive.fetch.task.conversion=more; + DESCRIBE FUNCTION explode; DESCRIBE FUNCTION EXTENDED explode; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q index 712d9598c755d..93d829d4ed175 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q @@ -2,17 +2,17 @@ create table json_t (key string, jstring string); insert overwrite table json_t select * from ( - select '1', '{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}' from src limit 1 + select '1', '{"f1": "value1", "f2": "value2", "f3": 3, "f5": 5.23}' from src tablesample (1 rows) union all - select '2', '{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}' from src limit 1 + select '2', '{"f1": "value12", "f3": "value3", "f2": 2, "f4": 4.01}' from src tablesample (1 rows) union all - select '3', '{"f1": "value13", "f4": "value44", "f3": "value33", "f2": 2, "f5": 5.01}' from src limit 1 + select '3', '{"f1": "value13", "f4": "value44", "f3": "value33", "f2": 2, "f5": 5.01}' from src tablesample (1 rows) union all - select '4', cast(null as string) from src limit 1 + select '4', cast(null as string) from src tablesample (1 rows) union all - select '5', '{"f1": "", "f5": null}' from src limit 1 + select '5', '{"f1": "", "f5": null}' from src tablesample (1 rows) union all - select '6', '[invalid JSON string]' from src limit 1 + select '6', '[invalid JSON string]' from src tablesample (1 rows) ) s; explain @@ -40,7 +40,7 @@ select f2, count(*) from json_t a lateral view json_tuple(a.jstring, 'f1', 'f2', CREATE TABLE dest1(c1 STRING) STORED AS RCFILE; -INSERT OVERWRITE TABLE dest1 SELECT '{"a":"b\nc"}' FROM src LIMIT 1; +INSERT OVERWRITE TABLE dest1 SELECT '{"a":"b\nc"}' FROM src tablesample (1 rows); SELECT * FROM dest1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q index 055e39b2973a6..0870cbc4a9950 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q @@ -2,17 +2,17 @@ create table url_t (key string, fullurl string); insert overwrite table url_t select * from ( - select '1', 'http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1' from src limit 1 + select '1', 'http://facebook.com/path1/p.php?k1=v1&k2=v2#Ref1' from src tablesample (1 rows) union all - select '2', 'https://www.socs.uts.edu.au:80/MosaicDocs-old/url-primer.html?k1=tps#chapter1' from src limit 1 + select '2', 'https://www.socs.uts.edu.au:80/MosaicDocs-old/url-primer.html?k1=tps#chapter1' from src tablesample (1 rows) union all - select '3', 'ftp://sites.google.com/a/example.com/site/page' from src limit 1 + select '3', 'ftp://sites.google.com/a/example.com/site/page' from src tablesample (1 rows) union all - select '4', cast(null as string) from src limit 1 + select '4', cast(null as string) from src tablesample (1 rows) union all - select '5', 'htttp://' from src limit 1 + select '5', 'htttp://' from src tablesample (1 rows) union all - select '6', '[invalid url string]' from src limit 1 + select '6', '[invalid url string]' from src tablesample (1 rows) ) s; describe function parse_url_tuple; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q new file mode 100644 index 0000000000000..343f08ba6fe4b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_posexplode.q @@ -0,0 +1,15 @@ +CREATE TABLE employees ( +name STRING, +salary FLOAT, +subordinates ARRAY, +deductions MAP, +address STRUCT); + +LOAD DATA LOCAL INPATH '../../data/files/posexplode_data.txt' INTO TABLE employees; + +SELECT + name, pos, sub +FROM + employees +LATERAL VIEW + posexplode(subordinates) subView AS pos, sub; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q index 91bbd1b07f08b..525eccbbfe227 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q @@ -6,13 +6,13 @@ FROM ( UNION ALL FROM src SELECT src.* WHERE src.key > 100 ) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; FROM ( FROM src select src.key, src.value WHERE src.key < 100 UNION ALL FROM src SELECT src.* WHERE src.key > 100 ) unioninput -INSERT OVERWRITE DIRECTORY '../build/ql/test/data/warehouse/union.out' SELECT unioninput.*; +INSERT OVERWRITE DIRECTORY 'target/warehouse/union.out' SELECT unioninput.*; -dfs -cat ../build/ql/test/data/warehouse/union.out/*; +dfs -cat ${system:test.warehouse.dir}/union.out/*; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q index 36bc865be81fe..238b583fd86a8 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q @@ -3,7 +3,7 @@ create table src10_2 (key string, value string); create table src10_3 (key string, value string); create table src10_4 (key string, value string); -from (select * from src limit 10) a +from (select * from src tablesample (10 rows)) a insert overwrite table src10_1 select * insert overwrite table src10_2 select * insert overwrite table src10_3 select * diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q index e332a8af7bb1b..dd6f08e5b4e21 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q @@ -17,8 +17,8 @@ create table union_date_2 ( FL_NUM int ); -LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1; -LOAD DATA LOCAL INPATH '../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2; +LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_1; +LOAD DATA LOCAL INPATH '../../data/files/flights_join.txt' OVERWRITE INTO TABLE union_date_2; select * from ( select fl_num, fl_date from union_date_1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q index 70147070feaca..4368b8a5b6c36 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q @@ -1,2 +1,5 @@ -- HIVE-2901 select x from (select value as x from src union all select NULL as x from src)a limit 10; + +-- HIVE-4837 +select * from (select null as N from src1 group by key UNION ALL select null as N from src1 group by key ) a; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q index c9f920c648a92..c87b3fef1e462 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q @@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q index a072fe3b56e43..6701952717eb5 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q index 6250a20210f9f..4b2fa42f1b54d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q index 168eac34a0301..69d0d0af9ff68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q index a88a83e152a5f..7605f0ec26a08 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q index e588e8fd62ab9..a4fdfc8ee44ef 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q @@ -25,7 +25,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q index 237f2e7629057..e3c937b7b1e06 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q @@ -25,7 +25,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q index 06d5043f9cf9d..537078b080433 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q @@ -25,7 +25,7 @@ set hive.exec.dynamic.partition=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile ; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q index 65b8255c3e954..d70f3d3da31a2 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q index 265acfd35758f..478650038cc9c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q @@ -24,7 +24,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, ds string) stored as textfile; create table outputTbl1(key string, values bigint) partitioned by (ds string) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 partition (ds) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q index 1450caa2d07a7..8c45953ed877c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q @@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q index 015c146ecadf0..83cd2887ab23f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q index ac727889273d7..f80f7c1dfdadd 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(values bigint, key string) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q index f89744624116f..8963c2576ca76 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q index f01053d27d0c0..b0c1ccd73db68 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q @@ -19,7 +19,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint, values2 bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q index 805dd76274c42..a1b989a0eb63c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q index 36fd947d6035d..ec561e0979e0c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q @@ -18,7 +18,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key double, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; EXPLAIN INSERT OVERWRITE TABLE outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q index da0f1c0473ffe..9617f737bc552 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q @@ -20,7 +20,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q index 18d4730a12024..cae323b1ef89d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q @@ -20,7 +20,7 @@ set hive.merge.smallfiles.avgsize=1; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q index a6fbeb03ddcc3..5df84e145c709 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q index 7ae5af30c1bae..bfce26d0fbe1f 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q @@ -17,7 +17,7 @@ create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as textfile; create table outputTbl2(key string, values bigint) stored as textfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain FROM ( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q index 5a639ca11700d..3a956747a473d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q @@ -21,7 +21,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q index 79b84e81181d3..a83a43e466f55 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q index f6038384f3fa8..e71f6dd001dd6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q @@ -22,7 +22,7 @@ set mapred.input.dir.recursive=true; create table inputTbl1(key string, val string) stored as textfile; create table outputTbl1(key string, values bigint) stored as rcfile; -load data local inpath '../data/files/T1.txt' into table inputTbl1; +load data local inpath '../../data/files/T1.txt' into table inputTbl1; explain insert overwrite table outputTbl1 diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q new file mode 100644 index 0000000000000..6a4b45fbc4599 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_top_level.q @@ -0,0 +1,106 @@ +-- top level +explain +select * from ( +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3 +) q1 order by key; + +select * from ( +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3 +) q1 order by key; + +explain +select * from ( +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +union all +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +) q1 order by k; + +select * from ( +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +union all +select s1.key as k, s2.value as v from src s1 join src s2 on (s1.key = s2.key) limit 10 +) q1 order by k; + +-- ctas +explain +create table union_top as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +create table union_top as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top order by key; + +truncate table union_top; + +-- insert into +explain +insert into table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +insert into table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top order by key; + +explain +insert overwrite table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +insert overwrite table union_top +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top order by key; + +-- create view +explain +create view union_top_view as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +create view union_top_view as +select key, 0 as value from src where key % 3 == 0 limit 3 +union all +select key, 1 as value from src where key % 3 == 1 limit 3 +union all +select key, 2 as value from src where key % 3 == 2 limit 3; + +select * from union_top_view order by key; + +drop table union_top; +drop view union_top_view; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q index 4f8bafe37a809..b727199cf4f25 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q @@ -1,3 +1,4 @@ +set hive.stats.dbclass=fs; CREATE TABLE src_union_1 (key int, value string) PARTITIONED BY (ds string); CREATE INDEX src_union_1_key_idx ON TABLE src_union_1(key) AS 'COMPACT' WITH DEFERRED REBUILD; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q index 51bcf22bfa0be..3bc8ef931bcd3 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q @@ -2,9 +2,9 @@ CREATE TABLE T1(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T2(key STRING, val STRING) STORED AS TEXTFILE; CREATE TABLE T3(key STRING, val STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/T1.txt' INTO TABLE T1; -LOAD DATA LOCAL INPATH '../data/files/T2.txt' INTO TABLE T2; -LOAD DATA LOCAL INPATH '../data/files/T3.txt' INTO TABLE T3; +LOAD DATA LOCAL INPATH '../../data/files/T1.txt' INTO TABLE T1; +LOAD DATA LOCAL INPATH '../../data/files/T2.txt' INTO TABLE T2; +LOAD DATA LOCAL INPATH '../../data/files/T3.txt' INTO TABLE T3; FROM UNIQUEJOIN PRESERVE T1 a (a.key), PRESERVE T2 b (b.key), PRESERVE T3 c (c.key) SELECT a.key, b.key, c.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q index 53273b3187a78..bed61f381ce96 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q @@ -5,7 +5,7 @@ create table varchar1 (key varchar(10), value varchar(20)); create table varchar1_1 (key string, value string); -- load from file -load data local inpath '../data/files/srcbucket0.txt' overwrite into table varchar1; +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table varchar1; select * from varchar1 order by key, value limit 2; -- insert overwrite, from same/different length varchar diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q index 550f3dc728df2..c356b1dbca6ab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- Cast from varchar to other data types select diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q index b6c6f406412fe..05cad852a2680 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q @@ -1,3 +1,4 @@ +set hive.fetch.task.conversion=more; -- Should all be true select diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q index 6a19efaa3c5a1..94226879c53ab 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q @@ -17,9 +17,9 @@ create table varchar_join1_str ( c2 string ); -load data local inpath '../data/files/vc1.txt' into table varchar_join1_vc1; -load data local inpath '../data/files/vc1.txt' into table varchar_join1_vc2; -load data local inpath '../data/files/vc1.txt' into table varchar_join1_str; +load data local inpath '../../data/files/vc1.txt' into table varchar_join1_vc1; +load data local inpath '../../data/files/vc1.txt' into table varchar_join1_vc2; +load data local inpath '../../data/files/vc1.txt' into table varchar_join1_str; -- Join varchar with same length varchar select * from varchar_join1_vc1 a join varchar_join1_vc1 b on (a.c2 = b.c2) order by a.c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q index 7351b688cf504..ea2a022b94037 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q @@ -18,7 +18,7 @@ with serdeproperties ( ) stored as textfile; -load data local inpath '../data/files/srcbucket0.txt' overwrite into table varchar_serde_regex; +load data local inpath '../../data/files/srcbucket0.txt' overwrite into table varchar_serde_regex; select * from varchar_serde_regex limit 5; select value, count(*) from varchar_serde_regex group by value limit 5; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q index cf90eab33e2d0..dd3cffe0db4fe 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q @@ -17,9 +17,9 @@ create table varchar_union1_str ( c2 string ); -load data local inpath '../data/files/vc1.txt' into table varchar_union1_vc1; -load data local inpath '../data/files/vc1.txt' into table varchar_union1_vc2; -load data local inpath '../data/files/vc1.txt' into table varchar_union1_str; +load data local inpath '../../data/files/vc1.txt' into table varchar_union1_vc1; +load data local inpath '../../data/files/vc1.txt' into table varchar_union1_vc2; +load data local inpath '../../data/files/vc1.txt' into table varchar_union1_str; -- union varchar with same length varchar select * from ( diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q new file mode 100644 index 0000000000000..1bc66118f8ff3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_between_in.q @@ -0,0 +1,35 @@ +SET hive.vectorized.execution.enabled=true; + +CREATE TABLE decimal_date_test STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, CAST(CAST((CAST(cint AS BIGINT) *ctinyint) AS TIMESTAMP) AS DATE) AS cdate FROM alltypesorc ORDER BY cdate; + +EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE)) ORDER BY cdate; + +EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdate NOT IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE), CAST("1970-01-21" AS DATE)); + +EXPLAIN SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 IN (2365.8945945946, 881.0135135135, -3367.6517567568) ORDER BY cdecimal1; + +EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT IN (2365.8945945946, 881.0135135135, -3367.6517567568); + +EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate BETWEEN CAST("1969-12-30" AS DATE) AND CAST("1970-01-02" AS DATE) ORDER BY cdate; + +EXPLAIN SELECT cdate FROM decimal_date_test WHERE cdate NOT BETWEEN CAST("1968-05-01" AS DATE) AND CAST("1971-09-01" AS DATE) ORDER BY cdate; + +EXPLAIN SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 BETWEEN -20 AND 45.9918918919 ORDER BY cdecimal1; + +EXPLAIN SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351; + +SELECT cdate FROM decimal_date_test WHERE cdate IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE)) ORDER BY cdate; + +SELECT COUNT(*) FROM decimal_date_test WHERE cdate NOT IN (CAST("1969-10-26" AS DATE), CAST("1969-07-14" AS DATE), CAST("1970-01-21" AS DATE)); + +SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 IN (2365.8945945946, 881.0135135135, -3367.6517567568) ORDER BY cdecimal1; + +SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT IN (2365.8945945946, 881.0135135135, -3367.6517567568); + +SELECT cdate FROM decimal_date_test WHERE cdate BETWEEN CAST("1969-12-30" AS DATE) AND CAST("1970-01-02" AS DATE) ORDER BY cdate; + +SELECT cdate FROM decimal_date_test WHERE cdate NOT BETWEEN CAST("1968-05-01" AS DATE) AND CAST("1971-09-01" AS DATE) ORDER BY cdate; + +SELECT cdecimal1 FROM decimal_date_test WHERE cdecimal1 BETWEEN -20 AND 45.9918918919 ORDER BY cdecimal1; + +SELECT COUNT(*) FROM decimal_date_test WHERE cdecimal1 NOT BETWEEN -2000 AND 4390.1351351351; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q new file mode 100644 index 0000000000000..052ab716bee1a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_coalesce.q @@ -0,0 +1,32 @@ +SET hive.vectorized.execution.enabled=true; +EXPLAIN SELECT cdouble, cstring1, cint, cfloat, csmallint, coalesce(cdouble, cstring1, cint, cfloat, csmallint) +FROM alltypesorc +WHERE (cdouble IS NULL) LIMIT 10; + +SELECT cdouble, cstring1, cint, cfloat, csmallint, coalesce(cdouble, cstring1, cint, cfloat, csmallint) +FROM alltypesorc +WHERE (cdouble IS NULL) LIMIT 10; + +EXPLAIN SELECT ctinyint, cdouble, cint, coalesce(ctinyint+10, (cdouble+log2(cint)), 0) +FROM alltypesorc +WHERE (ctinyint IS NULL) LIMIT 10; + +SELECT ctinyint, cdouble, cint, coalesce(ctinyint+10, (cdouble+log2(cint)), 0) +FROM alltypesorc +WHERE (ctinyint IS NULL) LIMIT 10; + +EXPLAIN SELECT cfloat, cbigint, coalesce(cfloat, cbigint, 0) +FROM alltypesorc +WHERE (cfloat IS NULL AND cbigint IS NULL) LIMIT 10; + +SELECT cfloat, cbigint, coalesce(cfloat, cbigint, 0) +FROM alltypesorc +WHERE (cfloat IS NULL AND cbigint IS NULL) LIMIT 10; + +EXPLAIN SELECT ctimestamp1, ctimestamp2, coalesce(ctimestamp1, ctimestamp2) +FROM alltypesorc +WHERE ctimestamp1 IS NOT NULL OR ctimestamp2 IS NOT NULL LIMIT 10; + +SELECT ctimestamp1, ctimestamp2, coalesce(ctimestamp1, ctimestamp2) +FROM alltypesorc +WHERE ctimestamp1 IS NOT NULL OR ctimestamp2 IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q new file mode 100644 index 0000000000000..eb9146e95d437 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_aggregate.q @@ -0,0 +1,20 @@ +CREATE TABLE decimal_vgby STORED AS ORC AS + SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, + CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, + cint + FROM alltypesorc; + +SET hive.vectorized.execution.enabled=true; + +EXPLAIN SELECT cint, + COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1), + COUNT(cdecimal2), MAX(cdecimal2), MIN(cdecimal2), SUM(cdecimal2), AVG(cdecimal2), STDDEV_POP(cdecimal2), STDDEV_SAMP(cdecimal2) + FROM decimal_vgby + GROUP BY cint + HAVING COUNT(*) > 1; +SELECT cint, + COUNT(cdecimal1), MAX(cdecimal1), MIN(cdecimal1), SUM(cdecimal1), AVG(cdecimal1), STDDEV_POP(cdecimal1), STDDEV_SAMP(cdecimal1), + COUNT(cdecimal2), MAX(cdecimal2), MIN(cdecimal2), SUM(cdecimal2), AVG(cdecimal2), STDDEV_POP(cdecimal2), STDDEV_SAMP(cdecimal2) + FROM decimal_vgby + GROUP BY cint + HAVING COUNT(*) > 1; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q new file mode 100644 index 0000000000000..ea7a5b817d7c0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_cast.q @@ -0,0 +1,5 @@ +SET hive.vectorized.execution.enabled=true; + +EXPLAIN SELECT cdouble, cint, cboolean1, ctimestamp1, CAST(cdouble AS DECIMAL(20,10)), CAST(cint AS DECIMAL(23,14)), CAST(cboolean1 AS DECIMAL(5,2)), CAST(ctimestamp1 AS DECIMAL(15,0)) FROM alltypesorc WHERE cdouble IS NOT NULL AND cint IS NOT NULL AND cboolean1 IS NOT NULL AND ctimestamp1 IS NOT NULL LIMIT 10; + +SELECT cdouble, cint, cboolean1, ctimestamp1, CAST(cdouble AS DECIMAL(20,10)), CAST(cint AS DECIMAL(23,14)), CAST(cboolean1 AS DECIMAL(5,2)), CAST(ctimestamp1 AS DECIMAL(15,0)) FROM alltypesorc WHERE cdouble IS NOT NULL AND cint IS NOT NULL AND cboolean1 IS NOT NULL AND ctimestamp1 IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q new file mode 100644 index 0000000000000..a74b17bd127d6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_expressions.q @@ -0,0 +1,5 @@ +CREATE TABLE decimal_test STORED AS ORC AS SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2 FROM alltypesorc; +SET hive.vectorized.execution.enabled=true; +EXPLAIN SELECT cdecimal1 + cdecimal2, cdecimal1 - (2*cdecimal2), ((cdecimal1+2.34)/cdecimal2), (cdecimal1 * (cdecimal2/3.4)), cdecimal1 % 10, CAST(cdecimal1 AS INT), CAST(cdecimal2 AS SMALLINT), CAST(cdecimal2 AS TINYINT), CAST(cdecimal1 AS BIGINT), CAST (cdecimal1 AS BOOLEAN), CAST(cdecimal2 AS DOUBLE), CAST(cdecimal1 AS FLOAT), CAST(cdecimal2 AS STRING), CAST(cdecimal1 AS TIMESTAMP) FROM decimal_test WHERE cdecimal1 > 0 AND cdecimal1 < 12345.5678 AND cdecimal2 != 0 AND cdecimal2 > 1000 AND cdouble IS NOT NULL LIMIT 10; + +SELECT cdecimal1 + cdecimal2, cdecimal1 - (2*cdecimal2), ((cdecimal1+2.34)/cdecimal2), (cdecimal1 * (cdecimal2/3.4)), cdecimal1 % 10, CAST(cdecimal1 AS INT), CAST(cdecimal2 AS SMALLINT), CAST(cdecimal2 AS TINYINT), CAST(cdecimal1 AS BIGINT), CAST (cdecimal1 AS BOOLEAN), CAST(cdecimal2 AS DOUBLE), CAST(cdecimal1 AS FLOAT), CAST(cdecimal2 AS STRING), CAST(cdecimal1 AS TIMESTAMP) FROM decimal_test WHERE cdecimal1 > 0 AND cdecimal1 < 12345.5678 AND cdecimal2 != 0 AND cdecimal2 > 1000 AND cdouble IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q new file mode 100644 index 0000000000000..d8b3d1a9ac13f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q @@ -0,0 +1,19 @@ +CREATE TABLE decimal_mapjoin STORED AS ORC AS + SELECT cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, + CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2, + cint + FROM alltypesorc; + +SET hive.auto.convert.join=true; +SET hive.auto.convert.join.nonconditionaltask=true; +SET hive.auto.convert.join.nonconditionaltask.size=1000000000; +SET hive.vectorized.execution.enabled=true; + +EXPLAIN SELECT l.cint, r.cint, l.cdecimal1, r.cdecimal2 + FROM decimal_mapjoin l + JOIN decimal_mapjoin r ON l.cint = r.cint + WHERE l.cint = 6981; +SELECT l.cint, r.cint, l.cdecimal1, r.cdecimal2 + FROM decimal_mapjoin l + JOIN decimal_mapjoin r ON l.cint = r.cint + WHERE l.cint = 6981; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q new file mode 100644 index 0000000000000..6e2c0b1fd403f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_decimal_math_funcs.q @@ -0,0 +1,77 @@ +CREATE TABLE decimal_test STORED AS ORC AS SELECT cbigint, cdouble, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal1, CAST (((cdouble*9.3)/13) AS DECIMAL(23,14)) AS cdecimal2 FROM alltypesorc; +SET hive.vectorized.execution.enabled=true; + +-- Test math functions in vectorized mode to verify they run correctly end-to-end. + +explain +select + cdecimal1 + ,Round(cdecimal1, 2) + ,Round(cdecimal1) + ,Floor(cdecimal1) + ,Ceil(cdecimal1) + ,Exp(cdecimal1) + ,Ln(cdecimal1) + ,Log10(cdecimal1) + -- Use log2 as a representative function to test all input types. + ,Log2(cdecimal1) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdecimal1 - 15601.0) + ,Log(2.0, cdecimal1) + ,Pow(log2(cdecimal1), 2.0) + ,Power(log2(cdecimal1), 2.0) + ,Sqrt(cdecimal1) + ,Abs(cdecimal1) + ,Sin(cdecimal1) + ,Asin(cdecimal1) + ,Cos(cdecimal1) + ,ACos(cdecimal1) + ,Atan(cdecimal1) + ,Degrees(cdecimal1) + ,Radians(cdecimal1) + ,Positive(cdecimal1) + ,Negative(cdecimal1) + ,Sign(cdecimal1) + -- Test nesting + ,cos(-sin(log(cdecimal1)) + 3.14159) +from decimal_test +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cdecimal1) >= -1.0; + +select + cdecimal1 + ,Round(cdecimal1, 2) + ,Round(cdecimal1) + ,Floor(cdecimal1) + ,Ceil(cdecimal1) + ,Exp(cdecimal1) + ,Ln(cdecimal1) + ,Log10(cdecimal1) + -- Use log2 as a representative function to test all input types. + ,Log2(cdecimal1) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdecimal1 - 15601.0) + ,Log(2.0, cdecimal1) + ,Pow(log2(cdecimal1), 2.0) + ,Power(log2(cdecimal1), 2.0) + ,Sqrt(cdecimal1) + ,Abs(cdecimal1) + ,Sin(cdecimal1) + ,Asin(cdecimal1) + ,Cos(cdecimal1) + ,ACos(cdecimal1) + ,Atan(cdecimal1) + ,Degrees(cdecimal1) + ,Radians(cdecimal1) + ,Positive(cdecimal1) + ,Negative(cdecimal1) + ,Sign(cdecimal1) + -- Test nesting + ,cos(-sin(log(cdecimal1)) + 3.14159) +from decimal_test +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cdecimal1) >= -1.0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q new file mode 100644 index 0000000000000..6e96690497390 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_left_outer_join.q @@ -0,0 +1,21 @@ +set hive.vectorized.execution.enabled=true; +set hive.auto.convert.join=true; +explain +select count(*) from (select c.ctinyint +from alltypesorc c +left outer join alltypesorc cd + on cd.cint = c.cint +left outer join alltypesorc hd + on hd.ctinyint = c.ctinyint +) t1 +; +select count(*) from (select c.ctinyint +from alltypesorc c +left outer join alltypesorc cd + on cd.cint = c.cint +left outer join alltypesorc hd + on hd.ctinyint = c.ctinyint +) t1; + +set hive.auto.convert.join=false; +set hive.vectorized.execution.enabled=false; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q new file mode 100644 index 0000000000000..fc1dc6d3b89f3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vector_non_string_partition.q @@ -0,0 +1,17 @@ +SET hive.vectorized.execution.enabled=true; +CREATE TABLE non_string_part(cint INT, cstring1 STRING, cdouble DOUBLE, ctimestamp1 TIMESTAMP) PARTITIONED BY (ctinyint tinyint) STORED AS ORC; +SET hive.exec.dynamic.partition.mode=nonstrict; +SET hive.exec.dynamic.partition=true; + +INSERT OVERWRITE TABLE non_string_part PARTITION(ctinyint) SELECT cint, cstring1, cdouble, ctimestamp1, ctinyint fROM alltypesorc +WHERE ctinyint IS NULL AND cdouble IS NOT NULL ORDER BY cdouble; + +SHOW PARTITIONS non_string_part; + +EXPLAIN SELECT cint, ctinyint FROM non_string_part WHERE cint > 0 ORDER BY cint LIMIT 10; + +SELECT cint, ctinyint FROM non_string_part WHERE cint > 0 ORDER BY cint LIMIT 10; + +EXPLAIN SELECT cint, cstring1 FROM non_string_part WHERE cint > 0 ORDER BY cint, cstring1 LIMIT 10; + +SELECT cint, cstring1 FROM non_string_part WHERE cint > 0 ORDER BY cint, cstring1 LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q new file mode 100644 index 0000000000000..39fba7d1ac32d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_0.q @@ -0,0 +1,27 @@ +SET hive.vectorized.execution.enabled=true; +SELECT AVG(cbigint), + (-(AVG(cbigint))), + (-6432 + AVG(cbigint)), + STDDEV_POP(cbigint), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) + (-6432 + AVG(cbigint))), + VAR_SAMP(cbigint), + (-((-6432 + AVG(cbigint)))), + (-6432 + (-((-6432 + AVG(cbigint))))), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) / (-((-6432 + AVG(cbigint))))), + COUNT(*), + SUM(cfloat), + (VAR_SAMP(cbigint) % STDDEV_POP(cbigint)), + (-(VAR_SAMP(cbigint))), + ((-((-6432 + AVG(cbigint)))) * (-(AVG(cbigint)))), + MIN(ctinyint), + (-(MIN(ctinyint))) +FROM alltypesorc +WHERE (((cstring2 LIKE '%b%') + OR ((79.553 != cint) + OR (cbigint < cdouble))) + OR ((ctinyint >= csmallint) + AND ((cboolean2 = 1) + AND (3569 = ctinyint)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q new file mode 100644 index 0000000000000..745aa06f2f524 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_1.q @@ -0,0 +1,21 @@ +SET hive.vectorized.execution.enabled=true; +SELECT VAR_POP(ctinyint), + (VAR_POP(ctinyint) / -26.28), + SUM(cfloat), + (-1.389 + SUM(cfloat)), + (SUM(cfloat) * (-1.389 + SUM(cfloat))), + MAX(ctinyint), + (-((SUM(cfloat) * (-1.389 + SUM(cfloat))))), + MAX(cint), + (MAX(cint) * 79.553), + VAR_SAMP(cdouble), + (10.175 % (-((SUM(cfloat) * (-1.389 + SUM(cfloat)))))), + COUNT(cint), + (-563 % MAX(cint)) +FROM alltypesorc +WHERE (((cdouble > ctinyint) + AND (cboolean2 > 0)) + OR ((cbigint < ctinyint) + OR ((cint > cbigint) + OR (cboolean1 < 0)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q new file mode 100644 index 0000000000000..720f38d02b072 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_10.q @@ -0,0 +1,24 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cdouble, + ctimestamp1, + ctinyint, + cboolean1, + cstring1, + (-(cdouble)), + (cdouble + csmallint), + ((cdouble + csmallint) % 33), + (-(cdouble)), + (ctinyint % cdouble), + (ctinyint % csmallint), + (-(cdouble)), + (cbigint * (ctinyint % csmallint)), + (9763215.5639 - (cdouble + csmallint)), + (-((-(cdouble)))) +FROM alltypesorc +WHERE (((cstring2 <= '10') + OR ((ctinyint > cdouble) + AND (-5638.15 >= ctinyint))) + OR ((cdouble > 6981) + AND ((csmallint = 9763215.5639) + OR (cstring1 LIKE '%a')))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q new file mode 100644 index 0000000000000..329ed28948434 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_11.q @@ -0,0 +1,15 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cstring1, + cboolean1, + cdouble, + ctimestamp1, + (-3728 * csmallint), + (cdouble - 9763215.5639), + (-(cdouble)), + ((-(cdouble)) + 6981), + (cdouble * -5638.15) +FROM alltypesorc +WHERE ((cstring2 = cstring1) + OR ((ctimestamp1 IS NULL) + AND (cstring1 LIKE '%a'))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q new file mode 100644 index 0000000000000..fef62fc7a0eb6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_12.q @@ -0,0 +1,32 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cbigint, + cboolean1, + cstring1, + ctimestamp1, + cdouble, + (-6432 * cdouble), + (-(cbigint)), + COUNT(cbigint), + (cbigint * COUNT(cbigint)), + STDDEV_SAMP(cbigint), + ((-6432 * cdouble) / -6432), + (-(((-6432 * cdouble) / -6432))), + AVG(cdouble), + (-((-6432 * cdouble))), + (-5638.15 + cbigint), + SUM(cbigint), + (AVG(cdouble) / (-6432 * cdouble)), + AVG(cdouble), + (-((-(((-6432 * cdouble) / -6432))))), + (((-6432 * cdouble) / -6432) + (-((-6432 * cdouble)))), + STDDEV_POP(cdouble) +FROM alltypesorc +WHERE (((ctimestamp1 IS NULL) + AND ((cboolean1 >= cboolean2) + OR (ctinyint != csmallint))) + AND ((cstring1 LIKE '%a') + OR ((cboolean2 <= 1) + AND (cbigint >= csmallint)))) +GROUP BY cbigint, cboolean1, cstring1, ctimestamp1, cdouble +ORDER BY ctimestamp1, cdouble, cbigint, cstring1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q new file mode 100644 index 0000000000000..fad2585d22725 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_13.q @@ -0,0 +1,31 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cboolean1, + ctinyint, + ctimestamp1, + cfloat, + cstring1, + (-(ctinyint)), + MAX(ctinyint), + ((-(ctinyint)) + MAX(ctinyint)), + SUM(cfloat), + (SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))), + (-(SUM(cfloat))), + (79.553 * cfloat), + STDDEV_POP(cfloat), + (-(SUM(cfloat))), + STDDEV_POP(ctinyint), + (((-(ctinyint)) + MAX(ctinyint)) - 10.175), + (-((-(SUM(cfloat))))), + (-26.28 / (-((-(SUM(cfloat)))))), + MAX(cfloat), + ((SUM(cfloat) * ((-(ctinyint)) + MAX(ctinyint))) / ctinyint), + MIN(ctinyint) +FROM alltypesorc +WHERE (((cfloat < 3569) + AND ((10.175 >= cdouble) + AND (cboolean1 != 1))) + OR ((ctimestamp1 > -29071) + AND ((ctimestamp2 != -29071) + AND (ctinyint < 9763215.5639)))) +GROUP BY cboolean1, ctinyint, ctimestamp1, cfloat, cstring1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q new file mode 100644 index 0000000000000..a121c64edc7b4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_14.q @@ -0,0 +1,33 @@ +SET hive.vectorized.execution.enabled=true; +SELECT ctimestamp1, + cfloat, + cstring1, + cboolean1, + cdouble, + (-26.28 + cdouble), + (-((-26.28 + cdouble))), + STDDEV_SAMP((-((-26.28 + cdouble)))), + (cfloat * -26.28), + MAX(cfloat), + (-(cfloat)), + (-(MAX(cfloat))), + ((-((-26.28 + cdouble))) / 10.175), + STDDEV_POP(cfloat), + COUNT(cfloat), + (-(((-((-26.28 + cdouble))) / 10.175))), + (-1.389 % STDDEV_SAMP((-((-26.28 + cdouble))))), + (cfloat - cdouble), + VAR_POP(cfloat), + (VAR_POP(cfloat) % 10.175), + VAR_SAMP(cfloat), + (-((cfloat - cdouble))) +FROM alltypesorc +WHERE (((ctinyint <= cbigint) + AND ((cint <= cdouble) + OR (ctimestamp2 < ctimestamp1))) + AND ((cdouble < ctinyint) + AND ((cbigint > -257) + OR (cfloat < cint)))) +GROUP BY ctimestamp1, cfloat, cstring1, cboolean1, cdouble +ORDER BY cstring1, cfloat, cdouble, ctimestamp1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q new file mode 100644 index 0000000000000..7daad0b2c50a5 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_15.q @@ -0,0 +1,31 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cfloat, + cboolean1, + cdouble, + cstring1, + ctinyint, + cint, + ctimestamp1, + STDDEV_SAMP(cfloat), + (-26.28 - cint), + MIN(cdouble), + (cdouble * 79.553), + (33 % cfloat), + STDDEV_SAMP(ctinyint), + VAR_POP(ctinyint), + (-23 % cdouble), + (-(ctinyint)), + VAR_SAMP(cint), + (cint - cfloat), + (-23 % ctinyint), + (-((-26.28 - cint))), + STDDEV_POP(cint) +FROM alltypesorc +WHERE (((cstring2 LIKE '%ss%') + OR (cstring1 LIKE '10%')) + OR ((cint >= -75) + AND ((ctinyint = csmallint) + AND (cdouble >= -3728)))) +GROUP BY cfloat, cboolean1, cdouble, cstring1, ctinyint, cint, ctimestamp1 +ORDER BY cfloat, cboolean1, cdouble, cstring1, ctinyint, cint, ctimestamp1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q new file mode 100644 index 0000000000000..39a9402680496 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_16.q @@ -0,0 +1,20 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cstring1, + cdouble, + ctimestamp1, + (cdouble - 9763215.5639), + (-((cdouble - 9763215.5639))), + COUNT(cdouble), + STDDEV_SAMP(cdouble), + (-(STDDEV_SAMP(cdouble))), + (STDDEV_SAMP(cdouble) * COUNT(cdouble)), + MIN(cdouble), + (9763215.5639 / cdouble), + (COUNT(cdouble) / -1.389), + STDDEV_SAMP(cdouble) +FROM alltypesorc +WHERE ((cstring2 LIKE '%b%') + AND ((cdouble >= -1.389) + OR (cstring1 < 'a'))) +GROUP BY cstring1, cdouble, ctimestamp1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q new file mode 100644 index 0000000000000..b8647a434414b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_2.q @@ -0,0 +1,23 @@ +SET hive.vectorized.execution.enabled=true; +SELECT AVG(csmallint), + (AVG(csmallint) % -563), + (AVG(csmallint) + 762), + SUM(cfloat), + VAR_POP(cbigint), + (-(VAR_POP(cbigint))), + (SUM(cfloat) - AVG(csmallint)), + COUNT(*), + (-((SUM(cfloat) - AVG(csmallint)))), + (VAR_POP(cbigint) - 762), + MIN(ctinyint), + ((-(VAR_POP(cbigint))) + MIN(ctinyint)), + AVG(cdouble), + (((-(VAR_POP(cbigint))) + MIN(ctinyint)) - SUM(cfloat)) +FROM alltypesorc +WHERE (((ctimestamp1 < ctimestamp2) + AND ((cstring2 LIKE 'b%') + AND (cfloat <= -5638.15))) + OR ((cdouble < ctinyint) + AND ((-10669 != ctimestamp2) + OR (359 > cint)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q new file mode 100644 index 0000000000000..1d53994519db3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_3.q @@ -0,0 +1,25 @@ +SET hive.vectorized.execution.enabled=true; +SELECT STDDEV_SAMP(csmallint), + (STDDEV_SAMP(csmallint) - 10.175), + STDDEV_POP(ctinyint), + (STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)), + (-(STDDEV_POP(ctinyint))), + (STDDEV_SAMP(csmallint) % 79.553), + (-((STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)))), + STDDEV_SAMP(cfloat), + (-(STDDEV_SAMP(csmallint))), + SUM(cfloat), + ((-((STDDEV_SAMP(csmallint) * (STDDEV_SAMP(csmallint) - 10.175)))) / (STDDEV_SAMP(csmallint) - 10.175)), + (-((STDDEV_SAMP(csmallint) - 10.175))), + AVG(cint), + (-3728 - STDDEV_SAMP(csmallint)), + STDDEV_POP(cint), + (AVG(cint) / STDDEV_SAMP(cfloat)) +FROM alltypesorc +WHERE (((cint <= cfloat) + AND ((79.553 != cbigint) + AND (ctimestamp2 = -29071))) + OR ((cbigint > cdouble) + AND ((79.553 <= csmallint) + AND (ctimestamp1 > ctimestamp2)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q new file mode 100644 index 0000000000000..1eb324d1902e8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_4.q @@ -0,0 +1,23 @@ +SET hive.vectorized.execution.enabled=true; +SELECT SUM(cint), + (SUM(cint) * -563), + (-3728 + SUM(cint)), + STDDEV_POP(cdouble), + (-(STDDEV_POP(cdouble))), + AVG(cdouble), + ((SUM(cint) * -563) % SUM(cint)), + (((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble)), + VAR_POP(cdouble), + (-((((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble)))), + ((-3728 + SUM(cint)) - (SUM(cint) * -563)), + MIN(ctinyint), + MIN(ctinyint), + (MIN(ctinyint) * (-((((SUM(cint) * -563) % SUM(cint)) / AVG(cdouble))))) +FROM alltypesorc +WHERE (((csmallint >= cint) + OR ((-89010 >= ctinyint) + AND (cdouble > 79.553))) + OR ((-563 != cbigint) + AND ((ctinyint != cbigint) + OR (-3728 >= cdouble)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q new file mode 100644 index 0000000000000..826b20fd4f124 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_5.q @@ -0,0 +1,20 @@ +SET hive.vectorized.execution.enabled=true; +SELECT MAX(csmallint), + (MAX(csmallint) * -75), + COUNT(*), + ((MAX(csmallint) * -75) / COUNT(*)), + (6981 * MAX(csmallint)), + MIN(csmallint), + (-(MIN(csmallint))), + (197 % ((MAX(csmallint) * -75) / COUNT(*))), + SUM(cint), + MAX(ctinyint), + (-(MAX(ctinyint))), + ((-(MAX(ctinyint))) + MAX(ctinyint)) +FROM alltypesorc +WHERE (((cboolean2 IS NOT NULL) + AND (cstring1 LIKE '%b%')) + OR ((ctinyint = cdouble) + AND ((ctimestamp2 IS NOT NULL) + AND (cstring2 LIKE 'a')))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q new file mode 100644 index 0000000000000..2b59f10ed89c3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_6.q @@ -0,0 +1,21 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cboolean1, + cfloat, + cstring1, + (988888 * csmallint), + (-(csmallint)), + (-(cfloat)), + (-26.28 / cfloat), + (cfloat * 359), + (cint % ctinyint), + (-(cdouble)), + (ctinyint - -75), + (762 * (cint % ctinyint)) +FROM alltypesorc +WHERE ((ctinyint != 0) + AND ((((cboolean1 <= 0) + AND (cboolean2 >= cboolean1)) + OR ((cbigint IS NOT NULL) + AND ((cstring2 LIKE '%a') + OR (cfloat <= -257)))))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q new file mode 100644 index 0000000000000..20c1148659195 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_7.q @@ -0,0 +1,25 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cboolean1, + cbigint, + csmallint, + ctinyint, + ctimestamp1, + cstring1, + (cbigint + cbigint), + (csmallint % -257), + (-(csmallint)), + (-(ctinyint)), + ((-(ctinyint)) + 17), + (cbigint * (-(csmallint))), + (cint % csmallint), + (-(ctinyint)), + ((-(ctinyint)) % ctinyint) +FROM alltypesorc +WHERE ((ctinyint != 0) + AND (((ctimestamp1 <= 0) + OR ((ctinyint = cint) + OR (cstring2 LIKE 'ss'))) + AND ((988888 < cdouble) + OR ((ctimestamp2 > -29071) + AND (3569 >= cdouble))))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q new file mode 100644 index 0000000000000..98b3385f165a9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_8.q @@ -0,0 +1,23 @@ +SET hive.vectorized.execution.enabled=true; +SELECT ctimestamp1, + cdouble, + cboolean1, + cstring1, + cfloat, + (-(cdouble)), + (-5638.15 - cdouble), + (cdouble * -257), + (cint + cfloat), + ((-(cdouble)) + cbigint), + (-(cdouble)), + (-1.389 - cfloat), + (-(cfloat)), + ((-5638.15 - cdouble) + (cint + cfloat)) +FROM alltypesorc +WHERE (((cstring2 IS NOT NULL) + AND ((ctimestamp1 <= -29071) + AND (ctimestamp2 != 16558))) + OR ((cfloat < -6432) + OR ((cboolean1 IS NOT NULL) + AND (cdouble = 988888)))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q new file mode 100644 index 0000000000000..252c426d64a5e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_9.q @@ -0,0 +1,24 @@ +SET hive.vectorized.execution.enabled=true; +SELECT cfloat, + cstring1, + cint, + ctimestamp1, + cdouble, + cbigint, + (cfloat / ctinyint), + (cint % cbigint), + (-(cdouble)), + (cdouble + (cfloat / ctinyint)), + (cdouble / cint), + (-((-(cdouble)))), + (9763215.5639 % cbigint), + (2563.58 + (-((-(cdouble))))) +FROM alltypesorc +WHERE (((cbigint > -23) + AND ((cdouble != 988888) + OR (cint > -863.257))) + AND ((ctinyint >= 33) + OR ((csmallint >= cbigint) + OR (cfloat = cdouble)))) +ORDER BY cbigint, cfloat; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q new file mode 100644 index 0000000000000..2b82a5aa0452a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_decimal_date.q @@ -0,0 +1,4 @@ +CREATE TABLE date_decimal_test STORED AS ORC AS SELECT cint, cdouble, CAST (CAST (cint AS TIMESTAMP) AS DATE) AS cdate, CAST (((cdouble*22.1)/37) AS DECIMAL(20,10)) AS cdecimal FROM alltypesorc; +SET hive.vectorized.execution.enabled=true; +EXPLAIN SELECT cdate, cdecimal from date_decimal_test where cint IS NOT NULL AND cdouble IS NOT NULL LIMIT 10; +SELECT cdate, cdecimal from date_decimal_test where cint IS NOT NULL AND cdouble IS NOT NULL LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q new file mode 100644 index 0000000000000..69e388a28e411 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_div0.q @@ -0,0 +1,24 @@ +SET hive.vectorized.execution.enabled = true; + +-- TODO: add more stuff here after HIVE-5918 is fixed, such as cbigint and constants +explain +select cdouble / 0.0 from alltypesorc limit 100; +select cdouble / 0.0 from alltypesorc limit 100; + +-- There are no zeros in the table, but there is 988888, so use it as zero + +-- TODO: add more stuff here after HIVE-5918 is fixed, such as cbigint and constants as numerators +explain +select (cbigint - 988888L) as s1, cdouble / (cbigint - 988888L) as s2, 1.2 / (cbigint - 988888L) +from alltypesorc where cbigint > 0 and cbigint < 100000000 order by s1, s2 limit 100; +select (cbigint - 988888L) as s1, cdouble / (cbigint - 988888L) as s2, 1.2 / (cbigint - 988888L) +from alltypesorc where cbigint > 0 and cbigint < 100000000 order by s1, s2 limit 100; + +-- There are no zeros in the table, but there is -200.0, so use it as zero + +explain +select (cdouble + 200.0) as s1, cbigint / (cdouble + 200.0) as s2, (cdouble + 200.0) / (cdouble + 200.0), cbigint / (cdouble + 200.0), 1 / (cdouble + 200.0), 1.2 / (cdouble + 200.0) +from alltypesorc where cdouble >= -500 and cdouble < -199 order by s1, s2 limit 100; +select (cdouble + 200.0) as s1, cbigint / (cdouble + 200.0) as s2, (cdouble + 200.0) / (cdouble + 200.0), cbigint / (cdouble + 200.0), 1 / (cdouble + 200.0), 1.2 / (cdouble + 200.0) +from alltypesorc where cdouble >= -500 and cdouble < -199 order by s1, s2 limit 100; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q new file mode 100644 index 0000000000000..094a8d26a3842 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_limit.q @@ -0,0 +1,37 @@ +SET hive.vectorized.execution.enabled=true; +explain SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; +SELECT cbigint, cdouble FROM alltypesorc WHERE cbigint < cdouble and cint > 0 limit 7; + +set hive.optimize.reducededuplication.min.reducer=1; +set hive.limit.pushdown.memory.usage=0.3f; + +-- HIVE-3562 Some limit can be pushed down to map stage - c/p parts from limit_pushdown + +explain +select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; +select ctinyint,cdouble,csmallint from alltypesorc where ctinyint is not null order by ctinyint,cdouble limit 20; + +-- deduped RS +explain +select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; +select ctinyint,avg(cdouble + 1) from alltypesorc group by ctinyint order by ctinyint limit 20; + +-- distincts +explain +select distinct(ctinyint) from alltypesorc limit 20; +select distinct(ctinyint) from alltypesorc limit 20; + +explain +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; +select ctinyint, count(distinct(cdouble)) from alltypesorc group by ctinyint limit 20; + +-- limit zero +explain +select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; +select ctinyint,cdouble from alltypesorc order by ctinyint limit 0; + +-- 2MR (applied to last RS) +explain +select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; +select cdouble, sum(ctinyint) as sum from alltypesorc where ctinyint is not null group by cdouble order by sum, cdouble limit 20; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q new file mode 100644 index 0000000000000..bb50f9b853287 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_nested_udf.q @@ -0,0 +1,3 @@ +SET hive.vectorized.execution.enabled=true; +SELECT SUM(abs(ctinyint)) from alltypesorc; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q new file mode 100644 index 0000000000000..bfd3dd722190c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_not.q @@ -0,0 +1,27 @@ +SET hive.vectorized.execution.enabled=true; +SELECT AVG(cbigint), + (-(AVG(cbigint))), + (-6432 + AVG(cbigint)), + STDDEV_POP(cbigint), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) + (-6432 + AVG(cbigint))), + VAR_SAMP(cbigint), + (-((-6432 + AVG(cbigint)))), + (-6432 + (-((-6432 + AVG(cbigint))))), + (-((-6432 + AVG(cbigint)))), + ((-((-6432 + AVG(cbigint)))) / (-((-6432 + AVG(cbigint))))), + COUNT(*), + SUM(cfloat), + (VAR_SAMP(cbigint) % STDDEV_POP(cbigint)), + (-(VAR_SAMP(cbigint))), + ((-((-6432 + AVG(cbigint)))) * (-(AVG(cbigint)))), + MIN(ctinyint), + (-(MIN(ctinyint))) +FROM alltypesorc +WHERE (((cstring2 LIKE '%b%') + OR ((79.553 != cint) + OR (NOT(cbigint >= cdouble)))) + OR ((ctinyint >= csmallint) + AND (NOT ((cboolean2 != 1) + OR (3569 != ctinyint))))); + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q new file mode 100644 index 0000000000000..0e34585b5a8ba --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part.q @@ -0,0 +1,7 @@ +SET hive.vectorized.execution.enabled=true; +CREATE TABLE alltypesorc_part(ctinyint tinyint, csmallint smallint, cint int, cbigint bigint, cfloat float, cdouble double, cstring1 string, cstring2 string, ctimestamp1 timestamp, ctimestamp2 timestamp, cboolean1 boolean, cboolean2 boolean) partitioned by (ds string) STORED AS ORC; +insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100; +insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100; + +select count(cdouble), cint from alltypesorc_part where ds='2011' group by cint limit 10; +select count(*) from alltypesorc_part A join alltypesorc_part B on A.ds=B.ds; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q new file mode 100644 index 0000000000000..c68ce56fd47db --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_part_project.q @@ -0,0 +1,7 @@ +SET hive.vectorized.execution.enabled=true; +CREATE TABLE alltypesorc_part(ctinyint tinyint, csmallint smallint, cint int, cbigint bigint, cfloat float, cdouble double, cstring1 string, cstring2 string, ctimestamp1 timestamp, ctimestamp2 timestamp, cboolean1 boolean, cboolean2 boolean) partitioned by (ds string) STORED AS ORC; +insert overwrite table alltypesorc_part partition (ds='2011') select * from alltypesorc limit 100; +insert overwrite table alltypesorc_part partition (ds='2012') select * from alltypesorc limit 100; + +explain select (cdouble+2) c1 from alltypesorc_part order by c1 limit 10; +select (cdouble+2) c1 from alltypesorc_part order by c1 limit 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q new file mode 100644 index 0000000000000..bafe5504789e8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_pushdown.q @@ -0,0 +1,4 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.optimize.index.filter=true; +explain SELECT AVG(cbigint) FROM alltypesorc WHERE cbigint < cdouble; +SELECT AVG(cbigint) FROM alltypesorc WHERE cbigint < cdouble; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q new file mode 100644 index 0000000000000..638a31ff413b7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorization_short_regress.q @@ -0,0 +1,852 @@ +SET hive.vectorized.execution.enabled=true; + +-- If you look at ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/OrcFileGenerator.java +-- which is the data generation class you'll see that those values are specified in the +-- initializeFixedPointValues for each data type. When I created the queries I usedthose values +-- where I needed scalar values to ensure that when the queries executed their predicates would be +-- filtering on values that are guaranteed to exist. + +-- Beyond those values, all the other data in the alltypesorc file is random, but there is a +-- specific pattern to the data that is important for coverage. In orc and subsequently +-- vectorization there are a number of optimizations for certain data patterns: AllValues, NoNulls, +-- RepeatingValue, RepeatingNull. The data in alltypesorc is generated such that each column has +-- exactly 3 batches of each data pattern. This gives us coverage for the vector expression +-- optimizations and ensure the metadata in appropriately set on the row batch object which are +-- reused across batches. + +-- For the queries themselves in order to efficiently cover as much of the new vectorization +-- functionality as I could I used a number of different techniques to create the +-- vectorization_short_regress.q test suite, primarily equivalence classes, and pairwise +-- combinations. + +-- First I divided the search space into a number of dimensions such as type, aggregate function, +-- filter operation, arithmetic operation, etc. The types were explored as equivalence classes of +-- long, double, time, string, and bool. Also, rather than creating a very large number of small +-- queries the resulting vectors were grouped by compatible dimensions to reduce the number of +-- queries. + +-- TargetTypeClasses: Long, Timestamp, Double, String, Bool +-- Functions: Avg, Sum, StDevP, StDev, Var, Min, Count +-- ArithmeticOps: Add, Multiply, Subtract, Divide +-- FilterOps: Equal, NotEqual, GreaterThan, LessThan, LessThanOrEqual +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT AVG(cint), + (AVG(cint) + -3728), + (-((AVG(cint) + -3728))), + (-((-((AVG(cint) + -3728))))), + ((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)), + SUM(cdouble), + (-(AVG(cint))), + STDDEV_POP(cint), + (((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)))))), + STDDEV_SAMP(csmallint), + (-(STDDEV_POP(cint))), + (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))), + ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))) * STDDEV_POP(cint)), + VAR_SAMP(cint), + AVG(cfloat), + (10.175 - VAR_SAMP(cint)), + (-((10.175 - VAR_SAMP(cint)))), + ((-(STDDEV_POP(cint))) / -563), + STDDEV_SAMP(cint), + (-(((-(STDDEV_POP(cint))) / -563))), + (AVG(cint) / SUM(cdouble)), + MIN(ctinyint), + COUNT(csmallint), + (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)), + (-((AVG(cint) / SUM(cdouble)))) +FROM alltypesorc +WHERE ((762 = cbigint) + OR ((csmallint < cfloat) + AND ((ctimestamp2 > -10669) + AND (cdouble != cint))) + OR (cstring1 = 'a') + OR ((cbigint <= -1.389) + AND ((cstring2 != 'a') + AND ((79.553 != cint) + AND (cboolean2 != cboolean1))))); +SELECT AVG(cint), + (AVG(cint) + -3728), + (-((AVG(cint) + -3728))), + (-((-((AVG(cint) + -3728))))), + ((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)), + SUM(cdouble), + (-(AVG(cint))), + STDDEV_POP(cint), + (((-((-((AVG(cint) + -3728))))) * (AVG(cint) + -3728)) * (-((-((AVG(cint) + -3728)))))), + STDDEV_SAMP(csmallint), + (-(STDDEV_POP(cint))), + (STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))), + ((STDDEV_POP(cint) - (-((-((AVG(cint) + -3728)))))) * STDDEV_POP(cint)), + VAR_SAMP(cint), + AVG(cfloat), + (10.175 - VAR_SAMP(cint)), + (-((10.175 - VAR_SAMP(cint)))), + ((-(STDDEV_POP(cint))) / -563), + STDDEV_SAMP(cint), + (-(((-(STDDEV_POP(cint))) / -563))), + (AVG(cint) / SUM(cdouble)), + MIN(ctinyint), + COUNT(csmallint), + (MIN(ctinyint) / ((-(STDDEV_POP(cint))) / -563)), + (-((AVG(cint) / SUM(cdouble)))) +FROM alltypesorc +WHERE ((762 = cbigint) + OR ((csmallint < cfloat) + AND ((ctimestamp2 > -10669) + AND (cdouble != cint))) + OR (cstring1 = 'a') + OR ((cbigint <= -1.389) + AND ((cstring2 != 'a') + AND ((79.553 != cint) + AND (cboolean2 != cboolean1))))); + +-- TargetTypeClasses: Long, Bool, Double, String, Timestamp +-- Functions: Max, VarP, StDevP, Avg, Min, StDev, Var +-- ArithmeticOps: Divide, Multiply, Remainder, Subtract +-- FilterOps: LessThan, LessThanOrEqual, GreaterThan, GreaterThanOrEqual, Like, RLike +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT MAX(cint), + (MAX(cint) / -3728), + (MAX(cint) * -3728), + VAR_POP(cbigint), + (-((MAX(cint) * -3728))), + STDDEV_POP(csmallint), + (-563 % (MAX(cint) * -3728)), + (VAR_POP(cbigint) / STDDEV_POP(csmallint)), + (-(STDDEV_POP(csmallint))), + MAX(cdouble), + AVG(ctinyint), + (STDDEV_POP(csmallint) - 10.175), + MIN(cint), + ((MAX(cint) * -3728) % (STDDEV_POP(csmallint) - 10.175)), + (-(MAX(cdouble))), + MIN(cdouble), + (MAX(cdouble) % -26.28), + STDDEV_SAMP(csmallint), + (-((MAX(cint) / -3728))), + ((-((MAX(cint) * -3728))) % (-563 % (MAX(cint) * -3728))), + ((MAX(cint) / -3728) - AVG(ctinyint)), + (-((MAX(cint) * -3728))), + VAR_SAMP(cint) +FROM alltypesorc +WHERE (((cbigint <= 197) + AND (cint < cbigint)) + OR ((cdouble >= -26.28) + AND (csmallint > cdouble)) + OR ((ctinyint > cfloat) + AND (cstring1 RLIKE '.*ss.*')) + OR ((cfloat > 79.553) + AND (cstring2 LIKE '10%'))); +SELECT MAX(cint), + (MAX(cint) / -3728), + (MAX(cint) * -3728), + VAR_POP(cbigint), + (-((MAX(cint) * -3728))), + STDDEV_POP(csmallint), + (-563 % (MAX(cint) * -3728)), + (VAR_POP(cbigint) / STDDEV_POP(csmallint)), + (-(STDDEV_POP(csmallint))), + MAX(cdouble), + AVG(ctinyint), + (STDDEV_POP(csmallint) - 10.175), + MIN(cint), + ((MAX(cint) * -3728) % (STDDEV_POP(csmallint) - 10.175)), + (-(MAX(cdouble))), + MIN(cdouble), + (MAX(cdouble) % -26.28), + STDDEV_SAMP(csmallint), + (-((MAX(cint) / -3728))), + ((-((MAX(cint) * -3728))) % (-563 % (MAX(cint) * -3728))), + ((MAX(cint) / -3728) - AVG(ctinyint)), + (-((MAX(cint) * -3728))), + VAR_SAMP(cint) +FROM alltypesorc +WHERE (((cbigint <= 197) + AND (cint < cbigint)) + OR ((cdouble >= -26.28) + AND (csmallint > cdouble)) + OR ((ctinyint > cfloat) + AND (cstring1 RLIKE '.*ss.*')) + OR ((cfloat > 79.553) + AND (cstring2 LIKE '10%'))); + +-- TargetTypeClasses: String, Long, Bool, Double, Timestamp +-- Functions: VarP, Count, Max, StDevP, StDev, Avg +-- ArithmeticOps: Subtract, Remainder, Multiply, Add +-- FilterOps: Equal, LessThanOrEqual, GreaterThan, Like, LessThan +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT VAR_POP(cbigint), + (-(VAR_POP(cbigint))), + (VAR_POP(cbigint) - (-(VAR_POP(cbigint)))), + COUNT(*), + (COUNT(*) % 79.553), + MAX(ctinyint), + (COUNT(*) - (-(VAR_POP(cbigint)))), + (-((-(VAR_POP(cbigint))))), + (-1 % (-(VAR_POP(cbigint)))), + COUNT(*), + (-(COUNT(*))), + STDDEV_POP(csmallint), + (-((-((-(VAR_POP(cbigint))))))), + (762 * (-(COUNT(*)))), + MAX(cint), + (MAX(ctinyint) + (762 * (-(COUNT(*))))), + ((-(VAR_POP(cbigint))) + MAX(cint)), + STDDEV_SAMP(cdouble), + ((-(COUNT(*))) % COUNT(*)), + COUNT(ctinyint), + AVG(ctinyint), + (-3728 % (MAX(ctinyint) + (762 * (-(COUNT(*)))))) +FROM alltypesorc +WHERE ((ctimestamp1 = ctimestamp2) + OR (762 = cfloat) + OR (cstring1 = 'ss') + OR ((csmallint <= cbigint) + AND (1 = cboolean2)) + OR ((cboolean1 IS NOT NULL) + AND ((ctimestamp2 IS NOT NULL) + AND (cstring2 > 'a')))); +SELECT VAR_POP(cbigint), + (-(VAR_POP(cbigint))), + (VAR_POP(cbigint) - (-(VAR_POP(cbigint)))), + COUNT(*), + (COUNT(*) % 79.553), + MAX(ctinyint), + (COUNT(*) - (-(VAR_POP(cbigint)))), + (-((-(VAR_POP(cbigint))))), + (-1 % (-(VAR_POP(cbigint)))), + COUNT(*), + (-(COUNT(*))), + STDDEV_POP(csmallint), + (-((-((-(VAR_POP(cbigint))))))), + (762 * (-(COUNT(*)))), + MAX(cint), + (MAX(ctinyint) + (762 * (-(COUNT(*))))), + ((-(VAR_POP(cbigint))) + MAX(cint)), + STDDEV_SAMP(cdouble), + ((-(COUNT(*))) % COUNT(*)), + COUNT(ctinyint), + AVG(ctinyint), + (-3728 % (MAX(ctinyint) + (762 * (-(COUNT(*)))))) +FROM alltypesorc +WHERE ((ctimestamp1 = ctimestamp2) + OR (762 = cfloat) + OR (cstring1 = 'ss') + OR ((csmallint <= cbigint) + AND (1 = cboolean2)) + OR ((cboolean1 IS NOT NULL) + AND ((ctimestamp2 IS NOT NULL) + AND (cstring2 > 'a')))); + +-- TargetTypeClasses: String, Bool, Timestamp, Long, Double +-- Functions: Avg, Max, StDev, VarP +-- ArithmeticOps: Add, Divide, Remainder, Multiply +-- FilterOps: LessThanOrEqual, NotEqual, GreaterThanOrEqual, LessThan, Equal +-- GroupBy: NoGroupByProjectAggs +EXPLAIN SELECT AVG(ctinyint), + (AVG(ctinyint) + 6981), + ((AVG(ctinyint) + 6981) + AVG(ctinyint)), + MAX(cbigint), + (((AVG(ctinyint) + 6981) + AVG(ctinyint)) / AVG(ctinyint)), + (-((AVG(ctinyint) + 6981))), + STDDEV_SAMP(cint), + (AVG(ctinyint) % (-((AVG(ctinyint) + 6981)))), + VAR_POP(cint), + VAR_POP(cbigint), + (-(MAX(cbigint))), + ((-(MAX(cbigint))) / STDDEV_SAMP(cint)), + MAX(cfloat), + (VAR_POP(cbigint) * -26.28) +FROM alltypesorc +WHERE (((ctimestamp2 <= ctimestamp1) + AND ((cbigint != cdouble) + AND ('ss' <= cstring1))) + OR ((csmallint < ctinyint) + AND (ctimestamp1 >= 0)) + OR (cfloat = 17)); +SELECT AVG(ctinyint), + (AVG(ctinyint) + 6981), + ((AVG(ctinyint) + 6981) + AVG(ctinyint)), + MAX(cbigint), + (((AVG(ctinyint) + 6981) + AVG(ctinyint)) / AVG(ctinyint)), + (-((AVG(ctinyint) + 6981))), + STDDEV_SAMP(cint), + (AVG(ctinyint) % (-((AVG(ctinyint) + 6981)))), + VAR_POP(cint), + VAR_POP(cbigint), + (-(MAX(cbigint))), + ((-(MAX(cbigint))) / STDDEV_SAMP(cint)), + MAX(cfloat), + (VAR_POP(cbigint) * -26.28) +FROM alltypesorc +WHERE (((ctimestamp2 <= ctimestamp1) + AND ((cbigint != cdouble) + AND ('ss' <= cstring1))) + OR ((csmallint < ctinyint) + AND (ctimestamp1 >= 0)) + OR (cfloat = 17)); + +-- TargetTypeClasses: Timestamp, String, Long, Double, Bool +-- Functions: Max, Avg, Min, Var, StDev, Count, StDevP, Sum +-- ArithmeticOps: Multiply, Subtract, Add, Divide +-- FilterOps: Like, NotEqual, LessThan, GreaterThanOrEqual, GreaterThan, RLike +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT cint, + cdouble, + ctimestamp2, + cstring1, + cboolean2, + ctinyint, + cfloat, + ctimestamp1, + csmallint, + cbigint, + (-3728 * cbigint), + (-(cint)), + (-863.257 - cint), + (-(csmallint)), + (csmallint - (-(csmallint))), + ((csmallint - (-(csmallint))) + (-(csmallint))), + (cint / cint), + ((-863.257 - cint) - -26.28), + (-(cfloat)), + (cdouble * -89010), + (ctinyint / 988888), + (-(ctinyint)), + (79.553 / ctinyint) +FROM alltypesorc +WHERE (((cstring1 RLIKE 'a.*') + AND (cstring2 LIKE '%ss%')) + OR ((1 != cboolean2) + AND ((csmallint < 79.553) + AND (-257 != ctinyint))) + OR ((cdouble > ctinyint) + AND (cfloat >= cint)) + OR ((cint < cbigint) + AND (ctinyint > cbigint))); +SELECT cint, + cdouble, + ctimestamp2, + cstring1, + cboolean2, + ctinyint, + cfloat, + ctimestamp1, + csmallint, + cbigint, + (-3728 * cbigint), + (-(cint)), + (-863.257 - cint), + (-(csmallint)), + (csmallint - (-(csmallint))), + ((csmallint - (-(csmallint))) + (-(csmallint))), + (cint / cint), + ((-863.257 - cint) - -26.28), + (-(cfloat)), + (cdouble * -89010), + (ctinyint / 988888), + (-(ctinyint)), + (79.553 / ctinyint) +FROM alltypesorc +WHERE (((cstring1 RLIKE 'a.*') + AND (cstring2 LIKE '%ss%')) + OR ((1 != cboolean2) + AND ((csmallint < 79.553) + AND (-257 != ctinyint))) + OR ((cdouble > ctinyint) + AND (cfloat >= cint)) + OR ((cint < cbigint) + AND (ctinyint > cbigint))); + +-- TargetTypeClasses: Long, String, Double, Bool, Timestamp +-- Functions: VarP, Var, StDev, StDevP, Max, Sum +-- ArithmeticOps: Divide, Remainder, Subtract, Multiply +-- FilterOps: Equal, LessThanOrEqual, LessThan, Like, GreaterThanOrEqual, NotEqual, GreaterThan +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT cint, + cbigint, + cstring1, + cboolean1, + cfloat, + cdouble, + ctimestamp2, + csmallint, + cstring2, + cboolean2, + (cint / cbigint), + (cbigint % 79.553), + (-((cint / cbigint))), + (10.175 % cfloat), + (-(cfloat)), + (cfloat - (-(cfloat))), + ((cfloat - (-(cfloat))) % -6432), + (cdouble * csmallint), + (-(cdouble)), + (-(cbigint)), + (cfloat - (cint / cbigint)), + (-(csmallint)), + (3569 % cbigint), + (359 - cdouble), + (-(csmallint)) +FROM alltypesorc +WHERE (((197 > ctinyint) + AND (cint = cbigint)) + OR (cbigint = 359) + OR (cboolean1 < 0) + OR ((cstring1 LIKE '%ss') + AND (cfloat <= ctinyint))); + +SELECT cint, + cbigint, + cstring1, + cboolean1, + cfloat, + cdouble, + ctimestamp2, + csmallint, + cstring2, + cboolean2, + (cint / cbigint), + (cbigint % 79.553), + (-((cint / cbigint))), + (10.175 % cfloat), + (-(cfloat)), + (cfloat - (-(cfloat))), + ((cfloat - (-(cfloat))) % -6432), + (cdouble * csmallint), + (-(cdouble)), + (-(cbigint)), + (cfloat - (cint / cbigint)), + (-(csmallint)), + (3569 % cbigint), + (359 - cdouble), + (-(csmallint)) +FROM alltypesorc +WHERE (((197 > ctinyint) + AND (cint = cbigint)) + OR (cbigint = 359) + OR (cboolean1 < 0) + OR ((cstring1 LIKE '%ss') + AND (cfloat <= ctinyint))); + +-- TargetTypeClasses: String, Bool, Double, Long, Timestamp +-- Functions: Sum, Max, Avg, Var, StDevP, VarP +-- ArithmeticOps: Add, Subtract, Divide, Multiply, Remainder +-- FilterOps: NotEqual, GreaterThanOrEqual, Like, LessThanOrEqual, Equal, GreaterThan +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT cint, + cstring1, + cboolean2, + ctimestamp2, + cdouble, + cfloat, + cbigint, + csmallint, + cboolean1, + (cint + csmallint), + (cbigint - ctinyint), + (-(cbigint)), + (-(cfloat)), + ((cbigint - ctinyint) + cbigint), + (cdouble / cdouble), + (-(cdouble)), + ((cint + csmallint) * (-(cbigint))), + ((-(cdouble)) + cbigint), + (-1.389 / ctinyint), + (cbigint % cdouble), + (-(csmallint)), + (csmallint + (cint + csmallint)) +FROM alltypesorc +WHERE (((csmallint > -26.28) + AND (cstring2 LIKE 'ss')) + OR ((cdouble <= cbigint) + AND ((cstring1 >= 'ss') + AND (cint != cdouble))) + OR (ctinyint = -89010) + OR ((cbigint <= cfloat) + AND (-26.28 <= csmallint))) +ORDER BY cboolean1, cstring1, ctimestamp2, cfloat, cbigint, cstring1, cdouble, cint, csmallint, cdouble; +SELECT cint, + cstring1, + cboolean2, + ctimestamp2, + cdouble, + cfloat, + cbigint, + csmallint, + cboolean1, + (cint + csmallint), + (cbigint - ctinyint), + (-(cbigint)), + (-(cfloat)), + ((cbigint - ctinyint) + cbigint), + (cdouble / cdouble), + (-(cdouble)), + ((cint + csmallint) * (-(cbigint))), + ((-(cdouble)) + cbigint), + (-1.389 / ctinyint), + (cbigint % cdouble), + (-(csmallint)), + (csmallint + (cint + csmallint)) +FROM alltypesorc +WHERE (((csmallint > -26.28) + AND (cstring2 LIKE 'ss')) + OR ((cdouble <= cbigint) + AND ((cstring1 >= 'ss') + AND (cint != cdouble))) + OR (ctinyint = -89010) + OR ((cbigint <= cfloat) + AND (-26.28 <= csmallint))) +ORDER BY cboolean1, cstring1, ctimestamp2, cfloat, cbigint, cstring1, cdouble, cint, csmallint, cdouble; + +-- TargetTypeClasses: Long, String, Double, Timestamp +-- Functions: Avg, Min, StDevP, Sum, Var +-- ArithmeticOps: Divide, Subtract, Multiply, Remainder +-- FilterOps: GreaterThan, LessThan, LessThanOrEqual, GreaterThanOrEqual, Like +-- GroupBy: NoGroupByProjectColumns +EXPLAIN SELECT ctimestamp1, + cstring2, + cdouble, + cfloat, + cbigint, + csmallint, + (cbigint / 3569), + (-257 - csmallint), + (-6432 * cfloat), + (-(cdouble)), + (cdouble * 10.175), + ((-6432 * cfloat) / cfloat), + (-(cfloat)), + (cint % csmallint), + (-(cdouble)), + (cdouble * (-(cdouble))) +FROM alltypesorc +WHERE (((-1.389 >= cint) + AND ((csmallint < ctinyint) + AND (-6432 > csmallint))) + OR ((cdouble >= cfloat) + AND (cstring2 <= 'a')) + OR ((cstring1 LIKE 'ss%') + AND (10.175 > cbigint))) +ORDER BY csmallint, cstring2, cdouble; +SELECT ctimestamp1, + cstring2, + cdouble, + cfloat, + cbigint, + csmallint, + (cbigint / 3569), + (-257 - csmallint), + (-6432 * cfloat), + (-(cdouble)), + (cdouble * 10.175), + ((-6432 * cfloat) / cfloat), + (-(cfloat)), + (cint % csmallint), + (-(cdouble)), + (cdouble * (-(cdouble))) +FROM alltypesorc +WHERE (((-1.389 >= cint) + AND ((csmallint < ctinyint) + AND (-6432 > csmallint))) + OR ((cdouble >= cfloat) + AND (cstring2 <= 'a')) + OR ((cstring1 LIKE 'ss%') + AND (10.175 > cbigint))) +ORDER BY csmallint, cstring2, cdouble; + +-- TargetTypeClasses: Double, String, Long +-- Functions: StDev, Sum, VarP, Count +-- ArithmeticOps: Remainder, Divide, Subtract +-- FilterOps: GreaterThanOrEqual, Equal, LessThanOrEqual +-- GroupBy: GroupBy +EXPLAIN SELECT csmallint, + (csmallint % -75), + STDDEV_SAMP(csmallint), + (-1.389 / csmallint), + SUM(cbigint), + ((csmallint % -75) / SUM(cbigint)), + (-((csmallint % -75))), + VAR_POP(ctinyint), + (-((-((csmallint % -75))))), + COUNT(*), + (COUNT(*) - -89010) +FROM alltypesorc +WHERE (((csmallint >= -257)) + AND ((-6432 = csmallint) + OR ((cint >= cdouble) + AND (ctinyint <= cint)))) +GROUP BY csmallint +ORDER BY csmallint; +SELECT csmallint, + (csmallint % -75), + STDDEV_SAMP(csmallint), + (-1.389 / csmallint), + SUM(cbigint), + ((csmallint % -75) / SUM(cbigint)), + (-((csmallint % -75))), + VAR_POP(ctinyint), + (-((-((csmallint % -75))))), + COUNT(*), + (COUNT(*) - -89010) +FROM alltypesorc +WHERE (((csmallint >= -257)) + AND ((-6432 = csmallint) + OR ((cint >= cdouble) + AND (ctinyint <= cint)))) +GROUP BY csmallint +ORDER BY csmallint; + +-- TargetTypeClasses: Long, Double, Timestamp +-- Functions: Var, Count, Sum, VarP, StDevP +-- ArithmeticOps: Multiply, Add, Subtract, Remainder +-- FilterOps: GreaterThan, LessThan, Equal, LessThanOrEqual, GreaterThanOrEqual +-- GroupBy: GroupBy +EXPLAIN SELECT cdouble, + VAR_SAMP(cdouble), + (2563.58 * VAR_SAMP(cdouble)), + (-(VAR_SAMP(cdouble))), + COUNT(cfloat), + ((2563.58 * VAR_SAMP(cdouble)) + -5638.15), + ((-(VAR_SAMP(cdouble))) * ((2563.58 * VAR_SAMP(cdouble)) + -5638.15)), + SUM(cfloat), + VAR_POP(cdouble), + (cdouble - (-(VAR_SAMP(cdouble)))), + STDDEV_POP(cdouble), + (cdouble + VAR_SAMP(cdouble)), + (cdouble * 762), + SUM(cdouble), + (-863.257 % (cdouble * 762)), + SUM(cdouble) +FROM alltypesorc +WHERE (((cdouble > 2563.58)) + AND (((cbigint >= cint) + AND ((csmallint < cint) + AND (cfloat < -5638.15))) + OR (2563.58 = ctinyint) + OR ((cdouble <= cbigint) + AND (-5638.15 > cbigint)))) +GROUP BY cdouble +ORDER BY cdouble; +SELECT cdouble, + VAR_SAMP(cdouble), + (2563.58 * VAR_SAMP(cdouble)), + (-(VAR_SAMP(cdouble))), + COUNT(cfloat), + ((2563.58 * VAR_SAMP(cdouble)) + -5638.15), + ((-(VAR_SAMP(cdouble))) * ((2563.58 * VAR_SAMP(cdouble)) + -5638.15)), + SUM(cfloat), + VAR_POP(cdouble), + (cdouble - (-(VAR_SAMP(cdouble)))), + STDDEV_POP(cdouble), + (cdouble + VAR_SAMP(cdouble)), + (cdouble * 762), + SUM(cdouble), + (-863.257 % (cdouble * 762)), + SUM(cdouble) +FROM alltypesorc +WHERE (((cdouble > 2563.58)) + AND (((cbigint >= cint) + AND ((csmallint < cint) + AND (cfloat < -5638.15))) + OR (2563.58 = ctinyint) + OR ((cdouble <= cbigint) + AND (-5638.15 > cbigint)))) +GROUP BY cdouble +ORDER BY cdouble; + +-- TargetTypeClasses: Bool, Timestamp, String, Double, Long +-- Functions: StDevP, Avg, Count, Min, Var, VarP, Sum +-- ArithmeticOps: Multiply, Subtract, Add, Divide, Remainder +-- FilterOps: NotEqual, LessThan, Like, Equal, RLike +-- GroupBy: GroupBy +EXPLAIN SELECT ctimestamp1, + cstring1, + STDDEV_POP(cint), + (STDDEV_POP(cint) * 10.175), + (-(STDDEV_POP(cint))), + AVG(csmallint), + (-(STDDEV_POP(cint))), + (-26.28 - STDDEV_POP(cint)), + COUNT(*), + (-(COUNT(*))), + ((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))), + MIN(ctinyint), + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*)))), + (-((STDDEV_POP(cint) * 10.175))), + VAR_SAMP(csmallint), + (VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-(STDDEV_POP(cint))))), + ((-(COUNT(*))) / STDDEV_POP(cint)), + VAR_POP(cfloat), + (10.175 / AVG(csmallint)), + AVG(cint), + VAR_SAMP(cfloat), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-((STDDEV_POP(cint) * 10.175))))), + AVG(cfloat), + (((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) * 10.175), + (10.175 % (10.175 / AVG(csmallint))), + (-(MIN(ctinyint))), + MIN(cdouble), + VAR_POP(csmallint), + (-(((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))))), + ((-(STDDEV_POP(cint))) % AVG(cfloat)), + (-26.28 / (-(MIN(ctinyint)))), + STDDEV_POP(ctinyint), + SUM(cint), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) / VAR_POP(cfloat)), + (-((-(COUNT(*))))), + COUNT(*), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) % -26.28) +FROM alltypesorc +WHERE (((ctimestamp1 != 0)) + AND ((((-257 != ctinyint) + AND (cboolean2 IS NOT NULL)) + AND ((cstring1 RLIKE '.*ss') + AND (-10669 < ctimestamp1))) + OR (ctimestamp2 = -10669) + OR ((ctimestamp1 < 0) + AND (cstring2 LIKE '%b%')) + OR (cdouble = cint) + OR ((cboolean1 IS NULL) + AND (cfloat < cint)))) +GROUP BY ctimestamp1, cstring1; +SELECT ctimestamp1, + cstring1, + STDDEV_POP(cint), + (STDDEV_POP(cint) * 10.175), + (-(STDDEV_POP(cint))), + AVG(csmallint), + (-(STDDEV_POP(cint))), + (-26.28 - STDDEV_POP(cint)), + COUNT(*), + (-(COUNT(*))), + ((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))), + MIN(ctinyint), + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*)))), + (-((STDDEV_POP(cint) * 10.175))), + VAR_SAMP(csmallint), + (VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-(STDDEV_POP(cint))))), + ((-(COUNT(*))) / STDDEV_POP(cint)), + VAR_POP(cfloat), + (10.175 / AVG(csmallint)), + AVG(cint), + VAR_SAMP(cfloat), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))), + (-((-((STDDEV_POP(cint) * 10.175))))), + AVG(cfloat), + (((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) - (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) * 10.175), + (10.175 % (10.175 / AVG(csmallint))), + (-(MIN(ctinyint))), + MIN(cdouble), + VAR_POP(csmallint), + (-(((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))))), + ((-(STDDEV_POP(cint))) % AVG(cfloat)), + (-26.28 / (-(MIN(ctinyint)))), + STDDEV_POP(ctinyint), + SUM(cint), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) / VAR_POP(cfloat)), + (-((-(COUNT(*))))), + COUNT(*), + ((VAR_SAMP(csmallint) + (((-26.28 - STDDEV_POP(cint)) * (-(STDDEV_POP(cint)))) * (-(COUNT(*))))) % -26.28) +FROM alltypesorc +WHERE (((ctimestamp1 != 0)) + AND ((((-257 != ctinyint) + AND (cboolean2 IS NOT NULL)) + AND ((cstring1 RLIKE '.*ss') + AND (-10669 < ctimestamp1))) + OR (ctimestamp2 = -10669) + OR ((ctimestamp1 < 0) + AND (cstring2 LIKE '%b%')) + OR (cdouble = cint) + OR ((cboolean1 IS NULL) + AND (cfloat < cint)))) +GROUP BY ctimestamp1, cstring1; + +-- TargetTypeClasses: Double, Long, String, Timestamp, Bool +-- Functions: Max, Sum, Var, Avg, Min, VarP, StDev, StDevP +-- ArithmeticOps: Divide, Subtract, Remainder, Add, Multiply +-- FilterOps: GreaterThan, LessThanOrEqual, Equal, LessThan, GreaterThanOrEqual, NotEqual, Like, RLike +-- GroupBy: GroupBy +EXPLAIN SELECT cboolean1, + MAX(cfloat), + (-(MAX(cfloat))), + (-26.28 / MAX(cfloat)), + SUM(cbigint), + (SUM(cbigint) - 10.175), + VAR_SAMP(cint), + (VAR_SAMP(cint) % MAX(cfloat)), + (10.175 + (-(MAX(cfloat)))), + AVG(cdouble), + ((SUM(cbigint) - 10.175) + VAR_SAMP(cint)), + MIN(cbigint), + VAR_POP(cbigint), + (-((10.175 + (-(MAX(cfloat)))))), + (79.553 / VAR_POP(cbigint)), + (VAR_SAMP(cint) % (79.553 / VAR_POP(cbigint))), + (-((10.175 + (-(MAX(cfloat)))))), + SUM(cint), + STDDEV_SAMP(ctinyint), + (-1.389 * MIN(cbigint)), + (SUM(cint) - (-1.389 * MIN(cbigint))), + STDDEV_POP(csmallint), + (-((SUM(cint) - (-1.389 * MIN(cbigint))))), + AVG(cint), + (-(AVG(cint))), + (AVG(cint) * SUM(cint)) +FROM alltypesorc +WHERE (((cboolean1 IS NOT NULL)) + AND (((cdouble < csmallint) + AND ((cboolean2 = cboolean1) + AND (cbigint <= -863.257))) + OR ((cint >= -257) + AND ((cstring1 IS NOT NULL) + AND (cboolean1 >= 1))) + OR (cstring2 RLIKE 'b') + OR ((csmallint >= ctinyint) + AND (ctimestamp2 IS NULL)))) +GROUP BY cboolean1 +ORDER BY cboolean1; +SELECT cboolean1, + MAX(cfloat), + (-(MAX(cfloat))), + (-26.28 / MAX(cfloat)), + SUM(cbigint), + (SUM(cbigint) - 10.175), + VAR_SAMP(cint), + (VAR_SAMP(cint) % MAX(cfloat)), + (10.175 + (-(MAX(cfloat)))), + AVG(cdouble), + ((SUM(cbigint) - 10.175) + VAR_SAMP(cint)), + MIN(cbigint), + VAR_POP(cbigint), + (-((10.175 + (-(MAX(cfloat)))))), + (79.553 / VAR_POP(cbigint)), + (VAR_SAMP(cint) % (79.553 / VAR_POP(cbigint))), + (-((10.175 + (-(MAX(cfloat)))))), + SUM(cint), + STDDEV_SAMP(ctinyint), + (-1.389 * MIN(cbigint)), + (SUM(cint) - (-1.389 * MIN(cbigint))), + STDDEV_POP(csmallint), + (-((SUM(cint) - (-1.389 * MIN(cbigint))))), + AVG(cint), + (-(AVG(cint))), + (AVG(cint) * SUM(cint)) +FROM alltypesorc +WHERE (((cboolean1 IS NOT NULL)) + AND (((cdouble < csmallint) + AND ((cboolean2 = cboolean1) + AND (cbigint <= -863.257))) + OR ((cint >= -257) + AND ((cstring1 IS NOT NULL) + AND (cboolean1 >= 1))) + OR (cstring2 RLIKE 'b') + OR ((csmallint >= ctinyint) + AND (ctimestamp2 IS NULL)))) +GROUP BY cboolean1 +ORDER BY cboolean1; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q new file mode 100644 index 0000000000000..e309713795af8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q @@ -0,0 +1,46 @@ +create table vsmb_bucket_1(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS ORC; +create table vsmb_bucket_2(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS ORC; + +create table vsmb_bucket_RC(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS RCFILE; + +create table vsmb_bucket_TXT(key int, value string) + CLUSTERED BY (key) + SORTED BY (key) INTO 1 BUCKETS + STORED AS TEXTFILE; + +insert into table vsmb_bucket_1 select cint, cstring1 from alltypesorc limit 2; +insert into table vsmb_bucket_2 select cint, cstring1 from alltypesorc limit 2; +insert into table vsmb_bucket_RC select cint, cstring1 from alltypesorc limit 2; +insert into table vsmb_bucket_TXT select cint, cstring1 from alltypesorc limit 2; + +set hive.vectorized.execution.enabled=true; +set hive.optimize.bucketmapjoin = true; +set hive.optimize.bucketmapjoin.sortedmerge = true; +set hive.auto.convert.sortmerge.join.noconditionaltask = true; +set hive.input.format = org.apache.hadoop.hive.ql.io.BucketizedHiveInputFormat; + +explain +select /*+MAPJOIN(a)*/ * from vsmb_bucket_1 a join vsmb_bucket_2 b on a.key = b.key; +select /*+MAPJOIN(a)*/ * from vsmb_bucket_1 a join vsmb_bucket_2 b on a.key = b.key; + +explain +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_RC b on a.key = b.key; +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_RC b on a.key = b.key; + +-- RC file does not yet provide the vectorized CommonRCFileformat out-of-the-box +-- explain +-- select /*+MAPJOIN(b)*/ * from vsmb_bucket_RC a join vsmb_bucket_2 b on a.key = b.key; +-- select /*+MAPJOIN(b)*/ * from vsmb_bucket_RC a join vsmb_bucket_2 b on a.key = b.key; + +explain +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_TXT b on a.key = b.key; +select /*+MAPJOIN(b)*/ * from vsmb_bucket_1 a join vsmb_bucket_TXT b on a.key = b.key; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q new file mode 100644 index 0000000000000..e448d51f6bc30 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_case.q @@ -0,0 +1,37 @@ +set hive.vectorized.execution.enabled = true +; +explain +select + csmallint, + case + when csmallint = 418 then "a" + when csmallint = 12205 then "b" + else "c" + end, + case csmallint + when 418 then "a" + when 12205 then "b" + else "c" + end +from alltypesorc +where csmallint = 418 +or csmallint = 12205 +or csmallint = 10583 +; +select + csmallint, + case + when csmallint = 418 then "a" + when csmallint = 12205 then "b" + else "c" + end, + case csmallint + when 418 then "a" + when 12205 then "b" + else "c" + end +from alltypesorc +where csmallint = 418 +or csmallint = 12205 +or csmallint = 10583 +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q new file mode 100644 index 0000000000000..3f818b18534af --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_casts.q @@ -0,0 +1,149 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test type casting in vectorized mode to verify end-to-end functionality. + +explain +select +-- to boolean + cast (ctinyint as boolean) + ,cast (csmallint as boolean) + ,cast (cint as boolean) + ,cast (cbigint as boolean) + ,cast (cfloat as boolean) + ,cast (cdouble as boolean) + ,cast (cboolean1 as boolean) + ,cast (cbigint * 0 as boolean) + ,cast (ctimestamp1 as boolean) + ,cast (cstring1 as boolean) +-- to int family + ,cast (ctinyint as int) + ,cast (csmallint as int) + ,cast (cint as int) + ,cast (cbigint as int) + ,cast (cfloat as int) + ,cast (cdouble as int) + ,cast (cboolean1 as int) + ,cast (ctimestamp1 as int) + ,cast (cstring1 as int) + ,cast (substr(cstring1, 1, 1) as int) + ,cast (cfloat as tinyint) + ,cast (cfloat as smallint) + ,cast (cfloat as bigint) +-- to float family + ,cast (ctinyint as double) + ,cast (csmallint as double) + ,cast (cint as double) + ,cast (cbigint as double) + ,cast (cfloat as double) + ,cast (cdouble as double) + ,cast (cboolean1 as double) + ,cast (ctimestamp1 as double) + ,cast (cstring1 as double) + ,cast (substr(cstring1, 1, 1) as double) + ,cast (cint as float) + ,cast (cdouble as float) +-- to timestamp + ,cast (ctinyint as timestamp) + ,cast (csmallint as timestamp) + ,cast (cint as timestamp) + ,cast (cbigint as timestamp) + ,cast (cfloat as timestamp) + ,cast (cdouble as timestamp) + ,cast (cboolean1 as timestamp) + ,cast (cbigint * 0 as timestamp) + ,cast (ctimestamp1 as timestamp) + ,cast (cstring1 as timestamp) + ,cast (substr(cstring1, 1, 1) as timestamp) +-- to string + ,cast (ctinyint as string) + ,cast (csmallint as string) + ,cast (cint as string) + ,cast (cbigint as string) + ,cast (cfloat as string) + ,cast (cdouble as string) + ,cast (cboolean1 as string) + ,cast (cbigint * 0 as string) + ,cast (ctimestamp1 as string) + ,cast (cstring1 as string) +-- nested and expression arguments + ,cast (cast (cfloat as int) as float) + ,cast (cint * 2 as double) + ,cast (sin(cfloat) as string) + ,cast (cint as float) + cast(cboolean1 as double) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 250 = 0; + + +select +-- to boolean + cast (ctinyint as boolean) + ,cast (csmallint as boolean) + ,cast (cint as boolean) + ,cast (cbigint as boolean) + ,cast (cfloat as boolean) + ,cast (cdouble as boolean) + ,cast (cboolean1 as boolean) + ,cast (cbigint * 0 as boolean) + ,cast (ctimestamp1 as boolean) + ,cast (cstring1 as boolean) +-- to int family + ,cast (ctinyint as int) + ,cast (csmallint as int) + ,cast (cint as int) + ,cast (cbigint as int) + ,cast (cfloat as int) + ,cast (cdouble as int) + ,cast (cboolean1 as int) + ,cast (ctimestamp1 as int) + ,cast (cstring1 as int) + ,cast (substr(cstring1, 1, 1) as int) + ,cast (cfloat as tinyint) + ,cast (cfloat as smallint) + ,cast (cfloat as bigint) +-- to float family + ,cast (ctinyint as double) + ,cast (csmallint as double) + ,cast (cint as double) + ,cast (cbigint as double) + ,cast (cfloat as double) + ,cast (cdouble as double) + ,cast (cboolean1 as double) + ,cast (ctimestamp1 as double) + ,cast (cstring1 as double) + ,cast (substr(cstring1, 1, 1) as double) + ,cast (cint as float) + ,cast (cdouble as float) +-- to timestamp + ,cast (ctinyint as timestamp) + ,cast (csmallint as timestamp) + ,cast (cint as timestamp) + ,cast (cbigint as timestamp) + ,cast (cfloat as timestamp) + ,cast (cdouble as timestamp) + ,cast (cboolean1 as timestamp) + ,cast (cbigint * 0 as timestamp) + ,cast (ctimestamp1 as timestamp) + ,cast (cstring1 as timestamp) + ,cast (substr(cstring1, 1, 1) as timestamp) +-- to string + ,cast (ctinyint as string) + ,cast (csmallint as string) + ,cast (cint as string) + ,cast (cbigint as string) + ,cast (cfloat as string) + ,cast (cdouble as string) + ,cast (cboolean1 as string) + ,cast (cbigint * 0 as string) + ,cast (ctimestamp1 as string) + ,cast (cstring1 as string) +-- nested and expression arguments + ,cast (cast (cfloat as int) as float) + ,cast (cint * 2 as double) + ,cast (sin(cfloat) as string) + ,cast (cint as float) + cast(cboolean1 as double) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 250 = 0; + + \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q new file mode 100644 index 0000000000000..381e4255ca51c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_context.q @@ -0,0 +1,47 @@ +create table store(s_store_sk int, s_city string) +stored as orc; +insert overwrite table store +select cint, cstring1 +from alltypesorc +where cint not in ( +-3728, -563, 762, 6981, 253665376, 528534767, 626923679); +create table store_sales(ss_store_sk int, ss_hdemo_sk int, ss_net_profit double) +stored as orc; +insert overwrite table store_sales +select cint, cint, cdouble +from alltypesorc +where cint not in ( +-3728, -563, 762, 6981, 253665376, 528534767, 626923679); +create table household_demographics(hd_demo_sk int) +stored as orc; +insert overwrite table household_demographics +select cint +from alltypesorc +where cint not in ( +-3728, -563, 762, 6981, 253665376, 528534767, 626923679); +set hive.auto.convert.join=true; +set hive.vectorized.execution.enabled=true; + + +explain +select store.s_city, ss_net_profit +from store_sales +JOIN store ON store_sales.ss_store_sk = store.s_store_sk +JOIN household_demographics ON store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk +limit 100 +; + +select store.s_city, ss_net_profit +from store_sales +JOIN store ON store_sales.ss_store_sk = store.s_store_sk +JOIN household_demographics ON store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk +limit 100 +; + +set hive.auto.convert.join=false; +set hive.vectorized.execution.enabled=false; + +drop table store; +drop table store_sales; +drop table household_demographics; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q new file mode 100644 index 0000000000000..b7aa3c28ac059 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_date_funcs.q @@ -0,0 +1,122 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test timestamp functions in vectorized mode to verify they run correctly end-to-end. + +CREATE TABLE date_udf_flight ( + origin_city_name STRING, + dest_city_name STRING, + fl_date DATE, + arr_delay FLOAT, + fl_num INT +); +LOAD DATA LOCAL INPATH '../../data/files/flights_tiny.txt.1' OVERWRITE INTO TABLE date_udf_flight; + +CREATE TABLE date_udf_flight_orc ( + fl_date DATE, + fl_time TIMESTAMP +) STORED AS ORC; + +INSERT INTO TABLE date_udf_flight_orc SELECT fl_date, to_utc_timestamp(fl_date, 'America/Los_Angeles') FROM date_udf_flight; + +SELECT * FROM date_udf_flight_orc; + +EXPLAIN SELECT + to_unix_timestamp(fl_time), + year(fl_time), + month(fl_time), + day(fl_time), + dayofmonth(fl_time), + weekofyear(fl_time), + date(fl_time), + to_date(fl_time), + date_add(fl_time, 2), + date_sub(fl_time, 2), + datediff(fl_time, "2000-01-01") +FROM date_udf_flight_orc; + +SELECT + to_unix_timestamp(fl_time), + year(fl_time), + month(fl_time), + day(fl_time), + dayofmonth(fl_time), + weekofyear(fl_time), + date(fl_time), + to_date(fl_time), + date_add(fl_time, 2), + date_sub(fl_time, 2), + datediff(fl_time, "2000-01-01") +FROM date_udf_flight_orc; + +EXPLAIN SELECT + to_unix_timestamp(fl_date), + year(fl_date), + month(fl_date), + day(fl_date), + dayofmonth(fl_date), + weekofyear(fl_date), + date(fl_date), + to_date(fl_date), + date_add(fl_date, 2), + date_sub(fl_date, 2), + datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +SELECT + to_unix_timestamp(fl_date), + year(fl_date), + month(fl_date), + day(fl_date), + dayofmonth(fl_date), + weekofyear(fl_date), + date(fl_date), + to_date(fl_date), + date_add(fl_date, 2), + date_sub(fl_date, 2), + datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +EXPLAIN SELECT + year(fl_time) = year(fl_date), + month(fl_time) = month(fl_date), + day(fl_time) = day(fl_date), + dayofmonth(fl_time) = dayofmonth(fl_date), + weekofyear(fl_time) = weekofyear(fl_date), + date(fl_time) = date(fl_date), + to_date(fl_time) = to_date(fl_date), + date_add(fl_time, 2) = date_add(fl_date, 2), + date_sub(fl_time, 2) = date_sub(fl_date, 2), + datediff(fl_time, "2000-01-01") = datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +-- Should all be true or NULL +SELECT + year(fl_time) = year(fl_date), + month(fl_time) = month(fl_date), + day(fl_time) = day(fl_date), + dayofmonth(fl_time) = dayofmonth(fl_date), + weekofyear(fl_time) = weekofyear(fl_date), + date(fl_time) = date(fl_date), + to_date(fl_time) = to_date(fl_date), + date_add(fl_time, 2) = date_add(fl_date, 2), + date_sub(fl_time, 2) = date_sub(fl_date, 2), + datediff(fl_time, "2000-01-01") = datediff(fl_date, "2000-01-01") +FROM date_udf_flight_orc; + +EXPLAIN SELECT + fl_date, + to_date(date_add(fl_date, 2)), + to_date(date_sub(fl_date, 2)), + datediff(fl_date, date_add(fl_date, 2)), + datediff(fl_date, date_sub(fl_date, 2)), + datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) +FROM date_udf_flight_orc LIMIT 10; + +SELECT + fl_date, + to_date(date_add(fl_date, 2)), + to_date(date_sub(fl_date, 2)), + datediff(fl_date, date_add(fl_date, 2)), + datediff(fl_date, date_sub(fl_date, 2)), + datediff(date_add(fl_date, 2), date_sub(fl_date, 2)) +FROM date_udf_flight_orc LIMIT 10; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q new file mode 100644 index 0000000000000..6e622007e5dcb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_distinct_gby.q @@ -0,0 +1,12 @@ +SET hive.vectorized.execution.enabled=true; + +SET hive.map.groupby.sorted=true; + +create table dtest(a int, b int) clustered by (a) sorted by (a) into 1 buckets stored as orc; +insert into table dtest select c,b from (select array(300,300,300,300,300) as a, 1 as b from src limit 1) y lateral view explode(a) t1 as c; + +explain select sum(distinct a), count(distinct a) from dtest; +select sum(distinct a), count(distinct a) from dtest; + +explain select sum(distinct cint), count(distinct cint), avg(distinct cint), std(distinct cint) from alltypesorc; +select sum(distinct cint), count(distinct cint), avg(distinct cint), std(distinct cint) from alltypesorc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q new file mode 100644 index 0000000000000..f390c2caafeb2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_mapjoin.q @@ -0,0 +1,12 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.auto.convert.join=true; +SET hive.auto.convert.join.nonconditionaltask=true; +SET hive.auto.convert.join.nonconditionaltask.size=1000000000; + +EXPLAIN SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; + +SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q new file mode 100644 index 0000000000000..d6b082467938f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_math_funcs.q @@ -0,0 +1,107 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test math functions in vectorized mode to verify they run correctly end-to-end. + +explain +select + cdouble + ,Round(cdouble, 2) + ,Floor(cdouble) + ,Ceil(cdouble) + ,Rand() + ,Rand(98007) + ,Exp(ln(cdouble)) + ,Ln(cdouble) + ,Ln(cfloat) + ,Log10(cdouble) + -- Use log2 as a representative function to test all input types. + ,Log2(cdouble) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdouble - 15601.0) + ,Log2(cfloat) + ,Log2(cbigint) + ,Log2(cint) + ,Log2(csmallint) + ,Log2(ctinyint) + ,Log(2.0, cdouble) + ,Pow(log2(cdouble), 2.0) + ,Power(log2(cdouble), 2.0) + ,Sqrt(cdouble) + ,Sqrt(cbigint) + ,Bin(cbigint) + ,Hex(cdouble) + ,Conv(cbigint, 10, 16) + ,Abs(cdouble) + ,Abs(ctinyint) + ,Pmod(cint, 3) + ,Sin(cdouble) + ,Asin(cdouble) + ,Cos(cdouble) + ,ACos(cdouble) + ,Atan(cdouble) + ,Degrees(cdouble) + ,Radians(cdouble) + ,Positive(cdouble) + ,Positive(cbigint) + ,Negative(cdouble) + ,Sign(cdouble) + ,Sign(cbigint) + -- Test nesting + ,cos(-sin(log(cdouble)) + 3.14159) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cfloat) >= -1.0; + +select + cdouble + ,Round(cdouble, 2) + ,Floor(cdouble) + ,Ceil(cdouble) + -- Omit rand() from runtime test because it's nondeterministic. + -- ,Rand() + ,Rand(98007) + ,Exp(ln(cdouble)) + ,Ln(cdouble) + ,Ln(cfloat) + ,Log10(cdouble) + -- Use log2 as a representative function to test all input types. + ,Log2(cdouble) + -- Use 15601.0 to test zero handling, as there are no zeroes in the table + ,Log2(cdouble - 15601.0) + ,Log2(cfloat) + ,Log2(cbigint) + ,Log2(cint) + ,Log2(csmallint) + ,Log2(ctinyint) + ,Log(2.0, cdouble) + ,Pow(log2(cdouble), 2.0) + ,Power(log2(cdouble), 2.0) + ,Sqrt(cdouble) + ,Sqrt(cbigint) + ,Bin(cbigint) + ,Hex(cdouble) + ,Conv(cbigint, 10, 16) + ,Abs(cdouble) + ,Abs(ctinyint) + ,Pmod(cint, 3) + ,Sin(cdouble) + ,Asin(cdouble) + ,Cos(cdouble) + ,ACos(cdouble) + ,Atan(cdouble) + ,Degrees(cdouble) + ,Radians(cdouble) + ,Positive(cdouble) + ,Positive(cbigint) + ,Negative(cdouble) + ,Sign(cdouble) + ,Sign(cbigint) + -- Test nesting + ,cos(-sin(log(cdouble)) + 3.14159) +from alltypesorc +-- limit output to a reasonably small number of rows +where cbigint % 500 = 0 +-- test use of a math function in the WHERE clause +and sin(cfloat) >= -1.0; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q new file mode 100644 index 0000000000000..ce4227cf0a700 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q @@ -0,0 +1,8 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.auto.convert.join=true; +SET hive.auto.convert.join.nonconditionaltask=true; +SET hive.auto.convert.join.nonconditionaltask.size=1000000000; + +explain select sum(t1.td) from (select v1.csmallint as tsi, v1.cdouble as td from alltypesorc v1, alltypesorc v2 where v1.ctinyint=v2.ctinyint) t1 join alltypesorc v3 on t1.tsi=v3.csmallint; + +select sum(t1.td) from (select v1.csmallint as tsi, v1.cdouble as td from alltypesorc v1, alltypesorc v2 where v1.ctinyint=v2.ctinyint) t1 join alltypesorc v3 on t1.tsi=v3.csmallint; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q new file mode 100644 index 0000000000000..488d2f38859b3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_rcfile_columnar.q @@ -0,0 +1,18 @@ +--This query must pass even when vectorized reader is not available for +--RC files. The query must fall back to the non-vector mode and run successfully. + +CREATE table columnTable (key STRING, value STRING) +ROW FORMAT SERDE + 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe' +STORED AS + INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' + OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat'; + +FROM src +INSERT OVERWRITE TABLE columnTable SELECT src.key, src.value LIMIT 10; +describe columnTable; + +SET hive.vectorized.execution.enabled=true; + +SELECT key, value FROM columnTable ORDER BY key; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q new file mode 100644 index 0000000000000..6b60aa08c581b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_shufflejoin.q @@ -0,0 +1,10 @@ +SET hive.vectorized.execution.enabled=true; +SET hive.auto.convert.join=false; + +EXPLAIN SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; + +SELECT COUNT(t1.cint), MAX(t2.cint), MIN(t1.cint), AVG(t1.cint+t2.cint) + FROM alltypesorc t1 + JOIN alltypesorc t2 ON t1.cint = t2.cint; \ No newline at end of file diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q new file mode 100644 index 0000000000000..96fe53da1ea35 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_string_funcs.q @@ -0,0 +1,46 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test string functions in vectorized mode to verify end-to-end functionality. + +explain +select + substr(cstring1, 1, 2) + ,substr(cstring1, 2) + ,lower(cstring1) + ,upper(cstring1) + ,ucase(cstring1) + ,length(cstring1) + ,trim(cstring1) + ,ltrim(cstring1) + ,rtrim(cstring1) + ,concat(cstring1, cstring2) + ,concat('>', cstring1) + ,concat(cstring1, '<') + ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2)) +from alltypesorc +-- Limit the number of rows of output to a reasonable amount. +where cbigint % 237 = 0 +-- Test function use in the WHERE clause. +and length(substr(cstring1, 1, 2)) <= 2 +and cstring1 like '%'; + +select + substr(cstring1, 1, 2) + ,substr(cstring1, 2) + ,lower(cstring1) + ,upper(cstring1) + ,ucase(cstring1) + ,length(cstring1) + ,trim(cstring1) + ,ltrim(cstring1) + ,rtrim(cstring1) + ,concat(cstring1, cstring2) + ,concat('>', cstring1) + ,concat(cstring1, '<') + ,concat(substr(cstring1, 1, 2), substr(cstring2, 1, 2)) +from alltypesorc +-- Limit the number of rows of output to a reasonable amount. +where cbigint % 237 = 0 +-- Test function use in the WHERE clause. +and length(substr(cstring1, 1, 2)) <= 2 +and cstring1 like '%'; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q new file mode 100644 index 0000000000000..95eedd3b581d6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/vectorized_timestamp_funcs.q @@ -0,0 +1,124 @@ +SET hive.vectorized.execution.enabled = true; + +-- Test timestamp functions in vectorized mode to verify they run correctly end-to-end. + +CREATE TABLE alltypesorc_string(ctimestamp1 timestamp, stimestamp1 string) STORED AS ORC; + +INSERT OVERWRITE TABLE alltypesorc_string +SELECT + to_utc_timestamp(ctimestamp1, 'America/Los_Angeles'), + CAST(to_utc_timestamp(ctimestamp1, 'America/Los_Angeles') AS STRING) +FROM alltypesorc +LIMIT 40; + +CREATE TABLE alltypesorc_wrong(stimestamp1 string) STORED AS ORC; + +INSERT INTO TABLE alltypesorc_wrong SELECT 'abcd' FROM alltypesorc LIMIT 1; +INSERT INTO TABLE alltypesorc_wrong SELECT '2000:01:01 00-00-00' FROM alltypesorc LIMIT 1; +INSERT INTO TABLE alltypesorc_wrong SELECT '0000-00-00 99:99:99' FROM alltypesorc LIMIT 1; + +EXPLAIN SELECT + to_unix_timestamp(ctimestamp1) AS c1, + year(ctimestamp1), + month(ctimestamp1), + day(ctimestamp1), + dayofmonth(ctimestamp1), + weekofyear(ctimestamp1), + hour(ctimestamp1), + minute(ctimestamp1), + second(ctimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +SELECT + to_unix_timestamp(ctimestamp1) AS c1, + year(ctimestamp1), + month(ctimestamp1), + day(ctimestamp1), + dayofmonth(ctimestamp1), + weekofyear(ctimestamp1), + hour(ctimestamp1), + minute(ctimestamp1), + second(ctimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +EXPLAIN SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +EXPLAIN SELECT + to_unix_timestamp(ctimestamp1) = to_unix_timestamp(stimestamp1) AS c1, + year(ctimestamp1) = year(stimestamp1), + month(ctimestamp1) = month(stimestamp1), + day(ctimestamp1) = day(stimestamp1), + dayofmonth(ctimestamp1) = dayofmonth(stimestamp1), + weekofyear(ctimestamp1) = weekofyear(stimestamp1), + hour(ctimestamp1) = hour(stimestamp1), + minute(ctimestamp1) = minute(stimestamp1), + second(ctimestamp1) = second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +-- Should all be true or NULL +SELECT + to_unix_timestamp(ctimestamp1) = to_unix_timestamp(stimestamp1) AS c1, + year(ctimestamp1) = year(stimestamp1), + month(ctimestamp1) = month(stimestamp1), + day(ctimestamp1) = day(stimestamp1), + dayofmonth(ctimestamp1) = dayofmonth(stimestamp1), + weekofyear(ctimestamp1) = weekofyear(stimestamp1), + hour(ctimestamp1) = hour(stimestamp1), + minute(ctimestamp1) = minute(stimestamp1), + second(ctimestamp1) = second(stimestamp1) +FROM alltypesorc_string +ORDER BY c1; + +-- Wrong format. Should all be NULL. +EXPLAIN SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_wrong +ORDER BY c1; + +SELECT + to_unix_timestamp(stimestamp1) AS c1, + year(stimestamp1), + month(stimestamp1), + day(stimestamp1), + dayofmonth(stimestamp1), + weekofyear(stimestamp1), + hour(stimestamp1), + minute(stimestamp1), + second(stimestamp1) +FROM alltypesorc_wrong +ORDER BY c1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q index 4e3d0572921d6..bc193554f9182 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q @@ -4,13 +4,13 @@ USE db1; CREATE TABLE table1 (key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE table1; CREATE TABLE table2 (key STRING, value STRING) STORED AS TEXTFILE; -LOAD DATA LOCAL INPATH '../data/files/kv1.txt' +LOAD DATA LOCAL INPATH '../../data/files/kv1.txt' OVERWRITE INTO TABLE table2; -- relative reference, no alias diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q index b0b078ec628b3..95517c3bcd367 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q @@ -1,11 +1,11 @@ DROP TABLE IF EXISTS atab; CREATE TABLE IF NOT EXISTS atab (ks_uid BIGINT, sr_uid STRING, sr_id STRING, tstamp STRING, m_id STRING, act STRING, at_sr_uid STRING, tstamp_type STRING, original_m_id STRING, original_tstamp STRING, registered_flag TINYINT, at_ks_uid BIGINT) PARTITIONED BY (dt STRING,nt STRING); -LOAD DATA LOCAL INPATH '../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130312', nt='tw'); -LOAD DATA LOCAL INPATH '../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130311', nt='tw'); +LOAD DATA LOCAL INPATH '../../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130312', nt='tw'); +LOAD DATA LOCAL INPATH '../../data/files/v1.txt' INTO TABLE atab PARTITION (dt='20130311', nt='tw'); DROP TABLE IF EXISTS mstab; CREATE TABLE mstab(ks_uid INT, csc INT) PARTITIONED BY (dt STRING); -LOAD DATA LOCAL INPATH '../data/files/v2.txt' INTO TABLE mstab PARTITION (dt='20130311'); +LOAD DATA LOCAL INPATH '../../data/files/v2.txt' INTO TABLE mstab PARTITION (dt='20130311'); DROP VIEW IF EXISTS aa_view_tw; CREATE VIEW aa_view_tw AS SELECT ks_uid, sr_id, act, at_ks_uid, at_sr_uid, from_unixtime(CAST(CAST( tstamp as BIGINT)/1000 AS BIGINT),'yyyyMMdd') AS act_date, from_unixtime(CAST(CAST( original_tstamp AS BIGINT)/1000 AS BIGINT),'yyyyMMdd') AS content_creation_date FROM atab WHERE dt='20130312' AND nt='tw' AND ks_uid != at_ks_uid; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q index a7297db6104c0..2f22145518c56 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testWindowing select p_mfgr, p_name, p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q index 9c7625dcd786a..67cab9f7b273a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; set hive.join.cache.size=1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q index 7c4ab386145d5..24f9ff73a30a1 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; -- 1. testQueryLevelPartitionColsNotInSelect select p_size, diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q index 2c3339095f22b..7e27c6b1c098d 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q @@ -13,7 +13,7 @@ CREATE TABLE part( p_comment STRING ); -LOAD DATA LOCAL INPATH '../data/files/part_tiny.txt' overwrite into table part; +LOAD DATA LOCAL INPATH '../../data/files/part_tiny.txt' overwrite into table part; drop table over10k; @@ -27,12 +27,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select p_mfgr, p_retailprice, p_size, round(sum(p_retailprice) over w1 , 2) = round(sum(lag(p_retailprice,1,0.0)) over w1 + last_value(p_retailprice) over w1 , 2), diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q index bb371e900975b..1c6e1aac37a1a 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q @@ -10,12 +10,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, rank() over (partition by s order by si), sum(b) over (partition by s order by si) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q index 8a9d0012593e6..05da2ba7efeaf 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q @@ -9,13 +9,13 @@ create table over10k( d double, bo boolean, s string, - ts timestamp, - dec decimal, + ts timestamp, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, row_number() over (partition by d order by dec) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q index 505c259f4b7b3..73e8192ee6e4c 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q @@ -10,12 +10,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select i, ntile(10) over (partition by s order by i) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q index bf76867813e4e..4b951179e09e6 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q @@ -10,12 +10,12 @@ create table over10k( bo boolean, s string, ts timestamp, - dec decimal, + dec decimal(4,2), bin binary) row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, rank() over (partition by f order by t) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q index f22b992cd4386..0173ab7a3ac56 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q @@ -15,7 +15,7 @@ create table over10k( row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, min(i) over (partition by s) from over10k limit 100; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q new file mode 100644 index 0000000000000..b813657baee47 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf2.q @@ -0,0 +1,4 @@ +-- user-added aggregates should be usable as windowing functions +create temporary function mysum as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum'; + +select sum(key) over (), mysum(key) over () from src limit 1; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q index 7cc1367306956..6d8ce670454d0 100644 --- a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q @@ -15,7 +15,7 @@ create table over10k( row format delimited fields terminated by '|'; -load data local inpath '../data/files/over10k' into table over10k; +load data local inpath '../../data/files/over10k' into table over10k; select s, sum(b) over (partition by i order by s,b rows unbounded preceding) from over10k limit 100; diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index aaefe84ce81ea..a90fc023e67d8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -21,11 +21,9 @@ import org.scalatest.BeforeAndAfterAll import scala.reflect.ClassTag - import org.apache.spark.sql.{SQLConf, QueryTest} import org.apache.spark.sql.catalyst.plans.logical.NativeCommand import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin} -import org.apache.spark.sql.hive.HiveShim import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala index 46b11b582b26d..ca78dfba4fa38 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveQLSuite.scala @@ -22,7 +22,7 @@ import scala.util.Try import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.sql.api.java.JavaSchemaRDD +import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD} import org.apache.spark.sql.execution.ExplainCommand import org.apache.spark.sql.hive.test.TestHive @@ -33,9 +33,7 @@ class JavaHiveQLSuite extends FunSuite { lazy val javaCtx = new JavaSparkContext(TestHive.sparkContext) // There is a little trickery here to avoid instantiating two HiveContexts in the same JVM - lazy val javaHiveCtx = new JavaHiveContext(javaCtx) { - override val sqlContext = TestHive - } + lazy val javaHiveCtx = new JavaHiveContext(TestHive) test("SELECT * FROM src") { assert( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 79cc7a3fcc7d6..44eb4cfa59335 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -142,14 +142,25 @@ abstract class HiveComparisonTest // Hack: Hive simply prints the result of a SET command to screen, // and does not return it as a query answer. case _: SetCommand => Seq("0") + case LogicalNativeCommand(c) if c.toLowerCase.contains("desc") => + answer + .filterNot(nonDeterministicLine) + .map(_.replaceAll("from deserializer", "")) + .map(_.replaceAll("None", "")) + .map(_.trim) + .filterNot(_ == "") case _: LogicalNativeCommand => answer.filterNot(nonDeterministicLine).filterNot(_ == "") case _: ExplainCommand => answer case _: DescribeCommand => // Filter out non-deterministic lines and lines which do not have actual results but // can introduce problems because of the way Hive formats these lines. // Then, remove empty lines. Do not sort the results. - answer.filterNot( - r => nonDeterministicLine(r) || ignoredLine(r)).map(_.trim).filterNot(_ == "") + answer + .filterNot(r => nonDeterministicLine(r) || ignoredLine(r)) + .map(_.replaceAll("from deserializer", "")) + .map(_.replaceAll("None", "")) + .map(_.trim) + .filterNot(_ == "") case plan => if (isSorted(plan)) answer else answer.sorted } orderedAnswer.map(cleanPaths) @@ -164,6 +175,7 @@ abstract class HiveComparisonTest "last_modified_by", "last_modified_time", "Owner:", + "COLUMN_STATS_ACCURATE", // The following are hive specific schema parameters which we do not need to match exactly. "numFiles", "numRows", @@ -237,6 +249,7 @@ abstract class HiveComparisonTest // the system to return the wrong answer. Since we have no intention of mirroring their // previously broken behavior we simply filter out changes to this setting. .filterNot(_ contains "hive.outerjoin.supports.filters") + .filterNot(_ contains "hive.exec.post.hooks") if (allQueries != queryList) logWarning(s"Simplifications made on unsupported operations for test $testCaseName") @@ -345,7 +358,7 @@ abstract class HiveComparisonTest (queryList, hiveResults, catalystResults).zipped.foreach { case (query, hive, (hiveQuery, catalyst)) => // Check that the results match unless its an EXPLAIN query. - val preparedHive = prepareAnswer(hiveQuery,hive) + val preparedHive = prepareAnswer(hiveQuery, hive) if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && preparedHive != catalyst) { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 5de20175d9f57..322a25bb20837 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -508,19 +508,19 @@ class HiveQuerySuite extends HiveComparisonTest { // Describe a partition is a native command assertResult( Array( - Array("key", "int", HiveShim.getEmptyCommentsFieldValue), - Array("value", "string", HiveShim.getEmptyCommentsFieldValue), - Array("dt", "string", HiveShim.getEmptyCommentsFieldValue), - Array("", "", ""), - Array("# Partition Information", "", ""), + Array("key", "int"), + Array("value", "string"), + Array("dt", "string"), + Array(""), + Array("# Partition Information"), Array("# col_name", "data_type", "comment"), - Array("", "", ""), - Array("dt", "string", HiveShim.getEmptyCommentsFieldValue)) + Array(""), + Array("dt", "string")) ) { sql("DESCRIBE test_describe_commands1 PARTITION (dt='2008-06-08')") .select('result) .collect() - .map(_.getString(0).split("\t").map(_.trim)) + .map(_.getString(0).replaceAll("None", "").trim.split("\t").map(_.trim)) } // Describe a registered temporary table. diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala deleted file mode 100644 index 6dde636965afd..0000000000000 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.net.URI -import java.util.{ArrayList => JArrayList} -import java.util.Properties -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} -import org.apache.hadoop.hive.ql.processors._ -import org.apache.hadoop.hive.ql.stats.StatsSetupConst -import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} -import org.apache.hadoop.{io => hadoopIo} -import org.apache.hadoop.mapred.InputFormat -import scala.collection.JavaConversions._ -import scala.language.implicitConversions - -/** - * A compatibility layer for interacting with Hive version 0.12.0. - */ -private[hive] object HiveShim { - val version = "0.12.0" - val metastoreDecimal = "decimal" - - def getTableDesc( - serdeClass: Class[_ <: Deserializer], - inputFormatClass: Class[_ <: InputFormat[_, _]], - outputFormatClass: Class[_], - properties: Properties) = { - new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) - } - - def createDriverResultsArray = new JArrayList[String] - - def processResults(results: JArrayList[String]) = results - - def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE - - def createDefaultDBIfNeeded(context: HiveContext) = { } - - /** The string used to denote an empty comments field in the schema. */ - def getEmptyCommentsFieldValue = "None" - - def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { - CommandProcessorFactory.get(cmd(0), conf) - } - - def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { - new HiveDecimal(bd) - } - - def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { - ColumnProjectionUtils.appendReadColumnIDs(conf, ids) - ColumnProjectionUtils.appendReadColumnNames(conf, names) - } - - def getExternalTmpPath(context: Context, uri: URI) = { - context.getExternalTmpFileURI(uri) - } - - def getDataLocationPath(p: Partition) = p.getPartitionPath - - def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) - -} - -class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) - extends FileSinkDesc(dir, tableInfo, compressed) { -} diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala new file mode 100644 index 0000000000000..2317d2e76341f --- /dev/null +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.net.URI +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors._ +import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.{io => hadoopIo} +import org.apache.hadoop.mapred.InputFormat +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[hive] object HiveShim { + val version = "0.12.0" + val metastoreDecimal = "decimal" + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[String] + + def processResults(results: JArrayList[String]) = results + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { } + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd(0), conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + new HiveDecimal(bd) + } + + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + ColumnProjectionUtils.appendReadColumnIDs(conf, ids) + ColumnProjectionUtils.appendReadColumnNames(conf, names) + } + + def getExternalTmpPath(context: Context, uri: URI) = { + context.getExternalTmpFileURI(uri) + } + + def getDataLocationPath(p: Partition) = p.getPartitionPath + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsForPruner(tbl) + + def compatibilityBlackList = Seq( + "decimal_.*", + "drop_partitions_filter2", + "show_.*", + "serde_regex", + "udf_to_date", + "udaf_collect_set", + "udf_concat" + ) + +} + +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends FileSinkDesc(dir, tableInfo, compressed) { +} diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala deleted file mode 100644 index 8678c0c475db4..0000000000000 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim.scala +++ /dev/null @@ -1,170 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive - -import java.util.{ArrayList => JArrayList} -import java.util.Properties -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.common.`type`.{HiveDecimal} -import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} -import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory -import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} -import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.Logging -import org.apache.hadoop.{io => hadoopIo} -import scala.collection.JavaConversions._ -import scala.language.implicitConversions - -/** - * A compatibility layer for interacting with Hive version 0.13.1. - */ -private[hive] object HiveShim { - val version = "0.13.1" - /* - * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded) - * Full support of new decimal feature need to be fixed in seperate PR. - */ - val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r - - def getTableDesc( - serdeClass: Class[_ <: Deserializer], - inputFormatClass: Class[_ <: InputFormat[_, _]], - outputFormatClass: Class[_], - properties: Properties) = { - new TableDesc(inputFormatClass, outputFormatClass, properties) - } - - def createDriverResultsArray = new JArrayList[Object] - - def processResults(results: JArrayList[Object]) = { - results.map { r => - r match { - case s: String => s - case a: Array[Object] => a(0).asInstanceOf[String] - } - } - } - - def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE - - def createDefaultDBIfNeeded(context: HiveContext) = { - context.runSqlHive("CREATE DATABASE default") - context.runSqlHive("USE default") - } - - /* The string used to denote an empty comments field in the schema. */ - def getEmptyCommentsFieldValue = "" - - def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { - CommandProcessorFactory.get(cmd, conf) - } - - def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { - HiveDecimal.create(bd) - } - - /* - * This function in hive-0.13 become private, but we have to do this to walkaround hive bug - */ - private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) { - val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "") - val result: StringBuilder = new StringBuilder(old) - var first: Boolean = old.isEmpty - - for (col <- cols) { - if (first) { - first = false - } else { - result.append(',') - } - result.append(col) - } - conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString) - } - - /* - * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty - */ - def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { - if (ids != null && ids.size > 0) { - ColumnProjectionUtils.appendReadColumns(conf, ids) - } - if (names != null && names.size > 0) { - appendReadColumnNames(conf, names) - } - } - - def getExternalTmpPath(context: Context, path: Path) = { - context.getExternalTmpPath(path.toUri) - } - - def getDataLocationPath(p: Partition) = p.getDataLocation - - def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsOf(tbl) - - /* - * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. - * Fix it through wrapper. - * */ - implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = { - var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed) - f.setCompressCodec(w.compressCodec) - f.setCompressType(w.compressType) - f.setTableInfo(w.tableInfo) - f.setDestTableId(w.destTableId) - f - } -} - -/* - * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. - * Fix it through wrapper. - */ -class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) - extends Serializable with Logging { - var compressCodec: String = _ - var compressType: String = _ - var destTableId: Int = _ - - def setCompressed(compressed: Boolean) { - this.compressed = compressed - } - - def getDirName = dir - - def setDestTableId(destTableId: Int) { - this.destTableId = destTableId - } - - def setTableInfo(tableInfo: TableDesc) { - this.tableInfo = tableInfo - } - - def setCompressCodec(intermediateCompressorCodec: String) { - compressCodec = intermediateCompressorCodec - } - - def setCompressType(intermediateCompressType: String) { - compressType = intermediateCompressType - } -} diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala new file mode 100644 index 0000000000000..b8d893d8c1319 --- /dev/null +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.util.{ArrayList => JArrayList} +import java.util.Properties +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.common.StatsSetupConst +import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} +import org.apache.hadoop.mapred.InputFormat +import org.apache.spark.Logging +import org.apache.hadoop.{io => hadoopIo} +import scala.collection.JavaConversions._ +import scala.language.implicitConversions + +/** + * A compatibility layer for interacting with Hive version 0.13.1. + */ +private[hive] object HiveShim { + val version = "0.13.1" + /* + * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded) + * Full support of new decimal feature need to be fixed in seperate PR. + */ + val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r + + def getTableDesc( + serdeClass: Class[_ <: Deserializer], + inputFormatClass: Class[_ <: InputFormat[_, _]], + outputFormatClass: Class[_], + properties: Properties) = { + new TableDesc(inputFormatClass, outputFormatClass, properties) + } + + def createDriverResultsArray = new JArrayList[Object] + + def processResults(results: JArrayList[Object]) = { + results.map { r => + r match { + case s: String => s + case a: Array[Object] => a(0).asInstanceOf[String] + } + } + } + + def getStatsSetupConstTotalSize = StatsSetupConst.TOTAL_SIZE + + def createDefaultDBIfNeeded(context: HiveContext) = { + context.runSqlHive("CREATE DATABASE default") + context.runSqlHive("USE default") + } + + def getCommandProcessor(cmd: Array[String], conf: HiveConf) = { + CommandProcessorFactory.get(cmd, conf) + } + + def createDecimal(bd: java.math.BigDecimal): HiveDecimal = { + HiveDecimal.create(bd) + } + + /* + * This function in hive-0.13 become private, but we have to do this to walkaround hive bug + */ + private def appendReadColumnNames(conf: Configuration, cols: Seq[String]) { + val old: String = conf.get(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, "") + val result: StringBuilder = new StringBuilder(old) + var first: Boolean = old.isEmpty + + for (col <- cols) { + if (first) { + first = false + } else { + result.append(',') + } + result.append(col) + } + conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, result.toString) + } + + /* + * Cannot use ColumnProjectionUtils.appendReadColumns directly, if ids is null or empty + */ + def appendReadColumns(conf: Configuration, ids: Seq[Integer], names: Seq[String]) { + if (ids != null && ids.size > 0) { + ColumnProjectionUtils.appendReadColumns(conf, ids) + } + if (names != null && names.size > 0) { + appendReadColumnNames(conf, names) + } + } + + def getExternalTmpPath(context: Context, path: Path) = { + context.getExternalTmpPath(path.toUri) + } + + def getDataLocationPath(p: Partition) = p.getDataLocation + + def getAllPartitionsOf(client: Hive, tbl: Table) = client.getAllPartitionsOf(tbl) + + def compatibilityBlackList = Seq() + + /* + * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. + * Fix it through wrapper. + * */ + implicit def wrapperToFileSinkDesc(w: ShimFileSinkDesc): FileSinkDesc = { + var f = new FileSinkDesc(new Path(w.dir), w.tableInfo, w.compressed) + f.setCompressCodec(w.compressCodec) + f.setCompressType(w.compressType) + f.setTableInfo(w.tableInfo) + f.setDestTableId(w.destTableId) + f + } +} + +/* + * Bug introdiced in hive-0.13. FileSinkDesc is serilizable, but its member path is not. + * Fix it through wrapper. + */ +class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) + extends Serializable with Logging { + var compressCodec: String = _ + var compressType: String = _ + var destTableId: Int = _ + + def setCompressed(compressed: Boolean) { + this.compressed = compressed + } + + def getDirName = dir + + def setDestTableId(destTableId: Int) { + this.destTableId = destTableId + } + + def setTableInfo(tableInfo: TableDesc) { + this.tableInfo = tableInfo + } + + def setCompressCodec(intermediateCompressorCodec: String) { + compressCodec = intermediateCompressorCodec + } + + def setCompressType(intermediateCompressType: String) { + compressType = intermediateCompressType + } +} From 9530316887612dca060a128fca34dd5a6ab2a9a9 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 25 Oct 2014 00:06:57 -0700 Subject: [PATCH 1072/1492] [SPARK-2321] Stable pull-based progress / status API This pull request is a first step towards the implementation of a stable, pull-based progress / status API for Spark (see [SPARK-2321](https://issues.apache.org/jira/browse/SPARK-2321)). For now, I'd like to discuss the basic implementation, API names, and overall interface design. Once we arrive at a good design, I'll go back and add additional methods to expose more information via these API. #### Design goals: - Pull-based API - Usable from Java / Scala / Python (eventually, likely with a wrapper) - Can be extended to expose more information without introducing binary incompatibilities. - Returns immutable objects. - Don't leak any implementation details, preserving our freedom to change the implementation. #### Implementation: - Add public methods (`getJobInfo`, `getStageInfo`) to SparkContext to allow status / progress information to be retrieved. - Add public interfaces (`SparkJobInfo`, `SparkStageInfo`) for our API return values. These interfaces consist entirely of Java-style getter methods. The interfaces are currently implemented in Java. I decided to explicitly separate the interface from its implementation (`SparkJobInfoImpl`, `SparkStageInfoImpl`) in order to prevent users from constructing these responses themselves. -Allow an existing JobProgressListener to be used when constructing a live SparkUI. This allows us to re-use this listeners in the implementation of this status API. There are a few reasons why this listener re-use makes sense: - The status API and web UI are guaranteed to show consistent information. - These listeners are already well-tested. - The same garbage-collection / information retention configurations can apply to both this API and the web UI. - Extend JobProgressListener to maintain `jobId -> Job` and `stageId -> Stage` mappings. The progress API methods are implemented in a separate trait that's mixed into SparkContext. This helps to avoid SparkContext.scala from becoming larger and more difficult to read. Author: Josh Rosen Author: Josh Rosen Closes #2696 from JoshRosen/progress-reporting-api and squashes the following commits: e6aa78d [Josh Rosen] Add tests. b585c16 [Josh Rosen] Accept SparkListenerBus instead of more specific subclasses. c96402d [Josh Rosen] Address review comments. 2707f98 [Josh Rosen] Expose current stage attempt id c28ba76 [Josh Rosen] Update demo code: 646ff1d [Josh Rosen] Document spark.ui.retainedJobs. 7f47d6d [Josh Rosen] Clean up SparkUI constructors, per Andrew's feedback. b77b3d8 [Josh Rosen] Merge remote-tracking branch 'origin/master' into progress-reporting-api 787444c [Josh Rosen] Move status API methods into trait that can be mixed into SparkContext. f9a9a00 [Josh Rosen] More review comments: 3dc79af [Josh Rosen] Remove creation of unused listeners in SparkContext. 249ca16 [Josh Rosen] Address several review comments: da5648e [Josh Rosen] Add example of basic progress reporting in Java. 7319ffd [Josh Rosen] Add getJobIdsForGroup() and num*Tasks() methods. cc568e5 [Josh Rosen] Add note explaining that interfaces should not be implemented outside of Spark. 6e840d4 [Josh Rosen] Remove getter-style names and "consistent snapshot" semantics: 08cbec9 [Josh Rosen] Begin to sketch the interfaces for a stable, public status API. ac2d13a [Josh Rosen] Add jobId->stage, stageId->stage mappings in JobProgressListener 24de263 [Josh Rosen] Create UI listeners in SparkContext instead of in Tabs: --- .../org/apache/spark/JobExecutionStatus.java | 25 +++ .../java/org/apache/spark/SparkJobInfo.java | 30 ++++ .../java/org/apache/spark/SparkStageInfo.java | 34 +++++ .../scala/org/apache/spark/SparkContext.scala | 76 ++-------- .../org/apache/spark/SparkStatusAPI.scala | 142 ++++++++++++++++++ .../org/apache/spark/StatusAPIImpl.scala | 34 +++++ .../spark/api/java/JavaSparkContext.scala | 19 +++ .../deploy/history/FsHistoryProvider.scala | 2 +- .../apache/spark/deploy/master/Master.scala | 4 +- .../scala/org/apache/spark/ui/SparkUI.scala | 108 ++++++++----- .../apache/spark/ui/env/EnvironmentTab.scala | 4 +- .../apache/spark/ui/exec/ExecutorsTab.scala | 3 +- .../spark/ui/jobs/JobProgressListener.scala | 49 +++++- .../spark/ui/jobs/JobProgressPage.scala | 9 +- .../apache/spark/ui/jobs/JobProgressTab.scala | 10 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/UIData.scala | 8 + .../apache/spark/ui/storage/StorageTab.scala | 3 +- .../org/apache/spark/StatusAPISuite.scala | 78 ++++++++++ docs/configuration.md | 11 +- .../spark/examples/JavaStatusAPIDemo.java | 70 +++++++++ 21 files changed, 588 insertions(+), 134 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/JobExecutionStatus.java create mode 100644 core/src/main/java/org/apache/spark/SparkJobInfo.java create mode 100644 core/src/main/java/org/apache/spark/SparkStageInfo.java create mode 100644 core/src/main/scala/org/apache/spark/SparkStatusAPI.scala create mode 100644 core/src/main/scala/org/apache/spark/StatusAPIImpl.scala create mode 100644 core/src/test/scala/org/apache/spark/StatusAPISuite.scala create mode 100644 examples/src/main/java/org/apache/spark/examples/JavaStatusAPIDemo.java diff --git a/core/src/main/java/org/apache/spark/JobExecutionStatus.java b/core/src/main/java/org/apache/spark/JobExecutionStatus.java new file mode 100644 index 0000000000000..6e161313702bb --- /dev/null +++ b/core/src/main/java/org/apache/spark/JobExecutionStatus.java @@ -0,0 +1,25 @@ +/* + * 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; + +public enum JobExecutionStatus { + RUNNING, + SUCCEEDED, + FAILED, + UNKNOWN +} diff --git a/core/src/main/java/org/apache/spark/SparkJobInfo.java b/core/src/main/java/org/apache/spark/SparkJobInfo.java new file mode 100644 index 0000000000000..4e3c983b1170a --- /dev/null +++ b/core/src/main/java/org/apache/spark/SparkJobInfo.java @@ -0,0 +1,30 @@ +/* + * 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; + +/** + * Exposes information about Spark Jobs. + * + * This interface is not designed to be implemented outside of Spark. We may add additional methods + * which may break binary compatibility with outside implementations. + */ +public interface SparkJobInfo { + int jobId(); + int[] stageIds(); + JobExecutionStatus status(); +} diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java new file mode 100644 index 0000000000000..04e2247210ecc --- /dev/null +++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java @@ -0,0 +1,34 @@ +/* + * 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; + +/** + * Exposes information about Spark Stages. + * + * This interface is not designed to be implemented outside of Spark. We may add additional methods + * which may break binary compatibility with outside implementations. + */ +public interface SparkStageInfo { + int stageId(); + int currentAttemptId(); + String name(); + int numTasks(); + int numActiveTasks(); + int numCompletedTasks(); + int numFailedTasks(); +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4565832334420..e8fdfff04390d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -26,7 +26,6 @@ import java.util.concurrent.atomic.AtomicInteger import java.util.{Properties, UUID} import java.util.UUID.randomUUID import scala.collection.{Map, Set} -import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} @@ -51,6 +50,7 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.jobs.JobProgressListener import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** @@ -61,7 +61,7 @@ import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, Metadat * this config overrides the default configs as well as system properties. */ -class SparkContext(config: SparkConf) extends Logging { +class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It @@ -224,10 +224,15 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - // Initialize the Spark UI, registering all associated listeners + + private[spark] val jobProgressListener = new JobProgressListener(conf) + listenerBus.addListener(jobProgressListener) + + // Initialize the Spark UI private[spark] val ui: Option[SparkUI] = if (conf.getBoolean("spark.ui.enabled", true)) { - Some(new SparkUI(this)) + Some(SparkUI.createLiveUI(this, conf, listenerBus, jobProgressListener, + env.securityManager,appName)) } else { // For tests, do not enable the UI None @@ -854,69 +859,6 @@ class SparkContext(config: SparkConf) extends Logging { /** The version of Spark on which this application is running. */ def version = SPARK_VERSION - /** - * Return a map from the slave to the max memory available for caching and the remaining - * memory available for caching. - */ - def getExecutorMemoryStatus: Map[String, (Long, Long)] = { - env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) => - (blockManagerId.host + ":" + blockManagerId.port, mem) - } - } - - /** - * :: DeveloperApi :: - * Return information about what RDDs are cached, if they are in mem or on disk, how much space - * they take, etc. - */ - @DeveloperApi - def getRDDStorageInfo: Array[RDDInfo] = { - val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray - StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) - rddInfos.filter(_.isCached) - } - - /** - * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call. - * Note that this does not necessarily mean the caching or computation was successful. - */ - def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - - /** - * :: DeveloperApi :: - * Return information about blocks stored in all of the slaves - */ - @DeveloperApi - def getExecutorStorageStatus: Array[StorageStatus] = { - env.blockManager.master.getStorageStatus - } - - /** - * :: DeveloperApi :: - * Return pools for fair scheduler - */ - @DeveloperApi - def getAllPools: Seq[Schedulable] = { - // TODO(xiajunluan): We should take nested pools into account - taskScheduler.rootPool.schedulableQueue.toSeq - } - - /** - * :: DeveloperApi :: - * Return the pool associated with the given name, if one exists - */ - @DeveloperApi - def getPoolForName(pool: String): Option[Schedulable] = { - Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)) - } - - /** - * Return current scheduling mode - */ - def getSchedulingMode: SchedulingMode.SchedulingMode = { - taskScheduler.schedulingMode - } - /** * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. diff --git a/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala b/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala new file mode 100644 index 0000000000000..1982499c5e1d3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkStatusAPI.scala @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import scala.collection.Map +import scala.collection.JavaConversions._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.{SchedulingMode, Schedulable} +import org.apache.spark.storage.{StorageStatus, StorageUtils, RDDInfo} + +/** + * Trait that implements Spark's status APIs. This trait is designed to be mixed into + * SparkContext; it allows the status API code to live in its own file. + */ +private[spark] trait SparkStatusAPI { this: SparkContext => + + /** + * Return a map from the slave to the max memory available for caching and the remaining + * memory available for caching. + */ + def getExecutorMemoryStatus: Map[String, (Long, Long)] = { + env.blockManager.master.getMemoryStatus.map { case(blockManagerId, mem) => + (blockManagerId.host + ":" + blockManagerId.port, mem) + } + } + + /** + * :: DeveloperApi :: + * Return information about what RDDs are cached, if they are in mem or on disk, how much space + * they take, etc. + */ + @DeveloperApi + def getRDDStorageInfo: Array[RDDInfo] = { + val rddInfos = persistentRdds.values.map(RDDInfo.fromRdd).toArray + StorageUtils.updateRddInfo(rddInfos, getExecutorStorageStatus) + rddInfos.filter(_.isCached) + } + + /** + * Returns an immutable map of RDDs that have marked themselves as persistent via cache() call. + * Note that this does not necessarily mean the caching or computation was successful. + */ + def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap + + /** + * :: DeveloperApi :: + * Return information about blocks stored in all of the slaves + */ + @DeveloperApi + def getExecutorStorageStatus: Array[StorageStatus] = { + env.blockManager.master.getStorageStatus + } + + /** + * :: DeveloperApi :: + * Return pools for fair scheduler + */ + @DeveloperApi + def getAllPools: Seq[Schedulable] = { + // TODO(xiajunluan): We should take nested pools into account + taskScheduler.rootPool.schedulableQueue.toSeq + } + + /** + * :: DeveloperApi :: + * Return the pool associated with the given name, if one exists + */ + @DeveloperApi + def getPoolForName(pool: String): Option[Schedulable] = { + Option(taskScheduler.rootPool.schedulableNameToSchedulable.get(pool)) + } + + /** + * Return current scheduling mode + */ + def getSchedulingMode: SchedulingMode.SchedulingMode = { + taskScheduler.schedulingMode + } + + + /** + * Return a list of all known jobs in a particular job group. The returned list may contain + * running, failed, and completed jobs, and may vary across invocations of this method. This + * method does not guarantee the order of the elements in its result. + */ + def getJobIdsForGroup(jobGroup: String): Array[Int] = { + jobProgressListener.synchronized { + val jobData = jobProgressListener.jobIdToData.valuesIterator + jobData.filter(_.jobGroup.exists(_ == jobGroup)).map(_.jobId).toArray + } + } + + /** + * Returns job information, or `None` if the job info could not be found or was garbage collected. + */ + def getJobInfo(jobId: Int): Option[SparkJobInfo] = { + jobProgressListener.synchronized { + jobProgressListener.jobIdToData.get(jobId).map { data => + new SparkJobInfoImpl(jobId, data.stageIds.toArray, data.status) + } + } + } + + /** + * Returns stage information, or `None` if the stage info could not be found or was + * garbage collected. + */ + def getStageInfo(stageId: Int): Option[SparkStageInfo] = { + jobProgressListener.synchronized { + for ( + info <- jobProgressListener.stageIdToInfo.get(stageId); + data <- jobProgressListener.stageIdToData.get((stageId, info.attemptId)) + ) yield { + new SparkStageInfoImpl( + stageId, + info.attemptId, + info.name, + info.numTasks, + data.numActiveTasks, + data.numCompleteTasks, + data.numFailedTasks) + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala new file mode 100644 index 0000000000000..90b47c847fbca --- /dev/null +++ b/core/src/main/scala/org/apache/spark/StatusAPIImpl.scala @@ -0,0 +1,34 @@ +/* + * 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 + +private class SparkJobInfoImpl ( + val jobId: Int, + val stageIds: Array[Int], + val status: JobExecutionStatus) + extends SparkJobInfo + +private class SparkStageInfoImpl( + val stageId: Int, + val currentAttemptId: Int, + val name: String, + val numTasks: Int, + val numActiveTasks: Int, + val numCompletedTasks: Int, + val numFailedTasks: Int) + extends SparkStageInfo diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 791d853a015a1..45168ba62d3c1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -132,6 +132,25 @@ class JavaSparkContext(val sc: SparkContext) /** Default min number of partitions for Hadoop RDDs when not given by user */ def defaultMinPartitions: java.lang.Integer = sc.defaultMinPartitions + + /** + * Return a list of all known jobs in a particular job group. The returned list may contain + * running, failed, and completed jobs, and may vary across invocations of this method. This + * method does not guarantee the order of the elements in its result. + */ + def getJobIdsForGroup(jobGroup: String): Array[Int] = sc.getJobIdsForGroup(jobGroup) + + /** + * Returns job information, or `null` if the job info could not be found or was garbage collected. + */ + def getJobInfo(jobId: Int): SparkJobInfo = sc.getJobInfo(jobId).orNull + + /** + * Returns stage information, or `null` if the stage info could not be found or was + * garbage collected. + */ + def getStageInfo(stageId: Int): SparkStageInfo = sc.getStageInfo(stageId).orNull + /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 481f6c93c6a8d..2d1609b973607 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -112,7 +112,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis val ui = { val conf = this.conf.clone() val appSecManager = new SecurityManager(conf) - new SparkUI(conf, appSecManager, replayBus, appId, + SparkUI.createHistoryUI(conf, replayBus, appSecManager, appId, s"${HistoryServer.UI_PATH_PREFIX}/$appId") // Do not call ui.bind() to avoid creating a new server for each application } 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 3b6bb9fe128a4..2f81d472d7b78 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 @@ -721,8 +721,8 @@ private[spark] class Master( try { val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI(new SparkConf, replayBus, appName + " (completed)", - HistoryServer.UI_PATH_PREFIX + s"/${app.id}") + val ui = SparkUI.createHistoryUI(new SparkConf, replayBus, new SecurityManager(conf), + appName + " (completed)", HistoryServer.UI_PATH_PREFIX + s"/${app.id}") replayBus.replay() appIdToUI(app.id) = ui webUi.attachSparkUI(ui) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index cccd59d122a92..049938f827291 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -21,47 +21,30 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.env.EnvironmentTab -import org.apache.spark.ui.exec.ExecutorsTab -import org.apache.spark.ui.jobs.JobProgressTab -import org.apache.spark.ui.storage.StorageTab +import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} +import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} +import org.apache.spark.ui.jobs.{JobProgressListener, JobProgressTab} +import org.apache.spark.ui.storage.{StorageListener, StorageTab} /** * Top level user interface for a Spark application. */ -private[spark] class SparkUI( - val sc: SparkContext, +private[spark] class SparkUI private ( + val sc: Option[SparkContext], val conf: SparkConf, val securityManager: SecurityManager, - val listenerBus: SparkListenerBus, + val environmentListener: EnvironmentListener, + val storageStatusListener: StorageStatusListener, + val executorsListener: ExecutorsListener, + val jobProgressListener: JobProgressListener, + val storageListener: StorageListener, var appName: String, - val basePath: String = "") + val basePath: String) extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") with Logging { - def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) - def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) - - def this( - conf: SparkConf, - securityManager: SecurityManager, - listenerBus: SparkListenerBus, - appName: String, - basePath: String) = - this(null, conf, securityManager, listenerBus, appName, basePath) - - // If SparkContext is not provided, assume the associated application is not live - val live = sc != null - - // Maintain executor storage status through Spark events - val storageStatusListener = new StorageStatusListener - - initialize() - /** Initialize all components of the server. */ def initialize() { - listenerBus.addListener(storageStatusListener) val jobProgressTab = new JobProgressTab(this) attachTab(jobProgressTab) attachTab(new StorageTab(this)) @@ -71,10 +54,10 @@ private[spark] class SparkUI( attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) attachHandler( createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) - if (live) { - sc.env.metricsSystem.getServletHandlers.foreach(attachHandler) - } + // If the UI is live, then serve + sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } + initialize() def getAppName = appName @@ -83,11 +66,6 @@ private[spark] class SparkUI( appName = name } - /** Register the given listener with the listener bus. */ - def registerListener(listener: SparkListener) { - listenerBus.addListener(listener) - } - /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() @@ -116,4 +94,60 @@ private[spark] object SparkUI { def getUIPort(conf: SparkConf): Int = { conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) } + + def createLiveUI( + sc: SparkContext, + conf: SparkConf, + listenerBus: SparkListenerBus, + jobProgressListener: JobProgressListener, + securityManager: SecurityManager, + appName: String): SparkUI = { + create(Some(sc), conf, listenerBus, securityManager, appName, + jobProgressListener = Some(jobProgressListener)) + } + + def createHistoryUI( + conf: SparkConf, + listenerBus: SparkListenerBus, + securityManager: SecurityManager, + appName: String, + basePath: String): SparkUI = { + create(None, conf, listenerBus, securityManager, appName, basePath) + } + + /** + * Create a new Spark UI. + * + * @param sc optional SparkContext; this can be None when reconstituting a UI from event logs. + * @param jobProgressListener if supplied, this JobProgressListener will be used; otherwise, the + * web UI will create and register its own JobProgressListener. + */ + private def create( + sc: Option[SparkContext], + conf: SparkConf, + listenerBus: SparkListenerBus, + securityManager: SecurityManager, + appName: String, + basePath: String = "", + jobProgressListener: Option[JobProgressListener] = None): SparkUI = { + + val _jobProgressListener: JobProgressListener = jobProgressListener.getOrElse { + val listener = new JobProgressListener(conf) + listenerBus.addListener(listener) + listener + } + + val environmentListener = new EnvironmentListener + val storageStatusListener = new StorageStatusListener + val executorsListener = new ExecutorsListener(storageStatusListener) + val storageListener = new StorageListener(storageStatusListener) + + listenerBus.addListener(environmentListener) + listenerBus.addListener(storageStatusListener) + listenerBus.addListener(executorsListener) + listenerBus.addListener(storageListener) + + new SparkUI(sc, conf, securityManager, environmentListener, storageStatusListener, + executorsListener, _jobProgressListener, storageListener, appName, basePath) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala index 0d158fbe638d3..f62260c6f6e1d 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -22,10 +22,8 @@ import org.apache.spark.scheduler._ import org.apache.spark.ui._ private[ui] class EnvironmentTab(parent: SparkUI) extends SparkUITab(parent, "environment") { - val listener = new EnvironmentListener - + val listener = parent.environmentListener attachPage(new EnvironmentPage(this)) - parent.registerListener(listener) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 61eb111cd9100..689cf02b25b70 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -26,10 +26,9 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { - val listener = new ExecutorsListener(parent.storageStatusListener) + val listener = parent.executorsListener attachPage(new ExecutorsPage(this)) - parent.registerListener(listener) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index eaeb861f59e5a..b5207360510dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -40,17 +40,25 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { import JobProgressListener._ + type JobId = Int + type StageId = Int + type StageAttemptId = Int + // How many stages to remember val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) + // How many jobs to remember + val retailedJobs = conf.getInt("spark.ui.retainedJobs", DEFAULT_RETAINED_JOBS) - // Map from stageId to StageInfo - val activeStages = new HashMap[Int, StageInfo] - - // Map from (stageId, attemptId) to StageUIData - val stageIdToData = new HashMap[(Int, Int), StageUIData] + val activeJobs = new HashMap[JobId, JobUIData] + val completedJobs = ListBuffer[JobUIData]() + val failedJobs = ListBuffer[JobUIData]() + val jobIdToData = new HashMap[JobId, JobUIData] + val activeStages = new HashMap[StageId, StageInfo] val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() + val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] + val stageIdToInfo = new HashMap[StageId, StageInfo] // Map from pool name to a hash map (map from stage id to StageInfo). val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() @@ -61,8 +69,32 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { def blockManagerIds = executorIdToBlockManagerId.values.toSeq + override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { + val jobGroup = Option(jobStart.properties).map(_.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + val jobData: JobUIData = + new JobUIData(jobStart.jobId, jobStart.stageIds, jobGroup, JobExecutionStatus.RUNNING) + jobIdToData(jobStart.jobId) = jobData + activeJobs(jobStart.jobId) = jobData + } + + override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { + val jobData = activeJobs.remove(jobEnd.jobId).getOrElse { + logWarning(s"Job completed for unknown job ${jobEnd.jobId}") + new JobUIData(jobId = jobEnd.jobId) + } + jobEnd.jobResult match { + case JobSucceeded => + completedJobs += jobData + jobData.status = JobExecutionStatus.SUCCEEDED + case JobFailed(exception) => + failedJobs += jobData + jobData.status = JobExecutionStatus.FAILED + } + } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { val stage = stageCompleted.stageInfo + stageIdToInfo(stage.stageId) = stage val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), { logWarning("Stage completed for unknown stage " + stage.stageId) new StageUIData @@ -89,7 +121,10 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) - stages.take(toRemove).foreach { s => stageIdToData.remove((s.stageId, s.attemptId)) } + stages.take(toRemove).foreach { s => + stageIdToData.remove((s.stageId, s.attemptId)) + stageIdToInfo.remove(s.stageId) + } stages.trimStart(toRemove) } } @@ -103,6 +138,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) }.getOrElse(DEFAULT_POOL_NAME) + stageIdToInfo(stage.stageId) = stage val stageData = stageIdToData.getOrElseUpdate((stage.stageId, stage.attemptId), new StageUIData) stageData.schedulingPool = poolName @@ -277,4 +313,5 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { private object JobProgressListener { val DEFAULT_POOL_NAME = "default" val DEFAULT_RETAINED_STAGES = 1000 + val DEFAULT_RETAINED_JOBS = 1000 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 1e02f1225d344..6e718eecdd52a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -26,7 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { - private val live = parent.live private val sc = parent.sc private val listener = parent.listener private def isFairScheduler = parent.isFairScheduler @@ -47,17 +46,17 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs - val pools = if (live) sc.getAllPools else Seq[Schedulable]() + val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) val poolTable = new PoolTable(pools, parent) val summary: NodeSeq =
      - {if (live) { + {if (sc.isDefined) { // Total duration is not meaningful unless the UI is live
    • Total Duration: - {UIUtils.formatDuration(now - sc.startTime)} + {UIUtils.formatDuration(now - sc.get.startTime)}
    • }}
    • @@ -80,7 +79,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")
    val content = summary ++ - {if (live && isFairScheduler) { + {if (sc.isDefined && isFairScheduler) {

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq } else { Seq[Node]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index c16542c9db30f..03ca918e2e8b3 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -25,16 +25,14 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") { - val live = parent.live val sc = parent.sc - val conf = if (live) sc.conf else new SparkConf - val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) - val listener = new JobProgressListener(conf) + val conf = sc.map(_.conf).getOrElse(new SparkConf) + val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val listener = parent.jobProgressListener attachPage(new JobProgressPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) - parent.registerListener(listener) def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) @@ -43,7 +41,7 @@ private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "st val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { - sc.cancelStage(stageId) + sc.get.cancelStage(stageId) } // Do a quick pause here to give Spark time to kill the stage so it shows up as // killed after the refresh. Note that this will block the serving thread so the diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 7a6c7d1a497ed..770d99eea1c9d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -26,7 +26,6 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { - private val live = parent.live private val sc = parent.sc private val listener = parent.listener @@ -42,7 +41,7 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs - val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() + val pools = sc.map(_.getPoolForName(poolName).get).toSeq val poolTable = new PoolTable(pools, parent) val content = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index a336bf7e1ed02..e2813f8eb5ab9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -17,6 +17,7 @@ package org.apache.spark.ui.jobs +import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} import org.apache.spark.util.collection.OpenHashSet @@ -36,6 +37,13 @@ private[jobs] object UIData { var diskBytesSpilled : Long = 0 } + class JobUIData( + var jobId: Int = -1, + var stageIds: Seq[Int] = Seq.empty, + var jobGroup: Option[String] = None, + var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN + ) + class StageUIData { var numActiveTasks: Int = _ var numCompleteTasks: Int = _ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 76097f1c51f8e..a81291d505583 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -26,11 +26,10 @@ import org.apache.spark.storage._ /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class StorageTab(parent: SparkUI) extends SparkUITab(parent, "storage") { - val listener = new StorageListener(parent.storageStatusListener) + val listener = parent.storageListener attachPage(new StoragePage(this)) attachPage(new RDDPage(this)) - parent.registerListener(listener) } /** diff --git a/core/src/test/scala/org/apache/spark/StatusAPISuite.scala b/core/src/test/scala/org/apache/spark/StatusAPISuite.scala new file mode 100644 index 0000000000000..4468fba8c1dff --- /dev/null +++ b/core/src/test/scala/org/apache/spark/StatusAPISuite.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import scala.concurrent.duration._ +import scala.language.implicitConversions +import scala.language.postfixOps + +import org.scalatest.{Matchers, FunSuite} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark.JobExecutionStatus._ +import org.apache.spark.SparkContext._ + +class StatusAPISuite extends FunSuite with Matchers with SharedSparkContext { + + test("basic status API usage") { + val jobFuture = sc.parallelize(1 to 10000, 2).map(identity).groupBy(identity).collectAsync() + val jobId: Int = eventually(timeout(10 seconds)) { + val jobIds = jobFuture.jobIds + jobIds.size should be(1) + jobIds.head + } + val jobInfo = eventually(timeout(10 seconds)) { + sc.getJobInfo(jobId).get + } + jobInfo.status() should not be FAILED + val stageIds = jobInfo.stageIds() + stageIds.size should be(2) + + val firstStageInfo = eventually(timeout(10 seconds)) { + sc.getStageInfo(stageIds(0)).get + } + firstStageInfo.stageId() should be(stageIds(0)) + firstStageInfo.currentAttemptId() should be(0) + firstStageInfo.numTasks() should be(2) + eventually(timeout(10 seconds)) { + val updatedFirstStageInfo = sc.getStageInfo(stageIds(0)).get + updatedFirstStageInfo.numCompletedTasks() should be(2) + updatedFirstStageInfo.numActiveTasks() should be(0) + updatedFirstStageInfo.numFailedTasks() should be(0) + } + } + + test("getJobIdsForGroup()") { + sc.setJobGroup("my-job-group", "description") + sc.getJobIdsForGroup("my-job-group") should be (Seq.empty) + val firstJobFuture = sc.parallelize(1 to 1000).countAsync() + val firstJobId = eventually(timeout(10 seconds)) { + firstJobFuture.jobIds.head + } + eventually(timeout(10 seconds)) { + sc.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId)) + } + val secondJobFuture = sc.parallelize(1 to 1000).countAsync() + val secondJobId = eventually(timeout(10 seconds)) { + secondJobFuture.jobIds.head + } + eventually(timeout(10 seconds)) { + sc.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId)) + } + } +} \ No newline at end of file diff --git a/docs/configuration.md b/docs/configuration.md index 66738d3ca754e..3007706a2586e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -375,7 +375,16 @@ Apart from these, the following properties are also available, and may be useful
    spark.ui.retainedStages 1000 - How many stages the Spark UI remembers before garbage collecting. + How many stages the Spark UI and status APIs remember before garbage + collecting. +
    spark.ui.retainedJobs1000 + How many stages the Spark UI and status APIs remember before garbage + collecting.
    +
    {headerRow} {data.map(r => generateDataRow(r))} diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 5d88ca403a674..9be65a4a39a09 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -82,7 +82,7 @@ private[spark] abstract class WebUI( } /** Detach a handler from this UI. */ - def detachHandler(handler: ServletContextHandler) { + protected def detachHandler(handler: ServletContextHandler) { handlers -= handler serverInfo.foreach { info => info.rootHandler.removeHandler(handler) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 8a0075ae8daf7..12d23a92878cf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -39,7 +39,8 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { // Worker table val workers = storageStatusList.map((rddId, _)) - val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers, + id = Some("rdd-storage-by-worker-table")) // Block table val blockLocations = StorageUtils.getRddBlockLocations(rddId, storageStatusList) @@ -49,7 +50,8 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { .map { case (blockId, status) => (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) } - val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) + val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks, + id = Some("rdd-storage-by-block-table")) val content =
    diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 83489ca0679ee..6ced6052d2b18 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -31,7 +31,7 @@ private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList - val content = UIUtils.listingTable(rddHeader, rddRow, rdds) + val content = UIUtils.listingTable(rddHeader, rddRow, rdds, id = Some("storage-by-rdd-table")) UIUtils.headerSparkPage("Storage", content, parent) } diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala new file mode 100644 index 0000000000000..bacf6a16fc233 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -0,0 +1,112 @@ +/* + * 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.ui + +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.{SparkException, SparkConf, SparkContext} +import org.openqa.selenium.WebDriver +import org.openqa.selenium.htmlunit.HtmlUnitDriver +import org.scalatest._ +import org.scalatest.concurrent.Eventually._ +import org.scalatest.selenium.WebBrowser +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.LocalSparkContext._ + +/** + * Selenium tests for the Spark Web UI. These tests are not run by default + * because they're slow. + */ +@DoNotDiscover +class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { + implicit val webDriver: WebDriver = new HtmlUnitDriver + + /** + * Create a test SparkContext with the SparkUI enabled. + * It is safe to `get` the SparkUI directly from the SparkContext returned here. + */ + private def newSparkContext(): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + val sc = new SparkContext(conf) + assert(sc.ui.isDefined) + sc + } + + test("effects of unpersist() / persist() should be reflected") { + // Regression test for SPARK-2527 + withSpark(newSparkContext()) { sc => + val ui = sc.ui.get + val rdd = sc.parallelize(Seq(1, 2, 3)) + rdd.persist(StorageLevels.DISK_ONLY).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage") + val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.DISK_ONLY.description) + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.DISK_ONLY.description) + } + + rdd.unpersist() + rdd.persist(StorageLevels.MEMORY_ONLY).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage") + val tableRowText = findAll(cssSelector("#storage-by-rdd-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.MEMORY_ONLY.description) + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (ui.appUIAddress.stripSuffix("/") + "/storage/rdd/?id=0") + val tableRowText = findAll(cssSelector("#rdd-storage-by-block-table td")).map(_.text).toSeq + tableRowText should contain (StorageLevels.MEMORY_ONLY.description) + } + } + } + + test("failed stages should not appear to be active") { + withSpark(newSparkContext()) { sc => + // Regression test for SPARK-3021 + intercept[SparkException] { + sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to sc.ui.get.appUIAddress + find(id("active")).get.text should be("Active Stages (0)") + find(id("failed")).get.text should be("Failed Stages (1)") + } + + // Regression test for SPARK-2105 + class NotSerializable + val unserializableObject = new NotSerializable + intercept[SparkException] { + sc.parallelize(1 to 10).map { x => unserializableObject}.collect() + } + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to sc.ui.get.appUIAddress + find(id("active")).get.text should be("Active Stages (0)") + // The failure occurs before the stage becomes active, hence we should still show only one + // failed stage, not two: + find(id("failed")).get.text should be("Failed Stages (1)") + } + } + } +} diff --git a/pom.xml b/pom.xml index 2faf0c7dcffd3..2ebe1b8da588a 100644 --- a/pom.xml +++ b/pom.xml @@ -312,6 +312,12 @@ jsr305 1.3.9 + + org.seleniumhq.selenium + selenium-java + 2.42.2 + test + org.slf4j slf4j-api @@ -520,7 +526,7 @@ org.scalatest scalatest_${scala.binary.version} - 2.1.5 + 2.2.1 test 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 6dc5942023f9e..f134d73450515 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 @@ -23,7 +23,7 @@ import scala.collection.immutable.HashSet import org.scalatest.FunSuite import org.scalatest.Matchers._ -import org.scalautils.TripleEqualsSupport.Spread +import org.scalactic.TripleEqualsSupport.Spread import org.apache.spark.sql.catalyst.types._ From 677852c3fa734326f9642245584b05d5604963ac Mon Sep 17 00:00:00 2001 From: anant asthana Date: Sun, 26 Oct 2014 14:14:12 -0700 Subject: [PATCH 1079/1492] Just fixing comment that shows usage Author: anant asthana Closes #2948 from anantasty/patch-1 and squashes the following commits: d8fea0b [anant asthana] Just fixing comment that shows usage --- examples/src/main/python/streaming/hdfs_wordcount.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py index 40faff0ccc7db..f7ffb5379681e 100644 --- a/examples/src/main/python/streaming/hdfs_wordcount.py +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -21,7 +21,7 @@ is the directory that Spark Streaming will use to find and read new text files. To run this on your local machine on directory `localdir`, run this example - $ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localdir + $ bin/spark-submit examples/src/main/python/streaming/hdfs_wordcount.py localdir Then create a text file in `localdir` and the words in the file will get counted. """ From 0af7e514c6790cb78ee216986c45dbfe1217cd7d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 26 Oct 2014 14:29:13 -0700 Subject: [PATCH 1080/1492] [SPARK-3925][SQL] Do not consider the ordering of qualifiers during comparison The orderings should not be considered during the comparison between old qualifiers and new qualifiers. Author: Liang-Chi Hsieh Closes #2783 from viirya/full_qualifier_comp and squashes the following commits: 89f652c [Liang-Chi Hsieh] modification for comment. abb5762 [Liang-Chi Hsieh] More comprehensive comparison of qualifiers. --- .../spark/sql/catalyst/expressions/namedExpressions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index d023db44d8543..fe13a661f6f7a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -156,7 +156,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea * Returns a copy of this [[AttributeReference]] with new qualifiers. */ override def withQualifiers(newQualifiers: Seq[String]) = { - if (newQualifiers == qualifiers) { + if (newQualifiers.toSet == qualifiers.toSet) { this } else { AttributeReference(name, dataType, nullable)(exprId, newQualifiers) From 879a16585808e8fe34bdede741565efc4c9f9bb3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 26 Oct 2014 15:24:39 -0700 Subject: [PATCH 1081/1492] [HOTFIX][SQL] Temporarily turn off hive-server tests. The thirift server is not available in the default (hive13) profile yet which is breaking all SQL only PRs. This turns off these test until #2685 is merged. Author: Michael Armbrust Closes #2950 from marmbrus/fixTests and squashes the following commits: 1a6dfee [Michael Armbrust] [HOTFIX][SQL] Temporarily turn of hive-server tests. --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index f55497ae2bfbd..972c8c8a21567 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -173,7 +173,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string #+ will be interpreted as a single test, which doesn't work. - SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "hive-thriftserver/test") + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test") else SBT_MAVEN_TEST_ARGS=("test") fi From 2838bf8aadd5228829c1a869863bc4da7877fdfb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 26 Oct 2014 16:10:09 -0700 Subject: [PATCH 1082/1492] [SPARK-3537][SPARK-3914][SQL] Refines in-memory columnar table statistics This PR refines in-memory columnar table statistics: 1. adds 2 more statistics for in-memory table columns: `count` and `sizeInBytes` 1. adds filter pushdown support for `IS NULL` and `IS NOT NULL`. 1. caches and propagates statistics in `InMemoryRelation` once the underlying cached RDD is materialized. Statistics are collected to driver side with an accumulator. This PR also fixes SPARK-3914 by properly propagating in-memory statistics. Author: Cheng Lian Closes #2860 from liancheng/propagates-in-mem-stats and squashes the following commits: 0cc5271 [Cheng Lian] Restricts visibility of o.a.s.s.c.p.l.Statistics c5ff904 [Cheng Lian] Fixes test table name conflict a8c818d [Cheng Lian] Refines tests 1d01074 [Cheng Lian] Bug fix: shouldn't call STRING.actualSize on null string value 7dc6a34 [Cheng Lian] Adds more in-memory table statistics and propagates them properly --- .../catalyst/expressions/AttributeMap.scala | 10 +- .../catalyst/plans/logical/LogicalPlan.scala | 31 +++-- .../spark/sql/columnar/ColumnStats.scala | 122 +++++++++--------- .../columnar/InMemoryColumnarTableScan.scala | 101 +++++++++------ .../spark/sql/execution/ExistingRDD.scala | 11 +- .../spark/sql/parquet/ParquetRelation.scala | 3 +- .../apache/spark/sql/CachedTableSuite.scala | 11 +- .../scala/org/apache/spark/sql/TestData.scala | 16 +-- .../spark/sql/columnar/ColumnStatsSuite.scala | 6 + .../columnar/PartitionBatchPruningSuite.scala | 76 ++++++----- .../spark/sql/execution/PlannerSuite.scala | 20 +++ 11 files changed, 240 insertions(+), 167 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala index 8364379644c90..82e760b6c6916 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala @@ -23,8 +23,7 @@ package org.apache.spark.sql.catalyst.expressions * of the name, or the expected nullability). */ object AttributeMap { - def apply[A](kvs: Seq[(Attribute, A)]) = - new AttributeMap(kvs.map(kv => (kv._1.exprId, (kv._1, kv._2))).toMap) + def apply[A](kvs: Seq[(Attribute, A)]) = new AttributeMap(kvs.map(kv => (kv._1.exprId, kv)).toMap) } class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) @@ -32,10 +31,9 @@ class AttributeMap[A](baseMap: Map[ExprId, (Attribute, A)]) override def get(k: Attribute): Option[A] = baseMap.get(k.exprId).map(_._2) - override def + [B1 >: A](kv: (Attribute, B1)): Map[Attribute, B1] = - (baseMap.map(_._2) + kv).toMap + override def + [B1 >: A](kv: (Attribute, B1)): Map[Attribute, B1] = baseMap.values.toMap + kv - override def iterator: Iterator[(Attribute, A)] = baseMap.map(_._2).iterator + override def iterator: Iterator[(Attribute, A)] = baseMap.valuesIterator - override def -(key: Attribute): Map[Attribute, A] = (baseMap.map(_._2) - key).toMap + override def -(key: Attribute): Map[Attribute, A] = baseMap.values.toMap - key } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 882e9c6110089..ed578e081be73 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -26,25 +26,24 @@ import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.StructType import org.apache.spark.sql.catalyst.trees +/** + * Estimates of various statistics. The default estimation logic simply lazily multiplies the + * corresponding statistic produced by the children. To override this behavior, override + * `statistics` and assign it an overriden version of `Statistics`. + * + * '''NOTE''': concrete and/or overriden versions of statistics fields should pay attention to the + * performance of the implementations. The reason is that estimations might get triggered in + * performance-critical processes, such as query plan planning. + * + * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it + * defaults to the product of children's `sizeInBytes`. + */ +private[sql] case class Statistics(sizeInBytes: BigInt) + abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { self: Product => - /** - * Estimates of various statistics. The default estimation logic simply lazily multiplies the - * corresponding statistic produced by the children. To override this behavior, override - * `statistics` and assign it an overriden version of `Statistics`. - * - * '''NOTE''': concrete and/or overriden versions of statistics fields should pay attention to the - * performance of the implementations. The reason is that estimations might get triggered in - * performance-critical processes, such as query plan planning. - * - * @param sizeInBytes Physical size in bytes. For leaf operators this defaults to 1, otherwise it - * defaults to the product of children's `sizeInBytes`. - */ - case class Statistics( - sizeInBytes: BigInt - ) - lazy val statistics: Statistics = { + def statistics: Statistics = { if (children.size == 0) { throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index b34ab255d084a..b9f9f8270045c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -24,11 +24,13 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeMap, Attribute, Attri import org.apache.spark.sql.catalyst.types._ private[sql] class ColumnStatisticsSchema(a: Attribute) extends Serializable { - val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = false)() - val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = false)() - val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() + val upperBound = AttributeReference(a.name + ".upperBound", a.dataType, nullable = true)() + val lowerBound = AttributeReference(a.name + ".lowerBound", a.dataType, nullable = true)() + val nullCount = AttributeReference(a.name + ".nullCount", IntegerType, nullable = false)() + val count = AttributeReference(a.name + ".count", IntegerType, nullable = false)() + val sizeInBytes = AttributeReference(a.name + ".sizeInBytes", LongType, nullable = false)() - val schema = Seq(lowerBound, upperBound, nullCount) + val schema = Seq(lowerBound, upperBound, nullCount, count, sizeInBytes) } private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Serializable { @@ -45,10 +47,21 @@ private[sql] class PartitionStatistics(tableSchema: Seq[Attribute]) extends Seri * brings significant performance penalty. */ private[sql] sealed trait ColumnStats extends Serializable { + protected var count = 0 + protected var nullCount = 0 + protected var sizeInBytes = 0L + /** * Gathers statistics information from `row(ordinal)`. */ - def gatherStats(row: Row, ordinal: Int): Unit + def gatherStats(row: Row, ordinal: Int): Unit = { + if (row.isNullAt(ordinal)) { + nullCount += 1 + // 4 bytes for null position + sizeInBytes += 4 + } + count += 1 + } /** * Column statistics represented as a single row, currently including closed lower bound, closed @@ -65,163 +78,154 @@ private[sql] class NoopColumnStats extends ColumnStats { } private[sql] class ByteColumnStats extends ColumnStats { - var upper = Byte.MinValue - var lower = Byte.MaxValue - var nullCount = 0 + protected var upper = Byte.MinValue + protected var lower = Byte.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getByte(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += BYTE.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class ShortColumnStats extends ColumnStats { - var upper = Short.MinValue - var lower = Short.MaxValue - var nullCount = 0 + protected var upper = Short.MinValue + protected var lower = Short.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getShort(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += SHORT.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class LongColumnStats extends ColumnStats { - var upper = Long.MinValue - var lower = Long.MaxValue - var nullCount = 0 + protected var upper = Long.MinValue + protected var lower = Long.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getLong(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += LONG.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class DoubleColumnStats extends ColumnStats { - var upper = Double.MinValue - var lower = Double.MaxValue - var nullCount = 0 + protected var upper = Double.MinValue + protected var lower = Double.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getDouble(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += DOUBLE.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class FloatColumnStats extends ColumnStats { - var upper = Float.MinValue - var lower = Float.MaxValue - var nullCount = 0 + protected var upper = Float.MinValue + protected var lower = Float.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getFloat(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += FLOAT.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class IntColumnStats extends ColumnStats { - var upper = Int.MinValue - var lower = Int.MaxValue - var nullCount = 0 + protected var upper = Int.MinValue + protected var lower = Int.MaxValue override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getInt(ordinal) if (value > upper) upper = value if (value < lower) lower = value - } else { - nullCount += 1 + sizeInBytes += INT.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class StringColumnStats extends ColumnStats { - var upper: String = null - var lower: String = null - var nullCount = 0 + protected var upper: String = null + protected var lower: String = null override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row.getString(ordinal) if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value - } else { - nullCount += 1 + sizeInBytes += STRING.actualSize(row, ordinal) } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class DateColumnStats extends ColumnStats { - var upper: Date = null - var lower: Date = null - var nullCount = 0 + protected var upper: Date = null + protected var lower: Date = null override def gatherStats(row: Row, ordinal: Int) { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row(ordinal).asInstanceOf[Date] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value - } else { - nullCount += 1 + sizeInBytes += DATE.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class TimestampColumnStats extends ColumnStats { - var upper: Timestamp = null - var lower: Timestamp = null - var nullCount = 0 + protected var upper: Timestamp = null + protected var lower: Timestamp = null override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { val value = row(ordinal).asInstanceOf[Timestamp] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value - } else { - nullCount += 1 + sizeInBytes += TIMESTAMP.defaultSize } } - def collectedStatistics = Row(lower, upper, nullCount) + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } 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 22ab0e2613f21..ee63134f56d8c 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 @@ -19,13 +19,15 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.execution.{LeafNode, SparkPlan} import org.apache.spark.storage.StorageLevel @@ -45,15 +47,51 @@ private[sql] case class InMemoryRelation( useCompression: Boolean, batchSize: Int, storageLevel: StorageLevel, - child: SparkPlan) - (private var _cachedColumnBuffers: RDD[CachedBatch] = null) + child: SparkPlan)( + private var _cachedColumnBuffers: RDD[CachedBatch] = null, + private var _statistics: Statistics = null) extends LogicalPlan with MultiInstanceRelation { - override lazy val statistics = - Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes) + private val batchStats = + child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) val partitionStatistics = new PartitionStatistics(output) + private def computeSizeInBytes = { + val sizeOfRow: Expression = + BindReferences.bindReference( + output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), + partitionStatistics.schema) + + batchStats.value.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum + } + + // Statistics propagation contracts: + // 1. Non-null `_statistics` must reflect the actual statistics of the underlying data + // 2. Only propagate statistics when `_statistics` is non-null + private def statisticsToBePropagated = if (_statistics == null) { + val updatedStats = statistics + if (_statistics == null) null else updatedStats + } else { + _statistics + } + + override def statistics = if (_statistics == null) { + if (batchStats.value.isEmpty) { + // Underlying columnar RDD hasn't been materialized, no useful statistics information + // available, return the default statistics. + Statistics(sizeInBytes = child.sqlContext.defaultSizeInBytes) + } else { + // Underlying columnar RDD has been materialized, required information has also been collected + // via the `batchStats` accumulator, compute the final statistics, and update `_statistics`. + _statistics = Statistics(sizeInBytes = computeSizeInBytes) + _statistics + } + } else { + // Pre-computed statistics + _statistics + } + // If the cached column buffers were not passed in, we calculate them in the constructor. // As in Spark, the actual work of caching is lazy. if (_cachedColumnBuffers == null) { @@ -91,6 +129,7 @@ private[sql] case class InMemoryRelation( val stats = Row.fromSeq( columnBuilders.map(_.columnStats.collectedStatistics).foldLeft(Seq.empty[Any])(_ ++ _)) + batchStats += stats CachedBatch(columnBuilders.map(_.build().array()), stats) } @@ -104,7 +143,8 @@ private[sql] case class InMemoryRelation( def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( - newOutput, useCompression, batchSize, storageLevel, child)(_cachedColumnBuffers) + newOutput, useCompression, batchSize, storageLevel, child)( + _cachedColumnBuffers, statisticsToBePropagated) } override def children = Seq.empty @@ -116,7 +156,8 @@ private[sql] case class InMemoryRelation( batchSize, storageLevel, child)( - _cachedColumnBuffers).asInstanceOf[this.type] + _cachedColumnBuffers, + statisticsToBePropagated).asInstanceOf[this.type] } def cachedColumnBuffers = _cachedColumnBuffers @@ -132,6 +173,8 @@ private[sql] case class InMemoryColumnarTableScan( override def output: Seq[Attribute] = attributes + private def statsFor(a: Attribute) = relation.partitionStatistics.forAttribute(a) + // Returned filter predicate should return false iff it is impossible for the input expression // to evaluate to `true' based on statistics collected about this partition batch. val buildFilter: PartialFunction[Expression, Expression] = { @@ -144,44 +187,24 @@ private[sql] case class InMemoryColumnarTableScan( buildFilter(lhs) || buildFilter(rhs) case EqualTo(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l && l <= aStats.upperBound - + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound case EqualTo(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l && l <= aStats.upperBound - - case LessThan(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound < l - - case LessThan(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - l < aStats.upperBound - - case LessThanOrEqual(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l + statsFor(a).lowerBound <= l && l <= statsFor(a).upperBound - case LessThanOrEqual(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - l <= aStats.upperBound + case LessThan(a: AttributeReference, l: Literal) => statsFor(a).lowerBound < l + case LessThan(l: Literal, a: AttributeReference) => l < statsFor(a).upperBound - case GreaterThan(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - l < aStats.upperBound + case LessThanOrEqual(a: AttributeReference, l: Literal) => statsFor(a).lowerBound <= l + case LessThanOrEqual(l: Literal, a: AttributeReference) => l <= statsFor(a).upperBound - case GreaterThan(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound < l + case GreaterThan(a: AttributeReference, l: Literal) => l < statsFor(a).upperBound + case GreaterThan(l: Literal, a: AttributeReference) => statsFor(a).lowerBound < l - case GreaterThanOrEqual(a: AttributeReference, l: Literal) => - val aStats = relation.partitionStatistics.forAttribute(a) - l <= aStats.upperBound + case GreaterThanOrEqual(a: AttributeReference, l: Literal) => l <= statsFor(a).upperBound + case GreaterThanOrEqual(l: Literal, a: AttributeReference) => statsFor(a).lowerBound <= l - case GreaterThanOrEqual(l: Literal, a: AttributeReference) => - val aStats = relation.partitionStatistics.forAttribute(a) - aStats.lowerBound <= l + case IsNull(a: Attribute) => statsFor(a).nullCount > 0 + case IsNotNull(a: Attribute) => statsFor(a).count - statsFor(a).nullCount > 0 } val partitionFilters = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 2ddf513b6fc98..04c51a1ee4b97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -17,16 +17,13 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - -import scala.reflect.runtime.universe.TypeTag - import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} +import org.apache.spark.sql.{Row, SQLContext} /** * :: DeveloperApi :: @@ -100,7 +97,7 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ override final def newInstance(): this.type = { SparkLogicalPlan( alreadyPlanned match { - case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) + case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance()), rdd) case _ => sys.error("Multiple instance of the same relation detected.") })(sqlContext).asInstanceOf[this.type] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 5ae768293a22e..82130b5459174 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -22,7 +22,6 @@ import java.io.IOException import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.permission.FsAction - import parquet.hadoop.ParquetOutputFormat import parquet.hadoop.metadata.CompressionCodecName import parquet.schema.MessageType @@ -30,7 +29,7 @@ import parquet.schema.MessageType import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} /** * Relation that consists of data stored in a Parquet columnar format. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index da5a358df3b1d..1a5d87d5240e9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.columnar.{InMemoryColumnarTableScan, InMemoryRelation} +import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.storage.{StorageLevel, RDDBlockId} @@ -234,4 +234,13 @@ class CachedTableSuite extends QueryTest { uncacheTable("testData") assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } + + test("InMemoryRelation statistics") { + sql("CACHE TABLE testData") + table("testData").queryExecution.withCachedData.collect { + case cached: InMemoryRelation => + val actualSizeInBytes = (1 to 100).map(i => INT.defaultSize + i.toString.length + 4).sum + assert(cached.statistics.sizeInBytes === actualSizeInBytes) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 10b7979df7375..1c21afc17e25e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -28,40 +28,40 @@ import org.apache.spark.sql.test.TestSQLContext._ case class TestData(key: Int, value: String) object TestData { - val testData: SchemaRDD = TestSQLContext.sparkContext.parallelize( - (1 to 100).map(i => TestData(i, i.toString))) + val testData = TestSQLContext.sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))).toSchemaRDD testData.registerTempTable("testData") case class LargeAndSmallInts(a: Int, b: Int) - val largeAndSmallInts: SchemaRDD = + val largeAndSmallInts = TestSQLContext.sparkContext.parallelize( LargeAndSmallInts(2147483644, 1) :: LargeAndSmallInts(1, 2) :: LargeAndSmallInts(2147483645, 1) :: LargeAndSmallInts(2, 2) :: LargeAndSmallInts(2147483646, 1) :: - LargeAndSmallInts(3, 2) :: Nil) + LargeAndSmallInts(3, 2) :: Nil).toSchemaRDD largeAndSmallInts.registerTempTable("largeAndSmallInts") case class TestData2(a: Int, b: Int) - val testData2: SchemaRDD = + val testData2 = TestSQLContext.sparkContext.parallelize( TestData2(1, 1) :: TestData2(1, 2) :: TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: - TestData2(3, 2) :: Nil) + TestData2(3, 2) :: Nil).toSchemaRDD testData2.registerTempTable("testData2") case class BinaryData(a: Array[Byte], b: Int) - val binaryData: SchemaRDD = + val binaryData = TestSQLContext.sparkContext.parallelize( BinaryData("12".getBytes(), 1) :: BinaryData("22".getBytes(), 5) :: BinaryData("122".getBytes(), 3) :: BinaryData("121".getBytes(), 2) :: - BinaryData("123".getBytes(), 4) :: Nil) + BinaryData("123".getBytes(), 4) :: Nil).toSchemaRDD binaryData.registerTempTable("binaryData") // TODO: There is no way to express null primitives as case classes currently... diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala index 6bdf741134e2f..a9f0851f8826c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -61,6 +61,12 @@ class ColumnStatsSuite extends FunSuite { assertResult(values.min(ordering), "Wrong lower bound")(stats(0)) assertResult(values.max(ordering), "Wrong upper bound")(stats(1)) assertResult(10, "Wrong null count")(stats(2)) + assertResult(20, "Wrong row count")(stats(3)) + assertResult(stats(4), "Wrong size in bytes") { + rows.map { row => + if (row.isNullAt(0)) 4 else columnType.actualSize(row, 0) + }.sum + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index f53acc8c9f718..9ba3c210171bd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -22,8 +22,6 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.sql._ import org.apache.spark.sql.test.TestSQLContext._ -case class IntegerData(i: Int) - class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { val originalColumnBatchSize = columnBatchSize val originalInMemoryPartitionPruning = inMemoryPartitionPruning @@ -31,8 +29,12 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be override protected def beforeAll(): Unit = { // Make a table with 5 partitions, 2 batches per partition, 10 elements per batch setConf(SQLConf.COLUMN_BATCH_SIZE, "10") - val rawData = sparkContext.makeRDD(1 to 100, 5).map(IntegerData) - rawData.registerTempTable("intData") + + val pruningData = sparkContext.makeRDD((1 to 100).map { key => + val string = if (((key - 1) / 10) % 2 == 0) null else key.toString + TestData(key, string) + }, 5) + pruningData.registerTempTable("pruningData") // Enable in-memory partition pruning setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") @@ -44,48 +46,64 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be } before { - cacheTable("intData") + cacheTable("pruningData") } after { - uncacheTable("intData") + uncacheTable("pruningData") } // Comparisons - checkBatchPruning("i = 1", Seq(1), 1, 1) - checkBatchPruning("1 = i", Seq(1), 1, 1) - checkBatchPruning("i < 12", 1 to 11, 1, 2) - checkBatchPruning("i <= 11", 1 to 11, 1, 2) - checkBatchPruning("i > 88", 89 to 100, 1, 2) - checkBatchPruning("i >= 89", 89 to 100, 1, 2) - checkBatchPruning("12 > i", 1 to 11, 1, 2) - checkBatchPruning("11 >= i", 1 to 11, 1, 2) - checkBatchPruning("88 < i", 89 to 100, 1, 2) - checkBatchPruning("89 <= i", 89 to 100, 1, 2) + checkBatchPruning("SELECT key FROM pruningData WHERE key = 1", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE 1 = key", 1, 1)(Seq(1)) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 12", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE key <= 11", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE key > 88", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE key >= 89", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE 12 > key", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE 11 >= key", 1, 2)(1 to 11) + checkBatchPruning("SELECT key FROM pruningData WHERE 88 < key", 1, 2)(89 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE 89 <= key", 1, 2)(89 to 100) + + // IS NULL + checkBatchPruning("SELECT key FROM pruningData WHERE value IS NULL", 5, 5) { + (1 to 10) ++ (21 to 30) ++ (41 to 50) ++ (61 to 70) ++ (81 to 90) + } + + // IS NOT NULL + checkBatchPruning("SELECT key FROM pruningData WHERE value IS NOT NULL", 5, 5) { + (11 to 20) ++ (31 to 40) ++ (51 to 60) ++ (71 to 80) ++ (91 to 100) + } // Conjunction and disjunction - checkBatchPruning("i > 8 AND i <= 21", 9 to 21, 2, 3) - checkBatchPruning("i < 2 OR i > 99", Seq(1, 100), 2, 2) - checkBatchPruning("i < 2 OR (i > 78 AND i < 92)", Seq(1) ++ (79 to 91), 3, 4) - checkBatchPruning("NOT (i < 88)", 88 to 100, 1, 2) + checkBatchPruning("SELECT key FROM pruningData WHERE key > 8 AND key <= 21", 2, 3)(9 to 21) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR key > 99", 2, 2)(Seq(1, 100)) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 2 OR (key > 78 AND key < 92)", 3, 4) { + Seq(1) ++ (79 to 91) + } // With unsupported predicate - checkBatchPruning("i < 12 AND i IS NOT NULL", 1 to 11, 1, 2) - checkBatchPruning(s"NOT (i in (${(1 to 30).mkString(",")}))", 31 to 100, 5, 10) + checkBatchPruning("SELECT key FROM pruningData WHERE NOT (key < 88)", 1, 2)(88 to 100) + checkBatchPruning("SELECT key FROM pruningData WHERE key < 12 AND key IS NOT NULL", 1, 2)(1 to 11) + + { + val seq = (1 to 30).mkString(", ") + checkBatchPruning(s"SELECT key FROM pruningData WHERE NOT (key IN ($seq))", 5, 10)(31 to 100) + } def checkBatchPruning( - filter: String, - expectedQueryResult: Seq[Int], + query: String, expectedReadPartitions: Int, - expectedReadBatches: Int): Unit = { + expectedReadBatches: Int)( + expectedQueryResult: => Seq[Int]): Unit = { - test(filter) { - val query = sql(s"SELECT * FROM intData WHERE $filter") + test(query) { + val schemaRdd = sql(query) assertResult(expectedQueryResult.toArray, "Wrong query result") { - query.collect().map(_.head).toArray + schemaRdd.collect().map(_.head).toArray } - val (readPartitions, readBatches) = query.queryExecution.executedPlan.collect { + val (readPartitions, readBatches) = schemaRdd.queryExecution.executedPlan.collect { case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) }.head diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index f14ffca0e4d35..a5af71acfc79a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -76,4 +76,24 @@ class PlannerSuite extends FunSuite { setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) } + + test("InMemoryRelation statistics propagation") { + val origThreshold = autoBroadcastJoinThreshold + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, 81920.toString) + + testData.limit(3).registerTempTable("tiny") + sql("CACHE TABLE tiny") + + val a = testData.as('a) + val b = table("tiny").as('b) + val planned = a.join(b, Inner, Some("a.key".attr === "b.key".attr)).queryExecution.executedPlan + + val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } + val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } + + assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") + assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") + + setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) + } } From 89e8a5d8ba57255c46cadfc803fcddeeec93325f Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sun, 26 Oct 2014 16:24:50 -0700 Subject: [PATCH 1083/1492] [SPARK-3997][Build]scalastyle should output the error location Author: GuoQiang Li Closes #2846 from witgo/SPARK-3997 and squashes the following commits: d6a57f8 [GuoQiang Li] scalastyle should output the error location --- dev/scalastyle | 2 +- project/SparkBuild.scala | 3 +-- project/plugins.sbt | 2 +- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/dev/scalastyle b/dev/scalastyle index c3b356bcb3c06..ed1b6b730af6e 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -25,7 +25,7 @@ echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn- 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 "\") +ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') rm scalastyle.txt if test ! -z "$ERRORS"; then diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index ea04473854007..6d5eb681c6131 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -23,7 +23,6 @@ import sbt.Classpaths.publishTask import sbt.Keys._ import sbtunidoc.Plugin.genjavadocSettings import sbtunidoc.Plugin.UnidocKeys.unidocGenjavadocVersion -import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} import com.typesafe.sbt.pom.{PomBuild, SbtPomKeys} import net.virtualvoid.sbt.graph.Plugin.graphSettings @@ -111,7 +110,7 @@ object SparkBuild extends PomBuild { 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 ++ genjavadocSettings ++ Seq ( + lazy val sharedSettings = graphSettings ++ genjavadocSettings ++ Seq ( javaHome := Properties.envOrNone("JAVA_HOME").map(file), incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, diff --git a/project/plugins.sbt b/project/plugins.sbt index 9d50a50b109af..ee45b6a51905e 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,7 +19,7 @@ addSbtPlugin("com.github.mpeltonen" % "sbt-idea" % "1.6.0") addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") -addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.5.0") +addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.6.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") From dc51f4d6d836fc2c65b0af8db1497427ad2742ec Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:27:29 -0700 Subject: [PATCH 1084/1492] [SQL][DOC] Wrong package name "scala.math.sql" in sql-programming-guide.md In sql-programming-guide.md, there is a wrong package name "scala.math.sql". Author: Kousuke Saruta Closes #2873 from sarutak/wrong-packagename-fix and squashes the following commits: 4d5ecf4 [Kousuke Saruta] Fixed wrong package name in sql-programming-guide.md --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 368c3d0008b07..d4ade939c3a6e 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1215,7 +1215,7 @@ import org.apache.spark.sql._
    - + From d518bc24af54a3853fac457e03c70970354150bb Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:28:33 -0700 Subject: [PATCH 1085/1492] [SPARK-3953][SQL][Minor] Confusable variable name. In SqlParser.scala, there is following code. case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => val base = r.getOrElse(NoRelation) val withFilter = f.map(f => Filter(f, base)).getOrElse(base) In the code above, there are 2 variables which have same name "f" in near place. One is receiver "f" and other is bound variable "f". Author: Kousuke Saruta Closes #2807 from sarutak/SPARK-3953 and squashes the following commits: 4957c32 [Kousuke Saruta] Improved variable name in SqlParser.scala --- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a277684f6327c..9bea990fcee4e 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -142,7 +142,7 @@ class SqlParser extends AbstractSparkSQLParser { (LIMIT ~> expression).? ^^ { case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l => val base = r.getOrElse(NoRelation) - val withFilter = f.map(f => Filter(f, base)).getOrElse(base) + val withFilter = f.map(Filter(_, base)).getOrElse(base) val withProjection = g .map(Aggregate(_, assignAliases(p), withFilter)) .getOrElse(Project(assignAliases(p), withFilter)) From 05308426f0f51273be95fb1ca2cb1ec19d83cec8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 26 Oct 2014 16:30:15 -0700 Subject: [PATCH 1086/1492] [SPARK-4052][SQL] Use scala.collection.Map for pattern matching instead of using Predef.Map (it is scala.collection.immutable.Map) Please check https://issues.apache.org/jira/browse/SPARK-4052 for cases triggering this bug. Author: Yin Huai Closes #2899 from yhuai/SPARK-4052 and squashes the following commits: 1188f70 [Yin Huai] Address liancheng's comments. b6712be [Yin Huai] Use scala.collection.Map instead of Predef.Map (scala.collection.immutable.Map). --- .../spark/sql/catalyst/ScalaReflection.scala | 3 +++ .../scala/org/apache/spark/sql/TestData.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 4 ++++ .../sql/hive/InsertIntoHiveTableSuite.scala | 18 ++++++++++++++++++ 4 files changed, 26 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 3d4296f9d7068..7d930fccd52d1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -27,6 +27,9 @@ import org.apache.spark.sql.catalyst.types._ * Provides experimental support for generating catalyst schemas for scala objects. */ object ScalaReflection { + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map import scala.reflect.runtime.universe._ case class Schema(dataType: DataType, nullable: Boolean) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 1c21afc17e25e..6c38575b13a2d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -99,7 +99,7 @@ object TestData { ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) arrayData.registerTempTable("arrayData") - case class MapData(data: Map[Int, String]) + case class MapData(data: scala.collection.Map[Int, String]) val mapData = TestSQLContext.sparkContext.parallelize( MapData(Map(1 -> "a1", 2 -> "b1", 3 -> "c1", 4 -> "d1", 5 -> "e1")) :: diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 7db5fd804d6ef..79234f8a66f05 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -93,6 +93,10 @@ case class InsertIntoHiveTable( (o: Any) => seqAsJavaList(o.asInstanceOf[Seq[_]].map(wrapper)) case moi: MapObjectInspector => + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map + val keyWrapper = wrapperFor(moi.getMapKeyObjectInspector) val valueWrapper = wrapperFor(moi.getMapValueObjectInspector) (o: Any) => mapAsJavaMap(o.asInstanceOf[Map[_, _]].map { case (key, value) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 7e323146f9da2..18dc937dd2b27 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.QueryTest +import org.apache.spark.sql._ import org.apache.spark.sql.hive.test.TestHive /* Implicits */ @@ -73,4 +74,21 @@ class InsertIntoHiveTableSuite extends QueryTest { createTable[TestData]("createAndInsertTest") createTable[TestData]("createAndInsertTest") } + + test("SPARK-4052: scala.collection.Map as value type of MapType") { + val schema = StructType(StructField("m", MapType(StringType, StringType), true) :: Nil) + val rowRDD = TestHive.sparkContext.parallelize( + (1 to 100).map(i => Row(scala.collection.mutable.HashMap(s"key$i" -> s"value$i")))) + val schemaRDD = applySchema(rowRDD, schema) + schemaRDD.registerTempTable("tableWithMapValue") + sql("CREATE TABLE hiveTableWithMapValue(m MAP )") + sql("INSERT OVERWRITE TABLE hiveTableWithMapValue SELECT m FROM tableWithMapValue") + + checkAnswer( + sql("SELECT * FROM hiveTableWithMapValue"), + rowRDD.collect().toSeq + ) + + sql("DROP TABLE hiveTableWithMapValue") + } } From 0481aaa8d751d0df81a1f79fe3f47ca62afd00e8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 26 Oct 2014 16:32:02 -0700 Subject: [PATCH 1087/1492] [SPARK-4068][SQL] NPE in jsonRDD schema inference Please refer to added tests for cases that can trigger the bug. JIRA: https://issues.apache.org/jira/browse/SPARK-4068 Author: Yin Huai Closes #2918 from yhuai/SPARK-4068 and squashes the following commits: d360eae [Yin Huai] Handle nulls when building key paths from elements of an array. --- .../org/apache/spark/sql/json/JsonRDD.scala | 4 +- .../org/apache/spark/sql/json/JsonSuite.scala | 35 ++++++++++++++++- .../apache/spark/sql/json/TestJsonData.scala | 39 +++++++++++-------- 3 files changed, 58 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 61ee960aad9d2..bf32da1b7181e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -242,14 +242,14 @@ private[sql] object JsonRDD extends Logging { def buildKeyPathForInnerStructs(v: Any, t: DataType): Seq[(String, DataType)] = t match { case ArrayType(StructType(Nil), containsNull) => { // The elements of this arrays are structs. - v.asInstanceOf[Seq[Map[String, Any]]].flatMap { + v.asInstanceOf[Seq[Map[String, Any]]].flatMap(Option(_)).flatMap { element => allKeysWithValueTypes(element) }.map { case (k, t) => (s"$key.$k", t) } } case ArrayType(t1, containsNull) => - v.asInstanceOf[Seq[Any]].flatMap { + v.asInstanceOf[Seq[Any]].flatMap(Option(_)).flatMap { element => buildKeyPathForInnerStructs(element, t1) } case other => Nil diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 7bb08f1b513ce..41927e83698a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -208,7 +208,7 @@ class JsonSuite extends QueryTest { } test("Complex field and type inferring") { - val jsonSchemaRDD = jsonRDD(complexFieldAndType) + val jsonSchemaRDD = jsonRDD(complexFieldAndType1) val expectedSchema = StructType( StructField("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: @@ -305,7 +305,7 @@ class JsonSuite extends QueryTest { } ignore("Complex field and type inferring (Ignored)") { - val jsonSchemaRDD = jsonRDD(complexFieldAndType) + val jsonSchemaRDD = jsonRDD(complexFieldAndType1) jsonSchemaRDD.registerTempTable("jsonTable") // Right now, "field1" and "field2" are treated as aliases. We should fix it. @@ -707,4 +707,35 @@ class JsonSuite extends QueryTest { TestSQLContext.setConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD, oldColumnNameOfCorruptRecord) } + + test("SPARK-4068: nulls in arrays") { + val jsonSchemaRDD = jsonRDD(nullsInArrays) + jsonSchemaRDD.registerTempTable("jsonTable") + + val schema = StructType( + StructField("field1", + ArrayType(ArrayType(ArrayType(ArrayType(StringType, false), false), true), false), true) :: + StructField("field2", + ArrayType(ArrayType( + StructType(StructField("Test", IntegerType, true) :: Nil), false), true), true) :: + StructField("field3", + ArrayType(ArrayType( + StructType(StructField("Test", StringType, true) :: Nil), true), false), true) :: + StructField("field4", + ArrayType(ArrayType(ArrayType(IntegerType, false), true), false), true) :: Nil) + + assert(schema === jsonSchemaRDD.schema) + + checkAnswer( + sql( + """ + |SELECT field1, field2, field3, field4 + |FROM jsonTable + """.stripMargin), + Seq(Seq(Seq(null), Seq(Seq(Seq("Test")))), null, null, null) :: + Seq(null, Seq(null, Seq(Seq(1))), null, null) :: + Seq(null, null, Seq(Seq(null), Seq(Seq("2"))), null) :: + Seq(null, null, null, Seq(Seq(null, Seq(1, 2, 3)))) :: Nil + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index eaca9f0508a12..c204162dd2fc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -32,22 +32,6 @@ object TestJsonData { "null":null }""" :: Nil) - val complexFieldAndType = - TestSQLContext.sparkContext.parallelize( - """{"struct":{"field1": true, "field2": 92233720368547758070}, - "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, - "arrayOfString":["str1", "str2"], - "arrayOfInteger":[1, 2147483647, -2147483648], - "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], - "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], - "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], - "arrayOfBoolean":[true, false, true], - "arrayOfNull":[null, null, null, null], - "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], - "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], - "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] - }""" :: Nil) - val primitiveFieldValueTypeConflict = TestSQLContext.sparkContext.parallelize( """{"num_num_1":11, "num_num_2":null, "num_num_3": 1.1, @@ -83,6 +67,22 @@ object TestJsonData { """{"d":{"field":true}}""" :: """{"e":"str"}""" :: Nil) + val complexFieldAndType1 = + TestSQLContext.sparkContext.parallelize( + """{"struct":{"field1": true, "field2": 92233720368547758070}, + "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, + "arrayOfString":["str1", "str2"], + "arrayOfInteger":[1, 2147483647, -2147483648], + "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], + "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], + "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], + "arrayOfBoolean":[true, false, true], + "arrayOfNull":[null, null, null, null], + "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], + "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], + "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] + }""" :: Nil) + val complexFieldAndType2 = TestSQLContext.sparkContext.parallelize( """{"arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], @@ -137,6 +137,13 @@ object TestJsonData { ]] }""" :: Nil) + val nullsInArrays = + TestSQLContext.sparkContext.parallelize( + """{"field1":[[null], [[["Test"]]]]}""" :: + """{"field2":[null, [{"Test":1}]]}""" :: + """{"field3":[[null], [{"Test":"2"}]]}""" :: + """{"field4":[[null, [1,2,3]]]}""" :: Nil) + val jsonArray = TestSQLContext.sparkContext.parallelize( """[{"a":"str_a_1"}]""" :: From 974d7b238b415791975c60e4e202265d6fd31e2d Mon Sep 17 00:00:00 2001 From: ravipesala Date: Sun, 26 Oct 2014 16:36:11 -0700 Subject: [PATCH 1088/1492] [SPARK-3483][SQL] Special chars in column names Supporting special chars in column names by using back ticks. Closed https://github.com/apache/spark/pull/2804 and created this PR as it has merge conflicts Author: ravipesala Closes #2927 from ravipesala/SPARK-3483-NEW and squashes the following commits: f6329f3 [ravipesala] Rebased with master --- .../org/apache/spark/sql/catalyst/SparkSQLParser.scala | 2 ++ .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 8 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala index 04467342e6ab5..219322c015bbf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -75,6 +75,8 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { { case chars => StringLit(chars mkString "") } | '"' ~> chrExcept('"', '\n', EofCh).* <~ '"' ^^ { case chars => StringLit(chars mkString "") } + | '`' ~> chrExcept('`', '\n', EofCh).* <~ '`' ^^ + { case chars => Identifier(chars mkString "") } | EofCh ^^^ EOF | '\'' ~> failure("unclosed string literal") | '"' ~> failure("unclosed string literal") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 3959925a2e529..ffb504b94992f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -748,4 +748,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { """.stripMargin), (1 to 100).map(i => Seq(i, i, i))) } + + test("SPARK-3483 Special chars in column names") { + val data = sparkContext.parallelize(Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) + jsonRDD(data).registerTempTable("records") + sql("SELECT `key?number1` FROM records") + } } From ace41e8bf2f4538115b28b90757b8424ca614682 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:40:29 -0700 Subject: [PATCH 1089/1492] [SPARK-3959][SPARK-3960][SQL] SqlParser fails to parse literal -9223372036854775808 (Long.MinValue). / We can apply unary minus only to literal. SqlParser fails to parse -9223372036854775808 (Long.MinValue) so we cannot write queries such like as follows. SELECT value FROM someTable WHERE value > -9223372036854775808 Additionally, because of the wrong syntax definition, we cannot apply unary minus only to literal. So, we cannot write such expressions. -(value1 + value2) // Parenthesized expressions -column // Columns -MAX(column) // Functions Author: Kousuke Saruta Closes #2816 from sarutak/spark-sql-dsl-improvement2 and squashes the following commits: 32a5005 [Kousuke Saruta] Remove test setting for thriftserver c2bab5e [Kousuke Saruta] Fixed SPARK-3959 and SPARK-3960 --- .../apache/spark/sql/catalyst/SqlParser.scala | 68 +++++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 129 ++++++++++++++++++ .../org/apache/spark/sql/json/JsonSuite.scala | 13 +- 3 files changed, 189 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 9bea990fcee4e..4e967713ede64 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -303,33 +303,73 @@ class SqlParser extends AbstractSparkSQLParser { CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { case exp ~ t => Cast(exp, t) } protected lazy val literal: Parser[Literal] = - ( numericLit ^^ { - case i if i.toLong > Int.MaxValue => Literal(i.toLong) - case i => Literal(i.toInt) - } - | NULL ^^^ Literal(null, NullType) - | floatLit ^^ {case f => Literal(f.toDouble) } + ( numericLiteral + | booleanLiteral | stringLit ^^ {case s => Literal(s, StringType) } + | NULL ^^^ Literal(null, NullType) + ) + + protected lazy val booleanLiteral: Parser[Literal] = + ( TRUE ^^^ Literal(true, BooleanType) + | FALSE ^^^ Literal(false, BooleanType) + ) + + protected lazy val numericLiteral: Parser[Literal] = + signedNumericLiteral | unsignedNumericLiteral + + protected lazy val sign: Parser[String] = + "+" | "-" + + protected lazy val signedNumericLiteral: Parser[Literal] = + ( sign ~ numericLit ^^ { case s ~ l => Literal(toNarrowestIntegerType(s + l)) } + | sign ~ floatLit ^^ { case s ~ f => Literal((s + f).toDouble) } + ) + + protected lazy val unsignedNumericLiteral: Parser[Literal] = + ( numericLit ^^ { n => Literal(toNarrowestIntegerType(n)) } + | floatLit ^^ { f => Literal(f.toDouble) } ) + private val longMax = BigDecimal(s"${Long.MaxValue}") + private val longMin = BigDecimal(s"${Long.MinValue}") + private val intMax = BigDecimal(s"${Int.MaxValue}") + private val intMin = BigDecimal(s"${Int.MinValue}") + + private def toNarrowestIntegerType(value: String) = { + val bigIntValue = BigDecimal(value) + + bigIntValue match { + case v if v < longMin || v > longMax => v + case v if v < intMin || v > intMax => v.toLong + case v => v.toInt + } + } + protected lazy val floatLit: Parser[String] = - elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) + ( "." ~> unsignedNumericLiteral ^^ { u => "0." + u } + | elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) + ) + + protected lazy val baseExpression: Parser[Expression] = + ( "*" ^^^ Star(None) + | primary + ) - protected lazy val baseExpression: PackratParser[Expression] = - ( expression ~ ("[" ~> expression <~ "]") ^^ + protected lazy val signedPrimary: Parser[Expression] = + sign ~ primary ^^ { case s ~ e => if (s == "-") UnaryMinus(e) else e} + + protected lazy val primary: PackratParser[Expression] = + ( literal + | expression ~ ("[" ~> expression <~ "]") ^^ { case base ~ ordinal => GetItem(base, ordinal) } | (expression <~ ".") ~ ident ^^ { case base ~ fieldName => GetField(base, fieldName) } - | TRUE ^^^ Literal(true, BooleanType) - | FALSE ^^^ Literal(false, BooleanType) | cast | "(" ~> expression <~ ")" | function - | "-" ~> literal ^^ UnaryMinus | dotExpressionHeader | ident ^^ UnresolvedAttribute - | "*" ^^^ Star(None) - | literal + | signedPrimary ) protected lazy val dotExpressionHeader: Parser[Expression] = diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index ffb504b94992f..12e1cfc1cb7eb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -738,6 +738,135 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAggregation("SELECT key + 1 + 1, COUNT(*) FROM testData GROUP BY key + 1", false) } + test("Test to check we can use Long.MinValue") { + checkAnswer( + sql(s"SELECT ${Long.MinValue} FROM testData ORDER BY key LIMIT 1"), Long.MinValue + ) + + checkAnswer( + sql(s"SELECT key FROM testData WHERE key > ${Long.MinValue}"), (1 to 100).map(Row(_)).toSeq + ) + } + + test("Floating point number format") { + checkAnswer( + sql("SELECT 0.3"), 0.3 + ) + + checkAnswer( + sql("SELECT -0.8"), -0.8 + ) + + checkAnswer( + sql("SELECT .5"), 0.5 + ) + + checkAnswer( + sql("SELECT -.18"), -0.18 + ) + } + + test("Auto cast integer type") { + checkAnswer( + sql(s"SELECT ${Int.MaxValue + 1L}"), Int.MaxValue + 1L + ) + + checkAnswer( + sql(s"SELECT ${Int.MinValue - 1L}"), Int.MinValue - 1L + ) + + checkAnswer( + sql("SELECT 9223372036854775808"), BigDecimal("9223372036854775808") + ) + + checkAnswer( + sql("SELECT -9223372036854775809"), BigDecimal("-9223372036854775809") + ) + } + + test("Test to check we can apply sign to expression") { + + checkAnswer( + sql("SELECT -100"), -100 + ) + + checkAnswer( + sql("SELECT +230"), 230 + ) + + checkAnswer( + sql("SELECT -5.2"), -5.2 + ) + + checkAnswer( + sql("SELECT +6.8"), 6.8 + ) + + checkAnswer( + sql("SELECT -key FROM testData WHERE key = 2"), -2 + ) + + checkAnswer( + sql("SELECT +key FROM testData WHERE key = 3"), 3 + ) + + checkAnswer( + sql("SELECT -(key + 1) FROM testData WHERE key = 1"), -2 + ) + + checkAnswer( + sql("SELECT - key + 1 FROM testData WHERE key = 10"), -9 + ) + + checkAnswer( + sql("SELECT +(key + 5) FROM testData WHERE key = 5"), 10 + ) + + checkAnswer( + sql("SELECT -MAX(key) FROM testData"), -100 + ) + + checkAnswer( + sql("SELECT +MAX(key) FROM testData"), 100 + ) + + checkAnswer( + sql("SELECT - (-10)"), 10 + ) + + checkAnswer( + sql("SELECT + (-key) FROM testData WHERE key = 32"), -32 + ) + + checkAnswer( + sql("SELECT - (+Max(key)) FROM testData"), -100 + ) + + checkAnswer( + sql("SELECT - - 3"), 3 + ) + + checkAnswer( + sql("SELECT - + 20"), -20 + ) + + checkAnswer( + sql("SELEcT - + 45"), -45 + ) + + checkAnswer( + sql("SELECT + + 100"), 100 + ) + + checkAnswer( + sql("SELECT - - Max(key) FROM testData"), 100 + ) + + checkAnswer( + sql("SELECT + - key FROM testData WHERE key = 33"), -33 + ) + } + test("Multiple join") { checkAnswer( sql( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 41927e83698a5..1ae75546aada1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -380,6 +380,12 @@ class JsonSuite extends QueryTest { 92233720368547758071.2 ) + // Number and String conflict: resolve the type as number in this query. + checkAnswer( + sql("select num_str + 1.2 from jsonTable where num_str > 92233720368547758060"), + BigDecimal("92233720368547758061.2").toDouble + ) + // String and Boolean conflict: resolve the type as string. checkAnswer( sql("select * from jsonTable where str_bool = 'str1'"), @@ -415,13 +421,6 @@ class JsonSuite extends QueryTest { false ) - // Right now, we have a parsing error. - // Number and String conflict: resolve the type as number in this query. - checkAnswer( - sql("select num_str + 1.2 from jsonTable where num_str > 92233720368547758060"), - BigDecimal("92233720368547758061.2") - ) - // The plan of the following DSL is // Project [(CAST(num_str#65:4, DoubleType) + 1.2) AS num#78] // Filter (CAST(CAST(num_str#65:4, DoubleType), DecimalType) > 92233720368547758060) From 3a9d66cf59ab7c9aee090e4c6067c73510e2ac26 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Sun, 26 Oct 2014 16:54:07 -0700 Subject: [PATCH 1090/1492] [SPARK-4061][SQL] We cannot use EOL character in the operand of LIKE predicate. We cannot use EOL character like \n or \r in the operand of LIKE predicate. So following condition is never true. -- someStr is 'hoge\nfuga' where someStr LIKE 'hoge_fuga' Author: Kousuke Saruta Closes #2908 from sarutak/spark-sql-like-match-modification and squashes the following commits: d15798b [Kousuke Saruta] Remove test setting for thriftserver f99a2f4 [Kousuke Saruta] Fixed LIKE predicate so that we can use EOL character as in a operand --- .../expressions/stringOperations.scala | 42 +++++++++---------- .../ExpressionEvaluationSuite.scala | 6 +++ 2 files changed, 25 insertions(+), 23 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 c2a3a5ca3ca8b..f6349767764a3 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 @@ -102,31 +102,27 @@ case class Like(left: Expression, right: Expression) // replace the _ with .{1} exactly match 1 time of any character // replace the % with .*, match 0 or more times with any character - override def escape(v: String) = { - val sb = new StringBuilder() - var i = 0; - while (i < v.length) { - // Make a special case for "\\_" and "\\%" - val n = v.charAt(i); - if (n == '\\' && i + 1 < v.length && (v.charAt(i + 1) == '_' || v.charAt(i + 1) == '%')) { - sb.append(v.charAt(i + 1)) - i += 1 - } else { - if (n == '_') { - sb.append("."); - } else if (n == '%') { - sb.append(".*"); - } else { - sb.append(Pattern.quote(Character.toString(n))); - } - } - - i += 1 + override def escape(v: String) = + if (!v.isEmpty) { + "(?s)" + (' ' +: v.init).zip(v).flatMap { + case (prev, '\\') => "" + case ('\\', c) => + c match { + case '_' => "_" + case '%' => "%" + case _ => Pattern.quote("\\" + c) + } + case (prev, c) => + c match { + case '_' => "." + case '%' => ".*" + case _ => Pattern.quote(Character.toString(c)) + } + }.mkString + } else { + v } - sb.toString() - } - override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() } 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 f134d73450515..53c53481f984e 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 @@ -191,6 +191,9 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like "a%", true) checkEvaluation("abc" like "b%", false) checkEvaluation("abc" like "bc%", false) + checkEvaluation("a\nb" like "a_b", true) + checkEvaluation("ab" like "a%b", true) + checkEvaluation("a\nb" like "a%b", true) } test("LIKE Non-literal Regular Expression") { @@ -207,6 +210,9 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) + checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a_b"))) + checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) + checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) } From f4e8c289d899fc03ee3cebd01d810f207868b448 Mon Sep 17 00:00:00 2001 From: scwf Date: Sun, 26 Oct 2014 16:56:03 -0700 Subject: [PATCH 1091/1492] [SPARK-4042][SQL] Append columns ids and names before broadcast Append columns ids and names before broadcast ```hiveExtraConf``` in ```HadoopTableReader```. Author: scwf Closes #2885 from scwf/HadoopTableReader and squashes the following commits: a8c498c [scwf] append columns ids and names before broadcast --- .../org/apache/spark/sql/hive/execution/HiveTableScan.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index 85965a6ea095a..b7f3ade4ea819 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -68,6 +68,9 @@ case class HiveTableScan( @transient private[this] val hiveExtraConf = new HiveConf(context.hiveconf) + // append columns ids and names before broadcast + addColumnMetadataToConf(hiveExtraConf) + @transient private[this] val hadoopReader = new HadoopTableReader(attributes, relation, context, hiveExtraConf) @@ -105,8 +108,6 @@ case class HiveTableScan( hiveConf.set(serdeConstants.LIST_COLUMNS, relation.attributes.map(_.name).mkString(",")) } - addColumnMetadataToConf(hiveExtraConf) - /** * Prunes partitions not involve the query plan. * From 6377adaf3212b4facb4af644b70b7e99455cef48 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 26 Oct 2014 18:02:06 -0700 Subject: [PATCH 1092/1492] [SPARK-3970] Remove duplicate removal of local dirs The shutdown hook of `DiskBlockManager` would remove localDirs. So do not need to register them with `Utils.registerShutdownDeleteDir`. It causes duplicate removal of these local dirs and corresponding exceptions. Author: Liang-Chi Hsieh Closes #2826 from viirya/fix_duplicate_localdir_remove and squashes the following commits: 051d4b5 [Liang-Chi Hsieh] check dir existing and return empty List as default. 2b91a9c [Liang-Chi Hsieh] remove duplicate removal of local dirs. --- .../org/apache/spark/storage/DiskBlockManager.scala | 1 - .../src/main/scala/org/apache/spark/util/Utils.scala | 12 ++++++++---- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 6633a1db57e59..99e925328a4b9 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -149,7 +149,6 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon } private def addShutdownHook() { - localDirs.foreach(localDir => Utils.registerShutdownDeleteDir(localDir)) Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run(): Unit = Utils.logUncaughtExceptions { logDebug("Shutdown hook called") 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 e1dc49238733c..d722ee5a97e94 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -739,11 +739,15 @@ private[spark] object Utils extends Logging { } private def listFilesSafely(file: File): Seq[File] = { - val files = file.listFiles() - if (files == null) { - throw new IOException("Failed to list files for dir: " + file) + if (file.exists()) { + val files = file.listFiles() + if (files == null) { + throw new IOException("Failed to list files for dir: " + file) + } + files + } else { + List() } - files } /** From 9aa340a23fd7532f5e72c3352df92ce3e857fc80 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Mon, 27 Oct 2014 08:45:36 -0700 Subject: [PATCH 1093/1492] [SPARK-4030] Make destroy public for broadcast variables This change makes the destroy function public for broadcast variables. Motivation for the change is described in https://issues.apache.org/jira/browse/SPARK-4030. This patch also logs where destroy was called from if a broadcast variable is used after destruction. Author: Shivaram Venkataraman Closes #2922 from shivaram/broadcast-destroy and squashes the following commits: a11abab [Shivaram Venkataraman] Fix scala style in Utils.scala bed9c9d [Shivaram Venkataraman] Make destroy blocking by default e80c1ab [Shivaram Venkataraman] Make destroy public for broadcast variables Also log where destroy was called from if a broadcast variable is used after destruction. --- .../apache/spark/broadcast/Broadcast.scala | 22 +++++++++++++++++-- .../scala/org/apache/spark/util/Utils.scala | 3 ++- .../spark/broadcast/BroadcastSuite.scala | 20 ++++++++++++++++- 3 files changed, 41 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 15fd30e65761d..87f5cf944ed85 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -20,6 +20,8 @@ package org.apache.spark.broadcast import java.io.Serializable import org.apache.spark.SparkException +import org.apache.spark.Logging +import org.apache.spark.util.Utils import scala.reflect.ClassTag @@ -52,7 +54,7 @@ import scala.reflect.ClassTag * @param id A unique identifier for the broadcast variable. * @tparam T Type of the data contained in the broadcast variable. */ -abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { +abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Logging { /** * Flag signifying whether the broadcast variable is valid @@ -60,6 +62,8 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { */ @volatile private var _isValid = true + private var _destroySite = "" + /** Get the broadcasted value. */ def value: T = { assertValid() @@ -84,13 +88,26 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { doUnpersist(blocking) } + + /** + * Destroy all data and metadata related to this broadcast variable. Use this with caution; + * once a broadcast variable has been destroyed, it cannot be used again. + * This method blocks until destroy has completed + */ + def destroy() { + destroy(blocking = true) + } + /** * Destroy all data and metadata related to this broadcast variable. Use this with caution; * once a broadcast variable has been destroyed, it cannot be used again. + * @param blocking Whether to block until destroy has completed */ private[spark] def destroy(blocking: Boolean) { assertValid() _isValid = false + _destroySite = Utils.getCallSite().shortForm + logInfo("Destroying %s (from %s)".format(toString, _destroySite)) doDestroy(blocking) } @@ -124,7 +141,8 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable { /** Check if this broadcast is valid. If not valid, exception is thrown. */ protected def assertValid() { if (!_isValid) { - throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString)) + throw new SparkException( + "Attempted to use %s after it was destroyed (%s) ".format(toString, _destroySite)) } } 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 d722ee5a97e94..84ed5db8f0a53 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -992,7 +992,8 @@ private[spark] object Utils extends Logging { private def coreExclusionFunction(className: String): Boolean = { // A regular expression to match classes of the "core" Spark API that we want to skip when // finding the call site of a method. - val SPARK_CORE_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r + val SPARK_CORE_CLASS_REGEX = + """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?(\.broadcast)?\.[A-Z]""".r val SCALA_CLASS_REGEX = """^scala""".r val isSparkCoreClass = SPARK_CORE_CLASS_REGEX.findFirstIn(className).isDefined val isScalaClass = SCALA_CLASS_REGEX.findFirstIn(className).isDefined diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index e096c8c3e9b46..1014fd62d9a75 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.broadcast import scala.util.Random -import org.scalatest.FunSuite +import org.scalatest.{Assertions, FunSuite} import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} import org.apache.spark.io.SnappyCompressionCodec @@ -136,6 +136,12 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { test("Unpersisting TorrentBroadcast on executors and driver in distributed mode") { testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = true) } + + test("Using broadcast after destroy prints callsite") { + sc = new SparkContext("local", "test") + testPackage.runCallSiteTest(sc) + } + /** * Verify the persistence of state associated with an HttpBroadcast in either local mode or * local-cluster mode (when distributed = true). @@ -311,3 +317,15 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { conf } } + +package object testPackage extends Assertions { + + def runCallSiteTest(sc: SparkContext) { + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) + val broadcast = sc.broadcast(rdd) + broadcast.destroy() + val thrown = intercept[SparkException] { broadcast.value } + assert(thrown.getMessage.contains("BroadcastSuite.scala")) + } + +} From c9e05ca27c9c702b510d424e3befc87213f24e0f Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 27 Oct 2014 10:02:48 -0700 Subject: [PATCH 1094/1492] [SPARK-4032] Deprecate YARN alpha support in Spark 1.2 Author: Prashant Sharma Closes #2878 from ScrapCodes/SPARK-4032/deprecate-yarn-alpha and squashes the following commits: 17e9857 [Prashant Sharma] added deperecated comment to Client and ExecutorRunnable. 3a34b1e [Prashant Sharma] Updated docs... 4608dea [Prashant Sharma] [SPARK-4032] Deprecate YARN alpha support in Spark 1.2 --- docs/building-spark.md | 4 +++- .../org/apache/spark/deploy/yarn/Client.scala | 2 ++ .../spark/deploy/yarn/ExecutorRunnable.scala | 2 +- yarn/pom.xml | 22 +++++++++++++++++++ 4 files changed, 28 insertions(+), 2 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 11fd56c145c01..4cc0b1f2e5116 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -67,11 +67,13 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN - +
    DecimalType scala.math.sql.BigDecimal scala.math.BigDecimal DecimalType
    YARN versionProfile required
    0.23.x to 2.1.xyarn-alpha
    0.23.x to 2.1.xyarn-alpha (Deprecated.)
    2.2.x and lateryarn
    +Note: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445). + Examples: {% highlight bash %} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 9c66c785848a5..73b705ba50051 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -35,6 +35,7 @@ import org.apache.spark.deploy.SparkHadoopUtil /** * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. */ +@deprecated("use yarn/stable", "1.2.0") private[spark] class Client( val args: ClientArguments, val hadoopConf: Configuration, @@ -131,6 +132,7 @@ object Client { println("WARNING: This client is deprecated and will be removed in a " + "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") } + println("WARNING: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445)") // Set an env variable indicating we are running in YARN mode. // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 229b7a09f456b..7ee4b5c842df1 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.spark.{SecurityManager, SparkConf, Logging} - +@deprecated("use yarn/stable", "1.2.0") class ExecutorRunnable( container: Container, conf: Configuration, diff --git a/yarn/pom.xml b/yarn/pom.xml index 8a7035c85e9f1..2885e6607ec24 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -73,6 +73,28 @@ yarn-alpha + + + + maven-antrun-plugin + + + validate + + run + + + + ******************************************************************************************* + ***WARNING***: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445).* + ******************************************************************************************* + + + + + + + alpha From dea302ddbd26b1f20fb8a3979bd1d8e1717479f8 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 27 Oct 2014 10:04:24 -0700 Subject: [PATCH 1095/1492] SPARK-2621. Update task InputMetrics incrementally The patch takes advantage an API provided in Hadoop 2.5 that allows getting accurate data on Hadoop FileSystem bytes read. It eliminates the old method, which naively accepts the split size as the input bytes. An impact of this change will be that input metrics go away when using against Hadoop versions earlier thatn 2.5. I can add this back in, but my opinion is that no metrics are better than inaccurate metrics. This is difficult to write a test for because we don't usually build against a version of Hadoop that contains the function we need. I've tested it manually on a pseudo-distributed cluster. Author: Sandy Ryza Closes #2087 from sryza/sandy-spark-2621 and squashes the following commits: 23010b8 [Sandy Ryza] Missing style fixes 74fc9bb [Sandy Ryza] Make getFSBytesReadOnThreadCallback private 1ab662d [Sandy Ryza] Clear things up a bit 984631f [Sandy Ryza] Switch from pull to push model and add test 7ef7b22 [Sandy Ryza] Add missing curly braces 219abc9 [Sandy Ryza] Fall back to split size 90dbc14 [Sandy Ryza] SPARK-2621. Update task InputMetrics incrementally --- .../apache/spark/deploy/SparkHadoopUtil.scala | 30 +++++++++++ .../apache/spark/executor/TaskMetrics.scala | 1 - .../org/apache/spark/rdd/HadoopRDD.scala | 48 +++++++++++++---- .../org/apache/spark/rdd/NewHadoopRDD.scala | 48 +++++++++++++---- .../scala/org/apache/spark/util/Utils.scala | 11 ++++ .../spark/metrics/InputMetricsSuite.scala | 53 +++++++++++++++++++ 6 files changed, 170 insertions(+), 21 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index fe0ad9ebbca12..e28eaad8a5180 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -20,12 +20,15 @@ package org.apache.spark.deploy import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{Logging, SparkContext, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils import scala.collection.JavaConversions._ @@ -121,6 +124,33 @@ class SparkHadoopUtil extends Logging { UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) } + /** + * Returns a function that can be called to find Hadoop FileSystem bytes read. If + * getFSBytesReadOnThreadCallback is called from thread r at time t, the returned callback will + * return the bytes read on r since t. Reflection is required because thread-level FileSystem + * statistics are only available as of Hadoop 2.5 (see HADOOP-10688). + * Returns None if the required method can't be found. + */ + private[spark] def getFSBytesReadOnThreadCallback(path: Path, conf: Configuration) + : Option[() => Long] = { + val qualifiedPath = path.getFileSystem(conf).makeQualified(path) + val scheme = qualifiedPath.toUri().getScheme() + val stats = FileSystem.getAllStatistics().filter(_.getScheme().equals(scheme)) + try { + val threadStats = stats.map(Utils.invoke(classOf[Statistics], _, "getThreadStatistics")) + val statisticsDataClass = + Class.forName("org.apache.hadoop.fs.FileSystem$Statistics$StatisticsData") + val getBytesReadMethod = statisticsDataClass.getDeclaredMethod("getBytesRead") + val f = () => threadStats.map(getBytesReadMethod.invoke(_).asInstanceOf[Long]).sum + val baselineBytesRead = f() + Some(() => f() - baselineBytesRead) + } catch { + case e: NoSuchMethodException => { + logDebug("Couldn't find method for retrieving thread-level FileSystem input data", e) + None + } + } + } } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 3e49b6235aff3..57bc2b40cec44 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -169,7 +169,6 @@ case class InputMetrics(readMethod: DataReadMethod.Value) { var bytesRead: Long = 0L } - /** * :: DeveloperApi :: * Metrics pertaining to shuffle data read in a given task. diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 775141775e06c..946fb5616d3ec 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -46,7 +46,6 @@ import org.apache.spark.rdd.HadoopRDD.HadoopMapPartitionsWithSplitRDD import org.apache.spark.util.{NextIterator, Utils} import org.apache.spark.scheduler.{HostTaskLocation, HDFSCacheTaskLocation} - /** * A Spark split class that wraps around a Hadoop InputSplit. */ @@ -224,18 +223,18 @@ class HadoopRDD[K, V]( val key: K = reader.createKey() val value: V = reader.createValue() - // Set the task input metrics. val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - try { - /* bytesRead may not exactly equal the bytes read by a task: split boundaries aren't - * always at record boundaries, so tasks may need to read into other splits to complete - * a record. */ - inputMetrics.bytesRead = split.inputSplit.value.getLength() - } catch { - case e: java.io.IOException => - logWarning("Unable to get input size to set InputMetrics for task", e) + // Find a function that will return the FileSystem bytes read by this thread. + val bytesReadCallback = if (split.inputSplit.value.isInstanceOf[FileSplit]) { + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( + split.inputSplit.value.asInstanceOf[FileSplit].getPath, jobConf) + } else { + None + } + if (bytesReadCallback.isDefined) { + context.taskMetrics.inputMetrics = Some(inputMetrics) } - context.taskMetrics.inputMetrics = Some(inputMetrics) + var recordsSinceMetricsUpdate = 0 override def getNext() = { try { @@ -244,12 +243,36 @@ class HadoopRDD[K, V]( case eof: EOFException => finished = true } + + // Update bytes read metric every few records + if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES + && bytesReadCallback.isDefined) { + recordsSinceMetricsUpdate = 0 + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else { + recordsSinceMetricsUpdate += 1 + } (key, value) } override def close() { try { reader.close() + if (bytesReadCallback.isDefined) { + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else if (split.inputSplit.value.isInstanceOf[FileSplit]) { + // If we can't get the bytes read from the FS stats, fall back to the split size, + // which may be inaccurate. + try { + inputMetrics.bytesRead = split.inputSplit.value.getLength + context.taskMetrics.inputMetrics = Some(inputMetrics) + } catch { + case e: java.io.IOException => + logWarning("Unable to get input size to set InputMetrics for task", e) + } + } } catch { case e: Exception => { if (!Utils.inShutdown()) { @@ -302,6 +325,9 @@ private[spark] object HadoopRDD extends Logging { */ val CONFIGURATION_INSTANTIATION_LOCK = new Object() + /** Update the input bytes read metric each time this number of records has been read */ + val RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES = 256 + /** * The three methods below are helpers for accessing the local map, a property of the SparkEnv of * the local process. diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 0cccdefc5ee09..324563248793c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,6 +25,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.input.FileSplit import org.apache.spark.annotation.DeveloperApi import org.apache.spark.input.WholeTextFileInputFormat @@ -36,6 +37,7 @@ import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.executor.{DataReadMethod, InputMetrics} import org.apache.spark.rdd.NewHadoopRDD.NewHadoopMapPartitionsWithSplitRDD import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil private[spark] class NewHadoopPartition( rddId: Int, @@ -118,21 +120,22 @@ class NewHadoopRDD[K, V]( reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - try { - /* bytesRead may not exactly equal the bytes read by a task: split boundaries aren't - * always at record boundaries, so tasks may need to read into other splits to complete - * a record. */ - inputMetrics.bytesRead = split.serializableHadoopSplit.value.getLength() - } catch { - case e: Exception => - logWarning("Unable to get input split size in order to set task input bytes", e) + // Find a function that will return the FileSystem bytes read by this thread. + val bytesReadCallback = if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { + SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( + split.serializableHadoopSplit.value.asInstanceOf[FileSplit].getPath, conf) + } else { + None + } + if (bytesReadCallback.isDefined) { + context.taskMetrics.inputMetrics = Some(inputMetrics) } - context.taskMetrics.inputMetrics = Some(inputMetrics) // Register an on-task-completion callback to close the input stream. context.addTaskCompletionListener(context => close()) var havePair = false var finished = false + var recordsSinceMetricsUpdate = 0 override def hasNext: Boolean = { if (!finished && !havePair) { @@ -147,12 +150,39 @@ class NewHadoopRDD[K, V]( throw new java.util.NoSuchElementException("End of stream") } havePair = false + + // Update bytes read metric every few records + if (recordsSinceMetricsUpdate == HadoopRDD.RECORDS_BETWEEN_BYTES_READ_METRIC_UPDATES + && bytesReadCallback.isDefined) { + recordsSinceMetricsUpdate = 0 + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else { + recordsSinceMetricsUpdate += 1 + } + (reader.getCurrentKey, reader.getCurrentValue) } private def close() { try { reader.close() + + // Update metrics with final amount + if (bytesReadCallback.isDefined) { + val bytesReadFn = bytesReadCallback.get + inputMetrics.bytesRead = bytesReadFn() + } else if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { + // If we can't get the bytes read from the FS stats, fall back to the split size, + // which may be inaccurate. + try { + inputMetrics.bytesRead = split.serializableHadoopSplit.value.getLength + context.taskMetrics.inputMetrics = Some(inputMetrics) + } catch { + case e: java.io.IOException => + logWarning("Unable to get input size to set InputMetrics for task", e) + } + } } catch { case e: Exception => { if (!Utils.inShutdown()) { 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 84ed5db8f0a53..93ac9f1c33ea8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1673,6 +1673,17 @@ private[spark] object Utils extends Logging { PropertyConfigurator.configure(pro) } + def invoke( + clazz: Class[_], + obj: AnyRef, + methodName: String, + args: (Class[_], AnyRef)*): AnyRef = { + val (types, values) = args.unzip + val method = clazz.getDeclaredMethod(methodName, types: _*) + method.setAccessible(true) + method.invoke(obj, values.toSeq: _*) + } + } /** diff --git a/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala new file mode 100644 index 0000000000000..33bd1afea2470 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala @@ -0,0 +1,53 @@ +/* + * 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.metrics + +import org.scalatest.FunSuite + +import org.apache.spark.SharedSparkContext +import org.apache.spark.scheduler.{SparkListenerTaskEnd, SparkListener} + +import scala.collection.mutable.ArrayBuffer + +import java.io.{FileWriter, PrintWriter, File} + +class InputMetricsSuite extends FunSuite with SharedSparkContext { + test("input metrics when reading text file") { + val file = new File(getClass.getSimpleName + ".txt") + val pw = new PrintWriter(new FileWriter(file)) + pw.println("some stuff") + pw.println("some other stuff") + pw.println("yet more stuff") + pw.println("too much stuff") + pw.close() + file.deleteOnExit() + + val taskBytesRead = new ArrayBuffer[Long]() + sc.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead + } + }) + sc.textFile("file://" + file.getAbsolutePath, 2).count() + + // Wait for task end events to come in + sc.listenerBus.waitUntilEmpty(500) + assert(taskBytesRead.length == 2) + assert(taskBytesRead.sum == file.length()) + } +} From 1d7bcc88401d66c8d17a075355acfc25a8b7615c Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 27 Oct 2014 10:06:05 -0700 Subject: [PATCH 1096/1492] [SQL] Fixes caching related JoinSuite failure PR #2860 refines in-memory table statistics and enables broader broadcasted hash join optimization for in-memory tables. This makes `JoinSuite` fail when some test suite caches test table `testData` and gets executed before `JoinSuite`. Because expected `ShuffledHashJoin`s are optimized to `BroadcastedHashJoin` according to collected in-memory table statistics. This PR fixes this issue by clearing the cache before testing join operator selection. A separate test case is also added to test broadcasted hash join operator selection. Author: Cheng Lian Closes #2960 from liancheng/fix-join-suite and squashes the following commits: 715b2de [Cheng Lian] Fixes caching related JoinSuite failure --- .../org/apache/spark/sql/JoinSuite.scala | 117 ++++++++++-------- .../scala/org/apache/spark/sql/TestData.scala | 4 +- 2 files changed, 64 insertions(+), 57 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 07f4d2946c1b5..8b4cf5bac0187 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -19,17 +19,13 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.catalyst.plans.JoinType -import org.apache.spark.sql.catalyst.plans.{LeftOuter, RightOuter, FullOuter, Inner, LeftSemi} -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, RightOuter} import org.apache.spark.sql.execution.joins._ -import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ class JoinSuite extends QueryTest with BeforeAndAfterEach { - // Ensures tables are loaded. TestData @@ -41,54 +37,65 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { assert(planned.size === 1) } - test("join operator selection") { - def assertJoin(sqlString: String, c: Class[_]): Any = { - val rdd = sql(sqlString) - val physical = rdd.queryExecution.sparkPlan - val operators = physical.collect { - case j: ShuffledHashJoin => j - case j: HashOuterJoin => j - case j: LeftSemiJoinHash => j - case j: BroadcastHashJoin => j - case j: LeftSemiJoinBNL => j - case j: CartesianProduct => j - case j: BroadcastNestedLoopJoin => j - } - - assert(operators.size === 1) - if (operators(0).getClass() != c) { - fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") - } + def assertJoin(sqlString: String, c: Class[_]): Any = { + val rdd = sql(sqlString) + val physical = rdd.queryExecution.sparkPlan + val operators = physical.collect { + case j: ShuffledHashJoin => j + case j: HashOuterJoin => j + case j: LeftSemiJoinHash => j + case j: BroadcastHashJoin => j + case j: LeftSemiJoinBNL => j + case j: CartesianProduct => j + case j: BroadcastNestedLoopJoin => j + } + + assert(operators.size === 1) + if (operators(0).getClass() != c) { + fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") } + } - val cases1 = Seq( - ("SELECT * FROM testData left semi join testData2 ON key = a", classOf[LeftSemiJoinHash]), - ("SELECT * FROM testData left semi join testData2", classOf[LeftSemiJoinBNL]), - ("SELECT * FROM testData join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData left join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData right join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData full outer join testData2", classOf[CartesianProduct]), - ("SELECT * FROM testData left join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData right join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData full outer join testData2 where key=2", classOf[CartesianProduct]), - ("SELECT * FROM testData join testData2 where key>a", classOf[CartesianProduct]), - ("SELECT * FROM testData full outer join testData2 where key>a", classOf[CartesianProduct]), - ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData left join testData2 ON key = a", classOf[HashOuterJoin]), - ("SELECT * FROM testData right join testData2 ON key = a where key=2", + test("join operator selection") { + clearCache() + + Seq( + ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]), + ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]), + ("SELECT * FROM testData JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData LEFT JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData RIGHT JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData FULL OUTER JOIN testData2", classOf[CartesianProduct]), + ("SELECT * FROM testData LEFT JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData RIGHT JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key = 2", classOf[CartesianProduct]), + ("SELECT * FROM testData JOIN testData2 WHERE key > a", classOf[CartesianProduct]), + ("SELECT * FROM testData FULL OUTER JOIN testData2 WHERE key > a", classOf[CartesianProduct]), + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[ShuffledHashJoin]), + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", classOf[HashOuterJoin]), - ("SELECT * FROM testData right join testData2 ON key = a and key=2", + ("SELECT * FROM testData right join testData2 ON key = a and key = 2", classOf[HashOuterJoin]), - ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]), - ("SELECT * FROM testData join testData2 ON key = a", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a and key=2", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key=2", classOf[ShuffledHashJoin]) - // TODO add BroadcastNestedLoopJoin - ) - cases1.foreach { c => assertJoin(c._1, c._2) } + ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]) + // TODO add BroadcastNestedLoopJoin + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } + + test("broadcasted hash join operator selection") { + clearCache() + sql("CACHE TABLE testData") + + Seq( + ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key = 2", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key = 2", classOf[BroadcastHashJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + + sql("UNCACHE TABLE testData") } test("multiple-key equi-join is hash-join") { @@ -171,7 +178,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) - + checkAnswer( upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'n > 1)), (1, "A", null, null) :: @@ -180,7 +187,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) - + checkAnswer( upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'N > 1)), (1, "A", null, null) :: @@ -189,7 +196,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "d") :: (5, "E", null, null) :: (6, "F", null, null) :: Nil) - + checkAnswer( upperCaseData.join(lowerCaseData, LeftOuter, Some('n === 'N && 'l > 'L)), (1, "A", 1, "a") :: @@ -300,7 +307,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) - + checkAnswer( left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("left.N".attr !== 3))), (1, "A", null, null) :: @@ -310,7 +317,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { (4, "D", 4, "D") :: (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) - + checkAnswer( left.join(right, FullOuter, Some(("left.N".attr === "right.N".attr) && ("right.N".attr !== 3))), (1, "A", null, null) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 6c38575b13a2d..c4dd3e860f5fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -80,7 +80,7 @@ object TestData { UpperCaseData(3, "C") :: UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: - UpperCaseData(6, "F") :: Nil) + UpperCaseData(6, "F") :: Nil).toSchemaRDD upperCaseData.registerTempTable("upperCaseData") case class LowerCaseData(n: Int, l: String) @@ -89,7 +89,7 @@ object TestData { LowerCaseData(1, "a") :: LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: - LowerCaseData(4, "d") :: Nil) + LowerCaseData(4, "d") :: Nil).toSchemaRDD lowerCaseData.registerTempTable("lowerCaseData") case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) From bfa614b12795f1cfce4de0950f90cb8c4f2a7d53 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 27 Oct 2014 10:53:15 -0700 Subject: [PATCH 1097/1492] SPARK-4022 [CORE] [MLLIB] Replace colt dependency (LGPL) with commons-math This change replaces usages of colt with commons-math3 equivalents, and makes some minor necessary adjustments to related code and tests to match. Author: Sean Owen Closes #2928 from srowen/SPARK-4022 and squashes the following commits: 61a232f [Sean Owen] Fix failure due to different sampling in JavaAPISuite.sample() 16d66b8 [Sean Owen] Simplify seeding with call to reseedRandomGenerator a1a78e0 [Sean Owen] Use Well19937c 31c7641 [Sean Owen] Fix Python Poisson test by choosing a different seed; about 88% of seeds should work but 1 didn't, it seems 5c9c67f [Sean Owen] Additional test fixes from review d8f88e0 [Sean Owen] Replace colt with commons-math3. Some tests do not pass yet. --- LICENSE | 12 --- assembly/pom.xml | 4 + core/pom.xml | 6 -- .../apache/spark/partial/CountEvaluator.scala | 5 +- .../spark/partial/GroupedCountEvaluator.scala | 5 +- .../apache/spark/partial/MeanEvaluator.scala | 7 +- .../apache/spark/partial/StudentTCacher.scala | 9 +- .../apache/spark/partial/SumEvaluator.scala | 7 +- .../org/apache/spark/rdd/SampledRDD.scala | 9 +- .../spark/util/random/RandomSampler.scala | 10 +- .../util/random/StratifiedSamplingUtils.scala | 32 +++--- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../util/random/RandomSamplerSuite.scala | 9 +- examples/pom.xml | 8 ++ .../org/apache/spark/examples/LocalALS.scala | 97 ++++++++----------- .../org/apache/spark/examples/SparkALS.scala | 78 +++++++-------- mllib/pom.xml | 8 +- .../mllib/random/RandomDataGenerator.scala | 10 +- .../spark/mllib/stat/test/ChiSqTest.scala | 20 ++-- .../spark/mllib/tree/impl/BaggedPoint.scala | 8 +- .../spark/mllib/tree/RandomForestSuite.scala | 2 +- pom.xml | 6 -- python/pyspark/mllib/random.py | 2 +- 23 files changed, 175 insertions(+), 181 deletions(-) diff --git a/LICENSE b/LICENSE index a7eee041129cb..f1732fb47afc0 100644 --- a/LICENSE +++ b/LICENSE @@ -712,18 +712,6 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -======================================================================== -For colt: -======================================================================== - -Copyright (c) 1999 CERN - European Organization for Nuclear Research. -Permission to use, copy, modify, distribute and sell this software and its documentation for any purpose is hereby granted without fee, provided that the above copyright notice appear in all copies and that both that copyright notice and this permission notice appear in supporting documentation. CERN makes no representations about the suitability of this software for any purpose. It is provided "as is" without expressed or implied warranty. - -Packages hep.aida.* - -Written by Pavel Binko, Dino Ferrero Merlino, Wolfgang Hoschek, Tony Johnson, Andreas Pfeiffer, and others. Check the FreeHEP home page for more info. Permission to use and/or redistribute this work is granted under the terms of the LGPL License, with the exception that any usage related to military applications is expressly forbidden. The software and documentation made available under the terms of this license are provided with no warranty. - - ======================================================================== For SnapTree: ======================================================================== diff --git a/assembly/pom.xml b/assembly/pom.xml index bfef95b8deb95..11d4bea9361ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -146,6 +146,10 @@ com/google/common/base/Present* + + org.apache.commons.math3 + org.spark-project.commons.math3 + diff --git a/core/pom.xml b/core/pom.xml index 320d1076f7c03..5cd21e18e8ca7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -85,8 +85,6 @@ org.apache.commons commons-math3 - 3.3 - test com.google.code.findbugs @@ -162,10 +160,6 @@ json4s-jackson_${scala.binary.version} 3.2.10 - - colt - colt - org.apache.mesos mesos diff --git a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala index 3155dfe165664..637492a97551b 100644 --- a/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/CountEvaluator.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.NormalDistribution /** * An ApproximateEvaluator for counts. @@ -46,7 +46,8 @@ private[spark] class CountEvaluator(totalOutputs: Int, confidence: Double) val mean = (sum + 1 - p) / p val variance = (sum + 1) * (1 - p) / (p * p) val stdev = math.sqrt(variance) - val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) + val confFactor = new NormalDistribution(). + inverseCumulativeProbability(1 - (1 - confidence) / 2) val low = mean - confFactor * stdev val high = mean + confFactor * stdev new BoundedDouble(mean, confidence, low, high) diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 8bb78123e3c9c..3ef3cc219dec6 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -24,7 +24,7 @@ import scala.collection.Map import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.NormalDistribution import org.apache.spark.util.collection.OpenHashMap @@ -55,7 +55,8 @@ private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, conf new HashMap[T, BoundedDouble] } else { val p = outputsMerged.toDouble / totalOutputs - val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) + val confFactor = new NormalDistribution(). + inverseCumulativeProbability(1 - (1 - confidence) / 2) val result = new JHashMap[T, BoundedDouble](sums.size) sums.foreach { case (key, sum) => val mean = (sum + 1 - p) / p diff --git a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala index d24959cba8727..787a21a61fdcf 100644 --- a/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/MeanEvaluator.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.{NormalDistribution, TDistribution} import org.apache.spark.util.StatCounter @@ -45,9 +45,10 @@ private[spark] class MeanEvaluator(totalOutputs: Int, confidence: Double) val stdev = math.sqrt(counter.sampleVariance / counter.count) val confFactor = { if (counter.count > 100) { - Probability.normalInverse(1 - (1 - confidence) / 2) + new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) } else { - Probability.studentTInverse(1 - confidence, (counter.count - 1).toInt) + val degreesOfFreedom = (counter.count - 1).toInt + new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) } } val low = mean - confFactor * stdev diff --git a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala index 92915ee66d29f..828bf96c2c0bd 100644 --- a/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala +++ b/core/src/main/scala/org/apache/spark/partial/StudentTCacher.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution} /** * A utility class for caching Student's T distribution values for a given confidence level @@ -25,8 +25,10 @@ import cern.jet.stat.Probability * confidence intervals for many keys. */ private[spark] class StudentTCacher(confidence: Double) { + val NORMAL_APPROX_SAMPLE_SIZE = 100 // For samples bigger than this, use Gaussian approximation - val normalApprox = Probability.normalInverse(1 - (1 - confidence) / 2) + + val normalApprox = new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) val cache = Array.fill[Double](NORMAL_APPROX_SAMPLE_SIZE)(-1.0) def get(sampleSize: Long): Double = { @@ -35,7 +37,8 @@ private[spark] class StudentTCacher(confidence: Double) { } else { val size = sampleSize.toInt if (cache(size) < 0) { - cache(size) = Probability.studentTInverse(1 - confidence, size - 1) + val tDist = new TDistribution(size - 1) + cache(size) = tDist.inverseCumulativeProbability(1 - (1 - confidence) / 2) } cache(size) } diff --git a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala index d5336284571d2..1753c2561b678 100644 --- a/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/SumEvaluator.scala @@ -17,7 +17,7 @@ package org.apache.spark.partial -import cern.jet.stat.Probability +import org.apache.commons.math3.distribution.{TDistribution, NormalDistribution} import org.apache.spark.util.StatCounter @@ -55,9 +55,10 @@ private[spark] class SumEvaluator(totalOutputs: Int, confidence: Double) val sumStdev = math.sqrt(sumVar) val confFactor = { if (counter.count > 100) { - Probability.normalInverse(1 - (1 - confidence) / 2) + new NormalDistribution().inverseCumulativeProbability(1 - (1 - confidence) / 2) } else { - Probability.studentTInverse(1 - confidence, (counter.count - 1).toInt) + val degreesOfFreedom = (counter.count - 1).toInt + new TDistribution(degreesOfFreedom).inverseCumulativeProbability(1 - (1 - confidence) / 2) } } val low = sumEstimate - confFactor * sumStdev diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index b097c30f8c231..9e8cee5331cf8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -21,8 +21,7 @@ import java.util.Random import scala.reflect.ClassTag -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.{Partition, TaskContext} @@ -53,9 +52,11 @@ private[spark] class SampledRDD[T: ClassTag]( if (withReplacement) { // For large datasets, the expected number of occurrences of each element in a sample with // replacement is Poisson(frac). We use that to get a count for each element. - val poisson = new Poisson(frac, new DRand(split.seed)) + val poisson = new PoissonDistribution(frac) + poisson.reseedRandomGenerator(split.seed) + firstParent[T].iterator(split.prev, context).flatMap { element => - val count = poisson.nextInt() + val count = poisson.sample() if (count == 0) { Iterator.empty // Avoid object allocation when we return 0 items, which is quite often } else { diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 32c5fdad75e58..ee389def20c8c 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -19,8 +19,7 @@ package org.apache.spark.util.random import java.util.Random -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.annotation.DeveloperApi @@ -87,15 +86,16 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) @DeveloperApi class PoissonSampler[T](mean: Double) extends RandomSampler[T, T] { - private[random] var rng = new Poisson(mean, new DRand) + private[random] var rng = new PoissonDistribution(mean) override def setSeed(seed: Long) { - rng = new Poisson(mean, new DRand(seed.toInt)) + rng = new PoissonDistribution(mean) + rng.reseedRandomGenerator(seed) } override def sample(items: Iterator[T]): Iterator[T] = { items.flatMap { item => - val count = rng.nextInt() + val count = rng.sample() if (count == 0) { Iterator.empty } else { diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 8f95d7c6b799b..4fa357edd6f07 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -22,8 +22,7 @@ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.Logging import org.apache.spark.SparkContext._ @@ -209,7 +208,7 @@ private[spark] object StratifiedSamplingUtils extends Logging { samplingRateByKey = computeThresholdByKey(finalResult, fractions) } (idx: Int, iter: Iterator[(K, V)]) => { - val rng = new RandomDataGenerator + val rng = new RandomDataGenerator() rng.reSeed(seed + idx) // Must use the same invoke pattern on the rng as in getSeqOp for without replacement // in order to generate the same sequence of random numbers when creating the sample @@ -245,9 +244,9 @@ private[spark] object StratifiedSamplingUtils extends Logging { // Must use the same invoke pattern on the rng as in getSeqOp for with replacement // in order to generate the same sequence of random numbers when creating the sample val copiesAccepted = if (acceptBound == 0) 0L else rng.nextPoisson(acceptBound) - val copiesWailisted = rng.nextPoisson(finalResult(key).waitListBound) + val copiesWaitlisted = rng.nextPoisson(finalResult(key).waitListBound) val copiesInSample = copiesAccepted + - (0 until copiesWailisted).count(i => rng.nextUniform() < thresholdByKey(key)) + (0 until copiesWaitlisted).count(i => rng.nextUniform() < thresholdByKey(key)) if (copiesInSample > 0) { Iterator.fill(copiesInSample.toInt)(item) } else { @@ -261,10 +260,10 @@ private[spark] object StratifiedSamplingUtils extends Logging { rng.reSeed(seed + idx) iter.flatMap { item => val count = rng.nextPoisson(fractions(item._1)) - if (count > 0) { - Iterator.fill(count)(item) - } else { + if (count == 0) { Iterator.empty + } else { + Iterator.fill(count)(item) } } } @@ -274,15 +273,24 @@ private[spark] object StratifiedSamplingUtils extends Logging { /** A random data generator that generates both uniform values and Poisson values. */ private class RandomDataGenerator { val uniform = new XORShiftRandom() - var poisson = new Poisson(1.0, new DRand) + // commons-math3 doesn't have a method to generate Poisson from an arbitrary mean; + // maintain a cache of Poisson(m) distributions for various m + val poissonCache = mutable.Map[Double, PoissonDistribution]() + var poissonSeed = 0L - def reSeed(seed: Long) { + def reSeed(seed: Long): Unit = { uniform.setSeed(seed) - poisson = new Poisson(1.0, new DRand(seed.toInt)) + poissonSeed = seed + poissonCache.clear() } def nextPoisson(mean: Double): Int = { - poisson.nextInt(mean) + val poisson = poissonCache.getOrElseUpdate(mean, { + val newPoisson = new PoissonDistribution(mean) + newPoisson.reseedRandomGenerator(poissonSeed) + newPoisson + }) + poisson.sample() } def nextUniform(): Double = { diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 814e40c4f77cc..0172876a264b8 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -142,7 +142,7 @@ public void sample() { JavaRDD rdd = sc.parallelize(ints); JavaRDD sample20 = rdd.sample(true, 0.2, 11); // expected 2 but of course result varies randomly a bit - Assert.assertEquals(3, sample20.count()); + Assert.assertEquals(1, sample20.count()); JavaRDD sample20NoReplacement = rdd.sample(false, 0.2, 11); Assert.assertEquals(2, sample20NoReplacement.count()); } diff --git a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala index 36877476e708e..ba67d766a775a 100644 --- a/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/RandomSamplerSuite.scala @@ -19,7 +19,8 @@ package org.apache.spark.util.random import java.util.Random -import cern.jet.random.Poisson +import org.apache.commons.math3.distribution.PoissonDistribution + import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar @@ -28,11 +29,11 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar val a = List(1, 2, 3, 4, 5, 6, 7, 8, 9) var random: Random = _ - var poisson: Poisson = _ + var poisson: PoissonDistribution = _ before { random = mock[Random] - poisson = mock[Poisson] + poisson = mock[PoissonDistribution] } test("BernoulliSamplerWithRange") { @@ -101,7 +102,7 @@ class RandomSamplerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar test("PoissonSampler") { expecting { for(x <- Seq(0, 1, 2, 0, 1, 1, 0, 0, 0)) { - poisson.nextInt().andReturn(x) + poisson.sample().andReturn(x) } } whenExecuting(poisson) { diff --git a/examples/pom.xml b/examples/pom.xml index eb49a0e5af22d..bc3291803c324 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -156,6 +156,10 @@ algebird-core_${scala.binary.version} 0.1.11 + + org.apache.commons + commons-math3 + org.scalatest scalatest_${scala.binary.version} @@ -268,6 +272,10 @@ com.google.common.base.Optional** + + org.apache.commons.math3 + org.spark-project.commons.math3 + diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 1f576319b3ca8..3d5259463003d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -17,11 +17,7 @@ package org.apache.spark.examples -import scala.math.sqrt - -import cern.colt.matrix._ -import cern.colt.matrix.linalg._ -import cern.jet.math._ +import org.apache.commons.math3.linear._ /** * Alternating least squares matrix factorization. @@ -30,84 +26,70 @@ import cern.jet.math._ * please refer to org.apache.spark.mllib.recommendation.ALS */ object LocalALS { + // Parameters set through command line arguments var M = 0 // Number of movies var U = 0 // Number of users var F = 0 // Number of features var ITERATIONS = 0 - val LAMBDA = 0.01 // Regularization coefficient - // Some COLT objects - val factory2D = DoubleFactory2D.dense - val factory1D = DoubleFactory1D.dense - val algebra = Algebra.DEFAULT - val blas = SeqBlas.seqBlas - - def generateR(): DoubleMatrix2D = { - val mh = factory2D.random(M, F) - val uh = factory2D.random(U, F) - algebra.mult(mh, algebra.transpose(uh)) + def generateR(): RealMatrix = { + val mh = randomMatrix(M, F) + val uh = randomMatrix(U, F) + mh.multiply(uh.transpose()) } - def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D], - us: Array[DoubleMatrix1D]): Double = - { - val r = factory2D.make(M, U) + def rmse(targetR: RealMatrix, ms: Array[RealVector], us: Array[RealVector]): Double = { + val r = new Array2DRowRealMatrix(M, U) for (i <- 0 until M; j <- 0 until U) { - r.set(i, j, blas.ddot(ms(i), us(j))) + r.setEntry(i, j, ms(i).dotProduct(us(j))) } - blas.daxpy(-1, targetR, r) - val sumSqs = r.aggregate(Functions.plus, Functions.square) - sqrt(sumSqs / (M * U)) + val diffs = r.subtract(targetR) + var sumSqs = 0.0 + for (i <- 0 until M; j <- 0 until U) { + val diff = diffs.getEntry(i, j) + sumSqs += diff * diff + } + math.sqrt(sumSqs / (M.toDouble * U.toDouble)) } - def updateMovie(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D], - R: DoubleMatrix2D) : DoubleMatrix1D = - { - val XtX = factory2D.make(F, F) - val Xty = factory1D.make(F) + def updateMovie(i: Int, m: RealVector, us: Array[RealVector], R: RealMatrix) : RealVector = { + var XtX: RealMatrix = new Array2DRowRealMatrix(F, F) + var Xty: RealVector = new ArrayRealVector(F) // For each user that rated the movie for (j <- 0 until U) { val u = us(j) // Add u * u^t to XtX - blas.dger(1, u, u, XtX) + XtX = XtX.add(u.outerProduct(u)) // Add u * rating to Xty - blas.daxpy(R.get(i, j), u, Xty) + Xty = Xty.add(u.mapMultiply(R.getEntry(i, j))) } - // Add regularization coefs to diagonal terms + // Add regularization coefficients to diagonal terms for (d <- 0 until F) { - XtX.set(d, d, XtX.get(d, d) + LAMBDA * U) + XtX.addToEntry(d, d, LAMBDA * U) } // Solve it with Cholesky - val ch = new CholeskyDecomposition(XtX) - val Xty2D = factory2D.make(Xty.toArray, F) - val solved2D = ch.solve(Xty2D) - solved2D.viewColumn(0) + new CholeskyDecomposition(XtX).getSolver.solve(Xty) } - def updateUser(j: Int, u: DoubleMatrix1D, ms: Array[DoubleMatrix1D], - R: DoubleMatrix2D) : DoubleMatrix1D = - { - val XtX = factory2D.make(F, F) - val Xty = factory1D.make(F) + def updateUser(j: Int, u: RealVector, ms: Array[RealVector], R: RealMatrix) : RealVector = { + var XtX: RealMatrix = new Array2DRowRealMatrix(F, F) + var Xty: RealVector = new ArrayRealVector(F) // For each movie that the user rated for (i <- 0 until M) { val m = ms(i) // Add m * m^t to XtX - blas.dger(1, m, m, XtX) + XtX = XtX.add(m.outerProduct(m)) // Add m * rating to Xty - blas.daxpy(R.get(i, j), m, Xty) + Xty = Xty.add(m.mapMultiply(R.getEntry(i, j))) } - // Add regularization coefs to diagonal terms + // Add regularization coefficients to diagonal terms for (d <- 0 until F) { - XtX.set(d, d, XtX.get(d, d) + LAMBDA * M) + XtX.addToEntry(d, d, LAMBDA * M) } // Solve it with Cholesky - val ch = new CholeskyDecomposition(XtX) - val Xty2D = factory2D.make(Xty.toArray, F) - val solved2D = ch.solve(Xty2D) - solved2D.viewColumn(0) + new CholeskyDecomposition(XtX).getSolver.solve(Xty) } def showWarning() { @@ -135,21 +117,28 @@ object LocalALS { showWarning() - printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) + println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS") val R = generateR() // Initialize m and u randomly - var ms = Array.fill(M)(factory1D.random(F)) - var us = Array.fill(U)(factory1D.random(F)) + var ms = Array.fill(M)(randomVector(F)) + var us = Array.fill(U)(randomVector(F)) // Iteratively update movies then users for (iter <- 1 to ITERATIONS) { - println("Iteration " + iter + ":") + println(s"Iteration $iter:") ms = (0 until M).map(i => updateMovie(i, ms(i), us, R)).toArray us = (0 until U).map(j => updateUser(j, us(j), ms, R)).toArray println("RMSE = " + rmse(R, ms, us)) println() } } + + private def randomVector(n: Int): RealVector = + new ArrayRealVector(Array.fill(n)(math.random)) + + private def randomMatrix(rows: Int, cols: Int): RealMatrix = + new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random)) + } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index fde8ffeedf8b4..6c0ac8013ce34 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -17,11 +17,7 @@ package org.apache.spark.examples -import scala.math.sqrt - -import cern.colt.matrix._ -import cern.colt.matrix.linalg._ -import cern.jet.math._ +import org.apache.commons.math3.linear._ import org.apache.spark._ @@ -32,62 +28,53 @@ import org.apache.spark._ * please refer to org.apache.spark.mllib.recommendation.ALS */ object SparkALS { + // Parameters set through command line arguments var M = 0 // Number of movies var U = 0 // Number of users var F = 0 // Number of features var ITERATIONS = 0 - val LAMBDA = 0.01 // Regularization coefficient - // Some COLT objects - val factory2D = DoubleFactory2D.dense - val factory1D = DoubleFactory1D.dense - val algebra = Algebra.DEFAULT - val blas = SeqBlas.seqBlas - - def generateR(): DoubleMatrix2D = { - val mh = factory2D.random(M, F) - val uh = factory2D.random(U, F) - algebra.mult(mh, algebra.transpose(uh)) + def generateR(): RealMatrix = { + val mh = randomMatrix(M, F) + val uh = randomMatrix(U, F) + mh.multiply(uh.transpose()) } - def rmse(targetR: DoubleMatrix2D, ms: Array[DoubleMatrix1D], - us: Array[DoubleMatrix1D]): Double = - { - val r = factory2D.make(M, U) + def rmse(targetR: RealMatrix, ms: Array[RealVector], us: Array[RealVector]): Double = { + val r = new Array2DRowRealMatrix(M, U) for (i <- 0 until M; j <- 0 until U) { - r.set(i, j, blas.ddot(ms(i), us(j))) + r.setEntry(i, j, ms(i).dotProduct(us(j))) } - blas.daxpy(-1, targetR, r) - val sumSqs = r.aggregate(Functions.plus, Functions.square) - sqrt(sumSqs / (M * U)) + val diffs = r.subtract(targetR) + var sumSqs = 0.0 + for (i <- 0 until M; j <- 0 until U) { + val diff = diffs.getEntry(i, j) + sumSqs += diff * diff + } + math.sqrt(sumSqs / (M.toDouble * U.toDouble)) } - def update(i: Int, m: DoubleMatrix1D, us: Array[DoubleMatrix1D], - R: DoubleMatrix2D) : DoubleMatrix1D = - { + def update(i: Int, m: RealVector, us: Array[RealVector], R: RealMatrix) : RealVector = { val U = us.size - val F = us(0).size - val XtX = factory2D.make(F, F) - val Xty = factory1D.make(F) + val F = us(0).getDimension + var XtX: RealMatrix = new Array2DRowRealMatrix(F, F) + var Xty: RealVector = new ArrayRealVector(F) // For each user that rated the movie for (j <- 0 until U) { val u = us(j) // Add u * u^t to XtX - blas.dger(1, u, u, XtX) + XtX = XtX.add(u.outerProduct(u)) // Add u * rating to Xty - blas.daxpy(R.get(i, j), u, Xty) + Xty = Xty.add(u.mapMultiply(R.getEntry(i, j))) } // Add regularization coefs to diagonal terms for (d <- 0 until F) { - XtX.set(d, d, XtX.get(d, d) + LAMBDA * U) + XtX.addToEntry(d, d, LAMBDA * U) } // Solve it with Cholesky - val ch = new CholeskyDecomposition(XtX) - val Xty2D = factory2D.make(Xty.toArray, F) - val solved2D = ch.solve(Xty2D) - solved2D.viewColumn(0) + new CholeskyDecomposition(XtX).getSolver.solve(Xty) } def showWarning() { @@ -118,7 +105,7 @@ object SparkALS { showWarning() - printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) + println(s"Running with M=$M, U=$U, F=$F, iters=$ITERATIONS") val sparkConf = new SparkConf().setAppName("SparkALS") val sc = new SparkContext(sparkConf) @@ -126,21 +113,21 @@ object SparkALS { val R = generateR() // Initialize m and u randomly - var ms = Array.fill(M)(factory1D.random(F)) - var us = Array.fill(U)(factory1D.random(F)) + var ms = Array.fill(M)(randomVector(F)) + var us = Array.fill(U)(randomVector(F)) // Iteratively update movies then users val Rc = sc.broadcast(R) var msb = sc.broadcast(ms) var usb = sc.broadcast(us) for (iter <- 1 to ITERATIONS) { - println("Iteration " + iter + ":") + println(s"Iteration $iter:") ms = sc.parallelize(0 until M, slices) .map(i => update(i, msb.value(i), usb.value, Rc.value)) .collect() msb = sc.broadcast(ms) // Re-broadcast ms because it was updated us = sc.parallelize(0 until U, slices) - .map(i => update(i, usb.value(i), msb.value, algebra.transpose(Rc.value))) + .map(i => update(i, usb.value(i), msb.value, Rc.value.transpose())) .collect() usb = sc.broadcast(us) // Re-broadcast us because it was updated println("RMSE = " + rmse(R, ms, us)) @@ -149,4 +136,11 @@ object SparkALS { sc.stop() } + + private def randomVector(n: Int): RealVector = + new ArrayRealVector(Array.fill(n)(math.random)) + + private def randomMatrix(rows: Int, cols: Int): RealMatrix = + new Array2DRowRealMatrix(Array.fill(rows, cols)(math.random)) + } diff --git a/mllib/pom.xml b/mllib/pom.xml index 696e9396f627c..de062a4901596 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -65,12 +65,12 @@ junit junit - - org.apache.commons - commons-math3 - + + org.apache.commons + commons-math3 + org.scalatest scalatest_${scala.binary.version} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala index 28179fbc450c0..51f9b8657c640 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/random/RandomDataGenerator.scala @@ -17,8 +17,7 @@ package org.apache.spark.mllib.random -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.random.{XORShiftRandom, Pseudorandom} @@ -89,12 +88,13 @@ class StandardNormalGenerator extends RandomDataGenerator[Double] { @DeveloperApi class PoissonGenerator(val mean: Double) extends RandomDataGenerator[Double] { - private var rng = new Poisson(mean, new DRand) + private var rng = new PoissonDistribution(mean) - override def nextValue(): Double = rng.nextDouble() + override def nextValue(): Double = rng.sample() override def setSeed(seed: Long) { - rng = new Poisson(mean, new DRand(seed.toInt)) + rng = new PoissonDistribution(mean) + rng.reseedRandomGenerator(seed) } override def copy(): PoissonGenerator = new PoissonGenerator(mean) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala index 0089419c2c5d4..ea82d39b72c03 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/ChiSqTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.stat.test import breeze.linalg.{DenseMatrix => BDM} -import cern.jet.stat.Probability.chiSquareComplemented +import org.apache.commons.math3.distribution.ChiSquaredDistribution import org.apache.spark.{SparkException, Logging} import org.apache.spark.mllib.linalg.{Matrices, Matrix, Vector, Vectors} @@ -33,7 +33,7 @@ import scala.collection.mutable * on an input of type `Matrix` in which independence between columns is assessed. * We also provide a method for computing the chi-squared statistic between each feature and the * label for an input `RDD[LabeledPoint]`, return an `Array[ChiSquaredTestResult]` of size = - * number of features in the inpuy RDD. + * number of features in the input RDD. * * Supported methods for goodness of fit: `pearson` (default) * Supported methods for independence: `pearson` (default) @@ -139,7 +139,7 @@ private[stat] object ChiSqTest extends Logging { } /* - * Pearon's goodness of fit test on the input observed and expected counts/relative frequencies. + * Pearson's goodness of fit test on the input observed and expected counts/relative frequencies. * Uniform distribution is assumed when `expected` is not passed in. */ def chiSquared(observed: Vector, @@ -188,12 +188,12 @@ private[stat] object ChiSqTest extends Logging { } } val df = size - 1 - val pValue = chiSquareComplemented(df, statistic) + val pValue = 1.0 - new ChiSquaredDistribution(df).cumulativeProbability(statistic) new ChiSqTestResult(pValue, df, statistic, PEARSON.name, NullHypothesis.goodnessOfFit.toString) } /* - * Pearon's independence test on the input contingency matrix. + * Pearson's independence test on the input contingency matrix. * TODO: optimize for SparseMatrix when it becomes supported. */ def chiSquaredMatrix(counts: Matrix, methodName:String = PEARSON.name): ChiSqTestResult = { @@ -238,7 +238,13 @@ private[stat] object ChiSqTest extends Logging { j += 1 } val df = (numCols - 1) * (numRows - 1) - val pValue = chiSquareComplemented(df, statistic) - new ChiSqTestResult(pValue, df, statistic, methodName, NullHypothesis.independence.toString) + if (df == 0) { + // 1 column or 1 row. Constant distribution is independent of anything. + // pValue = 1.0 and statistic = 0.0 in this case. + new ChiSqTestResult(1.0, 0, 0.0, methodName, NullHypothesis.independence.toString) + } else { + val pValue = 1.0 - new ChiSquaredDistribution(df).cumulativeProbability(statistic) + new ChiSqTestResult(pValue, df, statistic, methodName, NullHypothesis.independence.toString) + } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala index 937c8a2ac5836..e7a2127c5d2e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala @@ -17,8 +17,7 @@ package org.apache.spark.mllib.tree.impl -import cern.jet.random.Poisson -import cern.jet.random.engine.DRand +import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -60,12 +59,13 @@ private[tree] object BaggedPoint { input.mapPartitionsWithIndex { (partitionIndex, instances) => // TODO: Support different sampling rates, and sampling without replacement. // Use random seed = seed + partitionIndex + 1 to make generation reproducible. - val poisson = new Poisson(1.0, new DRand(seed + partitionIndex + 1)) + val poisson = new PoissonDistribution(1.0) + poisson.reseedRandomGenerator(seed + partitionIndex + 1) instances.map { instance => val subsampleWeights = new Array[Double](numSubsamples) var subsampleIndex = 0 while (subsampleIndex < numSubsamples) { - subsampleWeights(subsampleIndex) = poisson.nextInt() + subsampleWeights(subsampleIndex) = poisson.sample() subsampleIndex += 1 } new BaggedPoint(instance, subsampleWeights) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 6b13765b98f41..d3eff59aa0409 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -187,7 +187,7 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) val model = RandomForest.trainClassifier(input, strategy, numTrees = 2, featureSubsetStrategy = "sqrt", seed = 12345) - RandomForestSuite.validateClassifier(model, arr, 1.0) + RandomForestSuite.validateClassifier(model, arr, 0.0) } } diff --git a/pom.xml b/pom.xml index 2ebe1b8da588a..abcb97108c5d9 100644 --- a/pom.xml +++ b/pom.xml @@ -305,7 +305,6 @@ org.apache.commons commons-math3 3.3 - test com.google.code.findbugs @@ -431,11 +430,6 @@ akka-testkit_${scala.binary.version} ${akka.version} - - colt - colt - 1.2.0 - org.apache.mesos mesos diff --git a/python/pyspark/mllib/random.py b/python/pyspark/mllib/random.py index 73baba4ace5f6..2202c51ab9c06 100644 --- a/python/pyspark/mllib/random.py +++ b/python/pyspark/mllib/random.py @@ -107,7 +107,7 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): distribution with the input mean. >>> mean = 100.0 - >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=1L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2L) >>> stats = x.stats() >>> stats.count() 1000L From 7e3a1ada86e6adf1ddd4d8a321824daf5f3b2c75 Mon Sep 17 00:00:00 2001 From: coderxiang Date: Mon, 27 Oct 2014 19:43:39 -0700 Subject: [PATCH 1098/1492] [MLlib] SPARK-3987: add test case on objective value for NNLS Also update step parameter to pass the proposed test Author: coderxiang Closes #2965 from coderxiang/nnls-test and squashes the following commits: 24b06f9 [coderxiang] add test case on objective value for NNLS; update step parameter to pass the test --- .../spark/mllib/optimization/NNLS.scala | 2 +- .../spark/mllib/optimization/NNLSSuite.scala | 30 +++++++++++++++++++ 2 files changed, 31 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala index e4b436b023794..fef062e02b6ec 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/NNLS.scala @@ -79,7 +79,7 @@ private[mllib] object NNLS { // stopping condition def stop(step: Double, ndir: Double, nx: Double): Boolean = { ((step.isNaN) // NaN - || (step < 1e-6) // too small or negative + || (step < 1e-7) // too small or negative || (step > 1e40) // too small; almost certainly numerical problems || (ndir < 1e-12 * nx) // gradient relatively too small || (ndir < 1e-32) // gradient absolutely too small; numerical issues may lurk diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala index b781a6aed9a8c..82c327bd49fcd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/NNLSSuite.scala @@ -37,6 +37,12 @@ class NNLSSuite extends FunSuite { (ata, atb) } + /** Compute the objective value */ + def computeObjectiveValue(ata: DoubleMatrix, atb: DoubleMatrix, x: DoubleMatrix): Double = { + val res = (x.transpose().mmul(ata).mmul(x)).mul(0.5).sub(atb.dot(x)) + res.get(0) + } + test("NNLS: exact solution cases") { val n = 20 val rand = new Random(12346) @@ -79,4 +85,28 @@ class NNLSSuite extends FunSuite { assert(x(i) >= 0) } } + + test("NNLS: objective value test") { + val n = 5 + val ata = new DoubleMatrix(5, 5 + , 517399.13534, 242529.67289, -153644.98976, 130802.84503, -798452.29283 + , 242529.67289, 126017.69765, -75944.21743, 81785.36128, -405290.60884 + , -153644.98976, -75944.21743, 46986.44577, -45401.12659, 247059.51049 + , 130802.84503, 81785.36128, -45401.12659, 67457.31310, -253747.03819 + , -798452.29283, -405290.60884, 247059.51049, -253747.03819, 1310939.40814 + ) + val atb = new DoubleMatrix(5, 1, + -31755.05710, 13047.14813, -20191.24443, 25993.77580, 11963.55017) + + /** reference solution obtained from matlab function quadprog */ + val refx = new DoubleMatrix(Array(34.90751, 103.96254, 0.00000, 27.82094, 58.79627)) + val refObj = computeObjectiveValue(ata, atb, refx) + + + val ws = NNLS.createWorkspace(n) + val x = new DoubleMatrix(NNLS.solve(ata, atb, ws)) + val obj = computeObjectiveValue(ata, atb, x) + + assert(obj < refObj + 1E-5) + } } From 418ad83fe113f2f90552eb7247670279b55aed28 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 27 Oct 2014 20:42:05 -0700 Subject: [PATCH 1099/1492] [SPARK-3911] [SQL] HiveSimpleUdf can not be optimized in constant folding ``` explain extended select cos(null) from src limit 1; ``` outputs: ``` Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5] MetastoreRelation default, src, None == Optimized Logical Plan == Limit 1 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5] MetastoreRelation default, src, None == Physical Plan == Limit 1 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5] HiveTableScan [], (MetastoreRelation default, src, None), None ``` After patching this PR it outputs ``` == Parsed Logical Plan == Limit 1 Project ['cos(null) AS c_0#0] UnresolvedRelation None, src, None == Analyzed Logical Plan == Limit 1 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#0] MetastoreRelation default, src, None == Optimized Logical Plan == Limit 1 Project [null AS c_0#0] MetastoreRelation default, src, None == Physical Plan == Limit 1 Project [null AS c_0#0] HiveTableScan [], (MetastoreRelation default, src, None), None ``` Author: Cheng Hao Closes #2771 from chenghao-intel/hive_udf_constant_folding and squashes the following commits: 1379c73 [Cheng Hao] duplicate the PlanTest with catalyst/plans/PlanTest 1e52dda [Cheng Hao] add unit test for hive simple udf constant folding 01609ff [Cheng Hao] support constant folding for HiveSimpleUdf --- .../spark/sql/catalyst/plans/PlanTest.scala | 3 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 10 ++++ .../org/apache/spark/sql/QueryTest.scala | 4 +- .../spark/sql/catalyst/plans/PlanTest.scala | 57 +++++++++++++++++++ .../sql/hive/execution/HivePlanTest.scala | 32 +++++++++++ 5 files changed, 104 insertions(+), 2 deletions(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 7e9f47ef21df8..c4a1f899d8a13 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -33,7 +33,8 @@ class PlanTest extends FunSuite { * we must normalize them to check if two different queries are identical. */ protected def normalizeExprIds(plan: LogicalPlan) = { - val minId = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)).min + val list = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)) + val minId = if (list.isEmpty) 0 else list.min plan transformAllExpressions { case a: AttributeReference => AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 68f93f247d9bb..683c820dec305 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -99,6 +99,16 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ @transient protected lazy val arguments = children.map(c => toInspector(c.dataType)).toArray + @transient + protected lazy val isUDFDeterministic = { + val udfType = function.getClass().getAnnotation(classOf[HiveUDFType]) + udfType != null && udfType.deterministic() + } + + override def foldable = { + isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) + } + // Create parameter converters @transient protected lazy val conversionHelper = new ConversionHelper(method, arguments) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 95921c3d7ae09..6b0641052052d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.{ExprId, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ @@ -29,7 +31,7 @@ import org.apache.spark.sql.catalyst.util._ * It is hard to have maven allow one subproject depend on another subprojects test code. * So, we duplicate this code here. */ -class QueryTest extends FunSuite { +class QueryTest extends PlanTest { /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[SchemaRDD]] to be executed diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala new file mode 100644 index 0000000000000..081d94b6fc020 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans + +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, ExprId} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ +import org.scalatest.FunSuite + +/** + * *** DUPLICATED FROM sql/catalyst/plans. *** + * + * It is hard to have maven allow one subproject depend on another subprojects test code. + * So, we duplicate this code here. + */ +class PlanTest extends FunSuite { + + /** + * Since attribute references are given globally unique ids during analysis, + * we must normalize them to check if two different queries are identical. + */ + protected def normalizeExprIds(plan: LogicalPlan) = { + val list = plan.flatMap(_.expressions.flatMap(_.references).map(_.exprId.id)) + val minId = if (list.isEmpty) 0 else list.min + plan transformAllExpressions { + case a: AttributeReference => + AttributeReference(a.name, a.dataType, a.nullable)(exprId = ExprId(a.exprId.id - minId)) + } + } + + /** Fails the test if the two plans do not match */ + protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { + val normalized1 = normalizeExprIds(plan1) + val normalized2 = normalizeExprIds(plan2) + if (normalized1 != normalized2) + fail( + s""" + |== FAIL: Plans do not match === + |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} + """.stripMargin) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala new file mode 100644 index 0000000000000..c939e6e99d28a --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HivePlanTest.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive + +class HivePlanTest extends QueryTest { + import TestHive._ + + test("udf constant folding") { + val optimized = sql("SELECT cos(null) FROM src").queryExecution.optimizedPlan + val correctAnswer = sql("SELECT cast(null as double) FROM src").queryExecution.optimizedPlan + + comparePlans(optimized, correctAnswer) + } +} From 698a7eab7710cccdfd3a5234dc9572e4e674fff6 Mon Sep 17 00:00:00 2001 From: Alex Liu Date: Mon, 27 Oct 2014 20:43:29 -0700 Subject: [PATCH 1100/1492] [SPARK-3816][SQL] Add table properties from storage handler to output jobConf ...ob conf in SparkHadoopWriter class Author: Alex Liu Closes #2677 from alexliu68/SPARK-SQL-3816 and squashes the following commits: 79c269b [Alex Liu] [SPARK-3816][SQL] Add table properties from storage handler to job conf --- .../org/apache/spark/sql/hive/hiveWriterContainers.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index 981ab954da489..bf2ce9df67c58 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.{PlanUtils, TableDesc} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ @@ -47,6 +48,13 @@ private[hive] class SparkHiveWriterContainer( with Serializable { private val now = new Date() + private val tableDesc: TableDesc = fileSinkConf.getTableInfo + // Add table properties from storage handler to jobConf, so any custom storage + // handler settings can be set to jobConf + if (tableDesc != null) { + PlanUtils.configureOutputJobPropertiesForStorageHandler(tableDesc) + Utilities.copyTableJobPropertiesToConf(tableDesc, jobConf) + } protected val conf = new SerializableWritable(jobConf) private var jobID = 0 From 89af6dfc3afb2b8fc60fa74afb52541dbf3c4e8f Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 27 Oct 2014 20:46:20 -0700 Subject: [PATCH 1101/1492] [SPARK-4041][SQL] Attributes names in table scan should converted to lowercase when compare with relation attributes In ```MetastoreRelation``` the attributes name is lowercase because of hive using lowercase for fields name, so we should convert attributes name in table scan lowercase in ```indexWhere(_.name == a.name)```. ```neededColumnIDs``` may be not correct if not convert to lowercase. Author: wangfei Author: scwf Closes #2884 from scwf/fixColumnIds and squashes the following commits: 6174046 [scwf] use AttributeMap for this issue dc74a24 [wangfei] use lowerName and add a test case for this issue 3ff3a80 [wangfei] more safer change 294fcb7 [scwf] attributes names in table scan should convert lowercase in neededColumnsIDs --- .../org/apache/spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++++++ .../apache/spark/sql/hive/execution/HiveTableScan.scala | 9 +++++---- .../spark/sql/hive/execution/HiveTableScanSuite.scala | 9 +++++++++ 3 files changed, 20 insertions(+), 4 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 04c48c385966e..39d87a9d14fa7 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 @@ -307,4 +307,10 @@ private[hive] case class MetastoreRelation val attributes = hiveQlTable.getCols.map(_.toAttribute) val output = attributes ++ partitionKeys + + /** An attribute map that can be used to lookup original attributes based on expression id. */ + val attributeMap = AttributeMap(output.map(o => (o,o))) + + /** An attribute map for determining the ordinal for non-partition columns. */ + val columnOrdinals = AttributeMap(attributes.zipWithIndex) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index b7f3ade4ea819..d39413a44a6cb 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -44,7 +44,7 @@ import org.apache.spark.sql.hive._ */ @DeveloperApi case class HiveTableScan( - attributes: Seq[Attribute], + requestedAttributes: Seq[Attribute], relation: MetastoreRelation, partitionPruningPred: Option[Expression])( @transient val context: HiveContext) @@ -53,6 +53,9 @@ case class HiveTableScan( require(partitionPruningPred.isEmpty || relation.hiveQlTable.isPartitioned, "Partition pruning predicates only supported for partitioned tables.") + // Retrieve the original attributes based on expression ID so that capitalization matches. + val attributes = requestedAttributes.map(relation.attributeMap) + // Bind all partition key attribute references in the partition pruning predicate for later // evaluation. private[this] val boundPruningPred = partitionPruningPred.map { pred => @@ -81,9 +84,7 @@ case class HiveTableScan( private def addColumnMetadataToConf(hiveConf: HiveConf) { // Specifies needed column IDs for those non-partitioning columns. - val neededColumnIDs = - attributes.map(a => - relation.attributes.indexWhere(_.name == a.name): Integer).filter(index => index >= 0) + val neededColumnIDs = attributes.flatMap(relation.columnOrdinals.get).map(o => o: Integer) HiveShim.appendReadColumns(hiveConf, neededColumnIDs, attributes.map(_.name)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index c5736723b47c0..2f3db95882093 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.apache.spark.sql.hive.test.TestHive + class HiveTableScanSuite extends HiveComparisonTest { createQueryTest("partition_based_table_scan_with_different_serde", @@ -38,4 +40,11 @@ class HiveTableScanSuite extends HiveComparisonTest { | |SELECT * from part_scan_test; """.stripMargin) + + test("Spark-4041: lowercase issue") { + TestHive.sql("CREATE TABLE tb (KEY INT, VALUE STRING) STORED AS ORC") + TestHive.sql("insert into table tb select key, value from src") + TestHive.sql("select KEY from tb where VALUE='just_for_test' limit 5").collect() + TestHive.sql("drop table tb") + } } From 27470d3406bc0adde3da79ca34ebf9bc512514b6 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 27 Oct 2014 20:50:09 -0700 Subject: [PATCH 1102/1492] [SQL] Correct a variable name in JavaApplySchemaSuite.applySchemaToJSON `schemaRDD2` is not tested because `schemaRDD1` is registered again. Author: Yin Huai Closes #2869 from yhuai/JavaApplySchemaSuite and squashes the following commits: 95fe894 [Yin Huai] Correct variable name. --- .../org/apache/spark/sql/api/java/JavaApplySchemaSuite.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index 33e5020bc636a..9435a88009a5f 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -156,7 +156,7 @@ public void applySchemaToJSON() { JavaSchemaRDD schemaRDD2 = javaSqlCtx.jsonRDD(jsonRDD, expectedSchema); StructType actualSchema2 = schemaRDD2.schema(); Assert.assertEquals(expectedSchema, actualSchema2); - schemaRDD1.registerTempTable("jsonTable2"); + schemaRDD2.registerTempTable("jsonTable2"); List actual2 = javaSqlCtx.sql("select * from jsonTable2").collect(); Assert.assertEquals(expectedResult, actual2); } From 0c34fa5b4b3c1c20d7a2d7df3a8ae757b532dd32 Mon Sep 17 00:00:00 2001 From: wangxiaojing Date: Mon, 27 Oct 2014 22:00:36 -0700 Subject: [PATCH 1103/1492] [SPARK-3907][SQL] Add truncate table support JIRA issue: [SPARK-3907]https://issues.apache.org/jira/browse/SPARK-3907 Add turncate table support TRUNCATE TABLE table_name [PARTITION partition_spec]; partition_spec: : (partition_col = partition_col_value, partition_col = partiton_col_value, ...) Removes all rows from a table or partition(s). Currently target table should be native/managed table or exception will be thrown. User can specify partial partition_spec for truncating multiple partitions at once and omitting partition_spec will truncate all partitions in the table. Author: wangxiaojing Closes #2770 from wangxiaojing/spark-3907 and squashes the following commits: 63dbd81 [wangxiaojing] change hive scalastyle 7a03707 [wangxiaojing] add comment f6e710e [wangxiaojing] change truncate table a1f692c [wangxiaojing] Correct spelling mistakes 3b20007 [wangxiaojing] add truncate can not support column err message e483547 [wangxiaojing] add golden file 77b1f20 [wangxiaojing] add truncate table support --- .../spark/sql/hive/execution/HiveCompatibilitySuite.scala | 1 + .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 7 ++++++- .../truncate_column-0-616cad77ad5e7ac74da0d7425a7869a | 0 .../truncate_column-1-418ec894d08c33fd712eb358f579b7a0 | 1 + ...runcate_column_merge-0-46e8cc1556fa8586802a26267a906acf | 0 .../truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb | 0 .../truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 | 0 .../truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb | 0 .../truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 | 1 + .../truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 | 0 .../truncate_table-13-3230cfbe1871330193c3190c77582fe | 0 .../truncate_table-14-ae23925663d7e9b7e97c42b66086d835 | 0 .../truncate_table-15-7850dc059f9d00eb9439d477e92cb913 | 0 .../truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 | 0 .../truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 | 0 .../truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 | 0 .../truncate_table-19-1325d566d66f21a06543271c73a95a6f | 0 .../truncate_table-2-fc4118284bf8301cf0d1056c388f963a | 0 .../truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 | 0 .../truncate_table-21-f635675d59df31843e7be41af7b9e4fa | 0 .../truncate_table-22-f121fdc101603a8220c0f18e867f581e | 0 .../truncate_table-23-63988ac685a3bd645787116353f024d2 | 0 .../truncate_table-3-ecca1d24f36175932911a6e7a78ece2d | 0 .../truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 | 0 .../truncate_table-5-42aeecc67917d731e60fc46bde021d49 | 0 .../truncate_table-6-5a6776344f711298f27a8f1d3b47d107 | 0 .../truncate_table-7-65e270fb0b61886aa85255d77eb65794 | 0 .../truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e | 0 .../truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 | 0 29 files changed, 9 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/resources/golden/truncate_column-0-616cad77ad5e7ac74da0d7425a7869a create mode 100644 sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/truncate_column_merge-0-46e8cc1556fa8586802a26267a906acf create mode 100644 sql/hive/src/test/resources/golden/truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb create mode 100644 sql/hive/src/test/resources/golden/truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb create mode 100644 sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-13-3230cfbe1871330193c3190c77582fe create mode 100644 sql/hive/src/test/resources/golden/truncate_table-14-ae23925663d7e9b7e97c42b66086d835 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-15-7850dc059f9d00eb9439d477e92cb913 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-19-1325d566d66f21a06543271c73a95a6f create mode 100644 sql/hive/src/test/resources/golden/truncate_table-2-fc4118284bf8301cf0d1056c388f963a create mode 100644 sql/hive/src/test/resources/golden/truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-21-f635675d59df31843e7be41af7b9e4fa create mode 100644 sql/hive/src/test/resources/golden/truncate_table-22-f121fdc101603a8220c0f18e867f581e create mode 100644 sql/hive/src/test/resources/golden/truncate_table-23-63988ac685a3bd645787116353f024d2 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-3-ecca1d24f36175932911a6e7a78ece2d create mode 100644 sql/hive/src/test/resources/golden/truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-5-42aeecc67917d731e60fc46bde021d49 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-6-5a6776344f711298f27a8f1d3b47d107 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-7-65e270fb0b61886aa85255d77eb65794 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e create mode 100644 sql/hive/src/test/resources/golden/truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 15cd62d3bf869..4fc26d6f55511 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -767,6 +767,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "touch", "transform_ppr1", "transform_ppr2", + "truncate_table", "type_cast_1", "type_widening", "udaf_collect_set", 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 54c619722ee12..aa80b2f04de25 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 @@ -124,7 +124,8 @@ private[hive] object HiveQl { // Commands that we do not need to explain. protected val noExplainCommands = Seq( "TOK_CREATETABLE", - "TOK_DESCTABLE" + "TOK_DESCTABLE", + "TOK_TRUNCATETABLE" // truncate table" is a NativeCommand, does not need to explain. ) ++ nativeCommands protected val hqlParser = { @@ -490,6 +491,10 @@ private[hive] object HiveQl { // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder + // Support "TRUNCATE TABLE table_name [PARTITION partition_spec]" + case Token("TOK_TRUNCATETABLE", + Token("TOK_TABLE_PARTITION",table)::Nil) => NativePlaceholder + case Token("TOK_QUERY", Token("TOK_FROM", fromClause :: Nil) :: insertClauses) => diff --git a/sql/hive/src/test/resources/golden/truncate_column-0-616cad77ad5e7ac74da0d7425a7869a b/sql/hive/src/test/resources/golden/truncate_column-0-616cad77ad5e7ac74da0d7425a7869a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/truncate_column-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/truncate_column_merge-0-46e8cc1556fa8586802a26267a906acf b/sql/hive/src/test/resources/golden/truncate_column_merge-0-46e8cc1556fa8586802a26267a906acf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb b/sql/hive/src/test/resources/golden/truncate_table-0-d16efe9bac079f0c5fc6cc424a8fa3eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 b/sql/hive/src/test/resources/golden/truncate_table-1-ec0e3744208003f18c33a1f2c4c1e2c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb b/sql/hive/src/test/resources/golden/truncate_table-10-9ba46fdca3f0f4da8991cb5c7b01efdb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 b/sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/truncate_table-11-6e0b877ea24fa88c5461b02f7bda0746 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 b/sql/hive/src/test/resources/golden/truncate_table-12-7dee32ebe9887833a9ae2ea6e5568028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-13-3230cfbe1871330193c3190c77582fe b/sql/hive/src/test/resources/golden/truncate_table-13-3230cfbe1871330193c3190c77582fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-14-ae23925663d7e9b7e97c42b66086d835 b/sql/hive/src/test/resources/golden/truncate_table-14-ae23925663d7e9b7e97c42b66086d835 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-15-7850dc059f9d00eb9439d477e92cb913 b/sql/hive/src/test/resources/golden/truncate_table-15-7850dc059f9d00eb9439d477e92cb913 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 b/sql/hive/src/test/resources/golden/truncate_table-16-623e41aa678d5abc8341a8cee0ac8f94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 b/sql/hive/src/test/resources/golden/truncate_table-17-8c71d29e7db6a8d1cb5746458c7741e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 b/sql/hive/src/test/resources/golden/truncate_table-18-64d431f93d8a44fb143cb4b87d63a105 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-19-1325d566d66f21a06543271c73a95a6f b/sql/hive/src/test/resources/golden/truncate_table-19-1325d566d66f21a06543271c73a95a6f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-2-fc4118284bf8301cf0d1056c388f963a b/sql/hive/src/test/resources/golden/truncate_table-2-fc4118284bf8301cf0d1056c388f963a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 b/sql/hive/src/test/resources/golden/truncate_table-20-91f869cc79191b87d31cfd0eca2839f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-21-f635675d59df31843e7be41af7b9e4fa b/sql/hive/src/test/resources/golden/truncate_table-21-f635675d59df31843e7be41af7b9e4fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-22-f121fdc101603a8220c0f18e867f581e b/sql/hive/src/test/resources/golden/truncate_table-22-f121fdc101603a8220c0f18e867f581e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-23-63988ac685a3bd645787116353f024d2 b/sql/hive/src/test/resources/golden/truncate_table-23-63988ac685a3bd645787116353f024d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-3-ecca1d24f36175932911a6e7a78ece2d b/sql/hive/src/test/resources/golden/truncate_table-3-ecca1d24f36175932911a6e7a78ece2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 b/sql/hive/src/test/resources/golden/truncate_table-4-88e636ed8bdf647a02ff269aa3ebfe62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-5-42aeecc67917d731e60fc46bde021d49 b/sql/hive/src/test/resources/golden/truncate_table-5-42aeecc67917d731e60fc46bde021d49 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-6-5a6776344f711298f27a8f1d3b47d107 b/sql/hive/src/test/resources/golden/truncate_table-6-5a6776344f711298f27a8f1d3b47d107 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-7-65e270fb0b61886aa85255d77eb65794 b/sql/hive/src/test/resources/golden/truncate_table-7-65e270fb0b61886aa85255d77eb65794 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e b/sql/hive/src/test/resources/golden/truncate_table-8-e7699db3640f3b9b1fe44d6b8c9b507e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 b/sql/hive/src/test/resources/golden/truncate_table-9-eedfbb9479ac6c1b955b8e9b41994da4 new file mode 100644 index 0000000000000..e69de29bb2d1d From 7c0c26cd1241e1fde3c6f1f659a43b9c40ee3d42 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Mon, 27 Oct 2014 23:31:46 -0700 Subject: [PATCH 1104/1492] [SPARK-4064]NioBlockTransferService.fetchBlocks may cause spark to hang. cc @rxin Author: GuoQiang Li Closes #2929 from witgo/SPARK-4064 and squashes the following commits: 20110f2 [GuoQiang Li] Modify the exception msg 3425225 [GuoQiang Li] review commits 2b07e49 [GuoQiang Li] If we create a lot of big broadcast variables, Spark may hang --- .../network/nio/NioBlockTransferService.scala | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index 5add4fc433fb3..e3113205bebdc 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -95,16 +95,21 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa future.onSuccess { case message => val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) - - for (blockMessage <- blockMessageArray) { - if (blockMessage.getType != BlockMessage.TYPE_GOT_BLOCK) { - listener.onBlockFetchFailure( - new SparkException(s"Unexpected message ${blockMessage.getType} received from $cmId")) - } else { - val blockId = blockMessage.getId - val networkSize = blockMessage.getData.limit() - listener.onBlockFetchSuccess( - blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + // SPARK-4064: In some cases(eg. Remote block was removed) blockMessageArray may be empty. + if (blockMessageArray.isEmpty) { + listener.onBlockFetchFailure( + new SparkException(s"Received empty message from $cmId")) + } else { + for (blockMessage <- blockMessageArray) { + val msgType = blockMessage.getType + if (msgType != BlockMessage.TYPE_GOT_BLOCK) { + listener.onBlockFetchFailure( + new SparkException(s"Unexpected message ${msgType} received from $cmId")) + } else { + val blockId = blockMessage.getId + listener.onBlockFetchSuccess( + blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + } } } }(cm.futureExecContext) From 4ceb048b38949dd0a909d2ee6777607341c9c93a Mon Sep 17 00:00:00 2001 From: Ryan Williams Date: Mon, 27 Oct 2014 23:55:13 -0700 Subject: [PATCH 1105/1492] fix broken links in README.md seems like `building-spark.html` was renamed to `building-with-maven.html`? Is Maven the blessed build tool these days, or SBT? I couldn't find a building-with-sbt page so I went with the Maven one here. Author: Ryan Williams Closes #2859 from ryan-williams/broken-links-readme and squashes the following commits: 7692253 [Ryan Williams] fix broken links in README.md --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index dbf53dcd76b2d..9916ac7b1ae8e 100644 --- a/README.md +++ b/README.md @@ -84,7 +84,7 @@ storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs. Please refer to the build documentation at -["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version) +["Specifying the Hadoop Version"](http://spark.apache.org/docs/latest/building-with-maven.html#specifying-the-hadoop-version) for detailed guidance on building for a particular distribution of Hadoop, including building for particular Hive and Hive Thriftserver distributions. See also ["Third Party Hadoop Distributions"](http://spark.apache.org/docs/latest/hadoop-third-party-distributions.html) From 46c63417c1bb1aea07baf9036cc5b8f1c3781bbe Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 28 Oct 2014 00:04:16 -0700 Subject: [PATCH 1106/1492] [SPARK-4107] Fix incorrect handling of read() and skip() return values `read()` may return fewer bytes than requested; when this occurred, the old code would silently return less data than requested, which might cause stream corruption errors. `skip()` faces similar issues, too. This patch fixes several cases where we mis-handle these methods' return values. Author: Josh Rosen Closes #2969 from JoshRosen/file-channel-read-fix and squashes the following commits: e724a9f [Josh Rosen] Fix similar issue of not checking skip() return value. cbc03ce [Josh Rosen] Update the other log message, too. 01e6015 [Josh Rosen] file.getName -> file.getAbsolutePath d961d95 [Josh Rosen] Fix another issue in FileServerSuite. b9265d2 [Josh Rosen] Fix a similar (minor) issue in TestUtils. cd9d76f [Josh Rosen] Fix a similar error in Tachyon: 3db0008 [Josh Rosen] Fix a similar read() error in Utils.offsetBytes(). db985ed [Josh Rosen] Fix unsafe usage of FileChannel.read(): --- .../scala/org/apache/spark/TestUtils.scala | 9 ++------ .../apache/spark/network/ManagedBuffer.scala | 10 +++++++-- .../shuffle/IndexShuffleBlockManager.scala | 4 +++- .../org/apache/spark/storage/DiskStore.scala | 10 +++++++-- .../apache/spark/storage/TachyonStore.scala | 21 +++++++------------ .../scala/org/apache/spark/util/Utils.scala | 6 +++--- .../org/apache/spark/FileServerSuite.scala | 8 ++----- 7 files changed, 33 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index e72826dc25f41..34078142f5385 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -23,8 +23,8 @@ import java.util.jar.{JarEntry, JarOutputStream} import scala.collection.JavaConversions._ +import com.google.common.io.{ByteStreams, Files} import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} -import com.google.common.io.Files import org.apache.spark.util.Utils @@ -64,12 +64,7 @@ private[spark] object TestUtils { jarStream.putNextEntry(jarEntry) val in = new FileInputStream(file) - val buffer = new Array[Byte](10240) - var nRead = 0 - while (nRead <= 0) { - nRead = in.read(buffer, 0, buffer.length) - jarStream.write(buffer, 0, nRead) - } + ByteStreams.copy(in, jarStream) in.close() } jarStream.close() diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala index 4c9ca97a2a6b7..4211ba4e43b9e 100644 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala @@ -81,7 +81,13 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. if (length < MIN_MEMORY_MAP_BYTES) { val buf = ByteBuffer.allocate(length.toInt) - channel.read(buf, offset) + channel.position(offset) + while (buf.remaining() != 0) { + if (channel.read(buf) == -1) { + throw new IOException("Reached EOF before filling buffer\n" + + s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") + } + } buf.flip() buf } else { @@ -106,7 +112,7 @@ final class FileSegmentManagedBuffer(val file: File, val offset: Long, val lengt var is: FileInputStream = null try { is = new FileInputStream(file) - is.skip(offset) + ByteStreams.skipFully(is, offset) ByteStreams.limit(is, length) } catch { case e: IOException => diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index 4ab34336d3f01..b5cd34cacd520 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -20,6 +20,8 @@ package org.apache.spark.shuffle import java.io._ import java.nio.ByteBuffer +import com.google.common.io.ByteStreams + import org.apache.spark.SparkEnv import org.apache.spark.network.{ManagedBuffer, FileSegmentManagedBuffer} import org.apache.spark.storage._ @@ -101,7 +103,7 @@ class IndexShuffleBlockManager extends ShuffleBlockManager { val in = new DataInputStream(new FileInputStream(indexFile)) try { - in.skip(blockId.reduceId * 8) + ByteStreams.skipFully(in, blockId.reduceId * 8) val offset = in.readLong() val nextOffset = in.readLong() new FileSegmentManagedBuffer( diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index bac459e835a3f..8dadf6794039e 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{File, FileOutputStream, RandomAccessFile} +import java.io.{IOException, File, FileOutputStream, RandomAccessFile} import java.nio.ByteBuffer import java.nio.channels.FileChannel.MapMode @@ -110,7 +110,13 @@ private[spark] class DiskStore(blockManager: BlockManager, diskManager: DiskBloc // For small files, directly read rather than memory map if (length < minMemoryMapBytes) { val buf = ByteBuffer.allocate(length.toInt) - channel.read(buf, offset) + channel.position(offset) + while (buf.remaining() != 0) { + if (channel.read(buf) == -1) { + throw new IOException("Reached EOF before filling buffer\n" + + s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") + } + } buf.flip() Some(buf) } else { diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala index 932b5616043b4..6dbad5ff0518e 100644 --- a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.IOException import java.nio.ByteBuffer +import com.google.common.io.ByteStreams import tachyon.client.{ReadType, WriteType} import org.apache.spark.Logging @@ -105,25 +106,17 @@ private[spark] class TachyonStore( return None } val is = file.getInStream(ReadType.CACHE) - var buffer: ByteBuffer = null + assert (is != null) try { - if (is != null) { - val size = file.length - val bs = new Array[Byte](size.asInstanceOf[Int]) - val fetchSize = is.read(bs, 0, size.asInstanceOf[Int]) - buffer = ByteBuffer.wrap(bs) - if (fetchSize != size) { - logWarning(s"Failed to fetch the block $blockId from Tachyon: Size $size " + - s"is not equal to fetched size $fetchSize") - return None - } - } + val size = file.length + val bs = new Array[Byte](size.asInstanceOf[Int]) + ByteStreams.readFully(is, bs) + Some(ByteBuffer.wrap(bs)) } catch { case ioe: IOException => logWarning(s"Failed to fetch the block $blockId from Tachyon", ioe) - return None + None } - Some(buffer) } override def contains(blockId: BlockId): Boolean = { 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 93ac9f1c33ea8..4660030155854 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -33,7 +33,7 @@ import scala.reflect.ClassTag import scala.util.Try import scala.util.control.{ControlThrowable, NonFatal} -import com.google.common.io.Files +import com.google.common.io.{ByteStreams, Files} import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration @@ -1062,8 +1062,8 @@ private[spark] object Utils extends Logging { val stream = new FileInputStream(file) try { - stream.skip(effectiveStart) - stream.read(buff) + ByteStreams.skipFully(stream, effectiveStart) + ByteStreams.readFully(stream, buff) } finally { stream.close() } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index a8867020e457d..379c2a6ea4b55 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import java.io._ import java.util.jar.{JarEntry, JarOutputStream} +import com.google.common.io.ByteStreams import org.scalatest.FunSuite import org.apache.spark.SparkContext._ @@ -58,12 +59,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { jar.putNextEntry(jarEntry) val in = new FileInputStream(textFile) - val buffer = new Array[Byte](10240) - var nRead = 0 - while (nRead <= 0) { - nRead = in.read(buffer, 0, buffer.length) - jar.write(buffer, 0, nRead) - } + ByteStreams.copy(in, jar) in.close() jar.close() From fae095bc7c4097859af522ced77f09cf6be17691 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 28 Oct 2014 03:50:22 -0700 Subject: [PATCH 1107/1492] [SPARK-3961] [MLlib] [PySpark] Python API for mllib.feature Added completed Python API for MLlib.feature Normalizer StandardScalerModel StandardScaler HashTF IDFModel IDF cc mengxr Author: Davies Liu Author: Davies Liu Closes #2819 from davies/feature and squashes the following commits: 4f48f48 [Davies Liu] add a note for HashingTF 67f6d21 [Davies Liu] address comments b628693 [Davies Liu] rollback changes in Word2Vec efb4f4f [Davies Liu] Merge branch 'master' into feature 806c7c2 [Davies Liu] address comments 3abb8c2 [Davies Liu] address comments 59781b9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into feature a405ae7 [Davies Liu] fix tests 7a1891a [Davies Liu] fix tests 486795f [Davies Liu] update programming guide, HashTF -> HashingTF 8a50584 [Davies Liu] Python API for mllib.feature --- docs/mllib-feature-extraction.md | 85 ++++ .../mllib/api/python/PythonMLLibAPI.scala | 49 ++- .../mllib/feature/VectorTransformer.scala | 11 + .../apache/spark/mllib/feature/Word2Vec.scala | 4 +- python/pyspark/mllib/feature.py | 395 +++++++++++++++--- python/pyspark/mllib/linalg.py | 16 +- 6 files changed, 499 insertions(+), 61 deletions(-) diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 11622414494e4..886d71df474bc 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -95,8 +95,49 @@ tf.cache() val idf = new IDF(minDocFreq = 2).fit(tf) val tfidf: RDD[Vector] = idf.transform(tf) {% endhighlight %} + +
    + +TF and IDF are implemented in [HashingTF](api/python/pyspark.mllib.html#pyspark.mllib.feature.HashingTF) +and [IDF](api/python/pyspark.mllib.html#pyspark.mllib.feature.IDF). +`HashingTF` takes an RDD of list as the input. +Each record could be an iterable of strings or other types. + +{% highlight python %} +from pyspark import SparkContext +from pyspark.mllib.feature import HashingTF + +sc = SparkContext() +# Load documents (one per line). +documents = sc.textFile("...").map(lambda line: line.split(" ")) + +hashingTF = HashingTF() +tf = hashingTF.transform(documents) +{% endhighlight %} + +While applying `HashingTF` only needs a single pass to the data, applying `IDF` needs two passes: +first to compute the IDF vector and second to scale the term frequencies by IDF. + +{% highlight python %} +from pyspark.mllib.feature import IDF + +# ... continue from the previous example +tf.cache() +idf = IDF().fit(tf) +tfidf = idf.transform(tf) +{% endhighlight %} + +MLLib's IDF implementation provides an option for ignoring terms which occur in less than a +minimum number of documents. In such cases, the IDF for these terms is set to 0. This feature +can be used by passing the `minDocFreq` value to the IDF constructor. +{% highlight python %} +# ... continue from the previous example +tf.cache() +idf = IDF(minDocFreq=2).fit(tf) +tfidf = idf.transform(tf) +{% endhighlight %}
    @@ -223,6 +264,29 @@ val data1 = data.map(x => (x.label, scaler1.transform(x.features))) val data2 = data.map(x => (x.label, scaler2.transform(Vectors.dense(x.features.toArray)))) {% endhighlight %} + +
    +{% highlight python %} +from pyspark.mllib.util import MLUtils +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.feature import StandardScaler + +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +label = data.map(lambda x: x.label) +features = data.map(lambda x: x.features) + +scaler1 = StandardScaler().fit(features) +scaler2 = StandardScaler(withMean=True, withStd=True).fit(features) + +# data1 will be unit variance. +data1 = label.zip(scaler1.transform(features)) + +# Without converting the features into dense vectors, transformation with zero mean will raise +# exception on sparse vector. +# data2 will be unit variance and zero mean. +data2 = label.zip(scaler1.transform(features.map(lambda x: Vectors.dense(x.toArray())))) +{% endhighlight %} +
    ## Normalizer @@ -267,4 +331,25 @@ val data1 = data.map(x => (x.label, normalizer1.transform(x.features))) val data2 = data.map(x => (x.label, normalizer2.transform(x.features))) {% endhighlight %} + +
    +{% highlight python %} +from pyspark.mllib.util import MLUtils +from pyspark.mllib.linalg import Vectors +from pyspark.mllib.feature import Normalizer + +data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +labels = data.map(lambda x: x.label) +features = data.map(lambda x: x.features) + +normalizer1 = Normalizer() +normalizer2 = Normalizer(p=float("inf")) + +# Each sample in data1 will be normalized using $L^2$ norm. +data1 = labels.zip(normalizer1.transform(features)) + +# Each sample in data2 will be normalized using $L^\infty$ norm. +data2 = labels.zip(normalizer2.transform(features)) +{% endhighlight %} +
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index b478c21537c2a..485abe272326c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -31,8 +31,7 @@ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ -import org.apache.spark.mllib.feature.Word2Vec -import org.apache.spark.mllib.feature.Word2VecModel +import org.apache.spark.mllib.feature._ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.random.{RandomRDDs => RG} @@ -291,6 +290,43 @@ class PythonMLLibAPI extends Serializable { ALS.trainImplicit(ratingsJRDD.rdd, rank, iterations, lambda, blocks, alpha) } + /** + * Java stub for Normalizer.transform() + */ + def normalizeVector(p: Double, vector: Vector): Vector = { + new Normalizer(p).transform(vector) + } + + /** + * Java stub for Normalizer.transform() + */ + def normalizeVector(p: Double, rdd: JavaRDD[Vector]): JavaRDD[Vector] = { + new Normalizer(p).transform(rdd) + } + + /** + * Java stub for IDF.fit(). This stub returns a + * handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on + * exit; see the Py4J documentation. + */ + def fitStandardScaler( + withMean: Boolean, + withStd: Boolean, + data: JavaRDD[Vector]): StandardScalerModel = { + new StandardScaler(withMean, withStd).fit(data.rdd) + } + + /** + * Java stub for IDF.fit(). This stub returns a + * handle to the Java object instead of the content of the Java object. + * Extra care needs to be taken in the Python code to ensure it gets freed on + * exit; see the Py4J documentation. + */ + def fitIDF(minDocFreq: Int, dataset: JavaRDD[Vector]): IDFModel = { + new IDF(minDocFreq).fit(dataset) + } + /** * Java stub for Python mllib Word2Vec fit(). This stub returns a * handle to the Java object instead of the content of the Java object. @@ -328,6 +364,15 @@ class PythonMLLibAPI extends Serializable { model.transform(word) } + /** + * Transforms an RDD of words to its vector representation + * @param rdd an RDD of words + * @return an RDD of vector representations of words + */ + def transform(rdd: JavaRDD[String]): JavaRDD[Vector] = { + rdd.rdd.map(model.transform) + } + def findSynonyms(word: String, num: Int): java.util.List[java.lang.Object] = { val vec = transform(word) findSynonyms(vec, num) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala index 415a845332d45..7358c1c84f79c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/VectorTransformer.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD @@ -48,4 +49,14 @@ trait VectorTransformer extends Serializable { data.map(x => this.transform(x)) } + /** + * Applies transformation on an JavaRDD[Vector]. + * + * @param data JavaRDD[Vector] to be transformed. + * @return transformed JavaRDD[Vector]. + */ + def transform(data: JavaRDD[Vector]): JavaRDD[Vector] = { + transform(data.rdd) + } + } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index d321994c2a651..f5f7ad613d4c4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -432,7 +432,7 @@ class Word2VecModel private[mllib] ( throw new IllegalStateException(s"$word not in vocabulary") } } - + /** * Find synonyms of a word * @param word a word @@ -443,7 +443,7 @@ class Word2VecModel private[mllib] ( val vector = transform(word) findSynonyms(vector,num) } - + /** * Find synonyms of the vector representation of a word * @param vector vector representation of a word diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index b5a3f22c6907e..324343443ebdb 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -18,59 +18,357 @@ """ Python package for feature in MLlib. """ +import sys +import warnings + +import py4j.protocol +from py4j.protocol import Py4JJavaError +from py4j.java_gateway import JavaObject + +from pyspark import RDD, SparkContext from pyspark.serializers import PickleSerializer, AutoBatchedSerializer -from pyspark.mllib.linalg import _convert_to_vector, _to_java_object_rdd +from pyspark.mllib.linalg import Vectors, _to_java_object_rdd + +__all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', + 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] + + +# Hack for support float('inf') in Py4j +_old_smart_decode = py4j.protocol.smart_decode + +_float_str_mapping = { + u'nan': u'NaN', + u'inf': u'Infinity', + u'-inf': u'-Infinity', +} + + +def _new_smart_decode(obj): + if isinstance(obj, float): + s = unicode(obj) + return _float_str_mapping.get(s, s) + return _old_smart_decode(obj) + +py4j.protocol.smart_decode = _new_smart_decode + + +# TODO: move these helper functions into utils +_picklable_classes = [ + 'LinkedList', + 'SparseVector', + 'DenseVector', + 'DenseMatrix', + 'Rating', + 'LabeledPoint', +] + + +def _py2java(sc, a): + """ Convert Python object into Java """ + if isinstance(a, RDD): + a = _to_java_object_rdd(a) + elif not isinstance(a, (int, long, float, bool, basestring)): + bytes = bytearray(PickleSerializer().dumps(a)) + a = sc._jvm.SerDe.loads(bytes) + return a + + +def _java2py(sc, r): + if isinstance(r, JavaObject): + clsName = r.getClass().getSimpleName() + if clsName in ("RDD", "JavaRDD"): + if clsName == "RDD": + r = r.toJavaRDD() + jrdd = sc._jvm.SerDe.javaToPython(r) + return RDD(jrdd, sc, AutoBatchedSerializer(PickleSerializer())) -__all__ = ['Word2Vec', 'Word2VecModel'] + elif clsName in _picklable_classes: + r = sc._jvm.SerDe.dumps(r) + if isinstance(r, bytearray): + r = PickleSerializer().loads(str(r)) + return r -class Word2VecModel(object): + +def _callJavaFunc(sc, func, *args): + """ Call Java Function """ - class for Word2Vec model + args = [_py2java(sc, a) for a in args] + return _java2py(sc, func(*args)) + + +def _callAPI(sc, name, *args): + """ Call API in PythonMLLibAPI """ - def __init__(self, sc, java_model): + api = getattr(sc._jvm.PythonMLLibAPI(), name) + return _callJavaFunc(sc, api, *args) + + +class VectorTransformer(object): + """ + :: DeveloperApi :: + + Base class for transformation of a vector or RDD of vector + """ + def transform(self, vector): + """ + Applies transformation on a vector. + + :param vector: vector to be transformed. + """ + raise NotImplementedError + + +class Normalizer(VectorTransformer): + """ + :: Experimental :: + + Normalizes samples individually to unit L\ :sup:`p`\ norm + + For any 1 <= `p` <= float('inf'), normalizes samples using + sum(abs(vector). :sup:`p`) :sup:`(1/p)` as norm. + + For `p` = float('inf'), max(abs(vector)) will be used as norm for normalization. + + >>> v = Vectors.dense(range(3)) + >>> nor = Normalizer(1) + >>> nor.transform(v) + DenseVector([0.0, 0.3333, 0.6667]) + + >>> rdd = sc.parallelize([v]) + >>> nor.transform(rdd).collect() + [DenseVector([0.0, 0.3333, 0.6667])] + + >>> nor2 = Normalizer(float("inf")) + >>> nor2.transform(v) + DenseVector([0.0, 0.5, 1.0]) + """ + def __init__(self, p=2.0): """ - :param sc: Spark context - :param java_model: Handle to Java model object + :param p: Normalization in L^p^ space, p = 2 by default. """ + assert p >= 1.0, "p should be greater than 1.0" + self.p = float(p) + + def transform(self, vector): + """ + Applies unit length normalization on a vector. + + :param vector: vector to be normalized. + :return: normalized vector. If the norm of the input is zero, it + will return the input vector. + """ + sc = SparkContext._active_spark_context + assert sc is not None, "SparkContext should be initialized first" + return _callAPI(sc, "normalizeVector", self.p, vector) + + +class JavaModelWrapper(VectorTransformer): + """ + Wrapper for the model in JVM + """ + def __init__(self, sc, java_model): self._sc = sc self._java_model = java_model def __del__(self): self._sc._gateway.detach(self._java_model) - def transform(self, word): + def transform(self, dataset): + return _callJavaFunc(self._sc, self._java_model.transform, dataset) + + +class StandardScalerModel(JavaModelWrapper): + """ + :: Experimental :: + + Represents a StandardScaler model that can transform vectors. + """ + def transform(self, vector): """ - :param word: a word - :return: vector representation of word + Applies standardization transformation on a vector. + + :param vector: Vector to be standardized. + :return: Standardized vector. If the variance of a column is zero, + it will return default `0.0` for the column with zero variance. + """ + return JavaModelWrapper.transform(self, vector) + + +class StandardScaler(object): + """ + :: Experimental :: + + Standardizes features by removing the mean and scaling to unit + variance using column summary statistics on the samples in the + training set. + >>> vs = [Vectors.dense([-2.0, 2.3, 0]), Vectors.dense([3.8, 0.0, 1.9])] + >>> dataset = sc.parallelize(vs) + >>> standardizer = StandardScaler(True, True) + >>> model = standardizer.fit(dataset) + >>> result = model.transform(dataset) + >>> for r in result.collect(): r + DenseVector([-0.7071, 0.7071, -0.7071]) + DenseVector([0.7071, -0.7071, 0.7071]) + """ + def __init__(self, withMean=False, withStd=True): + """ + :param withMean: False by default. Centers the data with mean + before scaling. It will build a dense output, so this + does not work on sparse input and will raise an exception. + :param withStd: True by default. Scales the data to unit standard + deviation. + """ + if not (withMean or withStd): + warnings.warn("Both withMean and withStd are false. The model does nothing.") + self.withMean = withMean + self.withStd = withStd + + def fit(self, dataset): + """ + Computes the mean and variance and stores as a model to be used for later scaling. + + :param data: The data used to compute the mean and variance to build + the transformation model. + :return: a StandardScalarModel + """ + sc = dataset.context + jmodel = _callAPI(sc, "fitStandardScaler", self.withMean, self.withStd, dataset) + return StandardScalerModel(sc, jmodel) + + +class HashingTF(object): + """ + :: Experimental :: + + Maps a sequence of terms to their term frequencies using the hashing trick. + + Note: the terms must be hashable (can not be dict/set/list...). + + >>> htf = HashingTF(100) + >>> doc = "a a b b c d".split(" ") + >>> htf.transform(doc) + SparseVector(100, {1: 1.0, 14: 1.0, 31: 2.0, 44: 2.0}) + """ + def __init__(self, numFeatures=1 << 20): + """ + :param numFeatures: number of features (default: 2^20) + """ + self.numFeatures = numFeatures + + def indexOf(self, term): + """ Returns the index of the input term. """ + return hash(term) % self.numFeatures + + def transform(self, document): + """ + Transforms the input document (list of terms) to term frequency vectors, + or transform the RDD of document to RDD of term frequency vectors. + """ + if isinstance(document, RDD): + return document.map(self.transform) + + freq = {} + for term in document: + i = self.indexOf(term) + freq[i] = freq.get(i, 0) + 1.0 + return Vectors.sparse(self.numFeatures, freq.items()) + + +class IDFModel(JavaModelWrapper): + """ + Represents an IDF model that can transform term frequency vectors. + """ + def transform(self, dataset): + """ + Transforms term frequency (TF) vectors to TF-IDF vectors. + + If `minDocFreq` was set for the IDF calculation, + the terms which occur in fewer than `minDocFreq` + documents will have an entry of 0. + + :param dataset: an RDD of term frequency vectors + :return: an RDD of TF-IDF vectors + """ + return JavaModelWrapper.transform(self, dataset) + + +class IDF(object): + """ + :: Experimental :: + + Inverse document frequency (IDF). + + The standard formulation is used: `idf = log((m + 1) / (d(t) + 1))`, + where `m` is the total number of documents and `d(t)` is the number + of documents that contain term `t`. + + This implementation supports filtering out terms which do not appear + in a minimum number of documents (controlled by the variable `minDocFreq`). + For terms that are not in at least `minDocFreq` documents, the IDF is + found as 0, resulting in TF-IDFs of 0. + + >>> n = 4 + >>> freqs = [Vectors.sparse(n, (1, 3), (1.0, 2.0)), + ... Vectors.dense([0.0, 1.0, 2.0, 3.0]), + ... Vectors.sparse(n, [1], [1.0])] + >>> data = sc.parallelize(freqs) + >>> idf = IDF() + >>> model = idf.fit(data) + >>> tfidf = model.transform(data) + >>> for r in tfidf.collect(): r + SparseVector(4, {1: 0.0, 3: 0.5754}) + DenseVector([0.0, 0.0, 1.3863, 0.863]) + SparseVector(4, {1: 0.0}) + """ + def __init__(self, minDocFreq=0): + """ + :param minDocFreq: minimum of documents in which a term + should appear for filtering + """ + self.minDocFreq = minDocFreq + + def fit(self, dataset): + """ + Computes the inverse document frequency. + + :param dataset: an RDD of term frequency vectors + """ + sc = dataset.context + jmodel = _callAPI(sc, "fitIDF", self.minDocFreq, dataset) + return IDFModel(sc, jmodel) + + +class Word2VecModel(JavaModelWrapper): + """ + class for Word2Vec model + """ + def transform(self, word): + """ Transforms a word to its vector representation Note: local use only + + :param word: a word + :return: vector representation of word(s) """ - # TODO: make transform usable in RDD operations from python side - result = self._java_model.transform(word) - return PickleSerializer().loads(str(self._sc._jvm.SerDe.dumps(result))) + try: + return _callJavaFunc(self._sc, self._java_model.transform, word) + except Py4JJavaError: + raise ValueError("%s not found" % word) - def findSynonyms(self, x, num): + def findSynonyms(self, word, num): """ - :param x: a word or a vector representation of word + Find synonyms of a word + + :param word: a word or a vector representation of word :param num: number of synonyms to find :return: array of (word, cosineSimilarity) - Find synonyms of a word - Note: local use only """ - # TODO: make findSynonyms usable in RDD operations from python side - ser = PickleSerializer() - if type(x) == str: - jlist = self._java_model.findSynonyms(x, num) - else: - bytes = bytearray(ser.dumps(_convert_to_vector(x))) - vec = self._sc._jvm.SerDe.loads(bytes) - jlist = self._java_model.findSynonyms(vec, num) - words, similarity = ser.loads(str(self._sc._jvm.SerDe.dumps(jlist))) + words, similarity = _callJavaFunc(self._sc, self._java_model.findSynonyms, word, num) return zip(words, similarity) @@ -85,6 +383,7 @@ class Word2Vec(object): We used skip-gram model in our implementation and hierarchical softmax method to train the model. The variable names in the implementation matches the original C implementation. + For original C implementation, see https://code.google.com/p/word2vec/ For research papers, see Efficient Estimation of Word Representations in Vector Space @@ -95,33 +394,26 @@ class Word2Vec(object): >>> localDoc = [sentence, sentence] >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> syms = model.findSynonyms("a", 2) - >>> str(syms[0][0]) - 'b' - >>> str(syms[1][0]) - 'c' - >>> len(syms) - 2 + >>> [s[0] for s in syms] + [u'b', u'c'] >>> vec = model.transform("a") - >>> len(vec) - 10 >>> syms = model.findSynonyms(vec, 2) - >>> str(syms[0][0]) - 'b' - >>> str(syms[1][0]) - 'c' - >>> len(syms) - 2 + >>> [s[0] for s in syms] + [u'b', u'c'] """ def __init__(self): """ Construct Word2Vec instance """ + import random # this can't be on the top because of mllib.random + self.vectorSize = 100 self.learningRate = 0.025 self.numPartitions = 1 self.numIterations = 1 - self.seed = 42L + self.seed = random.randint(0, sys.maxint) def setVectorSize(self, vectorSize): """ @@ -164,20 +456,13 @@ def fit(self, data): Computes the vector representation of each word in vocabulary. :param data: training data. RDD of subtype of Iterable[String] - :return: python Word2VecModel instance + :return: Word2VecModel instance """ sc = data.context - ser = PickleSerializer() - vectorSize = self.vectorSize - learningRate = self.learningRate - numPartitions = self.numPartitions - numIterations = self.numIterations - seed = self.seed - - model = sc._jvm.PythonMLLibAPI().trainWord2Vec( - _to_java_object_rdd(data), vectorSize, - learningRate, numPartitions, numIterations, seed) - return Word2VecModel(sc, model) + jmodel = _callAPI(sc, "trainWord2Vec", data, int(self.vectorSize), + float(self.learningRate), int(self.numPartitions), + int(self.numIterations), long(self.seed)) + return Word2VecModel(sc, jmodel) def _test(): @@ -191,4 +476,8 @@ def _test(): exit(-1) if __name__ == "__main__": + # remove current path from list of search paths to avoid importing mllib.random + # for C{import random}, which is done in an external dependency of pyspark during doctests. + import sys + sys.path.pop(0) _test() diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 773d8d393805d..1b9bf596242df 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -111,6 +111,13 @@ def _vector_size(v): raise TypeError("Cannot treat type %s as a vector" % type(v)) +def _format_float(f, digits=4): + s = str(round(f, digits)) + if '.' in s: + s = s[:s.index('.') + 1 + digits] + return s + + class Vector(object): """ Abstract class for DenseVector and SparseVector @@ -228,7 +235,7 @@ def __str__(self): return "[" + ",".join([str(v) for v in self.array]) + "]" def __repr__(self): - return "DenseVector(%r)" % self.array + return "DenseVector([%s])" % (', '.join(_format_float(i) for i in self.array)) def __eq__(self, other): return isinstance(other, DenseVector) and self.array == other.array @@ -416,7 +423,7 @@ def toArray(self): Returns a copy of this SparseVector as a 1-dimensional NumPy array. """ arr = np.zeros((self.size,), dtype=np.float64) - for i in xrange(self.indices.size): + for i in xrange(len(self.indices)): arr[self.indices[i]] = self.values[i] return arr @@ -431,7 +438,8 @@ def __str__(self): def __repr__(self): inds = self.indices vals = self.values - entries = ", ".join(["{0}: {1}".format(inds[i], vals[i]) for i in xrange(len(inds))]) + entries = ", ".join(["{0}: {1}".format(inds[i], _format_float(vals[i])) + for i in xrange(len(inds))]) return "SparseVector({0}, {{{1}}})".format(self.size, entries) def __eq__(self, other): @@ -491,7 +499,7 @@ def dense(elements): returns a NumPy array. >>> Vectors.dense([1, 2, 3]) - DenseVector(array('d', [1.0, 2.0, 3.0])) + DenseVector([1.0, 2.0, 3.0]) """ return DenseVector(elements) From 47346cd029abc50c70582a721810a7cceb682d8a Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 08:46:31 -0500 Subject: [PATCH 1108/1492] [SPARK-4116][YARN]Delete the abandoned log4j-spark-container.properties Since its name reduced at https://github.com/apache/spark/pull/560, the log4j-spark-container.properties was never used again. And I have searched its name globally in code and found no cite. Author: WangTaoTheTonic Closes #2977 from WangTaoTheTonic/delLog4j and squashes the following commits: fb2729f [WangTaoTheTonic] delete the log4j file obsoleted --- .../log4j-spark-container.properties | 24 ------------------- 1 file changed, 24 deletions(-) delete mode 100644 yarn/common/src/main/resources/log4j-spark-container.properties diff --git a/yarn/common/src/main/resources/log4j-spark-container.properties b/yarn/common/src/main/resources/log4j-spark-container.properties deleted file mode 100644 index a1e37a0be27dd..0000000000000 --- a/yarn/common/src/main/resources/log4j-spark-container.properties +++ /dev/null @@ -1,24 +0,0 @@ -# -# Licensed 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. See accompanying LICENSE file. - -# Set everything to be logged to the console -log4j.rootCategory=INFO, console -log4j.appender.console=org.apache.log4j.ConsoleAppender -log4j.appender.console.target=System.err -log4j.appender.console.layout=org.apache.log4j.PatternLayout -log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n - -# Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO -log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO From e8813be6539aba1cd1f8854c204b7938464403ed Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 08:53:10 -0500 Subject: [PATCH 1109/1492] [SPARK-4095][YARN][Minor]extract val isLaunchingDriver in ClientBase Instead of checking if `args.userClass` is null repeatedly, we extract it to an global val as in `ApplicationMaster`. Author: WangTaoTheTonic Closes #2954 from WangTaoTheTonic/MemUnit and squashes the following commits: 13bda20 [WangTaoTheTonic] extract val isLaunchingDriver in ClientBase --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index fb0e34bf5985e..0417cdd00a393 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -55,6 +55,7 @@ private[spark] trait ClientBase extends Logging { protected val amMemoryOverhead = args.amMemoryOverhead // MB protected val executorMemoryOverhead = args.executorMemoryOverhead // MB private val distCacheMgr = new ClientDistributedCacheManager() + private val isLaunchingDriver = args.userClass != null /** * Fail fast if we have requested more resources per container than is available in the cluster. @@ -267,7 +268,6 @@ private[spark] trait ClientBase extends Logging { // Note that to warn the user about the deprecation in cluster mode, some code from // SparkConf#validateSettings() is duplicated here (to avoid triggering the condition // described above). - val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sys.env.get("SPARK_JAVA_OPTS").foreach { value => val warning = @@ -344,7 +344,6 @@ private[spark] trait ClientBase extends Logging { } // Include driver-specific java options if we are launching a driver - val isLaunchingDriver = args.userClass != null if (isLaunchingDriver) { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) @@ -357,7 +356,7 @@ private[spark] trait ClientBase extends Logging { javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) val userClass = - if (args.userClass != null) { + if (isLaunchingDriver) { Seq("--class", YarnSparkHadoopUtil.escapeForShell(args.userClass)) } else { Nil From 0ac52e30552530b247e37a470b8503346f19605c Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 09:51:44 -0500 Subject: [PATCH 1110/1492] [SPARK-4098][YARN]use appUIAddress instead of appUIHostPort in yarn-client mode https://issues.apache.org/jira/browse/SPARK-4098 Author: WangTaoTheTonic Closes #2958 from WangTaoTheTonic/useAddress and squashes the following commits: 29236e6 [WangTaoTheTonic] use appUIAddress instead of appUIHostPort in yarn-cluster mode --- .../spark/scheduler/cluster/YarnClientSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d948a2aeedd45..59b2b47aed2fe 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,7 +48,7 @@ private[spark] class YarnClientSchedulerBackend( val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort - sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIHostPort) } + sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.appUIAddress) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ("--arg", hostport) From 7768a800d4c4c19d76cef1ee40af6900bbac821c Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 28 Oct 2014 10:14:16 -0700 Subject: [PATCH 1111/1492] [SPARK-4031] Make torrent broadcast read blocks on use. This avoids reading torrent broadcast variables when they are referenced in the closure but not used in the closure. This is done by using a `lazy val` to read broadcast blocks cc rxin JoshRosen for review Author: Shivaram Venkataraman Closes #2871 from shivaram/broadcast-read-value and squashes the following commits: 1456d65 [Shivaram Venkataraman] Use getUsedTimeMs and remove readObject d6c5ee9 [Shivaram Venkataraman] Use laxy val to implement readBroadcastBlock 0b34df7 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/apache/spark into broadcast-read-value 9cec507 [Shivaram Venkataraman] Test if broadcast variables are read lazily 768b40b [Shivaram Venkataraman] Merge branch 'master' of https://github.com/apache/spark into broadcast-read-value 8792ed8 [Shivaram Venkataraman] Make torrent broadcast read blocks on use. This avoids reading broadcast variables when they are referenced in the closure but not used by the code. --- .../spark/broadcast/TorrentBroadcast.scala | 43 ++++++++++--------- .../scala/org/apache/spark/util/Utils.scala | 15 +++++++ .../spark/broadcast/BroadcastSuite.scala | 30 ++++++++++++- 3 files changed, 67 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 75e64c1bf401e..94142d33369c7 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -56,11 +56,13 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) extends Broadcast[T](id) with Logging with Serializable { /** - * Value of the broadcast object. On driver, this is set directly by the constructor. - * On executors, this is reconstructed by [[readObject]], which builds this value by reading - * blocks from the driver and/or other executors. + * Value of the broadcast object on executors. This is reconstructed by [[readBroadcastBlock]], + * which builds this value by reading blocks from the driver and/or other executors. + * + * On the driver, if the value is required, it is read lazily from the block manager. */ - @transient private var _value: T = obj + @transient private lazy val _value: T = readBroadcastBlock() + /** The compression codec to use, or None if compression is disabled */ @transient private var compressionCodec: Option[CompressionCodec] = _ /** Size of each block. Default value is 4MB. This value is only read by the broadcaster. */ @@ -79,22 +81,24 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) private val broadcastId = BroadcastBlockId(id) /** Total number of blocks this broadcast variable contains. */ - private val numBlocks: Int = writeBlocks() + private val numBlocks: Int = writeBlocks(obj) - override protected def getValue() = _value + override protected def getValue() = { + _value + } /** * Divide the object into multiple blocks and put those blocks in the block manager. - * + * @param value the object to divide * @return number of blocks this broadcast variable is divided into */ - private def writeBlocks(): Int = { + private def writeBlocks(value: T): Int = { // Store a copy of the broadcast variable in the driver so that tasks run on the driver // do not create a duplicate copy of the broadcast variable's value. - SparkEnv.get.blockManager.putSingle(broadcastId, _value, StorageLevel.MEMORY_AND_DISK, + SparkEnv.get.blockManager.putSingle(broadcastId, value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) val blocks = - TorrentBroadcast.blockifyObject(_value, blockSize, SparkEnv.get.serializer, compressionCodec) + TorrentBroadcast.blockifyObject(value, blockSize, SparkEnv.get.serializer, compressionCodec) blocks.zipWithIndex.foreach { case (block, i) => SparkEnv.get.blockManager.putBytes( BroadcastBlockId(id, "piece" + i), @@ -157,31 +161,30 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) out.defaultWriteObject() } - /** Used by the JVM when deserializing this object. */ - private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { - in.defaultReadObject() + private def readBroadcastBlock(): T = Utils.tryOrIOException { TorrentBroadcast.synchronized { setConf(SparkEnv.get.conf) SparkEnv.get.blockManager.getLocal(broadcastId).map(_.data.next()) match { case Some(x) => - _value = x.asInstanceOf[T] + x.asInstanceOf[T] case None => logInfo("Started reading broadcast variable " + id) - val start = System.nanoTime() + val startTimeMs = System.currentTimeMillis() val blocks = readBlocks() - val time = (System.nanoTime() - start) / 1e9 - logInfo("Reading broadcast variable " + id + " took " + time + " s") + logInfo("Reading broadcast variable " + id + " took" + Utils.getUsedTimeMs(startTimeMs)) - _value = - TorrentBroadcast.unBlockifyObject[T](blocks, SparkEnv.get.serializer, compressionCodec) + val obj = TorrentBroadcast.unBlockifyObject[T]( + blocks, SparkEnv.get.serializer, compressionCodec) // Store the merged copy in BlockManager so other tasks on this executor don't // need to re-fetch it. SparkEnv.get.blockManager.putSingle( - broadcastId, _value, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + broadcastId, obj, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + obj } } } + } 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 4660030155854..612eca308bf0b 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -988,6 +988,21 @@ private[spark] object Utils extends Logging { } } + /** + * Execute a block of code that returns a value, re-throwing any non-fatal uncaught + * exceptions as IOException. This is used when implementing Externalizable and Serializable's + * read and write methods, since Java's serializer will not report non-IOExceptions properly; + * see SPARK-4080 for more context. + */ + def tryOrIOException[T](block: => T): T = { + try { + block + } catch { + case e: IOException => throw e + case NonFatal(t) => throw new IOException(t) + } + } + /** Default filtering function for finding call sites using `getCallSite`. */ private def coreExclusionFunction(className: String): Boolean = { // A regular expression to match classes of the "core" Spark API that we want to skip when diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 1014fd62d9a75..b0a70f012f1f3 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -21,11 +21,28 @@ import scala.util.Random import org.scalatest.{Assertions, FunSuite} -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkEnv} import org.apache.spark.io.SnappyCompressionCodec +import org.apache.spark.rdd.RDD import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage._ +// Dummy class that creates a broadcast variable but doesn't use it +class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { + @transient val list = List(1, 2, 3, 4) + val broadcast = rdd.context.broadcast(list) + val bid = broadcast.id + + def doSomething() = { + rdd.map { x => + val bm = SparkEnv.get.blockManager + // Check if broadcast block was fetched + val isFound = bm.getLocal(BroadcastBlockId(bid)).isDefined + (x, isFound) + }.collect().toSet + } +} + class BroadcastSuite extends FunSuite with LocalSparkContext { private val httpConf = broadcastConf("HttpBroadcastFactory") @@ -105,6 +122,17 @@ class BroadcastSuite extends FunSuite with LocalSparkContext { } } + test("Test Lazy Broadcast variables with TorrentBroadcast") { + val numSlaves = 2 + val conf = torrentConf.clone + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", conf) + val rdd = sc.parallelize(1 to numSlaves) + + val results = new DummyBroadcastClass(rdd).doSomething() + + assert(results.toSet === (1 to numSlaves).map(x => (x, false)).toSet) + } + test("Unpersisting HttpBroadcast on executors only in local mode") { testUnpersistHttpBroadcast(distributed = false, removeFromDriver = false) } From 44d8b45a38c8d934628373a3b21084432516ee00 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:29:01 -0700 Subject: [PATCH 1112/1492] [SPARK-4110] Wrong comments about default settings in spark-daemon.sh In spark-daemon.sh, thare are following comments. # SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_PREFIX}/conf. # SPARK_LOG_DIR Where log files are stored. PWD by default. But, I think the default value for SPARK_CONF_DIR is `${SPARK_HOME}/conf` and for SPARK_LOG_DIR is `${SPARK_HOME}/logs`. Author: Kousuke Saruta Closes #2972 from sarutak/SPARK-4110 and squashes the following commits: 5a171a2 [Kousuke Saruta] Fixed wrong comments --- sbin/spark-daemon.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index cba475e2dd8c8..89608bc41b71d 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -21,8 +21,8 @@ # # Environment Variables # -# SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_PREFIX}/conf. -# SPARK_LOG_DIR Where log files are stored. PWD by default. +# SPARK_CONF_DIR Alternate conf dir. Default is ${SPARK_HOME}/conf. +# SPARK_LOG_DIR Where log files are stored. ${SPARK_HOME}/logs by default. # SPARK_MASTER host:path where spark code should be rsync'd from # SPARK_PID_DIR The pid files are stored. /tmp by default. # SPARK_IDENT_STRING A string representing this instance of spark. $USER by default From 1ea3e3dc9dd942402731751089bab2fb6ae29c7b Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 28 Oct 2014 12:31:42 -0700 Subject: [PATCH 1113/1492] [SPARK-4096][YARN]let ApplicationMaster accept executor memory argument in same format as JVM memory strings Here `ApplicationMaster` accept executor memory argument only in number format, we should let it accept JVM style memory strings as well. Author: WangTaoTheTonic Closes #2955 from WangTaoTheTonic/modifyDesc and squashes the following commits: ab98c70 [WangTaoTheTonic] append parameter passed in 3779767 [WangTaoTheTonic] Update executor memory description in the help message --- .../apache/spark/deploy/yarn/ApplicationMasterArguments.scala | 4 ++-- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 3e6b96fb63cea..5c54e3400301a 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 @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import org.apache.spark.util.IntParam +import org.apache.spark.util.{MemoryParam, IntParam} import collection.mutable.ArrayBuffer class ApplicationMasterArguments(val args: Array[String]) { @@ -55,7 +55,7 @@ class ApplicationMasterArguments(val args: Array[String]) { numExecutors = value args = tail - case ("--worker-memory" | "--executor-memory") :: IntParam(value) :: tail => + case ("--worker-memory" | "--executor-memory") :: MemoryParam(value) :: tail => executorMemory = value args = tail diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 0417cdd00a393..8ea0e7cf40a14 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -379,7 +379,7 @@ private[spark] trait ClientBase extends Logging { val amArgs = Seq(amClass) ++ userClass ++ userJar ++ userArgs ++ Seq( - "--executor-memory", args.executorMemory.toString, + "--executor-memory", args.executorMemory.toString + "m", "--executor-cores", args.executorCores.toString, "--num-executors ", args.numExecutors.toString) From 247c529b35d73c5a690bf6512b52999daba1a938 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:37:09 -0700 Subject: [PATCH 1114/1492] [SPARK-3657] yarn alpha YarnRMClientImpl throws NPE appMasterRequest.setTrackingUrl starting spark-shell tgravescs reported this issue. Following is quoted from tgravescs' report. YarnRMClientImpl.registerApplicationMaster can throw null pointer exception when setting the trackingurl if its empty: appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) I hit this just start spark-shell without the tracking url set. 14/09/23 16:18:34 INFO yarn.YarnRMClientImpl: Connecting to ResourceManager at kryptonitered-jt1.red.ygrid.yahoo.com/98.139.154.99:8030 Exception in thread "main" java.lang.NullPointerException at org.apache.hadoop.yarn.proto.YarnServiceProtos$RegisterApplicationMasterRequestProto$Builder.setTrackingUrl(YarnServiceProtos.java:710) at org.apache.hadoop.yarn.api.protocolrecords.impl.pb.RegisterApplicationMasterRequestPBImpl.setTrackingUrl(RegisterApplicationMasterRequestPBImpl.java:132) at org.apache.spark.deploy.yarn.YarnRMClientImpl.registerApplicationMaster(YarnRMClientImpl.scala:102) at org.apache.spark.deploy.yarn.YarnRMClientImpl.register(YarnRMClientImpl.scala:55) at org.apache.spark.deploy.yarn.YarnRMClientImpl.register(YarnRMClientImpl.scala:38) at org.apache.spark.deploy.yarn.ApplicationMaster.registerAM(ApplicationMaster.scala:168) at org.apache.spark.deploy.yarn.ApplicationMaster.runExecutorLauncher(ApplicationMaster.scala:206) at org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:120) Author: Kousuke Saruta Closes #2981 from sarutak/SPARK-3657-2 and squashes the following commits: e2fd6bc [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3657 70b8882 [Kousuke Saruta] Fixed NPE thrown --- .../org/apache/spark/deploy/yarn/YarnRMClientImpl.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala index 7faf55bc63372..e342cc82f454e 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.yarn import scala.collection.{Map, Set} -import java.net.URI; +import java.net.URI import org.apache.hadoop.net.NetUtils import org.apache.hadoop.yarn.api._ @@ -109,7 +109,9 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC appMasterRequest.setHost(Utils.localHostName()) appMasterRequest.setRpcPort(0) // remove the scheme from the url if it exists since Hadoop does not expect scheme - appMasterRequest.setTrackingUrl(new URI(uiAddress).getAuthority()) + val uri = new URI(uiAddress) + val authority = if (uri.getScheme == null) uiAddress else uri.getAuthority + appMasterRequest.setTrackingUrl(authority) resourceManager.registerApplicationMaster(appMasterRequest) } From 4d52cec21d98ee16066dcab0cc7959ccf4f412f3 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:44:12 -0700 Subject: [PATCH 1115/1492] [SPARK-4089][Doc][Minor] The version number of Spark in _config.yaml is wrong. The version number of Spark in docs/_config.yaml for master branch should be 1.2.0 for now. Author: Kousuke Saruta Closes #2943 from sarutak/SPARK-4089 and squashes the following commits: aba7fb4 [Kousuke Saruta] Fixed the version number of Spark in _config.yaml --- docs/_config.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/_config.yml b/docs/_config.yml index f4bf242ac191b..cdea02fcffbc5 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -11,10 +11,10 @@ kramdown: include: - _static -# These allow the documentation to be updated with nerw releases +# These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 1.0.0-SNAPSHOT -SPARK_VERSION_SHORT: 1.0.0 +SPARK_VERSION: 1.2.0-SNAPSHOT +SPARK_VERSION_SHORT: 1.2.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" MESOS_VERSION: 0.18.1 From 2f254dacf4b7ab9c59c7cef59fd364ca682162ae Mon Sep 17 00:00:00 2001 From: Michael Griffiths Date: Tue, 28 Oct 2014 12:47:21 -0700 Subject: [PATCH 1116/1492] [SPARK-4065] Add check for IPython on Windows This issue employs logic similar to the bash launcher (pyspark) to check if IPTYHON=1, and if so launch ipython with options in IPYTHON_OPTS. This fix assumes that ipython is available in the system Path, and can be invoked with a plain "ipython" command. Author: Michael Griffiths Closes #2910 from msjgriffiths/pyspark-windows and squashes the following commits: ef34678 [Michael Griffiths] Change build message to comply with [SPARK-3775] 361e3d8 [Michael Griffiths] [SPARK-4065] Add check for IPython on Windows 9ce72d1 [Michael Griffiths] [SPARK-4065] Add check for IPython on Windows --- bin/pyspark2.cmd | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index a0e66abcc26c9..59415e9bdec2c 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -59,7 +59,12 @@ for /f %%i in ('echo %1^| findstr /R "\.py"') do ( ) if [%PYTHON_FILE%] == [] ( - %PYSPARK_PYTHON% + set PYSPARK_SHELL=1 + if [%IPYTHON%] == [1] ( + ipython %IPYTHON_OPTS% + ) else ( + %PYSPARK_PYTHON% + ) ) else ( echo. echo WARNING: Running python applications through ./bin/pyspark.cmd is deprecated as of Spark 1.0. From 6c1b981c3fad671bff4795f061bd40e111956621 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 28 Oct 2014 12:58:25 -0700 Subject: [PATCH 1117/1492] [SPARK-4058] [PySpark] Log file name is hard coded even though there is a variable '$LOG_FILE ' In a script 'python/run-tests', log file name is represented by a variable 'LOG_FILE' and it is used in run-tests. But, there are some hard-coded log file name in the script. Author: Kousuke Saruta Closes #2905 from sarutak/SPARK-4058 and squashes the following commits: 7710490 [Kousuke Saruta] Fixed python/run-tests not to use hard-coded log file name --- python/run-tests | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/run-tests b/python/run-tests index 80acd002ab7eb..a4f0cac059ff3 100755 --- a/python/run-tests +++ b/python/run-tests @@ -41,7 +41,7 @@ function run_test() { # Fail and exit on the first test failure. if [[ $FAILED != 0 ]]; then - cat unit-tests.log | grep -v "^[0-9][0-9]*" # filter all lines starting with a number. + cat $LOG_FILE | grep -v "^[0-9][0-9]*" # filter all lines starting with a number. echo -en "\033[31m" # Red echo "Had test failures; see logs." echo -en "\033[0m" # No color @@ -87,7 +87,7 @@ function run_streaming_tests() { run_test "pyspark/streaming/tests.py" } -echo "Running PySpark tests. Output is in python/unit-tests.log." +echo "Running PySpark tests. Output is in python/$LOG_FILE." export PYSPARK_PYTHON="python" From 5807cb40ae178f0395c71b967f02aee853ef8bc9 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Tue, 28 Oct 2014 13:36:06 -0700 Subject: [PATCH 1118/1492] [SPARK-3814][SQL] Support for Bitwise AND(&), OR(|) ,XOR(^), NOT(~) in Spark HQL and SQL Currently there is no support of Bitwise & , | in Spark HiveQl and Spark SQL as well. So this PR support the same. I am closing https://github.com/apache/spark/pull/2926 as it has conflicts to merge. And also added support for Bitwise AND(&), OR(|) ,XOR(^), NOT(~) And I handled all review comments in that PR Author: ravipesala Closes #2961 from ravipesala/SPARK-3814-NEW4 and squashes the following commits: a391c7a [ravipesala] Rebase with master --- .../spark/sql/catalyst/SparkSQLParser.scala | 2 +- .../apache/spark/sql/catalyst/SqlParser.scala | 4 + .../spark/sql/catalyst/dsl/package.scala | 4 + .../sql/catalyst/expressions/arithmetic.scala | 89 +++++++++++++++++++ .../ExpressionEvaluationSuite.scala | 32 +++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 16 ++++ .../org/apache/spark/sql/hive/HiveQl.scala | 4 + .../sql/hive/execution/SQLQuerySuite.scala | 24 +++++ 8 files changed, 174 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala index 219322c015bbf..12e8346a6445d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -61,7 +61,7 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":", "[", "]", "." + ",", ";", "%", "{", "}", ":", "[", "]", ".", "&", "|", "^", "~" ) override lazy val token: Parser[Token] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 4e967713ede64..0acf7252ba3f0 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -260,6 +260,9 @@ class SqlParser extends AbstractSparkSQLParser { ( "*" ^^^ { (e1: Expression, e2: Expression) => Multiply(e1, e2) } | "/" ^^^ { (e1: Expression, e2: Expression) => Divide(e1, e2) } | "%" ^^^ { (e1: Expression, e2: Expression) => Remainder(e1, e2) } + | "&" ^^^ { (e1: Expression, e2: Expression) => BitwiseAnd(e1, e2) } + | "|" ^^^ { (e1: Expression, e2: Expression) => BitwiseOr(e1, e2) } + | "^" ^^^ { (e1: Expression, e2: Expression) => BitwiseXor(e1, e2) } ) protected lazy val function: Parser[Expression] = @@ -370,6 +373,7 @@ class SqlParser extends AbstractSparkSQLParser { | dotExpressionHeader | ident ^^ UnresolvedAttribute | signedPrimary + | "~" ~> expression ^^ BitwiseNot ) protected lazy val dotExpressionHeader: Parser[Expression] = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 75b6e37c2a1f9..23cfd483ec410 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -62,12 +62,16 @@ package object dsl { def unary_- = UnaryMinus(expr) def unary_! = Not(expr) + def unary_~ = BitwiseNot(expr) def + (other: Expression) = Add(expr, other) def - (other: Expression) = Subtract(expr, other) def * (other: Expression) = Multiply(expr, other) def / (other: Expression) = Divide(expr, other) def % (other: Expression) = Remainder(expr, other) + def & (other: Expression) = BitwiseAnd(expr, other) + def | (other: Expression) = BitwiseOr(expr, other) + def ^ (other: Expression) = BitwiseXor(expr, other) def && (other: Expression) = And(expr, other) def || (other: Expression) = Or(expr, other) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index fe825fdcdae37..83e8466ec2aa7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -64,6 +64,23 @@ abstract class BinaryArithmetic extends BinaryExpression { } left.dataType } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + if(evalE1 == null) { + null + } else { + val evalE2 = right.eval(input) + if (evalE2 == null) { + null + } else { + evalInternal(evalE1, evalE2) + } + } + } + + def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = + sys.error(s"BinaryExpressions must either override eval or evalInternal") } case class Add(left: Expression, right: Expression) extends BinaryArithmetic { @@ -100,6 +117,78 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } +/** + * A function that calculates bitwise and(&) of two numbers. + */ +case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic { + def symbol = "&" + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { + case ByteType => (evalE1.asInstanceOf[Byte] & evalE2.asInstanceOf[Byte]).toByte + case ShortType => (evalE1.asInstanceOf[Short] & evalE2.asInstanceOf[Short]).toShort + case IntegerType => evalE1.asInstanceOf[Int] & evalE2.asInstanceOf[Int] + case LongType => evalE1.asInstanceOf[Long] & evalE2.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise & operation on ${other}") + } +} + +/** + * A function that calculates bitwise or(|) of two numbers. + */ +case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic { + def symbol = "&" + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { + case ByteType => (evalE1.asInstanceOf[Byte] | evalE2.asInstanceOf[Byte]).toByte + case ShortType => (evalE1.asInstanceOf[Short] | evalE2.asInstanceOf[Short]).toShort + case IntegerType => evalE1.asInstanceOf[Int] | evalE2.asInstanceOf[Int] + case LongType => evalE1.asInstanceOf[Long] | evalE2.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise | operation on ${other}") + } +} + +/** + * A function that calculates bitwise xor(^) of two numbers. + */ +case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { + def symbol = "^" + + override def evalInternal(evalE1: EvaluatedType, evalE2: EvaluatedType): Any = dataType match { + case ByteType => (evalE1.asInstanceOf[Byte] ^ evalE2.asInstanceOf[Byte]).toByte + case ShortType => (evalE1.asInstanceOf[Short] ^ evalE2.asInstanceOf[Short]).toShort + case IntegerType => evalE1.asInstanceOf[Int] ^ evalE2.asInstanceOf[Int] + case LongType => evalE1.asInstanceOf[Long] ^ evalE2.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise ^ operation on ${other}") + } +} + +/** + * A function that calculates bitwise not(~) of a number. + */ +case class BitwiseNot(child: Expression) extends UnaryExpression { + type EvaluatedType = Any + + def dataType = child.dataType + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"-$child" + + override def eval(input: Row): Any = { + val evalE = child.eval(input) + if (evalE == null) { + null + } else { + dataType match { + case ByteType => (~(evalE.asInstanceOf[Byte])).toByte + case ShortType => (~(evalE.asInstanceOf[Short])).toShort + case IntegerType => ~(evalE.asInstanceOf[Int]) + case LongType => ~(evalE.asInstanceOf[Long]) + case other => sys.error(s"Unsupported bitwise ~ operation on ${other}") + } + } + } +} + case class MaxOf(left: Expression, right: Expression) extends Expression { type EvaluatedType = Any 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 53c53481f984e..5657bc555edf9 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 @@ -680,4 +680,36 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Sqrt(Literal(null, DoubleType)), null, new GenericRow(Array[Any](null))) } + + test("Bitwise operations") { + val row = new GenericRow(Array[Any](1, 2, 3, null)) + val c1 = 'a.int.at(0) + val c2 = 'a.int.at(1) + val c3 = 'a.int.at(2) + val c4 = 'a.int.at(3) + + checkEvaluation(BitwiseAnd(c1, c4), null, row) + checkEvaluation(BitwiseAnd(c1, c2), 0, row) + checkEvaluation(BitwiseAnd(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseAnd(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(BitwiseOr(c1, c4), null, row) + checkEvaluation(BitwiseOr(c1, c2), 3, row) + checkEvaluation(BitwiseOr(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseOr(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(BitwiseXor(c1, c4), null, row) + checkEvaluation(BitwiseXor(c1, c2), 3, row) + checkEvaluation(BitwiseXor(c1, Literal(null, IntegerType)), null, row) + checkEvaluation(BitwiseXor(Literal(null, IntegerType), Literal(null, IntegerType)), null, row) + + checkEvaluation(BitwiseNot(c4), null, row) + checkEvaluation(BitwiseNot(c1), -2, row) + checkEvaluation(BitwiseNot(Literal(null, IntegerType)), null, row) + + checkEvaluation(c1 & c2, 0, row) + checkEvaluation(c1 | c2, 3, row) + checkEvaluation(c1 ^ c2, 3, row) + checkEvaluation(~c1, -2, row) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 12e1cfc1cb7eb..1034c2d05f8cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -883,4 +883,20 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { jsonRDD(data).registerTempTable("records") sql("SELECT `key?number1` FROM records") } + + test("SPARK-3814 Support Bitwise & operator") { + checkAnswer(sql("SELECT key&1 FROM testData WHERE key = 1 "), 1) + } + + test("SPARK-3814 Support Bitwise | operator") { + checkAnswer(sql("SELECT key|0 FROM testData WHERE key = 1 "), 1) + } + + test("SPARK-3814 Support Bitwise ^ operator") { + checkAnswer(sql("SELECT key^0 FROM testData WHERE key = 1 "), 1) + } + + test("SPARK-3814 Support Bitwise ~ operator") { + checkAnswer(sql("SELECT ~key FROM testData WHERE key = 1 "), -2) + } } 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 aa80b2f04de25..ed07a280398bd 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 @@ -956,6 +956,7 @@ private[hive] object HiveQl { /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) + case Token("~", child :: Nil) => BitwiseNot(nodeToExpr(child)) case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right)) case Token("-", left :: right:: Nil) => Subtract(nodeToExpr(left), nodeToExpr(right)) case Token("*", left :: right:: Nil) => Multiply(nodeToExpr(left), nodeToExpr(right)) @@ -963,6 +964,9 @@ private[hive] object HiveQl { case Token(DIV(), left :: right:: Nil) => Cast(Divide(nodeToExpr(left), nodeToExpr(right)), LongType) case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) + case Token("&", left :: right:: Nil) => BitwiseAnd(nodeToExpr(left), nodeToExpr(right)) + case Token("|", left :: right:: Nil) => BitwiseOr(nodeToExpr(left), nodeToExpr(right)) + case Token("^", left :: right:: Nil) => BitwiseXor(nodeToExpr(left), nodeToExpr(right)) case Token("TOK_FUNCTION", Token(SQRT(), Nil) :: arg :: Nil) => Sqrt(nodeToExpr(arg)) /* Comparisons */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index fbe6ac765c009..a4aea31d3ff9e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -75,4 +75,28 @@ class SQLQuerySuite extends QueryTest { sql("SELECT a.key FROM (SELECT key FROM src) `a`"), sql("SELECT `key` FROM src").collect().toSeq) } + + test("SPARK-3814 Support Bitwise & operator") { + checkAnswer( + sql("SELECT case when 1&1=1 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } + + test("SPARK-3814 Support Bitwise | operator") { + checkAnswer( + sql("SELECT case when 1|0=1 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } + + test("SPARK-3814 Support Bitwise ^ operator") { + checkAnswer( + sql("SELECT case when 1^0=1 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } + + test("SPARK-3814 Support Bitwise ~ operator") { + checkAnswer( + sql("SELECT case when ~1=-2 then 1 else 0 end FROM src"), + sql("SELECT 1 FROM src").collect().toSeq) + } } From 47a40f60d62ea69b659959994918d4c640f39d5b Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 28 Oct 2014 13:43:25 -0700 Subject: [PATCH 1119/1492] [SPARK-3988][SQL] add public API for date type Add json and python api for date type. By using Pickle, `java.sql.Date` was serialized as calendar, and recognized in python as `datetime.datetime`. Author: Daoyuan Wang Closes #2901 from adrian-wang/spark3988 and squashes the following commits: c51a24d [Daoyuan Wang] convert datetime to date 5670626 [Daoyuan Wang] minor line combine f760d8e [Daoyuan Wang] fix indent 444f100 [Daoyuan Wang] fix a typo 1d74448 [Daoyuan Wang] fix scala style 8d7dd22 [Daoyuan Wang] add json and python api for date type --- python/pyspark/sql.py | 57 +++++++++++++------ .../spark/sql/catalyst/ScalaReflection.scala | 1 + .../spark/sql/catalyst/types/dataTypes.scala | 4 +- .../sql/catalyst/ScalaReflectionSuite.scala | 9 ++- .../org/apache/spark/sql/SQLContext.scala | 10 +++- .../org/apache/spark/sql/json/JsonRDD.scala | 20 +++++-- .../spark/sql/api/java/JavaRowSuite.java | 11 +++- .../java/JavaSideDataTypeConversionSuite.java | 1 + .../ScalaSideDataTypeConversionSuite.scala | 1 + .../org/apache/spark/sql/json/JsonSuite.scala | 9 ++- 10 files changed, 87 insertions(+), 36 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 7daf306f68479..93fd9d49096b8 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -49,7 +49,7 @@ __all__ = [ - "StringType", "BinaryType", "BooleanType", "TimestampType", "DecimalType", + "StringType", "BinaryType", "BooleanType", "DateType", "TimestampType", "DecimalType", "DoubleType", "FloatType", "ByteType", "IntegerType", "LongType", "ShortType", "ArrayType", "MapType", "StructField", "StructType", "SQLContext", "HiveContext", "SchemaRDD", "Row"] @@ -132,6 +132,14 @@ class BooleanType(PrimitiveType): """ +class DateType(PrimitiveType): + + """Spark SQL DateType + + The data type representing datetime.date values. + """ + + class TimestampType(PrimitiveType): """Spark SQL TimestampType @@ -438,7 +446,7 @@ def _parse_datatype_json_value(json_value): return _all_complex_types[json_value["type"]].fromJson(json_value) -# Mapping Python types to Spark SQL DateType +# Mapping Python types to Spark SQL DataType _type_mappings = { bool: BooleanType, int: IntegerType, @@ -448,8 +456,8 @@ def _parse_datatype_json_value(json_value): unicode: StringType, bytearray: BinaryType, decimal.Decimal: DecimalType, + datetime.date: DateType, datetime.datetime: TimestampType, - datetime.date: TimestampType, datetime.time: TimestampType, } @@ -656,10 +664,10 @@ def _infer_schema_type(obj, dataType): """ Fill the dataType with types infered from obj - >>> schema = _parse_schema_abstract("a b c") - >>> row = (1, 1.0, "str") + >>> schema = _parse_schema_abstract("a b c d") + >>> row = (1, 1.0, "str", datetime.date(2014, 10, 10)) >>> _infer_schema_type(row, schema) - StructType...IntegerType...DoubleType...StringType... + StructType...IntegerType...DoubleType...StringType...DateType... >>> row = [[1], {"key": (1, 2.0)}] >>> schema = _parse_schema_abstract("a[] b{c d}") >>> _infer_schema_type(row, schema) @@ -703,6 +711,7 @@ def _infer_schema_type(obj, dataType): DecimalType: (decimal.Decimal,), StringType: (str, unicode), BinaryType: (bytearray,), + DateType: (datetime.date,), TimestampType: (datetime.datetime,), ArrayType: (list, tuple, array), MapType: (dict,), @@ -740,7 +749,7 @@ def _verify_type(obj, dataType): # subclass of them can not be deserialized in JVM if type(obj) not in _acceptable_types[_type]: - raise TypeError("%s can not accept abject in type %s" + raise TypeError("%s can not accept object in type %s" % (dataType, type(obj))) if isinstance(dataType, ArrayType): @@ -767,7 +776,7 @@ def _restore_object(dataType, obj): """ Restore object during unpickling. """ # use id(dataType) as key to speed up lookup in dict # Because of batched pickling, dataType will be the - # same object in mose cases. + # same object in most cases. k = id(dataType) cls = _cached_cls.get(k) if cls is None: @@ -782,6 +791,10 @@ def _restore_object(dataType, obj): def _create_object(cls, v): """ Create an customized object with class `cls`. """ + # datetime.date would be deserialized as datetime.datetime + # from java type, so we need to set it back. + if cls is datetime.date and isinstance(v, datetime.datetime): + return v.date() return cls(v) if v is not None else v @@ -795,14 +808,16 @@ def getter(self): return getter -def _has_struct(dt): - """Return whether `dt` is or has StructType in it""" +def _has_struct_or_date(dt): + """Return whether `dt` is or has StructType/DateType in it""" if isinstance(dt, StructType): return True elif isinstance(dt, ArrayType): - return _has_struct(dt.elementType) + return _has_struct_or_date(dt.elementType) elif isinstance(dt, MapType): - return _has_struct(dt.valueType) + return _has_struct_or_date(dt.valueType) + elif isinstance(dt, DateType): + return True return False @@ -815,7 +830,7 @@ def _create_properties(fields): or keyword.iskeyword(name)): warnings.warn("field name %s can not be accessed in Python," "use position to access it instead" % name) - if _has_struct(f.dataType): + if _has_struct_or_date(f.dataType): # delay creating object until accessing it getter = _create_getter(f.dataType, i) else: @@ -870,6 +885,9 @@ def Dict(d): return Dict + elif isinstance(dataType, DateType): + return datetime.date + elif not isinstance(dataType, StructType): raise Exception("unexpected data type: %s" % dataType) @@ -1068,8 +1086,9 @@ def applySchema(self, rdd, schema): >>> srdd2.collect() [Row(field1=1, field2=u'row1'),..., Row(field1=3, field2=u'row3')] - >>> from datetime import datetime + >>> from datetime import date, datetime >>> rdd = sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + ... date(2010, 1, 1), ... datetime(2010, 1, 1, 1, 1, 1), ... {"a": 1}, (2,), [1, 2, 3], None)]) >>> schema = StructType([ @@ -1079,6 +1098,7 @@ def applySchema(self, rdd, schema): ... StructField("short2", ShortType(), False), ... StructField("int", IntegerType(), False), ... StructField("float", FloatType(), False), + ... StructField("date", DateType(), False), ... StructField("time", TimestampType(), False), ... StructField("map", ... MapType(StringType(), IntegerType(), False), False), @@ -1088,10 +1108,11 @@ def applySchema(self, rdd, schema): ... StructField("null", DoubleType(), True)]) >>> srdd = sqlCtx.applySchema(rdd, schema) >>> results = srdd.map( - ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.time, - ... x.map["a"], x.struct.b, x.list, x.null)) - >>> results.collect()[0] - (127, -128, -32768, 32767, 2147483647, 1.0, ...(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) + ... lambda x: (x.byte1, x.byte2, x.short1, x.short2, x.int, x.float, x.date, + ... x.time, x.map["a"], x.struct.b, x.list, x.null)) + >>> results.collect()[0] # doctest: +NORMALIZE_WHITESPACE + (127, -128, -32768, 32767, 2147483647, 1.0, datetime.date(2010, 1, 1), + datetime.datetime(2010, 1, 1, 1, 1, 1), 1, 2, [1, 2, 3], None) >>> srdd.registerTempTable("table2") >>> sqlCtx.sql( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 7d930fccd52d1..d76c743d3f652 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -112,6 +112,7 @@ object ScalaReflection { case obj: FloatType.JvmType => FloatType case obj: DoubleType.JvmType => DoubleType case obj: DecimalType.JvmType => DecimalType + case obj: DateType.JvmType => DateType case obj: TimestampType.JvmType => TimestampType case null => NullType // For other cases, there is no obvious mapping from the type of the given object to a 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 0cf139ebde417..b9cf37d53ffd2 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 @@ -91,6 +91,7 @@ object DataType { | "BinaryType" ^^^ BinaryType | "BooleanType" ^^^ BooleanType | "DecimalType" ^^^ DecimalType + | "DateType" ^^^ DateType | "TimestampType" ^^^ TimestampType ) @@ -198,7 +199,8 @@ trait PrimitiveType extends DataType { } object PrimitiveType { - private[sql] val all = Seq(DecimalType, TimestampType, BinaryType) ++ NativeType.all + private[sql] val all = Seq(DecimalType, DateType, TimestampType, BinaryType) ++ + NativeType.all private[sql] val nameToType = all.map(t => t.typeName -> t).toMap } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 488e373854bb3..430f0664b7d58 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst import java.math.BigInteger -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import org.scalatest.FunSuite @@ -43,6 +43,7 @@ case class NullableData( booleanField: java.lang.Boolean, stringField: String, decimalField: BigDecimal, + dateField: Date, timestampField: Timestamp, binaryField: Array[Byte]) @@ -96,6 +97,7 @@ class ScalaReflectionSuite extends FunSuite { StructField("booleanField", BooleanType, nullable = true), StructField("stringField", StringType, nullable = true), StructField("decimalField", DecimalType, nullable = true), + StructField("dateField", DateType, nullable = true), StructField("timestampField", TimestampType, nullable = true), StructField("binaryField", BinaryType, nullable = true))), nullable = true)) @@ -199,8 +201,11 @@ class ScalaReflectionSuite extends FunSuite { // DecimalType assert(DecimalType === typeOfObject(BigDecimal("1.7976931348623157E318"))) + // DateType + assert(DateType === typeOfObject(Date.valueOf("2014-07-25"))) + // TimestampType - assert(TimestampType === typeOfObject(java.sql.Timestamp.valueOf("2014-07-25 10:26:00"))) + assert(TimestampType === typeOfObject(Timestamp.valueOf("2014-07-25 10:26:00"))) // NullType assert(NullType === typeOfObject(null)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c4f4ef01d78df..ca8706ee68697 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -444,6 +444,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case ByteType => true case ShortType => true case FloatType => true + case DateType => true case TimestampType => true case ArrayType(_, _) => true case MapType(_, _, _) => true @@ -452,9 +453,9 @@ class SQLContext(@transient val sparkContext: SparkContext) } // Converts value to the type specified by the data type. - // Because Python does not have data types for TimestampType, FloatType, ShortType, and - // ByteType, we need to explicitly convert values in columns of these data types to the desired - // JVM data types. + // Because Python does not have data types for DateType, TimestampType, FloatType, ShortType, + // and ByteType, we need to explicitly convert values in columns of these data types to the + // desired JVM data types. def convert(obj: Any, dataType: DataType): Any = (obj, dataType) match { // TODO: We should check nullable case (null, _) => null @@ -474,6 +475,9 @@ class SQLContext(@transient val sparkContext: SparkContext) case (e, f) => convert(e, f.dataType) }): Row + case (c: java.util.Calendar, DateType) => + new java.sql.Date(c.getTime().getTime()) + case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index bf32da1b7181e..047dc85df6c1d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.json import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import com.fasterxml.jackson.core.JsonProcessingException import com.fasterxml.jackson.databind.ObjectMapper @@ -372,13 +372,20 @@ private[sql] object JsonRDD extends Logging { } } + private def toDate(value: Any): Date = { + value match { + // only support string as date + case value: java.lang.String => Date.valueOf(value) + } + } + private def toTimestamp(value: Any): Timestamp = { value match { - case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) - case value: java.lang.Long => new Timestamp(value) - case value: java.lang.String => Timestamp.valueOf(value) - } - } + case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) + case value: java.lang.Long => new Timestamp(value) + case value: java.lang.String => Timestamp.valueOf(value) + } + } private[json] def enforceCorrectType(value: Any, desiredType: DataType): Any ={ if (value == null) { @@ -396,6 +403,7 @@ private[sql] object JsonRDD extends Logging { case ArrayType(elementType, _) => value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case struct: StructType => asRow(value.asInstanceOf[Map[String, Any]], struct) + case DateType => toDate(value) case TimestampType => toTimestamp(value) } } diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java index 52d07b5425cc3..bc5cd66482add 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaRowSuite.java @@ -18,6 +18,7 @@ package org.apache.spark.sql.api.java; import java.math.BigDecimal; +import java.sql.Date; import java.sql.Timestamp; import java.util.Arrays; import java.util.HashMap; @@ -39,6 +40,7 @@ public class JavaRowSuite { private boolean booleanValue; private String stringValue; private byte[] binaryValue; + private Date dateValue; private Timestamp timestampValue; @Before @@ -53,6 +55,7 @@ public void setUp() { booleanValue = true; stringValue = "this is a string"; binaryValue = stringValue.getBytes(); + dateValue = Date.valueOf("2014-06-30"); timestampValue = Timestamp.valueOf("2014-06-30 09:20:00.0"); } @@ -76,6 +79,7 @@ public void constructSimpleRow() { new Boolean(booleanValue), stringValue, // StringType binaryValue, // BinaryType + dateValue, // DateType timestampValue, // TimestampType null // null ); @@ -114,9 +118,10 @@ public void constructSimpleRow() { Assert.assertEquals(stringValue, simpleRow.getString(15)); Assert.assertEquals(stringValue, simpleRow.get(15)); Assert.assertEquals(binaryValue, simpleRow.get(16)); - Assert.assertEquals(timestampValue, simpleRow.get(17)); - Assert.assertEquals(true, simpleRow.isNullAt(18)); - Assert.assertEquals(null, simpleRow.get(18)); + Assert.assertEquals(dateValue, simpleRow.get(17)); + Assert.assertEquals(timestampValue, simpleRow.get(18)); + Assert.assertEquals(true, simpleRow.isNullAt(19)); + Assert.assertEquals(null, simpleRow.get(19)); } @Test diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java index d099a48a1f4b6..d04396a5f8ec2 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java @@ -39,6 +39,7 @@ public void createDataTypes() { checkDataType(DataType.StringType); checkDataType(DataType.BinaryType); checkDataType(DataType.BooleanType); + checkDataType(DataType.DateType); checkDataType(DataType.TimestampType); checkDataType(DataType.DecimalType); checkDataType(DataType.DoubleType); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala index ff1debff0f8c1..8415af41be3af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala @@ -38,6 +38,7 @@ class ScalaSideDataTypeConversionSuite extends FunSuite { checkDataType(org.apache.spark.sql.StringType) checkDataType(org.apache.spark.sql.BinaryType) checkDataType(org.apache.spark.sql.BooleanType) + checkDataType(org.apache.spark.sql.DateType) checkDataType(org.apache.spark.sql.TimestampType) checkDataType(org.apache.spark.sql.DecimalType) checkDataType(org.apache.spark.sql.DoubleType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 1ae75546aada1..ce6184f5d8c9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.SQLConf import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ -import java.sql.Timestamp +import java.sql.{Date, Timestamp} class JsonSuite extends QueryTest { import TestJsonData._ @@ -58,8 +58,11 @@ class JsonSuite extends QueryTest { checkTypePromotion(new Timestamp(intNumber), enforceCorrectType(intNumber, TimestampType)) checkTypePromotion(new Timestamp(intNumber.toLong), enforceCorrectType(intNumber.toLong, TimestampType)) - val strDate = "2014-09-30 12:34:56" - checkTypePromotion(Timestamp.valueOf(strDate), enforceCorrectType(strDate, TimestampType)) + val strTime = "2014-09-30 12:34:56" + checkTypePromotion(Timestamp.valueOf(strTime), enforceCorrectType(strTime, TimestampType)) + + val strDate = "2014-10-15" + checkTypePromotion(Date.valueOf(strDate), enforceCorrectType(strDate, DateType)) } test("Get compatible type") { From abcafcfba38d7c8dba68a5510475c5c49ae54d92 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 28 Oct 2014 14:26:57 -0700 Subject: [PATCH 1120/1492] [Spark 3922] Refactor spark-core to use Utils.UTF_8 A global UTF8 constant is very helpful to handle encoding problems when converting between String and bytes. There are several solutions here: 1. Add `val UTF_8 = Charset.forName("UTF-8")` to Utils.scala 2. java.nio.charset.StandardCharsets.UTF_8 (require JDK7) 3. io.netty.util.CharsetUtil.UTF_8 4. com.google.common.base.Charsets.UTF_8 5. org.apache.commons.lang.CharEncoding.UTF_8 6. org.apache.commons.lang3.CharEncoding.UTF_8 IMO, I prefer option 1) because people can find it easily. This is a PR for option 1) and only fixes Spark Core. Author: zsxwing Closes #2781 from zsxwing/SPARK-3922 and squashes the following commits: f974edd [zsxwing] Merge branch 'master' into SPARK-3922 2d27423 [zsxwing] Refactor spark-core to use Refactor spark-core to use Utils.UTF_8 --- .../scala/org/apache/spark/SparkSaslClient.scala | 7 ++++--- .../scala/org/apache/spark/SparkSaslServer.scala | 10 ++++++---- .../org/apache/spark/api/python/PythonRDD.scala | 9 ++++----- .../python/WriteInputFormatTestDataGenerator.scala | 5 +++-- .../apache/spark/deploy/worker/DriverRunner.scala | 4 ++-- .../apache/spark/deploy/worker/ExecutorRunner.scala | 4 ++-- .../network/netty/client/BlockFetchingClient.scala | 4 ++-- .../netty/client/BlockFetchingClientHandler.scala | 5 +++-- .../spark/network/netty/server/BlockServer.scala | 4 ++-- .../netty/server/BlockServerChannelInitializer.scala | 6 +++--- .../apache/spark/network/nio/ConnectionManager.scala | 4 +++- .../scala/org/apache/spark/network/nio/Message.scala | 4 +++- .../client/BlockFetchingClientHandlerSuite.scala | 3 ++- .../netty/server/BlockHeaderEncoderSuite.scala | 8 ++++---- .../org/apache/spark/util/FileAppenderSuite.scala | 12 ++++++------ .../scala/org/apache/spark/util/UtilsSuite.scala | 12 ++++++------ 16 files changed, 55 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index 65003b6ac6a0a..a954fcc0c31fa 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -17,7 +17,6 @@ package org.apache.spark -import java.io.IOException import javax.security.auth.callback.Callback import javax.security.auth.callback.CallbackHandler import javax.security.auth.callback.NameCallback @@ -31,6 +30,8 @@ import javax.security.sasl.SaslException import scala.collection.JavaConversions.mapAsJavaMap +import com.google.common.base.Charsets.UTF_8 + /** * Implements SASL Client logic for Spark */ @@ -111,10 +112,10 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes(UTF_8)) private val secretKey = securityMgr.getSecretKey() private val userPassword: Array[Char] = SparkSaslServer.encodePassword( - if (secretKey != null) secretKey.getBytes("utf-8") else "".getBytes("utf-8")) + if (secretKey != null) secretKey.getBytes(UTF_8) else "".getBytes(UTF_8)) /** * Implementation used to respond to SASL request from the server. diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index f6b0a9132aca4..7c2afb364661f 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -28,6 +28,8 @@ import javax.security.sasl.Sasl import javax.security.sasl.SaslException import javax.security.sasl.SaslServer import scala.collection.JavaConversions.mapAsJavaMap + +import com.google.common.base.Charsets.UTF_8 import org.apache.commons.net.util.Base64 /** @@ -89,7 +91,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi extends CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes(UTF_8)) override def handle(callbacks: Array[Callback]) { logDebug("In the sasl server callback handler") @@ -101,7 +103,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi case pc: PasswordCallback => { logDebug("handle: SASL server callback: setting userPassword") val password: Array[Char] = - SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes("utf-8")) + SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes(UTF_8)) pc.setPassword(password) } case rc: RealmCallback => { @@ -159,7 +161,7 @@ private[spark] object SparkSaslServer { * @return Base64-encoded string */ def encodeIdentifier(identifier: Array[Byte]): String = { - new String(Base64.encodeBase64(identifier), "utf-8") + new String(Base64.encodeBase64(identifier), UTF_8) } /** @@ -168,7 +170,7 @@ private[spark] object SparkSaslServer { * @return password as a char array. */ def encodePassword(password: Array[Byte]): Array[Char] = { - new String(Base64.encodeBase64(password), "utf-8").toCharArray() + new String(Base64.encodeBase64(password), UTF_8).toCharArray() } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 163dca6cade5a..61b125ef7c6c1 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,7 +19,6 @@ package org.apache.spark.api.python import java.io._ import java.net._ -import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ @@ -27,6 +26,7 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.existentials +import com.google.common.base.Charsets.UTF_8 import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.hadoop.conf.Configuration @@ -134,7 +134,7 @@ private[spark] class PythonRDD( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj, "utf-8"), + throw new PythonException(new String(obj, UTF_8), writerThread.exception.getOrElse(null)) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still @@ -318,7 +318,6 @@ private object SpecialLengths { } private[spark] object PythonRDD extends Logging { - val UTF8 = Charset.forName("UTF-8") // remember the broadcasts sent to each worker private val workerBroadcasts = new mutable.WeakHashMap[Socket, mutable.Set[Long]]() @@ -586,7 +585,7 @@ private[spark] object PythonRDD extends Logging { } def writeUTF(str: String, dataOut: DataOutputStream) { - val bytes = str.getBytes(UTF8) + val bytes = str.getBytes(UTF_8) dataOut.writeInt(bytes.length) dataOut.write(bytes) } @@ -849,7 +848,7 @@ private[spark] object PythonRDD extends Logging { private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { - override def call(arr: Array[Byte]) : String = new String(arr, PythonRDD.UTF8) + override def call(arr: Array[Byte]) : String = new String(arr, UTF_8) } /** diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index d11db978b842e..e9ca9166eb4d6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -18,7 +18,8 @@ package org.apache.spark.api.python import java.io.{DataOutput, DataInput} -import java.nio.charset.Charset + +import com.google.common.base.Charsets.UTF_8 import org.apache.hadoop.io._ import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat @@ -136,7 +137,7 @@ object WriteInputFormatTestDataGenerator { sc.parallelize(intKeys).saveAsSequenceFile(intPath) sc.parallelize(intKeys.map{ case (k, v) => (k.toDouble, v) }).saveAsSequenceFile(doublePath) sc.parallelize(intKeys.map{ case (k, v) => (k.toString, v) }).saveAsSequenceFile(textPath) - sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(Charset.forName("UTF-8"))) } + sc.parallelize(intKeys.map{ case (k, v) => (k, v.getBytes(UTF_8)) } ).saveAsSequenceFile(bytesPath) val bools = Seq((1, true), (2, true), (2, false), (3, true), (2, false), (1, false)) sc.parallelize(bools).saveAsSequenceFile(boolPath) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 9f9911762505a..3bf0b9492df0a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import scala.collection.Map import akka.actor.ActorRef -import com.google.common.base.Charsets +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileUtil, Path} @@ -178,7 +178,7 @@ private[spark] class DriverRunner( val stderr = new File(baseDir, "stderr") val header = "Launch Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) - Files.append(header, stderr, Charsets.UTF_8) + Files.append(header, stderr, UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } runCommandWithRetry(ProcessBuilderLike(builder), initialize, supervise) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 71d7385b08eb9..030a651469b64 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io._ import akka.actor.ActorRef -import com.google.common.base.Charsets +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.spark.{SparkConf, Logging} @@ -151,7 +151,7 @@ private[spark] class ExecutorRunner( stdoutAppender = FileAppender(process.getInputStream, stdout, conf) val stderr = new File(executorDir, "stderr") - Files.write(header, stderr, Charsets.UTF_8) + Files.write(header, stderr, UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) state = ExecutorState.RUNNING diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala index 5aea7ba2f3673..3ab13b96d7f00 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala @@ -19,13 +19,13 @@ package org.apache.spark.network.netty.client import java.util.concurrent.TimeoutException +import com.google.common.base.Charsets.UTF_8 import io.netty.bootstrap.Bootstrap import io.netty.buffer.PooledByteBufAllocator import io.netty.channel.socket.SocketChannel import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} import io.netty.handler.codec.LengthFieldBasedFrameDecoder import io.netty.handler.codec.string.StringEncoder -import io.netty.util.CharsetUtil import org.apache.spark.Logging @@ -61,7 +61,7 @@ class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, b.handler(new ChannelInitializer[SocketChannel] { override def initChannel(ch: SocketChannel): Unit = { ch.pipeline - .addLast("encoder", new StringEncoder(CharsetUtil.UTF_8)) + .addLast("encoder", new StringEncoder(UTF_8)) // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) .addLast("handler", handler) diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala index 83265b164299d..d9d3f7bef0b20 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala @@ -17,6 +17,7 @@ package org.apache.spark.network.netty.client +import com.google.common.base.Charsets.UTF_8 import io.netty.buffer.ByteBuf import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} @@ -67,7 +68,7 @@ class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] wi val blockIdLen = in.readInt() val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) in.readBytes(blockIdBytes) - val blockId = new String(blockIdBytes) + val blockId = new String(blockIdBytes, UTF_8) val blockSize = totalLen - math.abs(blockIdLen) - 4 def server = ctx.channel.remoteAddress.toString @@ -76,7 +77,7 @@ class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] wi if (blockIdLen < 0) { val errorMessageBytes = new Array[Byte](blockSize) in.readBytes(errorMessageBytes) - val errorMsg = new String(errorMessageBytes) + val errorMsg = new String(errorMessageBytes, UTF_8) logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") val listener = outstandingRequests.get(blockId) diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala index 7b2f9a8d4dfd0..9194c7ced3547 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala @@ -19,6 +19,7 @@ package org.apache.spark.network.netty.server import java.net.InetSocketAddress +import com.google.common.base.Charsets.UTF_8 import io.netty.bootstrap.ServerBootstrap import io.netty.buffer.PooledByteBufAllocator import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} @@ -30,7 +31,6 @@ import io.netty.channel.socket.nio.NioServerSocketChannel import io.netty.channel.socket.oio.OioServerSocketChannel import io.netty.handler.codec.LineBasedFrameDecoder import io.netty.handler.codec.string.StringDecoder -import io.netty.util.CharsetUtil import org.apache.spark.{Logging, SparkConf} import org.apache.spark.network.netty.NettyConfig @@ -131,7 +131,7 @@ class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Lo override def initChannel(ch: SocketChannel): Unit = { ch.pipeline .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("stringDecoder", new StringDecoder(UTF_8)) .addLast("blockHeaderEncoder", new BlockHeaderEncoder) .addLast("handler", new BlockServerHandler(dataProvider)) } diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala index cc70bd0c5c477..188154d51d0eb 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala @@ -17,13 +17,13 @@ package org.apache.spark.network.netty.server +import com.google.common.base.Charsets.UTF_8 import io.netty.channel.ChannelInitializer import io.netty.channel.socket.SocketChannel import io.netty.handler.codec.LineBasedFrameDecoder import io.netty.handler.codec.string.StringDecoder -import io.netty.util.CharsetUtil -import org.apache.spark.storage.BlockDataProvider +import org.apache.spark.storage.BlockDataProvider /** Channel initializer that sets up the pipeline for the BlockServer. */ private[netty] @@ -33,7 +33,7 @@ class BlockServerChannelInitializer(dataProvider: BlockDataProvider) override def initChannel(ch: SocketChannel): Unit = { ch.pipeline .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(CharsetUtil.UTF_8)) + .addLast("stringDecoder", new StringDecoder(UTF_8)) .addLast("blockHeaderEncoder", new BlockHeaderEncoder) .addLast("handler", new BlockServerHandler(dataProvider)) } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index bda4bf50932c3..8408b75bb4d65 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -31,6 +31,8 @@ import scala.concurrent.duration._ import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.language.postfixOps +import com.google.common.base.Charsets.UTF_8 + import org.apache.spark._ import org.apache.spark.util.Utils @@ -923,7 +925,7 @@ private[nio] class ConnectionManager( val errorMsgByteBuf = ackMessage.asInstanceOf[BufferMessage].buffers.head val errorMsgBytes = new Array[Byte](errorMsgByteBuf.limit()) errorMsgByteBuf.get(errorMsgBytes) - val errorMsg = new String(errorMsgBytes, "utf-8") + val errorMsg = new String(errorMsgBytes, UTF_8) val e = new IOException( s"sendMessageReliably failed with ACK that signalled a remote error: $errorMsg") if (!promise.tryFailure(e)) { diff --git a/core/src/main/scala/org/apache/spark/network/nio/Message.scala b/core/src/main/scala/org/apache/spark/network/nio/Message.scala index 3ad04591da658..fb4a979b824c3 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/Message.scala @@ -22,6 +22,8 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import com.google.common.base.Charsets.UTF_8 + import org.apache.spark.util.Utils private[nio] abstract class Message(val typ: Long, val id: Int) { @@ -92,7 +94,7 @@ private[nio] object Message { */ def createErrorMessage(exception: Exception, ackId: Int): BufferMessage = { val exceptionString = Utils.exceptionString(exception) - val serializedExceptionString = ByteBuffer.wrap(exceptionString.getBytes("utf-8")) + val serializedExceptionString = ByteBuffer.wrap(exceptionString.getBytes(UTF_8)) val errorMessage = createBufferMessage(serializedExceptionString, ackId) errorMessage.hasError = true errorMessage diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala index 903ab09ae4322..f629322ff6838 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.network.netty.client import java.nio.ByteBuffer +import com.google.common.base.Charsets.UTF_8 import io.netty.buffer.Unpooled import io.netty.channel.embedded.EmbeddedChannel @@ -42,7 +43,7 @@ class BlockFetchingClientHandlerSuite extends FunSuite with PrivateMethodTester parsedBlockId = bid val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) refCntBuf.byteBuffer().get(bytes) - parsedBlockData = new String(bytes) + parsedBlockData = new String(bytes, UTF_8) } } ) diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala index 3ee281cb1350b..3f8d0cf8f3ff3 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.network.netty.server +import com.google.common.base.Charsets.UTF_8 import io.netty.buffer.ByteBuf import io.netty.channel.embedded.EmbeddedChannel import org.scalatest.FunSuite - class BlockHeaderEncoderSuite extends FunSuite { test("encode normal block data") { @@ -35,7 +35,7 @@ class BlockHeaderEncoderSuite extends FunSuite { val blockIdBytes = new Array[Byte](blockId.length) out.readBytes(blockIdBytes) - assert(new String(blockIdBytes) === blockId) + assert(new String(blockIdBytes, UTF_8) === blockId) assert(out.readableBytes() === 0) channel.close() @@ -52,11 +52,11 @@ class BlockHeaderEncoderSuite extends FunSuite { val blockIdBytes = new Array[Byte](blockId.length) out.readBytes(blockIdBytes) - assert(new String(blockIdBytes) === blockId) + assert(new String(blockIdBytes, UTF_8) === blockId) val errorMsgBytes = new Array[Byte](errorMsg.length) out.readBytes(errorMsgBytes) - assert(new String(errorMsgBytes) === errorMsg) + assert(new String(errorMsgBytes, UTF_8) === errorMsg) assert(out.readableBytes() === 0) channel.close() diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index d2bee448d4d3b..4dc5b6103db74 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -18,13 +18,13 @@ package org.apache.spark.util import java.io._ -import java.nio.charset.Charset import scala.collection.mutable.HashSet import scala.reflect._ import org.scalatest.{BeforeAndAfter, FunSuite} +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.apache.spark.{Logging, SparkConf} @@ -44,11 +44,11 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { test("basic file appender") { val testString = (1 to 1000).mkString(", ") - val inputStream = new ByteArrayInputStream(testString.getBytes(Charset.forName("UTF-8"))) + val inputStream = new ByteArrayInputStream(testString.getBytes(UTF_8)) val appender = new FileAppender(inputStream, testFile) inputStream.close() appender.awaitTermination() - assert(Files.toString(testFile, Charset.forName("UTF-8")) === testString) + assert(Files.toString(testFile, UTF_8) === testString) } test("rolling file appender - time-based rolling") { @@ -96,7 +96,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { val allGeneratedFiles = new HashSet[String]() val items = (1 to 10).map { _.toString * 10000 } for (i <- 0 until items.size) { - testOutputStream.write(items(i).getBytes(Charset.forName("UTF-8"))) + testOutputStream.write(items(i).getBytes(UTF_8)) testOutputStream.flush() allGeneratedFiles ++= RollingFileAppender.getSortedRolledOverFiles( testFile.getParentFile.toString, testFile.getName).map(_.toString) @@ -199,7 +199,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { // send data to appender through the input stream, and wait for the data to be written val expectedText = textToAppend.mkString("") for (i <- 0 until textToAppend.size) { - outputStream.write(textToAppend(i).getBytes(Charset.forName("UTF-8"))) + outputStream.write(textToAppend(i).getBytes(UTF_8)) outputStream.flush() Thread.sleep(sleepTimeBetweenTexts) } @@ -214,7 +214,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { logInfo("Filtered files: \n" + generatedFiles.mkString("\n")) assert(generatedFiles.size > 1) val allText = generatedFiles.map { file => - Files.toString(file, Charset.forName("UTF-8")) + Files.toString(file, UTF_8) }.mkString("") assert(allText === expectedText) generatedFiles diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index ea7ef0524d1e1..65579bb9affc7 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -23,7 +23,7 @@ import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStr import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} -import com.google.common.base.Charsets +import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files import org.scalatest.FunSuite @@ -118,7 +118,7 @@ class UtilsSuite extends FunSuite { tmpDir2.deleteOnExit() val f1Path = tmpDir2 + "/f1" val f1 = new FileOutputStream(f1Path) - f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(Charsets.UTF_8)) + f1.write("1\n2\n3\n4\n5\n6\n7\n8\n9\n".getBytes(UTF_8)) f1.close() // Read first few bytes @@ -146,9 +146,9 @@ class UtilsSuite extends FunSuite { val tmpDir = Utils.createTempDir() tmpDir.deleteOnExit() val files = (1 to 3).map(i => new File(tmpDir, i.toString)) - Files.write("0123456789", files(0), Charsets.UTF_8) - Files.write("abcdefghij", files(1), Charsets.UTF_8) - Files.write("ABCDEFGHIJ", files(2), Charsets.UTF_8) + Files.write("0123456789", files(0), UTF_8) + Files.write("abcdefghij", files(1), UTF_8) + Files.write("ABCDEFGHIJ", files(2), UTF_8) // Read first few bytes in the 1st file assert(Utils.offsetBytes(files, 0, 5) === "01234") @@ -339,7 +339,7 @@ class UtilsSuite extends FunSuite { try { System.setProperty("spark.test.fileNameLoadB", "2") Files.write("spark.test.fileNameLoadA true\n" + - "spark.test.fileNameLoadB 1\n", outFile, Charsets.UTF_8) + "spark.test.fileNameLoadB 1\n", outFile, UTF_8) val properties = Utils.getPropertiesFromFile(outFile.getAbsolutePath) properties .filter { case (k, v) => k.startsWith("spark.")} From 4b55482abf899c27da3d55401ad26b4e9247b327 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 28 Oct 2014 14:36:06 -0700 Subject: [PATCH 1121/1492] [SPARK-3343] [SQL] Add serde support for CTAS Currently, `CTAS` (Create Table As Select) doesn't support specifying the `SerDe` in HQL. This PR will pass down the `ASTNode` into the physical operator `execution.CreateTableAsSelect`, which will extract the `CreateTableDesc` object via Hive `SemanticAnalyzer`. In the meantime, I also update the `HiveMetastoreCatalog.createTable` to optionally support the `CreateTableDesc` for table creation. Author: Cheng Hao Closes #2570 from chenghao-intel/ctas_serde and squashes the following commits: e011ef5 [Cheng Hao] shim for both 0.12 & 0.13.1 cfb3662 [Cheng Hao] revert to hive 0.12 c8a547d [Cheng Hao] Support SerDe properties within CTAS --- .../plans/logical/basicOperators.scala | 8 +- .../org/apache/spark/sql/SchemaRDDLike.scala | 4 +- .../org/apache/spark/sql/QueryTest.scala | 19 ++ .../execution/HiveCompatibilitySuite.scala | 6 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 197 +++++++++++++++--- .../org/apache/spark/sql/hive/HiveQl.scala | 15 +- .../spark/sql/hive/HiveStrategies.scala | 17 +- .../hive/execution/CreateTableAsSelect.scala | 39 ++-- .../org/apache/spark/sql/QueryTest.scala | 19 ++ .../sql/hive/execution/HiveExplainSuite.scala | 37 ++-- .../sql/hive/execution/SQLQuerySuite.scala | 59 ++++++ .../org/apache/spark/sql/hive/Shim12.scala | 5 +- .../org/apache/spark/sql/hive/Shim13.scala | 6 +- 13 files changed, 337 insertions(+), 94 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 14b03c7445c13..00bdf108a8398 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -114,11 +114,13 @@ case class InsertIntoTable( } } -case class CreateTableAsSelect( +case class CreateTableAsSelect[T]( databaseName: Option[String], tableName: String, - child: LogicalPlan) extends UnaryNode { - override def output = child.output + child: LogicalPlan, + allowExisting: Boolean, + desc: Option[T] = None) extends UnaryNode { + override def output = Seq.empty[Attribute] override lazy val resolved = (databaseName != None && childrenResolved) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 25ba7d88ba538..15516afb95504 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -54,7 +54,7 @@ private[sql] trait SchemaRDDLike { @transient protected[spark] val logicalPlan: LogicalPlan = baseLogicalPlan match { // For various commands (like DDL) and queries with side effects, we force query optimization to // happen right away to let these side effects take place eagerly. - case _: Command | _: InsertIntoTable | _: CreateTableAsSelect |_: WriteToFile => + case _: Command | _: InsertIntoTable | _: CreateTableAsSelect[_] |_: WriteToFile => LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext) case _ => baseLogicalPlan @@ -123,7 +123,7 @@ private[sql] trait SchemaRDDLike { */ @Experimental def saveAsTable(tableName: String): Unit = - sqlContext.executePlan(CreateTableAsSelect(None, tableName, logicalPlan)).toRdd + sqlContext.executePlan(CreateTableAsSelect(None, tableName, logicalPlan, false)).toRdd /** Returns the schema as a string in the tree format. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 1fd8d27b34c59..042f61f5a4113 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -21,6 +21,25 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ class QueryTest extends PlanTest { + /** + * Runs the plan and makes sure the answer contains all of the keywords, or the + * none of keywords are listed in the answer + * @param rdd the [[SchemaRDD]] to be executed + * @param exists true for make sure the keywords are listed in the output, otherwise + * to make sure none of the keyword are not listed in the output + * @param keywords keyword in string array + */ + def checkExistence(rdd: SchemaRDD, exists: Boolean, keywords: String*) { + val outputs = rdd.collect().map(_.mkString).mkString + for (key <- keywords) { + if (exists) { + assert(outputs.contains(key), s"Failed for $rdd ($key doens't exist in result)") + } else { + assert(!outputs.contains(key), s"Failed for $rdd ($key existed in the result)") + } + } + } + /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[SchemaRDD]] to be executed diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 4fc26d6f55511..26d9ca05c851b 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -229,7 +229,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Needs constant object inspectors "udf_round", - "udf7" + "udf7", + + // Sort with Limit clause causes failure. + "ctas", + "ctas_hadoop20" ) ++ HiveShim.compatibilityBlackList /** 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 39d87a9d14fa7..2dd2c882a8420 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 @@ -17,17 +17,27 @@ package org.apache.spark.sql.hive +import java.io.IOException +import java.util.{List => JList} + import scala.util.parsing.combinator.RegexParsers -import org.apache.hadoop.hive.metastore.api.{FieldSchema, SerDeInfo, StorageDescriptor, Partition => TPartition, Table => TTable} -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} -import org.apache.hadoop.hive.ql.plan.TableDesc -import org.apache.hadoop.hive.serde2.Deserializer +import org.apache.hadoop.util.ReflectionUtils +import org.apache.hadoop.fs.Path + +import org.apache.hadoop.hive.metastore.TableType +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.metastore.api.{Table => TTable, Partition => TPartition} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table, HiveException} +import org.apache.hadoop.hive.ql.plan.{TableDesc, CreateTableDesc} +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.{Deserializer, SerDeException} +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.analysis.Catalog +import org.apache.spark.sql.catalyst.analysis.{Catalog, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ @@ -66,37 +76,164 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with table.getTTable, partitions.map(part => part.getTPartition))(hive) } + /** + * Create table with specified database, table name, table description and schema + * @param databaseName Database Name + * @param tableName Table Name + * @param schema Schema of the new table, if not specified, will use the schema + * specified in crtTbl + * @param allowExisting if true, ignore AlreadyExistsException + * @param desc CreateTableDesc object which contains the SerDe info. Currently + * we support most of the features except the bucket. + */ def createTable( databaseName: String, tableName: String, schema: Seq[Attribute], - allowExisting: Boolean = false): Unit = { + allowExisting: Boolean = false, + desc: Option[CreateTableDesc] = None) { + val hconf = hive.hiveconf + val (dbName, tblName) = processDatabaseAndTableName(databaseName, tableName) - val table = new Table(dbName, tblName) - val hiveSchema = + val tbl = new Table(dbName, tblName) + + val crtTbl: CreateTableDesc = desc.getOrElse(null) + + // We should respect the passed in schema, unless it's not set + val hiveSchema: JList[FieldSchema] = if (schema == null || schema.isEmpty) { + crtTbl.getCols + } else { schema.map(attr => new FieldSchema(attr.name, toMetastoreType(attr.dataType), "")) - table.setFields(hiveSchema) - - val sd = new StorageDescriptor() - table.getTTable.setSd(sd) - sd.setCols(hiveSchema) - - // TODO: THESE ARE ALL DEFAULTS, WE NEED TO PARSE / UNDERSTAND the output specs. - sd.setCompressed(false) - sd.setParameters(Map[String, String]()) - sd.setInputFormat("org.apache.hadoop.mapred.TextInputFormat") - sd.setOutputFormat("org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat") - val serDeInfo = new SerDeInfo() - serDeInfo.setName(tblName) - serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") - serDeInfo.setParameters(Map[String, String]()) - sd.setSerdeInfo(serDeInfo) + } + tbl.setFields(hiveSchema) + + // Most of code are similar with the DDLTask.createTable() of Hive, + if (crtTbl != null && crtTbl.getTblProps() != null) { + tbl.getTTable().getParameters().putAll(crtTbl.getTblProps()) + } + + if (crtTbl != null && crtTbl.getPartCols() != null) { + tbl.setPartCols(crtTbl.getPartCols()) + } + + if (crtTbl != null && crtTbl.getStorageHandler() != null) { + tbl.setProperty( + org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, + crtTbl.getStorageHandler()) + } + + /* + * We use LazySimpleSerDe by default. + * + * If the user didn't specify a SerDe, and any of the columns are not simple + * types, we will have to use DynamicSerDe instead. + */ + if (crtTbl == null || crtTbl.getSerName() == null) { + val storageHandler = tbl.getStorageHandler() + if (storageHandler == null) { + logInfo(s"Default to LazySimpleSerDe for table $dbName.$tblName") + tbl.setSerializationLib(classOf[LazySimpleSerDe].getName()) + + import org.apache.hadoop.mapred.TextInputFormat + import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat + import org.apache.hadoop.io.Text + + tbl.setInputFormatClass(classOf[TextInputFormat]) + tbl.setOutputFormatClass(classOf[HiveIgnoreKeyTextOutputFormat[Text, Text]]) + tbl.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe") + } else { + val serDeClassName = storageHandler.getSerDeClass().getName() + logInfo(s"Use StorageHandler-supplied $serDeClassName for table $dbName.$tblName") + tbl.setSerializationLib(serDeClassName) + } + } else { + // let's validate that the serde exists + val serdeName = crtTbl.getSerName() + try { + val d = ReflectionUtils.newInstance(hconf.getClassByName(serdeName), hconf) + if (d != null) { + logDebug("Found class for $serdeName") + } + } catch { + case e: SerDeException => throw new HiveException("Cannot validate serde: " + serdeName, e) + } + tbl.setSerializationLib(serdeName) + } + + if (crtTbl != null && crtTbl.getFieldDelim() != null) { + tbl.setSerdeParam(serdeConstants.FIELD_DELIM, crtTbl.getFieldDelim()) + tbl.setSerdeParam(serdeConstants.SERIALIZATION_FORMAT, crtTbl.getFieldDelim()) + } + if (crtTbl != null && crtTbl.getFieldEscape() != null) { + tbl.setSerdeParam(serdeConstants.ESCAPE_CHAR, crtTbl.getFieldEscape()) + } + + if (crtTbl != null && crtTbl.getCollItemDelim() != null) { + tbl.setSerdeParam(serdeConstants.COLLECTION_DELIM, crtTbl.getCollItemDelim()) + } + if (crtTbl != null && crtTbl.getMapKeyDelim() != null) { + tbl.setSerdeParam(serdeConstants.MAPKEY_DELIM, crtTbl.getMapKeyDelim()) + } + if (crtTbl != null && crtTbl.getLineDelim() != null) { + tbl.setSerdeParam(serdeConstants.LINE_DELIM, crtTbl.getLineDelim()) + } + + if (crtTbl != null && crtTbl.getSerdeProps() != null) { + val iter = crtTbl.getSerdeProps().entrySet().iterator() + while (iter.hasNext()) { + val m = iter.next() + tbl.setSerdeParam(m.getKey(), m.getValue()) + } + } + + if (crtTbl != null && crtTbl.getComment() != null) { + tbl.setProperty("comment", crtTbl.getComment()) + } + + if (crtTbl != null && crtTbl.getLocation() != null) { + HiveShim.setLocation(tbl, crtTbl) + } + + if (crtTbl != null && crtTbl.getSkewedColNames() != null) { + tbl.setSkewedColNames(crtTbl.getSkewedColNames()) + } + if (crtTbl != null && crtTbl.getSkewedColValues() != null) { + tbl.setSkewedColValues(crtTbl.getSkewedColValues()) + } + + if (crtTbl != null) { + tbl.setStoredAsSubDirectories(crtTbl.isStoredAsSubDirectories()) + tbl.setInputFormatClass(crtTbl.getInputFormat()) + tbl.setOutputFormatClass(crtTbl.getOutputFormat()) + } + + tbl.getTTable().getSd().setInputFormat(tbl.getInputFormatClass().getName()) + tbl.getTTable().getSd().setOutputFormat(tbl.getOutputFormatClass().getName()) + + if (crtTbl != null && crtTbl.isExternal()) { + tbl.setProperty("EXTERNAL", "TRUE") + tbl.setTableType(TableType.EXTERNAL_TABLE) + } + + // set owner + try { + tbl.setOwner(hive.hiveconf.getUser) + } catch { + case e: IOException => throw new HiveException("Unable to get current user", e) + } + + // set create time + tbl.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) + + // TODO add bucket support + // TODO set more info if Hive upgrade + // create the table synchronized { - try client.createTable(table) catch { - case e: org.apache.hadoop.hive.ql.metadata.HiveException - if e.getCause.isInstanceOf[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] && - allowExisting => // Do nothing. + try client.createTable(tbl, allowExisting) catch { + case e: org.apache.hadoop.hive.metastore.api.AlreadyExistsException + if allowExisting => // Do nothing + case e: Throwable => throw e } } } @@ -110,11 +247,11 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case CreateTableAsSelect(db, tableName, child) => + case CreateTableAsSelect(db, tableName, child, allowExisting, extra) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - CreateTableAsSelect(Some(databaseName), tableName, child) + CreateTableAsSelect(Some(databaseName), tableName, child, allowExisting, extra) } } 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 ed07a280398bd..9d9d68affa54b 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 @@ -448,14 +448,14 @@ private[hive] object HiveQl { } case Token("TOK_CREATETABLE", children) - if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => - // TODO: Parse other clauses. + if children.collect { case t @ Token("TOK_QUERY", _) => t }.nonEmpty => // Reference: https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DDL val ( Some(tableNameParts) :: _ /* likeTable */ :: - Some(query) +: - notImplemented) = + Some(query) :: + allowExisting +: + ignores) = getClauses( Seq( "TOK_TABNAME", @@ -479,14 +479,9 @@ private[hive] object HiveQl { "TOK_TABLELOCATION", "TOK_TABLEPROPERTIES"), children) - if (notImplemented.exists(token => !token.isEmpty)) { - throw new NotImplementedError( - s"Unhandled clauses: ${notImplemented.flatten.map(dumpTree(_)).mkString("\n")}") - } - val (db, tableName) = extractDbNameTableName(tableNameParts) - CreateTableAsSelect(db, tableName, nodeToPlan(query)) + CreateTableAsSelect(db, tableName, nodeToPlan(query), allowExisting != None, Some(node)) // If its not a "CREATE TABLE AS" like above then just pass it back to hive as a native command. case Token("TOK_CREATETABLE", _) => NativePlaceholder diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 5c66322f1ed99..e59d4d536a0af 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import org.apache.hadoop.hive.ql.parse.ASTNode + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ @@ -160,17 +162,14 @@ private[hive] trait HiveStrategies { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil - - case logical.CreateTableAsSelect(database, tableName, child) => - val query = planLater(child) + case logical.CreateTableAsSelect( + Some(database), tableName, child, allowExisting, Some(extra: ASTNode)) => CreateTableAsSelect( - database.get, + database, tableName, - query, - InsertIntoHiveTable(_: MetastoreRelation, - Map(), - query, - overwrite = true)(hiveContext)) :: Nil + child, + allowExisting, + extra) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 3625708d03175..2fce414734579 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -17,10 +17,12 @@ package org.apache.spark.sql.hive.execution +import org.apache.hadoop.hive.ql.Context +import org.apache.hadoop.hive.ql.parse.{SemanticAnalyzer, ASTNode} import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} import org.apache.spark.sql.execution.{SparkPlan, Command, LeafNode} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.MetastoreRelation @@ -30,33 +32,46 @@ import org.apache.spark.sql.hive.MetastoreRelation * Create table and insert the query result into it. * @param database the database name of the new relation * @param tableName the table name of the new relation - * @param insertIntoRelation function of creating the `InsertIntoHiveTable` - * by specifying the `MetaStoreRelation`, the data will be inserted into that table. - * TODO Add more table creating properties, e.g. SerDe, StorageHandler, in-memory cache etc. + * @param query the query whose result will be insert into the new relation + * @param allowExisting allow continue working if it's already exists, otherwise + * raise exception + * @param extra the extra information for this Operator, it should be the + * ASTNode object for extracting the CreateTableDesc. + */ @Experimental case class CreateTableAsSelect( database: String, tableName: String, - query: SparkPlan, - insertIntoRelation: MetastoreRelation => InsertIntoHiveTable) - extends LeafNode with Command { + query: LogicalPlan, + allowExisting: Boolean, + extra: ASTNode) extends LeafNode with Command { def output = Seq.empty + private[this] def sc = sqlContext.asInstanceOf[HiveContext] + // A lazy computing of the metastoreRelation private[this] lazy val metastoreRelation: MetastoreRelation = { - // Create the table - val sc = sqlContext.asInstanceOf[HiveContext] - sc.catalog.createTable(database, tableName, query.output, false) + // Get the CreateTableDesc from Hive SemanticAnalyzer + val sa = new SemanticAnalyzer(sc.hiveconf) + + sa.analyze(extra, new Context(sc.hiveconf)) + val desc = sa.getQB().getTableDesc + // Create Hive Table + sc.catalog.createTable(database, tableName, query.output, allowExisting, Some(desc)) + // Get the Metastore Relation sc.catalog.lookupRelation(Some(database), tableName, None) match { case r: MetastoreRelation => r } } - override protected lazy val sideEffectResult: Seq[Row] = { - insertIntoRelation(metastoreRelation).execute + override protected[sql] lazy val sideEffectResult: Seq[Row] = { + // TODO ideally, we should get the output data ready first and then + // add the relation into catalog, just in case of failure occurs while data + // processing. + sc.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd Seq.empty[Row] } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala index 6b0641052052d..f89c49d292c6c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -32,6 +32,25 @@ import org.apache.spark.sql.catalyst.util._ * So, we duplicate this code here. */ class QueryTest extends PlanTest { + /** + * Runs the plan and makes sure the answer contains all of the keywords, or the + * none of keywords are listed in the answer + * @param rdd the [[SchemaRDD]] to be executed + * @param exists true for make sure the keywords are listed in the output, otherwise + * to make sure none of the keyword are not listed in the output + * @param keywords keyword in string array + */ + def checkExistence(rdd: SchemaRDD, exists: Boolean, keywords: String*) { + val outputs = rdd.collect().map(_.mkString).mkString + for (key <- keywords) { + if (exists) { + assert(outputs.contains(key), s"Failed for $rdd ($key doens't exist in result)") + } else { + assert(!outputs.contains(key), s"Failed for $rdd ($key existed in the result)") + } + } + } + /** * Runs the plan and makes sure the answer matches the expected result. * @param rdd the [[SchemaRDD]] to be executed diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 4ed58f4be1167..a68fc2a803bb4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -18,37 +18,24 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -import org.apache.spark.sql.Row /** * A set of tests that validates support for Hive Explain command. */ class HiveExplainSuite extends QueryTest { - private def check(sqlCmd: String, exists: Boolean, keywords: String*) { - val outputs = sql(sqlCmd).collect().map(_.getString(0)).mkString - for (key <- keywords) { - if (exists) { - assert(outputs.contains(key), s"Failed for $sqlCmd ($key doens't exist in result)") - } else { - assert(!outputs.contains(key), s"Failed for $sqlCmd ($key existed in the result)") - } - } - } - test("explain extended command") { - check(" explain select * from src where key=123 ", true, - "== Physical Plan ==") - check(" explain select * from src where key=123 ", false, - "== Parsed Logical Plan ==", - "== Analyzed Logical Plan ==", - "== Optimized Logical Plan ==") - check(" explain extended select * from src where key=123 ", true, - "== Parsed Logical Plan ==", - "== Analyzed Logical Plan ==", - "== Optimized Logical Plan ==", - "== Physical Plan ==", - "Code Generation", "== RDD ==") + checkExistence(sql(" explain select * from src where key=123 "), true, + "== Physical Plan ==") + checkExistence(sql(" explain select * from src where key=123 "), false, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==") + checkExistence(sql(" explain extended select * from src where key=123 "), true, + "== Parsed Logical Plan ==", + "== Analyzed Logical Plan ==", + "== Optimized Logical Plan ==", + "== Physical Plan ==", + "Code Generation", "== RDD ==") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index a4aea31d3ff9e..4f96a327ee2c7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -32,6 +32,65 @@ case class Nested3(f3: Int) * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + test("CTAS with serde") { + sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect + sql( + """CREATE TABLE ctas2 + | ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe" + | WITH SERDEPROPERTIES("serde_p1"="p1","serde_p2"="p2") + | STORED AS RCFile + | TBLPROPERTIES("tbl_p1"="p11", "tbl_p2"="p22") + | AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin).collect + sql( + """CREATE TABLE ctas3 + | ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\012' + | STORED AS textfile AS + | SELECT key, value + | FROM src + | ORDER BY key, value""".stripMargin).collect + + // the table schema may like (key: integer, value: string) + sql( + """CREATE TABLE IF NOT EXISTS ctas4 AS + | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin).collect + // expect the string => integer for field key cause the table ctas4 already existed. + sql( + """CREATE TABLE IF NOT EXISTS ctas4 AS + | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect + + checkAnswer( + sql("SELECT k, value FROM ctas1 ORDER BY k, value"), + sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq) + checkAnswer( + sql("SELECT key, value FROM ctas2 ORDER BY key, value"), + sql( + """ + SELECT key, value + FROM src + ORDER BY key, value""").collect().toSeq) + checkAnswer( + sql("SELECT key, value FROM ctas3 ORDER BY key, value"), + sql( + """ + SELECT key, value + FROM src + ORDER BY key, value""").collect().toSeq) + checkAnswer( + sql("SELECT key, value FROM ctas4 ORDER BY key, value"), + sql("SELECT CAST(key AS int) k, value FROM src ORDER BY k, value").collect().toSeq) + + checkExistence(sql("DESC EXTENDED ctas2"), true, + "name:key", "type:string", "name:value", "ctas2", + "org.apache.hadoop.hive.ql.io.RCFileInputFormat", + "org.apache.hadoop.hive.ql.io.RCFileOutputFormat", + "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe", + "serde_p1=p1", "serde_p2=p2", "tbl_p1=p11", "tbl_p2=p22","MANAGED_TABLE" + ) + } + test("ordering not in select") { checkAnswer( sql("SELECT key FROM src ORDER BY value"), diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 2317d2e76341f..8cb81db8a9724 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} @@ -89,6 +89,9 @@ private[hive] object HiveShim { "udf_concat" ) + def setLocation(tbl: Table, crtTbl: CreateTableDesc): Unit = { + tbl.setDataLocation(new Path(crtTbl.getLocation()).toUri()) + } } class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index b8d893d8c1319..b9a742cc6ef67 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.hive.common.`type`.{HiveDecimal} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} -import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} +import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} import org.apache.hadoop.mapred.InputFormat @@ -121,6 +121,10 @@ private[hive] object HiveShim { def compatibilityBlackList = Seq() + def setLocation(tbl: Table, crtTbl: CreateTableDesc): Unit = { + tbl.setDataLocation(new Path(crtTbl.getLocation())) + } + /* * Bug introdiced in hive-0.13. FileSinkDesc is serializable, but its member path is not. * Fix it through wrapper. From 84e5da87e32256ba4f3dee6f8bf532ce88322028 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 28 Oct 2014 15:14:41 -0700 Subject: [PATCH 1122/1492] [SPARK-4084] Reuse sort key in Sorter Sorter uses generic-typed key for sorting. When data is large, it creates lots of key objects, which is not efficient. We should reuse the key in Sorter for memory efficiency. This change is part of the petabyte sort implementation from rxin . The `Sorter` class was written in Java and marked package private. So it is only available to `org.apache.spark.util.collection`. I renamed it to `TimSort` and add a simple wrapper of it, still called `Sorter`, in Scala, which is `private[spark]`. The benchmark code is updated, which now resets the array before each run. Here is the result on sorting primitive Int arrays of size 25 million using Sorter: ~~~ [info] - Sorter benchmark for key-value pairs !!! IGNORED !!! Java Arrays.sort() on non-primitive int array: Took 13237 ms Java Arrays.sort() on non-primitive int array: Took 13320 ms Java Arrays.sort() on non-primitive int array: Took 15718 ms Java Arrays.sort() on non-primitive int array: Took 13283 ms Java Arrays.sort() on non-primitive int array: Took 13267 ms Java Arrays.sort() on non-primitive int array: Took 15122 ms Java Arrays.sort() on non-primitive int array: Took 15495 ms Java Arrays.sort() on non-primitive int array: Took 14877 ms Java Arrays.sort() on non-primitive int array: Took 16429 ms Java Arrays.sort() on non-primitive int array: Took 14250 ms Java Arrays.sort() on non-primitive int array: (13878 ms first try, 14499 ms average) Java Arrays.sort() on primitive int array: Took 2683 ms Java Arrays.sort() on primitive int array: Took 2683 ms Java Arrays.sort() on primitive int array: Took 2701 ms Java Arrays.sort() on primitive int array: Took 2746 ms Java Arrays.sort() on primitive int array: Took 2685 ms Java Arrays.sort() on primitive int array: Took 2735 ms Java Arrays.sort() on primitive int array: Took 2669 ms Java Arrays.sort() on primitive int array: Took 2693 ms Java Arrays.sort() on primitive int array: Took 2680 ms Java Arrays.sort() on primitive int array: Took 2642 ms Java Arrays.sort() on primitive int array: (2948 ms first try, 2691 ms average) Sorter without key reuse on primitive int array: Took 10732 ms Sorter without key reuse on primitive int array: Took 12482 ms Sorter without key reuse on primitive int array: Took 10718 ms Sorter without key reuse on primitive int array: Took 12650 ms Sorter without key reuse on primitive int array: Took 10747 ms Sorter without key reuse on primitive int array: Took 10783 ms Sorter without key reuse on primitive int array: Took 12721 ms Sorter without key reuse on primitive int array: Took 10604 ms Sorter without key reuse on primitive int array: Took 10622 ms Sorter without key reuse on primitive int array: Took 11843 ms Sorter without key reuse on primitive int array: (11089 ms first try, 11390 ms average) Sorter with key reuse on primitive int array: Took 5141 ms Sorter with key reuse on primitive int array: Took 5298 ms Sorter with key reuse on primitive int array: Took 5066 ms Sorter with key reuse on primitive int array: Took 5164 ms Sorter with key reuse on primitive int array: Took 5203 ms Sorter with key reuse on primitive int array: Took 5274 ms Sorter with key reuse on primitive int array: Took 5186 ms Sorter with key reuse on primitive int array: Took 5159 ms Sorter with key reuse on primitive int array: Took 5164 ms Sorter with key reuse on primitive int array: Took 5078 ms Sorter with key reuse on primitive int array: (5311 ms first try, 5173 ms average) ~~~ So with key reuse, it is faster and less likely to trigger GC. Author: Xiangrui Meng Author: Reynold Xin Closes #2937 from mengxr/SPARK-4084 and squashes the following commits: d73c3d0 [Xiangrui Meng] address comments 0b7b682 [Xiangrui Meng] fix mima a72f53c [Xiangrui Meng] update timeIt 38ba50c [Xiangrui Meng] update timeIt 720f731 [Xiangrui Meng] add doc about JIT specialization 78f2879 [Xiangrui Meng] update tests 7de2efd [Xiangrui Meng] update the Sorter benchmark code to be correct 8626356 [Xiangrui Meng] add prepare to timeIt and update testsin SorterSuite 5f0d530 [Xiangrui Meng] update method modifiers of SortDataFormat 6ffbe66 [Xiangrui Meng] rename Sorter to TimSort and add a Scala wrapper that is private[spark] b00db4d [Xiangrui Meng] doc and tests cf94e8a [Xiangrui Meng] renaming 464ddce [Reynold Xin] cherry-pick rxin's commit --- .../collection/{Sorter.java => TimSort.java} | 77 ++++--- .../scala/org/apache/spark/util/Utils.scala | 26 ++- .../util/collection/SortDataFormat.scala | 41 +++- .../apache/spark/util/collection/Sorter.scala | 39 ++++ .../spark/util/random/XORShiftRandom.scala | 8 +- .../org/apache/spark/util/UtilsSuite.scala | 11 + .../spark/util/collection/SorterSuite.scala | 210 +++++++++++++----- project/MimaExcludes.scala | 4 +- 8 files changed, 310 insertions(+), 106 deletions(-) rename core/src/main/java/org/apache/spark/util/collection/{Sorter.java => TimSort.java} (92%) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/Sorter.scala diff --git a/core/src/main/java/org/apache/spark/util/collection/Sorter.java b/core/src/main/java/org/apache/spark/util/collection/TimSort.java similarity index 92% rename from core/src/main/java/org/apache/spark/util/collection/Sorter.java rename to core/src/main/java/org/apache/spark/util/collection/TimSort.java index 64ad18c0e463a..409e1a41c5d49 100644 --- a/core/src/main/java/org/apache/spark/util/collection/Sorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/TimSort.java @@ -20,18 +20,25 @@ import java.util.Comparator; /** - * A port of the Android Timsort class, which utilizes a "stable, adaptive, iterative mergesort." + * A port of the Android TimSort class, which utilizes a "stable, adaptive, iterative mergesort." * See the method comment on sort() for more details. * * This has been kept in Java with the original style in order to match very closely with the - * Anroid source code, and thus be easy to verify correctness. + * Android source code, and thus be easy to verify correctness. The class is package private. We put + * a simple Scala wrapper {@link org.apache.spark.util.collection.Sorter}, which is available to + * package org.apache.spark. * * The purpose of the port is to generalize the interface to the sort to accept input data formats * besides simple arrays where every element is sorted individually. For instance, the AppendOnlyMap * uses this to sort an Array with alternating elements of the form [key, value, key, value]. * This generalization comes with minimal overhead -- see SortDataFormat for more information. + * + * We allow key reuse to prevent creating many key objects -- see SortDataFormat. + * + * @see org.apache.spark.util.collection.SortDataFormat + * @see org.apache.spark.util.collection.Sorter */ -class Sorter { +class TimSort { /** * This is the minimum sized sequence that will be merged. Shorter @@ -54,7 +61,7 @@ class Sorter { private final SortDataFormat s; - public Sorter(SortDataFormat sortDataFormat) { + public TimSort(SortDataFormat sortDataFormat) { this.s = sortDataFormat; } @@ -91,7 +98,7 @@ public Sorter(SortDataFormat sortDataFormat) { * * @author Josh Bloch */ - void sort(Buffer a, int lo, int hi, Comparator c) { + public void sort(Buffer a, int lo, int hi, Comparator c) { assert c != null; int nRemaining = hi - lo; @@ -162,10 +169,13 @@ private void binarySort(Buffer a, int lo, int hi, int start, Comparator>> 1; - if (c.compare(pivot, s.getKey(a, mid)) < 0) + if (c.compare(pivot, s.getKey(a, mid, key1)) < 0) right = mid; else left = mid + 1; @@ -235,13 +245,16 @@ private int countRunAndMakeAscending(Buffer a, int lo, int hi, Comparator= 0) + while (runHi < hi && c.compare(s.getKey(a, runHi, key0), s.getKey(a, runHi - 1, key1)) >= 0) runHi++; } @@ -468,11 +481,13 @@ private void mergeAt(int i) { } stackSize--; + K key0 = s.newKey(); + /* * Find where the first element of run2 goes in run1. Prior elements * in run1 can be ignored (because they're already in place). */ - int k = gallopRight(s.getKey(a, base2), a, base1, len1, 0, c); + int k = gallopRight(s.getKey(a, base2, key0), a, base1, len1, 0, c); assert k >= 0; base1 += k; len1 -= k; @@ -483,7 +498,7 @@ private void mergeAt(int i) { * Find where the last element of run1 goes in run2. Subsequent elements * in run2 can be ignored (because they're already in place). */ - len2 = gallopLeft(s.getKey(a, base1 + len1 - 1), a, base2, len2, len2 - 1, c); + len2 = gallopLeft(s.getKey(a, base1 + len1 - 1, key0), a, base2, len2, len2 - 1, c); assert len2 >= 0; if (len2 == 0) return; @@ -517,10 +532,12 @@ private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator< assert len > 0 && hint >= 0 && hint < len; int lastOfs = 0; int ofs = 1; - if (c.compare(key, s.getKey(a, base + hint)) > 0) { + K key0 = s.newKey(); + + if (c.compare(key, s.getKey(a, base + hint, key0)) > 0) { // Gallop right until a[base+hint+lastOfs] < key <= a[base+hint+ofs] int maxOfs = len - hint; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) > 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key0)) > 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -535,7 +552,7 @@ private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator< } else { // key <= a[base + hint] // Gallop left until a[base+hint-ofs] < key <= a[base+hint-lastOfs] final int maxOfs = hint + 1; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) <= 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key0)) <= 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -560,7 +577,7 @@ private int gallopLeft(K key, Buffer a, int base, int len, int hint, Comparator< while (lastOfs < ofs) { int m = lastOfs + ((ofs - lastOfs) >>> 1); - if (c.compare(key, s.getKey(a, base + m)) > 0) + if (c.compare(key, s.getKey(a, base + m, key0)) > 0) lastOfs = m + 1; // a[base + m] < key else ofs = m; // key <= a[base + m] @@ -587,10 +604,12 @@ private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator int ofs = 1; int lastOfs = 0; - if (c.compare(key, s.getKey(a, base + hint)) < 0) { + K key1 = s.newKey(); + + if (c.compare(key, s.getKey(a, base + hint, key1)) < 0) { // Gallop left until a[b+hint - ofs] <= key < a[b+hint - lastOfs] int maxOfs = hint + 1; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs)) < 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint - ofs, key1)) < 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -606,7 +625,7 @@ private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator } else { // a[b + hint] <= key // Gallop right until a[b+hint + lastOfs] <= key < a[b+hint + ofs] int maxOfs = len - hint; - while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs)) >= 0) { + while (ofs < maxOfs && c.compare(key, s.getKey(a, base + hint + ofs, key1)) >= 0) { lastOfs = ofs; ofs = (ofs << 1) + 1; if (ofs <= 0) // int overflow @@ -630,7 +649,7 @@ private int gallopRight(K key, Buffer a, int base, int len, int hint, Comparator while (lastOfs < ofs) { int m = lastOfs + ((ofs - lastOfs) >>> 1); - if (c.compare(key, s.getKey(a, base + m)) < 0) + if (c.compare(key, s.getKey(a, base + m, key1)) < 0) ofs = m; // key < a[b + m] else lastOfs = m + 1; // a[b + m] <= key @@ -679,6 +698,9 @@ private void mergeLo(int base1, int len1, int base2, int len2) { return; } + K key0 = s.newKey(); + K key1 = s.newKey(); + Comparator c = this.c; // Use local variable for performance int minGallop = this.minGallop; // " " " " " outer: @@ -692,7 +714,7 @@ private void mergeLo(int base1, int len1, int base2, int len2) { */ do { assert len1 > 1 && len2 > 0; - if (c.compare(s.getKey(a, cursor2), s.getKey(tmp, cursor1)) < 0) { + if (c.compare(s.getKey(a, cursor2, key0), s.getKey(tmp, cursor1, key1)) < 0) { s.copyElement(a, cursor2++, a, dest++); count2++; count1 = 0; @@ -714,7 +736,7 @@ private void mergeLo(int base1, int len1, int base2, int len2) { */ do { assert len1 > 1 && len2 > 0; - count1 = gallopRight(s.getKey(a, cursor2), tmp, cursor1, len1, 0, c); + count1 = gallopRight(s.getKey(a, cursor2, key0), tmp, cursor1, len1, 0, c); if (count1 != 0) { s.copyRange(tmp, cursor1, a, dest, count1); dest += count1; @@ -727,7 +749,7 @@ private void mergeLo(int base1, int len1, int base2, int len2) { if (--len2 == 0) break outer; - count2 = gallopLeft(s.getKey(tmp, cursor1), a, cursor2, len2, 0, c); + count2 = gallopLeft(s.getKey(tmp, cursor1, key0), a, cursor2, len2, 0, c); if (count2 != 0) { s.copyRange(a, cursor2, a, dest, count2); dest += count2; @@ -784,6 +806,9 @@ private void mergeHi(int base1, int len1, int base2, int len2) { int cursor2 = len2 - 1; // Indexes into tmp array int dest = base2 + len2 - 1; // Indexes into a + K key0 = s.newKey(); + K key1 = s.newKey(); + // Move last element of first run and deal with degenerate cases s.copyElement(a, cursor1--, a, dest--); if (--len1 == 0) { @@ -811,7 +836,7 @@ private void mergeHi(int base1, int len1, int base2, int len2) { */ do { assert len1 > 0 && len2 > 1; - if (c.compare(s.getKey(tmp, cursor2), s.getKey(a, cursor1)) < 0) { + if (c.compare(s.getKey(tmp, cursor2, key0), s.getKey(a, cursor1, key1)) < 0) { s.copyElement(a, cursor1--, a, dest--); count1++; count2 = 0; @@ -833,7 +858,7 @@ private void mergeHi(int base1, int len1, int base2, int len2) { */ do { assert len1 > 0 && len2 > 1; - count1 = len1 - gallopRight(s.getKey(tmp, cursor2), a, base1, len1, len1 - 1, c); + count1 = len1 - gallopRight(s.getKey(tmp, cursor2, key0), a, base1, len1, len1 - 1, c); if (count1 != 0) { dest -= count1; cursor1 -= count1; @@ -846,7 +871,7 @@ private void mergeHi(int base1, int len1, int base2, int len2) { if (--len2 == 1) break outer; - count2 = len2 - gallopLeft(s.getKey(a, cursor1), tmp, 0, len2, len2 - 1, c); + count2 = len2 - gallopLeft(s.getKey(a, cursor1, key0), tmp, 0, len2, len2 - 1, c); if (count2 != 0) { dest -= count2; cursor2 -= count2; 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 612eca308bf0b..1e881da5114d3 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1272,12 +1272,28 @@ private[spark] object Utils extends Logging { /** * Timing method based on iterations that permit JVM JIT optimization. * @param numIters number of iterations - * @param f function to be executed + * @param f function to be executed. If prepare is not None, the running time of each call to f + * must be an order of magnitude longer than one millisecond for accurate timing. + * @param prepare function to be executed before each call to f. Its running time doesn't count. + * @return the total time across all iterations (not couting preparation time) */ - def timeIt(numIters: Int)(f: => Unit): Long = { - val start = System.currentTimeMillis - times(numIters)(f) - System.currentTimeMillis - start + def timeIt(numIters: Int)(f: => Unit, prepare: Option[() => Unit] = None): Long = { + if (prepare.isEmpty) { + val start = System.currentTimeMillis + times(numIters)(f) + System.currentTimeMillis - start + } else { + var i = 0 + var sum = 0L + while (i < numIters) { + prepare.get.apply() + val start = System.currentTimeMillis + f + sum += System.currentTimeMillis - start + i += 1 + } + sum + } } /** diff --git a/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala b/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala index ac1528969f0be..4f0bf8384afc9 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/SortDataFormat.scala @@ -27,33 +27,51 @@ import scala.reflect.ClassTag * Example format: an array of numbers, where each element is also the key. * See [[KVArraySortDataFormat]] for a more exciting format. * - * This trait extends Any to ensure it is universal (and thus compiled to a Java interface). + * Note: Declaring and instantiating multiple subclasses of this class would prevent JIT inlining + * overridden methods and hence decrease the shuffle performance. * * @tparam K Type of the sort key of each element * @tparam Buffer Internal data structure used by a particular format (e.g., Array[Int]). */ // TODO: Making Buffer a real trait would be a better abstraction, but adds some complexity. -private[spark] trait SortDataFormat[K, Buffer] extends Any { +private[spark] +abstract class SortDataFormat[K, Buffer] { + + /** + * Creates a new mutable key for reuse. This should be implemented if you want to override + * [[getKey(Buffer, Int, K)]]. + */ + def newKey(): K = null.asInstanceOf[K] + /** Return the sort key for the element at the given index. */ protected def getKey(data: Buffer, pos: Int): K + /** + * Returns the sort key for the element at the given index and reuse the input key if possible. + * The default implementation ignores the reuse parameter and invokes [[getKey(Buffer, Int]]. + * If you want to override this method, you must implement [[newKey()]]. + */ + def getKey(data: Buffer, pos: Int, reuse: K): K = { + getKey(data, pos) + } + /** Swap two elements. */ - protected def swap(data: Buffer, pos0: Int, pos1: Int): Unit + def swap(data: Buffer, pos0: Int, pos1: Int): Unit /** Copy a single element from src(srcPos) to dst(dstPos). */ - protected def copyElement(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int): Unit + def copyElement(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int): Unit /** * Copy a range of elements starting at src(srcPos) to dst, starting at dstPos. * Overlapping ranges are allowed. */ - protected def copyRange(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int, length: Int): Unit + def copyRange(src: Buffer, srcPos: Int, dst: Buffer, dstPos: Int, length: Int): Unit /** * Allocates a Buffer that can hold up to 'length' elements. * All elements of the buffer should be considered invalid until data is explicitly copied in. */ - protected def allocate(length: Int): Buffer + def allocate(length: Int): Buffer } /** @@ -67,9 +85,9 @@ private[spark] trait SortDataFormat[K, Buffer] extends Any { private[spark] class KVArraySortDataFormat[K, T <: AnyRef : ClassTag] extends SortDataFormat[K, Array[T]] { - override protected def getKey(data: Array[T], pos: Int): K = data(2 * pos).asInstanceOf[K] + override def getKey(data: Array[T], pos: Int): K = data(2 * pos).asInstanceOf[K] - override protected def swap(data: Array[T], pos0: Int, pos1: Int) { + override def swap(data: Array[T], pos0: Int, pos1: Int) { val tmpKey = data(2 * pos0) val tmpVal = data(2 * pos0 + 1) data(2 * pos0) = data(2 * pos1) @@ -78,17 +96,16 @@ class KVArraySortDataFormat[K, T <: AnyRef : ClassTag] extends SortDataFormat[K, data(2 * pos1 + 1) = tmpVal } - override protected def copyElement(src: Array[T], srcPos: Int, dst: Array[T], dstPos: Int) { + override def copyElement(src: Array[T], srcPos: Int, dst: Array[T], dstPos: Int) { dst(2 * dstPos) = src(2 * srcPos) dst(2 * dstPos + 1) = src(2 * srcPos + 1) } - override protected def copyRange(src: Array[T], srcPos: Int, - dst: Array[T], dstPos: Int, length: Int) { + override def copyRange(src: Array[T], srcPos: Int, dst: Array[T], dstPos: Int, length: Int) { System.arraycopy(src, 2 * srcPos, dst, 2 * dstPos, 2 * length) } - override protected def allocate(length: Int): Array[T] = { + override def allocate(length: Int): Array[T] = { new Array[T](2 * length) } } diff --git a/core/src/main/scala/org/apache/spark/util/collection/Sorter.scala b/core/src/main/scala/org/apache/spark/util/collection/Sorter.scala new file mode 100644 index 0000000000000..39f66b8c428c6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/Sorter.scala @@ -0,0 +1,39 @@ +/* + * 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.util.collection + +import java.util.Comparator + +/** + * A simple wrapper over the Java implementation [[TimSort]]. + * + * The Java implementation is package private, and hence it cannot be called outside package + * org.apache.spark.util.collection. This is a simple wrapper of it that is available to spark. + */ +private[spark] +class Sorter[K, Buffer](private val s: SortDataFormat[K, Buffer]) { + + private val timSort = new TimSort(s) + + /** + * Sorts the input buffer within range [lo, hi). + */ + def sort(a: Buffer, lo: Int, hi: Int, c: Comparator[_ >: K]): Unit = { + timSort.sort(a, lo, hi, c) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 55b5713706178..467b890fb4bb9 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -96,13 +96,9 @@ private[spark] object XORShiftRandom { xorRand.nextInt() } - val iters = timeIt(numIters)(_) - /* Return results as a map instead of just printing to screen in case the user wants to do something with them */ - Map("javaTime" -> iters {javaRand.nextInt()}, - "xorTime" -> iters {xorRand.nextInt()}) - + Map("javaTime" -> timeIt(numIters) { javaRand.nextInt() }, + "xorTime" -> timeIt(numIters) { xorRand.nextInt() }) } - } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 65579bb9affc7..1c112334cc2f5 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -351,4 +351,15 @@ class UtilsSuite extends FunSuite { outFile.delete() } } + + test("timeIt with prepare") { + var cnt = 0 + val prepare = () => { + cnt += 1 + Thread.sleep(1000) + } + val time = Utils.timeIt(2)({}, Some(prepare)) + require(cnt === 2, "prepare should be called twice") + require(time < 500, "preparation time should not count") + } } diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index 6fe1079c2719a..066d47c46a0d2 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util.collection -import java.lang.{Float => JFloat} +import java.lang.{Float => JFloat, Integer => JInteger} import java.util.{Arrays, Comparator} import org.scalatest.FunSuite @@ -30,11 +30,15 @@ class SorterSuite extends FunSuite { val rand = new XORShiftRandom(123) val data0 = Array.tabulate[Int](10000) { i => rand.nextInt() } val data1 = data0.clone() + val data2 = data0.clone() Arrays.sort(data0) new Sorter(new IntArraySortDataFormat).sort(data1, 0, data1.length, Ordering.Int) + new Sorter(new KeyReuseIntArraySortDataFormat) + .sort(data2, 0, data2.length, Ordering[IntWrapper]) - data0.zip(data1).foreach { case (x, y) => assert(x === y) } + assert(data0.view === data1.view) + assert(data0.view === data2.view) } test("KVArraySorter") { @@ -61,10 +65,33 @@ class SorterSuite extends FunSuite { } } + /** Runs an experiment several times. */ + def runExperiment(name: String, skip: Boolean = false)(f: => Unit, prepare: () => Unit): Unit = { + if (skip) { + println(s"Skipped experiment $name.") + return + } + + val firstTry = org.apache.spark.util.Utils.timeIt(1)(f, Some(prepare)) + System.gc() + + var i = 0 + var next10: Long = 0 + while (i < 10) { + val time = org.apache.spark.util.Utils.timeIt(1)(f, Some(prepare)) + next10 += time + println(s"$name: Took $time ms") + i += 1 + } + + println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") + } + /** * This provides a simple benchmark for comparing the Sorter with Java internal sorting. * Ideally these would be executed one at a time, each in their own JVM, so their listing - * here is mainly to have the code. + * here is mainly to have the code. Running multiple tests within the same JVM session would + * prevent JIT inlining overridden methods and hence hurt the performance. * * The goal of this code is to sort an array of key-value pairs, where the array physically * has the keys and values alternating. The basic Java sorts work only on the keys, so the @@ -72,96 +99,167 @@ class SorterSuite extends FunSuite { * those, while the Sorter approach can work directly on the input data format. * * Note that the Java implementation varies tremendously between Java 6 and Java 7, when - * the Java sort changed from merge sort to Timsort. + * the Java sort changed from merge sort to TimSort. */ - ignore("Sorter benchmark") { - - /** Runs an experiment several times. */ - def runExperiment(name: String)(f: => Unit): Unit = { - val firstTry = org.apache.spark.util.Utils.timeIt(1)(f) - System.gc() - - var i = 0 - var next10: Long = 0 - while (i < 10) { - val time = org.apache.spark.util.Utils.timeIt(1)(f) - next10 += time - println(s"$name: Took $time ms") - i += 1 - } - - println(s"$name: ($firstTry ms first try, ${next10 / 10} ms average)") - } - + ignore("Sorter benchmark for key-value pairs") { val numElements = 25000000 // 25 mil val rand = new XORShiftRandom(123) - val keys = Array.tabulate[JFloat](numElements) { i => - new JFloat(rand.nextFloat()) + // Test our key-value pairs where each element is a Tuple2[Float, Integer]. + + val kvTuples = Array.tabulate(numElements) { i => + (new JFloat(rand.nextFloat()), new JInteger(i)) } - // Test our key-value pairs where each element is a Tuple2[Float, Integer) - val kvTupleArray = Array.tabulate[AnyRef](numElements) { i => - (keys(i / 2): Float, i / 2: Int) + val kvTupleArray = new Array[AnyRef](numElements) + val prepareKvTupleArray = () => { + System.arraycopy(kvTuples, 0, kvTupleArray, 0, numElements) } - runExperiment("Tuple-sort using Arrays.sort()") { + runExperiment("Tuple-sort using Arrays.sort()")({ Arrays.sort(kvTupleArray, new Comparator[AnyRef] { override def compare(x: AnyRef, y: AnyRef): Int = - Ordering.Float.compare(x.asInstanceOf[(Float, _)]._1, y.asInstanceOf[(Float, _)]._1) + x.asInstanceOf[(JFloat, _)]._1.compareTo(y.asInstanceOf[(JFloat, _)]._1) }) - } + }, prepareKvTupleArray) // Test our Sorter where each element alternates between Float and Integer, non-primitive - val keyValueArray = Array.tabulate[AnyRef](numElements * 2) { i => - if (i % 2 == 0) keys(i / 2) else new Integer(i / 2) + + val keyValues = { + val data = new Array[AnyRef](numElements * 2) + var i = 0 + while (i < numElements) { + data(2 * i) = kvTuples(i)._1 + data(2 * i + 1) = kvTuples(i)._2 + i += 1 + } + data } + + val keyValueArray = new Array[AnyRef](numElements * 2) + val prepareKeyValueArray = () => { + System.arraycopy(keyValues, 0, keyValueArray, 0, numElements * 2) + } + val sorter = new Sorter(new KVArraySortDataFormat[JFloat, AnyRef]) - runExperiment("KV-sort using Sorter") { - sorter.sort(keyValueArray, 0, keys.length, new Comparator[JFloat] { - override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) + runExperiment("KV-sort using Sorter")({ + sorter.sort(keyValueArray, 0, numElements, new Comparator[JFloat] { + override def compare(x: JFloat, y: JFloat): Int = x.compareTo(y) }) + }, prepareKeyValueArray) + } + + /** + * Tests for sorting with primitive keys with/without key reuse. Java's Arrays.sort is used as + * reference, which is expected to be faster but it can only sort a single array. Sorter can be + * used to sort parallel arrays. + * + * Ideally these would be executed one at a time, each in their own JVM, so their listing + * here is mainly to have the code. Running multiple tests within the same JVM session would + * prevent JIT inlining overridden methods and hence hurt the performance. + */ + test("Sorter benchmark for primitive int array") { + val numElements = 25000000 // 25 mil + val rand = new XORShiftRandom(123) + + val ints = Array.fill(numElements)(rand.nextInt()) + val intObjects = { + val data = new Array[JInteger](numElements) + var i = 0 + while (i < numElements) { + data(i) = new JInteger(ints(i)) + i += 1 + } + data } - // Test non-primitive sort on float array - runExperiment("Java Arrays.sort()") { - Arrays.sort(keys, new Comparator[JFloat] { - override def compare(x: JFloat, y: JFloat): Int = Ordering.Float.compare(x, y) - }) + val intObjectArray = new Array[JInteger](numElements) + val prepareIntObjectArray = () => { + System.arraycopy(intObjects, 0, intObjectArray, 0, numElements) } - // Test primitive sort on float array - val primitiveKeys = Array.tabulate[Float](numElements) { i => rand.nextFloat() } - runExperiment("Java Arrays.sort() on primitive keys") { - Arrays.sort(primitiveKeys) + runExperiment("Java Arrays.sort() on non-primitive int array")({ + Arrays.sort(intObjectArray, new Comparator[JInteger] { + override def compare(x: JInteger, y: JInteger): Int = x.compareTo(y) + }) + }, prepareIntObjectArray) + + val intPrimitiveArray = new Array[Int](numElements) + val prepareIntPrimitiveArray = () => { + System.arraycopy(ints, 0, intPrimitiveArray, 0, numElements) } - } -} + runExperiment("Java Arrays.sort() on primitive int array")({ + Arrays.sort(intPrimitiveArray) + }, prepareIntPrimitiveArray) -/** Format to sort a simple Array[Int]. Could be easily generified and specialized. */ -class IntArraySortDataFormat extends SortDataFormat[Int, Array[Int]] { - override protected def getKey(data: Array[Int], pos: Int): Int = { - data(pos) + val sorterWithoutKeyReuse = new Sorter(new IntArraySortDataFormat) + runExperiment("Sorter without key reuse on primitive int array")({ + sorterWithoutKeyReuse.sort(intPrimitiveArray, 0, numElements, Ordering[Int]) + }, prepareIntPrimitiveArray) + + val sorterWithKeyReuse = new Sorter(new KeyReuseIntArraySortDataFormat) + runExperiment("Sorter with key reuse on primitive int array")({ + sorterWithKeyReuse.sort(intPrimitiveArray, 0, numElements, Ordering[IntWrapper]) + }, prepareIntPrimitiveArray) } +} - override protected def swap(data: Array[Int], pos0: Int, pos1: Int): Unit = { +abstract class AbstractIntArraySortDataFormat[K] extends SortDataFormat[K, Array[Int]] { + + override def swap(data: Array[Int], pos0: Int, pos1: Int): Unit = { val tmp = data(pos0) data(pos0) = data(pos1) data(pos1) = tmp } - override protected def copyElement(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int) { + override def copyElement(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int) { dst(dstPos) = src(srcPos) } /** Copy a range of elements starting at src(srcPos) to dest, starting at destPos. */ - override protected def copyRange(src: Array[Int], srcPos: Int, - dst: Array[Int], dstPos: Int, length: Int) { + override def copyRange(src: Array[Int], srcPos: Int, dst: Array[Int], dstPos: Int, length: Int) { System.arraycopy(src, srcPos, dst, dstPos, length) } /** Allocates a new structure that can hold up to 'length' elements. */ - override protected def allocate(length: Int): Array[Int] = { + override def allocate(length: Int): Array[Int] = { new Array[Int](length) } } + +/** Format to sort a simple Array[Int]. Could be easily generified and specialized. */ +class IntArraySortDataFormat extends AbstractIntArraySortDataFormat[Int] { + + override protected def getKey(data: Array[Int], pos: Int): Int = { + data(pos) + } +} + +/** Wrapper of Int for key reuse. */ +class IntWrapper(var key: Int = 0) extends Ordered[IntWrapper] { + + override def compare(that: IntWrapper): Int = { + Ordering.Int.compare(key, that.key) + } +} + +/** SortDataFormat for Array[Int] with reused keys. */ +class KeyReuseIntArraySortDataFormat extends AbstractIntArraySortDataFormat[IntWrapper] { + + override def newKey(): IntWrapper = { + new IntWrapper() + } + + override def getKey(data: Array[Int], pos: Int, reuse: IntWrapper): IntWrapper = { + if (reuse == null) { + new IntWrapper(data(pos)) + } else { + reuse.key = data(pos) + reuse + } + } + + override protected def getKey(data: Array[Int], pos: Int): IntWrapper = { + getKey(data, pos, null) + } +} diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c58666af84f24..95152b58e287e 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -53,7 +53,9 @@ object MimaExcludes { "org.apache.spark.scheduler.MapStatus"), // TaskContext was promoted to Abstract class ProblemFilters.exclude[AbstractClassProblem]( - "org.apache.spark.TaskContext") + "org.apache.spark.TaskContext"), + ProblemFilters.exclude[IncompatibleTemplateDefProblem]( + "org.apache.spark.util.collection.SortDataFormat") ) ++ Seq( // Adding new methods to the JavaRDDLike trait: ProblemFilters.exclude[MissingMethodProblem]( From 1536d70331e9a4f5b5ea9dabfd72592ca1fc8e35 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 28 Oct 2014 17:59:10 -0700 Subject: [PATCH 1123/1492] [SPARK-4008] Fix "kryo with fold" in KryoSerializerSuite `zeroValue` will be serialized by `spark.closure.serializer` but `spark.closure.serializer` only supports the default Java serializer. So it must not be `ClassWithoutNoArgConstructor`, which can not be serialized by the Java serializer. This PR changed `zeroValue` to null and updated the test to make it work correctly. Author: zsxwing Closes #2856 from zsxwing/SPARK-4008 and squashes the following commits: 51da655 [zsxwing] [SPARK-4008] Fix "kryo with fold" in KryoSerializerSuite --- .../spark/serializer/KryoSerializerSuite.scala | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 64ac6d2d920d2..a70f67af2e62e 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -201,12 +201,17 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { assert(control.sum === result) } - // TODO: this still doesn't work - ignore("kryo with fold") { + test("kryo with fold") { val control = 1 :: 2 :: Nil + // zeroValue must not be a ClassWithoutNoArgConstructor instance because it will be + // serialized by spark.closure.serializer but spark.closure.serializer only supports + // the default Java serializer. val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)) - .fold(new ClassWithoutNoArgConstructor(10))((t1, t2) => new ClassWithoutNoArgConstructor(t1.x + t2.x)).x - assert(10 + control.sum === result) + .fold(null)((t1, t2) => { + val t1x = if (t1 == null) 0 else t1.x + new ClassWithoutNoArgConstructor(t1x + t2.x) + }).x + assert(control.sum === result) } test("kryo with nonexistent custom registrator should fail") { From b5e79bf889700159d490cdac1f6322dff424b1d9 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 28 Oct 2014 19:11:57 -0700 Subject: [PATCH 1124/1492] [SPARK-3904] [SQL] add constant objectinspector support for udfs In HQL, we convert all of the data type into normal `ObjectInspector`s for UDFs, most of cases it works, however, some of the UDF actually requires its children `ObjectInspector` to be the `ConstantObjectInspector`, which will cause exception. e.g. select named_struct("x", "str") from src limit 1; I updated the method `wrap` by adding the one more parameter `ObjectInspector`(to describe what it expects to wrap to, for example: java.lang.Integer or IntWritable). As well as the `unwrap` method by providing the input `ObjectInspector`. Author: Cheng Hao Closes #2762 from chenghao-intel/udf_coi and squashes the following commits: bcacfd7 [Cheng Hao] Shim for both Hive 0.12 & 0.13.1 2416e5d [Cheng Hao] revert to hive 0.12 5793c01 [Cheng Hao] add space before while 4e56e1b [Cheng Hao] style issue 683d3fd [Cheng Hao] Add golden files fe591e4 [Cheng Hao] update HiveGenericUdf for set the ObjectInspector while constructing the DeferredObject f6740fe [Cheng Hao] Support Constant ObjectInspector for Map & List 8814c3a [Cheng Hao] Passing ContantObjectInspector(when necessary) for UDF initializing --- .../execution/HiveCompatibilitySuite.scala | 8 +- .../spark/sql/hive/HiveInspectors.scala | 185 ++++++++++++------ .../apache/spark/sql/hive/TableReader.scala | 2 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 62 +++--- ...nt array-0-761ef205b10ac4a10122c8b4ce10ada | 1 + ...d_struct-0-8f0ea83364b78634fbb3752c5a5c725 | 1 + ..._struct-1-380c9638cc6ea8ea42f187bf0cedf350 | 1 + ..._struct-2-22a79ac608b1249306f82f4bdc669b17 | 0 ..._struct-3-d7e4a555934307155784904ff9df188b | 1 + ...t_array-0-e86d559aeb84a4cc017a103182c22bfb | 0 ...t_array-1-976cd8b6b50a2748bbc768aa5e11cf82 | 1 + ..._array-10-9e047718e5fea6ea79124f1e899f1c13 | 1 + ...t_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 | 4 + ...t_array-3-55c4cdaf8438b06675d60848d68f35de | 0 ..._struct-0-f41043b7d9f14fa5e998c90454c7bdb1 | 1 + ..._struct-1-8ccdb20153debdab789ea8ad0228e2eb | 1 + ..._struct-2-4a62774a6de7571c8d2bcb77da63f8f3 | 0 ..._struct-3-abffdaacb0c7076ab538fbeec072daa2 | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 8 + .../org/apache/spark/sql/hive/Shim12.scala | 57 ++++++ .../org/apache/spark/sql/hive/Shim13.scala | 64 +++++- 21 files changed, 307 insertions(+), 92 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-2-22a79ac608b1249306f82f4bdc669b17 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de create mode 100644 sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 create mode 100644 sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb create mode 100644 sql/hive/src/test/resources/golden/udf_struct-2-4a62774a6de7571c8d2bcb77da63f8f3 create mode 100644 sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 26d9ca05c851b..1a3c24be420e6 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -233,7 +233,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Sort with Limit clause causes failure. "ctas", - "ctas_hadoop20" + "ctas_hadoop20", + + // timestamp in array, the output format of Hive contains double quotes, while + // Spark SQL doesn't + "udf_sort_array" ) ++ HiveShim.compatibilityBlackList /** @@ -861,6 +865,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_minute", "udf_modulo", "udf_month", + "udf_named_struct", "udf_negative", "udf_not", "udf_notequal", @@ -894,6 +899,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "udf_stddev_pop", "udf_stddev_samp", "udf_string", + "udf_struct", "udf_substring", "udf_subtract", "udf_sum", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index fad7373a2fa39..c6103a124df59 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.hive import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector._ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} @@ -78,44 +80,13 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == classOf[java.lang.Object] => NullType } - /** Converts hive types to native catalyst types. */ - def unwrap(a: Any): Any = a match { - case null => null - case i: hadoopIo.IntWritable => i.get - case t: hadoopIo.Text => t.toString - case l: hadoopIo.LongWritable => l.get - case d: hadoopIo.DoubleWritable => d.get - case d: hiveIo.DoubleWritable => d.get - case s: hiveIo.ShortWritable => s.get - case b: hadoopIo.BooleanWritable => b.get - case b: hiveIo.ByteWritable => b.get - case b: hadoopIo.FloatWritable => b.get - case b: hadoopIo.BytesWritable => { - val bytes = new Array[Byte](b.getLength) - System.arraycopy(b.getBytes(), 0, bytes, 0, b.getLength) - bytes - } - case d: hiveIo.DateWritable => d.get - case t: hiveIo.TimestampWritable => t.getTimestamp - case b: hiveIo.HiveDecimalWritable => BigDecimal(b.getHiveDecimal().bigDecimalValue()) - case list: java.util.List[_] => list.map(unwrap) - case map: java.util.Map[_,_] => map.map { case (k, v) => (unwrap(k), unwrap(v)) }.toMap - case array: Array[_] => array.map(unwrap).toSeq - case p: java.lang.Short => p - case p: java.lang.Long => p - case p: java.lang.Float => p - case p: java.lang.Integer => p - case p: java.lang.Double => p - case p: java.lang.Byte => p - case p: java.lang.Boolean => p - case str: String => str - case p: java.math.BigDecimal => p - case p: Array[Byte] => p - case p: java.sql.Date => p - case p: java.sql.Timestamp => p - } - - def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { + /** + * Converts hive types to native catalyst types. + * @param data the data in Hive type + * @param oi the ObjectInspector associated with the Hive Type + * @return convert the data into catalyst type + */ + def unwrap(data: Any, oi: ObjectInspector): Any = oi match { case hvoi: HiveVarcharObjectInspector => if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue case hdoi: HiveDecimalObjectInspector => @@ -123,43 +94,89 @@ private[hive] trait HiveInspectors { case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) case li: ListObjectInspector => Option(li.getList(data)) - .map(_.map(unwrapData(_, li.getListElementObjectInspector)).toSeq) + .map(_.map(unwrap(_, li.getListElementObjectInspector)).toSeq) .orNull case mi: MapObjectInspector => Option(mi.getMap(data)).map( _.map { case (k,v) => - (unwrapData(k, mi.getMapKeyObjectInspector), - unwrapData(v, mi.getMapValueObjectInspector)) + (unwrap(k, mi.getMapKeyObjectInspector), + unwrap(v, mi.getMapValueObjectInspector)) }.toMap).orNull case si: StructObjectInspector => val allRefs = si.getAllStructFieldRefs new GenericRow( allRefs.map(r => - unwrapData(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) + unwrap(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) } - /** Converts native catalyst types to the types expected by Hive */ - def wrap(a: Any): AnyRef = a match { - case s: String => s: java.lang.String - case i: Int => i: java.lang.Integer - case b: Boolean => b: java.lang.Boolean - case f: Float => f: java.lang.Float - case d: Double => d: java.lang.Double - case l: Long => l: java.lang.Long - case l: Short => l: java.lang.Short - case l: Byte => l: java.lang.Byte - case b: BigDecimal => HiveShim.createDecimal(b.underlying()) - case b: Array[Byte] => b - case d: java.sql.Date => d - case t: java.sql.Timestamp => t - case s: Seq[_] => seqAsJavaList(s.map(wrap)) - case m: Map[_,_] => - // Some UDFs seem to assume we pass in a HashMap. - val hashMap = new java.util.HashMap[AnyRef, AnyRef]() - hashMap.putAll(m.map { case (k, v) => wrap(k) -> wrap(v) }) - hashMap - case null => null + /** + * Converts native catalyst types to the types expected by Hive + * @param a the value to be wrapped + * @param oi This ObjectInspector associated with the value returned by this function, and + * the ObjectInspector should also be consistent with those returned from + * toInspector: DataType => ObjectInspector and + * toInspector: Expression => ObjectInspector + */ + def wrap(a: Any, oi: ObjectInspector): AnyRef = if (a == null) { + null + } else { + oi match { + case x: ConstantObjectInspector => x.getWritableConstantValue + case x: PrimitiveObjectInspector => a match { + // TODO what if x.preferWritable() == true? reuse the writable? + case s: String => s: java.lang.String + case i: Int => i: java.lang.Integer + case b: Boolean => b: java.lang.Boolean + case f: Float => f: java.lang.Float + case d: Double => d: java.lang.Double + case l: Long => l: java.lang.Long + case l: Short => l: java.lang.Short + case l: Byte => l: java.lang.Byte + case b: BigDecimal => HiveShim.createDecimal(b.underlying()) + case b: Array[Byte] => b + case d: java.sql.Date => d + case t: java.sql.Timestamp => t + } + case x: StructObjectInspector => + val fieldRefs = x.getAllStructFieldRefs + val row = a.asInstanceOf[Seq[_]] + val result = new java.util.ArrayList[AnyRef](fieldRefs.length) + var i = 0 + while (i < fieldRefs.length) { + result.add(wrap(row(i), fieldRefs.get(i).getFieldObjectInspector)) + i += 1 + } + + result + case x: ListObjectInspector => + val list = new java.util.ArrayList[Object] + a.asInstanceOf[Seq[_]].foreach { + v => list.add(wrap(v, x.getListElementObjectInspector)) + } + list + case x: MapObjectInspector => + // Some UDFs seem to assume we pass in a HashMap. + val hashMap = new java.util.HashMap[AnyRef, AnyRef]() + hashMap.putAll(a.asInstanceOf[Map[_, _]].map { + case (k, v) => + wrap(k, x.getMapKeyObjectInspector) -> wrap(v, x.getMapValueObjectInspector) + }) + + hashMap + } + } + + def wrap( + row: Seq[Any], + inspectors: Seq[ObjectInspector], + cache: Array[AnyRef]): Array[AnyRef] = { + var i = 0 + while (i < inspectors.length) { + cache(i) = wrap(row(i), inspectors(i)) + i += 1 + } + cache } def toInspector(dataType: DataType): ObjectInspector = dataType match { @@ -186,6 +203,48 @@ private[hive] trait HiveInspectors { fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) } + def toInspector(expr: Expression): ObjectInspector = expr match { + case Literal(value: String, StringType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Int, IntegerType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Double, DoubleType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Boolean, BooleanType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Long, LongType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Float, FloatType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Short, ShortType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Byte, ByteType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Array[Byte], BinaryType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: java.sql.Date, DateType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: java.sql.Timestamp, TimestampType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: BigDecimal, DecimalType) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(_, NullType) => + HiveShim.getPrimitiveNullWritableConstantObjectInspector + case Literal(value: Seq[_], ArrayType(dt, _)) => + val listObjectInspector = toInspector(dt) + val list = new java.util.ArrayList[Object]() + value.foreach(v => list.add(wrap(v, listObjectInspector))) + ObjectInspectorFactory.getStandardConstantListObjectInspector(listObjectInspector, list) + case Literal(map: Map[_, _], MapType(keyType, valueType, _)) => + val value = new java.util.HashMap[Object, Object]() + val keyOI = toInspector(keyType) + val valueOI = toInspector(valueType) + map.foreach (entry => value.put(wrap(entry._1, keyOI), wrap(entry._2, valueOI))) + ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, value) + case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") + case _ => toInspector(expr.dataType) + } + def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { case s: StructObjectInspector => StructType(s.getAllStructFieldRefs.map(f => { 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 9ff7ab5a124c1..e49f0957d188a 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 @@ -291,7 +291,7 @@ private[hive] object HadoopTableReader extends HiveInspectors { case oi: DoubleObjectInspector => (value: Any, row: MutableRow, ordinal: Int) => row.setDouble(ordinal, oi.get(value)) case oi => - (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrapData(value, oi) + (value: Any, row: MutableRow, ordinal: Int) => row(ordinal) = unwrap(value, oi) } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala index 683c820dec305..aff4ddce92272 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -21,7 +21,9 @@ import org.apache.hadoop.hive.ql.udf.generic.GenericUDFUtils.ConversionHelper import scala.collection.mutable.ArrayBuffer -import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory.ObjectInspectorOptions +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory import org.apache.hadoop.hive.ql.exec.{UDF, UDAF} import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType} @@ -97,7 +99,7 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) @transient - protected lazy val arguments = children.map(c => toInspector(c.dataType)).toArray + protected lazy val arguments = children.map(toInspector).toArray @transient protected lazy val isUDFDeterministic = { @@ -116,12 +118,19 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ @transient lazy val dataType = javaClassToDataType(method.getReturnType) + @transient + lazy val returnInspector = ObjectInspectorFactory.getReflectionObjectInspector( + method.getGenericReturnType(), ObjectInspectorOptions.JAVA) + + @transient + protected lazy val cached = new Array[AnyRef](children.length) + // TODO: Finish input output types. override def eval(input: Row): Any = { - val evaluatedChildren = children.map(c => wrap(c.eval(input))) - - unwrap(FunctionRegistry.invoke(method, function, conversionHelper - .convertIfNecessary(evaluatedChildren: _*): _*)) + unwrap( + FunctionRegistry.invoke(method, function, conversionHelper + .convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _*): _*), + returnInspector) } } @@ -133,7 +142,7 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq type UDFType = GenericUDF @transient - protected lazy val argumentInspectors = children.map(_.dataType).map(toInspector) + protected lazy val argumentInspectors = children.map(toInspector) @transient protected lazy val returnInspector = function.initialize(argumentInspectors.toArray) @@ -148,18 +157,18 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable) } - protected lazy val deferedObjects = Array.fill[DeferredObject](children.length)({ - new DeferredObjectAdapter - }) + @transient + protected lazy val deferedObjects = + argumentInspectors.map(new DeferredObjectAdapter(_)).toArray[DeferredObject] // Adapter from Catalyst ExpressionResult to Hive DeferredObject - class DeferredObjectAdapter extends DeferredObject { + class DeferredObjectAdapter(oi: ObjectInspector) extends DeferredObject { private var func: () => Any = _ def set(func: () => Any) { this.func = func } override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(func()) + override def get(): AnyRef = wrap(func(), oi) } lazy val dataType: DataType = inspectorToDataType(returnInspector) @@ -169,10 +178,13 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq var i = 0 while (i < children.length) { val idx = i - deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set(() => {children(idx).eval(input)}) + deferedObjects(i).asInstanceOf[DeferredObjectAdapter].set( + () => { + children(idx).eval(input) + }) i += 1 } - unwrap(function.evaluate(deferedObjects)) + unwrap(function.evaluate(deferedObjects), returnInspector) } } @@ -260,12 +272,14 @@ private[hive] case class HiveGenericUdtf( protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) @transient - protected lazy val outputInspectors = { - val structInspector = function.initialize(inputInspectors.toArray) - structInspector.getAllStructFieldRefs.map(_.getFieldObjectInspector) - } + protected lazy val outputInspector = function.initialize(inputInspectors.toArray) - protected lazy val outputDataTypes = outputInspectors.map(inspectorToDataType) + @transient + protected lazy val udtInput = new Array[AnyRef](children.length) + + protected lazy val outputDataTypes = outputInspector.getAllStructFieldRefs.map { + field => inspectorToDataType(field.getFieldObjectInspector) + } override protected def makeOutput() = { // Use column names when given, otherwise c_1, c_2, ... c_n. @@ -283,14 +297,12 @@ private[hive] case class HiveGenericUdtf( } override def eval(input: Row): TraversableOnce[Row] = { - outputInspectors // Make sure initialized. + outputInspector // Make sure initialized. val inputProjection = new InterpretedProjection(children) val collector = new UDTFCollector function.setCollector(collector) - - val udtInput = inputProjection(input).map(wrap).toArray - function.process(udtInput) + function.process(wrap(inputProjection(input), inputInspectors, udtInput)) collector.collectRows() } @@ -301,7 +313,7 @@ private[hive] case class HiveGenericUdtf( // We need to clone the input here because implementations of // GenericUDTF reuse the same object. Luckily they are always an array, so // it is easy to clone. - collected += new GenericRow(input.asInstanceOf[Array[_]].map(unwrap)) + collected += unwrap(input, outputInspector).asInstanceOf[Row] } def collectRows() = { @@ -342,7 +354,7 @@ private[hive] case class HiveUdafFunction( private val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - override def eval(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) + override def eval(input: Row): Any = unwrap(function.evaluate(buffer), returnInspector) @transient val inputProjection = new InterpretedProjection(exprs) diff --git a/sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada b/sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada new file mode 100644 index 0000000000000..94f18d09863a7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/constant array-0-761ef205b10ac4a10122c8b4ce10ada @@ -0,0 +1 @@ +["enterprise databases","hadoop distributed file system","hadoop map-reduce"] diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 b/sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 new file mode 100644 index 0000000000000..9bff96e7fa20e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-0-8f0ea83364b78634fbb3752c5a5c725 @@ -0,0 +1 @@ +named_struct(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 b/sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 new file mode 100644 index 0000000000000..9bff96e7fa20e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-1-380c9638cc6ea8ea42f187bf0cedf350 @@ -0,0 +1 @@ +named_struct(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-2-22a79ac608b1249306f82f4bdc669b17 b/sql/hive/src/test/resources/golden/udf_named_struct-2-22a79ac608b1249306f82f4bdc669b17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b b/sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b new file mode 100644 index 0000000000000..de25f51b5b56d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-3-d7e4a555934307155784904ff9df188b @@ -0,0 +1 @@ +{"foo":1,"bar":2} 1 diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 new file mode 100644 index 0000000000000..d514df4191b89 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 @@ -0,0 +1 @@ +sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 b/sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 new file mode 100644 index 0000000000000..9d33cd51fef04 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-10-9e047718e5fea6ea79124f1e899f1c13 @@ -0,0 +1 @@ +[1,2,3,4,5] [1,2,7,8,9] [4,8,16,32,64] [1,100,246,357,1000] [false,true] [1.414,1.618,2.718,3.141] [1.41421,1.61803,2.71828,3.14159] ["","aramis","athos","portos"] ["1970-01-05 13:51:04.042","1970-01-07 00:54:54.442","1970-01-16 12:50:35.242"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 new file mode 100644 index 0000000000000..43e36513de881 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 @@ -0,0 +1,4 @@ +sort_array(array(obj1, obj2,...)) - Sorts the input array in ascending order according to the natural ordering of the array elements. +Example: + > SELECT sort_array(array('b', 'd', 'c', 'a')) FROM src LIMIT 1; + 'a', 'b', 'c', 'd' diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 b/sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 new file mode 100644 index 0000000000000..062cb1bc683b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-0-f41043b7d9f14fa5e998c90454c7bdb1 @@ -0,0 +1 @@ +struct(col1, col2, col3, ...) - Creates a struct with the given field values diff --git a/sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb b/sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb new file mode 100644 index 0000000000000..062cb1bc683b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-1-8ccdb20153debdab789ea8ad0228e2eb @@ -0,0 +1 @@ +struct(col1, col2, col3, ...) - Creates a struct with the given field values diff --git a/sql/hive/src/test/resources/golden/udf_struct-2-4a62774a6de7571c8d2bcb77da63f8f3 b/sql/hive/src/test/resources/golden/udf_struct-2-4a62774a6de7571c8d2bcb77da63f8f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 b/sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 new file mode 100644 index 0000000000000..ff1a28fa47f18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-3-abffdaacb0c7076ab538fbeec072daa2 @@ -0,0 +1 @@ +{"col1":1} {"col1":1,"col2":"a"} 1 a diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 322a25bb20837..ffe1f0b90fcd0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -34,6 +34,14 @@ case class TestData(a: Int, b: String) * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest { + createQueryTest("constant array", + """ + |SELECT sort_array( + | sort_array( + | array("hadoop distributed file system", + | "enterprise databases", "hadoop map-reduce"))) + |FROM src LIMIT 1; + """.stripMargin) createQueryTest("count distinct 0 values", """ diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 8cb81db8a9724..afc252ac27987 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -29,7 +29,11 @@ import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} import org.apache.hadoop.mapred.InputFormat import scala.collection.JavaConversions._ @@ -50,6 +54,59 @@ private[hive] object HiveShim { new TableDesc(serdeClass, inputFormatClass, outputFormatClass, properties) } + def getPrimitiveWritableConstantObjectInspector(value: String): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.STRING, new hadoopIo.Text(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Int): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.INT, new hadoopIo.IntWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Double): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.DOUBLE, new hiveIo.DoubleWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Boolean): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.BOOLEAN, new hadoopIo.BooleanWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Long): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.LONG, new hadoopIo.LongWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Float): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.FLOAT, new hadoopIo.FloatWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Short): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.SHORT, new hiveIo.ShortWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Byte): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.BYTE, new hiveIo.ByteWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Array[Byte]): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.BINARY, new hadoopIo.BytesWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Date): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.DATE, new hiveIo.DateWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Timestamp): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.TIMESTAMP, new hiveIo.TimestampWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: BigDecimal): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.DECIMAL, + new hiveIo.HiveDecimalWritable(HiveShim.createDecimal(value.underlying()))) + + def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + PrimitiveCategory.VOID, null) + def createDriverResultsArray = new JArrayList[String] def processResults(results: JArrayList[String]) = results diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index b9a742cc6ef67..42cd65b2518c9 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -21,6 +21,7 @@ import java.util.{ArrayList => JArrayList} import java.util.Properties import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.common.`type`.{HiveDecimal} import org.apache.hadoop.hive.conf.HiveConf @@ -28,10 +29,16 @@ import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} -import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.Logging +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} +import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} +import org.apache.spark.Logging + import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -54,6 +61,59 @@ private[hive] object HiveShim { new TableDesc(inputFormatClass, outputFormatClass, properties) } + def getPrimitiveWritableConstantObjectInspector(value: String): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.stringTypeInfo, new hadoopIo.Text(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Int): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.intTypeInfo, new hadoopIo.IntWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Double): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.doubleTypeInfo, new hiveIo.DoubleWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Boolean): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.booleanTypeInfo, new hadoopIo.BooleanWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Long): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.longTypeInfo, new hadoopIo.LongWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Float): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.floatTypeInfo, new hadoopIo.FloatWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Short): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.shortTypeInfo, new hiveIo.ShortWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Byte): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.byteTypeInfo, new hiveIo.ByteWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: Array[Byte]): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.binaryTypeInfo, new hadoopIo.BytesWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Date): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.dateTypeInfo, new hiveIo.DateWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: java.sql.Timestamp): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.timestampTypeInfo, new hiveIo.TimestampWritable(value)) + + def getPrimitiveWritableConstantObjectInspector(value: BigDecimal): ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.decimalTypeInfo, + new hiveIo.HiveDecimalWritable(HiveShim.createDecimal(value.underlying()))) + + def getPrimitiveNullWritableConstantObjectInspector: ObjectInspector = + PrimitiveObjectInspectorFactory.getPrimitiveWritableConstantObjectInspector( + TypeInfoFactory.voidTypeInfo, null) + def createDriverResultsArray = new JArrayList[Object] def processResults(results: JArrayList[Object]) = { From 8c0bfd08fc19fa5de7d77bf8306d19834f907ec0 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 28 Oct 2014 19:38:16 -0700 Subject: [PATCH 1125/1492] [SPARK-4133] [SQL] [PySpark] type conversionfor python udf Call Python UDF on ArrayType/MapType/PrimitiveType, the returnType can also be ArrayType/MapType/PrimitiveType. For StructType, it will act as tuple (without attributes). If returnType is StructType, it also should be tuple. Author: Davies Liu Closes #2973 from davies/udf_array and squashes the following commits: 306956e [Davies Liu] Merge branch 'master' of github.com:apache/spark into udf_array 2c00e43 [Davies Liu] fix merge 11395fa [Davies Liu] Merge branch 'master' of github.com:apache/spark into udf_array 9df50a2 [Davies Liu] address comments 79afb4e [Davies Liu] type conversionfor python udf --- python/pyspark/tests.py | 16 +++- .../org/apache/spark/sql/SQLContext.scala | 43 +-------- .../org/apache/spark/sql/SchemaRDD.scala | 42 +-------- .../spark/sql/execution/pythonUdfs.scala | 91 +++++++++++++++++-- 4 files changed, 102 insertions(+), 90 deletions(-) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 047d85783089f..37a128907b3a7 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -49,7 +49,7 @@ from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter -from pyspark.sql import SQLContext, IntegerType, Row +from pyspark.sql import SQLContext, IntegerType, Row, ArrayType from pyspark import shuffle _have_scipy = False @@ -690,10 +690,20 @@ def test_udf(self): self.assertEqual(row[0], 5) def test_udf2(self): - self.sqlCtx.registerFunction("strlen", lambda string: len(string)) + self.sqlCtx.registerFunction("strlen", lambda string: len(string), IntegerType()) self.sqlCtx.inferSchema(self.sc.parallelize([Row(a="test")])).registerTempTable("test") [res] = self.sqlCtx.sql("SELECT strlen(a) FROM test WHERE strlen(a) > 1").collect() - self.assertEqual(u"4", res[0]) + self.assertEqual(4, res[0]) + + def test_udf_with_array_type(self): + d = [Row(l=range(3), d={"key": range(5)})] + rdd = self.sc.parallelize(d) + srdd = self.sqlCtx.inferSchema(rdd).registerTempTable("test") + self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) + self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType()) + [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect() + self.assertEqual(range(3), l1) + self.assertEqual(1, l2) def test_broadcast_in_udf(self): bar = {"a": "aa", "b": "bb", "c": "abc"} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index ca8706ee68697..a41a500c9a5d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -438,7 +438,6 @@ class SQLContext(@transient val sparkContext: SparkContext) private[sql] def applySchemaToPythonRDD( rdd: RDD[Array[Any]], schema: StructType): SchemaRDD = { - import scala.collection.JavaConversions._ def needsConversion(dataType: DataType): Boolean = dataType match { case ByteType => true @@ -452,49 +451,9 @@ class SQLContext(@transient val sparkContext: SparkContext) case other => false } - // Converts value to the type specified by the data type. - // Because Python does not have data types for DateType, TimestampType, FloatType, ShortType, - // and ByteType, we need to explicitly convert values in columns of these data types to the - // desired JVM data types. - def convert(obj: Any, dataType: DataType): Any = (obj, dataType) match { - // TODO: We should check nullable - case (null, _) => null - - case (c: java.util.List[_], ArrayType(elementType, _)) => - c.map { e => convert(e, elementType)}: Seq[Any] - - case (c, ArrayType(elementType, _)) if c.getClass.isArray => - c.asInstanceOf[Array[_]].map(e => convert(e, elementType)): Seq[Any] - - case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { - case (key, value) => (convert(key, keyType), convert(value, valueType)) - }.toMap - - case (c, StructType(fields)) if c.getClass.isArray => - new GenericRow(c.asInstanceOf[Array[_]].zip(fields).map { - case (e, f) => convert(e, f.dataType) - }): Row - - case (c: java.util.Calendar, DateType) => - new java.sql.Date(c.getTime().getTime()) - - case (c: java.util.Calendar, TimestampType) => - new java.sql.Timestamp(c.getTime().getTime()) - - case (c: Int, ByteType) => c.toByte - case (c: Long, ByteType) => c.toByte - case (c: Int, ShortType) => c.toShort - case (c: Long, ShortType) => c.toShort - case (c: Long, IntegerType) => c.toInt - case (c: Double, FloatType) => c.toFloat - case (c, StringType) if !c.isInstanceOf[String] => c.toString - - case (c, _) => c - } - val convertedRdd = if (schema.fields.exists(f => needsConversion(f.dataType))) { rdd.map(m => m.zip(schema.fields).map { - case (value, field) => convert(value, field.dataType) + case (value, field) => EvaluatePython.fromJava(value, field.dataType) }) } else { rdd diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 948122d42f0e1..8b96df10963b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} -import org.apache.spark.sql.execution.LogicalRDD +import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} import org.apache.spark.api.java.JavaRDD /** @@ -377,47 +377,15 @@ class SchemaRDD( */ def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan) - /** - * Helper for converting a Row to a simple Array suitable for pyspark serialization. - */ - private def rowToJArray(row: Row, structType: StructType): Array[Any] = { - import scala.collection.Map - - def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { - case (null, _) => null - - case (obj: Row, struct: StructType) => rowToJArray(obj, struct) - - case (seq: Seq[Any], array: ArrayType) => - seq.map(x => toJava(x, array.elementType)).asJava - case (list: JList[_], array: ArrayType) => - list.map(x => toJava(x, array.elementType)).asJava - case (arr, array: ArrayType) if arr.getClass.isArray => - arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) - - case (obj: Map[_, _], mt: MapType) => obj.map { - case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type - }.asJava - - // Pyrolite can handle Timestamp - case (other, _) => other - } - - val fields = structType.fields.map(field => field.dataType) - row.zip(fields).map { - case (obj, dataType) => toJava(obj, dataType) - }.toArray - } - /** * Converts a JavaRDD to a PythonRDD. It is used by pyspark. */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { - val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) + val fieldTypes = schema.fields.map(_.dataType) this.mapPartitions { iter => val pickle = new Pickler iter.map { row => - rowToJArray(row, rowSchema) + EvaluatePython.rowToArray(row, fieldTypes) }.grouped(100).map(batched => pickle.dumps(batched.toArray)) } } @@ -427,10 +395,10 @@ class SchemaRDD( * format as javaToPython. It is used by pyspark. */ private[sql] def collectToPython: JList[Array[Byte]] = { - val rowSchema = StructType.fromAttributes(this.queryExecution.analyzed.output) + val fieldTypes = schema.fields.map(_.dataType) val pickle = new Pickler new java.util.ArrayList(collect().map { row => - rowToJArray(row, rowSchema) + EvaluatePython.rowToArray(row, fieldTypes) }.grouped(100).map(batched => pickle.dumps(batched.toArray)).toIterable) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index be729e5d244b0..a1961bba1899e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -19,11 +19,14 @@ package org.apache.spark.sql.execution import java.util.{List => JList, Map => JMap} +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonRDD import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -31,8 +34,6 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.types._ import org.apache.spark.{Accumulator, Logging => SparkLogging} -import scala.collection.JavaConversions._ - /** * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. */ @@ -108,6 +109,80 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { object EvaluatePython { def apply(udf: PythonUDF, child: LogicalPlan) = new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) + + /** + * Helper for converting a Scala object to a java suitable for pyspark serialization. + */ + def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + case (null, _) => null + + case (row: Row, struct: StructType) => + val fields = struct.fields.map(field => field.dataType) + row.zip(fields).map { + case (obj, dataType) => toJava(obj, dataType) + }.toArray + + case (seq: Seq[Any], array: ArrayType) => + seq.map(x => toJava(x, array.elementType)).asJava + case (list: JList[_], array: ArrayType) => + list.map(x => toJava(x, array.elementType)).asJava + case (arr, array: ArrayType) if arr.getClass.isArray => + arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) + + case (obj: Map[_, _], mt: MapType) => obj.map { + case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type + }.asJava + + // Pyrolite can handle Timestamp + case (other, _) => other + } + + /** + * Convert Row into Java Array (for pickled into Python) + */ + def rowToArray(row: Row, fields: Seq[DataType]): Array[Any] = { + row.zip(fields).map {case (obj, dt) => toJava(obj, dt)}.toArray + } + + // Converts value to the type specified by the data type. + // Because Python does not have data types for TimestampType, FloatType, ShortType, and + // ByteType, we need to explicitly convert values in columns of these data types to the desired + // JVM data types. + def fromJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { + // TODO: We should check nullable + case (null, _) => null + + case (c: java.util.List[_], ArrayType(elementType, _)) => + c.map { e => fromJava(e, elementType)}: Seq[Any] + + case (c, ArrayType(elementType, _)) if c.getClass.isArray => + c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType)): Seq[Any] + + case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { + case (key, value) => (fromJava(key, keyType), fromJava(value, valueType)) + }.toMap + + case (c, StructType(fields)) if c.getClass.isArray => + new GenericRow(c.asInstanceOf[Array[_]].zip(fields).map { + case (e, f) => fromJava(e, f.dataType) + }): Row + + case (c: java.util.Calendar, DateType) => + new java.sql.Date(c.getTime().getTime()) + + case (c: java.util.Calendar, TimestampType) => + new java.sql.Timestamp(c.getTime().getTime()) + + case (c: Int, ByteType) => c.toByte + case (c: Long, ByteType) => c.toByte + case (c: Int, ShortType) => c.toShort + case (c: Long, ShortType) => c.toShort + case (c: Long, IntegerType) => c.toInt + case (c: Double, FloatType) => c.toFloat + case (c, StringType) if !c.isInstanceOf[String] => c.toString + + case (c, _) => c + } } /** @@ -141,8 +216,11 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: val parent = childResults.mapPartitions { iter => val pickle = new Pickler val currentRow = newMutableProjection(udf.children, child.output)() + val fields = udf.children.map(_.dataType) iter.grouped(1000).map { inputRows => - val toBePickled = inputRows.map(currentRow(_).toArray).toArray + val toBePickled = inputRows.map { row => + EvaluatePython.rowToArray(currentRow(row), fields) + }.toArray pickle.dumps(toBePickled) } } @@ -165,10 +243,7 @@ case class BatchPythonEvaluation(udf: PythonUDF, output: Seq[Attribute], child: }.mapPartitions { iter => val row = new GenericMutableRow(1) iter.map { result => - row(0) = udf.dataType match { - case StringType => result.toString - case other => result - } + row(0) = EvaluatePython.fromJava(result, udf.dataType) row: Row } } From 1559495dd961d299299a27aae2cb940e8c6697c5 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 28 Oct 2014 22:00:10 -0700 Subject: [PATCH 1126/1492] [FIX] disable benchmark code I forgot to disable the benchmark code in #2937, which increased the Jenkins build time by couple minutes. aarondav Author: Xiangrui Meng Closes #2990 from mengxr/disable-benchmark and squashes the following commits: c58f070 [Xiangrui Meng] disable benchmark code --- .../scala/org/apache/spark/util/collection/SorterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala index 066d47c46a0d2..0cb1ed7397655 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/SorterSuite.scala @@ -157,7 +157,7 @@ class SorterSuite extends FunSuite { * here is mainly to have the code. Running multiple tests within the same JVM session would * prevent JIT inlining overridden methods and hence hurt the performance. */ - test("Sorter benchmark for primitive int array") { + ignore("Sorter benchmark for primitive int array") { val numElements = 25000000 // 25 mil val rand = new XORShiftRandom(123) From 51ce997355465fc5c29d0e49b92f9bae0bab90ed Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Wed, 29 Oct 2014 10:14:53 -0700 Subject: [PATCH 1127/1492] [SPARK-4129][MLlib] Performance tuning in MultivariateOnlineSummarizer In MultivariateOnlineSummarizer, breeze's activeIterator is used to loop through the nonZero elements in the vector. However, activeIterator doesn't perform well due to lots of overhead. In this PR, native while loop is used for both DenseVector and SparseVector. The benchmark result with 20 executors using mnist8m dataset: Before: DenseVector: 48.2 seconds SparseVector: 16.3 seconds After: DenseVector: 17.8 seconds SparseVector: 11.2 seconds Since MultivariateOnlineSummarizer is used in several places, the overall performance gain in mllib library will be significant with this PR. Author: DB Tsai Closes #2992 from dbtsai/SPARK-4129 and squashes the following commits: b99db6c [DB Tsai] fixed java.lang.ArrayIndexOutOfBoundsException 2b5e882 [DB Tsai] small refactoring ebe3e74 [DB Tsai] First commit --- .../stat/MultivariateOnlineSummarizer.scala | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 3025d4837cab4..fab7c4405c65d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.stat import breeze.linalg.{DenseVector => BDV} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{Vectors, Vector} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector} /** * :: DeveloperApi :: @@ -72,9 +72,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S require(n == sample.size, s"Dimensions mismatch when adding new sample." + s" Expecting $n but got ${sample.size}.") - sample.toBreeze.activeIterator.foreach { - case (_, 0.0) => // Skip explicit zero elements. - case (i, value) => + @inline def update(i: Int, value: Double) = { + if (value != 0.0) { if (currMax(i) < value) { currMax(i) = value } @@ -89,6 +88,24 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currL1(i) += math.abs(value) nnz(i) += 1.0 + } + } + + sample match { + case dv: DenseVector => { + var j = 0 + while (j < dv.size) { + update(j, dv.values(j)) + j += 1 + } + } + case sv: SparseVector => + var j = 0 + while (j < sv.indices.size) { + update(sv.indices(j), sv.values(j)) + j += 1 + } + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } totalCnt += 1 From dff015533dd7b01b5e392f1ac5f3837e0a65f3f4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 29 Oct 2014 11:27:07 -0700 Subject: [PATCH 1128/1492] [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core This PR encapsulates #2330, which is itself a continuation of #2240. The first goal of this PR is to provide an alternate, simpler implementation of the ConnectionManager which is based on Netty. In addition to this goal, however, we want to resolve [SPARK-3796](https://issues.apache.org/jira/browse/SPARK-3796), which calls for a standalone shuffle service which can be integrated into the YARN NodeManager, Standalone Worker, or on its own. This PR makes the first step in this direction by ensuring that the actual Netty service is as small as possible and extracted from Spark core. Given this, we should be able to construct this standalone jar which can be included in other JVMs without incurring significant dependency or runtime issues. The actual work to ensure that such a standalone shuffle service would work in Spark will be left for a future PR, however. In order to minimize dependencies and allow for the service to be long-running (possibly much longer-running than Spark, and possibly having to support multiple version of Spark simultaneously), the entire service has been ported to Java, where we have full control over the binary compatibility of the components and do not depend on the Scala runtime or version. These issues: have been addressed by folding in #2330: SPARK-3453: Refactor Netty module to use BlockTransferService interface SPARK-3018: Release all buffers upon task completion/failure SPARK-3002: Create a connection pool and reuse clients across different threads SPARK-3017: Integration tests and unit tests for connection failures SPARK-3049: Make sure client doesn't block when server/connection has error(s) SPARK-3502: SO_RCVBUF and SO_SNDBUF should be bootstrap childOption, not option SPARK-3503: Disable thread local cache in PooledByteBufAllocator TODO before mergeable: - [x] Implement uploadBlock() - [x] Unit tests for RPC side of code - [x] Performance testing (see comments [here](https://github.com/apache/spark/pull/2753#issuecomment-59475022)) - [x] Turn OFF by default (currently on for unit testing) Author: Reynold Xin Author: Aaron Davidson Author: cocoatomo Author: Patrick Wendell Author: Prashant Sharma Author: Davies Liu Author: Anand Avati Closes #2753 from aarondav/netty and squashes the following commits: cadfd28 [Aaron Davidson] Turn netty off by default d7be11b [Aaron Davidson] Turn netty on by default 4a204b8 [Aaron Davidson] Fail block fetches if client connection fails 2b0d1c0 [Aaron Davidson] 100ch 0c5bca2 [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty 14e37f7 [Aaron Davidson] Address Reynold's comments 8dfcceb [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty 322dfc1 [Aaron Davidson] Address Reynold's comments, including major rename e5675a4 [Aaron Davidson] Fail outstanding RPCs as well ccd4959 [Aaron Davidson] Don't throw exception if client immediately fails 9da0bc1 [Aaron Davidson] Add RPC unit tests d236dfd [Aaron Davidson] Remove no-op serializer :) 7b7a26c [Aaron Davidson] Fix Nio compile issue dd420fd [Aaron Davidson] Merge branch 'master' of https://github.com/apache/spark into netty-test 939f276 [Aaron Davidson] Attempt to make comm. bidirectional aa58f67 [cocoatomo] [SPARK-3909][PySpark][Doc] A corrupted format in Sphinx documents and building warnings 8dc1ded [cocoatomo] [SPARK-3867][PySpark] ./python/run-tests failed when it run with Python 2.6 and unittest2 is not installed 5b5dbe6 [Prashant Sharma] [SPARK-2924] Required by scala 2.11, only one fun/ctor amongst overriden alternatives, can have default argument(s). 2c5d9dc [Patrick Wendell] HOTFIX: Fix build issue with Akka 2.3.4 upgrade. 020691e [Davies Liu] [SPARK-3886] [PySpark] use AutoBatchedSerializer by default ae4083a [Anand Avati] [SPARK-2805] Upgrade Akka to 2.3.4 29c6dcf [Aaron Davidson] [SPARK-3453] Netty-based BlockTransferService, extracted from Spark core f7e7568 [Reynold Xin] Fixed spark.shuffle.io.receiveBuffer setting. 5d98ce3 [Reynold Xin] Flip buffer. f6c220d [Reynold Xin] Merge with latest master. 407e59a [Reynold Xin] Fix style violation. a0518c7 [Reynold Xin] Implemented block uploads. 4b18db2 [Reynold Xin] Copy the buffer in fetchBlockSync. bec4ea2 [Reynold Xin] Removed OIO and added num threads settings. 1bdd7ee [Reynold Xin] Fixed tests. d68f328 [Reynold Xin] Logging close() in case close() fails. f63fb4c [Reynold Xin] Add more debug message. 6afc435 [Reynold Xin] Added logging. c066309 [Reynold Xin] Implement java.io.Closeable interface. 519d64d [Reynold Xin] Mark private package visibility and MimaExcludes. f0a16e9 [Reynold Xin] Fixed test hanging. 14323a5 [Reynold Xin] Removed BlockManager.getLocalShuffleFromDisk. b2f3281 [Reynold Xin] Added connection pooling. d23ed7b [Reynold Xin] Incorporated feedback from Norman: - use same pool for boss and worker - remove ioratio - disable caching of byte buf allocator - childoption sendbuf/receivebuf - fire exception through pipeline 9e0cb87 [Reynold Xin] Fixed BlockClientHandlerSuite 5cd33d7 [Reynold Xin] Fixed style violation. cb589ec [Reynold Xin] Added more test cases covering cleanup when fault happens in ShuffleBlockFetcherIteratorSuite 1be4e8e [Reynold Xin] Shorten NioManagedBuffer and NettyManagedBuffer class names. 108c9ed [Reynold Xin] Forgot to add TestSerializer to the commit list. b5c8d1f [Reynold Xin] Fixed ShuffleBlockFetcherIteratorSuite. 064747b [Reynold Xin] Reference count buffers and clean them up properly. 2b44cf1 [Reynold Xin] Added more documentation. 1760d32 [Reynold Xin] Use Epoll.isAvailable in BlockServer as well. 165eab1 [Reynold Xin] [SPARK-3453] Refactor Netty module to use BlockTransferService. --- core/pom.xml | 5 + .../scala/org/apache/spark/SparkEnv.scala | 9 +- .../spark/network/BlockDataManager.scala | 14 +- .../spark/network/BlockFetchingListener.scala | 11 +- .../spark/network/BlockTransferService.scala | 28 +- .../apache/spark/network/ManagedBuffer.scala | 166 ----------- .../network/netty/NettyBlockFetcher.scala | 95 +++++++ .../network/netty/NettyBlockRpcServer.scala | 76 +++++ .../netty/NettyBlockTransferService.scala | 111 ++++++++ .../spark/network/netty/NettyConfig.scala | 59 ---- .../netty/client/BlockFetchingClient.scala | 132 --------- .../client/BlockFetchingClientFactory.scala | 99 ------- .../client/BlockFetchingClientHandler.scala | 104 ------- .../netty/client/LazyInitIterator.scala | 44 --- .../netty/client/ReferenceCountedBuffer.scala | 47 ---- .../netty/server/BlockHeaderEncoder.scala | 47 ---- .../network/netty/server/BlockServer.scala | 162 ----------- .../BlockServerChannelInitializer.scala | 40 --- .../netty/server/BlockServerHandler.scala | 140 ---------- .../network/nio/NioBlockTransferService.scala | 51 ++-- .../apache/spark/serializer/Serializer.scala | 6 +- .../shuffle/FileShuffleBlockManager.scala | 4 +- .../shuffle/IndexShuffleBlockManager.scala | 2 +- .../spark/shuffle/ShuffleBlockManager.scala | 3 +- .../apache/spark/storage/BlockManager.scala | 52 ++-- .../storage/BlockNotFoundException.scala | 1 - .../storage/ShuffleBlockFetcherIterator.scala | 135 ++++++--- .../scala/org/apache/spark/util/Utils.scala | 1 - .../org/apache/spark/ShuffleNettySuite.scala | 4 +- .../netty/ServerClientIntegrationSuite.scala | 161 ----------- .../BlockFetchingClientHandlerSuite.scala | 106 ------- .../server/BlockHeaderEncoderSuite.scala | 64 ----- .../server/BlockServerHandlerSuite.scala | 107 ------- .../spark/serializer/TestSerializer.scala | 60 ++++ .../hash/HashShuffleManagerSuite.scala | 8 +- .../ShuffleBlockFetcherIteratorSuite.scala | 261 +++++++++++------- network/common/pom.xml | 94 +++++++ .../spark/network/TransportContext.java | 117 ++++++++ .../buffer/FileSegmentManagedBuffer.java | 154 +++++++++++ .../spark/network/buffer/ManagedBuffer.java | 71 +++++ .../network/buffer/NettyManagedBuffer.java | 76 +++++ .../network/buffer/NioManagedBuffer.java | 75 +++++ .../client/ChunkFetchFailureException.java | 21 +- .../network/client/ChunkReceivedCallback.java | 47 ++++ .../network/client/RpcResponseCallback.java | 22 +- .../spark/network/client/TransportClient.java | 159 +++++++++++ .../client/TransportClientFactory.java | 182 ++++++++++++ .../client/TransportResponseHandler.java | 167 +++++++++++ .../network/protocol/ChunkFetchFailure.java | 76 +++++ .../network/protocol/ChunkFetchRequest.java | 66 +++++ .../network/protocol/ChunkFetchSuccess.java | 80 ++++++ .../spark/network/protocol/Encodable.java | 41 +++ .../spark/network/protocol/Message.java | 58 ++++ .../network/protocol/MessageDecoder.java | 70 +++++ .../network/protocol/MessageEncoder.java | 80 ++++++ .../network/protocol/RequestMessage.java | 10 +- .../network/protocol/ResponseMessage.java | 14 +- .../spark/network/protocol/RpcFailure.java | 74 +++++ .../spark/network/protocol/RpcRequest.java | 81 ++++++ .../spark/network/protocol/RpcResponse.java | 72 +++++ .../spark/network/protocol/StreamChunkId.java | 73 +++++ .../network/server/DefaultStreamManager.java | 104 +++++++ .../spark/network/server/MessageHandler.java | 36 +++ .../spark/network/server/RpcHandler.java | 38 +++ .../spark/network/server/StreamManager.java | 52 ++++ .../server/TransportChannelHandler.java | 96 +++++++ .../server/TransportRequestHandler.java | 162 +++++++++++ .../spark/network/server/TransportServer.java | 121 ++++++++ .../spark/network/util/ConfigProvider.java | 52 ++++ .../org/apache/spark/network/util/IOMode.java | 27 ++ .../apache/spark/network/util/JavaUtils.java | 38 +++ .../apache/spark/network/util/NettyUtils.java | 102 +++++++ .../spark/network/util/TransportConf.java | 61 ++++ .../network/ChunkFetchIntegrationSuite.java | 217 +++++++++++++++ .../apache/spark/network/NoOpRpcHandler.java | 28 ++ .../apache/spark/network/ProtocolSuite.java | 86 ++++++ .../spark/network/RpcIntegrationSuite.java | 175 ++++++++++++ .../network/SystemPropertyConfigProvider.java | 34 +++ .../spark/network/TestManagedBuffer.java | 104 +++++++ .../org/apache/spark/network/TestUtils.java | 30 ++ .../network/TransportClientFactorySuite.java | 102 +++++++ .../TransportResponseHandlerSuite.java | 115 ++++++++ pom.xml | 1 + project/MimaExcludes.scala | 5 + 84 files changed, 4431 insertions(+), 1750 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala delete mode 100644 core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala create mode 100644 network/common/pom.xml create mode 100644 network/common/src/main/java/org/apache/spark/network/TransportContext.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java rename core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala => network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java (67%) create mode 100644 network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java rename core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala => network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java (55%) create mode 100644 network/common/src/main/java/org/apache/spark/network/client/TransportClient.java create mode 100644 network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java create mode 100644 network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/Message.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java rename core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala => network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java (77%) rename core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala => network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java (75%) create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java create mode 100644 network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/StreamManager.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java create mode 100644 network/common/src/main/java/org/apache/spark/network/server/TransportServer.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/IOMode.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java create mode 100644 network/common/src/main/java/org/apache/spark/network/util/TransportConf.java create mode 100644 network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java create mode 100644 network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TestUtils.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java create mode 100644 network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java diff --git a/core/pom.xml b/core/pom.xml index 5cd21e18e8ca7..8020a2daf81ec 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -44,6 +44,11 @@
    + + org.apache.spark + network + ${project.version} + net.java.dev.jets3t jets3t diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5c076e5f1c11d..6a6dfda363974 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -32,6 +32,7 @@ import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.netty.{NettyBlockTransferService} import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer @@ -272,7 +273,13 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) - val blockTransferService = new NioBlockTransferService(conf, securityManager) + val blockTransferService = + conf.get("spark.shuffle.blockTransferService", "nio").toLowerCase match { + case "netty" => + new NettyBlockTransferService(conf) + case "nio" => + new NioBlockTransferService(conf, securityManager) + } val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index e0e91724271c8..1745d52c81923 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -17,20 +17,20 @@ package org.apache.spark.network -import org.apache.spark.storage.StorageLevel - +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.storage.{BlockId, StorageLevel} +private[spark] trait BlockDataManager { /** - * Interface to get local block data. - * - * @return Some(buffer) if the block exists locally, and None if it doesn't. + * Interface to get local block data. Throws an exception if the block cannot be found or + * cannot be read successfully. */ - def getBlockData(blockId: String): Option[ManagedBuffer] + def getBlockData(blockId: BlockId): ManagedBuffer /** * Put the block locally, using the given storage level. */ - def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit + def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala index 34acaa563ca58..645793fde806d 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala @@ -19,19 +19,24 @@ package org.apache.spark.network import java.util.EventListener +import org.apache.spark.network.buffer.ManagedBuffer + /** * Listener callback interface for [[BlockTransferService.fetchBlocks]]. */ +private[spark] trait BlockFetchingListener extends EventListener { /** - * Called once per successfully fetched block. + * Called once per successfully fetched block. After this call returns, data will be released + * automatically. If the data will be passed to another thread, the receiver should retain() + * and release() the buffer on their own, or copy the data to a new buffer. */ def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit /** - * Called upon failures. For each failure, this is called only once (i.e. not once per block). + * Called at least once per block upon failures. */ - def onBlockFetchFailure(exception: Throwable): Unit + def onBlockFetchFailure(blockId: String, exception: Throwable): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 84d991fa6808c..b083f465334fe 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -17,13 +17,19 @@ package org.apache.spark.network +import java.io.Closeable +import java.nio.ByteBuffer + import scala.concurrent.{Await, Future} import scala.concurrent.duration.Duration -import org.apache.spark.storage.StorageLevel - +import org.apache.spark.Logging +import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.Utils -abstract class BlockTransferService { +private[spark] +abstract class BlockTransferService extends Closeable with Logging { /** * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch @@ -34,7 +40,7 @@ abstract class BlockTransferService { /** * Tear down the transfer service. */ - def stop(): Unit + def close(): Unit /** * Port number the service is listening on, available only after [[init]] is invoked. @@ -50,9 +56,6 @@ abstract class BlockTransferService { * Fetch a sequence of blocks from a remote node asynchronously, * available only after [[init]] is invoked. * - * Note that [[BlockFetchingListener.onBlockFetchSuccess]] is called once per block, - * while [[BlockFetchingListener.onBlockFetchFailure]] is called once per failure (not per block). - * * Note that this API takes a sequence so the implementation can batch requests, and does not * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as * the data of a block is fetched, rather than waiting for all blocks to be fetched. @@ -69,7 +72,7 @@ abstract class BlockTransferService { def uploadBlock( hostname: String, port: Int, - blockId: String, + blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Future[Unit] @@ -83,7 +86,7 @@ abstract class BlockTransferService { val lock = new Object @volatile var result: Either[ManagedBuffer, Throwable] = null fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener { - override def onBlockFetchFailure(exception: Throwable): Unit = { + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { lock.synchronized { result = Right(exception) lock.notify() @@ -91,7 +94,10 @@ abstract class BlockTransferService { } override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { lock.synchronized { - result = Left(data) + val ret = ByteBuffer.allocate(data.size.toInt) + ret.put(data.nioByteBuffer()) + ret.flip() + result = Left(new NioManagedBuffer(ret)) lock.notify() } } @@ -123,7 +129,7 @@ abstract class BlockTransferService { def uploadBlockSync( hostname: String, port: Int, - blockId: String, + blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel): Unit = { Await.result(uploadBlock(hostname, port, blockId, blockData, level), Duration.Inf) diff --git a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala b/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala deleted file mode 100644 index 4211ba4e43b9e..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/ManagedBuffer.scala +++ /dev/null @@ -1,166 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network - -import java.io._ -import java.nio.ByteBuffer -import java.nio.channels.FileChannel -import java.nio.channels.FileChannel.MapMode - -import scala.util.Try - -import com.google.common.io.ByteStreams -import io.netty.buffer.{ByteBufInputStream, ByteBuf} - -import org.apache.spark.util.{ByteBufferInputStream, Utils} - - -/** - * This interface provides an immutable view for data in the form of bytes. The implementation - * should specify how the data is provided: - * - * - FileSegmentManagedBuffer: data backed by part of a file - * - NioByteBufferManagedBuffer: data backed by a NIO ByteBuffer - * - NettyByteBufManagedBuffer: data backed by a Netty ByteBuf - */ -sealed abstract class ManagedBuffer { - // Note that all the methods are defined with parenthesis because their implementations can - // have side effects (io operations). - - /** Number of bytes of the data. */ - def size: Long - - /** - * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the - * returned ByteBuffer should not affect the content of this buffer. - */ - def nioByteBuffer(): ByteBuffer - - /** - * Exposes this buffer's data as an InputStream. The underlying implementation does not - * necessarily check for the length of bytes read, so the caller is responsible for making sure - * it does not go over the limit. - */ - def inputStream(): InputStream -} - - -/** - * A [[ManagedBuffer]] backed by a segment in a file - */ -final class FileSegmentManagedBuffer(val file: File, val offset: Long, val length: Long) - extends ManagedBuffer { - - /** - * Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889). - * Avoid unless there's a good reason not to. - */ - private val MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024; - - override def size: Long = length - - override def nioByteBuffer(): ByteBuffer = { - var channel: FileChannel = null - try { - channel = new RandomAccessFile(file, "r").getChannel - // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. - if (length < MIN_MEMORY_MAP_BYTES) { - val buf = ByteBuffer.allocate(length.toInt) - channel.position(offset) - while (buf.remaining() != 0) { - if (channel.read(buf) == -1) { - throw new IOException("Reached EOF before filling buffer\n" + - s"offset=$offset\nfile=${file.getAbsolutePath}\nbuf.remaining=${buf.remaining}") - } - } - buf.flip() - buf - } else { - channel.map(MapMode.READ_ONLY, offset, length) - } - } catch { - case e: IOException => - Try(channel.size).toOption match { - case Some(fileLen) => - throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) - case None => - throw new IOException(s"Error in opening $this", e) - } - } finally { - if (channel != null) { - Utils.tryLog(channel.close()) - } - } - } - - override def inputStream(): InputStream = { - var is: FileInputStream = null - try { - is = new FileInputStream(file) - ByteStreams.skipFully(is, offset) - ByteStreams.limit(is, length) - } catch { - case e: IOException => - if (is != null) { - Utils.tryLog(is.close()) - } - Try(file.length).toOption match { - case Some(fileLen) => - throw new IOException(s"Error in reading $this (actual file length $fileLen)", e) - case None => - throw new IOException(s"Error in opening $this", e) - } - case e: Throwable => - if (is != null) { - Utils.tryLog(is.close()) - } - throw e - } - } - - override def toString: String = s"${getClass.getName}($file, $offset, $length)" -} - - -/** - * A [[ManagedBuffer]] backed by [[java.nio.ByteBuffer]]. - */ -final class NioByteBufferManagedBuffer(buf: ByteBuffer) extends ManagedBuffer { - - override def size: Long = buf.remaining() - - override def nioByteBuffer() = buf.duplicate() - - override def inputStream() = new ByteBufferInputStream(buf) -} - - -/** - * A [[ManagedBuffer]] backed by a Netty [[ByteBuf]]. - */ -final class NettyByteBufManagedBuffer(buf: ByteBuf) extends ManagedBuffer { - - override def size: Long = buf.readableBytes() - - override def nioByteBuffer() = buf.nioBuffer() - - override def inputStream() = new ByteBufInputStream(buf) - - // TODO(rxin): Promote this to top level ManagedBuffer interface and add documentation for it. - def release(): Unit = buf.release() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala new file mode 100644 index 0000000000000..8c5ffd8da6bbb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.nio.ByteBuffer +import java.util + +import org.apache.spark.{SparkConf, Logging} +import org.apache.spark.network.BlockFetchingListener +import org.apache.spark.network.netty.NettyMessages._ +import org.apache.spark.serializer.{JavaSerializer, Serializer} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, TransportClient} +import org.apache.spark.storage.BlockId +import org.apache.spark.util.Utils + +/** + * Responsible for holding the state for a request for a single set of blocks. This assumes that + * the chunks will be returned in the same order as requested, and that there will be exactly + * one chunk per block. + * + * Upon receipt of any block, the listener will be called back. Upon failure part way through, + * the listener will receive a failure callback for each outstanding block. + */ +class NettyBlockFetcher( + serializer: Serializer, + client: TransportClient, + blockIds: Seq[String], + listener: BlockFetchingListener) + extends Logging { + + require(blockIds.nonEmpty) + + private val ser = serializer.newInstance() + + private var streamHandle: ShuffleStreamHandle = _ + + private val chunkCallback = new ChunkReceivedCallback { + // On receipt of a chunk, pass it upwards as a block. + def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions { + listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer) + } + + // On receipt of a failure, fail every block from chunkIndex onwards. + def onFailure(chunkIndex: Int, e: Throwable): Unit = { + blockIds.drop(chunkIndex).foreach { blockId => + listener.onBlockFetchFailure(blockId, e); + } + } + } + + /** Begins the fetching process, calling the listener with every block fetched. */ + def start(): Unit = { + // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle. + client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(), + new RpcResponseCallback { + override def onSuccess(response: Array[Byte]): Unit = { + try { + streamHandle = ser.deserialize[ShuffleStreamHandle](ByteBuffer.wrap(response)) + logTrace(s"Successfully opened block set: $streamHandle! Preparing to fetch chunks.") + + // Immediately request all chunks -- we expect that the total size of the request is + // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. + for (i <- 0 until streamHandle.numChunks) { + client.fetchChunk(streamHandle.streamId, i, chunkCallback) + } + } catch { + case e: Exception => + logError("Failed while starting block fetches", e) + blockIds.foreach(blockId => Utils.tryLog(listener.onBlockFetchFailure(blockId, e))) + } + } + + override def onFailure(e: Throwable): Unit = { + logError("Failed while starting block fetches", e) + blockIds.foreach(blockId => Utils.tryLog(listener.onBlockFetchFailure(blockId, e))) + } + }) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala new file mode 100644 index 0000000000000..02c657e1d61b5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import java.nio.ByteBuffer + +import org.apache.spark.Logging +import org.apache.spark.network.BlockDataManager +import org.apache.spark.serializer.Serializer +import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} +import org.apache.spark.network.client.{TransportClient, RpcResponseCallback} +import org.apache.spark.network.server.{DefaultStreamManager, RpcHandler} +import org.apache.spark.storage.{StorageLevel, BlockId} + +import scala.collection.JavaConversions._ + +object NettyMessages { + + /** Request to read a set of blocks. Returns [[ShuffleStreamHandle]] to identify the stream. */ + case class OpenBlocks(blockIds: Seq[BlockId]) + + /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */ + case class UploadBlock(blockId: BlockId, blockData: Array[Byte], level: StorageLevel) + + /** Identifier for a fixed number of chunks to read from a stream created by [[OpenBlocks]]. */ + case class ShuffleStreamHandle(streamId: Long, numChunks: Int) +} + +/** + * Serves requests to open blocks by simply registering one chunk per block requested. + */ +class NettyBlockRpcServer( + serializer: Serializer, + streamManager: DefaultStreamManager, + blockManager: BlockDataManager) + extends RpcHandler with Logging { + + import NettyMessages._ + + override def receive( + client: TransportClient, + messageBytes: Array[Byte], + responseContext: RpcResponseCallback): Unit = { + val ser = serializer.newInstance() + val message = ser.deserialize[AnyRef](ByteBuffer.wrap(messageBytes)) + logTrace(s"Received request: $message") + + message match { + case OpenBlocks(blockIds) => + val blocks: Seq[ManagedBuffer] = blockIds.map(blockManager.getBlockData) + val streamId = streamManager.registerStream(blocks.iterator) + logTrace(s"Registered streamId $streamId with ${blocks.size} buffers") + responseContext.onSuccess( + ser.serialize(new ShuffleStreamHandle(streamId, blocks.size)).array()) + + case UploadBlock(blockId, blockData, level) => + blockManager.putBlockData(blockId, new NioManagedBuffer(ByteBuffer.wrap(blockData)), level) + responseContext.onSuccess(new Array[Byte](0)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala new file mode 100644 index 0000000000000..38a3e945155e8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import scala.concurrent.{Promise, Future} + +import org.apache.spark.SparkConf +import org.apache.spark.network._ +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.client.{RpcResponseCallback, TransportClient, TransportClientFactory} +import org.apache.spark.network.netty.NettyMessages.UploadBlock +import org.apache.spark.network.server._ +import org.apache.spark.network.util.{ConfigProvider, TransportConf} +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.util.Utils + +/** + * A BlockTransferService that uses Netty to fetch a set of blocks at at time. + */ +class NettyBlockTransferService(conf: SparkConf) extends BlockTransferService { + // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. + val serializer = new JavaSerializer(conf) + + // Create a TransportConfig using SparkConf. + private[this] val transportConf = new TransportConf( + new ConfigProvider { override def get(name: String) = conf.get(name) }) + + private[this] var transportContext: TransportContext = _ + private[this] var server: TransportServer = _ + private[this] var clientFactory: TransportClientFactory = _ + + override def init(blockDataManager: BlockDataManager): Unit = { + val streamManager = new DefaultStreamManager + val rpcHandler = new NettyBlockRpcServer(serializer, streamManager, blockDataManager) + transportContext = new TransportContext(transportConf, streamManager, rpcHandler) + clientFactory = transportContext.createClientFactory() + server = transportContext.createServer() + } + + override def fetchBlocks( + hostname: String, + port: Int, + blockIds: Seq[String], + listener: BlockFetchingListener): Unit = { + try { + val client = clientFactory.createClient(hostname, port) + new NettyBlockFetcher(serializer, client, blockIds, listener).start() + } catch { + case e: Exception => + logError("Exception while beginning fetchBlocks", e) + blockIds.foreach(listener.onBlockFetchFailure(_, e)) + } + } + + override def hostName: String = Utils.localHostName() + + override def port: Int = server.getPort + + override def uploadBlock( + hostname: String, + port: Int, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel): Future[Unit] = { + val result = Promise[Unit]() + val client = clientFactory.createClient(hostname, port) + + // Convert or copy nio buffer into array in order to serialize it. + val nioBuffer = blockData.nioByteBuffer() + val array = if (nioBuffer.hasArray) { + nioBuffer.array() + } else { + val data = new Array[Byte](nioBuffer.remaining()) + nioBuffer.get(data) + data + } + + val ser = serializer.newInstance() + client.sendRpc(ser.serialize(new UploadBlock(blockId, array, level)).array(), + new RpcResponseCallback { + override def onSuccess(response: Array[Byte]): Unit = { + logTrace(s"Successfully uploaded block $blockId") + result.success() + } + override def onFailure(e: Throwable): Unit = { + logError(s"Error while uploading block $blockId", e) + result.failure(e) + } + }) + + result.future + } + + override def close(): Unit = server.close() +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala deleted file mode 100644 index b5870152c5a64..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyConfig.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import org.apache.spark.SparkConf - -/** - * A central location that tracks all the settings we exposed to users. - */ -private[spark] -class NettyConfig(conf: SparkConf) { - - /** Port the server listens on. Default to a random port. */ - private[netty] val serverPort = conf.getInt("spark.shuffle.io.port", 0) - - /** IO mode: nio, oio, epoll, or auto (try epoll first and then nio). */ - private[netty] val ioMode = conf.get("spark.shuffle.io.mode", "nio").toLowerCase - - /** Connect timeout in secs. Default 60 secs. */ - private[netty] val connectTimeoutMs = conf.getInt("spark.shuffle.io.connectionTimeout", 60) * 1000 - - /** - * Percentage of the desired amount of time spent for I/O in the child event loops. - * Only applicable in nio and epoll. - */ - private[netty] val ioRatio = conf.getInt("spark.shuffle.io.netty.ioRatio", 80) - - /** Requested maximum length of the queue of incoming connections. */ - private[netty] val backLog: Option[Int] = conf.getOption("spark.shuffle.io.backLog").map(_.toInt) - - /** - * Receive buffer size (SO_RCVBUF). - * Note: the optimal size for receive buffer and send buffer should be - * latency * network_bandwidth. - * Assuming latency = 1ms, network_bandwidth = 10Gbps - * buffer size should be ~ 1.25MB - */ - private[netty] val receiveBuf: Option[Int] = - conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) - - /** Send buffer size (SO_SNDBUF). */ - private[netty] val sendBuf: Option[Int] = - conf.getOption("spark.shuffle.io.sendBuffer").map(_.toInt) -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala deleted file mode 100644 index 3ab13b96d7f00..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClient.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import java.util.concurrent.TimeoutException - -import com.google.common.base.Charsets.UTF_8 -import io.netty.bootstrap.Bootstrap -import io.netty.buffer.PooledByteBufAllocator -import io.netty.channel.socket.SocketChannel -import io.netty.channel.{ChannelFutureListener, ChannelFuture, ChannelInitializer, ChannelOption} -import io.netty.handler.codec.LengthFieldBasedFrameDecoder -import io.netty.handler.codec.string.StringEncoder - -import org.apache.spark.Logging - -/** - * Client for fetching data blocks from [[org.apache.spark.network.netty.server.BlockServer]]. - * Use [[BlockFetchingClientFactory]] to instantiate this client. - * - * The constructor blocks until a connection is successfully established. - * - * See [[org.apache.spark.network.netty.server.BlockServer]] for client/server protocol. - * - * Concurrency: thread safe and can be called from multiple threads. - */ -@throws[TimeoutException] -private[spark] -class BlockFetchingClient(factory: BlockFetchingClientFactory, hostname: String, port: Int) - extends Logging { - - private val handler = new BlockFetchingClientHandler - - /** Netty Bootstrap for creating the TCP connection. */ - private val bootstrap: Bootstrap = { - val b = new Bootstrap - b.group(factory.workerGroup) - .channel(factory.socketChannelClass) - // Use pooled buffers to reduce temporary buffer allocation - .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - // Disable Nagle's Algorithm since we don't want packets to wait - .option(ChannelOption.TCP_NODELAY, java.lang.Boolean.TRUE) - .option(ChannelOption.SO_KEEPALIVE, java.lang.Boolean.TRUE) - .option[Integer](ChannelOption.CONNECT_TIMEOUT_MILLIS, factory.conf.connectTimeoutMs) - - b.handler(new ChannelInitializer[SocketChannel] { - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("encoder", new StringEncoder(UTF_8)) - // maxFrameLength = 2G, lengthFieldOffset = 0, lengthFieldLength = 4 - .addLast("framedLengthDecoder", new LengthFieldBasedFrameDecoder(Int.MaxValue, 0, 4)) - .addLast("handler", handler) - } - }) - b - } - - /** Netty ChannelFuture for the connection. */ - private val cf: ChannelFuture = bootstrap.connect(hostname, port) - if (!cf.awaitUninterruptibly(factory.conf.connectTimeoutMs)) { - throw new TimeoutException( - s"Connecting to $hostname:$port timed out (${factory.conf.connectTimeoutMs} ms)") - } - - /** - * Ask the remote server for a sequence of blocks, and execute the callback. - * - * Note that this is asynchronous and returns immediately. Upstream caller should throttle the - * rate of fetching; otherwise we could run out of memory. - * - * @param blockIds sequence of block ids to fetch. - * @param listener callback to fire on fetch success / failure. - */ - def fetchBlocks(blockIds: Seq[String], listener: BlockClientListener): Unit = { - // It's best to limit the number of "write" calls since it needs to traverse the whole pipeline. - // It's also best to limit the number of "flush" calls since it requires system calls. - // Let's concatenate the string and then call writeAndFlush once. - // This is also why this implementation might be more efficient than multiple, separate - // fetch block calls. - var startTime: Long = 0 - logTrace { - startTime = System.nanoTime - s"Sending request $blockIds to $hostname:$port" - } - - blockIds.foreach { blockId => - handler.addRequest(blockId, listener) - } - - val writeFuture = cf.channel().writeAndFlush(blockIds.mkString("\n") + "\n") - writeFuture.addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture): Unit = { - if (future.isSuccess) { - logTrace { - val timeTaken = (System.nanoTime - startTime).toDouble / 1000000 - s"Sending request $blockIds to $hostname:$port took $timeTaken ms" - } - } else { - // Fail all blocks. - val errorMsg = - s"Failed to send request $blockIds to $hostname:$port: ${future.cause.getMessage}" - logError(errorMsg, future.cause) - blockIds.foreach { blockId => - listener.onFetchFailure(blockId, errorMsg) - handler.removeRequest(blockId) - } - } - } - }) - } - - def waitForClose(): Unit = { - cf.channel().closeFuture().sync() - } - - def close(): Unit = cf.channel().close() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala deleted file mode 100644 index 2b28402c52b49..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientFactory.scala +++ /dev/null @@ -1,99 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import io.netty.channel.epoll.{EpollEventLoopGroup, EpollSocketChannel} -import io.netty.channel.nio.NioEventLoopGroup -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.nio.NioSocketChannel -import io.netty.channel.socket.oio.OioSocketChannel -import io.netty.channel.{EventLoopGroup, Channel} - -import org.apache.spark.SparkConf -import org.apache.spark.network.netty.NettyConfig -import org.apache.spark.util.Utils - -/** - * Factory for creating [[BlockFetchingClient]] by using createClient. This factory reuses - * the worker thread pool for Netty. - * - * Concurrency: createClient is safe to be called from multiple threads concurrently. - */ -private[spark] -class BlockFetchingClientFactory(val conf: NettyConfig) { - - def this(sparkConf: SparkConf) = this(new NettyConfig(sparkConf)) - - /** A thread factory so the threads are named (for debugging). */ - val threadFactory = Utils.namedThreadFactory("spark-shuffle-client") - - /** The following two are instantiated by the [[init]] method, depending ioMode. */ - var socketChannelClass: Class[_ <: Channel] = _ - var workerGroup: EventLoopGroup = _ - - init() - - /** Initialize [[socketChannelClass]] and [[workerGroup]] based on ioMode. */ - private def init(): Unit = { - def initOio(): Unit = { - socketChannelClass = classOf[OioSocketChannel] - workerGroup = new OioEventLoopGroup(0, threadFactory) - } - def initNio(): Unit = { - socketChannelClass = classOf[NioSocketChannel] - workerGroup = new NioEventLoopGroup(0, threadFactory) - } - def initEpoll(): Unit = { - socketChannelClass = classOf[EpollSocketChannel] - workerGroup = new EpollEventLoopGroup(0, threadFactory) - } - - conf.ioMode match { - case "nio" => initNio() - case "oio" => initOio() - case "epoll" => initEpoll() - case "auto" => - // For auto mode, first try epoll (only available on Linux), then nio. - try { - initEpoll() - } catch { - // TODO: Should we log the throwable? But that always happen on non-Linux systems. - // Perhaps the right thing to do is to check whether the system is Linux, and then only - // call initEpoll on Linux. - case e: Throwable => initNio() - } - } - } - - /** - * Create a new BlockFetchingClient connecting to the given remote host / port. - * - * This blocks until a connection is successfully established. - * - * Concurrency: This method is safe to call from multiple threads. - */ - def createClient(remoteHost: String, remotePort: Int): BlockFetchingClient = { - new BlockFetchingClient(this, remoteHost, remotePort) - } - - def stop(): Unit = { - if (workerGroup != null) { - workerGroup.shutdownGracefully() - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala deleted file mode 100644 index d9d3f7bef0b20..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandler.scala +++ /dev/null @@ -1,104 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import com.google.common.base.Charsets.UTF_8 -import io.netty.buffer.ByteBuf -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} - -import org.apache.spark.Logging - - -/** - * Handler that processes server responses. It uses the protocol documented in - * [[org.apache.spark.network.netty.server.BlockServer]]. - * - * Concurrency: thread safe and can be called from multiple threads. - */ -private[client] -class BlockFetchingClientHandler extends SimpleChannelInboundHandler[ByteBuf] with Logging { - - /** Tracks the list of outstanding requests and their listeners on success/failure. */ - private val outstandingRequests = java.util.Collections.synchronizedMap { - new java.util.HashMap[String, BlockClientListener] - } - - def addRequest(blockId: String, listener: BlockClientListener): Unit = { - outstandingRequests.put(blockId, listener) - } - - def removeRequest(blockId: String): Unit = { - outstandingRequests.remove(blockId) - } - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - val errorMsg = s"Exception in connection from ${ctx.channel.remoteAddress}: ${cause.getMessage}" - logError(errorMsg, cause) - - // Fire the failure callback for all outstanding blocks - outstandingRequests.synchronized { - val iter = outstandingRequests.entrySet().iterator() - while (iter.hasNext) { - val entry = iter.next() - entry.getValue.onFetchFailure(entry.getKey, errorMsg) - } - outstandingRequests.clear() - } - - ctx.close() - } - - override def channelRead0(ctx: ChannelHandlerContext, in: ByteBuf) { - val totalLen = in.readInt() - val blockIdLen = in.readInt() - val blockIdBytes = new Array[Byte](math.abs(blockIdLen)) - in.readBytes(blockIdBytes) - val blockId = new String(blockIdBytes, UTF_8) - val blockSize = totalLen - math.abs(blockIdLen) - 4 - - def server = ctx.channel.remoteAddress.toString - - // blockIdLen is negative when it is an error message. - if (blockIdLen < 0) { - val errorMessageBytes = new Array[Byte](blockSize) - in.readBytes(errorMessageBytes) - val errorMsg = new String(errorMessageBytes, UTF_8) - logTrace(s"Received block $blockId ($blockSize B) with error $errorMsg from $server") - - val listener = outstandingRequests.get(blockId) - if (listener == null) { - // Ignore callback - logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") - } else { - outstandingRequests.remove(blockId) - listener.onFetchFailure(blockId, errorMsg) - } - } else { - logTrace(s"Received block $blockId ($blockSize B) from $server") - - val listener = outstandingRequests.get(blockId) - if (listener == null) { - // Ignore callback - logWarning(s"Got a response for block $blockId but it is not in our outstanding requests") - } else { - outstandingRequests.remove(blockId) - listener.onFetchSuccess(blockId, new ReferenceCountedBuffer(in)) - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala b/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala deleted file mode 100644 index 9740ee64d1f2d..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/LazyInitIterator.scala +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -/** - * A simple iterator that lazily initializes the underlying iterator. - * - * The use case is that sometimes we might have many iterators open at the same time, and each of - * the iterator might initialize its own buffer (e.g. decompression buffer, deserialization buffer). - * This could lead to too many buffers open. If this iterator is used, we lazily initialize those - * buffers. - */ -private[spark] -class LazyInitIterator(createIterator: => Iterator[Any]) extends Iterator[Any] { - - lazy val proxy = createIterator - - override def hasNext: Boolean = { - val gotNext = proxy.hasNext - if (!gotNext) { - close() - } - gotNext - } - - override def next(): Any = proxy.next() - - def close(): Unit = Unit -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala b/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala deleted file mode 100644 index ea1abf5eccc26..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/client/ReferenceCountedBuffer.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import java.io.InputStream -import java.nio.ByteBuffer - -import io.netty.buffer.{ByteBuf, ByteBufInputStream} - - -/** - * A buffer abstraction based on Netty's ByteBuf so we don't expose Netty. - * This is a Scala value class. - * - * The buffer's life cycle is NOT managed by the JVM, and thus requiring explicit declaration of - * reference by the retain method and release method. - */ -private[spark] -class ReferenceCountedBuffer(val underlying: ByteBuf) extends AnyVal { - - /** Return the nio ByteBuffer view of the underlying buffer. */ - def byteBuffer(): ByteBuffer = underlying.nioBuffer - - /** Creates a new input stream that starts from the current position of the buffer. */ - def inputStream(): InputStream = new ByteBufInputStream(underlying) - - /** Increment the reference counter by one. */ - def retain(): Unit = underlying.retain() - - /** Decrement the reference counter by one and release the buffer if the ref count is 0. */ - def release(): Unit = underlying.release() -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala deleted file mode 100644 index 8e4dda4ef8595..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeaderEncoder.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import io.netty.buffer.ByteBuf -import io.netty.channel.ChannelHandlerContext -import io.netty.handler.codec.MessageToByteEncoder - -/** - * A simple encoder for BlockHeader. See [[BlockServer]] for the server to client protocol. - */ -private[server] -class BlockHeaderEncoder extends MessageToByteEncoder[BlockHeader] { - override def encode(ctx: ChannelHandlerContext, msg: BlockHeader, out: ByteBuf): Unit = { - // message = message length (4 bytes) + block id length (4 bytes) + block id + block data - // message length = block id length (4 bytes) + size of block id + size of block data - val blockIdBytes = msg.blockId.getBytes - msg.error match { - case Some(errorMsg) => - val errorBytes = errorMsg.getBytes - out.writeInt(4 + blockIdBytes.length + errorBytes.size) - out.writeInt(-blockIdBytes.length) // use negative block id length to represent errors - out.writeBytes(blockIdBytes) // next is blockId itself - out.writeBytes(errorBytes) // error message - case None => - out.writeInt(4 + blockIdBytes.length + msg.blockSize) - out.writeInt(blockIdBytes.length) // First 4 bytes is blockId length - out.writeBytes(blockIdBytes) // next is blockId itself - // msg of size blockSize will be written by ServerHandler - } - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala deleted file mode 100644 index 9194c7ced3547..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServer.scala +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import java.net.InetSocketAddress - -import com.google.common.base.Charsets.UTF_8 -import io.netty.bootstrap.ServerBootstrap -import io.netty.buffer.PooledByteBufAllocator -import io.netty.channel.{ChannelFuture, ChannelInitializer, ChannelOption} -import io.netty.channel.epoll.{EpollEventLoopGroup, EpollServerSocketChannel} -import io.netty.channel.nio.NioEventLoopGroup -import io.netty.channel.oio.OioEventLoopGroup -import io.netty.channel.socket.SocketChannel -import io.netty.channel.socket.nio.NioServerSocketChannel -import io.netty.channel.socket.oio.OioServerSocketChannel -import io.netty.handler.codec.LineBasedFrameDecoder -import io.netty.handler.codec.string.StringDecoder - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.network.netty.NettyConfig -import org.apache.spark.storage.BlockDataProvider -import org.apache.spark.util.Utils - - -/** - * Server for serving Spark data blocks. - * This should be used together with [[org.apache.spark.network.netty.client.BlockFetchingClient]]. - * - * Protocol for requesting blocks (client to server): - * One block id per line, e.g. to request 3 blocks: "block1\nblock2\nblock3\n" - * - * Protocol for sending blocks (server to client): - * frame-length (4 bytes), block-id-length (4 bytes), block-id, block-data. - * - * frame-length should not include the length of itself. - * If block-id-length is negative, then this is an error message rather than block-data. The real - * length is the absolute value of the frame-length. - * - */ -private[spark] -class BlockServer(conf: NettyConfig, dataProvider: BlockDataProvider) extends Logging { - - def this(sparkConf: SparkConf, dataProvider: BlockDataProvider) = { - this(new NettyConfig(sparkConf), dataProvider) - } - - def port: Int = _port - - def hostName: String = _hostName - - private var _port: Int = conf.serverPort - private var _hostName: String = "" - private var bootstrap: ServerBootstrap = _ - private var channelFuture: ChannelFuture = _ - - init() - - /** Initialize the server. */ - private def init(): Unit = { - bootstrap = new ServerBootstrap - val bossThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-boss") - val workerThreadFactory = Utils.namedThreadFactory("spark-shuffle-server-worker") - - // Use only one thread to accept connections, and 2 * num_cores for worker. - def initNio(): Unit = { - val bossGroup = new NioEventLoopGroup(1, bossThreadFactory) - val workerGroup = new NioEventLoopGroup(0, workerThreadFactory) - workerGroup.setIoRatio(conf.ioRatio) - bootstrap.group(bossGroup, workerGroup).channel(classOf[NioServerSocketChannel]) - } - def initOio(): Unit = { - val bossGroup = new OioEventLoopGroup(1, bossThreadFactory) - val workerGroup = new OioEventLoopGroup(0, workerThreadFactory) - bootstrap.group(bossGroup, workerGroup).channel(classOf[OioServerSocketChannel]) - } - def initEpoll(): Unit = { - val bossGroup = new EpollEventLoopGroup(1, bossThreadFactory) - val workerGroup = new EpollEventLoopGroup(0, workerThreadFactory) - workerGroup.setIoRatio(conf.ioRatio) - bootstrap.group(bossGroup, workerGroup).channel(classOf[EpollServerSocketChannel]) - } - - conf.ioMode match { - case "nio" => initNio() - case "oio" => initOio() - case "epoll" => initEpoll() - case "auto" => - // For auto mode, first try epoll (only available on Linux), then nio. - try { - initEpoll() - } catch { - // TODO: Should we log the throwable? But that always happen on non-Linux systems. - // Perhaps the right thing to do is to check whether the system is Linux, and then only - // call initEpoll on Linux. - case e: Throwable => initNio() - } - } - - // Use pooled buffers to reduce temporary buffer allocation - bootstrap.option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - bootstrap.childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) - - // Various (advanced) user-configured settings. - conf.backLog.foreach { backLog => - bootstrap.option[java.lang.Integer](ChannelOption.SO_BACKLOG, backLog) - } - conf.receiveBuf.foreach { receiveBuf => - bootstrap.option[java.lang.Integer](ChannelOption.SO_RCVBUF, receiveBuf) - } - conf.sendBuf.foreach { sendBuf => - bootstrap.option[java.lang.Integer](ChannelOption.SO_SNDBUF, sendBuf) - } - - bootstrap.childHandler(new ChannelInitializer[SocketChannel] { - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(UTF_8)) - .addLast("blockHeaderEncoder", new BlockHeaderEncoder) - .addLast("handler", new BlockServerHandler(dataProvider)) - } - }) - - channelFuture = bootstrap.bind(new InetSocketAddress(_port)) - channelFuture.sync() - - val addr = channelFuture.channel.localAddress.asInstanceOf[InetSocketAddress] - _port = addr.getPort - _hostName = addr.getHostName - } - - /** Shutdown the server. */ - def stop(): Unit = { - if (channelFuture != null) { - channelFuture.channel().close().awaitUninterruptibly() - channelFuture = null - } - if (bootstrap != null && bootstrap.group() != null) { - bootstrap.group().shutdownGracefully() - } - if (bootstrap != null && bootstrap.childGroup() != null) { - bootstrap.childGroup().shutdownGracefully() - } - bootstrap = null - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala deleted file mode 100644 index 188154d51d0eb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerChannelInitializer.scala +++ /dev/null @@ -1,40 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import com.google.common.base.Charsets.UTF_8 -import io.netty.channel.ChannelInitializer -import io.netty.channel.socket.SocketChannel -import io.netty.handler.codec.LineBasedFrameDecoder -import io.netty.handler.codec.string.StringDecoder - -import org.apache.spark.storage.BlockDataProvider - -/** Channel initializer that sets up the pipeline for the BlockServer. */ -private[netty] -class BlockServerChannelInitializer(dataProvider: BlockDataProvider) - extends ChannelInitializer[SocketChannel] { - - override def initChannel(ch: SocketChannel): Unit = { - ch.pipeline - .addLast("frameDecoder", new LineBasedFrameDecoder(1024)) // max block id length 1024 - .addLast("stringDecoder", new StringDecoder(UTF_8)) - .addLast("blockHeaderEncoder", new BlockHeaderEncoder) - .addLast("handler", new BlockServerHandler(dataProvider)) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala b/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala deleted file mode 100644 index 40dd5e5d1a2ac..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockServerHandler.scala +++ /dev/null @@ -1,140 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import java.io.FileInputStream -import java.nio.ByteBuffer -import java.nio.channels.FileChannel - -import io.netty.buffer.Unpooled -import io.netty.channel._ - -import org.apache.spark.Logging -import org.apache.spark.storage.{FileSegment, BlockDataProvider} - - -/** - * A handler that processes requests from clients and writes block data back. - * - * The messages should have been processed by a LineBasedFrameDecoder and a StringDecoder first - * so channelRead0 is called once per line (i.e. per block id). - */ -private[server] -class BlockServerHandler(dataProvider: BlockDataProvider) - extends SimpleChannelInboundHandler[String] with Logging { - - override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { - logError(s"Exception in connection from ${ctx.channel.remoteAddress}", cause) - ctx.close() - } - - override def channelRead0(ctx: ChannelHandlerContext, blockId: String): Unit = { - def client = ctx.channel.remoteAddress.toString - - // A helper function to send error message back to the client. - def respondWithError(error: String): Unit = { - ctx.writeAndFlush(new BlockHeader(-1, blockId, Some(error))).addListener( - new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (!future.isSuccess) { - // TODO: Maybe log the success case as well. - logError(s"Error sending error back to $client", future.cause) - ctx.close() - } - } - } - ) - } - - def writeFileSegment(segment: FileSegment): Unit = { - // Send error message back if the block is too large. Even though we are capable of sending - // large (2G+) blocks, the receiving end cannot handle it so let's fail fast. - // Once we fixed the receiving end to be able to process large blocks, this should be removed. - // Also make sure we update BlockHeaderEncoder to support length > 2G. - - // See [[BlockHeaderEncoder]] for the way length is encoded. - if (segment.length + blockId.length + 4 > Int.MaxValue) { - respondWithError(s"Block $blockId size ($segment.length) greater than 2G") - return - } - - var fileChannel: FileChannel = null - try { - fileChannel = new FileInputStream(segment.file).getChannel - } catch { - case e: Exception => - logError( - s"Error opening channel for $blockId in ${segment.file} for request from $client", e) - respondWithError(e.getMessage) - } - - // Found the block. Send it back. - if (fileChannel != null) { - // Write the header and block data. In the case of failures, the listener on the block data - // write should close the connection. - ctx.write(new BlockHeader(segment.length.toInt, blockId)) - - val region = new DefaultFileRegion(fileChannel, segment.offset, segment.length) - ctx.writeAndFlush(region).addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (future.isSuccess) { - logTrace(s"Sent block $blockId (${segment.length} B) back to $client") - } else { - logError(s"Error sending block $blockId to $client; closing connection", future.cause) - ctx.close() - } - } - }) - } - } - - def writeByteBuffer(buf: ByteBuffer): Unit = { - ctx.write(new BlockHeader(buf.remaining, blockId)) - ctx.writeAndFlush(Unpooled.wrappedBuffer(buf)).addListener(new ChannelFutureListener { - override def operationComplete(future: ChannelFuture) { - if (future.isSuccess) { - logTrace(s"Sent block $blockId (${buf.remaining} B) back to $client") - } else { - logError(s"Error sending block $blockId to $client; closing connection", future.cause) - ctx.close() - } - } - }) - } - - logTrace(s"Received request from $client to fetch block $blockId") - - var blockData: Either[FileSegment, ByteBuffer] = null - - // First make sure we can find the block. If not, send error back to the user. - try { - blockData = dataProvider.getBlockData(blockId) - } catch { - case e: Exception => - logError(s"Error opening block $blockId for request from $client", e) - respondWithError(e.getMessage) - return - } - - blockData match { - case Left(segment) => writeFileSegment(segment) - case Right(buf) => writeByteBuffer(buf) - } - - } // end of channelRead0 -} diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index e3113205bebdc..11793ea92adb1 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -19,12 +19,13 @@ package org.apache.spark.network.nio import java.nio.ByteBuffer -import scala.concurrent.Future - -import org.apache.spark.{SparkException, Logging, SecurityManager, SparkConf} import org.apache.spark.network._ +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} + +import scala.concurrent.Future /** @@ -71,7 +72,7 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa /** * Tear down the transfer service. */ - override def stop(): Unit = { + override def close(): Unit = { if (cm != null) { cm.stop() } @@ -95,27 +96,34 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa future.onSuccess { case message => val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) + // SPARK-4064: In some cases(eg. Remote block was removed) blockMessageArray may be empty. if (blockMessageArray.isEmpty) { - listener.onBlockFetchFailure( - new SparkException(s"Received empty message from $cmId")) + blockIds.foreach { id => + listener.onBlockFetchFailure(id, new SparkException(s"Received empty message from $cmId")) + } } else { - for (blockMessage <- blockMessageArray) { + for (blockMessage: BlockMessage <- blockMessageArray) { val msgType = blockMessage.getType if (msgType != BlockMessage.TYPE_GOT_BLOCK) { - listener.onBlockFetchFailure( - new SparkException(s"Unexpected message ${msgType} received from $cmId")) + if (blockMessage.getId != null) { + listener.onBlockFetchFailure(blockMessage.getId.toString, + new SparkException(s"Unexpected message $msgType received from $cmId")) + } } else { val blockId = blockMessage.getId + val networkSize = blockMessage.getData.limit() listener.onBlockFetchSuccess( - blockId.toString, new NioByteBufferManagedBuffer(blockMessage.getData)) + blockId.toString, new NioManagedBuffer(blockMessage.getData)) } } } }(cm.futureExecContext) future.onFailure { case exception => - listener.onBlockFetchFailure(exception) + blockIds.foreach { blockId => + listener.onBlockFetchFailure(blockId, exception) + } }(cm.futureExecContext) } @@ -127,12 +135,12 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa override def uploadBlock( hostname: String, port: Int, - blockId: String, + blockId: BlockId, blockData: ManagedBuffer, level: StorageLevel) : Future[Unit] = { checkInit() - val msg = PutBlock(BlockId(blockId), blockData.nioByteBuffer(), level) + val msg = PutBlock(blockId, blockData.nioByteBuffer(), level) val blockMessageArray = new BlockMessageArray(BlockMessage.fromPutBlock(msg)) val remoteCmId = new ConnectionManagerId(hostName, port) val reply = cm.sendMessageReliably(remoteCmId, blockMessageArray.toBufferMessage) @@ -154,10 +162,9 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa val responseMessages = blockMessages.map(processBlockMessage).filter(_ != None).map(_.get) Some(new BlockMessageArray(responseMessages).toBufferMessage) } catch { - case e: Exception => { + case e: Exception => logError("Exception handling buffer message", e) Some(Message.createErrorMessage(e, msg.id)) - } } case otherMessage: Any => @@ -172,13 +179,13 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa case BlockMessage.TYPE_PUT_BLOCK => val msg = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) logDebug("Received [" + msg + "]") - putBlock(msg.id.toString, msg.data, msg.level) + putBlock(msg.id, msg.data, msg.level) None case BlockMessage.TYPE_GET_BLOCK => val msg = new GetBlock(blockMessage.getId) logDebug("Received [" + msg + "]") - val buffer = getBlock(msg.id.toString) + val buffer = getBlock(msg.id) if (buffer == null) { return None } @@ -188,20 +195,20 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa } } - private def putBlock(blockId: String, bytes: ByteBuffer, level: StorageLevel) { + private def putBlock(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel) { val startTimeMs = System.currentTimeMillis() logDebug("PutBlock " + blockId + " started from " + startTimeMs + " with data: " + bytes) - blockDataManager.putBlockData(blockId, new NioByteBufferManagedBuffer(bytes), level) + blockDataManager.putBlockData(blockId, new NioManagedBuffer(bytes), level) logDebug("PutBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + " with data size: " + bytes.limit) } - private def getBlock(blockId: String): ByteBuffer = { + private def getBlock(blockId: BlockId): ByteBuffer = { val startTimeMs = System.currentTimeMillis() logDebug("GetBlock " + blockId + " started from " + startTimeMs) - val buffer = blockDataManager.getBlockData(blockId).orNull + val buffer = blockDataManager.getBlockData(blockId) logDebug("GetBlock " + blockId + " used " + Utils.getUsedTimeMs(startTimeMs) + " and got buffer " + buffer) - if (buffer == null) null else buffer.nioByteBuffer() + buffer.nioByteBuffer() } } diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index a9144cdd97b8c..ca6e971d227fb 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -17,14 +17,14 @@ package org.apache.spark.serializer -import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} +import java.io._ import java.nio.ByteBuffer import scala.reflect.ClassTag -import org.apache.spark.SparkEnv +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} +import org.apache.spark.util.{Utils, ByteBufferInputStream, NextIterator} /** * :: DeveloperApi :: diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 439981d232349..1fb5b2c4546bd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -24,9 +24,9 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.JavaConversions._ -import org.apache.spark.{SparkEnv, SparkConf, Logging} +import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup import org.apache.spark.storage._ diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index b5cd34cacd520..e9805c9c134b5 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -23,7 +23,7 @@ import java.nio.ByteBuffer import com.google.common.io.ByteStreams import org.apache.spark.SparkEnv -import org.apache.spark.network.{ManagedBuffer, FileSegmentManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.storage._ /** diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala index 63863cc0250a3..b521f0c7fc77e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockManager.scala @@ -18,8 +18,7 @@ package org.apache.spark.shuffle import java.nio.ByteBuffer - -import org.apache.spark.network.ManagedBuffer +import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.storage.ShuffleBlockId private[spark] diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 4cc97923658bc..58510d7232436 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -17,15 +17,13 @@ package org.apache.spark.storage -import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} +import java.io.{BufferedOutputStream, ByteArrayOutputStream, File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} -import scala.concurrent.ExecutionContext.Implicits.global - -import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.concurrent.{Await, Future} +import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ +import scala.concurrent.{Await, Future} import scala.util.Random import akka.actor.{ActorSystem, Props} @@ -35,11 +33,11 @@ import org.apache.spark._ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager import org.apache.spark.util._ - private[spark] sealed trait BlockValues private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends BlockValues private[spark] case class IteratorValues(iterator: Iterator[Any]) extends BlockValues @@ -212,21 +210,20 @@ private[spark] class BlockManager( } /** - * Interface to get local block data. - * - * @return Some(buffer) if the block exists locally, and None if it doesn't. + * Interface to get local block data. Throws an exception if the block cannot be found or + * cannot be read successfully. */ - override def getBlockData(blockId: String): Option[ManagedBuffer] = { - val bid = BlockId(blockId) - if (bid.isShuffle) { - Some(shuffleManager.shuffleBlockManager.getBlockData(bid.asInstanceOf[ShuffleBlockId])) + override def getBlockData(blockId: BlockId): ManagedBuffer = { + if (blockId.isShuffle) { + shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) } else { - val blockBytesOpt = doGetLocal(bid, asBlockResult = false).asInstanceOf[Option[ByteBuffer]] + val blockBytesOpt = doGetLocal(blockId, asBlockResult = false) + .asInstanceOf[Option[ByteBuffer]] if (blockBytesOpt.isDefined) { val buffer = blockBytesOpt.get - Some(new NioByteBufferManagedBuffer(buffer)) + new NioManagedBuffer(buffer) } else { - None + throw new BlockNotFoundException(blockId.toString) } } } @@ -234,8 +231,8 @@ private[spark] class BlockManager( /** * Put the block locally, using the given storage level. */ - override def putBlockData(blockId: String, data: ManagedBuffer, level: StorageLevel): Unit = { - putBytes(BlockId(blockId), data.nioByteBuffer(), level) + override def putBlockData(blockId: BlockId, data: ManagedBuffer, level: StorageLevel): Unit = { + putBytes(blockId, data.nioByteBuffer(), level) } /** @@ -340,17 +337,6 @@ private[spark] class BlockManager( locations } - /** - * A short-circuited method to get blocks directly from disk. This is used for getting - * shuffle blocks. It is safe to do so without a lock on block info since disk store - * never deletes (recent) items. - */ - def getLocalShuffleFromDisk(blockId: BlockId, serializer: Serializer): Option[Iterator[Any]] = { - val buf = shuffleManager.shuffleBlockManager.getBlockData(blockId.asInstanceOf[ShuffleBlockId]) - val is = wrapForCompression(blockId, buf.inputStream()) - Some(serializer.newInstance().deserializeStream(is).asIterator) - } - /** * Get block from local block manager. */ @@ -869,9 +855,9 @@ private[spark] class BlockManager( data.rewind() logTrace(s"Trying to replicate $blockId of ${data.limit()} bytes to $peer") blockTransferService.uploadBlockSync( - peer.host, peer.port, blockId.toString, new NioByteBufferManagedBuffer(data), tLevel) - logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %d ms" - .format((System.currentTimeMillis - onePeerStartTime))) + peer.host, peer.port, blockId, new NioManagedBuffer(data), tLevel) + logTrace(s"Replicated $blockId of ${data.limit()} bytes to $peer in %s ms" + .format(System.currentTimeMillis - onePeerStartTime)) peersReplicatedTo += peer peersForReplication -= peer replicationFailed = false @@ -1126,7 +1112,7 @@ private[spark] class BlockManager( } def stop(): Unit = { - blockTransferService.stop() + blockTransferService.close() diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala index 9ef453605f4f1..81f5f2d31dbd8 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockNotFoundException.scala @@ -17,5 +17,4 @@ package org.apache.spark.storage - class BlockNotFoundException(blockId: String) extends Exception(s"Block $blockId not found") diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 71b276b5f18e4..0d6f3bf003a9d 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -19,15 +19,13 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashSet -import scala.collection.mutable.Queue +import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} -import org.apache.spark.{TaskContext, Logging} -import org.apache.spark.network.{ManagedBuffer, BlockFetchingListener, BlockTransferService} +import org.apache.spark.{Logging, TaskContext} +import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} +import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.serializer.Serializer -import org.apache.spark.util.Utils - +import org.apache.spark.util.{CompletionIterator, Utils} /** * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block @@ -88,17 +86,51 @@ final class ShuffleBlockFetcherIterator( */ private[this] val results = new LinkedBlockingQueue[FetchResult] - // Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that - // the number of bytes in flight is limited to maxBytesInFlight + /** + * Current [[FetchResult]] being processed. We track this so we can release the current buffer + * in case of a runtime exception when processing the current buffer. + */ + private[this] var currentResult: FetchResult = null + + /** + * Queue of fetch requests to issue; we'll pull requests off this gradually to make sure that + * the number of bytes in flight is limited to maxBytesInFlight. + */ private[this] val fetchRequests = new Queue[FetchRequest] - // Current bytes in flight from our requests + /** Current bytes in flight from our requests */ private[this] var bytesInFlight = 0L private[this] val shuffleMetrics = context.taskMetrics.createShuffleReadMetricsForDependency() + /** + * Whether the iterator is still active. If isZombie is true, the callback interface will no + * longer place fetched blocks into [[results]]. + */ + @volatile private[this] var isZombie = false + initialize() + /** + * Mark the iterator as zombie, and release all buffers that haven't been deserialized yet. + */ + private[this] def cleanup() { + isZombie = true + // Release the current buffer if necessary + if (currentResult != null && !currentResult.failed) { + currentResult.buf.release() + } + + // Release buffers in the results queue + val iter = results.iterator() + while (iter.hasNext) { + val result = iter.next() + if (!result.failed) { + result.buf.release() + } + } + } + private[this] def sendRequest(req: FetchRequest) { logDebug("Sending request for %d blocks (%s) from %s".format( req.blocks.size, Utils.bytesToString(req.size), req.address.hostPort)) @@ -110,24 +142,23 @@ final class ShuffleBlockFetcherIterator( blockTransferService.fetchBlocks(req.address.host, req.address.port, blockIds, new BlockFetchingListener { - override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { - results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), - () => serializer.newInstance().deserializeStream( - blockManager.wrapForCompression(BlockId(blockId), data.inputStream())).asIterator - )) - shuffleMetrics.remoteBytesRead += data.size - shuffleMetrics.remoteBlocksFetched += 1 - logDebug("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = { + // Only add the buffer to results queue if the iterator is not zombie, + // i.e. cleanup() has not been called yet. + if (!isZombie) { + // Increment the ref count because we need to pass this to a different thread. + // This needs to be released after use. + buf.retain() + results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), buf)) + shuffleMetrics.remoteBytesRead += buf.size + shuffleMetrics.remoteBlocksFetched += 1 + } + logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) } - override def onBlockFetchFailure(e: Throwable): Unit = { + override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) - // Note that there is a chance that some blocks have been fetched successfully, but we - // still add them to the failed queue. This is fine because when the caller see a - // FetchFailedException, it is going to fail the entire task anyway. - for ((blockId, size) <- req.blocks) { - results.put(new FetchResult(blockId, -1, null)) - } + results.put(new FetchResult(BlockId(blockId), -1, null)) } } ) @@ -138,7 +169,7 @@ final class ShuffleBlockFetcherIterator( // smaller than maxBytesInFlight is to allow multiple, parallel fetches from up to 5 // nodes, rather than blocking on reading output from one node. val targetRequestSize = math.max(maxBytesInFlight / 5, 1L) - logInfo("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) + logDebug("maxBytesInFlight: " + maxBytesInFlight + ", targetRequestSize: " + targetRequestSize) // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. @@ -185,26 +216,34 @@ final class ShuffleBlockFetcherIterator( remoteRequests } + /** + * Fetch the local blocks while we are fetching remote blocks. This is ok because + * [[ManagedBuffer]]'s memory is allocated lazily when we create the input stream, so all we + * track in-memory are the ManagedBuffer references themselves. + */ private[this] def fetchLocalBlocks() { - // Get the local blocks while remote blocks are being fetched. Note that it's okay to do - // these all at once because they will just memory-map some files, so they won't consume - // any memory that might exceed our maxBytesInFlight - for (id <- localBlocks) { + val iter = localBlocks.iterator + while (iter.hasNext) { + val blockId = iter.next() try { + val buf = blockManager.getBlockData(blockId) shuffleMetrics.localBlocksFetched += 1 - results.put(new FetchResult( - id, 0, () => blockManager.getLocalShuffleFromDisk(id, serializer).get)) - logDebug("Got local block " + id) + buf.retain() + results.put(new FetchResult(blockId, 0, buf)) } catch { case e: Exception => + // If we see an exception, stop immediately. logError(s"Error occurred while fetching local blocks", e) - results.put(new FetchResult(id, -1, null)) + results.put(new FetchResult(blockId, -1, null)) return } } } private[this] def initialize(): Unit = { + // Add a task completion callback (called in both success case and failure case) to cleanup. + context.addTaskCompletionListener(_ => cleanup()) + // Split local and remote blocks. val remoteRequests = splitLocalRemoteBlocks() // Add the remote requests into our queue in a random order @@ -229,7 +268,8 @@ final class ShuffleBlockFetcherIterator( override def next(): (BlockId, Option[Iterator[Any]]) = { numBlocksProcessed += 1 val startFetchWait = System.currentTimeMillis() - val result = results.take() + currentResult = results.take() + val result = currentResult val stopFetchWait = System.currentTimeMillis() shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) if (!result.failed) { @@ -240,7 +280,21 @@ final class ShuffleBlockFetcherIterator( (bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { sendRequest(fetchRequests.dequeue()) } - (result.blockId, if (result.failed) None else Some(result.deserialize())) + + val iteratorOpt: Option[Iterator[Any]] = if (result.failed) { + None + } else { + val is = blockManager.wrapForCompression(result.blockId, result.buf.createInputStream()) + val iter = serializer.newInstance().deserializeStream(is).asIterator + Some(CompletionIterator[Any, Iterator[Any]](iter, { + // Once the iterator is exhausted, release the buffer and set currentResult to null + // so we don't release it again in cleanup. + currentResult = null + result.buf.release() + })) + } + + (result.blockId, iteratorOpt) } } @@ -254,7 +308,7 @@ object ShuffleBlockFetcherIterator { * @param blocks Sequence of tuple, where the first element is the block id, * and the second element is the estimated size, used to calculate bytesInFlight. */ - class FetchRequest(val address: BlockManagerId, val blocks: Seq[(BlockId, Long)]) { + case class FetchRequest(address: BlockManagerId, blocks: Seq[(BlockId, Long)]) { val size = blocks.map(_._2).sum } @@ -262,10 +316,11 @@ object ShuffleBlockFetcherIterator { * Result of a fetch from a remote block. A failure is represented as size == -1. * @param blockId block id * @param size estimated size of the block, used to calculate bytesInFlight. - * Note that this is NOT the exact bytes. - * @param deserialize closure to return the result in the form of an Iterator. + * Note that this is NOT the exact bytes. -1 if failure is present. + * @param buf [[ManagedBuffer]] for the content. null is error. */ - class FetchResult(val blockId: BlockId, val size: Long, val deserialize: () => Iterator[Any]) { + case class FetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) { def failed: Boolean = size == -1 + if (failed) assert(buf == null) else assert(buf != null) } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 1e881da5114d3..0daab91143e47 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -43,7 +43,6 @@ import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark._ -import org.apache.spark.util.SparkUncaughtExceptionHandler import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index d7b2d2e1e330f..840d8273cb6a8 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -24,10 +24,10 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. override def beforeAll() { - System.setProperty("spark.shuffle.use.netty", "true") + System.setProperty("spark.shuffle.blockTransferService", "netty") } override def afterAll() { - System.clearProperty("spark.shuffle.use.netty") + System.clearProperty("spark.shuffle.blockTransferService") } } diff --git a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala deleted file mode 100644 index 02d0ffc86f58f..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/ServerClientIntegrationSuite.scala +++ /dev/null @@ -1,161 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.io.{RandomAccessFile, File} -import java.nio.ByteBuffer -import java.util.{Collections, HashSet} -import java.util.concurrent.{TimeUnit, Semaphore} - -import scala.collection.JavaConversions._ - -import io.netty.buffer.{ByteBufUtil, Unpooled} - -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.SparkConf -import org.apache.spark.network.netty.client.{BlockClientListener, ReferenceCountedBuffer, BlockFetchingClientFactory} -import org.apache.spark.network.netty.server.BlockServer -import org.apache.spark.storage.{FileSegment, BlockDataProvider} - - -/** - * Test suite that makes sure the server and the client implementations share the same protocol. - */ -class ServerClientIntegrationSuite extends FunSuite with BeforeAndAfterAll { - - val bufSize = 100000 - var buf: ByteBuffer = _ - var testFile: File = _ - var server: BlockServer = _ - var clientFactory: BlockFetchingClientFactory = _ - - val bufferBlockId = "buffer_block" - val fileBlockId = "file_block" - - val fileContent = new Array[Byte](1024) - scala.util.Random.nextBytes(fileContent) - - override def beforeAll() = { - buf = ByteBuffer.allocate(bufSize) - for (i <- 1 to bufSize) { - buf.put(i.toByte) - } - buf.flip() - - testFile = File.createTempFile("netty-test-file", "txt") - val fp = new RandomAccessFile(testFile, "rw") - fp.write(fileContent) - fp.close() - - server = new BlockServer(new SparkConf, new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { - if (blockId == bufferBlockId) { - Right(buf) - } else if (blockId == fileBlockId) { - Left(new FileSegment(testFile, 10, testFile.length - 25)) - } else { - throw new Exception("Unknown block id " + blockId) - } - } - }) - - clientFactory = new BlockFetchingClientFactory(new SparkConf) - } - - override def afterAll() = { - server.stop() - clientFactory.stop() - } - - /** A ByteBuf for buffer_block */ - lazy val byteBufferBlockReference = Unpooled.wrappedBuffer(buf) - - /** A ByteBuf for file_block */ - lazy val fileBlockReference = Unpooled.wrappedBuffer(fileContent, 10, fileContent.length - 25) - - def fetchBlocks(blockIds: Seq[String]): (Set[String], Set[ReferenceCountedBuffer], Set[String]) = - { - val client = clientFactory.createClient(server.hostName, server.port) - val sem = new Semaphore(0) - val receivedBlockIds = Collections.synchronizedSet(new HashSet[String]) - val errorBlockIds = Collections.synchronizedSet(new HashSet[String]) - val receivedBuffers = Collections.synchronizedSet(new HashSet[ReferenceCountedBuffer]) - - client.fetchBlocks( - blockIds, - new BlockClientListener { - override def onFetchFailure(blockId: String, errorMsg: String): Unit = { - errorBlockIds.add(blockId) - sem.release() - } - - override def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit = { - receivedBlockIds.add(blockId) - data.retain() - receivedBuffers.add(data) - sem.release() - } - } - ) - if (!sem.tryAcquire(blockIds.size, 30, TimeUnit.SECONDS)) { - fail("Timeout getting response from the server") - } - client.close() - (receivedBlockIds.toSet, receivedBuffers.toSet, errorBlockIds.toSet) - } - - test("fetch a ByteBuffer block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId)) - assert(blockIds === Set(bufferBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch a FileSegment block via zero-copy send") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(fileBlockId)) - assert(blockIds === Set(fileBlockId)) - assert(buffers.map(_.underlying) === Set(fileBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch a non-existent block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq("random-block")) - assert(blockIds.isEmpty) - assert(buffers.isEmpty) - assert(failBlockIds === Set("random-block")) - } - - test("fetch both ByteBuffer block and FileSegment block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, fileBlockId)) - assert(blockIds === Set(bufferBlockId, fileBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference, fileBlockReference)) - assert(failBlockIds.isEmpty) - buffers.foreach(_.release()) - } - - test("fetch both ByteBuffer block and a non-existent block") { - val (blockIds, buffers, failBlockIds) = fetchBlocks(Seq(bufferBlockId, "random-block")) - assert(blockIds === Set(bufferBlockId)) - assert(buffers.map(_.underlying) === Set(byteBufferBlockReference)) - assert(failBlockIds === Set("random-block")) - buffers.foreach(_.release()) - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala deleted file mode 100644 index f629322ff6838..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/client/BlockFetchingClientHandlerSuite.scala +++ /dev/null @@ -1,106 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.client - -import java.nio.ByteBuffer - -import com.google.common.base.Charsets.UTF_8 -import io.netty.buffer.Unpooled -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.{PrivateMethodTester, FunSuite} - - -class BlockFetchingClientHandlerSuite extends FunSuite with PrivateMethodTester { - - test("handling block data (successful fetch)") { - val blockId = "test_block" - val blockData = "blahblahblahblahblah" - val totalLength = 4 + blockId.length + blockData.length - - var parsedBlockId: String = "" - var parsedBlockData: String = "" - val handler = new BlockFetchingClientHandler - handler.addRequest(blockId, - new BlockClientListener { - override def onFetchFailure(blockId: String, errorMsg: String): Unit = ??? - override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer): Unit = { - parsedBlockId = bid - val bytes = new Array[Byte](refCntBuf.byteBuffer().remaining) - refCntBuf.byteBuffer().get(bytes) - parsedBlockData = new String(bytes, UTF_8) - } - } - ) - - val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) - assert(handler.invokePrivate(outstandingRequests()).size === 1) - - val channel = new EmbeddedChannel(handler) - val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself - buf.putInt(totalLength) - buf.putInt(blockId.length) - buf.put(blockId.getBytes) - buf.put(blockData.getBytes) - buf.flip() - - channel.writeInbound(Unpooled.wrappedBuffer(buf)) - assert(parsedBlockId === blockId) - assert(parsedBlockData === blockData) - - assert(handler.invokePrivate(outstandingRequests()).size === 0) - - channel.close() - } - - test("handling error message (failed fetch)") { - val blockId = "test_block" - val errorMsg = "error erro5r error err4or error3 error6 error erro1r" - val totalLength = 4 + blockId.length + errorMsg.length - - var parsedBlockId: String = "" - var parsedErrorMsg: String = "" - val handler = new BlockFetchingClientHandler - handler.addRequest(blockId, new BlockClientListener { - override def onFetchFailure(bid: String, msg: String) ={ - parsedBlockId = bid - parsedErrorMsg = msg - } - override def onFetchSuccess(bid: String, refCntBuf: ReferenceCountedBuffer) = ??? - }) - - val outstandingRequests = PrivateMethod[java.util.Map[_, _]]('outstandingRequests) - assert(handler.invokePrivate(outstandingRequests()).size === 1) - - val channel = new EmbeddedChannel(handler) - val buf = ByteBuffer.allocate(totalLength + 4) // 4 bytes for the length field itself - buf.putInt(totalLength) - buf.putInt(-blockId.length) - buf.put(blockId.getBytes) - buf.put(errorMsg.getBytes) - buf.flip() - - channel.writeInbound(Unpooled.wrappedBuffer(buf)) - assert(parsedBlockId === blockId) - assert(parsedErrorMsg === errorMsg) - - assert(handler.invokePrivate(outstandingRequests()).size === 0) - - channel.close() - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala deleted file mode 100644 index 3f8d0cf8f3ff3..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockHeaderEncoderSuite.scala +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import com.google.common.base.Charsets.UTF_8 -import io.netty.buffer.ByteBuf -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.FunSuite - -class BlockHeaderEncoderSuite extends FunSuite { - - test("encode normal block data") { - val blockId = "test_block" - val channel = new EmbeddedChannel(new BlockHeaderEncoder) - channel.writeOutbound(new BlockHeader(17, blockId, None)) - val out = channel.readOutbound().asInstanceOf[ByteBuf] - assert(out.readInt() === 4 + blockId.length + 17) - assert(out.readInt() === blockId.length) - - val blockIdBytes = new Array[Byte](blockId.length) - out.readBytes(blockIdBytes) - assert(new String(blockIdBytes, UTF_8) === blockId) - assert(out.readableBytes() === 0) - - channel.close() - } - - test("encode error message") { - val blockId = "error_block" - val errorMsg = "error encountered" - val channel = new EmbeddedChannel(new BlockHeaderEncoder) - channel.writeOutbound(new BlockHeader(17, blockId, Some(errorMsg))) - val out = channel.readOutbound().asInstanceOf[ByteBuf] - assert(out.readInt() === 4 + blockId.length + errorMsg.length) - assert(out.readInt() === -blockId.length) - - val blockIdBytes = new Array[Byte](blockId.length) - out.readBytes(blockIdBytes) - assert(new String(blockIdBytes, UTF_8) === blockId) - - val errorMsgBytes = new Array[Byte](errorMsg.length) - out.readBytes(errorMsgBytes) - assert(new String(errorMsgBytes, UTF_8) === errorMsg) - assert(out.readableBytes() === 0) - - channel.close() - } -} diff --git a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala deleted file mode 100644 index 3239c710f1639..0000000000000 --- a/core/src/test/scala/org/apache/spark/network/netty/server/BlockServerHandlerSuite.scala +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty.server - -import java.io.{RandomAccessFile, File} -import java.nio.ByteBuffer - -import io.netty.buffer.{Unpooled, ByteBuf} -import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler, DefaultFileRegion} -import io.netty.channel.embedded.EmbeddedChannel - -import org.scalatest.FunSuite - -import org.apache.spark.storage.{BlockDataProvider, FileSegment} - - -class BlockServerHandlerSuite extends FunSuite { - - test("ByteBuffer block") { - val expectedBlockId = "test_bytebuffer_block" - val buf = ByteBuffer.allocate(10000) - for (i <- 1 to 10000) { - buf.put(i.toByte) - } - buf.flip() - - val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = Right(buf) - })) - - channel.writeInbound(expectedBlockId) - assert(channel.outboundMessages().size === 2) - - val out1 = channel.readOutbound().asInstanceOf[BlockHeader] - val out2 = channel.readOutbound().asInstanceOf[ByteBuf] - - assert(out1.blockId === expectedBlockId) - assert(out1.blockSize === buf.remaining) - assert(out1.error === None) - - assert(out2.equals(Unpooled.wrappedBuffer(buf))) - - channel.close() - } - - test("FileSegment block via zero-copy") { - val expectedBlockId = "test_file_block" - - // Create random file data - val fileContent = new Array[Byte](1024) - scala.util.Random.nextBytes(fileContent) - val testFile = File.createTempFile("netty-test-file", "txt") - val fp = new RandomAccessFile(testFile, "rw") - fp.write(fileContent) - fp.close() - - val channel = new EmbeddedChannel(new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = { - Left(new FileSegment(testFile, 15, testFile.length - 25)) - } - })) - - channel.writeInbound(expectedBlockId) - assert(channel.outboundMessages().size === 2) - - val out1 = channel.readOutbound().asInstanceOf[BlockHeader] - val out2 = channel.readOutbound().asInstanceOf[DefaultFileRegion] - - assert(out1.blockId === expectedBlockId) - assert(out1.blockSize === testFile.length - 25) - assert(out1.error === None) - - assert(out2.count === testFile.length - 25) - assert(out2.position === 15) - } - - test("pipeline exception propagation") { - val blockServerHandler = new BlockServerHandler(new BlockDataProvider { - override def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] = ??? - }) - val exceptionHandler = new SimpleChannelInboundHandler[String]() { - override def channelRead0(ctx: ChannelHandlerContext, msg: String): Unit = { - throw new Exception("this is an error") - } - } - - val channel = new EmbeddedChannel(exceptionHandler, blockServerHandler) - assert(channel.isOpen) - channel.writeInbound("a message to trigger the error") - assert(!channel.isOpen) - } -} diff --git a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala new file mode 100644 index 0000000000000..0ade1bab18d7e --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import java.io.{EOFException, OutputStream, InputStream} +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + + +/** + * A serializer implementation that always return a single element in a deserialization stream. + */ +class TestSerializer extends Serializer { + override def newInstance() = new TestSerializerInstance +} + + +class TestSerializerInstance extends SerializerInstance { + override def serialize[T: ClassTag](t: T): ByteBuffer = ??? + + override def serializeStream(s: OutputStream): SerializationStream = ??? + + override def deserializeStream(s: InputStream) = new TestDeserializationStream + + override def deserialize[T: ClassTag](bytes: ByteBuffer): T = ??? + + override def deserialize[T: ClassTag](bytes: ByteBuffer, loader: ClassLoader): T = ??? +} + + +class TestDeserializationStream extends DeserializationStream { + + private var count = 0 + + override def readObject[T: ClassTag](): T = { + count += 1 + if (count == 2) { + throw new EOFException + } + new Object().asInstanceOf[T] + } + + override def close(): Unit = {} +} diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index ba47fe5e25b9b..6790388f96603 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -25,7 +25,7 @@ import org.scalatest.FunSuite import org.apache.spark.{SparkEnv, SparkContext, LocalSparkContext, SparkConf} import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.network.{FileSegmentManagedBuffer, ManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.shuffle.FileShuffleBlockManager import org.apache.spark.storage.{ShuffleBlockId, FileSegment} @@ -36,9 +36,9 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { private def checkSegments(expected: FileSegment, buffer: ManagedBuffer) { assert(buffer.isInstanceOf[FileSegmentManagedBuffer]) val segment = buffer.asInstanceOf[FileSegmentManagedBuffer] - assert(expected.file.getCanonicalPath === segment.file.getCanonicalPath) - assert(expected.offset === segment.offset) - assert(expected.length === segment.length) + assert(expected.file.getCanonicalPath === segment.getFile.getCanonicalPath) + assert(expected.offset === segment.getOffset) + assert(expected.length === segment.getLength) } test("consolidated shuffle can write to shuffle group without messing existing offsets/lengths") { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index a8c049d749015..4e502cf65e6b3 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -17,6 +17,10 @@ package org.apache.spark.storage +import java.util.concurrent.Semaphore + +import scala.concurrent.future +import scala.concurrent.ExecutionContext.Implicits.global import org.apache.spark.{TaskContextImpl, TaskContext} import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} @@ -27,38 +31,64 @@ import org.mockito.stubbing.Answer import org.scalatest.FunSuite +import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.network._ +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.serializer.TestSerializer + class ShuffleBlockFetcherIteratorSuite extends FunSuite { + // Some of the tests are quite tricky because we are testing the cleanup behavior + // in the presence of faults. - test("handle local read failures in BlockManager") { + /** Creates a mock [[BlockTransferService]] that returns data from the given map. */ + private def createMockTransfer(data: Map[BlockId, ManagedBuffer]): BlockTransferService = { val transfer = mock(classOf[BlockTransferService]) - val blockManager = mock(classOf[BlockManager]) - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) - - val optItr = mock(classOf[Option[Iterator[Any]]]) - val answer = new Answer[Option[Iterator[Any]]] { - override def answer(invocation: InvocationOnMock) = Option[Iterator[Any]] { - throw new Exception + when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val blocks = invocation.getArguments()(2).asInstanceOf[Seq[String]] + val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + + for (blockId <- blocks) { + if (data.contains(BlockId(blockId))) { + listener.onBlockFetchSuccess(blockId, data(BlockId(blockId))) + } else { + listener.onBlockFetchFailure(blockId, new BlockNotFoundException(blockId)) + } + } } + }) + transfer + } + + private val conf = new SparkConf + + test("successful 3 local reads + 2 remote reads") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure blockManager.getBlockData would return the blocks + val localBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer])) + localBlocks.foreach { case (blockId, buf) => + doReturn(buf).when(blockManager).getBlockData(meq(blockId)) } - // 3rd block is going to fail - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doAnswer(answer).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val remoteBlocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 3, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 4, 0) -> mock(classOf[ManagedBuffer]) + ) + + val transfer = createMockTransfer(remoteBlocks) - val bmId = BlockManagerId("test-client", "test-client", 1) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) + (localBmId, localBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq), + (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq) ) val iterator = new ShuffleBlockFetcherIterator( @@ -66,118 +96,145 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { transfer, blockManager, blocksByAddress, - null, + new TestSerializer, 48 * 1024 * 1024) - // Without exhausting the iterator, the iterator should be lazy and not call - // getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - // the 2nd element of the tuple returned by iterator.next should be defined when - // fetching successfully - assert(iterator.next()._2.isDefined, - "1st element should be defined but is not actually defined") - verify(blockManager, times(1)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next()._2.isDefined, - "2nd element should be defined but is not actually defined") - verify(blockManager, times(2)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - // 3rd fetch should be failed - intercept[Exception] { - iterator.next() + // 3 local blocks fetched in initialization + verify(blockManager, times(3)).getBlockData(any()) + + for (i <- 0 until 5) { + assert(iterator.hasNext, s"iterator should have 5 elements but actually has $i elements") + val (blockId, subIterator) = iterator.next() + assert(subIterator.isDefined, + s"iterator should have 5 elements defined but actually has $i elements") + + // Make sure we release the buffer once the iterator is exhausted. + val mockBuf = localBlocks.getOrElse(blockId, remoteBlocks(blockId)) + verify(mockBuf, times(0)).release() + subIterator.get.foreach(_ => Unit) // exhaust the iterator + verify(mockBuf, times(1)).release() } - verify(blockManager, times(3)).getLocalShuffleFromDisk(any(), any()) + + // 3 local blocks, and 2 remote blocks + // (but from the same block manager so one call to fetchBlocks) + verify(blockManager, times(3)).getBlockData(any()) + verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any()) } - test("handle local read successes") { - val transfer = mock(classOf[BlockTransferService]) + test("release current unexhausted buffer in case the task completes early") { val blockManager = mock(classOf[BlockManager]) - doReturn(BlockManagerId("test-client", "test-client", 1)).when(blockManager).blockManagerId - - val blIds = Array[BlockId]( - ShuffleBlockId(0,0,0), - ShuffleBlockId(0,1,0), - ShuffleBlockId(0,2,0), - ShuffleBlockId(0,3,0), - ShuffleBlockId(0,4,0)) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val blocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer]) + ) - val optItr = mock(classOf[Option[Iterator[Any]]]) + // Semaphore to coordinate event sequence in two different threads. + val sem = new Semaphore(0) - // All blocks should be fetched successfully - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(0)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(1)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(2)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(3)), any()) - doReturn(optItr).when(blockManager).getLocalShuffleFromDisk(meq(blIds(4)), any()) + val transfer = mock(classOf[BlockTransferService]) + when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + override def answer(invocation: InvocationOnMock): Unit = { + val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + future { + // Return the first two blocks, and wait till task completion before returning the 3rd one + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 1, 0).toString, blocks(ShuffleBlockId(0, 1, 0))) + sem.acquire() + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 2, 0).toString, blocks(ShuffleBlockId(0, 2, 0))) + } + } + }) - val bmId = BlockManagerId("test-client", "test-client", 1) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, blIds.map(blId => (blId, 1.asInstanceOf[Long])).toSeq) - ) + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + val taskContext = new TaskContextImpl(0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( - new TaskContextImpl(0, 0, 0), + taskContext, transfer, blockManager, blocksByAddress, - null, + new TestSerializer, 48 * 1024 * 1024) - // Without exhausting the iterator, the iterator should be lazy and not call getLocalShuffleFromDisk. - verify(blockManager, times(0)).getLocalShuffleFromDisk(any(), any()) - - assert(iterator.hasNext, "iterator should have 5 elements but actually has no elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 1st element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 1 element") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 2nd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 2 elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 3rd element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 3 elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 4th element is not actually defined") - assert(iterator.hasNext, "iterator should have 5 elements but actually has 4 elements") - assert(iterator.next()._2.isDefined, - "All elements should be defined but 5th element is not actually defined") - - verify(blockManager, times(5)).getLocalShuffleFromDisk(any(), any()) + // Exhaust the first block, and then it should be released. + iterator.next()._2.get.foreach(_ => Unit) + verify(blocks(ShuffleBlockId(0, 0, 0)), times(1)).release() + + // Get the 2nd block but do not exhaust the iterator + val subIter = iterator.next()._2.get + + // Complete the task; then the 2nd block buffer should be exhausted + verify(blocks(ShuffleBlockId(0, 1, 0)), times(0)).release() + taskContext.markTaskCompleted() + verify(blocks(ShuffleBlockId(0, 1, 0)), times(1)).release() + + // The 3rd block should not be retained because the iterator is already in zombie state + sem.release() + verify(blocks(ShuffleBlockId(0, 2, 0)), times(0)).retain() + verify(blocks(ShuffleBlockId(0, 2, 0)), times(0)).release() } - test("handle remote fetch failures in BlockTransferService") { + test("fail all blocks if any of the remote request fails") { + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + + // Make sure remote blocks would return + val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) + val blocks = Map[BlockId, ManagedBuffer]( + ShuffleBlockId(0, 0, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 1, 0) -> mock(classOf[ManagedBuffer]), + ShuffleBlockId(0, 2, 0) -> mock(classOf[ManagedBuffer]) + ) + + // Semaphore to coordinate event sequence in two different threads. + val sem = new Semaphore(0) + val transfer = mock(classOf[BlockTransferService]) when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] - listener.onBlockFetchFailure(new Exception("blah")) + future { + // Return the first block, and then fail. + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, blocks(ShuffleBlockId(0, 0, 0))) + listener.onBlockFetchFailure( + ShuffleBlockId(0, 1, 0).toString, new BlockNotFoundException("blah")) + listener.onBlockFetchFailure( + ShuffleBlockId(0, 2, 0).toString, new BlockNotFoundException("blah")) + sem.release() + } } }) - val blockManager = mock(classOf[BlockManager]) - - when(blockManager.blockManagerId).thenReturn(BlockManagerId("test-client", "test-client", 1)) - - val blId1 = ShuffleBlockId(0, 0, 0) - val blId2 = ShuffleBlockId(0, 1, 0) - val bmId = BlockManagerId("test-server", "test-server", 1) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (bmId, Seq((blId1, 1L), (blId2, 1L)))) + (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)) + val taskContext = new TaskContextImpl(0, 0, 0) val iterator = new ShuffleBlockFetcherIterator( - new TaskContextImpl(0, 0, 0), + taskContext, transfer, blockManager, blocksByAddress, - null, + new TestSerializer, 48 * 1024 * 1024) - iterator.foreach { case (_, iterOption) => - assert(!iterOption.isDefined) - } + // Continue only after the mock calls onBlockFetchFailure + sem.acquire() + + // The first block should be defined, and the last two are not defined (due to failure) + assert(iterator.next()._2.isDefined === true) + assert(iterator.next()._2.isDefined === false) + assert(iterator.next()._2.isDefined === false) } } diff --git a/network/common/pom.xml b/network/common/pom.xml new file mode 100644 index 0000000000000..e3b7e328701b4 --- /dev/null +++ b/network/common/pom.xml @@ -0,0 +1,94 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.2.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + network + jar + Shuffle Streaming Service + http://spark.apache.org/ + + network + + + + + + io.netty + netty-all + + + org.slf4j + slf4j-api + + + + + com.google.guava + guava + provided + + + + + junit + junit + test + + + log4j + log4j + test + + + org.mockito + mockito-all + test + + + + + + target/java/classes + target/java/test-classes + + + org.apache.maven.plugins + maven-surefire-plugin + 2.17 + + false + + **/Test*.java + **/*Test.java + **/*Suite.java + + + + + + diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java new file mode 100644 index 0000000000000..854aa6685f85f --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import io.netty.channel.Channel; +import io.netty.channel.socket.SocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.MessageDecoder; +import org.apache.spark.network.protocol.MessageEncoder; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportChannelHandler; +import org.apache.spark.network.server.TransportRequestHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Contains the context to create a {@link TransportServer}, {@link TransportClientFactory}, and to + * setup Netty Channel pipelines with a {@link org.apache.spark.network.server.TransportChannelHandler}. + * + * There are two communication protocols that the TransportClient provides, control-plane RPCs and + * data-plane "chunk fetching". The handling of the RPCs is performed outside of the scope of the + * TransportContext (i.e., by a user-provided handler), and it is responsible for setting up streams + * which can be streamed through the data plane in chunks using zero-copy IO. + * + * The TransportServer and TransportClientFactory both create a TransportChannelHandler for each + * channel. As each TransportChannelHandler contains a TransportClient, this enables server + * processes to send messages back to the client on an existing channel. + */ +public class TransportContext { + private final Logger logger = LoggerFactory.getLogger(TransportContext.class); + + private final TransportConf conf; + private final StreamManager streamManager; + private final RpcHandler rpcHandler; + + private final MessageEncoder encoder; + private final MessageDecoder decoder; + + public TransportContext(TransportConf conf, StreamManager streamManager, RpcHandler rpcHandler) { + this.conf = conf; + this.streamManager = streamManager; + this.rpcHandler = rpcHandler; + this.encoder = new MessageEncoder(); + this.decoder = new MessageDecoder(); + } + + public TransportClientFactory createClientFactory() { + return new TransportClientFactory(this); + } + + public TransportServer createServer() { + return new TransportServer(this); + } + + /** + * Initializes a client or server Netty Channel Pipeline which encodes/decodes messages and + * has a {@link org.apache.spark.network.server.TransportChannelHandler} to handle request or + * response messages. + * + * @return Returns the created TransportChannelHandler, which includes a TransportClient that can + * be used to communicate on this channel. The TransportClient is directly associated with a + * ChannelHandler to ensure all users of the same channel get the same TransportClient object. + */ + public TransportChannelHandler initializePipeline(SocketChannel channel) { + try { + TransportChannelHandler channelHandler = createChannelHandler(channel); + channel.pipeline() + .addLast("encoder", encoder) + .addLast("frameDecoder", NettyUtils.createFrameDecoder()) + .addLast("decoder", decoder) + // NOTE: Chunks are currently guaranteed to be returned in the order of request, but this + // would require more logic to guarantee if this were not part of the same event loop. + .addLast("handler", channelHandler); + return channelHandler; + } catch (RuntimeException e) { + logger.error("Error while initializing Netty pipeline", e); + throw e; + } + } + + /** + * Creates the server- and client-side handler which is used to handle both RequestMessages and + * ResponseMessages. The channel is expected to have been successfully created, though certain + * properties (such as the remoteAddress()) may not be available yet. + */ + private TransportChannelHandler createChannelHandler(Channel channel) { + TransportResponseHandler responseHandler = new TransportResponseHandler(channel); + TransportClient client = new TransportClient(channel, responseHandler); + TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, + streamManager, rpcHandler); + return new TransportChannelHandler(client, responseHandler, requestHandler); + } + + public TransportConf getConf() { return conf; } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java new file mode 100644 index 0000000000000..89ed79bc63903 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/FileSegmentManagedBuffer.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.buffer; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; + +import com.google.common.base.Objects; +import com.google.common.io.ByteStreams; +import io.netty.channel.DefaultFileRegion; + +import org.apache.spark.network.util.JavaUtils; + +/** + * A {@link ManagedBuffer} backed by a segment in a file. + */ +public final class FileSegmentManagedBuffer extends ManagedBuffer { + + /** + * Memory mapping is expensive and can destabilize the JVM (SPARK-1145, SPARK-3889). + * Avoid unless there's a good reason not to. + */ + // TODO: Make this configurable + private static final long MIN_MEMORY_MAP_BYTES = 2 * 1024 * 1024; + + private final File file; + private final long offset; + private final long length; + + public FileSegmentManagedBuffer(File file, long offset, long length) { + this.file = file; + this.offset = offset; + this.length = length; + } + + @Override + public long size() { + return length; + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + FileChannel channel = null; + try { + channel = new RandomAccessFile(file, "r").getChannel(); + // Just copy the buffer if it's sufficiently small, as memory mapping has a high overhead. + if (length < MIN_MEMORY_MAP_BYTES) { + ByteBuffer buf = ByteBuffer.allocate((int) length); + channel.position(offset); + while (buf.remaining() != 0) { + if (channel.read(buf) == -1) { + throw new IOException(String.format("Reached EOF before filling buffer\n" + + "offset=%s\nfile=%s\nbuf.remaining=%s", + offset, file.getAbsoluteFile(), buf.remaining())); + } + } + buf.flip(); + return buf; + } else { + return channel.map(FileChannel.MapMode.READ_ONLY, offset, length); + } + } catch (IOException e) { + try { + if (channel != null) { + long size = channel.size(); + throw new IOException("Error in reading " + this + " (actual file length " + size + ")", + e); + } + } catch (IOException ignored) { + // ignore + } + throw new IOException("Error in opening " + this, e); + } finally { + JavaUtils.closeQuietly(channel); + } + } + + @Override + public InputStream createInputStream() throws IOException { + FileInputStream is = null; + try { + is = new FileInputStream(file); + ByteStreams.skipFully(is, offset); + return ByteStreams.limit(is, length); + } catch (IOException e) { + try { + if (is != null) { + long size = file.length(); + throw new IOException("Error in reading " + this + " (actual file length " + size + ")", + e); + } + } catch (IOException ignored) { + // ignore + } finally { + JavaUtils.closeQuietly(is); + } + throw new IOException("Error in opening " + this, e); + } catch (RuntimeException e) { + JavaUtils.closeQuietly(is); + throw e; + } + } + + @Override + public ManagedBuffer retain() { + return this; + } + + @Override + public ManagedBuffer release() { + return this; + } + + @Override + public Object convertToNetty() throws IOException { + FileChannel fileChannel = new FileInputStream(file).getChannel(); + return new DefaultFileRegion(fileChannel, offset, length); + } + + public File getFile() { return file; } + + public long getOffset() { return offset; } + + public long getLength() { return length; } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("file", file) + .add("offset", offset) + .add("length", length) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java new file mode 100644 index 0000000000000..a415db593a788 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +/** + * This interface provides an immutable view for data in the form of bytes. The implementation + * should specify how the data is provided: + * + * - {@link FileSegmentManagedBuffer}: data backed by part of a file + * - {@link NioManagedBuffer}: data backed by a NIO ByteBuffer + * - {@link NettyManagedBuffer}: data backed by a Netty ByteBuf + * + * The concrete buffer implementation might be managed outside the JVM garbage collector. + * For example, in the case of {@link NettyManagedBuffer}, the buffers are reference counted. + * In that case, if the buffer is going to be passed around to a different thread, retain/release + * should be called. + */ +public abstract class ManagedBuffer { + + /** Number of bytes of the data. */ + public abstract long size(); + + /** + * Exposes this buffer's data as an NIO ByteBuffer. Changing the position and limit of the + * returned ByteBuffer should not affect the content of this buffer. + */ + // TODO: Deprecate this, usage may require expensive memory mapping or allocation. + public abstract ByteBuffer nioByteBuffer() throws IOException; + + /** + * Exposes this buffer's data as an InputStream. The underlying implementation does not + * necessarily check for the length of bytes read, so the caller is responsible for making sure + * it does not go over the limit. + */ + public abstract InputStream createInputStream() throws IOException; + + /** + * Increment the reference count by one if applicable. + */ + public abstract ManagedBuffer retain(); + + /** + * If applicable, decrement the reference count by one and deallocates the buffer if the + * reference count reaches zero. + */ + public abstract ManagedBuffer release(); + + /** + * Convert the buffer into an Netty object, used to write the data out. + */ + public abstract Object convertToNetty() throws IOException; +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java new file mode 100644 index 0000000000000..c806bfa45bef3 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NettyManagedBuffer.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.ByteBufInputStream; + +/** + * A {@link ManagedBuffer} backed by a Netty {@link ByteBuf}. + */ +public final class NettyManagedBuffer extends ManagedBuffer { + private final ByteBuf buf; + + public NettyManagedBuffer(ByteBuf buf) { + this.buf = buf; + } + + @Override + public long size() { + return buf.readableBytes(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return buf.nioBuffer(); + } + + @Override + public InputStream createInputStream() throws IOException { + return new ByteBufInputStream(buf); + } + + @Override + public ManagedBuffer retain() { + buf.retain(); + return this; + } + + @Override + public ManagedBuffer release() { + buf.release(); + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return buf.duplicate(); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("buf", buf) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java new file mode 100644 index 0000000000000..f55b884bc45ce --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/buffer/NioManagedBuffer.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.buffer; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBufInputStream; +import io.netty.buffer.Unpooled; + +/** + * A {@link ManagedBuffer} backed by {@link ByteBuffer}. + */ +public final class NioManagedBuffer extends ManagedBuffer { + private final ByteBuffer buf; + + public NioManagedBuffer(ByteBuffer buf) { + this.buf = buf; + } + + @Override + public long size() { + return buf.remaining(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return buf.duplicate(); + } + + @Override + public InputStream createInputStream() throws IOException { + return new ByteBufInputStream(Unpooled.wrappedBuffer(buf)); + } + + @Override + public ManagedBuffer retain() { + return this; + } + + @Override + public ManagedBuffer release() { + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return Unpooled.wrappedBuffer(buf); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("buf", buf) + .toString(); + } +} + diff --git a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala b/network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java similarity index 67% rename from core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala rename to network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java index 5b6d086630834..1fbdcd6780785 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockDataProvider.scala +++ b/network/common/src/main/java/org/apache/spark/network/client/ChunkFetchFailureException.java @@ -15,18 +15,17 @@ * limitations under the License. */ -package org.apache.spark.storage - -import java.nio.ByteBuffer - +package org.apache.spark.network.client; /** - * An interface for providing data for blocks. - * - * getBlockData returns either a FileSegment (for zero-copy send), or a ByteBuffer. - * - * Aside from unit tests, [[BlockManager]] is the main class that implements this. + * General exception caused by a remote exception while fetching a chunk. */ -private[spark] trait BlockDataProvider { - def getBlockData(blockId: String): Either[FileSegment, ByteBuffer] +public class ChunkFetchFailureException extends RuntimeException { + public ChunkFetchFailureException(String errorMsg, Throwable cause) { + super(errorMsg, cause); + } + + public ChunkFetchFailureException(String errorMsg) { + super(errorMsg); + } } diff --git a/network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java b/network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java new file mode 100644 index 0000000000000..519e6cb470d0d --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/ChunkReceivedCallback.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * Callback for the result of a single chunk result. For a single stream, the callbacks are + * guaranteed to be called by the same thread in the same order as the requests for chunks were + * made. + * + * Note that if a general stream failure occurs, all outstanding chunk requests may be failed. + */ +public interface ChunkReceivedCallback { + /** + * Called upon receipt of a particular chunk. + * + * The given buffer will initially have a refcount of 1, but will be release()'d as soon as this + * call returns. You must therefore either retain() the buffer or copy its contents before + * returning. + */ + void onSuccess(int chunkIndex, ManagedBuffer buffer); + + /** + * Called upon failure to fetch a particular chunk. Note that this may actually be called due + * to failure to fetch a prior chunk in this stream. + * + * After receiving a failure, the stream may or may not be valid. The client should not assume + * that the server's side of the stream has been closed. + */ + void onFailure(int chunkIndex, Throwable e); +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala b/network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java similarity index 55% rename from core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala rename to network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java index 162e9cc6828d4..6ec960d795420 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/server/BlockHeader.scala +++ b/network/common/src/main/java/org/apache/spark/network/client/RpcResponseCallback.java @@ -15,18 +15,16 @@ * limitations under the License. */ -package org.apache.spark.network.netty.server +package org.apache.spark.network.client; /** - * Header describing a block. This is used only in the server pipeline. - * - * [[BlockServerHandler]] creates this, and [[BlockHeaderEncoder]] encodes it. - * - * @param blockSize length of the block content, excluding the length itself. - * If positive, this is the header for a block (not part of the header). - * If negative, this is the header and content for an error message. - * @param blockId block id - * @param error some error message from reading the block + * Callback for the result of a single RPC. This will be invoked once with either success or + * failure. */ -private[server] -class BlockHeader(val blockSize: Int, val blockId: String, val error: Option[String] = None) +public interface RpcResponseCallback { + /** Successful serialized result from server. */ + void onSuccess(byte[] response); + + /** Exception either propagated from server or raised on client side. */ + void onFailure(Throwable e); +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java new file mode 100644 index 0000000000000..b1732fcde21f1 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -0,0 +1,159 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +import java.io.Closeable; +import java.util.UUID; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Preconditions; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.util.NettyUtils; + +/** + * Client for fetching consecutive chunks of a pre-negotiated stream. This API is intended to allow + * efficient transfer of a large amount of data, broken up into chunks with size ranging from + * hundreds of KB to a few MB. + * + * Note that while this client deals with the fetching of chunks from a stream (i.e., data plane), + * the actual setup of the streams is done outside the scope of the transport layer. The convenience + * method "sendRPC" is provided to enable control plane communication between the client and server + * to perform this setup. + * + * For example, a typical workflow might be: + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100 + * client.fetchChunk(streamId = 100, chunkIndex = 0, callback) + * client.fetchChunk(streamId = 100, chunkIndex = 1, callback) + * ... + * client.sendRPC(new CloseStream(100)) + * + * Construct an instance of TransportClient using {@link TransportClientFactory}. A single + * TransportClient may be used for multiple streams, but any given stream must be restricted to a + * single client, in order to avoid out-of-order responses. + * + * NB: This class is used to make requests to the server, while {@link TransportResponseHandler} is + * responsible for handling responses from the server. + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportClient implements Closeable { + private final Logger logger = LoggerFactory.getLogger(TransportClient.class); + + private final Channel channel; + private final TransportResponseHandler handler; + + public TransportClient(Channel channel, TransportResponseHandler handler) { + this.channel = Preconditions.checkNotNull(channel); + this.handler = Preconditions.checkNotNull(handler); + } + + public boolean isActive() { + return channel.isOpen() || channel.isActive(); + } + + /** + * Requests a single chunk from the remote side, from the pre-negotiated streamId. + * + * Chunk indices go from 0 onwards. It is valid to request the same chunk multiple times, though + * some streams may not support this. + * + * Multiple fetchChunk requests may be outstanding simultaneously, and the chunks are guaranteed + * to be returned in the same order that they were requested, assuming only a single + * TransportClient is used to fetch the chunks. + * + * @param streamId Identifier that refers to a stream in the remote StreamManager. This should + * be agreed upon by client and server beforehand. + * @param chunkIndex 0-based index of the chunk to fetch + * @param callback Callback invoked upon successful receipt of chunk, or upon any failure. + */ + public void fetchChunk( + long streamId, + final int chunkIndex, + final ChunkReceivedCallback callback) { + final String serverAddr = NettyUtils.getRemoteAddress(channel); + final long startTime = System.currentTimeMillis(); + logger.debug("Sending fetch chunk request {} to {}", chunkIndex, serverAddr); + + final StreamChunkId streamChunkId = new StreamChunkId(streamId, chunkIndex); + handler.addFetchRequest(streamChunkId, callback); + + channel.writeAndFlush(new ChunkFetchRequest(streamChunkId)).addListener( + new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + long timeTaken = System.currentTimeMillis() - startTime; + logger.trace("Sending request {} to {} took {} ms", streamChunkId, serverAddr, + timeTaken); + } else { + String errorMsg = String.format("Failed to send request %s to %s: %s", streamChunkId, + serverAddr, future.cause()); + logger.error(errorMsg, future.cause()); + handler.removeFetchRequest(streamChunkId); + callback.onFailure(chunkIndex, new RuntimeException(errorMsg, future.cause())); + channel.close(); + } + } + }); + } + + /** + * Sends an opaque message to the RpcHandler on the server-side. The callback will be invoked + * with the server's response or upon any failure. + */ + public void sendRpc(byte[] message, final RpcResponseCallback callback) { + final String serverAddr = NettyUtils.getRemoteAddress(channel); + final long startTime = System.currentTimeMillis(); + logger.trace("Sending RPC to {}", serverAddr); + + final long requestId = UUID.randomUUID().getLeastSignificantBits(); + handler.addRpcRequest(requestId, callback); + + channel.writeAndFlush(new RpcRequest(requestId, message)).addListener( + new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + long timeTaken = System.currentTimeMillis() - startTime; + logger.trace("Sending request {} to {} took {} ms", requestId, serverAddr, timeTaken); + } else { + String errorMsg = String.format("Failed to send RPC %s to %s: %s", requestId, + serverAddr, future.cause()); + logger.error(errorMsg, future.cause()); + handler.removeRpcRequest(requestId); + callback.onFailure(new RuntimeException(errorMsg, future.cause())); + channel.close(); + } + } + }); + } + + @Override + public void close() { + // close is a local operation and should finish with milliseconds; timeout just to be safe + channel.close().awaitUninterruptibly(10, TimeUnit.SECONDS); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java new file mode 100644 index 0000000000000..10eb9ef7a025f --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +import java.io.Closeable; +import java.lang.reflect.Field; +import java.net.InetSocketAddress; +import java.net.SocketAddress; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; + +import io.netty.bootstrap.Bootstrap; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import io.netty.util.internal.PlatformDependent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.server.TransportChannelHandler; +import org.apache.spark.network.util.IOMode; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Factory for creating {@link TransportClient}s by using createClient. + * + * The factory maintains a connection pool to other hosts and should return the same + * {@link TransportClient} for the same remote host. It also shares a single worker thread pool for + * all {@link TransportClient}s. + */ +public class TransportClientFactory implements Closeable { + private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); + + private final TransportContext context; + private final TransportConf conf; + private final ConcurrentHashMap connectionPool; + + private final Class socketChannelClass; + private final EventLoopGroup workerGroup; + + public TransportClientFactory(TransportContext context) { + this.context = context; + this.conf = context.getConf(); + this.connectionPool = new ConcurrentHashMap(); + + IOMode ioMode = IOMode.valueOf(conf.ioMode()); + this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); + // TODO: Make thread pool name configurable. + this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client"); + } + + /** + * Create a new BlockFetchingClient connecting to the given remote host / port. + * + * This blocks until a connection is successfully established. + * + * Concurrency: This method is safe to call from multiple threads. + */ + public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException { + // Get connection from the connection pool first. + // If it is not found or not active, create a new one. + final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); + TransportClient cachedClient = connectionPool.get(address); + if (cachedClient != null && cachedClient.isActive()) { + return cachedClient; + } else if (cachedClient != null) { + connectionPool.remove(address, cachedClient); // Remove inactive clients. + } + + logger.debug("Creating new connection to " + address); + + Bootstrap bootstrap = new Bootstrap(); + bootstrap.group(workerGroup) + .channel(socketChannelClass) + // Disable Nagle's Algorithm since we don't want packets to wait + .option(ChannelOption.TCP_NODELAY, true) + .option(ChannelOption.SO_KEEPALIVE, true) + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()); + + // Use pooled buffers to reduce temporary buffer allocation + bootstrap.option(ChannelOption.ALLOCATOR, createPooledByteBufAllocator()); + + final AtomicReference client = new AtomicReference(); + + bootstrap.handler(new ChannelInitializer() { + @Override + public void initChannel(SocketChannel ch) { + TransportChannelHandler clientHandler = context.initializePipeline(ch); + client.set(clientHandler.getClient()); + } + }); + + // Connect to the remote server + ChannelFuture cf = bootstrap.connect(address); + if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { + throw new TimeoutException( + String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); + } else if (cf.cause() != null) { + throw new RuntimeException(String.format("Failed to connect to %s", address), cf.cause()); + } + + // Successful connection + assert client.get() != null : "Channel future completed successfully with null client"; + TransportClient oldClient = connectionPool.putIfAbsent(address, client.get()); + if (oldClient == null) { + return client.get(); + } else { + logger.debug("Two clients were created concurrently, second one will be disposed."); + client.get().close(); + return oldClient; + } + } + + /** Close all connections in the connection pool, and shutdown the worker thread pool. */ + @Override + public void close() { + for (TransportClient client : connectionPool.values()) { + try { + client.close(); + } catch (RuntimeException e) { + logger.warn("Ignoring exception during close", e); + } + } + connectionPool.clear(); + + if (workerGroup != null) { + workerGroup.shutdownGracefully(); + } + } + + /** + * Create a pooled ByteBuf allocator but disables the thread-local cache. Thread-local caches + * are disabled because the ByteBufs are allocated by the event loop thread, but released by the + * executor thread rather than the event loop thread. Those thread-local caches actually delay + * the recycling of buffers, leading to larger memory usage. + */ + private PooledByteBufAllocator createPooledByteBufAllocator() { + return new PooledByteBufAllocator( + PlatformDependent.directBufferPreferred(), + getPrivateStaticField("DEFAULT_NUM_HEAP_ARENA"), + getPrivateStaticField("DEFAULT_NUM_DIRECT_ARENA"), + getPrivateStaticField("DEFAULT_PAGE_SIZE"), + getPrivateStaticField("DEFAULT_MAX_ORDER"), + 0, // tinyCacheSize + 0, // smallCacheSize + 0 // normalCacheSize + ); + } + + /** Used to get defaults from Netty's private static fields. */ + private int getPrivateStaticField(String name) { + try { + Field f = PooledByteBufAllocator.DEFAULT.getClass().getDeclaredField(name); + f.setAccessible(true); + return f.getInt(null); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java new file mode 100644 index 0000000000000..d8965590b34da --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.client; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import com.google.common.annotations.VisibleForTesting; +import io.netty.channel.Channel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.ResponseMessage; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.server.MessageHandler; +import org.apache.spark.network.util.NettyUtils; + +/** + * Handler that processes server responses, in response to requests issued from a + * [[TransportClient]]. It works by tracking the list of outstanding requests (and their callbacks). + * + * Concurrency: thread safe and can be called from multiple threads. + */ +public class TransportResponseHandler extends MessageHandler { + private final Logger logger = LoggerFactory.getLogger(TransportResponseHandler.class); + + private final Channel channel; + + private final Map outstandingFetches; + + private final Map outstandingRpcs; + + public TransportResponseHandler(Channel channel) { + this.channel = channel; + this.outstandingFetches = new ConcurrentHashMap(); + this.outstandingRpcs = new ConcurrentHashMap(); + } + + public void addFetchRequest(StreamChunkId streamChunkId, ChunkReceivedCallback callback) { + outstandingFetches.put(streamChunkId, callback); + } + + public void removeFetchRequest(StreamChunkId streamChunkId) { + outstandingFetches.remove(streamChunkId); + } + + public void addRpcRequest(long requestId, RpcResponseCallback callback) { + outstandingRpcs.put(requestId, callback); + } + + public void removeRpcRequest(long requestId) { + outstandingRpcs.remove(requestId); + } + + /** + * Fire the failure callback for all outstanding requests. This is called when we have an + * uncaught exception or pre-mature connection termination. + */ + private void failOutstandingRequests(Throwable cause) { + for (Map.Entry entry : outstandingFetches.entrySet()) { + entry.getValue().onFailure(entry.getKey().chunkIndex, cause); + } + for (Map.Entry entry : outstandingRpcs.entrySet()) { + entry.getValue().onFailure(cause); + } + + // It's OK if new fetches appear, as they will fail immediately. + outstandingFetches.clear(); + outstandingRpcs.clear(); + } + + @Override + public void channelUnregistered() { + if (numOutstandingRequests() > 0) { + String remoteAddress = NettyUtils.getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(new RuntimeException("Connection from " + remoteAddress + " closed")); + } + } + + @Override + public void exceptionCaught(Throwable cause) { + if (numOutstandingRequests() > 0) { + String remoteAddress = NettyUtils.getRemoteAddress(channel); + logger.error("Still have {} requests outstanding when connection from {} is closed", + numOutstandingRequests(), remoteAddress); + failOutstandingRequests(cause); + } + } + + @Override + public void handle(ResponseMessage message) { + String remoteAddress = NettyUtils.getRemoteAddress(channel); + if (message instanceof ChunkFetchSuccess) { + ChunkFetchSuccess resp = (ChunkFetchSuccess) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} since it is not outstanding", + resp.streamChunkId, remoteAddress); + resp.buffer.release(); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onSuccess(resp.streamChunkId.chunkIndex, resp.buffer); + resp.buffer.release(); + } + } else if (message instanceof ChunkFetchFailure) { + ChunkFetchFailure resp = (ChunkFetchFailure) message; + ChunkReceivedCallback listener = outstandingFetches.get(resp.streamChunkId); + if (listener == null) { + logger.warn("Ignoring response for block {} from {} ({}) since it is not outstanding", + resp.streamChunkId, remoteAddress, resp.errorString); + } else { + outstandingFetches.remove(resp.streamChunkId); + listener.onFailure(resp.streamChunkId.chunkIndex, new ChunkFetchFailureException( + "Failure while fetching " + resp.streamChunkId + ": " + resp.errorString)); + } + } else if (message instanceof RpcResponse) { + RpcResponse resp = (RpcResponse) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({} bytes) since it is not outstanding", + resp.requestId, remoteAddress, resp.response.length); + } else { + outstandingRpcs.remove(resp.requestId); + listener.onSuccess(resp.response); + } + } else if (message instanceof RpcFailure) { + RpcFailure resp = (RpcFailure) message; + RpcResponseCallback listener = outstandingRpcs.get(resp.requestId); + if (listener == null) { + logger.warn("Ignoring response for RPC {} from {} ({}) since it is not outstanding", + resp.requestId, remoteAddress, resp.errorString); + } else { + outstandingRpcs.remove(resp.requestId); + listener.onFailure(new RuntimeException(resp.errorString)); + } + } else { + throw new IllegalStateException("Unknown response type: " + message.type()); + } + } + + /** Returns total number of outstanding requests (fetch requests + rpcs) */ + @VisibleForTesting + public int numOutstandingRequests() { + return outstandingFetches.size() + outstandingRpcs.size(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java new file mode 100644 index 0000000000000..152af98ced7ce --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Charsets; +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** + * Response to {@link ChunkFetchRequest} when there is an error fetching the chunk. + */ +public final class ChunkFetchFailure implements ResponseMessage { + public final StreamChunkId streamChunkId; + public final String errorString; + + public ChunkFetchFailure(StreamChunkId streamChunkId, String errorString) { + this.streamChunkId = streamChunkId; + this.errorString = errorString; + } + + @Override + public Type type() { return Type.ChunkFetchFailure; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength() + 4 + errorString.getBytes(Charsets.UTF_8).length; + } + + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + byte[] errorBytes = errorString.getBytes(Charsets.UTF_8); + buf.writeInt(errorBytes.length); + buf.writeBytes(errorBytes); + } + + public static ChunkFetchFailure decode(ByteBuf buf) { + StreamChunkId streamChunkId = StreamChunkId.decode(buf); + int numErrorStringBytes = buf.readInt(); + byte[] errorBytes = new byte[numErrorStringBytes]; + buf.readBytes(errorBytes); + return new ChunkFetchFailure(streamChunkId, new String(errorBytes, Charsets.UTF_8)); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchFailure) { + ChunkFetchFailure o = (ChunkFetchFailure) other; + return streamChunkId.equals(o.streamChunkId) && errorString.equals(o.errorString); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamChunkId", streamChunkId) + .add("errorString", errorString) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java new file mode 100644 index 0000000000000..980947cf13f6b --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** + * Request to fetch a sequence of a single chunk of a stream. This will correspond to a single + * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure). + */ +public final class ChunkFetchRequest implements RequestMessage { + public final StreamChunkId streamChunkId; + + public ChunkFetchRequest(StreamChunkId streamChunkId) { + this.streamChunkId = streamChunkId; + } + + @Override + public Type type() { return Type.ChunkFetchRequest; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength(); + } + + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + } + + public static ChunkFetchRequest decode(ByteBuf buf) { + return new ChunkFetchRequest(StreamChunkId.decode(buf)); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchRequest) { + ChunkFetchRequest o = (ChunkFetchRequest) other; + return streamChunkId.equals(o.streamChunkId); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamChunkId", streamChunkId) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java new file mode 100644 index 0000000000000..ff4936470c697 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** + * Response to {@link ChunkFetchRequest} when a chunk exists and has been successfully fetched. + * + * Note that the server-side encoding of this messages does NOT include the buffer itself, as this + * may be written by Netty in a more efficient manner (i.e., zero-copy write). + * Similarly, the client-side decoding will reuse the Netty ByteBuf as the buffer. + */ +public final class ChunkFetchSuccess implements ResponseMessage { + public final StreamChunkId streamChunkId; + public final ManagedBuffer buffer; + + public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) { + this.streamChunkId = streamChunkId; + this.buffer = buffer; + } + + @Override + public Type type() { return Type.ChunkFetchSuccess; } + + @Override + public int encodedLength() { + return streamChunkId.encodedLength(); + } + + /** Encoding does NOT include 'buffer' itself. See {@link MessageEncoder}. */ + @Override + public void encode(ByteBuf buf) { + streamChunkId.encode(buf); + } + + /** Decoding uses the given ByteBuf as our data, and will retain() it. */ + public static ChunkFetchSuccess decode(ByteBuf buf) { + StreamChunkId streamChunkId = StreamChunkId.decode(buf); + buf.retain(); + NettyManagedBuffer managedBuf = new NettyManagedBuffer(buf.duplicate()); + return new ChunkFetchSuccess(streamChunkId, managedBuf); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ChunkFetchSuccess) { + ChunkFetchSuccess o = (ChunkFetchSuccess) other; + return streamChunkId.equals(o.streamChunkId) && buffer.equals(o.buffer); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamChunkId", streamChunkId) + .add("buffer", buffer) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java b/network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java new file mode 100644 index 0000000000000..b4e299471b41a --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/Encodable.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import io.netty.buffer.ByteBuf; + +/** + * Interface for an object which can be encoded into a ByteBuf. Multiple Encodable objects are + * stored in a single, pre-allocated ByteBuf, so Encodables must also provide their length. + * + * Encodable objects should provide a static "decode(ByteBuf)" method which is invoked by + * {@link MessageDecoder}. During decoding, if the object uses the ByteBuf as its data (rather than + * just copying data from it), then you must retain() the ByteBuf. + * + * Additionally, when adding a new Encodable Message, add it to {@link Message.Type}. + */ +public interface Encodable { + /** Number of bytes of the encoded form of this object. */ + int encodedLength(); + + /** + * Serializes this object by writing into the given ByteBuf. + * This method must write exactly encodedLength() bytes. + */ + void encode(ByteBuf buf); +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/Message.java b/network/common/src/main/java/org/apache/spark/network/protocol/Message.java new file mode 100644 index 0000000000000..d568370125fd4 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/Message.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import io.netty.buffer.ByteBuf; + +/** An on-the-wire transmittable message. */ +public interface Message extends Encodable { + /** Used to identify this request type. */ + Type type(); + + /** Preceding every serialized Message is its type, which allows us to deserialize it. */ + public static enum Type implements Encodable { + ChunkFetchRequest(0), ChunkFetchSuccess(1), ChunkFetchFailure(2), + RpcRequest(3), RpcResponse(4), RpcFailure(5); + + private final byte id; + + private Type(int id) { + assert id < 128 : "Cannot have more than 128 message types"; + this.id = (byte) id; + } + + public byte id() { return id; } + + @Override public int encodedLength() { return 1; } + + @Override public void encode(ByteBuf buf) { buf.writeByte(id); } + + public static Type decode(ByteBuf buf) { + byte id = buf.readByte(); + switch (id) { + case 0: return ChunkFetchRequest; + case 1: return ChunkFetchSuccess; + case 2: return ChunkFetchFailure; + case 3: return RpcRequest; + case 4: return RpcResponse; + case 5: return RpcFailure; + default: throw new IllegalArgumentException("Unknown message type: " + id); + } + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java new file mode 100644 index 0000000000000..81f8d7f96350f --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageDecoder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Decoder used by the client side to encode server-to-client responses. + * This encoder is stateless so it is safe to be shared by multiple threads. + */ +@ChannelHandler.Sharable +public final class MessageDecoder extends MessageToMessageDecoder { + + private final Logger logger = LoggerFactory.getLogger(MessageDecoder.class); + @Override + public void decode(ChannelHandlerContext ctx, ByteBuf in, List out) { + Message.Type msgType = Message.Type.decode(in); + Message decoded = decode(msgType, in); + assert decoded.type() == msgType; + logger.trace("Received message " + msgType + ": " + decoded); + out.add(decoded); + } + + private Message decode(Message.Type msgType, ByteBuf in) { + switch (msgType) { + case ChunkFetchRequest: + return ChunkFetchRequest.decode(in); + + case ChunkFetchSuccess: + return ChunkFetchSuccess.decode(in); + + case ChunkFetchFailure: + return ChunkFetchFailure.decode(in); + + case RpcRequest: + return RpcRequest.decode(in); + + case RpcResponse: + return RpcResponse.decode(in); + + case RpcFailure: + return RpcFailure.decode(in); + + default: + throw new IllegalArgumentException("Unexpected message type: " + msgType); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java new file mode 100644 index 0000000000000..4cb8becc3ed22 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import java.util.List; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.ChannelHandler; +import io.netty.channel.ChannelHandlerContext; +import io.netty.handler.codec.MessageToMessageEncoder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Encoder used by the server side to encode server-to-client responses. + * This encoder is stateless so it is safe to be shared by multiple threads. + */ +@ChannelHandler.Sharable +public final class MessageEncoder extends MessageToMessageEncoder { + + private final Logger logger = LoggerFactory.getLogger(MessageEncoder.class); + + /*** + * Encodes a Message by invoking its encode() method. For non-data messages, we will add one + * ByteBuf to 'out' containing the total frame length, the message type, and the message itself. + * In the case of a ChunkFetchSuccess, we will also add the ManagedBuffer corresponding to the + * data to 'out', in order to enable zero-copy transfer. + */ + @Override + public void encode(ChannelHandlerContext ctx, Message in, List out) { + Object body = null; + long bodyLength = 0; + + // Only ChunkFetchSuccesses have data besides the header. + // The body is used in order to enable zero-copy transfer for the payload. + if (in instanceof ChunkFetchSuccess) { + ChunkFetchSuccess resp = (ChunkFetchSuccess) in; + try { + bodyLength = resp.buffer.size(); + body = resp.buffer.convertToNetty(); + } catch (Exception e) { + // Re-encode this message as BlockFetchFailure. + logger.error(String.format("Error opening block %s for client %s", + resp.streamChunkId, ctx.channel().remoteAddress()), e); + encode(ctx, new ChunkFetchFailure(resp.streamChunkId, e.getMessage()), out); + return; + } + } + + Message.Type msgType = in.type(); + // All messages have the frame length, message type, and message itself. + int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); + long frameLength = headerLength + bodyLength; + ByteBuf header = ctx.alloc().buffer(headerLength); + header.writeLong(frameLength); + msgType.encode(header); + in.encode(header); + assert header.writableBytes() == 0; + + out.add(header); + if (body != null && bodyLength > 0) { + out.add(body); + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala b/network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java similarity index 77% rename from core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala rename to network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java index 0d7695072a7b1..31b15bb17a327 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/PathResolver.scala +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RequestMessage.java @@ -15,11 +15,11 @@ * limitations under the License. */ -package org.apache.spark.network.netty +package org.apache.spark.network.protocol; -import org.apache.spark.storage.{BlockId, FileSegment} +import org.apache.spark.network.protocol.Message; -trait PathResolver { - /** Get the file segment in which the given block resides. */ - def getBlockLocation(blockId: BlockId): FileSegment +/** Messages from the client to the server. */ +public interface RequestMessage extends Message { + // token interface } diff --git a/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala b/network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java similarity index 75% rename from core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala rename to network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java index e28219dd7745b..6edffd11cf1e2 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/client/BlockClientListener.scala +++ b/network/common/src/main/java/org/apache/spark/network/protocol/ResponseMessage.java @@ -15,15 +15,11 @@ * limitations under the License. */ -package org.apache.spark.network.netty.client +package org.apache.spark.network.protocol; -import java.util.EventListener - - -trait BlockClientListener extends EventListener { - - def onFetchSuccess(blockId: String, data: ReferenceCountedBuffer): Unit - - def onFetchFailure(blockId: String, errorMsg: String): Unit +import org.apache.spark.network.protocol.Message; +/** Messages from the server to the client. */ +public interface ResponseMessage extends Message { + // token interface } diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java new file mode 100644 index 0000000000000..e239d4ffbd29c --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Charsets; +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** Response to {@link RpcRequest} for a failed RPC. */ +public final class RpcFailure implements ResponseMessage { + public final long requestId; + public final String errorString; + + public RpcFailure(long requestId, String errorString) { + this.requestId = requestId; + this.errorString = errorString; + } + + @Override + public Type type() { return Type.RpcFailure; } + + @Override + public int encodedLength() { + return 8 + 4 + errorString.getBytes(Charsets.UTF_8).length; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + byte[] errorBytes = errorString.getBytes(Charsets.UTF_8); + buf.writeInt(errorBytes.length); + buf.writeBytes(errorBytes); + } + + public static RpcFailure decode(ByteBuf buf) { + long requestId = buf.readLong(); + int numErrorStringBytes = buf.readInt(); + byte[] errorBytes = new byte[numErrorStringBytes]; + buf.readBytes(errorBytes); + return new RpcFailure(requestId, new String(errorBytes, Charsets.UTF_8)); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcFailure) { + RpcFailure o = (RpcFailure) other; + return requestId == o.requestId && errorString.equals(o.errorString); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("errorString", errorString) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java new file mode 100644 index 0000000000000..099e934ae018c --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** + * A generic RPC which is handled by a remote {@link org.apache.spark.network.server.RpcHandler}. + * This will correspond to a single + * {@link org.apache.spark.network.protocol.ResponseMessage} (either success or failure). + */ +public final class RpcRequest implements RequestMessage { + /** Used to link an RPC request with its response. */ + public final long requestId; + + /** Serialized message to send to remote RpcHandler. */ + public final byte[] message; + + public RpcRequest(long requestId, byte[] message) { + this.requestId = requestId; + this.message = message; + } + + @Override + public Type type() { return Type.RpcRequest; } + + @Override + public int encodedLength() { + return 8 + 4 + message.length; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + buf.writeInt(message.length); + buf.writeBytes(message); + } + + public static RpcRequest decode(ByteBuf buf) { + long requestId = buf.readLong(); + int messageLen = buf.readInt(); + byte[] message = new byte[messageLen]; + buf.readBytes(message); + return new RpcRequest(requestId, message); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcRequest) { + RpcRequest o = (RpcRequest) other; + return requestId == o.requestId && Arrays.equals(message, o.message); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("message", message) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java new file mode 100644 index 0000000000000..ed479478325b6 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** Response to {@link RpcRequest} for a successful RPC. */ +public final class RpcResponse implements ResponseMessage { + public final long requestId; + public final byte[] response; + + public RpcResponse(long requestId, byte[] response) { + this.requestId = requestId; + this.response = response; + } + + @Override + public Type type() { return Type.RpcResponse; } + + @Override + public int encodedLength() { return 8 + 4 + response.length; } + + @Override + public void encode(ByteBuf buf) { + buf.writeLong(requestId); + buf.writeInt(response.length); + buf.writeBytes(response); + } + + public static RpcResponse decode(ByteBuf buf) { + long requestId = buf.readLong(); + int responseLen = buf.readInt(); + byte[] response = new byte[responseLen]; + buf.readBytes(response); + return new RpcResponse(requestId, response); + } + + @Override + public boolean equals(Object other) { + if (other instanceof RpcResponse) { + RpcResponse o = (RpcResponse) other; + return requestId == o.requestId && Arrays.equals(response, o.response); + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("requestId", requestId) + .add("response", response) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java b/network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java new file mode 100644 index 0000000000000..d46a263884807 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/protocol/StreamChunkId.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +/** +* Encapsulates a request for a particular chunk of a stream. +*/ +public final class StreamChunkId implements Encodable { + public final long streamId; + public final int chunkIndex; + + public StreamChunkId(long streamId, int chunkIndex) { + this.streamId = streamId; + this.chunkIndex = chunkIndex; + } + + @Override + public int encodedLength() { + return 8 + 4; + } + + public void encode(ByteBuf buffer) { + buffer.writeLong(streamId); + buffer.writeInt(chunkIndex); + } + + public static StreamChunkId decode(ByteBuf buffer) { + assert buffer.readableBytes() >= 8 + 4; + long streamId = buffer.readLong(); + int chunkIndex = buffer.readInt(); + return new StreamChunkId(streamId, chunkIndex); + } + + @Override + public int hashCode() { + return Objects.hashCode(streamId, chunkIndex); + } + + @Override + public boolean equals(Object other) { + if (other instanceof StreamChunkId) { + StreamChunkId o = (StreamChunkId) other; + return streamId == o.streamId && chunkIndex == o.chunkIndex; + } + return false; + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamId", streamId) + .add("chunkIndex", chunkIndex) + .toString(); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java new file mode 100644 index 0000000000000..9688705569634 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.util.Iterator; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicLong; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * StreamManager which allows registration of an Iterator, which are individually + * fetched as chunks by the client. + */ +public class DefaultStreamManager extends StreamManager { + private final Logger logger = LoggerFactory.getLogger(DefaultStreamManager.class); + + private final AtomicLong nextStreamId; + private final Map streams; + + /** State of a single stream. */ + private static class StreamState { + final Iterator buffers; + + // Used to keep track of the index of the buffer that the user has retrieved, just to ensure + // that the caller only requests each chunk one at a time, in order. + int curChunk = 0; + + StreamState(Iterator buffers) { + this.buffers = buffers; + } + } + + public DefaultStreamManager() { + // For debugging purposes, start with a random stream id to help identifying different streams. + // This does not need to be globally unique, only unique to this class. + nextStreamId = new AtomicLong((long) new Random().nextInt(Integer.MAX_VALUE) * 1000); + streams = new ConcurrentHashMap(); + } + + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + StreamState state = streams.get(streamId); + if (chunkIndex != state.curChunk) { + throw new IllegalStateException(String.format( + "Received out-of-order chunk index %s (expected %s)", chunkIndex, state.curChunk)); + } else if (!state.buffers.hasNext()) { + throw new IllegalStateException(String.format( + "Requested chunk index beyond end %s", chunkIndex)); + } + state.curChunk += 1; + ManagedBuffer nextChunk = state.buffers.next(); + + if (!state.buffers.hasNext()) { + logger.trace("Removing stream id {}", streamId); + streams.remove(streamId); + } + + return nextChunk; + } + + @Override + public void connectionTerminated(long streamId) { + // Release all remaining buffers. + StreamState state = streams.remove(streamId); + if (state != null && state.buffers != null) { + while (state.buffers.hasNext()) { + state.buffers.next().release(); + } + } + } + + /** + * Registers a stream of ManagedBuffers which are served as individual chunks one at a time to + * callers. Each ManagedBuffer will be release()'d after it is transferred on the wire. If a + * client connection is closed before the iterator is fully drained, then the remaining buffers + * will all be release()'d. + */ + public long registerStream(Iterator buffers) { + long myStreamId = nextStreamId.getAndIncrement(); + streams.put(myStreamId, new StreamState(buffers)); + return myStreamId; + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java b/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java new file mode 100644 index 0000000000000..b80c15106ecbd --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/MessageHandler.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import org.apache.spark.network.protocol.Message; + +/** + * Handles either request or response messages coming off of Netty. A MessageHandler instance + * is associated with a single Netty Channel (though it may have multiple clients on the same + * Channel.) + */ +public abstract class MessageHandler { + /** Handles the receipt of a single message. */ + public abstract void handle(T message); + + /** Invoked when an exception was caught on the Channel. */ + public abstract void exceptionCaught(Throwable cause); + + /** Invoked when the channel this MessageHandler is on has been unregistered. */ + public abstract void channelUnregistered(); +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java new file mode 100644 index 0000000000000..f54a696b8ff79 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; + +/** + * Handler for sendRPC() messages sent by {@link org.apache.spark.network.client.TransportClient}s. + */ +public interface RpcHandler { + /** + * Receive a single RPC message. Any exception thrown while in this method will be sent back to + * the client in string form as a standard RPC failure. + * + * @param client A channel client which enables the handler to make requests back to the sender + * of this RPC. + * @param message The serialized bytes of the RPC. + * @param callback Callback which should be invoked exactly once upon success or failure of the + * RPC. + */ + void receive(TransportClient client, byte[] message, RpcResponseCallback callback); +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java new file mode 100644 index 0000000000000..5a9a14a180c10 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/StreamManager.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import org.apache.spark.network.buffer.ManagedBuffer; + +/** + * The StreamManager is used to fetch individual chunks from a stream. This is used in + * {@link TransportRequestHandler} in order to respond to fetchChunk() requests. Creation of the + * stream is outside the scope of the transport layer, but a given stream is guaranteed to be read + * by only one client connection, meaning that getChunk() for a particular stream will be called + * serially and that once the connection associated with the stream is closed, that stream will + * never be used again. + */ +public abstract class StreamManager { + /** + * Called in response to a fetchChunk() request. The returned buffer will be passed as-is to the + * client. A single stream will be associated with a single TCP connection, so this method + * will not be called in parallel for a particular stream. + * + * Chunks may be requested in any order, and requests may be repeated, but it is not required + * that implementations support this behavior. + * + * The returned ManagedBuffer will be release()'d after being written to the network. + * + * @param streamId id of a stream that has been previously registered with the StreamManager. + * @param chunkIndex 0-indexed chunk of the stream that's requested + */ + public abstract ManagedBuffer getChunk(long streamId, int chunkIndex); + + /** + * Indicates that the TCP connection that was tied to the given stream has been terminated. After + * this occurs, we are guaranteed not to read from the stream again, so any state can be cleaned + * up. + */ + public void connectionTerminated(long streamId) { } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java new file mode 100644 index 0000000000000..e491367fa4528 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportChannelHandler.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.SimpleChannelInboundHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.Message; +import org.apache.spark.network.protocol.RequestMessage; +import org.apache.spark.network.protocol.ResponseMessage; +import org.apache.spark.network.util.NettyUtils; + +/** + * The single Transport-level Channel handler which is used for delegating requests to the + * {@link TransportRequestHandler} and responses to the {@link TransportResponseHandler}. + * + * All channels created in the transport layer are bidirectional. When the Client initiates a Netty + * Channel with a RequestMessage (which gets handled by the Server's RequestHandler), the Server + * will produce a ResponseMessage (handled by the Client's ResponseHandler). However, the Server + * also gets a handle on the same Channel, so it may then begin to send RequestMessages to the + * Client. + * This means that the Client also needs a RequestHandler and the Server needs a ResponseHandler, + * for the Client's responses to the Server's requests. + */ +public class TransportChannelHandler extends SimpleChannelInboundHandler { + private final Logger logger = LoggerFactory.getLogger(TransportChannelHandler.class); + + private final TransportClient client; + private final TransportResponseHandler responseHandler; + private final TransportRequestHandler requestHandler; + + public TransportChannelHandler( + TransportClient client, + TransportResponseHandler responseHandler, + TransportRequestHandler requestHandler) { + this.client = client; + this.responseHandler = responseHandler; + this.requestHandler = requestHandler; + } + + public TransportClient getClient() { + return client; + } + + @Override + public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { + logger.warn("Exception in connection from " + NettyUtils.getRemoteAddress(ctx.channel()), + cause); + requestHandler.exceptionCaught(cause); + responseHandler.exceptionCaught(cause); + ctx.close(); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + try { + requestHandler.channelUnregistered(); + } catch (RuntimeException e) { + logger.error("Exception from request handler while unregistering channel", e); + } + try { + responseHandler.channelUnregistered(); + } catch (RuntimeException e) { + logger.error("Exception from response handler while unregistering channel", e); + } + super.channelUnregistered(ctx); + } + + @Override + public void channelRead0(ChannelHandlerContext ctx, Message request) { + if (request instanceof RequestMessage) { + requestHandler.handle((RequestMessage) request); + } else { + responseHandler.handle((ResponseMessage) request); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java new file mode 100644 index 0000000000000..352f865935b11 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.util.Set; + +import com.google.common.base.Throwables; +import com.google.common.collect.Sets; +import io.netty.channel.Channel; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelFutureListener; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.protocol.Encodable; +import org.apache.spark.network.protocol.RequestMessage; +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.util.NettyUtils; + +/** + * A handler that processes requests from clients and writes chunk data back. Each handler is + * attached to a single Netty channel, and keeps track of which streams have been fetched via this + * channel, in order to clean them up if the channel is terminated (see #channelUnregistered). + * + * The messages should have been processed by the pipeline setup by {@link TransportServer}. + */ +public class TransportRequestHandler extends MessageHandler { + private final Logger logger = LoggerFactory.getLogger(TransportRequestHandler.class); + + /** The Netty channel that this handler is associated with. */ + private final Channel channel; + + /** Client on the same channel allowing us to talk back to the requester. */ + private final TransportClient reverseClient; + + /** Returns each chunk part of a stream. */ + private final StreamManager streamManager; + + /** Handles all RPC messages. */ + private final RpcHandler rpcHandler; + + /** List of all stream ids that have been read on this handler, used for cleanup. */ + private final Set streamIds; + + public TransportRequestHandler( + Channel channel, + TransportClient reverseClient, + StreamManager streamManager, + RpcHandler rpcHandler) { + this.channel = channel; + this.reverseClient = reverseClient; + this.streamManager = streamManager; + this.rpcHandler = rpcHandler; + this.streamIds = Sets.newHashSet(); + } + + @Override + public void exceptionCaught(Throwable cause) { + } + + @Override + public void channelUnregistered() { + // Inform the StreamManager that these streams will no longer be read from. + for (long streamId : streamIds) { + streamManager.connectionTerminated(streamId); + } + } + + @Override + public void handle(RequestMessage request) { + if (request instanceof ChunkFetchRequest) { + processFetchRequest((ChunkFetchRequest) request); + } else if (request instanceof RpcRequest) { + processRpcRequest((RpcRequest) request); + } else { + throw new IllegalArgumentException("Unknown request type: " + request); + } + } + + private void processFetchRequest(final ChunkFetchRequest req) { + final String client = NettyUtils.getRemoteAddress(channel); + streamIds.add(req.streamChunkId.streamId); + + logger.trace("Received req from {} to fetch block {}", client, req.streamChunkId); + + ManagedBuffer buf; + try { + buf = streamManager.getChunk(req.streamChunkId.streamId, req.streamChunkId.chunkIndex); + } catch (Exception e) { + logger.error(String.format( + "Error opening block %s for request from %s", req.streamChunkId, client), e); + respond(new ChunkFetchFailure(req.streamChunkId, Throwables.getStackTraceAsString(e))); + return; + } + + respond(new ChunkFetchSuccess(req.streamChunkId, buf)); + } + + private void processRpcRequest(final RpcRequest req) { + try { + rpcHandler.receive(reverseClient, req.message, new RpcResponseCallback() { + @Override + public void onSuccess(byte[] response) { + respond(new RpcResponse(req.requestId, response)); + } + + @Override + public void onFailure(Throwable e) { + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + }); + } catch (Exception e) { + logger.error("Error while invoking RpcHandler#receive() on RPC id " + req.requestId, e); + respond(new RpcFailure(req.requestId, Throwables.getStackTraceAsString(e))); + } + } + + /** + * Responds to a single message with some Encodable object. If a failure occurs while sending, + * it will be logged and the channel closed. + */ + private void respond(final Encodable result) { + final String remoteAddress = channel.remoteAddress().toString(); + channel.writeAndFlush(result).addListener( + new ChannelFutureListener() { + @Override + public void operationComplete(ChannelFuture future) throws Exception { + if (future.isSuccess()) { + logger.trace(String.format("Sent result %s to client %s", result, remoteAddress)); + } else { + logger.error(String.format("Error sending result %s to %s; closing connection", + result, remoteAddress), future.cause()); + channel.close(); + } + } + } + ); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java new file mode 100644 index 0000000000000..243070750d6e7 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.server; + +import java.io.Closeable; +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; + +import io.netty.bootstrap.ServerBootstrap; +import io.netty.buffer.PooledByteBufAllocator; +import io.netty.channel.ChannelFuture; +import io.netty.channel.ChannelInitializer; +import io.netty.channel.ChannelOption; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.socket.SocketChannel; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.util.IOMode; +import org.apache.spark.network.util.NettyUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Server for the efficient, low-level streaming service. + */ +public class TransportServer implements Closeable { + private final Logger logger = LoggerFactory.getLogger(TransportServer.class); + + private final TransportContext context; + private final TransportConf conf; + + private ServerBootstrap bootstrap; + private ChannelFuture channelFuture; + private int port = -1; + + public TransportServer(TransportContext context) { + this.context = context; + this.conf = context.getConf(); + + init(); + } + + public int getPort() { + if (port == -1) { + throw new IllegalStateException("Server not initialized"); + } + return port; + } + + private void init() { + + IOMode ioMode = IOMode.valueOf(conf.ioMode()); + EventLoopGroup bossGroup = + NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); + EventLoopGroup workerGroup = bossGroup; + + bootstrap = new ServerBootstrap() + .group(bossGroup, workerGroup) + .channel(NettyUtils.getServerChannelClass(ioMode)) + .option(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT) + .childOption(ChannelOption.ALLOCATOR, PooledByteBufAllocator.DEFAULT); + + if (conf.backLog() > 0) { + bootstrap.option(ChannelOption.SO_BACKLOG, conf.backLog()); + } + + if (conf.receiveBuf() > 0) { + bootstrap.childOption(ChannelOption.SO_RCVBUF, conf.receiveBuf()); + } + + if (conf.sendBuf() > 0) { + bootstrap.childOption(ChannelOption.SO_SNDBUF, conf.sendBuf()); + } + + bootstrap.childHandler(new ChannelInitializer() { + @Override + protected void initChannel(SocketChannel ch) throws Exception { + context.initializePipeline(ch); + } + }); + + channelFuture = bootstrap.bind(new InetSocketAddress(conf.serverPort())); + channelFuture.syncUninterruptibly(); + + port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort(); + logger.debug("Shuffle server started on port :" + port); + } + + @Override + public void close() { + if (channelFuture != null) { + // close is a local operation and should finish with milliseconds; timeout just to be safe + channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS); + channelFuture = null; + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully(); + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully(); + } + bootstrap = null; + } + +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java b/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java new file mode 100644 index 0000000000000..d944d9da1c7f8 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/ConfigProvider.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.util.NoSuchElementException; + +/** + * Provides a mechanism for constructing a {@link TransportConf} using some sort of configuration. + */ +public abstract class ConfigProvider { + /** Obtains the value of the given config, throws NoSuchElementException if it doesn't exist. */ + public abstract String get(String name); + + public String get(String name, String defaultValue) { + try { + return get(name); + } catch (NoSuchElementException e) { + return defaultValue; + } + } + + public int getInt(String name, int defaultValue) { + return Integer.parseInt(get(name, Integer.toString(defaultValue))); + } + + public long getLong(String name, long defaultValue) { + return Long.parseLong(get(name, Long.toString(defaultValue))); + } + + public double getDouble(String name, double defaultValue) { + return Double.parseDouble(get(name, Double.toString(defaultValue))); + } + + public boolean getBoolean(String name, boolean defaultValue) { + return Boolean.parseBoolean(get(name, Boolean.toString(defaultValue))); + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/IOMode.java b/network/common/src/main/java/org/apache/spark/network/util/IOMode.java new file mode 100644 index 0000000000000..6b208d95bbfbc --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/IOMode.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +/** + * Selector for which form of low-level IO we should use. + * NIO is always available, while EPOLL is only available on Linux. + * AUTO is used to select EPOLL if it's available, or NIO otherwise. + */ +public enum IOMode { + NIO, EPOLL +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java new file mode 100644 index 0000000000000..32ba3f5b07f7a --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.io.Closeable; +import java.io.IOException; + +import com.google.common.io.Closeables; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JavaUtils { + private static final Logger logger = LoggerFactory.getLogger(JavaUtils.class); + + /** Closes the given object, ignoring IOExceptions. */ + public static void closeQuietly(Closeable closeable) { + try { + closeable.close(); + } catch (IOException e) { + logger.error("IOException should not have been thrown.", e); + } + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java new file mode 100644 index 0000000000000..b1872341198e0 --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +import java.util.concurrent.ThreadFactory; + +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.netty.channel.Channel; +import io.netty.channel.EventLoopGroup; +import io.netty.channel.ServerChannel; +import io.netty.channel.epoll.Epoll; +import io.netty.channel.epoll.EpollEventLoopGroup; +import io.netty.channel.epoll.EpollServerSocketChannel; +import io.netty.channel.epoll.EpollSocketChannel; +import io.netty.channel.nio.NioEventLoopGroup; +import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.socket.nio.NioSocketChannel; +import io.netty.handler.codec.ByteToMessageDecoder; +import io.netty.handler.codec.LengthFieldBasedFrameDecoder; + +/** + * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO. + */ +public class NettyUtils { + /** Creates a Netty EventLoopGroup based on the IOMode. */ + public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, String threadPrefix) { + + ThreadFactory threadFactory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(threadPrefix + "-%d") + .build(); + + switch (mode) { + case NIO: + return new NioEventLoopGroup(numThreads, threadFactory); + case EPOLL: + return new EpollEventLoopGroup(numThreads, threadFactory); + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** Returns the correct (client) SocketChannel class based on IOMode. */ + public static Class getClientChannelClass(IOMode mode) { + switch (mode) { + case NIO: + return NioSocketChannel.class; + case EPOLL: + return EpollSocketChannel.class; + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** Returns the correct ServerSocketChannel class based on IOMode. */ + public static Class getServerChannelClass(IOMode mode) { + switch (mode) { + case NIO: + return NioServerSocketChannel.class; + case EPOLL: + return EpollServerSocketChannel.class; + default: + throw new IllegalArgumentException("Unknown io mode: " + mode); + } + } + + /** + * Creates a LengthFieldBasedFrameDecoder where the first 8 bytes are the length of the frame. + * This is used before all decoders. + */ + public static ByteToMessageDecoder createFrameDecoder() { + // maxFrameLength = 2G + // lengthFieldOffset = 0 + // lengthFieldLength = 8 + // lengthAdjustment = -8, i.e. exclude the 8 byte length itself + // initialBytesToStrip = 8, i.e. strip out the length field itself + return new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 8, -8, 8); + } + + /** Returns the remote address on the channel or "" if none exists. */ + public static String getRemoteAddress(Channel channel) { + if (channel != null && channel.remoteAddress() != null) { + return channel.remoteAddress().toString(); + } + return ""; + } +} diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java new file mode 100644 index 0000000000000..80f65d98032da --- /dev/null +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.util; + +/** + * A central location that tracks all the settings we expose to users. + */ +public class TransportConf { + private final ConfigProvider conf; + + public TransportConf(ConfigProvider conf) { + this.conf = conf; + } + + /** Port the server listens on. Default to a random port. */ + public int serverPort() { return conf.getInt("spark.shuffle.io.port", 0); } + + /** IO mode: nio or epoll */ + public String ioMode() { return conf.get("spark.shuffle.io.mode", "NIO").toUpperCase(); } + + /** Connect timeout in secs. Default 120 secs. */ + public int connectionTimeoutMs() { + return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; + } + + /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ + public int backLog() { return conf.getInt("spark.shuffle.io.backLog", -1); } + + /** Number of threads used in the server thread pool. Default to 0, which is 2x#cores. */ + public int serverThreads() { return conf.getInt("spark.shuffle.io.serverThreads", 0); } + + /** Number of threads used in the client thread pool. Default to 0, which is 2x#cores. */ + public int clientThreads() { return conf.getInt("spark.shuffle.io.clientThreads", 0); } + + /** + * Receive buffer size (SO_RCVBUF). + * Note: the optimal size for receive buffer and send buffer should be + * latency * network_bandwidth. + * Assuming latency = 1ms, network_bandwidth = 10Gbps + * buffer size should be ~ 1.25MB + */ + public int receiveBuf() { return conf.getInt("spark.shuffle.io.receiveBuffer", -1); } + + /** Send buffer size (SO_SNDBUF). */ + public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } +} diff --git a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java new file mode 100644 index 0000000000000..738dca9b6a9ee --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.io.File; +import java.io.RandomAccessFile; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.TransportConf; + +public class ChunkFetchIntegrationSuite { + static final long STREAM_ID = 1; + static final int BUFFER_CHUNK_INDEX = 0; + static final int FILE_CHUNK_INDEX = 1; + + static TransportServer server; + static TransportClientFactory clientFactory; + static StreamManager streamManager; + static File testFile; + + static ManagedBuffer bufferChunk; + static ManagedBuffer fileChunk; + + @BeforeClass + public static void setUp() throws Exception { + int bufSize = 100000; + final ByteBuffer buf = ByteBuffer.allocate(bufSize); + for (int i = 0; i < bufSize; i ++) { + buf.put((byte) i); + } + buf.flip(); + bufferChunk = new NioManagedBuffer(buf); + + testFile = File.createTempFile("shuffle-test-file", "txt"); + testFile.deleteOnExit(); + RandomAccessFile fp = new RandomAccessFile(testFile, "rw"); + byte[] fileContent = new byte[1024]; + new Random().nextBytes(fileContent); + fp.write(fileContent); + fp.close(); + fileChunk = new FileSegmentManagedBuffer(testFile, 10, testFile.length() - 25); + + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + streamManager = new StreamManager() { + @Override + public ManagedBuffer getChunk(long streamId, int chunkIndex) { + assertEquals(STREAM_ID, streamId); + if (chunkIndex == BUFFER_CHUNK_INDEX) { + return new NioManagedBuffer(buf); + } else if (chunkIndex == FILE_CHUNK_INDEX) { + return new FileSegmentManagedBuffer(testFile, 10, testFile.length() - 25); + } else { + throw new IllegalArgumentException("Invalid chunk index: " + chunkIndex); + } + } + }; + TransportContext context = new TransportContext(conf, streamManager, new NoOpRpcHandler()); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @AfterClass + public static void tearDown() { + server.close(); + clientFactory.close(); + testFile.delete(); + } + + class FetchResult { + public Set successChunks; + public Set failedChunks; + public List buffers; + + public void releaseBuffers() { + for (ManagedBuffer buffer : buffers) { + buffer.release(); + } + } + } + + private FetchResult fetchChunks(List chunkIndices) throws Exception { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); + + final FetchResult res = new FetchResult(); + res.successChunks = Collections.synchronizedSet(new HashSet()); + res.failedChunks = Collections.synchronizedSet(new HashSet()); + res.buffers = Collections.synchronizedList(new LinkedList()); + + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + buffer.retain(); + res.successChunks.add(chunkIndex); + res.buffers.add(buffer); + sem.release(); + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + res.failedChunks.add(chunkIndex); + sem.release(); + } + }; + + for (int chunkIndex : chunkIndices) { + client.fetchChunk(STREAM_ID, chunkIndex, callback); + } + if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + client.close(); + return res; + } + + @Test + public void fetchBufferChunk() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX)); + assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + res.releaseBuffers(); + } + + @Test + public void fetchFileChunk() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(FILE_CHUNK_INDEX)); + assertEquals(res.successChunks, Sets.newHashSet(FILE_CHUNK_INDEX)); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(res.buffers, Lists.newArrayList(fileChunk)); + res.releaseBuffers(); + } + + @Test + public void fetchNonExistentChunk() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(12345)); + assertTrue(res.successChunks.isEmpty()); + assertEquals(res.failedChunks, Sets.newHashSet(12345)); + assertTrue(res.buffers.isEmpty()); + } + + @Test + public void fetchBothChunks() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX, FILE_CHUNK_INDEX)); + assertTrue(res.failedChunks.isEmpty()); + assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk, fileChunk)); + res.releaseBuffers(); + } + + @Test + public void fetchChunkAndNonExistent() throws Exception { + FetchResult res = fetchChunks(Lists.newArrayList(BUFFER_CHUNK_INDEX, 12345)); + assertEquals(res.successChunks, Sets.newHashSet(BUFFER_CHUNK_INDEX)); + assertEquals(res.failedChunks, Sets.newHashSet(12345)); + assertBufferListsEqual(res.buffers, Lists.newArrayList(bufferChunk)); + res.releaseBuffers(); + } + + private void assertBufferListsEqual(List list0, List list1) + throws Exception { + assertEquals(list0.size(), list1.size()); + for (int i = 0; i < list0.size(); i ++) { + assertBuffersEqual(list0.get(i), list1.get(i)); + } + } + + private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception { + ByteBuffer nio0 = buffer0.nioByteBuffer(); + ByteBuffer nio1 = buffer1.nioByteBuffer(); + + int len = nio0.remaining(); + assertEquals(nio0.remaining(), nio1.remaining()); + for (int i = 0; i < len; i ++) { + assertEquals(nio0.get(), nio1.get()); + } + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java b/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java new file mode 100644 index 0000000000000..7aa37efc582e4 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java @@ -0,0 +1,28 @@ +package org.apache.spark.network;/* + * 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. + */ + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.RpcHandler; + +/** Test RpcHandler which always returns a zero-sized success. */ +public class NoOpRpcHandler implements RpcHandler { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + callback.onSuccess(new byte[0]); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java new file mode 100644 index 0000000000000..43dc0cf8c7194 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/ProtocolSuite.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import io.netty.channel.embedded.EmbeddedChannel; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +import org.apache.spark.network.protocol.Message; +import org.apache.spark.network.protocol.StreamChunkId; +import org.apache.spark.network.protocol.ChunkFetchRequest; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.RpcRequest; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.MessageDecoder; +import org.apache.spark.network.protocol.MessageEncoder; +import org.apache.spark.network.util.NettyUtils; + +public class ProtocolSuite { + private void testServerToClient(Message msg) { + EmbeddedChannel serverChannel = new EmbeddedChannel(new MessageEncoder()); + serverChannel.writeOutbound(msg); + + EmbeddedChannel clientChannel = new EmbeddedChannel( + NettyUtils.createFrameDecoder(), new MessageDecoder()); + + while (!serverChannel.outboundMessages().isEmpty()) { + clientChannel.writeInbound(serverChannel.readOutbound()); + } + + assertEquals(1, clientChannel.inboundMessages().size()); + assertEquals(msg, clientChannel.readInbound()); + } + + private void testClientToServer(Message msg) { + EmbeddedChannel clientChannel = new EmbeddedChannel(new MessageEncoder()); + clientChannel.writeOutbound(msg); + + EmbeddedChannel serverChannel = new EmbeddedChannel( + NettyUtils.createFrameDecoder(), new MessageDecoder()); + + while (!clientChannel.outboundMessages().isEmpty()) { + serverChannel.writeInbound(clientChannel.readOutbound()); + } + + assertEquals(1, serverChannel.inboundMessages().size()); + assertEquals(msg, serverChannel.readInbound()); + } + + @Test + public void requests() { + testClientToServer(new ChunkFetchRequest(new StreamChunkId(1, 2))); + testClientToServer(new RpcRequest(12345, new byte[0])); + testClientToServer(new RpcRequest(12345, new byte[100])); + } + + @Test + public void responses() { + testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(10))); + testServerToClient(new ChunkFetchSuccess(new StreamChunkId(1, 2), new TestManagedBuffer(0))); + testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "this is an error")); + testServerToClient(new ChunkFetchFailure(new StreamChunkId(1, 2), "")); + testServerToClient(new RpcResponse(12345, new byte[0])); + testServerToClient(new RpcResponse(12345, new byte[1000])); + testServerToClient(new RpcFailure(0, "this is an error")); + testServerToClient(new RpcFailure(0, "")); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java new file mode 100644 index 0000000000000..9f216dd2d722d --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import com.google.common.base.Charsets; +import com.google.common.collect.Sets; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.DefaultStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.TransportConf; + +public class RpcIntegrationSuite { + static TransportServer server; + static TransportClientFactory clientFactory; + static RpcHandler rpcHandler; + + @BeforeClass + public static void setUp() throws Exception { + TransportConf conf = new TransportConf(new SystemPropertyConfigProvider()); + rpcHandler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + String msg = new String(message, Charsets.UTF_8); + String[] parts = msg.split("/"); + if (parts[0].equals("hello")) { + callback.onSuccess(("Hello, " + parts[1] + "!").getBytes(Charsets.UTF_8)); + } else if (parts[0].equals("return error")) { + callback.onFailure(new RuntimeException("Returned: " + parts[1])); + } else if (parts[0].equals("throw error")) { + throw new RuntimeException("Thrown: " + parts[1]); + } + } + }; + TransportContext context = new TransportContext(conf, new DefaultStreamManager(), rpcHandler); + server = context.createServer(); + clientFactory = context.createClientFactory(); + } + + @AfterClass + public static void tearDown() { + server.close(); + clientFactory.close(); + } + + class RpcResult { + public Set successMessages; + public Set errorMessages; + } + + private RpcResult sendRPC(String ... commands) throws Exception { + TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + final Semaphore sem = new Semaphore(0); + + final RpcResult res = new RpcResult(); + res.successMessages = Collections.synchronizedSet(new HashSet()); + res.errorMessages = Collections.synchronizedSet(new HashSet()); + + RpcResponseCallback callback = new RpcResponseCallback() { + @Override + public void onSuccess(byte[] message) { + res.successMessages.add(new String(message, Charsets.UTF_8)); + sem.release(); + } + + @Override + public void onFailure(Throwable e) { + res.errorMessages.add(e.getMessage()); + sem.release(); + } + }; + + for (String command : commands) { + client.sendRpc(command.getBytes(Charsets.UTF_8), callback); + } + + if (!sem.tryAcquire(commands.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + client.close(); + return res; + } + + @Test + public void singleRPC() throws Exception { + RpcResult res = sendRPC("hello/Aaron"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Aaron!")); + assertTrue(res.errorMessages.isEmpty()); + } + + @Test + public void doubleRPC() throws Exception { + RpcResult res = sendRPC("hello/Aaron", "hello/Reynold"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Aaron!", "Hello, Reynold!")); + assertTrue(res.errorMessages.isEmpty()); + } + + @Test + public void returnErrorRPC() throws Exception { + RpcResult res = sendRPC("return error/OK"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK")); + } + + @Test + public void throwErrorRPC() throws Exception { + RpcResult res = sendRPC("throw error/uh-oh"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: uh-oh")); + } + + @Test + public void doubleTrouble() throws Exception { + RpcResult res = sendRPC("return error/OK", "throw error/uh-oh"); + assertTrue(res.successMessages.isEmpty()); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Returned: OK", "Thrown: uh-oh")); + } + + @Test + public void sendSuccessAndFailure() throws Exception { + RpcResult res = sendRPC("hello/Bob", "throw error/the", "hello/Builder", "return error/!"); + assertEquals(res.successMessages, Sets.newHashSet("Hello, Bob!", "Hello, Builder!")); + assertErrorsContain(res.errorMessages, Sets.newHashSet("Thrown: the", "Returned: !")); + } + + private void assertErrorsContain(Set errors, Set contains) { + assertEquals(contains.size(), errors.size()); + + Set remainingErrors = Sets.newHashSet(errors); + for (String contain : contains) { + Iterator it = remainingErrors.iterator(); + boolean foundMatch = false; + while (it.hasNext()) { + if (it.next().contains(contain)) { + it.remove(); + foundMatch = true; + break; + } + } + assertTrue("Could not find error containing " + contain + "; errors: " + errors, foundMatch); + } + + assertTrue(remainingErrors.isEmpty()); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java b/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java new file mode 100644 index 0000000000000..f4e0a2426a3d2 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.util.NoSuchElementException; + +import org.apache.spark.network.util.ConfigProvider; + +/** Uses System properties to obtain config values. */ +public class SystemPropertyConfigProvider extends ConfigProvider { + @Override + public String get(String name) { + String value = System.getProperty(name); + if (value == null) { + throw new NoSuchElementException(name); + } + return value; + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java new file mode 100644 index 0000000000000..38113a918f795 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TestManagedBuffer.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import com.google.common.base.Preconditions; +import io.netty.buffer.Unpooled; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; + +/** + * A ManagedBuffer implementation that contains 0, 1, 2, 3, ..., (len-1). + * + * Used for testing. + */ +public class TestManagedBuffer extends ManagedBuffer { + + private final int len; + private NettyManagedBuffer underlying; + + public TestManagedBuffer(int len) { + Preconditions.checkArgument(len <= Byte.MAX_VALUE); + this.len = len; + byte[] byteArray = new byte[len]; + for (int i = 0; i < len; i ++) { + byteArray[i] = (byte) i; + } + this.underlying = new NettyManagedBuffer(Unpooled.wrappedBuffer(byteArray)); + } + + + @Override + public long size() { + return underlying.size(); + } + + @Override + public ByteBuffer nioByteBuffer() throws IOException { + return underlying.nioByteBuffer(); + } + + @Override + public InputStream createInputStream() throws IOException { + return underlying.createInputStream(); + } + + @Override + public ManagedBuffer retain() { + underlying.retain(); + return this; + } + + @Override + public ManagedBuffer release() { + underlying.release(); + return this; + } + + @Override + public Object convertToNetty() throws IOException { + return underlying.convertToNetty(); + } + + @Override + public boolean equals(Object other) { + if (other instanceof ManagedBuffer) { + try { + ByteBuffer nioBuf = ((ManagedBuffer) other).nioByteBuffer(); + if (nioBuf.remaining() != len) { + return false; + } else { + for (int i = 0; i < len; i ++) { + if (nioBuf.get() != i) { + return false; + } + } + return true; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return false; + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TestUtils.java b/network/common/src/test/java/org/apache/spark/network/TestUtils.java new file mode 100644 index 0000000000000..56a2b805f154c --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TestUtils.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.net.InetAddress; + +public class TestUtils { + public static String getLocalHost() { + try { + return InetAddress.getLocalHost().getHostAddress(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java new file mode 100644 index 0000000000000..3ef964616f0c5 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import java.util.concurrent.TimeoutException; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.DefaultStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; + +public class TransportClientFactorySuite { + private TransportConf conf; + private TransportContext context; + private TransportServer server1; + private TransportServer server2; + + @Before + public void setUp() { + conf = new TransportConf(new SystemPropertyConfigProvider()); + StreamManager streamManager = new DefaultStreamManager(); + RpcHandler rpcHandler = new NoOpRpcHandler(); + context = new TransportContext(conf, streamManager, rpcHandler); + server1 = context.createServer(); + server2 = context.createServer(); + } + + @After + public void tearDown() { + JavaUtils.closeQuietly(server1); + JavaUtils.closeQuietly(server2); + } + + @Test + public void createAndReuseBlockClients() throws TimeoutException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c3 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c3.isActive()); + assertTrue(c1 == c2); + assertTrue(c1 != c3); + factory.close(); + } + + @Test + public void neverReturnInactiveClients() throws Exception { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + c1.close(); + + long start = System.currentTimeMillis(); + while (c1.isActive() && (System.currentTimeMillis() - start) < 3000) { + Thread.sleep(10); + } + assertFalse(c1.isActive()); + + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertFalse(c1 == c2); + assertTrue(c2.isActive()); + factory.close(); + } + + @Test + public void closeBlockClientsWithFactory() throws TimeoutException { + TransportClientFactory factory = context.createClientFactory(); + TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + assertTrue(c1.isActive()); + assertTrue(c2.isActive()); + factory.close(); + assertFalse(c1.isActive()); + assertFalse(c2.isActive()); + } +} diff --git a/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java b/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java new file mode 100644 index 0000000000000..17a03ebe88a93 --- /dev/null +++ b/network/common/src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network; + +import io.netty.channel.local.LocalChannel; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportResponseHandler; +import org.apache.spark.network.protocol.ChunkFetchFailure; +import org.apache.spark.network.protocol.ChunkFetchSuccess; +import org.apache.spark.network.protocol.RpcFailure; +import org.apache.spark.network.protocol.RpcResponse; +import org.apache.spark.network.protocol.StreamChunkId; + +public class TransportResponseHandlerSuite { + @Test + public void handleSuccessfulFetch() { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(streamChunkId, new TestManagedBuffer(123))); + verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedFetch() { + StreamChunkId streamChunkId = new StreamChunkId(1, 0); + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(streamChunkId, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchFailure(streamChunkId, "some error msg")); + verify(callback, times(1)).onFailure(eq(0), (Throwable) any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void clearAllOutstandingRequests() { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + ChunkReceivedCallback callback = mock(ChunkReceivedCallback.class); + handler.addFetchRequest(new StreamChunkId(1, 0), callback); + handler.addFetchRequest(new StreamChunkId(1, 1), callback); + handler.addFetchRequest(new StreamChunkId(1, 2), callback); + assertEquals(3, handler.numOutstandingRequests()); + + handler.handle(new ChunkFetchSuccess(new StreamChunkId(1, 0), new TestManagedBuffer(12))); + handler.exceptionCaught(new Exception("duh duh duhhhh")); + + // should fail both b2 and b3 + verify(callback, times(1)).onSuccess(eq(0), (ManagedBuffer) any()); + verify(callback, times(1)).onFailure(eq(1), (Throwable) any()); + verify(callback, times(1)).onFailure(eq(2), (Throwable) any()); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleSuccessfulRPC() { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcResponse(54321, new byte[7])); // should be ignored + assertEquals(1, handler.numOutstandingRequests()); + + byte[] arr = new byte[10]; + handler.handle(new RpcResponse(12345, arr)); + verify(callback, times(1)).onSuccess(eq(arr)); + assertEquals(0, handler.numOutstandingRequests()); + } + + @Test + public void handleFailedRPC() { + TransportResponseHandler handler = new TransportResponseHandler(new LocalChannel()); + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.addRpcRequest(12345, callback); + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(54321, "uh-oh!")); // should be ignored + assertEquals(1, handler.numOutstandingRequests()); + + handler.handle(new RpcFailure(12345, "oh no")); + verify(callback, times(1)).onFailure((Throwable) any()); + assertEquals(0, handler.numOutstandingRequests()); + } +} diff --git a/pom.xml b/pom.xml index abcb97108c5d9..e4c92470fc03e 100644 --- a/pom.xml +++ b/pom.xml @@ -91,6 +91,7 @@ graphx mllib tools + network/common streaming sql/catalyst sql/core diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 95152b58e287e..adbdc5d1da3c1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -51,6 +51,11 @@ object MimaExcludes { // MapStatus should be private[spark] ProblemFilters.exclude[IncompatibleTemplateDefProblem]( "org.apache.spark.scheduler.MapStatus"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.network.netty.PathResolver"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.network.netty.client.BlockClientListener"), + // TaskContext was promoted to Abstract class ProblemFilters.exclude[AbstractClassProblem]( "org.apache.spark.TaskContext"), From 353546766384b1e80fc8cc75c532d8d1821012b4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 29 Oct 2014 12:10:58 -0700 Subject: [PATCH 1129/1492] [SPARK-4003] [SQL] add 3 types for java SQL context In JavaSqlContext, we need to let java program use big decimal, timestamp, date types. Author: Daoyuan Wang Closes #2850 from adrian-wang/javacontext and squashes the following commits: 4c4292c [Daoyuan Wang] change underlying type of JavaSchemaRDD as scala bb0508f [Daoyuan Wang] add test cases 3c58b0d [Daoyuan Wang] add 3 types for java SQL context --- .../spark/sql/api/java/JavaSQLContext.scala | 11 ++++- .../sql/types/util/DataTypeConversions.scala | 12 ++++++ .../spark/sql/api/java/JavaSQLSuite.scala | 41 +++++++++++++++++-- 3 files changed, 59 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index f8171c3be3207..082ae03eef03f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} import org.apache.spark.sql.parquet.ParquetRelation @@ -97,7 +98,9 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod) iter.map { row => - new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow + new GenericRow( + extractors.map(e => DataTypeConversions.convertJavaToCatalyst(e.invoke(row))).toArray[Any] + ): ScalaRow } } new JavaSchemaRDD(sqlContext, LogicalRDD(schema, rowRdd)(sqlContext)) @@ -226,6 +229,12 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { (org.apache.spark.sql.FloatType, true) case c: Class[_] if c == classOf[java.lang.Boolean] => (org.apache.spark.sql.BooleanType, true) + case c: Class[_] if c == classOf[java.math.BigDecimal] => + (org.apache.spark.sql.DecimalType, true) + case c: Class[_] if c == classOf[java.sql.Date] => + (org.apache.spark.sql.DateType, true) + case c: Class[_] if c == classOf[java.sql.Timestamp] => + (org.apache.spark.sql.TimestampType, true) } AttributeReference(property.getName, dataType, nullable)() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index e44cb08309523..609f7db562a31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -110,4 +110,16 @@ protected[sql] object DataTypeConversions { case structType: org.apache.spark.sql.api.java.StructType => StructType(structType.getFields.map(asScalaStructField)) } + + /** Converts Java objects to catalyst rows / types */ + def convertJavaToCatalyst(a: Any): Any = a match { + case d: java.math.BigDecimal => BigDecimal(d) + case other => other + } + + /** Converts Java objects to catalyst rows / types */ + def convertCatalystToJava(a: Any): Any = a match { + case d: scala.math.BigDecimal => d.underlying() + case other => other + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index 203ff847e94cc..d83f3e23a9468 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -45,6 +45,9 @@ class AllTypesBean extends Serializable { @BeanProperty var shortField: java.lang.Short = _ @BeanProperty var byteField: java.lang.Byte = _ @BeanProperty var booleanField: java.lang.Boolean = _ + @BeanProperty var dateField: java.sql.Date = _ + @BeanProperty var timestampField: java.sql.Timestamp = _ + @BeanProperty var bigDecimalField: java.math.BigDecimal = _ } class JavaSQLSuite extends FunSuite { @@ -73,6 +76,9 @@ class JavaSQLSuite extends FunSuite { bean.setShortField(0.toShort) bean.setByteField(0.toByte) bean.setBooleanField(false) + bean.setDateField(java.sql.Date.valueOf("2014-10-10")) + bean.setTimestampField(java.sql.Timestamp.valueOf("2014-10-10 00:00:00.0")) + bean.setBigDecimalField(new java.math.BigDecimal(0)) val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) @@ -82,10 +88,34 @@ class JavaSQLSuite extends FunSuite { javaSqlCtx.sql( """ |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, - | booleanField + | booleanField, dateField, timestampField, bigDecimalField |FROM allTypes """.stripMargin).collect.head.row === - Seq("", 0, 0L, 0F, 0.0, 0.toShort, 0.toByte, false)) + Seq("", 0, 0L, 0F, 0.0, 0.toShort, 0.toByte, false, java.sql.Date.valueOf("2014-10-10"), + java.sql.Timestamp.valueOf("2014-10-10 00:00:00.0"), scala.math.BigDecimal(0))) + } + + test("decimal types in JavaBeans") { + val bean = new AllTypesBean + bean.setStringField("") + bean.setIntField(0) + bean.setLongField(0) + bean.setFloatField(0.0F) + bean.setDoubleField(0.0) + bean.setShortField(0.toShort) + bean.setByteField(0.toByte) + bean.setBooleanField(false) + bean.setDateField(java.sql.Date.valueOf("2014-10-10")) + bean.setTimestampField(java.sql.Timestamp.valueOf("2014-10-10 00:00:00.0")) + bean.setBigDecimalField(new java.math.BigDecimal(0)) + + val rdd = javaCtx.parallelize(bean :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) + schemaRDD.registerTempTable("decimalTypes") + + assert(javaSqlCtx.sql( + "select bigDecimalField + bigDecimalField from decimalTypes" + ).collect.head.row === Seq(scala.math.BigDecimal(0))) } test("all types null in JavaBeans") { @@ -98,6 +128,9 @@ class JavaSQLSuite extends FunSuite { bean.setShortField(null) bean.setByteField(null) bean.setBooleanField(null) + bean.setDateField(null) + bean.setTimestampField(null) + bean.setBigDecimalField(null) val rdd = javaCtx.parallelize(bean :: Nil) val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean]) @@ -107,10 +140,10 @@ class JavaSQLSuite extends FunSuite { javaSqlCtx.sql( """ |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField, - | booleanField + | booleanField, dateField, timestampField, bigDecimalField |FROM allTypes """.stripMargin).collect.head.row === - Seq.fill(8)(null)) + Seq.fill(11)(null)) } test("loads JSON datasets") { From 1df05a40ebf3493b0aff46d18c0f30d2d5256c7b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 29 Oct 2014 14:01:00 -0700 Subject: [PATCH 1130/1492] [SPARK-3822] Executor scaling mechanism for Yarn This is part of a broader effort to enable dynamic scaling of executors ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This is intended to work alongside SPARK-3795 (#2746), SPARK-3796 and SPARK-3797, but is functionally independently of these other issues. The logic is built on top of PraveenSeluka's changes at #2798. This is different from the changes there in a few major ways: (1) the mechanism is implemented within the existing scheduler backend framework rather than in new `Actor` classes. This also introduces a parent abstract class `YarnSchedulerBackend` to encapsulate common logic to communicate with the Yarn `ApplicationMaster`. (2) The interface of requesting executors exposed to the `SparkContext` is the same, but the communication between the scheduler backend and the AM uses total number executors desired instead of an incremental number. This is discussed in #2746 and explained in the comments in the code. I have tested this significantly on a stable Yarn cluster. ------------ A remaining task for this issue is to tone down the error messages emitted when an executor is removed. Currently, `SparkContext` and its components react as if the executor has failed, resulting in many scary error messages and eventual timeouts. While it's not strictly necessary to fix this as of the first-cut implementation of this mechanism, it would be good to add logic to distinguish this case. I prefer to address this in a separate PR. I have filed a separate JIRA for this task at SPARK-4134. Author: Andrew Or Author: Andrew Or Closes #2840 from andrewor14/yarn-scaling-mechanism and squashes the following commits: 485863e [Andrew Or] Minor log message changes 4920be8 [Andrew Or] Clarify that public API is only for Yarn mode for now 1c57804 [Andrew Or] Reword a few comments + other review comments 6321140 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism 02836c0 [Andrew Or] Limit scope of synchronization 4e2ed7f [Andrew Or] Fix bug: keep track of removed executors properly 73ade46 [Andrew Or] Wording changes (minor) 2a7a6da [Andrew Or] Add `sc.killExecutor` as a shorthand (minor) 665f229 [Andrew Or] Mima excludes 79aa2df [Andrew Or] Simplify the request interface by asking for a total 04f625b [Andrew Or] Fix race condition that causes over-allocation of executors f4783f8 [Andrew Or] Change the semantics of requesting executors 005a124 [Andrew Or] Fix tests 4628b16 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism db4a679 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism 572f5c5 [Andrew Or] Unused import (minor) f30261c [Andrew Or] Kill multiple executors rather than one at a time de260d9 [Andrew Or] Simplify by skipping useless null check 9c52542 [Andrew Or] Simplify by skipping the TaskSchedulerImpl 97dd1a8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-scaling-mechanism d987b3e [Andrew Or] Move addWebUIFilters to Yarn scheduler backend 7b76d0a [Andrew Or] Expose mechanism in SparkContext as developer API 47466cd [Andrew Or] Refactor common Yarn scheduler backend logic c4dfaac [Andrew Or] Avoid thrashing when removing executors 53e8145 [Andrew Or] Start yarn actor early to listen for AM registration message bbee669 [Andrew Or] Add mechanism in yarn client mode --- .../scala/org/apache/spark/SparkContext.scala | 64 ++++++-- .../spark/scheduler/TaskSchedulerImpl.scala | 1 - .../cluster/CoarseGrainedClusterMessage.scala | 14 +- .../CoarseGrainedSchedulerBackend.scala | 105 ++++++++++--- .../cluster/YarnSchedulerBackend.scala | 142 ++++++++++++++++++ .../org/apache/spark/util/AkkaUtils.scala | 17 ++- .../SparkContextSchedulerCreationSuite.scala | 7 +- project/MimaExcludes.scala | 4 + .../spark/deploy/yarn/ApplicationMaster.scala | 34 ++++- .../spark/deploy/yarn/YarnAllocator.scala | 51 ++++++- .../cluster/YarnClientSchedulerBackend.scala | 19 +-- .../cluster/YarnClusterSchedulerBackend.scala | 12 +- 12 files changed, 391 insertions(+), 79 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e8fdfff04390d..40ea369f9ef93 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -294,7 +294,8 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler - private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) + private[spark] var (schedulerBackend, taskScheduler) = + SparkContext.createTaskScheduler(this, master) private val heartbeatReceiver = env.actorSystem.actorOf( Props(new HeartbeatReceiver(taskScheduler)), "HeartbeatReceiver") @volatile private[spark] var dagScheduler: DAGScheduler = _ @@ -856,6 +857,40 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { listenerBus.addListener(listener) } + /** + * :: DeveloperApi :: + * Request an additional number of executors from the cluster manager. + * This is currently only supported in Yarn mode. + */ + @DeveloperApi + def requestExecutors(numAdditionalExecutors: Int): Unit = { + schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) + case _ => logWarning("Requesting executors is only supported in coarse-grained mode") + } + } + + /** + * :: DeveloperApi :: + * Request that the cluster manager kill the specified executors. + * This is currently only supported in Yarn mode. + */ + @DeveloperApi + def killExecutors(executorIds: Seq[String]): Unit = { + schedulerBackend match { + case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) + case _ => logWarning("Killing executors is only supported in coarse-grained mode") + } + } + + /** + * :: DeveloperApi :: + * Request that cluster manager the kill the specified executor. + * This is currently only supported in Yarn mode. + */ + @DeveloperApi + def killExecutor(executorId: String): Unit = killExecutors(Seq(executorId)) + /** The version of Spark on which this application is running. */ def version = SPARK_VERSION @@ -1438,8 +1473,13 @@ object SparkContext extends Logging { res } - /** Creates a task scheduler based on a given master URL. Extracted for testing. */ - private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { + /** + * Create a task scheduler based on a given master URL. + * Return a 2-tuple of the scheduler backend and the task scheduler. + */ + private def createTaskScheduler( + sc: SparkContext, + master: String): (SchedulerBackend, TaskScheduler) = { // Regular expression used for local[N] and local[*] master formats val LOCAL_N_REGEX = """local\[([0-9]+|\*)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1461,7 +1501,7 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, 1) scheduler.initialize(backend) - scheduler + (backend, scheduler) case LOCAL_N_REGEX(threads) => def localCpuCount = Runtime.getRuntime.availableProcessors() @@ -1470,7 +1510,7 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) - scheduler + (backend, scheduler) case LOCAL_N_FAILURES_REGEX(threads, maxFailures) => def localCpuCount = Runtime.getRuntime.availableProcessors() @@ -1480,14 +1520,14 @@ object SparkContext extends Logging { val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) - scheduler + (backend, scheduler) case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) - scheduler + (backend, scheduler) case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. @@ -1507,7 +1547,7 @@ object SparkContext extends Logging { backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() } - scheduler + (backend, scheduler) case "yarn-standalone" | "yarn-cluster" => if (master == "yarn-standalone") { @@ -1536,7 +1576,7 @@ object SparkContext extends Logging { } } scheduler.initialize(backend) - scheduler + (backend, scheduler) case "yarn-client" => val scheduler = try { @@ -1563,7 +1603,7 @@ object SparkContext extends Logging { } scheduler.initialize(backend) - scheduler + (backend, scheduler) case mesosUrl @ MESOS_REGEX(_) => MesosNativeLibrary.load() @@ -1576,13 +1616,13 @@ object SparkContext extends Logging { new MesosSchedulerBackend(scheduler, sc, url) } scheduler.initialize(backend) - scheduler + (backend, scheduler) case SIMR_REGEX(simrUrl) => val scheduler = new TaskSchedulerImpl(sc) val backend = new SimrSchedulerBackend(scheduler, sc, simrUrl) scheduler.initialize(backend) - scheduler + (backend, scheduler) case _ => throw new SparkException("Could not parse Master URL: '" + master + "'") 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 2b39c7fc872da..cd3c015321e85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -34,7 +34,6 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId -import akka.actor.Props /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. 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 fb8160abc59db..1da6fe976da5b 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,7 +66,19 @@ private[spark] object CoarseGrainedClusterMessages { case class RemoveExecutor(executorId: String, reason: String) extends CoarseGrainedClusterMessage - case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase :String) + // Exchanged between the driver and the AM in Yarn client mode + case class AddWebUIFilter(filterName:String, filterParams: Map[String, String], proxyBase: String) extends CoarseGrainedClusterMessage + // Messages exchanged between the driver and the cluster manager for executor allocation + // In Yarn mode, these are exchanged between the driver and the AM + + case object RegisterClusterManager extends CoarseGrainedClusterMessage + + // Request executors by specifying the new total number of executors desired + // This includes executors already pending or running + case class RequestExecutors(requestedTotal: Int) extends CoarseGrainedClusterMessage + + case class KillExecutors(executorIds: Seq[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 59aed6b72fe42..7a6ee56f81689 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,7 +31,6 @@ 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.{ActorLogReceive, SerializableBuffer, AkkaUtils, Utils} -import org.apache.spark.ui.JettyUtils /** * A scheduler backend that waits for coarse grained executors to connect to it through Akka. @@ -42,7 +41,7 @@ import org.apache.spark.ui.JettyUtils * (spark.deploy.*). */ private[spark] -class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: ActorSystem) +class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSystem: ActorSystem) extends SchedulerBackend with Logging { // Use an atomic variable to track total number of cores in the cluster for simplicity and speed @@ -61,10 +60,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) val createTime = System.currentTimeMillis() + private val executorDataMap = new HashMap[String, ExecutorData] + + // Number of executors requested from the cluster manager that have not registered yet + private var numPendingExecutors = 0 + + // Executors we have requested the cluster manager to kill that have not died yet + private val executorsPendingToRemove = new HashSet[String] + class DriverActor(sparkProperties: Seq[(String, String)]) extends Actor with ActorLogReceive { override protected def log = CoarseGrainedSchedulerBackend.this.log private val addressToExecutorId = new HashMap[Address, String] - private val executorDataMap = new HashMap[String, ExecutorData] override def preStart() { // Listen for remote client disconnection events, since they don't go through Akka's watch() @@ -84,12 +90,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } else { logInfo("Registered executor: " + sender + " with ID " + executorId) sender ! RegisteredExecutor - executorDataMap.put(executorId, new ExecutorData(sender, sender.path.address, - Utils.parseHostPort(hostPort)._1, cores, cores)) addressToExecutorId(sender.path.address) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) + val (host, _) = Utils.parseHostPort(hostPort) + val data = new ExecutorData(sender, sender.path.address, host, cores, cores) + // This must be synchronized because variables mutated + // in this block are read when requesting executors + CoarseGrainedSchedulerBackend.this.synchronized { + executorDataMap.put(executorId, data) + if (numPendingExecutors > 0) { + numPendingExecutors -= 1 + logDebug(s"Decremented number of pending executors ($numPendingExecutors left)") + } + } makeOffers() } @@ -128,10 +143,6 @@ 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")) @@ -183,13 +194,18 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A } // Remove a disconnected slave from the cluster - def removeExecutor(executorId: String, reason: String) { + def removeExecutor(executorId: String, reason: String): Unit = { executorDataMap.get(executorId) match { case Some(executorInfo) => - executorDataMap -= executorId + // This must be synchronized because variables mutated + // in this block are read when requesting executors + CoarseGrainedSchedulerBackend.this.synchronized { + executorDataMap -= executorId + executorsPendingToRemove -= executorId + } totalCoreCount.addAndGet(-executorInfo.totalCores) scheduler.executorLost(executorId, SlaveLost(reason)) - case None => logError(s"Asked to remove non existant executor $executorId") + case None => logError(s"Asked to remove non-existent executor $executorId") } } } @@ -274,21 +290,62 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A false } - // Add filters to the SparkUI - def addWebUIFilter(filterName: String, filterParams: Map[String, String], proxyBase: String) { - if (proxyBase != null && proxyBase.nonEmpty) { - System.setProperty("spark.ui.proxyBase", proxyBase) - } + /** + * Return the number of executors currently registered with this backend. + */ + def numExistingExecutors: Int = executorDataMap.size + + /** + * Request an additional number of executors from the cluster manager. + * Return whether the request is acknowledged. + */ + final def requestExecutors(numAdditionalExecutors: Int): Boolean = synchronized { + logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") + logDebug(s"Number of pending executors is now $numPendingExecutors") + numPendingExecutors += numAdditionalExecutors + // Account for executors pending to be added or removed + val newTotal = numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size + doRequestTotalExecutors(newTotal) + } - val hasFilter = (filterName != null && filterName.nonEmpty && - filterParams != null && filterParams.nonEmpty) - if (hasFilter) { - logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") - conf.set("spark.ui.filters", filterName) - filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } - scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } + /** + * Request executors from the cluster manager by specifying the total number desired, + * including existing pending and running executors. + * + * The semantics here guarantee that we do not over-allocate executors for this application, + * since a later request overrides the value of any prior request. The alternative interface + * of requesting a delta of executors risks double counting new executors when there are + * insufficient resources to satisfy the first request. We make the assumption here that the + * cluster manager will eventually fulfill all requests when resources free up. + * + * Return whether the request is acknowledged. + */ + protected def doRequestTotalExecutors(requestedTotal: Int): Boolean = false + + /** + * Request that the cluster manager kill the specified executors. + * Return whether the kill request is acknowledged. + */ + final def killExecutors(executorIds: Seq[String]): Boolean = { + logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") + val filteredExecutorIds = new ArrayBuffer[String] + executorIds.foreach { id => + if (executorDataMap.contains(id)) { + filteredExecutorIds += id + } else { + logWarning(s"Executor to kill $id does not exist!") + } } + executorsPendingToRemove ++= filteredExecutorIds + doKillExecutors(filteredExecutorIds) } + + /** + * Kill the given list of executors through the cluster manager. + * Return whether the kill request is acknowledged. + */ + protected def doKillExecutors(executorIds: Seq[String]): Boolean = false + } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala new file mode 100644 index 0000000000000..50721b9d6cd6c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -0,0 +1,142 @@ +/* + * 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 akka.actor.{Actor, ActorRef, Props} +import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} + +import org.apache.spark.SparkContext +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.ui.JettyUtils +import org.apache.spark.util.AkkaUtils + +/** + * Abstract Yarn scheduler backend that contains common logic + * between the client and cluster Yarn scheduler backends. + */ +private[spark] abstract class YarnSchedulerBackend( + scheduler: TaskSchedulerImpl, + sc: SparkContext) + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { + + if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + minRegisteredRatio = 0.8 + } + + protected var totalExpectedExecutors = 0 + + private val yarnSchedulerActor: ActorRef = + actorSystem.actorOf( + Props(new YarnSchedulerActor), + name = YarnSchedulerBackend.ACTOR_NAME) + + private implicit val askTimeout = AkkaUtils.askTimeout(sc.conf) + + /** + * Request executors from the ApplicationMaster by specifying the total number desired. + * This includes executors already pending or running. + */ + override def doRequestTotalExecutors(requestedTotal: Int): Boolean = { + AkkaUtils.askWithReply[Boolean]( + RequestExecutors(requestedTotal), yarnSchedulerActor, askTimeout) + } + + /** + * Request that the ApplicationMaster kill the specified executors. + */ + override def doKillExecutors(executorIds: Seq[String]): Boolean = { + AkkaUtils.askWithReply[Boolean]( + KillExecutors(executorIds), yarnSchedulerActor, askTimeout) + } + + override def sufficientResourcesRegistered(): Boolean = { + totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio + } + + /** + * Add filters to the SparkUI. + */ + private def addWebUIFilter( + filterName: String, + filterParams: Map[String, String], + proxyBase: String): Unit = { + if (proxyBase != null && proxyBase.nonEmpty) { + System.setProperty("spark.ui.proxyBase", proxyBase) + } + + val hasFilter = + filterName != null && filterName.nonEmpty && + filterParams != null && filterParams.nonEmpty + if (hasFilter) { + logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") + conf.set("spark.ui.filters", filterName) + filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } + scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } + } + } + + /** + * An actor that communicates with the ApplicationMaster. + */ + private class YarnSchedulerActor extends Actor { + private var amActor: Option[ActorRef] = None + + override def preStart(): Unit = { + // Listen for disassociation events + context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + } + + override def receive = { + case RegisterClusterManager => + logInfo(s"ApplicationMaster registered as $sender") + amActor = Some(sender) + + case r: RequestExecutors => + amActor match { + case Some(actor) => + sender ! AkkaUtils.askWithReply[Boolean](r, actor, askTimeout) + case None => + logWarning("Attempted to request executors before the AM has registered!") + sender ! false + } + + case k: KillExecutors => + amActor match { + case Some(actor) => + sender ! AkkaUtils.askWithReply[Boolean](k, actor, askTimeout) + case None => + logWarning("Attempted to kill executors before the AM has registered!") + sender ! false + } + + case AddWebUIFilter(filterName, filterParams, proxyBase) => + addWebUIFilter(filterName, filterParams, proxyBase) + sender ! true + + case d: DisassociatedEvent => + if (amActor.isDefined && sender == amActor.get) { + logWarning(s"ApplicationMaster has disassociated: $d") + } + } + } +} + +private[spark] object YarnSchedulerBackend { + val ACTOR_NAME = "YarnScheduler" +} diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index f41c8d0315cb3..79e398eb8c104 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -159,17 +159,28 @@ private[spark] object AkkaUtils extends Logging { def askWithReply[T]( message: Any, actor: ActorRef, - retryAttempts: Int, + timeout: FiniteDuration): T = { + askWithReply[T](message, actor, maxAttempts = 1, retryInterval = Int.MaxValue, timeout) + } + + /** + * Send a message to the given actor and get its result within a default timeout, or + * throw a SparkException if this fails even after the specified number of retries. + */ + def askWithReply[T]( + message: Any, + actor: ActorRef, + maxAttempts: Int, retryInterval: Int, timeout: FiniteDuration): T = { // TODO: Consider removing multiple attempts if (actor == null) { - throw new SparkException("Error sending message as driverActor is null " + + throw new SparkException("Error sending message as actor is null " + "[message = " + message + "]") } var attempts = 0 var lastException: Exception = null - while (attempts < retryAttempts) { + while (attempts < maxAttempts) { attempts += 1 try { val future = actor.ask(message)(timeout) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 495a0d48633a4..df237ba796b38 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import org.scalatest.{BeforeAndAfterEach, FunSuite, PrivateMethodTester} -import org.apache.spark.scheduler.{TaskScheduler, TaskSchedulerImpl} +import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend @@ -31,8 +31,9 @@ class SparkContextSchedulerCreationSuite // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. val sc = new SparkContext("local", "test") - val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) - val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) + val createTaskSchedulerMethod = + PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) + val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index adbdc5d1da3c1..6a0495f8fd540 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -73,6 +73,10 @@ object MimaExcludes { "org.apache.spark.api.java.JavaRDDLike.foreachAsync"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.api.java.JavaRDDLike.collectAsync") + ) ++ Seq( + // SPARK-3822 + ProblemFilters.exclude[IncompatibleResultTypeProblem]( + "org.apache.spark.SparkContext.org$apache$spark$SparkContext$$createTaskScheduler") ) case v if v.startsWith("1.1") => diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index e6fe0265d8811..68073798886dd 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -36,8 +36,8 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, Spar import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer -import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.AddWebUIFilter +import org.apache.spark.scheduler.cluster.YarnSchedulerBackend +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.util.{AkkaUtils, SignalLogger, Utils} /** @@ -385,8 +385,8 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, SparkEnv.driverActorSystemName, driverHost, driverPort.toString, - CoarseGrainedSchedulerBackend.ACTOR_NAME) - actorSystem.actorOf(Props(new MonitorActor(driverUrl)), name = "YarnAM") + YarnSchedulerBackend.ACTOR_NAME) + actorSystem.actorOf(Props(new AMActor(driverUrl)), name = "YarnAM") } /** Add the Yarn IP filter that is required for properly securing the UI. */ @@ -479,9 +479,10 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, userThread } - // Actor used to monitor the driver when running in client deploy mode. - private class MonitorActor(driverUrl: String) extends Actor { - + /** + * Actor that communicates with the driver in client deploy mode. + */ + private class AMActor(driverUrl: String) extends Actor { var driver: ActorSelection = _ override def preStart() = { @@ -490,6 +491,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, // Send a hello message to establish the connection, after which // we can monitor Lifecycle Events. driver ! "Hello" + driver ! RegisterClusterManager context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) } @@ -497,11 +499,27 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, case x: DisassociatedEvent => logInfo(s"Driver terminated or disconnected! Shutting down. $x") finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + case x: AddWebUIFilter => logInfo(s"Add WebUI Filter. $x") driver ! x - } + case RequestExecutors(requestedTotal) => + logInfo(s"Driver requested a total number of executors of $requestedTotal.") + Option(allocator) match { + case Some(a) => a.requestTotalExecutors(requestedTotal) + case None => logWarning("Container allocator is not ready to request executors yet.") + } + sender ! true + + case KillExecutors(executorIds) => + logInfo(s"Driver requested to kill executor(s) ${executorIds.mkString(", ")}.") + Option(allocator) match { + case Some(a) => executorIds.foreach(a.killExecutor) + case None => logWarning("Container allocator is not ready to kill executors yet.") + } + sender ! true + } } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index e1af8d5a74cb1..7ae8ef237ff89 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -88,7 +88,10 @@ private[yarn] abstract class YarnAllocator( private val executorIdCounter = new AtomicInteger() private val numExecutorsFailed = new AtomicInteger() - private val maxExecutors = args.numExecutors + private var maxExecutors = args.numExecutors + + // Keep track of which container is running which executor to remove the executors later + private val executorIdToContainer = new HashMap[String, Container] protected val executorMemory = args.executorMemory protected val executorCores = args.executorCores @@ -111,7 +114,48 @@ private[yarn] abstract class YarnAllocator( def getNumExecutorsFailed: Int = numExecutorsFailed.intValue - def allocateResources() = { + /** + * Request as many executors from the ResourceManager as needed to reach the desired total. + * This takes into account executors already running or pending. + */ + def requestTotalExecutors(requestedTotal: Int): Unit = synchronized { + val currentTotal = numPendingAllocate.get + numExecutorsRunning.get + if (requestedTotal > currentTotal) { + maxExecutors += (requestedTotal - currentTotal) + // We need to call `allocateResources` here to avoid the following race condition: + // If we request executors twice before `allocateResources` is called, then we will end up + // double counting the number requested because `numPendingAllocate` is not updated yet. + allocateResources() + } else { + logInfo(s"Not allocating more executors because there are already $currentTotal " + + s"(application requested $requestedTotal total)") + } + } + + /** + * Request that the ResourceManager release the container running the specified executor. + */ + def killExecutor(executorId: String): Unit = synchronized { + if (executorIdToContainer.contains(executorId)) { + val container = executorIdToContainer.remove(executorId).get + internalReleaseContainer(container) + numExecutorsRunning.decrementAndGet() + maxExecutors -= 1 + assert(maxExecutors >= 0, "Allocator killed more executors than are allocated!") + } else { + logWarning(s"Attempted to kill unknown executor $executorId!") + } + } + + /** + * Allocate missing containers based on the number of executors currently pending and running. + * + * This method prioritizes the allocated container responses from the RM based on node and + * rack locality. Additionally, it releases any extra containers allocated for this application + * but are not needed. This must be synchronized because variables read in this block are + * mutated by other methods. + */ + def allocateResources(): Unit = synchronized { val missing = maxExecutors - numPendingAllocate.get() - numExecutorsRunning.get() // this is needed by alpha, do it here since we add numPending right after this @@ -119,7 +163,7 @@ private[yarn] abstract class YarnAllocator( if (missing > 0) { val totalExecutorMemory = executorMemory + memoryOverhead numPendingAllocate.addAndGet(missing) - logInfo(s"Will allocate $missing executor containers, each with $totalExecutorMemory MB " + + logInfo(s"Will allocate $missing executor containers, each with $totalExecutorMemory MB " + s"memory including $memoryOverhead MB overhead") } else { logDebug("Empty allocation request ...") @@ -269,6 +313,7 @@ private[yarn] abstract class YarnAllocator( CoarseGrainedSchedulerBackend.ACTOR_NAME) logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) + executorIdToContainer(executorId) = container // To be safe, remove the container from `releasedContainers`. releasedContainers.remove(containerId) 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 59b2b47aed2fe..f6f6dc52433e5 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 @@ -17,27 +17,23 @@ package org.apache.spark.scheduler.cluster +import scala.collection.mutable.ArrayBuffer + import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} + import org.apache.spark.{SparkException, Logging, SparkContext} import org.apache.spark.deploy.yarn.{Client, ClientArguments} import org.apache.spark.scheduler.TaskSchedulerImpl -import scala.collection.mutable.ArrayBuffer - private[spark] class YarnClientSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) + extends YarnSchedulerBackend(scheduler, sc) with Logging { - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { - minRegisteredRatio = 0.8 - } - private var client: Client = null private var appId: ApplicationId = null private var stopping: Boolean = false - private var totalExpectedExecutors = 0 /** * Create a Yarn client to submit an application to the ResourceManager. @@ -151,14 +147,11 @@ private[spark] class YarnClientSchedulerBackend( logInfo("Stopped") } - override def sufficientResourcesRegistered(): Boolean = { - totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio - } - - override def applicationId(): String = + override def applicationId(): String = { Option(appId).map(_.toString).getOrElse { logWarning("Application ID is not initialized yet.") super.applicationId } + } } 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 index 3a186cfeb4eeb..a96a54f66824c 100644 --- 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 @@ -25,13 +25,7 @@ import org.apache.spark.util.IntParam private[spark] class YarnClusterSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext) - extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) { - - var totalExpectedExecutors = 0 - - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { - minRegisteredRatio = 0.8 - } + extends YarnSchedulerBackend(scheduler, sc) { override def start() { super.start() @@ -44,10 +38,6 @@ private[spark] class YarnClusterSchedulerBackend( totalExpectedExecutors = sc.getConf.getInt("spark.executor.instances", totalExpectedExecutors) } - override def sufficientResourcesRegistered(): Boolean = { - totalRegisteredExecutors.get() >= totalExpectedExecutors * minRegisteredRatio - } - override def applicationId(): String = // In YARN Cluster mode, spark.yarn.app.id is expect to be set // before user application is launched. From e7fd80413d531e23b6c4def0ee32e52a39da36fa Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 29 Oct 2014 14:42:50 -0700 Subject: [PATCH 1131/1492] [SPARK-4097] Fix the race condition of 'thread' There is a chance that `thread` is null when calling `thread.interrupt()`. ```Scala override def cancel(): Unit = this.synchronized { _cancelled = true if (thread != null) { thread.interrupt() } } ``` Should put `thread = null` into a `synchronized` block to fix the race condition. Author: zsxwing Closes #2957 from zsxwing/SPARK-4097 and squashes the following commits: edf0aee [zsxwing] Add comments to explain the lock c5cfeca [zsxwing] Fix the race condition of 'thread' --- core/src/main/scala/org/apache/spark/FutureAction.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index d5c8f9d76c476..e97a7375a267b 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -210,7 +210,11 @@ class ComplexFutureAction[T] extends FutureAction[T] { } catch { case e: Exception => p.failure(e) } finally { - thread = null + // This lock guarantees when calling `thread.interrupt()` in `cancel`, + // thread won't be set to null. + ComplexFutureAction.this.synchronized { + thread = null + } } } this From 8d59b37b02eb36f37bcefafb952519d7dca744ad Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 29 Oct 2014 17:48:59 -0700 Subject: [PATCH 1132/1492] [SPARK-3795] Heuristics for dynamically scaling executors This is part of a bigger effort to provide elastic scaling of executors within a Spark application ([SPARK-3174](https://issues.apache.org/jira/browse/SPARK-3174)). This PR does not provide any functionality by itself; it is a skeleton that is missing a mechanism to be added later in [SPARK-3822](https://issues.apache.org/jira/browse/SPARK-3822). Comments and feedback are most welcome. For those of you reviewing this in detail, I highly recommend doing it through your favorite IDE instead of through the diff here. Author: Andrew Or Author: Andrew Or Closes #2746 from andrewor14/scaling-heuristics and squashes the following commits: 8a4fdaa [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics e045df8 [Andrew Or] Add warning message (minor) dfa31ec [Andrew Or] Fix tests c0becc4 [Andrew Or] Merging with SPARK-3822 4784f93 [Andrew Or] Reword an awkward log message 181f27f [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics c79e907 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 4672b90 [Andrew Or] It's nano time. a6a30f2 [Andrew Or] Do not allow min/max executors of 0 c60ec33 [Andrew Or] Rewrite test logic with clocks b00b680 [Andrew Or] Fix style c3caa65 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 7f9da14 [Andrew Or] Factor out logic to verify bounds on # executors (minor) f279019 [Andrew Or] Add time mocking tests for polling loop 685e347 [Andrew Or] Factor out clock in polling loop to facilitate testing 3cea7f7 [Andrew Or] Use PrivateMethodTester to keep original class private 3156d81 [Andrew Or] Update comments and exception messages 92f36f9 [Andrew Or] Address minor review comments abdea61 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 2aefd09 [Andrew Or] Correct listener behavior 9fe6e44 [Andrew Or] Rename variables and configs + update comments and log messages 149cc32 [Andrew Or] Fix style 254c958 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 5ff829b [Andrew Or] Add tests for ExecutorAllocationManager 19c6c4b [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 5896515 [Andrew Or] Move ExecutorAllocationManager out of scheduler package 9ca8945 [Andrew Or] Rewrite callbacks through the listener interface 5e336b9 [Andrew Or] Remove code from backend to avoid conflict with SPARK-3822 092d1fd [Andrew Or] Remove timeout logic for pending requests 1309fab [Andrew Or] Request executors by specifying the number pending 8bc0e9d [Andrew Or] Add logic to expire pending requests after timeouts b750ee1 [Andrew Or] Express timers in terms of expiration times + remove retry logic 7f8dd47 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 9d516cc [Andrew Or] Bug fix: Actually trigger the add timer / add retry timer 44f1832 [Andrew Or] Rename configs to include time units eaae7ef [Andrew Or] Address various review comments 6f8be6c [Andrew Or] Beef up comments on what each of the timers mean baaa403 [Andrew Or] Simplify variable names (minor) 42beec8 [Andrew Or] Reset whether the add threshold is crossed on cancellation 9bcc0bc [Andrew Or] ExecutorScalingManager -> ExecutorAllocationManager 2784398 [Andrew Or] Merge branch 'master' of github.com:apache/spark into scaling-heuristics 5a97d9e [Andrew Or] Log retry attempts in INFO + clean up logging 2f55c9f [Andrew Or] Do not keep requesting executors even after max attempts 0acd1cb [Andrew Or] Rewrite timer logic with polling b3c7d44 [Andrew Or] Start the retry timer for adding executors at the right time 9b5f2ea [Andrew Or] Wording changes in comments and log messages c2203a5 [Andrew Or] Simplify code to access the scheduler backend e519d08 [Andrew Or] Simplify initialization code 2cc87a7 [Andrew Or] Add retry logic for removing executors d0b34a6 [Andrew Or] Add retry logic for adding executors 9cc4649 [Andrew Or] Simplifying synchronization logic 67c03c7 [Andrew Or] Correct semantics of adding executors + update comments 6c48ab0 [Andrew Or] Update synchronization comment 8901900 [Andrew Or] Simplify remove policy + change the semantics of add policy 1cc8444 [Andrew Or] Minor wording change ae5b64a [Andrew Or] Add synchronization 20ec6b9 [Andrew Or] First cut implementation of removing executors dynamically 4077ae2 [Andrew Or] Minor code re-organization 6f1fa66 [Andrew Or] First cut implementation of adding executors dynamically b2e6dcc [Andrew Or] Add skeleton interface for requesting / killing executors --- .../spark/ExecutorAllocationManager.scala | 462 ++++++++++++ .../scala/org/apache/spark/SparkContext.scala | 35 +- .../ExecutorAllocationManagerSuite.scala | 662 ++++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- 4 files changed, 1150 insertions(+), 11 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala create mode 100644 core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala new file mode 100644 index 0000000000000..b2cf022baf29f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -0,0 +1,462 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import scala.collection.mutable + +import org.apache.spark.scheduler._ + +/** + * An agent that dynamically allocates and removes executors based on the workload. + * + * The add policy depends on whether there are backlogged tasks waiting to be scheduled. If + * the scheduler queue is not drained in N seconds, then new executors are added. If the queue + * persists for another M seconds, then more executors are added and so on. The number added + * in each round increases exponentially from the previous round until an upper bound on the + * number of executors has been reached. + * + * The rationale for the exponential increase is twofold: (1) Executors should be added slowly + * in the beginning in case the number of extra executors needed turns out to be small. Otherwise, + * we may add more executors than we need just to remove them later. (2) Executors should be added + * quickly over time in case the maximum number of executors is very high. Otherwise, it will take + * a long time to ramp up under heavy workloads. + * + * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not + * been scheduled to run any tasks, then it is removed. + * + * There is no retry logic in either case because we make the assumption that the cluster manager + * will eventually fulfill all requests it receives asynchronously. + * + * The relevant Spark properties include the following: + * + * spark.dynamicAllocation.enabled - Whether this feature is enabled + * spark.dynamicAllocation.minExecutors - Lower bound on the number of executors + * spark.dynamicAllocation.maxExecutors - Upper bound on the number of executors + * + * spark.dynamicAllocation.schedulerBacklogTimeout (M) - + * If there are backlogged tasks for this duration, add new executors + * + * spark.dynamicAllocation.sustainedSchedulerBacklogTimeout (N) - + * If the backlog is sustained for this duration, add more executors + * This is used only after the initial backlog timeout is exceeded + * + * spark.dynamicAllocation.executorIdleTimeout (K) - + * If an executor has been idle for this duration, remove it + */ +private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging { + import ExecutorAllocationManager._ + + private val conf = sc.conf + + // Lower and upper bounds on the number of executors. These are required. + private val minNumExecutors = conf.getInt("spark.dynamicAllocation.minExecutors", -1) + private val maxNumExecutors = conf.getInt("spark.dynamicAllocation.maxExecutors", -1) + verifyBounds() + + // How long there must be backlogged tasks for before an addition is triggered + private val schedulerBacklogTimeout = conf.getLong( + "spark.dynamicAllocation.schedulerBacklogTimeout", 60) + + // Same as above, but used only after `schedulerBacklogTimeout` is exceeded + private val sustainedSchedulerBacklogTimeout = conf.getLong( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) + + // How long an executor must be idle for before it is removed + private val removeThresholdSeconds = conf.getLong( + "spark.dynamicAllocation.executorIdleTimeout", 600) + + // Number of executors to add in the next round + private var numExecutorsToAdd = 1 + + // Number of executors that have been requested but have not registered yet + private var numExecutorsPending = 0 + + // Executors that have been requested to be removed but have not been killed yet + private val executorsPendingToRemove = new mutable.HashSet[String] + + // All known executors + private val executorIds = new mutable.HashSet[String] + + // A timestamp of when an addition should be triggered, or NOT_SET if it is not set + // This is set when pending tasks are added but not scheduled yet + private var addTime: Long = NOT_SET + + // A timestamp for each executor of when the executor should be removed, indexed by the ID + // This is set when an executor is no longer running a task, or when it first registers + private val removeTimes = new mutable.HashMap[String, Long] + + // Polling loop interval (ms) + private val intervalMillis: Long = 100 + + // Whether we are testing this class. This should only be used internally. + private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) + + // Clock used to schedule when executors should be added and removed + private var clock: Clock = new RealClock + + /** + * Verify that the lower and upper bounds on the number of executors are valid. + * If not, throw an appropriate exception. + */ + private def verifyBounds(): Unit = { + if (minNumExecutors < 0 || maxNumExecutors < 0) { + throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be set!") + } + if (minNumExecutors == 0 || maxNumExecutors == 0) { + throw new SparkException("spark.dynamicAllocation.{min/max}Executors cannot be 0!") + } + if (minNumExecutors > maxNumExecutors) { + throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") + } + } + + /** + * Use a different clock for this allocation manager. This is mainly used for testing. + */ + def setClock(newClock: Clock): Unit = { + clock = newClock + } + + /** + * Register for scheduler callbacks to decide when to add and remove executors. + */ + def start(): Unit = { + val listener = new ExecutorAllocationListener(this) + sc.addSparkListener(listener) + startPolling() + } + + /** + * Start the main polling thread that keeps track of when to add and remove executors. + */ + private def startPolling(): Unit = { + val t = new Thread { + override def run(): Unit = { + while (true) { + try { + schedule() + } catch { + case e: Exception => logError("Exception in dynamic executor allocation thread!", e) + } + Thread.sleep(intervalMillis) + } + } + } + t.setName("spark-dynamic-executor-allocation") + t.setDaemon(true) + t.start() + } + + /** + * If the add time has expired, request new executors and refresh the add time. + * If the remove time for an existing executor has expired, kill the executor. + * This is factored out into its own method for testing. + */ + private def schedule(): Unit = synchronized { + val now = clock.getTimeMillis + if (addTime != NOT_SET && now >= addTime) { + addExecutors() + logDebug(s"Starting timer to add more executors (to " + + s"expire in $sustainedSchedulerBacklogTimeout seconds)") + addTime += sustainedSchedulerBacklogTimeout * 1000 + } + + removeTimes.foreach { case (executorId, expireTime) => + if (now >= expireTime) { + removeExecutor(executorId) + removeTimes.remove(executorId) + } + } + } + + /** + * Request a number of executors from the cluster manager. + * If the cap on the number of executors is reached, give up and reset the + * number of executors to add next round instead of continuing to double it. + * Return the number actually requested. + */ + private def addExecutors(): Int = synchronized { + // Do not request more executors if we have already reached the upper bound + val numExistingExecutors = executorIds.size + numExecutorsPending + if (numExistingExecutors >= maxNumExecutors) { + logDebug(s"Not adding executors because there are already ${executorIds.size} " + + s"registered and $numExecutorsPending pending executor(s) (limit $maxNumExecutors)") + numExecutorsToAdd = 1 + return 0 + } + + // Request executors with respect to the upper bound + val actualNumExecutorsToAdd = + if (numExistingExecutors + numExecutorsToAdd <= maxNumExecutors) { + numExecutorsToAdd + } else { + maxNumExecutors - numExistingExecutors + } + val newTotalExecutors = numExistingExecutors + actualNumExecutorsToAdd + val addRequestAcknowledged = testing || sc.requestExecutors(actualNumExecutorsToAdd) + if (addRequestAcknowledged) { + logInfo(s"Requesting $actualNumExecutorsToAdd new executor(s) because " + + s"tasks are backlogged (new desired total will be $newTotalExecutors)") + numExecutorsToAdd = + if (actualNumExecutorsToAdd == numExecutorsToAdd) numExecutorsToAdd * 2 else 1 + numExecutorsPending += actualNumExecutorsToAdd + actualNumExecutorsToAdd + } else { + logWarning(s"Unable to reach the cluster manager " + + s"to request $actualNumExecutorsToAdd executors!") + 0 + } + } + + /** + * Request the cluster manager to remove the given executor. + * Return whether the request is received. + */ + private def removeExecutor(executorId: String): Boolean = synchronized { + // Do not kill the executor if we are not aware of it (should never happen) + if (!executorIds.contains(executorId)) { + logWarning(s"Attempted to remove unknown executor $executorId!") + return false + } + + // Do not kill the executor again if it is already pending to be killed (should never happen) + if (executorsPendingToRemove.contains(executorId)) { + logWarning(s"Attempted to remove executor $executorId " + + s"when it is already pending to be removed!") + return false + } + + // Do not kill the executor if we have already reached the lower bound + val numExistingExecutors = executorIds.size - executorsPendingToRemove.size + if (numExistingExecutors - 1 < minNumExecutors) { + logInfo(s"Not removing idle executor $executorId because there are only " + + s"$numExistingExecutors executor(s) left (limit $minNumExecutors)") + return false + } + + // Send a request to the backend to kill this executor + val removeRequestAcknowledged = testing || sc.killExecutor(executorId) + if (removeRequestAcknowledged) { + logInfo(s"Removing executor $executorId because it has been idle for " + + s"$removeThresholdSeconds seconds (new desired total will be ${numExistingExecutors - 1})") + executorsPendingToRemove.add(executorId) + true + } else { + logWarning(s"Unable to reach the cluster manager to kill executor $executorId!") + false + } + } + + /** + * Callback invoked when the specified executor has been added. + */ + private def onExecutorAdded(executorId: String): Unit = synchronized { + if (!executorIds.contains(executorId)) { + executorIds.add(executorId) + executorIds.foreach(onExecutorIdle) + logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})") + if (numExecutorsPending > 0) { + numExecutorsPending -= 1 + logDebug(s"Decremented number of pending executors ($numExecutorsPending left)") + } + } else { + logWarning(s"Duplicate executor $executorId has registered") + } + } + + /** + * Callback invoked when the specified executor has been removed. + */ + private def onExecutorRemoved(executorId: String): Unit = synchronized { + if (executorIds.contains(executorId)) { + executorIds.remove(executorId) + removeTimes.remove(executorId) + logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})") + if (executorsPendingToRemove.contains(executorId)) { + executorsPendingToRemove.remove(executorId) + logDebug(s"Executor $executorId is no longer pending to " + + s"be removed (${executorsPendingToRemove.size} left)") + } + } else { + logWarning(s"Unknown executor $executorId has been removed!") + } + } + + /** + * Callback invoked when the scheduler receives new pending tasks. + * This sets a time in the future that decides when executors should be added + * if it is not already set. + */ + private def onSchedulerBacklogged(): Unit = synchronized { + if (addTime == NOT_SET) { + logDebug(s"Starting timer to add executors because pending tasks " + + s"are building up (to expire in $schedulerBacklogTimeout seconds)") + addTime = clock.getTimeMillis + schedulerBacklogTimeout * 1000 + } + } + + /** + * Callback invoked when the scheduler queue is drained. + * This resets all variables used for adding executors. + */ + private def onSchedulerQueueEmpty(): Unit = synchronized { + logDebug(s"Clearing timer to add executors because there are no more pending tasks") + addTime = NOT_SET + numExecutorsToAdd = 1 + } + + /** + * Callback invoked when the specified executor is no longer running any tasks. + * This sets a time in the future that decides when this executor should be removed if + * the executor is not already marked as idle. + */ + private def onExecutorIdle(executorId: String): Unit = synchronized { + if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { + logDebug(s"Starting idle timer for $executorId because there are no more tasks " + + s"scheduled to run on the executor (to expire in $removeThresholdSeconds seconds)") + removeTimes(executorId) = clock.getTimeMillis + removeThresholdSeconds * 1000 + } + } + + /** + * Callback invoked when the specified executor is now running a task. + * This resets all variables used for removing this executor. + */ + private def onExecutorBusy(executorId: String): Unit = synchronized { + logDebug(s"Clearing idle timer for $executorId because it is now running a task") + removeTimes.remove(executorId) + } + + /** + * A listener that notifies the given allocation manager of when to add and remove executors. + * + * This class is intentionally conservative in its assumptions about the relative ordering + * and consistency of events returned by the listener. For simplicity, it does not account + * for speculated tasks. + */ + private class ExecutorAllocationListener(allocationManager: ExecutorAllocationManager) + extends SparkListener { + + private val stageIdToNumTasks = new mutable.HashMap[Int, Int] + private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] + private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]] + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { + synchronized { + val stageId = stageSubmitted.stageInfo.stageId + val numTasks = stageSubmitted.stageInfo.numTasks + stageIdToNumTasks(stageId) = numTasks + allocationManager.onSchedulerBacklogged() + } + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + synchronized { + val stageId = stageCompleted.stageInfo.stageId + stageIdToNumTasks -= stageId + stageIdToTaskIndices -= stageId + + // If this is the last stage with pending tasks, mark the scheduler queue as empty + // This is needed in case the stage is aborted for any reason + if (stageIdToNumTasks.isEmpty) { + allocationManager.onSchedulerQueueEmpty() + } + } + } + + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { + val stageId = taskStart.stageId + val taskId = taskStart.taskInfo.taskId + val taskIndex = taskStart.taskInfo.index + val executorId = taskStart.taskInfo.executorId + + // If this is the last pending task, mark the scheduler queue as empty + stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex + val numTasksScheduled = stageIdToTaskIndices(stageId).size + val numTasksTotal = stageIdToNumTasks.getOrElse(stageId, -1) + if (numTasksScheduled == numTasksTotal) { + // No more pending tasks for this stage + stageIdToNumTasks -= stageId + if (stageIdToNumTasks.isEmpty) { + allocationManager.onSchedulerQueueEmpty() + } + } + + // Mark the executor on which this task is scheduled as busy + executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId + allocationManager.onExecutorBusy(executorId) + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { + val executorId = taskEnd.taskInfo.executorId + val taskId = taskEnd.taskInfo.taskId + + // If the executor is no longer running scheduled any tasks, mark it as idle + if (executorIdToTaskIds.contains(executorId)) { + executorIdToTaskIds(executorId) -= taskId + if (executorIdToTaskIds(executorId).isEmpty) { + executorIdToTaskIds -= executorId + allocationManager.onExecutorIdle(executorId) + } + } + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { + val executorId = blockManagerAdded.blockManagerId.executorId + if (executorId != "") { + allocationManager.onExecutorAdded(executorId) + } + } + + override def onBlockManagerRemoved( + blockManagerRemoved: SparkListenerBlockManagerRemoved): Unit = { + allocationManager.onExecutorRemoved(blockManagerRemoved.blockManagerId.executorId) + } + } + +} + +private object ExecutorAllocationManager { + val NOT_SET = Long.MaxValue +} + +/** + * An abstract clock for measuring elapsed time. + */ +private trait Clock { + def getTimeMillis: Long +} + +/** + * A clock backed by a monotonically increasing time source. + * The time returned by this clock does not correspond to any notion of wall-clock time. + */ +private class RealClock extends Clock { + override def getTimeMillis: Long = System.nanoTime / (1000 * 1000) +} + +/** + * A clock that allows the caller to customize the time. + * This is used mainly for testing. + */ +private class TestClock(startTimeMillis: Long) extends Clock { + private var time: Long = startTimeMillis + override def getTimeMillis: Long = time + def tick(ms: Long): Unit = { time += ms } +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 40ea369f9ef93..73668e83bbb1d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -330,6 +330,15 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { } else None } + // Optionally scale number of executors dynamically based on workload. Exposed for testing. + private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = + if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + Some(new ExecutorAllocationManager(this)) + } else { + None + } + executorAllocationManager.foreach(_.start()) + // At this point, all relevant SparkListeners have been registered, so begin releasing events listenerBus.start() @@ -860,36 +869,42 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { /** * :: DeveloperApi :: * Request an additional number of executors from the cluster manager. - * This is currently only supported in Yarn mode. + * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def requestExecutors(numAdditionalExecutors: Int): Unit = { + def requestExecutors(numAdditionalExecutors: Int): Boolean = { schedulerBackend match { - case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) - case _ => logWarning("Requesting executors is only supported in coarse-grained mode") + case b: CoarseGrainedSchedulerBackend => + b.requestExecutors(numAdditionalExecutors) + case _ => + logWarning("Requesting executors is only supported in coarse-grained mode") + false } } /** * :: DeveloperApi :: * Request that the cluster manager kill the specified executors. - * This is currently only supported in Yarn mode. + * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutors(executorIds: Seq[String]): Unit = { + def killExecutors(executorIds: Seq[String]): Boolean = { schedulerBackend match { - case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) - case _ => logWarning("Killing executors is only supported in coarse-grained mode") + case b: CoarseGrainedSchedulerBackend => + b.killExecutors(executorIds) + case _ => + logWarning("Killing executors is only supported in coarse-grained mode") + false } } /** * :: DeveloperApi :: * Request that cluster manager the kill the specified executor. - * This is currently only supported in Yarn mode. + * This is currently only supported in Yarn mode. Return whether the request is received. */ @DeveloperApi - def killExecutor(executorId: String): Unit = killExecutors(Seq(executorId)) + def killExecutor(executorId: String): Boolean = killExecutors(Seq(executorId)) /** The version of Spark on which this application is running. */ def version = SPARK_VERSION diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala new file mode 100644 index 0000000000000..f0aa914cfedb6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -0,0 +1,662 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import org.scalatest.{FunSuite, PrivateMethodTester} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.scheduler._ +import org.apache.spark.storage.BlockManagerId + +/** + * Test add and remove behavior of ExecutorAllocationManager. + */ +class ExecutorAllocationManagerSuite extends FunSuite { + import ExecutorAllocationManager._ + import ExecutorAllocationManagerSuite._ + + test("verify min/max executors") { + // No min or max + val conf = new SparkConf() + .setMaster("local") + .setAppName("test-executor-allocation-manager") + .set("spark.dynamicAllocation.enabled", "true") + intercept[SparkException] { new SparkContext(conf) } + + // Only min + val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") + intercept[SparkException] { new SparkContext(conf1) } + + // Only max + val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") + intercept[SparkException] { new SparkContext(conf2) } + + // Both min and max, but min > max + intercept[SparkException] { createSparkContext(2, 1) } + + // Both min and max, and min == max + val sc1 = createSparkContext(1, 1) + assert(sc1.executorAllocationManager.isDefined) + sc1.stop() + + // Both min and max, and min < max + val sc2 = createSparkContext(1, 2) + assert(sc2.executorAllocationManager.isDefined) + sc2.stop() + } + + test("starting state") { + val sc = createSparkContext() + val manager = sc.executorAllocationManager.get + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + assert(executorIds(manager).isEmpty) + assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) + assert(removeTimes(manager).isEmpty) + sc.stop() + } + + test("add executors") { + val sc = createSparkContext(1, 10) + val manager = sc.executorAllocationManager.get + + // Keep adding until the limit is reached + assert(numExecutorsPending(manager) === 0) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 1) + assert(numExecutorsPending(manager) === 1) + assert(numExecutorsToAdd(manager) === 2) + assert(addExecutors(manager) === 2) + assert(numExecutorsPending(manager) === 3) + assert(numExecutorsToAdd(manager) === 4) + assert(addExecutors(manager) === 4) + assert(numExecutorsPending(manager) === 7) + assert(numExecutorsToAdd(manager) === 8) + assert(addExecutors(manager) === 3) // reached the limit of 10 + assert(numExecutorsPending(manager) === 10) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 0) + assert(numExecutorsPending(manager) === 10) + assert(numExecutorsToAdd(manager) === 1) + + // Register previously requested executors + onExecutorAdded(manager, "first") + assert(numExecutorsPending(manager) === 9) + onExecutorAdded(manager, "second") + onExecutorAdded(manager, "third") + onExecutorAdded(manager, "fourth") + assert(numExecutorsPending(manager) === 6) + onExecutorAdded(manager, "first") // duplicates should not count + onExecutorAdded(manager, "second") + assert(numExecutorsPending(manager) === 6) + + // Try adding again + // This should still fail because the number pending + running is still at the limit + assert(addExecutors(manager) === 0) + assert(numExecutorsPending(manager) === 6) + assert(numExecutorsToAdd(manager) === 1) + assert(addExecutors(manager) === 0) + assert(numExecutorsPending(manager) === 6) + assert(numExecutorsToAdd(manager) === 1) + sc.stop() + } + + test("remove executors") { + val sc = createSparkContext(5, 10) + val manager = sc.executorAllocationManager.get + (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } + + // Keep removing until the limit is reached + assert(executorsPendingToRemove(manager).isEmpty) + assert(removeExecutor(manager, "1")) + assert(executorsPendingToRemove(manager).size === 1) + assert(executorsPendingToRemove(manager).contains("1")) + assert(removeExecutor(manager, "2")) + assert(removeExecutor(manager, "3")) + assert(executorsPendingToRemove(manager).size === 3) + assert(executorsPendingToRemove(manager).contains("2")) + assert(executorsPendingToRemove(manager).contains("3")) + assert(!removeExecutor(manager, "100")) // remove non-existent executors + assert(!removeExecutor(manager, "101")) + assert(executorsPendingToRemove(manager).size === 3) + assert(removeExecutor(manager, "4")) + assert(removeExecutor(manager, "5")) + assert(!removeExecutor(manager, "6")) // reached the limit of 5 + assert(executorsPendingToRemove(manager).size === 5) + assert(executorsPendingToRemove(manager).contains("4")) + assert(executorsPendingToRemove(manager).contains("5")) + assert(!executorsPendingToRemove(manager).contains("6")) + + // Kill executors previously requested to remove + onExecutorRemoved(manager, "1") + assert(executorsPendingToRemove(manager).size === 4) + assert(!executorsPendingToRemove(manager).contains("1")) + onExecutorRemoved(manager, "2") + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 2) + assert(!executorsPendingToRemove(manager).contains("2")) + assert(!executorsPendingToRemove(manager).contains("3")) + onExecutorRemoved(manager, "2") // duplicates should not count + onExecutorRemoved(manager, "3") + assert(executorsPendingToRemove(manager).size === 2) + onExecutorRemoved(manager, "4") + onExecutorRemoved(manager, "5") + assert(executorsPendingToRemove(manager).isEmpty) + + // Try removing again + // This should still fail because the number pending + running is still at the limit + assert(!removeExecutor(manager, "7")) + assert(executorsPendingToRemove(manager).isEmpty) + assert(!removeExecutor(manager, "8")) + assert(executorsPendingToRemove(manager).isEmpty) + sc.stop() + } + + test ("interleaving add and remove") { + val sc = createSparkContext(5, 10) + val manager = sc.executorAllocationManager.get + + // Add a few executors + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 2) + assert(addExecutors(manager) === 4) + onExecutorAdded(manager, "1") + onExecutorAdded(manager, "2") + onExecutorAdded(manager, "3") + onExecutorAdded(manager, "4") + onExecutorAdded(manager, "5") + onExecutorAdded(manager, "6") + onExecutorAdded(manager, "7") + assert(executorIds(manager).size === 7) + + // Remove until limit + assert(removeExecutor(manager, "1")) + assert(removeExecutor(manager, "2")) + assert(!removeExecutor(manager, "3")) // lower limit reached + assert(!removeExecutor(manager, "4")) + onExecutorRemoved(manager, "1") + onExecutorRemoved(manager, "2") + assert(executorIds(manager).size === 5) + + // Add until limit + assert(addExecutors(manager) === 5) // upper limit reached + assert(addExecutors(manager) === 0) + assert(!removeExecutor(manager, "3")) // still at lower limit + assert(!removeExecutor(manager, "4")) + onExecutorAdded(manager, "8") + onExecutorAdded(manager, "9") + onExecutorAdded(manager, "10") + onExecutorAdded(manager, "11") + onExecutorAdded(manager, "12") + assert(executorIds(manager).size === 10) + + // Remove succeeds again, now that we are no longer at the lower limit + assert(removeExecutor(manager, "3")) + assert(removeExecutor(manager, "4")) + assert(removeExecutor(manager, "5")) + assert(removeExecutor(manager, "6")) + assert(executorIds(manager).size === 10) + assert(addExecutors(manager) === 0) // still at upper limit + onExecutorRemoved(manager, "3") + onExecutorRemoved(manager, "4") + assert(executorIds(manager).size === 8) + + // Add succeeds again, now that we are no longer at the upper limit + // Number of executors added restarts at 1 + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 1) // upper limit reached again + assert(addExecutors(manager) === 0) + assert(executorIds(manager).size === 8) + onExecutorRemoved(manager, "5") + onExecutorRemoved(manager, "6") + onExecutorAdded(manager, "13") + onExecutorAdded(manager, "14") + assert(executorIds(manager).size === 8) + assert(addExecutors(manager) === 1) + assert(addExecutors(manager) === 1) // upper limit reached again + assert(addExecutors(manager) === 0) + onExecutorAdded(manager, "15") + onExecutorAdded(manager, "16") + assert(executorIds(manager).size === 10) + sc.stop() + } + + test("starting/canceling add timer") { + val sc = createSparkContext(2, 10) + val clock = new TestClock(8888L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Starting add timer is idempotent + assert(addTime(manager) === NOT_SET) + onSchedulerBacklogged(manager) + val firstAddTime = addTime(manager) + assert(firstAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) + clock.tick(100L) + onSchedulerBacklogged(manager) + assert(addTime(manager) === firstAddTime) // timer is already started + clock.tick(200L) + onSchedulerBacklogged(manager) + assert(addTime(manager) === firstAddTime) + onSchedulerQueueEmpty(manager) + + // Restart add timer + clock.tick(1000L) + assert(addTime(manager) === NOT_SET) + onSchedulerBacklogged(manager) + val secondAddTime = addTime(manager) + assert(secondAddTime === clock.getTimeMillis + schedulerBacklogTimeout * 1000) + clock.tick(100L) + onSchedulerBacklogged(manager) + assert(addTime(manager) === secondAddTime) // timer is already started + assert(addTime(manager) !== firstAddTime) + assert(firstAddTime !== secondAddTime) + } + + test("starting/canceling remove timers") { + val sc = createSparkContext(2, 10) + val clock = new TestClock(14444L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Starting remove timer is idempotent for each executor + assert(removeTimes(manager).isEmpty) + onExecutorIdle(manager, "1") + assert(removeTimes(manager).size === 1) + assert(removeTimes(manager).contains("1")) + val firstRemoveTime = removeTimes(manager)("1") + assert(firstRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) + clock.tick(100L) + onExecutorIdle(manager, "1") + assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started + clock.tick(200L) + onExecutorIdle(manager, "1") + assert(removeTimes(manager)("1") === firstRemoveTime) + clock.tick(300L) + onExecutorIdle(manager, "2") + assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor + assert(removeTimes(manager)("2") === clock.getTimeMillis + executorIdleTimeout * 1000) + clock.tick(400L) + onExecutorIdle(manager, "3") + assert(removeTimes(manager)("3") !== firstRemoveTime) + assert(removeTimes(manager)("3") === clock.getTimeMillis + executorIdleTimeout * 1000) + assert(removeTimes(manager).size === 3) + assert(removeTimes(manager).contains("2")) + assert(removeTimes(manager).contains("3")) + + // Restart remove timer + clock.tick(1000L) + onExecutorBusy(manager, "1") + assert(removeTimes(manager).size === 2) + onExecutorIdle(manager, "1") + assert(removeTimes(manager).size === 3) + assert(removeTimes(manager).contains("1")) + val secondRemoveTime = removeTimes(manager)("1") + assert(secondRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) + assert(removeTimes(manager)("1") === secondRemoveTime) // timer is already started + assert(removeTimes(manager)("1") !== firstRemoveTime) + assert(firstRemoveTime !== secondRemoveTime) + } + + test("mock polling loop with no events") { + val sc = createSparkContext(1, 20) + val manager = sc.executorAllocationManager.get + val clock = new TestClock(2020L) + manager.setClock(clock) + + // No events - we should not be adding or removing + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(100L) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(1000L) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(10000L) + schedule(manager) + assert(numExecutorsPending(manager) === 0) + assert(executorsPendingToRemove(manager).isEmpty) + } + + test("mock polling loop add behavior") { + val sc = createSparkContext(1, 20) + val clock = new TestClock(2020L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Scheduler queue backlogged + onSchedulerBacklogged(manager) + clock.tick(schedulerBacklogTimeout * 1000 / 2) + schedule(manager) + assert(numExecutorsPending(manager) === 0) // timer not exceeded yet + clock.tick(schedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 1) // first timer exceeded + clock.tick(sustainedSchedulerBacklogTimeout * 1000 / 2) + schedule(manager) + assert(numExecutorsPending(manager) === 1) // second timer not exceeded yet + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 1 + 2) // second timer exceeded + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 1 + 2 + 4) // third timer exceeded + + // Scheduler queue drained + onSchedulerQueueEmpty(manager) + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7) // timer is canceled + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7) + + // Scheduler queue backlogged again + onSchedulerBacklogged(manager) + clock.tick(schedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7 + 1) // timer restarted + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7 + 1 + 2) + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 7 + 1 + 2 + 4) + clock.tick(sustainedSchedulerBacklogTimeout * 1000) + schedule(manager) + assert(numExecutorsPending(manager) === 20) // limit reached + } + + test("mock polling loop remove behavior") { + val sc = createSparkContext(1, 20) + val clock = new TestClock(2020L) + val manager = sc.executorAllocationManager.get + manager.setClock(clock) + + // Remove idle executors on timeout + onExecutorAdded(manager, "executor-1") + onExecutorAdded(manager, "executor-2") + onExecutorAdded(manager, "executor-3") + assert(removeTimes(manager).size === 3) + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(executorIdleTimeout * 1000 / 2) + schedule(manager) + assert(removeTimes(manager).size === 3) // idle threshold not reached yet + assert(executorsPendingToRemove(manager).isEmpty) + clock.tick(executorIdleTimeout * 1000) + schedule(manager) + assert(removeTimes(manager).isEmpty) // idle threshold exceeded + assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) + + // Mark a subset as busy - only idle executors should be removed + onExecutorAdded(manager, "executor-4") + onExecutorAdded(manager, "executor-5") + onExecutorAdded(manager, "executor-6") + onExecutorAdded(manager, "executor-7") + assert(removeTimes(manager).size === 5) // 5 active executors + assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed + onExecutorBusy(manager, "executor-4") + onExecutorBusy(manager, "executor-5") + onExecutorBusy(manager, "executor-6") // 3 busy and 2 idle (of the 5 active ones) + schedule(manager) + assert(removeTimes(manager).size === 2) // remove only idle executors + assert(!removeTimes(manager).contains("executor-4")) + assert(!removeTimes(manager).contains("executor-5")) + assert(!removeTimes(manager).contains("executor-6")) + assert(executorsPendingToRemove(manager).size === 2) + clock.tick(executorIdleTimeout * 1000) + schedule(manager) + assert(removeTimes(manager).isEmpty) // idle executors are removed + assert(executorsPendingToRemove(manager).size === 4) + assert(!executorsPendingToRemove(manager).contains("executor-4")) + assert(!executorsPendingToRemove(manager).contains("executor-5")) + assert(!executorsPendingToRemove(manager).contains("executor-6")) + + // Busy executors are now idle and should be removed + onExecutorIdle(manager, "executor-4") + onExecutorIdle(manager, "executor-5") + onExecutorIdle(manager, "executor-6") + schedule(manager) + assert(removeTimes(manager).size === 3) // 0 busy and 3 idle + assert(removeTimes(manager).contains("executor-4")) + assert(removeTimes(manager).contains("executor-5")) + assert(removeTimes(manager).contains("executor-6")) + assert(executorsPendingToRemove(manager).size === 4) + clock.tick(executorIdleTimeout * 1000) + schedule(manager) + assert(removeTimes(manager).isEmpty) + assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining) + } + + test("listeners trigger add executors correctly") { + val sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(addTime(manager) === NOT_SET) + + // Starting a stage should start the add timer + val numTasks = 10 + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(0, numTasks))) + assert(addTime(manager) !== NOT_SET) + + // Starting a subset of the tasks should not cancel the add timer + val taskInfos = (0 to numTasks - 1).map { i => createTaskInfo(i, i, "executor-1") } + taskInfos.tail.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, info)) } + assert(addTime(manager) !== NOT_SET) + + // Starting all remaining tasks should cancel the add timer + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, taskInfos.head)) + assert(addTime(manager) === NOT_SET) + + // Start two different stages + // The add timer should be canceled only if all tasks in both stages start running + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(1, numTasks))) + sc.listenerBus.postToAll(SparkListenerStageSubmitted(createStageInfo(2, numTasks))) + assert(addTime(manager) !== NOT_SET) + taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(1, 0, info)) } + assert(addTime(manager) !== NOT_SET) + taskInfos.foreach { info => sc.listenerBus.postToAll(SparkListenerTaskStart(2, 0, info)) } + assert(addTime(manager) === NOT_SET) + } + + test("listeners trigger remove executors correctly") { + val sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(removeTimes(manager).isEmpty) + + // Added executors should start the remove timers for each executor + (1 to 5).map("executor-" + _).foreach { id => onExecutorAdded(manager, id) } + assert(removeTimes(manager).size === 5) + + // Starting a task cancel the remove timer for that executor + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1"))) + sc.listenerBus.postToAll(SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2"))) + assert(removeTimes(manager).size === 3) + assert(!removeTimes(manager).contains("executor-1")) + assert(!removeTimes(manager).contains("executor-2")) + + // Finishing all tasks running on an executor should start the remove timer for that executor + sc.listenerBus.postToAll(SparkListenerTaskEnd( + 0, 0, "task-type", Success, createTaskInfo(0, 0, "executor-1"), new TaskMetrics)) + sc.listenerBus.postToAll(SparkListenerTaskEnd( + 0, 0, "task-type", Success, createTaskInfo(2, 2, "executor-2"), new TaskMetrics)) + assert(removeTimes(manager).size === 4) + assert(!removeTimes(manager).contains("executor-1")) // executor-1 has not finished yet + assert(removeTimes(manager).contains("executor-2")) + sc.listenerBus.postToAll(SparkListenerTaskEnd( + 0, 0, "task-type", Success, createTaskInfo(1, 1, "executor-1"), new TaskMetrics)) + assert(removeTimes(manager).size === 5) + assert(removeTimes(manager).contains("executor-1")) // executor-1 has now finished + } + + test("listeners trigger add and remove executor callbacks correctly") { + val sc = createSparkContext(2, 10) + val manager = sc.executorAllocationManager.get + assert(executorIds(manager).isEmpty) + assert(removeTimes(manager).isEmpty) + + // New executors have registered + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-1", "host1", 1), 100L)) + assert(executorIds(manager).size === 1) + assert(executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 1) + assert(removeTimes(manager).contains("executor-1")) + sc.listenerBus.postToAll(SparkListenerBlockManagerAdded( + 0L, BlockManagerId("executor-2", "host2", 1), 100L)) + assert(executorIds(manager).size === 2) + assert(executorIds(manager).contains("executor-2")) + assert(removeTimes(manager).size === 2) + assert(removeTimes(manager).contains("executor-2")) + + // Existing executors have disconnected + sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( + 0L, BlockManagerId("executor-1", "host1", 1))) + assert(executorIds(manager).size === 1) + assert(!executorIds(manager).contains("executor-1")) + assert(removeTimes(manager).size === 1) + assert(!removeTimes(manager).contains("executor-1")) + + // Unknown executor has disconnected + sc.listenerBus.postToAll(SparkListenerBlockManagerRemoved( + 0L, BlockManagerId("executor-3", "host3", 1))) + assert(executorIds(manager).size === 1) + assert(removeTimes(manager).size === 1) + } + +} + +/** + * Helper methods for testing ExecutorAllocationManager. + * This includes methods to access private methods and fields in ExecutorAllocationManager. + */ +private object ExecutorAllocationManagerSuite extends PrivateMethodTester { + private val schedulerBacklogTimeout = 1L + private val sustainedSchedulerBacklogTimeout = 2L + private val executorIdleTimeout = 3L + + private def createSparkContext(minExecutors: Int = 1, maxExecutors: Int = 5): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test-executor-allocation-manager") + .set("spark.dynamicAllocation.enabled", "true") + .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) + .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) + .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", + sustainedSchedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) + .set("spark.dynamicAllocation.testing", "true") + new SparkContext(conf) + } + + private def createStageInfo(stageId: Int, numTasks: Int): StageInfo = { + new StageInfo(stageId, 0, "name", numTasks, Seq.empty, "no details") + } + + private def createTaskInfo(taskId: Int, taskIndex: Int, executorId: String): TaskInfo = { + new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative = false) + } + + /* ------------------------------------------------------- * + | Helper methods for accessing private methods and fields | + * ------------------------------------------------------- */ + + private val _numExecutorsToAdd = PrivateMethod[Int]('numExecutorsToAdd) + private val _numExecutorsPending = PrivateMethod[Int]('numExecutorsPending) + private val _executorsPendingToRemove = + PrivateMethod[collection.Set[String]]('executorsPendingToRemove) + private val _executorIds = PrivateMethod[collection.Set[String]]('executorIds) + private val _addTime = PrivateMethod[Long]('addTime) + private val _removeTimes = PrivateMethod[collection.Map[String, Long]]('removeTimes) + private val _schedule = PrivateMethod[Unit]('schedule) + private val _addExecutors = PrivateMethod[Int]('addExecutors) + private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor) + private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded) + private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved) + private val _onSchedulerBacklogged = PrivateMethod[Unit]('onSchedulerBacklogged) + private val _onSchedulerQueueEmpty = PrivateMethod[Unit]('onSchedulerQueueEmpty) + private val _onExecutorIdle = PrivateMethod[Unit]('onExecutorIdle) + private val _onExecutorBusy = PrivateMethod[Unit]('onExecutorBusy) + + private def numExecutorsToAdd(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _numExecutorsToAdd() + } + + private def numExecutorsPending(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _numExecutorsPending() + } + + private def executorsPendingToRemove( + manager: ExecutorAllocationManager): collection.Set[String] = { + manager invokePrivate _executorsPendingToRemove() + } + + private def executorIds(manager: ExecutorAllocationManager): collection.Set[String] = { + manager invokePrivate _executorIds() + } + + private def addTime(manager: ExecutorAllocationManager): Long = { + manager invokePrivate _addTime() + } + + private def removeTimes(manager: ExecutorAllocationManager): collection.Map[String, Long] = { + manager invokePrivate _removeTimes() + } + + private def schedule(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _schedule() + } + + private def addExecutors(manager: ExecutorAllocationManager): Int = { + manager invokePrivate _addExecutors() + } + + private def removeExecutor(manager: ExecutorAllocationManager, id: String): Boolean = { + manager invokePrivate _removeExecutor(id) + } + + private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorAdded(id) + } + + private def onExecutorRemoved(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorRemoved(id) + } + + private def onSchedulerBacklogged(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _onSchedulerBacklogged() + } + + private def onSchedulerQueueEmpty(manager: ExecutorAllocationManager): Unit = { + manager invokePrivate _onSchedulerQueueEmpty() + } + + private def onExecutorIdle(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorIdle(id) + } + + private def onExecutorBusy(manager: ExecutorAllocationManager, id: String): Unit = { + manager invokePrivate _onExecutorBusy(id) + } +} diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 68073798886dd..e90672c004d4b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -505,7 +505,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, driver ! x case RequestExecutors(requestedTotal) => - logInfo(s"Driver requested a total number of executors of $requestedTotal.") + logInfo(s"Driver requested a total number of $requestedTotal executor(s).") Option(allocator) match { case Some(a) => a.requestTotalExecutors(requestedTotal) case None => logWarning("Container allocator is not ready to request executors yet.") From 1234258077b1f4050845e9fb73066b37f981c72a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 29 Oct 2014 17:59:16 -0700 Subject: [PATCH 1133/1492] [SPARK-4053][Streaming] Made the ReceiverSuite test more reliable, by fixing block generator throttling In the unit test that checked whether blocks generated by throttled block generator had expected number of records, the thresholds are too tight, which sometimes led to the test failing. This PR fixes it by relaxing the thresholds and the time intervals for testing. Author: Tathagata Das Closes #2900 from tdas/receiver-suite-flakiness and squashes the following commits: 28508a2 [Tathagata Das] Made the ReceiverSuite test more reliable --- ...eceiverSuite.scala => ReceiverSuite.scala} | 44 +++++++++++++------ 1 file changed, 30 insertions(+), 14 deletions(-) rename streaming/src/test/scala/org/apache/spark/streaming/{NetworkReceiverSuite.scala => ReceiverSuite.scala} (86%) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala similarity index 86% rename from streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala rename to streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index eb6e88cf5520d..0f6a9489dbe0d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -31,9 +31,9 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ /** Testsuite for testing the network receiver behavior */ -class NetworkReceiverSuite extends FunSuite with Timeouts { +class ReceiverSuite extends FunSuite with Timeouts { - test("network receiver life cycle") { + test("receiver life cycle") { val receiver = new FakeReceiver val executor = new FakeReceiverSupervisor(receiver) @@ -152,8 +152,8 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { test("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 50 - val maxRate = 200 + val blockInterval = 100 + val maxRate = 100 val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) @@ -175,19 +175,35 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { } blockGenerator.stop() - val recordedData = blockGeneratorListener.arrayBuffers - assert(blockGeneratorListener.arrayBuffers.size > 0) - assert(recordedData.flatten.toSet === generatedData.toSet) + val recordedBlocks = blockGeneratorListener.arrayBuffers + val recordedData = recordedBlocks.flatten + assert(blockGeneratorListener.arrayBuffers.size > 0, "No blocks received") + assert(recordedData.toSet === generatedData.toSet, "Received data not same") + // recordedData size should be close to the expected rate - assert(recordedData.flatten.size >= expectedMessages * 0.9 && - recordedData.flatten.size <= expectedMessages * 1.1 ) - // the first and last block may be incomplete, so we slice them out - recordedData.slice(1, recordedData.size - 1).foreach { block => - assert(block.size >= expectedMessagesPerBlock * 0.8 && - block.size <= expectedMessagesPerBlock * 1.2 ) - } + val minExpectedMessages = expectedMessages - 3 + val maxExpectedMessages = expectedMessages + 1 + val numMessages = recordedData.size + assert( + numMessages >= minExpectedMessages && numMessages <= maxExpectedMessages, + s"#records received = $numMessages, not between $minExpectedMessages and $maxExpectedMessages" + ) + + val minExpectedMessagesPerBlock = expectedMessagesPerBlock - 3 + val maxExpectedMessagesPerBlock = expectedMessagesPerBlock + 1 + val receivedBlockSizes = recordedBlocks.map { _.size }.mkString(",") + println(minExpectedMessagesPerBlock, maxExpectedMessagesPerBlock, ":", receivedBlockSizes) + assert( + // the first and last block may be incomplete, so we slice them out + recordedBlocks.drop(1).dropRight(1).forall { block => + block.size >= minExpectedMessagesPerBlock && block.size <= maxExpectedMessagesPerBlock + }, + s"# records in received blocks = [$receivedBlockSizes], not between " + + s"$minExpectedMessagesPerBlock and $maxExpectedMessagesPerBlock" + ) } + /** * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. */ From cd739bd756875bd52e9bd8ae801e0ae10a1f6937 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Wed, 29 Oct 2014 23:02:58 -0700 Subject: [PATCH 1134/1492] [SPARK-1720][SPARK-1719] use LD_LIBRARY_PATH instead of -Djava.library.path - [X] Standalone - [X] YARN - [X] Mesos - [X] Mac OS X - [X] Linux - [ ] Windows This is another implementation about #1031 Author: GuoQiang Li Closes #2711 from witgo/SPARK-1719 and squashes the following commits: c7b26f6 [GuoQiang Li] review commits 4488e41 [GuoQiang Li] Refactoring CommandUtils a444094 [GuoQiang Li] review commits 40c0b4a [GuoQiang Li] Add buildLocalCommand method c1a0ddd [GuoQiang Li] fix comments 156ce88 [GuoQiang Li] review commit 38aa377 [GuoQiang Li] Refactor CommandUtils.scala 4269e00 [GuoQiang Li] Refactor SparkSubmitDriverBootstrapper.scala 7a1d634 [GuoQiang Li] use LD_LIBRARY_PATH instead of -Djava.library.path --- bin/spark-class | 6 +- .../scala/org/apache/spark/SparkConf.scala | 13 ++++ .../SparkSubmitDriverBootstrapper.scala | 17 ++--- .../spark/deploy/worker/CommandUtils.scala | 68 ++++++++++++++++--- .../spark/deploy/worker/DriverRunner.scala | 23 ++----- .../spark/deploy/worker/ExecutorRunner.scala | 26 +++---- .../mesos/CoarseMesosSchedulerBackend.scala | 22 +++--- .../cluster/mesos/MesosSchedulerBackend.scala | 18 ++--- .../scala/org/apache/spark/util/Utils.scala | 42 +++++++++++- .../spark/deploy/CommandUtilsSuite.scala | 37 ++++++++++ .../deploy/worker/ExecutorRunnerTest.scala | 5 +- .../apache/spark/deploy/yarn/ClientBase.scala | 14 +++- .../deploy/yarn/ExecutorRunnableUtil.scala | 11 ++- 13 files changed, 221 insertions(+), 81 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala diff --git a/bin/spark-class b/bin/spark-class index 91d858bc063d0..925367b0dd187 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -81,7 +81,11 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_SUBMIT_OPTS" OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} if [ -n "$SPARK_SUBMIT_LIBRARY_PATH" ]; then - OUR_JAVA_OPTS="$OUR_JAVA_OPTS -Djava.library.path=$SPARK_SUBMIT_LIBRARY_PATH" + if [[ $OSTYPE == darwin* ]]; then + export DYLD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$DYLD_LIBRARY_PATH" + else + export LD_LIBRARY_PATH="$SPARK_SUBMIT_LIBRARY_PATH:$LD_LIBRARY_PATH" + fi fi if [ -n "$SPARK_SUBMIT_DRIVER_MEMORY" ]; then OUR_JAVA_MEM="$SPARK_SUBMIT_DRIVER_MEMORY" diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index dbbcc23305c50..ad0a9017afead 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -244,6 +244,19 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { val executorClasspathKey = "spark.executor.extraClassPath" val driverOptsKey = "spark.driver.extraJavaOptions" val driverClassPathKey = "spark.driver.extraClassPath" + val driverLibraryPathKey = "spark.driver.extraLibraryPath" + + // Used by Yarn in 1.1 and before + sys.props.get("spark.driver.libraryPath").foreach { value => + val warning = + s""" + |spark.driver.libraryPath was detected (set to '$value'). + |This is deprecated in Spark 1.2+. + | + |Please instead use: $driverLibraryPathKey + """.stripMargin + logWarning(warning) + } // Validate spark.executor.extraJavaOptions settings.get(executorOptsKey).map { javaOpts => diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index 0125330589da5..2b894a796c8c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -82,17 +82,8 @@ private[spark] object SparkSubmitDriverBootstrapper { .orElse(confDriverMemory) .getOrElse(defaultDriverMemory) - val newLibraryPath = - if (submitLibraryPath.isDefined) { - // SPARK_SUBMIT_LIBRARY_PATH is already captured in JAVA_OPTS - "" - } else { - confLibraryPath.map("-Djava.library.path=" + _).getOrElse("") - } - val newClasspath = if (submitClasspath.isDefined) { - // SPARK_SUBMIT_CLASSPATH is already captured in CLASSPATH classpath } else { classpath + confClasspath.map(sys.props("path.separator") + _).getOrElse("") @@ -114,7 +105,6 @@ private[spark] object SparkSubmitDriverBootstrapper { val command: Seq[String] = Seq(runner) ++ Seq("-cp", newClasspath) ++ - Seq(newLibraryPath) ++ filteredJavaOpts ++ Seq(s"-Xms$newDriverMemory", s"-Xmx$newDriverMemory") ++ Seq("org.apache.spark.deploy.SparkSubmit") ++ @@ -130,6 +120,13 @@ private[spark] object SparkSubmitDriverBootstrapper { // Start the driver JVM val filteredCommand = command.filter(_.nonEmpty) val builder = new ProcessBuilder(filteredCommand) + val env = builder.environment() + + if (submitLibraryPath.isEmpty && confLibraryPath.nonEmpty) { + val libraryPaths = confLibraryPath ++ sys.env.get(Utils.libraryPathEnvName) + env.put(Utils.libraryPathEnvName, libraryPaths.mkString(sys.props("path.separator"))) + } + val process = builder.start() // Redirect stdout and stderr from the child JVM diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 2e9be2a180c68..aba2e20118d7a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -20,6 +20,8 @@ package org.apache.spark.deploy.worker import java.io.{File, FileOutputStream, InputStream, IOException} import java.lang.System._ +import scala.collection.Map + import org.apache.spark.Logging import org.apache.spark.deploy.Command import org.apache.spark.util.Utils @@ -29,7 +31,29 @@ import org.apache.spark.util.Utils */ private[spark] object CommandUtils extends Logging { - def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { + + /** + * Build a ProcessBuilder based on the given parameters. + * The `env` argument is exposed for testing. + */ + def buildProcessBuilder( + command: Command, + memory: Int, + sparkHome: String, + substituteArguments: String => String, + classPaths: Seq[String] = Seq[String](), + env: Map[String, String] = sys.env): ProcessBuilder = { + val localCommand = buildLocalCommand(command, substituteArguments, classPaths, env) + val commandSeq = buildCommandSeq(localCommand, memory, sparkHome) + val builder = new ProcessBuilder(commandSeq: _*) + val environment = builder.environment() + for ((key, value) <- localCommand.environment) { + environment.put(key, value) + } + builder + } + + private def buildCommandSeq(command: Command, memory: Int, sparkHome: String): Seq[String] = { val runner = sys.env.get("JAVA_HOME").map(_ + "/bin/java").getOrElse("java") // SPARK-698: do not call the run.cmd script, as process.destroy() @@ -38,11 +62,41 @@ object CommandUtils extends Logging { command.arguments } + /** + * Build a command based on the given one, taking into account the local environment + * of where this command is expected to run, substitute any placeholders, and append + * any extra class paths. + */ + private def buildLocalCommand( + command: Command, + substituteArguments: String => String, + classPath: Seq[String] = Seq[String](), + env: Map[String, String]): Command = { + val libraryPathName = Utils.libraryPathEnvName + val libraryPathEntries = command.libraryPathEntries + val cmdLibraryPath = command.environment.get(libraryPathName) + + val newEnvironment = if (libraryPathEntries.nonEmpty && libraryPathName.nonEmpty) { + val libraryPaths = libraryPathEntries ++ cmdLibraryPath ++ env.get(libraryPathName) + command.environment + ((libraryPathName, libraryPaths.mkString(File.pathSeparator))) + } else { + command.environment + } + + Command( + command.mainClass, + command.arguments.map(substituteArguments), + newEnvironment, + command.classPathEntries ++ classPath, + Seq[String](), // library path already captured in environment variable + command.javaOpts) + } + /** * Attention: this must always be aligned with the environment variables in the run scripts and * the way the JAVA_OPTS are assembled there. */ - def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { + private def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") // Exists for backwards compatibility with older Spark versions @@ -53,14 +107,6 @@ object CommandUtils extends Logging { logWarning("Set SPARK_LOCAL_DIRS for node-specific storage locations.") } - val libraryOpts = - if (command.libraryPathEntries.size > 0) { - val joined = command.libraryPathEntries.mkString(File.pathSeparator) - Seq(s"-Djava.library.path=$joined") - } else { - Seq() - } - // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( @@ -71,7 +117,7 @@ object CommandUtils extends Logging { val javaVersion = System.getProperty("java.version") val permGenOpt = if (!javaVersion.startsWith("1.8")) Some("-XX:MaxPermSize=128m") else None Seq("-cp", userClassPath.filterNot(_.isEmpty).mkString(File.pathSeparator)) ++ - permGenOpt ++ libraryOpts ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts + permGenOpt ++ workerLocalOpts ++ command.javaOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index 3bf0b9492df0a..28cab36c7b9e2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -76,17 +76,9 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val classPath = driverDesc.command.classPathEntries ++ Seq(s"$localJarFilename") - val newCommand = Command( - driverDesc.command.mainClass, - driverDesc.command.arguments.map(substituteVariables), - driverDesc.command.environment, - classPath, - driverDesc.command.libraryPathEntries, - driverDesc.command.javaOpts) - val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, - sparkHome.getAbsolutePath) - launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) + val builder = CommandUtils.buildProcessBuilder(driverDesc.command, driverDesc.mem, + sparkHome.getAbsolutePath, substituteVariables, Seq(localJarFilename)) + launchDriver(builder, driverDir, driverDesc.supervise) } catch { case e: Exception => finalException = Some(e) @@ -165,11 +157,8 @@ private[spark] class DriverRunner( localJarFilename } - private def launchDriver(command: Seq[String], envVars: Map[String, String], baseDir: File, - supervise: Boolean) { - val builder = new ProcessBuilder(command: _*).directory(baseDir) - envVars.map{ case(k,v) => builder.environment().put(k, v) } - + private def launchDriver(builder: ProcessBuilder, baseDir: File, supervise: Boolean) { + builder.directory(baseDir) def initialize(process: Process) = { // Redirect stdout and stderr to files val stdout = new File(baseDir, "stdout") @@ -177,7 +166,7 @@ private[spark] class DriverRunner( val stderr = new File(baseDir, "stderr") val header = "Launch Command: %s\n%s\n\n".format( - command.mkString("\"", "\" \"", "\""), "=" * 40) + builder.command.mkString("\"", "\" \"", "\""), "=" * 40) Files.append(header, stderr, UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 030a651469b64..8ba6a01bbcb97 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.worker import java.io._ +import scala.collection.JavaConversions._ + import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files @@ -115,33 +117,21 @@ private[spark] class ExecutorRunner( case other => other } - def getCommandSeq = { - val command = Command( - appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables), - appDesc.command.environment, - appDesc.command.classPathEntries, - appDesc.command.libraryPathEntries, - appDesc.command.javaOpts) - CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) - } - /** * Download and run the executor described in our ApplicationDescription */ def fetchAndRunExecutor() { try { // Launch the process - val command = getCommandSeq + val builder = CommandUtils.buildProcessBuilder(appDesc.command, memory, + sparkHome.getAbsolutePath, substituteVariables) + val command = builder.command() logInfo("Launch command: " + command.mkString("\"", "\" \"", "\"")) - val builder = new ProcessBuilder(command: _*).directory(executorDir) - val env = builder.environment() - for ((key, value) <- appDesc.command.environment) { - env.put(key, value) - } + + builder.directory(executorDir) // In case we are running this from within the Spark Shell, avoid creating a "scala" // parent process for the executor command - env.put("SPARK_LAUNCH_WITH_SCALA", "0") + builder.environment.put("SPARK_LAUNCH_WITH_SCALA", "0") process = builder.start() val header = "Spark Executor Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index d7f88de4b40aa..d8c0e2f66df01 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -31,6 +31,7 @@ import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, TaskState => MesosTas import org.apache.spark.{Logging, SparkContext, SparkEnv, SparkException} import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -120,16 +121,18 @@ private[spark] class CoarseMesosSchedulerBackend( environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions") + val extraJavaOpts = conf.get("spark.executor.extraJavaOptions", "") - val libraryPathOption = "spark.executor.extraLibraryPath" - val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") - val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + // Set the environment variable through a command prefix + // to append to the existing value of the variable + val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => + Utils.libraryPathEnvPrefix(Seq(p)) + }.getOrElse("") environment.addVariables( Environment.Variable.newBuilder() .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraOpts) + .setValue(extraJavaOpts) .build()) sc.executorEnvs.foreach { case (key, value) => @@ -150,16 +153,17 @@ private[spark] class CoarseMesosSchedulerBackend( if (uri == null) { val runScript = new File(executorSparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores, appId)) + "%s \"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s".format( + prefixEnv, runScript, driverUrl, offer.getSlaveId.getValue, + offer.getHostname, numCores, appId)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( - ("cd %s*; " + + ("cd %s*; %s " + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d %s") - .format(basename, driverUrl, offer.getSlaveId.getValue, + .format(basename, prefixEnv, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores, appId)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index e0f2fd622f54c..8e2faff90f9b2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -98,15 +98,16 @@ private[spark] class MesosSchedulerBackend( environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") - val extraLibraryPath = sc.conf.getOption("spark.executor.extraLibraryPath").map { lp => - s"-Djava.library.path=$lp" - } - val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").getOrElse("") + + val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => + Utils.libraryPathEnvPrefix(Seq(p)) + }.getOrElse("") + environment.addVariables( Environment.Variable.newBuilder() .setName("SPARK_EXECUTOR_OPTS") - .setValue(extraOpts) + .setValue(extraJavaOpts) .build()) sc.executorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() @@ -118,12 +119,13 @@ private[spark] class MesosSchedulerBackend( .setEnvironment(environment) val uri = sc.conf.get("spark.executor.uri", null) if (uri == null) { - command.setValue(new File(executorSparkHome, "/sbin/spark-executor").getCanonicalPath) + val executorPath = new File(executorSparkHome, "/sbin/spark-executor").getCanonicalPath + command.setValue("%s %s".format(prefixEnv, executorPath)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head - command.setValue("cd %s*; ./sbin/spark-executor".format(basename)) + command.setValue("cd %s*; %s ./sbin/spark-executor".format(basename, prefixEnv)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } val cpus = Resource.newBuilder() 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 0daab91143e47..063895d3c548d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -23,8 +23,6 @@ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} -import org.eclipse.jetty.util.MultiException - import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer @@ -39,6 +37,7 @@ import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration import org.apache.log4j.PropertyConfigurator import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.eclipse.jetty.util.MultiException import org.json4s._ import tachyon.client.{TachyonFile,TachyonFS} @@ -1381,6 +1380,11 @@ private[spark] object Utils extends Logging { */ val isWindows = SystemUtils.IS_OS_WINDOWS + /** + * Whether the underlying operating system is Mac OS X. + */ + val isMac = SystemUtils.IS_OS_MAC_OSX + /** * Pattern for matching a Windows drive, which contains only a single alphabet character. */ @@ -1714,6 +1718,40 @@ private[spark] object Utils extends Logging { method.invoke(obj, values.toSeq: _*) } + /** + * Return the current system LD_LIBRARY_PATH name + */ + def libraryPathEnvName: String = { + if (isWindows) { + "PATH" + } else if (isMac) { + "DYLD_LIBRARY_PATH" + } else { + "LD_LIBRARY_PATH" + } + } + + /** + * Return the prefix of a command that appends the given library paths to the + * system-specific library path environment variable. On Unix, for instance, + * this returns the string LD_LIBRARY_PATH="path1:path2:$LD_LIBRARY_PATH". + */ + def libraryPathEnvPrefix(libraryPaths: Seq[String]): String = { + val libraryPathScriptVar = if (isWindows) { + s"%${libraryPathEnvName}%" + } else { + "$" + libraryPathEnvName + } + val libraryPath = (libraryPaths :+ libraryPathScriptVar).mkString("\"", + File.pathSeparator, "\"") + val ampersand = if (Utils.isWindows) { + " &" + } else { + "" + } + s"$libraryPathEnvName=$libraryPath$ampersand" + } + } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala new file mode 100644 index 0000000000000..7915ee75d8778 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/CommandUtilsSuite.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import org.apache.spark.deploy.worker.CommandUtils +import org.apache.spark.util.Utils + +import org.scalatest.{FunSuite, Matchers} + +class CommandUtilsSuite extends FunSuite with Matchers { + + test("set libraryPath correctly") { + val appId = "12345-worker321-9876" + val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) + val cmd = new Command("mainClass", Seq(), Map(), Seq(), Seq("libraryPathToB"), Seq()) + val builder = CommandUtils.buildProcessBuilder(cmd, 512, sparkHome, t => t) + val libraryPath = Utils.libraryPathEnvName + val env = builder.environment + env.keySet should contain(libraryPath) + assert(env.get(libraryPath).startsWith("libraryPathToB")) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 5e2592e8d2e8d..196217062991e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -19,6 +19,8 @@ package org.apache.spark.deploy.worker import java.io.File +import scala.collection.JavaConversions._ + import org.scalatest.FunSuite import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} @@ -32,6 +34,7 @@ class ExecutorRunnerTest extends FunSuite { Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", new File(sparkHome), new File("ooga"), "blah", new SparkConf, ExecutorState.RUNNING) - assert(er.getCommandSeq.last === appId) + val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) + assert(builder.command().last === appId) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8ea0e7cf40a14..f95d72379171c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} +import org.apache.spark.util.Utils /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. @@ -312,6 +313,10 @@ private[spark] trait ClientBase extends Logging { val javaOpts = ListBuffer[String]() + // Set the environment variable through a command prefix + // to append to the existing value of the variable + var prefixEnv: Option[String] = None + // Add Xmx for AM memory javaOpts += "-Xmx" + args.amMemory + "m" @@ -348,8 +353,11 @@ private[spark] trait ClientBase extends Logging { sparkConf.getOption("spark.driver.extraJavaOptions") .orElse(sys.env.get("SPARK_JAVA_OPTS")) .foreach(opts => javaOpts += opts) - sparkConf.getOption("spark.driver.libraryPath") - .foreach(p => javaOpts += s"-Djava.library.path=$p") + val libraryPaths = Seq(sys.props.get("spark.driver.extraLibraryPath"), + sys.props.get("spark.driver.libraryPath")).flatten + if (libraryPaths.nonEmpty) { + prefixEnv = Some(Utils.libraryPathEnvPrefix(libraryPaths)) + } } // For log4j configuration to reference @@ -384,7 +392,7 @@ private[spark] trait ClientBase extends Logging { "--num-executors ", args.numExecutors.toString) // Command for the ApplicationMaster - val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ javaOpts ++ amArgs ++ Seq( "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 5cb4753de2e84..88dad0febd03f 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils trait ExecutorRunnableUtil extends Logging { @@ -47,6 +48,11 @@ trait ExecutorRunnableUtil extends Logging { localResources: HashMap[String, LocalResource]): List[String] = { // Extra options for the JVM val javaOpts = ListBuffer[String]() + + // Set the environment variable through a command prefix + // to append to the existing value of the variable + var prefixEnv: Option[String] = None + // Set the JVM memory val executorMemoryString = executorMemory + "m" javaOpts += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " @@ -58,6 +64,9 @@ trait ExecutorRunnableUtil extends Logging { sys.env.get("SPARK_JAVA_OPTS").foreach { opts => javaOpts += opts } + sys.props.get("spark.executor.extraLibraryPath").foreach { p => + prefixEnv = Some(Utils.libraryPathEnvPrefix(Seq(p))) + } javaOpts += "-Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) @@ -101,7 +110,7 @@ trait ExecutorRunnableUtil extends Logging { // For log4j configuration to reference javaOpts += ("-Dspark.yarn.app.container.log.dir=" + ApplicationConstants.LOG_DIR_EXPANSION_VAR) - val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", + val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server", // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in From 6db3157464e36f7a572ada5f1e7f88730aa23dbd Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 29 Oct 2014 23:52:46 -0700 Subject: [PATCH 1135/1492] [SPARK-4102] Remove unused ShuffleReader.stop() method. This method is not implemented by the only subclass (HashShuffleReader), nor is it ever called. While the use of Scala's fancy "???" was pretty exciting, the method's existence can only lead to confusion and it therefore should be deleted. mateiz was there a reason for adding this that I'm missing? Author: Kay Ousterhout Closes #2966 from kayousterhout/SPARK-4102 and squashes the following commits: 532c564 [Kay Ousterhout] Added back commented-out method, as per Matei's request 904655e [Kay Ousterhout] [SPARK-4102] Remove unused ShuffleReader.stop() method. --- .../scala/org/apache/spark/shuffle/ShuffleReader.scala | 8 ++++++-- .../org/apache/spark/shuffle/hash/HashShuffleReader.scala | 3 --- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala index b30e366d06006..292e48314ee10 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleReader.scala @@ -24,6 +24,10 @@ private[spark] trait ShuffleReader[K, C] { /** Read the combined key-values for this reduce task */ def read(): Iterator[Product2[K, C]] - /** Close this reader */ - def stop(): Unit + /** + * Close this reader. + * TODO: Add this back when we make the ShuffleReader a developer API that others can implement + * (at which point this will likely be necessary). + */ + // def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 88a5f1e5ddf58..5baf45db45c17 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -66,7 +66,4 @@ private[spark] class HashShuffleReader[K, C]( aggregatedIter } } - - /** Close this reader */ - override def stop(): Unit = ??? } From c7ad0852084dc28f3ebc144adfd4928b23f1c8ea Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Thu, 30 Oct 2014 00:05:57 -0700 Subject: [PATCH 1136/1492] [SPARK-4130][MLlib] Fixing libSVM parser bug with extra whitespace This simple patch filters out extra whitespace entries. Author: Joseph E. Gonzalez Author: Joey Closes #2996 from jegonzal/loadLibSVM and squashes the following commits: e0227ab [Joey] improving readability e028e84 [Joseph E. Gonzalez] fixing whitespace bug in loadLibSVMFile when parsing libSVM files --- mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index dce0adffa6249..b88e08bf148ae 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -76,7 +76,7 @@ object MLUtils { .map { line => val items = line.split(' ') val label = items.head.toDouble - val (indices, values) = items.tail.map { item => + val (indices, values) = items.tail.filter(_.nonEmpty).map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based. val value = indexAndValue(1).toDouble From d9327192eee7f18e92381c59a42b0e1770f1f8f4 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 30 Oct 2014 12:00:56 -0700 Subject: [PATCH 1137/1492] SPARK-4111 [MLlib] add regression metrics Add RegressionMetrics.scala as regression metrics used for evaluation and corresponding test case RegressionMetricsSuite.scala. Author: Yanbo Liang Author: liangyanbo Closes #2978 from yanbohappy/regression_metrics and squashes the following commits: 730d0a9 [Yanbo Liang] more clearly annotation 3d0bec1 [Yanbo Liang] rename and keep code style a8ad3e3 [Yanbo Liang] simplify code for keeping style d454909 [Yanbo Liang] rename parameter and function names, delete unused columns, add reference 2e56282 [liangyanbo] rename r2_score() and remove unused column 43bb12b [liangyanbo] add regression metrics --- .../mllib/evaluation/RegressionMetrics.scala | 89 +++++++++++++++++++ .../evaluation/RegressionMetricsSuite.scala | 52 +++++++++++ 2 files changed, 141 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala new file mode 100644 index 0000000000000..693117d820580 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -0,0 +1,89 @@ +/* + * 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.apache.spark.annotation.Experimental +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, MultivariateOnlineSummarizer} + +/** + * :: Experimental :: + * Evaluator for regression. + * + * @param predictionAndObservations an RDD of (prediction, observation) pairs. + */ +@Experimental +class RegressionMetrics(predictionAndObservations: RDD[(Double, Double)]) extends Logging { + + /** + * Use MultivariateOnlineSummarizer to calculate summary statistics of observations and errors. + */ + private lazy val summary: MultivariateStatisticalSummary = { + val summary: MultivariateStatisticalSummary = predictionAndObservations.map { + case (prediction, observation) => Vectors.dense(observation, observation - prediction) + }.aggregate(new MultivariateOnlineSummarizer())( + (summary, v) => summary.add(v), + (sum1, sum2) => sum1.merge(sum2) + ) + summary + } + + /** + * Returns the explained variance regression score. + * explainedVariance = 1 - variance(y - \hat{y}) / variance(y) + * Reference: [[http://en.wikipedia.org/wiki/Explained_variation]] + */ + def explainedVariance: Double = { + 1 - summary.variance(1) / summary.variance(0) + } + + /** + * Returns the mean absolute error, which is a risk function corresponding to the + * expected value of the absolute error loss or l1-norm loss. + */ + def meanAbsoluteError: Double = { + summary.normL1(1) / summary.count + } + + /** + * Returns the mean squared error, which is a risk function corresponding to the + * expected value of the squared error loss or quadratic loss. + */ + def meanSquaredError: Double = { + val rmse = summary.normL2(1) / math.sqrt(summary.count) + rmse * rmse + } + + /** + * Returns the root mean squared error, which is defined as the square root of + * the mean squared error. + */ + def rootMeanSquaredError: Double = { + summary.normL2(1) / math.sqrt(summary.count) + } + + /** + * Returns R^2^, the coefficient of determination. + * Reference: [[http://en.wikipedia.org/wiki/Coefficient_of_determination]] + */ + def r2: Double = { + 1 - math.pow(summary.normL2(1), 2) / (summary.variance(0) * (summary.count - 1)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala new file mode 100644 index 0000000000000..5396d7b2b74fa --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class RegressionMetricsSuite extends FunSuite with LocalSparkContext { + + test("regression metrics") { + val predictionAndObservations = sc.parallelize( + Seq((2.5,3.0),(0.0,-0.5),(2.0,2.0),(8.0,7.0)), 2) + val metrics = new RegressionMetrics(predictionAndObservations) + assert(metrics.explainedVariance ~== 0.95717 absTol 1E-5, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.5 absTol 1E-5, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.375 absTol 1E-5, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.61237 absTol 1E-5, + "root mean squared error mismatch") + assert(metrics.r2 ~== 0.94861 absTol 1E-5, "r2 score mismatch") + } + + test("regression metrics with complete fitting") { + val predictionAndObservations = sc.parallelize( + Seq((3.0,3.0),(0.0,0.0),(2.0,2.0),(8.0,8.0)), 2) + val metrics = new RegressionMetrics(predictionAndObservations) + assert(metrics.explainedVariance ~== 1.0 absTol 1E-5, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.0 absTol 1E-5, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.0 absTol 1E-5, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.0 absTol 1E-5, + "root mean squared error mismatch") + assert(metrics.r2 ~== 1.0 absTol 1E-5, "r2 score mismatch") + } +} From 234de9232bcfa212317a8073c4a82c3863b36b14 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 30 Oct 2014 14:51:13 -0700 Subject: [PATCH 1138/1492] [SPARK-4028][Streaming] ReceivedBlockHandler interface to abstract the functionality of storage of received data As part of the initiative to prevent data loss on streaming driver failure, this JIRA tracks the subtask of implementing a ReceivedBlockHandler, that abstracts the functionality of storage of received data blocks. The default implementation will maintain the current behavior of storing the data into BlockManager. The optional implementation will store the data to both BlockManager as well as a write ahead log. Author: Tathagata Das Closes #2940 from tdas/driver-ha-rbh and squashes the following commits: 78a4aaa [Tathagata Das] Fixed bug causing test failures. f192f47 [Tathagata Das] Fixed import order. df5f320 [Tathagata Das] Updated code to use ReceivedBlockStoreResult as the return type for handler's storeBlock 33c30c9 [Tathagata Das] Added license, and organized imports. 2f025b3 [Tathagata Das] Updates based on PR comments. 18aec1e [Tathagata Das] Moved ReceivedBlockInfo back into spark.streaming.scheduler package 95a4987 [Tathagata Das] Added ReceivedBlockHandler and its associated tests --- .../dstream/ReceiverInputDStream.scala | 7 +- .../streaming/receiver/ReceivedBlock.scala | 35 +++ .../receiver/ReceivedBlockHandler.scala | 193 +++++++++++++ .../receiver/ReceiverSupervisorImpl.scala | 88 +++--- .../spark/streaming/scheduler/BatchInfo.scala | 2 +- .../spark/streaming/scheduler/JobSet.scala | 3 +- .../scheduler/ReceivedBlockInfo.scala | 28 ++ .../streaming/scheduler/ReceiverTracker.scala | 24 +- .../util/WriteAheadLogRandomReader.scala | 1 - .../streaming/ReceivedBlockHandlerSuite.scala | 258 ++++++++++++++++++ .../streaming/util/WriteAheadLogSuite.scala | 34 ++- 11 files changed, 603 insertions(+), 70 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 391e40924f38a..bb47d373de63d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -23,8 +23,9 @@ import scala.reflect.ClassTag import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.BlockId import org.apache.spark.streaming._ -import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.receiver.{WriteAheadLogBasedStoreResult, BlockManagerBasedStoreResult, Receiver} import org.apache.spark.streaming.scheduler.ReceivedBlockInfo +import org.apache.spark.SparkException /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -65,10 +66,10 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont if (validTime >= graph.startTime) { val blockInfo = ssc.scheduler.receiverTracker.getReceivedBlockInfo(id) receivedBlockInfo(validTime) = blockInfo - val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) + val blockIds = blockInfo.map { _.blockStoreResult.blockId.asInstanceOf[BlockId] } Some(new BlockRDD[T](ssc.sc, blockIds)) } else { - Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) + Some(new BlockRDD[T](ssc.sc, Array.empty)) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala new file mode 100644 index 0000000000000..47968afef2dbf --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.language.existentials + +/** Trait representing a received block */ +private[streaming] sealed trait ReceivedBlock + +/** class representing a block received as an ArrayBuffer */ +private[streaming] case class ArrayBufferBlock(arrayBuffer: ArrayBuffer[_]) extends ReceivedBlock + +/** class representing a block received as an Iterator */ +private[streaming] case class IteratorBlock(iterator: Iterator[_]) extends ReceivedBlock + +/** class representing a block received as an ByteBuffer */ +private[streaming] case class ByteBufferBlock(byteBuffer: ByteBuffer) extends ReceivedBlock diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala new file mode 100644 index 0000000000000..fdf995320beb4 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.receiver + +import scala.concurrent.{Await, ExecutionContext, Future} +import scala.concurrent.duration._ +import scala.language.{existentials, postfixOps} + +import WriteAheadLogBasedBlockHandler._ +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.storage._ +import org.apache.spark.streaming.util.{Clock, SystemClock, WriteAheadLogFileSegment, WriteAheadLogManager} +import org.apache.spark.util.Utils + +/** Trait that represents the metadata related to storage of blocks */ +private[streaming] trait ReceivedBlockStoreResult { + def blockId: StreamBlockId // Any implementation of this trait will store a block id +} + +/** Trait that represents a class that handles the storage of blocks received by receiver */ +private[streaming] trait ReceivedBlockHandler { + + /** Store a received block with the given block id and return related metadata */ + def storeBlock(blockId: StreamBlockId, receivedBlock: ReceivedBlock): ReceivedBlockStoreResult + + /** Cleanup old blocks older than the given threshold time */ + def cleanupOldBlock(threshTime: Long) +} + + +/** + * Implementation of [[org.apache.spark.streaming.receiver.ReceivedBlockStoreResult]] + * that stores the metadata related to storage of blocks using + * [[org.apache.spark.streaming.receiver.BlockManagerBasedBlockHandler]] + */ +private[streaming] case class BlockManagerBasedStoreResult(blockId: StreamBlockId) + extends ReceivedBlockStoreResult + + +/** + * Implementation of a [[org.apache.spark.streaming.receiver.ReceivedBlockHandler]] which + * stores the received blocks into a block manager with the specified storage level. + */ +private[streaming] class BlockManagerBasedBlockHandler( + blockManager: BlockManager, storageLevel: StorageLevel) + extends ReceivedBlockHandler with Logging { + + def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { + val putResult: Seq[(BlockId, BlockStatus)] = block match { + case ArrayBufferBlock(arrayBuffer) => + blockManager.putIterator(blockId, arrayBuffer.iterator, storageLevel, tellMaster = true) + case IteratorBlock(iterator) => + blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) + case ByteBufferBlock(byteBuffer) => + blockManager.putBytes(blockId, byteBuffer, storageLevel, tellMaster = true) + case o => + throw new SparkException( + s"Could not store $blockId to block manager, unexpected block type ${o.getClass.getName}") + } + if (!putResult.map { _._1 }.contains(blockId)) { + throw new SparkException( + s"Could not store $blockId to block manager with storage level $storageLevel") + } + BlockManagerBasedStoreResult(blockId) + } + + def cleanupOldBlock(threshTime: Long) { + // this is not used as blocks inserted into the BlockManager are cleared by DStream's clearing + // of BlockRDDs. + } +} + + +/** + * Implementation of [[org.apache.spark.streaming.receiver.ReceivedBlockStoreResult]] + * that stores the metadata related to storage of blocks using + * [[org.apache.spark.streaming.receiver.WriteAheadLogBasedBlockHandler]] + */ +private[streaming] case class WriteAheadLogBasedStoreResult( + blockId: StreamBlockId, + segment: WriteAheadLogFileSegment + ) extends ReceivedBlockStoreResult + + +/** + * Implementation of a [[org.apache.spark.streaming.receiver.ReceivedBlockHandler]] which + * stores the received blocks in both, a write ahead log and a block manager. + */ +private[streaming] class WriteAheadLogBasedBlockHandler( + blockManager: BlockManager, + streamId: Int, + storageLevel: StorageLevel, + conf: SparkConf, + hadoopConf: Configuration, + checkpointDir: String, + clock: Clock = new SystemClock + ) extends ReceivedBlockHandler with Logging { + + private val blockStoreTimeout = conf.getInt( + "spark.streaming.receiver.blockStoreTimeout", 30).seconds + private val rollingInterval = conf.getInt( + "spark.streaming.receiver.writeAheadLog.rollingInterval", 60) + private val maxFailures = conf.getInt( + "spark.streaming.receiver.writeAheadLog.maxFailures", 3) + + // Manages rolling log files + private val logManager = new WriteAheadLogManager( + checkpointDirToLogDir(checkpointDir, streamId), + hadoopConf, rollingInterval, maxFailures, + callerName = this.getClass.getSimpleName, + clock = clock + ) + + // For processing futures used in parallel block storing into block manager and write ahead log + // # threads = 2, so that both writing to BM and WAL can proceed in parallel + implicit private val executionContext = ExecutionContext.fromExecutorService( + Utils.newDaemonFixedThreadPool(2, this.getClass.getSimpleName)) + + /** + * This implementation stores the block into the block manager as well as a write ahead log. + * It does this in parallel, using Scala Futures, and returns only after the block has + * been stored in both places. + */ + def storeBlock(blockId: StreamBlockId, block: ReceivedBlock): ReceivedBlockStoreResult = { + + // Serialize the block so that it can be inserted into both + val serializedBlock = block match { + case ArrayBufferBlock(arrayBuffer) => + blockManager.dataSerialize(blockId, arrayBuffer.iterator) + case IteratorBlock(iterator) => + blockManager.dataSerialize(blockId, iterator) + case ByteBufferBlock(byteBuffer) => + byteBuffer + case _ => + throw new Exception(s"Could not push $blockId to block manager, unexpected block type") + } + + // Store the block in block manager + val storeInBlockManagerFuture = Future { + val putResult = + blockManager.putBytes(blockId, serializedBlock, storageLevel, tellMaster = true) + if (!putResult.map { _._1 }.contains(blockId)) { + throw new SparkException( + s"Could not store $blockId to block manager with storage level $storageLevel") + } + } + + // Store the block in write ahead log + val storeInWriteAheadLogFuture = Future { + logManager.writeToLog(serializedBlock) + } + + // Combine the futures, wait for both to complete, and return the write ahead log segment + val combinedFuture = for { + _ <- storeInBlockManagerFuture + fileSegment <- storeInWriteAheadLogFuture + } yield fileSegment + val segment = Await.result(combinedFuture, blockStoreTimeout) + WriteAheadLogBasedStoreResult(blockId, segment) + } + + def cleanupOldBlock(threshTime: Long) { + logManager.cleanupOldLogs(threshTime) + } + + def stop() { + logManager.stop() + } +} + +private[streaming] object WriteAheadLogBasedBlockHandler { + def checkpointDirToLogDir(checkpointDir: String, streamId: Int): String = { + new Path(checkpointDir, new Path("receivedData", streamId.toString)).toString + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 53a3e6200e340..5360412330d37 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -25,16 +25,13 @@ import scala.concurrent.Await import akka.actor.{Actor, Props} import akka.pattern.ask - import com.google.common.base.Throwables - -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.streaming.scheduler._ -import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.hadoop.conf.Configuration +import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.storage.StreamBlockId -import org.apache.spark.streaming.scheduler.DeregisterReceiver -import org.apache.spark.streaming.scheduler.AddBlock -import org.apache.spark.streaming.scheduler.RegisterReceiver +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.util.WriteAheadLogFileSegment +import org.apache.spark.util.{AkkaUtils, Utils} /** * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] @@ -44,12 +41,26 @@ import org.apache.spark.streaming.scheduler.RegisterReceiver */ private[streaming] class ReceiverSupervisorImpl( receiver: Receiver[_], - env: SparkEnv + env: SparkEnv, + hadoopConf: Configuration, + checkpointDirOption: Option[String] ) extends ReceiverSupervisor(receiver, env.conf) with Logging { - private val blockManager = env.blockManager + private val receivedBlockHandler: ReceivedBlockHandler = { + if (env.conf.getBoolean("spark.streaming.receiver.writeAheadLog.enable", false)) { + if (checkpointDirOption.isEmpty) { + throw new SparkException( + "Cannot enable receiver write-ahead log without checkpoint directory set. " + + "Please use streamingContext.checkpoint() to set the checkpoint directory. " + + "See documentation for more details.") + } + new WriteAheadLogBasedBlockHandler(env.blockManager, receiver.streamId, + receiver.storageLevel, env.conf, hadoopConf, checkpointDirOption.get) + } else { + new BlockManagerBasedBlockHandler(env.blockManager, receiver.storageLevel) + } + } - private val storageLevel = receiver.storageLevel /** Remote Akka actor for the ReceiverTracker */ private val trackerActor = { @@ -105,47 +116,50 @@ private[streaming] class ReceiverSupervisorImpl( /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ def pushArrayBuffer( arrayBuffer: ArrayBuffer[_], - optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] ) { - val blockId = optionalBlockId.getOrElse(nextBlockId) - val time = System.currentTimeMillis - blockManager.putArray(blockId, arrayBuffer.toArray[Any], storageLevel, tellMaster = true) - logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") - reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) + pushAndReportBlock(ArrayBufferBlock(arrayBuffer), metadataOption, blockIdOption) } /** Store a iterator of received data as a data block into Spark's memory. */ def pushIterator( iterator: Iterator[_], - optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] ) { - val blockId = optionalBlockId.getOrElse(nextBlockId) - val time = System.currentTimeMillis - blockManager.putIterator(blockId, iterator, storageLevel, tellMaster = true) - logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") - reportPushedBlock(blockId, -1, optionalMetadata) + pushAndReportBlock(IteratorBlock(iterator), metadataOption, blockIdOption) } /** Store the bytes of received data as a data block into Spark's memory. */ def pushBytes( bytes: ByteBuffer, - optionalMetadata: Option[Any], - optionalBlockId: Option[StreamBlockId] + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] ) { - val blockId = optionalBlockId.getOrElse(nextBlockId) - val time = System.currentTimeMillis - blockManager.putBytes(blockId, bytes, storageLevel, tellMaster = true) - logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") - reportPushedBlock(blockId, -1, optionalMetadata) + pushAndReportBlock(ByteBufferBlock(bytes), metadataOption, blockIdOption) } - /** Report pushed block */ - def reportPushedBlock(blockId: StreamBlockId, numRecords: Long, optionalMetadata: Option[Any]) { - val blockInfo = ReceivedBlockInfo(streamId, blockId, numRecords, optionalMetadata.orNull) - trackerActor ! AddBlock(blockInfo) - logDebug("Reported block " + blockId) + /** Store block and report it to driver */ + def pushAndReportBlock( + receivedBlock: ReceivedBlock, + metadataOption: Option[Any], + blockIdOption: Option[StreamBlockId] + ) { + val blockId = blockIdOption.getOrElse(nextBlockId) + val numRecords = receivedBlock match { + case ArrayBufferBlock(arrayBuffer) => arrayBuffer.size + case _ => -1 + } + + val time = System.currentTimeMillis + val blockStoreResult = receivedBlockHandler.storeBlock(blockId, receivedBlock) + logDebug(s"Pushed block $blockId in ${(System.currentTimeMillis - time)} ms") + + val blockInfo = ReceivedBlockInfo(streamId, numRecords, blockStoreResult) + val future = trackerActor.ask(AddBlock(blockInfo))(askTimeout) + Await.result(future, askTimeout) + logDebug(s"Reported block $blockId") } /** Report error to the receiver tracker */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index a68aecb881117..92dc113f397ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -17,8 +17,8 @@ package org.apache.spark.streaming.scheduler -import org.apache.spark.streaming.Time import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.streaming.Time /** * :: DeveloperApi :: diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index a69d74362173e..8c15a75b1b0e0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -17,7 +17,8 @@ package org.apache.spark.streaming.scheduler -import scala.collection.mutable.{ArrayBuffer, HashSet} +import scala.collection.mutable.HashSet + import org.apache.spark.streaming.Time /** Class representing a set of Jobs diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala new file mode 100644 index 0000000000000..94beb590f52d6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.scheduler + +import org.apache.spark.streaming.receiver.ReceivedBlockStoreResult + +/** Information about blocks received by the receiver */ +private[streaming] case class ReceivedBlockInfo( + streamId: Int, + numRecords: Long, + blockStoreResult: ReceivedBlockStoreResult + ) + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 7149dbc12a365..d696563bcee83 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -21,21 +21,12 @@ import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import scala.language.existentials import akka.actor._ -import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.{SerializableWritable, Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ -import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} import org.apache.spark.util.AkkaUtils -/** Information about blocks received by the receiver */ -private[streaming] case class ReceivedBlockInfo( - streamId: Int, - blockId: StreamBlockId, - numRecords: Long, - metadata: Any - ) - /** * Messages used by the NetworkReceiver and the ReceiverTracker to communicate * with each other. @@ -153,7 +144,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { def addBlocks(receivedBlockInfo: ReceivedBlockInfo) { getReceivedBlockInfoQueue(receivedBlockInfo.streamId) += receivedBlockInfo logDebug("Stream " + receivedBlockInfo.streamId + " received new blocks: " + - receivedBlockInfo.blockId) + receivedBlockInfo.blockStoreResult.blockId) } /** Report error sent by a receiver */ @@ -188,6 +179,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { sender ! true case AddBlock(receivedBlockInfo) => addBlocks(receivedBlockInfo) + sender ! true case ReportError(streamId, message, error) => reportError(streamId, message, error) case DeregisterReceiver(streamId, message, error) => @@ -252,6 +244,9 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { ssc.sc.makeRDD(receivers, receivers.size) } + val checkpointDirOption = Option(ssc.checkpointDir) + val serializableHadoopConf = new SerializableWritable(ssc.sparkContext.hadoopConfiguration) + // Function to start the receiver on the worker node val startReceiver = (iterator: Iterator[Receiver[_]]) => { if (!iterator.hasNext) { @@ -259,9 +254,10 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { "Could not start receiver as object not found.") } val receiver = iterator.next() - val executor = new ReceiverSupervisorImpl(receiver, SparkEnv.get) - executor.start() - executor.awaitTermination() + val supervisor = new ReceiverSupervisorImpl( + receiver, SparkEnv.get, serializableHadoopConf.value, checkpointDirOption) + supervisor.start() + supervisor.awaitTermination() } // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala index 92bad7a882a65..003989092a42a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogRandomReader.scala @@ -52,4 +52,3 @@ private[streaming] class WriteAheadLogRandomReader(path: String, conf: Configura HdfsUtils.checkState(!closed, "Stream is closed. Create a new Reader to read from the file.") } } - diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala new file mode 100644 index 0000000000000..ad1a6f01b3a57 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -0,0 +1,258 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming + +import java.io.File +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.postfixOps + +import akka.actor.{ActorSystem, Props} +import com.google.common.io.Files +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} +import org.scalatest.concurrent.Eventually._ + +import org.apache.spark._ +import org.apache.spark.network.nio.NioBlockTransferService +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.storage._ +import org.apache.spark.streaming.receiver._ +import org.apache.spark.streaming.util._ +import org.apache.spark.util.AkkaUtils +import WriteAheadLogBasedBlockHandler._ +import WriteAheadLogSuite._ + +class ReceivedBlockHandlerSuite extends FunSuite with BeforeAndAfter with Matchers with Logging { + + val conf = new SparkConf().set("spark.streaming.receiver.writeAheadLog.rollingInterval", "1") + val hadoopConf = new Configuration() + val storageLevel = StorageLevel.MEMORY_ONLY_SER + val streamId = 1 + val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) + val shuffleManager = new HashShuffleManager(conf) + val serializer = new KryoSerializer(conf) + val manualClock = new ManualClock + val blockManagerSize = 10000000 + + var actorSystem: ActorSystem = null + var blockManagerMaster: BlockManagerMaster = null + var blockManager: BlockManager = null + var tempDirectory: File = null + + before { + val (actorSystem, boundPort) = AkkaUtils.createActorSystem( + "test", "localhost", 0, conf = conf, securityManager = securityMgr) + this.actorSystem = actorSystem + conf.set("spark.driver.port", boundPort.toString) + + blockManagerMaster = new BlockManagerMaster( + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf, true) + + blockManager = new BlockManager("bm", actorSystem, blockManagerMaster, serializer, + blockManagerSize, conf, mapOutputTracker, shuffleManager, + new NioBlockTransferService(conf, securityMgr)) + + tempDirectory = Files.createTempDir() + manualClock.setTime(0) + } + + after { + if (blockManager != null) { + blockManager.stop() + blockManager = null + } + if (blockManagerMaster != null) { + blockManagerMaster.stop() + blockManagerMaster = null + } + actorSystem.shutdown() + actorSystem.awaitTermination() + actorSystem = null + + if (tempDirectory != null && tempDirectory.exists()) { + FileUtils.deleteDirectory(tempDirectory) + tempDirectory = null + } + } + + test("BlockManagerBasedBlockHandler - store blocks") { + withBlockManagerBasedBlockHandler { handler => + testBlockStoring(handler) { case (data, blockIds, storeResults) => + // Verify the data in block manager is correct + val storedData = blockIds.flatMap { blockId => + blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + }.toList + storedData shouldEqual data + + // Verify that the store results are instances of BlockManagerBasedStoreResult + assert( + storeResults.forall { _.isInstanceOf[BlockManagerBasedStoreResult] }, + "Unexpected store result type" + ) + } + } + } + + test("BlockManagerBasedBlockHandler - handle errors in storing block") { + withBlockManagerBasedBlockHandler { handler => + testErrorHandling(handler) + } + } + + test("WriteAheadLogBasedBlockHandler - store blocks") { + withWriteAheadLogBasedBlockHandler { handler => + testBlockStoring(handler) { case (data, blockIds, storeResults) => + // Verify the data in block manager is correct + val storedData = blockIds.flatMap { blockId => + blockManager.getLocal(blockId).map { _.data.map {_.toString}.toList }.getOrElse(List.empty) + }.toList + storedData shouldEqual data + + // Verify that the store results are instances of WriteAheadLogBasedStoreResult + assert( + storeResults.forall { _.isInstanceOf[WriteAheadLogBasedStoreResult] }, + "Unexpected store result type" + ) + // Verify the data in write ahead log files is correct + val fileSegments = storeResults.map { _.asInstanceOf[WriteAheadLogBasedStoreResult].segment} + val loggedData = fileSegments.flatMap { segment => + val reader = new WriteAheadLogRandomReader(segment.path, hadoopConf) + val bytes = reader.read(segment) + reader.close() + blockManager.dataDeserialize(generateBlockId(), bytes).toList + } + loggedData shouldEqual data + } + } + } + + test("WriteAheadLogBasedBlockHandler - handle errors in storing block") { + withWriteAheadLogBasedBlockHandler { handler => + testErrorHandling(handler) + } + } + + test("WriteAheadLogBasedBlockHandler - cleanup old blocks") { + withWriteAheadLogBasedBlockHandler { handler => + val blocks = Seq.tabulate(10) { i => IteratorBlock(Iterator(1 to i)) } + storeBlocks(handler, blocks) + + val preCleanupLogFiles = getWriteAheadLogFiles() + preCleanupLogFiles.size should be > 1 + + // this depends on the number of blocks inserted using generateAndStoreData() + manualClock.currentTime() shouldEqual 5000L + + val cleanupThreshTime = 3000L + handler.cleanupOldBlock(cleanupThreshTime) + eventually(timeout(10000 millis), interval(10 millis)) { + getWriteAheadLogFiles().size should be < preCleanupLogFiles.size + } + } + } + + /** + * Test storing of data using different forms of ReceivedBlocks and verify that they succeeded + * using the given verification function + */ + private def testBlockStoring(receivedBlockHandler: ReceivedBlockHandler) + (verifyFunc: (Seq[String], Seq[StreamBlockId], Seq[ReceivedBlockStoreResult]) => Unit) { + val data = Seq.tabulate(100) { _.toString } + + def storeAndVerify(blocks: Seq[ReceivedBlock]) { + blocks should not be empty + val (blockIds, storeResults) = storeBlocks(receivedBlockHandler, blocks) + withClue(s"Testing with ${blocks.head.getClass.getSimpleName}s:") { + // Verify returns store results have correct block ids + (storeResults.map { _.blockId }) shouldEqual blockIds + + // Call handler-specific verification function + verifyFunc(data, blockIds, storeResults) + } + } + + def dataToByteBuffer(b: Seq[String]) = blockManager.dataSerialize(generateBlockId, b.iterator) + + val blocks = data.grouped(10).toSeq + + storeAndVerify(blocks.map { b => IteratorBlock(b.toIterator) }) + storeAndVerify(blocks.map { b => ArrayBufferBlock(new ArrayBuffer ++= b) }) + storeAndVerify(blocks.map { b => ByteBufferBlock(dataToByteBuffer(b)) }) + } + + /** Test error handling when blocks that cannot be stored */ + private def testErrorHandling(receivedBlockHandler: ReceivedBlockHandler) { + // Handle error in iterator (e.g. divide-by-zero error) + intercept[Exception] { + val iterator = (10 to (-10, -1)).toIterator.map { _ / 0 } + receivedBlockHandler.storeBlock(StreamBlockId(1, 1), IteratorBlock(iterator)) + } + + // Handler error in block manager storing (e.g. too big block) + intercept[SparkException] { + val byteBuffer = ByteBuffer.wrap(new Array[Byte](blockManagerSize + 1)) + receivedBlockHandler.storeBlock(StreamBlockId(1, 1), ByteBufferBlock(byteBuffer)) + } + } + + /** Instantiate a BlockManagerBasedBlockHandler and run a code with it */ + private def withBlockManagerBasedBlockHandler(body: BlockManagerBasedBlockHandler => Unit) { + body(new BlockManagerBasedBlockHandler(blockManager, storageLevel)) + } + + /** Instantiate a WriteAheadLogBasedBlockHandler and run a code with it */ + private def withWriteAheadLogBasedBlockHandler(body: WriteAheadLogBasedBlockHandler => Unit) { + val receivedBlockHandler = new WriteAheadLogBasedBlockHandler(blockManager, 1, + storageLevel, conf, hadoopConf, tempDirectory.toString, manualClock) + try { + body(receivedBlockHandler) + } finally { + receivedBlockHandler.stop() + } + } + + /** Store blocks using a handler */ + private def storeBlocks( + receivedBlockHandler: ReceivedBlockHandler, + blocks: Seq[ReceivedBlock] + ): (Seq[StreamBlockId], Seq[ReceivedBlockStoreResult]) = { + val blockIds = Seq.fill(blocks.size)(generateBlockId()) + val storeResults = blocks.zip(blockIds).map { + case (block, id) => + manualClock.addToTime(500) // log rolling interval set to 1000 ms through SparkConf + logDebug("Inserting block " + id) + receivedBlockHandler.storeBlock(id, block) + }.toList + logDebug("Done inserting") + (blockIds, storeResults) + } + + private def getWriteAheadLogFiles(): Seq[String] = { + getLogFilesInDirectory(checkpointDirToLogDir(tempDirectory.toString, streamId)) + } + + private def generateBlockId(): StreamBlockId = StreamBlockId(streamId, scala.util.Random.nextLong) +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 5eba93c208c50..1956a4f1db90a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -58,7 +58,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { test("WriteAheadLogWriter - writing data") { val dataToWrite = generateRandomData() val segments = writeDataUsingWriter(testFile, dataToWrite) - val writtenData = readDataManually(testFile, segments) + val writtenData = readDataManually(segments) assert(writtenData === dataToWrite) } @@ -67,7 +67,7 @@ class WriteAheadLogSuite extends FunSuite with BeforeAndAfter { val writer = new WriteAheadLogWriter(testFile, hadoopConf) dataToWrite.foreach { data => val segment = writer.write(stringToByteBuffer(data)) - val dataRead = readDataManually(testFile, Seq(segment)).head + val dataRead = readDataManually(Seq(segment)).head assert(data === dataRead) } writer.close() @@ -281,14 +281,20 @@ object WriteAheadLogSuite { } /** Read data from a segments of a log file directly and return the list of byte buffers.*/ - def readDataManually(file: String, segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { - val reader = HdfsUtils.getInputStream(file, hadoopConf) - segments.map { x => - reader.seek(x.offset) - val data = new Array[Byte](x.length) - reader.readInt() - reader.readFully(data) - Utils.deserialize[String](data) + def readDataManually(segments: Seq[WriteAheadLogFileSegment]): Seq[String] = { + segments.map { segment => + val reader = HdfsUtils.getInputStream(segment.path, hadoopConf) + try { + reader.seek(segment.offset) + val bytes = new Array[Byte](segment.length) + reader.readInt() + reader.readFully(bytes) + val data = Utils.deserialize[String](bytes) + reader.close() + data + } finally { + reader.close() + } } } @@ -335,9 +341,11 @@ object WriteAheadLogSuite { val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) if (fileSystem.exists(logDirectoryPath) && fileSystem.getFileStatus(logDirectoryPath).isDir) { - fileSystem.listStatus(logDirectoryPath).map { - _.getPath.toString.stripPrefix("file:") - }.sorted + fileSystem.listStatus(logDirectoryPath).map { _.getPath() }.sortBy { + _.getName().split("-")(1).toLong + }.map { + _.toString.stripPrefix("file:") + } } else { Seq.empty } From fb1fbca204250840ffdbc0fcbf80b8dfeebf9edb Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 30 Oct 2014 15:17:02 -0700 Subject: [PATCH 1139/1492] [SPARK-4027][Streaming] WriteAheadLogBackedBlockRDD to read received either from BlockManager or WAL in HDFS As part of the initiative of preventing data loss on streaming driver failure, this sub-task implements a BlockRDD that is backed by HDFS. This BlockRDD can either read data from the Spark's BlockManager, or read the data from file-segments in write ahead log in HDFS. Most of this code has been written by @harishreedharan Author: Tathagata Das Author: Hari Shreedharan Closes #2931 from tdas/driver-ha-rdd and squashes the following commits: 209e49c [Tathagata Das] Better fix to style issue. 4a5866f [Tathagata Das] Addressed one more comment. ed5fbf0 [Tathagata Das] Minor updates. b0a18b1 [Tathagata Das] Fixed import order. 20aa7c6 [Tathagata Das] Fixed more line length issues. 29aa099 [Tathagata Das] Fixed line length issues. 9e47b5b [Tathagata Das] Renamed class, simplified+added unit tests. 6e1bfb8 [Tathagata Das] Tweaks testuite to create spark contxt lazily to prevent contxt leaks. 9c86a61 [Tathagata Das] Merge pull request #22 from harishreedharan/driver-ha-rdd 2878c38 [Hari Shreedharan] Shutdown spark context after tests. Formatting/minor fixes c709f2f [Tathagata Das] Merge pull request #21 from harishreedharan/driver-ha-rdd 5cce16f [Hari Shreedharan] Make sure getBlockLocations uses offset and length to find the blocks on HDFS eadde56 [Tathagata Das] Transferred HDFSBackedBlockRDD for the driver-ha-working branch --- .../scala/org/apache/spark/rdd/BlockRDD.scala | 4 + .../rdd/WriteAheadLogBackedBlockRDD.scala | 125 +++++++++++++++ .../spark/streaming/util/HdfsUtils.scala | 8 +- .../WriteAheadLogBackedBlockRDDSuite.scala | 151 ++++++++++++++++++ 4 files changed, 285 insertions(+), 3 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index 2673ec22509e9..fffa1911f5bc2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -84,5 +84,9 @@ class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds "Attempted to use %s after its blocks have been removed!".format(toString)) } } + + protected def getBlockIdLocations(): Map[BlockId, Seq[String]] = { + locations_ + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala new file mode 100644 index 0000000000000..23295bf658712 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.rdd + +import scala.reflect.ClassTag + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark._ +import org.apache.spark.rdd.BlockRDD +import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.streaming.util.{HdfsUtils, WriteAheadLogFileSegment, WriteAheadLogRandomReader} + +/** + * Partition class for [[org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD]]. + * It contains information about the id of the blocks having this partition's data and + * the segment of the write ahead log that backs the partition. + * @param index index of the partition + * @param blockId id of the block having the partition data + * @param segment segment of the write ahead log having the partition data + */ +private[streaming] +class WriteAheadLogBackedBlockRDDPartition( + val index: Int, + val blockId: BlockId, + val segment: WriteAheadLogFileSegment) + extends Partition + + +/** + * This class represents a special case of the BlockRDD where the data blocks in + * the block manager are also backed by segments in write ahead logs. For reading + * the data, this RDD first looks up the blocks by their ids in the block manager. + * If it does not find them, it looks up the corresponding file segment. + * + * @param sc SparkContext + * @param hadoopConfig Hadoop configuration + * @param blockIds Ids of the blocks that contains this RDD's data + * @param segments Segments in write ahead logs that contain this RDD's data + * @param storeInBlockManager Whether to store in the block manager after reading from the segment + * @param storageLevel storage level to store when storing in block manager + * (applicable when storeInBlockManager = true) + */ +private[streaming] +class WriteAheadLogBackedBlockRDD[T: ClassTag]( + @transient sc: SparkContext, + @transient hadoopConfig: Configuration, + @transient blockIds: Array[BlockId], + @transient segments: Array[WriteAheadLogFileSegment], + storeInBlockManager: Boolean, + storageLevel: StorageLevel) + extends BlockRDD[T](sc, blockIds) { + + require( + blockIds.length == segments.length, + s"Number of block ids (${blockIds.length}) must be " + + s"the same as number of segments (${segments.length}})!") + + // Hadoop configuration is not serializable, so broadcast it as a serializable. + private val broadcastedHadoopConf = new SerializableWritable(hadoopConfig) + + override def getPartitions: Array[Partition] = { + assertValid() + Array.tabulate(blockIds.size) { i => + new WriteAheadLogBackedBlockRDDPartition(i, blockIds(i), segments(i)) + } + } + + /** + * Gets the partition data by getting the corresponding block from the block manager. + * If the block does not exist, then the data is read from the corresponding segment + * in write ahead log files. + */ + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + assertValid() + val hadoopConf = broadcastedHadoopConf.value + val blockManager = SparkEnv.get.blockManager + val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] + val blockId = partition.blockId + blockManager.get(blockId) match { + case Some(block) => // Data is in Block Manager + val iterator = block.data.asInstanceOf[Iterator[T]] + logDebug(s"Read partition data of $this from block manager, block $blockId") + iterator + case None => // Data not found in Block Manager, grab it from write ahead log file + val reader = new WriteAheadLogRandomReader(partition.segment.path, hadoopConf) + val dataRead = reader.read(partition.segment) + reader.close() + logInfo(s"Read partition data of $this from write ahead log, segment ${partition.segment}") + if (storeInBlockManager) { + blockManager.putBytes(blockId, dataRead, storageLevel) + logDebug(s"Stored partition data of $this into block manager with level $storageLevel") + dataRead.rewind() + } + blockManager.dataDeserialize(blockId, dataRead).asInstanceOf[Iterator[T]] + } + } + + /** + * Get the preferred location of the partition. This returns the locations of the block + * if it is present in the block manager, else it returns the location of the + * corresponding segment in HDFS. + */ + override def getPreferredLocations(split: Partition): Seq[String] = { + val partition = split.asInstanceOf[WriteAheadLogBackedBlockRDDPartition] + val blockLocations = getBlockIdLocations().get(partition.blockId) + def segmentLocations = HdfsUtils.getFileSegmentLocations( + partition.segment.path, partition.segment.offset, partition.segment.length, hadoopConfig) + blockLocations.getOrElse(segmentLocations) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index 491f1175576e6..27a28bab83ed5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -52,12 +52,14 @@ private[streaming] object HdfsUtils { } } - def getBlockLocations(path: String, conf: Configuration): Option[Array[String]] = { + /** Get the locations of the HDFS blocks containing the given file segment. */ + def getFileSegmentLocations( + path: String, offset: Long, length: Long, conf: Configuration): Array[String] = { val dfsPath = new Path(path) val dfs = getFileSystemForPath(dfsPath, conf) val fileStatus = dfs.getFileStatus(dfsPath) - val blockLocs = Option(dfs.getFileBlockLocations(fileStatus, 0, fileStatus.getLen)) - blockLocs.map(_.flatMap(_.getHosts)) + val blockLocs = Option(dfs.getFileBlockLocations(fileStatus, offset, length)) + blockLocs.map(_.flatMap(_.getHosts)).getOrElse(Array.empty) } def getFileSystemForPath(path: Path, conf: Configuration): FileSystem = { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala new file mode 100644 index 0000000000000..10160244bcc91 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDDSuite.scala @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.streaming.rdd + +import java.io.File + +import scala.util.Random + +import com.google.common.io.Files +import org.apache.hadoop.conf.Configuration +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.storage.{BlockId, BlockManager, StorageLevel, StreamBlockId} +import org.apache.spark.streaming.util.{WriteAheadLogFileSegment, WriteAheadLogWriter} + +class WriteAheadLogBackedBlockRDDSuite extends FunSuite with BeforeAndAfterAll { + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName(this.getClass.getSimpleName) + val hadoopConf = new Configuration() + + var sparkContext: SparkContext = null + var blockManager: BlockManager = null + var dir: File = null + + override def beforeAll(): Unit = { + sparkContext = new SparkContext(conf) + blockManager = sparkContext.env.blockManager + dir = Files.createTempDir() + } + + override def afterAll(): Unit = { + // Copied from LocalSparkContext, simpler than to introduced test dependencies to core tests. + sparkContext.stop() + dir.delete() + System.clearProperty("spark.driver.port") + } + + test("Read data available in block manager and write ahead log") { + testRDD(5, 5) + } + + test("Read data available only in block manager, not in write ahead log") { + testRDD(5, 0) + } + + test("Read data available only in write ahead log, not in block manager") { + testRDD(0, 5) + } + + test("Read data available only in write ahead log, and test storing in block manager") { + testRDD(0, 5, testStoreInBM = true) + } + + test("Read data with partially available in block manager, and rest in write ahead log") { + testRDD(3, 2) + } + + /** + * Test the WriteAheadLogBackedRDD, by writing some partitions of the data to block manager + * and the rest to a write ahead log, and then reading reading it all back using the RDD. + * It can also test if the partitions that were read from the log were again stored in + * block manager. + * @param numPartitionsInBM Number of partitions to write to the Block Manager + * @param numPartitionsInWAL Number of partitions to write to the Write Ahead Log + * @param testStoreInBM Test whether blocks read from log are stored back into block manager + */ + private def testRDD(numPartitionsInBM: Int, numPartitionsInWAL: Int, testStoreInBM: Boolean = false) { + val numBlocks = numPartitionsInBM + numPartitionsInWAL + val data = Seq.fill(numBlocks, 10)(scala.util.Random.nextString(50)) + + // Put the necessary blocks in the block manager + val blockIds = Array.fill(numBlocks)(StreamBlockId(Random.nextInt(), Random.nextInt())) + data.zip(blockIds).take(numPartitionsInBM).foreach { case(block, blockId) => + blockManager.putIterator(blockId, block.iterator, StorageLevel.MEMORY_ONLY_SER) + } + + // Generate write ahead log segments + val segments = generateFakeSegments(numPartitionsInBM) ++ + writeLogSegments(data.takeRight(numPartitionsInWAL), blockIds.takeRight(numPartitionsInWAL)) + + // Make sure that the left `numPartitionsInBM` blocks are in block manager, and others are not + require( + blockIds.take(numPartitionsInBM).forall(blockManager.get(_).nonEmpty), + "Expected blocks not in BlockManager" + ) + require( + blockIds.takeRight(numPartitionsInWAL).forall(blockManager.get(_).isEmpty), + "Unexpected blocks in BlockManager" + ) + + // Make sure that the right `numPartitionsInWAL` blocks are in write ahead logs, and other are not + require( + segments.takeRight(numPartitionsInWAL).forall(s => + new File(s.path.stripPrefix("file://")).exists()), + "Expected blocks not in write ahead log" + ) + require( + segments.take(numPartitionsInBM).forall(s => + !new File(s.path.stripPrefix("file://")).exists()), + "Unexpected blocks in write ahead log" + ) + + // Create the RDD and verify whether the returned data is correct + val rdd = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, + segments.toArray, storeInBlockManager = false, StorageLevel.MEMORY_ONLY) + assert(rdd.collect() === data.flatten) + + if (testStoreInBM) { + val rdd2 = new WriteAheadLogBackedBlockRDD[String](sparkContext, hadoopConf, blockIds.toArray, + segments.toArray, storeInBlockManager = true, StorageLevel.MEMORY_ONLY) + assert(rdd2.collect() === data.flatten) + assert( + blockIds.forall(blockManager.get(_).nonEmpty), + "All blocks not found in block manager" + ) + } + } + + private def writeLogSegments( + blockData: Seq[Seq[String]], + blockIds: Seq[BlockId] + ): Seq[WriteAheadLogFileSegment] = { + require(blockData.size === blockIds.size) + val writer = new WriteAheadLogWriter(new File(dir, Random.nextString(10)).toString, hadoopConf) + val segments = blockData.zip(blockIds).map { case (data, id) => + writer.write(blockManager.dataSerialize(id, data.iterator)) + } + writer.close() + segments + } + + private def generateFakeSegments(count: Int): Seq[WriteAheadLogFileSegment] = { + Array.fill(count)(new WriteAheadLogFileSegment("random", 0l, 0)) + } +} From 9142c9b80bfe12e0be8a2b795bf52e403b2c5f30 Mon Sep 17 00:00:00 2001 From: Grace Date: Thu, 30 Oct 2014 15:27:32 -0700 Subject: [PATCH 1140/1492] [SPARK-4078] New FsPermission instance w/o FsPermission.createImmutable in eventlog By default, Spark builds its package against Hadoop 1.0.4 version. In that version, it has some FsPermission bug (see [HADOOP-7629] (https://issues.apache.org/jira/browse/HADOOP-7629) by Todd Lipcon). This bug got fixed since 1.1 version. By using that FsPermission.createImmutable() API, end-user may see some RPC exception like below (if turn on eventlog over HDFS). Here proposes a quick fix to avoid certain exception for all hadoop versions. ``` Exception in thread "main" java.io.IOException: Call to sr484/10.1.2.84:54310 failed on local exception: java.io.EOFException at org.apache.hadoop.ipc.Client.wrapException(Client.java:1150) at org.apache.hadoop.ipc.Client.call(Client.java:1118) at org.apache.hadoop.ipc.RPC$Invoker.invoke(RPC.java:229) at $Proxy6.setPermission(Unknown Source) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:85) at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:62) at $Proxy6.setPermission(Unknown Source) at org.apache.hadoop.hdfs.DFSClient.setPermission(DFSClient.java:1285) at org.apache.hadoop.hdfs.DistributedFileSystem.setPermission(DistributedFileSystem.java:572) at org.apache.spark.util.FileLogger.createLogDir(FileLogger.scala:138) at org.apache.spark.util.FileLogger.start(FileLogger.scala:115) at org.apache.spark.scheduler.EventLoggingListener.start(EventLoggingListener.scala:74) at org.apache.spark.SparkContext.(SparkContext.scala:324) ``` Author: Grace Closes #2892 from GraceH/eventlog-rpc and squashes the following commits: 58ea038 [Grace] new FsPermission Instance w/o FsPermission.createImmutable --- .../scala/org/apache/spark/scheduler/EventLoggingListener.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 100c9ba9b7809..597dbc884913c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -142,7 +142,7 @@ private[spark] object EventLoggingListener extends Logging { val SPARK_VERSION_PREFIX = "SPARK_VERSION_" val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" - val LOG_FILE_PERMISSIONS = FsPermission.createImmutable(Integer.parseInt("770", 8).toShort) + val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) // A cache for compression codecs to avoid creating the same codec many times private val codecMap = new mutable.HashMap[String, CompressionCodec] From 24c5129257ce6e3b734f168e860b714c2730b55f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:29:07 -0700 Subject: [PATCH 1141/1492] [SPARK-3319] [SPARK-3338] Resolve Spark submit config paths The bulk of this PR is comprised of tests. All changes in functionality are made in `SparkSubmit.scala` (~20 lines). **SPARK-3319.** There is currently a divergence in behavior when the user passes in additional jars through `--jars` and through setting `spark.jars` in the default properties file. The former will happily resolve the paths (e.g. convert `my.jar` to `file:/absolute/path/to/my.jar`), while the latter does not. We should resolve paths consistently in both cases. This also applies to the following pairs of command line arguments and Spark configs: - `--jars` ~ `spark.jars` - `--files` ~ `spark.files` / `spark.yarn.dist.files` - `--archives` ~ `spark.yarn.dist.archives` - `--py-files` ~ `spark.submit.pyFiles` **SPARK-3338.** This PR also fixes the following bug: if the user sets `spark.submit.pyFiles` in his/her properties file, it does not actually get picked up even if `--py-files` is not set. This is simply because the config is overridden by an empty string. Author: Andrew Or Author: Andrew Or Closes #2232 from andrewor14/resolve-config-paths and squashes the following commits: fff2869 [Andrew Or] Add spark.yarn.jar da3a1c1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into resolve-config-paths f0fae64 [Andrew Or] Merge branch 'master' of github.com:apache/spark into resolve-config-paths 05e03d6 [Andrew Or] Add tests for resolving both command line and config paths 460117e [Andrew Or] Resolve config paths properly fe039d3 [Andrew Or] Beef up tests to test fixed-pointed-ness of Utils.resolveURI(s) --- .../org/apache/spark/deploy/SparkSubmit.scala | 28 ++++- .../spark/deploy/SparkSubmitSuite.scala | 106 +++++++++++++++++- .../org/apache/spark/util/UtilsSuite.scala | 38 +++++-- 3 files changed, 158 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index f97bf67fa5a3b..0379adeb07b83 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -158,8 +158,9 @@ object SparkSubmit { args.files = mergeFileLists(args.files, args.primaryResource) } args.files = mergeFileLists(args.files, args.pyFiles) - // Format python file paths properly before adding them to the PYTHONPATH - sysProps("spark.submit.pyFiles") = PythonRunner.formatPaths(args.pyFiles).mkString(",") + if (args.pyFiles != null) { + sysProps("spark.submit.pyFiles") = args.pyFiles + } } // Special flag to avoid deprecation warnings at the client @@ -284,6 +285,29 @@ object SparkSubmit { sysProps.getOrElseUpdate(k, v) } + // Resolve paths in certain spark properties + val pathConfigs = Seq( + "spark.jars", + "spark.files", + "spark.yarn.jar", + "spark.yarn.dist.files", + "spark.yarn.dist.archives") + pathConfigs.foreach { config => + // Replace old URIs with resolved URIs, if they exist + sysProps.get(config).foreach { oldValue => + sysProps(config) = Utils.resolveURIs(oldValue) + } + } + + // Resolve and format python file paths properly before adding them to the PYTHONPATH. + // The resolving part is redundant in the case of --py-files, but necessary if the user + // explicitly sets `spark.submit.pyFiles` in his/her default properties file. + sysProps.get("spark.submit.pyFiles").foreach { pyFiles => + val resolvedPyFiles = Utils.resolveURIs(pyFiles) + val formattedPyFiles = PythonRunner.formatPaths(resolvedPyFiles).mkString(",") + sysProps("spark.submit.pyFiles") = formattedPyFiles + } + (childArgs, childClasspath, sysProps, childMainClass) } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 1cdf50d5c08c7..d8cd0ff2c9026 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -292,7 +292,7 @@ class SparkSubmitSuite extends FunSuite with Matchers { runSparkSubmit(args) } - test("spark submit includes jars passed in through --jar") { + test("includes jars passed in through --jars") { val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) @@ -306,6 +306,110 @@ class SparkSubmitSuite extends FunSuite with Matchers { runSparkSubmit(args) } + test("resolves command line argument paths correctly") { + val jars = "/jar1,/jar2" // --jars + val files = "hdfs:/file1,file2" // --files + val archives = "file:/archive1,archive2" // --archives + val pyFiles = "py-file1,py-file2" // --py-files + + // Test jars and files + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--jars", jars, + "--files", files, + "thejar.jar") + val appArgs = new SparkSubmitArguments(clArgs) + val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + appArgs.jars should be (Utils.resolveURIs(jars)) + appArgs.files should be (Utils.resolveURIs(files)) + sysProps("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar")) + sysProps("spark.files") should be (Utils.resolveURIs(files)) + + // Test files and archives (Yarn) + val clArgs2 = Seq( + "--master", "yarn-client", + "--class", "org.SomeClass", + "--files", files, + "--archives", archives, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + appArgs2.files should be (Utils.resolveURIs(files)) + appArgs2.archives should be (Utils.resolveURIs(archives)) + sysProps2("spark.yarn.dist.files") should be (Utils.resolveURIs(files)) + sysProps2("spark.yarn.dist.archives") should be (Utils.resolveURIs(archives)) + + // Test python files + val clArgs3 = Seq( + "--master", "local", + "--py-files", pyFiles, + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) + sysProps3("spark.submit.pyFiles") should be ( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + } + + test("resolves config paths correctly") { + val jars = "/jar1,/jar2" // spark.jars + val files = "hdfs:/file1,file2" // spark.files / spark.yarn.dist.files + val archives = "file:/archive1,archive2" // spark.yarn.dist.archives + val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles + + // Test jars and files + val f1 = File.createTempFile("test-submit-jars-files", "") + val writer1 = new PrintWriter(f1) + writer1.println("spark.jars " + jars) + writer1.println("spark.files " + files) + writer1.close() + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--properties-file", f1.getPath, + "thejar.jar" + ) + val appArgs = new SparkSubmitArguments(clArgs) + val sysProps = SparkSubmit.createLaunchEnv(appArgs)._3 + sysProps("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) + sysProps("spark.files") should be(Utils.resolveURIs(files)) + + // Test files and archives (Yarn) + val f2 = File.createTempFile("test-submit-files-archives", "") + val writer2 = new PrintWriter(f2) + writer2.println("spark.yarn.dist.files " + files) + writer2.println("spark.yarn.dist.archives " + archives) + writer2.close() + val clArgs2 = Seq( + "--master", "yarn-client", + "--class", "org.SomeClass", + "--properties-file", f2.getPath, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val sysProps2 = SparkSubmit.createLaunchEnv(appArgs2)._3 + sysProps2("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) + sysProps2("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) + + // Test python files + val f3 = File.createTempFile("test-submit-python-files", "") + val writer3 = new PrintWriter(f3) + writer3.println("spark.submit.pyFiles " + pyFiles) + writer3.close() + val clArgs3 = Seq( + "--master", "local", + "--properties-file", f3.getPath, + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val sysProps3 = SparkSubmit.createLaunchEnv(appArgs3)._3 + sysProps3("spark.submit.pyFiles") should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) + } + test("SPARK_CONF_DIR overrides spark-defaults.conf") { forConfDir(Map("spark.executor.memory" -> "2.3g")) { path => val unusedJar = TestUtils.createJarWithClasses(Seq.empty) diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 1c112334cc2f5..8ffe3e2b139c3 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -217,9 +217,14 @@ class UtilsSuite extends FunSuite { test("resolveURI") { def assertResolves(before: String, after: String, testWindows: Boolean = false): Unit = { - assume(before.split(",").length == 1) - assert(Utils.resolveURI(before, testWindows) === new URI(after)) - assert(Utils.resolveURI(after, testWindows) === new URI(after)) + // This should test only single paths + assume(before.split(",").length === 1) + // Repeated invocations of resolveURI should yield the same result + def resolve(uri: String): String = Utils.resolveURI(uri, testWindows).toString + assert(resolve(after) === after) + assert(resolve(resolve(after)) === after) + assert(resolve(resolve(resolve(after))) === after) + // Also test resolveURIs with single paths assert(new URI(Utils.resolveURIs(before, testWindows)) === new URI(after)) assert(new URI(Utils.resolveURIs(after, testWindows)) === new URI(after)) } @@ -235,16 +240,27 @@ class UtilsSuite extends FunSuite { assertResolves("file:/C:/file.txt#alias.txt", "file:/C:/file.txt#alias.txt", testWindows = true) intercept[IllegalArgumentException] { Utils.resolveURI("file:foo") } intercept[IllegalArgumentException] { Utils.resolveURI("file:foo:baby") } + } - // Test resolving comma-delimited paths - assert(Utils.resolveURIs("jar1,jar2") === s"file:$cwd/jar1,file:$cwd/jar2") - assert(Utils.resolveURIs("file:/jar1,file:/jar2") === "file:/jar1,file:/jar2") - assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3") === - s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") - assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,jar4#jar5") === + test("resolveURIs with multiple paths") { + def assertResolves(before: String, after: String, testWindows: Boolean = false): Unit = { + assume(before.split(",").length > 1) + assert(Utils.resolveURIs(before, testWindows) === after) + assert(Utils.resolveURIs(after, testWindows) === after) + // Repeated invocations of resolveURIs should yield the same result + def resolve(uri: String): String = Utils.resolveURIs(uri, testWindows) + assert(resolve(after) === after) + assert(resolve(resolve(after)) === after) + assert(resolve(resolve(resolve(after))) === after) + } + val cwd = System.getProperty("user.dir") + assertResolves("jar1,jar2", s"file:$cwd/jar1,file:$cwd/jar2") + assertResolves("file:/jar1,file:/jar2", "file:/jar1,file:/jar2") + assertResolves("hdfs:/jar1,file:/jar2,jar3", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3") + assertResolves("hdfs:/jar1,file:/jar2,jar3,jar4#jar5", s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:$cwd/jar4#jar5") - assert(Utils.resolveURIs("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", testWindows = true) === - s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi") + assertResolves("hdfs:/jar1,file:/jar2,jar3,C:\\pi.py#py.pi", + s"hdfs:/jar1,file:/jar2,file:$cwd/jar3,file:/C:/pi.py#py.pi", testWindows = true) } test("nonLocalPaths") { From 26f092d4e32cc1f7e279646075eaf1e495395923 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:31:23 -0700 Subject: [PATCH 1142/1492] [SPARK-4138][SPARK-4139] Improve dynamic allocation settings This should be merged after #2746 (SPARK-3795). **SPARK-4138**. If the user sets both the number of executors and `spark.dynamicAllocation.enabled`, we should throw an exception. **SPARK-4139**. If the user sets `spark.dynamicAllocation.enabled`, we should use the max number of executors as the starting number of executors because the first job is likely to run immediately after application startup. If the latter is not set, throw an exception. Author: Andrew Or Closes #3002 from andrewor14/yarn-set-executors and squashes the following commits: c528fce [Andrew Or] Merge branch 'master' of github.com:apache/spark into yarn-set-executors 55d4699 [Andrew Or] Bug fix: `isDynamicAllocationEnabled` was always false 2b0ccec [Andrew Or] Start the number of executors at the max 022bfde [Andrew Or] Guard against incompatible settings of number of executors --- .../yarn/ApplicationMasterArguments.scala | 3 +- .../spark/deploy/yarn/ClientArguments.scala | 30 ++++++++++++++----- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 ++ .../cluster/YarnClusterSchedulerBackend.scala | 4 +-- 4 files changed, 29 insertions(+), 10 deletions(-) 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 5c54e3400301a..104db4f65f030 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 @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import org.apache.spark.util.{MemoryParam, IntParam} +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import collection.mutable.ArrayBuffer class ApplicationMasterArguments(val args: Array[String]) { @@ -26,7 +27,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 var executorCores = 1 - var numExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS + var numExecutors = DEFAULT_NUMBER_EXECUTORS parseArgs(args.toList) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index a12f82d2fbe70..4d859450efc63 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -20,8 +20,8 @@ package org.apache.spark.deploy.yarn import scala.collection.mutable.ArrayBuffer import org.apache.spark.SparkConf -import org.apache.spark.util.{Utils, IntParam, MemoryParam} import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ +import org.apache.spark.util.{Utils, IntParam, MemoryParam} // TODO: Add code and support for ensuring that yarn resource 'tasks' are location aware ! private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) { @@ -33,23 +33,25 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 // MB var executorCores = 1 - var numExecutors = 2 + var numExecutors = DEFAULT_NUMBER_EXECUTORS var amQueue = sparkConf.get("spark.yarn.queue", "default") var amMemory: Int = 512 // MB var appName: String = "Spark" var priority = 0 - parseArgs(args.toList) - loadEnvironmentArgs() - // Additional memory to allocate to containers // For now, use driver's memory overhead as our AM container's memory overhead - val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", + val amMemoryOverhead = sparkConf.getInt("spark.yarn.driver.memoryOverhead", math.max((MEMORY_OVERHEAD_FACTOR * amMemory).toInt, MEMORY_OVERHEAD_MIN)) - val executorMemoryOverhead = sparkConf.getInt("spark.yarn.executor.memoryOverhead", + val executorMemoryOverhead = sparkConf.getInt("spark.yarn.executor.memoryOverhead", math.max((MEMORY_OVERHEAD_FACTOR * executorMemory).toInt, MEMORY_OVERHEAD_MIN)) + private val isDynamicAllocationEnabled = + sparkConf.getBoolean("spark.dynamicAllocation.enabled", false) + + parseArgs(args.toList) + loadEnvironmentArgs() validateArgs() /** Load any default arguments provided through environment variables and Spark properties. */ @@ -64,6 +66,15 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) .orElse(sys.env.get("SPARK_YARN_DIST_ARCHIVES")) .orElse(sparkConf.getOption("spark.yarn.dist.archives").map(p => Utils.resolveURIs(p))) .orNull + // If dynamic allocation is enabled, start at the max number of executors + if (isDynamicAllocationEnabled) { + val maxExecutorsConf = "spark.dynamicAllocation.maxExecutors" + if (!sparkConf.contains(maxExecutorsConf)) { + throw new IllegalArgumentException( + s"$maxExecutorsConf must be set if dynamic allocation is enabled!") + } + numExecutors = sparkConf.get(maxExecutorsConf).toInt + } } /** @@ -113,6 +124,11 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) if (args(0) == "--num-workers") { println("--num-workers is deprecated. Use --num-executors instead.") } + // Dynamic allocation is not compatible with this option + if (isDynamicAllocationEnabled) { + throw new IllegalArgumentException("Explicitly setting the number " + + "of executors is not compatible with spark.dynamicAllocation.enabled!") + } numExecutors = value args = tail diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index e1e0144f46fe9..7d453ecb7983c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -93,6 +93,8 @@ object YarnSparkHadoopUtil { val ANY_HOST = "*" + val DEFAULT_NUMBER_EXECUTORS = 2 + // All RM requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) val RM_REQUEST_PRIORITY = 1 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 index a96a54f66824c..b1de81e6a8b0f 100644 --- 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 @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.SparkContext -import org.apache.spark.deploy.yarn.ApplicationMasterArguments +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.IntParam @@ -29,7 +29,7 @@ private[spark] class YarnClusterSchedulerBackend( override def start() { super.start() - totalExpectedExecutors = ApplicationMasterArguments.DEFAULT_NUMBER_EXECUTORS + totalExpectedExecutors = DEFAULT_NUMBER_EXECUTORS if (System.getenv("SPARK_EXECUTOR_INSTANCES") != null) { totalExpectedExecutors = IntParam.unapply(System.getenv("SPARK_EXECUTOR_INSTANCES")) .getOrElse(totalExpectedExecutors) From 5231a3f228b5482cba09ae23a9f68498eba03c88 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:32:11 -0700 Subject: [PATCH 1143/1492] [Minor] A few typos in comments and log messages Author: Andrew Or Author: Andrew Or Closes #3021 from andrewor14/typos and squashes the following commits: daaf417 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos 4838ae4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into typos 026d426 [Andrew Or] Merge branch 'master' of github.com:andrewor14/spark into typos a81ae8f [Andrew Or] Some typos --- .../scala/org/apache/spark/scheduler/TaskResultGetter.scala | 2 +- .../main/scala/org/apache/spark/scheduler/TaskScheduler.scala | 4 ++-- .../apache/spark/deploy/yarn/ApplicationMasterArguments.scala | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 3f345ceeaaf7a..4b5be68ec5f92 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -93,7 +93,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } } catch { case cnd: ClassNotFoundException => - // Log an error but keep going here -- the task failed, so not catastropic if we can't + // Log an error but keep going here -- the task failed, so not catastrophic if we can't // deserialize the reason. val loader = Utils.getContextOrSparkClassLoader logError( diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index a129a434c9a1a..f095915352b17 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.BlockManagerId /** * Low-level task scheduler interface, currently implemented exclusively by TaskSchedulerImpl. - * This interface allows plugging in different task schedulers. Each TaskScheduler schedulers tasks + * This interface allows plugging in different task schedulers. Each TaskScheduler schedules tasks * for a single SparkContext. These schedulers get sets of tasks submitted to them from the * DAGScheduler for each stage, and are responsible for sending the tasks to the cluster, running * them, retrying if there are failures, and mitigating stragglers. They return events to the @@ -41,7 +41,7 @@ private[spark] trait TaskScheduler { // Invoked after system has successfully initialized (typically in spark context). // Yarn uses this to bootstrap allocation of resources based on preferred locations, - // wait for slave registerations, etc. + // wait for slave registrations, etc. def postStartHook() { } // Disconnect from the cluster. 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 104db4f65f030..8b32c76d14037 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 @@ -82,7 +82,7 @@ class ApplicationMasterArguments(val args: Array[String]) { | --jar JAR_PATH Path to your application's JAR file | --class CLASS_NAME Name of your application's main class | --args ARGS Arguments to be passed to your application's main class. - | Mutliple invocations are possible, each will be passed in order. + | Multiple invocations are possible, each will be passed in order. | --num-executors NUM Number of executors to start (Default: 2) | --executor-cores NUM Number of cores for the executors (Default: 1) | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) From 9334d699671edd8f18370255017ad40c1d0340ee Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:32:46 -0700 Subject: [PATCH 1144/1492] [SPARK-4155] Consolidate usages of We use "\" everywhere. Let's not do that. Author: Andrew Or Closes #3020 from andrewor14/consolidate-driver and squashes the following commits: c1c2204 [Andrew Or] Just use "" for local executor ID 3d751e9 [Andrew Or] Consolidate usages of --- .../apache/spark/ExecutorAllocationManager.scala | 2 +- .../main/scala/org/apache/spark/SparkContext.scala | 2 ++ core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- .../apache/spark/scheduler/local/LocalBackend.scala | 4 ++-- .../org/apache/spark/storage/BlockManagerId.scala | 3 ++- .../spark/storage/StorageStatusListener.scala | 13 ++----------- .../org/apache/spark/ui/exec/ExecutorsTab.scala | 6 ++---- .../storage/BlockManagerReplicationSuite.scala | 8 +++++--- .../apache/spark/storage/BlockManagerSuite.scala | 10 ++++++---- 9 files changed, 23 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index b2cf022baf29f..c11f1db0064fd 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -419,7 +419,7 @@ private[spark] class ExecutorAllocationManager(sc: SparkContext) extends Logging override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded): Unit = { val executorId = blockManagerAdded.blockManagerId.executorId - if (executorId != "") { + if (executorId != SparkContext.DRIVER_IDENTIFIER) { allocationManager.onExecutorAdded(executorId) } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 73668e83bbb1d..6bfcd8ceae3c1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1333,6 +1333,8 @@ object SparkContext extends Logging { private[spark] val SPARK_UNKNOWN_USER = "" + private[spark] val DRIVER_IDENTIFIER = "" + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 6a6dfda363974..557d2f51283ae 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -156,7 +156,7 @@ object SparkEnv extends Logging { assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") val hostname = conf.get("spark.driver.host") val port = conf.get("spark.driver.port").toInt - create(conf, "", hostname, port, true, isLocal, listenerBus) + create(conf, SparkContext.DRIVER_IDENTIFIER, hostname, port, true, isLocal, listenerBus) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 58b78f041cd85..c0264836de738 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import akka.actor.{Actor, ActorRef, Props} -import org.apache.spark.{Logging, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} @@ -47,7 +47,7 @@ private[spark] class LocalActor( private var freeCores = totalCores - private val localExecutorId = "localhost" + private val localExecutorId = SparkContext.DRIVER_IDENTIFIER private val localExecutorHostname = "localhost" val executor = new Executor( diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 259f423c73e6b..b177a59c721df 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} import java.util.concurrent.ConcurrentHashMap +import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -59,7 +60,7 @@ class BlockManagerId private ( def port: Int = port_ - def isDriver: Boolean = (executorId == "") + def isDriver: Boolean = { executorId == SparkContext.DRIVER_IDENTIFIER } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { out.writeUTF(executorId_) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index d9066f766476e..def49e80a3605 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import scala.collection.mutable +import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ @@ -59,10 +60,9 @@ class StorageStatusListener extends SparkListener { val info = taskEnd.taskInfo val metrics = taskEnd.taskMetrics if (info != null && metrics != null) { - val execId = formatExecutorId(info.executorId) val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) if (updatedBlocks.length > 0) { - updateStorageStatus(execId, updatedBlocks) + updateStorageStatus(info.executorId, updatedBlocks) } } } @@ -88,13 +88,4 @@ class StorageStatusListener extends SparkListener { } } - /** - * In the local mode, there is a discrepancy between the executor ID according to the - * task ("localhost") and that according to SparkEnv (""). In the UI, this - * results in duplicate rows for the same executor. Thus, in this mode, we aggregate - * these two rows and use the executor ID of "" to be consistent. - */ - def formatExecutorId(execId: String): String = { - if (execId == "localhost") "" else execId - } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 689cf02b25b70..9e0e71a51a408 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -48,14 +48,14 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp def storageStatusList = storageStatusListener.storageStatusList override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val eid = formatExecutorId(taskStart.taskInfo.executorId) + val eid = taskStart.taskInfo.executorId executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { val info = taskEnd.taskInfo if (info != null) { - val eid = formatExecutorId(info.executorId) + val eid = info.executorId executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration taskEnd.reason match { @@ -84,6 +84,4 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener) extends Sp } } - // This addresses executor ID inconsistencies in the local mode - private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 1f1d53a1ee3b0..c6d7105592096 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -27,7 +27,7 @@ import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.network.BlockTransferService import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus @@ -57,7 +57,9 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) - private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + private def makeBlockManager( + maxMem: Long, + name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) val store = new BlockManager(name, actorSystem, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer) @@ -108,7 +110,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd storeIds.filterNot { _ == stores(2).blockManagerId }) // Add driver store and test whether it is filtered out - val driverStore = makeBlockManager(1000, "") + val driverStore = makeBlockManager(1000, SparkContext.DRIVER_IDENTIFIER) assert(master.getPeers(stores(0).blockManagerId).forall(!_.isDriver)) assert(master.getPeers(stores(1).blockManagerId).forall(!_.isDriver)) assert(master.getPeers(stores(2).blockManagerId).forall(!_.isDriver)) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 9d96202a3e7ac..715b740b857b2 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -37,7 +37,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite, Matchers, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.Timeouts._ -import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SparkConf, SparkContext, SecurityManager} import org.apache.spark.executor.DataReadMethod import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus @@ -69,7 +69,9 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) - private def makeBlockManager(maxMem: Long, name: String = ""): BlockManager = { + private def makeBlockManager( + maxMem: Long, + name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { val transfer = new NioBlockTransferService(conf, securityMgr) new BlockManager(name, actorSystem, master, serializer, maxMem, conf, mapOutputTracker, shuffleManager, transfer) @@ -790,8 +792,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfter test("block store put failure") { // Use Java serializer so we can create an unserializable error. val transfer = new NioBlockTransferService(conf, securityMgr) - store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - mapOutputTracker, shuffleManager, transfer) + store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, actorSystem, master, + new JavaSerializer(conf), 1200, conf, mapOutputTracker, shuffleManager, transfer) // The put should fail since a1 is not serializable. class UnserializableClass From 849b43ec0f9e4f2ef962a054eb78cd0fc94a142a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 30 Oct 2014 15:33:34 -0700 Subject: [PATCH 1145/1492] Minor style hot fix after #2711 I had planned to fix this when I merged it but I forgot to. witgo Author: Andrew Or Closes #3018 from andrewor14/command-utils-style and squashes the following commits: c2959fb [Andrew Or] Style hot fix --- .../apache/spark/deploy/worker/CommandUtils.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index aba2e20118d7a..28e9662db5da9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -37,12 +37,12 @@ object CommandUtils extends Logging { * The `env` argument is exposed for testing. */ def buildProcessBuilder( - command: Command, - memory: Int, - sparkHome: String, - substituteArguments: String => String, - classPaths: Seq[String] = Seq[String](), - env: Map[String, String] = sys.env): ProcessBuilder = { + command: Command, + memory: Int, + sparkHome: String, + substituteArguments: String => String, + classPaths: Seq[String] = Seq[String](), + env: Map[String, String] = sys.env): ProcessBuilder = { val localCommand = buildLocalCommand(command, substituteArguments, classPaths, env) val commandSeq = buildCommandSeq(localCommand, memory, sparkHome) val builder = new ProcessBuilder(commandSeq: _*) From d3450578357d6f7598243ee2ab11c338085ad9c1 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 30 Oct 2014 15:33:56 -0700 Subject: [PATCH 1146/1492] [SPARK-4153][WebUI] Update the sort keys for HistoryPage Sort "Started", "Completed", "Duration" and "Last Updated" by time. Author: zsxwing Closes #3014 from zsxwing/SPARK-4153 and squashes the following commits: ec8b9ad [zsxwing] Sort "Started", "Completed", "Duration" and "Last Updated" by time --- .../org/apache/spark/deploy/history/HistoryPage.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index d25c29113d6da..0e249e51a77d8 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -84,11 +84,11 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") {
    {info.id} {info.name}{startTime}{endTime}{duration}{startTime}{endTime}{duration} {info.sparkUser}{lastUpdated}{lastUpdated}
    {h} -
      - { h.split("\n").map { case t =>
    • {t}
    • } } -
    -
    {getHeaderContent(x._1)}
    diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 2414e4c65237e..961224a300a70 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,10 +22,11 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, Unparsed} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.ui.{ToolTips, WebUIPage, UIUtils} import org.apache.spark.ui.jobs.UIData._ import org.apache.spark.util.{Utils, Distribution} -import org.apache.spark.scheduler.AccumulableInfo +import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { @@ -57,7 +58,6 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val hasShuffleWrite = stageData.shuffleWriteBytes > 0 val hasBytesSpilled = stageData.memoryBytesSpilled > 0 && stageData.diskBytesSpilled > 0 - // scalastyle:off val summary =
      @@ -65,55 +65,102 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { Total task time across all tasks: {UIUtils.formatDuration(stageData.executorRunTime)} - {if (hasInput) + {if (hasInput) {
    • Input: {Utils.bytesToString(stageData.inputBytes)}
    • - } - {if (hasShuffleRead) + }} + {if (hasShuffleRead) {
    • Shuffle read: {Utils.bytesToString(stageData.shuffleReadBytes)}
    • - } - {if (hasShuffleWrite) + }} + {if (hasShuffleWrite) {
    • Shuffle write: {Utils.bytesToString(stageData.shuffleWriteBytes)}
    • - } - {if (hasBytesSpilled) -
    • - Shuffle spill (memory): - {Utils.bytesToString(stageData.memoryBytesSpilled)} -
    • -
    • - Shuffle spill (disk): - {Utils.bytesToString(stageData.diskBytesSpilled)} -
    • - } + }} + {if (hasBytesSpilled) { +
    • + Shuffle spill (memory): + {Utils.bytesToString(stageData.memoryBytesSpilled)} +
    • +
    • + Shuffle spill (disk): + {Utils.bytesToString(stageData.diskBytesSpilled)} +
    • + }}
    - // scalastyle:on + + val showAdditionalMetrics = +
    + + + Show additional metrics + + +
    + val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") def accumulableRow(acc: AccumulableInfo) = val accumulableTable = UIUtils.listingTable(accumulableHeaders, accumulableRow, accumulables.values.toSeq) - val taskHeaders: Seq[String] = + val taskHeadersAndCssClasses: Seq[(String, String)] = Seq( - "Index", "ID", "Attempt", "Status", "Locality Level", "Executor ID / Host", - "Launch Time", "Duration", "GC Time", "Accumulators") ++ - {if (hasInput) Seq("Input") else Nil} ++ - {if (hasShuffleRead) Seq("Shuffle Read") else Nil} ++ - {if (hasShuffleWrite) Seq("Write Time", "Shuffle Write") else Nil} ++ - {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ - Seq("Errors") + ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), + ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Accumulators", ""), + ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), + ("GC Time", TaskDetailsClassNames.GC_TIME), + ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), + ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ + {if (hasInput) Seq(("Input", "")) else Nil} ++ + {if (hasShuffleRead) Seq(("Shuffle Read", "")) else Nil} ++ + {if (hasShuffleWrite) Seq(("Write Time", ""), ("Shuffle Write", "")) else Nil} ++ + {if (hasBytesSpilled) Seq(("Shuffle Spill (Memory)", ""), ("Shuffle Spill (Disk)", "")) + else Nil} ++ + Seq(("Errors", "")) + + val unzipped = taskHeadersAndCssClasses.unzip val taskTable = UIUtils.listingTable( - taskHeaders, taskRow(hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) - + unzipped._1, taskRow(hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks, + headerClasses = unzipped._2) // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) @@ -122,18 +169,37 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { None } else { - val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => - metrics.get.resultSerializationTime.toDouble + def getFormattedTimeQuantiles(times: Seq[Double]): Seq[Node] = { + Distribution(times).get.getQuantiles().map { millis => + + } } - val serializationQuantiles = - +: Distribution(serializationTimes). - get.getQuantiles().map(ms => ) val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } - val serviceQuantiles = +: Distribution(serviceTimes).get.getQuantiles() - .map(ms => ) + val serviceQuantiles = +: getFormattedTimeQuantiles(serviceTimes) + + val gcTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.jvmGCTime.toDouble + } + val gcQuantiles = + +: getFormattedTimeQuantiles(gcTimes) + + val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.resultSerializationTime.toDouble + } + val serializationQuantiles = + +: getFormattedTimeQuantiles(serializationTimes) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { @@ -142,76 +208,75 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { 0.0 } } - val gettingResultQuantiles = +: - Distribution(gettingResultTimes).get.getQuantiles().map { millis => - - } + val gettingResultQuantiles = + +: + getFormattedTimeQuantiles(gettingResultTimes) // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => - val totalExecutionTime = { - if (info.gettingResultTime > 0) { - (info.gettingResultTime - info.launchTime).toDouble - } else { - (info.finishTime - info.launchTime).toDouble - } - } - totalExecutionTime - metrics.get.executorRunTime + getSchedulerDelay(info, metrics.get).toDouble } val schedulerDelayTitle = + title={ToolTips.SCHEDULER_DELAY} data-placement="right">Scheduler Delay val schedulerDelayQuantiles = schedulerDelayTitle +: - Distribution(schedulerDelays).get.getQuantiles().map { millis => - - } + getFormattedTimeQuantiles(schedulerDelays) - def getQuantileCols(data: Seq[Double]) = + def getFormattedSizeQuantiles(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => ) val inputSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.inputMetrics.map(_.bytesRead).getOrElse(0L).toDouble } - val inputQuantiles = +: getQuantileCols(inputSizes) + val inputQuantiles = +: getFormattedSizeQuantiles(inputSizes) val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadQuantiles = +: - getQuantileCols(shuffleReadSizes) + getFormattedSizeQuantiles(shuffleReadSizes) val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } - val shuffleWriteQuantiles = +: getQuantileCols(shuffleWriteSizes) + val shuffleWriteQuantiles = +: + getFormattedSizeQuantiles(shuffleWriteSizes) val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.memoryBytesSpilled.toDouble } val memoryBytesSpilledQuantiles = +: - getQuantileCols(memoryBytesSpilledSizes) + getFormattedSizeQuantiles(memoryBytesSpilledSizes) val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.diskBytesSpilled.toDouble } val diskBytesSpilledQuantiles = +: - getQuantileCols(diskBytesSpilledSizes) + getFormattedSizeQuantiles(diskBytesSpilledSizes) val listings: Seq[Seq[Node]] = Seq( - serializationQuantiles, - serviceQuantiles, - gettingResultQuantiles, - schedulerDelayQuantiles, - if (hasInput) inputQuantiles else Nil, - if (hasShuffleRead) shuffleReadQuantiles else Nil, - if (hasShuffleWrite) shuffleWriteQuantiles else Nil, - if (hasBytesSpilled) memoryBytesSpilledQuantiles else Nil, - if (hasBytesSpilled) diskBytesSpilledQuantiles else Nil) + {serviceQuantiles}, + {schedulerDelayQuantiles}, + {gcQuantiles}, + + {serializationQuantiles} + , + {gettingResultQuantiles}, + if (hasInput) {inputQuantiles} else Nil, + if (hasShuffleRead) {shuffleReadQuantiles} else Nil, + if (hasShuffleWrite) {shuffleWriteQuantiles} else Nil, + if (hasBytesSpilled) {memoryBytesSpilledQuantiles} else Nil, + if (hasBytesSpilled) {diskBytesSpilledQuantiles} else Nil) val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") - def quantileRow(data: Seq[Node]): Seq[Node] = {data} - Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) + Some(UIUtils.listingTable( + quantileHeaders, identity[Seq[Node]], listings, fixedWidth = true)) } val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) @@ -221,6 +286,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val content = summary ++ + showAdditionalMetrics ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++ @@ -241,8 +307,10 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { else metrics.map(_.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") + val schedulerDelay = getSchedulerDelay(info, metrics.get) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) + val gettingResultTime = info.gettingResultTime val maybeInput = metrics.flatMap(_.inputMetrics) val inputSortable = maybeInput.map(_.bytesRead.toString).getOrElse("") @@ -287,20 +355,25 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { - + - {if (hasInput) { } } + + private def getSchedulerDelay(info: TaskInfo, metrics: TaskMetrics): Long = { + val totalExecutionTime = { + if (info.gettingResultTime > 0) { + (info.gettingResultTime - info.launchTime) + } else { + (info.finishTime - info.launchTime) + } + } + totalExecutionTime - metrics.executorRunTime + } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala new file mode 100644 index 0000000000000..23d672cabda07 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.jobs + +/** + * Names of the CSS classes corresponding to each type of task detail. Used to allow users + * to optionally show/hide columns. + */ +private object TaskDetailsClassNames { + val SCHEDULER_DELAY = "scheduler_delay" + val GC_TIME = "gc_time" + val RESULT_SERIALIZATION_TIME = "serialization_time" + val GETTING_RESULT_TIME = "getting_result_time" +} From 7c41d135709c148d4fa3a1b06b5905715c970519 Mon Sep 17 00:00:00 2001 From: wangfei Date: Fri, 31 Oct 2014 11:27:59 -0700 Subject: [PATCH 1159/1492] [SPARK-3826][SQL]enable hive-thriftserver to support hive-0.13.1 In #2241 hive-thriftserver is not enabled. This patch enable hive-thriftserver to support hive-0.13.1 by using a shim layer refer to #2241. 1 A light shim layer(code in sql/hive-thriftserver/hive-version) for each different hive version to handle api compatibility 2 New pom profiles "hive-default" and "hive-versions"(copy from #2241) to activate different hive version 3 SBT cmd for different version as follows: hive-0.12.0 --- sbt/sbt -Phive,hadoop-2.3 -Phive-0.12.0 assembly hive-0.13.1 --- sbt/sbt -Phive,hadoop-2.3 -Phive-0.13.1 assembly 4 Since hive-thriftserver depend on hive subproject, this patch should be merged with #2241 to enable hive-0.13.1 for hive-thriftserver Author: wangfei Author: scwf Closes #2685 from scwf/shim-thriftserver1 and squashes the following commits: f26f3be [wangfei] remove clean to save time f5cac74 [wangfei] remove local hivecontext test 578234d [wangfei] use new shaded hive 18fb1ff [wangfei] exclude kryo in hive pom fa21d09 [wangfei] clean package assembly/assembly 8a4daf2 [wangfei] minor fix 0d7f6cf [wangfei] address comments f7c93ae [wangfei] adding build with hive 0.13 before running tests bcf943f [wangfei] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver1 c359822 [wangfei] reuse getCommandProcessor in hiveshim 52674a4 [scwf] sql/hive included since examples depend on it 3529e98 [scwf] move hive module to hive profile f51ff4e [wangfei] update and fix conflicts f48d3a5 [scwf] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver1 41f727b [scwf] revert pom changes 13afde0 [scwf] fix small bug 4b681f4 [scwf] enable thriftserver in profile hive-0.13.1 0bc53aa [scwf] fixed when result filed is null dfd1c63 [scwf] update run-tests to run hive-0.12.0 default now c6da3ce [scwf] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver 7c66b8e [scwf] update pom according spark-2706 ae47489 [scwf] update and fix conflicts --- assembly/pom.xml | 6 - dev/run-tests | 13 +- pom.xml | 29 +- python/pyspark/sql.py | 27 -- sql/hive-thriftserver/pom.xml | 18 ++ ...ver.scala => AbstractSparkSQLDriver.scala} | 18 +- .../hive/thriftserver/SparkSQLCLIDriver.scala | 6 +- .../thriftserver/SparkSQLCLIService.scala | 19 +- .../server/SparkSQLOperationManager.scala | 169 +---------- .../spark/sql/hive/thriftserver/Shim12.scala | 225 +++++++++++++++ .../spark/sql/hive/thriftserver/Shim13.scala | 267 ++++++++++++++++++ sql/hive/pom.xml | 4 + 12 files changed, 571 insertions(+), 230 deletions(-) rename sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/{SparkSQLDriver.scala => AbstractSparkSQLDriver.scala} (86%) create mode 100644 sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala create mode 100644 sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala diff --git a/assembly/pom.xml b/assembly/pom.xml index 11d4bea9361ab..9e8525dd46157 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -201,12 +201,6 @@ spark-hive_${scala.binary.version} ${project.version} - - - - - hive-0.12.0 - org.apache.spark spark-hive-thriftserver_${scala.binary.version} diff --git a/dev/run-tests b/dev/run-tests index 972c8c8a21567..0e9eefa76a18b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -142,17 +142,24 @@ CURRENT_BLOCK=$BLOCK_BUILD # We always build with Hive because the PySpark Spark SQL tests need it. BUILD_MVN_PROFILE_ARGS="$SBT_MAVEN_PROFILES_ARGS -Phive -Phive-0.12.0" - echo "[info] Building Spark with these arguments: $BUILD_MVN_PROFILE_ARGS" # NOTE: echo "q" is needed because sbt on encountering a build file with failure #+ (either resolution or compilation) prompts the user for input either q, r, etc #+ to quit or retry. This echo is there to make it not block. - # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a + # NOTE: Do not quote $BUILD_MVN_PROFILE_ARGS or else it will be interpreted as a #+ single argument! # QUESTION: Why doesn't 'yes "q"' work? # QUESTION: Why doesn't 'grep -v -e "^\[info\] Resolving"' work? + # First build with 0.12 to ensure patches do not break the hive 12 build + echo "[info] Compile with hive 0.12" echo -e "q\n" \ - | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean package assembly/assembly \ + | sbt/sbt $BUILD_MVN_PROFILE_ARGS clean hive/compile hive-thriftserver/compile \ + | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" + + # Then build with default version(0.13.1) because tests are based on this version + echo "[info] Building Spark with these arguments: $SBT_MAVEN_PROFILES_ARGS -Phive" + echo -e "q\n" \ + | sbt/sbt $SBT_MAVEN_PROFILES_ARGS -Phive package assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" } diff --git a/pom.xml b/pom.xml index 379274d0b16e3..42fdbb9e097db 100644 --- a/pom.xml +++ b/pom.xml @@ -129,7 +129,7 @@ 1.4.0 3.4.5 - 0.13.1 + 0.13.1a 0.13.1 10.10.1.1 @@ -240,6 +240,18 @@ false + + + spark-staging-hive13 + Spring Staging Repository Hive 13 + https://oss.sonatype.org/content/repositories/orgspark-project-1089/ + + true + + + false + + @@ -908,9 +920,9 @@ by Spark SQL for code generation. --> - org.scalamacros - paradise_${scala.version} - ${scala.macros.version} + org.scalamacros + paradise_${scala.version} + ${scala.macros.version} @@ -1314,14 +1326,19 @@ - hive-0.12.0 + hive false - sql/hive-thriftserver + + + hive-0.12.0 + + false + 0.12.0-protobuf-2.5 0.12.0 diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 93fd9d49096b8..f0bd3cbd985da 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1400,33 +1400,6 @@ def hql(self, hqlQuery): class LocalHiveContext(HiveContext): - """Starts up an instance of hive where metadata is stored locally. - - An in-process metadata data is created with data stored in ./metadata. - Warehouse data is stored in in ./warehouse. - - >>> import os - >>> hiveCtx = LocalHiveContext(sc) - >>> try: - ... supress = hiveCtx.sql("DROP TABLE src") - ... except Exception: - ... pass - >>> kv1 = os.path.join(os.environ["SPARK_HOME"], - ... 'examples/src/main/resources/kv1.txt') - >>> supress = hiveCtx.sql( - ... "CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - >>> supress = hiveCtx.sql("LOAD DATA LOCAL INPATH '%s' INTO TABLE src" - ... % kv1) - >>> results = hiveCtx.sql("FROM src SELECT value" - ... ).map(lambda r: int(r.value.split('_')[1])) - >>> num = results.count() - >>> reduce_sum = results.reduce(lambda x, y: x + y) - >>> num - 500 - >>> reduce_sum - 130091 - """ - def __init__(self, sparkContext, sqlContext=None): HiveContext.__init__(self, sparkContext, sqlContext) warnings.warn("LocalHiveContext is deprecated. " diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 124fc107cb8aa..8db3010624100 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -70,6 +70,24 @@ org.scalatest scalatest-maven-plugin + + org.codehaus.mojo + build-helper-maven-plugin + + + add-default-sources + generate-sources + + add-source + + + + v${hive.version.short}/src/main/scala + + + + + org.apache.maven.plugins maven-deploy-plugin diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala similarity index 86% rename from sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala rename to sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala index a5c457c677564..fcb302edbffa8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala @@ -29,11 +29,11 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.Logging import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveContext) - extends Driver with Logging { +private[hive] abstract class AbstractSparkSQLDriver( + val context: HiveContext = SparkSQLEnv.hiveContext) extends Driver with Logging { - private var tableSchema: Schema = _ - private var hiveResponse: Seq[String] = _ + private[hive] var tableSchema: Schema = _ + private[hive] var hiveResponse: Seq[String] = _ override def init(): Unit = { } @@ -74,16 +74,6 @@ private[hive] class SparkSQLDriver(val context: HiveContext = SparkSQLEnv.hiveCo override def getSchema: Schema = tableSchema - override def getResults(res: JArrayList[String]): Boolean = { - if (hiveResponse == null) { - false - } else { - res.addAll(hiveResponse) - hiveResponse = null - true - } - } - override def destroy() { super.destroy() hiveResponse = null diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 7ba4564602ecd..2cd02ae9269f5 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -38,6 +38,8 @@ import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket import org.apache.spark.Logging +import org.apache.spark.sql.hive.HiveShim +import org.apache.spark.sql.hive.thriftserver.HiveThriftServerShim private[hive] object SparkSQLCLIDriver { private var prompt = "spark-sql" @@ -116,7 +118,7 @@ private[hive] object SparkSQLCLIDriver { } } - if (!sessionState.isRemoteMode && !ShimLoader.getHadoopShims.usesJobShell()) { + if (!sessionState.isRemoteMode) { // Hadoop-20 and above - we need to augment classpath using hiveconf // components. // See also: code in ExecDriver.java @@ -258,7 +260,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } else { var ret = 0 val hconf = conf.asInstanceOf[HiveConf] - val proc: CommandProcessor = CommandProcessorFactory.get(tokens(0), hconf) + val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf) if (proc != null) { if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 42cbf363b274f..a78311fc48635 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -24,6 +24,7 @@ import java.util.{List => JList} import javax.security.auth.login.LoginException import org.apache.commons.logging.Log +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hive.service.Service.STATE @@ -44,15 +45,17 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext) val sparkSqlSessionManager = new SparkSQLSessionManager(hiveContext) setSuperField(this, "sessionManager", sparkSqlSessionManager) addService(sparkSqlSessionManager) + var sparkServiceUGI: UserGroupInformation = null - try { - HiveAuthFactory.loginFromKeytab(hiveConf) - val serverUserName = ShimLoader.getHadoopShims - .getShortUserName(ShimLoader.getHadoopShims.getUGIForConf(hiveConf)) - setSuperField(this, "serverUserName", serverUserName) - } catch { - case e @ (_: IOException | _: LoginException) => - throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + if (ShimLoader.getHadoopShims().isSecurityEnabled()) { + try { + HiveAuthFactory.loginFromKeytab(hiveConf) + sparkServiceUGI = ShimLoader.getHadoopShims.getUGIForConf(hiveConf) + HiveThriftServerShim.setServerUserName(sparkServiceUGI, this) + } catch { + case e @ (_: IOException | _: LoginException) => + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e) + } } initCompositeService(hiveConf) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index accf61576b804..2a4f24132cc5e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -17,24 +17,15 @@ package org.apache.spark.sql.hive.thriftserver.server -import java.sql.Timestamp import java.util.{Map => JMap} +import scala.collection.mutable.Map -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, Map} -import scala.math.{random, round} - -import org.apache.hadoop.hive.common.`type`.HiveDecimal -import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging -import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} -import org.apache.spark.sql.catalyst.plans.logical.SetCommand -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.hive.thriftserver.ReflectionUtils +import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.thriftserver.{SparkExecuteStatementOperation, ReflectionUtils} /** * Executes queries using Spark SQL, and maintains a list of handles to active queries. @@ -54,158 +45,8 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) confOverlay: JMap[String, String], async: Boolean): ExecuteStatementOperation = synchronized { - val operation = new ExecuteStatementOperation(parentSession, statement, confOverlay) { - private var result: SchemaRDD = _ - private var iter: Iterator[SparkRow] = _ - private var dataTypes: Array[DataType] = _ - - def close(): Unit = { - // RDDs will be cleaned automatically upon garbage collection. - logDebug("CLOSING") - } - - def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { - if (!iter.hasNext) { - new RowSet() - } else { - // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int - val maxRows = maxRowsL.toInt - var curRow = 0 - var rowSet = new ArrayBuffer[Row](maxRows.min(1024)) - - while (curRow < maxRows && iter.hasNext) { - val sparkRow = iter.next() - val row = new Row() - var curCol = 0 - - while (curCol < sparkRow.length) { - if (sparkRow.isNullAt(curCol)) { - addNullColumnValue(sparkRow, row, curCol) - } else { - addNonNullColumnValue(sparkRow, row, curCol) - } - curCol += 1 - } - rowSet += row - curRow += 1 - } - new RowSet(rowSet, 0) - } - } - - def addNonNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { - dataTypes(ordinal) match { - case StringType => - to.addString(from(ordinal).asInstanceOf[String]) - case IntegerType => - to.addColumnValue(ColumnValue.intValue(from.getInt(ordinal))) - case BooleanType => - to.addColumnValue(ColumnValue.booleanValue(from.getBoolean(ordinal))) - case DoubleType => - to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal))) - case FloatType => - to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal))) - case DecimalType => - val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal - to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) - case LongType => - to.addColumnValue(ColumnValue.longValue(from.getLong(ordinal))) - case ByteType => - to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) - case ShortType => - to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) - case TimestampType => - to.addColumnValue( - ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - val hiveString = result - .queryExecution - .asInstanceOf[HiveContext#QueryExecution] - .toHiveString((from.get(ordinal), dataTypes(ordinal))) - to.addColumnValue(ColumnValue.stringValue(hiveString)) - } - } - - def addNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { - dataTypes(ordinal) match { - case StringType => - to.addString(null) - case IntegerType => - to.addColumnValue(ColumnValue.intValue(null)) - case BooleanType => - to.addColumnValue(ColumnValue.booleanValue(null)) - case DoubleType => - to.addColumnValue(ColumnValue.doubleValue(null)) - case FloatType => - to.addColumnValue(ColumnValue.floatValue(null)) - case DecimalType => - to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal)) - case LongType => - to.addColumnValue(ColumnValue.longValue(null)) - case ByteType => - to.addColumnValue(ColumnValue.byteValue(null)) - case ShortType => - to.addColumnValue(ColumnValue.shortValue(null)) - case TimestampType => - to.addColumnValue(ColumnValue.timestampValue(null)) - case BinaryType | _: ArrayType | _: StructType | _: MapType => - to.addColumnValue(ColumnValue.stringValue(null: String)) - } - } - - def getResultSetSchema: TableSchema = { - logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") - if (result.queryExecution.analyzed.output.size == 0) { - new TableSchema(new FieldSchema("Result", "string", "") :: Nil) - } else { - val schema = result.queryExecution.analyzed.output.map { attr => - new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") - } - new TableSchema(schema) - } - } - - def run(): Unit = { - logInfo(s"Running query '$statement'") - setState(OperationState.RUNNING) - try { - result = hiveContext.sql(statement) - logDebug(result.queryExecution.toString()) - result.queryExecution.logical match { - case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => - sessionToActivePool(parentSession) = value - logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") - case _ => - } - - val groupId = round(random * 1000000).toString - hiveContext.sparkContext.setJobGroup(groupId, statement) - sessionToActivePool.get(parentSession).foreach { pool => - hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) - } - iter = { - val resultRdd = result.queryExecution.toRdd - val useIncrementalCollect = - hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean - if (useIncrementalCollect) { - resultRdd.toLocalIterator - } else { - resultRdd.collect().iterator - } - } - dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray - setHasResultSet(true) - } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - logError("Error executing query:",e) - throw new HiveSQLException(e.toString) - } - setState(OperationState.FINISHED) - } - } - + val operation = new SparkExecuteStatementOperation(parentSession, statement, confOverlay)( + hiveContext, sessionToActivePool) handleToOperation.put(operation.getHandle, operation) operation } diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala new file mode 100644 index 0000000000000..bbd727c686bbc --- /dev/null +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -0,0 +1,225 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.sql.Timestamp +import java.util.{ArrayList => JArrayList, Map => JMap} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, Map => SMap} +import scala.math._ + +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.ExecuteStatementOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.plans.logical.SetCommand +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} +import org.apache.spark.sql.hive.{HiveMetastoreTypes, HiveContext} +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[thriftserver] object HiveThriftServerShim { + val version = "0.12.0" + + def setServerUserName(sparkServiceUGI: UserGroupInformation, sparkCliService:SparkSQLCLIService) = { + val serverUserName = ShimLoader.getHadoopShims.getShortUserName(sparkServiceUGI) + setSuperField(sparkCliService, "serverUserName", serverUserName) + } +} + +private[hive] class SparkSQLDriver(val _context: HiveContext = SparkSQLEnv.hiveContext) + extends AbstractSparkSQLDriver(_context) { + override def getResults(res: JArrayList[String]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.addAll(hiveResponse) + hiveResponse = null + true + } + } +} + +private[hive] class SparkExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String])( + hiveContext: HiveContext, + sessionToActivePool: SMap[HiveSession, String]) extends ExecuteStatementOperation( + parentSession, statement, confOverlay) with Logging { + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logDebug("CLOSING") + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + if (!iter.hasNext) { + new RowSet() + } else { + // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int + val maxRows = maxRowsL.toInt + var curRow = 0 + var rowSet = new ArrayBuffer[Row](maxRows.min(1024)) + + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = new Row() + var curCol = 0 + + while (curCol < sparkRow.length) { + if (sparkRow.isNullAt(curCol)) { + addNullColumnValue(sparkRow, row, curCol) + } else { + addNonNullColumnValue(sparkRow, row, curCol) + } + curCol += 1 + } + rowSet += row + curRow += 1 + } + new RowSet(rowSet, 0) + } + } + + def addNonNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(from(ordinal).asInstanceOf[String]) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(from.getInt(ordinal))) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(from.getBoolean(ordinal))) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal))) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal))) + case DecimalType => + val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal + to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) + case LongType => + to.addColumnValue(ColumnValue.longValue(from.getLong(ordinal))) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(from.getByte(ordinal))) + case ShortType => + to.addColumnValue(ColumnValue.shortValue(from.getShort(ordinal))) + case TimestampType => + to.addColumnValue( + ColumnValue.timestampValue(from.get(ordinal).asInstanceOf[Timestamp])) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + val hiveString = result + .queryExecution + .asInstanceOf[HiveContext#QueryExecution] + .toHiveString((from.get(ordinal), dataTypes(ordinal))) + to.addColumnValue(ColumnValue.stringValue(hiveString)) + } + } + + def addNullColumnValue(from: SparkRow, to: Row, ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to.addString(null) + case IntegerType => + to.addColumnValue(ColumnValue.intValue(null)) + case BooleanType => + to.addColumnValue(ColumnValue.booleanValue(null)) + case DoubleType => + to.addColumnValue(ColumnValue.doubleValue(null)) + case FloatType => + to.addColumnValue(ColumnValue.floatValue(null)) + case DecimalType => + to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal)) + case LongType => + to.addColumnValue(ColumnValue.longValue(null)) + case ByteType => + to.addColumnValue(ColumnValue.byteValue(null)) + case ShortType => + to.addColumnValue(ColumnValue.shortValue(null)) + case TimestampType => + to.addColumnValue(ColumnValue.timestampValue(null)) + case BinaryType | _: ArrayType | _: StructType | _: MapType => + to.addColumnValue(ColumnValue.stringValue(null: String)) + } + } + + def getResultSetSchema: TableSchema = { + logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + def run(): Unit = { + logInfo(s"Running query '$statement'") + setState(OperationState.RUNNING) + try { + result = hiveContext.sql(statement) + logDebug(result.queryExecution.toString()) + result.queryExecution.logical match { + case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => + sessionToActivePool(parentSession) = value + logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") + case _ => + } + + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + sessionToActivePool.get(parentSession).foreach { pool => + hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + } + iter = { + val resultRdd = result.queryExecution.toRdd + val useIncrementalCollect = + hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean + if (useIncrementalCollect) { + resultRdd.toLocalIterator + } else { + resultRdd.collect().iterator + } + } + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + setHasResultSet(true) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logError("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + setState(OperationState.FINISHED) + } +} diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala new file mode 100644 index 0000000000000..e59681bfbe43e --- /dev/null +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -0,0 +1,267 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.security.PrivilegedExceptionAction +import java.sql.Timestamp +import java.util.concurrent.Future +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, Map => SMap} +import scala.math._ + +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.metastore.api.FieldSchema +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.ExecuteStatementOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.{Row => SparkRow, SchemaRDD} +import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} +import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ + +/** + * A compatibility layer for interacting with Hive version 0.12.0. + */ +private[thriftserver] object HiveThriftServerShim { + val version = "0.13.1" + + def setServerUserName(sparkServiceUGI: UserGroupInformation, sparkCliService:SparkSQLCLIService) = { + setSuperField(sparkCliService, "serviceUGI", sparkServiceUGI) + } +} + +private[hive] class SparkSQLDriver(val _context: HiveContext = SparkSQLEnv.hiveContext) + extends AbstractSparkSQLDriver(_context) { + override def getResults(res: JList[_]): Boolean = { + if (hiveResponse == null) { + false + } else { + res.asInstanceOf[JArrayList[String]].addAll(hiveResponse) + hiveResponse = null + true + } + } +} + +private[hive] class SparkExecuteStatementOperation( + parentSession: HiveSession, + statement: String, + confOverlay: JMap[String, String], + runInBackground: Boolean = true)( + hiveContext: HiveContext, + sessionToActivePool: SMap[HiveSession, String]) extends ExecuteStatementOperation( + parentSession, statement, confOverlay, runInBackground) with Logging { + + private var result: SchemaRDD = _ + private var iter: Iterator[SparkRow] = _ + private var dataTypes: Array[DataType] = _ + + private def runInternal(cmd: String) = { + try { + result = hiveContext.sql(cmd) + logDebug(result.queryExecution.toString()) + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + iter = { + val resultRdd = result.queryExecution.toRdd + val useIncrementalCollect = + hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean + if (useIncrementalCollect) { + resultRdd.toLocalIterator + } else { + resultRdd.collect().iterator + } + } + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logError("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + } + + def close(): Unit = { + // RDDs will be cleaned automatically upon garbage collection. + logDebug("CLOSING") + } + + def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { + dataTypes(ordinal) match { + case StringType => + to += from.get(ordinal).asInstanceOf[String] + case IntegerType => + to += from.getInt(ordinal) + case BooleanType => + to += from.getBoolean(ordinal) + case DoubleType => + to += from.getDouble(ordinal) + case FloatType => + to += from.getFloat(ordinal) + case DecimalType => + to += from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal + case LongType => + to += from.getLong(ordinal) + case ByteType => + to += from.getByte(ordinal) + case ShortType => + to += from.getShort(ordinal) + case TimestampType => + to += from.get(ordinal).asInstanceOf[Timestamp] + case BinaryType => + to += from.get(ordinal).asInstanceOf[String] + case _: ArrayType => + to += from.get(ordinal).asInstanceOf[String] + case _: StructType => + to += from.get(ordinal).asInstanceOf[String] + case _: MapType => + to += from.get(ordinal).asInstanceOf[String] + } + } + + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + validateDefaultFetchOrientation(order) + assertState(OperationState.FINISHED) + setHasResultSet(true) + val reultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) + if (!iter.hasNext) { + reultRowSet + } else { + // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int + val maxRows = maxRowsL.toInt + var curRow = 0 + while (curRow < maxRows && iter.hasNext) { + val sparkRow = iter.next() + val row = ArrayBuffer[Any]() + var curCol = 0 + while (curCol < sparkRow.length) { + if (sparkRow.isNullAt(curCol)) { + row += null + } else { + addNonNullColumnValue(sparkRow, row, curCol) + } + curCol += 1 + } + reultRowSet.addRow(row.toArray.asInstanceOf[Array[Object]]) + curRow += 1 + } + reultRowSet + } + } + + def getResultSetSchema: TableSchema = { + logInfo(s"Result Schema: ${result.queryExecution.analyzed.output}") + if (result.queryExecution.analyzed.output.size == 0) { + new TableSchema(new FieldSchema("Result", "string", "") :: Nil) + } else { + val schema = result.queryExecution.analyzed.output.map { attr => + new FieldSchema(attr.name, HiveMetastoreTypes.toMetastoreType(attr.dataType), "") + } + new TableSchema(schema) + } + } + + private def getConfigForOperation: HiveConf = { + var sqlOperationConf: HiveConf = getParentSession.getHiveConf + if (!getConfOverlay.isEmpty || shouldRunAsync) { + sqlOperationConf = new HiveConf(sqlOperationConf) + import scala.collection.JavaConversions._ + for (confEntry <- getConfOverlay.entrySet) { + try { + sqlOperationConf.verifyAndSet(confEntry.getKey, confEntry.getValue) + } + catch { + case e: IllegalArgumentException => { + throw new HiveSQLException("Error applying statement specific settings", e) + } + } + } + } + return sqlOperationConf + } + + def run(): Unit = { + logInfo(s"Running query '$statement'") + val opConfig: HiveConf = getConfigForOperation + setState(OperationState.RUNNING) + setHasResultSet(true) + + if (!shouldRunAsync) { + runInternal(statement) + setState(OperationState.FINISHED) + } else { + val parentSessionState = SessionState.get + val sessionHive: Hive = Hive.get + val currentUGI: UserGroupInformation = ShimLoader.getHadoopShims.getUGIForConf(opConfig) + + val backgroundOperation: Runnable = new Runnable { + def run { + val doAsAction: PrivilegedExceptionAction[AnyRef] = + new PrivilegedExceptionAction[AnyRef] { + def run: AnyRef = { + Hive.set(sessionHive) + SessionState.setCurrentSessionState(parentSessionState) + try { + runInternal(statement) + } + catch { + case e: HiveSQLException => { + setOperationException(e) + logError("Error running hive query: ", e) + } + } + return null + } + } + try { + ShimLoader.getHadoopShims.doAs(currentUGI, doAsAction) + } + catch { + case e: Exception => { + setOperationException(new HiveSQLException(e)) + logError("Error running hive query as user : " + currentUGI.getShortUserName, e) + } + } + setState(OperationState.FINISHED) + } + } + + try { + val backgroundHandle: Future[_] = getParentSession.getSessionManager. + submitBackgroundOperation(backgroundOperation) + setBackgroundHandle(backgroundHandle) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + logError("Error executing query:",e) + throw new HiveSQLException(e.toString) + } + } + } +} diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index db01363b4d629..67e36a951e506 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -65,6 +65,10 @@ commons-logging commons-logging + + com.esotericsoftware.kryo + kryo + From fa712b309c0e59943aae289dab629b34a13fe20e Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Fri, 31 Oct 2014 11:30:28 -0700 Subject: [PATCH 1160/1492] [SPARK-4077][SQL] Spark SQL return wrong values for valid string timestamp values In org.apache.hadoop.hive.serde2.io.TimestampWritable.set , if the next entry is null then current time stamp object is being reset. However because of this hiveinspectors:unwrap cannot use the same timestamp object without creating a copy. Author: Venkata Ramana G Author: Venkata Ramana Gollamudi Closes #3019 from gvramana/spark_4077 and squashes the following commits: 32d818f [Venkata Ramana Gollamudi] fixed check style fa01e71 [Venkata Ramana Gollamudi] cloned timestamp object as org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object --- .../spark/sql/hive/HiveInspectors.scala | 3 +++ .../resources/data/files/issue-4077-data.txt | 2 ++ .../hive/execution/HiveTableScanSuite.scala | 22 +++++++++++++++++++ 3 files changed, 27 insertions(+) create mode 100644 sql/hive/src/test/resources/data/files/issue-4077-data.txt diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index c6103a124df59..0439ab97d80be 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -91,6 +91,9 @@ private[hive] trait HiveInspectors { if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue case hdoi: HiveDecimalObjectInspector => if (data == null) null else BigDecimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object + // if next timestamp is null, so Timestamp object is cloned + case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone() case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) case li: ListObjectInspector => Option(li.getList(data)) diff --git a/sql/hive/src/test/resources/data/files/issue-4077-data.txt b/sql/hive/src/test/resources/data/files/issue-4077-data.txt new file mode 100644 index 0000000000000..18067b0a64c9c --- /dev/null +++ b/sql/hive/src/test/resources/data/files/issue-4077-data.txt @@ -0,0 +1,2 @@ +2014-12-11 00:00:00,1 +2014-12-11astring00:00:00,2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala index 2f3db95882093..54c0f017d4cb6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala @@ -18,6 +18,9 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.{Row, SchemaRDD} + +import org.apache.spark.util.Utils class HiveTableScanSuite extends HiveComparisonTest { @@ -47,4 +50,23 @@ class HiveTableScanSuite extends HiveComparisonTest { TestHive.sql("select KEY from tb where VALUE='just_for_test' limit 5").collect() TestHive.sql("drop table tb") } + + test("Spark-4077: timestamp query for null value") { + TestHive.sql("DROP TABLE IF EXISTS timestamp_query_null") + TestHive.sql( + """ + CREATE EXTERNAL TABLE timestamp_query_null (time TIMESTAMP,id INT) + ROW FORMAT DELIMITED + FIELDS TERMINATED BY ',' + LINES TERMINATED BY '\n' + """.stripMargin) + val location = + Utils.getSparkClassLoader.getResource("data/files/issue-4077-data.txt").getFile() + + TestHive.sql(s"LOAD DATA LOCAL INPATH '$location' INTO TABLE timestamp_query_null") + assert(TestHive.sql("SELECT time from timestamp_query_null limit 2").collect() + === Array(Row(java.sql.Timestamp.valueOf("2014-12-11 00:00:00")),Row(null))) + TestHive.sql("DROP TABLE timestamp_query_null") + } + } From ea465af12ddae424af9b4e742c3d5aed2a0bc8ec Mon Sep 17 00:00:00 2001 From: ravipesala Date: Fri, 31 Oct 2014 11:33:20 -0700 Subject: [PATCH 1161/1492] [SPARK-4154][SQL] Query does not work if it has "not between " in Spark SQL and HQL if the query contains "not between" does not work like. SELECT * FROM src where key not between 10 and 20' Author: ravipesala Closes #3017 from ravipesala/SPARK-4154 and squashes the following commits: 65fc89e [ravipesala] Handled admin comments 32e6d42 [ravipesala] 'not between' is not working --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 6 ++++-- .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 +++++ .../scala/org/apache/spark/sql/hive/HiveQl.scala | 13 +++++++++---- .../spark/sql/hive/execution/SQLQuerySuite.scala | 5 +++++ 4 files changed, 23 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 942b843104958..b1e7570f577c3 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -232,8 +232,10 @@ class SqlParser extends AbstractSparkSQLParser { | termExpression ~ (">=" ~> termExpression) ^^ { case e1 ~ e2 => GreaterThanOrEqual(e1, e2) } | termExpression ~ ("!=" ~> termExpression) ^^ { case e1 ~ e2 => Not(EqualTo(e1, e2)) } | termExpression ~ ("<>" ~> termExpression) ^^ { case e1 ~ e2 => Not(EqualTo(e1, e2)) } - | termExpression ~ (BETWEEN ~> termExpression) ~ (AND ~> termExpression) ^^ { - case e ~ el ~ eu => And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + | termExpression ~ NOT.? ~ (BETWEEN ~> termExpression) ~ (AND ~> termExpression) ^^ { + case e ~ not ~ el ~ eu => + val betweenExpr: Expression = And(GreaterThanOrEqual(e, el), LessThanOrEqual(e, eu)) + not.fold(betweenExpr)(f=> Not(betweenExpr)) } | termExpression ~ (RLIKE ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } | termExpression ~ (REGEXP ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4c36ca0f41174..4acd92d33d180 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -909,4 +909,9 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { """.stripMargin), (1 to 100).map(i => Seq(i, i, i))) } + + test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") { + checkAnswer(sql("SELECT key FROM testData WHERE key not between 0 and 10 order by key"), + (11 to 100).map(i => Seq(i))) + } } 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 9d9d68affa54b..a3573e6502de8 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 @@ -985,15 +985,20 @@ private[hive] object HiveQl { In(nodeToExpr(value), list.map(nodeToExpr)) case Token("TOK_FUNCTION", Token(BETWEEN(), Nil) :: - Token("KW_FALSE", Nil) :: + kw :: target :: minValue :: maxValue :: Nil) => val targetExpression = nodeToExpr(target) - And( - GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), - LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) + val betweenExpr = + And( + GreaterThanOrEqual(targetExpression, nodeToExpr(minValue)), + LessThanOrEqual(targetExpression, nodeToExpr(maxValue))) + kw match { + case Token("KW_FALSE", Nil) => betweenExpr + case Token("KW_TRUE", Nil) => Not(betweenExpr) + } /* Boolean Logic */ case Token(AND(), left :: right:: Nil) => And(nodeToExpr(left), nodeToExpr(right)) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 4f96a327ee2c7..76a0ec01a6075 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -158,4 +158,9 @@ class SQLQuerySuite extends QueryTest { sql("SELECT case when ~1=-2 then 1 else 0 end FROM src"), sql("SELECT 1 FROM src").collect().toSeq) } + + test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") { + checkAnswer(sql("SELECT key FROM src WHERE key not between 0 and 10 order by key"), + sql("SELECT key FROM src WHERE key between 11 and 500 order by key").collect().toSeq) + } } From 23468e7e96bf047ba53806352558b9d661567b23 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 31 Oct 2014 11:34:51 -0700 Subject: [PATCH 1162/1492] [SPARK-2220][SQL] Fixes remaining Hive commands This PR adds support for the `ADD FILE` Hive command, and removes `ShellCommand` and `SourceCommand`. The reason is described in [this SPARK-2220 comment](https://issues.apache.org/jira/browse/SPARK-2220?focusedCommentId=14191841&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14191841). Author: Cheng Lian Closes #3038 from liancheng/hive-commands and squashes the following commits: 6db61e0 [Cheng Lian] Fixes remaining Hive commands --- .../spark/sql/catalyst/SparkSQLParser.scala | 14 +----------- .../sql/catalyst/plans/logical/commands.scala | 22 +++++++++---------- .../spark/sql/hive/HiveStrategies.scala | 2 ++ .../spark/sql/hive/execution/commands.scala | 16 ++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 19 +++++++++++++--- 5 files changed, 45 insertions(+), 28 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala index 12e8346a6445d..f5c19ee69c37a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -137,7 +137,6 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr protected val LAZY = Keyword("LAZY") protected val SET = Keyword("SET") protected val TABLE = Keyword("TABLE") - protected val SOURCE = Keyword("SOURCE") protected val UNCACHE = Keyword("UNCACHE") protected implicit def asParser(k: Keyword): Parser[String] = @@ -152,8 +151,7 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr override val lexical = new SqlLexical(reservedWords) - override protected lazy val start: Parser[LogicalPlan] = - cache | uncache | set | shell | source | others + override protected lazy val start: Parser[LogicalPlan] = cache | uncache | set | others private lazy val cache: Parser[LogicalPlan] = CACHE ~> LAZY.? ~ (TABLE ~> ident) ~ (AS ~> restInput).? ^^ { @@ -171,16 +169,6 @@ private[sql] class SparkSQLParser(fallback: String => LogicalPlan) extends Abstr case input => SetCommandParser(input) } - private lazy val shell: Parser[LogicalPlan] = - "!" ~> restInput ^^ { - case input => ShellCommand(input.trim) - } - - private lazy val source: Parser[LogicalPlan] = - SOURCE ~> restInput ^^ { - case input => SourceCommand(input.trim) - } - private lazy val others: Parser[LogicalPlan] = wholeInput ^^ { case input => fallback(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala index b8ba2ee428a20..1d513d7789763 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/commands.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, BoundReference} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.types.StringType /** @@ -41,6 +41,15 @@ case class NativeCommand(cmd: String) extends Command { /** * Commands of the form "SET [key [= value] ]". */ +case class DFSCommand(kv: Option[(String, Option[String])]) extends Command { + override def output = Seq( + AttributeReference("DFS output", StringType, nullable = false)()) +} + +/** + * + * Commands of the form "SET [key [= value] ]". + */ case class SetCommand(kv: Option[(String, Option[String])]) extends Command { override def output = Seq( AttributeReference("", StringType, nullable = false)()) @@ -81,14 +90,3 @@ case class DescribeCommand( AttributeReference("data_type", StringType, nullable = false)(), AttributeReference("comment", StringType, nullable = false)()) } - -/** - * Returned for the "! shellCommand" command - */ -case class ShellCommand(cmd: String) extends Command - - -/** - * Returned for the "SOURCE file" command - */ -case class SourceCommand(filePath: String) extends Command diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index e59d4d536a0af..3207ad81d9571 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -206,6 +206,8 @@ private[hive] trait HiveStrategies { case hive.AddJar(path) => execution.AddJar(path) :: Nil + case hive.AddFile(path) => execution.AddFile(path) :: Nil + case hive.AnalyzeTable(tableName) => execution.AnalyzeTable(tableName) :: Nil case describe: logical.DescribeCommand => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 0fc674af31885..903075edf7e04 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -76,3 +76,19 @@ case class AddJar(path: String) extends LeafNode with Command { Seq.empty[Row] } } + +/** + * :: DeveloperApi :: + */ +@DeveloperApi +case class AddFile(path: String) extends LeafNode with Command { + def hiveContext = sqlContext.asInstanceOf[HiveContext] + + override def output = Seq.empty + + override protected lazy val sideEffectResult: Seq[Row] = { + hiveContext.runSqlHive(s"ADD FILE $path") + hiveContext.sparkContext.addFile(path) + Seq.empty[Row] + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index ffe1f0b90fcd0..5918f888c8f4c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.hive.execution +import java.io.File + import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.spark.SparkException +import org.apache.spark.{SparkFiles, SparkException} import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.test.TestHive @@ -569,7 +571,7 @@ class HiveQuerySuite extends HiveComparisonTest { |WITH serdeproperties('s1'='9') """.stripMargin) } - // Now only verify 0.12.0, and ignore other versions due to binary compatability + // Now only verify 0.12.0, and ignore other versions due to binary compatibility // current TestSerDe.jar is from 0.12.0 if (HiveShim.version == "0.12.0") { sql(s"ADD JAR $testJar") @@ -581,6 +583,17 @@ class HiveQuerySuite extends HiveComparisonTest { sql("DROP TABLE alter1") } + test("ADD FILE command") { + val testFile = TestHive.getHiveFile("data/files/v1.txt").getCanonicalFile + sql(s"ADD FILE $testFile") + + val checkAddFileRDD = sparkContext.parallelize(1 to 2, 1).mapPartitions { _ => + Iterator.single(new File(SparkFiles.get("v1.txt")).canRead) + } + + assert(checkAddFileRDD.first()) + } + case class LogEntry(filename: String, message: String) case class LogFile(name: String) @@ -816,7 +829,7 @@ class HiveQuerySuite extends HiveComparisonTest { createQueryTest("select from thrift based table", "SELECT * from src_thrift") - + // Put tests that depend on specific Hive settings before these last two test, // since they modify /clear stuff. } From a68ecf32812b5ef150e4b455b0f13b3fe120dc80 Mon Sep 17 00:00:00 2001 From: Mark Mims Date: Fri, 31 Oct 2014 11:41:03 -0700 Subject: [PATCH 1163/1492] [SPARK-4141] Hide Accumulators column on stage page when no accumulators exist WebUI Author: Mark Mims This patch had conflicts when merged, resolved by Committer: Josh Rosen Closes #3031 from mmm/remove-accumulators-col and squashes the following commits: 6141cb3 [Mark Mims] reformat to satisfy scalastyle linelength. build failed from jenkins https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/22604/ 390893b [Mark Mims] cleanup c28c449 [Mark Mims] looking much better now... minimal explicit formatting. Now, see if any sort keys make sense fb72156 [Mark Mims] mimic hasInput. The basics work here, but wanna clean this up with maybeAccumulators for column content --- .../org/apache/spark/ui/jobs/StagePage.scala | 21 +++++++++++++------ 1 file changed, 15 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 961224a300a70..bf45272aefde4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -53,6 +53,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val numCompleted = tasks.count(_.taskInfo.finished) val accumulables = listener.stageIdToData((stageId, stageAttemptId)).accumulables + val hasAccumulators = accumulables.size > 0 val hasInput = stageData.inputBytes > 0 val hasShuffleRead = stageData.shuffleReadBytes > 0 val hasShuffleWrite = stageData.shuffleWriteBytes > 0 @@ -144,11 +145,12 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val taskHeadersAndCssClasses: Seq[(String, String)] = Seq( ("Index", ""), ("ID", ""), ("Attempt", ""), ("Status", ""), ("Locality Level", ""), - ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Accumulators", ""), + ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), ("GC Time", TaskDetailsClassNames.GC_TIME), ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ + {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ {if (hasInput) Seq(("Input", "")) else Nil} ++ {if (hasShuffleRead) Seq(("Shuffle Read", "")) else Nil} ++ {if (hasShuffleWrite) Seq(("Write Time", ""), ("Shuffle Write", "")) else Nil} ++ @@ -159,7 +161,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val unzipped = taskHeadersAndCssClasses.unzip val taskTable = UIUtils.listingTable( - unzipped._1, taskRow(hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks, + unzipped._1, + taskRow(hasAccumulators, hasInput, hasShuffleRead, hasShuffleWrite, hasBytesSpilled), + tasks, headerClasses = unzipped._2) // Excludes tasks which failed and have incomplete metrics val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) @@ -298,6 +302,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { } def taskRow( + hasAccumulators: Boolean, hasInput: Boolean, hasShuffleRead: Boolean, hasShuffleWrite: Boolean, @@ -312,6 +317,9 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = info.gettingResultTime + val maybeAccumulators = info.accumulables + val accumulatorsReadable = maybeAccumulators.map{acc => s"${acc.name}: ${acc.update.get}"} + val maybeInput = metrics.flatMap(_.inputMetrics) val inputSortable = maybeInput.map(_.bytesRead.toString).getOrElse("") val inputReadable = maybeInput @@ -355,10 +363,6 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { - + {if (hasAccumulators) { + + }} {if (hasInput) { + + + + +
    {acc.name}{acc.value}
    {UIUtils.formatDuration(millis.toLong)}Result serialization time{UIUtils.formatDuration(ms.toLong)}Duration{UIUtils.formatDuration(ms.toLong)}Duration + GC Time + + + + Result Serialization Time + + Time spent fetching task results{UIUtils.formatDuration(millis.toLong)} + + Getting Result Time + + Scheduler delay{UIUtils.formatDuration(millis.toLong)}{Utils.bytesToString(d.toLong)}InputInputShuffle Read (Remote)Shuffle WriteShuffle WriteShuffle spill (memory)Shuffle spill (disk)
    {formatDuration} - {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} - {Unparsed( - info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
    ") - )} + info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
    "))} +
    + {UIUtils.formatDuration(schedulerDelay.toLong)} {inputReadable} @@ -333,4 +406,15 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") {
    {formatDuration} - {Unparsed( - info.accumulables.map{acc => s"${acc.name}: ${acc.update.get}"}.mkString("
    "))} -
    {UIUtils.formatDuration(schedulerDelay.toLong)} @@ -374,6 +378,11 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { class={TaskDetailsClassNames.GETTING_RESULT_TIME}> {UIUtils.formatDuration(gettingResultTime)} + {Unparsed(accumulatorsReadable.mkString("
    "))} +
    {inputReadable} From f1e7361f66fc24ae79bf48cc25f7fe395b016285 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 31 Oct 2014 12:07:48 -0700 Subject: [PATCH 1164/1492] [SPARK-4150][PySpark] return self in rdd.setName Then we can do `rdd.setName('abc').cache().count()`. Author: Xiangrui Meng Closes #3011 from mengxr/rdd-setname and squashes the following commits: 10d0d60 [Xiangrui Meng] update test 4ac3bbd [Xiangrui Meng] return self in rdd.setName --- python/pyspark/rdd.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 15be4bfec92f9..550c9dd80522f 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1867,11 +1867,11 @@ def setName(self, name): Assign a name to this RDD. >>> rdd1 = sc.parallelize([1,2]) - >>> rdd1.setName('RDD1') - >>> rdd1.name() + >>> rdd1.setName('RDD1').name() 'RDD1' """ self._jrdd.setName(name) + return self def toDebugString(self): """ From 55ab77707891408e635843cf80079747625bd28e Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Fri, 31 Oct 2014 12:39:52 -0700 Subject: [PATCH 1165/1492] [SPARK-3870] EOL character enforcement We have shell scripts and Windows batch files, so we should enforce proper EOL character. Author: Kousuke Saruta Closes #2726 from sarutak/eol-enforcement and squashes the following commits: 9748c3f [Kousuke Saruta] Fixed make.bat 252de89 [Kousuke Saruta] Removed extra characters from make.bat 5b81c00 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into eol-enforcement 8633ed2 [Kousuke Saruta] merge branch 'master' of git://git.apache.org/spark into eol-enforcement 5d630d8 [Kousuke Saruta] Merged ba10797 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into eol-enforcement 7407515 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into eol-enforcement 772fd4e [Kousuke Saruta] Normized EOL character in make.bat and compute-classpath.cmd ac7f873 [Kousuke Saruta] Added an entry for .gitattributes to .rat-excludes 1570e77 [Kousuke Saruta] Added .gitattributes --- .gitattributes | 2 + .rat-excludes | 1 + bin/compute-classpath.cmd | 234 +++++++++--------- python/docs/make.bat | 12 +- python/docs/make2.bat | 486 +++++++++++++++++++------------------- 5 files changed, 369 insertions(+), 366 deletions(-) create mode 100644 .gitattributes diff --git a/.gitattributes b/.gitattributes new file mode 100644 index 0000000000000..2b65f6fe3cc80 --- /dev/null +++ b/.gitattributes @@ -0,0 +1,2 @@ +*.bat text eol=crlf +*.cmd text eol=crlf diff --git a/.rat-excludes b/.rat-excludes index ae9745673c87d..20e3372464386 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -1,5 +1,6 @@ target .gitignore +.gitattributes .project .classpath .mima-excludes diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 3cd0579aea8d3..a4c099fb45b14 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,117 +1,117 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting -rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we -rem need to set it here because we use !datanucleus_jars! below. -if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion -setlocal enabledelayedexpansion -:skip_delayed_expansion - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" - -rem Build up classpath -set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% - -if not "x%SPARK_CONF_DIR%"=="x" ( - set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% -) else ( - set CLASSPATH=%CLASSPATH%;%FWDIR%conf -) - -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) - -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -rem When Hive support is needed, Datanucleus jars must be included on the classpath. -rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. -rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is -rem built with Hive, so look for them there. -if exist "%FWDIR%RELEASE" ( - set datanucleus_dir=%FWDIR%lib -) else ( - set datanucleus_dir=%FWDIR%lib_managed\jars -) -set "datanucleus_jars=" -for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( - set datanucleus_jars=!datanucleus_jars!;%%d -) -set CLASSPATH=%CLASSPATH%;%datanucleus_jars% - -set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes -set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes - -set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes -set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH - rem so that local compilation takes precedence over assembled jar - set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +rem If we're called from spark-class2.cmd, it already set enabledelayedexpansion and setting +rem it here would stop us from affecting its copy of the CLASSPATH variable; otherwise we +rem need to set it here because we use !datanucleus_jars! below. +if "%DONT_PRINT_CLASSPATH%"=="1" goto skip_delayed_expansion +setlocal enabledelayedexpansion +:skip_delayed_expansion + +set SCALA_VERSION=2.10 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0..\ + +rem Load environment variables from conf\spark-env.cmd, if it exists +if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" + +rem Build up classpath +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH% + +if not "x%SPARK_CONF_DIR%"=="x" ( + set CLASSPATH=%CLASSPATH%;%SPARK_CONF_DIR% +) else ( + set CLASSPATH=%CLASSPATH%;%FWDIR%conf +) + +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) else ( + for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) + +set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% + +rem When Hive support is needed, Datanucleus jars must be included on the classpath. +rem Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +rem Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +rem built with Hive, so look for them there. +if exist "%FWDIR%RELEASE" ( + set datanucleus_dir=%FWDIR%lib +) else ( + set datanucleus_dir=%FWDIR%lib_managed\jars +) +set "datanucleus_jars=" +for %%d in ("%datanucleus_dir%\datanucleus-*.jar") do ( + set datanucleus_jars=!datanucleus_jars!;%%d +) +set CLASSPATH=%CLASSPATH%;%datanucleus_jars% + +set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes + +set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes + +if "x%SPARK_TESTING%"=="x1" ( + rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH + rem so that local compilation takes precedence over assembled jar + set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% +) + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/python/docs/make.bat b/python/docs/make.bat index c011e82b4a35a..cc29acdc19686 100644 --- a/python/docs/make.bat +++ b/python/docs/make.bat @@ -1,6 +1,6 @@ -@ECHO OFF - -rem This is the entry point for running Sphinx documentation. To avoid polluting the -rem environment, it just launches a new cmd to do the real work. - -cmd /V /E /C %~dp0make2.bat %* +@ECHO OFF + +rem This is the entry point for running Sphinx documentation. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0make2.bat %* diff --git a/python/docs/make2.bat b/python/docs/make2.bat index 7bcaeafad13d7..05d22eb5cdd23 100644 --- a/python/docs/make2.bat +++ b/python/docs/make2.bat @@ -1,243 +1,243 @@ -@ECHO OFF - -REM Command file for Sphinx documentation - - -if "%SPHINXBUILD%" == "" ( - set SPHINXBUILD=sphinx-build -) -set BUILDDIR=_build -set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% . -set I18NSPHINXOPTS=%SPHINXOPTS% . -if NOT "%PAPER%" == "" ( - set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% - set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% -) - -if "%1" == "" goto help - -if "%1" == "help" ( - :help - echo.Please use `make ^` where ^ is one of - echo. html to make standalone HTML files - echo. dirhtml to make HTML files named index.html in directories - echo. singlehtml to make a single large HTML file - echo. pickle to make pickle files - echo. json to make JSON files - echo. htmlhelp to make HTML files and a HTML help project - echo. qthelp to make HTML files and a qthelp project - echo. devhelp to make HTML files and a Devhelp project - echo. epub to make an epub - echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter - echo. text to make text files - echo. man to make manual pages - echo. texinfo to make Texinfo files - echo. gettext to make PO message catalogs - echo. changes to make an overview over all changed/added/deprecated items - echo. xml to make Docutils-native XML files - echo. pseudoxml to make pseudoxml-XML files for display purposes - echo. linkcheck to check all external links for integrity - echo. doctest to run all doctests embedded in the documentation if enabled - goto end -) - -if "%1" == "clean" ( - for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i - del /q /s %BUILDDIR%\* - goto end -) - - -%SPHINXBUILD% 2> nul -if errorlevel 9009 ( - echo. - echo.The 'sphinx-build' command was not found. Make sure you have Sphinx - echo.installed, then set the SPHINXBUILD environment variable to point - echo.to the full path of the 'sphinx-build' executable. Alternatively you - echo.may add the Sphinx directory to PATH. - echo. - echo.If you don't have Sphinx installed, grab it from - echo.http://sphinx-doc.org/ - exit /b 1 -) - -if "%1" == "html" ( - %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/html. - goto end -) - -if "%1" == "dirhtml" ( - %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. - goto end -) - -if "%1" == "singlehtml" ( - %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. - goto end -) - -if "%1" == "pickle" ( - %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the pickle files. - goto end -) - -if "%1" == "json" ( - %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can process the JSON files. - goto end -) - -if "%1" == "htmlhelp" ( - %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run HTML Help Workshop with the ^ -.hhp project file in %BUILDDIR%/htmlhelp. - goto end -) - -if "%1" == "qthelp" ( - %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; now you can run "qcollectiongenerator" with the ^ -.qhcp project file in %BUILDDIR%/qthelp, like this: - echo.^> qcollectiongenerator %BUILDDIR%\qthelp\pyspark.qhcp - echo.To view the help file: - echo.^> assistant -collectionFile %BUILDDIR%\qthelp\pyspark.ghc - goto end -) - -if "%1" == "devhelp" ( - %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. - goto end -) - -if "%1" == "epub" ( - %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The epub file is in %BUILDDIR%/epub. - goto end -) - -if "%1" == "latex" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - if errorlevel 1 exit /b 1 - echo. - echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdf" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf - cd %BUILDDIR%/.. - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "latexpdfja" ( - %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex - cd %BUILDDIR%/latex - make all-pdf-ja - cd %BUILDDIR%/.. - echo. - echo.Build finished; the PDF files are in %BUILDDIR%/latex. - goto end -) - -if "%1" == "text" ( - %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The text files are in %BUILDDIR%/text. - goto end -) - -if "%1" == "man" ( - %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The manual pages are in %BUILDDIR%/man. - goto end -) - -if "%1" == "texinfo" ( - %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. - goto end -) - -if "%1" == "gettext" ( - %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The message catalogs are in %BUILDDIR%/locale. - goto end -) - -if "%1" == "changes" ( - %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes - if errorlevel 1 exit /b 1 - echo. - echo.The overview file is in %BUILDDIR%/changes. - goto end -) - -if "%1" == "linkcheck" ( - %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck - if errorlevel 1 exit /b 1 - echo. - echo.Link check complete; look for any errors in the above output ^ -or in %BUILDDIR%/linkcheck/output.txt. - goto end -) - -if "%1" == "doctest" ( - %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest - if errorlevel 1 exit /b 1 - echo. - echo.Testing of doctests in the sources finished, look at the ^ -results in %BUILDDIR%/doctest/output.txt. - goto end -) - -if "%1" == "xml" ( - %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The XML files are in %BUILDDIR%/xml. - goto end -) - -if "%1" == "pseudoxml" ( - %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml - if errorlevel 1 exit /b 1 - echo. - echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. - goto end -) - -:end +@ECHO OFF + +REM Command file for Sphinx documentation + + +if "%SPHINXBUILD%" == "" ( + set SPHINXBUILD=sphinx-build +) +set BUILDDIR=_build +set ALLSPHINXOPTS=-d %BUILDDIR%/doctrees %SPHINXOPTS% . +set I18NSPHINXOPTS=%SPHINXOPTS% . +if NOT "%PAPER%" == "" ( + set ALLSPHINXOPTS=-D latex_paper_size=%PAPER% %ALLSPHINXOPTS% + set I18NSPHINXOPTS=-D latex_paper_size=%PAPER% %I18NSPHINXOPTS% +) + +if "%1" == "" goto help + +if "%1" == "help" ( + :help + echo.Please use `make ^` where ^ is one of + echo. html to make standalone HTML files + echo. dirhtml to make HTML files named index.html in directories + echo. singlehtml to make a single large HTML file + echo. pickle to make pickle files + echo. json to make JSON files + echo. htmlhelp to make HTML files and a HTML help project + echo. qthelp to make HTML files and a qthelp project + echo. devhelp to make HTML files and a Devhelp project + echo. epub to make an epub + echo. latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter + echo. text to make text files + echo. man to make manual pages + echo. texinfo to make Texinfo files + echo. gettext to make PO message catalogs + echo. changes to make an overview over all changed/added/deprecated items + echo. xml to make Docutils-native XML files + echo. pseudoxml to make pseudoxml-XML files for display purposes + echo. linkcheck to check all external links for integrity + echo. doctest to run all doctests embedded in the documentation if enabled + goto end +) + +if "%1" == "clean" ( + for /d %%i in (%BUILDDIR%\*) do rmdir /q /s %%i + del /q /s %BUILDDIR%\* + goto end +) + + +%SPHINXBUILD% 2> nul +if errorlevel 9009 ( + echo. + echo.The 'sphinx-build' command was not found. Make sure you have Sphinx + echo.installed, then set the SPHINXBUILD environment variable to point + echo.to the full path of the 'sphinx-build' executable. Alternatively you + echo.may add the Sphinx directory to PATH. + echo. + echo.If you don't have Sphinx installed, grab it from + echo.http://sphinx-doc.org/ + exit /b 1 +) + +if "%1" == "html" ( + %SPHINXBUILD% -b html %ALLSPHINXOPTS% %BUILDDIR%/html + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/html. + goto end +) + +if "%1" == "dirhtml" ( + %SPHINXBUILD% -b dirhtml %ALLSPHINXOPTS% %BUILDDIR%/dirhtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/dirhtml. + goto end +) + +if "%1" == "singlehtml" ( + %SPHINXBUILD% -b singlehtml %ALLSPHINXOPTS% %BUILDDIR%/singlehtml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The HTML pages are in %BUILDDIR%/singlehtml. + goto end +) + +if "%1" == "pickle" ( + %SPHINXBUILD% -b pickle %ALLSPHINXOPTS% %BUILDDIR%/pickle + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the pickle files. + goto end +) + +if "%1" == "json" ( + %SPHINXBUILD% -b json %ALLSPHINXOPTS% %BUILDDIR%/json + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can process the JSON files. + goto end +) + +if "%1" == "htmlhelp" ( + %SPHINXBUILD% -b htmlhelp %ALLSPHINXOPTS% %BUILDDIR%/htmlhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run HTML Help Workshop with the ^ +.hhp project file in %BUILDDIR%/htmlhelp. + goto end +) + +if "%1" == "qthelp" ( + %SPHINXBUILD% -b qthelp %ALLSPHINXOPTS% %BUILDDIR%/qthelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; now you can run "qcollectiongenerator" with the ^ +.qhcp project file in %BUILDDIR%/qthelp, like this: + echo.^> qcollectiongenerator %BUILDDIR%\qthelp\pyspark.qhcp + echo.To view the help file: + echo.^> assistant -collectionFile %BUILDDIR%\qthelp\pyspark.ghc + goto end +) + +if "%1" == "devhelp" ( + %SPHINXBUILD% -b devhelp %ALLSPHINXOPTS% %BUILDDIR%/devhelp + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. + goto end +) + +if "%1" == "epub" ( + %SPHINXBUILD% -b epub %ALLSPHINXOPTS% %BUILDDIR%/epub + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The epub file is in %BUILDDIR%/epub. + goto end +) + +if "%1" == "latex" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + if errorlevel 1 exit /b 1 + echo. + echo.Build finished; the LaTeX files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdf" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "latexpdfja" ( + %SPHINXBUILD% -b latex %ALLSPHINXOPTS% %BUILDDIR%/latex + cd %BUILDDIR%/latex + make all-pdf-ja + cd %BUILDDIR%/.. + echo. + echo.Build finished; the PDF files are in %BUILDDIR%/latex. + goto end +) + +if "%1" == "text" ( + %SPHINXBUILD% -b text %ALLSPHINXOPTS% %BUILDDIR%/text + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The text files are in %BUILDDIR%/text. + goto end +) + +if "%1" == "man" ( + %SPHINXBUILD% -b man %ALLSPHINXOPTS% %BUILDDIR%/man + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The manual pages are in %BUILDDIR%/man. + goto end +) + +if "%1" == "texinfo" ( + %SPHINXBUILD% -b texinfo %ALLSPHINXOPTS% %BUILDDIR%/texinfo + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The Texinfo files are in %BUILDDIR%/texinfo. + goto end +) + +if "%1" == "gettext" ( + %SPHINXBUILD% -b gettext %I18NSPHINXOPTS% %BUILDDIR%/locale + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The message catalogs are in %BUILDDIR%/locale. + goto end +) + +if "%1" == "changes" ( + %SPHINXBUILD% -b changes %ALLSPHINXOPTS% %BUILDDIR%/changes + if errorlevel 1 exit /b 1 + echo. + echo.The overview file is in %BUILDDIR%/changes. + goto end +) + +if "%1" == "linkcheck" ( + %SPHINXBUILD% -b linkcheck %ALLSPHINXOPTS% %BUILDDIR%/linkcheck + if errorlevel 1 exit /b 1 + echo. + echo.Link check complete; look for any errors in the above output ^ +or in %BUILDDIR%/linkcheck/output.txt. + goto end +) + +if "%1" == "doctest" ( + %SPHINXBUILD% -b doctest %ALLSPHINXOPTS% %BUILDDIR%/doctest + if errorlevel 1 exit /b 1 + echo. + echo.Testing of doctests in the sources finished, look at the ^ +results in %BUILDDIR%/doctest/output.txt. + goto end +) + +if "%1" == "xml" ( + %SPHINXBUILD% -b xml %ALLSPHINXOPTS% %BUILDDIR%/xml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The XML files are in %BUILDDIR%/xml. + goto end +) + +if "%1" == "pseudoxml" ( + %SPHINXBUILD% -b pseudoxml %ALLSPHINXOPTS% %BUILDDIR%/pseudoxml + if errorlevel 1 exit /b 1 + echo. + echo.Build finished. The pseudo-XML files are in %BUILDDIR%/pseudoxml. + goto end +) + +:end From 087e31a7a0550f520c2e21b0ceaf36b969e050b4 Mon Sep 17 00:00:00 2001 From: andrewor14 Date: Fri, 31 Oct 2014 14:36:55 -0700 Subject: [PATCH 1166/1492] [HOT FIX] Yarn stable tests don't compile This is caused by this commit: acd4ac7c9a503445e27739708cf36e19119b8ddc Author: andrewor14 Author: Andrew Or Closes #3041 from andrewor14/yarn-hot-fix and squashes the following commits: e5deba1 [andrewor14] Add new line at the end (minor) aa998e8 [Andrew Or] Compilation hot fix --- .../spark/deploy/yarn/YarnAllocator.scala | 30 +++++++++++-------- .../deploy/yarn/YarnAllocatorSuite.scala | 4 +-- 2 files changed, 19 insertions(+), 15 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index e6196194acbb4..b32e15738f28b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -64,6 +64,8 @@ private[yarn] abstract class YarnAllocator( securityMgr: SecurityManager) extends Logging { + import YarnAllocator._ + // These three are locked on allocatedHostToContainersMap. Complementary data structures // allocatedHostToContainersMap : containers which are running : host, Set // allocatedContainerToHostMap: container to host mapping. @@ -439,19 +441,6 @@ private[yarn] abstract class YarnAllocator( } } - private val MEM_REGEX = "[0-9.]+ [KMG]B" - private val PMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") - private val VMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") - - def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { - val matcher = pattern.matcher(diagnostics) - val diag = if (matcher.find()) " " + matcher.group() + "." else "" - ("Container killed by YARN for exceeding memory limits." + diag - + " Consider boosting spark.yarn.executor.memoryOverhead.") - } - protected def allocatedContainersOnHost(host: String): Int = { var retval = 0 allocatedHostToContainersMap.synchronized { @@ -532,3 +521,18 @@ private[yarn] abstract class YarnAllocator( } } + +private object YarnAllocator { + val MEM_REGEX = "[0-9.]+ [KMG]B" + val PMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") + val VMEM_EXCEEDED_PATTERN = + Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") + + def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { + val matcher = pattern.matcher(diagnostics) + val diag = if (matcher.find()) " " + matcher.group() + "." else "" + ("Container killed by YARN for exceeding memory limits." + diag + + " Consider boosting spark.yarn.executor.memoryOverhead.") + } +} diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 9fff63fb25156..8d184a09d64cc 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.yarn -import org.apache.spark.deploy.yarn.MemLimitLogger._ +import org.apache.spark.deploy.yarn.YarnAllocator._ import org.scalatest.FunSuite class YarnAllocatorSuite extends FunSuite { @@ -31,4 +31,4 @@ class YarnAllocatorSuite extends FunSuite { assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used.")) assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used.")) } -} \ No newline at end of file +} From 23f73f525ce3d2b4a614e60f4f9170c860ab93da Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 31 Oct 2014 17:22:52 -0700 Subject: [PATCH 1167/1492] SPARK-4175. Exception on stage page Author: Sandy Ryza Closes #3043 from sryza/sandy-spark-4175 and squashes the following commits: e327340 [Sandy Ryza] SPARK-4175. Exception on stage page --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index bf45272aefde4..7cc03b7d333df 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -312,7 +312,7 @@ private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { else metrics.map(_.executorRunTime).getOrElse(1L) val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") - val schedulerDelay = getSchedulerDelay(info, metrics.get) + val schedulerDelay = metrics.map(getSchedulerDelay(info, _)).getOrElse(0L) val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) val gettingResultTime = info.gettingResultTime From 62d01d255c001a6d397cc166a10aba3894f43459 Mon Sep 17 00:00:00 2001 From: Alexander Ulanov Date: Fri, 31 Oct 2014 18:31:03 -0700 Subject: [PATCH 1168/1492] [MLLIB] SPARK-2329 Add multi-label evaluation metrics Implementation of various multi-label classification measures, including: Hamming-loss, strict and default Accuracy, macro-averaged Precision, Recall and F1-measure based on documents and labels, micro-averaged measures: https://issues.apache.org/jira/browse/SPARK-2329 Multi-class measures are currently in the following pull request: https://github.com/apache/spark/pull/1155 Author: Alexander Ulanov Author: avulanov Closes #1270 from avulanov/multilabelmetrics and squashes the following commits: fc8175e [Alexander Ulanov] Merge with previous updates 43a613e [Alexander Ulanov] Addressing reviewers comments: change Set to Array 517a594 [avulanov] Addressing reviewers comments: Scala style cf4222bc [avulanov] Addressing reviewers comments: renaming. Added label method that returns the list of labels 1843f73 [Alexander Ulanov] Scala style fix 79e8476 [Alexander Ulanov] Replacing fold(_ + _) with sum as suggested by srowen ca46765 [Alexander Ulanov] Cosmetic changes: Apache header and parameter explanation 40593f5 [Alexander Ulanov] Multi-label metrics: Hamming-loss, strict and normal accuracy, fix to macro measures, bunch of tests ad62df0 [Alexander Ulanov] Comments and scala style check 154164b [Alexander Ulanov] Multilabel evaluation metics and tests: macro precision and recall averaged by docs, micro and per-class precision and recall averaged by class --- .../mllib/evaluation/MultilabelMetrics.scala | 157 ++++++++++++++++++ .../evaluation/MultilabelMetricsSuite.scala | 103 ++++++++++++ 2 files changed, 260 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala new file mode 100644 index 0000000000000..ea10bde5fa252 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala @@ -0,0 +1,157 @@ +/* + * 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.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ + +/** + * Evaluator for multilabel classification. + * @param predictionAndLabels an RDD of (predictions, labels) pairs, + * both are non-null Arrays, each with unique elements. + */ +class MultilabelMetrics(predictionAndLabels: RDD[(Array[Double], Array[Double])]) { + + private lazy val numDocs: Long = predictionAndLabels.count() + + private lazy val numLabels: Long = predictionAndLabels.flatMap { case (_, labels) => + labels}.distinct().count() + + /** + * Returns subset accuracy + * (for equal sets of labels) + */ + lazy val subsetAccuracy: Double = predictionAndLabels.filter { case (predictions, labels) => + predictions.deep == labels.deep + }.count().toDouble / numDocs + + /** + * Returns accuracy + */ + lazy val accuracy: Double = predictionAndLabels.map { case (predictions, labels) => + labels.intersect(predictions).size.toDouble / + (labels.size + predictions.size - labels.intersect(predictions).size)}.sum / numDocs + + + /** + * Returns Hamming-loss + */ + lazy val hammingLoss: Double = predictionAndLabels.map { case (predictions, labels) => + labels.size + predictions.size - 2 * labels.intersect(predictions).size + }.sum / (numDocs * numLabels) + + /** + * Returns document-based precision averaged by the number of documents + */ + lazy val precision: Double = predictionAndLabels.map { case (predictions, labels) => + if (predictions.size > 0) { + predictions.intersect(labels).size.toDouble / predictions.size + } else { + 0 + } + }.sum / numDocs + + /** + * Returns document-based recall averaged by the number of documents + */ + lazy val recall: Double = predictionAndLabels.map { case (predictions, labels) => + labels.intersect(predictions).size.toDouble / labels.size + }.sum / numDocs + + /** + * Returns document-based f1-measure averaged by the number of documents + */ + lazy val f1Measure: Double = predictionAndLabels.map { case (predictions, labels) => + 2.0 * predictions.intersect(labels).size / (predictions.size + labels.size) + }.sum / numDocs + + private lazy val tpPerClass = predictionAndLabels.flatMap { case (predictions, labels) => + predictions.intersect(labels) + }.countByValue() + + private lazy val fpPerClass = predictionAndLabels.flatMap { case (predictions, labels) => + predictions.diff(labels) + }.countByValue() + + private lazy val fnPerClass = predictionAndLabels.flatMap { case(predictions, labels) => + labels.diff(predictions) + }.countByValue() + + /** + * Returns precision for a given label (category) + * @param label the label. + */ + def precision(label: Double) = { + val tp = tpPerClass(label) + val fp = fpPerClass.getOrElse(label, 0L) + 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) = { + val tp = tpPerClass(label) + val fn = fnPerClass.getOrElse(label, 0L) + if (tp + fn == 0) 0 else tp.toDouble / (tp + fn) + } + + /** + * Returns f1-measure for a given label (category) + * @param label the label. + */ + def f1Measure(label: Double) = { + val p = precision(label) + val r = recall(label) + if((p + r) == 0) 0 else 2 * p * r / (p + r) + } + + private lazy val sumTp = tpPerClass.foldLeft(0L) { case (sum, (_, tp)) => sum + tp } + private lazy val sumFpClass = fpPerClass.foldLeft(0L) { case (sum, (_, fp)) => sum + fp } + private lazy val sumFnClass = fnPerClass.foldLeft(0L) { case (sum, (_, fn)) => sum + fn } + + /** + * Returns micro-averaged label-based precision + * (equals to micro-averaged document-based precision) + */ + lazy val microPrecision = { + val sumFp = fpPerClass.foldLeft(0L){ case(cum, (_, fp)) => cum + fp} + sumTp.toDouble / (sumTp + sumFp) + } + + /** + * Returns micro-averaged label-based recall + * (equals to micro-averaged document-based recall) + */ + lazy val microRecall = { + val sumFn = fnPerClass.foldLeft(0.0){ case(cum, (_, fn)) => cum + fn} + sumTp.toDouble / (sumTp + sumFn) + } + + /** + * Returns micro-averaged label-based f1-measure + * (equals to micro-averaged document-based f1-measure) + */ + lazy val microF1Measure = 2.0 * sumTp / (2 * sumTp + sumFnClass + sumFpClass) + + /** + * Returns the sequence of labels in ascending order + */ + lazy val labels: Array[Double] = tpPerClass.keys.toArray.sorted +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala new file mode 100644 index 0000000000000..342baa0274e9c --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD + +class MultilabelMetricsSuite extends FunSuite with LocalSparkContext { + test("Multilabel evaluation metrics") { + /* + * Documents true labels (5x class0, 3x class1, 4x class2): + * doc 0 - predict 0, 1 - class 0, 2 + * doc 1 - predict 0, 2 - class 0, 1 + * doc 2 - predict none - class 0 + * doc 3 - predict 2 - class 2 + * doc 4 - predict 2, 0 - class 2, 0 + * doc 5 - predict 0, 1, 2 - class 0, 1 + * doc 6 - predict 1 - class 1, 2 + * + * predicted classes + * class 0 - doc 0, 1, 4, 5 (total 4) + * class 1 - doc 0, 5, 6 (total 3) + * class 2 - doc 1, 3, 4, 5 (total 4) + * + * true classes + * class 0 - doc 0, 1, 2, 4, 5 (total 5) + * class 1 - doc 1, 5, 6 (total 3) + * class 2 - doc 0, 3, 4, 6 (total 4) + * + */ + val scoreAndLabels: RDD[(Array[Double], Array[Double])] = sc.parallelize( + Seq((Array(0.0, 1.0), Array(0.0, 2.0)), + (Array(0.0, 2.0), Array(0.0, 1.0)), + (Array(), Array(0.0)), + (Array(2.0), Array(2.0)), + (Array(2.0, 0.0), Array(2.0, 0.0)), + (Array(0.0, 1.0, 2.0), Array(0.0, 1.0)), + (Array(1.0), Array(1.0, 2.0))), 2) + val metrics = new MultilabelMetrics(scoreAndLabels) + val delta = 0.00001 + val precision0 = 4.0 / (4 + 0) + val precision1 = 2.0 / (2 + 1) + val precision2 = 2.0 / (2 + 2) + val recall0 = 4.0 / (4 + 1) + val recall1 = 2.0 / (2 + 1) + val recall2 = 2.0 / (2 + 2) + val f1measure0 = 2 * precision0 * recall0 / (precision0 + recall0) + val f1measure1 = 2 * precision1 * recall1 / (precision1 + recall1) + val f1measure2 = 2 * precision2 * recall2 / (precision2 + recall2) + val sumTp = 4 + 2 + 2 + assert(sumTp == (1 + 1 + 0 + 1 + 2 + 2 + 1)) + val microPrecisionClass = sumTp.toDouble / (4 + 0 + 2 + 1 + 2 + 2) + val microRecallClass = sumTp.toDouble / (4 + 1 + 2 + 1 + 2 + 2) + val microF1MeasureClass = 2.0 * sumTp.toDouble / + (2 * sumTp.toDouble + (1 + 1 + 2) + (0 + 1 + 2)) + val macroPrecisionDoc = 1.0 / 7 * + (1.0 / 2 + 1.0 / 2 + 0 + 1.0 / 1 + 2.0 / 2 + 2.0 / 3 + 1.0 / 1.0) + val macroRecallDoc = 1.0 / 7 * + (1.0 / 2 + 1.0 / 2 + 0 / 1 + 1.0 / 1 + 2.0 / 2 + 2.0 / 2 + 1.0 / 2) + val macroF1MeasureDoc = (1.0 / 7) * + 2 * ( 1.0 / (2 + 2) + 1.0 / (2 + 2) + 0 + 1.0 / (1 + 1) + + 2.0 / (2 + 2) + 2.0 / (3 + 2) + 1.0 / (1 + 2) ) + val hammingLoss = (1.0 / (7 * 3)) * (2 + 2 + 1 + 0 + 0 + 1 + 1) + val strictAccuracy = 2.0 / 7 + val accuracy = 1.0 / 7 * (1.0 / 3 + 1.0 /3 + 0 + 1.0 / 1 + 2.0 / 2 + 2.0 / 3 + 1.0 / 2) + 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.f1Measure(0.0) - f1measure0) < delta) + assert(math.abs(metrics.f1Measure(1.0) - f1measure1) < delta) + assert(math.abs(metrics.f1Measure(2.0) - f1measure2) < delta) + assert(math.abs(metrics.microPrecision - microPrecisionClass) < delta) + assert(math.abs(metrics.microRecall - microRecallClass) < delta) + assert(math.abs(metrics.microF1Measure - microF1MeasureClass) < delta) + assert(math.abs(metrics.precision - macroPrecisionDoc) < delta) + assert(math.abs(metrics.recall - macroRecallDoc) < delta) + assert(math.abs(metrics.f1Measure - macroF1MeasureDoc) < delta) + assert(math.abs(metrics.hammingLoss - hammingLoss) < delta) + assert(math.abs(metrics.subsetAccuracy - strictAccuracy) < delta) + assert(math.abs(metrics.accuracy - accuracy) < delta) + assert(metrics.labels.sameElements(Array(0.0, 1.0, 2.0))) + } +} From e07fb6a41ee949f8dba44d5a3b6c0615f27f0eaf Mon Sep 17 00:00:00 2001 From: Anant Date: Fri, 31 Oct 2014 18:33:19 -0700 Subject: [PATCH 1169/1492] [SPARK-3838][examples][mllib][python] Word2Vec example in python This pull request refers to issue: https://issues.apache.org/jira/browse/SPARK-3838 Python example for word2vec mengxr Author: Anant Closes #2952 from anantasty/SPARK-3838 and squashes the following commits: 87bd723 [Anant] remove stop line 4bd439e [Anant] Changes as per code review. Fized error in word2vec python example, simplified example in docs. 3d3c9ee [Anant] Added empty line after python imports 0c90c31 [Anant] Fixed erroneous code. I was still treating each line to be a single word instead of 16 words ee4f5f6 [Anant] Fixes from code review comments c637bcf [Anant] Added word2vec python example to docs 269f31f [Anant] added example in docs c015b14 [Anant] Added python example for word2vec --- docs/mllib-feature-extraction.md | 17 ++++++++ examples/src/main/python/mllib/word2vec.py | 50 ++++++++++++++++++++++ 2 files changed, 67 insertions(+) create mode 100644 examples/src/main/python/mllib/word2vec.py diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index 886d71df474bc..197bc77d506c6 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -203,6 +203,23 @@ for((synonym, cosineSimilarity) <- synonyms) { } {% endhighlight %} +
    +{% highlight python %} +from pyspark import SparkContext +from pyspark.mllib.feature import Word2Vec + +sc = SparkContext(appName='Word2Vec') +inp = sc.textFile("text8_lines").map(lambda row: row.split(" ")) + +word2vec = Word2Vec() +model = word2vec.fit(inp) + +synonyms = model.findSynonyms('china', 40) + +for word, cosine_distance in synonyms: + print "{}: {}".format(word, cosine_distance) +{% endhighlight %} +
    ## StandardScaler diff --git a/examples/src/main/python/mllib/word2vec.py b/examples/src/main/python/mllib/word2vec.py new file mode 100644 index 0000000000000..99fef4276a369 --- /dev/null +++ b/examples/src/main/python/mllib/word2vec.py @@ -0,0 +1,50 @@ +# +# 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. +# + +# This example uses text8 file from http://mattmahoney.net/dc/text8.zip +# The file was downloadded, unziped and split into multiple lines using +# +# wget http://mattmahoney.net/dc/text8.zip +# unzip text8.zip +# grep -o -E '\w+(\W+\w+){0,15}' text8 > text8_lines +# This was done so that the example can be run in local mode + + +import sys + +from pyspark import SparkContext +from pyspark.mllib.feature import Word2Vec + +USAGE = ("bin/spark-submit --driver-memory 4g " + "examples/src/main/python/mllib/word2vec.py text8_lines") + +if __name__ == "__main__": + if len(sys.argv) < 2: + print USAGE + sys.exit("Argument for file not provided") + file_path = sys.argv[1] + sc = SparkContext(appName='Word2Vec') + inp = sc.textFile(file_path).map(lambda row: row.split(" ")) + + word2vec = Word2Vec() + model = word2vec.fit(inp) + + synonyms = model.findSynonyms('china', 40) + + for word, cosine_distance in synonyms: + print "{}: {}".format(word, cosine_distance) + sc.stop() From 8602195510f5821b37746bb7fa24902f43a1bd93 Mon Sep 17 00:00:00 2001 From: Manish Amde Date: Fri, 31 Oct 2014 18:57:55 -0700 Subject: [PATCH 1170/1492] [MLLIB] SPARK-1547: Add Gradient Boosting to MLlib MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Given the popular demand for gradient boosting and AdaBoost in MLlib, I am creating a WIP branch for early feedback on gradient boosting with AdaBoost to follow soon after this PR is accepted. This is based on work done along with hirakendu that was pending due to decision tree optimizations and random forests work. Ideally, boosting algorithms should work with any base learners. This will soon be possible once the MLlib API is finalized -- we want to ensure we use a consistent interface for the underlying base learners. In the meantime, this PR uses decision trees as base learners for the gradient boosting algorithm. The current PR allows "pluggable" loss functions and provides least squares error and least absolute error by default. Here is the task list: - [x] Gradient boosting support - [x] Pluggable loss functions - [x] Stochastic gradient boosting support – Re-use the BaggedPoint approach used for RandomForest. - [x] Binary classification support - [x] Support configurable checkpointing – This approach will avoid long lineage chains. - [x] Create classification and regression APIs - [x] Weighted Ensemble Model -- created a WeightedEnsembleModel class that can be used by ensemble algorithms such as random forests and boosting. - [x] Unit Tests Future work: + Multi-class classification is currently not supported by this PR since it requires discussion on the best way to support "deviance" as a loss function. + BaggedRDD caching -- Avoid repeating feature to bin mapping for each tree estimator after standard API work is completed. cc: jkbradley hirakendu mengxr etrain atalwalkar chouqin Author: Manish Amde Author: manishamde Closes #2607 from manishamde/gbt and squashes the following commits: 991c7b5 [Manish Amde] public api ff2a796 [Manish Amde] addressing comments b4c1318 [Manish Amde] removing spaces 8476b6b [Manish Amde] fixing line length 0183cb9 [Manish Amde] fixed naming and formatting issues 1c40c33 [Manish Amde] add newline, removed spaces e33ab61 [Manish Amde] minor comment eadbf09 [Manish Amde] parameter renaming 035a2ed [Manish Amde] jkbradley formatting suggestions 9f7359d [Manish Amde] simplified gbt logic and added more tests 49ba107 [Manish Amde] merged from master eff21fe [Manish Amde] Added gradient boosting tests 3fd0528 [Manish Amde] moved helper methods to new class a32a5ab [Manish Amde] added test for subsampling without replacement 781542a [Manish Amde] added support for fractional subsampling with replacement 3a18cc1 [Manish Amde] cleaned up api for conversion to bagged point and moved tests to it's own test suite 0e81906 [Manish Amde] improving caching unpersisting logic d971f73 [Manish Amde] moved RF code to use WeightedEnsembleModel class fee06d3 [Manish Amde] added weighted ensemble model 1b01943 [Manish Amde] add weights for base learners 9bc6e74 [Manish Amde] adding random seed as parameter d2c8323 [Manish Amde] Merge branch 'master' into gbt 2ae97b7 [Manish Amde] added documentation for the loss classes 9366b8f [Manish Amde] minor: using numTrees instead of trees.size 3b43896 [Manish Amde] added learning rate for prediction 9b2e35e [Manish Amde] Merge branch 'master' into gbt 6a11c02 [manishamde] fixing formatting 823691b [Manish Amde] fixing RF test 1f47941 [Manish Amde] changing access modifier 5b67102 [Manish Amde] shortened parameter list 5ab3796 [Manish Amde] minor reformatting 9155a9d [Manish Amde] consolidated boosting configuration and added public API 631baea [Manish Amde] Merge branch 'master' into gbt 2cb1258 [Manish Amde] public API support 3b8ffc0 [Manish Amde] added documentation 8e10c63 [Manish Amde] modified unpersist strategy f62bc48 [Manish Amde] added unpersist bdca43a [Manish Amde] added timing parameters 2fbc9c7 [Manish Amde] fixing binomial classification prediction 6dd4dd8 [Manish Amde] added support for log loss 9af0231 [Manish Amde] classification attempt 62cc000 [Manish Amde] basic checkpointing 4784091 [Manish Amde] formatting 78ed452 [Manish Amde] added newline and fixed if statement 3973dd1 [Manish Amde] minor indicating subsample is double during comparison aa8fae7 [Manish Amde] minor refactoring 1a8031c [Manish Amde] sampling with replacement f1c9ef7 [Manish Amde] Merge branch 'master' into gbt cdceeef [Manish Amde] added documentation 6251fd5 [Manish Amde] modified method name 5538521 [Manish Amde] disable checkpointing for now 0ae1c0a [Manish Amde] basic gradient boosting code from earlier branches --- .../examples/mllib/DecisionTreeRunner.scala | 4 +- .../spark/mllib/tree/DecisionTree.scala | 2 +- .../spark/mllib/tree/GradientBoosting.scala | 314 ++++++++++++++++++ .../spark/mllib/tree/RandomForest.scala | 49 +-- .../tree/configuration/BoostingStrategy.scala | 109 ++++++ .../EnsembleCombiningStrategy.scala | 30 ++ .../mllib/tree/configuration/Strategy.scala | 23 +- .../spark/mllib/tree/impl/BaggedPoint.scala | 69 +++- .../spark/mllib/tree/loss/AbsoluteError.scala | 66 ++++ .../spark/mllib/tree/loss/LogLoss.scala | 63 ++++ .../apache/spark/mllib/tree/loss/Loss.scala | 52 +++ .../apache/spark/mllib/tree/loss/Losses.scala | 29 ++ .../spark/mllib/tree/loss/SquaredError.scala | 66 ++++ .../mllib/tree/model/RandomForestModel.scala | 115 ------- .../tree/model/WeightedEnsembleModel.scala | 158 +++++++++ .../spark/mllib/tree/DecisionTreeSuite.scala | 6 +- .../spark/mllib/tree/EnsembleTestHelper.scala | 94 ++++++ .../mllib/tree/GradientBoostingSuite.scala | 132 ++++++++ .../spark/mllib/tree/RandomForestSuite.scala | 117 +------ .../mllib/tree/impl/BaggedPointSuite.scala | 100 ++++++ 20 files changed, 1331 insertions(+), 267 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 0890e6263e165..f98730366bcb7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.{RandomForest, DecisionTree, impurity} import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.model.{RandomForestModel, DecisionTreeModel} +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -317,7 +317,7 @@ object DecisionTreeRunner { /** * Calculates the mean squared error for regression. */ - private def meanSquaredError(tree: RandomForestModel, data: RDD[LabeledPoint]): Double = { + private def meanSquaredError(tree: WeightedEnsembleModel, data: RDD[LabeledPoint]): Double = { data.map { y => val err = tree.predict(y.features) - y.label err * err diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 6737a2f4176c2..752ed59a030d3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -62,7 +62,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo // Note: random seed will not be used since numTrees = 1. val rf = new RandomForest(strategy, numTrees = 1, featureSubsetStrategy = "all", seed = 0) val rfModel = rf.train(input) - rfModel.trees(0) + rfModel.weakHypotheses(0) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala new file mode 100644 index 0000000000000..1a847201ce157 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoosting.scala @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.tree.configuration.{Strategy, BoostingStrategy} +import org.apache.spark.Logging +import org.apache.spark.mllib.tree.impl.TimeTracker +import org.apache.spark.mllib.tree.loss.Losses +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel} +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Sum + +/** + * :: Experimental :: + * A class that implements gradient boosting for regression and binary classification problems. + * @param boostingStrategy Parameters for the gradient boosting algorithm + */ +@Experimental +class GradientBoosting ( + private val boostingStrategy: BoostingStrategy) extends Serializable with Logging { + + /** + * Method to train a gradient boosting model + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @return WeightedEnsembleModel that can be used for prediction + */ + def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = { + val algo = boostingStrategy.algo + algo match { + case Regression => GradientBoosting.boost(input, boostingStrategy) + case Classification => + val remappedInput = input.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + GradientBoosting.boost(remappedInput, boostingStrategy) + case _ => + throw new IllegalArgumentException(s"$algo is not supported by the gradient boosting.") + } + } + +} + + +object GradientBoosting extends Logging { + + /** + * Method to train a gradient boosting model. + * + * Note: Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]] + * is recommended to clearly specify regression. + * Using [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]] + * is recommended to clearly specify regression. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. + * @param boostingStrategy Configuration options for the boosting algorithm. + * @return WeightedEnsembleModel that can be used for prediction + */ + def train( + input: RDD[LabeledPoint], + boostingStrategy: BoostingStrategy): WeightedEnsembleModel = { + new GradientBoosting(boostingStrategy).train(input) + } + + /** + * Method to train a gradient boosting classification model. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. + * @param boostingStrategy Configuration options for the boosting algorithm. + * @return WeightedEnsembleModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + boostingStrategy: BoostingStrategy): WeightedEnsembleModel = { + val algo = boostingStrategy.algo + require(algo == Classification, s"Only Classification algo supported. Provided algo is $algo.") + new GradientBoosting(boostingStrategy).train(input) + } + + /** + * Method to train a gradient boosting regression model. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. + * @param boostingStrategy Configuration options for the boosting algorithm. + * @return WeightedEnsembleModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + boostingStrategy: BoostingStrategy): WeightedEnsembleModel = { + val algo = boostingStrategy.algo + require(algo == Regression, s"Only Regression algo supported. Provided algo is $algo.") + new GradientBoosting(boostingStrategy).train(input) + } + + /** + * Method to train a gradient boosting binary classification model. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. + * @param numEstimators Number of estimators used in boosting stages. In other words, + * number of boosting iterations performed. + * @param loss Loss function used for minimization during gradient boosting. + * @param learningRate Learning rate for shrinking the contribution of each estimator. The + * learning rate should be between in the interval (0, 1] + * @param subsamplingRate Fraction of the training data used for learning the decision tree. + * @param numClassesForClassification Number of classes for classification. + * (Ignored for regression.) + * @param categoricalFeaturesInfo A map storing information about the categorical variables and + * the number of discrete values they take. For example, + * an entry (n -> k) implies the feature n is categorical with k + * categories 0, 1, 2, ... , k-1. It's important to note that + * features are zero-indexed. + * @param weakLearnerParams Parameters for the weak learner. (Currently only decision tree is + * supported.) + * @return WeightedEnsembleModel that can be used for prediction + */ + def trainClassifier( + input: RDD[LabeledPoint], + numEstimators: Int, + loss: String, + learningRate: Double, + subsamplingRate: Double, + numClassesForClassification: Int, + categoricalFeaturesInfo: Map[Int, Int], + weakLearnerParams: Strategy): WeightedEnsembleModel = { + val lossType = Losses.fromString(loss) + val boostingStrategy = new BoostingStrategy(Classification, numEstimators, lossType, + learningRate, subsamplingRate, numClassesForClassification, categoricalFeaturesInfo, + weakLearnerParams) + new GradientBoosting(boostingStrategy).train(input) + } + + /** + * Method to train a gradient boosting regression model. + * + * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * For classification, labels should take values {0, 1, ..., numClasses-1}. + * For regression, labels are real numbers. + * @param numEstimators Number of estimators used in boosting stages. In other words, + * number of boosting iterations performed. + * @param loss Loss function used for minimization during gradient boosting. + * @param learningRate Learning rate for shrinking the contribution of each estimator. The + * learning rate should be between in the interval (0, 1] + * @param subsamplingRate Fraction of the training data used for learning the decision tree. + * @param numClassesForClassification Number of classes for classification. + * (Ignored for regression.) + * @param categoricalFeaturesInfo A map storing information about the categorical variables and + * the number of discrete values they take. For example, + * an entry (n -> k) implies the feature n is categorical with k + * categories 0, 1, 2, ... , k-1. It's important to note that + * features are zero-indexed. + * @param weakLearnerParams Parameters for the weak learner. (Currently only decision tree is + * supported.) + * @return WeightedEnsembleModel that can be used for prediction + */ + def trainRegressor( + input: RDD[LabeledPoint], + numEstimators: Int, + loss: String, + learningRate: Double, + subsamplingRate: Double, + numClassesForClassification: Int, + categoricalFeaturesInfo: Map[Int, Int], + weakLearnerParams: Strategy): WeightedEnsembleModel = { + val lossType = Losses.fromString(loss) + val boostingStrategy = new BoostingStrategy(Regression, numEstimators, lossType, + learningRate, subsamplingRate, numClassesForClassification, categoricalFeaturesInfo, + weakLearnerParams) + new GradientBoosting(boostingStrategy).train(input) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainClassifier]] + */ + def trainClassifier( + input: RDD[LabeledPoint], + numEstimators: Int, + loss: String, + learningRate: Double, + subsamplingRate: Double, + numClassesForClassification: Int, + categoricalFeaturesInfo:java.util.Map[java.lang.Integer, java.lang.Integer], + weakLearnerParams: Strategy): WeightedEnsembleModel = { + trainClassifier(input, numEstimators, loss, learningRate, subsamplingRate, + numClassesForClassification, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + weakLearnerParams) + } + + /** + * Java-friendly API for [[org.apache.spark.mllib.tree.GradientBoosting$#trainRegressor]] + */ + def trainRegressor( + input: RDD[LabeledPoint], + numEstimators: Int, + loss: String, + learningRate: Double, + subsamplingRate: Double, + numClassesForClassification: Int, + categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], + weakLearnerParams: Strategy): WeightedEnsembleModel = { + trainRegressor(input, numEstimators, loss, learningRate, subsamplingRate, + numClassesForClassification, + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + weakLearnerParams) + } + + + /** + * Internal method for performing regression using trees as base learners. + * @param input training dataset + * @param boostingStrategy boosting parameters + * @return + */ + private def boost( + input: RDD[LabeledPoint], + boostingStrategy: BoostingStrategy): WeightedEnsembleModel = { + + val timer = new TimeTracker() + timer.start("total") + timer.start("init") + + // Initialize gradient boosting parameters + val numEstimators = boostingStrategy.numEstimators + val baseLearners = new Array[DecisionTreeModel](numEstimators) + val baseLearnerWeights = new Array[Double](numEstimators) + val loss = boostingStrategy.loss + val learningRate = boostingStrategy.learningRate + val strategy = boostingStrategy.weakLearnerParams + + // Cache input + input.persist(StorageLevel.MEMORY_AND_DISK) + + timer.stop("init") + + logDebug("##########") + logDebug("Building tree 0") + logDebug("##########") + var data = input + + // 1. Initialize tree + timer.start("building tree 0") + val firstTreeModel = new DecisionTree(strategy).train(data) + baseLearners(0) = firstTreeModel + baseLearnerWeights(0) = 1.0 + val startingModel = new WeightedEnsembleModel(Array(firstTreeModel), Array(1.0), Regression, + Sum) + logDebug("error of gbt = " + loss.computeError(startingModel, input)) + // Note: A model of type regression is used since we require raw prediction + timer.stop("building tree 0") + + // psuedo-residual for second iteration + data = input.map(point => LabeledPoint(loss.gradient(startingModel, point), + point.features)) + + var m = 1 + while (m < numEstimators) { + timer.start(s"building tree $m") + logDebug("###################################################") + logDebug("Gradient boosting tree iteration " + m) + logDebug("###################################################") + val model = new DecisionTree(strategy).train(data) + timer.stop(s"building tree $m") + // Create partial model + baseLearners(m) = model + baseLearnerWeights(m) = learningRate + // Note: A model of type regression is used since we require raw prediction + val partialModel = new WeightedEnsembleModel(baseLearners.slice(0, m + 1), + baseLearnerWeights.slice(0, m + 1), Regression, Sum) + logDebug("error of gbt = " + loss.computeError(partialModel, input)) + // Update data with pseudo-residuals + data = input.map(point => LabeledPoint(-loss.gradient(partialModel, point), + point.features)) + m += 1 + } + + timer.stop("total") + + logInfo("Internal timing for DecisionTree:") + logInfo(s"$timer") + + + // 3. Output classifier + new WeightedEnsembleModel(baseLearners, baseLearnerWeights, boostingStrategy.algo, Sum) + + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index ebbd8e0257209..1dcaf914381ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -26,6 +26,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Average import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker} import org.apache.spark.mllib.tree.impurity.Impurities @@ -59,7 +60,7 @@ import org.apache.spark.util.Utils * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "sqrt" for classification and * to "onethird" for regression. - * @param seed Random seed for bootstrapping and choosing feature subsets. + * @param seed Random seed for bootstrapping and choosing feature subsets. */ @Experimental private class RandomForest ( @@ -78,9 +79,9 @@ private class RandomForest ( /** * Method to train a decision tree model over an RDD * @param input Training data: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] - * @return RandomForestModel that can be used for prediction + * @return WeightedEnsembleModel that can be used for prediction */ - def train(input: RDD[LabeledPoint]): RandomForestModel = { + def train(input: RDD[LabeledPoint]): WeightedEnsembleModel = { val timer = new TimeTracker() @@ -111,11 +112,20 @@ private class RandomForest ( // Bin feature values (TreePoint representation). // Cache input RDD for speedup during multiple passes. val treeInput = TreePoint.convertToTreeRDD(retaggedInput, bins, metadata) - val baggedInput = if (numTrees > 1) { - BaggedPoint.convertToBaggedRDD(treeInput, numTrees, seed) - } else { - BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) - }.persist(StorageLevel.MEMORY_AND_DISK) + + val (subsample, withReplacement) = { + // TODO: Have a stricter check for RF in the strategy + val isRandomForest = numTrees > 1 + if (isRandomForest) { + (1.0, true) + } else { + (strategy.subsamplingRate, false) + } + } + + val baggedInput + = BaggedPoint.convertToBaggedRDD(treeInput, subsample, numTrees, withReplacement, seed) + .persist(StorageLevel.MEMORY_AND_DISK) // depth of the decision tree val maxDepth = strategy.maxDepth @@ -184,7 +194,8 @@ private class RandomForest ( logInfo(s"$timer") val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo)) - RandomForestModel.build(trees) + val treeWeights = Array.fill[Double](numTrees)(1.0) + new WeightedEnsembleModel(trees, treeWeights, strategy.algo, Average) } } @@ -205,14 +216,14 @@ object RandomForest extends Serializable with Logging { * if numTrees > 1 (forest) set to "sqrt" for classification and * to "onethird" for regression. * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return RandomForestModel that can be used for prediction + * @return WeightedEnsembleModel that can be used for prediction */ def trainClassifier( input: RDD[LabeledPoint], strategy: Strategy, numTrees: Int, featureSubsetStrategy: String, - seed: Int): RandomForestModel = { + seed: Int): WeightedEnsembleModel = { require(strategy.algo == Classification, s"RandomForest.trainClassifier given Strategy with invalid algo: ${strategy.algo}") val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed) @@ -243,7 +254,7 @@ object RandomForest extends Serializable with Logging { * @param maxBins maximum number of bins used for splitting features * (suggested value: 100) * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return RandomForestModel that can be used for prediction + * @return WeightedEnsembleModel that can be used for prediction */ def trainClassifier( input: RDD[LabeledPoint], @@ -254,7 +265,7 @@ object RandomForest extends Serializable with Logging { impurity: String, maxDepth: Int, maxBins: Int, - seed: Int = Utils.random.nextInt()): RandomForestModel = { + seed: Int = Utils.random.nextInt()): WeightedEnsembleModel = { val impurityType = Impurities.fromString(impurity) val strategy = new Strategy(Classification, impurityType, maxDepth, numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo) @@ -273,7 +284,7 @@ object RandomForest extends Serializable with Logging { impurity: String, maxDepth: Int, maxBins: Int, - seed: Int): RandomForestModel = { + seed: Int): WeightedEnsembleModel = { trainClassifier(input.rdd, numClassesForClassification, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) @@ -293,14 +304,14 @@ object RandomForest extends Serializable with Logging { * if numTrees > 1 (forest) set to "sqrt" for classification and * to "onethird" for regression. * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return RandomForestModel that can be used for prediction + * @return WeightedEnsembleModel that can be used for prediction */ def trainRegressor( input: RDD[LabeledPoint], strategy: Strategy, numTrees: Int, featureSubsetStrategy: String, - seed: Int): RandomForestModel = { + seed: Int): WeightedEnsembleModel = { require(strategy.algo == Regression, s"RandomForest.trainRegressor given Strategy with invalid algo: ${strategy.algo}") val rf = new RandomForest(strategy, numTrees, featureSubsetStrategy, seed) @@ -330,7 +341,7 @@ object RandomForest extends Serializable with Logging { * @param maxBins maximum number of bins used for splitting features * (suggested value: 100) * @param seed Random seed for bootstrapping and choosing feature subsets. - * @return RandomForestModel that can be used for prediction + * @return WeightedEnsembleModel that can be used for prediction */ def trainRegressor( input: RDD[LabeledPoint], @@ -340,7 +351,7 @@ object RandomForest extends Serializable with Logging { impurity: String, maxDepth: Int, maxBins: Int, - seed: Int = Utils.random.nextInt()): RandomForestModel = { + seed: Int = Utils.random.nextInt()): WeightedEnsembleModel = { val impurityType = Impurities.fromString(impurity) val strategy = new Strategy(Regression, impurityType, maxDepth, 0, maxBins, Sort, categoricalFeaturesInfo) @@ -358,7 +369,7 @@ object RandomForest extends Serializable with Logging { impurity: String, maxDepth: Int, maxBins: Int, - seed: Int): RandomForestModel = { + seed: Int): WeightedEnsembleModel = { trainRegressor(input.rdd, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala new file mode 100644 index 0000000000000..501d9ff9ea9b7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.configuration + +import scala.beans.BeanProperty + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.{Gini, Variance} +import org.apache.spark.mllib.tree.loss.{LogLoss, SquaredError, Loss} + +/** + * :: Experimental :: + * Stores all the configuration options for the boosting algorithms + * @param algo Learning goal. Supported: + * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], + * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] + * @param numEstimators Number of estimators used in boosting stages. In other words, + * number of boosting iterations performed. + * @param loss Loss function used for minimization during gradient boosting. + * @param learningRate Learning rate for shrinking the contribution of each estimator. The + * learning rate should be between in the interval (0, 1] + * @param subsamplingRate Fraction of the training data used for learning the decision tree. + * @param numClassesForClassification Number of classes for classification. + * (Ignored for regression.) + * Default value is 2 (binary classification). + * @param categoricalFeaturesInfo A map storing information about the categorical variables and the + * number of discrete values they take. For example, an entry (n -> + * k) implies the feature n is categorical with k categories 0, + * 1, 2, ... , k-1. It's important to note that features are + * zero-indexed. + * @param weakLearnerParams Parameters for weak learners. Currently only decision trees are + * supported. + */ +@Experimental +case class BoostingStrategy( + // Required boosting parameters + algo: Algo, + @BeanProperty var numEstimators: Int, + @BeanProperty var loss: Loss, + // Optional boosting parameters + @BeanProperty var learningRate: Double = 0.1, + @BeanProperty var subsamplingRate: Double = 1.0, + @BeanProperty var numClassesForClassification: Int = 2, + @BeanProperty var categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), + @BeanProperty var weakLearnerParams: Strategy) extends Serializable { + + require(learningRate <= 1, "Learning rate should be <= 1. Provided learning rate is " + + s"$learningRate.") + require(learningRate > 0, "Learning rate should be > 0. Provided learning rate is " + + s"$learningRate.") + + // Ensure values for weak learner are the same as what is provided to the boosting algorithm. + weakLearnerParams.categoricalFeaturesInfo = categoricalFeaturesInfo + weakLearnerParams.numClassesForClassification = numClassesForClassification + weakLearnerParams.subsamplingRate = subsamplingRate + +} + +@Experimental +object BoostingStrategy { + + /** + * Returns default configuration for the boosting algorithm + * @param algo Learning goal. Supported: + * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], + * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] + * @return Configuration for boosting algorithm + */ + def defaultParams(algo: Algo): BoostingStrategy = { + val treeStrategy = defaultWeakLearnerParams(algo) + algo match { + case Classification => + new BoostingStrategy(algo, 100, LogLoss, weakLearnerParams = treeStrategy) + case Regression => + new BoostingStrategy(algo, 100, SquaredError, weakLearnerParams = treeStrategy) + case _ => + throw new IllegalArgumentException(s"$algo is not supported by the boosting.") + } + } + + /** + * Returns default configuration for the weak learner (decision tree) algorithm + * @param algo Learning goal. Supported: + * [[org.apache.spark.mllib.tree.configuration.Algo.Classification]], + * [[org.apache.spark.mllib.tree.configuration.Algo.Regression]] + * @return Configuration for weak learner + */ + def defaultWeakLearnerParams(algo: Algo): Strategy = { + // Note: Regression tree used even for classification for GBT. + new Strategy(Regression, Variance, 3) + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala new file mode 100644 index 0000000000000..82889dc00cdad --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/EnsembleCombiningStrategy.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.configuration + +import org.apache.spark.annotation.DeveloperApi + +/** + * :: Experimental :: + * Enum to select ensemble combining strategy for base learners + */ +@DeveloperApi +object EnsembleCombiningStrategy extends Enumeration { + type EnsembleCombiningStrategy = Value + val Sum, Average = Value +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index caaccbfb8ad16..2ed63cf002efb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.tree.configuration +import scala.beans.BeanProperty import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental @@ -43,7 +44,7 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * for choosing how to split on features at each node. * More bins give higher granularity. * @param quantileCalculationStrategy Algorithm for calculating quantiles. Supported: - * [[org.apache.spark.mllib.tree.configuration.QuantileStrategy.Sort]] + * [[org.apache.spark.mllib.tree.configuration.QuantileStrategy.Sort]] * @param categoricalFeaturesInfo A map storing information about the categorical variables and the * number of discrete values they take. For example, an entry (n -> * k) implies the feature n is categorical with k categories 0, @@ -58,19 +59,21 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * this split will not be considered as a valid split. * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is * 256 MB. + * @param subsamplingRate Fraction of the training data used for learning decision tree. */ @Experimental class Strategy ( val algo: Algo, - val impurity: Impurity, - val maxDepth: Int, - val numClassesForClassification: Int = 2, - val maxBins: Int = 32, - val quantileCalculationStrategy: QuantileStrategy = Sort, - val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), - val minInstancesPerNode: Int = 1, - val minInfoGain: Double = 0.0, - val maxMemoryInMB: Int = 256) extends Serializable { + @BeanProperty var impurity: Impurity, + @BeanProperty var maxDepth: Int, + @BeanProperty var numClassesForClassification: Int = 2, + @BeanProperty var maxBins: Int = 32, + @BeanProperty var quantileCalculationStrategy: QuantileStrategy = Sort, + @BeanProperty var categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), + @BeanProperty var minInstancesPerNode: Int = 1, + @BeanProperty var minInfoGain: Double = 0.0, + @BeanProperty var maxMemoryInMB: Int = 256, + @BeanProperty var subsamplingRate: Double = 1) extends Serializable { if (algo == Classification) { require(numClassesForClassification >= 2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala index e7a2127c5d2e7..089010c81ffb6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/BaggedPoint.scala @@ -21,13 +21,14 @@ import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils +import org.apache.spark.util.random.XORShiftRandom /** * Internal representation of a datapoint which belongs to several subsamples of the same dataset, * particularly for bagging (e.g., for random forests). * * This holds one instance, as well as an array of weights which represent the (weighted) - * number of times which this instance appears in each subsample. + * number of times which this instance appears in each subsamplingRate. * E.g., (datum, [1, 0, 4]) indicates that there are 3 subsamples of the dataset and that * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively. * @@ -44,22 +45,65 @@ private[tree] object BaggedPoint { /** * Convert an input dataset into its BaggedPoint representation, - * choosing subsample counts for each instance. - * Each subsample has the same number of instances as the original dataset, - * and is created by subsampling with replacement. - * @param input Input dataset. - * @param numSubsamples Number of subsamples of this RDD to take. - * @param seed Random seed. - * @return BaggedPoint dataset representation + * choosing subsamplingRate counts for each instance. + * Each subsamplingRate has the same number of instances as the original dataset, + * and is created by subsampling without replacement. + * @param input Input dataset. + * @param subsamplingRate Fraction of the training data used for learning decision tree. + * @param numSubsamples Number of subsamples of this RDD to take. + * @param withReplacement Sampling with/without replacement. + * @param seed Random seed. + * @return BaggedPoint dataset representation. */ - def convertToBaggedRDD[Datum]( + def convertToBaggedRDD[Datum] ( input: RDD[Datum], + subsamplingRate: Double, numSubsamples: Int, + withReplacement: Boolean, seed: Int = Utils.random.nextInt()): RDD[BaggedPoint[Datum]] = { + if (withReplacement) { + convertToBaggedRDDSamplingWithReplacement(input, subsamplingRate, numSubsamples, seed) + } else { + if (numSubsamples == 1 && subsamplingRate == 1.0) { + convertToBaggedRDDWithoutSampling(input) + } else { + convertToBaggedRDDSamplingWithoutReplacement(input, subsamplingRate, numSubsamples, seed) + } + } + } + + private def convertToBaggedRDDSamplingWithoutReplacement[Datum] ( + input: RDD[Datum], + subsamplingRate: Double, + numSubsamples: Int, + seed: Int): RDD[BaggedPoint[Datum]] = { + input.mapPartitionsWithIndex { (partitionIndex, instances) => + // Use random seed = seed + partitionIndex + 1 to make generation reproducible. + val rng = new XORShiftRandom + rng.setSeed(seed + partitionIndex + 1) + instances.map { instance => + val subsampleWeights = new Array[Double](numSubsamples) + var subsampleIndex = 0 + while (subsampleIndex < numSubsamples) { + val x = rng.nextDouble() + subsampleWeights(subsampleIndex) = { + if (x < subsamplingRate) 1.0 else 0.0 + } + subsampleIndex += 1 + } + new BaggedPoint(instance, subsampleWeights) + } + } + } + + private def convertToBaggedRDDSamplingWithReplacement[Datum] ( + input: RDD[Datum], + subsample: Double, + numSubsamples: Int, + seed: Int): RDD[BaggedPoint[Datum]] = { input.mapPartitionsWithIndex { (partitionIndex, instances) => - // TODO: Support different sampling rates, and sampling without replacement. // Use random seed = seed + partitionIndex + 1 to make generation reproducible. - val poisson = new PoissonDistribution(1.0) + val poisson = new PoissonDistribution(subsample) poisson.reseedRandomGenerator(seed + partitionIndex + 1) instances.map { instance => val subsampleWeights = new Array[Double](numSubsamples) @@ -73,7 +117,8 @@ private[tree] object BaggedPoint { } } - def convertToBaggedRDDWithoutSampling[Datum](input: RDD[Datum]): RDD[BaggedPoint[Datum]] = { + private def convertToBaggedRDDWithoutSampling[Datum] ( + input: RDD[Datum]): RDD[BaggedPoint[Datum]] = { input.map(datum => new BaggedPoint(datum, Array(1.0))) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala new file mode 100644 index 0000000000000..d111ffe30ed9e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.loss + +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel +import org.apache.spark.rdd.RDD + +/** + * :: DeveloperApi :: + * Class for least absolute error loss calculation. + * The features x and the corresponding label y is predicted using the function F. + * For each instance: + * Loss: |y - F| + * Negative gradient: sign(y - F) + */ +@DeveloperApi +object AbsoluteError extends Loss { + + /** + * Method to calculate the gradients for the gradient boosting calculation for least + * absolute error calculation. + * @param model Model of the weak learner + * @param point Instance of the training dataset + * @return Loss gradient + */ + override def gradient( + model: WeightedEnsembleModel, + point: LabeledPoint): Double = { + if ((point.label - model.predict(point.features)) < 0) 1.0 else -1.0 + } + + /** + * Method to calculate error of the base learner for the gradient boosting calculation. + * Note: This method is not used by the gradient boosting algorithm but is useful for debugging + * purposes. + * @param model Model of the weak learner. + * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @return + */ + override def computeError(model: WeightedEnsembleModel, data: RDD[LabeledPoint]): Double = { + val sumOfAbsolutes = data.map { y => + val err = model.predict(y.features) - y.label + math.abs(err) + }.sum() + sumOfAbsolutes / data.count() + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala new file mode 100644 index 0000000000000..6f3d4340f0d3b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -0,0 +1,63 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.loss + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel +import org.apache.spark.rdd.RDD + +/** + * :: DeveloperApi :: + * Class for least squares error loss calculation. + * + * The features x and the corresponding label y is predicted using the function F. + * For each instance: + * Loss: log(1 + exp(-2yF)), y in {-1, 1} + * Negative gradient: 2y / ( 1 + exp(2yF)) + */ +@DeveloperApi +object LogLoss extends Loss { + + /** + * Method to calculate the loss gradients for the gradient boosting calculation for binary + * classification + * @param model Model of the weak learner + * @param point Instance of the training dataset + * @return Loss gradient + */ + override def gradient( + model: WeightedEnsembleModel, + point: LabeledPoint): Double = { + val prediction = model.predict(point.features) + 1.0 / (1.0 + math.exp(-prediction)) - point.label + } + + /** + * Method to calculate error of the base learner for the gradient boosting calculation. + * Note: This method is not used by the gradient boosting algorithm but is useful for debugging + * purposes. + * @param model Model of the weak learner. + * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @return + */ + override def computeError(model: WeightedEnsembleModel, data: RDD[LabeledPoint]): Double = { + val wrongPredictions = data.filter(lp => model.predict(lp.features) != lp.label).count() + wrongPredictions / data.count + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala new file mode 100644 index 0000000000000..5580866c879e2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.loss + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel +import org.apache.spark.rdd.RDD + +/** + * :: DeveloperApi :: + * Trait for adding "pluggable" loss functions for the gradient boosting algorithm. + */ +@DeveloperApi +trait Loss extends Serializable { + + /** + * Method to calculate the gradients for the gradient boosting calculation. + * @param model Model of the weak learner. + * @param point Instance of the training dataset. + * @return Loss gradient. + */ + def gradient( + model: WeightedEnsembleModel, + point: LabeledPoint): Double + + /** + * Method to calculate error of the base learner for the gradient boosting calculation. + * Note: This method is not used by the gradient boosting algorithm but is useful for debugging + * purposes. + * @param model Model of the weak learner. + * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @return + */ + def computeError(model: WeightedEnsembleModel, data: RDD[LabeledPoint]): Double + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala new file mode 100644 index 0000000000000..42c9ead9884b4 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Losses.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.loss + +object Losses { + + def fromString(name: String): Loss = name match { + case "leastSquaresError" => SquaredError + case "leastAbsoluteError" => AbsoluteError + case "logLoss" => LogLoss + case _ => throw new IllegalArgumentException(s"Did not recognize Loss name: $name") + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala new file mode 100644 index 0000000000000..4349fefef2c74 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.loss + +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel +import org.apache.spark.rdd.RDD + +/** + * :: DeveloperApi :: + * Class for least squares error loss calculation. + * + * The features x and the corresponding label y is predicted using the function F. + * For each instance: + * Loss: (y - F)**2/2 + * Negative gradient: y - F + */ +@DeveloperApi +object SquaredError extends Loss { + + /** + * Method to calculate the gradients for the gradient boosting calculation for least + * squares error calculation. + * @param model Model of the weak learner + * @param point Instance of the training dataset + * @return Loss gradient + */ + override def gradient( + model: WeightedEnsembleModel, + point: LabeledPoint): Double = { + model.predict(point.features) - point.label + } + + /** + * Method to calculate error of the base learner for the gradient boosting calculation. + * Note: This method is not used by the gradient boosting algorithm but is useful for debugging + * purposes. + * @param model Model of the weak learner. + * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. + * @return + */ + override def computeError(model: WeightedEnsembleModel, data: RDD[LabeledPoint]): Double = { + data.map { y => + val err = model.predict(y.features) - y.label + err * err + }.mean() + } + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala deleted file mode 100644 index 6a22e2abe59bd..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/RandomForestModel.scala +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.mllib.tree.model - -import scala.collection.mutable - -import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.Vector -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.rdd.RDD - -/** - * :: Experimental :: - * Random forest model for classification or regression. - * This model stores a collection of [[DecisionTreeModel]] instances and uses them to make - * aggregate predictions. - * @param trees Trees which make up this forest. This cannot be empty. - * @param algo algorithm type -- classification or regression - */ -@Experimental -class RandomForestModel(val trees: Array[DecisionTreeModel], val algo: Algo) extends Serializable { - - require(trees.size > 0, s"RandomForestModel cannot be created with empty trees collection.") - - /** - * Predict values for a single data point. - * - * @param features array representing a single data point - * @return Double prediction from the trained model - */ - def predict(features: Vector): Double = { - algo match { - case Classification => - val predictionToCount = new mutable.HashMap[Int, Int]() - trees.foreach { tree => - val prediction = tree.predict(features).toInt - predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1 - } - predictionToCount.maxBy(_._2)._1 - case Regression => - trees.map(_.predict(features)).sum / trees.size - } - } - - /** - * Predict values for the given data set. - * - * @param features RDD representing data points to be predicted - * @return RDD[Double] where each entry contains the corresponding prediction - */ - def predict(features: RDD[Vector]): RDD[Double] = { - features.map(x => predict(x)) - } - - /** - * Get number of trees in forest. - */ - def numTrees: Int = trees.size - - /** - * Get total number of nodes, summed over all trees in the forest. - */ - def totalNumNodes: Int = trees.map(tree => tree.numNodes).sum - - /** - * Print a summary of the model. - */ - override def toString: String = algo match { - case Classification => - s"RandomForestModel classifier with $numTrees trees and $totalNumNodes total nodes" - case Regression => - s"RandomForestModel regressor with $numTrees trees and $totalNumNodes total nodes" - case _ => throw new IllegalArgumentException( - s"RandomForestModel given unknown algo parameter: $algo.") - } - - /** - * Print the full model to a string. - */ - def toDebugString: String = { - val header = toString + "\n" - header + trees.zipWithIndex.map { case (tree, treeIndex) => - s" Tree $treeIndex:\n" + tree.topNode.subtreeToString(4) - }.fold("")(_ + _) - } - -} - -private[tree] object RandomForestModel { - - def build(trees: Array[DecisionTreeModel]): RandomForestModel = { - require(trees.size > 0, s"RandomForestModel cannot be created with empty trees collection.") - val algo: Algo = trees(0).algo - require(trees.forall(_.algo == algo), - "RandomForestModel cannot combine trees which have different output types" + - " (classification/regression).") - new RandomForestModel(trees, algo) - } - -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala new file mode 100644 index 0000000000000..7b052d9163a13 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/WeightedEnsembleModel.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.model + +import org.apache.spark.annotation.Experimental +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy._ +import org.apache.spark.rdd.RDD + +import scala.collection.mutable + +@Experimental +class WeightedEnsembleModel( + val weakHypotheses: Array[DecisionTreeModel], + val weakHypothesisWeights: Array[Double], + val algo: Algo, + val combiningStrategy: EnsembleCombiningStrategy) extends Serializable { + + require(numWeakHypotheses > 0, s"WeightedEnsembleModel cannot be created without weakHypotheses" + + s". Number of weakHypotheses = $weakHypotheses") + + /** + * Predict values for a single data point using the model trained. + * + * @param features array representing a single data point + * @return predicted category from the trained model + */ + private def predictRaw(features: Vector): Double = { + val treePredictions = weakHypotheses.map(learner => learner.predict(features)) + if (numWeakHypotheses == 1){ + treePredictions(0) + } else { + var prediction = treePredictions(0) + var index = 1 + while (index < numWeakHypotheses) { + prediction += weakHypothesisWeights(index) * treePredictions(index) + index += 1 + } + prediction + } + } + + /** + * Predict values for a single data point using the model trained. + * + * @param features array representing a single data point + * @return predicted category from the trained model + */ + private def predictBySumming(features: Vector): Double = { + algo match { + case Regression => predictRaw(features) + case Classification => { + // TODO: predicted labels are +1 or -1 for GBT. Need a better way to store this info. + if (predictRaw(features) > 0 ) 1.0 else 0.0 + } + case _ => throw new IllegalArgumentException( + s"WeightedEnsembleModel given unknown algo parameter: $algo.") + } + } + + /** + * Predict values for a single data point. + * + * @param features array representing a single data point + * @return Double prediction from the trained model + */ + private def predictByAveraging(features: Vector): Double = { + algo match { + case Classification => + val predictionToCount = new mutable.HashMap[Int, Int]() + weakHypotheses.foreach { learner => + val prediction = learner.predict(features).toInt + predictionToCount(prediction) = predictionToCount.getOrElse(prediction, 0) + 1 + } + predictionToCount.maxBy(_._2)._1 + case Regression => + weakHypotheses.map(_.predict(features)).sum / weakHypotheses.size + } + } + + + /** + * Predict values for a single data point using the model trained. + * + * @param features array representing a single data point + * @return predicted category from the trained model + */ + def predict(features: Vector): Double = { + combiningStrategy match { + case Sum => predictBySumming(features) + case Average => predictByAveraging(features) + case _ => throw new IllegalArgumentException( + s"WeightedEnsembleModel given unknown combining parameter: $combiningStrategy.") + } + } + + /** + * Predict values for the given data set. + * + * @param features RDD representing data points to be predicted + * @return RDD[Double] where each entry contains the corresponding prediction + */ + def predict(features: RDD[Vector]): RDD[Double] = features.map(x => predict(x)) + + /** + * Print a summary of the model. + */ + override def toString: String = { + algo match { + case Classification => + s"WeightedEnsembleModel classifier with $numWeakHypotheses trees\n" + case Regression => + s"WeightedEnsembleModel regressor with $numWeakHypotheses trees\n" + case _ => throw new IllegalArgumentException( + s"WeightedEnsembleModel given unknown algo parameter: $algo.") + } + } + + /** + * Print the full model to a string. + */ + def toDebugString: String = { + val header = toString + "\n" + header + weakHypotheses.zipWithIndex.map { case (tree, treeIndex) => + s" Tree $treeIndex:\n" + tree.topNode.subtreeToString(4) + }.fold("")(_ + _) + } + + /** + * Get number of trees in forest. + */ + def numWeakHypotheses: Int = weakHypotheses.size + + // TODO: Remove these helpers methods once class is generalized to support any base learning + // algorithms. + + /** + * Get total number of nodes, summed over all trees in the forest. + */ + def totalNumNodes: Int = weakHypotheses.map(tree => tree.numNodes).sum + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 8fc5e111bbc17..c579cb58549f5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -493,7 +493,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { assert(rootNode1.rightNode.nonEmpty) val treeInput = TreePoint.convertToTreeRDD(rdd, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) // Single group second level tree construction. val nodesForGroup = Map((0, Array(rootNode1.leftNode.get, rootNode1.rightNode.get))) @@ -786,7 +786,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) val topNode = Node.emptyNode(nodeIndex = 1) assert(topNode.predict.predict === Double.MinValue) @@ -829,7 +829,7 @@ class DecisionTreeSuite extends FunSuite with LocalSparkContext { val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) val treeInput = TreePoint.convertToTreeRDD(input, bins, metadata) - val baggedInput = BaggedPoint.convertToBaggedRDDWithoutSampling(treeInput) + val baggedInput = BaggedPoint.convertToBaggedRDD(treeInput, 1.0, 1, false) val topNode = Node.emptyNode(nodeIndex = 1) assert(topNode.predict.predict === Double.MinValue) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala new file mode 100644 index 0000000000000..effb7b8259ffb --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.model.WeightedEnsembleModel +import org.apache.spark.util.StatCounter + +import scala.collection.mutable + +object EnsembleTestHelper { + + /** + * Aggregates all values in data, and tests whether the empirical mean and stddev are within + * epsilon of the expected values. + * @param data Every element of the data should be an i.i.d. sample from some distribution. + */ + def testRandomArrays( + data: Array[Array[Double]], + numCols: Int, + expectedMean: Double, + expectedStddev: Double, + epsilon: Double) { + val values = new mutable.ArrayBuffer[Double]() + data.foreach { row => + assert(row.size == numCols) + values ++= row + } + val stats = new StatCounter(values) + assert(math.abs(stats.mean - expectedMean) < epsilon) + assert(math.abs(stats.stdev - expectedStddev) < epsilon) + } + + def validateClassifier( + model: WeightedEnsembleModel, + input: Seq[LabeledPoint], + requiredAccuracy: Double) { + val predictions = input.map(x => model.predict(x.features)) + val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => + prediction != expected.label + } + val accuracy = (input.length - numOffPredictions).toDouble / input.length + assert(accuracy >= requiredAccuracy, + s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") + } + + def validateRegressor( + model: WeightedEnsembleModel, + input: Seq[LabeledPoint], + requiredMSE: Double) { + val predictions = input.map(x => model.predict(x.features)) + val squaredError = predictions.zip(input).map { case (prediction, expected) => + val err = prediction - expected.label + err * err + }.sum + val mse = squaredError / input.length + assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") + } + + def generateOrderedLabeledPoints(numFeatures: Int, numInstances: Int): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](numInstances) + for (i <- 0 until numInstances) { + val label = if (i < numInstances / 10) { + 0.0 + } else if (i < numInstances / 2) { + 1.0 + } else if (i < numInstances * 0.9) { + 0.0 + } else { + 1.0 + } + val features = Array.fill[Double](numFeatures)(i.toDouble) + arr(i) = new LabeledPoint(label, Vectors.dense(features)) + } + arr + } + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala new file mode 100644 index 0000000000000..970fff82215e2 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostingSuite.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Strategy} +import org.apache.spark.mllib.tree.impurity.{Variance, Gini} +import org.apache.spark.mllib.tree.loss.{SquaredError, LogLoss} +import org.apache.spark.mllib.tree.model.{WeightedEnsembleModel, DecisionTreeModel} + +import org.apache.spark.mllib.util.LocalSparkContext + +/** + * Test suite for [[GradientBoosting]]. + */ +class GradientBoostingSuite extends FunSuite with LocalSparkContext { + + test("Regression with continuous features: SquaredError") { + + GradientBoostingSuite.testCombinations.foreach { + case (numEstimators, learningRate, subsamplingRate) => + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + + val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + subsamplingRate = subsamplingRate) + + val dt = DecisionTree.train(remappedInput, treeStrategy) + + val boostingStrategy = new BoostingStrategy(Regression, numEstimators, SquaredError, + subsamplingRate, learningRate, 1, categoricalFeaturesInfo, treeStrategy) + + val gbt = GradientBoosting.trainRegressor(rdd, boostingStrategy) + assert(gbt.weakHypotheses.size === numEstimators) + val gbtTree = gbt.weakHypotheses(0) + + EnsembleTestHelper.validateRegressor(gbt, arr, 0.02) + + // Make sure trees are the same. + assert(gbtTree.toString == dt.toString) + } + } + + test("Regression with continuous features: Absolute Error") { + + GradientBoostingSuite.testCombinations.foreach { + case (numEstimators, learningRate, subsamplingRate) => + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + + val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + subsamplingRate = subsamplingRate) + + val dt = DecisionTree.train(remappedInput, treeStrategy) + + val boostingStrategy = new BoostingStrategy(Regression, numEstimators, SquaredError, + subsamplingRate, learningRate, 1, categoricalFeaturesInfo, treeStrategy) + + val gbt = GradientBoosting.trainRegressor(rdd, boostingStrategy) + assert(gbt.weakHypotheses.size === numEstimators) + val gbtTree = gbt.weakHypotheses(0) + + EnsembleTestHelper.validateRegressor(gbt, arr, 0.02) + + // Make sure trees are the same. + assert(gbtTree.toString == dt.toString) + } + } + + + test("Binary classification with continuous features: Log Loss") { + + GradientBoostingSuite.testCombinations.foreach { + case (numEstimators, learningRate, subsamplingRate) => + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) + val rdd = sc.parallelize(arr) + val categoricalFeaturesInfo = Map.empty[Int, Int] + + val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + subsamplingRate = subsamplingRate) + + val dt = DecisionTree.train(remappedInput, treeStrategy) + + val boostingStrategy = new BoostingStrategy(Classification, numEstimators, LogLoss, + subsamplingRate, learningRate, 1, categoricalFeaturesInfo, treeStrategy) + + val gbt = GradientBoosting.trainClassifier(rdd, boostingStrategy) + assert(gbt.weakHypotheses.size === numEstimators) + val gbtTree = gbt.weakHypotheses(0) + + EnsembleTestHelper.validateClassifier(gbt, arr, 0.9) + + // Make sure trees are the same. + assert(gbtTree.toString == dt.toString) + } + } + +} + +object GradientBoostingSuite { + + // Combinations for estimators, learning rates and subsamplingRate + val testCombinations + = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 1.0, 0.75), (10, 0.1, 0.75)) + +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index d3eff59aa0409..10c046e07f562 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -25,45 +25,20 @@ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.Strategy -import org.apache.spark.mllib.tree.impl.{BaggedPoint, DecisionTreeMetadata} +import org.apache.spark.mllib.tree.impl.DecisionTreeMetadata import org.apache.spark.mllib.tree.impurity.{Gini, Variance} -import org.apache.spark.mllib.tree.model.{Node, RandomForestModel} +import org.apache.spark.mllib.tree.model.Node import org.apache.spark.mllib.util.LocalSparkContext -import org.apache.spark.util.StatCounter /** * Test suite for [[RandomForest]]. */ class RandomForestSuite extends FunSuite with LocalSparkContext { - test("BaggedPoint RDD: without subsampling") { - val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1) - val rdd = sc.parallelize(arr) - val baggedRDD = BaggedPoint.convertToBaggedRDDWithoutSampling(rdd) - baggedRDD.collect().foreach { baggedPoint => - assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1) - } - } - - test("BaggedPoint RDD: with subsampling") { - val numSubsamples = 100 - val (expectedMean, expectedStddev) = (1.0, 1.0) - - val seeds = Array(123, 5354, 230, 349867, 23987) - val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 1) - val rdd = sc.parallelize(arr) - seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, numSubsamples, seed = seed) - val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() - RandomForestSuite.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, - expectedStddev, epsilon = 0.01) - } - } - test("Binary classification with continuous features:" + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { - val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 50) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) val categoricalFeaturesInfo = Map.empty[Int, Int] val numTrees = 1 @@ -73,12 +48,12 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { val rf = RandomForest.trainClassifier(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) - assert(rf.trees.size === 1) - val rfTree = rf.trees(0) + assert(rf.weakHypotheses.size === 1) + val rfTree = rf.weakHypotheses(0) val dt = DecisionTree.train(rdd, strategy) - RandomForestSuite.validateClassifier(rf, arr, 0.9) + EnsembleTestHelper.validateClassifier(rf, arr, 0.9) DecisionTreeSuite.validateClassifier(dt, arr, 0.9) // Make sure trees are the same. @@ -88,7 +63,7 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { test("Regression with continuous features:" + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { - val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures = 50) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) val categoricalFeaturesInfo = Map.empty[Int, Int] val numTrees = 1 @@ -99,12 +74,12 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { val rf = RandomForest.trainRegressor(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) - assert(rf.trees.size === 1) - val rfTree = rf.trees(0) + assert(rf.weakHypotheses.size === 1) + val rfTree = rf.weakHypotheses(0) val dt = DecisionTree.train(rdd, strategy) - RandomForestSuite.validateRegressor(rf, arr, 0.01) + EnsembleTestHelper.validateRegressor(rf, arr, 0.01) DecisionTreeSuite.validateRegressor(dt, arr, 0.01) // Make sure trees are the same. @@ -113,7 +88,7 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { test("Binary classification with continuous features: subsampling features") { val numFeatures = 50 - val arr = RandomForestSuite.generateOrderedLabeledPoints(numFeatures) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) val rdd = sc.parallelize(arr) val categoricalFeaturesInfo = Map.empty[Int, Int] @@ -187,77 +162,9 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { numClassesForClassification = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) val model = RandomForest.trainClassifier(input, strategy, numTrees = 2, featureSubsetStrategy = "sqrt", seed = 12345) - RandomForestSuite.validateClassifier(model, arr, 0.0) + EnsembleTestHelper.validateClassifier(model, arr, 1.0) } } -object RandomForestSuite { - - /** - * Aggregates all values in data, and tests whether the empirical mean and stddev are within - * epsilon of the expected values. - * @param data Every element of the data should be an i.i.d. sample from some distribution. - */ - def testRandomArrays( - data: Array[Array[Double]], - numCols: Int, - expectedMean: Double, - expectedStddev: Double, - epsilon: Double) { - val values = new mutable.ArrayBuffer[Double]() - data.foreach { row => - assert(row.size == numCols) - values ++= row - } - val stats = new StatCounter(values) - assert(math.abs(stats.mean - expectedMean) < epsilon) - assert(math.abs(stats.stdev - expectedStddev) < epsilon) - } - - def validateClassifier( - model: RandomForestModel, - input: Seq[LabeledPoint], - requiredAccuracy: Double) { - val predictions = input.map(x => model.predict(x.features)) - val numOffPredictions = predictions.zip(input).count { case (prediction, expected) => - prediction != expected.label - } - val accuracy = (input.length - numOffPredictions).toDouble / input.length - assert(accuracy >= requiredAccuracy, - s"validateClassifier calculated accuracy $accuracy but required $requiredAccuracy.") - } - - def validateRegressor( - model: RandomForestModel, - input: Seq[LabeledPoint], - requiredMSE: Double) { - val predictions = input.map(x => model.predict(x.features)) - val squaredError = predictions.zip(input).map { case (prediction, expected) => - val err = prediction - expected.label - err * err - }.sum - val mse = squaredError / input.length - assert(mse <= requiredMSE, s"validateRegressor calculated MSE $mse but required $requiredMSE.") - } - def generateOrderedLabeledPoints(numFeatures: Int): Array[LabeledPoint] = { - val numInstances = 1000 - val arr = new Array[LabeledPoint](numInstances) - for (i <- 0 until numInstances) { - val label = if (i < numInstances / 10) { - 0.0 - } else if (i < numInstances / 2) { - 1.0 - } else if (i < numInstances * 0.9) { - 0.0 - } else { - 1.0 - } - val features = Array.fill[Double](numFeatures)(i.toDouble) - arr(i) = new LabeledPoint(label, Vectors.dense(features)) - } - arr - } - -} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala new file mode 100644 index 0000000000000..c0a62e00432a3 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.tree.EnsembleTestHelper +import org.apache.spark.mllib.util.LocalSparkContext + +/** + * Test suite for [[BaggedPoint]]. + */ +class BaggedPointSuite extends FunSuite with LocalSparkContext { + + test("BaggedPoint RDD: without subsampling") { + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + val rdd = sc.parallelize(arr) + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false) + baggedRDD.collect().foreach { baggedPoint => + assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1) + } + } + + test("BaggedPoint RDD: with subsampling with replacement (fraction = 1.0)") { + val numSubsamples = 100 + val (expectedMean, expectedStddev) = (1.0, 1.0) + + val seeds = Array(123, 5354, 230, 349867, 23987) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + val rdd = sc.parallelize(arr) + seeds.foreach { seed => + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true) + val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, + expectedStddev, epsilon = 0.01) + } + } + + test("BaggedPoint RDD: with subsampling with replacement (fraction = 0.5)") { + val numSubsamples = 100 + val subsample = 0.5 + val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample)) + + val seeds = Array(123, 5354, 230, 349867, 23987) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + val rdd = sc.parallelize(arr) + seeds.foreach { seed => + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true) + val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, + expectedStddev, epsilon = 0.01) + } + } + + test("BaggedPoint RDD: with subsampling without replacement (fraction = 1.0)") { + val numSubsamples = 100 + val (expectedMean, expectedStddev) = (1.0, 0) + + val seeds = Array(123, 5354, 230, 349867, 23987) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + val rdd = sc.parallelize(arr) + seeds.foreach { seed => + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false) + val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, + expectedStddev, epsilon = 0.01) + } + } + + test("BaggedPoint RDD: with subsampling without replacement (fraction = 0.5)") { + val numSubsamples = 100 + val subsample = 0.5 + val (expectedMean, expectedStddev) = (subsample, math.sqrt(subsample * (1 - subsample))) + + val seeds = Array(123, 5354, 230, 349867, 23987) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + val rdd = sc.parallelize(arr) + seeds.foreach { seed => + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false) + val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, + expectedStddev, epsilon = 0.01) + } + } +} From 98c556ebbca6a815813daaefd292d2e46fb16cc2 Mon Sep 17 00:00:00 2001 From: freeman Date: Fri, 31 Oct 2014 22:30:12 -0700 Subject: [PATCH 1171/1492] Streaming KMeans [MLLIB][SPARK-3254] This adds a Streaming KMeans algorithm to MLlib. It uses an update rule that generalizes the mini-batch KMeans update to incorporate a decay factor, which allows past data to be forgotten. The decay factor can be specified explicitly, or via a more intuitive "fractional decay" setting, in units of either data points or batches. The PR includes: - StreamingKMeans algorithm with decay factor settings - Usage example - Additions to documentation clustering page - Unit tests of basic behavior and decay behaviors tdas mengxr rezazadeh Author: freeman Author: Jeremy Freeman Author: Xiangrui Meng Closes #2942 from freeman-lab/streaming-kmeans and squashes the following commits: b2e5b4a [freeman] Fixes to docs / examples 078617c [Jeremy Freeman] Merge pull request #1 from mengxr/SPARK-3254 2e682c0 [Xiangrui Meng] take discount on previous weights; use BLAS; detect dying clusters 0411bf5 [freeman] Change decay parameterization 9f7aea9 [freeman] Style fixes 374a706 [freeman] Formatting ad9bdc2 [freeman] Use labeled points and predictOnValues in examples 77dbd3f [freeman] Make initialization check an assertion 9cfc301 [freeman] Make random seed an argument 44050a9 [freeman] Simpler constructor c7050d5 [freeman] Fix spacing 2899623 [freeman] Use pattern matching for clarity a4a316b [freeman] Use collect 1472ec5 [freeman] Doc formatting ea22ec8 [freeman] Fix imports 2086bdc [freeman] Log cluster center updates ea9877c [freeman] More documentation 9facbe3 [freeman] Bug fix 5db7074 [freeman] Example usage for StreamingKMeans f33684b [freeman] Add explanation and example to docs b5b5f8d [freeman] Add better documentation a0fd790 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-kmeans 9fd9c15 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-kmeans b93350f [freeman] Streaming KMeans with decay --- docs/mllib-clustering.md | 96 ++++++- .../examples/mllib/StreamingKMeans.scala | 77 +++++ .../mllib/clustering/StreamingKMeans.scala | 268 ++++++++++++++++++ .../clustering/StreamingKMeansSuite.scala | 157 ++++++++++ 4 files changed, 597 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 7978e934fb36b..c696ae9c8e8c8 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -34,7 +34,7 @@ a given dataset, the algorithm returns the best clustering result). * *initializationSteps* determines the number of steps in the k-means\|\| algorithm. * *epsilon* determines the distance threshold within which we consider k-means to have converged. -## Examples +### Examples
    @@ -153,3 +153,97 @@ provided in the [Self-Contained Applications](quick-start.html#self-contained-ap section of the Spark Quick Start guide. Be sure to also include *spark-mllib* to your build file as a dependency. + +## Streaming clustering + +When data arrive in a stream, we may want to estimate clusters dynamically, +updating them as new data arrive. MLlib provides support for streaming k-means clustering, +with parameters to control the decay (or "forgetfulness") of the estimates. The algorithm +uses a generalization of the mini-batch k-means update rule. For each batch of data, we assign +all points to their nearest cluster, compute new cluster centers, then update each cluster using: + +`\begin{equation} + c_{t+1} = \frac{c_tn_t\alpha + x_tm_t}{n_t\alpha+m_t} +\end{equation}` +`\begin{equation} + n_{t+1} = n_t + m_t +\end{equation}` + +Where `$c_t$` is the previous center for the cluster, `$n_t$` is the number of points assigned +to the cluster thus far, `$x_t$` is the new cluster center from the current batch, and `$m_t$` +is the number of points added to the cluster in the current batch. The decay factor `$\alpha$` +can be used to ignore the past: with `$\alpha$=1` all data will be used from the beginning; +with `$\alpha$=0` only the most recent data will be used. This is analogous to an +exponentially-weighted moving average. + +The decay can be specified using a `halfLife` parameter, which determines the +correct decay factor `a` such that, for data acquired +at time `t`, its contribution by time `t + halfLife` will have dropped to 0.5. +The unit of time can be specified either as `batches` or `points` and the update rule +will be adjusted accordingly. + +### Examples + +This example shows how to estimate clusters on streaming data. + +
    + +
    + +First we import the neccessary classes. + +{% highlight scala %} + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.clustering.StreamingKMeans + +{% endhighlight %} + +Then we make an input stream of vectors for training, as well as a stream of labeled data +points for testing. We assume a StreamingContext `ssc` has been created, see +[Spark Streaming Programming Guide](streaming-programming-guide.html#initializing) for more info. + +{% highlight scala %} + +val trainingData = ssc.textFileStream("/training/data/dir").map(Vectors.parse) +val testData = ssc.textFileStream("/testing/data/dir").map(LabeledPoint.parse) + +{% endhighlight %} + +We create a model with random clusters and specify the number of clusters to find + +{% highlight scala %} + +val numDimensions = 3 +val numClusters = 2 +val model = new StreamingKMeans() + .setK(numClusters) + .setDecayFactor(1.0) + .setRandomCenters(numDimensions, 0.0) + +{% endhighlight %} + +Now register the streams for training and testing and start the job, printing +the predicted cluster assignments on new data points as they arrive. + +{% highlight scala %} + +model.trainOn(trainingData) +model.predictOnValues(testData).print() + +ssc.start() +ssc.awaitTermination() + +{% endhighlight %} + +As you add new text files with data the cluster centers will update. Each training +point should be formatted as `[x1, x2, x3]`, and each test data point +should be formatted as `(y, [x1, x2, x3])`, where `y` is some useful label or identifier +(e.g. a true category assignment). Anytime a text file is placed in `/training/data/dir` +the model will update. Anytime a text file is placed in `/testing/data/dir` +you will see predictions. With new data, the cluster centers will change! + +
    + +
    diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala new file mode 100644 index 0000000000000..33e5760aed997 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.clustering.StreamingKMeans +import org.apache.spark.SparkConf +import org.apache.spark.streaming.{Seconds, StreamingContext} + +/** + * Estimate clusters on one stream of data and make predictions + * on another stream, where the data streams arrive as text files + * into two different directories. + * + * The rows of the training text files must be vector data in the form + * `[x1,x2,x3,...,xn]` + * Where n is the number of dimensions. + * + * The rows of the test text files must be labeled data in the form + * `(y,[x1,x2,x3,...,xn])` + * Where y is some identifier. n must be the same for train and test. + * + * Usage: StreamingKmeans + * + * To run on your local machine using the two directories `trainingDir` and `testDir`, + * with updates every 5 seconds, 2 dimensions per data point, and 3 clusters, call: + * $ bin/run-example \ + * org.apache.spark.examples.mllib.StreamingKMeans trainingDir testDir 5 3 2 + * + * As you add text files to `trainingDir` the clusters will continuously update. + * Anytime you add text files to `testDir`, you'll see predicted labels using the current model. + * + */ +object StreamingKMeans { + + def main(args: Array[String]) { + if (args.length != 5) { + System.err.println( + "Usage: StreamingKMeans " + + " ") + System.exit(1) + } + + val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") + val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) + + val trainingData = ssc.textFileStream(args(0)).map(Vectors.parse) + val testData = ssc.textFileStream(args(1)).map(LabeledPoint.parse) + + val model = new StreamingKMeans() + .setK(args(3).toInt) + .setDecayFactor(1.0) + .setRandomCenters(args(4).toInt, 0.0) + + model.trainOn(trainingData) + model.predictOnValues(testData.map(lp => (lp.label, lp.features))).print() + + ssc.start() + ssc.awaitTermination() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala new file mode 100644 index 0000000000000..6189dce9b27da --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -0,0 +1,268 @@ +/* + * 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.clustering + +import scala.reflect.ClassTag + +import org.apache.spark.Logging +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.StreamingContext._ +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.util.Utils +import org.apache.spark.util.random.XORShiftRandom + +/** + * :: DeveloperApi :: + * StreamingKMeansModel extends MLlib's KMeansModel for streaming + * algorithms, so it can keep track of a continuously updated weight + * associated with each cluster, and also update the model by + * doing a single iteration of the standard k-means algorithm. + * + * The update algorithm uses the "mini-batch" KMeans rule, + * generalized to incorporate forgetfullness (i.e. decay). + * The update rule (for each cluster) is: + * + * c_t+1 = [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] + * n_t+t = n_t * a + m_t + * + * Where c_t is the previously estimated centroid for that cluster, + * n_t is the number of points assigned to it thus far, x_t is the centroid + * estimated on the current batch, and m_t is the number of points assigned + * to that centroid in the current batch. + * + * The decay factor 'a' scales the contribution of the clusters as estimated thus far, + * by applying a as a discount weighting on the current point when evaluating + * new incoming data. If a=1, all batches are weighted equally. If a=0, new centroids + * are determined entirely by recent data. Lower values correspond to + * more forgetting. + * + * Decay can optionally be specified by a half life and associated + * time unit. The time unit can either be a batch of data or a single + * data point. Considering data arrived at time t, the half life h is defined + * such that at time t + h the discount applied to the data from t is 0.5. + * The definition remains the same whether the time unit is given + * as batches or points. + * + */ +@DeveloperApi +class StreamingKMeansModel( + override val clusterCenters: Array[Vector], + val clusterWeights: Array[Double]) extends KMeansModel(clusterCenters) with Logging { + + /** Perform a k-means update on a batch of data. */ + def update(data: RDD[Vector], decayFactor: Double, timeUnit: String): StreamingKMeansModel = { + + // find nearest cluster to each point + val closest = data.map(point => (this.predict(point), (point, 1L))) + + // get sums and counts for updating each cluster + val mergeContribs: ((Vector, Long), (Vector, Long)) => (Vector, Long) = (p1, p2) => { + BLAS.axpy(1.0, p2._1, p1._1) + (p1._1, p1._2 + p2._2) + } + val dim = clusterCenters(0).size + val pointStats: Array[(Int, (Vector, Long))] = closest + .aggregateByKey((Vectors.zeros(dim), 0L))(mergeContribs, mergeContribs) + .collect() + + val discount = timeUnit match { + case StreamingKMeans.BATCHES => decayFactor + case StreamingKMeans.POINTS => + val numNewPoints = pointStats.view.map { case (_, (_, n)) => + n + }.sum + math.pow(decayFactor, numNewPoints) + } + + // apply discount to weights + BLAS.scal(discount, Vectors.dense(clusterWeights)) + + // implement update rule + pointStats.foreach { case (label, (sum, count)) => + val centroid = clusterCenters(label) + + val updatedWeight = clusterWeights(label) + count + val lambda = count / math.max(updatedWeight, 1e-16) + + clusterWeights(label) = updatedWeight + BLAS.scal(1.0 - lambda, centroid) + BLAS.axpy(lambda / count, sum, centroid) + + // display the updated cluster centers + val display = clusterCenters(label).size match { + case x if x > 100 => centroid.toArray.take(100).mkString("[", ",", "...") + case _ => centroid.toArray.mkString("[", ",", "]") + } + + logInfo(s"Cluster $label updated with weight $updatedWeight and centroid: $display") + } + + // Check whether the smallest cluster is dying. If so, split the largest cluster. + val weightsWithIndex = clusterWeights.view.zipWithIndex + val (maxWeight, largest) = weightsWithIndex.maxBy(_._1) + val (minWeight, smallest) = weightsWithIndex.minBy(_._1) + if (minWeight < 1e-8 * maxWeight) { + logInfo(s"Cluster $smallest is dying. Split the largest cluster $largest into two.") + val weight = (maxWeight + minWeight) / 2.0 + clusterWeights(largest) = weight + clusterWeights(smallest) = weight + val largestClusterCenter = clusterCenters(largest) + val smallestClusterCenter = clusterCenters(smallest) + var j = 0 + while (j < dim) { + val x = largestClusterCenter(j) + val p = 1e-14 * math.max(math.abs(x), 1.0) + largestClusterCenter.toBreeze(j) = x + p + smallestClusterCenter.toBreeze(j) = x - p + j += 1 + } + } + + this + } +} + +/** + * :: DeveloperApi :: + * StreamingKMeans provides methods for configuring a + * streaming k-means analysis, training the model on streaming, + * and using the model to make predictions on streaming data. + * See KMeansModel for details on algorithm and update rules. + * + * Use a builder pattern to construct a streaming k-means analysis + * in an application, like: + * + * val model = new StreamingKMeans() + * .setDecayFactor(0.5) + * .setK(3) + * .setRandomCenters(5, 100.0) + * .trainOn(DStream) + */ +@DeveloperApi +class StreamingKMeans( + var k: Int, + var decayFactor: Double, + var timeUnit: String) extends Logging { + + def this() = this(2, 1.0, StreamingKMeans.BATCHES) + + protected var model: StreamingKMeansModel = new StreamingKMeansModel(null, null) + + /** Set the number of clusters. */ + def setK(k: Int): this.type = { + this.k = k + this + } + + /** Set the decay factor directly (for forgetful algorithms). */ + def setDecayFactor(a: Double): this.type = { + this.decayFactor = decayFactor + this + } + + /** Set the half life and time unit ("batches" or "points") for forgetful algorithms. */ + def setHalfLife(halfLife: Double, timeUnit: String): this.type = { + if (timeUnit != StreamingKMeans.BATCHES && timeUnit != StreamingKMeans.POINTS) { + throw new IllegalArgumentException("Invalid time unit for decay: " + timeUnit) + } + this.decayFactor = math.exp(math.log(0.5) / halfLife) + logInfo("Setting decay factor to: %g ".format (this.decayFactor)) + this.timeUnit = timeUnit + this + } + + /** Specify initial centers directly. */ + def setInitialCenters(centers: Array[Vector], weights: Array[Double]): this.type = { + model = new StreamingKMeansModel(centers, weights) + this + } + + /** + * Initialize random centers, requiring only the number of dimensions. + * + * @param dim Number of dimensions + * @param weight Weight for each center + * @param seed Random seed + */ + def setRandomCenters(dim: Int, weight: Double, seed: Long = Utils.random.nextLong): this.type = { + val random = new XORShiftRandom(seed) + val centers = Array.fill(k)(Vectors.dense(Array.fill(dim)(random.nextGaussian()))) + val weights = Array.fill(k)(weight) + model = new StreamingKMeansModel(centers, weights) + this + } + + /** Return the latest model. */ + def latestModel(): StreamingKMeansModel = { + model + } + + /** + * Update the clustering model by training on batches of data from a DStream. + * This operation registers a DStream for training the model, + * checks whether the cluster centers have been initialized, + * and updates the model using each batch of data from the stream. + * + * @param data DStream containing vector data + */ + def trainOn(data: DStream[Vector]) { + assertInitialized() + data.foreachRDD { (rdd, time) => + model = model.update(rdd, decayFactor, timeUnit) + } + } + + /** + * Use the clustering model to make predictions on batches of data from a DStream. + * + * @param data DStream containing vector data + * @return DStream containing predictions + */ + def predictOn(data: DStream[Vector]): DStream[Int] = { + assertInitialized() + data.map(model.predict) + } + + /** + * Use the model to make predictions on the values of a DStream and carry over its keys. + * + * @param data DStream containing (key, feature vector) pairs + * @tparam K key type + * @return DStream containing the input keys and the predictions as values + */ + def predictOnValues[K: ClassTag](data: DStream[(K, Vector)]): DStream[(K, Int)] = { + assertInitialized() + data.mapValues(model.predict) + } + + /** Check whether cluster centers have been initialized. */ + private[this] def assertInitialized(): Unit = { + if (model.clusterCenters == null) { + throw new IllegalStateException( + "Initial cluster centers must be set before starting predictions") + } + } +} + +private[clustering] object StreamingKMeans { + final val BATCHES = "batches" + final val POINTS = "points" +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala new file mode 100644 index 0000000000000..850c9fce507cd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala @@ -0,0 +1,157 @@ +/* + * 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.clustering + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.streaming.TestSuiteBase +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.util.random.XORShiftRandom + +class StreamingKMeansSuite extends FunSuite with TestSuiteBase { + + override def maxWaitTimeMillis = 30000 + + test("accuracy for single center and equivalence to grand average") { + // set parameters + val numBatches = 10 + val numPoints = 50 + val k = 1 + val d = 5 + val r = 0.1 + + // create model with one cluster + val model = new StreamingKMeans() + .setK(1) + .setDecayFactor(1.0) + .setInitialCenters(Array(Vectors.dense(0.0, 0.0, 0.0, 0.0, 0.0)), Array(0.0)) + + // generate random data for k-means + val (input, centers) = StreamingKMeansDataGenerator(numPoints, numBatches, k, d, r, 42) + + // setup and run the model training + val ssc = setupStreams(input, (inputDStream: DStream[Vector]) => { + model.trainOn(inputDStream) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) + + // estimated center should be close to true center + assert(centers(0) ~== model.latestModel().clusterCenters(0) absTol 1E-1) + + // estimated center from streaming should exactly match the arithmetic mean of all data points + // because the decay factor is set to 1.0 + val grandMean = + input.flatten.map(x => x.toBreeze).reduce(_+_) / (numBatches * numPoints).toDouble + assert(model.latestModel().clusterCenters(0) ~== Vectors.dense(grandMean.toArray) absTol 1E-5) + } + + test("accuracy for two centers") { + val numBatches = 10 + val numPoints = 5 + val k = 2 + val d = 5 + val r = 0.1 + + // create model with two clusters + val kMeans = new StreamingKMeans() + .setK(2) + .setHalfLife(2, "batches") + .setInitialCenters( + Array(Vectors.dense(-0.1, 0.1, -0.2, -0.3, -0.1), + Vectors.dense(0.1, -0.2, 0.0, 0.2, 0.1)), + Array(5.0, 5.0)) + + // generate random data for k-means + val (input, centers) = StreamingKMeansDataGenerator(numPoints, numBatches, k, d, r, 42) + + // setup and run the model training + val ssc = setupStreams(input, (inputDStream: DStream[Vector]) => { + kMeans.trainOn(inputDStream) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) + + // check that estimated centers are close to true centers + // NOTE exact assignment depends on the initialization! + assert(centers(0) ~== kMeans.latestModel().clusterCenters(0) absTol 1E-1) + assert(centers(1) ~== kMeans.latestModel().clusterCenters(1) absTol 1E-1) + } + + test("detecting dying clusters") { + val numBatches = 10 + val numPoints = 5 + val k = 1 + val d = 1 + val r = 1.0 + + // create model with two clusters + val kMeans = new StreamingKMeans() + .setK(2) + .setHalfLife(0.5, "points") + .setInitialCenters( + Array(Vectors.dense(0.0), Vectors.dense(1000.0)), + Array(1.0, 1.0)) + + // new data are all around the first cluster 0.0 + val (input, _) = + StreamingKMeansDataGenerator(numPoints, numBatches, k, d, r, 42, Array(Vectors.dense(0.0))) + + // setup and run the model training + val ssc = setupStreams(input, (inputDStream: DStream[Vector]) => { + kMeans.trainOn(inputDStream) + inputDStream.count() + }) + runStreams(ssc, numBatches, numBatches) + + // check that estimated centers are close to true centers + // NOTE exact assignment depends on the initialization! + val model = kMeans.latestModel() + val c0 = model.clusterCenters(0)(0) + val c1 = model.clusterCenters(1)(0) + + assert(c0 * c1 < 0.0, "should have one positive center and one negative center") + // 0.8 is the mean of half-normal distribution + assert(math.abs(c0) ~== 0.8 absTol 0.6) + assert(math.abs(c1) ~== 0.8 absTol 0.6) + } + + def StreamingKMeansDataGenerator( + numPoints: Int, + numBatches: Int, + k: Int, + d: Int, + r: Double, + seed: Int, + initCenters: Array[Vector] = null): (IndexedSeq[IndexedSeq[Vector]], Array[Vector]) = { + val rand = new XORShiftRandom(seed) + val centers = initCenters match { + case null => Array.fill(k)(Vectors.dense(Array.fill(d)(rand.nextGaussian()))) + case _ => initCenters + } + val data = (0 until numBatches).map { i => + (0 until numPoints).map { idx => + val center = centers(idx % k) + Vectors.dense(Array.tabulate(d)(x => center(x) + rand.nextGaussian() * r)) + } + } + (data, centers) + } +} From 680fd87c65e3e7ef223e6a1573c7afe55bff6324 Mon Sep 17 00:00:00 2001 From: Daniel Lemire Date: Sat, 1 Nov 2014 01:13:47 -0700 Subject: [PATCH 1172/1492] Upgrading to roaring 0.4.5 (bug fix release) I recommend upgrading roaring to 0.4.5 as it fixes a rarely occurring bug in iterators (that would otherwise throw an unwarranted exception). The upgrade should have no other consequence. Author: Daniel Lemire Closes #3044 from lemire/master and squashes the following commits: 54018c5 [Daniel Lemire] Recommended update to roaring 0.4.5 (bug fix release) 048933e [Daniel Lemire] Merge remote-tracking branch 'upstream/master' 431f3a0 [Daniel Lemire] Recommended bug fix release --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 42fdbb9e097db..4c7806c416e34 100644 --- a/pom.xml +++ b/pom.xml @@ -458,7 +458,7 @@ org.roaringbitmap RoaringBitmap - 0.4.3 + 0.4.5 commons-net From f4e0b28c859412ec8bdfdf452b6a1b2e1bee310e Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Sat, 1 Nov 2014 01:18:07 -0700 Subject: [PATCH 1173/1492] [SPARK-4142][GraphX] Default numEdgePartitions Changing the default number of edge partitions to match spark parallelism. Author: Joseph E. Gonzalez Closes #3006 from jegonzal/default_partitions and squashes the following commits: a9a5c4f [Joseph E. Gonzalez] Changing the default number of edge partitions to match spark parallelism --- .../org/apache/spark/examples/graphx/Analytics.scala | 6 +++--- .../scala/org/apache/spark/graphx/GraphLoader.scala | 12 +++++++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index d70d93608a57c..828cffb01ca1e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -77,7 +77,7 @@ object Analytics extends Logging { val sc = new SparkContext(conf.setAppName("PageRank(" + fname + ")")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart, + numEdgePartitions = numEPart, edgeStorageLevel = edgeStorageLevel, vertexStorageLevel = vertexStorageLevel).cache() val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) @@ -110,7 +110,7 @@ object Analytics extends Logging { val sc = new SparkContext(conf.setAppName("ConnectedComponents(" + fname + ")")) val unpartitionedGraph = GraphLoader.edgeListFile(sc, fname, - minEdgePartitions = numEPart, + numEdgePartitions = numEPart, edgeStorageLevel = edgeStorageLevel, vertexStorageLevel = vertexStorageLevel).cache() val graph = partitionStrategy.foldLeft(unpartitionedGraph)(_.partitionBy(_)) @@ -131,7 +131,7 @@ object Analytics extends Logging { val sc = new SparkContext(conf.setAppName("TriangleCount(" + fname + ")")) val graph = GraphLoader.edgeListFile(sc, fname, canonicalOrientation = true, - minEdgePartitions = numEPart, + numEdgePartitions = numEPart, edgeStorageLevel = edgeStorageLevel, vertexStorageLevel = vertexStorageLevel) // TriangleCount requires the graph to be partitioned diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index f4c79365b16da..4933aecba1286 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -48,7 +48,8 @@ object GraphLoader extends Logging { * @param path the path to the file (e.g., /home/data/file or hdfs://file) * @param canonicalOrientation whether to orient edges in the positive * direction - * @param minEdgePartitions the number of partitions for the edge RDD + * @param numEdgePartitions the number of partitions for the edge RDD + * Setting this value to -1 will use the default parallelism. * @param edgeStorageLevel the desired storage level for the edge partitions * @param vertexStorageLevel the desired storage level for the vertex partitions */ @@ -56,7 +57,7 @@ object GraphLoader extends Logging { sc: SparkContext, path: String, canonicalOrientation: Boolean = false, - minEdgePartitions: Int = 1, + numEdgePartitions: Int = -1, edgeStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY, vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) : Graph[Int, Int] = @@ -64,7 +65,12 @@ object GraphLoader extends Logging { val startTime = System.currentTimeMillis // Parse the edge data table directly into edge partitions - val lines = sc.textFile(path, minEdgePartitions).coalesce(minEdgePartitions) + val lines = + if (numEdgePartitions > 0) { + sc.textFile(path, numEdgePartitions).coalesce(numEdgePartitions) + } else { + sc.textFile(path) + } val edges = lines.mapPartitionsWithIndex { (pid, iter) => val builder = new EdgePartitionBuilder[Int, Int] iter.foreach { line => From ee29ef3800438501e0ff207feb00a28973fc0769 Mon Sep 17 00:00:00 2001 From: luluorta Date: Sat, 1 Nov 2014 01:22:46 -0700 Subject: [PATCH 1174/1492] [SPARK-4115][GraphX] Add overrided count for edge counting of EdgeRDD. Accumulate sizes of all the EdgePartitions just like the VertexRDD. Author: luluorta Closes #2975 from luluorta/graph-edge-count and squashes the following commits: 86ef0e5 [luluorta] Add overrided count for edge counting of EdgeRDD. --- .../src/main/scala/org/apache/spark/graphx/EdgeRDD.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index 5bcb96b136ed7..5267560b3e5ce 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -82,12 +82,17 @@ class EdgeRDD[@specialized ED: ClassTag, VD: ClassTag]( this } - /** Persists the vertex partitions using `targetStorageLevel`, which defaults to MEMORY_ONLY. */ + /** Persists the edge partitions using `targetStorageLevel`, which defaults to MEMORY_ONLY. */ override def cache(): this.type = { partitionsRDD.persist(targetStorageLevel) this } + /** The number of edges in the RDD. */ + override def count(): Long = { + partitionsRDD.map(_._2.size.toLong).reduce(_ + _) + } + private[graphx] def mapEdgePartitions[ED2: ClassTag, VD2: ClassTag]( f: (PartitionID, EdgePartition[ED, VD]) => EdgePartition[ED2, VD2]): EdgeRDD[ED2, VD2] = { this.withPartitionsRDD[ED2, VD2](partitionsRDD.mapPartitions({ iter => From 7136719b7d53ee1360abaa5e178ba9f8b00f3da8 Mon Sep 17 00:00:00 2001 From: Kevin Mader Date: Sat, 1 Nov 2014 11:22:46 -0700 Subject: [PATCH 1175/1492] [SPARK-2759][CORE] Generic Binary File Support in Spark The additions add the abstract BinaryFileInputFormat and BinaryRecordReader classes for reading in data as a byte stream and converting it to another format using the ```def parseByteArray(inArray: Array[Byte]): T``` function. As a trivial example ```ByteInputFormat``` and ```ByteRecordReader``` are included which just return the Array[Byte] from a given file. Finally a RDD for ```BinaryFileInputFormat``` (to allow for easier partitioning changes as was done for WholeFileInput) was added and the appropriate byteFiles to the ```SparkContext``` so the functions can be easily used by others. A common use case might be to read in a folder ``` sc.byteFiles("s3://mydrive/tif/*.tif").map(rawData => ReadTiffFromByteArray(rawData)) ``` Author: Kevin Mader Author: Kevin Mader Closes #1658 from kmader/master and squashes the following commits: 3c49a30 [Kevin Mader] fixing wholetextfileinput to it has the same setMinPartitions function as in BinaryData files 359a096 [Kevin Mader] making the final corrections suggested by @mateiz and renaming a few functions to make their usage clearer 6379be4 [Kevin Mader] reorganizing code 7b9d181 [Kevin Mader] removing developer API, cleaning up imports 8ac288b [Kevin Mader] fixed a single slightly over 100 character line 92bda0d [Kevin Mader] added new tests, renamed files, fixed several of the javaapi functions, formatted code more nicely a32fef7 [Kevin Mader] removed unneeded classes added DeveloperApi note to portabledatastreams since the implementation might change 49174d9 [Kevin Mader] removed unneeded classes added DeveloperApi note to portabledatastreams since the implementation might change c27a8f1 [Kevin Mader] jenkins crashed before running anything last time, so making minor change b348ce1 [Kevin Mader] fixed order in check (prefix only appears on jenkins not when I run unit tests locally) 0588737 [Kevin Mader] filename check in "binary file input as byte array" test now ignores prefixes and suffixes which might get added by Hadoop 4163e38 [Kevin Mader] fixing line length and output from FSDataInputStream to DataInputStream to minimize sensitivity to Hadoop API changes 19812a8 [Kevin Mader] Fixed the serialization issue with PortableDataStream since neither CombineFileSplit nor TaskAttemptContext implement the Serializable interface, by using ByteArrays for storing both and then recreating the objects from these bytearrays as needed. 238c83c [Kevin Mader] fixed several scala-style issues, changed structure of binaryFiles, removed excessive classes added new tests. The caching tests still have a serialization issue, but that should be easily fixed as well. 932a206 [Kevin Mader] Update RawFileInput.scala a01c9cf [Kevin Mader] Update RawFileInput.scala 441f79a [Kevin Mader] fixed a few small comments and dependency 12e7be1 [Kevin Mader] removing imglib from maven (definitely not ready yet) 5deb79e [Kevin Mader] added new portabledatastream to code so that it can be serialized correctly f032bc0 [Kevin Mader] fixed bug in path name, renamed tests bc5c0b9 [Kevin Mader] made minor stylistic adjustments from mateiz df8e528 [Kevin Mader] fixed line lengths and changed java test 9a313d5 [Kevin Mader] making classes that needn't be public private, adding automatic file closure, adding new tests edf5829 [Kevin Mader] fixing line lengths, adding new lines f4841dc [Kevin Mader] un-optimizing imports, silly intellij eacfaa6 [Kevin Mader] Added FixedLengthBinaryInputFormat and RecordReader from freeman-lab and added them to both the JavaSparkContext and the SparkContext as fixedLengthBinaryFile 1622935 [Kevin Mader] changing the line lengths to make jenkins happy 1cfa38a [Kevin Mader] added apache headers, added datainputstream directly as an output option for more complicated readers (HDF5 perhaps), and renamed several of the functions and files to be more consistent. Also added parallel functions to the java api 84035f1 [Kevin Mader] adding binary and byte file support spark 81c5f12 [Kevin Mader] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/SparkContext.scala | 65 +++++- .../spark/api/java/JavaSparkContext.scala | 82 ++++++- .../input/FixedLengthBinaryInputFormat.scala | 85 +++++++ .../input/FixedLengthBinaryRecordReader.scala | 126 ++++++++++ .../spark/input/PortableDataStream.scala | 218 ++++++++++++++++++ .../input/WholeTextFileInputFormat.scala | 5 +- .../org/apache/spark/rdd/BinaryFileRDD.scala | 51 ++++ .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../java/org/apache/spark/JavaAPISuite.java | 79 +++++++ .../scala/org/apache/spark/FileSuite.scala | 184 +++++++++++++++ 10 files changed, 892 insertions(+), 5 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala create mode 100644 core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala create mode 100644 core/src/main/scala/org/apache/spark/input/PortableDataStream.scala create mode 100644 core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6bfcd8ceae3c1..8b4db783979ec 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -41,7 +41,7 @@ import akka.actor.Props import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} -import org.apache.spark.input.WholeTextFileInputFormat +import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ @@ -533,6 +533,69 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { minPartitions).setName(path) } + + /** + * Get an RDD for a Hadoop-readable dataset as PortableDataStream for each file + * (useful for binary data) + * + * For example, if you have the following files: + * {{{ + * hdfs://a-hdfs-path/part-00000 + * hdfs://a-hdfs-path/part-00001 + * ... + * hdfs://a-hdfs-path/part-nnnnn + * }}} + * + * Do + * `val rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + * + * then `rdd` contains + * {{{ + * (a-hdfs-path/part-00000, its content) + * (a-hdfs-path/part-00001, its content) + * ... + * (a-hdfs-path/part-nnnnn, its content) + * }}} + * + * @param minPartitions A suggestion value of the minimal splitting number for input data. + * + * @note Small files are preferred; very large files may cause bad performance. + */ + @Experimental + def binaryFiles(path: String, minPartitions: Int = defaultMinPartitions): + RDD[(String, PortableDataStream)] = { + val job = new NewHadoopJob(hadoopConfiguration) + NewFileInputFormat.addInputPath(job, new Path(path)) + val updateConf = job.getConfiguration + new BinaryFileRDD( + this, + classOf[StreamInputFormat], + classOf[String], + classOf[PortableDataStream], + updateConf, + minPartitions).setName(path) + } + + /** + * Load data from a flat binary file, assuming the length of each record is constant. + * + * @param path Directory to the input data files + * @param recordLength The length at which to split the records + * @return An RDD of data with values, represented as byte arrays + */ + @Experimental + def binaryRecords(path: String, recordLength: Int, conf: Configuration = hadoopConfiguration) + : RDD[Array[Byte]] = { + conf.setInt(FixedLengthBinaryInputFormat.RECORD_LENGTH_PROPERTY, recordLength) + val br = newAPIHadoopFile[LongWritable, BytesWritable, FixedLengthBinaryInputFormat](path, + classOf[FixedLengthBinaryInputFormat], + classOf[LongWritable], + classOf[BytesWritable], + conf=conf) + val data = br.map{ case (k, v) => v.getBytes} + data + } + /** * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other * necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable), diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 0565adf4d4ead..e3aeba7e6c39d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -21,6 +21,11 @@ import java.io.Closeable import java.util import java.util.{Map => JMap} +import java.io.DataInputStream + +import org.apache.hadoop.io.{BytesWritable, LongWritable} +import org.apache.spark.input.{PortableDataStream, FixedLengthBinaryInputFormat} + import scala.collection.JavaConversions import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -32,7 +37,8 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ -import org.apache.spark.SparkContext.{DoubleAccumulatorParam, IntAccumulatorParam} +import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} @@ -202,6 +208,8 @@ class JavaSparkContext(val sc: SparkContext) def textFile(path: String, minPartitions: Int): JavaRDD[String] = sc.textFile(path, minPartitions) + + /** * Read a directory of text files from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI. Each file is read as a single record and returned in a @@ -245,6 +253,78 @@ class JavaSparkContext(val sc: SparkContext) def wholeTextFiles(path: String): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(path)) + /** + * Read a directory of binary files from HDFS, a local file system (available on all nodes), + * or any Hadoop-supported file system URI as a byte array. Each file is read as a single + * record and returned in a key-value pair, where the key is the path of each file, + * the value is the content of each file. + * + * For example, if you have the following files: + * {{{ + * hdfs://a-hdfs-path/part-00000 + * hdfs://a-hdfs-path/part-00001 + * ... + * hdfs://a-hdfs-path/part-nnnnn + * }}} + * + * Do + * `JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + * + * then `rdd` contains + * {{{ + * (a-hdfs-path/part-00000, its content) + * (a-hdfs-path/part-00001, its content) + * ... + * (a-hdfs-path/part-nnnnn, its content) + * }}} + * + * @note Small files are preferred; very large files but may cause bad performance. + * + * @param minPartitions A suggestion value of the minimal splitting number for input data. + */ + def binaryFiles(path: String, minPartitions: Int): JavaPairRDD[String, PortableDataStream] = + new JavaPairRDD(sc.binaryFiles(path, minPartitions)) + + /** + * Read a directory of binary files from HDFS, a local file system (available on all nodes), + * or any Hadoop-supported file system URI as a byte array. Each file is read as a single + * record and returned in a key-value pair, where the key is the path of each file, + * the value is the content of each file. + * + * For example, if you have the following files: + * {{{ + * hdfs://a-hdfs-path/part-00000 + * hdfs://a-hdfs-path/part-00001 + * ... + * hdfs://a-hdfs-path/part-nnnnn + * }}} + * + * Do + * `JavaPairRDD rdd = sparkContext.dataStreamFiles("hdfs://a-hdfs-path")`, + * + * then `rdd` contains + * {{{ + * (a-hdfs-path/part-00000, its content) + * (a-hdfs-path/part-00001, its content) + * ... + * (a-hdfs-path/part-nnnnn, its content) + * }}} + * + * @note Small files are preferred; very large files but may cause bad performance. + */ + def binaryFiles(path: String): JavaPairRDD[String, PortableDataStream] = + new JavaPairRDD(sc.binaryFiles(path, defaultMinPartitions)) + + /** + * Load data from a flat binary file, assuming the length of each record is constant. + * + * @param path Directory to the input data files + * @return An RDD of data with values, represented as byte arrays + */ + def binaryRecords(path: String, recordLength: Int): JavaRDD[Array[Byte]] = { + new JavaRDD(sc.binaryRecords(path, recordLength)) + } + /** Get an RDD for a Hadoop SequenceFile with given key and value types. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala new file mode 100644 index 0000000000000..89b29af2000c8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryInputFormat.scala @@ -0,0 +1,85 @@ +/* + * 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.input + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{BytesWritable, LongWritable} +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat +import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} + +/** + * Custom Input Format for reading and splitting flat binary files that contain records, + * each of which are a fixed size in bytes. The fixed record size is specified through + * a parameter recordLength in the Hadoop configuration. + */ +private[spark] object FixedLengthBinaryInputFormat { + /** Property name to set in Hadoop JobConfs for record length */ + val RECORD_LENGTH_PROPERTY = "org.apache.spark.input.FixedLengthBinaryInputFormat.recordLength" + + /** Retrieves the record length property from a Hadoop configuration */ + def getRecordLength(context: JobContext): Int = { + context.getConfiguration.get(RECORD_LENGTH_PROPERTY).toInt + } +} + +private[spark] class FixedLengthBinaryInputFormat + extends FileInputFormat[LongWritable, BytesWritable] { + + private var recordLength = -1 + + /** + * Override of isSplitable to ensure initial computation of the record length + */ + override def isSplitable(context: JobContext, filename: Path): Boolean = { + if (recordLength == -1) { + recordLength = FixedLengthBinaryInputFormat.getRecordLength(context) + } + if (recordLength <= 0) { + println("record length is less than 0, file cannot be split") + false + } else { + true + } + } + + /** + * This input format overrides computeSplitSize() to make sure that each split + * only contains full records. Each InputSplit passed to FixedLengthBinaryRecordReader + * will start at the first byte of a record, and the last byte will the last byte of a record. + */ + override def computeSplitSize(blockSize: Long, minSize: Long, maxSize: Long): Long = { + val defaultSize = super.computeSplitSize(blockSize, minSize, maxSize) + // If the default size is less than the length of a record, make it equal to it + // Otherwise, make sure the split size is as close to possible as the default size, + // but still contains a complete set of records, with the first record + // starting at the first byte in the split and the last record ending with the last byte + if (defaultSize < recordLength) { + recordLength.toLong + } else { + (Math.floor(defaultSize / recordLength) * recordLength).toLong + } + } + + /** + * Create a FixedLengthBinaryRecordReader + */ + override def createRecordReader(split: InputSplit, context: TaskAttemptContext) + : RecordReader[LongWritable, BytesWritable] = { + new FixedLengthBinaryRecordReader + } +} diff --git a/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala new file mode 100644 index 0000000000000..5164a74bec4e9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/FixedLengthBinaryRecordReader.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.input + +import java.io.IOException + +import org.apache.hadoop.fs.FSDataInputStream +import org.apache.hadoop.io.compress.CompressionCodecFactory +import org.apache.hadoop.io.{BytesWritable, LongWritable} +import org.apache.hadoop.mapreduce.{InputSplit, RecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.FileSplit + +/** + * FixedLengthBinaryRecordReader is returned by FixedLengthBinaryInputFormat. + * It uses the record length set in FixedLengthBinaryInputFormat to + * read one record at a time from the given InputSplit. + * + * Each call to nextKeyValue() updates the LongWritable key and BytesWritable value. + * + * key = record index (Long) + * value = the record itself (BytesWritable) + */ +private[spark] class FixedLengthBinaryRecordReader + extends RecordReader[LongWritable, BytesWritable] { + + private var splitStart: Long = 0L + private var splitEnd: Long = 0L + private var currentPosition: Long = 0L + private var recordLength: Int = 0 + private var fileInputStream: FSDataInputStream = null + private var recordKey: LongWritable = null + private var recordValue: BytesWritable = null + + override def close() { + if (fileInputStream != null) { + fileInputStream.close() + } + } + + override def getCurrentKey: LongWritable = { + recordKey + } + + override def getCurrentValue: BytesWritable = { + recordValue + } + + override def getProgress: Float = { + splitStart match { + case x if x == splitEnd => 0.0.toFloat + case _ => Math.min( + ((currentPosition - splitStart) / (splitEnd - splitStart)).toFloat, 1.0 + ).toFloat + } + } + + override def initialize(inputSplit: InputSplit, context: TaskAttemptContext) { + // the file input + val fileSplit = inputSplit.asInstanceOf[FileSplit] + + // the byte position this fileSplit starts at + splitStart = fileSplit.getStart + + // splitEnd byte marker that the fileSplit ends at + splitEnd = splitStart + fileSplit.getLength + + // the actual file we will be reading from + val file = fileSplit.getPath + // job configuration + val job = context.getConfiguration + // check compression + val codec = new CompressionCodecFactory(job).getCodec(file) + if (codec != null) { + throw new IOException("FixedLengthRecordReader does not support reading compressed files") + } + // get the record length + recordLength = FixedLengthBinaryInputFormat.getRecordLength(context) + // get the filesystem + val fs = file.getFileSystem(job) + // open the File + fileInputStream = fs.open(file) + // seek to the splitStart position + fileInputStream.seek(splitStart) + // set our current position + currentPosition = splitStart + } + + override def nextKeyValue(): Boolean = { + if (recordKey == null) { + recordKey = new LongWritable() + } + // the key is a linear index of the record, given by the + // position the record starts divided by the record length + recordKey.set(currentPosition / recordLength) + // the recordValue to place the bytes into + if (recordValue == null) { + recordValue = new BytesWritable(new Array[Byte](recordLength)) + } + // read a record if the currentPosition is less than the split end + if (currentPosition < splitEnd) { + // setup a buffer to store the record + val buffer = recordValue.getBytes + fileInputStream.read(buffer, 0, recordLength) + // update our current position + currentPosition = currentPosition + recordLength + // return true + return true + } + false + } +} diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala new file mode 100644 index 0000000000000..457472547fcbb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -0,0 +1,218 @@ +/* + * 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.input + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import scala.collection.JavaConversions._ + +import com.google.common.io.ByteStreams +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{InputSplit, JobContext, RecordReader, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.input.{CombineFileInputFormat, CombineFileRecordReader, CombineFileSplit} + +import org.apache.spark.annotation.Experimental + +/** + * A general format for reading whole files in as streams, byte arrays, + * or other functions to be added + */ +private[spark] abstract class StreamFileInputFormat[T] + extends CombineFileInputFormat[String, T] +{ + override protected def isSplitable(context: JobContext, file: Path): Boolean = false + + /** + * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API + * which is set through setMaxSplitSize + */ + def setMinPartitions(context: JobContext, minPartitions: Int) { + val files = listStatus(context) + val totalLen = files.map { file => + if (file.isDir) 0L else file.getLen + }.sum + + val maxSplitSize = Math.ceil(totalLen * 1.0 / files.length).toLong + super.setMaxSplitSize(maxSplitSize) + } + + def createRecordReader(split: InputSplit, taContext: TaskAttemptContext): RecordReader[String, T] + +} + +/** + * An abstract class of [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] + * to reading files out as streams + */ +private[spark] abstract class StreamBasedRecordReader[T]( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends RecordReader[String, T] { + + // True means the current file has been processed, then skip it. + private var processed = false + + private var key = "" + private var value: T = null.asInstanceOf[T] + + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + override def close() = {} + + override def getProgress = if (processed) 1.0f else 0.0f + + override def getCurrentKey = key + + override def getCurrentValue = value + + override def nextKeyValue = { + if (!processed) { + val fileIn = new PortableDataStream(split, context, index) + value = parseStream(fileIn) + fileIn.close() // if it has not been open yet, close does nothing + key = fileIn.getPath + processed = true + true + } else { + false + } + } + + /** + * Parse the stream (and close it afterwards) and return the value as in type T + * @param inStream the stream to be read in + * @return the data formatted as + */ + def parseStream(inStream: PortableDataStream): T +} + +/** + * Reads the record in directly as a stream for other objects to manipulate and handle + */ +private[spark] class StreamRecordReader( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends StreamBasedRecordReader[PortableDataStream](split, context, index) { + + def parseStream(inStream: PortableDataStream): PortableDataStream = inStream +} + +/** + * The format for the PortableDataStream files + */ +private[spark] class StreamInputFormat extends StreamFileInputFormat[PortableDataStream] { + override def createRecordReader(split: InputSplit, taContext: TaskAttemptContext) = { + new CombineFileRecordReader[String, PortableDataStream]( + split.asInstanceOf[CombineFileSplit], taContext, classOf[StreamRecordReader]) + } +} + +/** + * A class that allows DataStreams to be serialized and moved around by not creating them + * until they need to be read + * @note TaskAttemptContext is not serializable resulting in the confBytes construct + * @note CombineFileSplit is not serializable resulting in the splitBytes construct + */ +@Experimental +class PortableDataStream( + @transient isplit: CombineFileSplit, + @transient context: TaskAttemptContext, + index: Integer) + extends Serializable { + + // transient forces file to be reopened after being serialization + // it is also used for non-serializable classes + + @transient private var fileIn: DataInputStream = null + @transient private var isOpen = false + + private val confBytes = { + val baos = new ByteArrayOutputStream() + context.getConfiguration.write(new DataOutputStream(baos)) + baos.toByteArray + } + + private val splitBytes = { + val baos = new ByteArrayOutputStream() + isplit.write(new DataOutputStream(baos)) + baos.toByteArray + } + + @transient private lazy val split = { + val bais = new ByteArrayInputStream(splitBytes) + val nsplit = new CombineFileSplit() + nsplit.readFields(new DataInputStream(bais)) + nsplit + } + + @transient private lazy val conf = { + val bais = new ByteArrayInputStream(confBytes) + val nconf = new Configuration() + nconf.readFields(new DataInputStream(bais)) + nconf + } + /** + * Calculate the path name independently of opening the file + */ + @transient private lazy val path = { + val pathp = split.getPath(index) + pathp.toString + } + + /** + * Create a new DataInputStream from the split and context + */ + def open(): DataInputStream = { + if (!isOpen) { + val pathp = split.getPath(index) + val fs = pathp.getFileSystem(conf) + fileIn = fs.open(pathp) + isOpen = true + } + fileIn + } + + /** + * Read the file as a byte array + */ + def toArray(): Array[Byte] = { + open() + val innerBuffer = ByteStreams.toByteArray(fileIn) + close() + innerBuffer + } + + /** + * Close the file (if it is currently open) + */ + def close() = { + if (isOpen) { + try { + fileIn.close() + isOpen = false + } catch { + case ioe: java.io.IOException => // do nothing + } + } + } + + def getPath(): String = path +} + diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index 4cb450577796a..183bce3d8d8d3 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -48,9 +48,10 @@ private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[Str } /** - * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API. + * Allow minPartitions set by end-user in order to keep compatibility with old Hadoop API, + * which is set through setMaxSplitSize */ - def setMaxSplitSize(context: JobContext, minPartitions: Int) { + def setMinPartitions(context: JobContext, minPartitions: Int) { val files = listStatus(context) val totalLen = files.map { file => if (file.isDir) 0L else file.getLen diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala new file mode 100644 index 0000000000000..6e66ddbdef788 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -0,0 +1,51 @@ +/* + * 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.rdd + +import org.apache.hadoop.conf.{ Configurable, Configuration } +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapreduce._ +import org.apache.spark.input.StreamFileInputFormat +import org.apache.spark.{ Partition, SparkContext } + +private[spark] class BinaryFileRDD[T]( + sc: SparkContext, + inputFormatClass: Class[_ <: StreamFileInputFormat[T]], + keyClass: Class[String], + valueClass: Class[T], + @transient conf: Configuration, + minPartitions: Int) + extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { + + override def getPartitions: Array[Partition] = { + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = newJobContext(conf, jobId) + inputFormat.setMinPartitions(jobContext, minPartitions) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[Partition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } +} diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 324563248793c..6d6b86721ca74 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -263,7 +263,7 @@ private[spark] class WholeTextFileRDD( case _ => } val jobContext = newJobContext(conf, jobId) - inputFormat.setMaxSplitSize(jobContext, minPartitions) + inputFormat.setMinPartitions(jobContext, minPartitions) val rawSplits = inputFormat.getSplits(jobContext).toArray val result = new Array[Partition](rawSplits.size) for (i <- 0 until rawSplits.size) { diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index c21a4b30d7726..59c86eecac5e8 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,10 +18,13 @@ package org.apache.spark; import java.io.*; +import java.nio.channels.FileChannel; +import java.nio.ByteBuffer; import java.net.URI; import java.util.*; import java.util.concurrent.*; +import org.apache.spark.input.PortableDataStream; import scala.Tuple2; import scala.Tuple3; import scala.Tuple4; @@ -863,6 +866,82 @@ public Tuple2 call(Tuple2 pair) { Assert.assertEquals(pairs, readRDD.collect()); } + @Test + public void binaryFiles() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + channel1.write(bbuf); + channel1.close(); + JavaPairRDD readRDD = sc.binaryFiles(tempDirName, 3); + List> result = readRDD.collect(); + for (Tuple2 res : result) { + Assert.assertArrayEquals(content1, res._2().toArray()); + } + } + + @Test + public void binaryFilesCaching() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + channel1.write(bbuf); + channel1.close(); + + JavaPairRDD readRDD = sc.binaryFiles(tempDirName).cache(); + readRDD.foreach(new VoidFunction>() { + @Override + public void call(Tuple2 pair) throws Exception { + pair._2().toArray(); // force the file to read + } + }); + + List> result = readRDD.collect(); + for (Tuple2 res : result) { + Assert.assertArrayEquals(content1, res._2().toArray()); + } + } + + @Test + public void binaryRecords() throws Exception { + // Reusing the wholeText files example + byte[] content1 = "spark isn't always easy to use.\n".getBytes("utf-8"); + int numOfCopies = 10; + String tempDirName = tempDir.getAbsolutePath(); + File file1 = new File(tempDirName + "/part-00000"); + + FileOutputStream fos1 = new FileOutputStream(file1); + + FileChannel channel1 = fos1.getChannel(); + + for (int i = 0; i < numOfCopies; i++) { + ByteBuffer bbuf = java.nio.ByteBuffer.wrap(content1); + channel1.write(bbuf); + } + channel1.close(); + + JavaRDD readRDD = sc.binaryRecords(tempDirName, content1.length); + Assert.assertEquals(numOfCopies,readRDD.count()); + List result = readRDD.collect(); + for (byte[] res : result) { + Assert.assertArrayEquals(content1, res); + } + } + @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index a2b74c4419d46..5e24196101fbc 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -19,6 +19,9 @@ package org.apache.spark import java.io.{File, FileWriter} +import org.apache.spark.input.PortableDataStream +import org.apache.spark.storage.StorageLevel + import scala.io.Source import org.apache.hadoop.io._ @@ -224,6 +227,187 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } + test("binary file input as byte array") { + sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName) + val (infile: String, indata: PortableDataStream) = inRdd.collect.head + + // Make sure the name and array match + assert(infile.contains(outFileName)) // a prefix may get added + assert(indata.toArray === testOutput) + } + + test("portabledatastream caching tests") { + sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName).cache() + inRdd.foreach{ + curData: (String, PortableDataStream) => + curData._2.toArray() // force the file to read + } + val mappedRdd = inRdd.map { + curData: (String, PortableDataStream) => + (curData._2.getPath(), curData._2) + } + val (infile: String, indata: PortableDataStream) = mappedRdd.collect.head + + // Try reading the output back as an object file + + assert(indata.toArray === testOutput) + } + + test("portabledatastream persist disk storage") { + sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName).persist(StorageLevel.DISK_ONLY) + inRdd.foreach{ + curData: (String, PortableDataStream) => + curData._2.toArray() // force the file to read + } + val mappedRdd = inRdd.map { + curData: (String, PortableDataStream) => + (curData._2.getPath(), curData._2) + } + val (infile: String, indata: PortableDataStream) = mappedRdd.collect.head + + // Try reading the output back as an object file + + assert(indata.toArray === testOutput) + } + + test("portabledatastream flatmap tests") { + sc = new SparkContext("local", "test") + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val numOfCopies = 3 + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + channel.write(bbuf) + channel.close() + file.close() + + val inRdd = sc.binaryFiles(outFileName) + val mappedRdd = inRdd.map { + curData: (String, PortableDataStream) => + (curData._2.getPath(), curData._2) + } + val copyRdd = mappedRdd.flatMap { + curData: (String, PortableDataStream) => + for(i <- 1 to numOfCopies) yield (i, curData._2) + } + + val copyArr: Array[(Int, PortableDataStream)] = copyRdd.collect() + + // Try reading the output back as an object file + assert(copyArr.length == numOfCopies) + copyArr.foreach{ + cEntry: (Int, PortableDataStream) => + assert(cEntry._2.toArray === testOutput) + } + + } + + test("fixed record length binary file as byte array") { + // a fixed length of 6 bytes + + sc = new SparkContext("local", "test") + + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val testOutputCopies = 10 + + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + for(i <- 1 to testOutputCopies) { + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + channel.write(bbuf) + } + channel.close() + file.close() + + val inRdd = sc.binaryRecords(outFileName, testOutput.length) + // make sure there are enough elements + assert(inRdd.count == testOutputCopies) + + // now just compare the first one + val indata: Array[Byte] = inRdd.collect.head + assert(indata === testOutput) + } + + test ("negative binary record length should raise an exception") { + // a fixed length of 6 bytes + sc = new SparkContext("local", "test") + + val outFile = new File(tempDir, "record-bytestream-00000.bin") + val outFileName = outFile.getAbsolutePath() + + // create file + val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) + val testOutputCopies = 10 + + // write data to file + val file = new java.io.FileOutputStream(outFile) + val channel = file.getChannel + for(i <- 1 to testOutputCopies) { + val bbuf = java.nio.ByteBuffer.wrap(testOutput) + channel.write(bbuf) + } + channel.close() + file.close() + + val inRdd = sc.binaryRecords(outFileName, -1) + + intercept[SparkException] { + inRdd.count + } + } + test("file caching") { sc = new SparkContext("local", "test") val out = new FileWriter(tempDir + "/input") From 59e626c701227634336110e1bc23afd94c535ede Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sat, 1 Nov 2014 13:15:24 -0700 Subject: [PATCH 1176/1492] [SPARK-4183] Enable NettyBlockTransferService by default Note that we're turning this on for at least the first part of the QA period as a trial. We want to enable this (and deprecate the NioBlockTransferService) as soon as possible in the hopes that NettyBlockTransferService will be more stable and easier to maintain. We will turn it off if we run into major issues. Author: Aaron Davidson Closes #3049 from aarondav/enable-netty and squashes the following commits: bb981cc [Aaron Davidson] [SPARK-4183] Enable NettyBlockTransferService by default --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- docs/configuration.md | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 557d2f51283ae..16c5d6648d828 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -274,7 +274,7 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) val blockTransferService = - conf.get("spark.shuffle.blockTransferService", "nio").toLowerCase match { + conf.get("spark.shuffle.blockTransferService", "netty").toLowerCase match { case "netty" => new NettyBlockTransferService(conf) case "nio" => diff --git a/docs/configuration.md b/docs/configuration.md index 3007706a2586e..78c4bf332c5d1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -359,6 +359,16 @@ Apart from these, the following properties are also available, and may be useful map-side aggregation and there are at most this many reduce partitions.
    spark.shuffle.blockTransferServicenetty + Implementation to use for transferring shuffle and cached blocks between executors. There + are two implementations available: netty and nio. Netty-based + block transfer is intended to be simpler but equally efficient and is the default option + starting in 1.2. +
    #### Spark UI From 1d4f3552037cb667971bea2e5078d8b3ce6c2eae Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 1 Nov 2014 14:37:00 -0700 Subject: [PATCH 1177/1492] [SPARK-3569][SQL] Add metadata field to StructField Add `metadata: Metadata` to `StructField` to store extra information of columns. `Metadata` is a simple wrapper over `Map[String, Any]` with value types restricted to Boolean, Long, Double, String, Metadata, and arrays of those types. SerDe is via JSON. Metadata is preserved through simple operations like `SELECT`. marmbrus liancheng Author: Xiangrui Meng Author: Michael Armbrust Closes #2701 from mengxr/structfield-metadata and squashes the following commits: dedda56 [Xiangrui Meng] merge remote 5ef930a [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata c35203f [Xiangrui Meng] Merge pull request #1 from marmbrus/pr/2701 886b85c [Michael Armbrust] Expose Metadata and MetadataBuilder through the public scala and java packages. 589f314 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata 1e2abcf [Xiangrui Meng] change default value of metadata to None in python 611d3c2 [Xiangrui Meng] move metadata from Expr to NamedExpr ddfcfad [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata a438440 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata 4266f4d [Xiangrui Meng] add StructField.toString back for backward compatibility 3f49aab [Xiangrui Meng] remove StructField.toString 24a9f80 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata 473a7c5 [Xiangrui Meng] merge master c9d7301 [Xiangrui Meng] organize imports 1fcbf13 [Xiangrui Meng] change metadata type in StructField for Scala/Java 60cc131 [Xiangrui Meng] add doc and header 60614c7 [Xiangrui Meng] add metadata e42c452 [Xiangrui Meng] merge master 93518fb [Xiangrui Meng] support metadata in python 905bb89 [Xiangrui Meng] java conversions 618e349 [Xiangrui Meng] make tests work in scala 61b8e0f [Xiangrui Meng] merge master 7e5a322 [Xiangrui Meng] do not output metadata in StructField.toString c41a664 [Xiangrui Meng] merge master d8af0ed [Xiangrui Meng] move tests to SQLQuerySuite 67fdebb [Xiangrui Meng] add test on join d65072e [Xiangrui Meng] remove Map.empty 367d237 [Xiangrui Meng] add test c194d5e [Xiangrui Meng] add metadata field to StructField and Attribute --- python/pyspark/sql.py | 15 +- .../spark/sql/catalyst/ScalaReflection.scala | 2 +- .../sql/catalyst/expressions/Expression.scala | 1 + .../sql/catalyst/expressions/generators.scala | 2 +- .../expressions/namedExpressions.scala | 31 ++- .../spark/sql/catalyst/types/dataTypes.scala | 25 +- .../spark/sql/catalyst/util/Metadata.scala | 255 ++++++++++++++++++ .../sql/catalyst/util/MetadataSuite.scala | 82 ++++++ .../apache/spark/sql/api/java/DataType.java | 25 +- .../apache/spark/sql/api/java/Metadata.java | 31 +++ .../spark/sql/api/java/MetadataBuilder.java | 28 ++ .../spark/sql/api/java/StructField.java | 19 +- .../org/apache/spark/sql/SQLContext.scala | 1 - .../org/apache/spark/sql/json/JsonRDD.scala | 13 +- .../scala/org/apache/spark/sql/package.scala | 23 ++ .../sql/types/util/DataTypeConversions.scala | 8 +- .../org/apache/spark/sql/DataTypeSuite.scala | 8 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 37 ++- .../scala/org/apache/spark/sql/TestData.scala | 11 + .../ScalaSideDataTypeConversionSuite.scala | 12 +- 20 files changed, 573 insertions(+), 56 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index f0bd3cbd985da..93bfc25bca855 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -313,12 +313,15 @@ class StructField(DataType): """ - def __init__(self, name, dataType, nullable): + def __init__(self, name, dataType, nullable, metadata=None): """Creates a StructField :param name: the name of this field. :param dataType: the data type of this field. :param nullable: indicates whether values of this field can be null. + :param metadata: metadata of this field, which is a map from string + to simple type that can be serialized to JSON + automatically >>> (StructField("f1", StringType, True) ... == StructField("f1", StringType, True)) @@ -330,6 +333,7 @@ def __init__(self, name, dataType, nullable): self.name = name self.dataType = dataType self.nullable = nullable + self.metadata = metadata or {} def __repr__(self): return "StructField(%s,%s,%s)" % (self.name, self.dataType, @@ -338,13 +342,15 @@ def __repr__(self): def jsonValue(self): return {"name": self.name, "type": self.dataType.jsonValue(), - "nullable": self.nullable} + "nullable": self.nullable, + "metadata": self.metadata} @classmethod def fromJson(cls, json): return StructField(json["name"], _parse_datatype_json_value(json["type"]), - json["nullable"]) + json["nullable"], + json["metadata"]) class StructType(DataType): @@ -423,7 +429,8 @@ def _parse_datatype_json_string(json_string): ... StructField("simpleArray", simple_arraytype, True), ... StructField("simpleMap", simple_maptype, True), ... StructField("simpleStruct", simple_structtype, True), - ... StructField("boolean", BooleanType(), False)]) + ... StructField("boolean", BooleanType(), False), + ... StructField("withMeta", DoubleType(), False, {"name": "age"})]) >>> check_datatype(complex_structtype) True >>> # Complex ArrayType. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index d76c743d3f652..75923d9e8d729 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -46,7 +46,7 @@ object ScalaReflection { /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { case Schema(s: StructType, _) => - s.fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) + s.fields.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) } /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 1eb260efa6387..39b120e8de485 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType} +import org.apache.spark.sql.catalyst.util.Metadata abstract class Expression extends TreeNode[Expression] { self: Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 9c865254e0be9..ab0701fd9a80b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -43,7 +43,7 @@ abstract class Generator extends Expression { override type EvaluatedType = TraversableOnce[Row] override lazy val dataType = - ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable)))) + ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata)))) override def nullable = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index fe13a661f6f7a..3310566087b3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.util.Metadata object NamedExpression { private val curId = new java.util.concurrent.atomic.AtomicLong() @@ -43,6 +44,9 @@ abstract class NamedExpression extends Expression { def toAttribute: Attribute + /** Returns the metadata when an expression is a reference to another expression with metadata. */ + def metadata: Metadata = Metadata.empty + protected def typeSuffix = if (resolved) { dataType match { @@ -88,10 +92,16 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable + override def metadata: Metadata = { + child match { + case named: NamedExpression => named.metadata + case _ => Metadata.empty + } + } override def toAttribute = { if (resolved) { - AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) + AttributeReference(name, child.dataType, child.nullable, metadata)(exprId, qualifiers) } else { UnresolvedAttribute(name) } @@ -108,15 +118,20 @@ case class Alias(child: Expression, name: String) * @param name The name of this attribute, should only be used during analysis or for debugging. * @param dataType The [[DataType]] of this attribute. * @param nullable True if null is a valid value for this attribute. + * @param metadata The metadata of this attribute. * @param exprId A globally unique id used to check if different AttributeReferences refer to the * same attribute. * @param qualifiers a list of strings that can be used to referred to this attribute in a fully * qualified way. Consider the examples tableName.name, subQueryAlias.name. * tableName and subQueryAlias are possible qualifiers. */ -case class AttributeReference(name: String, dataType: DataType, nullable: Boolean = true) - (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) - extends Attribute with trees.LeafNode[Expression] { +case class AttributeReference( + name: String, + dataType: DataType, + nullable: Boolean = true, + override val metadata: Metadata = Metadata.empty)( + val exprId: ExprId = NamedExpression.newExprId, + val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType @@ -128,10 +143,12 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea var h = 17 h = h * 37 + exprId.hashCode() h = h * 37 + dataType.hashCode() + h = h * 37 + metadata.hashCode() h } - override def newInstance() = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) + override def newInstance() = + AttributeReference(name, dataType, nullable, metadata)(qualifiers = qualifiers) /** * Returns a copy of this [[AttributeReference]] with changed nullability. @@ -140,7 +157,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea if (nullable == newNullability) { this } else { - AttributeReference(name, dataType, newNullability)(exprId, qualifiers) + AttributeReference(name, dataType, newNullability, metadata)(exprId, qualifiers) } } @@ -159,7 +176,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea if (newQualifiers.toSet == qualifiers.toSet) { this } else { - AttributeReference(name, dataType, nullable)(exprId, newQualifiers) + AttributeReference(name, dataType, nullable, metadata)(exprId, newQualifiers) } } 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 4e6e1166bfffb..6069f9b0a68dc 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 @@ -24,16 +24,16 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} import scala.util.parsing.combinator.RegexParsers -import org.json4s.JsonAST.JValue import org.json4s._ +import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.catalyst.util.Metadata import org.apache.spark.util.Utils - object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) @@ -70,10 +70,11 @@ object DataType { private def parseStructField(json: JValue): StructField = json match { case JSortedObject( + ("metadata", metadata: JObject), ("name", JString(name)), ("nullable", JBool(nullable)), ("type", dataType: JValue)) => - StructField(name, parseDataType(dataType), nullable) + StructField(name, parseDataType(dataType), nullable, Metadata.fromJObject(metadata)) } @deprecated("Use DataType.fromJson instead", "1.2.0") @@ -388,24 +389,34 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT * @param name The name of this field. * @param dataType The data type of this field. * @param nullable Indicates if values of this field can be `null` values. + * @param metadata The metadata of this field. The metadata should be preserved during + * transformation if the content of the column is not modified, e.g, in selection. */ -case class StructField(name: String, dataType: DataType, nullable: Boolean) { +case class StructField( + name: String, + dataType: DataType, + nullable: Boolean, + metadata: Metadata = Metadata.empty) { private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { builder.append(s"$prefix-- $name: ${dataType.typeName} (nullable = $nullable)\n") DataType.buildFormattedString(dataType, s"$prefix |", builder) } + // override the default toString to be compatible with legacy parquet files. + override def toString: String = s"StructField($name,$dataType,$nullable)" + private[sql] def jsonValue: JValue = { ("name" -> name) ~ ("type" -> dataType.jsonValue) ~ - ("nullable" -> nullable) + ("nullable" -> nullable) ~ + ("metadata" -> metadata.jsonValue) } } object StructType { protected[sql] def fromAttributes(attributes: Seq[Attribute]): StructType = - StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable))) + StructType(attributes.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) } case class StructType(fields: Seq[StructField]) extends DataType { @@ -439,7 +450,7 @@ case class StructType(fields: Seq[StructField]) extends DataType { } protected[sql] def toAttributes = - fields.map(f => AttributeReference(f.name, f.dataType, f.nullable)()) + fields.map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) def treeString: String = { val builder = new StringBuilder diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala new file mode 100644 index 0000000000000..2f2082fa3c863 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/Metadata.scala @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import scala.collection.mutable + +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + +/** + * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, + * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and + * Array[Metadata]. JSON is used for serialization. + * + * The default constructor is private. User should use either [[MetadataBuilder]] or + * [[Metadata$#fromJson]] to create Metadata instances. + * + * @param map an immutable map that stores the data + */ +sealed class Metadata private[util] (private[util] val map: Map[String, Any]) extends Serializable { + + /** Gets a Long. */ + def getLong(key: String): Long = get(key) + + /** Gets a Double. */ + def getDouble(key: String): Double = get(key) + + /** Gets a Boolean. */ + def getBoolean(key: String): Boolean = get(key) + + /** Gets a String. */ + def getString(key: String): String = get(key) + + /** Gets a Metadata. */ + def getMetadata(key: String): Metadata = get(key) + + /** Gets a Long array. */ + def getLongArray(key: String): Array[Long] = get(key) + + /** Gets a Double array. */ + def getDoubleArray(key: String): Array[Double] = get(key) + + /** Gets a Boolean array. */ + def getBooleanArray(key: String): Array[Boolean] = get(key) + + /** Gets a String array. */ + def getStringArray(key: String): Array[String] = get(key) + + /** Gets a Metadata array. */ + def getMetadataArray(key: String): Array[Metadata] = get(key) + + /** Converts to its JSON representation. */ + def json: String = compact(render(jsonValue)) + + override def toString: String = json + + override def equals(obj: Any): Boolean = { + obj match { + case that: Metadata => + if (map.keySet == that.map.keySet) { + map.keys.forall { k => + (map(k), that.map(k)) match { + case (v0: Array[_], v1: Array[_]) => + v0.view == v1.view + case (v0, v1) => + v0 == v1 + } + } + } else { + false + } + case other => + false + } + } + + override def hashCode: Int = Metadata.hash(this) + + private def get[T](key: String): T = { + map(key).asInstanceOf[T] + } + + private[sql] def jsonValue: JValue = Metadata.toJsonValue(this) +} + +object Metadata { + + /** Returns an empty Metadata. */ + def empty: Metadata = new Metadata(Map.empty) + + /** Creates a Metadata instance from JSON. */ + def fromJson(json: String): Metadata = { + fromJObject(parse(json).asInstanceOf[JObject]) + } + + /** Creates a Metadata instance from JSON AST. */ + private[sql] def fromJObject(jObj: JObject): Metadata = { + val builder = new MetadataBuilder + jObj.obj.foreach { + case (key, JInt(value)) => + builder.putLong(key, value.toLong) + case (key, JDouble(value)) => + builder.putDouble(key, value) + case (key, JBool(value)) => + builder.putBoolean(key, value) + case (key, JString(value)) => + builder.putString(key, value) + case (key, o: JObject) => + builder.putMetadata(key, fromJObject(o)) + case (key, JArray(value)) => + if (value.isEmpty) { + // If it is an empty array, we cannot infer its element type. We put an empty Array[Long]. + builder.putLongArray(key, Array.empty) + } else { + value.head match { + case _: JInt => + builder.putLongArray(key, value.asInstanceOf[List[JInt]].map(_.num.toLong).toArray) + case _: JDouble => + builder.putDoubleArray(key, value.asInstanceOf[List[JDouble]].map(_.num).toArray) + case _: JBool => + builder.putBooleanArray(key, value.asInstanceOf[List[JBool]].map(_.value).toArray) + case _: JString => + builder.putStringArray(key, value.asInstanceOf[List[JString]].map(_.s).toArray) + case _: JObject => + builder.putMetadataArray( + key, value.asInstanceOf[List[JObject]].map(fromJObject).toArray) + case other => + throw new RuntimeException(s"Do not support array of type ${other.getClass}.") + } + } + case other => + throw new RuntimeException(s"Do not support type ${other.getClass}.") + } + builder.build() + } + + /** Converts to JSON AST. */ + private def toJsonValue(obj: Any): JValue = { + obj match { + case map: Map[_, _] => + val fields = map.toList.map { case (k: String, v) => (k, toJsonValue(v)) } + JObject(fields) + case arr: Array[_] => + val values = arr.toList.map(toJsonValue) + JArray(values) + case x: Long => + JInt(x) + case x: Double => + JDouble(x) + case x: Boolean => + JBool(x) + case x: String => + JString(x) + case x: Metadata => + toJsonValue(x.map) + case other => + throw new RuntimeException(s"Do not support type ${other.getClass}.") + } + } + + /** Computes the hash code for the types we support. */ + private def hash(obj: Any): Int = { + obj match { + case map: Map[_, _] => + map.mapValues(hash).## + case arr: Array[_] => + // Seq.empty[T] has the same hashCode regardless of T. + arr.toSeq.map(hash).## + case x: Long => + x.## + case x: Double => + x.## + case x: Boolean => + x.## + case x: String => + x.## + case x: Metadata => + hash(x.map) + case other => + throw new RuntimeException(s"Do not support type ${other.getClass}.") + } + } +} + +/** + * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. + */ +class MetadataBuilder { + + private val map: mutable.Map[String, Any] = mutable.Map.empty + + /** Returns the immutable version of this map. Used for java interop. */ + protected def getMap = map.toMap + + /** Include the content of an existing [[Metadata]] instance. */ + def withMetadata(metadata: Metadata): this.type = { + map ++= metadata.map + this + } + + /** Puts a Long. */ + def putLong(key: String, value: Long): this.type = put(key, value) + + /** Puts a Double. */ + def putDouble(key: String, value: Double): this.type = put(key, value) + + /** Puts a Boolean. */ + def putBoolean(key: String, value: Boolean): this.type = put(key, value) + + /** Puts a String. */ + def putString(key: String, value: String): this.type = put(key, value) + + /** Puts a [[Metadata]]. */ + def putMetadata(key: String, value: Metadata): this.type = put(key, value) + + /** Puts a Long array. */ + def putLongArray(key: String, value: Array[Long]): this.type = put(key, value) + + /** Puts a Double array. */ + def putDoubleArray(key: String, value: Array[Double]): this.type = put(key, value) + + /** Puts a Boolean array. */ + def putBooleanArray(key: String, value: Array[Boolean]): this.type = put(key, value) + + /** Puts a String array. */ + def putStringArray(key: String, value: Array[String]): this.type = put(key, value) + + /** Puts a [[Metadata]] array. */ + def putMetadataArray(key: String, value: Array[Metadata]): this.type = put(key, value) + + /** Builds the [[Metadata]] instance. */ + def build(): Metadata = { + new Metadata(map.toMap) + } + + private def put(key: String, value: Any): this.type = { + map.put(key, value) + this + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala new file mode 100644 index 0000000000000..0063d31666c85 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.json4s.jackson.JsonMethods.parse +import org.scalatest.FunSuite + +class MetadataSuite extends FunSuite { + + val baseMetadata = new MetadataBuilder() + .putString("purpose", "ml") + .putBoolean("isBase", true) + .build() + + val summary = new MetadataBuilder() + .putLong("numFeatures", 10L) + .build() + + val age = new MetadataBuilder() + .putString("name", "age") + .putLong("index", 1L) + .putBoolean("categorical", false) + .putDouble("average", 45.0) + .build() + + val gender = new MetadataBuilder() + .putString("name", "gender") + .putLong("index", 5) + .putBoolean("categorical", true) + .putStringArray("categories", Array("male", "female")) + .build() + + val metadata = new MetadataBuilder() + .withMetadata(baseMetadata) + .putBoolean("isBase", false) // overwrite an existing key + .putMetadata("summary", summary) + .putLongArray("long[]", Array(0L, 1L)) + .putDoubleArray("double[]", Array(3.0, 4.0)) + .putBooleanArray("boolean[]", Array(true, false)) + .putMetadataArray("features", Array(age, gender)) + .build() + + test("metadata builder and getters") { + assert(age.getLong("index") === 1L) + assert(age.getDouble("average") === 45.0) + assert(age.getBoolean("categorical") === false) + assert(age.getString("name") === "age") + assert(metadata.getString("purpose") === "ml") + assert(metadata.getBoolean("isBase") === false) + assert(metadata.getMetadata("summary") === summary) + assert(metadata.getLongArray("long[]").toSeq === Seq(0L, 1L)) + assert(metadata.getDoubleArray("double[]").toSeq === Seq(3.0, 4.0)) + assert(metadata.getBooleanArray("boolean[]").toSeq === Seq(true, false)) + assert(gender.getStringArray("categories").toSeq === Seq("male", "female")) + assert(metadata.getMetadataArray("features").toSeq === Seq(age, gender)) + } + + test("metadata json conversion") { + val json = metadata.json + withClue("toJson must produce a valid JSON string") { + parse(json) + } + val parsed = Metadata.fromJson(json) + assert(parsed === metadata) + assert(parsed.## === metadata.##) + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index 37e88d72b9172..0c85cdc0aa640 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -17,9 +17,7 @@ package org.apache.spark.sql.api.java; -import java.util.HashSet; -import java.util.List; -import java.util.Set; +import java.util.*; /** * The base type of all Spark SQL data types. @@ -151,15 +149,31 @@ public static MapType createMapType( * Creates a StructField by specifying the name ({@code name}), data type ({@code dataType}) and * whether values of this field can be null values ({@code nullable}). */ - public static StructField createStructField(String name, DataType dataType, boolean nullable) { + public static StructField createStructField( + String name, + DataType dataType, + boolean nullable, + Metadata metadata) { if (name == null) { throw new IllegalArgumentException("name should not be null."); } if (dataType == null) { throw new IllegalArgumentException("dataType should not be null."); } + if (metadata == null) { + throw new IllegalArgumentException("metadata should not be null."); + } + + return new StructField(name, dataType, nullable, metadata); + } - return new StructField(name, dataType, nullable); + /** + * Creates a StructField with empty metadata. + * + * @see #createStructField(String, DataType, boolean, Metadata) + */ + public static StructField createStructField(String name, DataType dataType, boolean nullable) { + return createStructField(name, dataType, nullable, (new MetadataBuilder()).build()); } /** @@ -191,5 +205,4 @@ public static StructType createStructType(StructField[] fields) { return new StructType(fields); } - } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java new file mode 100644 index 0000000000000..0f819fb01a76a --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/Metadata.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, + * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and + * Array[Metadata]. JSON is used for serialization. + * + * The default constructor is private. User should use [[MetadataBuilder]]. + */ +class Metadata extends org.apache.spark.sql.catalyst.util.Metadata { + Metadata(scala.collection.immutable.Map map) { + super(map); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java new file mode 100644 index 0000000000000..6e6b12f0722c5 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/MetadataBuilder.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +/** + * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. + */ +public class MetadataBuilder extends org.apache.spark.sql.catalyst.util.MetadataBuilder { + @Override + public Metadata build() { + return new Metadata(getMap()); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java index b48e2a2c5f953..7c60d492bcdf0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/StructField.java @@ -17,6 +17,8 @@ package org.apache.spark.sql.api.java; +import java.util.Map; + /** * A StructField object represents a field in a StructType object. * A StructField object comprises three fields, {@code String name}, {@code DataType dataType}, @@ -24,20 +26,27 @@ * The field of {@code dataType} specifies the data type of a StructField. * The field of {@code nullable} specifies if values of a StructField can contain {@code null} * values. + * The field of {@code metadata} provides extra information of the StructField. * * To create a {@link StructField}, - * {@link DataType#createStructField(String, DataType, boolean)} + * {@link DataType#createStructField(String, DataType, boolean, Metadata)} * should be used. */ public class StructField { private String name; private DataType dataType; private boolean nullable; + private Metadata metadata; - protected StructField(String name, DataType dataType, boolean nullable) { + protected StructField( + String name, + DataType dataType, + boolean nullable, + Metadata metadata) { this.name = name; this.dataType = dataType; this.nullable = nullable; + this.metadata = metadata; } public String getName() { @@ -52,6 +61,10 @@ public boolean isNullable() { return nullable; } + public Metadata getMetadata() { + return metadata; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -62,6 +75,7 @@ public boolean equals(Object o) { if (nullable != that.nullable) return false; if (!dataType.equals(that.dataType)) return false; if (!name.equals(that.name)) return false; + if (!metadata.equals(that.metadata)) return false; return true; } @@ -71,6 +85,7 @@ public int hashCode() { int result = name.hashCode(); result = 31 * result + dataType.hashCode(); result = 31 * result + (nullable ? 1 : 0); + result = 31 * result + metadata.hashCode(); return result; } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index a41a500c9a5d0..4953f8399a96b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -32,7 +32,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types.DataType import org.apache.spark.sql.execution.{SparkStrategies, _} import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 047dc85df6c1d..eabe312f92371 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -117,10 +117,7 @@ private[sql] object JsonRDD extends Logging { } }.flatMap(field => field).toSeq - StructType( - (topLevelFields ++ structFields).sortBy { - case StructField(name, _, _) => name - }) + StructType((topLevelFields ++ structFields).sortBy(_.name)) } makeStruct(resolved.keySet.toSeq, Nil) @@ -128,7 +125,7 @@ private[sql] object JsonRDD extends Logging { private[sql] def nullTypeToStringType(struct: StructType): StructType = { val fields = struct.fields.map { - case StructField(fieldName, dataType, nullable) => { + case StructField(fieldName, dataType, nullable, _) => { val newType = dataType match { case NullType => StringType case ArrayType(NullType, containsNull) => ArrayType(StringType, containsNull) @@ -163,9 +160,7 @@ private[sql] object JsonRDD extends Logging { StructField(name, dataType, true) } } - StructType(newFields.toSeq.sortBy { - case StructField(name, _, _) => name - }) + StructType(newFields.toSeq.sortBy(_.name)) } case (ArrayType(elementType1, containsNull1), ArrayType(elementType2, containsNull2)) => ArrayType(compatibleType(elementType1, elementType2), containsNull1 || containsNull2) @@ -413,7 +408,7 @@ private[sql] object JsonRDD extends Logging { // TODO: Reuse the row instead of creating a new one for every record. val row = new GenericMutableRow(schema.fields.length) schema.fields.zipWithIndex.foreach { - case (StructField(name, dataType, _), i) => + case (StructField(name, dataType, _, _), i) => row.update(i, json.get(name).flatMap(v => Option(v)).map( enforceCorrectType(_, dataType)).orNull) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index e98d151286818..f0e57e2a7447b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -125,6 +125,9 @@ package object sql { @DeveloperApi type DataType = catalyst.types.DataType + @DeveloperApi + val DataType = catalyst.types.DataType + /** * :: DeveloperApi :: * @@ -414,4 +417,24 @@ package object sql { */ @DeveloperApi val StructField = catalyst.types.StructField + + /** + * :: DeveloperApi :: + * + * Metadata is a wrapper over Map[String, Any] that limits the value type to simple ones: Boolean, + * Long, Double, String, Metadata, Array[Boolean], Array[Long], Array[Double], Array[String], and + * Array[Metadata]. JSON is used for serialization. + * + * The default constructor is private. User should use either [[MetadataBuilder]] or + * [[Metadata$#fromJson]] to create Metadata instances. + * + * @param map an immutable map that stores the data + */ + @DeveloperApi + type Metadata = catalyst.util.Metadata + + /** + * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. + */ + type MetadataBuilder = catalyst.util.MetadataBuilder } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 609f7db562a31..142598c904b37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.types.util import org.apache.spark.sql._ -import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField} +import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, MetadataBuilder => JMetaDataBuilder} import scala.collection.JavaConverters._ @@ -31,7 +31,8 @@ protected[sql] object DataTypeConversions { JDataType.createStructField( scalaStructField.name, asJavaDataType(scalaStructField.dataType), - scalaStructField.nullable) + scalaStructField.nullable, + (new JMetaDataBuilder).withMetadata(scalaStructField.metadata).build()) } /** @@ -68,7 +69,8 @@ protected[sql] object DataTypeConversions { StructField( javaStructField.getName, asScalaDataType(javaStructField.getDataType), - javaStructField.isNullable) + javaStructField.isNullable, + javaStructField.getMetadata) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala index 100ecb45e9e88..6c9db639c0f6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types.DataType - class DataTypeSuite extends FunSuite { test("construct an ArrayType") { @@ -79,8 +77,12 @@ class DataTypeSuite extends FunSuite { checkDataTypeJsonRepr(ArrayType(StringType, false)) checkDataTypeJsonRepr(MapType(IntegerType, StringType, true)) checkDataTypeJsonRepr(MapType(IntegerType, ArrayType(DoubleType), false)) + val metadata = new MetadataBuilder() + .putString("name", "age") + .build() checkDataTypeJsonRepr( StructType(Seq( StructField("a", IntegerType, nullable = true), - StructField("b", ArrayType(DoubleType), nullable = false)))) + StructField("b", ArrayType(DoubleType), nullable = false), + StructField("c", DoubleType, nullable = false, metadata)))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4acd92d33d180..6befe1b755cc6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,17 +17,16 @@ package org.apache.spark.sql +import java.util.TimeZone + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.joins.BroadcastHashJoin -import org.apache.spark.sql.test._ -import org.scalatest.BeforeAndAfterAll -import java.util.TimeZone -/* Implicits */ -import TestSQLContext._ -import TestData._ +import org.apache.spark.sql.test.TestSQLContext._ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { // Make sure the tables are loaded. @@ -697,6 +696,30 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { ("true", "false") :: Nil) } + test("metadata is propagated correctly") { + val person = sql("SELECT * FROM person") + val schema = person.schema + val docKey = "doc" + val docValue = "first name" + val metadata = new MetadataBuilder() + .putString(docKey, docValue) + .build() + val schemaWithMeta = new StructType(Seq( + schema("id"), schema("name").copy(metadata = metadata), schema("age"))) + val personWithMeta = applySchema(person, schemaWithMeta) + def validateMetadata(rdd: SchemaRDD): Unit = { + assert(rdd.schema("name").metadata.getString(docKey) == docValue) + } + personWithMeta.registerTempTable("personWithMeta") + validateMetadata(personWithMeta.select('name)) + validateMetadata(personWithMeta.select("name".attr)) + validateMetadata(personWithMeta.select('id, 'name)) + validateMetadata(sql("SELECT * FROM personWithMeta")) + validateMetadata(sql("SELECT id, name FROM personWithMeta")) + validateMetadata(sql("SELECT * FROM personWithMeta JOIN salary ON id = personId")) + validateMetadata(sql("SELECT name, salary FROM personWithMeta JOIN salary ON id = personId")) + } + test("SPARK-3371 Renaming a function expression with group by gives error") { registerFunction("len", (s: String) => s.length) checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index c4dd3e860f5fd..836dd17fcc3a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -166,4 +166,15 @@ object TestData { // An RDD with 4 elements and 8 partitions val withEmptyParts = TestSQLContext.sparkContext.parallelize((1 to 4).map(IntField), 8) withEmptyParts.registerTempTable("withEmptyParts") + + case class Person(id: Int, name: String, age: Int) + case class Salary(personId: Int, salary: Double) + val person = TestSQLContext.sparkContext.parallelize( + Person(0, "mike", 30) :: + Person(1, "jim", 20) :: Nil) + person.registerTempTable("person") + val salary = TestSQLContext.sparkContext.parallelize( + Salary(0, 2000.0) :: + Salary(1, 1000.0) :: Nil) + salary.registerTempTable("salary") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala index 8415af41be3af..e0e0ff9cb3d3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.api.java -import org.apache.spark.sql.types.util.DataTypeConversions import org.scalatest.FunSuite -import org.apache.spark.sql.{DataType => SDataType, StructField => SStructField} -import org.apache.spark.sql.{StructType => SStructType} -import DataTypeConversions._ +import org.apache.spark.sql.{DataType => SDataType, StructField => SStructField, StructType => SStructType} +import org.apache.spark.sql.types.util.DataTypeConversions._ class ScalaSideDataTypeConversionSuite extends FunSuite { @@ -67,11 +65,15 @@ class ScalaSideDataTypeConversionSuite extends FunSuite { checkDataType(simpleScalaStructType) // Complex StructType. + val metadata = new MetadataBuilder() + .putString("name", "age") + .build() val complexScalaStructType = SStructType( SStructField("simpleArray", simpleScalaArrayType, true) :: SStructField("simpleMap", simpleScalaMapType, true) :: SStructField("simpleStruct", simpleScalaStructType, true) :: - SStructField("boolean", org.apache.spark.sql.BooleanType, false) :: Nil) + SStructField("boolean", org.apache.spark.sql.BooleanType, false) :: + SStructField("withMeta", org.apache.spark.sql.DoubleType, false, metadata) :: Nil) checkDataType(complexScalaStructType) // Complex ArrayType. From f55218aeb1e9d638df6229b36a59a15ce5363482 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sat, 1 Nov 2014 14:37:45 -0700 Subject: [PATCH 1178/1492] [SPARK-3796] Create external service which can serve shuffle files This patch introduces the tooling necessary to construct an external shuffle service which is independent of Spark executors, and then use this service inside Spark. An example (just for the sake of this PR) of the service creation can be found in Worker, and the service itself is used by plugging in the StandaloneShuffleClient as Spark's ShuffleClient (setup in BlockManager). This PR continues the work from #2753, which extracted out the transport layer of Spark's block transfer into an independent package within Spark. A new package was created which contains the Spark business logic necessary to retrieve the actual shuffle data, which is completely independent of the transport layer introduced in the previous patch. Similar to the transport layer, this package must not depend on Spark as we anticipate plugging this service as a lightweight process within, say, the YARN NodeManager, and do not wish to include Spark's dependencies (including Scala itself). There are several outstanding tasks which must be complete before this PR can be merged: - [x] Complete unit testing of network/shuffle package. - [x] Performance and correctness testing on a real cluster. - [x] Remove example service instantiation from Worker.scala. There are even more shortcomings of this PR which should be addressed in followup patches: - Don't use Java serializer for RPC layer! It is not cross-version compatible. - Handle shuffle file cleanup for dead executors once the application terminates or the ContextCleaner triggers. - Documentation of the feature in the Spark docs. - Improve behavior if the shuffle service itself goes down (right now we don't blacklist it, and new executors cannot spawn on that machine). - SSL and SASL integration - Nice to have: Handle shuffle file consolidation (this would requires changes to Spark's implementation). Author: Aaron Davidson Closes #3001 from aarondav/shuffle-service and squashes the following commits: 4d1f8c1 [Aaron Davidson] Remove changes to Worker 705748f [Aaron Davidson] Rename Standalone* to External* fd3928b [Aaron Davidson] Do not unregister executor outputs unduly 9883918 [Aaron Davidson] Make suggested build changes 3d62679 [Aaron Davidson] Add Spark integration test 7fe51d5 [Aaron Davidson] Fix SBT integration 56caa50 [Aaron Davidson] Address comments c8d1ac3 [Aaron Davidson] Add unit tests 2f70c0c [Aaron Davidson] Fix unit tests 5483e96 [Aaron Davidson] Fix unit tests 46a70bf [Aaron Davidson] Whoops, bracket 5ea4df6 [Aaron Davidson] [SPARK-3796] Create external service which can serve shuffle files --- core/pom.xml | 5 + .../org/apache/spark/MapOutputTracker.scala | 4 +- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../apache/spark/deploy/worker/Worker.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 2 +- .../spark/network/BlockTransferService.scala | 55 ++-- .../network/netty/NettyBlockFetcher.scala | 95 ------ .../network/netty/NettyBlockRpcServer.scala | 26 +- .../netty/NettyBlockTransferService.scala | 29 +- .../network/netty/SparkTransportConf.scala | 32 ++ .../network/nio/NioBlockTransferService.scala | 8 +- .../apache/spark/scheduler/DAGScheduler.scala | 41 ++- .../org/apache/spark/scheduler/Stage.scala | 10 + .../spark/scheduler/TaskSetManager.scala | 5 +- .../shuffle/FileShuffleBlockManager.scala | 3 +- .../shuffle/IndexShuffleBlockManager.scala | 2 + .../hash/BlockStoreShuffleFetcher.scala | 2 +- .../shuffle/hash/HashShuffleWriter.scala | 2 +- .../shuffle/sort/SortShuffleWriter.scala | 2 +- .../org/apache/spark/storage/BlockId.scala | 2 + .../apache/spark/storage/BlockManager.scala | 71 ++++- .../spark/storage/DiskBlockManager.scala | 25 +- .../storage/ShuffleBlockFetcherIterator.scala | 14 +- .../scala/org/apache/spark/util/Utils.scala | 2 + .../org/apache/spark/DistributedSuite.scala | 3 +- .../spark/ExternalShuffleServiceSuite.scala | 76 +++++ .../org/apache/spark/HashShuffleSuite.scala | 6 +- .../org/apache/spark/ShuffleNettySuite.scala | 6 +- .../scala/org/apache/spark/ShuffleSuite.scala | 34 +- .../org/apache/spark/SortShuffleSuite.scala | 6 +- .../spark/metrics/MetricsSystemSuite.scala | 4 +- .../ShuffleBlockFetcherIteratorSuite.scala | 25 +- network/common/pom.xml | 20 +- .../spark/network/TransportContext.java | 14 +- .../spark/network/client/TransportClient.java | 32 +- .../client/TransportClientFactory.java | 17 +- .../spark/network/server}/NoOpRpcHandler.java | 18 +- ...nager.java => OneForOneStreamManager.java} | 8 +- .../spark/network/server/RpcHandler.java | 6 + .../server/TransportRequestHandler.java | 9 +- .../spark/network/server/TransportServer.java | 8 +- .../apache/spark/network/util/JavaUtils.java | 38 +++ .../util}/SystemPropertyConfigProvider.java | 2 +- .../spark/network/util/TransportConf.java | 3 - .../network/ChunkFetchIntegrationSuite.java | 16 +- .../spark/network/RpcIntegrationSuite.java | 9 +- .../network/TransportClientFactorySuite.java | 7 +- network/shuffle/pom.xml | 96 ++++++ .../shuffle/BlockFetchingListener.java | 18 +- .../network/shuffle/ExecutorShuffleInfo.java | 64 ++++ .../shuffle/ExternalShuffleBlockHandler.java | 102 ++++++ .../shuffle/ExternalShuffleBlockManager.java | 154 +++++++++ .../shuffle/ExternalShuffleClient.java | 88 ++++++ .../shuffle/ExternalShuffleMessages.java | 106 +++++++ .../shuffle/OneForOneBlockFetcher.java | 121 ++++++++ .../spark/network/shuffle/ShuffleClient.java | 35 +++ .../network/shuffle/ShuffleStreamHandle.java | 60 ++++ .../ExternalShuffleBlockHandlerSuite.java | 123 ++++++++ .../ExternalShuffleBlockManagerSuite.java | 125 ++++++++ .../ExternalShuffleIntegrationSuite.java | 291 ++++++++++++++++++ .../shuffle/OneForOneBlockFetcherSuite.java | 167 ++++++++++ .../network/shuffle/ShuffleMessagesSuite.java | 51 +++ .../shuffle/TestShuffleDataContext.java | 107 +++++++ pom.xml | 1 + project/SparkBuild.scala | 11 +- 65 files changed, 2216 insertions(+), 312 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala create mode 100644 core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala create mode 100644 core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala rename network/common/src/{test/java/org/apache/spark/network => main/java/org/apache/spark/network/server}/NoOpRpcHandler.java (72%) rename network/common/src/main/java/org/apache/spark/network/server/{DefaultStreamManager.java => OneForOneStreamManager.java} (93%) rename network/common/src/{test/java/org/apache/spark/network => main/java/org/apache/spark/network/util}/SystemPropertyConfigProvider.java (96%) create mode 100644 network/shuffle/pom.xml rename core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala => network/shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java (73%) create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorShuffleInfo.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java create mode 100644 network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleStreamHandle.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleMessagesSuite.java create mode 100644 network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java diff --git a/core/pom.xml b/core/pom.xml index 6963ce4777e6f..41296e0eca330 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -49,6 +49,11 @@ spark-network-common_2.10 ${project.version} + + org.apache.spark + spark-network-shuffle_2.10 + ${project.version} + net.java.dev.jets3t jets3t diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 4cb0bd4142435..7d96962c4acd7 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -178,6 +178,7 @@ private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) } } else { + logError("Missing all output locations for shuffle " + shuffleId) throw new MetadataFetchFailedException( shuffleId, reduceId, "Missing all output locations for shuffle " + shuffleId) } @@ -348,7 +349,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr new ConcurrentHashMap[Int, Array[MapStatus]] } -private[spark] object MapOutputTracker { +private[spark] object MapOutputTracker extends Logging { // Serialize an array of map output locations into an efficient byte format so that we can send // it to reduce tasks. We do this by compressing the serialized bytes using GZIP. They will @@ -381,6 +382,7 @@ private[spark] object MapOutputTracker { statuses.map { status => if (status == null) { + logError("Missing an output location for shuffle " + shuffleId) throw new MetadataFetchFailedException( shuffleId, reduceId, "Missing an output location for shuffle " + shuffleId) } else { diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 16c5d6648d828..e2f13accdfab5 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -32,7 +32,7 @@ import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.network.BlockTransferService -import org.apache.spark.network.netty.{NettyBlockTransferService} +import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.network.nio.NioBlockTransferService import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index c4a8ec2e5e7b0..f1f66d0903f1c 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -186,11 +186,11 @@ private[spark] class Worker( private def retryConnectToMaster() { Utils.tryOrExit { connectionAttemptCount += 1 - logInfo(s"Attempting to connect to master (attempt # $connectionAttemptCount") if (registered) { registrationRetryTimer.foreach(_.cancel()) registrationRetryTimer = None } else if (connectionAttemptCount <= TOTAL_REGISTRATION_RETRIES) { + logInfo(s"Retrying connection to master (attempt # $connectionAttemptCount)") tryRegisterAllMasters() if (connectionAttemptCount == INITIAL_REGISTRATION_RETRIES) { registrationRetryTimer.foreach(_.cancel()) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 2889f59e33e84..c78e0ffca25bb 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -78,7 +78,7 @@ private[spark] class Executor( val executorSource = new ExecutorSource(this, executorId) // Initialize Spark environment (using system properties read above) - conf.set("spark.executor.id", "executor." + executorId) + conf.set("spark.executor.id", executorId) private val env = { if (!isLocal) { val port = conf.getInt("spark.executor.port", 0) diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index b083f465334fe..210a581db466e 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -20,16 +20,16 @@ package org.apache.spark.network import java.io.Closeable import java.nio.ByteBuffer -import scala.concurrent.{Await, Future} +import scala.concurrent.{Promise, Await, Future} import scala.concurrent.duration.Duration import org.apache.spark.Logging import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} -import org.apache.spark.storage.{BlockId, StorageLevel} -import org.apache.spark.util.Utils +import org.apache.spark.network.shuffle.{ShuffleClient, BlockFetchingListener} +import org.apache.spark.storage.{BlockManagerId, BlockId, StorageLevel} private[spark] -abstract class BlockTransferService extends Closeable with Logging { +abstract class BlockTransferService extends ShuffleClient with Closeable with Logging { /** * Initialize the transfer service by giving it the BlockDataManager that can be used to fetch @@ -60,10 +60,11 @@ abstract class BlockTransferService extends Closeable with Logging { * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as * the data of a block is fetched, rather than waiting for all blocks to be fetched. */ - def fetchBlocks( - hostName: String, + override def fetchBlocks( + host: String, port: Int, - blockIds: Seq[String], + execId: String, + blockIds: Array[String], listener: BlockFetchingListener): Unit /** @@ -81,43 +82,23 @@ abstract class BlockTransferService extends Closeable with Logging { * * It is also only available after [[init]] is invoked. */ - def fetchBlockSync(hostName: String, port: Int, blockId: String): ManagedBuffer = { + def fetchBlockSync(host: String, port: Int, execId: String, blockId: String): ManagedBuffer = { // A monitor for the thread to wait on. - val lock = new Object - @volatile var result: Either[ManagedBuffer, Throwable] = null - fetchBlocks(hostName, port, Seq(blockId), new BlockFetchingListener { - override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { - lock.synchronized { - result = Right(exception) - lock.notify() + val result = Promise[ManagedBuffer]() + fetchBlocks(host, port, execId, Array(blockId), + new BlockFetchingListener { + override def onBlockFetchFailure(blockId: String, exception: Throwable): Unit = { + result.failure(exception) } - } - override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { - lock.synchronized { + override def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit = { val ret = ByteBuffer.allocate(data.size.toInt) ret.put(data.nioByteBuffer()) ret.flip() - result = Left(new NioManagedBuffer(ret)) - lock.notify() + result.success(new NioManagedBuffer(ret)) } - } - }) + }) - // Sleep until result is no longer null - lock.synchronized { - while (result == null) { - try { - lock.wait() - } catch { - case e: InterruptedException => - } - } - } - - result match { - case Left(data) => data - case Right(e) => throw e - } + Await.result(result.future, Duration.Inf) } /** diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala deleted file mode 100644 index 8c5ffd8da6bbb..0000000000000 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockFetcher.scala +++ /dev/null @@ -1,95 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.netty - -import java.nio.ByteBuffer -import java.util - -import org.apache.spark.{SparkConf, Logging} -import org.apache.spark.network.BlockFetchingListener -import org.apache.spark.network.netty.NettyMessages._ -import org.apache.spark.serializer.{JavaSerializer, Serializer} -import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.client.{RpcResponseCallback, ChunkReceivedCallback, TransportClient} -import org.apache.spark.storage.BlockId -import org.apache.spark.util.Utils - -/** - * Responsible for holding the state for a request for a single set of blocks. This assumes that - * the chunks will be returned in the same order as requested, and that there will be exactly - * one chunk per block. - * - * Upon receipt of any block, the listener will be called back. Upon failure part way through, - * the listener will receive a failure callback for each outstanding block. - */ -class NettyBlockFetcher( - serializer: Serializer, - client: TransportClient, - blockIds: Seq[String], - listener: BlockFetchingListener) - extends Logging { - - require(blockIds.nonEmpty) - - private val ser = serializer.newInstance() - - private var streamHandle: ShuffleStreamHandle = _ - - private val chunkCallback = new ChunkReceivedCallback { - // On receipt of a chunk, pass it upwards as a block. - def onSuccess(chunkIndex: Int, buffer: ManagedBuffer): Unit = Utils.logUncaughtExceptions { - listener.onBlockFetchSuccess(blockIds(chunkIndex), buffer) - } - - // On receipt of a failure, fail every block from chunkIndex onwards. - def onFailure(chunkIndex: Int, e: Throwable): Unit = { - blockIds.drop(chunkIndex).foreach { blockId => - listener.onBlockFetchFailure(blockId, e); - } - } - } - - /** Begins the fetching process, calling the listener with every block fetched. */ - def start(): Unit = { - // Send the RPC to open the given set of blocks. This will return a ShuffleStreamHandle. - client.sendRpc(ser.serialize(OpenBlocks(blockIds.map(BlockId.apply))).array(), - new RpcResponseCallback { - override def onSuccess(response: Array[Byte]): Unit = { - try { - streamHandle = ser.deserialize[ShuffleStreamHandle](ByteBuffer.wrap(response)) - logTrace(s"Successfully opened block set: $streamHandle! Preparing to fetch chunks.") - - // Immediately request all chunks -- we expect that the total size of the request is - // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. - for (i <- 0 until streamHandle.numChunks) { - client.fetchChunk(streamHandle.streamId, i, chunkCallback) - } - } catch { - case e: Exception => - logError("Failed while starting block fetches", e) - blockIds.foreach(blockId => Utils.tryLog(listener.onBlockFetchFailure(blockId, e))) - } - } - - override def onFailure(e: Throwable): Unit = { - logError("Failed while starting block fetches", e) - blockIds.foreach(blockId => Utils.tryLog(listener.onBlockFetchFailure(blockId, e))) - } - }) - } -} diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 02c657e1d61b5..1950e7bd634ee 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -19,39 +19,41 @@ package org.apache.spark.network.netty import java.nio.ByteBuffer +import scala.collection.JavaConversions._ + import org.apache.spark.Logging import org.apache.spark.network.BlockDataManager +import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} +import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager} +import org.apache.spark.network.shuffle.ShuffleStreamHandle import org.apache.spark.serializer.Serializer -import org.apache.spark.network.buffer.{NioManagedBuffer, ManagedBuffer} -import org.apache.spark.network.client.{TransportClient, RpcResponseCallback} -import org.apache.spark.network.server.{DefaultStreamManager, RpcHandler} -import org.apache.spark.storage.{StorageLevel, BlockId} - -import scala.collection.JavaConversions._ +import org.apache.spark.storage.{BlockId, StorageLevel} object NettyMessages { - /** Request to read a set of blocks. Returns [[ShuffleStreamHandle]] to identify the stream. */ case class OpenBlocks(blockIds: Seq[BlockId]) /** Request to upload a block with a certain StorageLevel. Returns nothing (empty byte array). */ case class UploadBlock(blockId: BlockId, blockData: Array[Byte], level: StorageLevel) - - /** Identifier for a fixed number of chunks to read from a stream created by [[OpenBlocks]]. */ - case class ShuffleStreamHandle(streamId: Long, numChunks: Int) } /** * Serves requests to open blocks by simply registering one chunk per block requested. + * Handles opening and uploading arbitrary BlockManager blocks. + * + * Opened blocks are registered with the "one-for-one" strategy, meaning each Transport-layer Chunk + * is equivalent to one Spark-level shuffle block. */ class NettyBlockRpcServer( serializer: Serializer, - streamManager: DefaultStreamManager, blockManager: BlockDataManager) extends RpcHandler with Logging { import NettyMessages._ + private val streamManager = new OneForOneStreamManager() + override def receive( client: TransportClient, messageBytes: Array[Byte], @@ -73,4 +75,6 @@ class NettyBlockRpcServer( responseContext.onSuccess(new Array[Byte](0)) } } + + override def getStreamManager(): StreamManager = streamManager } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 38a3e945155e8..ec3000e722a95 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -17,15 +17,15 @@ package org.apache.spark.network.netty -import scala.concurrent.{Promise, Future} +import scala.concurrent.{Future, Promise} import org.apache.spark.SparkConf import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer -import org.apache.spark.network.client.{RpcResponseCallback, TransportClient, TransportClientFactory} -import org.apache.spark.network.netty.NettyMessages.UploadBlock +import org.apache.spark.network.client.{RpcResponseCallback, TransportClientFactory} +import org.apache.spark.network.netty.NettyMessages.{OpenBlocks, UploadBlock} import org.apache.spark.network.server._ -import org.apache.spark.network.util.{ConfigProvider, TransportConf} +import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils @@ -37,30 +37,29 @@ class NettyBlockTransferService(conf: SparkConf) extends BlockTransferService { // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. val serializer = new JavaSerializer(conf) - // Create a TransportConfig using SparkConf. - private[this] val transportConf = new TransportConf( - new ConfigProvider { override def get(name: String) = conf.get(name) }) - private[this] var transportContext: TransportContext = _ private[this] var server: TransportServer = _ private[this] var clientFactory: TransportClientFactory = _ override def init(blockDataManager: BlockDataManager): Unit = { - val streamManager = new DefaultStreamManager - val rpcHandler = new NettyBlockRpcServer(serializer, streamManager, blockDataManager) - transportContext = new TransportContext(transportConf, streamManager, rpcHandler) + val rpcHandler = new NettyBlockRpcServer(serializer, blockDataManager) + transportContext = new TransportContext(SparkTransportConf.fromSparkConf(conf), rpcHandler) clientFactory = transportContext.createClientFactory() server = transportContext.createServer() + logInfo("Server created on " + server.getPort) } override def fetchBlocks( - hostname: String, + host: String, port: Int, - blockIds: Seq[String], + execId: String, + blockIds: Array[String], listener: BlockFetchingListener): Unit = { + logTrace(s"Fetch blocks from $host:$port (executor id $execId)") try { - val client = clientFactory.createClient(hostname, port) - new NettyBlockFetcher(serializer, client, blockIds, listener).start() + val client = clientFactory.createClient(host, port) + new OneForOneBlockFetcher(client, blockIds.toArray, listener) + .start(OpenBlocks(blockIds.map(BlockId.apply))) } catch { case e: Exception => logError("Exception while beginning fetchBlocks", e) diff --git a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala new file mode 100644 index 0000000000000..9fa4fa77b8817 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import org.apache.spark.SparkConf +import org.apache.spark.network.util.{TransportConf, ConfigProvider} + +/** + * Utility for creating a [[TransportConf]] from a [[SparkConf]]. + */ +object SparkTransportConf { + def fromSparkConf(conf: SparkConf): TransportConf = { + new TransportConf(new ConfigProvider { + override def get(name: String): String = conf.get(name) + }) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala index 11793ea92adb1..f56d165daba55 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/NioBlockTransferService.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.Utils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} @@ -79,13 +80,14 @@ final class NioBlockTransferService(conf: SparkConf, securityManager: SecurityMa } override def fetchBlocks( - hostName: String, + host: String, port: Int, - blockIds: Seq[String], + execId: String, + blockIds: Array[String], listener: BlockFetchingListener): Unit = { checkInit() - val cmId = new ConnectionManagerId(hostName, port) + val cmId = new ConnectionManagerId(host, port) val blockMessageArray = new BlockMessageArray(blockIds.map { blockId => BlockMessage.fromGetBlock(GetBlock(BlockId(blockId))) }) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f81fa6d8089fc..af17b5d5d2571 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -124,6 +124,9 @@ class DAGScheduler( /** If enabled, we may run certain actions like take() and first() locally. */ private val localExecutionEnabled = sc.getConf.getBoolean("spark.localExecution.enabled", false) + /** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */ + private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false) + private def initializeEventProcessActor() { // blocking the thread until supervisor is started, which ensures eventProcessActor is // not null before any job is submitted @@ -1064,7 +1067,9 @@ class DAGScheduler( runningStages -= failedStage } - if (failedStages.isEmpty && eventProcessActor != null) { + if (disallowStageRetryForTest) { + abortStage(failedStage, "Fetch failure will not retry stage due to testing config") + } else if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. eventProcessActor may be // null during unit tests. @@ -1086,7 +1091,7 @@ class DAGScheduler( // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { - handleExecutorLost(bmAddress.executorId, Some(task.epoch)) + handleExecutorLost(bmAddress.executorId, fetchFailed = true, Some(task.epoch)) } case ExceptionFailure(className, description, stackTrace, metrics) => @@ -1106,25 +1111,35 @@ class DAGScheduler( * Responds to an executor being lost. This is called inside the event loop, so it assumes it can * modify the scheduler's internal state. Use executorLost() to post a loss event from outside. * + * We will also assume that we've lost all shuffle blocks associated with the executor if the + * executor serves its own blocks (i.e., we're not using external shuffle) OR a FetchFailed + * occurred, in which case we presume all shuffle data related to this executor to be lost. + * * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - private[scheduler] def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { + private[scheduler] def handleExecutorLost( + execId: String, + fetchFailed: Boolean, + maybeEpoch: Option[Long] = None) { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { failedEpoch(execId) = currentEpoch logInfo("Executor lost: %s (epoch %d)".format(execId, currentEpoch)) blockManagerMaster.removeExecutor(execId) - // TODO: This will be really slow if we keep accumulating shuffle map stages - for ((shuffleId, stage) <- shuffleToMapStage) { - stage.removeOutputsOnExecutor(execId) - val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray - mapOutputTracker.registerMapOutputs(shuffleId, locs, changeEpoch = true) - } - if (shuffleToMapStage.isEmpty) { - mapOutputTracker.incrementEpoch() + + if (!env.blockManager.externalShuffleServiceEnabled || fetchFailed) { + // TODO: This will be really slow if we keep accumulating shuffle map stages + for ((shuffleId, stage) <- shuffleToMapStage) { + stage.removeOutputsOnExecutor(execId) + val locs = stage.outputLocs.map(list => if (list.isEmpty) null else list.head).toArray + mapOutputTracker.registerMapOutputs(shuffleId, locs, changeEpoch = true) + } + if (shuffleToMapStage.isEmpty) { + mapOutputTracker.incrementEpoch() + } + clearCacheLocs() } - clearCacheLocs() } else { logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") @@ -1382,7 +1397,7 @@ private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGSchedule dagScheduler.handleExecutorAdded(execId, host) case ExecutorLost(execId) => - dagScheduler.handleExecutorLost(execId) + dagScheduler.handleExecutorLost(execId, fetchFailed = false) case BeginEvent(task, taskInfo) => dagScheduler.handleBeginEvent(task, taskInfo) diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index 071568cdfb429..cc13f57a49b89 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -102,6 +102,11 @@ private[spark] class Stage( } } + /** + * Removes all shuffle outputs associated with this executor. Note that this will also remove + * outputs which are served by an external shuffle server (if one exists), as they are still + * registered with this execId. + */ def removeOutputsOnExecutor(execId: String) { var becameUnavailable = false for (partition <- 0 until numPartitions) { @@ -131,4 +136,9 @@ private[spark] class Stage( override def toString = "Stage " + id override def hashCode(): Int = id + + override def equals(other: Any): Boolean = other match { + case stage: Stage => stage != null && stage.id == id + case _ => false + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a6c23fc85a1b0..376821f89c6b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -687,10 +687,11 @@ private[spark] class TaskSetManager( addPendingTask(index, readding=true) } - // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage. + // Re-enqueue any tasks that ran on the failed executor if this is a shuffle map stage, + // and we are not using an external shuffle server which could serve the shuffle outputs. // The reason is the next stage wouldn't be able to fetch the data from this dead executor // so we would need to rerun these tasks on other executors. - if (tasks(0).isInstanceOf[ShuffleMapTask]) { + if (tasks(0).isInstanceOf[ShuffleMapTask] && !env.blockManager.externalShuffleServiceEnabled) { for ((tid, info) <- taskInfos if info.executorId == execId) { val index = taskInfos(tid).index if (successful(index)) { diff --git a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala index 1fb5b2c4546bd..f03e8e4bf1b7e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala @@ -62,7 +62,8 @@ private[spark] trait ShuffleWriterGroup { * each block stored in each file. In order to find the location of a shuffle block, we search the * files within a ShuffleFileGroups associated with the block's reducer. */ - +// Note: Changes to the format in this file should be kept in sync with +// org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getHashBasedShuffleBlockData(). private[spark] class FileShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager with Logging { diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala index e9805c9c134b5..a48f0c9eceb5e 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala @@ -35,6 +35,8 @@ import org.apache.spark.storage._ * as the filename postfix for data file, and ".index" as the filename postfix for index file. * */ +// Note: Changes to the format in this file should be kept in sync with +// org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getSortBasedShuffleBlockData(). private[spark] class IndexShuffleBlockManager extends ShuffleBlockManager { diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index 6cf9305977a3c..f49917b7fe833 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -74,7 +74,7 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { val blockFetcherItr = new ShuffleBlockFetcherIterator( context, - SparkEnv.get.blockTransferService, + SparkEnv.get.blockManager.shuffleClient, blockManager, blocksByAddress, serializer, diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 746ed33b54c00..183a30373b28c 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -107,7 +107,7 @@ private[spark] class HashShuffleWriter[K, V]( writer.commitAndClose() writer.fileSegment().length } - MapStatus(blockManager.blockManagerId, sizes) + MapStatus(blockManager.shuffleServerId, sizes) } private def revertWrites(): Unit = { diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 927481b72cf4f..d75f9d7311fad 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -70,7 +70,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val partitionLengths = sorter.writePartitionedFile(blockId, context, outputFile) shuffleBlockManager.writeIndexFile(dep.shuffleId, mapId, partitionLengths) - mapStatus = MapStatus(blockManager.blockManagerId, partitionLengths) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } /** Close this writer, passing along whether the map completed */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 8df5ec6bde184..1f012941c85ab 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -53,6 +53,8 @@ case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockId { def name = "rdd_" + rddId + "_" + splitIndex } +// Format of the shuffle block ids (including data and index) should be kept in sync with +// org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getBlockData(). @DeveloperApi case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 58510d7232436..1f8de2896116c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -21,9 +21,9 @@ import java.io.{BufferedOutputStream, ByteArrayOutputStream, File, InputStream, import java.nio.{ByteBuffer, MappedByteBuffer} import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.concurrent.{Await, Future} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ -import scala.concurrent.{Await, Future} import scala.util.Random import akka.actor.{ActorSystem, Props} @@ -34,8 +34,13 @@ import org.apache.spark.executor._ import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.netty.{SparkTransportConf, NettyBlockTransferService} +import org.apache.spark.network.shuffle.{ExecutorShuffleInfo, ExternalShuffleClient} +import org.apache.spark.network.util.{ConfigProvider, TransportConf} import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.ShuffleManager +import org.apache.spark.shuffle.hash.HashShuffleManager +import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.util._ private[spark] sealed trait BlockValues @@ -85,9 +90,38 @@ private[spark] class BlockManager( new TachyonStore(this, tachyonBlockManager) } + private[spark] + val externalShuffleServiceEnabled = conf.getBoolean("spark.shuffle.service.enabled", false) + private val externalShuffleServicePort = conf.getInt("spark.shuffle.service.port", 7337) + // Check that we're not using external shuffle service with consolidated shuffle files. + if (externalShuffleServiceEnabled + && conf.getBoolean("spark.shuffle.consolidateFiles", false) + && shuffleManager.isInstanceOf[HashShuffleManager]) { + throw new UnsupportedOperationException("Cannot use external shuffle service with consolidated" + + " shuffle files in hash-based shuffle. Please disable spark.shuffle.consolidateFiles or " + + " switch to sort-based shuffle.") + } + val blockManagerId = BlockManagerId( executorId, blockTransferService.hostName, blockTransferService.port) + // Address of the server that serves this executor's shuffle files. This is either an external + // service, or just our own Executor's BlockManager. + private[spark] val shuffleServerId = if (externalShuffleServiceEnabled) { + BlockManagerId(executorId, blockTransferService.hostName, externalShuffleServicePort) + } else { + blockManagerId + } + + // Client to read other executors' shuffle files. This is either an external service, or just the + // standard BlockTranserService to directly connect to other Executors. + private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { + val appId = conf.get("spark.app.id", "unknown-app-id") + new ExternalShuffleClient(SparkTransportConf.fromSparkConf(conf), appId) + } else { + blockTransferService + } + // Whether to compress broadcast variables that are stored private val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) // Whether to compress shuffle output that are stored @@ -143,10 +177,41 @@ private[spark] class BlockManager( /** * Initialize the BlockManager. Register to the BlockManagerMaster, and start the - * BlockManagerWorker actor. + * BlockManagerWorker actor. Additionally registers with a local shuffle service if configured. */ private def initialize(): Unit = { master.registerBlockManager(blockManagerId, maxMemory, slaveActor) + + // Register Executors' configuration with the local shuffle service, if one should exist. + if (externalShuffleServiceEnabled && !blockManagerId.isDriver) { + registerWithExternalShuffleServer() + } + } + + private def registerWithExternalShuffleServer() { + logInfo("Registering executor with local external shuffle service.") + val shuffleConfig = new ExecutorShuffleInfo( + diskBlockManager.localDirs.map(_.toString), + diskBlockManager.subDirsPerLocalDir, + shuffleManager.getClass.getName) + + val MAX_ATTEMPTS = 3 + val SLEEP_TIME_SECS = 5 + + for (i <- 1 to MAX_ATTEMPTS) { + try { + // Synchronous and will throw an exception if we cannot connect. + shuffleClient.asInstanceOf[ExternalShuffleClient].registerWithShuffleServer( + shuffleServerId.host, shuffleServerId.port, shuffleServerId.executorId, shuffleConfig) + return + } catch { + case e: Exception if i < MAX_ATTEMPTS => + val attemptsRemaining = + logError(s"Failed to connect to external shuffle server, will retry ${MAX_ATTEMPTS - i}}" + + s" more times after waiting $SLEEP_TIME_SECS seconds...", e) + Thread.sleep(SLEEP_TIME_SECS * 1000) + } + } } /** @@ -506,7 +571,7 @@ private[spark] class BlockManager( for (loc <- locations) { logDebug(s"Getting remote block $blockId from $loc") val data = blockTransferService.fetchBlockSync( - loc.host, loc.port, blockId.toString).nioByteBuffer() + loc.host, loc.port, loc.executorId, blockId.toString).nioByteBuffer() if (data != null) { if (asBlockResult) { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 99e925328a4b9..58fba54710510 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -38,12 +38,13 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon extends Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 - private val subDirsPerLocalDir = blockManager.conf.getInt("spark.diskStore.subDirectories", 64) + private[spark] + val subDirsPerLocalDir = blockManager.conf.getInt("spark.diskStore.subDirectories", 64) /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid * having really large inodes at the top level. */ - val localDirs: Array[File] = createLocalDirs(conf) + private[spark] val localDirs: Array[File] = createLocalDirs(conf) if (localDirs.isEmpty) { logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) @@ -52,6 +53,9 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon addShutdownHook() + /** Looks up a file by hashing it into one of our local subdirectories. */ + // This method should be kept in sync with + // org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getFile(). def getFile(filename: String): File = { // Figure out which local directory it hashes to, and which subdirectory in that val hash = Utils.nonNegativeHash(filename) @@ -159,13 +163,16 @@ private[spark] class DiskBlockManager(blockManager: BlockManager, conf: SparkCon /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { - localDirs.foreach { localDir => - if (localDir.isDirectory() && localDir.exists()) { - try { - if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) - } catch { - case e: Exception => - logError(s"Exception while deleting local spark dir: $localDir", e) + // Only perform cleanup if an external service is not serving our shuffle files. + if (!blockManager.externalShuffleServiceEnabled) { + localDirs.foreach { localDir => + if (localDir.isDirectory() && localDir.exists()) { + try { + if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } catch { + case e: Exception => + logError(s"Exception while deleting local spark dir: $localDir", e) + } } } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 0d6f3bf003a9d..ee89c7e521f4e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -22,7 +22,8 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} import org.apache.spark.{Logging, TaskContext} -import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} +import org.apache.spark.network.BlockTransferService +import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.serializer.Serializer import org.apache.spark.util.{CompletionIterator, Utils} @@ -38,8 +39,8 @@ import org.apache.spark.util.{CompletionIterator, Utils} * using too much memory. * * @param context [[TaskContext]], used for metrics update - * @param blockTransferService [[BlockTransferService]] for fetching remote blocks - * @param blockManager [[BlockManager]] for reading local blocks + * @param shuffleClient [[ShuffleClient]] for fetching remote blocks + * @param blockManager [[BlockManager]] for reading local blocks * @param blocksByAddress list of blocks to fetch grouped by the [[BlockManagerId]]. * For each block we also require the size (in bytes as a long field) in * order to throttle the memory usage. @@ -49,7 +50,7 @@ import org.apache.spark.util.{CompletionIterator, Utils} private[spark] final class ShuffleBlockFetcherIterator( context: TaskContext, - blockTransferService: BlockTransferService, + shuffleClient: ShuffleClient, blockManager: BlockManager, blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer, @@ -140,7 +141,8 @@ final class ShuffleBlockFetcherIterator( val sizeMap = req.blocks.map { case (blockId, size) => (blockId.toString, size) }.toMap val blockIds = req.blocks.map(_._1.toString) - blockTransferService.fetchBlocks(req.address.host, req.address.port, blockIds, + val address = req.address + shuffleClient.fetchBlocks(address.host, address.port, address.executorId, blockIds.toArray, new BlockFetchingListener { override def onBlockFetchSuccess(blockId: String, buf: ManagedBuffer): Unit = { // Only add the buffer to results queue if the iterator is not zombie, @@ -179,7 +181,7 @@ final class ShuffleBlockFetcherIterator( var totalBlocks = 0 for ((address, blockInfos) <- blocksByAddress) { totalBlocks += blockInfos.size - if (address == blockManager.blockManagerId) { + if (address.executorId == blockManager.blockManagerId.executorId) { // Filter out zero-sized blocks localBlocks ++= blockInfos.filter(_._2 != 0).map(_._1) numBlocksToFetch += localBlocks.size 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 063895d3c548d..68d378f3a212d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1237,6 +1237,8 @@ private[spark] object Utils extends Logging { } // Handles idiosyncracies with hash (add more as required) + // This method should be kept in sync with + // org.apache.spark.network.util.JavaUtils#nonNegativeHash(). def nonNegativeHash(obj: AnyRef): Int = { // Required ? diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 81b64c36ddca1..429199f2075c6 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -202,7 +202,8 @@ class DistributedSuite extends FunSuite with Matchers with BeforeAndAfter val blockManager = SparkEnv.get.blockManager val blockTransfer = SparkEnv.get.blockTransferService blockManager.master.getLocations(blockId).foreach { cmId => - val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, blockId.toString) + val bytes = blockTransfer.fetchBlockSync(cmId.host, cmId.port, cmId.executorId, + blockId.toString) val deserialized = blockManager.dataDeserialize(blockId, bytes.nioByteBuffer()) .asInstanceOf[Iterator[Int]].toList assert(deserialized === (1 to 100).toList) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala new file mode 100644 index 0000000000000..792b9cd8b6ff2 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark + +import java.util.concurrent.atomic.AtomicInteger + +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkContext._ +import org.apache.spark.network.TransportContext +import org.apache.spark.network.netty.SparkTransportConf +import org.apache.spark.network.server.TransportServer +import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleClient} + +/** + * This suite creates an external shuffle server and routes all shuffle fetches through it. + * Note that failures in this suite may arise due to changes in Spark that invalidate expectations + * set up in [[ExternalShuffleBlockHandler]], such as changing the format of shuffle files or how + * we hash files into folders. + */ +class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { + var server: TransportServer = _ + var rpcHandler: ExternalShuffleBlockHandler = _ + + override def beforeAll() { + val transportConf = SparkTransportConf.fromSparkConf(conf) + rpcHandler = new ExternalShuffleBlockHandler() + val transportContext = new TransportContext(transportConf, rpcHandler) + server = transportContext.createServer() + + conf.set("spark.shuffle.manager", "sort") + conf.set("spark.shuffle.service.enabled", "true") + conf.set("spark.shuffle.service.port", server.getPort.toString) + } + + override def afterAll() { + server.close() + } + + // This test ensures that the external shuffle service is actually in use for the other tests. + test("using external shuffle service") { + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) + sc.env.blockManager.externalShuffleServiceEnabled should equal(true) + sc.env.blockManager.shuffleClient.getClass should equal(classOf[ExternalShuffleClient]) + + val rdd = sc.parallelize(0 until 1000, 10).map(i => (i, 1)).reduceByKey(_ + _) + + rdd.count() + rdd.count() + + // Invalidate the registered executors, disallowing access to their shuffle blocks. + rpcHandler.clearRegisteredExecutors() + + // Now Spark will receive FetchFailed, and not retry the stage due to "spark.test.noStageRetry" + // being set. + val e = intercept[SparkException] { + rdd.count() + } + e.getMessage should include ("Fetch failure will not retry stage due to testing config") + } +} diff --git a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala index 2acc02a54fa3d..19180e88ebe0a 100644 --- a/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/HashShuffleSuite.scala @@ -24,10 +24,6 @@ class HashShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with hash-based shuffle. override def beforeAll() { - System.setProperty("spark.shuffle.manager", "hash") - } - - override def afterAll() { - System.clearProperty("spark.shuffle.manager") + conf.set("spark.shuffle.manager", "hash") } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala index 840d8273cb6a8..d78c99c2e1e06 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleNettySuite.scala @@ -24,10 +24,6 @@ class ShuffleNettySuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with Netty shuffle mode. override def beforeAll() { - System.setProperty("spark.shuffle.blockTransferService", "netty") - } - - override def afterAll() { - System.clearProperty("spark.shuffle.blockTransferService") + conf.set("spark.shuffle.blockTransferService", "netty") } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 2bdd84ce69ab8..cda942e15a704 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -30,10 +30,14 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex val conf = new SparkConf(loadDefaults = false) + // Ensure that the DAGScheduler doesn't retry stages whose fetches fail, so that we accurately + // test that the shuffle works (rather than retrying until all blocks are local to one Executor). + conf.set("spark.test.noStageRetry", "true") + test("groupByKey without compression") { try { System.setProperty("spark.shuffle.compress", "false") - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() assert(groups.size === 2) @@ -47,7 +51,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex } test("shuffle non-zero block size") { - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) val NUM_BLOCKS = 3 val a = sc.parallelize(1 to 10, 2) @@ -73,7 +77,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("shuffle serializer") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) val a = sc.parallelize(1 to 10, 2) val b = a.map { x => (x, new NonJavaSerializableClass(x * 2)) @@ -89,7 +93,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("zero sized blocks") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) // 10 partitions from 4 keys val NUM_BLOCKS = 10 @@ -116,7 +120,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("zero sized blocks without kryo") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) // 10 partitions from 4 keys val NUM_BLOCKS = 10 @@ -141,7 +145,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("shuffle on mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) @@ -154,7 +158,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("sorting on mutable pairs") { // This is not in SortingSuite because of the local cluster setup. // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data = Array(p(1, 11), p(3, 33), p(100, 100), p(2, 22)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) @@ -168,7 +172,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("cogroup using mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3")) @@ -195,7 +199,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("subtract mutable pairs") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - sc = new SparkContext("local-cluster[2,1,512]", "test") + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1), p(3, 33)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22")) @@ -209,11 +213,8 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("sort with Java non serializable class - Kryo") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - val conf = new SparkConf() - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .setAppName("test") - .setMaster("local-cluster[2,1,512]") - sc = new SparkContext(conf) + val myConf = conf.clone().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sc = new SparkContext("local-cluster[2,1,512]", "test", myConf) val a = sc.parallelize(1 to 10, 2) val b = a.map { x => (new NonJavaSerializableClass(x), x) @@ -226,10 +227,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("sort with Java non serializable class - Java") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - val conf = new SparkConf() - .setAppName("test") - .setMaster("local-cluster[2,1,512]") - sc = new SparkContext(conf) + sc = new SparkContext("local-cluster[2,1,512]", "test", conf) val a = sc.parallelize(1 to 10, 2) val b = a.map { x => (new NonJavaSerializableClass(x), x) diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index 639e56c488db4..63358172ea1f4 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -24,10 +24,6 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // This test suite should run all tests in ShuffleSuite with sort-based shuffle. override def beforeAll() { - System.setProperty("spark.shuffle.manager", "sort") - } - - override def afterAll() { - System.clearProperty("spark.shuffle.manager") + conf.set("spark.shuffle.manager", "sort") } } diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index 3925f0ccbdbf0..bbdc9568a6ddb 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -121,7 +121,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod } val appId = "testId" - val executorId = "executor.1" + val executorId = "1" conf.set("spark.app.id", appId) conf.set("spark.executor.id", executorId) @@ -138,7 +138,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter with PrivateMethod override val metricRegistry = new MetricRegistry() } - val executorId = "executor.1" + val executorId = "1" conf.set("spark.executor.id", executorId) val instanceName = "executor" diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 4e502cf65e6b3..28f766570e96f 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -21,22 +21,19 @@ import java.util.concurrent.Semaphore import scala.concurrent.future import scala.concurrent.ExecutionContext.Implicits.global -import org.apache.spark.{TaskContextImpl, TaskContext} -import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} -import org.mockito.Mockito._ import org.mockito.Matchers.{any, eq => meq} +import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer - import org.scalatest.FunSuite -import org.apache.spark.{SparkConf, TaskContext} +import org.apache.spark.{SparkConf, TaskContextImpl} import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.BlockFetchingListener import org.apache.spark.serializer.TestSerializer - class ShuffleBlockFetcherIteratorSuite extends FunSuite { // Some of the tests are quite tricky because we are testing the cleanup behavior // in the presence of faults. @@ -44,10 +41,10 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { /** Creates a mock [[BlockTransferService]] that returns data from the given map. */ private def createMockTransfer(data: Map[BlockId, ManagedBuffer]): BlockTransferService = { val transfer = mock(classOf[BlockTransferService]) - when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { - val blocks = invocation.getArguments()(2).asInstanceOf[Seq[String]] - val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + val blocks = invocation.getArguments()(3).asInstanceOf[Array[String]] + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] for (blockId <- blocks) { if (data.contains(BlockId(blockId))) { @@ -118,7 +115,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { // 3 local blocks, and 2 remote blocks // (but from the same block manager so one call to fetchBlocks) verify(blockManager, times(3)).getBlockData(any()) - verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any()) + verify(transfer, times(1)).fetchBlocks(any(), any(), any(), any(), any()) } test("release current unexhausted buffer in case the task completes early") { @@ -138,9 +135,9 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val sem = new Semaphore(0) val transfer = mock(classOf[BlockTransferService]) - when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { - val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] future { // Return the first two blocks, and wait till task completion before returning the 3rd one listener.onBlockFetchSuccess( @@ -201,9 +198,9 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { val sem = new Semaphore(0) val transfer = mock(classOf[BlockTransferService]) - when(transfer.fetchBlocks(any(), any(), any(), any())).thenAnswer(new Answer[Unit] { + when(transfer.fetchBlocks(any(), any(), any(), any(), any())).thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { - val listener = invocation.getArguments()(3).asInstanceOf[BlockFetchingListener] + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] future { // Return the first block, and then fail. listener.onBlockFetchSuccess( diff --git a/network/common/pom.xml b/network/common/pom.xml index a33e44b63d981..ea887148d98ba 100644 --- a/network/common/pom.xml +++ b/network/common/pom.xml @@ -85,9 +85,25 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + - org.scalatest - scalatest-maven-plugin + org.apache.maven.plugins + maven-jar-plugin + 2.2 + + + + test-jar + + + + test-jar-on-test-compile + test-compile + + test-jar + + + diff --git a/network/common/src/main/java/org/apache/spark/network/TransportContext.java b/network/common/src/main/java/org/apache/spark/network/TransportContext.java index 854aa6685f85f..a271841e4e56c 100644 --- a/network/common/src/main/java/org/apache/spark/network/TransportContext.java +++ b/network/common/src/main/java/org/apache/spark/network/TransportContext.java @@ -52,15 +52,13 @@ public class TransportContext { private final Logger logger = LoggerFactory.getLogger(TransportContext.class); private final TransportConf conf; - private final StreamManager streamManager; private final RpcHandler rpcHandler; private final MessageEncoder encoder; private final MessageDecoder decoder; - public TransportContext(TransportConf conf, StreamManager streamManager, RpcHandler rpcHandler) { + public TransportContext(TransportConf conf, RpcHandler rpcHandler) { this.conf = conf; - this.streamManager = streamManager; this.rpcHandler = rpcHandler; this.encoder = new MessageEncoder(); this.decoder = new MessageDecoder(); @@ -70,8 +68,14 @@ public TransportClientFactory createClientFactory() { return new TransportClientFactory(this); } + /** Create a server which will attempt to bind to a specific port. */ + public TransportServer createServer(int port) { + return new TransportServer(this, port); + } + + /** Creates a new server, binding to any available ephemeral port. */ public TransportServer createServer() { - return new TransportServer(this); + return new TransportServer(this, 0); } /** @@ -109,7 +113,7 @@ private TransportChannelHandler createChannelHandler(Channel channel) { TransportResponseHandler responseHandler = new TransportResponseHandler(channel); TransportClient client = new TransportClient(channel, responseHandler); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, - streamManager, rpcHandler); + rpcHandler); return new TransportChannelHandler(client, responseHandler, requestHandler); } diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java index b1732fcde21f1..01c143fff423c 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -19,9 +19,13 @@ import java.io.Closeable; import java.util.UUID; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; +import com.google.common.util.concurrent.SettableFuture; import io.netty.channel.Channel; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelFutureListener; @@ -129,7 +133,7 @@ public void sendRpc(byte[] message, final RpcResponseCallback callback) { final long startTime = System.currentTimeMillis(); logger.trace("Sending RPC to {}", serverAddr); - final long requestId = UUID.randomUUID().getLeastSignificantBits(); + final long requestId = Math.abs(UUID.randomUUID().getLeastSignificantBits()); handler.addRpcRequest(requestId, callback); channel.writeAndFlush(new RpcRequest(requestId, message)).addListener( @@ -151,6 +155,32 @@ public void operationComplete(ChannelFuture future) throws Exception { }); } + /** + * Synchronously sends an opaque message to the RpcHandler on the server-side, waiting for up to + * a specified timeout for a response. + */ + public byte[] sendRpcSync(byte[] message, long timeoutMs) { + final SettableFuture result = SettableFuture.create(); + + sendRpc(message, new RpcResponseCallback() { + @Override + public void onSuccess(byte[] response) { + result.set(response); + } + + @Override + public void onFailure(Throwable e) { + result.setException(e); + } + }); + + try { + return result.get(timeoutMs, TimeUnit.MILLISECONDS); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + @Override public void close() { // close is a local operation and should finish with milliseconds; timeout just to be safe diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 10eb9ef7a025f..e7fa4f6bf3b5d 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -78,15 +78,17 @@ public TransportClientFactory(TransportContext context) { * * Concurrency: This method is safe to call from multiple threads. */ - public TransportClient createClient(String remoteHost, int remotePort) throws TimeoutException { + public TransportClient createClient(String remoteHost, int remotePort) { // Get connection from the connection pool first. // If it is not found or not active, create a new one. final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); TransportClient cachedClient = connectionPool.get(address); - if (cachedClient != null && cachedClient.isActive()) { - return cachedClient; - } else if (cachedClient != null) { - connectionPool.remove(address, cachedClient); // Remove inactive clients. + if (cachedClient != null) { + if (cachedClient.isActive()) { + return cachedClient; + } else { + connectionPool.remove(address, cachedClient); // Remove inactive clients. + } } logger.debug("Creating new connection to " + address); @@ -115,13 +117,14 @@ public void initChannel(SocketChannel ch) { // Connect to the remote server ChannelFuture cf = bootstrap.connect(address); if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { - throw new TimeoutException( + throw new RuntimeException( String.format("Connecting to %s timed out (%s ms)", address, conf.connectionTimeoutMs())); } else if (cf.cause() != null) { throw new RuntimeException(String.format("Failed to connect to %s", address), cf.cause()); } - // Successful connection + // Successful connection -- in the event that two threads raced to create a client, we will + // use the first one that was put into the connectionPool and close the one we made here. assert client.get() != null : "Channel future completed successfully with null client"; TransportClient oldClient = connectionPool.putIfAbsent(address, client.get()); if (oldClient == null) { diff --git a/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java similarity index 72% rename from network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java rename to network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java index 7aa37efc582e4..5a3f003726fc1 100644 --- a/network/common/src/test/java/org/apache/spark/network/NoOpRpcHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/NoOpRpcHandler.java @@ -1,4 +1,6 @@ -package org.apache.spark.network;/* +package org.apache.spark.network.server; + +/* * 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. @@ -17,12 +19,20 @@ import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; -import org.apache.spark.network.server.RpcHandler; -/** Test RpcHandler which always returns a zero-sized success. */ +/** An RpcHandler suitable for a client-only TransportContext, which cannot receive RPCs. */ public class NoOpRpcHandler implements RpcHandler { + private final StreamManager streamManager; + + public NoOpRpcHandler() { + streamManager = new OneForOneStreamManager(); + } + @Override public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { - callback.onSuccess(new byte[0]); + throw new UnsupportedOperationException("Cannot handle messages"); } + + @Override + public StreamManager getStreamManager() { return streamManager; } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java similarity index 93% rename from network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java rename to network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index 9688705569634..731d48d4d9c6c 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/DefaultStreamManager.java +++ b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -30,10 +30,10 @@ /** * StreamManager which allows registration of an Iterator, which are individually - * fetched as chunks by the client. + * fetched as chunks by the client. Each registered buffer is one chunk. */ -public class DefaultStreamManager extends StreamManager { - private final Logger logger = LoggerFactory.getLogger(DefaultStreamManager.class); +public class OneForOneStreamManager extends StreamManager { + private final Logger logger = LoggerFactory.getLogger(OneForOneStreamManager.class); private final AtomicLong nextStreamId; private final Map streams; @@ -51,7 +51,7 @@ private static class StreamState { } } - public DefaultStreamManager() { + public OneForOneStreamManager() { // For debugging purposes, start with a random stream id to help identifying different streams. // This does not need to be globally unique, only unique to this class. nextStreamId = new AtomicLong((long) new Random().nextInt(Integer.MAX_VALUE) * 1000); diff --git a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java index f54a696b8ff79..2369dc6203944 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/RpcHandler.java @@ -35,4 +35,10 @@ public interface RpcHandler { * RPC. */ void receive(TransportClient client, byte[] message, RpcResponseCallback callback); + + /** + * Returns the StreamManager which contains the state about which streams are currently being + * fetched by a TransportClient. + */ + StreamManager getStreamManager(); } diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index 352f865935b11..17fe9001b35cc 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -56,24 +56,23 @@ public class TransportRequestHandler extends MessageHandler { /** Client on the same channel allowing us to talk back to the requester. */ private final TransportClient reverseClient; - /** Returns each chunk part of a stream. */ - private final StreamManager streamManager; - /** Handles all RPC messages. */ private final RpcHandler rpcHandler; + /** Returns each chunk part of a stream. */ + private final StreamManager streamManager; + /** List of all stream ids that have been read on this handler, used for cleanup. */ private final Set streamIds; public TransportRequestHandler( Channel channel, TransportClient reverseClient, - StreamManager streamManager, RpcHandler rpcHandler) { this.channel = channel; this.reverseClient = reverseClient; - this.streamManager = streamManager; this.rpcHandler = rpcHandler; + this.streamManager = rpcHandler.getStreamManager(); this.streamIds = Sets.newHashSet(); } diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java index 243070750d6e7..d1a1877a98f46 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -49,11 +49,11 @@ public class TransportServer implements Closeable { private ChannelFuture channelFuture; private int port = -1; - public TransportServer(TransportContext context) { + public TransportServer(TransportContext context, int portToBind) { this.context = context; this.conf = context.getConf(); - init(); + init(portToBind); } public int getPort() { @@ -63,7 +63,7 @@ public int getPort() { return port; } - private void init() { + private void init(int portToBind) { IOMode ioMode = IOMode.valueOf(conf.ioMode()); EventLoopGroup bossGroup = @@ -95,7 +95,7 @@ protected void initChannel(SocketChannel ch) throws Exception { } }); - channelFuture = bootstrap.bind(new InetSocketAddress(conf.serverPort())); + channelFuture = bootstrap.bind(new InetSocketAddress(portToBind)); channelFuture.syncUninterruptibly(); port = ((InetSocketAddress) channelFuture.channel().localAddress()).getPort(); diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 32ba3f5b07f7a..40b71b0c87a47 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -17,8 +17,12 @@ package org.apache.spark.network.util; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; import com.google.common.io.Closeables; import org.slf4j.Logger; @@ -35,4 +39,38 @@ public static void closeQuietly(Closeable closeable) { logger.error("IOException should not have been thrown.", e); } } + + // TODO: Make this configurable, do not use Java serialization! + public static T deserialize(byte[] bytes) { + try { + ObjectInputStream is = new ObjectInputStream(new ByteArrayInputStream(bytes)); + Object out = is.readObject(); + is.close(); + return (T) out; + } catch (ClassNotFoundException e) { + throw new RuntimeException("Could not deserialize object", e); + } catch (IOException e) { + throw new RuntimeException("Could not deserialize object", e); + } + } + + // TODO: Make this configurable, do not use Java serialization! + public static byte[] serialize(Object object) { + try { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ObjectOutputStream os = new ObjectOutputStream(baos); + os.writeObject(object); + os.close(); + return baos.toByteArray(); + } catch (IOException e) { + throw new RuntimeException("Could not serialize object", e); + } + } + + /** Returns a hash consistent with Spark's Utils.nonNegativeHash(). */ + public static int nonNegativeHash(Object obj) { + if (obj == null) { return 0; } + int hash = obj.hashCode(); + return hash != Integer.MIN_VALUE ? Math.abs(hash) : 0; + } } diff --git a/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java b/network/common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java similarity index 96% rename from network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java rename to network/common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java index f4e0a2426a3d2..5f20b70678d1e 100644 --- a/network/common/src/test/java/org/apache/spark/network/SystemPropertyConfigProvider.java +++ b/network/common/src/main/java/org/apache/spark/network/util/SystemPropertyConfigProvider.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.network; +package org.apache.spark.network.util; import java.util.NoSuchElementException; diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 80f65d98032da..a68f38e0e94c9 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -27,9 +27,6 @@ public TransportConf(ConfigProvider conf) { this.conf = conf; } - /** Port the server listens on. Default to a random port. */ - public int serverPort() { return conf.getInt("spark.shuffle.io.port", 0); } - /** IO mode: nio or epoll */ public String ioMode() { return conf.get("spark.shuffle.io.mode", "NIO").toUpperCase(); } diff --git a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index 738dca9b6a9ee..c4158833976aa 100644 --- a/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -41,10 +41,13 @@ import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.buffer.NioManagedBuffer; import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; public class ChunkFetchIntegrationSuite { @@ -93,7 +96,18 @@ public ManagedBuffer getChunk(long streamId, int chunkIndex) { } } }; - TransportContext context = new TransportContext(conf, streamManager, new NoOpRpcHandler()); + RpcHandler handler = new RpcHandler() { + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + throw new UnsupportedOperationException(); + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + }; + TransportContext context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); } diff --git a/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 9f216dd2d722d..64b457b4b3f01 100644 --- a/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/network/common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -35,9 +35,11 @@ import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientFactory; -import org.apache.spark.network.server.DefaultStreamManager; +import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; public class RpcIntegrationSuite { @@ -61,8 +63,11 @@ public void receive(TransportClient client, byte[] message, RpcResponseCallback throw new RuntimeException("Thrown: " + parts[1]); } } + + @Override + public StreamManager getStreamManager() { return new OneForOneStreamManager(); } }; - TransportContext context = new TransportContext(conf, new DefaultStreamManager(), rpcHandler); + TransportContext context = new TransportContext(conf, rpcHandler); server = context.createServer(); clientFactory = context.createClientFactory(); } diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 3ef964616f0c5..5a10fdb3842ef 100644 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -28,11 +28,11 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.client.TransportClientFactory; -import org.apache.spark.network.server.DefaultStreamManager; +import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; -import org.apache.spark.network.server.StreamManager; import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; public class TransportClientFactorySuite { @@ -44,9 +44,8 @@ public class TransportClientFactorySuite { @Before public void setUp() { conf = new TransportConf(new SystemPropertyConfigProvider()); - StreamManager streamManager = new DefaultStreamManager(); RpcHandler rpcHandler = new NoOpRpcHandler(); - context = new TransportContext(conf, streamManager, rpcHandler); + context = new TransportContext(conf, rpcHandler); server1 = context.createServer(); server2 = context.createServer(); } diff --git a/network/shuffle/pom.xml b/network/shuffle/pom.xml new file mode 100644 index 0000000000000..d271704d98a7a --- /dev/null +++ b/network/shuffle/pom.xml @@ -0,0 +1,96 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.2.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-network-shuffle_2.10 + jar + Spark Project Shuffle Streaming Service Code + http://spark.apache.org/ + + network-shuffle + + + + + + org.apache.spark + spark-network-common_2.10 + ${project.version} + + + org.slf4j + slf4j-api + + + + + com.google.guava + guava + provided + + + + + org.apache.spark + spark-network-common_2.10 + ${project.version} + test-jar + test + + + junit + junit + test + + + com.novocode + junit-interface + test + + + log4j + log4j + test + + + org.mockito + mockito-all + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java similarity index 73% rename from core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala rename to network/shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java index 645793fde806d..138fd5389c20a 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockFetchingListener.scala +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/BlockFetchingListener.java @@ -15,28 +15,22 @@ * limitations under the License. */ -package org.apache.spark.network +package org.apache.spark.network.shuffle; -import java.util.EventListener +import java.util.EventListener; -import org.apache.spark.network.buffer.ManagedBuffer - - -/** - * Listener callback interface for [[BlockTransferService.fetchBlocks]]. - */ -private[spark] -trait BlockFetchingListener extends EventListener { +import org.apache.spark.network.buffer.ManagedBuffer; +public interface BlockFetchingListener extends EventListener { /** * Called once per successfully fetched block. After this call returns, data will be released * automatically. If the data will be passed to another thread, the receiver should retain() * and release() the buffer on their own, or copy the data to a new buffer. */ - def onBlockFetchSuccess(blockId: String, data: ManagedBuffer): Unit + void onBlockFetchSuccess(String blockId, ManagedBuffer data); /** * Called at least once per block upon failures. */ - def onBlockFetchFailure(blockId: String, exception: Throwable): Unit + void onBlockFetchFailure(String blockId, Throwable exception); } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorShuffleInfo.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorShuffleInfo.java new file mode 100644 index 0000000000000..d45e64656a0e3 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorShuffleInfo.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.Serializable; +import java.util.Arrays; + +import com.google.common.base.Objects; + +/** Contains all configuration necessary for locating the shuffle files of an executor. */ +public class ExecutorShuffleInfo implements Serializable { + /** The base set of local directories that the executor stores its shuffle files in. */ + final String[] localDirs; + /** Number of subdirectories created within each localDir. */ + final int subDirsPerLocalDir; + /** Shuffle manager (SortShuffleManager or HashShuffleManager) that the executor is using. */ + final String shuffleManager; + + public ExecutorShuffleInfo(String[] localDirs, int subDirsPerLocalDir, String shuffleManager) { + this.localDirs = localDirs; + this.subDirsPerLocalDir = subDirsPerLocalDir; + this.shuffleManager = shuffleManager; + } + + @Override + public int hashCode() { + return Objects.hashCode(subDirsPerLocalDir, shuffleManager) * 41 + Arrays.hashCode(localDirs); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("localDirs", Arrays.toString(localDirs)) + .add("subDirsPerLocalDir", subDirsPerLocalDir) + .add("shuffleManager", shuffleManager) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof ExecutorShuffleInfo) { + ExecutorShuffleInfo o = (ExecutorShuffleInfo) other; + return Arrays.equals(localDirs, o.localDirs) + && Objects.equal(subDirsPerLocalDir, o.subDirsPerLocalDir) + && Objects.equal(shuffleManager, o.shuffleManager); + } + return false; + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java new file mode 100644 index 0000000000000..a9dff31decc83 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.util.List; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.spark.network.shuffle.ExternalShuffleMessages.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.server.StreamManager; +import org.apache.spark.network.util.JavaUtils; + +/** + * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. + * + * Handles registering executors and opening shuffle blocks from them. Shuffle blocks are registered + * with the "one-for-one" strategy, meaning each Transport-layer Chunk is equivalent to one Spark- + * level shuffle block. + */ +public class ExternalShuffleBlockHandler implements RpcHandler { + private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); + + private final ExternalShuffleBlockManager blockManager; + private final OneForOneStreamManager streamManager; + + public ExternalShuffleBlockHandler() { + this(new OneForOneStreamManager(), new ExternalShuffleBlockManager()); + } + + /** Enables mocking out the StreamManager and BlockManager. */ + @VisibleForTesting + ExternalShuffleBlockHandler( + OneForOneStreamManager streamManager, + ExternalShuffleBlockManager blockManager) { + this.streamManager = streamManager; + this.blockManager = blockManager; + } + + @Override + public void receive(TransportClient client, byte[] message, RpcResponseCallback callback) { + Object msgObj = JavaUtils.deserialize(message); + + logger.trace("Received message: " + msgObj); + + if (msgObj instanceof OpenShuffleBlocks) { + OpenShuffleBlocks msg = (OpenShuffleBlocks) msgObj; + List blocks = Lists.newArrayList(); + + for (String blockId : msg.blockIds) { + blocks.add(blockManager.getBlockData(msg.appId, msg.execId, blockId)); + } + long streamId = streamManager.registerStream(blocks.iterator()); + logger.trace("Registered streamId {} with {} buffers", streamId, msg.blockIds.length); + callback.onSuccess(JavaUtils.serialize( + new ShuffleStreamHandle(streamId, msg.blockIds.length))); + + } else if (msgObj instanceof RegisterExecutor) { + RegisterExecutor msg = (RegisterExecutor) msgObj; + blockManager.registerExecutor(msg.appId, msg.execId, msg.executorInfo); + callback.onSuccess(new byte[0]); + + } else { + throw new UnsupportedOperationException(String.format( + "Unexpected message: %s (class = %s)", msgObj, msgObj.getClass())); + } + } + + @Override + public StreamManager getStreamManager() { + return streamManager; + } + + /** For testing, clears all executors registered with "RegisterExecutor". */ + @VisibleForTesting + public void clearRegisteredExecutors() { + blockManager.clearRegisteredExecutors(); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java new file mode 100644 index 0000000000000..6589889fe1be7 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManager.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.DataInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.concurrent.ConcurrentHashMap; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.util.JavaUtils; + +/** + * Manages converting shuffle BlockIds into physical segments of local files, from a process outside + * of Executors. Each Executor must register its own configuration about where it stores its files + * (local dirs) and how (shuffle manager). The logic for retrieval of individual files is replicated + * from Spark's FileShuffleBlockManager and IndexShuffleBlockManager. + * + * Executors with shuffle file consolidation are not currently supported, as the index is stored in + * the Executor's memory, unlike the IndexShuffleBlockManager. + */ +public class ExternalShuffleBlockManager { + private final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockManager.class); + + // Map from "appId-execId" to the executor's configuration. + private final ConcurrentHashMap executors = + new ConcurrentHashMap(); + + // Returns an id suitable for a single executor within a single application. + private String getAppExecId(String appId, String execId) { + return appId + "-" + execId; + } + + /** Registers a new Executor with all the configuration we need to find its shuffle files. */ + public void registerExecutor( + String appId, + String execId, + ExecutorShuffleInfo executorInfo) { + String fullId = getAppExecId(appId, execId); + logger.info("Registered executor {} with {}", fullId, executorInfo); + executors.put(fullId, executorInfo); + } + + /** + * Obtains a FileSegmentManagedBuffer from a shuffle block id. We expect the blockId has the + * format "shuffle_ShuffleId_MapId_ReduceId" (from ShuffleBlockId), and additionally make + * assumptions about how the hash and sort based shuffles store their data. + */ + public ManagedBuffer getBlockData(String appId, String execId, String blockId) { + String[] blockIdParts = blockId.split("_"); + if (blockIdParts.length < 4) { + throw new IllegalArgumentException("Unexpected block id format: " + blockId); + } else if (!blockIdParts[0].equals("shuffle")) { + throw new IllegalArgumentException("Expected shuffle block id, got: " + blockId); + } + int shuffleId = Integer.parseInt(blockIdParts[1]); + int mapId = Integer.parseInt(blockIdParts[2]); + int reduceId = Integer.parseInt(blockIdParts[3]); + + ExecutorShuffleInfo executor = executors.get(getAppExecId(appId, execId)); + if (executor == null) { + throw new RuntimeException( + String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); + } + + if ("org.apache.spark.shuffle.hash.HashShuffleManager".equals(executor.shuffleManager)) { + return getHashBasedShuffleBlockData(executor, blockId); + } else if ("org.apache.spark.shuffle.sort.SortShuffleManager".equals(executor.shuffleManager)) { + return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); + } else { + throw new UnsupportedOperationException( + "Unsupported shuffle manager: " + executor.shuffleManager); + } + } + + /** + * Hash-based shuffle data is simply stored as one file per block. + * This logic is from FileShuffleBlockManager. + */ + // TODO: Support consolidated hash shuffle files + private ManagedBuffer getHashBasedShuffleBlockData(ExecutorShuffleInfo executor, String blockId) { + File shuffleFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId); + return new FileSegmentManagedBuffer(shuffleFile, 0, shuffleFile.length()); + } + + /** + * Sort-based shuffle data uses an index called "shuffle_ShuffleId_MapId_0.index" into a data file + * called "shuffle_ShuffleId_MapId_0.data". This logic is from IndexShuffleBlockManager, + * and the block id format is from ShuffleDataBlockId and ShuffleIndexBlockId. + */ + private ManagedBuffer getSortBasedShuffleBlockData( + ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { + File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, + "shuffle_" + shuffleId + "_" + mapId + "_0.index"); + + DataInputStream in = null; + try { + in = new DataInputStream(new FileInputStream(indexFile)); + in.skipBytes(reduceId * 8); + long offset = in.readLong(); + long nextOffset = in.readLong(); + return new FileSegmentManagedBuffer( + getFile(executor.localDirs, executor.subDirsPerLocalDir, + "shuffle_" + shuffleId + "_" + mapId + "_0.data"), + offset, + nextOffset - offset); + } catch (IOException e) { + throw new RuntimeException("Failed to open file: " + indexFile, e); + } finally { + if (in != null) { + JavaUtils.closeQuietly(in); + } + } + } + + /** + * Hashes a filename into the corresponding local directory, in a manner consistent with + * Spark's DiskBlockManager.getFile(). + */ + @VisibleForTesting + static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) { + int hash = JavaUtils.nonNegativeHash(filename); + String localDir = localDirs[hash % localDirs.length]; + int subDirId = (hash / localDirs.length) % subDirsPerLocalDir; + return new File(new File(localDir, String.format("%02x", subDirId)), filename); + } + + /** For testing, clears all registered executors. */ + @VisibleForTesting + void clearRegisteredExecutors() { + executors.clear(); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java new file mode 100644 index 0000000000000..cc2f6261ca302 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.client.TransportClientFactory; +import org.apache.spark.network.server.NoOpRpcHandler; +import org.apache.spark.network.shuffle.ExternalShuffleMessages.RegisterExecutor; +import org.apache.spark.network.util.JavaUtils; +import org.apache.spark.network.util.TransportConf; + +/** + * Client for reading shuffle blocks which points to an external (outside of executor) server. + * This is instead of reading shuffle blocks directly from other executors (via + * BlockTransferService), which has the downside of losing the shuffle data if we lose the + * executors. + */ +public class ExternalShuffleClient implements ShuffleClient { + private final Logger logger = LoggerFactory.getLogger(ExternalShuffleClient.class); + + private final TransportClientFactory clientFactory; + private final String appId; + + public ExternalShuffleClient(TransportConf conf, String appId) { + TransportContext context = new TransportContext(conf, new NoOpRpcHandler()); + this.clientFactory = context.createClientFactory(); + this.appId = appId; + } + + @Override + public void fetchBlocks( + String host, + int port, + String execId, + String[] blockIds, + BlockFetchingListener listener) { + logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId); + try { + TransportClient client = clientFactory.createClient(host, port); + new OneForOneBlockFetcher(client, blockIds, listener) + .start(new ExternalShuffleMessages.OpenShuffleBlocks(appId, execId, blockIds)); + } catch (Exception e) { + logger.error("Exception while beginning fetchBlocks", e); + for (String blockId : blockIds) { + listener.onBlockFetchFailure(blockId, e); + } + } + } + + /** + * Registers this executor with an external shuffle server. This registration is required to + * inform the shuffle server about where and how we store our shuffle files. + * + * @param host Host of shuffle server. + * @param port Port of shuffle server. + * @param execId This Executor's id. + * @param executorInfo Contains all info necessary for the service to find our shuffle files. + */ + public void registerWithShuffleServer( + String host, + int port, + String execId, + ExecutorShuffleInfo executorInfo) { + TransportClient client = clientFactory.createClient(host, port); + byte[] registerExecutorMessage = + JavaUtils.serialize(new RegisterExecutor(appId, execId, executorInfo)); + client.sendRpcSync(registerExecutorMessage, 5000 /* timeoutMs */); + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java new file mode 100644 index 0000000000000..e79420ed8254f --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleMessages.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.Serializable; +import java.util.Arrays; + +import com.google.common.base.Objects; + +/** Messages handled by the {@link ExternalShuffleBlockHandler}. */ +public class ExternalShuffleMessages { + + /** Request to read a set of shuffle blocks. Returns [[ShuffleStreamHandle]]. */ + public static class OpenShuffleBlocks implements Serializable { + public final String appId; + public final String execId; + public final String[] blockIds; + + public OpenShuffleBlocks(String appId, String execId, String[] blockIds) { + this.appId = appId; + this.execId = execId; + this.blockIds = blockIds; + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId) * 41 + Arrays.hashCode(blockIds); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("blockIds", Arrays.toString(blockIds)) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof OpenShuffleBlocks) { + OpenShuffleBlocks o = (OpenShuffleBlocks) other; + return Objects.equal(appId, o.appId) + && Objects.equal(execId, o.execId) + && Arrays.equals(blockIds, o.blockIds); + } + return false; + } + } + + /** Initial registration message between an executor and its local shuffle server. */ + public static class RegisterExecutor implements Serializable { + public final String appId; + public final String execId; + public final ExecutorShuffleInfo executorInfo; + + public RegisterExecutor( + String appId, + String execId, + ExecutorShuffleInfo executorInfo) { + this.appId = appId; + this.execId = execId; + this.executorInfo = executorInfo; + } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId, executorInfo); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("executorInfo", executorInfo) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof RegisterExecutor) { + RegisterExecutor o = (RegisterExecutor) other; + return Objects.equal(appId, o.appId) + && Objects.equal(execId, o.execId) + && Objects.equal(executorInfo, o.executorInfo); + } + return false; + } + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java new file mode 100644 index 0000000000000..39b6f30f92baf --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.util.Arrays; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.util.JavaUtils; + +/** + * Simple wrapper on top of a TransportClient which interprets each chunk as a whole block, and + * invokes the BlockFetchingListener appropriately. This class is agnostic to the actual RPC + * handler, as long as there is a single "open blocks" message which returns a ShuffleStreamHandle, + * and Java serialization is used. + * + * Note that this typically corresponds to a + * {@link org.apache.spark.network.server.OneForOneStreamManager} on the server side. + */ +public class OneForOneBlockFetcher { + private final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class); + + private final TransportClient client; + private final String[] blockIds; + private final BlockFetchingListener listener; + private final ChunkReceivedCallback chunkCallback; + + private ShuffleStreamHandle streamHandle = null; + + public OneForOneBlockFetcher( + TransportClient client, + String[] blockIds, + BlockFetchingListener listener) { + if (blockIds.length == 0) { + throw new IllegalArgumentException("Zero-sized blockIds array"); + } + this.client = client; + this.blockIds = blockIds; + this.listener = listener; + this.chunkCallback = new ChunkCallback(); + } + + /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ + private class ChunkCallback implements ChunkReceivedCallback { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + // On receipt of a chunk, pass it upwards as a block. + listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer); + } + + @Override + public void onFailure(int chunkIndex, Throwable e) { + // On receipt of a failure, fail every block from chunkIndex onwards. + String[] remainingBlockIds = Arrays.copyOfRange(blockIds, chunkIndex, blockIds.length); + failRemainingBlocks(remainingBlockIds, e); + } + } + + /** + * Begins the fetching process, calling the listener with every block fetched. + * The given message will be serialized with the Java serializer, and the RPC must return a + * {@link ShuffleStreamHandle}. We will send all fetch requests immediately, without throttling. + */ + public void start(Object openBlocksMessage) { + client.sendRpc(JavaUtils.serialize(openBlocksMessage), new RpcResponseCallback() { + @Override + public void onSuccess(byte[] response) { + try { + streamHandle = JavaUtils.deserialize(response); + logger.trace("Successfully opened blocks {}, preparing to fetch chunks.", streamHandle); + + // Immediately request all chunks -- we expect that the total size of the request is + // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. + for (int i = 0; i < streamHandle.numChunks; i++) { + client.fetchChunk(streamHandle.streamId, i, chunkCallback); + } + } catch (Exception e) { + logger.error("Failed while starting block fetches", e); + failRemainingBlocks(blockIds, e); + } + } + + @Override + public void onFailure(Throwable e) { + logger.error("Failed while starting block fetches", e); + failRemainingBlocks(blockIds, e); + } + }); + } + + /** Invokes the "onBlockFetchFailure" callback for every listed block id. */ + private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { + for (String blockId : failedBlockIds) { + try { + listener.onBlockFetchFailure(blockId, e); + } catch (Exception e2) { + logger.error("Error in block fetch failure callback", e2); + } + } + } +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java new file mode 100644 index 0000000000000..9fa87c2c6e1c2 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +/** Provides an interface for reading shuffle files, either from an Executor or external service. */ +public interface ShuffleClient { + /** + * Fetch a sequence of blocks from a remote node asynchronously, + * + * Note that this API takes a sequence so the implementation can batch requests, and does not + * return a future so the underlying implementation can invoke onBlockFetchSuccess as soon as + * the data of a block is fetched, rather than waiting for all blocks to be fetched. + */ + public void fetchBlocks( + String host, + int port, + String execId, + String[] blockIds, + BlockFetchingListener listener); +} diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleStreamHandle.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleStreamHandle.java new file mode 100644 index 0000000000000..9c94691224328 --- /dev/null +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleStreamHandle.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.Serializable; +import java.util.Arrays; + +import com.google.common.base.Objects; + +/** + * Identifier for a fixed number of chunks to read from a stream created by an "open blocks" + * message. This is used by {@link OneForOneBlockFetcher}. + */ +public class ShuffleStreamHandle implements Serializable { + public final long streamId; + public final int numChunks; + + public ShuffleStreamHandle(long streamId, int numChunks) { + this.streamId = streamId; + this.numChunks = numChunks; + } + + @Override + public int hashCode() { + return Objects.hashCode(streamId, numChunks); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("streamId", streamId) + .add("numChunks", numChunks) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof ShuffleStreamHandle) { + ShuffleStreamHandle o = (ShuffleStreamHandle) other; + return Objects.equal(streamId, o.streamId) + && Objects.equal(numChunks, o.numChunks); + } + return false; + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java new file mode 100644 index 0000000000000..7939cb4d32690 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -0,0 +1,123 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.nio.ByteBuffer; +import java.util.Iterator; + +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; + +import static org.apache.spark.network.shuffle.ExternalShuffleMessages.OpenShuffleBlocks; +import static org.apache.spark.network.shuffle.ExternalShuffleMessages.RegisterExecutor; +import static org.junit.Assert.*; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.server.OneForOneStreamManager; +import org.apache.spark.network.server.RpcHandler; +import org.apache.spark.network.util.JavaUtils; + +public class ExternalShuffleBlockHandlerSuite { + TransportClient client = mock(TransportClient.class); + + OneForOneStreamManager streamManager; + ExternalShuffleBlockManager blockManager; + RpcHandler handler; + + @Before + public void beforeEach() { + streamManager = mock(OneForOneStreamManager.class); + blockManager = mock(ExternalShuffleBlockManager.class); + handler = new ExternalShuffleBlockHandler(streamManager, blockManager); + } + + @Test + public void testRegisterExecutor() { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + ExecutorShuffleInfo config = new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort"); + byte[] registerMessage = JavaUtils.serialize( + new RegisterExecutor("app0", "exec1", config)); + handler.receive(client, registerMessage, callback); + verify(blockManager, times(1)).registerExecutor("app0", "exec1", config); + + verify(callback, times(1)).onSuccess((byte[]) any()); + verify(callback, never()).onFailure((Throwable) any()); + } + + @SuppressWarnings("unchecked") + @Test + public void testOpenShuffleBlocks() { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); + ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); + when(blockManager.getBlockData("app0", "exec1", "b0")).thenReturn(block0Marker); + when(blockManager.getBlockData("app0", "exec1", "b1")).thenReturn(block1Marker); + byte[] openBlocksMessage = JavaUtils.serialize( + new OpenShuffleBlocks("app0", "exec1", new String[] { "b0", "b1" })); + handler.receive(client, openBlocksMessage, callback); + verify(blockManager, times(1)).getBlockData("app0", "exec1", "b0"); + verify(blockManager, times(1)).getBlockData("app0", "exec1", "b1"); + + ArgumentCaptor response = ArgumentCaptor.forClass(byte[].class); + verify(callback, times(1)).onSuccess(response.capture()); + verify(callback, never()).onFailure((Throwable) any()); + + ShuffleStreamHandle handle = JavaUtils.deserialize(response.getValue()); + assertEquals(2, handle.numChunks); + + ArgumentCaptor stream = ArgumentCaptor.forClass(Iterator.class); + verify(streamManager, times(1)).registerStream(stream.capture()); + Iterator buffers = (Iterator) stream.getValue(); + assertEquals(block0Marker, buffers.next()); + assertEquals(block1Marker, buffers.next()); + assertFalse(buffers.hasNext()); + } + + @Test + public void testBadMessages() { + RpcResponseCallback callback = mock(RpcResponseCallback.class); + + byte[] unserializableMessage = new byte[] { 0x12, 0x34, 0x56 }; + try { + handler.receive(client, unserializableMessage, callback); + fail("Should have thrown"); + } catch (Exception e) { + // pass + } + + byte[] unexpectedMessage = JavaUtils.serialize( + new ExecutorShuffleInfo(new String[] {"/a", "/b"}, 16, "sort")); + try { + handler.receive(client, unexpectedMessage, callback); + fail("Should have thrown"); + } catch (UnsupportedOperationException e) { + // pass + } + + verify(callback, never()).onSuccess((byte[]) any()); + verify(callback, never()).onFailure((Throwable) any()); + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java new file mode 100644 index 0000000000000..da54797e8923c --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockManagerSuite.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; + +import com.google.common.io.CharStreams; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class ExternalShuffleBlockManagerSuite { + static String sortBlock0 = "Hello!"; + static String sortBlock1 = "World!"; + + static String hashBlock0 = "Elementary"; + static String hashBlock1 = "Tabular"; + + static TestShuffleDataContext dataContext; + + @BeforeClass + public static void beforeAll() throws IOException { + dataContext = new TestShuffleDataContext(2, 5); + + dataContext.create(); + // Write some sort and hash data. + dataContext.insertSortShuffleData(0, 0, + new byte[][] { sortBlock0.getBytes(), sortBlock1.getBytes() } ); + dataContext.insertHashShuffleData(1, 0, + new byte[][] { hashBlock0.getBytes(), hashBlock1.getBytes() } ); + } + + @AfterClass + public static void afterAll() { + dataContext.cleanup(); + } + + @Test + public void testBadRequests() { + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(); + // Unregistered executor + try { + manager.getBlockData("app0", "exec1", "shuffle_1_1_0"); + fail("Should have failed"); + } catch (RuntimeException e) { + assertTrue("Bad error message: " + e, e.getMessage().contains("not registered")); + } + + // Invalid shuffle manager + manager.registerExecutor("app0", "exec2", dataContext.createExecutorInfo("foobar")); + try { + manager.getBlockData("app0", "exec2", "shuffle_1_1_0"); + fail("Should have failed"); + } catch (UnsupportedOperationException e) { + // pass + } + + // Nonexistent shuffle block + manager.registerExecutor("app0", "exec3", + dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); + try { + manager.getBlockData("app0", "exec3", "shuffle_1_1_0"); + fail("Should have failed"); + } catch (Exception e) { + // pass + } + } + + @Test + public void testSortShuffleBlocks() throws IOException { + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(); + manager.registerExecutor("app0", "exec0", + dataContext.createExecutorInfo("org.apache.spark.shuffle.sort.SortShuffleManager")); + + InputStream block0Stream = + manager.getBlockData("app0", "exec0", "shuffle_0_0_0").createInputStream(); + String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); + block0Stream.close(); + assertEquals(sortBlock0, block0); + + InputStream block1Stream = + manager.getBlockData("app0", "exec0", "shuffle_0_0_1").createInputStream(); + String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); + block1Stream.close(); + assertEquals(sortBlock1, block1); + } + + @Test + public void testHashShuffleBlocks() throws IOException { + ExternalShuffleBlockManager manager = new ExternalShuffleBlockManager(); + manager.registerExecutor("app0", "exec0", + dataContext.createExecutorInfo("org.apache.spark.shuffle.hash.HashShuffleManager")); + + InputStream block0Stream = + manager.getBlockData("app0", "exec0", "shuffle_1_0_0").createInputStream(); + String block0 = CharStreams.toString(new InputStreamReader(block0Stream)); + block0Stream.close(); + assertEquals(hashBlock0, block0); + + InputStream block1Stream = + manager.getBlockData("app0", "exec0", "shuffle_1_0_1").createInputStream(); + String block1 = CharStreams.toString(new InputStreamReader(block1Stream)); + block1Stream.close(); + assertEquals(hashBlock1, block1); + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java new file mode 100644 index 0000000000000..b3bcf5fd68e73 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.TestUtils; +import org.apache.spark.network.TransportContext; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.SystemPropertyConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class ExternalShuffleIntegrationSuite { + + static String APP_ID = "app-id"; + static String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; + static String HASH_MANAGER = "org.apache.spark.shuffle.hash.HashShuffleManager"; + + // Executor 0 is sort-based + static TestShuffleDataContext dataContext0; + // Executor 1 is hash-based + static TestShuffleDataContext dataContext1; + + static ExternalShuffleBlockHandler handler; + static TransportServer server; + static TransportConf conf; + + static byte[][] exec0Blocks = new byte[][] { + new byte[123], + new byte[12345], + new byte[1234567], + }; + + static byte[][] exec1Blocks = new byte[][] { + new byte[321], + new byte[54321], + }; + + @BeforeClass + public static void beforeAll() throws IOException { + Random rand = new Random(); + + for (byte[] block : exec0Blocks) { + rand.nextBytes(block); + } + for (byte[] block: exec1Blocks) { + rand.nextBytes(block); + } + + dataContext0 = new TestShuffleDataContext(2, 5); + dataContext0.create(); + dataContext0.insertSortShuffleData(0, 0, exec0Blocks); + + dataContext1 = new TestShuffleDataContext(6, 2); + dataContext1.create(); + dataContext1.insertHashShuffleData(1, 0, exec1Blocks); + + conf = new TransportConf(new SystemPropertyConfigProvider()); + handler = new ExternalShuffleBlockHandler(); + TransportContext transportContext = new TransportContext(conf, handler); + server = transportContext.createServer(); + } + + @AfterClass + public static void afterAll() { + dataContext0.cleanup(); + dataContext1.cleanup(); + server.close(); + } + + @After + public void afterEach() { + handler.clearRegisteredExecutors(); + } + + class FetchResult { + public Set successBlocks; + public Set failedBlocks; + public List buffers; + + public void releaseBuffers() { + for (ManagedBuffer buffer : buffers) { + buffer.release(); + } + } + } + + // Fetch a set of blocks from a pre-registered executor. + private FetchResult fetchBlocks(String execId, String[] blockIds) throws Exception { + return fetchBlocks(execId, blockIds, server.getPort()); + } + + // Fetch a set of blocks from a pre-registered executor. Connects to the server on the given port, + // to allow connecting to invalid servers. + private FetchResult fetchBlocks(String execId, String[] blockIds, int port) throws Exception { + final FetchResult res = new FetchResult(); + res.successBlocks = Collections.synchronizedSet(new HashSet()); + res.failedBlocks = Collections.synchronizedSet(new HashSet()); + res.buffers = Collections.synchronizedList(new LinkedList()); + + final Semaphore requestsRemaining = new Semaphore(0); + + ExternalShuffleClient client = new ExternalShuffleClient(conf, APP_ID); + client.fetchBlocks(TestUtils.getLocalHost(), port, execId, blockIds, + new BlockFetchingListener() { + @Override + public void onBlockFetchSuccess(String blockId, ManagedBuffer data) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + data.retain(); + res.successBlocks.add(blockId); + res.buffers.add(data); + requestsRemaining.release(); + } + } + } + + @Override + public void onBlockFetchFailure(String blockId, Throwable exception) { + synchronized (this) { + if (!res.successBlocks.contains(blockId) && !res.failedBlocks.contains(blockId)) { + res.failedBlocks.add(blockId); + requestsRemaining.release(); + } + } + } + }); + + if (!requestsRemaining.tryAcquire(blockIds.length, 5, TimeUnit.SECONDS)) { + fail("Timeout getting response from the server"); + } + return res; + } + + @Test + public void testFetchOneSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", new String[] { "shuffle_0_0_0" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0"), exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks[0])); + exec0Fetch.releaseBuffers(); + } + + @Test + public void testFetchThreeSort() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult exec0Fetch = fetchBlocks("exec-0", + new String[] { "shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2" }); + assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_0_0_1", "shuffle_0_0_2"), + exec0Fetch.successBlocks); + assertTrue(exec0Fetch.failedBlocks.isEmpty()); + assertBufferListsEqual(exec0Fetch.buffers, Lists.newArrayList(exec0Blocks)); + exec0Fetch.releaseBuffers(); + } + + @Test + public void testFetchHash() throws Exception { + registerExecutor("exec-1", dataContext1.createExecutorInfo(HASH_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-1", + new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }); + assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.successBlocks); + assertTrue(execFetch.failedBlocks.isEmpty()); + assertBufferListsEqual(execFetch.buffers, Lists.newArrayList(exec1Blocks)); + execFetch.releaseBuffers(); + } + + @Test + public void testFetchWrongShuffle() throws Exception { + registerExecutor("exec-1", dataContext1.createExecutorInfo(SORT_MANAGER /* wrong manager */)); + FetchResult execFetch = fetchBlocks("exec-1", + new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + } + + @Test + public void testFetchInvalidShuffle() throws Exception { + registerExecutor("exec-1", dataContext1.createExecutorInfo("unknown sort manager")); + FetchResult execFetch = fetchBlocks("exec-1", + new String[] { "shuffle_1_0_0" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_1_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchWrongBlockId() throws Exception { + registerExecutor("exec-1", dataContext1.createExecutorInfo(SORT_MANAGER /* wrong manager */)); + FetchResult execFetch = fetchBlocks("exec-1", + new String[] { "rdd_1_0_0" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchNonexistent() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-0", + new String[] { "shuffle_2_0_0" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_2_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchWrongExecutor() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-0", + new String[] { "shuffle_0_0_0" /* right */, "shuffle_1_0_0" /* wrong */ }); + // Both still fail, as we start by checking for all block. + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchUnregisteredExecutor() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-2", + new String[] { "shuffle_0_0_0", "shuffle_1_0_0" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_0_0_0", "shuffle_1_0_0"), execFetch.failedBlocks); + } + + @Test + public void testFetchNoServer() throws Exception { + registerExecutor("exec-0", dataContext0.createExecutorInfo(SORT_MANAGER)); + FetchResult execFetch = fetchBlocks("exec-0", + new String[] { "shuffle_1_0_0", "shuffle_1_0_1" }, 1 /* port */); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("shuffle_1_0_0", "shuffle_1_0_1"), execFetch.failedBlocks); + } + + private void registerExecutor(String executorId, ExecutorShuffleInfo executorInfo) { + ExternalShuffleClient client = new ExternalShuffleClient(conf, APP_ID); + client.registerWithShuffleServer(TestUtils.getLocalHost(), server.getPort(), + executorId, executorInfo); + } + + private void assertBufferListsEqual(List list0, List list1) + throws Exception { + assertEquals(list0.size(), list1.size()); + for (int i = 0; i < list0.size(); i ++) { + assertBuffersEqual(list0.get(i), new NioManagedBuffer(ByteBuffer.wrap(list1.get(i)))); + } + } + + private void assertBuffersEqual(ManagedBuffer buffer0, ManagedBuffer buffer1) throws Exception { + ByteBuffer nio0 = buffer0.nioByteBuffer(); + ByteBuffer nio1 = buffer1.nioByteBuffer(); + + int len = nio0.remaining(); + assertEquals(nio0.remaining(), nio1.remaining()); + for (int i = 0; i < len; i ++) { + assertEquals(nio0.get(), nio1.get()); + } + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java new file mode 100644 index 0000000000000..c18346f6966d6 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +import com.google.common.collect.Maps; +import io.netty.buffer.Unpooled; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +import static org.junit.Assert.*; +import static org.junit.Assert.assertEquals; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.*; + +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.buffer.NettyManagedBuffer; +import org.apache.spark.network.buffer.NioManagedBuffer; +import org.apache.spark.network.client.ChunkReceivedCallback; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.client.TransportClient; +import org.apache.spark.network.util.JavaUtils; + +public class OneForOneBlockFetcherSuite { + @Test + public void testFetchOne() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + + BlockFetchingListener listener = fetchBlocks(blocks); + + verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); + } + + @Test + public void testFetchThree() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + + BlockFetchingListener listener = fetchBlocks(blocks); + + for (int i = 0; i < 3; i ++) { + verify(listener, times(1)).onBlockFetchSuccess("b" + i, blocks.get("b" + i)); + } + } + + @Test + public void testFailure() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", null); + blocks.put("b2", null); + + BlockFetchingListener listener = fetchBlocks(blocks); + + // Each failure will cause a failure to be invoked in all remaining block fetches. + verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, times(2)).onBlockFetchFailure(eq("b2"), (Throwable) any()); + } + + @Test + public void testFailureAndSuccess() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("b1", null); + blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[21]))); + + BlockFetchingListener listener = fetchBlocks(blocks); + + // We may call both success and failure for the same block. + verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); + verify(listener, times(1)).onBlockFetchFailure(eq("b1"), (Throwable) any()); + verify(listener, times(1)).onBlockFetchSuccess("b2", blocks.get("b2")); + verify(listener, times(1)).onBlockFetchFailure(eq("b2"), (Throwable) any()); + } + + @Test + public void testEmptyBlockFetch() { + try { + fetchBlocks(Maps.newLinkedHashMap()); + fail(); + } catch (IllegalArgumentException e) { + assertEquals("Zero-sized blockIds array", e.getMessage()); + } + } + + /** + * Begins a fetch on the given set of blocks by mocking out the server side of the RPC which + * simply returns the given (BlockId, Block) pairs. + * As "blocks" is a LinkedHashMap, the blocks are guaranteed to be returned in the same order + * that they were inserted in. + * + * If a block's buffer is "null", an exception will be thrown instead. + */ + private BlockFetchingListener fetchBlocks(final LinkedHashMap blocks) { + TransportClient client = mock(TransportClient.class); + BlockFetchingListener listener = mock(BlockFetchingListener.class); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + OneForOneBlockFetcher fetcher = new OneForOneBlockFetcher(client, blockIds, listener); + + // Respond to the "OpenBlocks" message with an appropirate ShuffleStreamHandle with streamId 123 + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) throws Throwable { + String message = JavaUtils.deserialize((byte[]) invocationOnMock.getArguments()[0]); + RpcResponseCallback callback = (RpcResponseCallback) invocationOnMock.getArguments()[1]; + callback.onSuccess(JavaUtils.serialize(new ShuffleStreamHandle(123, blocks.size()))); + assertEquals("OpenZeBlocks", message); + return null; + } + }).when(client).sendRpc((byte[]) any(), (RpcResponseCallback) any()); + + // Respond to each chunk request with a single buffer from our blocks array. + final AtomicInteger expectedChunkIndex = new AtomicInteger(0); + final Iterator blockIterator = blocks.values().iterator(); + doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + try { + long streamId = (Long) invocation.getArguments()[0]; + int myChunkIndex = (Integer) invocation.getArguments()[1]; + assertEquals(123, streamId); + assertEquals(expectedChunkIndex.getAndIncrement(), myChunkIndex); + + ChunkReceivedCallback callback = (ChunkReceivedCallback) invocation.getArguments()[2]; + ManagedBuffer result = blockIterator.next(); + if (result != null) { + callback.onSuccess(myChunkIndex, result); + } else { + callback.onFailure(myChunkIndex, new RuntimeException("Failed " + myChunkIndex)); + } + } catch (Exception e) { + e.printStackTrace(); + fail("Unexpected failure"); + } + return null; + } + }).when(client).fetchChunk(anyLong(), anyInt(), (ChunkReceivedCallback) any()); + + fetcher.start("OpenZeBlocks"); + return listener; + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleMessagesSuite.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleMessagesSuite.java new file mode 100644 index 0000000000000..ee9482b49cfc3 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/ShuffleMessagesSuite.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import org.junit.Test; + +import static org.junit.Assert.*; + +import org.apache.spark.network.util.JavaUtils; + +import static org.apache.spark.network.shuffle.ExternalShuffleMessages.*; + +public class ShuffleMessagesSuite { + @Test + public void serializeOpenShuffleBlocks() { + OpenShuffleBlocks msg = new OpenShuffleBlocks("app-1", "exec-2", + new String[] { "block0", "block1" }); + OpenShuffleBlocks msg2 = JavaUtils.deserialize(JavaUtils.serialize(msg)); + assertEquals(msg, msg2); + } + + @Test + public void serializeRegisterExecutor() { + RegisterExecutor msg = new RegisterExecutor("app-1", "exec-2", new ExecutorShuffleInfo( + new String[] { "/local1", "/local2" }, 32, "MyShuffleManager")); + RegisterExecutor msg2 = JavaUtils.deserialize(JavaUtils.serialize(msg)); + assertEquals(msg, msg2); + } + + @Test + public void serializeShuffleStreamHandle() { + ShuffleStreamHandle msg = new ShuffleStreamHandle(12345, 16); + ShuffleStreamHandle msg2 = JavaUtils.deserialize(JavaUtils.serialize(msg)); + assertEquals(msg, msg2); + } +} diff --git a/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java new file mode 100644 index 0000000000000..442b756467442 --- /dev/null +++ b/network/shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +import com.google.common.io.Files; + +/** + * Manages some sort- and hash-based shuffle data, including the creation + * and cleanup of directories that can be read by the {@link ExternalShuffleBlockManager}. + */ +public class TestShuffleDataContext { + private final String[] localDirs; + private final int subDirsPerLocalDir; + + public TestShuffleDataContext(int numLocalDirs, int subDirsPerLocalDir) { + this.localDirs = new String[numLocalDirs]; + this.subDirsPerLocalDir = subDirsPerLocalDir; + } + + public void create() { + for (int i = 0; i < localDirs.length; i ++) { + localDirs[i] = Files.createTempDir().getAbsolutePath(); + + for (int p = 0; p < subDirsPerLocalDir; p ++) { + new File(localDirs[i], String.format("%02x", p)).mkdirs(); + } + } + } + + public void cleanup() { + for (String localDir : localDirs) { + deleteRecursively(new File(localDir)); + } + } + + /** Creates reducer blocks in a sort-based data format within our local dirs. */ + public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_0"; + + OutputStream dataStream = new FileOutputStream( + ExternalShuffleBlockManager.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); + DataOutputStream indexStream = new DataOutputStream(new FileOutputStream( + ExternalShuffleBlockManager.getFile(localDirs, subDirsPerLocalDir, blockId + ".index"))); + + long offset = 0; + indexStream.writeLong(offset); + for (byte[] block : blocks) { + offset += block.length; + dataStream.write(block); + indexStream.writeLong(offset); + } + + dataStream.close(); + indexStream.close(); + } + + /** Creates reducer blocks in a hash-based data format within our local dirs. */ + public void insertHashShuffleData(int shuffleId, int mapId, byte[][] blocks) throws IOException { + for (int i = 0; i < blocks.length; i ++) { + String blockId = "shuffle_" + shuffleId + "_" + mapId + "_" + i; + Files.write(blocks[i], + ExternalShuffleBlockManager.getFile(localDirs, subDirsPerLocalDir, blockId)); + } + } + + /** + * Creates an ExecutorShuffleInfo object based on the given shuffle manager which targets this + * context's directories. + */ + public ExecutorShuffleInfo createExecutorInfo(String shuffleManager) { + return new ExecutorShuffleInfo(localDirs, subDirsPerLocalDir, shuffleManager); + } + + private static void deleteRecursively(File f) { + assert f != null; + if (f.isDirectory()) { + File[] children = f.listFiles(); + if (children != null) { + for (File child : children) { + deleteRecursively(child); + } + } + } + f.delete(); + } +} diff --git a/pom.xml b/pom.xml index 4c7806c416e34..61a508a0ea592 100644 --- a/pom.xml +++ b/pom.xml @@ -92,6 +92,7 @@ mllib tools network/common + network/shuffle streaming sql/catalyst sql/core diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 77083518bbab3..33618f5401768 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -31,11 +31,12 @@ object BuildCommons { private val buildLocation = file(".").getAbsoluteFile.getParentFile val allProjects@Seq(bagel, catalyst, core, graphx, hive, hiveThriftServer, mllib, repl, - sql, networkCommon, streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingMqtt, - streamingTwitter, streamingZeromq) = + sql, networkCommon, networkShuffle, streaming, streamingFlumeSink, streamingFlume, streamingKafka, + streamingMqtt, streamingTwitter, streamingZeromq) = Seq("bagel", "catalyst", "core", "graphx", "hive", "hive-thriftserver", "mllib", "repl", - "sql", "network-common", "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka", - "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) + "sql", "network-common", "network-shuffle", "streaming", "streaming-flume-sink", + "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", + "streaming-zeromq").map(ProjectRef(buildLocation, _)) val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, sparkGangliaLgpl, sparkKinesisAsl) = Seq("yarn", "yarn-stable", "yarn-alpha", "java8-tests", "ganglia-lgpl", "kinesis-asl") @@ -142,7 +143,7 @@ object SparkBuild extends PomBuild { // TODO: Add Sql to mima checks allProjects.filterNot(x => Seq(spark, sql, hive, hiveThriftServer, catalyst, repl, - streamingFlumeSink, networkCommon).contains(x)).foreach { + streamingFlumeSink, networkCommon, networkShuffle).contains(x)).foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) } From ad0fde10b2285e780349be5a8f333db0974a502f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sat, 1 Nov 2014 15:03:11 -0700 Subject: [PATCH 1179/1492] [SPARK-4037][SQL] Removes the SessionState instance created in HiveThriftServer2 `HiveThriftServer2` creates a global singleton `SessionState` instance and overrides `HiveContext` to inject the `SessionState` object. This messes up `SessionState` initialization and causes problems. This PR replaces the global `SessionState` with `HiveContext.sessionState` to avoid the initialization conflict. Also `HiveContext` reuses existing started `SessionState` if any (this is required by `SparkSQLCLIDriver`, which uses specialized `CliSessionState`). Author: Cheng Lian Closes #2887 from liancheng/spark-4037 and squashes the following commits: 8446675 [Cheng Lian] Removes redundant Driver initialization a28fef5 [Cheng Lian] Avoid starting HiveContext.sessionState multiple times 49b1c5b [Cheng Lian] Reuses existing started SessionState if any 3cd6fab [Cheng Lian] Fixes SPARK-4037 --- .../hive/thriftserver/HiveThriftServer2.scala | 17 +------ .../sql/hive/thriftserver/SparkSQLEnv.scala | 18 +++----- .../thriftserver/HiveThriftServer2Suite.scala | 10 +++-- .../apache/spark/sql/hive/HiveContext.scala | 44 +++++++++++++------ 4 files changed, 44 insertions(+), 45 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 3d468d804622c..bd4e99492b395 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -17,11 +17,8 @@ package org.apache.spark.sql.hive.thriftserver -import scala.collection.JavaConversions._ - import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} @@ -51,24 +48,12 @@ object HiveThriftServer2 extends Logging { def main(args: Array[String]) { val optionsProcessor = new ServerOptionsProcessor("HiveThriftServer2") - if (!optionsProcessor.process(args)) { System.exit(-1) } - val ss = new SessionState(new HiveConf(classOf[SessionState])) - - // Set all properties specified via command line. - val hiveConf: HiveConf = ss.getConf - hiveConf.getAllProperties.toSeq.sortBy(_._1).foreach { case (k, v) => - logDebug(s"HiveConf var: $k=$v") - } - - SessionState.start(ss) - logInfo("Starting SparkContext") SparkSQLEnv.init() - SessionState.start(ss) Runtime.getRuntime.addShutdownHook( new Thread() { @@ -80,7 +65,7 @@ object HiveThriftServer2 extends Logging { try { val server = new HiveThriftServer2(SparkSQLEnv.hiveContext) - server.init(hiveConf) + server.init(SparkSQLEnv.hiveContext.hiveconf) server.start() logInfo("HiveThriftServer2 started") } catch { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 2136a2ea63543..50425863518c3 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.hive.thriftserver -import org.apache.hadoop.hive.ql.session.SessionState - -import org.apache.spark.scheduler.{SplitInfo, StatsReportListener} -import org.apache.spark.Logging +import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} +import scala.collection.JavaConversions._ /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { @@ -37,14 +35,12 @@ private[hive] object SparkSQLEnv extends Logging { .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) sparkContext.addSparkListener(new StatsReportListener()) + hiveContext = new HiveContext(sparkContext) - hiveContext = new HiveContext(sparkContext) { - @transient override lazy val sessionState = { - val state = SessionState.get() - setConf(state.getConf.getAllProperties) - state + if (log.isDebugEnabled) { + hiveContext.hiveconf.getAllProperties.toSeq.sorted.foreach { case (k, v) => + logDebug(s"HiveConf var: $k=$v") } - @transient override lazy val hiveconf = sessionState.getConf } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index e3b4e45a3d68c..c60e8fa5b1259 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -150,10 +150,12 @@ class HiveThriftServer2Suite extends FunSuite with Logging { val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") - val queries = Seq( - "CREATE TABLE test(key INT, val STRING)", - s"LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test", - "CACHE TABLE test") + val queries = + s"""SET spark.sql.shuffle.partitions=3; + |CREATE TABLE test(key INT, val STRING); + |LOAD DATA LOCAL INPATH '$dataFilePath' OVERWRITE INTO TABLE test; + |CACHE TABLE test; + """.stripMargin.split(";").map(_.trim).filter(_.nonEmpty) queries.foreach(statement.execute) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index fad4091d48a89..ff8fa44194d98 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -224,21 +224,29 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } /** - * SQLConf and HiveConf contracts: when the hive session is first initialized, params in - * HiveConf will get picked up by the SQLConf. Additionally, any properties set by - * set() or a SET command inside sql() will be set in the SQLConf *as well as* - * in the HiveConf. + * SQLConf and HiveConf contracts: + * + * 1. reuse existing started SessionState if any + * 2. when the Hive session is first initialized, params in HiveConf will get picked up by the + * SQLConf. Additionally, any properties set by set() or a SET command inside sql() will be + * set in the SQLConf *as well as* in the HiveConf. */ - @transient lazy val hiveconf = new HiveConf(classOf[SessionState]) - @transient protected[hive] lazy val sessionState = { - val ss = new SessionState(hiveconf) - setConf(hiveconf.getAllProperties) // Have SQLConf pick up the initial set of HiveConf. - SessionState.start(ss) - ss.err = new PrintStream(outputBuffer, true, "UTF-8") - ss.out = new PrintStream(outputBuffer, true, "UTF-8") - - ss - } + @transient protected[hive] lazy val (hiveconf, sessionState) = + Option(SessionState.get()) + .orElse { + val newState = new SessionState(new HiveConf(classOf[SessionState])) + // Only starts newly created `SessionState` instance. Any existing `SessionState` instance + // returned by `SessionState.get()` must be the most recently started one. + SessionState.start(newState) + Some(newState) + } + .map { state => + setConf(state.getConf.getAllProperties) + if (state.out == null) state.out = new PrintStream(outputBuffer, true, "UTF-8") + if (state.err == null) state.err = new PrintStream(outputBuffer, true, "UTF-8") + (state.getConf, state) + } + .get override def setConf(key: String, value: String): Unit = { super.setConf(key, value) @@ -288,6 +296,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hiveconf) + // Makes sure the session represented by the `sessionState` field is activated. This implies + // Spark SQL Hive support uses a single `SessionState` for all Hive operations and breaks + // session isolation under multi-user scenarios (i.e. HiveThriftServer2). + // TODO Fix session isolation + if (SessionState.get() != sessionState) { + SessionState.start(sessionState) + } + proc match { case driver: Driver => val results = HiveShim.createDriverResultsArray From 7894de276b8d0b0e4efc654d0b254fc2a6f6077c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 1 Nov 2014 15:18:58 -0700 Subject: [PATCH 1180/1492] Revert "[SPARK-4183] Enable NettyBlockTransferService by default" This reverts commit 59e626c701227634336110e1bc23afd94c535ede. --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 2 +- docs/configuration.md | 10 ---------- 2 files changed, 1 insertion(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index e2f13accdfab5..7fb2b91377076 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -274,7 +274,7 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) val blockTransferService = - conf.get("spark.shuffle.blockTransferService", "netty").toLowerCase match { + conf.get("spark.shuffle.blockTransferService", "nio").toLowerCase match { case "netty" => new NettyBlockTransferService(conf) case "nio" => diff --git a/docs/configuration.md b/docs/configuration.md index 78c4bf332c5d1..3007706a2586e 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -359,16 +359,6 @@ Apart from these, the following properties are also available, and may be useful map-side aggregation and there are at most this many reduce partitions. - - spark.shuffle.blockTransferService - netty - - Implementation to use for transferring shuffle and cached blocks between executors. There - are two implementations available: netty and nio. Netty-based - block transfer is intended to be simpler but equally efficient and is the default option - starting in 1.2. - - #### Spark UI From d8176b1c2f22247ee724041aefa1af9118cf861d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sat, 1 Nov 2014 15:21:36 -0700 Subject: [PATCH 1181/1492] [SPARK-4121] Set commons-math3 version based on hadoop profiles, instead of shading In #2928 , we shade commons-math3 to prevent future conflicts with hadoop. It caused problems with our Jenkins master build with maven. Some tests used local-cluster mode, where the assembly jar contains relocated math3 classes, while mllib test code still compiles with core and the untouched math3 classes. This PR sets commons-math3 version based on hadoop profiles. pwendell JoshRosen srowen Author: Xiangrui Meng Closes #3023 from mengxr/SPARK-4121-alt and squashes the following commits: 580f6d9 [Xiangrui Meng] replace tab by spaces 7f71f08 [Xiangrui Meng] revert changes to PoissonSampler to avoid conflicts d3353d9 [Xiangrui Meng] do not shade commons-math3 b4180dc [Xiangrui Meng] temp work --- assembly/pom.xml | 4 ---- mllib/pom.xml | 4 ++++ pom.xml | 9 ++++++++- 3 files changed, 12 insertions(+), 5 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9e8525dd46157..31a01e4d8e1de 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -146,10 +146,6 @@ com/google/common/base/Present* - - org.apache.commons.math3 - org.spark-project.commons.math3 - diff --git a/mllib/pom.xml b/mllib/pom.xml index de062a4901596..fb7239e779aae 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -65,6 +65,10 @@ junit junit + + org.apache.commons + commons-math3 + diff --git a/pom.xml b/pom.xml index 61a508a0ea592..6191cd3a541e2 100644 --- a/pom.xml +++ b/pom.xml @@ -145,6 +145,7 @@ 1.8.3 1.1.0 4.2.6 + 3.1.1 64m 512m @@ -318,7 +319,7 @@ org.apache.commons commons-math3 - 3.3 + ${commons.math3.version} com.google.code.findbugs @@ -1175,6 +1176,10 @@ + + hadoop-0.23 @@ -1204,6 +1209,7 @@ 2.3.0 2.5.0 0.9.0 + 3.1.1 hadoop2 @@ -1214,6 +1220,7 @@ 2.4.0 2.5.0 0.9.0 + 3.1.1 hadoop2 From 56f2c61cde3f5d906c2a58e9af1a661222f2c679 Mon Sep 17 00:00:00 2001 From: Sung Chung Date: Sat, 1 Nov 2014 16:58:26 -0700 Subject: [PATCH 1182/1492] [SPARK-3161][MLLIB] Adding a node Id caching mechanism for training deci... ...sion trees. jkbradley mengxr chouqin Please review this. Author: Sung Chung Closes #2868 from codedeft/SPARK-3161 and squashes the following commits: 5f5a156 [Sung Chung] [SPARK-3161][MLLIB] Adding a node Id caching mechanism for training decision trees. --- .../examples/mllib/DecisionTreeRunner.scala | 25 ++- .../spark/mllib/tree/DecisionTree.scala | 114 ++++++++-- .../spark/mllib/tree/RandomForest.scala | 22 +- .../mllib/tree/configuration/Strategy.scala | 12 +- .../spark/mllib/tree/impl/NodeIdCache.scala | 204 ++++++++++++++++++ .../spark/mllib/tree/RandomForestSuite.scala | 69 ++++-- 6 files changed, 405 insertions(+), 41 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index f98730366bcb7..49751a30491d0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -62,7 +62,10 @@ object DecisionTreeRunner { minInfoGain: Double = 0.0, numTrees: Int = 1, featureSubsetStrategy: String = "auto", - fracTest: Double = 0.2) extends AbstractParams[Params] + fracTest: Double = 0.2, + useNodeIdCache: Boolean = false, + checkpointDir: Option[String] = None, + checkpointInterval: Int = 10) extends AbstractParams[Params] def main(args: Array[String]) { val defaultParams = Params() @@ -102,6 +105,21 @@ object DecisionTreeRunner { .text(s"fraction of data to hold out for testing. If given option testInput, " + s"this option is ignored. default: ${defaultParams.fracTest}") .action((x, c) => c.copy(fracTest = x)) + opt[Boolean]("useNodeIdCache") + .text(s"whether to use node Id cache during training, " + + s"default: ${defaultParams.useNodeIdCache}") + .action((x, c) => c.copy(useNodeIdCache = x)) + opt[String]("checkpointDir") + .text(s"checkpoint directory where intermediate node Id caches will be stored, " + + s"default: ${defaultParams.checkpointDir match { + case Some(strVal) => strVal + case None => "None" + }}") + .action((x, c) => c.copy(checkpointDir = Some(x))) + opt[Int]("checkpointInterval") + .text(s"how often to checkpoint the node Id cache, " + + s"default: ${defaultParams.checkpointInterval}") + .action((x, c) => c.copy(checkpointInterval = x)) opt[String]("testInput") .text(s"input path to test dataset. If given, option fracTest is ignored." + s" default: ${defaultParams.testInput}") @@ -236,7 +254,10 @@ object DecisionTreeRunner { maxBins = params.maxBins, numClassesForClassification = numClasses, minInstancesPerNode = params.minInstancesPerNode, - minInfoGain = params.minInfoGain) + minInfoGain = params.minInfoGain, + useNodeIdCache = params.useNodeIdCache, + checkpointDir = params.checkpointDir, + checkpointInterval = params.checkpointInterval) if (params.numTrees == 1) { val startTime = System.nanoTime() val model = DecisionTree.train(training, strategy) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 752ed59a030d3..78acc17f901c1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -437,6 +437,11 @@ object DecisionTree extends Serializable with Logging { * @param bins possible bins for all features, indexed (numFeatures)(numBins) * @param nodeQueue Queue of nodes to split, with values (treeIndex, node). * Updated with new non-leaf nodes which are created. + * @param nodeIdCache Node Id cache containing an RDD of Array[Int] where + * each value in the array is the data point's node Id + * for a corresponding tree. This is used to prevent the need + * to pass the entire tree to the executors during + * the node stat aggregation phase. */ private[tree] def findBestSplits( input: RDD[BaggedPoint[TreePoint]], @@ -447,7 +452,8 @@ object DecisionTree extends Serializable with Logging { splits: Array[Array[Split]], bins: Array[Array[Bin]], nodeQueue: mutable.Queue[(Int, Node)], - timer: TimeTracker = new TimeTracker): Unit = { + timer: TimeTracker = new TimeTracker, + nodeIdCache: Option[NodeIdCache] = None): Unit = { /* * The high-level descriptions of the best split optimizations are noted here. @@ -479,6 +485,37 @@ object DecisionTree extends Serializable with Logging { logDebug("isMulticlass = " + metadata.isMulticlass) logDebug("isMulticlassWithCategoricalFeatures = " + metadata.isMulticlassWithCategoricalFeatures) + logDebug("using nodeIdCache = " + nodeIdCache.nonEmpty.toString) + + /** + * Performs a sequential aggregation over a partition for a particular tree and node. + * + * For each feature, the aggregate sufficient statistics are updated for the relevant + * bins. + * + * @param treeIndex Index of the tree that we want to perform aggregation for. + * @param nodeInfo The node info for the tree node. + * @param agg Array storing aggregate calculation, with a set of sufficient statistics + * for each (node, feature, bin). + * @param baggedPoint Data point being aggregated. + */ + def nodeBinSeqOp( + treeIndex: Int, + nodeInfo: RandomForest.NodeIndexInfo, + agg: Array[DTStatsAggregator], + baggedPoint: BaggedPoint[TreePoint]): Unit = { + if (nodeInfo != null) { + val aggNodeIndex = nodeInfo.nodeIndexInGroup + val featuresForNode = nodeInfo.featureSubset + val instanceWeight = baggedPoint.subsampleWeights(treeIndex) + if (metadata.unorderedFeatures.isEmpty) { + orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) + } else { + mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, bins, metadata.unorderedFeatures, + instanceWeight, featuresForNode) + } + } + } /** * Performs a sequential aggregation over a partition. @@ -497,20 +534,25 @@ object DecisionTree extends Serializable with Logging { treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => val nodeIndex = predictNodeIndex(topNodes(treeIndex), baggedPoint.datum.binnedFeatures, bins, metadata.unorderedFeatures) - val nodeInfo = nodeIndexToInfo.getOrElse(nodeIndex, null) - // If the example does not reach a node in this group, then nodeIndex = null. - if (nodeInfo != null) { - val aggNodeIndex = nodeInfo.nodeIndexInGroup - val featuresForNode = nodeInfo.featureSubset - val instanceWeight = baggedPoint.subsampleWeights(treeIndex) - if (metadata.unorderedFeatures.isEmpty) { - orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) - } else { - mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, bins, metadata.unorderedFeatures, - instanceWeight, featuresForNode) - } - } + nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null), agg, baggedPoint) + } + + agg + } + + /** + * Do the same thing as binSeqOp, but with nodeIdCache. + */ + def binSeqOpWithNodeIdCache( + agg: Array[DTStatsAggregator], + dataPoint: (BaggedPoint[TreePoint], Array[Int])): Array[DTStatsAggregator] = { + treeToNodeToIndexInfo.foreach { case (treeIndex, nodeIndexToInfo) => + val baggedPoint = dataPoint._1 + val nodeIdCache = dataPoint._2 + val nodeIndex = nodeIdCache(treeIndex) + nodeBinSeqOp(treeIndex, nodeIndexToInfo.getOrElse(nodeIndex, null), agg, baggedPoint) } + agg } @@ -553,7 +595,26 @@ object DecisionTree extends Serializable with Logging { // Finally, only best Splits for nodes are collected to driver to construct decision tree. val nodeToFeatures = getNodeToFeatures(treeToNodeToIndexInfo) val nodeToFeaturesBc = input.sparkContext.broadcast(nodeToFeatures) - val nodeToBestSplits = + + val partitionAggregates : RDD[(Int, DTStatsAggregator)] = if (nodeIdCache.nonEmpty) { + input.zip(nodeIdCache.get.nodeIdsForInstances).mapPartitions { points => + // Construct a nodeStatsAggregators array to hold node aggregate stats, + // each node will have a nodeStatsAggregator + val nodeStatsAggregators = Array.tabulate(numNodes) { nodeIndex => + val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => + Some(nodeToFeatures(nodeIndex)) + } + new DTStatsAggregator(metadata, featuresForNode) + } + + // iterator all instances in current partition and update aggregate stats + points.foreach(binSeqOpWithNodeIdCache(nodeStatsAggregators, _)) + + // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs, + // which can be combined with other partition using `reduceByKey` + nodeStatsAggregators.view.zipWithIndex.map(_.swap).iterator + } + } else { input.mapPartitions { points => // Construct a nodeStatsAggregators array to hold node aggregate stats, // each node will have a nodeStatsAggregator @@ -570,7 +631,10 @@ object DecisionTree extends Serializable with Logging { // transform nodeStatsAggregators array to (nodeIndex, nodeAggregateStats) pairs, // which can be combined with other partition using `reduceByKey` nodeStatsAggregators.view.zipWithIndex.map(_.swap).iterator - }.reduceByKey((a, b) => a.merge(b)) + } + } + + val nodeToBestSplits = partitionAggregates.reduceByKey((a, b) => a.merge(b)) .map { case (nodeIndex, aggStats) => val featuresForNode = nodeToFeaturesBc.value.flatMap { nodeToFeatures => Some(nodeToFeatures(nodeIndex)) @@ -584,6 +648,13 @@ object DecisionTree extends Serializable with Logging { timer.stop("chooseSplits") + val nodeIdUpdaters = if (nodeIdCache.nonEmpty) { + Array.fill[mutable.Map[Int, NodeIndexUpdater]]( + metadata.numTrees)(mutable.Map[Int, NodeIndexUpdater]()) + } else { + null + } + // Iterate over all nodes in this group. nodesForGroup.foreach { case (treeIndex, nodesForTree) => nodesForTree.foreach { node => @@ -613,6 +684,13 @@ object DecisionTree extends Serializable with Logging { node.rightNode = Some(Node(Node.rightChildIndex(nodeIndex), stats.rightPredict, stats.rightImpurity, rightChildIsLeaf)) + if (nodeIdCache.nonEmpty) { + val nodeIndexUpdater = NodeIndexUpdater( + split = split, + nodeIndex = nodeIndex) + nodeIdUpdaters(treeIndex).put(nodeIndex, nodeIndexUpdater) + } + // enqueue left child and right child if they are not leaves if (!leftChildIsLeaf) { nodeQueue.enqueue((treeIndex, node.leftNode.get)) @@ -629,6 +707,10 @@ object DecisionTree extends Serializable with Logging { } } + if (nodeIdCache.nonEmpty) { + // Update the cache if needed. + nodeIdCache.get.updateNodeIndices(input, nodeIdUpdaters, bins) + } } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 1dcaf914381ce..9683916d9b3f1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -28,7 +28,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ import org.apache.spark.mllib.tree.configuration.EnsembleCombiningStrategy.Average import org.apache.spark.mllib.tree.configuration.Strategy -import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker} +import org.apache.spark.mllib.tree.impl.{BaggedPoint, TreePoint, DecisionTreeMetadata, TimeTracker, NodeIdCache } import org.apache.spark.mllib.tree.impurity.Impurities import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD @@ -160,6 +160,19 @@ private class RandomForest ( * in lower levels). */ + // Create an RDD of node Id cache. + // At first, all the rows belong to the root nodes (node Id == 1). + val nodeIdCache = if (strategy.useNodeIdCache) { + Some(NodeIdCache.init( + data = baggedInput, + numTrees = numTrees, + checkpointDir = strategy.checkpointDir, + checkpointInterval = strategy.checkpointInterval, + initVal = 1)) + } else { + None + } + // FIFO queue of nodes to train: (treeIndex, node) val nodeQueue = new mutable.Queue[(Int, Node)]() @@ -182,7 +195,7 @@ private class RandomForest ( // Choose node splits, and enqueue new nodes as needed. timer.start("findBestSplits") DecisionTree.findBestSplits(baggedInput, metadata, topNodes, nodesForGroup, - treeToNodeToIndexInfo, splits, bins, nodeQueue, timer) + treeToNodeToIndexInfo, splits, bins, nodeQueue, timer, nodeIdCache = nodeIdCache) timer.stop("findBestSplits") } @@ -193,6 +206,11 @@ private class RandomForest ( logInfo("Internal timing for DecisionTree:") logInfo(s"$timer") + // Delete any remaining checkpoints used for node Id cache. + if (nodeIdCache.nonEmpty) { + nodeIdCache.get.deleteAllCheckpoints() + } + val trees = topNodes.map(topNode => new DecisionTreeModel(topNode, strategy.algo)) val treeWeights = Array.fill[Double](numTrees)(1.0) new WeightedEnsembleModel(trees, treeWeights, strategy.algo, Average) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 2ed63cf002efb..d09295c507d67 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -60,6 +60,13 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. Default value is * 256 MB. * @param subsamplingRate Fraction of the training data used for learning decision tree. + * @param useNodeIdCache If this is true, instead of passing trees to executors, the algorithm will + * maintain a separate RDD of node Id cache for each row. + * @param checkpointDir If the node Id cache is used, it will help to checkpoint + * the node Id cache periodically. This is the checkpoint directory + * to be used for the node Id cache. + * @param checkpointInterval How often to checkpoint when the node Id cache gets updated. + * E.g. 10 means that the cache will get checkpointed every 10 updates. */ @Experimental class Strategy ( @@ -73,7 +80,10 @@ class Strategy ( @BeanProperty var minInstancesPerNode: Int = 1, @BeanProperty var minInfoGain: Double = 0.0, @BeanProperty var maxMemoryInMB: Int = 256, - @BeanProperty var subsamplingRate: Double = 1) extends Serializable { + @BeanProperty var subsamplingRate: Double = 1, + @BeanProperty var useNodeIdCache: Boolean = false, + @BeanProperty var checkpointDir: Option[String] = None, + @BeanProperty var checkpointInterval: Int = 10) extends Serializable { if (algo == Classification) { require(numClassesForClassification >= 2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala new file mode 100644 index 0000000000000..83011b48b7d9b --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/NodeIdCache.scala @@ -0,0 +1,204 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.tree.impl + +import scala.collection.mutable + +import org.apache.hadoop.fs.{Path, FileSystem} + +import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.mllib.tree.model.{Bin, Node, Split} + +/** + * :: DeveloperApi :: + * This is used by the node id cache to find the child id that a data point would belong to. + * @param split Split information. + * @param nodeIndex The current node index of a data point that this will update. + */ +@DeveloperApi +private[tree] case class NodeIndexUpdater( + split: Split, + nodeIndex: Int) { + /** + * Determine a child node index based on the feature value and the split. + * @param binnedFeatures Binned feature values. + * @param bins Bin information to convert the bin indices to approximate feature values. + * @return Child node index to update to. + */ + def updateNodeIndex(binnedFeatures: Array[Int], bins: Array[Array[Bin]]): Int = { + if (split.featureType == Continuous) { + val featureIndex = split.feature + val binIndex = binnedFeatures(featureIndex) + val featureValueUpperBound = bins(featureIndex)(binIndex).highSplit.threshold + if (featureValueUpperBound <= split.threshold) { + Node.leftChildIndex(nodeIndex) + } else { + Node.rightChildIndex(nodeIndex) + } + } else { + if (split.categories.contains(binnedFeatures(split.feature).toDouble)) { + Node.leftChildIndex(nodeIndex) + } else { + Node.rightChildIndex(nodeIndex) + } + } + } +} + +/** + * :: DeveloperApi :: + * A given TreePoint would belong to a particular node per tree. + * Each row in the nodeIdsForInstances RDD is an array over trees of the node index + * in each tree. Initially, values should all be 1 for root node. + * The nodeIdsForInstances RDD needs to be updated at each iteration. + * @param nodeIdsForInstances The initial values in the cache + * (should be an Array of all 1's (meaning the root nodes)). + * @param checkpointDir The checkpoint directory where + * the checkpointed files will be stored. + * @param checkpointInterval The checkpointing interval + * (how often should the cache be checkpointed.). + */ +@DeveloperApi +private[tree] class NodeIdCache( + var nodeIdsForInstances: RDD[Array[Int]], + val checkpointDir: Option[String], + val checkpointInterval: Int) { + + // Keep a reference to a previous node Ids for instances. + // Because we will keep on re-persisting updated node Ids, + // we want to unpersist the previous RDD. + private var prevNodeIdsForInstances: RDD[Array[Int]] = null + + // To keep track of the past checkpointed RDDs. + private val checkpointQueue = mutable.Queue[RDD[Array[Int]]]() + private var rddUpdateCount = 0 + + // If a checkpoint directory is given, and there's no prior checkpoint directory, + // then set the checkpoint directory with the given one. + if (checkpointDir.nonEmpty && nodeIdsForInstances.sparkContext.getCheckpointDir.isEmpty) { + nodeIdsForInstances.sparkContext.setCheckpointDir(checkpointDir.get) + } + + /** + * Update the node index values in the cache. + * This updates the RDD and its lineage. + * TODO: Passing bin information to executors seems unnecessary and costly. + * @param data The RDD of training rows. + * @param nodeIdUpdaters A map of node index updaters. + * The key is the indices of nodes that we want to update. + * @param bins Bin information needed to find child node indices. + */ + def updateNodeIndices( + data: RDD[BaggedPoint[TreePoint]], + nodeIdUpdaters: Array[mutable.Map[Int, NodeIndexUpdater]], + bins: Array[Array[Bin]]): Unit = { + if (prevNodeIdsForInstances != null) { + // Unpersist the previous one if one exists. + prevNodeIdsForInstances.unpersist() + } + + prevNodeIdsForInstances = nodeIdsForInstances + nodeIdsForInstances = data.zip(nodeIdsForInstances).map { + dataPoint => { + var treeId = 0 + while (treeId < nodeIdUpdaters.length) { + val nodeIdUpdater = nodeIdUpdaters(treeId).getOrElse(dataPoint._2(treeId), null) + if (nodeIdUpdater != null) { + val newNodeIndex = nodeIdUpdater.updateNodeIndex( + binnedFeatures = dataPoint._1.datum.binnedFeatures, + bins = bins) + dataPoint._2(treeId) = newNodeIndex + } + + treeId += 1 + } + + dataPoint._2 + } + } + + // Keep on persisting new ones. + nodeIdsForInstances.persist(StorageLevel.MEMORY_AND_DISK) + rddUpdateCount += 1 + + // Handle checkpointing if the directory is not None. + if (nodeIdsForInstances.sparkContext.getCheckpointDir.nonEmpty && + (rddUpdateCount % checkpointInterval) == 0) { + // Let's see if we can delete previous checkpoints. + var canDelete = true + while (checkpointQueue.size > 1 && canDelete) { + // We can delete the oldest checkpoint iff + // the next checkpoint actually exists in the file system. + if (checkpointQueue.get(1).get.getCheckpointFile != None) { + val old = checkpointQueue.dequeue() + + // Since the old checkpoint is not deleted by Spark, + // we'll manually delete it here. + val fs = FileSystem.get(old.sparkContext.hadoopConfiguration) + fs.delete(new Path(old.getCheckpointFile.get), true) + } else { + canDelete = false + } + } + + nodeIdsForInstances.checkpoint() + checkpointQueue.enqueue(nodeIdsForInstances) + } + } + + /** + * Call this after training is finished to delete any remaining checkpoints. + */ + def deleteAllCheckpoints(): Unit = { + while (checkpointQueue.size > 0) { + val old = checkpointQueue.dequeue() + if (old.getCheckpointFile != None) { + val fs = FileSystem.get(old.sparkContext.hadoopConfiguration) + fs.delete(new Path(old.getCheckpointFile.get), true) + } + } + } +} + +@DeveloperApi +private[tree] object NodeIdCache { + /** + * Initialize the node Id cache with initial node Id values. + * @param data The RDD of training rows. + * @param numTrees The number of trees that we want to create cache for. + * @param checkpointDir The checkpoint directory where the checkpointed files will be stored. + * @param checkpointInterval The checkpointing interval + * (how often should the cache be checkpointed.). + * @param initVal The initial values in the cache. + * @return A node Id cache containing an RDD of initial root node Indices. + */ + def init( + data: RDD[BaggedPoint[TreePoint]], + numTrees: Int, + checkpointDir: Option[String], + checkpointInterval: Int, + initVal: Int = 1): NodeIdCache = { + new NodeIdCache( + data.map(_ => Array.fill[Int](numTrees)(initVal)), + checkpointDir, + checkpointInterval) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 10c046e07f562..73c4393c3581a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -34,18 +34,11 @@ import org.apache.spark.mllib.util.LocalSparkContext * Test suite for [[RandomForest]]. */ class RandomForestSuite extends FunSuite with LocalSparkContext { - - test("Binary classification with continuous features:" + - " comparing DecisionTree vs. RandomForest(numTrees = 1)") { - + def binaryClassificationTestWithContinuousFeatures(strategy: Strategy) { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) - val categoricalFeaturesInfo = Map.empty[Int, Int] val numTrees = 1 - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) - val rf = RandomForest.trainClassifier(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) assert(rf.weakHypotheses.size === 1) @@ -60,18 +53,27 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { assert(rfTree.toString == dt.toString) } - test("Regression with continuous features:" + + test("Binary classification with continuous features:" + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + binaryClassificationTestWithContinuousFeatures(strategy) + } + test("Binary classification with continuous features and node Id cache :" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) + binaryClassificationTestWithContinuousFeatures(strategy) + } + + def regressionTestWithContinuousFeatures(strategy: Strategy) { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 50, 1000) val rdd = sc.parallelize(arr) - val categoricalFeaturesInfo = Map.empty[Int, Int] val numTrees = 1 - val strategy = new Strategy(algo = Regression, impurity = Variance, - maxDepth = 2, maxBins = 10, numClassesForClassification = 2, - categoricalFeaturesInfo = categoricalFeaturesInfo) - val rf = RandomForest.trainRegressor(rdd, strategy, numTrees = numTrees, featureSubsetStrategy = "auto", seed = 123) assert(rf.weakHypotheses.size === 1) @@ -86,14 +88,28 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { assert(rfTree.toString == dt.toString) } - test("Binary classification with continuous features: subsampling features") { + test("Regression with continuous features:" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new Strategy(algo = Regression, impurity = Variance, + maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + categoricalFeaturesInfo = categoricalFeaturesInfo) + regressionTestWithContinuousFeatures(strategy) + } + + test("Regression with continuous features and node Id cache :" + + " comparing DecisionTree vs. RandomForest(numTrees = 1)") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new Strategy(algo = Regression, impurity = Variance, + maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) + regressionTestWithContinuousFeatures(strategy) + } + + def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: Strategy) { val numFeatures = 50 val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) val rdd = sc.parallelize(arr) - val categoricalFeaturesInfo = Map.empty[Int, Int] - - val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) // Select feature subset for top nodes. Return true if OK. def checkFeatureSubsetStrategy( @@ -149,6 +165,20 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { checkFeatureSubsetStrategy(numTrees = 2, "onethird", (numFeatures / 3.0).ceil.toInt) } + test("Binary classification with continuous features: subsampling features") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) + } + + test("Binary classification with continuous features and node Id cache: subsampling features") { + val categoricalFeaturesInfo = Map.empty[Int, Int] + val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, + numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) + binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) + } + test("alternating categorical and continuous features with multiclass labels to test indexing") { val arr = new Array[LabeledPoint](4) arr(0) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0, 0.0, 3.0, 1.0)) @@ -164,7 +194,6 @@ class RandomForestSuite extends FunSuite with LocalSparkContext { featureSubsetStrategy = "sqrt", seed = 12345) EnsembleTestHelper.validateClassifier(model, arr, 1.0) } - } From 23f966f47523f85ba440b4080eee665271f53b5e Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 1 Nov 2014 19:29:14 -0700 Subject: [PATCH 1183/1492] [SPARK-3930] [SPARK-3933] Support fixed-precision decimal in SQL, and some optimizations - Adds optional precision and scale to Spark SQL's decimal type, which behave similarly to those in Hive 13 (https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf) - Replaces our internal representation of decimals with a Decimal class that can store small values in a mutable Long, saving memory in this situation and letting some operations happen directly on Longs This is still marked WIP because there are a few TODOs, but I'll remove that tag when done. Author: Matei Zaharia Closes #2983 from mateiz/decimal-1 and squashes the following commits: 35e6b02 [Matei Zaharia] Fix issues after merge 227f24a [Matei Zaharia] Review comments 31f915e [Matei Zaharia] Implement Davies's suggestions in Python eb84820 [Matei Zaharia] Support reading/writing decimals as fixed-length binary in Parquet 4dc6bae [Matei Zaharia] Fix decimal support in PySpark d1d9d68 [Matei Zaharia] Fix compile error and test issues after rebase b28933d [Matei Zaharia] Support decimal precision/scale in Hive metastore 2118c0d [Matei Zaharia] Some test and bug fixes 81db9cb [Matei Zaharia] Added mutable Decimal that will be more efficient for small precisions 7af0c3b [Matei Zaharia] Add optional precision and scale to DecimalType, but use Unlimited for now ec0a947 [Matei Zaharia] Make the result of AVG on Decimals be Decimal, not Double --- python/pyspark/sql.py | 35 +- .../spark/sql/catalyst/ScalaReflection.scala | 20 +- .../apache/spark/sql/catalyst/SqlParser.scala | 14 +- .../catalyst/analysis/HiveTypeCoercion.scala | 146 +++++++- .../spark/sql/catalyst/dsl/package.scala | 11 +- .../spark/sql/catalyst/expressions/Cast.scala | 78 ++-- .../sql/catalyst/expressions/aggregates.scala | 55 ++- .../sql/catalyst/expressions/arithmetic.scala | 10 +- .../expressions/codegen/CodeGenerator.scala | 31 +- .../expressions/decimalFunctions.scala | 59 +++ .../sql/catalyst/expressions/literals.scala | 6 +- .../sql/catalyst/optimizer/Optimizer.scala | 38 +- .../spark/sql/catalyst/types/dataTypes.scala | 84 ++++- .../sql/catalyst/types/decimal/Decimal.scala | 335 ++++++++++++++++++ .../sql/catalyst/ScalaReflectionSuite.scala | 14 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 6 +- .../analysis/DecimalPrecisionSuite.scala | 88 +++++ .../analysis/HiveTypeCoercionSuite.scala | 17 +- .../ExpressionEvaluationSuite.scala | 90 ++++- .../catalyst/types/decimal/DecimalSuite.scala | 158 +++++++++ .../apache/spark/sql/api/java/DataType.java | 5 - .../spark/sql/api/java/DecimalType.java | 58 ++- .../org/apache/spark/sql/SchemaRDD.scala | 3 +- .../spark/sql/api/java/JavaSQLContext.scala | 2 +- .../org/apache/spark/sql/api/java/Row.scala | 4 + .../sql/execution/GeneratedAggregate.scala | 41 ++- .../spark/sql/execution/SparkPlan.scala | 4 +- .../sql/execution/SparkSqlSerializer.scala | 2 + .../spark/sql/execution/basicOperators.scala | 7 +- .../execution/joins/BroadcastHashJoin.scala | 3 +- .../spark/sql/execution/pythonUdfs.scala | 6 +- .../org/apache/spark/sql/json/JsonRDD.scala | 20 +- .../scala/org/apache/spark/sql/package.scala | 14 + .../spark/sql/parquet/ParquetConverter.scala | 43 +++ .../sql/parquet/ParquetTableSupport.scala | 28 ++ .../spark/sql/parquet/ParquetTypes.scala | 79 +++-- .../sql/types/util/DataTypeConversions.scala | 13 +- .../sql/api/java/JavaApplySchemaSuite.java | 2 +- .../java/JavaSideDataTypeConversionSuite.java | 9 +- .../org/apache/spark/sql/DataTypeSuite.scala | 2 +- .../sql/ScalaReflectionRelationSuite.scala | 5 +- .../spark/sql/api/java/JavaSQLSuite.scala | 2 + .../ScalaSideDataTypeConversionSuite.scala | 4 +- .../org/apache/spark/sql/json/JsonSuite.scala | 46 +-- .../spark/sql/parquet/ParquetQuerySuite.scala | 35 +- .../server/SparkSQLOperationManager.scala | 4 +- .../spark/sql/hive/thriftserver/Shim12.scala | 4 +- .../spark/sql/hive/thriftserver/Shim13.scala | 2 +- .../apache/spark/sql/hive/HiveContext.scala | 9 +- .../spark/sql/hive/HiveInspectors.scala | 24 +- .../spark/sql/hive/HiveMetastoreCatalog.scala | 14 +- .../org/apache/spark/sql/hive/HiveQl.scala | 15 +- .../hive/execution/InsertIntoHiveTable.scala | 3 +- .../org/apache/spark/sql/hive/Shim12.scala | 22 +- .../org/apache/spark/sql/hive/Shim13.scala | 39 +- 55 files changed, 1636 insertions(+), 232 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 93bfc25bca855..98e41f8575679 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -35,6 +35,7 @@ import keyword import warnings import json +import re from array import array from operator import itemgetter from itertools import imap @@ -148,13 +149,30 @@ class TimestampType(PrimitiveType): """ -class DecimalType(PrimitiveType): +class DecimalType(DataType): """Spark SQL DecimalType The data type representing decimal.Decimal values. """ + def __init__(self, precision=None, scale=None): + self.precision = precision + self.scale = scale + self.hasPrecisionInfo = precision is not None + + def jsonValue(self): + if self.hasPrecisionInfo: + return "decimal(%d,%d)" % (self.precision, self.scale) + else: + return "decimal" + + def __repr__(self): + if self.hasPrecisionInfo: + return "DecimalType(%d,%d)" % (self.precision, self.scale) + else: + return "DecimalType()" + class DoubleType(PrimitiveType): @@ -446,9 +464,20 @@ def _parse_datatype_json_string(json_string): return _parse_datatype_json_value(json.loads(json_string)) +_FIXED_DECIMAL = re.compile("decimal\\((\\d+),(\\d+)\\)") + + def _parse_datatype_json_value(json_value): - if type(json_value) is unicode and json_value in _all_primitive_types.keys(): - return _all_primitive_types[json_value]() + if type(json_value) is unicode: + if json_value in _all_primitive_types.keys(): + return _all_primitive_types[json_value]() + elif json_value == u'decimal': + return DecimalType() + elif _FIXED_DECIMAL.match(json_value): + m = _FIXED_DECIMAL.match(json_value) + return DecimalType(int(m.group(1)), int(m.group(2))) + else: + raise ValueError("Could not parse datatype: %s" % json_value) else: return _all_complex_types[json_value["type"]].fromJson(json_value) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 75923d9e8d729..8fbdf664b71e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal /** * Provides experimental support for generating catalyst schemas for scala objects. @@ -40,9 +41,20 @@ object ScalaReflection { case s: Seq[_] => s.map(convertToCatalyst) case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) + case d: BigDecimal => Decimal(d) case other => other } + /** Converts Catalyst types used internally in rows to standard Scala types */ + def convertToScala(a: Any): Any = a match { + case s: Seq[_] => s.map(convertToScala) + case m: Map[_, _] => m.map { case (k, v) => convertToScala(k) -> convertToScala(v) } + case d: Decimal => d.toBigDecimal + case other => other + } + + def convertRowToScala(r: Row): Row = new GenericRow(r.toArray.map(convertToScala)) + /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { case Schema(s: StructType, _) => @@ -83,7 +95,8 @@ object ScalaReflection { case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[Date] => Schema(DateType, nullable = true) - case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) + case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) case t if t <:< typeOf[java.lang.Double] => Schema(DoubleType, nullable = true) @@ -111,8 +124,9 @@ object ScalaReflection { case obj: LongType.JvmType => LongType case obj: FloatType.JvmType => FloatType case obj: DoubleType.JvmType => DoubleType - case obj: DecimalType.JvmType => DecimalType case obj: DateType.JvmType => DateType + case obj: BigDecimal => DecimalType.Unlimited + case obj: Decimal => DecimalType.Unlimited case obj: TimestampType.JvmType => TimestampType case null => NullType // For other cases, there is no obvious mapping from the type of the given object to a diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index b1e7570f577c3..00fc4d75c9ea9 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -52,11 +52,13 @@ class SqlParser extends AbstractSparkSQLParser { protected val CASE = Keyword("CASE") protected val CAST = Keyword("CAST") protected val COUNT = Keyword("COUNT") + protected val DECIMAL = Keyword("DECIMAL") protected val DESC = Keyword("DESC") protected val DISTINCT = Keyword("DISTINCT") protected val ELSE = Keyword("ELSE") protected val END = Keyword("END") protected val EXCEPT = Keyword("EXCEPT") + protected val DOUBLE = Keyword("DOUBLE") protected val FALSE = Keyword("FALSE") protected val FIRST = Keyword("FIRST") protected val FROM = Keyword("FROM") @@ -385,5 +387,15 @@ class SqlParser extends AbstractSparkSQLParser { } protected lazy val dataType: Parser[DataType] = - STRING ^^^ StringType | TIMESTAMP ^^^ TimestampType + ( STRING ^^^ StringType + | TIMESTAMP ^^^ TimestampType + | DOUBLE ^^^ DoubleType + | fixedDecimalType + | DECIMAL ^^^ DecimalType.Unlimited + ) + + protected lazy val fixedDecimalType: Parser[DataType] = + (DECIMAL ~ "(" ~> numericLit) ~ ("," ~> numericLit <~ ")") ^^ { + case precision ~ scale => DecimalType(precision.toInt, scale.toInt) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 2b69c02b28285..e38114ab3cf25 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -25,19 +25,31 @@ import org.apache.spark.sql.catalyst.types._ object HiveTypeCoercion { // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types. // The conversion for integral and floating point types have a linear widening hierarchy: - val numericPrecedence = - Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType) - val allPromotions: Seq[Seq[DataType]] = numericPrecedence :: Nil + private val numericPrecedence = + Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType.Unlimited) + /** + * Find the tightest common type of two types that might be used in a binary expression. + * This handles all numeric types except fixed-precision decimals interacting with each other or + * with primitive types, because in that case the precision and scale of the result depends on + * the operation. Those rules are implemented in [[HiveTypeCoercion.DecimalPrecision]]. + */ def findTightestCommonType(t1: DataType, t2: DataType): Option[DataType] = { val valueTypes = Seq(t1, t2).filter(t => t != NullType) if (valueTypes.distinct.size > 1) { - // Try and find a promotion rule that contains both types in question. - val applicableConversion = - HiveTypeCoercion.allPromotions.find(p => p.contains(t1) && p.contains(t2)) - - // If found return the widest common type, otherwise None - applicableConversion.map(_.filter(t => t == t1 || t == t2).last) + // Promote numeric types to the highest of the two and all numeric types to unlimited decimal + if (numericPrecedence.contains(t1) && numericPrecedence.contains(t2)) { + Some(numericPrecedence.filter(t => t == t1 || t == t2).last) + } else if (t1.isInstanceOf[DecimalType] && t2.isInstanceOf[DecimalType]) { + // Fixed-precision decimals can up-cast into unlimited + if (t1 == DecimalType.Unlimited || t2 == DecimalType.Unlimited) { + Some(DecimalType.Unlimited) + } else { + None + } + } else { + None + } } else { Some(if (valueTypes.size == 0) NullType else valueTypes.head) } @@ -59,6 +71,7 @@ trait HiveTypeCoercion { ConvertNaNs :: WidenTypes :: PromoteStrings :: + DecimalPrecision :: BooleanComparisons :: BooleanCasts :: StringToIntegralCasts :: @@ -151,6 +164,7 @@ trait HiveTypeCoercion { import HiveTypeCoercion._ def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // TODO: unions with fixed-precision decimals case u @ Union(left, right) if u.childrenResolved && !u.resolved => val castedInput = left.output.zip(right.output).map { // When a string is found on one side, make the other side a string too. @@ -265,6 +279,110 @@ trait HiveTypeCoercion { } } + // scalastyle:off + /** + * Calculates and propagates precision for fixed-precision decimals. Hive has a number of + * rules for this based on the SQL standard and MS SQL: + * https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf + * + * In particular, if we have expressions e1 and e2 with precision/scale p1/s2 and p2/s2 + * respectively, then the following operations have the following precision / scale: + * + * Operation Result Precision Result Scale + * ------------------------------------------------------------------------ + * e1 + e2 max(s1, s2) + max(p1-s1, p2-s2) + 1 max(s1, s2) + * e1 - e2 max(s1, s2) + max(p1-s1, p2-s2) + 1 max(s1, s2) + * e1 * e2 p1 + p2 + 1 s1 + s2 + * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) + * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) + * sum(e1) p1 + 10 s1 + * avg(e1) p1 + 4 s1 + 4 + * + * Catalyst also has unlimited-precision decimals. For those, all ops return unlimited precision. + * + * To implement the rules for fixed-precision types, we introduce casts to turn them to unlimited + * precision, do the math on unlimited-precision numbers, then introduce casts back to the + * required fixed precision. This allows us to do all rounding and overflow handling in the + * cast-to-fixed-precision operator. + * + * In addition, when mixing non-decimal types with decimals, we use the following rules: + * - BYTE gets turned into DECIMAL(3, 0) + * - SHORT gets turned into DECIMAL(5, 0) + * - INT gets turned into DECIMAL(10, 0) + * - LONG gets turned into DECIMAL(20, 0) + * - FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive, + * but note that unlimited decimals are considered bigger than doubles in WidenTypes) + */ + // scalastyle:on + object DecimalPrecision extends Rule[LogicalPlan] { + import scala.math.{max, min} + + // Conversion rules for integer types into fixed-precision decimals + val intTypeToFixed: Map[DataType, DecimalType] = Map( + ByteType -> DecimalType(3, 0), + ShortType -> DecimalType(5, 0), + IntegerType -> DecimalType(10, 0), + LongType -> DecimalType(20, 0) + ) + + def isFloat(t: DataType): Boolean = t == FloatType || t == DoubleType + + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + // Skip nodes whose children have not been resolved yet + case e if !e.childrenResolved => e + + case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Add(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) + ) + + case Subtract(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Subtract(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) + ) + + case Multiply(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Multiply(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(p1 + p2 + 1, s1 + s2) + ) + + case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Divide(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(p1 - s1 + s2 + max(6, s1 + p2 + 1), max(6, s1 + p2 + 1)) + ) + + case Remainder(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => + Cast( + Remainder(Cast(e1, DecimalType.Unlimited), Cast(e2, DecimalType.Unlimited)), + DecimalType(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2)) + ) + + // Promote integers inside a binary expression with fixed-precision decimals to decimals, + // and fixed-precision decimals in an expression with floats / doubles to doubles + case b: BinaryExpression if b.left.dataType != b.right.dataType => + (b.left.dataType, b.right.dataType) match { + case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) => + b.makeCopy(Array(Cast(b.left, intTypeToFixed(t)), b.right)) + case (DecimalType.Fixed(p, s), t) if intTypeToFixed.contains(t) => + b.makeCopy(Array(b.left, Cast(b.right, intTypeToFixed(t)))) + case (t, DecimalType.Fixed(p, s)) if isFloat(t) => + b.makeCopy(Array(b.left, Cast(b.right, DoubleType))) + case (DecimalType.Fixed(p, s), t) if isFloat(t) => + b.makeCopy(Array(Cast(b.left, DoubleType), b.right)) + case _ => + b + } + + // TODO: MaxOf, MinOf, etc might want other rules + + // SUM and AVERAGE are handled by the implementations of those expressions + } + } + /** * Changes Boolean values to Bytes so that expressions like true < false can be Evaluated. */ @@ -330,7 +448,7 @@ trait HiveTypeCoercion { case e if !e.childrenResolved => e case Cast(e @ StringType(), t: IntegralType) => - Cast(Cast(e, DecimalType), t) + Cast(Cast(e, DecimalType.Unlimited), t) } } @@ -383,10 +501,12 @@ trait HiveTypeCoercion { // Decimal and Double remain the same case d: Divide if d.resolved && d.dataType == DoubleType => d - case d: Divide if d.resolved && d.dataType == DecimalType => d + case d: Divide if d.resolved && d.dataType.isInstanceOf[DecimalType] => d - case Divide(l, r) if l.dataType == DecimalType => Divide(l, Cast(r, DecimalType)) - case Divide(l, r) if r.dataType == DecimalType => Divide(Cast(l, DecimalType), r) + case Divide(l, r) if l.dataType.isInstanceOf[DecimalType] => + Divide(l, Cast(r, DecimalType.Unlimited)) + case Divide(l, r) if r.dataType.isInstanceOf[DecimalType] => + Divide(Cast(l, DecimalType.Unlimited), r) case Divide(l, r) => Divide(Cast(l, DoubleType), Cast(r, DoubleType)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 23cfd483ec410..7e6d770314f5a 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} +import org.apache.spark.sql.catalyst.types.decimal.Decimal + import scala.language.implicitConversions import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute @@ -124,7 +126,8 @@ package object dsl { implicit def doubleToLiteral(d: Double) = Literal(d) implicit def stringToLiteral(s: String) = Literal(s) implicit def dateToLiteral(d: Date) = Literal(d) - implicit def decimalToLiteral(d: BigDecimal) = Literal(d) + implicit def bigDecimalToLiteral(d: BigDecimal) = Literal(d) + implicit def decimalToLiteral(d: Decimal) = Literal(d) implicit def timestampToLiteral(t: Timestamp) = Literal(t) implicit def binaryToLiteral(a: Array[Byte]) = Literal(a) @@ -183,7 +186,11 @@ package object dsl { def date = AttributeReference(s, DateType, nullable = true)() /** Creates a new AttributeReference of type decimal */ - def decimal = AttributeReference(s, DecimalType, nullable = true)() + def decimal = AttributeReference(s, DecimalType.Unlimited, nullable = true)() + + /** Creates a new AttributeReference of type decimal */ + def decimal(precision: Int, scale: Int) = + AttributeReference(s, DecimalType(precision, scale), nullable = true)() /** Creates a new AttributeReference of type timestamp */ def timestamp = AttributeReference(s, TimestampType, nullable = true)() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 8e5baf0eb82d6..22009666196a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -23,6 +23,7 @@ import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression with Logging { @@ -36,6 +37,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case (BooleanType, DateType) => true case (DateType, _: NumericType) => true case (DateType, BooleanType) => true + case (_, DecimalType.Fixed(_, _)) => true // TODO: not all upcasts here can really give null case _ => child.nullable } @@ -76,8 +78,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Short](_, _ != 0) case ByteType => buildCast[Byte](_, _ != 0) - case DecimalType => - buildCast[BigDecimal](_, _ != 0) + case DecimalType() => + buildCast[Decimal](_, _ != 0) case DoubleType => buildCast[Double](_, _ != 0) case FloatType => @@ -109,19 +111,19 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case DateType => buildCast[Date](_, d => new Timestamp(d.getTime)) // TimestampWritable.decimalToTimestamp - case DecimalType => - buildCast[BigDecimal](_, d => decimalToTimestamp(d)) + case DecimalType() => + buildCast[Decimal](_, d => decimalToTimestamp(d)) // TimestampWritable.doubleToTimestamp case DoubleType => - buildCast[Double](_, d => decimalToTimestamp(d)) + buildCast[Double](_, d => decimalToTimestamp(Decimal(d))) // TimestampWritable.floatToTimestamp case FloatType => - buildCast[Float](_, f => decimalToTimestamp(f)) + buildCast[Float](_, f => decimalToTimestamp(Decimal(f))) } - private[this] def decimalToTimestamp(d: BigDecimal) = { + private[this] def decimalToTimestamp(d: Decimal) = { val seconds = Math.floor(d.toDouble).toLong - val bd = (d - seconds) * 1000000000 + val bd = (d.toBigDecimal - seconds) * 1000000000 val nanos = bd.intValue() val millis = seconds * 1000 @@ -196,8 +198,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t)) - case DecimalType => - buildCast[BigDecimal](_, _.toLong) + case DecimalType() => + buildCast[Decimal](_, _.toLong) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) } @@ -214,8 +216,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toInt) - case DecimalType => - buildCast[BigDecimal](_, _.toInt) + case DecimalType() => + buildCast[Decimal](_, _.toInt) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) } @@ -232,8 +234,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toShort) - case DecimalType => - buildCast[BigDecimal](_, _.toShort) + case DecimalType() => + buildCast[Decimal](_, _.toShort) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort } @@ -250,27 +252,45 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToLong(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToLong(t).toByte) - case DecimalType => - buildCast[BigDecimal](_, _.toByte) + case DecimalType() => + buildCast[Decimal](_, _.toByte) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } - // DecimalConverter - private[this] def castToDecimal: Any => Any = child.dataType match { + /** + * Change the precision / scale in a given decimal to those set in `decimalType` (if any), + * returning null if it overflows or modifying `value` in-place and returning it if successful. + * + * NOTE: this modifies `value` in-place, so don't call it on external data. + */ + private[this] def changePrecision(value: Decimal, decimalType: DecimalType): Decimal = { + decimalType match { + case DecimalType.Unlimited => + value + case DecimalType.Fixed(precision, scale) => + if (value.changePrecision(precision, scale)) value else null + } + } + + private[this] def castToDecimal(target: DecimalType): Any => Any = child.dataType match { case StringType => - buildCast[String](_, s => try BigDecimal(s.toDouble) catch { + buildCast[String](_, s => try changePrecision(Decimal(s.toDouble), target) catch { case _: NumberFormatException => null }) case BooleanType => - buildCast[Boolean](_, b => if (b) BigDecimal(1) else BigDecimal(0)) + buildCast[Boolean](_, b => changePrecision(if (b) Decimal(1) else Decimal(0), target)) case DateType => - buildCast[Date](_, d => dateToDouble(d)) + buildCast[Date](_, d => changePrecision(null, target)) // date can't cast to decimal in Hive case TimestampType => // Note that we lose precision here. - buildCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) - case x: NumericType => - b => BigDecimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)) + buildCast[Timestamp](_, t => changePrecision(Decimal(timestampToDouble(t)), target)) + case DecimalType() => + b => changePrecision(b.asInstanceOf[Decimal].clone(), target) + case LongType => + b => changePrecision(Decimal(b.asInstanceOf[Long]), target) + case x: NumericType => // All other numeric types can be represented precisely as Doubles + b => changePrecision(Decimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)), target) } // DoubleConverter @@ -285,8 +305,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t)) - case DecimalType => - buildCast[BigDecimal](_, _.toDouble) + case DecimalType() => + buildCast[Decimal](_, _.toDouble) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) } @@ -303,8 +323,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w buildCast[Date](_, d => dateToDouble(d)) case TimestampType => buildCast[Timestamp](_, t => timestampToDouble(t).toFloat) - case DecimalType => - buildCast[BigDecimal](_, _.toFloat) + case DecimalType() => + buildCast[Decimal](_, _.toFloat) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) } @@ -313,8 +333,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case dt if dt == child.dataType => identity[Any] case StringType => castToString case BinaryType => castToBinary - case DecimalType => castToDecimal case DateType => castToDate + case decimal: DecimalType => castToDecimal(decimal) case TimestampType => castToTimestamp case BooleanType => castToBoolean case ByteType => castToByte diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 1b4d892625dbb..2b364fc1df1d8 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -286,18 +286,38 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def nullable = false - override def dataType = DoubleType + + override def dataType = child.dataType match { + case DecimalType.Fixed(precision, scale) => + DecimalType(precision + 4, scale + 4) // Add 4 digits after decimal point, like Hive + case DecimalType.Unlimited => + DecimalType.Unlimited + case _ => + DoubleType + } + override def toString = s"AVG($child)" override def asPartial: SplitEvaluation = { val partialSum = Alias(Sum(child), "PartialSum")() val partialCount = Alias(Count(child), "PartialCount")() - val castedSum = Cast(Sum(partialSum.toAttribute), dataType) - val castedCount = Cast(Sum(partialCount.toAttribute), dataType) - SplitEvaluation( - Divide(castedSum, castedCount), - partialCount :: partialSum :: Nil) + child.dataType match { + case DecimalType.Fixed(_, _) => + // Turn the results to unlimited decimals for the divsion, before going back to fixed + val castedSum = Cast(Sum(partialSum.toAttribute), DecimalType.Unlimited) + val castedCount = Cast(Sum(partialCount.toAttribute), DecimalType.Unlimited) + SplitEvaluation( + Cast(Divide(castedSum, castedCount), dataType), + partialCount :: partialSum :: Nil) + + case _ => + val castedSum = Cast(Sum(partialSum.toAttribute), dataType) + val castedCount = Cast(Sum(partialCount.toAttribute), dataType) + SplitEvaluation( + Divide(castedSum, castedCount), + partialCount :: partialSum :: Nil) + } } override def newInstance() = new AverageFunction(child, this) @@ -306,7 +326,16 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { override def nullable = false - override def dataType = child.dataType + + override def dataType = child.dataType match { + case DecimalType.Fixed(precision, scale) => + DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive + case DecimalType.Unlimited => + DecimalType.Unlimited + case _ => + child.dataType + } + override def toString = s"SUM($child)" override def asPartial: SplitEvaluation = { @@ -322,9 +351,17 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case class SumDistinct(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { - override def nullable = false - override def dataType = child.dataType + + override def dataType = child.dataType match { + case DecimalType.Fixed(precision, scale) => + DecimalType(precision + 10, scale) // Add 10 digits left of decimal point, like Hive + case DecimalType.Unlimited => + DecimalType.Unlimited + case _ => + child.dataType + } + override def toString = s"SUM(DISTINCT $child)" override def newInstance() = new SumDistinctFunction(child, this) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 83e8466ec2aa7..8574cabc43525 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -36,7 +36,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { case class Sqrt(child: Expression) extends UnaryExpression { type EvaluatedType = Any - + def dataType = DoubleType override def foldable = child.foldable def nullable = child.nullable @@ -55,7 +55,9 @@ abstract class BinaryArithmetic extends BinaryExpression { def nullable = left.nullable || right.nullable override lazy val resolved = - left.resolved && right.resolved && left.dataType == right.dataType + left.resolved && right.resolved && + left.dataType == right.dataType && + !DecimalType.isFixed(left.dataType) def dataType = { if (!resolved) { @@ -104,6 +106,8 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "/" + override def nullable = left.nullable || right.nullable || dataType.isInstanceOf[DecimalType] + override def eval(input: Row): Any = dataType match { case _: FractionalType => f2(input, left, right, _.div(_, _)) case _: IntegralType => i2(input, left , right, _.quot(_, _)) @@ -114,6 +118,8 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" + override def nullable = left.nullable || right.nullable || dataType.isInstanceOf[DecimalType] + override def eval(input: Row): Any = i2(input, left, right, _.rem(_, _)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 5a3f013c34579..67f8d411b6bb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import com.google.common.cache.{CacheLoader, CacheBuilder} +import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.language.existentials @@ -485,6 +486,34 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } """.children + case UnscaledValue(child) => + val childEval = expressionEvaluator(child) + + childEval.code ++ + q""" + var $nullTerm = ${childEval.nullTerm} + var $primitiveTerm: Long = if (!$nullTerm) { + ${childEval.primitiveTerm}.toUnscaledLong + } else { + ${defaultPrimitive(LongType)} + } + """.children + + case MakeDecimal(child, precision, scale) => + val childEval = expressionEvaluator(child) + + childEval.code ++ + q""" + var $nullTerm = ${childEval.nullTerm} + var $primitiveTerm: org.apache.spark.sql.catalyst.types.decimal.Decimal = + ${defaultPrimitive(DecimalType())} + + if (!$nullTerm) { + $primitiveTerm = new org.apache.spark.sql.catalyst.types.decimal.Decimal() + $primitiveTerm = $primitiveTerm.setOrNull(${childEval.primitiveTerm}, $precision, $scale) + $nullTerm = $primitiveTerm == null + } + """.children } // If there was no match in the partial function above, we fall back on calling the interpreted @@ -562,7 +591,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case LongType => ru.Literal(Constant(1L)) case ByteType => ru.Literal(Constant(-1.toByte)) case DoubleType => ru.Literal(Constant(-1.toDouble)) - case DecimalType => ru.Literal(Constant(-1)) // Will get implicity converted as needed. + case DecimalType() => q"org.apache.spark.sql.catalyst.types.decimal.Decimal(-1)" case IntegerType => ru.Literal(Constant(-1)) case _ => ru.Literal(Constant(null)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala new file mode 100644 index 0000000000000..d1eab2eb4ed56 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.catalyst.types.{DecimalType, LongType, DoubleType, DataType} + +/** Return the unscaled Long value of a Decimal, assuming it fits in a Long */ +case class UnscaledValue(child: Expression) extends UnaryExpression { + override type EvaluatedType = Any + + override def dataType: DataType = LongType + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"UnscaledValue($child)" + + override def eval(input: Row): Any = { + val childResult = child.eval(input) + if (childResult == null) { + null + } else { + childResult.asInstanceOf[Decimal].toUnscaledLong + } + } +} + +/** Create a Decimal from an unscaled Long value */ +case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends UnaryExpression { + override type EvaluatedType = Decimal + + override def dataType: DataType = DecimalType(precision, scale) + override def foldable = child.foldable + def nullable = child.nullable + override def toString = s"MakeDecimal($child,$precision,$scale)" + + override def eval(input: Row): Decimal = { + val childResult = child.eval(input) + if (childResult == null) { + null + } else { + new Decimal().setOrNull(childResult.asInstanceOf[Long], precision, scale) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index ba240233cae61..93c19325151bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal object Literal { def apply(v: Any): Literal = v match { @@ -31,7 +32,8 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(s, StringType) case b: Boolean => Literal(b, BooleanType) - case d: BigDecimal => Literal(d, DecimalType) + case d: BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) + case d: Decimal => Literal(d, DecimalType.Unlimited) case t: Timestamp => Literal(t, TimestampType) case d: Date => Literal(d, DateType) case a: Array[Byte] => Literal(a, BinaryType) @@ -62,7 +64,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { } // TODO: Specialize -case class MutableLiteral(var value: Any, dataType: DataType, nullable: Boolean = true) +case class MutableLiteral(var value: Any, dataType: DataType, nullable: Boolean = true) extends LeafExpression { type EvaluatedType = Any 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 9ce7c78195830..a4aa322fc52d8 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.LeftSemi 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.types.decimal.Decimal abstract class Optimizer extends RuleExecutor[LogicalPlan] @@ -43,6 +44,8 @@ object DefaultOptimizer extends Optimizer { SimplifyCasts, SimplifyCaseConversionExpressions, OptimizeIn) :: + Batch("Decimal Optimizations", FixedPoint(100), + DecimalAggregates) :: Batch("Filter Pushdown", FixedPoint(100), UnionPushdown, CombineFilters, @@ -390,9 +393,9 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { * evaluated using only the attributes of the left or right side of a join. Other * [[Filter]] conditions are moved into the `condition` of the [[Join]]. * - * And also Pushes down the join filter, where the `condition` can be evaluated using only the - * attributes of the left or right side of sub query when applicable. - * + * And also Pushes down the join filter, where the `condition` can be evaluated using only the + * attributes of the left or right side of sub query when applicable. + * * Check https://cwiki.apache.org/confluence/display/Hive/OuterJoinBehavior for more details */ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { @@ -404,7 +407,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { private def split(condition: Seq[Expression], left: LogicalPlan, right: LogicalPlan) = { val (leftEvaluateCondition, rest) = condition.partition(_.references subsetOf left.outputSet) - val (rightEvaluateCondition, commonCondition) = + val (rightEvaluateCondition, commonCondition) = rest.partition(_.references subsetOf right.outputSet) (leftEvaluateCondition, rightEvaluateCondition, commonCondition) @@ -413,7 +416,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // push the where condition down into join filter case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition)) => - val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = + val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = split(splitConjunctivePredicates(filterCondition), left, right) joinType match { @@ -451,7 +454,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { // push down the join filter into sub query scanning if applicable case f @ Join(left, right, joinType, joinCondition) => - val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = + val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) joinType match { @@ -519,3 +522,26 @@ object SimplifyCaseConversionExpressions extends Rule[LogicalPlan] { } } } + +/** + * Speeds up aggregates on fixed-precision decimals by executing them on unscaled Long values. + * + * This uses the same rules for increasing the precision and scale of the output as + * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion.DecimalPrecision]]. + */ +object DecimalAggregates extends Rule[LogicalPlan] { + import Decimal.MAX_LONG_DIGITS + + /** Maximum number of decimal digits representable precisely in a Double */ + val MAX_DOUBLE_DIGITS = 15 + + def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case Sum(e @ DecimalType.Expression(prec, scale)) if prec + 10 <= MAX_LONG_DIGITS => + MakeDecimal(Sum(UnscaledValue(e)), prec + 10, scale) + + case Average(e @ DecimalType.Expression(prec, scale)) if prec + 4 <= MAX_DOUBLE_DIGITS => + Cast( + Divide(Average(UnscaledValue(e)), Literal(math.pow(10.0, scale), DoubleType)), + DecimalType(prec + 4, scale + 4)) + } +} 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 6069f9b0a68dc..8dda0b182805c 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,7 +19,7 @@ package org.apache.spark.sql.catalyst.types import java.sql.{Date, Timestamp} -import scala.math.Numeric.{BigDecimalAsIfIntegral, DoubleAsIfIntegral, FloatAsIfIntegral} +import scala.math.Numeric.{FloatAsIfIntegral, BigDecimalAsIfIntegral, DoubleAsIfIntegral} import scala.reflect.ClassTag import scala.reflect.runtime.universe.{TypeTag, runtimeMirror, typeTag} import scala.util.parsing.combinator.RegexParsers @@ -33,6 +33,7 @@ import org.apache.spark.sql.catalyst.ScalaReflectionLock import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.sql.catalyst.util.Metadata import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.types.decimal._ object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) @@ -91,11 +92,17 @@ object DataType { | "LongType" ^^^ LongType | "BinaryType" ^^^ BinaryType | "BooleanType" ^^^ BooleanType - | "DecimalType" ^^^ DecimalType | "DateType" ^^^ DateType + | "DecimalType()" ^^^ DecimalType.Unlimited + | fixedDecimalType | "TimestampType" ^^^ TimestampType ) + protected lazy val fixedDecimalType: Parser[DataType] = + ("DecimalType(" ~> "[0-9]+".r) ~ ("," ~> "[0-9]+".r <~ ")") ^^ { + case precision ~ scale => DecimalType(precision.toInt, scale.toInt) + } + protected lazy val arrayType: Parser[DataType] = "ArrayType" ~> "(" ~> dataType ~ "," ~ boolVal <~ ")" ^^ { case tpe ~ _ ~ containsNull => ArrayType(tpe, containsNull) @@ -200,10 +207,18 @@ trait PrimitiveType extends DataType { } object PrimitiveType { - private[sql] val all = Seq(DecimalType, DateType, TimestampType, BinaryType) ++ - NativeType.all - - private[sql] val nameToType = all.map(t => t.typeName -> t).toMap + private val nonDecimals = Seq(DateType, TimestampType, BinaryType) ++ NativeType.all + private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap + + /** Given the string representation of a type, return its DataType */ + private[sql] def nameToType(name: String): DataType = { + val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)""".r + name match { + case "decimal" => DecimalType.Unlimited + case FIXED_DECIMAL(precision, scale) => DecimalType(precision.toInt, scale.toInt) + case other => nonDecimalNameToType(other) + } + } } abstract class NativeType extends DataType { @@ -332,13 +347,58 @@ abstract class FractionalType extends NumericType { private[sql] val asIntegral: Integral[JvmType] } -case object DecimalType extends FractionalType { - private[sql] type JvmType = BigDecimal +/** Precision parameters for a Decimal */ +case class PrecisionInfo(precision: Int, scale: Int) + +/** A Decimal that might have fixed precision and scale, or unlimited values for these */ +case class DecimalType(precisionInfo: Option[PrecisionInfo]) extends FractionalType { + private[sql] type JvmType = Decimal @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } - private[sql] val numeric = implicitly[Numeric[BigDecimal]] - private[sql] val fractional = implicitly[Fractional[BigDecimal]] - private[sql] val ordering = implicitly[Ordering[JvmType]] - private[sql] val asIntegral = BigDecimalAsIfIntegral + private[sql] val numeric = Decimal.DecimalIsFractional + private[sql] val fractional = Decimal.DecimalIsFractional + private[sql] val ordering = Decimal.DecimalIsFractional + private[sql] val asIntegral = Decimal.DecimalAsIfIntegral + + override def typeName: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"decimal($precision,$scale)" + case None => "decimal" + } + + override def toString: String = precisionInfo match { + case Some(PrecisionInfo(precision, scale)) => s"DecimalType($precision,$scale)" + case None => "DecimalType()" + } +} + +/** Extra factory methods and pattern matchers for Decimals */ +object DecimalType { + val Unlimited: DecimalType = DecimalType(None) + + object Fixed { + def unapply(t: DecimalType): Option[(Int, Int)] = + t.precisionInfo.map(p => (p.precision, p.scale)) + } + + object Expression { + def unapply(e: Expression): Option[(Int, Int)] = e.dataType match { + case t: DecimalType => t.precisionInfo.map(p => (p.precision, p.scale)) + case _ => None + } + } + + def apply(): DecimalType = Unlimited + + def apply(precision: Int, scale: Int): DecimalType = + DecimalType(Some(PrecisionInfo(precision, scale))) + + def unapply(t: DataType): Boolean = t.isInstanceOf[DecimalType] + + def unapply(e: Expression): Boolean = e.dataType.isInstanceOf[DecimalType] + + def isFixed(dataType: DataType): Boolean = dataType match { + case DecimalType.Fixed(_, _) => true + case _ => false + } } case object DoubleType extends FractionalType { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala new file mode 100644 index 0000000000000..708362acf32dc --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/decimal/Decimal.scala @@ -0,0 +1,335 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.types.decimal + +import org.apache.spark.annotation.DeveloperApi + +/** + * A mutable implementation of BigDecimal that can hold a Long if values are small enough. + * + * The semantics of the fields are as follows: + * - _precision and _scale represent the SQL precision and scale we are looking for + * - If decimalVal is set, it represents the whole decimal value + * - Otherwise, the decimal value is longVal / (10 ** _scale) + */ +final class Decimal extends Ordered[Decimal] with Serializable { + import Decimal.{MAX_LONG_DIGITS, POW_10, ROUNDING_MODE, BIG_DEC_ZERO} + + private var decimalVal: BigDecimal = null + private var longVal: Long = 0L + private var _precision: Int = 1 + private var _scale: Int = 0 + + def precision: Int = _precision + def scale: Int = _scale + + /** + * Set this Decimal to the given Long. Will have precision 20 and scale 0. + */ + def set(longVal: Long): Decimal = { + if (longVal <= -POW_10(MAX_LONG_DIGITS) || longVal >= POW_10(MAX_LONG_DIGITS)) { + // We can't represent this compactly as a long without risking overflow + this.decimalVal = BigDecimal(longVal) + this.longVal = 0L + } else { + this.decimalVal = null + this.longVal = longVal + } + this._precision = 20 + this._scale = 0 + this + } + + /** + * Set this Decimal to the given Int. Will have precision 10 and scale 0. + */ + def set(intVal: Int): Decimal = { + this.decimalVal = null + this.longVal = intVal + this._precision = 10 + this._scale = 0 + this + } + + /** + * Set this Decimal to the given unscaled Long, with a given precision and scale. + */ + def set(unscaled: Long, precision: Int, scale: Int): Decimal = { + if (setOrNull(unscaled, precision, scale) == null) { + throw new IllegalArgumentException("Unscaled value too large for precision") + } + this + } + + /** + * Set this Decimal to the given unscaled Long, with a given precision and scale, + * and return it, or return null if it cannot be set due to overflow. + */ + def setOrNull(unscaled: Long, precision: Int, scale: Int): Decimal = { + if (unscaled <= -POW_10(MAX_LONG_DIGITS) || unscaled >= POW_10(MAX_LONG_DIGITS)) { + // We can't represent this compactly as a long without risking overflow + if (precision < 19) { + return null // Requested precision is too low to represent this value + } + this.decimalVal = BigDecimal(longVal) + this.longVal = 0L + } else { + val p = POW_10(math.min(precision, MAX_LONG_DIGITS)) + if (unscaled <= -p || unscaled >= p) { + return null // Requested precision is too low to represent this value + } + this.decimalVal = null + this.longVal = unscaled + } + this._precision = precision + this._scale = scale + this + } + + /** + * Set this Decimal to the given BigDecimal value, with a given precision and scale. + */ + def set(decimal: BigDecimal, precision: Int, scale: Int): Decimal = { + this.decimalVal = decimal.setScale(scale, ROUNDING_MODE) + require(decimalVal.precision <= precision, "Overflowed precision") + this.longVal = 0L + this._precision = precision + this._scale = scale + this + } + + /** + * Set this Decimal to the given BigDecimal value, inheriting its precision and scale. + */ + def set(decimal: BigDecimal): Decimal = { + this.decimalVal = decimal + this.longVal = 0L + this._precision = decimal.precision + this._scale = decimal.scale + this + } + + /** + * Set this Decimal to the given Decimal value. + */ + def set(decimal: Decimal): Decimal = { + this.decimalVal = decimal.decimalVal + this.longVal = decimal.longVal + this._precision = decimal._precision + this._scale = decimal._scale + this + } + + def toBigDecimal: BigDecimal = { + if (decimalVal.ne(null)) { + decimalVal + } else { + BigDecimal(longVal, _scale) + } + } + + def toUnscaledLong: Long = { + if (decimalVal.ne(null)) { + decimalVal.underlying().unscaledValue().longValue() + } else { + longVal + } + } + + override def toString: String = toBigDecimal.toString() + + @DeveloperApi + def toDebugString: String = { + if (decimalVal.ne(null)) { + s"Decimal(expanded,$decimalVal,$precision,$scale})" + } else { + s"Decimal(compact,$longVal,$precision,$scale})" + } + } + + def toDouble: Double = toBigDecimal.doubleValue() + + def toFloat: Float = toBigDecimal.floatValue() + + def toLong: Long = { + if (decimalVal.eq(null)) { + longVal / POW_10(_scale) + } else { + decimalVal.longValue() + } + } + + def toInt: Int = toLong.toInt + + def toShort: Short = toLong.toShort + + def toByte: Byte = toLong.toByte + + /** + * Update precision and scale while keeping our value the same, and return true if successful. + * + * @return true if successful, false if overflow would occur + */ + def changePrecision(precision: Int, scale: Int): Boolean = { + // First, update our longVal if we can, or transfer over to using a BigDecimal + if (decimalVal.eq(null)) { + if (scale < _scale) { + // Easier case: we just need to divide our scale down + val diff = _scale - scale + val droppedDigits = longVal % POW_10(diff) + longVal /= POW_10(diff) + if (math.abs(droppedDigits) * 2 >= POW_10(diff)) { + longVal += (if (longVal < 0) -1L else 1L) + } + } else if (scale > _scale) { + // We might be able to multiply longVal by a power of 10 and not overflow, but if not, + // switch to using a BigDecimal + val diff = scale - _scale + val p = POW_10(math.max(MAX_LONG_DIGITS - diff, 0)) + if (diff <= MAX_LONG_DIGITS && longVal > -p && longVal < p) { + // Multiplying longVal by POW_10(diff) will still keep it below MAX_LONG_DIGITS + longVal *= POW_10(diff) + } else { + // Give up on using Longs; switch to BigDecimal, which we'll modify below + decimalVal = BigDecimal(longVal, _scale) + } + } + // In both cases, we will check whether our precision is okay below + } + + if (decimalVal.ne(null)) { + // We get here if either we started with a BigDecimal, or we switched to one because we would + // have overflowed our Long; in either case we must rescale decimalVal to the new scale. + val newVal = decimalVal.setScale(scale, ROUNDING_MODE) + if (newVal.precision > precision) { + return false + } + decimalVal = newVal + } else { + // We're still using Longs, but we should check whether we match the new precision + val p = POW_10(math.min(_precision, MAX_LONG_DIGITS)) + if (longVal <= -p || longVal >= p) { + // Note that we shouldn't have been able to fix this by switching to BigDecimal + return false + } + } + + _precision = precision + _scale = scale + true + } + + override def clone(): Decimal = new Decimal().set(this) + + override def compare(other: Decimal): Int = { + if (decimalVal.eq(null) && other.decimalVal.eq(null) && _scale == other._scale) { + if (longVal < other.longVal) -1 else if (longVal == other.longVal) 0 else 1 + } else { + toBigDecimal.compare(other.toBigDecimal) + } + } + + override def equals(other: Any) = other match { + case d: Decimal => + compare(d) == 0 + case _ => + false + } + + override def hashCode(): Int = toBigDecimal.hashCode() + + def isZero: Boolean = if (decimalVal.ne(null)) decimalVal == BIG_DEC_ZERO else longVal == 0 + + def + (that: Decimal): Decimal = Decimal(toBigDecimal + that.toBigDecimal) + + def - (that: Decimal): Decimal = Decimal(toBigDecimal - that.toBigDecimal) + + def * (that: Decimal): Decimal = Decimal(toBigDecimal * that.toBigDecimal) + + def / (that: Decimal): Decimal = + if (that.isZero) null else Decimal(toBigDecimal / that.toBigDecimal) + + def % (that: Decimal): Decimal = + if (that.isZero) null else Decimal(toBigDecimal % that.toBigDecimal) + + def remainder(that: Decimal): Decimal = this % that + + def unary_- : Decimal = { + if (decimalVal.ne(null)) { + Decimal(-decimalVal) + } else { + Decimal(-longVal, precision, scale) + } + } +} + +object Decimal { + private val ROUNDING_MODE = BigDecimal.RoundingMode.HALF_UP + + /** Maximum number of decimal digits a Long can represent */ + val MAX_LONG_DIGITS = 18 + + private val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong) + + private val BIG_DEC_ZERO = BigDecimal(0) + + def apply(value: Double): Decimal = new Decimal().set(value) + + def apply(value: Long): Decimal = new Decimal().set(value) + + def apply(value: Int): Decimal = new Decimal().set(value) + + def apply(value: BigDecimal): Decimal = new Decimal().set(value) + + def apply(value: BigDecimal, precision: Int, scale: Int): Decimal = + new Decimal().set(value, precision, scale) + + def apply(unscaled: Long, precision: Int, scale: Int): Decimal = + new Decimal().set(unscaled, precision, scale) + + def apply(value: String): Decimal = new Decimal().set(BigDecimal(value)) + + // Evidence parameters for Decimal considered either as Fractional or Integral. We provide two + // parameters inheriting from a common trait since both traits define mkNumericOps. + // See scala.math's Numeric.scala for examples for Scala's built-in types. + + /** Common methods for Decimal evidence parameters */ + trait DecimalIsConflicted extends Numeric[Decimal] { + override def plus(x: Decimal, y: Decimal): Decimal = x + y + override def times(x: Decimal, y: Decimal): Decimal = x * y + override def minus(x: Decimal, y: Decimal): Decimal = x - y + override def negate(x: Decimal): Decimal = -x + override def toDouble(x: Decimal): Double = x.toDouble + override def toFloat(x: Decimal): Float = x.toFloat + override def toInt(x: Decimal): Int = x.toInt + override def toLong(x: Decimal): Long = x.toLong + override def fromInt(x: Int): Decimal = new Decimal().set(x) + override def compare(x: Decimal, y: Decimal): Int = x.compare(y) + } + + /** A [[scala.math.Fractional]] evidence parameter for Decimals. */ + object DecimalIsFractional extends DecimalIsConflicted with Fractional[Decimal] { + override def div(x: Decimal, y: Decimal): Decimal = x / y + } + + /** A [[scala.math.Integral]] evidence parameter for Decimals. */ + object DecimalAsIfIntegral extends DecimalIsConflicted with Integral[Decimal] { + override def quot(x: Decimal, y: Decimal): Decimal = x / y + override def rem(x: Decimal, y: Decimal): Decimal = x % y + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 430f0664b7d58..21b2c8e20d4db 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -96,7 +96,7 @@ class ScalaReflectionSuite extends FunSuite { StructField("byteField", ByteType, nullable = true), StructField("booleanField", BooleanType, nullable = true), StructField("stringField", StringType, nullable = true), - StructField("decimalField", DecimalType, nullable = true), + StructField("decimalField", DecimalType.Unlimited, nullable = true), StructField("dateField", DateType, nullable = true), StructField("timestampField", TimestampType, nullable = true), StructField("binaryField", BinaryType, nullable = true))), @@ -199,7 +199,7 @@ class ScalaReflectionSuite extends FunSuite { assert(DoubleType === typeOfObject(1.7976931348623157E308)) // DecimalType - assert(DecimalType === typeOfObject(BigDecimal("1.7976931348623157E318"))) + assert(DecimalType.Unlimited === typeOfObject(BigDecimal("1.7976931348623157E318"))) // DateType assert(DateType === typeOfObject(Date.valueOf("2014-07-25"))) @@ -211,19 +211,19 @@ class ScalaReflectionSuite extends FunSuite { assert(NullType === typeOfObject(null)) def typeOfObject1: PartialFunction[Any, DataType] = typeOfObject orElse { - case value: java.math.BigInteger => DecimalType - case value: java.math.BigDecimal => DecimalType + case value: java.math.BigInteger => DecimalType.Unlimited + case value: java.math.BigDecimal => DecimalType.Unlimited case _ => StringType } - assert(DecimalType === typeOfObject1( + assert(DecimalType.Unlimited === typeOfObject1( new BigInteger("92233720368547758070"))) - assert(DecimalType === typeOfObject1( + assert(DecimalType.Unlimited === typeOfObject1( new java.math.BigDecimal("1.7976931348623157E318"))) assert(StringType === typeOfObject1(BigInt("92233720368547758070"))) def typeOfObject2: PartialFunction[Any, DataType] = typeOfObject orElse { - case value: java.math.BigInteger => DecimalType + case value: java.math.BigInteger => DecimalType.Unlimited } intercept[MatchError](typeOfObject2(BigInt("92233720368547758070"))) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 7b45738c4fc95..33a3cba3d4c0e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -38,7 +38,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("a", StringType)(), AttributeReference("b", StringType)(), AttributeReference("c", DoubleType)(), - AttributeReference("d", DecimalType)(), + AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) before { @@ -119,7 +119,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { AttributeReference("a", StringType)(), AttributeReference("b", StringType)(), AttributeReference("c", DoubleType)(), - AttributeReference("d", DecimalType)(), + AttributeReference("d", DecimalType.Unlimited)(), AttributeReference("e", ShortType)()) val expr0 = 'a / 2 @@ -137,7 +137,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { assert(pl(0).dataType == DoubleType) assert(pl(1).dataType == DoubleType) assert(pl(2).dataType == DoubleType) - assert(pl(3).dataType == DecimalType) + assert(pl(3).dataType == DecimalType.Unlimited) assert(pl(4).dataType == DoubleType) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala new file mode 100644 index 0000000000000..d5b7d2789a103 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{Project, LocalRelation} +import org.apache.spark.sql.catalyst.types._ +import org.scalatest.{BeforeAndAfter, FunSuite} + +class DecimalPrecisionSuite extends FunSuite with BeforeAndAfter { + val catalog = new SimpleCatalog(false) + val analyzer = new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = false) + + val relation = LocalRelation( + AttributeReference("i", IntegerType)(), + AttributeReference("d1", DecimalType(2, 1))(), + AttributeReference("d2", DecimalType(5, 2))(), + AttributeReference("u", DecimalType.Unlimited)(), + AttributeReference("f", FloatType)() + ) + + val i: Expression = UnresolvedAttribute("i") + val d1: Expression = UnresolvedAttribute("d1") + val d2: Expression = UnresolvedAttribute("d2") + val u: Expression = UnresolvedAttribute("u") + val f: Expression = UnresolvedAttribute("f") + + before { + catalog.registerTable(None, "table", relation) + } + + private def checkType(expression: Expression, expectedType: DataType): Unit = { + val plan = Project(Seq(Alias(expression, "c")()), relation) + assert(analyzer(plan).schema.fields(0).dataType === expectedType) + } + + test("basic operations") { + checkType(Add(d1, d2), DecimalType(6, 2)) + checkType(Subtract(d1, d2), DecimalType(6, 2)) + checkType(Multiply(d1, d2), DecimalType(8, 3)) + checkType(Divide(d1, d2), DecimalType(10, 7)) + checkType(Divide(d2, d1), DecimalType(10, 6)) + checkType(Remainder(d1, d2), DecimalType(3, 2)) + checkType(Remainder(d2, d1), DecimalType(3, 2)) + checkType(Sum(d1), DecimalType(12, 1)) + checkType(Average(d1), DecimalType(6, 5)) + + checkType(Add(Add(d1, d2), d1), DecimalType(7, 2)) + checkType(Add(Add(Add(d1, d2), d1), d2), DecimalType(8, 2)) + checkType(Add(Add(d1, d2), Add(d1, d2)), DecimalType(7, 2)) + } + + test("bringing in primitive types") { + checkType(Add(d1, i), DecimalType(12, 1)) + checkType(Add(d1, f), DoubleType) + checkType(Add(i, d1), DecimalType(12, 1)) + checkType(Add(f, d1), DoubleType) + checkType(Add(d1, Cast(i, LongType)), DecimalType(22, 1)) + checkType(Add(d1, Cast(i, ShortType)), DecimalType(7, 1)) + checkType(Add(d1, Cast(i, ByteType)), DecimalType(5, 1)) + checkType(Add(d1, Cast(i, DoubleType)), DoubleType) + } + + test("unlimited decimals make everything else cast up") { + for (expr <- Seq(d1, d2, i, f, u)) { + checkType(Add(expr, u), DecimalType.Unlimited) + checkType(Subtract(expr, u), DecimalType.Unlimited) + checkType(Multiply(expr, u), DecimalType.Unlimited) + checkType(Divide(expr, u), DecimalType.Unlimited) + checkType(Remainder(expr, u), DecimalType.Unlimited) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index baeb9b0cf5964..dfa2d958c0faf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -68,6 +68,21 @@ class HiveTypeCoercionSuite extends FunSuite { widenTest(LongType, FloatType, Some(FloatType)) widenTest(LongType, DoubleType, Some(DoubleType)) + // Casting up to unlimited-precision decimal + widenTest(IntegerType, DecimalType.Unlimited, Some(DecimalType.Unlimited)) + widenTest(DoubleType, DecimalType.Unlimited, Some(DecimalType.Unlimited)) + widenTest(DecimalType(3, 2), DecimalType.Unlimited, Some(DecimalType.Unlimited)) + widenTest(DecimalType.Unlimited, IntegerType, Some(DecimalType.Unlimited)) + widenTest(DecimalType.Unlimited, DoubleType, Some(DecimalType.Unlimited)) + widenTest(DecimalType.Unlimited, DecimalType(3, 2), Some(DecimalType.Unlimited)) + + // No up-casting for fixed-precision decimal (this is handled by arithmetic rules) + widenTest(DecimalType(2, 1), DecimalType(3, 2), None) + widenTest(DecimalType(2, 1), DoubleType, None) + widenTest(DecimalType(2, 1), IntegerType, None) + widenTest(DoubleType, DecimalType(2, 1), None) + widenTest(IntegerType, DecimalType(2, 1), None) + // StringType widenTest(NullType, StringType, Some(StringType)) widenTest(StringType, StringType, Some(StringType)) @@ -92,7 +107,7 @@ class HiveTypeCoercionSuite extends FunSuite { def ruleTest(initial: Expression, transformed: Expression) { val testRelation = LocalRelation(AttributeReference("a", IntegerType)()) assert(booleanCasts(Project(Seq(Alias(initial, "a")()), testRelation)) == - Project(Seq(Alias(transformed, "a")()), testRelation)) + Project(Seq(Alias(transformed, "a")()), testRelation)) } // Remove superflous boolean -> boolean casts. ruleTest(Cast(Literal(true), BooleanType), Literal(true)) 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 5657bc555edf9..6bfa0dbd65ba7 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 @@ -21,6 +21,7 @@ import java.sql.{Date, Timestamp} import scala.collection.immutable.HashSet +import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.scalatest.FunSuite import org.scalatest.Matchers._ import org.scalactic.TripleEqualsSupport.Spread @@ -138,7 +139,7 @@ class ExpressionEvaluationSuite extends FunSuite { val actual = try evaluate(expression, inputRow) catch { case e: Exception => fail(s"Exception evaluating $expression", e) } - actual.asInstanceOf[Double] shouldBe expected + actual.asInstanceOf[Double] shouldBe expected } test("IN") { @@ -165,7 +166,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(InSet(three, nS, three +: nullS), false) checkEvaluation(InSet(one, hS, one +: s) && InSet(two, hS, two +: s), true) } - + test("MaxOf") { checkEvaluation(MaxOf(1, 2), 2) checkEvaluation(MaxOf(2, 1), 2) @@ -265,9 +266,9 @@ class ExpressionEvaluationSuite extends FunSuite { val ts = Timestamp.valueOf(nts) checkEvaluation("abdef" cast StringType, "abdef") - checkEvaluation("abdef" cast DecimalType, null) + checkEvaluation("abdef" cast DecimalType.Unlimited, null) checkEvaluation("abdef" cast TimestampType, null) - checkEvaluation("12.65" cast DecimalType, BigDecimal(12.65)) + checkEvaluation("12.65" cast DecimalType.Unlimited, Decimal(12.65)) checkEvaluation(Literal(1) cast LongType, 1) checkEvaluation(Cast(Literal(1000) cast TimestampType, LongType), 1.toLong) @@ -289,12 +290,12 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Cast(Cast(Cast(Cast( Cast("5" cast ByteType, ShortType), IntegerType), FloatType), DoubleType), LongType), 5) - checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast ByteType, TimestampType), DecimalType), LongType), StringType), ShortType), 0) - checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast TimestampType, ByteType), DecimalType), LongType), StringType), ShortType), null) - checkEvaluation(Cast(Cast(Cast(Cast( - Cast("5" cast DecimalType, ByteType), TimestampType), LongType), StringType), ShortType), 0) + checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast + ByteType, TimestampType), DecimalType.Unlimited), LongType), StringType), ShortType), 0) + checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast + TimestampType, ByteType), DecimalType.Unlimited), LongType), StringType), ShortType), null) + checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast + DecimalType.Unlimited, ByteType), TimestampType), LongType), StringType), ShortType), 0) checkEvaluation(Literal(true) cast IntegerType, 1) checkEvaluation(Literal(false) cast IntegerType, 0) checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1) @@ -302,7 +303,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("23" cast DoubleType, 23d) checkEvaluation("23" cast IntegerType, 23) checkEvaluation("23" cast FloatType, 23f) - checkEvaluation("23" cast DecimalType, 23: BigDecimal) + checkEvaluation("23" cast DecimalType.Unlimited, Decimal(23)) checkEvaluation("23" cast ByteType, 23.toByte) checkEvaluation("23" cast ShortType, 23.toShort) checkEvaluation("2012-12-11" cast DoubleType, null) @@ -311,7 +312,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24d) checkEvaluation(Literal(23) + Cast(true, IntegerType), 24) checkEvaluation(Literal(23f) + Cast(true, FloatType), 24f) - checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24: BigDecimal) + checkEvaluation(Literal(Decimal(23)) + Cast(true, DecimalType.Unlimited), Decimal(24)) checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24.toByte) checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24.toShort) @@ -325,7 +326,8 @@ class ExpressionEvaluationSuite extends FunSuite { assert(("abcdef" cast IntegerType).nullable === true) assert(("abcdef" cast ShortType).nullable === true) assert(("abcdef" cast ByteType).nullable === true) - assert(("abcdef" cast DecimalType).nullable === true) + assert(("abcdef" cast DecimalType.Unlimited).nullable === true) + assert(("abcdef" cast DecimalType(4, 2)).nullable === true) assert(("abcdef" cast DoubleType).nullable === true) assert(("abcdef" cast FloatType).nullable === true) @@ -338,6 +340,64 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(d1) < Literal(d2), true) } + test("casting to fixed-precision decimals") { + // Overflow and rounding for casting to fixed-precision decimals: + // - Values should round with HALF_UP mode by default when you lower scale + // - Values that would overflow the target precision should turn into null + // - Because of this, casts to fixed-precision decimals should be nullable + + assert(Cast(Literal(123), DecimalType.Unlimited).nullable === false) + assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === false) + assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === false) + assert(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable === false) + + assert(Cast(Literal(123), DecimalType(2, 1)).nullable === true) + assert(Cast(Literal(10.03f), DecimalType(2, 1)).nullable === true) + assert(Cast(Literal(10.03), DecimalType(2, 1)).nullable === true) + assert(Cast(Literal(Decimal(10.03)), DecimalType(2, 1)).nullable === true) + + checkEvaluation(Cast(Literal(123), DecimalType.Unlimited), Decimal(123)) + checkEvaluation(Cast(Literal(123), DecimalType(3, 0)), Decimal(123)) + checkEvaluation(Cast(Literal(123), DecimalType(3, 1)), null) + checkEvaluation(Cast(Literal(123), DecimalType(2, 0)), null) + + checkEvaluation(Cast(Literal(10.03), DecimalType.Unlimited), Decimal(10.03)) + checkEvaluation(Cast(Literal(10.03), DecimalType(4, 2)), Decimal(10.03)) + checkEvaluation(Cast(Literal(10.03), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(Cast(Literal(10.03), DecimalType(2, 0)), Decimal(10)) + checkEvaluation(Cast(Literal(10.03), DecimalType(1, 0)), null) + checkEvaluation(Cast(Literal(10.03), DecimalType(2, 1)), null) + checkEvaluation(Cast(Literal(10.03), DecimalType(3, 2)), null) + checkEvaluation(Cast(Literal(Decimal(10.03)), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(Cast(Literal(Decimal(10.03)), DecimalType(3, 2)), null) + + checkEvaluation(Cast(Literal(10.05), DecimalType.Unlimited), Decimal(10.05)) + checkEvaluation(Cast(Literal(10.05), DecimalType(4, 2)), Decimal(10.05)) + checkEvaluation(Cast(Literal(10.05), DecimalType(3, 1)), Decimal(10.1)) + checkEvaluation(Cast(Literal(10.05), DecimalType(2, 0)), Decimal(10)) + checkEvaluation(Cast(Literal(10.05), DecimalType(1, 0)), null) + checkEvaluation(Cast(Literal(10.05), DecimalType(2, 1)), null) + checkEvaluation(Cast(Literal(10.05), DecimalType(3, 2)), null) + checkEvaluation(Cast(Literal(Decimal(10.05)), DecimalType(3, 1)), Decimal(10.1)) + checkEvaluation(Cast(Literal(Decimal(10.05)), DecimalType(3, 2)), null) + + checkEvaluation(Cast(Literal(9.95), DecimalType(3, 2)), Decimal(9.95)) + checkEvaluation(Cast(Literal(9.95), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(Cast(Literal(9.95), DecimalType(2, 0)), Decimal(10)) + checkEvaluation(Cast(Literal(9.95), DecimalType(2, 1)), null) + checkEvaluation(Cast(Literal(9.95), DecimalType(1, 0)), null) + checkEvaluation(Cast(Literal(Decimal(9.95)), DecimalType(3, 1)), Decimal(10.0)) + checkEvaluation(Cast(Literal(Decimal(9.95)), DecimalType(1, 0)), null) + + checkEvaluation(Cast(Literal(-9.95), DecimalType(3, 2)), Decimal(-9.95)) + checkEvaluation(Cast(Literal(-9.95), DecimalType(3, 1)), Decimal(-10.0)) + checkEvaluation(Cast(Literal(-9.95), DecimalType(2, 0)), Decimal(-10)) + checkEvaluation(Cast(Literal(-9.95), DecimalType(2, 1)), null) + checkEvaluation(Cast(Literal(-9.95), DecimalType(1, 0)), null) + checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(3, 1)), Decimal(-10.0)) + checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(1, 0)), null) + } + test("timestamp") { val ts1 = new Timestamp(12) val ts2 = new Timestamp(123) @@ -374,7 +434,7 @@ class ExpressionEvaluationSuite extends FunSuite { millis.toFloat / 1000) checkEvaluation(Cast(Cast(millis.toDouble / 1000, TimestampType), DoubleType), millis.toDouble / 1000) - checkEvaluation(Cast(Literal(BigDecimal(1)) cast TimestampType, DecimalType), 1) + checkEvaluation(Cast(Literal(Decimal(1)) cast TimestampType, DecimalType.Unlimited), Decimal(1)) // A test for higher precision than millis checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) @@ -673,7 +733,7 @@ class ExpressionEvaluationSuite extends FunSuite { val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble)) val rowSequence = inputSequence.map(l => new GenericRow(Array[Any](l.toDouble))) val d = 'a.double.at(0) - + for ((row, expected) <- rowSequence zip expectedResults) { checkEvaluation(Sqrt(d), expected, row) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala new file mode 100644 index 0000000000000..5aa263484d5ed --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/types/decimal/DecimalSuite.scala @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.types.decimal + +import org.scalatest.{PrivateMethodTester, FunSuite} + +import scala.language.postfixOps + +class DecimalSuite extends FunSuite with PrivateMethodTester { + test("creating decimals") { + /** Check that a Decimal has the given string representation, precision and scale */ + def checkDecimal(d: Decimal, string: String, precision: Int, scale: Int): Unit = { + assert(d.toString === string) + assert(d.precision === precision) + assert(d.scale === scale) + } + + checkDecimal(new Decimal(), "0", 1, 0) + checkDecimal(Decimal(BigDecimal("10.030")), "10.030", 5, 3) + checkDecimal(Decimal(BigDecimal("10.030"), 4, 1), "10.0", 4, 1) + checkDecimal(Decimal(BigDecimal("-9.95"), 4, 1), "-10.0", 4, 1) + checkDecimal(Decimal("10.030"), "10.030", 5, 3) + checkDecimal(Decimal(10.03), "10.03", 4, 2) + checkDecimal(Decimal(17L), "17", 20, 0) + checkDecimal(Decimal(17), "17", 10, 0) + checkDecimal(Decimal(17L, 2, 1), "1.7", 2, 1) + checkDecimal(Decimal(170L, 4, 2), "1.70", 4, 2) + checkDecimal(Decimal(17L, 24, 1), "1.7", 24, 1) + checkDecimal(Decimal(1e17.toLong, 18, 0), 1e17.toLong.toString, 18, 0) + checkDecimal(Decimal(Long.MaxValue), Long.MaxValue.toString, 20, 0) + checkDecimal(Decimal(Long.MinValue), Long.MinValue.toString, 20, 0) + intercept[IllegalArgumentException](Decimal(170L, 2, 1)) + intercept[IllegalArgumentException](Decimal(170L, 2, 0)) + intercept[IllegalArgumentException](Decimal(BigDecimal("10.030"), 2, 1)) + intercept[IllegalArgumentException](Decimal(BigDecimal("-9.95"), 2, 1)) + intercept[IllegalArgumentException](Decimal(1e17.toLong, 17, 0)) + } + + test("double and long values") { + /** Check that a Decimal converts to the given double and long values */ + def checkValues(d: Decimal, doubleValue: Double, longValue: Long): Unit = { + assert(d.toDouble === doubleValue) + assert(d.toLong === longValue) + } + + checkValues(new Decimal(), 0.0, 0L) + checkValues(Decimal(BigDecimal("10.030")), 10.03, 10L) + checkValues(Decimal(BigDecimal("10.030"), 4, 1), 10.0, 10L) + checkValues(Decimal(BigDecimal("-9.95"), 4, 1), -10.0, -10L) + checkValues(Decimal(10.03), 10.03, 10L) + checkValues(Decimal(17L), 17.0, 17L) + checkValues(Decimal(17), 17.0, 17L) + checkValues(Decimal(17L, 2, 1), 1.7, 1L) + checkValues(Decimal(170L, 4, 2), 1.7, 1L) + checkValues(Decimal(1e16.toLong), 1e16, 1e16.toLong) + checkValues(Decimal(1e17.toLong), 1e17, 1e17.toLong) + checkValues(Decimal(1e18.toLong), 1e18, 1e18.toLong) + checkValues(Decimal(2e18.toLong), 2e18, 2e18.toLong) + checkValues(Decimal(Long.MaxValue), Long.MaxValue.toDouble, Long.MaxValue) + checkValues(Decimal(Long.MinValue), Long.MinValue.toDouble, Long.MinValue) + checkValues(Decimal(Double.MaxValue), Double.MaxValue, 0L) + checkValues(Decimal(Double.MinValue), Double.MinValue, 0L) + } + + // Accessor for the BigDecimal value of a Decimal, which will be null if it's using Longs + private val decimalVal = PrivateMethod[BigDecimal]('decimalVal) + + /** Check whether a decimal is represented compactly (passing whether we expect it to be) */ + private def checkCompact(d: Decimal, expected: Boolean): Unit = { + val isCompact = d.invokePrivate(decimalVal()).eq(null) + assert(isCompact == expected, s"$d ${if (expected) "was not" else "was"} compact") + } + + test("small decimals represented as unscaled long") { + checkCompact(new Decimal(), true) + checkCompact(Decimal(BigDecimal(10.03)), false) + checkCompact(Decimal(BigDecimal(1e20)), false) + checkCompact(Decimal(17L), true) + checkCompact(Decimal(17), true) + checkCompact(Decimal(17L, 2, 1), true) + checkCompact(Decimal(170L, 4, 2), true) + checkCompact(Decimal(17L, 24, 1), true) + checkCompact(Decimal(1e16.toLong), true) + checkCompact(Decimal(1e17.toLong), true) + checkCompact(Decimal(1e18.toLong - 1), true) + checkCompact(Decimal(- 1e18.toLong + 1), true) + checkCompact(Decimal(1e18.toLong - 1, 30, 10), true) + checkCompact(Decimal(- 1e18.toLong + 1, 30, 10), true) + checkCompact(Decimal(1e18.toLong), false) + checkCompact(Decimal(-1e18.toLong), false) + checkCompact(Decimal(1e18.toLong, 30, 10), false) + checkCompact(Decimal(-1e18.toLong, 30, 10), false) + checkCompact(Decimal(Long.MaxValue), false) + checkCompact(Decimal(Long.MinValue), false) + } + + test("hash code") { + assert(Decimal(123).hashCode() === (123).##) + assert(Decimal(-123).hashCode() === (-123).##) + assert(Decimal(123.312).hashCode() === (123.312).##) + assert(Decimal(Int.MaxValue).hashCode() === Int.MaxValue.##) + assert(Decimal(Long.MaxValue).hashCode() === Long.MaxValue.##) + assert(Decimal(BigDecimal(123)).hashCode() === (123).##) + + val reallyBig = BigDecimal("123182312312313232112312312123.1231231231") + assert(Decimal(reallyBig).hashCode() === reallyBig.hashCode) + } + + test("equals") { + // The decimals on the left are stored compactly, while the ones on the right aren't + checkCompact(Decimal(123), true) + checkCompact(Decimal(BigDecimal(123)), false) + checkCompact(Decimal("123"), false) + assert(Decimal(123) === Decimal(BigDecimal(123))) + assert(Decimal(123) === Decimal(BigDecimal("123.00"))) + assert(Decimal(-123) === Decimal(BigDecimal(-123))) + assert(Decimal(-123) === Decimal(BigDecimal("-123.00"))) + } + + test("isZero") { + assert(Decimal(0).isZero) + assert(Decimal(0, 4, 2).isZero) + assert(Decimal("0").isZero) + assert(Decimal("0.000").isZero) + assert(!Decimal(1).isZero) + assert(!Decimal(1, 4, 2).isZero) + assert(!Decimal("1").isZero) + assert(!Decimal("0.001").isZero) + } + + test("arithmetic") { + assert(Decimal(100) + Decimal(-100) === Decimal(0)) + assert(Decimal(100) + Decimal(-100) === Decimal(0)) + assert(Decimal(100) * Decimal(-100) === Decimal(-10000)) + assert(Decimal(1e13) * Decimal(1e13) === Decimal(1e26)) + assert(Decimal(100) / Decimal(-100) === Decimal(-1)) + assert(Decimal(100) / Decimal(0) === null) + assert(Decimal(100) % Decimal(-100) === Decimal(0)) + assert(Decimal(100) % Decimal(3) === Decimal(1)) + assert(Decimal(-100) % Decimal(3) === Decimal(-1)) + assert(Decimal(100) % Decimal(0) === null) + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index 0c85cdc0aa640..c38354039d686 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -52,11 +52,6 @@ public abstract class DataType { */ public static final TimestampType TimestampType = new TimestampType(); - /** - * Gets the DecimalType object. - */ - public static final DecimalType DecimalType = new DecimalType(); - /** * Gets the DoubleType object. */ diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java index bc54c078d7a4e..60752451ecfc7 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DecimalType.java @@ -19,9 +19,61 @@ /** * The data type representing java.math.BigDecimal values. - * - * {@code DecimalType} is represented by the singleton object {@link DataType#DecimalType}. */ public class DecimalType extends DataType { - protected DecimalType() {} + private boolean hasPrecisionInfo; + private int precision; + private int scale; + + public DecimalType(int precision, int scale) { + this.hasPrecisionInfo = true; + this.precision = precision; + this.scale = scale; + } + + public DecimalType() { + this.hasPrecisionInfo = false; + this.precision = -1; + this.scale = -1; + } + + public boolean isUnlimited() { + return !hasPrecisionInfo; + } + + public boolean isFixed() { + return hasPrecisionInfo; + } + + /** Return the precision, or -1 if no precision is set */ + public int getPrecision() { + return precision; + } + + /** Return the scale, or -1 if no precision is set */ + public int getScale() { + return scale; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DecimalType that = (DecimalType) o; + + if (hasPrecisionInfo != that.hasPrecisionInfo) return false; + if (precision != that.precision) return false; + if (scale != that.scale) return false; + + return true; + } + + @Override + public int hashCode() { + int result = (hasPrecisionInfo ? 1 : 0); + result = 31 * result + precision; + result = 31 * result + scale; + return result; + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 8b96df10963b3..018a18c4ac214 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import java.util.{Map => JMap, List => JList} +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.storage.StorageLevel import scala.collection.JavaConversions._ @@ -113,7 +114,7 @@ class SchemaRDD( // ========================================================================================= override def compute(split: Partition, context: TaskContext): Iterator[Row] = - firstParent[Row].compute(split, context).map(_.copy()) + firstParent[Row].compute(split, context).map(ScalaReflection.convertRowToScala) override def getPartitions: Array[Partition] = firstParent[Row].partitions diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 082ae03eef03f..876b1c6edef20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -230,7 +230,7 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { case c: Class[_] if c == classOf[java.lang.Boolean] => (org.apache.spark.sql.BooleanType, true) case c: Class[_] if c == classOf[java.math.BigDecimal] => - (org.apache.spark.sql.DecimalType, true) + (org.apache.spark.sql.DecimalType(), true) case c: Class[_] if c == classOf[java.sql.Date] => (org.apache.spark.sql.DateType, true) case c: Class[_] if c == classOf[java.sql.Timestamp] => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala index df01411f60a05..401798e317e96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.api.java +import org.apache.spark.sql.catalyst.types.decimal.Decimal + import scala.annotation.varargs import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} import scala.collection.JavaConversions @@ -106,6 +108,8 @@ class Row(private[spark] val row: ScalaRow) extends Serializable { } override def hashCode(): Int = row.hashCode() + + override def toString: String = row.toString } object Row { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index b3edd5020fa8c..087b0ecbb25c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -70,16 +70,29 @@ case class GeneratedAggregate( val computeFunctions = aggregatesToCompute.map { case c @ Count(expr) => + // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its + // UnscaledValue will be null if and only if x is null; helps with Average on decimals + val toCount = expr match { + case UnscaledValue(e) => e + case _ => expr + } val currentCount = AttributeReference("currentCount", LongType, nullable = false)() val initialValue = Literal(0L) - val updateFunction = If(IsNotNull(expr), Add(currentCount, Literal(1L)), currentCount) + val updateFunction = If(IsNotNull(toCount), Add(currentCount, Literal(1L)), currentCount) val result = currentCount AggregateEvaluation(currentCount :: Nil, initialValue :: Nil, updateFunction :: Nil, result) case Sum(expr) => - val currentSum = AttributeReference("currentSum", expr.dataType, nullable = false)() - val initialValue = Cast(Literal(0L), expr.dataType) + val resultType = expr.dataType match { + case DecimalType.Fixed(precision, scale) => + DecimalType(precision + 10, scale) + case _ => + expr.dataType + } + + val currentSum = AttributeReference("currentSum", resultType, nullable = false)() + val initialValue = Cast(Literal(0L), resultType) // Coalasce avoids double calculation... // but really, common sub expression elimination would be better.... @@ -93,10 +106,26 @@ case class GeneratedAggregate( val currentSum = AttributeReference("currentSum", expr.dataType, nullable = false)() val initialCount = Literal(0L) val initialSum = Cast(Literal(0L), expr.dataType) - val updateCount = If(IsNotNull(expr), Add(currentCount, Literal(1L)), currentCount) + + // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its + // UnscaledValue will be null if and only if x is null; helps with Average on decimals + val toCount = expr match { + case UnscaledValue(e) => e + case _ => expr + } + + val updateCount = If(IsNotNull(toCount), Add(currentCount, Literal(1L)), currentCount) val updateSum = Coalesce(Add(expr, currentSum) :: currentSum :: Nil) - val result = Divide(Cast(currentSum, DoubleType), Cast(currentCount, DoubleType)) + val resultType = expr.dataType match { + case DecimalType.Fixed(precision, scale) => + DecimalType(precision + 4, scale + 4) + case DecimalType.Unlimited => + DecimalType.Unlimited + case _ => + DoubleType + } + val result = Divide(Cast(currentSum, resultType), Cast(currentCount, resultType)) AggregateEvaluation( currentCount :: currentSum :: Nil, @@ -142,7 +171,7 @@ case class GeneratedAggregate( val computationSchema = computeFunctions.flatMap(_.schema) - val resultMap: Map[TreeNodeRef, Expression] = + val resultMap: Map[TreeNodeRef, Expression] = aggregatesToCompute.zip(computeFunctions).map { case (agg, func) => new TreeNodeRef(agg) -> func.result }.toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index b1a7948b66cb6..aafcce0572b25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -23,7 +23,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext -import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.{ScalaReflection, trees} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ @@ -82,7 +82,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Runs this query returning the result as an array. */ - def executeCollect(): Array[Row] = execute().map(_.copy()).collect() + def executeCollect(): Array[Row] = execute().map(ScalaReflection.convertRowToScala).collect() protected def newProjection( expressions: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 077e6ebc5f11e..84d96e612f0dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -29,6 +29,7 @@ import com.twitter.chill.{AllScalaRegistrar, ResourcePool} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils @@ -51,6 +52,7 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[LongHashSet], new LongHashSetSerializer) kryo.register(classOf[org.apache.spark.util.collection.OpenHashSet[_]], new OpenHashSetSerializer) + kryo.register(classOf[Decimal]) kryo.setReferences(false) kryo.setClassLoader(Utils.getSparkClassLoader) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 977f3c9f32096..e6cd1a9d04278 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -143,7 +143,7 @@ case class Limit(limit: Int, child: SparkPlan) partsScanned += numPartsToTry } - buf.toArray + buf.toArray.map(ScalaReflection.convertRowToScala) } override def execute() = { @@ -176,10 +176,11 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) override def output = child.output override def outputPartitioning = SinglePartition - val ordering = new RowOrdering(sortOrder, child.output) + val ord = new RowOrdering(sortOrder, child.output) // TODO: Is this copying for no reason? - override def executeCollect() = child.execute().map(_.copy()).takeOrdered(limit)(ordering) + override def executeCollect() = + child.execute().map(_.copy()).takeOrdered(limit)(ord).map(ScalaReflection.convertRowToScala) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala index 8fd35880eedfe..5cf2a785adc7d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoin.scala @@ -49,7 +49,8 @@ case class BroadcastHashJoin( @transient private val broadcastFuture = future { - val input: Array[Row] = buildPlan.executeCollect() + // Note that we use .execute().collect() because we don't want to convert data to Scala types + val input: Array[Row] = buildPlan.execute().map(_.copy()).collect() val hashed = HashedRelation(input.iterator, buildSideKeyGenerator, input.length) sparkContext.broadcast(hashed) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index a1961bba1899e..997669051ed07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution import java.util.{List => JList, Map => JMap} +import org.apache.spark.sql.catalyst.types.decimal.Decimal + import scala.collection.JavaConversions._ import scala.collection.JavaConverters._ @@ -116,7 +118,7 @@ object EvaluatePython { def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { case (null, _) => null - case (row: Row, struct: StructType) => + case (row: Seq[Any], struct: StructType) => val fields = struct.fields.map(field => field.dataType) row.zip(fields).map { case (obj, dataType) => toJava(obj, dataType) @@ -133,6 +135,8 @@ object EvaluatePython { case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type }.asJava + case (dec: BigDecimal, dt: DecimalType) => dec.underlying() // Pyrolite can handle BigDecimal + // Pyrolite can handle Timestamp case (other, _) => other } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index eabe312f92371..5bb6f6c85d801 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.json +import org.apache.spark.sql.catalyst.types.decimal.Decimal + import scala.collection.Map import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} import scala.math.BigDecimal @@ -175,9 +177,9 @@ private[sql] object JsonRDD extends Logging { ScalaReflection.typeOfObject orElse { // Since we do not have a data type backed by BigInteger, // when we see a Java BigInteger, we use DecimalType. - case value: java.math.BigInteger => DecimalType + case value: java.math.BigInteger => DecimalType.Unlimited // DecimalType's JVMType is scala BigDecimal. - case value: java.math.BigDecimal => DecimalType + case value: java.math.BigDecimal => DecimalType.Unlimited // Unexpected data type. case _ => StringType } @@ -319,13 +321,13 @@ private[sql] object JsonRDD extends Logging { } } - private def toDecimal(value: Any): BigDecimal = { + private def toDecimal(value: Any): Decimal = { value match { - case value: java.lang.Integer => BigDecimal(value) - case value: java.lang.Long => BigDecimal(value) - case value: java.math.BigInteger => BigDecimal(value) - case value: java.lang.Double => BigDecimal(value) - case value: java.math.BigDecimal => BigDecimal(value) + case value: java.lang.Integer => Decimal(value) + case value: java.lang.Long => Decimal(value) + case value: java.math.BigInteger => Decimal(BigDecimal(value)) + case value: java.lang.Double => Decimal(value) + case value: java.math.BigDecimal => Decimal(BigDecimal(value)) } } @@ -391,7 +393,7 @@ private[sql] object JsonRDD extends Logging { case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) case DoubleType => toDouble(value) - case DecimalType => toDecimal(value) + case DecimalType() => toDecimal(value) case BooleanType => value.asInstanceOf[BooleanType.JvmType] case NullType => null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index f0e57e2a7447b..05926a24c5307 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -183,6 +183,20 @@ package object sql { * * The data type representing `scala.math.BigDecimal` values. * + * TODO(matei): explain precision and scale + * + * @group dataType + */ + @DeveloperApi + type DecimalType = catalyst.types.DecimalType + + /** + * :: DeveloperApi :: + * + * The data type representing `scala.math.BigDecimal` values. + * + * TODO(matei): explain precision and scale + * * @group dataType */ @DeveloperApi 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 2fc7e1cf23ab7..08feced61a899 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import org.apache.spark.sql.catalyst.types.decimal.Decimal + import scala.collection.mutable.{Buffer, ArrayBuffer, HashMap} import parquet.io.api.{PrimitiveConverter, GroupConverter, Binary, Converter} @@ -117,6 +119,12 @@ private[sql] object CatalystConverter { parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.JvmType]) } } + case d: DecimalType => { + new CatalystPrimitiveConverter(parent, fieldIndex) { + override def addBinary(value: Binary): Unit = + parent.updateDecimal(fieldIndex, value, d) + } + } // All other primitive types use the default converter case ctype: PrimitiveType => { // note: need the type tag here! new CatalystPrimitiveConverter(parent, fieldIndex) @@ -191,6 +199,10 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, value.toStringUsingUTF8) + protected[parquet] def updateDecimal(fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = { + updateField(fieldIndex, readDecimal(new Decimal(), value, ctype)) + } + protected[parquet] def isRootConverter: Boolean = parent == null protected[parquet] def clearBuffer(): Unit @@ -201,6 +213,27 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { * @return */ def getCurrentRecord: Row = throw new UnsupportedOperationException + + /** + * Read a decimal value from a Parquet Binary into "dest". Only supports decimals that fit in + * a long (i.e. precision <= 18) + */ + protected[parquet] def readDecimal(dest: Decimal, value: Binary, ctype: DecimalType): Unit = { + val precision = ctype.precisionInfo.get.precision + val scale = ctype.precisionInfo.get.scale + val bytes = value.getBytes + require(bytes.length <= 16, "Decimal field too large to read") + var unscaled = 0L + var i = 0 + while (i < bytes.length) { + unscaled = (unscaled << 8) | (bytes(i) & 0xFF) + i += 1 + } + // Make sure unscaled has the right sign, by sign-extending the first bit + val numBits = 8 * bytes.length + unscaled = (unscaled << (64 - numBits)) >> (64 - numBits) + dest.set(unscaled, precision, scale) + } } /** @@ -352,6 +385,16 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateString(fieldIndex: Int, value: Binary): Unit = current.setString(fieldIndex, value.toStringUsingUTF8) + + override protected[parquet] def updateDecimal( + fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = { + var decimal = current(fieldIndex).asInstanceOf[Decimal] + if (decimal == null) { + decimal = new Decimal + current(fieldIndex) = decimal + } + readDecimal(decimal, value, ctype) + } } /** 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 bdf02401b21be..2a5f23b24e8e8 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.parquet import java.util.{HashMap => JHashMap} import org.apache.hadoop.conf.Configuration +import org.apache.spark.sql.catalyst.types.decimal.Decimal import parquet.column.ParquetProperties import parquet.hadoop.ParquetOutputFormat import parquet.hadoop.api.ReadSupport.ReadContext @@ -204,6 +205,11 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { case DoubleType => writer.addDouble(value.asInstanceOf[Double]) case FloatType => writer.addFloat(value.asInstanceOf[Float]) case BooleanType => writer.addBoolean(value.asInstanceOf[Boolean]) + case d: DecimalType => + if (d.precisionInfo == None || d.precisionInfo.get.precision > 18) { + sys.error(s"Unsupported datatype $d, cannot write to consumer") + } + writeDecimal(value.asInstanceOf[Decimal], d.precisionInfo.get.precision) case _ => sys.error(s"Do not know how to writer $schema to consumer") } } @@ -283,6 +289,23 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } writer.endGroup() } + + // Scratch array used to write decimals as fixed-length binary + private val scratchBytes = new Array[Byte](8) + + private[parquet] def writeDecimal(decimal: Decimal, precision: Int): Unit = { + val numBytes = ParquetTypesConverter.BYTES_FOR_PRECISION(precision) + val unscaledLong = decimal.toUnscaledLong + var i = 0 + var shift = 8 * (numBytes - 1) + while (i < numBytes) { + scratchBytes(i) = (unscaledLong >> shift).toByte + i += 1 + shift -= 8 + } + writer.addBinary(Binary.fromByteArray(scratchBytes, 0, numBytes)) + } + } // Optimized for non-nested rows @@ -326,6 +349,11 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { case DoubleType => writer.addDouble(record.getDouble(index)) case FloatType => writer.addFloat(record.getFloat(index)) case BooleanType => writer.addBoolean(record.getBoolean(index)) + case d: DecimalType => + if (d.precisionInfo == None || d.precisionInfo.get.precision > 18) { + sys.error(s"Unsupported datatype $d, cannot write to consumer") + } + writeDecimal(record(index).asInstanceOf[Decimal], d.precisionInfo.get.precision) case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer") } } 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 e6389cf77a4c9..e5077de8dd908 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 @@ -29,8 +29,8 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter import parquet.hadoop.{ParquetFileReader, Footer, ParquetFileWriter} import parquet.hadoop.metadata.{ParquetMetadata, FileMetaData} import parquet.hadoop.util.ContextUtil -import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType} -import parquet.schema.{GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns} +import parquet.schema.{Type => ParquetType, Types => ParquetTypes, PrimitiveType => ParquetPrimitiveType, MessageType} +import parquet.schema.{GroupType => ParquetGroupType, OriginalType => ParquetOriginalType, ConversionPatterns, DecimalMetadata} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.Type.Repetition @@ -41,17 +41,25 @@ import org.apache.spark.sql.catalyst.types._ // Implicits import scala.collection.JavaConversions._ +/** A class representing Parquet info fields we care about, for passing back to Parquet */ +private[parquet] case class ParquetTypeInfo( + primitiveType: ParquetPrimitiveTypeName, + originalType: Option[ParquetOriginalType] = None, + decimalMetadata: Option[DecimalMetadata] = None, + length: Option[Int] = None) + private[parquet] object ParquetTypesConverter extends Logging { def isPrimitiveType(ctype: DataType): Boolean = classOf[PrimitiveType] isAssignableFrom ctype.getClass def toPrimitiveDataType( parquetType: ParquetPrimitiveType, - binayAsString: Boolean): DataType = + binaryAsString: Boolean): DataType = { + val originalType = parquetType.getOriginalType + val decimalInfo = parquetType.getDecimalMetadata parquetType.getPrimitiveTypeName match { case ParquetPrimitiveTypeName.BINARY - if (parquetType.getOriginalType == ParquetOriginalType.UTF8 || - binayAsString) => StringType + if (originalType == ParquetOriginalType.UTF8 || binaryAsString) => StringType case ParquetPrimitiveTypeName.BINARY => BinaryType case ParquetPrimitiveTypeName.BOOLEAN => BooleanType case ParquetPrimitiveTypeName.DOUBLE => DoubleType @@ -61,9 +69,14 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetPrimitiveTypeName.INT96 => // TODO: add BigInteger type? TODO(andre) use DecimalType instead???? sys.error("Potential loss of precision: cannot convert INT96") + case ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY + if (originalType == ParquetOriginalType.DECIMAL && decimalInfo.getPrecision <= 18) => + // TODO: for now, our reader only supports decimals that fit in a Long + DecimalType(decimalInfo.getPrecision, decimalInfo.getScale) case _ => sys.error( s"Unsupported parquet datatype $parquetType") } + } /** * Converts a given Parquet `Type` into the corresponding @@ -183,23 +196,40 @@ private[parquet] object ParquetTypesConverter extends Logging { * is not primitive. * * @param ctype The type to convert - * @return The name of the corresponding Parquet primitive type + * @return The name of the corresponding Parquet type properties */ - def fromPrimitiveDataType(ctype: DataType): - 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) + def fromPrimitiveDataType(ctype: DataType): Option[ParquetTypeInfo] = ctype match { + case StringType => Some(ParquetTypeInfo( + ParquetPrimitiveTypeName.BINARY, Some(ParquetOriginalType.UTF8))) + case BinaryType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.BINARY)) + case BooleanType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.BOOLEAN)) + case DoubleType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.DOUBLE)) + case FloatType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.FLOAT)) + case IntegerType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) // There is no type for Byte or Short so we promote them to INT32. - case ShortType => Some(ParquetPrimitiveTypeName.INT32, None) - case ByteType => Some(ParquetPrimitiveTypeName.INT32, None) - case LongType => Some(ParquetPrimitiveTypeName.INT64, None) + case ShortType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) + case ByteType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT32)) + case LongType => Some(ParquetTypeInfo(ParquetPrimitiveTypeName.INT64)) + case DecimalType.Fixed(precision, scale) if precision <= 18 => + // TODO: for now, our writer only supports decimals that fit in a Long + Some(ParquetTypeInfo(ParquetPrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, + Some(ParquetOriginalType.DECIMAL), + Some(new DecimalMetadata(precision, scale)), + Some(BYTES_FOR_PRECISION(precision)))) case _ => None } + /** + * Compute the FIXED_LEN_BYTE_ARRAY length needed to represent a given DECIMAL precision. + */ + private[parquet] val BYTES_FOR_PRECISION = Array.tabulate[Int](38) { precision => + var length = 1 + while (math.pow(2.0, 8 * length - 1) < math.pow(10.0, precision)) { + length += 1 + } + length + } + /** * Converts a given Catalyst [[org.apache.spark.sql.catalyst.types.DataType]] into * the corresponding Parquet `Type`. @@ -247,10 +277,17 @@ private[parquet] object ParquetTypesConverter extends Logging { } else { if (nullable) Repetition.OPTIONAL else Repetition.REQUIRED } - val primitiveType = fromPrimitiveDataType(ctype) - primitiveType.map { - case (primitiveType, originalType) => - new ParquetPrimitiveType(repetition, primitiveType, name, originalType.orNull) + val typeInfo = fromPrimitiveDataType(ctype) + typeInfo.map { + case ParquetTypeInfo(primitiveType, originalType, decimalMetadata, length) => + val builder = ParquetTypes.primitive(primitiveType, repetition).as(originalType.orNull) + for (len <- length) { + builder.length(len) + } + for (metadata <- decimalMetadata) { + builder.precision(metadata.getPrecision).scale(metadata.getScale) + } + builder.named(name) }.getOrElse { ctype match { case ArrayType(elementType, false) => { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 142598c904b37..7564bf3923032 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.types.util import org.apache.spark.sql._ import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, MetadataBuilder => JMetaDataBuilder} +import org.apache.spark.sql.api.java.{DecimalType => JDecimalType} +import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.collection.JavaConverters._ @@ -44,7 +46,8 @@ protected[sql] object DataTypeConversions { case BooleanType => JDataType.BooleanType case DateType => JDataType.DateType case TimestampType => JDataType.TimestampType - case DecimalType => JDataType.DecimalType + case DecimalType.Fixed(precision, scale) => new JDecimalType(precision, scale) + case DecimalType.Unlimited => new JDecimalType() case DoubleType => JDataType.DoubleType case FloatType => JDataType.FloatType case ByteType => JDataType.ByteType @@ -88,7 +91,11 @@ protected[sql] object DataTypeConversions { case timestampType: org.apache.spark.sql.api.java.TimestampType => TimestampType case decimalType: org.apache.spark.sql.api.java.DecimalType => - DecimalType + if (decimalType.isFixed) { + DecimalType(decimalType.getPrecision, decimalType.getScale) + } else { + DecimalType.Unlimited + } case doubleType: org.apache.spark.sql.api.java.DoubleType => DoubleType case floatType: org.apache.spark.sql.api.java.FloatType => @@ -115,7 +122,7 @@ protected[sql] object DataTypeConversions { /** Converts Java objects to catalyst rows / types */ def convertJavaToCatalyst(a: Any): Any = a match { - case d: java.math.BigDecimal => BigDecimal(d) + case d: java.math.BigDecimal => Decimal(BigDecimal(d)) case other => other } diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java index 9435a88009a5f..a04b8060cd658 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaApplySchemaSuite.java @@ -118,7 +118,7 @@ public void applySchemaToJSON() { "\"bigInteger\":92233720368547758069, \"double\":1.7976931348623157E305, " + "\"boolean\":false, \"null\":null}")); List fields = new ArrayList(7); - fields.add(DataType.createStructField("bigInteger", DataType.DecimalType, true)); + fields.add(DataType.createStructField("bigInteger", new DecimalType(), true)); fields.add(DataType.createStructField("boolean", DataType.BooleanType, true)); fields.add(DataType.createStructField("double", DataType.DoubleType, true)); fields.add(DataType.createStructField("integer", DataType.IntegerType, true)); diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java index d04396a5f8ec2..8396a29c61c4c 100644 --- a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaSideDataTypeConversionSuite.java @@ -41,7 +41,8 @@ public void createDataTypes() { checkDataType(DataType.BooleanType); checkDataType(DataType.DateType); checkDataType(DataType.TimestampType); - checkDataType(DataType.DecimalType); + checkDataType(new DecimalType()); + checkDataType(new DecimalType(10, 4)); checkDataType(DataType.DoubleType); checkDataType(DataType.FloatType); checkDataType(DataType.ByteType); @@ -59,7 +60,7 @@ public void createDataTypes() { // Simple StructType. List simpleFields = new ArrayList(); - simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); simpleFields.add(DataType.createStructField("d", DataType.BinaryType, false)); @@ -128,7 +129,7 @@ public void illegalArgument() { // StructType try { List simpleFields = new ArrayList(); - simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); simpleFields.add(DataType.createStructField("b", DataType.BooleanType, true)); simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); simpleFields.add(null); @@ -138,7 +139,7 @@ public void illegalArgument() { } try { List simpleFields = new ArrayList(); - simpleFields.add(DataType.createStructField("a", DataType.DecimalType, false)); + simpleFields.add(DataType.createStructField("a", new DecimalType(), false)); simpleFields.add(DataType.createStructField("a", DataType.BooleanType, true)); simpleFields.add(DataType.createStructField("c", DataType.LongType, true)); DataType.createStructType(simpleFields); diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala index 6c9db639c0f6c..e9740d913cf57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -69,7 +69,7 @@ class DataTypeSuite extends FunSuite { checkDataTypeJsonRepr(LongType) checkDataTypeJsonRepr(FloatType) checkDataTypeJsonRepr(DoubleType) - checkDataTypeJsonRepr(DecimalType) + checkDataTypeJsonRepr(DecimalType.Unlimited) checkDataTypeJsonRepr(TimestampType) checkDataTypeJsonRepr(StringType) checkDataTypeJsonRepr(BinaryType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index bfa9ea416266d..cf3a59e545905 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} +import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ @@ -81,7 +82,9 @@ class ScalaReflectionRelationSuite extends FunSuite { val rdd = sparkContext.parallelize(data :: Nil) rdd.registerTempTable("reflectData") - assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq) + assert(sql("SELECT * FROM reflectData").collect().head === + Seq("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, + BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3))) } test("query case class RDD with nulls") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index d83f3e23a9468..c9012c9e47cff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.api.java +import org.apache.spark.sql.catalyst.types.decimal.Decimal + import scala.beans.BeanProperty import org.scalatest.FunSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala index e0e0ff9cb3d3d..62fe59dd345d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/ScalaSideDataTypeConversionSuite.scala @@ -38,7 +38,7 @@ class ScalaSideDataTypeConversionSuite extends FunSuite { checkDataType(org.apache.spark.sql.BooleanType) checkDataType(org.apache.spark.sql.DateType) checkDataType(org.apache.spark.sql.TimestampType) - checkDataType(org.apache.spark.sql.DecimalType) + checkDataType(org.apache.spark.sql.DecimalType.Unlimited) checkDataType(org.apache.spark.sql.DoubleType) checkDataType(org.apache.spark.sql.FloatType) checkDataType(org.apache.spark.sql.ByteType) @@ -58,7 +58,7 @@ class ScalaSideDataTypeConversionSuite extends FunSuite { // Simple StructType. val simpleScalaStructType = SStructType( - SStructField("a", org.apache.spark.sql.DecimalType, false) :: + SStructField("a", org.apache.spark.sql.DecimalType.Unlimited, false) :: SStructField("b", org.apache.spark.sql.BooleanType, true) :: SStructField("c", org.apache.spark.sql.LongType, true) :: SStructField("d", org.apache.spark.sql.BinaryType, false) :: Nil) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index ce6184f5d8c9d..1cb6c23c58f36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.json import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} import org.apache.spark.sql.QueryTest @@ -44,19 +45,22 @@ class JsonSuite extends QueryTest { checkTypePromotion(intNumber, enforceCorrectType(intNumber, IntegerType)) checkTypePromotion(intNumber.toLong, enforceCorrectType(intNumber, LongType)) checkTypePromotion(intNumber.toDouble, enforceCorrectType(intNumber, DoubleType)) - checkTypePromotion(BigDecimal(intNumber), enforceCorrectType(intNumber, DecimalType)) + checkTypePromotion( + Decimal(intNumber), enforceCorrectType(intNumber, DecimalType.Unlimited)) val longNumber: Long = 9223372036854775807L checkTypePromotion(longNumber, enforceCorrectType(longNumber, LongType)) checkTypePromotion(longNumber.toDouble, enforceCorrectType(longNumber, DoubleType)) - checkTypePromotion(BigDecimal(longNumber), enforceCorrectType(longNumber, DecimalType)) + checkTypePromotion( + Decimal(longNumber), enforceCorrectType(longNumber, DecimalType.Unlimited)) val doubleNumber: Double = 1.7976931348623157E308d checkTypePromotion(doubleNumber.toDouble, enforceCorrectType(doubleNumber, DoubleType)) - checkTypePromotion(BigDecimal(doubleNumber), enforceCorrectType(doubleNumber, DecimalType)) - + checkTypePromotion( + Decimal(doubleNumber), enforceCorrectType(doubleNumber, DecimalType.Unlimited)) + checkTypePromotion(new Timestamp(intNumber), enforceCorrectType(intNumber, TimestampType)) - checkTypePromotion(new Timestamp(intNumber.toLong), + checkTypePromotion(new Timestamp(intNumber.toLong), enforceCorrectType(intNumber.toLong, TimestampType)) val strTime = "2014-09-30 12:34:56" checkTypePromotion(Timestamp.valueOf(strTime), enforceCorrectType(strTime, TimestampType)) @@ -80,7 +84,7 @@ class JsonSuite extends QueryTest { checkDataType(NullType, IntegerType, IntegerType) checkDataType(NullType, LongType, LongType) checkDataType(NullType, DoubleType, DoubleType) - checkDataType(NullType, DecimalType, DecimalType) + checkDataType(NullType, DecimalType.Unlimited, DecimalType.Unlimited) checkDataType(NullType, StringType, StringType) checkDataType(NullType, ArrayType(IntegerType), ArrayType(IntegerType)) checkDataType(NullType, StructType(Nil), StructType(Nil)) @@ -91,7 +95,7 @@ class JsonSuite extends QueryTest { checkDataType(BooleanType, IntegerType, StringType) checkDataType(BooleanType, LongType, StringType) checkDataType(BooleanType, DoubleType, StringType) - checkDataType(BooleanType, DecimalType, StringType) + checkDataType(BooleanType, DecimalType.Unlimited, StringType) checkDataType(BooleanType, StringType, StringType) checkDataType(BooleanType, ArrayType(IntegerType), StringType) checkDataType(BooleanType, StructType(Nil), StringType) @@ -100,7 +104,7 @@ class JsonSuite extends QueryTest { checkDataType(IntegerType, IntegerType, IntegerType) checkDataType(IntegerType, LongType, LongType) checkDataType(IntegerType, DoubleType, DoubleType) - checkDataType(IntegerType, DecimalType, DecimalType) + checkDataType(IntegerType, DecimalType.Unlimited, DecimalType.Unlimited) checkDataType(IntegerType, StringType, StringType) checkDataType(IntegerType, ArrayType(IntegerType), StringType) checkDataType(IntegerType, StructType(Nil), StringType) @@ -108,23 +112,23 @@ class JsonSuite extends QueryTest { // LongType checkDataType(LongType, LongType, LongType) checkDataType(LongType, DoubleType, DoubleType) - checkDataType(LongType, DecimalType, DecimalType) + checkDataType(LongType, DecimalType.Unlimited, DecimalType.Unlimited) checkDataType(LongType, StringType, StringType) checkDataType(LongType, ArrayType(IntegerType), StringType) checkDataType(LongType, StructType(Nil), StringType) // DoubleType checkDataType(DoubleType, DoubleType, DoubleType) - checkDataType(DoubleType, DecimalType, DecimalType) + checkDataType(DoubleType, DecimalType.Unlimited, DecimalType.Unlimited) checkDataType(DoubleType, StringType, StringType) checkDataType(DoubleType, ArrayType(IntegerType), StringType) checkDataType(DoubleType, StructType(Nil), StringType) // DoubleType - checkDataType(DecimalType, DecimalType, DecimalType) - checkDataType(DecimalType, StringType, StringType) - checkDataType(DecimalType, ArrayType(IntegerType), StringType) - checkDataType(DecimalType, StructType(Nil), StringType) + checkDataType(DecimalType.Unlimited, DecimalType.Unlimited, DecimalType.Unlimited) + checkDataType(DecimalType.Unlimited, StringType, StringType) + checkDataType(DecimalType.Unlimited, ArrayType(IntegerType), StringType) + checkDataType(DecimalType.Unlimited, StructType(Nil), StringType) // StringType checkDataType(StringType, StringType, StringType) @@ -178,7 +182,7 @@ class JsonSuite extends QueryTest { checkDataType( StructType( StructField("f1", IntegerType, true) :: Nil), - DecimalType, + DecimalType.Unlimited, StringType) } @@ -186,7 +190,7 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(primitiveFieldAndType) val expectedSchema = StructType( - StructField("bigInteger", DecimalType, true) :: + StructField("bigInteger", DecimalType.Unlimited, true) :: StructField("boolean", BooleanType, true) :: StructField("double", DoubleType, true) :: StructField("integer", IntegerType, true) :: @@ -216,7 +220,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, false), false), true) :: - StructField("arrayOfBigInteger", ArrayType(DecimalType, false), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType.Unlimited, false), true) :: StructField("arrayOfBoolean", ArrayType(BooleanType, false), true) :: StructField("arrayOfDouble", ArrayType(DoubleType, false), true) :: StructField("arrayOfInteger", ArrayType(IntegerType, false), true) :: @@ -230,7 +234,7 @@ class JsonSuite extends QueryTest { StructField("field3", StringType, true) :: Nil), false), true) :: StructField("struct", StructType( StructField("field1", BooleanType, true) :: - StructField("field2", DecimalType, true) :: Nil), true) :: + StructField("field2", DecimalType.Unlimited, true) :: Nil), true) :: StructField("structWithArrayFields", StructType( StructField("field1", ArrayType(IntegerType, false), true) :: StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) @@ -331,7 +335,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("num_bool", StringType, true) :: StructField("num_num_1", LongType, true) :: - StructField("num_num_2", DecimalType, true) :: + StructField("num_num_2", DecimalType.Unlimited, true) :: StructField("num_num_3", DoubleType, true) :: StructField("num_str", StringType, true) :: StructField("str_bool", StringType, true) :: Nil) @@ -521,7 +525,7 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonFile(path) val expectedSchema = StructType( - StructField("bigInteger", DecimalType, true) :: + StructField("bigInteger", DecimalType.Unlimited, true) :: StructField("boolean", BooleanType, true) :: StructField("double", DoubleType, true) :: StructField("integer", IntegerType, true) :: @@ -551,7 +555,7 @@ class JsonSuite extends QueryTest { primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) val schema = StructType( - StructField("bigInteger", DecimalType, true) :: + StructField("bigInteger", DecimalType.Unlimited, true) :: StructField("boolean", BooleanType, true) :: StructField("double", DoubleType, true) :: StructField("integer", IntegerType, true) :: 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 9979ab446d8b5..08d9da27f1b11 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 @@ -77,6 +77,8 @@ case class AllDataTypesWithNonPrimitiveType( case class BinaryData(binaryData: Array[Byte]) +case class NumericData(i: Int, d: Double) + class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { TestData // Load test data tables. @@ -560,7 +562,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(stringResult.size === 1) assert(stringResult(0).getString(2) == "100", "stringvalue incorrect") assert(stringResult(0).getInt(1) === 100) - + val query7 = sql(s"SELECT * FROM testfiltersource WHERE myoptint < 40") assert( query7.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], @@ -869,4 +871,35 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(a.dataType === b.dataType) } } + + test("read/write fixed-length decimals") { + for ((precision, scale) <- Seq((5, 2), (1, 0), (1, 1), (18, 10), (18, 17))) { + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + val data = sparkContext.parallelize(0 to 1000) + .map(i => NumericData(i, i / 100.0)) + .select('i, 'd cast DecimalType(precision, scale)) + data.saveAsParquetFile(tempDir) + checkAnswer(parquetFile(tempDir), data.toSchemaRDD.collect().toSeq) + } + + // Decimals with precision above 18 are not yet supported + intercept[RuntimeException] { + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + val data = sparkContext.parallelize(0 to 1000) + .map(i => NumericData(i, i / 100.0)) + .select('i, 'd cast DecimalType(19, 10)) + data.saveAsParquetFile(tempDir) + checkAnswer(parquetFile(tempDir), data.toSchemaRDD.collect().toSeq) + } + + // Unlimited-length decimals are not yet supported + intercept[RuntimeException] { + val tempDir = getTempFilePath("parquetTest").getCanonicalPath + val data = sparkContext.parallelize(0 to 1000) + .map(i => NumericData(i, i / 100.0)) + .select('i, 'd cast DecimalType.Unlimited) + data.saveAsParquetFile(tempDir) + checkAnswer(parquetFile(tempDir), data.toSchemaRDD.collect().toSeq) + } + } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index 2a4f24132cc5e..99c4f46a82b8e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -47,7 +47,7 @@ private[thriftserver] class SparkSQLOperationManager(hiveContext: HiveContext) val operation = new SparkExecuteStatementOperation(parentSession, statement, confOverlay)( hiveContext, sessionToActivePool) - handleToOperation.put(operation.getHandle, operation) - operation + handleToOperation.put(operation.getHandle, operation) + operation } } diff --git a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala index bbd727c686bbc..8077d0ec46fd7 100644 --- a/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala +++ b/sql/hive-thriftserver/v0.12.0/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim12.scala @@ -123,7 +123,7 @@ private[hive] class SparkExecuteStatementOperation( to.addColumnValue(ColumnValue.doubleValue(from.getDouble(ordinal))) case FloatType => to.addColumnValue(ColumnValue.floatValue(from.getFloat(ordinal))) - case DecimalType => + case DecimalType() => val hiveDecimal = from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal to.addColumnValue(ColumnValue.stringValue(new HiveDecimal(hiveDecimal))) case LongType => @@ -156,7 +156,7 @@ private[hive] class SparkExecuteStatementOperation( to.addColumnValue(ColumnValue.doubleValue(null)) case FloatType => to.addColumnValue(ColumnValue.floatValue(null)) - case DecimalType => + case DecimalType() => to.addColumnValue(ColumnValue.stringValue(null: HiveDecimal)) case LongType => to.addColumnValue(ColumnValue.longValue(null)) diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index e59681bfbe43e..2c1983de1d0d5 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -123,7 +123,7 @@ private[hive] class SparkExecuteStatementOperation( to += from.getDouble(ordinal) case FloatType => to += from.getFloat(ordinal) - case DecimalType => + case DecimalType() => to += from.get(ordinal).asInstanceOf[BigDecimal].bigDecimal case LongType => to += from.getLong(ordinal) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index ff8fa44194d98..2e27817d60221 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -21,6 +21,10 @@ import java.io.{BufferedReader, File, InputStreamReader, PrintStream} import java.sql.{Date, Timestamp} import java.util.{ArrayList => JArrayList} +import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.spark.sql.catalyst.types.DecimalType +import org.apache.spark.sql.catalyst.types.decimal.Decimal + import scala.collection.JavaConversions._ import scala.language.implicitConversions import scala.reflect.runtime.universe.{TypeTag, typeTag} @@ -370,7 +374,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DecimalType, DateType, TimestampType, BinaryType) + ShortType, DateType, TimestampType, BinaryType) protected[sql] def toHiveString(a: (Any, DataType)): String = a match { case (struct: Row, StructType(fields)) => @@ -388,6 +392,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { case (d: Date, DateType) => new DateWritable(d).toString case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") + case (decimal: Decimal, DecimalType()) => // Hive strips trailing zeros so use its toString + HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString case (other, tpe) if primitiveTypes contains tpe => other.toString } @@ -406,6 +412,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { }.toSeq.sorted.mkString("{", ",", "}") case (null, _) => "null" case (s: String, StringType) => "\"" + s + "\"" + case (decimal, DecimalType()) => decimal.toString case (other, tpe) if primitiveTypes contains tpe => other.toString } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 0439ab97d80be..1e2bf5cc4b0b4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -38,7 +39,7 @@ private[hive] trait HiveInspectors { // writable case c: Class[_] if c == classOf[hadoopIo.DoubleWritable] => DoubleType case c: Class[_] if c == classOf[hiveIo.DoubleWritable] => DoubleType - case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType + case c: Class[_] if c == classOf[hiveIo.HiveDecimalWritable] => DecimalType.Unlimited case c: Class[_] if c == classOf[hiveIo.ByteWritable] => ByteType case c: Class[_] if c == classOf[hiveIo.ShortWritable] => ShortType case c: Class[_] if c == classOf[hiveIo.DateWritable] => DateType @@ -54,8 +55,8 @@ private[hive] trait HiveInspectors { case c: Class[_] if c == classOf[java.lang.String] => StringType case c: Class[_] if c == classOf[java.sql.Date] => DateType case c: Class[_] if c == classOf[java.sql.Timestamp] => TimestampType - case c: Class[_] if c == classOf[HiveDecimal] => DecimalType - case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType + case c: Class[_] if c == classOf[HiveDecimal] => DecimalType.Unlimited + case c: Class[_] if c == classOf[java.math.BigDecimal] => DecimalType.Unlimited case c: Class[_] if c == classOf[Array[Byte]] => BinaryType case c: Class[_] if c == classOf[java.lang.Short] => ShortType case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType @@ -90,7 +91,7 @@ private[hive] trait HiveInspectors { case hvoi: HiveVarcharObjectInspector => if (data == null) null else hvoi.getPrimitiveJavaObject(data).getValue case hdoi: HiveDecimalObjectInspector => - if (data == null) null else BigDecimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + if (data == null) null else HiveShim.toCatalystDecimal(hdoi, data) // org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object // if next timestamp is null, so Timestamp object is cloned case ti: TimestampObjectInspector => ti.getPrimitiveJavaObject(data).clone() @@ -137,8 +138,9 @@ private[hive] trait HiveInspectors { case l: Short => l: java.lang.Short case l: Byte => l: java.lang.Byte case b: BigDecimal => HiveShim.createDecimal(b.underlying()) + case d: Decimal => HiveShim.createDecimal(d.toBigDecimal.underlying()) case b: Array[Byte] => b - case d: java.sql.Date => d + case d: java.sql.Date => d case t: java.sql.Timestamp => t } case x: StructObjectInspector => @@ -200,7 +202,7 @@ private[hive] trait HiveInspectors { case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector case DateType => PrimitiveObjectInspectorFactory.javaDateObjectInspector case TimestampType => PrimitiveObjectInspectorFactory.javaTimestampObjectInspector - case DecimalType => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector + case DecimalType() => PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector case StructType(fields) => ObjectInspectorFactory.getStandardStructObjectInspector( fields.map(f => f.name), fields.map(f => toInspector(f.dataType))) @@ -229,8 +231,10 @@ private[hive] trait HiveInspectors { HiveShim.getPrimitiveWritableConstantObjectInspector(value) case Literal(value: java.sql.Timestamp, TimestampType) => HiveShim.getPrimitiveWritableConstantObjectInspector(value) - case Literal(value: BigDecimal, DecimalType) => + case Literal(value: BigDecimal, DecimalType()) => HiveShim.getPrimitiveWritableConstantObjectInspector(value) + case Literal(value: Decimal, DecimalType()) => + HiveShim.getPrimitiveWritableConstantObjectInspector(value.toBigDecimal) case Literal(_, NullType) => HiveShim.getPrimitiveNullWritableConstantObjectInspector case Literal(value: Seq[_], ArrayType(dt, _)) => @@ -277,8 +281,8 @@ private[hive] trait HiveInspectors { case _: JavaFloatObjectInspector => FloatType case _: WritableBinaryObjectInspector => BinaryType case _: JavaBinaryObjectInspector => BinaryType - case _: WritableHiveDecimalObjectInspector => DecimalType - case _: JavaHiveDecimalObjectInspector => DecimalType + case w: WritableHiveDecimalObjectInspector => HiveShim.decimalTypeInfoToCatalyst(w) + case j: JavaHiveDecimalObjectInspector => HiveShim.decimalTypeInfoToCatalyst(j) case _: WritableDateObjectInspector => DateType case _: JavaDateObjectInspector => DateType case _: WritableTimestampObjectInspector => TimestampType @@ -307,7 +311,7 @@ private[hive] trait HiveInspectors { case LongType => longTypeInfo case ShortType => shortTypeInfo case StringType => stringTypeInfo - case DecimalType => decimalTypeInfo + case d: DecimalType => HiveShim.decimalTypeInfo(d) case DateType => dateTypeInfo case TimestampType => timestampTypeInfo case NullType => voidTypeInfo 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 2dd2c882a8420..096b4a07aa2ea 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import java.io.IOException import java.util.{List => JList} +import scala.util.matching.Regex import scala.util.parsing.combinator.RegexParsers import org.apache.hadoop.util.ReflectionUtils @@ -321,11 +322,18 @@ object HiveMetastoreTypes extends RegexParsers { "bigint" ^^^ LongType | "binary" ^^^ BinaryType | "boolean" ^^^ BooleanType | - HiveShim.metastoreDecimal ^^^ DecimalType | + fixedDecimalType | // Hive 0.13+ decimal with precision/scale + "decimal" ^^^ DecimalType.Unlimited | // Hive 0.12 decimal with no precision/scale "date" ^^^ DateType | "timestamp" ^^^ TimestampType | "varchar\\((\\d+)\\)".r ^^^ StringType + protected lazy val fixedDecimalType: Parser[DataType] = + ("decimal" ~> "(" ~> "\\d+".r) ~ ("," ~> "\\d+".r <~ ")") ^^ { + case precision ~ scale => + DecimalType(precision.toInt, scale.toInt) + } + protected lazy val arrayType: Parser[DataType] = "array" ~> "<" ~> dataType <~ ">" ^^ { case tpe => ArrayType(tpe) @@ -373,7 +381,7 @@ object HiveMetastoreTypes extends RegexParsers { case BinaryType => "binary" case BooleanType => "boolean" case DateType => "date" - case DecimalType => "decimal" + case d: DecimalType => HiveShim.decimalMetastoreString(d) case TimestampType => "timestamp" case NullType => "void" } @@ -441,7 +449,7 @@ private[hive] case class MetastoreRelation val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) /** Non-partitionKey attributes */ - val attributes = hiveQlTable.getCols.map(_.toAttribute) + val attributes = hiveQlTable.getCols.map(_.toAttribute) val output = attributes ++ partitionKeys 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 a3573e6502de8..74f68d0f95317 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 @@ -31,6 +31,7 @@ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -325,7 +326,11 @@ private[hive] object HiveQl { } protected def nodeToDataType(node: Node): DataType = node match { - case Token("TOK_DECIMAL", Nil) => DecimalType + case Token("TOK_DECIMAL", precision :: scale :: Nil) => + DecimalType(precision.getText.toInt, scale.getText.toInt) + case Token("TOK_DECIMAL", precision :: Nil) => + DecimalType(precision.getText.toInt, 0) + case Token("TOK_DECIMAL", Nil) => DecimalType.Unlimited case Token("TOK_BIGINT", Nil) => LongType case Token("TOK_INT", Nil) => IntegerType case Token("TOK_TINYINT", Nil) => ByteType @@ -942,8 +947,12 @@ private[hive] object HiveQl { Cast(nodeToExpr(arg), BinaryType) case Token("TOK_FUNCTION", Token("TOK_BOOLEAN", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), BooleanType) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: scale :: nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType(precision.getText.toInt, scale.getText.toInt)) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", precision :: Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType(precision.getText.toInt, 0)) case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => - Cast(nodeToExpr(arg), DecimalType) + Cast(nodeToExpr(arg), DecimalType.Unlimited) case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), TimestampType) case Token("TOK_FUNCTION", Token("TOK_DATE", Nil) :: arg :: Nil) => @@ -1063,7 +1072,7 @@ private[hive] object HiveQl { } else if (ast.getText.endsWith("BD") || ast.getText.endsWith("D")) { // Literal decimal val strVal = ast.getText.stripSuffix("D").stripSuffix("B") - v = Literal(BigDecimal(strVal)) + v = Literal(Decimal(strVal)) } else { v = Literal(ast.getText.toDouble, DoubleType) v = Literal(ast.getText.toLong, LongType) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 79234f8a66f05..92bc1c6625892 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.execution.{Command, SparkPlan, UnaryNode} import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.{ ShimFileSinkDesc => FileSinkDesc} @@ -76,7 +77,7 @@ case class InsertIntoHiveTable( (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) case _: JavaHiveDecimalObjectInspector => - (o: Any) => HiveShim.createDecimal(o.asInstanceOf[BigDecimal].underlying()) + (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toBigDecimal.underlying()) case soi: StandardStructObjectInspector => val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index afc252ac27987..8e946b7e82f5d 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -30,21 +30,24 @@ import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.stats.StatsSetupConst import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} +import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} import org.apache.hadoop.mapred.InputFormat +import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.collection.JavaConversions._ import scala.language.implicitConversions +import org.apache.spark.sql.catalyst.types.DecimalType + /** * A compatibility layer for interacting with Hive version 0.12.0. */ private[hive] object HiveShim { val version = "0.12.0" - val metastoreDecimal = "decimal" def getTableDesc( serdeClass: Class[_ <: Deserializer], @@ -149,6 +152,19 @@ private[hive] object HiveShim { def setLocation(tbl: Table, crtTbl: CreateTableDesc): Unit = { tbl.setDataLocation(new Path(crtTbl.getLocation()).toUri()) } + + def decimalMetastoreString(decimalType: DecimalType): String = "decimal" + + def decimalTypeInfo(decimalType: DecimalType): TypeInfo = + TypeInfoFactory.decimalTypeInfo + + def decimalTypeInfoToCatalyst(inspector: PrimitiveObjectInspector): DecimalType = { + DecimalType.Unlimited + } + + def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = { + Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue()) + } } class ShimFileSinkDesc(var dir: String, var tableInfo: TableDesc, var compressed: Boolean) diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 42cd65b2518c9..0bc330cdbecb1 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -29,15 +29,15 @@ import org.apache.hadoop.hive.ql.Context import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory -import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory -import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer} -import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory -import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory -import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector +import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory} +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.{PrimitiveObjectInspector, ObjectInspector} import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.types.DecimalType +import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.collection.JavaConversions._ import scala.language.implicitConversions @@ -47,11 +47,6 @@ import scala.language.implicitConversions */ private[hive] object HiveShim { val version = "0.13.1" - /* - * TODO: hive-0.13 support DECIMAL(precision, scale), DECIMAL in hive-0.12 is actually DECIMAL(38,unbounded) - * Full support of new decimal feature need to be fixed in seperate PR. - */ - val metastoreDecimal = "decimal\\((\\d+),(\\d+)\\)".r def getTableDesc( serdeClass: Class[_ <: Deserializer], @@ -197,6 +192,30 @@ private[hive] object HiveShim { f.setDestTableId(w.destTableId) f } + + // Precision and scale to pass for unlimited decimals; these are the same as the precision and + // scale Hive 0.13 infers for BigDecimals from sources that don't specify them (e.g. UDFs) + private val UNLIMITED_DECIMAL_PRECISION = 38 + private val UNLIMITED_DECIMAL_SCALE = 18 + + def decimalMetastoreString(decimalType: DecimalType): String = decimalType match { + case DecimalType.Fixed(precision, scale) => s"decimal($precision,$scale)" + case _ => s"decimal($UNLIMITED_DECIMAL_PRECISION,$UNLIMITED_DECIMAL_SCALE)" + } + + def decimalTypeInfo(decimalType: DecimalType): TypeInfo = decimalType match { + case DecimalType.Fixed(precision, scale) => new DecimalTypeInfo(precision, scale) + case _ => new DecimalTypeInfo(UNLIMITED_DECIMAL_PRECISION, UNLIMITED_DECIMAL_SCALE) + } + + def decimalTypeInfoToCatalyst(inspector: PrimitiveObjectInspector): DecimalType = { + val info = inspector.getTypeInfo.asInstanceOf[DecimalTypeInfo] + DecimalType(info.precision(), info.scale()) + } + + def toCatalystDecimal(hdoi: HiveDecimalObjectInspector, data: Any): Decimal = { + Decimal(hdoi.getPrimitiveJavaObject(data).bigDecimalValue(), hdoi.precision(), hdoi.scale()) + } } /* From 6181577e9935f46b646ba3925b873d031aa3d6ba Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sun, 2 Nov 2014 00:03:51 -0700 Subject: [PATCH 1184/1492] [SPARK-3466] Limit size of results that a driver collects for each action Right now, operations like collect() and take() can crash the driver with an OOM if they bring back too many data. This PR will introduce spark.driver.maxResultSize, after setting it, the driver will abort a job if its result is bigger than it. By default, it's 1g (for backward compatibility for most the cases). In local mode, the driver and executor share the same JVM, the default setting can not protect JVM from OOM. cc mateiz Author: Davies Liu Closes #3003 from davies/collect and squashes the following commits: 248ed5e [Davies Liu] fix compile 272522e [Davies Liu] address comments 2c35773 [Davies Liu] add sizes in message of abort() 5d62303 [Davies Liu] address comments bc3c077 [Davies Liu] Merge branch 'master' of github.com:apache/spark into collect 11f97c5 [Davies Liu] address comments 47b144f [Davies Liu] check the size of result before send and fetch 3d81af2 [Davies Liu] address comments ca8267d [Davies Liu] limit the size of data by collect --- .../org/apache/spark/executor/Executor.scala | 25 ++++++++------ .../apache/spark/scheduler/TaskResult.scala | 4 +-- .../spark/scheduler/TaskResultGetter.scala | 20 ++++++++--- .../spark/scheduler/TaskSetManager.scala | 33 ++++++++++++++++--- .../scala/org/apache/spark/util/Utils.scala | 5 +++ .../scheduler/TaskResultGetterSuite.scala | 2 +- .../spark/scheduler/TaskSetManagerSuite.scala | 25 ++++++++++++++ docs/configuration.md | 12 +++++++ 8 files changed, 104 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c78e0ffca25bb..e24a15f015e1c 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -104,6 +104,9 @@ private[spark] class Executor( // to send the result back. private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) + // Limit of bytes for total size of results (default is 1GB) + private val maxResultSize = Utils.getMaxResultSize(conf) + // Start worker thread pool val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") @@ -210,25 +213,27 @@ private[spark] class Executor( val resultSize = serializedDirectResult.limit // directSend = sending directly back to the driver - val (serializedResult, directSend) = { - if (resultSize >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { + val serializedResult = { + if (resultSize > maxResultSize) { + logWarning(s"Finished $taskName (TID $taskId). Result is larger than maxResultSize " + + s"(${Utils.bytesToString(resultSize)} > ${Utils.bytesToString(maxResultSize)}), " + + s"dropping it.") + ser.serialize(new IndirectTaskResult[Any](TaskResultBlockId(taskId), resultSize)) + } else if (resultSize >= akkaFrameSize - AkkaUtils.reservedSizeBytes) { val blockId = TaskResultBlockId(taskId) env.blockManager.putBytes( blockId, serializedDirectResult, StorageLevel.MEMORY_AND_DISK_SER) - (ser.serialize(new IndirectTaskResult[Any](blockId)), false) + logInfo( + s"Finished $taskName (TID $taskId). $resultSize bytes result sent via BlockManager)") + ser.serialize(new IndirectTaskResult[Any](blockId, resultSize)) } else { - (serializedDirectResult, true) + logInfo(s"Finished $taskName (TID $taskId). $resultSize bytes result sent to driver") + serializedDirectResult } } execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) - if (directSend) { - logInfo(s"Finished $taskName (TID $taskId). $resultSize bytes result sent to driver") - } else { - logInfo( - s"Finished $taskName (TID $taskId). $resultSize bytes result sent via BlockManager)") - } } catch { case ffe: FetchFailedException => { val reason = ffe.toTaskEndReason diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala index 11c19eeb6e42c..1f114a0207f7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResult.scala @@ -31,8 +31,8 @@ import org.apache.spark.util.Utils private[spark] sealed trait TaskResult[T] /** A reference to a DirectTaskResult that has been stored in the worker's BlockManager. */ -private[spark] -case class IndirectTaskResult[T](blockId: BlockId) extends TaskResult[T] with Serializable +private[spark] case class IndirectTaskResult[T](blockId: BlockId, size: Int) + extends TaskResult[T] with Serializable /** A TaskResult that contains the task's return value and accumulator updates. */ private[spark] diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index 4b5be68ec5f92..819b51e12ad8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -47,9 +47,18 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul getTaskResultExecutor.execute(new Runnable { override def run(): Unit = Utils.logUncaughtExceptions { try { - val result = serializer.get().deserialize[TaskResult[_]](serializedData) match { - case directResult: DirectTaskResult[_] => directResult - case IndirectTaskResult(blockId) => + val (result, size) = serializer.get().deserialize[TaskResult[_]](serializedData) match { + case directResult: DirectTaskResult[_] => + if (!taskSetManager.canFetchMoreResults(serializedData.limit())) { + return + } + (directResult, serializedData.limit()) + case IndirectTaskResult(blockId, size) => + if (!taskSetManager.canFetchMoreResults(size)) { + // dropped by executor if size is larger than maxResultSize + sparkEnv.blockManager.master.removeBlock(blockId) + return + } logDebug("Fetching indirect task result for TID %s".format(tid)) scheduler.handleTaskGettingResult(taskSetManager, tid) val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId) @@ -64,9 +73,10 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul val deserializedResult = serializer.get().deserialize[DirectTaskResult[_]]( serializedTaskResult.get) sparkEnv.blockManager.master.removeBlock(blockId) - deserializedResult + (deserializedResult, size) } - result.metrics.resultSize = serializedData.limit() + + result.metrics.resultSize = size scheduler.handleSuccessfulTask(taskSetManager, tid, result) } catch { case cnf: ClassNotFoundException => diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 376821f89c6b8..a9767340074a8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -23,13 +23,12 @@ import java.util.Arrays import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet -import scala.math.max -import scala.math.min +import scala.math.{min, max} import org.apache.spark._ -import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.{Clock, SystemClock} +import org.apache.spark.TaskState.TaskState +import org.apache.spark.util.{Clock, SystemClock, Utils} /** * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of @@ -68,6 +67,9 @@ private[spark] class TaskSetManager( val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5) + // Limit of bytes for total size of results (default is 1GB) + val maxResultSize = Utils.getMaxResultSize(conf) + // Serializer for closures and tasks. val env = SparkEnv.get val ser = env.closureSerializer.newInstance() @@ -89,6 +91,8 @@ private[spark] class TaskSetManager( var stageId = taskSet.stageId var name = "TaskSet_" + taskSet.stageId.toString var parent: Pool = null + var totalResultSize = 0L + var calculatedTasks = 0 val runningTasksSet = new HashSet[Long] override def runningTasks = runningTasksSet.size @@ -515,12 +519,33 @@ private[spark] class TaskSetManager( index } + /** + * Marks the task as getting result and notifies the DAG Scheduler + */ def handleTaskGettingResult(tid: Long) = { val info = taskInfos(tid) info.markGettingResult() sched.dagScheduler.taskGettingResult(info) } + /** + * Check whether has enough quota to fetch the result with `size` bytes + */ + def canFetchMoreResults(size: Long): Boolean = synchronized { + totalResultSize += size + calculatedTasks += 1 + if (maxResultSize > 0 && totalResultSize > maxResultSize) { + val msg = s"Total size of serialized results of ${calculatedTasks} tasks " + + s"(${Utils.bytesToString(totalResultSize)}) is bigger than maxResultSize " + + s"(${Utils.bytesToString(maxResultSize)})" + logError(msg) + abort(msg) + false + } else { + true + } + } + /** * Marks the task as successful and notifies the DAGScheduler that a task has ended. */ 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 68d378f3a212d..4e30d0d3813a2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1720,6 +1720,11 @@ private[spark] object Utils extends Logging { method.invoke(obj, values.toSeq: _*) } + // Limit of bytes for total size of results (default is 1GB) + def getMaxResultSize(conf: SparkConf): Long = { + memoryStringToMb(conf.get("spark.driver.maxResultSize", "1g")).toLong << 20 + } + /** * Return the current system LD_LIBRARY_PATH name */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index c4e7a4bb7d385..5768a3a733f00 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -40,7 +40,7 @@ class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedule // Only remove the result once, since we'd like to test the case where the task eventually // succeeds. serializer.get().deserialize[TaskResult[_]](serializedData) match { - case IndirectTaskResult(blockId) => + case IndirectTaskResult(blockId, size) => sparkEnv.blockManager.master.removeBlock(blockId) case directResult: DirectTaskResult[_] => taskSetManager.abort("Internal error: expect only indirect results") diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index c0b07649eb6dd..1809b5396d53e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -563,6 +563,31 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.emittedTaskSizeWarning) } + test("abort the job if total size of results is too large") { + val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") + sc = new SparkContext("local", "test", conf) + + def genBytes(size: Int) = { (x: Int) => + val bytes = Array.ofDim[Byte](size) + scala.util.Random.nextBytes(bytes) + bytes + } + + // multiple 1k result + val r = sc.makeRDD(0 until 10, 10).map(genBytes(1024)).collect() + assert(10 === r.size ) + + // single 10M result + val thrown = intercept[SparkException] {sc.makeRDD(genBytes(10 << 20)(0), 1).collect()} + assert(thrown.getMessage().contains("bigger than maxResultSize")) + + // multiple 1M results + val thrown2 = intercept[SparkException] { + sc.makeRDD(0 until 10, 10).map(genBytes(1 << 20)).collect() + } + assert(thrown2.getMessage().contains("bigger than maxResultSize")) + } + test("speculative and noPref task should be scheduled after node-local") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) diff --git a/docs/configuration.md b/docs/configuration.md index 3007706a2586e..099972ca1af70 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -111,6 +111,18 @@ of the most common options to set are: (e.g. 512m, 2g). + + spark.driver.maxResultSize + 1g + + Limit of total size of serialized results of all partitions for each Spark action (e.g. collect). + Should be at least 1M, or 0 for unlimited. Jobs will be aborted if the total size + is above this limit. + Having a high limit may cause out-of-memory errors in driver (depends on spark.driver.memory + and memory overhead of objects in JVM). Setting a proper limit can protect the driver from + out-of-memory errors. + + spark.serializer org.apache.spark.serializer.
    JavaSerializer From 4e6a7a0b3e55098374a22f3ae9500404f7e4e91a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 2 Nov 2014 10:44:52 -0800 Subject: [PATCH 1185/1492] [SPARK-4166][Core][WebUI] Display the executor ID in the Web UI when ExecutorLostFailure happens Now when ExecutorLostFailure happens, it only displays `ExecutorLostFailure (executor lost)`. Adding the executor id will help locate the faulted executor. Author: zsxwing Closes #3033 from zsxwing/SPARK-4166 and squashes the following commits: ff4664c [zsxwing] Backward-compatible support c5c4cf2 [zsxwing] Display the executor ID in the Web UI when ExecutorLostFailure happens --- core/src/main/scala/org/apache/spark/TaskEndReason.scala | 4 ++-- .../scala/org/apache/spark/scheduler/TaskSetManager.scala | 2 +- .../main/scala/org/apache/spark/util/JsonProtocol.scala | 8 ++++++-- .../apache/spark/ui/jobs/JobProgressListenerSuite.scala | 2 +- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 5 +++-- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 8f0c5e78416c2..202fba699ab26 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -117,8 +117,8 @@ case object TaskKilled extends TaskFailedReason { * the task crashed the JVM. */ @DeveloperApi -case object ExecutorLostFailure extends TaskFailedReason { - override def toErrorString: String = "ExecutorLostFailure (executor lost)" +case class ExecutorLostFailure(execId: String) extends TaskFailedReason { + override def toErrorString: String = s"ExecutorLostFailure (executor ${execId} lost)" } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a9767340074a8..d8fb640350343 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -732,7 +732,7 @@ private[spark] class TaskSetManager( } // Also re-enqueue any tasks that were running on the node for ((tid, info) <- taskInfos if info.running && info.executorId == execId) { - handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure) + handleFailedTask(tid, TaskState.FAILED, ExecutorLostFailure(execId)) } // recalculate valid locality levels and waits when executor is lost recomputeLocality() diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 5b2e7d3a7edb9..43c7fba06694a 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -272,7 +272,7 @@ private[spark] object JsonProtocol { def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { val reason = Utils.getFormattedClassName(taskEndReason) - val json = taskEndReason match { + val json: JObject = taskEndReason match { case fetchFailed: FetchFailed => val blockManagerAddress = Option(fetchFailed.bmAddress). map(blockManagerIdToJson).getOrElse(JNothing) @@ -287,6 +287,8 @@ private[spark] object JsonProtocol { ("Description" -> exceptionFailure.description) ~ ("Stack Trace" -> stackTrace) ~ ("Metrics" -> metrics) + case ExecutorLostFailure(executorId) => + ("Executor ID" -> executorId) case _ => Utils.emptyJson } ("Reason" -> reason) ~ json @@ -636,7 +638,9 @@ private[spark] object JsonProtocol { new ExceptionFailure(className, description, stackTrace, metrics) case `taskResultLost` => TaskResultLost case `taskKilled` => TaskKilled - case `executorLostFailure` => ExecutorLostFailure + case `executorLostFailure` => + val executorId = Utils.jsonOption(json \ "Executor ID").map(_.extract[String]) + ExecutorLostFailure(executorId.getOrElse("Unknown")) case `unknownReason` => UnknownReason } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 3370dd4156c3f..6567c5ab836e7 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -119,7 +119,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc new ExceptionFailure("Exception", "description", null, None), TaskResultLost, TaskKilled, - ExecutorLostFailure, + ExecutorLostFailure("0"), UnknownReason) var failCount = 0 for (reason <- taskFailedReasons) { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index f1f88c5fd3634..d235d7a0ed839 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -115,7 +115,7 @@ class JsonProtocolSuite extends FunSuite { testTaskEndReason(exceptionFailure) testTaskEndReason(TaskResultLost) testTaskEndReason(TaskKilled) - testTaskEndReason(ExecutorLostFailure) + testTaskEndReason(ExecutorLostFailure("100")) testTaskEndReason(UnknownReason) // BlockId @@ -403,7 +403,8 @@ class JsonProtocolSuite extends FunSuite { assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals) case (TaskResultLost, TaskResultLost) => case (TaskKilled, TaskKilled) => - case (ExecutorLostFailure, ExecutorLostFailure) => + case (ExecutorLostFailure(execId1), ExecutorLostFailure(execId2)) => + assert(execId1 === execId2) case (UnknownReason, UnknownReason) => case _ => fail("Task end reasons don't match in types!") } From f0a4b630abf0766cc0c41e682691e0d435caca04 Mon Sep 17 00:00:00 2001 From: wangfei Date: Sun, 2 Nov 2014 14:59:41 -0800 Subject: [PATCH 1186/1492] [HOTFIX][SQL] hive test missing some golden files cc marmbrus Author: wangfei Closes #3055 from scwf/hotfix and squashes the following commits: d881bd7 [wangfei] miss golden files --- .../golden/truncate_table-1-7fc255c86d7c3a9ff088f9eb29a42565 | 0 .../golden/truncate_table-10-c32b771845f4d5a0330e2cfa09f89a7f | 0 .../golden/truncate_table-7-1ad5d350714e3d4ea17201153772d58d | 0 .../golden/truncate_table-8-76c754eac44c7254b45807255d4dbc3a | 0 .../golden/truncate_table-9-f4286b5657674a6a6b6bc6680f72f89a | 0 .../golden/udf_named_struct-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + .../golden/udf_named_struct-1-8f0ea83364b78634fbb3752c5a5c725 | 1 + .../golden/udf_named_struct-2-380c9638cc6ea8ea42f187bf0cedf350 | 1 + .../golden/udf_named_struct-3-c069e28293a12a813f8e881f776bae90 | 0 .../golden/udf_named_struct-4-b499d4120e009f222f2fab160a9006d7 | 1 + .../golden/udf_struct-0-50131c0ba7b7a6b65c789a5a8497bada | 1 + .../golden/udf_struct-1-f41043b7d9f14fa5e998c90454c7bdb1 | 1 + .../golden/udf_struct-2-8ccdb20153debdab789ea8ad0228e2eb | 1 + .../golden/udf_struct-3-71361a92b74c4d026ac7ae6e1e6746f1 | 0 .../golden/udf_struct-4-b196b5d8849d52bbe5e2ee683f29e051 | 1 + 15 files changed, 8 insertions(+) create mode 100644 sql/hive/src/test/resources/golden/truncate_table-1-7fc255c86d7c3a9ff088f9eb29a42565 create mode 100644 sql/hive/src/test/resources/golden/truncate_table-10-c32b771845f4d5a0330e2cfa09f89a7f create mode 100644 sql/hive/src/test/resources/golden/truncate_table-7-1ad5d350714e3d4ea17201153772d58d create mode 100644 sql/hive/src/test/resources/golden/truncate_table-8-76c754eac44c7254b45807255d4dbc3a create mode 100644 sql/hive/src/test/resources/golden/truncate_table-9-f4286b5657674a6a6b6bc6680f72f89a create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-1-8f0ea83364b78634fbb3752c5a5c725 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-2-380c9638cc6ea8ea42f187bf0cedf350 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-3-c069e28293a12a813f8e881f776bae90 create mode 100644 sql/hive/src/test/resources/golden/udf_named_struct-4-b499d4120e009f222f2fab160a9006d7 create mode 100644 sql/hive/src/test/resources/golden/udf_struct-0-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/udf_struct-1-f41043b7d9f14fa5e998c90454c7bdb1 create mode 100644 sql/hive/src/test/resources/golden/udf_struct-2-8ccdb20153debdab789ea8ad0228e2eb create mode 100644 sql/hive/src/test/resources/golden/udf_struct-3-71361a92b74c4d026ac7ae6e1e6746f1 create mode 100644 sql/hive/src/test/resources/golden/udf_struct-4-b196b5d8849d52bbe5e2ee683f29e051 diff --git a/sql/hive/src/test/resources/golden/truncate_table-1-7fc255c86d7c3a9ff088f9eb29a42565 b/sql/hive/src/test/resources/golden/truncate_table-1-7fc255c86d7c3a9ff088f9eb29a42565 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-10-c32b771845f4d5a0330e2cfa09f89a7f b/sql/hive/src/test/resources/golden/truncate_table-10-c32b771845f4d5a0330e2cfa09f89a7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-7-1ad5d350714e3d4ea17201153772d58d b/sql/hive/src/test/resources/golden/truncate_table-7-1ad5d350714e3d4ea17201153772d58d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-8-76c754eac44c7254b45807255d4dbc3a b/sql/hive/src/test/resources/golden/truncate_table-8-76c754eac44c7254b45807255d4dbc3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/truncate_table-9-f4286b5657674a6a6b6bc6680f72f89a b/sql/hive/src/test/resources/golden/truncate_table-9-f4286b5657674a6a6b6bc6680f72f89a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_named_struct-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-1-8f0ea83364b78634fbb3752c5a5c725 b/sql/hive/src/test/resources/golden/udf_named_struct-1-8f0ea83364b78634fbb3752c5a5c725 new file mode 100644 index 0000000000000..9bff96e7fa20e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-1-8f0ea83364b78634fbb3752c5a5c725 @@ -0,0 +1 @@ +named_struct(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-2-380c9638cc6ea8ea42f187bf0cedf350 b/sql/hive/src/test/resources/golden/udf_named_struct-2-380c9638cc6ea8ea42f187bf0cedf350 new file mode 100644 index 0000000000000..9bff96e7fa20e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-2-380c9638cc6ea8ea42f187bf0cedf350 @@ -0,0 +1 @@ +named_struct(name1, val1, name2, val2, ...) - Creates a struct with the given field names and values diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-3-c069e28293a12a813f8e881f776bae90 b/sql/hive/src/test/resources/golden/udf_named_struct-3-c069e28293a12a813f8e881f776bae90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_named_struct-4-b499d4120e009f222f2fab160a9006d7 b/sql/hive/src/test/resources/golden/udf_named_struct-4-b499d4120e009f222f2fab160a9006d7 new file mode 100644 index 0000000000000..de25f51b5b56d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_named_struct-4-b499d4120e009f222f2fab160a9006d7 @@ -0,0 +1 @@ +{"foo":1,"bar":2} 1 diff --git a/sql/hive/src/test/resources/golden/udf_struct-0-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/udf_struct-0-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-0-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_struct-1-f41043b7d9f14fa5e998c90454c7bdb1 b/sql/hive/src/test/resources/golden/udf_struct-1-f41043b7d9f14fa5e998c90454c7bdb1 new file mode 100644 index 0000000000000..062cb1bc683b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-1-f41043b7d9f14fa5e998c90454c7bdb1 @@ -0,0 +1 @@ +struct(col1, col2, col3, ...) - Creates a struct with the given field values diff --git a/sql/hive/src/test/resources/golden/udf_struct-2-8ccdb20153debdab789ea8ad0228e2eb b/sql/hive/src/test/resources/golden/udf_struct-2-8ccdb20153debdab789ea8ad0228e2eb new file mode 100644 index 0000000000000..062cb1bc683b1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-2-8ccdb20153debdab789ea8ad0228e2eb @@ -0,0 +1 @@ +struct(col1, col2, col3, ...) - Creates a struct with the given field values diff --git a/sql/hive/src/test/resources/golden/udf_struct-3-71361a92b74c4d026ac7ae6e1e6746f1 b/sql/hive/src/test/resources/golden/udf_struct-3-71361a92b74c4d026ac7ae6e1e6746f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_struct-4-b196b5d8849d52bbe5e2ee683f29e051 b/sql/hive/src/test/resources/golden/udf_struct-4-b196b5d8849d52bbe5e2ee683f29e051 new file mode 100644 index 0000000000000..ff1a28fa47f18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_struct-4-b196b5d8849d52bbe5e2ee683f29e051 @@ -0,0 +1 @@ +{"col1":1} {"col1":1,"col2":"a"} 1 a From 9c0eb57c737dd7d97d2cbd4516ddd2cf5d06e4b2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Nov 2014 15:08:35 -0800 Subject: [PATCH 1187/1492] [SPARK-3247][SQL] An API for adding data sources to Spark SQL This PR introduces a new set of APIs to Spark SQL to allow other developers to add support for reading data from new sources in `org.apache.spark.sql.sources`. New sources must implement the interface `BaseRelation`, which is responsible for describing the schema of the data. BaseRelations have three `Scan` subclasses, which are responsible for producing an RDD containing row objects. The [various Scan interfaces](https://github.com/marmbrus/spark/blob/foreign/sql/core/src/main/scala/org/apache/spark/sql/sources/package.scala#L50) allow for optimizations such as column pruning and filter push down, when the underlying data source can handle these operations. By implementing a class that inherits from RelationProvider these data sources can be accessed using using pure SQL. I've used the functionality to update the JSON support so it can now be used in this way as follows: ```sql CREATE TEMPORARY TABLE jsonTableSQL USING org.apache.spark.sql.json OPTIONS ( path '/home/michael/data.json' ) ``` Further example usage can be found in the test cases: https://github.com/marmbrus/spark/tree/foreign/sql/core/src/test/scala/org/apache/spark/sql/sources There is also a library that uses this new API to read avro data available here: https://github.com/marmbrus/sql-avro Author: Michael Armbrust Closes #2475 from marmbrus/foreign and squashes the following commits: 1ed6010 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into foreign ab2c31f [Michael Armbrust] fix test 1d41bb5 [Michael Armbrust] unify argument names 5b47901 [Michael Armbrust] Remove sealed, more filter types fab154a [Michael Armbrust] Merge remote-tracking branch 'origin/master' into foreign e3e690e [Michael Armbrust] Add hook for extraStrategies a70d602 [Michael Armbrust] Fix style, more tests, FilteredSuite => PrunedFilteredSuite 70da6d9 [Michael Armbrust] Modify API to ease binary compatibility and interop with Java 7d948ae [Michael Armbrust] Fix equality of AttributeReference. 5545491 [Michael Armbrust] Address comments 5031ac3 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into foreign 22963ef [Michael Armbrust] package objects compile wierdly... b069146 [Michael Armbrust] traits => abstract classes 34f836a [Michael Armbrust] Make @DeveloperApi 0d74bcf [Michael Armbrust] Add documention on object life cycle 3e06776 [Michael Armbrust] remove line wraps de3b68c [Michael Armbrust] Remove empty file 360cb30 [Michael Armbrust] style and java api 2957875 [Michael Armbrust] add override 0fd3a07 [Michael Armbrust] Draft of data sources API --- .../expressions/namedExpressions.scala | 2 +- .../apache/spark/sql/catalyst/package.scala | 4 + .../sql/catalyst/planning/QueryPlanner.scala | 20 +- .../spark/sql/catalyst/types/dataTypes.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 25 ++- .../spark/sql/api/java/JavaSQLContext.scala | 5 + .../spark/sql/execution/ExistingRDD.scala | 6 - .../spark/sql/execution/SparkStrategies.scala | 3 +- .../apache/spark/sql/execution/commands.scala | 35 +++- .../apache/spark/sql/json/JSONRelation.scala | 49 +++++ .../scala/org/apache/spark/sql/package.scala | 9 + .../sql/sources/DataSourceStrategy.scala | 112 +++++++++++ .../spark/sql/sources/LogicalRelation.scala | 54 ++++++ .../org/apache/spark/sql/sources/ddl.scala | 108 +++++++++++ .../apache/spark/sql/sources/filters.scala | 26 +++ .../apache/spark/sql/sources/interfaces.scala | 86 +++++++++ .../apache/spark/sql/sources/package.scala | 22 +++ .../apache/spark/sql/CachedTableSuite.scala | 12 -- .../org/apache/spark/sql/QueryTest.scala | 30 ++- .../org/apache/spark/sql/json/JsonSuite.scala | 26 +++ .../spark/sql/sources/DataSourceTest.scala | 34 ++++ .../spark/sql/sources/FilteredScanSuite.scala | 176 ++++++++++++++++++ .../spark/sql/sources/PrunedScanSuite.scala | 137 ++++++++++++++ .../spark/sql/sources/TableScanSuite.scala | 125 +++++++++++++ .../apache/spark/sql/hive/HiveContext.scala | 6 +- .../spark/sql/hive/HiveStrategies.scala | 2 +- 26 files changed, 1074 insertions(+), 42 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/sources/package.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 3310566087b3d..fc90a54a58259 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -134,7 +134,7 @@ case class AttributeReference( val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { override def equals(other: Any) = other match { - case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType + case ar: AttributeReference => name == ar.name && exprId == ar.exprId && dataType == ar.dataType case _ => false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala index bdd07bbeb2230..a38079ced34b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala @@ -17,6 +17,10 @@ package org.apache.spark.sql +/** + * Catalyst is a library for manipulating relational query plans. All classes in catalyst are + * considered an internal API to Spark SQL and are subject to change between minor releases. + */ package object catalyst { /** * A JVM-global lock that should be used to prevent thread safety issues when using things in diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index 5839c9f7c43ef..51b5699affed5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -21,6 +21,15 @@ import org.apache.spark.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode +/** + * Given a [[plans.logical.LogicalPlan LogicalPlan]], returns a list of `PhysicalPlan`s that can + * be used for execution. If this strategy does not apply to the give logical operation then an + * empty list should be returned. + */ +abstract class GenericStrategy[PhysicalPlan <: TreeNode[PhysicalPlan]] extends Logging { + def apply(plan: LogicalPlan): Seq[PhysicalPlan] +} + /** * Abstract class for transforming [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans. * Child classes are responsible for specifying a list of [[Strategy]] objects that each of which @@ -35,16 +44,7 @@ import org.apache.spark.sql.catalyst.trees.TreeNode */ abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] { /** A list of execution strategies that can be used by the planner */ - def strategies: Seq[Strategy] - - /** - * Given a [[plans.logical.LogicalPlan LogicalPlan]], returns a list of `PhysicalPlan`s that can - * be used for execution. If this strategy does not apply to the give logical operation then an - * empty list should be returned. - */ - abstract protected class Strategy extends Logging { - def apply(plan: LogicalPlan): Seq[PhysicalPlan] - } + def strategies: Seq[GenericStrategy[PhysicalPlan]] /** * Returns a placeholder for a physical plan that executes `plan`. This placeholder will be 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 8dda0b182805c..d25f3a619dd75 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 @@ -455,7 +455,7 @@ case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataT case class StructField( name: String, dataType: DataType, - nullable: Boolean, + nullable: Boolean = true, metadata: Metadata = Metadata.empty) { private[sql] def buildFormattedString(prefix: String, builder: StringBuilder): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4953f8399a96b..4cded98c803f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.{SparkStrategies, _} import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.sources.{DataSourceStrategy, BaseRelation, DDLParser, LogicalRelation} /** * :: AlphaComponent :: @@ -68,13 +69,19 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] lazy val optimizer: Optimizer = DefaultOptimizer + @transient + protected[sql] val ddlParser = new DDLParser + @transient protected[sql] val sqlParser = { val fallback = new catalyst.SqlParser new catalyst.SparkSQLParser(fallback(_)) } - protected[sql] def parseSql(sql: String): LogicalPlan = sqlParser(sql) + protected[sql] def parseSql(sql: String): LogicalPlan = { + ddlParser(sql).getOrElse(sqlParser(sql)) + } + protected[sql] def executeSql(sql: String): this.QueryExecution = executePlan(parseSql(sql)) protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } @@ -104,6 +111,10 @@ class SQLContext(@transient val sparkContext: SparkContext) LogicalRDD(ScalaReflection.attributesFor[A], RDDConversions.productToRowRdd(rdd))(self)) } + implicit def baseRelationToSchemaRDD(baseRelation: BaseRelation): SchemaRDD = { + logicalPlanToSparkQuery(LogicalRelation(baseRelation)) + } + /** * :: DeveloperApi :: * Creates a [[SchemaRDD]] from an [[RDD]] containing [[Row]]s by applying a schema to this RDD. @@ -283,6 +294,14 @@ class SQLContext(@transient val sparkContext: SparkContext) def table(tableName: String): SchemaRDD = new SchemaRDD(this, catalog.lookupRelation(None, tableName)) + /** + * :: DeveloperApi :: + * Allows extra strategies to be injected into the query planner at runtime. Note this API + * should be consider experimental and is not intended to be stable across releases. + */ + @DeveloperApi + var extraStrategies: Seq[Strategy] = Nil + protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext: SparkContext = self.sparkContext @@ -293,7 +312,9 @@ class SQLContext(@transient val sparkContext: SparkContext) def numPartitions = self.numShufflePartitions val strategies: Seq[Strategy] = + extraStrategies ++ ( CommandStrategy(self) :: + DataSourceStrategy :: TakeOrdered :: HashAggregation :: LeftSemiJoin :: @@ -302,7 +323,7 @@ class SQLContext(@transient val sparkContext: SparkContext) ParquetOperations :: BasicOperators :: CartesianProduct :: - BroadcastNestedLoopJoin :: Nil + BroadcastNestedLoopJoin :: Nil) /** * Used to build table scan operators where complex projection and filtering are done using diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 876b1c6edef20..60065509bfbbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation} import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.{SQLContext, StructType => SStructType} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} @@ -39,6 +40,10 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { def this(sparkContext: JavaSparkContext) = this(new SQLContext(sparkContext.sc)) + def baseRelationToSchemaRDD(baseRelation: BaseRelation): JavaSchemaRDD = { + new JavaSchemaRDD(sqlContext, LogicalRelation(baseRelation)) + } + /** * Executes a SQL query using Spark, returning the result as a SchemaRDD. The dialect that is * used for SQL parsing can be configured with 'spark.sql.dialect'. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 04c51a1ee4b97..d64c5af89ec99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -50,12 +50,6 @@ object RDDConversions { } } } - - /* - def toLogicalPlan[A <: Product : TypeTag](productRdd: RDD[A]): LogicalPlan = { - LogicalRDD(ScalaReflection.attributesFor[A], productToRowRdd(productRdd)) - } - */ } case class LogicalRDD(output: Seq[Attribute], rdd: RDD[Row])(sqlContext: SQLContext) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 79e4ddb8c4f5d..2cd3063bc3097 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -import org.apache.spark.sql.{SQLContext, execution} +import org.apache.spark.sql.{SQLContext, Strategy, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ @@ -304,6 +304,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case class CommandStrategy(context: SQLContext) extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case r: RunnableCommand => ExecutedCommand(r) :: Nil case logical.SetCommand(kv) => Seq(execution.SetCommand(kv, plan.output)(context)) case logical.ExplainCommand(logicalPlan, extended) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index 5859eba408ee1..e658e6fc4d5d5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -21,10 +21,12 @@ import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Row, Attribute} +import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.{Row, SQLConf, SQLContext} +import org.apache.spark.sql.{SQLConf, SQLContext} +// TODO: DELETE ME... trait Command { this: SparkPlan => @@ -44,6 +46,35 @@ trait Command { override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) } +// TODO: Replace command with runnable command. +trait RunnableCommand extends logical.Command { + self: Product => + + def output: Seq[Attribute] + def run(sqlContext: SQLContext): Seq[Row] +} + +case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { + /** + * A concrete command should override this lazy field to wrap up any side effects caused by the + * command or any other computation that should be evaluated exactly once. The value of this field + * can be used as the contents of the corresponding RDD generated from the physical plan of this + * command. + * + * The `execute()` method of all the physical command classes should reference `sideEffectResult` + * so that the command can be executed eagerly right after the command query is created. + */ + protected[sql] lazy val sideEffectResult: Seq[Row] = cmd.run(sqlContext) + + override def output = cmd.output + + override def children = Nil + + override def executeCollect(): Array[Row] = sideEffectResult.toArray + + override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) +} + /** * :: DeveloperApi :: */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala new file mode 100644 index 0000000000000..fc70c183437f6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.json + +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.sources._ + +private[sql] class DefaultSource extends RelationProvider { + /** Returns a new base relation with the given parameters. */ + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + val fileName = parameters.getOrElse("path", sys.error("Option 'path' not specified")) + val samplingRatio = parameters.get("samplingRatio").map(_.toDouble).getOrElse(1.0) + + JSONRelation(fileName, samplingRatio)(sqlContext) + } +} + +private[sql] case class JSONRelation(fileName: String, samplingRatio: Double)( + @transient val sqlContext: SQLContext) + extends TableScan { + + private def baseRDD = sqlContext.sparkContext.textFile(fileName) + + override val schema = + JsonRDD.inferSchema( + baseRDD, + samplingRatio, + sqlContext.columnNameOfCorruptRecord) + + override def buildScan() = + JsonRDD.jsonStringToRow(baseRDD, schema, sqlContext.columnNameOfCorruptRecord) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 05926a24c5307..51dad54f1a3f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.execution.SparkPlan /** * Allows the execution of relational queries, including those expressed in SQL using Spark. @@ -432,6 +433,12 @@ package object sql { @DeveloperApi val StructField = catalyst.types.StructField + /** + * Converts a logical plan into zero or more SparkPlans. + */ + @DeveloperApi + type Strategy = org.apache.spark.sql.catalyst.planning.GenericStrategy[SparkPlan] + /** * :: DeveloperApi :: * @@ -448,7 +455,9 @@ package object sql { type Metadata = catalyst.util.Metadata /** + * :: DeveloperApi :: * Builder for [[Metadata]]. If there is a key collision, the latter will overwrite the former. */ + @DeveloperApi type MetadataBuilder = catalyst.util.MetadataBuilder } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala new file mode 100644 index 0000000000000..9b8c6a56b94b4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan + +/** + * A Strategy for planning scans over data sources defined using the sources API. + */ +private[sql] object DataSourceStrategy extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: PrunedFilteredScan)) => + pruneFilterProject( + l, + projectList, + filters, + (a, f) => t.buildScan(a, f)) :: Nil + + case PhysicalOperation(projectList, filters, l @ LogicalRelation(t: PrunedScan)) => + pruneFilterProject( + l, + projectList, + filters, + (a, _) => t.buildScan(a)) :: Nil + + case l @ LogicalRelation(t: TableScan) => + execution.PhysicalRDD(l.output, t.buildScan()) :: Nil + + case _ => Nil + } + + protected def pruneFilterProject( + relation: LogicalRelation, + projectList: Seq[NamedExpression], + filterPredicates: Seq[Expression], + scanBuilder: (Array[String], Array[Filter]) => RDD[Row]) = { + + val projectSet = AttributeSet(projectList.flatMap(_.references)) + val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) + val filterCondition = filterPredicates.reduceLeftOption(And) + + val pushedFilters = selectFilters(filterPredicates.map { _ transform { + case a: AttributeReference => relation.attributeMap(a) // Match original case of attributes. + }}).toArray + + if (projectList.map(_.toAttribute) == projectList && + projectSet.size == projectList.size && + filterSet.subsetOf(projectSet)) { + // When it is possible to just use column pruning to get the right projection and + // when the columns of this projection are enough to evaluate all filter conditions, + // just do a scan followed by a filter, with no extra project. + val requestedColumns = + projectList.asInstanceOf[Seq[Attribute]] // Safe due to if above. + .map(relation.attributeMap) // Match original case of attributes. + .map(_.name) + .toArray + + val scan = + execution.PhysicalRDD( + projectList.map(_.toAttribute), + scanBuilder(requestedColumns, pushedFilters)) + filterCondition.map(execution.Filter(_, scan)).getOrElse(scan) + } else { + val requestedColumns = (projectSet ++ filterSet).map(relation.attributeMap).toSeq + val columnNames = requestedColumns.map(_.name).toArray + + val scan = execution.PhysicalRDD(requestedColumns, scanBuilder(columnNames, pushedFilters)) + execution.Project(projectList, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)) + } + } + + protected def selectFilters(filters: Seq[Expression]): Seq[Filter] = filters.collect { + case expressions.EqualTo(a: Attribute, Literal(v, _)) => EqualTo(a.name, v) + case expressions.EqualTo(Literal(v, _), a: Attribute) => EqualTo(a.name, v) + + case expressions.GreaterThan(a: Attribute, Literal(v, _)) => GreaterThan(a.name, v) + case expressions.GreaterThan(Literal(v, _), a: Attribute) => LessThan(a.name, v) + + case expressions.LessThan(a: Attribute, Literal(v, _)) => LessThan(a.name, v) + case expressions.LessThan(Literal(v, _), a: Attribute) => GreaterThan(a.name, v) + + case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, _)) => + GreaterThanOrEqual(a.name, v) + case expressions.GreaterThanOrEqual(Literal(v, _), a: Attribute) => + LessThanOrEqual(a.name, v) + + case expressions.LessThanOrEqual(a: Attribute, Literal(v, _)) => LessThanOrEqual(a.name, v) + case expressions.LessThanOrEqual(Literal(v, _), a: Attribute) => GreaterThanOrEqual(a.name, v) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala new file mode 100644 index 0000000000000..82a2cf8402f8f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/LogicalRelation.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.sources + +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.expressions.AttributeMap +import org.apache.spark.sql.catalyst.plans.logical.{Statistics, LeafNode, LogicalPlan} + +/** + * Used to link a [[BaseRelation]] in to a logical query plan. + */ +private[sql] case class LogicalRelation(relation: BaseRelation) + extends LeafNode + with MultiInstanceRelation { + + override val output = relation.schema.toAttributes + + // Logical Relations are distinct if they have different output for the sake of transformations. + override def equals(other: Any) = other match { + case l @ LogicalRelation(otherRelation) => relation == otherRelation && output == l.output + case _ => false + } + + override def sameResult(otherPlan: LogicalPlan) = otherPlan match { + case LogicalRelation(otherRelation) => relation == otherRelation + case _ => false + } + + @transient override lazy val statistics = Statistics( + // TODO: Allow datasources to provide statistics as well. + sizeInBytes = BigInt(relation.sqlContext.defaultSizeInBytes) + ) + + /** Used to lookup original attribute capitalization */ + val attributeMap = AttributeMap(output.map(o => (o, o))) + + def newInstance() = LogicalRelation(relation).asInstanceOf[this.type] + + override def simpleString = s"Relation[${output.mkString(",")}] $relation" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala new file mode 100644 index 0000000000000..9168ca2fc6fec --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +import org.apache.spark.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.execution.RunnableCommand +import org.apache.spark.util.Utils + +import scala.language.implicitConversions +import scala.util.parsing.combinator.lexical.StdLexical +import scala.util.parsing.combinator.syntactical.StandardTokenParsers +import scala.util.parsing.combinator.PackratParsers + +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.SqlLexical + +/** + * A parser for foreign DDL commands. + */ +private[sql] class DDLParser extends StandardTokenParsers with PackratParsers with Logging { + + def apply(input: String): Option[LogicalPlan] = { + phrase(ddl)(new lexical.Scanner(input)) match { + case Success(r, x) => Some(r) + case x => + logDebug(s"Not recognized as DDL: $x") + None + } + } + + protected case class Keyword(str: String) + + protected implicit def asParser(k: Keyword): Parser[String] = + lexical.allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) + + protected val CREATE = Keyword("CREATE") + protected val TEMPORARY = Keyword("TEMPORARY") + protected val TABLE = Keyword("TABLE") + protected val USING = Keyword("USING") + protected val OPTIONS = Keyword("OPTIONS") + + // Use reflection to find the reserved words defined in this class. + protected val reservedWords = + this.getClass + .getMethods + .filter(_.getReturnType == classOf[Keyword]) + .map(_.invoke(this).asInstanceOf[Keyword].str) + + override val lexical = new SqlLexical(reservedWords) + + protected lazy val ddl: Parser[LogicalPlan] = createTable + + /** + * CREATE FOREIGN TEMPORARY TABLE avroTable + * USING org.apache.spark.sql.avro + * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro") + */ + protected lazy val createTable: Parser[LogicalPlan] = + CREATE ~ TEMPORARY ~ TABLE ~> ident ~ (USING ~> className) ~ (OPTIONS ~> options) ^^ { + case tableName ~ provider ~ opts => + CreateTableUsing(tableName, provider, opts) + } + + protected lazy val options: Parser[Map[String, String]] = + "(" ~> repsep(pair, ",") <~ ")" ^^ { case s: Seq[(String, String)] => s.toMap } + + protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")} + + protected lazy val pair: Parser[(String, String)] = ident ~ stringLit ^^ { case k ~ v => (k,v) } +} + +private[sql] case class CreateTableUsing( + tableName: String, + provider: String, + options: Map[String, String]) extends RunnableCommand { + + def run(sqlContext: SQLContext) = { + val loader = Utils.getContextOrSparkClassLoader + val clazz: Class[_] = try loader.loadClass(provider) catch { + case cnf: java.lang.ClassNotFoundException => + try loader.loadClass(provider + ".DefaultSource") catch { + case cnf: java.lang.ClassNotFoundException => + sys.error(s"Failed to load class for data source: $provider") + } + } + val dataSource = clazz.newInstance().asInstanceOf[org.apache.spark.sql.sources.RelationProvider] + val relation = dataSource.createRelation(sqlContext, options) + + sqlContext.baseRelationToSchemaRDD(relation).registerTempTable(tableName) + Seq.empty + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala new file mode 100644 index 0000000000000..e72a2aeb8f310 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -0,0 +1,26 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources + +abstract class Filter + +case class EqualTo(attribute: String, value: Any) extends Filter +case class GreaterThan(attribute: String, value: Any) extends Filter +case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter +case class LessThan(attribute: String, value: Any) extends Filter +case class LessThanOrEqual(attribute: String, value: Any) extends Filter diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala new file mode 100644 index 0000000000000..ac3bf9d8e1a21 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -0,0 +1,86 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +package org.apache.spark.sql.sources + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext, StructType} +import org.apache.spark.sql.catalyst.expressions.{Expression, Attribute} + +/** + * Implemented by objects that produce relations for a specific kind of data source. When + * Spark SQL is given a DDL operation with a USING clause specified, this interface is used to + * pass in the parameters specified by a user. + * + * Users may specify the fully qualified class name of a given data source. When that class is + * not found Spark SQL will append the class name `DefaultSource` to the path, allowing for + * less verbose invocation. For example, 'org.apache.spark.sql.json' would resolve to the + * data source 'org.apache.spark.sql.json.DefaultSource' + * + * A new instance of this class with be instantiated each time a DDL call is made. + */ +@DeveloperApi +trait RelationProvider { + /** Returns a new base relation with the given parameters. */ + def createRelation(sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation +} + +/** + * Represents a collection of tuples with a known schema. Classes that extend BaseRelation must + * be able to produce the schema of their data in the form of a [[StructType]] Concrete + * implementation should inherit from one of the descendant `Scan` classes, which define various + * abstract methods for execution. + * + * BaseRelations must also define a equality function that only returns true when the two + * instances will return the same data. This equality function is used when determining when + * it is safe to substitute cached results for a given relation. + */ +@DeveloperApi +abstract class BaseRelation { + def sqlContext: SQLContext + def schema: StructType +} + +/** + * A BaseRelation that can produce all of its tuples as an RDD of Row objects. + */ +@DeveloperApi +abstract class TableScan extends BaseRelation { + def buildScan(): RDD[Row] +} + +/** + * A BaseRelation that can eliminate unneeded columns before producing an RDD + * containing all of its tuples as Row objects. + */ +@DeveloperApi +abstract class PrunedScan extends BaseRelation { + def buildScan(requiredColumns: Array[String]): RDD[Row] +} + +/** + * A BaseRelation that can eliminate unneeded columns and filter using selected + * predicates before producing an RDD containing all matching tuples as Row objects. + * + * The pushed down filters are currently purely an optimization as they will all be evaluated + * again. This means it is safe to use them with methods that produce false positives such + * as filtering partitions based on a bloom filter. + */ +@DeveloperApi +abstract class PrunedFilteredScan extends BaseRelation { + def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/package.scala new file mode 100644 index 0000000000000..8393c510f4f6d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/package.scala @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql + +/** + * A set of APIs for adding data sources to Spark SQL. + */ +package object sources diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 1a5d87d5240e9..44a2961b27eda 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -27,18 +27,6 @@ case class BigData(s: String) class CachedTableSuite extends QueryTest { TestData // Load test tables. - def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = { - val planWithCaching = query.queryExecution.withCachedData - val cachedData = planWithCaching collect { - case cached: InMemoryRelation => cached - } - - assert( - cachedData.size == numCachedTables, - s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + - planWithCaching) - } - def rddIdOf(tableName: String): Int = { val executedPlan = table(tableName).queryExecution.executedPlan executedPlan.collect { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 042f61f5a4113..3d9f0cbf80fe7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -19,8 +19,10 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.columnar.InMemoryRelation class QueryTest extends PlanTest { + /** * Runs the plan and makes sure the answer contains all of the keywords, or the * none of keywords are listed in the answer @@ -78,11 +80,31 @@ class QueryTest extends PlanTest { |${rdd.queryExecution.executedPlan} |== Results == |${sideBySide( - s"== Correct Answer - ${convertedAnswer.size} ==" +: - prepareAnswer(convertedAnswer).map(_.toString), - s"== Spark Answer - ${sparkAnswer.size} ==" +: - prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} + s"== Correct Answer - ${convertedAnswer.size} ==" +: + prepareAnswer(convertedAnswer).map(_.toString), + s"== Spark Answer - ${sparkAnswer.size} ==" +: + prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} """.stripMargin) } } + + def sqlTest(sqlString: String, expectedAnswer: Any)(implicit sqlContext: SQLContext): Unit = { + test(sqlString) { + checkAnswer(sqlContext.sql(sqlString), expectedAnswer) + } + } + + /** Asserts that a given SchemaRDD will be executed using the given number of cached results. */ + def assertCached(query: SchemaRDD, numCachedTables: Int = 1): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val cachedData = planWithCaching collect { + case cached: InMemoryRelation => cached + } + + assert( + cachedData.size == numCachedTables, + s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" + + planWithCaching) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 1cb6c23c58f36..362c7e1a52482 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -549,6 +549,32 @@ class JsonSuite extends QueryTest { ) } + test("Loading a JSON dataset from a text file with SQL") { + val file = getTempFilePath("json") + val path = file.toString + primitiveFieldAndType.map(record => record.replaceAll("\n", " ")).saveAsTextFile(path) + + sql( + s""" + |CREATE TEMPORARY TABLE jsonTableSQL + |USING org.apache.spark.sql.json + |OPTIONS ( + | path '$path' + |) + """.stripMargin) + + checkAnswer( + sql("select * from jsonTableSQL"), + (BigDecimal("92233720368547758070"), + true, + 1.7976931348623157E308, + 10, + 21474836470L, + null, + "this is a simple string.") :: Nil + ) + } + test("Applying schemas") { val file = getTempFilePath("json") val path = file.toString diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala new file mode 100644 index 0000000000000..9626252e742e5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala @@ -0,0 +1,34 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.sources + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.analysis.Analyzer +import org.apache.spark.sql.test.TestSQLContext +import org.scalatest.BeforeAndAfter + +abstract class DataSourceTest extends QueryTest with BeforeAndAfter { + // Case sensitivity is not configurable yet, but we want to test some edge cases. + // TODO: Remove when it is configurable + implicit val caseInsensisitiveContext = new SQLContext(TestSQLContext.sparkContext) { + @transient + override protected[sql] lazy val analyzer: Analyzer = + new Analyzer(catalog, functionRegistry, caseSensitive = false) + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala new file mode 100644 index 0000000000000..8b2f1591d5bf3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -0,0 +1,176 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.sources + +import scala.language.existentials + +import org.apache.spark.sql._ + +class FilteredScanSource extends RelationProvider { + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + SimpleFilteredScan(parameters("from").toInt, parameters("to").toInt)(sqlContext) + } +} + +case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) + extends PrunedFilteredScan { + + override def schema = + StructType( + StructField("a", IntegerType, nullable = false) :: + StructField("b", IntegerType, nullable = false) :: Nil) + + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { + val rowBuilders = requiredColumns.map { + case "a" => (i: Int) => Seq(i) + case "b" => (i: Int) => Seq(i * 2) + } + + FiltersPushed.list = filters + + val filterFunctions = filters.collect { + case EqualTo("a", v) => (a: Int) => a == v + case LessThan("a", v: Int) => (a: Int) => a < v + case LessThanOrEqual("a", v: Int) => (a: Int) => a <= v + case GreaterThan("a", v: Int) => (a: Int) => a > v + case GreaterThanOrEqual("a", v: Int) => (a: Int) => a >= v + } + + def eval(a: Int) = !filterFunctions.map(_(a)).contains(false) + + sqlContext.sparkContext.parallelize(from to to).filter(eval).map(i => + Row.fromSeq(rowBuilders.map(_(i)).reduceOption(_ ++ _).getOrElse(Seq.empty))) + } +} + +// A hack for better error messages when filter pushdown fails. +object FiltersPushed { + var list: Seq[Filter] = Nil +} + +class FilteredScanSuite extends DataSourceTest { + + import caseInsensisitiveContext._ + + before { + sql( + """ + |CREATE TEMPORARY TABLE oneToTenFiltered + |USING org.apache.spark.sql.sources.FilteredScanSource + |OPTIONS ( + | from '1', + | to '10' + |) + """.stripMargin) + } + + sqlTest( + "SELECT * FROM oneToTenFiltered", + (1 to 10).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT a, b FROM oneToTenFiltered", + (1 to 10).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT b, a FROM oneToTenFiltered", + (1 to 10).map(i => Row(i * 2, i)).toSeq) + + sqlTest( + "SELECT a FROM oneToTenFiltered", + (1 to 10).map(i => Row(i)).toSeq) + + sqlTest( + "SELECT b FROM oneToTenFiltered", + (1 to 10).map(i => Row(i * 2)).toSeq) + + sqlTest( + "SELECT a * 2 FROM oneToTenFiltered", + (1 to 10).map(i => Row(i * 2)).toSeq) + + sqlTest( + "SELECT A AS b FROM oneToTenFiltered", + (1 to 10).map(i => Row(i)).toSeq) + + sqlTest( + "SELECT x.b, y.a FROM oneToTenFiltered x JOIN oneToTenFiltered y ON x.a = y.b", + (1 to 5).map(i => Row(i * 4, i)).toSeq) + + sqlTest( + "SELECT x.a, y.b FROM oneToTenFiltered x JOIN oneToTenFiltered y ON x.a = y.b", + (2 to 10 by 2).map(i => Row(i, i)).toSeq) + + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE a = 1", + Seq(1).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE A = 1", + Seq(1).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT * FROM oneToTenFiltered WHERE b = 2", + Seq(1).map(i => Row(i, i * 2)).toSeq) + + testPushDown("SELECT * FROM oneToTenFiltered WHERE A = 1", 1) + testPushDown("SELECT a FROM oneToTenFiltered WHERE A = 1", 1) + testPushDown("SELECT b FROM oneToTenFiltered WHERE A = 1", 1) + testPushDown("SELECT a, b FROM oneToTenFiltered WHERE A = 1", 1) + testPushDown("SELECT * FROM oneToTenFiltered WHERE a = 1", 1) + testPushDown("SELECT * FROM oneToTenFiltered WHERE 1 = a", 1) + + testPushDown("SELECT * FROM oneToTenFiltered WHERE a > 1", 9) + testPushDown("SELECT * FROM oneToTenFiltered WHERE a >= 2", 9) + + testPushDown("SELECT * FROM oneToTenFiltered WHERE 1 < a", 9) + testPushDown("SELECT * FROM oneToTenFiltered WHERE 2 <= a", 9) + + testPushDown("SELECT * FROM oneToTenFiltered WHERE 1 > a", 0) + testPushDown("SELECT * FROM oneToTenFiltered WHERE 2 >= a", 2) + + testPushDown("SELECT * FROM oneToTenFiltered WHERE a < 1", 0) + testPushDown("SELECT * FROM oneToTenFiltered WHERE a <= 2", 2) + + testPushDown("SELECT * FROM oneToTenFiltered WHERE a > 1 AND a < 10", 8) + + testPushDown("SELECT * FROM oneToTenFiltered WHERE a = 20", 0) + testPushDown("SELECT * FROM oneToTenFiltered WHERE b = 1", 10) + + def testPushDown(sqlString: String, expectedCount: Int): Unit = { + test(s"PushDown Returns $expectedCount: $sqlString") { + val queryExecution = sql(sqlString).queryExecution + val rawPlan = queryExecution.executedPlan.collect { + case p: execution.PhysicalRDD => p + } match { + case Seq(p) => p + case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") + } + val rawCount = rawPlan.execute().count() + + if (rawCount != expectedCount) { + fail( + s"Wrong # of results for pushed filter. Got $rawCount, Expected $expectedCount\n" + + s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" + + queryExecution) + } + } + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala new file mode 100644 index 0000000000000..fee2e22611cdc --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -0,0 +1,137 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.sources + +import org.apache.spark.sql._ + +class PrunedScanSource extends RelationProvider { + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + SimplePrunedScan(parameters("from").toInt, parameters("to").toInt)(sqlContext) + } +} + +case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) + extends PrunedScan { + + override def schema = + StructType( + StructField("a", IntegerType, nullable = false) :: + StructField("b", IntegerType, nullable = false) :: Nil) + + override def buildScan(requiredColumns: Array[String]) = { + val rowBuilders = requiredColumns.map { + case "a" => (i: Int) => Seq(i) + case "b" => (i: Int) => Seq(i * 2) + } + + sqlContext.sparkContext.parallelize(from to to).map(i => + Row.fromSeq(rowBuilders.map(_(i)).reduceOption(_ ++ _).getOrElse(Seq.empty))) + } +} + +class PrunedScanSuite extends DataSourceTest { + import caseInsensisitiveContext._ + + before { + sql( + """ + |CREATE TEMPORARY TABLE oneToTenPruned + |USING org.apache.spark.sql.sources.PrunedScanSource + |OPTIONS ( + | from '1', + | to '10' + |) + """.stripMargin) + } + + sqlTest( + "SELECT * FROM oneToTenPruned", + (1 to 10).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT a, b FROM oneToTenPruned", + (1 to 10).map(i => Row(i, i * 2)).toSeq) + + sqlTest( + "SELECT b, a FROM oneToTenPruned", + (1 to 10).map(i => Row(i * 2, i)).toSeq) + + sqlTest( + "SELECT a FROM oneToTenPruned", + (1 to 10).map(i => Row(i)).toSeq) + + sqlTest( + "SELECT a, a FROM oneToTenPruned", + (1 to 10).map(i => Row(i, i)).toSeq) + + sqlTest( + "SELECT b FROM oneToTenPruned", + (1 to 10).map(i => Row(i * 2)).toSeq) + + sqlTest( + "SELECT a * 2 FROM oneToTenPruned", + (1 to 10).map(i => Row(i * 2)).toSeq) + + sqlTest( + "SELECT A AS b FROM oneToTenPruned", + (1 to 10).map(i => Row(i)).toSeq) + + sqlTest( + "SELECT x.b, y.a FROM oneToTenPruned x JOIN oneToTenPruned y ON x.a = y.b", + (1 to 5).map(i => Row(i * 4, i)).toSeq) + + sqlTest( + "SELECT x.a, y.b FROM oneToTenPruned x JOIN oneToTenPruned y ON x.a = y.b", + (2 to 10 by 2).map(i => Row(i, i)).toSeq) + + testPruning("SELECT * FROM oneToTenPruned", "a", "b") + testPruning("SELECT a, b FROM oneToTenPruned", "a", "b") + testPruning("SELECT b, a FROM oneToTenPruned", "b", "a") + testPruning("SELECT b, b FROM oneToTenPruned", "b") + testPruning("SELECT a FROM oneToTenPruned", "a") + testPruning("SELECT b FROM oneToTenPruned", "b") + + def testPruning(sqlString: String, expectedColumns: String*): Unit = { + test(s"Columns output ${expectedColumns.mkString(",")}: $sqlString") { + val queryExecution = sql(sqlString).queryExecution + val rawPlan = queryExecution.executedPlan.collect { + case p: execution.PhysicalRDD => p + } match { + case Seq(p) => p + case _ => fail(s"More than one PhysicalRDD found\n$queryExecution") + } + val rawColumns = rawPlan.output.map(_.name) + val rawOutput = rawPlan.execute().first() + + if (rawColumns != expectedColumns) { + fail( + s"Wrong column names. Got $rawColumns, Expected $expectedColumns\n" + + s"Filters pushed: ${FiltersPushed.list.mkString(",")}\n" + + queryExecution) + } + + if (rawOutput.size != expectedColumns.size) { + fail(s"Wrong output row. Got $rawOutput\n$queryExecution") + } + } + } + +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala new file mode 100644 index 0000000000000..b254b0620c779 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -0,0 +1,125 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ + +package org.apache.spark.sql.sources + +import org.apache.spark.sql._ + +class DefaultSource extends SimpleScanSource + +class SimpleScanSource extends RelationProvider { + override def createRelation( + sqlContext: SQLContext, + parameters: Map[String, String]): BaseRelation = { + SimpleScan(parameters("from").toInt, parameters("to").toInt)(sqlContext) + } +} + +case class SimpleScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) + extends TableScan { + + override def schema = + StructType(StructField("i", IntegerType, nullable = false) :: Nil) + + override def buildScan() = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) +} + +class TableScanSuite extends DataSourceTest { + import caseInsensisitiveContext._ + + before { + sql( + """ + |CREATE TEMPORARY TABLE oneToTen + |USING org.apache.spark.sql.sources.SimpleScanSource + |OPTIONS ( + | from '1', + | to '10' + |) + """.stripMargin) + } + + sqlTest( + "SELECT * FROM oneToTen", + (1 to 10).map(Row(_)).toSeq) + + sqlTest( + "SELECT i FROM oneToTen", + (1 to 10).map(Row(_)).toSeq) + + sqlTest( + "SELECT i FROM oneToTen WHERE i < 5", + (1 to 4).map(Row(_)).toSeq) + + sqlTest( + "SELECT i * 2 FROM oneToTen", + (1 to 10).map(i => Row(i * 2)).toSeq) + + sqlTest( + "SELECT a.i, b.i FROM oneToTen a JOIN oneToTen b ON a.i = b.i + 1", + (2 to 10).map(i => Row(i, i - 1)).toSeq) + + + test("Caching") { + // Cached Query Execution + cacheTable("oneToTen") + assertCached(sql("SELECT * FROM oneToTen")) + checkAnswer( + sql("SELECT * FROM oneToTen"), + (1 to 10).map(Row(_)).toSeq) + + assertCached(sql("SELECT i FROM oneToTen")) + checkAnswer( + sql("SELECT i FROM oneToTen"), + (1 to 10).map(Row(_)).toSeq) + + assertCached(sql("SELECT i FROM oneToTen WHERE i < 5")) + checkAnswer( + sql("SELECT i FROM oneToTen WHERE i < 5"), + (1 to 4).map(Row(_)).toSeq) + + assertCached(sql("SELECT i * 2 FROM oneToTen")) + checkAnswer( + sql("SELECT i * 2 FROM oneToTen"), + (1 to 10).map(i => Row(i * 2)).toSeq) + + assertCached(sql("SELECT a.i, b.i FROM oneToTen a JOIN oneToTen b ON a.i = b.i + 1"), 2) + checkAnswer( + sql("SELECT a.i, b.i FROM oneToTen a JOIN oneToTen b ON a.i = b.i + 1"), + (2 to 10).map(i => Row(i, i - 1)).toSeq) + + // Verify uncaching + uncacheTable("oneToTen") + assertCached(sql("SELECT * FROM oneToTen"), 0) + } + + test("defaultSource") { + sql( + """ + |CREATE TEMPORARY TABLE oneToTenDef + |USING org.apache.spark.sql.sources + |OPTIONS ( + | from '1', + | to '10' + |) + """.stripMargin) + + checkAnswer( + sql("SELECT * FROM oneToTenDef"), + (1 to 10).map(Row(_)).toSeq) + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 2e27817d60221..dca5367f244de 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -50,6 +50,7 @@ import org.apache.spark.sql.execution.ExtractPythonUdfs import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.{Command => PhysicalCommand} import org.apache.spark.sql.hive.execution.DescribeHiveTableCommand +import org.apache.spark.sql.sources.DataSourceStrategy /** * DEPRECATED: Use HiveContext instead. @@ -99,7 +100,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { if (dialect == "sql") { super.sql(sqlText) } else if (dialect == "hiveql") { - new SchemaRDD(this, HiveQl.parseSql(sqlText)) + new SchemaRDD(this, ddlParser(sqlText).getOrElse(HiveQl.parseSql(sqlText))) } else { sys.error(s"Unsupported SQL dialect: $dialect. Try 'sql' or 'hiveql'") } @@ -345,7 +346,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val hivePlanner = new SparkPlanner with HiveStrategies { val hiveContext = self - override val strategies: Seq[Strategy] = Seq( + override val strategies: Seq[Strategy] = extraStrategies ++ Seq( + DataSourceStrategy, CommandStrategy(self), HiveCommandStrategy(self), TakeOrdered, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 3207ad81d9571..989740c8d43b6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.{SQLContext, SchemaRDD} +import org.apache.spark.sql.{SQLContext, SchemaRDD, Strategy} import scala.collection.JavaConversions._ From e4b80894bdb72c0acf8832fd48421c546fbc37e6 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 2 Nov 2014 15:14:44 -0800 Subject: [PATCH 1188/1492] [SPARK-4182][SQL] Fixes ColumnStats classes for boolean, binary and complex data types `NoopColumnStats` was once used for binary, boolean and complex data types. This `ColumnStats` doesn't return properly shaped column statistics and causes caching failure if a table contains columns of the aforementioned types. This PR adds `BooleanColumnStats`, `BinaryColumnStats` and `GenericColumnStats`, used for boolean, binary and all complex data types respectively. In addition, `NoopColumnStats` returns properly shaped column statistics containing null count and row count, but this class is now used for testing purpose only. Author: Cheng Lian Closes #3059 from liancheng/spark-4182 and squashes the following commits: b398cfd [Cheng Lian] Fixes failed test case fb3ee85 [Cheng Lian] Fixes SPARK-4182 --- .../spark/sql/columnar/ColumnBuilder.scala | 10 +++-- .../spark/sql/columnar/ColumnStats.scala | 45 ++++++++++++++++++- .../columnar/InMemoryColumnarTableScan.scala | 3 ++ .../org/apache/spark/sql/SQLQuerySuite.scala | 7 +-- .../scala/org/apache/spark/sql/TestData.scala | 8 ++++ .../columnar/InMemoryColumnarQuerySuite.scala | 28 +++++++----- 6 files changed, 82 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 300cef15bf8a4..c68dceef3b142 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -79,8 +79,9 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( } private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType]( + columnStats: ColumnStats, columnType: ColumnType[T, JvmType]) - extends BasicColumnBuilder[T, JvmType](new NoopColumnStats, columnType) + extends BasicColumnBuilder[T, JvmType](columnStats, columnType) with NullableColumnBuilder private[sql] abstract class NativeColumnBuilder[T <: NativeType]( @@ -91,7 +92,7 @@ private[sql] abstract class NativeColumnBuilder[T <: NativeType]( with AllCompressionSchemes with CompressibleColumnBuilder[T] -private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new NoopColumnStats, BOOLEAN) +private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN) private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT) @@ -112,10 +113,11 @@ private[sql] class DateColumnBuilder extends NativeColumnBuilder(new DateColumnS private[sql] class TimestampColumnBuilder extends NativeColumnBuilder(new TimestampColumnStats, TIMESTAMP) -private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY) +private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(new BinaryColumnStats, BINARY) // TODO (lian) Add support for array, struct and map -private[sql] class GenericColumnBuilder extends ComplexColumnBuilder(GENERIC) +private[sql] class GenericColumnBuilder + extends ComplexColumnBuilder(new GenericColumnStats, GENERIC) private[sql] object ColumnBuilder { val DEFAULT_INITIAL_BUFFER_SIZE = 1024 * 1024 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index b9f9f8270045c..668efe4a3b2a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -70,11 +70,30 @@ private[sql] sealed trait ColumnStats extends Serializable { def collectedStatistics: Row } +/** + * A no-op ColumnStats only used for testing purposes. + */ private[sql] class NoopColumnStats extends ColumnStats { + override def gatherStats(row: Row, ordinal: Int): Unit = super.gatherStats(row, ordinal) + + def collectedStatistics = Row(null, null, nullCount, count, 0L) +} - override def gatherStats(row: Row, ordinal: Int): Unit = {} +private[sql] class BooleanColumnStats extends ColumnStats { + protected var upper = false + protected var lower = true - override def collectedStatistics = Row() + override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + val value = row.getBoolean(ordinal) + if (value > upper) upper = value + if (value < lower) lower = value + sizeInBytes += BOOLEAN.defaultSize + } + } + + def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } private[sql] class ByteColumnStats extends ColumnStats { @@ -229,3 +248,25 @@ private[sql] class TimestampColumnStats extends ColumnStats { def collectedStatistics = Row(lower, upper, nullCount, count, sizeInBytes) } + +private[sql] class BinaryColumnStats extends ColumnStats { + override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + sizeInBytes += BINARY.actualSize(row, ordinal) + } + } + + def collectedStatistics = Row(null, null, nullCount, count, sizeInBytes) +} + +private[sql] class GenericColumnStats extends ColumnStats { + override def gatherStats(row: Row, ordinal: Int): Unit = { + super.gatherStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + sizeInBytes += GENERIC.actualSize(row, ordinal) + } + } + + def collectedStatistics = Row(null, null, nullCount, count, sizeInBytes) +} 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 ee63134f56d8c..455b415d9d959 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 @@ -161,6 +161,9 @@ private[sql] case class InMemoryRelation( } def cachedColumnBuffers = _cachedColumnBuffers + + override protected def otherCopyArgs: Seq[AnyRef] = + Seq(_cachedColumnBuffers, statisticsToBePropagated) } private[sql] case class InMemoryColumnarTableScan( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6befe1b755cc6..6bf439377aa3e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -21,11 +21,12 @@ import java.util.TimeZone import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +/* Implicits */ +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.test.TestSQLContext._ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { @@ -719,7 +720,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { validateMetadata(sql("SELECT * FROM personWithMeta JOIN salary ON id = personId")) validateMetadata(sql("SELECT name, salary FROM personWithMeta JOIN salary ON id = personId")) } - + test("SPARK-3371 Renaming a function expression with group by gives error") { registerFunction("len", (s: String) => s.length) checkAnswer( @@ -934,7 +935,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-4154 Query does not work if it has 'not between' in Spark SQL and HQL") { - checkAnswer(sql("SELECT key FROM testData WHERE key not between 0 and 10 order by key"), + checkAnswer(sql("SELECT key FROM testData WHERE key not between 0 and 10 order by key"), (11 to 100).map(i => Seq(i))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 836dd17fcc3a2..ef87a230639bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -177,4 +177,12 @@ object TestData { Salary(0, 2000.0) :: Salary(1, 1000.0) :: Nil) salary.registerTempTable("salary") + + case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) + val complexData = + TestSQLContext.sparkContext.parallelize( + ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true) + :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false) + :: Nil).toSchemaRDD + complexData.registerTempTable("complexData") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 9775dd26b7773..15903d07df29a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,17 +17,18 @@ package org.apache.spark.sql.columnar +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY class InMemoryColumnarQuerySuite extends QueryTest { - import org.apache.spark.sql.TestData._ - import org.apache.spark.sql.test.TestSQLContext._ + // Make sure the tables are loaded. + TestData test("simple columnar query") { - val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan + val plan = executePlan(testData.logicalPlan).executedPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().toSeq) @@ -42,7 +43,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { } test("projection") { - val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan + val plan = executePlan(testData.select('value, 'key).logicalPlan).executedPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().map { @@ -51,7 +52,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { } test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { - val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan + val plan = executePlan(testData.logicalPlan).executedPlan val scan = InMemoryRelation(useCompression = true, 5, MEMORY_ONLY, plan) checkAnswer(scan, testData.collect().toSeq) @@ -63,7 +64,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM repeatedData"), repeatedData.collect().toSeq) - TestSQLContext.cacheTable("repeatedData") + cacheTable("repeatedData") checkAnswer( sql("SELECT * FROM repeatedData"), @@ -75,7 +76,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM nullableRepeatedData"), nullableRepeatedData.collect().toSeq) - TestSQLContext.cacheTable("nullableRepeatedData") + cacheTable("nullableRepeatedData") checkAnswer( sql("SELECT * FROM nullableRepeatedData"), @@ -87,7 +88,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT time FROM timestamps"), timestamps.collect().toSeq) - TestSQLContext.cacheTable("timestamps") + cacheTable("timestamps") checkAnswer( sql("SELECT time FROM timestamps"), @@ -99,10 +100,17 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM withEmptyParts"), withEmptyParts.collect().toSeq) - TestSQLContext.cacheTable("withEmptyParts") + cacheTable("withEmptyParts") checkAnswer( sql("SELECT * FROM withEmptyParts"), withEmptyParts.collect().toSeq) } + + test("SPARK-4182 Caching complex types") { + complexData.cache().count() + // Shouldn't throw + complexData.count() + complexData.unpersist() + } } From 495a132031ae002c787371f2fd0ba4be2437e7c8 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 2 Nov 2014 15:15:52 -0800 Subject: [PATCH 1189/1492] [SQL] Fixes race condition in CliSuite `CliSuite` has been flaky for a while, this PR tries to improve this situation by fixing a race condition in `CliSuite`. The `captureOutput` function is used to capture both stdout and stderr output of the forked external process in two background threads and search for expected strings, but wasn't been properly synchronized before. Author: Cheng Lian Closes #3060 from liancheng/fix-cli-suite and squashes the following commits: a70569c [Cheng Lian] Fixes race condition in CliSuite --- .../sql/hive/thriftserver/CliSuite.scala | 35 ++++++++----------- 1 file changed, 15 insertions(+), 20 deletions(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 8a72e9d2aef57..e8ffbc5b954d4 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -18,19 +18,17 @@ package org.apache.spark.sql.hive.thriftserver +import java.io._ + import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ -import scala.concurrent.{Await, Future, Promise} +import scala.concurrent.{Await, Promise} import scala.sys.process.{Process, ProcessLogger} -import java.io._ -import java.util.concurrent.atomic.AtomicInteger - import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.apache.spark.{SparkException, Logging} +import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { @@ -53,23 +51,20 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { """.stripMargin.split("\\s+").toSeq ++ extraArgs } - // AtomicInteger is needed because stderr and stdout of the forked process are handled in - // different threads. - val next = new AtomicInteger(0) + var next = 0 val foundAllExpectedAnswers = Promise.apply[Unit]() val queryStream = new ByteArrayInputStream(queries.mkString("\n").getBytes) val buffer = new ArrayBuffer[String]() + val lock = new Object - def captureOutput(source: String)(line: String) { + def captureOutput(source: String)(line: String): Unit = lock.synchronized { buffer += s"$source> $line" - // If we haven't found all expected answers... - if (next.get() < expectedAnswers.size) { - // If another expected answer is found... - if (line.startsWith(expectedAnswers(next.get()))) { - // If all expected answers have been found... - if (next.incrementAndGet() == expectedAnswers.size) { - foundAllExpectedAnswers.trySuccess(()) - } + // If we haven't found all expected answers and another expected answer comes up... + if (next < expectedAnswers.size && line.startsWith(expectedAnswers(next))) { + next += 1 + // If all expected answers have been found... + if (next == expectedAnswers.size) { + foundAllExpectedAnswers.trySuccess(()) } } } @@ -88,8 +83,8 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { |======================= |Spark SQL CLI command line: ${command.mkString(" ")} | - |Executed query ${next.get()} "${queries(next.get())}", - |But failed to capture expected output "${expectedAnswers(next.get())}" within $timeout. + |Executed query $next "${queries(next)}", + |But failed to capture expected output "${expectedAnswers(next)}" within $timeout. | |${buffer.mkString("\n")} |=========================== From c9f840046f8c45b1137f0289eeb0c980de72ea5e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 2 Nov 2014 15:18:29 -0800 Subject: [PATCH 1190/1492] [SPARK-3791][SQL] Provides Spark version and Hive version in HiveThriftServer2 This PR overrides the `GetInfo` Hive Thrift API to provide correct version information. Another property `spark.sql.hive.version` is added to reveal the underlying Hive version. These are generally useful for Spark SQL ODBC driver providers. The Spark version information is extracted from the jar manifest. Also took the chance to remove the `SET -v` hack, which was a workaround for Simba ODBC driver connectivity. TODO - [x] Find a general way to figure out Hive (or even any dependency) version. This [blog post](http://blog.soebes.de/blog/2014/01/02/version-information-into-your-appas-with-maven/) suggests several methods to inspect application version. In the case of Spark, this can be tricky because the chosen method: 1. must applies to both Maven build and SBT build For Maven builds, we can retrieve the version information from the META-INF/maven directory within the assembly jar. But this doesn't work for SBT builds. 2. must not rely on the original jars of dependencies to extract specific dependency version, because Spark uses assembly jar. This implies we can't read Hive version from Hive jar files since standard Spark distribution doesn't include them. 3. should play well with `SPARK_PREPEND_CLASSES` to ease local testing during development. `SPARK_PREPEND_CLASSES` prevents classes to be loaded from the assembly jar, thus we can't locate the jar file and read its manifest. Given these, maybe the only reliable method is to generate a source file containing version information at build time. pwendell Do you have any suggestions from the perspective of the build process? **Update** Hive version is now retrieved from the newly introduced `HiveShim` object. Author: Cheng Lian Author: Cheng Lian Closes #2843 from liancheng/get-info and squashes the following commits: a873d0f [Cheng Lian] Updates test case 53f43cd [Cheng Lian] Retrieves underlying Hive verson via HiveShim 1d282b8 [Cheng Lian] Removes the Simba ODBC "SET -v" hack f857fce [Cheng Lian] Overrides Hive GetInfo Thrift API and adds Hive version property --- .../scala/org/apache/spark/util/Utils.scala | 8 + .../apache/spark/sql/execution/commands.scala | 69 ++++----- .../thriftserver/SparkSQLCLIService.scala | 14 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 11 +- .../thriftserver/HiveThriftServer2Suite.scala | 144 +++++++++++++----- .../apache/spark/sql/hive/HiveContext.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 35 +---- 7 files changed, 173 insertions(+), 112 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 4e30d0d3813a2..b402c5f334bb0 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -20,8 +20,10 @@ package org.apache.spark.util import java.io._ import java.net._ import java.nio.ByteBuffer +import java.util.jar.Attributes.Name import java.util.{Properties, Locale, Random, UUID} import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.jar.{Manifest => JarManifest} import scala.collection.JavaConversions._ import scala.collection.Map @@ -1759,6 +1761,12 @@ private[spark] object Utils extends Logging { s"$libraryPathEnvName=$libraryPath$ampersand" } + lazy val sparkVersion = + SparkContext.jarOfObject(this).map { path => + val manifestUrl = new URL(s"jar:file:$path!/META-INF/MANIFEST.MF") + val manifest = new JarManifest(manifestUrl.openStream()) + manifest.getMainAttributes.getValue(Name.IMPLEMENTATION_VERSION) + }.getOrElse("Unknown") } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index e658e6fc4d5d5..f23b9c48cfb40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -84,50 +84,35 @@ case class SetCommand(kv: Option[(String, Option[String])], output: Seq[Attribut extends LeafNode with Command with Logging { override protected lazy val sideEffectResult: Seq[Row] = kv match { - // Set value for the key. - case Some((key, Some(value))) => - if (key == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { - logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + // Configures the deprecated "mapred.reduce.tasks" property. + case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, Some(value))) => + logWarning( + s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + s"automatically converted to ${SQLConf.SHUFFLE_PARTITIONS} instead.") - context.setConf(SQLConf.SHUFFLE_PARTITIONS, value) - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) - } else { - context.setConf(key, value) - Seq(Row(s"$key=$value")) - } - - // Query the value bound to the key. + context.setConf(SQLConf.SHUFFLE_PARTITIONS, value) + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=$value")) + + // Configures a single property. + case Some((key, Some(value))) => + context.setConf(key, value) + Seq(Row(s"$key=$value")) + + // Queries all key-value pairs that are set in the SQLConf of the context. Notice that different + // from Hive, here "SET -v" is an alias of "SET". (In Hive, "SET" returns all changed properties + // while "SET -v" returns all properties.) + case Some(("-v", None)) | None => + context.getAllConfs.map { case (k, v) => Row(s"$k=$v") }.toSeq + + // Queries the deprecated "mapred.reduce.tasks" property. + case Some((SQLConf.Deprecated.MAPRED_REDUCE_TASKS, None)) => + logWarning( + s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + + s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") + Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${context.numShufflePartitions}")) + + // Queries a single property. case Some((key, None)) => - // TODO (lian) This is just a workaround to make the Simba ODBC driver work. - // Should remove this once we get the ODBC driver updated. - if (key == "-v") { - val hiveJars = Seq( - "hive-exec-0.12.0.jar", - "hive-service-0.12.0.jar", - "hive-common-0.12.0.jar", - "hive-hwi-0.12.0.jar", - "hive-0.12.0.jar").mkString(":") - - context.getAllConfs.map { case (k, v) => - Row(s"$k=$v") - }.toSeq ++ Seq( - Row("system:java.class.path=" + hiveJars), - Row("system:sun.java.command=shark.SharkServer2")) - } else { - if (key == SQLConf.Deprecated.MAPRED_REDUCE_TASKS) { - logWarning(s"Property ${SQLConf.Deprecated.MAPRED_REDUCE_TASKS} is deprecated, " + - s"showing ${SQLConf.SHUFFLE_PARTITIONS} instead.") - Seq(Row(s"${SQLConf.SHUFFLE_PARTITIONS}=${context.numShufflePartitions}")) - } else { - Seq(Row(s"$key=${context.getConf(key, "")}")) - } - } - - // Query all key-value pairs that are set in the SQLConf of the context. - case _ => - context.getAllConfs.map { case (k, v) => - Row(s"$k=$v") - }.toSeq + Seq(Row(s"$key=${context.getConf(key, "")}")) } override def otherCopyArgs = context :: Nil diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index a78311fc48635..ecfb74473e921 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.thriftserver +import java.util.jar.Attributes.Name + import scala.collection.JavaConversions._ import java.io.IOException @@ -29,11 +31,12 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hive.service.Service.STATE import org.apache.hive.service.auth.HiveAuthFactory -import org.apache.hive.service.cli.CLIService +import org.apache.hive.service.cli._ import org.apache.hive.service.{AbstractService, Service, ServiceException} import org.apache.spark.sql.hive.HiveContext import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ +import org.apache.spark.util.Utils private[hive] class SparkSQLCLIService(hiveContext: HiveContext) extends CLIService @@ -60,6 +63,15 @@ private[hive] class SparkSQLCLIService(hiveContext: HiveContext) initCompositeService(hiveConf) } + + override def getInfo(sessionHandle: SessionHandle, getInfoType: GetInfoType): GetInfoValue = { + getInfoType match { + case GetInfoType.CLI_SERVER_NAME => new GetInfoValue("Spark SQL") + case GetInfoType.CLI_DBMS_NAME => new GetInfoValue("Spark SQL") + case GetInfoType.CLI_DBMS_VER => new GetInfoValue(Utils.sparkVersion) + case _ => super.getInfo(sessionHandle, getInfoType) + } + } } private[thriftserver] trait ReflectedCompositeService { this: AbstractService => diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 50425863518c3..89732c939b0ec 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -17,10 +17,11 @@ package org.apache.spark.sql.hive.thriftserver +import scala.collection.JavaConversions._ + import org.apache.spark.scheduler.StatsReportListener -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.sql.hive.{HiveShim, HiveContext} import org.apache.spark.{Logging, SparkConf, SparkContext} -import scala.collection.JavaConversions._ /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { @@ -31,8 +32,10 @@ private[hive] object SparkSQLEnv extends Logging { def init() { if (hiveContext == null) { - sparkContext = new SparkContext(new SparkConf() - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}")) + val sparkConf = new SparkConf() + .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}") + .set("spark.sql.hive.version", HiveShim.version) + sparkContext = new SparkContext(sparkConf) sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala index c60e8fa5b1259..65d910a0c3ffc 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suite.scala @@ -30,42 +30,95 @@ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver +import org.apache.hive.service.auth.PlainSaslHelper +import org.apache.hive.service.cli.GetInfoType +import org.apache.hive.service.cli.thrift.TCLIService.Client +import org.apache.hive.service.cli.thrift._ +import org.apache.thrift.protocol.TBinaryProtocol +import org.apache.thrift.transport.TSocket import org.scalatest.FunSuite import org.apache.spark.Logging import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.hive.HiveShim /** * Tests for the HiveThriftServer2 using JDBC. + * + * NOTE: SPARK_PREPEND_CLASSES is explicitly disabled in this test suite. Assembly jar must be + * rebuilt after changing HiveThriftServer2 related code. */ class HiveThriftServer2Suite extends FunSuite with Logging { Class.forName(classOf[HiveDriver].getCanonicalName) - def startThriftServerWithin(timeout: FiniteDuration = 1.minute)(f: Statement => Unit) { + def randomListeningPort = { + // Let the system to choose a random available port to avoid collision with other parallel + // builds. + val socket = new ServerSocket(0) + val port = socket.getLocalPort + socket.close() + port + } + + def withJdbcStatement(serverStartTimeout: FiniteDuration = 1.minute)(f: Statement => Unit) { + val port = randomListeningPort + + startThriftServer(port, serverStartTimeout) { + val jdbcUri = s"jdbc:hive2://${"localhost"}:$port/" + val user = System.getProperty("user.name") + val connection = DriverManager.getConnection(jdbcUri, user, "") + val statement = connection.createStatement() + + try { + f(statement) + } finally { + statement.close() + connection.close() + } + } + } + + def withCLIServiceClient( + serverStartTimeout: FiniteDuration = 1.minute)( + f: ThriftCLIServiceClient => Unit) { + val port = randomListeningPort + + startThriftServer(port) { + // Transport creation logics below mimics HiveConnection.createBinaryTransport + val rawTransport = new TSocket("localhost", port) + val user = System.getProperty("user.name") + val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) + val protocol = new TBinaryProtocol(transport) + val client = new ThriftCLIServiceClient(new Client(protocol)) + + transport.open() + + try { + f(client) + } finally { + transport.close() + } + } + } + + def startThriftServer( + port: Int, + serverStartTimeout: FiniteDuration = 1.minute)( + f: => Unit) { val startScript = "../../sbin/start-thriftserver.sh".split("/").mkString(File.separator) val stopScript = "../../sbin/stop-thriftserver.sh".split("/").mkString(File.separator) val warehousePath = getTempFilePath("warehouse") val metastorePath = getTempFilePath("metastore") val metastoreJdbcUri = s"jdbc:derby:;databaseName=$metastorePath;create=true" - val listeningHost = "localhost" - val listeningPort = { - // Let the system to choose a random available port to avoid collision with other parallel - // builds. - val socket = new ServerSocket(0) - val port = socket.getLocalPort - socket.close() - port - } - val command = s"""$startScript | --master local | --hiveconf hive.root.logger=INFO,console | --hiveconf ${ConfVars.METASTORECONNECTURLKEY}=$metastoreJdbcUri | --hiveconf ${ConfVars.METASTOREWAREHOUSE}=$warehousePath - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=$listeningHost - | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$listeningPort + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST}=${"localhost"} + | --hiveconf ${ConfVars.HIVE_SERVER2_THRIFT_PORT}=$port """.stripMargin.split("\\s+").toSeq val serverRunning = Promise[Unit]() @@ -92,31 +145,25 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } } - // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths - Process(command, None, "SPARK_TESTING" -> "0").run(ProcessLogger( + val env = Seq( + // Resets SPARK_TESTING to avoid loading Log4J configurations in testing class paths + "SPARK_TESTING" -> "0", + // Prevents loading classes out of the assembly jar. Otherwise Utils.sparkVersion can't read + // proper version information from the jar manifest. + "SPARK_PREPEND_CLASSES" -> "") + + Process(command, None, env: _*).run(ProcessLogger( captureThriftServerOutput("stdout"), captureThriftServerOutput("stderr"))) - val jdbcUri = s"jdbc:hive2://$listeningHost:$listeningPort/" - val user = System.getProperty("user.name") - try { - Await.result(serverRunning.future, timeout) - - val connection = DriverManager.getConnection(jdbcUri, user, "") - val statement = connection.createStatement() - - try { - f(statement) - } finally { - statement.close() - connection.close() - } + Await.result(serverRunning.future, serverStartTimeout) + f } catch { case cause: Exception => cause match { case _: TimeoutException => - logError(s"Failed to start Hive Thrift server within $timeout", cause) + logError(s"Failed to start Hive Thrift server within $serverStartTimeout", cause) case _ => } logError( @@ -125,8 +172,8 @@ class HiveThriftServer2Suite extends FunSuite with Logging { |HiveThriftServer2Suite failure output |===================================== |HiveThriftServer2 command line: ${command.mkString(" ")} - |JDBC URI: $jdbcUri - |User: $user + |Binding port: $port + |System user: ${System.getProperty("user.name")} | |${buffer.mkString("\n")} |========================================= @@ -146,7 +193,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } test("Test JDBC query execution") { - startThriftServerWithin() { statement => + withJdbcStatement() { statement => val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") @@ -168,7 +215,7 @@ class HiveThriftServer2Suite extends FunSuite with Logging { } test("SPARK-3004 regression: result set containing NULL") { - startThriftServerWithin() { statement => + withJdbcStatement() { statement => val dataFilePath = Thread.currentThread().getContextClassLoader.getResource( "data/files/small_kv_with_null.txt") @@ -191,4 +238,33 @@ class HiveThriftServer2Suite extends FunSuite with Logging { assert(!resultSet.next()) } } + + test("GetInfo Thrift API") { + withCLIServiceClient() { client => + val user = System.getProperty("user.name") + val sessionHandle = client.openSession(user, "") + + assertResult("Spark SQL", "Wrong GetInfo(CLI_DBMS_NAME) result") { + client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_NAME).getStringValue + } + + assertResult("Spark SQL", "Wrong GetInfo(CLI_SERVER_NAME) result") { + client.getInfo(sessionHandle, GetInfoType.CLI_SERVER_NAME).getStringValue + } + + assertResult(true, "Spark version shouldn't be \"Unknown\"") { + val version = client.getInfo(sessionHandle, GetInfoType.CLI_DBMS_VER).getStringValue + logInfo(s"Spark version: $version") + version != "Unknown" + } + } + } + + test("Checks Hive version") { + withJdbcStatement() { statement => + val resultSet = statement.executeQuery("SET spark.sql.hive.version") + resultSet.next() + assert(resultSet.getString(1) === s"spark.sql.hive.version=${HiveShim.version}") + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index dca5367f244de..0fe59f42f21ff 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -323,7 +323,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { driver.close() HiveShim.processResults(results) case _ => - sessionState.out.println(tokens(0) + " " + cmd_1) + if (sessionState.out != null) { + sessionState.out.println(tokens(0) + " " + cmd_1) + } Seq(proc.run(cmd_1).getResponseCode.toString) } } catch { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 5918f888c8f4c..b897dff0159ff 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -769,7 +769,7 @@ class HiveQuerySuite extends HiveComparisonTest { }.toSet clear() - // "set" itself returns all config variables currently specified in SQLConf. + // "SET" itself returns all config variables currently specified in SQLConf. // TODO: Should we be listing the default here always? probably... assert(sql("SET").collect().size == 0) @@ -778,44 +778,19 @@ class HiveQuerySuite extends HiveComparisonTest { } assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(sql("SET")) - } + assertResult(Set(testKey -> testVal))(collectResults(sql("SET"))) + assertResult(Set(testKey -> testVal))(collectResults(sql("SET -v"))) sql(s"SET ${testKey + testKey}=${testVal + testVal}") assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { collectResults(sql("SET")) } - - // "set key" - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey")) - } - - assertResult(Set(nonexistentKey -> "")) { - collectResults(sql(s"SET $nonexistentKey")) - } - - // Assert that sql() should have the same effects as sql() by repeating the above using sql(). - clear() - assert(sql("SET").collect().size == 0) - - assertResult(Set(testKey -> testVal)) { - collectResults(sql(s"SET $testKey=$testVal")) - } - - assert(hiveconf.get(testKey, "") == testVal) - assertResult(Set(testKey -> testVal)) { - collectResults(sql("SET")) - } - - sql(s"SET ${testKey + testKey}=${testVal + testVal}") - assert(hiveconf.get(testKey + testKey, "") == testVal + testVal) assertResult(Set(testKey -> testVal, (testKey + testKey) -> (testVal + testVal))) { - collectResults(sql("SET")) + collectResults(sql("SET -v")) } + // "SET key" assertResult(Set(testKey -> testVal)) { collectResults(sql(s"SET $testKey")) } From e749f5dedbad412430b86e7290085095f8dec0d1 Mon Sep 17 00:00:00 2001 From: wangfei Date: Sun, 2 Nov 2014 15:45:55 -0800 Subject: [PATCH 1191/1492] [SPARK-4191][SQL]move wrapperFor to HiveInspectors to reuse it Move wrapperFor in InsertIntoHiveTable to HiveInspectors to reuse them, this method can be reused when writing date with ObjectInspector(such as orc support) Author: wangfei Author: scwf Closes #3057 from scwf/reuse-wraperfor and squashes the following commits: 7ccf932 [scwf] fix conflicts d44f4da [wangfei] fix imports 9bf1b50 [wangfei] revert no related change 9a5276a [wangfei] move wrapfor to hiveinspector to reuse them --- .../spark/sql/hive/HiveInspectors.scala | 47 ++++++++++++++++++- .../hive/execution/InsertIntoHiveTable.scala | 44 +---------------- 2 files changed, 48 insertions(+), 43 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 1e2bf5cc4b0b4..58815daa82276 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive -import org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector._ @@ -114,6 +114,51 @@ private[hive] trait HiveInspectors { unwrap(si.getStructFieldData(data,r), r.getFieldObjectInspector)).toArray) } + + /** + * Wraps with Hive types based on object inspector. + * TODO: Consolidate all hive OI/data interface code. + */ + /** + * Wraps with Hive types based on object inspector. + * TODO: Consolidate all hive OI/data interface code. + */ + protected def wrapperFor(oi: ObjectInspector): Any => Any = oi match { + case _: JavaHiveVarcharObjectInspector => + (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) + + case _: JavaHiveDecimalObjectInspector => + (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toBigDecimal.underlying()) + + case soi: StandardStructObjectInspector => + val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) + (o: Any) => { + val struct = soi.create() + (soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[Row]).zipped.foreach { + (field, wrapper, data) => soi.setStructFieldData(struct, field, wrapper(data)) + } + struct + } + + case loi: ListObjectInspector => + val wrapper = wrapperFor(loi.getListElementObjectInspector) + (o: Any) => seqAsJavaList(o.asInstanceOf[Seq[_]].map(wrapper)) + + case moi: MapObjectInspector => + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map + + val keyWrapper = wrapperFor(moi.getMapKeyObjectInspector) + val valueWrapper = wrapperFor(moi.getMapValueObjectInspector) + (o: Any) => mapAsJavaMap(o.asInstanceOf[Map[_, _]].map { case (key, value) => + keyWrapper(key) -> valueWrapper(value) + }) + + case _ => + identity[Any] + } + /** * Converts native catalyst types to the types expected by Hive * @param a the value to be wrapped diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 92bc1c6625892..74b4e7aaa47a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import scala.collection.JavaConversions._ -import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} +import org.apache.hadoop.hive.common.`type`.HiveVarchar import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.metastore.MetaStoreUtils @@ -52,7 +52,7 @@ case class InsertIntoHiveTable( child: SparkPlan, overwrite: Boolean) (@transient sc: HiveContext) - extends UnaryNode with Command { + extends UnaryNode with Command with HiveInspectors { @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @transient private lazy val hiveContext = new Context(sc.hiveconf) @@ -68,46 +68,6 @@ case class InsertIntoHiveTable( def output = child.output - /** - * Wraps with Hive types based on object inspector. - * TODO: Consolidate all hive OI/data interface code. - */ - protected def wrapperFor(oi: ObjectInspector): Any => Any = oi match { - case _: JavaHiveVarcharObjectInspector => - (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) - - case _: JavaHiveDecimalObjectInspector => - (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toBigDecimal.underlying()) - - case soi: StandardStructObjectInspector => - val wrappers = soi.getAllStructFieldRefs.map(ref => wrapperFor(ref.getFieldObjectInspector)) - (o: Any) => { - val struct = soi.create() - (soi.getAllStructFieldRefs, wrappers, o.asInstanceOf[Row]).zipped.foreach { - (field, wrapper, data) => soi.setStructFieldData(struct, field, wrapper(data)) - } - struct - } - - case loi: ListObjectInspector => - val wrapper = wrapperFor(loi.getListElementObjectInspector) - (o: Any) => seqAsJavaList(o.asInstanceOf[Seq[_]].map(wrapper)) - - case moi: MapObjectInspector => - // The Predef.Map is scala.collection.immutable.Map. - // Since the map values can be mutable, we explicitly import scala.collection.Map at here. - import scala.collection.Map - - val keyWrapper = wrapperFor(moi.getMapKeyObjectInspector) - val valueWrapper = wrapperFor(moi.getMapValueObjectInspector) - (o: Any) => mapAsJavaMap(o.asInstanceOf[Map[_, _]].map { case (key, value) => - keyWrapper(key) -> valueWrapper(value) - }) - - case _ => - identity[Any] - } - def saveAsHiveFile( rdd: RDD[Row], valueClass: Class[_], From 06232d23ff2a6344c49fff81364d9f6b02af326b Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sun, 2 Nov 2014 15:46:56 -0800 Subject: [PATCH 1192/1492] [SPARK-4185][SQL] JSON schema inference failed when dealing with type conflicts in arrays JIRA: https://issues.apache.org/jira/browse/SPARK-4185. This PR also has the fix of #3052. Author: Yin Huai Closes #3056 from yhuai/SPARK-4185 and squashes the following commits: ed3a5a8 [Yin Huai] Correctly handle type conflicts between structs and primitive types in an array. --- .../org/apache/spark/sql/json/JsonRDD.scala | 16 +++++++++++----- .../org/apache/spark/sql/json/JsonSuite.scala | 9 ++++++--- .../org/apache/spark/sql/json/TestJsonData.scala | 4 +++- 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 5bb6f6c85d801..0f2dcdcacf0ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -73,16 +73,18 @@ private[sql] object JsonRDD extends Logging { def makeStruct(values: Seq[Seq[String]], prefix: Seq[String]): StructType = { val (topLevel, structLike) = values.partition(_.size == 1) + val topLevelFields = topLevel.filter { name => resolved.get(prefix ++ name).get match { case ArrayType(elementType, _) => { def hasInnerStruct(t: DataType): Boolean = t match { - case s: StructType => false + case s: StructType => true case ArrayType(t1, _) => hasInnerStruct(t1) - case o => true + case o => false } - hasInnerStruct(elementType) + // Check if this array has inner struct. + !hasInnerStruct(elementType) } case struct: StructType => false case _ => true @@ -90,8 +92,11 @@ private[sql] object JsonRDD extends Logging { }.map { a => StructField(a.head, resolved.get(prefix ++ a).get, nullable = true) } + val topLevelFieldNameSet = topLevelFields.map(_.name) - val structFields: Seq[StructField] = structLike.groupBy(_(0)).map { + val structFields: Seq[StructField] = structLike.groupBy(_(0)).filter { + case (name, _) => !topLevelFieldNameSet.contains(name) + }.map { case (name, fields) => { val nestedFields = fields.map(_.tail) val structType = makeStruct(nestedFields, prefix :+ name) @@ -354,7 +359,8 @@ private[sql] object JsonRDD extends Logging { case (key, value) => if (count > 0) builder.append(",") count += 1 - builder.append(s"""\"${key}\":${toString(value)}""") + val stringValue = if (value.isInstanceOf[String]) s"""\"$value\"""" else toString(value) + builder.append(s"""\"${key}\":${stringValue}""") } builder.append("}") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 362c7e1a52482..4b851d1b96152 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -483,7 +483,8 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("array1", ArrayType(StringType, true), true) :: StructField("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil), false), true) :: Nil) + StructField("field", LongType, true) :: Nil), false), true) :: + StructField("array3", ArrayType(StringType, false), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -492,12 +493,14 @@ class JsonSuite extends QueryTest { checkAnswer( sql("select * from jsonTable"), Seq(Seq("1", "1.1", "true", null, "[]", "{}", "[2,3,4]", - """{"field":str}"""), Seq(Seq(214748364700L), Seq(1))) :: Nil + """{"field":"str"}"""), Seq(Seq(214748364700L), Seq(1)), null) :: + Seq(null, null, Seq("""{"field":"str"}""", """{"field":1}""")) :: + Seq(null, null, Seq("1", "2", "3")) :: Nil ) // Treat an element as a number. checkAnswer( - sql("select array1[0] + 1 from jsonTable"), + sql("select array1[0] + 1 from jsonTable where array1 is not null"), 2 ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index c204162dd2fc1..e5773a55875bc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -57,7 +57,9 @@ object TestJsonData { val arrayElementTypeConflict = TestSQLContext.sparkContext.parallelize( """{"array1": [1, 1.1, true, null, [], {}, [2,3,4], {"field":"str"}], - "array2": [{"field":214748364700}, {"field":1}]}""" :: Nil) + "array2": [{"field":214748364700}, {"field":1}]}""" :: + """{"array3": [{"field":"str"}, {"field":1}]}""" :: + """{"array3": [1, 2, 3]}""" :: Nil) val missingFields = TestSQLContext.sparkContext.parallelize( From 9081b9f9f79b78f0b20a5fc3bc4e7c1d3e717130 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 2 Nov 2014 16:00:24 -0800 Subject: [PATCH 1193/1492] [SPARK-2189][SQL] Adds dropTempTable API This PR adds an API for unregistering temporary tables. If a temporary table has been cached before, it's unpersisted as well. Author: Cheng Lian Closes #3039 from liancheng/unregister-temp-table and squashes the following commits: 54ae99f [Cheng Lian] Fixes Scala styling issue 1948c14 [Cheng Lian] Removes the unpersist argument aca41d3 [Cheng Lian] Ensures thread safety 7d4fb2b [Cheng Lian] Adds unregisterTempTable API --- .../org/apache/spark/sql/CacheManager.scala | 13 ++++++++++++ .../org/apache/spark/sql/SQLContext.scala | 13 ++++++++++++ .../apache/spark/sql/CachedTableSuite.scala | 20 +++++++++++++++++++ 3 files changed, 46 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index 3ced11a5e6c11..2e7abac1f1bdb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -103,6 +103,19 @@ private[sql] trait CacheManager { cachedData.remove(dataIndex) } + /** Tries to remove the data for the given SchemaRDD from the cache if it's cached */ + private[sql] def tryUncacheQuery( + query: SchemaRDD, + blocking: Boolean = true): Boolean = writeLock { + val planToCache = query.queryExecution.analyzed + val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) + val found = dataIndex >= 0 + if (found) { + cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData.remove(dataIndex) + } + found + } /** Optionally returns cached data for the given SchemaRDD */ private[sql] def lookupCachedData(query: SchemaRDD): Option[CachedData] = readLock { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4cded98c803f8..3cf6af5f7a3c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -276,6 +276,19 @@ class SQLContext(@transient val sparkContext: SparkContext) catalog.registerTable(None, tableName, rdd.queryExecution.logical) } + /** + * Drops the temporary table with the given table name in the catalog. If the table has been + * cached/persisted before, it's also unpersisted. + * + * @param tableName the name of the table to be unregistered. + * + * @group userf + */ + def dropTempTable(tableName: String): Unit = { + tryUncacheQuery(table(tableName)) + catalog.unregisterTable(None, tableName) + } + /** * Executes a SQL query using Spark, returning the result as a SchemaRDD. The dialect that is * used for SQL parsing can be configured with 'spark.sql.dialect'. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 44a2961b27eda..765fa82776341 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -231,4 +231,24 @@ class CachedTableSuite extends QueryTest { assert(cached.statistics.sizeInBytes === actualSizeInBytes) } } + + test("Drops temporary table") { + testData.select('key).registerTempTable("t1") + table("t1") + dropTempTable("t1") + assert(intercept[RuntimeException](table("t1")).getMessage.startsWith("Table Not Found")) + } + + test("Drops cached temporary table") { + testData.select('key).registerTempTable("t1") + testData.select('key).registerTempTable("t2") + cacheTable("t1") + + assert(isCached("t1")) + assert(isCached("t2")) + + dropTempTable("t1") + assert(intercept[RuntimeException](table("t1")).getMessage.startsWith("Table Not Found")) + assert(!isCached("t2")) + } } From 2ebd1df3f17993f3cb472ec44c8832213976d99a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 2 Nov 2014 16:26:24 -0800 Subject: [PATCH 1194/1492] [SPARK-4183] Close transport-related resources between SparkContexts A leak of event loops may be causing test failures. Author: Aaron Davidson Closes #3053 from aarondav/leak and squashes the following commits: e676d18 [Aaron Davidson] Typo! 8f96475 [Aaron Davidson] Keep original ssc semantics 7e49f10 [Aaron Davidson] A leak of event loops may be causing test failures. --- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../netty/NettyBlockTransferService.scala | 5 ++- .../apache/spark/storage/BlockManager.scala | 4 +++ .../ExecutorAllocationManagerSuite.scala | 34 +++++++++---------- .../apache/spark/MapOutputTrackerSuite.scala | 21 +++++++++++- .../SparkContextSchedulerCreationSuite.scala | 6 ++-- .../flume/FlumePollingStreamSuite.scala | 15 +++++--- .../client/TransportClientFactory.java | 3 +- .../spark/network/server/TransportServer.java | 5 +-- .../shuffle/ExternalShuffleClient.java | 7 ++++ .../spark/network/shuffle/ShuffleClient.java | 4 ++- .../streaming/StreamingContextSuite.scala | 4 +++ 12 files changed, 78 insertions(+), 32 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 7fb2b91377076..e2f13accdfab5 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -274,7 +274,7 @@ object SparkEnv extends Logging { val shuffleMemoryManager = new ShuffleMemoryManager(conf) val blockTransferService = - conf.get("spark.shuffle.blockTransferService", "nio").toLowerCase match { + conf.get("spark.shuffle.blockTransferService", "netty").toLowerCase match { case "netty" => new NettyBlockTransferService(conf) case "nio" => diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index ec3000e722a95..1c4327cf13b51 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -106,5 +106,8 @@ class NettyBlockTransferService(conf: SparkConf) extends BlockTransferService { result.future } - override def close(): Unit = server.close() + override def close(): Unit = { + server.close() + clientFactory.close() + } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1f8de2896116c..5f5dd0dc1c63f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1178,6 +1178,10 @@ private[spark] class BlockManager( def stop(): Unit = { blockTransferService.close() + if (shuffleClient ne blockTransferService) { + // Closing should be idempotent, but maybe not for the NioBlockTransferService. + shuffleClient.close() + } diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index f0aa914cfedb6..66cf60d25f6d1 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.BlockManagerId /** * Test add and remove behavior of ExecutorAllocationManager. */ -class ExecutorAllocationManagerSuite extends FunSuite { +class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { import ExecutorAllocationManager._ import ExecutorAllocationManagerSuite._ @@ -36,17 +36,21 @@ class ExecutorAllocationManagerSuite extends FunSuite { .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") intercept[SparkException] { new SparkContext(conf) } + SparkEnv.get.stop() // cleanup the created environment // Only min val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "1") intercept[SparkException] { new SparkContext(conf1) } + SparkEnv.get.stop() // Only max val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "2") intercept[SparkException] { new SparkContext(conf2) } + SparkEnv.get.stop() // Both min and max, but min > max intercept[SparkException] { createSparkContext(2, 1) } + SparkEnv.get.stop() // Both min and max, and min == max val sc1 = createSparkContext(1, 1) @@ -60,18 +64,17 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("starting state") { - val sc = createSparkContext() + sc = createSparkContext() val manager = sc.executorAllocationManager.get assert(numExecutorsPending(manager) === 0) assert(executorsPendingToRemove(manager).isEmpty) assert(executorIds(manager).isEmpty) assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) assert(removeTimes(manager).isEmpty) - sc.stop() } test("add executors") { - val sc = createSparkContext(1, 10) + sc = createSparkContext(1, 10) val manager = sc.executorAllocationManager.get // Keep adding until the limit is reached @@ -112,11 +115,10 @@ class ExecutorAllocationManagerSuite extends FunSuite { assert(addExecutors(manager) === 0) assert(numExecutorsPending(manager) === 6) assert(numExecutorsToAdd(manager) === 1) - sc.stop() } test("remove executors") { - val sc = createSparkContext(5, 10) + sc = createSparkContext(5, 10) val manager = sc.executorAllocationManager.get (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } @@ -163,11 +165,10 @@ class ExecutorAllocationManagerSuite extends FunSuite { assert(executorsPendingToRemove(manager).isEmpty) assert(!removeExecutor(manager, "8")) assert(executorsPendingToRemove(manager).isEmpty) - sc.stop() } test ("interleaving add and remove") { - val sc = createSparkContext(5, 10) + sc = createSparkContext(5, 10) val manager = sc.executorAllocationManager.get // Add a few executors @@ -232,11 +233,10 @@ class ExecutorAllocationManagerSuite extends FunSuite { onExecutorAdded(manager, "15") onExecutorAdded(manager, "16") assert(executorIds(manager).size === 10) - sc.stop() } test("starting/canceling add timer") { - val sc = createSparkContext(2, 10) + sc = createSparkContext(2, 10) val clock = new TestClock(8888L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -268,7 +268,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("starting/canceling remove timers") { - val sc = createSparkContext(2, 10) + sc = createSparkContext(2, 10) val clock = new TestClock(14444L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -313,7 +313,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("mock polling loop with no events") { - val sc = createSparkContext(1, 20) + sc = createSparkContext(1, 20) val manager = sc.executorAllocationManager.get val clock = new TestClock(2020L) manager.setClock(clock) @@ -339,7 +339,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("mock polling loop add behavior") { - val sc = createSparkContext(1, 20) + sc = createSparkContext(1, 20) val clock = new TestClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -388,7 +388,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("mock polling loop remove behavior") { - val sc = createSparkContext(1, 20) + sc = createSparkContext(1, 20) val clock = new TestClock(2020L) val manager = sc.executorAllocationManager.get manager.setClock(clock) @@ -449,7 +449,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("listeners trigger add executors correctly") { - val sc = createSparkContext(2, 10) + sc = createSparkContext(2, 10) val manager = sc.executorAllocationManager.get assert(addTime(manager) === NOT_SET) @@ -479,7 +479,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("listeners trigger remove executors correctly") { - val sc = createSparkContext(2, 10) + sc = createSparkContext(2, 10) val manager = sc.executorAllocationManager.get assert(removeTimes(manager).isEmpty) @@ -510,7 +510,7 @@ class ExecutorAllocationManagerSuite extends FunSuite { } test("listeners trigger add and remove executor callbacks correctly") { - val sc = createSparkContext(2, 10) + sc = createSparkContext(2, 10) val manager = sc.executorAllocationManager.get assert(executorIds(manager).isEmpty) assert(removeTimes(manager).isEmpty) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index cbc0bd178d894..d27880f4bc32f 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.AkkaUtils -class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { +class MapOutputTrackerSuite extends FunSuite { private val conf = new SparkConf test("master start and stop") { @@ -37,6 +37,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.stop() + actorSystem.shutdown() } test("master register shuffle and fetch") { @@ -56,6 +57,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { assert(statuses.toSeq === Seq((BlockManagerId("a", "hostA", 1000), size1000), (BlockManagerId("b", "hostB", 1000), size10000))) tracker.stop() + actorSystem.shutdown() } test("master register and unregister shuffle") { @@ -74,6 +76,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.unregisterShuffle(10) assert(!tracker.containsShuffle(10)) assert(tracker.getServerStatuses(10, 0).isEmpty) + + tracker.stop() + actorSystem.shutdown() } test("master register shuffle and unregister map output and fetch") { @@ -97,6 +102,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // this should cause it to fail, and the scheduler will ignore the failure due to the // stage already being aborted. intercept[FetchFailedException] { tracker.getServerStatuses(10, 1) } + + tracker.stop() + actorSystem.shutdown() } test("remote fetch") { @@ -136,6 +144,11 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // failure should be cached intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } + + masterTracker.stop() + slaveTracker.stop() + actorSystem.shutdown() + slaveSystem.shutdown() } test("remote fetch below akka frame size") { @@ -154,6 +167,9 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { masterTracker.registerMapOutput(10, 0, MapStatus( BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) masterActor.receive(GetMapOutputStatuses(10)) + +// masterTracker.stop() // this throws an exception + actorSystem.shutdown() } test("remote fetch exceeds akka frame size") { @@ -176,5 +192,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } + +// masterTracker.stop() // this throws an exception + actorSystem.shutdown() } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index df237ba796b38..0390a2e4f1dbb 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.scalatest.{BeforeAndAfterEach, FunSuite, PrivateMethodTester} +import org.scalatest.{FunSuite, PrivateMethodTester} import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} @@ -25,12 +25,12 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite - extends FunSuite with PrivateMethodTester with Logging with BeforeAndAfterEach { + extends FunSuite with LocalSparkContext with PrivateMethodTester with Logging { def createTaskScheduler(master: String): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - val sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala index 32a19787a28e1..475026e8eb140 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala @@ -145,11 +145,16 @@ class FlumePollingStreamSuite extends TestSuiteBase { outputStream.register() ssc.start() - writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) - assertChannelIsEmpty(channel) - assertChannelIsEmpty(channel2) - sink.stop() - channel.stop() + try { + writeAndVerify(Seq(channel, channel2), ssc, outputBuffer) + assertChannelIsEmpty(channel) + assertChannelIsEmpty(channel2) + } finally { + sink.stop() + sink2.stop() + channel.stop() + channel2.stop() + } } def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext, diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index e7fa4f6bf3b5d..0b4a1d8286407 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -58,7 +58,7 @@ public class TransportClientFactory implements Closeable { private final ConcurrentHashMap connectionPool; private final Class socketChannelClass; - private final EventLoopGroup workerGroup; + private EventLoopGroup workerGroup; public TransportClientFactory(TransportContext context) { this.context = context; @@ -150,6 +150,7 @@ public void close() { if (workerGroup != null) { workerGroup.shutdownGracefully(); + workerGroup = null; } } diff --git a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java index d1a1877a98f46..70da48ca8ee79 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/network/common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -49,6 +49,7 @@ public class TransportServer implements Closeable { private ChannelFuture channelFuture; private int port = -1; + /** Creates a TransportServer that binds to the given port, or to any available if 0. */ public TransportServer(TransportContext context, int portToBind) { this.context = context; this.conf = context.getConf(); @@ -67,7 +68,7 @@ private void init(int portToBind) { IOMode ioMode = IOMode.valueOf(conf.ioMode()); EventLoopGroup bossGroup = - NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); + NettyUtils.createEventLoop(ioMode, conf.serverThreads(), "shuffle-server"); EventLoopGroup workerGroup = bossGroup; bootstrap = new ServerBootstrap() @@ -105,7 +106,7 @@ protected void initChannel(SocketChannel ch) throws Exception { @Override public void close() { if (channelFuture != null) { - // close is a local operation and should finish with milliseconds; timeout just to be safe + // close is a local operation and should finish within milliseconds; timeout just to be safe channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS); channelFuture = null; } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index cc2f6261ca302..6bbabc44b958b 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -17,6 +17,8 @@ package org.apache.spark.network.shuffle; +import java.io.Closeable; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,4 +87,9 @@ public void registerWithShuffleServer( JavaUtils.serialize(new RegisterExecutor(appId, execId, executorInfo)); client.sendRpcSync(registerExecutorMessage, 5000 /* timeoutMs */); } + + @Override + public void close() { + clientFactory.close(); + } } diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java index 9fa87c2c6e1c2..d46a562394557 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -17,8 +17,10 @@ package org.apache.spark.network.shuffle; +import java.io.Closeable; + /** Provides an interface for reading shuffle files, either from an Executor or external service. */ -public interface ShuffleClient { +public interface ShuffleClient extends Closeable { /** * Fetch a sequence of blocks from a remote node asynchronously, * diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 655cec1573f58..f47772947d67c 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -46,6 +46,10 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w after { if (ssc != null) { ssc.stop() + if (ssc.sc != null) { + // Calling ssc.stop() does not always stop the associated SparkContext. + ssc.sc.stop() + } ssc = null } if (sc != null) { From ebd6480587f96e9964d37157253523e0a179171a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 2 Nov 2014 17:55:55 -0800 Subject: [PATCH 1195/1492] [SPARK-3572] [SQL] Internal API for User-Defined Types This PR adds User-Defined Types (UDTs) to SQL. It is a precursor to using SchemaRDD as a Dataset for the new MLlib API. Currently, the UDT API is private since there is incomplete support (e.g., no Java or Python support yet). Author: Joseph K. Bradley Author: Michael Armbrust Author: Xiangrui Meng Closes #3063 from marmbrus/udts and squashes the following commits: 7ccfc0d [Michael Armbrust] remove println 46a3aee [Michael Armbrust] Slightly easier to read test output. 6cc434d [Michael Armbrust] Recursively convert rows. e369b91 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udts 15c10a6 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into sql-udt2 f3c72fe [Joseph K. Bradley] Fixing merge e13cd8a [Joseph K. Bradley] Removed Vector UDTs 5817b2b [Joseph K. Bradley] style edits 30ce5b2 [Joseph K. Bradley] updates based on code review d063380 [Joseph K. Bradley] Cleaned up Java UDT Suite, and added warning about element ordering when creating schema from Java Bean a571bb6 [Joseph K. Bradley] Removed old UDT code (registry and Java UDTs). Cleaned up other code. Extended JavaUserDefinedTypeSuite 6fddc1c [Joseph K. Bradley] Made MyLabeledPoint into a Java Bean 20630bc [Joseph K. Bradley] fixed scalastyle fa86b20 [Joseph K. Bradley] Removed Java UserDefinedType, and made UDTs private[spark] for now 8de957c [Joseph K. Bradley] Modified UserDefinedType to store Java class of user type so that registerUDT takes only the udt argument. 8b242ea [Joseph K. Bradley] Fixed merge error after last merge. Note: Last merge commit also removed SQL UDT examples from mllib. 7f29656 [Joseph K. Bradley] Moved udt case to top of all matches. Small cleanups b028675 [Xiangrui Meng] allow any type in UDT 4500d8a [Xiangrui Meng] update example code 87264a5 [Xiangrui Meng] remove debug code 3143ac3 [Xiangrui Meng] remove unnecessary changes cfbc321 [Xiangrui Meng] support UDT in parquet db16139 [Joseph K. Bradley] Added more doc for UserDefinedType. Removed unused code in Suite 759af7a [Joseph K. Bradley] Added more doc to UserDefineType 63626a4 [Joseph K. Bradley] Updated ScalaReflectionsSuite per @marmbrus suggestions 51e5282 [Joseph K. Bradley] fixed 1 test f025035 [Joseph K. Bradley] Cleanups before PR. Added new tests 85872f6 [Michael Armbrust] Allow schema calculation to be lazy, but ensure its available on executors. dff99d6 [Joseph K. Bradley] Added UDTs for Vectors in MLlib, plus DatasetExample using the UDTs cd60cb4 [Joseph K. Bradley] Trying to get other SQL tests to run 34a5831 [Joseph K. Bradley] Added MLlib dependency on SQL. e1f7b9c [Joseph K. Bradley] blah 2f40c02 [Joseph K. Bradley] renamed UDT types 3579035 [Joseph K. Bradley] udt annotation now working b226b9e [Joseph K. Bradley] Changing UDT to annotation fea04af [Joseph K. Bradley] more cleanups 964b32e [Joseph K. Bradley] some cleanups 893ee4c [Joseph K. Bradley] udt finallly working 50f9726 [Joseph K. Bradley] udts 04303c9 [Joseph K. Bradley] udts 39f8707 [Joseph K. Bradley] removed old udt suite 273ac96 [Joseph K. Bradley] basic UDT is working, but deserialization has yet to be done 8bebf24 [Joseph K. Bradley] commented out convertRowToScala for debugging 53de70f [Joseph K. Bradley] more udts... 982c035 [Joseph K. Bradley] still working on UDTs 19b2f60 [Joseph K. Bradley] still working on UDTs 0eaeb81 [Joseph K. Bradley] Still working on UDTs 105c5a3 [Joseph K. Bradley] Adding UserDefinedType to SQL, not done yet. --- .../spark/sql/catalyst/ScalaReflection.scala | 155 +++++++++++------- .../annotation/SQLUserDefinedType.java | 46 ++++++ .../sql/catalyst/expressions/ScalaUdf.scala | 6 +- .../spark/sql/catalyst/types/dataTypes.scala | 53 +++++- .../sql/catalyst/ScalaReflectionSuite.scala | 13 +- .../spark/sql/api/java/UserDefinedType.java | 53 ++++++ .../org/apache/spark/sql/SQLContext.scala | 6 +- .../org/apache/spark/sql/SchemaRDD.scala | 30 ++-- .../org/apache/spark/sql/SchemaRDDLike.scala | 2 +- .../apache/spark/sql/UdfRegistration.scala | 46 +++--- .../spark/sql/api/java/JavaSQLContext.scala | 29 +++- .../spark/sql/api/java/UDTWrappers.scala | 75 +++++++++ .../spark/sql/execution/ExistingRDD.scala | 11 +- .../spark/sql/execution/SparkPlan.scala | 5 +- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../spark/sql/execution/basicOperators.scala | 6 +- .../spark/sql/parquet/ParquetConverter.scala | 13 +- .../sql/parquet/ParquetTableSupport.scala | 3 +- .../spark/sql/parquet/ParquetTypes.scala | 3 + .../sql/types/util/DataTypeConversions.scala | 22 ++- .../api/java/JavaUserDefinedTypeSuite.java | 88 ++++++++++ .../spark/sql/UserDefinedTypeSuite.scala | 83 ++++++++++ .../org/apache/spark/sql/json/JsonSuite.scala | 11 +- .../apache/spark/sql/hive/HiveContext.scala | 4 +- 24 files changed, 620 insertions(+), 146 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala create mode 100644 sql/core/src/test/java/org/apache/spark/sql/api/java/JavaUserDefinedTypeSuite.java create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 8fbdf664b71e4..9cda373623cb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} +import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType import org.apache.spark.sql.catalyst.expressions.{GenericRow, Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.types._ @@ -35,25 +37,46 @@ object ScalaReflection { case class Schema(dataType: DataType, nullable: Boolean) - /** Converts Scala objects to catalyst rows / types */ - def convertToCatalyst(a: Any): Any = a match { - case o: Option[_] => o.map(convertToCatalyst).orNull - case s: Seq[_] => s.map(convertToCatalyst) - case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } - case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) - case d: BigDecimal => Decimal(d) - case other => other + /** + * Converts Scala objects to catalyst rows / types. + * Note: This is always called after schemaFor has been called. + * This ordering is important for UDT registration. + */ + def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (obj, udt: UserDefinedType[_]) => udt.serialize(obj) + case (o: Option[_], _) => o.map(convertToCatalyst(_, dataType)).orNull + case (s: Seq[_], arrayType: ArrayType) => s.map(convertToCatalyst(_, arrayType.elementType)) + case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => + convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, mapType.valueType) + } + case (p: Product, structType: StructType) => + new GenericRow( + p.productIterator.toSeq.zip(structType.fields).map { case (elem, field) => + convertToCatalyst(elem, field.dataType) + }.toArray) + case (d: BigDecimal, _) => Decimal(d) + case (other, _) => other } /** Converts Catalyst types used internally in rows to standard Scala types */ - def convertToScala(a: Any): Any = a match { - case s: Seq[_] => s.map(convertToScala) - case m: Map[_, _] => m.map { case (k, v) => convertToScala(k) -> convertToScala(v) } - case d: Decimal => d.toBigDecimal - case other => other + def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (d, udt: UserDefinedType[_]) => udt.deserialize(d) + case (s: Seq[_], arrayType: ArrayType) => s.map(convertToScala(_, arrayType.elementType)) + case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => + convertToScala(k, mapType.keyType) -> convertToScala(v, mapType.valueType) + } + case (r: Row, s: StructType) => convertRowToScala(r, s) + case (d: Decimal, _: DecimalType) => d.toBigDecimal + case (other, _) => other } - def convertRowToScala(r: Row): Row = new GenericRow(r.toArray.map(convertToScala)) + def convertRowToScala(r: Row, schema: StructType): Row = { + new GenericRow( + r.zip(schema.fields.map(_.dataType)) + .map(r_dt => convertToScala(r_dt._1, r_dt._2)).toArray) + } /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { @@ -65,52 +88,64 @@ object ScalaReflection { def schemaFor[T: TypeTag]: Schema = schemaFor(typeOf[T]) /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ - def schemaFor(tpe: `Type`): Schema = tpe match { - case t if t <:< typeOf[Option[_]] => - val TypeRef(_, _, Seq(optType)) = t - Schema(schemaFor(optType).dataType, nullable = true) - case t if t <:< typeOf[Product] => - val formalTypeArgs = t.typeSymbol.asClass.typeParams - val TypeRef(_, _, actualTypeArgs) = t - val params = t.member(nme.CONSTRUCTOR).asMethod.paramss - Schema(StructType( - params.head.map { p => - val Schema(dataType, nullable) = - schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) - StructField(p.name.toString, dataType, nullable) - }), nullable = true) - // Need to decide if we actually need a special type here. - case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) - case t if t <:< typeOf[Array[_]] => - sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") - case t if t <:< typeOf[Seq[_]] => - val TypeRef(_, _, Seq(elementType)) = t - val Schema(dataType, nullable) = schemaFor(elementType) - Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< typeOf[Map[_,_]] => - val TypeRef(_, _, Seq(keyType, valueType)) = t - val Schema(valueDataType, valueNullable) = schemaFor(valueType) - Schema(MapType(schemaFor(keyType).dataType, - valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) - case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) - case t if t <:< typeOf[Date] => Schema(DateType, nullable = true) - case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) - case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) - case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) - case t if t <:< typeOf[java.lang.Double] => Schema(DoubleType, nullable = true) - case t if t <:< typeOf[java.lang.Float] => Schema(FloatType, nullable = true) - case t if t <:< typeOf[java.lang.Short] => Schema(ShortType, nullable = true) - case t if t <:< typeOf[java.lang.Byte] => Schema(ByteType, nullable = true) - case t if t <:< typeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) - case t if t <:< definitions.IntTpe => Schema(IntegerType, nullable = false) - case t if t <:< definitions.LongTpe => Schema(LongType, nullable = false) - case t if t <:< definitions.DoubleTpe => Schema(DoubleType, nullable = false) - case t if t <:< definitions.FloatTpe => Schema(FloatType, nullable = false) - case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) - case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) - case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) + def schemaFor(tpe: `Type`): Schema = { + val className: String = tpe.erasure.typeSymbol.asClass.fullName + tpe match { + case t if Utils.classIsLoadable(className) && + Utils.classForName(className).isAnnotationPresent(classOf[SQLUserDefinedType]) => + // Note: We check for classIsLoadable above since Utils.classForName uses Java reflection, + // whereas className is from Scala reflection. This can make it hard to find classes + // in some cases, such as when a class is enclosed in an object (in which case + // Java appends a '$' to the object name but Scala does not). + val udt = Utils.classForName(className) + .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + Schema(udt, nullable = true) + case t if t <:< typeOf[Option[_]] => + val TypeRef(_, _, Seq(optType)) = t + Schema(schemaFor(optType).dataType, nullable = true) + case t if t <:< typeOf[Product] => + val formalTypeArgs = t.typeSymbol.asClass.typeParams + val TypeRef(_, _, actualTypeArgs) = t + val params = t.member(nme.CONSTRUCTOR).asMethod.paramss + Schema(StructType( + params.head.map { p => + val Schema(dataType, nullable) = + schemaFor(p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs)) + StructField(p.name.toString, dataType, nullable) + }), nullable = true) + // Need to decide if we actually need a special type here. + case t if t <:< typeOf[Array[Byte]] => Schema(BinaryType, nullable = true) + case t if t <:< typeOf[Array[_]] => + sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") + case t if t <:< typeOf[Seq[_]] => + val TypeRef(_, _, Seq(elementType)) = t + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) + case t if t <:< typeOf[Map[_, _]] => + val TypeRef(_, _, Seq(keyType, valueType)) = t + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + Schema(MapType(schemaFor(keyType).dataType, + valueDataType, valueContainsNull = valueNullable), nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) + case t if t <:< typeOf[Date] => Schema(DateType, nullable = true) + case t if t <:< typeOf[BigDecimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< typeOf[Decimal] => Schema(DecimalType.Unlimited, nullable = true) + case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) + case t if t <:< typeOf[java.lang.Long] => Schema(LongType, nullable = true) + case t if t <:< typeOf[java.lang.Double] => Schema(DoubleType, nullable = true) + case t if t <:< typeOf[java.lang.Float] => Schema(FloatType, nullable = true) + case t if t <:< typeOf[java.lang.Short] => Schema(ShortType, nullable = true) + case t if t <:< typeOf[java.lang.Byte] => Schema(ByteType, nullable = true) + case t if t <:< typeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) + case t if t <:< definitions.IntTpe => Schema(IntegerType, nullable = false) + case t if t <:< definitions.LongTpe => Schema(LongType, nullable = false) + case t if t <:< definitions.DoubleTpe => Schema(DoubleType, nullable = false) + case t if t <:< definitions.FloatTpe => Schema(FloatType, nullable = false) + case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) + case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) + case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) + } } def typeOfObject: PartialFunction[Any, DataType] = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java new file mode 100644 index 0000000000000..e966aeea1cb23 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/annotation/SQLUserDefinedType.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.annotation; + +import java.lang.annotation.*; + +import org.apache.spark.annotation.DeveloperApi; +import org.apache.spark.sql.catalyst.types.UserDefinedType; + +/** + * ::DeveloperApi:: + * A user-defined type which can be automatically recognized by a SQLContext and registered. + * + * WARNING: This annotation will only work if both Java and Scala reflection return the same class + * names (after erasure) for the UDT. This will NOT be the case when, e.g., the UDT class + * is enclosed in an object (a singleton). + * + * WARNING: UDTs are currently only supported from Scala. + */ +// TODO: Should I used @Documented ? +@DeveloperApi +@Retention(RetentionPolicy.RUNTIME) +@Target(ElementType.TYPE) +public @interface SQLUserDefinedType { + + /** + * Returns an instance of the UserDefinedType which can serialize and deserialize the user + * class to and from Catalyst built-in types. + */ + Class > udt(); +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 1b687a443ef8b..fa1786e74bb3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -21,6 +21,10 @@ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.types.DataType import org.apache.spark.util.ClosureCleaner +/** + * User-defined function. + * @param dataType Return type of function. + */ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) extends Expression { @@ -347,6 +351,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi } // scalastyle:on - ScalaReflection.convertToCatalyst(result) + ScalaReflection.convertToCatalyst(result, dataType) } } 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 d25f3a619dd75..cc5015ad3c013 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 @@ -29,11 +29,12 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.ScalaReflectionLock -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression, Row} +import org.apache.spark.sql.catalyst.types.decimal._ import org.apache.spark.sql.catalyst.util.Metadata import org.apache.spark.util.Utils -import org.apache.spark.sql.catalyst.types.decimal._ object DataType { def fromJson(json: String): DataType = parseDataType(parse(json)) @@ -67,6 +68,11 @@ object DataType { ("fields", JArray(fields)), ("type", JString("struct"))) => StructType(fields.map(parseStructField)) + + case JSortedObject( + ("class", JString(udtClass)), + ("type", JString("udt"))) => + Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] } private def parseStructField(json: JValue): StructField = json match { @@ -342,6 +348,7 @@ object FractionalType { case _ => false } } + abstract class FractionalType extends NumericType { private[sql] val fractional: Fractional[JvmType] private[sql] val asIntegral: Integral[JvmType] @@ -565,3 +572,45 @@ case class MapType( ("valueType" -> valueType.jsonValue) ~ ("valueContainsNull" -> valueContainsNull) } + +/** + * ::DeveloperApi:: + * The data type for User Defined Types (UDTs). + * + * This interface allows a user to make their own classes more interoperable with SparkSQL; + * e.g., by creating a [[UserDefinedType]] for a class X, it becomes possible to create + * a SchemaRDD which has class X in the schema. + * + * For SparkSQL to recognize UDTs, the UDT must be annotated with + * [[org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType]]. + * + * The conversion via `serialize` occurs when instantiating a `SchemaRDD` from another RDD. + * The conversion via `deserialize` occurs when reading from a `SchemaRDD`. + */ +@DeveloperApi +abstract class UserDefinedType[UserType] extends DataType with Serializable { + + /** Underlying storage type for this UDT */ + def sqlType: DataType + + /** + * Convert the user type to a SQL datum + * + * TODO: Can we make this take obj: UserType? The issue is in ScalaReflection.convertToCatalyst, + * where we need to convert Any to UserType. + */ + def serialize(obj: Any): Any + + /** Convert a SQL datum to the user type */ + def deserialize(datum: Any): UserType + + override private[sql] def jsonValue: JValue = { + ("type" -> "udt") ~ + ("class" -> this.getClass.getName) + } + + /** + * Class object for the UserType + */ + def userClass: java.lang.Class[UserType] +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index 21b2c8e20d4db..ddc3d44869c98 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -22,6 +22,7 @@ import java.sql.{Date, Timestamp} import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.types._ case class PrimitiveData( @@ -239,13 +240,17 @@ class ScalaReflectionSuite extends FunSuite { test("convert PrimitiveData to catalyst") { val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) val convertedData = Seq(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true) - assert(convertToCatalyst(data) === convertedData) + val dataType = schemaFor[PrimitiveData].dataType + assert(convertToCatalyst(data, dataType) === convertedData) } test("convert Option[Product] to catalyst") { val primitiveData = PrimitiveData(1, 1, 1, 1, 1, 1, true) - val data = OptionalData(Some(1), Some(1), Some(1), Some(1), Some(1), Some(1), Some(true), Some(primitiveData)) - val convertedData = Seq(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true, convertToCatalyst(primitiveData)) - assert(convertToCatalyst(data) === convertedData) + val data = OptionalData(Some(2), Some(2), Some(2), Some(2), Some(2), Some(2), Some(true), + Some(primitiveData)) + val dataType = schemaFor[OptionalData].dataType + val convertedData = Row(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true, + Row(1, 1, 1, 1, 1, 1, true)) + assert(convertToCatalyst(data, dataType) === convertedData) } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java new file mode 100644 index 0000000000000..b751847b464fd --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UserDefinedType.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; + +import org.apache.spark.annotation.DeveloperApi; + +/** + * ::DeveloperApi:: + * The data type representing User-Defined Types (UDTs). + * UDTs may use any other DataType for an underlying representation. + */ +@DeveloperApi +public abstract class UserDefinedType extends DataType implements Serializable { + + protected UserDefinedType() { } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + UserDefinedType that = (UserDefinedType) o; + return this.sqlType().equals(that.sqlType()); + } + + /** Underlying storage type for this UDT */ + public abstract DataType sqlType(); + + /** Convert the user type to a SQL datum */ + public abstract Object serialize(Object obj); + + /** Convert a SQL datum to the user type */ + public abstract UserType deserialize(Object datum); + + /** Class object for the UserType */ + public abstract Class userClass(); +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 3cf6af5f7a3c0..9e61d18f7e926 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -107,8 +107,10 @@ class SQLContext(@transient val sparkContext: SparkContext) */ implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = { SparkPlan.currentContext.set(self) - new SchemaRDD(this, - LogicalRDD(ScalaReflection.attributesFor[A], RDDConversions.productToRowRdd(rdd))(self)) + val attributeSeq = ScalaReflection.attributesFor[A] + val schema = StructType.fromAttributes(attributeSeq) + val rowRDD = RDDConversions.productToRowRdd(rdd, schema) + new SchemaRDD(this, LogicalRDD(attributeSeq, rowRDD)(self)) } implicit def baseRelationToSchemaRDD(baseRelation: BaseRelation): SchemaRDD = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 018a18c4ac214..3ee2ea05cfa2d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,26 +17,24 @@ package org.apache.spark.sql -import java.util.{Map => JMap, List => JList} - -import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.storage.StorageLevel +import java.util.{List => JList} import scala.collection.JavaConversions._ -import scala.collection.JavaConverters._ import net.razorvine.pickle.Pickler import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext} import org.apache.spark.annotation.{AlphaComponent, Experimental} +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.sql.api.java.JavaSchemaRDD +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.{LogicalRDD, EvaluatePython} -import org.apache.spark.api.java.JavaRDD +import org.apache.spark.storage.StorageLevel /** * :: AlphaComponent :: @@ -114,18 +112,22 @@ class SchemaRDD( // ========================================================================================= override def compute(split: Partition, context: TaskContext): Iterator[Row] = - firstParent[Row].compute(split, context).map(ScalaReflection.convertRowToScala) + firstParent[Row].compute(split, context).map(ScalaReflection.convertRowToScala(_, this.schema)) override def getPartitions: Array[Partition] = firstParent[Row].partitions - override protected def getDependencies: Seq[Dependency[_]] = + override protected def getDependencies: Seq[Dependency[_]] = { + schema // Force reification of the schema so it is available on executors. + List(new OneToOneDependency(queryExecution.toRdd)) + } - /** Returns the schema of this SchemaRDD (represented by a [[StructType]]). - * - * @group schema - */ - def schema: StructType = queryExecution.analyzed.schema + /** + * Returns the schema of this SchemaRDD (represented by a [[StructType]]). + * + * @group schema + */ + lazy val schema: StructType = queryExecution.analyzed.schema // ======================================================================= // Query DSL diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 15516afb95504..fd5f4abcbcd65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.execution.LogicalRDD * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) */ private[sql] trait SchemaRDDLike { - @transient val sqlContext: SQLContext + @transient def sqlContext: SQLContext @transient val baseLogicalPlan: LogicalPlan private[sql] def baseSchemaRDD: SchemaRDD diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index 595b4aa36eae3..6d4c0d82ac7af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -78,7 +78,7 @@ private[sql] trait UDFRegistration { s""" def registerFunction[T: TypeTag](name: String, func: Function$x[$types, T]): Unit = { def builder(e: Seq[Expression]) = - ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } """ @@ -87,112 +87,112 @@ private[sql] trait UDFRegistration { // scalastyle:off def registerFunction[T: TypeTag](name: String, func: Function1[_, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function2[_, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function3[_, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function4[_, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function5[_, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function6[_, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function7[_, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function8[_, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function9[_, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function10[_, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } def registerFunction[T: TypeTag](name: String, func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]): Unit = { - def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor(typeTag[T]).dataType, e) + def builder(e: Seq[Expression]) = ScalaUdf(func, ScalaReflection.schemaFor[T].dataType, e) functionRegistry.registerFunction(name, builder) } // scalastyle:on diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 60065509bfbbd..4c0869e05b029 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -23,13 +23,14 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} -import org.apache.spark.sql.json.JsonRDD -import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation} -import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.{SQLContext, StructType => SStructType} +import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} -import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.LogicalRDD +import org.apache.spark.sql.json.JsonRDD +import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.sources.{LogicalRelation, BaseRelation} +import org.apache.spark.sql.types.util.DataTypeConversions import org.apache.spark.sql.types.util.DataTypeConversions.asScalaDataType import org.apache.spark.util.Utils @@ -91,9 +92,12 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { /** * Applies a schema to an RDD of Java Beans. + * + * WARNING: Since there is no guaranteed ordering for fields in a Java Bean, + * SELECT * queries will return the columns in an undefined order. */ def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): JavaSchemaRDD = { - val schema = getSchema(beanClass) + val attributeSeq = getSchema(beanClass) val className = beanClass.getName val rowRdd = rdd.rdd.mapPartitions { iter => // BeanInfo is not serializable so we must rediscover it remotely for each partition. @@ -104,11 +108,13 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { iter.map { row => new GenericRow( - extractors.map(e => DataTypeConversions.convertJavaToCatalyst(e.invoke(row))).toArray[Any] + extractors.zip(attributeSeq).map { case (e, attr) => + DataTypeConversions.convertJavaToCatalyst(e.invoke(row), attr.dataType) + }.toArray[Any] ): ScalaRow } } - new JavaSchemaRDD(sqlContext, LogicalRDD(schema, rowRdd)(sqlContext)) + new JavaSchemaRDD(sqlContext, LogicalRDD(attributeSeq, rowRdd)(sqlContext)) } /** @@ -195,14 +201,21 @@ class JavaSQLContext(val sqlContext: SQLContext) extends UDFRegistration { sqlContext.registerRDDAsTable(rdd.baseSchemaRDD, tableName) } - /** Returns a Catalyst Schema for the given java bean class. */ + /** + * Returns a Catalyst Schema for the given java bean class. + */ protected def getSchema(beanClass: Class[_]): Seq[AttributeReference] = { // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. val beanInfo = Introspector.getBeanInfo(beanClass) + // Note: The ordering of elements may differ from when the schema is inferred in Scala. + // This is because beanInfo.getPropertyDescriptors gives no guarantees about + // element ordering. val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") fields.map { property => val (dataType, nullable) = property.getPropertyType match { + case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => + (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) case c: Class[_] if c == classOf[java.lang.String] => (org.apache.spark.sql.StringType, true) case c: Class[_] if c == java.lang.Short.TYPE => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala new file mode 100644 index 0000000000000..a7d0f4f127ecc --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/UDTWrappers.scala @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java + +import org.apache.spark.sql.catalyst.types.{UserDefinedType => ScalaUserDefinedType} +import org.apache.spark.sql.{DataType => ScalaDataType} +import org.apache.spark.sql.types.util.DataTypeConversions + +/** + * Scala wrapper for a Java UserDefinedType + */ +private[sql] class JavaToScalaUDTWrapper[UserType](val javaUDT: UserDefinedType[UserType]) + extends ScalaUserDefinedType[UserType] with Serializable { + + /** Underlying storage type for this UDT */ + val sqlType: ScalaDataType = DataTypeConversions.asScalaDataType(javaUDT.sqlType()) + + /** Convert the user type to a SQL datum */ + def serialize(obj: Any): Any = javaUDT.serialize(obj) + + /** Convert a SQL datum to the user type */ + def deserialize(datum: Any): UserType = javaUDT.deserialize(datum) + + val userClass: java.lang.Class[UserType] = javaUDT.userClass() +} + +/** + * Java wrapper for a Scala UserDefinedType + */ +private[sql] class ScalaToJavaUDTWrapper[UserType](val scalaUDT: ScalaUserDefinedType[UserType]) + extends UserDefinedType[UserType] with Serializable { + + /** Underlying storage type for this UDT */ + val sqlType: DataType = DataTypeConversions.asJavaDataType(scalaUDT.sqlType) + + /** Convert the user type to a SQL datum */ + def serialize(obj: Any): java.lang.Object = scalaUDT.serialize(obj).asInstanceOf[java.lang.Object] + + /** Convert a SQL datum to the user type */ + def deserialize(datum: Any): UserType = scalaUDT.deserialize(datum) + + val userClass: java.lang.Class[UserType] = scalaUDT.userClass +} + +private[sql] object UDTWrappers { + + def wrapAsScala(udtType: UserDefinedType[_]): ScalaUserDefinedType[_] = { + udtType match { + case t: ScalaToJavaUDTWrapper[_] => t.scalaUDT + case _ => new JavaToScalaUDTWrapper(udtType) + } + } + + def wrapAsJava(udtType: ScalaUserDefinedType[_]): UserDefinedType[_] = { + udtType match { + case t: JavaToScalaUDTWrapper[_] => t.javaUDT + case _ => new ScalaToJavaUDTWrapper(udtType) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index d64c5af89ec99..ed6b95dc6d9d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,29 +19,32 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataType, StructType, Row, SQLContext} import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.ScalaReflection.Schema import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} -import org.apache.spark.sql.{Row, SQLContext} +import org.apache.spark.sql.catalyst.types.UserDefinedType /** * :: DeveloperApi :: */ @DeveloperApi object RDDConversions { - def productToRowRdd[A <: Product](data: RDD[A]): RDD[Row] = { + def productToRowRdd[A <: Product](data: RDD[A], schema: StructType): RDD[Row] = { data.mapPartitions { iterator => if (iterator.isEmpty) { Iterator.empty } else { val bufferedIterator = iterator.buffered val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) - + val schemaFields = schema.fields.toArray bufferedIterator.map { r => var i = 0 while (i < mutableRow.length) { - mutableRow(i) = ScalaReflection.convertToCatalyst(r.productElement(i)) + mutableRow(i) = + ScalaReflection.convertToCatalyst(r.productElement(i), schemaFields(i).dataType) i += 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index aafcce0572b25..81c60e00505c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD - - import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.{ScalaReflection, trees} import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation @@ -82,7 +80,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Runs this query returning the result as an array. */ - def executeCollect(): Array[Row] = execute().map(ScalaReflection.convertRowToScala).collect() + def executeCollect(): Array[Row] = + execute().map(ScalaReflection.convertRowToScala(_, schema)).collect() protected def newProjection( expressions: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 2cd3063bc3097..cc7e0c05ffc70 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -280,7 +280,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val nPartitions = if (data.isEmpty) 1 else numPartitions PhysicalRDD( output, - RDDConversions.productToRowRdd(sparkContext.parallelize(data, nPartitions))) :: Nil + RDDConversions.productToRowRdd(sparkContext.parallelize(data, nPartitions), + StructType.fromAttributes(output))) :: Nil case logical.Limit(IntegerLiteral(limit), child) => execution.Limit(limit, planLater(child)) :: Nil case Unions(unionChildren) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index e6cd1a9d04278..1b8ba3ace2a82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -143,7 +143,7 @@ case class Limit(limit: Int, child: SparkPlan) partsScanned += numPartsToTry } - buf.toArray.map(ScalaReflection.convertRowToScala) + buf.toArray.map(ScalaReflection.convertRowToScala(_, this.schema)) } override def execute() = { @@ -179,8 +179,8 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) val ord = new RowOrdering(sortOrder, child.output) // TODO: Is this copying for no reason? - override def executeCollect() = - child.execute().map(_.copy()).takeOrdered(limit)(ord).map(ScalaReflection.convertRowToScala) + override def executeCollect() = child.execute().map(_.copy()).takeOrdered(limit)(ord) + .map(ScalaReflection.convertRowToScala(_, this.schema)) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. 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 08feced61a899..1bbb66aaa19a3 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 @@ -77,6 +77,9 @@ private[sql] object CatalystConverter { parent: CatalystConverter): Converter = { val fieldType: DataType = field.dataType fieldType match { + case udt: UserDefinedType[_] => { + createConverter(field.copy(dataType = udt.sqlType), fieldIndex, parent) + } // For native JVM types we use a converter with native arrays case ArrayType(elementType: NativeType, false) => { new CatalystNativeArrayConverter(elementType, fieldIndex, parent) @@ -255,8 +258,8 @@ private[parquet] class CatalystGroupConverter( schema, index, parent, - current=null, - buffer=new ArrayBuffer[Row]( + current = null, + buffer = new ArrayBuffer[Row]( CatalystArrayConverter.INITIAL_ARRAY_SIZE)) /** @@ -301,7 +304,7 @@ private[parquet] class CatalystGroupConverter( override def end(): Unit = { if (!isRootConverter) { - assert(current!=null) // there should be no empty groups + assert(current != null) // there should be no empty groups buffer.append(new GenericRow(current.toArray)) parent.updateField(index, new GenericRow(buffer.toArray.asInstanceOf[Array[Any]])) } @@ -358,7 +361,7 @@ private[parquet] class CatalystPrimitiveRowConverter( override def end(): Unit = {} - // Overriden here to avoid auto-boxing for primitive types + // Overridden here to avoid auto-boxing for primitive types override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = current.setBoolean(fieldIndex, value) @@ -533,7 +536,7 @@ private[parquet] class CatalystNativeArrayConverter( override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = throw new UnsupportedOperationException - // Overriden here to avoid auto-boxing for primitive types + // Overridden here to avoid auto-boxing for primitive types override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { checkGrowBuffer() buffer(elements) = value.asInstanceOf[NativeType] 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 2a5f23b24e8e8..7bc249660053a 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 @@ -20,7 +20,6 @@ package org.apache.spark.sql.parquet import java.util.{HashMap => JHashMap} import org.apache.hadoop.conf.Configuration -import org.apache.spark.sql.catalyst.types.decimal.Decimal import parquet.column.ParquetProperties import parquet.hadoop.ParquetOutputFormat import parquet.hadoop.api.ReadSupport.ReadContext @@ -31,6 +30,7 @@ import parquet.schema.MessageType import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.types.decimal.Decimal /** * A `parquet.io.api.RecordMaterializer` for Rows. @@ -174,6 +174,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeValue(schema: DataType, value: Any): Unit = { if (value != null) { schema match { + case t: UserDefinedType[_] => writeValue(t.sqlType, value) case t @ ArrayType(_, _) => writeArray( t, value.asInstanceOf[CatalystConverter.ArrayScalaType[_]]) 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 e5077de8dd908..fa37d1f2ae7e6 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 @@ -290,6 +290,9 @@ private[parquet] object ParquetTypesConverter extends Logging { builder.named(name) }.getOrElse { ctype match { + case udt: UserDefinedType[_] => { + fromDataType(udt.sqlType, name, nullable, inArray) + } case ArrayType(elementType, false) => { val parquetElementType = fromDataType( elementType, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 7564bf3923032..1bc15146f0fe8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -17,12 +17,16 @@ package org.apache.spark.sql.types.util +import scala.collection.JavaConverters._ + import org.apache.spark.sql._ -import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, MetadataBuilder => JMetaDataBuilder} +import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, + MetadataBuilder => JMetaDataBuilder, UDTWrappers, JavaToScalaUDTWrapper} import org.apache.spark.sql.api.java.{DecimalType => JDecimalType} import org.apache.spark.sql.catalyst.types.decimal.Decimal +import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.types.UserDefinedType -import scala.collection.JavaConverters._ protected[sql] object DataTypeConversions { @@ -41,6 +45,9 @@ protected[sql] object DataTypeConversions { * Returns the equivalent DataType in Java for the given DataType in Scala. */ def asJavaDataType(scalaDataType: DataType): JDataType = scalaDataType match { + case udtType: UserDefinedType[_] => + UDTWrappers.wrapAsJava(udtType) + case StringType => JDataType.StringType case BinaryType => JDataType.BinaryType case BooleanType => JDataType.BooleanType @@ -80,6 +87,9 @@ protected[sql] object DataTypeConversions { * Returns the equivalent DataType in Scala for the given DataType in Java. */ def asScalaDataType(javaDataType: JDataType): DataType = javaDataType match { + case udtType: org.apache.spark.sql.api.java.UserDefinedType[_] => + UDTWrappers.wrapAsScala(udtType) + case stringType: org.apache.spark.sql.api.java.StringType => StringType case binaryType: org.apache.spark.sql.api.java.BinaryType => @@ -121,9 +131,11 @@ protected[sql] object DataTypeConversions { } /** Converts Java objects to catalyst rows / types */ - def convertJavaToCatalyst(a: Any): Any = a match { - case d: java.math.BigDecimal => Decimal(BigDecimal(d)) - case other => other + def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { + case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type + case (d: java.math.BigDecimal, _) => Decimal(BigDecimal(d)) + case (d: java.math.BigDecimal, _) => BigDecimal(d) + case (other, _) => other } /** Converts Java objects to catalyst rows / types */ diff --git a/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaUserDefinedTypeSuite.java b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaUserDefinedTypeSuite.java new file mode 100644 index 0000000000000..0caa8219a63e9 --- /dev/null +++ b/sql/core/src/test/java/org/apache/spark/sql/api/java/JavaUserDefinedTypeSuite.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.api.java; + +import java.io.Serializable; +import java.util.*; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.MyDenseVector; +import org.apache.spark.sql.MyLabeledPoint; + +public class JavaUserDefinedTypeSuite implements Serializable { + private transient JavaSparkContext javaCtx; + private transient JavaSQLContext javaSqlCtx; + + @Before + public void setUp() { + javaCtx = new JavaSparkContext("local", "JavaUserDefinedTypeSuite"); + javaSqlCtx = new JavaSQLContext(javaCtx); + } + + @After + public void tearDown() { + javaCtx.stop(); + javaCtx = null; + javaSqlCtx = null; + } + + @Test + public void useScalaUDT() { + List points = Arrays.asList( + new MyLabeledPoint(1.0, new MyDenseVector(new double[]{0.1, 1.0})), + new MyLabeledPoint(0.0, new MyDenseVector(new double[]{0.2, 2.0}))); + JavaRDD pointsRDD = javaCtx.parallelize(points); + + JavaSchemaRDD schemaRDD = javaSqlCtx.applySchema(pointsRDD, MyLabeledPoint.class); + schemaRDD.registerTempTable("points"); + + List actualLabelRows = javaSqlCtx.sql("SELECT label FROM points").collect(); + List actualLabels = new LinkedList(); + for (Row r : actualLabelRows) { + actualLabels.add(r.getDouble(0)); + } + for (MyLabeledPoint lp : points) { + Assert.assertTrue(actualLabels.contains(lp.label())); + } + + List actualFeatureRows = javaSqlCtx.sql("SELECT features FROM points").collect(); + List actualFeatures = new LinkedList(); + for (Row r : actualFeatureRows) { + actualFeatures.add((MyDenseVector)r.get(0)); + } + for (MyLabeledPoint lp : points) { + Assert.assertTrue(actualFeatures.contains(lp.features())); + } + + List actual = javaSqlCtx.sql("SELECT label, features FROM points").collect(); + List actualPoints = + new LinkedList(); + for (Row r : actual) { + actualPoints.add(new MyLabeledPoint(r.getDouble(0), (MyDenseVector)r.get(1))); + } + for (MyLabeledPoint lp : points) { + Assert.assertTrue(actualPoints.contains(lp)); + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala new file mode 100644 index 0000000000000..666235e57f812 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import scala.beans.{BeanInfo, BeanProperty} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType +import org.apache.spark.sql.catalyst.types.UserDefinedType +import org.apache.spark.sql.test.TestSQLContext._ + +@SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) +private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { + override def equals(other: Any): Boolean = other match { + case v: MyDenseVector => + java.util.Arrays.equals(this.data, v.data) + case _ => false + } +} + +@BeanInfo +private[sql] case class MyLabeledPoint( + @BeanProperty label: Double, + @BeanProperty features: MyDenseVector) + +private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { + + override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) + + override def serialize(obj: Any): Seq[Double] = { + obj match { + case features: MyDenseVector => + features.data.toSeq + } + } + + override def deserialize(datum: Any): MyDenseVector = { + datum match { + case data: Seq[_] => + new MyDenseVector(data.asInstanceOf[Seq[Double]].toArray) + } + } + + override def userClass = classOf[MyDenseVector] +} + +class UserDefinedTypeSuite extends QueryTest { + + test("register user type: MyDenseVector for MyLabeledPoint") { + val points = Seq( + MyLabeledPoint(1.0, new MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new MyDenseVector(Array(0.2, 2.0)))) + val pointsRDD: RDD[MyLabeledPoint] = sparkContext.parallelize(points) + + val labels: RDD[Double] = pointsRDD.select('label).map { case Row(v: Double) => v } + val labelsArrays: Array[Double] = labels.collect() + assert(labelsArrays.size === 2) + assert(labelsArrays.contains(1.0)) + assert(labelsArrays.contains(0.0)) + + val features: RDD[MyDenseVector] = + pointsRDD.select('features).map { case Row(v: MyDenseVector) => v } + val featuresArrays: Array[MyDenseVector] = features.collect() + assert(featuresArrays.size === 2) + assert(featuresArrays.contains(new MyDenseVector(Array(0.1, 1.0)))) + assert(featuresArrays.contains(new MyDenseVector(Array(0.2, 2.0)))) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 4b851d1b96152..cade244f7ac39 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -21,8 +21,7 @@ import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.json.JsonRDD.{enforceCorrectType, compatibleType} -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.{Row, SQLConf, QueryTest} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ @@ -233,8 +232,8 @@ class JsonSuite extends QueryTest { StructField("field2", StringType, true) :: StructField("field3", StringType, true) :: Nil), false), true) :: StructField("struct", StructType( - StructField("field1", BooleanType, true) :: - StructField("field2", DecimalType.Unlimited, true) :: Nil), true) :: + StructField("field1", BooleanType, true) :: + StructField("field2", DecimalType.Unlimited, true) :: Nil), true) :: StructField("structWithArrayFields", StructType( StructField("field1", ArrayType(IntegerType, false), true) :: StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) @@ -292,8 +291,8 @@ class JsonSuite extends QueryTest { // Access a struct and fields inside of it. checkAnswer( sql("select struct, struct.field1, struct.field2 from jsonTable"), - ( - Seq(true, BigDecimal("92233720368547758070")), + Row( + Row(true, BigDecimal("92233720368547758070")), true, BigDecimal("92233720368547758070")) :: Nil ) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 0fe59f42f21ff..f025169ad5063 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -374,8 +374,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /** Extends QueryExecution with hive specific features. */ protected[sql] abstract class QueryExecution extends super.QueryExecution { - override lazy val toRdd: RDD[Row] = executedPlan.execute().map(_.copy()) - protected val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) @@ -433,7 +431,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { command.executeCollect().map(_.head.toString) case other => - val result: Seq[Seq[Any]] = toRdd.collect().toSeq + val result: Seq[Seq[Any]] = toRdd.map(_.copy()).collect().toSeq // We need the types so we can output struct field names val types = analyzed.output.map(_.dataType) // Reformat to match hive tab delimited output. From 1ae51f6dc7e79a88dfe54a2c488b540fdbf51212 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sat, 1 Nov 2014 13:15:24 -0700 Subject: [PATCH 1196/1492] [SPARK-4183] Enable NettyBlockTransferService by default Note that we're turning this on for at least the first part of the QA period as a trial. We want to enable this (and deprecate the NioBlockTransferService) as soon as possible in the hopes that NettyBlockTransferService will be more stable and easier to maintain. We will turn it off if we run into major issues. Author: Aaron Davidson Closes #3049 from aarondav/enable-netty and squashes the following commits: bb981cc [Aaron Davidson] [SPARK-4183] Enable NettyBlockTransferService by default --- docs/configuration.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 099972ca1af70..685101ea5c9c9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -371,6 +371,16 @@ Apart from these, the following properties are also available, and may be useful map-side aggregation and there are at most this many reduce partitions. + + spark.shuffle.blockTransferService + netty + + Implementation to use for transferring shuffle and cached blocks between executors. There + are two implementations available: netty and nio. Netty-based + block transfer is intended to be simpler but equally efficient and is the default option + starting in 1.2. + + #### Spark UI From d6e4c5917522b9fb6653ddc0634e93ff2dcf82be Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 2 Nov 2014 21:56:07 -0800 Subject: [PATCH 1197/1492] Close #2971. From 001acc446345ccb1e494af9ff1d16dd65db8034e Mon Sep 17 00:00:00 2001 From: wangfei Date: Sun, 2 Nov 2014 22:02:05 -0800 Subject: [PATCH 1198/1492] [SPARK-4177][Doc]update build doc since JDBC/CLI support hive 13 now Author: wangfei Closes #3042 from scwf/patch-9 and squashes the following commits: 3784ed1 [wangfei] remove 'TODO' 1891553 [wangfei] update build doc since JDBC/CLI support hive 13 --- docs/building-spark.md | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 4cc0b1f2e5116..238ddae15545e 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -99,14 +99,11 @@ mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package {% endhighlight %} - - # Building With Hive and JDBC Support To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` profile to your existing build options. By default Spark will build with Hive 0.13.1 bindings. You can also build for Hive 0.12.0 using -the `-Phive-0.12.0` profile. NOTE: currently the JDBC server is only -supported for Hive 0.12.0. +the `-Phive-0.12.0` profile. {% highlight bash %} # Apache Hadoop 2.4.X with Hive 13 support mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -DskipTests clean package @@ -121,8 +118,8 @@ Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.o Some of the tests require Spark to be packaged first, so always run `mvn package` with `-DskipTests` the first time. The following is an example of a correct (build, test) sequence: - mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive -Phive-0.12.0 clean package - mvn -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test + mvn -Pyarn -Phadoop-2.3 -DskipTests -Phive clean package + mvn -Pyarn -Phadoop-2.3 -Phive test The ScalaTest plugin also supports running only a specific test suite as follows: @@ -185,16 +182,16 @@ can be set to control the SBT build. For example: Some of the tests require Spark to be packaged first, so always run `sbt/sbt assembly` the first time. The following is an example of a correct (build, test) sequence: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 assembly - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive assembly + sbt/sbt -Pyarn -Phadoop-2.3 -Phive test To run only a specific test suite as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 "test-only org.apache.spark.repl.ReplSuite" + sbt/sbt -Pyarn -Phadoop-2.3 -Phive "test-only org.apache.spark.repl.ReplSuite" To run test suites of a specific sub project as follows: - sbt/sbt -Pyarn -Phadoop-2.3 -Phive -Phive-0.12.0 core/test + sbt/sbt -Pyarn -Phadoop-2.3 -Phive core/test # Speeding up Compilation with Zinc From 76386e1a23c55a58c0aeea67820aab2bac71b24b Mon Sep 17 00:00:00 2001 From: zsxwing Date: Sun, 2 Nov 2014 23:20:22 -0800 Subject: [PATCH 1199/1492] [SPARK-4163][Core][WebUI] Send the fetch failure message back to Web UI This is a PR to send the fetch failure message back to Web UI. Before: ![f1](https://cloud.githubusercontent.com/assets/1000778/4856595/1f036c80-60be-11e4-956f-335147fbccb7.png) ![f2](https://cloud.githubusercontent.com/assets/1000778/4856596/1f11cbea-60be-11e4-8fe9-9f9b2b35c884.png) After (Please ignore the meaning of exception, I threw it in the code directly because it's hard to simulate a fetch failure): ![e1](https://cloud.githubusercontent.com/assets/1000778/4856600/2657ea38-60be-11e4-9f2d-d56c5f900f10.png) ![e2](https://cloud.githubusercontent.com/assets/1000778/4856601/26595008-60be-11e4-912b-2744af786991.png) Author: zsxwing Closes #3032 from zsxwing/SPARK-4163 and squashes the following commits: f7e1faf [zsxwing] Discard changes for FetchFailedException and minor modification 4e946f7 [zsxwing] Add e as the cause of SparkException 316767d [zsxwing] Add private[storage] to FetchResult d51b0b6 [zsxwing] Set e as the cause of FetchFailedException b88c919 [zsxwing] Use 'private[storage]' for case classes instead of 'sealed' 62103fd [zsxwing] Update as per review 0c07d1f [zsxwing] Backward-compatible support a3bca65 [zsxwing] Send the fetch failure message back to Web UI --- .../org/apache/spark/TaskEndReason.scala | 6 +- .../apache/spark/scheduler/DAGScheduler.scala | 4 +- .../apache/spark/scheduler/JobLogger.scala | 2 +- .../spark/shuffle/FetchFailedException.scala | 16 ++-- .../hash/BlockStoreShuffleFetcher.scala | 14 ++-- .../storage/ShuffleBlockFetcherIterator.scala | 82 ++++++++++++------- .../org/apache/spark/util/JsonProtocol.scala | 7 +- .../scala/org/apache/spark/util/Utils.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 10 +-- .../ShuffleBlockFetcherIteratorSuite.scala | 8 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- 12 files changed, 92 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 202fba699ab26..f45b463fb6f62 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -69,11 +69,13 @@ case class FetchFailed( bmAddress: BlockManagerId, // Note that bmAddress can be null shuffleId: Int, mapId: Int, - reduceId: Int) + reduceId: Int, + message: String) extends TaskFailedReason { override def toErrorString: String = { val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString - s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId)" + s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " + + s"message=\n$message\n)" } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index af17b5d5d2571..96114c0423a9e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1053,7 +1053,7 @@ class DAGScheduler( logInfo("Resubmitted " + task + ", so marking it as still running") stage.pendingTasks += task - case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => + case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) => val failedStage = stageIdToStage(task.stageId) val mapStage = shuffleToMapStage(shuffleId) @@ -1063,7 +1063,7 @@ class DAGScheduler( if (runningStages.contains(failedStage)) { logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + s"due to a fetch failure from $mapStage (${mapStage.name})") - markStageAsFinished(failedStage, Some("Fetch failure")) + markStageAsFinished(failedStage, Some("Fetch failure: " + failureMessage)) runningStages -= failedStage } diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 54904bffdf10b..4e3d9de540783 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -215,7 +215,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + " STAGE_ID=" + taskEnd.stageId stageLogInfo(taskEnd.stageId, taskStatus) - case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => + case FetchFailed(bmAddress, shuffleId, mapId, reduceId, message) => taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + taskEnd.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId diff --git a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala index 71c08e9d5a8c3..0c1b6f4defdb3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala @@ -19,6 +19,7 @@ package org.apache.spark.shuffle import org.apache.spark.storage.BlockManagerId import org.apache.spark.{FetchFailed, TaskEndReason} +import org.apache.spark.util.Utils /** * Failed to fetch a shuffle block. The executor catches this exception and propagates it @@ -30,13 +31,11 @@ private[spark] class FetchFailedException( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, - reduceId: Int) - extends Exception { - - override def getMessage: String = - "Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId) + reduceId: Int, + message: String) + extends Exception(message) { - def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId) + def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, message) } /** @@ -46,7 +45,4 @@ private[spark] class MetadataFetchFailedException( shuffleId: Int, reduceId: Int, message: String) - extends FetchFailedException(null, shuffleId, -1, reduceId) { - - override def getMessage: String = message -} + extends FetchFailedException(null, shuffleId, -1, reduceId, message) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala index f49917b7fe833..0d5247f4176d4 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala @@ -19,12 +19,13 @@ package org.apache.spark.shuffle.hash import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap +import scala.util.{Failure, Success, Try} import org.apache.spark._ import org.apache.spark.serializer.Serializer import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId} -import org.apache.spark.util.CompletionIterator +import org.apache.spark.util.{CompletionIterator, Utils} private[hash] object BlockStoreShuffleFetcher extends Logging { def fetch[T]( @@ -52,21 +53,22 @@ private[hash] object BlockStoreShuffleFetcher extends Logging { (address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2))) } - def unpackBlock(blockPair: (BlockId, Option[Iterator[Any]])) : Iterator[T] = { + def unpackBlock(blockPair: (BlockId, Try[Iterator[Any]])) : Iterator[T] = { val blockId = blockPair._1 val blockOption = blockPair._2 blockOption match { - case Some(block) => { + case Success(block) => { block.asInstanceOf[Iterator[T]] } - case None => { + case Failure(e) => { blockId match { case ShuffleBlockId(shufId, mapId, _) => val address = statuses(mapId.toInt)._1 - throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId) + throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, + Utils.exceptionString(e)) case _ => throw new SparkException( - "Failed to get block " + blockId + ", which is not a shuffle block") + "Failed to get block " + blockId + ", which is not a shuffle block", e) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index ee89c7e521f4e..1e579187e4193 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} +import scala.util.{Failure, Success, Try} import org.apache.spark.{Logging, TaskContext} import org.apache.spark.network.BlockTransferService @@ -55,7 +56,7 @@ final class ShuffleBlockFetcherIterator( blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer, maxBytesInFlight: Long) - extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { + extends Iterator[(BlockId, Try[Iterator[Any]])] with Logging { import ShuffleBlockFetcherIterator._ @@ -118,16 +119,18 @@ final class ShuffleBlockFetcherIterator( private[this] def cleanup() { isZombie = true // Release the current buffer if necessary - if (currentResult != null && !currentResult.failed) { - currentResult.buf.release() + currentResult match { + case SuccessFetchResult(_, _, buf) => buf.release() + case _ => } // Release buffers in the results queue val iter = results.iterator() while (iter.hasNext) { val result = iter.next() - if (!result.failed) { - result.buf.release() + result match { + case SuccessFetchResult(_, _, buf) => buf.release() + case _ => } } } @@ -151,7 +154,7 @@ final class ShuffleBlockFetcherIterator( // Increment the ref count because we need to pass this to a different thread. // This needs to be released after use. buf.retain() - results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), buf)) + results.put(new SuccessFetchResult(BlockId(blockId), sizeMap(blockId), buf)) shuffleMetrics.remoteBytesRead += buf.size shuffleMetrics.remoteBlocksFetched += 1 } @@ -160,7 +163,7 @@ final class ShuffleBlockFetcherIterator( override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) - results.put(new FetchResult(BlockId(blockId), -1, null)) + results.put(new FailureFetchResult(BlockId(blockId), e)) } } ) @@ -231,12 +234,12 @@ final class ShuffleBlockFetcherIterator( val buf = blockManager.getBlockData(blockId) shuffleMetrics.localBlocksFetched += 1 buf.retain() - results.put(new FetchResult(blockId, 0, buf)) + results.put(new SuccessFetchResult(blockId, 0, buf)) } catch { case e: Exception => // If we see an exception, stop immediately. logError(s"Error occurred while fetching local blocks", e) - results.put(new FetchResult(blockId, -1, null)) + results.put(new FailureFetchResult(blockId, e)) return } } @@ -267,15 +270,17 @@ final class ShuffleBlockFetcherIterator( override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch - override def next(): (BlockId, Option[Iterator[Any]]) = { + override def next(): (BlockId, Try[Iterator[Any]]) = { numBlocksProcessed += 1 val startFetchWait = System.currentTimeMillis() currentResult = results.take() val result = currentResult val stopFetchWait = System.currentTimeMillis() shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) - if (!result.failed) { - bytesInFlight -= result.size + + result match { + case SuccessFetchResult(_, size, _) => bytesInFlight -= size + case _ => } // Send fetch requests up to maxBytesInFlight while (fetchRequests.nonEmpty && @@ -283,20 +288,21 @@ final class ShuffleBlockFetcherIterator( sendRequest(fetchRequests.dequeue()) } - val iteratorOpt: Option[Iterator[Any]] = if (result.failed) { - None - } else { - val is = blockManager.wrapForCompression(result.blockId, result.buf.createInputStream()) - val iter = serializer.newInstance().deserializeStream(is).asIterator - Some(CompletionIterator[Any, Iterator[Any]](iter, { - // Once the iterator is exhausted, release the buffer and set currentResult to null - // so we don't release it again in cleanup. - currentResult = null - result.buf.release() - })) + val iteratorTry: Try[Iterator[Any]] = result match { + case FailureFetchResult(_, e) => Failure(e) + case SuccessFetchResult(blockId, _, buf) => { + val is = blockManager.wrapForCompression(blockId, buf.createInputStream()) + val iter = serializer.newInstance().deserializeStream(is).asIterator + Success(CompletionIterator[Any, Iterator[Any]](iter, { + // Once the iterator is exhausted, release the buffer and set currentResult to null + // so we don't release it again in cleanup. + currentResult = null + buf.release() + })) + } } - (result.blockId, iteratorOpt) + (result.blockId, iteratorTry) } } @@ -315,14 +321,30 @@ object ShuffleBlockFetcherIterator { } /** - * Result of a fetch from a remote block. A failure is represented as size == -1. + * Result of a fetch from a remote block. + */ + private[storage] sealed trait FetchResult { + val blockId: BlockId + } + + /** + * Result of a fetch from a remote block successfully. * @param blockId block id * @param size estimated size of the block, used to calculate bytesInFlight. - * Note that this is NOT the exact bytes. -1 if failure is present. - * @param buf [[ManagedBuffer]] for the content. null is error. + * Note that this is NOT the exact bytes. + * @param buf [[ManagedBuffer]] for the content. */ - case class FetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) { - def failed: Boolean = size == -1 - if (failed) assert(buf == null) else assert(buf != null) + private[storage] case class SuccessFetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) + extends FetchResult { + require(buf != null) + require(size >= 0) } + + /** + * Result of a fetch from a remote block unsuccessfully. + * @param blockId block id + * @param e the failure exception + */ + private[storage] case class FailureFetchResult(blockId: BlockId, e: Throwable) + extends FetchResult } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 43c7fba06694a..f7ae1f7f334de 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -279,7 +279,8 @@ private[spark] object JsonProtocol { ("Block Manager Address" -> blockManagerAddress) ~ ("Shuffle ID" -> fetchFailed.shuffleId) ~ ("Map ID" -> fetchFailed.mapId) ~ - ("Reduce ID" -> fetchFailed.reduceId) + ("Reduce ID" -> fetchFailed.reduceId) ~ + ("Message" -> fetchFailed.message) case exceptionFailure: ExceptionFailure => val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) @@ -629,7 +630,9 @@ private[spark] object JsonProtocol { val shuffleId = (json \ "Shuffle ID").extract[Int] val mapId = (json \ "Map ID").extract[Int] val reduceId = (json \ "Reduce ID").extract[Int] - new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId) + val message = Utils.jsonOption(json \ "Message").map(_.extract[String]) + new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId, + message.getOrElse("Unknown reason")) case `exceptionFailure` => val className = (json \ "Class Name").extract[String] val description = (json \ "Description").extract[String] 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 b402c5f334bb0..a33046d2040d8 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1597,7 +1597,7 @@ private[spark] object Utils extends Logging { } /** Return a nice string representation of the exception, including the stack trace. */ - def exceptionString(e: Exception): String = { + def exceptionString(e: Throwable): String = { if (e == null) "" else exceptionString(getFormattedClassName(e), e.getMessage, e.getStackTrace) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index a2e4f712db55b..819f95634bcdc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -431,7 +431,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), - (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null))) // this will get called // blockManagerMaster.removeExecutor("exec-hostA") // ask the scheduler to try it again @@ -461,7 +461,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F // The first result task fails, with a fetch failure for the output from the first mapper. runEvent(CompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null, Map[Long, Any](), null, @@ -472,7 +472,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(CompletionEvent( taskSets(1).tasks(0), - FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"), null, Map[Long, Any](), null, @@ -624,7 +624,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F (Success, makeMapStatus("hostC", 1)))) // fail the third stage because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // have DAGScheduler try again @@ -655,7 +655,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F (Success, makeMapStatus("hostB", 1)))) // pretend stage 0 failed because hostA went down complete(taskSets(2), Seq( - (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null))) + (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null))) // TODO assert this: // blockManagerMaster.removeExecutor("exec-hostA") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index 28f766570e96f..1eaabb93adbed 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -102,7 +102,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { for (i <- 0 until 5) { assert(iterator.hasNext, s"iterator should have 5 elements but actually has $i elements") val (blockId, subIterator) = iterator.next() - assert(subIterator.isDefined, + assert(subIterator.isSuccess, s"iterator should have 5 elements defined but actually has $i elements") // Make sure we release the buffer once the iterator is exhausted. @@ -230,8 +230,8 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite { sem.acquire() // The first block should be defined, and the last two are not defined (due to failure) - assert(iterator.next()._2.isDefined === true) - assert(iterator.next()._2.isDefined === false) - assert(iterator.next()._2.isDefined === false) + assert(iterator.next()._2.isSuccess) + assert(iterator.next()._2.isFailure) + assert(iterator.next()._2.isFailure) } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 6567c5ab836e7..2efbae689771a 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -115,7 +115,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc // Go through all the failure cases to make sure we are counting them as failures. val taskFailedReasons = Seq( Resubmitted, - new FetchFailed(null, 0, 0, 0), + new FetchFailed(null, 0, 0, 0, "ignored"), new ExceptionFailure("Exception", "description", null, None), TaskResultLost, TaskKilled, diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index d235d7a0ed839..a91c9ddeaef36 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -107,7 +107,8 @@ class JsonProtocolSuite extends FunSuite { testJobResult(jobFailed) // TaskEndReason - val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19) + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + "Some exception") val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, None) testTaskEndReason(Success) testTaskEndReason(Resubmitted) @@ -396,6 +397,7 @@ class JsonProtocolSuite extends FunSuite { assert(r1.mapId === r2.mapId) assert(r1.reduceId === r2.reduceId) assertEquals(r1.bmAddress, r2.bmAddress) + assert(r1.message === r2.message) case (r1: ExceptionFailure, r2: ExceptionFailure) => assert(r1.className === r2.className) assert(r1.description === r2.description) From 2aca97c7cfdefea8b6f9dbb88951e9acdfd606d9 Mon Sep 17 00:00:00 2001 From: Nicholas Chammas Date: Mon, 3 Nov 2014 09:02:35 -0800 Subject: [PATCH 1200/1492] [EC2] Factor out Mesos spark-ec2 branch We reference a specific branch in two places. This patch makes it one place. Author: Nicholas Chammas Closes #3008 from nchammas/mesos-spark-ec2-branch and squashes the following commits: 10a6089 [Nicholas Chammas] factor out mess spark-ec2 branch --- ec2/spark_ec2.py | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0d6b82b4944f3..50f88f735650e 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -41,8 +41,9 @@ DEFAULT_SPARK_VERSION = "1.1.0" +MESOS_SPARK_EC2_BRANCH = "v4" # A URL prefix from which to fetch AMI information -AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/v2/ami-list" +AMI_PREFIX = "https://raw.github.com/mesos/spark-ec2/{b}/ami-list".format(b=MESOS_SPARK_EC2_BRANCH) class UsageError(Exception): @@ -583,7 +584,13 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - ssh(master, opts, "rm -rf spark-ec2 && git clone https://github.com/mesos/spark-ec2.git -b v4") + ssh( + host=master, + opts=opts, + command="rm -rf spark-ec2" + + " && " + + "git clone https://github.com/mesos/spark-ec2.git -b {b}".format(b=MESOS_SPARK_EC2_BRANCH) + ) print "Deploying files to master..." deploy_files(conn, "deploy.generic", opts, master_nodes, slave_nodes, modules) From 3cca1962207745814b9d83e791713c91b659c36c Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 3 Nov 2014 12:24:24 -0800 Subject: [PATCH 1201/1492] [SPARK-4148][PySpark] fix seed distribution and add some tests for rdd.sample The current way of seed distribution makes the random sequences from partition i and i+1 offset by 1. ~~~ In [14]: import random In [15]: r1 = random.Random(10) In [16]: r1.randint(0, 1) Out[16]: 1 In [17]: r1.random() Out[17]: 0.4288890546751146 In [18]: r1.random() Out[18]: 0.5780913011344704 In [19]: r2 = random.Random(10) In [20]: r2.randint(0, 1) Out[20]: 1 In [21]: r2.randint(0, 1) Out[21]: 0 In [22]: r2.random() Out[22]: 0.5780913011344704 ~~~ Note: The new tests are not for this bug fix. Author: Xiangrui Meng Closes #3010 from mengxr/SPARK-4148 and squashes the following commits: 869ae4b [Xiangrui Meng] move tests tests.py c1bacd9 [Xiangrui Meng] fix seed distribution and add some tests for rdd.sample --- python/pyspark/rdd.py | 3 --- python/pyspark/rddsampler.py | 11 +++++------ python/pyspark/tests.py | 15 +++++++++++++++ 3 files changed, 20 insertions(+), 9 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 550c9dd80522f..4f025b9f11707 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -316,9 +316,6 @@ def sample(self, withReplacement, fraction, seed=None): """ Return a sampled subset of this RDD (relies on numpy and falls back on default random generator if numpy is unavailable). - - >>> sc.parallelize(range(0, 100)).sample(False, 0.1, 2).collect() #doctest: +SKIP - [2, 3, 20, 21, 24, 41, 42, 66, 67, 89, 90, 98] """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 528a181e8905a..f5c3cfd259a5b 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -40,14 +40,13 @@ def __init__(self, withReplacement, seed=None): def initRandomGenerator(self, split): if self._use_numpy: import numpy - self._random = numpy.random.RandomState(self._seed) + self._random = numpy.random.RandomState(self._seed ^ split) else: - self._random = random.Random(self._seed) + self._random = random.Random(self._seed ^ split) - for _ in range(0, split): - # discard the next few values in the sequence to have a - # different seed for the different splits - self._random.randint(0, 2 ** 32 - 1) + # mixing because the initial seeds are close to each other + for _ in xrange(10): + self._random.randint(0, 1) self._split = split self._rand_initialized = True diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 37a128907b3a7..253a471849c3a 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -648,6 +648,21 @@ def test_distinct(self): self.assertEquals(result.getNumPartitions(), 5) self.assertEquals(result.count(), 3) + def test_sample(self): + rdd = self.sc.parallelize(range(0, 100), 4) + wo = rdd.sample(False, 0.1, 2).collect() + wo_dup = rdd.sample(False, 0.1, 2).collect() + self.assertSetEqual(set(wo), set(wo_dup)) + wr = rdd.sample(True, 0.2, 5).collect() + wr_dup = rdd.sample(True, 0.2, 5).collect() + self.assertSetEqual(set(wr), set(wr_dup)) + wo_s10 = rdd.sample(False, 0.3, 10).collect() + wo_s20 = rdd.sample(False, 0.3, 20).collect() + self.assertNotEqual(set(wo_s10), set(wo_s20)) + wr_s11 = rdd.sample(True, 0.4, 11).collect() + wr_s21 = rdd.sample(True, 0.4, 21).collect() + self.assertNotEqual(set(wr_s11), set(wr_s21)) + class ProfilerTests(PySparkTestCase): From df607da025488d6c924d3d70eddb67f5523080d3 Mon Sep 17 00:00:00 2001 From: fi Date: Mon, 3 Nov 2014 12:56:56 -0800 Subject: [PATCH 1202/1492] [SPARK-4211][Build] Fixes hive.version in Maven profile hive-0.13.1 instead of `hive.version=0.13.1`. e.g. mvn -Phive -Phive=0.13.1 Note: `hive.version=0.13.1a` is the default property value. However, when explicitly specifying the `hive-0.13.1` maven profile, the wrong one would be selected. References: PR #2685, which resolved a package incompatibility issue with Hive-0.13.1 by introducing a special version Hive-0.13.1a Author: fi Closes #3072 from coderfi/master and squashes the following commits: 7ca4b1e [fi] Fixes the `hive-0.13.1` maven profile referencing `hive.version=0.13.1` instead of the Spark compatible `hive.version=0.13.1a` Note: `hive.version=0.13.1a` is the default version. However, when explicitly specifying the `hive-0.13.1` maven profile, the wrong one would be selected. e.g. mvn -Phive -Phive=0.13.1 See PR #2685 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 6191cd3a541e2..eb613531b8a5f 100644 --- a/pom.xml +++ b/pom.xml @@ -1359,7 +1359,7 @@ false - 0.13.1 + 0.13.1a 0.13.1 10.10.1.1 From 2b6e1ce6ee7b1ba8160bcbee97f5bbff5c46ca09 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Mon, 3 Nov 2014 13:07:41 -0800 Subject: [PATCH 1203/1492] [SPARK-4207][SQL] Query which has syntax like 'not like' is not working in Spark SQL Queries which has 'not like' is not working spark sql. sql("SELECT * FROM records where value not like 'val%'") same query works in Spark HiveQL Author: ravipesala Closes #3075 from ravipesala/SPARK-4207 and squashes the following commits: 35c11e7 [ravipesala] Supported 'not like' syntax in sql --- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 1 + .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 5 +++++ 2 files changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 00fc4d75c9ea9..5e613e0f18ba6 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -242,6 +242,7 @@ class SqlParser extends AbstractSparkSQLParser { | termExpression ~ (RLIKE ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } | termExpression ~ (REGEXP ~> termExpression) ^^ { case e1 ~ e2 => RLike(e1, e2) } | termExpression ~ (LIKE ~> termExpression) ^^ { case e1 ~ e2 => Like(e1, e2) } + | termExpression ~ (NOT ~ LIKE ~> termExpression) ^^ { case e1 ~ e2 => Not(Like(e1, e2)) } | termExpression ~ (IN ~ "(" ~> rep1sep(termExpression, ",")) <~ ")" ^^ { case e1 ~ e2 => In(e1, e2) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6bf439377aa3e..702714af5308d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -938,4 +938,9 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAnswer(sql("SELECT key FROM testData WHERE key not between 0 and 10 order by key"), (11 to 100).map(i => Seq(i))) } + + test("SPARK-4207 Query which has syntax like 'not like' is not working in Spark SQL") { + checkAnswer(sql("SELECT key FROM testData WHERE value not like '100%' order by key"), + (1 to 99).map(i => Seq(i))) + } } From 24544fbce05665ab4999a1fe5aac434d29cd912c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 3 Nov 2014 13:17:09 -0800 Subject: [PATCH 1204/1492] [SPARK-3594] [PySpark] [SQL] take more rows to infer schema or sampling This patch will try to infer schema for RDD which has empty value (None, [], {}) in the first row. It will try first 100 rows and merge the types into schema, also merge fields of StructType together. If there is still NullType in schema, then it will show an warning, tell user to try with sampling. If sampling is presented, it will infer schema from all the rows after sampling. Also, add samplingRatio for jsonFile() and jsonRDD() Author: Davies Liu Author: Davies Liu Closes #2716 from davies/infer and squashes the following commits: e678f6d [Davies Liu] Merge branch 'master' of github.com:apache/spark into infer 34b5c63 [Davies Liu] Merge branch 'master' of github.com:apache/spark into infer 567dc60 [Davies Liu] update docs 9767b27 [Davies Liu] Merge branch 'master' into infer e48d7fb [Davies Liu] fix tests 29e94d5 [Davies Liu] let NullType inherit from PrimitiveType ee5d524 [Davies Liu] Merge branch 'master' of github.com:apache/spark into infer 540d1d5 [Davies Liu] merge fields for StructType f93fd84 [Davies Liu] add more tests 3603e00 [Davies Liu] take more rows to infer schema, or infer the schema by sampling the RDD --- python/pyspark/sql.py | 196 ++++++++++++------ python/pyspark/tests.py | 19 ++ .../spark/sql/catalyst/types/dataTypes.scala | 2 +- 3 files changed, 148 insertions(+), 69 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 98e41f8575679..675df084bf303 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -109,6 +109,15 @@ def __eq__(self, other): return self is other +class NullType(PrimitiveType): + + """Spark SQL NullType + + The data type representing None, used for the types which has not + been inferred. + """ + + class StringType(PrimitiveType): """Spark SQL StringType @@ -331,7 +340,7 @@ class StructField(DataType): """ - def __init__(self, name, dataType, nullable, metadata=None): + def __init__(self, name, dataType, nullable=True, metadata=None): """Creates a StructField :param name: the name of this field. :param dataType: the data type of this field. @@ -484,6 +493,7 @@ def _parse_datatype_json_value(json_value): # Mapping Python types to Spark SQL DataType _type_mappings = { + type(None): NullType, bool: BooleanType, int: IntegerType, long: LongType, @@ -500,22 +510,22 @@ def _parse_datatype_json_value(json_value): def _infer_type(obj): """Infer the DataType from obj""" - if obj is None: - raise ValueError("Can not infer type for None") - dataType = _type_mappings.get(type(obj)) if dataType is not None: return dataType() if isinstance(obj, dict): - if not obj: - raise ValueError("Can not infer type for empty dict") - key, value = obj.iteritems().next() - return MapType(_infer_type(key), _infer_type(value), True) + for key, value in obj.iteritems(): + if key is not None and value is not None: + return MapType(_infer_type(key), _infer_type(value), True) + else: + return MapType(NullType(), NullType(), True) elif isinstance(obj, (list, array)): - if not obj: - raise ValueError("Can not infer type for empty list/array") - return ArrayType(_infer_type(obj[0]), True) + for v in obj: + if v is not None: + return ArrayType(_infer_type(obj[0]), True) + else: + return ArrayType(NullType(), True) else: try: return _infer_schema(obj) @@ -548,60 +558,93 @@ def _infer_schema(row): return StructType(fields) -def _create_converter(obj, dataType): +def _has_nulltype(dt): + """ Return whether there is NullType in `dt` or not """ + if isinstance(dt, StructType): + return any(_has_nulltype(f.dataType) for f in dt.fields) + elif isinstance(dt, ArrayType): + return _has_nulltype((dt.elementType)) + elif isinstance(dt, MapType): + return _has_nulltype(dt.keyType) or _has_nulltype(dt.valueType) + else: + return isinstance(dt, NullType) + + +def _merge_type(a, b): + if isinstance(a, NullType): + return b + elif isinstance(b, NullType): + return a + elif type(a) is not type(b): + # TODO: type cast (such as int -> long) + raise TypeError("Can not merge type %s and %s" % (a, b)) + + # same type + if isinstance(a, StructType): + nfs = dict((f.name, f.dataType) for f in b.fields) + fields = [StructField(f.name, _merge_type(f.dataType, nfs.get(f.name, NullType()))) + for f in a.fields] + names = set([f.name for f in fields]) + for n in nfs: + if n not in names: + fields.append(StructField(n, nfs[n])) + return StructType(fields) + + elif isinstance(a, ArrayType): + return ArrayType(_merge_type(a.elementType, b.elementType), True) + + elif isinstance(a, MapType): + return MapType(_merge_type(a.keyType, b.keyType), + _merge_type(a.valueType, b.valueType), + True) + else: + return a + + +def _create_converter(dataType): """Create an converter to drop the names of fields in obj """ if isinstance(dataType, ArrayType): - conv = _create_converter(obj[0], dataType.elementType) + conv = _create_converter(dataType.elementType) return lambda row: map(conv, row) elif isinstance(dataType, MapType): - value = obj.values()[0] - conv = _create_converter(value, dataType.valueType) + conv = _create_converter(dataType.valueType) return lambda row: dict((k, conv(v)) for k, v in row.iteritems()) + elif isinstance(dataType, NullType): + return lambda x: None + elif not isinstance(dataType, StructType): return lambda x: x # dataType must be StructType names = [f.name for f in dataType.fields] + converters = [_create_converter(f.dataType) for f in dataType.fields] + + def convert_struct(obj): + if obj is None: + return + + if isinstance(obj, tuple): + if hasattr(obj, "fields"): + d = dict(zip(obj.fields, obj)) + if hasattr(obj, "__FIELDS__"): + d = dict(zip(obj.__FIELDS__, obj)) + elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): + d = dict(obj) + else: + raise ValueError("unexpected tuple: %s" % obj) - if isinstance(obj, dict): - conv = lambda o: tuple(o.get(n) for n in names) - - elif isinstance(obj, tuple): - if hasattr(obj, "_fields"): # namedtuple - conv = tuple - elif hasattr(obj, "__FIELDS__"): - conv = tuple - elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): - conv = lambda o: tuple(v for k, v in o) + elif isinstance(obj, dict): + d = obj + elif hasattr(obj, "__dict__"): # object + d = obj.__dict__ else: - raise ValueError("unexpected tuple") + raise ValueError("Unexpected obj: %s" % obj) - elif hasattr(obj, "__dict__"): # object - conv = lambda o: [o.__dict__.get(n, None) for n in names] + return tuple([conv(d.get(name)) for name, conv in zip(names, converters)]) - if all(isinstance(f.dataType, PrimitiveType) for f in dataType.fields): - return conv - - row = conv(obj) - convs = [_create_converter(v, f.dataType) - for v, f in zip(row, dataType.fields)] - - def nested_conv(row): - return tuple(f(v) for f, v in zip(convs, conv(row))) - - return nested_conv - - -def _drop_schema(rows, schema): - """ all the names of fields, becoming tuples""" - iterator = iter(rows) - row = iterator.next() - converter = _create_converter(row, schema) - yield converter(row) - for i in iterator: - yield converter(i) + return convert_struct _BRACKETS = {'(': ')', '[': ']', '{': '}'} @@ -713,7 +756,7 @@ def _infer_schema_type(obj, dataType): return _infer_type(obj) if not obj: - raise ValueError("Can not infer type from empty value") + return NullType() if isinstance(dataType, ArrayType): eType = _infer_schema_type(obj[0], dataType.elementType) @@ -1049,18 +1092,20 @@ def registerFunction(self, name, f, returnType=StringType()): self._sc._javaAccumulator, returnType.json()) - def inferSchema(self, rdd): + def inferSchema(self, rdd, samplingRatio=None): """Infer and apply a schema to an RDD of L{Row}. - We peek at the first row of the RDD to determine the fields' names - and types. Nested collections are supported, which include array, - dict, list, Row, tuple, namedtuple, or object. + When samplingRatio is specified, the schema is inferred by looking + at the types of each row in the sampled dataset. Otherwise, the + first 100 rows of the RDD are inspected. Nested collections are + supported, which can include array, dict, list, Row, tuple, + namedtuple, or object. - All the rows in `rdd` should have the same type with the first one, - or it will cause runtime exceptions. + Each row could be L{pyspark.sql.Row} object or namedtuple or objects. + Using top level dicts is deprecated, as dict is used to represent Maps. - Each row could be L{pyspark.sql.Row} object or namedtuple or objects, - using dict is deprecated. + If a single column has multiple distinct inferred types, it may cause + runtime exceptions. >>> rdd = sc.parallelize( ... [Row(field1=1, field2="row1"), @@ -1097,8 +1142,23 @@ def inferSchema(self, rdd): warnings.warn("Using RDD of dict to inferSchema is deprecated," "please use pyspark.sql.Row instead") - schema = _infer_schema(first) - rdd = rdd.mapPartitions(lambda rows: _drop_schema(rows, schema)) + if samplingRatio is None: + schema = _infer_schema(first) + if _has_nulltype(schema): + for row in rdd.take(100)[1:]: + schema = _merge_type(schema, _infer_schema(row)) + if not _has_nulltype(schema): + break + else: + warnings.warn("Some of types cannot be determined by the " + "first 100 rows, please try again with sampling") + else: + if samplingRatio > 0.99: + rdd = rdd.sample(False, float(samplingRatio)) + schema = rdd.map(_infer_schema).reduce(_merge_type) + + converter = _create_converter(schema) + rdd = rdd.map(converter) return self.applySchema(rdd, schema) def applySchema(self, rdd, schema): @@ -1219,7 +1279,7 @@ def parquetFile(self, path): jschema_rdd = self._ssql_ctx.parquetFile(path).toJavaSchemaRDD() return SchemaRDD(jschema_rdd, self) - def jsonFile(self, path, schema=None): + def jsonFile(self, path, schema=None, samplingRatio=1.0): """ Loads a text file storing one JSON object per line as a L{SchemaRDD}. @@ -1227,8 +1287,8 @@ def jsonFile(self, path, schema=None): If the schema is provided, applies the given schema to this JSON dataset. - Otherwise, it goes through the entire dataset once to determine - the schema. + Otherwise, it samples the dataset with ratio `samplingRatio` to + determine the schema. >>> import tempfile, shutil >>> jsonFile = tempfile.mkdtemp() @@ -1274,20 +1334,20 @@ def jsonFile(self, path, schema=None): [Row(f1=u'row1', f2=None, f3=None)...Row(f1=u'row3', f2=[], f3=None)] """ if schema is None: - srdd = self._ssql_ctx.jsonFile(path) + srdd = self._ssql_ctx.jsonFile(path, samplingRatio) else: scala_datatype = self._ssql_ctx.parseDataType(schema.json()) srdd = self._ssql_ctx.jsonFile(path, scala_datatype) return SchemaRDD(srdd.toJavaSchemaRDD(), self) - def jsonRDD(self, rdd, schema=None): + def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): """Loads an RDD storing one JSON object per string as a L{SchemaRDD}. If the schema is provided, applies the given schema to this JSON dataset. - Otherwise, it goes through the entire dataset once to determine - the schema. + Otherwise, it samples the dataset with ratio `samplingRatio` to + determine the schema. >>> srdd1 = sqlCtx.jsonRDD(json) >>> sqlCtx.registerRDDAsTable(srdd1, "table1") @@ -1344,7 +1404,7 @@ def func(iterator): keyed._bypass_serializer = True jrdd = keyed._jrdd.map(self._jvm.BytesToString()) if schema is None: - srdd = self._ssql_ctx.jsonRDD(jrdd.rdd()) + srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), samplingRatio) else: scala_datatype = self._ssql_ctx.parseDataType(schema.json()) srdd = self._ssql_ctx.jsonRDD(jrdd.rdd(), scala_datatype) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 253a471849c3a..68fd756876219 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -796,6 +796,25 @@ def test_serialize_nested_array_and_map(self): self.assertEqual(1.0, row.c) self.assertEqual("2", row.d) + def test_infer_schema(self): + d = [Row(l=[], d={}), + Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}, s="")] + rdd = self.sc.parallelize(d) + srdd = self.sqlCtx.inferSchema(rdd) + self.assertEqual([], srdd.map(lambda r: r.l).first()) + self.assertEqual([None, ""], srdd.map(lambda r: r.s).collect()) + srdd.registerTempTable("test") + result = self.sqlCtx.sql("SELECT l[0].a from test where d['key'].d = '2'") + self.assertEqual(1, result.first()[0]) + + srdd2 = self.sqlCtx.inferSchema(rdd, 1.0) + self.assertEqual(srdd.schema(), srdd2.schema()) + self.assertEqual({}, srdd2.map(lambda r: r.d).first()) + self.assertEqual([None, ""], srdd2.map(lambda r: r.s).collect()) + srdd2.registerTempTable("test2") + result = self.sqlCtx.sql("SELECT l[0].a from test2 where d['key'].d = '2'") + self.assertEqual(1, result.first()[0]) + def test_convert_row_to_dict(self): row = Row(l=[Row(a=1, b='s')], d={"key": Row(c=1.0, d="2")}) self.assertEqual(1, row.asDict()['l'][0].a) 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 cc5015ad3c013..e1b5992a36e5f 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 @@ -213,7 +213,7 @@ trait PrimitiveType extends DataType { } object PrimitiveType { - private val nonDecimals = Seq(DateType, TimestampType, BinaryType) ++ NativeType.all + private val nonDecimals = Seq(NullType, DateType, TimestampType, BinaryType) ++ NativeType.all private val nonDecimalNameToType = nonDecimals.map(t => t.typeName -> t).toMap /** Given the string representation of a type, return its DataType */ From c238fb423d1011bd1b1e6201d769b72e52664fc6 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 3 Nov 2014 13:20:33 -0800 Subject: [PATCH 1205/1492] [SPARK-4202][SQL] Simple DSL support for Scala UDF This feature is based on an offline discussion with mengxr, hopefully can be useful for the new MLlib pipeline API. For the following test snippet ```scala case class KeyValue(key: Int, value: String) val testData = sc.parallelize(1 to 10).map(i => KeyValue(i, i.toString)).toSchemaRDD def foo(a: Int, b: String) => a.toString + b ``` the newly introduced DSL enables the following syntax ```scala import org.apache.spark.sql.catalyst.dsl._ testData.select(Star(None), foo.call('key, 'value) as 'result) ``` which is equivalent to ```scala testData.registerTempTable("testData") sqlContext.registerFunction("foo", foo) sql("SELECT *, foo(key, value) AS result FROM testData") ``` Author: Cheng Lian Closes #3067 from liancheng/udf-dsl and squashes the following commits: f132818 [Cheng Lian] Adds DSL support for Scala UDF --- .../spark/sql/catalyst/dsl/package.scala | 59 +++++++++++++++++++ .../org/apache/spark/sql/DslQuerySuite.scala | 17 ++++-- 2 files changed, 72 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 7e6d770314f5a..3314e15477016 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -22,6 +22,7 @@ import java.sql.{Date, Timestamp} import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.language.implicitConversions +import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ @@ -285,4 +286,62 @@ package object dsl { def writeToFile(path: String) = WriteToFile(path, logicalPlan) } } + + case class ScalaUdfBuilder[T: TypeTag](f: AnyRef) { + def call(args: Expression*) = ScalaUdf(f, ScalaReflection.schemaFor(typeTag[T]).dataType, args) + } + + // scalastyle:off + /** functionToUdfBuilder 1-22 were generated by this script + + (1 to 22).map { x => + val argTypes = Seq.fill(x)("_").mkString(", ") + s"implicit def functionToUdfBuilder[T: TypeTag](func: Function$x[$argTypes, T]) = ScalaUdfBuilder(func)" + } + */ + + implicit def functionToUdfBuilder[T: TypeTag](func: Function1[_, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function2[_, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function3[_, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function4[_, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function5[_, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function6[_, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function7[_, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function8[_, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function9[_, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function10[_, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function11[_, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function12[_, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function13[_, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + + implicit def functionToUdfBuilder[T: TypeTag](func: Function22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, T]) = ScalaUdfBuilder(func) + // scalastyle:on } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 45e58afe9d9a2..e70ad891eea36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -19,14 +19,13 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.test._ /* Implicits */ -import TestSQLContext._ +import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.test.TestSQLContext._ class DslQuerySuite extends QueryTest { - import TestData._ + import org.apache.spark.sql.TestData._ test("table scan") { checkAnswer( @@ -216,4 +215,14 @@ class DslQuerySuite extends QueryTest { (4, "d") :: Nil) checkAnswer(lowerCaseData.intersect(upperCaseData), Nil) } + + test("udf") { + val foo = (a: Int, b: String) => a.toString + b + + checkAnswer( + // SELECT *, foo(key, value) FROM testData + testData.select(Star(None), foo.call('key, 'value)).limit(3), + (1, "1", "11") :: (2, "2", "22") :: (3, "3", "33") :: Nil + ) + } } From e83f13e8d37ca33f4e183e977d077221b90c6025 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 3 Nov 2014 13:59:43 -0800 Subject: [PATCH 1206/1492] [SPARK-4152] [SQL] Avoid data change in CTAS while table already existed MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit CREATE TABLE t1 (a String); CREATE TABLE t1 AS SELECT key FROM src; – throw exception CREATE TABLE if not exists t1 AS SELECT key FROM src; – expect do nothing, currently it will overwrite the t1, which is incorrect. Author: Cheng Hao Closes #3013 from chenghao-intel/ctas_unittest and squashes the following commits: 194113e [Cheng Hao] fix bug in CTAS when table already existed --- .../spark/sql/catalyst/analysis/Catalog.scala | 22 +++++++++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 +++++ .../hive/execution/CreateTableAsSelect.scala | 12 +++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 9 ++++++-- 4 files changed, 46 insertions(+), 3 deletions(-) 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 2059a91ba0612..0415d74bd8141 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 @@ -28,6 +28,8 @@ trait Catalog { def caseSensitive: Boolean + def tableExists(db: Option[String], tableName: String): Boolean + def lookupRelation( databaseName: Option[String], tableName: String, @@ -82,6 +84,14 @@ class SimpleCatalog(val caseSensitive: Boolean) extends Catalog { tables.clear() } + override def tableExists(db: Option[String], tableName: String): Boolean = { + val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + tables.get(tblName) match { + case Some(_) => true + case None => false + } + } + override def lookupRelation( databaseName: Option[String], tableName: String, @@ -107,6 +117,14 @@ trait OverrideCatalog extends Catalog { // TODO: This doesn't work when the database changes... val overrides = new mutable.HashMap[(Option[String],String), LogicalPlan]() + abstract override def tableExists(db: Option[String], tableName: String): Boolean = { + val (dbName, tblName) = processDatabaseAndTableName(db, tableName) + overrides.get((dbName, tblName)) match { + case Some(_) => true + case None => super.tableExists(db, tableName) + } + } + abstract override def lookupRelation( databaseName: Option[String], tableName: String, @@ -149,6 +167,10 @@ object EmptyCatalog extends Catalog { val caseSensitive: Boolean = true + def tableExists(db: Option[String], tableName: String): Boolean = { + throw new UnsupportedOperationException + } + def lookupRelation( databaseName: Option[String], tableName: String, 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 096b4a07aa2ea..0baf4c9f8c7ab 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 @@ -57,6 +57,12 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val caseSensitive: Boolean = false + def tableExists(db: Option[String], tableName: String): Boolean = { + val (databaseName, tblName) = processDatabaseAndTableName( + db.getOrElse(hive.sessionState.getCurrentDatabase), tableName) + client.getTable(databaseName, tblName, false) != null + } + def lookupRelation( db: Option[String], tableName: String, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 2fce414734579..3d24d87bc3d38 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -71,7 +71,17 @@ case class CreateTableAsSelect( // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data // processing. - sc.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd + if (sc.catalog.tableExists(Some(database), tableName)) { + if (allowExisting) { + // table already exists, will do nothing, to keep consistent with Hive + } else { + throw + new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName") + } + } else { + sc.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd + } + Seq.empty[Row] } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index 76a0ec01a6075..e9b1943ff8db7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -56,7 +56,7 @@ class SQLQuerySuite extends QueryTest { sql( """CREATE TABLE IF NOT EXISTS ctas4 AS | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin).collect - // expect the string => integer for field key cause the table ctas4 already existed. + // do nothing cause the table ctas4 already existed. sql( """CREATE TABLE IF NOT EXISTS ctas4 AS | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect @@ -78,9 +78,14 @@ class SQLQuerySuite extends QueryTest { SELECT key, value FROM src ORDER BY key, value""").collect().toSeq) + intercept[org.apache.hadoop.hive.metastore.api.AlreadyExistsException] { + sql( + """CREATE TABLE ctas4 AS + | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect + } checkAnswer( sql("SELECT key, value FROM ctas4 ORDER BY key, value"), - sql("SELECT CAST(key AS int) k, value FROM src ORDER BY k, value").collect().toSeq) + sql("SELECT key, value FROM ctas4 LIMIT 1").collect().toSeq) checkExistence(sql("DESC EXTENDED ctas2"), true, "name:key", "type:string", "name:value", "ctas2", From 25bef7e6951301e93004567fc0cef96bf8d1a224 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Nov 2014 14:08:27 -0800 Subject: [PATCH 1207/1492] [SQL] More aggressive defaults - Turns on compression for in-memory cached data by default - Changes the default parquet compression format back to gzip (we have seen more OOMs with production workloads due to the way Snappy allocates memory) - Ups the batch size to 10,000 rows - Increases the broadcast threshold to 10mb. - Uses our parquet implementation instead of the hive one by default. - Cache parquet metadata by default. Author: Michael Armbrust Closes #3064 from marmbrus/fasterDefaults and squashes the following commits: 97ee9f8 [Michael Armbrust] parquet codec docs e641694 [Michael Armbrust] Remote also a12866a [Michael Armbrust] Cache metadata. 2d73acc [Michael Armbrust] Update docs defaults. d63d2d5 [Michael Armbrust] document parquet option da373f9 [Michael Armbrust] More aggressive defaults --- docs/sql-programming-guide.md | 18 +++++++++++++----- .../scala/org/apache/spark/sql/SQLConf.scala | 10 +++++----- .../sql/parquet/ParquetTableOperations.scala | 6 +++--- .../apache/spark/sql/hive/HiveContext.scala | 2 +- 4 files changed, 22 insertions(+), 14 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d4ade939c3a6e..e399fecbbc78c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -582,19 +582,27 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or spark.sql.parquet.cacheMetadata - false + true Turns on caching of Parquet schema metadata. Can speed up querying of static data. spark.sql.parquet.compression.codec - snappy + gzip Sets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo. + + spark.sql.hive.convertMetastoreParquet + true + + When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in + support. + + ## JSON Datasets @@ -815,7 +823,7 @@ Configuration of in-memory caching can be done using the `setConf` method on SQL Property NameDefaultMeaning spark.sql.inMemoryColumnarStorage.compressed - false + true When set to true Spark SQL will automatically select a compression codec for each column based on statistics of the data. @@ -823,7 +831,7 @@ Configuration of in-memory caching can be done using the `setConf` method on SQL spark.sql.inMemoryColumnarStorage.batchSize - 1000 + 10000 Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization and compression, but risk OOMs when caching data. @@ -841,7 +849,7 @@ that these options will be deprecated in future release as more optimizations ar Property NameDefaultMeaning spark.sql.autoBroadcastJoinThreshold - 10000 + 10485760 (10 MB) Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 07e6e2eccddf4..279495aa64755 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -79,13 +79,13 @@ private[sql] trait SQLConf { private[spark] def dialect: String = getConf(DIALECT, "sql") /** When true tables cached using the in-memory columnar caching will be compressed. */ - private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "false").toBoolean + private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "true").toBoolean /** The compression codec for writing to a Parquetfile */ - private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "snappy") + private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "gzip") /** The number of rows that will be */ - private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt + private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "10000").toInt /** Number of partitions to use for shuffle operators. */ private[spark] def numShufflePartitions: Int = getConf(SHUFFLE_PARTITIONS, "200").toInt @@ -106,10 +106,10 @@ private[sql] trait SQLConf { * a broadcast value during the physical executions of join operations. Setting this to -1 * effectively disables auto conversion. * - * Hive setting: hive.auto.convert.join.noconditionaltask.size, whose default value is also 10000. + * Hive setting: hive.auto.convert.join.noconditionaltask.size, whose default value is 10000. */ private[spark] def autoBroadcastJoinThreshold: Int = - getConf(AUTO_BROADCASTJOIN_THRESHOLD, "10000").toInt + getConf(AUTO_BROADCASTJOIN_THRESHOLD, (10 * 1024 * 1024).toString).toInt /** * The default size in bytes to assign to a logical operator's estimation statistics. By default, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 9664c565a0b86..d00860a8bb8a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -123,7 +123,7 @@ case class ParquetTableScan( // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata conf.set( SQLConf.PARQUET_CACHE_METADATA, - sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "false")) + sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA, "true")) val baseRDD = new org.apache.spark.rdd.NewHadoopRDD( @@ -394,7 +394,7 @@ private[parquet] class FilteringParquetRowInputFormat if (footers eq null) { val conf = ContextUtil.getConfiguration(jobContext) - val cacheMetadata = conf.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) + val cacheMetadata = conf.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true) val statuses = listStatus(jobContext) fileStatuses = statuses.map(file => file.getPath -> file).toMap if (statuses.isEmpty) { @@ -493,7 +493,7 @@ private[parquet] class FilteringParquetRowInputFormat import parquet.filter2.compat.FilterCompat.Filter; import parquet.filter2.compat.RowGroupFilter; - val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, false) + val cacheMetadata = configuration.getBoolean(SQLConf.PARQUET_CACHE_METADATA, true) val splits = mutable.ArrayBuffer.empty[ParquetInputSplit] val filter: Filter = ParquetInputFormat.getFilter(configuration) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index f025169ad5063..e88afaaf001c0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -90,7 +90,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { * SerDe. */ private[spark] def convertMetastoreParquet: Boolean = - getConf("spark.sql.hive.convertMetastoreParquet", "false") == "true" + getConf("spark.sql.hive.convertMetastoreParquet", "true") == "true" override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } From 28128150e7e0c2b7d1c483e67214bdaef59f7d75 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 3 Nov 2014 15:19:01 -0800 Subject: [PATCH 1208/1492] SPARK-4178. Hadoop input metrics ignore bytes read in RecordReader insta... ...ntiation Author: Sandy Ryza Closes #3045 from sryza/sandy-spark-4178 and squashes the following commits: 8d2e70e [Sandy Ryza] Kostas's review feedback e5b27c0 [Sandy Ryza] SPARK-4178. Hadoop input metrics ignore bytes read in RecordReader instantiation --- .../org/apache/spark/rdd/HadoopRDD.scala | 25 +++++++++-------- .../org/apache/spark/rdd/NewHadoopRDD.scala | 26 +++++++++--------- .../spark/metrics/InputMetricsSuite.scala | 27 +++++++++++++++++-- 3 files changed, 53 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 946fb5616d3ec..a157e36e2286e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -211,20 +211,11 @@ class HadoopRDD[K, V]( val split = theSplit.asInstanceOf[HadoopPartition] logInfo("Input split: " + split.inputSplit) - var reader: RecordReader[K, V] = null val jobConf = getJobConf() - val inputFormat = getInputFormat(jobConf) - HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), - context.stageId, theSplit.index, context.attemptId.toInt, jobConf) - reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) - - // Register an on-task-completion callback to close the input stream. - context.addTaskCompletionListener{ context => closeIfNeeded() } - val key: K = reader.createKey() - val value: V = reader.createValue() val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - // Find a function that will return the FileSystem bytes read by this thread. + // Find a function that will return the FileSystem bytes read by this thread. Do this before + // creating RecordReader, because RecordReader's constructor might read some bytes val bytesReadCallback = if (split.inputSplit.value.isInstanceOf[FileSplit]) { SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( split.inputSplit.value.asInstanceOf[FileSplit].getPath, jobConf) @@ -234,6 +225,18 @@ class HadoopRDD[K, V]( if (bytesReadCallback.isDefined) { context.taskMetrics.inputMetrics = Some(inputMetrics) } + + var reader: RecordReader[K, V] = null + val inputFormat = getInputFormat(jobConf) + HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), + context.stageId, theSplit.index, context.attemptId.toInt, jobConf) + reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) + + // Register an on-task-completion callback to close the input stream. + context.addTaskCompletionListener{ context => closeIfNeeded() } + val key: K = reader.createKey() + val value: V = reader.createValue() + var recordsSinceMetricsUpdate = 0 override def getNext() = { diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 6d6b86721ca74..351e145f96f9a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -107,20 +107,10 @@ class NewHadoopRDD[K, V]( val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) - val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) - val format = inputFormatClass.newInstance - format match { - case configurable: Configurable => - configurable.setConf(conf) - case _ => - } - val reader = format.createRecordReader( - split.serializableHadoopSplit.value, hadoopAttemptContext) - reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) val inputMetrics = new InputMetrics(DataReadMethod.Hadoop) - // Find a function that will return the FileSystem bytes read by this thread. + // Find a function that will return the FileSystem bytes read by this thread. Do this before + // creating RecordReader, because RecordReader's constructor might read some bytes val bytesReadCallback = if (split.serializableHadoopSplit.value.isInstanceOf[FileSplit]) { SparkHadoopUtil.get.getFSBytesReadOnThreadCallback( split.serializableHadoopSplit.value.asInstanceOf[FileSplit].getPath, conf) @@ -131,6 +121,18 @@ class NewHadoopRDD[K, V]( context.taskMetrics.inputMetrics = Some(inputMetrics) } + val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) + val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) + val format = inputFormatClass.newInstance + format match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val reader = format.createRecordReader( + split.serializableHadoopSplit.value, hadoopAttemptContext) + reader.initialize(split.serializableHadoopSplit.value, hadoopAttemptContext) + // Register an on-task-completion callback to close the input stream. context.addTaskCompletionListener(context => close()) var havePair = false diff --git a/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala index 33bd1afea2470..48c386ba04311 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputMetricsSuite.scala @@ -27,7 +27,7 @@ import scala.collection.mutable.ArrayBuffer import java.io.{FileWriter, PrintWriter, File} class InputMetricsSuite extends FunSuite with SharedSparkContext { - test("input metrics when reading text file") { + test("input metrics when reading text file with single split") { val file = new File(getClass.getSimpleName + ".txt") val pw = new PrintWriter(new FileWriter(file)) pw.println("some stuff") @@ -48,6 +48,29 @@ class InputMetricsSuite extends FunSuite with SharedSparkContext { // Wait for task end events to come in sc.listenerBus.waitUntilEmpty(500) assert(taskBytesRead.length == 2) - assert(taskBytesRead.sum == file.length()) + assert(taskBytesRead.sum >= file.length()) + } + + test("input metrics when reading text file with multiple splits") { + val file = new File(getClass.getSimpleName + ".txt") + val pw = new PrintWriter(new FileWriter(file)) + for (i <- 0 until 10000) { + pw.println("some stuff") + } + pw.close() + file.deleteOnExit() + + val taskBytesRead = new ArrayBuffer[Long]() + sc.addSparkListener(new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + taskBytesRead += taskEnd.taskMetrics.inputMetrics.get.bytesRead + } + }) + sc.textFile("file://" + file.getAbsolutePath, 2).count() + + // Wait for task end events to come in + sc.listenerBus.waitUntilEmpty(500) + assert(taskBytesRead.length == 2) + assert(taskBytesRead.sum >= file.length()) } } From 15b58a2234ab7ba30c9c0cbb536177a3c725e350 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 3 Nov 2014 18:04:51 -0800 Subject: [PATCH 1209/1492] [SQL] Convert arguments to Scala UDFs Author: Michael Armbrust Closes #3077 from marmbrus/udfsWithUdts and squashes the following commits: 34b5f27 [Michael Armbrust] style 504adef [Michael Armbrust] Convert arguments to Scala UDFs --- .../sql/catalyst/expressions/ScalaUdf.scala | 560 ++++++++++-------- .../spark/sql/UserDefinedTypeSuite.scala | 18 +- 2 files changed, 316 insertions(+), 262 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index fa1786e74bb3e..18c96da2f87fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -34,320 +34,366 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi override def toString = s"scalaUDF(${children.mkString(",")})" + // scalastyle:off + /** This method has been generated by this script (1 to 22).map { x => val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) - val evals = (0 to x - 1).map(x => s"children($x).eval(input)").reduce(_ + ",\n " + _) + val evals = (0 to x - 1).map(x => s" ScalaReflection.convertToScala(children($x).eval(input), children($x).dataType)").reduce(_ + ",\n " + _) s""" case $x => function.asInstanceOf[($anys) => Any]( - $evals) + $evals) """ - } + }.foreach(println) */ - // scalastyle:off override def eval(input: Row): Any = { val result = children.size match { case 0 => function.asInstanceOf[() => Any]() - case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) + case 1 => + function.asInstanceOf[(Any) => Any]( + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType)) + + case 2 => function.asInstanceOf[(Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType)) + + case 3 => function.asInstanceOf[(Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType)) + + case 4 => function.asInstanceOf[(Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType)) + + case 5 => function.asInstanceOf[(Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType)) + + case 6 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType)) + + case 7 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType)) + + case 8 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType)) + + case 9 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType)) + + case 10 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType)) + + case 11 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType)) + + case 12 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType)) + + case 13 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType)) + + case 14 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType)) + + case 15 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType)) + + case 16 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input), - children(15).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), + ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType)) + + case 17 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input), - children(15).eval(input), - children(16).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), + ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), + ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType)) + + case 18 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input), - children(15).eval(input), - children(16).eval(input), - children(17).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), + ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), + ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), + ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType)) + + case 19 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input), - children(15).eval(input), - children(16).eval(input), - children(17).eval(input), - children(18).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), + ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), + ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), + ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), + ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType)) + + case 20 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input), - children(15).eval(input), - children(16).eval(input), - children(17).eval(input), - children(18).eval(input), - children(19).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), + ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), + ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), + ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), + ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), + ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType)) + + case 21 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input), - children(15).eval(input), - children(16).eval(input), - children(17).eval(input), - children(18).eval(input), - children(19).eval(input), - children(20).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), + ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), + ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), + ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), + ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), + ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType), + ScalaReflection.convertToScala(children(20).eval(input), children(20).dataType)) + + case 22 => function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any]( - children(0).eval(input), - children(1).eval(input), - children(2).eval(input), - children(3).eval(input), - children(4).eval(input), - children(5).eval(input), - children(6).eval(input), - children(7).eval(input), - children(8).eval(input), - children(9).eval(input), - children(10).eval(input), - children(11).eval(input), - children(12).eval(input), - children(13).eval(input), - children(14).eval(input), - children(15).eval(input), - children(16).eval(input), - children(17).eval(input), - children(18).eval(input), - children(19).eval(input), - children(20).eval(input), - children(21).eval(input)) + ScalaReflection.convertToScala(children(0).eval(input), children(0).dataType), + ScalaReflection.convertToScala(children(1).eval(input), children(1).dataType), + ScalaReflection.convertToScala(children(2).eval(input), children(2).dataType), + ScalaReflection.convertToScala(children(3).eval(input), children(3).dataType), + ScalaReflection.convertToScala(children(4).eval(input), children(4).dataType), + ScalaReflection.convertToScala(children(5).eval(input), children(5).dataType), + ScalaReflection.convertToScala(children(6).eval(input), children(6).dataType), + ScalaReflection.convertToScala(children(7).eval(input), children(7).dataType), + ScalaReflection.convertToScala(children(8).eval(input), children(8).dataType), + ScalaReflection.convertToScala(children(9).eval(input), children(9).dataType), + ScalaReflection.convertToScala(children(10).eval(input), children(10).dataType), + ScalaReflection.convertToScala(children(11).eval(input), children(11).dataType), + ScalaReflection.convertToScala(children(12).eval(input), children(12).dataType), + ScalaReflection.convertToScala(children(13).eval(input), children(13).dataType), + ScalaReflection.convertToScala(children(14).eval(input), children(14).dataType), + ScalaReflection.convertToScala(children(15).eval(input), children(15).dataType), + ScalaReflection.convertToScala(children(16).eval(input), children(16).dataType), + ScalaReflection.convertToScala(children(17).eval(input), children(17).dataType), + ScalaReflection.convertToScala(children(18).eval(input), children(18).dataType), + ScalaReflection.convertToScala(children(19).eval(input), children(19).dataType), + ScalaReflection.convertToScala(children(20).eval(input), children(20).dataType), + ScalaReflection.convertToScala(children(21).eval(input), children(21).dataType)) + } // scalastyle:on diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 666235e57f812..1806a1dd82023 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -60,13 +60,13 @@ private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { } class UserDefinedTypeSuite extends QueryTest { + val points = Seq( + MyLabeledPoint(1.0, new MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new MyDenseVector(Array(0.2, 2.0)))) + val pointsRDD: RDD[MyLabeledPoint] = sparkContext.parallelize(points) - test("register user type: MyDenseVector for MyLabeledPoint") { - val points = Seq( - MyLabeledPoint(1.0, new MyDenseVector(Array(0.1, 1.0))), - MyLabeledPoint(0.0, new MyDenseVector(Array(0.2, 2.0)))) - val pointsRDD: RDD[MyLabeledPoint] = sparkContext.parallelize(points) + test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).map { case Row(v: Double) => v } val labelsArrays: Array[Double] = labels.collect() assert(labelsArrays.size === 2) @@ -80,4 +80,12 @@ class UserDefinedTypeSuite extends QueryTest { assert(featuresArrays.contains(new MyDenseVector(Array(0.1, 1.0)))) assert(featuresArrays.contains(new MyDenseVector(Array(0.2, 2.0)))) } + + test("UDTs and UDFs") { + registerFunction("testType", (d: MyDenseVector) => d.isInstanceOf[MyDenseVector]) + pointsRDD.registerTempTable("points") + checkAnswer( + sql("SELECT testType(features) from points"), + Seq(Row(true), Row(true))) + } } From 97a466eca0a629f17e9662ca2b59eeca99142c54 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Mon, 3 Nov 2014 18:17:32 -0800 Subject: [PATCH 1210/1492] [SPARK-4168][WebUI] web statges number should show correctly when stages are more than 1000 The number of completed stages and failed stages showed on webUI will always be less than 1000. This is really misleading when there are already thousands of stages completed or failed. The number should be correct even when only partial stages listed on the webUI (stage info will be removed if the number is too large). Author: Zhang, Liye Closes #3035 from liyezhang556520/webStageNum and squashes the following commits: d9e29fb [Zhang, Liye] add detailed comments for variables 4ea8fd1 [Zhang, Liye] change variable name accroding to comments f4c404d [Zhang, Liye] [SPARK-4168][WebUI] web statges number should show correctly when stages are more than 1000 --- .../org/apache/spark/ui/jobs/JobProgressListener.scala | 9 +++++++++ .../org/apache/spark/ui/jobs/JobProgressPage.scala | 10 ++++++---- 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index b5207360510dd..e3223403c17f4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -59,6 +59,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val failedStages = ListBuffer[StageInfo]() val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] val stageIdToInfo = new HashMap[StageId, StageInfo] + + // Number of completed and failed stages, may not actually equal to completedStages.size and + // failedStages.size respectively due to completedStage and failedStages only maintain the latest + // part of the stages, the earlier ones will be removed when there are too many stages for + // memory sake. + var numCompletedStages = 0 + var numFailedStages = 0 // Map from pool name to a hash map (map from stage id to StageInfo). val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() @@ -110,9 +117,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { activeStages.remove(stage.stageId) if (stage.failureReason.isEmpty) { completedStages += stage + numCompletedStages += 1 trimIfNecessary(completedStages) } else { failedStages += stage + numFailedStages += 1 trimIfNecessary(failedStages) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 6e718eecdd52a..83a7898071c9b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -34,7 +34,9 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") listener.synchronized { val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq + val numCompletedStages = listener.numCompletedStages val failedStages = listener.failedStages.reverse.toSeq + val numFailedStages = listener.numFailedStages val now = System.currentTimeMillis val activeStagesTable = @@ -69,11 +71,11 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")
  • Completed Stages: - {completedStages.size} + {numCompletedStages}
  • Failed Stages: - {failedStages.size} + {numFailedStages}
  • @@ -86,9 +88,9 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") }} ++

    Active Stages ({activeStages.size})

    ++ activeStagesTable.toNodeSeq ++ -

    Completed Stages ({completedStages.size})

    ++ +

    Completed Stages ({numCompletedStages})

    ++ completedStagesTable.toNodeSeq ++ -

    Failed Stages ({failedStages.size})

    ++ +

    Failed Stages ({numFailedStages})

    ++ failedStagesTable.toNodeSeq UIUtils.headerSparkPage("Spark Stages", content, parent) From 4f035dd2cd6f1ec9059811f3495f3e0a8ec5fb84 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 3 Nov 2014 18:18:47 -0800 Subject: [PATCH 1211/1492] [SPARK-611] Display executor thread dumps in web UI This patch allows executor thread dumps to be collected on-demand and viewed in the Spark web UI. The thread dumps are collected using Thread.getAllStackTraces(). To allow remote thread dumps to be triggered from the web UI, I added a new `ExecutorActor` that runs inside of the Executor actor system and responds to RPCs from the driver. The driver's mechanism for obtaining a reference to this actor is a little bit hacky: it uses the block manager master actor to determine the host/port of the executor actor systems in order to construct ActorRefs to ExecutorActor. Unfortunately, I couldn't find a much cleaner way to do this without a big refactoring of the executor -> driver communication. Screenshots: ![image](https://cloud.githubusercontent.com/assets/50748/4781793/7e7a0776-5cbf-11e4-874d-a91cd04620bd.png) ![image](https://cloud.githubusercontent.com/assets/50748/4781794/8bce76aa-5cbf-11e4-8d13-8477748c9f7e.png) ![image](https://cloud.githubusercontent.com/assets/50748/4781797/bd11a8b8-5cbf-11e4-9ad7-a7459467ec8e.png) Author: Josh Rosen Closes #2944 from JoshRosen/jstack-in-web-ui and squashes the following commits: 3c21a5d [Josh Rosen] Address review comments: 880f7f7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui f719266 [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui 19707b0 [Josh Rosen] Add one comment. 127a130 [Josh Rosen] Update to use SparkContext.DRIVER_IDENTIFIER b8e69aa [Josh Rosen] Merge remote-tracking branch 'origin/master' into jstack-in-web-ui 3dfc2d4 [Josh Rosen] Add missing file. bc1e675 [Josh Rosen] Undo some leftover changes from the earlier approach. f4ac1c1 [Josh Rosen] Switch to on-demand collection of thread dumps dfec08b [Josh Rosen] Add option to disable thread dumps in UI. 4c87d7f [Josh Rosen] Use separate RPC for sending thread dumps. 2b8bdf3 [Josh Rosen] Enable thread dumps from the driver when running in non-local mode. cc3e6b3 [Josh Rosen] Fix test code in DAGSchedulerSuite. 87b8b65 [Josh Rosen] Add new listener event for thread dumps. 8c10216 [Josh Rosen] Add missing file. 0f198ac [Josh Rosen] [SPARK-611] Display executor thread dumps in web UI --- .../scala/org/apache/spark/SparkContext.scala | 29 +++++++- .../CoarseGrainedExecutorBackend.scala | 3 +- .../org/apache/spark/executor/Executor.scala | 7 +- .../apache/spark/executor/ExecutorActor.scala | 41 +++++++++++ .../spark/storage/BlockManagerMaster.scala | 4 + .../storage/BlockManagerMasterActor.scala | 18 +++++ .../spark/storage/BlockManagerMessages.scala | 2 + .../ui/exec/ExecutorThreadDumpPage.scala | 73 +++++++++++++++++++ .../apache/spark/ui/exec/ExecutorsPage.scala | 15 +++- .../apache/spark/ui/exec/ExecutorsTab.scala | 8 +- .../org/apache/spark/util/AkkaUtils.scala | 14 ++++ .../apache/spark/util/ThreadStackTrace.scala | 27 +++++++ .../scala/org/apache/spark/util/Utils.scala | 13 ++++ 13 files changed, 247 insertions(+), 7 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala create mode 100644 core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8b4db783979ec..40444c237b738 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -21,9 +21,8 @@ import scala.language.implicitConversions import java.io._ import java.net.URI -import java.util.Arrays +import java.util.{Arrays, Properties, UUID} import java.util.concurrent.atomic.AtomicInteger -import java.util.{Properties, UUID} import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable @@ -41,6 +40,7 @@ import akka.actor.Props import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import org.apache.spark.executor.TriggerThreadDump import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ @@ -51,7 +51,7 @@ import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage._ import org.apache.spark.ui.SparkUI import org.apache.spark.ui.jobs.JobProgressListener -import org.apache.spark.util.{CallSite, ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} +import org.apache.spark.util._ /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -361,6 +361,29 @@ class SparkContext(config: SparkConf) extends SparkStatusAPI with Logging { override protected def childValue(parent: Properties): Properties = new Properties(parent) } + /** + * Called by the web UI to obtain executor thread dumps. This method may be expensive. + * Logs an error and returns None if we failed to obtain a thread dump, which could occur due + * to an executor being dead or unresponsive or due to network issues while sending the thread + * dump message back to the driver. + */ + private[spark] def getExecutorThreadDump(executorId: String): Option[Array[ThreadStackTrace]] = { + try { + if (executorId == SparkContext.DRIVER_IDENTIFIER) { + Some(Utils.getThreadDump()) + } else { + val (host, port) = env.blockManager.master.getActorSystemHostPortForExecutor(executorId).get + val actorRef = AkkaUtils.makeExecutorRef("ExecutorActor", conf, host, port, env.actorSystem) + Some(AkkaUtils.askWithReply[Array[ThreadStackTrace]](TriggerThreadDump, actorRef, + AkkaUtils.numRetries(conf), AkkaUtils.retryWaitMs(conf), AkkaUtils.askTimeout(conf))) + } + } catch { + case e: Exception => + logError(s"Exception getting thread dump from executor $executorId", e) + None + } + } + private[spark] def getLocalProperties: Properties = localProperties.get() private[spark] def setLocalProperties(props: Properties) { diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 697154d762d41..3711824a40cfc 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -131,7 +131,8 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // Create a new ActorSystem using driver's Spark properties to run the backend. val driverConf = new SparkConf().setAll(props) val (actorSystem, boundPort) = AkkaUtils.createActorSystem( - "sparkExecutor", hostname, port, driverConf, new SecurityManager(driverConf)) + SparkEnv.executorActorSystemName, + hostname, port, driverConf, new SecurityManager(driverConf)) // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e24a15f015e1c..8b095e23f32ff 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -26,7 +26,7 @@ import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.util.control.NonFatal -import akka.actor.ActorSystem +import akka.actor.{Props, ActorSystem} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil @@ -92,6 +92,10 @@ private[spark] class Executor( } } + // Create an actor for receiving RPCs from the driver + private val executorActor = env.actorSystem.actorOf( + Props(new ExecutorActor(executorId)), "ExecutorActor") + // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager private val urlClassLoader = createClassLoader() @@ -131,6 +135,7 @@ private[spark] class Executor( def stop() { env.metricsSystem.report() + env.actorSystem.stop(executorActor) isStopped = true threadPool.shutdown() if (!isLocal) { diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala new file mode 100644 index 0000000000000..41925f7e97e84 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorActor.scala @@ -0,0 +1,41 @@ +/* + * 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.executor + +import akka.actor.Actor +import org.apache.spark.Logging + +import org.apache.spark.util.{Utils, ActorLogReceive} + +/** + * Driver -> Executor message to trigger a thread dump. + */ +private[spark] case object TriggerThreadDump + +/** + * Actor that runs inside of executors to enable driver -> executor RPC. + */ +private[spark] +class ExecutorActor(executorId: String) extends Actor with ActorLogReceive with Logging { + + override def receiveWithLogging = { + case TriggerThreadDump => + sender ! Utils.getThreadDump() + } + +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index d08e1419e3e41..b63c7f191155c 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -88,6 +88,10 @@ class BlockManagerMaster( askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId)) } + def getActorSystemHostPortForExecutor(executorId: String): Option[(String, Int)] = { + askDriverWithReply[Option[(String, Int)]](GetActorSystemHostPortForExecutor(executorId)) + } + /** * Remove a block from the slaves that have it. This can only be used to remove * blocks that the driver knows about. 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 5e375a2553979..685b2e11440fb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -86,6 +86,9 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case GetPeers(blockManagerId) => sender ! getPeers(blockManagerId) + case GetActorSystemHostPortForExecutor(executorId) => + sender ! getActorSystemHostPortForExecutor(executorId) + case GetMemoryStatus => sender ! memoryStatus @@ -412,6 +415,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus Seq.empty } } + + /** + * Returns the hostname and port of an executor's actor system, based on the Akka address of its + * BlockManagerSlaveActor. + */ + private def getActorSystemHostPortForExecutor(executorId: String): Option[(String, Int)] = { + for ( + blockManagerId <- blockManagerIdByExecutor.get(executorId); + info <- blockManagerInfo.get(blockManagerId); + host <- info.slaveActor.path.address.host; + port <- info.slaveActor.path.address.port + ) yield { + (host, port) + } + } } @DeveloperApi diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 291ddfcc113ac..3f32099d08cc9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -92,6 +92,8 @@ private[spark] object BlockManagerMessages { case class GetPeers(blockManagerId: BlockManagerId) extends ToBlockManagerMaster + case class GetActorSystemHostPortForExecutor(executorId: String) extends ToBlockManagerMaster + case class RemoveExecutor(execId: String) extends ToBlockManagerMaster case object StopBlockManagerMaster extends ToBlockManagerMaster diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala new file mode 100644 index 0000000000000..e9c755e36f716 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -0,0 +1,73 @@ +/* + * 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.ui.exec + +import javax.servlet.http.HttpServletRequest + +import scala.util.Try +import scala.xml.{Text, Node} + +import org.apache.spark.ui.{UIUtils, WebUIPage} + +private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage("threadDump") { + + private val sc = parent.sc + + def render(request: HttpServletRequest): Seq[Node] = { + val executorId = Option(request.getParameter("executorId")).getOrElse { + return Text(s"Missing executorId parameter") + } + val time = System.currentTimeMillis() + val maybeThreadDump = sc.get.getExecutorThreadDump(executorId) + + val content = maybeThreadDump.map { threadDump => + val dumpRows = threadDump.map { thread => + + } + +
    +

    Updated at {UIUtils.formatDate(time)}

    + { + // scalastyle:off +

    + Expand All +

    +

    + // scalastyle:on + } +
    {dumpRows}
    +
    + }.getOrElse(Text("Error fetching thread dump")) + UIUtils.headerSparkPage(s"Thread dump for executor $executorId", content, parent) + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index b0e3bb3b552fd..048fee3ce1ff4 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -41,7 +41,10 @@ private case class ExecutorSummaryInfo( totalShuffleWrite: Long, maxMemory: Long) -private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { +private[ui] class ExecutorsPage( + parent: ExecutorsTab, + threadDumpEnabled: Boolean) + extends WebUIPage("") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -75,6 +78,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { Shuffle Write + {if (threadDumpEnabled) Thread Dump else Seq.empty} {execInfoSorted.map(execRow)} @@ -133,6 +137,15 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { {Utils.bytesToString(info.totalShuffleWrite)} + { + if (threadDumpEnabled) { + + Thread Dump + + } else { + Seq.empty + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 9e0e71a51a408..ba97630f025c1 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -27,8 +27,14 @@ import org.apache.spark.ui.{SparkUI, SparkUITab} private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { val listener = parent.executorsListener + val sc = parent.sc + val threadDumpEnabled = + sc.isDefined && parent.conf.getBoolean("spark.ui.threadDumpsEnabled", true) - attachPage(new ExecutorsPage(this)) + attachPage(new ExecutorsPage(this, threadDumpEnabled)) + if (threadDumpEnabled) { + attachPage(new ExecutorThreadDumpPage(this)) + } } /** diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 79e398eb8c104..10010bdfa1a51 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -212,4 +212,18 @@ private[spark] object AkkaUtils extends Logging { logInfo(s"Connecting to $name: $url") Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) } + + def makeExecutorRef( + name: String, + conf: SparkConf, + host: String, + port: Int, + actorSystem: ActorSystem): ActorRef = { + val executorActorSystemName = SparkEnv.executorActorSystemName + Utils.checkHost(host, "Expected hostname") + val url = s"akka.tcp://$executorActorSystemName@$host:$port/user/$name" + val timeout = AkkaUtils.lookupTimeout(conf) + logInfo(s"Connecting to $name: $url") + Await.result(actorSystem.actorSelection(url).resolveOne(timeout), timeout) + } } diff --git a/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala b/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala new file mode 100644 index 0000000000000..d4e0ad93b966a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ThreadStackTrace.scala @@ -0,0 +1,27 @@ +/* + * 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.util + +/** + * Used for shipping per-thread stacktraces from the executors to driver. + */ +private[spark] case class ThreadStackTrace( + threadId: Long, + threadName: String, + threadState: Thread.State, + stackTrace: String) 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 a33046d2040d8..6ab94af9f3739 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -18,6 +18,7 @@ package org.apache.spark.util import java.io._ +import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer import java.util.jar.Attributes.Name @@ -1611,6 +1612,18 @@ private[spark] object Utils extends Logging { s"$className: $desc\n$st" } + /** Return a thread dump of all threads' stacktraces. Used to capture dumps for the web UI */ + def getThreadDump(): Array[ThreadStackTrace] = { + // We need to filter out null values here because dumpAllThreads() may return null array + // elements for threads that are dead / don't exist. + val threadInfos = ManagementFactory.getThreadMXBean.dumpAllThreads(true, true).filter(_ != null) + threadInfos.sortBy(_.getThreadId).map { case threadInfo => + val stackTrace = threadInfo.getStackTrace.map(_.toString).mkString("\n") + ThreadStackTrace(threadInfo.getThreadId, threadInfo.getThreadName, + threadInfo.getThreadState, stackTrace) + } + } + /** * Convert all spark properties set in the given SparkConf to a sequence of java options. */ From c5912ecc7b392a13089ae735c07c2d7256de36c6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 3 Nov 2014 18:50:37 -0800 Subject: [PATCH 1212/1492] [FIX][MLLIB] fix seed in BaggedPointSuite Saw Jenkins test failures due to random seeds. jkbradley manishamde Author: Xiangrui Meng Closes #3084 from mengxr/fix-baggedpoint-suite and squashes the following commits: f735a43 [Xiangrui Meng] fix seed in BaggedPointSuite --- .../spark/mllib/tree/impl/BaggedPointSuite.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala index c0a62e00432a3..5cb433232e714 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/impl/BaggedPointSuite.scala @@ -30,7 +30,7 @@ class BaggedPointSuite extends FunSuite with LocalSparkContext { test("BaggedPoint RDD: without subsampling") { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) val rdd = sc.parallelize(arr) - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false) + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false, 42) baggedRDD.collect().foreach { baggedPoint => assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1) } @@ -44,7 +44,7 @@ class BaggedPointSuite extends FunSuite with LocalSparkContext { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true) + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true, seed) val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) @@ -60,7 +60,7 @@ class BaggedPointSuite extends FunSuite with LocalSparkContext { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true) + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true, seed) val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) @@ -75,7 +75,7 @@ class BaggedPointSuite extends FunSuite with LocalSparkContext { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false) + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false, seed) val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) @@ -91,7 +91,7 @@ class BaggedPointSuite extends FunSuite with LocalSparkContext { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false) + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false, seed) val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) From 04450d11548cfb25d4fb77d4a33e3a7cd4254183 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 3 Nov 2014 19:29:11 -0800 Subject: [PATCH 1213/1492] [SPARK-4192][SQL] Internal API for Python UDT Following #2919, this PR adds Python UDT (for internal use only) with tests under "pyspark.tests". Before `SQLContext.applySchema`, we check whether we need to convert user-type instances into SQL recognizable data. In the current implementation, a Python UDT must be paired with a Scala UDT for serialization on the JVM side. A following PR will add VectorUDT in MLlib for both Scala and Python. marmbrus jkbradley davies Author: Xiangrui Meng Closes #3068 from mengxr/SPARK-4192-sql and squashes the following commits: acff637 [Xiangrui Meng] merge master dba5ea7 [Xiangrui Meng] only use pyClass for Python UDT output sqlType as well 2c9d7e4 [Xiangrui Meng] move import to global setup; update needsConversion 7c4a6a9 [Xiangrui Meng] address comments 75223db [Xiangrui Meng] minor update f740379 [Xiangrui Meng] remove UDT from default imports e98d9d0 [Xiangrui Meng] fix py style 4e84fce [Xiangrui Meng] remove local hive tests and add more tests 39f19e0 [Xiangrui Meng] add tests b7f666d [Xiangrui Meng] add Python UDT --- python/pyspark/sql.py | 206 +++++++++++++++++- python/pyspark/tests.py | 93 +++++++- .../spark/sql/catalyst/types/dataTypes.scala | 9 +- .../org/apache/spark/sql/SQLContext.scala | 2 + .../spark/sql/execution/pythonUdfs.scala | 5 + .../spark/sql/test/ExamplePointUDT.scala | 64 ++++++ .../sql/types/util/DataTypeConversions.scala | 1 - 7 files changed, 375 insertions(+), 5 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index 675df084bf303..d16c18bc79fe4 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -417,6 +417,75 @@ def fromJson(cls, json): return StructType([StructField.fromJson(f) for f in json["fields"]]) +class UserDefinedType(DataType): + """ + :: WARN: Spark Internal Use Only :: + SQL User-Defined Type (UDT). + """ + + @classmethod + def typeName(cls): + return cls.__name__.lower() + + @classmethod + def sqlType(cls): + """ + Underlying SQL storage type for this UDT. + """ + raise NotImplementedError("UDT must implement sqlType().") + + @classmethod + def module(cls): + """ + The Python module of the UDT. + """ + raise NotImplementedError("UDT must implement module().") + + @classmethod + def scalaUDT(cls): + """ + The class name of the paired Scala UDT. + """ + raise NotImplementedError("UDT must have a paired Scala UDT.") + + def serialize(self, obj): + """ + Converts the a user-type object into a SQL datum. + """ + raise NotImplementedError("UDT must implement serialize().") + + def deserialize(self, datum): + """ + Converts a SQL datum into a user-type object. + """ + raise NotImplementedError("UDT must implement deserialize().") + + def json(self): + return json.dumps(self.jsonValue(), separators=(',', ':'), sort_keys=True) + + def jsonValue(self): + schema = { + "type": "udt", + "class": self.scalaUDT(), + "pyClass": "%s.%s" % (self.module(), type(self).__name__), + "sqlType": self.sqlType().jsonValue() + } + return schema + + @classmethod + def fromJson(cls, json): + pyUDT = json["pyClass"] + split = pyUDT.rfind(".") + pyModule = pyUDT[:split] + pyClass = pyUDT[split+1:] + m = __import__(pyModule, globals(), locals(), [pyClass], -1) + UDT = getattr(m, pyClass) + return UDT() + + def __eq__(self, other): + return type(self) == type(other) + + _all_primitive_types = dict((v.typeName(), v) for v in globals().itervalues() if type(v) is PrimitiveTypeSingleton and @@ -469,6 +538,12 @@ def _parse_datatype_json_string(json_string): ... complex_arraytype, False) >>> check_datatype(complex_maptype) True + >>> check_datatype(ExamplePointUDT()) + True + >>> structtype_with_udt = StructType([StructField("label", DoubleType(), False), + ... StructField("point", ExamplePointUDT(), False)]) + >>> check_datatype(structtype_with_udt) + True """ return _parse_datatype_json_value(json.loads(json_string)) @@ -488,7 +563,13 @@ def _parse_datatype_json_value(json_value): else: raise ValueError("Could not parse datatype: %s" % json_value) else: - return _all_complex_types[json_value["type"]].fromJson(json_value) + tpe = json_value["type"] + if tpe in _all_complex_types: + return _all_complex_types[tpe].fromJson(json_value) + elif tpe == 'udt': + return UserDefinedType.fromJson(json_value) + else: + raise ValueError("not supported type: %s" % tpe) # Mapping Python types to Spark SQL DataType @@ -509,7 +590,18 @@ def _parse_datatype_json_value(json_value): def _infer_type(obj): - """Infer the DataType from obj""" + """Infer the DataType from obj + + >>> p = ExamplePoint(1.0, 2.0) + >>> _infer_type(p) + ExamplePointUDT + """ + if obj is None: + raise ValueError("Can not infer type for None") + + if hasattr(obj, '__UDT__'): + return obj.__UDT__ + dataType = _type_mappings.get(type(obj)) if dataType is not None: return dataType() @@ -558,6 +650,93 @@ def _infer_schema(row): return StructType(fields) +def _need_python_to_sql_conversion(dataType): + """ + Checks whether we need python to sql conversion for the given type. + For now, only UDTs need this conversion. + + >>> _need_python_to_sql_conversion(DoubleType()) + False + >>> schema0 = StructType([StructField("indices", ArrayType(IntegerType(), False), False), + ... StructField("values", ArrayType(DoubleType(), False), False)]) + >>> _need_python_to_sql_conversion(schema0) + False + >>> _need_python_to_sql_conversion(ExamplePointUDT()) + True + >>> schema1 = ArrayType(ExamplePointUDT(), False) + >>> _need_python_to_sql_conversion(schema1) + True + >>> schema2 = StructType([StructField("label", DoubleType(), False), + ... StructField("point", ExamplePointUDT(), False)]) + >>> _need_python_to_sql_conversion(schema2) + True + """ + if isinstance(dataType, StructType): + return any([_need_python_to_sql_conversion(f.dataType) for f in dataType.fields]) + elif isinstance(dataType, ArrayType): + return _need_python_to_sql_conversion(dataType.elementType) + elif isinstance(dataType, MapType): + return _need_python_to_sql_conversion(dataType.keyType) or \ + _need_python_to_sql_conversion(dataType.valueType) + elif isinstance(dataType, UserDefinedType): + return True + else: + return False + + +def _python_to_sql_converter(dataType): + """ + Returns a converter that converts a Python object into a SQL datum for the given type. + + >>> conv = _python_to_sql_converter(DoubleType()) + >>> conv(1.0) + 1.0 + >>> conv = _python_to_sql_converter(ArrayType(DoubleType(), False)) + >>> conv([1.0, 2.0]) + [1.0, 2.0] + >>> conv = _python_to_sql_converter(ExamplePointUDT()) + >>> conv(ExamplePoint(1.0, 2.0)) + [1.0, 2.0] + >>> schema = StructType([StructField("label", DoubleType(), False), + ... StructField("point", ExamplePointUDT(), False)]) + >>> conv = _python_to_sql_converter(schema) + >>> conv((1.0, ExamplePoint(1.0, 2.0))) + (1.0, [1.0, 2.0]) + """ + if not _need_python_to_sql_conversion(dataType): + return lambda x: x + + if isinstance(dataType, StructType): + names, types = zip(*[(f.name, f.dataType) for f in dataType.fields]) + converters = map(_python_to_sql_converter, types) + + def converter(obj): + if isinstance(obj, dict): + return tuple(c(obj.get(n)) for n, c in zip(names, converters)) + elif isinstance(obj, tuple): + if hasattr(obj, "_fields") or hasattr(obj, "__FIELDS__"): + return tuple(c(v) for c, v in zip(converters, obj)) + elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs + d = dict(obj) + return tuple(c(d.get(n)) for n, c in zip(names, converters)) + else: + return tuple(c(v) for c, v in zip(converters, obj)) + else: + raise ValueError("Unexpected tuple %r with type %r" % (obj, dataType)) + return converter + elif isinstance(dataType, ArrayType): + element_converter = _python_to_sql_converter(dataType.elementType) + return lambda a: [element_converter(v) for v in a] + elif isinstance(dataType, MapType): + key_converter = _python_to_sql_converter(dataType.keyType) + value_converter = _python_to_sql_converter(dataType.valueType) + return lambda m: dict([(key_converter(k), value_converter(v)) for k, v in m.items()]) + elif isinstance(dataType, UserDefinedType): + return lambda obj: dataType.serialize(obj) + else: + raise ValueError("Unexpected type %r" % dataType) + + def _has_nulltype(dt): """ Return whether there is NullType in `dt` or not """ if isinstance(dt, StructType): @@ -818,11 +997,22 @@ def _verify_type(obj, dataType): Traceback (most recent call last): ... ValueError:... + >>> _verify_type(ExamplePoint(1.0, 2.0), ExamplePointUDT()) + >>> _verify_type([1.0, 2.0], ExamplePointUDT()) # doctest: +IGNORE_EXCEPTION_DETAIL + Traceback (most recent call last): + ... + ValueError:... """ # all objects are nullable if obj is None: return + if isinstance(dataType, UserDefinedType): + if not (hasattr(obj, '__UDT__') and obj.__UDT__ == dataType): + raise ValueError("%r is not an instance of type %r" % (obj, dataType)) + _verify_type(dataType.serialize(obj), dataType.sqlType()) + return + _type = type(dataType) assert _type in _acceptable_types, "unkown datatype: %s" % dataType @@ -897,6 +1087,8 @@ def _has_struct_or_date(dt): return _has_struct_or_date(dt.valueType) elif isinstance(dt, DateType): return True + elif isinstance(dt, UserDefinedType): + return True return False @@ -967,6 +1159,9 @@ def Dict(d): elif isinstance(dataType, DateType): return datetime.date + elif isinstance(dataType, UserDefinedType): + return lambda datum: dataType.deserialize(datum) + elif not isinstance(dataType, StructType): raise Exception("unexpected data type: %s" % dataType) @@ -1244,6 +1439,10 @@ def applySchema(self, rdd, schema): for row in rows: _verify_type(row, schema) + # convert python objects to sql data + converter = _python_to_sql_converter(schema) + rdd = rdd.map(converter) + batched = isinstance(rdd._jrdd_deserializer, BatchedSerializer) jrdd = self._pythonToJava(rdd._jrdd, batched) srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) @@ -1877,6 +2076,7 @@ def _test(): # let doctest run in pyspark.sql, so DataTypes can be picklable import pyspark.sql from pyspark.sql import Row, SQLContext + from pyspark.tests import ExamplePoint, ExamplePointUDT globs = pyspark.sql.__dict__.copy() # The small batch size here ensures that we see multiple batches, # even in these small test examples: @@ -1888,6 +2088,8 @@ def _test(): Row(field1=2, field2="row2"), Row(field1=3, field2="row3")] ) + globs['ExamplePoint'] = ExamplePoint + globs['ExamplePointUDT'] = ExamplePointUDT jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', '{"field1" : 2, "field3":{"field4":22, "field5": [10, 11]},' diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 68fd756876219..e947b09468108 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -49,7 +49,8 @@ from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ CloudPickleSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter -from pyspark.sql import SQLContext, IntegerType, Row, ArrayType +from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ + UserDefinedType, DoubleType from pyspark import shuffle _have_scipy = False @@ -694,8 +695,65 @@ def heavy_foo(x): self.assertTrue("rdd_%d.pstats" % id in os.listdir(d)) +class ExamplePointUDT(UserDefinedType): + """ + User-defined type (UDT) for ExamplePoint. + """ + + @classmethod + def sqlType(self): + return ArrayType(DoubleType(), False) + + @classmethod + def module(cls): + return 'pyspark.tests' + + @classmethod + def scalaUDT(cls): + return 'org.apache.spark.sql.test.ExamplePointUDT' + + def serialize(self, obj): + return [obj.x, obj.y] + + def deserialize(self, datum): + return ExamplePoint(datum[0], datum[1]) + + +class ExamplePoint: + """ + An example class to demonstrate UDT in Scala, Java, and Python. + """ + + __UDT__ = ExamplePointUDT() + + def __init__(self, x, y): + self.x = x + self.y = y + + def __repr__(self): + return "ExamplePoint(%s,%s)" % (self.x, self.y) + + def __str__(self): + return "(%s,%s)" % (self.x, self.y) + + def __eq__(self, other): + return isinstance(other, ExamplePoint) and \ + other.x == self.x and other.y == self.y + + class SQLTests(ReusedPySparkTestCase): + @classmethod + def setUpClass(cls): + ReusedPySparkTestCase.setUpClass() + cls.tempdir = tempfile.NamedTemporaryFile(delete=False) + os.unlink(cls.tempdir.name) + + @classmethod + def tearDownClass(cls): + ReusedPySparkTestCase.tearDownClass() + shutil.rmtree(cls.tempdir.name) + def setUp(self): self.sqlCtx = SQLContext(self.sc) @@ -824,6 +882,39 @@ def test_convert_row_to_dict(self): row = self.sqlCtx.sql("select l[0].a AS la from test").first() self.assertEqual(1, row.asDict()["la"]) + def test_infer_schema_with_udt(self): + from pyspark.tests import ExamplePoint, ExamplePointUDT + row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) + rdd = self.sc.parallelize([row]) + srdd = self.sqlCtx.inferSchema(rdd) + schema = srdd.schema() + field = [f for f in schema.fields if f.name == "point"][0] + self.assertEqual(type(field.dataType), ExamplePointUDT) + srdd.registerTempTable("labeled_point") + point = self.sqlCtx.sql("SELECT point FROM labeled_point").first().point + self.assertEqual(point, ExamplePoint(1.0, 2.0)) + + def test_apply_schema_with_udt(self): + from pyspark.tests import ExamplePoint, ExamplePointUDT + row = (1.0, ExamplePoint(1.0, 2.0)) + rdd = self.sc.parallelize([row]) + schema = StructType([StructField("label", DoubleType(), False), + StructField("point", ExamplePointUDT(), False)]) + srdd = self.sqlCtx.applySchema(rdd, schema) + point = srdd.first().point + self.assertEquals(point, ExamplePoint(1.0, 2.0)) + + def test_parquet_with_udt(self): + from pyspark.tests import ExamplePoint + row = Row(label=1.0, point=ExamplePoint(1.0, 2.0)) + rdd = self.sc.parallelize([row]) + srdd0 = self.sqlCtx.inferSchema(rdd) + output_dir = os.path.join(self.tempdir.name, "labeled_point") + srdd0.saveAsParquetFile(output_dir) + srdd1 = self.sqlCtx.parquetFile(output_dir) + point = srdd1.first().point + self.assertEquals(point, ExamplePoint(1.0, 2.0)) + class InputFormatTests(ReusedPySparkTestCase): 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 e1b5992a36e5f..5dd19dd12d8dd 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 @@ -71,6 +71,8 @@ object DataType { case JSortedObject( ("class", JString(udtClass)), + ("pyClass", _), + ("sqlType", _), ("type", JString("udt"))) => Class.forName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] } @@ -593,6 +595,9 @@ abstract class UserDefinedType[UserType] extends DataType with Serializable { /** Underlying storage type for this UDT */ def sqlType: DataType + /** Paired Python UDT class, if exists. */ + def pyUDT: String = null + /** * Convert the user type to a SQL datum * @@ -606,7 +611,9 @@ abstract class UserDefinedType[UserType] extends DataType with Serializable { override private[sql] def jsonValue: JValue = { ("type" -> "udt") ~ - ("class" -> this.getClass.getName) + ("class" -> this.getClass.getName) ~ + ("pyClass" -> pyUDT) ~ + ("sqlType" -> sqlType.jsonValue) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 9e61d18f7e926..84eaf401f240c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{Optimizer, DefaultOptimizer} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.types.UserDefinedType import org.apache.spark.sql.execution.{SparkStrategies, _} import org.apache.spark.sql.json._ import org.apache.spark.sql.parquet.ParquetRelation @@ -483,6 +484,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case ArrayType(_, _) => true case MapType(_, _, _) => true case StructType(_) => true + case udt: UserDefinedType[_] => needsConversion(udt.sqlType) case other => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index 997669051ed07..a83cf5d441d1e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -135,6 +135,8 @@ object EvaluatePython { case (k, v) => (k, toJava(v, mt.valueType)) // key should be primitive type }.asJava + case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) + case (dec: BigDecimal, dt: DecimalType) => dec.underlying() // Pyrolite can handle BigDecimal // Pyrolite can handle Timestamp @@ -177,6 +179,9 @@ object EvaluatePython { case (c: java.util.Calendar, TimestampType) => new java.sql.Timestamp(c.getTime().getTime()) + case (_, udt: UserDefinedType[_]) => + fromJava(obj, udt.sqlType) + case (c: Int, ByteType) => c.toByte case (c: Long, ByteType) => c.toByte case (c: Int, ShortType) => c.toShort diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala new file mode 100644 index 0000000000000..b9569e96c0312 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.test + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType +import org.apache.spark.sql.catalyst.types._ + +/** + * An example class to demonstrate UDT in Scala, Java, and Python. + * @param x x coordinate + * @param y y coordinate + */ +@SQLUserDefinedType(udt = classOf[ExamplePointUDT]) +private[sql] class ExamplePoint(val x: Double, val y: Double) + +/** + * User-defined type for [[ExamplePoint]]. + */ +private[sql] class ExamplePointUDT extends UserDefinedType[ExamplePoint] { + + override def sqlType: DataType = ArrayType(DoubleType, false) + + override def pyUDT: String = "pyspark.tests.ExamplePointUDT" + + override def serialize(obj: Any): Seq[Double] = { + obj match { + case p: ExamplePoint => + Seq(p.x, p.y) + } + } + + override def deserialize(datum: Any): ExamplePoint = { + datum match { + case values: Seq[_] => + val xy = values.asInstanceOf[Seq[Double]] + assert(xy.length == 2) + new ExamplePoint(xy(0), xy(1)) + case values: util.ArrayList[_] => + val xy = values.asInstanceOf[util.ArrayList[Double]].asScala + new ExamplePoint(xy(0), xy(1)) + } + } + + override def userClass: Class[ExamplePoint] = classOf[ExamplePoint] +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 1bc15146f0fe8..3fa4a7c6481d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.types.UserDefinedType - protected[sql] object DataTypeConversions { /** From 1a9c6cddadebdc53d083ac3e0da276ce979b5d1f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 3 Nov 2014 22:29:48 -0800 Subject: [PATCH 1214/1492] [SPARK-3573][MLLIB] Make MLlib's Vector compatible with SQL's SchemaRDD Register MLlib's Vector as a SQL user-defined type (UDT) in both Scala and Python. With this PR, we can easily map a RDD[LabeledPoint] to a SchemaRDD, and then select columns or save to a Parquet file. Examples in Scala/Python are attached. The Scala code was copied from jkbradley. ~~This PR contains the changes from #3068 . I will rebase after #3068 is merged.~~ marmbrus jkbradley Author: Xiangrui Meng Closes #3070 from mengxr/SPARK-3573 and squashes the following commits: 3a0b6e5 [Xiangrui Meng] organize imports 236f0a0 [Xiangrui Meng] register vector as UDT and provide dataset examples --- dev/run-tests | 2 +- .../src/main/python/mllib/dataset_example.py | 62 +++++++++ .../spark/examples/mllib/DatasetExample.scala | 121 ++++++++++++++++++ mllib/pom.xml | 5 + .../apache/spark/mllib/linalg/Vectors.scala | 69 +++++++++- .../spark/mllib/linalg/VectorsSuite.scala | 11 ++ python/pyspark/mllib/linalg.py | 50 ++++++++ python/pyspark/mllib/tests.py | 39 +++++- 8 files changed, 353 insertions(+), 6 deletions(-) create mode 100644 examples/src/main/python/mllib/dataset_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala diff --git a/dev/run-tests b/dev/run-tests index 0e9eefa76a18b..de607e4344453 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -180,7 +180,7 @@ CURRENT_BLOCK=$BLOCK_SPARK_UNIT_TESTS if [ -n "$_SQL_TESTS_ONLY" ]; then # This must be an array of individual arguments. Otherwise, having one long string #+ will be interpreted as a single test, which doesn't work. - SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test") + SBT_MAVEN_TEST_ARGS=("catalyst/test" "sql/test" "hive/test" "mllib/test") else SBT_MAVEN_TEST_ARGS=("test") fi diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py new file mode 100644 index 0000000000000..540dae785f6ea --- /dev/null +++ b/examples/src/main/python/mllib/dataset_example.py @@ -0,0 +1,62 @@ +# +# 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. +# + +""" +An example of how to use SchemaRDD as a dataset for ML. Run with:: + bin/spark-submit examples/src/main/python/mllib/dataset_example.py +""" + +import os +import sys +import tempfile +import shutil + +from pyspark import SparkContext +from pyspark.sql import SQLContext +from pyspark.mllib.util import MLUtils +from pyspark.mllib.stat import Statistics + + +def summarize(dataset): + print "schema: %s" % dataset.schema().json() + labels = dataset.map(lambda r: r.label) + print "label average: %f" % labels.mean() + features = dataset.map(lambda r: r.features) + summary = Statistics.colStats(features) + print "features average: %r" % summary.mean() + +if __name__ == "__main__": + if len(sys.argv) > 2: + print >> sys.stderr, "Usage: dataset_example.py " + exit(-1) + sc = SparkContext(appName="DatasetExample") + sqlCtx = SQLContext(sc) + if len(sys.argv) == 2: + input = sys.argv[1] + else: + input = "data/mllib/sample_libsvm_data.txt" + points = MLUtils.loadLibSVMFile(sc, input) + dataset0 = sqlCtx.inferSchema(points).setName("dataset0").cache() + summarize(dataset0) + tempdir = tempfile.NamedTemporaryFile(delete=False).name + os.unlink(tempdir) + print "Save dataset as a Parquet file to %s." % tempdir + dataset0.saveAsParquetFile(tempdir) + print "Load it back and summarize it again." + dataset1 = sqlCtx.parquetFile(tempdir).setName("dataset1").cache() + summarize(dataset1) + shutil.rmtree(tempdir) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala new file mode 100644 index 0000000000000..f8d83f4ec7327 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DatasetExample.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib + +import java.io.File + +import com.google.common.io.Files +import scopt.OptionParser + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext, SchemaRDD} + +/** + * An example of how to use [[org.apache.spark.sql.SchemaRDD]] as a Dataset for ML. Run with + * {{{ + * ./bin/run-example org.apache.spark.examples.mllib.DatasetExample [options] + * }}} + * If you use it as a template to create your own app, please use `spark-submit` to submit your app. + */ +object DatasetExample { + + case class Params( + input: String = "data/mllib/sample_libsvm_data.txt", + dataFormat: String = "libsvm") extends AbstractParams[Params] + + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("DatasetExample") { + head("Dataset: an example app using SchemaRDD as a Dataset for ML.") + opt[String]("input") + .text(s"input path to dataset") + .action((x, c) => c.copy(input = x)) + opt[String]("dataFormat") + .text("data format: libsvm (default), dense (deprecated in Spark v1.1)") + .action((x, c) => c.copy(input = x)) + checkConfig { params => + success + } + } + + parser.parse(args, defaultParams).map { params => + run(params) + }.getOrElse { + sys.exit(1) + } + } + + def run(params: Params) { + + val conf = new SparkConf().setAppName(s"DatasetExample with $params") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext._ // for implicit conversions + + // Load input data + val origData: RDD[LabeledPoint] = params.dataFormat match { + case "dense" => MLUtils.loadLabeledPoints(sc, params.input) + case "libsvm" => MLUtils.loadLibSVMFile(sc, params.input) + } + println(s"Loaded ${origData.count()} instances from file: ${params.input}") + + // Convert input data to SchemaRDD explicitly. + val schemaRDD: SchemaRDD = origData + println(s"Inferred schema:\n${schemaRDD.schema.prettyJson}") + println(s"Converted to SchemaRDD with ${schemaRDD.count()} records") + + // Select columns, using implicit conversion to SchemaRDD. + val labelsSchemaRDD: SchemaRDD = origData.select('label) + val labels: RDD[Double] = labelsSchemaRDD.map { case Row(v: Double) => v } + val numLabels = labels.count() + val meanLabel = labels.fold(0.0)(_ + _) / numLabels + println(s"Selected label column with average value $meanLabel") + + val featuresSchemaRDD: SchemaRDD = origData.select('features) + val features: RDD[Vector] = featuresSchemaRDD.map { case Row(v: Vector) => v } + val featureSummary = features.aggregate(new MultivariateOnlineSummarizer())( + (summary, feat) => summary.add(feat), + (sum1, sum2) => sum1.merge(sum2)) + println(s"Selected features column with average values:\n ${featureSummary.mean.toString}") + + val tmpDir = Files.createTempDir() + tmpDir.deleteOnExit() + val outputDir = new File(tmpDir, "dataset").toString + println(s"Saving to $outputDir as Parquet file.") + schemaRDD.saveAsParquetFile(outputDir) + + println(s"Loading Parquet file with UDT from $outputDir.") + val newDataset = sqlContext.parquetFile(outputDir) + + println(s"Schema from Parquet: ${newDataset.schema.prettyJson}") + val newFeatures = newDataset.select('features).map { case Row(v: Vector) => v } + val newFeaturesSummary = newFeatures.aggregate(new MultivariateOnlineSummarizer())( + (summary, feat) => summary.add(feat), + (sum1, sum2) => sum1.merge(sum2)) + println(s"Selected features column with average values:\n ${newFeaturesSummary.mean.toString}") + + sc.stop() + } + +} diff --git a/mllib/pom.xml b/mllib/pom.xml index fb7239e779aae..87a7ddaba97f2 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -45,6 +45,11 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + org.eclipse.jetty jetty-server diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 6af225b7f49f7..ac217edc619ab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -17,22 +17,26 @@ package org.apache.spark.mllib.linalg -import java.lang.{Double => JavaDouble, Integer => JavaInteger, Iterable => JavaIterable} import java.util +import java.lang.{Double => JavaDouble, Integer => JavaInteger, Iterable => JavaIterable} import scala.annotation.varargs import scala.collection.JavaConverters._ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} -import org.apache.spark.mllib.util.NumericParser import org.apache.spark.SparkException +import org.apache.spark.mllib.util.NumericParser +import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Row} +import org.apache.spark.sql.catalyst.types._ /** * Represents a numeric vector, whose index type is Int and value type is Double. * * Note: Users should not implement this interface. */ +@SQLUserDefinedType(udt = classOf[VectorUDT]) sealed trait Vector extends Serializable { /** @@ -74,6 +78,65 @@ sealed trait Vector extends Serializable { } } +/** + * User-defined type for [[Vector]] which allows easy interaction with SQL + * via [[org.apache.spark.sql.SchemaRDD]]. + */ +private[spark] class VectorUDT extends UserDefinedType[Vector] { + + override def sqlType: StructType = { + // type: 0 = sparse, 1 = dense + // We only use "values" for dense vectors, and "size", "indices", and "values" for sparse + // vectors. The "values" field is nullable because we might want to add binary vectors later, + // which uses "size" and "indices", but not "values". + StructType(Seq( + StructField("type", ByteType, nullable = false), + StructField("size", IntegerType, nullable = true), + StructField("indices", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true))) + } + + override def serialize(obj: Any): Row = { + val row = new GenericMutableRow(4) + obj match { + case sv: SparseVector => + row.setByte(0, 0) + row.setInt(1, sv.size) + row.update(2, sv.indices.toSeq) + row.update(3, sv.values.toSeq) + case dv: DenseVector => + row.setByte(0, 1) + row.setNullAt(1) + row.setNullAt(2) + row.update(3, dv.values.toSeq) + } + row + } + + override def deserialize(datum: Any): Vector = { + datum match { + case row: Row => + require(row.length == 4, + s"VectorUDT.deserialize given row with length ${row.length} but requires length == 4") + val tpe = row.getByte(0) + tpe match { + case 0 => + val size = row.getInt(1) + val indices = row.getAs[Iterable[Int]](2).toArray + val values = row.getAs[Iterable[Double]](3).toArray + new SparseVector(size, indices, values) + case 1 => + val values = row.getAs[Iterable[Double]](3).toArray + new DenseVector(values) + } + } + } + + override def pyUDT: String = "pyspark.mllib.linalg.VectorUDT" + + override def userClass: Class[Vector] = classOf[Vector] +} + /** * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. * We don't use the name `Vector` because Scala imports @@ -191,6 +254,7 @@ object Vectors { /** * A dense vector represented by a value array. */ +@SQLUserDefinedType(udt = classOf[VectorUDT]) class DenseVector(val values: Array[Double]) extends Vector { override def size: Int = values.length @@ -215,6 +279,7 @@ class DenseVector(val values: Array[Double]) extends Vector { * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ +@SQLUserDefinedType(udt = classOf[VectorUDT]) class SparseVector( override val size: Int, val indices: Array[Int], diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index cd651fe2d2ddf..93a84fe07b32a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -155,4 +155,15 @@ class VectorsSuite extends FunSuite { throw new RuntimeException(s"copy returned ${dvCopy.getClass} on ${dv.getClass}.") } } + + test("VectorUDT") { + val dv0 = Vectors.dense(Array.empty[Double]) + val dv1 = Vectors.dense(1.0, 2.0) + val sv0 = Vectors.sparse(2, Array.empty, Array.empty) + val sv1 = Vectors.sparse(2, Array(1), Array(2.0)) + val udt = new VectorUDT() + for (v <- Seq(dv0, dv1, sv0, sv1)) { + assert(v === udt.deserialize(udt.serialize(v))) + } + } } diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index d0a0e102a1a07..c0c3dff31e7f8 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -29,6 +29,9 @@ import numpy as np +from pyspark.sql import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ + IntegerType, ByteType, Row + __all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors'] @@ -106,7 +109,54 @@ def _format_float(f, digits=4): return s +class VectorUDT(UserDefinedType): + """ + SQL user-defined type (UDT) for Vector. + """ + + @classmethod + def sqlType(cls): + return StructType([ + StructField("type", ByteType(), False), + StructField("size", IntegerType(), True), + StructField("indices", ArrayType(IntegerType(), False), True), + StructField("values", ArrayType(DoubleType(), False), True)]) + + @classmethod + def module(cls): + return "pyspark.mllib.linalg" + + @classmethod + def scalaUDT(cls): + return "org.apache.spark.mllib.linalg.VectorUDT" + + def serialize(self, obj): + if isinstance(obj, SparseVector): + indices = [int(i) for i in obj.indices] + values = [float(v) for v in obj.values] + return (0, obj.size, indices, values) + elif isinstance(obj, DenseVector): + values = [float(v) for v in obj] + return (1, None, None, values) + else: + raise ValueError("cannot serialize %r of type %r" % (obj, type(obj))) + + def deserialize(self, datum): + assert len(datum) == 4, \ + "VectorUDT.deserialize given row with length %d but requires 4" % len(datum) + tpe = datum[0] + if tpe == 0: + return SparseVector(datum[1], datum[2], datum[3]) + elif tpe == 1: + return DenseVector(datum[3]) + else: + raise ValueError("do not recognize type %r" % tpe) + + class Vector(object): + + __UDT__ = VectorUDT() + """ Abstract class for DenseVector and SparseVector """ diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index d6fb87b378b4a..9fa4d6f6a2f5f 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -33,14 +33,14 @@ else: import unittest -from pyspark.serializers import PickleSerializer -from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, _convert_to_vector +from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics +from pyspark.serializers import PickleSerializer +from pyspark.sql import SQLContext from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase - _have_scipy = False try: import scipy.sparse @@ -221,6 +221,39 @@ def test_col_with_different_rdds(self): self.assertEqual(10, summary.count()) +class VectorUDTTests(PySparkTestCase): + + dv0 = DenseVector([]) + dv1 = DenseVector([1.0, 2.0]) + sv0 = SparseVector(2, [], []) + sv1 = SparseVector(2, [1], [2.0]) + udt = VectorUDT() + + def test_json_schema(self): + self.assertEqual(VectorUDT.fromJson(self.udt.jsonValue()), self.udt) + + def test_serialization(self): + for v in [self.dv0, self.dv1, self.sv0, self.sv1]: + self.assertEqual(v, self.udt.deserialize(self.udt.serialize(v))) + + def test_infer_schema(self): + sqlCtx = SQLContext(self.sc) + rdd = self.sc.parallelize([LabeledPoint(1.0, self.dv1), LabeledPoint(0.0, self.sv1)]) + srdd = sqlCtx.inferSchema(rdd) + schema = srdd.schema() + field = [f for f in schema.fields if f.name == "features"][0] + self.assertEqual(field.dataType, self.udt) + vectors = srdd.map(lambda p: p.features).collect() + self.assertEqual(len(vectors), 2) + for v in vectors: + if isinstance(v, SparseVector): + self.assertEqual(v, self.sv1) + elif isinstance(v, DenseVector): + self.assertEqual(v, self.dv1) + else: + raise ValueError("expecting a vector but got %r of type %r" % (v, type(v))) + + @unittest.skipIf(not _have_scipy, "SciPy not installed") class SciPyTests(PySparkTestCase): From 9bdc8412a0160e06e8182bd8b2f9bb65b478c590 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 3 Nov 2014 22:40:43 -0800 Subject: [PATCH 1215/1492] [SPARK-4163][Core] Add a backward compatibility test for FetchFailed /cc aarondav Author: zsxwing Closes #3086 from zsxwing/SPARK-4163-back-comp and squashes the following commits: 21cb2a8 [zsxwing] Add a backward compatibility test for FetchFailed --- .../org/apache/spark/util/JsonProtocolSuite.scala | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index a91c9ddeaef36..01030120ae548 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -177,6 +177,17 @@ class JsonProtocolSuite extends FunSuite { deserializedBmRemoved) } + test("FetchFailed backwards compatibility") { + // FetchFailed in Spark 1.1.0 does not have an "Message" property. + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + "ignored") + val oldEvent = JsonProtocol.taskEndReasonToJson(fetchFailed) + .removeField({ _._1 == "Message" }) + val expectedFetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, + "Unknown reason") + assert(expectedFetchFailed === JsonProtocol.taskEndReasonFromJson(oldEvent)) + } + test("SparkListenerApplicationStart backwards compatibility") { // SparkListenerApplicationStart in Spark 1.0.0 do not have an "appId" property. val applicationStart = SparkListenerApplicationStart("test", None, 1L, "user") From b671ce047d036b8923007902826038b01e836e8a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 3 Nov 2014 22:47:45 -0800 Subject: [PATCH 1216/1492] [SPARK-4166][Core] Add a backward compatibility test for ExecutorLostFailure Author: zsxwing Closes #3085 from zsxwing/SPARK-4166-back-comp and squashes the following commits: 89329f4 [zsxwing] Add a backward compatibility test for ExecutorLostFailure --- .../scala/org/apache/spark/util/JsonProtocolSuite.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 01030120ae548..aec1e409db95c 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -196,6 +196,15 @@ class JsonProtocolSuite extends FunSuite { assert(applicationStart === JsonProtocol.applicationStartFromJson(oldEvent)) } + test("ExecutorLostFailure backward compatibility") { + // ExecutorLostFailure in Spark 1.1.0 does not have an "Executor ID" property. + val executorLostFailure = ExecutorLostFailure("100") + val oldEvent = JsonProtocol.taskEndReasonToJson(executorLostFailure) + .removeField({ _._1 == "Executor ID" }) + val expectedExecutorLostFailure = ExecutorLostFailure("Unknown") + assert(expectedExecutorLostFailure === JsonProtocol.taskEndReasonFromJson(oldEvent)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ From e4f42631a68b473ce706429915f3f08042af2119 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 3 Nov 2014 23:56:14 -0800 Subject: [PATCH 1217/1492] [SPARK-3886] [PySpark] simplify serializer, use AutoBatchedSerializer by default. This PR simplify serializer, always use batched serializer (AutoBatchedSerializer as default), even batch size is 1. Author: Davies Liu This patch had conflicts when merged, resolved by Committer: Josh Rosen Closes #2920 from davies/fix_autobatch and squashes the following commits: e544ef9 [Davies Liu] revert unrelated change 6880b14 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch 1d557fc [Davies Liu] fix tests 8180907 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch 76abdce [Davies Liu] clean up 53fa60b [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch d7ac751 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch 2cc2497 [Davies Liu] Merge branch 'master' of github.com:apache/spark into fix_autobatch b4292ce [Davies Liu] fix bug in master d79744c [Davies Liu] recover hive tests be37ece [Davies Liu] refactor eb3938d [Davies Liu] refactor serializer in scala 8d77ef2 [Davies Liu] simplify serializer, use AutoBatchedSerializer by default. --- .../spark/api/python/PythonHadoopUtil.scala | 6 +- .../apache/spark/api/python/PythonRDD.scala | 110 +--------------- .../apache/spark/api/python/SerDeUtil.scala | 121 +++++++++++++----- .../WriteInputFormatTestDataGenerator.scala | 10 +- .../mllib/api/python/PythonMLLibAPI.scala | 2 +- python/pyspark/context.py | 58 +++------ python/pyspark/mllib/common.py | 2 +- python/pyspark/mllib/recommendation.py | 2 +- python/pyspark/rdd.py | 91 ++++++------- python/pyspark/serializers.py | 36 ++---- python/pyspark/shuffle.py | 7 +- python/pyspark/sql.py | 18 +-- python/pyspark/tests.py | 66 ++-------- .../org/apache/spark/sql/SchemaRDD.scala | 10 +- 14 files changed, 201 insertions(+), 338 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index 49dc95f349eac..5ba66178e2b78 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -61,8 +61,7 @@ private[python] object Converter extends Logging { * Other objects are passed through without conversion. */ private[python] class WritableToJavaConverter( - conf: Broadcast[SerializableWritable[Configuration]], - batchSize: Int) extends Converter[Any, Any] { + conf: Broadcast[SerializableWritable[Configuration]]) extends Converter[Any, Any] { /** * Converts a [[org.apache.hadoop.io.Writable]] to the underlying primitive, String or @@ -94,8 +93,7 @@ private[python] class WritableToJavaConverter( map.put(convertWritable(k), convertWritable(v)) } map - case w: Writable => - if (batchSize > 1) WritableUtils.clone(w, conf.value.value) else w + case w: Writable => WritableUtils.clone(w, conf.value.value) case other => other } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 61b125ef7c6c1..e94ccdcd47bb7 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -22,12 +22,10 @@ import java.net._ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ -import scala.collection.JavaConverters._ import scala.collection.mutable import scala.language.existentials import com.google.common.base.Charsets.UTF_8 -import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec @@ -442,7 +440,7 @@ private[spark] object PythonRDD extends Logging { val rdd = sc.sc.sequenceFile[K, V](path, kc, vc, minSplits) val confBroadcasted = sc.sc.broadcast(new SerializableWritable(sc.hadoopConfiguration())) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, - new WritableToJavaConverter(confBroadcasted, batchSize)) + new WritableToJavaConverter(confBroadcasted)) JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } @@ -468,7 +466,7 @@ private[spark] object PythonRDD extends Logging { Some(path), inputFormatClass, keyClass, valueClass, mergedConf) val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf)) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, - new WritableToJavaConverter(confBroadcasted, batchSize)) + new WritableToJavaConverter(confBroadcasted)) JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } @@ -494,7 +492,7 @@ private[spark] object PythonRDD extends Logging { None, inputFormatClass, keyClass, valueClass, conf) val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf)) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, - new WritableToJavaConverter(confBroadcasted, batchSize)) + new WritableToJavaConverter(confBroadcasted)) JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } @@ -537,7 +535,7 @@ private[spark] object PythonRDD extends Logging { Some(path), inputFormatClass, keyClass, valueClass, mergedConf) val confBroadcasted = sc.sc.broadcast(new SerializableWritable(mergedConf)) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, - new WritableToJavaConverter(confBroadcasted, batchSize)) + new WritableToJavaConverter(confBroadcasted)) JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } @@ -563,7 +561,7 @@ private[spark] object PythonRDD extends Logging { None, inputFormatClass, keyClass, valueClass, conf) val confBroadcasted = sc.sc.broadcast(new SerializableWritable(conf)) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, - new WritableToJavaConverter(confBroadcasted, batchSize)) + new WritableToJavaConverter(confBroadcasted)) JavaRDD.fromRDD(SerDeUtil.pairRDDToPython(converted, batchSize)) } @@ -746,104 +744,6 @@ private[spark] object PythonRDD extends Logging { converted.saveAsHadoopDataset(new JobConf(conf)) } } - - - /** - * Convert an RDD of serialized Python dictionaries to Scala Maps (no recursive conversions). - */ - @deprecated("PySpark does not use it anymore", "1.1") - def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { - pyRDD.rdd.mapPartitions { iter => - val unpickle = new Unpickler - SerDeUtil.initialize() - iter.flatMap { row => - unpickle.loads(row) match { - // in case of objects are pickled in batch mode - case objs: JArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) - // not in batch mode - case obj: JMap[String @unchecked, _] => Seq(obj.toMap) - } - } - } - } - - /** - * Convert an RDD of serialized Python tuple to Array (no recursive conversions). - * It is only used by pyspark.sql. - */ - def pythonToJavaArray(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Array[_]] = { - - def toArray(obj: Any): Array[_] = { - obj match { - case objs: JArrayList[_] => - objs.toArray - case obj if obj.getClass.isArray => - obj.asInstanceOf[Array[_]].toArray - } - } - - pyRDD.rdd.mapPartitions { iter => - val unpickle = new Unpickler - iter.flatMap { row => - val obj = unpickle.loads(row) - if (batched) { - obj.asInstanceOf[JArrayList[_]].map(toArray) - } else { - Seq(toArray(obj)) - } - } - }.toJavaRDD() - } - - private[spark] class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { - private val pickle = new Pickler() - private var batch = 1 - private val buffer = new mutable.ArrayBuffer[Any] - - override def hasNext(): Boolean = iter.hasNext - - override def next(): Array[Byte] = { - while (iter.hasNext && buffer.length < batch) { - buffer += iter.next() - } - val bytes = pickle.dumps(buffer.toArray) - val size = bytes.length - // let 1M < size < 10M - if (size < 1024 * 1024) { - batch *= 2 - } else if (size > 1024 * 1024 * 10 && batch > 1) { - batch /= 2 - } - buffer.clear() - bytes - } - } - - /** - * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by - * PySpark. - */ - def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { - jRDD.rdd.mapPartitions { iter => new AutoBatchedPickler(iter) } - } - - /** - * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. - */ - def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { - pyRDD.rdd.mapPartitions { iter => - SerDeUtil.initialize() - val unpickle = new Unpickler - iter.flatMap { row => - val obj = unpickle.loads(row) - if (batched) { - obj.asInstanceOf[JArrayList[_]].asScala - } else { - Seq(obj) - } - } - }.toJavaRDD() - } } private diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index ebdc3533e0992..a4153aaa926f8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -18,8 +18,13 @@ package org.apache.spark.api.python import java.nio.ByteOrder +import java.util.{ArrayList => JArrayList} + +import org.apache.spark.api.java.JavaRDD import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.util.Failure import scala.util.Try @@ -89,6 +94,73 @@ private[spark] object SerDeUtil extends Logging { } initialize() + + /** + * Convert an RDD of Java objects to Array (no recursive conversions). + * It is only used by pyspark.sql. + */ + def toJavaArray(jrdd: JavaRDD[Any]): JavaRDD[Array[_]] = { + jrdd.rdd.map { + case objs: JArrayList[_] => + objs.toArray + case obj if obj.getClass.isArray => + obj.asInstanceOf[Array[_]].toArray + }.toJavaRDD() + } + + /** + * Choose batch size based on size of objects + */ + private[spark] class AutoBatchedPickler(iter: Iterator[Any]) extends Iterator[Array[Byte]] { + private val pickle = new Pickler() + private var batch = 1 + private val buffer = new mutable.ArrayBuffer[Any] + + override def hasNext: Boolean = iter.hasNext + + override def next(): Array[Byte] = { + while (iter.hasNext && buffer.length < batch) { + buffer += iter.next() + } + val bytes = pickle.dumps(buffer.toArray) + val size = bytes.length + // let 1M < size < 10M + if (size < 1024 * 1024) { + batch *= 2 + } else if (size > 1024 * 1024 * 10 && batch > 1) { + batch /= 2 + } + buffer.clear() + bytes + } + } + + /** + * Convert an RDD of Java objects to an RDD of serialized Python objects, that is usable by + * PySpark. + */ + private[spark] def javaToPython(jRDD: JavaRDD[_]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => new AutoBatchedPickler(iter) } + } + + /** + * Convert an RDD of serialized Python objects to RDD of objects, that is usable by PySpark. + */ + def pythonToJava(pyRDD: JavaRDD[Array[Byte]], batched: Boolean): JavaRDD[Any] = { + pyRDD.rdd.mapPartitions { iter => + initialize() + val unpickle = new Unpickler + iter.flatMap { row => + val obj = unpickle.loads(row) + if (batched) { + obj.asInstanceOf[JArrayList[_]].asScala + } else { + Seq(obj) + } + } + }.toJavaRDD() + } + private def checkPickle(t: (Any, Any)): (Boolean, Boolean) = { val pickle = new Pickler val kt = Try { @@ -128,17 +200,18 @@ private[spark] object SerDeUtil extends Logging { */ def pairRDDToPython(rdd: RDD[(Any, Any)], batchSize: Int): RDD[Array[Byte]] = { val (keyFailed, valueFailed) = checkPickle(rdd.first()) + rdd.mapPartitions { iter => - val pickle = new Pickler val cleaned = iter.map { case (k, v) => val key = if (keyFailed) k.toString else k val value = if (valueFailed) v.toString else v Array[Any](key, value) } - if (batchSize > 1) { - cleaned.grouped(batchSize).map(batched => pickle.dumps(seqAsJavaList(batched))) + if (batchSize == 0) { + new AutoBatchedPickler(cleaned) } else { - cleaned.map(pickle.dumps(_)) + val pickle = new Pickler + cleaned.grouped(batchSize).map(batched => pickle.dumps(seqAsJavaList(batched))) } } } @@ -146,36 +219,22 @@ private[spark] object SerDeUtil extends Logging { /** * Convert an RDD of serialized Python tuple (K, V) to RDD[(K, V)]. */ - def pythonToPairRDD[K, V](pyRDD: RDD[Array[Byte]], batchSerialized: Boolean): RDD[(K, V)] = { + def pythonToPairRDD[K, V](pyRDD: RDD[Array[Byte]], batched: Boolean): RDD[(K, V)] = { def isPair(obj: Any): Boolean = { - Option(obj.getClass.getComponentType).map(!_.isPrimitive).getOrElse(false) && + Option(obj.getClass.getComponentType).exists(!_.isPrimitive) && obj.asInstanceOf[Array[_]].length == 2 } - pyRDD.mapPartitions { iter => - initialize() - val unpickle = new Unpickler - val unpickled = - if (batchSerialized) { - iter.flatMap { batch => - unpickle.loads(batch) match { - case objs: java.util.List[_] => collectionAsScalaIterable(objs) - case other => throw new SparkException( - s"Unexpected type ${other.getClass.getName} for batch serialized Python RDD") - } - } - } else { - iter.map(unpickle.loads(_)) - } - unpickled.map { - case obj if isPair(obj) => - // we only accept (K, V) - val arr = obj.asInstanceOf[Array[_]] - (arr.head.asInstanceOf[K], arr.last.asInstanceOf[V]) - case other => throw new SparkException( - s"RDD element of type ${other.getClass.getName} cannot be used") - } + + val rdd = pythonToJava(pyRDD, batched).rdd + rdd.first match { + case obj if isPair(obj) => + // we only accept (K, V) + case other => throw new SparkException( + s"RDD element of type ${other.getClass.getName} cannot be used") + } + rdd.map { obj => + val arr = obj.asInstanceOf[Array[_]] + (arr.head.asInstanceOf[K], arr.last.asInstanceOf[V]) } } - } - diff --git a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala index e9ca9166eb4d6..c0cbd28a845be 100644 --- a/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala +++ b/core/src/main/scala/org/apache/spark/api/python/WriteInputFormatTestDataGenerator.scala @@ -176,11 +176,11 @@ object WriteInputFormatTestDataGenerator { // Create test data for arbitrary custom writable TestWritable val testClass = Seq( - ("1", TestWritable("test1", 123, 54.0)), - ("2", TestWritable("test2", 456, 8762.3)), - ("1", TestWritable("test3", 123, 423.1)), - ("3", TestWritable("test56", 456, 423.5)), - ("2", TestWritable("test2", 123, 5435.2)) + ("1", TestWritable("test1", 1, 1.0)), + ("2", TestWritable("test2", 2, 2.3)), + ("3", TestWritable("test3", 3, 3.1)), + ("5", TestWritable("test56", 5, 5.5)), + ("4", TestWritable("test4", 4, 4.2)) ) val rdd = sc.parallelize(testClass, numSlices = 2).map{ case (k, v) => (new Text(k), v) } rdd.saveAsNewAPIHadoopFile(classPath, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index acdc67ddc660a..65b98a8ceea55 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -736,7 +736,7 @@ private[spark] object SerDe extends Serializable { def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { jRDD.rdd.mapPartitions { iter => initialize() // let it called in executor - new PythonRDD.AutoBatchedPickler(iter) + new SerDeUtil.AutoBatchedPickler(iter) } } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 5f8dcedb1eea2..a0e4821728c8b 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -63,7 +63,6 @@ class SparkContext(object): _active_spark_context = None _lock = Lock() _python_includes = None # zip and egg files that need to be added to PYTHONPATH - _default_batch_size_for_serialized_input = 10 def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, environment=None, batchSize=0, serializer=PickleSerializer(), conf=None, @@ -115,9 +114,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self._conf = conf or SparkConf(_jvm=self._jvm) self._batchSize = batchSize # -1 represents an unlimited batch size self._unbatched_serializer = serializer - if batchSize == 1: - self.serializer = self._unbatched_serializer - elif batchSize == 0: + if batchSize == 0: self.serializer = AutoBatchedSerializer(self._unbatched_serializer) else: self.serializer = BatchedSerializer(self._unbatched_serializer, @@ -305,12 +302,8 @@ def parallelize(self, c, numSlices=None): # Make sure we distribute data evenly if it's smaller than self.batchSize if "__len__" not in dir(c): c = list(c) # Make it a list so we can compute its length - batchSize = min(len(c) // numSlices, self._batchSize) - if batchSize > 1: - serializer = BatchedSerializer(self._unbatched_serializer, - batchSize) - else: - serializer = self._unbatched_serializer + batchSize = max(1, min(len(c) // numSlices, self._batchSize)) + serializer = BatchedSerializer(self._unbatched_serializer, batchSize) serializer.dump_stream(c, tempFile) tempFile.close() readRDDFromFile = self._jvm.PythonRDD.readRDDFromFile @@ -328,8 +321,7 @@ def pickleFile(self, name, minPartitions=None): [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """ minPartitions = minPartitions or self.defaultMinPartitions - return RDD(self._jsc.objectFile(name, minPartitions), self, - BatchedSerializer(PickleSerializer())) + return RDD(self._jsc.objectFile(name, minPartitions), self) def textFile(self, name, minPartitions=None, use_unicode=True): """ @@ -405,7 +397,7 @@ def _dictToJavaMap(self, d): return jm def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, - valueConverter=None, minSplits=None, batchSize=None): + valueConverter=None, minSplits=None, batchSize=0): """ Read a Hadoop SequenceFile with arbitrary key and value Writable class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -427,17 +419,15 @@ def sequenceFile(self, path, keyClass=None, valueClass=None, keyConverter=None, :param minSplits: minimum splits in dataset (default min(2, sc.defaultParallelism)) :param batchSize: The number of Python objects represented as a single - Java object. (default sc._default_batch_size_for_serialized_input) + Java object. (default 0, choose batchSize automatically) """ minSplits = minSplits or min(self.defaultParallelism, 2) - batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) - ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.sequenceFile(self._jsc, path, keyClass, valueClass, keyConverter, valueConverter, minSplits, batchSize) - return RDD(jrdd, self, ser) + return RDD(jrdd, self) def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None, batchSize=None): + valueConverter=None, conf=None, batchSize=0): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -458,18 +448,16 @@ def newAPIHadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConv :param conf: Hadoop configuration, passed in as a dict (None by default) :param batchSize: The number of Python objects represented as a single - Java object. (default sc._default_batch_size_for_serialized_input) + Java object. (default 0, choose batchSize automatically) """ jconf = self._dictToJavaMap(conf) - batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) - ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopFile(self._jsc, path, inputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf, batchSize) - return RDD(jrdd, self, ser) + return RDD(jrdd, self) def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None, batchSize=None): + valueConverter=None, conf=None, batchSize=0): """ Read a 'new API' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -487,18 +475,16 @@ def newAPIHadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=N :param conf: Hadoop configuration, passed in as a dict (None by default) :param batchSize: The number of Python objects represented as a single - Java object. (default sc._default_batch_size_for_serialized_input) + Java object. (default 0, choose batchSize automatically) """ jconf = self._dictToJavaMap(conf) - batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) - ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.newAPIHadoopRDD(self._jsc, inputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf, batchSize) - return RDD(jrdd, self, ser) + return RDD(jrdd, self) def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None, batchSize=None): + valueConverter=None, conf=None, batchSize=0): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class from HDFS, a local file system (available on all nodes), or any Hadoop-supported file system URI. @@ -519,18 +505,16 @@ def hadoopFile(self, path, inputFormatClass, keyClass, valueClass, keyConverter= :param conf: Hadoop configuration, passed in as a dict (None by default) :param batchSize: The number of Python objects represented as a single - Java object. (default sc._default_batch_size_for_serialized_input) + Java object. (default 0, choose batchSize automatically) """ jconf = self._dictToJavaMap(conf) - batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) - ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.hadoopFile(self._jsc, path, inputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf, batchSize) - return RDD(jrdd, self, ser) + return RDD(jrdd, self) def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, - valueConverter=None, conf=None, batchSize=None): + valueConverter=None, conf=None, batchSize=0): """ Read an 'old' Hadoop InputFormat with arbitrary key and value class, from an arbitrary Hadoop configuration, which is passed in as a Python dict. @@ -548,15 +532,13 @@ def hadoopRDD(self, inputFormatClass, keyClass, valueClass, keyConverter=None, :param conf: Hadoop configuration, passed in as a dict (None by default) :param batchSize: The number of Python objects represented as a single - Java object. (default sc._default_batch_size_for_serialized_input) + Java object. (default 0, choose batchSize automatically) """ jconf = self._dictToJavaMap(conf) - batchSize = max(1, batchSize or self._default_batch_size_for_serialized_input) - ser = BatchedSerializer(PickleSerializer()) if (batchSize > 1) else PickleSerializer() jrdd = self._jvm.PythonRDD.hadoopRDD(self._jsc, inputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf, batchSize) - return RDD(jrdd, self, ser) + return RDD(jrdd, self) def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) @@ -836,7 +818,7 @@ def _test(): import doctest import tempfile globs = globals().copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + globs['sc'] = SparkContext('local[4]', 'PythonTest') globs['tempdir'] = tempfile.mkdtemp() atexit.register(lambda: shutil.rmtree(globs['tempdir'])) (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index 76864d8163586..dbe5f698b7345 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -96,7 +96,7 @@ def _java2py(sc, r): if clsName == 'JavaRDD': jrdd = sc._jvm.SerDe.javaToPython(r) - return RDD(jrdd, sc, AutoBatchedSerializer(PickleSerializer())) + return RDD(jrdd, sc) elif isinstance(r, (JavaArray, JavaList)) or clsName in _picklable_classes: r = sc._jvm.SerDe.dumps(r) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 6b32af07c9be2..e8b998414d319 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -117,7 +117,7 @@ def _test(): import doctest import pyspark.mllib.recommendation globs = pyspark.mllib.recommendation.__dict__.copy() - globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) + globs['sc'] = SparkContext('local[4]', 'PythonTest') (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 4f025b9f11707..879655dc53f4a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -120,7 +120,7 @@ class RDD(object): operated on in parallel. """ - def __init__(self, jrdd, ctx, jrdd_deserializer): + def __init__(self, jrdd, ctx, jrdd_deserializer=AutoBatchedSerializer(PickleSerializer())): self._jrdd = jrdd self.is_cached = False self.is_checkpointed = False @@ -129,12 +129,8 @@ def __init__(self, jrdd, ctx, jrdd_deserializer): self._id = jrdd.id() self._partitionFunc = None - def _toPickleSerialization(self): - if (self._jrdd_deserializer == PickleSerializer() or - self._jrdd_deserializer == BatchedSerializer(PickleSerializer())): - return self - else: - return self._reserialize(BatchedSerializer(PickleSerializer(), 10)) + def _pickled(self): + return self._reserialize(AutoBatchedSerializer(PickleSerializer())) def id(self): """ @@ -446,12 +442,11 @@ def intersection(self, other): def _reserialize(self, serializer=None): serializer = serializer or self.ctx.serializer - if self._jrdd_deserializer == serializer: - return self - else: - converted = self.map(lambda x: x, preservesPartitioning=True) - converted._jrdd_deserializer = serializer - return converted + if self._jrdd_deserializer != serializer: + if not isinstance(self, PipelinedRDD): + self = self.map(lambda x: x, preservesPartitioning=True) + self._jrdd_deserializer = serializer + return self def __add__(self, other): """ @@ -1120,9 +1115,8 @@ def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None :param valueConverter: (None by default) """ jconf = self.ctx._dictToJavaMap(conf) - pickledRDD = self._toPickleSerialization() - batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) - self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf, + pickledRDD = self._pickled() + self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, True, jconf, keyConverter, valueConverter, True) def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None, @@ -1147,9 +1141,8 @@ def saveAsNewAPIHadoopFile(self, path, outputFormatClass, keyClass=None, valueCl :param conf: Hadoop job configuration, passed in as a dict (None by default) """ jconf = self.ctx._dictToJavaMap(conf) - pickledRDD = self._toPickleSerialization() - batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) - self.ctx._jvm.PythonRDD.saveAsNewAPIHadoopFile(pickledRDD._jrdd, batched, path, + pickledRDD = self._pickled() + self.ctx._jvm.PythonRDD.saveAsNewAPIHadoopFile(pickledRDD._jrdd, True, path, outputFormatClass, keyClass, valueClass, keyConverter, valueConverter, jconf) @@ -1166,9 +1159,8 @@ def saveAsHadoopDataset(self, conf, keyConverter=None, valueConverter=None): :param valueConverter: (None by default) """ jconf = self.ctx._dictToJavaMap(conf) - pickledRDD = self._toPickleSerialization() - batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) - self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, batched, jconf, + pickledRDD = self._pickled() + self.ctx._jvm.PythonRDD.saveAsHadoopDataset(pickledRDD._jrdd, True, jconf, keyConverter, valueConverter, False) def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=None, @@ -1195,9 +1187,8 @@ def saveAsHadoopFile(self, path, outputFormatClass, keyClass=None, valueClass=No :param compressionCodecClass: (None by default) """ jconf = self.ctx._dictToJavaMap(conf) - pickledRDD = self._toPickleSerialization() - batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) - self.ctx._jvm.PythonRDD.saveAsHadoopFile(pickledRDD._jrdd, batched, path, + pickledRDD = self._pickled() + self.ctx._jvm.PythonRDD.saveAsHadoopFile(pickledRDD._jrdd, True, path, outputFormatClass, keyClass, valueClass, keyConverter, valueConverter, @@ -1215,9 +1206,8 @@ def saveAsSequenceFile(self, path, compressionCodecClass=None): :param path: path to sequence file :param compressionCodecClass: (None by default) """ - pickledRDD = self._toPickleSerialization() - batched = isinstance(pickledRDD._jrdd_deserializer, BatchedSerializer) - self.ctx._jvm.PythonRDD.saveAsSequenceFile(pickledRDD._jrdd, batched, + pickledRDD = self._pickled() + self.ctx._jvm.PythonRDD.saveAsSequenceFile(pickledRDD._jrdd, True, path, compressionCodecClass) def saveAsPickleFile(self, path, batchSize=10): @@ -1232,8 +1222,11 @@ def saveAsPickleFile(self, path, batchSize=10): >>> sorted(sc.pickleFile(tmpFile.name, 5).collect()) [1, 2, 'rdd', 'spark'] """ - self._reserialize(BatchedSerializer(PickleSerializer(), - batchSize))._jrdd.saveAsObjectFile(path) + if batchSize == 0: + ser = AutoBatchedSerializer(PickleSerializer()) + else: + ser = BatchedSerializer(PickleSerializer(), batchSize) + self._reserialize(ser)._jrdd.saveAsObjectFile(path) def saveAsTextFile(self, path): """ @@ -1774,13 +1767,10 @@ def zip(self, other): >>> x.zip(y).collect() [(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)] """ - if self.getNumPartitions() != other.getNumPartitions(): - raise ValueError("Can only zip with RDD which has the same number of partitions") - def get_batch_size(ser): if isinstance(ser, BatchedSerializer): return ser.batchSize - return 0 + return 1 def batch_as(rdd, batchSize): ser = rdd._jrdd_deserializer @@ -1790,12 +1780,16 @@ def batch_as(rdd, batchSize): my_batch = get_batch_size(self._jrdd_deserializer) other_batch = get_batch_size(other._jrdd_deserializer) - if my_batch != other_batch: - # use the greatest batchSize to batch the other one. - if my_batch > other_batch: - other = batch_as(other, my_batch) - else: - self = batch_as(self, other_batch) + # use the smallest batchSize for both of them + batchSize = min(my_batch, other_batch) + if batchSize <= 0: + # auto batched or unlimited + batchSize = 100 + other = batch_as(other, batchSize) + self = batch_as(self, batchSize) + + if self.getNumPartitions() != other.getNumPartitions(): + raise ValueError("Can only zip with RDD which has the same number of partitions") # There will be an Exception in JVM if there are different number # of items in each partitions. @@ -1934,25 +1928,14 @@ def lookup(self, key): return values.collect() - def _is_pickled(self): - """ Return this RDD is serialized by Pickle or not. """ - der = self._jrdd_deserializer - if isinstance(der, PickleSerializer): - return True - if isinstance(der, BatchedSerializer) and isinstance(der.serializer, PickleSerializer): - return True - return False - def _to_java_object_rdd(self): """ Return an JavaRDD of Object by unpickling It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. """ - rdd = self._reserialize(AutoBatchedSerializer(PickleSerializer())) \ - if not self._is_pickled() else self - is_batch = isinstance(rdd._jrdd_deserializer, BatchedSerializer) - return self.ctx._jvm.PythonRDD.pythonToJava(rdd._jrdd, is_batch) + rdd = self._pickled() + return self.ctx._jvm.SerDeUtil.pythonToJava(rdd._jrdd, True) def countApprox(self, timeout, confidence=0.95): """ @@ -2132,7 +2115,7 @@ def _test(): globs = globals().copy() # 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) + globs['sc'] = SparkContext('local[4]', 'PythonTest') (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 904bd9f2652d3..d597cbf94e1b1 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -33,9 +33,8 @@ [0, 2, 4, 6, 8, 10, 12, 14, 16, 18] >>> sc.stop() -By default, PySpark serialize objects in batches; the batch size can be -controlled through SparkContext's C{batchSize} parameter -(the default size is 1024 objects): +PySpark serialize objects in batches; By default, the batch size is chosen based +on the size of objects, also configurable by SparkContext's C{batchSize} parameter: >>> sc = SparkContext('local', 'test', batchSize=2) >>> rdd = sc.parallelize(range(16), 4).map(lambda x: x) @@ -48,16 +47,6 @@ >>> rdd._jrdd.count() 8L >>> sc.stop() - -A batch size of -1 uses an unlimited batch size, and a size of 1 disables -batching: - ->>> sc = SparkContext('local', 'test', batchSize=1) ->>> rdd = sc.parallelize(range(16), 4).map(lambda x: x) ->>> rdd.glom().collect() -[[0, 1, 2, 3], [4, 5, 6, 7], [8, 9, 10, 11], [12, 13, 14, 15]] ->>> rdd._jrdd.count() -16L """ import cPickle @@ -73,7 +62,7 @@ from pyspark import cloudpickle -__all__ = ["PickleSerializer", "MarshalSerializer"] +__all__ = ["PickleSerializer", "MarshalSerializer", "UTF8Deserializer"] class SpecialLengths(object): @@ -113,7 +102,7 @@ def __ne__(self, other): return not self.__eq__(other) def __repr__(self): - return "<%s object>" % self.__class__.__name__ + return "%s()" % self.__class__.__name__ def __hash__(self): return hash(str(self)) @@ -181,6 +170,7 @@ class BatchedSerializer(Serializer): """ UNLIMITED_BATCH_SIZE = -1 + UNKNOWN_BATCH_SIZE = 0 def __init__(self, serializer, batchSize=UNLIMITED_BATCH_SIZE): self.serializer = serializer @@ -213,10 +203,10 @@ def _load_stream_without_unbatching(self, stream): def __eq__(self, other): return (isinstance(other, BatchedSerializer) and - other.serializer == self.serializer) + other.serializer == self.serializer and other.batchSize == self.batchSize) def __repr__(self): - return "BatchedSerializer<%s>" % str(self.serializer) + return "BatchedSerializer(%s, %d)" % (str(self.serializer), self.batchSize) class AutoBatchedSerializer(BatchedSerializer): @@ -225,7 +215,7 @@ class AutoBatchedSerializer(BatchedSerializer): """ def __init__(self, serializer, bestSize=1 << 16): - BatchedSerializer.__init__(self, serializer, -1) + BatchedSerializer.__init__(self, serializer, self.UNKNOWN_BATCH_SIZE) self.bestSize = bestSize def dump_stream(self, iterator, stream): @@ -248,10 +238,10 @@ def dump_stream(self, iterator, stream): def __eq__(self, other): return (isinstance(other, AutoBatchedSerializer) and - other.serializer == self.serializer) + other.serializer == self.serializer and other.bestSize == self.bestSize) def __str__(self): - return "AutoBatchedSerializer<%s>" % str(self.serializer) + return "AutoBatchedSerializer(%s)" % str(self.serializer) class CartesianDeserializer(FramedSerializer): @@ -284,7 +274,7 @@ def __eq__(self, other): self.key_ser == other.key_ser and self.val_ser == other.val_ser) def __repr__(self): - return "CartesianDeserializer<%s, %s>" % \ + return "CartesianDeserializer(%s, %s)" % \ (str(self.key_ser), str(self.val_ser)) @@ -311,7 +301,7 @@ def __eq__(self, other): self.key_ser == other.key_ser and self.val_ser == other.val_ser) def __repr__(self): - return "PairDeserializer<%s, %s>" % (str(self.key_ser), str(self.val_ser)) + return "PairDeserializer(%s, %s)" % (str(self.key_ser), str(self.val_ser)) class NoOpSerializer(FramedSerializer): @@ -430,7 +420,7 @@ def loads(self, obj): class AutoSerializer(FramedSerializer): """ - Choose marshal or cPickle as serialization protocol autumatically + Choose marshal or cPickle as serialization protocol automatically """ def __init__(self): diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index d57a802e4734a..5931e923c2e36 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -25,7 +25,7 @@ import random import pyspark.heapq3 as heapq -from pyspark.serializers import BatchedSerializer, PickleSerializer +from pyspark.serializers import AutoBatchedSerializer, PickleSerializer try: import psutil @@ -213,8 +213,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, Merger.__init__(self, aggregator) self.memory_limit = memory_limit # default serializer is only used for tests - self.serializer = serializer or \ - BatchedSerializer(PickleSerializer(), 1024) + self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) self.localdirs = localdirs or _get_local_dirs(str(id(self))) # number of partitions when spill data into disks self.partitions = partitions @@ -470,7 +469,7 @@ class ExternalSorter(object): def __init__(self, memory_limit, serializer=None): self.memory_limit = memory_limit self.local_dirs = _get_local_dirs("sort") - self.serializer = serializer or BatchedSerializer(PickleSerializer(), 1024) + self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) def _get_path(self, n): """ Choose one directory for spill by number n """ diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d16c18bc79fe4..e5d62a466cab6 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -44,7 +44,8 @@ from py4j.java_collections import ListConverter, MapConverter from pyspark.rdd import RDD -from pyspark.serializers import BatchedSerializer, PickleSerializer, CloudPickleSerializer +from pyspark.serializers import BatchedSerializer, AutoBatchedSerializer, PickleSerializer, \ + CloudPickleSerializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -1233,7 +1234,6 @@ def __init__(self, sparkContext, sqlContext=None): self._sc = sparkContext self._jsc = self._sc._jsc self._jvm = self._sc._jvm - self._pythonToJava = self._jvm.PythonRDD.pythonToJavaArray self._scala_SQLContext = sqlContext @property @@ -1263,8 +1263,8 @@ def registerFunction(self, name, f, returnType=StringType()): """ func = lambda _, it: imap(lambda x: f(*x), it) command = (func, None, - BatchedSerializer(PickleSerializer(), 1024), - BatchedSerializer(PickleSerializer(), 1024)) + AutoBatchedSerializer(PickleSerializer()), + AutoBatchedSerializer(PickleSerializer())) ser = CloudPickleSerializer() pickled_command = ser.dumps(command) if len(pickled_command) > (1 << 20): # 1M @@ -1443,8 +1443,7 @@ def applySchema(self, rdd, schema): converter = _python_to_sql_converter(schema) rdd = rdd.map(converter) - batched = isinstance(rdd._jrdd_deserializer, BatchedSerializer) - jrdd = self._pythonToJava(rdd._jrdd, batched) + jrdd = self._jvm.SerDeUtil.toJavaArray(rdd._to_java_object_rdd()) srdd = self._ssql_ctx.applySchemaToPythonRDD(jrdd.rdd(), schema.json()) return SchemaRDD(srdd.toJavaSchemaRDD(), self) @@ -1841,7 +1840,7 @@ def __init__(self, jschema_rdd, sql_ctx): self.is_checkpointed = False self.ctx = self.sql_ctx._sc # the _jrdd is created by javaToPython(), serialized by pickle - self._jrdd_deserializer = BatchedSerializer(PickleSerializer()) + self._jrdd_deserializer = AutoBatchedSerializer(PickleSerializer()) @property def _jrdd(self): @@ -2071,16 +2070,13 @@ def subtract(self, other, numPartitions=None): def _test(): import doctest - from array import array from pyspark.context import SparkContext # let doctest run in pyspark.sql, so DataTypes can be picklable import pyspark.sql from pyspark.sql import Row, SQLContext from pyspark.tests import ExamplePoint, ExamplePointUDT globs = pyspark.sql.__dict__.copy() - # The small batch size here ensures that we see multiple batches, - # even in these small test examples: - sc = SparkContext('local[4]', 'PythonTest', batchSize=2) + sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc globs['sqlCtx'] = SQLContext(sc) globs['rdd'] = sc.parallelize( diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index e947b09468108..7e61b017efa75 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -242,7 +242,7 @@ class PySparkTestCase(unittest.TestCase): def setUp(self): self._old_sys_path = list(sys.path) class_name = self.__class__.__name__ - self.sc = SparkContext('local[4]', class_name, batchSize=2) + self.sc = SparkContext('local[4]', class_name) def tearDown(self): self.sc.stop() @@ -253,7 +253,7 @@ class ReusedPySparkTestCase(unittest.TestCase): @classmethod def setUpClass(cls): - cls.sc = SparkContext('local[4]', cls.__name__, batchSize=2) + cls.sc = SparkContext('local[4]', cls.__name__) @classmethod def tearDownClass(cls): @@ -671,7 +671,7 @@ def setUp(self): self._old_sys_path = list(sys.path) class_name = self.__class__.__name__ conf = SparkConf().set("spark.python.profile", "true") - self.sc = SparkContext('local[4]', class_name, batchSize=2, conf=conf) + self.sc = SparkContext('local[4]', class_name, conf=conf) def test_profiler(self): @@ -1012,16 +1012,19 @@ def test_sequencefiles(self): clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", "org.apache.hadoop.io.Text", "org.apache.spark.api.python.TestWritable").collect()) - ec = (u'1', - {u'__class__': u'org.apache.spark.api.python.TestWritable', - u'double': 54.0, u'int': 123, u'str': u'test1'}) - self.assertEqual(clazz[0], ec) + cname = u'org.apache.spark.api.python.TestWritable' + ec = [(u'1', {u'__class__': cname, u'double': 1.0, u'int': 1, u'str': u'test1'}), + (u'2', {u'__class__': cname, u'double': 2.3, u'int': 2, u'str': u'test2'}), + (u'3', {u'__class__': cname, u'double': 3.1, u'int': 3, u'str': u'test3'}), + (u'4', {u'__class__': cname, u'double': 4.2, u'int': 4, u'str': u'test4'}), + (u'5', {u'__class__': cname, u'double': 5.5, u'int': 5, u'str': u'test56'})] + self.assertEqual(clazz, ec) unbatched_clazz = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfclass/", "org.apache.hadoop.io.Text", "org.apache.spark.api.python.TestWritable", - batchSize=1).collect()) - self.assertEqual(unbatched_clazz[0], ec) + ).collect()) + self.assertEqual(unbatched_clazz, ec) def test_oldhadoop(self): basepath = self.tempdir.name @@ -1341,51 +1344,6 @@ def test_reserialization(self): result5 = sorted(self.sc.sequenceFile(basepath + "/reserialize/newdataset").collect()) self.assertEqual(result5, data) - def test_unbatched_save_and_read(self): - basepath = self.tempdir.name - ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] - self.sc.parallelize(ei, len(ei)).saveAsSequenceFile( - basepath + "/unbatched/") - - unbatched_sequence = sorted(self.sc.sequenceFile( - basepath + "/unbatched/", - batchSize=1).collect()) - self.assertEqual(unbatched_sequence, ei) - - unbatched_hadoopFile = sorted(self.sc.hadoopFile( - basepath + "/unbatched/", - "org.apache.hadoop.mapred.SequenceFileInputFormat", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.Text", - batchSize=1).collect()) - self.assertEqual(unbatched_hadoopFile, ei) - - unbatched_newAPIHadoopFile = sorted(self.sc.newAPIHadoopFile( - basepath + "/unbatched/", - "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.Text", - batchSize=1).collect()) - self.assertEqual(unbatched_newAPIHadoopFile, ei) - - oldconf = {"mapred.input.dir": basepath + "/unbatched/"} - unbatched_hadoopRDD = sorted(self.sc.hadoopRDD( - "org.apache.hadoop.mapred.SequenceFileInputFormat", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.Text", - conf=oldconf, - batchSize=1).collect()) - self.assertEqual(unbatched_hadoopRDD, ei) - - newconf = {"mapred.input.dir": basepath + "/unbatched/"} - unbatched_newAPIHadoopRDD = sorted(self.sc.newAPIHadoopRDD( - "org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.Text", - conf=newconf, - batchSize=1).collect()) - self.assertEqual(unbatched_newAPIHadoopRDD, ei) - def test_malformed_RDD(self): basepath = self.tempdir.name # non-batch-serialized RDD[[(K, V)]] should be rejected diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 3ee2ea05cfa2d..fbec2f9f4b2c1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql import java.util.{List => JList} +import org.apache.spark.api.python.SerDeUtil + import scala.collection.JavaConversions._ import net.razorvine.pickle.Pickler @@ -385,12 +387,8 @@ class SchemaRDD( */ private[sql] def javaToPython: JavaRDD[Array[Byte]] = { val fieldTypes = schema.fields.map(_.dataType) - this.mapPartitions { iter => - val pickle = new Pickler - iter.map { row => - EvaluatePython.rowToArray(row, fieldTypes) - }.grouped(100).map(batched => pickle.dumps(batched.toArray)) - } + val jrdd = this.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD() + SerDeUtil.javaToPython(jrdd) } /** From bcecd73fdd4d2ec209259cfd57d3ad1d63f028f2 Mon Sep 17 00:00:00 2001 From: Dariusz Kobylarz Date: Tue, 4 Nov 2014 09:53:43 -0800 Subject: [PATCH 1218/1492] fixed MLlib Naive-Bayes java example bug the filter tests Double objects by references whereas it should test their values Author: Dariusz Kobylarz Closes #3081 from dkobylarz/master and squashes the following commits: 5d43a39 [Dariusz Kobylarz] naive bayes example update a304b93 [Dariusz Kobylarz] fixed MLlib Naive-Bayes java example bug --- docs/mllib-naive-bayes.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 7f9d4c6563944..d5b044d94fdd7 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -88,11 +88,11 @@ JavaPairRDD predictionAndLabel = return new Tuple2(model.predict(p.features()), p.label()); } }); -double accuracy = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { +double accuracy = predictionAndLabel.filter(new Function, Boolean>() { @Override public Boolean call(Tuple2 pl) { - return pl._1() == pl._2(); + return pl._1().equals(pl._2()); } - }).count() / test.count(); + }).count() / (double) test.count(); {% endhighlight %}